From 97356277231ba61d06a7cbd185e38af3d7b7df16 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 27 Aug 2018 14:36:22 -0700 Subject: [PATCH 0001/1642] Issue 1626: Add tests for LedgerUnderredplicationManager#markLedgerUnderreplicatedAsync *Motivation* We introduced LedgerUnderredplicationManager#markLedgerUnderreplicatedAsync in #1619. This exposes the async api to the public. Let's make sure we have enough test coverage for this new async api, including negative tests and concurrent tests. *Changes* - Add basic tests - Add negative tests - Add concurrent tests on resolving conflicts --- .../meta/ZkLedgerUnderreplicationManager.java | 3 +- .../ZkLedgerUnderreplicationManagerTest.java | 244 ++++++++++++++++++ .../suites/BookKeeperClusterTestSuite.java | 146 +++++++++++ .../bookkeeper/suites/MetadataStore.java | 41 +++ .../bookkeeper/suites/ZKMetadataStore.java | 45 ++++ 5 files changed, 478 insertions(+), 1 deletion(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManagerTest.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/BookKeeperClusterTestSuite.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/MetadataStore.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/ZKMetadataStore.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java index 32723706fd0..5ae90809159 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java @@ -243,7 +243,8 @@ public static String getUrLedgerLockZnode(String base, long ledgerId) { return String.format("%s/urL%010d", base, ledgerId); } - private String getUrLedgerZnode(long ledgerId) { + @VisibleForTesting + String getUrLedgerZnode(long ledgerId) { return getUrLedgerZnode(urLedgerPath, ledgerId); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManagerTest.java new file mode 100644 index 00000000000..2464a538c90 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManagerTest.java @@ -0,0 +1,244 @@ +/* + * Licensed 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.meta; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.proto.DataFormats.UnderreplicatedLedgerFormat; +import org.apache.bookkeeper.replication.ReplicationException; +import org.apache.bookkeeper.suites.BookKeeperClusterTestSuite; +import org.apache.bookkeeper.util.ZkUtils; +import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; +import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.commons.lang3.StringUtils; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.NoNodeException; +import org.apache.zookeeper.ZooKeeper; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Unit test {@link ZkLedgerUnderreplicationManager}. + */ +public class ZkLedgerUnderreplicationManagerTest extends BookKeeperClusterTestSuite { + + @BeforeClass + public static void setUpCluster() throws Exception { + BookKeeperClusterTestSuite.setUpCluster(0); + } + + @AfterClass + public static void tearDownCluster() throws Exception { + BookKeeperClusterTestSuite.tearDownCluster(); + } + + private static String getZooKeeperConnectString() throws Exception { + String[] serviceHosts = metadataStore.getServiceUri().getServiceHosts(); + return StringUtils.join(serviceHosts, ','); + } + + private static ZooKeeper createZooKeeper() throws Exception { + return ZooKeeperClient.newBuilder() + .connectString(getZooKeeperConnectString()) + .connectRetryPolicy( + new BoundExponentialBackoffRetryPolicy(1, 10, 100)) + .operationRetryPolicy( + new BoundExponentialBackoffRetryPolicy(1, 10, 100)) + .sessionTimeoutMs(60000) + .build(); + } + + private ZooKeeper zk; + private ZkLedgerUnderreplicationManager urMgr; + + @Before + public void setUp() throws Exception { + this.zk = createZooKeeper(); + ServerConfiguration conf = new ServerConfiguration(baseServerConf); + conf.setStoreSystemTimeAsLedgerUnderreplicatedMarkTime(true); + this.urMgr = new ZkLedgerUnderreplicationManager(conf, zk); + } + + @After + public void tearDown() throws Exception { + if (null != urMgr) { + this.urMgr.close(); + } + if (null != zk) { + this.zk.close(); + } + } + + /** + * Test basic operation on {@link ZkLedgerUnderreplicationManager#markLedgerUnderreplicatedAsync(long, Collection)}. + */ + @Test + public void testMarkLedgerUnderreplicatedBasic() throws Exception { + long ledgerId = 0xabcdef; + Collection missingBookies = Lists.newArrayList("bookie-1"); + + long prevCtime = -1L; + + // mark when it hasn't been marked before + FutureUtils.result(urMgr.markLedgerUnderreplicatedAsync(ledgerId, missingBookies)); + UnderreplicatedLedgerFormat urLedgerFormat = urMgr.getLedgerUnreplicationInfo(ledgerId); + assertEquals(missingBookies, urLedgerFormat.getReplicaList()); + assertTrue(urLedgerFormat.getCtime() > prevCtime); + prevCtime = urLedgerFormat.getCtime(); + + // mark when it has been marked. but the missing bookies already duplicated there + FutureUtils.result(urMgr.markLedgerUnderreplicatedAsync(ledgerId, missingBookies)); + urLedgerFormat = urMgr.getLedgerUnreplicationInfo(ledgerId); + assertEquals(missingBookies, urLedgerFormat.getReplicaList()); + assertTrue(urLedgerFormat.getCtime() >= prevCtime); + prevCtime = urLedgerFormat.getCtime(); + + // mark with new bookies when it has been marked + Collection newMissingBookies = Lists.newArrayList("bookie-2", "bookie-3"); + FutureUtils.result(urMgr.markLedgerUnderreplicatedAsync(ledgerId, newMissingBookies)); + urLedgerFormat = urMgr.getLedgerUnreplicationInfo(ledgerId); + assertEquals( + Lists.newArrayList("bookie-1", "bookie-2", "bookie-3"), + urLedgerFormat.getReplicaList() + ); + assertTrue(urLedgerFormat.getCtime() >= prevCtime); + } + + /** + * Test {@link ZkLedgerUnderreplicationManager#markLedgerUnderreplicatedAsync(long, Collection)} handles corrupted + * data. + */ + @Test + public void testMarkLedgerWithCorruptedDataExists() throws Exception { + long ledgerId = 0xabcdee; + String ledgerPath = urMgr.getUrLedgerZnode(ledgerId); + ZkUtils.createFullPathOptimistic( + zk, ledgerPath, "junk data".getBytes(UTF_8), ZkUtils.getACLs(baseServerConf), CreateMode.PERSISTENT); + Collection missingBookies = Lists.newArrayList("bookie-1"); + try { + FutureUtils.result(urMgr.markLedgerUnderreplicatedAsync(ledgerId, missingBookies)); + fail("Should fail to mark ledger underreplicated if there is already corrupted data on zookeeper"); + } catch (ReplicationException re) { + assertTrue(re.getMessage().contains("Invalid underreplicated ledger data for ledger " + ledgerPath)); + } + byte[] data = zk.getData(ledgerPath, null, null); + assertEquals("junk data", new String(data, UTF_8)); + } + + @Test + public void testMarkLedgerUnderreplicatedConcurrently() throws Exception { + final int numLedgers = 20; + List> futures = Lists.newArrayListWithExpectedSize(numLedgers); + long ledgerId = 0xabcc00; + Set expectedBookies = Sets.newHashSet(); + for (int i = 0; i < numLedgers; i++) { + futures.add( + urMgr.markLedgerUnderreplicatedAsync( + ledgerId, Lists.newArrayList("bookie-" + i))); + expectedBookies.add("bookie-" + i); + } + FutureUtils.result(FutureUtils.collect(futures)); + + UnderreplicatedLedgerFormat urLedgerFormat = urMgr.getLedgerUnreplicationInfo(ledgerId); + Set actualBookies = Sets.newHashSet(); + actualBookies.addAll(urLedgerFormat.getReplicaList()); + + assertEquals(expectedBookies, actualBookies); + } + + @Test + public void testMarkLedgerUnderreplicatedConcurrentlyWithDifferentClients() throws Exception { + final int numLedgers = 20; + List zks = new ArrayList<>(numLedgers); + List urMgrs = new ArrayList<>(numLedgers); + + for (int i = 0; i < numLedgers; i++) { + zks.add(createZooKeeper()); + urMgrs.add(new ZkLedgerUnderreplicationManager(baseServerConf, zks.get(i))); + } + + List> futures = Lists.newArrayListWithExpectedSize(numLedgers); + long ledgerId = 0xabcd00; + Set expectedBookies = Sets.newHashSet(); + for (int i = 0; i < numLedgers; i++) { + futures.add( + urMgrs.get(i).markLedgerUnderreplicatedAsync( + ledgerId, Lists.newArrayList("bookie-" + i))); + expectedBookies.add("bookie-" + i); + } + + FutureUtils.result(FutureUtils.collect(futures)); + + UnderreplicatedLedgerFormat urLedgerFormat = urMgr.getLedgerUnreplicationInfo(ledgerId); + Set actualBookies = Sets.newHashSet(); + actualBookies.addAll(urLedgerFormat.getReplicaList()); + + assertEquals(expectedBookies, actualBookies); + + for (LedgerUnderreplicationManager urMgr : urMgrs) { + urMgr.close(); + } + for (ZooKeeper zk : zks) { + zk.close(); + } + } + + + @Test + public void testMarkLedgerUnderreplicatedWhenSessionExpired() throws Exception { + final long ledgerId = 0xabbd00; + try (ZooKeeper zk = new ZooKeeper(getZooKeeperConnectString(), 60000, null)) { + try (LedgerUnderreplicationManager urMgr = new ZkLedgerUnderreplicationManager(baseServerConf, zk)) { + // open another zookeeper client to expire current session + try (ZooKeeper newZk = new ZooKeeper( + getZooKeeperConnectString(), 60000, null, zk.getSessionId(), zk.getSessionPasswd() + )) { + try { + FutureUtils.result(urMgr.markLedgerUnderreplicatedAsync( + ledgerId, Lists.newArrayList("bookie-1"))); + fail("Should fail if encountered zookeeper exceptions"); + } catch (KeeperException ke) { + // expected + } + try { + UnderreplicatedLedgerFormat urLedgerFormat = + ZkLedgerUnderreplicationManagerTest.this.urMgr.getLedgerUnreplicationInfo(ledgerId); + fail("The ledger shouldn't been marked as underreplicated"); + } catch (NoNodeException nee) { + // expected + } + } + } + } + } + + +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/BookKeeperClusterTestSuite.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/BookKeeperClusterTestSuite.java new file mode 100644 index 00000000000..df156dff053 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/BookKeeperClusterTestSuite.java @@ -0,0 +1,146 @@ +/* + * Licensed 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.suites; + +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.test.PortManager; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.util.IOUtils; +import org.apache.commons.io.FileUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +/** + * A class runs a bookkeeper cluster for testing. + * + *

The cluster will be setup and teardown before class. It will not be restarted between different + * test methods. It is more suitable for running tests that don't require restarting bookies. + */ +@Slf4j +public abstract class BookKeeperClusterTestSuite { + + protected static MetadataStore metadataStore; + protected static ClientConfiguration baseClientConf; + protected static ServerConfiguration baseServerConf; + protected static final int NUM_BOOKIES = 3; + protected static final List BOOKIES = new ArrayList<>(NUM_BOOKIES); + protected static final List TMP_DIRS = new ArrayList<>(NUM_BOOKIES); + + protected static File createTempDir(String prefix, String suffix) throws IOException { + File dir = IOUtils.createTempDir(prefix, suffix); + TMP_DIRS.add(dir); + return dir; + } + + protected static ServerConfiguration newServerConfiguration() throws Exception { + File f = createTempDir("bookie", "test"); + int port = PortManager.nextFreePort(); + return newServerConfiguration(port, f, new File[] { f }); + } + + protected static ServerConfiguration newServerConfiguration(int port, File journalDir, File[] ledgerDirs) { + ServerConfiguration conf = new ServerConfiguration(baseServerConf); + conf.setBookiePort(port); + conf.setJournalDirName(journalDir.getPath()); + String[] ledgerDirNames = new String[ledgerDirs.length]; + for (int i = 0; i < ledgerDirs.length; i++) { + ledgerDirNames[i] = ledgerDirs[i].getPath(); + } + conf.setLedgerDirNames(ledgerDirNames); + conf.setEnableTaskExecutionStats(true); + return conf; + } + + @BeforeClass + public static void setUpCluster() throws Exception { + setUpCluster(NUM_BOOKIES); + } + + protected static void setUpCluster(int numBookies) throws Exception { + // set up the metadata store + metadataStore = new ZKMetadataStore(); + metadataStore.start(); + ServiceURI uri = metadataStore.getServiceUri(); + log.info("Setting up cluster at service uri : {}", uri.getUri()); + + baseClientConf = new ClientConfiguration() + .setMetadataServiceUri(uri.getUri().toString()); + baseServerConf = TestBKConfiguration.newServerConfiguration() + .setMetadataServiceUri(uri.getUri().toString()); + + // format the cluster + assertTrue(BookKeeperAdmin.format(baseServerConf, false, true)); + + // start bookies + startNumBookies(numBookies); + } + + private static void startNumBookies(int numBookies) throws Exception { + for (int i = 0; i < numBookies; i++) { + ServerConfiguration conf = newServerConfiguration(); + log.info("Starting new bookie on port : {}", conf.getBookiePort()); + BookieServer server = startBookie(conf); + synchronized (BOOKIES) { + BOOKIES.add(server); + } + } + } + + private static BookieServer startBookie(ServerConfiguration conf) throws Exception { + conf.setAutoRecoveryDaemonEnabled(true); + TestStatsProvider provider = new TestStatsProvider(); + BookieServer server = new BookieServer(conf, provider.getStatsLogger("")); + server.start(); + return server; + } + + @AfterClass + public static void tearDownCluster() throws Exception { + // stop bookies + stopBookies(); + + // stop metadata store + metadataStore.close(); + log.info("Stopped the metadata store."); + + // clean up temp dirs + for (File f : TMP_DIRS) { + FileUtils.deleteDirectory(f); + } + log.info("Clean up all the temp directories."); + } + + private static void stopBookies() { + synchronized (BOOKIES) { + BOOKIES.forEach(BookieServer::shutdown); + log.info("Stopped all the bookies."); + } + } + + + +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/MetadataStore.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/MetadataStore.java new file mode 100644 index 00000000000..853dee5a22e --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/MetadataStore.java @@ -0,0 +1,41 @@ +/* + * Licensed 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.suites; + +import org.apache.bookkeeper.common.net.ServiceURI; + +/** + * A metadata store interface to start up and teardown a metadata service. + */ +public interface MetadataStore extends AutoCloseable { + + /** + * Start the metadata store. + */ + void start() throws Exception; + + /** + * Close the metadata store. + */ + void close() throws Exception; + + /** + * Return the service uri for bookies and clients to use. + * + * @return the service uri. + */ + ServiceURI getServiceUri(); + +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/ZKMetadataStore.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/ZKMetadataStore.java new file mode 100644 index 00000000000..2c42b230ce6 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/suites/ZKMetadataStore.java @@ -0,0 +1,45 @@ +/* + * Licensed 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.suites; + +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.test.ZooKeeperUtil; + +/** + * Start the zookeeper based metadata store. + */ +class ZKMetadataStore implements MetadataStore { + + private final ZooKeeperUtil zkUtil; + + ZKMetadataStore() { + this.zkUtil = new ZooKeeperUtil(); + } + + @Override + public void start() throws Exception { + this.zkUtil.startServer(); + } + + @Override + public void close() throws Exception { + this.zkUtil.killServer(); + } + + @Override + public ServiceURI getServiceUri() { + return ServiceURI.create(zkUtil.getMetadataServiceUri()); + } +} From 39f012ad0e56a098b4231126c77306affcba2bbf Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 27 Aug 2018 15:36:33 -0700 Subject: [PATCH 0002/1642] use @cleanup --- .../ZkLedgerUnderreplicationManagerTest.java | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManagerTest.java index 2464a538c90..55bc0eeeb77 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManagerTest.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; +import lombok.Cleanup; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.proto.DataFormats.UnderreplicatedLedgerFormat; @@ -215,28 +216,27 @@ public void testMarkLedgerUnderreplicatedConcurrentlyWithDifferentClients() thro @Test public void testMarkLedgerUnderreplicatedWhenSessionExpired() throws Exception { final long ledgerId = 0xabbd00; - try (ZooKeeper zk = new ZooKeeper(getZooKeeperConnectString(), 60000, null)) { - try (LedgerUnderreplicationManager urMgr = new ZkLedgerUnderreplicationManager(baseServerConf, zk)) { - // open another zookeeper client to expire current session - try (ZooKeeper newZk = new ZooKeeper( - getZooKeeperConnectString(), 60000, null, zk.getSessionId(), zk.getSessionPasswd() - )) { - try { - FutureUtils.result(urMgr.markLedgerUnderreplicatedAsync( - ledgerId, Lists.newArrayList("bookie-1"))); - fail("Should fail if encountered zookeeper exceptions"); - } catch (KeeperException ke) { - // expected - } - try { - UnderreplicatedLedgerFormat urLedgerFormat = - ZkLedgerUnderreplicationManagerTest.this.urMgr.getLedgerUnreplicationInfo(ledgerId); - fail("The ledger shouldn't been marked as underreplicated"); - } catch (NoNodeException nee) { - // expected - } - } - } + @Cleanup + ZooKeeper zk = new ZooKeeper(getZooKeeperConnectString(), 60000, null); + @Cleanup + LedgerUnderreplicationManager urMgr = new ZkLedgerUnderreplicationManager(baseServerConf, zk); + // open another zookeeper client to expire current session + @Cleanup + ZooKeeper newZk = new ZooKeeper( + getZooKeeperConnectString(), 60000, null, zk.getSessionId(), zk.getSessionPasswd()); + try { + FutureUtils.result(urMgr.markLedgerUnderreplicatedAsync( + ledgerId, Lists.newArrayList("bookie-1"))); + fail("Should fail if encountered zookeeper exceptions"); + } catch (KeeperException ke) { + // expected + } + try { + UnderreplicatedLedgerFormat urLedgerFormat = + ZkLedgerUnderreplicationManagerTest.this.urMgr.getLedgerUnreplicationInfo(ledgerId); + fail("The ledger shouldn't been marked as underreplicated"); + } catch (NoNodeException nee) { + // expected } } From ad6e4a3a9a55679a08e1c027c1c41f4cc65262da Mon Sep 17 00:00:00 2001 From: Samuel Just Date: Mon, 27 Aug 2018 22:40:27 -0700 Subject: [PATCH 0003/1642] ISSUE #1623: ReadOnlyLedgerHandle: don't schedule monitorPendingAddOps() The LedgerHandle constructor schedules an addEntryQuorumTimeout check with the bk client scheduler. However, the only place this callback is canceled is in the closeAsync (the one which returns a future, not to be confused with asyncClose) method. asyncClose and close() both leak this callback. Moreover, ReadOnlyLedgerHandle invokes the LedgerHandle constructor and so also creates this callback, but it overrides close() and asyncClose() without passing them through. ReadOnlyLedgerHandle already overrides initializeExplicitLacFlushPolicy() to avoid write specific state. This patch generalizes that hack to initializeWriteHandleState() and the cleanup to tearDownWriteHandleState(). tearDownWriteHandleState() is moved into doAsyncClose(), which appears to be called for closes in general. (rev cguttapalem) (bug W-5362724) Signed-off-by: Samuel Just Author: Samuel Just Reviewers: Ivan Kelly , Enrico Olivelli , Sijie Guo This closes #1624 from athanatos/forupstream/wip-1623, closes #1623 --- .../bookkeeper/client/LedgerHandle.java | 40 ++++++++++--------- .../client/ReadOnlyLedgerHandle.java | 3 +- 2 files changed, 23 insertions(+), 20 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 3e9f1c2a349..741610dcef8 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 @@ -238,15 +238,25 @@ public Integer getSample() { return pendingAddOps.size(); } }); - initializeExplicitLacFlushPolicy(); + + initializeWriteHandleState(); + } + + protected void initializeWriteHandleState() { + if (clientCtx.getConf().explicitLacInterval > 0) { + explicitLacFlushPolicy = new ExplicitLacFlushPolicy.ExplicitLacFlushPolicyImpl( + this, clientCtx); + } else { + explicitLacFlushPolicy = ExplicitLacFlushPolicy.VOID_EXPLICITLAC_FLUSH_POLICY; + } if (clientCtx.getConf().addEntryQuorumTimeoutNanos > 0) { SafeRunnable monitor = new SafeRunnable() { - @Override - public void safeRun() { - monitorPendingAddOps(); - } - }; + @Override + public void safeRun() { + monitorPendingAddOps(); + } + }; this.timeoutFuture = clientCtx.getScheduler().scheduleAtFixedRate( monitor, clientCtx.getConf().timeoutMonitorIntervalSec, @@ -255,14 +265,10 @@ public void safeRun() { } } - protected void initializeExplicitLacFlushPolicy() { - if (!getLedgerMetadata().isClosed() - && !(this instanceof ReadOnlyLedgerHandle) - && clientCtx.getConf().explicitLacInterval > 0) { - explicitLacFlushPolicy = new ExplicitLacFlushPolicy.ExplicitLacFlushPolicyImpl( - this, clientCtx); - } else { - explicitLacFlushPolicy = ExplicitLacFlushPolicy.VOID_EXPLICITLAC_FLUSH_POLICY; + private void tearDownWriteHandleState() { + explicitLacFlushPolicy.stopExplicitLacFlush(); + if (timeoutFuture != null) { + timeoutFuture.cancel(false); } } @@ -445,10 +451,6 @@ public CompletableFuture closeAsync() { CompletableFuture result = new CompletableFuture<>(); SyncCloseCallback callback = new SyncCloseCallback(result); asyncClose(callback, null); - explicitLacFlushPolicy.stopExplicitLacFlush(); - if (timeoutFuture != null) { - timeoutFuture.cancel(false); - } return result; } @@ -625,7 +627,7 @@ public String toString() { } writeLedgerConfig(new CloseCb()); - + tearDownWriteHandleState(); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java index e9f790015fc..aa0290b3065 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java @@ -183,7 +183,8 @@ public String toString() { } @Override - protected void initializeExplicitLacFlushPolicy() { + protected void initializeWriteHandleState() { + // Essentially a noop, we don't want to set up write handle state here for a ReadOnlyLedgerHandle explicitLacFlushPolicy = ExplicitLacFlushPolicy.VOID_EXPLICITLAC_FLUSH_POLICY; } From 9bab746b8827de9e50eee37b74a512ac797c1338 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Tue, 28 Aug 2018 15:42:54 +0200 Subject: [PATCH 0004/1642] Allow construction of mock client context from scratch So that ledger handle functionality can be tested without instantiating a BookKeeper client. Author: Ivan Kelly Reviewers: Enrico Olivelli , Sijie Guo This closes #1625 from ivankelly/mock-client-context --- .../bookkeeper/client/MockClientContext.java | 98 +++++++++++++++---- 1 file changed, 79 insertions(+), 19 deletions(-) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java index 040402dc102..2f5b2dc5a40 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java @@ -20,15 +20,27 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkState; + import java.util.function.BooleanSupplier; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.discover.MockRegistrationClient; import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.MockLedgerManager; import org.apache.bookkeeper.proto.BookieClient; +import org.apache.bookkeeper.proto.MockBookieClient; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.mockito.Mockito; -class MockClientContext implements ClientContext { - private ClientInternalConf conf; +/** + * Mock client context to allow testing client functionality with no external dependencies. + * The client context can be created with defaults, copied from another context or constructed from scratch. + */ +public class MockClientContext implements ClientContext { + private ClientInternalConf internalConf; private LedgerManager ledgerManager; private BookieWatcher bookieWatcher; private EnsemblePlacementPolicy placementPolicy; @@ -37,6 +49,26 @@ class MockClientContext implements ClientContext { private OrderedScheduler scheduler; private BookKeeperClientStats clientStats; private BooleanSupplier isClientClosed; + private MockRegistrationClient regClient; + + static MockClientContext create() { + ClientConfiguration conf = new ClientConfiguration(); + OrderedScheduler scheduler = OrderedScheduler.newSchedulerBuilder().name("mock-executor").numThreads(1).build(); + MockRegistrationClient regClient = new MockRegistrationClient(); + EnsemblePlacementPolicy placementPolicy = new DefaultEnsemblePlacementPolicy(); + + return new MockClientContext() + .setConf(ClientInternalConf.fromConfig(conf)) + .setLedgerManager(new MockLedgerManager()) + .setBookieWatcher(new BookieWatcherImpl(conf, placementPolicy, regClient, NullStatsLogger.INSTANCE)) + .setPlacementPolicy(placementPolicy) + .setRegistrationClient(regClient) + .setBookieClient(new MockBookieClient(scheduler)) + .setMainWorkerPool(scheduler) + .setScheduler(scheduler) + .setClientStats(BookKeeperClientStats.newInstance(NullStatsLogger.INSTANCE)) + .setIsClientClosed(() -> false); + } static MockClientContext copyOf(ClientContext other) { return new MockClientContext() @@ -51,54 +83,82 @@ static MockClientContext copyOf(ClientContext other) { .setIsClientClosed(other::isClientClosed); } - MockClientContext setConf(ClientInternalConf conf) { - this.conf = conf; + public MockRegistrationClient getMockRegistrationClient() { + checkState(regClient != null); + return regClient; + } + + public MockLedgerManager getMockLedgerManager() { + checkState(ledgerManager instanceof MockLedgerManager); + return (MockLedgerManager) ledgerManager; + } + + public MockBookieClient getMockBookieClient() { + checkState(bookieClient instanceof MockBookieClient); + return (MockBookieClient) bookieClient; + } + + public MockClientContext setConf(ClientInternalConf internalConf) { + this.internalConf = maybeSpy(internalConf); return this; } - MockClientContext setLedgerManager(LedgerManager ledgerManager) { - this.ledgerManager = ledgerManager; + public MockClientContext setLedgerManager(LedgerManager ledgerManager) { + this.ledgerManager = maybeSpy(ledgerManager); return this; } - MockClientContext setBookieWatcher(BookieWatcher bookieWatcher) { - this.bookieWatcher = bookieWatcher; + public MockClientContext setBookieWatcher(BookieWatcher bookieWatcher) { + this.bookieWatcher = maybeSpy(bookieWatcher); return this; } - MockClientContext setPlacementPolicy(EnsemblePlacementPolicy placementPolicy) { - this.placementPolicy = placementPolicy; + public MockClientContext setPlacementPolicy(EnsemblePlacementPolicy placementPolicy) { + this.placementPolicy = maybeSpy(placementPolicy); return this; } - MockClientContext setBookieClient(BookieClient bookieClient) { - this.bookieClient = bookieClient; + public MockClientContext setBookieClient(BookieClient bookieClient) { + this.bookieClient = maybeSpy(bookieClient); return this; } - MockClientContext setMainWorkerPool(OrderedExecutor mainWorkerPool) { - this.mainWorkerPool = mainWorkerPool; + public MockClientContext setMainWorkerPool(OrderedExecutor mainWorkerPool) { + this.mainWorkerPool = maybeSpy(mainWorkerPool); return this; } - MockClientContext setScheduler(OrderedScheduler scheduler) { - this.scheduler = scheduler; + public MockClientContext setScheduler(OrderedScheduler scheduler) { + this.scheduler = maybeSpy(scheduler); return this; } - MockClientContext setClientStats(BookKeeperClientStats clientStats) { + public MockClientContext setClientStats(BookKeeperClientStats clientStats) { this.clientStats = clientStats; return this; } - MockClientContext setIsClientClosed(BooleanSupplier isClientClosed) { + public MockClientContext setIsClientClosed(BooleanSupplier isClientClosed) { this.isClientClosed = isClientClosed; return this; } + public MockClientContext setRegistrationClient(MockRegistrationClient regClient) { + this.regClient = maybeSpy(regClient); + return this; + } + + private static T maybeSpy(T orig) { + if (Mockito.mockingDetails(orig).isSpy()) { + return orig; + } else { + return Mockito.spy(orig); + } + } + @Override public ClientInternalConf getConf() { - return this.conf; + return this.internalConf; } @Override From f1f4d13a61e775e24e09d031419fc69c64c8d867 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 29 Aug 2018 10:37:30 -0700 Subject: [PATCH 0005/1642] [RELEASE] update release script to generate sha512 instead of sha1 *Motivation* According to [checksums requirements](http://www.apache.org/dev/release-distribution#sigs-and-sums), the project should supply a sha256 and/or sha512 checksum file, and should not supply a MD5 and sha1 checksum file. *Changes* Update the release script to generate sha512 files Descriptions of the changes in this PR: Author: Sijie Guo Reviewers: Enrico Olivelli This closes #1631 from sijie/sha512_files --- dev/release/004-stage-packages.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/dev/release/004-stage-packages.sh b/dev/release/004-stage-packages.sh index 8926ccf9d30..23941342f9a 100755 --- a/dev/release/004-stage-packages.sh +++ b/dev/release/004-stage-packages.sh @@ -52,11 +52,12 @@ cp ${SRC_DIR}/bookkeeper-dist/all/target/bookkeeper-all-${VERSION}-bin.tar.gz cp ${SRC_DIR}/bookkeeper-dist/all/target/bookkeeper-all-${VERSION}-bin.tar.gz.asc ${DEST_DIR}/bookkeeper-all-${VERSION}-bin.tar.gz.asc echo "Copied packages." -echo "Generating sha1 files ..." -sha1sum ${DEST_DIR}/bookkeeper-${VERSION}-src.tar.gz > ${DEST_DIR}/bookkeeper-${VERSION}-src.tar.gz.sha1 -sha1sum ${DEST_DIR}/bookkeeper-server-${VERSION}-bin.tar.gz > ${DEST_DIR}/bookkeeper-server-${VERSION}-bin.tar.gz.sha1 -sha1sum ${DEST_DIR}/bookkeeper-all-${VERSION}-bin.tar.gz > ${DEST_DIR}/bookkeeper-all-${VERSION}-bin.tar.gz.sha1 -echo "Generated sha1 files." +echo "Generating sha512 files ..." +cd ${DEST_DIR} +shasum -a 512 bookkeeper-${VERSION}-src.tar.gz > bookkeeper-${VERSION}-src.tar.gz.sha512 +shasum -a 512 bookkeeper-server-${VERSION}-bin.tar.gz > bookkeeper-server-${VERSION}-bin.tar.gz.sha512 +shasum -a 512 bookkeeper-all-${VERSION}-bin.tar.gz > bookkeeper-all-${VERSION}-bin.tar.gz.sha512 +echo "Generated sha512 files." cd ${DIST_DEV_DIR}/bookkeeper svn add ${RC_DIR} From 8ee926580b1f44d09305be13c0e2f9ea28c48fb8 Mon Sep 17 00:00:00 2001 From: Qi Wang Date: Thu, 30 Aug 2018 11:24:54 -0700 Subject: [PATCH 0006/1642] Update testcontainers to latest version Descriptions of the changes in this PR: ### Motivation the latest version of testcontainers provides a `doStart` method to allow adding logic before `doStart`. Author: Qi Wang Reviewers: Enrico Olivelli , Sijie Guo This closes #1636 from codingwangqi/update_testcontainers --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b29f423731b..609411c2c5a 100644 --- a/pom.xml +++ b/pom.xml @@ -154,7 +154,7 @@ 3.0.1 1.7.25 3.1.1 - 1.7.0 + 1.8.3 1.29.0 3.4.1 3.4.13 From 9993406e59550d5769f7cc1271dd4189df9a190d Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 30 Aug 2018 23:31:02 -0700 Subject: [PATCH 0007/1642] [RELEASE] [WEBSITE] Create documentation for release 4.7.2 Author: Sijie Guo Reviewers: Enrico Olivelli This closes #1634 from sijie/website_472 --- site/_config.yml | 5 +- site/docs/4.7.2/admin/autorecovery.md | 128 +++ site/docs/4.7.2/admin/bookies.md | 180 ++++ site/docs/4.7.2/admin/geo-replication.md | 22 + site/docs/4.7.2/admin/http.md | 394 +++++++++ site/docs/4.7.2/admin/metrics.md | 41 + site/docs/4.7.2/admin/perf.md | 3 + site/docs/4.7.2/admin/placement.md | 3 + site/docs/4.7.2/admin/upgrade.md | 179 ++++ site/docs/4.7.2/api/distributedlog-api.md | 395 +++++++++ site/docs/4.7.2/api/ledger-adv-api.md | 111 +++ site/docs/4.7.2/api/ledger-api.md | 802 ++++++++++++++++++ site/docs/4.7.2/api/overview.md | 17 + site/docs/4.7.2/deployment/dcos.md | 142 ++++ site/docs/4.7.2/deployment/kubernetes.md | 181 ++++ site/docs/4.7.2/deployment/manual.md | 56 ++ site/docs/4.7.2/development/codebase.md | 3 + site/docs/4.7.2/development/protocol.md | 148 ++++ site/docs/4.7.2/getting-started/concepts.md | 202 +++++ .../4.7.2/getting-started/installation.md | 74 ++ .../docs/4.7.2/getting-started/run-locally.md | 16 + site/docs/4.7.2/overview/overview.md | 58 ++ site/docs/4.7.2/overview/releaseNotes.md | 44 + .../4.7.2/overview/releaseNotesTemplate.md | 17 + site/docs/4.7.2/reference/cli.md | 10 + site/docs/4.7.2/reference/config.md | 9 + site/docs/4.7.2/reference/metrics.md | 3 + site/docs/4.7.2/security/overview.md | 21 + site/docs/4.7.2/security/sasl.md | 202 +++++ site/docs/4.7.2/security/tls.md | 210 +++++ site/docs/4.7.2/security/zookeeper.md | 41 + site/releases.md | 10 + 32 files changed, 3725 insertions(+), 2 deletions(-) create mode 100644 site/docs/4.7.2/admin/autorecovery.md create mode 100644 site/docs/4.7.2/admin/bookies.md create mode 100644 site/docs/4.7.2/admin/geo-replication.md create mode 100644 site/docs/4.7.2/admin/http.md create mode 100644 site/docs/4.7.2/admin/metrics.md create mode 100644 site/docs/4.7.2/admin/perf.md create mode 100644 site/docs/4.7.2/admin/placement.md create mode 100644 site/docs/4.7.2/admin/upgrade.md create mode 100644 site/docs/4.7.2/api/distributedlog-api.md create mode 100644 site/docs/4.7.2/api/ledger-adv-api.md create mode 100644 site/docs/4.7.2/api/ledger-api.md create mode 100644 site/docs/4.7.2/api/overview.md create mode 100644 site/docs/4.7.2/deployment/dcos.md create mode 100644 site/docs/4.7.2/deployment/kubernetes.md create mode 100644 site/docs/4.7.2/deployment/manual.md create mode 100644 site/docs/4.7.2/development/codebase.md create mode 100644 site/docs/4.7.2/development/protocol.md create mode 100644 site/docs/4.7.2/getting-started/concepts.md create mode 100644 site/docs/4.7.2/getting-started/installation.md create mode 100644 site/docs/4.7.2/getting-started/run-locally.md create mode 100644 site/docs/4.7.2/overview/overview.md create mode 100644 site/docs/4.7.2/overview/releaseNotes.md create mode 100644 site/docs/4.7.2/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.7.2/reference/cli.md create mode 100644 site/docs/4.7.2/reference/config.md create mode 100644 site/docs/4.7.2/reference/metrics.md create mode 100644 site/docs/4.7.2/security/overview.md create mode 100644 site/docs/4.7.2/security/sasl.md create mode 100644 site/docs/4.7.2/security/tls.md create mode 100644 site/docs/4.7.2/security/zookeeper.md diff --git a/site/_config.yml b/site/_config.yml index a0b9a1070c0..ff35a8929ad 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -9,6 +9,7 @@ twitter_url: https://twitter.com/asfbookkeeper versions: # [next_version_placeholder] +- "4.7.2" - "4.7.1" - "4.7.0" - "4.6.2" @@ -30,9 +31,9 @@ archived_versions: - "4.1.0" - "4.0.0" latest_version: "4.8.0-SNAPSHOT" -latest_release: "4.7.1" +latest_release: "4.7.2" stable_release: "4.6.2" -distributedlog_version: "4.7.1" +distributedlog_version: "4.7.2" defaults: - scope: diff --git a/site/docs/4.7.2/admin/autorecovery.md b/site/docs/4.7.2/admin/autorecovery.md new file mode 100644 index 00000000000..b1dd078f9b2 --- /dev/null +++ b/site/docs/4.7.2/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: + +* the `shell recover` option +* 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 shell recover \ + 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 shell recover \ + 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.7.2/admin/bookies.md b/site/docs/4.7.2/admin/bookies.md new file mode 100644 index 00000000000..1b0427dae3c --- /dev/null +++ b/site/docs/4.7.2/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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + \ + \ + + ``` + + 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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + 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.7.2/admin/geo-replication.md b/site/docs/4.7.2/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.7.2/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.7.2/admin/http.md b/site/docs/4.7.2/admin/http.md new file mode 100644 index 00000000000..0097adc62b8 --- /dev/null +++ b/site/docs/4.7.2/admin/http.md @@ -0,0 +1,394 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 4 components: +1. Heartbeat: heartbeat for a specific bookie. +1. Config: doing the server configuration for a specific bookie. +1. Ledger: HTTP endpoints related to ledgers. +1. Bookie: HTTP endpoints related to bookies. +1. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.7.2/admin/metrics.md b/site/docs/4.7.2/admin/metrics.md new file mode 100644 index 00000000000..142df3dcd2d --- /dev/null +++ b/site/docs/4.7.2/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.prometheus.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.7.2/admin/perf.md b/site/docs/4.7.2/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.7.2/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.7.2/admin/placement.md b/site/docs/4.7.2/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.7.2/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.7.2/admin/upgrade.md b/site/docs/4.7.2/admin/upgrade.md new file mode 100644 index 00000000000..ddbaf0eda11 --- /dev/null +++ b/site/docs/4.7.2/admin/upgrade.md @@ -0,0 +1,179 @@ +--- +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.7.1 to 4.7.2 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.2. So you can follow the general upgrade sequence to upgrade from 4.7.1 to 4.7.2. + +### 4.7.0 to 4.7.1 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.1. So you can follow the general upgrade sequence to upgrade from 4.7.0 to 4.7.1. + +### 4.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.7.2/api/distributedlog-api.md b/site/docs/4.7.2/api/distributedlog-api.md new file mode 100644 index 00000000000..786d7a95a1e --- /dev/null +++ b/site/docs/4.7.2/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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.7.2/api/ledger-adv-api.md b/site/docs/4.7.2/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.7.2/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.7.2/api/ledger-api.md b/site/docs/4.7.2/api/ledger-api.md new file mode 100644 index 00000000000..d30c4c8e4cb --- /dev/null +++ b/site/docs/4.7.2/api/ledger-api.md @@ -0,0 +1,802 @@ +--- +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.7.2 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.7.2 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.7.2' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.7.2' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for +guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` diff --git a/site/docs/4.7.2/api/overview.md b/site/docs/4.7.2/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.7.2/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.7.2/deployment/dcos.md b/site/docs/4.7.2/deployment/dcos.md new file mode 100644 index 00000000000..ed7cdf59ae3 --- /dev/null +++ b/site/docs/4.7.2/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.7.2] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.7.2/deployment/kubernetes.md b/site/docs/4.7.2/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.7.2/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.7.2/deployment/manual.md b/site/docs/4.7.2/deployment/manual.md new file mode 100644 index 00000000000..daafd5556f5 --- /dev/null +++ b/site/docs/4.7.2/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.7.2/development/codebase.md b/site/docs/4.7.2/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.7.2/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.7.2/development/protocol.md b/site/docs/4.7.2/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.7.2/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.7.2/getting-started/concepts.md b/site/docs/4.7.2/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.7.2/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.7.2/getting-started/installation.md b/site/docs/4.7.2/getting-started/installation.md new file mode 100644 index 00000000000..9986cd8e043 --- /dev/null +++ b/site/docs/4.7.2/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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.7.2/getting-started/run-locally.md b/site/docs/4.7.2/getting-started/run-locally.md new file mode 100644 index 00000000000..edbfab9fda6 --- /dev/null +++ b/site/docs/4.7.2/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 +$ bookkeeper-server/bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.7.2/overview/overview.md b/site/docs/4.7.2/overview/overview.md new file mode 100644 index 00000000000..0c30a6e51b3 --- /dev/null +++ b/site/docs/4.7.2/overview/overview.md @@ -0,0 +1,58 @@ +--- +title: Apache BookKeeper™ 4.7.2 +--- + + +This documentation is for Apache BookKeeper™ version 4.7.2. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.7.2 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.7.2/overview/releaseNotes.md b/site/docs/4.7.2/overview/releaseNotes.md new file mode 100644 index 00000000000..cb9d33fc537 --- /dev/null +++ b/site/docs/4.7.2/overview/releaseNotes.md @@ -0,0 +1,44 @@ +--- +title: Apache BookKeeper 4.7.2 Release Notes +--- + +This is the eleventh release of Apache BookKeeper! + +The 4.7.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.7.0. + +Apache BookKeeper users who are using 4.7.0 are encouraged to upgrade to 4.7.2. The technical details of this release are summarized +below. + +## Highlights + +- Performance enhancement on eliminating bytes copying in `AddEntry` code path, see [apache/bookkeeper#1361](https://github.com/apache/bookkeeper/pull/1361) + +- Introduce Fast and Garbage-Free Statistics Timers in Codahale Stats Provider, see [apache/bookkeeper#1364](https://github.com/apache/bookkeeper/pull/1364) + +- Fix OrderedScheduler handling null key, see [apache/bookkeeper#1372](https://github.com/apache/bookkeeper/pull/1372) + +- Fix zookeeper ledger manager on handling no ledger exists, see [apache/bookkeeper#1382](https://github.com/apache/bookkeeper/pull/1382) + +- Fix long poll reads when ensemble size is larger than write quorum size, see [apache/bookkeeper#1404](https://github.com/apache/bookkeeper/pull/1404) + +- Fix IllegalReferenceCount on filling readahead cache for DbLedgerStorage, see [apache/bookkeeper#1487](https://github.com/apache/bookkeeper/issues/1487) + +- Fix LedgerEntry recycling issue on long poll speculative reads, see [apache/bookkeeper#1509](https://github.com/apache/bookkeeper/pull/1509) + +- Various bug fixes and improvements around bookkeeper table service, see changes under [apache/bookkeeper#release/4.7.2](https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93&q=is%3Aclosed+label%3Aarea%2Ftableservice+label%3Arelease%2F4.7.1) + +### Dependencies Upgrade + +Here is a list of dependencies changed in 4.7.2: + +- [Grpc](https://grpc.io/) is upgraded from `1.5.0` to `1.12.0`. See [apache/bookkeeper#1441](https://github.com/apache/bookkeeper/pull/1441) +- [Netty](http://netty.io/) is upgraded from `4.1.12` to `4.1.22`. See [apache/bookkeeper#1441](https://github.com/apache/bookkeeper/pull/1441) +- [Protobuf](https://developers.google.com/protocol-buffers/) is upgraded from `3.4.0` to `3.5.1`. See [apache/bookkeeper#1466](https://github.com/apache/bookkeeper/pull/1466) +- [RocksDB](http://rocksdb.org/) is upgraded from `5.8.6` to `5.13.1`. See [apache/bookkeeper#1466](https://github.com/apache/bookkeeper/pull/1466) + +`Reflective setAccessible(true)` is disabled by default in Netty while using java9+. This might result in performance degradation. Consider reenabling `Reflective setAccessible(true)` by setting +environment value `io.netty.tryReflectionSetAccessible` to `true`. See [netty/netty#7650](https://github.com/netty/netty/pull/7650) for more details. + +## Full list of changes + +- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.1+is%3Aclosed) diff --git a/site/docs/4.7.2/overview/releaseNotesTemplate.md b/site/docs/4.7.2/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..cd7570d4db6 --- /dev/null +++ b/site/docs/4.7.2/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.7.2 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.7.2. The technical details of this release are summarized +below. + +## Highlights + +[List the highlights] + +## Details + +[list to issues list] + diff --git a/site/docs/4.7.2/reference/cli.md b/site/docs/4.7.2/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.7.2/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.7.2/reference/config.md b/site/docs/4.7.2/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.7.2/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.7.2/reference/metrics.md b/site/docs/4.7.2/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.7.2/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.7.2/security/overview.md b/site/docs/4.7.2/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.7.2/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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.7.2/security/sasl.md b/site/docs/4.7.2/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.7.2/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.7.2/security/tls.md b/site/docs/4.7.2/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.7.2/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.7.2/security/zookeeper.md b/site/docs/4.7.2/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.7.2/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) diff --git a/site/releases.md b/site/releases.md index 876c175e8fd..d8233c6afd1 100644 --- a/site/releases.md +++ b/site/releases.md @@ -62,6 +62,16 @@ Client Guide | API docs ## News +### 29 August, 2018: Release 4.7.2 available + +This is the 12th release of Apache BookKeeper! + +The 4.7.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.7.1. These fixes include +bug fixes around DbLedgerStorage, failure handling around ensemble changes, bookie shutdown and such. + +See [BookKeeper 4.7.2 Release Notes](../docs/4.7.2/overview/releaseNotes) for details. + + ### 19 June, 2018: Release 4.7.1 available This is the 11th release of Apache BookKeeper! From dfb6462964959f3a1819f17286f472dd7a2695ac Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 31 Aug 2018 08:43:38 -0700 Subject: [PATCH 0008/1642] [RELEASE] Release Notes for 4.7.2 Descriptions of the changes in this PR: Add the release notes for 4.7.2 release Author: Sijie Guo Reviewers: Andrey Yegorov , Enrico Olivelli , Matteo Merli This closes #1628 from sijie/release_notes_472 --- site/docs/4.7.2/overview/releaseNotes.md | 37 +++++++++--------------- 1 file changed, 14 insertions(+), 23 deletions(-) diff --git a/site/docs/4.7.2/overview/releaseNotes.md b/site/docs/4.7.2/overview/releaseNotes.md index cb9d33fc537..45b0592bf2e 100644 --- a/site/docs/4.7.2/overview/releaseNotes.md +++ b/site/docs/4.7.2/overview/releaseNotes.md @@ -2,43 +2,34 @@ title: Apache BookKeeper 4.7.2 Release Notes --- -This is the eleventh release of Apache BookKeeper! +This is the 12th release of Apache BookKeeper! -The 4.7.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.7.0. +The 4.7.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.7.1. -Apache BookKeeper users who are using 4.7.0 are encouraged to upgrade to 4.7.2. The technical details of this release are summarized +Apache BookKeeper users who are using 4.7.1 are encouraged to upgrade to 4.7.2. The technical details of this release are summarized below. ## Highlights -- Performance enhancement on eliminating bytes copying in `AddEntry` code path, see [apache/bookkeeper#1361](https://github.com/apache/bookkeeper/pull/1361) +- Fix high cpu usage issue in DbLedgerStorage by avoiding using RocksDD#deleteRange, see [apache/bookkeeper#1620](https://github.com/apache/bookkeeper/pull/1620) -- Introduce Fast and Garbage-Free Statistics Timers in Codahale Stats Provider, see [apache/bookkeeper#1364](https://github.com/apache/bookkeeper/pull/1364) +- Fix deadlock in Auditor blocking zookeeper thread, see [apache/bookkeeper#1619](https://github.com/apache/bookkeeper/pull/1619) -- Fix OrderedScheduler handling null key, see [apache/bookkeeper#1372](https://github.com/apache/bookkeeper/pull/1372) +- Fix ArrayIndexOutOfBoundsException on ConcurrentLongHashMap, see [apache/bookkeeper#1606](https://github.com/apache/bookkeeper/pull/1606) -- Fix zookeeper ledger manager on handling no ledger exists, see [apache/bookkeeper#1382](https://github.com/apache/bookkeeper/pull/1382) +- Fix deferred failure handling causes data loss, see [apache/bookkeeper#1591](https://github.com/apache/bookkeeper/pull/1591) -- Fix long poll reads when ensemble size is larger than write quorum size, see [apache/bookkeeper#1404](https://github.com/apache/bookkeeper/pull/1404) +- Fix ConcurrentModificationException using nonblocking logReader#readNext, see [apache/bookkeeper#1544](https://github.com/apache/bookkeeper/pull/1544) -- Fix IllegalReferenceCount on filling readahead cache for DbLedgerStorage, see [apache/bookkeeper#1487](https://github.com/apache/bookkeeper/issues/1487) +- Fix Bookie shutdown fails to exit, see [apache/bookkeeper#1543](https://github.com/apache/bookkeeper/issues/1543) -- Fix LedgerEntry recycling issue on long poll speculative reads, see [apache/bookkeeper#1509](https://github.com/apache/bookkeeper/pull/1509) +- Fix race conditions on accessing guava multimap in PCBC when using v2 protocol, see [apache/bookkeeper#1618](https://github.com/apache/bookkeeper/pull/1618) -- Various bug fixes and improvements around bookkeeper table service, see changes under [apache/bookkeeper#release/4.7.2](https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93&q=is%3Aclosed+label%3Aarea%2Ftableservice+label%3Arelease%2F4.7.1) +### Dependency Changes -### Dependencies Upgrade - -Here is a list of dependencies changed in 4.7.2: - -- [Grpc](https://grpc.io/) is upgraded from `1.5.0` to `1.12.0`. See [apache/bookkeeper#1441](https://github.com/apache/bookkeeper/pull/1441) -- [Netty](http://netty.io/) is upgraded from `4.1.12` to `4.1.22`. See [apache/bookkeeper#1441](https://github.com/apache/bookkeeper/pull/1441) -- [Protobuf](https://developers.google.com/protocol-buffers/) is upgraded from `3.4.0` to `3.5.1`. See [apache/bookkeeper#1466](https://github.com/apache/bookkeeper/pull/1466) -- [RocksDB](http://rocksdb.org/) is upgraded from `5.8.6` to `5.13.1`. See [apache/bookkeeper#1466](https://github.com/apache/bookkeeper/pull/1466) - -`Reflective setAccessible(true)` is disabled by default in Netty while using java9+. This might result in performance degradation. Consider reenabling `Reflective setAccessible(true)` by setting -environment value `io.netty.tryReflectionSetAccessible` to `true`. See [netty/netty#7650](https://github.com/netty/netty/pull/7650) for more details. +In 4.7.2, [Zookeeper](https://zookeeper.apache.org/) version is downgraded from `3.5.3-beta` to `3.4.13` to avoid having a `beta` dependency and address maturity concerns. +The downgrade is safe and smooth. No extra actions are required from switching bookkeeper 4.7.1 to 4.7.2. ## Full list of changes -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.1+is%3Aclosed) +- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.2+is%3Aclosed) From f193748c468642910f887a3e160616c5418216a8 Mon Sep 17 00:00:00 2001 From: cguttapalem Date: Tue, 4 Sep 2018 12:29:51 -0700 Subject: [PATCH 0009/1642] Fix EntryLoggerAllocator.stop Descriptions of the changes in this PR: - make sure EntryLoggerAllocator.stop terminates pending allocatorExecutor's task. Otherwise tests would flap, since even after EntryLoggerAllocator.stop, there is possibility of new entrylog creation, which is not expected. Author: cguttapalem Reviewers: Andrey Yegorov , Enrico Olivelli , Sijie Guo This closes #1627 from reddycharan/entryalocshut --- .../bookkeeper/bookie/EntryLoggerAllocator.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java index 10e7715cd31..3ddd8e2e738 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java @@ -40,6 +40,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.EntryLogger.BufferedLogChannel; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -203,6 +205,16 @@ private void setLastLogId(File dir, long logId) throws IOException { void stop() { // wait until the preallocation finished. allocatorExecutor.shutdown(); + try { + if (!allocatorExecutor.awaitTermination(5, TimeUnit.SECONDS)) { + log.warn("Timedout while awaiting for allocatorExecutor's termination, so force shuttingdown"); + } + } catch (InterruptedException e) { + log.warn("Got InterruptedException while awaiting termination of allocatorExecutor, so force shuttingdown"); + Thread.currentThread().interrupt(); + } + allocatorExecutor.shutdownNow(); + log.info("Stopped entry logger preallocator."); } From 9a880cbbcb3cbe532422ff66af3368ea17ff6896 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Tue, 4 Sep 2018 12:31:15 -0700 Subject: [PATCH 0010/1642] Recovery uses immutable metadata And MetadataUpdateLoop to update it. Master issue: #281 Author: Ivan Kelly Reviewers: Enrico Olivelli , Sijie Guo This closes #1621 from ivankelly/recovery-immutable --- .../apache/bookkeeper/client/BKException.java | 13 + .../bookkeeper/client/LedgerHandle.java | 112 +------ .../bookkeeper/client/LedgerMetadata.java | 10 +- .../client/LedgerMetadataBuilder.java | 21 +- .../bookkeeper/client/LedgerOpenOp.java | 2 +- .../bookkeeper/client/LedgerRecoveryOp.java | 63 ++-- .../client/ReadOnlyLedgerHandle.java | 254 ++++++++++++++-- .../bookkeeper/client/BKExceptionTest.java | 67 +++++ .../client/LedgerRecovery2Test.java | 278 ++++++++++++++++++ .../bookkeeper/client/LedgerRecoveryTest.java | 23 +- .../client/MetadataUpdateLoopTest.java | 2 +- .../client/MockBookKeeperTestCase.java | 21 +- .../bookkeeper/client/MockClientContext.java | 7 +- .../client/ParallelLedgerRecoveryTest.java | 6 +- .../discover/MockRegistrationClient.java | 8 +- .../bookkeeper/proto/MockBookieClient.java | 166 ++++++----- .../bookkeeper/client/BookKeeperAccessor.java | 6 +- 17 files changed, 764 insertions(+), 295 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BKExceptionTest.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java index ddfc795acbc..8ab563be60c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java @@ -426,4 +426,17 @@ public BKLedgerIdOverflowException() { super(Code.LedgerIdOverflowException); } } + + /** + * Extract an exception code from an BKException, or use a default if it's another type. + */ + public static int getExceptionCode(Throwable t, int defaultCode) { + if (t instanceof BKException) { + return ((BKException) t).getCode(); + } else if (t.getCause() != null) { + return getExceptionCode(t.getCause(), defaultCode); + } else { + return defaultCode; + } + } } 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 741610dcef8..1b859a3bb6b 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 @@ -76,9 +76,7 @@ import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.TimedGenericCallback; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.State; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.proto.checksum.MacDigestManager; @@ -329,6 +327,10 @@ boolean setLedgerMetadata(LedgerMetadata expected, LedgerMetadata newMetadata) { // ensure that we only update the metadata if it is the object we expect it to be if (metadata == expected) { metadata = newMetadata; + if (metadata.isClosed()) { + lastAddConfirmed = lastAddPushed = metadata.getLastEntryId(); + length = metadata.getLength(); + } return true; } else { return false; @@ -2224,112 +2226,6 @@ void registerOperationFailureOnBookie(BookieSocketAddress bookie, long entryId) } } - - void recover(GenericCallback finalCb) { - recover(finalCb, null, false); - } - - /** - * Recover the ledger. - * - * @param finalCb - * callback after recovery is done. - * @param listener - * read entry listener on recovery reads. - * @param forceRecovery - * force the recovery procedure even the ledger metadata shows the ledger is closed. - */ - void recover(GenericCallback finalCb, - final @VisibleForTesting BookkeeperInternalCallbacks.ReadEntryListener listener, - final boolean forceRecovery) { - final GenericCallback cb = new TimedGenericCallback( - finalCb, - BKException.Code.OK, - clientCtx.getClientStats().getRecoverOpLogger()); - boolean wasClosed = false; - boolean wasInRecovery = false; - - LedgerMetadata metadata = getLedgerMetadata(); - synchronized (this) { - if (metadata.isClosed()) { - if (forceRecovery) { - wasClosed = false; - // mark the ledger back to in recovery state, so it would proceed ledger recovery again. - wasInRecovery = false; - metadata.markLedgerInRecovery(); - } else { - lastAddConfirmed = lastAddPushed = metadata.getLastEntryId(); - length = metadata.getLength(); - wasClosed = true; - } - } else { - wasClosed = false; - if (metadata.isInRecovery()) { - wasInRecovery = true; - } else { - wasInRecovery = false; - metadata.markLedgerInRecovery(); - } - } - } - - if (wasClosed) { - // We are already closed, nothing to do - cb.operationComplete(BKException.Code.OK, null); - return; - } - - if (wasInRecovery) { - // if metadata is already in recover, dont try to write again, - // just do the recovery from the starting point - new LedgerRecoveryOp(LedgerHandle.this, clientCtx, cb) - .setEntryListener(listener) - .initiate(); - return; - } - - writeLedgerConfig(new OrderedGenericCallback(clientCtx.getMainWorkerPool(), ledgerId) { - @Override - public void safeOperationComplete(final int rc, LedgerMetadata writtenMetadata) { - if (rc == BKException.Code.MetadataVersionException) { - rereadMetadata(new OrderedGenericCallback(clientCtx.getMainWorkerPool(), - ledgerId) { - @Override - public void safeOperationComplete(int rc, LedgerMetadata newMeta) { - if (rc != BKException.Code.OK) { - cb.operationComplete(rc, null); - } else { - LedgerHandle.this.metadata = newMeta; - recover(cb, listener, forceRecovery); - } - } - - @Override - public String toString() { - return String.format("ReReadMetadataForRecover(%d)", ledgerId); - } - }); - } else if (rc == BKException.Code.OK) { - // we only could issue recovery operation after we successfully update the ledger state to - // in recovery otherwise, it couldn't prevent us advancing last confirmed while the other writer is - // closing the ledger, which will cause inconsistent last add confirmed on bookies & zookeeper - // metadata. - new LedgerRecoveryOp(LedgerHandle.this, clientCtx, cb) - .setEntryListener(listener) - .initiate(); - } else { - LOG.error("Error writing ledger {} config: {}", ledgerId, BKException.codeLogger(rc)); - cb.operationComplete(rc, null); - } - } - - @Override - public String toString() { - return String.format("WriteLedgerConfigForRecover(%d)", ledgerId); - } - }); - } - static class NoopCloseCallback implements CloseCallback { static NoopCloseCallback instance = new NoopCloseCallback(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 4f58aa8b88a..ffb0d6aa7bc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -135,6 +135,7 @@ public LedgerMetadata(int ensembleSize, int ackQuorumSize, LedgerMetadataFormat.State state, java.util.Optional lastEntryId, + java.util.Optional length, Map> ensembles, DigestType digestType, java.util.Optional password, @@ -148,7 +149,7 @@ public LedgerMetadata(int ensembleSize, this.ackQuorumSize = ackQuorumSize; this.state = state; lastEntryId.ifPresent((eid) -> this.lastEntryId = eid); - + length.ifPresent((l) -> this.length = l); setEnsembles(ensembles); if (state != LedgerMetadataFormat.State.CLOSED) { currentEnsemble = this.ensembles.lastEntry().getValue(); @@ -787,4 +788,11 @@ Set getBookiesInThisLedger() { return bookies; } + java.util.Optional getLastEnsembleKey() { + if (ensembles.size() > 0) { + return java.util.Optional.of(ensembles.lastKey()); + } else { + return java.util.Optional.empty(); + } + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index 8c37bd53917..a9d83b04757 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableMap; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -40,6 +41,7 @@ class LedgerMetadataBuilder { private LedgerMetadataFormat.State state = LedgerMetadataFormat.State.OPEN; private Optional lastEntryId = Optional.empty(); + private Optional length = Optional.empty(); private TreeMap> ensembles = new TreeMap<>(); @@ -67,6 +69,10 @@ static LedgerMetadataBuilder from(LedgerMetadata other) { if (lastEntryId != LedgerHandle.INVALID_ENTRY_ID) { builder.lastEntryId = Optional.of(lastEntryId); } + long length = other.getLength(); + if (length > 0) { + builder.length = Optional.of(length); + } builder.ensembles.putAll(other.getEnsembles()); @@ -85,6 +91,11 @@ static LedgerMetadataBuilder from(LedgerMetadata other) { return builder; } + LedgerMetadataBuilder withPassword(byte[] password) { + this.password = Optional.of(Arrays.copyOf(password, password.length)); + return this; + } + LedgerMetadataBuilder withEnsembleSize(int ensembleSize) { checkState(ensembles.size() == 0, "Can only set ensemble size before adding ensembles to the builder"); this.ensembleSize = ensembleSize; @@ -109,17 +120,21 @@ LedgerMetadataBuilder replaceEnsembleEntry(long firstEntry, List { final long ledgerId; final OpenCallback cb; final Object ctx; - LedgerHandle lh; + ReadOnlyLedgerHandle lh; final byte[] passwd; boolean doRecovery = true; boolean administrativeOpen = false; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java index 6ab25d60322..923c4df2025 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java @@ -18,11 +18,10 @@ package org.apache.bookkeeper.client; import com.google.common.annotations.VisibleForTesting; -import java.util.concurrent.atomic.AtomicBoolean; + +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; -import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener; import org.apache.bookkeeper.proto.checksum.DigestManager.RecoveryData; import org.slf4j.Logger; @@ -32,9 +31,7 @@ * This class encapsulated the ledger recovery operation. It first does a read * with entry-id of -1 (BookieProtocol.LAST_ADD_CONFIRMED) to all bookies. Then * starting from the last confirmed entry (from hints in the ledger entries), - * it reads forward until it is not able to find a particular entry. It closes - * the ledger at that entry. - * + * it reads forward until it is not able to find a particular entry. */ class LedgerRecoveryOp implements ReadEntryListener, AddCallback { @@ -42,13 +39,13 @@ class LedgerRecoveryOp implements ReadEntryListener, AddCallback { final LedgerHandle lh; final ClientContext clientCtx; + final CompletableFuture promise; final AtomicLong readCount, writeCount; volatile boolean readDone; - final AtomicBoolean callbackDone; volatile long startEntryToRead; volatile long endEntryToRead; - final GenericCallback cb; + // keep a copy of metadata for recovery. LedgerMetadata metadataForRecovery; @@ -72,13 +69,11 @@ protected LedgerMetadata getLedgerMetadata() { } - public LedgerRecoveryOp(LedgerHandle lh, ClientContext clientCtx, - GenericCallback cb) { + public LedgerRecoveryOp(LedgerHandle lh, ClientContext clientCtx) { readCount = new AtomicLong(0); writeCount = new AtomicLong(0); readDone = false; - callbackDone = new AtomicBoolean(false); - this.cb = cb; + this.promise = new CompletableFuture<>(); this.lh = lh; this.clientCtx = clientCtx; } @@ -96,7 +91,7 @@ LedgerRecoveryOp setEntryListener(ReadEntryListener entryListener) { return this; } - public void initiate() { + public CompletableFuture initiate() { ReadLastConfirmedOp rlcop = new ReadLastConfirmedOp(lh, clientCtx.getBookieClient(), lh.getCurrentEnsemble(), new ReadLastConfirmedOp.LastConfirmedDataCallback() { public void readLastConfirmedDataComplete(int rc, RecoveryData data) { @@ -125,24 +120,27 @@ public void readLastConfirmedDataComplete(int rc, RecoveryData data) { * from writing to it. */ rlcop.initiateWithFencing(); + + return promise; } private void submitCallback(int rc) { if (BKException.Code.OK == rc) { clientCtx.getClientStats().getRecoverAddCountLogger().registerSuccessfulValue(writeCount.get()); clientCtx.getClientStats().getRecoverReadCountLogger().registerSuccessfulValue(readCount.get()); + promise.complete(lh); } else { clientCtx.getClientStats().getRecoverAddCountLogger().registerFailedValue(writeCount.get()); clientCtx.getClientStats().getRecoverReadCountLogger().registerFailedValue(readCount.get()); + promise.completeExceptionally(BKException.create(rc)); } - cb.operationComplete(rc, null); } /** * Try to read past the last confirmed. */ private void doRecoveryRead() { - if (!callbackDone.get()) { + if (!promise.isDone()) { startEntryToRead = endEntryToRead + 1; endEntryToRead = endEntryToRead + clientCtx.getConf().recoveryReadBatchSize; new RecoveryReadOp(lh, clientCtx, startEntryToRead, endEntryToRead, this, null) @@ -150,26 +148,6 @@ private void doRecoveryRead() { } } - private void closeAndCallback() { - if (callbackDone.compareAndSet(false, true)) { - lh.asyncCloseInternal(new CloseCallback() { - @Override - public void closeComplete(int rc, LedgerHandle lh, Object ctx) { - if (rc != BKException.Code.OK) { - LOG.warn("Close ledger {} failed during recovery: ", - LedgerRecoveryOp.this.lh.getId(), BKException.getMessage(rc)); - submitCallback(rc); - } else { - submitCallback(BKException.Code.OK); - if (LOG.isDebugEnabled()) { - LOG.debug("After closing length is: {}", lh.getLength()); - } - } - } - }, null, BKException.Code.LedgerClosedException); - } - } - @Override public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object ctx) { // notify entry listener on individual entries being read during ledger recovery. @@ -179,7 +157,7 @@ public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object c } // we only trigger recovery add an entry when readDone == false && callbackDone == false - if (!callbackDone.get() && !readDone && rc == BKException.Code.OK) { + if (!promise.isDone() && !readDone && rc == BKException.Code.OK) { readCount.incrementAndGet(); byte[] data = entry.getEntry(); @@ -211,15 +189,15 @@ public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object c if (rc == BKException.Code.NoSuchEntryException || rc == BKException.Code.NoSuchLedgerExistsException) { readDone = true; if (readCount.get() == writeCount.get()) { - closeAndCallback(); + submitCallback(BKException.Code.OK); } return; } // otherwise, some other error, we can't handle - if (BKException.Code.OK != rc && callbackDone.compareAndSet(false, true)) { + if (BKException.Code.OK != rc && !promise.isDone()) { LOG.error("Failure {} while reading entries: ({} - {}), ledger: {} while recovering ledger", - BKException.getMessage(rc), startEntryToRead, endEntryToRead, lh.getId()); + BKException.getMessage(rc), startEntryToRead, endEntryToRead, lh.getId()); submitCallback(rc); } else if (BKException.Code.OK == rc) { // we are here is because we successfully read an entry but readDone was already set to true. @@ -235,15 +213,12 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { if (rc != BKException.Code.OK) { LOG.error("Failure {} while writing entry: {} while recovering ledger: {}", BKException.codeLogger(rc), entryId + 1, lh.ledgerId); - if (callbackDone.compareAndSet(false, true)) { - // Give up, we can't recover from this error - submitCallback(rc); - } + submitCallback(rc); return; } long numAdd = writeCount.incrementAndGet(); if (readDone && readCount.get() == numAdd) { - closeAndCallback(); + submitCallback(rc); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java index aa0290b3065..e2c9a44a3c4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java @@ -20,8 +20,21 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.annotations.VisibleForTesting; + import java.security.GeneralSecurityException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.NavigableMap; +import java.util.Optional; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.RejectedExecutionException; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; @@ -30,10 +43,16 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.TimedGenericCallback; import org.apache.bookkeeper.util.SafeRunnable; import org.apache.bookkeeper.versioning.Version; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Read only ledger handle. This ledger handle allows you to * read from a ledger but not to write to it. It overrides all @@ -41,6 +60,10 @@ * It should be returned for BookKeeper#openLedger operations. */ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListener { + private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyLedgerHandle.class); + + private Object metadataLock = new Object(); + private final NavigableMap> newEnsemblesFromRecovery = new TreeMap<>(); class MetadataUpdater extends SafeRunnable { @@ -127,29 +150,6 @@ public void asyncAddEntry(final byte[] data, final int offset, final int length, LedgerHandle.INVALID_ENTRY_ID, ctx); } - @Override - void handleBookieFailure(final Map failedBookies) { - blockAddCompletions.incrementAndGet(); - synchronized (getLedgerMetadata()) { - try { - EnsembleInfo ensembleInfo = replaceBookieInMetadata(failedBookies, - numEnsembleChanges.incrementAndGet()); - if (ensembleInfo.replacedBookies.isEmpty()) { - blockAddCompletions.decrementAndGet(); - return; - } - blockAddCompletions.decrementAndGet(); - // the failed bookie has been replaced - unsetSuccessAndSendWriteRequest(ensembleInfo.newEnsemble, ensembleInfo.replacedBookies); - } catch (BKException.BKNotEnoughBookiesException e) { - LOG.error("Could not get additional bookie to " - + "remake ensemble, closing ledger: " + ledgerId); - handleUnrecoverableErrorDuringAdd(e.getCode()); - return; - } - } - } - @Override public void onChanged(long lid, LedgerMetadata newMetadata) { if (LOG.isDebugEnabled()) { @@ -208,4 +208,212 @@ public void readLastConfirmedComplete(int rc, long lastConfirmed, Object ctx) { } }, ctx); } + + List replaceBookiesInEnsemble(LedgerMetadata metadata, + List oldEnsemble, + Map failedBookies) + throws BKException.BKNotEnoughBookiesException { + List newEnsemble = new ArrayList<>(oldEnsemble); + + int ensembleSize = metadata.getEnsembleSize(); + int writeQ = metadata.getWriteQuorumSize(); + int ackQ = metadata.getAckQuorumSize(); + Map customMetadata = metadata.getCustomMetadata(); + + Set exclude = new HashSet<>(failedBookies.values()); + + int replaced = 0; + for (Map.Entry entry : failedBookies.entrySet()) { + int idx = entry.getKey(); + BookieSocketAddress addr = entry.getValue(); + if (LOG.isDebugEnabled()) { + LOG.debug("[EnsembleChange-L{}] replacing bookie: {} index: {}", getId(), addr, idx); + } + + if (!newEnsemble.get(idx).equals(addr)) { + if (LOG.isDebugEnabled()) { + LOG.debug("[EnsembleChange-L{}] Not changing failed bookie {} at index {}, already changed to {}", + getId(), addr, idx, newEnsemble.get(idx)); + } + continue; + } + try { + BookieSocketAddress newBookie = clientCtx.getBookieWatcher().replaceBookie( + ensembleSize, writeQ, ackQ, customMetadata, newEnsemble, idx, exclude); + newEnsemble.set(idx, newBookie); + + replaced++; + } catch (BKException.BKNotEnoughBookiesException e) { + // if there is no bookie replaced, we throw not enough bookie exception + if (replaced <= 0) { + throw e; + } else { + break; + } + } + } + return newEnsemble; + } + + private static Set diffEnsemble(List e1, + List e2) { + checkArgument(e1.size() == e2.size(), "Ensembles must be of same size"); + Set diff = new HashSet<>(); + for (int i = 0; i < e1.size(); i++) { + if (!e1.get(i).equals(e2.get(i))) { + diff.add(i); + } + } + return diff; + } + + /** + * For a read only ledger handle, this method will only ever be called during recovery, + * when we are reading forward from LAC and writing back those entries. As such, + * unlike with LedgerHandle, we do not want to persist changes to the metadata as they occur, + * but rather, we want to defer the persistence until recovery has completed, and do it all + * on the close. + */ + @Override + void handleBookieFailure(final Map failedBookies) { + blockAddCompletions.incrementAndGet(); + + // handleBookieFailure should always run in the ordered executor thread for this + // ledger, so this synchronized should be unnecessary, but putting it here now + // just in case (can be removed when we validate threads) + synchronized (metadataLock) { + long lac = getLastAddConfirmed(); + LedgerMetadata metadata = getLedgerMetadata(); + List currentEnsemble = getCurrentEnsemble(); + try { + List newEnsemble = replaceBookiesInEnsemble(metadata, currentEnsemble, + failedBookies); + + Set replaced = diffEnsemble(currentEnsemble, newEnsemble); + blockAddCompletions.decrementAndGet(); + if (!replaced.isEmpty()) { + newEnsemblesFromRecovery.put(lac + 1, newEnsemble); + unsetSuccessAndSendWriteRequest(newEnsemble, replaced); + } + } catch (BKException.BKNotEnoughBookiesException e) { + LOG.error("Could not get additional bookie to remake ensemble, closing ledger: {}", ledgerId); + + handleUnrecoverableErrorDuringAdd(e.getCode()); + return; + } + } + } + + @Override + void handleUnrecoverableErrorDuringAdd(int rc) { + errorOutPendingAdds(rc); + } + + void recover(GenericCallback finalCb) { + recover(finalCb, null, false); + } + + /** + * Recover the ledger. + * + * @param finalCb + * callback after recovery is done. + * @param listener + * read entry listener on recovery reads. + * @param forceRecovery + * force the recovery procedure even the ledger metadata shows the ledger is closed. + */ + void recover(GenericCallback finalCb, + final @VisibleForTesting ReadEntryListener listener, + final boolean forceRecovery) { + final GenericCallback cb = new TimedGenericCallback( + finalCb, + BKException.Code.OK, + clientCtx.getClientStats().getRecoverOpLogger()); + + MetadataUpdateLoop.NeedsUpdatePredicate needsUpdate = + (metadata) -> !(metadata.isClosed() || metadata.isInRecovery()); + if (forceRecovery) { + // in the force recovery case, we want to update the metadata + // to IN_RECOVERY, even if the ledger is already closed + needsUpdate = (metadata) -> !metadata.isInRecovery(); + } + new MetadataUpdateLoop( + clientCtx.getLedgerManager(), getId(), + this::getLedgerMetadata, + needsUpdate, + (metadata) -> LedgerMetadataBuilder.from(metadata).withInRecoveryState().build(), + this::setLedgerMetadata) + .run() + .thenCompose((metadata) -> { + if (metadata.isClosed()) { + return CompletableFuture.completedFuture(ReadOnlyLedgerHandle.this); + } else { + return new LedgerRecoveryOp(ReadOnlyLedgerHandle.this, clientCtx) + .setEntryListener(listener) + .initiate(); + } + }) + .thenCompose((ignore) -> closeRecovered()) + .whenComplete((ignore, ex) -> { + if (ex != null) { + cb.operationComplete( + BKException.getExceptionCode(ex, BKException.Code.UnexpectedConditionException), null); + } else { + cb.operationComplete(BKException.Code.OK, null); + } + }); + } + + CompletableFuture closeRecovered() { + long lac, len; + synchronized (this) { + lac = lastAddConfirmed; + len = length; + } + LOG.info("Closing recovered ledger {} at entry {}", getId(), lac); + CompletableFuture f = new MetadataUpdateLoop( + clientCtx.getLedgerManager(), getId(), + this::getLedgerMetadata, + (metadata) -> metadata.isInRecovery(), + (metadata) -> { + LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata); + Optional lastEnsembleKey = metadata.getLastEnsembleKey(); + checkState(lastEnsembleKey.isPresent(), + "Metadata shouldn't have been created without at least one ensemble"); + synchronized (metadataLock) { + newEnsemblesFromRecovery.entrySet().forEach( + (e) -> { + checkState(e.getKey() >= lastEnsembleKey.get(), + "Once a ledger is in recovery, noone can add ensembles without closing"); + // Occurs when a bookie need to be replaced at very start of recovery + if (lastEnsembleKey.get().equals(e.getKey())) { + builder.replaceEnsembleEntry(e.getKey(), e.getValue()); + } else { + builder.newEnsembleEntry(e.getKey(), e.getValue()); + } + }); + } + return builder.closingAt(lac, len).build(); + }, + this::setLedgerMetadata).run(); + f.thenRun(() -> { + synchronized (metadataLock) { + newEnsemblesFromRecovery.clear(); + } + }); + return f; + } + + @Override + List getCurrentEnsemble() { + synchronized (metadataLock) { + if (!newEnsemblesFromRecovery.isEmpty()) { + return newEnsemblesFromRecovery.lastEntry().getValue(); + } else { + return super.getCurrentEnsemble(); + } + } + } + } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BKExceptionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BKExceptionTest.java new file mode 100644 index 00000000000..2692df87216 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BKExceptionTest.java @@ -0,0 +1,67 @@ +/* + * + * 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.client; + +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Test for extracting codes from BKException. + */ +public class BKExceptionTest { + @Test + public void testBKExceptionCode() { + Assert.assertEquals(BKException.Code.WriteException, + BKException.getExceptionCode(new BKException.BKWriteException(), + BKException.Code.ReadException)); + } + + @Test + public void testNonBKExceptionCode() { + Assert.assertEquals(BKException.Code.ReadException, + BKException.getExceptionCode(new Exception(), + BKException.Code.ReadException)); + + } + + @Test + public void testNestedBKExceptionCode() { + Assert.assertEquals(BKException.Code.WriteException, + BKException.getExceptionCode( + new ExecutionException("test", new BKException.BKWriteException()), + BKException.Code.ReadException)); + } + + @Test + public void testDoubleNestedBKExceptionCode() { + Assert.assertEquals(BKException.Code.WriteException, + BKException.getExceptionCode( + new ExecutionException("test", + new CompletionException("blah", + new BKException.BKWriteException())), + BKException.Code.ReadException)); + + } +} + diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java new file mode 100644 index 00000000000..9a7cc7fa880 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java @@ -0,0 +1,278 @@ +/* + * + * 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.client; + +import com.google.common.collect.Lists; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; +import org.junit.Assert; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Ledger recovery tests using mocks rather than a real cluster. + */ +public class LedgerRecovery2Test { + private static final Logger log = LoggerFactory.getLogger(LedgerRecovery2Test.class); + + private static final byte[] PASSWD = "foobar".getBytes(); + private static final BookieSocketAddress b1 = new BookieSocketAddress("b1", 3181); + private static final BookieSocketAddress b2 = new BookieSocketAddress("b2", 3181); + private static final BookieSocketAddress b3 = new BookieSocketAddress("b3", 3181); + private static final BookieSocketAddress b4 = new BookieSocketAddress("b4", 3181); + private static final BookieSocketAddress b5 = new BookieSocketAddress("b5", 3181); + + private static LedgerMetadata setupLedger(ClientContext clientCtx, long ledgerId, + List bookies) throws Exception { + LedgerMetadata md = LedgerMetadataBuilder.create() + .withPassword(PASSWD) + .newEnsembleEntry(0, bookies).build(); + GenericCallbackFuture mdPromise = new GenericCallbackFuture<>(); + clientCtx.getLedgerManager().createLedgerMetadata(1L, md, mdPromise); + return mdPromise.get(); + } + + @Test + public void testCantRecoverAllDown() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + + LedgerMetadata md = setupLedger(clientCtx, 1L, Lists.newArrayList(b1, b2, b3)); + + clientCtx.getMockBookieClient().errorBookies(b1, b2, b3); + + ReadOnlyLedgerHandle lh = new ReadOnlyLedgerHandle( + clientCtx, 1L, md, BookKeeper.DigestType.CRC32C, PASSWD, false); + try { + GenericCallbackFuture promise = new GenericCallbackFuture<>(); + lh.recover(promise, null, false); + promise.get(); + Assert.fail("Recovery shouldn't have been able to complete"); + } catch (ExecutionException ee) { + Assert.assertEquals(BKException.BKReadException.class, ee.getCause().getClass()); + } + } + + @Test + public void testCanReadLacButCantWrite() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + + LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + + clientCtx.getMockBookieClient().seedEntries(b1, 1L, 0L, -1L); + clientCtx.getMockBookieClient().setPreWriteHook( + (bookie, ledgerId, entryId) -> FutureUtils.exception(new BKException.BKWriteException())); + + ReadOnlyLedgerHandle lh = new ReadOnlyLedgerHandle( + clientCtx, 1L, md, BookKeeper.DigestType.CRC32C, PASSWD, false); + try { + GenericCallbackFuture promise = new GenericCallbackFuture<>(); + lh.recover(promise, null, false); + promise.get(); + Assert.fail("Recovery shouldn't have been able to complete"); + } catch (ExecutionException ee) { + Assert.assertEquals(BKException.BKNotEnoughBookiesException.class, ee.getCause().getClass()); + } + } + + @Test + public void testMetadataClosedDuringRecovery() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + + LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + + CompletableFuture writingBack = new CompletableFuture<>(); + CompletableFuture blocker = new CompletableFuture<>(); + clientCtx.getMockBookieClient().seedEntries(b1, 1L, 0L, -1L); + // will block recovery at the writeback phase + clientCtx.getMockBookieClient().setPreWriteHook( + (bookie, ledgerId, entryId) -> { + writingBack.complete(null); + return blocker; + }); + + ReadOnlyLedgerHandle lh = new ReadOnlyLedgerHandle( + clientCtx, 1L, md, BookKeeper.DigestType.CRC32C, PASSWD, false); + + GenericCallbackFuture recoveryPromise = new GenericCallbackFuture<>(); + lh.recover(recoveryPromise, null, false); + + writingBack.get(10, TimeUnit.SECONDS); + + GenericCallbackFuture readPromise = new GenericCallbackFuture<>(); + clientCtx.getLedgerManager().readLedgerMetadata(1L, readPromise); + LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(readPromise.get()); + GenericCallbackFuture writePromise = new GenericCallbackFuture<>(); + clientCtx.getLedgerManager().writeLedgerMetadata(1L, builder.closingAt(-1, 0).build(), writePromise); + writePromise.get(); + + // allow recovery to continue + blocker.complete(null); + + recoveryPromise.get(); + + Assert.assertEquals(lh.getLastAddConfirmed(), -1); + Assert.assertEquals(lh.getLength(), 0); + } + + @Test + public void testNewEnsembleAddedDuringRecovery() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + + LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + + CompletableFuture writingBack = new CompletableFuture<>(); + CompletableFuture blocker = new CompletableFuture<>(); + CompletableFuture failing = new CompletableFuture<>(); + clientCtx.getMockBookieClient().seedEntries(b1, 1L, 0L, -1L); + // will block recovery at the writeback phase + clientCtx.getMockBookieClient().setPreWriteHook( + (bookie, ledgerId, entryId) -> { + writingBack.complete(null); + if (bookie.equals(b3)) { + return failing; + } else { + return blocker; + } + }); + + ReadOnlyLedgerHandle lh = new ReadOnlyLedgerHandle( + clientCtx, 1L, md, BookKeeper.DigestType.CRC32C, PASSWD, false); + + GenericCallbackFuture recoveryPromise = new GenericCallbackFuture<>(); + lh.recover(recoveryPromise, null, false); + + writingBack.get(10, TimeUnit.SECONDS); + + GenericCallbackFuture readPromise = new GenericCallbackFuture<>(); + clientCtx.getLedgerManager().readLedgerMetadata(1L, readPromise); + LedgerMetadata newMeta = LedgerMetadataBuilder.from(readPromise.get()) + .newEnsembleEntry(1L, Lists.newArrayList(b1, b2, b4)).build(); + GenericCallbackFuture writePromise = new GenericCallbackFuture<>(); + clientCtx.getLedgerManager().writeLedgerMetadata(1L, newMeta, writePromise); + writePromise.get(); + + // allow recovery to continue + failing.completeExceptionally(new BKException.BKWriteException()); + blocker.complete(null); + + try { + recoveryPromise.get(); + Assert.fail("Should fail on the update"); + } catch (ExecutionException ee) { + Assert.assertEquals(BKException.BKUnexpectedConditionException.class, ee.getCause().getClass()); + } + } + + @Test + public void testRecoveryBookieFailedAtStart() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + + LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + + CompletableFuture writingBack = new CompletableFuture<>(); + CompletableFuture blocker = new CompletableFuture<>(); + CompletableFuture failing = new CompletableFuture<>(); + clientCtx.getMockBookieClient().seedEntries(b1, 1L, 0L, -1L); + clientCtx.getMockBookieClient().errorBookies(b2); + + ReadOnlyLedgerHandle lh = new ReadOnlyLedgerHandle( + clientCtx, 1L, md, BookKeeper.DigestType.CRC32C, PASSWD, false); + + GenericCallbackFuture recoveryPromise = new GenericCallbackFuture<>(); + lh.recover(recoveryPromise, null, false); + recoveryPromise.get(); + + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), + Lists.newArrayList(b1, b4, b3)); + } + + @Test + public void testRecoveryOneBookieFailsDuring() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + + LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + clientCtx.getMockBookieClient().seedEntries(b1, 1L, 0L, -1L); + clientCtx.getMockBookieClient().seedEntries(b3, 1L, 1L, -1L); + clientCtx.getMockBookieClient().setPreWriteHook( + (bookie, ledgerId, entryId) -> { + if (bookie.equals(b2) && entryId == 1L) { + return FutureUtils.exception(new BKException.BKWriteException()); + } else { + return FutureUtils.value(null); + } + }); + + ReadOnlyLedgerHandle lh = new ReadOnlyLedgerHandle( + clientCtx, 1L, md, BookKeeper.DigestType.CRC32C, PASSWD, false); + + GenericCallbackFuture recoveryPromise = new GenericCallbackFuture<>(); + lh.recover(recoveryPromise, null, false); + recoveryPromise.get(); + + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), + Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), + Lists.newArrayList(b1, b4, b3)); + Assert.assertEquals(lh.getLastAddConfirmed(), 1L); + } + + @Test + public void testRecoveryTwoBookiesFailOnSameEntry() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + clientCtx.getMockRegistrationClient().addBookies(b4, b5).get(); + + LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + clientCtx.getMockBookieClient().seedEntries(b1, 1L, 0L, -1L); + clientCtx.getMockBookieClient().setPreWriteHook( + (bookie, ledgerId, entryId) -> { + if (bookie.equals(b1) || bookie.equals(b2)) { + return FutureUtils.exception(new BKException.BKWriteException()); + } else { + return FutureUtils.value(null); + } + }); + + ReadOnlyLedgerHandle lh = new ReadOnlyLedgerHandle( + clientCtx, 1L, md, BookKeeper.DigestType.CRC32C, PASSWD, false); + + GenericCallbackFuture recoveryPromise = new GenericCallbackFuture<>(); + lh.recover(recoveryPromise, null, false); + recoveryPromise.get(); + + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b3)); + Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b4)); + Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b5)); + Assert.assertEquals(lh.getLastAddConfirmed(), 0L); + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java index e1260015344..6a21da2d61c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java @@ -29,9 +29,9 @@ import java.io.IOException; import java.util.Enumeration; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.bookie.Bookie; @@ -42,7 +42,6 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; @@ -357,7 +356,6 @@ public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] // start a new good server startNewBookie(); - LedgerHandle lhafter = bkc.openLedger(lhbefore.getId(), digestType, "".getBytes()); assertEquals("Fenced ledger should have correct lastAddConfirmed", @@ -479,24 +477,13 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { LedgerHandle recoverLh = newBk.openLedgerNoRecovery(lh.getId(), digestType, "".getBytes()); assertEquals(BookieProtocol.INVALID_ENTRY_ID, recoverLh.getLastAddConfirmed()); - final CountDownLatch recoverLatch = new CountDownLatch(1); - final AtomicBoolean success = new AtomicBoolean(false); - MockClientContext parallelReadCtx = MockClientContext.copyOf(bkc.getClientCtx()) .setConf(ClientInternalConf.fromConfig(newConf.setEnableParallelRecoveryRead(true))); - LedgerRecoveryOp recoveryOp = new LedgerRecoveryOp( - recoverLh, parallelReadCtx, - new BookkeeperInternalCallbacks.GenericCallback() { - @Override - public void operationComplete(int rc, Void result) { - success.set(BKException.Code.OK == rc); - recoverLatch.countDown(); - } - }); - recoveryOp.initiate(); - recoverLatch.await(10, TimeUnit.SECONDS); - assertTrue(success.get()); + LedgerRecoveryOp recoveryOp = new LedgerRecoveryOp(recoverLh, parallelReadCtx); + CompletableFuture f = recoveryOp.initiate(); + f.get(10, TimeUnit.SECONDS); + assertEquals(numEntries, recoveryOp.readCount.get()); assertEquals(numEntries, recoveryOp.writeCount.get()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java index 5ed75ce9770..e70a7d31e1f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java @@ -361,7 +361,7 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { ledgerId, reference::get, (currentMetadata) -> !currentMetadata.isClosed(), - (currentMetadata) -> LedgerMetadataBuilder.from(currentMetadata).closingAtEntry(10L).build(), + (currentMetadata) -> LedgerMetadataBuilder.from(currentMetadata).closingAt(10L, 100L).build(), reference::compareAndSet).run(); CompletableFuture loop2 = new MetadataUpdateLoop( lm, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java index 783e435d6c2..c290a7e01be 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java @@ -430,16 +430,17 @@ private void setupCreateLedgerMetadata() { @SuppressWarnings("unchecked") private void setupWriteLedgerMetadata() { doAnswer(invocation -> { - Object[] args = invocation.getArguments(); - Long ledgerId = (Long) args[0]; - LedgerMetadata metadata = (LedgerMetadata) args[1]; - BookkeeperInternalCallbacks.GenericCallback cb = (BookkeeperInternalCallbacks.GenericCallback) args[2]; - executor.executeOrdered(ledgerId, () -> { - mockLedgerMetadataRegistry.put(ledgerId, new LedgerMetadata(metadata)); - cb.operationComplete(BKException.Code.OK, null); - }); - return null; - }).when(ledgerManager).writeLedgerMetadata(anyLong(), any(), any()); + Object[] args = invocation.getArguments(); + Long ledgerId = (Long) args[0]; + LedgerMetadata metadata = (LedgerMetadata) args[1]; + BookkeeperInternalCallbacks.GenericCallback cb = (BookkeeperInternalCallbacks.GenericCallback) args[2]; + executor.executeOrdered(ledgerId, () -> { + LedgerMetadata newMetadata = LedgerMetadataBuilder.from(metadata).build(); + mockLedgerMetadataRegistry.put(ledgerId, newMetadata); + cb.operationComplete(BKException.Code.OK, newMetadata); + }); + return null; + }).when(ledgerManager).writeLedgerMetadata(anyLong(), any(), any()); } @SuppressWarnings("unchecked") diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java index 2f5b2dc5a40..f36c00885bf 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java @@ -51,16 +51,19 @@ public class MockClientContext implements ClientContext { private BooleanSupplier isClientClosed; private MockRegistrationClient regClient; - static MockClientContext create() { + static MockClientContext create() throws Exception { ClientConfiguration conf = new ClientConfiguration(); OrderedScheduler scheduler = OrderedScheduler.newSchedulerBuilder().name("mock-executor").numThreads(1).build(); MockRegistrationClient regClient = new MockRegistrationClient(); EnsemblePlacementPolicy placementPolicy = new DefaultEnsemblePlacementPolicy(); + BookieWatcherImpl bookieWatcherImpl = new BookieWatcherImpl(conf, placementPolicy, + regClient, NullStatsLogger.INSTANCE); + bookieWatcherImpl.initialBlockingBookieRead(); return new MockClientContext() .setConf(ClientInternalConf.fromConfig(conf)) .setLedgerManager(new MockLedgerManager()) - .setBookieWatcher(new BookieWatcherImpl(conf, placementPolicy, regClient, NullStatsLogger.INSTANCE)) + .setBookieWatcher(bookieWatcherImpl) .setPlacementPolicy(placementPolicy) .setRegistrationClient(regClient) .setBookieClient(new MockBookieClient(scheduler)) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index 8b5f5a60321..9bdbfacce30 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -342,7 +342,7 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { final CountDownLatch recoverLatch = new CountDownLatch(1); final AtomicBoolean success = new AtomicBoolean(false); - recoverLh.recover(new GenericCallback() { + ((ReadOnlyLedgerHandle) recoverLh).recover(new GenericCallback() { @Override public void operationComplete(int rc, Void result) { LOG.info("Recovering ledger {} completed : {}.", lh.getId(), rc); @@ -459,7 +459,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, final AtomicBoolean isMetadataClosed = new AtomicBoolean(false); final AtomicInteger numSuccessCalls = new AtomicInteger(0); final AtomicInteger numFailureCalls = new AtomicInteger(0); - recoverLh.recover(new GenericCallback() { + ((ReadOnlyLedgerHandle) recoverLh).recover(new GenericCallback() { @Override public void operationComplete(int rc, Void result) { if (BKException.Code.OK == rc) { @@ -639,7 +639,7 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { tlm1.setLatch(metadataLatch); final CountDownLatch recoverLatch = new CountDownLatch(1); final AtomicBoolean recoverSuccess = new AtomicBoolean(false); - lh1.recover(new GenericCallback() { + ((ReadOnlyLedgerHandle) lh1).recover(new GenericCallback() { @Override public void operationComplete(int rc, Void result) { LOG.info("Recovering ledger {} completed : {}", lh1.getId(), rc); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java index 40178b98f88..8ce3686c20a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java @@ -55,7 +55,7 @@ private static Versioned> versioned(Set(Collections.unmodifiableSet(bookies), new LongVersion(version)); } - CompletableFuture addBookies(BookieSocketAddress... bookies) { + public CompletableFuture addBookies(BookieSocketAddress... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; @@ -68,7 +68,7 @@ CompletableFuture addBookies(BookieSocketAddress... bookies) { return promise; } - CompletableFuture removeBookies(BookieSocketAddress... bookies) { + public CompletableFuture removeBookies(BookieSocketAddress... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; @@ -81,7 +81,7 @@ CompletableFuture removeBookies(BookieSocketAddress... bookies) { return promise; } - CompletableFuture addReadOnlyBookies(BookieSocketAddress... bookies) { + public CompletableFuture addReadOnlyBookies(BookieSocketAddress... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; @@ -94,7 +94,7 @@ CompletableFuture addReadOnlyBookies(BookieSocketAddress... bookies) { return promise; } - CompletableFuture removeReadOnlyBookies(BookieSocketAddress... bookies) { + public CompletableFuture removeReadOnlyBookies(BookieSocketAddress... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java index 79cc5ba3f5f..04119d51543 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java @@ -25,19 +25,19 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import java.util.ArrayList; import java.util.Collections; import java.util.EnumSet; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Consumer; + import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.api.WriteFlag; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback; @@ -46,8 +46,9 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadLacCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteLacCallback; +import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; +import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.util.ByteBufList; -import org.apache.bookkeeper.util.SafeRunnable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,24 +64,36 @@ public class MockBookieClient implements BookieClient { final Set errorBookies = Collections.newSetFromMap(new ConcurrentHashMap()); - final Map stalledBookies = new HashMap<>(); - final Map>> stalledRequests = new HashMap<>(); + /** + * Runs before or after an operation. Can stall the operation or error it. + */ + public interface Hook { + CompletableFuture runHook(BookieSocketAddress bookie, long ledgerId, long entryId); + } + + private Hook preReadHook = (bookie, ledgerId, entryId) -> FutureUtils.value(null); + private Hook postReadHook = (bookie, ledgerId, entryId) -> FutureUtils.value(null); + private Hook preWriteHook = (bookie, ledgerId, entryId) -> FutureUtils.value(null); + private Hook postWriteHook = (bookie, ledgerId, entryId) -> FutureUtils.value(null); public MockBookieClient(OrderedExecutor executor) { this.executor = executor; } - public void stallBookie(BookieSocketAddress bookie) { - synchronized (this) { - stalledBookies.put(bookie, true); - } + public void setPreReadHook(Hook hook) { + this.preReadHook = hook; } - public void releaseStalledBookie(BookieSocketAddress bookie, int rc) { - synchronized (this) { - stalledBookies.remove(bookie); - stalledRequests.remove(bookie).forEach((r) -> r.accept(rc)); - } + public void setPostReadHook(Hook hook) { + this.postReadHook = hook; + } + + public void setPreWriteHook(Hook hook) { + this.preWriteHook = hook; + } + + public void setPostWriteHook(Hook hook) { + this.postWriteHook = hook; } public void errorBookies(BookieSocketAddress... bookies) { @@ -95,6 +108,15 @@ public void removeErrors(BookieSocketAddress... bookies) { } } + public void seedEntries(BookieSocketAddress bookie, long ledgerId, long entryId, long lac) throws Exception { + DigestManager digestManager = DigestManager.instantiate(ledgerId, new byte[0], DigestType.CRC32C); + ByteBuf entry = ByteBufList.coalesce(digestManager.computeDigestAndPackageForSending( + entryId, lac, 0, Unpooled.buffer(10))); + + LedgerData ledger = getBookieData(bookie).computeIfAbsent(ledgerId, LedgerData::new); + ledger.addEntry(entryId, entry); + } + @Override public List getFaultyBookies() { return Collections.emptyList(); @@ -134,41 +156,29 @@ public void writeLac(BookieSocketAddress addr, long ledgerId, byte[] masterKey, public void addEntry(BookieSocketAddress addr, long ledgerId, byte[] masterKey, long entryId, ByteBufList toSend, WriteCallback cb, Object ctx, int options, boolean allowFastFail, EnumSet writeFlags) { - SafeRunnable write = safeRun(() -> { - LOG.info("[{};L{}] write entry {}", addr, ledgerId, entryId); - if (errorBookies.contains(addr)) { - LOG.warn("[{};L{}] erroring write {}", addr, ledgerId, entryId); - cb.writeComplete(BKException.Code.WriteException, ledgerId, entryId, addr, ctx); - return; - } - LedgerData ledger = getBookieData(addr).computeIfAbsent(ledgerId, LedgerData::new); - ledger.addEntry(entryId, copyData(toSend)); - cb.writeComplete(BKException.Code.OK, ledgerId, entryId, addr, ctx); - toSend.release(); - }); - toSend.retain(); - synchronized (this) { - if (stalledBookies.getOrDefault(addr, false)) { - LOG.info("[{};{};{}] Stalling write {}", addr, ledgerId, System.identityHashCode(write), entryId); - stalledRequests.computeIfAbsent(addr, (key) -> new ArrayList<>()) - .add((rc) -> { - LOG.info("[{};{};{}] Unstalled write {}", - addr, ledgerId, System.identityHashCode(write), entryId); - if (rc == BKException.Code.OK) { - executor.executeOrdered(ledgerId, write); - } else { - executor.executeOrdered( - ledgerId, safeRun(() -> { - cb.writeComplete(rc, ledgerId, entryId, addr, ctx); - toSend.release(); - })); - } - }); - } else { - executor.executeOrdered(ledgerId, write); - } - } + preWriteHook.runHook(addr, ledgerId, entryId) + .thenComposeAsync( + (ignore) -> { + LOG.info("[{};L{}] write entry {}", addr, ledgerId, entryId); + if (errorBookies.contains(addr)) { + LOG.warn("[{};L{}] erroring write {}", addr, ledgerId, entryId); + return FutureUtils.exception(new BKException.BKWriteException()); + } + LedgerData ledger = getBookieData(addr).computeIfAbsent(ledgerId, LedgerData::new); + ledger.addEntry(entryId, copyData(toSend)); + toSend.release(); + return FutureUtils.value(null); + }, executor.chooseThread(ledgerId)) + .thenCompose((res) -> postWriteHook.runHook(addr, ledgerId, entryId)) + .whenCompleteAsync((res, ex) -> { + if (ex != null) { + cb.writeComplete(BKException.getExceptionCode(ex, BKException.Code.WriteException), + ledgerId, entryId, addr, ctx); + } else { + cb.writeComplete(BKException.Code.OK, ledgerId, entryId, addr, ctx); + } + }, executor.chooseThread(ledgerId)); } @Override @@ -184,34 +194,38 @@ public void readLac(BookieSocketAddress addr, long ledgerId, ReadLacCallback cb, public void readEntry(BookieSocketAddress addr, long ledgerId, long entryId, ReadEntryCallback cb, Object ctx, int flags, byte[] masterKey, boolean allowFastFail) { - executor.executeOrdered(ledgerId, - safeRun(() -> { - LOG.info("[{};L{}] read entry {}", addr, ledgerId, entryId); - if (errorBookies.contains(addr)) { - LOG.warn("[{};L{}] erroring read {}", addr, ledgerId, entryId); - cb.readEntryComplete(BKException.Code.ReadException, ledgerId, entryId, null, ctx); - return; - } - - LedgerData ledger = getBookieData(addr).get(ledgerId); - if (ledger == null) { - LOG.warn("[{};L{}] ledger not found", addr, ledgerId); - cb.readEntryComplete(BKException.Code.NoSuchLedgerExistsException, - ledgerId, entryId, null, ctx); - return; - } - - ByteBuf entry = ledger.getEntry(entryId); - if (entry == null) { - LOG.warn("[{};L{}] entry({}) not found", addr, ledgerId, entryId); - cb.readEntryComplete(BKException.Code.NoSuchEntryException, - ledgerId, entryId, null, ctx); - return; - } - + preReadHook.runHook(addr, ledgerId, entryId) + .thenComposeAsync((res) -> { + LOG.info("[{};L{}] read entry {}", addr, ledgerId, entryId); + if (errorBookies.contains(addr)) { + LOG.warn("[{};L{}] erroring read {}", addr, ledgerId, entryId); + return FutureUtils.exception(new BKException.BKReadException()); + } + + LedgerData ledger = getBookieData(addr).get(ledgerId); + if (ledger == null) { + LOG.warn("[{};L{}] ledger not found", addr, ledgerId); + return FutureUtils.exception(new BKException.BKNoSuchLedgerExistsException()); + } + + ByteBuf entry = ledger.getEntry(entryId); + if (entry == null) { + LOG.warn("[{};L{}] entry({}) not found", addr, ledgerId, entryId); + return FutureUtils.exception(new BKException.BKNoSuchEntryException()); + } + + return FutureUtils.value(entry); + }, executor.chooseThread(ledgerId)) + .thenCompose((buf) -> postReadHook.runHook(addr, ledgerId, entryId).thenApply((res) -> buf)) + .whenCompleteAsync((res, ex) -> { + if (ex != null) { + cb.readEntryComplete(BKException.getExceptionCode(ex, BKException.Code.ReadException), + ledgerId, entryId, null, ctx); + } else { cb.readEntryComplete(BKException.Code.OK, - ledgerId, entryId, entry.slice(), ctx); - })); + ledgerId, entryId, res.slice(), ctx); + } + }, executor.chooseThread(ledgerId)); } @Override diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java index 8978ff3f1e7..3152a74c563 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java +++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java @@ -17,6 +17,8 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkArgument; + import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; @@ -31,7 +33,9 @@ public static LedgerManager getLedgerManager(BookKeeper bk) { public static void forceRecoverLedger(LedgerHandle lh, BookkeeperInternalCallbacks.GenericCallback cb) { - lh.recover(cb, null, true); + checkArgument(lh instanceof ReadOnlyLedgerHandle, + "Recovery can only run on ReadOnlyLedgerHandle"); + ((ReadOnlyLedgerHandle) lh).recover(cb, null, true); } public static LedgerMetadata getLedgerMetadata(LedgerHandle lh) { From 3cc711311269de4238e01a3d28685fb3c60306a5 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 4 Sep 2018 13:40:05 -0700 Subject: [PATCH 0011/1642] [DOCKER] update package to 4.7.2 in dockerfile Descriptions of the changes in this PR: ### Motivation We need to generate 4.7.2 docker image. Author: Sijie Guo Reviewers: Enrico Olivelli , Matteo Merli This closes #1644 from sijie/update_docker_image_47 --- docker/Dockerfile | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/Dockerfile b/docker/Dockerfile index 8c7f255072f..6257b8b83d6 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -20,7 +20,7 @@ FROM centos:7 MAINTAINER Apache BookKeeper -ARG BK_VERSION=4.7.1 +ARG BK_VERSION=4.7.2 ARG DISTRO_NAME=bookkeeper-server-${BK_VERSION}-bin ARG GPG_KEY=FD74402C @@ -39,13 +39,13 @@ RUN set -x \ && cd /opt \ && wget -q "https://archive.apache.org/dist/bookkeeper/bookkeeper-${BK_VERSION}/${DISTRO_NAME}.tar.gz" \ && wget -q "https://archive.apache.org/dist/bookkeeper/bookkeeper-${BK_VERSION}/${DISTRO_NAME}.tar.gz.asc" \ - && wget -q "https://archive.apache.org/dist/bookkeeper/bookkeeper-${BK_VERSION}/${DISTRO_NAME}.tar.gz.sha1" \ - && sha1sum -c ${DISTRO_NAME}.tar.gz.sha1 \ + && wget -q "https://archive.apache.org/dist/bookkeeper/bookkeeper-${BK_VERSION}/${DISTRO_NAME}.tar.gz.sha512" \ + && sha512sum -c ${DISTRO_NAME}.tar.gz.sha512 \ && gpg --keyserver ha.pool.sks-keyservers.net --recv-key "$GPG_KEY" \ && gpg --batch --verify "$DISTRO_NAME.tar.gz.asc" "$DISTRO_NAME.tar.gz" \ && tar -xzf "$DISTRO_NAME.tar.gz" \ && mv bookkeeper-server-${BK_VERSION}/ /opt/bookkeeper/ \ - && rm -rf "$DISTRO_NAME.tar.gz" "$DISTRO_NAME.tar.gz.asc" "$DISTRO_NAME.tar.gz.sha1" \ + && rm -rf "$DISTRO_NAME.tar.gz" "$DISTRO_NAME.tar.gz.asc" "$DISTRO_NAME.tar.gz.sha512" \ && yum remove -y wget \ && yum clean all From f3aeec2f22690084de79773de799f08f2e039acc Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 4 Sep 2018 14:54:05 -0700 Subject: [PATCH 0012/1642] [WEBSITE] 4.7.2 only has sha512 file not sha1 file Descriptions of the changes in this PR: *Motivation* Since 4.7.2 bookkeeper releases only provide sha512 file, not sha1 file. Author: Sijie Guo Reviewers: Matteo Merli This closes #1647 from sijie/update_website_472 --- site/releases.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/site/releases.md b/site/releases.md index d8233c6afd1..7d68af921f7 100644 --- a/site/releases.md +++ b/site/releases.md @@ -24,8 +24,8 @@ If you want to download older, archived releases, they are available in the [Apa Release | Link | Crypto files :-------|:-----|:------------ -Source | [bookkeeper-{{ site.latest_release }}-src.tar.gz]({{ latest_source_url }}) | [asc]({{ latest_source_dist_url }}.asc), [sha1]({{ latest_source_dist_url }}.sha1) -Binary | [bookkeeper-server-{{ site.latest_release }}-bin.tar.gz]({{ latest_bin_url }}) | [asc]({{ latest_bin_dist_url }}.asc), [sha1]({{ latest_bin_dist_url }}.sha1) +Source | [bookkeeper-{{ site.latest_release }}-src.tar.gz]({{ latest_source_url }}) | [asc]({{ latest_source_dist_url }}.asc), [sha512]({{ latest_source_dist_url }}.sha512) +Binary | [bookkeeper-server-{{ site.latest_release }}-bin.tar.gz]({{ latest_bin_url }}) | [asc]({{ latest_bin_dist_url }}.asc), [sha512]({{ latest_bin_dist_url }}.sha512) ## Latest stable release (version {{ site.stable_release }}) From 54232a594495997a5e829117da414d329d4f2be9 Mon Sep 17 00:00:00 2001 From: Qi Wang Date: Wed, 5 Sep 2018 12:07:29 -0700 Subject: [PATCH 0013/1642] ISSUE #1630: TestHttpService#testWhoIsAuditorService is flaky Descriptions of the changes in this PR: ### Motivation Fixes #1630. The problem is auditor elector was started async, the test itself doesn't wait until election completed. so the test can be running before auditor runs elector. ### Changes The fix is simple, just wait until auditor completes election. Master Issue: #1630 Author: Qi Wang Reviewers: Ivan Kelly , Enrico Olivelli , Sijie Guo This closes #1649 from codingwangqi/issue_1630, closes #1630 --- .../apache/bookkeeper/replication/AuditorElector.java | 9 +++++---- .../apache/bookkeeper/server/http/TestHttpService.java | 8 +++++--- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java index 8cdf3b36570..1ea179c7b2e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicBoolean; @@ -206,9 +207,9 @@ public void process(WatchedEvent event) { } } - public void start() { + public Future start() { running.set(true); - submitElectionTask(); + return submitElectionTask(); } /** @@ -242,7 +243,7 @@ public void run() { * Auditor. */ @VisibleForTesting - void submitElectionTask() { + Future submitElectionTask() { Runnable r = new Runnable() { public void run() { @@ -302,7 +303,7 @@ public void run() { } } }; - executor.submit(r); + return executor.submit(r); } @VisibleForTesting diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index bc57424915a..d14b8a4f9e2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -31,6 +31,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.Future; import lombok.Cleanup; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; @@ -586,7 +587,7 @@ public void testRecoveryBookieService() throws Exception { ZooKeeper auditorZookeeper; AuditorElector auditorElector; - private void startAuditorElector() throws Exception { + private Future startAuditorElector() throws Exception { auditorZookeeper = ZooKeeperClient.newBuilder() .connectString(zkUtil.getZooKeeperConnectString()) .sessionTimeoutMs(10000) @@ -597,7 +598,7 @@ private void startAuditorElector() throws Exception { conf.setMetadataServiceUri("zk://" + zkUtil.getZooKeeperConnectString() + "/ledgers"); auditorElector = new AuditorElector(addr, conf, auditorZookeeper); - auditorElector.start(); + return auditorElector.start(); } private void stopAuditorElector() throws Exception { @@ -628,7 +629,8 @@ public void testTriggerAuditService() throws Exception { @Test public void testWhoIsAuditorService() throws Exception { - startAuditorElector(); + // start the auditor elector and wait until auditor finishes election. + startAuditorElector().get(); HttpEndpointService whoIsAuditorService = bkHttpServiceProvider .provideHttpEndpointService(HttpServer.ApiType.WHO_IS_AUDITOR); From 18b44900f8d16aad7d0c36093f8f34f23668e747 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 5 Sep 2018 12:11:58 -0700 Subject: [PATCH 0014/1642] [WEBSITE] fix download link to allow user choosing mirror site Address the download link according to https://www.apache.org/dev/release-distribution#download-links Author: Sijie Guo Reviewers: Enrico Olivelli , Matteo Merli This closes #1650 from sijie/fix_download_links --- site/releases.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/releases.md b/site/releases.md index 7d68af921f7..fed90ea3a9b 100644 --- a/site/releases.md +++ b/site/releases.md @@ -3,7 +3,7 @@ title: Apache BookKeeper™ Releases layout: community --- -{% capture mirror_url %}https://www.apache.org/dyn/mirrors/mirrors.cgi?action=download&filename=bookkeeper{% endcapture %} +{% capture mirror_url %}https://www.apache.org/dyn/closer.lua/bookkeeper{% endcapture %} {% capture latest_source_url %}{{ mirror_url }}/bookkeeper-{{ site.latest_release }}/bookkeeper-{{ site.latest_release }}-src.tar.gz{% endcapture %} {% capture latest_bin_url %}{{ mirror_url }}/bookkeeper-{{ site.latest_release }}/bookkeeper-server-{{ site.latest_release }}-bin.tar.gz{% endcapture %} {% capture dist_url %}https://www.apache.org/dist/bookkeeper{% endcapture %} From a1714d60504fa950ca3a2b2a0338b3c87ef5187d Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 6 Sep 2018 09:34:23 +0200 Subject: [PATCH 0015/1642] Make LedgerFragmentReplicator use MetadataUpdateLoop Master Issue: #281 Author: Ivan Kelly Reviewers: Enrico Olivelli , Sijie Guo This closes #1645 from ivankelly/ledger-fragment-immutable-metadata2 --- .../bookkeeper/client/BookKeeperAdmin.java | 4 +- .../client/LedgerFragmentReplicator.java | 145 +++++------------- 2 files changed, 42 insertions(+), 107 deletions(-) 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 59c203ba0fa..e5b59143548 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 @@ -893,7 +893,7 @@ public void processResult(int rc, String path, Object ctx) { try { LedgerFragmentReplicator.SingleFragmentCallback cb = new LedgerFragmentReplicator.SingleFragmentCallback(ledgerFragmentsMcb, lh, - bkc.getMainWorkerPool(), + bkc.getLedgerManager(), startEntryId, getReplacementBookiesMap(ensemble, targetBookieAddresses)); LedgerFragment ledgerFragment = new LedgerFragment(lh, startEntryId, endEntryId, targetBookieAddresses.keySet()); @@ -1047,7 +1047,7 @@ private void replicateLedgerFragment(LedgerHandle lh, SingleFragmentCallback cb = new SingleFragmentCallback( resultCallBack, lh, - bkc.getMainWorkerPool(), + bkc.getLedgerManager(), ledgerFragment.getFirstEntryId(), getReplacementBookiesMap(ledgerFragment, targetBookieAddresses)); 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 eb11b30c5ac..dc7302c46f1 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 @@ -23,7 +23,6 @@ import io.netty.buffer.Unpooled; -import java.util.ArrayList; import java.util.Enumeration; import java.util.HashSet; import java.util.Iterator; @@ -33,13 +32,13 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; import org.apache.bookkeeper.client.api.WriteFlag; -import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.MultiCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.replication.ReplicationStats; @@ -48,7 +47,6 @@ import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.ByteBufList; -import org.apache.bookkeeper.util.OrderedGenericCallback; import org.apache.zookeeper.AsyncCallback; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -341,16 +339,16 @@ public void readComplete(int rc, LedgerHandle lh, static class SingleFragmentCallback implements AsyncCallback.VoidCallback { final AsyncCallback.VoidCallback ledgerFragmentsMcb; final LedgerHandle lh; - final OrderedExecutor mainWorkerPool; + final LedgerManager ledgerManager; final long fragmentStartId; final Map oldBookie2NewBookie; SingleFragmentCallback(AsyncCallback.VoidCallback ledgerFragmentsMcb, - LedgerHandle lh, OrderedExecutor mainWorkerPool, long fragmentStartId, - Map oldBookie2NewBookie) { + LedgerHandle lh, LedgerManager ledgerManager, long fragmentStartId, + Map oldBookie2NewBookie) { this.ledgerFragmentsMcb = ledgerFragmentsMcb; this.lh = lh; - this.mainWorkerPool = mainWorkerPool; + this.ledgerManager = ledgerManager; this.fragmentStartId = fragmentStartId; this.oldBookie2NewBookie = oldBookie2NewBookie; } @@ -363,7 +361,7 @@ public void processResult(int rc, String path, Object ctx) { ledgerFragmentsMcb.processResult(rc, null, null); return; } - updateEnsembleInfo(ledgerFragmentsMcb, fragmentStartId, lh, mainWorkerPool, oldBookie2NewBookie); + updateEnsembleInfo(ledgerManager, ledgerFragmentsMcb, fragmentStartId, lh, oldBookie2NewBookie); } } @@ -371,104 +369,41 @@ public void processResult(int rc, String path, Object ctx) { * Updates the ensemble with newBookie and notify the ensembleUpdatedCb. */ private static void updateEnsembleInfo( - AsyncCallback.VoidCallback ensembleUpdatedCb, long fragmentStartId, - LedgerHandle lh, OrderedExecutor mainWorkerPool, - Map oldBookie2NewBookie) { - /* - * Update the ledger metadata's ensemble info to point to the new - * bookie. - */ - List ensemble = lh.getLedgerMetadata().getEnsembles().get(fragmentStartId); - List newEnsemble = new ArrayList<>(ensemble); - for (Map.Entry entry : oldBookie2NewBookie.entrySet()) { - int deadBookieIndex = newEnsemble.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) { - newEnsemble.set(deadBookieIndex, entry.getValue()); - } else { - LOG.info("Bookie {} doesn't exist in ensemble {} anymore.", entry.getKey(), ensemble); - } - } - lh.getLedgerMetadata().updateEnsemble(fragmentStartId, newEnsemble); - lh.writeLedgerConfig(new UpdateEnsembleCb(ensembleUpdatedCb, - fragmentStartId, lh, mainWorkerPool, oldBookie2NewBookie)); - } + LedgerManager ledgerManager, AsyncCallback.VoidCallback ensembleUpdatedCb, long fragmentStartId, + LedgerHandle lh, Map oldBookie2NewBookie) { - /** - * Update the ensemble data with newBookie. re-reads the metadata on - * MetadataVersionException and update ensemble again. On successfull - * updation, it will also notify to super call back - */ - private static class UpdateEnsembleCb implements GenericCallback { - final AsyncCallback.VoidCallback ensembleUpdatedCb; - final LedgerHandle lh; - final OrderedExecutor mainWorkerPool; - final long fragmentStartId; - final Map oldBookie2NewBookie; + MetadataUpdateLoop updateLoop = new MetadataUpdateLoop( + ledgerManager, + lh.getId(), + lh::getLedgerMetadata, + (metadata) -> { + // returns true if any of old bookies exist in ensemble + List ensemble = metadata.getEnsembles().get(fragmentStartId); + return oldBookie2NewBookie.keySet().stream().anyMatch(ensemble::contains); + }, + (currentMetadata) -> { + // replace all old bookies with new bookies in ensemble + List newEnsemble = currentMetadata.getEnsembles().get(fragmentStartId) + .stream().map((bookie) -> oldBookie2NewBookie.getOrDefault(bookie, bookie)) + .collect(Collectors.toList()); + return LedgerMetadataBuilder.from(currentMetadata) + .replaceEnsembleEntry(fragmentStartId, newEnsemble).build(); + }, + lh::setLedgerMetadata); - public UpdateEnsembleCb(AsyncCallback.VoidCallback ledgerFragmentsMcb, - long fragmentStartId, LedgerHandle lh, - OrderedExecutor mainWorkerPool, - Map oldBookie2NewBookie) { - this.ensembleUpdatedCb = ledgerFragmentsMcb; - this.lh = lh; - this.mainWorkerPool = mainWorkerPool; - this.fragmentStartId = fragmentStartId; - this.oldBookie2NewBookie = oldBookie2NewBookie; - } + updateLoop.run().whenComplete((result, ex) -> { + if (ex == null) { + LOG.info("Updated ZK for ledgerId: ({}:{}) to point ledger fragments" + + " from old bookies to new bookies: {}", oldBookie2NewBookie); - @Override - public void operationComplete(int rc, LedgerMetadata writtenMetadata) { - if (rc == BKException.Code.MetadataVersionException) { - LOG.warn("Two fragments attempted update at once; ledger id: " - + lh.getId() + " startid: " + fragmentStartId); - // 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 OrderedGenericCallback(mainWorkerPool, lh.getId()) { - @Override - public void safeOperationComplete(int rc, - LedgerMetadata newMeta) { - if (rc != BKException.Code.OK) { - LOG - .error("Error reading updated ledger metadata for ledger " - + lh.getId()); - ensembleUpdatedCb.processResult(rc, null, - null); - } else { - while (true) { - // temporary change, metadata really shouldn't be updated - // until the new metadata has been written successfully - LedgerMetadata currentMetadata = lh.getLedgerMetadata(); - if (lh.setLedgerMetadata(currentMetadata, newMeta)) { - break; - } - } - updateEnsembleInfo(ensembleUpdatedCb, - fragmentStartId, lh, mainWorkerPool, oldBookie2NewBookie); - } - } - @Override - public String toString() { - return String.format("ReReadMetadataForUpdateEnsemble(%d)", lh.getId()); - } - }); - return; - } else if (rc != BKException.Code.OK) { - LOG.error("Error updating ledger config metadata for ledgerId {} : {}", - lh.getId(), BKException.codeLogger(rc)); - } else { - LOG.info("Updated ZK for ledgerId: (" + lh.getId() + " : " - + fragmentStartId - + ") to point ledger fragments from old bookies to new bookies: " - + oldBookie2NewBookie); - } - /* - * Pass the return code result up the chain with the parent - * callback. - */ - ensembleUpdatedCb.processResult(rc, null, null); - } + ensembleUpdatedCb.processResult(BKException.Code.OK, null, null); + } else { + LOG.error("Error updating ledger config metadata for ledgerId {}", lh.getId(), ex); + + ensembleUpdatedCb.processResult( + BKException.getExceptionCode(ex, BKException.Code.UnexpectedConditionException), + null, null); + } + }); } } From 6118133f0b57f33898d85366993739530606e43f Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 6 Sep 2018 10:10:38 +0200 Subject: [PATCH 0016/1642] ISSUE #1658: License file: typo "Google Protocal Buffers" instead of "Google Protocol Buffers" Fix a typo in LICENSE files for binary distributions. Master issue #1658 Author: Enrico Olivelli Reviewers: Sijie Guo This closes #1659 from eolivelli/fix/protocal-typo, closes #1658 --- bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt | 2 +- bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index 5303d528625..2997d7bdd6a 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -492,7 +492,7 @@ a "3-clause BSD" license. For details, see deps/jsr-305/LICENSE. Bundled as lib/com.google.code.findbugs-jsr305-3.0.2.jar Source available at https://storage.googleapis.com/google-code-archive-source/v2/code.google.com/jsr-305/source-archive.zip ------------------------------------------------------------------------------------ -This product bundles Google Protocal Buffers, which is available under a "3-clause BSD" +This product bundles Google Protocol Buffers, which is available under a "3-clause BSD" license. Bundled as diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index eac571468ea..7ed32ad92fb 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -404,7 +404,7 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. ------------------------------------------------------------------------------------ -This product bundles Google Protocal Buffers, which is available under a "3-clause BSD" +This product bundles Google Protocol Buffers, which is available under a "3-clause BSD" license. Bundled as From 5a8fe5fe8b44236de07c4f5d67fcea8925d89032 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 6 Sep 2018 11:16:09 -0700 Subject: [PATCH 0017/1642] [WEBSITE] Issue #1652: fix anchor links at download page Descriptions of the changes in this PR: As reported in #1652, the anchor links at download page are broken. Author: Sijie Guo Reviewers: Ivan Kelly , Enrico Olivelli This closes #1656 from sijie/fix_download_links_2, closes #1652 --- site/releases.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/site/releases.md b/site/releases.md index fed90ea3a9b..e9238de7269 100644 --- a/site/releases.md +++ b/site/releases.md @@ -14,13 +14,14 @@ layout: community {% capture stable_source_url %}{{ archive_url }}/bookkeeper-{{ site.stable_release }}/bookkeeper-{{ site.stable_release }}-src.tar.gz{% endcapture %} {% capture stable_bin_url %}{{ archive_url }}/bookkeeper-{{ site.stable_release }}/bookkeeper-server-{{ site.stable_release }}-bin.tar.gz{% endcapture %} -Version **{{ site.latest_release }}** is the [latest release](#latest-releases-version-{{ site.latest_release | remove: "." }}) of BookKeeper. The current [stable version](#latest-stable-releases-version-{{ site.stable_release | remove: "." }}) is **{{ site.stable_release }}**. +Version **{{ site.latest_release }}** is the [latest release](#latest-release) of BookKeeper. The current [stable version](#latest-stable-release) is **{{ site.stable_release }}**. > You can verify your download by following these [procedures](http://www.apache.org/info/verification.html) and using these [KEYS](https://www.apache.org/dist/bookkeeper/KEYS). If you want to download older, archived releases, they are available in the [Apache archive](http://archive.apache.org/dist/bookkeeper/). ## Latest release (version {{ site.latest_release }}) + Release | Link | Crypto files :-------|:-----|:------------ @@ -28,6 +29,7 @@ Source | [bookkeeper-{{ site.latest_release }}-src.tar.gz]({{ latest_source_url Binary | [bookkeeper-server-{{ site.latest_release }}-bin.tar.gz]({{ latest_bin_url }}) | [asc]({{ latest_bin_dist_url }}.asc), [sha512]({{ latest_bin_dist_url }}.sha512) ## Latest stable release (version {{ site.stable_release }}) + Release | Link | Crypto files :-------|:-----|:------------ From 6615f4fe7eb1264fd9f2095e0230cf78b2524ce0 Mon Sep 17 00:00:00 2001 From: Qi Wang Date: Thu, 6 Sep 2018 14:14:13 -0700 Subject: [PATCH 0018/1642] Open ledger returns no ledger exception if ledger id is negative Descriptions of the changes in this PR: ### Motivation if I open a ledger with negative value, bookkeeper client responds 'NoSuchLedgerException'. But I think it should return `IllegalOpException`, because it is an illegal exception. Author: Qi Wang Reviewers: Enrico Olivelli , Sijie Guo This closes #1638 from codingwangqi/fix_return_code --- .../java/org/apache/bookkeeper/client/LedgerOpenOp.java | 6 ++++-- .../org/apache/bookkeeper/client/impl/OpenBuilderBase.java | 7 ++++--- .../java/org/apache/bookkeeper/client/MockBookKeeper.java | 6 ++++-- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java index da825319794..7f169ab1ba8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java @@ -33,6 +33,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncOpenCallback; +import org.apache.bookkeeper.client.api.BKException.Code; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.client.impl.OpenBuilderBase; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; @@ -245,8 +246,9 @@ public CompletableFuture execute() { } private void open(OpenCallback cb) { - if (!validate()) { - cb.openComplete(BKException.Code.NoSuchLedgerExistsException, null, null); + final int validateRc = validate(); + if (Code.OK != validateRc) { + cb.openComplete(validateRc, null, null); return; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java index b22effc3d57..c2c4c35e1fa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java @@ -21,6 +21,7 @@ import java.util.Arrays; import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.api.BKException.Code; import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.OpenBuilder; @@ -62,11 +63,11 @@ public OpenBuilder withDigestType(DigestType digestType) { return this; } - protected boolean validate() { + protected int validate() { if (ledgerId < 0) { LOG.error("invalid ledgerId {} < 0", ledgerId); - return false; + return Code.NoSuchLedgerExistsException; } - return true; + return Code.OK; } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java index 3b63cfd3708..c65e0702810 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java @@ -35,6 +35,7 @@ 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.api.BKException.Code; import org.apache.bookkeeper.client.api.OpenBuilder; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.client.impl.OpenBuilderBase; @@ -221,8 +222,9 @@ public OpenBuilder newOpenLedgerOp() { public CompletableFuture execute() { CompletableFuture promise = new CompletableFuture(); - if (!validate()) { - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + final int validateRc = validate(); + if (Code.OK != validateRc) { + promise.completeExceptionally(BKException.create(validateRc)); return promise; } else if (getProgrammedFailStatus()) { if (failReturnCode != BkTimeoutOperation) { From 6986ad1f3368b19365ceea59b967c819529f326b Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 6 Sep 2018 21:40:10 -0700 Subject: [PATCH 0019/1642] Utility to rebuild interleaved storage index files We came across a case where the a ledger had been deleted from zookeeper accidently. It was possible to recover the ledger metadata from the zookeeper journal and old snapshots, but the bookies had deleted the indices by this time. However, even if the index is deleted, the data still exists in the entrylog. This utility scans the entrylog to rebuild the index, thereby making the ledger available again. Author: Ivan Kelly Reviewers: Enrico Olivelli , Sijie Guo This closes #1642 from ivankelly/regen-from-entrylogger --- .../org/apache/bookkeeper/bookie/Bookie.java | 35 ++- .../apache/bookkeeper/bookie/BookieShell.java | 69 +++++- .../InterleavedStorageRegenerateIndexOp.java | 230 ++++++++++++++++++ .../bookkeeper/client/LedgerHandle.java | 14 +- .../proto/checksum/DigestManager.java | 5 + .../proto/checksum/MacDigestManager.java | 9 + conf/log4j.shell.properties | 1 + .../utils/BookKeeperClusterUtils.java | 19 +- .../integration/BookieShellTestBase.java | 4 +- .../integration/TestBookieShellCluster.java | 99 ++++++++ 10 files changed, 450 insertions(+), 35 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.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 e9e4d15023e..3795b37b91b 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 @@ -629,17 +629,10 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger) for (File journalDirectory : conf.getJournalDirs()) { this.journalDirectories.add(getCurrentDirectory(journalDirectory)); } - DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()); - this.ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), diskChecker, - statsLogger.scope(LD_LEDGER_SCOPE)); - - File[] idxDirs = conf.getIndexDirs(); - if (null == idxDirs) { - this.indexDirsManager = this.ledgerDirsManager; - } else { - this.indexDirsManager = new LedgerDirsManager(conf, idxDirs, diskChecker, - statsLogger.scope(LD_INDEX_SCOPE)); - } + DiskChecker diskChecker = createDiskChecker(conf); + this.ledgerDirsManager = createLedgerDirsManager(conf, diskChecker, statsLogger.scope(LD_LEDGER_SCOPE)); + this.indexDirsManager = createIndexDirsManager(conf, diskChecker, statsLogger.scope(LD_INDEX_SCOPE), + this.ledgerDirsManager); // instantiate zookeeper client to initialize ledger manager this.metadataDriver = instantiateMetadataDriver(conf); @@ -675,7 +668,7 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger) } } - if (null == idxDirs) { + if (ledgerDirsManager == indexDirsManager) { this.idxMonitor = this.ledgerMonitor; } else { this.idxMonitor = new LedgerDirsMonitor(conf, diskChecker, indexDirsManager); @@ -1546,4 +1539,22 @@ public int getExitCode() { return exitCode; } + static DiskChecker createDiskChecker(ServerConfiguration conf) { + return new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()); + } + + static LedgerDirsManager createLedgerDirsManager(ServerConfiguration conf, DiskChecker diskChecker, + StatsLogger statsLogger) { + return new LedgerDirsManager(conf, conf.getLedgerDirs(), diskChecker, statsLogger); + } + + static LedgerDirsManager createIndexDirsManager(ServerConfiguration conf, DiskChecker diskChecker, + StatsLogger statsLogger, LedgerDirsManager fallback) { + File[] idxDirs = conf.getIndexDirs(); + if (null == idxDirs) { + return fallback; + } else { + return new LedgerDirsManager(conf, idxDirs, diskChecker, statsLogger); + } + } } 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 4434cfc3aef..8eebce58be8 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 @@ -18,7 +18,7 @@ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithMetadataBookieDriver; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; @@ -51,6 +51,7 @@ import java.text.DecimalFormat; import java.util.ArrayList; import java.util.Arrays; +import java.util.Base64; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -72,6 +73,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; +import java.util.stream.Collectors; import java.util.stream.LongStream; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; @@ -187,6 +189,7 @@ public class BookieShell implements Tool { static final String CMD_CONVERT_TO_DB_STORAGE = "convert-to-db-storage"; static final String CMD_CONVERT_TO_INTERLEAVED_STORAGE = "convert-to-interleaved-storage"; static final String CMD_REBUILD_DB_LEDGER_LOCATIONS_INDEX = "rebuild-db-ledger-locations-index"; + static final String CMD_REGENERATE_INTERLEAVED_STORAGE_INDEX_FILE = "regenerate-interleaved-storage-index-file"; static final String CMD_HELP = "help"; final ServerConfiguration bkConf = new ServerConfiguration(); @@ -2816,6 +2819,69 @@ int runCmd(CommandLine cmdLine) throws Exception { } } + /** + * Regenerate an index file for interleaved storage. + */ + class RegenerateInterleavedStorageIndexFile extends MyCommand { + Options opts = new Options(); + + public RegenerateInterleavedStorageIndexFile() { + super(CMD_REGENERATE_INTERLEAVED_STORAGE_INDEX_FILE); + Option ledgerOption = new Option("l", "ledgerIds", true, + "Ledger(s) whose index needs to be regenerated." + + " Multiple can be specified, comma separated."); + ledgerOption.setRequired(true); + ledgerOption.setValueSeparator(','); + ledgerOption.setArgs(Option.UNLIMITED_VALUES); + + opts.addOption(ledgerOption); + opts.addOption("dryRun", false, + "Process the entryLogger, but don't write anything."); + opts.addOption("password", true, + "The bookie stores the password in the index file, so we need it to regenerate. " + + "This must match the value in the ledger metadata."); + opts.addOption("b64password", true, + "The password in base64 encoding, for cases where the password is not UTF-8."); + } + + @Override + Options getOptions() { + return opts; + } + + @Override + String getDescription() { + return "Regenerate an interleaved storage index file, from available entrylogger files."; + } + + @Override + String getUsage() { + return CMD_REGENERATE_INTERLEAVED_STORAGE_INDEX_FILE; + } + + @Override + int runCmd(CommandLine cmdLine) throws Exception { + byte[] password; + if (cmdLine.hasOption("password")) { + password = cmdLine.getOptionValue("password").getBytes(UTF_8); + } else if (cmdLine.hasOption("b64password")) { + password = Base64.getDecoder().decode(cmdLine.getOptionValue("b64password")); + } else { + LOG.error("The password must be specified to regenerate the index file."); + return 1; + } + Set ledgerIds = Arrays.stream(cmdLine.getOptionValues("ledgerIds")) + .map((id) -> Long.parseLong(id)).collect(Collectors.toSet()); + boolean dryRun = cmdLine.hasOption("dryRun"); + + LOG.info("=== Rebuilding index file for {} ===", ledgerIds); + ServerConfiguration conf = new ServerConfiguration(bkConf); + new InterleavedStorageRegenerateIndexOp(conf, ledgerIds, password).initiate(dryRun); + LOG.info("-- Done rebuilding index file for {} --", ledgerIds); + return 0; + } + } + final Map commands = new HashMap(); { @@ -2849,6 +2915,7 @@ int runCmd(CommandLine cmdLine) throws Exception { commands.put(CMD_CONVERT_TO_DB_STORAGE, new ConvertToDbStorageCmd()); commands.put(CMD_CONVERT_TO_INTERLEAVED_STORAGE, new ConvertToInterleavedStorageCmd()); commands.put(CMD_REBUILD_DB_LEDGER_LOCATIONS_INDEX, new RebuildDbLedgerLocationsIndexCmd()); + commands.put(CMD_REGENERATE_INTERLEAVED_STORAGE_INDEX_FILE, new RegenerateInterleavedStorageIndexFile()); commands.put(CMD_HELP, new HelpCmd()); commands.put(CMD_LOSTBOOKIERECOVERYDELAY, new LostBookieRecoveryDelayCmd()); commands.put(CMD_TRIGGERAUDIT, new TriggerAuditCmd()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java new file mode 100644 index 00000000000..de48eafe5d4 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java @@ -0,0 +1,230 @@ +/** + * + * 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 io.netty.buffer.ByteBuf; + +import java.io.IOException; +import java.security.NoSuchAlgorithmException; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner; +import org.apache.bookkeeper.common.util.Watcher; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.proto.checksum.DigestManager; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.util.DiskChecker; +import org.apache.bookkeeper.util.SnapshotMap; +import org.apache.commons.lang.time.DurationFormatUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Scan all entries in the entry log and rebuild the index file for one ledger. + */ +public class InterleavedStorageRegenerateIndexOp { + private static final Logger LOG = LoggerFactory.getLogger(InterleavedStorageRegenerateIndexOp.class); + + private final ServerConfiguration conf; + private final Set ledgerIds; + private final byte[] masterKey; + + public InterleavedStorageRegenerateIndexOp(ServerConfiguration conf, Set ledgerIds, byte[] password) + throws NoSuchAlgorithmException { + this.conf = conf; + this.ledgerIds = ledgerIds; + this.masterKey = DigestManager.generateMasterKey(password); + } + + static class RecoveryStats { + long firstEntry = Long.MAX_VALUE; + long lastEntry = Long.MIN_VALUE; + long numEntries = 0; + + void registerEntry(long entryId) { + numEntries++; + if (entryId < firstEntry) { + firstEntry = entryId; + } + if (entryId > lastEntry) { + lastEntry = entryId; + } + } + + long getNumEntries() { + return numEntries; + } + + long getFirstEntry() { + return firstEntry; + } + + long getLastEntry() { + return lastEntry; + } + } + + public void initiate(boolean dryRun) throws IOException { + LOG.info("Starting index rebuilding"); + + DiskChecker diskChecker = Bookie.createDiskChecker(conf); + LedgerDirsManager ledgerDirsManager = Bookie.createLedgerDirsManager( + conf, diskChecker, NullStatsLogger.INSTANCE); + LedgerDirsManager indexDirsManager = Bookie.createIndexDirsManager( + conf, diskChecker, NullStatsLogger.INSTANCE, ledgerDirsManager); + EntryLogger entryLogger = new EntryLogger(conf, ledgerDirsManager); + final LedgerCache ledgerCache; + if (dryRun) { + ledgerCache = new DryRunLedgerCache(); + } else { + ledgerCache = new LedgerCacheImpl(conf, new SnapshotMap(), + indexDirsManager, NullStatsLogger.INSTANCE); + } + + Set entryLogs = entryLogger.getEntryLogsSet(); + + int totalEntryLogs = entryLogs.size(); + int completedEntryLogs = 0; + long startTime = System.nanoTime(); + + LOG.info("Scanning {} entry logs", totalEntryLogs); + + Map stats = new HashMap<>(); + for (long entryLogId : entryLogs) { + LOG.info("Scanning {}", entryLogId); + entryLogger.scanEntryLog(entryLogId, new EntryLogScanner() { + @Override + public void process(long ledgerId, long offset, ByteBuf entry) throws IOException { + long entryId = entry.getLong(8); + + stats.computeIfAbsent(ledgerId, (ignore) -> new RecoveryStats()).registerEntry(entryId); + + // Actual location indexed is pointing past the entry size + long location = (entryLogId << 32L) | (offset + 4); + + if (LOG.isDebugEnabled()) { + LOG.debug("Rebuilding {}:{} at location {} / {}", ledgerId, entryId, location >> 32, + location & (Integer.MAX_VALUE - 1)); + } + + if (!ledgerCache.ledgerExists(ledgerId)) { + ledgerCache.setMasterKey(ledgerId, masterKey); + ledgerCache.setFenced(ledgerId); + } + ledgerCache.putEntryOffset(ledgerId, entryId, location); + } + + @Override + public boolean accept(long ledgerId) { + return ledgerIds.contains(ledgerId); + } + }); + + ledgerCache.flushLedger(true); + + ++completedEntryLogs; + LOG.info("Completed scanning of log {}.log -- {} / {}", Long.toHexString(entryLogId), completedEntryLogs, + totalEntryLogs); + } + + LOG.info("Rebuilding indices done"); + for (long ledgerId : ledgerIds) { + RecoveryStats ledgerStats = stats.get(ledgerId); + if (ledgerStats == null || ledgerStats.getNumEntries() == 0) { + LOG.info(" {} - No entries found", ledgerId); + } else { + LOG.info(" {} - Found {} entries, from {} to {}", ledgerId, + ledgerStats.getNumEntries(), ledgerStats.getFirstEntry(), ledgerStats.getLastEntry()); + } + } + LOG.info("Total time: {}", DurationFormatUtils.formatDurationHMS( + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime))); + } + + + static class DryRunLedgerCache implements LedgerCache { + @Override + public void close() { + } + @Override + public boolean setFenced(long ledgerId) throws IOException { + return false; + } + @Override + public boolean isFenced(long ledgerId) throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public void setMasterKey(long ledgerId, byte[] masterKey) throws IOException { + } + @Override + public byte[] readMasterKey(long ledgerId) throws IOException, BookieException { + throw new UnsupportedOperationException(); + } + @Override + public boolean ledgerExists(long ledgerId) throws IOException { + return false; + } + @Override + public void putEntryOffset(long ledger, long entry, long offset) throws IOException { + } + @Override + public long getEntryOffset(long ledger, long entry) throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public void flushLedger(boolean doAll) throws IOException { + } + @Override + public long getLastEntry(long ledgerId) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public Long getLastAddConfirmed(long ledgerId) throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public long updateLastAddConfirmed(long ledgerId, long lac) throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public boolean waitForLastAddConfirmedUpdate(long ledgerId, + long previousLAC, + Watcher watcher) + throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public void deleteLedger(long ledgerId) throws IOException { + } + @Override + public void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException { + } + @Override + public ByteBuf getExplicitLac(long ledgerId) { + throw new UnsupportedOperationException(); + } + } +} 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 1b859a3bb6b..beddaede3b4 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 @@ -34,7 +34,6 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import java.security.GeneralSecurityException; -import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.Arrays; import java.util.EnumSet; @@ -79,7 +78,6 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.State; import org.apache.bookkeeper.proto.checksum.DigestManager; -import org.apache.bookkeeper.proto.checksum.MacDigestManager; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.OpStatsLogger; @@ -150,16 +148,6 @@ public class LedgerHandle implements WriteHandle { final Counter lacUpdateMissesCounter; private final OpStatsLogger clientChannelWriteWaitStats; - // This empty master key is used when an empty password is provided which is the hash of an empty string - private static final byte[] emptyLedgerKey; - static { - try { - emptyLedgerKey = MacDigestManager.genDigest("ledger", new byte[0]); - } catch (NoSuchAlgorithmException e) { - throw new RuntimeException(e); - } - } - public Map getDelayedWriteFailedBookies() { return delayedWriteFailedBookies; } @@ -197,7 +185,7 @@ public Map getDelayedWriteFailedBookies() { // If the password is empty, pass the same random ledger key which is generated by the hash of the empty // password, so that the bookie can avoid processing the keys for each entry - this.ledgerKey = password.length > 0 ? MacDigestManager.genDigest("ledger", password) : emptyLedgerKey; + this.ledgerKey = DigestManager.generateMasterKey(password); distributionSchedule = new RoundRobinDistributionSchedule( metadata.getWriteQuorumSize(), metadata.getAckQuorumSize(), metadata.getEnsembleSize()); this.bookieFailureHistory = CacheBuilder.newBuilder() diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java index 4c174a8df16..19286374540 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java @@ -23,6 +23,7 @@ import io.netty.util.ReferenceCountUtil; import java.security.GeneralSecurityException; +import java.security.NoSuchAlgorithmException; import org.apache.bookkeeper.client.BKException.BKDigestMatchException; import org.apache.bookkeeper.client.LedgerHandle; @@ -86,6 +87,10 @@ public static DigestManager instantiate(long ledgerId, byte[] passwd, DigestType } } + public static byte[] generateMasterKey(byte[] password) throws NoSuchAlgorithmException { + return password.length > 0 ? MacDigestManager.genDigest("ledger", password) : MacDigestManager.EMPTY_LEDGER_KEY; + } + /** * Computes the digest for an entry and put bytes together for sending. * diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java index 8d830a488d5..e71c077eabc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java @@ -46,6 +46,15 @@ public class MacDigestManager extends DigestManager { final byte[] passwd; + static final byte[] EMPTY_LEDGER_KEY; + static { + try { + EMPTY_LEDGER_KEY = MacDigestManager.genDigest("ledger", new byte[0]); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException(e); + } + } + private final ThreadLocal mac = new ThreadLocal() { @Override protected Mac initialValue() { diff --git a/conf/log4j.shell.properties b/conf/log4j.shell.properties index caec94859e8..7f8c00f3357 100644 --- a/conf/log4j.shell.properties +++ b/conf/log4j.shell.properties @@ -48,3 +48,4 @@ log4j.logger.org.apache.zookeeper=ERROR log4j.logger.org.apache.bookkeeper=ERROR log4j.logger.org.apache.bookkeeper.bookie.BookieShell=INFO log4j.logger.org.apache.bookkeeper.client.BookKeeperAdmin=INFO +log4j.logger.org.apache.bookkeeper.bookie.InterleavedStorageRegenerateIndexOp=INFO \ No newline at end of file diff --git a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/integration/utils/BookKeeperClusterUtils.java b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/integration/utils/BookKeeperClusterUtils.java index 8d025964b0b..2d4fc27cc8b 100644 --- a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/integration/utils/BookKeeperClusterUtils.java +++ b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/integration/utils/BookKeeperClusterUtils.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.net.Socket; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -136,13 +137,13 @@ public static void updateBookieConf(DockerClient docker, String containerId, public static void updateAllBookieConf(DockerClient docker, String version, String key, String value) throws Exception { - for (String b : DockerUtils.cubeIdsMatching("bookkeeper")) { + for (String b : allBookies()) { updateBookieConf(docker, b, version, key, value); } } public static boolean runOnAnyBookie(DockerClient docker, String... cmds) throws Exception { - Optional bookie = DockerUtils.cubeIdsMatching("bookkeeper").stream().findAny(); + Optional bookie = allBookies().stream().findAny(); if (bookie.isPresent()) { DockerUtils.runCommand(docker, bookie.get(), cmds); return true; @@ -152,7 +153,7 @@ public static boolean runOnAnyBookie(DockerClient docker, String... cmds) throws } public static String getAnyBookie() throws Exception { - Optional bookie = DockerUtils.cubeIdsMatching("bookkeeper").stream().findAny(); + Optional bookie = allBookies().stream().findAny(); if (bookie.isPresent()) { return bookie.get(); } else { @@ -161,11 +162,15 @@ public static String getAnyBookie() throws Exception { } public static void runOnAllBookies(DockerClient docker, String... cmds) throws Exception { - for (String b : DockerUtils.cubeIdsMatching("bookkeeper")) { + for (String b : allBookies()) { DockerUtils.runCommand(docker, b, cmds); } } + public static Set allBookies() { + return DockerUtils.cubeIdsMatching("bookkeeper"); + } + private static boolean waitBookieState(DockerClient docker, String containerId, int timeout, TimeUnit timeoutUnit, boolean upOrDown) { @@ -219,7 +224,7 @@ private static boolean allTrue(boolean accumulator, boolean result) { public static boolean startAllBookiesWithVersion(DockerClient docker, String version) throws Exception { - return DockerUtils.cubeIdsMatching("bookkeeper").stream() + return allBookies().stream() .map((b) -> startBookieWithVersion(docker, b, version)) .reduce(true, BookKeeperClusterUtils::allTrue); } @@ -235,13 +240,13 @@ public static boolean stopBookie(DockerClient docker, String containerId) { } public static boolean stopAllBookies(DockerClient docker) { - return DockerUtils.cubeIdsMatching("bookkeeper").stream() + return allBookies().stream() .map((b) -> stopBookie(docker, b)) .reduce(true, BookKeeperClusterUtils::allTrue); } public static boolean waitAllBookieUp(DockerClient docker) { - return DockerUtils.cubeIdsMatching("bookkeeper").stream() + return allBookies().stream() .map((b) -> waitBookieUp(docker, b, 10, TimeUnit.SECONDS)) .reduce(true, BookKeeperClusterUtils::allTrue); } diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/BookieShellTestBase.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/BookieShellTestBase.java index 276b842f6ed..05ff030470a 100644 --- a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/BookieShellTestBase.java +++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/BookieShellTestBase.java @@ -30,8 +30,8 @@ @Slf4j public abstract class BookieShellTestBase { - private String currentVersion = System.getProperty("currentVersion"); - private String bkScript; + String currentVersion = System.getProperty("currentVersion"); + String bkScript; @Before public void setup() { diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java index 8b138c7b465..47fbc554715 100644 --- a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java +++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java @@ -18,14 +18,23 @@ package org.apache.bookkeeper.tests.integration; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertTrue; import com.github.dockerjava.api.DockerClient; import lombok.extern.slf4j.Slf4j; + +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.api.BookKeeper; +import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.client.api.WriteHandle; +import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.tests.integration.utils.BookKeeperClusterUtils; import org.apache.bookkeeper.tests.integration.utils.DockerUtils; import org.jboss.arquillian.junit.Arquillian; import org.jboss.arquillian.test.api.ArquillianResource; +import org.junit.Assert; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runner.RunWith; @@ -44,6 +53,8 @@ public class TestBookieShellCluster extends BookieShellTestBase { private String currentVersion = System.getProperty("currentVersion"); + private static final byte[] PASSWORD = "foobar".getBytes(UTF_8); + @Test @Override public void test000_Setup() throws Exception { @@ -83,4 +94,92 @@ public void test002_ListROBookies() throws Exception { public void test003_ListRWBookies() throws Exception { super.test003_ListRWBookies(); } + + private static long writeNEntries(BookKeeper bk, int n, int numBookies) throws Exception { + try (WriteHandle writer = bk.newCreateLedgerOp().withEnsembleSize(numBookies) + .withWriteQuorumSize(numBookies).withAckQuorumSize(numBookies) + .withPassword(PASSWORD).execute().get()) { + int i = 0; + for (; i < n - 1; i++) { + writer.appendAsync(("entry" + i).getBytes(UTF_8)); + } + writer.append(("entry" + i).getBytes(UTF_8)); + + return writer.getId(); + } + } + + private static void validateNEntries(BookKeeper bk, long ledgerId, int n) throws Exception { + try (ReadHandle reader = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(PASSWORD) + .execute().get(); + LedgerEntries entries = reader.read(0, n - 1)) { + Assert.assertEquals(reader.getLastAddConfirmed(), n - 1); + + for (int i = 0; i < n; i++) { + Assert.assertEquals("entry" + i, new String(entries.getEntry(i).getEntryBytes(), UTF_8)); + } + } + } + + /** + * These tests on being able to access cluster internals, so can't be put in test base. + */ + @Test + public void test101_RegenerateIndex() throws Exception { + String zookeeper = String.format("zk+hierarchical://%s/ledgers", + BookKeeperClusterUtils.zookeeperConnectString(docker)); + int numEntries = 100; + + try (BookKeeper bk = BookKeeper.newBuilder( + new ClientConfiguration().setMetadataServiceUri(zookeeper)).build()) { + log.info("Writing entries"); + long ledgerId1 = writeNEntries(bk, numEntries, BookKeeperClusterUtils.allBookies().size()); + long ledgerId2 = writeNEntries(bk, numEntries, BookKeeperClusterUtils.allBookies().size()); + + log.info("Validate that we can read back"); + validateNEntries(bk, ledgerId1, numEntries); + validateNEntries(bk, ledgerId2, numEntries); + + String indexFileName1 = String.format("/opt/bookkeeper/data/ledgers/current/0/%d/%d.idx", + ledgerId1, ledgerId1); + String indexFileName2 = String.format("/opt/bookkeeper/data/ledgers/current/0/%d/%d.idx", + ledgerId2, ledgerId2); + + log.info("Stop bookies to flush, delete the index and start again"); + assertTrue(BookKeeperClusterUtils.stopAllBookies(docker)); + + BookKeeperClusterUtils.runOnAllBookies(docker, "rm", indexFileName1, indexFileName2); + assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, currentVersion)); + + log.info("Validate that we cannot read back"); + try { + validateNEntries(bk, ledgerId1, numEntries); + Assert.fail("Shouldn't have been able to find anything"); + } catch (BKException.BKNoSuchLedgerExistsException e) { + // expected + } + try { + validateNEntries(bk, ledgerId2, numEntries); + Assert.fail("Shouldn't have been able to find anything"); + } catch (BKException.BKNoSuchLedgerExistsException e) { + // expected + } + + assertTrue(BookKeeperClusterUtils.stopAllBookies(docker)); + + log.info("Regenerate the index file"); + BookKeeperClusterUtils.runOnAllBookies(docker, + bkScript, "shell", "regenerate-interleaved-storage-index-file", + "--ledgerIds", String.format("%d,%d", ledgerId1, ledgerId2), + "--password", new String(PASSWORD, UTF_8)); + assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, currentVersion)); + + log.info("Validate that we can read back, after regeneration"); + validateNEntries(bk, ledgerId1, numEntries); + validateNEntries(bk, ledgerId2, numEntries); + } + } + } From 51c687f350a51811e5b095c6528ff0c17e3141e1 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 6 Sep 2018 22:02:04 -0700 Subject: [PATCH 0020/1642] BP-35: 128 bits support Descriptions of the changes in this PR: ### Motivation BookKeeper coordinates with a metadata store to generate a cluster wide `ledgerId`. Currently this is a signed `64 bit` number (effectively 63 bits). This method works great because we have a centralized metadata store for coordinating the id generation. However this method may not scale as the cluster size and number of ledgers grow. [Universally unique identifier - Wikipedia](https://en.wikipedia.org/wiki/Globally_unique_identifier) is a preferred way to generate decentralized globally unique IDs and it takes `128 bits`. This method can scale well as it doesn't need a centralized coordination. This BP proposes the changes for increasing ledger id from `63 bits` to `128 bits`. Master Issue: #603 Author: Sijie Guo Reviewers: Andrey Yegorov , Enrico Olivelli , Jia Zhai , Venkateswararao Jujjuri (JV) This closes #1611 from sijie/bp_35_128bits --- site/bps/BP-35-128-bits-support.md | 399 +++++++++++++++++++++++++ site/community/bookkeeper_proposals.md | 3 +- 2 files changed, 401 insertions(+), 1 deletion(-) create mode 100644 site/bps/BP-35-128-bits-support.md diff --git a/site/bps/BP-35-128-bits-support.md b/site/bps/BP-35-128-bits-support.md new file mode 100644 index 00000000000..cae6b8c4ac3 --- /dev/null +++ b/site/bps/BP-35-128-bits-support.md @@ -0,0 +1,399 @@ +--- +title: "BP-34: 128 bits support" +issue: https://github.com/apache/bookkeeper/603 +state: "Under Discussion" +design doc: https://docs.google.com/document/d/1cu54dNSV2ZrdWCi40LcyX8NxXGRCW0609T_ewmK9BWM +release: "4.9.0" +--- + +### Motivation + +BookKeeper coordinates with a metadata store to generate a cluster wide `ledgerId`. +Currently this is a signed `64 bit` number (effectively 63 bits). This method works +great because we have a centralized metadata store for coordinating the id generation. +However this method may not scale as the cluster size and number of ledgers grow. + +[Universally unique identifier - Wikipedia](https://en.wikipedia.org/wiki/Globally_unique_identifier) +is a preferred way to generate decentralized globally unique IDs and it takes `128 bits`. +This method can scale well as it doesn't need a centralized coordination. + +This BP proposes the changes for increasing ledger id from `63 bits` to `128 bits`. + +### 128 bits + +Since there is no native support for `128 bits` in both Java and +[Protobuf](https://github.com/google/protobuf/issues/2180), we have to break `128 bits` +into 2 `64 bits` numbers for representing the `128 bits` id: + +- ledger-id-msb: the most significant 64 bits, bit 64 - 127 +- ledger-id-lsb: the least significant 64 bits, bit 0 - 63 + +For backward compatibility, the `ledger-id-lsb` is the current `64 bits` ledger-id. +The `ledger-id-msb` will be added as a new field in both API and protocol. + +I am proposing calling `ledger-id-msb` as `ledger-scope-id`. So the current 64bits `ledgerId` and +the newly introduced 64bits `ledgerScopeId` together will be forming the new `128 bits` ledger id. + +The default `ledgerScopeId` is `0`. That means any ledgers created prior to this change are allocated +under scope `0`. Hence it maintains backward compatibility during upgrade. + +The combination of `ledgerScopeId` and `ledgerId` forms the `128 bits` ledger id. We can introduce a +hex representation of this `128 bits` ledger id - `ledgerQualifiedName` . This `ledgerQualifiedName` can +be useful for CLI tooling, REST api and troubleshooting purpose. The API internally can convert +`ledgerQualifiedName` to `ledgerScopeId` and `ledgerId`. + +### Public Interfaces + +#### API Change + +The API will be introducing `ledgerScopeId` across the interfaces. This field will be optional and default to `0`. + +##### Handle + +Introduce a new method `getScopeId` for representing the scope id (the most significant `128 bits` ledger id). + +```java +public interface Handle extends AutoCloseable { + + ... + + /** + * Return the ledger scope id. The most significant 64 bits of 128 bits. + */ + long getScopeId(); + + /** + * Return the ledger id. The least significant 64 bits of 128 bits. + */ + long getId(); + + ... + +} +``` + +##### Create LedgerAdv + +Introduce a new method `withLedgerScopeId` in `CreateAdvBuilder` for providing `scopeId` +(the most significant 64 bits for 128 bits ledger id) on creating a ledger. + +```java +public interface CreateAdvBuilder extends OpBuilder { + ... + + /** + * Set the scope id for the newly created ledger. + * If no explicit scopeId is passed, the new ledger + * will be created under scope `0`. + */ + CreateAdvBuilder withLedgerScopeId(long scopeId); + + ... +} +``` + +##### Open Ledger + +Introduce a new method `withLedgerScopeId` in `OpenBuilder` for providing `scopeId` +(the most significant 64 bits for 128 bits ledger id) on opening a ledger. + +```java +public interface OpenBuilder extends OpBuilder { + ... + /** + * Set the scope id of the ledger to open. + */ + OpenBuilder withLedgerScopeId(long scopeId); + ... +} +``` + +##### Delete Ledger + +Introduce a new method `withLedgerScopeId` in `DeleteBuilder` for providing `scopeId` +(the most significant 64 bits for 128 bits ledger id) on deleting a ledger. + +```java +public interface DeleteBuilder extends OpBuilder { + ... + /** + * Set the scope id of the ledger to delete. + */ + DeleteBuilder withLedgerScopeId(long scopeId); + ... +} +``` + +#### CLI + +All BookKeeper CLI tools will be updated with additional option `—ledger-scope-id`. +Optionally we can add option `—ledger-qualified-name` (the hex representation of 128 bits). +Internally all the CLI tools will convert ledger qualified name to `ledgerId` and `ledgerScopeId`. + +#### REST + +1. All ledger related endpoints will be adding a new parameter `ledger_scope_id`. +2. `ListLedgerService` only supports listing ledgers under a given ledger scope id. + If `ledger_scope_id` is missing, it will be listing ledgers under scope `0`. + +#### Wire Protocol + +> There will be no plan for supporting 128 bits in v2 protocol, due to the limitation in v2 protocol. +> So any operations in v2 protocol with scope id not equal to 0 will be failed immediately with +> `ILLEGAL_OP` exceptions. + +All the request and response messages will be adding an optional field `optional int64 ledgerScopeId`. + +#### Entry Format + +Currently all the entries written to bookies are encoded in a certain format, including `metadata`, +`digest code` and `payload`. The entry format is not *versioned*. + +In order to support adding another field `ledgerScopeId` in the `metadata` section, we are introducing +`version` in the entry format. + +##### Entry Format V1 + +```json +Entry Format V1 +=============== +--- header --- +Bytes (0 - 7) : Ledger ID +Bytes (8 - 15) : Entry ID +Bytes (16 - 23) : LastAddConfirmed +Bytes (24 - 31) : Length +--- digest --- +Bytes (32 - (32 + x - 1)) : Digest Code (e.g. CRC32) +--- payload --- +Bytes ((32 + x) - ) : Payload +``` + +> `x` is the length of digest code. + +> Prior to introducing `ledgerScopeId`, ledgerId is assumed to be a positive value. + +##### Entry Format V2 + +```json +Entry Format V2 +=============== +--- header --- +Bytes (0 - 7) : Metadata Flags +Bytes (8 - 15) : Ledger Scope ID +Bytes (16 - 23) : Ledger ID +Bytes (24 - 31) : Entry ID +Bytes (32 - 39) : LastAddConfirmed +Bytes (40 - 47) : Length +--- digest --- +Bytes (37 - (37 + x - 1)) : Digest Code (e.g. CRC32) +--- payload --- +Bytes ((37 + x) - ) : Payload +``` + +> `x` is the length of digest code. + +###### Metadata Flags + +```json +Metadata: 1 Bytes (Long) +------------------------ +0x 0 0 + |__| + | + version + +---- +Bit 0 - 3: digest type (e.g. CRC32, CRC32C and such) +Bit 4 - 7: version, the most significant bit of this byte will be always set to 1. +it will be used for differentiating entry format v1 and v2. + +``` + +We are setting the most significant bit to be `1`. So the first byte in entry v2 will +be a negative value, which can be used for differentiating entry format v1 and v2. +The version will be encoded into the first byte. The version will be used for describing +the entry format. + +##### Decoding Entry + +The pseudo code for decoding an entry will be described as followings: + +```java + +ByteBuf entry = ...; + +int metadataFlags = entry.getByte(); + +if (metadataFlags <= 128) { // the entry is encoded in v1 format + // decoding the entry in v1 format + ... +} else { + // decoding the entry in v2 format +} + +``` + +#### Bookie Storage + +##### Journal + +A new method should be added in journal `WriteCallback` to handle `ledgerScopeId`. + +```java +public interface WriteCallback { + + void writeComplete(int rc, + long ledgerScopeId, + long ledgerId, + long entryId, + BookieSocketAddress addr, + Object ctx); + + default void writeComplete(int rc, + long ledgerId, + long entryId, + BookieSocketAddress addr, + Object ctx) { + writeComplete(rc, 0L, ledgerId, entryId, addr, ctx); + } + +} +``` + +The journal should be changed to be able to retrieve `ledgerScopeId` from the entry +payload based on [Entry Format](#entry-format). + +##### Ledger Storage + +###### EntryLogger + +1. Methods in `EntryLogger` should be able to accept `ledgerScopeId` as a parameter. +2. EntryLogger should be updated to retrieve `ledgerScopeId` from the entry payload + based on [Entry Format](#entry-format). + +###### EntryMemTable + +`ledgerScopeId` should be added as part of `EntryKey`. + +###### IndexPersistenceMgr + +Currently the ledger index files (64 bits) are stored into 2-level-hirechicy +directories - `//.idx`. + +If `ledgerScopeId` is 0, it will be using existing scheme for storing and retrieving +ledger index files. + +If `ledgerScopeId` is not 0, that means the ledgers are produced by new clients that +support 128-bits, those ledgers will be stored in a 4-level-hirechicy +directories - +`///`. + +All the file info caches should be updated to use `` +as index keys. + +###### IndexInMemPageMgr + +The LRU pages map will be updated to use `` as index +keys. + +###### DBLedgerStorage + +Currently DBLedgerStorage use `` as the index key for indexing entry +locations for each entry. + +Similar as `SortedLedgerStorage` and `InterleavedLedgerStorage`, for ledgers whose +`ledgerScopeId` is 0, they will be using existing scheme for storing their entry locations. + +For ledgers whose `ledgerScopeId` is not 0, they will be stored in a new rocksdb, +whose index key will be ``. + +#### Metadata Store + +##### LedgerManager + +All the interfaces should be updated with accepting `ledgerScopeId`. + +The actual implementation should decide how to store metadata +for ``. + +###### ZooKeeper Ledger Manager + +We need to introduce a LongLongHierchicalLedgerManager for storing metadata +indexing by ``. + +If `ledgerScopeId` is 0, then it will be falling back to `LongHierachicalLedgerManager`. +So no behavior is changed. + +If `ledgerScopeId` is not 0, those ledgers will be indexed in new hierarchy +(possible under a different znode). + +###### Ledger ID generation + +When upgrading from 64bit to 128bits, we probably don't need any centralized mechanism +for generating ledger id. It can be implemented using UUID generation. + +Especially since we are supporting 128bits by introducing `ledgerScopeId`. That means +application of bookkeeper can decide its own way for generating their `scopeId`. +An application or even bookkeeper client can generate its ledgerId using UUID generation, +then breaks the 128 bits UUID into two parts, one serves as `ledgerScopeId` and the other +one serves as `ledgerId`. + +###### Etcd + +Since Etcd has a better key/value presentation, we can basically just combine +`` as the index key for storing ledger metadata in Etcd. +Nothing is needed for special consideration. + +### Performance Concerns + +There shouldn't be any performance difference when not using 128 bit ledger id +(`ledgerScopeId` is omitted). + +Performance concerns can be arised in following areas: + +- **Wire Protocol**: additional 9 bytes will be added per entry, one byte for version + and 8 bytes for the msb of 128 bit ledger id +- **Journal**: additional 9 bytes will be added per entry (same as wire protocol). +- **EntryLogger**: additional 9 bytes will be added per entry (same as wire protocol) +- **Memtable**: additional 8 bytes will be added per indexed entry. +- **FileInfo**: there is no change to the index file format itself. +- **IndexPersistenceManager**: Files will be organized in more directory hierarchy. + It shouldn't be a big deal. +- **IndexInMemoryManager (LedgerCache)**: additional 8 bytes per index page. +- **DbLedgerStorage**: additional 8 bytes per entry for entry location. +- **Metadata**: on zookeeper, we need a 128 bit ledger manager, that means more znode + hierarchy than 64 bit ledger manager. Etcd like key/value metadata store is probably + more preferrable for 128 bit ledger manager. + +However increasing ledger id from 64 bits to 128 bits can get rid of the only remaining +central point, since we don't need to use zookeeper for ledger id generation. The id +generation can become decentralized. + +### Proposed Changes + +All the required changes are described above. In summary, the changes can +happen in following 2 phases: + +1. Ensure all components have `ledgerScopeId` added (both wire protocol, storage and such). + Assuming `ledgerScopeId` will be 0. The changes can happen independently and ensure + they are backward compatible with old clients. +2. Add `ledgerScopeId` into public API, so application can start using `ledgerScopeId`. + After that, applications can use UUID to generate ledger id and break UUID into two parts, + one is `ledgerScopeId`, while the other one is `ledgerId`. + +### Compatibility, Deprecation, and Migration Plan + +All the changes are backward compatible, since we are doing the changes by adding an optional +field `ledgerScopeId`. Old clients can still operating in the mode of `ledgerScopeId == 0`. +The new application can activate the feature by starting using `ledgerScopeId` in the new API. + +### Test Plan + +1. Add unit tests for individual components on introducing `ledgerScopeId`. +2. Add backward compatibility tests for individual components. +3. Add end-to-end integration tests for introducing `ledgerScopeId`. +4. Add end-to-end backward compatibility tests. + +### Rejected Alternatives + +N/A diff --git a/site/community/bookkeeper_proposals.md b/site/community/bookkeeper_proposals.md index ee8899e7a18..94bd6a5e00c 100644 --- a/site/community/bookkeeper_proposals.md +++ b/site/community/bookkeeper_proposals.md @@ -85,7 +85,7 @@ using Google Doc. This section lists all the _bookkeeper proposals_ made to BookKeeper. -*Next Proposal Number: 35* +*Next Proposal Number: 36* ### Inprogress @@ -106,6 +106,7 @@ Proposal | State [BP-32: Advisory (optimistic) write close](../../bps/BP-32-advisory-write-close) | Accepted [BP-33: Move releasing docker images out of main repo](../../bps/BP-33-building-official-docker-imags) | Draft [BP-34: Cluster Metadata Checker](../../bps/BP-34-cluster-metadata-checker) | Accepted +[BP-35: 128 bits support](../../bps/BP-35-128-bits-support) | Under Discussion ### Adopted From 3083cdae0eab58bd94dbe7cd813500a09775f4fa Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 7 Sep 2018 11:42:11 -0700 Subject: [PATCH 0021/1642] [DEV] clean up merge script Descriptions of the changes in this PR: ### Motivation The development of bookkeeper has been moved from JIRA to Github. So we can clean up the merge script to remove those instructions assuming we are still on apache mirror and JIRA. ### Changes This change follows the changes that I have adjusted at apache/incubator-pulsar#2526 - remove related logic about JIRA from merge script - change merge script to use github merge api Author: Reviewers: Enrico Olivelli This closes #1663 from sijie/improve_merge_script_2 --- dev/bk-merge-pr.py | 236 +++++++++++---------------------------------- 1 file changed, 57 insertions(+), 179 deletions(-) diff --git a/dev/bk-merge-pr.py b/dev/bk-merge-pr.py index c7d2adf4cb1..2c698908b9f 100755 --- a/dev/bk-merge-pr.py +++ b/dev/bk-merge-pr.py @@ -23,9 +23,7 @@ # Usage: ./bk-merge-pr.py (see config env vars below) # # This utility assumes you already have local a bookkeeper git folder and that you -# have added remotes corresponding to both: -# (i) the github apache bookkeeper mirror and -# (ii) the apache bookkeeper git repo. +# have added remotes corresponding to the github apache bookkeeper repo. import json import os @@ -34,27 +32,15 @@ import sys import urllib2 -try: - import jira.client - JIRA_IMPORTED = True -except ImportError: - JIRA_IMPORTED = False - PROJECT_NAME = "bookkeeper" -CAPITALIZED_PROJECT_NAME = "bookkeeper".upper() +CAPITALIZED_PROJECT_NAME = PROJECT_NAME.upper() GITHUB_ISSUES_NAME = "issue".upper() # Location of the local git repository REPO_HOME = os.environ.get("%s_HOME" % CAPITALIZED_PROJECT_NAME, os.getcwd()) # Remote name which points to the GitHub site -PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") -# Remote name which points to Apache git -PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") -# ASF JIRA username -JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "") -# ASF JIRA password -JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "") +PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache") # Github API page size GITHUB_PAGE_SIZE = os.environ.get("GH_PAGE_SIZE", "100") # OAuth key used for issuing requests against the GitHub API. If this is not defined, then requests @@ -67,8 +53,6 @@ GITHUB_BASE = "https://github.com/%s/%s/pull" % (GITHUB_USER, PROJECT_NAME) GITHUB_API_URL = "https://api.github.com" GITHUB_API_BASE = "%s/repos/%s/%s" % (GITHUB_API_URL, GITHUB_USER, PROJECT_NAME) -JIRA_BASE = "https://issues.apache.org/jira/browse" -JIRA_API_BASE = "https://issues.apache.org/jira" # Prefix added to temporary branches TEMP_BRANCH_PREFIX = "PR_TOOL" RELEASE_BRANCH_PREFIX = "branch-" @@ -95,8 +79,22 @@ def get_json(url, preview_api = False): def post_json(url, data): try: request = urllib2.Request(url, data, { 'Content-Type': 'application/json' }) - if GITHUB_OAUTH_KEY: - request.add_header('Authorization', 'token %s' % GITHUB_OAUTH_KEY) + request.add_header('Authorization', 'token %s' % GITHUB_OAUTH_KEY) + return json.load(urllib2.urlopen(request)) + except urllib2.HTTPError as e: + if "X-RateLimit-Remaining" in e.headers and e.headers["X-RateLimit-Remaining"] == '0': + print "Exceeded the GitHub API rate limit; see the instructions in " + \ + "bk-merge-pr.py to configure an OAuth token for making authenticated " + \ + "GitHub requests." + else: + print "Unable to fetch URL, exiting: %s - %s" % (url, e) + sys.exit(-1) + +def put_json(url, data): + try: + request = urllib2.Request(url, data, { 'Content-Type': 'application/json' }) + request.get_method = lambda: 'PUT' + request.add_header('Authorization', 'token %s' % GITHUB_OAUTH_KEY) return json.load(urllib2.urlopen(request)) except urllib2.HTTPError as e: if "X-RateLimit-Remaining" in e.headers and e.headers["X-RateLimit-Remaining"] == '0': @@ -105,8 +103,10 @@ def post_json(url, data): "GitHub requests." else: print "Unable to fetch URL, exiting: %s - %s" % (url, e) + print e sys.exit(-1) + def fail(msg): print msg clean_up() @@ -150,38 +150,7 @@ def get_all_labels(): # merge the requested PR and return the merge hash def merge_pr(pr_num, target_ref, title, body, default_pr_reviewers, pr_repo_desc): pr_branch_name = "%s_MERGE_PR_%s" % (TEMP_BRANCH_PREFIX, pr_num) - target_branch_name = "%s_MERGE_PR_%s_%s" % (TEMP_BRANCH_PREFIX, pr_num, target_ref.upper()) run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) - run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) - run_cmd("git checkout %s" % target_branch_name) - - had_conflicts = False - try: - run_cmd(['git', 'merge', pr_branch_name, '--squash']) - except Exception as e: - msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e - continue_maybe(msg) - msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" - continue_maybe(msg) - had_conflicts = True - - # Offer to run unit tests before committing - result = raw_input('Do you want to validate unit tests after the merge? (y/n): ') - if result.lower() == 'y': - test_res = subprocess.call('mvn clean install'.split()) - if test_res == 0: - print('Unit tests execution succeeded') - else: - continue_maybe("Unit tests execution FAILED. Do you want to continue with the merge anyway?") - - # Offer to run spotbugs and rat before committing - result = raw_input('Do you want to validate spotbugs and rat after the merge? (y/n): ') - if result.lower() == 'y': - test_res = subprocess.call('mvn clean install -DskipTests spotbugs:check apache-rat:check'.split()) - if test_res == 0: - print('QA tests execution succeeded') - else: - continue_maybe("QA tests execution FAILED. Do you want to continue with the merge anyway?") commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, '--pretty=format:%an <%ae>']).split("\n") @@ -211,25 +180,17 @@ def merge_pr(pr_num, target_ref, title, body, default_pr_reviewers, pr_repo_desc merge_message_flags = [] - merge_message_flags += ["-m", title] if body is not None: # We remove @ symbols from the body to avoid triggering e-mails # to people every time someone creates a public fork of the project. - merge_message_flags += ["-m", body.replace("@", "")] + merge_message_flags += [body.replace("@", "")] authors = "\n".join(["Author: %s" % a for a in distinct_authors]) - merge_message_flags += ["-m", authors] + merge_message_flags += [authors] if (reviewers != ""): - merge_message_flags += ["-m", "Reviewers: %s" % reviewers] - - if had_conflicts: - committer_name = run_cmd("git config --get user.name").strip() - committer_email = run_cmd("git config --get user.email").strip() - message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % ( - committer_name, committer_email) - merge_message_flags += ["-m", message] + merge_message_flags += ["Reviewers: %s" % reviewers] # The string "Closes #%s" string is required for GitHub to correctly close the PR close_line = "This closes #%s from %s" % (pr_num, pr_repo_desc) @@ -242,24 +203,28 @@ def merge_pr(pr_num, target_ref, title, body, default_pr_reviewers, pr_repo_desc if should_list_commits: close_line += " and squashes the following commits:" - merge_message_flags += ["-m", close_line] + merge_message_flags += [close_line] if should_list_commits: - merge_message_flags += ["-m", "\n".join(commits)] + merge_message_flags += ["\n".join(commits)] - run_cmd(['git', 'commit', '--author="%s"' % primary_author] + merge_message_flags) + pr_sha = run_cmd("git rev-parse %s" % pr_branch_name).rstrip() - continue_maybe("Merge complete (local ref %s). Push to %s?" % ( - target_branch_name, PUSH_REMOTE_NAME)) + merge_url = get_github_issue_merge_url(pr_num) + data = json.dumps({ + 'commit_title': title, + 'commit_message': "\n\n".join(merge_message_flags), + 'sha': pr_sha, + 'merge_method': 'squash' + }, indent = 4) - try: - run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, target_branch_name, target_ref)) - except Exception as e: - clean_up() - fail("Exception while pushing: %s" % e) + continue_maybe("Merge Pull Request (%s) with following details:\n%s" % ( + pr_num, data)) + + resp = put_json(merge_url, data) - merge_hash = run_cmd("git rev-parse %s" % target_branch_name)[:8] - merge_log = run_cmd("git show --format=fuller -q %s" % target_branch_name) + merge_hash = resp['sha'] + merge_log = title + '\n' + "\n\n".join(merge_message_flags) clean_up() print("Pull request #%s merged!" % pr_num) print("Merge hash: %s" % merge_hash) @@ -274,7 +239,7 @@ def ask_for_branch(default_branch): def cherry_pick(pr_num, merge_hash, pick_ref): pick_branch_name = "%s_PICK_PR_%s_%s" % (TEMP_BRANCH_PREFIX, pr_num, pick_ref.upper()) - run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name)) + run_cmd("git fetch %s %s:%s" % (PR_REMOTE_NAME, pick_ref, pick_branch_name)) run_cmd("git checkout %s" % pick_branch_name) try: @@ -286,10 +251,10 @@ def cherry_pick(pr_num, merge_hash, pick_ref): continue_maybe(msg) continue_maybe("Pick complete (local ref %s). Push to %s?" % ( - pick_branch_name, PUSH_REMOTE_NAME)) + pick_branch_name, PR_REMOTE_NAME)) try: - run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, pick_branch_name, pick_ref)) + run_cmd('git push %s %s:%s' % (PR_REMOTE_NAME, pick_branch_name, pick_ref)) except Exception as e: clean_up() fail("Exception while pushing: %s" % e) @@ -317,90 +282,16 @@ def fix_version_from_branch(branch, versions, target_ref): else: return None - -def resolve_jira_issue(merge_branches, comment, jira_id, target_ref): - asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, - basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) - - result = raw_input("Resolve JIRA %s ? (y/n): " % jira_id) - if result.lower() != "y": - return - - try: - issue = asf_jira.issue(jira_id) - except Exception as e: - fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) - - cur_status = issue.fields.status.name - cur_summary = issue.fields.summary - cur_assignee = issue.fields.assignee - if cur_assignee is None: - cur_assignee = "NOT ASSIGNED!!!" - else: - cur_assignee = cur_assignee.displayName - - if cur_status == "Resolved" or cur_status == "Closed": - fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) - print ("=== JIRA %s ===" % jira_id) - print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( - cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) - - versions = asf_jira.project_versions(CAPITALIZED_PROJECT_NAME) - versions = sorted(versions, key=lambda x: x.name, reverse=True) - versions = filter(lambda x: x.raw['released'] is False, versions) - - version_names = map(lambda x: x.name, versions) - default_fix_versions = map(lambda x: fix_version_from_branch(x, version_names), merge_branches, target_ref) - default_fix_versions = filter(lambda x: x != None, default_fix_versions) - default_fix_versions = ",".join(default_fix_versions) - - fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) - if fix_versions == "": - fix_versions = default_fix_versions - fix_versions = fix_versions.replace(" ", "").split(",") - - def get_version_json(version_str): - return filter(lambda v: v.name == version_str, versions)[0].raw - - jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) - - resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] - resolution = filter(lambda r: r.raw['name'] == "Fixed", asf_jira.resolutions())[0] - asf_jira.transition_issue( - jira_id, resolve["id"], fixVersions = jira_fix_versions, - comment = comment, resolution = {'id': resolution.raw['id']}) - - print "Successfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) - - -def resolve_jira_issues(title, merge_branches, comment, target_ref): - jira_ids = re.findall("%s-[0-9]{3,6}" % CAPITALIZED_PROJECT_NAME, title) - - if len(jira_ids) == 0: - print "No JIRA issue found to update" - for jira_id in jira_ids: - resolve_jira_issue(merge_branches, comment, jira_id, target_ref) - - -def standardize_jira_ref(text): +def standardize_issue_ref(text): """ - Standardize the jira reference commit message prefix to "PROJECT_NAME-XXX: Issue" + Standardize the github reference commit message prefix to "Issue #XXX: Issue" - 'BOOKKEEPER-877: Script for generating patch for reviews' 'ISSUE #376: Script for generating patch for reviews' """ - jira_refs = [] github_issue_refs = [] github_issue_ids = [] components = [] - # Extract JIRA ref(s): - pattern = re.compile(r'(%s[-\s]*[0-9]{3,6})+' % CAPITALIZED_PROJECT_NAME, re.IGNORECASE) - for ref in pattern.findall(text): - # Add brackets, replace spaces with a dash, & convert to uppercase - jira_refs.append(re.sub(r'\s+', '-', ref.upper())) - text = text.replace(ref, '') - # Extract Github Issue ref(s) pattern = re.compile(r'(%s[-\s]*([0-9]{3,6}))+' % GITHUB_ISSUES_NAME, re.IGNORECASE) for ref in pattern.findall(text): @@ -421,17 +312,14 @@ def standardize_jira_ref(text): if (pattern.search(text) is not None): text = pattern.search(text).groups()[0] - # Assemble full text (JIRA ref(s), module(s), remaining text) + # Assemble full text (github ref(s), module(s), remaining text) prefix = '' - jira_prefix = ' '.join(jira_refs).strip() - if jira_prefix: - prefix = jira_prefix + ": " github_prefix = ' '.join(github_issue_refs).strip() if github_prefix: prefix = github_prefix + ": " clean_text = prefix + ' '.join(components).strip() + " " + text.strip() - # Replace multiple spaces with a single space, e.g. if no jira refs and/or components were included + # Replace multiple spaces with a single space, e.g. if no refs and/or components were included clean_text = re.sub(r'\s+', ' ', clean_text.strip()) return clean_text, github_issue_ids @@ -630,6 +518,9 @@ def ask_for_labels(prefix, labels, issue_labels): def get_github_issue_url(github_issue_id): return "https://api.github.com/repos/%s/%s/issues/%s" % (GITHUB_USER, PROJECT_NAME, github_issue_id) +def get_github_issue_merge_url(github_issue_id): + return "https://api.github.com/repos/%s/%s/pulls/%s/merge" % (GITHUB_USER, PROJECT_NAME, github_issue_id) + def get_assignees_url(github_issue_id): return "https://api.github.com/repos/%s/%s/issues/%s/assignees" % (GITHUB_USER, PROJECT_NAME, github_issue_id) @@ -725,7 +616,7 @@ def main(): commit_title = pr_title # Decide whether to use the modified title or not - modified_title, github_issue_ids = standardize_jira_ref(commit_title) + modified_title, github_issue_ids = standardize_issue_ref(commit_title) if modified_title != commit_title: print "I've re-written the title as follows to match the standard format:" print "Original: %s" % commit_title @@ -744,6 +635,7 @@ def main(): if line.startswith('>'): continue modified_body = modified_body + line + "\n" + modified_body = modified_body.rstrip("\n") if modified_body != body: print "I've re-written the body as follows to match the standard formats:" print "Original: " @@ -819,8 +711,8 @@ def main(): if not bool(pr["mergeable"]): msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ - "Continue? (experts only!)" - continue_maybe(msg) + "You may need to rebase the PR." + fail(msg) print ("\n=== Pull Request #%s ===" % pr_num) print ("PR title\t%s\nCommit title\t%s\nSource\t\t%s\nTarget\t\t%s\nURL\t\t%s" % ( @@ -831,6 +723,9 @@ def main(): # proceed with the merge merge_hash, merge_commit_log = merge_pr(pr_num, target_ref, commit_title, body, pr_reviewers, pr_repo_desc) + # once the pr is merged, refresh the local repo + run_cmd("git fetch %s" % PR_REMOTE_NAME) + pick_prompt = "Would you like to pick %s into another branch?" % merge_hash while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": pick_ref = ask_for_branch(latest_branch) @@ -841,23 +736,6 @@ def main(): all_issue_labels = add_release_to_github_issues(github_issue_ids, all_issue_labels, fix_releases[0]) merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, pick_ref)] - if JIRA_IMPORTED: - if JIRA_USERNAME and JIRA_PASSWORD: - jira_comment = '''Issue resolved by merging pull request %s - [%s/%s] - - {noformat} - %s - {noformat} - ''' % (pr_num, GITHUB_BASE, pr_num, merge_commit_log) - resolve_jira_issues(commit_title, merged_refs, jira_comment, target_ref) - else: - print "JIRA_USERNAME and JIRA_PASSWORD not set" - print "Exiting without trying to close the associated JIRA." - else: - print "Could not find jira-python library. Run 'sudo pip install jira' to install." - print "Exiting without trying to close the associated JIRA." - if __name__ == "__main__": import doctest (failure_count, test_count) = doctest.testmod() From 2f89796d2ac910370285e645fee715a75bf0bd2f Mon Sep 17 00:00:00 2001 From: Samuel Just Date: Fri, 7 Sep 2018 15:07:18 -0700 Subject: [PATCH 0022/1642] ISSUE #1653: Clients with zk:// uri's should read layout from store zk+null:// will cause the layout to be read from the store. This patch updates zk:// to do the same. This matches past behavior with an unspecified ledgerManagerFactoryClass. Some tests needed to be updated with the new behavior. (bug W-5415120) (rev cguttapalem) Signed-off-by: Samuel Just Author: Samuel Just Reviewers: Sijie Guo This closes #1668 from athanatos/forupstream/wip-1653, closes #1653 --- .../bookkeeper/meta/zk/ZKMetadataDriverBase.java | 3 ++- .../apache/bookkeeper/meta/TestLedgerManager.java | 2 +- .../meta/zk/ZKMetadataDriverBaseStaticTest.java | 15 ++++++++++++--- .../test/BookKeeperClusterTestCase.java | 4 ++++ 4 files changed, 19 insertions(+), 5 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java index a9e1a13966d..e9efc82cd22 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java @@ -117,7 +117,8 @@ public static Class resolveLedgerManagerFactory( + schemeParts[1] + "' at uri : " + metadataServiceUri); } } else { - ledgerManagerFactoryClass = HierarchicalLedgerManagerFactory.class; + // behave as in the +null case, infer the layout from the store or fall back to the default + ledgerManagerFactoryClass = null; } return ledgerManagerFactoryClass; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerManager.java index 499a6e9521b..a91485a1d32 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerManager.java @@ -183,7 +183,7 @@ public void testBadZkContents() throws Exception { String root0 = "/badzk0"; zkc.create(root0, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - conf.setMetadataServiceUri(newMetadataServiceUri(root0)); + conf.setMetadataServiceUri(newMetadataServiceUri(root0, HierarchicalLedgerManagerFactory.NAME)); LedgerLayout layout = new LedgerLayout("DoesNotExist", 0xdeadbeef); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBaseStaticTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBaseStaticTest.java index 0ed3f7d31b6..66b096172bc 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBaseStaticTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBaseStaticTest.java @@ -64,11 +64,20 @@ public void testResolveLedgerManagerFactoryUnknownScheme() { } @Test - public void testResolveLedgerManagerFactoryDefaultValue() { + public void testResolveLedgerManagerFactoryUnspecifiedLayout() { assertEquals( - HierarchicalLedgerManagerFactory.class, + null, ZKMetadataDriverBase.resolveLedgerManagerFactory( - URI.create("zk://127.0.0.1/ledgers")) + URI.create("zk://127.0.0.1/ledgers")) + ); + } + + @Test + public void testResolveLedgerManagerFactoryNullLayout() { + assertEquals( + null, + ZKMetadataDriverBase.resolveLedgerManagerFactory( + URI.create("zk+null://127.0.0.1/ledgers")) ); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index e42b8187fb8..932abd9d1cf 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -315,6 +315,10 @@ protected String newMetadataServiceUri(String ledgersRootPath) { return zkUtil.getMetadataServiceUri(ledgersRootPath); } + protected String newMetadataServiceUri(String ledgersRootPath, String type) { + return zkUtil.getMetadataServiceUri(ledgersRootPath, type); + } + /** * Get bookie address for bookie at index. */ From 550ee52f0620b59cc2c147baaee2b6e309c8a590 Mon Sep 17 00:00:00 2001 From: Samuel Just Date: Fri, 7 Sep 2018 15:08:57 -0700 Subject: [PATCH 0023/1642] ISSUE #1662: LocalBookKeeper use layout manager and ledger path configs The URI patch had a side effect of causing LocalBookKeeper to always use a URI string of the form zk:// which defaults to HLM. This patch updates LocalBookKeeper to generate zk+:// based on the legacy LEDGER_MANAGER_FACTORY_CLASS config when present. +null will be used otherwise and defaults to HLM. Further, use the (admittedly deprecated) zkLedgerRootPath config to determine the ledger path (defaults to /ledger, so the behavior will be unchanged if unspecified). (bug W-5415276) (rev cguttapalem) Signed-off-by: Samuel Just Author: Samuel Just Reviewers: Sijie Guo This closes #1669 from athanatos/forupstream/wip-1662, closes #1662 --- .../conf/AbstractConfiguration.java | 58 +++++++++++-------- .../bookkeeper/util/LocalBookKeeper.java | 52 ++++++++++------- .../conf/AbstractConfigurationTest.java | 3 +- 3 files changed, 67 insertions(+), 46 deletions(-) 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 1ce4cf359d5..9f9547cef50 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 @@ -225,35 +225,12 @@ public String getMetadataServiceUriUnchecked() throws UncheckedConfigurationExce * @return metadata service uri. * @throws ConfigurationException if the metadata service uri is invalid. */ - @SuppressWarnings("deprecation") public String getMetadataServiceUri() throws ConfigurationException { String serviceUri = getString(METADATA_SERVICE_URI); if (null == serviceUri) { // no service uri is defined, fallback to old settings String ledgerManagerType; - Class factoryClass = getLedgerManagerFactoryClass(); - if (factoryClass == null) { - // set the ledger manager type to "null", so the driver implementation knows that the type is not set. - ledgerManagerType = "null"; - } else { - if (!AbstractZkLedgerManagerFactory.class.isAssignableFrom(factoryClass)) { - // this is a non-zk implementation - throw new UnsupportedOperationException("metadata service uri is not supported for " - + factoryClass); - } - if (factoryClass == HierarchicalLedgerManagerFactory.class) { - ledgerManagerType = HierarchicalLedgerManagerFactory.NAME; - } else if (factoryClass == org.apache.bookkeeper.meta.FlatLedgerManagerFactory.class) { - ledgerManagerType = org.apache.bookkeeper.meta.FlatLedgerManagerFactory.NAME; - } else if (factoryClass == LongHierarchicalLedgerManagerFactory.class) { - ledgerManagerType = LongHierarchicalLedgerManagerFactory.NAME; - } else if (factoryClass == org.apache.bookkeeper.meta.MSLedgerManagerFactory.class) { - ledgerManagerType = org.apache.bookkeeper.meta.MSLedgerManagerFactory.NAME; - } else { - throw new IllegalArgumentException("Unknown zookeeper based ledger manager factory : " - + factoryClass); - } - } + ledgerManagerType = getLedgerManagerLayoutStringFromFactoryClass(); String zkServers = getZkServers(); if (null != zkServers) { // URI doesn't accept ',' @@ -426,6 +403,39 @@ public String getLedgerManagerFactoryClassName() { return getString(LEDGER_MANAGER_FACTORY_CLASS); } + /** + * Get layout string ("null" if unconfigured). + * + * @return null, hierarchical, longhierarchical, or flat based on LEDGER_MANAGER_FACTORY_CLASS + */ + @SuppressWarnings("deprecation") + public String getLedgerManagerLayoutStringFromFactoryClass() throws ConfigurationException { + String ledgerManagerType; + Class factoryClass = getLedgerManagerFactoryClass(); + if (factoryClass == null) { + // set the ledger manager type to "null", so the driver implementation knows that the type is not set. + ledgerManagerType = "null"; + } else { + if (!AbstractZkLedgerManagerFactory.class.isAssignableFrom(factoryClass)) { + // this is a non-zk implementation + throw new ConfigurationException("metadata service uri is not supported for " + factoryClass); + } + if (factoryClass == HierarchicalLedgerManagerFactory.class) { + ledgerManagerType = HierarchicalLedgerManagerFactory.NAME; + } else if (factoryClass == org.apache.bookkeeper.meta.FlatLedgerManagerFactory.class) { + ledgerManagerType = org.apache.bookkeeper.meta.FlatLedgerManagerFactory.NAME; + } else if (factoryClass == LongHierarchicalLedgerManagerFactory.class) { + ledgerManagerType = LongHierarchicalLedgerManagerFactory.NAME; + } else if (factoryClass == org.apache.bookkeeper.meta.MSLedgerManagerFactory.class) { + ledgerManagerType = org.apache.bookkeeper.meta.MSLedgerManagerFactory.NAME; + } else { + throw new IllegalArgumentException("Unknown zookeeper based ledger manager factory : " + + factoryClass); + } + } + return ledgerManagerType; + } + /** * Set Ledger Manager Factory Class. * 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 72ba48a69fc..9555474e069 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 @@ -37,7 +37,6 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.proto.BookieServer; @@ -47,6 +46,7 @@ import org.apache.bookkeeper.shims.zk.ZooKeeperServerShimFactory; import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.io.FileUtils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -62,8 +62,8 @@ public class LocalBookKeeper { protected static final Logger LOG = LoggerFactory.getLogger(LocalBookKeeper.class); public static final int CONNECTION_TIMEOUT = 30000; - private static String newMetadataServiceUri(String zkServers, int port) { - return "zk://" + zkServers + ":" + port + "/ledgers"; + private static String newMetadataServiceUri(String zkServers, int port, String layout, String ledgerPath) { + return "zk+" + layout + "://" + zkServers + ":" + port + ledgerPath; } int numberOfBookies; @@ -73,13 +73,18 @@ public LocalBookKeeper() { } public LocalBookKeeper(int numberOfBookies) { - this(numberOfBookies, 5000, defaultLocalBookiesConfigDir); + this(numberOfBookies, 5000, new ServerConfiguration(), defaultLocalBookiesConfigDir); } - public LocalBookKeeper(int numberOfBookies, int initialPort, String localBookiesConfigDirName) { + public LocalBookKeeper( + int numberOfBookies, + int initialPort, + ServerConfiguration baseConf, + String localBookiesConfigDirName) { this.numberOfBookies = numberOfBookies; this.initialPort = initialPort; this.localBookiesConfigDir = new File(localBookiesConfigDirName); + this.baseConf = baseConf; LOG.info("Running {} bookie(s) on zkServer {}.", this.numberOfBookies); } @@ -94,6 +99,7 @@ public LocalBookKeeper(int numberOfBookies, int initialPort, String localBookies BookieServer bs[]; ServerConfiguration bsConfs[]; Integer initialPort = 5000; + private ServerConfiguration baseConf; File localBookiesConfigDir; /** @@ -121,7 +127,7 @@ public static ZooKeeperServerShim runZookeeper(int maxCC, int zookeeperPort, Fil } @SuppressWarnings("deprecation") - private void initializeZookeeper(AbstractConfiguration conf, String zkHost, int zkPort) throws IOException { + private void initializeZookeeper(String zkHost, int zkPort) throws IOException { LOG.info("Instantiate ZK Client"); //initialize the zk client with values try (ZooKeeperClient zkc = ZooKeeperClient.newBuilder() @@ -129,7 +135,7 @@ private void initializeZookeeper(AbstractConfiguration conf, String zkHost, int .sessionTimeoutMs(zkSessionTimeOut) .build()) { List multiOps = Lists.newArrayListWithExpectedSize(3); - String zkLedgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf); + String zkLedgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseConf); multiOps.add( Op.create(zkLedgersRootPath, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)); multiOps.add( @@ -158,12 +164,13 @@ private static void cleanupDirectories(List dirs) throws IOException { } } - private List runBookies(ServerConfiguration baseConf, String dirSuffix) + private List runBookies(String dirSuffix) throws IOException, KeeperException, InterruptedException, BookieException, - UnavailableException, CompatibilityException, SecurityException, BKException { + UnavailableException, CompatibilityException, SecurityException, BKException, + ConfigurationException { List tempDirs = new ArrayList(); try { - runBookies(baseConf, tempDirs, dirSuffix); + runBookies(tempDirs, dirSuffix); return tempDirs; } catch (IOException ioe) { cleanupDirectories(tempDirs); @@ -188,9 +195,9 @@ private List runBookies(ServerConfiguration baseConf, String dirSuffix) } @SuppressWarnings("deprecation") - private void runBookies(ServerConfiguration baseConf, List tempDirs, String dirSuffix) + private void runBookies(List tempDirs, String dirSuffix) throws IOException, KeeperException, InterruptedException, BookieException, UnavailableException, - CompatibilityException, SecurityException, BKException { + CompatibilityException, SecurityException, BKException, ConfigurationException { LOG.info("Starting Bookie(s)"); // Create Bookie Servers (B1, B2, B3) @@ -256,8 +263,7 @@ private void runBookies(ServerConfiguration baseConf, List tempDirs, Strin } if (null == baseConf.getMetadataServiceUriUnchecked()) { - bsConfs[i].setMetadataServiceUri( - newMetadataServiceUri(InetAddress.getLocalHost().getHostAddress(), zooKeeperDefaultPort)); + bsConfs[i].setMetadataServiceUri(baseConf.getMetadataServiceUri()); } bsConfs[i].setJournalDirName(journalDirs[i].getPath()); @@ -285,8 +291,7 @@ private void runBookies(ServerConfiguration baseConf, List tempDirs, Strin ServerConfiguration baseConfWithCorrectZKServers = new ServerConfiguration( (ServerConfiguration) baseConf.clone()); if (null == baseConf.getMetadataServiceUriUnchecked()) { - baseConfWithCorrectZKServers.setMetadataServiceUri( - newMetadataServiceUri(InetAddress.getLocalHost().getHostAddress(), zooKeeperDefaultPort)); + baseConfWithCorrectZKServers.setMetadataServiceUri(baseConf.getMetadataServiceUri()); } serializeLocalBookieConfig(baseConfWithCorrectZKServers, "baseconf.conf"); } @@ -340,7 +345,14 @@ static void startLocalBookiesInternal(ServerConfiguration conf, String zkDataDir, String localBookiesConfigDirName) throws Exception { - LocalBookKeeper lb = new LocalBookKeeper(numBookies, initialBookiePort, localBookiesConfigDirName); + + conf.setMetadataServiceUri( + newMetadataServiceUri( + zkHost, + zkPort, + conf.getLedgerManagerLayoutStringFromFactoryClass(), + conf.getZkLedgersRootPath())); + LocalBookKeeper lb = new LocalBookKeeper(numBookies, initialBookiePort, conf, localBookiesConfigDirName); ZooKeeperServerShim zks = null; File zkTmpDir = null; @@ -363,10 +375,8 @@ static void startLocalBookiesInternal(ServerConfiguration conf, zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkTmpDir); } - conf.setMetadataServiceUri(newMetadataServiceUri(zkHost, zkPort)); - - lb.initializeZookeeper(conf, zkHost, zkPort); - bkTmpDirs = lb.runBookies(conf, dirSuffix); + lb.initializeZookeeper(zkHost, zkPort); + bkTmpDirs = lb.runBookies(dirSuffix); try { while (true) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java index 8039b0caad3..54e07bd5992 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java @@ -26,6 +26,7 @@ import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LongHierarchicalLedgerManagerFactory; +import org.apache.commons.configuration.ConfigurationException; import org.junit.Before; import org.junit.Test; @@ -76,7 +77,7 @@ public void testSetMetadataServiceUri() throws Exception { } @SuppressWarnings({ "unchecked" }) - @Test(expected = UnsupportedOperationException.class) + @Test(expected = ConfigurationException.class) public void testUnsupportedLedgerManagerFactory() throws Exception { LedgerManagerFactory mockFactory = mock(LedgerManagerFactory.class, CALLS_REAL_METHODS); conf.setLedgerManagerFactoryClass(mockFactory.getClass()); From 9f0eece91d7a5efc115807bf4fabffebdd7bbe73 Mon Sep 17 00:00:00 2001 From: JV Jujjuri Date: Mon, 10 Sep 2018 12:58:47 -0700 Subject: [PATCH 0024/1642] Issue #1664 Cancel Scheduled SpeculativeReads If configured every read request schedules a Future task to send speculative reads on speculativeReadTimeout. When the read is completed successfully, this task must be canceled otherwise it leads to memory consumption and under heavy load the tasks get accumulated which forces lengthy GC cycles. These lengthy GC cycles may cause ZK lease expiry and all other sorts of problems eventually resulting in application errors. This fix makes sure that the scheduled Futures are cancelled at the end of read task. Signed-off-by: Venkateswararao Jujjuri (JV) (ref Andrey) Descriptions of the changes in this PR: ### Motivation (Explain: why you're making that change, what is the problem you're trying to solve) ### Changes (Describe: what changes you have made) Master Issue: # Author: JV Jujjuri Author: Sijie Guo Reviewers: Andrey Yegorov , Enrico Olivelli , Sijie Guo This closes #1665 from jvrao/ups_speculative_cancel, closes #1664 --- ...aultSpeculativeRequestExecutionPolicy.java | 11 ++++++---- .../bookkeeper/client/PendingReadOp.java | 6 ++++- .../client/ReadLastConfirmedAndEntryOp.java | 11 +++++++++- .../SpeculativeRequestExecutionPolicy.java | 5 ++++- .../client/TestSpeculativeRead.java | 22 ++++++++++++++++++- 5 files changed, 47 insertions(+), 8 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java index 7474e56acb4..b2874e5d67f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java @@ -26,6 +26,7 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import org.slf4j.Logger; @@ -64,18 +65,19 @@ public DefaultSpeculativeRequestExecutionPolicy(int firstSpeculativeRequestTimeo * * @param scheduler The scheduler service to issue the speculative request * @param requestExecutor The executor is used to issue the actual speculative requests + * @return ScheduledFuture, in case caller needs to cancel it. */ @Override - public void initiateSpeculativeRequest(final ScheduledExecutorService scheduler, + public ScheduledFuture initiateSpeculativeRequest(final ScheduledExecutorService scheduler, final SpeculativeRequestExecutor requestExecutor) { - scheduleSpeculativeRead(scheduler, requestExecutor, firstSpeculativeRequestTimeout); + return scheduleSpeculativeRead(scheduler, requestExecutor, firstSpeculativeRequestTimeout); } - private void scheduleSpeculativeRead(final ScheduledExecutorService scheduler, + private ScheduledFuture scheduleSpeculativeRead(final ScheduledExecutorService scheduler, final SpeculativeRequestExecutor requestExecutor, final int speculativeRequestTimeout) { try { - scheduler.schedule(new Runnable() { + return scheduler.schedule(new Runnable() { @Override public void run() { ListenableFuture issueNextRequest = requestExecutor.issueSpeculativeRequest(); @@ -107,5 +109,6 @@ public void onFailure(Throwable thrown) { requestExecutor, speculativeRequestTimeout, re); } } + return null; } } 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 d31a744f676..4ee9c925f7a 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 @@ -482,6 +482,10 @@ protected void cancelSpeculativeTask(boolean mayInterruptIfRunning) { } } + public ScheduledFuture getSpeculativeTask() { + return speculativeTask; + } + // I don't think this is ever used in production code -Ivan PendingReadOp parallelRead(boolean enabled) { this.parallelRead = enabled; @@ -518,7 +522,7 @@ void initiate() { for (LedgerEntryRequest entry : seq) { entry.read(); if (!parallelRead && clientCtx.getConf().readSpeculativeRequestPolicy.isPresent()) { - clientCtx.getConf().readSpeculativeRequestPolicy.get() + speculativeTask = clientCtx.getConf().readSpeculativeRequestPolicy.get() .initiateSpeculativeRequest(clientCtx.getScheduler(), entry); } } 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 e61e666e1f7..cb9de32111f 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 @@ -25,6 +25,7 @@ import java.util.BitSet; import java.util.List; import java.util.concurrent.Callable; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; @@ -64,6 +65,7 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt private long lastAddConfirmed; private long timeOutInMillis; private final List currentEnsemble; + private ScheduledFuture speculativeTask = null; abstract class ReadLACAndEntryRequest implements AutoCloseable { @@ -461,6 +463,12 @@ ReadLastConfirmedAndEntryOp parallelRead(boolean enabled) { return this; } + protected void cancelSpeculativeTask(boolean mayInterruptIfRunning) { + if (speculativeTask != null) { + speculativeTask.cancel(mayInterruptIfRunning); + speculativeTask = null; + } + } /** * Speculative Read Logic. */ @@ -491,7 +499,7 @@ public void initiate() { request.read(); if (!parallelRead && clientCtx.getConf().readLACSpeculativeRequestPolicy.isPresent()) { - clientCtx.getConf().readLACSpeculativeRequestPolicy.get() + speculativeTask = clientCtx.getConf().readLACSpeculativeRequestPolicy.get() .initiateSpeculativeRequest(clientCtx.getScheduler(), this); } } @@ -521,6 +529,7 @@ interface LastConfirmedAndEntryCallback { private void submitCallback(int rc) { long latencyMicros = MathUtils.elapsedMicroSec(requestTimeNano); LedgerEntry entry; + cancelSpeculativeTask(true); if (BKException.Code.OK != rc) { clientCtx.getClientStats().getReadLacAndEntryOpLogger() .registerFailedEvent(latencyMicros, TimeUnit.MICROSECONDS); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/SpeculativeRequestExecutionPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/SpeculativeRequestExecutionPolicy.java index bff4bb3f8ea..e04dc98d907 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/SpeculativeRequestExecutionPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/SpeculativeRequestExecutionPolicy.java @@ -21,6 +21,7 @@ package org.apache.bookkeeper.client; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; /** * Define a policy for speculative request execution. @@ -37,6 +38,8 @@ public interface SpeculativeRequestExecutionPolicy { * * @param scheduler The scheduler service to issue the speculative request * @param requestExectuor The executor is used to issue the actual speculative requests + * @return ScheduledFuture, in case caller needs to cancel it. */ - void initiateSpeculativeRequest(ScheduledExecutorService scheduler, SpeculativeRequestExecutor requestExectuor); + ScheduledFuture initiateSpeculativeRequest(ScheduledExecutorService scheduler, + SpeculativeRequestExecutor requestExectuor); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java index 5d251a954a9..0ae13ba546f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java @@ -297,6 +297,26 @@ public void testSpeculativeReadFirstReadCompleteIsOk() throws Exception { } } + /** + * Unit test to check if the scheduled speculative task gets cancelled + * on successful read. + */ + @Test + public void testSpeculativeReadScheduledTaskCancel() throws Exception { + long id = getLedgerToRead(3, 2); + int timeout = 1000; + BookKeeper bkspec = createClient(timeout); + LedgerHandle l = bkspec.openLedger(id, digestType, passwd); + PendingReadOp op = null; + try { + op = new PendingReadOp(l, bkspec.getClientCtx(), 0, 5, false); + op.initiate(); + op.future().get(); + } finally { + assertNull("Speculative Read tasks must be null", op.getSpeculativeTask()); + } + } + /** * Unit test for the speculative read scheduling method. */ @@ -353,7 +373,7 @@ public void testSpeculativeReadScheduling() throws Exception { } Thread.sleep(1000); } - assertTrue("Request should be done", req0.isComplete()); + assertTrue("Request should be done", req.isComplete()); } } From cfe63e27552beb08a112acb06d89fa81b26ee5b7 Mon Sep 17 00:00:00 2001 From: Qi Wang Date: Mon, 10 Sep 2018 13:14:08 -0700 Subject: [PATCH 0025/1642] BookieWatcherImpl doesn't initCause when throwing MetaStoreException Descriptions of the changes in this PR: ### Motivation when BookieWatcherImpl throws MetaStoreException, it doesn't provide the cause. so when the exception is logged, we can't tell the actual failure cause. Author: Qi Wang Reviewers: Enrico Olivelli , Sijie Guo This closes #1637 from codingwangqi/fill_cause_in_exception --- .../org/apache/bookkeeper/client/BKException.java | 8 ++++++++ .../apache/bookkeeper/client/BookieWatcherImpl.java | 3 ++- .../org/apache/bookkeeper/client/api/BKException.java | 11 +++++++++++ 3 files changed, 21 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java index 8ab563be60c..d6304927e5e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java @@ -47,6 +47,10 @@ public abstract class BKException extends org.apache.bookkeeper.client.api.BKExc super(code); } + BKException(int code, Throwable cause) { + super(code, cause); + } + /** * Create an exception from an error code. * @param code return error code @@ -299,6 +303,10 @@ public static class MetaStoreException extends BKException { public MetaStoreException() { super(Code.MetaStoreException); } + + public MetaStoreException(Throwable cause) { + super(Code.MetaStoreException, cause); + } } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index ff707f0f995..2941a41fa1f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -62,7 +62,8 @@ class BookieWatcherImpl implements BookieWatcher { log.error("Interrupted reading bookie list : ", cause); return new BKInterruptedException(); } else { - return new MetaStoreException(); + MetaStoreException mse = new MetaStoreException(cause); + return mse; } }; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java index 09bb8f36098..cc28158938a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java @@ -59,6 +59,17 @@ public BKException(int code) { this.code = code; } + /** + * Create a new exception with the cause. + * + * @param code exception code + * @param cause the exception cause + */ + public BKException(int code, Throwable cause) { + super(getMessage(code), cause); + this.code = code; + } + /** * Get the return code for the exception. * From 020d8545df45895d1d2da8c7003fd239cd70e799 Mon Sep 17 00:00:00 2001 From: cguttapalem Date: Mon, 10 Sep 2018 13:20:45 -0700 Subject: [PATCH 0026/1642] Remove unnecessary log statement in LedgerDirsManager.getWritableLedgerDirs Descriptions of the changes in this PR: ### Motivation - when Bookie reaches diskusage threshold, LedgerDirsMonitor for every diskCheckInterval it logs NoWritableLedgerDirException call stack and message "All ledger directories are non writable". ### Changes So remove unneccessary log statement in LedgerDirsManager.getWritableLedgerDirs and let the caller deal with the thrown exception. Author: cguttapalem Reviewers: Andrey Yegorov , Sijie Guo This closes #1670 from reddycharan/fixdirsmonitorlog --- .../java/org/apache/bookkeeper/bookie/LedgerDirsManager.java | 1 - .../java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java index cdaa66814b9..5fb8215f093 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java @@ -157,7 +157,6 @@ public List getWritableLedgerDirs() String errMsg = "All ledger directories are non writable"; NoWritableLedgerDirException e = new NoWritableLedgerDirException( errMsg); - LOG.error(errMsg, e); throw e; } return writableLedgerDirectories; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java index 4ef02fae4da..fedebb74222 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java @@ -108,6 +108,7 @@ private void check() { // bookie cannot get writable dir but considered to be writable ldm.getWritableLedgerDirs(); } catch (NoWritableLedgerDirException e) { + LOG.warn("LedgerDirsMonitor check process: All ledger directories are non writable"); boolean highPriorityWritesAllowed = true; try { // disk check can be frequent, so disable 'loggingNoWritable' to avoid log flooding. From 9c2dc606d69f75d2c6c797af35c49196b53556f6 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 11 Sep 2018 17:25:23 +0200 Subject: [PATCH 0027/1642] Use useradd -l option for docker scripts This is a workaround for the Docker issue https://github.com/moby/moby/issues/5419 If you run the scripts with an very large uid the script hangs and it fills up the disk which contains /var/lib/docker Author: Enrico Olivelli Reviewers: Sijie Guo This closes #1673 from eolivelli/fix/docker-large-uid --- dev/release/000-run-docker.sh | 2 +- site/docker/run.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/release/000-run-docker.sh b/dev/release/000-run-docker.sh index 255a6a210ac..edc5a948a79 100755 --- a/dev/release/000-run-docker.sh +++ b/dev/release/000-run-docker.sh @@ -52,7 +52,7 @@ fi docker build -t "${IMAGE_NAME}-${USER_NAME}" - < Date: Tue, 11 Sep 2018 19:57:26 +0200 Subject: [PATCH 0028/1642] Release script: Use realpath() to map to current user home This change enables to run the release scripts on linux boxes in which the user's home is not at /home/username Author: Enrico Olivelli Reviewers: Sijie Guo This closes #1674 from eolivelli/fix/better-userland-mapping --- dev/release/000-run-docker.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/release/000-run-docker.sh b/dev/release/000-run-docker.sh index edc5a948a79..66a94ffff71 100755 --- a/dev/release/000-run-docker.sh +++ b/dev/release/000-run-docker.sh @@ -107,8 +107,8 @@ docker run -i -t \ --rm=true \ -w ${BOOKKEEPER_ROOT} \ -u "${USER}" \ - -v "${BOOKKEEPER_ROOT}:${BOOKKEEPER_ROOT}" \ - -v "${LOCAL_HOME}:/home/${USER_NAME}" \ + -v "${BOOKKEEPER_ROOT}:$(realpath $BOOKKEEPER_ROOT)" \ + -v "${LOCAL_HOME}:$(realpath ~)" \ -e VERSION=${VERSION} \ -e MAJOR_VERSION=${MAJOR_VERSION} \ -e NEXT_VERSION=${NEXT_VERSION} \ From d341cfa920be5bae37b009cee556b90a0893500c Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 12 Sep 2018 10:00:53 +0200 Subject: [PATCH 0029/1642] Release script: swap mapping of directories The script assumed that the layout of /home on the host machine is the same as in the container. This change makes the script work even on machines in which the HOME is /home/xxxx/username and not simply /home/username Author: Enrico Olivelli Reviewers: Sijie Guo This closes #1675 from eolivelli/fix/invert-mapping --- dev/release/000-run-docker.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/release/000-run-docker.sh b/dev/release/000-run-docker.sh index 66a94ffff71..d63e2221b3b 100755 --- a/dev/release/000-run-docker.sh +++ b/dev/release/000-run-docker.sh @@ -107,8 +107,8 @@ docker run -i -t \ --rm=true \ -w ${BOOKKEEPER_ROOT} \ -u "${USER}" \ - -v "${BOOKKEEPER_ROOT}:$(realpath $BOOKKEEPER_ROOT)" \ - -v "${LOCAL_HOME}:$(realpath ~)" \ + -v "$(realpath $BOOKKEEPER_ROOT):${BOOKKEEPER_ROOT}" \ + -v "$(realpath ~):${LOCAL_HOME}" \ -e VERSION=${VERSION} \ -e MAJOR_VERSION=${MAJOR_VERSION} \ -e NEXT_VERSION=${NEXT_VERSION} \ From 94944c7ceacf69db7448cd79cbac5e291e944af8 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 14 Sep 2018 08:47:26 +0200 Subject: [PATCH 0030/1642] Utility to dump/restore ledger metadata to/from file Dumping will dump the serialized metadata to a file. Restoring will parse the metadata from the file and write it to the metadata store. Also changes the LedgerIdFormatter to default to long formatter (it was UUID before, and we don't even currently support UUID ledger ids). This fixes #1604 Author: Ivan Kelly Author: Enrico Olivelli Author: cguttapalem Author: Qi Wang Reviewers: Enrico Olivelli , Sijie Guo , Jia Zhai This closes #1667 from ivankelly/dump-metadata-to-from-file --- .../apache/bookkeeper/bookie/BookieShell.java | 55 +++++++++---------- .../conf/AbstractConfiguration.java | 2 +- .../bookkeeper/util/LedgerIdFormatter.java | 4 +- .../integration/TestBookieShellCluster.java | 43 +++++++++++++++ 4 files changed, 73 insertions(+), 31 deletions(-) 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 8eebce58be8..a6e0f44bc43 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 @@ -25,9 +25,9 @@ import static org.apache.bookkeeper.tools.cli.helpers.CommandHelpers.getBookieSocketAddrStringRepresentation; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.util.concurrent.AbstractFuture; import com.google.common.util.concurrent.UncheckedExecutionException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; @@ -105,6 +105,7 @@ import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.replication.ReplicationException; @@ -1084,28 +1085,6 @@ void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { } } - static class ReadMetadataCallback extends AbstractFuture - implements GenericCallback { - final long ledgerId; - - ReadMetadataCallback(long ledgerId) { - this.ledgerId = ledgerId; - } - - long getLedgerId() { - return ledgerId; - } - - @Override - public void operationComplete(int rc, LedgerMetadata result) { - if (rc != 0) { - setException(BKException.create(rc)); - } else { - set(result); - } - } - } - /** * Print the metadata for a ledger. */ @@ -1115,6 +1094,8 @@ class LedgerMetadataCmd extends MyCommand { LedgerMetadataCmd() { super(CMD_LEDGERMETADATA); lOpts.addOption("l", "ledgerid", true, "Ledger ID"); + lOpts.addOption("dumptofile", true, "Dump metadata for ledger, to a file"); + lOpts.addOption("restorefromfile", true, "Restore metadata for ledger, from a file"); } @Override @@ -1125,11 +1106,29 @@ public int runCmd(CommandLine cmdLine) throws Exception { return -1; } + if (cmdLine.hasOption("dumptofile") && cmdLine.hasOption("restorefromfile")) { + System.err.println("Only one of --dumptofile and --restorefromfile can be specified"); + return -2; + } runFunctionWithLedgerManagerFactory(bkConf, mFactory -> { try (LedgerManager m = mFactory.newLedgerManager()) { - ReadMetadataCallback cb = new ReadMetadataCallback(lid); - m.readLedgerMetadata(lid, cb); - printLedgerMetadata(lid, cb.get(), true); + if (cmdLine.hasOption("dumptofile")) { + GenericCallbackFuture cb = new GenericCallbackFuture<>(); + m.readLedgerMetadata(lid, cb); + Files.write(FileSystems.getDefault().getPath(cmdLine.getOptionValue("dumptofile")), + cb.join().serialize()); + } else if (cmdLine.hasOption("restorefromfile")) { + byte[] serialized = Files.readAllBytes( + FileSystems.getDefault().getPath(cmdLine.getOptionValue("restorefromfile"))); + LedgerMetadata md = LedgerMetadata.parseConfig(serialized, Version.NEW, Optional.absent()); + GenericCallbackFuture cb = new GenericCallbackFuture<>(); + m.createLedgerMetadata(lid, md, cb); + cb.join(); + } else { + GenericCallbackFuture cb = new GenericCallbackFuture<>(); + m.readLedgerMetadata(lid, cb); + printLedgerMetadata(lid, cb.get(), true); + } } catch (Exception e) { throw new UncheckedExecutionException(e); } @@ -1141,12 +1140,12 @@ public int runCmd(CommandLine cmdLine) throws Exception { @Override String getDescription() { - return "Print the metadata for a ledger."; + return "Print the metadata for a ledger, or optionally dump to a file."; } @Override String getUsage() { - return "ledgermetadata -ledgerid "; + return "ledgermetadata -ledgerid [--dump-to-file FILENAME|--restore-from-file FILENAME]"; } @Override 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 9f9547cef50..b72b0ba5e7b 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 @@ -105,7 +105,7 @@ public abstract class AbstractConfiguration // Default formatter classes protected static final Class DEFAULT_ENTRY_FORMATTER = StringEntryFormatter.class; protected static final Class DEFAULT_LEDGERID_FORMATTER = - LedgerIdFormatter.UUIDLedgerIdFormatter.class; + LedgerIdFormatter.LongLedgerIdFormatter.class; /** * This list will be passed to {@link SSLEngine#setEnabledCipherSuites(java.lang.String[]) }. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java index ec52cdf5cc8..f037f6d6d5e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java @@ -62,8 +62,8 @@ public static LedgerIdFormatter newLedgerIdFormatter(AbstractConfiguration co formatter = ReflectionUtils.newInstance(ledgerIdFormatterClass); } catch (Exception e) { LOG.warn("No formatter class found", e); - LOG.warn("Using Default UUID Formatter."); - formatter = new UUIDLedgerIdFormatter(); + LOG.warn("Using Default Long Formatter."); + formatter = new LongLedgerIdFormatter(); } return formatter; } diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java index 47fbc554715..c68a3e17122 100644 --- a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java +++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import com.github.dockerjava.api.DockerClient; +import java.util.concurrent.ExecutionException; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException; @@ -182,4 +183,46 @@ public void test101_RegenerateIndex() throws Exception { } } + @Test + public void test102_DumpRestoreMetadata() throws Exception { + String zookeeper = String.format("zk+hierarchical://%s/ledgers", + BookKeeperClusterUtils.zookeeperConnectString(docker)); + int numEntries = 100; + + try (BookKeeper bk = BookKeeper.newBuilder( + new ClientConfiguration().setMetadataServiceUri(zookeeper)).build()) { + log.info("Writing entries"); + long ledgerId = writeNEntries(bk, numEntries, 1); + + log.info("Dumping ledger metadata to file"); + String bookie = BookKeeperClusterUtils.getAnyBookie(); + String dumpFile = String.format("/tmp/ledger-%d-%d", ledgerId, System.nanoTime()); + DockerUtils.runCommand(docker, bookie, + bkScript, "shell", "ledgermetadata", + "--ledgerid", String.valueOf(ledgerId), + "--dumptofile", dumpFile); + + log.info("Delete the ledger metadata"); + bk.newDeleteLedgerOp().withLedgerId(ledgerId).execute().get(); + + // hopefully ledger gc doesn't kick in + log.info("Verify that we cannot open ledger"); + try { + validateNEntries(bk, ledgerId, numEntries); + Assert.fail("Shouldn't have been able to find anything"); + } catch (ExecutionException ee) { + Assert.assertEquals(ee.getCause().getClass(), BKException.BKNoSuchLedgerExistsException.class); + } + + log.info("Restore the ledger metadata"); + DockerUtils.runCommand(docker, bookie, + bkScript, "shell", "ledgermetadata", + "--ledgerid", String.valueOf(ledgerId), + "--restorefromfile", dumpFile); + + log.info("Validate that we can read back, after regeneration"); + validateNEntries(bk, ledgerId, numEntries); + } + } + } From 0c9e3a50e2ee5a6ef59fdab0cdf863825bbd0e32 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 14 Sep 2018 10:33:17 -0700 Subject: [PATCH 0031/1642] [DEV] make merge script work with python 3 Descriptions of the changes in this PR: ### Motivation make sure the script works for python 3 as well. ### Changes since python2 and python3 have different syntax and library, make a copy of the merge script for python3. Author: Reviewers: Enrico Olivelli This closes #1677 from sijie/make_merge_script_works_for_37 --- dev/bk-merge-pr.py | 114 +++---- dev/bk-merge-pr3.py | 746 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 803 insertions(+), 57 deletions(-) create mode 100755 dev/bk-merge-pr3.py diff --git a/dev/bk-merge-pr.py b/dev/bk-merge-pr.py index 2c698908b9f..4dec84d6d15 100755 --- a/dev/bk-merge-pr.py +++ b/dev/bk-merge-pr.py @@ -69,11 +69,11 @@ def get_json(url, preview_api = False): return json.load(urllib2.urlopen(request)) except urllib2.HTTPError as e: if "X-RateLimit-Remaining" in e.headers and e.headers["X-RateLimit-Remaining"] == '0': - print "Exceeded the GitHub API rate limit; see the instructions in " + \ + print("Exceeded the GitHub API rate limit; see the instructions in " + \ "bk-merge-pr.py to configure an OAuth token for making authenticated " + \ - "GitHub requests." + "GitHub requests.") else: - print "Unable to fetch URL, exiting: %s" % url + print("Unable to fetch URL, exiting: %s" % url) sys.exit(-1) def post_json(url, data): @@ -83,11 +83,11 @@ def post_json(url, data): return json.load(urllib2.urlopen(request)) except urllib2.HTTPError as e: if "X-RateLimit-Remaining" in e.headers and e.headers["X-RateLimit-Remaining"] == '0': - print "Exceeded the GitHub API rate limit; see the instructions in " + \ + print("Exceeded the GitHub API rate limit; see the instructions in " + \ "bk-merge-pr.py to configure an OAuth token for making authenticated " + \ - "GitHub requests." + "GitHub requests.") else: - print "Unable to fetch URL, exiting: %s - %s" % (url, e) + print("Unable to fetch URL, exiting: %s - %s" % (url, e)) sys.exit(-1) def put_json(url, data): @@ -98,23 +98,23 @@ def put_json(url, data): return json.load(urllib2.urlopen(request)) except urllib2.HTTPError as e: if "X-RateLimit-Remaining" in e.headers and e.headers["X-RateLimit-Remaining"] == '0': - print "Exceeded the GitHub API rate limit; see the instructions in " + \ + print("Exceeded the GitHub API rate limit; see the instructions in " + \ "bk-merge-pr.py to configure an OAuth token for making authenticated " + \ - "GitHub requests." + "GitHub requests.") else: - print "Unable to fetch URL, exiting: %s - %s" % (url, e) - print e + print("Unable to fetch URL, exiting: %s - %s" % (url, e)) + print(e) sys.exit(-1) def fail(msg): - print msg + print(msg) clean_up() sys.exit(-1) def run_cmd(cmd): - print cmd + print(cmd) if isinstance(cmd, list): return subprocess.check_output(cmd) else: @@ -128,13 +128,13 @@ def continue_maybe(prompt): def clean_up(): if original_head != get_current_branch(): - print "Restoring head pointer to %s" % original_head + print("Restoring head pointer to %s" % original_head) run_cmd("git checkout %s" % original_head) branches = run_cmd("git branch").replace(" ", "").split("\n") for branch in filter(lambda x: x.startswith(TEMP_BRANCH_PREFIX), branches): - print "Deleting local branch %s" % branch + print("Deleting local branch %s" % branch) run_cmd("git branch -D %s" % branch) def get_current_branch(): @@ -369,9 +369,9 @@ def check_ci_status(pr): comments = get_json(pr["comments_url"]) ignore_ci_comments = [c for c in comments if c["body"].upper() == "IGNORE CI"] if len(ignore_ci_comments) > 0: - print "\n\nWARNING: The PR has not passed CI (state is %s)" % (state) \ + print("\n\nWARNING: The PR has not passed CI (state is %s)" % (state) \ + ", but this has been overridden by %s. \n" % (ignore_ci_comments[0]["user"]["login"]) \ - + "Proceed at your own peril!\n\n" + + "Proceed at your own peril!\n\n") else: check_individual_ci_status(ci_status, comments) @@ -411,14 +411,14 @@ def check_individual_ci_status(ci_status, comments): # all ci passed except integration tests ignore_it_ci_comments = [c for c in comments if c["body"].upper() == "IGNORE IT CI"] if len(ignore_it_ci_comments) > 0: - print "\n\nWARNING: The PR has not passed integration tests CI" \ + print("\n\nWARNING: The PR has not passed integration tests CI" \ + ", but this has been overridden by %s. \n" % (ignore_it_ci_comments[0]["user"]["login"]) \ - + "Proceed at your own peril!\n\n" + + "Proceed at your own peril!\n\n") else: fail("The PR has not passed integration tests CI") elif len(ci_failures) != 0 or len(ci_integration_test_failures) != 0: fail_msg = "The PR has not passed CI:\n" - print "" + print("") for status in ci_failures: fail_msg += "\t %s = %s\n" % (status["context"], status["state"]) for status in ci_integration_test_failures: @@ -426,16 +426,16 @@ def check_individual_ci_status(ci_status, comments): fail(fail_msg) def ask_release_for_github_issues(branch, labels): - print "=== Add release to github issues ===" + print("=== Add release to github issues ===") while True: fix_releases = ask_for_labels("release/%s" % branch, labels, []) if len(fix_releases) != 1: - print "Please choose only one release to add for branch '%s'." % branch + print("Please choose only one release to add for branch '%s'." % branch) continue - print "=== Apply following releases to github issues ==" - print "Fix Releases: %s" % ', '.join(fix_releases) - print "" + print("=== Apply following releases to github issues ==") + print("Fix Releases: %s" % ', '.join(fix_releases)) + print("") if raw_input("Would you like to add these releases to github issues? (y/n): ") == "y": break @@ -446,12 +446,12 @@ def ask_updates_for_github_issues(milestones, labels, issue_labels, milestone_re fix_milestone, fix_milestone_number, fix_areas, fix_types = \ get_updates_for_github_issues(milestones, labels, issue_labels, milestone_required) - print "=== Apply following milestone, area, type to github issues ==" - print "Fix Types: %s" % ', '.join(fix_types) - print "Fix Areas: %s" % ', '.join(fix_areas) + print("=== Apply following milestone, area, type to github issues ==") + print("Fix Types: %s" % ', '.join(fix_types)) + print("Fix Areas: %s" % ', '.join(fix_areas)) if milestone_required: - print "Fix Milestone: %s" % fix_milestone - print "" + print("Fix Milestone: %s" % fix_milestone) + print("") if raw_input("Would you like to update github issues with these labels? (y/n): ") == "y": break @@ -475,7 +475,7 @@ def get_updates_for_github_issues(milestones, labels, issue_labels, milestone_re elif fix_milestone in milestone_map: break else: - print "Invalid milestone: %s." % fix_milestone + print("Invalid milestone: %s." % fix_milestone) fix_milestone_number = milestone_map[fix_milestone] # get area @@ -494,19 +494,19 @@ def ask_for_labels(prefix, labels, issue_labels): % (prefix, ', '.join(filtered_labels).strip(), ', '.join(issue_filtered_labels).strip())) if fix_labels == "": if not issue_filtered_labels: - print "Please specify a '%s' label to close the issue!" % prefix + print("Please specify a '%s' label to close the issue!" % prefix) continue else: fix_labels = issue_filtered_labels break fix_labels = fix_labels.replace(" ", "").split(",") if not fix_labels: - print "Please specify a '%s' label to close the issue!" % prefix + print("Please specify a '%s' label to close the issue!" % prefix) continue invalid_label = False for label in fix_labels: if label not in filtered_labels: - print "Invalid '%s' label: %s." % (prefix, label) + print("Invalid '%s' label: %s." % (prefix, label)) invalid_label = True break if invalid_label: @@ -581,12 +581,12 @@ def main(): global original_head if not GITHUB_OAUTH_KEY: - print "OAuth key is needed for merging bookkeeper pull requests." - print "If environment variable 'GITHUB_OAUTH_KEY' is not defined," - print "then requests will be unauthenticated." - print "You can create an OAuth key at https://github.com/settings/tokens" - print "and set it to the environment variable 'GITHUB_OAUTH_KEY'." - print "(This token only needs the 'public_repo' scope permissions)" + print("OAuth key is needed for merging bookkeeper pull requests.") + print("If environment variable 'GITHUB_OAUTH_KEY' is not defined,") + print("then requests will be unauthenticated.") + print("You can create an OAuth key at https://github.com/settings/tokens") + print("and set it to the environment variable 'GITHUB_OAUTH_KEY'.") + print("(This token only needs the 'public_repo' scope permissions)") exit(-1) # 0. get the current state so we can go back @@ -618,16 +618,16 @@ def main(): # Decide whether to use the modified title or not modified_title, github_issue_ids = standardize_issue_ref(commit_title) if modified_title != commit_title: - print "I've re-written the title as follows to match the standard format:" - print "Original: %s" % commit_title - print "Modified: %s" % modified_title + print("I've re-written the title as follows to match the standard format:") + print("Original: %s" % commit_title) + print("Modified: %s" % modified_title) result = raw_input("Would you like to use the modified title? (y/n): ") if result.lower() == "y": commit_title = modified_title - print "Using modified title:" + print("Using modified title:") else: - print "Using original title:" - print commit_title + print("Using original title:") + print(commit_title) body = pr["body"] modified_body = "" @@ -635,19 +635,19 @@ def main(): if line.startswith('>'): continue modified_body = modified_body + line + "\n" - modified_body = modified_body.rstrip("\n") + modified_body = modified_body[:-1] if modified_body != body: - print "I've re-written the body as follows to match the standard formats:" - print "Original: " - print body - print "Modified: " - print modified_body + print("I've re-written the body as follows to match the standard formats:") + print("Original: ") + print(body) + print("Modified: ") + print(modified_body) result = raw_input("Would you like to use the modified body? (y/n): ") if result.lower() == "y": body = modified_body - print "Using modified body." + print("Using modified body.") else: - print "Using original body." + print("Using original body.") target_ref = pr["base"]["ref"] user_login = pr["user"]["login"] @@ -698,13 +698,13 @@ def main(): merge_hash = merge_commits[0]["commit_id"] message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] - print "Pull request %s has already been merged, assuming you want to backport" % pr_num + print("Pull request %s has already been merged, assuming you want to backport" % pr_num) commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', "%s^{commit}" % merge_hash]).strip() != "" if not commit_is_downloaded: fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) - print "Found commit %s:\n%s" % (merge_hash, message) + print("Found commit %s:\n%s" % (merge_hash, message)) cherry_pick(pr_num, merge_hash, ask_for_branch(latest_branch)) sys.exit(0) @@ -714,8 +714,8 @@ def main(): "You may need to rebase the PR." fail(msg) - print ("\n=== Pull Request #%s ===" % pr_num) - print ("PR title\t%s\nCommit title\t%s\nSource\t\t%s\nTarget\t\t%s\nURL\t\t%s" % ( + print("\n=== Pull Request #%s ===" % pr_num) + print("PR title\t%s\nCommit title\t%s\nSource\t\t%s\nTarget\t\t%s\nURL\t\t%s" % ( pr_title, commit_title, pr_repo_desc, target_ref, url)) continue_maybe("Proceed with merging pull request #%s?" % pr_num) diff --git a/dev/bk-merge-pr3.py b/dev/bk-merge-pr3.py new file mode 100755 index 00000000000..4f755f9cfb9 --- /dev/null +++ b/dev/bk-merge-pr3.py @@ -0,0 +1,746 @@ +#!/usr/bin/env python + +# +# 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. +# + +# Utility for creating well-formed pull request merges and pushing them to Apache. This script is a modified version +# of the one created by the Spark project (https://github.com/apache/spark/blob/master/dev/merge_spark_pr.py). +# +# Usage: ./bk-merge-pr.py (see config env vars below) +# +# This utility assumes you already have local a bookkeeper git folder and that you +# have added remotes corresponding to the github apache bookkeeper repo. + +import json +import os +import re +import subprocess +import sys +from urllib.request import urlopen, Request +from urllib.error import HTTPError + +PROJECT_NAME = "bookkeeper" + +CAPITALIZED_PROJECT_NAME = PROJECT_NAME.upper() +GITHUB_ISSUES_NAME = "issue".upper() + +# Location of the local git repository +REPO_HOME = os.environ.get("%s_HOME" % CAPITALIZED_PROJECT_NAME, os.getcwd()) +# Remote name which points to the GitHub site +PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache") +# Github API page size +GITHUB_PAGE_SIZE = os.environ.get("GH_PAGE_SIZE", "100") +# OAuth key used for issuing requests against the GitHub API. If this is not defined, then requests +# will be unauthenticated. You should only need to configure this if you find yourself regularly +# exceeding your IP's unauthenticated request rate limit. You can create an OAuth key at +# https://github.com/settings/tokens. This script only requires the "public_repo" scope. +GITHUB_OAUTH_KEY = os.environ.get("GITHUB_OAUTH_KEY") + +GITHUB_USER = os.environ.get("GITHUB_USER", "apache") +GITHUB_BASE = "https://github.com/%s/%s/pull" % (GITHUB_USER, PROJECT_NAME) +GITHUB_API_URL = "https://api.github.com" +GITHUB_API_BASE = "%s/repos/%s/%s" % (GITHUB_API_URL, GITHUB_USER, PROJECT_NAME) +# Prefix added to temporary branches +TEMP_BRANCH_PREFIX = "PR_TOOL" +RELEASE_BRANCH_PREFIX = "branch-" + +DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "0.9.1.0") + +def get_json(url, preview_api = False): + try: + request = Request(url) + if GITHUB_OAUTH_KEY: + request.add_header('Authorization', 'token %s' % GITHUB_OAUTH_KEY) + if preview_api: + request.add_header('Accept', 'application/vnd.github.black-cat-preview+json') + return json.loads(urlopen(request).read()) + except HTTPError as e: + if "X-RateLimit-Remaining" in e.headers and e.headers["X-RateLimit-Remaining"] == '0': + print("Exceeded the GitHub API rate limit; see the instructions in " + \ + "bk-merge-pr.py to configure an OAuth token for making authenticated " + \ + "GitHub requests.") + else: + print("Unable to fetch URL, exiting: %s" % url) + sys.exit(-1) + +def post_json(url, data): + try: + request = Request(url, data.encode(encoding='utf-8'), { 'Content-Type': 'application/json' }) + request.add_header('Authorization', 'token %s' % GITHUB_OAUTH_KEY) + return json.loads(urlopen(request).read()) + except HTTPError as e: + if "X-RateLimit-Remaining" in e.headers and e.headers["X-RateLimit-Remaining"] == '0': + print("Exceeded the GitHub API rate limit; see the instructions in " + \ + "bk-merge-pr.py to configure an OAuth token for making authenticated " + \ + "GitHub requests.") + else: + print("Unable to fetch URL, exiting: %s - %s" % (url, e)) + sys.exit(-1) + +def put_json(url, data): + try: + request = Request(url, data.encode(encoding='utf-8'), { 'Content-Type': 'application/json' }) + request.get_method = lambda: 'PUT' + request.add_header('Authorization', 'token %s' % GITHUB_OAUTH_KEY) + return json.loads(urlopen(request).read()) + except HTTPError as e: + if "X-RateLimit-Remaining" in e.headers and e.headers["X-RateLimit-Remaining"] == '0': + print("Exceeded the GitHub API rate limit; see the instructions in " + \ + "bk-merge-pr.py to configure an OAuth token for making authenticated " + \ + "GitHub requests.") + else: + print("Unable to fetch URL, exiting: %s - %s" % (url, e)) + print(e) + sys.exit(-1) + + +def fail(msg): + print(msg) + clean_up() + sys.exit(-1) + + +def run_cmd(cmd): + print(cmd) + if isinstance(cmd, list): + return subprocess.check_output(cmd).decode(encoding='utf-8') + else: + return subprocess.check_output(cmd.split(" ")).decode(encoding='utf-8') + + +def continue_maybe(prompt): + result = input("\n%s (y/n): " % prompt) + if result.lower() != "y": + fail("Okay, exiting") + +def clean_up(): + if original_head != get_current_branch(): + print("Restoring head pointer to %s" % original_head) + run_cmd("git checkout %s" % original_head) + + branches = run_cmd("git branch").replace(" ", "").split("\n") + + for branch in list(filter(lambda x: x.startswith(TEMP_BRANCH_PREFIX), branches)): + print("Deleting local branch %s" % branch) + run_cmd("git branch -D %s" % branch) + +def get_current_branch(): + return run_cmd("git rev-parse --abbrev-ref HEAD").replace("\n", "") + +def get_milestones(): + return get_json("https://api.github.com/repos/%s/%s/milestones?state=open&sort=due_on&direction=asc" % (GITHUB_USER, PROJECT_NAME)) + +def get_all_labels(): + result = get_json("https://api.github.com/repos/%s/%s/labels?per_page=%s" % (GITHUB_USER, PROJECT_NAME, GITHUB_PAGE_SIZE)) + return list(map(lambda x: x['name'], result)) + +# merge the requested PR and return the merge hash +def merge_pr(pr_num, target_ref, title, body, default_pr_reviewers, pr_repo_desc): + pr_branch_name = "%s_MERGE_PR_%s" % (TEMP_BRANCH_PREFIX, pr_num) + run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) + + commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%an <%ae>']).split("\n") + distinct_authors = sorted(set(commit_authors), + key=lambda x: commit_authors.count(x), reverse=True) + primary_author = input( + "Enter primary author in the format of \"name \" [%s]: " % + distinct_authors[0]) + if primary_author == "": + primary_author = distinct_authors[0] + + reviewers = input("Enter reviewers [%s]: " % default_pr_reviewers).strip() + if reviewers == '': + reviewers = default_pr_reviewers + + commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%h [%an] %s']).split("\n") + + if len(commits) > 1: + result = input("List pull request commits in squashed commit message? (y/n): ") + if result.lower() == "y": + should_list_commits = True + else: + should_list_commits = False + else: + should_list_commits = False + + merge_message_flags = [] + + if body is not None: + # We remove @ symbols from the body to avoid triggering e-mails + # to people every time someone creates a public fork of the project. + merge_message_flags += [body.replace("@", "")] + + authors = "\n".join(["Author: %s" % a for a in distinct_authors]) + + merge_message_flags += [authors] + + if (reviewers != ""): + merge_message_flags += ["Reviewers: %s" % reviewers] + + # The string "Closes #%s" string is required for GitHub to correctly close the PR + close_line = "This closes #%s from %s" % (pr_num, pr_repo_desc) + # Find the github issues to close + github_issues = re.findall("#[0-9]{3,6}", title) + + if len(github_issues) != 0: + for issue_id in github_issues: + close_line += ", closes %s" % (issue_id) + + if should_list_commits: + close_line += " and squashes the following commits:" + merge_message_flags += [close_line] + + if should_list_commits: + merge_message_flags += ["\n".join(commits)] + + pr_sha = run_cmd("git rev-parse %s" % pr_branch_name).rstrip() + + merge_url = get_github_issue_merge_url(pr_num) + data = json.dumps({ + 'commit_title': title, + 'commit_message': "\n\n".join(merge_message_flags), + 'sha': pr_sha, + 'merge_method': 'squash' + }, indent = 4) + + continue_maybe("Merge Pull Request (%s) with following details:\n%s" % ( + pr_num, data)) + + resp = put_json(merge_url, data) + + merge_hash = resp['sha'] + merge_log = title + '\n' + "\n\n".join(merge_message_flags) + clean_up() + print("Pull request #%s merged!" % pr_num) + print("Merge hash: %s" % merge_hash) + return merge_hash, merge_log + +def ask_for_branch(default_branch): + pick_ref = input("Enter a branch name [%s]: " % default_branch) + if pick_ref == "": + pick_ref = default_branch + return pick_ref + +def cherry_pick(pr_num, merge_hash, pick_ref): + pick_branch_name = "%s_PICK_PR_%s_%s" % (TEMP_BRANCH_PREFIX, pr_num, pick_ref.upper()) + + run_cmd("git fetch %s %s:%s" % (PR_REMOTE_NAME, pick_ref, pick_branch_name)) + run_cmd("git checkout %s" % pick_branch_name) + + try: + run_cmd("git cherry-pick -sx %s" % merge_hash) + except Exception as e: + msg = "Error cherry-picking: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and finish the cherry-pick. Finished?" + continue_maybe(msg) + + continue_maybe("Pick complete (local ref %s). Push to %s?" % ( + pick_branch_name, PR_REMOTE_NAME)) + + try: + run_cmd('git push %s %s:%s' % (PR_REMOTE_NAME, pick_branch_name, pick_ref)) + except Exception as e: + clean_up() + fail("Exception while pushing: %s" % e) + + pick_hash = run_cmd("git rev-parse %s" % pick_branch_name)[:8] + clean_up() + + print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) + print("Pick hash: %s" % pick_hash) + return pick_ref + + +def fix_version_from_branch(branch, versions, target_ref): + # Note: Assumes this is a sorted (newest->oldest) list of un-released versions + if branch == target_ref: + versions = list(filter(lambda x: x == DEFAULT_FIX_VERSION, versions)) + if len(versions) > 0: + return versions[0] + else: + return None + else: + versions = list(filter(lambda x: x.startswith(branch), versions)) + if len(versions) > 0: + return versions[-1] + else: + return None + +def standardize_issue_ref(text): + """ + Standardize the github reference commit message prefix to "Issue #XXX: Issue" + + 'ISSUE #376: Script for generating patch for reviews' + """ + github_issue_refs = [] + github_issue_ids = [] + components = [] + + # Extract Github Issue ref(s) + pattern = re.compile(r'(%s[-\s]*([0-9]{3,6}))+' % GITHUB_ISSUES_NAME, re.IGNORECASE) + for ref in pattern.findall(text): + # Add brackets, replace spaces or a dash with ' #', & convert to uppercase + github_issue_refs.append(re.sub(r'[-\s]+', ' #', ref[0].upper())) + text = text.replace(ref[0], '') + github_issue_ids.append(ref[1].upper()) + + # Extract project name component(s): + # Look for alphanumeric chars, spaces, dashes, periods, and/or commas + pattern = re.compile(r'(\[[\w\s,-\.]+\])', re.IGNORECASE) + for component in pattern.findall(text): + components.append(component.upper()) + text = text.replace(component, '') + + # Cleanup any remaining symbols: + pattern = re.compile(r'^\W+(.*)', re.IGNORECASE) + if (pattern.search(text) is not None): + text = pattern.search(text).groups()[0] + + # Assemble full text (github ref(s), module(s), remaining text) + prefix = '' + github_prefix = ' '.join(github_issue_refs).strip() + if github_prefix: + prefix = github_prefix + ": " + clean_text = prefix + ' '.join(components).strip() + " " + text.strip() + + # Replace multiple spaces with a single space, e.g. if no refs and/or components were included + clean_text = re.sub(r'\s+', ' ', clean_text.strip()) + + return clean_text, github_issue_ids + +def get_reviewers(pr_num): + """ + Get a candidate list of reviewers that have commented on the PR with '+1' or 'LGTM' + """ + approval_msgs = ['+1', 'lgtm'] + + pr_comments = get_json("%s/issues/%s/comments" % (GITHUB_API_BASE, pr_num)) + + reviewers_ids = set() + for comment in pr_comments: + for approval_msg in approval_msgs: + if approval_msg in comment['body'].lower(): + reviewers_ids.add(comment['user']['login']) + + approval_review_states = ['approved'] + pr_reviews = get_json('{0}/pulls/{1}/reviews?per_page=100'.format(GITHUB_API_BASE, pr_num), True) + for review in pr_reviews: + for approval_state in approval_review_states: + if approval_state in review['state'].lower(): + reviewers_ids.add(review['user']['login']) + + if len(reviewers_ids) == 0: + fail("No approvals found in this pull request") + + reviewers_emails = [] + for reviewer_id in reviewers_ids: + username = None + useremail = None + user = get_json("%s/users/%s" % (GITHUB_API_URL, reviewer_id)) + if user['email']: + useremail = user['email'].strip() + if user['name']: + username = user['name'].strip() + if username is None: + continue + reviewers_emails.append('{0} <{1}>'.format(username, useremail)) + return ', '.join(reviewers_emails) + +def check_ci_status(pr): + ci_status = get_json("%s/commits/%s/status" % (GITHUB_API_BASE, pr["head"]["sha"])) + state = ci_status["state"] + if state != "success": + comments = get_json(pr["comments_url"]) + ignore_ci_comments = [c for c in comments if c["body"].upper() == "IGNORE CI"] + if len(ignore_ci_comments) > 0: + print("\n\nWARNING: The PR has not passed CI (state is %s)" % (state) \ + + ", but this has been overridden by %s. \n" % (ignore_ci_comments[0]["user"]["login"]) \ + + "Proceed at your own peril!\n\n") + else: + check_individual_ci_status(ci_status, comments) + +def is_check_passed(check): + passed = check["state"] == "success" + if (not passed) and is_jenkins_check(check): + try: + return is_jenkins_passed(check["target_url"]) + except: + fail("failed to fetch the jenkins build status for check '%s'.\nPlease manually check its build status at %s" % (check["context"], check["target_url"])) + return passed + +def is_jenkins_check(check): + return check["context"].startswith("Jenkins:") + +def is_jenkins_passed(url): + jenkins_status = get_json("%sapi/json?tree=result" % (url)) + return "SUCCESS" == jenkins_status['result'] + +def is_integration_test_check(check): + return check["context"] == u"Jenkins: Integration Tests" + +def check_individual_ci_status(ci_status, comments): + ci_failures = [] + ci_integration_test_failures = [] + for status in ci_status["statuses"]: + is_passed = is_check_passed(status) + is_integration_test = is_integration_test_check(status) + + if is_integration_test and (not is_passed): + ci_integration_test_failures.append(status) + else: + if not is_passed: + ci_failures.append(status) + + if len(ci_integration_test_failures) != 0 and len(ci_failures) == 0: + # all ci passed except integration tests + ignore_it_ci_comments = [c for c in comments if c["body"].upper() == "IGNORE IT CI"] + if len(ignore_it_ci_comments) > 0: + print("\n\nWARNING: The PR has not passed integration tests CI" \ + + ", but this has been overridden by %s. \n" % (ignore_it_ci_comments[0]["user"]["login"]) \ + + "Proceed at your own peril!\n\n") + else: + fail("The PR has not passed integration tests CI") + elif len(ci_failures) != 0 or len(ci_integration_test_failures) != 0: + fail_msg = "The PR has not passed CI:\n" + print("") + for status in ci_failures: + fail_msg += "\t %s = %s\n" % (status["context"], status["state"]) + for status in ci_integration_test_failures: + fail_msg += "\t %s = %s\n" % (status["context"], status["state"]) + fail(fail_msg) + +def ask_release_for_github_issues(branch, labels): + print("=== Add release to github issues ===") + while True: + fix_releases = ask_for_labels("release/%s" % branch, labels, []) + if len(fix_releases) != 1: + print("Please choose only one release to add for branch '%s'." % branch) + continue + + print("=== Apply following releases to github issues ==") + print("Fix Releases: %s" % ', '.join(fix_releases)) + print("") + + if input("Would you like to add these releases to github issues? (y/n): ") == "y": + break + return fix_releases + +def ask_updates_for_github_issues(milestones, labels, issue_labels, milestone_required): + while True: + fix_milestone, fix_milestone_number, fix_areas, fix_types = \ + get_updates_for_github_issues(milestones, labels, issue_labels, milestone_required) + + print("=== Apply following milestone, area, type to github issues ==") + print("Fix Types: %s" % ', '.join(fix_types)) + print("Fix Areas: %s" % ', '.join(fix_areas)) + if milestone_required: + print("Fix Milestone: %s" % fix_milestone) + print("") + + if input("Would you like to update github issues with these labels? (y/n): ") == "y": + break + + return fix_milestone, fix_milestone_number, fix_areas, fix_types + +def get_updates_for_github_issues(milestones, labels, issue_labels, milestone_required): + # get milestone + fix_milestone = "" + fix_milestone_number = "" + if milestone_required: + default_milestone_name = milestones[0]['title'] + milestone_list = list(map(lambda x: x['title'], milestones)) + milestone_map = dict((milestone['title'], milestone['number']) for milestone in milestones) + while True: + fix_milestone = input("Choose fix milestone : options are [%s] - default: [%s]: " % (', '.join(milestone_list).strip(), default_milestone_name)) + fix_milestone = fix_milestone.strip() + if fix_milestone == "": + fix_milestone = default_milestone_name + break + elif fix_milestone in milestone_map: + break + else: + print("Invalid milestone: %s." % fix_milestone) + fix_milestone_number = milestone_map[fix_milestone] + + # get area + fix_areas = ask_for_labels("area/", labels, issue_labels) + + # get types + fix_types = ask_for_labels("type/", labels, issue_labels) + + return fix_milestone, fix_milestone_number, fix_areas, fix_types + +def ask_for_labels(prefix, labels, issue_labels): + issue_filtered_labels = list(map(lambda l: l.split('/')[1], filter(lambda x: x.startswith(prefix), issue_labels))) + filtered_labels = list(map(lambda l: l.split('/')[1], filter(lambda x: x.startswith(prefix), labels))) + while True: + fix_labels = input("Choose label '%s' - options are: [%s] - default: [%s] (comma separated): " + % (prefix, ', '.join(filtered_labels).strip(), ', '.join(issue_filtered_labels).strip())) + if fix_labels == "": + if not issue_filtered_labels: + print("Please specify a '%s' label to close the issue!" % prefix) + continue + else: + fix_labels = issue_filtered_labels + break + fix_labels = fix_labels.replace(" ", "").split(",") + if not fix_labels: + print("Please specify a '%s' label to close the issue!" % prefix) + continue + invalid_label = False + for label in fix_labels: + if label not in filtered_labels: + print("Invalid '%s' label: %s." % (prefix, label)) + invalid_label = True + break + if invalid_label: + continue + else: + break + return fix_labels + +def get_github_issue_url(github_issue_id): + return "https://api.github.com/repos/%s/%s/issues/%s" % (GITHUB_USER, PROJECT_NAME, github_issue_id) + +def get_github_issue_merge_url(github_issue_id): + return "https://api.github.com/repos/%s/%s/pulls/%s/merge" % (GITHUB_USER, PROJECT_NAME, github_issue_id) + +def get_assignees_url(github_issue_id): + return "https://api.github.com/repos/%s/%s/issues/%s/assignees" % (GITHUB_USER, PROJECT_NAME, github_issue_id) + +def get_github_issue_labels(github_issue_id): + url = "https://api.github.com/repos/%s/%s/issues/%s/labels" % (GITHUB_USER, PROJECT_NAME, github_issue_id) + result = get_json(url) + return list(map(lambda x: x["name"], result)) + +def add_release_to_github_issues(github_issue_ids, labels, fix_release): + for github_issue_id in github_issue_ids: + labels = add_release_to_github_issue(github_issue_id, labels, fix_release) + return labels + +def add_release_to_github_issue(github_issue_id, labels, fix_release): + url = get_github_issue_url(github_issue_id) + labels = ["release/%s" % fix_release] + labels + data = json.dumps({ + 'labels': labels + }) + post_json(url, data) + return labels + +def update_github_issue(github_issue_id, fix_milestone_number, fix_milestone, fix_areas, fix_types, other_labels): + url = get_github_issue_url(github_issue_id) + labels = other_labels + list(map(lambda x: ("area/%s" % x), fix_areas)) + labels = labels + list(map(lambda x: ("type/%s" % x), fix_types)) + if fix_milestone_number == '': + data = json.dumps({ + 'labels': labels, + }) + else: + labels.append("release/%s" % fix_milestone) + data = json.dumps({ + 'milestone': int(fix_milestone_number), + 'labels': labels, + }) + + post_json(url, data) + return labels + +def update_github_issues(github_issue_ids, fix_milestone_number, fix_milestone, fix_areas, fix_types, other_labels): + for github_issue_id in github_issue_ids: + labels = update_github_issue(github_issue_id, fix_milestone_number, fix_milestone, fix_areas, fix_types, other_labels) + return labels + +def add_assignees_to_github_issues(github_issue_ids, assignees): + for github_issue_id in github_issue_ids: + add_assignees_to_github_issue(github_issue_id, assignees) + +def add_assignees_to_github_issue(github_issue_id, assignees): + url = get_assignees_url(github_issue_id) + data = json.dumps({ + "assignees": assignees + }) + post_json(url, data) + +def main(): + global original_head + + if not GITHUB_OAUTH_KEY: + print("OAuth key is needed for merging bookkeeper pull requests.") + print("If environment variable 'GITHUB_OAUTH_KEY' is not defined,") + print("then requests will be unauthenticated.") + print("You can create an OAuth key at https://github.com/settings/tokens") + print("and set it to the environment variable 'GITHUB_OAUTH_KEY'.") + print("(This token only needs the 'public_repo' scope permissions)") + exit(-1) + + # 0. get the current state so we can go back + original_head = get_current_branch() + + # 1. retrieve milestones, labels, branches + milestones = get_milestones() + labels = get_all_labels() + branches = get_json("%s/branches" % GITHUB_API_BASE) + branch_names = list(filter(lambda x: x.startswith(RELEASE_BRANCH_PREFIX), [x['name'] for x in branches])) + # Assumes branch names can be sorted lexicographically + latest_branch = sorted(branch_names, reverse=True)[0] + + # 2. retrieve the details for a given pull request + pr_num = input("Which pull request would you like to merge? (e.g. 34): ") + pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) + pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) + pr_reviewers = get_reviewers(pr_num) + check_ci_status(pr) + + url = pr["url"] + + # 3. repair the title for commit message + pr_title = pr["title"] + commit_title = input("Commit title [%s]: " % pr_title) + if commit_title == "": + commit_title = pr_title + + # Decide whether to use the modified title or not + modified_title, github_issue_ids = standardize_issue_ref(commit_title) + if modified_title != commit_title: + print("I've re-written the title as follows to match the standard format:") + print("Original: %s" % commit_title) + print("Modified: %s" % modified_title) + result = input("Would you like to use the modified title? (y/n): ") + if result.lower() == "y": + commit_title = modified_title + print("Using modified title:") + else: + print("Using original title:") + print(commit_title) + + body = pr["body"] + modified_body = "" + for line in body.split('\n'): + if line.startswith('>'): + continue + modified_body = modified_body + line + "\n" + modified_body = modified_body[:-1] + if modified_body != body: + print("I've re-written the body as follows to match the standard formats:") + print("Original: ") + print(body) + print("Modified: ") + print(modified_body) + result = input("Would you like to use the modified body? (y/n): ") + if result.lower() == "y": + body = modified_body + print("Using modified body.") + else: + print("Using original body.") + + target_ref = pr["base"]["ref"] + user_login = pr["user"]["login"] + base_ref = pr["head"]["ref"] + pr_repo_desc = "%s/%s" % (user_login, base_ref) + + # append pr num to the github issues - we need to attach label and milestone to them + github_issue_ids.append(pr_num) + + # + # 4. attach milestone, area, type and release to github issues + # + + # get issue labels + issue_labels = get_github_issue_labels(pr_num) + # ask for fix milestone, area and type + fix_milestone, fix_milestone_number, fix_areas, fix_types = \ + ask_updates_for_github_issues(milestones, labels, issue_labels, target_ref == "master") + # update issues with fix milestone, are and type + other_labels = list(filter(lambda x: not x.startswith("area"), issue_labels)) + all_issue_labels = update_github_issues( \ + github_issue_ids, \ + fix_milestone_number, \ + fix_milestone, \ + fix_areas, \ + fix_types, \ + other_labels) + # add the pr author to the assignees + add_assignees_to_github_issues(github_issue_ids, [ user_login ]) + + if target_ref != "master": + branch_version = target_ref.split('-')[1] + # add releases + fix_releases = ask_release_for_github_issues(branch_version, labels) + if len(fix_releases) > 0: + all_issue_labels = add_release_to_github_issues(github_issue_ids, all_issue_labels, fix_releases[0]) + + # + # 5. Process the merge + # + + # Merged pull requests don't appear as merged in the GitHub API; + # Instead, they're closed by asfgit. + merge_commits = \ + [e for e in pr_events if e["actor"]["login"] == "asfgit" and e["event"] == "closed"] + + if merge_commits: + merge_hash = merge_commits[0]["commit_id"] + message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] + + print("Pull request %s has already been merged, assuming you want to backport" % pr_num) + commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', + "%s^{commit}" % merge_hash]).strip() != "" + if not commit_is_downloaded: + fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) + + print("Found commit %s:\n%s" % (merge_hash, message)) + + cherry_pick(pr_num, merge_hash, ask_for_branch(latest_branch)) + sys.exit(0) + + if not bool(pr["mergeable"]): + msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ + "You may need to rebase the PR." + fail(msg) + + print("\n=== Pull Request #%s ===" % pr_num) + print("PR title\t%s\nCommit title\t%s\nSource\t\t%s\nTarget\t\t%s\nURL\t\t%s" % ( + pr_title, commit_title, pr_repo_desc, target_ref, url)) + continue_maybe("Proceed with merging pull request #%s?" % pr_num) + + merged_refs = [target_ref] + # proceed with the merge + merge_hash, merge_commit_log = merge_pr(pr_num, target_ref, commit_title, body, pr_reviewers, pr_repo_desc) + + # once the pr is merged, refresh the local repo + run_cmd("git fetch %s" % PR_REMOTE_NAME) + + pick_prompt = "Would you like to pick %s into another branch?" % merge_hash + while input("\n%s (y/n): " % pick_prompt).lower() == "y": + pick_ref = ask_for_branch(latest_branch) + branch_version = pick_ref.split('-')[1] + # add releases + fix_releases = ask_release_for_github_issues(branch_version, labels) + if len(fix_releases) > 0: + all_issue_labels = add_release_to_github_issues(github_issue_ids, all_issue_labels, fix_releases[0]) + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, pick_ref)] + +if __name__ == "__main__": + import doctest + (failure_count, test_count) = doctest.testmod() + if (failure_count): + exit(-1) + + main() From 7de7164639509215b745ca187b082df2f7c35702 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Mon, 17 Sep 2018 01:06:45 -0700 Subject: [PATCH 0032/1642] Issue #850: Added request context across client and bookies. Descriptions of the changes in this PR: - MDC context is passed to runnables, callables etc. - protocol extended, context is sent to bookie servers, restored there and back on client with the response. Hopefully did not miss some nuance on the server side, largely rely on changes in ordered executors to do all the magic. - did microbenchmarking of the protocol changes (strings added to protobuf, MDC context preserved/restored). Looks ok. - added unit tests. (bug W-5291641) (bug W-5291648) ### Motivation Troubleshooting of request-level failures/errors can be simplified if request id was passed through all the stages of the request, from threadpools on the client to bookies to the response back on the client. Log4j/Slf4j allows logging of MDC data so it makes sense to use this functionality for logging. ### Changes - MDC context is passed to runnables, callables etc. - protocol extended, context is sent to bookie servers, restored there and back on client with the response. Hopefully did not miss some nuance on the server side, largely rely on changes in ordered executors to do all the magic. - did microbenchmarking of the protocol changes (strings added to protobuf, MDC context preserved/restored). Looks ok. - added unit tests. Master Issue: #850 Author: Andrey Yegorov Reviewers: Enrico Olivelli , Jia Zhai , Venkateswararao Jujjuri (JV) , Sijie Guo This closes #1672 from dlg99/feature/correlation_id, closes #850 --- .../bookkeeper/common/util/MdcUtils.java | 40 +++ .../common/util/OrderedExecutor.java | 137 +++++++++- .../common/util/OrderedScheduler.java | 21 +- .../src/main/proto/BookkeeperProtocol.proto | 7 + .../apache/bookkeeper/client/BookKeeper.java | 1 + .../conf/AbstractConfiguration.java | 23 ++ .../proto/BookieRequestProcessor.java | 112 ++++---- .../proto/PerChannelBookieClient.java | 82 +++++- .../util/OrderedGenericCallback.java | 55 ++-- .../bookkeeper/client/MdcContextTest.java | 249 ++++++++++++++++++ .../bookkeeper/proto/ProtocolBenchmark.java | 79 +++++- 11 files changed, 703 insertions(+), 103 deletions(-) create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/MdcUtils.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/MdcUtils.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/MdcUtils.java new file mode 100644 index 00000000000..f10f2540124 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/MdcUtils.java @@ -0,0 +1,40 @@ +/* + * + * 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.common.util; + +import java.util.Map; + +import org.slf4j.MDC; + +/** + * Utils for work with Slf4j MDC. + */ +public class MdcUtils { + + public static void restoreContext(Map mdcContextMap) { + if (mdcContextMap == null || mdcContextMap.isEmpty()) { + MDC.clear(); + } else { + MDC.setContextMap(mdcContextMap); + } + } +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java index 6a86141256f..520787b3835 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java @@ -28,6 +28,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; @@ -38,6 +39,7 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; @@ -46,6 +48,7 @@ import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.lang.StringUtils; +import org.slf4j.MDC; /** * This class provides 2 things over the java {@link ExecutorService}. @@ -73,6 +76,7 @@ public class OrderedExecutor implements ExecutorService { final OpStatsLogger taskExecutionStats; final OpStatsLogger taskPendingStats; final boolean traceTaskExecution; + final boolean preserveMdcForTaskExecution; final long warnTimeMicroSec; final int maxTasksInQueue; @@ -92,7 +96,8 @@ public OrderedExecutor build() { threadFactory = new DefaultThreadFactory("bookkeeper-ordered-safe-executor"); } return new OrderedExecutor(name, numThreads, threadFactory, statsLogger, - traceTaskExecution, warnTimeMicroSec, maxTasksInQueue); + traceTaskExecution, preserveMdcForTaskExecution, + warnTimeMicroSec, maxTasksInQueue); } } @@ -105,6 +110,7 @@ public abstract static class AbstractBuilder { protected ThreadFactory threadFactory = null; protected StatsLogger statsLogger = NullStatsLogger.INSTANCE; protected boolean traceTaskExecution = false; + protected boolean preserveMdcForTaskExecution = false; protected long warnTimeMicroSec = WARN_TIME_MICRO_SEC_DEFAULT; protected int maxTasksInQueue = NO_TASK_LIMIT; @@ -138,6 +144,11 @@ public AbstractBuilder traceTaskExecution(boolean enabled) { return this; } + public AbstractBuilder preserveMdcForTaskExecution(boolean enabled) { + this.preserveMdcForTaskExecution = enabled; + return this; + } + public AbstractBuilder traceTaskWarnTimeMicroSec(long warnTimeMicroSec) { this.warnTimeMicroSec = warnTimeMicroSec; return this; @@ -154,6 +165,7 @@ public T build() { threadFactory, statsLogger, traceTaskExecution, + preserveMdcForTaskExecution, warnTimeMicroSec, maxTasksInQueue); } @@ -185,6 +197,81 @@ public void run() { } } + /** + * Decorator class for a callable that measure the execution time. + */ + protected class TimedCallable implements Callable { + final Callable callable; + final long initNanos; + + TimedCallable(Callable callable) { + this.callable = callable; + this.initNanos = MathUtils.nowInNano(); + } + + @Override + public T call() throws Exception { + taskPendingStats.registerSuccessfulEvent(MathUtils.elapsedNanos(initNanos), TimeUnit.NANOSECONDS); + long startNanos = MathUtils.nowInNano(); + try { + return this.callable.call(); + } finally { + long elapsedMicroSec = MathUtils.elapsedMicroSec(startNanos); + taskExecutionStats.registerSuccessfulEvent(elapsedMicroSec, TimeUnit.MICROSECONDS); + if (elapsedMicroSec >= warnTimeMicroSec) { + log.warn("Callable {}:{} took too long {} micros to execute.", callable, callable.getClass(), + elapsedMicroSec); + } + } + } + } + + /** + * Decorator class for a runnable that preserves MDC context. + */ + static class ContextPreservingRunnable implements Runnable { + private final Runnable runnable; + private final Map mdcContextMap; + + ContextPreservingRunnable(Runnable runnable) { + this.runnable = runnable; + this.mdcContextMap = MDC.getCopyOfContextMap(); + } + + @Override + public void run() { + MdcUtils.restoreContext(mdcContextMap); + try { + runnable.run(); + } finally { + MDC.clear(); + } + } + } + + /** + * Decorator class for a callable that preserves MDC context. + */ + static class ContextPreservingCallable implements Callable { + private final Callable callable; + private final Map mdcContextMap; + + ContextPreservingCallable(Callable callable) { + this.callable = callable; + this.mdcContextMap = MDC.getCopyOfContextMap(); + } + + @Override + public T call() throws Exception { + MdcUtils.restoreContext(mdcContextMap); + try { + return callable.call(); + } finally { + MDC.clear(); + } + } + } + protected ThreadPoolExecutor createSingleThreadExecutor(ThreadFactory factory) { return new ThreadPoolExecutor(1, 1, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(), factory); } @@ -206,6 +293,8 @@ protected ExecutorService getBoundedExecutor(ThreadPoolExecutor executor) { * - for reporting executor stats * @param traceTaskExecution * - should we stat task execution + * @param preserveMdcForTaskExecution + * - should we preserve MDC for task execution * @param warnTimeMicroSec * - log long task exec warning after this interval * @param maxTasksInQueue @@ -213,7 +302,7 @@ protected ExecutorService getBoundedExecutor(ThreadPoolExecutor executor) { */ protected OrderedExecutor(String baseName, int numThreads, ThreadFactory threadFactory, StatsLogger statsLogger, boolean traceTaskExecution, - long warnTimeMicroSec, int maxTasksInQueue) { + boolean preserveMdcForTaskExecution, long warnTimeMicroSec, int maxTasksInQueue) { checkArgument(numThreads > 0); checkArgument(!StringUtils.isBlank(baseName)); @@ -280,6 +369,17 @@ public Number getSample() { this.taskExecutionStats = statsLogger.scope(name).getOpStatsLogger("task_execution"); this.taskPendingStats = statsLogger.scope(name).getOpStatsLogger("task_queued"); this.traceTaskExecution = traceTaskExecution; + this.preserveMdcForTaskExecution = preserveMdcForTaskExecution; + } + + /** + * Flag describing executor's expectation in regards of MDC. + * All tasks submitted through executor's submit/execute methods will automatically respect this. + * + * @return true if runnable/callable is expected to preserve MDC, false otherwise. + */ + public boolean preserveMdc() { + return preserveMdcForTaskExecution; } /** @@ -369,12 +469,23 @@ public ExecutorService chooseThread(long orderingKey) { return threads[MathUtils.signSafeMod(orderingKey, threads.length)]; } - private Runnable timedRunnable(Runnable r) { - if (traceTaskExecution) { - return new TimedRunnable(r); - } else { - return r; + protected Runnable timedRunnable(Runnable r) { + final Runnable runMe = traceTaskExecution ? new TimedRunnable(r) : r; + return preserveMdcForTaskExecution ? new ContextPreservingRunnable(runMe) : runMe; + } + + protected Callable timedCallable(Callable c) { + final Callable callMe = traceTaskExecution ? new TimedCallable<>(c) : c; + return preserveMdcForTaskExecution ? new ContextPreservingCallable<>(callMe) : callMe; + } + + protected Collection> timedCallables(Collection> tasks) { + if (traceTaskExecution || preserveMdcForTaskExecution) { + return tasks.stream() + .map(this::timedCallable) + .collect(Collectors.toList()); } + return tasks; } /** @@ -382,7 +493,7 @@ private Runnable timedRunnable(Runnable r) { */ @Override public Future submit(Callable task) { - return chooseThread().submit(task); + return chooseThread().submit(timedCallable(task)); } /** @@ -407,7 +518,7 @@ public Future submit(Runnable task) { @Override public List> invokeAll(Collection> tasks) throws InterruptedException { - return chooseThread().invokeAll(tasks); + return chooseThread().invokeAll(timedCallables(tasks)); } /** @@ -418,7 +529,7 @@ public List> invokeAll(Collection> tasks, long timeout, TimeUnit unit) throws InterruptedException { - return chooseThread().invokeAll(tasks, timeout, unit); + return chooseThread().invokeAll(timedCallables(tasks), timeout, unit); } /** @@ -427,7 +538,7 @@ public List> invokeAll(Collection> tasks, @Override public T invokeAny(Collection> tasks) throws InterruptedException, ExecutionException { - return chooseThread().invokeAny(tasks); + return chooseThread().invokeAny(timedCallables(tasks)); } /** @@ -436,7 +547,7 @@ public T invokeAny(Collection> tasks) @Override public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - return chooseThread().invokeAny(tasks, timeout, unit); + return chooseThread().invokeAny(timedCallables(tasks), timeout, unit); } /** @@ -444,7 +555,7 @@ public T invokeAny(Collection> tasks, long timeout, Ti */ @Override public void execute(Runnable command) { - chooseThread().execute(command); + chooseThread().execute(timedRunnable(command)); } diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java index 6f05832b9d5..6f366fdbc87 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java @@ -72,6 +72,7 @@ public OrderedScheduler build() { threadFactory, statsLogger, traceTaskExecution, + preserveMdcForTaskExecution, warnTimeMicroSec, maxTasksInQueue); } @@ -90,6 +91,8 @@ public OrderedScheduler build() { * - for reporting executor stats * @param traceTaskExecution * - should we stat task execution + * @param preserveMdcForTaskExecution + * - should we preserve MDC for task execution * @param warnTimeMicroSec * - log long task exec warning after this interval */ @@ -98,9 +101,11 @@ private OrderedScheduler(String baseName, ThreadFactory threadFactory, StatsLogger statsLogger, boolean traceTaskExecution, + boolean preserveMdcForTaskExecution, long warnTimeMicroSec, int maxTasksInQueue) { - super(baseName, numThreads, threadFactory, statsLogger, traceTaskExecution, warnTimeMicroSec, maxTasksInQueue); + super(baseName, numThreads, threadFactory, statsLogger, traceTaskExecution, + preserveMdcForTaskExecution, warnTimeMicroSec, maxTasksInQueue); } @@ -150,7 +155,7 @@ public ListenableFuture submitOrdered(Object orderingKey, * will return null upon completion */ public ScheduledFuture schedule(SafeRunnable command, long delay, TimeUnit unit) { - return chooseThread().schedule(command, delay, unit); + return chooseThread().schedule(timedRunnable(command), delay, unit); } /** @@ -181,7 +186,7 @@ public ScheduledFuture scheduleOrdered(Object orderingKey, SafeRunnable comma * method will throw an exception upon cancellation */ public ScheduledFuture scheduleAtFixedRate(SafeRunnable command, long initialDelay, long period, TimeUnit unit) { - return chooseThread().scheduleAtFixedRate(command, initialDelay, period, unit); + return chooseThread().scheduleAtFixedRate(timedRunnable(command), initialDelay, period, unit); } /** @@ -219,7 +224,7 @@ public ScheduledFuture scheduleAtFixedRateOrdered(Object orderingKey, SafeRun */ public ScheduledFuture scheduleWithFixedDelay(SafeRunnable command, long initialDelay, long delay, TimeUnit unit) { - return chooseThread().scheduleWithFixedDelay(command, initialDelay, delay, unit); + return chooseThread().scheduleWithFixedDelay(timedRunnable(command), initialDelay, delay, unit); } /** @@ -252,7 +257,7 @@ public ScheduledFuture scheduleWithFixedDelayOrdered(Object orderingKey, Safe */ @Override public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - return chooseThread().schedule(command, delay, unit); + return chooseThread().schedule(timedRunnable(command), delay, unit); } /** @@ -260,7 +265,7 @@ public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) */ @Override public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { - return chooseThread().schedule(callable, delay, unit); + return chooseThread().schedule(timedCallable(callable), delay, unit); } /** @@ -269,7 +274,7 @@ public ScheduledFuture schedule(Callable callable, long delay, TimeUni @Override public ScheduledFuture scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) { - return chooseThread().scheduleAtFixedRate(command, initialDelay, period, unit); + return chooseThread().scheduleAtFixedRate(timedRunnable(command), initialDelay, period, unit); } /** @@ -278,7 +283,7 @@ public ScheduledFuture scheduleAtFixedRate(Runnable command, @Override public ScheduledFuture scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) { - return chooseThread().scheduleWithFixedDelay(command, initialDelay, delay, unit); + return chooseThread().scheduleWithFixedDelay(timedRunnable(command), initialDelay, delay, unit); } } diff --git a/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto b/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto index bac9411330c..f34d56e5919 100644 --- a/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto +++ b/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto @@ -76,6 +76,11 @@ message BKPacketHeader { optional uint32 priority = 4 [default = 0]; } +message ContextPair { + required string key = 1; + required string value = 2; +} + message Request { required BKPacketHeader header = 1; // Requests @@ -87,6 +92,8 @@ message Request { optional GetBookieInfoRequest getBookieInfoRequest = 105; optional StartTLSRequest startTLSRequest = 106; optional ForceLedgerRequest forceLedgerRequest = 107; + // to pass MDC context + repeated ContextPair requestContext = 200; } message ReadRequest { 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 1396ee75a64..d434ffa39e5 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 @@ -402,6 +402,7 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo .numThreads(conf.getNumWorkerThreads()) .statsLogger(rootStatsLogger) .traceTaskExecution(conf.getEnableTaskExecutionStats()) + .preserveMdcForTaskExecution(conf.getPreserveMdcForTaskExecution()) .traceTaskWarnTimeMicroSec(conf.getTaskExecutionWarnTimeMicros()) .build(); 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 b72b0ba5e7b..666b377b30b 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 @@ -102,6 +102,9 @@ public abstract class AbstractConfiguration // Enable authentication of the other connection end point (mutual authentication) protected static final String TLS_CLIENT_AUTHENTICATION = "tlsClientAuthentication"; + // Preserve MDC or not for tasks in executor + protected static final String PRESERVE_MDC_FOR_TASK_EXECUTION = "preserveMdcForTaskExecution"; + // Default formatter classes protected static final Class DEFAULT_ENTRY_FORMATTER = StringEntryFormatter.class; protected static final Class DEFAULT_LEDGERID_FORMATTER = @@ -851,6 +854,26 @@ public boolean getStoreSystemTimeAsLedgerUnderreplicatedMarkTime() { return getBoolean(STORE_SYSTEMTIME_AS_LEDGER_UNDERREPLICATED_MARK_TIME, false); } + /** + * Whether to preserve MDC for tasks in Executor. + * + * @return flag to enable/disable MDC preservation in Executor. + */ + public boolean getPreserveMdcForTaskExecution() { + return getBoolean(PRESERVE_MDC_FOR_TASK_EXECUTION, false); + } + + /** + * Whether to preserve MDC for tasks in Executor. + * + * @param enabled + * flag to enable/disable MDC preservation in Executor. + * @return configuration. + */ + public T setPreserveMdcForTaskExecution(boolean enabled) { + setProperty(PRESERVE_MDC_FOR_TASK_EXECUTION, enabled); + return getThis(); + } /** * Trickery to allow inheritance with fluent style. */ 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 8ee363fc78f..5a50bf69291 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 @@ -89,6 +89,7 @@ import org.apache.bookkeeper.tls.SecurityHandlerFactory.NodeType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.MDC; /** * An implementation of the RequestProcessor interface. @@ -104,6 +105,7 @@ public class BookieRequestProcessor implements RequestProcessor { */ private final ServerConfiguration serverCfg; private final long waitTimeoutOnBackpressureMillis; + private final boolean preserveMdcForTaskExecution; /** * This is the Bookie instance that is used to handle all read and write requests. @@ -187,6 +189,7 @@ public BookieRequestProcessor(ServerConfiguration serverCfg, Bookie bookie, StatsLogger statsLogger, SecurityHandlerFactory shFactory) throws SecurityException { this.serverCfg = serverCfg; this.waitTimeoutOnBackpressureMillis = serverCfg.getWaitTimeoutOnResponseBackpressureMillis(); + this.preserveMdcForTaskExecution = serverCfg.getPreserveMdcForTaskExecution(); this.bookie = bookie; this.readThreadPool = createExecutor( this.serverCfg.getNumReadWorkerThreads(), @@ -412,6 +415,7 @@ private OrderedExecutor createExecutor( .numThreads(numThreads) .name(nameFormat) .traceTaskExecution(serverCfg.getEnableTaskExecutionStats()) + .preserveMdcForTaskExecution(serverCfg.getPreserveMdcForTaskExecution()) .statsLogger(statsLogger) .maxTasksInQueue(maxTasksInQueue) .build(); @@ -430,52 +434,57 @@ public void processRequest(Object msg, Channel c) { // it as a version 3 packet. Else, just use the old protocol. if (msg instanceof BookkeeperProtocol.Request) { BookkeeperProtocol.Request r = (BookkeeperProtocol.Request) msg; - BookkeeperProtocol.BKPacketHeader header = r.getHeader(); - switch (header.getOperation()) { - case ADD_ENTRY: - processAddRequestV3(r, c); - break; - case READ_ENTRY: - processReadRequestV3(r, c); - break; - case FORCE_LEDGER: - processForceLedgerRequestV3(r, c); - break; - case AUTH: - LOG.info("Ignoring auth operation from client {}", c.remoteAddress()); - BookkeeperProtocol.AuthMessage message = BookkeeperProtocol.AuthMessage - .newBuilder() - .setAuthPluginName(AuthProviderFactoryFactory.AUTHENTICATION_DISABLED_PLUGIN_NAME) - .setPayload(ByteString.copyFrom(AuthToken.NULL.getData())) - .build(); - BookkeeperProtocol.Response.Builder authResponse = BookkeeperProtocol.Response - .newBuilder().setHeader(r.getHeader()) - .setStatus(BookkeeperProtocol.StatusCode.EOK) - .setAuthResponse(message); - c.writeAndFlush(authResponse.build()); - break; - case WRITE_LAC: - processWriteLacRequestV3(r, c); - break; - case READ_LAC: - processReadLacRequestV3(r, c); - break; - case GET_BOOKIE_INFO: - processGetBookieInfoRequestV3(r, c); - break; - case START_TLS: - processStartTLSRequestV3(r, c); - break; - default: - LOG.info("Unknown operation type {}", header.getOperation()); - BookkeeperProtocol.Response.Builder response = - BookkeeperProtocol.Response.newBuilder().setHeader(r.getHeader()) - .setStatus(BookkeeperProtocol.StatusCode.EBADREQ); - c.writeAndFlush(response.build()); - if (statsEnabled) { - bkStats.getOpStats(BKStats.STATS_UNKNOWN).incrementFailedOps(); - } - break; + restoreMdcContextFromRequest(r); + try { + BookkeeperProtocol.BKPacketHeader header = r.getHeader(); + switch (header.getOperation()) { + case ADD_ENTRY: + processAddRequestV3(r, c); + break; + case READ_ENTRY: + processReadRequestV3(r, c); + break; + case FORCE_LEDGER: + processForceLedgerRequestV3(r, c); + break; + case AUTH: + LOG.info("Ignoring auth operation from client {}", c.remoteAddress()); + BookkeeperProtocol.AuthMessage message = BookkeeperProtocol.AuthMessage + .newBuilder() + .setAuthPluginName(AuthProviderFactoryFactory.AUTHENTICATION_DISABLED_PLUGIN_NAME) + .setPayload(ByteString.copyFrom(AuthToken.NULL.getData())) + .build(); + BookkeeperProtocol.Response.Builder authResponse = BookkeeperProtocol.Response + .newBuilder().setHeader(r.getHeader()) + .setStatus(BookkeeperProtocol.StatusCode.EOK) + .setAuthResponse(message); + c.writeAndFlush(authResponse.build()); + break; + case WRITE_LAC: + processWriteLacRequestV3(r, c); + break; + case READ_LAC: + processReadLacRequestV3(r, c); + break; + case GET_BOOKIE_INFO: + processGetBookieInfoRequestV3(r, c); + break; + case START_TLS: + processStartTLSRequestV3(r, c); + break; + default: + LOG.info("Unknown operation type {}", header.getOperation()); + BookkeeperProtocol.Response.Builder response = + BookkeeperProtocol.Response.newBuilder().setHeader(r.getHeader()) + .setStatus(BookkeeperProtocol.StatusCode.EBADREQ); + c.writeAndFlush(response.build()); + if (statsEnabled) { + bkStats.getOpStats(BKStats.STATS_UNKNOWN).incrementFailedOps(); + } + break; + } + } finally { + MDC.clear(); } } else { BookieProtocol.Request r = (BookieProtocol.Request) msg; @@ -500,7 +509,16 @@ public void processRequest(Object msg, Channel c) { } } - private void processWriteLacRequestV3(final BookkeeperProtocol.Request r, final Channel c) { + private void restoreMdcContextFromRequest(BookkeeperProtocol.Request req) { + if (preserveMdcForTaskExecution) { + MDC.clear(); + for (BookkeeperProtocol.ContextPair pair: req.getRequestContextList()) { + MDC.put(pair.getKey(), pair.getValue()); + } + } + } + + private void processWriteLacRequestV3(final BookkeeperProtocol.Request r, final Channel c) { WriteLacProcessorV3 writeLac = new WriteLacProcessorV3(r, c, this); if (null == writeThreadPool) { writeLac.run(); 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 860079737fd..79e4de912bc 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 @@ -70,6 +70,7 @@ import java.util.Collections; import java.util.EnumSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Queue; import java.util.Set; @@ -87,6 +88,7 @@ import org.apache.bookkeeper.client.BookKeeperClientStats; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.api.WriteFlag; +import org.apache.bookkeeper.common.util.MdcUtils; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -132,6 +134,7 @@ import org.apache.bookkeeper.util.collections.SynchronizedHashMultiMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.MDC; /** * This class manages all details of connection to a particular bookie. It also @@ -198,6 +201,7 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter { private final Counter failedTlsHandshakeCounter; private final boolean useV2WireProtocol; + private final boolean preserveMdcForTaskExecution; /** * The following member variables do not need to be concurrent, or volatile @@ -267,6 +271,7 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor this.getBookieInfoTimeout = conf.getBookieInfoTimeout(); this.startTLSTimeout = conf.getStartTLSTimeout(); this.useV2WireProtocol = conf.getUseV2WireProtocol(); + this.preserveMdcForTaskExecution = conf.getPreserveMdcForTaskExecution(); this.authProviderFactory = authProviderFactory; this.extRegistry = extRegistry; @@ -466,7 +471,7 @@ protected void initChannel(Channel ch) throws Exception { } ChannelFuture future = bootstrap.connect(bookieAddr); - future.addListener(new ConnectionFutureListener(startTime)); + future.addListener(contextPreservingListener(new ConnectionFutureListener(startTime))); future.addListener(x -> makeWritable()); return future; } @@ -565,7 +570,7 @@ void writeLac(final long ledgerId, final byte[] masterKey, final long lac, ByteB .setMasterKey(UnsafeByteOperations.unsafeWrap(masterKey)) .setBody(body); - final Request writeLacRequest = Request.newBuilder() + final Request writeLacRequest = withRequestContext(Request.newBuilder()) .setHeader(headerBuilder) .setWriteLacRequest(writeLacBuilder) .build(); @@ -596,7 +601,7 @@ void forceLedger(final long ledgerId, ForceLedgerCallback cb, Object ctx) { ForceLedgerRequest.Builder writeLacBuilder = ForceLedgerRequest.newBuilder() .setLedgerId(ledgerId); - final Request forceLedgerRequest = Request.newBuilder() + final Request forceLedgerRequest = withRequestContext(Request.newBuilder()) .setHeader(headerBuilder) .setForceLedgerRequest(writeLacBuilder) .build(); @@ -676,7 +681,7 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBuf addBuilder.setWriteFlags(WriteFlag.getWriteFlagsValue(writeFlags)); } - request = Request.newBuilder() + request = withRequestContext(Request.newBuilder()) .setHeader(headerBuilder) .setAddRequest(addBuilder) .build(); @@ -716,7 +721,7 @@ public void readLac(final long ledgerId, ReadLacCallback cb, Object ctx) { .setTxnId(txnId); ReadLacRequest.Builder readLacBuilder = ReadLacRequest.newBuilder() .setLedgerId(ledgerId); - request = Request.newBuilder() + request = withRequestContext(Request.newBuilder()) .setHeader(headerBuilder) .setReadLacRequest(readLacBuilder) .build(); @@ -823,7 +828,7 @@ private void readEntryInternal(final long ledgerId, readBuilder.setMasterKey(ByteString.copyFrom(masterKey)); } - request = Request.newBuilder() + request = withRequestContext(Request.newBuilder()) .setHeader(headerBuilder) .setReadRequest(readBuilder) .build(); @@ -851,7 +856,7 @@ public void getBookieInfo(final long requested, GetBookieInfoCallback cb, Object GetBookieInfoRequest.Builder getBookieInfoBuilder = GetBookieInfoRequest.newBuilder() .setRequested(requested); - final Request getBookieInfoRequest = Request.newBuilder() + final Request getBookieInfoRequest = withRequestContext(Request.newBuilder()) .setHeader(headerBuilder) .setGetBookieInfoRequest(getBookieInfoBuilder) .build(); @@ -1290,8 +1295,8 @@ private static StatusCode getStatusCodeFromErrorCode(int errorCode) { private void readV3Response(final Response response) { final BKPacketHeader header = response.getHeader(); - final CompletionValue completionValue = completionObjects.get(newCompletionKey(header.getTxnId(), - header.getOperation())); + final CompletionKey key = newCompletionKey(header.getTxnId(), header.getOperation()); + final CompletionValue completionValue = completionObjects.get(key); if (null == completionValue) { // Unexpected response, so log it. The txnId should have been present. @@ -1304,6 +1309,7 @@ private void readV3Response(final Response response) { executor.executeOrdered(orderingKey, new SafeRunnable() { @Override public void safeRun() { + completionValue.restoreMdcContext(); completionValue.handleV3Response(response); } @@ -1316,7 +1322,7 @@ public String toString() { }); } - completionObjects.remove(newCompletionKey(header.getTxnId(), header.getOperation())); + completionObjects.remove(key); } void initTLSHandshake() { @@ -1399,6 +1405,7 @@ abstract class CompletionValue { private final OpStatsLogger opLogger; private final OpStatsLogger timeoutOpLogger; private final String operationName; + private final Map mdcContextMap; protected Object ctx; protected long ledgerId; protected long entryId; @@ -1416,6 +1423,7 @@ public CompletionValue(String operationName, this.startTime = MathUtils.nowInNano(); this.opLogger = opLogger; this.timeoutOpLogger = timeoutOpLogger; + this.mdcContextMap = preserveMdcForTaskExecution ? MDC.getCopyOfContextMap() : null; } private long latency() { @@ -1469,6 +1477,9 @@ protected int convertStatus(StatusCode status, int defaultStatus) { } } + public void restoreMdcContext() { + MdcUtils.restoreContext(mdcContextMap); + } public abstract void errorOut(); public abstract void errorOut(int rc); @@ -2128,6 +2139,55 @@ public void release() { } } + Request.Builder withRequestContext(Request.Builder builder) { + if (preserveMdcForTaskExecution) { + return appendRequestContext(builder); + } + return builder; + } + + static Request.Builder appendRequestContext(Request.Builder builder) { + final Map mdcContextMap = MDC.getCopyOfContextMap(); + if (mdcContextMap == null || mdcContextMap.isEmpty()) { + return builder; + } + for (Map.Entry kv : mdcContextMap.entrySet()) { + final BookkeeperProtocol.ContextPair context = BookkeeperProtocol.ContextPair.newBuilder() + .setKey(kv.getKey()) + .setValue(kv.getValue()) + .build(); + builder.addRequestContext(context); + } + return builder; + } + + ChannelFutureListener contextPreservingListener(ChannelFutureListener listener) { + return preserveMdcForTaskExecution ? new ContextPreservingFutureListener(listener) : listener; + } + + /** + * Decorator to preserve MDC for connection listener. + */ + static class ContextPreservingFutureListener implements ChannelFutureListener { + private final ChannelFutureListener listener; + private final Map mdcContextMap; + + ContextPreservingFutureListener(ChannelFutureListener listener) { + this.listener = listener; + this.mdcContextMap = MDC.getCopyOfContextMap(); + } + + @Override + public void operationComplete(ChannelFuture future) throws Exception { + MdcUtils.restoreContext(mdcContextMap); + try { + listener.operationComplete(future); + } finally { + MDC.clear(); + } + } + } + /** * Connection listener. */ @@ -2228,7 +2288,7 @@ private void initiateTLS() { final CompletionKey completionKey = new V3CompletionKey(txnId, OperationType.START_TLS); completionObjects.put(completionKey, new StartTLSCompletion(completionKey)); - BookkeeperProtocol.Request.Builder h = BookkeeperProtocol.Request.newBuilder(); + BookkeeperProtocol.Request.Builder h = withRequestContext(BookkeeperProtocol.Request.newBuilder()); BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() .setVersion(ProtocolVersion.VERSION_THREE) .setOperation(OperationType.START_TLS) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/OrderedGenericCallback.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/OrderedGenericCallback.java index 73150ad0112..5c16de8e13b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/OrderedGenericCallback.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/OrderedGenericCallback.java @@ -17,13 +17,16 @@ */ package org.apache.bookkeeper.util; +import java.util.Map; import java.util.concurrent.RejectedExecutionException; +import org.apache.bookkeeper.common.util.MdcUtils; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.common.util.SafeRunnable; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.MDC; /** * Generic callback implementation which will run the @@ -34,6 +37,7 @@ public abstract class OrderedGenericCallback implements GenericCallback { private final OrderedExecutor executor; private final long orderingKey; + private final Map mdcContextMap; /** * @param executor The executor on which to run the callback @@ -43,33 +47,40 @@ public abstract class OrderedGenericCallback implements GenericCallback { public OrderedGenericCallback(OrderedExecutor executor, long orderingKey) { this.executor = executor; this.orderingKey = orderingKey; + this.mdcContextMap = executor.preserveMdc() ? MDC.getCopyOfContextMap() : null; } @Override public final void operationComplete(final int rc, final T result) { - // during closing, callbacks that are error out might try to submit to - // the scheduler again. if the submission will go to same thread, we - // don't need to submit to executor again. this is also an optimization for - // callback submission - if (Thread.currentThread().getId() == executor.getThreadID(orderingKey)) { - safeOperationComplete(rc, result); - } else { - try { - executor.executeOrdered(orderingKey, new SafeRunnable() { - @Override - public void safeRun() { - safeOperationComplete(rc, result); - } - @Override - public String toString() { - return String.format("Callback(key=%s, name=%s)", - orderingKey, - OrderedGenericCallback.this); - } - }); - } catch (RejectedExecutionException re) { - LOG.warn("Failed to submit callback for {} : ", orderingKey, re); + MdcUtils.restoreContext(mdcContextMap); + try { + // during closing, callbacks that are error out might try to submit to + // the scheduler again. if the submission will go to same thread, we + // don't need to submit to executor again. this is also an optimization for + // callback submission + if (Thread.currentThread().getId() == executor.getThreadID(orderingKey)) { + safeOperationComplete(rc, result); + } else { + try { + executor.executeOrdered(orderingKey, new SafeRunnable() { + @Override + public void safeRun() { + safeOperationComplete(rc, result); + } + + @Override + public String toString() { + return String.format("Callback(key=%s, name=%s)", + orderingKey, + OrderedGenericCallback.this); + } + }); + } catch (RejectedExecutionException re) { + LOG.warn("Failed to submit callback for {} : ", orderingKey, re); + } } + } finally { + MDC.clear(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java new file mode 100644 index 00000000000..f10427b8b94 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java @@ -0,0 +1,249 @@ +/* + * + * 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.client; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.hasItem; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; +import static org.mockito.AdditionalAnswers.answerVoid; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.util.LinkedList; +import java.util.List; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; +import org.apache.bookkeeper.bookie.LedgerDirsManager; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.log4j.Appender; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.log4j.MDC; +import org.apache.log4j.spi.LoggingEvent; +import org.hamcrest.CoreMatchers; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + + +/** + * Test passing of MDC context. + */ +@SuppressWarnings("deprecation") +@Slf4j +public class MdcContextTest extends BookKeeperClusterTestCase { + public static final String MDC_REQUEST_ID = "request_id"; + + final byte[] entry = "Test Entry".getBytes(); + + BookKeeper bkc; + LedgerHandle lh; + + private Appender mockAppender; + private List capturedEvents; + private Logger rootLogger = LogManager.getRootLogger(); + + public MdcContextTest() { + super(3); + baseConf.setNumAddWorkerThreads(0); + baseConf.setNumReadWorkerThreads(0); + baseConf.setPreserveMdcForTaskExecution(true); + baseConf.setReadOnlyModeEnabled(true); + + // for read-only bookie + baseConf.setLedgerStorageClass(InterleavedLedgerStorage.class.getName()); + baseConf.setEntryLogFilePreAllocationEnabled(false); + baseConf.setMinUsableSizeForEntryLogCreation(Long.MAX_VALUE); + } + + + public static String mdcFormat(Object mdc, String message) { + return mdc == null + ? "[" + MDC_REQUEST_ID + ":] - " + message + : "[" + MDC_REQUEST_ID + ":" + mdc.toString() + + "] - " + message; + } + + public void assertLogWithMdc(String mdc, String msgSubstring) { + assertThat(capturedEvents, + hasItem(CoreMatchers.allOf( + containsString("[" + MDC_REQUEST_ID + ":" + mdc + "] - "), + containsString(msgSubstring) + ))); + } + + @Before + public void setUp() throws Exception { + super.setUp(); + ClientConfiguration conf = new ClientConfiguration(); + conf.setReadTimeout(360) + .setMetadataServiceUri(zkUtil.getMetadataServiceUri()) + .setPreserveMdcForTaskExecution(true); + + MDC.clear(); + bkc = new BookKeeper(conf); + + MDC.put(MDC_REQUEST_ID, "ledger_create"); + log.info("creating ledger"); + lh = bkc.createLedgerAdv(3, 3, 3, BookKeeper.DigestType.CRC32, new byte[] {}); + MDC.clear(); + + mockAppender = mock(Appender.class); + when(mockAppender.getName()).thenReturn("MockAppender"); + + rootLogger.addAppender(mockAppender); + rootLogger.setLevel(Level.INFO); + capturedEvents = new LinkedList<>(); + + doAnswer(answerVoid((LoggingEvent event) -> capturedEvents.add( + mdcFormat(event.getMDC(MDC_REQUEST_ID), event.getRenderedMessage()) + ))).when(mockAppender).doAppend(any()); + } + + @After + public void tearDown() throws Exception { + lh.close(); + bkc.close(); + rootLogger.removeAppender(mockAppender); + capturedEvents = null; + MDC.clear(); + super.tearDown(); + } + + @Test + public void testLedgerCreateFails() throws Exception { + MDC.put(MDC_REQUEST_ID, "ledger_create_fail"); + try { + bkc.createLedgerAdv(99, 3, 2, BookKeeper.DigestType.CRC32, new byte[]{}); + Assert.fail("should not get here"); + } catch (BKException bke) { + // expected + } + assertLogWithMdc("ledger_create_fail", "Not enough bookies to create ledger"); + } + + @Test + public void testSimpleAdd() throws Exception { + MDC.put(MDC_REQUEST_ID, "ledger_add_entry"); + lh.addEntry(0, entry); + + // client msg + assertLogWithMdc("ledger_add_entry", "Successfully connected to bookie"); + // bookie msg + assertLogWithMdc("ledger_add_entry", "Created new entry log file"); + } + + @Test + public void testAddWithEnsembleChange() throws Exception { + lh.addEntry(0, entry); + startNewBookie(); + killBookie(0); + + MDC.put(MDC_REQUEST_ID, "ledger_add_entry"); + lh.addEntry(1, entry); + assertLogWithMdc("ledger_add_entry", "Could not connect to bookie"); + assertLogWithMdc("ledger_add_entry", "Failed to write entry"); + assertLogWithMdc("ledger_add_entry", "New Ensemble"); + } + + @Test + public void testAddFailsWithReadOnlyBookie() throws Exception { + for (int i = 0; i < 3; ++i) { + Bookie bookie = bs.get(i).getBookie(); + File[] ledgerDirs = bsConfs.get(i).getLedgerDirs(); + LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager(); + ledgerDirsManager.addToFilledDirs(new File(ledgerDirs[0], "current")); + } + + MDC.put(MDC_REQUEST_ID, "ledger_add_entry"); + try { + lh.addEntry(0, entry); + Assert.fail("should not get here"); + } catch (BKException bke) { + // expected, pass + } + + assertLogWithMdc("ledger_add_entry", "No writable ledger dirs below diskUsageThreshold"); + assertLogWithMdc("ledger_add_entry", "All ledger directories are non writable and no reserved space"); + assertLogWithMdc("ledger_add_entry", "Error writing entry:0 to ledger:0"); + assertLogWithMdc("ledger_add_entry", "Add for failed on bookie"); + assertLogWithMdc("ledger_add_entry", "Failed to find 1 bookies"); + assertLogWithMdc("ledger_add_entry", "Could not get additional bookie to remake ensemble, closing ledger: 0"); + } + + @Test + public void testAddFailsDuplicateEntry() throws Exception { + lh.addEntry(0, entry); + + MDC.put(MDC_REQUEST_ID, "ledger_add_duplicate_entry"); + try { + lh.addEntry(0, entry); + Assert.fail("should not get here"); + } catch (BKException bke) { + // expected, pass + } + + assertLogWithMdc("ledger_add_duplicate_entry", "Trying to re-add duplicate entryid:0"); + assertLogWithMdc("ledger_add_duplicate_entry", "Write of ledger entry to quorum failed"); + } + + @Test + public void testReadEntryBeyondLac() throws Exception { + MDC.put(MDC_REQUEST_ID, "ledger_read_entry"); + + try { + lh.readEntries(100, 100); + fail("should not get here"); + } catch (BKException.BKReadException e) { + // pass + } + assertLogWithMdc("ledger_read_entry", "ReadException on ledgerId:0 firstEntry:100 lastEntry:100"); + } + + @Test + public void testReadFromDeletedLedger() throws Exception { + lh.addEntry(0, entry); + lh.close(); + bkc.deleteLedger(lh.ledgerId); + + MDC.put(MDC_REQUEST_ID, "ledger_read_entry"); + + try { + lh.readEntries(100, 100); + fail("should not get here"); + } catch (BKException.BKReadException e) { + // pass + } + assertLogWithMdc("ledger_read_entry", "ReadException on ledgerId:0 firstEntry:100 lastEntry:100"); + } + +} diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java index ce1f02bd57d..6ababbe324d 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java @@ -24,6 +24,8 @@ import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.util.ReferenceCountUtil; + +import java.util.Map; import java.util.Random; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.proto.BookieProtoEncoding.EnDecoder; @@ -43,9 +45,10 @@ import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.Setup; import org.openjdk.jmh.annotations.State; +import org.slf4j.MDC; /** - * Benchmarking serialization and deserilization. + * Benchmarking serialization and deserialization. */ @BenchmarkMode({ Mode.Throughput }) @OutputTimeUnit(TimeUnit.MILLISECONDS) @@ -82,7 +85,7 @@ public void prepare() { @Benchmark public void testAddEntryV2() throws Exception { - ByteBufList list = ByteBufList.get(entry.slice()); + ByteBufList list = ByteBufList.get(entry.retainedSlice()); BookieProtocol.AddRequest req = BookieProtocol.AddRequest.create( BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, @@ -122,4 +125,76 @@ public void testAddEntryV3() throws Exception { ReferenceCountUtil.release(res); } + @Benchmark + public void testAddEntryV3WithMdc() throws Exception { + MDC.put("parent_id", "LetsPutSomeLongParentRequestIdHere"); + MDC.put("request_id", "LetsPutSomeLongRequestIdHere"); + // Build the request and calculate the total size to be included in the packet. + BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.ADD_ENTRY) + .setTxnId(0L); + + ByteBuf toSend = entry.slice(); + byte[] toSendArray = new byte[toSend.readableBytes()]; + toSend.getBytes(toSend.readerIndex(), toSendArray); + AddRequest.Builder addBuilder = AddRequest.newBuilder() + .setLedgerId(ledgerId) + .setEntryId(entryId) + .setMasterKey(ByteString.copyFrom(masterKey)) + .setBody(ByteString.copyFrom(toSendArray)) + .setFlag(AddRequest.Flag.RECOVERY_ADD); + + Request request = PerChannelBookieClient.appendRequestContext(Request.newBuilder()) + .setHeader(headerBuilder) + .setAddRequest(addBuilder) + .build(); + + Object res = this.reqEnDeV3.encode(request, ByteBufAllocator.DEFAULT); + ReferenceCountUtil.release(res); + MDC.clear(); + } + + static Request.Builder appendRequestContextNoMdc(Request.Builder builder) { + final BookkeeperProtocol.ContextPair context1 = BookkeeperProtocol.ContextPair.newBuilder() + .setKey("parent_id") + .setValue("LetsPutSomeLongParentRequestIdHere") + .build(); + builder.addRequestContext(context1); + + final BookkeeperProtocol.ContextPair context2 = BookkeeperProtocol.ContextPair.newBuilder() + .setKey("request_id") + .setValue("LetsPutSomeLongRequestIdHere") + .build(); + builder.addRequestContext(context2); + + return builder; + } + + @Benchmark + public void testAddEntryV3WithExtraContextDataNoMdc() throws Exception { + // Build the request and calculate the total size to be included in the packet. + BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.ADD_ENTRY) + .setTxnId(0L); + + ByteBuf toSend = entry.slice(); + byte[] toSendArray = new byte[toSend.readableBytes()]; + toSend.getBytes(toSend.readerIndex(), toSendArray); + AddRequest.Builder addBuilder = AddRequest.newBuilder() + .setLedgerId(ledgerId) + .setEntryId(entryId) + .setMasterKey(ByteString.copyFrom(masterKey)) + .setBody(ByteString.copyFrom(toSendArray)) + .setFlag(AddRequest.Flag.RECOVERY_ADD); + + Request request = appendRequestContextNoMdc(Request.newBuilder()) + .setHeader(headerBuilder) + .setAddRequest(addBuilder) + .build(); + + Object res = this.reqEnDeV3.encode(request, ByteBufAllocator.DEFAULT); + ReferenceCountUtil.release(res); + } } From 6b7c5008f517de85df43ded94e74b111ae68df06 Mon Sep 17 00:00:00 2001 From: SongWang Date: Tue, 18 Sep 2018 17:25:03 +0800 Subject: [PATCH 0033/1642] Issue #1679 : Fix build failure due to javadoc Descriptions of the changes in this PR: ### Motivation Fix build failure due to javadoc, additionalOptions' type is an array . ### Changes correct assignment strategy of additionalOptions to avoid type mismatch Master Issue: #1679 Author: songsong Reviewers: Enrico Olivelli , Sijie Guo This closes #1681 from aCoder2013/fix-javadoc-build, closes #1679 --- pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 609411c2c5a..701b9a55e6d 100644 --- a/pom.xml +++ b/pom.xml @@ -764,7 +764,9 @@ -Xdoclint:none -notimestamp - -Xdoclint:none -notimestamp + + -Xdoclint:none -notimestamp + org.apache.bookkeeper.client:org.apache.bookkeeper.client.api:org.apache.bookkeeper.common.annotation:org.apache.bookkeeper.conf:org.apache.bookkeeper.feature:org.apache.bookkeeper.stats From d18201c989dc84ff4ca86e4ee833964550388620 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Tue, 18 Sep 2018 19:42:52 +0200 Subject: [PATCH 0034/1642] Fix license issues in master The only blocking error is that the GRPC NOTICE had not been bubbled up to our notice. The links for common-lang 3.6 and jackson 1.9.11 were also wrong, and while these aren't required in license files, bad links makes it harder to check if a dependency has been correct noted in the LICENSE and NOTICE. Author: Andrey Yegorov Author: Ivan Kelly Author: SongWang Reviewers: Sijie Guo , Enrico Olivelli This closes #1684 from ivankelly/v48-license --- .../src/main/resources/LICENSE-all.bin.txt | 73 +++++++++---------- .../src/main/resources/LICENSE-server.bin.txt | 73 +++++++++---------- .../src/main/resources/NOTICE-all.bin.txt | 48 ++++++++++++ .../src/main/resources/NOTICE-server.bin.txt | 48 ++++++++++++ 4 files changed, 168 insertions(+), 74 deletions(-) diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index 2997d7bdd6a..0b181f1e20f 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -275,30 +275,30 @@ Apache Software License, Version 2. - lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [37] - lib/net.jpountz.lz4-lz4-1.3.0.jar [38] - lib/org.codehaus.jackson-jackson-core-asl-1.9.11.jar [39] -- lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [40] -- lib/com.google.api.grpc-proto-google-common-protos-1.0.0.jar [41] -- lib/com.google.code.gson-gson-2.7.jar [42] -- lib/io.opencensus-opencensus-api-0.11.0.jar [43] -- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.11.0.jar [43] -- lib/com.squareup.okhttp-okhttp-2.5.0.jar [44] -- lib/com.squareup.okio-okio-1.13.0.jar [45] -- lib/io.grpc-grpc-all-1.12.0.jar [46] -- lib/io.grpc-grpc-auth-1.12.0.jar [46] -- lib/io.grpc-grpc-context-1.12.0.jar [46] -- lib/io.grpc-grpc-core-1.12.0.jar [46] -- lib/io.grpc-grpc-netty-1.12.0.jar [46] -- lib/io.grpc-grpc-okhttp-1.12.0.jar [46] -- lib/io.grpc-grpc-protobuf-1.12.0.jar [46] -- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar [46] -- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar [46] -- lib/io.grpc-grpc-stub-1.12.0.jar [46] -- lib/io.grpc-grpc-testing-1.12.0.jar [46] -- lib/org.apache.curator-curator-client-4.0.1.jar [47] -- lib/org.apache.curator-curator-framework-4.0.1.jar [47] -- lib/org.apache.curator-curator-recipes-4.0.1.jar [47] -- lib/org.inferred-freebuilder-1.14.9.jar [48] -- lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [49] -- lib/org.apache.yetus-audience-annotations-0.5.0.jar [50] +- lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [39] +- lib/com.google.api.grpc-proto-google-common-protos-1.0.0.jar [40] +- lib/com.google.code.gson-gson-2.7.jar [41] +- lib/io.opencensus-opencensus-api-0.11.0.jar [42] +- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.11.0.jar [42] +- lib/com.squareup.okhttp-okhttp-2.5.0.jar [43] +- lib/com.squareup.okio-okio-1.13.0.jar [44] +- lib/io.grpc-grpc-all-1.12.0.jar [45] +- lib/io.grpc-grpc-auth-1.12.0.jar [45] +- lib/io.grpc-grpc-context-1.12.0.jar [45] +- lib/io.grpc-grpc-core-1.12.0.jar [45] +- lib/io.grpc-grpc-netty-1.12.0.jar [45] +- lib/io.grpc-grpc-okhttp-1.12.0.jar [45] +- lib/io.grpc-grpc-protobuf-1.12.0.jar [45] +- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar [45] +- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar [45] +- lib/io.grpc-grpc-stub-1.12.0.jar [45] +- lib/io.grpc-grpc-testing-1.12.0.jar [45] +- lib/org.apache.curator-curator-client-4.0.1.jar [46] +- lib/org.apache.curator-curator-framework-4.0.1.jar [46] +- lib/org.apache.curator-curator-recipes-4.0.1.jar [46] +- lib/org.inferred-freebuilder-1.14.9.jar [47] +- lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [48] +- lib/org.apache.yetus-audience-annotations-0.5.0.jar [49] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9 @@ -311,7 +311,7 @@ Apache Software License, Version 2. [9] Source available at https://github.com/ben-manes/caffeine/tree/v2.3.4 [10] Source available at http://svn.apache.org/viewvc/commons/proper/configuration/tags/CONFIGURATION_1_10/ [11] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-io.git;a=tag;h=603579 -[12] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=tag;h=375459 +[12] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=shortlog;h=refs/tags/LANG_3_6 [13] Source available at http://svn.apache.org/viewvc/commons/proper/logging/tags/commons-logging-1.1.1/ [14] Source available at https://github.com/twitter/finagle/tree/finagle-6.44.0 [15] Source available at https://github.com/twitter/finagle/tree/finagle-6.43.0 @@ -337,18 +337,17 @@ Apache Software License, Version 2. [36] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48 [37] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3 [38] Source available at https://github.com/lz4/lz4-java/tree/1.3.0 -[39] Source available at https://github.com/codehaus/jackson/tree/1.9 -[40] Source available at https://github.com/codehaus/jackson/tree/1.9 -[41] Source available at https://github.com/googleapis/googleapis -[42] Source available at https://github.com/google/gson/tree/gson-parent-2.7 -[43] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.11.0 -[44] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 -[45] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 -[46] Source available at https://github.com/grpc/grpc-java/tree/v1.12.0 -[47] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 -[48] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 -[49] Source available at https://github.com/google/error-prone/tree/v2.1.2 -[50] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 +[39] Source available at https://github.com/codehaus/jackson/tree/d1c26ec538a8701cff7548009254fd6183b71873 +[40] Source available at https://github.com/googleapis/googleapis +[41] Source available at https://github.com/google/gson/tree/gson-parent-2.7 +[42] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.11.0 +[43] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 +[44] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 +[45] Source available at https://github.com/grpc/grpc-java/tree/v1.12.0 +[46] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 +[47] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 +[48] Source available at https://github.com/google/error-prone/tree/v2.1.2 +[49] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 ------------------------------------------------------------------------------------ diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index 7ed32ad92fb..6b630b4ccde 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -240,30 +240,30 @@ Apache Software License, Version 2. - lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [24] - lib/net.jpountz.lz4-lz4-1.3.0.jar [25] - lib/org.codehaus.jackson-jackson-core-asl-1.9.11.jar [26] -- lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [27] -- lib/com.google.api.grpc-proto-google-common-protos-1.0.0.jar [28] -- lib/com.google.code.gson-gson-2.7.jar [29] -- lib/io.opencensus-opencensus-api-0.11.0.jar [30] -- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.11.0.jar [30] -- lib/com.squareup.okhttp-okhttp-2.5.0.jar [31] -- lib/com.squareup.okio-okio-1.13.0.jar [32] -- lib/io.grpc-grpc-all-1.12.0.jar [33] -- lib/io.grpc-grpc-auth-1.12.0.jar [33] -- lib/io.grpc-grpc-context-1.12.0.jar [33] -- lib/io.grpc-grpc-core-1.12.0.jar [33] -- lib/io.grpc-grpc-netty-1.12.0.jar [33] -- lib/io.grpc-grpc-okhttp-1.12.0.jar [33] -- lib/io.grpc-grpc-protobuf-1.12.0.jar [33] -- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar [33] -- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar [33] -- lib/io.grpc-grpc-stub-1.12.0.jar [33] -- lib/io.grpc-grpc-testing-1.12.0.jar [33] -- lib/org.apache.curator-curator-client-4.0.1.jar [34] -- lib/org.apache.curator-curator-framework-4.0.1.jar [34] -- lib/org.apache.curator-curator-recipes-4.0.1.jar [34] -- lib/org.inferred-freebuilder-1.14.9.jar [35] -- lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [36] -- lib/org.apache.yetus-audience-annotations-0.5.0.jar [37] +- lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [26] +- lib/com.google.api.grpc-proto-google-common-protos-1.0.0.jar [27] +- lib/com.google.code.gson-gson-2.7.jar [28] +- lib/io.opencensus-opencensus-api-0.11.0.jar [29] +- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.11.0.jar [29] +- lib/com.squareup.okhttp-okhttp-2.5.0.jar [30] +- lib/com.squareup.okio-okio-1.13.0.jar [31] +- lib/io.grpc-grpc-all-1.12.0.jar [32] +- lib/io.grpc-grpc-auth-1.12.0.jar [32] +- lib/io.grpc-grpc-context-1.12.0.jar [32] +- lib/io.grpc-grpc-core-1.12.0.jar [32] +- lib/io.grpc-grpc-netty-1.12.0.jar [32] +- lib/io.grpc-grpc-okhttp-1.12.0.jar [32] +- lib/io.grpc-grpc-protobuf-1.12.0.jar [32] +- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar [32] +- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar [32] +- lib/io.grpc-grpc-stub-1.12.0.jar [32] +- lib/io.grpc-grpc-testing-1.12.0.jar [32] +- lib/org.apache.curator-curator-client-4.0.1.jar [33] +- lib/org.apache.curator-curator-framework-4.0.1.jar [33] +- lib/org.apache.curator-curator-recipes-4.0.1.jar [33] +- lib/org.inferred-freebuilder-1.14.9.jar [34] +- lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [35] +- lib/org.apache.yetus-audience-annotations-0.5.0.jar [36] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9 @@ -283,25 +283,24 @@ Apache Software License, Version 2. [16] Source available at http://logging.apache.org/log4j/1.2/download.html [17] Source available at https://github.com/java-native-access/jna/tree/3.2.7 [18] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad -[19] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=tag;h=3ad2e8 +[19] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=shortlog;h=refs/tags/LANG_3_6 [20] Source available at https://github.com/apache/zookeeper/tree/release-3.4.13 [21] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502 [22] Source available at https://github.com/facebook/rocksdb/tree/v5.13.1 [23] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48 [24] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3 [25] Source available at https://github.com/lz4/lz4-java/tree/1.3.0 -[26] Source available at https://github.com/codehaus/jackson/tree/1.9 -[27] Source available at https://github.com/codehaus/jackson/tree/1.9 -[28] Source available at https://github.com/googleapis/googleapis -[29] Source available at https://github.com/google/gson/tree/gson-parent-2.7 -[30] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.11.0 -[31] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 -[32] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 -[33] Source available at https://github.com/grpc/grpc-java/tree/v1.12.0 -[34] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 -[35] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 -[36] Source available at https://github.com/google/error-prone/tree/v2.1.2 -[37] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 +[26] Source available at https://github.com/codehaus/jackson/tree/d1c26ec538a8701cff7548009254fd6183b71873 +[27] Source available at https://github.com/googleapis/googleapis +[28] Source available at https://github.com/google/gson/tree/gson-parent-2.7 +[29] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.11.0 +[30] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 +[31] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 +[32] Source available at https://github.com/grpc/grpc-java/tree/v1.12.0 +[33] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 +[34] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 +[35] Source available at https://github.com/google/error-prone/tree/v2.1.2 +[36] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 ------------------------------------------------------------------------------------ lib/io.netty-netty-all-4.1.22.Final.jar bundles some 3rd party dependencies diff --git a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt index d9984b6908b..dea8a3f9ae2 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt @@ -127,3 +127,51 @@ granted provided that the copyright notice appears in all copies. Copyright 2010 Cedric Beust cedric@beust.com ------------------------------------------------------------------------------------ +- lib/io.grpc-grpc-all-1.12.0.jar +- lib/io.grpc-grpc-auth-1.12.0.jar +- lib/io.grpc-grpc-context-1.12.0.jar +- lib/io.grpc-grpc-core-1.12.0.jar +- lib/io.grpc-grpc-netty-1.12.0.jar +- lib/io.grpc-grpc-okhttp-1.12.0.jar +- lib/io.grpc-grpc-protobuf-1.12.0.jar +- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar +- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar +- lib/io.grpc-grpc-stub-1.12.0.jar +- lib/io.grpc-grpc-testing-1.12.0.jar + +Copyright 2014, gRPC Authors All rights reserved. + +Licensed 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. + +This product contains a modified portion of 'OkHttp', an open source +HTTP & SPDY client for Android and Java applications, which can be obtained +at: + + * LICENSE: + * okhttp/third_party/okhttp/LICENSE (Apache License 2.0) + * HOMEPAGE: + * https://github.com/square/okhttp + * LOCATION_IN_GRPC: + * okhttp/third_party/okhttp + +This product contains a modified portion of 'Netty', an open source +networking library, which can be obtained at: + + * LICENSE: + * netty/third_party/netty/LICENSE.txt (Apache License 2.0) + * HOMEPAGE: + * https://netty.io + * LOCATION_IN_GRPC: + * netty/third_party/netty + +------------------------------------------------------------------------------------ diff --git a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt index 1fd2f53ba3b..d2f4f403f99 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt @@ -87,3 +87,51 @@ granted provided that the copyright notice appears in all copies. Copyright 2010 Cedric Beust cedric@beust.com ------------------------------------------------------------------------------------ +- lib/io.grpc-grpc-all-1.12.0.jar +- lib/io.grpc-grpc-auth-1.12.0.jar +- lib/io.grpc-grpc-context-1.12.0.jar +- lib/io.grpc-grpc-core-1.12.0.jar +- lib/io.grpc-grpc-netty-1.12.0.jar +- lib/io.grpc-grpc-okhttp-1.12.0.jar +- lib/io.grpc-grpc-protobuf-1.12.0.jar +- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar +- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar +- lib/io.grpc-grpc-stub-1.12.0.jar +- lib/io.grpc-grpc-testing-1.12.0.jar + +Copyright 2014, gRPC Authors All rights reserved. + +Licensed 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. + +This product contains a modified portion of 'OkHttp', an open source +HTTP & SPDY client for Android and Java applications, which can be obtained +at: + + * LICENSE: + * okhttp/third_party/okhttp/LICENSE (Apache License 2.0) + * HOMEPAGE: + * https://github.com/square/okhttp + * LOCATION_IN_GRPC: + * okhttp/third_party/okhttp + +This product contains a modified portion of 'Netty', an open source +networking library, which can be obtained at: + + * LICENSE: + * netty/third_party/netty/LICENSE.txt (Apache License 2.0) + * HOMEPAGE: + * https://netty.io + * LOCATION_IN_GRPC: + * netty/third_party/netty + +------------------------------------------------------------------------------------ From 467c067eeaa45b64ca4c8da917a1e2ebe09d0e0d Mon Sep 17 00:00:00 2001 From: Mate Varga Date: Thu, 20 Sep 2018 10:08:33 +0200 Subject: [PATCH 0035/1642] ISSUE #1687: Using eval instead of exec to handle quotes in healtcheck.sh Descriptions of the changes in this PR: Fixes https://github.com/apache/bookkeeper/issues/1687 ### Motivation https://github.com/apache/bookkeeper/issues/1687 ### Changes Using eval instead of exec to run the health check script. Master Issue: #1687 Author: matevarga Reviewers: Sijie Guo , Enrico Olivelli This closes #1688 from matevarga/master, closes #1687 --- docker/scripts/healthcheck.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/scripts/healthcheck.sh b/docker/scripts/healthcheck.sh index 47a21bf50a6..943be2669c7 100755 --- a/docker/scripts/healthcheck.sh +++ b/docker/scripts/healthcheck.sh @@ -31,4 +31,4 @@ DEFAULT_HEALTH_CHECK_CMD="/opt/bookkeeper/bin/bookkeeper shell bookiesanity" HEALTH_CHECK_CMD=${HEALTH_CHECK_CMD:-"${DEFAULT_HEALTH_CHECK_CMD}"} -exec "${HEALTH_CHECK_CMD}" +eval "${HEALTH_CHECK_CMD}" From 35b0f10ad00737a83b82d33053bcff96571b079d Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 20 Sep 2018 10:04:51 -0700 Subject: [PATCH 0036/1642] Shutdown appropriate EntryLogger instance. Descriptions of the changes in this PR: - EntryLogger's instance shutdown method should be called - Otherwise test would flap, since in tearDown method, FileUtils.deleteDirectory can fail because of EntryLogFile PreAllocation task. Author: cguttapalem Reviewers: Enrico Olivelli , Sijie Guo This closes #1686 from reddycharan/entrylogtest --- .../test/java/org/apache/bookkeeper/bookie/EntryLogTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java index 9694fae3e98..9062261c1f2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java @@ -1076,6 +1076,7 @@ public void testEntryLogManagerExpiryRemoval() throws Exception { */ @Test public void testCacheMaximumSizeEvictionPolicy() throws Exception { + entryLogger.shutdown(); final int cacheMaximumSize = 20; ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); @@ -1086,7 +1087,7 @@ public void testCacheMaximumSizeEvictionPolicy() throws Exception { LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); - EntryLogger entryLogger = new EntryLogger(conf, ledgerDirsManager); + entryLogger = new EntryLogger(conf, ledgerDirsManager); EntryLogManagerForEntryLogPerLedger entryLogManager = (EntryLogManagerForEntryLogPerLedger) entryLogger.getEntryLogManager(); From 62cb1be11f2064cdf35919081e6f763276b22933 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 20 Sep 2018 10:09:30 -0700 Subject: [PATCH 0037/1642] [TABLE SERVICE] remove build warnings "Import xxx.proto but not used" Descriptions of the changes in this PR: *Motivation* There are build warnings about `Import xyz.proto but not used`. for example: ``` cluster.proto: warning: Import common.proto but not used. kv_store.proto: warning: Import kv.proto but not used. ``` *Changes* Remove unused imports Author: Sijie Guo Reviewers: Enrico Olivelli This closes #1685 from sijie/remove_warnings --- stream/proto/src/main/proto/cluster.proto | 2 -- stream/proto/src/main/proto/kv_store.proto | 1 - stream/proto/src/main/proto/stream.proto | 2 -- 3 files changed, 5 deletions(-) diff --git a/stream/proto/src/main/proto/cluster.proto b/stream/proto/src/main/proto/cluster.proto index 1454071ab6c..20e9d6ddb6d 100644 --- a/stream/proto/src/main/proto/cluster.proto +++ b/stream/proto/src/main/proto/cluster.proto @@ -17,8 +17,6 @@ */ syntax = "proto3"; -import "common.proto"; - package bookkeeper.proto.cluster; option java_multiple_files = true; diff --git a/stream/proto/src/main/proto/kv_store.proto b/stream/proto/src/main/proto/kv_store.proto index fbaa58f2ea2..46257639ce7 100644 --- a/stream/proto/src/main/proto/kv_store.proto +++ b/stream/proto/src/main/proto/kv_store.proto @@ -17,7 +17,6 @@ */ syntax = "proto3"; -import "kv.proto"; import "kv_rpc.proto"; package bookkeeper.proto.kv.store; diff --git a/stream/proto/src/main/proto/stream.proto b/stream/proto/src/main/proto/stream.proto index 150e30f98dd..7a59868d42d 100644 --- a/stream/proto/src/main/proto/stream.proto +++ b/stream/proto/src/main/proto/stream.proto @@ -17,8 +17,6 @@ */ syntax = "proto3"; -import "common.proto"; - package bookkeeper.proto.stream; option java_multiple_files = true; From f0169ff1c97fca84eb255cfadec49cceeb0c51d1 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Fri, 21 Sep 2018 08:28:51 -0700 Subject: [PATCH 0038/1642] Bump journal and fileinfo version. Descriptions of the changes in this PR: - As part of ISSUE #1527, new versions of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are introduced. Since 4.8 release version is created, default values of these versions should be bumped. Author: cguttapalem Reviewers: Sijie Guo , Enrico Olivelli This closes #1689 from reddycharan/bumpjournalversion --- .../bookkeeper/conf/ServerConfiguration.java | 4 ++-- conf/bk_server.conf | 22 +++++++++---------- site/_data/config/bk_server.yaml | 10 ++++----- 3 files changed, 18 insertions(+), 18 deletions(-) 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 db40e0f5021..2475084677d 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 @@ -554,7 +554,7 @@ public ServerConfiguration setFileInfoMaxIdleTime(long idleTime) { * @return fileinfo format version to write. */ public int getFileInfoFormatVersionToWrite() { - return this.getInt(FILEINFO_FORMAT_VERSION_TO_WRITE, 0); + return this.getInt(FILEINFO_FORMAT_VERSION_TO_WRITE, 1); } /** @@ -657,7 +657,7 @@ public ServerConfiguration setJournalAlignmentSize(int size) { * @return journal format version to write. */ public int getJournalFormatVersionToWrite() { - return this.getInt(JOURNAL_FORMAT_VERSION_TO_WRITE, 4); + return this.getInt(JOURNAL_FORMAT_VERSION_TO_WRITE, 6); } /** diff --git a/conf/bk_server.conf b/conf/bk_server.conf index 1eaa1914e04..93c12a142cd 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -272,20 +272,19 @@ journalDirectories=/tmp/bk-txn # journalDirectory=/tmp/bk-txn # The journal format version to write. -# Available formats are 1-5: +# Available formats are 1-6: # 1: no header # 2: a header section was added # 3: ledger key was introduced # 4: fencing key was introduced # 5: expanding header to 512 and padding writes to align sector size configured by `journalAlignmentSize` # 6: persisting explicitLac is introduced -# By default, it is `4`. If you'd like to enable `padding-writes` feature, you can set journal version to `5`. +# By default, it is `6`. +# If you'd like to disable persisting ExplicitLac, you can set this config to < `6` and also +# fileInfoFormatVersionToWrite should be set to 0. If there is mismatch then the serverconfig is considered invalid. # You can disable `padding-writes` by setting journal version back to `4`. This feature is available in 4.5.0 # and onward versions. -# If you'd like to enable persisting ExplicitLac, you can set this config to 6 and also -# fileInfoFormatVersionToWrite should be atleast 1. If there is mismatch then the serverconfig is considered -# invalid. -# journalFormatVersionToWrite=4 +# journalFormatVersionToWrite=6 # Max file size of journal file, in mega bytes # A new journal file will be created when the old one reaches the file size limitation @@ -445,7 +444,7 @@ ledgerDirectories=/tmp/bk-data # active at a given point in time # maximumNumberOfActiveEntryLogs=500 -# in EntryLogManagerForEntryLogPerLedger, this config value specifies the metrics cache size +# in EntryLogManagerForEntryLogPerLedger, this config value specifies the metrics cache size # limits in multiples of entrylogMap cache size limits. # entryLogPerLedgerCounterLimitsMultFactor=10 @@ -600,10 +599,11 @@ ledgerDirectories=/tmp/bk-data # Available formats are 0-1: # 0: Initial version # 1: persisting explicitLac is introduced -# By default, it is `0`. If you'd like to enable persisting ExplicitLac, you can set -# this config to 1 and also journalFormatVersionToWrite should be atleast 6. If -# there is mismatch then the serverconfig is considered invalid. -# fileInfoFormatVersionToWrite = 0 +# By default, it is `1`. +# If you'd like to disable persisting ExplicitLac, you can set this config to 0 and +# also journalFormatVersionToWrite should be set to < 6. If there is mismatch then the +# serverconfig is considered invalid. +# fileInfoFormatVersionToWrite = 1 # Size of a index page in ledger cache, in bytes # A larger index page can improve performance writing page to disk, diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index fd4f9a2ceca..991d6f1fceb 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -200,10 +200,10 @@ groups: 5: expanding header to 512 and padding writes to align sector size configured by `journalAlignmentSize` 6: persisting explicitLac is introduced - By default, it is `4`. If you'd like to enable `padding-writes` feature, you can set journal version to `5`. + By default, it is `6`. + If you'd like to disable persisting ExplicitLac, you can set this config to < `6` and also fileInfoFormatVersionToWrite should be set to 0. If there is mismatch then the serverconfig is considered invalid. You can disable `padding-writes` by setting journal version back to `4`. This feature is available in 4.5.0 and onward versions. - If you'd like to enable persisting ExplicitLac, you can set this config to 6 and also fileInfoFormatVersionToWrite should be atleast 1. If there is mismatch then the serverconfig is considered invalid. - default: 4 + default: 6 - param: journalMaxSizeMB description: Max file size of journal file, in mega bytes. A new journal file will be created when the old one reaches the file size limitation. default: 2048 @@ -422,8 +422,8 @@ groups: 0: Initial version 1: persisting explicitLac is introduced - By default, it is `0`. If you'd like to enable persisting ExplicitLac, you can set this config to 1 and also journalFormatVersionToWrite should be atleast 6. If there is mismatch then the serverconfig is considered invalid. - default: 0 + By default, it is `1`. If you'd like to disable persisting ExplicitLac, you can set this config to 0 and also journalFormatVersionToWrite should be set to < 6. If there is mismatch then the serverconfig is considered invalid. + default: 1 - param: pageSize description: | Size of a index page in ledger cache, in bytes. A larger index page can improve performance writing page to disk, which is efficent when you have small number of ledgers and these ledgers have similar number of entries. If you have large number of ledgers and each ledger has fewer entries, smaller index page would improve memory usage. From 3a93b79ee1bf693e3a92c0dc5fea642ba2f51c07 Mon Sep 17 00:00:00 2001 From: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Date: Sun, 23 Sep 2018 16:37:58 -0700 Subject: [PATCH 0039/1642] Issue #1639: Etcd based metadata driver implementation Descriptions of the changes in this PR: Provide a metadata driver based on Etcd. The changes were based on sijie's prototype at #1114 Master Issue: #1639 Author: Qi Wang Reviewers: Sijie Guo This closes #1640 from codingwangqi/etcd_metadata_driver, closes #1639 --- metadata-drivers/etcd/pom.xml | 65 ++ .../jetcd/internal/impl/EtcdClientUtils.java | 37 ++ .../internal/impl/EtcdConnectionManager.java | 64 ++ .../jetcd/internal/impl/package-info.java | 22 + .../metadata/etcd/Etcd64bitIdGenerator.java | 141 +++++ .../metadata/etcd/EtcdBookieRegister.java | 188 ++++++ .../metadata/etcd/EtcdConstants.java | 53 ++ .../metadata/etcd/EtcdLayoutManager.java | 108 ++++ .../metadata/etcd/EtcdLedgerManager.java | 457 ++++++++++++++ .../etcd/EtcdLedgerManagerFactory.java | 121 ++++ .../etcd/EtcdMetadataBookieDriver.java | 94 +++ .../etcd/EtcdMetadataClientDriver.java | 75 +++ .../metadata/etcd/EtcdMetadataDriverBase.java | 133 ++++ .../metadata/etcd/EtcdRegistrationClient.java | 169 ++++++ .../etcd/EtcdRegistrationManager.java | 569 ++++++++++++++++++ .../bookkeeper/metadata/etcd/EtcdUtils.java | 211 +++++++ .../metadata/etcd/EtcdWatchClient.java | 400 ++++++++++++ .../bookkeeper/metadata/etcd/EtcdWatcher.java | 115 ++++ .../metadata/etcd/LedgerMetadataConsumer.java | 72 +++ .../metadata/etcd/helpers/KeyIterator.java | 68 +++ .../metadata/etcd/helpers/KeySetReader.java | 282 +++++++++ .../metadata/etcd/helpers/KeyStream.java | 120 ++++ .../etcd/helpers/RevisionedConsumer.java | 67 +++ .../metadata/etcd/helpers/ValueStream.java | 310 ++++++++++ .../metadata/etcd/helpers/package-info.java | 22 + .../metadata/etcd/package-info.java | 22 + .../etcd/Etcd64bitIdGeneratorTest.java | 153 +++++ .../metadata/etcd/EtcdClusterTest.java | 341 +++++++++++ .../metadata/etcd/EtcdCookieTest.java | 163 +++++ .../metadata/etcd/EtcdLayoutManagerTest.java | 104 ++++ .../metadata/etcd/EtcdLedgerManagerTest.java | 412 +++++++++++++ .../metadata/etcd/EtcdRegistrationTest.java | 416 +++++++++++++ .../metadata/etcd/helpers/HelpersTest.java | 210 +++++++ .../etcd/helpers/KeySetReaderTest.java | 414 +++++++++++++ .../metadata/etcd/integration/SmokeTest.java | 105 ++++ .../etcd/testing/EtcdBKClusterTestBase.java | 166 +++++ .../metadata/etcd/testing/EtcdContainer.java | 141 +++++ .../metadata/etcd/testing/EtcdTestBase.java | 91 +++ .../etcd/src/test/resources/log4j.properties | 39 ++ metadata-drivers/pom.xml | 34 ++ pom.xml | 2 + 41 files changed, 6776 insertions(+) create mode 100644 metadata-drivers/etcd/pom.xml create mode 100644 metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/EtcdClientUtils.java create mode 100644 metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/EtcdConnectionManager.java create mode 100644 metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/package-info.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/Etcd64bitIdGenerator.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdBookieRegister.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdConstants.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLayoutManager.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataBookieDriver.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataClientDriver.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdUtils.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdWatchClient.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdWatcher.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeyIterator.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeySetReader.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeyStream.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/RevisionedConsumer.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/ValueStream.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/package-info.java create mode 100644 metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/package-info.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/Etcd64bitIdGeneratorTest.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdClusterTest.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdCookieTest.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLayoutManagerTest.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/helpers/HelpersTest.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/helpers/KeySetReaderTest.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/integration/SmokeTest.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdContainer.java create mode 100644 metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdTestBase.java create mode 100644 metadata-drivers/etcd/src/test/resources/log4j.properties create mode 100644 metadata-drivers/pom.xml diff --git a/metadata-drivers/etcd/pom.xml b/metadata-drivers/etcd/pom.xml new file mode 100644 index 00000000000..9e463bd89b8 --- /dev/null +++ b/metadata-drivers/etcd/pom.xml @@ -0,0 +1,65 @@ + + + + + org.apache.bookkeeper.metadata.drivers + metadata-drivers-parent + 4.9.0-SNAPSHOT + .. + + 4.0.0 + org.apache.bookkeeper.metadata.drivers + metadata-stores-etcd + Apache BookKeeper :: Metadata Drivers:: Etcd + + + org.apache.bookkeeper + bookkeeper-server + ${project.version} + + + com.coreos + jetcd-core + ${etcd.version} + + + io.grpc + * + + + + + io.grpc + grpc-all + + + org.testcontainers + testcontainers + test + + + org.apache.bookkeeper + bookkeeper-server + ${project.version} + test-jar + test + + + diff --git a/metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/EtcdClientUtils.java b/metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/EtcdClientUtils.java new file mode 100644 index 00000000000..0c414218c16 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/EtcdClientUtils.java @@ -0,0 +1,37 @@ +/* + * 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 com.coreos.jetcd.internal.impl; + +import java.lang.reflect.Field; + +/** + * Utils to access fields in Etcd client. + */ +class EtcdClientUtils { + + @SuppressWarnings("unchecked") + static T getField(Object obj, String fieldName) + throws NoSuchFieldException, IllegalAccessException { + Class cls = obj.getClass(); + Field field = cls.getDeclaredField(fieldName); + field.setAccessible(true); + return (T) field.get(obj); + } + +} diff --git a/metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/EtcdConnectionManager.java b/metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/EtcdConnectionManager.java new file mode 100644 index 00000000000..6428f5c96ad --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/EtcdConnectionManager.java @@ -0,0 +1,64 @@ +/* + * 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 com.coreos.jetcd.internal.impl; + +import com.coreos.jetcd.Client; +import com.coreos.jetcd.api.WatchGrpc; +import lombok.extern.slf4j.Slf4j; + +/** + * Keep a reference to etcd internal connection manager. + */ +@Slf4j +public class EtcdConnectionManager { + + private final ClientImpl client; + private ClientConnectionManager connMgr; + + public EtcdConnectionManager(Client client) { + this((ClientImpl) client); + } + + EtcdConnectionManager(ClientImpl client) { + this.client = client; + try { + this.connMgr = EtcdClientUtils.getField( + client, "connectionManager" + ); + } catch (NoSuchFieldException e) { + log.error("No `connectionManager` field found in etcd client", e); + throw new RuntimeException( + "No `connectionManager` field found in etcd client", e); + } catch (IllegalAccessException e) { + log.error("Illegal access to `connectionManager` field in etcd client", e); + throw new RuntimeException( + "Illegal access to `connectionManager` field in etcd client", e); + } + } + + /** + * Create a watch api grpc stub. + * + * @return a watch api grpc stub. + */ + public WatchGrpc.WatchStub newWatchStub() { + return connMgr.newStub(WatchGrpc::newStub); + } + +} diff --git a/metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/package-info.java b/metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/package-info.java new file mode 100644 index 00000000000..44d9109b3c9 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/com/coreos/jetcd/internal/impl/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. + */ + +/** + * Etcd client changes for bookkeeper metadata driver. + */ +package com.coreos.jetcd.internal.impl; \ No newline at end of file diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/Etcd64bitIdGenerator.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/Etcd64bitIdGenerator.java new file mode 100644 index 00000000000..f21cb811b09 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/Etcd64bitIdGenerator.java @@ -0,0 +1,141 @@ +/* + * 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.metadata.etcd; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.EMPTY_BS; + +import com.coreos.jetcd.KV; +import com.coreos.jetcd.Txn; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.data.KeyValue; +import com.coreos.jetcd.kv.GetResponse; +import com.coreos.jetcd.op.Cmp; +import com.coreos.jetcd.op.Cmp.Op; +import com.coreos.jetcd.op.CmpTarget; +import com.coreos.jetcd.options.GetOption; +import com.coreos.jetcd.options.PutOption; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BKException.Code; +import org.apache.bookkeeper.meta.LedgerIdGenerator; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; + +/** + * Generate 64-bit ledger ids from a bucket. + * + *

The most significant 8 bits is used as bucket id. The remaining 56 bits are + * used as the id generated per bucket. + */ +@Slf4j +class Etcd64bitIdGenerator implements LedgerIdGenerator { + + static final long MAX_ID_PER_BUCKET = 0x00ffffffffffffffL; + static final long BUCKET_ID_MASK = 0xff00000000000000L; + static final int BUCKET_ID_SHIFT = 56; + static final int NUM_BUCKETS = 0x80; + + static int getBucketId(long lid) { + return (int) ((lid & BUCKET_ID_MASK) >>> BUCKET_ID_SHIFT); + } + + static long getIdInBucket(long lid) { + return lid & MAX_ID_PER_BUCKET; + } + + private static final AtomicIntegerFieldUpdater nextBucketIdUpdater = + AtomicIntegerFieldUpdater.newUpdater(Etcd64bitIdGenerator.class, "nextBucketId"); + + private final String scope; + private final KV kvClient; + private volatile int nextBucketId; + + Etcd64bitIdGenerator(KV kvClient, String scope) { + this.kvClient = kvClient; + this.scope = scope; + this.nextBucketId = ThreadLocalRandom.current().nextInt(NUM_BUCKETS); + } + + int nextBucketId() { + while (true) { + int bucketId = nextBucketIdUpdater.incrementAndGet(this); + if (bucketId >= NUM_BUCKETS) { + if (nextBucketIdUpdater.compareAndSet(this, bucketId, 0)) { + bucketId = 0; + } else { + // someone has been updated bucketId, try it again. + continue; + } + } + return bucketId; + } + } + + @Override + public void generateLedgerId(GenericCallback cb) { + int bucketId = nextBucketId(); + checkArgument(bucketId >= 0 && bucketId < NUM_BUCKETS, + "Invalid bucket id : " + bucketId); + + ByteSequence bucketKey = ByteSequence.fromString(EtcdUtils.getBucketPath(scope, bucketId)); + Txn txn = kvClient.txn() + .If(new Cmp(bucketKey, Op.GREATER, CmpTarget.createRevision(0))) + .Then( + com.coreos.jetcd.op.Op.put(bucketKey, EMPTY_BS, PutOption.DEFAULT), + com.coreos.jetcd.op.Op.get(bucketKey, GetOption.DEFAULT) + ) + .Else( + com.coreos.jetcd.op.Op.put(bucketKey, EMPTY_BS, PutOption.DEFAULT), + com.coreos.jetcd.op.Op.get(bucketKey, GetOption.DEFAULT) + ); + txn.commit() + .thenAccept(txnResponse -> { + if (txnResponse.getGetResponses().size() <= 0) { + cb.operationComplete(Code.UnexpectedConditionException, null); + } else { + GetResponse resp = txnResponse.getGetResponses().get(0); + if (resp.getCount() > 0) { + KeyValue kv = resp.getKvs().get(0); + if (kv.getVersion() > MAX_ID_PER_BUCKET) { + log.warn("Etcd bucket '{}' is overflowed", bucketKey.toStringUtf8()); + // the bucket is overflowed, moved to next bucket. + generateLedgerId(cb); + } else { + long version = kv.getVersion(); + long lid = ((((long) bucketId) << BUCKET_ID_SHIFT) & BUCKET_ID_MASK) + | (version & MAX_ID_PER_BUCKET); + cb.operationComplete(Code.OK, lid); + } + } else { + cb.operationComplete(Code.UnexpectedConditionException, null); + } + } + }) + .exceptionally(cause -> { + cb.operationComplete(Code.MetaStoreException, null); + return null; + }); + } + + @Override + public void close() { + // no-op + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdBookieRegister.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdBookieRegister.java new file mode 100644 index 00000000000..9c16ef0c7ee --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdBookieRegister.java @@ -0,0 +1,188 @@ +/* + * 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.metadata.etcd; + +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.msResult; + +import com.coreos.jetcd.Lease; +import com.coreos.jetcd.Lease.KeepAliveListener; +import com.coreos.jetcd.common.exception.EtcdException; +import com.coreos.jetcd.lease.LeaseKeepAliveResponse; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; +import lombok.AccessLevel; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.discover.RegistrationManager.RegistrationListener; + +/** + * Register to register a bookie in Etcd. + */ +@Slf4j +class EtcdBookieRegister implements AutoCloseable, Runnable, Supplier { + + private final Lease leaseClient; + private final long ttlSeconds; + private final ScheduledExecutorService executor; + private final RegistrationListener regListener; + private volatile CompletableFuture leaseFuture = new CompletableFuture<>(); + @Getter(AccessLevel.PACKAGE) + private volatile long leaseId = -0xabcd; + private volatile KeepAliveListener kaListener = null; + private volatile boolean running = true; + private long nextWaitTimeMs = 200; + private Future runFuture = null; + + EtcdBookieRegister(Lease leaseClient, + long ttlSeconds, + RegistrationListener regListener) { + this.regListener = regListener; + this.leaseClient = leaseClient; + this.ttlSeconds = ttlSeconds; + this.executor = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat("bookie-etcd-keepalive-thread") + .build()); + } + + long getTtlSeconds() { + return ttlSeconds; + } + + public synchronized EtcdBookieRegister start() { + if (null == runFuture) { + runFuture = executor.submit(this); + } + return this; + } + + private void newLeaseIfNeeded() throws MetadataStoreException { + boolean newLeaseNeeded; + synchronized (this) { + newLeaseNeeded = !leaseFuture.isDone(); + } + if (newLeaseNeeded) { + long leaseId = msResult(leaseClient.grant(ttlSeconds)).getID(); + this.kaListener = leaseClient.keepAlive(leaseId); + this.leaseId = leaseId; + leaseFuture.complete(leaseId); + log.info("New lease '{}' is granted.", leaseId); + } + } + + private void waitForNewLeaseId() { + while (running) { + try { + newLeaseIfNeeded(); + nextWaitTimeMs = 100L; + } catch (MetadataStoreException e) { + log.error("Failed to grant a new lease", e); + try { + TimeUnit.MILLISECONDS.sleep(nextWaitTimeMs); + nextWaitTimeMs *= 2; + nextWaitTimeMs = Math.min(nextWaitTimeMs, TimeUnit.SECONDS.toMillis(ttlSeconds)); + } catch (InterruptedException e1) { + Thread.currentThread().interrupt(); + log.warn("Interrupted at backing off granting a new lease"); + } + continue; + } + } + } + + @Override + public void run() { + while (running) { + waitForNewLeaseId(); + // here we get a lease, keep it alive + try { + log.info("Keeping Alive at lease = {}", get()); + LeaseKeepAliveResponse kaResponse = kaListener.listen(); + log.info("KeepAlive response : lease = {}, ttl = {}", + kaResponse.getID(), kaResponse.getTTL()); + continue; + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + log.warn("Interrupted at keeping lease '{}' alive", leaseId); + resetLease(); + } catch (EtcdException ee) { + log.warn("Failed to keep alive lease '{}'", leaseId, ee); + resetLease(); + } + } + } + + private void resetLease() { + synchronized (this) { + leaseFuture = new CompletableFuture<>(); + } + kaListener.close(); + if (null != regListener) { + regListener.onRegistrationExpired(); + } + } + + @Override + public void close() { + synchronized (this) { + if (!running) { + return; + } else { + running = false; + } + if (null != runFuture) { + if (runFuture.cancel(true)) { + log.info("Successfully interrupted bookie register."); + } + } + } + CompletableFuture closeFuture = new CompletableFuture<>(); + executor.submit(() -> { + FutureUtils.complete(closeFuture, (Void) null); + }); + closeFuture.join(); + } + + @Override + public Long get() { + while (true) { + try { + return leaseFuture.get(100, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + log.warn("Interrupted at getting lease id", e); + return -1L; + } catch (ExecutionException e) { + throw new IllegalArgumentException("Should never reach here"); + } catch (TimeoutException e) { + continue; + } + + } + } + +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdConstants.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdConstants.java new file mode 100644 index 00000000000..fed70b51e4c --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdConstants.java @@ -0,0 +1,53 @@ +/** + * 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.metadata.etcd; + +import com.coreos.jetcd.data.ByteSequence; + +/** + * Constants used in the Etcd metadata drivers. + */ +final class EtcdConstants { + + private EtcdConstants() {} + + public static final String END_SEP = "0"; + + public static final String LAYOUT_NODE = "layout"; + public static final String INSTANCEID_NODE = "instanceid"; + public static final String COOKIES_NODE = "cookies"; + public static final String LEDGERS_NODE = "ledgers"; + public static final String BUCKETS_NODE = "buckets"; + + // + // membership related constants + // + + public static final String MEMBERS_NODE = "bookies"; + public static final String WRITEABLE_NODE = "writable"; + public static final String READONLY_NODE = "readonly"; + + // + // underreplication related constants + // + + public static final String UR_NODE = "underreplication"; + + public static final ByteSequence EMPTY_BS = ByteSequence.fromBytes(new byte[0]); + +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLayoutManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLayoutManager.java new file mode 100644 index 00000000000..a20886d4d93 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLayoutManager.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.metadata.etcd; + +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.ioResult; + +import com.coreos.jetcd.Client; +import com.coreos.jetcd.KV; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.kv.DeleteResponse; +import com.coreos.jetcd.kv.GetResponse; +import com.coreos.jetcd.kv.TxnResponse; +import com.coreos.jetcd.op.Cmp; +import com.coreos.jetcd.op.Cmp.Op; +import com.coreos.jetcd.op.CmpTarget; +import com.coreos.jetcd.options.GetOption; +import com.coreos.jetcd.options.PutOption; +import java.io.IOException; +import lombok.AccessLevel; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.meta.LayoutManager; +import org.apache.bookkeeper.meta.LedgerLayout; + +/** + * Etcd based layout manager. + */ +@Slf4j +@Getter(AccessLevel.PACKAGE) +class EtcdLayoutManager implements LayoutManager { + + private final Client client; + private final KV kvClient; + private final String scope; + private final ByteSequence layoutKey; + + EtcdLayoutManager(Client client, String scope) { + this.client = client; + this.kvClient = client.getKVClient(); + this.scope = scope; + this.layoutKey = ByteSequence.fromString(EtcdUtils.getLayoutKey(scope)); + } + + @Override + public LedgerLayout readLedgerLayout() throws IOException { + GetResponse response = ioResult(kvClient.get(layoutKey, GetOption.DEFAULT)); + if (response.getCount() <= 0) { + return null; + } else { + byte[] layoutData = response.getKvs().get(0).getValue().getBytes(); + return LedgerLayout.parseLayout(layoutData); + } + } + + @Override + public void storeLedgerLayout(LedgerLayout layout) throws IOException { + ByteSequence layoutData = ByteSequence.fromBytes(layout.serialize()); + TxnResponse response = ioResult(kvClient.txn() + .If(new Cmp(layoutKey, Op.GREATER, CmpTarget.createRevision(0))) + .Then(com.coreos.jetcd.op.Op.get(layoutKey, GetOption.DEFAULT)) + .Else(com.coreos.jetcd.op.Op.put(layoutKey, layoutData, PutOption.DEFAULT)) + .commit()); + // key doesn't exist and we created the key + if (!response.isSucceeded()) { + return; + // key exists and we retrieved the key + } else { + GetResponse resp = response.getGetResponses().get(0); + if (resp.getCount() <= 0) { + // fail to put key/value but key is not found + throw new IOException("Creating layout node '" + layoutKey.toStringUtf8() + + "' failed due to it already exists but no layout node is found"); + } else { + throw new LedgerLayoutExistsException( + "Ledger layout already exists under '" + layoutKey.toStringUtf8() + "'"); + } + } + } + + @Override + public void deleteLedgerLayout() throws IOException { + DeleteResponse response = ioResult(kvClient.delete(layoutKey)); + if (response.getDeleted() > 0) { + if (log.isDebugEnabled()) { + log.debug("Successfully delete layout '{}'", layoutKey.toStringUtf8()); + } + return; + } else { + throw new IOException("No ledger layout is found under '" + layoutKey.toStringUtf8() + "'"); + } + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java new file mode 100644 index 00000000000..1dc44398452 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -0,0 +1,457 @@ +/* + * 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.metadata.etcd; + +import com.coreos.jetcd.Client; +import com.coreos.jetcd.KV; +import com.coreos.jetcd.Txn; +import com.coreos.jetcd.common.exception.ClosedClientException; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.data.KeyValue; +import com.coreos.jetcd.kv.GetResponse; +import com.coreos.jetcd.op.Cmp; +import com.coreos.jetcd.op.Cmp.Op; +import com.coreos.jetcd.op.CmpTarget; +import com.coreos.jetcd.options.DeleteOption; +import com.coreos.jetcd.options.GetOption; +import com.coreos.jetcd.options.PutOption; +import com.google.common.base.Optional; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Consumer; +import java.util.function.Function; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BKException.Code; +import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.metadata.etcd.helpers.KeyIterator; +import org.apache.bookkeeper.metadata.etcd.helpers.KeyStream; +import org.apache.bookkeeper.metadata.etcd.helpers.ValueStream; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; +import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Version; +import org.apache.zookeeper.AsyncCallback.VoidCallback; + +/** + * Etcd ledger manager. + */ +@Slf4j +class EtcdLedgerManager implements LedgerManager { + + private static final Function LEDGER_METADATA_FUNCTION = bs -> { + try { + return LedgerMetadata.parseConfig( + bs.getBytes(), + Version.ANY, + Optional.absent() + ); + } catch (IOException ioe) { + log.error("Could not parse ledger metadata : {}", bs.toStringUtf8(), ioe); + throw new RuntimeException( + "Could not parse ledger metadata : " + bs.toStringUtf8(), ioe); + } + }; + + private final String scope; + private final Client client; + private final KV kvClient; + private final EtcdWatchClient watchClient; + private final ConcurrentLongHashMap> watchers = + new ConcurrentLongHashMap<>(); + private final ConcurrentMap listeners = + new ConcurrentHashMap<>(); + private volatile boolean closed = false; + + EtcdLedgerManager(Client client, + String scope) { + this.client = client; + this.kvClient = client.getKVClient(); + this.scope = scope; + this.watchClient = new EtcdWatchClient(client); + } + + private boolean isClosed() { + return closed; + } + + ValueStream getLedgerMetadataStream(long ledgerId) { + return watchers.get(ledgerId); + } + + @Override + public void createLedgerMetadata(long ledgerId, + LedgerMetadata metadata, + GenericCallback cb) { + String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); + ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); + log.info("Create ledger metadata under key {}", ledgerKey); + kvClient.txn() + .If(new Cmp( + ledgerKeyBs, + Op.GREATER, + CmpTarget.createRevision(0L))) + .Then(com.coreos.jetcd.op.Op.get( + ledgerKeyBs, + GetOption.newBuilder() + .withCountOnly(true) + .build())) + .Else(com.coreos.jetcd.op.Op.put( + ledgerKeyBs, + ByteSequence.fromBytes(metadata.serialize()), + PutOption.DEFAULT)) + .commit() + .thenAccept(resp -> { + if (resp.isSucceeded()) { + GetResponse getResp = resp.getGetResponses().get(0); + if (getResp.getCount() <= 0) { + // key doesn't exist but we fail to put the key + cb.operationComplete(Code.UnexpectedConditionException, null); + } else { + // key exists + cb.operationComplete(Code.LedgerExistException, null); + } + } else { + metadata.setVersion(new LongVersion(resp.getHeader().getRevision())); + cb.operationComplete(Code.OK, metadata); + } + }) + .exceptionally(cause -> { + cb.operationComplete(Code.MetaStoreException, null); + return null; + }); + } + + @Override + public void removeLedgerMetadata(long ledgerId, + Version version, + GenericCallback cb) { + long revision = -0xabcd; + if (Version.NEW == version) { + log.error("Request to delete ledger {} metadata with version set to the initial one", ledgerId); + cb.operationComplete(Code.MetadataVersionException, null); + return; + } else if (Version.ANY != version) { + if (!(version instanceof LongVersion)) { + log.info("Not an instance of LongVersion : {}", ledgerId); + cb.operationComplete(Code.MetadataVersionException, null); + return; + } else { + revision = ((LongVersion) version).getLongVersion(); + } + } + + String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); + ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); + Txn txn = kvClient.txn(); + if (revision == -0xabcd) { + txn = txn.If(new Cmp( + ledgerKeyBs, + Op.GREATER, + CmpTarget.createRevision(0L) + )); + } else { + txn = txn.If(new Cmp( + ledgerKeyBs, + Op.EQUAL, + CmpTarget.modRevision(revision) + )); + } + txn + .Then(com.coreos.jetcd.op.Op.delete( + ledgerKeyBs, + DeleteOption.DEFAULT + )) + .Else(com.coreos.jetcd.op.Op.get( + ledgerKeyBs, + GetOption.DEFAULT + )) + .commit() + .thenAccept(txnResp -> { + if (txnResp.isSucceeded()) { + cb.operationComplete(Code.OK, null); + } else { + GetResponse getResp = txnResp.getGetResponses().get(0); + if (getResp.getCount() > 0) { + // fail to delete the ledger + cb.operationComplete(Code.MetadataVersionException, null); + } else { + log.warn("Deleting ledger {} failed due to : ledger key {} doesn't exist", ledgerId, ledgerKey); + cb.operationComplete(Code.NoSuchLedgerExistsException, null); + } + } + }) + .exceptionally(cause -> { + cb.operationComplete(Code.MetaStoreException, null); + return null; + }); + } + + @Override + public void readLedgerMetadata(long ledgerId, GenericCallback readCb) { + String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); + ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); + log.info("read ledger metadata under key {}", ledgerKey); + kvClient.get(ledgerKeyBs) + .thenAccept(getResp -> { + if (getResp.getCount() > 0) { + KeyValue kv = getResp.getKvs().get(0); + byte[] data = kv.getValue().getBytes(); + LedgerMetadata metadata; + try { + metadata = LedgerMetadata.parseConfig( + data, + new LongVersion(kv.getModRevision()), + Optional.absent() + ); + readCb.operationComplete(Code.OK, metadata); + } catch (IOException ioe) { + log.error("Could not parse ledger metadata for ledger : {}", ledgerId, ioe); + readCb.operationComplete(Code.MetaStoreException, null); + return; + } + } else { + readCb.operationComplete(Code.NoSuchLedgerExistsException, null); + } + }) + .exceptionally(cause -> { + readCb.operationComplete(Code.MetaStoreException, null); + return null; + }); + } + + @Override + public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback cb) { + Version v = metadata.getVersion(); + if (Version.NEW == v || !(v instanceof LongVersion)) { + cb.operationComplete(Code.MetadataVersionException, null); + return; + } + final LongVersion lv = (LongVersion) v; + String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); + ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); + kvClient.txn() + .If(new Cmp( + ledgerKeyBs, + Op.EQUAL, + CmpTarget.modRevision(lv.getLongVersion()))) + .Then(com.coreos.jetcd.op.Op.put( + ledgerKeyBs, + ByteSequence.fromBytes(metadata.serialize()), + PutOption.DEFAULT)) + .Else(com.coreos.jetcd.op.Op.get( + ledgerKeyBs, + GetOption.DEFAULT)) + .commit() + .thenAccept(resp -> { + if (resp.isSucceeded()) { + metadata.setVersion(new LongVersion(resp.getHeader().getRevision())); + cb.operationComplete(Code.OK, metadata); + } else { + GetResponse getResp = resp.getGetResponses().get(0); + if (getResp.getCount() > 0) { + log.warn("Conditional update ledger metadata failed :" + + " expected version = {}, actual version = {}", + getResp.getKvs().get(0).getModRevision(), lv); + cb.operationComplete(Code.MetadataVersionException, null); + } else { + cb.operationComplete(Code.NoSuchLedgerExistsException, null); + } + } + }) + .exceptionally(cause -> { + cb.operationComplete(Code.MetaStoreException, null); + return null; + }); + } + + private LedgerMetadataConsumer listenerToConsumer(long ledgerId, + LedgerMetadataListener listener, + Consumer onDeletedConsumer) { + return new LedgerMetadataConsumer( + ledgerId, + listener, + onDeletedConsumer + ); + } + + @Override + public void registerLedgerMetadataListener(long ledgerId, LedgerMetadataListener listener) { + if (listeners.containsKey(listener)) { + return; + } + + ValueStream lmStream = watchers.computeIfAbsent( + ledgerId, (lid) -> new ValueStream<>( + client, + watchClient, + LEDGER_METADATA_FUNCTION, + ByteSequence.fromString(EtcdUtils.getLedgerKey(scope, ledgerId))) + ); + LedgerMetadataConsumer lmConsumer = listenerToConsumer(ledgerId, listener, + (lid) -> { + if (watchers.remove(lid, lmStream)) { + log.info("Closed ledger metadata watcher on ledger {} deletion.", lid); + lmStream.closeAsync(); + } + }); + LedgerMetadataConsumer oldConsumer = listeners.putIfAbsent(listener, lmConsumer); + if (null != oldConsumer) { + return; + } else { + lmStream.readAndWatch(lmConsumer) + .whenComplete((values, cause) -> { + if (null != cause && !(cause instanceof ClosedClientException)) { + // fail to register ledger metadata listener, re-attempt it + registerLedgerMetadataListener(ledgerId, listener); + } + }); + } + } + + @Override + public void unregisterLedgerMetadataListener(long ledgerId, LedgerMetadataListener listener) { + LedgerMetadataConsumer lmConsumer = listeners.remove(listener); + unregisterLedgerMetadataListener(ledgerId, lmConsumer); + } + + private void unregisterLedgerMetadataListener(long ledgerId, LedgerMetadataConsumer lmConsumer) { + ValueStream lmStream = watchers.get(ledgerId); + if (null == lmStream) { + return; + } else { + lmStream.unwatch(lmConsumer).thenAccept(noConsumers -> { + if (noConsumers) { + if (watchers.remove(ledgerId, lmStream)) { + log.info("Closed ledger metadata watcher on ledger {} since there are no listeners any more.", + ledgerId); + lmStream.closeAsync(); + } + } + }).exceptionally(cause -> { + if (cause instanceof ClosedClientException) { + // fail to unwatch a consumer + unregisterLedgerMetadataListener(ledgerId, lmConsumer); + } + return null; + }); + } + } + + @Override + public void asyncProcessLedgers(Processor processor, + VoidCallback finalCb, + Object context, + int successRc, + int failureRc) { + KeyStream ks = new KeyStream<>( + kvClient, + ByteSequence.fromString(EtcdUtils.getLedgerKey(scope, 0L)), + ByteSequence.fromString(EtcdUtils.getLedgerKey(scope, Long.MAX_VALUE)), + bs -> { + UUID uuid = EtcdUtils.parseLedgerKey(bs.toStringUtf8()); + return uuid.getLeastSignificantBits(); + } + ); + processLedgers( + ks, processor, finalCb, context, successRc, failureRc); + } + + private void processLedgers(KeyStream ks, + Processor processor, + VoidCallback finalCb, + Object context, + int successRc, + int failureRc) { + ks.readNext().whenCompleteAsync((ledgers, cause) -> { + if (null != cause) { + finalCb.processResult(failureRc, null, context); + } else { + if (ledgers.isEmpty()) { + finalCb.processResult(successRc, null, context); + } else { + ledgers.forEach(l -> processor.process(l, finalCb)); + processLedgers(ks, processor, finalCb, context, successRc, failureRc); + } + } + }); + } + + @Override + public LedgerRangeIterator getLedgerRanges() { + KeyStream ks = new KeyStream<>( + kvClient, + ByteSequence.fromString(EtcdUtils.getLedgerKey(scope, 0L)), + ByteSequence.fromString(EtcdUtils.getLedgerKey(scope, Long.MAX_VALUE)), + bs -> { + UUID uuid = EtcdUtils.parseLedgerKey(bs.toStringUtf8()); + return uuid.getLeastSignificantBits(); + } + ); + KeyIterator ki = new KeyIterator<>(ks); + return new LedgerRangeIterator() { + @Override + public boolean hasNext() throws IOException { + try { + return ki.hasNext(); + } catch (Exception e) { + if (e instanceof IOException) { + throw ((IOException) e); + } else { + throw new IOException(e); + } + } + } + + @Override + public LedgerRange next() throws IOException { + try { + final List values = ki.next(); + final Set ledgers = Sets.newTreeSet(); + ledgers.addAll(values); + return new LedgerRange(ledgers); + } catch (Exception e) { + if (e instanceof IOException) { + throw ((IOException) e); + } else { + throw new IOException(e); + } + } + } + }; + } + + @Override + public void close() { + synchronized (this) { + if (closed) { + return; + } + closed = true; + } + watchClient.close(); + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java new file mode 100644 index 00000000000..dc7d1517b2b --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java @@ -0,0 +1,121 @@ +/* + * 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.metadata.etcd; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.coreos.jetcd.Client; +import java.io.IOException; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.meta.LayoutManager; +import org.apache.bookkeeper.meta.LedgerIdGenerator; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; +import org.apache.commons.configuration.ConfigurationException; +import org.apache.zookeeper.KeeperException; + +/** + * Etcd based ledger manager factory. + */ +class EtcdLedgerManagerFactory implements LedgerManagerFactory { + + static final int VERSION = 0; + + private String scope; + private Client client; + + @Override + public int getCurrentVersion() { + return VERSION; + } + + @Override + public LedgerManagerFactory initialize(AbstractConfiguration conf, + LayoutManager layoutManager, + int factoryVersion) throws IOException { + checkArgument(layoutManager instanceof EtcdLayoutManager); + + EtcdLayoutManager etcdLayoutManager = (EtcdLayoutManager) layoutManager; + + if (VERSION != factoryVersion) { + throw new IOException("Incompatible layout version found : " + factoryVersion); + } + try { + ServiceURI uri = ServiceURI.create(conf.getMetadataServiceUri()); + this.scope = uri.getServicePath(); + } catch (ConfigurationException e) { + throw new IOException("Invalid metadata service uri", e); + } + this.client = etcdLayoutManager.getClient(); + return this; + } + + @Override + public void close() { + // since layout manager is passed from outside. + // we don't need to close it here + } + + @Override + public LedgerIdGenerator newLedgerIdGenerator() { + return new Etcd64bitIdGenerator(client.getKVClient(), scope); + } + + @Override + public LedgerManager newLedgerManager() { + return new EtcdLedgerManager(client, scope); + } + + @Override + public LedgerUnderreplicationManager newLedgerUnderreplicationManager() + throws KeeperException, InterruptedException, CompatibilityException { + throw new UnsupportedOperationException(); + } + + @Override + public void format(AbstractConfiguration conf, LayoutManager lm) + throws InterruptedException, KeeperException, IOException { + try { + EtcdRegistrationManager.format(client.getKVClient(), scope); + } catch (Exception e) { + if (e instanceof IOException) { + throw (IOException) e; + } else { + throw new IOException(e); + } + } + } + + @Override + public boolean validateAndNukeExistingCluster(AbstractConfiguration conf, LayoutManager lm) + throws InterruptedException, KeeperException, IOException { + try { + return EtcdRegistrationManager.nukeExistingCluster(client.getKVClient(), scope); + } catch (Exception e) { + if (e instanceof IOException) { + throw (IOException) e; + } else { + throw new IOException(e); + } + } + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataBookieDriver.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataBookieDriver.java new file mode 100644 index 00000000000..764018bb99c --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataBookieDriver.java @@ -0,0 +1,94 @@ +/* + * Licensed 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.metadata.etcd; + +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.discover.RegistrationManager.RegistrationListener; +import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.stats.StatsLogger; + +/** + * Etcd based metadata bookie driver. + */ +@Slf4j +public class EtcdMetadataBookieDriver extends EtcdMetadataDriverBase implements MetadataBookieDriver { + + // register myself + static { + MetadataDrivers.registerBookieDriver( + SCHEME, EtcdMetadataBookieDriver.class); + log.info("Registered etcd metadata bookie driver"); + } + + ServerConfiguration conf; + EtcdBookieRegister bkRegister; + RegistrationManager regMgr; + RegistrationListener listener; + + @Override + public synchronized MetadataBookieDriver initialize(ServerConfiguration conf, + RegistrationListener listener, + StatsLogger statsLogger) + throws MetadataException { + super.initialize(conf, statsLogger); + this.conf = conf; + this.listener = listener; + this.statsLogger = statsLogger; + return null; + } + + @Override + public synchronized RegistrationManager getRegistrationManager() { + if (null == bkRegister) { + bkRegister = new EtcdBookieRegister( + client.getLeaseClient(), + TimeUnit.MILLISECONDS.toSeconds(conf.getZkTimeout()), + listener + ).start(); + } + if (null == regMgr) { + regMgr = new EtcdRegistrationManager( + client, + keyPrefix, + bkRegister + ); + } + return regMgr; + } + + @Override + public void close() { + RegistrationManager rmToClose; + EtcdBookieRegister bkRegisterToClose; + synchronized (this) { + rmToClose = regMgr; + regMgr = null; + bkRegisterToClose = bkRegister; + bkRegister = null; + } + if (null != rmToClose) { + rmToClose.close(); + } + if (null != bkRegisterToClose) { + bkRegisterToClose.close(); + } + super.close(); + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataClientDriver.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataClientDriver.java new file mode 100644 index 00000000000..8fd7b973b99 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataClientDriver.java @@ -0,0 +1,75 @@ +/* + * 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.metadata.etcd; + +import java.util.Optional; +import java.util.concurrent.ScheduledExecutorService; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.discover.RegistrationClient; +import org.apache.bookkeeper.meta.MetadataClientDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.stats.StatsLogger; + +/** + * Etcd based metadata client driver. + */ +@Slf4j +public class EtcdMetadataClientDriver extends EtcdMetadataDriverBase implements MetadataClientDriver { + + // register myself to driver manager + static { + MetadataDrivers.registerClientDriver( + SCHEME, EtcdMetadataClientDriver.class); + log.info("Registered etcd metadata client driver."); + } + + ClientConfiguration conf; + ScheduledExecutorService scheduler; + RegistrationClient regClient; + + @Override + public MetadataClientDriver initialize(ClientConfiguration conf, + ScheduledExecutorService scheduler, + StatsLogger statsLogger, + Optional ctx) + throws MetadataException { + super.initialize(conf, statsLogger); + this.conf = conf; + this.statsLogger = statsLogger; + return this; + } + + @Override + public synchronized RegistrationClient getRegistrationClient() { + if (null == regClient) { + regClient = new EtcdRegistrationClient(keyPrefix, client); + } + return regClient; + } + + @Override + public synchronized void close() { + if (null != regClient) { + regClient.close(); + regClient = null; + } + super.close(); + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java new file mode 100644 index 00000000000..47b034c00ef --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java @@ -0,0 +1,133 @@ +/* + * 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.metadata.etcd; + +import com.beust.jcommander.internal.Lists; +import com.coreos.jetcd.Client; +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.meta.LayoutManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.exceptions.Code; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.commons.configuration.ConfigurationException; + +/** + * This is a mixin class for supporting etcd based metadata drivers. + */ +@Slf4j +class EtcdMetadataDriverBase implements AutoCloseable { + + static final String SCHEME = "etcd"; + + protected AbstractConfiguration conf; + protected StatsLogger statsLogger; + + // service uri + protected Client client; + protected String keyPrefix; + + // managers + protected LayoutManager layoutManager; + protected LedgerManagerFactory lmFactory; + + public String getScheme() { + return SCHEME; + } + + /** + * Initialize metadata driver with provided configuration and statsLogger. + * + * @param conf configuration to initialize metadata driver + * @param statsLogger stats logger + * @throws MetadataException + */ + protected void initialize(AbstractConfiguration conf, StatsLogger statsLogger) + throws MetadataException { + this.conf = conf; + this.statsLogger = statsLogger; + + final String metadataServiceUriStr; + try { + metadataServiceUriStr = conf.getMetadataServiceUri(); + } catch (ConfigurationException ce) { + log.error("Failed to retrieve metadata service uri from configuration", ce); + throw new MetadataException(Code.INVALID_METADATA_SERVICE_URI, ce); + } + ServiceURI serviceURI = ServiceURI.create(metadataServiceUriStr); + this.keyPrefix = serviceURI.getServicePath(); + + List etcdEndpoints = Lists.newArrayList(serviceURI.getServiceHosts()) + .stream() + .map(host -> String.format("http://%s", host)) + .collect(Collectors.toList()); + + log.info("Initializing etcd metadata driver : etcd endpoints = {}, key scope = {}", + etcdEndpoints, keyPrefix); + + synchronized (this) { + this.client = Client.builder() + .endpoints(etcdEndpoints.toArray(new String[etcdEndpoints.size()])) + .build(); + } + + this.layoutManager = new EtcdLayoutManager( + client, + keyPrefix + ); + } + + public LayoutManager getLayoutManager() { + return layoutManager; + } + + public synchronized LedgerManagerFactory getLedgerManagerFactory() + throws MetadataException { + if (null == lmFactory) { + try { + lmFactory = new EtcdLedgerManagerFactory(); + lmFactory.initialize(conf, layoutManager, EtcdLedgerManagerFactory.VERSION); + } catch (IOException ioe) { + throw new MetadataException( + Code.METADATA_SERVICE_ERROR, "Failed to initialize ledger manager factory", ioe); + } + } + return lmFactory; + } + + @Override + public synchronized void close() { + if (null != lmFactory) { + try { + lmFactory.close(); + } catch (IOException e) { + log.error("Failed to close ledger manager factory", e); + } + lmFactory = null; + } + if (null != client) { + client.close(); + client = null; + } + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java new file mode 100644 index 00000000000..f1e77157ffb --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java @@ -0,0 +1,169 @@ +/* + * 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.metadata.etcd; + +import com.coreos.jetcd.Client; +import com.coreos.jetcd.data.ByteSequence; +import com.google.common.collect.Maps; +import java.net.UnknownHostException; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; +import java.util.function.Function; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.discover.RegistrationClient; +import org.apache.bookkeeper.metadata.etcd.helpers.KeySetReader; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * Etcd based registration client. + */ +@Slf4j +class EtcdRegistrationClient implements RegistrationClient { + + private static Function newBookieSocketAddressFunc(String prefix) { + return bs -> { + String addrStr = bs.toStringUtf8(); + try { + return new BookieSocketAddress(addrStr.replace(prefix, "")); + } catch (UnknownHostException e) { + throw new RuntimeException("Unknown bookie address '" + addrStr + "' : ", e); + } + }; + } + + private final EtcdWatchClient watchClient; + private final KeySetReader writableBookiesReader; + private final KeySetReader readonlyBookiesReader; + private Map>>> writableListeners = + Maps.newHashMap(); + private Map>>> readonlyListeners = + Maps.newHashMap(); + + EtcdRegistrationClient(String scope, + Client client) { + this.watchClient = new EtcdWatchClient(client); + this.writableBookiesReader = new KeySetReader<>( + client, + watchClient, + newBookieSocketAddressFunc(EtcdUtils.getWritableBookiesBeginPath(scope)), + ByteSequence.fromString(EtcdUtils.getWritableBookiesBeginPath(scope)), + ByteSequence.fromString(EtcdUtils.getWritableBookiesEndPath(scope)) + ); + this.readonlyBookiesReader = new KeySetReader<>( + client, + watchClient, + newBookieSocketAddressFunc(EtcdUtils.getReadonlyBookiesBeginPath(scope)), + ByteSequence.fromString(EtcdUtils.getReadonlyBookiesBeginPath(scope)), + ByteSequence.fromString(EtcdUtils.getReadonlyBookiesEndPath(scope)) + ); + } + + + @Override + public void close() { + this.writableBookiesReader.close(); + this.readonlyBookiesReader.close(); + this.watchClient.close(); + } + + @Override + public CompletableFuture>> getWritableBookies() { + return writableBookiesReader.read(); + } + + @Override + public CompletableFuture>> getReadOnlyBookies() { + return readonlyBookiesReader.read(); + } + + private static CompletableFuture registerListener( + KeySetReader keySetReader, + Map>>> listeners, + RegistrationListener listener + ) { + Consumer>> consumer; + synchronized (listeners) { + consumer = listeners.get(listener); + if (null != consumer) { + // already registered + return FutureUtils.Void(); + } else { + consumer = bookies -> listener.onBookiesChanged(bookies); + listeners.put(listener, consumer); + } + } + return keySetReader + .readAndWatch(consumer) + .thenApply(ignored -> null); + } + + private static CompletableFuture unregisterListener( + KeySetReader keySetReader, + Map>>> listeners, + RegistrationListener listener + ) { + Consumer>> consumer = listeners.get(listener); + if (null == consumer) { + return FutureUtils.Void(); + } else { + return keySetReader.unwatch(consumer); + } + } + + @Override + public CompletableFuture watchWritableBookies(RegistrationListener listener) { + return registerListener( + writableBookiesReader, + writableListeners, + listener + ); + } + + @Override + public void unwatchWritableBookies(RegistrationListener listener) { + unregisterListener( + writableBookiesReader, + writableListeners, + listener + ); + } + + @Override + public CompletableFuture watchReadOnlyBookies(RegistrationListener listener) { + return registerListener( + readonlyBookiesReader, + readonlyListeners, + listener + ); + } + + @Override + public void unwatchReadOnlyBookies(RegistrationListener listener) { + unregisterListener( + readonlyBookiesReader, + readonlyListeners, + listener + ); + } + +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java new file mode 100644 index 00000000000..87c831c17e4 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java @@ -0,0 +1,569 @@ +/* + * 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.metadata.etcd; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getBookiesEndPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getBookiesPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getBucketsPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getClusterInstanceIdPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getCookiePath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getCookiesPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getLayoutKey; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getLedgersPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getReadonlyBookiePath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getReadonlyBookiesPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getScopeEndKey; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getUnderreplicationPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getWritableBookiePath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getWritableBookiesPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.msResult; + +import com.coreos.jetcd.Client; +import com.coreos.jetcd.KV; +import com.coreos.jetcd.Txn; +import com.coreos.jetcd.Watch; +import com.coreos.jetcd.Watch.Watcher; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.data.KeyValue; +import com.coreos.jetcd.kv.DeleteResponse; +import com.coreos.jetcd.kv.GetResponse; +import com.coreos.jetcd.kv.TxnResponse; +import com.coreos.jetcd.op.Cmp; +import com.coreos.jetcd.op.Cmp.Op; +import com.coreos.jetcd.op.CmpTarget; +import com.coreos.jetcd.options.DeleteOption; +import com.coreos.jetcd.options.GetOption; +import com.coreos.jetcd.options.PutOption; +import com.coreos.jetcd.options.WatchOption; +import com.coreos.jetcd.watch.WatchEvent; +import com.coreos.jetcd.watch.WatchEvent.EventType; +import com.coreos.jetcd.watch.WatchResponse; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import lombok.AccessLevel; +import lombok.Getter; +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.discover.RegistrationManager; +import org.apache.bookkeeper.meta.LedgerLayout; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * Etcd registration manager. + */ +@Slf4j +class EtcdRegistrationManager implements RegistrationManager { + + private final String scope; + @Getter(AccessLevel.PACKAGE) + private final Client client; + private final boolean ownClient; + private final KV kvClient; + @Getter(AccessLevel.PACKAGE) + private final EtcdBookieRegister bkRegister; + + @VisibleForTesting + EtcdRegistrationManager(Client client, + String scope) { + this(client, scope, 60); + } + + @VisibleForTesting + EtcdRegistrationManager(Client client, + String scope, + long ttlSeconds) { + this(client, scope, ttlSeconds, () -> {}); + } + + @VisibleForTesting + EtcdRegistrationManager(Client client, + String scope, + long ttlSeconds, + RegistrationListener listener) { + this( + client, + scope, + new EtcdBookieRegister( + client.getLeaseClient(), + ttlSeconds, + listener + ).start(), + true); + } + + EtcdRegistrationManager(Client client, + String scope, + EtcdBookieRegister bkRegister) { + this(client, scope, bkRegister, false); + } + + private EtcdRegistrationManager(Client client, + String scope, + EtcdBookieRegister bkRegister, + boolean ownClient) { + this.scope = scope; + this.client = client; + this.kvClient = client.getKVClient(); + this.bkRegister = bkRegister; + this.ownClient = ownClient; + } + + @Override + public void close() { + if (ownClient) { + log.info("Closing registration manager under scope '{}'", scope); + bkRegister.close(); + client.close(); + log.info("Successfully closed registration manager under scope '{}'", scope); + } + } + + @Override + public void registerBookie(String bookieId, boolean readOnly) throws BookieException { + if (readOnly) { + doRegisterReadonlyBookie(bookieId, bkRegister.get()); + } else { + doRegisterBookie(getWritableBookiePath(scope, bookieId), bkRegister.get()); + } + } + + private boolean checkRegNodeAndWaitExpired(String regPath, long leaseId) + throws MetadataStoreException { + ByteSequence regPathBs = ByteSequence.fromString(regPath); + GetResponse getResp = msResult(kvClient.get(regPathBs)); + if (getResp.getCount() <= 0) { + // key doesn't exist anymore + return false; + } else { + return waitUntilRegNodeExpired(regPath, leaseId); + } + } + + private boolean waitUntilRegNodeExpired(String regPath, long leaseId) + throws MetadataStoreException { + ByteSequence regPathBs = ByteSequence.fromString(regPath); + // check regPath again + GetResponse getResp = msResult(kvClient.get(regPathBs)); + if (getResp.getCount() <= 0) { + // key disappears after watching it + return false; + } else { + KeyValue kv = getResp.getKvs().get(0); + if (kv.getLease() != leaseId) { + Watch watchClient = client.getWatchClient(); + Watcher watcher = watchClient.watch( + regPathBs, + WatchOption.newBuilder() + .withRevision(getResp.getHeader().getRevision() + 1) + .build()); + log.info("Previous bookie registration (lease = {}) still exists at {}, " + + "so new lease '{}' will be waiting previous lease for {} seconds to be expired", + kv.getLease(), regPath, leaseId, bkRegister.getTtlSeconds()); + CompletableFuture watchFuture = + CompletableFuture.runAsync(() -> { + try { + while (true) { + log.info("Listening on '{}' until it is expired", regPath); + WatchResponse response = watcher.listen(); + for (WatchEvent event : response.getEvents()) { + log.info("Received watch event on '{}' : EventType = {}", + regPath, event.getEventType()); + if (EventType.DELETE == event.getEventType()) { + return; + } + } + } + } catch (InterruptedException e) { + throw new UncheckedExecutionException( + "Interrupted at waiting previous registration under " + + regPath + " (lease = " + kv.getLease() + ") to be expired", e); + } + }); + + try { + msResult(watchFuture, 2 * bkRegister.getTtlSeconds(), TimeUnit.SECONDS); + return false; + } catch (TimeoutException e) { + watchFuture.cancel(true); + throw new MetadataStoreException( + "Previous bookie registration still exists at " + + regPath + " (lease = " + kv.getLease() + ") after " + + (2 * bkRegister.getTtlSeconds()) + " seconds elapsed"); + } catch (UncheckedExecutionException uee) { + throw new MetadataStoreException(uee.getMessage(), uee.getCause()); + } finally { + watcher.close(); + } + } else { + // key exists with same lease + return true; + } + } + } + + private void doRegisterBookie(String regPath, long leaseId) throws MetadataStoreException { + if (checkRegNodeAndWaitExpired(regPath, leaseId)) { + // the bookie is already registered under `${regPath}` with `${leaseId}`. + return; + } + + ByteSequence regPathBs = ByteSequence.fromString(regPath); + Txn txn = kvClient.txn() + .If(new Cmp( + regPathBs, + Op.GREATER, + CmpTarget.createRevision(0))) + .Then(com.coreos.jetcd.op.Op.get(regPathBs, GetOption.DEFAULT)) + .Else(com.coreos.jetcd.op.Op.put( + regPathBs, + ByteSequence.fromBytes(new byte[0]), + PutOption.newBuilder() + .withLeaseId(bkRegister.get()) + .build() + )); + TxnResponse txnResp = msResult(txn.commit()); + if (txnResp.isSucceeded()) { + // the key already exists + GetResponse getResp = txnResp.getGetResponses().get(0); + if (getResp.getCount() <= 0) { + throw new MetadataStoreException( + "Failed to register bookie under '" + regPath + + "', but no bookie is registered there."); + } else { + KeyValue kv = getResp.getKvs().get(0); + throw new MetadataStoreException("Another bookie already registered under '" + + regPath + "': lease = " + kv.getLease()); + } + } else { + log.info("Successfully registered bookie at {}", regPath); + } + } + + private void doRegisterReadonlyBookie(String bookieId, long leaseId) throws MetadataStoreException { + String readonlyRegPath = getReadonlyBookiePath(scope, bookieId); + doRegisterBookie(readonlyRegPath, leaseId); + String writableRegPath = getWritableBookiePath(scope, bookieId); + msResult(kvClient.delete(ByteSequence.fromString(writableRegPath))); + } + + @Override + public void unregisterBookie(String bookieId, boolean readOnly) throws BookieException { + String regPath; + if (readOnly) { + regPath = getReadonlyBookiePath(scope, bookieId); + } else { + regPath = getWritableBookiePath(scope, bookieId); + } + DeleteResponse delResp = msResult(kvClient.delete(ByteSequence.fromString(regPath))); + if (delResp.getDeleted() > 0) { + log.info("Successfully unregistered bookie {} from {}", bookieId, regPath); + } else { + log.info("Bookie disappeared from {} before unregistering", regPath); + } + } + + @Override + public boolean isBookieRegistered(String bookieId) throws BookieException { + CompletableFuture getWritableFuture = kvClient.get( + ByteSequence.fromString(getWritableBookiePath(scope, bookieId)), + GetOption.newBuilder() + .withCountOnly(true) + .build()); + CompletableFuture getReadonlyFuture = kvClient.get( + ByteSequence.fromString(getReadonlyBookiePath(scope, bookieId)), + GetOption.newBuilder() + .withCountOnly(true) + .build()); + + return msResult(getWritableFuture).getCount() > 0 + || msResult(getReadonlyFuture).getCount() > 0; + } + + @Override + public void writeCookie(String bookieId, Versioned cookieData) throws BookieException { + ByteSequence cookiePath = ByteSequence.fromString(getCookiePath(scope, bookieId)); + Txn txn = kvClient.txn(); + if (Version.NEW == cookieData.getVersion()) { + txn.If(new Cmp( + cookiePath, + Op.GREATER, + CmpTarget.createRevision(0L)) + ) + // if key not exists, create one. + .Else(com.coreos.jetcd.op.Op.put( + cookiePath, + ByteSequence.fromBytes(cookieData.getValue()), + PutOption.DEFAULT) + ); + } else { + if (!(cookieData.getVersion() instanceof LongVersion)) { + throw new BookieIllegalOpException("Invalid version type, expected it to be LongVersion"); + } + txn.If(new Cmp( + cookiePath, + Op.EQUAL, + CmpTarget.modRevision(((LongVersion) cookieData.getVersion()).getLongVersion())) + ) + .Then(com.coreos.jetcd.op.Op.put( + cookiePath, + ByteSequence.fromBytes(cookieData.getValue()), + PutOption.DEFAULT) + ); + } + TxnResponse response = msResult(txn.commit()); + if (response.isSucceeded() != (Version.NEW != cookieData.getVersion())) { + throw new MetadataStoreException( + "Conflict on writing cookie for bookie " + bookieId); + } + } + + @Override + public Versioned readCookie(String bookieId) throws BookieException { + ByteSequence cookiePath = ByteSequence.fromString(getCookiePath(scope, bookieId)); + GetResponse resp = msResult(kvClient.get(cookiePath)); + if (resp.getCount() <= 0) { + throw new CookieNotFoundException(bookieId); + } else { + KeyValue kv = resp.getKvs().get(0); + return new Versioned<>( + kv.getValue().getBytes(), + new LongVersion(kv.getModRevision())); + } + } + + @Override + public void removeCookie(String bookieId, Version version) throws BookieException { + ByteSequence cookiePath = ByteSequence.fromString(getCookiePath(scope, bookieId)); + Txn delTxn = kvClient.txn() + .If(new Cmp( + cookiePath, + Op.EQUAL, + CmpTarget.modRevision(((LongVersion) version).getLongVersion()) + )) + .Then(com.coreos.jetcd.op.Op.delete( + cookiePath, + DeleteOption.DEFAULT + )) + .Else(com.coreos.jetcd.op.Op.get( + cookiePath, + GetOption.newBuilder().withCountOnly(true).build() + )); + TxnResponse txnResp = msResult(delTxn.commit()); + if (!txnResp.isSucceeded()) { + GetResponse getResp = txnResp.getGetResponses().get(0); + if (getResp.getCount() > 0) { + throw new MetadataStoreException( + "Failed to remove cookie from " + cookiePath.toStringUtf8() + + " for bookie " + bookieId + " : bad version '" + version + "'"); + } else { + throw new CookieNotFoundException(bookieId); + } + } else { + log.info("Removed cookie from {} for bookie {}", + cookiePath.toStringUtf8(), bookieId); + } + } + + @Override + public String getClusterInstanceId() throws BookieException { + GetResponse response = msResult( + kvClient.get(ByteSequence.fromString(getClusterInstanceIdPath(scope)))); + if (response.getCount() <= 0) { + log.error("BookKeeper metadata doesn't exist in Etcd. " + + "Has the cluster been initialized? " + + "Try running bin/bookkeeper shell initNewCluster"); + throw new MetadataStoreException("BookKeeper is not initialized under '" + scope + "' yet"); + } else { + KeyValue kv = response.getKvs().get(0); + return new String(kv.getValue().getBytes(), UTF_8); + } + } + + @Override + public boolean prepareFormat() throws Exception { + ByteSequence rootScopeKey = ByteSequence.fromString(scope); + GetResponse resp = msResult(kvClient.get(rootScopeKey)); + return resp.getCount() > 0; + } + + @Override + public boolean initNewCluster() throws Exception { + return initNewCluster(kvClient, scope); + } + + static boolean initNewCluster(KV kvClient, String scope) throws Exception { + ByteSequence rootScopeKey = ByteSequence.fromString(scope); + String instanceId = UUID.randomUUID().toString(); + LedgerLayout layout = new LedgerLayout( + EtcdLedgerManagerFactory.class.getName(), + EtcdLedgerManagerFactory.VERSION + ); + Txn initTxn = kvClient.txn() + .If(new Cmp( + rootScopeKey, + Op.GREATER, + CmpTarget.createRevision(0L) + )) + // only put keys when root scope doesn't exist + .Else( + // `${scope}` + com.coreos.jetcd.op.Op.put( + rootScopeKey, + EtcdConstants.EMPTY_BS, + PutOption.DEFAULT + ), + // `${scope}/layout` + com.coreos.jetcd.op.Op.put( + ByteSequence.fromString(getLayoutKey(scope)), + ByteSequence.fromBytes(layout.serialize()), + PutOption.DEFAULT + ), + // `${scope}/instanceid` + com.coreos.jetcd.op.Op.put( + ByteSequence.fromString(getClusterInstanceIdPath(scope)), + ByteSequence.fromString(instanceId), + PutOption.DEFAULT + ), + // `${scope}/cookies` + com.coreos.jetcd.op.Op.put( + ByteSequence.fromString(getCookiesPath(scope)), + EtcdConstants.EMPTY_BS, + PutOption.DEFAULT + ), + // `${scope}/bookies` + com.coreos.jetcd.op.Op.put( + ByteSequence.fromString(getBookiesPath(scope)), + EtcdConstants.EMPTY_BS, + PutOption.DEFAULT + ), + // `${scope}/bookies/writable` + com.coreos.jetcd.op.Op.put( + ByteSequence.fromString(getWritableBookiesPath(scope)), + EtcdConstants.EMPTY_BS, + PutOption.DEFAULT + ), + // `${scope}/bookies/readonly` + com.coreos.jetcd.op.Op.put( + ByteSequence.fromString(getReadonlyBookiesPath(scope)), + EtcdConstants.EMPTY_BS, + PutOption.DEFAULT + ), + // `${scope}/ledgers` + com.coreos.jetcd.op.Op.put( + ByteSequence.fromString(getLedgersPath(scope)), + EtcdConstants.EMPTY_BS, + PutOption.DEFAULT + ), + // `${scope}/buckets` + com.coreos.jetcd.op.Op.put( + ByteSequence.fromString(getBucketsPath(scope)), + EtcdConstants.EMPTY_BS, + PutOption.DEFAULT + ), + // `${scope}/underreplication` + com.coreos.jetcd.op.Op.put( + ByteSequence.fromString(getUnderreplicationPath(scope)), + EtcdConstants.EMPTY_BS, + PutOption.DEFAULT + ) + ); + + return !msResult(initTxn.commit()).isSucceeded(); + } + + @Override + public boolean format() throws Exception { + return format(kvClient, scope); + } + + static boolean format(KV kvClient, String scope) throws Exception { + ByteSequence rootScopeKey = ByteSequence.fromString(scope); + GetResponse resp = msResult(kvClient.get(rootScopeKey)); + if (resp.getCount() <= 0) { + // cluster doesn't exist + return initNewCluster(kvClient, scope); + } else if (nukeExistingCluster(kvClient, scope)) { // cluster exists and has successfully nuked it + return initNewCluster(kvClient, scope); + } else { + return false; + } + } + + @Override + public boolean nukeExistingCluster() throws Exception { + return nukeExistingCluster(kvClient, scope); + } + + static boolean nukeExistingCluster(KV kvClient, String scope) throws Exception { + ByteSequence rootScopeKey = ByteSequence.fromString(scope); + GetResponse resp = msResult(kvClient.get(rootScopeKey)); + if (resp.getCount() <= 0) { + log.info("There is no existing cluster with under scope '{}' in Etcd, " + + "so exiting nuke operation", scope); + return true; + } + + String bookiesPath = getBookiesPath(scope); + String bookiesEndPath = getBookiesEndPath(scope); + resp = msResult(kvClient.get( + ByteSequence.fromString(bookiesPath), + GetOption.newBuilder() + .withRange(ByteSequence.fromString(bookiesEndPath)) + .withKeysOnly(true) + .build() + )); + String writableBookiesPath = getWritableBookiesPath(scope); + String readonlyBookiesPath = getReadonlyBookiesPath(scope); + boolean hasBookiesAlive = false; + for (KeyValue kv : resp.getKvs()) { + String keyStr = new String(kv.getKey().getBytes(), UTF_8); + if (keyStr.equals(bookiesPath) + || keyStr.equals(writableBookiesPath) + || keyStr.equals(readonlyBookiesPath)) { + continue; + } else { + hasBookiesAlive = true; + break; + } + } + if (hasBookiesAlive) { + log.error("Bookies are still up and connected to this cluster, " + + "stop all bookies before nuking the cluster"); + return false; + } + DeleteResponse delResp = msResult(kvClient.delete( + rootScopeKey, + DeleteOption.newBuilder() + .withRange(ByteSequence.fromString(getScopeEndKey(scope))) + .build())); + log.info("Successfully nuked cluster under scope '{}' : {} kv pairs deleted", + scope, delResp.getDeleted()); + return true; + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdUtils.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdUtils.java new file mode 100644 index 00000000000..296ff6c5092 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdUtils.java @@ -0,0 +1,211 @@ +/* + * 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.metadata.etcd; + +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.BUCKETS_NODE; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.COOKIES_NODE; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.END_SEP; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.INSTANCEID_NODE; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.LAYOUT_NODE; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.LEDGERS_NODE; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.MEMBERS_NODE; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.READONLY_NODE; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.UR_NODE; +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.WRITEABLE_NODE; + +import java.io.IOException; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import lombok.AccessLevel; +import lombok.NoArgsConstructor; +import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.commons.lang3.StringUtils; + +/** + * Utils for etcd based metadata store. + */ +@NoArgsConstructor(access = AccessLevel.PRIVATE) +final class EtcdUtils { + + static String getScopeEndKey(String scope) { + return String.format("%s%s", scope, END_SEP); + } + + static String getBucketsPath(String scope) { + return String.format("%s/%s", scope, BUCKETS_NODE); + } + + static String getBucketPath(String scope, int bucket) { + return String.format("%s/%s/%03d", + scope, + BUCKETS_NODE, + bucket); + } + + static String getLayoutKey(String scope) { + return String.format("%s/%s", scope, LAYOUT_NODE); + } + + static String getLedgersPath(String scope) { + return String.format("%s/%s", scope, LEDGERS_NODE); + } + + static String getLedgerKey(String scope, long ledgerId) { + return getLedgerKey(scope, 0, ledgerId); + } + + static String getLedgerKey(String scope, long scopeId, long ledgerId) { + UUID uuid = new UUID(scopeId, ledgerId); + return String.format("%s/ledgers/%s", scope, uuid.toString()); + } + + static UUID parseLedgerKey(String ledgerKey) { + String[] keyParts = StringUtils.split(ledgerKey, '/'); + return UUID.fromString(keyParts[keyParts.length - 1]); + } + + static String getBookiesPath(String scope) { + return String.format("%s/%s", scope, MEMBERS_NODE); + } + + static String getBookiesEndPath(String scope) { + return String.format("%s/%s%s", scope, MEMBERS_NODE, END_SEP); + } + + static String getWritableBookiesPath(String scope) { + return String.format("%s/%s/%s", scope, MEMBERS_NODE, WRITEABLE_NODE); + } + + static String getWritableBookiesBeginPath(String scope) { + return String.format("%s/%s/%s/", scope, MEMBERS_NODE, WRITEABLE_NODE); + } + + static String getWritableBookiesEndPath(String scope) { + return String.format("%s/%s/%s%s", scope, MEMBERS_NODE, WRITEABLE_NODE, END_SEP); + } + + static String getWritableBookiePath(String scope, String bookieId) { + return String.format("%s/%s/%s/%s", + scope, MEMBERS_NODE, WRITEABLE_NODE, bookieId); + } + + static String getReadonlyBookiesPath(String scope) { + return String.format("%s/%s/%s", scope, MEMBERS_NODE, READONLY_NODE); + } + + static String getReadonlyBookiesBeginPath(String scope) { + return String.format("%s/%s/%s/", scope, MEMBERS_NODE, READONLY_NODE); + } + + static String getReadonlyBookiesEndPath(String scope) { + return String.format("%s/%s/%s%s", scope, MEMBERS_NODE, READONLY_NODE, END_SEP); + } + + static String getReadonlyBookiePath(String scope, String bookieId) { + return String.format("%s/%s/%s/%s", + scope, MEMBERS_NODE, READONLY_NODE, bookieId); + } + + static String getCookiesPath(String scope) { + return String.format("%s/%s", scope, COOKIES_NODE); + } + + static String getCookiePath(String scope, String bookieId) { + return String.format("%s/%s/%s", scope, COOKIES_NODE, bookieId); + } + + static String getClusterInstanceIdPath(String scope) { + return String.format("%s/%s", scope, INSTANCEID_NODE); + } + + static String getUnderreplicationPath(String scope) { + return String.format("%s/%s", scope, UR_NODE); + } + + static T ioResult(CompletableFuture future) throws IOException { + return FutureUtils.result(future, cause -> { + if (cause instanceof IOException) { + return (IOException) cause; + } else { + return new IOException(cause); + } + }); + } + + static T msResult(CompletableFuture future) throws MetadataStoreException { + return FutureUtils.result(future, cause -> { + if (cause instanceof MetadataStoreException) { + return (MetadataStoreException) cause; + } else { + return new MetadataStoreException(cause); + } + }); + } + + static T msResult(CompletableFuture future, + long timeout, + TimeUnit timeUnit) + throws MetadataStoreException, TimeoutException { + return FutureUtils.result(future, cause -> { + if (cause instanceof MetadataStoreException) { + return (MetadataStoreException) cause; + } else { + return new MetadataStoreException(cause); + } + }, timeout, timeUnit); + } + + public static long toLong(byte[] memory, int index) { + return ((long) memory[index] & 0xff) << 56 + | ((long) memory[index + 1] & 0xff) << 48 + | ((long) memory[index + 2] & 0xff) << 40 + | ((long) memory[index + 3] & 0xff) << 32 + | ((long) memory[index + 4] & 0xff) << 24 + | ((long) memory[index + 5] & 0xff) << 16 + | ((long) memory[index + 6] & 0xff) << 8 + | (long) memory[index + 7] & 0xff; + } + + /** + * Convert a long number to a bytes array. + * + * @param value the long number + * @return the bytes array + */ + public static byte[] toBytes(long value) { + byte[] memory = new byte[8]; + toBytes(value, memory, 0); + return memory; + } + + public static void toBytes(long value, byte[] memory, int index) { + memory[index] = (byte) (value >>> 56); + memory[index + 1] = (byte) (value >>> 48); + memory[index + 2] = (byte) (value >>> 40); + memory[index + 3] = (byte) (value >>> 32); + memory[index + 4] = (byte) (value >>> 24); + memory[index + 5] = (byte) (value >>> 16); + memory[index + 6] = (byte) (value >>> 8); + memory[index + 7] = (byte) value; + } + +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdWatchClient.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdWatchClient.java new file mode 100644 index 00000000000..b3aabfe9702 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdWatchClient.java @@ -0,0 +1,400 @@ +/* + * 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.metadata.etcd; + +import static com.coreos.jetcd.common.exception.EtcdExceptionFactory.newClosedWatchClientException; +import static com.coreos.jetcd.common.exception.EtcdExceptionFactory.newEtcdException; +import static com.coreos.jetcd.common.exception.EtcdExceptionFactory.toEtcdException; + +import com.coreos.jetcd.Client; +import com.coreos.jetcd.api.WatchCancelRequest; +import com.coreos.jetcd.api.WatchCreateRequest; +import com.coreos.jetcd.api.WatchGrpc; +import com.coreos.jetcd.api.WatchRequest; +import com.coreos.jetcd.api.WatchResponse; +import com.coreos.jetcd.common.exception.ErrorCode; +import com.coreos.jetcd.common.exception.EtcdException; +import com.coreos.jetcd.common.exception.EtcdExceptionFactory; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.internal.impl.EtcdConnectionManager; +import com.coreos.jetcd.options.WatchOption; +import com.coreos.jetcd.watch.WatchResponseWithError; +import com.google.common.base.Strings; +import com.google.protobuf.ByteString; +import com.google.protobuf.UnsafeByteOperations; +import io.grpc.Status; +import io.grpc.Status.Code; +import io.grpc.stub.StreamObserver; +import java.util.LinkedList; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap; +import org.apache.bookkeeper.util.collections.ConcurrentLongHashSet; + +/** + * An async watch implementation. + */ +@Slf4j +public class EtcdWatchClient implements AutoCloseable { + + private final EtcdConnectionManager connMgr; + private final WatchGrpc.WatchStub stub; + private volatile StreamObserver grpcWatchStreamObserver; + // watchers stores a mapping between watchID -> EtcdWatcher. + private final ConcurrentLongHashMap watchers = + new ConcurrentLongHashMap<>(); + private final LinkedList pendingWatchers = new LinkedList<>(); + private final ConcurrentLongHashSet cancelSet = new ConcurrentLongHashSet(); + + // scheduler + private final OrderedScheduler scheduler; + private final ScheduledExecutorService watchExecutor; + + // close state + private CompletableFuture closeFuture = null; + + public EtcdWatchClient(Client client) { + this.connMgr = new EtcdConnectionManager(client); + this.stub = connMgr.newWatchStub(); + this.scheduler = OrderedScheduler.newSchedulerBuilder() + .name("etcd-watcher-scheduler") + .numThreads(Runtime.getRuntime().availableProcessors()) + .build(); + this.watchExecutor = this.scheduler.chooseThread(); + } + + public synchronized boolean isClosed() { + return closeFuture != null; + } + + public CompletableFuture watch(ByteSequence key, + BiConsumer consumer) { + return watch(key, WatchOption.DEFAULT, consumer); + } + + public CompletableFuture watch(ByteSequence key, + WatchOption watchOption, + BiConsumer consumer) { + return CompletableFuture.supplyAsync(() -> { + if (isClosed()) { + throw EtcdExceptionFactory.newClosedWatchClientException(); + } + + EtcdWatcher watcher = new EtcdWatcher(key, watchOption, scheduler.chooseThread(), this); + watcher.addConsumer(consumer); + pendingWatchers.add(watcher); + if (pendingWatchers.size() == 1) { + WatchRequest request = toWatchCreateRequest(watcher); + getGrpcWatchStreamObserver().onNext(request); + } + return watcher; + }, watchExecutor); + } + + // notifies all watchers about a exception. it doesn't close watchers. + // it is the responsibility of user to close watchers. + private void notifyWatchers(EtcdException e) { + WatchResponseWithError wre = new WatchResponseWithError(e); + this.pendingWatchers.forEach(watcher -> watcher.notifyWatchResponse(wre)); + this.pendingWatchers.clear(); + this.watchers.values().forEach(watcher -> watcher.notifyWatchResponse(wre)); + this.watchers.clear(); + } + + public CompletableFuture unwatch(EtcdWatcher watcher) { + return CompletableFuture.runAsync(() -> cancelWatcher(watcher.getWatchID()), watchExecutor); + } + + private void cancelWatcher(long watchID) { + if (isClosed()) { + return; + } + + if (cancelSet.contains(watchID)) { + return; + } + + watchers.remove(watchID); + cancelSet.add(watchID); + + WatchCancelRequest watchCancelRequest = WatchCancelRequest.newBuilder() + .setWatchId(watchID) + .build(); + WatchRequest cancelRequest = WatchRequest.newBuilder() + .setCancelRequest(watchCancelRequest) + .build(); + getGrpcWatchStreamObserver().onNext(cancelRequest); + } + + public CompletableFuture closeAsync() { + CompletableFuture future; + synchronized (this) { + if (null == closeFuture) { + log.info("Closing watch client"); + closeFuture = CompletableFuture.runAsync(() -> { + notifyWatchers(newClosedWatchClientException()); + closeGrpcWatchStreamObserver(); + }, watchExecutor); + } + future = closeFuture; + } + return future.whenComplete((ignored, cause) -> { + this.scheduler.shutdown(); + }); + } + + @Override + public void close() { + try { + FutureUtils.result(closeAsync()); + } catch (Exception e) { + log.warn("Encountered exceptions on closing watch client", e); + } + this.scheduler.forceShutdown(10, TimeUnit.SECONDS); + } + + private StreamObserver createWatchStreamObserver() { + return new StreamObserver() { + @Override + public void onNext(WatchResponse watchResponse) { + if (isClosed()) { + return; + } + watchExecutor.submit(() -> processWatchResponse(watchResponse)); + } + + @Override + public void onError(Throwable t) { + if (isClosed()) { + return; + } + watchExecutor.submit(() -> processError(t)); + } + + @Override + public void onCompleted() { + } + }; + } + + private void processWatchResponse(WatchResponse watchResponse) { + // prevents grpc on sending watchResponse to a closed watch client. + if (isClosed()) { + return; + } + + if (watchResponse.getCreated()) { + processCreate(watchResponse); + } else if (watchResponse.getCanceled()) { + processCanceled(watchResponse); + } else { + processEvents(watchResponse); + } + } + + private void processError(Throwable t) { + // prevents grpc on sending error to a closed watch client. + if (this.isClosed()) { + return; + } + + Status status = Status.fromThrowable(t); + if (this.isHaltError(status) || this.isNoLeaderError(status)) { + this.notifyWatchers(toEtcdException(status)); + this.closeGrpcWatchStreamObserver(); + this.cancelSet.clear(); + return; + } + // resume with a delay; avoiding immediate retry on a long connection downtime. + scheduler.schedule(this::resume, 500, TimeUnit.MILLISECONDS); + } + + private void resume() { + this.closeGrpcWatchStreamObserver(); + this.cancelSet.clear(); + this.resumeWatchers(); + } + + private synchronized StreamObserver getGrpcWatchStreamObserver() { + if (this.grpcWatchStreamObserver == null) { + this.grpcWatchStreamObserver = this.stub.watch(this.createWatchStreamObserver()); + } + return this.grpcWatchStreamObserver; + } + + // closeGrpcWatchStreamObserver closes the underlying grpc watch stream. + private void closeGrpcWatchStreamObserver() { + if (this.grpcWatchStreamObserver == null) { + return; + } + this.grpcWatchStreamObserver.onCompleted(); + this.grpcWatchStreamObserver = null; + } + + private void processCreate(WatchResponse response) { + EtcdWatcher watcher = this.pendingWatchers.poll(); + + this.sendNextWatchCreateRequest(); + + if (watcher == null) { + // shouldn't happen + // may happen due to duplicate watch create responses. + log.warn("Watch client receives watch create response but find no corresponding watcher"); + return; + } + + if (watcher.isClosed()) { + return; + } + + if (response.getWatchId() == -1) { + watcher.notifyWatchResponse(new WatchResponseWithError( + newEtcdException(ErrorCode.INTERNAL, "etcd server failed to create watch id"))); + return; + } + + if (watcher.getRevision() == 0) { + watcher.setRevision(response.getHeader().getRevision()); + } + + watcher.setWatchID(response.getWatchId()); + this.watchers.put(watcher.getWatchID(), watcher); + } + + /** + * chooses the next resuming watcher to register with the grpc stream. + */ + private Optional nextResume() { + EtcdWatcher pendingWatcher = this.pendingWatchers.peek(); + if (pendingWatcher != null) { + return Optional.of(this.toWatchCreateRequest(pendingWatcher)); + } + return Optional.empty(); + } + + private void sendNextWatchCreateRequest() { + this.nextResume().ifPresent( + (nextWatchRequest -> this.getGrpcWatchStreamObserver().onNext(nextWatchRequest))); + } + + private void processEvents(WatchResponse response) { + EtcdWatcher watcher = this.watchers.get(response.getWatchId()); + if (watcher == null) { + // cancel server side watcher. + this.cancelWatcher(response.getWatchId()); + return; + } + + if (response.getCompactRevision() != 0) { + watcher.notifyWatchResponse(new WatchResponseWithError( + EtcdExceptionFactory + .newCompactedException(response.getCompactRevision()))); + return; + } + + if (response.getEventsCount() == 0) { + watcher.setRevision(response.getHeader().getRevision()); + return; + } + + watcher.notifyWatchResponse(new WatchResponseWithError(response)); + watcher.setRevision( + response + .getEvents(response.getEventsCount() - 1) + .getKv().getModRevision() + 1); + } + + private void resumeWatchers() { + this.watchers.values().forEach(watcher -> { + if (watcher.isClosed()) { + return; + } + watcher.setWatchID(-1); + this.pendingWatchers.add(watcher); + }); + + this.watchers.clear(); + + this.sendNextWatchCreateRequest(); + } + + private void processCanceled(WatchResponse response) { + EtcdWatcher watcher = this.watchers.get(response.getWatchId()); + this.cancelSet.remove(response.getWatchId()); + if (watcher == null) { + return; + } + String reason = response.getCancelReason(); + if (Strings.isNullOrEmpty(reason)) { + watcher.notifyWatchResponse(new WatchResponseWithError(newEtcdException( + ErrorCode.OUT_OF_RANGE, + "etcdserver: mvcc: required revision is a future revision")) + ); + + } else { + watcher.notifyWatchResponse( + new WatchResponseWithError(newEtcdException(ErrorCode.FAILED_PRECONDITION, reason))); + } + } + + private static boolean isNoLeaderError(Status status) { + return status.getCode() == Code.UNAVAILABLE + && "etcdserver: no leader".equals(status.getDescription()); + } + + private static boolean isHaltError(Status status) { + // Unavailable codes mean the system will be right back. + // (e.g., can't connect, lost leader) + // Treat Internal codes as if something failed, leaving the + // system in an inconsistent state, but retrying could make progress. + // (e.g., failed in middle of send, corrupted frame) + return status.getCode() != Code.UNAVAILABLE && status.getCode() != Code.INTERNAL; + } + + private static WatchRequest toWatchCreateRequest(EtcdWatcher watcher) { + ByteString key = UnsafeByteOperations.unsafeWrap(watcher.getKey().getBytes()); + WatchOption option = watcher.getWatchOption(); + WatchCreateRequest.Builder builder = WatchCreateRequest.newBuilder() + .setKey(key) + .setPrevKv(option.isPrevKV()) + .setProgressNotify(option.isProgressNotify()) + .setStartRevision(watcher.getRevision()); + + option.getEndKey() + .ifPresent(endKey -> builder.setRangeEnd(UnsafeByteOperations.unsafeWrap(endKey.getBytes()))); + + if (option.isNoDelete()) { + builder.addFilters(WatchCreateRequest.FilterType.NODELETE); + } + + if (option.isNoPut()) { + builder.addFilters(WatchCreateRequest.FilterType.NOPUT); + } + + return WatchRequest.newBuilder().setCreateRequest(builder).build(); + } + + +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdWatcher.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdWatcher.java new file mode 100644 index 00000000000..ad3b6a38907 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdWatcher.java @@ -0,0 +1,115 @@ +/* + * 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.metadata.etcd; + +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.options.WatchOption; +import com.coreos.jetcd.watch.WatchResponse; +import com.coreos.jetcd.watch.WatchResponseWithError; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.BiConsumer; +import lombok.Getter; +import lombok.Setter; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.concurrent.FutureUtils; + +/** + * Watcher class holds watcher information. + */ +@Slf4j +public class EtcdWatcher implements AutoCloseable { + + private final ScheduledExecutorService executor; + @Getter + private final WatchOption watchOption; + @Getter + private final ByteSequence key; + // watch listener + private final CopyOnWriteArraySet> consumers; + @Getter + @Setter + private long watchID; + // the revision to watch on. + @Getter + @Setter + private long revision; + private boolean closed = false; + // the client owns this watcher + private final EtcdWatchClient owner; + + EtcdWatcher(ByteSequence key, + WatchOption watchOption, + ScheduledExecutorService executor, + EtcdWatchClient owner) { + this.key = key; + this.watchOption = watchOption; + this.executor = executor; + this.owner = owner; + this.consumers = new CopyOnWriteArraySet<>(); + } + + public void addConsumer(BiConsumer consumer) { + this.consumers.add(consumer); + } + + synchronized boolean isClosed() { + return closed; + } + + void notifyWatchResponse(WatchResponseWithError watchResponse) { + synchronized (this) { + if (closed) { + return; + } + } + + this.executor.submit(() -> consumers.forEach(c -> { + if (watchResponse.getException() != null) { + c.accept(null, watchResponse.getException()); + } else { + c.accept( + new WatchResponse(watchResponse.getWatchResponse()), + null); + } + })); + } + + public CompletableFuture closeAsync() { + return owner.unwatch(this); + } + + @Override + public void close() { + synchronized (this) { + if (closed) { + return; + } + closed = true; + } + try { + FutureUtils.result(closeAsync()); + } catch (Exception e) { + log.warn("Encountered error on removing watcher '{}' from watch client : {}", + watchID, e.getMessage()); + } + consumers.clear(); + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java new file mode 100644 index 00000000000..a201b253811 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java @@ -0,0 +1,72 @@ +/* + * Licensed 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.metadata.etcd; + +import java.util.Objects; +import java.util.function.Consumer; +import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * A consumer wrapper over ledger metadata listener. + */ +class LedgerMetadataConsumer implements Consumer> { + + private final long ledgerId; + private final LedgerMetadataListener listener; + private final Consumer onDeletedConsumer; + + LedgerMetadataConsumer(long ledgerId, + LedgerMetadataListener listener, + Consumer onDeletedConsumer) { + this.ledgerId = ledgerId; + this.listener = listener; + this.onDeletedConsumer = onDeletedConsumer; + } + + @Override + public int hashCode() { + return listener.hashCode(); + } + + @Override + public String toString() { + return listener.toString(); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof LedgerMetadataConsumer)) { + return false; + } + + LedgerMetadataConsumer another = (LedgerMetadataConsumer) obj; + return ledgerId == another.ledgerId + && Objects.equals(listener, another.listener) + && Objects.equals(onDeletedConsumer, another.onDeletedConsumer); + } + + @Override + public void accept(Versioned ledgerMetadataVersioned) { + LedgerMetadata lm = ledgerMetadataVersioned.getValue(); + if (null == lm) { + onDeletedConsumer.accept(ledgerId); + } else { + lm.setVersion(ledgerMetadataVersioned.getVersion()); + listener.onChanged(ledgerId, lm); + } + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeyIterator.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeyIterator.java new file mode 100644 index 00000000000..7bf628412a4 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeyIterator.java @@ -0,0 +1,68 @@ +/* + * 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.metadata.etcd.helpers; + +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; + +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; + +/** + * Iterator over a range of key/value pairs. + */ +public class KeyIterator { + + private final KeyStream stream; + private CompletableFuture> readFuture = null; + private boolean hasNext = true; + private List keys = null; + + public KeyIterator(KeyStream stream) { + this.stream = stream; + } + + public synchronized boolean hasNext() throws Exception { + if (hasNext) { + if (null == readFuture) { + readFuture = stream.readNext(); + } + keys = result(readFuture); + if (keys.isEmpty()) { + hasNext = false; + } + return hasNext; + } else { + return false; + } + } + + public synchronized List next() throws Exception { + try { + if (!hasNext()) { + throw new NoSuchElementException("Reach end of key stream"); + } + return keys; + } finally { + // read next + readFuture = stream.readNext(); + } + } + +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeySetReader.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeySetReader.java new file mode 100644 index 00000000000..d3da0597fa6 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeySetReader.java @@ -0,0 +1,282 @@ +/* + * 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.metadata.etcd.helpers; + +import com.coreos.jetcd.Client; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.data.KeyValue; +import com.coreos.jetcd.kv.GetResponse; +import com.coreos.jetcd.options.GetOption; +import com.coreos.jetcd.options.WatchOption; +import com.coreos.jetcd.watch.WatchResponse; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Sets; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.metadata.etcd.EtcdWatchClient; +import org.apache.bookkeeper.metadata.etcd.EtcdWatcher; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * A helper class to read a set of keys and watch them. + */ +@Slf4j +public class KeySetReader implements BiConsumer, AutoCloseable { + + private final Client client; + private final boolean ownWatchClient; + private final EtcdWatchClient watchClient; + private final Function encoder; + private final ByteSequence beginKey; + private final ByteSequence endKey; + private final Set keys; + private final CopyOnWriteArraySet>>> consumers = + new CopyOnWriteArraySet<>(); + private volatile long revision = -1L; + private CompletableFuture watchFuture = null; + private CompletableFuture closeFuture = null; + + public KeySetReader(Client client, + Function encoder, + ByteSequence beginKey, + ByteSequence endKey) { + this(client, new EtcdWatchClient(client), encoder, beginKey, endKey); + } + + public KeySetReader(Client client, + EtcdWatchClient watchClient, + Function encoder, + ByteSequence beginKey, + ByteSequence endKey) { + this.client = client; + this.watchClient = watchClient; + this.ownWatchClient = false; + this.encoder = encoder; + this.beginKey = beginKey; + this.endKey = endKey; + this.keys = Collections.synchronizedSet(Sets.newHashSet()); + } + + public CompletableFuture>> read() { + GetOption.Builder optionBuilder = GetOption.newBuilder() + .withKeysOnly(true); + if (null != endKey) { + optionBuilder.withRange(endKey); + } + return client.getKVClient().get( + beginKey, + optionBuilder.build() + ).thenApply(getResp -> { + boolean updated = updateLocalValue(getResp); + Versioned> localValue = getLocalValue(); + try { + return localValue; + } finally { + if (updated) { + notifyConsumers(localValue); + } + } + }); + } + + @VisibleForTesting + long getRevision() { + return revision; + } + + private void notifyConsumers(Versioned> localValue) { + consumers.forEach(consumer -> consumer.accept(localValue)); + } + + private synchronized boolean updateLocalValue(GetResponse response) { + if (revision < response.getHeader().getRevision()) { + revision = response.getHeader().getRevision(); + keys.clear(); + for (KeyValue kv : response.getKvs()) { + ByteSequence key = kv.getKey(); + keys.add(key); + } + return true; + } else { + return false; + } + } + + private synchronized Versioned> processWatchResponse(WatchResponse response) { + if (null != closeFuture) { + return null; + } + if (log.isDebugEnabled()) { + log.debug("Received watch response : revision = {}, {} events = {}", + response.getHeader().getRevision(), response.getEvents().size(), response.getEvents()); + } + + if (response.getHeader().getRevision() <= revision) { + return null; + } + revision = response.getHeader().getRevision(); + response.getEvents().forEach(event -> { + switch (event.getEventType()) { + case PUT: + keys.add(event.getKeyValue().getKey()); + break; + case DELETE: + keys.remove(event.getKeyValue().getKey()); + break; + default: + // ignore + break; + } + + }); + return getLocalValue(); + } + + @VisibleForTesting + synchronized Versioned> getLocalValue() { + return new Versioned<>( + keys.stream().map(encoder).collect(Collectors.toSet()), + new LongVersion(revision) + ); + } + + private CompletableFuture>> getOrRead() { + boolean shouldRead = false; + synchronized (this) { + if (revision < 0L) { + // the value is never cached. + shouldRead = true; + } + } + if (shouldRead) { + return read(); + } else { + return FutureUtils.value(getLocalValue()); + } + } + + @VisibleForTesting + synchronized boolean isWatcherSet() { + return null != watchFuture; + } + + public CompletableFuture>> readAndWatch(Consumer>> consumer) { + if (!consumers.add(consumer) || isWatcherSet()) { + return getOrRead(); + } + + return getOrRead() + .thenCompose(versionedKeys -> { + long revision = ((LongVersion) versionedKeys.getVersion()).getLongVersion(); + return watch(revision).thenApply(ignored -> versionedKeys); + }); + } + + public CompletableFuture unwatch(Consumer>> consumer) { + if (consumers.remove(consumer) && consumers.isEmpty()) { + return closeOrRewatch(false); + } else { + return FutureUtils.Void(); + } + } + + private synchronized CompletableFuture watch(long revision) { + if (null != watchFuture) { + return watchFuture; + } + + WatchOption.Builder optionBuilder = WatchOption.newBuilder() + .withRevision(revision); + if (null != endKey) { + optionBuilder.withRange(endKey); + } + watchFuture = watchClient.watch(beginKey, optionBuilder.build(), this); + return watchFuture.whenComplete((watcher, cause) -> { + if (null != cause) { + synchronized (KeySetReader.this) { + watchFuture = null; + } + } + }); + } + + private CompletableFuture closeOrRewatch(boolean rewatch) { + CompletableFuture oldWatcherFuture; + synchronized (this) { + oldWatcherFuture = watchFuture; + if (rewatch && null == closeFuture) { + watchFuture = watch(revision); + } else { + watchFuture = null; + } + } + if (null != oldWatcherFuture) { + return oldWatcherFuture.thenCompose(EtcdWatcher::closeAsync); + } else { + return FutureUtils.Void(); + } + } + + @Override + public void accept(WatchResponse watchResponse, Throwable throwable) { + if (null == throwable) { + Versioned> localValue = processWatchResponse(watchResponse); + if (null != localValue) { + notifyConsumers(localValue); + } + } else { + closeOrRewatch(true); + } + } + + public CompletableFuture closeAsync() { + CompletableFuture future; + synchronized (this) { + if (null == closeFuture) { + closeFuture = closeOrRewatch(false).thenCompose(ignored -> { + if (ownWatchClient) { + return watchClient.closeAsync(); + } else { + return FutureUtils.Void(); + } + }); + } + future = closeFuture; + } + return future; + } + + @Override + public void close() { + try { + FutureUtils.result(closeAsync()); + } catch (Exception e) { + log.warn("Encountered exceptions on closing key reader : {}", e.getMessage()); + } + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeyStream.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeyStream.java new file mode 100644 index 00000000000..1f23557e5e8 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/KeyStream.java @@ -0,0 +1,120 @@ +/* + * 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.metadata.etcd.helpers; + +import com.coreos.jetcd.KV; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.data.KeyValue; +import com.coreos.jetcd.options.GetOption; +import com.coreos.jetcd.options.GetOption.SortOrder; +import com.coreos.jetcd.options.GetOption.SortTarget; +import com.google.common.primitives.UnsignedBytes; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.concurrent.FutureUtils; + +/** + * Read a range of key/value pairs in a streaming way. + */ +@Slf4j +public class KeyStream { + + private final KV kvClient; + private final ByteSequence startKey; + private final ByteSequence endKey; + private final Function encoder; + private final int batchSize; + private ByteSequence nextKey; + private ByteSequence lastKey = null; + private boolean hasMore = true; + + public KeyStream(KV kvClient, + ByteSequence startKey, + ByteSequence endKey, + Function encoder) { + this(kvClient, startKey, endKey, encoder, 100); + } + + public KeyStream(KV kvClient, + ByteSequence startKey, + ByteSequence endKey, + Function encoder, + int batchSize) { + this.kvClient = kvClient; + this.startKey = startKey; + this.endKey = endKey; + this.nextKey = startKey; + this.encoder = encoder; + this.batchSize = batchSize; + } + + public CompletableFuture> readNext() { + ByteSequence beginKey; + int batchSize = this.batchSize; + synchronized (this) { + if (!hasMore) { + return FutureUtils.value(Collections.emptyList()); + } + + beginKey = nextKey; + if (null != lastKey) { + // read one more in since we are including last key. + batchSize += 1; + } + } + if (log.isTraceEnabled()) { + log.trace("Read keys between {} and {}", beginKey.toStringUtf8(), endKey.toStringUtf8()); + } + return kvClient.get( + beginKey, + GetOption.newBuilder() + .withRange(endKey) + .withKeysOnly(true) + .withLimit(batchSize) + .withSortField(SortTarget.KEY) + .withSortOrder(SortOrder.ASCEND) + .build() + ).thenApply(getResp -> { + List kvs = getResp.getKvs(); + ByteSequence lkey; + synchronized (KeyStream.this) { + hasMore = getResp.isMore(); + lkey = lastKey; + if (kvs.size() > 0) { + lastKey = nextKey = kvs.get(kvs.size() - 1).getKey(); + } + } + if (null != lkey + && kvs.size() > 0 + && UnsignedBytes.lexicographicalComparator().compare( + lkey.getBytes(), + kvs.get(0).getKey().getBytes()) == 0) { + kvs.remove(0); + } + return kvs.stream() + .map(kv -> encoder.apply(kv.getKey())) + .collect(Collectors.toList()); + }); + } + +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/RevisionedConsumer.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/RevisionedConsumer.java new file mode 100644 index 00000000000..69095a01287 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/RevisionedConsumer.java @@ -0,0 +1,67 @@ +/* + * Licensed 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.metadata.etcd.helpers; + +import java.util.function.Consumer; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Version.Occurred; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * A consumer that cache last version. + */ +public class RevisionedConsumer implements Consumer> { + + protected final Consumer> consumer; + protected volatile Version localVersion = null; + + protected RevisionedConsumer(Consumer> consumer) { + this.consumer = consumer; + } + + @Override + public void accept(Versioned versionedVal) { + synchronized (this) { + if (localVersion != null + && Occurred.BEFORE != localVersion.compare(versionedVal.getVersion())) { + return; + } + localVersion = versionedVal.getVersion(); + } + consumer.accept(versionedVal); + } + + @Override + public int hashCode() { + return consumer.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof Consumer)) { + return false; + } + if (obj instanceof RevisionedConsumer) { + return consumer.equals(((RevisionedConsumer) obj).consumer); + } else { + return consumer.equals(obj); + } + } + + @Override + public String toString() { + return consumer.toString(); + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/ValueStream.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/ValueStream.java new file mode 100644 index 00000000000..a12e99f0a0b --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/ValueStream.java @@ -0,0 +1,310 @@ +/* + * Licensed 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.metadata.etcd.helpers; + +import com.coreos.jetcd.Client; +import com.coreos.jetcd.common.exception.ClosedClientException; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.kv.GetResponse; +import com.coreos.jetcd.options.WatchOption; +import com.coreos.jetcd.watch.WatchResponse; +import com.google.common.annotations.VisibleForTesting; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.metadata.etcd.EtcdWatchClient; +import org.apache.bookkeeper.metadata.etcd.EtcdWatcher; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * A helper class to read the stream of values of a given key. + */ +@Slf4j +public class ValueStream implements BiConsumer, AutoCloseable { + + private final Client client; + private final boolean ownWatchClient; + private final EtcdWatchClient watchClient; + private final Function encoder; + private final ByteSequence key; + private final Map>, RevisionedConsumer> consumers = + new HashMap<>(); + private volatile T localValue = null; + private volatile long revision = -1L; + private CompletableFuture watchFuture = null; + private CompletableFuture closeFuture = null; + + public ValueStream(Client client, + Function encoder, + ByteSequence key) { + this(client, new EtcdWatchClient(client), encoder, key); + } + + public ValueStream(Client client, + EtcdWatchClient watchClient, + Function encoder, + ByteSequence key) { + this.client = client; + this.watchClient = watchClient; + this.ownWatchClient = false; + this.encoder = encoder; + this.key = key; + } + + public CompletableFuture> read() { + return client.getKVClient().get( + key + ).thenApply(getResp -> { + boolean updated = updateLocalValue(getResp); + Versioned localValue = getLocalValue(); + try { + return localValue; + } finally { + if (updated) { + notifyConsumers(localValue); + } + } + }); + } + + @VisibleForTesting + public int getNumConsumers() { + synchronized (consumers) { + return consumers.size(); + } + } + + private void notifyConsumers(Versioned localValue) { + synchronized (consumers) { + consumers.values().forEach(consumer -> consumer.accept(localValue)); + } + } + + private synchronized boolean updateLocalValue(GetResponse response) { + if (revision < response.getHeader().getRevision()) { + revision = response.getHeader().getRevision(); + if (response.getCount() > 0) { + localValue = encoder.apply(response.getKvs().get(0).getValue()); + } else { + localValue = null; + } + return true; + } else { + return false; + } + } + + private synchronized Versioned processWatchResponse(WatchResponse response) { + if (null != closeFuture) { + return null; + } + if (log.isDebugEnabled()) { + log.debug("Received watch response : revision = {}, {} events = {}", + response.getHeader().getRevision(), response.getEvents().size(), response.getEvents()); + } + + if (response.getHeader().getRevision() <= revision) { + return null; + } + revision = response.getHeader().getRevision(); + response.getEvents().forEach(event -> { + switch (event.getEventType()) { + case PUT: + this.localValue = encoder.apply(event.getKeyValue().getValue()); + break; + case DELETE: + this.localValue = null; + break; + default: + // ignore + break; + } + + }); + return getLocalValue(); + } + + @VisibleForTesting + synchronized Versioned getLocalValue() { + return new Versioned<>( + localValue, + new LongVersion(revision) + ); + } + + private CompletableFuture> getOrRead() { + boolean shouldRead = false; + synchronized (this) { + if (revision < 0L) { + // the value is never cached. + shouldRead = true; + } + } + if (shouldRead) { + return read(); + } else { + return FutureUtils.value(getLocalValue()); + } + } + + @VisibleForTesting + synchronized boolean isWatcherSet() { + return null != watchFuture; + } + + private synchronized CompletableFuture getWatchFuture() { + return this.watchFuture; + } + + @VisibleForTesting + public CompletableFuture waitUntilWatched() { + CompletableFuture wf; + while ((wf = getWatchFuture()) == null) { + try { + TimeUnit.MILLISECONDS.sleep(100); + } catch (InterruptedException e) { + if (log.isDebugEnabled()) { + log.debug("Interrupted at waiting until the key is watched", e); + } + } + } + return wf; + } + + public CompletableFuture> readAndWatch(Consumer> consumer) { + final RevisionedConsumer revisionedConsumer = new RevisionedConsumer<>(consumer); + final boolean consumerExisted; + synchronized (consumers) { + consumerExisted = (null != consumers.put(consumer, revisionedConsumer)); + } + if (consumerExisted) { + return getOrRead(); + } + + return getOrRead() + .thenCompose(versionedVal -> { + long revision = ((LongVersion) versionedVal.getVersion()).getLongVersion(); + synchronized (this) { + notifyConsumers(versionedVal); + } + return watchIfNeeded(revision).thenApply(ignored -> versionedVal); + }); + } + + public CompletableFuture unwatch(Consumer> consumer) { + boolean lastConsumer; + synchronized (consumers) { + lastConsumer = (null != consumers.remove(consumer) && consumers.isEmpty()); + } + if (lastConsumer) { + return closeOrRewatch(false).thenApply(ignored -> true); + } else { + return FutureUtils.value(false); + } + } + + private synchronized CompletableFuture watchIfNeeded(long revision) { + if (null != watchFuture) { + return watchFuture; + } + watchFuture = watch(revision); + return watchFuture; + } + + private CompletableFuture watch(long revision) { + WatchOption.Builder optionBuilder = WatchOption.newBuilder() + .withRevision(revision); + return watchClient.watch(key, optionBuilder.build(), this) + .whenComplete((watcher, cause) -> { + if (null != cause) { + synchronized (ValueStream.this) { + watchFuture = null; + } + } + }); + } + + private CompletableFuture closeOrRewatch(boolean rewatch) { + CompletableFuture oldWatcherFuture; + synchronized (this) { + oldWatcherFuture = watchFuture; + if (rewatch && null == closeFuture) { + watchFuture = watch(revision); + } else { + watchFuture = null; + } + } + if (null != oldWatcherFuture) { + return oldWatcherFuture.thenCompose(EtcdWatcher::closeAsync); + } else { + return FutureUtils.Void(); + } + } + + @Override + public void accept(WatchResponse watchResponse, Throwable throwable) { + if (null == throwable) { + if (log.isDebugEnabled()) { + log.debug("Received watch response : revision = {}, {} events = {}", + watchResponse.getHeader().getRevision(), + watchResponse.getEvents().size(), + watchResponse.getEvents()); + } + + synchronized (this) { + Versioned localValue = processWatchResponse(watchResponse); + if (null != localValue) { + notifyConsumers(localValue); + } + } + } else { + // rewatch if it is not a `ClosedClientException` + closeOrRewatch(!(throwable instanceof ClosedClientException)); + } + } + + public CompletableFuture closeAsync() { + CompletableFuture future; + synchronized (this) { + if (null == closeFuture) { + closeFuture = closeOrRewatch(false).thenCompose(ignored -> { + if (ownWatchClient) { + return watchClient.closeAsync(); + } else { + return FutureUtils.Void(); + } + }); + } + future = closeFuture; + } + return future; + } + + @Override + public void close() { + try { + FutureUtils.result(closeAsync()); + } catch (Exception e) { + log.warn("Encountered exceptions on closing key reader : {}", e.getMessage()); + } + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/package-info.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/package-info.java new file mode 100644 index 00000000000..ce8f2e515ed --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/helpers/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. + */ + +/** + * Etcd helper classes. + */ +package org.apache.bookkeeper.metadata.etcd.helpers; \ No newline at end of file diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/package-info.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/package-info.java new file mode 100644 index 00000000000..81369d82cc1 --- /dev/null +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/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. + */ + +/** + * Etcd based metadata driver. + */ +package org.apache.bookkeeper.metadata.etcd; \ No newline at end of file diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/Etcd64bitIdGeneratorTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/Etcd64bitIdGeneratorTest.java new file mode 100644 index 00000000000..53ad49bdc8f --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/Etcd64bitIdGeneratorTest.java @@ -0,0 +1,153 @@ +/* + * 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.metadata.etcd; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.coreos.jetcd.Client; +import com.google.common.util.concurrent.RateLimiter; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; +import org.apache.commons.lang.RandomStringUtils; +import org.junit.Before; +import org.junit.Test; + +/** + * Integration test {@link Etcd64bitIdGenerator}. + */ +@Slf4j +public class Etcd64bitIdGeneratorTest extends EtcdTestBase { + + private String scope; + private Etcd64bitIdGenerator generator; + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + this.scope = "/" + RandomStringUtils.randomAlphabetic(8); + this.generator = new Etcd64bitIdGenerator(etcdClient.getKVClient(), scope); + log.info("Setup id generator under scope {}", scope); + } + + @Test + public void testGenerateIdSequence() throws Exception { + Map buckets = new HashMap<>(); + + int numIterations = 10; + + for (int i = 0; i < numIterations; i++) { + log.info("Id generation iteration : {}", i); + for (int j = 0; j < Etcd64bitIdGenerator.NUM_BUCKETS; j++) { + GenericCallbackFuture future = new GenericCallbackFuture<>(); + generator.generateLedgerId(future); + long lid = future.get(); + int bucketId = Etcd64bitIdGenerator.getBucketId(lid); + long idInBucket = Etcd64bitIdGenerator.getIdInBucket(lid); + Long prevIdInBucket = buckets.put(bucketId, idInBucket); + if (null == prevIdInBucket) { + assertEquals(1, idInBucket); + } else { + assertEquals(prevIdInBucket + 1, idInBucket); + } + } + } + + assertEquals(Etcd64bitIdGenerator.NUM_BUCKETS, buckets.size()); + for (Map.Entry bucketEntry : buckets.entrySet()) { + assertEquals(numIterations, bucketEntry.getValue().intValue()); + } + } + + /** + * Test generating id in parallel and ensure there is no duplicated id. + */ + @Test + public void testGenerateIdParallel() throws Exception { + final int numThreads = 10; + @Cleanup("shutdown") + ExecutorService executor = Executors.newFixedThreadPool(numThreads); + + final int numIds = 10000; + final AtomicLong totalIds = new AtomicLong(numIds); + final Set ids = Collections.newSetFromMap(new ConcurrentHashMap<>()); + final RateLimiter limiter = RateLimiter.create(1000); + final CompletableFuture doneFuture = new CompletableFuture<>(); + for (int i = 0; i < numThreads; i++) { + executor.submit(() -> { + Client client = Client.builder() + .endpoints(etcdContainer.getClientEndpoint()) + .build(); + Etcd64bitIdGenerator gen = new Etcd64bitIdGenerator( + client.getKVClient(), + scope + ); + + AtomicBoolean running = new AtomicBoolean(true); + + while (running.get()) { + limiter.acquire(); + + GenericCallbackFuture genFuture = new GenericCallbackFuture<>(); + gen.generateLedgerId(genFuture); + + genFuture + .thenAccept(lid -> { + boolean duplicatedFound = !(ids.add(lid)); + if (duplicatedFound) { + running.set(false); + doneFuture.completeExceptionally( + new IllegalStateException("Duplicated id " + lid + " generated : " + ids)); + return; + } else { + if (totalIds.decrementAndGet() <= 0) { + running.set(false); + doneFuture.complete(null); + } + } + }) + .exceptionally(cause -> { + running.set(false); + doneFuture.completeExceptionally(cause); + return null; + }); + } + }); + } + + FutureUtils.result(doneFuture); + assertTrue(totalIds.get() <= 0); + assertTrue(ids.size() >= numIds); + } + +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdClusterTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdClusterTest.java new file mode 100644 index 00000000000..c13e56f84ba --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdClusterTest.java @@ -0,0 +1,341 @@ +/* + * 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.metadata.etcd; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getBookiesPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getBucketsPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getClusterInstanceIdPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getLayoutKey; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getLedgersPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getReadonlyBookiesPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getScopeEndKey; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getUnderreplicationPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.getWritableBookiesPath; +import static org.apache.bookkeeper.metadata.etcd.EtcdUtils.msResult; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.coreos.jetcd.Client; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.kv.GetResponse; +import com.coreos.jetcd.options.GetOption; +import java.util.UUID; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.LedgerLayout; +import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +import org.apache.commons.lang.RandomStringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * Test cluster related operation on Etcd based registration manager. + */ +@Slf4j +public class EtcdClusterTest extends EtcdTestBase { + + private String scope; + private RegistrationManager regMgr; + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + this.scope = RandomStringUtils.randomAlphabetic(32); + this.regMgr = new EtcdRegistrationManager( + newEtcdClient(), scope + ); + } + + @After + @Override + public void tearDown() throws Exception { + this.regMgr.close(); + super.tearDown(); + } + + @Test + public void testGetClusterInstanceIdIfClusterNotInitialized() throws Exception { + try { + regMgr.getClusterInstanceId(); + fail("Should fail getting cluster instance id if cluster not initialized"); + } catch (MetadataStoreException e) { + assertTrue(e.getMessage().contains("BookKeeper is not initialized")); + } + } + + @Test + public void testGetClusterInstanceId() throws Exception { + assertClusterNotExists(etcdClient, scope); + regMgr.initNewCluster(); + String instanceId = regMgr.getClusterInstanceId(); + UUID uuid = UUID.fromString(instanceId); + log.info("Cluster instance id : {}", uuid); + } + + @Test + public void testNukeNonExistingCluster() throws Exception { + assertClusterNotExists(etcdClient, scope); + assertTrue(regMgr.nukeExistingCluster()); + assertClusterNotExists(etcdClient, scope); + } + + @Test + public void testNukeExistingCluster() throws Exception { + assertTrue(regMgr.initNewCluster()); + assertClusterExists(etcdClient, scope); + assertTrue(regMgr.nukeExistingCluster()); + assertClusterNotExists(etcdClient, scope); + } + + @Test + public void testInitNewClusterTwice() throws Exception { + assertTrue(regMgr.initNewCluster()); + assertClusterExists(etcdClient, scope); + String instanceId = regMgr.getClusterInstanceId(); + assertFalse(regMgr.initNewCluster()); + assertClusterExists(etcdClient, scope); + assertEquals(instanceId, regMgr.getClusterInstanceId()); + } + + @Test + public void testPrepareFormatNonExistingCluster() throws Exception { + assertFalse(regMgr.prepareFormat()); + } + + @Test + public void testPrepareFormatExistingCluster() throws Exception { + assertTrue(regMgr.initNewCluster()); + assertClusterExists(etcdClient, scope); + assertTrue(regMgr.prepareFormat()); + } + + @Test + public void testNukeExistingClusterWithWritableBookies() throws Exception { + testNukeExistingClusterWithBookies(false); + } + + @Test + public void testNukeExistingClusterWithReadonlyBookies() throws Exception { + testNukeExistingClusterWithBookies(true); + } + + private void testNukeExistingClusterWithBookies(boolean readonly) throws Exception { + assertTrue(regMgr.initNewCluster()); + assertClusterExists(etcdClient, scope); + createNumBookies(etcdClient, scope, 3, readonly); + assertFalse(regMgr.nukeExistingCluster()); + assertClusterExists(etcdClient, scope); + removeNumBookies(etcdClient, scope, 3, readonly); + assertTrue(regMgr.nukeExistingCluster()); + assertClusterNotExists(etcdClient, scope); + } + + @Test + public void testNukeExistingClusterWithAllBookies() throws Exception { + assertTrue(regMgr.initNewCluster()); + assertClusterExists(etcdClient, scope); + createNumBookies(etcdClient, scope, 1, false); + createNumBookies(etcdClient, scope, 2, true); + assertFalse(regMgr.nukeExistingCluster()); + assertClusterExists(etcdClient, scope); + removeNumBookies(etcdClient, scope, 1, false); + removeNumBookies(etcdClient, scope, 2, true); + assertTrue(regMgr.nukeExistingCluster()); + assertClusterNotExists(etcdClient, scope); + } + + @Test + public void testFormatNonExistingCluster() throws Exception { + assertClusterNotExists(etcdClient, scope); + assertTrue(regMgr.format()); + assertClusterExists(etcdClient, scope); + } + + @Test + public void testFormatExistingCluster() throws Exception { + assertClusterNotExists(etcdClient, scope); + assertTrue(regMgr.initNewCluster()); + assertClusterExists(etcdClient, scope); + String clusterInstanceId = regMgr.getClusterInstanceId(); + assertTrue(regMgr.format()); + assertClusterExists(etcdClient, scope); + assertNotEquals(clusterInstanceId, regMgr.getClusterInstanceId()); + } + + @Test + public void testFormatExistingClusterWithBookies() throws Exception { + assertClusterNotExists(etcdClient, scope); + assertTrue(regMgr.initNewCluster()); + assertClusterExists(etcdClient, scope); + String clusterInstanceId = regMgr.getClusterInstanceId(); + createNumBookies(etcdClient, scope, 3, false); + assertFalse(regMgr.format()); + assertClusterExists(etcdClient, scope); + assertEquals(clusterInstanceId, regMgr.getClusterInstanceId()); + } + + private static void createNumBookies(Client client, + String scope, + int numBookies, + boolean readonly) throws Exception { + for (int i = 0; i < numBookies; i++) { + String bookieId = "bookie-" + i + ":3181"; + String bookiePath; + if (readonly) { + bookiePath = EtcdUtils.getReadonlyBookiePath(scope, bookieId); + } else { + bookiePath = EtcdUtils.getWritableBookiePath(scope, bookieId); + } + msResult(client.getKVClient().put( + ByteSequence.fromString(bookiePath), + EtcdConstants.EMPTY_BS + )); + } + } + + private static void removeNumBookies(Client client, + String scope, + int numBookies, + boolean readonly) throws Exception { + for (int i = 0; i < numBookies; i++) { + String bookieId = "bookie-" + i + ":3181"; + String bookiePath; + if (readonly) { + bookiePath = EtcdUtils.getReadonlyBookiePath(scope, bookieId); + } else { + bookiePath = EtcdUtils.getWritableBookiePath(scope, bookieId); + } + msResult(client.getKVClient().delete( + ByteSequence.fromString(bookiePath) + )); + } + } + + private static void assertClusterScope(Client client, + String scope) throws Exception { + GetResponse resp = msResult( + client.getKVClient().get( + ByteSequence.fromString(scope))); + assertEquals(1, resp.getCount()); + } + + private static void assertClusterLayout(Client client, + String scope) throws Exception { + String layoutPath = getLayoutKey(scope); + GetResponse resp = msResult( + client.getKVClient().get( + ByteSequence.fromString(layoutPath))); + assertEquals(1, resp.getCount()); + LedgerLayout layout = LedgerLayout.parseLayout( + resp.getKvs().get(0).getValue().getBytes() + ); + assertEquals( + EtcdLedgerManagerFactory.class.getName(), + layout.getManagerFactoryClass() + ); + assertEquals(EtcdLedgerManagerFactory.VERSION, layout.getManagerVersion()); + assertEquals(LedgerLayout.LAYOUT_FORMAT_VERSION, layout.getLayoutFormatVersion()); + } + + private static void assertClusterInstanceId(Client client, + String scope) throws Exception { + String instanceIdPath = getClusterInstanceIdPath(scope); + GetResponse resp = msResult( + client.getKVClient().get(ByteSequence.fromString(instanceIdPath))); + assertEquals(1, resp.getCount()); + String instanceId = new String(resp.getKvs().get(0).getValue().getBytes(), UTF_8); + UUID uuid = UUID.fromString(instanceId); + log.info("Cluster instance id : {}", uuid); + } + + private static void assertBookiesPath(Client client, + String scope) throws Exception { + String bookiesPath = getBookiesPath(scope); + GetResponse resp = msResult( + client.getKVClient().get(ByteSequence.fromString(bookiesPath))); + assertEquals(1, resp.getCount()); + } + + private static void assertWritableBookiesPath(Client client, + String scope) throws Exception { + String bookiesPath = getWritableBookiesPath(scope); + GetResponse resp = msResult( + client.getKVClient().get(ByteSequence.fromString(bookiesPath))); + assertEquals(1, resp.getCount()); + } + + private static void assertReadonlyBookiesPath(Client client, + String scope) throws Exception { + String bookiesPath = getReadonlyBookiesPath(scope); + GetResponse resp = msResult( + client.getKVClient().get(ByteSequence.fromString(bookiesPath))); + assertEquals(1, resp.getCount()); + } + + private static void assertLedgersPath(Client client, String scope) throws Exception { + String ledgersPath = getLedgersPath(scope); + GetResponse resp = msResult( + client.getKVClient().get(ByteSequence.fromString(ledgersPath))); + assertEquals(1, resp.getCount()); + } + + private static void assertBucketsPath(Client client, String scope) throws Exception { + String bucketsPath = getBucketsPath(scope); + GetResponse resp = msResult( + client.getKVClient().get(ByteSequence.fromString(bucketsPath))); + assertEquals(1, resp.getCount()); + } + + private static void assertUnderreplicationPath(Client client, String scope) throws Exception { + String urPath = getUnderreplicationPath(scope); + GetResponse resp = msResult( + client.getKVClient().get(ByteSequence.fromString(urPath))); + assertEquals(1, resp.getCount()); + } + + private static void assertClusterExists(Client client, String scope) throws Exception { + assertClusterScope(client, scope); + assertClusterLayout(client, scope); + assertClusterInstanceId(client, scope); + assertBookiesPath(client, scope); + assertWritableBookiesPath(client, scope); + assertReadonlyBookiesPath(client, scope); + assertLedgersPath(client, scope); + assertBucketsPath(client, scope); + assertUnderreplicationPath(client, scope); + } + + private static void assertClusterNotExists(Client client, String scope) throws Exception { + GetResponse response = msResult( + client.getKVClient().get( + ByteSequence.fromString(scope), + GetOption.newBuilder() + .withRange(ByteSequence.fromString(getScopeEndKey(scope))) + .build())); + assertEquals(0, response.getCount()); + } + +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdCookieTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdCookieTest.java new file mode 100644 index 00000000000..692ff87ca7d --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdCookieTest.java @@ -0,0 +1,163 @@ +/* + * 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.metadata.etcd; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; +import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +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.commons.lang.RandomStringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; + +/** + * Test Etcd based cookie management. + */ +public class EtcdCookieTest extends EtcdTestBase { + + @Rule + public final TestName runtime = new TestName(); + + private RegistrationManager regMgr; + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + String scope = RandomStringUtils.randomAlphabetic(16); + this.regMgr = new EtcdRegistrationManager( + newEtcdClient(), + scope + ); + } + + @After + @Override + public void tearDown() throws Exception { + this.regMgr.close(); + super.tearDown(); + } + + private static void assertCookieEquals(Versioned expected, Versioned actual) { + assertEquals(Occurred.CONCURRENTLY, expected.getVersion().compare(actual.getVersion())); + assertArrayEquals(expected.getValue(), actual.getValue()); + } + + @Test + public void readWriteRemoveCookie() throws Exception { + String bookieId = runtime.getMethodName() + ":3181"; + + // read the cookie doesn't exist + try { + regMgr.readCookie(bookieId); + fail("Should fail reading cookie if cookie doesn't exist"); + } catch (CookieNotFoundException cnfe) { + // expected + } + + // create the cookie + String cookieData = RandomStringUtils.randomAlphanumeric(1024); + Versioned cookie = new Versioned<>( + cookieData.getBytes(UTF_8), Version.NEW + ); + regMgr.writeCookie(bookieId, cookie); + + // read the cookie + Versioned readCookie = regMgr.readCookie(bookieId); + assertEquals(cookieData, new String(readCookie.getValue(), UTF_8)); + + // attempt to create the cookie again + String newCookieData = RandomStringUtils.randomAlphabetic(512); + Versioned newCookie = new Versioned<>( + newCookieData.getBytes(UTF_8), Version.NEW + ); + try { + regMgr.writeCookie(bookieId, newCookie); + fail("Should fail creating cookie if the cookie already exists"); + } catch (MetadataStoreException mse) { + assertTrue(mse.getMessage().contains("Conflict on writing cookie")); + } + Versioned readCookie2 = regMgr.readCookie(bookieId); + assertCookieEquals(readCookie, readCookie2); + + // attempt to update the cookie with a wrong version + newCookie = new Versioned<>( + newCookieData.getBytes(UTF_8), new LongVersion(Long.MAX_VALUE) + ); + try { + regMgr.writeCookie(bookieId, newCookie); + } catch (MetadataStoreException mse) { + assertTrue(mse.getMessage().contains("Conflict on writing cookie")); + } + readCookie2 = regMgr.readCookie(bookieId); + assertCookieEquals(readCookie, readCookie2); + + // delete the cookie with a wrong version + LongVersion badVersion = new LongVersion(Long.MAX_VALUE); + try { + regMgr.removeCookie(bookieId, badVersion); + fail("Should fail to remove cookie with bad version"); + } catch (MetadataStoreException mse) { + assertTrue(mse.getMessage().contains( + "bad version '" + badVersion + "'" + )); + } + readCookie2 = regMgr.readCookie(bookieId); + assertCookieEquals(readCookie, readCookie2); + + // update the cookie with right version + newCookie = new Versioned<>( + newCookieData.getBytes(UTF_8), readCookie2.getVersion()); + regMgr.writeCookie(bookieId, newCookie); + readCookie2 = regMgr.readCookie(bookieId); + assertEquals(newCookieData, new String(readCookie2.getValue(), UTF_8)); + assertEquals(Occurred.AFTER, readCookie2.getVersion().compare(readCookie.getVersion())); + + // delete the cookie with right version + regMgr.removeCookie(bookieId, readCookie2.getVersion()); + try { + regMgr.readCookie(bookieId); + fail("Should fail reading cookie if cookie doesn't exist"); + } catch (CookieNotFoundException cnfe) { + // expected + } + + // remove a cookie that doesn't exist + try { + regMgr.removeCookie(bookieId, readCookie2.getVersion()); + fail("Should fail removing cookie if cookie doesn't exist"); + } catch (CookieNotFoundException cnfe) { + // expected + } + } + +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLayoutManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLayoutManagerTest.java new file mode 100644 index 00000000000..a0a1f696c49 --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLayoutManagerTest.java @@ -0,0 +1,104 @@ +/* + * 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.metadata.etcd; + +import static org.apache.bookkeeper.metadata.etcd.EtcdConstants.LAYOUT_NODE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +import java.io.IOException; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.meta.LayoutManager.LedgerLayoutExistsException; +import org.apache.bookkeeper.meta.LedgerLayout; +import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +import org.apache.commons.lang.RandomStringUtils; +import org.junit.Before; +import org.junit.Test; + +/** + * Integration test {@link EtcdLayoutManager}. + */ +@Slf4j +public class EtcdLayoutManagerTest extends EtcdTestBase { + + private static final int managerVersion = 0xabcd; + + private String scope; + private EtcdLayoutManager layoutManager; + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + this.scope = "/" + RandomStringUtils.randomAlphabetic(8); + this.layoutManager = new EtcdLayoutManager(etcdClient, scope); + log.info("setup layout manager under scope {}", scope); + } + + @Test + public void testReadCreateDeleteLayout() throws Exception { + // layout doesn't exist + assertNull(layoutManager.readLedgerLayout()); + + // create the layout + LedgerLayout layout = new LedgerLayout( + EtcdLedgerManagerFactory.class.getName(), + managerVersion + ); + layoutManager.storeLedgerLayout(layout); + + // read the layout + LedgerLayout readLayout = layoutManager.readLedgerLayout(); + assertEquals(layout, readLayout); + + // attempts to create the layout again and it should fail + LedgerLayout newLayout = new LedgerLayout( + "new layout", + managerVersion + 1 + ); + try { + layoutManager.storeLedgerLayout(newLayout); + fail("Should fail storeLedgerLayout if layout already exists"); + } catch (LedgerLayoutExistsException e) { + // expected + } + + // read the layout again (layout should not be changed) + readLayout = layoutManager.readLedgerLayout(); + assertEquals(layout, readLayout); + + // delete the layout + layoutManager.deleteLedgerLayout(); + + // the layout should be gone now + assertNull(layoutManager.readLedgerLayout()); + + // delete the layout again. it should fail since layout doesn't exist + try { + layoutManager.deleteLedgerLayout(); + fail("Should fail deleteLedgerLayout is layout not found"); + } catch (IOException ioe) { + assertEquals( + "No ledger layout is found under '" + scope + "/" + LAYOUT_NODE + "'", + ioe.getMessage()); + } + } + +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java new file mode 100644 index 00000000000..226fd48a830 --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -0,0 +1,412 @@ +/* + * Licensed 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. + */ + +/* + * Licensed 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.metadata.etcd; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BKException.Code; +import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; +import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; +import org.apache.bookkeeper.metadata.etcd.helpers.ValueStream; +import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Version; +import org.apache.commons.lang.RandomStringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * Integration test {@link EtcdLedgerManager}. + */ +@Slf4j +public class EtcdLedgerManagerTest extends EtcdTestBase { + + private String scope; + private EtcdLedgerManager lm; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + this.scope = RandomStringUtils.randomAlphabetic(8); + this.lm = new EtcdLedgerManager(etcdClient, scope); + } + + @Override + @After + public void tearDown() throws Exception { + if (null != lm) { + lm.close(); + } + super.tearDown(); + } + + @Test + public void testLedgerCRUD() throws Exception { + long ledgerId = System.currentTimeMillis(); + LedgerMetadata metadata = new LedgerMetadata( + 3, 3, 2, + DigestType.CRC32C, + "test-password".getBytes(UTF_8) + ); + + // ledger doesn't exist: read + + GenericCallbackFuture readFuture = new GenericCallbackFuture<>(); + lm.readLedgerMetadata(ledgerId, readFuture); + try { + result(readFuture); + fail("Should fail on reading ledger metadata if the ledger doesn't exist"); + } catch (BKException bke) { + assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); + } + + // ledger doesn't exist : delete + + GenericCallbackFuture deleteFuture = new GenericCallbackFuture<>(); + lm.removeLedgerMetadata(ledgerId, new LongVersion(999L), deleteFuture); + try { + result(deleteFuture); + fail("Should fail on deleting ledger metadata if the ledger doesn't exist"); + } catch (BKException bke) { + assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); + } + + // ledger doesn't exist : write + + GenericCallbackFuture writeFuture = new GenericCallbackFuture<>(); + metadata.setVersion(new LongVersion(999L)); + lm.writeLedgerMetadata(ledgerId, metadata, writeFuture); + try { + result(deleteFuture); + fail("Should fail on updating ledger metadata if the ledger doesn't exist"); + } catch (BKException bke) { + assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); + } + + // ledger doesn't exist : create + + GenericCallbackFuture createFuture = new GenericCallbackFuture<>(); + lm.createLedgerMetadata(ledgerId, metadata, createFuture); + LedgerMetadata writtenMetadata = result(createFuture); + assertSame(metadata, writtenMetadata); + Version version = writtenMetadata.getVersion(); + assertNotNull(version); + assertTrue(version instanceof LongVersion); + assertTrue(((LongVersion) version).getLongVersion() > 0L); + + // ledger exists : create + + // attempt to create the ledger again will result in exception `LedgerExistsException` + createFuture = new GenericCallbackFuture<>(); + try { + lm.createLedgerMetadata(ledgerId, metadata, createFuture); + result(createFuture); + fail("Should fail on creating ledger metadata if the ledger already exists"); + } catch (BKException bke) { + assertEquals(Code.LedgerExistException, bke.getCode()); + } + + // ledger exists: get + + readFuture = new GenericCallbackFuture<>(); + lm.readLedgerMetadata(ledgerId, readFuture); + LedgerMetadata readMetadata = result(readFuture); + assertEquals(metadata, readMetadata); + + // ledger exists: update metadata with wrong version + + readMetadata.setVersion(new LongVersion(Long.MAX_VALUE)); + writeFuture = new GenericCallbackFuture<>(); + lm.writeLedgerMetadata(ledgerId, readMetadata, writeFuture); + try { + result(writeFuture); + fail("Should fail to write metadata using a wrong version"); + } catch (BKException bke) { + assertEquals(Code.MetadataVersionException, bke.getCode()); + } + readFuture = new GenericCallbackFuture<>(); + lm.readLedgerMetadata(ledgerId, readFuture); + readMetadata = result(readFuture); + assertEquals(metadata, readMetadata); + + // ledger exists: delete metadata with wrong version + + deleteFuture = new GenericCallbackFuture<>(); + lm.removeLedgerMetadata(ledgerId, new LongVersion(Long.MAX_VALUE), deleteFuture); + try { + result(deleteFuture); + fail("Should fail to delete metadata using a wrong version"); + } catch (BKException bke) { + assertEquals(Code.MetadataVersionException, bke.getCode()); + } + readFuture = new GenericCallbackFuture<>(); + lm.readLedgerMetadata(ledgerId, readFuture); + readMetadata = result(readFuture); + assertEquals(metadata, readMetadata); + + // ledger exists: update metadata with the right version + + LongVersion curVersion = (LongVersion) readMetadata.getVersion(); + writeFuture = new GenericCallbackFuture<>(); + lm.writeLedgerMetadata(ledgerId, readMetadata, writeFuture); + writtenMetadata = result(writeFuture); + LongVersion newVersion = (LongVersion) writtenMetadata.getVersion(); + assertTrue(curVersion.getLongVersion() < newVersion.getLongVersion()); + readFuture = new GenericCallbackFuture<>(); + lm.readLedgerMetadata(ledgerId, readFuture); + readMetadata = result(readFuture); + assertEquals(writtenMetadata, readMetadata); + + // ledger exists: delete metadata with the right version + + deleteFuture = new GenericCallbackFuture<>(); + lm.removeLedgerMetadata(ledgerId, newVersion, deleteFuture); + result(deleteFuture); + readFuture = new GenericCallbackFuture<>(); + try { + lm.readLedgerMetadata(ledgerId, readFuture); + result(readFuture); + fail("Should fail to read ledger if it is deleted"); + } catch (BKException bke) { + assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); + } + + } + + @Test + public void testProcessLedgers() throws Exception { + final int numLedgers = 100; + createNumLedgers(numLedgers); + + final CountDownLatch processLatch = new CountDownLatch(numLedgers); + final CompletableFuture doneFuture = new CompletableFuture<>(); + lm.asyncProcessLedgers( + (l, cb) -> processLatch.countDown(), + (rc, path, ctx) -> { + if (Code.OK == rc) { + FutureUtils.complete(doneFuture, null); + } else { + FutureUtils.completeExceptionally(doneFuture, BKException.create(rc)); + } + }, + null, + Code.OK, + Code.MetaStoreException); + + result(doneFuture); + processLatch.await(); + } + + @Test + public void testLedgerRangeIterator() throws Exception { + final int numLedgers = 100; + createNumLedgers(numLedgers); + + long nextLedgerId = 0L; + LedgerRangeIterator iter = lm.getLedgerRanges(); + while (iter.hasNext()) { + LedgerRange lr = iter.next(); + for (Long lid : lr.getLedgers()) { + assertEquals(nextLedgerId, lid.longValue()); + ++nextLedgerId; + } + } + assertEquals((long) numLedgers, nextLedgerId); + } + + private void createNumLedgers(int numLedgers) throws Exception { + List> createFutures = new ArrayList<>(numLedgers); + for (int i = 0; i < numLedgers; i++) { + GenericCallbackFuture createFuture = new GenericCallbackFuture<>(); + createFutures.add(createFuture); + LedgerMetadata metadata = new LedgerMetadata( + 3, 3, 2, + DigestType.CRC32C, + "test-password".getBytes(UTF_8) + ); + lm.createLedgerMetadata(i, metadata, createFuture); + } + FutureUtils.result(FutureUtils.collect(createFutures)); + } + + @Test + public void testRegisterLedgerMetadataListener() throws Exception { + long ledgerId = System.currentTimeMillis(); + + // create a ledger metadata + LedgerMetadata metadata = new LedgerMetadata( + 3, 3, 2, + DigestType.CRC32C, + "test-password".getBytes(UTF_8) + ); + metadata.addEnsemble(0L, createNumBookies(3)); + GenericCallbackFuture createFuture = new GenericCallbackFuture<>(); + lm.createLedgerMetadata(ledgerId, metadata, createFuture); + result(createFuture); + LedgerMetadata readMetadata = readLedgerMetadata(ledgerId); + log.info("Create ledger metadata : {}", readMetadata); + + // register first listener + + LinkedBlockingQueue metadataQueue1 = new LinkedBlockingQueue<>(); + LedgerMetadataListener listener1 = (lid, m) -> { + log.info("[listener1] Received ledger {} metadata : {}", lid, m); + metadataQueue1.add(m); + }; + log.info("Registered first listener for ledger {}", ledgerId); + lm.registerLedgerMetadataListener(ledgerId, listener1); + // we should receive a metadata notification when a ledger is created + LedgerMetadata notifiedMetadata = metadataQueue1.take(); + assertEquals(readMetadata, notifiedMetadata); + ValueStream lms = lm.getLedgerMetadataStream(ledgerId); + assertNotNull(lms.waitUntilWatched()); + assertNotNull(result(lms.waitUntilWatched())); + + // register second listener + + LinkedBlockingQueue metadataQueue2 = new LinkedBlockingQueue<>(); + LedgerMetadataListener listener2 = (lid, m) -> { + log.info("[listener2] Received ledger {} metadata : {}", lid, m); + metadataQueue2.add(m); + }; + log.info("Registered second listener for ledger {}", ledgerId); + lm.registerLedgerMetadataListener(ledgerId, listener2); + LedgerMetadata notifiedMetadata2 = metadataQueue2.take(); + assertEquals(readMetadata, notifiedMetadata2); + assertNotNull(lm.getLedgerMetadataStream(ledgerId)); + + // update the metadata + metadata.setVersion(notifiedMetadata.getVersion()); + metadata.addEnsemble(10L, createNumBookies(3)); + writeLedgerMetadata(ledgerId, metadata); + readMetadata = readLedgerMetadata(ledgerId); + assertEquals(readMetadata, metadataQueue1.take()); + assertEquals(readMetadata, metadataQueue2.take()); + lms = lm.getLedgerMetadataStream(ledgerId); + assertNotNull(lms); + assertEquals(2, lms.getNumConsumers()); + + // remove listener2 + lm.unregisterLedgerMetadataListener(ledgerId, listener2); + lms = lm.getLedgerMetadataStream(ledgerId); + assertNotNull(lms); + assertEquals(1, lms.getNumConsumers()); + + // update the metadata again + metadata.setVersion(readMetadata.getVersion()); + metadata.addEnsemble(20L, createNumBookies(3)); + writeLedgerMetadata(ledgerId, metadata); + readMetadata = readLedgerMetadata(ledgerId); + assertEquals(readMetadata, metadataQueue1.take()); + assertNull(metadataQueue2.poll()); + + // remove listener1 + lm.unregisterLedgerMetadataListener(ledgerId, listener1); + // the value stream will be removed + while (lm.getLedgerMetadataStream(ledgerId) != null) { + TimeUnit.MILLISECONDS.sleep(100); + } + assertEquals(0, lms.getNumConsumers()); + + // update the metadata again + metadata.setVersion(readMetadata.getVersion()); + metadata.addEnsemble(30L, createNumBookies(3)); + writeLedgerMetadata(ledgerId, metadata); + readMetadata = readLedgerMetadata(ledgerId); + assertNull(metadataQueue1.poll()); + assertNull(metadataQueue2.poll()); + + log.info("Registered first listener for ledger {} again", ledgerId); + lm.registerLedgerMetadataListener(ledgerId, listener1); + notifiedMetadata = metadataQueue1.take(); + assertEquals(readMetadata, notifiedMetadata); + lms = lm.getLedgerMetadataStream(ledgerId); + assertNotNull(lms); + assertEquals(1, lms.getNumConsumers()); + + // delete the ledger + removeLedgerMetadata(ledgerId, readMetadata.getVersion()); + // the listener will eventually be removed + while (lm.getLedgerMetadataStream(ledgerId) != null) { + TimeUnit.MILLISECONDS.sleep(100); + } + assertEquals(1, lms.getNumConsumers()); + assertNull(metadataQueue1.poll()); + assertNull(metadataQueue2.poll()); + } + + LedgerMetadata readLedgerMetadata(long lid) throws Exception { + GenericCallbackFuture readFuture = new GenericCallbackFuture<>(); + lm.readLedgerMetadata(lid, readFuture); + return result(readFuture); + } + + void writeLedgerMetadata(long lid, LedgerMetadata metadata) throws Exception { + GenericCallbackFuture writeFuture = new GenericCallbackFuture<>(); + lm.writeLedgerMetadata(lid, metadata, writeFuture); + result(writeFuture); + } + + void removeLedgerMetadata(long lid, Version version) throws Exception { + GenericCallbackFuture deleteFuture = new GenericCallbackFuture<>(); + lm.removeLedgerMetadata(lid, version, deleteFuture); + result(deleteFuture); + } + + static List createNumBookies(int numBookies) { + return IntStream.range(0, numBookies) + .mapToObj(idx -> new BookieSocketAddress("127.0.0.1", 3181 + idx)) + .collect(Collectors.toList()); + } +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java new file mode 100644 index 00000000000..f36a496c215 --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java @@ -0,0 +1,416 @@ +/* + * 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.metadata.etcd; + +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.coreos.jetcd.Client; +import com.google.common.collect.Sets; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.discover.RegistrationClient; +import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; +import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +import org.apache.bookkeeper.net.BookieSocketAddress; +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.commons.lang.RandomStringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; + +/** + * Test etcd based bookie registration. + */ +@Slf4j +public class EtcdRegistrationTest extends EtcdTestBase { + + static String newBookie(int i) { + return "127.0.0.1:" + (3181 + i); + } + + @Rule + public final TestName runtime = new TestName(); + + private String scope; + private RegistrationClient regClient; + + protected static RegistrationListener newRegistrationListener( + LinkedBlockingQueue>> notifications) { + return bookies -> { + log.info("Received new bookies: {}", bookies); + try { + notifications.put(bookies); + } catch (InterruptedException e) { + log.error("Interrupted at enqueuing updated key set", e); + } + }; + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + this.scope = RandomStringUtils.randomAlphabetic(16); + this.regClient = new EtcdRegistrationClient(scope, etcdClient); + } + + @After + @Override + public void tearDown() throws Exception { + this.regClient.close(); + super.tearDown(); + } + + interface MultiBookiesTester { + + void test(String scope, int numBookies, boolean readonly) throws Exception; + + } + + private static void runNumBookiesTest(final String scope, + final int numBookies, + final boolean readonly, + MultiBookiesTester tester) throws Exception { + + final List bookies = createNumBookies(readonly, numBookies, scope); + try { + tester.test(scope, numBookies, readonly); + } finally { + bookies.forEach(EtcdRegistrationManager::close); + } + + } + + @Test + public void testRegisterWritableBookies() throws Exception { + testRegisterBookie(false); + } + + @Test + public void testRegisterReadonlyBookies() throws Exception { + testRegisterBookie(true); + } + + private void testRegisterBookie(boolean readonly) throws Exception { + runNumBookiesTest(scope, 3, readonly, (scope, numBookies, ro) -> { + Set expectedBookies = Sets.newHashSet(); + for (int i = 0; i < numBookies; i++) { + expectedBookies.add(new BookieSocketAddress(newBookie(i))); + } + Set writableBookies = result(regClient.getWritableBookies()).getValue(); + Set readonlyBookies = result(regClient.getReadOnlyBookies()).getValue(); + if (ro) { + assertEquals(0, writableBookies.size()); + assertEquals(numBookies, readonlyBookies.size()); + assertEquals(expectedBookies, readonlyBookies); + } else { + assertEquals(0, readonlyBookies.size()); + assertEquals(numBookies, writableBookies.size()); + assertEquals(expectedBookies, writableBookies); + } + + }); + } + + @Test + public void testWatchWritableBookies() throws Exception { + testWatchBookies(false); + } + + @Test + public void testWatchReadonlyBookies() throws Exception { + testWatchBookies(true); + } + + private void testWatchBookies(boolean readonly) throws Exception { + LinkedBlockingQueue>> writableChanges = new LinkedBlockingQueue<>(); + LinkedBlockingQueue>> readonlyChanges = new LinkedBlockingQueue<>(); + result(regClient.watchReadOnlyBookies(newRegistrationListener(readonlyChanges))); + result(regClient.watchWritableBookies(newRegistrationListener(writableChanges))); + Versioned> versionedBookies = writableChanges.take(); + assertTrue(versionedBookies.getValue().isEmpty()); + versionedBookies = readonlyChanges.take(); + assertTrue(versionedBookies.getValue().isEmpty()); + + final int numBookies = 3; + final List bookies = createNumBookies(readonly, numBookies, scope, 1); + + LinkedBlockingQueue>> changes; + if (readonly) { + changes = readonlyChanges; + } else { + changes = writableChanges; + } + + Version preVersion = new LongVersion(-1); + Set expectedBookies = new HashSet<>(); + for (int i = 0; i < numBookies; i++) { + BookieSocketAddress address = new BookieSocketAddress(newBookie(i)); + expectedBookies.add(address); + + versionedBookies = changes.take(); + Version curVersion = versionedBookies.getVersion(); + assertEquals(Occurred.AFTER, curVersion.compare(preVersion)); + assertEquals(expectedBookies, versionedBookies.getValue()); + preVersion = curVersion; + } + + bookies.forEach(EtcdRegistrationManager::close); + for (int i = 0; i < numBookies; i++) { + versionedBookies = changes.take(); + Version curVersion = versionedBookies.getVersion(); + assertEquals(Occurred.AFTER, curVersion.compare(preVersion)); + assertEquals(numBookies - i - 1, versionedBookies.getValue().size()); + preVersion = curVersion; + } + if (readonly) { + assertEquals(0, writableChanges.size()); + } else { + assertEquals(0, readonlyChanges.size()); + } + } + + private static List createNumBookies(boolean readonly, + int numBookies, + String scope, + long ttlSeconds) throws BookieException { + List bookies = new ArrayList<>(numBookies); + for (int i = 0; i < numBookies; i++) { + Client client = newEtcdClient(); + EtcdRegistrationManager regMgr = new EtcdRegistrationManager(client, scope, ttlSeconds); + bookies.add(regMgr); + regMgr.registerBookie(newBookie(i), readonly); + } + return bookies; + } + + private static List createNumBookies(boolean readonly, + int numBookies, + String scope) throws BookieException { + return createNumBookies(readonly, numBookies, scope, 60); + } + + @Test + public void testRegisterBookieWaitUntilPreviousExpiredSuccess() throws Exception { + long ttlSeconds = 1; + long leaseId = -0xabcd; + String bookieId = runtime.getMethodName() + ":3181"; + try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( + newEtcdClient(), scope, ttlSeconds) + ) { + regManager.registerBookie(bookieId, false); + leaseId = regManager.getBkRegister().getLeaseId(); + log.info("Registered bookie under scope '{}' with lease = {}", scope, leaseId); + } + assertNotEquals(-0xabcd, leaseId); + final long prevLeaseId = leaseId; + try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( + newEtcdClient(), scope, 100000 * ttlSeconds) + ) { + regManager.registerBookie(bookieId, false); + leaseId = regManager.getBkRegister().getLeaseId(); + log.info("Registered bookie under scope '{}' with new lease = {}", scope, leaseId); + } + assertNotEquals(prevLeaseId, leaseId); + } + + @Test + public void testRegisterBookieWaitUntilPreviousExpiredFailure() throws Exception { + long ttlSeconds = 1; + long leaseId = -0xabcd; + String bookieId = runtime.getMethodName() + ":3181"; + try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( + newEtcdClient(), scope, 10000000 * ttlSeconds) + ) { + regManager.registerBookie(bookieId, false); + leaseId = regManager.getBkRegister().getLeaseId(); + log.info("Registered bookie under scope '{}' with lease = {}", scope, leaseId); + } + assertNotEquals(-0xabcd, leaseId); + try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( + newEtcdClient(), scope, ttlSeconds) + ) { + regManager.registerBookie(bookieId, false); + fail("Should fail to register bookie under scope '{}'" + + " since previous registration has not been expired yet"); + } catch (MetadataStoreException mse) { + log.info("Encountered exception on registering bookie under scope '{}'", scope, mse); + // expected + } + } + + @Test + public void testRegisterWritableBookieWithSameLeaseId() throws Exception { + testRegisterBookieWithSameLeaseId(false); + } + + @Test + public void testRegisterReadonlyBookieWithSameLeaseId() throws Exception { + testRegisterBookieWithSameLeaseId(true); + } + + private void testRegisterBookieWithSameLeaseId(boolean readonly) throws Exception { + long ttlSeconds = 1; + long leaseId = -0xabcd; + String bookieId = runtime.getMethodName() + ":3181"; + try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( + newEtcdClient(), scope, 10000000 * ttlSeconds) + ) { + regManager.registerBookie(bookieId, readonly); + leaseId = regManager.getBkRegister().getLeaseId(); + log.info("Registered bookie under scope '{}' with lease = {}", scope, leaseId); + log.info("Trying to register using same lease '{}'", leaseId); + try (EtcdRegistrationManager regManager2 = new EtcdRegistrationManager( + regManager.getClient(), scope, regManager.getBkRegister() + )) { + regManager.registerBookie(bookieId, readonly); + } + } + } + + private Set getBookies(boolean readonly) throws Exception { + Set bookies; + if (readonly) { + bookies = result(regClient.getReadOnlyBookies()).getValue(); + } else { + bookies = result(regClient.getWritableBookies()).getValue(); + } + return bookies; + } + + @Test + public void testRegisterUnregisterWritableBookie() throws Exception { + testRegisterUnregister(false); + } + + @Test + public void testRegisterUnregisterReadonlyBookie() throws Exception { + testRegisterUnregister(true); + } + + private void testRegisterUnregister(boolean readonly) throws Exception { + String bookieId = runtime.getMethodName(); + if (readonly) { + bookieId += "-readonly"; + } + bookieId += ":3181"; + try (EtcdRegistrationManager regMgr = new EtcdRegistrationManager( + newEtcdClient(), scope, 1000000000 + )) { + // before registration + Set bookies = getBookies(readonly); + log.info("before registration : bookies = {}", bookies); + assertEquals(0, bookies.size()); + // registered + regMgr.registerBookie(bookieId, readonly); + bookies = getBookies(readonly); + log.info("after registered: bookies = {}", bookies); + assertEquals(1, bookies.size()); + assertEquals( + Sets.newHashSet(new BookieSocketAddress(bookieId)), + bookies); + // unregistered + regMgr.unregisterBookie(bookieId, readonly); + bookies = getBookies(readonly); + log.info("after unregistered: bookies = {}", bookies); + assertEquals(0, bookies.size()); + } + } + + @Test + public void testConcurrentWritableRegistration() throws Exception { + testConcurrentRegistration(false); + } + + @Test + public void testConcurrentReadonlyRegistration() throws Exception { + testConcurrentRegistration(true); + } + + private void testConcurrentRegistration(boolean readonly) throws Exception { + final String bookieId; + if (readonly) { + bookieId = runtime.getMethodName() + "-readonly:3181"; + } else { + bookieId = runtime.getMethodName() + ":3181"; + } + final int numBookies = 10; + @Cleanup("shutdown") + ExecutorService executor = Executors.newFixedThreadPool(numBookies); + final CyclicBarrier startBarrier = new CyclicBarrier(numBookies); + final CyclicBarrier completeBarrier = new CyclicBarrier(numBookies); + final CompletableFuture doneFuture = new CompletableFuture<>(); + final AtomicInteger numSuccesses = new AtomicInteger(0); + final AtomicInteger numFailures = new AtomicInteger(0); + for (int i = 0; i < numBookies; i++) { + executor.submit(() -> { + try (EtcdRegistrationManager regMgr = new EtcdRegistrationManager( + newEtcdClient(), scope, 1 + )) { + try { + startBarrier.await(); + regMgr.registerBookie(bookieId, readonly); + numSuccesses.incrementAndGet(); + } catch (InterruptedException e) { + log.warn("Interrupted at waiting for the other threads to start", e); + } catch (BrokenBarrierException e) { + log.warn("Start barrier is broken", e); + } catch (BookieException e) { + numFailures.incrementAndGet(); + } + try { + completeBarrier.await(); + } catch (InterruptedException e) { + log.warn("Interrupted at waiting for the other threads to complete", e); + } catch (BrokenBarrierException e) { + log.warn("Complete barrier is broken", e); + } + FutureUtils.complete(doneFuture, null); + } + }); + } + doneFuture.join(); + assertEquals(1, numSuccesses.get()); + assertEquals(numBookies - 1, numFailures.get()); + } + +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/helpers/HelpersTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/helpers/HelpersTest.java new file mode 100644 index 00000000000..ecd295db2f2 --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/helpers/HelpersTest.java @@ -0,0 +1,210 @@ +/* + * 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.metadata.etcd.helpers; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.coreos.jetcd.data.ByteSequence; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +import org.apache.commons.lang.RandomStringUtils; +import org.apache.commons.lang3.StringUtils; +import org.junit.Before; +import org.junit.Test; + +/** + * Integration test helpers. + */ +@Slf4j +public class HelpersTest extends EtcdTestBase { + + private static final Function BYTE_SEQUENCE_STRING_FUNCTION = bs -> bs.toStringUtf8(); + + private static String getKey(String scope, int i) { + return String.format("%s-key-%010d", scope, i); + } + + private String scope; + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + scope = RandomStringUtils.randomAlphabetic(8); + } + + @Test + public void testEmptyKeyStream() throws Exception { + KeyStream ks = new KeyStream<>( + etcdClient.getKVClient(), + ByteSequence.fromString(getKey(scope, 0)), + ByteSequence.fromString(getKey(scope, 100)), + BYTE_SEQUENCE_STRING_FUNCTION + ); + List values = result(ks.readNext()); + assertTrue(values.isEmpty()); + + // read the values again + values = result(ks.readNext()); + assertTrue(values.isEmpty()); + } + + @Test + public void testKeyStreamBatch1() throws Exception { + testKeyStream(20, 1); + } + + @Test + public void testKeyStreamBatch2() throws Exception { + testKeyStream(20, 2); + } + + @Test + public void testKeyStreamBatch7() throws Exception { + testKeyStream(20, 7); + } + + @Test + public void testKeyStreamBatch10() throws Exception { + testKeyStream(20, 10); + } + + @Test + public void testKeyStreamBatch20() throws Exception { + testKeyStream(20, 20); + } + + @Test + public void testKeyStreamBatch40() throws Exception { + testKeyStream(20, 40); + } + + @Test + public void testKeyStreamBatchUnlimited() throws Exception { + testKeyStream(20, 0); + } + + private void testKeyStream(int numKeys, int batchSize) throws Exception { + for (int i = 0; i < numKeys; i++) { + String key = getKey(scope, i); + ByteSequence keyBs = ByteSequence.fromBytes(key.getBytes(UTF_8)); + result(etcdClient.getKVClient().put(keyBs, keyBs)); + } + + KeyStream ks = openKeyStream(batchSize); + AtomicInteger numReceived = new AtomicInteger(0); + while (true) { + List values = result(ks.readNext()); + log.info("Received values : {}", values); + if (values.isEmpty()) { + break; + } + for (int value : values) { + assertEquals(numReceived.getAndIncrement(), value); + } + } + assertEquals(numKeys, numReceived.get()); + } + + private void testKeyIterator(int numKeys, int batchSize) throws Exception { + for (int i = 0; i < numKeys; i++) { + String key = getKey(scope, i); + ByteSequence keyBs = ByteSequence.fromString(key); + result(etcdClient.getKVClient().put(keyBs, keyBs)); + } + + KeyStream ks = openKeyStream(batchSize); + KeyIterator ki = new KeyIterator<>(ks); + + AtomicInteger numReceived = new AtomicInteger(0); + while (ki.hasNext()) { + List values = ki.next(); + log.info("Received values : {}", values); + if (values.isEmpty()) { + break; + } + for (int value : values) { + assertEquals(numReceived.getAndIncrement(), value); + } + } + assertEquals(numKeys, numReceived.get()); + } + + @Test + public void testKeyIteratorBatch1() throws Exception { + testKeyIterator(20, 1); + } + + @Test + public void testKeyIteratorBatch2() throws Exception { + testKeyIterator(20, 2); + } + + @Test + public void testKeyIteratorBatch7() throws Exception { + testKeyIterator(20, 7); + } + + @Test + public void testKeyIteratorBatch10() throws Exception { + testKeyIterator(20, 10); + } + + @Test + public void testKeyIteratorBatch20() throws Exception { + testKeyIterator(20, 20); + } + + @Test + public void testKeyIteratorBatch40() throws Exception { + testKeyIterator(20, 40); + } + + @Test + public void testKeyIteratorBatchUnlimited() throws Exception { + testKeyIterator(20, 0); + } + + private KeyStream openKeyStream(int batchSize) { + KeyStream ks = new KeyStream<>( + etcdClient.getKVClient(), + ByteSequence.fromBytes(getKey(scope, 0).getBytes(UTF_8)), + ByteSequence.fromBytes(getKey(scope, Integer.MAX_VALUE).getBytes(UTF_8)), + bs -> { + String[] keyParts = StringUtils.split(bs.toString(UTF_8), '-'); + try { + return Integer.parseInt(keyParts[2]); + } catch (NumberFormatException nfe) { + log.error("Failed to parse key string '{}' : ", + bs.toStringUtf8(), nfe); + return -0xabcd; + } + }, + batchSize + ); + return ks; + } + +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/helpers/KeySetReaderTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/helpers/KeySetReaderTest.java new file mode 100644 index 00000000000..5bc52bd877b --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/helpers/KeySetReaderTest.java @@ -0,0 +1,414 @@ +/* + * 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.metadata.etcd.helpers; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import com.coreos.jetcd.Lease.KeepAliveListener; +import com.coreos.jetcd.data.ByteSequence; +import com.coreos.jetcd.options.PutOption; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.function.Consumer; +import java.util.function.Function; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Version.Occurred; +import org.apache.bookkeeper.versioning.Versioned; +import org.apache.commons.compress.utils.Sets; +import org.junit.Test; +import org.testcontainers.shaded.org.apache.commons.lang.RandomStringUtils; + +/** + * Integration test {@link KeySetReader}. + */ +@Slf4j +public class KeySetReaderTest extends EtcdTestBase { + + private static final Function BYTE_SEQUENCE_STRING_FUNCTION = bs -> bs.toStringUtf8(); + + @Test + public void testReadSingleKey() throws Exception { + String key = RandomStringUtils.randomAlphabetic(16); + ByteSequence keyBs = ByteSequence.fromString(key); + try (KeySetReader ksReader = new KeySetReader<>( + etcdClient, + BYTE_SEQUENCE_STRING_FUNCTION, + keyBs, + null + )) { + // key not exists + Versioned> versionedKeys = FutureUtils.result(ksReader.read()); + assertTrue( + "VersionedKeys : " + versionedKeys, + ((LongVersion) versionedKeys.getVersion()).getLongVersion() > 0L); + assertEquals(0, versionedKeys.getValue().size()); + assertFalse(ksReader.isWatcherSet()); + + // keys should be cached + assertEquals(versionedKeys, ksReader.getLocalValue()); + + // update a value + String value = RandomStringUtils.randomAlphabetic(32); + ByteSequence valueBs = ByteSequence.fromString(value); + FutureUtils.result(etcdClient.getKVClient().put(keyBs, valueBs)); + + // update the value should not change local value + assertEquals(versionedKeys, ksReader.getLocalValue()); + + // read the key again + Versioned> newVersionedKey = FutureUtils.result(ksReader.read()); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(1, newVersionedKey.getValue().size()); + assertEquals(Sets.newHashSet(key), newVersionedKey.getValue()); + + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + } + } + + @Test + public void testWatchSingleKey() throws Exception { + String key = RandomStringUtils.randomAlphabetic(16); + ByteSequence keyBs = ByteSequence.fromString(key); + KeySetReader ksReader = null; + try { + ksReader = new KeySetReader<>( + etcdClient, + BYTE_SEQUENCE_STRING_FUNCTION, + keyBs, + null + ); + LinkedBlockingQueue>> notifications = new LinkedBlockingQueue<>(); + Consumer>> keyConsumer = consumeVersionedKeySet(notifications); + + // key not exists + Versioned> versionedKeys = FutureUtils.result(ksReader.readAndWatch(keyConsumer)); + assertTrue( + "VersionedKeys : " + versionedKeys, + ((LongVersion) versionedKeys.getVersion()).getLongVersion() > 0L); + assertEquals(0, versionedKeys.getValue().size()); + assertTrue(ksReader.isWatcherSet()); + + // keys should be cached + assertEquals(versionedKeys, ksReader.getLocalValue()); + Versioned> newVersionedKey = notifications.take(); + assertEquals(Occurred.CONCURRENTLY, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(versionedKeys, newVersionedKey); + versionedKeys = newVersionedKey; + + // update a value + String value = RandomStringUtils.randomAlphabetic(32); + ByteSequence valueBs = ByteSequence.fromString(value); + FutureUtils.result(etcdClient.getKVClient().put(keyBs, valueBs)); + + // we should get notified with updated key set + newVersionedKey = notifications.take(); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(1, newVersionedKey.getValue().size()); + assertEquals(Sets.newHashSet(key), newVersionedKey.getValue()); + + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + versionedKeys = newVersionedKey; + + // delete the key + FutureUtils.result(etcdClient.getKVClient().delete(keyBs)); + newVersionedKey = notifications.take(); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(0, newVersionedKey.getValue().size()); + + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + } finally { + if (null != ksReader) { + ksReader.close(); + } + } + assertNotNull(ksReader); + assertFalse(ksReader.isWatcherSet()); + } + + @Test + public void testWatchSingleKeyWithTTL() throws Exception { + String key = RandomStringUtils.randomAlphabetic(16); + ByteSequence keyBs = ByteSequence.fromString(key); + KeySetReader ksReader = null; + try { + ksReader = new KeySetReader<>( + etcdClient, + BYTE_SEQUENCE_STRING_FUNCTION, + keyBs, + null + ); + LinkedBlockingQueue>> notifications = new LinkedBlockingQueue<>(); + Consumer>> keyConsumer = consumeVersionedKeySet(notifications); + + // key not exists + Versioned> versionedKeys = FutureUtils.result(ksReader.readAndWatch(keyConsumer)); + assertTrue( + "VersionedKeys : " + versionedKeys, + ((LongVersion) versionedKeys.getVersion()).getLongVersion() > 0L); + assertEquals(0, versionedKeys.getValue().size()); + assertTrue(ksReader.isWatcherSet()); + + // keys should be cached + assertEquals(versionedKeys, ksReader.getLocalValue()); + // no watch event should be issued + Versioned> newVersionedKey = notifications.take(); + assertEquals(Occurred.CONCURRENTLY, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(versionedKeys, newVersionedKey); + versionedKeys = newVersionedKey; + + // create a key with ttl + long leaseId = FutureUtils.result(etcdClient.getLeaseClient().grant(1)).getID(); + String value = RandomStringUtils.randomAlphabetic(32); + ByteSequence valueBs = ByteSequence.fromString(value); + FutureUtils.result(etcdClient.getKVClient() + .put(keyBs, valueBs, PutOption.newBuilder().withLeaseId(leaseId).build())); + + // we should get notified with updated key set + newVersionedKey = notifications.take(); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(1, newVersionedKey.getValue().size()); + assertEquals(Sets.newHashSet(key), newVersionedKey.getValue()); + + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + versionedKeys = newVersionedKey; + + // the key will be deleted after TTL + newVersionedKey = notifications.take(); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(0, newVersionedKey.getValue().size()); + + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + } finally { + if (null != ksReader) { + ksReader.close(); + } + } + assertNotNull(ksReader); + assertFalse(ksReader.isWatcherSet()); + } + + @Test + public void testReadKeySet() throws Exception { + String prefix = RandomStringUtils.randomAlphabetic(16); + ByteSequence beginKeyBs = ByteSequence.fromString(prefix + "-000"); + ByteSequence endKeyBs = ByteSequence.fromString(prefix + "-999"); + try (KeySetReader ksReader = new KeySetReader<>( + etcdClient, + BYTE_SEQUENCE_STRING_FUNCTION, + beginKeyBs, + endKeyBs + )) { + // key not exists + Versioned> versionedKeys = FutureUtils.result(ksReader.read()); + assertTrue( + "VersionedKeys : " + versionedKeys, + ((LongVersion) versionedKeys.getVersion()).getLongVersion() > 0L); + assertEquals(0, versionedKeys.getValue().size()); + assertFalse(ksReader.isWatcherSet()); + + // keys should be cached + assertEquals(versionedKeys, ksReader.getLocalValue()); + + Set expectedKeySet = new HashSet<>(); + for (int i = 0; i < 20; i++) { + // update a value + String key = String.format("%s-%03d", prefix, i); + String value = RandomStringUtils.randomAlphabetic(32); + ByteSequence keyBs = ByteSequence.fromString(key); + ByteSequence valueBs = ByteSequence.fromString(value); + expectedKeySet.add(key); + FutureUtils.result(etcdClient.getKVClient().put(keyBs, valueBs)); + + // update the value should not change local value + assertEquals(versionedKeys, ksReader.getLocalValue()); + + // read the key again + Versioned> newVersionedKey = FutureUtils.result(ksReader.read()); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(expectedKeySet, newVersionedKey.getValue()); + + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + versionedKeys = newVersionedKey; + } + } + } + + @Test + public void testWatchKeySet() throws Exception { + String prefix = RandomStringUtils.randomAlphabetic(16); + ByteSequence beginKeyBs = ByteSequence.fromString(prefix + "-000"); + ByteSequence endKeyBs = ByteSequence.fromString(prefix + "-999"); + KeySetReader ksReader = null; + try { + ksReader = new KeySetReader<>( + etcdClient, + BYTE_SEQUENCE_STRING_FUNCTION, + beginKeyBs, + endKeyBs + ); + LinkedBlockingQueue>> notifications = new LinkedBlockingQueue<>(); + Consumer>> keyConsumer = consumeVersionedKeySet(notifications); + + // key not exists + Versioned> versionedKeys = FutureUtils.result(ksReader.readAndWatch(keyConsumer)); + assertTrue( + "VersionedKeys : " + versionedKeys, + ((LongVersion) versionedKeys.getVersion()).getLongVersion() > 0L); + assertEquals(0, versionedKeys.getValue().size()); + assertTrue(ksReader.isWatcherSet()); + + // keys should be cached + assertEquals(versionedKeys, ksReader.getLocalValue()); + Versioned> newVersionedKey = notifications.take(); + assertEquals(Occurred.CONCURRENTLY, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(versionedKeys, newVersionedKey); + versionedKeys = newVersionedKey; + + Set expectedKeySet = new HashSet<>(); + for (int i = 0; i < 20; i++) { + // update a value + String key = String.format("%s-%03d", prefix, i); + String value = RandomStringUtils.randomAlphabetic(32); + ByteSequence keyBs = ByteSequence.fromString(key); + ByteSequence valueBs = ByteSequence.fromString(value); + expectedKeySet.add(key); + FutureUtils.result(etcdClient.getKVClient().put(keyBs, valueBs)); + + // we should get notified with updated key set + newVersionedKey = notifications.take(); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(expectedKeySet, newVersionedKey.getValue()); + + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + versionedKeys = newVersionedKey; + } + + for (int i = 0; i < 20; i++) { + // delete the key + String key = String.format("%s-%03d", prefix, i); + ByteSequence keyBs = ByteSequence.fromString(key); + expectedKeySet.remove(key); + FutureUtils.result(etcdClient.getKVClient().delete(keyBs)); + + // we should get notified with updated key set + newVersionedKey = notifications.take(); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(expectedKeySet, newVersionedKey.getValue()); + + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + versionedKeys = newVersionedKey; + } + } finally { + if (null != ksReader) { + ksReader.close(); + } + } + assertNotNull(ksReader); + assertFalse(ksReader.isWatcherSet()); + } + + @Test + public void testWatchKeySetWithTTL() throws Exception { + String prefix = RandomStringUtils.randomAlphabetic(16); + ByteSequence beginKeyBs = ByteSequence.fromString(prefix + "-000"); + ByteSequence endKeyBs = ByteSequence.fromString(prefix + "-999"); + KeySetReader ksReader = null; + try { + ksReader = new KeySetReader<>( + etcdClient, + BYTE_SEQUENCE_STRING_FUNCTION, + beginKeyBs, + endKeyBs + ); + LinkedBlockingQueue>> notifications = new LinkedBlockingQueue<>(); + Consumer>> keyConsumer = consumeVersionedKeySet(notifications); + + // key not exists + Versioned> versionedKeys = FutureUtils.result(ksReader.readAndWatch(keyConsumer)); + assertTrue( + "VersionedKeys : " + versionedKeys, + ((LongVersion) versionedKeys.getVersion()).getLongVersion() > 0L); + assertEquals(0, versionedKeys.getValue().size()); + assertTrue(ksReader.isWatcherSet()); + + // keys should be cached + assertEquals(versionedKeys, ksReader.getLocalValue()); + // no watch event should be issued + Versioned> newVersionedKey = notifications.take(); + assertEquals(Occurred.CONCURRENTLY, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(versionedKeys, newVersionedKey); + versionedKeys = newVersionedKey; + + // create keys with ttl + long leaseId = FutureUtils.result(etcdClient.getLeaseClient().grant(1)).getID(); + KeepAliveListener ka = etcdClient.getLeaseClient().keepAlive(leaseId); + + Set expectedKeySet = new HashSet<>(); + for (int i = 0; i < 20; i++) { + String key = String.format("%s-%03d", prefix, i); + String value = RandomStringUtils.randomAlphabetic(32); + ByteSequence keyBs = ByteSequence.fromString(key); + ByteSequence valueBs = ByteSequence.fromString(value); + expectedKeySet.add(key); + FutureUtils.result(etcdClient.getKVClient() + .put(keyBs, valueBs, PutOption.newBuilder().withLeaseId(leaseId).build())); + + // we should get notified with updated key set + newVersionedKey = notifications.take(); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertEquals(expectedKeySet, newVersionedKey.getValue()); + + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + versionedKeys = newVersionedKey; + } + + // stop keep alive all the keys should be expired. + ka.close(); + + // all the keys will be deleted after TTL in same batch. + newVersionedKey = notifications.take(); + // local value should be changed + assertEquals(newVersionedKey, ksReader.getLocalValue()); + assertEquals(Occurred.AFTER, newVersionedKey.getVersion().compare(versionedKeys.getVersion())); + assertTrue(newVersionedKey.getValue().isEmpty()); + } finally { + if (null != ksReader) { + ksReader.close(); + } + } + assertNotNull(ksReader); + assertFalse(ksReader.isWatcherSet()); + } +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/integration/SmokeTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/integration/SmokeTest.java new file mode 100644 index 00000000000..c29ad426628 --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/integration/SmokeTest.java @@ -0,0 +1,105 @@ +/* + * Licensed 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.metadata.etcd.integration; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.junit.Assert.assertEquals; + +import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.api.BookKeeper; +import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.client.api.WriteAdvHandle; +import org.apache.bookkeeper.client.api.WriteHandle; +import org.apache.bookkeeper.metadata.etcd.testing.EtcdBKClusterTestBase; +import org.junit.Test; + +/** + * Smoke testing etcd metadata drives. + */ +@Slf4j +public class SmokeTest extends EtcdBKClusterTestBase { + + private static final byte[] PASSWD = "smoketest".getBytes(UTF_8); + + private static void readEntries(BookKeeper bk, + long ledgerId, + int numExpectedEntries) throws Exception { + try (ReadHandle readlh = result(bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withDigestType(DigestType.CRC32C) + .withPassword(PASSWD) + .execute() + )) { + long lac = readlh.getLastAddConfirmed(); + AtomicInteger idx = new AtomicInteger(0); + try (LedgerEntries entries = readlh.read(0, lac)) { + entries.forEach(e -> assertEquals( + String.format("entry-%03d", idx.getAndIncrement()), + new String(e.getEntryBytes(), UTF_8))); + } + assertEquals(idx.get(), numExpectedEntries); + } + } + + @Test + public void testReadWrite() throws Exception { + int numEntries = 100; + try (BookKeeper bk = BookKeeper.newBuilder(conf).build()) { + long ledgerId; + try (WriteHandle wh = result(bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32C) + .withPassword(PASSWD) + .execute())) { + ledgerId = wh.getId(); + log.info("Successfully created ledger {} to append entries.", ledgerId); + for (int i = 0; i < numEntries; i++) { + wh.append(String.format("entry-%03d", i).getBytes(UTF_8)); + } + } + log.info("Opening ledger {} to read entries ...", ledgerId); + readEntries(bk, ledgerId, numEntries); + log.info("Successfully read {} entries from ledger {}", numEntries, ledgerId); + } + } + + @Test + public void testReadWriteAdv() throws Exception { + final int numEntries = 100; + try (BookKeeper bk = BookKeeper.newBuilder(conf).build()) { + long ledgerId; + try (WriteAdvHandle wah = result(bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32C) + .withPassword(PASSWD) + .makeAdv() + .execute())) { + ledgerId = wah.getId(); + log.info("Successfully created adv ledger {} to append entries.", ledgerId); + for (int i = 0; i < numEntries; i++) { + wah.write(i, String.format("entry-%03d", i).getBytes(UTF_8)); + } + } + log.info("Opening adv ledger {} to read entries ...", ledgerId); + readEntries(bk, ledgerId, numEntries); + log.info("Successfully read {} entries from adv ledger {}", numEntries, ledgerId); + } + } + + + +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java new file mode 100644 index 00000000000..ccc9a1e8174 --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java @@ -0,0 +1,166 @@ +/* + * Licensed 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.metadata.etcd.testing; + +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.api.BookKeeper; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.metadata.etcd.EtcdMetadataBookieDriver; +import org.apache.bookkeeper.metadata.etcd.EtcdMetadataClientDriver; +import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.test.PortManager; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.util.IOUtils; +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +/** + * A test base that run an Etcd based bookkeeper cluster. + */ +@Slf4j +public abstract class EtcdBKClusterTestBase extends EtcdTestBase { + + protected static ClientConfiguration baseClientConf; + protected static ServerConfiguration baseServerConf; + protected static final int NUM_BOOKIES = 3; + protected static final List BOOKIES = new ArrayList<>(NUM_BOOKIES); + protected static final List TMP_DIRS = new ArrayList<>(NUM_BOOKIES); + + protected static File createTempDir(String prefix, String suffix) throws IOException { + File dir = IOUtils.createTempDir(prefix, suffix); + TMP_DIRS.add(dir); + return dir; + } + + protected static ServerConfiguration newServerConfiguration() throws Exception { + File f = createTempDir("bookie", "test"); + int port = PortManager.nextFreePort(); + return newServerConfiguration(port, f, new File[] { f }); + } + + protected static ServerConfiguration newServerConfiguration(int port, File journalDir, File[] ledgerDirs) { + ServerConfiguration conf = new ServerConfiguration(baseServerConf); + conf.setBookiePort(port); + conf.setJournalDirName(journalDir.getPath()); + String[] ledgerDirNames = new String[ledgerDirs.length]; + for (int i = 0; i < ledgerDirs.length; i++) { + ledgerDirNames[i] = ledgerDirs[i].getPath(); + } + conf.setLedgerDirNames(ledgerDirNames); + conf.setEnableTaskExecutionStats(true); + return conf; + } + + @BeforeClass + public static void setupCluster() throws Exception { + setupCluster(NUM_BOOKIES); + } + protected static void setupCluster(int numBookies) throws Exception { + EtcdTestBase.setupCluster(); + + MetadataDrivers.registerBookieDriver( + "etcd", EtcdMetadataBookieDriver.class + ); + MetadataDrivers.registerClientDriver( + "etcd", EtcdMetadataClientDriver.class + ); + + log.info("Successfully started etcd at:" + + " internal service uri = {}, external service uri = {}", + etcdContainer.getInternalServiceUri(), etcdContainer.getExternalServiceUri()); + + ServiceURI uri = ServiceURI.create(etcdContainer.getExternalServiceUri()); + + baseClientConf = new ClientConfiguration() + .setMetadataServiceUri(uri.getUri().toString()); + baseServerConf = TestBKConfiguration.newServerConfiguration() + .setMetadataServiceUri(uri.getUri().toString()); + // format the cluster + assertTrue(BookKeeperAdmin.format(baseServerConf, false, true)); + // start bookies + startNumBookies(numBookies); + } + + private static void startNumBookies(int numBookies) throws Exception { + for (int i = 0; i < numBookies; i++) { + ServerConfiguration conf = newServerConfiguration(); + log.info("Starting new bookie on port : {}", conf.getBookiePort()); + BookieServer server = startBookie(conf); + synchronized (BOOKIES) { + BOOKIES.add(server); + } + } + } + private static BookieServer startBookie(ServerConfiguration conf) throws Exception { + conf.setAutoRecoveryDaemonEnabled(true); + TestStatsProvider provider = new TestStatsProvider(); + BookieServer server = new BookieServer(conf, provider.getStatsLogger("")); + server.start(); + return server; + } + + @AfterClass + public static void teardownCluster() throws Exception { + // stop bookies + stopBookies(); + // stop metadata store + EtcdTestBase.teardownCluster(); + log.info("Stopped the metadata store."); + // clean up temp dirs + for (File f : TMP_DIRS) { + FileUtils.deleteDirectory(f); + } + log.info("Clean up all the temp directories."); + } + + private static void stopBookies() { + synchronized (BOOKIES) { + BOOKIES.forEach(BookieServer::shutdown); + log.info("Stopped all the bookies."); + } + } + + protected ClientConfiguration conf; + protected BookKeeper bk; + + @Before + public void setUp() throws Exception { + conf = new ClientConfiguration() + .setMetadataServiceUri(etcdContainer.getExternalServiceUri()); + bk = BookKeeper.newBuilder(conf).build(); + } + + @After + public void tearDown() throws Exception { + if (null != bk) { + bk.close(); + } + } + +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdContainer.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdContainer.java new file mode 100644 index 00000000000..ccc3035bad1 --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdContainer.java @@ -0,0 +1,141 @@ +/* + * 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.metadata.etcd.testing; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.github.dockerjava.api.DockerClient; +import com.github.dockerjava.api.command.LogContainerCmd; +import com.github.dockerjava.api.model.Frame; +import com.github.dockerjava.core.command.LogContainerResultCallback; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import lombok.extern.slf4j.Slf4j; +import org.testcontainers.DockerClientFactory; +import org.testcontainers.containers.ContainerLaunchException; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.WaitingConsumer; +import org.testcontainers.containers.wait.strategy.WaitStrategy; +import org.testcontainers.utility.LogUtils; + +/** + * Etcd test container. + */ +@Slf4j +public class EtcdContainer extends GenericContainer { + + public static final String NAME = "etcd"; + public static final int CLIENT_PORT = 2379; + + private final String clusterName; + + public EtcdContainer(String clusterName) { + super("quay.io/coreos/etcd:v3.3"); + this.clusterName = clusterName; + } + + public String getExternalServiceUri() { + return "etcd://" + getContainerIpAddress() + ":" + getEtcdClientPort() + "/clusters/" + clusterName; + } + + public String getInternalServiceUri() { + return "etcd://" + NAME + ":" + CLIENT_PORT + "/clusters/" + clusterName; + } + + @Override + protected void configure() { + super.configure(); + + String[] command = new String[] { + "/usr/local/bin/etcd", + "--name", NAME + "0", + "--initial-advertise-peer-urls", "http://" + NAME + ":2380", + "--listen-peer-urls", "http://0.0.0.0:2380", + "--advertise-client-urls", "http://" + NAME + ":2379", + "--listen-client-urls", "http://0.0.0.0:2379", + "--initial-cluster", NAME + "0=http://" + NAME + ":2380" + }; + + this.withNetworkAliases(NAME) + .withExposedPorts(CLIENT_PORT) + .withCreateContainerCmdModifier(createContainerCmd -> { + createContainerCmd.withHostName(NAME); + createContainerCmd.withName(clusterName + "-" + NAME); + }) + .withCommand(command) + .withNetworkAliases(NAME) + .waitingFor(waitStrategy()); + tailContainerLog(); + } + + public void tailContainerLog() { + CompletableFuture.runAsync(() -> { + while (null == containerId) { + try { + TimeUnit.MILLISECONDS.sleep(100); + } catch (InterruptedException e) { + return; + } + } + + LogContainerCmd logContainerCmd = this.dockerClient.logContainerCmd(containerId); + logContainerCmd.withStdOut(true).withStdErr(true).withFollowStream(true); + logContainerCmd.exec(new LogContainerResultCallback() { + @Override + public void onNext(Frame item) { + log.info(new String(item.getPayload(), UTF_8)); + } + }); + }); + } + + public int getEtcdClientPort() { + return getMappedPort(CLIENT_PORT); + } + + public String getClientEndpoint() { + return String.format("http://%s:%d", getContainerIpAddress(), getEtcdClientPort()); + } + + private WaitStrategy waitStrategy() { + return new org.testcontainers.containers.wait.strategy.AbstractWaitStrategy() { + @Override + protected void waitUntilReady() { + final DockerClient client = DockerClientFactory.instance().client(); + final WaitingConsumer waitingConsumer = new WaitingConsumer(); + + LogUtils.followOutput(client, waitStrategyTarget.getContainerId(), waitingConsumer); + + try { + waitingConsumer.waitUntil( + f -> f.getUtf8String().contains("ready to serve client requests"), + startupTimeout.getSeconds(), + TimeUnit.SECONDS, + 1 + ); + } catch (TimeoutException e) { + throw new ContainerLaunchException("Timed out"); + } + } + }; + } + + +} diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdTestBase.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdTestBase.java new file mode 100644 index 00000000000..0ab38e3aceb --- /dev/null +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdTestBase.java @@ -0,0 +1,91 @@ +/* + * 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.metadata.etcd.testing; + +import com.coreos.jetcd.Client; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.function.Consumer; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.versioning.Versioned; +import org.apache.commons.lang.RandomStringUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +/** + * A test base that setup etcd cluster for testing. + */ +@Slf4j +public abstract class EtcdTestBase { + + protected static EtcdContainer etcdContainer; + + @BeforeClass + public static void setupCluster() throws Exception { + etcdContainer = new EtcdContainer(RandomStringUtils.randomAlphabetic(8)); + etcdContainer.start(); + log.info("Successfully started etcd at {}", etcdContainer.getClientEndpoint()); + } + + @AfterClass + public static void teardownCluster() throws Exception { + if (null != etcdContainer) { + etcdContainer.stop(); + log.info("Successfully stopped etcd."); + } + } + + protected Client etcdClient; + + protected static Client newEtcdClient() { + Client client = Client.builder() + .endpoints(etcdContainer.getClientEndpoint()) + .build(); + return client; + } + + protected static Consumer>> consumeVersionedKeySet( + LinkedBlockingQueue>> notifications) { + return versionedKeys -> { + log.info("Received new keyset : {}", versionedKeys); + try { + notifications.put(versionedKeys); + } catch (InterruptedException e) { + log.error("Interrupted at enqueuing updated key set", e); + } + }; + } + + @Before + public void setUp() throws Exception { + etcdClient = newEtcdClient(); + log.info("Successfully build etcd client to endpoint {}", etcdContainer.getClientEndpoint()); + } + + @After + public void tearDown() throws Exception { + if (null != etcdClient) { + etcdClient.close(); + log.info("Successfully close etcd client to endpoint {}", etcdContainer.getClientEndpoint()); + } + } + +} diff --git a/metadata-drivers/etcd/src/test/resources/log4j.properties b/metadata-drivers/etcd/src/test/resources/log4j.properties new file mode 100644 index 00000000000..e6e9f783730 --- /dev/null +++ b/metadata-drivers/etcd/src/test/resources/log4j.properties @@ -0,0 +1,39 @@ +# +# 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. +# + +# +# Bookkeeper Logging Configuration +# + +# Format is " (, )+ + +# 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.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n + +#disable zookeeper logging +log4j.logger.org.apache.zookeeper=OFF +log4j.logger.org.apache.bookkeeper.bookie=INFO +log4j.logger.org.apache.bookkeeper.meta=INFO diff --git a/metadata-drivers/pom.xml b/metadata-drivers/pom.xml new file mode 100644 index 00000000000..9cc47ca894d --- /dev/null +++ b/metadata-drivers/pom.xml @@ -0,0 +1,34 @@ + + + + + bookkeeper + org.apache.bookkeeper + 4.9.0-SNAPSHOT + + 4.0.0 + org.apache.bookkeeper.metadata.drivers + metadata-drivers-parent + pom + Apache BookKeeper :: Metadata Drivers :: Parent + + etcd + + diff --git a/pom.xml b/pom.xml index 701b9a55e6d..a788fcf36af 100644 --- a/pom.xml +++ b/pom.xml @@ -66,6 +66,7 @@ microbenchmarks stream/distributedlog tools + metadata-drivers @@ -120,6 +121,7 @@ 2.4 4.0.1 3.1.0 + 0.0.2 6.44.0 1.14.9 3.0.2 From 45ca65139522a881c3886173308874366ddb7925 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sun, 23 Sep 2018 23:08:42 -0700 Subject: [PATCH 0040/1642] [TABLE SERVICE] [PYTHON] introduce a python client for table service Descriptions of the changes in this PR: *Motivation* We need a python client for accessing table service. *Changes* This is the first change for introducing a python client for table service. - setup the directory structure for the python client - setup build, lint, test for the client - implement the admin client for creating/deleting/getting namespaces/streams - add examples for admin client Master Issue: #1690 Author: Sijie Guo Reviewers: Jia Zhai This closes #1691 from sijie/stream_python_client --- pom.xml | 17 + stream/clients/python/.coveragerc | 30 + stream/clients/python/.flake8 | 23 + stream/clients/python/.gitignore | 22 + stream/clients/python/README.md | 4 + stream/clients/python/bookkeeper/__init__.py | 20 + .../python/bookkeeper/admin/__init__.py | 19 + .../clients/python/bookkeeper/admin/client.py | 72 + .../python/bookkeeper/admin/exceptions.py | 19 + .../python/bookkeeper/admin/namespace.py | 60 + .../python/bookkeeper/admin/namespaces.py | 53 + .../python/bookkeeper/common/__init__.py | 13 + .../python/bookkeeper/common/constants.py | 45 + .../bookkeeper/common/datetime_helpers.py | 224 +++ .../python/bookkeeper/common/exceptions.py | 185 +++ .../bookkeeper/common/future/__init__.py | 19 + .../bookkeeper/common/future/_helpers.py | 37 + .../python/bookkeeper/common/future/base.py | 64 + .../bookkeeper/common/future/polling.py | 161 ++ .../bookkeeper/common/general_helpers.py | 30 + .../python/bookkeeper/common/grpc_helpers.py | 313 ++++ .../bookkeeper/common/protobuf_helpers.py | 321 ++++ .../clients/python/bookkeeper/common/retry.py | 282 ++++ .../python/bookkeeper/common/service_uri.py | 31 + .../clients/python/bookkeeper/kv/__init__.py | 19 + stream/clients/python/bookkeeper/kv/client.py | 60 + .../python/bookkeeper/kv/exceptions.py | 19 + .../clients/python/bookkeeper/kv/futures.py | 170 +++ .../python/bookkeeper/proto/cluster_pb2.py | 199 +++ .../python/bookkeeper/proto/common_pb2.py | 77 + .../clients/python/bookkeeper/proto/kv_pb2.py | 192 +++ .../python/bookkeeper/proto/kv_rpc_pb2.py | 1232 +++++++++++++++ .../bookkeeper/proto/kv_rpc_pb2_grpc.py | 122 ++ .../python/bookkeeper/proto/kv_store_pb2.py | 297 ++++ .../python/bookkeeper/proto/storage_pb2.py | 1276 ++++++++++++++++ .../bookkeeper/proto/storage_pb2_grpc.py | 219 +++ .../python/bookkeeper/proto/stream_pb2.py | 1323 +++++++++++++++++ stream/clients/python/bookkeeper/types.py | 71 + stream/clients/python/examples/admin.py | 58 + stream/clients/python/noxfile.py | 86 ++ stream/clients/python/pylint.config.py | 29 + stream/clients/python/setup.cfg | 14 + stream/clients/python/setup.py | 87 ++ stream/clients/python/tests/unit/__init__.py | 11 + .../bookkeeper/common/future/test__helpers.py | 35 + .../bookkeeper/common/future/test_polling.py | 155 ++ .../bookkeeper/common/test_grpc_helpers.py | 172 +++ .../common/test_protobuf_helpers.py | 274 ++++ .../tests/unit/bookkeeper/test_futures.py | 145 ++ 49 files changed, 8406 insertions(+) create mode 100644 stream/clients/python/.coveragerc create mode 100644 stream/clients/python/.flake8 create mode 100644 stream/clients/python/.gitignore create mode 100644 stream/clients/python/README.md create mode 100644 stream/clients/python/bookkeeper/__init__.py create mode 100644 stream/clients/python/bookkeeper/admin/__init__.py create mode 100644 stream/clients/python/bookkeeper/admin/client.py create mode 100644 stream/clients/python/bookkeeper/admin/exceptions.py create mode 100644 stream/clients/python/bookkeeper/admin/namespace.py create mode 100644 stream/clients/python/bookkeeper/admin/namespaces.py create mode 100644 stream/clients/python/bookkeeper/common/__init__.py create mode 100644 stream/clients/python/bookkeeper/common/constants.py create mode 100644 stream/clients/python/bookkeeper/common/datetime_helpers.py create mode 100644 stream/clients/python/bookkeeper/common/exceptions.py create mode 100644 stream/clients/python/bookkeeper/common/future/__init__.py create mode 100644 stream/clients/python/bookkeeper/common/future/_helpers.py create mode 100644 stream/clients/python/bookkeeper/common/future/base.py create mode 100644 stream/clients/python/bookkeeper/common/future/polling.py create mode 100644 stream/clients/python/bookkeeper/common/general_helpers.py create mode 100644 stream/clients/python/bookkeeper/common/grpc_helpers.py create mode 100644 stream/clients/python/bookkeeper/common/protobuf_helpers.py create mode 100644 stream/clients/python/bookkeeper/common/retry.py create mode 100644 stream/clients/python/bookkeeper/common/service_uri.py create mode 100644 stream/clients/python/bookkeeper/kv/__init__.py create mode 100644 stream/clients/python/bookkeeper/kv/client.py create mode 100644 stream/clients/python/bookkeeper/kv/exceptions.py create mode 100644 stream/clients/python/bookkeeper/kv/futures.py create mode 100644 stream/clients/python/bookkeeper/proto/cluster_pb2.py create mode 100644 stream/clients/python/bookkeeper/proto/common_pb2.py create mode 100644 stream/clients/python/bookkeeper/proto/kv_pb2.py create mode 100644 stream/clients/python/bookkeeper/proto/kv_rpc_pb2.py create mode 100644 stream/clients/python/bookkeeper/proto/kv_rpc_pb2_grpc.py create mode 100644 stream/clients/python/bookkeeper/proto/kv_store_pb2.py create mode 100644 stream/clients/python/bookkeeper/proto/storage_pb2.py create mode 100644 stream/clients/python/bookkeeper/proto/storage_pb2_grpc.py create mode 100644 stream/clients/python/bookkeeper/proto/stream_pb2.py create mode 100644 stream/clients/python/bookkeeper/types.py create mode 100644 stream/clients/python/examples/admin.py create mode 100644 stream/clients/python/noxfile.py create mode 100644 stream/clients/python/pylint.config.py create mode 100644 stream/clients/python/setup.cfg create mode 100644 stream/clients/python/setup.py create mode 100644 stream/clients/python/tests/unit/__init__.py create mode 100644 stream/clients/python/tests/unit/bookkeeper/common/future/test__helpers.py create mode 100644 stream/clients/python/tests/unit/bookkeeper/common/future/test_polling.py create mode 100644 stream/clients/python/tests/unit/bookkeeper/common/test_grpc_helpers.py create mode 100644 stream/clients/python/tests/unit/bookkeeper/common/test_protobuf_helpers.py create mode 100644 stream/clients/python/tests/unit/bookkeeper/test_futures.py diff --git a/pom.xml b/pom.xml index a788fcf36af..e5a3ffde00a 100644 --- a/pom.xml +++ b/pom.xml @@ -824,6 +824,9 @@ ${apache-rat-plugin.version} + + **/.idea/** + .git/**/* .github/**/* @@ -867,6 +870,20 @@ dev/.vagrant/** + + + **/proto/**.py + + **/python/.coverage + **/python/.Python + **/python/bin/** + **/python/include/** + **/python/lib/** + **/.nox/** + **/.pytest_cache/** + **/__pycache__/** + **/bookkeeper.egg-info/** + **/pip-selfcheck.json true diff --git a/stream/clients/python/.coveragerc b/stream/clients/python/.coveragerc new file mode 100644 index 00000000000..2a6c4e711f5 --- /dev/null +++ b/stream/clients/python/.coveragerc @@ -0,0 +1,30 @@ +# Licensed 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. + +[run] +branch = True +source = + bookkeeper + tests.unit + +[report] +omit = + */proto/* +show_missing = True + +exclude_lines = + # Re-enable the standard pragma + pragma: NO COVER + # Ignore debug-only repr + def __repr__ + # Ignore abstract methods + raise NotImplementedError diff --git a/stream/clients/python/.flake8 b/stream/clients/python/.flake8 new file mode 100644 index 00000000000..6c93f2015b1 --- /dev/null +++ b/stream/clients/python/.flake8 @@ -0,0 +1,23 @@ +# Licensed 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. + +[flake8] +exclude = + # Exclude generated code. + **/proto/** + *_pb2.py + + # Standard linting exemptions. + __pycache__, + .git, + *.pyc, + conf.py diff --git a/stream/clients/python/.gitignore b/stream/clients/python/.gitignore new file mode 100644 index 00000000000..a520e657a93 --- /dev/null +++ b/stream/clients/python/.gitignore @@ -0,0 +1,22 @@ +# virtualenv +.Python +bin/ +include/ + +# coverage +.coverage + +# nox +.nox/ + +# pytest +.pytest_cache/ + +# pycache +__pycache__/ + +# build files +bookkeeper.egg-info/ + +# pip +pip-selfcheck.json diff --git a/stream/clients/python/README.md b/stream/clients/python/README.md new file mode 100644 index 00000000000..df1b7efd248 --- /dev/null +++ b/stream/clients/python/README.md @@ -0,0 +1,4 @@ +Python Client for Apache BookKeeper +=================================== + +|pypi| |versions| diff --git a/stream/clients/python/bookkeeper/__init__.py b/stream/clients/python/bookkeeper/__init__.py new file mode 100644 index 00000000000..c1052c168bf --- /dev/null +++ b/stream/clients/python/bookkeeper/__init__.py @@ -0,0 +1,20 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +try: + import pkg_resources + pkg_resources.declare_namespace(__name__) +except ImportError: + import pkgutil + __path__ = pkgutil.extend_path(__path__, __name__) diff --git a/stream/clients/python/bookkeeper/admin/__init__.py b/stream/clients/python/bookkeeper/admin/__init__.py new file mode 100644 index 00000000000..dce620fe6eb --- /dev/null +++ b/stream/clients/python/bookkeeper/admin/__init__.py @@ -0,0 +1,19 @@ +# Licensed 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. + +from __future__ import absolute_import + +from bookkeeper.admin.client import Client + +__all__ = ( + 'Client', +) diff --git a/stream/clients/python/bookkeeper/admin/client.py b/stream/clients/python/bookkeeper/admin/client.py new file mode 100644 index 00000000000..d3dff3fab77 --- /dev/null +++ b/stream/clients/python/bookkeeper/admin/client.py @@ -0,0 +1,72 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +from __future__ import absolute_import + +import grpc +import logging +import pkg_resources + +from bookkeeper import types +from bookkeeper.admin.namespace import Namespace +from bookkeeper.admin.namespaces import Namespaces +from bookkeeper.common.service_uri import ServiceURI +from bookkeeper.proto.storage_pb2_grpc import RootRangeServiceStub + +__version__ = pkg_resources.get_distribution('bookkeeper').version +__logger__ = logging.getLogger("bookkeeper.admin.Client") + + +class Client(object): + """An admin client for Apache BookKeeper. + + This creates an object that is capable of administrating bookkeeper + resources. Generally, you can instantiate this client with no arguments, + and you get sensible defaults. + + Args: + storage_client_settings (~bookkeeper.types.StorageClientSettings): The + settings for bookkeeper storage client . + kwargs (dict): Any additional arguments provided are sent as keyword + arguments to the underlying grpc client. + """ + + def __init__(self, storage_client_settings=(), **kwargs): + # init the storage client settings + self.storage_client_settings =\ + types.StorageClientSettings(*storage_client_settings) + __logger__.info("Creating an admin client to cluster '%s'", + self.storage_client_settings.service_uri) + + service_uri = ServiceURI(self.storage_client_settings.service_uri) + assert service_uri.service_name.lower() == 'bk' + + # create channel + self.channel = grpc.insecure_channel( + target=service_uri.service_location + ) + __logger__.info("Successfully created an admin client to cluster '%s'", + self.storage_client_settings.service_uri) + + # create the rpc stub + self.root_range = RootRangeServiceStub(channel=self.channel) + + # services + self.__namespaces__ = Namespaces(client=self) + + def namespaces(self): + return self.__namespaces__ + + def namespace(self, namespace): + return Namespace(self, namespace) diff --git a/stream/clients/python/bookkeeper/admin/exceptions.py b/stream/clients/python/bookkeeper/admin/exceptions.py new file mode 100644 index 00000000000..3bc01a8588f --- /dev/null +++ b/stream/clients/python/bookkeeper/admin/exceptions.py @@ -0,0 +1,19 @@ +# Licensed 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. + +from __future__ import absolute_import + +from concurrent.futures import TimeoutError + +__all__ = ( + 'TimeoutError', +) diff --git a/stream/clients/python/bookkeeper/admin/namespace.py b/stream/clients/python/bookkeeper/admin/namespace.py new file mode 100644 index 00000000000..029c2c488be --- /dev/null +++ b/stream/clients/python/bookkeeper/admin/namespace.py @@ -0,0 +1,60 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +from __future__ import absolute_import + +from bookkeeper.common.constants import __DEFAULT_STREAM_CONF__ +from bookkeeper.common.constants import __ROOT_RANGE_METADATA__ +from bookkeeper.proto import storage_pb2 +from bookkeeper.proto import stream_pb2 + + +class Namespace(object): + + def __init__(self, client, namespace): + self.client = client + self.namespace = namespace + + def create(self, stream_name, stream_config=__DEFAULT_STREAM_CONF__): + create_stream_req = storage_pb2.CreateStreamRequest( + ns_name=self.namespace, + name=stream_name, + stream_conf=stream_config + ) + return self.client.root_range.CreateStream( + request=create_stream_req, + metadata=__ROOT_RANGE_METADATA__ + ) + + def get(self, stream_name): + get_stream_req = storage_pb2.GetStreamRequest( + stream_name=stream_pb2.StreamName( + namespace_name=self.namespace, + stream_name=stream_name + ) + ) + return self.client.root_range.GetStream( + request=get_stream_req, + metadata=__ROOT_RANGE_METADATA__ + ) + + def delete(self, stream_name): + del_stream_req = storage_pb2.DeleteStreamRequest( + ns_name=self.namespace, + name=stream_name + ) + return self.client.root_range.DeleteStream( + request=del_stream_req, + metadata=__ROOT_RANGE_METADATA__ + ) diff --git a/stream/clients/python/bookkeeper/admin/namespaces.py b/stream/clients/python/bookkeeper/admin/namespaces.py new file mode 100644 index 00000000000..7589d081323 --- /dev/null +++ b/stream/clients/python/bookkeeper/admin/namespaces.py @@ -0,0 +1,53 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +from __future__ import absolute_import + +from bookkeeper.common.constants import __DEFAULT_NS_CONF__ +from bookkeeper.common.constants import __ROOT_RANGE_METADATA__ +from bookkeeper.proto import storage_pb2 + + +class Namespaces(object): + + def __init__(self, client): + self.client = client + + def create(self, namespace, namespace_config=__DEFAULT_NS_CONF__): + create_ns_req = storage_pb2.CreateNamespaceRequest( + name=namespace, + ns_conf=namespace_config + ) + return self.client.root_range.CreateNamespace( + request=create_ns_req, + metadata=__ROOT_RANGE_METADATA__ + ) + + def get(self, namespace): + get_ns_req = storage_pb2.GetNamespaceRequest( + name=namespace + ) + return self.client.root_range.GetNamespace( + request=get_ns_req, + metadata=__ROOT_RANGE_METADATA__ + ) + + def delete(self, namespace): + del_ns_req = storage_pb2.DeleteNamespaceRequest( + name=namespace + ) + return self.client.root_range.DeleteNamespace( + request=del_ns_req, + metadata=__ROOT_RANGE_METADATA__ + ) diff --git a/stream/clients/python/bookkeeper/common/__init__.py b/stream/clients/python/bookkeeper/common/__init__.py new file mode 100644 index 00000000000..89c7831391e --- /dev/null +++ b/stream/clients/python/bookkeeper/common/__init__.py @@ -0,0 +1,13 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. diff --git a/stream/clients/python/bookkeeper/common/constants.py b/stream/clients/python/bookkeeper/common/constants.py new file mode 100644 index 00000000000..bec9e83d182 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/constants.py @@ -0,0 +1,45 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +from bookkeeper.proto import stream_pb2 + +__ROOT_RANGE_ID__ = 0 +__ROOT_RANGE_METADATA__ = [ + ('sc-id-bin', __ROOT_RANGE_ID__.to_bytes(8, "little")) +] +__DEFAULT_STREAM_CONF__ = stream_pb2.StreamConfiguration( + key_type=stream_pb2.RangeKeyType.values()[0], + min_num_ranges=24, + initial_num_ranges=4, + split_policy=stream_pb2.SplitPolicy( + type=stream_pb2.SplitPolicyType.values()[0], + fixed_range_policy=stream_pb2.FixedRangeSplitPolicy( + num_ranges=2 + ) + ), + rolling_policy=stream_pb2.SegmentRollingPolicy( + size_policy=stream_pb2.SizeBasedSegmentRollingPolicy( + max_segment_size=128*1024*1024 + ) + ), + retention_policy=stream_pb2.RetentionPolicy( + time_policy=stream_pb2.TimeBasedRetentionPolicy( + retention_minutes=-1 + ) + ), + storage_type=stream_pb2.StorageType.values()[1] +) +__DEFAULT_NS_CONF__ = stream_pb2.NamespaceConfiguration( + default_stream_conf=__DEFAULT_STREAM_CONF__ +) diff --git a/stream/clients/python/bookkeeper/common/datetime_helpers.py b/stream/clients/python/bookkeeper/common/datetime_helpers.py new file mode 100644 index 00000000000..a4671648d8e --- /dev/null +++ b/stream/clients/python/bookkeeper/common/datetime_helpers.py @@ -0,0 +1,224 @@ +# Licensed 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. + +"""Helpers for :mod:`datetime`.""" + +import calendar +import datetime +import re + +import pytz + + +_UTC_EPOCH = datetime.datetime.utcfromtimestamp(0).replace(tzinfo=pytz.utc) +_RFC3339_MICROS = '%Y-%m-%dT%H:%M:%S.%fZ' +_RFC3339_NO_FRACTION = '%Y-%m-%dT%H:%M:%S' +# datetime.strptime cannot handle nanosecond precision: parse w/ regex +_RFC3339_NANOS = re.compile(r""" + (?P + \d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2} # YYYY-MM-DDTHH:MM:SS + ) + ( # Optional decimal part + \. # decimal point + (?P\d{1,9}) # nanoseconds, maybe truncated + )? + Z # Zulu +""", re.VERBOSE) + + +def utcnow(): + """A :meth:`datetime.datetime.utcnow()` alias to allow mocking in tests.""" + return datetime.datetime.utcnow() + + +def to_milliseconds(value): + """Convert a zone-aware datetime to milliseconds since the unix epoch. + Args: + value (datetime.datetime): The datetime to covert. + Returns: + int: Milliseconds since the unix epoch. + """ + micros = to_microseconds(value) + return micros // 1000 + + +def from_microseconds(value): + """Convert timestamp in microseconds since the unix epoch to datetime. + Args: + value (float): The timestamp to convert, in microseconds. + Returns: + datetime.datetime: The datetime object equivalent to the timestamp in + UTC. + """ + return _UTC_EPOCH + datetime.timedelta(microseconds=value) + + +def to_microseconds(value): + """Convert a datetime to microseconds since the unix epoch. + Args: + value (datetime.datetime): The datetime to covert. + Returns: + int: Microseconds since the unix epoch. + """ + if not value.tzinfo: + value = value.replace(tzinfo=pytz.utc) + # Regardless of what timezone is on the value, convert it to UTC. + value = value.astimezone(pytz.utc) + # Convert the datetime to a microsecond timestamp. + return int(calendar.timegm(value.timetuple()) * 1e6) + value.microsecond + + +def from_iso8601_date(value): + """Convert a ISO8601 date string to a date. + Args: + value (str): The ISO8601 date string. + Returns: + datetime.date: A date equivalent to the date string. + """ + return datetime.datetime.strptime(value, '%Y-%m-%d').date() + + +def from_iso8601_time(value): + """Convert a zoneless ISO8601 time string to a time. + Args: + value (str): The ISO8601 time string. + Returns: + datetime.time: A time equivalent to the time string. + """ + return datetime.datetime.strptime(value, '%H:%M:%S').time() + + +def from_rfc3339(value): + """Convert a microsecond-precision timestamp to datetime. + Args: + value (str): The RFC3339 string to convert. + Returns: + datetime.datetime: The datetime object equivalent to the timestamp in + UTC. + """ + return datetime.datetime.strptime( + value, _RFC3339_MICROS).replace(tzinfo=pytz.utc) + + +def from_rfc3339_nanos(value): + """Convert a nanosecond-precision timestamp to a native datetime. + .. note:: + Python datetimes do not support nanosecond precision; this function + therefore truncates such values to microseconds. + Args: + value (str): The RFC3339 string to convert. + Returns: + datetime.datetime: The datetime object equivalent to the timestamp in + UTC. + Raises: + ValueError: If the timestamp does not match the RFC 3339 + regular expression. + """ + with_nanos = _RFC3339_NANOS.match(value) + + if with_nanos is None: + raise ValueError( + 'Timestamp: {!r}, does not match pattern: {!r}'.format( + value, _RFC3339_NANOS.pattern)) + + bare_seconds = datetime.datetime.strptime( + with_nanos.group('no_fraction'), _RFC3339_NO_FRACTION) + fraction = with_nanos.group('nanos') + + if fraction is None: + micros = 0 + else: + scale = 9 - len(fraction) + nanos = int(fraction) * (10 ** scale) + micros = nanos // 1000 + + return bare_seconds.replace(microsecond=micros, tzinfo=pytz.utc) + + +def to_rfc3339(value, ignore_zone=True): + """Convert a datetime to an RFC3339 timestamp string. + Args: + value (datetime.datetime): + The datetime object to be converted to a string. + ignore_zone (bool): If True, then the timezone (if any) of the + datetime object is ignored and the datetime is treated as UTC. + Returns: + str: The RFC3339 formated string representing the datetime. + """ + if not ignore_zone and value.tzinfo is not None: + # Convert to UTC and remove the time zone info. + value = value.replace(tzinfo=None) - value.utcoffset() + + return value.strftime(_RFC3339_MICROS) + + +class DatetimeWithNanoseconds(datetime.datetime): + """Track nanosecond in addition to normal datetime attrs. + Nanosecond can be passed only as a keyword argument. + """ + __slots__ = ('_nanosecond',) + + # pylint: disable=arguments-differ + def __new__(cls, *args, **kw): + nanos = kw.pop('nanosecond', 0) + if nanos > 0: + if 'microsecond' in kw: + raise TypeError( + "Specify only one of 'microsecond' or 'nanosecond'") + kw['microsecond'] = nanos // 1000 + inst = datetime.datetime.__new__(cls, *args, **kw) + inst._nanosecond = nanos or 0 + return inst + # pylint: disable=arguments-differ + + @property + def nanosecond(self): + """Read-only: nanosecond precision.""" + return self._nanosecond + + def rfc3339(self): + """Return an RFC 3339-compliant timestamp. + Returns: + (str): Timestamp string according to RFC 3339 spec. + """ + if self._nanosecond == 0: + return to_rfc3339(self) + nanos = str(self._nanosecond).rstrip('0') + return '{}.{}Z'.format(self.strftime(_RFC3339_NO_FRACTION), nanos) + + @classmethod + def from_rfc3339(cls, stamp): + """Parse RFC 3339-compliant timestamp, preserving nanoseconds. + Args: + stamp (str): RFC 3339 stamp, with up to nanosecond precision + Returns: + :class:`DatetimeWithNanoseconds`: + an instance matching the timestamp string + Raises: + ValueError: if `stamp` does not match the expected format + """ + with_nanos = _RFC3339_NANOS.match(stamp) + if with_nanos is None: + raise ValueError( + 'Timestamp: {}, does not match pattern: {}'.format( + stamp, _RFC3339_NANOS.pattern)) + bare = datetime.datetime.strptime( + with_nanos.group('no_fraction'), _RFC3339_NO_FRACTION) + fraction = with_nanos.group('nanos') + if fraction is None: + nanos = 0 + else: + scale = 9 - len(fraction) + nanos = int(fraction) * (10 ** scale) + return cls(bare.year, bare.month, bare.day, + bare.hour, bare.minute, bare.second, + nanosecond=nanos, tzinfo=pytz.UTC) diff --git a/stream/clients/python/bookkeeper/common/exceptions.py b/stream/clients/python/bookkeeper/common/exceptions.py new file mode 100644 index 00000000000..34ff1227dcf --- /dev/null +++ b/stream/clients/python/bookkeeper/common/exceptions.py @@ -0,0 +1,185 @@ +# Licensed 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. + +"""Exceptions raised by bookkeeper clients. +This module provides base classes for all errors raised by libraries based +on :mod:`bookkeeper.common`. +""" + +from __future__ import absolute_import +from __future__ import unicode_literals +from bookkeeper.proto.storage_pb2 import BAD_REQUEST +from bookkeeper.proto.storage_pb2 import INTERNAL_SERVER_ERROR + +import six + +try: + import grpc +except ImportError: # pragma: NO COVER + grpc = None + +# Lookup tables for mapping exceptions from gRPC transports. +# Populated by _APICallErrorMeta +_GRPC_CODE_TO_EXCEPTION = {} + + +class BKError(Exception): + """Base class for all exceptions raised by bookkeeper Clients.""" + pass + + +@six.python_2_unicode_compatible +class RetryError(BKError): + """Raised when a function has exhausted all of its available retries. + Args: + message (str): The exception message. + cause (Exception): The last exception raised when retring the + function. + """ + def __init__(self, message, cause): + super(RetryError, self).__init__(message) + self.message = message + self._cause = cause + + @property + def cause(self): + """The last exception raised when retrying the function.""" + return self._cause + + def __str__(self): + return '{}, last exception: {}'.format(self.message, self.cause) + + +class _BKGRpcCallErrorMeta(type): + """Metaclass for registering BKGRpcCallError subclasses.""" + def __new__(mcs, name, bases, class_dict): + cls = type.__new__(mcs, name, bases, class_dict) + if cls.grpc_status_code is not None: + _GRPC_CODE_TO_EXCEPTION.setdefault(cls.grpc_status_code, cls) + return cls + + +@six.python_2_unicode_compatible +@six.add_metaclass(_BKGRpcCallErrorMeta) +class BKGrpcCallError(BKError): + """Base class for exceptions raised by calling API methods. + Args: + message (str): The exception message. + errors (Sequence[Any]): An optional list of error details. + response (Union[requests.Request, grpc.Call]): The response or + gRPC call metadata. + """ + + grpc_status_code = None + """Optional[grpc.StatusCode]: The gRPC status code associated with this + error. + This may be ``None`` if the exception does not match up to a gRPC error. + """ + + bk_status_code = None + """Optional[bookkeeper.proto.StatusCode]: The bookkeeper storage status code + associated with this error. + This may be ```None` if the exception is a gRPC channel error. + """ + + def __init__(self, message, errors=(), response=None): + super(BKGrpcCallError, self).__init__(message) + self.message = message + """str: The exception message.""" + self._errors = errors + self._response = response + + def __str__(self): + return '{} {}'.format(self.code, self.message) + + @property + def errors(self): + """Detailed error information. + Returns: + Sequence[Any]: A list of additional error details. + """ + return list(self._errors) + + @property + def response(self): + """Optional[Union[requests.Request, grpc.Call]]: The response or + gRPC call metadata.""" + return self._response + + +class ClientError(BKGrpcCallError): + """Base class for all client error responses.""" + + +class BadRequest(ClientError): + """Exception mapping a ``400 Bad Request`` response.""" + code = BAD_REQUEST + + +class ServerError(BKGrpcCallError): + """Base for 5xx responses.""" + + +class InternalServerError(ServerError): + """Exception mapping a ``500 Internal Server Error`` response. or a + :attr:`grpc.StatusCode.INTERNAL` error.""" + code = INTERNAL_SERVER_ERROR + grpc_status_code = grpc.StatusCode.INTERNAL if grpc is not None else None + + +def exception_class_for_grpc_status(status_code): + """Return the exception class for a specific :class:`grpc.StatusCode`. + Args: + status_code (grpc.StatusCode): The gRPC status code. + Returns: + :func:`type`: the appropriate subclass of :class:`BKGrpcCallError`. + """ + return _GRPC_CODE_TO_EXCEPTION.get(status_code, BKGrpcCallError) + + +def from_grpc_status(status_code, message, **kwargs): + """Create a :class:`BKGrpcCallError` from a :class:`grpc.StatusCode`. + Args: + status_code (grpc.StatusCode): The gRPC status code. + message (str): The exception message. + kwargs: Additional arguments passed to the :class:`BKGrpcCallError` + constructor. + Returns: + BKGrpcCallError: An instance of the appropriate subclass of + :class:`BKGrpcCallError`. + """ + error_class = exception_class_for_grpc_status(status_code) + error = error_class(message, **kwargs) + + if error.grpc_status_code is None: + error.grpc_status_code = status_code + + return error + + +def from_grpc_error(rpc_exc): + """Create a :class:`BKGrpcCallError` from a :class:`grpc.RpcError`. + Args: + rpc_exc (grpc.RpcError): The gRPC error. + Returns: + BKGrpcCallError: An instance of the appropriate subclass of + :class:`BKGrpcError`. + """ + if isinstance(rpc_exc, grpc.Call): + return from_grpc_status( + rpc_exc.code(), + rpc_exc.details(), + errors=(rpc_exc,), + response=rpc_exc) + else: + return BKGrpcCallError( + str(rpc_exc), errors=(rpc_exc,), response=rpc_exc) diff --git a/stream/clients/python/bookkeeper/common/future/__init__.py b/stream/clients/python/bookkeeper/common/future/__init__.py new file mode 100644 index 00000000000..f9d985cb5ea --- /dev/null +++ b/stream/clients/python/bookkeeper/common/future/__init__.py @@ -0,0 +1,19 @@ +# Licensed 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. + +"""Futures for dealing with asynchronous operations.""" + +from bookkeeper.common.future.base import Future + +__all__ = [ + 'Future', +] diff --git a/stream/clients/python/bookkeeper/common/future/_helpers.py b/stream/clients/python/bookkeeper/common/future/_helpers.py new file mode 100644 index 00000000000..0aa841dba04 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/future/_helpers.py @@ -0,0 +1,37 @@ +# Licensed 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. + +"""Private helpers for futures.""" + +import logging +import threading + + +_LOGGER = logging.getLogger(__name__) + + +def start_daemon_thread(*args, **kwargs): + """Starts a thread and marks it as a daemon thread.""" + thread = threading.Thread(*args, **kwargs) + thread.daemon = True + thread.start() + return thread + + +def safe_invoke_callback(callback, *args, **kwargs): + """Invoke a callback, swallowing and logging any exceptions.""" + # pylint: disable=bare-except + # We intentionally want to swallow all exceptions. + try: + return callback(*args, **kwargs) + except Exception: + _LOGGER.exception('Error while executing Future callback.') diff --git a/stream/clients/python/bookkeeper/common/future/base.py b/stream/clients/python/bookkeeper/common/future/base.py new file mode 100644 index 00000000000..b548c434f78 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/future/base.py @@ -0,0 +1,64 @@ +# Licensed 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. + +"""Abstract and helper bases for Future implementations.""" + +import abc + +import six + + +@six.add_metaclass(abc.ABCMeta) +class Future(object): + # pylint: disable=missing-docstring + # We inherit the interfaces here from concurrent.futures. + + """Future interface. + This interface is based on :class:`concurrent.futures.Future`. + """ + + @abc.abstractmethod + def cancel(self): + raise NotImplementedError() + + @abc.abstractmethod + def cancelled(self): + raise NotImplementedError() + + @abc.abstractmethod + def running(self): + raise NotImplementedError() + + @abc.abstractmethod + def done(self): + raise NotImplementedError() + + @abc.abstractmethod + def result(self, timeout=None): + raise NotImplementedError() + + @abc.abstractmethod + def exception(self, timeout=None): + raise NotImplementedError() + + @abc.abstractmethod + def add_done_callback(self, fn): + # pylint: disable=invalid-name + raise NotImplementedError() + + @abc.abstractmethod + def set_result(self, result): + raise NotImplementedError() + + @abc.abstractmethod + def set_exception(self, exception): + raise NotImplementedError() diff --git a/stream/clients/python/bookkeeper/common/future/polling.py b/stream/clients/python/bookkeeper/common/future/polling.py new file mode 100644 index 00000000000..2bde3a87913 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/future/polling.py @@ -0,0 +1,161 @@ +# Licensed 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. + +"""Abstract and helper bases for Future implementations.""" + +import abc +import concurrent.futures + +from bookkeeper.common import exceptions +from bookkeeper.common import retry +from bookkeeper.common.future import _helpers +from bookkeeper.common.future import base + + +class _OperationNotComplete(Exception): + """Private exception used for polling via retry.""" + pass + + +RETRY_PREDICATE = retry.if_exception_type(_OperationNotComplete) +DEFAULT_RETRY = retry.Retry(predicate=RETRY_PREDICATE) + + +class PollingFuture(base.Future): + """A Future that needs to poll some service to check its status. + The :meth:`done` method should be implemented by subclasses. The polling + behavior will repeatedly call ``done`` until it returns True. + .. note: Privacy here is intended to prevent the final class from + overexposing, not to prevent subclasses from accessing methods. + Args: + retry (bookkeeper.common.retry.Retry): The retry configuration used + when polling. This can be used to control how often :meth:`done` + is polled. Regardless of the retry's ``deadline``, it will be + overridden by the ``timeout`` argument to :meth:`result`. + """ + def __init__(self, retry=DEFAULT_RETRY): + super(PollingFuture, self).__init__() + self._retry = retry + self._result = None + self._exception = None + self._result_set = False + """bool: Set to True when the result has been set via set_result or + set_exception.""" + self._polling_thread = None + self._done_callbacks = [] + + @abc.abstractmethod + def done(self): + """Checks to see if the operation is complete. + Returns: + bool: True if the operation is complete, False otherwise. + """ + # pylint: disable=redundant-returns-doc, missing-raises-doc + raise NotImplementedError() + + def _done_or_raise(self): + """Check if the future is done and raise if it's not.""" + if not self.done(): + raise _OperationNotComplete() + + def running(self): + """True if the operation is currently running.""" + return not self.done() + + def _blocking_poll(self, timeout=None): + """Poll and wait for the Future to be resolved. + Args: + timeout (int): + How long (in seconds) to wait for the operation to complete. + If None, wait indefinitely. + """ + if self._result_set: + return + + retry_ = self._retry.with_deadline(timeout) + + try: + retry_(self._done_or_raise)() + except exceptions.RetryError: + raise concurrent.futures.TimeoutError( + 'Operation did not complete within the designated ' + 'timeout.') + + def result(self, timeout=None): + """Get the result of the operation, blocking if necessary. + Args: + timeout (int): + How long (in seconds) to wait for the operation to complete. + If None, wait indefinitely. + Returns: + google.protobuf.Message: The Operation's result. + Raises: + bookkeeper.common.exceptions.BKGrpcAPICallError: If the operation errors or if + the timeout is reached before the operation completes. + """ + self._blocking_poll(timeout=timeout) + + if self._exception is not None: + # pylint: disable=raising-bad-type + # Pylint doesn't recognize that this is valid in this case. + raise self._exception + + return self._result + + def exception(self, timeout=None): + """Get the exception from the operation, blocking if necessary. + Args: + timeout (int): How long to wait for the operation to complete. + If None, wait indefinitely. + Returns: + Optional[bookkeeper.common.exceptions.BKGrpcAPICallError]: The operation's + error. + """ + self._blocking_poll() + return self._exception + + def add_done_callback(self, fn): + """Add a callback to be executed when the operation is complete. + If the operation is not already complete, this will start a helper + thread to poll for the status of the operation in the background. + Args: + fn (Callable[Future]): The callback to execute when the operation + is complete. + """ + if self._result_set: + _helpers.safe_invoke_callback(fn, self) + return + + self._done_callbacks.append(fn) + + if self._polling_thread is None: + # The polling thread will exit on its own as soon as the operation + # is done. + self._polling_thread = _helpers.start_daemon_thread( + target=self._blocking_poll) + + def _invoke_callbacks(self, *args, **kwargs): + """Invoke all done callbacks.""" + for callback in self._done_callbacks: + _helpers.safe_invoke_callback(callback, *args, **kwargs) + + def set_result(self, result): + """Set the Future's result.""" + self._result = result + self._result_set = True + self._invoke_callbacks(self) + + def set_exception(self, exception): + """Set the Future's exception.""" + self._exception = exception + self._result_set = True + self._invoke_callbacks(self) diff --git a/stream/clients/python/bookkeeper/common/general_helpers.py b/stream/clients/python/bookkeeper/common/general_helpers.py new file mode 100644 index 00000000000..353f5e963c3 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/general_helpers.py @@ -0,0 +1,30 @@ +# Licensed 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. + +"""Helpers for general Python functionality.""" + +import functools + +import six + + +# functools.partial objects lack several attributes present on real function +# objects. In Python 2 wraps fails on this so use a restricted set instead. +_PARTIAL_VALID_ASSIGNMENTS = ('__doc__',) + + +def wraps(wrapped): + """A functools.wraps helper that handles partial objects on Python 2.""" + if isinstance(wrapped, functools.partial): + return six.wraps(wrapped, assigned=_PARTIAL_VALID_ASSIGNMENTS) + else: + return six.wraps(wrapped) diff --git a/stream/clients/python/bookkeeper/common/grpc_helpers.py b/stream/clients/python/bookkeeper/common/grpc_helpers.py new file mode 100644 index 00000000000..9e7cfd097d8 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/grpc_helpers.py @@ -0,0 +1,313 @@ +# Licensed 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. + +"""Helpers for :mod:`grpc`.""" + +import collections + +import grpc +import six + +from bookkeeper.common import exceptions +from bookkeeper.common import general_helpers + +# The list of gRPC Callable interfaces that return iterators. +_STREAM_WRAP_CLASSES = ( + grpc.UnaryStreamMultiCallable, + grpc.StreamStreamMultiCallable, +) + + +def _patch_callable_name(callable_): + """Fix-up gRPC callable attributes. + gRPC callable lack the ``__name__`` attribute which causes + :func:`functools.wraps` to error. This adds the attribute if needed. + """ + if not hasattr(callable_, '__name__'): + callable_.__name__ = callable_.__class__.__name__ + + +def _wrap_unary_errors(callable_): + """Map errors for Unary-Unary and Stream-Unary gRPC callables.""" + _patch_callable_name(callable_) + + @six.wraps(callable_) + def error_remapped_callable(*args, **kwargs): + try: + return callable_(*args, **kwargs) + except grpc.RpcError as exc: + six.raise_from(exceptions.from_grpc_error(exc), exc) + + return error_remapped_callable + + +class _StreamingResponseIterator(grpc.Call): + def __init__(self, wrapped): + self._wrapped = wrapped + + def __iter__(self): + """This iterator is also an iterable that returns itself.""" + return self + + def next(self): + """Get the next response from the stream. + Returns: + protobuf.Message: A single response from the stream. + """ + try: + return six.next(self._wrapped) + except grpc.RpcError as exc: + six.raise_from(exceptions.from_grpc_error(exc), exc) + + # Alias needed for Python 2/3 support. + __next__ = next + + # grpc.Call & grpc.RpcContext interface + + def add_callback(self, callback): + return self._wrapped.add_callback(callback) + + def cancel(self): + return self._wrapped.cancel() + + def code(self): + return self._wrapped.code() + + def details(self): + return self._wrapped.details() + + def initial_metadata(self): + return self._wrapped.initial_metadata() + + def is_active(self): + return self._wrapped.is_active() + + def time_remaining(self): + return self._wrapped.time_remaining() + + def trailing_metadata(self): + return self._wrapped.trailing_metadata() + + +def _wrap_stream_errors(callable_): + """Wrap errors for Unary-Stream and Stream-Stream gRPC callables. + The callables that return iterators require a bit more logic to re-map + errors when iterating. This wraps both the initial invocation and the + iterator of the return value to re-map errors. + """ + _patch_callable_name(callable_) + + @general_helpers.wraps(callable_) + def error_remapped_callable(*args, **kwargs): + try: + result = callable_(*args, **kwargs) + return _StreamingResponseIterator(result) + except grpc.RpcError as exc: + six.raise_from(exceptions.from_grpc_error(exc), exc) + + return error_remapped_callable + + +def wrap_errors(callable_): + """Wrap a gRPC callable and map :class:`grpc.RpcErrors` to friendly error + classes. + Errors raised by the gRPC callable are mapped to the appropriate + :class:`bookkeeper.common.exceptions.BKGrpcAPICallError` subclasses. + The original `grpc.RpcError` (which is usually also a `grpc.Call`) is + available from the ``response`` property on the mapped exception. This + is useful for extracting metadata from the original error. + Args: + callable_ (Callable): A gRPC callable. + Returns: + Callable: The wrapped gRPC callable. + """ + if isinstance(callable_, _STREAM_WRAP_CLASSES): + return _wrap_stream_errors(callable_) + else: + return _wrap_unary_errors(callable_) + + +def create_channel(target, + **kwargs): + """Create a secure channel with credentials. + Args: + target (str): The target service address in the format 'hostname:port'. + kwargs: Additional key-word args passed to + :func:`grpc_gcp.secure_channel` or :func:`grpc.secure_channel`. + Returns: + grpc.Channel: The created channel. + """ + return grpc.secure_channel(target, None, **kwargs) + + +_MethodCall = collections.namedtuple( + '_MethodCall', ('request', 'timeout', 'metadata', 'credentials')) + +_ChannelRequest = collections.namedtuple( + '_ChannelRequest', ('method', 'request')) + + +class _CallableStub(object): + """Stub for the grpc.*MultiCallable interfaces.""" + + def __init__(self, method, channel): + self._method = method + self._channel = channel + self.response = None + """Union[protobuf.Message, Callable[protobuf.Message], exception]: + The response to give when invoking this callable. If this is a + callable, it will be invoked with the request protobuf. If it's an + exception, the exception will be raised when this is invoked. + """ + self.responses = None + """Iterator[ + Union[protobuf.Message, Callable[protobuf.Message], exception]]: + An iterator of responses. If specified, self.response will be populated + on each invocation by calling ``next(self.responses)``.""" + self.requests = [] + """List[protobuf.Message]: All requests sent to this callable.""" + self.calls = [] + """List[Tuple]: All invocations of this callable. Each tuple is the + request, timeout, metadata, and credentials.""" + + def __call__(self, request, timeout=None, metadata=None, credentials=None): + self._channel.requests.append( + _ChannelRequest(self._method, request)) + self.calls.append( + _MethodCall(request, timeout, metadata, credentials)) + self.requests.append(request) + + response = self.response + if self.responses is not None: + if response is None: + response = next(self.responses) + else: + raise ValueError( + '{method}.response and {method}.responses are mutually ' + 'exclusive.'.format(method=self._method)) + + if callable(response): + return response(request) + + if isinstance(response, Exception): + raise response + + if response is not None: + return response + + raise ValueError( + 'Method stub for "{}" has no response.'.format(self._method)) + + +def _simplify_method_name(method): + """Simplifies a gRPC method name. + When gRPC invokes the channel to create a callable, it gives a full + method name like "/org.apache.bookkeeper.Table/Put". This + returns just the name of the method, in this case "Put". + Args: + method (str): The name of the method. + Returns: + str: The simplified name of the method. + """ + return method.rsplit('/', 1).pop() + + +class ChannelStub(grpc.Channel): + """A testing stub for the grpc.Channel interface. + This can be used to test any client that eventually uses a gRPC channel + to communicate. By passing in a channel stub, you can configure which + responses are returned and track which requests are made. + For example: + .. code-block:: python + channel_stub = grpc_helpers.ChannelStub() + client = FooClient(channel=channel_stub) + channel_stub.GetFoo.response = foo_pb2.Foo(name='bar') + foo = client.get_foo(labels=['baz']) + assert foo.name == 'bar' + assert channel_stub.GetFoo.requests[0].labels = ['baz'] + Each method on the stub can be accessed and configured on the channel. + Here's some examples of various configurations: + .. code-block:: python + # Return a basic response: + channel_stub.GetFoo.response = foo_pb2.Foo(name='bar') + assert client.get_foo().name == 'bar' + # Raise an exception: + channel_stub.GetFoo.response = NotFound('...') + with pytest.raises(NotFound): + client.get_foo() + # Use a sequence of responses: + channel_stub.GetFoo.responses = iter([ + foo_pb2.Foo(name='bar'), + foo_pb2.Foo(name='baz'), + ]) + assert client.get_foo().name == 'bar' + assert client.get_foo().name == 'baz' + # Use a callable + def on_get_foo(request): + return foo_pb2.Foo(name='bar' + request.id) + channel_stub.GetFoo.response = on_get_foo + assert client.get_foo(id='123').name == 'bar123' + """ + + def __init__(self, responses=[]): + self.requests = [] + """Sequence[Tuple[str, protobuf.Message]]: A list of all requests made + on this channel in order. The tuple is of method name, request + message.""" + self._method_stubs = {} + + def _stub_for_method(self, method): + method = _simplify_method_name(method) + self._method_stubs[method] = _CallableStub(method, self) + return self._method_stubs[method] + + def __getattr__(self, key): + try: + return self._method_stubs[key] + except KeyError: + raise AttributeError + + def unary_unary( + self, method, + request_serializer=None, response_deserializer=None): + """grpc.Channel.unary_unary implementation.""" + return self._stub_for_method(method) + + def unary_stream( + self, method, + request_serializer=None, response_deserializer=None): + """grpc.Channel.unary_stream implementation.""" + return self._stub_for_method(method) + + def stream_unary( + self, method, + request_serializer=None, response_deserializer=None): + """grpc.Channel.stream_unary implementation.""" + return self._stub_for_method(method) + + def stream_stream( + self, method, + request_serializer=None, response_deserializer=None): + """grpc.Channel.stream_stream implementation.""" + return self._stub_for_method(method) + + def subscribe(self, callback, try_to_connect=False): + """grpc.Channel.subscribe implementation.""" + pass + + def unsubscribe(self, callback): + """grpc.Channel.unsubscribe implementation.""" + pass + + def close(self): + """grpc.Channel.close implementation.""" + pass diff --git a/stream/clients/python/bookkeeper/common/protobuf_helpers.py b/stream/clients/python/bookkeeper/common/protobuf_helpers.py new file mode 100644 index 00000000000..fcba501b97b --- /dev/null +++ b/stream/clients/python/bookkeeper/common/protobuf_helpers.py @@ -0,0 +1,321 @@ +# Licensed 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. + +"""Helpers for :mod:`protobuf`.""" + +import collections +import copy +import inspect + +from google.protobuf import field_mask_pb2 +from google.protobuf import message +from google.protobuf import wrappers_pb2 + +_SENTINEL = object() +_WRAPPER_TYPES = ( + wrappers_pb2.BoolValue, + wrappers_pb2.BytesValue, + wrappers_pb2.DoubleValue, + wrappers_pb2.FloatValue, + wrappers_pb2.Int32Value, + wrappers_pb2.Int64Value, + wrappers_pb2.StringValue, + wrappers_pb2.UInt32Value, + wrappers_pb2.UInt64Value, +) + + +def from_any_pb(pb_type, any_pb): + """Converts an ``Any`` protobuf to the specified message type. + Args: + pb_type (type): the type of the message that any_pb stores an instance + of. + any_pb (google.protobuf.any_pb2.Any): the object to be converted. + Returns: + pb_type: An instance of the pb_type message. + Raises: + TypeError: if the message could not be converted. + """ + msg = pb_type() + if not any_pb.Unpack(msg): + raise TypeError( + 'Could not convert {} to {}'.format( + any_pb.__class__.__name__, pb_type.__name__)) + + return msg + + +def check_oneof(**kwargs): + """Raise ValueError if more than one keyword argument is not ``None``. + Args: + kwargs (dict): The keyword arguments sent to the function. + Raises: + ValueError: If more than one entry in ``kwargs`` is not ``None``. + """ + # Sanity check: If no keyword arguments were sent, this is fine. + if not kwargs: + return + + not_nones = [val for val in kwargs.values() if val is not None] + if len(not_nones) > 1: + raise ValueError('Only one of {fields} should be set.'.format( + fields=', '.join(sorted(kwargs.keys())), + )) + + +def get_messages(module): + """Discovers all protobuf Message classes in a given import module. + Args: + module (module): A Python module; :func:`dir` will be run against this + module to find Message subclasses. + Returns: + dict[str, google.protobuf.message.Message]: A dictionary with the + Message class names as keys, and the Message subclasses themselves + as values. + """ + answer = collections.OrderedDict() + for name in dir(module): + candidate = getattr(module, name) + if (inspect.isclass(candidate) and + issubclass(candidate, message.Message)): + answer[name] = candidate + return answer + + +def _resolve_subkeys(key, separator='.'): + """Resolve a potentially nested key. + If the key contains the ``separator`` (e.g. ``.``) then the key will be + split on the first instance of the subkey:: + >>> _resolve_subkeys('a.b.c') + ('a', 'b.c') + >>> _resolve_subkeys('d|e|f', separator='|') + ('d', 'e|f') + If not, the subkey will be :data:`None`:: + >>> _resolve_subkeys('foo') + ('foo', None) + Args: + key (str): A string that may or may not contain the separator. + separator (str): The namespace separator. Defaults to `.`. + Returns: + Tuple[str, str]: The key and subkey(s). + """ + parts = key.split(separator, 1) + + if len(parts) > 1: + return parts + else: + return parts[0], None + + +def get(msg_or_dict, key, default=_SENTINEL): + """Retrieve a key's value from a protobuf Message or dictionary. + Args: + mdg_or_dict (Union[~google.protobuf.message.Message, Mapping]): the + object. + key (str): The key to retrieve from the object. + default (Any): If the key is not present on the object, and a default + is set, returns that default instead. A type-appropriate falsy + default is generally recommended, as protobuf messages almost + always have default values for unset values and it is not always + possible to tell the difference between a falsy value and an + unset one. If no default is set then :class:`KeyError` will be + raised if the key is not present in the object. + Returns: + Any: The return value from the underlying Message or dict. + Raises: + KeyError: If the key is not found. Note that, for unset values, + messages and dictionaries may not have consistent behavior. + TypeError: If ``msg_or_dict`` is not a Message or Mapping. + """ + # We may need to get a nested key. Resolve this. + key, subkey = _resolve_subkeys(key) + + # Attempt to get the value from the two types of objects we know about. + # If we get something else, complain. + if isinstance(msg_or_dict, message.Message): + answer = getattr(msg_or_dict, key, default) + elif isinstance(msg_or_dict, collections.Mapping): + answer = msg_or_dict.get(key, default) + else: + raise TypeError( + 'get() expected a dict or protobuf message, got {!r}.'.format( + type(msg_or_dict))) + + # If the object we got back is our sentinel, raise KeyError; this is + # a "not found" case. + if answer is _SENTINEL: + raise KeyError(key) + + # If a subkey exists, call this method recursively against the answer. + if subkey is not None and answer is not default: + return get(answer, subkey, default=default) + + return answer + + +def _set_field_on_message(msg, key, value): + """Set helper for protobuf Messages.""" + # Attempt to set the value on the types of objects we know how to deal + # with. + if isinstance(value, (collections.MutableSequence, tuple)): + # Clear the existing repeated protobuf message of any elements + # currently inside it. + while getattr(msg, key): + getattr(msg, key).pop() + + # Write our new elements to the repeated field. + for item in value: + if isinstance(item, collections.Mapping): + getattr(msg, key).add(**item) + else: + # protobuf's RepeatedCompositeContainer doesn't support + # append. + getattr(msg, key).extend([item]) + elif isinstance(value, collections.Mapping): + # Assign the dictionary values to the protobuf message. + for item_key, item_value in value.items(): + set(getattr(msg, key), item_key, item_value) + elif isinstance(value, message.Message): + getattr(msg, key).CopyFrom(value) + else: + setattr(msg, key, value) + + +def set(msg_or_dict, key, value): + """Set a key's value on a protobuf Message or dictionary. + Args: + msg_or_dict (Union[~google.protobuf.message.Message, Mapping]): the + object. + key (str): The key to set. + value (Any): The value to set. + Raises: + TypeError: If ``msg_or_dict`` is not a Message or dictionary. + """ + # Sanity check: Is our target object valid? + if (not isinstance(msg_or_dict, + (collections.MutableMapping, message.Message))): + raise TypeError( + 'set() expected a dict or protobuf message, got {!r}.'.format( + type(msg_or_dict))) + + # We may be setting a nested key. Resolve this. + basekey, subkey = _resolve_subkeys(key) + + # If a subkey exists, then get that object and call this method + # recursively against it using the subkey. + if subkey is not None: + if isinstance(msg_or_dict, collections.MutableMapping): + msg_or_dict.setdefault(basekey, {}) + set(get(msg_or_dict, basekey), subkey, value) + return + + if isinstance(msg_or_dict, collections.MutableMapping): + msg_or_dict[key] = value + else: + _set_field_on_message(msg_or_dict, key, value) + + +def setdefault(msg_or_dict, key, value): + """Set the key on a protobuf Message or dictionary to a given value if the + current value is falsy. + Because protobuf Messages do not distinguish between unset values and + falsy ones particularly well (by design), this method treats any falsy + value (e.g. 0, empty list) as a target to be overwritten, on both Messages + and dictionaries. + Args: + msg_or_dict (Union[~google.protobuf.message.Message, Mapping]): the + object. + key (str): The key on the object in question. + value (Any): The value to set. + Raises: + TypeError: If ``msg_or_dict`` is not a Message or dictionary. + """ + if not get(msg_or_dict, key, default=None): + set(msg_or_dict, key, value) + + +def field_mask(original, modified): + """Create a field mask by comparing two messages. + Args: + original (~google.protobuf.message.Message): the original message. + If set to None, this field will be interpretted as an empty + message. + modified (~google.protobuf.message.Message): the modified message. + If set to None, this field will be interpretted as an empty + message. + Returns: + google.protobuf.field_mask_pb2.FieldMask: field mask that contains + the list of field names that have different values between the two + messages. If the messages are equivalent, then the field mask is empty. + Raises: + ValueError: If the ``original`` or ``modified`` are not the same type. + """ + if original is None and modified is None: + return field_mask_pb2.FieldMask() + + if original is None and modified is not None: + original = copy.deepcopy(modified) + original.Clear() + + if modified is None and original is not None: + modified = copy.deepcopy(original) + modified.Clear() + + if type(original) != type(modified): + raise ValueError( + 'expected that both original and modified should be of the ' + 'same type, received "{!r}" and "{!r}".'. + format(type(original), type(modified))) + + return field_mask_pb2.FieldMask( + paths=_field_mask_helper(original, modified)) + + +def _field_mask_helper(original, modified, current=''): + answer = [] + + for name in original.DESCRIPTOR.fields_by_name: + field_path = _get_path(current, name) + + original_val = getattr(original, name) + modified_val = getattr(modified, name) + + if _is_message(original_val) or _is_message(modified_val): + if original_val != modified_val: + # Wrapper types do not need to include the .value part of the + # path. + if _is_wrapper(original_val) or _is_wrapper(modified_val): + answer.append(field_path) + elif not modified_val.ListFields(): + answer.append(field_path) + else: + answer.extend(_field_mask_helper(original_val, + modified_val, field_path)) + else: + if original_val != modified_val: + answer.append(field_path) + + return answer + + +def _get_path(current, name): + if not current: + return name + return '%s.%s' % (current, name) + + +def _is_message(value): + return isinstance(value, message.Message) + + +def _is_wrapper(value): + return type(value) in _WRAPPER_TYPES diff --git a/stream/clients/python/bookkeeper/common/retry.py b/stream/clients/python/bookkeeper/common/retry.py new file mode 100644 index 00000000000..f75905a02f7 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/retry.py @@ -0,0 +1,282 @@ +# Licensed 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. + +"""Helpers for retrying functions with exponential back-off. +The :class:`Retry` decorator can be used to retry functions that raise +exceptions using exponential backoff. Because a exponential sleep algorithm is +used, the retry is limited by a `deadline`. The deadline is the maxmimum amount +of time a method can block. This is used instead of total number of retries +because it is difficult to ascertain the amount of time a function can block +when using total number of retries and exponential backoff. +By default, this decorator will retry transient +API errors (see :func:`if_transient_error`). For example: +.. code-block:: python + @retry.Retry() + def call_flaky_rpc(): + return client.flaky_rpc() + # Will retry flaky_rpc() if it raises transient API errors. + result = call_flaky_rpc() +You can pass a custom predicate to retry on different exceptions, such as +waiting for an eventually consistent item to be available: +.. code-block:: python + @retry.Retry(predicate=if_exception_type(exceptions.NotFound)) + def check_if_exists(): + return client.does_thing_exist() + is_available = check_if_exists() +Some client library methods apply retry automatically. These methods can accept +a ``retry`` parameter that allows you to configure the behavior: +.. code-block:: python + my_retry = retry.Retry(deadline=60) + result = client.some_method(retry=my_retry) +""" + +from __future__ import unicode_literals + +import datetime +import functools +import logging +import random +import time + +import six + +from bookkeeper.common import datetime_helpers +from bookkeeper.common import exceptions +from bookkeeper.common import general_helpers + +_LOGGER = logging.getLogger(__name__) +_DEFAULT_INITIAL_DELAY = 1.0 # seconds +_DEFAULT_MAXIMUM_DELAY = 60.0 # seconds +_DEFAULT_DELAY_MULTIPLIER = 2.0 +_DEFAULT_DEADLINE = 60.0 * 2.0 # seconds + + +def if_exception_type(*exception_types): + """Creates a predicate to check if the exception is of a given type. + Args: + exception_types (Sequence[:func:`type`]): The exception types to check + for. + Returns: + Callable[Exception]: A predicate that returns True if the provided + exception is of the given type(s). + """ + def if_exception_type_predicate(exception): + """Bound predicate for checking an exception type.""" + return isinstance(exception, exception_types) + return if_exception_type_predicate + + +# pylint: disable=invalid-name +# Pylint sees this as a constant, but it is also an alias that should be +# considered a function. +if_transient_error = if_exception_type(( + exceptions.InternalServerError)) +"""A predicate that checks if an exception is a transient API error. +The following server errors are considered transient: +- :class:`bookkeeper.common.exceptions.InternalServerError` - gRPC + ``INTERNAL(13)``, bk ``INTERNAL_SERVER_ERROR`` and its subclasses. +""" +# pylint: enable=invalid-name + + +def exponential_sleep_generator( + initial, maximum, multiplier=_DEFAULT_DELAY_MULTIPLIER): + """Generates sleep intervals based on the exponential back-off algorithm. + This implements the `Truncated Exponential Back-off`_ algorithm. + .. _Truncated Exponential Back-off: + https://cloud.google.com/storage/docs/exponential-backoff + Args: + initial (float): The minimum about of time to delay. This must + be greater than 0. + maximum (float): The maximum about of time to delay. + multiplier (float): The multiplier applied to the delay. + Yields: + float: successive sleep intervals. + """ + delay = initial + while True: + # Introduce jitter by yielding a delay that is uniformly distributed + # to average out to the delay time. + yield min(random.uniform(0.0, delay * 2.0), maximum) + delay = delay * multiplier + + +def retry_target(target, predicate, sleep_generator, deadline, on_error=None): + """Call a function and retry if it fails. + This is the lowest-level retry helper. Generally, you'll use the + higher-level retry helper :class:`Retry`. + Args: + target(Callable): The function to call and retry. This must be a + nullary function - apply arguments with `functools.partial`. + predicate (Callable[Exception]): A callable used to determine if an + exception raised by the target should be considered retryable. + It should return True to retry or False otherwise. + sleep_generator (Iterable[float]): An infinite iterator that determines + how long to sleep between retries. + deadline (float): How long to keep retrying the target. + on_error (Callable): A function to call while processing a retryable + exception. Any error raised by this function will *not* be + caught. + Returns: + Any: the return value of the target function. + Raises: + bookkeeper.common.RetryError: If the deadline is exceeded while + retrying. + ValueError: If the sleep generator stops yielding values. + Exception: If the target raises a method that isn't retryable. + """ + if deadline is not None: + deadline_datetime = ( + datetime_helpers.utcnow() + datetime.timedelta(seconds=deadline)) + else: + deadline_datetime = None + + last_exc = None + + for sleep in sleep_generator: + try: + return target() + + # pylint: disable=broad-except + # This function explicitly must deal with broad exceptions. + except Exception as exc: + if not predicate(exc): + raise + last_exc = exc + if on_error is not None: + on_error(exc) + + now = datetime_helpers.utcnow() + if deadline_datetime is not None and deadline_datetime < now: + six.raise_from( + exceptions.RetryError( + 'Deadline of {:.1f}s exceeded while calling {}'.format( + deadline, target), + last_exc), + last_exc) + + _LOGGER.debug('Retrying due to {}, sleeping {:.1f}s ...'.format( + last_exc, sleep)) + time.sleep(sleep) + + raise ValueError('Sleep generator stopped yielding sleep values.') + + +@six.python_2_unicode_compatible +class Retry(object): + """Exponential retry decorator. + This class is a decorator used to add exponential back-off retry behavior + to an RPC call. + Although the default behavior is to retry transient API errors, a + different predicate can be provided to retry other exceptions. + Args: + predicate (Callable[Exception]): A callable that should return ``True`` + if the given exception is retryable. + initial (float): The minimum about of time to delay in seconds. This + must be greater than 0. + maximum (float): The maximum about of time to delay in seconds. + multiplier (float): The multiplier applied to the delay. + deadline (float): How long to keep retrying in seconds. + """ + def __init__( + self, + predicate=if_transient_error, + initial=_DEFAULT_INITIAL_DELAY, + maximum=_DEFAULT_MAXIMUM_DELAY, + multiplier=_DEFAULT_DELAY_MULTIPLIER, + deadline=_DEFAULT_DEADLINE): + self._predicate = predicate + self._initial = initial + self._multiplier = multiplier + self._maximum = maximum + self._deadline = deadline + + def __call__(self, func, on_error=None): + """Wrap a callable with retry behavior. + Args: + func (Callable): The callable to add retry behavior to. + on_error (Callable): A function to call while processing a + retryable exception. Any error raised by this function will + *not* be caught. + Returns: + Callable: A callable that will invoke ``func`` with retry + behavior. + """ + @general_helpers.wraps(func) + def retry_wrapped_func(*args, **kwargs): + """A wrapper that calls target function with retry.""" + target = functools.partial(func, *args, **kwargs) + sleep_generator = exponential_sleep_generator( + self._initial, self._maximum, multiplier=self._multiplier) + return retry_target( + target, + self._predicate, + sleep_generator, + self._deadline, + on_error=on_error, + ) + + return retry_wrapped_func + + def with_deadline(self, deadline): + """Return a copy of this retry with the given deadline. + Args: + deadline (float): How long to keep retrying. + Returns: + Retry: A new retry instance with the given deadline. + """ + return Retry( + predicate=self._predicate, + initial=self._initial, + maximum=self._maximum, + multiplier=self._multiplier, + deadline=deadline) + + def with_predicate(self, predicate): + """Return a copy of this retry with the given predicate. + Args: + predicate (Callable[Exception]): A callable that should return + ``True`` if the given exception is retryable. + Returns: + Retry: A new retry instance with the given predicate. + """ + return Retry( + predicate=predicate, + initial=self._initial, + maximum=self._maximum, + multiplier=self._multiplier, + deadline=self._deadline) + + def with_delay( + self, initial=None, maximum=None, multiplier=None): + """Return a copy of this retry with the given delay options. + Args: + initial (float): The minimum about of time to delay. This must + be greater than 0. + maximum (float): The maximum about of time to delay. + multiplier (float): The multiplier applied to the delay. + Returns: + Retry: A new retry instance with the given predicate. + """ + return Retry( + predicate=self._predicate, + initial=initial if initial is not None else self._initial, + maximum=maximum if maximum is not None else self._maximum, + multiplier=multiplier if maximum is not None else self._multiplier, + deadline=self._deadline) + + def __str__(self): + return ( + ''.format( + self._predicate, self._initial, self._maximum, + self._multiplier, self._deadline)) diff --git a/stream/clients/python/bookkeeper/common/service_uri.py b/stream/clients/python/bookkeeper/common/service_uri.py new file mode 100644 index 00000000000..ee8cd93070d --- /dev/null +++ b/stream/clients/python/bookkeeper/common/service_uri.py @@ -0,0 +1,31 @@ +# Licensed 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. + +import sys + +if sys.version_info[0] < 3: + USE_PYTHON3 = False + from urlparse import urlparse +else: + USE_PYTHON3 = True + from urllib.parse import urlparse + + +class ServiceURI(object): + + def __init__(self, service_uri): + self.uri = urlparse(service_uri) + self.service_name = self.uri.scheme + self.service_user = self.uri.username + self.service_path = self.uri.path + self.service_location = self.uri.netloc + self.service_hosts = self.uri.netloc.split(',') diff --git a/stream/clients/python/bookkeeper/kv/__init__.py b/stream/clients/python/bookkeeper/kv/__init__.py new file mode 100644 index 00000000000..459fe518f72 --- /dev/null +++ b/stream/clients/python/bookkeeper/kv/__init__.py @@ -0,0 +1,19 @@ +# Licensed 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. + +from __future__ import absolute_import + +from bookkeeper.kv.client import Client + +__all__ = ( + 'Client', +) diff --git a/stream/clients/python/bookkeeper/kv/client.py b/stream/clients/python/bookkeeper/kv/client.py new file mode 100644 index 00000000000..314fab51c3d --- /dev/null +++ b/stream/clients/python/bookkeeper/kv/client.py @@ -0,0 +1,60 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +from __future__ import absolute_import + +import grpc +import logging +import pkg_resources + +from bookkeeper import types +from bookkeeper.common.service_uri import ServiceURI +from bookkeeper.proto.storage_pb2_grpc import RootRangeServiceStub + +__version__ = pkg_resources.get_distribution('bookkeeper').version +__logger__ = logging.getLogger("bookkeeper.kv.Client") + + +class Client(object): + """A key/value client for Apache BookKeeper Table Service. + + This creates an object that is capable of interacting with bookkeeper + table service. Generally, you can instantiate this client with no + arguments, and you get sensible defaults. + + Args: + storage_client_settings (~bookkeeper.types.StorageClientSettings): The + settings for bookkeeper storage client . + kwargs (dict): Any additional arguments provided are sent as keyword + arguments to the underlying grpc client. + """ + def __init__(self, storage_client_settings=(), **kwargs): + # init the storage client settings + self.storage_client_settings =\ + types.StorageClientSettings(*storage_client_settings) + __logger__.info("Creating an admin client to cluster '%s'", + self.storage_client_settings.service_uri) + + service_uri = ServiceURI(self.storage_client_settings.service_uri) + assert service_uri.service_name.lower() == 'bk' + + # create channel + self.channel = grpc.insecure_channel( + target=service_uri.service_location + ) + __logger__.info("Successfully created an admin client to cluster '%s'", + self.storage_client_settings.service_uri) + + # create the rpc stub + self.root_range = RootRangeServiceStub(channel=self.channel) diff --git a/stream/clients/python/bookkeeper/kv/exceptions.py b/stream/clients/python/bookkeeper/kv/exceptions.py new file mode 100644 index 00000000000..3bc01a8588f --- /dev/null +++ b/stream/clients/python/bookkeeper/kv/exceptions.py @@ -0,0 +1,19 @@ +# Licensed 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. + +from __future__ import absolute_import + +from concurrent.futures import TimeoutError + +__all__ = ( + 'TimeoutError', +) diff --git a/stream/clients/python/bookkeeper/kv/futures.py b/stream/clients/python/bookkeeper/kv/futures.py new file mode 100644 index 00000000000..ae626a7e98c --- /dev/null +++ b/stream/clients/python/bookkeeper/kv/futures.py @@ -0,0 +1,170 @@ +# Licensed 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. + +from __future__ import absolute_import + +import threading +import uuid + +import bookkeeper.common.future +from bookkeeper.kv import exceptions + + +class Future(bookkeeper.common.future.Future): + """Encapsulation of the asynchronous execution of an action. + This object is returned from asychronous bookkeeper calls, and is the + interface to determine the status of those calls. + This object should not be created directly, but is returned by other + methods in this library. + Args: + completed (Optional[Any]): An event, with the same interface as + :class:`threading.Event`. This is provided so that callers + with different concurrency models (e.g. ``threading`` or + ``multiprocessing``) can supply an event that is compatible + with that model. The ``wait()`` and ``set()`` methods will be + used. If this argument is not provided, then a new + :class:`threading.Event` will be created and used. + """ + + # This could be a sentinel object or None, but the sentinel object's ID + # can change if the process is forked, and None has the possibility of + # actually being a result. + _SENTINEL = uuid.uuid4() + + def __init__(self, completed=None): + self._result = self._SENTINEL + self._exception = self._SENTINEL + self._callbacks = [] + if completed is None: + completed = threading.Event() + self._completed = completed + + def cancel(self): + """Actions in bookkeeper generally may not be canceled. + This method always returns False. + """ + return False + + def cancelled(self): + """Actions in bookkeeper generally may not be canceled. + This method always returns False. + """ + return False + + def running(self): + """Actions in bookkeeper generally may not be canceled. + Returns: + bool: ``True`` if this method has not yet completed, or + ``False`` if it has completed. + """ + if self.done(): + return False + return True + + def done(self): + """Return True the future is done, False otherwise. + This still returns True in failure cases; checking :meth:`result` or + :meth:`exception` is the canonical way to assess success or failure. + """ + return (self._exception != self._SENTINEL or + self._result != self._SENTINEL) + + def result(self, timeout=None): + """Return the message ID, or raise an exception. + This blocks until the message has successfully been published, and + returns the message ID. + Args: + timeout (Union[int, float]): The number of seconds before this call + times out and raises TimeoutError. + Returns: + str: The message ID. + Raises: + ~.pubsub_v1.TimeoutError: If the request times out. + Exception: For undefined exceptions in the underlying + call execution. + """ + # Attempt to get the exception if there is one. + # If there is not one, then we know everything worked, and we can + # return an appropriate value. + err = self.exception(timeout=timeout) + if err is None: + return self._result + raise err + + def exception(self, timeout=None): + """Return the exception raised by the call, if any. + This blocks until the message has successfully been published, and + returns the exception. If the call succeeded, return None. + Args: + timeout (Union[int, float]): The number of seconds before this call + times out and raises TimeoutError. + Raises: + TimeoutError: If the request times out. + Returns: + Exception: The exception raised by the call, if any. + """ + # Wait until the future is done. + if not self._completed.wait(timeout=timeout): + raise exceptions.TimeoutError('Timed out waiting for result.') + + # If the batch completed successfully, this should return None. + if self._result != self._SENTINEL: + return None + + # Okay, this batch had an error; this should return it. + return self._exception + + def add_done_callback(self, fn): + """Attach the provided callable to the future. + The provided function is called, with this future as its only argument, + when the future finishes running. + """ + if self.done(): + return fn(self) + self._callbacks.append(fn) + + def set_result(self, result): + """Set the result of the future to the provided result. + Args: + result (Any): The result + """ + # Sanity check: A future can only complete once. + if self.done(): + raise RuntimeError('set_result can only be called once.') + + # Set the result and trigger the future. + self._result = result + self._trigger() + + def set_exception(self, exception): + """Set the result of the future to the given exception. + Args: + exception (:exc:`Exception`): The exception raised. + """ + # Sanity check: A future can only complete once. + if self.done(): + raise RuntimeError('set_exception can only be called once.') + + # Set the exception and trigger the future. + self._exception = exception + self._trigger() + + def _trigger(self): + """Trigger all callbacks registered to this Future. + This method is called internally by the batch once the batch + completes. + Args: + message_id (str): The message ID, as a string. + """ + self._completed.set() + for callback in self._callbacks: + callback(self) diff --git a/stream/clients/python/bookkeeper/proto/cluster_pb2.py b/stream/clients/python/bookkeeper/proto/cluster_pb2.py new file mode 100644 index 00000000000..65f5d6dc12d --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/cluster_pb2.py @@ -0,0 +1,199 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: cluster.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +import bookkeeper.proto.common_pb2 as common__pb2 + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='cluster.proto', + package='bookkeeper.proto.cluster', + syntax='proto3', + serialized_options=_b('\n*org.apache.bookkeeper.stream.proto.clusterP\001'), + serialized_pb=_b('\n\rcluster.proto\x12\x18\x62ookkeeper.proto.cluster\x1a\x0c\x63ommon.proto\"*\n\x14ServerAssignmentData\x12\x12\n\ncontainers\x18\x01 \x03(\x03\"\xc6\x01\n\x15\x43lusterAssignmentData\x12M\n\x07servers\x18\x01 \x03(\x0b\x32<.bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry\x1a^\n\x0cServersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12=\n\x05value\x18\x02 \x01(\x0b\x32..bookkeeper.proto.cluster.ServerAssignmentData:\x02\x38\x01\"1\n\x0f\x43lusterMetadata\x12\x1e\n\x16num_storage_containers\x18\x01 \x01(\x03\x42.\n*org.apache.bookkeeper.stream.proto.clusterP\x01\x62\x06proto3') + , + dependencies=[common__pb2.DESCRIPTOR,]) + + + + +_SERVERASSIGNMENTDATA = _descriptor.Descriptor( + name='ServerAssignmentData', + full_name='bookkeeper.proto.cluster.ServerAssignmentData', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='containers', full_name='bookkeeper.proto.cluster.ServerAssignmentData.containers', index=0, + number=1, type=3, cpp_type=2, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=57, + serialized_end=99, +) + + +_CLUSTERASSIGNMENTDATA_SERVERSENTRY = _descriptor.Descriptor( + name='ServersEntry', + full_name='bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry.key', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='value', full_name='bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry.value', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=_b('8\001'), + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=206, + serialized_end=300, +) + +_CLUSTERASSIGNMENTDATA = _descriptor.Descriptor( + name='ClusterAssignmentData', + full_name='bookkeeper.proto.cluster.ClusterAssignmentData', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='servers', full_name='bookkeeper.proto.cluster.ClusterAssignmentData.servers', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[_CLUSTERASSIGNMENTDATA_SERVERSENTRY, ], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=102, + serialized_end=300, +) + + +_CLUSTERMETADATA = _descriptor.Descriptor( + name='ClusterMetadata', + full_name='bookkeeper.proto.cluster.ClusterMetadata', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='num_storage_containers', full_name='bookkeeper.proto.cluster.ClusterMetadata.num_storage_containers', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=302, + serialized_end=351, +) + +_CLUSTERASSIGNMENTDATA_SERVERSENTRY.fields_by_name['value'].message_type = _SERVERASSIGNMENTDATA +_CLUSTERASSIGNMENTDATA_SERVERSENTRY.containing_type = _CLUSTERASSIGNMENTDATA +_CLUSTERASSIGNMENTDATA.fields_by_name['servers'].message_type = _CLUSTERASSIGNMENTDATA_SERVERSENTRY +DESCRIPTOR.message_types_by_name['ServerAssignmentData'] = _SERVERASSIGNMENTDATA +DESCRIPTOR.message_types_by_name['ClusterAssignmentData'] = _CLUSTERASSIGNMENTDATA +DESCRIPTOR.message_types_by_name['ClusterMetadata'] = _CLUSTERMETADATA +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +ServerAssignmentData = _reflection.GeneratedProtocolMessageType('ServerAssignmentData', (_message.Message,), dict( + DESCRIPTOR = _SERVERASSIGNMENTDATA, + __module__ = 'cluster_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.cluster.ServerAssignmentData) + )) +_sym_db.RegisterMessage(ServerAssignmentData) + +ClusterAssignmentData = _reflection.GeneratedProtocolMessageType('ClusterAssignmentData', (_message.Message,), dict( + + ServersEntry = _reflection.GeneratedProtocolMessageType('ServersEntry', (_message.Message,), dict( + DESCRIPTOR = _CLUSTERASSIGNMENTDATA_SERVERSENTRY, + __module__ = 'cluster_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry) + )) + , + DESCRIPTOR = _CLUSTERASSIGNMENTDATA, + __module__ = 'cluster_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.cluster.ClusterAssignmentData) + )) +_sym_db.RegisterMessage(ClusterAssignmentData) +_sym_db.RegisterMessage(ClusterAssignmentData.ServersEntry) + +ClusterMetadata = _reflection.GeneratedProtocolMessageType('ClusterMetadata', (_message.Message,), dict( + DESCRIPTOR = _CLUSTERMETADATA, + __module__ = 'cluster_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.cluster.ClusterMetadata) + )) +_sym_db.RegisterMessage(ClusterMetadata) + + +DESCRIPTOR._options = None +_CLUSTERASSIGNMENTDATA_SERVERSENTRY._options = None +# @@protoc_insertion_point(module_scope) diff --git a/stream/clients/python/bookkeeper/proto/common_pb2.py b/stream/clients/python/bookkeeper/proto/common_pb2.py new file mode 100644 index 00000000000..84bb846730a --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/common_pb2.py @@ -0,0 +1,77 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: common.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='common.proto', + package='bookkeeper.proto.common', + syntax='proto3', + serialized_options=_b('\n)org.apache.bookkeeper.stream.proto.commonP\001'), + serialized_pb=_b('\n\x0c\x63ommon.proto\x12\x17\x62ookkeeper.proto.common\"*\n\x08\x45ndpoint\x12\x10\n\x08hostname\x18\x01 \x01(\t\x12\x0c\n\x04port\x18\x02 \x01(\x05\x42-\n)org.apache.bookkeeper.stream.proto.commonP\x01\x62\x06proto3') +) + + + + +_ENDPOINT = _descriptor.Descriptor( + name='Endpoint', + full_name='bookkeeper.proto.common.Endpoint', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='hostname', full_name='bookkeeper.proto.common.Endpoint.hostname', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='port', full_name='bookkeeper.proto.common.Endpoint.port', index=1, + number=2, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=41, + serialized_end=83, +) + +DESCRIPTOR.message_types_by_name['Endpoint'] = _ENDPOINT +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +Endpoint = _reflection.GeneratedProtocolMessageType('Endpoint', (_message.Message,), dict( + DESCRIPTOR = _ENDPOINT, + __module__ = 'common_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.common.Endpoint) + )) +_sym_db.RegisterMessage(Endpoint) + + +DESCRIPTOR._options = None +# @@protoc_insertion_point(module_scope) diff --git a/stream/clients/python/bookkeeper/proto/kv_pb2.py b/stream/clients/python/bookkeeper/proto/kv_pb2.py new file mode 100644 index 00000000000..51fedc3c7ae --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/kv_pb2.py @@ -0,0 +1,192 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: kv.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='kv.proto', + package='bookkeeper.proto.kv', + syntax='proto3', + serialized_options=_b('\n%org.apache.bookkeeper.stream.proto.kvP\001'), + serialized_pb=_b('\n\x08kv.proto\x12\x13\x62ookkeeper.proto.kv\"\x8f\x01\n\x08KeyValue\x12\x0b\n\x03key\x18\x01 \x01(\x0c\x12\x17\n\x0f\x63reate_revision\x18\x02 \x01(\x03\x12\x14\n\x0cmod_revision\x18\x03 \x01(\x03\x12\x0f\n\x07version\x18\x04 \x01(\x03\x12\r\n\x05value\x18\x05 \x01(\x0c\x12\x11\n\tis_number\x18\x06 \x01(\x08\x12\x14\n\x0cnumber_value\x18\x07 \x01(\x03\"\xb8\x01\n\x05\x45vent\x12\x32\n\x04type\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.kv.Event.EventType\x12)\n\x02kv\x18\x02 \x01(\x0b\x32\x1d.bookkeeper.proto.kv.KeyValue\x12.\n\x07prev_kv\x18\x03 \x01(\x0b\x32\x1d.bookkeeper.proto.kv.KeyValue\" \n\tEventType\x12\x07\n\x03PUT\x10\x00\x12\n\n\x06\x44\x45LETE\x10\x01\x42)\n%org.apache.bookkeeper.stream.proto.kvP\x01\x62\x06proto3') +) + + + +_EVENT_EVENTTYPE = _descriptor.EnumDescriptor( + name='EventType', + full_name='bookkeeper.proto.kv.Event.EventType', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='PUT', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='DELETE', index=1, number=1, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=332, + serialized_end=364, +) +_sym_db.RegisterEnumDescriptor(_EVENT_EVENTTYPE) + + +_KEYVALUE = _descriptor.Descriptor( + name='KeyValue', + full_name='bookkeeper.proto.kv.KeyValue', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='bookkeeper.proto.kv.KeyValue.key', index=0, + number=1, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='create_revision', full_name='bookkeeper.proto.kv.KeyValue.create_revision', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='mod_revision', full_name='bookkeeper.proto.kv.KeyValue.mod_revision', index=2, + number=3, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='version', full_name='bookkeeper.proto.kv.KeyValue.version', index=3, + number=4, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='value', full_name='bookkeeper.proto.kv.KeyValue.value', index=4, + number=5, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='is_number', full_name='bookkeeper.proto.kv.KeyValue.is_number', index=5, + number=6, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='number_value', full_name='bookkeeper.proto.kv.KeyValue.number_value', index=6, + number=7, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=34, + serialized_end=177, +) + + +_EVENT = _descriptor.Descriptor( + name='Event', + full_name='bookkeeper.proto.kv.Event', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='type', full_name='bookkeeper.proto.kv.Event.type', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='kv', full_name='bookkeeper.proto.kv.Event.kv', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='prev_kv', full_name='bookkeeper.proto.kv.Event.prev_kv', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + _EVENT_EVENTTYPE, + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=180, + serialized_end=364, +) + +_EVENT.fields_by_name['type'].enum_type = _EVENT_EVENTTYPE +_EVENT.fields_by_name['kv'].message_type = _KEYVALUE +_EVENT.fields_by_name['prev_kv'].message_type = _KEYVALUE +_EVENT_EVENTTYPE.containing_type = _EVENT +DESCRIPTOR.message_types_by_name['KeyValue'] = _KEYVALUE +DESCRIPTOR.message_types_by_name['Event'] = _EVENT +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +KeyValue = _reflection.GeneratedProtocolMessageType('KeyValue', (_message.Message,), dict( + DESCRIPTOR = _KEYVALUE, + __module__ = 'kv_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.KeyValue) + )) +_sym_db.RegisterMessage(KeyValue) + +Event = _reflection.GeneratedProtocolMessageType('Event', (_message.Message,), dict( + DESCRIPTOR = _EVENT, + __module__ = 'kv_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.Event) + )) +_sym_db.RegisterMessage(Event) + + +DESCRIPTOR._options = None +# @@protoc_insertion_point(module_scope) diff --git a/stream/clients/python/bookkeeper/proto/kv_rpc_pb2.py b/stream/clients/python/bookkeeper/proto/kv_rpc_pb2.py new file mode 100644 index 00000000000..2a3f7a9fe6d --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/kv_rpc_pb2.py @@ -0,0 +1,1232 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: kv_rpc.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +import bookkeeper.proto.kv_pb2 as kv__pb2 +import bookkeeper.proto.storage_pb2 as storage__pb2 + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='kv_rpc.proto', + package='bookkeeper.proto.kv.rpc', + syntax='proto3', + serialized_options=_b('\n)org.apache.bookkeeper.stream.proto.kv.rpcP\001'), + serialized_pb=_b('\n\x0ckv_rpc.proto\x12\x17\x62ookkeeper.proto.kv.rpc\x1a\x08kv.proto\x1a\rstorage.proto\"C\n\rRoutingHeader\x12\x11\n\tstream_id\x18\x01 \x01(\x03\x12\x10\n\x08range_id\x18\x02 \x01(\x03\x12\r\n\x05r_key\x18\x03 \x01(\x0c\"\x84\x01\n\x0eResponseHeader\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12>\n\x0erouting_header\x18\x63 \x01(\x0b\x32&.bookkeeper.proto.kv.rpc.RoutingHeader\"\xb2\x04\n\x0cRangeRequest\x12\x0b\n\x03key\x18\x01 \x01(\x0c\x12\x11\n\trange_end\x18\x02 \x01(\x0c\x12\r\n\x05limit\x18\x03 \x01(\x03\x12\x10\n\x08revision\x18\x04 \x01(\x03\x12\x43\n\nsort_order\x18\x05 \x01(\x0e\x32/.bookkeeper.proto.kv.rpc.RangeRequest.SortOrder\x12\x45\n\x0bsort_target\x18\x06 \x01(\x0e\x32\x30.bookkeeper.proto.kv.rpc.RangeRequest.SortTarget\x12\x14\n\x0cserializable\x18\x07 \x01(\x08\x12\x11\n\tkeys_only\x18\x08 \x01(\x08\x12\x12\n\ncount_only\x18\t \x01(\x08\x12\x18\n\x10min_mod_revision\x18\n \x01(\x03\x12\x18\n\x10max_mod_revision\x18\x0b \x01(\x03\x12\x1b\n\x13min_create_revision\x18\x0c \x01(\x03\x12\x1b\n\x13max_create_revision\x18\r \x01(\x03\x12\x36\n\x06header\x18\x63 \x01(\x0b\x32&.bookkeeper.proto.kv.rpc.RoutingHeader\".\n\tSortOrder\x12\x08\n\x04NONE\x10\x00\x12\n\n\x06\x41SCEND\x10\x01\x12\x0b\n\x07\x44\x45SCEND\x10\x02\"B\n\nSortTarget\x12\x07\n\x03KEY\x10\x00\x12\x0b\n\x07VERSION\x10\x01\x12\n\n\x06\x43REATE\x10\x02\x12\x07\n\x03MOD\x10\x03\x12\t\n\x05VALUE\x10\x04\"\x91\x01\n\rRangeResponse\x12\x37\n\x06header\x18\x01 \x01(\x0b\x32\'.bookkeeper.proto.kv.rpc.ResponseHeader\x12*\n\x03kvs\x18\x02 \x03(\x0b\x32\x1d.bookkeeper.proto.kv.KeyValue\x12\x0c\n\x04more\x18\x03 \x01(\x08\x12\r\n\x05\x63ount\x18\x04 \x01(\x03\"\xc6\x01\n\nPutRequest\x12\x0b\n\x03key\x18\x01 \x01(\x0c\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\r\n\x05lease\x18\x03 \x01(\x03\x12\x0f\n\x07prev_kv\x18\x04 \x01(\x08\x12\x14\n\x0cignore_value\x18\x05 \x01(\x08\x12\x14\n\x0cignore_lease\x18\x06 \x01(\x08\x12\x36\n\x06header\x18\x63 \x01(\x0b\x32&.bookkeeper.proto.kv.rpc.RoutingHeader\x12\x18\n\x10\x65xpected_version\x18\x64 \x01(\x03\"v\n\x0bPutResponse\x12\x37\n\x06header\x18\x01 \x01(\x0b\x32\'.bookkeeper.proto.kv.rpc.ResponseHeader\x12.\n\x07prev_kv\x18\x02 \x01(\x0b\x32\x1d.bookkeeper.proto.kv.KeyValue\"z\n\x10IncrementRequest\x12\x0b\n\x03key\x18\x01 \x01(\x0c\x12\x0e\n\x06\x61mount\x18\x02 \x01(\x03\x12\x11\n\tget_total\x18\x03 \x01(\x08\x12\x36\n\x06header\x18\x63 \x01(\x0b\x32&.bookkeeper.proto.kv.rpc.RoutingHeader\"b\n\x11IncrementResponse\x12\x37\n\x06header\x18\x01 \x01(\x0b\x32\'.bookkeeper.proto.kv.rpc.ResponseHeader\x12\x14\n\x0ctotal_amount\x18\x02 \x01(\x03\"}\n\x12\x44\x65leteRangeRequest\x12\x0b\n\x03key\x18\x01 \x01(\x0c\x12\x11\n\trange_end\x18\x02 \x01(\x0c\x12\x0f\n\x07prev_kv\x18\x03 \x01(\x08\x12\x36\n\x06header\x18\x63 \x01(\x0b\x32&.bookkeeper.proto.kv.rpc.RoutingHeader\"\x90\x01\n\x13\x44\x65leteRangeResponse\x12\x37\n\x06header\x18\x01 \x01(\x0b\x32\'.bookkeeper.proto.kv.rpc.ResponseHeader\x12\x0f\n\x07\x64\x65leted\x18\x02 \x01(\x03\x12/\n\x08prev_kvs\x18\x03 \x03(\x0b\x32\x1d.bookkeeper.proto.kv.KeyValue\"\xdf\x01\n\tRequestOp\x12>\n\rrequest_range\x18\x01 \x01(\x0b\x32%.bookkeeper.proto.kv.rpc.RangeRequestH\x00\x12:\n\x0brequest_put\x18\x02 \x01(\x0b\x32#.bookkeeper.proto.kv.rpc.PutRequestH\x00\x12K\n\x14request_delete_range\x18\x03 \x01(\x0b\x32+.bookkeeper.proto.kv.rpc.DeleteRangeRequestH\x00\x42\t\n\x07request\"\xe7\x01\n\nResponseOp\x12@\n\x0eresponse_range\x18\x01 \x01(\x0b\x32&.bookkeeper.proto.kv.rpc.RangeResponseH\x00\x12<\n\x0cresponse_put\x18\x02 \x01(\x0b\x32$.bookkeeper.proto.kv.rpc.PutResponseH\x00\x12M\n\x15response_delete_range\x18\x03 \x01(\x0b\x32,.bookkeeper.proto.kv.rpc.DeleteRangeResponseH\x00\x42\n\n\x08response\"\xfd\x02\n\x07\x43ompare\x12>\n\x06result\x18\x01 \x01(\x0e\x32..bookkeeper.proto.kv.rpc.Compare.CompareResult\x12>\n\x06target\x18\x02 \x01(\x0e\x32..bookkeeper.proto.kv.rpc.Compare.CompareTarget\x12\x0b\n\x03key\x18\x03 \x01(\x0c\x12\x11\n\x07version\x18\x04 \x01(\x03H\x00\x12\x19\n\x0f\x63reate_revision\x18\x05 \x01(\x03H\x00\x12\x16\n\x0cmod_revision\x18\x06 \x01(\x03H\x00\x12\x0f\n\x05value\x18\x07 \x01(\x0cH\x00\"@\n\rCompareResult\x12\t\n\x05\x45QUAL\x10\x00\x12\x0b\n\x07GREATER\x10\x01\x12\x08\n\x04LESS\x10\x02\x12\r\n\tNOT_EQUAL\x10\x03\"<\n\rCompareTarget\x12\x0b\n\x07VERSION\x10\x00\x12\n\n\x06\x43REATE\x10\x01\x12\x07\n\x03MOD\x10\x02\x12\t\n\x05VALUE\x10\x03\x42\x0e\n\x0ctarget_union\"\xe1\x01\n\nTxnRequest\x12\x31\n\x07\x63ompare\x18\x01 \x03(\x0b\x32 .bookkeeper.proto.kv.rpc.Compare\x12\x33\n\x07success\x18\x02 \x03(\x0b\x32\".bookkeeper.proto.kv.rpc.RequestOp\x12\x33\n\x07\x66\x61ilure\x18\x03 \x03(\x0b\x32\".bookkeeper.proto.kv.rpc.RequestOp\x12\x36\n\x06header\x18\x63 \x01(\x0b\x32&.bookkeeper.proto.kv.rpc.RoutingHeader\"\x91\x01\n\x0bTxnResponse\x12\x37\n\x06header\x18\x01 \x01(\x0b\x32\'.bookkeeper.proto.kv.rpc.ResponseHeader\x12\x11\n\tsucceeded\x18\x02 \x01(\x08\x12\x36\n\tresponses\x18\x03 \x03(\x0b\x32#.bookkeeper.proto.kv.rpc.ResponseOp2\xdd\x03\n\x0cTableService\x12X\n\x05Range\x12%.bookkeeper.proto.kv.rpc.RangeRequest\x1a&.bookkeeper.proto.kv.rpc.RangeResponse\"\x00\x12R\n\x03Put\x12#.bookkeeper.proto.kv.rpc.PutRequest\x1a$.bookkeeper.proto.kv.rpc.PutResponse\"\x00\x12\x65\n\x06\x44\x65lete\x12+.bookkeeper.proto.kv.rpc.DeleteRangeRequest\x1a,.bookkeeper.proto.kv.rpc.DeleteRangeResponse\"\x00\x12R\n\x03Txn\x12#.bookkeeper.proto.kv.rpc.TxnRequest\x1a$.bookkeeper.proto.kv.rpc.TxnResponse\"\x00\x12\x64\n\tIncrement\x12).bookkeeper.proto.kv.rpc.IncrementRequest\x1a*.bookkeeper.proto.kv.rpc.IncrementResponse\"\x00\x42-\n)org.apache.bookkeeper.stream.proto.kv.rpcP\x01\x62\x06proto3') + , + dependencies=[kv__pb2.DESCRIPTOR,storage__pb2.DESCRIPTOR,]) + + + +_RANGEREQUEST_SORTORDER = _descriptor.EnumDescriptor( + name='SortOrder', + full_name='bookkeeper.proto.kv.rpc.RangeRequest.SortOrder', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='NONE', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='ASCEND', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='DESCEND', index=2, number=2, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=719, + serialized_end=765, +) +_sym_db.RegisterEnumDescriptor(_RANGEREQUEST_SORTORDER) + +_RANGEREQUEST_SORTTARGET = _descriptor.EnumDescriptor( + name='SortTarget', + full_name='bookkeeper.proto.kv.rpc.RangeRequest.SortTarget', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='KEY', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='VERSION', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='CREATE', index=2, number=2, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='MOD', index=3, number=3, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='VALUE', index=4, number=4, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=767, + serialized_end=833, +) +_sym_db.RegisterEnumDescriptor(_RANGEREQUEST_SORTTARGET) + +_COMPARE_COMPARERESULT = _descriptor.EnumDescriptor( + name='CompareResult', + full_name='bookkeeper.proto.kv.rpc.Compare.CompareResult', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='EQUAL', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='GREATER', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='LESS', index=2, number=2, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='NOT_EQUAL', index=3, number=3, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=2502, + serialized_end=2566, +) +_sym_db.RegisterEnumDescriptor(_COMPARE_COMPARERESULT) + +_COMPARE_COMPARETARGET = _descriptor.EnumDescriptor( + name='CompareTarget', + full_name='bookkeeper.proto.kv.rpc.Compare.CompareTarget', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='VERSION', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='CREATE', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='MOD', index=2, number=2, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='VALUE', index=3, number=3, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=2568, + serialized_end=2628, +) +_sym_db.RegisterEnumDescriptor(_COMPARE_COMPARETARGET) + + +_ROUTINGHEADER = _descriptor.Descriptor( + name='RoutingHeader', + full_name='bookkeeper.proto.kv.rpc.RoutingHeader', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='stream_id', full_name='bookkeeper.proto.kv.rpc.RoutingHeader.stream_id', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='range_id', full_name='bookkeeper.proto.kv.rpc.RoutingHeader.range_id', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='r_key', full_name='bookkeeper.proto.kv.rpc.RoutingHeader.r_key', index=2, + number=3, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=66, + serialized_end=133, +) + + +_RESPONSEHEADER = _descriptor.Descriptor( + name='ResponseHeader', + full_name='bookkeeper.proto.kv.rpc.ResponseHeader', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='code', full_name='bookkeeper.proto.kv.rpc.ResponseHeader.code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='routing_header', full_name='bookkeeper.proto.kv.rpc.ResponseHeader.routing_header', index=1, + number=99, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=136, + serialized_end=268, +) + + +_RANGEREQUEST = _descriptor.Descriptor( + name='RangeRequest', + full_name='bookkeeper.proto.kv.rpc.RangeRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='bookkeeper.proto.kv.rpc.RangeRequest.key', index=0, + number=1, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='range_end', full_name='bookkeeper.proto.kv.rpc.RangeRequest.range_end', index=1, + number=2, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='limit', full_name='bookkeeper.proto.kv.rpc.RangeRequest.limit', index=2, + number=3, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.revision', index=3, + number=4, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='sort_order', full_name='bookkeeper.proto.kv.rpc.RangeRequest.sort_order', index=4, + number=5, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='sort_target', full_name='bookkeeper.proto.kv.rpc.RangeRequest.sort_target', index=5, + number=6, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='serializable', full_name='bookkeeper.proto.kv.rpc.RangeRequest.serializable', index=6, + number=7, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='keys_only', full_name='bookkeeper.proto.kv.rpc.RangeRequest.keys_only', index=7, + number=8, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='count_only', full_name='bookkeeper.proto.kv.rpc.RangeRequest.count_only', index=8, + number=9, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='min_mod_revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.min_mod_revision', index=9, + number=10, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='max_mod_revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.max_mod_revision', index=10, + number=11, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='min_create_revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.min_create_revision', index=11, + number=12, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='max_create_revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.max_create_revision', index=12, + number=13, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.RangeRequest.header', index=13, + number=99, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + _RANGEREQUEST_SORTORDER, + _RANGEREQUEST_SORTTARGET, + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=271, + serialized_end=833, +) + + +_RANGERESPONSE = _descriptor.Descriptor( + name='RangeResponse', + full_name='bookkeeper.proto.kv.rpc.RangeResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.RangeResponse.header', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='kvs', full_name='bookkeeper.proto.kv.rpc.RangeResponse.kvs', index=1, + number=2, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='more', full_name='bookkeeper.proto.kv.rpc.RangeResponse.more', index=2, + number=3, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='count', full_name='bookkeeper.proto.kv.rpc.RangeResponse.count', index=3, + number=4, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=836, + serialized_end=981, +) + + +_PUTREQUEST = _descriptor.Descriptor( + name='PutRequest', + full_name='bookkeeper.proto.kv.rpc.PutRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='bookkeeper.proto.kv.rpc.PutRequest.key', index=0, + number=1, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='value', full_name='bookkeeper.proto.kv.rpc.PutRequest.value', index=1, + number=2, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='lease', full_name='bookkeeper.proto.kv.rpc.PutRequest.lease', index=2, + number=3, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='prev_kv', full_name='bookkeeper.proto.kv.rpc.PutRequest.prev_kv', index=3, + number=4, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='ignore_value', full_name='bookkeeper.proto.kv.rpc.PutRequest.ignore_value', index=4, + number=5, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='ignore_lease', full_name='bookkeeper.proto.kv.rpc.PutRequest.ignore_lease', index=5, + number=6, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.PutRequest.header', index=6, + number=99, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='expected_version', full_name='bookkeeper.proto.kv.rpc.PutRequest.expected_version', index=7, + number=100, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=984, + serialized_end=1182, +) + + +_PUTRESPONSE = _descriptor.Descriptor( + name='PutResponse', + full_name='bookkeeper.proto.kv.rpc.PutResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.PutResponse.header', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='prev_kv', full_name='bookkeeper.proto.kv.rpc.PutResponse.prev_kv', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1184, + serialized_end=1302, +) + + +_INCREMENTREQUEST = _descriptor.Descriptor( + name='IncrementRequest', + full_name='bookkeeper.proto.kv.rpc.IncrementRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='bookkeeper.proto.kv.rpc.IncrementRequest.key', index=0, + number=1, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='amount', full_name='bookkeeper.proto.kv.rpc.IncrementRequest.amount', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='get_total', full_name='bookkeeper.proto.kv.rpc.IncrementRequest.get_total', index=2, + number=3, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.IncrementRequest.header', index=3, + number=99, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1304, + serialized_end=1426, +) + + +_INCREMENTRESPONSE = _descriptor.Descriptor( + name='IncrementResponse', + full_name='bookkeeper.proto.kv.rpc.IncrementResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.IncrementResponse.header', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='total_amount', full_name='bookkeeper.proto.kv.rpc.IncrementResponse.total_amount', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1428, + serialized_end=1526, +) + + +_DELETERANGEREQUEST = _descriptor.Descriptor( + name='DeleteRangeRequest', + full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest.key', index=0, + number=1, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='range_end', full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest.range_end', index=1, + number=2, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='prev_kv', full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest.prev_kv', index=2, + number=3, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest.header', index=3, + number=99, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1528, + serialized_end=1653, +) + + +_DELETERANGERESPONSE = _descriptor.Descriptor( + name='DeleteRangeResponse', + full_name='bookkeeper.proto.kv.rpc.DeleteRangeResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.DeleteRangeResponse.header', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='deleted', full_name='bookkeeper.proto.kv.rpc.DeleteRangeResponse.deleted', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='prev_kvs', full_name='bookkeeper.proto.kv.rpc.DeleteRangeResponse.prev_kvs', index=2, + number=3, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1656, + serialized_end=1800, +) + + +_REQUESTOP = _descriptor.Descriptor( + name='RequestOp', + full_name='bookkeeper.proto.kv.rpc.RequestOp', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='request_range', full_name='bookkeeper.proto.kv.rpc.RequestOp.request_range', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='request_put', full_name='bookkeeper.proto.kv.rpc.RequestOp.request_put', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='request_delete_range', full_name='bookkeeper.proto.kv.rpc.RequestOp.request_delete_range', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='request', full_name='bookkeeper.proto.kv.rpc.RequestOp.request', + index=0, containing_type=None, fields=[]), + ], + serialized_start=1803, + serialized_end=2026, +) + + +_RESPONSEOP = _descriptor.Descriptor( + name='ResponseOp', + full_name='bookkeeper.proto.kv.rpc.ResponseOp', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='response_range', full_name='bookkeeper.proto.kv.rpc.ResponseOp.response_range', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='response_put', full_name='bookkeeper.proto.kv.rpc.ResponseOp.response_put', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='response_delete_range', full_name='bookkeeper.proto.kv.rpc.ResponseOp.response_delete_range', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='response', full_name='bookkeeper.proto.kv.rpc.ResponseOp.response', + index=0, containing_type=None, fields=[]), + ], + serialized_start=2029, + serialized_end=2260, +) + + +_COMPARE = _descriptor.Descriptor( + name='Compare', + full_name='bookkeeper.proto.kv.rpc.Compare', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='result', full_name='bookkeeper.proto.kv.rpc.Compare.result', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='target', full_name='bookkeeper.proto.kv.rpc.Compare.target', index=1, + number=2, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='key', full_name='bookkeeper.proto.kv.rpc.Compare.key', index=2, + number=3, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='version', full_name='bookkeeper.proto.kv.rpc.Compare.version', index=3, + number=4, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='create_revision', full_name='bookkeeper.proto.kv.rpc.Compare.create_revision', index=4, + number=5, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='mod_revision', full_name='bookkeeper.proto.kv.rpc.Compare.mod_revision', index=5, + number=6, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='value', full_name='bookkeeper.proto.kv.rpc.Compare.value', index=6, + number=7, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + _COMPARE_COMPARERESULT, + _COMPARE_COMPARETARGET, + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='target_union', full_name='bookkeeper.proto.kv.rpc.Compare.target_union', + index=0, containing_type=None, fields=[]), + ], + serialized_start=2263, + serialized_end=2644, +) + + +_TXNREQUEST = _descriptor.Descriptor( + name='TxnRequest', + full_name='bookkeeper.proto.kv.rpc.TxnRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='compare', full_name='bookkeeper.proto.kv.rpc.TxnRequest.compare', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='success', full_name='bookkeeper.proto.kv.rpc.TxnRequest.success', index=1, + number=2, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='failure', full_name='bookkeeper.proto.kv.rpc.TxnRequest.failure', index=2, + number=3, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.TxnRequest.header', index=3, + number=99, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2647, + serialized_end=2872, +) + + +_TXNRESPONSE = _descriptor.Descriptor( + name='TxnResponse', + full_name='bookkeeper.proto.kv.rpc.TxnResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='header', full_name='bookkeeper.proto.kv.rpc.TxnResponse.header', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='succeeded', full_name='bookkeeper.proto.kv.rpc.TxnResponse.succeeded', index=1, + number=2, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='responses', full_name='bookkeeper.proto.kv.rpc.TxnResponse.responses', index=2, + number=3, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2875, + serialized_end=3020, +) + +_RESPONSEHEADER.fields_by_name['code'].enum_type = storage__pb2._STATUSCODE +_RESPONSEHEADER.fields_by_name['routing_header'].message_type = _ROUTINGHEADER +_RANGEREQUEST.fields_by_name['sort_order'].enum_type = _RANGEREQUEST_SORTORDER +_RANGEREQUEST.fields_by_name['sort_target'].enum_type = _RANGEREQUEST_SORTTARGET +_RANGEREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER +_RANGEREQUEST_SORTORDER.containing_type = _RANGEREQUEST +_RANGEREQUEST_SORTTARGET.containing_type = _RANGEREQUEST +_RANGERESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER +_RANGERESPONSE.fields_by_name['kvs'].message_type = kv__pb2._KEYVALUE +_PUTREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER +_PUTRESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER +_PUTRESPONSE.fields_by_name['prev_kv'].message_type = kv__pb2._KEYVALUE +_INCREMENTREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER +_INCREMENTRESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER +_DELETERANGEREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER +_DELETERANGERESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER +_DELETERANGERESPONSE.fields_by_name['prev_kvs'].message_type = kv__pb2._KEYVALUE +_REQUESTOP.fields_by_name['request_range'].message_type = _RANGEREQUEST +_REQUESTOP.fields_by_name['request_put'].message_type = _PUTREQUEST +_REQUESTOP.fields_by_name['request_delete_range'].message_type = _DELETERANGEREQUEST +_REQUESTOP.oneofs_by_name['request'].fields.append( + _REQUESTOP.fields_by_name['request_range']) +_REQUESTOP.fields_by_name['request_range'].containing_oneof = _REQUESTOP.oneofs_by_name['request'] +_REQUESTOP.oneofs_by_name['request'].fields.append( + _REQUESTOP.fields_by_name['request_put']) +_REQUESTOP.fields_by_name['request_put'].containing_oneof = _REQUESTOP.oneofs_by_name['request'] +_REQUESTOP.oneofs_by_name['request'].fields.append( + _REQUESTOP.fields_by_name['request_delete_range']) +_REQUESTOP.fields_by_name['request_delete_range'].containing_oneof = _REQUESTOP.oneofs_by_name['request'] +_RESPONSEOP.fields_by_name['response_range'].message_type = _RANGERESPONSE +_RESPONSEOP.fields_by_name['response_put'].message_type = _PUTRESPONSE +_RESPONSEOP.fields_by_name['response_delete_range'].message_type = _DELETERANGERESPONSE +_RESPONSEOP.oneofs_by_name['response'].fields.append( + _RESPONSEOP.fields_by_name['response_range']) +_RESPONSEOP.fields_by_name['response_range'].containing_oneof = _RESPONSEOP.oneofs_by_name['response'] +_RESPONSEOP.oneofs_by_name['response'].fields.append( + _RESPONSEOP.fields_by_name['response_put']) +_RESPONSEOP.fields_by_name['response_put'].containing_oneof = _RESPONSEOP.oneofs_by_name['response'] +_RESPONSEOP.oneofs_by_name['response'].fields.append( + _RESPONSEOP.fields_by_name['response_delete_range']) +_RESPONSEOP.fields_by_name['response_delete_range'].containing_oneof = _RESPONSEOP.oneofs_by_name['response'] +_COMPARE.fields_by_name['result'].enum_type = _COMPARE_COMPARERESULT +_COMPARE.fields_by_name['target'].enum_type = _COMPARE_COMPARETARGET +_COMPARE_COMPARERESULT.containing_type = _COMPARE +_COMPARE_COMPARETARGET.containing_type = _COMPARE +_COMPARE.oneofs_by_name['target_union'].fields.append( + _COMPARE.fields_by_name['version']) +_COMPARE.fields_by_name['version'].containing_oneof = _COMPARE.oneofs_by_name['target_union'] +_COMPARE.oneofs_by_name['target_union'].fields.append( + _COMPARE.fields_by_name['create_revision']) +_COMPARE.fields_by_name['create_revision'].containing_oneof = _COMPARE.oneofs_by_name['target_union'] +_COMPARE.oneofs_by_name['target_union'].fields.append( + _COMPARE.fields_by_name['mod_revision']) +_COMPARE.fields_by_name['mod_revision'].containing_oneof = _COMPARE.oneofs_by_name['target_union'] +_COMPARE.oneofs_by_name['target_union'].fields.append( + _COMPARE.fields_by_name['value']) +_COMPARE.fields_by_name['value'].containing_oneof = _COMPARE.oneofs_by_name['target_union'] +_TXNREQUEST.fields_by_name['compare'].message_type = _COMPARE +_TXNREQUEST.fields_by_name['success'].message_type = _REQUESTOP +_TXNREQUEST.fields_by_name['failure'].message_type = _REQUESTOP +_TXNREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER +_TXNRESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER +_TXNRESPONSE.fields_by_name['responses'].message_type = _RESPONSEOP +DESCRIPTOR.message_types_by_name['RoutingHeader'] = _ROUTINGHEADER +DESCRIPTOR.message_types_by_name['ResponseHeader'] = _RESPONSEHEADER +DESCRIPTOR.message_types_by_name['RangeRequest'] = _RANGEREQUEST +DESCRIPTOR.message_types_by_name['RangeResponse'] = _RANGERESPONSE +DESCRIPTOR.message_types_by_name['PutRequest'] = _PUTREQUEST +DESCRIPTOR.message_types_by_name['PutResponse'] = _PUTRESPONSE +DESCRIPTOR.message_types_by_name['IncrementRequest'] = _INCREMENTREQUEST +DESCRIPTOR.message_types_by_name['IncrementResponse'] = _INCREMENTRESPONSE +DESCRIPTOR.message_types_by_name['DeleteRangeRequest'] = _DELETERANGEREQUEST +DESCRIPTOR.message_types_by_name['DeleteRangeResponse'] = _DELETERANGERESPONSE +DESCRIPTOR.message_types_by_name['RequestOp'] = _REQUESTOP +DESCRIPTOR.message_types_by_name['ResponseOp'] = _RESPONSEOP +DESCRIPTOR.message_types_by_name['Compare'] = _COMPARE +DESCRIPTOR.message_types_by_name['TxnRequest'] = _TXNREQUEST +DESCRIPTOR.message_types_by_name['TxnResponse'] = _TXNRESPONSE +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +RoutingHeader = _reflection.GeneratedProtocolMessageType('RoutingHeader', (_message.Message,), dict( + DESCRIPTOR = _ROUTINGHEADER, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.RoutingHeader) + )) +_sym_db.RegisterMessage(RoutingHeader) + +ResponseHeader = _reflection.GeneratedProtocolMessageType('ResponseHeader', (_message.Message,), dict( + DESCRIPTOR = _RESPONSEHEADER, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.ResponseHeader) + )) +_sym_db.RegisterMessage(ResponseHeader) + +RangeRequest = _reflection.GeneratedProtocolMessageType('RangeRequest', (_message.Message,), dict( + DESCRIPTOR = _RANGEREQUEST, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.RangeRequest) + )) +_sym_db.RegisterMessage(RangeRequest) + +RangeResponse = _reflection.GeneratedProtocolMessageType('RangeResponse', (_message.Message,), dict( + DESCRIPTOR = _RANGERESPONSE, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.RangeResponse) + )) +_sym_db.RegisterMessage(RangeResponse) + +PutRequest = _reflection.GeneratedProtocolMessageType('PutRequest', (_message.Message,), dict( + DESCRIPTOR = _PUTREQUEST, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.PutRequest) + )) +_sym_db.RegisterMessage(PutRequest) + +PutResponse = _reflection.GeneratedProtocolMessageType('PutResponse', (_message.Message,), dict( + DESCRIPTOR = _PUTRESPONSE, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.PutResponse) + )) +_sym_db.RegisterMessage(PutResponse) + +IncrementRequest = _reflection.GeneratedProtocolMessageType('IncrementRequest', (_message.Message,), dict( + DESCRIPTOR = _INCREMENTREQUEST, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.IncrementRequest) + )) +_sym_db.RegisterMessage(IncrementRequest) + +IncrementResponse = _reflection.GeneratedProtocolMessageType('IncrementResponse', (_message.Message,), dict( + DESCRIPTOR = _INCREMENTRESPONSE, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.IncrementResponse) + )) +_sym_db.RegisterMessage(IncrementResponse) + +DeleteRangeRequest = _reflection.GeneratedProtocolMessageType('DeleteRangeRequest', (_message.Message,), dict( + DESCRIPTOR = _DELETERANGEREQUEST, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.DeleteRangeRequest) + )) +_sym_db.RegisterMessage(DeleteRangeRequest) + +DeleteRangeResponse = _reflection.GeneratedProtocolMessageType('DeleteRangeResponse', (_message.Message,), dict( + DESCRIPTOR = _DELETERANGERESPONSE, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.DeleteRangeResponse) + )) +_sym_db.RegisterMessage(DeleteRangeResponse) + +RequestOp = _reflection.GeneratedProtocolMessageType('RequestOp', (_message.Message,), dict( + DESCRIPTOR = _REQUESTOP, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.RequestOp) + )) +_sym_db.RegisterMessage(RequestOp) + +ResponseOp = _reflection.GeneratedProtocolMessageType('ResponseOp', (_message.Message,), dict( + DESCRIPTOR = _RESPONSEOP, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.ResponseOp) + )) +_sym_db.RegisterMessage(ResponseOp) + +Compare = _reflection.GeneratedProtocolMessageType('Compare', (_message.Message,), dict( + DESCRIPTOR = _COMPARE, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.Compare) + )) +_sym_db.RegisterMessage(Compare) + +TxnRequest = _reflection.GeneratedProtocolMessageType('TxnRequest', (_message.Message,), dict( + DESCRIPTOR = _TXNREQUEST, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.TxnRequest) + )) +_sym_db.RegisterMessage(TxnRequest) + +TxnResponse = _reflection.GeneratedProtocolMessageType('TxnResponse', (_message.Message,), dict( + DESCRIPTOR = _TXNRESPONSE, + __module__ = 'kv_rpc_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.TxnResponse) + )) +_sym_db.RegisterMessage(TxnResponse) + + +DESCRIPTOR._options = None + +_TABLESERVICE = _descriptor.ServiceDescriptor( + name='TableService', + full_name='bookkeeper.proto.kv.rpc.TableService', + file=DESCRIPTOR, + index=0, + serialized_options=None, + serialized_start=3023, + serialized_end=3500, + methods=[ + _descriptor.MethodDescriptor( + name='Range', + full_name='bookkeeper.proto.kv.rpc.TableService.Range', + index=0, + containing_service=None, + input_type=_RANGEREQUEST, + output_type=_RANGERESPONSE, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='Put', + full_name='bookkeeper.proto.kv.rpc.TableService.Put', + index=1, + containing_service=None, + input_type=_PUTREQUEST, + output_type=_PUTRESPONSE, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='Delete', + full_name='bookkeeper.proto.kv.rpc.TableService.Delete', + index=2, + containing_service=None, + input_type=_DELETERANGEREQUEST, + output_type=_DELETERANGERESPONSE, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='Txn', + full_name='bookkeeper.proto.kv.rpc.TableService.Txn', + index=3, + containing_service=None, + input_type=_TXNREQUEST, + output_type=_TXNRESPONSE, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='Increment', + full_name='bookkeeper.proto.kv.rpc.TableService.Increment', + index=4, + containing_service=None, + input_type=_INCREMENTREQUEST, + output_type=_INCREMENTRESPONSE, + serialized_options=None, + ), +]) +_sym_db.RegisterServiceDescriptor(_TABLESERVICE) + +DESCRIPTOR.services_by_name['TableService'] = _TABLESERVICE + +# @@protoc_insertion_point(module_scope) diff --git a/stream/clients/python/bookkeeper/proto/kv_rpc_pb2_grpc.py b/stream/clients/python/bookkeeper/proto/kv_rpc_pb2_grpc.py new file mode 100644 index 00000000000..364ab74999a --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/kv_rpc_pb2_grpc.py @@ -0,0 +1,122 @@ +# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT! +import grpc + +import bookkeeper.proto.kv_rpc_pb2 as kv__rpc__pb2 + + +class TableServiceStub(object): + # missing associated documentation comment in .proto file + pass + + def __init__(self, channel): + """Constructor. + + Args: + channel: A grpc.Channel. + """ + self.Range = channel.unary_unary( + '/bookkeeper.proto.kv.rpc.TableService/Range', + request_serializer=kv__rpc__pb2.RangeRequest.SerializeToString, + response_deserializer=kv__rpc__pb2.RangeResponse.FromString, + ) + self.Put = channel.unary_unary( + '/bookkeeper.proto.kv.rpc.TableService/Put', + request_serializer=kv__rpc__pb2.PutRequest.SerializeToString, + response_deserializer=kv__rpc__pb2.PutResponse.FromString, + ) + self.Delete = channel.unary_unary( + '/bookkeeper.proto.kv.rpc.TableService/Delete', + request_serializer=kv__rpc__pb2.DeleteRangeRequest.SerializeToString, + response_deserializer=kv__rpc__pb2.DeleteRangeResponse.FromString, + ) + self.Txn = channel.unary_unary( + '/bookkeeper.proto.kv.rpc.TableService/Txn', + request_serializer=kv__rpc__pb2.TxnRequest.SerializeToString, + response_deserializer=kv__rpc__pb2.TxnResponse.FromString, + ) + self.Increment = channel.unary_unary( + '/bookkeeper.proto.kv.rpc.TableService/Increment', + request_serializer=kv__rpc__pb2.IncrementRequest.SerializeToString, + response_deserializer=kv__rpc__pb2.IncrementResponse.FromString, + ) + + +class TableServiceServicer(object): + # missing associated documentation comment in .proto file + pass + + def Range(self, request, context): + """Range gets the keys in the range from the key-value store. + NOT supported yet. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def Put(self, request, context): + """Put puts the given key into the key-value store. + A put request increments the revision of the key-value store + and generates one event in the event history. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def Delete(self, request, context): + """DeleteRange deletes the given range from the key-value store. + A delete request increments the revision of the key-value store + and generates a delete event in the event history for every deleted key. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def Txn(self, request, context): + """Txn processes multiple requests in a single transaction. + A txn request increments the revision of the key-value store + and generates events with the same revision for every completed request. + It is not allowed to modify the same key several times within one txn. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def Increment(self, request, context): + """Increment increments the amount associated with the keys + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + +def add_TableServiceServicer_to_server(servicer, server): + rpc_method_handlers = { + 'Range': grpc.unary_unary_rpc_method_handler( + servicer.Range, + request_deserializer=kv__rpc__pb2.RangeRequest.FromString, + response_serializer=kv__rpc__pb2.RangeResponse.SerializeToString, + ), + 'Put': grpc.unary_unary_rpc_method_handler( + servicer.Put, + request_deserializer=kv__rpc__pb2.PutRequest.FromString, + response_serializer=kv__rpc__pb2.PutResponse.SerializeToString, + ), + 'Delete': grpc.unary_unary_rpc_method_handler( + servicer.Delete, + request_deserializer=kv__rpc__pb2.DeleteRangeRequest.FromString, + response_serializer=kv__rpc__pb2.DeleteRangeResponse.SerializeToString, + ), + 'Txn': grpc.unary_unary_rpc_method_handler( + servicer.Txn, + request_deserializer=kv__rpc__pb2.TxnRequest.FromString, + response_serializer=kv__rpc__pb2.TxnResponse.SerializeToString, + ), + 'Increment': grpc.unary_unary_rpc_method_handler( + servicer.Increment, + request_deserializer=kv__rpc__pb2.IncrementRequest.FromString, + response_serializer=kv__rpc__pb2.IncrementResponse.SerializeToString, + ), + } + generic_handler = grpc.method_handlers_generic_handler( + 'bookkeeper.proto.kv.rpc.TableService', rpc_method_handlers) + server.add_generic_rpc_handlers((generic_handler,)) diff --git a/stream/clients/python/bookkeeper/proto/kv_store_pb2.py b/stream/clients/python/bookkeeper/proto/kv_store_pb2.py new file mode 100644 index 00000000000..7c4ea29e253 --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/kv_store_pb2.py @@ -0,0 +1,297 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: kv_store.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf.internal import enum_type_wrapper +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +import bookkeeper.proto.kv_pb2 as kv__pb2 +import bookkeeper.proto.kv_rpc_pb2 as kv__rpc__pb2 + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='kv_store.proto', + package='bookkeeper.proto.kv.store', + syntax='proto3', + serialized_options=_b('\n+org.apache.bookkeeper.stream.proto.kv.storeP\001'), + serialized_pb=_b('\n\x0ekv_store.proto\x12\x19\x62ookkeeper.proto.kv.store\x1a\x08kv.proto\x1a\x0ckv_rpc.proto\"\x83\x01\n\x07KeyMeta\x12\x17\n\x0f\x63reate_revision\x18\x01 \x01(\x03\x12\x14\n\x0cmod_revision\x18\x02 \x01(\x03\x12\x0f\n\x07version\x18\x03 \x01(\x03\x12\x38\n\nvalue_type\x18\x04 \x01(\x0e\x32$.bookkeeper.proto.kv.store.ValueType\"\x0c\n\nNopRequest\"\xbc\x02\n\x07\x43ommand\x12\x38\n\x07nop_req\x18\x01 \x01(\x0b\x32%.bookkeeper.proto.kv.store.NopRequestH\x00\x12\x36\n\x07put_req\x18\x02 \x01(\x0b\x32#.bookkeeper.proto.kv.rpc.PutRequestH\x00\x12\x41\n\ndelete_req\x18\x03 \x01(\x0b\x32+.bookkeeper.proto.kv.rpc.DeleteRangeRequestH\x00\x12\x36\n\x07txn_req\x18\x04 \x01(\x0b\x32#.bookkeeper.proto.kv.rpc.TxnRequestH\x00\x12=\n\x08incr_req\x18\x05 \x01(\x0b\x32).bookkeeper.proto.kv.rpc.IncrementRequestH\x00\x42\x05\n\x03req\"E\n\x12\x43heckpointMetadata\x12\r\n\x05\x66iles\x18\x01 \x03(\t\x12\x0c\n\x04txid\x18\x02 \x01(\x0c\x12\x12\n\ncreated_at\x18\x03 \x01(\x04*\"\n\tValueType\x12\t\n\x05\x42YTES\x10\x00\x12\n\n\x06NUMBER\x10\x01\x42/\n+org.apache.bookkeeper.stream.proto.kv.storeP\x01\x62\x06proto3') + , + dependencies=[kv__pb2.DESCRIPTOR,kv__rpc__pb2.DESCRIPTOR,]) + +_VALUETYPE = _descriptor.EnumDescriptor( + name='ValueType', + full_name='bookkeeper.proto.kv.store.ValueType', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='BYTES', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='NUMBER', index=1, number=1, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=607, + serialized_end=641, +) +_sym_db.RegisterEnumDescriptor(_VALUETYPE) + +ValueType = enum_type_wrapper.EnumTypeWrapper(_VALUETYPE) +BYTES = 0 +NUMBER = 1 + + + +_KEYMETA = _descriptor.Descriptor( + name='KeyMeta', + full_name='bookkeeper.proto.kv.store.KeyMeta', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='create_revision', full_name='bookkeeper.proto.kv.store.KeyMeta.create_revision', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='mod_revision', full_name='bookkeeper.proto.kv.store.KeyMeta.mod_revision', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='version', full_name='bookkeeper.proto.kv.store.KeyMeta.version', index=2, + number=3, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='value_type', full_name='bookkeeper.proto.kv.store.KeyMeta.value_type', index=3, + number=4, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=70, + serialized_end=201, +) + + +_NOPREQUEST = _descriptor.Descriptor( + name='NopRequest', + full_name='bookkeeper.proto.kv.store.NopRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=203, + serialized_end=215, +) + + +_COMMAND = _descriptor.Descriptor( + name='Command', + full_name='bookkeeper.proto.kv.store.Command', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='nop_req', full_name='bookkeeper.proto.kv.store.Command.nop_req', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='put_req', full_name='bookkeeper.proto.kv.store.Command.put_req', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='delete_req', full_name='bookkeeper.proto.kv.store.Command.delete_req', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='txn_req', full_name='bookkeeper.proto.kv.store.Command.txn_req', index=3, + number=4, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='incr_req', full_name='bookkeeper.proto.kv.store.Command.incr_req', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='req', full_name='bookkeeper.proto.kv.store.Command.req', + index=0, containing_type=None, fields=[]), + ], + serialized_start=218, + serialized_end=534, +) + + +_CHECKPOINTMETADATA = _descriptor.Descriptor( + name='CheckpointMetadata', + full_name='bookkeeper.proto.kv.store.CheckpointMetadata', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='files', full_name='bookkeeper.proto.kv.store.CheckpointMetadata.files', index=0, + number=1, type=9, cpp_type=9, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='txid', full_name='bookkeeper.proto.kv.store.CheckpointMetadata.txid', index=1, + number=2, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='created_at', full_name='bookkeeper.proto.kv.store.CheckpointMetadata.created_at', index=2, + number=3, type=4, cpp_type=4, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=536, + serialized_end=605, +) + +_KEYMETA.fields_by_name['value_type'].enum_type = _VALUETYPE +_COMMAND.fields_by_name['nop_req'].message_type = _NOPREQUEST +_COMMAND.fields_by_name['put_req'].message_type = kv__rpc__pb2._PUTREQUEST +_COMMAND.fields_by_name['delete_req'].message_type = kv__rpc__pb2._DELETERANGEREQUEST +_COMMAND.fields_by_name['txn_req'].message_type = kv__rpc__pb2._TXNREQUEST +_COMMAND.fields_by_name['incr_req'].message_type = kv__rpc__pb2._INCREMENTREQUEST +_COMMAND.oneofs_by_name['req'].fields.append( + _COMMAND.fields_by_name['nop_req']) +_COMMAND.fields_by_name['nop_req'].containing_oneof = _COMMAND.oneofs_by_name['req'] +_COMMAND.oneofs_by_name['req'].fields.append( + _COMMAND.fields_by_name['put_req']) +_COMMAND.fields_by_name['put_req'].containing_oneof = _COMMAND.oneofs_by_name['req'] +_COMMAND.oneofs_by_name['req'].fields.append( + _COMMAND.fields_by_name['delete_req']) +_COMMAND.fields_by_name['delete_req'].containing_oneof = _COMMAND.oneofs_by_name['req'] +_COMMAND.oneofs_by_name['req'].fields.append( + _COMMAND.fields_by_name['txn_req']) +_COMMAND.fields_by_name['txn_req'].containing_oneof = _COMMAND.oneofs_by_name['req'] +_COMMAND.oneofs_by_name['req'].fields.append( + _COMMAND.fields_by_name['incr_req']) +_COMMAND.fields_by_name['incr_req'].containing_oneof = _COMMAND.oneofs_by_name['req'] +DESCRIPTOR.message_types_by_name['KeyMeta'] = _KEYMETA +DESCRIPTOR.message_types_by_name['NopRequest'] = _NOPREQUEST +DESCRIPTOR.message_types_by_name['Command'] = _COMMAND +DESCRIPTOR.message_types_by_name['CheckpointMetadata'] = _CHECKPOINTMETADATA +DESCRIPTOR.enum_types_by_name['ValueType'] = _VALUETYPE +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +KeyMeta = _reflection.GeneratedProtocolMessageType('KeyMeta', (_message.Message,), dict( + DESCRIPTOR = _KEYMETA, + __module__ = 'kv_store_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.store.KeyMeta) + )) +_sym_db.RegisterMessage(KeyMeta) + +NopRequest = _reflection.GeneratedProtocolMessageType('NopRequest', (_message.Message,), dict( + DESCRIPTOR = _NOPREQUEST, + __module__ = 'kv_store_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.store.NopRequest) + )) +_sym_db.RegisterMessage(NopRequest) + +Command = _reflection.GeneratedProtocolMessageType('Command', (_message.Message,), dict( + DESCRIPTOR = _COMMAND, + __module__ = 'kv_store_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.store.Command) + )) +_sym_db.RegisterMessage(Command) + +CheckpointMetadata = _reflection.GeneratedProtocolMessageType('CheckpointMetadata', (_message.Message,), dict( + DESCRIPTOR = _CHECKPOINTMETADATA, + __module__ = 'kv_store_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.store.CheckpointMetadata) + )) +_sym_db.RegisterMessage(CheckpointMetadata) + + +DESCRIPTOR._options = None +# @@protoc_insertion_point(module_scope) diff --git a/stream/clients/python/bookkeeper/proto/storage_pb2.py b/stream/clients/python/bookkeeper/proto/storage_pb2.py new file mode 100644 index 00000000000..7f7854f574c --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/storage_pb2.py @@ -0,0 +1,1276 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: storage.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf.internal import enum_type_wrapper +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +import bookkeeper.proto.common_pb2 as common__pb2 +import bookkeeper.proto.stream_pb2 as stream__pb2 + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='storage.proto', + package='bookkeeper.proto.storage', + syntax='proto3', + serialized_options=_b('\n*org.apache.bookkeeper.stream.proto.storageP\001'), + serialized_pb=_b('\n\rstorage.proto\x12\x18\x62ookkeeper.proto.storage\x1a\x0c\x63ommon.proto\x1a\x0cstream.proto\"l\n\x16GetActiveRangesRequest\x12\x11\n\tstream_id\x18\x01 \x01(\x03\x12?\n\x0cstream_props\x18\x02 \x01(\x0b\x32).bookkeeper.proto.stream.StreamProperties\"\x86\x01\n\x17GetActiveRangesResponse\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12\x37\n\x06ranges\x18\x02 \x03(\x0b\x32\'.bookkeeper.proto.storage.RelatedRanges\"\x96\x01\n\rRelatedRanges\x12\x37\n\x05props\x18\x01 \x01(\x0b\x32(.bookkeeper.proto.stream.RangeProperties\x12\x34\n\x04type\x18\x02 \x01(\x0e\x32&.bookkeeper.proto.storage.RelationType\x12\x16\n\x0erelated_ranges\x18\x03 \x03(\x03\"h\n\x16\x43reateNamespaceRequest\x12\x0c\n\x04name\x18\x01 \x01(\t\x12@\n\x07ns_conf\x18\x02 \x01(\x0b\x32/.bookkeeper.proto.stream.NamespaceConfiguration\"\x8d\x01\n\x17\x43reateNamespaceResponse\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12>\n\x08ns_props\x18\x02 \x01(\x0b\x32,.bookkeeper.proto.stream.NamespaceProperties\"&\n\x16\x44\x65leteNamespaceRequest\x12\x0c\n\x04name\x18\x01 \x01(\t\"M\n\x17\x44\x65leteNamespaceResponse\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\"#\n\x13GetNamespaceRequest\x12\x0c\n\x04name\x18\x01 \x01(\t\"\x8a\x01\n\x14GetNamespaceResponse\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12>\n\x08ns_props\x18\x02 \x01(\x0b\x32,.bookkeeper.proto.stream.NamespaceProperties\"w\n\x13\x43reateStreamRequest\x12\x0f\n\x07ns_name\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x41\n\x0bstream_conf\x18\x03 \x01(\x0b\x32,.bookkeeper.proto.stream.StreamConfiguration\"\x8b\x01\n\x14\x43reateStreamResponse\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12?\n\x0cstream_props\x18\x02 \x01(\x0b\x32).bookkeeper.proto.stream.StreamProperties\"4\n\x13\x44\x65leteStreamRequest\x12\x0f\n\x07ns_name\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\"J\n\x14\x44\x65leteStreamResponse\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\"i\n\x10GetStreamRequest\x12:\n\x0bstream_name\x18\x01 \x01(\x0b\x32#.bookkeeper.proto.stream.StreamNameH\x00\x12\x13\n\tstream_id\x18\x02 \x01(\x03H\x00\x42\x04\n\x02id\"\x88\x01\n\x11GetStreamResponse\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12?\n\x0cstream_props\x18\x02 \x01(\x0b\x32).bookkeeper.proto.stream.StreamProperties\"\xba\x01\n\x18StorageContainerEndpoint\x12\x1c\n\x14storage_container_id\x18\x01 \x01(\x03\x12\x10\n\x08revision\x18\x02 \x01(\x03\x12\x36\n\x0brw_endpoint\x18\x03 \x01(\x0b\x32!.bookkeeper.proto.common.Endpoint\x12\x36\n\x0bro_endpoint\x18\x04 \x03(\x0b\x32!.bookkeeper.proto.common.Endpoint\"Q\n\"OneStorageContainerEndpointRequest\x12\x19\n\x11storage_container\x18\x01 \x01(\x03\x12\x10\n\x08revision\x18\x02 \x01(\x03\"t\n\"GetStorageContainerEndpointRequest\x12N\n\x08requests\x18\x01 \x03(\x0b\x32<.bookkeeper.proto.storage.OneStorageContainerEndpointRequest\"\xa6\x01\n#OneStorageContainerEndpointResponse\x12\x39\n\x0bstatus_code\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12\x44\n\x08\x65ndpoint\x18\x02 \x01(\x0b\x32\x32.bookkeeper.proto.storage.StorageContainerEndpoint\"\xb2\x01\n#GetStorageContainerEndpointResponse\x12\x39\n\x0bstatus_code\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12P\n\tresponses\x18\x02 \x03(\x0b\x32=.bookkeeper.proto.storage.OneStorageContainerEndpointResponse*\xca\x04\n\nStatusCode\x12\x0b\n\x07SUCCESS\x10\x00\x12\x0b\n\x07\x46\x41ILURE\x10\x01\x12\x10\n\x0b\x42\x41\x44_REQUEST\x10\x90\x03\x12\x0f\n\nILLEGAL_OP\x10\x93\x03\x12\x1a\n\x15INTERNAL_SERVER_ERROR\x10\xf4\x03\x12\x14\n\x0fNOT_IMPLEMENTED\x10\xf5\x03\x12\x0f\n\nUNEXPECTED\x10\xd8\x04\x12\x10\n\x0b\x42\x41\x44_VERSION\x10\x84\x07\x12\x11\n\x0c\x42\x41\x44_REVISION\x10\x85\x07\x12\x1b\n\x16INVALID_NAMESPACE_NAME\x10\xd0\x0f\x12\x15\n\x10NAMESPACE_EXISTS\x10\xd1\x0f\x12\x18\n\x13NAMESPACE_NOT_FOUND\x10\xd2\x0f\x12\x18\n\x13INVALID_STREAM_NAME\x10\xb4\x10\x12\x12\n\rSTREAM_EXISTS\x10\xb5\x10\x12\x15\n\x10STREAM_NOT_FOUND\x10\xb6\x10\x12 \n\x1bINVALID_STREAMSNAPSHOT_NAME\x10\xb7\x10\x12\x1a\n\x15STREAMSNAPSHOT_EXISTS\x10\xb8\x10\x12\x1d\n\x18STREAMSNAPSHOT_NOT_FOUND\x10\xb9\x10\x12\x19\n\x14WRONG_GROUP_LOCATION\x10\xb8\x17\x12\x15\n\x10INVALID_GROUP_ID\x10\xb9\x17\x12\x11\n\x0cGROUP_EXISTS\x10\xba\x17\x12\x14\n\x0fGROUP_NOT_FOUND\x10\xbb\x17\x12\x15\n\x10STALE_GROUP_INFO\x10\xbc\x17\x12\x10\n\x0bINVALID_KEY\x10\xf0.\x12\x0f\n\nKEY_EXISTS\x10\xf1.\x12\x12\n\rKEY_NOT_FOUND\x10\xf2.*?\n\x0cRelationType\x12\x0c\n\x08\x43HILDREN\x10\x00\x12\x0b\n\x07PARENTS\x10\x01\x12\x14\n\x10\x43HILDREN_PARENTS\x10\x02\x32\x8a\x01\n\x10MetaRangeService\x12v\n\x0fGetActiveRanges\x12\x30.bookkeeper.proto.storage.GetActiveRangesRequest\x1a\x31.bookkeeper.proto.storage.GetActiveRangesResponse2\xb5\x05\n\x10RootRangeService\x12v\n\x0f\x43reateNamespace\x12\x30.bookkeeper.proto.storage.CreateNamespaceRequest\x1a\x31.bookkeeper.proto.storage.CreateNamespaceResponse\x12v\n\x0f\x44\x65leteNamespace\x12\x30.bookkeeper.proto.storage.DeleteNamespaceRequest\x1a\x31.bookkeeper.proto.storage.DeleteNamespaceResponse\x12m\n\x0cGetNamespace\x12-.bookkeeper.proto.storage.GetNamespaceRequest\x1a..bookkeeper.proto.storage.GetNamespaceResponse\x12m\n\x0c\x43reateStream\x12-.bookkeeper.proto.storage.CreateStreamRequest\x1a..bookkeeper.proto.storage.CreateStreamResponse\x12m\n\x0c\x44\x65leteStream\x12-.bookkeeper.proto.storage.DeleteStreamRequest\x1a..bookkeeper.proto.storage.DeleteStreamResponse\x12\x64\n\tGetStream\x12*.bookkeeper.proto.storage.GetStreamRequest\x1a+.bookkeeper.proto.storage.GetStreamResponse2\xb6\x01\n\x17StorageContainerService\x12\x9a\x01\n\x1bGetStorageContainerEndpoint\x12<.bookkeeper.proto.storage.GetStorageContainerEndpointRequest\x1a=.bookkeeper.proto.storage.GetStorageContainerEndpointResponseB.\n*org.apache.bookkeeper.stream.proto.storageP\x01\x62\x06proto3') + , + dependencies=[common__pb2.DESCRIPTOR,stream__pb2.DESCRIPTOR,]) + +_STATUSCODE = _descriptor.EnumDescriptor( + name='StatusCode', + full_name='bookkeeper.proto.storage.StatusCode', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='SUCCESS', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='FAILURE', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='BAD_REQUEST', index=2, number=400, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='ILLEGAL_OP', index=3, number=403, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='INTERNAL_SERVER_ERROR', index=4, number=500, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='NOT_IMPLEMENTED', index=5, number=501, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='UNEXPECTED', index=6, number=600, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='BAD_VERSION', index=7, number=900, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='BAD_REVISION', index=8, number=901, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='INVALID_NAMESPACE_NAME', index=9, number=2000, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='NAMESPACE_EXISTS', index=10, number=2001, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='NAMESPACE_NOT_FOUND', index=11, number=2002, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='INVALID_STREAM_NAME', index=12, number=2100, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='STREAM_EXISTS', index=13, number=2101, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='STREAM_NOT_FOUND', index=14, number=2102, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='INVALID_STREAMSNAPSHOT_NAME', index=15, number=2103, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='STREAMSNAPSHOT_EXISTS', index=16, number=2104, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='STREAMSNAPSHOT_NOT_FOUND', index=17, number=2105, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='WRONG_GROUP_LOCATION', index=18, number=3000, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='INVALID_GROUP_ID', index=19, number=3001, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='GROUP_EXISTS', index=20, number=3002, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='GROUP_NOT_FOUND', index=21, number=3003, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='STALE_GROUP_INFO', index=22, number=3004, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='INVALID_KEY', index=23, number=6000, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='KEY_EXISTS', index=24, number=6001, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='KEY_NOT_FOUND', index=25, number=6002, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=2398, + serialized_end=2984, +) +_sym_db.RegisterEnumDescriptor(_STATUSCODE) + +StatusCode = enum_type_wrapper.EnumTypeWrapper(_STATUSCODE) +_RELATIONTYPE = _descriptor.EnumDescriptor( + name='RelationType', + full_name='bookkeeper.proto.storage.RelationType', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='CHILDREN', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='PARENTS', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='CHILDREN_PARENTS', index=2, number=2, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=2986, + serialized_end=3049, +) +_sym_db.RegisterEnumDescriptor(_RELATIONTYPE) + +RelationType = enum_type_wrapper.EnumTypeWrapper(_RELATIONTYPE) +SUCCESS = 0 +FAILURE = 1 +BAD_REQUEST = 400 +ILLEGAL_OP = 403 +INTERNAL_SERVER_ERROR = 500 +NOT_IMPLEMENTED = 501 +UNEXPECTED = 600 +BAD_VERSION = 900 +BAD_REVISION = 901 +INVALID_NAMESPACE_NAME = 2000 +NAMESPACE_EXISTS = 2001 +NAMESPACE_NOT_FOUND = 2002 +INVALID_STREAM_NAME = 2100 +STREAM_EXISTS = 2101 +STREAM_NOT_FOUND = 2102 +INVALID_STREAMSNAPSHOT_NAME = 2103 +STREAMSNAPSHOT_EXISTS = 2104 +STREAMSNAPSHOT_NOT_FOUND = 2105 +WRONG_GROUP_LOCATION = 3000 +INVALID_GROUP_ID = 3001 +GROUP_EXISTS = 3002 +GROUP_NOT_FOUND = 3003 +STALE_GROUP_INFO = 3004 +INVALID_KEY = 6000 +KEY_EXISTS = 6001 +KEY_NOT_FOUND = 6002 +CHILDREN = 0 +PARENTS = 1 +CHILDREN_PARENTS = 2 + + + +_GETACTIVERANGESREQUEST = _descriptor.Descriptor( + name='GetActiveRangesRequest', + full_name='bookkeeper.proto.storage.GetActiveRangesRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='stream_id', full_name='bookkeeper.proto.storage.GetActiveRangesRequest.stream_id', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='stream_props', full_name='bookkeeper.proto.storage.GetActiveRangesRequest.stream_props', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=71, + serialized_end=179, +) + + +_GETACTIVERANGESRESPONSE = _descriptor.Descriptor( + name='GetActiveRangesResponse', + full_name='bookkeeper.proto.storage.GetActiveRangesResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='code', full_name='bookkeeper.proto.storage.GetActiveRangesResponse.code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='ranges', full_name='bookkeeper.proto.storage.GetActiveRangesResponse.ranges', index=1, + number=2, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=182, + serialized_end=316, +) + + +_RELATEDRANGES = _descriptor.Descriptor( + name='RelatedRanges', + full_name='bookkeeper.proto.storage.RelatedRanges', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='props', full_name='bookkeeper.proto.storage.RelatedRanges.props', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='type', full_name='bookkeeper.proto.storage.RelatedRanges.type', index=1, + number=2, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='related_ranges', full_name='bookkeeper.proto.storage.RelatedRanges.related_ranges', index=2, + number=3, type=3, cpp_type=2, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=319, + serialized_end=469, +) + + +_CREATENAMESPACEREQUEST = _descriptor.Descriptor( + name='CreateNamespaceRequest', + full_name='bookkeeper.proto.storage.CreateNamespaceRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='name', full_name='bookkeeper.proto.storage.CreateNamespaceRequest.name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='ns_conf', full_name='bookkeeper.proto.storage.CreateNamespaceRequest.ns_conf', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=471, + serialized_end=575, +) + + +_CREATENAMESPACERESPONSE = _descriptor.Descriptor( + name='CreateNamespaceResponse', + full_name='bookkeeper.proto.storage.CreateNamespaceResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='code', full_name='bookkeeper.proto.storage.CreateNamespaceResponse.code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='ns_props', full_name='bookkeeper.proto.storage.CreateNamespaceResponse.ns_props', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=578, + serialized_end=719, +) + + +_DELETENAMESPACEREQUEST = _descriptor.Descriptor( + name='DeleteNamespaceRequest', + full_name='bookkeeper.proto.storage.DeleteNamespaceRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='name', full_name='bookkeeper.proto.storage.DeleteNamespaceRequest.name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=721, + serialized_end=759, +) + + +_DELETENAMESPACERESPONSE = _descriptor.Descriptor( + name='DeleteNamespaceResponse', + full_name='bookkeeper.proto.storage.DeleteNamespaceResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='code', full_name='bookkeeper.proto.storage.DeleteNamespaceResponse.code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=761, + serialized_end=838, +) + + +_GETNAMESPACEREQUEST = _descriptor.Descriptor( + name='GetNamespaceRequest', + full_name='bookkeeper.proto.storage.GetNamespaceRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='name', full_name='bookkeeper.proto.storage.GetNamespaceRequest.name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=840, + serialized_end=875, +) + + +_GETNAMESPACERESPONSE = _descriptor.Descriptor( + name='GetNamespaceResponse', + full_name='bookkeeper.proto.storage.GetNamespaceResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='code', full_name='bookkeeper.proto.storage.GetNamespaceResponse.code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='ns_props', full_name='bookkeeper.proto.storage.GetNamespaceResponse.ns_props', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=878, + serialized_end=1016, +) + + +_CREATESTREAMREQUEST = _descriptor.Descriptor( + name='CreateStreamRequest', + full_name='bookkeeper.proto.storage.CreateStreamRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='ns_name', full_name='bookkeeper.proto.storage.CreateStreamRequest.ns_name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='name', full_name='bookkeeper.proto.storage.CreateStreamRequest.name', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='stream_conf', full_name='bookkeeper.proto.storage.CreateStreamRequest.stream_conf', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1018, + serialized_end=1137, +) + + +_CREATESTREAMRESPONSE = _descriptor.Descriptor( + name='CreateStreamResponse', + full_name='bookkeeper.proto.storage.CreateStreamResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='code', full_name='bookkeeper.proto.storage.CreateStreamResponse.code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='stream_props', full_name='bookkeeper.proto.storage.CreateStreamResponse.stream_props', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1140, + serialized_end=1279, +) + + +_DELETESTREAMREQUEST = _descriptor.Descriptor( + name='DeleteStreamRequest', + full_name='bookkeeper.proto.storage.DeleteStreamRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='ns_name', full_name='bookkeeper.proto.storage.DeleteStreamRequest.ns_name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='name', full_name='bookkeeper.proto.storage.DeleteStreamRequest.name', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1281, + serialized_end=1333, +) + + +_DELETESTREAMRESPONSE = _descriptor.Descriptor( + name='DeleteStreamResponse', + full_name='bookkeeper.proto.storage.DeleteStreamResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='code', full_name='bookkeeper.proto.storage.DeleteStreamResponse.code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1335, + serialized_end=1409, +) + + +_GETSTREAMREQUEST = _descriptor.Descriptor( + name='GetStreamRequest', + full_name='bookkeeper.proto.storage.GetStreamRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='stream_name', full_name='bookkeeper.proto.storage.GetStreamRequest.stream_name', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='stream_id', full_name='bookkeeper.proto.storage.GetStreamRequest.stream_id', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='id', full_name='bookkeeper.proto.storage.GetStreamRequest.id', + index=0, containing_type=None, fields=[]), + ], + serialized_start=1411, + serialized_end=1516, +) + + +_GETSTREAMRESPONSE = _descriptor.Descriptor( + name='GetStreamResponse', + full_name='bookkeeper.proto.storage.GetStreamResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='code', full_name='bookkeeper.proto.storage.GetStreamResponse.code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='stream_props', full_name='bookkeeper.proto.storage.GetStreamResponse.stream_props', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1519, + serialized_end=1655, +) + + +_STORAGECONTAINERENDPOINT = _descriptor.Descriptor( + name='StorageContainerEndpoint', + full_name='bookkeeper.proto.storage.StorageContainerEndpoint', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='storage_container_id', full_name='bookkeeper.proto.storage.StorageContainerEndpoint.storage_container_id', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='revision', full_name='bookkeeper.proto.storage.StorageContainerEndpoint.revision', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='rw_endpoint', full_name='bookkeeper.proto.storage.StorageContainerEndpoint.rw_endpoint', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='ro_endpoint', full_name='bookkeeper.proto.storage.StorageContainerEndpoint.ro_endpoint', index=3, + number=4, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1658, + serialized_end=1844, +) + + +_ONESTORAGECONTAINERENDPOINTREQUEST = _descriptor.Descriptor( + name='OneStorageContainerEndpointRequest', + full_name='bookkeeper.proto.storage.OneStorageContainerEndpointRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='storage_container', full_name='bookkeeper.proto.storage.OneStorageContainerEndpointRequest.storage_container', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='revision', full_name='bookkeeper.proto.storage.OneStorageContainerEndpointRequest.revision', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1846, + serialized_end=1927, +) + + +_GETSTORAGECONTAINERENDPOINTREQUEST = _descriptor.Descriptor( + name='GetStorageContainerEndpointRequest', + full_name='bookkeeper.proto.storage.GetStorageContainerEndpointRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='requests', full_name='bookkeeper.proto.storage.GetStorageContainerEndpointRequest.requests', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1929, + serialized_end=2045, +) + + +_ONESTORAGECONTAINERENDPOINTRESPONSE = _descriptor.Descriptor( + name='OneStorageContainerEndpointResponse', + full_name='bookkeeper.proto.storage.OneStorageContainerEndpointResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='status_code', full_name='bookkeeper.proto.storage.OneStorageContainerEndpointResponse.status_code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='endpoint', full_name='bookkeeper.proto.storage.OneStorageContainerEndpointResponse.endpoint', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2048, + serialized_end=2214, +) + + +_GETSTORAGECONTAINERENDPOINTRESPONSE = _descriptor.Descriptor( + name='GetStorageContainerEndpointResponse', + full_name='bookkeeper.proto.storage.GetStorageContainerEndpointResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='status_code', full_name='bookkeeper.proto.storage.GetStorageContainerEndpointResponse.status_code', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='responses', full_name='bookkeeper.proto.storage.GetStorageContainerEndpointResponse.responses', index=1, + number=2, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2217, + serialized_end=2395, +) + +_GETACTIVERANGESREQUEST.fields_by_name['stream_props'].message_type = stream__pb2._STREAMPROPERTIES +_GETACTIVERANGESRESPONSE.fields_by_name['code'].enum_type = _STATUSCODE +_GETACTIVERANGESRESPONSE.fields_by_name['ranges'].message_type = _RELATEDRANGES +_RELATEDRANGES.fields_by_name['props'].message_type = stream__pb2._RANGEPROPERTIES +_RELATEDRANGES.fields_by_name['type'].enum_type = _RELATIONTYPE +_CREATENAMESPACEREQUEST.fields_by_name['ns_conf'].message_type = stream__pb2._NAMESPACECONFIGURATION +_CREATENAMESPACERESPONSE.fields_by_name['code'].enum_type = _STATUSCODE +_CREATENAMESPACERESPONSE.fields_by_name['ns_props'].message_type = stream__pb2._NAMESPACEPROPERTIES +_DELETENAMESPACERESPONSE.fields_by_name['code'].enum_type = _STATUSCODE +_GETNAMESPACERESPONSE.fields_by_name['code'].enum_type = _STATUSCODE +_GETNAMESPACERESPONSE.fields_by_name['ns_props'].message_type = stream__pb2._NAMESPACEPROPERTIES +_CREATESTREAMREQUEST.fields_by_name['stream_conf'].message_type = stream__pb2._STREAMCONFIGURATION +_CREATESTREAMRESPONSE.fields_by_name['code'].enum_type = _STATUSCODE +_CREATESTREAMRESPONSE.fields_by_name['stream_props'].message_type = stream__pb2._STREAMPROPERTIES +_DELETESTREAMRESPONSE.fields_by_name['code'].enum_type = _STATUSCODE +_GETSTREAMREQUEST.fields_by_name['stream_name'].message_type = stream__pb2._STREAMNAME +_GETSTREAMREQUEST.oneofs_by_name['id'].fields.append( + _GETSTREAMREQUEST.fields_by_name['stream_name']) +_GETSTREAMREQUEST.fields_by_name['stream_name'].containing_oneof = _GETSTREAMREQUEST.oneofs_by_name['id'] +_GETSTREAMREQUEST.oneofs_by_name['id'].fields.append( + _GETSTREAMREQUEST.fields_by_name['stream_id']) +_GETSTREAMREQUEST.fields_by_name['stream_id'].containing_oneof = _GETSTREAMREQUEST.oneofs_by_name['id'] +_GETSTREAMRESPONSE.fields_by_name['code'].enum_type = _STATUSCODE +_GETSTREAMRESPONSE.fields_by_name['stream_props'].message_type = stream__pb2._STREAMPROPERTIES +_STORAGECONTAINERENDPOINT.fields_by_name['rw_endpoint'].message_type = common__pb2._ENDPOINT +_STORAGECONTAINERENDPOINT.fields_by_name['ro_endpoint'].message_type = common__pb2._ENDPOINT +_GETSTORAGECONTAINERENDPOINTREQUEST.fields_by_name['requests'].message_type = _ONESTORAGECONTAINERENDPOINTREQUEST +_ONESTORAGECONTAINERENDPOINTRESPONSE.fields_by_name['status_code'].enum_type = _STATUSCODE +_ONESTORAGECONTAINERENDPOINTRESPONSE.fields_by_name['endpoint'].message_type = _STORAGECONTAINERENDPOINT +_GETSTORAGECONTAINERENDPOINTRESPONSE.fields_by_name['status_code'].enum_type = _STATUSCODE +_GETSTORAGECONTAINERENDPOINTRESPONSE.fields_by_name['responses'].message_type = _ONESTORAGECONTAINERENDPOINTRESPONSE +DESCRIPTOR.message_types_by_name['GetActiveRangesRequest'] = _GETACTIVERANGESREQUEST +DESCRIPTOR.message_types_by_name['GetActiveRangesResponse'] = _GETACTIVERANGESRESPONSE +DESCRIPTOR.message_types_by_name['RelatedRanges'] = _RELATEDRANGES +DESCRIPTOR.message_types_by_name['CreateNamespaceRequest'] = _CREATENAMESPACEREQUEST +DESCRIPTOR.message_types_by_name['CreateNamespaceResponse'] = _CREATENAMESPACERESPONSE +DESCRIPTOR.message_types_by_name['DeleteNamespaceRequest'] = _DELETENAMESPACEREQUEST +DESCRIPTOR.message_types_by_name['DeleteNamespaceResponse'] = _DELETENAMESPACERESPONSE +DESCRIPTOR.message_types_by_name['GetNamespaceRequest'] = _GETNAMESPACEREQUEST +DESCRIPTOR.message_types_by_name['GetNamespaceResponse'] = _GETNAMESPACERESPONSE +DESCRIPTOR.message_types_by_name['CreateStreamRequest'] = _CREATESTREAMREQUEST +DESCRIPTOR.message_types_by_name['CreateStreamResponse'] = _CREATESTREAMRESPONSE +DESCRIPTOR.message_types_by_name['DeleteStreamRequest'] = _DELETESTREAMREQUEST +DESCRIPTOR.message_types_by_name['DeleteStreamResponse'] = _DELETESTREAMRESPONSE +DESCRIPTOR.message_types_by_name['GetStreamRequest'] = _GETSTREAMREQUEST +DESCRIPTOR.message_types_by_name['GetStreamResponse'] = _GETSTREAMRESPONSE +DESCRIPTOR.message_types_by_name['StorageContainerEndpoint'] = _STORAGECONTAINERENDPOINT +DESCRIPTOR.message_types_by_name['OneStorageContainerEndpointRequest'] = _ONESTORAGECONTAINERENDPOINTREQUEST +DESCRIPTOR.message_types_by_name['GetStorageContainerEndpointRequest'] = _GETSTORAGECONTAINERENDPOINTREQUEST +DESCRIPTOR.message_types_by_name['OneStorageContainerEndpointResponse'] = _ONESTORAGECONTAINERENDPOINTRESPONSE +DESCRIPTOR.message_types_by_name['GetStorageContainerEndpointResponse'] = _GETSTORAGECONTAINERENDPOINTRESPONSE +DESCRIPTOR.enum_types_by_name['StatusCode'] = _STATUSCODE +DESCRIPTOR.enum_types_by_name['RelationType'] = _RELATIONTYPE +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +GetActiveRangesRequest = _reflection.GeneratedProtocolMessageType('GetActiveRangesRequest', (_message.Message,), dict( + DESCRIPTOR = _GETACTIVERANGESREQUEST, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetActiveRangesRequest) + )) +_sym_db.RegisterMessage(GetActiveRangesRequest) + +GetActiveRangesResponse = _reflection.GeneratedProtocolMessageType('GetActiveRangesResponse', (_message.Message,), dict( + DESCRIPTOR = _GETACTIVERANGESRESPONSE, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetActiveRangesResponse) + )) +_sym_db.RegisterMessage(GetActiveRangesResponse) + +RelatedRanges = _reflection.GeneratedProtocolMessageType('RelatedRanges', (_message.Message,), dict( + DESCRIPTOR = _RELATEDRANGES, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.RelatedRanges) + )) +_sym_db.RegisterMessage(RelatedRanges) + +CreateNamespaceRequest = _reflection.GeneratedProtocolMessageType('CreateNamespaceRequest', (_message.Message,), dict( + DESCRIPTOR = _CREATENAMESPACEREQUEST, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.CreateNamespaceRequest) + )) +_sym_db.RegisterMessage(CreateNamespaceRequest) + +CreateNamespaceResponse = _reflection.GeneratedProtocolMessageType('CreateNamespaceResponse', (_message.Message,), dict( + DESCRIPTOR = _CREATENAMESPACERESPONSE, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.CreateNamespaceResponse) + )) +_sym_db.RegisterMessage(CreateNamespaceResponse) + +DeleteNamespaceRequest = _reflection.GeneratedProtocolMessageType('DeleteNamespaceRequest', (_message.Message,), dict( + DESCRIPTOR = _DELETENAMESPACEREQUEST, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.DeleteNamespaceRequest) + )) +_sym_db.RegisterMessage(DeleteNamespaceRequest) + +DeleteNamespaceResponse = _reflection.GeneratedProtocolMessageType('DeleteNamespaceResponse', (_message.Message,), dict( + DESCRIPTOR = _DELETENAMESPACERESPONSE, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.DeleteNamespaceResponse) + )) +_sym_db.RegisterMessage(DeleteNamespaceResponse) + +GetNamespaceRequest = _reflection.GeneratedProtocolMessageType('GetNamespaceRequest', (_message.Message,), dict( + DESCRIPTOR = _GETNAMESPACEREQUEST, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetNamespaceRequest) + )) +_sym_db.RegisterMessage(GetNamespaceRequest) + +GetNamespaceResponse = _reflection.GeneratedProtocolMessageType('GetNamespaceResponse', (_message.Message,), dict( + DESCRIPTOR = _GETNAMESPACERESPONSE, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetNamespaceResponse) + )) +_sym_db.RegisterMessage(GetNamespaceResponse) + +CreateStreamRequest = _reflection.GeneratedProtocolMessageType('CreateStreamRequest', (_message.Message,), dict( + DESCRIPTOR = _CREATESTREAMREQUEST, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.CreateStreamRequest) + )) +_sym_db.RegisterMessage(CreateStreamRequest) + +CreateStreamResponse = _reflection.GeneratedProtocolMessageType('CreateStreamResponse', (_message.Message,), dict( + DESCRIPTOR = _CREATESTREAMRESPONSE, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.CreateStreamResponse) + )) +_sym_db.RegisterMessage(CreateStreamResponse) + +DeleteStreamRequest = _reflection.GeneratedProtocolMessageType('DeleteStreamRequest', (_message.Message,), dict( + DESCRIPTOR = _DELETESTREAMREQUEST, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.DeleteStreamRequest) + )) +_sym_db.RegisterMessage(DeleteStreamRequest) + +DeleteStreamResponse = _reflection.GeneratedProtocolMessageType('DeleteStreamResponse', (_message.Message,), dict( + DESCRIPTOR = _DELETESTREAMRESPONSE, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.DeleteStreamResponse) + )) +_sym_db.RegisterMessage(DeleteStreamResponse) + +GetStreamRequest = _reflection.GeneratedProtocolMessageType('GetStreamRequest', (_message.Message,), dict( + DESCRIPTOR = _GETSTREAMREQUEST, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetStreamRequest) + )) +_sym_db.RegisterMessage(GetStreamRequest) + +GetStreamResponse = _reflection.GeneratedProtocolMessageType('GetStreamResponse', (_message.Message,), dict( + DESCRIPTOR = _GETSTREAMRESPONSE, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetStreamResponse) + )) +_sym_db.RegisterMessage(GetStreamResponse) + +StorageContainerEndpoint = _reflection.GeneratedProtocolMessageType('StorageContainerEndpoint', (_message.Message,), dict( + DESCRIPTOR = _STORAGECONTAINERENDPOINT, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.StorageContainerEndpoint) + )) +_sym_db.RegisterMessage(StorageContainerEndpoint) + +OneStorageContainerEndpointRequest = _reflection.GeneratedProtocolMessageType('OneStorageContainerEndpointRequest', (_message.Message,), dict( + DESCRIPTOR = _ONESTORAGECONTAINERENDPOINTREQUEST, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.OneStorageContainerEndpointRequest) + )) +_sym_db.RegisterMessage(OneStorageContainerEndpointRequest) + +GetStorageContainerEndpointRequest = _reflection.GeneratedProtocolMessageType('GetStorageContainerEndpointRequest', (_message.Message,), dict( + DESCRIPTOR = _GETSTORAGECONTAINERENDPOINTREQUEST, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetStorageContainerEndpointRequest) + )) +_sym_db.RegisterMessage(GetStorageContainerEndpointRequest) + +OneStorageContainerEndpointResponse = _reflection.GeneratedProtocolMessageType('OneStorageContainerEndpointResponse', (_message.Message,), dict( + DESCRIPTOR = _ONESTORAGECONTAINERENDPOINTRESPONSE, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.OneStorageContainerEndpointResponse) + )) +_sym_db.RegisterMessage(OneStorageContainerEndpointResponse) + +GetStorageContainerEndpointResponse = _reflection.GeneratedProtocolMessageType('GetStorageContainerEndpointResponse', (_message.Message,), dict( + DESCRIPTOR = _GETSTORAGECONTAINERENDPOINTRESPONSE, + __module__ = 'storage_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetStorageContainerEndpointResponse) + )) +_sym_db.RegisterMessage(GetStorageContainerEndpointResponse) + + +DESCRIPTOR._options = None + +_METARANGESERVICE = _descriptor.ServiceDescriptor( + name='MetaRangeService', + full_name='bookkeeper.proto.storage.MetaRangeService', + file=DESCRIPTOR, + index=0, + serialized_options=None, + serialized_start=3052, + serialized_end=3190, + methods=[ + _descriptor.MethodDescriptor( + name='GetActiveRanges', + full_name='bookkeeper.proto.storage.MetaRangeService.GetActiveRanges', + index=0, + containing_service=None, + input_type=_GETACTIVERANGESREQUEST, + output_type=_GETACTIVERANGESRESPONSE, + serialized_options=None, + ), +]) +_sym_db.RegisterServiceDescriptor(_METARANGESERVICE) + +DESCRIPTOR.services_by_name['MetaRangeService'] = _METARANGESERVICE + + +_ROOTRANGESERVICE = _descriptor.ServiceDescriptor( + name='RootRangeService', + full_name='bookkeeper.proto.storage.RootRangeService', + file=DESCRIPTOR, + index=1, + serialized_options=None, + serialized_start=3193, + serialized_end=3886, + methods=[ + _descriptor.MethodDescriptor( + name='CreateNamespace', + full_name='bookkeeper.proto.storage.RootRangeService.CreateNamespace', + index=0, + containing_service=None, + input_type=_CREATENAMESPACEREQUEST, + output_type=_CREATENAMESPACERESPONSE, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='DeleteNamespace', + full_name='bookkeeper.proto.storage.RootRangeService.DeleteNamespace', + index=1, + containing_service=None, + input_type=_DELETENAMESPACEREQUEST, + output_type=_DELETENAMESPACERESPONSE, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='GetNamespace', + full_name='bookkeeper.proto.storage.RootRangeService.GetNamespace', + index=2, + containing_service=None, + input_type=_GETNAMESPACEREQUEST, + output_type=_GETNAMESPACERESPONSE, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='CreateStream', + full_name='bookkeeper.proto.storage.RootRangeService.CreateStream', + index=3, + containing_service=None, + input_type=_CREATESTREAMREQUEST, + output_type=_CREATESTREAMRESPONSE, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='DeleteStream', + full_name='bookkeeper.proto.storage.RootRangeService.DeleteStream', + index=4, + containing_service=None, + input_type=_DELETESTREAMREQUEST, + output_type=_DELETESTREAMRESPONSE, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='GetStream', + full_name='bookkeeper.proto.storage.RootRangeService.GetStream', + index=5, + containing_service=None, + input_type=_GETSTREAMREQUEST, + output_type=_GETSTREAMRESPONSE, + serialized_options=None, + ), +]) +_sym_db.RegisterServiceDescriptor(_ROOTRANGESERVICE) + +DESCRIPTOR.services_by_name['RootRangeService'] = _ROOTRANGESERVICE + + +_STORAGECONTAINERSERVICE = _descriptor.ServiceDescriptor( + name='StorageContainerService', + full_name='bookkeeper.proto.storage.StorageContainerService', + file=DESCRIPTOR, + index=2, + serialized_options=None, + serialized_start=3889, + serialized_end=4071, + methods=[ + _descriptor.MethodDescriptor( + name='GetStorageContainerEndpoint', + full_name='bookkeeper.proto.storage.StorageContainerService.GetStorageContainerEndpoint', + index=0, + containing_service=None, + input_type=_GETSTORAGECONTAINERENDPOINTREQUEST, + output_type=_GETSTORAGECONTAINERENDPOINTRESPONSE, + serialized_options=None, + ), +]) +_sym_db.RegisterServiceDescriptor(_STORAGECONTAINERSERVICE) + +DESCRIPTOR.services_by_name['StorageContainerService'] = _STORAGECONTAINERSERVICE + +# @@protoc_insertion_point(module_scope) diff --git a/stream/clients/python/bookkeeper/proto/storage_pb2_grpc.py b/stream/clients/python/bookkeeper/proto/storage_pb2_grpc.py new file mode 100644 index 00000000000..1f89bd3b748 --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/storage_pb2_grpc.py @@ -0,0 +1,219 @@ +# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT! +import grpc + +import bookkeeper.proto.storage_pb2 as storage__pb2 + + +class MetaRangeServiceStub(object): + """public service for other operations in range server + """ + + def __init__(self, channel): + """Constructor. + + Args: + channel: A grpc.Channel. + """ + self.GetActiveRanges = channel.unary_unary( + '/bookkeeper.proto.storage.MetaRangeService/GetActiveRanges', + request_serializer=storage__pb2.GetActiveRangesRequest.SerializeToString, + response_deserializer=storage__pb2.GetActiveRangesResponse.FromString, + ) + + +class MetaRangeServiceServicer(object): + """public service for other operations in range server + """ + + def GetActiveRanges(self, request, context): + # missing associated documentation comment in .proto file + pass + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + +def add_MetaRangeServiceServicer_to_server(servicer, server): + rpc_method_handlers = { + 'GetActiveRanges': grpc.unary_unary_rpc_method_handler( + servicer.GetActiveRanges, + request_deserializer=storage__pb2.GetActiveRangesRequest.FromString, + response_serializer=storage__pb2.GetActiveRangesResponse.SerializeToString, + ), + } + generic_handler = grpc.method_handlers_generic_handler( + 'bookkeeper.proto.storage.MetaRangeService', rpc_method_handlers) + server.add_generic_rpc_handlers((generic_handler,)) + + +class RootRangeServiceStub(object): + """public service for metadata services + """ + + def __init__(self, channel): + """Constructor. + + Args: + channel: A grpc.Channel. + """ + self.CreateNamespace = channel.unary_unary( + '/bookkeeper.proto.storage.RootRangeService/CreateNamespace', + request_serializer=storage__pb2.CreateNamespaceRequest.SerializeToString, + response_deserializer=storage__pb2.CreateNamespaceResponse.FromString, + ) + self.DeleteNamespace = channel.unary_unary( + '/bookkeeper.proto.storage.RootRangeService/DeleteNamespace', + request_serializer=storage__pb2.DeleteNamespaceRequest.SerializeToString, + response_deserializer=storage__pb2.DeleteNamespaceResponse.FromString, + ) + self.GetNamespace = channel.unary_unary( + '/bookkeeper.proto.storage.RootRangeService/GetNamespace', + request_serializer=storage__pb2.GetNamespaceRequest.SerializeToString, + response_deserializer=storage__pb2.GetNamespaceResponse.FromString, + ) + self.CreateStream = channel.unary_unary( + '/bookkeeper.proto.storage.RootRangeService/CreateStream', + request_serializer=storage__pb2.CreateStreamRequest.SerializeToString, + response_deserializer=storage__pb2.CreateStreamResponse.FromString, + ) + self.DeleteStream = channel.unary_unary( + '/bookkeeper.proto.storage.RootRangeService/DeleteStream', + request_serializer=storage__pb2.DeleteStreamRequest.SerializeToString, + response_deserializer=storage__pb2.DeleteStreamResponse.FromString, + ) + self.GetStream = channel.unary_unary( + '/bookkeeper.proto.storage.RootRangeService/GetStream', + request_serializer=storage__pb2.GetStreamRequest.SerializeToString, + response_deserializer=storage__pb2.GetStreamResponse.FromString, + ) + + +class RootRangeServiceServicer(object): + """public service for metadata services + """ + + def CreateNamespace(self, request, context): + """ + Namespace Methods + + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def DeleteNamespace(self, request, context): + # missing associated documentation comment in .proto file + pass + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def GetNamespace(self, request, context): + # missing associated documentation comment in .proto file + pass + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def CreateStream(self, request, context): + """ + Stream Methods + + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def DeleteStream(self, request, context): + # missing associated documentation comment in .proto file + pass + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def GetStream(self, request, context): + # missing associated documentation comment in .proto file + pass + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + +def add_RootRangeServiceServicer_to_server(servicer, server): + rpc_method_handlers = { + 'CreateNamespace': grpc.unary_unary_rpc_method_handler( + servicer.CreateNamespace, + request_deserializer=storage__pb2.CreateNamespaceRequest.FromString, + response_serializer=storage__pb2.CreateNamespaceResponse.SerializeToString, + ), + 'DeleteNamespace': grpc.unary_unary_rpc_method_handler( + servicer.DeleteNamespace, + request_deserializer=storage__pb2.DeleteNamespaceRequest.FromString, + response_serializer=storage__pb2.DeleteNamespaceResponse.SerializeToString, + ), + 'GetNamespace': grpc.unary_unary_rpc_method_handler( + servicer.GetNamespace, + request_deserializer=storage__pb2.GetNamespaceRequest.FromString, + response_serializer=storage__pb2.GetNamespaceResponse.SerializeToString, + ), + 'CreateStream': grpc.unary_unary_rpc_method_handler( + servicer.CreateStream, + request_deserializer=storage__pb2.CreateStreamRequest.FromString, + response_serializer=storage__pb2.CreateStreamResponse.SerializeToString, + ), + 'DeleteStream': grpc.unary_unary_rpc_method_handler( + servicer.DeleteStream, + request_deserializer=storage__pb2.DeleteStreamRequest.FromString, + response_serializer=storage__pb2.DeleteStreamResponse.SerializeToString, + ), + 'GetStream': grpc.unary_unary_rpc_method_handler( + servicer.GetStream, + request_deserializer=storage__pb2.GetStreamRequest.FromString, + response_serializer=storage__pb2.GetStreamResponse.SerializeToString, + ), + } + generic_handler = grpc.method_handlers_generic_handler( + 'bookkeeper.proto.storage.RootRangeService', rpc_method_handlers) + server.add_generic_rpc_handlers((generic_handler,)) + + +class StorageContainerServiceStub(object): + """A general range server service + """ + + def __init__(self, channel): + """Constructor. + + Args: + channel: A grpc.Channel. + """ + self.GetStorageContainerEndpoint = channel.unary_unary( + '/bookkeeper.proto.storage.StorageContainerService/GetStorageContainerEndpoint', + request_serializer=storage__pb2.GetStorageContainerEndpointRequest.SerializeToString, + response_deserializer=storage__pb2.GetStorageContainerEndpointResponse.FromString, + ) + + +class StorageContainerServiceServicer(object): + """A general range server service + """ + + def GetStorageContainerEndpoint(self, request, context): + """Get the storage container endpoints + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + +def add_StorageContainerServiceServicer_to_server(servicer, server): + rpc_method_handlers = { + 'GetStorageContainerEndpoint': grpc.unary_unary_rpc_method_handler( + servicer.GetStorageContainerEndpoint, + request_deserializer=storage__pb2.GetStorageContainerEndpointRequest.FromString, + response_serializer=storage__pb2.GetStorageContainerEndpointResponse.SerializeToString, + ), + } + generic_handler = grpc.method_handlers_generic_handler( + 'bookkeeper.proto.storage.StorageContainerService', rpc_method_handlers) + server.add_generic_rpc_handlers((generic_handler,)) diff --git a/stream/clients/python/bookkeeper/proto/stream_pb2.py b/stream/clients/python/bookkeeper/proto/stream_pb2.py new file mode 100644 index 00000000000..2374de6c12b --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/stream_pb2.py @@ -0,0 +1,1323 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: stream.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf.internal import enum_type_wrapper +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +import bookkeeper.proto.common_pb2 as common__pb2 + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='stream.proto', + package='bookkeeper.proto.stream', + syntax='proto3', + serialized_options=_b('\n\"org.apache.bookkeeper.stream.protoP\001'), + serialized_pb=_b('\n\x0cstream.proto\x12\x17\x62ookkeeper.proto.stream\x1a\x0c\x63ommon.proto\"=\n\x07RangeId\x12\r\n\x05sc_id\x18\x01 \x01(\x03\x12\x11\n\tstream_id\x18\x02 \x01(\x03\x12\x10\n\x08range_id\x18\x03 \x01(\x03\"8\n\x08KeyRange\x12\x16\n\x0estart_hash_key\x18\x01 \x01(\x03\x12\x14\n\x0c\x65nd_hash_key\x18\x02 \x01(\x03\"o\n\x0fRangeProperties\x12\x16\n\x0estart_hash_key\x18\x01 \x01(\x03\x12\x14\n\x0c\x65nd_hash_key\x18\x02 \x01(\x03\x12\x10\n\x08range_id\x18\x03 \x01(\x03\x12\x1c\n\x14storage_container_id\x18\x04 \x01(\x03\"\xda\x01\n\rRangeMetadata\x12\x37\n\x05props\x18\x01 \x01(\x0b\x32(.bookkeeper.proto.stream.RangeProperties\x12\x10\n\x08revision\x18\x02 \x01(\x03\x12\x32\n\x05state\x18\x03 \x01(\x0e\x32#.bookkeeper.proto.stream.RangeState\x12\x13\n\x0b\x63reate_time\x18\n \x01(\x03\x12\x12\n\nfence_time\x18\x0b \x01(\x03\x12\x10\n\x08\x63hildren\x18\x14 \x03(\x03\x12\x0f\n\x07parents\x18\x15 \x03(\x03\":\n\x0cParentRanges\x12\x10\n\x08range_id\x18\x01 \x01(\x03\x12\x18\n\x10parent_range_ids\x18\x02 \x03(\x03\"O\n\x10ParentRangesList\x12;\n\x0c\x63hild_ranges\x18\x01 \x03(\x0b\x32%.bookkeeper.proto.stream.ParentRanges\"+\n\x15\x46ixedRangeSplitPolicy\x12\x12\n\nnum_ranges\x18\x01 \x01(\x05\"\xa5\x01\n\x19\x42\x61ndwidthBasedSplitPolicy\x12\x19\n\x11max_rate_in_bytes\x18\x01 \x01(\x05\x12\x19\n\x11min_rate_in_bytes\x18\x02 \x01(\x05\x12\x1b\n\x13max_rate_in_records\x18\x03 \x01(\x05\x12\x1b\n\x13min_rate_in_records\x18\x04 \x01(\x05\x12\x18\n\x10max_split_factor\x18\x05 \x01(\x05\"\xed\x01\n\x0bSplitPolicy\x12\x36\n\x04type\x18\x01 \x01(\x0e\x32(.bookkeeper.proto.stream.SplitPolicyType\x12L\n\x12\x66ixed_range_policy\x18\x02 \x01(\x0b\x32..bookkeeper.proto.stream.FixedRangeSplitPolicyH\x00\x12N\n\x10\x62\x61ndwidth_policy\x18\x03 \x01(\x0b\x32\x32.bookkeeper.proto.stream.BandwidthBasedSplitPolicyH\x00\x42\x08\n\x06policy\"9\n\x1dSizeBasedSegmentRollingPolicy\x12\x18\n\x10max_segment_size\x18\x01 \x01(\x03\"9\n\x1dTimeBasedSegmentRollingPolicy\x12\x18\n\x10interval_seconds\x18\x01 \x01(\x03\"\xb0\x01\n\x14SegmentRollingPolicy\x12K\n\x0bsize_policy\x18\x01 \x01(\x0b\x32\x36.bookkeeper.proto.stream.SizeBasedSegmentRollingPolicy\x12K\n\x0btime_policy\x18\x02 \x01(\x0b\x32\x36.bookkeeper.proto.stream.TimeBasedSegmentRollingPolicy\"5\n\x18TimeBasedRetentionPolicy\x12\x19\n\x11retention_minutes\x18\x01 \x01(\x03\"Y\n\x0fRetentionPolicy\x12\x46\n\x0btime_policy\x18\x01 \x01(\x0b\x32\x31.bookkeeper.proto.stream.TimeBasedRetentionPolicy\"\x85\x03\n\x13StreamConfiguration\x12\x37\n\x08key_type\x18\x01 \x01(\x0e\x32%.bookkeeper.proto.stream.RangeKeyType\x12\x16\n\x0emin_num_ranges\x18\x02 \x01(\x05\x12\x1a\n\x12initial_num_ranges\x18\x03 \x01(\x05\x12:\n\x0csplit_policy\x18\x04 \x01(\x0b\x32$.bookkeeper.proto.stream.SplitPolicy\x12\x45\n\x0erolling_policy\x18\x05 \x01(\x0b\x32-.bookkeeper.proto.stream.SegmentRollingPolicy\x12\x42\n\x10retention_policy\x18\x06 \x01(\x0b\x32(.bookkeeper.proto.stream.RetentionPolicy\x12:\n\x0cstorage_type\x18\x07 \x01(\x0e\x32$.bookkeeper.proto.stream.StorageType\"\x9b\x01\n\x10StreamProperties\x12\x11\n\tstream_id\x18\x01 \x01(\x03\x12\x1c\n\x14storage_container_id\x18\x02 \x01(\x03\x12\x13\n\x0bstream_name\x18\x03 \x01(\t\x12\x41\n\x0bstream_conf\x18\x04 \x01(\x0b\x32,.bookkeeper.proto.stream.StreamConfiguration\"9\n\nStreamName\x12\x16\n\x0enamespace_name\x18\x01 \x01(\t\x12\x13\n\x0bstream_name\x18\x02 \x01(\t\"\xb5\x03\n\x0eStreamMetadata\x12\x38\n\x05props\x18\x01 \x01(\x0b\x32).bookkeeper.proto.stream.StreamProperties\x12O\n\x0flifecycle_state\x18\x02 \x01(\x0e\x32\x36.bookkeeper.proto.stream.StreamMetadata.LifecycleState\x12K\n\rserving_state\x18\x03 \x01(\x0e\x32\x34.bookkeeper.proto.stream.StreamMetadata.ServingState\x12\x0e\n\x06\x63_time\x18\x04 \x01(\x04\x12\x0e\n\x06m_time\x18\x05 \x01(\x04\x12\x15\n\rnext_range_id\x18\x64 \x01(\x04\x12\x16\n\x0e\x63urrent_ranges\x18\x65 \x03(\x04\"P\n\x0eLifecycleState\x12\n\n\x06UNINIT\x10\x00\x12\x0c\n\x08\x43REATING\x10\x01\x12\x0b\n\x07\x43REATED\x10\x02\x12\x0b\n\x07\x46\x45NCING\x10\x03\x12\n\n\x06\x46\x45NCED\x10\x04\"*\n\x0cServingState\x12\x0c\n\x08WRITABLE\x10\x00\x12\x0c\n\x08READONLY\x10\x01\"c\n\x16NamespaceConfiguration\x12I\n\x13\x64\x65\x66\x61ult_stream_conf\x18\x01 \x01(\x0b\x32,.bookkeeper.proto.stream.StreamConfiguration\"\x8e\x01\n\x13NamespaceProperties\x12\x14\n\x0cnamespace_id\x18\x01 \x01(\x03\x12\x16\n\x0enamespace_name\x18\x02 \x01(\t\x12I\n\x13\x64\x65\x66\x61ult_stream_conf\x18\x03 \x01(\x0b\x32,.bookkeeper.proto.stream.StreamConfiguration\"P\n\x11NamespaceMetadata\x12;\n\x05props\x18\x01 \x01(\x0b\x32,.bookkeeper.proto.stream.NamespaceProperties*C\n\nRangeState\x12\x10\n\x0cRANGE_ACTIVE\x10\x00\x12\x11\n\rRANGE_FENCING\x10\x01\x12\x10\n\x0cRANGE_FENCED\x10\x02*+\n\x0cRangeKeyType\x12\x08\n\x04NULL\x10\x00\x12\x08\n\x04HASH\x10\x01\x12\x07\n\x03RAW\x10\x02*$\n\x0bStorageType\x12\n\n\x06STREAM\x10\x00\x12\t\n\x05TABLE\x10\x01*+\n\x0fSplitPolicyType\x12\t\n\x05\x46IXED\x10\x00\x12\r\n\tBANDWIDTH\x10\x01\x42&\n\"org.apache.bookkeeper.stream.protoP\x01\x62\x06proto3') + , + dependencies=[common__pb2.DESCRIPTOR,]) + +_RANGESTATE = _descriptor.EnumDescriptor( + name='RangeState', + full_name='bookkeeper.proto.stream.RangeState', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='RANGE_ACTIVE', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='RANGE_FENCING', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='RANGE_FENCED', index=2, number=2, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=2924, + serialized_end=2991, +) +_sym_db.RegisterEnumDescriptor(_RANGESTATE) + +RangeState = enum_type_wrapper.EnumTypeWrapper(_RANGESTATE) +_RANGEKEYTYPE = _descriptor.EnumDescriptor( + name='RangeKeyType', + full_name='bookkeeper.proto.stream.RangeKeyType', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='NULL', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='HASH', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='RAW', index=2, number=2, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=2993, + serialized_end=3036, +) +_sym_db.RegisterEnumDescriptor(_RANGEKEYTYPE) + +RangeKeyType = enum_type_wrapper.EnumTypeWrapper(_RANGEKEYTYPE) +_STORAGETYPE = _descriptor.EnumDescriptor( + name='StorageType', + full_name='bookkeeper.proto.stream.StorageType', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='STREAM', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='TABLE', index=1, number=1, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=3038, + serialized_end=3074, +) +_sym_db.RegisterEnumDescriptor(_STORAGETYPE) + +StorageType = enum_type_wrapper.EnumTypeWrapper(_STORAGETYPE) +_SPLITPOLICYTYPE = _descriptor.EnumDescriptor( + name='SplitPolicyType', + full_name='bookkeeper.proto.stream.SplitPolicyType', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='FIXED', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='BANDWIDTH', index=1, number=1, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=3076, + serialized_end=3119, +) +_sym_db.RegisterEnumDescriptor(_SPLITPOLICYTYPE) + +SplitPolicyType = enum_type_wrapper.EnumTypeWrapper(_SPLITPOLICYTYPE) +RANGE_ACTIVE = 0 +RANGE_FENCING = 1 +RANGE_FENCED = 2 +NULL = 0 +HASH = 1 +RAW = 2 +STREAM = 0 +TABLE = 1 +FIXED = 0 +BANDWIDTH = 1 + + +_STREAMMETADATA_LIFECYCLESTATE = _descriptor.EnumDescriptor( + name='LifecycleState', + full_name='bookkeeper.proto.stream.StreamMetadata.LifecycleState', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='UNINIT', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='CREATING', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='CREATED', index=2, number=2, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='FENCING', index=3, number=3, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='FENCED', index=4, number=4, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=2470, + serialized_end=2550, +) +_sym_db.RegisterEnumDescriptor(_STREAMMETADATA_LIFECYCLESTATE) + +_STREAMMETADATA_SERVINGSTATE = _descriptor.EnumDescriptor( + name='ServingState', + full_name='bookkeeper.proto.stream.StreamMetadata.ServingState', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='WRITABLE', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='READONLY', index=1, number=1, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=2552, + serialized_end=2594, +) +_sym_db.RegisterEnumDescriptor(_STREAMMETADATA_SERVINGSTATE) + + +_RANGEID = _descriptor.Descriptor( + name='RangeId', + full_name='bookkeeper.proto.stream.RangeId', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='sc_id', full_name='bookkeeper.proto.stream.RangeId.sc_id', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='stream_id', full_name='bookkeeper.proto.stream.RangeId.stream_id', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='range_id', full_name='bookkeeper.proto.stream.RangeId.range_id', index=2, + number=3, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=55, + serialized_end=116, +) + + +_KEYRANGE = _descriptor.Descriptor( + name='KeyRange', + full_name='bookkeeper.proto.stream.KeyRange', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='start_hash_key', full_name='bookkeeper.proto.stream.KeyRange.start_hash_key', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='end_hash_key', full_name='bookkeeper.proto.stream.KeyRange.end_hash_key', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=118, + serialized_end=174, +) + + +_RANGEPROPERTIES = _descriptor.Descriptor( + name='RangeProperties', + full_name='bookkeeper.proto.stream.RangeProperties', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='start_hash_key', full_name='bookkeeper.proto.stream.RangeProperties.start_hash_key', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='end_hash_key', full_name='bookkeeper.proto.stream.RangeProperties.end_hash_key', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='range_id', full_name='bookkeeper.proto.stream.RangeProperties.range_id', index=2, + number=3, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='storage_container_id', full_name='bookkeeper.proto.stream.RangeProperties.storage_container_id', index=3, + number=4, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=176, + serialized_end=287, +) + + +_RANGEMETADATA = _descriptor.Descriptor( + name='RangeMetadata', + full_name='bookkeeper.proto.stream.RangeMetadata', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='props', full_name='bookkeeper.proto.stream.RangeMetadata.props', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='revision', full_name='bookkeeper.proto.stream.RangeMetadata.revision', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='state', full_name='bookkeeper.proto.stream.RangeMetadata.state', index=2, + number=3, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='create_time', full_name='bookkeeper.proto.stream.RangeMetadata.create_time', index=3, + number=10, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='fence_time', full_name='bookkeeper.proto.stream.RangeMetadata.fence_time', index=4, + number=11, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='children', full_name='bookkeeper.proto.stream.RangeMetadata.children', index=5, + number=20, type=3, cpp_type=2, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='parents', full_name='bookkeeper.proto.stream.RangeMetadata.parents', index=6, + number=21, type=3, cpp_type=2, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=290, + serialized_end=508, +) + + +_PARENTRANGES = _descriptor.Descriptor( + name='ParentRanges', + full_name='bookkeeper.proto.stream.ParentRanges', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='range_id', full_name='bookkeeper.proto.stream.ParentRanges.range_id', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='parent_range_ids', full_name='bookkeeper.proto.stream.ParentRanges.parent_range_ids', index=1, + number=2, type=3, cpp_type=2, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=510, + serialized_end=568, +) + + +_PARENTRANGESLIST = _descriptor.Descriptor( + name='ParentRangesList', + full_name='bookkeeper.proto.stream.ParentRangesList', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='child_ranges', full_name='bookkeeper.proto.stream.ParentRangesList.child_ranges', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=570, + serialized_end=649, +) + + +_FIXEDRANGESPLITPOLICY = _descriptor.Descriptor( + name='FixedRangeSplitPolicy', + full_name='bookkeeper.proto.stream.FixedRangeSplitPolicy', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='num_ranges', full_name='bookkeeper.proto.stream.FixedRangeSplitPolicy.num_ranges', index=0, + number=1, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=651, + serialized_end=694, +) + + +_BANDWIDTHBASEDSPLITPOLICY = _descriptor.Descriptor( + name='BandwidthBasedSplitPolicy', + full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='max_rate_in_bytes', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.max_rate_in_bytes', index=0, + number=1, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='min_rate_in_bytes', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.min_rate_in_bytes', index=1, + number=2, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='max_rate_in_records', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.max_rate_in_records', index=2, + number=3, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='min_rate_in_records', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.min_rate_in_records', index=3, + number=4, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='max_split_factor', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.max_split_factor', index=4, + number=5, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=697, + serialized_end=862, +) + + +_SPLITPOLICY = _descriptor.Descriptor( + name='SplitPolicy', + full_name='bookkeeper.proto.stream.SplitPolicy', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='type', full_name='bookkeeper.proto.stream.SplitPolicy.type', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='fixed_range_policy', full_name='bookkeeper.proto.stream.SplitPolicy.fixed_range_policy', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='bandwidth_policy', full_name='bookkeeper.proto.stream.SplitPolicy.bandwidth_policy', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='policy', full_name='bookkeeper.proto.stream.SplitPolicy.policy', + index=0, containing_type=None, fields=[]), + ], + serialized_start=865, + serialized_end=1102, +) + + +_SIZEBASEDSEGMENTROLLINGPOLICY = _descriptor.Descriptor( + name='SizeBasedSegmentRollingPolicy', + full_name='bookkeeper.proto.stream.SizeBasedSegmentRollingPolicy', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='max_segment_size', full_name='bookkeeper.proto.stream.SizeBasedSegmentRollingPolicy.max_segment_size', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1104, + serialized_end=1161, +) + + +_TIMEBASEDSEGMENTROLLINGPOLICY = _descriptor.Descriptor( + name='TimeBasedSegmentRollingPolicy', + full_name='bookkeeper.proto.stream.TimeBasedSegmentRollingPolicy', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='interval_seconds', full_name='bookkeeper.proto.stream.TimeBasedSegmentRollingPolicy.interval_seconds', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1163, + serialized_end=1220, +) + + +_SEGMENTROLLINGPOLICY = _descriptor.Descriptor( + name='SegmentRollingPolicy', + full_name='bookkeeper.proto.stream.SegmentRollingPolicy', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='size_policy', full_name='bookkeeper.proto.stream.SegmentRollingPolicy.size_policy', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='time_policy', full_name='bookkeeper.proto.stream.SegmentRollingPolicy.time_policy', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1223, + serialized_end=1399, +) + + +_TIMEBASEDRETENTIONPOLICY = _descriptor.Descriptor( + name='TimeBasedRetentionPolicy', + full_name='bookkeeper.proto.stream.TimeBasedRetentionPolicy', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='retention_minutes', full_name='bookkeeper.proto.stream.TimeBasedRetentionPolicy.retention_minutes', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1401, + serialized_end=1454, +) + + +_RETENTIONPOLICY = _descriptor.Descriptor( + name='RetentionPolicy', + full_name='bookkeeper.proto.stream.RetentionPolicy', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='time_policy', full_name='bookkeeper.proto.stream.RetentionPolicy.time_policy', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1456, + serialized_end=1545, +) + + +_STREAMCONFIGURATION = _descriptor.Descriptor( + name='StreamConfiguration', + full_name='bookkeeper.proto.stream.StreamConfiguration', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key_type', full_name='bookkeeper.proto.stream.StreamConfiguration.key_type', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='min_num_ranges', full_name='bookkeeper.proto.stream.StreamConfiguration.min_num_ranges', index=1, + number=2, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='initial_num_ranges', full_name='bookkeeper.proto.stream.StreamConfiguration.initial_num_ranges', index=2, + number=3, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='split_policy', full_name='bookkeeper.proto.stream.StreamConfiguration.split_policy', index=3, + number=4, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='rolling_policy', full_name='bookkeeper.proto.stream.StreamConfiguration.rolling_policy', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='retention_policy', full_name='bookkeeper.proto.stream.StreamConfiguration.retention_policy', index=5, + number=6, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='storage_type', full_name='bookkeeper.proto.stream.StreamConfiguration.storage_type', index=6, + number=7, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1548, + serialized_end=1937, +) + + +_STREAMPROPERTIES = _descriptor.Descriptor( + name='StreamProperties', + full_name='bookkeeper.proto.stream.StreamProperties', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='stream_id', full_name='bookkeeper.proto.stream.StreamProperties.stream_id', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='storage_container_id', full_name='bookkeeper.proto.stream.StreamProperties.storage_container_id', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='stream_name', full_name='bookkeeper.proto.stream.StreamProperties.stream_name', index=2, + number=3, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='stream_conf', full_name='bookkeeper.proto.stream.StreamProperties.stream_conf', index=3, + number=4, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1940, + serialized_end=2095, +) + + +_STREAMNAME = _descriptor.Descriptor( + name='StreamName', + full_name='bookkeeper.proto.stream.StreamName', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='namespace_name', full_name='bookkeeper.proto.stream.StreamName.namespace_name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='stream_name', full_name='bookkeeper.proto.stream.StreamName.stream_name', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2097, + serialized_end=2154, +) + + +_STREAMMETADATA = _descriptor.Descriptor( + name='StreamMetadata', + full_name='bookkeeper.proto.stream.StreamMetadata', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='props', full_name='bookkeeper.proto.stream.StreamMetadata.props', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='lifecycle_state', full_name='bookkeeper.proto.stream.StreamMetadata.lifecycle_state', index=1, + number=2, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='serving_state', full_name='bookkeeper.proto.stream.StreamMetadata.serving_state', index=2, + number=3, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='c_time', full_name='bookkeeper.proto.stream.StreamMetadata.c_time', index=3, + number=4, type=4, cpp_type=4, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='m_time', full_name='bookkeeper.proto.stream.StreamMetadata.m_time', index=4, + number=5, type=4, cpp_type=4, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='next_range_id', full_name='bookkeeper.proto.stream.StreamMetadata.next_range_id', index=5, + number=100, type=4, cpp_type=4, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='current_ranges', full_name='bookkeeper.proto.stream.StreamMetadata.current_ranges', index=6, + number=101, type=4, cpp_type=4, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + _STREAMMETADATA_LIFECYCLESTATE, + _STREAMMETADATA_SERVINGSTATE, + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2157, + serialized_end=2594, +) + + +_NAMESPACECONFIGURATION = _descriptor.Descriptor( + name='NamespaceConfiguration', + full_name='bookkeeper.proto.stream.NamespaceConfiguration', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='default_stream_conf', full_name='bookkeeper.proto.stream.NamespaceConfiguration.default_stream_conf', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2596, + serialized_end=2695, +) + + +_NAMESPACEPROPERTIES = _descriptor.Descriptor( + name='NamespaceProperties', + full_name='bookkeeper.proto.stream.NamespaceProperties', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='namespace_id', full_name='bookkeeper.proto.stream.NamespaceProperties.namespace_id', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='namespace_name', full_name='bookkeeper.proto.stream.NamespaceProperties.namespace_name', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='default_stream_conf', full_name='bookkeeper.proto.stream.NamespaceProperties.default_stream_conf', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2698, + serialized_end=2840, +) + + +_NAMESPACEMETADATA = _descriptor.Descriptor( + name='NamespaceMetadata', + full_name='bookkeeper.proto.stream.NamespaceMetadata', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='props', full_name='bookkeeper.proto.stream.NamespaceMetadata.props', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2842, + serialized_end=2922, +) + +_RANGEMETADATA.fields_by_name['props'].message_type = _RANGEPROPERTIES +_RANGEMETADATA.fields_by_name['state'].enum_type = _RANGESTATE +_PARENTRANGESLIST.fields_by_name['child_ranges'].message_type = _PARENTRANGES +_SPLITPOLICY.fields_by_name['type'].enum_type = _SPLITPOLICYTYPE +_SPLITPOLICY.fields_by_name['fixed_range_policy'].message_type = _FIXEDRANGESPLITPOLICY +_SPLITPOLICY.fields_by_name['bandwidth_policy'].message_type = _BANDWIDTHBASEDSPLITPOLICY +_SPLITPOLICY.oneofs_by_name['policy'].fields.append( + _SPLITPOLICY.fields_by_name['fixed_range_policy']) +_SPLITPOLICY.fields_by_name['fixed_range_policy'].containing_oneof = _SPLITPOLICY.oneofs_by_name['policy'] +_SPLITPOLICY.oneofs_by_name['policy'].fields.append( + _SPLITPOLICY.fields_by_name['bandwidth_policy']) +_SPLITPOLICY.fields_by_name['bandwidth_policy'].containing_oneof = _SPLITPOLICY.oneofs_by_name['policy'] +_SEGMENTROLLINGPOLICY.fields_by_name['size_policy'].message_type = _SIZEBASEDSEGMENTROLLINGPOLICY +_SEGMENTROLLINGPOLICY.fields_by_name['time_policy'].message_type = _TIMEBASEDSEGMENTROLLINGPOLICY +_RETENTIONPOLICY.fields_by_name['time_policy'].message_type = _TIMEBASEDRETENTIONPOLICY +_STREAMCONFIGURATION.fields_by_name['key_type'].enum_type = _RANGEKEYTYPE +_STREAMCONFIGURATION.fields_by_name['split_policy'].message_type = _SPLITPOLICY +_STREAMCONFIGURATION.fields_by_name['rolling_policy'].message_type = _SEGMENTROLLINGPOLICY +_STREAMCONFIGURATION.fields_by_name['retention_policy'].message_type = _RETENTIONPOLICY +_STREAMCONFIGURATION.fields_by_name['storage_type'].enum_type = _STORAGETYPE +_STREAMPROPERTIES.fields_by_name['stream_conf'].message_type = _STREAMCONFIGURATION +_STREAMMETADATA.fields_by_name['props'].message_type = _STREAMPROPERTIES +_STREAMMETADATA.fields_by_name['lifecycle_state'].enum_type = _STREAMMETADATA_LIFECYCLESTATE +_STREAMMETADATA.fields_by_name['serving_state'].enum_type = _STREAMMETADATA_SERVINGSTATE +_STREAMMETADATA_LIFECYCLESTATE.containing_type = _STREAMMETADATA +_STREAMMETADATA_SERVINGSTATE.containing_type = _STREAMMETADATA +_NAMESPACECONFIGURATION.fields_by_name['default_stream_conf'].message_type = _STREAMCONFIGURATION +_NAMESPACEPROPERTIES.fields_by_name['default_stream_conf'].message_type = _STREAMCONFIGURATION +_NAMESPACEMETADATA.fields_by_name['props'].message_type = _NAMESPACEPROPERTIES +DESCRIPTOR.message_types_by_name['RangeId'] = _RANGEID +DESCRIPTOR.message_types_by_name['KeyRange'] = _KEYRANGE +DESCRIPTOR.message_types_by_name['RangeProperties'] = _RANGEPROPERTIES +DESCRIPTOR.message_types_by_name['RangeMetadata'] = _RANGEMETADATA +DESCRIPTOR.message_types_by_name['ParentRanges'] = _PARENTRANGES +DESCRIPTOR.message_types_by_name['ParentRangesList'] = _PARENTRANGESLIST +DESCRIPTOR.message_types_by_name['FixedRangeSplitPolicy'] = _FIXEDRANGESPLITPOLICY +DESCRIPTOR.message_types_by_name['BandwidthBasedSplitPolicy'] = _BANDWIDTHBASEDSPLITPOLICY +DESCRIPTOR.message_types_by_name['SplitPolicy'] = _SPLITPOLICY +DESCRIPTOR.message_types_by_name['SizeBasedSegmentRollingPolicy'] = _SIZEBASEDSEGMENTROLLINGPOLICY +DESCRIPTOR.message_types_by_name['TimeBasedSegmentRollingPolicy'] = _TIMEBASEDSEGMENTROLLINGPOLICY +DESCRIPTOR.message_types_by_name['SegmentRollingPolicy'] = _SEGMENTROLLINGPOLICY +DESCRIPTOR.message_types_by_name['TimeBasedRetentionPolicy'] = _TIMEBASEDRETENTIONPOLICY +DESCRIPTOR.message_types_by_name['RetentionPolicy'] = _RETENTIONPOLICY +DESCRIPTOR.message_types_by_name['StreamConfiguration'] = _STREAMCONFIGURATION +DESCRIPTOR.message_types_by_name['StreamProperties'] = _STREAMPROPERTIES +DESCRIPTOR.message_types_by_name['StreamName'] = _STREAMNAME +DESCRIPTOR.message_types_by_name['StreamMetadata'] = _STREAMMETADATA +DESCRIPTOR.message_types_by_name['NamespaceConfiguration'] = _NAMESPACECONFIGURATION +DESCRIPTOR.message_types_by_name['NamespaceProperties'] = _NAMESPACEPROPERTIES +DESCRIPTOR.message_types_by_name['NamespaceMetadata'] = _NAMESPACEMETADATA +DESCRIPTOR.enum_types_by_name['RangeState'] = _RANGESTATE +DESCRIPTOR.enum_types_by_name['RangeKeyType'] = _RANGEKEYTYPE +DESCRIPTOR.enum_types_by_name['StorageType'] = _STORAGETYPE +DESCRIPTOR.enum_types_by_name['SplitPolicyType'] = _SPLITPOLICYTYPE +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +RangeId = _reflection.GeneratedProtocolMessageType('RangeId', (_message.Message,), dict( + DESCRIPTOR = _RANGEID, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.RangeId) + )) +_sym_db.RegisterMessage(RangeId) + +KeyRange = _reflection.GeneratedProtocolMessageType('KeyRange', (_message.Message,), dict( + DESCRIPTOR = _KEYRANGE, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.KeyRange) + )) +_sym_db.RegisterMessage(KeyRange) + +RangeProperties = _reflection.GeneratedProtocolMessageType('RangeProperties', (_message.Message,), dict( + DESCRIPTOR = _RANGEPROPERTIES, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.RangeProperties) + )) +_sym_db.RegisterMessage(RangeProperties) + +RangeMetadata = _reflection.GeneratedProtocolMessageType('RangeMetadata', (_message.Message,), dict( + DESCRIPTOR = _RANGEMETADATA, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.RangeMetadata) + )) +_sym_db.RegisterMessage(RangeMetadata) + +ParentRanges = _reflection.GeneratedProtocolMessageType('ParentRanges', (_message.Message,), dict( + DESCRIPTOR = _PARENTRANGES, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.ParentRanges) + )) +_sym_db.RegisterMessage(ParentRanges) + +ParentRangesList = _reflection.GeneratedProtocolMessageType('ParentRangesList', (_message.Message,), dict( + DESCRIPTOR = _PARENTRANGESLIST, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.ParentRangesList) + )) +_sym_db.RegisterMessage(ParentRangesList) + +FixedRangeSplitPolicy = _reflection.GeneratedProtocolMessageType('FixedRangeSplitPolicy', (_message.Message,), dict( + DESCRIPTOR = _FIXEDRANGESPLITPOLICY, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.FixedRangeSplitPolicy) + )) +_sym_db.RegisterMessage(FixedRangeSplitPolicy) + +BandwidthBasedSplitPolicy = _reflection.GeneratedProtocolMessageType('BandwidthBasedSplitPolicy', (_message.Message,), dict( + DESCRIPTOR = _BANDWIDTHBASEDSPLITPOLICY, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.BandwidthBasedSplitPolicy) + )) +_sym_db.RegisterMessage(BandwidthBasedSplitPolicy) + +SplitPolicy = _reflection.GeneratedProtocolMessageType('SplitPolicy', (_message.Message,), dict( + DESCRIPTOR = _SPLITPOLICY, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.SplitPolicy) + )) +_sym_db.RegisterMessage(SplitPolicy) + +SizeBasedSegmentRollingPolicy = _reflection.GeneratedProtocolMessageType('SizeBasedSegmentRollingPolicy', (_message.Message,), dict( + DESCRIPTOR = _SIZEBASEDSEGMENTROLLINGPOLICY, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.SizeBasedSegmentRollingPolicy) + )) +_sym_db.RegisterMessage(SizeBasedSegmentRollingPolicy) + +TimeBasedSegmentRollingPolicy = _reflection.GeneratedProtocolMessageType('TimeBasedSegmentRollingPolicy', (_message.Message,), dict( + DESCRIPTOR = _TIMEBASEDSEGMENTROLLINGPOLICY, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.TimeBasedSegmentRollingPolicy) + )) +_sym_db.RegisterMessage(TimeBasedSegmentRollingPolicy) + +SegmentRollingPolicy = _reflection.GeneratedProtocolMessageType('SegmentRollingPolicy', (_message.Message,), dict( + DESCRIPTOR = _SEGMENTROLLINGPOLICY, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.SegmentRollingPolicy) + )) +_sym_db.RegisterMessage(SegmentRollingPolicy) + +TimeBasedRetentionPolicy = _reflection.GeneratedProtocolMessageType('TimeBasedRetentionPolicy', (_message.Message,), dict( + DESCRIPTOR = _TIMEBASEDRETENTIONPOLICY, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.TimeBasedRetentionPolicy) + )) +_sym_db.RegisterMessage(TimeBasedRetentionPolicy) + +RetentionPolicy = _reflection.GeneratedProtocolMessageType('RetentionPolicy', (_message.Message,), dict( + DESCRIPTOR = _RETENTIONPOLICY, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.RetentionPolicy) + )) +_sym_db.RegisterMessage(RetentionPolicy) + +StreamConfiguration = _reflection.GeneratedProtocolMessageType('StreamConfiguration', (_message.Message,), dict( + DESCRIPTOR = _STREAMCONFIGURATION, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.StreamConfiguration) + )) +_sym_db.RegisterMessage(StreamConfiguration) + +StreamProperties = _reflection.GeneratedProtocolMessageType('StreamProperties', (_message.Message,), dict( + DESCRIPTOR = _STREAMPROPERTIES, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.StreamProperties) + )) +_sym_db.RegisterMessage(StreamProperties) + +StreamName = _reflection.GeneratedProtocolMessageType('StreamName', (_message.Message,), dict( + DESCRIPTOR = _STREAMNAME, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.StreamName) + )) +_sym_db.RegisterMessage(StreamName) + +StreamMetadata = _reflection.GeneratedProtocolMessageType('StreamMetadata', (_message.Message,), dict( + DESCRIPTOR = _STREAMMETADATA, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.StreamMetadata) + )) +_sym_db.RegisterMessage(StreamMetadata) + +NamespaceConfiguration = _reflection.GeneratedProtocolMessageType('NamespaceConfiguration', (_message.Message,), dict( + DESCRIPTOR = _NAMESPACECONFIGURATION, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.NamespaceConfiguration) + )) +_sym_db.RegisterMessage(NamespaceConfiguration) + +NamespaceProperties = _reflection.GeneratedProtocolMessageType('NamespaceProperties', (_message.Message,), dict( + DESCRIPTOR = _NAMESPACEPROPERTIES, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.NamespaceProperties) + )) +_sym_db.RegisterMessage(NamespaceProperties) + +NamespaceMetadata = _reflection.GeneratedProtocolMessageType('NamespaceMetadata', (_message.Message,), dict( + DESCRIPTOR = _NAMESPACEMETADATA, + __module__ = 'stream_pb2' + # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.NamespaceMetadata) + )) +_sym_db.RegisterMessage(NamespaceMetadata) + + +DESCRIPTOR._options = None +# @@protoc_insertion_point(module_scope) diff --git a/stream/clients/python/bookkeeper/types.py b/stream/clients/python/bookkeeper/types.py new file mode 100644 index 00000000000..e9643299787 --- /dev/null +++ b/stream/clients/python/bookkeeper/types.py @@ -0,0 +1,71 @@ +# Licensed 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. + +from __future__ import absolute_import +import collections +import sys + +from google.protobuf import descriptor_pb2 +from google.protobuf import duration_pb2 +from google.protobuf import empty_pb2 +from google.protobuf import field_mask_pb2 +from google.protobuf import timestamp_pb2 +from bookkeeper.proto import common_pb2 +from bookkeeper.proto import kv_rpc_pb2 +from bookkeeper.proto import stream_pb2 +from bookkeeper.proto import storage_pb2 + +from bookkeeper.common.protobuf_helpers import get_messages + + +# Define the default values for storage client settings. +# +# This class is used when creating a bookkeeper client, and +# these settings can be altered to tweak client behavior. +# The defaults should be fine for most use cases. +StorageClientSettings = collections.namedtuple( + 'StorageClientSettings', + ['service_uri'], +) +StorageClientSettings.__new__.__defaults__ = ( + "bk://localhost:4181", # bookkeeper service uri +) + +_shared_modules = [ + descriptor_pb2, + duration_pb2, + empty_pb2, + field_mask_pb2, + timestamp_pb2, +] + +_local_modules = [ + common_pb2, + kv_rpc_pb2, + stream_pb2, + storage_pb2 +] + +names = ['StorageClientSettings'] + +for module in _shared_modules: + for name, message in get_messages(module).items(): + setattr(sys.modules[__name__], name, message) + names.append(name) + +for module in _local_modules: + for name, message in get_messages(module).items(): + message.__module__ = 'bookkeeper.types' + setattr(sys.modules[__name__], name, message) + names.append(name) + +__all__ = tuple(sorted(names)) diff --git a/stream/clients/python/examples/admin.py b/stream/clients/python/examples/admin.py new file mode 100644 index 00000000000..d39d50cf5c7 --- /dev/null +++ b/stream/clients/python/examples/admin.py @@ -0,0 +1,58 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +from bookkeeper import admin + +ns_name = "test" +ns_name_2 = "test2" +stream_name = "test_stream" +stream_name_2 = "test_stream_2" + +client = admin.Client() + +# create first namespace +ns_resp = client.namespaces().create(ns_name) +print("Created first namespace '%s' : %s" % (ns_name, ns_resp)) + +# create second namespace +ns_resp = client.namespaces().create(ns_name_2) +print("Created second namespace '%s' : %s" % (ns_name_2, ns_resp)) + +# get first namespace +ns_props = client.namespaces().get(ns_name) +print("Get first namespace '%s' : %s" % (ns_name, ns_props)) +ns_props = client.namespaces().get(ns_name_2) +print("Get second namespace '%s' : %s" % (ns_name_2, ns_props)) + +# test operations on namespace 'test' +ns = client.namespace(ns_name) +stream_resp = ns.create(stream_name) +print("Create first stream '%s' : %s" % (stream_name, stream_resp)) +stream_resp = ns.create(stream_name_2) +print("Create second stream '%s' : %s" % (stream_name_2, stream_resp)) + +stream_props = ns.get(stream_name) +print("Get first stream '%s' : %s" % (stream_name, stream_props)) +stream_props = ns.get(stream_name_2) +print("Get second stream '%s' : %s" % (stream_name_2, stream_props)) + +del_resp = ns.delete(stream_name) +print("Delete first stream '%s' : %s" % (stream_name, del_resp)) +del_resp = ns.delete(stream_name_2) +print("Delete second stream '%s' : %s" % (stream_name_2, del_resp)) + +del_resp = client.namespaces().delete(ns_name) +print("Delete first namespace '%s' : %s" % (ns_name, del_resp)) +del_resp = client.namespaces().delete(ns_name_2) +print("Delete second namespace '%s' : %s" % (ns_name_2, del_resp)) diff --git a/stream/clients/python/noxfile.py b/stream/clients/python/noxfile.py new file mode 100644 index 00000000000..0d25fa2a1e1 --- /dev/null +++ b/stream/clients/python/noxfile.py @@ -0,0 +1,86 @@ +# Licensed 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. + +from __future__ import absolute_import + +import os + +import nox + + +LOCAL_DEPS = ( +) + + +@nox.session +def default(session): + """Default unit test session. + This is intended to be run **without** an interpreter set, so + that the current ``python`` (on the ``PATH``) or the version of + Python corresponding to the ``nox`` binary the ``PATH`` can + run the tests. + """ + # Install all test dependencies, then install local packages in-place. + session.install('mock', 'pytest', 'pytest-cov') + for local_dep in LOCAL_DEPS: + session.install('-e', local_dep) + session.install('-e', '.') + + # Run py.test against the unit tests. + session.run( + 'py.test', + '--quiet', + '--cov-append', + '--cov-report=', + '--cov=bookkeeper', + '--cov-config=.coveragerc', + os.path.join('tests', 'unit'), + *session.posargs + ) + + +@nox.session +@nox.parametrize('py', ['3.7']) +def unit(session, py): + """Run the unit test suite.""" + default(session) + + +@nox.session +def lint(session): + """Run linters. + Returns a failure if the linters find linting errors or sufficiently + serious code quality issues. + """ + session.install('flake8', *LOCAL_DEPS) + session.install('.') + session.run('flake8', 'bookkeeper', 'tests') + + +@nox.session +def lint_setup_py(session): + """Verify that setup.py is valid (including RST check).""" + session.install('docutils', 'Pygments') + session.run( + 'python', 'setup.py', 'check', '--restructuredtext', '--strict') + + +# TODO: Enable coverage report +# @nox.session +def cover(session): + """Run the final coverage report. + This outputs the coverage report aggregating coverage from the unit + test runs (not system test runs), and then erases coverage data. + """ + session.install('coverage', 'pytest-cov') + session.run('coverage', 'report', '--show-missing', '--fail-under=100') + session.run('coverage', 'erase') diff --git a/stream/clients/python/pylint.config.py b/stream/clients/python/pylint.config.py new file mode 100644 index 00000000000..a4f86b4500c --- /dev/null +++ b/stream/clients/python/pylint.config.py @@ -0,0 +1,29 @@ +# Licensed 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. + +"""This module is used to configure gcp-devrel-py-tools run-pylint.""" + +import copy + +from gcp_devrel.tools import pylint + +# Library configuration + +# library_additions = {} +# Ignore generated code +library_replacements = copy.deepcopy(pylint.DEFAULT_LIBRARY_RC_REPLACEMENTS) +library_replacements['MASTER']['ignore'].append('proto') + +# Test configuration + +# test_additions = copy.deepcopy(library_additions) +# test_replacements = copy.deepcopy(library_replacements) \ No newline at end of file diff --git a/stream/clients/python/setup.cfg b/stream/clients/python/setup.cfg new file mode 100644 index 00000000000..d3eef106e10 --- /dev/null +++ b/stream/clients/python/setup.cfg @@ -0,0 +1,14 @@ +# Licensed 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. + +[bdist_wheel] +universal = 1 diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py new file mode 100644 index 00000000000..5fd3e62a449 --- /dev/null +++ b/stream/clients/python/setup.py @@ -0,0 +1,87 @@ +# Licensed 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. + +import io +import os + +import setuptools + +# Package metadata. + +name = 'bookkeeper' +description = 'Apache BookKeeper client library' +version = '4.9.0' +# Should be one of: +# 'Development Status :: 3 - Alpha' +# 'Development Status :: 4 - Beta' +# 'Development Status :: 5 - Production/Stable' +release_status = 'Development Status :: 3 - Alpha' +dependencies = [ + 'protobuf>=3.0.0', + 'requests<3.0.0dev,>=2.18.0', + 'setuptools>=34.0.0', + 'six>=1.10.0', + 'pytz', + 'futures>=3.2.0;python_version<"3.2"', + 'grpcio>=1.8.2', +] +extras = { +} + +# Setup boilerplate below this line. + +package_root = os.path.abspath(os.path.dirname(__file__)) + +readme_filename = os.path.join(package_root, 'README.md') +with io.open(readme_filename, encoding='utf-8') as readme_file: + readme = readme_file.read() + +# Only include packages under the 'bookkeeper' namespace. Do not include tests, +# benchmarks, etc. +packages = [ + package for package in setuptools.find_packages() + if package.startswith('bookkeeper')] + +# Determine which namespaces are needed. +namespaces = ['bookkeeper'] + +setuptools.setup( + name=name, + version=version, + description=description, + long_description=readme, + author='Apache BookKeeper', + author_email='dev@bookkeeper.apache.org', + license='Apache 2.0', + url='https://github.com/apache/bookkeeper/tree/master/stream/clients/python', + classifiers=[ + release_status, + 'Intended Audience :: Developers', + 'License :: Apache Software License', + 'Programming Language :: Python', + 'Programming Language :: Python :: 2', + 'Programming Language :: Python :: 2.7', + 'Programming Language :: Python :: 3', + 'Programming Language :: Python :: 3.4', + 'Programming Language :: Python :: 3.5', + 'Programming Language :: Python :: 3.6', + 'Operating System :: OS Independent', + 'Topic :: Internet', + ], + platforms='Posix; MacOS X; Windows', + packages=packages, + namespace_packages=namespaces, + install_requires=dependencies, + extras_require=extras, + include_package_data=True, + zip_safe=False, +) diff --git a/stream/clients/python/tests/unit/__init__.py b/stream/clients/python/tests/unit/__init__.py new file mode 100644 index 00000000000..4d9a92490ba --- /dev/null +++ b/stream/clients/python/tests/unit/__init__.py @@ -0,0 +1,11 @@ +# Licensed 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. diff --git a/stream/clients/python/tests/unit/bookkeeper/common/future/test__helpers.py b/stream/clients/python/tests/unit/bookkeeper/common/future/test__helpers.py new file mode 100644 index 00000000000..804e3bef880 --- /dev/null +++ b/stream/clients/python/tests/unit/bookkeeper/common/future/test__helpers.py @@ -0,0 +1,35 @@ +# Licensed 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. + +import mock + +from bookkeeper.common.future import _helpers + + +@mock.patch('threading.Thread', autospec=True) +def test_start_deamon_thread(unused_thread): + deamon_thread = _helpers.start_daemon_thread(target=mock.sentinel.target) + assert deamon_thread.daemon is True + + +def test_safe_invoke_callback(): + callback = mock.Mock(spec=['__call__'], return_value=42) + result = _helpers.safe_invoke_callback(callback, 'a', b='c') + assert result == 42 + callback.assert_called_once_with('a', b='c') + + +def test_safe_invoke_callback_exception(): + callback = mock.Mock(spec=['__call__'], side_effect=ValueError()) + result = _helpers.safe_invoke_callback(callback, 'a', b='c') + assert result is None + callback.assert_called_once_with('a', b='c') diff --git a/stream/clients/python/tests/unit/bookkeeper/common/future/test_polling.py b/stream/clients/python/tests/unit/bookkeeper/common/future/test_polling.py new file mode 100644 index 00000000000..2ab7568e1ea --- /dev/null +++ b/stream/clients/python/tests/unit/bookkeeper/common/future/test_polling.py @@ -0,0 +1,155 @@ +# Licensed 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. + +import concurrent.futures +import threading +import time + +import mock +import pytest + +from bookkeeper.common.future import polling + + +class PollingFutureImpl(polling.PollingFuture): + def done(self): + return False + + def cancel(self): + return True + + def cancelled(self): + return False + + def running(self): + return True + + +def test_polling_future_constructor(): + future = PollingFutureImpl() + assert not future.done() + assert not future.cancelled() + assert future.running() + assert future.cancel() + + +def test_set_result(): + future = PollingFutureImpl() + callback = mock.Mock() + + future.set_result(1) + + assert future.result() == 1 + future.add_done_callback(callback) + callback.assert_called_once_with(future) + + +def test_set_exception(): + future = PollingFutureImpl() + exception = ValueError('meep') + + future.set_exception(exception) + + assert future.exception() == exception + with pytest.raises(ValueError): + future.result() + + callback = mock.Mock() + future.add_done_callback(callback) + callback.assert_called_once_with(future) + + +def test_invoke_callback_exception(): + future = PollingFutureImplWithPoll() + future.set_result(42) + + # This should not raise, despite the callback causing an exception. + callback = mock.Mock(side_effect=ValueError) + future.add_done_callback(callback) + callback.assert_called_once_with(future) + + +class PollingFutureImplWithPoll(PollingFutureImpl): + def __init__(self): + super(PollingFutureImplWithPoll, self).__init__() + self.poll_count = 0 + self.event = threading.Event() + + def done(self): + self.poll_count += 1 + self.event.wait() + self.set_result(42) + return True + + +def test_result_with_polling(): + future = PollingFutureImplWithPoll() + + future.event.set() + result = future.result() + + assert result == 42 + assert future.poll_count == 1 + # Repeated calls should not cause additional polling + assert future.result() == result + assert future.poll_count == 1 + + +class PollingFutureImplTimeout(PollingFutureImplWithPoll): + def done(self): + time.sleep(1) + return False + + +def test_result_timeout(): + future = PollingFutureImplTimeout() + with pytest.raises(concurrent.futures.TimeoutError): + future.result(timeout=1) + + +def test_callback_background_thread(): + future = PollingFutureImplWithPoll() + callback = mock.Mock() + + future.add_done_callback(callback) + + assert future._polling_thread is not None + + # Give the thread a second to poll + time.sleep(1) + assert future.poll_count == 1 + + future.event.set() + future._polling_thread.join() + + callback.assert_called_once_with(future) + + +def test_double_callback_background_thread(): + future = PollingFutureImplWithPoll() + callback = mock.Mock() + callback2 = mock.Mock() + + future.add_done_callback(callback) + current_thread = future._polling_thread + assert current_thread is not None + + # only one polling thread should be created. + future.add_done_callback(callback2) + assert future._polling_thread is current_thread + + future.event.set() + future._polling_thread.join() + + assert future.poll_count == 1 + callback.assert_called_once_with(future) + callback2.assert_called_once_with(future) diff --git a/stream/clients/python/tests/unit/bookkeeper/common/test_grpc_helpers.py b/stream/clients/python/tests/unit/bookkeeper/common/test_grpc_helpers.py new file mode 100644 index 00000000000..ad9daf8b396 --- /dev/null +++ b/stream/clients/python/tests/unit/bookkeeper/common/test_grpc_helpers.py @@ -0,0 +1,172 @@ +# Licensed 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. + +import grpc +import mock +import pytest + +from bookkeeper.common import exceptions +from bookkeeper.common import grpc_helpers + + +def test__patch_callable_name(): + callable = mock.Mock(spec=['__class__']) + callable.__class__ = mock.Mock(spec=['__name__']) + callable.__class__.__name__ = 'TestCallable' + + grpc_helpers._patch_callable_name(callable) + + assert callable.__name__ == 'TestCallable' + + +def test__patch_callable_name_no_op(): + callable = mock.Mock(spec=['__name__']) + callable.__name__ = 'test_callable' + + grpc_helpers._patch_callable_name(callable) + + assert callable.__name__ == 'test_callable' + + +class RpcErrorImpl(grpc.RpcError, grpc.Call): + def __init__(self, code): + super(RpcErrorImpl, self).__init__() + self._code = code + + def code(self): + return self._code + + def details(self): + return None + + +def test_wrap_unary_errors(): + grpc_error = RpcErrorImpl(grpc.StatusCode.INTERNAL) + callable_ = mock.Mock(spec=['__call__'], side_effect=grpc_error) + + wrapped_callable = grpc_helpers._wrap_unary_errors(callable_) + + with pytest.raises(exceptions.InternalServerError) as exc_info: + wrapped_callable(1, 2, three='four') + + callable_.assert_called_once_with(1, 2, three='four') + assert exc_info.value.response == grpc_error + + +def test_wrap_stream_okay(): + expected_responses = [1, 2, 3] + callable_ = mock.Mock(spec=[ + '__call__'], return_value=iter(expected_responses)) + + wrapped_callable = grpc_helpers._wrap_stream_errors(callable_) + + got_iterator = wrapped_callable(1, 2, three='four') + + responses = list(got_iterator) + + callable_.assert_called_once_with(1, 2, three='four') + assert responses == expected_responses + + +def test_wrap_stream_iterable_iterface(): + response_iter = mock.create_autospec(grpc.Call, instance=True) + callable_ = mock.Mock(spec=['__call__'], return_value=response_iter) + + wrapped_callable = grpc_helpers._wrap_stream_errors(callable_) + + got_iterator = wrapped_callable() + + callable_.assert_called_once_with() + + # Check each aliased method in the grpc.Call interface + got_iterator.add_callback(mock.sentinel.callback) + response_iter.add_callback.assert_called_once_with(mock.sentinel.callback) + + got_iterator.cancel() + response_iter.cancel.assert_called_once_with() + + got_iterator.code() + response_iter.code.assert_called_once_with() + + got_iterator.details() + response_iter.details.assert_called_once_with() + + got_iterator.initial_metadata() + response_iter.initial_metadata.assert_called_once_with() + + got_iterator.is_active() + response_iter.is_active.assert_called_once_with() + + got_iterator.time_remaining() + response_iter.time_remaining.assert_called_once_with() + + got_iterator.trailing_metadata() + response_iter.trailing_metadata.assert_called_once_with() + + +def test_wrap_stream_errors_invocation(): + grpc_error = RpcErrorImpl(grpc.StatusCode.INTERNAL) + callable_ = mock.Mock(spec=['__call__'], side_effect=grpc_error) + + wrapped_callable = grpc_helpers._wrap_stream_errors(callable_) + + with pytest.raises(exceptions.InternalServerError) as exc_info: + wrapped_callable(1, 2, three='four') + + callable_.assert_called_once_with(1, 2, three='four') + assert exc_info.value.response == grpc_error + + +class RpcResponseIteratorImpl(object): + def __init__(self, exception): + self._exception = exception + + def next(self): + raise self._exception + + __next__ = next + + +def test_wrap_stream_errors_iterator(): + grpc_error = RpcErrorImpl(grpc.StatusCode.INTERNAL) + response_iter = RpcResponseIteratorImpl(grpc_error) + callable_ = mock.Mock(spec=['__call__'], return_value=response_iter) + + wrapped_callable = grpc_helpers._wrap_stream_errors(callable_) + + got_iterator = wrapped_callable(1, 2, three='four') + + with pytest.raises(exceptions.InternalServerError) as exc_info: + next(got_iterator) + + callable_.assert_called_once_with(1, 2, three='four') + assert exc_info.value.response == grpc_error + + +@mock.patch('bookkeeper.common.grpc_helpers._wrap_unary_errors') +def test_wrap_errors_non_streaming(wrap_unary_errors): + callable_ = mock.create_autospec(grpc.UnaryUnaryMultiCallable) + + result = grpc_helpers.wrap_errors(callable_) + + assert result == wrap_unary_errors.return_value + wrap_unary_errors.assert_called_once_with(callable_) + + +@mock.patch('bookkeeper.common.grpc_helpers._wrap_stream_errors') +def test_wrap_errors_streaming(wrap_stream_errors): + callable_ = mock.create_autospec(grpc.UnaryStreamMultiCallable) + + result = grpc_helpers.wrap_errors(callable_) + + assert result == wrap_stream_errors.return_value + wrap_stream_errors.assert_called_once_with(callable_) diff --git a/stream/clients/python/tests/unit/bookkeeper/common/test_protobuf_helpers.py b/stream/clients/python/tests/unit/bookkeeper/common/test_protobuf_helpers.py new file mode 100644 index 00000000000..9851f18171d --- /dev/null +++ b/stream/clients/python/tests/unit/bookkeeper/common/test_protobuf_helpers.py @@ -0,0 +1,274 @@ +# Licensed 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. + +import pytest + +from bookkeeper.common import protobuf_helpers +from bookkeeper.proto import common_pb2 +from bookkeeper.proto import kv_pb2 + +from google.protobuf import any_pb2 +from google.protobuf import message +from google.protobuf import source_context_pb2 +from google.protobuf import struct_pb2 +from google.protobuf import timestamp_pb2 +from google.protobuf import type_pb2 + + +def test_from_any_pb_success(): + in_message = common_pb2.Endpoint(port=5181) + in_message_any = any_pb2.Any() + in_message_any.Pack(in_message) + out_message =\ + protobuf_helpers.from_any_pb(common_pb2.Endpoint, in_message_any) + + assert in_message == out_message + + +def test_from_any_pb_failure(): + in_message = any_pb2.Any() + in_message.Pack(common_pb2.Endpoint(port=5181)) + + with pytest.raises(TypeError): + protobuf_helpers.from_any_pb(kv_pb2.KeyValue, in_message) + + +def test_check_protobuf_helpers_ok(): + assert protobuf_helpers.check_oneof() is None + assert protobuf_helpers.check_oneof(foo='bar') is None + assert protobuf_helpers.check_oneof(foo='bar', baz=None) is None + assert protobuf_helpers.check_oneof(foo=None, baz='bacon') is None + assert (protobuf_helpers.check_oneof(foo='bar', spam=None, eggs=None) + is None) + + +def test_check_protobuf_helpers_failures(): + with pytest.raises(ValueError): + protobuf_helpers.check_oneof(foo='bar', spam='eggs') + with pytest.raises(ValueError): + protobuf_helpers.check_oneof(foo='bar', baz='bacon', spam='eggs') + with pytest.raises(ValueError): + protobuf_helpers.check_oneof(foo='bar', spam=0, eggs=None) + + +def test_get_messages(): + kv = protobuf_helpers.get_messages(kv_pb2) + + # Ensure that Date was exported properly. + assert kv['KeyValue'] is kv_pb2.KeyValue + + # Ensure that no non-Message objects were exported. + for value in kv.values(): + assert issubclass(value, message.Message) + + +def test_get_dict_absent(): + with pytest.raises(KeyError): + assert protobuf_helpers.get({}, 'foo') + + +def test_get_dict_present(): + assert protobuf_helpers.get({'foo': 'bar'}, 'foo') == 'bar' + + +def test_get_dict_default(): + assert protobuf_helpers.get({}, 'foo', default='bar') == 'bar' + + +def test_get_dict_nested(): + assert protobuf_helpers.get({'foo': {'bar': 'baz'}}, 'foo.bar') == 'baz' + + +def test_get_dict_nested_default(): + assert protobuf_helpers.get({}, 'foo.baz', default='bacon') == 'bacon' + assert ( + protobuf_helpers.get({'foo': {}}, 'foo.baz', default='bacon') == + 'bacon') + + +def test_get_msg_sentinel(): + msg = timestamp_pb2.Timestamp() + with pytest.raises(KeyError): + assert protobuf_helpers.get(msg, 'foo') + + +def test_get_msg_present(): + msg = timestamp_pb2.Timestamp(seconds=42) + assert protobuf_helpers.get(msg, 'seconds') == 42 + + +def test_get_msg_default(): + msg = timestamp_pb2.Timestamp() + assert protobuf_helpers.get(msg, 'foo', default='bar') == 'bar' + + +def test_invalid_object(): + with pytest.raises(TypeError): + protobuf_helpers.get(object(), 'foo', 'bar') + + +def test_set_dict(): + mapping = {} + protobuf_helpers.set(mapping, 'foo', 'bar') + assert mapping == {'foo': 'bar'} + + +def test_set_msg(): + msg = timestamp_pb2.Timestamp() + protobuf_helpers.set(msg, 'seconds', 42) + assert msg.seconds == 42 + + +def test_set_dict_nested(): + mapping = {} + protobuf_helpers.set(mapping, 'foo.bar', 'baz') + assert mapping == {'foo': {'bar': 'baz'}} + + +def test_set_invalid_object(): + with pytest.raises(TypeError): + protobuf_helpers.set(object(), 'foo', 'bar') + + +def test_setdefault_dict_unset(): + mapping = {} + protobuf_helpers.setdefault(mapping, 'foo', 'bar') + assert mapping == {'foo': 'bar'} + + +def test_setdefault_dict_falsy(): + mapping = {'foo': None} + protobuf_helpers.setdefault(mapping, 'foo', 'bar') + assert mapping == {'foo': 'bar'} + + +def test_setdefault_dict_truthy(): + mapping = {'foo': 'bar'} + protobuf_helpers.setdefault(mapping, 'foo', 'baz') + assert mapping == {'foo': 'bar'} + + +def test_field_mask_singular_field_diffs(): + original = type_pb2.Type(name='name') + modified = type_pb2.Type() + assert (protobuf_helpers.field_mask(original, modified).paths == + ['name']) + + original = type_pb2.Type(name='name') + modified = type_pb2.Type() + assert (protobuf_helpers.field_mask(original, modified).paths == + ['name']) + + original = None + modified = type_pb2.Type(name='name') + assert (protobuf_helpers.field_mask(original, modified).paths == + ['name']) + + original = type_pb2.Type(name='name') + modified = None + assert (protobuf_helpers.field_mask(original, modified).paths == + ['name']) + + +def test_field_mask_message_diffs(): + original = type_pb2.Type() + modified = type_pb2.Type(source_context=source_context_pb2.SourceContext( + file_name='name')) + assert (protobuf_helpers.field_mask(original, modified).paths == + ['source_context.file_name']) + + original = type_pb2.Type(source_context=source_context_pb2.SourceContext( + file_name='name')) + modified = type_pb2.Type() + assert (protobuf_helpers.field_mask(original, modified).paths == + ['source_context']) + + original = type_pb2.Type(source_context=source_context_pb2.SourceContext( + file_name='name')) + modified = type_pb2.Type(source_context=source_context_pb2.SourceContext( + file_name='other_name')) + assert (protobuf_helpers.field_mask(original, modified).paths == + ['source_context.file_name']) + + original = None + modified = type_pb2.Type(source_context=source_context_pb2.SourceContext( + file_name='name')) + assert (protobuf_helpers.field_mask(original, modified).paths == + ['source_context.file_name']) + + original = type_pb2.Type(source_context=source_context_pb2.SourceContext( + file_name='name')) + modified = None + assert (protobuf_helpers.field_mask(original, modified).paths == + ['source_context']) + + +def test_field_mask_repeated_diffs(): + original = struct_pb2.ListValue() + modified = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0), + struct_pb2.Value(number_value=2.0)]) + assert protobuf_helpers.field_mask(original, modified).paths == ['values'] + + original = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0), + struct_pb2.Value(number_value=2.0)]) + modified = struct_pb2.ListValue() + assert protobuf_helpers.field_mask(original, modified).paths == ['values'] + + original = None + modified = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0), + struct_pb2.Value(number_value=2.0)]) + assert protobuf_helpers.field_mask(original, modified).paths == ['values'] + + original = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0), + struct_pb2.Value(number_value=2.0)]) + modified = None + assert protobuf_helpers.field_mask(original, modified).paths == ['values'] + + original = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0), + struct_pb2.Value(number_value=2.0)]) + modified = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=2.0), + struct_pb2.Value(number_value=1.0)]) + assert protobuf_helpers.field_mask(original, modified).paths == ['values'] + + +def test_field_mask_map_diffs(): + original = struct_pb2.Struct() + modified = struct_pb2.Struct( + fields={'foo': struct_pb2.Value(number_value=1.0)}) + assert protobuf_helpers.field_mask(original, modified).paths == ['fields'] + + original = struct_pb2.Struct( + fields={'foo': struct_pb2.Value(number_value=1.0)}) + modified = struct_pb2.Struct() + assert protobuf_helpers.field_mask(original, modified).paths == ['fields'] + + original = None + modified = struct_pb2.Struct( + fields={'foo': struct_pb2.Value(number_value=1.0)}) + assert protobuf_helpers.field_mask(original, modified).paths == ['fields'] + + original = struct_pb2.Struct( + fields={'foo': struct_pb2.Value(number_value=1.0)}) + modified = None + assert protobuf_helpers.field_mask(original, modified).paths == ['fields'] + + original = struct_pb2.Struct( + fields={'foo': struct_pb2.Value(number_value=1.0)}) + modified = struct_pb2.Struct( + fields={'foo': struct_pb2.Value(number_value=2.0)}) + assert protobuf_helpers.field_mask(original, modified).paths == ['fields'] + + original = struct_pb2.Struct( + fields={'foo': struct_pb2.Value(number_value=1.0)}) + modified = struct_pb2.Struct( + fields={'bar': struct_pb2.Value(number_value=1.0)}) + assert protobuf_helpers.field_mask(original, modified).paths == ['fields'] diff --git a/stream/clients/python/tests/unit/bookkeeper/test_futures.py b/stream/clients/python/tests/unit/bookkeeper/test_futures.py new file mode 100644 index 00000000000..4f95c4cb09b --- /dev/null +++ b/stream/clients/python/tests/unit/bookkeeper/test_futures.py @@ -0,0 +1,145 @@ +# Licensed 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. + +import threading + +import mock +import pytest + +from bookkeeper.kv import futures, exceptions + + +def _future(*args, **kwargs): + return futures.Future(*args, **kwargs) + + +def test_constructor_defaults(): + with mock.patch.object(threading, 'Event', autospec=True) as Event: + future = _future() + + assert future._result == futures.Future._SENTINEL + assert future._exception == futures.Future._SENTINEL + assert future._callbacks == [] + assert future._completed is Event.return_value + + Event.assert_called_once_with() + + +def test_constructor_explicit_completed(): + completed = mock.sentinel.completed + future = _future(completed=completed) + + assert future._result == futures.Future._SENTINEL + assert future._exception == futures.Future._SENTINEL + assert future._callbacks == [] + assert future._completed is completed + + +def test_cancel(): + assert _future().cancel() is False + + +def test_cancelled(): + assert _future().cancelled() is False + + +def test_running(): + future = _future() + assert future.running() is True + future.set_result('foobar') + assert future.running() is False + + +def test_done(): + future = _future() + assert future.done() is False + future.set_result('12345') + assert future.done() is True + + +def test_exception_no_error(): + future = _future() + future.set_result('12345') + assert future.exception() is None + + +def test_exception_with_error(): + future = _future() + error = RuntimeError('Something really bad happened.') + future.set_exception(error) + + # Make sure that the exception that is returned is the batch's error. + # Also check the type to ensure the batch's error did not somehow + # change internally. + assert future.exception() is error + assert isinstance(future.exception(), RuntimeError) + with pytest.raises(RuntimeError): + future.result() + + +def test_exception_timeout(): + future = _future() + with pytest.raises(exceptions.TimeoutError): + future.exception(timeout=0.01) + + +def test_result_no_error(): + future = _future() + future.set_result('42') + assert future.result() == '42' + + +def test_result_with_error(): + future = _future() + future.set_exception(RuntimeError('Something really bad happened.')) + with pytest.raises(RuntimeError): + future.result() + + +def test_add_done_callback_pending_batch(): + future = _future() + callback = mock.Mock() + future.add_done_callback(callback) + assert len(future._callbacks) == 1 + assert callback in future._callbacks + assert callback.call_count == 0 + + +def test_add_done_callback_completed_batch(): + future = _future() + future.set_result('12345') + callback = mock.Mock(spec=()) + future.add_done_callback(callback) + callback.assert_called_once_with(future) + + +def test_trigger(): + future = _future() + callback = mock.Mock(spec=()) + future.add_done_callback(callback) + assert callback.call_count == 0 + future.set_result('12345') + callback.assert_called_once_with(future) + + +def test_set_result_once_only(): + future = _future() + future.set_result('12345') + with pytest.raises(RuntimeError): + future.set_result('67890') + + +def test_set_exception_once_only(): + future = _future() + future.set_exception(ValueError('wah wah')) + with pytest.raises(RuntimeError): + future.set_exception(TypeError('other wah wah')) From f0fbca412d320236d905832499b2b79b27379a59 Mon Sep 17 00:00:00 2001 From: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Date: Sun, 23 Sep 2018 23:09:37 -0700 Subject: [PATCH 0041/1642] autorecovery-use-metadata-driver (part 1) : move AutoRecoveryMain to use MetadataBookieDriver Descriptions of the changes in this PR: ### Motivation We are introducing Etcd as the metadata storage. However AutoRecovery currently is still tied to zookeeper. In order to use Etcd as the metadata storage, we have to move AutoRecovery related classes to use metadata driver API. ### Changes This is the first change for changing AutoRecovery to use metadata driver. It changed AutoRecoveryMain to use metadata driver api and also removed the shutdown logic on session expired, which doesn't make any sense for current retryable zookeeper. Author: Qi Wang Reviewers: Sijie Guo , Enrico Olivelli This closes #1693 from codingwangqi/autorecovery_metadata --- .../replication/AutoRecoveryMain.java | 87 +++++++++---------- .../replication/AutoRecoveryMainTest.java | 33 ------- 2 files changed, 39 insertions(+), 81 deletions(-) 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 9830c592904..3369de8b3da 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,6 +20,7 @@ */ package org.apache.bookkeeper.replication; +import static com.google.common.base.Preconditions.checkArgument; import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_WORKER_SCOPE; @@ -28,8 +29,7 @@ import java.io.File; import java.io.IOException; import java.net.MalformedURLException; -import java.util.HashSet; -import java.util.Set; +import java.net.URI; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieCriticalThread; @@ -37,13 +37,15 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.http.HttpServer; import org.apache.bookkeeper.http.HttpServerLoader; -import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.server.http.BKHttpServiceProvider; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.HelpFormatter; @@ -51,8 +53,6 @@ import org.apache.commons.cli.ParseException; import org.apache.commons.configuration.ConfigurationException; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,12 +66,12 @@ public class AutoRecoveryMain { private static final Logger LOG = LoggerFactory .getLogger(AutoRecoveryMain.class); - private ServerConfiguration conf; - ZooKeeper zk; - AuditorElector auditorElector; - ReplicationWorker replicationWorker; - private AutoRecoveryDeathWatcher deathWatcher; - private int exitCode; + private final ServerConfiguration conf; + final MetadataBookieDriver metadataBookieDriver; + final AuditorElector auditorElector; + final ReplicationWorker replicationWorker; + final AutoRecoveryDeathWatcher deathWatcher; + int exitCode; private volatile boolean shuttingDown = false; private volatile boolean running = false; @@ -85,40 +85,36 @@ public AutoRecoveryMain(ServerConfiguration conf, StatsLogger statsLogger) throws IOException, InterruptedException, KeeperException, UnavailableException, CompatibilityException { this.conf = conf; - Set watchers = new HashSet(); - // TODO: better session handling for auto recovery daemon https://issues.apache.org/jira/browse/BOOKKEEPER-594 - // since {@link org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager} - // use Watcher, need to ensure the logic works correctly after recreating - // a new zookeeper client when session expired. - // for now just shutdown it. - watchers.add(new Watcher() { - @Override - public void process(WatchedEvent event) { - // Check for expired connection. - if (event.getState().equals(Watcher.Event.KeeperState.Expired)) { - LOG.error("ZK client connection to the ZK server has expired!"); - shutdown(ExitCode.ZK_EXPIRED); - } - } - }); - zk = ZooKeeperClient.newBuilder() - .connectString(ZKMetadataDriverBase.resolveZkServers(conf)) - .sessionTimeoutMs(conf.getZkTimeout()) - .watchers(watchers) - .build(); + this.metadataBookieDriver = initializeMetadataDriver(conf, statsLogger); + auditorElector = new AuditorElector(Bookie.getBookieAddress(conf).toString(), conf, - zk, statsLogger.scope(AUDITOR_SCOPE)); - replicationWorker = new ReplicationWorker(zk, conf, statsLogger.scope(REPLICATION_WORKER_SCOPE)); + getZooKeeperFromMetadataDriver(metadataBookieDriver), + statsLogger.scope(AUDITOR_SCOPE)); + replicationWorker = new ReplicationWorker( + getZooKeeperFromMetadataDriver(metadataBookieDriver), + conf, + statsLogger.scope(REPLICATION_WORKER_SCOPE)); deathWatcher = new AutoRecoveryDeathWatcher(this); } - public AutoRecoveryMain(ServerConfiguration conf, ZooKeeper zk) throws IOException, InterruptedException, - KeeperException, UnavailableException, CompatibilityException { - this.conf = conf; - this.zk = zk; - auditorElector = new AuditorElector(Bookie.getBookieAddress(conf).toString(), conf, zk); - replicationWorker = new ReplicationWorker(zk, conf); - deathWatcher = new AutoRecoveryDeathWatcher(this); + private MetadataBookieDriver initializeMetadataDriver(ServerConfiguration conf, StatsLogger statsLogger) + throws IOException { + String metadataServiceUri = conf.getMetadataServiceUriUnchecked(); + MetadataBookieDriver driver = MetadataDrivers.getBookieDriver( + URI.create(metadataServiceUri)); + try { + driver.initialize(conf, () -> {}, statsLogger); + } catch (MetadataException e) { + throw new IOException("Failed to initialize metadata driver at " + metadataServiceUri, e); + } + return driver; + } + + // it existing because AuditorElector takes zookeeper + ZooKeeper getZooKeeperFromMetadataDriver(MetadataBookieDriver driver) { + checkArgument(driver instanceof ZKMetadataBookieDriver); + ZKMetadataBookieDriver zkDriver = (ZKMetadataBookieDriver) driver; + return zkDriver.getZk(); } /* @@ -169,12 +165,7 @@ private void shutdown(int exitCode) { LOG.warn("Interrupted shutting down auditor elector", e); } replicationWorker.shutdown(); - try { - zk.close(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.warn("Interrupted shutting down auto recovery", e); - } + metadataBookieDriver.close(); } private int getExitCode() { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java index 2970d3c5665..8bb8049ef09 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java @@ -74,37 +74,4 @@ public void testShutdown() throws Exception { main.replicationWorker.isRunning()); } - /** - * Test that, if an autorecovery looses its ZK connection/session - * it will shutdown. - */ - @Test - public void testAutoRecoverySessionLoss() throws Exception { - AutoRecoveryMain main1 = new AutoRecoveryMain(bsConfs.get(0)); - AutoRecoveryMain main2 = new AutoRecoveryMain(bsConfs.get(1)); - main1.start(); - main2.start(); - Thread.sleep(500); - assertTrue("AuditorElectors should be running", - main1.auditorElector.isRunning() && main2.auditorElector.isRunning()); - assertTrue("Replication workers should be running", - main1.replicationWorker.isRunning() && main2.replicationWorker.isRunning()); - - zkUtil.expireSession(main1.zk); - zkUtil.expireSession(main2.zk); - - for (int i = 0; i < 10; i++) { // give it 10 seconds to shutdown - if (!main1.auditorElector.isRunning() - && !main2.auditorElector.isRunning() - && !main1.replicationWorker.isRunning() - && !main2.replicationWorker.isRunning()) { - break; - } - Thread.sleep(1000); - } - assertFalse("Elector1 should have shutdown", main1.auditorElector.isRunning()); - assertFalse("Elector2 should have shutdown", main2.auditorElector.isRunning()); - assertFalse("RW1 should have shutdown", main1.replicationWorker.isRunning()); - assertFalse("RW2 should have shutdown", main2.replicationWorker.isRunning()); - } } From 31aa848f3e772276747eb48d3fd9dd185e2e183f Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Mon, 24 Sep 2018 00:18:21 -0700 Subject: [PATCH 0042/1642] Enable storeSystemTimeAsLedgerUnderreplicatedMarkTime Descriptions of the changes in this PR: - Enable the Auditor to use system time as underreplicated ledger mark time. If this is enabled, Auditor will write a ctime field into the underreplicated ledger znode. https://github.com/apache/bookkeeper/commit/f4094c4992d5b22630fa633085a00a9152b87ffe introduced feature to Record ctime for underreplicated ledger mark time. Author: Sijie Guo Author: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Author: cguttapalem Author: Charan Reddy Guttapalem Reviewers: Sijie Guo , Enrico Olivelli This closes #1692 from reddycharan/enablectime --- .../java/org/apache/bookkeeper/conf/AbstractConfiguration.java | 2 +- conf/bk_server.conf | 2 +- site/_data/config/bk_server.yaml | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) 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 666b377b30b..a3c47abda7a 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 @@ -851,7 +851,7 @@ public T setStoreSystemTimeAsLedgerUnderreplicatedMarkTime(boolean enabled) { * underreplicated ledger mark time. */ public boolean getStoreSystemTimeAsLedgerUnderreplicatedMarkTime() { - return getBoolean(STORE_SYSTEMTIME_AS_LEDGER_UNDERREPLICATED_MARK_TIME, false); + return getBoolean(STORE_SYSTEMTIME_AS_LEDGER_UNDERREPLICATED_MARK_TIME, true); } /** diff --git a/conf/bk_server.conf b/conf/bk_server.conf index 93c12a142cd..97404fb1bb4 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -911,7 +911,7 @@ zkEnableSecurity=false # Enable the Auditor to use system time as underreplicated ledger mark time. # If this is enabled, Auditor will write a ctime field into the underreplicated ledger znode. -# storeSystemTimeAsLedgerUnderreplicatedMarkTime=false +# storeSystemTimeAsLedgerUnderreplicatedMarkTime=true ############################################################################# ## Replication Worker settings diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index 991d6f1fceb..b60bfbf9930 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -648,6 +648,7 @@ groups: default: 0 - param: storeSystemTimeAsLedgerUnderreplicatedMarkTime description: Enable the Auditor to use system time as underreplicated ledger mark time. If this is enabled, Auditor will write a ctime field into the underreplicated ledger znode. + default: true - name: AutoRecovery replication worker settings params: From 2cebbdea03d439c18a6c1abbb3254b0a62be6fce Mon Sep 17 00:00:00 2001 From: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Date: Mon, 24 Sep 2018 19:15:33 -0700 Subject: [PATCH 0043/1642] autorecovery-use-metadata-driver (part 2) : remove ZooKeeper from ReplicationWorker Descriptions of the changes in this PR: *Changes* This is the second part of changing AutoRecovery to use metadata driver. It removes the zookeeper reference from ReplicationWorker, because the zookeeper instance is not used anyway This change is based on #1693 Author: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Author: Qi Wang Author: Sijie Guo Author: Charan Reddy Guttapalem Reviewers: Sijie Guo , Enrico Olivelli This closes #1694 from codingwangqi/autorecovery_metadata_part2 --- .../replication/AutoRecoveryMain.java | 1 - .../replication/ReplicationWorker.java | 15 ++-------- .../replication/AuditorPeriodicCheckTest.java | 3 +- .../replication/TestReplicationWorker.java | 28 ++++++++----------- 4 files changed, 15 insertions(+), 32 deletions(-) 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 3369de8b3da..b43ff850e69 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 @@ -91,7 +91,6 @@ public AutoRecoveryMain(ServerConfiguration conf, StatsLogger statsLogger) getZooKeeperFromMetadataDriver(metadataBookieDriver), statsLogger.scope(AUDITOR_SCOPE)); replicationWorker = new ReplicationWorker( - getZooKeeperFromMetadataDriver(metadataBookieDriver), conf, statsLogger.scope(REPLICATION_WORKER_SCOPE)); deathWatcher = new AutoRecoveryDeathWatcher(this); 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 d5c85f16107..b536fb09725 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 @@ -67,7 +67,6 @@ import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,7 +82,6 @@ public class ReplicationWorker implements Runnable { private final LedgerUnderreplicationManager underreplicationManager; private final ServerConfiguration conf; - private final ZooKeeper zkc; private volatile boolean workerRunning = false; private final BookKeeperAdmin admin; private final LedgerChecker ledgerChecker; @@ -107,16 +105,13 @@ public class ReplicationWorker implements Runnable { * UnderReplicationManager to the targetBookie. This target bookie will be a * local bookie. * - * @param zkc - * - ZK instance * @param conf * - configurations */ - public ReplicationWorker(final ZooKeeper zkc, - final ServerConfiguration conf) + public ReplicationWorker(final ServerConfiguration conf) throws CompatibilityException, KeeperException, InterruptedException, IOException { - this(zkc, conf, NullStatsLogger.INSTANCE); + this(conf, NullStatsLogger.INSTANCE); } /** @@ -124,19 +119,15 @@ public ReplicationWorker(final ZooKeeper zkc, * UnderReplicationManager to the targetBookie. This target bookie will be a * local bookie. * - * @param zkc - * - ZK instance * @param conf * - configurations * @param statsLogger * - stats logger */ - public ReplicationWorker(final ZooKeeper zkc, - final ServerConfiguration conf, + public ReplicationWorker(final ServerConfiguration conf, StatsLogger statsLogger) throws CompatibilityException, KeeperException, InterruptedException, IOException { - this.zkc = zkc; this.conf = conf; try { this.bkc = BookKeeper.forConfig(new ClientConfiguration(conf)) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 4e3bda8ebae..0668088ed51 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -449,8 +449,7 @@ public void testFailedWriteRecovery() throws Exception { // now start the replication workers List l = new ArrayList(); for (int i = 0; i < numBookies; i++) { - ReplicationWorker rw = new ReplicationWorker( - zkc, bsConfs.get(i), NullStatsLogger.INSTANCE); + ReplicationWorker rw = new ReplicationWorker(bsConfs.get(i), NullStatsLogger.INSTANCE); rw.start(); l.add(rw); } 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 e4b1232e775..a4b280a3b7f 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 @@ -52,7 +52,6 @@ import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; -import org.apache.zookeeper.ZooKeeper; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -153,7 +152,7 @@ public void testRWShouldReplicateFragmentsToTargetBookie() throws Exception { lh.addEntry(data); } - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); + ReplicationWorker rw = new ReplicationWorker(baseConf); rw.start(); try { @@ -198,7 +197,7 @@ public void testRWShouldRetryUntilThereAreEnoughBksAvailableForReplication() LOG.info("New Bookie addr :" + newBkAddr); killAllBookies(lh, newBkAddr); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); + ReplicationWorker rw = new ReplicationWorker(baseConf); rw.start(); try { @@ -248,16 +247,12 @@ public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplicatio // Starte RW1 BookieSocketAddress newBkAddr1 = startNewBookieAndReturnAddress(); LOG.info("New Bookie addr : {}", newBkAddr1); - ReplicationWorker rw1 = new ReplicationWorker(zkc, baseConf); + ReplicationWorker rw1 = new ReplicationWorker(baseConf); // Starte RW2 BookieSocketAddress newBkAddr2 = startNewBookieAndReturnAddress(); LOG.info("New Bookie addr : {}", newBkAddr2); - ZooKeeper zkc1 = ZooKeeperClient.newBuilder() - .connectString(zkUtil.getZooKeeperConnectString()) - .sessionTimeoutMs(10000) - .build(); - ReplicationWorker rw2 = new ReplicationWorker(zkc1, baseConf); + ReplicationWorker rw2 = new ReplicationWorker(baseConf); rw1.start(); rw2.start(); @@ -283,7 +278,6 @@ public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplicatio } finally { rw1.shutdown(); rw2.shutdown(); - zkc1.close(); } } @@ -308,7 +302,7 @@ public void testRWShouldCleanTheLedgerFromUnderReplicationIfLedgerAlreadyDeleted BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); LOG.info("New Bookie addr : {}", newBkAddr); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); + ReplicationWorker rw = new ReplicationWorker(baseConf); rw.start(); try { @@ -363,7 +357,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); LOG.info("New Bookie addr : {}", newBkAddr); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); + ReplicationWorker rw = new ReplicationWorker(baseConf); rw.start(); try { @@ -419,7 +413,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR() // set to 3s instead of default 30s baseConf.setOpenLedgerRereplicationGracePeriod("3000"); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); + ReplicationWorker rw = new ReplicationWorker(baseConf); @Cleanup MetadataClientDriver clientDriver = MetadataDrivers.getClientDriver( URI.create(baseClientConf.getMetadataServiceUri())); @@ -489,8 +483,8 @@ public void testBookiesNotAvailableScenarioForReplicationWorker() throws Excepti // create couple of replicationworkers baseConf.setLockReleaseOfFailedLedgerGracePeriod("500"); - ReplicationWorker rw1 = new ReplicationWorker(zkc, baseConf); - ReplicationWorker rw2 = new ReplicationWorker(zkc, baseConf); + ReplicationWorker rw1 = new ReplicationWorker(baseConf); + ReplicationWorker rw2 = new ReplicationWorker(baseConf); @Cleanup MetadataClientDriver clientDriver = MetadataDrivers @@ -610,7 +604,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR() lh.addEntry(data); } - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); + ReplicationWorker rw = new ReplicationWorker(baseConf); baseClientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); @@ -660,7 +654,7 @@ public void testRWZKConnectionLost() throws Exception { .sessionTimeoutMs(10000) .build()) { - ReplicationWorker rw = new ReplicationWorker(zk, baseConf); + ReplicationWorker rw = new ReplicationWorker(baseConf); rw.start(); for (int i = 0; i < 10; i++) { if (rw.isRunning()) { From d67d7ae99af32b1c82c8dd80fe83957cc12f448c Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 25 Sep 2018 10:10:37 -0700 Subject: [PATCH 0044/1642] [TABLE SERVICE] [PYTHON] add hash router for computing the hash routing value for a given key Descriptions of the changes in this PR: *Motivation* As part of implementing python client for #1690, we need a hash router to compute the hash routing value for a given key. *Changes* - Murmur2 is not well supported in python 3. so standarized hash computation in using Murmur3. - Change java hash router to use Murmur3 - Add python hash router - Add unit tests to cover the hash value computation and make sure it is consistent across java and python Master Issue: #1690 Author: Reviewers: Jia Zhai This closes #1698 from sijie/message_router --- .../resources/bookkeeper/suppressions.xml | 1 + .../bookkeeper/common/router/__init__.py | 17 + .../python/bookkeeper/common/router/router.py | 25 + stream/clients/python/setup.py | 1 + .../bookkeeper/common/router/test_router.py | 19 + .../bookkeeper/common/hash/Murmur3.java | 443 ++++++++++++++++++ .../common/router/AbstractHashRouter.java | 9 +- .../common/router/HashRouterTest.java | 64 +++ 8 files changed, 574 insertions(+), 5 deletions(-) create mode 100644 stream/clients/python/bookkeeper/common/router/__init__.py create mode 100644 stream/clients/python/bookkeeper/common/router/router.py create mode 100644 stream/clients/python/tests/unit/bookkeeper/common/router/test_router.py create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/hash/Murmur3.java create mode 100644 stream/common/src/test/java/org/apache/bookkeeper/common/router/HashRouterTest.java diff --git a/buildtools/src/main/resources/bookkeeper/suppressions.xml b/buildtools/src/main/resources/bookkeeper/suppressions.xml index c45e2fa2b32..232f19b083d 100644 --- a/buildtools/src/main/resources/bookkeeper/suppressions.xml +++ b/buildtools/src/main/resources/bookkeeper/suppressions.xml @@ -25,6 +25,7 @@ + diff --git a/stream/clients/python/bookkeeper/common/router/__init__.py b/stream/clients/python/bookkeeper/common/router/__init__.py new file mode 100644 index 00000000000..d7329cf23c4 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/router/__init__.py @@ -0,0 +1,17 @@ +# Licensed 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. + +from bookkeeper.common.router.router import BytesHashRouter + +__all__ = [ + 'BytesHashRouter', +] diff --git a/stream/clients/python/bookkeeper/common/router/router.py b/stream/clients/python/bookkeeper/common/router/router.py new file mode 100644 index 00000000000..de4f31dc692 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/router/router.py @@ -0,0 +1,25 @@ +# Licensed 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. + +import mmh3 + +__SEED__ = 383242705 + + +class BytesHashRouter(object): + """A router that computes hash values for keys using MurmurHash3""" + + def __init__(self): + return + + def getRoutingKey(self, key): + return mmh3.hash64(key, seed=__SEED__, signed=True)[0] diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index 5fd3e62a449..49afd7a424c 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -33,6 +33,7 @@ 'pytz', 'futures>=3.2.0;python_version<"3.2"', 'grpcio>=1.8.2', + 'mmh3>=2.5.1' ] extras = { } diff --git a/stream/clients/python/tests/unit/bookkeeper/common/router/test_router.py b/stream/clients/python/tests/unit/bookkeeper/common/router/test_router.py new file mode 100644 index 00000000000..55e9eda30de --- /dev/null +++ b/stream/clients/python/tests/unit/bookkeeper/common/router/test_router.py @@ -0,0 +1,19 @@ +# Licensed 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. + +from bookkeeper.common.router import BytesHashRouter + + +def test_get_routing_hash(): + router = BytesHashRouter() + hash_key = router.getRoutingKey("foo") + assert hash_key == -2336792112830167536 diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/hash/Murmur3.java b/stream/common/src/main/java/org/apache/bookkeeper/common/hash/Murmur3.java new file mode 100644 index 00000000000..47852ba7649 --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/hash/Murmur3.java @@ -0,0 +1,443 @@ +/* + * Licensed 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.common.hash; + +import io.netty.buffer.ByteBuf; + +/** + * Murmur3 is successor to Murmur2 fast non-crytographic hash algorithms. + * + * Murmur3 32 and 128 bit variants. + * 32-bit Java port of https://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp#94 + * 128-bit Java port of https://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp#255 + * + * This is a public domain code with no copyrights. + * From homepage of MurmurHash (https://code.google.com/p/smhasher/), + * "All MurmurHash versions are public domain software, and the author disclaims all copyright + * to their code." + */ +public class Murmur3 { + // Constants for 32 bit variant + private static final int C1_32 = 0xcc9e2d51; + private static final int C2_32 = 0x1b873593; + private static final int R1_32 = 15; + private static final int R2_32 = 13; + private static final int M_32 = 5; + private static final int N_32 = 0xe6546b64; + + // Constants for 128 bit variant + private static final long C1 = 0x87c37b91114253d5L; + private static final long C2 = 0x4cf5ad432745937fL; + private static final int R1 = 31; + private static final int R2 = 27; + private static final int R3 = 33; + private static final int M = 5; + private static final int N1 = 0x52dce729; + private static final int N2 = 0x38495ab5; + + public static final int DEFAULT_SEED = 104729; + + /** + * Murmur3 32-bit variant. + * + * @param data - input byte array + * @return - hashcode + */ + public static int hash32(byte[] data) { + return hash32(data, 0, data.length, DEFAULT_SEED); + } + + /** + * Murmur3 32-bit variant. + * + * @param data - input byte array + * @param length - length of array + * @return - hashcode + */ + public static int hash32(byte[] data, int length) { + return hash32(data, 0, length, DEFAULT_SEED); + } + + /** + * Murmur3 32-bit variant. + * + * @param data - input byte array + * @param length - length of array + * @param seed - seed. (default 0) + * @return - hashcode + */ + public static int hash32(byte[] data, int length, int seed) { + return hash32(data, 0, length, seed); + } + + /** + * Murmur3 32-bit variant. + * + * @param data - input byte array + * @param offset - offset of data + * @param length - length of array + * @param seed - seed. (default 0) + * @return - hashcode + */ + public static int hash32(byte[] data, int offset, int length, int seed) { + int hash = seed; + final int nblocks = length >> 2; + + // body + for (int i = 0; i < nblocks; i++) { + int i_4 = i << 2; + int k = (data[offset + i_4] & 0xff) + | ((data[offset + i_4 + 1] & 0xff) << 8) + | ((data[offset + i_4 + 2] & 0xff) << 16) + | ((data[offset + i_4 + 3] & 0xff) << 24); + + // mix functions + k *= C1_32; + k = Integer.rotateLeft(k, R1_32); + k *= C2_32; + hash ^= k; + hash = Integer.rotateLeft(hash, R2_32) * M_32 + N_32; + } + + // tail + int idx = nblocks << 2; + int k1 = 0; + switch (length - idx) { + case 3: + k1 ^= data[offset + idx + 2] << 16; + case 2: + k1 ^= data[offset + idx + 1] << 8; + case 1: + k1 ^= data[offset + idx]; + + // mix functions + k1 *= C1_32; + k1 = Integer.rotateLeft(k1, R1_32); + k1 *= C2_32; + hash ^= k1; + } + + // finalization + hash ^= length; + hash ^= (hash >>> 16); + hash *= 0x85ebca6b; + hash ^= (hash >>> 13); + hash *= 0xc2b2ae35; + hash ^= (hash >>> 16); + + return hash; + } + + public static int hash32(ByteBuf data, int offset, int length, int seed) { + int hash = seed; + final int nblocks = length >> 2; + + // body + for (int i = 0; i < nblocks; i++) { + int i_4 = i << 2; + int k = (data.getByte(offset + i_4) & 0xff) + | ((data.getByte(offset + i_4 + 1) & 0xff) << 8) + | ((data.getByte(offset + i_4 + 2) & 0xff) << 16) + | ((data.getByte(offset + i_4 + 3) & 0xff) << 24); + + // mix functions + k *= C1_32; + k = Integer.rotateLeft(k, R1_32); + k *= C2_32; + hash ^= k; + hash = Integer.rotateLeft(hash, R2_32) * M_32 + N_32; + } + + // tail + int idx = nblocks << 2; + int k1 = 0; + switch (length - idx) { + case 3: + k1 ^= data.getByte(offset + idx + 2) << 16; + case 2: + k1 ^= data.getByte(offset + idx + 1) << 8; + case 1: + k1 ^= data.getByte(offset + idx); + + // mix functions + k1 *= C1_32; + k1 = Integer.rotateLeft(k1, R1_32); + k1 *= C2_32; + hash ^= k1; + } + + // finalization + hash ^= length; + hash ^= (hash >>> 16); + hash *= 0x85ebca6b; + hash ^= (hash >>> 13); + hash *= 0xc2b2ae35; + hash ^= (hash >>> 16); + + return hash; + } + + /** + * Murmur3 128-bit variant. + * + * @param data - input byte array + * @return - hashcode (2 longs) + */ + public static long[] hash128(byte[] data) { + return hash128(data, 0, data.length, DEFAULT_SEED); + } + + /** + * Murmur3 128-bit variant. + * + * @param data - input byte array + * @param offset - the first element of array + * @param length - length of array + * @param seed - seed. (default is 0) + * @return - hashcode (2 longs) + */ + public static long[] hash128(byte[] data, int offset, int length, int seed) { + long h1 = seed; + long h2 = seed; + final int nblocks = length >> 4; + + // body + for (int i = 0; i < nblocks; i++) { + final int i16 = i << 4; + long k1 = ((long) data[offset + i16] & 0xff) + | (((long) data[offset + i16 + 1] & 0xff) << 8) + | (((long) data[offset + i16 + 2] & 0xff) << 16) + | (((long) data[offset + i16 + 3] & 0xff) << 24) + | (((long) data[offset + i16 + 4] & 0xff) << 32) + | (((long) data[offset + i16 + 5] & 0xff) << 40) + | (((long) data[offset + i16 + 6] & 0xff) << 48) + | (((long) data[offset + i16 + 7] & 0xff) << 56); + + long k2 = ((long) data[offset + i16 + 8] & 0xff) + | (((long) data[offset + i16 + 9] & 0xff) << 8) + | (((long) data[offset + i16 + 10] & 0xff) << 16) + | (((long) data[offset + i16 + 11] & 0xff) << 24) + | (((long) data[offset + i16 + 12] & 0xff) << 32) + | (((long) data[offset + i16 + 13] & 0xff) << 40) + | (((long) data[offset + i16 + 14] & 0xff) << 48) + | (((long) data[offset + i16 + 15] & 0xff) << 56); + + // mix functions for k1 + k1 *= C1; + k1 = Long.rotateLeft(k1, R1); + k1 *= C2; + h1 ^= k1; + h1 = Long.rotateLeft(h1, R2); + h1 += h2; + h1 = h1 * M + N1; + + // mix functions for k2 + k2 *= C2; + k2 = Long.rotateLeft(k2, R3); + k2 *= C1; + h2 ^= k2; + h2 = Long.rotateLeft(h2, R1); + h2 += h1; + h2 = h2 * M + N2; + } + + // tail + long k1 = 0; + long k2 = 0; + int tailStart = nblocks << 4; + switch (length - tailStart) { + case 15: + k2 ^= (long) (data[offset + tailStart + 14] & 0xff) << 48; + case 14: + k2 ^= (long) (data[offset + tailStart + 13] & 0xff) << 40; + case 13: + k2 ^= (long) (data[offset + tailStart + 12] & 0xff) << 32; + case 12: + k2 ^= (long) (data[offset + tailStart + 11] & 0xff) << 24; + case 11: + k2 ^= (long) (data[offset + tailStart + 10] & 0xff) << 16; + case 10: + k2 ^= (long) (data[offset + tailStart + 9] & 0xff) << 8; + case 9: + k2 ^= (long) (data[offset + tailStart + 8] & 0xff); + k2 *= C2; + k2 = Long.rotateLeft(k2, R3); + k2 *= C1; + h2 ^= k2; + + case 8: + k1 ^= (long) (data[offset + tailStart + 7] & 0xff) << 56; + case 7: + k1 ^= (long) (data[offset + tailStart + 6] & 0xff) << 48; + case 6: + k1 ^= (long) (data[offset + tailStart + 5] & 0xff) << 40; + case 5: + k1 ^= (long) (data[offset + tailStart + 4] & 0xff) << 32; + case 4: + k1 ^= (long) (data[offset + tailStart + 3] & 0xff) << 24; + case 3: + k1 ^= (long) (data[offset + tailStart + 2] & 0xff) << 16; + case 2: + k1 ^= (long) (data[offset + tailStart + 1] & 0xff) << 8; + case 1: + k1 ^= (long) (data[offset + tailStart] & 0xff); + k1 *= C1; + k1 = Long.rotateLeft(k1, R1); + k1 *= C2; + h1 ^= k1; + } + + // finalization + h1 ^= length; + h2 ^= length; + + h1 += h2; + h2 += h1; + + h1 = fmix64(h1); + h2 = fmix64(h2); + + h1 += h2; + h2 += h1; + + return new long[]{h1, h2}; + } + + /** + * Murmur3 128-bit variant. + * + * @param data - input byte array + * @param offset - the first element of array + * @param length - length of array + * @param seed - seed. (default is 0) + * @return - hashcode (2 longs) + */ + public static long[] hash128(ByteBuf data, int offset, int length, long seed) { + long h1 = seed; + long h2 = seed; + final int nblocks = length >> 4; + + // body + for (int i = 0; i < nblocks; i++) { + final int i16 = i << 4; + long k1 = ((long) data.getByte(offset + i16) & 0xff) + | (((long) data.getByte(offset + i16 + 1) & 0xff) << 8) + | (((long) data.getByte(offset + i16 + 2) & 0xff) << 16) + | (((long) data.getByte(offset + i16 + 3) & 0xff) << 24) + | (((long) data.getByte(offset + i16 + 4) & 0xff) << 32) + | (((long) data.getByte(offset + i16 + 5) & 0xff) << 40) + | (((long) data.getByte(offset + i16 + 6) & 0xff) << 48) + | (((long) data.getByte(offset + i16 + 7) & 0xff) << 56); + + long k2 = ((long) data.getByte(offset + i16 + 8) & 0xff) + | (((long) data.getByte(offset + i16 + 9) & 0xff) << 8) + | (((long) data.getByte(offset + i16 + 10) & 0xff) << 16) + | (((long) data.getByte(offset + i16 + 11) & 0xff) << 24) + | (((long) data.getByte(offset + i16 + 12) & 0xff) << 32) + | (((long) data.getByte(offset + i16 + 13) & 0xff) << 40) + | (((long) data.getByte(offset + i16 + 14) & 0xff) << 48) + | (((long) data.getByte(offset + i16 + 15) & 0xff) << 56); + + // mix functions for k1 + k1 *= C1; + k1 = Long.rotateLeft(k1, R1); + k1 *= C2; + h1 ^= k1; + h1 = Long.rotateLeft(h1, R2); + h1 += h2; + h1 = h1 * M + N1; + + // mix functions for k2 + k2 *= C2; + k2 = Long.rotateLeft(k2, R3); + k2 *= C1; + h2 ^= k2; + h2 = Long.rotateLeft(h2, R1); + h2 += h1; + h2 = h2 * M + N2; + } + + // tail + long k1 = 0; + long k2 = 0; + int tailStart = nblocks << 4; + switch (length - tailStart) { + case 15: + k2 ^= (long) (data.getByte(offset + tailStart + 14) & 0xff) << 48; + case 14: + k2 ^= (long) (data.getByte(offset + tailStart + 13) & 0xff) << 40; + case 13: + k2 ^= (long) (data.getByte(offset + tailStart + 12) & 0xff) << 32; + case 12: + k2 ^= (long) (data.getByte(offset + tailStart + 11) & 0xff) << 24; + case 11: + k2 ^= (long) (data.getByte(offset + tailStart + 10) & 0xff) << 16; + case 10: + k2 ^= (long) (data.getByte(offset + tailStart + 9) & 0xff) << 8; + case 9: + k2 ^= (long) (data.getByte(offset + tailStart + 8) & 0xff); + k2 *= C2; + k2 = Long.rotateLeft(k2, R3); + k2 *= C1; + h2 ^= k2; + + case 8: + k1 ^= (long) (data.getByte(offset + tailStart + 7) & 0xff) << 56; + case 7: + k1 ^= (long) (data.getByte(offset + tailStart + 6) & 0xff) << 48; + case 6: + k1 ^= (long) (data.getByte(offset + tailStart + 5) & 0xff) << 40; + case 5: + k1 ^= (long) (data.getByte(offset + tailStart + 4) & 0xff) << 32; + case 4: + k1 ^= (long) (data.getByte(offset + tailStart + 3) & 0xff) << 24; + case 3: + k1 ^= (long) (data.getByte(offset + tailStart + 2) & 0xff) << 16; + case 2: + k1 ^= (long) (data.getByte(offset + tailStart + 1) & 0xff) << 8; + case 1: + k1 ^= (long) (data.getByte(offset + tailStart) & 0xff); + k1 *= C1; + k1 = Long.rotateLeft(k1, R1); + k1 *= C2; + h1 ^= k1; + } + + // finalization + h1 ^= length; + h2 ^= length; + + h1 += h2; + h2 += h1; + + h1 = fmix64(h1); + h2 = fmix64(h2); + + h1 += h2; + h2 += h1; + + return new long[]{h1, h2}; + } + + private static long fmix64(long h) { + h ^= (h >>> 33); + h *= 0xff51afd7ed558ccdL; + h ^= (h >>> 33); + h *= 0xc4ceb9fe1a85ec53L; + h ^= (h >>> 33); + return h; + } + +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/router/AbstractHashRouter.java b/stream/common/src/main/java/org/apache/bookkeeper/common/router/AbstractHashRouter.java index 2c6e052d97b..7dadf2d4ab7 100644 --- a/stream/common/src/main/java/org/apache/bookkeeper/common/router/AbstractHashRouter.java +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/router/AbstractHashRouter.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.common.router; import io.netty.buffer.ByteBuf; -import org.apache.bookkeeper.common.hash.MurmurHash; +import org.apache.bookkeeper.common.hash.Murmur3; /** * The base hash router. @@ -26,8 +26,7 @@ public abstract class AbstractHashRouter implements HashRouter { private static final long serialVersionUID = -7979076779920023308L; - // the MurmurHash2 value of "ZSTREAM" - static final long HASH_SEED = 383242705L; + public static final long HASH_SEED = 383242705L; protected AbstractHashRouter() { } @@ -36,8 +35,8 @@ protected AbstractHashRouter() { public Long getRoutingKey(K key) { ByteBuf keyData = getRoutingKeyData(key); try { - return MurmurHash.hash64( - keyData, keyData.readerIndex(), keyData.readableBytes(), HASH_SEED); + return Murmur3.hash128( + keyData, keyData.readerIndex(), keyData.readableBytes(), HASH_SEED)[0]; } finally { keyData.release(); } diff --git a/stream/common/src/test/java/org/apache/bookkeeper/common/router/HashRouterTest.java b/stream/common/src/test/java/org/apache/bookkeeper/common/router/HashRouterTest.java new file mode 100644 index 00000000000..5d4ed79ade7 --- /dev/null +++ b/stream/common/src/test/java/org/apache/bookkeeper/common/router/HashRouterTest.java @@ -0,0 +1,64 @@ +/* + * Licensed 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.common.router; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertEquals; + +import com.google.common.hash.Hashing; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.hash.Murmur3; +import org.junit.Test; + +/** + * Unit test {@link HashRouter}s. + */ +@Slf4j +public class HashRouterTest { + + @Test + public void testByteBufHashRouter() { + ByteBuf key = Unpooled.wrappedBuffer("foo".getBytes(UTF_8)); + + // murmur3 - 32 bits + int hash32 = Murmur3.hash32( + key, key.readerIndex(), key.readableBytes(), (int) AbstractHashRouter.HASH_SEED); + int guavaHash32 = Hashing.murmur3_32((int) AbstractHashRouter.HASH_SEED) + .newHasher() + .putString("foo", UTF_8) + .hash() + .asInt(); + assertEquals(hash32, guavaHash32); + + // murmur3 - 128 bits + long[] hash128 = Murmur3.hash128( + key, key.readerIndex(), key.readableBytes(), AbstractHashRouter.HASH_SEED); + log.info("hash128: {}", hash128); + long guavaHash128 = Hashing.murmur3_128((int) AbstractHashRouter.HASH_SEED) + .newHasher() + .putString("foo", UTF_8) + .hash() + .asLong(); + assertEquals(hash128[0], guavaHash128); + + ByteBufHashRouter router = ByteBufHashRouter.of(); + long routingHash = router.getRoutingKey(key).longValue(); + log.info("Routing hash = {}", routingHash); + assertEquals(hash128[0], routingHash); + } + +} From 7d354b88a2fd50c6a1f8ce0c3e838687c399cd1e Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 25 Sep 2018 10:38:12 -0700 Subject: [PATCH 0045/1642] [TOOLS] Add a perf tool for benchmarking bookkeeper Descriptions of the changes in this PR: ### Motivation when investigating performance on scanning entries in bookkeeper, it is hard to know what is the expectation of what kind of throughput that application can get. we need a perf tool to understand the baseline of performance that bookkeeper can provide. ### Changes this tool is following what pulsar perf tool is doing and using dlog library for evaluating the write and read throughput on bookkeeper using one (or a few) dlog streams. Author: Sijie Guo Author: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Author: Charan Reddy Guttapalem Author: Sijie Guo Reviewers: Enrico Olivelli , Jia Zhai This closes #1697 from sijie/bkperf --- bin/bkperf | 66 +++ conf/log4j.cli.properties | 1 + pom.xml | 8 + tools/perf/README.md | 100 +++++ tools/perf/pom.xml | 43 ++ .../apache/bookkeeper/tools/perf/BKPerf.java | 46 ++ .../tools/perf/DlogPerfCommandGroup.java | 43 ++ .../tools/perf/PerfCommandGroup.java | 25 ++ .../tools/perf/dlog/PerfReader.java | 290 ++++++++++++ .../tools/perf/dlog/PerfWriter.java | 413 ++++++++++++++++++ .../tools/perf/dlog/ReadCommand.java | 58 +++ .../tools/perf/dlog/WriteCommand.java | 58 +++ .../tools/perf/dlog/package-info.java | 18 + .../bookkeeper/tools/perf/package-info.java | 18 + .../perf/utils/PaddingDecimalFormat.java | 85 ++++ .../tools/perf/utils/package-info.java | 18 + tools/pom.xml | 1 + 17 files changed, 1291 insertions(+) create mode 100755 bin/bkperf create mode 100644 tools/perf/README.md create mode 100644 tools/perf/pom.xml create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/DlogPerfCommandGroup.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/PerfCommandGroup.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/PerfReader.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/PerfWriter.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/ReadCommand.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/WriteCommand.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/package-info.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/package-info.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/utils/PaddingDecimalFormat.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/utils/package-info.java diff --git a/bin/bkperf b/bin/bkperf new file mode 100755 index 00000000000..6ec2909989d --- /dev/null +++ b/bin/bkperf @@ -0,0 +1,66 @@ +#!/usr/bin/env bash +# +#/** +# * 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. +# */ + +# BookKeeper Perf Tool (experimental) + +BINDIR=`dirname "$0"` +BK_HOME=`cd ${BINDIR}/..;pwd` + +source ${BK_HOME}/bin/common.sh +source ${BK_HOME}/conf/bk_cli_env.sh + +CLI_MODULE_PATH=tools/perf +CLI_MODULE_NAME="(org.apache.bookkeeper-)?bookkeeper-perf" +CLI_MODULE_HOME=${BK_HOME}/${CLI_MODULE_PATH} + +# find the module jar +CLI_JAR=$(find_module_jar ${CLI_MODULE_PATH} ${CLI_MODULE_NAME}) + +# set up the classpath +CLI_CLASSPATH=$(set_module_classpath ${CLI_MODULE_PATH}) + +DEFAULT_CONF=${BK_HOME}/conf/bk_server.conf +if [ -z "${CLI_CONF}" ]; then + CLI_CONF=${DEFAULT_CONF} +fi + +DEFAULT_LOG_CONF=${BK_HOME}/conf/log4j.cli.properties +if [ -z "${CLI_LOG_CONF}" ]; then + CLI_LOG_CONF=${DEFAULT_LOG_CONF} +fi +CLI_LOG_DIR=${CLI_LOG_DIR:-"$BK_HOME/logs"} +CLI_LOG_FILE=${CLI_LOG_FILE:-"bkperf.log"} +CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,ROLLINGFILE"} + +# Configure the classpath +CLI_CLASSPATH="$CLI_JAR:$CLI_CLASSPATH:$CLI_EXTRA_CLASSPATH" +CLI_CLASSPATH="`dirname $CLI_LOG_CONF`:$CLI_CLASSPATH" + +# Build the OPTs +BOOKIE_OPTS=$(build_bookie_opts) +GC_OPTS=$(build_cli_jvm_opts ${CLI_LOG_DIR} "bkperf-gc.log") +NETTY_OPTS=$(build_netty_opts) +LOGGING_OPTS=$(build_cli_logging_opts ${CLI_LOG_CONF} ${CLI_LOG_DIR} ${CLI_LOG_FILE} ${CLI_ROOT_LOGGER}) + +OPTS="${OPTS} -cp ${CLI_CLASSPATH} ${BOOKIE_OPTS} ${GC_OPTS} ${NETTY_OPTS} ${LOGGING_OPTS} ${CLI_EXTRA_OPTS}" + +#Change to BK_HOME to support relative paths +cd "$BK_HOME" +exec ${JAVA} ${OPTS} org.apache.bookkeeper.tools.perf.BKPerf --conf ${CLI_CONF} $@ diff --git a/conf/log4j.cli.properties b/conf/log4j.cli.properties index 51c95f58c88..ceb77cc934a 100644 --- a/conf/log4j.cli.properties +++ b/conf/log4j.cli.properties @@ -55,5 +55,6 @@ log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{ log4j.logger.verbose=INFO,VERBOSECONSOLE log4j.logger.org.apache.zookeeper=ERROR log4j.logger.org.apache.bookkeeper=ERROR +log4j.logger.org.apache.bookkeeper.tools=INFO log4j.logger.org.apache.bookkeeper.bookie.BookieShell=INFO log4j.logger.org.apache.bookkeeper.client.BookKeeperAdmin=INFO diff --git a/pom.xml b/pom.xml index e5a3ffde00a..557025965fd 100644 --- a/pom.xml +++ b/pom.xml @@ -130,6 +130,7 @@ 21.0 2.7.3 1.3 + 2.1.10 2.8.9 1.9.11 1.48 @@ -543,6 +544,13 @@ ${jcommander.version} + + + org.hdrhistogram + HdrHistogram + ${hdrhistogram.version} + + junit diff --git a/tools/perf/README.md b/tools/perf/README.md new file mode 100644 index 00000000000..945d80c81ba --- /dev/null +++ b/tools/perf/README.md @@ -0,0 +1,100 @@ +## BookKeeper Perf Tool + +### Dlog + +```shell +$ bin/bkperf dlog +Commands on evaluating performance of distributedlog library + +Usage: bkperf dlog [command] [command options] + +Commands: + + read Read log records to distributedlog streams + write Write log records to distributedlog streams + + help Display help information about it +``` + +#### Write records to logs + +```shell +$ bin/bkperf dlog write -h +Write log records to distributedlog streams + +Usage: bkperf dlog write [flags] + +Flags: + + -a, --ack-quorum-size + Ledger ack quorum size + + -e, --ensemble-size + Ledger ensemble size + + -ln, --log-name + Log name or log name pattern if more than 1 log is specified at + `--num-logs` + + -b, --num-bytes + Number of bytes to write in total. If 0, it will keep writing + + -l, --num-logs + Number of log streams + + -n, --num-records + Number of records to write in total. If 0, it will keep writing + + -r, --rate + Write rate bytes/s across log streams + + -rs, --record-size + Log record size + + --threads + Number of threads writing + + -w, --write-quorum-size + Ledger write quorum size + + + -h, --help + Display help information +``` + +Example: write to log stream `test-log` at `100mb/second`, using 1-bookie ensemble. + +```shell +$ bin/bkperf dlog write -w 1 -a 1 -e 1 -r 104857600 --log-name test-log +``` + +### Read records from logs + +```shell +$ bin/bkperf dlog read -h +Read log records from distributedlog streams + +Usage: bkperf dlog read [flags] + +Flags: + + -ln, --log-name + Log name or log name pattern if more than 1 log is specified at + `--num-logs` + + -l, --num-logs + Number of log streams + + --threads + Number of threads reading + + + -h, --help + Display help information +``` + +Example: read from log stream `test-log-000000`. + +```shell +$ bin/bkperf dlog read --log-name test-log-000000 +``` diff --git a/tools/perf/pom.xml b/tools/perf/pom.xml new file mode 100644 index 00000000000..d3a617f60b1 --- /dev/null +++ b/tools/perf/pom.xml @@ -0,0 +1,43 @@ + + + + 4.0.0 + + org.apache.bookkeeper + bookkeeper-tools-parent + 4.9.0-SNAPSHOT + + bookkeeper-perf + Apache BookKeeper :: Tools :: Perf + + + + org.apache.bookkeeper + bookkeeper-tools-framework + ${project.version} + + + org.apache.distributedlog + distributedlog-core + ${project.version} + + + org.hdrhistogram + HdrHistogram + + + + diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java new file mode 100644 index 00000000000..23021ee930a --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java @@ -0,0 +1,46 @@ +/* + * Licensed 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.tools.perf; + +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.Cli; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * bkperf evaluates the performance of Apache BookKeeper cluster. + */ +@Slf4j +public class BKPerf { + + public static final String NAME = "bkperf"; + + @SuppressWarnings("unchecked") + public static void main(String[] args) { + CliSpec.Builder specBuilder = CliSpec.newBuilder() + .withName(NAME) + .withUsage(NAME + " [flags] [command group] [commands]") + .withDescription(NAME + " evaluates the performance of Apache BookKeeper clusters") + .withFlags(new BKFlags()) + .withConsole(System.out) + .addCommand(new DlogPerfCommandGroup()); + + CliSpec spec = specBuilder.build(); + + int retCode = Cli.runCli(spec, args); + Runtime.getRuntime().exit(retCode); + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/DlogPerfCommandGroup.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/DlogPerfCommandGroup.java new file mode 100644 index 00000000000..75adc4b292f --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/DlogPerfCommandGroup.java @@ -0,0 +1,43 @@ +/* + * Licensed 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.tools.perf; + +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliCommandGroup; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.tools.perf.dlog.ReadCommand; +import org.apache.bookkeeper.tools.perf.dlog.WriteCommand; + +/** + * Commands that evaluate performance of distributedlog library. + */ +public class DlogPerfCommandGroup extends CliCommandGroup implements PerfCommandGroup { + + private static final String NAME = "dlog"; + private static final String DESC = "Commands on evaluating performance of distributedlog library"; + + private static final CliSpec spec = CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withParent(BKPerf.NAME) + .addCommand(new WriteCommand()) + .addCommand(new ReadCommand()) + .build(); + + public DlogPerfCommandGroup() { + super(spec); + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/PerfCommandGroup.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/PerfCommandGroup.java new file mode 100644 index 00000000000..c64d77f2d8c --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/PerfCommandGroup.java @@ -0,0 +1,25 @@ +/* + * Licensed 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.tools.perf; + +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CommandGroup; + +/** + * A command group that group commands together for performance evaluations. + */ +public interface PerfCommandGroup + extends CommandGroup { +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/PerfReader.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/PerfReader.java new file mode 100644 index 00000000000..7497d7df3bb --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/PerfReader.java @@ -0,0 +1,290 @@ +/* + * Licensed 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. + */ + +/* + * Licensed 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.tools.perf.dlog; + +import com.beust.jcommander.Parameter; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.text.DecimalFormat; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.LongAdder; +import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; +import org.HdrHistogram.Histogram; +import org.HdrHistogram.Recorder; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.perf.utils.PaddingDecimalFormat; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.distributedlog.DLSN; +import org.apache.distributedlog.DistributedLogConfiguration; +import org.apache.distributedlog.LogRecordWithDLSN; +import org.apache.distributedlog.api.DistributedLogManager; +import org.apache.distributedlog.api.LogReader; +import org.apache.distributedlog.api.namespace.Namespace; +import org.apache.distributedlog.api.namespace.NamespaceBuilder; + +/** + * A perf writer to evaluate write performance. + */ +@Slf4j +public class PerfReader implements Runnable { + + /** + * Flags for the write command. + */ + public static class Flags extends CliFlags { + + @Parameter( + names = { + "-ln", "--log-name" + }, + description = "Log name or log name pattern if more than 1 log is specified at `--num-logs`") + public String logName = "test-log-%06d"; + + @Parameter( + names = { + "-l", "--num-logs" + }, + description = "Number of log streams") + public int numLogs = 1; + + @Parameter( + names = { + "-t", "--threads" + }, + description = "Number of threads reading") + public int numThreads = 1; + + @Parameter( + names = { + "-mr", "--max-readahead-records" + }, + description = "Max readhead records") + public int maxReadAheadRecords = 1000000; + + @Parameter( + names = { + "-bs", "--readahead-batch-size" + }, + description = "ReadAhead Batch Size, in entries" + ) + public int readAheadBatchSize = 4; + + } + + + // stats + private final LongAdder recordsRead = new LongAdder(); + private final LongAdder bytesRead = new LongAdder(); + + private final ServiceURI serviceURI; + private final Flags flags; + private final Recorder recorder = new Recorder( + TimeUnit.SECONDS.toMillis(120000), 5 + ); + private final Recorder cumulativeRecorder = new Recorder( + TimeUnit.SECONDS.toMillis(120000), 5 + ); + private final AtomicBoolean isDone = new AtomicBoolean(false); + + PerfReader(ServiceURI serviceURI, Flags flags) { + this.serviceURI = serviceURI; + this.flags = flags; + } + + @Override + public void run() { + try { + execute(); + } catch (Exception e) { + log.error("Encountered exception at running dlog perf writer", e); + } + } + + void execute() throws Exception { + ObjectMapper m = new ObjectMapper(); + ObjectWriter w = m.writerWithDefaultPrettyPrinter(); + log.info("Starting dlog perf reader with config : {}", w.writeValueAsString(flags)); + + DistributedLogConfiguration conf = newDlogConf(flags); + try (Namespace namespace = NamespaceBuilder.newBuilder() + .conf(conf) + .uri(serviceURI.getUri()) + .build()) { + execute(namespace); + } + } + + void execute(Namespace namespace) throws Exception { + List> managers = new ArrayList<>(flags.numLogs); + for (int i = 0; i < flags.numLogs; i++) { + String logName = String.format(flags.logName, i); + managers.add(Pair.of(i, namespace.openLog(logName))); + } + log.info("Successfully open {} logs", managers.size()); + + // register shutdown hook to aggregate stats + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + isDone.set(true); + printAggregatedStats(cumulativeRecorder); + })); + + ExecutorService executor = Executors.newFixedThreadPool(flags.numThreads); + try { + for (int i = 0; i < flags.numThreads; i++) { + final int idx = i; + final List logsThisThread = managers + .stream() + .filter(pair -> pair.getLeft() % flags.numThreads == idx) + .map(pair -> pair.getRight()) + .collect(Collectors.toList()); + executor.submit(() -> { + try { + read(logsThisThread); + } catch (Exception e) { + log.error("Encountered error at writing records", e); + } + }); + } + log.info("Started {} write threads", flags.numThreads); + reportStats(); + } finally { + executor.shutdown(); + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + managers.forEach(manager -> manager.getRight().asyncClose()); + } + } + + void read(List logs) throws Exception { + log.info("Read thread started with : logs = {}", + logs.stream().map(l -> l.getStreamName()).collect(Collectors.toList())); + + List readers = logs.stream() + .map(manager -> { + try { + return manager.openLogReader(DLSN.InitialDLSN); + } catch (IOException e) { + log.error("Failed to open reader for log stream {}", manager.getStreamName(), e); + throw new UncheckedIOException(e); + } + }) + .collect(Collectors.toList()); + + final int numLogs = logs.size(); + while (true) { + for (int i = 0; i < numLogs; i++) { + LogRecordWithDLSN record = readers.get(i).readNext(true); + if (null != record) { + recordsRead.increment(); + bytesRead.add(record.getPayloadBuf().readableBytes()); + } + } + } + } + + void reportStats() { + // Print report stats + long oldTime = System.nanoTime(); + + Histogram reportHistogram = null; + + while (true) { + try { + Thread.sleep(10000); + } catch (InterruptedException e) { + break; + } + + if (isDone.get()) { + break; + } + + long now = System.nanoTime(); + double elapsed = (now - oldTime) / 1e9; + + double rate = recordsRead.sumThenReset() / elapsed; + double throughput = bytesRead.sumThenReset() / elapsed / 1024 / 1024; + + reportHistogram = recorder.getIntervalHistogram(reportHistogram); + + log.info("Throughput read : {} records/s --- {} MB/s --- Latency: mean:" + + " {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}", + throughputFormat.format(rate), throughputFormat.format(throughput), + dec.format(reportHistogram.getMean() / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(50) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(95) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.9) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.99) / 1000.0), + dec.format(reportHistogram.getMaxValue() / 1000.0)); + + reportHistogram.reset(); + + oldTime = now; + } + + } + + private static DistributedLogConfiguration newDlogConf(Flags flags) { + return new DistributedLogConfiguration() + .setReadAheadBatchSize(flags.readAheadBatchSize) + .setReadAheadMaxRecords(flags.maxReadAheadRecords) + .setReadAheadWaitTime(200); + } + + + private static final DecimalFormat throughputFormat = new PaddingDecimalFormat("0.0", 8); + private static final DecimalFormat dec = new PaddingDecimalFormat("0.000", 7); + + private static void printAggregatedStats(Recorder recorder) { + Histogram reportHistogram = recorder.getIntervalHistogram(); + + log.info("Aggregated latency stats --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {}" + + " - 99.9pct: {} - 99.99pct: {} - 99.999pct: {} - Max: {}", + dec.format(reportHistogram.getMean() / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(50) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(95) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.9) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.999) / 1000.0), + dec.format(reportHistogram.getMaxValue() / 1000.0)); + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/PerfWriter.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/PerfWriter.java new file mode 100644 index 00000000000..e29dc6441b3 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/PerfWriter.java @@ -0,0 +1,413 @@ +/* + * Licensed 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.tools.perf.dlog; + +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; + +import com.beust.jcommander.Parameter; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.common.util.concurrent.RateLimiter; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.netty.buffer.Unpooled; +import java.text.DecimalFormat; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.LongAdder; +import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; +import org.HdrHistogram.Histogram; +import org.HdrHistogram.Recorder; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.perf.utils.PaddingDecimalFormat; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.distributedlog.DistributedLogConfiguration; +import org.apache.distributedlog.LogRecord; +import org.apache.distributedlog.api.AsyncLogWriter; +import org.apache.distributedlog.api.DistributedLogManager; +import org.apache.distributedlog.api.namespace.Namespace; +import org.apache.distributedlog.api.namespace.NamespaceBuilder; + +/** + * A perf writer to evaluate write performance. + */ +@Slf4j +public class PerfWriter implements Runnable { + + /** + * Flags for the write command. + */ + public static class Flags extends CliFlags { + + @Parameter( + names = { + "-r", "--rate" + }, + description = "Write rate bytes/s across log streams") + public int writeRate = 0; + + @Parameter( + names = { + "-rs", "--record-size" + }, + description = "Log record size") + public int recordSize = 1024; + + @Parameter( + names = { + "-ln", "--log-name" + }, + description = "Log name or log name pattern if more than 1 log is specified at `--num-logs`") + public String logName = "test-log-%06d"; + + @Parameter( + names = { + "-l", "--num-logs" + }, + description = "Number of log streams") + public int numLogs = 1; + + @Parameter( + names = { + "-t", "--threads" + }, + description = "Number of threads writing") + public int numThreads = 1; + + @Parameter( + names = { + "-mob", "--max-outstanding-megabytes" + }, + description = "Number of threads writing") + public long maxOutstandingMB = 200; + + @Parameter( + names = { + "-n", "--num-records" + }, + description = "Number of records to write in total. If 0, it will keep writing") + public long numRecords = 0; + + @Parameter( + names = { + "-b", "--num-bytes" + }, + description = "Number of bytes to write in total. If 0, it will keep writing") + public long numBytes = 0; + + @Parameter( + names = { + "-e", "--ensemble-size" + }, + description = "Ledger ensemble size") + public int ensembleSize = 1; + + @Parameter( + names = { + "-w", "--write-quorum-size" + }, + description = "Ledger write quorum size") + public int writeQuorumSize = 1; + + @Parameter( + names = { + "-a", "--ack-quorum-size" + }, + description = "Ledger ack quorum size") + public int ackQuorumSize = 1; + + } + + + // stats + private final LongAdder recordsWritten = new LongAdder(); + private final LongAdder bytesWritten = new LongAdder(); + + private final byte[] payload; + private final ServiceURI serviceURI; + private final Flags flags; + private final Recorder recorder = new Recorder( + TimeUnit.SECONDS.toMillis(120000), 5 + ); + private final Recorder cumulativeRecorder = new Recorder( + TimeUnit.SECONDS.toMillis(120000), 5 + ); + private final AtomicBoolean isDone = new AtomicBoolean(false); + + PerfWriter(ServiceURI serviceURI, Flags flags) { + this.serviceURI = serviceURI; + this.flags = flags; + this.payload = new byte[flags.recordSize]; + ThreadLocalRandom.current().nextBytes(payload); + } + + @Override + public void run() { + try { + execute(); + } catch (Exception e) { + log.error("Encountered exception at running dlog perf writer", e); + } + } + + void execute() throws Exception { + ObjectMapper m = new ObjectMapper(); + ObjectWriter w = m.writerWithDefaultPrettyPrinter(); + log.info("Starting dlog perf writer with config : {}", w.writeValueAsString(flags)); + + DistributedLogConfiguration conf = newDlogConf(flags); + try (Namespace namespace = NamespaceBuilder.newBuilder() + .conf(conf) + .uri(serviceURI.getUri()) + .build()) { + execute(namespace); + } + } + + void execute(Namespace namespace) throws Exception { + List> managers = new ArrayList<>(flags.numLogs); + for (int i = 0; i < flags.numLogs; i++) { + String logName = String.format(flags.logName, i); + managers.add(Pair.of(i, namespace.openLog(logName))); + } + log.info("Successfully open {} logs", managers.size()); + + // register shutdown hook to aggregate stats + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + isDone.set(true); + printAggregatedStats(cumulativeRecorder); + })); + + ExecutorService executor = Executors.newFixedThreadPool(flags.numThreads); + try { + for (int i = 0; i < flags.numThreads; i++) { + final int idx = i; + final List logsThisThread = managers + .stream() + .filter(pair -> pair.getLeft() % flags.numThreads == idx) + .map(pair -> pair.getRight()) + .collect(Collectors.toList()); + final long numRecordsForThisThread = flags.numRecords / flags.numThreads; + final long numBytesForThisThread = flags.numBytes / flags.numThreads; + final double writeRateForThisThread = flags.writeRate / (double) flags.numThreads; + final long maxOutstandingBytesForThisThread = flags.maxOutstandingMB * 1024 * 1024 / flags.numThreads; + executor.submit(() -> { + try { + write( + logsThisThread, + writeRateForThisThread, + (int) maxOutstandingBytesForThisThread, + numRecordsForThisThread, + numBytesForThisThread); + } catch (Exception e) { + log.error("Encountered error at writing records", e); + } + }); + } + log.info("Started {} write threads", flags.numThreads); + reportStats(); + } finally { + executor.shutdown(); + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + managers.forEach(manager -> manager.getRight().asyncClose()); + } + } + + void write(List logs, + double writeRate, + int maxOutstandingBytesForThisThread, + long numRecordsForThisThread, + long numBytesForThisThread) throws Exception { + log.info("Write thread started with : logs = {}, rate = {}," + + " num records = {}, num bytes = {}, max outstanding bytes = {}", + logs.stream().map(l -> l.getStreamName()).collect(Collectors.toList()), + writeRate, + numRecordsForThisThread, + numBytesForThisThread, + maxOutstandingBytesForThisThread); + + List> writerFutures = logs.stream() + .map(manager -> manager.openAsyncLogWriter()) + .collect(Collectors.toList()); + List writers = result(FutureUtils.collect(writerFutures)); + + long txid = writers + .stream() + .mapToLong(writer -> writer.getLastTxId()) + .max() + .orElse(0L); + txid = Math.max(0L, txid); + + RateLimiter limiter; + if (writeRate > 0) { + limiter = RateLimiter.create(writeRate); + } else { + limiter = null; + } + final Semaphore semaphore; + if (maxOutstandingBytesForThisThread > 0) { + semaphore = new Semaphore(maxOutstandingBytesForThisThread); + } else { + semaphore = null; + } + + // Acquire 1 second worth of records to have a slower ramp-up + if (limiter != null) { + limiter.acquire((int) writeRate); + } + + long totalWritten = 0L; + long totalBytesWritten = 0L; + final int numLogs = logs.size(); + while (true) { + for (int i = 0; i < numLogs; i++) { + if (numRecordsForThisThread > 0 + && totalWritten >= numRecordsForThisThread) { + markPerfDone(); + } + if (numBytesForThisThread > 0 + && totalBytesWritten >= numBytesForThisThread) { + markPerfDone(); + } + if (null != semaphore) { + semaphore.acquire(payload.length); + } + + totalWritten++; + totalBytesWritten += payload.length; + if (null != limiter) { + limiter.acquire(payload.length); + } + final long sendTime = System.nanoTime(); + writers.get(i).write( + new LogRecord(++txid, Unpooled.wrappedBuffer(payload)) + ).thenAccept(dlsn -> { + if (null != semaphore) { + semaphore.release(payload.length); + } + + recordsWritten.increment(); + bytesWritten.add(payload.length); + + long latencyMicros = TimeUnit.NANOSECONDS.toMicros( + System.nanoTime() - sendTime + ); + recorder.recordValue(latencyMicros); + cumulativeRecorder.recordValue(latencyMicros); + }).exceptionally(cause -> { + log.warn("Error at writing records", cause); + System.exit(-1); + return null; + }); + } + } + } + + @SuppressFBWarnings("DM_EXIT") + void markPerfDone() throws Exception { + log.info("------------------- DONE -----------------------"); + printAggregatedStats(cumulativeRecorder); + isDone.set(true); + Thread.sleep(5000); + System.exit(0); + } + + void reportStats() { + // Print report stats + long oldTime = System.nanoTime(); + + Histogram reportHistogram = null; + + while (true) { + try { + Thread.sleep(10000); + } catch (InterruptedException e) { + break; + } + + if (isDone.get()) { + break; + } + + long now = System.nanoTime(); + double elapsed = (now - oldTime) / 1e9; + + double rate = recordsWritten.sumThenReset() / elapsed; + double throughput = bytesWritten.sumThenReset() / elapsed / 1024 / 1024; + + reportHistogram = recorder.getIntervalHistogram(reportHistogram); + + log.info( + "Throughput written : {} records/s --- {} MB/s --- Latency: mean:" + + " {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}", + throughputFormat.format(rate), throughputFormat.format(throughput), + dec.format(reportHistogram.getMean() / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(50) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(95) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.9) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.99) / 1000.0), + dec.format(reportHistogram.getMaxValue() / 1000.0)); + + reportHistogram.reset(); + + oldTime = now; + } + + } + + private static DistributedLogConfiguration newDlogConf(Flags flags) { + return new DistributedLogConfiguration() + .setEnsembleSize(flags.ensembleSize) + .setWriteQuorumSize(flags.writeQuorumSize) + .setAckQuorumSize(flags.ackQuorumSize) + .setOutputBufferSize(512 * 1024) + .setPeriodicFlushFrequencyMilliSeconds(2) + .setWriteLockEnabled(false) + .setMaxLogSegmentBytes(512 * 1024 * 1024) // 512MB + .setExplicitTruncationByApplication(true); + } + + + private static final DecimalFormat throughputFormat = new PaddingDecimalFormat("0.0", 8); + private static final DecimalFormat dec = new PaddingDecimalFormat("0.000", 7); + + private static void printAggregatedStats(Recorder recorder) { + Histogram reportHistogram = recorder.getIntervalHistogram(); + + log.info("Aggregated latency stats --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {}" + + " - 99.9pct: {} - 99.99pct: {} - 99.999pct: {} - Max: {}", + dec.format(reportHistogram.getMean() / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(50) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(95) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.9) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.999) / 1000.0), + dec.format(reportHistogram.getMaxValue() / 1000.0)); + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/ReadCommand.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/ReadCommand.java new file mode 100644 index 00000000000..001cacba3d7 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/ReadCommand.java @@ -0,0 +1,58 @@ +/* + * Licensed 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.tools.perf.dlog; + +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.tools.common.BKCommand; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.tools.perf.dlog.PerfReader.Flags; +import org.apache.commons.configuration.CompositeConfiguration; + +/** + * Command to read log records to distributedlog streams. + */ +@Slf4j +public class ReadCommand extends BKCommand { + + private static final String NAME = "read"; + private static final String DESC = "Read log records from distributedlog streams"; + + public ReadCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new Flags()) + .build()); + } + + @Override + protected boolean apply(ServiceURI serviceURI, + CompositeConfiguration conf, + BKFlags globalFlags, Flags cmdFlags) { + + + if (serviceURI == null) { + log.warn("No service uri is provided. Use default 'distributedlog://localhost/distributedlog'."); + serviceURI = ServiceURI.create("distributedlog://localhost/distributedlog"); + } + + PerfReader reader = new PerfReader(serviceURI, cmdFlags); + reader.run(); + return true; + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/WriteCommand.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/WriteCommand.java new file mode 100644 index 00000000000..aa7c92e46e7 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/WriteCommand.java @@ -0,0 +1,58 @@ +/* + * Licensed 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.tools.perf.dlog; + +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.tools.common.BKCommand; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.tools.perf.dlog.PerfWriter.Flags; +import org.apache.commons.configuration.CompositeConfiguration; + +/** + * Command to write log records to distributedlog streams. + */ +@Slf4j +public class WriteCommand extends BKCommand { + + private static final String NAME = "write"; + private static final String DESC = "Write log records to distributedlog streams"; + + public WriteCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new Flags()) + .build()); + } + + @Override + protected boolean apply(ServiceURI serviceURI, + CompositeConfiguration conf, + BKFlags globalFlags, Flags cmdFlags) { + + + if (serviceURI == null) { + log.warn("No service uri is provided. Use default 'distributedlog://localhost/distributedlog'."); + serviceURI = ServiceURI.create("distributedlog://localhost/distributedlog"); + } + + PerfWriter writer = new PerfWriter(serviceURI, cmdFlags); + writer.run(); + return true; + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/package-info.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/package-info.java new file mode 100644 index 00000000000..596d419d585 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/dlog/package-info.java @@ -0,0 +1,18 @@ +/* + * Licensed 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. + */ + +/** + * Dlog related perf command. + */ +package org.apache.bookkeeper.tools.perf.dlog; \ No newline at end of file diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/package-info.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/package-info.java new file mode 100644 index 00000000000..ca7aee64bc7 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/package-info.java @@ -0,0 +1,18 @@ +/* + * Licensed 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. + */ + +/** + * BookKeeper Perf Tool. + */ +package org.apache.bookkeeper.tools.perf; diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/utils/PaddingDecimalFormat.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/utils/PaddingDecimalFormat.java new file mode 100644 index 00000000000..0bc92af86fc --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/utils/PaddingDecimalFormat.java @@ -0,0 +1,85 @@ +/* + * Licensed 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.tools.perf.utils; + +import java.text.DecimalFormat; +import java.text.DecimalFormatSymbols; +import java.text.FieldPosition; + +/** + * A decimal format that adds padding zeros. + */ +public class PaddingDecimalFormat extends DecimalFormat { + private int minimumLength; + + /** + * Creates a PaddingDecimalFormat using the given pattern and minimum minimumLength and the symbols for the default + * locale. + */ + public PaddingDecimalFormat(String pattern, int minLength) { + super(pattern); + minimumLength = minLength; + } + + /** + * Creates a PaddingDecimalFormat using the given pattern, symbols and minimum minimumLength. + */ + public PaddingDecimalFormat(String pattern, DecimalFormatSymbols symbols, int minLength) { + super(pattern, symbols); + minimumLength = minLength; + } + + @Override + public StringBuffer format(double number, StringBuffer toAppendTo, FieldPosition pos) { + int initLength = toAppendTo.length(); + super.format(number, toAppendTo, pos); + return pad(toAppendTo, initLength); + } + + @Override + public StringBuffer format(long number, StringBuffer toAppendTo, FieldPosition pos) { + int initLength = toAppendTo.length(); + super.format(number, toAppendTo, pos); + return pad(toAppendTo, initLength); + } + + private StringBuffer pad(StringBuffer toAppendTo, int initLength) { + int numLength = toAppendTo.length() - initLength; + int padLength = minimumLength - numLength; + if (padLength > 0) { + StringBuffer pad = new StringBuffer(padLength); + for (int i = 0; i < padLength; i++) { + pad.append(' '); + } + toAppendTo.insert(initLength, pad); + } + return toAppendTo; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof PaddingDecimalFormat)) { + return false; + } + + PaddingDecimalFormat other = (PaddingDecimalFormat) obj; + return minimumLength == other.minimumLength && super.equals(obj); + } + + @Override + public int hashCode() { + return 31 * super.hashCode() + minimumLength; + } +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/utils/package-info.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/utils/package-info.java new file mode 100644 index 00000000000..76a7e427d91 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/utils/package-info.java @@ -0,0 +1,18 @@ +/* + * Licensed 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. + */ + +/** + * Utils used in the perf tool. + */ +package org.apache.bookkeeper.tools.perf.utils; \ No newline at end of file diff --git a/tools/pom.xml b/tools/pom.xml index 988de817fb2..37e63751ed5 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,6 +27,7 @@ pom framework + perf all From 4869d0c4792269850cf9a859e2da714354e1eda0 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 26 Sep 2018 02:39:32 -0700 Subject: [PATCH 0046/1642] [BUILD] Ignore pid files Descriptions of the changes in this PR: *Motivation* when running bookkeeper servers using `bin/bookkeeper-daemon`, `pid` files will be added in current directory. *Changes* Add `.pid` in `.gitignore`, so those pid files will be ignored. Author: Sijie Guo Author: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Author: Charan Reddy Guttapalem Author: Sijie Guo Reviewers: Enrico Olivelli , Jia Zhai This closes #1706 from sijie/ignore_pid_files --- .gitignore | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index 4865c98eb45..dcdcb75890a 100644 --- a/.gitignore +++ b/.gitignore @@ -29,3 +29,6 @@ logs/ # Data directory data/ + +# Pid files +**/*.pid From 1329a293086e5a9ca9ed07540e9c4fa3e944738c Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Wed, 26 Sep 2018 02:41:42 -0700 Subject: [PATCH 0047/1642] Issue #1700: fixed: TimedRunnable does not anticipate exception from wrapped runnable Descriptions of the changes in this PR: added try/finally around runnable.run() ### Motivation To correctly track (metrics/log) runnable that throw unexpected exception. ### Changes added try/finally around runnable.run() Master Issue: #1700 Author: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Author: Charan Reddy Guttapalem Author: Sijie Guo Author: Andrey Yegorov Reviewers: Sijie Guo , Enrico Olivelli This closes #1702 from dlg99/fix/issue_1700_timedrunnable, closes #1700 --- .../bookkeeper/common/util/OrderedExecutor.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java index 520787b3835..ab62a687e10 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java @@ -187,12 +187,15 @@ protected class TimedRunnable implements Runnable { public void run() { taskPendingStats.registerSuccessfulEvent(MathUtils.elapsedNanos(initNanos), TimeUnit.NANOSECONDS); long startNanos = MathUtils.nowInNano(); - this.runnable.run(); - long elapsedMicroSec = MathUtils.elapsedMicroSec(startNanos); - taskExecutionStats.registerSuccessfulEvent(elapsedMicroSec, TimeUnit.MICROSECONDS); - if (elapsedMicroSec >= warnTimeMicroSec) { - log.warn("Runnable {}:{} took too long {} micros to execute.", runnable, runnable.getClass(), - elapsedMicroSec); + try { + this.runnable.run(); + } finally { + long elapsedMicroSec = MathUtils.elapsedMicroSec(startNanos); + taskExecutionStats.registerSuccessfulEvent(elapsedMicroSec, TimeUnit.MICROSECONDS); + if (elapsedMicroSec >= warnTimeMicroSec) { + log.warn("Runnable {}:{} took too long {} micros to execute.", runnable, runnable.getClass(), + elapsedMicroSec); + } } } } From 54d22f6c47afb26756233c77e64dd59ca81b146e Mon Sep 17 00:00:00 2001 From: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Date: Wed, 26 Sep 2018 02:42:51 -0700 Subject: [PATCH 0048/1642] autorecovery-use-metadata-driver (part 3) : remove zookeeper from Auditor class Descriptions of the changes in this PR: This is the 3rd change to remove zookeeper from Auditor. It also changes how we construct metadata driver from AutoRecoveryMain. Since we are using bookkeeper client and admin across auditor and replication. So change the AutoRecoveryMain to construct the bookkeeper client from client configuration and pass bookkeeper client down to replication worker and auditor. Author: Qi Wang Author: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Author: Sijie Guo Author: Charan Reddy Guttapalem Reviewers: Enrico Olivelli , Sijie Guo This closes #1695 from codingwangqi/autorecovery_metadata_part3 --- .../bookkeeper/replication/Auditor.java | 79 +++++++++++++------ .../replication/AuditorElector.java | 50 +++++++++--- .../replication/AutoRecoveryMain.java | 53 +++++-------- .../replication/ReplicationWorker.java | 38 +++++---- .../replication/AuditorBookieTest.java | 9 +-- .../replication/AuditorLedgerCheckerTest.java | 3 +- .../AuditorPeriodicBookieCheckTest.java | 13 +-- .../replication/AuditorPeriodicCheckTest.java | 18 +---- .../server/http/TestHttpService.java | 11 +-- 9 files changed, 142 insertions(+), 132 deletions(-) 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 788aaa2764d..c22889c87dd 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 @@ -52,7 +52,6 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; -import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; @@ -62,11 +61,9 @@ import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.commons.collections4.CollectionUtils; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,7 +79,8 @@ public class Auditor implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(Auditor.class); private final ServerConfiguration conf; - private BookKeeper bkc; + private final BookKeeper bkc; + private final boolean ownBkc; private BookKeeperAdmin admin; private BookieLedgerIndexer bookieLedgerIndexer; private LedgerManager ledgerManager; @@ -104,8 +102,48 @@ public class Auditor implements AutoCloseable { private Set bookiesToBeAudited = Sets.newHashSet(); private volatile int lostBookieRecoveryDelayBeforeChange; - public Auditor(final String bookieIdentifier, ServerConfiguration conf, - ZooKeeper zkc, StatsLogger statsLogger) throws UnavailableException { + static BookKeeper createBookKeeperClient(ServerConfiguration conf) + throws InterruptedException, IOException { + ClientConfiguration clientConfiguration = new ClientConfiguration(conf); + clientConfiguration.setClientRole(ClientConfiguration.CLIENT_ROLE_SYSTEM); + try { + return BookKeeper.forConfig(clientConfiguration).build(); + } catch (BKException e) { + throw new IOException("Failed to create bookkeeper client", e); + } + } + + static BookKeeper createBookKeeperClientThrowUnavailableException(ServerConfiguration conf) + throws UnavailableException { + try { + return createBookKeeperClient(conf); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UnavailableException("Failed to create bookkeeper client", e); + } catch (IOException e) { + throw new UnavailableException("Failed to create bookkeeper client", e); + } + } + + + public Auditor(final String bookieIdentifier, + ServerConfiguration conf, + StatsLogger statsLogger) + throws UnavailableException { + this( + bookieIdentifier, + conf, + createBookKeeperClientThrowUnavailableException(conf), + true, + statsLogger); + } + + public Auditor(final String bookieIdentifier, + ServerConfiguration conf, + BookKeeper bkc, + boolean ownBkc, + StatsLogger statsLogger) + throws UnavailableException { this.conf = conf; this.bookieIdentifier = bookieIdentifier; this.statsLogger = statsLogger; @@ -123,7 +161,9 @@ public Auditor(final String bookieIdentifier, ServerConfiguration conf, numDelayedBookieAuditsCancelled = this.statsLogger .getCounter(ReplicationStats.NUM_DELAYED_BOOKIE_AUDITS_DELAYES_CANCELLED); - initialize(conf, zkc); + this.bkc = bkc; + this.ownBkc = ownBkc; + initialize(conf, bkc); executor = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() { @Override @@ -135,15 +175,9 @@ public Thread newThread(Runnable r) { }); } - private void initialize(ServerConfiguration conf, ZooKeeper zkc) + private void initialize(ServerConfiguration conf, BookKeeper bkc) throws UnavailableException { try { - ClientConfiguration clientConfiguration = new ClientConfiguration(conf); - clientConfiguration.setClientRole(ClientConfiguration.CLIENT_ROLE_SYSTEM); - LOG.info("AuthProvider used by the Auditor is {}", - clientConfiguration.getClientAuthProviderFactoryClass()); - this.bkc = new BookKeeper(clientConfiguration, zkc); - LedgerManagerFactory ledgerManagerFactory = AbstractZkLedgerManagerFactory .newLedgerManagerFactory( conf, @@ -154,6 +188,8 @@ private void initialize(ServerConfiguration conf, ZooKeeper zkc) this.ledgerUnderreplicationManager = ledgerManagerFactory .newLedgerUnderreplicationManager(); this.admin = new BookKeeperAdmin(bkc, statsLogger); + LOG.info("AuthProvider used by the Auditor is {}", + admin.getConf().getClientAuthProviderFactoryClass()); if (this.ledgerUnderreplicationManager .initializeLostBookieRecoveryDelay(conf.getLostBookieRecoveryDelay())) { LOG.info("Initializing lostBookieRecoveryDelay zNode to the conif value: {}", @@ -166,7 +202,7 @@ private void initialize(ServerConfiguration conf, ZooKeeper zkc) } catch (CompatibilityException ce) { throw new UnavailableException( "CompatibilityException while initializing Auditor", ce); - } catch (IOException | BKException | KeeperException ioe) { + } catch (IOException | KeeperException ioe) { throw new UnavailableException( "Exception while initializing Auditor", ioe); } catch (InterruptedException ie) { @@ -601,13 +637,7 @@ public void operationComplete(int rc, Set fragments) { * be run very often. */ void checkAllLedgers() throws BKException, IOException, InterruptedException, KeeperException { - ZooKeeper newzk = ZooKeeperClient.newBuilder() - .connectString(ZKMetadataDriverBase.resolveZkServers(conf)) - .sessionTimeoutMs(conf.getZkTimeout()) - .build(); - - final BookKeeper client = new BookKeeper(new ClientConfiguration(conf), - newzk); + final BookKeeper client = createBookKeeperClient(conf); final BookKeeperAdmin admin = new BookKeeperAdmin(client, statsLogger); try { @@ -664,7 +694,6 @@ void checkAllLedgers() throws BKException, IOException, InterruptedException, Ke } finally { admin.close(); client.close(); - newzk.close(); } } @@ -680,7 +709,9 @@ public void shutdown() { executor.shutdownNow(); } admin.close(); - bkc.close(); + if (ownBkc) { + bkc.close(); + } } catch (InterruptedException ie) { Thread.currentThread().interrupt(); LOG.warn("Interrupted while shutting down auditor bookie", ie); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java index 1ea179c7b2e..cddb1f0f431 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java @@ -37,7 +37,10 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.ZkLayoutManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.DataFormats.AuditorVoteFormat; @@ -85,7 +88,9 @@ public class AuditorElector { private final String bookieId; private final ServerConfiguration conf; + private final BookKeeper bkc; private final ZooKeeper zkc; + private final boolean ownBkc; private final ExecutorService executor; private String myVote; @@ -97,6 +102,15 @@ public class AuditorElector { private final StatsLogger statsLogger; + @VisibleForTesting + public AuditorElector(final String bookieId, ServerConfiguration conf) throws UnavailableException { + this( + bookieId, + conf, + Auditor.createBookKeeperClientThrowUnavailableException(conf), + true); + } + /** * AuditorElector for performing the auditor election. * @@ -104,14 +118,16 @@ public class AuditorElector { * - bookie identifier, comprises HostAddress:Port * @param conf * - configuration - * @param zkc - * - ZK instance + * @param bkc + * - bookkeeper instance * @throws UnavailableException * throws unavailable exception while initializing the elector */ - public AuditorElector(final String bookieId, ServerConfiguration conf, - ZooKeeper zkc) throws UnavailableException { - this(bookieId, conf, zkc, NullStatsLogger.INSTANCE); + public AuditorElector(final String bookieId, + ServerConfiguration conf, + BookKeeper bkc, + boolean ownBkc) throws UnavailableException { + this(bookieId, conf, bkc, NullStatsLogger.INSTANCE, ownBkc); } /** @@ -121,18 +137,23 @@ public AuditorElector(final String bookieId, ServerConfiguration conf, * - bookie identifier, comprises HostAddress:Port * @param conf * - configuration - * @param zkc - * - ZK instance + * @param bkc + * - bookkeeper instance * @param statsLogger * - stats logger * @throws UnavailableException * throws unavailable exception while initializing the elector */ - public AuditorElector(final String bookieId, ServerConfiguration conf, - ZooKeeper zkc, StatsLogger statsLogger) throws UnavailableException { + public AuditorElector(final String bookieId, + ServerConfiguration conf, + BookKeeper bkc, + StatsLogger statsLogger, + boolean ownBkc) throws UnavailableException { this.bookieId = bookieId; this.conf = conf; - this.zkc = zkc; + this.bkc = bkc; + this.ownBkc = ownBkc; + this.zkc = ((ZkLayoutManager) bkc.getMetadataClientDriver().getLayoutManager()).getZk(); this.statsLogger = statsLogger; this.electionAttempts = statsLogger.getCounter(ELECTION_ATTEMPTS); basePath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf) + '/' @@ -274,7 +295,7 @@ public void run() { zkc.setData(getVotePath(""), TextFormat.printToString(builder.build()).getBytes(UTF_8), -1); - auditor = new Auditor(bookieId, conf, zkc, statsLogger); + auditor = new Auditor(bookieId, conf, bkc, false, statsLogger); auditor.start(); } else { // If not an auditor, will be watching to my predecessor and @@ -352,6 +373,13 @@ public void shutdown() throws InterruptedException { auditor.shutdown(); auditor = null; } + if (ownBkc) { + try { + bkc.close(); + } catch (BKException e) { + LOG.warn("Failed to close bookkeeper client", e); + } + } } /** 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 b43ff850e69..d63e19eac31 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,7 +20,6 @@ */ package org.apache.bookkeeper.replication; -import static com.google.common.base.Preconditions.checkArgument; import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_WORKER_SCOPE; @@ -29,18 +28,15 @@ import java.io.File; import java.io.IOException; import java.net.MalformedURLException; -import java.net.URI; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieCriticalThread; import org.apache.bookkeeper.bookie.ExitCode; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.http.HttpServer; import org.apache.bookkeeper.http.HttpServerLoader; -import org.apache.bookkeeper.meta.MetadataBookieDriver; -import org.apache.bookkeeper.meta.MetadataDrivers; -import org.apache.bookkeeper.meta.exceptions.MetadataException; -import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.server.http.BKHttpServiceProvider; @@ -53,7 +49,6 @@ import org.apache.commons.cli.ParseException; import org.apache.commons.configuration.ConfigurationException; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,7 +62,7 @@ public class AutoRecoveryMain { .getLogger(AutoRecoveryMain.class); private final ServerConfiguration conf; - final MetadataBookieDriver metadataBookieDriver; + final BookKeeper bkc; final AuditorElector auditorElector; final ReplicationWorker replicationWorker; final AutoRecoveryDeathWatcher deathWatcher; @@ -85,37 +80,22 @@ public AutoRecoveryMain(ServerConfiguration conf, StatsLogger statsLogger) throws IOException, InterruptedException, KeeperException, UnavailableException, CompatibilityException { this.conf = conf; - this.metadataBookieDriver = initializeMetadataDriver(conf, statsLogger); + this.bkc = Auditor.createBookKeeperClient(conf); - auditorElector = new AuditorElector(Bookie.getBookieAddress(conf).toString(), conf, - getZooKeeperFromMetadataDriver(metadataBookieDriver), - statsLogger.scope(AUDITOR_SCOPE)); + auditorElector = new AuditorElector( + Bookie.getBookieAddress(conf).toString(), + conf, + bkc, + statsLogger.scope(AUDITOR_SCOPE), + false); replicationWorker = new ReplicationWorker( conf, + bkc, + false, statsLogger.scope(REPLICATION_WORKER_SCOPE)); deathWatcher = new AutoRecoveryDeathWatcher(this); } - private MetadataBookieDriver initializeMetadataDriver(ServerConfiguration conf, StatsLogger statsLogger) - throws IOException { - String metadataServiceUri = conf.getMetadataServiceUriUnchecked(); - MetadataBookieDriver driver = MetadataDrivers.getBookieDriver( - URI.create(metadataServiceUri)); - try { - driver.initialize(conf, () -> {}, statsLogger); - } catch (MetadataException e) { - throw new IOException("Failed to initialize metadata driver at " + metadataServiceUri, e); - } - return driver; - } - - // it existing because AuditorElector takes zookeeper - ZooKeeper getZooKeeperFromMetadataDriver(MetadataBookieDriver driver) { - checkArgument(driver instanceof ZKMetadataBookieDriver); - ZKMetadataBookieDriver zkDriver = (ZKMetadataBookieDriver) driver; - return zkDriver.getZk(); - } - /* * Start daemons */ @@ -164,7 +144,14 @@ private void shutdown(int exitCode) { LOG.warn("Interrupted shutting down auditor elector", e); } replicationWorker.shutdown(); - metadataBookieDriver.close(); + try { + bkc.close(); + } catch (BKException e) { + LOG.warn("Failed to close bookkeeper client for auto recovery", e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.warn("Interrupted closing bookkeeper client for auto recovery", e); + } } private int getExitCode() { 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 b536fb09725..e3f9ac679d3 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,7 +19,6 @@ */ package org.apache.bookkeeper.replication; -import static org.apache.bookkeeper.replication.ReplicationStats.BK_CLIENT_SCOPE; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_DEFER_LEDGER_LOCK_RELEASE_OF_FAILED_LEDGER; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED; import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATE_EXCEPTION; @@ -53,7 +52,6 @@ import org.apache.bookkeeper.client.LedgerFragment; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.LedgerMetadata; -import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerManagerFactory; @@ -86,6 +84,7 @@ public class ReplicationWorker implements Runnable { private final BookKeeperAdmin admin; private final LedgerChecker ledgerChecker; private final BookKeeper bkc; + private final boolean ownBkc; private final Thread workerThread; private final long rwRereplicateBackoffMs; private final long openLedgerRereplicationGracePeriod; @@ -127,15 +126,20 @@ public ReplicationWorker(final ServerConfiguration conf) public ReplicationWorker(final ServerConfiguration conf, StatsLogger statsLogger) throws CompatibilityException, KeeperException, + + InterruptedException, IOException { + this(conf, Auditor.createBookKeeperClient(conf), true, statsLogger); + } + + ReplicationWorker(final ServerConfiguration conf, + BookKeeper bkc, + boolean ownBkc, + StatsLogger statsLogger) + throws CompatibilityException, KeeperException, InterruptedException, IOException { this.conf = conf; - try { - this.bkc = BookKeeper.forConfig(new ClientConfiguration(conf)) - .statsLogger(statsLogger.scope(BK_CLIENT_SCOPE)) - .build(); - } catch (BKException e) { - throw new IOException("Failed to instantiate replication worker", e); - } + this.bkc = bkc; + this.ownBkc = ownBkc; LedgerManagerFactory mFactory = AbstractZkLedgerManagerFactory .newLedgerManagerFactory( this.conf, @@ -505,13 +509,15 @@ public void shutdown() { e); Thread.currentThread().interrupt(); } - try { - bkc.close(); - } catch (InterruptedException e) { - LOG.warn("Interrupted while closing the Bookie client", e); - Thread.currentThread().interrupt(); - } catch (BKException e) { - LOG.warn("Exception while closing the Bookie client", e); + if (ownBkc) { + try { + bkc.close(); + } catch (InterruptedException e) { + LOG.warn("Interrupted while closing the Bookie client", e); + Thread.currentThread().interrupt(); + } catch (BKException e) { + LOG.warn("Exception while closing the Bookie client", e); + } } try { underreplicationManager.close(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java index 25878655a11..f32b7876938 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java @@ -35,7 +35,6 @@ import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.zookeeper.ZooKeeper; import org.junit.Test; import org.slf4j.Logger; @@ -209,14 +208,8 @@ public void testRestartAuditorBookieAfterCrashing() throws Exception { } private void startAuditorElector(String addr) throws Exception { - ZooKeeper zk = ZooKeeperClient.newBuilder() - .connectString(zkUtil.getZooKeeperConnectString()) - .sessionTimeoutMs(10000) - .build(); - zkClients.add(zk); - AuditorElector auditorElector = new AuditorElector(addr, - baseConf, zk); + baseConf); auditorElectors.put(addr, auditorElector); auditorElector.start(); LOG.debug("Starting Auditor Elector"); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index 65c69226f02..a09720b2013 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -146,8 +146,7 @@ public void tearDown() throws Exception { private void startAuditorElectors() throws Exception { for (BookieServer bserver : bs) { String addr = bserver.getLocalAddress().toString(); - AuditorElector auditorElector = new AuditorElector(addr, - baseConf, zkc); + AuditorElector auditorElector = new AuditorElector(addr, baseConf); auditorElectors.put(addr, auditorElector); auditorElector.start(); LOG.debug("Starting Auditor Elector"); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java index 3ec8ae17ef0..e53b3fffa44 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java @@ -35,12 +35,9 @@ import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; -import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; -import org.apache.zookeeper.ZooKeeper; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -56,7 +53,6 @@ public class AuditorPeriodicBookieCheckTest extends BookKeeperClusterTestCase { .getLogger(AuditorPeriodicBookieCheckTest.class); private AuditorElector auditorElector = null; - private ZooKeeper auditorZookeeper = null; private static final int CHECK_INTERVAL = 1; // run every second @@ -75,13 +71,7 @@ public void setUp() throws Exception { conf.setMetadataServiceUri(metadataServiceUri); String addr = bs.get(0).getLocalAddress().toString(); - auditorZookeeper = ZooKeeperClient.newBuilder() - .connectString(ZKMetadataDriverBase.resolveZkServers(conf)) - .sessionTimeoutMs(10000) - .build(); - - auditorElector = new AuditorElector(addr, conf, - auditorZookeeper); + auditorElector = new AuditorElector(addr, conf); auditorElector.start(); } @@ -89,7 +79,6 @@ public void setUp() throws Exception { @Override public void tearDown() throws Exception { auditorElector.shutdown(); - auditorZookeeper.close(); super.tearDown(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 0668088ed51..84788a35324 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -58,8 +58,6 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; -import org.apache.zookeeper.ZooKeeper; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -76,7 +74,6 @@ public class AuditorPeriodicCheckTest extends BookKeeperClusterTestCase { private MetadataBookieDriver driver; private HashMap auditorElectors = new HashMap(); - private List zkClients = new LinkedList(); private static final int CHECK_INTERVAL = 1; // run every second @@ -96,14 +93,7 @@ public void setUp() throws Exception { String addr = bs.get(i).getLocalAddress().toString(); - ZooKeeper zk = ZooKeeperClient.newBuilder() - .connectString(zkUtil.getZooKeeperConnectString()) - .sessionTimeoutMs(10000) - .build(); - zkClients.add(zk); - - AuditorElector auditorElector = new AuditorElector(addr, - conf, zk); + AuditorElector auditorElector = new AuditorElector(addr, conf); auditorElectors.put(addr, auditorElector); auditorElector.start(); LOG.debug("Starting Auditor Elector"); @@ -127,10 +117,6 @@ public void tearDown() throws Exception { for (AuditorElector e : auditorElectors.values()) { e.shutdown(); } - for (ZooKeeper zk : zkClients) { - zk.close(); - } - zkClients.clear(); super.tearDown(); } @@ -339,7 +325,7 @@ public void testPeriodicCheckWhenLedgerDeleted() throws Exception { try (final Auditor auditor = new Auditor( Bookie.getBookieAddress(bsConfs.get(0)).toString(), - bsConfs.get(0), zkc, NullStatsLogger.INSTANCE)) { + bsConfs.get(0), NullStatsLogger.INSTANCE)) { final AtomicBoolean exceptionCaught = new AtomicBoolean(false); final CountDownLatch latch = new CountDownLatch(1); Thread t = new Thread() { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index d14b8a4f9e2..d328bd4ac0b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -51,8 +51,6 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; -import org.apache.zookeeper.ZooKeeper; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -585,25 +583,18 @@ public void testRecoveryBookieService() throws Exception { assertEquals(HttpServer.StatusCode.OK.getValue(), response5.getStatusCode()); } - ZooKeeper auditorZookeeper; AuditorElector auditorElector; private Future startAuditorElector() throws Exception { - auditorZookeeper = ZooKeeperClient.newBuilder() - .connectString(zkUtil.getZooKeeperConnectString()) - .sessionTimeoutMs(10000) - .build(); String addr = bs.get(0).getLocalAddress().toString(); ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); conf.setAuditorPeriodicBookieCheckInterval(1); conf.setMetadataServiceUri("zk://" + zkUtil.getZooKeeperConnectString() + "/ledgers"); - auditorElector = new AuditorElector(addr, conf, - auditorZookeeper); + auditorElector = new AuditorElector(addr, conf); return auditorElector.start(); } private void stopAuditorElector() throws Exception { auditorElector.shutdown(); - auditorZookeeper.close(); } @Test From c01718e477a2245a463913790a643aa330ceb991 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 26 Sep 2018 02:43:57 -0700 Subject: [PATCH 0049/1642] [TABLE SERVICE] [LOGGING] improve logging at zk storage container manager Descriptions of the changes in this PR: *Motivation* the storage container manager probes containers to ensure containers are started and stopped to reflect to topology change. however the logging can be very annoying if there is no topology changed. *Changes* Improve the logic to only log the message when topology is changed. Author: Sijie Guo Author: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Author: Sijie Guo Author: Charan Reddy Guttapalem Reviewers: Enrico Olivelli , Jia Zhai This closes #1704 from sijie/reducing_logging_storage_containers --- .../impl/sc/ZkStorageContainerManager.java | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManager.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManager.java index b6350fec178..f4fdc4d0d95 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManager.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManager.java @@ -184,7 +184,9 @@ private boolean refreshMyAssignment() { private void processServersJoined(Set serversJoined, Map newAssignmentMap) { - log.info("Servers joined : {}", serversJoined); + if (!serversJoined.isEmpty()) { + log.info("Servers joined : {}", serversJoined); + } serversJoined.forEach(ep -> { ServerAssignmentData sad = newAssignmentMap.get(ep); if (null != sad) { @@ -195,7 +197,9 @@ private void processServersJoined(Set serversJoined, private void processServersLeft(Set serversLeft, Map oldAssignmentMap) { - log.info("Servers left : {}", serversLeft); + if (!serversLeft.isEmpty()) { + log.info("Servers left : {}", serversLeft); + } serversLeft.forEach(ep -> { ServerAssignmentData sad = oldAssignmentMap.get(ep); if (null != sad) { @@ -246,9 +250,15 @@ private void processMyAssignment(ServerAssignmentData myAssignment) { containersToStop = Sets.filter(containersToStop, container -> !pendingStartStopContainers.contains(container)); - log.info("Process container changes:\n\tIdeal = {}\n\tLive = {}\n\t" - + "Pending = {}\n\tToStart = {}\n\tToStop = {}", - assignedContainerSet, liveContainerSet, pendingStartStopContainers, containersToStart, containersToStop); + if (!containersToStart.isEmpty() || !containersToStop.isEmpty()) { + log.info("Process container changes:\n\tIdeal = {}\n\tLive = {}\n\t" + + "Pending = {}\n\tToStart = {}\n\tToStop = {}", + assignedContainerSet, + liveContainerSet, + pendingStartStopContainers, + containersToStart, + containersToStop); + } containersToStart.forEach(this::startStorageContainer); containersToStop.forEach(this::stopStorageContainer); From e0f9a59e98abe766bd5e2ec008fb90ca9bdaf981 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 26 Sep 2018 02:44:42 -0700 Subject: [PATCH 0050/1642] [TABLE SERVICE] improve the bookkeeper script to better detect whether table service is enabled or not Descriptions of the changes in this PR: *motivation* Currently `bin/bookkeeper` automatically choose modules based on the command. However it would be much better to detect which module to use by checking the bookkeeper configuration settings. *changes* Improve `bin/bookkeeper` to better detect which module to use based on bookkeeper server configuration. Additionally adding dlog related settings in the server configuration file. Author: Sijie Guo Author: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Author: Charan Reddy Guttapalem Author: Sijie Guo Reviewers: Enrico Olivelli , Jia Zhai This closes #1705 from sijie/improve_bookkeeper_scripts --- bin/bookkeeper | 24 +++++++++++++++++++----- conf/bk_server.conf | 7 +++++++ 2 files changed, 26 insertions(+), 5 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index 07e7db3ecd7..38280a1d766 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -23,7 +23,25 @@ BK_HOME=`cd ${BINDIR}/..;pwd` source ${BK_HOME}/bin/common.sh -if [ "x$1" == "xstandalone" ]; then +# default variables +DEFAULT_CONF=${BK_HOME}/conf/bk_server.conf +DEFAULT_ZK_CONF=${BK_HOME}/conf/zookeeper.conf + +if [ -z "$BOOKIE_CONF" ]; then + BOOKIE_CONF_TO_CHECK=${DEFAULT_CONF} +else + BOOKIE_CONF_TO_CHECK=${BOOKIE_CONF} +fi + +# check the configuration to see if table service is enabled or not. +if [ -z "${ENABLE_TABLE_SERVICE}" ]; then + TABLE_SERVICE_SETTING=$(grep StreamStorageLifecycleComponent ${BOOKIE_CONF_TO_CHECK}) + if [ "${TABLE_SERVICE_SETTING}" != \#* ]; then + ENABLE_TABLE_SERVICE="true" + fi +fi + +if [ \( "x$1" == "xstandalone" \) -o \( "x${ENABLE_TABLE_SERVICE}" != "x" \) ]; then BOOKIE_MODULE_PATH=stream/server BOOKIE_MODULE_NAME="(org.apache.bookkeeper-)?stream-storage-server" else @@ -37,10 +55,6 @@ BOOKIE_JAR=$(find_module_jar ${BOOKIE_MODULE_PATH} ${BOOKIE_MODULE_NAME}) # set up the classpath BOOKIE_CLASSPATH=$(set_module_classpath ${BOOKIE_MODULE_PATH}) -# default variables -DEFAULT_CONF=${BK_HOME}/conf/bk_server.conf -DEFAULT_ZK_CONF=${BK_HOME}/conf/zookeeper.conf - bookkeeper_help() { cat < diff --git a/conf/bk_server.conf b/conf/bk_server.conf index 97404fb1bb4..df6c9916ca3 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -939,6 +939,13 @@ zkEnableSecurity=false # the grpc server port to listen on. default is 4181 storageserver.grpc.port=4181 +### Dlog Settings for table service ### + +#### Replication Settings +dlog.bkcEnsembleSize=3 +dlog.bkcWriteQuorumSize=2 +dlog.bkcAckQuorumSize=2 + ### Storage ### # local storage directories for storing table ranges data (e.g. rocksdb sst files) From 8e48839dca54f827037cf4a76169fcd6e44de28e Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 26 Sep 2018 13:20:21 +0200 Subject: [PATCH 0051/1642] Release notes for Apache BookKeeper 4.8.0 Update the website with the releases notes for Apache BookKeeper 4.8.0 Author: Enrico Olivelli Reviewers: Sijie Guo This closes #1676 from eolivelli/fix/release-notes-480 --- site/_config.yml | 7 +- site/docs/4.8.0/admin/autorecovery.md | 128 +++ site/docs/4.8.0/admin/bookies.md | 180 ++++ site/docs/4.8.0/admin/geo-replication.md | 22 + site/docs/4.8.0/admin/http.md | 407 +++++++++ site/docs/4.8.0/admin/metrics.md | 41 + site/docs/4.8.0/admin/perf.md | 3 + site/docs/4.8.0/admin/placement.md | 3 + site/docs/4.8.0/admin/upgrade.md | 181 ++++ site/docs/4.8.0/api/distributedlog-api.md | 395 ++++++++ site/docs/4.8.0/api/ledger-adv-api.md | 111 +++ site/docs/4.8.0/api/ledger-api.md | 841 ++++++++++++++++++ site/docs/4.8.0/api/overview.md | 17 + site/docs/4.8.0/deployment/dcos.md | 142 +++ site/docs/4.8.0/deployment/kubernetes.md | 181 ++++ site/docs/4.8.0/deployment/manual.md | 56 ++ site/docs/4.8.0/development/codebase.md | 3 + site/docs/4.8.0/development/protocol.md | 148 +++ site/docs/4.8.0/getting-started/concepts.md | 202 +++++ .../4.8.0/getting-started/installation.md | 74 ++ .../docs/4.8.0/getting-started/run-locally.md | 16 + site/docs/4.8.0/overview/overview.md | 60 ++ site/docs/4.8.0/overview/releaseNotes.md | 73 ++ .../4.8.0/overview/releaseNotesTemplate.md | 17 + site/docs/4.8.0/reference/cli.md | 10 + site/docs/4.8.0/reference/config.md | 9 + site/docs/4.8.0/reference/metrics.md | 3 + site/docs/4.8.0/security/overview.md | 21 + site/docs/4.8.0/security/sasl.md | 202 +++++ site/docs/4.8.0/security/tls.md | 210 +++++ site/docs/4.8.0/security/zookeeper.md | 41 + site/docs/latest/overview/overview.md | 2 +- site/docs/latest/overview/releaseNotes.md | 4 +- .../latest/overview/releaseNotesTemplate.md | 2 +- site/releases.md | 4 + 35 files changed, 3809 insertions(+), 7 deletions(-) create mode 100644 site/docs/4.8.0/admin/autorecovery.md create mode 100644 site/docs/4.8.0/admin/bookies.md create mode 100644 site/docs/4.8.0/admin/geo-replication.md create mode 100644 site/docs/4.8.0/admin/http.md create mode 100644 site/docs/4.8.0/admin/metrics.md create mode 100644 site/docs/4.8.0/admin/perf.md create mode 100644 site/docs/4.8.0/admin/placement.md create mode 100644 site/docs/4.8.0/admin/upgrade.md create mode 100644 site/docs/4.8.0/api/distributedlog-api.md create mode 100644 site/docs/4.8.0/api/ledger-adv-api.md create mode 100644 site/docs/4.8.0/api/ledger-api.md create mode 100644 site/docs/4.8.0/api/overview.md create mode 100644 site/docs/4.8.0/deployment/dcos.md create mode 100644 site/docs/4.8.0/deployment/kubernetes.md create mode 100644 site/docs/4.8.0/deployment/manual.md create mode 100644 site/docs/4.8.0/development/codebase.md create mode 100644 site/docs/4.8.0/development/protocol.md create mode 100644 site/docs/4.8.0/getting-started/concepts.md create mode 100644 site/docs/4.8.0/getting-started/installation.md create mode 100644 site/docs/4.8.0/getting-started/run-locally.md create mode 100644 site/docs/4.8.0/overview/overview.md create mode 100644 site/docs/4.8.0/overview/releaseNotes.md create mode 100644 site/docs/4.8.0/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.8.0/reference/cli.md create mode 100644 site/docs/4.8.0/reference/config.md create mode 100644 site/docs/4.8.0/reference/metrics.md create mode 100644 site/docs/4.8.0/security/overview.md create mode 100644 site/docs/4.8.0/security/sasl.md create mode 100644 site/docs/4.8.0/security/tls.md create mode 100644 site/docs/4.8.0/security/zookeeper.md diff --git a/site/_config.yml b/site/_config.yml index ff35a8929ad..2b871aa8e4c 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -9,6 +9,7 @@ twitter_url: https://twitter.com/asfbookkeeper versions: # [next_version_placeholder] +- "4.8.0" - "4.7.2" - "4.7.1" - "4.7.0" @@ -30,9 +31,9 @@ archived_versions: - "4.2.0" - "4.1.0" - "4.0.0" -latest_version: "4.8.0-SNAPSHOT" -latest_release: "4.7.2" -stable_release: "4.6.2" +latest_version: "4.9.0-SNAPSHOT" +latest_release: "4.8.0" +stable_release: "4.7.2" distributedlog_version: "4.7.2" defaults: diff --git a/site/docs/4.8.0/admin/autorecovery.md b/site/docs/4.8.0/admin/autorecovery.md new file mode 100644 index 00000000000..b1dd078f9b2 --- /dev/null +++ b/site/docs/4.8.0/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: + +* the `shell recover` option +* 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 shell recover \ + 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 shell recover \ + 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.8.0/admin/bookies.md b/site/docs/4.8.0/admin/bookies.md new file mode 100644 index 00000000000..1b0427dae3c --- /dev/null +++ b/site/docs/4.8.0/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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + \ + \ + + ``` + + 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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + 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.8.0/admin/geo-replication.md b/site/docs/4.8.0/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.8.0/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.8.0/admin/http.md b/site/docs/4.8.0/admin/http.md new file mode 100644 index 00000000000..dc647449621 --- /dev/null +++ b/site/docs/4.8.0/admin/http.md @@ -0,0 +1,407 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 4 components: +1. Heartbeat: heartbeat for a specific bookie. +1. Config: doing the server configuration for a specific bookie. +1. Ledger: HTTP endpoints related to ledgers. +1. Bookie: HTTP endpoints related to bookies. +1. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Config + +### Endpoint: /metrics +1. Method: GET + * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.8.0/admin/metrics.md b/site/docs/4.8.0/admin/metrics.md new file mode 100644 index 00000000000..142df3dcd2d --- /dev/null +++ b/site/docs/4.8.0/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.prometheus.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.8.0/admin/perf.md b/site/docs/4.8.0/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.8.0/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.8.0/admin/placement.md b/site/docs/4.8.0/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.8.0/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.8.0/admin/upgrade.md b/site/docs/4.8.0/admin/upgrade.md new file mode 100644 index 00000000000..690de2dcc9e --- /dev/null +++ b/site/docs/4.8.0/admin/upgrade.md @@ -0,0 +1,181 @@ +--- +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.7.x to 4.8.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.7.x to 4.8.0. + +In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. +(Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it). +Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. +But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. + + +### 4.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.8.0/api/distributedlog-api.md b/site/docs/4.8.0/api/distributedlog-api.md new file mode 100644 index 00000000000..40ac4782cc3 --- /dev/null +++ b/site/docs/4.8.0/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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.8.0/api/ledger-adv-api.md b/site/docs/4.8.0/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.8.0/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.8.0/api/ledger-api.md b/site/docs/4.8.0/api/ledger-api.md new file mode 100644 index 00000000000..38573610113 --- /dev/null +++ b/site/docs/4.8.0/api/ledger-api.md @@ -0,0 +1,841 @@ +--- +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.8.0 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.8.0 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.8.0' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.8.0' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` + +### Relaxing Durability + +In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. +In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will +be always readable by other clients. + +On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after +writing the entry to SO buffers without waiting for an fsync. +In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. +Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. + +In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble ackknowledge the call after +performing an fsync to the disk which is storing the journal. +This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. + +The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. +In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. + + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete + + +wh.force().get(); // wait for fsync, make data available to readers and to the replicator + +wh.close(); // seal the ledger + +``` diff --git a/site/docs/4.8.0/api/overview.md b/site/docs/4.8.0/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.8.0/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.8.0/deployment/dcos.md b/site/docs/4.8.0/deployment/dcos.md new file mode 100644 index 00000000000..9aca61c2e14 --- /dev/null +++ b/site/docs/4.8.0/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.8.0] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.8.0/deployment/kubernetes.md b/site/docs/4.8.0/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.8.0/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.8.0/deployment/manual.md b/site/docs/4.8.0/deployment/manual.md new file mode 100644 index 00000000000..daafd5556f5 --- /dev/null +++ b/site/docs/4.8.0/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.8.0/development/codebase.md b/site/docs/4.8.0/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.8.0/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.8.0/development/protocol.md b/site/docs/4.8.0/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.8.0/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.8.0/getting-started/concepts.md b/site/docs/4.8.0/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.8.0/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.8.0/getting-started/installation.md b/site/docs/4.8.0/getting-started/installation.md new file mode 100644 index 00000000000..9986cd8e043 --- /dev/null +++ b/site/docs/4.8.0/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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.8.0/getting-started/run-locally.md b/site/docs/4.8.0/getting-started/run-locally.md new file mode 100644 index 00000000000..edbfab9fda6 --- /dev/null +++ b/site/docs/4.8.0/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 +$ bookkeeper-server/bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.8.0/overview/overview.md b/site/docs/4.8.0/overview/overview.md new file mode 100644 index 00000000000..80de28b55bf --- /dev/null +++ b/site/docs/4.8.0/overview/overview.md @@ -0,0 +1,60 @@ +--- +title: Apache BookKeeper™ 4.8.0 +--- + + +This documentation is for Apache BookKeeper™ version 4.8.0. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.8.0 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.8.0/overview/releaseNotes.md b/site/docs/4.8.0/overview/releaseNotes.md new file mode 100644 index 00000000000..7614131d64b --- /dev/null +++ b/site/docs/4.8.0/overview/releaseNotes.md @@ -0,0 +1,73 @@ +--- +title: Apache BookKeeper 4.8.0 Release Notes +--- + +This is the 13th release of Apache BookKeeper! + +The 4.8.0 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.7.0. +It is a new big milestone in Apache BookKeeper community, +this release include great new features, like Relaxed Durability, Stream Storage service and Multiple active Entrylogs. + +Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.8.0](../../admin/upgrade). The technical details of +this release are summarized below. + +## Highlights + +The main features in 4.8.0 cover are around following areas: + +- Public API +- Durability +- ExplicitLAC feature +- New Table Storage Service +- Bug Fixes + + +### Public API + +There are multiple new client features introduced in 4.8.0. Here are two highlighted features: + +#### New WriteFlag DEFERRED_SYNC + +The writer may ask for temporary relaxed durability writes, that is to receive early acknowledge from Bookies, before an fsync() on Journal. +Together with this new flag we introduced the new WriteHandle#force() API, this this API the writer is able to request an explicit guarantee of durability to the Bookies +it is mostly like and explicit fsync() on a file system. + +See [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) and [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) for reference + +#### New behaviour for Netty ByteBuf reference count management + +All the client side APIs which take ByteBufs now will have the net effect of decrementing by 1 the refcount. +This is consistent with general contract of Netty. +It is expected that the client passes the ownership of the ByteBuf to BookKeeper client. + +#### Multiple Active Entrylogs + +It is now possible on the Bookie to have multiple active entry loggers, +this new feature will help with compaction performance and some specific workloads. + +See [Multiple active entrylogs](https://github.com/apache/bookkeeper/issues/570) + +## Table Storage Service + +From this version we are providing the a table (key/value) service embedded in Bookies. + +See [BP-30: BookKeeper Table Service](https://github.com/apache/bookkeeper/issues/1205) + +## Make ExplicitLAC persistent + +ExplicitLAC was contributed from Salesforce in 4.5.0 release, but in the first release +it was a beft-effort in-memory mechanism. Now you can configure Bookies to store durably ExplicitLAC. + +See [Make ExplicitLAC persistent](https://github.com/apache/bookkeeper/issues/1527) + +## Ensemble change on Delayed Write Failure + +We are handling more gracefully the case of a failure of a Bookie in spite of a succeeded write. +If you are writing with Ack Quorum = 2 and Write Quorum = 3, writes will succeeed even if 1 of 3 Bookies fail, +now BookKeeper will trigger an *ensemble change* and replace the failed bookie earlier. + +See [Ensemble change on Delayed Write Failure](https://github.com/apache/bookkeeper/issues/1390) + +## Full list of changes + +- [https://github.com/apache/bookkeeper/milestone/4](https://github.com/apache/bookkeeper/milestone/4?closed=1) diff --git a/site/docs/4.8.0/overview/releaseNotesTemplate.md b/site/docs/4.8.0/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..3ad4d98b1b5 --- /dev/null +++ b/site/docs/4.8.0/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.8.0 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.8.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.8.0/reference/cli.md b/site/docs/4.8.0/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.8.0/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.8.0/reference/config.md b/site/docs/4.8.0/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.8.0/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.8.0/reference/metrics.md b/site/docs/4.8.0/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.8.0/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.8.0/security/overview.md b/site/docs/4.8.0/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.8.0/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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.8.0/security/sasl.md b/site/docs/4.8.0/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.8.0/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.8.0/security/tls.md b/site/docs/4.8.0/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.8.0/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.8.0/security/zookeeper.md b/site/docs/4.8.0/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.8.0/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) diff --git a/site/docs/latest/overview/overview.md b/site/docs/latest/overview/overview.md index 2b89711d117..66452dc96d0 100644 --- a/site/docs/latest/overview/overview.md +++ b/site/docs/latest/overview/overview.md @@ -1,5 +1,5 @@ --- -title: Apache BookKeeper™ 4.8.0-SNAPSHOT +title: Apache BookKeeper™ 4.9.0-SNAPSHOT --- + true +
org.apache.bookkeeper.tests stream-storage-tests-common diff --git a/stream/distributedlog/core/pom.xml b/stream/distributedlog/core/pom.xml index 0e7d8603664..5617f2c4c1e 100644 --- a/stream/distributedlog/core/pom.xml +++ b/stream/distributedlog/core/pom.xml @@ -54,6 +54,12 @@ bookkeeper-server ${project.parent.version} + + javax.annotation + javax.annotation-api + + true + org.jmock jmock diff --git a/stream/proto/pom.xml b/stream/proto/pom.xml index 8a6ad978741..7c3e86fdfde 100644 --- a/stream/proto/pom.xml +++ b/stream/proto/pom.xml @@ -42,6 +42,12 @@ com.google.protobuf protobuf-java + + javax.annotation + javax.annotation-api + + true + org.apache.bookkeeper.tests stream-storage-tests-common diff --git a/stream/tests-common/pom.xml b/stream/tests-common/pom.xml index 971aecc792f..6f760251c55 100644 --- a/stream/tests-common/pom.xml +++ b/stream/tests-common/pom.xml @@ -37,6 +37,12 @@ com.google.protobuf protobuf-java + + javax.annotation + javax.annotation-api + + true + From 7d9ee98946239c02c78b2a9e59994b259d8cc52e Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 2 Oct 2018 08:33:58 -0700 Subject: [PATCH 0065/1642] [TABLE SERVICE] [CLIENT] Provide a simple version of storage clients Descriptions of the changes in this PR: *Motivation* The PR #1721 provides a routing service for proxying grpc requests to the correct storage container. That means that the storage client can be implemented using simple grpc calls since all containers can serve same grpc requests. *Changes* Implement a simple version of storage clients by using server-side routing to route grpc requests to the correct storage containers. Author: Sijie Guo Reviewers: Jia Zhai This closes #1722 from sijie/proxy_table_requests2 --- .../bookkeeper/clients/SimpleClientBase.java | 71 ++++ .../clients/SimpleStorageClientImpl.java | 110 ++++++ .../clients/StorageClientBuilder.java | 19 +- .../admin/SimpleStorageAdminClientImpl.java | 154 +++++++++ .../clients/config/StorageClientSettings.java | 14 + .../StorageContainerClientInterceptor.java | 19 ++ .../bookkeeper/clients/utils/RetryUtils.java | 123 +++++++ .../impl/kv/PByteBufSimpleTableImpl.java | 320 ++++++++++++++++++ .../stream/StorageAdminClientTest.java | 46 ++- .../stream/StreamClusterTestBase.java | 5 + .../integration/stream/TableClientTest.java | 50 ++- 11 files changed, 885 insertions(+), 46 deletions(-) create mode 100644 stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleClientBase.java create mode 100644 stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java create mode 100644 stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java create mode 100644 stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/RetryUtils.java create mode 100644 stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleClientBase.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleClientBase.java new file mode 100644 index 00000000000..3f6bc776ce7 --- /dev/null +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleClientBase.java @@ -0,0 +1,71 @@ +/* + * 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.clients; + +import io.grpc.Channel; +import io.grpc.ClientInterceptors; +import io.grpc.ManagedChannel; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.clients.impl.container.StorageContainerClientInterceptor; +import org.apache.bookkeeper.clients.utils.ClientResources; +import org.apache.bookkeeper.clients.utils.GrpcChannels; +import org.apache.bookkeeper.clients.utils.RetryUtils; +import org.apache.bookkeeper.common.util.AbstractAutoAsyncCloseable; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.common.util.SharedResourceManager; +import org.apache.bookkeeper.common.util.SharedResourceManager.Resource; + +/** + * Simple client implementation base. + */ +public class SimpleClientBase extends AbstractAutoAsyncCloseable { + + protected final Resource schedulerResource; + protected final OrderedScheduler scheduler; + protected final ManagedChannel managedChannel; + protected final Channel channel; + protected final RetryUtils retryUtils; + + protected SimpleClientBase(StorageClientSettings settings) { + this(settings, ClientResources.create().scheduler()); + } + + protected SimpleClientBase(StorageClientSettings settings, + Resource schedulerResource) { + this.managedChannel = GrpcChannels.createChannelBuilder(settings.serviceUri(), settings).build(); + this.channel = ClientInterceptors.intercept( + managedChannel, + new StorageContainerClientInterceptor(0L)); + + this.schedulerResource = schedulerResource; + this.scheduler = SharedResourceManager.shared().get(schedulerResource); + this.retryUtils = RetryUtils.create(settings.backoffPolicy(), scheduler); + } + + @Override + protected void closeAsyncOnce(CompletableFuture closeFuture) { + managedChannel.shutdown(); + scheduler.submit(() -> { + SharedResourceManager.shared().release(schedulerResource, scheduler); + closeFuture.complete(null); + }); + } +} diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java new file mode 100644 index 00000000000..4817dccb1dd --- /dev/null +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java @@ -0,0 +1,110 @@ +/* + * 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.clients; + +import static org.apache.bookkeeper.clients.impl.internal.ProtocolInternalUtils.createRootRangeException; +import static org.apache.bookkeeper.common.util.ListenableFutures.fromListenableFuture; +import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.createGetStreamRequest; + +import io.grpc.CallOptions; +import io.netty.buffer.ByteBuf; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.api.StorageClient; +import org.apache.bookkeeper.api.exceptions.ApiException; +import org.apache.bookkeeper.api.kv.PTable; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.clients.impl.kv.ByteBufTableImpl; +import org.apache.bookkeeper.clients.impl.kv.PByteBufSimpleTableImpl; +import org.apache.bookkeeper.clients.utils.GrpcUtils; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.common.util.ExceptionUtils; +import org.apache.bookkeeper.stream.proto.StorageType; +import org.apache.bookkeeper.stream.proto.StreamProperties; +import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc; +import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc.RootRangeServiceFutureStub; +import org.apache.bookkeeper.stream.proto.storage.StatusCode; + +/** + * The implementation of {@link StorageClient} client. + */ +@Slf4j +class SimpleStorageClientImpl extends SimpleClientBase implements StorageClient { + + private static final String COMPONENT_NAME = SimpleStorageClientImpl.class.getSimpleName(); + + private final String namespaceName; + private final RootRangeServiceFutureStub rootRangeService; + + public SimpleStorageClientImpl(String namespaceName, + StorageClientSettings settings) { + super(settings); + this.namespaceName = namespaceName; + this.rootRangeService = GrpcUtils.configureGrpcStub( + RootRangeServiceGrpc.newFutureStub(channel), + Optional.empty()); + } + + // + // Materialized Views + // + + @Override + public CompletableFuture> openPTable(String streamName) { + return ExceptionUtils.callAndHandleClosedAsync( + COMPONENT_NAME, + isClosed(), + (future) -> openStreamAsTableImpl(streamName, future)); + } + + @Override + public CompletableFuture> openTable(String table) { + return openPTable(table) + .thenApply(pTable -> new ByteBufTableImpl(pTable)); + } + + private void openStreamAsTableImpl(String streamName, + CompletableFuture> future) { + CompletableFuture getStreamFuture = retryUtils.execute(() -> + fromListenableFuture(rootRangeService.getStream( + createGetStreamRequest(namespaceName, streamName))) + ).thenCompose(resp -> { + if (StatusCode.SUCCESS == resp.getCode()) { + StreamProperties streamProps = resp.getStreamProps(); + log.info("Retrieved table properties for table {} : {}", streamName, streamProps); + if (StorageType.TABLE != streamProps.getStreamConf().getStorageType()) { + return FutureUtils.exception(new ApiException( + "Can't open a non-table storage entity : " + streamProps.getStreamConf().getStorageType())); + } else { + return FutureUtils.value(streamProps); + } + } else { + return FutureUtils.exception(createRootRangeException(namespaceName, resp.getCode())); + } + }); + FutureUtils.proxyTo( + getStreamFuture.thenApply(streamProps -> + new PByteBufSimpleTableImpl(streamProps, managedChannel, CallOptions.DEFAULT, retryUtils)), + future); + } + +} diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientBuilder.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientBuilder.java index 10282b29ae3..3a7ff4c8ee1 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientBuilder.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientBuilder.java @@ -23,6 +23,7 @@ import java.util.function.Supplier; import org.apache.bookkeeper.api.StorageClient; +import org.apache.bookkeeper.clients.admin.SimpleStorageAdminClientImpl; import org.apache.bookkeeper.clients.admin.StorageAdminClient; import org.apache.bookkeeper.clients.admin.StorageAdminClientImpl; import org.apache.bookkeeper.clients.config.StorageClientSettings; @@ -83,10 +84,14 @@ public StorageClient build() { checkNotNull(settings, "Stream settings is null"); checkArgument(validateNamespaceName(namespaceName), "Namespace name '" + namespaceName + "'is invalid"); - return new StorageClientImpl( - namespaceName, - settings, - ClientResources.create()); + if (settings.enableServerSideRouting()) { + return new SimpleStorageClientImpl(namespaceName, settings); + } else { + return new StorageClientImpl( + namespaceName, + settings, + ClientResources.create()); + } } /** @@ -97,7 +102,11 @@ public StorageClient build() { public StorageAdminClient buildAdmin() { checkNotNull(settings, "Storage client settings is null"); - return new StorageAdminClientImpl(settings); + if (settings.enableServerSideRouting()) { + return new SimpleStorageAdminClientImpl(settings); + } else { + return new StorageAdminClientImpl(settings); + } } @Override diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java new file mode 100644 index 00000000000..cbacbc396dd --- /dev/null +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java @@ -0,0 +1,154 @@ +/* + * 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.clients.admin; + +import static org.apache.bookkeeper.clients.impl.internal.ProtocolInternalUtils.createRootRangeException; +import static org.apache.bookkeeper.common.util.ListenableFutures.fromListenableFuture; +import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.createCreateNamespaceRequest; +import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.createCreateStreamRequest; +import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.createDeleteNamespaceRequest; +import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.createDeleteStreamRequest; +import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.createGetNamespaceRequest; +import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.createGetStreamRequest; + +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.clients.SimpleClientBase; +import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.clients.utils.ClientResources; +import org.apache.bookkeeper.clients.utils.GrpcUtils; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.common.util.SharedResourceManager.Resource; +import org.apache.bookkeeper.stream.proto.NamespaceConfiguration; +import org.apache.bookkeeper.stream.proto.NamespaceProperties; +import org.apache.bookkeeper.stream.proto.StreamConfiguration; +import org.apache.bookkeeper.stream.proto.StreamProperties; +import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc; +import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc.RootRangeServiceFutureStub; +import org.apache.bookkeeper.stream.proto.storage.StatusCode; + +/** + * A simple implementation of {@link StorageAdminClient}. + * + *

Unlike {@link StorageAdminClientImpl} which handles all the routing logic, this implementation + * just make simple grpc calls to the storage service, the storage service handles the grpc requests + * and handle proper routing at the server side. + */ +public class SimpleStorageAdminClientImpl extends SimpleClientBase implements StorageAdminClient { + + private final RootRangeServiceFutureStub rootRangeService; + + public SimpleStorageAdminClientImpl(StorageClientSettings settings) { + this(settings, ClientResources.create().scheduler()); + } + + public SimpleStorageAdminClientImpl(StorageClientSettings settings, + Resource schedulerResource) { + super(settings, schedulerResource); + this.rootRangeService = GrpcUtils.configureGrpcStub( + RootRangeServiceGrpc.newFutureStub(channel), + Optional.empty()); + } + + @Override + public CompletableFuture createNamespace(String namespace, NamespaceConfiguration conf) { + return retryUtils.execute(() -> + fromListenableFuture(rootRangeService.createNamespace(createCreateNamespaceRequest(namespace, conf))) + ).thenCompose(resp -> { + if (StatusCode.SUCCESS == resp.getCode()) { + return FutureUtils.value(resp.getNsProps()); + } else { + return FutureUtils.exception(createRootRangeException(namespace, resp.getCode())); + } + }); + } + + @Override + public CompletableFuture deleteNamespace(String namespace) { + return retryUtils.execute(() -> + fromListenableFuture(rootRangeService.deleteNamespace(createDeleteNamespaceRequest(namespace))) + ).thenCompose(resp -> { + if (StatusCode.SUCCESS == resp.getCode()) { + return FutureUtils.value(true); + } else { + return FutureUtils.exception(createRootRangeException(namespace, resp.getCode())); + } + }); + } + + @Override + public CompletableFuture getNamespace(String namespace) { + return retryUtils.execute(() -> + fromListenableFuture(rootRangeService.getNamespace(createGetNamespaceRequest(namespace))) + ).thenCompose(resp -> { + if (StatusCode.SUCCESS == resp.getCode()) { + return FutureUtils.value(resp.getNsProps()); + } else { + return FutureUtils.exception(createRootRangeException(namespace, resp.getCode())); + } + }); + } + + @Override + public CompletableFuture createStream(String namespace, + String streamName, + StreamConfiguration streamConfiguration) { + return retryUtils.execute(() -> + fromListenableFuture(rootRangeService.createStream( + createCreateStreamRequest(namespace, streamName, streamConfiguration))) + ).thenCompose(resp -> { + if (StatusCode.SUCCESS == resp.getCode()) { + return FutureUtils.value(resp.getStreamProps()); + } else { + return FutureUtils.exception(createRootRangeException(namespace, resp.getCode())); + } + }); + } + + @Override + public CompletableFuture deleteStream(String namespace, String streamName) { + return retryUtils.execute(() -> + fromListenableFuture(rootRangeService.deleteStream( + createDeleteStreamRequest(namespace, streamName))) + ).thenCompose(resp -> { + if (StatusCode.SUCCESS == resp.getCode()) { + return FutureUtils.value(true); + } else { + return FutureUtils.exception(createRootRangeException(namespace, resp.getCode())); + } + }); + } + + @Override + public CompletableFuture getStream(String namespace, String streamName) { + return retryUtils.execute(() -> + fromListenableFuture(rootRangeService.getStream( + createGetStreamRequest(namespace, streamName))) + ).thenCompose(resp -> { + if (StatusCode.SUCCESS == resp.getCode()) { + return FutureUtils.value(resp.getStreamProps()); + } else { + return FutureUtils.exception(createRootRangeException(namespace, resp.getCode())); + } + }); + } + +} diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/config/StorageClientSettings.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/config/StorageClientSettings.java index 9a0fc6d821d..e78ca73e835 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/config/StorageClientSettings.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/config/StorageClientSettings.java @@ -81,6 +81,19 @@ public interface StorageClientSettings { */ Backoff.Policy backoffPolicy(); + /** + * Configure whether to enable server side routing or not. + * + *

By default, the client implementation will does client side routing, which will talk to storage containers + * directly, however sometimes if you can simply expose storage containers addresses due to network security + * constraints, you can enable server side routing. in server side routing mode, the clients simply make + * grpc calls to any storage container, those storage containers will route the requests accordingly to the + * right storage container. In this mode, the storage containers act as grpc proxies. + * + * @return flag whether to enable server side routing or not. + */ + boolean enableServerSideRouting(); + /** * Builder of {@link StorageClientSettings} instances. */ @@ -91,6 +104,7 @@ class Builder extends StorageClientSettings_Builder { usePlaintext(true); backoffPolicy(ClientConstants.DEFAULT_INFINIT_BACKOFF_POLICY); endpointResolver(EndpointResolver.identity()); + enableServerSideRouting(false); } @Override diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerClientInterceptor.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerClientInterceptor.java index 1e1de080258..efdb47f1f22 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerClientInterceptor.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerClientInterceptor.java @@ -27,11 +27,13 @@ import io.grpc.ClientInterceptors.CheckedForwardingClientCall; import io.grpc.Metadata; import io.grpc.MethodDescriptor; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.grpc.netty.LongBinaryMarshaller; /** * A client interceptor that intercepting outgoing calls to storage containers. */ +@Slf4j public class StorageContainerClientInterceptor implements ClientInterceptor { private final long scId; @@ -48,13 +50,30 @@ public StorageContainerClientInterceptor(long scId) { public ClientCall interceptCall(MethodDescriptor method, CallOptions callOptions, Channel next) { + if (log.isTraceEnabled()) { + log.trace("Intercepting method {} : req marshaller = {}, resp marshaller = {}", + method.getFullMethodName(), + method.getRequestMarshaller(), + method.getResponseMarshaller()); + } return new CheckedForwardingClientCall(next.newCall(method, callOptions)) { @Override protected void checkedStart(Listener responseListener, Metadata headers) throws Exception { + if (log.isTraceEnabled()) { + log.trace("Attaching storage container {},", scId); + } headers.put(scIdKey, scId); delegate().start(responseListener, headers); } + + @Override + public void request(int numMessages) { + if (log.isTraceEnabled()) { + log.trace("request {} messages", numMessages); + } + super.request(numMessages); + } }; } } diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/RetryUtils.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/RetryUtils.java new file mode 100644 index 00000000000..06dcc137be2 --- /dev/null +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/RetryUtils.java @@ -0,0 +1,123 @@ +/* + * 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.clients.utils; + +import static org.apache.bookkeeper.common.util.ListenableFutures.fromListenableFuture; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.ListenableFuture; +import io.grpc.Status; +import io.grpc.StatusException; +import io.grpc.StatusRuntimeException; +import java.util.concurrent.CompletableFuture; +import java.util.function.Predicate; +import java.util.function.Supplier; +import org.apache.bookkeeper.common.util.Backoff; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.common.util.Retries; + +/** + * Retry Utils. + */ +public class RetryUtils { + + @VisibleForTesting + static final Predicate DEFAULT_CLIENT_RETRY_PREDICATE = + cause -> shouldRetryOnException(cause); + + private static boolean shouldRetryOnException(Throwable cause) { + if (cause instanceof StatusRuntimeException || cause instanceof StatusException) { + Status status; + if (cause instanceof StatusException) { + status = ((StatusException) cause).getStatus(); + } else { + status = ((StatusRuntimeException) cause).getStatus(); + } + switch (status.getCode()) { + case INVALID_ARGUMENT: + case ALREADY_EXISTS: + case PERMISSION_DENIED: + case UNAUTHENTICATED: + return false; + default: + return true; + } + } else if (cause instanceof RuntimeException) { + return false; + } else { + // storage level exceptions + return false; + } + } + + public static RetryUtils create(Backoff.Policy backoffPolicy, + OrderedScheduler scheduler) { + return create(DEFAULT_CLIENT_RETRY_PREDICATE, backoffPolicy, scheduler); + } + + public static RetryUtils create(Predicate retryPredicate, + Backoff.Policy backoffPolicy, + OrderedScheduler scheduler) { + return new RetryUtils(retryPredicate, backoffPolicy, scheduler); + } + + private final Predicate retryPredicate; + private final Backoff.Policy backoffPolicy; + private final OrderedScheduler scheduler; + + private RetryUtils(Predicate retryPredicate, + Backoff.Policy backoffPolicy, + OrderedScheduler scheduler) { + this.retryPredicate = retryPredicate; + this.backoffPolicy = backoffPolicy; + this.scheduler = scheduler; + } + + /** + * Run the action with retries. + * + * @param action action to run + * @return the result of the action + */ + public CompletableFuture execute(Supplier> action) { + return Retries.run( + backoffPolicy.toBackoffs(), + retryPredicate, + action, + scheduler + ); + } + + /** + * Run the action with retries. + * + * @param action action to run + * @return the result of the action + */ + public CompletableFuture executeListenable(Supplier> action) { + return Retries.run( + backoffPolicy.toBackoffs(), + retryPredicate, + () -> fromListenableFuture(action.get()), + scheduler + ); + } + +} diff --git a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java new file mode 100644 index 00000000000..c9170051fb9 --- /dev/null +++ b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java @@ -0,0 +1,320 @@ +/* + * 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.clients.impl.kv; + +import static org.apache.bookkeeper.clients.impl.kv.KvUtils.toProtoCompare; +import static org.apache.bookkeeper.clients.impl.kv.KvUtils.toProtoRequest; +import static org.apache.bookkeeper.common.util.ListenableFutures.fromListenableFuture; +import static org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.getDeleteMethod; +import static org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.getIncrementMethod; +import static org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.getPutMethod; +import static org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.getRangeMethod; +import static org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.getTxnMethod; +import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.RK_METADATA_KEY; +import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.SID_METADATA_KEY; + +import com.google.common.collect.Lists; +import com.google.protobuf.UnsafeByteOperations; +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.ClientCall; +import io.grpc.ClientInterceptor; +import io.grpc.ClientInterceptors; +import io.grpc.ClientInterceptors.CheckedForwardingClientCall; +import io.grpc.Metadata; +import io.grpc.MethodDescriptor; +import io.grpc.stub.AbstractStub; +import io.grpc.stub.ClientCalls; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufUtil; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.api.kv.PTable; +import org.apache.bookkeeper.api.kv.Txn; +import org.apache.bookkeeper.api.kv.impl.op.OpFactoryImpl; +import org.apache.bookkeeper.api.kv.impl.result.KeyValueFactory; +import org.apache.bookkeeper.api.kv.impl.result.ResultFactory; +import org.apache.bookkeeper.api.kv.op.CompareOp; +import org.apache.bookkeeper.api.kv.op.Op; +import org.apache.bookkeeper.api.kv.op.OpFactory; +import org.apache.bookkeeper.api.kv.options.DeleteOption; +import org.apache.bookkeeper.api.kv.options.IncrementOption; +import org.apache.bookkeeper.api.kv.options.PutOption; +import org.apache.bookkeeper.api.kv.options.RangeOption; +import org.apache.bookkeeper.api.kv.result.DeleteResult; +import org.apache.bookkeeper.api.kv.result.IncrementResult; +import org.apache.bookkeeper.api.kv.result.PutResult; +import org.apache.bookkeeper.api.kv.result.RangeResult; +import org.apache.bookkeeper.api.kv.result.TxnResult; +import org.apache.bookkeeper.clients.utils.RetryUtils; +import org.apache.bookkeeper.stream.proto.StreamProperties; +import org.apache.bookkeeper.stream.proto.kv.rpc.RoutingHeader; +import org.apache.bookkeeper.stream.proto.kv.rpc.TxnRequest; + +/** + * A {@link PTable} implementation using simple grpc calls. + */ +@Slf4j +public class PByteBufSimpleTableImpl + extends AbstractStub + implements PTable { + + private static class RoutingHeaderInterceptor implements ClientInterceptor { + + private final long streamId; + private final ByteBuf rKey; + + RoutingHeaderInterceptor(long streamId, ByteBuf rKey) { + this.streamId = streamId; + this.rKey = rKey; + } + + @Override + public ClientCall interceptCall(MethodDescriptor method, + CallOptions callOptions, + Channel next) { + return new CheckedForwardingClientCall(next.newCall(method, callOptions)) { + @Override + protected void checkedStart(Listener responseListener, Metadata headers) throws Exception { + headers.put(SID_METADATA_KEY, streamId); + headers.put(RK_METADATA_KEY, ByteBufUtil.getBytes(rKey)); + delegate().start(responseListener, headers); + } + }; + } + } + + private final OpFactory opFactory; + private final ResultFactory resultFactory; + private final KeyValueFactory kvFactory; + private final StreamProperties streamProps; + private final long streamId; + private final RetryUtils retryUtils; + + public PByteBufSimpleTableImpl(StreamProperties streamProps, + Channel channel, + CallOptions callOptions, + RetryUtils retryUtils) { + super(channel, callOptions); + this.streamProps = streamProps; + this.streamId = streamProps.getStreamId(); + this.opFactory = new OpFactoryImpl<>(); + this.resultFactory = new ResultFactory<>(); + this.kvFactory = new KeyValueFactory<>(); + this.retryUtils = retryUtils; + } + + private RoutingHeader.Builder newRoutingHeader(ByteBuf pKey) { + return RoutingHeader.newBuilder() + .setStreamId(streamId) + .setRKey(UnsafeByteOperations.unsafeWrap(pKey.nioBuffer())); + } + + private Channel getChannel(ByteBuf pKey) { + RoutingHeaderInterceptor interceptor = new RoutingHeaderInterceptor(streamId, pKey); + return ClientInterceptors.intercept(getChannel(), interceptor); + } + + @Override + public CompletableFuture> get( + ByteBuf pKey, ByteBuf lKey, RangeOption option + ) { + pKey.retain(); + lKey.retain(); + if (null != option.endKey()) { + option.endKey().retain(); + } + return retryUtils.execute(() -> fromListenableFuture( + ClientCalls.futureUnaryCall( + getChannel(pKey).newCall(getRangeMethod(), getCallOptions()), + KvUtils.newRangeRequest(lKey, option) + .setHeader(newRoutingHeader(pKey)) + .build()) + )) + .thenApply(response -> KvUtils.newRangeResult(response, resultFactory, kvFactory)) + .whenComplete((value, cause) -> { + pKey.release(); + lKey.release(); + if (null != option.endKey()) { + option.endKey().release(); + } + }); + } + + @Override + public CompletableFuture> put( + ByteBuf pKey, ByteBuf lKey, ByteBuf value, PutOption option + ) { + pKey.retain(); + lKey.retain(); + value.retain(); + return retryUtils.execute(() -> fromListenableFuture( + ClientCalls.futureUnaryCall( + getChannel(pKey).newCall(getPutMethod(), getCallOptions()), + KvUtils.newPutRequest(lKey, value, option) + .setHeader(newRoutingHeader(pKey)) + .build()) + )) + .thenApply(response -> KvUtils.newPutResult(response, resultFactory, kvFactory)) + .whenComplete((ignored, cause) -> { + pKey.release(); + lKey.release(); + value.release(); + }); + } + + @Override + public CompletableFuture> delete( + ByteBuf pKey, ByteBuf lKey, DeleteOption option + ) { + pKey.retain(); + lKey.retain(); + if (null != option.endKey()) { + option.endKey().retain(); + } + return retryUtils.execute(() -> fromListenableFuture( + ClientCalls.futureUnaryCall( + getChannel(pKey).newCall(getDeleteMethod(), getCallOptions()), + KvUtils.newDeleteRequest(lKey, option) + .setHeader(newRoutingHeader(pKey)) + .build()) + )) + .thenApply(response -> KvUtils.newDeleteResult(response, resultFactory, kvFactory)) + .whenComplete((ignored, cause) -> { + pKey.release(); + lKey.release(); + if (null != option.endKey()) { + option.endKey().release(); + } + }); + } + + @Override + public CompletableFuture> increment( + ByteBuf pKey, ByteBuf lKey, long amount, IncrementOption option + ) { + pKey.retain(); + lKey.retain(); + return retryUtils.execute(() -> fromListenableFuture( + ClientCalls.futureUnaryCall( + getChannel(pKey).newCall(getIncrementMethod(), getCallOptions()), + KvUtils.newIncrementRequest(lKey, amount, option) + .setHeader(newRoutingHeader(pKey)) + .build()) + )) + .thenApply(response -> KvUtils.newIncrementResult(response, resultFactory, kvFactory)) + .whenComplete((ignored, cause) -> { + pKey.release(); + lKey.release(); + }); + } + + @Override + public Txn txn(ByteBuf pKey) { + return new TxnImpl(pKey); + } + + @Override + public OpFactory opFactory() { + return opFactory; + } + + @Override + public void close() { + // no-op + } + + // + // Txn Implementation + // + + class TxnImpl implements Txn { + + private final ByteBuf pKey; + private final TxnRequest.Builder txnBuilder; + private final List resourcesToRelease; + + TxnImpl(ByteBuf pKey) { + this.pKey = pKey.retain(); + this.txnBuilder = TxnRequest.newBuilder(); + this.resourcesToRelease = Lists.newArrayList(); + } + + @SuppressWarnings("unchecked") + @Override + public Txn If(CompareOp... cmps) { + for (CompareOp cmp : cmps) { + txnBuilder.addCompare(toProtoCompare(cmp)); + resourcesToRelease.add(cmp); + } + return this; + } + + @SuppressWarnings("unchecked") + @Override + public Txn Then(Op... ops) { + for (Op op : ops) { + txnBuilder.addSuccess(toProtoRequest(op)); + resourcesToRelease.add(op); + } + return this; + } + + @SuppressWarnings("unchecked") + @Override + public Txn Else(Op... ops) { + for (Op op : ops) { + txnBuilder.addFailure(toProtoRequest(op)); + resourcesToRelease.add(op); + } + return this; + } + + @Override + public CompletableFuture> commit() { + return retryUtils.execute(() -> fromListenableFuture( + ClientCalls.futureUnaryCall( + getChannel(pKey).newCall(getTxnMethod(), getCallOptions()), + txnBuilder.setHeader(newRoutingHeader(pKey)).build()) + )) + .thenApply(response -> KvUtils.newKvTxnResult(response, resultFactory, kvFactory)) + .whenComplete((ignored, cause) -> { + pKey.release(); + for (AutoCloseable resource : resourcesToRelease) { + closeResource(resource); + } + }); + } + + private void closeResource(AutoCloseable resource) { + try { + resource.close(); + } catch (Exception e) { + log.warn("Fail to close resource {}", resource, e); + } + } + } + + @Override + protected PByteBufSimpleTableImpl build(Channel channel, CallOptions callOptions) { + return new PByteBufSimpleTableImpl(streamProps, channel, callOptions, retryUtils); + } +} diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StorageAdminClientTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StorageAdminClientTest.java index 5aaf89e20ef..e897564d05c 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StorageAdminClientTest.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StorageAdminClientTest.java @@ -30,15 +30,13 @@ import org.apache.bookkeeper.clients.exceptions.StreamExistsException; import org.apache.bookkeeper.clients.exceptions.StreamNotFoundException; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.common.testing.annotations.FlakyTest; import org.apache.bookkeeper.stream.proto.NamespaceConfiguration; import org.apache.bookkeeper.stream.proto.NamespaceProperties; import org.apache.bookkeeper.stream.proto.StreamConfiguration; import org.apache.bookkeeper.stream.proto.StreamProperties; import org.apache.bookkeeper.stream.proto.storage.StatusCode; -import org.junit.After; -import org.junit.Before; import org.junit.Rule; +import org.junit.Test; import org.junit.rules.TestName; /** @@ -49,22 +47,24 @@ public class StorageAdminClientTest extends StreamClusterTestBase { @Rule public final TestName testName = new TestName(); - private StorageAdminClient adminClient; + @Test + public void testNamespaceAPIClientSideRouting() throws Exception { + testNamespaceAPI(false); + } - @Before - public void setup() { - adminClient = createStorageAdminClient(newStorageClientSettings()); + @Test + public void testNamespaceAPIServerSideRouting() throws Exception { + testNamespaceAPI(true); } - @After - public void teardown() { - if (null != adminClient) { - adminClient.close(); + private void testNamespaceAPI(boolean enableServerSideRouting) throws Exception { + try (StorageAdminClient adminClient = + createStorageAdminClient(newStorageClientSettings(enableServerSideRouting))) { + testNamespaceAPI(adminClient); } } - @FlakyTest("https://github.com/apache/bookkeeper/issues/1440") - public void testNamespaceAPI() throws Exception { + private void testNamespaceAPI(StorageAdminClient adminClient) throws Exception { // Create a namespace String nsName = testName.getMethodName(); NamespaceConfiguration colConf = NamespaceConfiguration.newBuilder() @@ -119,8 +119,24 @@ public void testNamespaceAPI() throws Exception { } } - @FlakyTest("https://github.com/apache/bookkeeper/issues/1440") - public void testStreamAPI() throws Exception { + @Test + public void testStreamAPIClientSideRouting() throws Exception { + testStreamAPI(false); + } + + @Test + public void testStreamAPIServerSideRouting() throws Exception { + testStreamAPI(false); + } + + private void testStreamAPI(boolean enableServerSideRouting) throws Exception { + try (StorageAdminClient adminClient = + createStorageAdminClient(newStorageClientSettings(enableServerSideRouting))) { + testStreamAPI(adminClient); + } + } + + private void testStreamAPI(StorageAdminClient adminClient) throws Exception { // Create a namespace String nsName = testName.getMethodName() + "_ns"; NamespaceConfiguration colConf = NamespaceConfiguration.newBuilder() diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StreamClusterTestBase.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StreamClusterTestBase.java index bbde846be2a..04a91a4b496 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StreamClusterTestBase.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StreamClusterTestBase.java @@ -99,6 +99,10 @@ protected static List getInternalStreamEndpoints() { // protected static StorageClientSettings newStorageClientSettings() { + return newStorageClientSettings(false); + } + + protected static StorageClientSettings newStorageClientSettings(boolean enableServerSideRouting) { String serviceUri = String.format( "bk://%s/", getExsternalStreamEndpoints().stream() @@ -114,6 +118,7 @@ protected static StorageClientSettings newStorageClientSettings() { return NetUtils.parseEndpoint(externalEndpointStr); }) .usePlaintext(true) + .enableServerSideRouting(enableServerSideRouting) .build(); } diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientTest.java index 467e455fad9..81c3c853109 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientTest.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientTest.java @@ -50,15 +50,13 @@ import org.apache.bookkeeper.clients.admin.StorageAdminClient; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.common.testing.annotations.FlakyTest; import org.apache.bookkeeper.stream.proto.NamespaceConfiguration; import org.apache.bookkeeper.stream.proto.NamespaceProperties; import org.apache.bookkeeper.stream.proto.StorageType; import org.apache.bookkeeper.stream.proto.StreamConfiguration; import org.apache.bookkeeper.stream.proto.StreamProperties; -import org.junit.After; -import org.junit.Before; import org.junit.Rule; +import org.junit.Test; import org.junit.rules.TestName; /** @@ -70,29 +68,8 @@ public class TableClientTest extends StreamClusterTestBase { @Rule public final TestName testName = new TestName(); - private final String namespace = "test_namespace"; - private StorageAdminClient adminClient; - private StorageClient storageClient; private final OptionFactory optionFactory = new OptionFactoryImpl<>(); - @Before - public void setup() { - StorageClientSettings settings = newStorageClientSettings(); - String namespace = "test_namespace"; - adminClient = createStorageAdminClient(settings); - storageClient = createStorageClient(settings, namespace); - } - - @After - public void teardown() { - if (null != adminClient) { - adminClient.close(); - } - if (null != storageClient) { - storageClient.close(); - } - } - private static ByteBuf getLKey(int i) { return Unpooled.wrappedBuffer(String.format("test-lkey-%06d", i).getBytes(UTF_8)); } @@ -101,8 +78,29 @@ private static ByteBuf getValue(int i) { return Unpooled.wrappedBuffer(String.format("test-val-%06d", i).getBytes(UTF_8)); } - @FlakyTest("https://github.com/apache/bookkeeper/issues/1440") - public void testTableAPI() throws Exception { + @Test + public void testTableAPIClientSideRouting() throws Exception { + testTableAPI(false); + } + + @Test + public void testTableAPIServerSideRouting() throws Exception { + testTableAPI(true); + } + + private void testTableAPI(boolean enableServerSideRouting) throws Exception { + StorageClientSettings setting = newStorageClientSettings(enableServerSideRouting); + try (StorageAdminClient adminClient = createStorageAdminClient(setting)) { + final String namespace = testName.getMethodName(); + try (StorageClient storageClient = createStorageClient(setting, namespace)) { + testTableAPI(namespace, adminClient, storageClient); + } + } + } + + private void testTableAPI(String namespace, + StorageAdminClient adminClient, + StorageClient storageClient) throws Exception { // Create a namespace NamespaceConfiguration nsConf = NamespaceConfiguration.newBuilder() .setDefaultStreamConf(DEFAULT_STREAM_CONF) From 327d9056302435f98e8347f29334801fb1416836 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 2 Oct 2018 11:35:47 -0700 Subject: [PATCH 0066/1642] [TABLE SERVICE] [CLIENT] Implement the python client to access tables Descriptions of the changes in this PR: *Motivation* Providing a python client to access tables Fixes #1690 *Changes* Implement the python client to access tables using the table grpc requests Author: Sijie Guo Reviewers: Jia Zhai This closes #1723 from sijie/python_client --- .../clients/python/bookkeeper/admin/client.py | 8 +- .../python/bookkeeper/admin/namespace.py | 44 +++- .../python/bookkeeper/admin/namespaces.py | 36 ++- .../python/bookkeeper/common/constants.py | 2 +- .../python/bookkeeper/common/exceptions.py | 169 +++++++++++++- .../bookkeeper/common/future/polling.py | 9 +- .../python/bookkeeper/common/method.py | 214 ++++++++++++++++++ .../clients/python/bookkeeper/common/retry.py | 4 +- .../python/bookkeeper/common/timeout.py | 213 +++++++++++++++++ stream/clients/python/bookkeeper/kv/client.py | 27 ++- stream/clients/python/bookkeeper/kv/table.py | 124 ++++++++++ .../python/bookkeeper/proto/__init__.py | 13 ++ stream/clients/python/examples/admin.py | 43 ++++ .../stream/protocol/ProtocolConstants.java | 8 +- 14 files changed, 875 insertions(+), 39 deletions(-) create mode 100644 stream/clients/python/bookkeeper/common/method.py create mode 100644 stream/clients/python/bookkeeper/common/timeout.py create mode 100644 stream/clients/python/bookkeeper/kv/table.py create mode 100644 stream/clients/python/bookkeeper/proto/__init__.py diff --git a/stream/clients/python/bookkeeper/admin/client.py b/stream/clients/python/bookkeeper/admin/client.py index d3dff3fab77..fdb82d2699a 100644 --- a/stream/clients/python/bookkeeper/admin/client.py +++ b/stream/clients/python/bookkeeper/admin/client.py @@ -53,20 +53,20 @@ def __init__(self, storage_client_settings=(), **kwargs): assert service_uri.service_name.lower() == 'bk' # create channel - self.channel = grpc.insecure_channel( + self.__channel__ = grpc.insecure_channel( target=service_uri.service_location ) __logger__.info("Successfully created an admin client to cluster '%s'", self.storage_client_settings.service_uri) # create the rpc stub - self.root_range = RootRangeServiceStub(channel=self.channel) + self.__root_range__ = RootRangeServiceStub(channel=self.__channel__) # services - self.__namespaces__ = Namespaces(client=self) + self.__namespaces__ = Namespaces(self.__root_range__) def namespaces(self): return self.__namespaces__ def namespace(self, namespace): - return Namespace(self, namespace) + return Namespace(self.__root_range__, namespace) diff --git a/stream/clients/python/bookkeeper/admin/namespace.py b/stream/clients/python/bookkeeper/admin/namespace.py index 029c2c488be..5c3910b1ce6 100644 --- a/stream/clients/python/bookkeeper/admin/namespace.py +++ b/stream/clients/python/bookkeeper/admin/namespace.py @@ -16,45 +16,71 @@ from bookkeeper.common.constants import __DEFAULT_STREAM_CONF__ from bookkeeper.common.constants import __ROOT_RANGE_METADATA__ +from bookkeeper.common.exceptions import from_root_range_rpc_response +from bookkeeper.common.method import wrap_method +from bookkeeper.common.retry import Retry +from bookkeeper.common.timeout import ExponentialTimeout from bookkeeper.proto import storage_pb2 from bookkeeper.proto import stream_pb2 class Namespace(object): - def __init__(self, client, namespace): - self.client = client - self.namespace = namespace + def __init__(self, root_range_service, namespace): + self.__root_range_service__ = root_range_service + self.__namespace__ = namespace + self.__default_retry__ = Retry(deadline=60) + self.__default_timeout__ = ExponentialTimeout() + self.__create_with_retries__ =\ + wrap_method(self.__create_stream__, self.__default_retry__) + self.__get_with_retries__ =\ + wrap_method(self.__get_stream__, self.__default_retry__) + self.__delete_with_retries__ =\ + wrap_method(self.__delete_stream__, self.__default_retry__) def create(self, stream_name, stream_config=__DEFAULT_STREAM_CONF__): + return self.__create_with_retries__(stream_name, stream_config) + + def __create_stream__(self, stream_name, stream_config): create_stream_req = storage_pb2.CreateStreamRequest( - ns_name=self.namespace, + ns_name=self.__namespace__, name=stream_name, stream_conf=stream_config ) - return self.client.root_range.CreateStream( + create_stream_resp = self.__root_range_service__.CreateStream( request=create_stream_req, metadata=__ROOT_RANGE_METADATA__ ) + create_stream_resp = from_root_range_rpc_response(create_stream_resp) + return create_stream_resp.stream_props def get(self, stream_name): + return self.__get_with_retries__(stream_name) + + def __get_stream__(self, stream_name): get_stream_req = storage_pb2.GetStreamRequest( stream_name=stream_pb2.StreamName( - namespace_name=self.namespace, + namespace_name=self.__namespace__, stream_name=stream_name ) ) - return self.client.root_range.GetStream( + get_stream_resp = self.__root_range_service__.GetStream( request=get_stream_req, metadata=__ROOT_RANGE_METADATA__ ) + get_stream_resp = from_root_range_rpc_response(get_stream_resp) + return get_stream_resp.stream_props def delete(self, stream_name): + return self.__delete_with_retries__(stream_name) + + def __delete_stream__(self, stream_name): del_stream_req = storage_pb2.DeleteStreamRequest( - ns_name=self.namespace, + ns_name=self.__namespace__, name=stream_name ) - return self.client.root_range.DeleteStream( + del_stream_resp = self.__root_range_service__.DeleteStream( request=del_stream_req, metadata=__ROOT_RANGE_METADATA__ ) + from_root_range_rpc_response(del_stream_resp) diff --git a/stream/clients/python/bookkeeper/admin/namespaces.py b/stream/clients/python/bookkeeper/admin/namespaces.py index 7589d081323..3df8cbd57d3 100644 --- a/stream/clients/python/bookkeeper/admin/namespaces.py +++ b/stream/clients/python/bookkeeper/admin/namespaces.py @@ -16,38 +16,64 @@ from bookkeeper.common.constants import __DEFAULT_NS_CONF__ from bookkeeper.common.constants import __ROOT_RANGE_METADATA__ +from bookkeeper.common.exceptions import from_root_range_rpc_response +from bookkeeper.common.method import wrap_method +from bookkeeper.common.retry import Retry +from bookkeeper.common.timeout import ExponentialTimeout from bookkeeper.proto import storage_pb2 class Namespaces(object): - def __init__(self, client): - self.client = client + def __init__(self, root_range_service): + self.__root_range_service__ = root_range_service + self.__default_retry__ = Retry(deadline=60) + self.__default_timeout__ = ExponentialTimeout() + self.__create_with_retries__ =\ + wrap_method(self.__create_ns__, self.__default_retry__) + self.__get_with_retries__ =\ + wrap_method(self.__get_ns__, self.__default_retry__) + self.__delete_with_retries__ =\ + wrap_method(self.__delete_ns__, self.__default_retry__) def create(self, namespace, namespace_config=__DEFAULT_NS_CONF__): + return self.__create_with_retries__(namespace, namespace_config) + + def __create_ns__(self, namespace, namespace_config): create_ns_req = storage_pb2.CreateNamespaceRequest( name=namespace, ns_conf=namespace_config ) - return self.client.root_range.CreateNamespace( + create_ns_resp = self.__root_range_service__.CreateNamespace( request=create_ns_req, metadata=__ROOT_RANGE_METADATA__ ) + create_ns_resp = from_root_range_rpc_response(create_ns_resp) + return create_ns_resp.ns_props def get(self, namespace): + return self.__get_with_retries__(namespace) + + def __get_ns__(self, namespace): get_ns_req = storage_pb2.GetNamespaceRequest( name=namespace ) - return self.client.root_range.GetNamespace( + get_ns_resp = self.__root_range_service__.GetNamespace( request=get_ns_req, metadata=__ROOT_RANGE_METADATA__ ) + get_ns_resp = from_root_range_rpc_response(get_ns_resp) + return get_ns_resp.ns_props def delete(self, namespace): + return self.__delete_with_retries__(namespace) + + def __delete_ns__(self, namespace): del_ns_req = storage_pb2.DeleteNamespaceRequest( name=namespace ) - return self.client.root_range.DeleteNamespace( + del_ns_resp = self.__root_range_service__.DeleteNamespace( request=del_ns_req, metadata=__ROOT_RANGE_METADATA__ ) + from_root_range_rpc_response(del_ns_resp) diff --git a/stream/clients/python/bookkeeper/common/constants.py b/stream/clients/python/bookkeeper/common/constants.py index bec9e83d182..deb85a93468 100644 --- a/stream/clients/python/bookkeeper/common/constants.py +++ b/stream/clients/python/bookkeeper/common/constants.py @@ -16,7 +16,7 @@ __ROOT_RANGE_ID__ = 0 __ROOT_RANGE_METADATA__ = [ - ('sc-id-bin', __ROOT_RANGE_ID__.to_bytes(8, "little")) + ('bk-rt-sc-id-bin', __ROOT_RANGE_ID__.to_bytes(8, "big")) ] __DEFAULT_STREAM_CONF__ = stream_pb2.StreamConfiguration( key_type=stream_pb2.RangeKeyType.values()[0], diff --git a/stream/clients/python/bookkeeper/common/exceptions.py b/stream/clients/python/bookkeeper/common/exceptions.py index 34ff1227dcf..782a579d5b7 100644 --- a/stream/clients/python/bookkeeper/common/exceptions.py +++ b/stream/clients/python/bookkeeper/common/exceptions.py @@ -17,8 +17,7 @@ from __future__ import absolute_import from __future__ import unicode_literals -from bookkeeper.proto.storage_pb2 import BAD_REQUEST -from bookkeeper.proto.storage_pb2 import INTERNAL_SERVER_ERROR +from bookkeeper.proto import storage_pb2 import six @@ -122,18 +121,132 @@ class ClientError(BKGrpcCallError): class BadRequest(ClientError): """Exception mapping a ``400 Bad Request`` response.""" - code = BAD_REQUEST + code = storage_pb2.BAD_REQUEST + grpc_status_code =\ + grpc.StatusCode.FAILED_PRECONDITION if grpc is not None else None + + +class IllegalOpError(ClientError): + """Exception mapping a ``403 Illegal Op`` response.""" + code = storage_pb2.ILLEGAL_OP + grpc_status_code =\ + grpc.StatusCode.FAILED_PRECONDITION if grpc is not None else None class ServerError(BKGrpcCallError): """Base for 5xx responses.""" +class StorageContainerNotFoundError(BKGrpcCallError): + """Exception raised when storage container is not found""" + code = None + grpc_status_code =\ + grpc.StatusCode.NOT_FOUND if grpc is not None else None + + class InternalServerError(ServerError): """Exception mapping a ``500 Internal Server Error`` response. or a :attr:`grpc.StatusCode.INTERNAL` error.""" - code = INTERNAL_SERVER_ERROR - grpc_status_code = grpc.StatusCode.INTERNAL if grpc is not None else None + code = storage_pb2.INTERNAL_SERVER_ERROR + grpc_status_code =\ + grpc.StatusCode.INTERNAL if grpc is not None else None + + +class UnimplementedError(ServerError): + code = storage_pb2.NOT_IMPLEMENTED + grpc_status_code =\ + grpc.StatusCode.UNIMPLEMENTED if grpc is not None else None + + +class UnexpectedError(BKGrpcCallError): + code = storage_pb2.UNEXPECTED + grpc_status_code =\ + grpc.StatusCode.UNKNOWN if grpc is not None else None + + +class StorageError(BKGrpcCallError): + grpc_status_code = None + + +class FailureError(StorageError): + code = storage_pb2.FAILURE + + +class BadVersionError(StorageError): + code = storage_pb2.BAD_VERSION + + +class BadRevisionError(StorageError): + code = storage_pb2.BAD_REVISION + + +class NamespaceError(StorageError): + code = storage_pb2.UNEXPECTED + + +class InvalidNamespaceNameError(NamespaceError): + code = storage_pb2.INVALID_NAMESPACE_NAME + + +class NamespaceNotFoundError(NamespaceError): + code = storage_pb2.NAMESPACE_NOT_FOUND + + +class NamespaceExistsError(NamespaceError): + code = storage_pb2.NAMESPACE_EXISTS + + +class StreamError(StorageError): + code = storage_pb2.UNEXPECTED + + +class InvalidStreamNameError(StreamError): + code = storage_pb2.INVALID_STREAM_NAME + + +class StreamNotFoundError(StreamError): + code = storage_pb2.STREAM_NOT_FOUND + + +class StreamExistsError(StreamError): + code = storage_pb2.STREAM_EXISTS + + +class TableError(StorageError): + code = storage_pb2.UNEXPECTED + + +class InvalidKeyError(TableError): + code = storage_pb2.INVALID_KEY + + +class KeyNotFoundError(TableError): + code = storage_pb2.KEY_NOT_FOUND + + +class KeyExistsError(TableError): + code = storage_pb2.KEY_EXISTS + + +_BK_CODE_TO_EXCEPTION_ = { + storage_pb2.FAILURE: FailureError, + storage_pb2.BAD_REQUEST: BadRequest, + storage_pb2.ILLEGAL_OP: IllegalOpError, + storage_pb2.INTERNAL_SERVER_ERROR: InternalServerError, + storage_pb2.NOT_IMPLEMENTED: UnimplementedError, + storage_pb2.UNEXPECTED: UnexpectedError, + storage_pb2.BAD_VERSION: BadVersionError, + storage_pb2.BAD_REVISION: BadRevisionError, + storage_pb2.INVALID_NAMESPACE_NAME: InvalidNamespaceNameError, + storage_pb2.NAMESPACE_EXISTS: NamespaceExistsError, + storage_pb2.NAMESPACE_NOT_FOUND: NamespaceNotFoundError, + storage_pb2.INVALID_STREAM_NAME: InvalidStreamNameError, + storage_pb2.STREAM_EXISTS: StreamExistsError, + storage_pb2.STREAM_NOT_FOUND: StreamNotFoundError, + storage_pb2.INVALID_KEY: InvalidKeyError, + storage_pb2.KEY_EXISTS: KeyExistsError, + storage_pb2.KEY_NOT_FOUND: KeyNotFoundError +} def exception_class_for_grpc_status(status_code): @@ -180,6 +293,52 @@ def from_grpc_error(rpc_exc): rpc_exc.details(), errors=(rpc_exc,), response=rpc_exc) + elif isinstance(rpc_exc, grpc.RpcError): + return from_grpc_error( + rpc_exc.code(), + rpc_exc.details(), + errors=(rpc_exc,), + response=rpc_exc + ) else: return BKGrpcCallError( str(rpc_exc), errors=(rpc_exc,), response=rpc_exc) + + +def exception_class_for_bk_status_code(status_code): + return _BK_CODE_TO_EXCEPTION_.get(status_code, BKGrpcCallError) + + +def from_bk_status(status_code, message, **kwargs): + error_class = exception_class_for_bk_status_code(status_code) + error = error_class(message, **kwargs) + + if error.bk_status_code is None: + error.bk_status_code = status_code + + return error + + +def from_table_rpc_response(rpc_resp): + routing_header = rpc_resp.header + status_code = routing_header.code + if storage_pb2.SUCCESS == status_code: + return rpc_resp + else: + raise from_bk_status( + status_code, + "", + errors=(rpc_resp,), + response=rpc_resp) + + +def from_root_range_rpc_response(rpc_resp): + status_code = rpc_resp.code + if storage_pb2.SUCCESS == status_code: + return rpc_resp + else: + raise from_bk_status( + status_code, + "", + errors=(rpc_resp,), + response=rpc_resp) diff --git a/stream/clients/python/bookkeeper/common/future/polling.py b/stream/clients/python/bookkeeper/common/future/polling.py index 2bde3a87913..d6842076594 100644 --- a/stream/clients/python/bookkeeper/common/future/polling.py +++ b/stream/clients/python/bookkeeper/common/future/polling.py @@ -99,8 +99,9 @@ def result(self, timeout=None): Returns: google.protobuf.Message: The Operation's result. Raises: - bookkeeper.common.exceptions.BKGrpcAPICallError: If the operation errors or if - the timeout is reached before the operation completes. + bookkeeper.common.exceptions.BKGrpcAPICallError: If the operation + errors or if the timeout is reached before the operation + completes. """ self._blocking_poll(timeout=timeout) @@ -117,8 +118,8 @@ def exception(self, timeout=None): timeout (int): How long to wait for the operation to complete. If None, wait indefinitely. Returns: - Optional[bookkeeper.common.exceptions.BKGrpcAPICallError]: The operation's - error. + Optional[bookkeeper.common.exceptions.BKGrpcAPICallError]: + The operation's error. """ self._blocking_poll() return self._exception diff --git a/stream/clients/python/bookkeeper/common/method.py b/stream/clients/python/bookkeeper/common/method.py new file mode 100644 index 00000000000..be6cd65a527 --- /dev/null +++ b/stream/clients/python/bookkeeper/common/method.py @@ -0,0 +1,214 @@ +# Licensed 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. + +"""Helpers for wrapping low-level gRPC methods with common functionality. +""" + +from bookkeeper.common import general_helpers +from bookkeeper.common import grpc_helpers +from bookkeeper.common import timeout + +DEFAULT = object() +"""Sentinel value indicating that a retry or timeout argument was unspecified, +so the default should be used.""" + + +def _is_not_none_or_false(value): + return value is not None and value is not False + + +def _apply_decorators(func, decorators): + """Apply a list of decorators to a given function. + + ``decorators`` may contain items that are ``None`` or ``False`` which will + be ignored. + """ + decorators = filter(_is_not_none_or_false, reversed(decorators)) + + for decorator in decorators: + func = decorator(func) + + return func + + +def _determine_timeout(default_timeout, specified_timeout, retry): + """Determines how timeout should be applied to a wrapped method. + + Args: + default_timeout (Optional[Timeout]): The default timeout specified + at method creation time. + specified_timeout (Optional[Timeout]): The timeout specified at + invocation time. If :attr:`DEFAULT`, this will be set to + the ``default_timeout``. + retry (Optional[Retry]): The retry specified at invocation time. + + Returns: + Optional[Timeout]: The timeout to apply to the method or ``None``. + """ + if specified_timeout is DEFAULT: + specified_timeout = default_timeout + + if specified_timeout is default_timeout: + # If timeout is the default and the default timeout is exponential and + # a non-default retry is specified, make sure the timeout's deadline + # matches the retry's. This handles the case where the user leaves + # the timeout default but specifies a lower deadline via the retry. + if (retry and retry is not DEFAULT + and isinstance(default_timeout, timeout.ExponentialTimeout)): + return default_timeout.with_deadline(retry._deadline) + else: + return default_timeout + + # If timeout is specified as a number instead of a Timeout instance, + # convert it to a ConstantTimeout. + if isinstance(specified_timeout, (int, float)): + return timeout.ConstantTimeout(specified_timeout) + else: + return specified_timeout + + +class _GrpcCallable(object): + """Callable that applies retry, timeout, and metadata logic. + + Args: + target (Callable): The low-level RPC method. + retry (bookkeeper.common.retry.Retry): The default retry for the + callable. If ``None``, this callable will not retry by default + timeout (bookkeeper.common.timeout.Timeout): The default timeout + for the callable. If ``None``, this callable will not specify + a timeout argument to the low-level RPC method by default. + metadata (Sequence[Tuple[str, str]]): Additional metadata that is + provided to the RPC method on every invocation. This is merged with + any metadata specified during invocation. If ``None``, no + additional metadata will be passed to the RPC method. + """ + + def __init__(self, target, retry, timeout, metadata=None): + self._target = target + self._retry = retry + self._timeout = timeout + self._metadata = metadata + + def __call__(self, *args, **kwargs): + """Invoke the low-level RPC with retry, timeout, and metadata.""" + # Note: Due to Python 2 lacking keyword-only arguments we use kwargs to + # extract the retry and timeout params. + timeout_ = _determine_timeout( + self._timeout, + kwargs.pop('timeout', self._timeout), + # Use only the invocation-specified retry only for this, as we only + # want to adjust the timeout deadline if the *user* specified + # a different retry. + kwargs.get('retry', None)) + + retry = kwargs.pop('retry', self._retry) + + if retry is DEFAULT: + retry = self._retry + + # Apply all applicable decorators. + wrapped_func = _apply_decorators(self._target, [retry, timeout_]) + + # Add the user agent metadata to the call. + if self._metadata is not None: + metadata = kwargs.get('metadata', []) + # Due to the nature of invocation, None should be treated the same + # as not specified. + if metadata is None: + metadata = [] + metadata = list(metadata) + metadata.extend(self._metadata) + kwargs['metadata'] = metadata + + return wrapped_func(*args, **kwargs) + + +def wrap_method(func, default_retry=None, default_timeout=None): + """Wrap an RPC method with common behavior. + + This applies common error wrapping, retry, and timeout behavior a function. + The wrapped function will take optional ``retry`` and ``timeout`` + arguments. + + For example:: + + import bookkeeper.common.method + from bookkeeper.common import retry + from bookkeeper.common import timeout + + # The original RPC method. + def get_topic(name, timeout=None): + request = publisher_v2.GetTopicRequest(name=name) + return publisher_stub.GetTopic(request, timeout=timeout) + + default_retry = retry.Retry(deadline=60) + default_timeout = timeout.Timeout(deadline=60) + wrapped_get_topic = bookkeeper.common.method.wrap_method( + get_topic, default_retry) + + # Execute get_topic with default retry and timeout: + response = wrapped_get_topic() + + # Execute get_topic without doing any retying but with the default + # timeout: + response = wrapped_get_topic(retry=None) + + # Execute get_topic but only retry on 5xx errors: + my_retry = retry.Retry(retry.if_exception_type( + exceptions.InternalServerError)) + response = wrapped_get_topic(retry=my_retry) + + The way this works is by late-wrapping the given function with the retry + and timeout decorators. Essentially, when ``wrapped_get_topic()`` is + called: + + * ``get_topic()`` is first wrapped with the ``timeout`` into + ``get_topic_with_timeout``. + * ``get_topic_with_timeout`` is wrapped with the ``retry`` into + ``get_topic_with_timeout_and_retry()``. + * The final ``get_topic_with_timeout_and_retry`` is called passing through + the ``args`` and ``kwargs``. + + The callstack is therefore:: + + method.__call__() -> + Retry.__call__() -> + Timeout.__call__() -> + wrap_errors() -> + get_topic() + + Note that if ``timeout`` or ``retry`` is ``None``, then they are not + applied to the function. For example, + ``wrapped_get_topic(timeout=None, retry=None)`` is more or less + equivalent to just calling ``get_topic`` but with error re-mapping. + + Args: + func (Callable[Any]): The function to wrap. It should accept an + optional ``timeout`` argument. If ``metadata`` is not ``None``, it + should accept a ``metadata`` argument. + default_retry (Optional[bookkeeper.common.Retry]): The default retry + strategy. If ``None``, the method will not retry by default. + default_timeout (Optional[bookkeeper.common.Timeout]): The default + timeout strategy. Can also be specified as an int or float. If + ``None``, the method will not have timeout specified by default. + + Returns: + Callable: A new callable that takes optional ``retry`` and ``timeout`` + arguments and applies the common error mapping, retry, timeout, + and metadata behavior to the low-level RPC method. + """ + func = grpc_helpers.wrap_errors(func) + + return general_helpers.wraps(func)( + _GrpcCallable( + func, default_retry, default_timeout + )) diff --git a/stream/clients/python/bookkeeper/common/retry.py b/stream/clients/python/bookkeeper/common/retry.py index f75905a02f7..fac1fcc3c77 100644 --- a/stream/clients/python/bookkeeper/common/retry.py +++ b/stream/clients/python/bookkeeper/common/retry.py @@ -79,7 +79,9 @@ def if_exception_type_predicate(exception): # Pylint sees this as a constant, but it is also an alias that should be # considered a function. if_transient_error = if_exception_type(( - exceptions.InternalServerError)) + exceptions.InternalServerError, + exceptions.StorageContainerNotFoundError +)) """A predicate that checks if an exception is a transient API error. The following server errors are considered transient: - :class:`bookkeeper.common.exceptions.InternalServerError` - gRPC diff --git a/stream/clients/python/bookkeeper/common/timeout.py b/stream/clients/python/bookkeeper/common/timeout.py new file mode 100644 index 00000000000..c705b28a0dc --- /dev/null +++ b/stream/clients/python/bookkeeper/common/timeout.py @@ -0,0 +1,213 @@ +# Licensed 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. + +"""Decorators for applying timeout arguments to functions. + +These decorators are used to wrap API methods to apply either a constant +or exponential timeout argument. + +For example, imagine an API method that can take a while to return results, +such as one that might block until a resource is ready: + +.. code-block:: python + + def is_thing_ready(timeout=None): + response = requests.get('https://example.com/is_thing_ready') + response.raise_for_status() + return response.json() + +This module allows a function like this to be wrapped so that timeouts are +automatically determined, for example: + +.. code-block:: python + + timeout_ = timeout.ExponentialTimeout() + is_thing_ready_with_timeout = timeout_(is_thing_ready) + + for n in range(10): + try: + is_thing_ready_with_timeout({'example': 'data'}) + except: + pass + +In this example the first call to ``is_thing_ready`` will have a relatively +small timeout (like 1 second). If the resource is available and the request +completes quickly, the loop exits. But, if the resource isn't yet available +and the request times out, it'll be retried - this time with a larger timeout. + +In the broader context these decorators are typically combined with +:mod:`bookkeeper.common.retry` to implement API methods with a signature that +matches ``api_method(request, timeout=None, retry=None)``. +""" + +from __future__ import unicode_literals + +import datetime + +import six + +from bookkeeper.common import datetime_helpers +from bookkeeper.common import general_helpers + +_DEFAULT_INITIAL_TIMEOUT = 5.0 # seconds +_DEFAULT_MAXIMUM_TIMEOUT = 30.0 # seconds +_DEFAULT_TIMEOUT_MULTIPLIER = 2.0 +# If specified, must be in seconds. If none, deadline is not used in the +# timeout calculation. +_DEFAULT_DEADLINE = None + + +@six.python_2_unicode_compatible +class ConstantTimeout(object): + """A decorator that adds a constant timeout argument. + + This is effectively equivalent to + ``functools.partial(func, timeout=timeout)``. + + Args: + timeout (Optional[float]): the timeout (in seconds) to applied to the + wrapped function. If `None`, the target function is expected to + never timeout. + """ + def __init__(self, timeout=None): + self._timeout = timeout + + def __call__(self, func): + """Apply the timeout decorator. + + Args: + func (Callable): The function to apply the timeout argument to. + This function must accept a timeout keyword argument. + + Returns: + Callable: The wrapped function. + """ + @general_helpers.wraps(func) + def func_with_timeout(*args, **kwargs): + """Wrapped function that adds timeout.""" + kwargs['timeout'] = self._timeout + return func(*args, **kwargs) + return func_with_timeout + + def __str__(self): + return ''.format(self._timeout) + + +def _exponential_timeout_generator(initial, maximum, multiplier, deadline): + """A generator that yields exponential timeout values. + + Args: + initial (float): The initial timeout. + maximum (float): The maximum timeout. + multiplier (float): The multiplier applied to the timeout. + deadline (float): The overall deadline across all invocations. + + Yields: + float: A timeout value. + """ + if deadline is not None: + deadline_datetime = ( + datetime_helpers.utcnow() + + datetime.timedelta(seconds=deadline)) + else: + deadline_datetime = datetime.datetime.max + + timeout = initial + while True: + now = datetime_helpers.utcnow() + yield min( + # The calculated timeout based on invocations. + timeout, + # The set maximum timeout. + maximum, + # The remaining time before the deadline is reached. + float((deadline_datetime - now).seconds)) + timeout = timeout * multiplier + + +@six.python_2_unicode_compatible +class ExponentialTimeout(object): + """A decorator that adds an exponentially increasing timeout argument. + + This is useful if a function is called multiple times. Each time the + function is called this decorator will calculate a new timeout parameter + based on the the number of times the function has been called. + + For example + + .. code-block:: python + + Args: + initial (float): The initial timeout to pass. + maximum (float): The maximum timeout for any one call. + multiplier (float): The multiplier applied to the timeout for each + invocation. + deadline (Optional[float]): The overall deadline across all + invocations. This is used to prevent a very large calculated + timeout from pushing the overall execution time over the deadline. + This is especially useful in conjuction with + :mod:`bookkeeper.common.retry`. If ``None``, the timeouts will not + be adjusted to accomodate an overall deadline. + """ + def __init__( + self, + initial=_DEFAULT_INITIAL_TIMEOUT, + maximum=_DEFAULT_MAXIMUM_TIMEOUT, + multiplier=_DEFAULT_TIMEOUT_MULTIPLIER, + deadline=_DEFAULT_DEADLINE): + self._initial = initial + self._maximum = maximum + self._multiplier = multiplier + self._deadline = deadline + + def with_deadline(self, deadline): + """Return a copy of this teimout with the given deadline. + + Args: + deadline (float): The overall deadline across all invocations. + + Returns: + ExponentialTimeout: A new instance with the given deadline. + """ + return ExponentialTimeout( + initial=self._initial, + maximum=self._maximum, + multiplier=self._multiplier, + deadline=deadline) + + def __call__(self, func): + """Apply the timeout decorator. + + Args: + func (Callable): The function to apply the timeout argument to. + This function must accept a timeout keyword argument. + + Returns: + Callable: The wrapped function. + """ + timeouts = _exponential_timeout_generator( + self._initial, self._maximum, self._multiplier, self._deadline) + + @general_helpers.wraps(func) + def func_with_timeout(*args, **kwargs): + """Wrapped function that adds timeout.""" + kwargs['timeout'] = next(timeouts) + return func(*args, **kwargs) + + return func_with_timeout + + def __str__(self): + return ( + ''.format( + self._initial, self._maximum, self._multiplier, + self._deadline)) diff --git a/stream/clients/python/bookkeeper/kv/client.py b/stream/clients/python/bookkeeper/kv/client.py index 314fab51c3d..3de12919905 100644 --- a/stream/clients/python/bookkeeper/kv/client.py +++ b/stream/clients/python/bookkeeper/kv/client.py @@ -19,7 +19,9 @@ import pkg_resources from bookkeeper import types +from bookkeeper.admin.namespace import Namespace from bookkeeper.common.service_uri import ServiceURI +from bookkeeper.kv.table import Table from bookkeeper.proto.storage_pb2_grpc import RootRangeServiceStub __version__ = pkg_resources.get_distribution('bookkeeper').version @@ -36,25 +38,38 @@ class Client(object): Args: storage_client_settings (~bookkeeper.types.StorageClientSettings): The settings for bookkeeper storage client . + namespace (str): namespace name kwargs (dict): Any additional arguments provided are sent as keyword arguments to the underlying grpc client. """ - def __init__(self, storage_client_settings=(), **kwargs): + def __init__(self, + storage_client_settings=(), + namespace="public", + **kwargs): # init the storage client settings self.storage_client_settings =\ types.StorageClientSettings(*storage_client_settings) - __logger__.info("Creating an admin client to cluster '%s'", + __logger__.info("Creating a storage client to cluster '%s'", self.storage_client_settings.service_uri) service_uri = ServiceURI(self.storage_client_settings.service_uri) assert service_uri.service_name.lower() == 'bk' # create channel - self.channel = grpc.insecure_channel( + self.__channel__ = grpc.insecure_channel( target=service_uri.service_location ) - __logger__.info("Successfully created an admin client to cluster '%s'", - self.storage_client_settings.service_uri) + __logger__.info( + "Successfully created a storage client to cluster '%s'", + self.storage_client_settings.service_uri) # create the rpc stub - self.root_range = RootRangeServiceStub(channel=self.channel) + self.__root_range__ = RootRangeServiceStub(channel=self.__channel__) + + # assign the namespace + self.__namespace__ = namespace + + def table(self, table_name): + ns = Namespace(self.__root_range__, self.__namespace__) + stream_props = ns.get(table_name) + return Table(self.__channel__, stream_props) diff --git a/stream/clients/python/bookkeeper/kv/table.py b/stream/clients/python/bookkeeper/kv/table.py new file mode 100644 index 00000000000..81ac6a8ccf0 --- /dev/null +++ b/stream/clients/python/bookkeeper/kv/table.py @@ -0,0 +1,124 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +from __future__ import absolute_import + +import logging + +from bookkeeper.common.exceptions import from_table_rpc_response +from bookkeeper.common.method import wrap_method +from bookkeeper.common.retry import Retry +from bookkeeper.common.timeout import ExponentialTimeout +from bookkeeper.proto import kv_rpc_pb2 +from bookkeeper.proto.kv_rpc_pb2_grpc import TableServiceStub + +__logger__ = logging.getLogger("bookkeeper.kv.Table") + + +class Table(object): + + def __init__(self, channel, stream_props): + self.__table_service__ = TableServiceStub(channel=channel) + self.__stream_props__ = stream_props + self.__default_retry__ = Retry(deadline=60) + self.__default_timeout__ = ExponentialTimeout() + self.__put_with_retries__ =\ + wrap_method(self.__do_put__, self.__default_retry__) + self.__get_with_retries__ =\ + wrap_method(self.__do_get__, self.__default_retry__) + self.__del_with_retries__ =\ + wrap_method(self.__do_del__, self.__default_retry__) + __logger__.info("initialized table instance with properties : %s", + stream_props) + + def __make_routing_metadata__(self, key): + return [ + ('bk-rt-sid-bin', + self.__stream_props__.stream_id.to_bytes(8, "big")), + ('bk-rt-key-bin', + key) + ] + + def __make_routing_header__(self, key): + return kv_rpc_pb2.RoutingHeader( + stream_id=self.__stream_props__.stream_id, + r_key=key + ) + + def put_str(self, key_str, val_str): + key = key_str.encode('utf-8') + value = val_str.encode('utf-8') + return self.put(key, value) + + def put(self, key, value): + metadata = self.__make_routing_metadata__(key) + header = self.__make_routing_header__(key) + return self.__put_with_retries__(key, value, header, metadata) + + def __do_put__(self, key, value, routing_header, grpc_metadata): + put_req = kv_rpc_pb2.PutRequest( + key=key, + value=value, + header=routing_header + ) + put_resp = self.__table_service__.Put( + request=put_req, + metadata=grpc_metadata + ) + from_table_rpc_response(put_resp) + + def get_str(self, key_str): + key = key_str.encode('utf-8') + return self.get(key) + + def get(self, key): + metadata = self.__make_routing_metadata__(key) + header = self.__make_routing_header__(key) + return self.__get_with_retries__(key, header, metadata) + + def __do_get__(self, key, routing_header, grpc_metadata): + get_req = kv_rpc_pb2.RangeRequest( + key=key, + header=routing_header + ) + get_resp = self.__table_service__.Range( + request=get_req, + metadata=grpc_metadata + ) + get_resp = from_table_rpc_response(get_resp) + if get_resp.count == 0: + return None + else: + return get_resp.kvs[0] + + def delete_str(self, key_str): + key = key_str.encode('utf-8') + return self.delete(key) + + def delete(self, key): + metadata = self.__make_routing_metadata__(key) + header = self.__make_routing_header__(key) + return self.__del_with_retries__(key, header, metadata) + + def __do_del__(self, key, routing_header, grpc_metadata): + del_req = kv_rpc_pb2.DeleteRangeRequest( + key=key, + header=routing_header + ) + del_resp = self.__table_service__.Delete( + request=del_req, + metadata=grpc_metadata + ) + del_resp = from_table_rpc_response(del_resp) + return del_resp.deleted diff --git a/stream/clients/python/bookkeeper/proto/__init__.py b/stream/clients/python/bookkeeper/proto/__init__.py new file mode 100644 index 00000000000..89c7831391e --- /dev/null +++ b/stream/clients/python/bookkeeper/proto/__init__.py @@ -0,0 +1,13 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. diff --git a/stream/clients/python/examples/admin.py b/stream/clients/python/examples/admin.py index d39d50cf5c7..d023ce1af88 100644 --- a/stream/clients/python/examples/admin.py +++ b/stream/clients/python/examples/admin.py @@ -13,6 +13,9 @@ # limitations under the License. from bookkeeper import admin +from bookkeeper import kv +from bookkeeper.common.exceptions import KeyNotFoundError +from bookkeeper.common.exceptions import NamespaceNotFoundError ns_name = "test" ns_name_2 = "test2" @@ -21,6 +24,16 @@ client = admin.Client() +try: + client.namespaces().delete(ns_name) +except NamespaceNotFoundError: + print("Namespace '%s' doesn't not exist" % ns_name) + +try: + client.namespaces().delete(ns_name_2) +except NamespaceNotFoundError: + print("Namespace '%s' doesn't not exist" % ns_name_2) + # create first namespace ns_resp = client.namespaces().create(ns_name) print("Created first namespace '%s' : %s" % (ns_name, ns_resp)) @@ -47,6 +60,36 @@ stream_props = ns.get(stream_name_2) print("Get second stream '%s' : %s" % (stream_name_2, stream_props)) +kv_client = kv.Client(namespace=ns_name) +table = kv_client.table(stream_name) +num_keys = 10 + +for i in range(num_keys): + put_resp = table.put_str("key-%s" % i, "value-%s" % i) +print("Successfully added %d keys" % num_keys) + +for i in range(10): + get_resp = table.get_str("key-%s" % i) + print("Get response : %s" % get_resp) +print("Successfully retrieved %d keys" % num_keys) + +for i in range(10): + del_resp = table.delete_str("key-%s" % i) + print("Delete response : %s" % del_resp) +print("Successfully deleted %d keys" % num_keys) + +print("Try to retrieve %d keys again" % num_keys) +for i in range(10): + get_resp = table.get_str("key-%s" % i) + assert get_resp is None +print("All %d keys should not exist anymore" % num_keys) + +for i in range(10): + try: + table.delete_str("key-%s" % i) + except KeyNotFoundError: + print("Key 'key-%s' doesn't exist" % i) + del_resp = ns.delete(stream_name) print("Delete first stream '%s' : %s" % (stream_name, del_resp)) del_resp = ns.delete(stream_name_2) diff --git a/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/ProtocolConstants.java b/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/ProtocolConstants.java index b888b45353b..e8c14efed5a 100644 --- a/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/ProtocolConstants.java +++ b/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/ProtocolConstants.java @@ -111,12 +111,12 @@ private ProtocolConstants() { .build(); // storage container request metadata key - public static final String SC_ID_KEY = "sc-id" + Metadata.BINARY_HEADER_SUFFIX; + public static final String SC_ID_KEY = "bk-rt-sc-id" + Metadata.BINARY_HEADER_SUFFIX; // request metadata key for routing requests - public static final String ROUTING_KEY = "rk" + Metadata.BINARY_HEADER_SUFFIX; - public static final String STREAM_ID_KEY = "sid-" + Metadata.BINARY_HEADER_SUFFIX; - public static final String RANGE_ID_KEY = "rid-" + Metadata.BINARY_HEADER_SUFFIX; + public static final String ROUTING_KEY = "bk-rt-key" + Metadata.BINARY_HEADER_SUFFIX; + public static final String STREAM_ID_KEY = "bk-rt-sid" + Metadata.BINARY_HEADER_SUFFIX; + public static final String RANGE_ID_KEY = "bk-rt-rid" + Metadata.BINARY_HEADER_SUFFIX; // the metadata keys in grpc call metadata public static final Metadata.Key SCID_METADATA_KEY = Metadata.Key.of( From 7c83a12e6552bded8e28cf0a57f720001a64846c Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Tue, 2 Oct 2018 20:52:18 +0200 Subject: [PATCH 0067/1642] Decorate OrderedExecutor threads rather than runnables Actually, decorate the runnables too, but apply the decoration in a decorator on the thread so that if a runnable/callable is passed to the thread, the decoration will be applied. This allows users of OrderedExecutor/Scheduler to use executor.chooseThread() for CompletableFuture async handlers and still get things like the MDC context. Author: Reviewers: Andrey Yegorov , Ivan Kelly , Enrico Olivelli , Sijie Guo This closes #1729 from ivankelly/decorate-exec --- .../common/util/OrderedExecutor.java | 68 +++++++- .../common/util/OrderedScheduler.java | 94 +++++++++++ .../util/TestOrderedExecutorDecorators.java | 157 ++++++++++++++++++ 3 files changed, 313 insertions(+), 6 deletions(-) create mode 100644 bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestOrderedExecutorDecorators.java diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java index ab62a687e10..220e1af9c02 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java @@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.util.concurrent.ForwardingExecutorService; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -283,6 +284,61 @@ protected ExecutorService getBoundedExecutor(ThreadPoolExecutor executor) { return new BoundedExecutorService(executor, this.maxTasksInQueue); } + protected ExecutorService addExecutorDecorators(ExecutorService executor) { + return new ForwardingExecutorService() { + @Override + protected ExecutorService delegate() { + return executor; + } + + @Override + public List> invokeAll(Collection> tasks) + throws InterruptedException { + return super.invokeAll(timedCallables(tasks)); + } + + @Override + public List> invokeAll(Collection> tasks, + long timeout, TimeUnit unit) + throws InterruptedException { + return super.invokeAll(timedCallables(tasks), timeout, unit); + } + + @Override + public T invokeAny(Collection> tasks) + throws InterruptedException, ExecutionException { + return super.invokeAny(timedCallables(tasks)); + } + + @Override + public T invokeAny(Collection> tasks, + long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { + return super.invokeAny(timedCallables(tasks), timeout, unit); + } + + @Override + public void execute(Runnable command) { + super.execute(timedRunnable(command)); + } + + @Override + public Future submit(Callable task) { + return super.submit(timedCallable(task)); + } + + @Override + public Future submit(Runnable task) { + return super.submit(timedRunnable(task)); + } + + @Override + public Future submit(Runnable task, T result) { + return super.submit(timedRunnable(task), result); + } + }; + } + /** * Constructs Safe executor. * @@ -318,7 +374,7 @@ protected OrderedExecutor(String baseName, int numThreads, ThreadFactory threadF ThreadPoolExecutor thread = createSingleThreadExecutor( new ThreadFactoryBuilder().setNameFormat(name + "-" + getClass().getSimpleName() + "-" + i + "-%d") .setThreadFactory(threadFactory).build()); - threads[i] = getBoundedExecutor(thread); + threads[i] = addExecutorDecorators(getBoundedExecutor(thread)); final int idx = i; try { @@ -391,7 +447,7 @@ public boolean preserveMdc() { * @param r */ public void executeOrdered(Object orderingKey, SafeRunnable r) { - chooseThread(orderingKey).execute(timedRunnable(r)); + chooseThread(orderingKey).execute(r); } /** @@ -400,7 +456,7 @@ public void executeOrdered(Object orderingKey, SafeRunnable r) { * @param r */ public void executeOrdered(long orderingKey, SafeRunnable r) { - chooseThread(orderingKey).execute(timedRunnable(r)); + chooseThread(orderingKey).execute(r); } /** @@ -409,7 +465,7 @@ public void executeOrdered(long orderingKey, SafeRunnable r) { * @param r */ public void executeOrdered(int orderingKey, SafeRunnable r) { - chooseThread(orderingKey).execute(timedRunnable(r)); + chooseThread(orderingKey).execute(r); } public ListenableFuture submitOrdered(long orderingKey, Callable task) { @@ -504,7 +560,7 @@ public Future submit(Callable task) { */ @Override public Future submit(Runnable task, T result) { - return chooseThread().submit(timedRunnable(task), result); + return chooseThread().submit(task, result); } /** @@ -512,7 +568,7 @@ public Future submit(Runnable task, T result) { */ @Override public Future submit(Runnable task) { - return chooseThread().submit(timedRunnable(task)); + return chooseThread().submit(task); } /** diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java index 6f366fdbc87..3c1fef5f797 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java @@ -17,18 +17,27 @@ */ package org.apache.bookkeeper.common.util; +import com.google.common.util.concurrent.ForwardingListeningExecutorService; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListenableScheduledFuture; +import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.ListeningScheduledExecutorService; import io.netty.util.concurrent.DefaultThreadFactory; +import java.util.Collection; +import java.util.List; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import org.apache.bookkeeper.stats.StatsLogger; @@ -119,6 +128,11 @@ protected ListeningScheduledExecutorService getBoundedExecutor(ThreadPoolExecuto return new BoundedScheduledExecutorService((ScheduledThreadPoolExecutor) executor, this.maxTasksInQueue); } + @Override + protected ListeningScheduledExecutorService addExecutorDecorators(ExecutorService executor) { + return new OrderedSchedulerDecoratedThread((ListeningScheduledExecutorService) executor); + } + @Override public ListeningScheduledExecutorService chooseThread() { return (ListeningScheduledExecutorService) super.chooseThread(); @@ -286,4 +300,84 @@ public ScheduledFuture scheduleWithFixedDelay(Runnable command, return chooseThread().scheduleWithFixedDelay(timedRunnable(command), initialDelay, delay, unit); } + class OrderedSchedulerDecoratedThread extends ForwardingListeningExecutorService + implements ListeningScheduledExecutorService { + private final ListeningScheduledExecutorService delegate; + + private OrderedSchedulerDecoratedThread(ListeningScheduledExecutorService delegate) { + this.delegate = delegate; + } + + @Override + protected ListeningExecutorService delegate() { + return delegate; + } + + @Override + public ListenableScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { + return delegate.schedule(timedRunnable(command), delay, unit); + } + + @Override + public ListenableScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { + return delegate.schedule(timedCallable(callable), delay, unit); + } + + @Override + public ListenableScheduledFuture scheduleAtFixedRate(Runnable command, + long initialDelay, long period, TimeUnit unit) { + return delegate.scheduleAtFixedRate(timedRunnable(command), initialDelay, period, unit); + } + + @Override + public ListenableScheduledFuture scheduleWithFixedDelay(Runnable command, + long initialDelay, long delay, TimeUnit unit) { + return delegate.scheduleAtFixedRate(timedRunnable(command), initialDelay, delay, unit); + } + + @Override + public ListenableFuture submit(Callable task) { + return super.submit(timedCallable(task)); + } + + @Override + public ListenableFuture submit(Runnable task) { + return super.submit(timedRunnable(task)); + } + + @Override + public List> invokeAll(Collection> tasks) throws InterruptedException { + return super.invokeAll(timedCallables(tasks)); + } + + @Override + public List> invokeAll(Collection> tasks, + long timeout, TimeUnit unit) throws InterruptedException { + return super.invokeAll(timedCallables(tasks), timeout, unit); + } + + @Override + public T invokeAny(Collection> tasks) + throws InterruptedException, ExecutionException { + return super.invokeAny(timedCallables(tasks)); + } + + @Override + public T invokeAny(Collection> tasks, long timeout, + TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + + return super.invokeAny(timedCallables(tasks), timeout, unit); + } + + @Override + public ListenableFuture submit(Runnable task, T result) { + return super.submit(timedRunnable(task), result); + } + + @Override + public void execute(Runnable command) { + super.execute(timedRunnable(command)); + } + }; + } diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestOrderedExecutorDecorators.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestOrderedExecutorDecorators.java new file mode 100644 index 00000000000..f03b24df950 --- /dev/null +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestOrderedExecutorDecorators.java @@ -0,0 +1,157 @@ +/* + * + * 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.common.util; + +import static org.apache.bookkeeper.common.util.SafeRunnable.safeRun; +import static org.hamcrest.Matchers.hasItem; +import static org.junit.Assert.assertThat; +import static org.mockito.AdditionalAnswers.answerVoid; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; + +import org.apache.log4j.Appender; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.apache.log4j.MDC; +import org.apache.log4j.spi.LoggingEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test that decorators applied by OrderedExecutor/Scheduler are correctly applied. + */ +public class TestOrderedExecutorDecorators { + private static final Logger log = LoggerFactory.getLogger(TestOrderedExecutorDecorators.class); + private static final String MDC_KEY = "mdc-key"; + + private Appender mockAppender; + private final Queue capturedEvents = new ConcurrentLinkedQueue<>(); + + public static String mdcFormat(Object mdc, String message) { + return String.format("[%s:%s] %s", MDC_KEY, mdc, message); + } + + @Before + public void setUp() throws Exception { + MDC.clear(); + mockAppender = mock(Appender.class); + when(mockAppender.getName()).thenReturn("MockAppender"); + + LogManager.getRootLogger().addAppender(mockAppender); + LogManager.getRootLogger().setLevel(Level.INFO); + + doAnswer(answerVoid((LoggingEvent event) -> { + capturedEvents.add(mdcFormat(event.getMDC(MDC_KEY), + event.getRenderedMessage())); + })).when(mockAppender).doAppend(any()); + } + + @After + public void tearDown() throws Exception { + LogManager.getRootLogger().removeAppender(mockAppender); + capturedEvents.clear(); + MDC.clear(); + } + + @Test + public void testMDCInvokeOrdered() throws Exception { + OrderedExecutor executor = OrderedExecutor.newBuilder() + .name("test").numThreads(20).preserveMdcForTaskExecution(true).build(); + + try { + MDC.put(MDC_KEY, "testMDCInvokeOrdered"); + executor.submitOrdered(10, () -> { + log.info("foobar"); + return 10; + }).get(); + assertThat(capturedEvents, + hasItem(mdcFormat("testMDCInvokeOrdered", "foobar"))); + } finally { + executor.shutdown(); + } + } + + @Test + public void testMDCInvokeDirectOnChosen() throws Exception { + OrderedExecutor executor = OrderedExecutor.newBuilder() + .name("test").numThreads(20).preserveMdcForTaskExecution(true).build(); + + try { + MDC.put(MDC_KEY, "testMDCInvokeOrdered"); + executor.chooseThread(10).submit(() -> { + log.info("foobar"); + return 10; + }).get(); + assertThat(capturedEvents, + hasItem(mdcFormat("testMDCInvokeOrdered", "foobar"))); + } finally { + executor.shutdown(); + } + + } + + + @Test + public void testMDCScheduleOrdered() throws Exception { + OrderedScheduler scheduler = OrderedScheduler.newSchedulerBuilder() + .name("test").numThreads(20).preserveMdcForTaskExecution(true).build(); + + try { + MDC.put(MDC_KEY, "testMDCInvokeOrdered"); + scheduler.scheduleOrdered(10, safeRun(() -> { + log.info("foobar"); + }), 0, TimeUnit.DAYS).get(); + assertThat(capturedEvents, + hasItem(mdcFormat("testMDCInvokeOrdered", "foobar"))); + } finally { + scheduler.shutdown(); + } + } + + @Test + public void testMDCScheduleDirectOnChosen() throws Exception { + OrderedScheduler scheduler = OrderedScheduler.newSchedulerBuilder() + .name("test").numThreads(20).preserveMdcForTaskExecution(true).build(); + + try { + MDC.put(MDC_KEY, "testMDCInvokeOrdered"); + scheduler.chooseThread(10).schedule(safeRun(() -> { + log.info("foobar"); + }), 0, TimeUnit.DAYS).get(); + assertThat(capturedEvents, + hasItem(mdcFormat("testMDCInvokeOrdered", "foobar"))); + } finally { + scheduler.shutdown(); + } + } + +} From 9f013b43138ec2e32039f1108ff2379dc6dc05fd Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 2 Oct 2018 15:00:19 -0700 Subject: [PATCH 0068/1642] [TABLE SERVICES] [PERF] Provides a simple perf tool for testing put and inc performance Descriptions of the changes in this PR: *Motivation* Need to know what is the performance of table service. *Changes* Provide a simple perf tool following what RocksDB is doing. Currently support simple testing for put and inc operations. Author: Sijie Guo Reviewers: Jia Zhai , Enrico Olivelli This closes #1727 from sijie/async_benchmark --- .../service/DLNamespaceProviderService.java | 2 +- .../statelib/impl/kv/RocksdbKVStore.java | 2 +- tools/perf/pom.xml | 5 + .../apache/bookkeeper/tools/perf/BKPerf.java | 3 +- .../tools/perf/TablePerfCommandGroup.java | 46 +++ .../tools/perf/table/BenchmarkCommand.java | 60 +++ .../tools/perf/table/BenchmarkTask.java | 77 ++++ .../tools/perf/table/IncrementRandomTask.java | 50 +++ .../perf/table/IncrementSequentialTask.java | 50 +++ .../tools/perf/table/IncrementTask.java | 105 +++++ .../tools/perf/table/KeyGenerator.java | 62 +++ .../tools/perf/table/PerfClient.java | 374 ++++++++++++++++++ .../tools/perf/table/WriteRandomTask.java | 50 +++ .../tools/perf/table/WriteSequentialTask.java | 50 +++ .../tools/perf/table/WriteTask.java | 112 ++++++ .../tools/perf/table/package-info.java | 23 ++ 16 files changed, 1068 insertions(+), 3 deletions(-) create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/TablePerfCommandGroup.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/BenchmarkCommand.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/BenchmarkTask.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementRandomTask.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementSequentialTask.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementTask.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/KeyGenerator.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/PerfClient.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteRandomTask.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteSequentialTask.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteTask.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/package-info.java diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/DLNamespaceProviderService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/DLNamespaceProviderService.java index 4550df55aa9..0a87add7888 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/DLNamespaceProviderService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/DLNamespaceProviderService.java @@ -91,7 +91,7 @@ public DLNamespaceProviderService(ServerConfiguration bkServerConf, this.dlConf.setWriteLockEnabled(false); // setting the flush policy this.dlConf.setImmediateFlushEnabled(false); - this.dlConf.setOutputBufferSize(0); + this.dlConf.setOutputBufferSize(512 * 1024); this.dlConf.setPeriodicFlushFrequencyMilliSeconds(2); // flush every 1 ms // explicit truncation is required this.dlConf.setExplicitTruncationByApplication(true); diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java index 4e72e1f5cb4..46ba7f1c13e 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java @@ -275,7 +275,7 @@ protected void openRocksdb(StateStoreSpec spec) throws StateStoreException { // initialize the write options writeOpts = new WriteOptions(); - writeOpts.setDisableWAL(false); // disable wal, since the source of truth will be on distributedlog + writeOpts.setDisableWAL(true); // disable wal, since the source of truth will be on distributedlog // initialize the flush options diff --git a/tools/perf/pom.xml b/tools/perf/pom.xml index d3a617f60b1..3142063a2b6 100644 --- a/tools/perf/pom.xml +++ b/tools/perf/pom.xml @@ -34,6 +34,11 @@ distributedlog-core ${project.version} + + org.apache.bookkeeper + stream-storage-java-client + ${project.version} + org.hdrhistogram HdrHistogram diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java index 23021ee930a..3aac74eef8b 100644 --- a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java @@ -35,7 +35,8 @@ public static void main(String[] args) { .withDescription(NAME + " evaluates the performance of Apache BookKeeper clusters") .withFlags(new BKFlags()) .withConsole(System.out) - .addCommand(new DlogPerfCommandGroup()); + .addCommand(new DlogPerfCommandGroup()) + .addCommand(new TablePerfCommandGroup()); CliSpec spec = specBuilder.build(); diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/TablePerfCommandGroup.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/TablePerfCommandGroup.java new file mode 100644 index 00000000000..5ab16b8af87 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/TablePerfCommandGroup.java @@ -0,0 +1,46 @@ +/* + * 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.tools.perf; + +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliCommandGroup; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.tools.perf.table.BenchmarkCommand; + +/** + * Commands that evaluate performance of table service. + */ +public class TablePerfCommandGroup extends CliCommandGroup implements PerfCommandGroup { + + private static final String NAME = "table"; + private static final String DESC = "Commands on evaluating performance of table service"; + + private static final CliSpec spec = CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withParent(BKPerf.NAME) + .addCommand(new BenchmarkCommand()) + .build(); + + public TablePerfCommandGroup() { + super(spec); + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/BenchmarkCommand.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/BenchmarkCommand.java new file mode 100644 index 00000000000..19322f5c584 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/BenchmarkCommand.java @@ -0,0 +1,60 @@ +/* + * 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.tools.perf.table; + +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.tools.common.BKCommand; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.tools.perf.table.PerfClient.Flags; +import org.apache.commons.configuration.CompositeConfiguration; + +/** + * Command to benchmark table service. + */ +@Slf4j +public class BenchmarkCommand extends BKCommand { + + private static final String NAME = "benchmark"; + private static final String DESC = "benchmark table service"; + + public BenchmarkCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new Flags()) + .build()); + } + + @Override + protected boolean apply(ServiceURI serviceURI, + CompositeConfiguration conf, + BKFlags globalFlags, + Flags cmdFlags) { + if (serviceURI == null) { + log.warn("No service uri is provided. Use default 'bk://localhost:4181'."); + serviceURI = ServiceURI.create("bk://localhost:4181"); + } + PerfClient client = new PerfClient(serviceURI, cmdFlags); + client.run(); + return true; + } +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/BenchmarkTask.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/BenchmarkTask.java new file mode 100644 index 00000000000..ed2321abe13 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/BenchmarkTask.java @@ -0,0 +1,77 @@ +/* + * 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.tools.perf.table; + +import io.netty.buffer.ByteBuf; +import java.util.Random; +import java.util.concurrent.Callable; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.tools.perf.table.PerfClient.Flags; + +/** + * Abstract benchmark task. + */ +abstract class BenchmarkTask implements Callable { + + protected final Table table; + protected final int tid; + protected final Random random; + protected final long numRecords; + protected final long keyRange; + protected final Flags flags; + protected final KeyGenerator generator; + + BenchmarkTask(Table table, + int tid, + long randSeed, + long numRecords, + long keyRange, + Flags flags, + KeyGenerator generator) { + this.table = table; + this.tid = tid; + this.random = new Random(randSeed + tid * 1000); + this.numRecords = numRecords; + this.keyRange = keyRange; + this.flags = flags; + this.generator = generator; + } + + @Override + public Void call() throws Exception { + runTask(); + return null; + } + + protected abstract void runTask() throws Exception; + + protected void getFixedKey(ByteBuf key, long sn) { + generator.generateKeyFromLong(key, sn); + } + + protected void getRandomKey(ByteBuf key, long range) { + generator.generateKeyFromLong(key, Math.abs(random.nextLong() % range)); + } + + protected abstract void reportStats(long oldTime); + + protected abstract void printAggregatedStats(); + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementRandomTask.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementRandomTask.java new file mode 100644 index 00000000000..65ffb35bd61 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementRandomTask.java @@ -0,0 +1,50 @@ +/* + * 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.tools.perf.table; + +import com.google.common.util.concurrent.RateLimiter; +import io.netty.buffer.ByteBuf; +import java.util.concurrent.Semaphore; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.tools.perf.table.PerfClient.Flags; + +/** + * Increment the amount of keys in sequence. + */ +class IncrementRandomTask extends IncrementTask { + + IncrementRandomTask(Table table, + int tid, + long randSeed, + long numRecords, + long keyRange, + Flags flags, + KeyGenerator generator, + RateLimiter limiter, + Semaphore semaphore) { + super(table, tid, randSeed, numRecords, keyRange, flags, generator, limiter, semaphore); + } + + + @Override + protected void getKey(ByteBuf key, long id, long range) { + getRandomKey(key, range); + } +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementSequentialTask.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementSequentialTask.java new file mode 100644 index 00000000000..82610f170d5 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementSequentialTask.java @@ -0,0 +1,50 @@ +/* + * 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.tools.perf.table; + +import com.google.common.util.concurrent.RateLimiter; +import io.netty.buffer.ByteBuf; +import java.util.concurrent.Semaphore; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.tools.perf.table.PerfClient.Flags; + +/** + * Increment the amount of keys in sequence. + */ +class IncrementSequentialTask extends IncrementTask { + + IncrementSequentialTask(Table table, + int tid, + long randSeed, + long numRecords, + long keyRange, + Flags flags, + KeyGenerator generator, + RateLimiter limiter, + Semaphore semaphore) { + super(table, tid, randSeed, numRecords, keyRange, flags, generator, limiter, semaphore); + } + + + @Override + protected void getKey(ByteBuf key, long id, long range) { + getFixedKey(key, id); + } +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementTask.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementTask.java new file mode 100644 index 00000000000..3f047d9c265 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/IncrementTask.java @@ -0,0 +1,105 @@ +/* + * 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.tools.perf.table; + +import com.google.common.util.concurrent.RateLimiter; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.tools.perf.table.PerfClient.Flags; +import org.apache.bookkeeper.tools.perf.table.PerfClient.OP; +import org.apache.bookkeeper.tools.perf.table.PerfClient.OpStats; + +/** + * Write task to inject key/value pairs to the table. + */ +@Slf4j +abstract class IncrementTask extends BenchmarkTask { + + protected final RateLimiter limiter; + protected final Semaphore semaphore; + protected final OpStats writeOpStats; + + IncrementTask(Table table, + int tid, + long randSeed, + long numRecords, + long keyRange, + Flags flags, + KeyGenerator generator, + RateLimiter limiter, + Semaphore semaphore) { + super(table, tid, randSeed, numRecords, keyRange, flags, generator); + this.limiter = limiter; + this.semaphore = semaphore; + this.writeOpStats = new OpStats(OP.INC.name()); + } + + @Override + protected void runTask() throws Exception { + for (long i = 0L; i < numRecords; ++i) { + if (null != semaphore) { + semaphore.acquire(); + } + if (null != limiter) { + limiter.acquire(); + } + incKey(i); + } + } + + protected abstract void getKey(ByteBuf key, long id, long range); + + void incKey(long i) { + ByteBuf keyBuf = PooledByteBufAllocator.DEFAULT.heapBuffer(flags.keySize); + getKey(keyBuf, i, keyRange); + keyBuf.writerIndex(keyBuf.readerIndex() + keyBuf.writableBytes()); + + final long startTime = System.nanoTime(); + table.increment(keyBuf, 100) + .whenComplete((result, cause) -> { + if (null != semaphore) { + semaphore.release(); + } + if (null != cause) { + log.error("Error at increment key/amount", cause); + } else { + long latencyMicros = TimeUnit.NANOSECONDS.toMicros( + System.nanoTime() - startTime + ); + writeOpStats.recordOp(latencyMicros); + } + keyBuf.release(); + }); + } + + @Override + protected void reportStats(long oldTime) { + writeOpStats.reportStats(oldTime); + } + + @Override + protected void printAggregatedStats() { + writeOpStats.printAggregatedStats(); + } +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/KeyGenerator.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/KeyGenerator.java new file mode 100644 index 00000000000..458aa1c1d7c --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/KeyGenerator.java @@ -0,0 +1,62 @@ +/* + * 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.tools.perf.table; + +import io.netty.buffer.ByteBuf; + +/** + * Util class used for perf benchmarks. + */ +final class KeyGenerator { + + private final long numKeys; + private final long keysPerPrefix; + private final int prefixSize; + + KeyGenerator(long numKeys, + long keysPerPrefix, + int prefixSize) { + this.numKeys = numKeys; + this.keysPerPrefix = keysPerPrefix; + this.prefixSize = prefixSize; + } + + public void generateKeyFromLong(ByteBuf slice, long n) { + int startPos = 0; + if (keysPerPrefix > 0) { + long numPrefix = (numKeys + keysPerPrefix - 1) / keysPerPrefix; + long prefix = n % numPrefix; + int bytesToFill = Math.min(prefixSize, 8); + for (int i = 0; i < bytesToFill; i++) { + slice.setByte(i, (byte) (prefix % 256)); + prefix /= 256; + } + for (int i = 8; i < bytesToFill; ++i) { + slice.setByte(i, '0'); + } + startPos = bytesToFill; + } + for (int i = slice.writableBytes() - 1; i >= startPos; --i) { + slice.setByte(i, (byte) ('0' + (n % 10))); + n /= 10; + } + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/PerfClient.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/PerfClient.java new file mode 100644 index 00000000000..4b3442700cf --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/PerfClient.java @@ -0,0 +1,374 @@ +/* + * 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.tools.perf.table; + +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; + +import com.beust.jcommander.Parameter; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.common.util.concurrent.RateLimiter; +import io.netty.buffer.ByteBuf; +import java.text.DecimalFormat; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.LongAdder; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.HdrHistogram.Histogram; +import org.HdrHistogram.Recorder; +import org.apache.bookkeeper.api.StorageClient; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.clients.StorageClientBuilder; +import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.perf.utils.PaddingDecimalFormat; + +/** + * Perf client to evaluate the performance of table service. + */ +@Slf4j +public class PerfClient implements Runnable { + + enum OP { + PUT, + GET, + INC, + DEL + } + + /** + * Flags for the perf client. + */ + public static class Flags extends CliFlags { + + @Parameter( + names = { + "-r", "--rate" + }, + description = "Request rate - requests/second") + public int rate = 100000; + + @Parameter( + names = { + "-mor", "--max-outstanding-requests" + }, + description = "Max outstanding request") + public int maxOutstandingRequests = 10000; + + @Parameter( + names = { + "-ks", "--key-size" + }, + description = "Key size") + public int keySize = 16; + + @Parameter( + names = { + "-vs", "--value-size" + }, + description = "Value size") + public int valueSize = 100; + + @Parameter( + names = { + "-t", "--table-name" + }, + description = "Table name") + public String tableName = "test-table"; + + @Parameter( + names = { + "-nk", "--num-keys" + }, + description = "Number of the keys to test") + public int numKeys = 1000000; + + @Parameter( + names = { + "-kpp", "--keys-per-prefix" + }, + description = "control average number of keys generated per prefix," + + " 0 means no special handling of the prefix, i.e. use the" + + " prefix comes with the generated random number" + ) + public int keysPerPrefix = 0; + + @Parameter( + names = { + "-ps", "--prefix-size" + }, + description = "Prefix size" + ) + public int prefixSize = 0; + + @Parameter( + names = { + "-no", "--num-ops" + }, + description = "Number of client operations to test") + public int numOps = 0; + + @Parameter( + names = { + "-ns", "--namespace" + }, + description = "Namespace of the tables to benchmark") + public String namespace = "benchmark"; + + @Parameter( + names = { + "-b", "--benchmarks" + }, + description = "List of benchamrks to run") + public List benchmarks; + + } + + static class OpStats { + + private final String name; + private final LongAdder ops = new LongAdder(); + private final Recorder recorder = new Recorder( + TimeUnit.SECONDS.toMillis(120000), 5 + ); + private final Recorder cumulativeRecorder = new Recorder( + TimeUnit.SECONDS.toMillis(120000), 5 + ); + private Histogram reportHistogram; + + OpStats(String name) { + this.name = name; + } + + void recordOp(long latencyMicros) { + ops.increment(); + recorder.recordValue(latencyMicros); + cumulativeRecorder.recordValue(latencyMicros); + } + + void reportStats(long oldTime) { + long now = System.nanoTime(); + double elapsed = (now - oldTime) / 1e9; + double rate = ops.sumThenReset() / elapsed; + reportHistogram = recorder.getIntervalHistogram(reportHistogram); + log.info( + "[{}] Throughput: {} ops/s --- Latency: mean:" + + " {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}", + name, + throughputFormat.format(rate), + dec.format(reportHistogram.getMean() / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(50) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(95) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.9) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.99) / 1000.0), + dec.format(reportHistogram.getMaxValue() / 1000.0)); + reportHistogram.reset(); + } + + void printAggregatedStats() { + Histogram reportHistogram = cumulativeRecorder.getIntervalHistogram(); + log.info("[{}] latency stats --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {}" + + " - 99.9pct: {} - 99.99pct: {} - 99.999pct: {} - Max: {}", + name, + dec.format(reportHistogram.getMean() / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(50) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(95) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.9) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.999) / 1000.0), + dec.format(reportHistogram.getMaxValue() / 1000.0)); + } + } + + private final ServiceURI serviceURI; + private final Flags flags; + + PerfClient(ServiceURI serviceURI, Flags flags) { + this.serviceURI = serviceURI; + this.flags = flags; + } + + @Override + public void run() { + try { + execute(); + } catch (Exception e) { + log.error("Encountered exception at running table perf client", e); + } + } + + void execute() throws Exception { + ObjectMapper m = new ObjectMapper(); + ObjectWriter w = m.writerWithDefaultPrettyPrinter(); + log.info("Starting table perf client with config : {}", w.writeValueAsString(flags)); + + runBenchmarkTasks(); + } + + private void runBenchmarkTasks() throws Exception { + StorageClientSettings settings = StorageClientSettings.newBuilder() + .serviceUri(serviceURI.getUri().toString()) + .build(); + try (StorageClient client = StorageClientBuilder.newBuilder() + .withSettings(settings) + .withNamespace(flags.namespace) + .build()) { + try (Table table = result(client.openTable(flags.tableName))) { + + long randSeed = System.currentTimeMillis(); + KeyGenerator generator = new KeyGenerator(flags.numKeys, flags.keysPerPrefix, flags.prefixSize); + RateLimiter limiter; + if (flags.rate <= 0) { + limiter = null; + } else { + limiter = RateLimiter.create(flags.rate); + } + + for (String benchmark : flags.benchmarks) { + List tasks = new ArrayList<>(); + int currentTaskId = 0; + Semaphore semaphore; + if (flags.maxOutstandingRequests <= 0) { + semaphore = null; + } else { + semaphore = new Semaphore(flags.maxOutstandingRequests); + } + + switch (benchmark) { + case "fillseq": + tasks.add(new WriteSequentialTask( + table, + currentTaskId++, + randSeed, + Math.max(flags.numOps, flags.numKeys), + flags.numKeys, + flags, + generator, + limiter, + semaphore + )); + break; + case "fillrandom": + tasks.add(new WriteRandomTask( + table, + currentTaskId++, + randSeed, + Math.max(flags.numOps, flags.numKeys), + flags.numKeys, + flags, + generator, + limiter, + semaphore + )); + break; + case "incseq": + tasks.add(new IncrementSequentialTask( + table, + currentTaskId++, + randSeed, + Math.max(flags.numOps, flags.numKeys), + flags.numKeys, + flags, + generator, + limiter, + semaphore + )); + break; + case "incrandom": + tasks.add(new IncrementRandomTask( + table, + currentTaskId++, + randSeed, + Math.max(flags.numOps, flags.numKeys), + flags.numKeys, + flags, + generator, + limiter, + semaphore + )); + break; + default: + System.err.println("Unknown benchmark: " + benchmark); + break; + } + + if (tasks.isEmpty()) { + continue; + } + + final CountDownLatch latch = new CountDownLatch(tasks.size()); + @Cleanup("shutdown") + ExecutorService executor = Executors.newCachedThreadPool(); + for (BenchmarkTask task : tasks) { + executor.submit(() -> { + try { + task.runTask(); + } catch (Exception e) { + log.error("Encountered issue at running benchmark task {}", + task.tid, e); + } finally { + latch.countDown(); + } + + }); + } + + @Cleanup("shutdown") + ExecutorService statsExecutor = Executors.newSingleThreadExecutor(); + statsExecutor.submit(() -> reportStats(tasks)); + + latch.await(); + + log.info("------------------- DONE -----------------------"); + tasks.forEach(task -> task.printAggregatedStats()); + } + } + } + } + + private void reportStats(List tasks) { + long oldTime = System.nanoTime(); + + while (true) { + try { + Thread.sleep(10000); + } catch (InterruptedException ie) { + break; + } + + final long startTime = oldTime; + tasks.forEach(task -> task.reportStats(startTime)); + oldTime = System.nanoTime(); + } + } + + private static final DecimalFormat throughputFormat = new PaddingDecimalFormat("0.0", 8); + private static final DecimalFormat dec = new PaddingDecimalFormat("0.000", 7); + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteRandomTask.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteRandomTask.java new file mode 100644 index 00000000000..ee9d4cc3433 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteRandomTask.java @@ -0,0 +1,50 @@ +/* + * 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.tools.perf.table; + +import com.google.common.util.concurrent.RateLimiter; +import io.netty.buffer.ByteBuf; +import java.util.concurrent.Semaphore; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.tools.perf.table.PerfClient.Flags; + +/** + * Write key/values in sequence. + */ +class WriteRandomTask extends WriteTask { + + WriteRandomTask(Table table, + int tid, + long randSeed, + long numRecords, + long keyRange, + Flags flags, + KeyGenerator generator, + RateLimiter limiter, + Semaphore semaphore) { + super(table, tid, randSeed, numRecords, keyRange, flags, generator, limiter, semaphore); + } + + @Override + protected void getKey(ByteBuf key, long id, long range) { + getRandomKey(key, range); + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteSequentialTask.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteSequentialTask.java new file mode 100644 index 00000000000..963cfa74cd3 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteSequentialTask.java @@ -0,0 +1,50 @@ +/* + * 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.tools.perf.table; + +import com.google.common.util.concurrent.RateLimiter; +import io.netty.buffer.ByteBuf; +import java.util.concurrent.Semaphore; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.tools.perf.table.PerfClient.Flags; + +/** + * Write key/values in sequence. + */ +class WriteSequentialTask extends WriteTask { + + WriteSequentialTask(Table table, + int tid, + long randSeed, + long numRecords, + long keyRange, + Flags flags, + KeyGenerator generator, + RateLimiter limiter, + Semaphore semaphore) { + super(table, tid, randSeed, numRecords, keyRange, flags, generator, limiter, semaphore); + } + + @Override + protected void getKey(ByteBuf key, long id, long range) { + getFixedKey(key, id); + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteTask.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteTask.java new file mode 100644 index 00000000000..a74c67abc1e --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/WriteTask.java @@ -0,0 +1,112 @@ +/* + * 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.tools.perf.table; + +import com.google.common.util.concurrent.RateLimiter; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.buffer.Unpooled; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.tools.perf.table.PerfClient.Flags; +import org.apache.bookkeeper.tools.perf.table.PerfClient.OP; +import org.apache.bookkeeper.tools.perf.table.PerfClient.OpStats; + +/** + * Write task to inject key/value pairs to the table. + */ +@Slf4j +abstract class WriteTask extends BenchmarkTask { + + protected final RateLimiter limiter; + protected final Semaphore semaphore; + protected final byte[] valueBytes; + protected final OpStats writeOpStats; + + WriteTask(Table table, + int tid, + long randSeed, + long numRecords, + long keyRange, + Flags flags, + KeyGenerator generator, + RateLimiter limiter, + Semaphore semaphore) { + super(table, tid, randSeed, numRecords, keyRange, flags, generator); + this.limiter = limiter; + this.semaphore = semaphore; + this.valueBytes = new byte[flags.valueSize]; + ThreadLocalRandom.current().nextBytes(valueBytes); + this.writeOpStats = new OpStats(OP.PUT.name()); + } + + @Override + protected void runTask() throws Exception { + for (long i = 0L; i < numRecords; ++i) { + if (null != semaphore) { + semaphore.acquire(); + } + if (null != limiter) { + limiter.acquire(); + } + writeKey(i, valueBytes); + } + } + + protected abstract void getKey(ByteBuf key, long id, long range); + + void writeKey(long i, byte[] valueBytes) { + final ByteBuf keyBuf = PooledByteBufAllocator.DEFAULT.heapBuffer(flags.keySize); + getKey(keyBuf, i, keyRange); + keyBuf.writerIndex(keyBuf.readerIndex() + keyBuf.writableBytes()); + final ByteBuf valBuf = Unpooled.wrappedBuffer(valueBytes); + + final long startTime = System.nanoTime(); + table.put(keyBuf, valBuf) + .whenComplete((result, cause) -> { + if (null != semaphore) { + semaphore.release(); + } + if (null != cause) { + log.error("Error at put key/value", cause); + } else { + long latencyMicros = TimeUnit.NANOSECONDS.toMicros( + System.nanoTime() - startTime + ); + writeOpStats.recordOp(latencyMicros); + } + keyBuf.release(); + valBuf.release(); + }); + } + + @Override + protected void reportStats(long oldTime) { + writeOpStats.reportStats(oldTime); + } + + @Override + protected void printAggregatedStats() { + writeOpStats.printAggregatedStats(); + } +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/package-info.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/package-info.java new file mode 100644 index 00000000000..b36e39b3439 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/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. + */ + +/** + * Table related perf command. + */ +package org.apache.bookkeeper.tools.perf.table; \ No newline at end of file From 5a03dc4d57102467fb3306bcbbb6772545110f68 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Tue, 2 Oct 2018 22:03:28 -0700 Subject: [PATCH 0069/1642] Enhance Auditor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Descriptions of the changes in this PR: - store last checkAllLedgers executed time in metadataserver. - use that value for determining the initialDelay of checkAllLedgers call - add dedicated metric for bookieCheck - add more log lines in Auditor ### Motivation In Auditor checkAllLedgers is scheduled to run for every ‘auditorPeriodicCheckInterval’ and with initial delay of ‘auditorPeriodicCheckInterval’. But the problem is if Auditor role keeps transforming to another AR process, it might not get chance to do ‘checkAllLedgers’ task. Considering default value for 'auditorPeriodicCheckInterval' is 1 week, it is not uncommon situation. ### Changes Thats why I'm planning to persist last successful checkAllLedgers execution ctime in ZK, so that next time when a new AR becomes Auditor, it would use that as reference to decide the initial delay value. Author: cguttapalem Author: Sijie Guo Reviewers: Sijie Guo , Venkateswararao Jujjuri (JV) , Enrico Olivelli This closes #1683 from reddycharan/auditormetricslogs --- .travis.yml | 5 +- .../src/main/proto/DataFormats.proto | 7 + .../meta/LedgerUnderreplicationManager.java | 16 +++ .../meta/ZkLedgerUnderreplicationManager.java | 51 ++++++- .../bookkeeper/replication/Auditor.java | 133 ++++++++++++------ .../replication/ReplicationStats.java | 1 + .../bookkeeper/util/BookKeeperConstants.java | 1 + .../replication/AuditorLedgerCheckerTest.java | 1 + .../replication/AuditorPeriodicCheckTest.java | 129 ++++++++++++++++- .../TestLedgerUnderreplicationManager.java | 16 +++ 10 files changed, 311 insertions(+), 49 deletions(-) diff --git a/.travis.yml b/.travis.yml index 56b6805467e..c645809bf6f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -32,8 +32,9 @@ matrix: osx_image: xcode9.2 - os: linux env: CUSTOM_JDK="oraclejdk8" - - os: linux - env: CUSTOM_JDK="oraclejdk9" + # disabled oraclejdk9 since it has unknown failures on https://github.com/apache/bookkeeper/pull/1683 + #- os: linux + # env: CUSTOM_JDK="oraclejdk9" - os: linux env: CUSTOM_JDK="oraclejdk10" diff --git a/bookkeeper-proto/src/main/proto/DataFormats.proto b/bookkeeper-proto/src/main/proto/DataFormats.proto index 92eaa5fd396..79d9b2f6716 100644 --- a/bookkeeper-proto/src/main/proto/DataFormats.proto +++ b/bookkeeper-proto/src/main/proto/DataFormats.proto @@ -95,3 +95,10 @@ message LockDataFormat { message AuditorVoteFormat { optional string bookieId = 1; } + +/** + * information of checkAllLedgers execution + */ +message CheckAllLedgersFormat { + optional int64 checkAllLedgersCTime = 1; +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerUnderreplicationManager.java index b5447beee2f..304f1848b08 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerUnderreplicationManager.java @@ -168,6 +168,22 @@ boolean initializeLostBookieRecoveryDelay(int lostBookieRecoveryDelay) */ int getLostBookieRecoveryDelay() throws ReplicationException.UnavailableException; + /** + * Setter for the CheckAllLedgers last executed ctime. + * + * @param checkAllLedgersCTime + * @throws ReplicationException.UnavailableException + */ + void setCheckAllLedgersCTime(long checkAllLedgersCTime) throws ReplicationException.UnavailableException; + + /** + * Getter for the CheckAllLedgers last executed ctime. + * + * @return the long value of checkAllLedgersCTime + * @throws ReplicationException.UnavailableException + */ + long getCheckAllLedgersCTime() throws ReplicationException.UnavailableException; + /** * Receive notification asynchronously when the lostBookieRecoveryDelay value is Changed. * diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java index 5ae90809159..f641aaa1e1c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; +import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.TextFormat; import com.google.protobuf.TextFormat.ParseException; @@ -46,6 +47,7 @@ import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.DNS; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; +import org.apache.bookkeeper.proto.DataFormats.CheckAllLedgersFormat; import org.apache.bookkeeper.proto.DataFormats.LedgerRereplicationLayoutFormat; import org.apache.bookkeeper.proto.DataFormats.LockDataFormat; import org.apache.bookkeeper.proto.DataFormats.UnderreplicatedLedgerFormat; @@ -115,6 +117,7 @@ int getLedgerZNodeVersion() { private final String layoutZNode; private final AbstractConfiguration conf; private final String lostBookieRecoveryDelayZnode; + private final String checkAllLedgersCtimeZnode; private final ZooKeeper zkc; private final SubTreeCache subTreeCache; @@ -127,7 +130,7 @@ public ZkLedgerUnderreplicationManager(AbstractConfiguration conf, ZooKeeper zkc + BookKeeperConstants.DEFAULT_ZK_LEDGERS_ROOT_PATH; urLockPath = basePath + '/' + BookKeeperConstants.UNDER_REPLICATION_LOCK; lostBookieRecoveryDelayZnode = basePath + '/' + BookKeeperConstants.LOSTBOOKIERECOVERYDELAY_NODE; - + checkAllLedgersCtimeZnode = basePath + '/' + BookKeeperConstants.CHECK_ALL_LEDGERS_CTIME; idExtractionPattern = Pattern.compile("urL(\\d+)$"); this.zkc = zkc; this.subTreeCache = new SubTreeCache(new SubTreeCache.TreeProvider() { @@ -885,4 +888,50 @@ public String getReplicationWorkerIdRereplicatingLedger(long ledgerId) } return replicationWorkerId; } + + @Override + public void setCheckAllLedgersCTime(long checkAllLedgersCTime) throws UnavailableException { + if (LOG.isDebugEnabled()) { + LOG.debug("setCheckAllLedgersCTime"); + } + try { + List zkAcls = ZkUtils.getACLs(conf); + CheckAllLedgersFormat.Builder builder = CheckAllLedgersFormat.newBuilder(); + builder.setCheckAllLedgersCTime(checkAllLedgersCTime); + byte[] checkAllLedgersFormatByteArray = builder.build().toByteArray(); + if (zkc.exists(checkAllLedgersCtimeZnode, false) != null) { + zkc.setData(checkAllLedgersCtimeZnode, checkAllLedgersFormatByteArray, -1); + } else { + zkc.create(checkAllLedgersCtimeZnode, checkAllLedgersFormatByteArray, zkAcls, CreateMode.PERSISTENT); + } + } catch (KeeperException ke) { + throw new ReplicationException.UnavailableException("Error contacting zookeeper", ke); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new ReplicationException.UnavailableException("Interrupted while contacting zookeeper", ie); + } + } + + @Override + public long getCheckAllLedgersCTime() throws UnavailableException { + if (LOG.isDebugEnabled()) { + LOG.debug("setCheckAllLedgersCTime"); + } + try { + byte[] data = zkc.getData(checkAllLedgersCtimeZnode, false, null); + CheckAllLedgersFormat checkAllLedgersFormat = CheckAllLedgersFormat.parseFrom(data); + return checkAllLedgersFormat.hasCheckAllLedgersCTime() ? checkAllLedgersFormat.getCheckAllLedgersCTime() + : -1; + } catch (KeeperException.NoNodeException ne) { + LOG.warn("checkAllLedgersCtimeZnode is not yet available"); + return -1; + } catch (KeeperException ke) { + throw new ReplicationException.UnavailableException("Error contacting zookeeper", ke); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new ReplicationException.UnavailableException("Interrupted while contacting zookeeper", ie); + } catch (InvalidProtocolBufferException ipbe) { + throw new ReplicationException.UnavailableException("Error while parsing ZK protobuf binary data", ipbe); + } + } } 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 c22889c87dd..1fcd0f88877 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 @@ -93,6 +93,7 @@ public class Auditor implements AutoCloseable { private final OpStatsLogger uRLPublishTimeForLostBookies; private final OpStatsLogger bookieToLedgersMapCreationTime; private final OpStatsLogger checkAllLedgersTime; + private final OpStatsLogger auditBookiesTime; private final Counter numLedgersChecked; private final OpStatsLogger numFragmentsPerLedger; private final OpStatsLogger numBookiesPerLedger; @@ -154,6 +155,7 @@ public Auditor(final String bookieIdentifier, bookieToLedgersMapCreationTime = this.statsLogger .getOpStatsLogger(ReplicationStats.BOOKIE_TO_LEDGERS_MAP_CREATION_TIME); checkAllLedgersTime = this.statsLogger.getOpStatsLogger(ReplicationStats.CHECK_ALL_LEDGERS_TIME); + auditBookiesTime = this.statsLogger.getOpStatsLogger(ReplicationStats.AUDIT_BOOKIES_TIME); numLedgersChecked = this.statsLogger.getCounter(ReplicationStats.NUM_LEDGERS_CHECKED); numFragmentsPerLedger = statsLogger.getOpStatsLogger(ReplicationStats.NUM_FRAGMENTS_PER_LEDGER); numBookiesPerLedger = statsLogger.getOpStatsLogger(ReplicationStats.NUM_BOOKIES_PER_LEDGER); @@ -214,16 +216,19 @@ private void initialize(ServerConfiguration conf, BookKeeper bkc) private void submitShutdownTask() { synchronized (this) { + LOG.info("Executing submitShutdownTask"); if (executor.isShutdown()) { + LOG.info("executor is already shutdown"); return; } executor.submit(new Runnable() { - public void run() { - synchronized (Auditor.this) { - executor.shutdown(); - } + public void run() { + synchronized (Auditor.this) { + LOG.info("Shutting down Auditor's Executor"); + executor.shutdown(); } - }); + } + }); } } @@ -381,46 +386,11 @@ public void start() { return; } - long interval = conf.getAuditorPeriodicCheckInterval(); - - if (interval > 0) { - LOG.info("Auditor periodic ledger checking enabled" - + " 'auditorPeriodicCheckInterval' {} seconds", interval); - executor.scheduleAtFixedRate(new Runnable() { - public void run() { - try { - if (!ledgerUnderreplicationManager.isLedgerReplicationEnabled()) { - LOG.info("Ledger replication disabled, skipping"); - return; - } - - Stopwatch stopwatch = Stopwatch.createStarted(); - checkAllLedgers(); - checkAllLedgersTime.registerSuccessfulEvent(stopwatch.stop() - .elapsed(TimeUnit.MILLISECONDS), - TimeUnit.MILLISECONDS); - } catch (KeeperException ke) { - LOG.error("Exception while running periodic check", ke); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - LOG.error("Interrupted while running periodic check", ie); - } catch (BKException bke) { - LOG.error("Exception running periodic check", bke); - } catch (IOException ioe) { - LOG.error("I/O exception running periodic check", ioe); - } catch (ReplicationException.UnavailableException ue) { - LOG.error("Underreplication manager unavailable running periodic check", ue); - } - } - }, interval, interval, TimeUnit.SECONDS); - } else { - LOG.info("Periodic checking disabled"); - } try { watchBookieChanges(); knownBookies = getAvailableBookies(); } catch (BKException bke) { - LOG.error("Couldn't get bookie list, exiting", bke); + LOG.error("Couldn't get bookie list, so exiting", bke); submitShutdownTask(); } @@ -428,7 +398,8 @@ public void run() { this.ledgerUnderreplicationManager .notifyLostBookieRecoveryDelayChanged(new LostBookieRecoveryDelayChangedCb()); } catch (UnavailableException ue) { - LOG.error("Exception while registering for LostBookieRecoveryDelay change notification", ue); + LOG.error("Exception while registering for LostBookieRecoveryDelay change notification, so exiting", + ue); submitShutdownTask(); } @@ -441,6 +412,71 @@ public void run() { + " 'auditorPeriodicBookieCheckInterval' {} seconds", bookieCheckInterval); executor.scheduleAtFixedRate(bookieCheck, 0, bookieCheckInterval, TimeUnit.SECONDS); } + + long interval = conf.getAuditorPeriodicCheckInterval(); + + if (interval > 0) { + LOG.info("Auditor periodic ledger checking enabled" + " 'auditorPeriodicCheckInterval' {} seconds", + interval); + + long checkAllLedgersLastExecutedCTime; + long durationSinceLastExecutionInSecs; + long initialDelay; + try { + checkAllLedgersLastExecutedCTime = ledgerUnderreplicationManager.getCheckAllLedgersCTime(); + } catch (UnavailableException ue) { + LOG.error("Got UnavailableException while trying to get checkAllLedgersCTime", ue); + checkAllLedgersLastExecutedCTime = -1; + } + if (checkAllLedgersLastExecutedCTime == -1) { + durationSinceLastExecutionInSecs = -1; + initialDelay = 0; + } else { + durationSinceLastExecutionInSecs = (System.currentTimeMillis() - checkAllLedgersLastExecutedCTime) + / 1000; + if (durationSinceLastExecutionInSecs < 0) { + // this can happen if there is no strict time ordering + durationSinceLastExecutionInSecs = 0; + } + initialDelay = durationSinceLastExecutionInSecs > interval ? 0 + : (interval - durationSinceLastExecutionInSecs); + } + LOG.info( + "checkAllLedgers scheduling info. checkAllLedgersLastExecutedCTime: {} " + + "durationSinceLastExecutionInSecs: {} initialDelay: {} interval: {}", + checkAllLedgersLastExecutedCTime, durationSinceLastExecutionInSecs, initialDelay, interval); + + executor.scheduleAtFixedRate(new Runnable() { + public void run() { + try { + if (!ledgerUnderreplicationManager.isLedgerReplicationEnabled()) { + LOG.info("Ledger replication disabled, skipping checkAllLedgers"); + return; + } + + Stopwatch stopwatch = Stopwatch.createStarted(); + LOG.info("Starting checkAllLedgers"); + checkAllLedgers(); + long checkAllLedgersDuration = stopwatch.stop().elapsed(TimeUnit.MILLISECONDS); + LOG.info("Completed checkAllLedgers in {} milliSeconds", checkAllLedgersDuration); + checkAllLedgersTime.registerSuccessfulEvent(checkAllLedgersDuration, TimeUnit.MILLISECONDS); + } catch (KeeperException ke) { + LOG.error("Exception while running periodic check", ke); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + LOG.error("Interrupted while running periodic check", ie); + } catch (BKException bke) { + LOG.error("Exception running periodic check", bke); + } catch (IOException ioe) { + LOG.error("I/O exception running periodic check", ioe); + } catch (ReplicationException.UnavailableException ue) { + LOG.error("Underreplication manager unavailable running periodic check", ue); + } + } + }, initialDelay, interval, TimeUnit.SECONDS); + } else { + LOG.info("Periodic checking disabled"); + } } } @@ -519,7 +555,7 @@ private void auditBookies() + "Will retry after a period"); return; } - + LOG.info("Starting auditBookies"); Stopwatch stopwatch = Stopwatch.createStarted(); // put exit cases here Map> ledgerDetails = generateBookie2LedgersIndex(); @@ -551,10 +587,12 @@ private void auditBookies() } catch (ReplicationException e) { throw new BKAuditException(e.getMessage(), e.getCause()); } - uRLPublishTimeForLostBookies.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS), + uRLPublishTimeForLostBookies.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS); } - + LOG.info("Completed auditBookies"); + auditBookiesTime.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS), + TimeUnit.MILLISECONDS); } private Map> generateBookie2LedgersIndex() @@ -691,6 +729,11 @@ void checkAllLedgers() throws BKException, IOException, InterruptedException, Ke } }, null, BKException.Code.OK, BKException.Code.ReadException); FutureUtils.result(processFuture, BKException.HANDLER); + try { + ledgerUnderreplicationManager.setCheckAllLedgersCTime(System.currentTimeMillis()); + } catch (UnavailableException ue) { + LOG.error("Got exception while trying to set checkAllLedgersCTime", ue); + } } finally { admin.close(); client.close(); 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 b1afa816d04..eac30eca891 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 @@ -33,6 +33,7 @@ public interface ReplicationStats { String URL_PUBLISH_TIME_FOR_LOST_BOOKIE = "URL_PUBLISH_TIME_FOR_LOST_BOOKIE"; String BOOKIE_TO_LEDGERS_MAP_CREATION_TIME = "BOOKIE_TO_LEDGERS_MAP_CREATION_TIME"; String CHECK_ALL_LEDGERS_TIME = "CHECK_ALL_LEDGERS_TIME"; + String AUDIT_BOOKIES_TIME = "AUDIT_BOOKIES_TIME"; String NUM_FRAGMENTS_PER_LEDGER = "NUM_FRAGMENTS_PER_LEDGER"; String NUM_BOOKIES_PER_LEDGER = "NUM_BOOKIES_PER_LEDGER"; String NUM_LEDGERS_CHECKED = "NUM_LEDGERS_CHECKED"; 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 bd6801f9404..8cfe9422fb5 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 @@ -42,6 +42,7 @@ public class BookKeeperConstants { public static final String UNDER_REPLICATION_LOCK = "locks"; public static final String DISABLE_NODE = "disable"; public static final String LOSTBOOKIERECOVERYDELAY_NODE = "lostBookieRecoveryDelay"; + public static final String CHECK_ALL_LEDGERS_CTIME = "checkallledgersctime"; public static final String DEFAULT_ZK_LEDGERS_ROOT_PATH = "/ledgers"; public static final String LAYOUT_ZNODE = "LAYOUT"; public static final String INSTANCEID = "INSTANCEID"; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index a09720b2013..342c443add6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -129,6 +129,7 @@ public void setUp() throws Exception { + "/underreplication/auditorelection"; urLedgerMgr = new ZkLedgerUnderreplicationManager(baseClientConf, zkc); + urLedgerMgr.setCheckAllLedgersCTime(System.currentTimeMillis()); startAuditorElectors(); rng = new Random(System.currentTimeMillis()); // Initialize the Random urLedgerList = new HashSet(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 84788a35324..52580c56b19 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -20,17 +20,20 @@ */ package org.apache.bookkeeper.replication; +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import io.netty.buffer.ByteBuf; + import java.io.File; import java.io.FileOutputStream; import java.io.FilenameFilter; import java.io.IOException; import java.net.URI; +import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; @@ -38,14 +41,18 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieAccessor; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.IndexPersistenceMgr; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.LedgerHandleAdapter; @@ -56,8 +63,14 @@ import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; +import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; +import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -207,7 +220,7 @@ public void testIndexCorruption() throws Exception { out.close(); long underReplicatedLedger = -1; - for (int i = 0; i < 10; i++) { + for (int i = 0; i < 15; i++) { underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); if (underReplicatedLedger != -1) { break; @@ -351,6 +364,120 @@ public void run() { } } + @Test + public void testInitialDelayOfCheckAllLedgers() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(bsConfs.get(0)); + validateInitialDelayOfCheckAllLedgers(urm, -1, 1000, servConf, bkc); + validateInitialDelayOfCheckAllLedgers(urm, 999, 1000, servConf, bkc); + validateInitialDelayOfCheckAllLedgers(urm, 1001, 1000, servConf, bkc); + } + + void validateInitialDelayOfCheckAllLedgers(LedgerUnderreplicationManager urm, long timeSinceLastExecutedInSecs, + long auditorPeriodicCheckInterval, ServerConfiguration servConf, BookKeeper bkc) + throws UnavailableException, UnknownHostException, InterruptedException { + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + TestOpStatsLogger checkAllLedgersStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.CHECK_ALL_LEDGERS_TIME); + servConf.setAuditorPeriodicCheckInterval(auditorPeriodicCheckInterval); + final TestAuditor auditor = new TestAuditor(Bookie.getBookieAddress(servConf).toString(), servConf, bkc, false, + statsLogger); + CountDownLatch latch = auditor.getLatch(); + assertEquals("CHECK_ALL_LEDGERS_TIME SuccessCount", 0, checkAllLedgersStatsLogger.getSuccessCount()); + long curTimeBeforeStart = System.currentTimeMillis(); + long checkAllLedgersCTime = -1; + long initialDelayInMsecs = -1; + long nextExpectedCheckAllLedgersExecutionTime = -1; + long bufferTimeInMsecs = 12000L; + if (timeSinceLastExecutedInSecs == -1) { + /* + * if we are setting checkAllLedgersCTime to -1, it means that + * checkAllLedgers hasn't run before. So initialDelay for + * checkAllLedgers should be 0. + */ + checkAllLedgersCTime = -1; + initialDelayInMsecs = 0; + } else { + checkAllLedgersCTime = curTimeBeforeStart - timeSinceLastExecutedInSecs * 1000L; + initialDelayInMsecs = timeSinceLastExecutedInSecs > auditorPeriodicCheckInterval ? 0 + : (auditorPeriodicCheckInterval - timeSinceLastExecutedInSecs) * 1000L; + } + /* + * next checkAllLedgers should happen atleast after + * nextExpectedCheckAllLedgersExecutionTime. + */ + nextExpectedCheckAllLedgersExecutionTime = curTimeBeforeStart + initialDelayInMsecs; + + urm.setCheckAllLedgersCTime(checkAllLedgersCTime); + auditor.start(); + /* + * since auditorPeriodicCheckInterval are higher values (in the order of + * 100s of seconds), its ok bufferTimeInMsecs to be ` 10 secs. + */ + assertTrue("checkAllLedgers should have executed with initialDelay " + initialDelayInMsecs, + latch.await(initialDelayInMsecs + bufferTimeInMsecs, TimeUnit.MILLISECONDS)); + for (int i = 0; i < 10; i++) { + Thread.sleep(100); + if (checkAllLedgersStatsLogger.getSuccessCount() >= 1) { + break; + } + } + assertEquals("CHECK_ALL_LEDGERS_TIME SuccessCount", 1, checkAllLedgersStatsLogger.getSuccessCount()); + long currentCheckAllLedgersCTime = urm.getCheckAllLedgersCTime(); + assertTrue( + "currentCheckAllLedgersCTime: " + currentCheckAllLedgersCTime + + " should be greater than nextExpectedCheckAllLedgersExecutionTime: " + + nextExpectedCheckAllLedgersExecutionTime, + currentCheckAllLedgersCTime > nextExpectedCheckAllLedgersExecutionTime); + assertTrue( + "currentCheckAllLedgersCTime: " + currentCheckAllLedgersCTime + + " should be lesser than nextExpectedCheckAllLedgersExecutionTime+bufferTimeInMsecs: " + + (nextExpectedCheckAllLedgersExecutionTime + bufferTimeInMsecs), + currentCheckAllLedgersCTime < (nextExpectedCheckAllLedgersExecutionTime + bufferTimeInMsecs)); + auditor.close(); + } + + class TestAuditor extends Auditor { + + final AtomicReference latchRef = new AtomicReference(new CountDownLatch(1)); + + public TestAuditor(String bookieIdentifier, ServerConfiguration conf, BookKeeper bkc, boolean ownBkc, + StatsLogger statsLogger) throws UnavailableException { + super(bookieIdentifier, conf, bkc, ownBkc, statsLogger); + } + + void checkAllLedgers() throws BKException, IOException, InterruptedException, KeeperException { + super.checkAllLedgers(); + latchRef.get().countDown(); + } + + CountDownLatch getLatch() { + return latchRef.get(); + } + + void setLatch(CountDownLatch latch) { + latchRef.set(latch); + } + } + private BookieSocketAddress replaceBookieWithWriteFailingBookie(LedgerHandle lh) throws Exception { int bookieIdx = -1; Long entryId = LedgerHandleAdapter.getLedgerMetadata(lh).getEnsembles().firstKey(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java index db0e308eb41..26285786698 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java @@ -43,6 +43,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory; @@ -764,6 +765,21 @@ public void run() { assertEquals("All hierarchies should be cleaned up", 0, children.size()); } + @Test + public void testCheckAllLedgersCTime() throws Exception { + @Cleanup + LedgerUnderreplicationManager underReplicaMgr1 = lmf1.newLedgerUnderreplicationManager(); + @Cleanup + LedgerUnderreplicationManager underReplicaMgr2 = lmf2.newLedgerUnderreplicationManager(); + assertEquals(-1, underReplicaMgr1.getCheckAllLedgersCTime()); + long curTime = System.currentTimeMillis(); + underReplicaMgr2.setCheckAllLedgersCTime(curTime); + assertEquals(curTime, underReplicaMgr1.getCheckAllLedgersCTime()); + curTime = System.currentTimeMillis(); + underReplicaMgr2.setCheckAllLedgersCTime(curTime); + assertEquals(curTime, underReplicaMgr1.getCheckAllLedgersCTime()); + } + private void verifyMarkLedgerUnderreplicated(Collection missingReplica) throws KeeperException, InterruptedException, ReplicationException { Long ledgerA = 0xfeadeefdacL; From 2b05be0534b099e4e5f9fbd1f11706363489cd10 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 3 Oct 2018 08:15:35 +0200 Subject: [PATCH 0070/1642] Fix Kerberos tests on JDK11: Descriptions of the changes in this PR: - move to Kerby (MiniKDC code copied from Apache ZooKeeper repository) - adapt tests to run with new KDC - force TCP on krb5.conf for tests (this is needed for Kerby, which opens only TCP by default config) Master Issue: #1710 Author: Enrico Olivelli Reviewers: Ivan Kelly , Sijie Guo This closes #1716 from eolivelli/fix/kerby-11 --- bookkeeper-server/pom.xml | 33 +- .../meta/LedgerManagerIteratorTest.java | 4 +- .../bookkeeper/sasl/GSSAPIBookKeeperTest.java | 39 +- .../org/apache/bookkeeper/sasl/MiniKdc.java | 349 ++++++++++++++++++ pom.xml | 4 +- 5 files changed, 395 insertions(+), 34 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/MiniKdc.java diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index 90dcf0a5c46..e154a36ebf1 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -121,9 +121,28 @@ test - org.apache.hadoop - hadoop-minikdc - test + org.apache.kerby + kerby-config + ${kerby.version} + test + + + org.slf4j + * + + + + + org.apache.kerby + kerb-simplekdc + ${kerby.version} + test + + + org.slf4j + * + + org.apache.zookeeper @@ -134,14 +153,6 @@ - - - org.apache.felix - maven-bundle-plugin - ${maven-bundle-plugin.version} - true - true - org.apache.maven.plugins maven-jar-plugin diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java index 1804b2a99f3..ac8223bf00b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java @@ -39,6 +39,7 @@ import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -52,7 +53,6 @@ import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.Version; -import org.apache.mina.util.ConcurrentHashSet; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs; import org.junit.After; @@ -445,7 +445,7 @@ public void checkConcurrentModifications() throws Throwable { final long start = MathUtils.nowInNano(); final CountDownLatch latch = new CountDownLatch(1); ArrayList threads = new ArrayList<>(); - final ConcurrentHashSet createdLedgers = new ConcurrentHashSet<>(); + final ConcurrentSkipListSet createdLedgers = new ConcurrentSkipListSet<>(); for (int i = 0; i < numWriters; ++i) { Thread thread = new Thread(safeWrapper(() -> { LedgerManager writerLM = getIndependentLedgerManager(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java index db39ae734c5..a538950f777 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java @@ -43,14 +43,13 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.hadoop.minikdc.MiniKdc; import org.apache.zookeeper.KeeperException; -import org.junit.After; import org.junit.AfterClass; -import org.junit.Before; -import org.junit.Rule; +import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; @@ -66,23 +65,24 @@ public class GSSAPIBookKeeperTest extends BookKeeperClusterTestCase { private static final byte[] PASSWD = "testPasswd".getBytes(); private static final byte[] ENTRY = "TestEntry".getBytes(); - private MiniKdc kdc; - private Properties conf; + private static MiniKdc kdc; + private static Properties conf; - @Rule - public TemporaryFolder kdcDir = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder kdcDir = new TemporaryFolder(); - @Rule - public TemporaryFolder kerberosWorkDir = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder kerberosWorkDir = new TemporaryFolder(); - @Before - public void startMiniKdc() throws Exception { + @BeforeClass + public static void startMiniKdc() throws Exception { conf = MiniKdc.createConf(); kdc = new MiniKdc(conf, kdcDir.getRoot()); kdc.start(); - ServerConfiguration bookieConf = newServerConfiguration(); + // this is just to calculate "localhostName" the same way the bookie does + ServerConfiguration bookieConf = TestBKConfiguration.newServerConfiguration(); bookieConf.setUseHostNameAsBookieID(true); String localhostName = Bookie.getBookieAddress(bookieConf).getHostName(); @@ -127,16 +127,17 @@ public void startMiniKdc() throws Exception { File krb5file = new File(kerberosWorkDir.getRoot(), "krb5.conf"); try (FileWriter writer = new FileWriter(krb5file)) { - writer.write("[libdefaults]\n" + String conf = "[libdefaults]\n" + " default_realm = " + kdc.getRealm() + "\n" + + " udp_preference_limit = 1\n" // force use TCP + "\n" + "\n" + "[realms]\n" + " " + kdc.getRealm() + " = {\n" + " kdc = " + kdc.getHost() + ":" + kdc.getPort() + "\n" - + " }" - ); - + + " }"; + writer.write(conf); + LOG.info("krb5.conf:\n" + conf); } System.setProperty("java.security.auth.login.config", jaasFile.getAbsolutePath()); @@ -145,8 +146,8 @@ public void startMiniKdc() throws Exception { } - @After - public void stopMiniKdc() { + @AfterClass + public static void stopMiniKdc() { System.clearProperty("java.security.auth.login.config"); System.clearProperty("java.security.krb5.conf"); if (kdc != null) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/MiniKdc.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/MiniKdc.java new file mode 100644 index 00000000000..f5cb16164ed --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/MiniKdc.java @@ -0,0 +1,349 @@ +/** + * 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.sasl; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.HashSet; +import java.util.Locale; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import org.apache.kerby.kerberos.kerb.KrbException; +import org.apache.kerby.kerberos.kerb.server.KdcConfigKey; +import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer; +import org.apache.kerby.util.IOUtil; +import org.apache.kerby.util.NetworkUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Mini KDC based on Apache Directory Server that can be embedded in testcases + * or used from command line as a standalone KDC. + * + *

From within testcases: + * + *

MiniKdc sets one System property when started and un-set when stopped: + *

    + *
  • sun.security.krb5.debug: set to the debug value provided in the + * configuration
  • + *
+ * Because of this, multiple MiniKdc instances cannot be started in parallel. + * For example, running testcases in parallel that start a KDC each. To + * accomplish this a single MiniKdc should be used for all testcases running + * in parallel. + * + *

MiniKdc default configuration values are: + *

    + *
  • org.name=EXAMPLE (used to create the REALM)
  • + *
  • org.domain=COM (used to create the REALM)
  • + *
  • kdc.bind.address=localhost
  • + *
  • kdc.port=0 (ephemeral port)
  • + *
  • instance=DefaultKrbServer
  • + *
  • max.ticket.lifetime=86400000 (1 day)
  • + *
  • max.renewable.lifetime=604800000 (7 days)
  • + *
  • transport=TCP
  • + *
  • debug=false
  • + *
+ * The generated krb5.conf forces TCP connections. + * This code is originally from HDFS, see the file name MiniKdc there + * in case of bug fixing, history, etc. + * https://github.com/apache/hadoop/blob/trunk/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java + */ +public class MiniKdc { + + public static final String JAVA_SECURITY_KRB5_CONF = + "java.security.krb5.conf"; + public static final String SUN_SECURITY_KRB5_DEBUG = + "sun.security.krb5.debug"; + + + private static final Logger LOG = LoggerFactory.getLogger(MiniKdc.class); + + public static final String ORG_NAME = "org.name"; + public static final String ORG_DOMAIN = "org.domain"; + public static final String KDC_BIND_ADDRESS = "kdc.bind.address"; + public static final String KDC_PORT = "kdc.port"; + public static final String INSTANCE = "instance"; + public static final String MAX_TICKET_LIFETIME = "max.ticket.lifetime"; + public static final String MAX_RENEWABLE_LIFETIME = "max.renewable.lifetime"; + public static final String TRANSPORT = "transport"; + public static final String DEBUG = "debug"; + + private static final Set PROPERTIES = new HashSet(); + private static final Properties DEFAULT_CONFIG = new Properties(); + + static { + PROPERTIES.add(ORG_NAME); + PROPERTIES.add(ORG_DOMAIN); + PROPERTIES.add(KDC_BIND_ADDRESS); + PROPERTIES.add(KDC_BIND_ADDRESS); + PROPERTIES.add(KDC_PORT); + PROPERTIES.add(INSTANCE); + PROPERTIES.add(TRANSPORT); + PROPERTIES.add(MAX_TICKET_LIFETIME); + PROPERTIES.add(MAX_RENEWABLE_LIFETIME); + + DEFAULT_CONFIG.setProperty(KDC_BIND_ADDRESS, "localhost"); + DEFAULT_CONFIG.setProperty(KDC_PORT, "0"); + DEFAULT_CONFIG.setProperty(INSTANCE, "DefaultKrbServer"); + DEFAULT_CONFIG.setProperty(ORG_NAME, "EXAMPLE"); + DEFAULT_CONFIG.setProperty(ORG_DOMAIN, "COM"); + DEFAULT_CONFIG.setProperty(TRANSPORT, "TCP"); + DEFAULT_CONFIG.setProperty(MAX_TICKET_LIFETIME, "86400000"); + DEFAULT_CONFIG.setProperty(MAX_RENEWABLE_LIFETIME, "604800000"); + DEFAULT_CONFIG.setProperty(DEBUG, "false"); + } + + /** + * Convenience method that returns MiniKdc default configuration. + * + *

The returned configuration is a copy, it can be customized before using + * it to create a MiniKdc. + * @return a MiniKdc default configuration. + */ + public static Properties createConf() { + return (Properties) DEFAULT_CONFIG.clone(); + } + + private Properties conf; + private SimpleKdcServer simpleKdc; + private int port; + private String realm; + private File workDir; + private File krb5conf; + private String transport; + private boolean krb5Debug; + + public void setTransport(String transport) { + this.transport = transport; + } + + /** + * Creates a MiniKdc. + * + * @param conf MiniKdc configuration. + * @param workDir working directory, it should be the build directory. Under + * this directory an ApacheDS working directory will be created, this + * directory will be deleted when the MiniKdc stops. + * @throws Exception thrown if the MiniKdc could not be created. + */ + public MiniKdc(Properties conf, File workDir) throws Exception { + if (!conf.keySet().containsAll(PROPERTIES)) { + Set missingProperties = new HashSet(PROPERTIES); + missingProperties.removeAll(conf.keySet()); + throw new IllegalArgumentException("Missing configuration properties: " + + missingProperties); + } + this.workDir = new File(workDir, Long.toString(System.currentTimeMillis())); + if (!this.workDir.exists() + && !this.workDir.mkdirs()) { + throw new RuntimeException("Cannot create directory " + this.workDir); + } + LOG.info("Configuration:"); + LOG.info("---------------------------------------------------------------"); + for (Map.Entry entry : conf.entrySet()) { + LOG.info(" {}: {}", entry.getKey(), entry.getValue()); + } + LOG.info("---------------------------------------------------------------"); + this.conf = conf; + port = Integer.parseInt(conf.getProperty(KDC_PORT)); + String orgName = conf.getProperty(ORG_NAME); + String orgDomain = conf.getProperty(ORG_DOMAIN); + realm = orgName.toUpperCase(Locale.ENGLISH) + "." + + orgDomain.toUpperCase(Locale.ENGLISH); + } + + /** + * Returns the port of the MiniKdc. + * + * @return the port of the MiniKdc. + */ + public int getPort() { + return port; + } + + /** + * Returns the host of the MiniKdc. + * + * @return the host of the MiniKdc. + */ + public String getHost() { + return conf.getProperty(KDC_BIND_ADDRESS); + } + + /** + * Returns the realm of the MiniKdc. + * + * @return the realm of the MiniKdc. + */ + public String getRealm() { + return realm; + } + + public File getKrb5conf() { + krb5conf = new File(System.getProperty(JAVA_SECURITY_KRB5_CONF)); + return krb5conf; + } + + /** + * Starts the MiniKdc. + * + * @throws Exception thrown if the MiniKdc could not be started. + */ + public synchronized void start() throws Exception { + if (simpleKdc != null) { + throw new RuntimeException("Already started"); + } + simpleKdc = new SimpleKdcServer(); + prepareKdcServer(); + simpleKdc.init(); + resetDefaultRealm(); + simpleKdc.start(); + LOG.info("MiniKdc stated."); + } + + private void resetDefaultRealm() throws IOException { + InputStream templateResource = new FileInputStream( + getKrb5conf().getAbsolutePath()); + String content = IOUtil.readInput(templateResource); + content = content.replaceAll("default_realm = .*\n", + "default_realm = " + getRealm() + "\n"); + IOUtil.writeFile(content, getKrb5conf()); + } + + private void prepareKdcServer() throws Exception { + // transport + simpleKdc.setWorkDir(workDir); + simpleKdc.setKdcHost(getHost()); + simpleKdc.setKdcRealm(realm); + if (transport == null) { + transport = conf.getProperty(TRANSPORT); + } + if (port == 0) { + port = NetworkUtil.getServerPort(); + } + if (transport != null) { + if (transport.trim().equals("TCP")) { + simpleKdc.setKdcTcpPort(port); + simpleKdc.setAllowUdp(false); + } else if (transport.trim().equals("UDP")) { + simpleKdc.setKdcUdpPort(port); + simpleKdc.setAllowTcp(false); + } else { + throw new IllegalArgumentException("Invalid transport: " + transport); + } + } else { + throw new IllegalArgumentException("Need to set transport!"); + } + simpleKdc.getKdcConfig().setString(KdcConfigKey.KDC_SERVICE_NAME, + conf.getProperty(INSTANCE)); + if (conf.getProperty(DEBUG) != null) { + krb5Debug = getAndSet(SUN_SECURITY_KRB5_DEBUG, conf.getProperty(DEBUG)); + } + } + + /** + * Stops the MiniKdc. + */ + public synchronized void stop() { + if (simpleKdc != null) { + try { + simpleKdc.stop(); + } catch (KrbException e) { + e.printStackTrace(); + } finally { + if (conf.getProperty(DEBUG) != null) { + System.setProperty(SUN_SECURITY_KRB5_DEBUG, + Boolean.toString(krb5Debug)); + } + } + } + delete(workDir); + try { + // Will be fixed in next Kerby version. + Thread.sleep(1000); + } catch (InterruptedException e) { + e.printStackTrace(); + } + LOG.info("MiniKdc stopped."); + } + + private void delete(File f) { + if (f.isFile()) { + if (!f.delete()) { + LOG.warn("WARNING: cannot delete file " + f.getAbsolutePath()); + } + } else { + for (File c: f.listFiles()) { + delete(c); + } + if (!f.delete()) { + LOG.warn("WARNING: cannot delete directory " + f.getAbsolutePath()); + } + } + } + + /** + * Creates a principal in the KDC with the specified user and password. + * + * @param principal principal name, do not include the domain. + * @param password password. + * @throws Exception thrown if the principal could not be created. + */ + public synchronized void createPrincipal(String principal, String password) + throws Exception { + simpleKdc.createPrincipal(principal, password); + } + + /** + * Creates multiple principals in the KDC and adds them to a keytab file. + * + * @param keytabFile keytab file to add the created principals. + * @param principals principals to add to the KDC, do not include the domain. + * @throws Exception thrown if the principals or the keytab file could not be + * created. + */ + public synchronized void createPrincipal(File keytabFile, + String ... principals) + throws Exception { + simpleKdc.createPrincipals(principals); + if (keytabFile.exists() && !keytabFile.delete()) { + LOG.error("Failed to delete keytab file: " + keytabFile); + } + for (String principal : principals) { + simpleKdc.getKadmin().exportKeytab(keytabFile, principal); + } + } + + /** + * Set the System property; return the old value for caching. + * + * @param sysprop property + * @param debug true or false + * @return the previous value + */ + private boolean getAndSet(String sysprop, String debug) { + boolean old = Boolean.getBoolean(sysprop); + System.setProperty(sysprop, debug); + return old; + } +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 0b158cffbda..418e9a4f5d3 100644 --- a/pom.xml +++ b/pom.xml @@ -128,6 +128,7 @@ 2.1.2 1.12.0 21.0 + 1.1.1 2.7.3 1.3 2.1.10 @@ -171,7 +172,6 @@ 0.8.0 1.8 3.1.0 - 3.2.0 3.0.0 2.5 3.7.0 @@ -591,7 +591,7 @@ org.apache.hadoop hadoop-minikdc - ${hadoop.version} + ${hadoop.minikdc.version} org.arquillian.cube From e3beb89d6ebf380ffe71b2e70279713184e384fa Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 3 Oct 2018 01:31:58 -0700 Subject: [PATCH 0071/1642] [TABLE SERVICE] [STORAGE] improve the logic on creating the meta range for a table Descriptions of the changes in this PR: *Motivation* Currently the creation of meta range for table is deferred until we first try to use it. The propagation of stream properties is carried by the client. *Changes* Improve the creation logic to fetch stream properties from the root range. Author: Sijie Guo Reviewers: Jia Zhai This closes #1730 from sijie/cache_stream_props --- .../storage/StorageContainerStoreBuilder.java | 2 +- .../storage/impl/metadata/MetaRangeStoreImpl.java | 13 +++++++++---- .../storage/impl/metadata/RootRangeStoreImpl.java | 6 +----- .../impl/service/RangeStoreServiceFactoryImpl.java | 11 +++++------ .../impl/service/RangeStoreServiceImpl.java | 12 +++++------- .../impl/metadata/MetaRangeStoreImplTest.java | 14 ++++++++++---- .../impl/metadata/TestRootRangeStoreImpl.java | 4 ---- 7 files changed, 31 insertions(+), 31 deletions(-) diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/StorageContainerStoreBuilder.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/StorageContainerStoreBuilder.java index cac363f41ee..9dda00202c3 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/StorageContainerStoreBuilder.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/StorageContainerStoreBuilder.java @@ -159,7 +159,7 @@ public StorageContainerStore build() { placementPolicyFactory.newPlacementPolicy(), storeResources.scheduler(), mvccStoreFactory, - defaultBackendUri); + clientManagerSupplier.get()); RangeStoreContainerServiceFactoryImpl containerServiceFactory = new RangeStoreContainerServiceFactoryImpl(serviceFactory); diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java index 530d4cdce0b..b64d3205f71 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java @@ -23,6 +23,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.clients.impl.internal.api.StorageServerClientManager; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.statelib.api.mvcc.MVCCAsyncStore; import org.apache.bookkeeper.stream.proto.RangeMetadata; @@ -48,14 +49,17 @@ public class MetaRangeStoreImpl private final ScheduledExecutorService executor; private final StorageContainerPlacementPolicy rangePlacementPolicy; private final Map streams; + private final StorageServerClientManager clientManager; public MetaRangeStoreImpl(MVCCAsyncStore store, StorageContainerPlacementPolicy rangePlacementPolicy, - ScheduledExecutorService executor) { + ScheduledExecutorService executor, + StorageServerClientManager clientManager) { this.store = store; this.executor = executor; this.rangePlacementPolicy = rangePlacementPolicy; this.streams = Maps.newHashMap(); + this.clientManager = clientManager; } // @@ -100,9 +104,10 @@ public CompletableFuture getActiveRanges(GetActiveRange return metaRangeImpl.load(streamId) .thenCompose(mr -> { if (null == mr) { - StreamProperties streamProps = request.hasStreamProps() - ? request.getStreamProps() : null; - return createStreamIfMissing(streamId, metaRangeImpl, streamProps); + // meta range doesn't exist, talk to root range to get the stream props + return clientManager.getStreamProperties(streamId) + .thenCompose(streamProperties -> + createStreamIfMissing(streamId, metaRangeImpl, streamProperties)); } else { synchronized (streams) { streams.put(streamId, (MetaRangeImpl) mr); diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java index 2f2c4048e59..c066bd14e67 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java @@ -24,7 +24,6 @@ import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.validateStreamName; import com.google.protobuf.InvalidProtocolBufferException; -import java.net.URI; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import lombok.extern.slf4j.Slf4j; @@ -145,16 +144,13 @@ static final byte[] getStreamIdKey(long streamId) { return streamIdBytes; } - private final URI defaultServiceUri; private final MVCCAsyncStore store; private final StorageContainerPlacementPolicy placementPolicy; private final ScheduledExecutorService executor; - public RootRangeStoreImpl(URI defaultServiceUri, - MVCCAsyncStore store, + public RootRangeStoreImpl(MVCCAsyncStore store, StorageContainerPlacementPolicy placementPolicy, ScheduledExecutorService executor) { - this.defaultServiceUri = defaultServiceUri; this.store = store; this.placementPolicy = placementPolicy; this.executor = executor; diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceFactoryImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceFactoryImpl.java index 02dd8a3c932..937a9fa61fd 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceFactoryImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceFactoryImpl.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.stream.storage.impl.service; -import java.net.URI; +import org.apache.bookkeeper.clients.impl.internal.api.StorageServerClientManager; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.common.util.SharedResourceManager; import org.apache.bookkeeper.common.util.SharedResourceManager.Resource; @@ -38,30 +38,29 @@ public class RangeStoreServiceFactoryImpl implements RangeStoreServiceFactory { private final Resource schedulerResource; private final OrderedScheduler scheduler; private final MVCCStoreFactory storeFactory; - private final URI defaultBackendUri; + private final StorageServerClientManager clientManager; public RangeStoreServiceFactoryImpl(StorageConfiguration storageConf, StorageContainerPlacementPolicy rangePlacementPolicy, Resource schedulerResource, MVCCStoreFactory storeFactory, - URI defaultBackendUri) { + StorageServerClientManager clientManager) { this.storageConf = storageConf; this.rangePlacementPolicy = rangePlacementPolicy; this.schedulerResource = schedulerResource; this.scheduler = SharedResourceManager.shared().get(schedulerResource); this.storeFactory = storeFactory; - this.defaultBackendUri = defaultBackendUri; + this.clientManager = clientManager; } @Override public RangeStoreService createService(long scId) { return new RangeStoreServiceImpl( - storageConf, scId, rangePlacementPolicy, scheduler, storeFactory, - defaultBackendUri); + clientManager); } @Override diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java index 7adcefdb752..e3b2d3991df 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java @@ -25,12 +25,12 @@ import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.ROOT_STREAM_ID; import com.google.common.collect.Lists; -import java.net.URI; import java.util.List; import java.util.concurrent.CompletableFuture; import lombok.AccessLevel; import lombok.Getter; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.clients.impl.internal.api.StorageServerClientManager; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; @@ -64,7 +64,6 @@ import org.apache.bookkeeper.stream.storage.api.metadata.MetaRangeStore; import org.apache.bookkeeper.stream.storage.api.metadata.RangeStoreService; import org.apache.bookkeeper.stream.storage.api.metadata.RootRangeStore; -import org.apache.bookkeeper.stream.storage.conf.StorageConfiguration; import org.apache.bookkeeper.stream.storage.impl.kv.TableStoreCache; import org.apache.bookkeeper.stream.storage.impl.kv.TableStoreFactory; import org.apache.bookkeeper.stream.storage.impl.kv.TableStoreImpl; @@ -100,19 +99,18 @@ class RangeStoreServiceImpl implements RangeStoreService, AutoCloseable { @Getter(value = AccessLevel.PACKAGE) private final TableStoreFactory tableStoreFactory; - RangeStoreServiceImpl(StorageConfiguration storageConf, - long scId, + RangeStoreServiceImpl(long scId, StorageContainerPlacementPolicy rangePlacementPolicy, OrderedScheduler scheduler, MVCCStoreFactory storeFactory, - URI defaultBackendUri) { + StorageServerClientManager clientManager) { this( scId, scheduler, storeFactory, store -> new RootRangeStoreImpl( - defaultBackendUri, store, rangePlacementPolicy, scheduler.chooseThread(scId)), - store -> new MetaRangeStoreImpl(store, rangePlacementPolicy, scheduler.chooseThread(scId)), + store, rangePlacementPolicy, scheduler.chooseThread(scId)), + store -> new MetaRangeStoreImpl(store, rangePlacementPolicy, scheduler.chooseThread(scId), clientManager), store -> new TableStoreImpl(store)); } diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImplTest.java index d0156c55d27..5e5f4b3d7e8 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImplTest.java @@ -20,12 +20,16 @@ import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.DEFAULT_STREAM_CONF; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.google.common.collect.Lists; import java.util.Collections; import java.util.List; import java.util.NavigableMap; import java.util.stream.LongStream; +import org.apache.bookkeeper.clients.impl.internal.api.StorageServerClientManager; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.RangeMetadata; import org.apache.bookkeeper.stream.proto.RangeState; @@ -48,6 +52,7 @@ public class MetaRangeStoreImplTest extends MVCCAsyncStoreTestBase { private StreamProperties streamProps; private MetaRangeStoreImpl mrStoreImpl; + private StorageServerClientManager clientManager; @Override protected void doSetup() throws Exception { @@ -57,10 +62,12 @@ protected void doSetup() throws Exception { .setStreamName(name.getMethodName() + "_stream") .setStreamId(System.currentTimeMillis()) .build(); + this.clientManager = mock(StorageServerClientManager.class); this.mrStoreImpl = new MetaRangeStoreImpl( this.store, StorageContainerPlacementPolicyImpl.of(1024), - this.scheduler.chooseThread()); + this.scheduler.chooseThread(), + clientManager); } @Override @@ -68,11 +75,10 @@ protected void doTeardown() throws Exception { } GetActiveRangesRequest createRequest(StreamProperties streamProperties) { + when(clientManager.getStreamProperties(eq(this.streamProps.getStreamId()))) + .thenReturn(FutureUtils.value(streamProperties)); GetActiveRangesRequest.Builder reqBuilder = GetActiveRangesRequest.newBuilder() .setStreamId(this.streamProps.getStreamId()); - if (null != streamProperties) { - reqBuilder = reqBuilder.setStreamProps(streamProperties); - } return reqBuilder.build(); } diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java index da3b6d11146..9cdbf6424df 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java @@ -33,7 +33,6 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import java.net.URI; import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -57,8 +56,6 @@ @Slf4j public class TestRootRangeStoreImpl extends MVCCAsyncStoreTestBase { - private static final String DEFAULT_SERVICE_URI = "distributedlog://127.0.0.1/stream/storage"; - private final NamespaceConfiguration namespaceConf = NamespaceConfiguration.newBuilder() .setDefaultStreamConf(DEFAULT_STREAM_CONF) @@ -74,7 +71,6 @@ public class TestRootRangeStoreImpl extends MVCCAsyncStoreTestBase { @Override protected void doSetup() throws Exception { rootRangeStore = new RootRangeStoreImpl( - URI.create(DEFAULT_SERVICE_URI), store, StorageContainerPlacementPolicyImpl.of(1024), scheduler.chooseThread()); From 0a53b0de8a64283e496d844c587bdf22e3dad295 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 3 Oct 2018 10:34:44 +0200 Subject: [PATCH 0072/1642] Add Oracle JDK11 to Travis CI build Descriptions of the changes in this PR: ### Motivation Start testing builds on JDK11, as it is the latest available version of Java ### Changes Add Oracle JDK 11 to Travis configuration file Master Issue: #1710 Author: Reviewers: Sijie Guo This closes #1732 from eolivelli/fix/travis-jdk11 --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index c645809bf6f..7c9b257a686 100644 --- a/.travis.yml +++ b/.travis.yml @@ -37,6 +37,8 @@ matrix: # env: CUSTOM_JDK="oraclejdk9" - os: linux env: CUSTOM_JDK="oraclejdk10" + - os: linux + env: CUSTOM_JDK="oraclejdk11" before_install: - | From 9e0385014dd596fdac98cab4d91c41ed0fced01e Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 3 Oct 2018 08:29:52 -0700 Subject: [PATCH 0073/1642] [TABLE SERVICE] [CLI] improve table service related cli commands Descriptions of the changes in this PR: *Motivation* Add missing commands for table service related operations. *Changes* - Add missing commands such as get/delete - Fill the default service uri if it is not provided - Define the command ops in `Commands` util class Author: Sijie Guo Reviewers: Jia Zhai This closes #1724 from sijie/improve_console --- .../bookkeeper/common/net/ServiceURI.java | 6 ++ conf/log4j.cli.properties | 5 +- .../stream/cluster/StreamCluster.java | 5 +- .../tests/integration/stream/BkCtlTest.java | 2 +- .../standalone/StandaloneTest.java | 5 +- .../bookkeeper/stream/cli/Commands.java | 38 ++++++++ .../stream/cli/NamespaceCommandGroup.java | 4 + .../stream/cli/TableAdminCommandGroup.java | 4 + .../stream/cli/TableCommandGroup.java | 2 + .../cli/commands/AbstractStreamCommand.java | 62 +++++++++++++ .../stream/cli/commands/AdminCommand.java | 24 ++--- .../stream/cli/commands/ClientCommand.java | 19 ++-- .../commands/cluster/InitClusterCommand.java | 10 ++- .../namespace/CreateNamespaceCommand.java | 25 +++--- .../namespace/DeleteNamespaceCommand.java | 74 ++++++++++++++++ .../namespace/GetNamespaceCommand.java | 76 ++++++++++++++++ .../commands/table/CreateTableCommand.java | 52 ++++++++--- .../stream/cli/commands/table/DelCommand.java | 78 +++++++++++++++++ .../commands/table/DeleteTableCommand.java | 80 +++++++++++++++++ .../stream/cli/commands/table/GetCommand.java | 3 +- .../cli/commands/table/GetTableCommand.java | 87 +++++++++++++++++++ .../cli/commands/table/IncrementCommand.java | 3 +- .../stream/cli/commands/table/PutCommand.java | 3 +- 23 files changed, 600 insertions(+), 67 deletions(-) create mode 100644 tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/Commands.java create mode 100644 tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AbstractStreamCommand.java create mode 100644 tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/DeleteNamespaceCommand.java create mode 100644 tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/GetNamespaceCommand.java create mode 100644 tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DelCommand.java create mode 100644 tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DeleteTableCommand.java create mode 100644 tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetTableCommand.java diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java index 289c3f3726c..4e2b158df40 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java @@ -123,6 +123,12 @@ public class ServiceURI { */ public static final String SERVICE_BK = "bk"; + /** + * The default local bk service uri. + */ + public static final ServiceURI DEFAULT_LOCAL_STREAM_STORAGE_SERVICE_URI = + ServiceURI.create("bk://localhost:4181"); + private static final String SERVICE_SEP = "+"; private static final String SERVICE_DLOG_SEP = "-"; diff --git a/conf/log4j.cli.properties b/conf/log4j.cli.properties index ceb77cc934a..f90d9551aa6 100644 --- a/conf/log4j.cli.properties +++ b/conf/log4j.cli.properties @@ -54,7 +54,4 @@ log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{ log4j.logger.verbose=INFO,VERBOSECONSOLE log4j.logger.org.apache.zookeeper=ERROR -log4j.logger.org.apache.bookkeeper=ERROR -log4j.logger.org.apache.bookkeeper.tools=INFO -log4j.logger.org.apache.bookkeeper.bookie.BookieShell=INFO -log4j.logger.org.apache.bookkeeper.client.BookKeeperAdmin=INFO +log4j.logger.org.apache.bookkeeper=INFO diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java index 407b27fbe23..5490dee4724 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java @@ -245,10 +245,11 @@ private void createDefaultNamespaces() throws Exception { boolean created = false; while (!created) { try { - result(admin.getNamespace(namespaceName)); + NamespaceProperties nsProps = result(admin.getNamespace(namespaceName)); + log.info("Namespace '{}':\n{}", namespaceName, nsProps); created = true; } catch (NamespaceNotFoundException nnfe) { - log.info("Namespace '{}' is not found."); + log.info("Namespace '{}' is not found.", namespaceName); log.info("Creating namespace '{}' ...", namespaceName); try { NamespaceProperties nsProps = result( diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/BkCtlTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/BkCtlTest.java index 0bb11a13655..6067776492b 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/BkCtlTest.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/BkCtlTest.java @@ -118,7 +118,7 @@ public void createTable() throws Exception { tableName ); assertTrue( - result.getStdout().contains("Successfully created stream '" + tableName + "'")); + result.getStdout().contains("Successfully created table '" + tableName + "'")); } // diff --git a/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java b/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java index f7b70778109..29c30c369bf 100644 --- a/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java +++ b/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java @@ -80,17 +80,20 @@ public void createNamespace() throws Exception { @Test public void createTable() throws Exception { + createNamespace(); String tableName = testName.getMethodName(); ExecResult result = bkContainer.execCmd( "/opt/bookkeeper/bin/bkctl", "-u bk://localhost:4181", + "--namespace", + testName.getMethodName(), "tables", "create", tableName ); assertTrue( result.getStdout(), - result.getStdout().contains("Successfully created stream '" + tableName + "'")); + result.getStdout().contains("Successfully created table '" + tableName + "'")); } } diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/Commands.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/Commands.java new file mode 100644 index 00000000000..2c88c003df9 --- /dev/null +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/Commands.java @@ -0,0 +1,38 @@ +/* + * 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.stream.cli; + +/** + * Utils for commands. + */ +public final class Commands { + + private Commands() {} + + public static final String OP_INIT = "init"; + public static final String OP_CREATE = "create"; + public static final String OP_DELETE = "delete"; + public static final String OP_LIST = "list"; + public static final String OP_GET = "get"; + public static final String OP_DEL = "del"; + public static final String OP_PUT = "put"; + public static final String OP_INC = "inc"; + +} diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java index be9e06295bc..1300bf2515d 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java @@ -18,6 +18,8 @@ package org.apache.bookkeeper.stream.cli; import org.apache.bookkeeper.stream.cli.commands.namespace.CreateNamespaceCommand; +import org.apache.bookkeeper.stream.cli.commands.namespace.DeleteNamespaceCommand; +import org.apache.bookkeeper.stream.cli.commands.namespace.GetNamespaceCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -35,6 +37,8 @@ public class NamespaceCommandGroup extends CliCommandGroup { .withDescription(DESC) .withParent("bkctl") .addCommand(new CreateNamespaceCommand()) + .addCommand(new GetNamespaceCommand()) + .addCommand(new DeleteNamespaceCommand()) .build(); public NamespaceCommandGroup() { diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java index 60b98928e22..bce4fc58713 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java @@ -18,6 +18,8 @@ package org.apache.bookkeeper.stream.cli; import org.apache.bookkeeper.stream.cli.commands.table.CreateTableCommand; +import org.apache.bookkeeper.stream.cli.commands.table.DeleteTableCommand; +import org.apache.bookkeeper.stream.cli.commands.table.GetTableCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -35,6 +37,8 @@ public class TableAdminCommandGroup extends CliCommandGroup { .withDescription(DESC) .withParent("bkctl") .addCommand(new CreateTableCommand()) + .addCommand(new GetTableCommand()) + .addCommand(new DeleteTableCommand()) .build(); public TableAdminCommandGroup() { diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java index 09d3be61bf0..f7d96b7ab5d 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java @@ -17,6 +17,7 @@ */ package org.apache.bookkeeper.stream.cli; +import org.apache.bookkeeper.stream.cli.commands.table.DelCommand; import org.apache.bookkeeper.stream.cli.commands.table.GetCommand; import org.apache.bookkeeper.stream.cli.commands.table.IncrementCommand; import org.apache.bookkeeper.stream.cli.commands.table.PutCommand; @@ -39,6 +40,7 @@ public class TableCommandGroup extends CliCommandGroup { .addCommand(new PutCommand()) .addCommand(new GetCommand()) .addCommand(new IncrementCommand()) + .addCommand(new DelCommand()) .build(); public TableCommandGroup() { diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AbstractStreamCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AbstractStreamCommand.java new file mode 100644 index 00000000000..583dcff3df3 --- /dev/null +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AbstractStreamCommand.java @@ -0,0 +1,62 @@ +/* + * 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.stream.cli.commands; + +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.tools.common.BKCommand; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.commons.configuration.CompositeConfiguration; + + +/** + * Abstract stream storage related commands. + */ +@Slf4j +abstract class AbstractStreamCommand extends BKCommand { + + protected AbstractStreamCommand(CliSpec spec) { + super(spec); + } + + @Override + protected boolean acceptServiceUri(ServiceURI serviceURI) { + return ServiceURI.SERVICE_BK.equals(serviceURI.getServiceName()); + } + + @Override + protected boolean apply(ServiceURI serviceURI, + CompositeConfiguration conf, + BKFlags globalFlags, + CommandFlagsT cmdFlags) { + if (serviceURI == null) { + serviceURI = ServiceURI.DEFAULT_LOCAL_STREAM_STORAGE_SERVICE_URI; + log.info("Service Uri is not specified. Using default service uri : {}", serviceURI); + } + return doApply(serviceURI, conf, globalFlags, cmdFlags); + } + + protected abstract boolean doApply(ServiceURI serviceURI, + CompositeConfiguration conf, + BKFlags globalFlags, + CommandFlagsT cmdFlags); +} diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AdminCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AdminCommand.java index eb319aae8a2..bbe47b71dd3 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AdminCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AdminCommand.java @@ -17,14 +17,11 @@ */ package org.apache.bookkeeper.stream.cli.commands; -import static com.google.common.base.Preconditions.checkArgument; - import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.clients.StorageClientBuilder; import org.apache.bookkeeper.clients.admin.StorageAdminClient; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.common.net.ServiceURI; -import org.apache.bookkeeper.tools.common.BKCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -34,26 +31,17 @@ * An admin command interface provides a run method to execute admin commands. */ @Slf4j -public abstract class AdminCommand extends BKCommand { +public abstract class AdminCommand extends AbstractStreamCommand { protected AdminCommand(CliSpec spec) { super(spec); } @Override - protected boolean acceptServiceUri(ServiceURI serviceURI) { - return ServiceURI.SERVICE_BK.equals(serviceURI.getServiceName()); - } - - @Override - protected boolean apply(ServiceURI serviceURI, - CompositeConfiguration conf, - BKFlags bkFlags, - ClientFlagsT cmdFlags) { - checkArgument( - null != serviceURI, - "No service URI is provided"); - + protected boolean doApply(ServiceURI serviceURI, + CompositeConfiguration conf, + BKFlags bkFlags, + ClientFlagsT cmdFlags) { StorageClientSettings settings = StorageClientSettings.newBuilder() .clientName("bkctl") .serviceUri(serviceURI.getUri().toString()) @@ -66,6 +54,8 @@ protected boolean apply(ServiceURI serviceURI, return true; } catch (Exception e) { log.error("Failed to process stream admin command", e); + spec.console().println("Failed to process stream admin command"); + e.printStackTrace(spec.console()); return false; } } diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/ClientCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/ClientCommand.java index 03232a3b4c7..14ad838cdd6 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/ClientCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/ClientCommand.java @@ -17,14 +17,11 @@ */ package org.apache.bookkeeper.stream.cli.commands; -import static com.google.common.base.Preconditions.checkArgument; - import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.api.StorageClient; import org.apache.bookkeeper.clients.StorageClientBuilder; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.common.net.ServiceURI; -import org.apache.bookkeeper.tools.common.BKCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -34,21 +31,17 @@ * An admin command interface provides a run method to execute admin commands. */ @Slf4j -public abstract class ClientCommand extends BKCommand { +public abstract class ClientCommand extends AbstractStreamCommand { protected ClientCommand(CliSpec spec) { super(spec); } @Override - protected boolean apply(ServiceURI serviceURI, - CompositeConfiguration conf, - BKFlags globalFlags, - ClientFlagsT cmdFlags) { - checkArgument( - null != serviceURI, - "No service uri is provided"); - + protected boolean doApply(ServiceURI serviceURI, + CompositeConfiguration conf, + BKFlags globalFlags, + ClientFlagsT cmdFlags) { StorageClientSettings settings = StorageClientSettings.newBuilder() .clientName("bkctl") .serviceUri(serviceURI.getUri().toString()) @@ -63,6 +56,8 @@ protected boolean apply(ServiceURI serviceURI, } catch (Exception e) { log.error("Failed to process commands under namespace '{}'", globalFlags.namespace, e); + spec.console().println("Failed to process stream admin command"); + e.printStackTrace(spec.console()); return false; } } diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/cluster/InitClusterCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/cluster/InitClusterCommand.java index 2337f000d76..4ee3d399d43 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/cluster/InitClusterCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/cluster/InitClusterCommand.java @@ -19,6 +19,7 @@ package org.apache.bookkeeper.stream.cli.commands.cluster; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.stream.cli.Commands.OP_INIT; import com.beust.jcommander.Parameter; import com.google.common.base.Strings; @@ -52,7 +53,7 @@ @Slf4j public class InitClusterCommand extends BKCommand { - private static final String NAME = "init"; + private static final String NAME = OP_INIT; private static final String DESC = "Init a cluster"; /** @@ -95,6 +96,7 @@ public InitClusterCommand() { .withName(NAME) .withDescription(DESC) .withFlags(new Flags()) + .withUsage("bkctl cluster init [flags] ") .build()); } @@ -105,14 +107,16 @@ protected boolean acceptServiceUri(ServiceURI serviceURI) { } @Override - protected boolean apply(ServiceURI serviceURI, + protected boolean apply(ServiceURI ignored, CompositeConfiguration conf, BKFlags globalFlags, Flags cmdFlags) { checkArgument( - null != serviceURI, + !cmdFlags.arguments.isEmpty(), "No service URI is provided"); + ServiceURI serviceURI = ServiceURI.create(cmdFlags.arguments.get(0)); + if (null != cmdFlags.clusterName) { checkArgument( !cmdFlags.clusterName.contains("/"), diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java index 6f5e43537f2..2c572092b25 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java @@ -19,9 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_CREATE; import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.DEFAULT_STREAM_CONF; import org.apache.bookkeeper.clients.admin.StorageAdminClient; +import org.apache.bookkeeper.clients.exceptions.NamespaceExistsException; import org.apache.bookkeeper.stream.cli.commands.AdminCommand; import org.apache.bookkeeper.stream.cli.commands.namespace.CreateNamespaceCommand.Flags; import org.apache.bookkeeper.stream.proto.NamespaceConfiguration; @@ -35,7 +37,7 @@ */ public class CreateNamespaceCommand extends AdminCommand { - private static final String NAME = "create"; + private static final String NAME = OP_CREATE; private static final String DESC = "Create a namespace"; /** @@ -62,15 +64,18 @@ protected void run(StorageAdminClient admin, String namespaceName = flags.arguments.get(0); - spec.console().println("Creating namespace '" + namespaceName + "' ..."); - NamespaceProperties nsProps = result( - admin.createNamespace( - namespaceName, - NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build())); - spec.console().println("Successfully created namespace '" + namespaceName + "':"); - spec.console().println(nsProps); + try { + NamespaceProperties nsProps = result( + admin.createNamespace( + namespaceName, + NamespaceConfiguration.newBuilder() + .setDefaultStreamConf(DEFAULT_STREAM_CONF) + .build())); + spec.console().println("Successfully created namespace '" + namespaceName + "':"); + spec.console().println(nsProps); + } catch (NamespaceExistsException nee) { + spec.console().println("Namespace '" + namespaceName + "' already exists"); + } } } diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/DeleteNamespaceCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/DeleteNamespaceCommand.java new file mode 100644 index 00000000000..cc106559508 --- /dev/null +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/DeleteNamespaceCommand.java @@ -0,0 +1,74 @@ +/* + * 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.stream.cli.commands.namespace; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_DELETE; + +import org.apache.bookkeeper.clients.admin.StorageAdminClient; +import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException; +import org.apache.bookkeeper.stream.cli.commands.AdminCommand; +import org.apache.bookkeeper.stream.cli.commands.namespace.DeleteNamespaceCommand.Flags; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Command to get a namespace. + */ +public class DeleteNamespaceCommand extends AdminCommand { + + private static final String NAME = OP_DELETE; + private static final String DESC = "Delete a namespace"; + + /** + * Flags for the get namespace command. + */ + public static class Flags extends CliFlags { + } + + public DeleteNamespaceCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new Flags()) + .withArgumentsUsage("") + .build()); + } + + @Override + protected void run(StorageAdminClient admin, + BKFlags globalFlags, + Flags cmdFlags) throws Exception { + checkArgument(!cmdFlags.arguments.isEmpty(), + "Namespace name is not provided"); + + String namespaceName = cmdFlags.arguments.get(0); + try { + result( + admin.deleteNamespace(namespaceName)); + spec.console().println("Successfully deleted namespace '" + namespaceName + "'"); + } catch (NamespaceNotFoundException nfe) { + spec.console().println("Namespace '" + namespaceName + "' does not exist"); + } + } + +} diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/GetNamespaceCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/GetNamespaceCommand.java new file mode 100644 index 00000000000..b8342f5b1c9 --- /dev/null +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/GetNamespaceCommand.java @@ -0,0 +1,76 @@ +/* + * 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.stream.cli.commands.namespace; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_GET; + +import org.apache.bookkeeper.clients.admin.StorageAdminClient; +import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException; +import org.apache.bookkeeper.stream.cli.commands.AdminCommand; +import org.apache.bookkeeper.stream.cli.commands.namespace.GetNamespaceCommand.Flags; +import org.apache.bookkeeper.stream.proto.NamespaceProperties; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Command to get a namespace. + */ +public class GetNamespaceCommand extends AdminCommand { + + private static final String NAME = OP_GET; + private static final String DESC = "Get the details of a namespace"; + + /** + * Flags for the get namespace command. + */ + public static class Flags extends CliFlags { + } + + public GetNamespaceCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new Flags()) + .withArgumentsUsage("") + .build()); + } + + @Override + protected void run(StorageAdminClient admin, + BKFlags globalFlags, + Flags cmdFlags) throws Exception { + checkArgument(!cmdFlags.arguments.isEmpty(), + "Namespace name is not provided"); + + String namespaceName = cmdFlags.arguments.get(0); + try { + NamespaceProperties ns = result( + admin.getNamespace(namespaceName)); + spec.console().println("Namespace '" + namespaceName + "' :"); + spec.console().println(ns); + } catch (NamespaceNotFoundException nfe) { + spec.console().println("Namespace '" + namespaceName + "' does not exist"); + } + } + +} diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java index 5b50d1fa73e..2979b35e561 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java @@ -19,9 +19,13 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_CREATE; import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.DEFAULT_STREAM_CONF; +import com.beust.jcommander.Parameter; import org.apache.bookkeeper.clients.admin.StorageAdminClient; +import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException; +import org.apache.bookkeeper.clients.exceptions.StreamExistsException; import org.apache.bookkeeper.stream.cli.commands.AdminCommand; import org.apache.bookkeeper.stream.cli.commands.table.CreateTableCommand.Flags; import org.apache.bookkeeper.stream.proto.StorageType; @@ -36,13 +40,22 @@ */ public class CreateTableCommand extends AdminCommand { - private static final String NAME = "create"; - private static final String DESC = "Create a stream"; + private static final String NAME = OP_CREATE; + private static final String DESC = "Create a table"; /** * Flags for the create table command. */ public static class Flags extends CliFlags { + + @Parameter( + names = { + "-r", "--num-ranges" + }, + description = "num of ranges for the table" + ) + private int numRanges = 3; + } public CreateTableCommand() { @@ -50,7 +63,7 @@ public CreateTableCommand() { .withName(NAME) .withDescription(DESC) .withFlags(new Flags()) - .withArgumentsUsage("") + .withArgumentsUsage("") .build()); } @@ -59,20 +72,31 @@ protected void run(StorageAdminClient admin, BKFlags globalFlags, Flags flags) throws Exception { checkArgument(!flags.arguments.isEmpty(), - "Stream name is not provided"); + "Table name is not provided"); + checkArgument(flags.numRanges >= 1, + "Invalid number of ranges specified for creating table : " + flags.numRanges); String streamName = flags.arguments.get(0); - spec.console().println("Creating stream '" + streamName + "' ..."); - StreamProperties nsProps = result( - admin.createStream( - globalFlags.namespace, - streamName, - StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.TABLE) - .build())); - spec.console().println("Successfully created stream '" + streamName + "':"); - spec.console().println(nsProps); + try { + StreamConfiguration conf = StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) + .setMinNumRanges(flags.numRanges) + .setInitialNumRanges(flags.numRanges) + .build(); + StreamProperties nsProps = result( + admin.createStream( + globalFlags.namespace, + streamName, + StreamConfiguration.newBuilder(conf) + .setStorageType(StorageType.TABLE) + .build())); + spec.console().println("Successfully created table '" + streamName + "':"); + spec.console().println(nsProps); + } catch (NamespaceNotFoundException nfe) { + spec.console().println("Namespace '" + globalFlags.namespace + "' does not exist"); + } catch (StreamExistsException see) { + spec.console().println("Table '" + globalFlags.namespace + "' already exists"); + } } } diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DelCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DelCommand.java new file mode 100644 index 00000000000..44d17d4de0c --- /dev/null +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DelCommand.java @@ -0,0 +1,78 @@ +/* + * 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.stream.cli.commands.table; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_DEL; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.apache.bookkeeper.api.StorageClient; +import org.apache.bookkeeper.api.kv.Table; +import org.apache.bookkeeper.stream.cli.commands.ClientCommand; +import org.apache.bookkeeper.stream.cli.commands.table.DelCommand.Flags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Commands to put kvs. + */ +public class DelCommand extends ClientCommand { + + private static final String NAME = OP_DEL; + private static final String DESC = "Put key/value pair to a table"; + + /** + * Flags of the put command. + */ + public static class Flags extends CliFlags { + } + + public DelCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new Flags()) + .withArgumentsUsage(" ") + .build()); + } + + @Override + protected void run(StorageClient client, Flags flags) throws Exception { + checkArgument(flags.arguments.size() >= 2, + "table and key/value are not provided"); + + String tableName = flags.arguments.get(0); + String key = flags.arguments.get(1); + + try (Table table = result(client.openTable(tableName))) { + ByteBuf value = result(table.delete( + Unpooled.wrappedBuffer(key.getBytes(UTF_8)))); + if (null != value) { + value.release(); + spec.console().println("Successfully deleted key: ('" + key + "')."); + } else { + spec.console().println("key '" + key + "' doesn't exist."); + } + } + } + +} diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DeleteTableCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DeleteTableCommand.java new file mode 100644 index 00000000000..3022ca20bb4 --- /dev/null +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DeleteTableCommand.java @@ -0,0 +1,80 @@ +/* + * 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.stream.cli.commands.table; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_DELETE; + +import org.apache.bookkeeper.clients.admin.StorageAdminClient; +import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException; +import org.apache.bookkeeper.clients.exceptions.StreamNotFoundException; +import org.apache.bookkeeper.stream.cli.commands.AdminCommand; +import org.apache.bookkeeper.stream.cli.commands.table.DeleteTableCommand.Flags; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Command to create a namespace. + */ +public class DeleteTableCommand extends AdminCommand { + + private static final String NAME = OP_DELETE; + private static final String DESC = "Delete a table"; + + /** + * Flags for the create table command. + */ + public static class Flags extends CliFlags { + } + + public DeleteTableCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new Flags()) + .withArgumentsUsage("") + .build()); + } + + @Override + protected void run(StorageAdminClient admin, + BKFlags globalFlags, + Flags flags) throws Exception { + checkArgument(!flags.arguments.isEmpty(), + "Table name is not provided"); + + String streamName = flags.arguments.get(0); + + try { + result( + admin.deleteStream( + globalFlags.namespace, + streamName)); + spec.console().println("Successfully deleted table '" + streamName + + "' from namespace '" + globalFlags.namespace + "'"); + } catch (NamespaceNotFoundException nfe) { + spec.console().println("Namespace '" + globalFlags.namespace + "' does not exist"); + } catch (StreamNotFoundException snfe) { + spec.console().println("Table '" + streamName + "' does not exist"); + } + } + +} diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetCommand.java index b7090fcaa4f..093b669f246 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetCommand.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_GET; import com.beust.jcommander.Parameter; import io.netty.buffer.ByteBuf; @@ -38,7 +39,7 @@ */ public class GetCommand extends ClientCommand { - private static final String NAME = "get"; + private static final String NAME = OP_GET; private static final String DESC = "Get key/value pair from a table"; /** diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetTableCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetTableCommand.java new file mode 100644 index 00000000000..bc4b461fc09 --- /dev/null +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetTableCommand.java @@ -0,0 +1,87 @@ +/* + * 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.stream.cli.commands.table; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_GET; + +import org.apache.bookkeeper.clients.admin.StorageAdminClient; +import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException; +import org.apache.bookkeeper.clients.exceptions.StreamNotFoundException; +import org.apache.bookkeeper.stream.cli.commands.AdminCommand; +import org.apache.bookkeeper.stream.cli.commands.table.GetTableCommand.Flags; +import org.apache.bookkeeper.stream.proto.StorageType; +import org.apache.bookkeeper.stream.proto.StreamProperties; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Command to create a namespace. + */ +public class GetTableCommand extends AdminCommand { + + private static final String NAME = OP_GET; + private static final String DESC = "Get the details of a table"; + + /** + * Flags for the create table command. + */ + public static class Flags extends CliFlags { + } + + public GetTableCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new Flags()) + .withArgumentsUsage("") + .build()); + } + + @Override + protected void run(StorageAdminClient admin, + BKFlags globalFlags, + Flags flags) throws Exception { + checkArgument(!flags.arguments.isEmpty(), + "Table name is not provided"); + + String streamName = flags.arguments.get(0); + + try { + StreamProperties streamProps = result( + admin.getStream( + globalFlags.namespace, + streamName)); + if (streamProps.getStreamConf().getStorageType() != StorageType.TABLE) { + spec.console().println("'" + streamName + "' is not a table storage entity, its storage type is " + + streamProps.getStreamConf().getStorageType()); + } else { + spec.console().println("Table '" + streamName + "' :"); + } + spec.console().println(streamProps); + } catch (NamespaceNotFoundException nfe) { + spec.console().println("Namespace '" + globalFlags.namespace + "' does not exist"); + } catch (StreamNotFoundException snfe) { + spec.console().println("Table '" + streamName + "' does not exist"); + } + } + +} diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/IncrementCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/IncrementCommand.java index 0e8f2e536ec..25a2fb50b84 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/IncrementCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/IncrementCommand.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_INC; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -35,7 +36,7 @@ */ public class IncrementCommand extends ClientCommand { - private static final String NAME = "inc"; + private static final String NAME = OP_INC; private static final String DESC = "Increment the amount of a key in a table"; /** diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java index 5c89c664a88..b43f91efbef 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.apache.bookkeeper.stream.cli.Commands.OP_PUT; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -35,7 +36,7 @@ */ public class PutCommand extends ClientCommand { - private static final String NAME = "put"; + private static final String NAME = OP_PUT; private static final String DESC = "Put key/value pair to a table"; /** From 6947f917dd5d81062385882b03d2dc6ac78a2064 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 3 Oct 2018 08:30:42 -0700 Subject: [PATCH 0074/1642] [TABLE SERVICE] [STATS] enable stats for grpc calls on both client and server Descriptions of the changes in this PR: *Motivation* We need visibility on both client and server on table service. *Changes* - introduce grpc monitoring interceptors for both client and server - enable monitoring interceptors on both client and server Author: Sijie Guo Reviewers: Enrico Olivelli , Jia Zhai This closes #1731 from sijie/grpc_stats --- bookkeeper-benchmark/pom.xml | 11 +- .../bookkeeper/test/TestStatsProvider.java | 6 +- metadata-drivers/etcd/pom.xml | 7 + .../clients/config/StorageClientSettings.java | 8 + .../impl/channel/StorageServerChannel.java | 24 +- stream/common/pom.xml | 7 + .../common/grpc/stats/ClientStats.java | 126 +++++++ .../grpc/stats/MonitoringClientCall.java | 52 +++ .../stats/MonitoringClientCallListener.java | 69 ++++ .../stats/MonitoringClientInterceptor.java | 84 +++++ .../grpc/stats/MonitoringServerCall.java | 59 ++++ .../stats/MonitoringServerCallListener.java | 54 +++ .../stats/MonitoringServerInterceptor.java | 88 +++++ .../common/grpc/stats/ServerStats.java | 126 +++++++ .../common/grpc/stats/package-info.java | 23 ++ .../common/grpc/stats/ClientStatsTest.java | 163 +++++++++ .../grpc/stats/GrpcStatsIntegrationTest.java | 324 ++++++++++++++++++ .../common/grpc/stats/ServerStatsTest.java | 163 +++++++++ .../stream/server/grpc/GrpcServer.java | 16 +- 19 files changed, 1397 insertions(+), 13 deletions(-) rename {bookkeeper-server => bookkeeper-common}/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java (97%) create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/ClientStats.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientCall.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientCallListener.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientInterceptor.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerCall.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerCallListener.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerInterceptor.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/ServerStats.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/package-info.java create mode 100644 stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/ClientStatsTest.java create mode 100644 stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/GrpcStatsIntegrationTest.java create mode 100644 stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/ServerStatsTest.java diff --git a/bookkeeper-benchmark/pom.xml b/bookkeeper-benchmark/pom.xml index 163c1bd5f26..9b86140719b 100644 --- a/bookkeeper-benchmark/pom.xml +++ b/bookkeeper-benchmark/pom.xml @@ -52,13 +52,20 @@ org.apache.bookkeeper bookkeeper-server - ${project.parent.version} + ${project.version} compile + + org.apache.bookkeeper + bookkeeper-common + ${project.version} + test-jar + test + org.apache.bookkeeper bookkeeper-server - ${project.parent.version} + ${project.version} test-jar test diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java similarity index 97% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java rename to bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java index 9b9fdb54dd6..aca154ec0c2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java @@ -1,5 +1,4 @@ /* - * * 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 @@ -16,7 +15,6 @@ * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. - * */ package org.apache.bookkeeper.test; @@ -103,12 +101,12 @@ public class TestOpStatsLogger implements OpStatsLogger { @Override public void registerFailedEvent(long eventLatency, TimeUnit unit) { - registerFailedValue(unit.convert(eventLatency, TimeUnit.NANOSECONDS)); + registerFailedValue(TimeUnit.NANOSECONDS.convert(eventLatency, unit)); } @Override public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) { - registerSuccessfulValue(unit.convert(eventLatency, TimeUnit.NANOSECONDS)); + registerSuccessfulValue(TimeUnit.NANOSECONDS.convert(eventLatency, unit)); } @Override diff --git a/metadata-drivers/etcd/pom.xml b/metadata-drivers/etcd/pom.xml index 9e463bd89b8..0d9e0dd164d 100644 --- a/metadata-drivers/etcd/pom.xml +++ b/metadata-drivers/etcd/pom.xml @@ -54,6 +54,13 @@ testcontainers test + + org.apache.bookkeeper + bookkeeper-common + ${project.version} + test-jar + test + org.apache.bookkeeper bookkeeper-server diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/config/StorageClientSettings.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/config/StorageClientSettings.java index e78ca73e835..b5541becbe6 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/config/StorageClientSettings.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/config/StorageClientSettings.java @@ -23,6 +23,7 @@ import org.apache.bookkeeper.clients.utils.ClientConstants; import org.apache.bookkeeper.common.net.ServiceURI; import org.apache.bookkeeper.common.util.Backoff; +import org.apache.bookkeeper.stats.StatsLogger; import org.inferred.freebuilder.FreeBuilder; /** @@ -72,6 +73,13 @@ public interface StorageClientSettings { */ Optional clientName(); + /** + * Configure a stats logger to collect stats exposed by this client. + * + * @return stats logger. + */ + Optional statsLogger(); + /** * Configure a backoff policy for the client. * diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannel.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannel.java index e5206de6a3e..afacce18b72 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannel.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannel.java @@ -31,6 +31,7 @@ import org.apache.bookkeeper.clients.impl.container.StorageContainerClientInterceptor; import org.apache.bookkeeper.clients.resolver.EndpointResolver; import org.apache.bookkeeper.clients.utils.GrpcUtils; +import org.apache.bookkeeper.common.grpc.stats.MonitoringClientInterceptor; import org.apache.bookkeeper.stream.proto.common.Endpoint; import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc; import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.TableServiceFutureStub; @@ -49,11 +50,24 @@ public class StorageServerChannel implements AutoCloseable { public static Function factory(StorageClientSettings settings) { - return (endpoint) -> new StorageServerChannel( - endpoint, - Optional.empty(), - settings.usePlaintext(), - settings.endpointResolver()); + return new Function() { + + private final Optional interceptor = + settings.statsLogger().map(statsLogger -> + MonitoringClientInterceptor.create(statsLogger, true)); + + @Override + public StorageServerChannel apply(Endpoint endpoint) { + StorageServerChannel channel = new StorageServerChannel( + endpoint, + Optional.empty(), + settings.usePlaintext(), + settings.endpointResolver()); + return interceptor + .map(interceptor -> channel.intercept(interceptor)) + .orElse(channel); + } + }; } private final Optional token; diff --git a/stream/common/pom.xml b/stream/common/pom.xml index 40c8b07dde8..c5d0339fe70 100644 --- a/stream/common/pom.xml +++ b/stream/common/pom.xml @@ -47,6 +47,13 @@ true + + org.apache.bookkeeper + bookkeeper-common + ${project.version} + test-jar + test + org.apache.bookkeeper.tests stream-storage-tests-common diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/ClientStats.java b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/ClientStats.java new file mode 100644 index 00000000000..2b7ba216484 --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/ClientStats.java @@ -0,0 +1,126 @@ +/* + * 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.common.grpc.stats; + +import io.grpc.MethodDescriptor; +import io.grpc.MethodDescriptor.MethodType; +import io.grpc.Status.Code; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; + +/** + * Client side monitoring for grpc services. + */ +class ClientStats { + + private final Counter rpcStarted; + private final Counter rpcCompleted; + private final Counter streamMessagesReceived; + private final Counter streamMessagesSent; + private final Optional completedLatencyMicros; + + private ClientStats(StatsLogger rootStatsLogger, + boolean includeLatencyHistograms, + boolean streamRequests, + boolean streamResponses) { + this.rpcStarted = rootStatsLogger.getCounter("grpc_started"); + this.rpcCompleted = rootStatsLogger.getCounter("grpc_completed"); + if (streamResponses) { + this.streamMessagesReceived = rootStatsLogger.getCounter("grpc_msg_received"); + } else { + this.streamMessagesReceived = NullStatsLogger.INSTANCE.getCounter("grpc_msg_received"); + } + if (streamRequests) { + this.streamMessagesSent = rootStatsLogger.getCounter("grpc_msg_sent"); + } else { + this.streamMessagesSent = NullStatsLogger.INSTANCE.getCounter("grpc_msg_sent"); + } + if (includeLatencyHistograms) { + this.completedLatencyMicros = Optional.of( + rootStatsLogger.getOpStatsLogger("grpc_latency_micros") + ); + } else { + this.completedLatencyMicros = Optional.empty(); + } + } + + public void recordCallStarted() { + rpcStarted.inc(); + } + + public void recordClientHandled(Code code) { + rpcCompleted.inc(); + } + + public void recordStreamMessageSent() { + streamMessagesSent.inc(); + } + + public void recordStreamMessageReceived() { + streamMessagesReceived.inc(); + } + + public boolean shouldRecordLatency() { + return completedLatencyMicros.isPresent(); + } + + public void recordLatency(boolean success, long latencyMicros) { + completedLatencyMicros.ifPresent(latencyLogger -> { + if (success) { + latencyLogger.registerSuccessfulEvent(latencyMicros, TimeUnit.MICROSECONDS); + } else { + latencyLogger.registerFailedEvent(latencyMicros, TimeUnit.MICROSECONDS); + } + }); + } + + /** + * Knows how to produce {@link ClientStats} instances for individual methods. + */ + static class Factory { + + private final boolean includeLatencyHistograms; + + Factory(boolean includeLatencyHistograms) { + this.includeLatencyHistograms = includeLatencyHistograms; + } + + /** Creates a {@link ClientStats} for the supplied method. */ + ClientStats createMetricsForMethod(MethodDescriptor methodDescriptor, + StatsLogger statsLogger) { + + String fullMethodName = methodDescriptor.getFullMethodName(); + String serviceName = MethodDescriptor.extractFullServiceName(fullMethodName); + String methodName = fullMethodName.substring(serviceName.length() + 1); + + MethodType type = methodDescriptor.getType(); + return new ClientStats( + statsLogger.scope(methodName), + includeLatencyHistograms, + type == MethodType.CLIENT_STREAMING || type == MethodType.BIDI_STREAMING, + type == MethodType.SERVER_STREAMING || type == MethodType.BIDI_STREAMING); + } + } + +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientCall.java b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientCall.java new file mode 100644 index 00000000000..763ca98cb8b --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientCall.java @@ -0,0 +1,52 @@ +/* + * 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.common.grpc.stats; + +import io.grpc.ClientCall; +import io.grpc.ForwardingClientCall.SimpleForwardingClientCall; +import io.grpc.Metadata; + +/** + * A {@link SimpleForwardingClientCall} which increments counters for rpc calls. + */ +class MonitoringClientCall extends SimpleForwardingClientCall { + + private final ClientStats stats; + + MonitoringClientCall(ClientCall delegate, + ClientStats stats) { + super(delegate); + this.stats = stats; + } + + @Override + public void start(Listener responseListener, Metadata headers) { + stats.recordCallStarted(); + super.start(new MonitoringClientCallListener<>( + responseListener, stats + ), headers); + } + + @Override + public void sendMessage(ReqT message) { + stats.recordStreamMessageSent(); + super.sendMessage(message); + } +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientCallListener.java b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientCallListener.java new file mode 100644 index 00000000000..651439e01ae --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientCallListener.java @@ -0,0 +1,69 @@ +/* + * 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.common.grpc.stats; + +import io.grpc.ClientCall.Listener; +import io.grpc.ForwardingClientCallListener; +import io.grpc.Metadata; +import io.grpc.Status; +import org.apache.bookkeeper.common.util.MathUtils; + +/** + * A {@link ForwardingClientCallListener} that monitors stats on grpc clients. + */ +class MonitoringClientCallListener extends ForwardingClientCallListener { + + private final Listener delegate; + private final ClientStats stats; + private final long startNanos; + + MonitoringClientCallListener(Listener delegate, + ClientStats stats) { + this.delegate = delegate; + this.stats = stats; + this.startNanos = MathUtils.nowInNano(); + } + + @Override + protected Listener delegate() { + return delegate; + } + + @Override + public void onMessage(RespT message) { + stats.recordStreamMessageReceived(); + super.onMessage(message); + } + + @Override + public void onClose(Status status, Metadata trailers) { + stats.recordClientHandled(status.getCode()); + if (stats.shouldRecordLatency()) { + long latencyMicros = MathUtils.elapsedMicroSec(startNanos); + stats.recordLatency(Status.OK == status, latencyMicros); + } + super.onClose(status, trailers); + } + + @Override + public String toString() { + return delegate.toString(); + } +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientInterceptor.java b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientInterceptor.java new file mode 100644 index 00000000000..868f249ccfe --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringClientInterceptor.java @@ -0,0 +1,84 @@ +/* + * 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.common.grpc.stats; + +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.ClientCall; +import io.grpc.ClientInterceptor; +import io.grpc.MethodDescriptor; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.bookkeeper.common.grpc.stats.ClientStats.Factory; +import org.apache.bookkeeper.stats.StatsLogger; + +/** + * A {@link ClientInterceptor} that sends stats about grpc calls to stats logger. + */ +public class MonitoringClientInterceptor implements ClientInterceptor { + + /** + * Create a monitoring client interceptor with provided stats logger and configuration. + * + * @param statsLogger stats logger to collect grpc stats + * @param includeLatencyHistograms flag indicates whether to include latency histograms or not + * @return a monitoring client interceptor + */ + public static MonitoringClientInterceptor create(StatsLogger statsLogger, + boolean includeLatencyHistograms) { + return new MonitoringClientInterceptor( + new Factory(includeLatencyHistograms), statsLogger); + } + + private final Factory statsFactory; + private final StatsLogger statsLogger; + private final ConcurrentMap methods; + + private MonitoringClientInterceptor(Factory statsFactory, + StatsLogger statsLogger) { + this.statsFactory = statsFactory; + this.statsLogger = statsLogger; + this.methods = new ConcurrentHashMap<>(); + } + + private ClientStats getMethodStats(MethodDescriptor method) { + ClientStats stats = methods.get(method.getFullMethodName()); + if (null != stats) { + return stats; + } + ClientStats newStats = statsFactory.createMetricsForMethod(method, statsLogger); + ClientStats oldStats = methods.putIfAbsent(method.getFullMethodName(), newStats); + if (null != oldStats) { + return oldStats; + } else { + return newStats; + } + } + + @Override + public ClientCall interceptCall( + MethodDescriptor method, CallOptions callOptions, Channel next) { + ClientStats stats = getMethodStats(method); + return new MonitoringClientCall<>( + next.newCall(method, callOptions), + stats + ); + } +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerCall.java b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerCall.java new file mode 100644 index 00000000000..abb0969b07d --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerCall.java @@ -0,0 +1,59 @@ +/* + * 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.common.grpc.stats; + +import io.grpc.ForwardingServerCall.SimpleForwardingServerCall; +import io.grpc.Metadata; +import io.grpc.ServerCall; +import io.grpc.Status; +import org.apache.bookkeeper.common.util.MathUtils; + +/** + * A {@link SimpleForwardingServerCall} which increments counters for rpc calls. + */ +class MonitoringServerCall extends SimpleForwardingServerCall { + + private final ServerStats stats; + private final long startNanos; + + MonitoringServerCall(ServerCall delegate, + ServerStats stats) { + super(delegate); + this.stats = stats; + this.startNanos = MathUtils.nowInNano(); + stats.recordCallStarted(); + } + + @Override + public void sendMessage(RespT message) { + stats.recordStreamMessageSent(); + super.sendMessage(message); + } + + @Override + public void close(Status status, Metadata trailers) { + stats.recordServerHandled(status.getCode()); + if (stats.shouldRecordLatency()) { + long latencyMicros = MathUtils.elapsedMicroSec(startNanos); + stats.recordLatency(Status.OK == status, latencyMicros); + } + super.close(status, trailers); + } +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerCallListener.java b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerCallListener.java new file mode 100644 index 00000000000..6eef140c200 --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerCallListener.java @@ -0,0 +1,54 @@ +/* + * 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.common.grpc.stats; + +import io.grpc.ForwardingServerCallListener; +import io.grpc.ServerCall.Listener; + +/** + * A {@link ForwardingServerCallListener} that monitors stats on grpc clients. + */ +class MonitoringServerCallListener extends ForwardingServerCallListener { + + private final Listener delegate; + private final ServerStats stats; + + MonitoringServerCallListener(Listener delegate, + ServerStats stats) { + this.delegate = delegate; + this.stats = stats; + } + + @Override + protected Listener delegate() { + return delegate; + } + + @Override + public void onMessage(RespT message) { + stats.recordStreamMessageReceived(); + super.onMessage(message); + } + + @Override + public String toString() { + return delegate.toString(); + } +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerInterceptor.java b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerInterceptor.java new file mode 100644 index 00000000000..1c3ed255291 --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/MonitoringServerInterceptor.java @@ -0,0 +1,88 @@ +/* + * 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.common.grpc.stats; + +import io.grpc.Metadata; +import io.grpc.MethodDescriptor; +import io.grpc.ServerCall; +import io.grpc.ServerCall.Listener; +import io.grpc.ServerCallHandler; +import io.grpc.ServerInterceptor; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.bookkeeper.common.grpc.stats.ServerStats.Factory; +import org.apache.bookkeeper.stats.StatsLogger; + +/** + * A {@link ServerInterceptor} that sends stats about grpc calls to stats logger. + */ +public class MonitoringServerInterceptor implements ServerInterceptor { + + /** + * Create a monitoring client interceptor with provided stats logger and configuration. + * + * @param statsLogger stats logger to collect grpc stats + * @param includeLatencyHistograms flag indicates whether to include latency histograms or not + * @return a monitoring client interceptor + */ + public static MonitoringServerInterceptor create(StatsLogger statsLogger, + boolean includeLatencyHistograms) { + return new MonitoringServerInterceptor( + new Factory(includeLatencyHistograms), statsLogger); + } + + private final Factory statsFactory; + private final StatsLogger statsLogger; + private final ConcurrentMap methods; + + private MonitoringServerInterceptor(Factory statsFactory, + StatsLogger statsLogger) { + this.statsFactory = statsFactory; + this.statsLogger = statsLogger; + this.methods = new ConcurrentHashMap<>(); + } + + private ServerStats getMethodStats(MethodDescriptor method) { + ServerStats stats = methods.get(method.getFullMethodName()); + if (null != stats) { + return stats; + } + ServerStats newStats = statsFactory.createMetricsForMethod(method, statsLogger); + ServerStats oldStats = methods.putIfAbsent(method.getFullMethodName(), newStats); + if (null != oldStats) { + return oldStats; + } else { + return newStats; + } + } + + + @Override + public Listener interceptCall(ServerCall call, + Metadata headers, + ServerCallHandler next) { + MethodDescriptor method = call.getMethodDescriptor(); + ServerStats stats = getMethodStats(method); + ServerCall monitoringCall = new MonitoringServerCall<>(call, stats); + return new MonitoringServerCallListener<>( + next.startCall(monitoringCall, headers), stats + ); + } +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/ServerStats.java b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/ServerStats.java new file mode 100644 index 00000000000..eae8348eae5 --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/ServerStats.java @@ -0,0 +1,126 @@ +/* + * 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.common.grpc.stats; + +import io.grpc.MethodDescriptor; +import io.grpc.MethodDescriptor.MethodType; +import io.grpc.Status.Code; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; + +/** + * Client side monitoring for grpc services. + */ +class ServerStats { + + private final Counter rpcStarted; + private final Counter rpcCompleted; + private final Counter streamMessagesReceived; + private final Counter streamMessagesSent; + private final Optional completedLatencyMicros; + + private ServerStats(StatsLogger rootStatsLogger, + boolean includeLatencyHistograms, + boolean streamRequests, + boolean streamResponses) { + this.rpcStarted = rootStatsLogger.getCounter("grpc_started"); + this.rpcCompleted = rootStatsLogger.getCounter("grpc_completed"); + if (streamRequests) { + this.streamMessagesReceived = rootStatsLogger.getCounter("grpc_msg_received"); + } else { + this.streamMessagesReceived = NullStatsLogger.INSTANCE.getCounter("grpc_msg_received"); + } + if (streamResponses) { + this.streamMessagesSent = rootStatsLogger.getCounter("grpc_msg_sent"); + } else { + this.streamMessagesSent = NullStatsLogger.INSTANCE.getCounter("grpc_msg_sent"); + } + if (includeLatencyHistograms) { + this.completedLatencyMicros = Optional.of( + rootStatsLogger.getOpStatsLogger("grpc_latency_micros") + ); + } else { + this.completedLatencyMicros = Optional.empty(); + } + } + + public void recordCallStarted() { + rpcStarted.inc(); + } + + public void recordServerHandled(Code code) { + rpcCompleted.inc(); + } + + public void recordStreamMessageSent() { + streamMessagesSent.inc(); + } + + public void recordStreamMessageReceived() { + streamMessagesReceived.inc(); + } + + public boolean shouldRecordLatency() { + return completedLatencyMicros.isPresent(); + } + + public void recordLatency(boolean success, long latencyMicros) { + completedLatencyMicros.ifPresent(latencyLogger -> { + if (success) { + latencyLogger.registerSuccessfulEvent(latencyMicros, TimeUnit.MICROSECONDS); + } else { + latencyLogger.registerFailedEvent(latencyMicros, TimeUnit.MICROSECONDS); + } + }); + } + + /** + * Knows how to produce {@link ServerStats} instances for individual methods. + */ + static class Factory { + + private final boolean includeLatencyHistograms; + + Factory(boolean includeLatencyHistograms) { + this.includeLatencyHistograms = includeLatencyHistograms; + } + + /** Creates a {@link ServerStats} for the supplied method. */ + ServerStats createMetricsForMethod(MethodDescriptor methodDescriptor, + StatsLogger statsLogger) { + + String fullMethodName = methodDescriptor.getFullMethodName(); + String serviceName = MethodDescriptor.extractFullServiceName(fullMethodName); + String methodName = fullMethodName.substring(serviceName.length() + 1); + + MethodType type = methodDescriptor.getType(); + return new ServerStats( + statsLogger.scope(methodName), + includeLatencyHistograms, + type == MethodType.CLIENT_STREAMING || type == MethodType.BIDI_STREAMING, + type == MethodType.SERVER_STREAMING || type == MethodType.BIDI_STREAMING); + } + } + +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/package-info.java b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/stats/package-info.java new file mode 100644 index 00000000000..a3e2d3ee6eb --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/grpc/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. + */ + +/** + * Collecting grpc related stats. + */ +package org.apache.bookkeeper.common.grpc.stats; \ No newline at end of file diff --git a/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/ClientStatsTest.java b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/ClientStatsTest.java new file mode 100644 index 00000000000..d5cf2d356a9 --- /dev/null +++ b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/ClientStatsTest.java @@ -0,0 +1,163 @@ +/* + * 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.common.grpc.stats; + +import static org.junit.Assert.assertEquals; + +import io.grpc.MethodDescriptor; +import io.grpc.Status; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.common.grpc.stats.ClientStats.Factory; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; +import org.bookkeeper.tests.proto.rpc.PingPongServiceGrpc; +import org.junit.Before; +import org.junit.Test; + +/** + * Unit test {@link ClientStats}. + */ +public class ClientStatsTest { + + private Factory factoryWithHistograms; + private Factory factoryWithoutHistograms; + private TestStatsProvider statsProvider; + + @Before + public void setup() { + this.statsProvider = new TestStatsProvider(); + this.factoryWithHistograms = new Factory(true); + this.factoryWithoutHistograms = new Factory(false); + } + + @Test + public void testClientStatsWithHistogram() { + testClientStats(factoryWithHistograms, true); + } + + @Test + public void testClientStatsWithoutHistogram() { + testClientStats(factoryWithoutHistograms, false); + } + + private void testClientStats(Factory clientStatsFactory, + boolean includeLatencyHistogram) { + // test unary method + MethodDescriptor unaryMethod = PingPongServiceGrpc.getPingPongMethod(); + testClientStats( + clientStatsFactory, + includeLatencyHistogram, + unaryMethod, + "PingPong", + "unary", + 1, + 1, + 0, + 0 + ); + // test client streaming + MethodDescriptor clientStreamingMethod = PingPongServiceGrpc.getLotsOfPingsMethod(); + testClientStats( + clientStatsFactory, + includeLatencyHistogram, + clientStreamingMethod, + "LotsOfPings", + "client_streaming", + 1, + 1, + 1, + 0 + ); + // test server streaming + MethodDescriptor serverStreamingMethod = PingPongServiceGrpc.getLotsOfPongsMethod(); + testClientStats( + clientStatsFactory, + includeLatencyHistogram, + serverStreamingMethod, + "LotsOfPongs", + "server_streaming", + 1, + 1, + 0, + 2 + ); + // test server streaming + MethodDescriptor biStreamingMethod = PingPongServiceGrpc.getBidiPingPongMethod(); + testClientStats( + clientStatsFactory, + includeLatencyHistogram, + biStreamingMethod, + "BidiPingPong", + "bidi_streaming", + 1, + 1, + 1, + 2 + ); + } + + private void testClientStats(Factory clientStatsFactory, + boolean includeLatencyHistogram, + MethodDescriptor method, + String methodName, + String statsScope, + long expectedCallStarted, + long expectedCallCompleted, + long expectedStreamMsgsSent, + long expectedStreamMsgsReceived) { + StatsLogger statsLogger = statsProvider.getStatsLogger(statsScope); + ClientStats unaryStats = clientStatsFactory.createMetricsForMethod( + method, + statsLogger + ); + unaryStats.recordCallStarted(); + assertEquals( + expectedCallStarted, + statsLogger.scope(methodName).getCounter("grpc_started").get().longValue()); + unaryStats.recordClientHandled(Status.OK.getCode()); + assertEquals( + expectedCallCompleted, + statsLogger.scope(methodName).getCounter("grpc_completed").get().longValue()); + unaryStats.recordStreamMessageSent(); + assertEquals( + expectedStreamMsgsSent, + statsLogger.scope(methodName).getCounter("grpc_msg_sent").get().longValue()); + unaryStats.recordStreamMessageReceived(); + unaryStats.recordStreamMessageReceived(); + assertEquals( + expectedStreamMsgsReceived, + statsLogger.scope(methodName).getCounter("grpc_msg_received").get().longValue()); + long latencyMicros = 12345L; + unaryStats.recordLatency(true, latencyMicros); + TestOpStatsLogger opStatsLogger = + (TestOpStatsLogger) statsLogger.scope(methodName).getOpStatsLogger("grpc_latency_micros"); + if (includeLatencyHistogram) { + assertEquals(1, opStatsLogger.getSuccessCount()); + assertEquals( + TimeUnit.MICROSECONDS.toNanos(latencyMicros), + (long) opStatsLogger.getSuccessAverage()); + } else { + assertEquals(0, opStatsLogger.getSuccessCount()); + assertEquals(0, (long) opStatsLogger.getSuccessAverage()); + } + } + +} diff --git a/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/GrpcStatsIntegrationTest.java b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/GrpcStatsIntegrationTest.java new file mode 100644 index 00000000000..e34f30e1d2d --- /dev/null +++ b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/GrpcStatsIntegrationTest.java @@ -0,0 +1,324 @@ +/* + * 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.common.grpc.stats; + +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import io.grpc.Channel; +import io.grpc.ClientInterceptors; +import io.grpc.ManagedChannel; +import io.grpc.Server; +import io.grpc.ServerInterceptors; +import io.grpc.ServerServiceDefinition; +import io.grpc.inprocess.InProcessChannelBuilder; +import io.grpc.inprocess.InProcessServerBuilder; +import io.grpc.stub.StreamObserver; +import io.grpc.util.MutableHandlerRegistry; +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; +import org.apache.bookkeeper.tests.rpc.PingPongService; +import org.bookkeeper.tests.proto.rpc.PingPongServiceGrpc; +import org.bookkeeper.tests.proto.rpc.PingPongServiceGrpc.PingPongServiceBlockingStub; +import org.bookkeeper.tests.proto.rpc.PingPongServiceGrpc.PingPongServiceStub; +import org.bookkeeper.tests.proto.rpc.PingRequest; +import org.bookkeeper.tests.proto.rpc.PongResponse; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * End-to-end integration test on grpc stats. + */ +public class GrpcStatsIntegrationTest { + + private static final int NUM_PONGS_PER_PING = 10; + private static final String SERVICE_NAME = "pingpong"; + + private Server server; + private PingPongService service; + private ManagedChannel channel; + private Channel monitoredChannel; + private PingPongServiceBlockingStub client; + private PingPongServiceStub clientNonBlocking; + private TestStatsProvider statsProvider; + private TestStatsLogger clientStatsLogger; + private TestStatsLogger serverStatsLogger; + + + @Before + public void setup() throws Exception { + statsProvider = new TestStatsProvider(); + clientStatsLogger = statsProvider.getStatsLogger("client"); + serverStatsLogger = statsProvider.getStatsLogger("server"); + service = new PingPongService(NUM_PONGS_PER_PING); + ServerServiceDefinition monitoredService = ServerInterceptors.intercept( + service, + MonitoringServerInterceptor.create(serverStatsLogger, true) + ); + MutableHandlerRegistry registry = new MutableHandlerRegistry(); + server = InProcessServerBuilder + .forName(SERVICE_NAME) + .fallbackHandlerRegistry(registry) + .directExecutor() + .build() + .start(); + registry.addService(monitoredService); + + channel = InProcessChannelBuilder.forName(SERVICE_NAME) + .usePlaintext() + .build(); + monitoredChannel = ClientInterceptors.intercept( + channel, + MonitoringClientInterceptor.create(clientStatsLogger, true) + ); + client = PingPongServiceGrpc.newBlockingStub(monitoredChannel); + clientNonBlocking = PingPongServiceGrpc.newStub(monitoredChannel); + } + + @After + public void teardown() { + if (null != channel) { + channel.shutdown(); + } + if (null != server) { + server.shutdown(); + } + } + + private void assertStats(String methodName, + long numCalls, + long numClientMsgSent, + long numClientMsgReceived, + long numServerMsgSent, + long numServerMsgReceived) { + // client stats + assertEquals( + numCalls, + clientStatsLogger.scope(methodName).getCounter("grpc_started").get().longValue() + ); + assertEquals( + numCalls, + clientStatsLogger.scope(methodName).getCounter("grpc_completed").get().longValue() + ); + assertEquals( + numClientMsgSent, + clientStatsLogger.scope(methodName).getCounter("grpc_msg_sent").get().longValue() + ); + assertEquals( + numClientMsgReceived, + clientStatsLogger.scope(methodName).getCounter("grpc_msg_received").get().longValue() + ); + TestOpStatsLogger opStatsLogger = + (TestOpStatsLogger) clientStatsLogger.scope(methodName).getOpStatsLogger("grpc_latency_micros"); + assertEquals( + numCalls, + opStatsLogger.getSuccessCount() + ); + // server stats + assertEquals( + numCalls, + serverStatsLogger.scope(methodName).getCounter("grpc_started").get().longValue() + ); + assertEquals( + numCalls, + serverStatsLogger.scope(methodName).getCounter("grpc_completed").get().longValue() + ); + assertEquals( + numServerMsgSent, + serverStatsLogger.scope(methodName).getCounter("grpc_msg_sent").get().longValue() + ); + assertEquals( + numServerMsgReceived, + serverStatsLogger.scope(methodName).getCounter("grpc_msg_received").get().longValue() + ); + opStatsLogger = + (TestOpStatsLogger) serverStatsLogger.scope(methodName).getOpStatsLogger("grpc_latency_micros"); + assertEquals( + numCalls, + opStatsLogger.getSuccessCount() + ); + } + + @Test + public void testUnary() { + long sequence = ThreadLocalRandom.current().nextLong(); + PingRequest request = PingRequest.newBuilder() + .setSequence(sequence) + .build(); + PongResponse response = client.pingPong(request); + assertEquals(sequence, response.getLastSequence()); + assertEquals(1, response.getNumPingReceived()); + assertEquals(0, response.getSlotId()); + + // verify the stats + assertStats( + "PingPong", + 1, + 0, + 0, + 0, + 0); + } + + @Test + public void testServerStreaming() { + long sequence = ThreadLocalRandom.current().nextLong(100000); + PingRequest request = PingRequest.newBuilder() + .setSequence(sequence) + .build(); + Iterator respIter = client.lotsOfPongs(request); + int count = 0; + while (respIter.hasNext()) { + PongResponse resp = respIter.next(); + assertEquals(sequence, resp.getLastSequence()); + assertEquals(1, resp.getNumPingReceived()); + assertEquals(count, resp.getSlotId()); + ++count; + } + + assertStats( + "LotsOfPongs", + 1, + 0, + NUM_PONGS_PER_PING, + NUM_PONGS_PER_PING, + 0); + } + + @Test + public void testClientStreaming() throws Exception { + final int numPings = 100; + final long sequence = ThreadLocalRandom.current().nextLong(100000); + final CompletableFuture respFuture = new CompletableFuture<>(); + final LinkedBlockingQueue respQueue = new LinkedBlockingQueue<>(); + StreamObserver pinger = clientNonBlocking.lotsOfPings(new StreamObserver() { + @Override + public void onNext(PongResponse resp) { + respQueue.offer(resp); + } + + @Override + public void onError(Throwable t) { + respFuture.completeExceptionally(t); + } + + @Override + public void onCompleted() { + FutureUtils.complete(respFuture, null); + } + }); + + for (int i = 0; i < numPings; i++) { + PingRequest request = PingRequest.newBuilder() + .setSequence(sequence + i) + .build(); + pinger.onNext(request); + } + pinger.onCompleted(); + + // wait for response to be received. + result(respFuture); + + assertEquals(1, respQueue.size()); + + PongResponse resp = respQueue.take(); + assertEquals(sequence + numPings - 1, resp.getLastSequence()); + assertEquals(numPings, resp.getNumPingReceived()); + assertEquals(0, resp.getSlotId()); + + assertStats( + "LotsOfPings", + 1, + numPings, + 0, + 0, + numPings + ); + } + + @Test + public void testBidiStreaming() throws Exception { + final int numPings = 100; + + final CompletableFuture respFuture = new CompletableFuture<>(); + final LinkedBlockingQueue respQueue = new LinkedBlockingQueue<>(); + StreamObserver pinger = clientNonBlocking.bidiPingPong(new StreamObserver() { + @Override + public void onNext(PongResponse resp) { + respQueue.offer(resp); + } + + @Override + public void onError(Throwable t) { + respFuture.completeExceptionally(t); + } + + @Override + public void onCompleted() { + FutureUtils.complete(respFuture, null); + } + }); + + final LinkedBlockingQueue reqQueue = new LinkedBlockingQueue<>(); + for (int i = 0; i < numPings; i++) { + final long sequence = ThreadLocalRandom.current().nextLong(100000); + PingRequest request = PingRequest.newBuilder() + .setSequence(sequence) + .build(); + reqQueue.put(request); + pinger.onNext(request); + } + pinger.onCompleted(); + + // wait for response to be received + result(respFuture); + + assertEquals(numPings, respQueue.size()); + + int count = 0; + for (PingRequest request : reqQueue) { + PongResponse response = respQueue.take(); + + assertEquals(request.getSequence(), response.getLastSequence()); + assertEquals(++count, response.getNumPingReceived()); + assertEquals(0, response.getSlotId()); + } + assertNull(respQueue.poll()); + assertEquals(numPings, count); + + assertStats( + "BidiPingPong", + 1, + numPings, + numPings, + numPings, + numPings + ); + } + +} diff --git a/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/ServerStatsTest.java b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/ServerStatsTest.java new file mode 100644 index 00000000000..8300aa6327e --- /dev/null +++ b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/ServerStatsTest.java @@ -0,0 +1,163 @@ +/* + * 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.common.grpc.stats; + +import static org.junit.Assert.assertEquals; + +import io.grpc.MethodDescriptor; +import io.grpc.Status; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.common.grpc.stats.ServerStats.Factory; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; +import org.bookkeeper.tests.proto.rpc.PingPongServiceGrpc; +import org.junit.Before; +import org.junit.Test; + +/** + * Unit test {@link ServerStats}. + */ +public class ServerStatsTest { + + private Factory factoryWithHistograms; + private Factory factoryWithoutHistograms; + private TestStatsProvider statsProvider; + + @Before + public void setup() { + this.statsProvider = new TestStatsProvider(); + this.factoryWithHistograms = new Factory(true); + this.factoryWithoutHistograms = new Factory(false); + } + + @Test + public void testServerStatsWithHistogram() { + testServerStats(factoryWithHistograms, true); + } + + @Test + public void testServerStatsWithoutHistogram() { + testServerStats(factoryWithoutHistograms, false); + } + + private void testServerStats(Factory clientStatsFactory, + boolean includeLatencyHistogram) { + // test unary method + MethodDescriptor unaryMethod = PingPongServiceGrpc.getPingPongMethod(); + testServerStats( + clientStatsFactory, + includeLatencyHistogram, + unaryMethod, + "PingPong", + "unary", + 1, + 1, + 0, + 0 + ); + // test client streaming + MethodDescriptor clientStreamingMethod = PingPongServiceGrpc.getLotsOfPingsMethod(); + testServerStats( + clientStatsFactory, + includeLatencyHistogram, + clientStreamingMethod, + "LotsOfPings", + "client_streaming", + 1, + 1, + 0, + 2 + ); + // test server streaming + MethodDescriptor serverStreamingMethod = PingPongServiceGrpc.getLotsOfPongsMethod(); + testServerStats( + clientStatsFactory, + includeLatencyHistogram, + serverStreamingMethod, + "LotsOfPongs", + "server_streaming", + 1, + 1, + 1, + 0 + ); + // test server streaming + MethodDescriptor biStreamingMethod = PingPongServiceGrpc.getBidiPingPongMethod(); + testServerStats( + clientStatsFactory, + includeLatencyHistogram, + biStreamingMethod, + "BidiPingPong", + "bidi_streaming", + 1, + 1, + 1, + 2 + ); + } + + private void testServerStats(Factory clientStatsFactory, + boolean includeLatencyHistogram, + MethodDescriptor method, + String methodName, + String statsScope, + long expectedCallStarted, + long expectedCallCompleted, + long expectedStreamMsgsSent, + long expectedStreamMsgsReceived) { + StatsLogger statsLogger = statsProvider.getStatsLogger(statsScope); + ServerStats unaryStats = clientStatsFactory.createMetricsForMethod( + method, + statsLogger + ); + unaryStats.recordCallStarted(); + assertEquals( + expectedCallStarted, + statsLogger.scope(methodName).getCounter("grpc_started").get().longValue()); + unaryStats.recordServerHandled(Status.OK.getCode()); + assertEquals( + expectedCallCompleted, + statsLogger.scope(methodName).getCounter("grpc_completed").get().longValue()); + unaryStats.recordStreamMessageSent(); + assertEquals( + expectedStreamMsgsSent, + statsLogger.scope(methodName).getCounter("grpc_msg_sent").get().longValue()); + unaryStats.recordStreamMessageReceived(); + unaryStats.recordStreamMessageReceived(); + assertEquals( + expectedStreamMsgsReceived, + statsLogger.scope(methodName).getCounter("grpc_msg_received").get().longValue()); + long latencyMicros = 12345L; + unaryStats.recordLatency(true, latencyMicros); + TestOpStatsLogger opStatsLogger = + (TestOpStatsLogger) statsLogger.scope(methodName).getOpStatsLogger("grpc_latency_micros"); + if (includeLatencyHistogram) { + assertEquals(1, opStatsLogger.getSuccessCount()); + assertEquals( + TimeUnit.MICROSECONDS.toNanos(latencyMicros), + (long) opStatsLogger.getSuccessAverage()); + } else { + assertEquals(0, opStatsLogger.getSuccessCount()); + assertEquals(0, (long) opStatsLogger.getSuccessAverage()); + } + } + +} diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcServer.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcServer.java index 7a5f16306ef..1ef26423a5e 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcServer.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcServer.java @@ -18,12 +18,14 @@ import io.grpc.HandlerRegistry; import io.grpc.Server; import io.grpc.ServerBuilder; +import io.grpc.ServerInterceptors; import io.grpc.ServerServiceDefinition; import io.grpc.inprocess.InProcessServerBuilder; import java.io.IOException; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.common.grpc.proxy.ProxyHandlerRegistry; +import org.apache.bookkeeper.common.grpc.stats.MonitoringServerInterceptor; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stream.proto.common.Endpoint; import org.apache.bookkeeper.stream.server.conf.StorageServerConfiguration; @@ -75,13 +77,23 @@ public GrpcServer(StorageContainerStore storageContainerStore, } this.grpcServer = serverBuilder.build(); } else { + MonitoringServerInterceptor monitoringInterceptor = + MonitoringServerInterceptor.create(statsLogger.scope("services"), true); ProxyHandlerRegistry.Builder proxyRegistryBuilder = ProxyHandlerRegistry.newBuilder() .setChannelFinder(storageContainerStore); for (ServerServiceDefinition definition : GrpcServices.create(null)) { - proxyRegistryBuilder = proxyRegistryBuilder.addService(definition); + ServerServiceDefinition monitoredService = ServerInterceptors.intercept( + definition, + monitoringInterceptor + ); + proxyRegistryBuilder = proxyRegistryBuilder.addService(monitoredService); } + ServerServiceDefinition locationService = ServerInterceptors.intercept( + new GrpcStorageContainerService(storageContainerStore), + monitoringInterceptor + ); this.grpcServer = ServerBuilder.forPort(this.myEndpoint.getPort()) - .addService(new GrpcStorageContainerService(storageContainerStore)) + .addService(locationService) .fallbackHandlerRegistry(proxyRegistryBuilder.build()) .build(); } From 615604d328581609dda6a83adabf5552900fe8a1 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 4 Oct 2018 02:16:47 -0700 Subject: [PATCH 0075/1642] Introduce lifecycle components for managing components in AutoRecovery Descriptions of the changes in this PR: - lifecycle components for managing components in AutoRecovery - expose metrics of AR in the same http admin endpoint Author: cguttapalem Reviewers: Enrico Olivelli , Sijie Guo This closes #1726 from reddycharan/stackar --- bookkeeper-server/pom.xml | 12 ++ .../replication/AutoRecoveryMain.java | 154 ++++++++++++------ .../org/apache/bookkeeper/server/Main.java | 2 +- .../server/service/AutoRecoveryService.java | 11 ++ .../bookie/BookieInitializationTest.java | 130 +++++++++++++++ 5 files changed, 261 insertions(+), 48 deletions(-) diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index e154a36ebf1..e093d8051b0 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -150,6 +150,18 @@ test-jar test + + org.apache.bookkeeper.stats + prometheus-metrics-provider + ${project.parent.version} + test + + + org.apache.bookkeeper.http + vertx-http-server + ${project.parent.version} + test + 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 d63e19eac31..cd4aee2ce0d 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 @@ -27,19 +27,26 @@ import java.io.File; import java.io.IOException; +import java.lang.Thread.UncaughtExceptionHandler; import java.net.MalformedURLException; +import java.util.concurrent.ExecutionException; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieCriticalThread; import org.apache.bookkeeper.bookie.ExitCode; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.common.component.ComponentStarter; +import org.apache.bookkeeper.common.component.LifecycleComponent; +import org.apache.bookkeeper.common.component.LifecycleComponentStack; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.http.HttpServer; -import org.apache.bookkeeper.http.HttpServerLoader; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.server.conf.BookieConfiguration; import org.apache.bookkeeper.server.http.BKHttpServiceProvider; +import org.apache.bookkeeper.server.service.AutoRecoveryService; +import org.apache.bookkeeper.server.service.HttpService; +import org.apache.bookkeeper.server.service.StatsProviderService; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.cli.BasicParser; @@ -70,6 +77,9 @@ public class AutoRecoveryMain { private volatile boolean shuttingDown = false; private volatile boolean running = false; + // Exception handler + private volatile UncaughtExceptionHandler uncaughtExceptionHandler = null; + public AutoRecoveryMain(ServerConfiguration conf) throws IOException, InterruptedException, KeeperException, UnavailableException, CompatibilityException { @@ -102,6 +112,9 @@ public AutoRecoveryMain(ServerConfiguration conf, StatsLogger statsLogger) public void start() throws UnavailableException { auditorElector.start(); replicationWorker.start(); + if (null != uncaughtExceptionHandler) { + deathWatcher.setUncaughtExceptionHandler(uncaughtExceptionHandler); + } deathWatcher.start(); running = true; } @@ -129,13 +142,6 @@ private void shutdown(int exitCode) { shuttingDown = true; running = false; this.exitCode = exitCode; - try { - deathWatcher.interrupt(); - deathWatcher.join(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.warn("Interrupted shutting down auto recovery", e); - } try { auditorElector.shutdown(); @@ -158,6 +164,18 @@ private int getExitCode() { return exitCode; } + /** + * Currently the uncaught exception handler is used for DeathWatcher to notify + * lifecycle management that a bookie is dead for some reasons. + * + *

in future, we can register this exceptionHandler to critical threads + * so when those threads are dead, it will automatically trigger lifecycle management + * to shutdown the process. + */ + public void setExceptionHandler(UncaughtExceptionHandler exceptionHandler) { + this.uncaughtExceptionHandler = exceptionHandler; + } + @VisibleForTesting public Auditor getAuditor() { return auditorElector.getAuditor(); @@ -171,7 +189,7 @@ public boolean isAutoRecoveryRunning() { /* * DeathWatcher for AutoRecovery daemons. */ - private static class AutoRecoveryDeathWatcher extends BookieCriticalThread { + private class AutoRecoveryDeathWatcher extends BookieCriticalThread { private int watchInterval; private AutoRecoveryMain autoRecoveryMain; @@ -180,6 +198,13 @@ public AutoRecoveryDeathWatcher(AutoRecoveryMain autoRecoveryMain) { + autoRecoveryMain.conf.getBookiePort()); this.autoRecoveryMain = autoRecoveryMain; watchInterval = autoRecoveryMain.conf.getDeathWatchInterval(); + // set a default uncaught exception handler to shutdown the AutoRecovery + // when it notices the AutoRecovery is not running any more. + setUncaughtExceptionHandler((thread, cause) -> { + LOG.info("AutoRecoveryDeathWatcher exited loop due to uncaught exception from thread {}", + thread.getName(), cause); + shutdown(); + }); } @Override @@ -189,13 +214,20 @@ public void run() { Thread.sleep(watchInterval); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); - break; } // If any one service not running, then shutdown peer. - if (!autoRecoveryMain.auditorElector.isRunning() - || !autoRecoveryMain.replicationWorker.isRunning()) { - autoRecoveryMain.shutdown(ExitCode.SERVER_EXCEPTION); - break; + if (!autoRecoveryMain.auditorElector.isRunning() || !autoRecoveryMain.replicationWorker.isRunning()) { + LOG.info( + "AutoRecoveryDeathWatcher noticed the AutoRecovery is not running any more," + + "exiting the watch loop!"); + /* + * death watcher has noticed that AutoRecovery is not + * running any more throw an exception to fail the death + * watcher thread and it will trigger the uncaught exception + * handler to handle this "AutoRecovery not running" + * situation. + */ + throw new RuntimeException("AutoRecovery is not running any more"); } } } @@ -266,45 +298,73 @@ private static ServerConfiguration parseArgs(String[] args) } public static void main(String[] args) { - ServerConfiguration conf = null; + int retCode = doMain(args); + Runtime.getRuntime().exit(retCode); + } + + static int doMain(String[] args) { + + ServerConfiguration conf; + + // 0. parse command line try { conf = parseArgs(args); } catch (IllegalArgumentException iae) { - LOG.error("Error parsing command line arguments : ", iae); - System.err.println(iae.getMessage()); - printUsage(); - System.exit(ExitCode.INVALID_CONF); + return ExitCode.INVALID_CONF; } + // 1. building the component stack: + LifecycleComponent server; try { - final AutoRecoveryMain autoRecoveryMain = new AutoRecoveryMain(conf); - autoRecoveryMain.start(); - HttpServerLoader.loadHttpServer(conf); - final HttpServer httpServer = HttpServerLoader.get(); - if (conf.isHttpServerEnabled() && httpServer != null) { - BKHttpServiceProvider serviceProvider = new BKHttpServiceProvider.Builder() - .setAutoRecovery(autoRecoveryMain) - .setServerConfiguration(conf) - .build(); - httpServer.initialize(serviceProvider); - httpServer.startServer(conf.getHttpServerPort()); - } - Runtime.getRuntime().addShutdownHook(new Thread() { - @Override - public void run() { - autoRecoveryMain.shutdown(); - if (httpServer != null && httpServer.isRunning()) { - httpServer.stopServer(); - } - LOG.info("Shutdown AutoRecoveryMain successfully"); - } - }); - LOG.info("Register shutdown hook successfully"); - autoRecoveryMain.join(); - System.exit(autoRecoveryMain.getExitCode()); + server = buildAutoRecoveryServer(new BookieConfiguration(conf)); } catch (Exception e) { - LOG.error("Exception running AutoRecoveryMain : ", e); - System.exit(ExitCode.SERVER_EXCEPTION); + LOG.error("Failed to build AutoRecovery Server", e); + return ExitCode.SERVER_EXCEPTION; + } + + // 2. start the server + try { + ComponentStarter.startComponent(server).get(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + // the server is interrupted + LOG.info("AutoRecovery server is interrupted. Exiting ..."); + } catch (ExecutionException ee) { + LOG.error("Error in bookie shutdown", ee.getCause()); + return ExitCode.SERVER_EXCEPTION; } + return ExitCode.OK; + } + + public static LifecycleComponentStack buildAutoRecoveryServer(BookieConfiguration conf) throws Exception { + LifecycleComponentStack.Builder serverBuilder = LifecycleComponentStack.newBuilder() + .withName("autorecovery-server"); + + // 1. build stats provider + StatsProviderService statsProviderService = new StatsProviderService(conf); + StatsLogger rootStatsLogger = statsProviderService.getStatsProvider().getStatsLogger(""); + + serverBuilder.addComponent(statsProviderService); + LOG.info("Load lifecycle component : {}", StatsProviderService.class.getName()); + + // 2. build AutoRecovery server + AutoRecoveryService autoRecoveryService = new AutoRecoveryService(conf, rootStatsLogger); + + serverBuilder.addComponent(autoRecoveryService); + LOG.info("Load lifecycle component : {}", AutoRecoveryService.class.getName()); + + // 4. build http service + if (conf.getServerConf().isHttpServerEnabled()) { + BKHttpServiceProvider provider = new BKHttpServiceProvider.Builder() + .setAutoRecovery(autoRecoveryService.getAutoRecoveryServer()) + .setServerConfiguration(conf.getServerConf()) + .setStatsProvider(statsProviderService.getStatsProvider()).build(); + HttpService httpService = new HttpService(provider, conf, rootStatsLogger); + + serverBuilder.addComponent(httpService); + LOG.info("Load lifecycle component : {}", HttpService.class.getName()); + } + + return serverBuilder.build(); } } 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 ae92955a326..b991d31b896 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 @@ -284,7 +284,7 @@ private static ServerConfiguration parseCommandLine(String[] args) * @param conf bookie server configuration * @return lifecycle stack */ - static LifecycleComponentStack buildBookieServer(BookieConfiguration conf) throws Exception { + public static LifecycleComponentStack buildBookieServer(BookieConfiguration conf) throws Exception { LifecycleComponentStack.Builder serverBuilder = LifecycleComponentStack.newBuilder().withName("bookie-server"); // 1. build stats provider diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java index b2b8f07d0c9..f8389df69d0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java @@ -19,6 +19,8 @@ package org.apache.bookkeeper.server.service; import java.io.IOException; +import java.lang.Thread.UncaughtExceptionHandler; + import org.apache.bookkeeper.replication.AutoRecoveryMain; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.server.component.ServerLifecycleComponent; @@ -41,6 +43,15 @@ public AutoRecoveryService(BookieConfiguration conf, StatsLogger statsLogger) th statsLogger); } + @Override + public void setExceptionHandler(UncaughtExceptionHandler handler) { + main.setExceptionHandler(handler); + } + + public AutoRecoveryMain getAutoRecoveryServer() { + return main; + } + @Override protected void doStart() { try { 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 48ea817f5ec..6500e0b5446 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 @@ -33,16 +33,23 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.BufferedReader; import java.io.BufferedWriter; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.io.InputStreamReader; import java.io.OutputStreamWriter; import java.net.BindException; import java.net.InetAddress; +import java.net.URL; +import java.net.URLConnection; import java.security.AccessControlException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.concurrent.CompletableFuture; @@ -55,20 +62,29 @@ import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.component.ComponentStarter; +import org.apache.bookkeeper.common.component.Lifecycle; +import org.apache.bookkeeper.common.component.LifecycleComponent; 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.http.HttpRouter; +import org.apache.bookkeeper.http.HttpServerLoader; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.replication.AutoRecoveryMain; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.replication.ReplicationStats; +import org.apache.bookkeeper.server.Main; import org.apache.bookkeeper.server.conf.BookieConfiguration; +import org.apache.bookkeeper.server.service.AutoRecoveryService; import org.apache.bookkeeper.server.service.BookieService; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.tls.SecurityException; @@ -91,6 +107,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase { private static final Logger LOG = LoggerFactory .getLogger(BookieInitializationTest.class); + private static ObjectMapper om = new ObjectMapper(); + @Rule public final TestName runtime = new TestName(); ZKMetadataBookieDriver driver; @@ -495,6 +513,22 @@ public void testBookieServiceExceptionHandler() throws Exception { startFuture.get(); } + @Test + public void testAutoRecoveryServiceExceptionHandler() throws Exception { + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + conf.setMetadataServiceUri(metadataServiceUri); + + BookieConfiguration bkConf = new BookieConfiguration(conf); + AutoRecoveryService service = new AutoRecoveryService(bkConf, NullStatsLogger.INSTANCE); + CompletableFuture startFuture = ComponentStarter.startComponent(service); + + // shutdown the AutoRecovery service + service.getAutoRecoveryServer().shutdown(); + + // the AutoRecovery lifecycle component should be shutdown. + startFuture.get(); + } + /** * Verify bookie server starts up on ephemeral ports. */ @@ -1072,4 +1106,100 @@ private void corruptFile(File file) throws IOException { } } + @Test + public void testIOVertexHTTPServerEndpointForBookieWithPrometheusProvider() throws Exception { + File tmpDir = createTempDir("bookie", "test"); + + final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setJournalDirName(tmpDir.getPath()).setLedgerDirNames(new String[] { tmpDir.getPath() }) + .setBookiePort(PortManager.nextFreePort()).setMetadataServiceUri(metadataServiceUri) + .setListeningInterface(null); + + /* + * enable io.vertx http server + */ + int nextFreePort = PortManager.nextFreePort(); + conf.setStatsProviderClass(PrometheusMetricsProvider.class); + conf.setHttpServerEnabled(true); + conf.setProperty(HttpServerLoader.HTTP_SERVER_CLASS, "org.apache.bookkeeper.http.vertx.VertxHttpServer"); + conf.setHttpServerPort(nextFreePort); + + // 1. building the component stack: + LifecycleComponent server = Main.buildBookieServer(new BookieConfiguration(conf)); + // 2. start the server + CompletableFuture stackComponentFuture = ComponentStarter.startComponent(server); + while (server.lifecycleState() != Lifecycle.State.STARTED) { + Thread.sleep(100); + } + + // Now, hit the rest endpoint for metrics + URL url = new URL("http://localhost:" + nextFreePort + HttpRouter.METRICS); + URLConnection urlc = url.openConnection(); + BufferedReader in = new BufferedReader(new InputStreamReader(urlc.getInputStream())); + String inputLine; + StringBuilder metricsStringBuilder = new StringBuilder(); + while ((inputLine = in.readLine()) != null) { + metricsStringBuilder.append(inputLine); + } + in.close(); + String metrics = metricsStringBuilder.toString(); + // do primitive checks if metrics string contains some stats + assertTrue("Metrics should contain basic counters", metrics.contains(BookKeeperServerStats.BOOKIE_ADD_ENTRY)); + + // Now, hit the rest endpoint for configs + url = new URL("http://localhost:" + nextFreePort + HttpRouter.SERVER_CONFIG); + @SuppressWarnings("unchecked") + Map configMap = om.readValue(url, Map.class); + if (configMap.isEmpty() || !configMap.containsKey("bookiePort")) { + Assert.fail("Failed to map configurations to valid JSON entries."); + } + stackComponentFuture.cancel(true); + } + + @Test + public void testIOVertexHTTPServerEndpointForARWithPrometheusProvider() throws Exception { + final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setMetadataServiceUri(metadataServiceUri).setListeningInterface(null); + + /* + * enable io.vertx http server + */ + int nextFreePort = PortManager.nextFreePort(); + conf.setStatsProviderClass(PrometheusMetricsProvider.class); + conf.setHttpServerEnabled(true); + conf.setProperty(HttpServerLoader.HTTP_SERVER_CLASS, "org.apache.bookkeeper.http.vertx.VertxHttpServer"); + conf.setHttpServerPort(nextFreePort); + + // 1. building the component stack: + LifecycleComponent server = AutoRecoveryMain.buildAutoRecoveryServer(new BookieConfiguration(conf)); + // 2. start the server + CompletableFuture stackComponentFuture = ComponentStarter.startComponent(server); + while (server.lifecycleState() != Lifecycle.State.STARTED) { + Thread.sleep(100); + } + + // Now, hit the rest endpoint for metrics + URL url = new URL("http://localhost:" + nextFreePort + HttpRouter.METRICS); + URLConnection urlc = url.openConnection(); + BufferedReader in = new BufferedReader(new InputStreamReader(urlc.getInputStream())); + String inputLine; + StringBuilder metricsStringBuilder = new StringBuilder(); + while ((inputLine = in.readLine()) != null) { + metricsStringBuilder.append(inputLine); + } + in.close(); + String metrics = metricsStringBuilder.toString(); + // do primitive checks if metrics string contains some stats + assertTrue("Metrics should contain basic counters", + metrics.contains(ReplicationStats.NUM_UNDER_REPLICATED_LEDGERS)); + + // Now, hit the rest endpoint for configs + url = new URL("http://localhost:" + nextFreePort + HttpRouter.SERVER_CONFIG); + @SuppressWarnings("unchecked") + Map configMap = om.readValue(url, Map.class); + if (configMap.isEmpty() || !configMap.containsKey("metadataServiceUri")) { + Assert.fail("Failed to map configurations to valid JSON entries."); + } + stackComponentFuture.cancel(true); + } } From 1d55c3699d1b2f6986ab02a86312abe5bab53e75 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 4 Oct 2018 02:20:58 -0700 Subject: [PATCH 0076/1642] [TABLE SERVICE] [CLIENT] provide scripts for uploading python clients to pypi Descriptions of the changes in this PR: *Motivation* make the python client available for usage *Changes* - update README and setup.py - set the client version to `4.9.0-alpha-0` - scripts to build python client and upload it to pypi *Result* https://pypi.org/project/apache-bookkeeper-client/4.9.0a0/ Author: Sijie Guo Reviewers: Enrico Olivelli This closes #1734 from sijie/upload_python_client --- pom.xml | 1 + stream/clients/python/.gitignore | 3 ++ stream/clients/python/README.md | 4 --- stream/clients/python/README.rst | 4 +++ stream/clients/python/scripts/publish.sh | 30 +++++++++++++++++++ .../clients/python/scripts/publish_staging.sh | 30 +++++++++++++++++++ stream/clients/python/setup.py | 8 ++--- 7 files changed, 72 insertions(+), 8 deletions(-) delete mode 100644 stream/clients/python/README.md create mode 100644 stream/clients/python/README.rst create mode 100755 stream/clients/python/scripts/publish.sh create mode 100755 stream/clients/python/scripts/publish_staging.sh diff --git a/pom.xml b/pom.xml index 418e9a4f5d3..ecac42afce7 100644 --- a/pom.xml +++ b/pom.xml @@ -854,6 +854,7 @@ **/README.md + **/README.rst **/apidocs/* **/src/main/resources/deps/** **/META-INF/** diff --git a/stream/clients/python/.gitignore b/stream/clients/python/.gitignore index a520e657a93..5d2d92bef5b 100644 --- a/stream/clients/python/.gitignore +++ b/stream/clients/python/.gitignore @@ -20,3 +20,6 @@ bookkeeper.egg-info/ # pip pip-selfcheck.json + +# egg-info +**egg-info/ diff --git a/stream/clients/python/README.md b/stream/clients/python/README.md deleted file mode 100644 index df1b7efd248..00000000000 --- a/stream/clients/python/README.md +++ /dev/null @@ -1,4 +0,0 @@ -Python Client for Apache BookKeeper -=================================== - -|pypi| |versions| diff --git a/stream/clients/python/README.rst b/stream/clients/python/README.rst new file mode 100644 index 00000000000..65afe63d770 --- /dev/null +++ b/stream/clients/python/README.rst @@ -0,0 +1,4 @@ +Python Client for Apache BookKeeper +=================================== + +Apache BookKeeper is a scalable, fault tolerant and low latency storage service optimized for append-only workloads. diff --git a/stream/clients/python/scripts/publish.sh b/stream/clients/python/scripts/publish.sh new file mode 100755 index 00000000000..caa2a3b5b4e --- /dev/null +++ b/stream/clients/python/scripts/publish.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +# +#/** +# * 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. +# */ + +# Upload python client to pypi staging + +BINDIR=`dirname "$0"` +BK_HOME=`cd ${BINDIR}/..;pwd` + +python -m pip install --user --upgrade setuptools wheel twine + +rm ${BK_HOME}/dist/* +python setup.py sdist bdist_wheel +twine upload dist/* diff --git a/stream/clients/python/scripts/publish_staging.sh b/stream/clients/python/scripts/publish_staging.sh new file mode 100755 index 00000000000..363a768e07d --- /dev/null +++ b/stream/clients/python/scripts/publish_staging.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +# +#/** +# * 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. +# */ + +# Upload python client to pypi staging + +BINDIR=`dirname "$0"` +BK_HOME=`cd ${BINDIR}/..;pwd` + +python -m pip install --user --upgrade setuptools wheel twine + +rm ${BK_HOME}/dist/* +python setup.py sdist bdist_wheel +twine upload --repository-url https://test.pypi.org/legacy/ dist/* diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index 49afd7a424c..74a8b0ca07f 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -17,9 +17,9 @@ # Package metadata. -name = 'bookkeeper' +name = 'apache-bookkeeper-client' description = 'Apache BookKeeper client library' -version = '4.9.0' +version = '4.9.0-alpha-0' # Should be one of: # 'Development Status :: 3 - Alpha' # 'Development Status :: 4 - Beta' @@ -42,7 +42,7 @@ package_root = os.path.abspath(os.path.dirname(__file__)) -readme_filename = os.path.join(package_root, 'README.md') +readme_filename = os.path.join(package_root, 'README.rst') with io.open(readme_filename, encoding='utf-8') as readme_file: readme = readme_file.read() @@ -67,7 +67,7 @@ classifiers=[ release_status, 'Intended Audience :: Developers', - 'License :: Apache Software License', + 'License :: OSI Approved :: Apache Software License', 'Programming Language :: Python', 'Programming Language :: Python :: 2', 'Programming Language :: Python :: 2.7', From 18eb1e26ead0e87561e811e230d08125c68fc69e Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Fri, 5 Oct 2018 15:31:28 -0700 Subject: [PATCH 0077/1642] Issue #1740: IOUtils.close() only accepts Closeable as a vararg which results in unnecessary Object[] created if only one Closeable passed there Descriptions of the changes in this PR: added close() variant that takes one Closeable. ### Motivation 100% of close() usage is with one Closeable. ### Changes light refactoring. Master Issue: #1740 Author: Andrey Yegorov Reviewers: Sijie Guo , Enrico Olivelli This closes #1741 from dlg99/nit/ioutils, closes #1740 --- .../org/apache/bookkeeper/util/IOUtils.java | 27 ++++++++++++++----- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/IOUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/IOUtils.java index 53ef2ac9f3d..43b6ff39c76 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/IOUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/IOUtils.java @@ -43,13 +43,26 @@ public class IOUtils { */ public static void close(Logger log, java.io.Closeable... closeables) { for (java.io.Closeable c : closeables) { - if (c != null) { - try { - c.close(); - } catch (IOException e) { - if (log != null && log.isDebugEnabled()) { - log.debug("Exception in closing " + c, e); - } + close(log, c); + } + } + + /** + * Close the Closeable object and ignore any {@link IOException} or + * null pointers. Must only be used for cleanup in exception handlers. + * + * @param log + * the log to record problems to at debug level. Can be null. + * @param closeable + * the objects to close + */ + public static void close(Logger log, java.io.Closeable closeable) { + if (closeable != null) { + try { + closeable.close(); + } catch (IOException e) { + if (log != null && log.isDebugEnabled()) { + log.debug("Exception in closing " + closeable, e); } } } From a8d7c7af2b7e100ecd71017bca7fd26097e8f511 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Fri, 5 Oct 2018 15:34:32 -0700 Subject: [PATCH 0078/1642] Issue #1738: Journal: add a metric for the time ForceWriteRequest spent in the queue ### Motivation Investigated latency spike in one of the bookie clusters. At some point the question at hands was "how much time ForceWriteRequests spend in the queue in this case". There is no metric to answer this question. ### Changes Added new metric. Master Issue: #1738 Author: Andrey Yegorov Reviewers: Sijie Guo , Enrico Olivelli This closes #1739 from dlg99/fix/forcewrite-metric, closes #1738 --- .../org/apache/bookkeeper/bookie/BookKeeperServerStats.java | 1 + .../src/main/java/org/apache/bookkeeper/bookie/Journal.java | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java index 9afd8a58f7c..e048bd732ee 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java @@ -92,6 +92,7 @@ public interface BookKeeperServerStats { String JOURNAL_ADD_ENTRY = "JOURNAL_ADD_ENTRY"; String JOURNAL_FORCE_LEDGER = "JOURNAL_FORCE_LEDGER"; String JOURNAL_SYNC = "JOURNAL_SYNC"; + String JOURNAL_FORCE_WRITE_ENQUEUE = "JOURNAL_FORCE_WRITE_ENQUEUE"; String JOURNAL_MEM_ADD_ENTRY = "JOURNAL_MEM_ADD_ENTRY"; String JOURNAL_PREALLOCATION = "JOURNAL_PREALLOCATION"; String JOURNAL_FORCE_WRITE_LATENCY = "JOURNAL_FORCE_WRITE_LATENCY"; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 1b0d7070bac..0db23c63453 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -360,9 +360,12 @@ public class ForceWriteRequest { private boolean isMarker; private long lastFlushedPosition; private long logId; + private long enqueueTime; public int process(boolean shouldForceWrite) throws IOException { forceWriteQueueSize.dec(); + fwEnqueueTimeStats.registerSuccessfulEvent(MathUtils.elapsedNanos(enqueueTime), TimeUnit.NANOSECONDS); + if (isMarker) { return 0; } @@ -434,6 +437,7 @@ private ForceWriteRequest createForceWriteRequest(JournalChannel logFile, req.lastFlushedPosition = lastFlushedPosition; req.shouldClose = shouldClose; req.isMarker = isMarker; + req.enqueueTime = MathUtils.nowInNano(); forceWriteQueueSize.inc(); return req; } @@ -613,6 +617,7 @@ static void writePaddingBytes(JournalChannel jc, ByteBuf paddingBuffer, int jour private final OpStatsLogger journalAddEntryStats; private final OpStatsLogger journalForceLedgerStats; private final OpStatsLogger journalSyncStats; + private final OpStatsLogger fwEnqueueTimeStats; private final OpStatsLogger journalCreationStats; private final OpStatsLogger journalFlushStats; private final OpStatsLogger journalProcessTimeStats; @@ -677,6 +682,7 @@ public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf journalAddEntryStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_ADD_ENTRY); journalForceLedgerStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_LEDGER); journalSyncStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_SYNC); + fwEnqueueTimeStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_WRITE_ENQUEUE); journalCreationStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_CREATION_LATENCY); journalFlushStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FLUSH_LATENCY); journalQueueStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_QUEUE_LATENCY); From 2327ecd20fad654198eea656e6c2d673656fa057 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 5 Oct 2018 15:36:00 -0700 Subject: [PATCH 0079/1642] ISSUE #1703: IllegalReferenceCountException at closing EntryLogManagerForSingleEntryLog Descriptions of the changes in this PR: *Motivation* Fixes #1703. The active entry log channel in EntryLogManagerForSingleEntryLog is closed twice during shutdown. One is by EntryLogManagerForSingleEntryLog#close and the other one is EntryLogManagerForSingleEntryLog#forceClose(). *Changes* This change is adding logic in BufferedChannel to make sure BufferedChannel can be closed multiple times. Author: Sijie Guo Reviewers: Charan Reddy Guttapalem , Enrico Olivelli , Andrey Yegorov This closes #1735 from sijie/issue_1703, closes #1703 --- .../java/org/apache/bookkeeper/bookie/BufferedChannel.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java index b2dd4be605a..633c5400dab 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java @@ -63,6 +63,8 @@ public class BufferedChannel extends BufferedReadChannel implements Closeable { */ protected final AtomicLong unpersistedBytes; + private boolean closed = false; + // make constructor to be public for unit test public BufferedChannel(FileChannel fc, int capacity) throws IOException { // Use the same capacity for read and write buffers. @@ -88,8 +90,12 @@ public BufferedChannel(FileChannel fc, int writeCapacity, int readCapacity, long @Override public synchronized void close() throws IOException { + if (closed) { + return; + } ReferenceCountUtil.safeRelease(writeBuffer); fileChannel.close(); + closed = true; } /** From ae76131bfcdbd1e41be3be6a1fbb81515d6e37e3 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 5 Oct 2018 15:38:53 -0700 Subject: [PATCH 0080/1642] [TABLE SERVICE] [TESTS] add integration tests for python clients Descriptions of the changes in this PR: *Motivation* We need have test coverage for python clients. *Changes* - add scripts to run python client tests for different python versions - make sure the client can be built on both python2 and python3 - add a postcommit job to run python tests Author: Sijie Guo Reviewers: Enrico Olivelli This closes #1742 from sijie/python_integration_tests --- ...bookkeeper_postcommit_master_python.groovy | 52 +++++++++++++ ...okkeeper_precommit_integrationtests.groovy | 11 +-- .test-infra/scripts/post-docker-tests.sh | 22 ++++++ .test-infra/scripts/pre-docker-tests.sh | 31 ++++++++ pom.xml | 1 + stream/clients/python/.gitignore | 3 + .../python/bookkeeper/common/constants.py | 3 +- .../clients/python/bookkeeper/common/util.py | 26 +++++++ stream/clients/python/bookkeeper/kv/table.py | 3 +- stream/clients/python/nox.py | 77 +++++++++++++++++++ stream/clients/python/noxfile.py | 7 +- stream/clients/python/scripts/docker_tests.sh | 49 ++++++++++++ stream/clients/python/scripts/test.sh | 25 ++++++ 13 files changed, 294 insertions(+), 16 deletions(-) create mode 100644 .test-infra/jenkins/job_bookkeeper_postcommit_master_python.groovy create mode 100755 .test-infra/scripts/post-docker-tests.sh create mode 100755 .test-infra/scripts/pre-docker-tests.sh create mode 100644 stream/clients/python/bookkeeper/common/util.py create mode 100644 stream/clients/python/nox.py create mode 100755 stream/clients/python/scripts/docker_tests.sh create mode 100755 stream/clients/python/scripts/test.sh diff --git a/.test-infra/jenkins/job_bookkeeper_postcommit_master_python.groovy b/.test-infra/jenkins/job_bookkeeper_postcommit_master_python.groovy new file mode 100644 index 00000000000..38be2e36baf --- /dev/null +++ b/.test-infra/jenkins/job_bookkeeper_postcommit_master_python.groovy @@ -0,0 +1,52 @@ +/* + * 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. + */ + +import common_job_properties + +// This job runs postcommit tests on python client +freeStyleJob('bookkeeper_postcommit_master_python') { + description('Runs nightly build for bookkeeper python client.') + + // clean up the workspace before build + wrappers { preBuildCleanup() } + + // Set common parameters. + common_job_properties.setTopLevelMainJobProperties( + delegate, 'master', 'JDK 1.8 (latest)') + + throttleConcurrentBuilds { + // limit builds to 1 per node to avoid conflicts on building docker images + maxPerNode(1) + } + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit( + delegate, + 'H 12 * * *', + false) + + steps { + shell('.test-infra/scripts/pre-docker-tests.sh') + shell('docker pull python:3.7') + shell('docker pull python:3.6') + shell('docker pull python:3.5') + shell('docker pull python:2.7') + shell('./stream/clients/python/scripts/test.sh') + shell('.test-infra/scripts/post-docker-tests.sh') + } +} diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy index f070822bee9..10958ee308e 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy @@ -47,14 +47,7 @@ freeStyleJob('bookkeeper_precommit_integrationtests') { '.*\\[x\\] \\[skip integration tests\\].*') steps { - // Temporary information gathering to see if full disks are causing the builds to flake - shell('id') - shell('ulimit -a') - shell('pwd') - shell('df -h') - shell('ps -eo euser,pid,ppid,pgid,start,pcpu,pmem,cmd') - shell('docker network prune -f --filter name=testnetwork_*') // clean up any dangling networks from previous runs - shell('docker system events > docker.log & echo $! > docker-log.pid') + shell('.test-infra/scripts/pre-docker-tests.sh') shell('docker pull apachebookkeeper/bookkeeper-all-released-versions:latest') @@ -74,7 +67,7 @@ freeStyleJob('bookkeeper_precommit_integrationtests') { goals('-B test -Dstream -DintegrationTests') } - shell('kill $(cat docker-log.pid) || true') + shell('.test-infra/scripts/post-docker-tests.sh') } publishers { diff --git a/.test-infra/scripts/post-docker-tests.sh b/.test-infra/scripts/post-docker-tests.sh new file mode 100755 index 00000000000..8af7ab8b914 --- /dev/null +++ b/.test-infra/scripts/post-docker-tests.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# +# 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. +# + +set -ex +kill $(cat docker-log.pid) || true diff --git a/.test-infra/scripts/pre-docker-tests.sh b/.test-infra/scripts/pre-docker-tests.sh new file mode 100755 index 00000000000..3e92513e1fd --- /dev/null +++ b/.test-infra/scripts/pre-docker-tests.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash +# +# 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. +# + +set -ex + +id +ulimit -a +pwd +df -h +ps -eo euser,pid,ppid,pgid,start,pcpu,pmem,cmd +docker system prune -f +# clean up any dangling networks from previous runs +docker network prune -f --filter name=testnetwork_* +docker system events > docker.debug-info & echo $! > docker-log.pid diff --git a/pom.xml b/pom.xml index ecac42afce7..af3cefa88a6 100644 --- a/pom.xml +++ b/pom.xml @@ -894,6 +894,7 @@ **/python/bin/** **/python/include/** **/python/lib/** + **/**.pyc **/.nox/** **/.pytest_cache/** **/__pycache__/** diff --git a/stream/clients/python/.gitignore b/stream/clients/python/.gitignore index 5d2d92bef5b..f01f19c4a34 100644 --- a/stream/clients/python/.gitignore +++ b/stream/clients/python/.gitignore @@ -15,6 +15,9 @@ include/ # pycache __pycache__/ +# .pyc +*.pyc + # build files bookkeeper.egg-info/ diff --git a/stream/clients/python/bookkeeper/common/constants.py b/stream/clients/python/bookkeeper/common/constants.py index deb85a93468..b21f4bcceb0 100644 --- a/stream/clients/python/bookkeeper/common/constants.py +++ b/stream/clients/python/bookkeeper/common/constants.py @@ -12,11 +12,12 @@ # See the License for the specific language governing permissions and # limitations under the License. +from bookkeeper.common import util from bookkeeper.proto import stream_pb2 __ROOT_RANGE_ID__ = 0 __ROOT_RANGE_METADATA__ = [ - ('bk-rt-sc-id-bin', __ROOT_RANGE_ID__.to_bytes(8, "big")) + ('bk-rt-sc-id-bin', util.to_bytes(__ROOT_RANGE_ID__, 8, "big")) ] __DEFAULT_STREAM_CONF__ = stream_pb2.StreamConfiguration( key_type=stream_pb2.RangeKeyType.values()[0], diff --git a/stream/clients/python/bookkeeper/common/util.py b/stream/clients/python/bookkeeper/common/util.py new file mode 100644 index 00000000000..54f11d9621d --- /dev/null +++ b/stream/clients/python/bookkeeper/common/util.py @@ -0,0 +1,26 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +import sys + +__PYTHON3__ = sys.version_info >= (3, 0) + + +def to_bytes(n, length, endianess='big'): + if __PYTHON3__: + return n.to_bytes(length, endianess) + else: + h = '%x' % n + s = ('0'*(len(h) % 2) + h).zfill(length*2).decode('hex') + return s if endianess == 'big' else s[::-1] diff --git a/stream/clients/python/bookkeeper/kv/table.py b/stream/clients/python/bookkeeper/kv/table.py index 81ac6a8ccf0..605ab5d525a 100644 --- a/stream/clients/python/bookkeeper/kv/table.py +++ b/stream/clients/python/bookkeeper/kv/table.py @@ -16,6 +16,7 @@ import logging +from bookkeeper.common import util from bookkeeper.common.exceptions import from_table_rpc_response from bookkeeper.common.method import wrap_method from bookkeeper.common.retry import Retry @@ -45,7 +46,7 @@ def __init__(self, channel, stream_props): def __make_routing_metadata__(self, key): return [ ('bk-rt-sid-bin', - self.__stream_props__.stream_id.to_bytes(8, "big")), + util.to_bytes(self.__stream_props__.stream_id, 8, "big")), ('bk-rt-key-bin', key) ] diff --git a/stream/clients/python/nox.py b/stream/clients/python/nox.py new file mode 100644 index 00000000000..5966201c4a0 --- /dev/null +++ b/stream/clients/python/nox.py @@ -0,0 +1,77 @@ +# Licensed 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. + +from __future__ import absolute_import + +import os +import nox + + +LOCAL_DEPS = ( +) + +@nox.session +def default(session): + """Default unit test session. + This is intended to be run **without** an interpreter set, so + that the current ``python`` (on the ``PATH``) or the version of + Python corresponding to the ``nox`` binary the ``PATH`` can + run the tests. + """ + # Install all test dependencies, then install local packages in-place. + session.install('mock', 'pytest', 'pytest-cov') + for local_dep in LOCAL_DEPS: + session.install('-e', local_dep) + session.install('-e', '.') + + # Run py.test against the unit tests. + session.run( + 'py.test', + '--quiet', + '--cov-append', + '--cov-report=', + '--cov=bookkeeper', + '--cov-config=.coveragerc', + os.path.join('tests', 'unit'), + *session.posargs + ) + + +@nox.session +def lint(session): + """Run linters. + Returns a failure if the linters find linting errors or sufficiently + serious code quality issues. + """ + session.install('flake8', *LOCAL_DEPS) + session.install('.') + session.run('flake8', 'bookkeeper', 'tests') + + +@nox.session +def lint_setup_py(session): + """Verify that setup.py is valid (including RST check).""" + session.install('docutils', 'Pygments') + session.run( + 'python', 'setup.py', 'check', '--restructuredtext', '--strict') + + +# TODO: Enable coverage report +# @nox.session +def cover(session): + """Run the final coverage report. + This outputs the coverage report aggregating coverage from the unit + test runs (not system test runs), and then erases coverage data. + """ + session.install('coverage', 'pytest-cov') + session.run('coverage', 'report', '--show-missing', '--fail-under=100') + session.run('coverage', 'erase') diff --git a/stream/clients/python/noxfile.py b/stream/clients/python/noxfile.py index 0d25fa2a1e1..3027c9212ca 100644 --- a/stream/clients/python/noxfile.py +++ b/stream/clients/python/noxfile.py @@ -13,7 +13,6 @@ from __future__ import absolute_import import os - import nox @@ -21,7 +20,6 @@ ) -@nox.session def default(session): """Default unit test session. This is intended to be run **without** an interpreter set, so @@ -48,9 +46,8 @@ def default(session): ) -@nox.session -@nox.parametrize('py', ['3.7']) -def unit(session, py): +@nox.session(python=[os.environ['PY_VERSION']]) +def unit(session): """Run the unit test suite.""" default(session) diff --git a/stream/clients/python/scripts/docker_tests.sh b/stream/clients/python/scripts/docker_tests.sh new file mode 100755 index 00000000000..3b91d40fb6b --- /dev/null +++ b/stream/clients/python/scripts/docker_tests.sh @@ -0,0 +1,49 @@ +#!/bin/bash + +# 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. + +set -e -x -u + +SCRIPT_DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +PY_VERSION=${PY_VERSION:-"3.7"} +NOXSESSION=${NOXSESSION:-"unit"} + +# nox only support python 3+ +if [[ ${PY_VERSION} == 3* ]]; then + TEST_COMMANDS=`cat < Date: Mon, 8 Oct 2018 10:35:25 -0700 Subject: [PATCH 0081/1642] ISSUE #1737: EntryMemTable.newEntry: always make a copy Retaining a reference to that array assumes that the caller won't reuse the array for something else -- an assumption violated by Journal.scanJournal and probably other callers. (bug W-5499346) (rev cguttapalem) Signed-off-by: Samuel Just Author: Reviewers: Enrico Olivelli , Matteo Merli , Sijie Guo This closes #1744 from athanatos/forupstream/wip-1737, closes #1737 --- .../bookkeeper/bookie/EntryMemTable.java | 9 ++--- .../bookkeeper/conf/ServerConfiguration.java | 12 +++++++ .../bookkeeper/bookie/BookieJournalTest.java | 33 +++++++++++++++++++ 3 files changed, 47 insertions(+), 7 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java index 73283989bb0..0a95fe92158 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java @@ -366,13 +366,8 @@ private EntryKeyValue newEntry(long ledgerId, long entryId, final ByteBuffer ent int offset = 0; int length = entry.remaining(); - if (entry.hasArray()) { - buf = entry.array(); - offset = entry.arrayOffset(); - } else { - buf = new byte[length]; - entry.get(buf); - } + buf = new byte[length]; + entry.get(buf); return new EntryKeyValue(ledgerId, entryId, buf, offset, length); } 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 2475084677d..e9c551e3dc4 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 @@ -1755,6 +1755,18 @@ public int getSkipListArenaMaxAllocSize() { return getInt(SKIP_LIST_MAX_ALLOC_ENTRY, 128 * 1024); } + /** + * Set the max size we should allocate from the skiplist arena. Allocations + * larger than this should be allocated directly by the VM to avoid fragmentation. + * + * @param size max alloc size. + * @return server configuration object. + */ + public ServerConfiguration setSkipListArenaMaxAllocSize(int size) { + setProperty(SKIP_LIST_MAX_ALLOC_ENTRY, size); + return this; + } + /** * Should the data be fsynced on journal before acknowledgment. * diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java index 1a0342b46e7..080ebfebb36 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java @@ -640,6 +640,39 @@ public void testTruncatedInEntryJournal() throws Exception { } } + /** + * Test journal replay with SortedLedgerStorage and a very small max + * arena size. + */ + @Test + public void testSortedLedgerStorageReplayWithSmallMaxArenaSize() throws Exception { + File journalDir = createTempDir("bookie", "journal"); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir)); + + File ledgerDir = createTempDir("bookie", "ledger"); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir)); + + JournalChannel jc = writeV2Journal( + Bookie.getCurrentDirectory(journalDir), 100); + + jc.fc.force(false); + + writeIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir), + 1, "testPasswd".getBytes()); + + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + conf.setLedgerStorageClass("org.apache.bookkeeper.bookie.SortedLedgerStorage"); + conf.setSkipListArenaMaxAllocSize(0); + conf.setJournalDirName(journalDir.getPath()) + .setLedgerDirNames(new String[] { ledgerDir.getPath() }); + + Bookie b = new Bookie(conf); + b.readJournal(); + b.ledgerStorage.flush(); + b.readEntry(1, 80); + b.readEntry(1, 99); + } + /** * Test partial index (truncate master key) with pre-v3 journals. */ From 98212a3d83154dcafba22183bf6d05d04092ab74 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Mon, 8 Oct 2018 17:54:30 -0700 Subject: [PATCH 0082/1642] ReadEntryCallback in ReadLedgerEntriesCmd shouldn't release buffer. Descriptions of the changes in this PR: - ReadEntryCallback in ReadLedgerEntriesCmd shouldn't release buffer, which is not owned by the callback. ### Motivation with the following change, Per channel bookie clients owns the buffer for read responses. So it is not correct for ReadEntryCallback in ReadLedgerEntriesCmd to release buffer https://github.com/apache/bookkeeper/commit/8d048abce486c63d428041f77ee9a506756f4d1e#diff-e50ee2c1aec1539ea185a94605b0e550R1611 because of this issue I'm seeing following error with ReadLedgerEntriesCmd ``` /Workspace/SFStorage/bookkeeper/bookkeeper-server/bin$ ./bookkeeper shell -localbookie readledger -bookie **** -ledgerid 00000000-0000-0000-0000-000000000003 -firstentryid 1 -lastentryid 3 JAVA_HOME not set, using java from PATH. (/usr/bin/java) --------- Lid=00000000-0000-0000-0000-000000000003, Eid=1 --------- 18:32:03,724 ERROR Unexpected throwable caught io.netty.util.IllegalReferenceCountException: refCnt: 0, increment: 1 at io.netty.buffer.AbstractReferenceCountedByteBuf.release0(AbstractReferenceCountedByteBuf.java:100) ~[netty-all-4.1.22.Final.jar:4.1.22.Final] at io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:84) ~[netty-all-4.1.22.Final.jar:4.1.22.Final] at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV3Response(PerChannelBookieClient.java:1699) ~[bookkeeper-server-4.7.0-SNAPSHOT.jar:4.7.0-SNAPSHOT] at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1286) ~[bookkeeper-server-4.7.0-SNAPSHOT.jar:4.7.0-SNAPSHOT] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [bookkeeper-common-4.7.0-SNAPSHOT.jar:4.7.0-SNAPSHOT] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_172] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_172] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [netty-all-4.1.22.Final.jar:4.1.22.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_172] --------- Lid=00000000-0000-0000-0000-000000000003, Eid=2 --------- 18:32:03,733 ERROR Unexpected throwable caught io.netty.util.IllegalReferenceCountException: refCnt: 0, increment: 1 at io.netty.buffer.AbstractReferenceCountedByteBuf.release0(AbstractReferenceCountedByteBuf.java:100) ~[netty-all-4.1.22.Final.jar:4.1.22.Final] at io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:84) ~[netty-all-4.1.22.Final.jar:4.1.22.Final] at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV3Response(PerChannelBookieClient.java:1699) ~[bookkeeper-server-4.7.0-SNAPSHOT.jar:4.7.0-SNAPSHOT] at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1286) ~[bookkeeper-server-4.7.0-SNAPSHOT.jar:4.7.0-SNAPSHOT] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [bookkeeper-common-4.7.0-SNAPSHOT.jar:4.7.0-SNAPSHOT] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_172] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_172] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [netty-all-4.1.22.Final.jar:4.1.22.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_172] ``` Author: cguttapalem Reviewers: Sijie Guo , Enrico Olivelli , Andrey Yegorov This closes #1736 from reddycharan/readledgerfix --- .../src/main/java/org/apache/bookkeeper/bookie/BookieShell.java | 1 - 1 file changed, 1 deletion(-) 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 a6e0f44bc43..2829a730d3e 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 @@ -854,7 +854,6 @@ int runCmd(CommandLine cmdLine) throws Exception { System.out.println("Data: " + ByteBufUtil.prettyHexDump(buffer)); } - buffer.release(); future.complete(null); }, null, BookieProtocol.FLAG_NONE); From c26dc17ef706014fb021f531c8b8a0a0299dca7a Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 8 Oct 2018 18:02:35 -0700 Subject: [PATCH 0083/1642] [TABLE SERVICE] [CLIENT] storage client can open tables on a different namespace Descriptions of the changes in this PR: *Motivation* In some cases, we are using same storage client for opening tables under different namespaces. It would be good that a client instance can do that. *Changes* - Add methods to `StorageClient` to open tables under different namespaces - Add `asClient` to `StorageAdminClient` to convert admin client to storage client Author: Sijie Guo Reviewers: Enrico Olivelli , Jia Zhai This closes #1733 from sijie/improve_client_interface --- .../apache/bookkeeper/api/StorageClient.java | 32 ++++++ .../bookkeeper/clients/SimpleClientBase.java | 28 +++-- .../clients/SimpleStorageClientImpl.java | 46 ++++++-- .../clients/StorageClientBuilder.java | 2 +- .../bookkeeper/clients/StorageClientImpl.java | 64 ++++++++--- .../admin/SimpleStorageAdminClientImpl.java | 12 +++ .../clients/admin/StorageAdminClient.java | 65 +++++++++++ .../clients/admin/StorageAdminClientImpl.java | 38 +++---- .../clients/StorageClientImplTest.java | 101 +++++++++++++++++- .../admin/TestStorageAdminClientImpl.java | 9 +- 10 files changed, 341 insertions(+), 56 deletions(-) diff --git a/stream/api/src/main/java/org/apache/bookkeeper/api/StorageClient.java b/stream/api/src/main/java/org/apache/bookkeeper/api/StorageClient.java index 969a8bbfe56..419a8289e3d 100644 --- a/stream/api/src/main/java/org/apache/bookkeeper/api/StorageClient.java +++ b/stream/api/src/main/java/org/apache/bookkeeper/api/StorageClient.java @@ -33,8 +33,40 @@ @Evolving public interface StorageClient extends AutoAsyncCloseable { + /** + * Open a {@link PTable} table under namespace. + * + * @param namespace namespace + * @param table table name + * @return a future represents the open result + */ + CompletableFuture> openPTable(String namespace, String table); + + /** + * Open a {@link PTable} table under the default namespace of this client. + * The default namespace is configured when creating {@link StorageClient}. + * + * @param table table name + * @return a future represents the open result + */ CompletableFuture> openPTable(String table); + /** + * Open a {@link Table} table under namespace. + * + * @param namespace namespace + * @param table table name + * @return a future represents the open result + */ + CompletableFuture> openTable(String namespace, String table); + + /** + * Open a {@link Table} table under namespace. + * The default namespace is configured when creating {@link StorageClient}. + * + * @param table table name + * @return a future represents the open result + */ CompletableFuture> openTable(String table); } diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleClientBase.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleClientBase.java index 3f6bc776ce7..c213687e91e 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleClientBase.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleClientBase.java @@ -38,9 +38,11 @@ */ public class SimpleClientBase extends AbstractAutoAsyncCloseable { + protected final StorageClientSettings settings; protected final Resource schedulerResource; protected final OrderedScheduler scheduler; protected final ManagedChannel managedChannel; + protected final boolean ownChannel; protected final Channel channel; protected final RetryUtils retryUtils; @@ -50,11 +52,23 @@ protected SimpleClientBase(StorageClientSettings settings) { protected SimpleClientBase(StorageClientSettings settings, Resource schedulerResource) { - this.managedChannel = GrpcChannels.createChannelBuilder(settings.serviceUri(), settings).build(); + this( + settings, + schedulerResource, + GrpcChannels.createChannelBuilder(settings.serviceUri(), settings).build(), + true); + } + + protected SimpleClientBase(StorageClientSettings settings, + Resource schedulerResource, + ManagedChannel managedChannel, + boolean ownChannel) { + this.settings = settings; + this.managedChannel = managedChannel; + this.ownChannel = ownChannel; this.channel = ClientInterceptors.intercept( managedChannel, new StorageContainerClientInterceptor(0L)); - this.schedulerResource = schedulerResource; this.scheduler = SharedResourceManager.shared().get(schedulerResource); this.retryUtils = RetryUtils.create(settings.backoffPolicy(), scheduler); @@ -62,10 +76,10 @@ protected SimpleClientBase(StorageClientSettings settings, @Override protected void closeAsyncOnce(CompletableFuture closeFuture) { - managedChannel.shutdown(); - scheduler.submit(() -> { - SharedResourceManager.shared().release(schedulerResource, scheduler); - closeFuture.complete(null); - }); + if (ownChannel) { + managedChannel.shutdown(); + } + SharedResourceManager.shared().release(schedulerResource, scheduler); + closeFuture.complete(null); } } diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java index 4817dccb1dd..79ffb0ee2a7 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java @@ -24,6 +24,7 @@ import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.createGetStreamRequest; import io.grpc.CallOptions; +import io.grpc.ManagedChannel; import io.netty.buffer.ByteBuf; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -38,6 +39,8 @@ import org.apache.bookkeeper.clients.utils.GrpcUtils; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.ExceptionUtils; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.common.util.SharedResourceManager.Resource; import org.apache.bookkeeper.stream.proto.StorageType; import org.apache.bookkeeper.stream.proto.StreamProperties; import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc; @@ -48,17 +51,28 @@ * The implementation of {@link StorageClient} client. */ @Slf4j -class SimpleStorageClientImpl extends SimpleClientBase implements StorageClient { +public class SimpleStorageClientImpl extends SimpleClientBase implements StorageClient { private static final String COMPONENT_NAME = SimpleStorageClientImpl.class.getSimpleName(); - private final String namespaceName; + private final String defaultNamespace; private final RootRangeServiceFutureStub rootRangeService; public SimpleStorageClientImpl(String namespaceName, StorageClientSettings settings) { super(settings); - this.namespaceName = namespaceName; + this.defaultNamespace = namespaceName; + this.rootRangeService = GrpcUtils.configureGrpcStub( + RootRangeServiceGrpc.newFutureStub(channel), + Optional.empty()); + } + + public SimpleStorageClientImpl(String namespaceName, + StorageClientSettings settings, + Resource schedulerResource, + ManagedChannel channel) { + super(settings, schedulerResource, channel, false); + this.defaultNamespace = namespaceName; this.rootRangeService = GrpcUtils.configureGrpcStub( RootRangeServiceGrpc.newFutureStub(channel), Optional.empty()); @@ -69,28 +83,42 @@ public SimpleStorageClientImpl(String namespaceName, // @Override - public CompletableFuture> openPTable(String streamName) { + public CompletableFuture> openPTable(String tableName) { + return openPTable(defaultNamespace, tableName); + } + + @Override + public CompletableFuture> openPTable(String namespaceName, + String tableName) { return ExceptionUtils.callAndHandleClosedAsync( COMPONENT_NAME, isClosed(), - (future) -> openStreamAsTableImpl(streamName, future)); + (future) -> openTableImpl(namespaceName, tableName, future)); } @Override public CompletableFuture> openTable(String table) { - return openPTable(table) + return openTable(defaultNamespace, table); + } + + @Override + public CompletableFuture> openTable(String namespaceName, + String table) { + return openPTable(namespaceName, table) .thenApply(pTable -> new ByteBufTableImpl(pTable)); } - private void openStreamAsTableImpl(String streamName, - CompletableFuture> future) { + private void openTableImpl(String namespaceName, + String streamName, + CompletableFuture> future) { CompletableFuture getStreamFuture = retryUtils.execute(() -> fromListenableFuture(rootRangeService.getStream( createGetStreamRequest(namespaceName, streamName))) ).thenCompose(resp -> { if (StatusCode.SUCCESS == resp.getCode()) { StreamProperties streamProps = resp.getStreamProps(); - log.info("Retrieved table properties for table {} : {}", streamName, streamProps); + log.info("Retrieved table properties for table {}/{} : {}", + namespaceName, streamName, streamProps); if (StorageType.TABLE != streamProps.getStreamConf().getStorageType()) { return FutureUtils.exception(new ApiException( "Can't open a non-table storage entity : " + streamProps.getStreamConf().getStorageType())); diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientBuilder.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientBuilder.java index 3a7ff4c8ee1..82bad79a4d8 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientBuilder.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientBuilder.java @@ -105,7 +105,7 @@ public StorageAdminClient buildAdmin() { if (settings.enableServerSideRouting()) { return new SimpleStorageAdminClientImpl(settings); } else { - return new StorageAdminClientImpl(settings); + return new StorageAdminClientImpl(settings, ClientResources.create()); } } diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientImpl.java index 9d495569706..2b5bc4817c2 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientImpl.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientImpl.java @@ -44,57 +44,85 @@ * The implementation of {@link StorageClient} client. */ @Slf4j -class StorageClientImpl extends AbstractAutoAsyncCloseable implements StorageClient { +public class StorageClientImpl extends AbstractAutoAsyncCloseable implements StorageClient { private static final String COMPONENT_NAME = StorageClientImpl.class.getSimpleName(); - private final String namespaceName; + private final String defaultNamespace; private final StorageClientSettings settings; private final ClientResources resources; private final OrderedScheduler scheduler; // clients private final StorageServerClientManager serverManager; + private final boolean ownServerManager; + + StorageClientImpl(String namespaceName, + StorageClientSettings settings, + ClientResources resources) { + this( + namespaceName, + settings, + resources, + new StorageServerClientManagerImpl(settings, resources.scheduler()), + true); + } public StorageClientImpl(String namespaceName, StorageClientSettings settings, - ClientResources resources) { - this.namespaceName = namespaceName; + ClientResources resources, + StorageServerClientManager serverManager, + boolean ownServerManager) { + this.defaultNamespace = namespaceName; this.settings = settings; this.resources = resources; - this.serverManager = new StorageServerClientManagerImpl(settings, resources.scheduler()); + this.serverManager = serverManager; + this.ownServerManager = ownServerManager; this.scheduler = SharedResourceManager.shared().get(resources.scheduler()); - } - CompletableFuture getStreamProperties(String streamName) { + CompletableFuture getStreamProperties(String namespaceName, + String streamName) { return this.serverManager.getRootRangeClient().getStream(namespaceName, streamName); } // - // Materialized Views + // Tables // @Override - public CompletableFuture> openPTable(String streamName) { + public CompletableFuture> openPTable(String tableName) { + return openPTable(defaultNamespace, tableName); + } + + @Override + public CompletableFuture> openPTable(String namespaceName, + String tableName) { return ExceptionUtils.callAndHandleClosedAsync( COMPONENT_NAME, isClosed(), - (future) -> openStreamAsTableImpl(streamName, future)); + (future) -> openTableImpl(namespaceName, tableName, future)); } @Override public CompletableFuture> openTable(String table) { - return openPTable(table) + return openTable(defaultNamespace, table); + } + + @Override + public CompletableFuture> openTable(String namespaceName, + String table) { + return openPTable(namespaceName, table) .thenApply(pTable -> new ByteBufTableImpl(pTable)); } - private void openStreamAsTableImpl(String streamName, - CompletableFuture> future) { + private void openTableImpl(String namespaceName, + String tableName, + CompletableFuture> future) { FutureUtils.proxyTo( - getStreamProperties(streamName).thenComposeAsync(props -> { + getStreamProperties(namespaceName, tableName).thenComposeAsync(props -> { if (log.isInfoEnabled()) { - log.info("Retrieved table properties for table {} : {}", streamName, props); + log.info("Retrieved table properties for table {}/{} : {}", namespaceName, tableName, props); } if (StorageType.TABLE != props.getStreamConf().getStorageType()) { return FutureUtils.exception(new ApiException( @@ -102,7 +130,7 @@ private void openStreamAsTableImpl(String streamName, ); } return new PByteBufTableImpl( - streamName, + tableName, props, serverManager, scheduler.chooseThread(props.getStreamId()), @@ -120,7 +148,9 @@ private void openStreamAsTableImpl(String streamName, @Override protected void closeAsyncOnce(CompletableFuture closeFuture) { scheduler.submit(() -> { - serverManager.close(); + if (ownServerManager) { + serverManager.close(); + } closeFuture.complete(null); SharedResourceManager.shared().release(resources.scheduler(), scheduler); }); diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java index cbacbc396dd..04072e1952f 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java @@ -30,7 +30,9 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.api.StorageClient; import org.apache.bookkeeper.clients.SimpleClientBase; +import org.apache.bookkeeper.clients.SimpleStorageClientImpl; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.clients.utils.ClientResources; import org.apache.bookkeeper.clients.utils.GrpcUtils; @@ -68,6 +70,16 @@ public SimpleStorageAdminClientImpl(StorageClientSettings settings, Optional.empty()); } + @Override + public StorageClient asClient(String namespace) { + return new SimpleStorageClientImpl( + namespace, + settings, + schedulerResource, + managedChannel + ); + } + @Override public CompletableFuture createNamespace(String namespace, NamespaceConfiguration conf) { return retryUtils.execute(() -> diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClient.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClient.java index 55cc63a0ef9..ec3485aecc1 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClient.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClient.java @@ -18,6 +18,9 @@ package org.apache.bookkeeper.clients.admin; import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.api.StorageClient; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; +import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving; import org.apache.bookkeeper.common.util.AutoAsyncCloseable; import org.apache.bookkeeper.stream.proto.NamespaceConfiguration; import org.apache.bookkeeper.stream.proto.NamespaceProperties; @@ -27,21 +30,83 @@ /** * A storage admin client. */ +@Public +@Evolving public interface StorageAdminClient extends AutoAsyncCloseable { + /** + * Convert the storage admin client to a client. + * + * @return storage client + */ + default StorageClient asClient() { + return asClient(null); + } + + /** + * Convert the storage admin client to a client with default namespace. + * + * @param namespace namespace + * @return storage client + */ + StorageClient asClient(String namespace); + + /** + * Create a namespace with the provided namespace configuration conf. + * + * @param namespace namespace + * @param conf namespace configuration + * @return a future represent the creation result + */ CompletableFuture createNamespace(String namespace, NamespaceConfiguration conf); + /** + * Delete a namespace. + * + * @param namespace namespace + * @return a future represents the deletion result + */ CompletableFuture deleteNamespace(String namespace); + /** + * Get the namespace properties of a given namespace. + * + * @param namespace namespace + * @return a future represents the get result + */ CompletableFuture getNamespace(String namespace); + /** + * Create a stream streamName under namespace namespace + * with the provided stream configuration streamConfiguration. + * + * @param namespace namespace + * @param streamName stream name + * @param streamConfiguration stream configuration + * @return a future represents the creation result + */ CompletableFuture createStream(String namespace, String streamName, StreamConfiguration streamConfiguration); + /** + * Delete a stream from the provided namespace. + * + * @param namespace namespace + * @param streamName stream name + * @return a future represents the deletion result + */ CompletableFuture deleteStream(String namespace, String streamName); + /** + * Retrieve the stream properties of a given stream under + * the provided namespace. + * + * @param namespace namespace + * @param streamName stream name + * @return a future represents the get result + */ CompletableFuture getStream(String namespace, String streamName); } diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClientImpl.java index 8c392a07aa8..ea571bdf50c 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClientImpl.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClientImpl.java @@ -18,19 +18,18 @@ package org.apache.bookkeeper.clients.admin; -import com.google.common.annotations.VisibleForTesting; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.function.Supplier; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.api.StorageClient; +import org.apache.bookkeeper.clients.StorageClientImpl; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.clients.impl.internal.StorageServerClientManagerImpl; import org.apache.bookkeeper.clients.impl.internal.api.RootRangeClient; import org.apache.bookkeeper.clients.impl.internal.api.StorageServerClientManager; import org.apache.bookkeeper.clients.utils.ClientResources; import org.apache.bookkeeper.common.util.AbstractAutoAsyncCloseable; -import org.apache.bookkeeper.common.util.OrderedScheduler; -import org.apache.bookkeeper.common.util.SharedResourceManager.Resource; import org.apache.bookkeeper.stream.proto.NamespaceConfiguration; import org.apache.bookkeeper.stream.proto.NamespaceProperties; import org.apache.bookkeeper.stream.proto.StreamConfiguration; @@ -43,6 +42,8 @@ public class StorageAdminClientImpl extends AbstractAutoAsyncCloseable implements StorageAdminClient { // clients + private final StorageClientSettings settings; + private final ClientResources resources; private final StorageServerClientManager clientManager; private final RootRangeClient rootRangeClient; @@ -50,30 +51,31 @@ public class StorageAdminClientImpl extends AbstractAutoAsyncCloseable implement * Create a stream admin client with provided {@code withSettings}. * * @param settings withSettings to create an admin client. + * @param resources resources used by this client */ - public StorageAdminClientImpl(StorageClientSettings settings) { + public StorageAdminClientImpl(StorageClientSettings settings, + ClientResources resources) { this( settings, - ClientResources.create().scheduler()); - } - - /** - * Create a stream admin client with provided {@code withSettings} and {@code scheduler}. - * - * @param settings withSettings to create an admin client. - * @param schedulerResource scheduler to execute. - */ - public StorageAdminClientImpl(StorageClientSettings settings, - Resource schedulerResource) { - this(() -> new StorageServerClientManagerImpl(settings, schedulerResource)); + resources, + () -> new StorageServerClientManagerImpl(settings, resources.scheduler())); } - @VisibleForTesting - StorageAdminClientImpl(Supplier factory) { + StorageAdminClientImpl(StorageClientSettings settings, + ClientResources resources, + Supplier factory) { + this.settings = settings; + this.resources = resources; this.clientManager = factory.get(); this.rootRangeClient = this.clientManager.getRootRangeClient(); } + @Override + public StorageClient asClient(String namespace) { + return new StorageClientImpl( + namespace, settings, resources, clientManager, false); + } + @Override public CompletableFuture createNamespace(String namespace, NamespaceConfiguration colConf) { diff --git a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java index 6ae9035a553..9f0e6dfe8d5 100644 --- a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java +++ b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -90,7 +91,7 @@ public void testOpenPTable() throws Exception { .setStorageType(StorageType.TABLE) .build()) .build(); - when(client.getStreamProperties(anyString())) + when(client.getStreamProperties(anyString(), anyString())) .thenReturn(FutureUtils.value(streamProps)); PByteBufTableImpl tableImpl = mock(PByteBufTableImpl.class); @@ -107,6 +108,51 @@ public void testOpenPTable() throws Exception { assertSame(tableImpl, returnedTableImpl); } + @SuppressWarnings("unchecked") + @Test + public void testOpenPTableDiffernetNamespace() throws Exception { + StreamProperties tableProps1 = StreamProperties.newBuilder(STREAM_PROPERTIES) + .setStreamName("table1") + .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) + .setStorageType(StorageType.TABLE) + .build()) + .build(); + when(client.getStreamProperties(eq(NAMESPACE), eq("table1"))) + .thenReturn(FutureUtils.value(tableProps1)); + + StreamProperties tableProps2 = StreamProperties.newBuilder(STREAM_PROPERTIES) + .setStreamName("table2") + .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) + .setStorageType(StorageType.TABLE) + .build()) + .build(); + when(client.getStreamProperties(eq(NAMESPACE), eq("table2"))) + .thenReturn(FutureUtils.value(tableProps2)); + + PByteBufTableImpl tableImpl1 = mock(PByteBufTableImpl.class); + when(tableImpl1.initialize()).thenReturn(FutureUtils.value(tableImpl1)); + PByteBufTableImpl tableImpl2 = mock(PByteBufTableImpl.class); + when(tableImpl2.initialize()).thenReturn(FutureUtils.value(tableImpl2)); + + PowerMockito.whenNew(PByteBufTableImpl.class) + .withAnyArguments() + .thenReturn(tableImpl1); + + PTable returnedTableImpl1 = FutureUtils.result( + client.openPTable("table1") + ); + assertSame(tableImpl1, returnedTableImpl1); + + PowerMockito.whenNew(PByteBufTableImpl.class) + .withAnyArguments() + .thenReturn(tableImpl2); + + PTable returnedTableImpl2 = FutureUtils.result( + client.openPTable("table2") + ); + assertSame(tableImpl2, returnedTableImpl2); + } + @SuppressWarnings("unchecked") @Test public void testOpenTable() throws Exception { @@ -115,7 +161,7 @@ public void testOpenTable() throws Exception { .setStorageType(StorageType.TABLE) .build()) .build(); - when(client.getStreamProperties(anyString())) + when(client.getStreamProperties(anyString(), anyString())) .thenReturn(FutureUtils.value(streamProps)); PByteBufTableImpl tableImpl = mock(PByteBufTableImpl.class); @@ -134,6 +180,55 @@ public void testOpenTable() throws Exception { assertSame(tableImpl, Whitebox.getInternalState(bytesTableImpl, "underlying")); } + @SuppressWarnings("unchecked") + @Test + public void testOpenTableWithDifferentNamespace() throws Exception { + StreamProperties tableProps1 = StreamProperties.newBuilder(STREAM_PROPERTIES) + .setStreamName("table1") + .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) + .setStorageType(StorageType.TABLE) + .build()) + .build(); + when(client.getStreamProperties(eq(NAMESPACE), eq("table1"))) + .thenReturn(FutureUtils.value(tableProps1)); + + StreamProperties tableProps2 = StreamProperties.newBuilder(STREAM_PROPERTIES) + .setStreamName("table2") + .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) + .setStorageType(StorageType.TABLE) + .build()) + .build(); + when(client.getStreamProperties(eq(NAMESPACE), eq("table2"))) + .thenReturn(FutureUtils.value(tableProps2)); + + PByteBufTableImpl tableImpl1 = mock(PByteBufTableImpl.class); + when(tableImpl1.initialize()).thenReturn(FutureUtils.value(tableImpl1)); + PByteBufTableImpl tableImpl2 = mock(PByteBufTableImpl.class); + when(tableImpl2.initialize()).thenReturn(FutureUtils.value(tableImpl2)); + + PowerMockito.whenNew(PByteBufTableImpl.class) + .withAnyArguments() + .thenReturn(tableImpl1); + + Table returnedTableImpl1 = FutureUtils.result( + client.openTable("table1") + ); + assertTrue(returnedTableImpl1 instanceof ByteBufTableImpl); + ByteBufTableImpl bytesTableImpl1 = (ByteBufTableImpl) returnedTableImpl1; + assertSame(tableImpl1, Whitebox.getInternalState(bytesTableImpl1, "underlying")); + + PowerMockito.whenNew(PByteBufTableImpl.class) + .withAnyArguments() + .thenReturn(tableImpl2); + + Table returnedTableImpl2 = FutureUtils.result( + client.openTable("table2") + ); + assertTrue(returnedTableImpl2 instanceof ByteBufTableImpl); + ByteBufTableImpl bytesTableImpl2 = (ByteBufTableImpl) returnedTableImpl2; + assertSame(tableImpl2, Whitebox.getInternalState(bytesTableImpl2, "underlying")); + } + @SuppressWarnings("unchecked") @Test public void testOpenPTableIllegalOp() throws Exception { @@ -142,7 +237,7 @@ public void testOpenPTableIllegalOp() throws Exception { .setStorageType(StorageType.STREAM) .build()) .build(); - when(client.getStreamProperties(anyString())) + when(client.getStreamProperties(anyString(), anyString())) .thenReturn(FutureUtils.value(streamProps)); PByteBufTableImpl tableImpl = mock(PByteBufTableImpl.class); diff --git a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/admin/TestStorageAdminClientImpl.java b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/admin/TestStorageAdminClientImpl.java index 0787abc33e7..d9a791a5efb 100644 --- a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/admin/TestStorageAdminClientImpl.java +++ b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/admin/TestStorageAdminClientImpl.java @@ -25,8 +25,10 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.clients.impl.internal.api.RootRangeClient; import org.apache.bookkeeper.clients.impl.internal.api.StorageServerClientManager; +import org.apache.bookkeeper.clients.utils.ClientResources; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.NamespaceConfiguration; import org.apache.bookkeeper.stream.proto.NamespaceProperties; @@ -66,7 +68,12 @@ public class TestStorageAdminClientImpl { @Before public void setUp() { when(mockManager.getRootRangeClient()).thenReturn(mockRootRangeClient); - this.adminClient = new StorageAdminClientImpl(() -> mockManager); + this.adminClient = new StorageAdminClientImpl( + StorageClientSettings.newBuilder() + .serviceUri("bk://localhost:4181") + .build(), + ClientResources.create(), + () -> mockManager); } @Test From 4469c9a97bd523fdb54fb869bcc1a501a9c09ef6 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Wed, 17 Oct 2018 03:56:41 -0700 Subject: [PATCH 0084/1642] Set ConnectionExpired Listener to MetadataClientDriver in AR Descriptions of the changes in this PR: - add setConnectionExpiredListener method to MetadataClientDriver interface. - add listener to shutdown AR in the case of metadata connection expiry ### Motivation This commit - 4f0d2a195bd9be3788876b47813cee1440cf005c, has removed the shutdown logic in AutoRecoveryMain incase of ZK client session expiry, with the following reason "which doesn't make any sense for current retryable zookeeper". But if the ZK session has expired then it is not completely correct to let AutoRecoveryMain to continue to run in that state. Author: Sijie Guo Author: Andrey Yegorov Author: cguttapalem Author: Charan Reddy Guttapalem Author: Samuel Just Reviewers: Sijie Guo , Enrico Olivelli This closes #1747 from reddycharan/conexpirylisten --- .../bookkeeper/meta/MetadataClientDriver.java | 19 +++ .../meta/zk/ZKMetadataClientDriver.java | 14 ++ .../replication/AuditorElector.java | 4 + .../replication/AutoRecoveryMain.java | 6 + .../bookkeeper/meta/MetadataDriversTest.java | 5 + .../replication/AutoRecoveryMainTest.java | 133 +++++++++++++++++- .../etcd/EtcdMetadataClientDriver.java | 8 ++ 7 files changed, 188 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java index b53836790cc..7647a5bed85 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java @@ -84,4 +84,23 @@ LedgerManagerFactory getLedgerManagerFactory() @Override void close(); + /** + * State Listener on listening the metadata client session states. + */ + @FunctionalInterface + interface SessionStateListener { + + /** + * Signal when client session is expired. + */ + void onSessionExpired(); + } + + /** + * sets session state listener. + * + * @param sessionStateListener + * listener listening on metadata client session states. + */ + void setSessionStateListener(SessionStateListener sessionStateListener); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataClientDriver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataClientDriver.java index a5dcaa740cf..99b942771da 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataClientDriver.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataClientDriver.java @@ -20,7 +20,9 @@ import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; + import lombok.extern.slf4j.Slf4j; + import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.discover.ZKRegistrationClient; @@ -29,6 +31,8 @@ import org.apache.bookkeeper.meta.exceptions.MetadataException; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; +import org.apache.zookeeper.Watcher.Event.EventType; +import org.apache.zookeeper.Watcher.Event.KeeperState; /** * ZooKeeper based metadata client driver. @@ -88,4 +92,14 @@ public synchronized void close() { } super.close(); } + + @Override + public void setSessionStateListener(SessionStateListener sessionStateListener) { + zk.register((event) -> { + // Check for expired connection. + if (event.getType().equals(EventType.None) && event.getState().equals(KeeperState.Expired)) { + sessionStateListener.onSessionExpired(); + } + }); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java index cddb1f0f431..14c5c538562 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java @@ -179,6 +179,10 @@ private void createMyVote() throws KeeperException, InterruptedException { } } + String getMyVote() { + return myVote; + } + private String getVotePath(String vote) { return electionPath + vote; } 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 cd4aee2ce0d..c4954438232 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 @@ -40,6 +40,7 @@ import org.apache.bookkeeper.common.component.LifecycleComponent; import org.apache.bookkeeper.common.component.LifecycleComponentStack; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.MetadataClientDriver; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.server.conf.BookieConfiguration; @@ -91,6 +92,11 @@ public AutoRecoveryMain(ServerConfiguration conf, StatsLogger statsLogger) CompatibilityException { this.conf = conf; this.bkc = Auditor.createBookKeeperClient(conf); + MetadataClientDriver metadataClientDriver = bkc.getMetadataClientDriver(); + metadataClientDriver.setSessionStateListener(() -> { + LOG.error("Client connection to the Metadata server has expired, so shutting down AutoRecoveryMain!"); + shutdown(ExitCode.ZK_EXPIRED); + }); auditorElector = new AuditorElector( Bookie.getBookieAddress(conf).toString(), diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MetadataDriversTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MetadataDriversTest.java index 2d69cd2bec1..593c52005ad 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MetadataDriversTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MetadataDriversTest.java @@ -79,6 +79,10 @@ public LayoutManager getLayoutManager() { @Override public void close() { } + + @Override + public void setSessionStateListener(SessionStateListener sessionStateListener) { + } } static class ClientDriver1 extends TestClientDriver { @@ -88,6 +92,7 @@ public String getScheme() { return "driver1"; } + } static class ClientDriver2 extends TestClientDriver { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java index 8bb8049ef09..8aa969f5291 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java @@ -23,8 +23,13 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.meta.zk.ZKMetadataClientDriver; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; - +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; import org.junit.Test; /** @@ -74,4 +79,130 @@ public void testShutdown() throws Exception { main.replicationWorker.isRunning()); } + /** + * Test that, if an autorecovery looses its ZK connection/session it will + * shutdown. + */ + @Test + public void testAutoRecoverySessionLoss() throws Exception { + /* + * initialize three AutoRecovery instances. + */ + AutoRecoveryMain main1 = new AutoRecoveryMain(bsConfs.get(0)); + AutoRecoveryMain main2 = new AutoRecoveryMain(bsConfs.get(1)); + AutoRecoveryMain main3 = new AutoRecoveryMain(bsConfs.get(2)); + + /* + * start main1, make sure all the components are started and main1 is + * the current Auditor + */ + ZKMetadataClientDriver zkMetadataClientDriver1 = startAutoRecoveryMain(main1); + ZooKeeper zk1 = zkMetadataClientDriver1.getZk(); + Auditor auditor1 = main1.auditorElector.getAuditor(); + BookieSocketAddress currentAuditor = AuditorElector.getCurrentAuditor(bsConfs.get(0), zk1); + assertTrue("Current Auditor should be AR1", currentAuditor.equals(Bookie.getBookieAddress(bsConfs.get(0)))); + assertTrue("Auditor of AR1 should be running", auditor1.isRunning()); + + /* + * start main2 and main3 + */ + ZKMetadataClientDriver zkMetadataClientDriver2 = startAutoRecoveryMain(main2); + ZooKeeper zk2 = zkMetadataClientDriver2.getZk(); + ZKMetadataClientDriver zkMetadataClientDriver3 = startAutoRecoveryMain(main3); + ZooKeeper zk3 = zkMetadataClientDriver3.getZk(); + + /* + * make sure AR1 is still the current Auditor and AR2's and AR3's + * auditors are not running. + */ + assertTrue("Current Auditor should still be AR1", + currentAuditor.equals(Bookie.getBookieAddress(bsConfs.get(0)))); + Auditor auditor2 = main2.auditorElector.getAuditor(); + Auditor auditor3 = main3.auditorElector.getAuditor(); + assertTrue("AR2's Auditor should not be running", (auditor2 == null || !auditor2.isRunning())); + assertTrue("AR3's Auditor should not be running", (auditor3 == null || !auditor3.isRunning())); + + /* + * expire zk2 and zk1 sessions. + */ + zkUtil.expireSession(zk2); + zkUtil.expireSession(zk1); + + /* + * wait for some time for all the components of AR1 and AR2 are + * shutdown. + */ + for (int i = 0; i < 10; i++) { + if (!main1.auditorElector.isRunning() && !main1.replicationWorker.isRunning() + && !main1.isAutoRecoveryRunning() && !main2.auditorElector.isRunning() + && !main2.replicationWorker.isRunning() && !main2.isAutoRecoveryRunning()) { + break; + } + Thread.sleep(1000); + } + + /* + * since zk1 and zk2 sessions are expired, the 'myVote' ephemeral nodes + * of AR1 and AR2 should not be existing anymore. + */ + assertTrue("AR1's vote node should not be existing", + zk3.exists(main1.auditorElector.getMyVote(), false) == null); + assertTrue("AR2's vote node should not be existing", + zk3.exists(main2.auditorElector.getMyVote(), false) == null); + + /* + * the AR3 should be current auditor. + */ + currentAuditor = AuditorElector.getCurrentAuditor(bsConfs.get(2), zk3); + assertTrue("Current Auditor should be AR3", currentAuditor.equals(Bookie.getBookieAddress(bsConfs.get(2)))); + auditor3 = main3.auditorElector.getAuditor(); + assertTrue("Auditor of AR3 should be running", auditor3.isRunning()); + + /* + * since AR3 is current auditor, AR1's auditor should not be running + * anymore. + */ + assertFalse("AR1's auditor should not be running", auditor1.isRunning()); + + /* + * components of AR2 and AR3 should not be running since zk1 and zk2 + * sessions are expired. + */ + assertFalse("Elector1 should have shutdown", main1.auditorElector.isRunning()); + assertFalse("RW1 should have shutdown", main1.replicationWorker.isRunning()); + assertFalse("AR1 should have shutdown", main1.isAutoRecoveryRunning()); + assertFalse("Elector2 should have shutdown", main2.auditorElector.isRunning()); + assertFalse("RW2 should have shutdown", main2.replicationWorker.isRunning()); + assertFalse("AR2 should have shutdown", main2.isAutoRecoveryRunning()); + } + + /* + * start autoRecoveryMain and make sure all its components are running and + * myVote node is existing + */ + ZKMetadataClientDriver startAutoRecoveryMain(AutoRecoveryMain autoRecoveryMain) + throws InterruptedException, KeeperException, UnavailableException { + autoRecoveryMain.start(); + ZKMetadataClientDriver metadataClientDriver = (ZKMetadataClientDriver) autoRecoveryMain.bkc + .getMetadataClientDriver(); + ZooKeeper zk = metadataClientDriver.getZk(); + String myVote; + for (int i = 0; i < 10; i++) { + if (autoRecoveryMain.auditorElector.isRunning() && autoRecoveryMain.replicationWorker.isRunning() + && autoRecoveryMain.isAutoRecoveryRunning()) { + myVote = autoRecoveryMain.auditorElector.getMyVote(); + if (myVote != null) { + if (null != zk.exists(myVote, false)) { + break; + } + } + } + Thread.sleep(100); + } + assertTrue("autoRecoveryMain components should be running", autoRecoveryMain.auditorElector.isRunning() + && autoRecoveryMain.replicationWorker.isRunning() && autoRecoveryMain.isAutoRecoveryRunning()); + assertTrue("autoRecoveryMain's vote node should be existing", + zk.exists(autoRecoveryMain.auditorElector.getMyVote(), false) != null); + return metadataClientDriver; + } } diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataClientDriver.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataClientDriver.java index 8fd7b973b99..07d64186aa3 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataClientDriver.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataClientDriver.java @@ -72,4 +72,12 @@ public synchronized void close() { } super.close(); } + + @Override + public void setSessionStateListener(SessionStateListener sessionStateListener) { + /* + * TODO: EtcdMetadataClientDriver has to implement this method. + */ + throw new UnsupportedOperationException(); + } } From 9fc99370fd9f97a0e92b1b4da08d65422469f3a1 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 19 Oct 2018 08:33:30 +0800 Subject: [PATCH 0085/1642] [BUILD] Fix build & bash script error when not using `stream` profile Descriptions of the changes in this PR: *Motivation* There are two issues in current master when building without stream - build `bkperf` failed - bin/bookkeeper is using the wrong module *Changes* - build `bkperf` module as part of `stream` profile - fix the problem in `bin/bookkeeper` locating the right module - add two tests in travis to test builds without `stream` profile Author: Sijie Guo Author: Sijie Guo Author: Charan Reddy Guttapalem Author: Andrey Yegorov Author: Samuel Just Reviewers: Enrico Olivelli , Charan Reddy Guttapalem This closes #1749 from sijie/fix_build_without_stream_profile --- .travis.yml | 8 ++++++++ .travis_scripts/build.sh | 30 ++++++++++++++++++------------ bin/bookkeeper | 6 ++++-- pom.xml | 8 ++++---- tools/pom.xml | 2 +- 5 files changed, 35 insertions(+), 19 deletions(-) diff --git a/.travis.yml b/.travis.yml index 7c9b257a686..ad53fd5f29f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -39,6 +39,14 @@ matrix: env: CUSTOM_JDK="oraclejdk10" - os: linux env: CUSTOM_JDK="oraclejdk11" + - os: osx + osx_image: xcode8 + env: + - STREAM_DISABLED="true" + - os: linux + env: + - CUSTOM_JDK="oraclejdk8" + - STREAM_DISABLED="true" before_install: - | diff --git a/.travis_scripts/build.sh b/.travis_scripts/build.sh index f751bd84cf5..49e33a9aea9 100755 --- a/.travis_scripts/build.sh +++ b/.travis_scripts/build.sh @@ -22,16 +22,22 @@ set -ev BINDIR=`dirname "$0"` BK_HOME=`cd $BINDIR/..;pwd` -mvn --batch-mode clean apache-rat:check compile spotbugs:check install -DskipTests -Dstream -$BK_HOME/dev/check-binary-license ./bookkeeper-dist/all/target/bookkeeper-all-*-bin.tar.gz; -$BK_HOME/dev/check-binary-license ./bookkeeper-dist/server/target/bookkeeper-server-*-bin.tar.gz; -if [ "$DLOG_MODIFIED" == "true" ]; then - cd $BK_HOME/stream/distributedlog - mvn --batch-mode clean package -Ddistributedlog -fi -if [ "$TRAVIS_OS_NAME" == "linux" ] && [ "$WEBSITE_MODIFIED" == "true" ]; then - cd $BK_HOME/site - make clean - # run the docker image to build the website - ./docker/ci.sh +if [ "xtrue" == "x${STREAM_DISABLED}" ]; then + mvn --batch-mode clean install -DskipTests + # this command should be executed correctly + ${BK_HOME}/bin/bookkeeper help +else + mvn --batch-mode clean apache-rat:check compile spotbugs:check install -DskipTests -Dstream + $BK_HOME/dev/check-binary-license ./bookkeeper-dist/all/target/bookkeeper-all-*-bin.tar.gz; + $BK_HOME/dev/check-binary-license ./bookkeeper-dist/server/target/bookkeeper-server-*-bin.tar.gz; + if [ "$DLOG_MODIFIED" == "true" ]; then + cd $BK_HOME/stream/distributedlog + mvn --batch-mode clean package -Ddistributedlog + fi + if [ "$TRAVIS_OS_NAME" == "linux" ] && [ "$WEBSITE_MODIFIED" == "true" ]; then + cd $BK_HOME/site + make clean + # run the docker image to build the website + ./docker/ci.sh + fi fi diff --git a/bin/bookkeeper b/bin/bookkeeper index 38280a1d766..a207ff73290 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -18,6 +18,8 @@ # * limitations under the License. # */ +set -e + BINDIR=`dirname "$0"` BK_HOME=`cd ${BINDIR}/..;pwd` @@ -36,12 +38,12 @@ fi # check the configuration to see if table service is enabled or not. if [ -z "${ENABLE_TABLE_SERVICE}" ]; then TABLE_SERVICE_SETTING=$(grep StreamStorageLifecycleComponent ${BOOKIE_CONF_TO_CHECK}) - if [ "${TABLE_SERVICE_SETTING}" != \#* ]; then + if [[ "${TABLE_SERVICE_SETTING}" =~ ^extraServerComponents.* ]]; then ENABLE_TABLE_SERVICE="true" fi fi -if [ \( "x$1" == "xstandalone" \) -o \( "x${ENABLE_TABLE_SERVICE}" != "x" \) ]; then +if [ \( "x$1" == "xstandalone" \) -o \( "x${ENABLE_TABLE_SERVICE}" == "xtrue" \) ]; then BOOKIE_MODULE_PATH=stream/server BOOKIE_MODULE_NAME="(org.apache.bookkeeper-)?stream-storage-server" else diff --git a/pom.xml b/pom.xml index af3cefa88a6..4d210a34af6 100644 --- a/pom.xml +++ b/pom.xml @@ -60,13 +60,13 @@ bookkeeper-benchmark bookkeeper-stats-providers bookkeeper-http - shaded - tests - bookkeeper-dist - microbenchmarks stream/distributedlog tools metadata-drivers + bookkeeper-dist + shaded + microbenchmarks + tests diff --git a/tools/pom.xml b/tools/pom.xml index 37e63751ed5..2981bbb0221 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,7 +27,6 @@ pom framework - perf all @@ -42,6 +41,7 @@ stream + perf From 737950945ea1792fe757830f832077d9d8fa0e90 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 25 Oct 2018 05:06:58 -0700 Subject: [PATCH 0086/1642] Netty allocator wrapper ### Motivation Configuring the correct JVM memory settings and cache sizes for a BookKeeper cluster should be simplified. There are currently many knobs in Netty or JVM flags for different components and while with a good setup the systems is very stable, it's easy to setup non-optimal configurations which might result in OutOfMemory errors under load. Ideally, there should be very minimal configuration required to bring up a BookKeeper cluster that can work under a wide set of traffic loads. In any case, we should prefer to automatically fallback to slower alternatives, when possible, instead of throwing OOM exceptions. * Provide a wrapper to have a single unified configuration point for Netty allocator configuration. * Provide fallback policy when dealing with direct memory OOM errors ### Changes * This is the first PR. It only contains the allocator wrapper implementation. Subsequent PR will add the changes to use it. * Added `bookkeeper-common-allocator` module to have a no-dependencues module that can be used directly from Pulsar client library too (which doesn't depend on BK). Author: Sijie Guo Author: Charan Reddy Guttapalem Author: Matteo Merli Author: Andrey Yegorov Author: Samuel Just Reviewers: Qi Wang , Sijie Guo , Ivan Kelly , Enrico Olivelli This closes #1754 from merlimat/allocator --- bookkeeper-common-allocator/pom.xml | 60 ++++ .../allocator/ByteBufAllocatorBuilder.java | 97 +++++++ .../common/allocator/LeakDetectionPolicy.java | 47 +++ .../common/allocator/OutOfMemoryPolicy.java | 39 +++ .../common/allocator/PoolingPolicy.java | 45 +++ .../impl/ByteBufAllocatorBuilderImpl.java | 90 ++++++ .../allocator/impl/ByteBufAllocatorImpl.java | 163 +++++++++++ .../common/allocator/impl/package-info.java | 21 ++ .../common/allocator/package-info.java | 21 ++ .../impl/ByteBufAllocatorBuilderTest.java | 270 ++++++++++++++++++ pom.xml | 1 + 11 files changed, 854 insertions(+) create mode 100644 bookkeeper-common-allocator/pom.xml create mode 100644 bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorBuilder.java create mode 100644 bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/LeakDetectionPolicy.java create mode 100644 bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/OutOfMemoryPolicy.java create mode 100644 bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/PoolingPolicy.java create mode 100644 bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderImpl.java create mode 100644 bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java create mode 100644 bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/package-info.java create mode 100644 bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/package-info.java create mode 100644 bookkeeper-common-allocator/src/test/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderTest.java diff --git a/bookkeeper-common-allocator/pom.xml b/bookkeeper-common-allocator/pom.xml new file mode 100644 index 00000000000..a98889923ee --- /dev/null +++ b/bookkeeper-common-allocator/pom.xml @@ -0,0 +1,60 @@ + + + + 4.0.0 + + org.apache.bookkeeper + bookkeeper + 4.9.0-SNAPSHOT + + bookkeeper-common-allocator + Apache BookKeeper :: Common :: Allocator + + + io.netty + netty-buffer + + + + + + com.github.spotbugs + spotbugs-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorBuilder.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorBuilder.java new file mode 100644 index 00000000000..d749efd5cef --- /dev/null +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/ByteBufAllocatorBuilder.java @@ -0,0 +1,97 @@ +/** + * 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.common.allocator; + +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.buffer.UnpooledByteBufAllocator; + +import java.util.function.Consumer; + +import org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorBuilderImpl; + +/** + * Builder object to customize a ByteBuf allocator. + */ +public interface ByteBufAllocatorBuilder { + /** + * Creates a new {@link ByteBufAllocatorBuilder}. + */ + static ByteBufAllocatorBuilder create() { + return new ByteBufAllocatorBuilderImpl(); + } + + /** + * Finalize the configured {@link ByteBufAllocator}. + */ + ByteBufAllocator build(); + + /** + * Specify a custom allocator where the allocation requests should be + * forwarded to. + * + *

Default is to use a new instance of {@link PooledByteBufAllocator}. + */ + ByteBufAllocatorBuilder pooledAllocator(ByteBufAllocator pooledAllocator); + + /** + * Specify a custom allocator where the allocation requests should be + * forwarded to. + * + *

Default is to use {@link UnpooledByteBufAllocator#DEFAULT}. + */ + ByteBufAllocatorBuilder unpooledAllocator(ByteBufAllocator unpooledAllocator); + + /** + * Define the memory pooling policy. + * + *

Default is {@link PoolingPolicy#PooledDirect} + */ + ByteBufAllocatorBuilder poolingPolicy(PoolingPolicy policy); + + /** + * Controls the amount of concurrency for the memory pool. + * + *

Default is to have a number of allocator arenas equals to 2 * CPUS. + * + *

Decreasing this number will reduce the amount of memory overhead, at the + * expense of increased allocation contention. + */ + ByteBufAllocatorBuilder poolingConcurrency(int poolingConcurrency); + + /** + * Define the OutOfMemory handling policy. + * + *

Default is {@link OutOfMemoryPolicy#FallbackToHeap} + */ + ByteBufAllocatorBuilder outOfMemoryPolicy(OutOfMemoryPolicy policy); + + /** + * Add a listener that is triggered whenever there is an allocation failure. + * + *

Application can use this to trigger alerting or process restarting. + */ + ByteBufAllocatorBuilder outOfMemoryListener(Consumer listener); + + /** + * Enable the leak detection for the allocator. + * + *

Default is {@link LeakDetectionPolicy#Disabled} + */ + ByteBufAllocatorBuilder leakDetectionPolicy(LeakDetectionPolicy leakDetectionPolicy); +} diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/LeakDetectionPolicy.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/LeakDetectionPolicy.java new file mode 100644 index 00000000000..476684778a3 --- /dev/null +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/LeakDetectionPolicy.java @@ -0,0 +1,47 @@ +/** + * 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.common.allocator; + +/** + * Define the policy for the Netty leak detector. + */ +public enum LeakDetectionPolicy { + + /** + * No leak detection and no overhead. + */ + Disabled, + + /** + * Instruments 1% of the allocated buffer to track for leaks. + */ + Simple, + + /** + * Instruments 1% of the allocated buffer to track for leaks, reporting + * stack traces of places where the buffer was used. + */ + Advanced, + + /** + * Instruments 100% of the allocated buffer to track for leaks, reporting + * stack traces of places where the buffer was used. Introduce very + * significant overhead. + */ + Paranoid, +} diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/OutOfMemoryPolicy.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/OutOfMemoryPolicy.java new file mode 100644 index 00000000000..ff720507097 --- /dev/null +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/OutOfMemoryPolicy.java @@ -0,0 +1,39 @@ +/** + * 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.common.allocator; + +/** + * Represents the action to take when it's not possible to allocate memory. + */ +public enum OutOfMemoryPolicy { + + /** + * Throw regular OOM exception without taking addition actions. + */ + ThrowException, + + /** + * If it's not possible to allocate a buffer from direct memory, fallback to + * allocate an unpooled buffer from JVM heap. + * + *

This will help absorb memory allocation spikes because the heap + * allocations will naturally slow down the process and will result if full + * GC cleanup if the Heap itself is full. + */ + FallbackToHeap, +} diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/PoolingPolicy.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/PoolingPolicy.java new file mode 100644 index 00000000000..352a55ed1ed --- /dev/null +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/PoolingPolicy.java @@ -0,0 +1,45 @@ +/** + * 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.common.allocator; + +/** + * Define a policy for allocating buffers. + */ +public enum PoolingPolicy { + + /** + * Allocate memory from JVM heap without any pooling. + * + *

This option has the least overhead in terms of memory usage since the + * memory will be automatically reclaimed by the JVM GC but might impose a + * performance penalty at high throughput. + */ + UnpooledHeap, + + /** + * Use Direct memory for all buffers and pool the memory. + * + *

Direct memory will avoid the overhead of JVM GC and most memory copies + * when reading and writing to socket channel. + * + *

Pooling will add memory space overhead due to the fact that there will be + * fragmentation in the allocator and that threads will keep a portion of + * memory as thread-local to avoid contention when possible. + */ + PooledDirect +} diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderImpl.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderImpl.java new file mode 100644 index 00000000000..fc6bd9dc258 --- /dev/null +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderImpl.java @@ -0,0 +1,90 @@ +/** + * 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.common.allocator.impl; + +import io.netty.buffer.ByteBufAllocator; + +import java.util.function.Consumer; + +import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder; +import org.apache.bookkeeper.common.allocator.LeakDetectionPolicy; +import org.apache.bookkeeper.common.allocator.OutOfMemoryPolicy; +import org.apache.bookkeeper.common.allocator.PoolingPolicy; + +/** + * Implementation of {@link ByteBufAllocatorBuilder}. + */ +public class ByteBufAllocatorBuilderImpl implements ByteBufAllocatorBuilder { + + ByteBufAllocator pooledAllocator = null; + ByteBufAllocator unpooledAllocator = null; + PoolingPolicy poolingPolicy = PoolingPolicy.PooledDirect; + int poolingConcurrency = 2 * Runtime.getRuntime().availableProcessors(); + OutOfMemoryPolicy outOfMemoryPolicy = OutOfMemoryPolicy.FallbackToHeap; + Consumer outOfMemoryListener = null; + LeakDetectionPolicy leakDetectionPolicy = LeakDetectionPolicy.Disabled; + + @Override + public ByteBufAllocator build() { + return new ByteBufAllocatorImpl(pooledAllocator, unpooledAllocator, poolingPolicy, poolingConcurrency, + outOfMemoryPolicy, outOfMemoryListener, leakDetectionPolicy); + } + + @Override + public ByteBufAllocatorBuilder pooledAllocator(ByteBufAllocator pooledAllocator) { + this.pooledAllocator = pooledAllocator; + return this; + } + + @Override + public ByteBufAllocatorBuilder unpooledAllocator(ByteBufAllocator unpooledAllocator) { + this.unpooledAllocator = unpooledAllocator; + return this; + } + + @Override + public ByteBufAllocatorBuilder poolingPolicy(PoolingPolicy policy) { + this.poolingPolicy = policy; + return this; + } + + @Override + public ByteBufAllocatorBuilder poolingConcurrency(int poolingConcurrency) { + this.poolingConcurrency = poolingConcurrency; + return this; + } + + @Override + public ByteBufAllocatorBuilder outOfMemoryPolicy(OutOfMemoryPolicy policy) { + this.outOfMemoryPolicy = policy; + return this; + } + + @Override + public ByteBufAllocatorBuilder outOfMemoryListener(Consumer listener) { + this.outOfMemoryListener = listener; + return this; + } + + @Override + public ByteBufAllocatorBuilder leakDetectionPolicy(LeakDetectionPolicy leakDetectionPolicy) { + this.leakDetectionPolicy = leakDetectionPolicy; + return this; + } + +} diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java new file mode 100644 index 00000000000..35441659a05 --- /dev/null +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java @@ -0,0 +1,163 @@ +/** + * 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.common.allocator.impl; + +import io.netty.buffer.AbstractByteBufAllocator; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.util.ResourceLeakDetector; +import io.netty.util.ResourceLeakDetector.Level; +import java.util.function.Consumer; + +import org.apache.bookkeeper.common.allocator.LeakDetectionPolicy; +import org.apache.bookkeeper.common.allocator.OutOfMemoryPolicy; +import org.apache.bookkeeper.common.allocator.PoolingPolicy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implementation of {@link ByteBufAllocator}. + */ +public class ByteBufAllocatorImpl extends AbstractByteBufAllocator implements ByteBufAllocator { + + private static final Logger log = LoggerFactory.getLogger(ByteBufAllocatorImpl.class); + + private final ByteBufAllocator pooledAllocator; + private final ByteBufAllocator unpooledAllocator; + private final PoolingPolicy poolingPolicy; + private final OutOfMemoryPolicy outOfMemoryPolicy; + private final Consumer outOfMemoryListener; + + ByteBufAllocatorImpl(ByteBufAllocator pooledAllocator, ByteBufAllocator unpooledAllocator, + PoolingPolicy poolingPolicy, int poolingConcurrency, OutOfMemoryPolicy outOfMemoryPolicy, + Consumer outOfMemoryListener, + LeakDetectionPolicy leakDetectionPolicy) { + super(poolingPolicy == PoolingPolicy.PooledDirect /* preferDirect */); + + this.poolingPolicy = poolingPolicy; + this.outOfMemoryPolicy = outOfMemoryPolicy; + if (outOfMemoryListener == null) { + this.outOfMemoryListener = (v) -> { + log.error("Unable to allocate memory", v); + }; + } else { + this.outOfMemoryListener = outOfMemoryListener; + } + + if (poolingPolicy == PoolingPolicy.PooledDirect) { + if (pooledAllocator == null) { + this.pooledAllocator = new PooledByteBufAllocator( + true /* preferDirect */, + poolingConcurrency /* nHeapArena */, + poolingConcurrency /* nDirectArena */, + PooledByteBufAllocator.defaultPageSize(), + PooledByteBufAllocator.defaultMaxOrder(), + PooledByteBufAllocator.defaultTinyCacheSize(), + PooledByteBufAllocator.defaultSmallCacheSize(), + PooledByteBufAllocator.defaultNormalCacheSize(), + PooledByteBufAllocator.defaultUseCacheForAllThreads()); + } else { + this.pooledAllocator = pooledAllocator; + } + } else { + this.pooledAllocator = null; + } + + this.unpooledAllocator = (unpooledAllocator != null) ? unpooledAllocator : UnpooledByteBufAllocator.DEFAULT; + + // The setting is static in Netty, so it will actually affect all + // allocators + switch (leakDetectionPolicy) { + case Disabled: + if (log.isDebugEnabled()) { + log.debug("Disable Netty allocator leak detector"); + } + ResourceLeakDetector.setLevel(Level.DISABLED); + break; + + case Simple: + log.info("Setting Netty allocator leak detector to Simple"); + ResourceLeakDetector.setLevel(Level.SIMPLE); + break; + + case Advanced: + log.info("Setting Netty allocator leak detector to Advanced"); + ResourceLeakDetector.setLevel(Level.ADVANCED); + break; + + case Paranoid: + log.info("Setting Netty allocator leak detector to Paranoid"); + ResourceLeakDetector.setLevel(Level.PARANOID); + break; + } + } + + @Override + protected ByteBuf newHeapBuffer(int initialCapacity, int maxCapacity) { + try { + // There are few cases in which we ask explicitly for a pooled + // heap buffer. + ByteBufAllocator alloc = (poolingPolicy == PoolingPolicy.PooledDirect) ? pooledAllocator + : unpooledAllocator; + return alloc.heapBuffer(initialCapacity, maxCapacity); + } catch (OutOfMemoryError e) { + outOfMemoryListener.accept(e); + throw e; + } + } + + @Override + protected ByteBuf newDirectBuffer(int initialCapacity, int maxCapacity) { + if (poolingPolicy == PoolingPolicy.PooledDirect) { + try { + return pooledAllocator.directBuffer(initialCapacity, maxCapacity); + } catch (OutOfMemoryError e) { + switch (outOfMemoryPolicy) { + case ThrowException: + outOfMemoryListener.accept(e); + throw e; + + case FallbackToHeap: + try { + return unpooledAllocator.heapBuffer(initialCapacity, maxCapacity); + } catch (OutOfMemoryError e2) { + outOfMemoryListener.accept(e2); + throw e2; + } + } + return null; + } + } else { + // Unpooled heap buffer. Force heap buffers because unpooled direct + // buffers have very high overhead of allocation/reclaiming + try { + return unpooledAllocator.heapBuffer(initialCapacity, maxCapacity); + } catch (OutOfMemoryError e) { + outOfMemoryListener.accept(e); + throw e; + } + } + } + + @Override + public boolean isDirectBufferPooled() { + return pooledAllocator != null && pooledAllocator.isDirectBufferPooled(); + } +} diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/package-info.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/package-info.java new file mode 100644 index 00000000000..10133096ccc --- /dev/null +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/package-info.java @@ -0,0 +1,21 @@ +/** + * 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. + */ +/** + * Implements the utilities for allocator used across the project. + */ +package org.apache.bookkeeper.common.allocator.impl; \ No newline at end of file diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/package-info.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/package-info.java new file mode 100644 index 00000000000..512911402db --- /dev/null +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/package-info.java @@ -0,0 +1,21 @@ +/** + * 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. + */ +/** + * defines the utilities for allocator used across the project. + */ +package org.apache.bookkeeper.common.allocator; \ No newline at end of file diff --git a/bookkeeper-common-allocator/src/test/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderTest.java b/bookkeeper-common-allocator/src/test/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderTest.java new file mode 100644 index 00000000000..8ff66c33171 --- /dev/null +++ b/bookkeeper-common-allocator/src/test/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderTest.java @@ -0,0 +1,270 @@ +/** + * 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.common.allocator.impl; + +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 static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.buffer.UnpooledByteBufAllocator; + +import java.lang.reflect.Constructor; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder; +import org.apache.bookkeeper.common.allocator.OutOfMemoryPolicy; +import org.apache.bookkeeper.common.allocator.PoolingPolicy; +import org.junit.Test; + +/** + * Tests for {@link ByteBufAllocatorBuilderImpl}. + */ +public class ByteBufAllocatorBuilderTest { + + private static final OutOfMemoryError outOfDirectMemException; + + static { + try { + Class clazz = (Class) ByteBufAllocatorBuilderTest.class.getClassLoader() + .loadClass("io.netty.util.internal.OutOfDirectMemoryError"); + @SuppressWarnings("unchecked") + Constructor constructor = (Constructor) clazz + .getDeclaredConstructor(String.class); + + constructor.setAccessible(true); + outOfDirectMemException = constructor.newInstance("no mem"); + } catch (Exception e) { + throw new RuntimeException(e); + } + + } + + @Test + public void testOomWithException() { + ByteBufAllocator baseAlloc = mock(ByteBufAllocator.class); + when(baseAlloc.directBuffer(anyInt(), anyInt())).thenThrow(outOfDirectMemException); + + AtomicReference receivedException = new AtomicReference<>(); + + ByteBufAllocator alloc = ByteBufAllocatorBuilder.create() + .pooledAllocator(baseAlloc) + .outOfMemoryPolicy(OutOfMemoryPolicy.ThrowException) + .outOfMemoryListener((e) -> { + receivedException.set(e); + }) + .build(); + + try { + alloc.buffer(); + fail("Should have thrown exception"); + } catch (OutOfMemoryError e) { + // Expected + assertEquals(outOfDirectMemException, e); + } + + // Ensure the notification was triggered even when exception is thrown + assertEquals(outOfDirectMemException, receivedException.get()); + } + + @Test + public void testOomWithFallback() { + ByteBufAllocator baseAlloc = mock(ByteBufAllocator.class); + when(baseAlloc.directBuffer(anyInt(), anyInt())).thenThrow(outOfDirectMemException); + + AtomicReference receivedException = new AtomicReference<>(); + + ByteBufAllocator alloc = ByteBufAllocatorBuilder.create() + .pooledAllocator(baseAlloc) + .unpooledAllocator(UnpooledByteBufAllocator.DEFAULT) + .outOfMemoryPolicy(OutOfMemoryPolicy.FallbackToHeap) + .outOfMemoryListener((e) -> { + receivedException.set(e); + }) + .build(); + + // Should not throw exception + ByteBuf buf = alloc.buffer(); + assertEquals(UnpooledByteBufAllocator.DEFAULT, buf.alloc()); + + // No notification should have been triggered + assertEquals(null, receivedException.get()); + } + + @Test + public void testOomWithFallbackAndNoMoreHeap() { + ByteBufAllocator baseAlloc = mock(ByteBufAllocator.class); + when(baseAlloc.directBuffer(anyInt(), anyInt())).thenThrow(outOfDirectMemException); + + ByteBufAllocator heapAlloc = mock(ByteBufAllocator.class); + OutOfMemoryError noHeapError = new OutOfMemoryError("no more heap"); + when(heapAlloc.heapBuffer(anyInt(), anyInt())).thenThrow(noHeapError); + + AtomicReference receivedException = new AtomicReference<>(); + + ByteBufAllocator alloc = ByteBufAllocatorBuilder.create() + .pooledAllocator(baseAlloc) + .unpooledAllocator(heapAlloc) + .outOfMemoryPolicy(OutOfMemoryPolicy.FallbackToHeap) + .outOfMemoryListener((e) -> { + receivedException.set(e); + }) + .build(); + + try { + alloc.buffer(); + fail("Should have thrown exception"); + } catch (OutOfMemoryError e) { + // Expected + assertEquals(noHeapError, e); + } + + // Ensure the notification was triggered even when exception is thrown + assertEquals(noHeapError, receivedException.get()); + } + + @Test + public void testOomUnpooled() { + ByteBufAllocator heapAlloc = mock(ByteBufAllocator.class); + OutOfMemoryError noHeapError = new OutOfMemoryError("no more heap"); + when(heapAlloc.heapBuffer(anyInt(), anyInt())).thenThrow(noHeapError); + + AtomicReference receivedException = new AtomicReference<>(); + + ByteBufAllocator alloc = ByteBufAllocatorBuilder.create() + .poolingPolicy(PoolingPolicy.UnpooledHeap) + .unpooledAllocator(heapAlloc) + .outOfMemoryPolicy(OutOfMemoryPolicy.FallbackToHeap) + .outOfMemoryListener((e) -> { + receivedException.set(e); + }) + .build(); + + try { + alloc.directBuffer(); + fail("Should have thrown exception"); + } catch (OutOfMemoryError e) { + // Expected + assertEquals(noHeapError, e); + } + + // Ensure the notification was triggered even when exception is thrown + assertEquals(noHeapError, receivedException.get()); + } + + @Test + public void testOomUnpooledWithHeap() { + ByteBufAllocator heapAlloc = mock(ByteBufAllocator.class); + OutOfMemoryError noHeapError = new OutOfMemoryError("no more heap"); + when(heapAlloc.heapBuffer(anyInt(), anyInt())).thenThrow(noHeapError); + + AtomicReference receivedException = new AtomicReference<>(); + + ByteBufAllocator alloc = ByteBufAllocatorBuilder.create() + .poolingPolicy(PoolingPolicy.UnpooledHeap) + .unpooledAllocator(heapAlloc) + .outOfMemoryPolicy(OutOfMemoryPolicy.FallbackToHeap) + .outOfMemoryListener((e) -> { + receivedException.set(e); + }) + .build(); + + try { + alloc.heapBuffer(); + fail("Should have thrown exception"); + } catch (OutOfMemoryError e) { + // Expected + assertEquals(noHeapError, e); + } + + // Ensure the notification was triggered even when exception is thrown + assertEquals(noHeapError, receivedException.get()); + } + + @Test + public void testUnpooled() { + ByteBufAllocator alloc = ByteBufAllocatorBuilder.create() + .poolingPolicy(PoolingPolicy.UnpooledHeap) + .build(); + + ByteBuf buf = alloc.buffer(); + assertEquals(UnpooledByteBufAllocator.DEFAULT, buf.alloc()); + assertTrue(buf.hasArray()); + + ByteBuf buf2 = alloc.directBuffer(); + assertEquals(UnpooledByteBufAllocator.DEFAULT, buf2.alloc()); + assertTrue(buf2.hasArray()); + } + + @Test + public void testPooled() { + PooledByteBufAllocator pooledAlloc = new PooledByteBufAllocator(true); + + ByteBufAllocator alloc = ByteBufAllocatorBuilder.create() + .poolingPolicy(PoolingPolicy.PooledDirect) + .pooledAllocator(pooledAlloc) + .build(); + + assertTrue(alloc.isDirectBufferPooled()); + + ByteBuf buf1 = alloc.buffer(); + assertEquals(pooledAlloc, buf1.alloc()); + assertFalse(buf1.hasArray()); + buf1.release(); + + ByteBuf buf2 = alloc.directBuffer(); + assertEquals(pooledAlloc, buf2.alloc()); + assertFalse(buf2.hasArray()); + buf2.release(); + + ByteBuf buf3 = alloc.heapBuffer(); + assertEquals(pooledAlloc, buf3.alloc()); + assertTrue(buf3.hasArray()); + buf3.release(); + } + + @Test + public void testPooledWithDefaultAllocator() { + ByteBufAllocator alloc = ByteBufAllocatorBuilder.create() + .poolingPolicy(PoolingPolicy.PooledDirect) + .poolingConcurrency(3) + .build(); + + assertTrue(alloc.isDirectBufferPooled()); + + ByteBuf buf1 = alloc.buffer(); + assertEquals(PooledByteBufAllocator.class, buf1.alloc().getClass()); + assertEquals(3, ((PooledByteBufAllocator) buf1.alloc()).metric().numDirectArenas()); + assertFalse(buf1.hasArray()); + buf1.release(); + + ByteBuf buf2 = alloc.directBuffer(); + assertFalse(buf2.hasArray()); + buf2.release(); + + ByteBuf buf3 = alloc.heapBuffer(); + assertTrue(buf3.hasArray()); + buf3.release(); + } +} diff --git a/pom.xml b/pom.xml index 4d210a34af6..c13d4878ce6 100644 --- a/pom.xml +++ b/pom.xml @@ -54,6 +54,7 @@ buildtools circe-checksum bookkeeper-common + bookkeeper-common-allocator bookkeeper-stats bookkeeper-proto bookkeeper-server From 1723d1f36246859970b03ec839cf2ae2171529f4 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 25 Oct 2018 05:12:27 -0700 Subject: [PATCH 0087/1642] Adding multi-node ZKCluster test util class Descriptions of the changes in this PR: - adding multi-node ZKCluster util class - adding new interface for ZKCluster util and having separate implementaions for single node ZKCluster and multi-node ZKCluster ### Motivation This helps in adding more test coverage to Zookeeper aspects. Author: Sijie Guo Author: Charan Reddy Guttapalem Author: cguttapalem Author: Andrey Yegorov Author: Samuel Just Reviewers: Sijie Guo , Enrico Olivelli This closes #1753 from reddycharan/multinodezkcluster --- .../bookie/BookieInitializationTest.java | 2 +- .../bookie/LedgerStorageCheckpointTest.java | 4 +- .../bookkeeper/client/BookKeeperTest.java | 4 +- .../meta/TestLongZkLedgerIdGenerator.java | 4 +- .../meta/TestZkLedgerIdGenerator.java | 4 +- .../TestLedgerUnderreplicationManager.java | 4 +- .../test/BookKeeperClusterTestCase.java | 19 ++- .../bookkeeper/test/ZooKeeperCluster.java | 79 ++++++++++ .../bookkeeper/test/ZooKeeperClusterUtil.java | 140 ++++++++++++++++++ .../apache/bookkeeper/test/ZooKeeperUtil.java | 53 +++---- .../apache/bookkeeper/util/TestZkUtils.java | 4 +- .../zookeeper/TestZooKeeperClient.java | 36 ++++- 12 files changed, 297 insertions(+), 56 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperClusterUtil.java 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 6500e0b5446..530969bf68d 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 @@ -568,7 +568,7 @@ public void testBookieServerStartupOnEphemeralPorts() throws Exception { */ @Test public void testStartBookieWithoutZKServer() throws Exception { - zkUtil.killServer(); + zkUtil.killCluster(); File tmpDir = createTempDir("bookie", "test"); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java index 921d3101888..89a5063514c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java @@ -143,7 +143,7 @@ public void tearDown() throws Exception { * @throws Exception */ protected void startZKCluster() throws Exception { - zkUtil.startServer(); + zkUtil.startCluster(); } /** @@ -152,7 +152,7 @@ protected void startZKCluster() throws Exception { * @throws Exception */ protected void stopZKCluster() throws Exception { - zkUtil.killServer(); + zkUtil.killCluster(); } protected void cleanupTempDirs() throws Exception { 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 1a22e613e5f..0471e50b1ea 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 @@ -72,7 +72,7 @@ public void testConstructionZkDelay() throws Exception { .setZkTimeout(20000); CountDownLatch l = new CountDownLatch(1); - zkUtil.sleepServer(200, TimeUnit.MILLISECONDS, l); + zkUtil.sleepCluster(200, TimeUnit.MILLISECONDS, l); l.await(); BookKeeper bkc = new BookKeeper(conf); @@ -87,7 +87,7 @@ public void testConstructionNotConnectedExplicitZk() throws Exception { .setZkTimeout(20000); CountDownLatch l = new CountDownLatch(1); - zkUtil.sleepServer(200, TimeUnit.MILLISECONDS, l); + zkUtil.sleepCluster(200, TimeUnit.MILLISECONDS, l); l.await(); ZooKeeper zk = new ZooKeeper( diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLongZkLedgerIdGenerator.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLongZkLedgerIdGenerator.java index bc6ac03db41..7245cc125a5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLongZkLedgerIdGenerator.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLongZkLedgerIdGenerator.java @@ -58,7 +58,7 @@ public void setUp() throws Exception { super.setUp(); zkutil = new ZooKeeperUtil(); - zkutil.startServer(); + zkutil.startCluster(); zk = zkutil.getZooKeeperClient(); ZkLedgerIdGenerator shortLedgerIdGenerator = new ZkLedgerIdGenerator(zk, @@ -73,7 +73,7 @@ public void tearDown() throws Exception { LOG.info("Tearing down test"); ledgerIdGenerator.close(); zk.close(); - zkutil.killServer(); + zkutil.killCluster(); super.tearDown(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestZkLedgerIdGenerator.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestZkLedgerIdGenerator.java index 3779af40bfe..5c5f6b690dc 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestZkLedgerIdGenerator.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestZkLedgerIdGenerator.java @@ -55,7 +55,7 @@ public void setUp() throws Exception { super.setUp(); zkutil = new ZooKeeperUtil(); - zkutil.startServer(); + zkutil.startCluster(); zk = zkutil.getZooKeeperClient(); ledgerIdGenerator = new ZkLedgerIdGenerator(zk, @@ -68,7 +68,7 @@ public void tearDown() throws Exception { LOG.info("Tearing down test"); ledgerIdGenerator.close(); zk.close(); - zkutil.killServer(); + zkutil.killCluster(); super.tearDown(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java index 26285786698..e84113ba44b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java @@ -93,7 +93,7 @@ public class TestLedgerUnderreplicationManager { @Before public void setupZooKeeper() throws Exception { zkUtil = new ZooKeeperUtil(); - zkUtil.startServer(); + zkUtil.startCluster(); conf = TestBKConfiguration.newServerConfiguration(); conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); @@ -134,7 +134,7 @@ public void setupZooKeeper() throws Exception { @After public void teardownZooKeeper() throws Exception { if (zkUtil != null) { - zkUtil.killServer(); + zkUtil.killCluster(); zkUtil = null; } if (executor != null) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index 932abd9d1cf..b1feef059a5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -79,7 +79,7 @@ public abstract class BookKeeperClusterTestCase { public final Timeout globalTimeout; // Metadata service related variables - protected final ZooKeeperUtil zkUtil = new ZooKeeperUtil(); + protected final ZooKeeperCluster zkUtil; protected ZooKeeper zkc; protected String metadataServiceUri; @@ -118,8 +118,21 @@ public BookKeeperClusterTestCase(int numBookies) { } public BookKeeperClusterTestCase(int numBookies, int testTimeoutSecs) { + this(numBookies, 1, 120); + } + + public BookKeeperClusterTestCase(int numBookies, int numOfZKNodes, int testTimeoutSecs) { this.numBookies = numBookies; this.globalTimeout = Timeout.seconds(testTimeoutSecs); + if (numOfZKNodes == 1) { + zkUtil = new ZooKeeperUtil(); + } else { + try { + zkUtil = new ZooKeeperClusterUtil(numOfZKNodes); + } catch (IOException | KeeperException | InterruptedException e) { + throw new RuntimeException(e); + } + } } @Before @@ -202,7 +215,7 @@ protected File createTempDir(String prefix, String suffix) throws IOException { * @throws Exception */ protected void startZKCluster() throws Exception { - zkUtil.startServer(); + zkUtil.startCluster(); zkc = zkUtil.getZooKeeperClient(); } @@ -212,7 +225,7 @@ protected void startZKCluster() throws Exception { * @throws Exception */ protected void stopZKCluster() throws Exception { - zkUtil.killServer(); + zkUtil.killCluster(); } /** diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java new file mode 100644 index 00000000000..7713503916d --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java @@ -0,0 +1,79 @@ +/* + * + * 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.test; + +import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; +import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; + +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.Transaction; +import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooKeeper; + +/** + * Interface for ZooKeeperCluster. + */ +public interface ZooKeeperCluster { + ZooKeeper getZooKeeperClient(); + + String getZooKeeperConnectString(); + + String getMetadataServiceUri(); + + String getMetadataServiceUri(String zkLedgersRootPath); + + String getMetadataServiceUri(String zkLedgersRootPath, String type); + + void startCluster() throws Exception; + + void stopCluster() throws Exception; + + void restartCluster() throws Exception; + + void killCluster() throws Exception; + + void sleepCluster(final int time, final TimeUnit timeUnit, final CountDownLatch l) + throws InterruptedException, IOException; + + default void expireSession(ZooKeeper zk) throws Exception { + long id = zk.getSessionId(); + byte[] password = zk.getSessionPasswd(); + ZooKeeperWatcherBase w = new ZooKeeperWatcherBase(10000); + ZooKeeper zk2 = new ZooKeeper(getZooKeeperConnectString(), zk.getSessionTimeout(), w, id, password); + w.waitForConnection(); + zk2.close(); + } + + default void createBKEnsemble(String ledgersPath) throws KeeperException, InterruptedException { + Transaction txn = getZooKeeperClient().transaction(); + txn.create(ledgersPath, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + txn.create(ledgersPath + "/" + AVAILABLE_NODE, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + txn.create(ledgersPath + "/" + AVAILABLE_NODE + "/" + READONLY, new byte[0], Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + txn.commit(); + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperClusterUtil.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperClusterUtil.java new file mode 100644 index 00000000000..5eef1d0b189 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperClusterUtil.java @@ -0,0 +1,140 @@ +/* + * + * 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.test; + +import com.google.common.io.Files; +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.meta.LongHierarchicalLedgerManagerFactory; +import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.test.QuorumUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Provides multi node zookeeper cluster. + */ +@Slf4j +public class ZooKeeperClusterUtil implements ZooKeeperCluster { + + static { + enableZookeeperTestEnvVariables(); + } + + static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClusterUtil.class); + private final int numOfZKNodes; + public QuorumUtil quorumUtil; + String connectString; + protected ZooKeeper zkc; // zookeeper client + + public static void enableZookeeperTestEnvVariables() { + /* + * org.apache.zookeeper.test.ClientBase uses FourLetterWordMain, from + * 3.5.3 four letter words are disabled by default due to security + * reasons + */ + System.setProperty("zookeeper.4lw.commands.whitelist", "*"); + System.setProperty("zookeeper.admin.enableServer", "false"); + try { + System.setProperty("build.test.dir", Files.createTempDir().getCanonicalPath()); + } catch (IOException e) { + log.error("Failed to create temp dir, so setting build.test.dir system property to /tmp"); + System.setProperty("build.test.dir", "/tmp"); + } + } + + public ZooKeeperClusterUtil(int numOfZKNodes) throws IOException, KeeperException, InterruptedException { + if ((numOfZKNodes < 3) || (numOfZKNodes % 2 == 0)) { + throw new IllegalArgumentException("numOfZKNodes should be atleast 3 and it should not be even number"); + } + this.numOfZKNodes = numOfZKNodes; + } + + @Override + public String getZooKeeperConnectString() { + return connectString; + } + + @Override + public String getMetadataServiceUri() { + return getMetadataServiceUri("/ledgers"); + } + + @Override + public String getMetadataServiceUri(String zkLedgersRootPath) { + return getMetadataServiceUri(zkLedgersRootPath, LongHierarchicalLedgerManagerFactory.NAME); + } + + @Override + public String getMetadataServiceUri(String zkLedgersRootPath, String type) { + /* + * URI doesn't accept ',', for more info. check + * AbstractConfiguration.getMetadataServiceUri() + */ + return "zk+" + type + "://" + connectString.replace(",", ";") + zkLedgersRootPath; + } + + @Override + public ZooKeeper getZooKeeperClient() { + return zkc; + } + + @Override + public void startCluster() throws Exception { + // QuorumUtil will start 2*n+1 nodes. + quorumUtil = new QuorumUtil(numOfZKNodes / 2); + quorumUtil.startAll(); + connectString = quorumUtil.getConnString(); + // create a zookeeper client + LOG.debug("Instantiate ZK Client"); + zkc = ZooKeeperClient.newBuilder().connectString(getZooKeeperConnectString()).sessionTimeoutMs(10000).build(); + + // create default bk ensemble + createBKEnsemble("/ledgers"); + } + + @Override + public void stopCluster() throws Exception { + if (zkc != null) { + zkc.close(); + } + quorumUtil.shutdownAll(); + } + + @Override + public void restartCluster() throws Exception { + quorumUtil.startAll(); + } + + @Override + public void killCluster() throws Exception { + quorumUtil.tearDown(); + } + + @Override + public void sleepCluster(int time, TimeUnit timeUnit, CountDownLatch l) throws InterruptedException, IOException { + throw new UnsupportedOperationException("sleepServer operation is not supported for ZooKeeperClusterUtil"); + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperUtil.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperUtil.java index 09f50495742..6868f9249f7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperUtil.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperUtil.java @@ -21,8 +21,6 @@ package org.apache.bookkeeper.test; -import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; -import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; import static org.junit.Assert.assertTrue; import java.io.File; @@ -34,12 +32,7 @@ import org.apache.bookkeeper.util.IOUtils; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; -import org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase; import org.apache.commons.io.FileUtils; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.Transaction; -import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.server.NIOServerCnxnFactory; import org.apache.zookeeper.server.ZooKeeperServer; @@ -50,7 +43,7 @@ /** * Test the zookeeper utilities. */ -public class ZooKeeperUtil { +public class ZooKeeperUtil implements ZooKeeperCluster { static { // org.apache.zookeeper.test.ClientBase uses FourLetterWordMain, from 3.5.3 four letter words @@ -75,50 +68,47 @@ public ZooKeeperUtil() { connectString = loopbackIPAddr + ":" + zooKeeperPort; } + @Override public ZooKeeper getZooKeeperClient() { return zkc; } + @Override public String getZooKeeperConnectString() { return connectString; } + @Override public String getMetadataServiceUri() { return getMetadataServiceUri("/ledgers"); } + @Override public String getMetadataServiceUri(String zkLedgersRootPath) { return "zk://" + connectString + zkLedgersRootPath; } + @Override public String getMetadataServiceUri(String zkLedgersRootPath, String type) { return "zk+" + type + "://" + connectString + zkLedgersRootPath; } - public void startServer() throws Exception { + @Override + public void startCluster() throws Exception { // create a ZooKeeper server(dataDir, dataLogDir, port) LOG.debug("Running ZK server"); ClientBase.setupTestEnv(); zkTmpDir = IOUtils.createTempDir("zookeeper", "test"); // start the server and client. - restartServer(); + restartCluster(); // create default bk ensemble createBKEnsemble("/ledgers"); } - public void createBKEnsemble(String ledgersPath) throws KeeperException, InterruptedException { - Transaction txn = zkc.transaction(); - txn.create(ledgersPath, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - txn.create(ledgersPath + "/" + AVAILABLE_NODE, - new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - txn.create(ledgersPath + "/" + AVAILABLE_NODE + "/" + READONLY, - new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - txn.commit(); - } - - public void restartServer() throws Exception { + @Override + public void restartCluster() throws Exception { zks = new ZooKeeperServer(zkTmpDir, zkTmpDir, ZooKeeperServer.DEFAULT_TICK_TIME); serverFactory = new NIOServerCnxnFactory(); @@ -143,7 +133,8 @@ public void restartServer() throws Exception { .build(); } - public void sleepServer(final int time, + @Override + public void sleepCluster(final int time, final TimeUnit timeUnit, final CountDownLatch l) throws InterruptedException, IOException { @@ -171,17 +162,8 @@ public void run() { throw new IOException("ZooKeeper thread not found"); } - public void expireSession(ZooKeeper zk) throws Exception { - long id = zk.getSessionId(); - byte[] password = zk.getSessionPasswd(); - ZooKeeperWatcherBase w = new ZooKeeperWatcherBase(10000); - ZooKeeper zk2 = new ZooKeeper(getZooKeeperConnectString(), - zk.getSessionTimeout(), w, id, password); - w.waitForConnection(); - zk2.close(); - } - - public void stopServer() throws Exception { + @Override + public void stopCluster() throws Exception { if (zkc != null) { zkc.close(); } @@ -198,8 +180,9 @@ public void stopServer() throws Exception { } } - public void killServer() throws Exception { - stopServer(); + @Override + public void killCluster() throws Exception { + stopCluster(); FileUtils.deleteDirectory(zkTmpDir); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestZkUtils.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestZkUtils.java index 527a38460eb..cdc936d7294 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestZkUtils.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestZkUtils.java @@ -49,13 +49,13 @@ public class TestZkUtils extends TestCase { @Override public void setUp() throws Exception { logger.info("Setting up test {}.", getName()); - zkUtil.startServer(); + zkUtil.startCluster(); } @After @Override public void tearDown() throws Exception { - zkUtil.killServer(); + zkUtil.killCluster(); logger.info("Teared down test {}.", getName()); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java index bb9554ac2e4..fca65baa57f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java @@ -22,6 +22,8 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -32,6 +34,8 @@ import junit.framework.TestCase; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.ZooKeeperCluster; +import org.apache.bookkeeper.test.ZooKeeperClusterUtil; import org.apache.bookkeeper.test.ZooKeeperUtil; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.AsyncCallback.ACLCallback; @@ -54,30 +58,52 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Test the wrapper of {@link org.apache.zookeeper.ZooKeeper} client. */ +@RunWith(Parameterized.class) public class TestZooKeeperClient extends TestCase { + static { + ZooKeeperClusterUtil.enableZookeeperTestEnvVariables(); + } + private static final Logger logger = LoggerFactory.getLogger(TestZooKeeperClient.class); // ZooKeeper related variables - protected ZooKeeperUtil zkUtil = new ZooKeeperUtil(); + protected ZooKeeperCluster zkUtil; + + @Parameters + public static Collection zooKeeperUtilClass() { + return Arrays.asList(new Object[][] { { ZooKeeperUtil.class }, { ZooKeeperClusterUtil.class } }); + } + + public TestZooKeeperClient(Class zooKeeperUtilClass) + throws IOException, KeeperException, InterruptedException { + if (zooKeeperUtilClass.equals(ZooKeeperUtil.class)) { + zkUtil = new ZooKeeperUtil(); + } else { + zkUtil = new ZooKeeperClusterUtil(3); + } + } @Before @Override public void setUp() throws Exception { logger.info("Setting up test {}.", getName()); - zkUtil.startServer(); + zkUtil.startCluster(); } @After @Override public void tearDown() throws Exception { - zkUtil.killServer(); + zkUtil.killCluster(); logger.info("Teared down test {}.", getName()); } @@ -121,7 +147,7 @@ class ShutdownZkServerClient extends ZooKeeperClient { public void process(WatchedEvent event) { if (event.getType() == EventType.None && event.getState() == KeeperState.Expired) { try { - zkUtil.stopServer(); + zkUtil.stopCluster(); } catch (Exception e) { logger.error("Failed to stop zookeeper server : ", e); } @@ -175,7 +201,7 @@ public void process(WatchedEvent event) { // expected } - zkUtil.restartServer(); + zkUtil.restartCluster(); // wait for a reconnect cycle Thread.sleep(2 * timeout); From 0e0dc8eca62fb42c745d83bc110f86dcad0730a7 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 25 Oct 2018 05:15:18 -0700 Subject: [PATCH 0088/1642] Modules dependencies do not need to be included in checksum Jar The current assembly configuration is including the jars of the modules dependency inside its own jar. These jars are completely ignored, but they take up non trivial space. With this change the size of `circe-checksum.jar` goes from 2.8 MB to 62 KB. Author: Matteo Merli Reviewers: Sijie Guo , Enrico Olivelli This closes #1751 from merlimat/master --- circe-checksum/src/main/assembly/assembly.xml | 9 --------- 1 file changed, 9 deletions(-) diff --git a/circe-checksum/src/main/assembly/assembly.xml b/circe-checksum/src/main/assembly/assembly.xml index bded0cab8bf..432c1ac4282 100644 --- a/circe-checksum/src/main/assembly/assembly.xml +++ b/circe-checksum/src/main/assembly/assembly.xml @@ -26,15 +26,6 @@ false - - - - org.slf4j:slf4j-api - io.netty - *:nar:* - - - ${project.build.directory}/nar/${project.artifactId}-${project.version}-${os.arch}-MacOSX-gpp-jni/lib/${os.arch}-MacOSX-gpp/jni From c0824f51bfe370757c426e00704359caabc6d353 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 25 Oct 2018 14:28:57 +0200 Subject: [PATCH 0089/1642] Use immutable metadata in LedgerHandle Which means that for the two LedgerHandle operations that mutate the metadata, ensemble change and closing, ensure that metadata is written to the metadata store before the client ever uses it. Master issue: #281 Author: Ivan Kelly Author: Sijie Guo Author: Charan Reddy Guttapalem Author: Andrey Yegorov Author: Samuel Just Reviewers: Sijie Guo , Venkateswararao Jujjuri (JV) , Samuel Just This closes #1646 from ivankelly/immutable-handle-failures --- .../bookkeeper/client/EnsembleUtils.java | 98 +++ .../bookkeeper/client/LedgerHandle.java | 667 +++++------------- .../bookkeeper/client/LedgerHandleAdv.java | 6 +- .../bookkeeper/client/LedgerMetadata.java | 22 +- .../client/LedgerMetadataBuilder.java | 13 + .../bookkeeper/client/PendingAddOp.java | 8 +- .../client/ReadOnlyLedgerHandle.java | 82 +-- .../bookkeeper/client/api/WriteHandle.java | 16 + .../bookkeeper/client/BookKeeperTest.java | 9 +- .../apache/bookkeeper/client/ClientUtil.java | 23 + .../bookkeeper/client/DeferredSyncTest.java | 3 +- .../bookkeeper/client/HandleFailuresTest.java | 444 ++++++++++++ .../bookkeeper/client/LedgerClose2Test.java | 269 +++++++ .../bookkeeper/client/MdcContextTest.java | 6 +- .../client/TestDisableEnsembleChange.java | 6 +- .../client/TestReadLastConfirmedLongPoll.java | 5 +- .../bookkeeper/meta/MockLedgerManager.java | 95 +-- .../bookkeeper/test/ConditionalSetTest.java | 14 +- .../org/apache/bookkeeper/util/TestUtils.java | 17 +- 19 files changed, 1157 insertions(+), 646 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java new file mode 100644 index 00000000000..e4ab118f7dd --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java @@ -0,0 +1,98 @@ +/* + * + * 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.client; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.bookkeeper.net.BookieSocketAddress; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class EnsembleUtils { + private static final Logger LOG = LoggerFactory.getLogger(EnsembleUtils.class); + + static List replaceBookiesInEnsemble(BookieWatcher bookieWatcher, + LedgerMetadata metadata, + List oldEnsemble, + Map failedBookies, + String logContext) + throws BKException.BKNotEnoughBookiesException { + List newEnsemble = new ArrayList<>(oldEnsemble); + + int ensembleSize = metadata.getEnsembleSize(); + int writeQ = metadata.getWriteQuorumSize(); + int ackQ = metadata.getAckQuorumSize(); + Map customMetadata = metadata.getCustomMetadata(); + + Set exclude = new HashSet<>(failedBookies.values()); + + int replaced = 0; + for (Map.Entry entry : failedBookies.entrySet()) { + int idx = entry.getKey(); + BookieSocketAddress addr = entry.getValue(); + if (LOG.isDebugEnabled()) { + LOG.debug("{} replacing bookie: {} index: {}", logContext, addr, idx); + } + + if (!newEnsemble.get(idx).equals(addr)) { + if (LOG.isDebugEnabled()) { + LOG.debug("{} Not changing failed bookie {} at index {}, already changed to {}", + logContext, addr, idx, newEnsemble.get(idx)); + } + continue; + } + try { + BookieSocketAddress newBookie = bookieWatcher.replaceBookie( + ensembleSize, writeQ, ackQ, customMetadata, newEnsemble, idx, exclude); + newEnsemble.set(idx, newBookie); + + replaced++; + } catch (BKException.BKNotEnoughBookiesException e) { + // if there is no bookie replaced, we throw not enough bookie exception + if (replaced <= 0) { + throw e; + } else { + break; + } + } + } + return newEnsemble; + } + + static Set diffEnsemble(List e1, + List e2) { + checkArgument(e1.size() == e2.size(), "Ensembles must be of same size"); + Set diff = new HashSet<>(); + for (int i = 0; i < e1.size(); i++) { + if (!e1.get(i).equals(e2.get(i))) { + diff.add(i); + } + } + return diff; + } +} 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 beddaede3b4..a340cc7f6c6 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 @@ -20,11 +20,12 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkState; + import static org.apache.bookkeeper.client.api.BKException.Code.ClientClosedException; import static org.apache.bookkeeper.client.api.BKException.Code.WriteException; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Objects; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; @@ -39,7 +40,6 @@ import java.util.EnumSet; import java.util.Enumeration; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Queue; @@ -76,12 +76,10 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.State; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.OpStatsLogger; -import org.apache.bookkeeper.util.OrderedGenericCallback; import org.apache.bookkeeper.util.SafeRunnable; import org.apache.commons.collections4.IteratorUtils; import org.slf4j.Logger; @@ -101,6 +99,13 @@ public class LedgerHandle implements WriteHandle { final long ledgerId; long lastAddPushed; + private enum HandleState { + OPEN, + CLOSED + }; + + private HandleState handleState = HandleState.OPEN; + /** * Last entryId which has been confirmed to be written durably to the bookies. * This value is used by readers, the the LAC protocol @@ -123,8 +128,9 @@ public class LedgerHandle implements WriteHandle { ScheduledFuture timeoutFuture = null; - private final Map delayedWriteFailedBookies = - new HashMap(); + @VisibleForTesting + final Map delayedWriteFailedBookies = + new HashMap(); /** * Invalid entry id. This value is returned from methods which @@ -138,7 +144,8 @@ public class LedgerHandle implements WriteHandle { */ public static final long INVALID_LEDGER_ID = -0xABCDABCDL; - final AtomicInteger blockAddCompletions = new AtomicInteger(0); + final Object metadataLock = new Object(); + boolean changingEnsemble = false; final AtomicInteger numEnsembleChanges = new AtomicInteger(0); Queue pendingAddOps; ExplicitLacFlushPolicy explicitLacFlushPolicy; @@ -148,10 +155,6 @@ public class LedgerHandle implements WriteHandle { final Counter lacUpdateMissesCounter; private final OpStatsLogger clientChannelWriteWaitStats; - public Map getDelayedWriteFailedBookies() { - return delayedWriteFailedBookies; - } - LedgerHandle(ClientContext clientCtx, long ledgerId, LedgerMetadata metadata, BookKeeper.DigestType digestType, byte[] password, @@ -468,6 +471,10 @@ public synchronized boolean isClosed() { return getLedgerMetadata().isClosed(); } + boolean isHandleWritable() { + return !getLedgerMetadata().isClosed() && handleState == HandleState.OPEN; + } + void asyncCloseInternal(final CloseCallback cb, final Object ctx, final int rc) { try { doAsyncCloseInternal(cb, ctx, rc); @@ -494,135 +501,75 @@ void doAsyncCloseInternal(final CloseCallback cb, final Object ctx, final int rc clientCtx.getMainWorkerPool().executeOrdered(ledgerId, new SafeRunnable() { @Override public void safeRun() { - final long prevLastEntryId; - final long prevLength; - final State prevState; - List pendingAdds; - - if (isClosed()) { - // TODO: make ledger metadata immutable {@link https://github.com/apache/bookkeeper/issues/281} - // Although the metadata is already closed, we don't need to proceed zookeeper metadata update, but - // we still need to error out the pending add ops. - // - // There is a race condition a pending add op is enqueued, after a close op reset ledger metadata - // state to unclosed to resolve metadata conflicts. If we don't error out these pending add ops, - // they would be leak and never callback. - // - // The race condition happen in following sequence: - // a) ledger L is fenced - // b) write entry E encountered LedgerFencedException, trigger ledger close procedure - // c) ledger close encountered metadata version exception and set ledger metadata back to open - // d) writer tries to write entry E+1, since ledger metadata is still open (reset by c)) - // e) the close procedure in c) resolved the metadata conflicts and set ledger metadata to closed - // f) writing entry E+1 encountered LedgerFencedException which will enter ledger close procedure - // g) it would find that ledger metadata is closed, then it callbacks immediately without erroring - // out any pendings - synchronized (LedgerHandle.this) { - pendingAdds = drainPendingAddsToErrorOut(); - } - errorOutPendingAdds(rc, pendingAdds); - cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx); - return; - } + final HandleState prevHandleState; + final List pendingAdds; + final long lastEntry; + final long finalLength; synchronized (LedgerHandle.this) { - LedgerMetadata metadata = getLedgerMetadata(); - prevState = metadata.getState(); - prevLastEntryId = metadata.getLastEntryId(); - prevLength = metadata.getLength(); + prevHandleState = handleState; // drain pending adds first - pendingAdds = drainPendingAddsToErrorOut(); - - // synchronized on LedgerHandle.this to ensure that - // lastAddPushed can not be updated after the metadata - // is closed. - metadata.setLength(length); - metadata.close(lastAddConfirmed); - lastAddPushed = lastAddConfirmed; + pendingAdds = drainPendingAddsAndAdjustLength(); + + // taking the length must occur after draining, as draining changes the length + lastEntry = lastAddPushed = LedgerHandle.this.lastAddConfirmed; + finalLength = LedgerHandle.this.length; + handleState = HandleState.CLOSED; } // error out all pending adds during closing, the callbacks shouldn't be // running under any bk locks. errorOutPendingAdds(rc, pendingAdds); - if (LOG.isDebugEnabled()) { - LedgerMetadata metadata = getLedgerMetadata(); - LOG.debug("Closing ledger: " + ledgerId + " at entryId: " - + metadata.getLastEntryId() + " with this many bytes: " + metadata.getLength()); - } - - final class CloseCb extends OrderedGenericCallback { - CloseCb() { - super(clientCtx.getMainWorkerPool(), ledgerId); + if (prevHandleState == HandleState.CLOSED) { + cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Closing ledger: {} at entryId {} with {} bytes", getId(), lastEntry, finalLength); } - @Override - public void safeOperationComplete(final int rc, LedgerMetadata writtenMetadata) { - if (rc == BKException.Code.MetadataVersionException) { - rereadMetadata(new OrderedGenericCallback(clientCtx.getMainWorkerPool(), - ledgerId) { - @Override - public void safeOperationComplete(int newrc, LedgerMetadata newMeta) { - if (newrc != BKException.Code.OK) { - LOG.error("Error reading new metadata from ledger {} when closing: {}", - ledgerId, BKException.codeLogger(newrc)); - cb.closeComplete(rc, LedgerHandle.this, ctx); + tearDownWriteHandleState(); + new MetadataUpdateLoop( + clientCtx.getLedgerManager(), getId(), + LedgerHandle.this::getLedgerMetadata, + (metadata) -> { + if (metadata.isClosed()) { + /* If the ledger has been closed with the same lastEntry + * and length that we planned to close with, we have nothing to do, + * so just return success */ + if (lastEntry == metadata.getLastEntryId() + && finalLength == metadata.getLength()) { + return false; } else { - LedgerMetadata metadata = getLedgerMetadata(); - metadata.setState(prevState); - if (prevState.equals(State.CLOSED)) { - metadata.close(prevLastEntryId); - } - - metadata.setLength(prevLength); - if (!metadata.isNewerThan(newMeta) - && !metadata.isConflictWith(newMeta)) { - // use the new metadata's ensemble, in case re-replication already - // replaced some bookies in the ensemble. - metadata.setEnsembles(newMeta.getEnsembles()); - metadata.setVersion(newMeta.version); - metadata.setLength(length); - metadata.close(getLastAddConfirmed()); - writeLedgerConfig(new CloseCb()); - return; - } else { - metadata.setLength(length); - metadata.close(getLastAddConfirmed()); - LOG.warn("Conditional update ledger metadata for ledger {} failed.", - ledgerId); - cb.closeComplete(rc, LedgerHandle.this, ctx); - } + LOG.error("Metadata conflict when closing ledger {}." + + " Another client may have recovered the ledger while there" + + " were writes outstanding. (local lastEntry:{} length:{}) " + + " (metadata lastEntry:{} length:{})", + getId(), lastEntry, finalLength, + metadata.getLastEntryId(), metadata.getLength()); + throw new BKException.BKMetadataVersionException(); } + } else { + return true; } - - @Override - public String toString() { - return String.format("ReReadMetadataForClose(%d)", ledgerId); + }, + (metadata) -> { + return LedgerMetadataBuilder.from(metadata) + .closingAt(lastEntry, finalLength).build(); + }, + LedgerHandle.this::setLedgerMetadata) + .run().whenComplete((metadata, ex) -> { + if (ex != null) { + cb.closeComplete( + BKException.getExceptionCode( + ex, BKException.Code.UnexpectedConditionException), + LedgerHandle.this, ctx); + } else { + cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx); } - }); - } else if (rc != BKException.Code.OK) { - LOG.error("Error update ledger metadata for ledger {} : {}", - ledgerId, BKException.codeLogger(rc)); - cb.closeComplete(rc, LedgerHandle.this, ctx); - } else { - cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx); - } - } - - @Override - public String toString() { - return String.format("WriteLedgerConfigForClose(%d)", ledgerId); - } + }); } - - writeLedgerConfig(new CloseCb()); - tearDownWriteHandleState(); - } - - @Override - public String toString() { - return String.format("CloseLedgerHandle(%d)", ledgerId); } }); } @@ -1133,7 +1080,7 @@ public CompletableFuture force() { // synchronized on this to ensure that // the ledger isn't closed between checking and // updating lastAddPushed - if (getLedgerMetadata().isClosed()) { + if (!isHandleWritable()) { wasClosed = true; } } @@ -1292,14 +1239,14 @@ protected void doAsyncAddEntry(final PendingAddOp op) { // synchronized on this to ensure that // the ledger isn't closed between checking and // updating lastAddPushed - if (getLedgerMetadata().isClosed()) { - wasClosed = true; - } else { + if (isHandleWritable()) { long entryId = ++lastAddPushed; long currentLedgerLength = addToLength(op.payload.readableBytes()); op.setEntryId(entryId); op.setLedgerLength(currentLedgerLength); pendingAddOps.add(op); + } else { + wasClosed = true; } } @@ -1755,10 +1702,10 @@ private void monitorPendingAddOps() { } void errorOutPendingAdds(int rc) { - errorOutPendingAdds(rc, drainPendingAddsToErrorOut()); + errorOutPendingAdds(rc, drainPendingAddsAndAdjustLength()); } - synchronized List drainPendingAddsToErrorOut() { + synchronized List drainPendingAddsAndAdjustLength() { PendingAddOp pendingAddOp; List opsDrained = new ArrayList(pendingAddOps.size()); while ((pendingAddOp = pendingAddOps.poll()) != null) { @@ -1780,7 +1727,7 @@ void sendAddSuccessCallbacks() { PendingAddOp pendingAddOp; while ((pendingAddOp = pendingAddOps.peek()) != null - && blockAddCompletions.get() == 0) { + && !changingEnsemble) { if (!pendingAddOp.completed) { if (LOG.isDebugEnabled()) { LOG.debug("pending add not completed: {}", pendingAddOp); @@ -1808,77 +1755,35 @@ void sendAddSuccessCallbacks() { } - EnsembleInfo replaceBookieInMetadata(final Map failedBookies, - int ensembleChangeIdx) - throws BKException.BKNotEnoughBookiesException { - final ArrayList newEnsemble = new ArrayList(); - final long newEnsembleStartEntry = getLastAddConfirmed() + 1; - final HashSet replacedBookies = new HashSet(); - final LedgerMetadata metadata = getLedgerMetadata(); - synchronized (metadata) { - newEnsemble.addAll(getCurrentEnsemble()); - for (Map.Entry entry : failedBookies.entrySet()) { - int idx = entry.getKey(); - BookieSocketAddress addr = entry.getValue(); - if (LOG.isDebugEnabled()) { - LOG.debug("[EnsembleChange-L{}-{}] : replacing bookie: {} index: {}", - getId(), ensembleChangeIdx, addr, idx); - } - if (!newEnsemble.get(idx).equals(addr)) { - // ensemble has already changed, failure of this addr is immaterial - if (LOG.isDebugEnabled()) { - LOG.debug("Write did not succeed to {}, bookieIndex {}, but we have already fixed it.", - addr, idx); - } - continue; - } - try { - BookieSocketAddress newBookie = clientCtx.getBookieWatcher().replaceBookie( - metadata.getEnsembleSize(), - metadata.getWriteQuorumSize(), - metadata.getAckQuorumSize(), - metadata.getCustomMetadata(), - newEnsemble, - idx, - new HashSet(failedBookies.values())); - newEnsemble.set(idx, newBookie); - replacedBookies.add(idx); - } catch (BKException.BKNotEnoughBookiesException e) { - // if there is no bookie replaced, we throw not enough bookie exception - if (replacedBookies.size() <= 0) { - throw e; - } else { - break; - } - } - } - if (LOG.isDebugEnabled()) { - LOG.debug("[EnsembleChange-L{}-{}] : changing ensemble from: {} to: {} starting at entry: {}," - + " failed bookies: {}, replaced bookies: {}", - ledgerId, ensembleChangeIdx, getCurrentEnsemble(), newEnsemble, - (getLastAddConfirmed() + 1), failedBookies, replacedBookies); - } - metadata.addEnsemble(newEnsembleStartEntry, newEnsemble); + @VisibleForTesting + boolean hasDelayedWriteFailedBookies() { + return !delayedWriteFailedBookies.isEmpty(); + } + + void notifyWriteFailed(int index, BookieSocketAddress addr) { + synchronized (metadataLock) { + delayedWriteFailedBookies.put(index, addr); } - return new EnsembleInfo(newEnsemble, failedBookies, replacedBookies); } - void handleDelayedWriteBookieFailure() { - final Map copyDelayedWriteFailedBookies = - new HashMap(delayedWriteFailedBookies); - delayedWriteFailedBookies.clear(); + void maybeHandleDelayedWriteBookieFailure() { + synchronized (metadataLock) { + if (delayedWriteFailedBookies.isEmpty()) { + return; + } + Map toReplace = new HashMap<>(delayedWriteFailedBookies); + delayedWriteFailedBookies.clear(); - // Original intent of this change is to do a best-effort ensemble change. - // But this is not possible until the local metadata is completely immutable. - // Until the feature "Make LedgerMetadata Immutable #610" Is complete we will use - // handleBookieFailure() to handle delayed writes as regular bookie failures. - handleBookieFailure(copyDelayedWriteFailedBookies); + // Original intent of this change is to do a best-effort ensemble change. + // But this is not possible until the local metadata is completely immutable. + // Until the feature "Make LedgerMetadata Immutable #610" Is complete we will use + // handleBookieFailure() to handle delayed writes as regular bookie failures. + handleBookieFailure(toReplace); + } } void handleBookieFailure(final Map failedBookies) { - int curBlockAddCompletions = blockAddCompletions.incrementAndGet(); if (clientCtx.getConf().disableEnsembleChangeFeature.isAvailable()) { - blockAddCompletions.decrementAndGet(); if (LOG.isDebugEnabled()) { LOG.debug("Ensemble change is disabled. Retry sending to failed bookies {} for ledger {}.", failedBookies, ledgerId); @@ -1888,7 +1793,6 @@ void handleBookieFailure(final Map failedBookies) } if (writeFlags.contains(WriteFlag.DEFERRED_SYNC)) { - blockAddCompletions.decrementAndGet(); if (LOG.isDebugEnabled()) { LOG.debug("Cannot perform ensemble change with write flags {}. " + "Failed bookies {} for ledger {}.", @@ -1898,302 +1802,113 @@ void handleBookieFailure(final Map failedBookies) return; } - int curNumEnsembleChanges = numEnsembleChanges.incrementAndGet(); - // when the ensemble changes are too frequent, close handle - if (curNumEnsembleChanges > clientCtx.getConf().maxAllowedEnsembleChanges) { - if (LOG.isDebugEnabled()) { - LOG.debug("Ledger {} reaches max allowed ensemble change number {}", - ledgerId, clientCtx.getConf().maxAllowedEnsembleChanges); - } - handleUnrecoverableErrorDuringAdd(WriteException); - return; - } - LedgerMetadata metadata = getLedgerMetadata(); - synchronized (metadata) { - try { - EnsembleInfo ensembleInfo = replaceBookieInMetadata(failedBookies, curNumEnsembleChanges); - if (ensembleInfo.replacedBookies.isEmpty()) { - blockAddCompletions.decrementAndGet(); - return; - } - if (LOG.isDebugEnabled()) { - LOG.debug("[EnsembleChange-L{}-{}] : writing new ensemble info = {}, block add completions = {}", - getId(), curNumEnsembleChanges, ensembleInfo, curBlockAddCompletions); - } - writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, curBlockAddCompletions, - curNumEnsembleChanges)); - // clear if there are any delayed write failures were recorded. - delayedWriteFailedBookies.clear(); - } catch (BKException.BKNotEnoughBookiesException e) { - LOG.error("Could not get additional bookie to remake ensemble, closing ledger: {}", ledgerId); - handleUnrecoverableErrorDuringAdd(e.getCode()); - return; - } - } - } - - // Contains newly reformed ensemble, bookieIndex, failedBookieAddress - static final class EnsembleInfo { - final ArrayList newEnsemble; - private final Map failedBookies; - final Set replacedBookies; - - public EnsembleInfo(ArrayList newEnsemble, - Map failedBookies, - Set replacedBookies) { - this.newEnsemble = newEnsemble; - this.failedBookies = failedBookies; - this.replacedBookies = replacedBookies; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("Ensemble Info : failed bookies = ").append(failedBookies) - .append(", replaced bookies = ").append(replacedBookies) - .append(", new ensemble = ").append(newEnsemble); - return sb.toString(); - } - } - - /** - * Callback which is updating the ledgerMetadata in zk with the newly - * reformed ensemble. On MetadataVersionException, will reread latest - * ledgerMetadata and act upon. - */ - private final class ChangeEnsembleCb extends OrderedGenericCallback { - private final EnsembleInfo ensembleInfo; - private final int curBlockAddCompletions; - private final int ensembleChangeIdx; - - ChangeEnsembleCb(EnsembleInfo ensembleInfo, - int curBlockAddCompletions, - int ensembleChangeIdx) { - super(clientCtx.getMainWorkerPool(), ledgerId); - this.ensembleInfo = ensembleInfo; - this.curBlockAddCompletions = curBlockAddCompletions; - this.ensembleChangeIdx = ensembleChangeIdx; - } - - @Override - public void safeOperationComplete(final int rc, LedgerMetadata writtenMetadata) { - if (rc == BKException.Code.MetadataVersionException) { - // We changed the ensemble, but got a version exception. We - // should still consider this as an ensemble change - ensembleChangeCounter.inc(); - - if (LOG.isDebugEnabled()) { - LOG.info("[EnsembleChange-L{}-{}] : encountered version conflicts, re-read ledger metadata.", - getId(), ensembleChangeIdx); - } - - rereadMetadata(new ReReadLedgerMetadataCb(rc, - ensembleInfo, curBlockAddCompletions, ensembleChangeIdx)); - return; - } else if (rc != BKException.Code.OK) { - LOG.error("[EnsembleChange-L{}-{}] : could not persist ledger metadata : info = {}, " - + "closing ledger : {}.", getId(), ensembleChangeIdx, ensembleInfo, rc); - handleUnrecoverableErrorDuringAdd(rc); - return; - } - int newBlockAddCompletions = blockAddCompletions.decrementAndGet(); + boolean triggerLoop = false; + Map toReplace = null; + List origEnsemble = null; + synchronized (metadataLock) { + if (changingEnsemble) { + delayedWriteFailedBookies.putAll(failedBookies); + } else { + changingEnsemble = true; + triggerLoop = true; + toReplace = new HashMap<>(delayedWriteFailedBookies); + delayedWriteFailedBookies.clear(); + toReplace.putAll(failedBookies); - if (LOG.isDebugEnabled()) { - LOG.info("[EnsembleChange-L{}-{}] : completed ensemble change, block add completion {} => {}", - getId(), ensembleChangeIdx, curBlockAddCompletions, newBlockAddCompletions); + origEnsemble = getCurrentEnsemble(); } - - // We've successfully changed an ensemble - ensembleChangeCounter.inc(); - LOG.info("New Ensemble: {} for ledger: {}", ensembleInfo.newEnsemble, ledgerId); - - // the failed bookie has been replaced - unsetSuccessAndSendWriteRequest(ensembleInfo.newEnsemble, ensembleInfo.replacedBookies); } - - @Override - public String toString() { - return String.format("ChangeEnsemble(%d)", ledgerId); + if (triggerLoop) { + ensembleChangeLoop(origEnsemble, toReplace); } } - /** - * Callback which is reading the ledgerMetadata present in zk. This will try - * to resolve the version conflicts. - */ - private final class ReReadLedgerMetadataCb extends OrderedGenericCallback { - private final int rc; - private final EnsembleInfo ensembleInfo; - private final int curBlockAddCompletions; - private final int ensembleChangeIdx; - - ReReadLedgerMetadataCb(int rc, - EnsembleInfo ensembleInfo, - int curBlockAddCompletions, - int ensembleChangeIdx) { - super(clientCtx.getMainWorkerPool(), ledgerId); - this.rc = rc; - this.ensembleInfo = ensembleInfo; - this.curBlockAddCompletions = curBlockAddCompletions; - this.ensembleChangeIdx = ensembleChangeIdx; - } + void ensembleChangeLoop(List origEnsemble, Map failedBookies) { + int ensembleChangeId = numEnsembleChanges.incrementAndGet(); + String logContext = String.format("[EnsembleChange(ledger:%d, change-id:%010d)]", ledgerId, ensembleChangeId); - @Override - public void safeOperationComplete(int newrc, LedgerMetadata newMeta) { - if (newrc != BKException.Code.OK) { - LOG.error("[EnsembleChange-L{}-{}] : error re-reading metadata " - + "to address ensemble change conflicts: {}", - ledgerId, ensembleChangeIdx, BKException.codeLogger(newrc)); - handleUnrecoverableErrorDuringAdd(rc); - } else { - if (!resolveConflict(newMeta)) { - LOG.error("[EnsembleChange-L{}-{}] : could not resolve ledger metadata conflict" - + " while changing ensemble to: {}, local meta data is \n {} \n," - + " zk meta data is \n {} \n, closing ledger", - ledgerId, ensembleChangeIdx, ensembleInfo.newEnsemble, getLedgerMetadata(), newMeta); - handleUnrecoverableErrorDuringAdd(rc); - } - } + // when the ensemble changes are too frequent, close handle + if (ensembleChangeId > clientCtx.getConf().maxAllowedEnsembleChanges) { + LOG.info("{} reaches max allowed ensemble change number {}", + logContext, clientCtx.getConf().maxAllowedEnsembleChanges); + handleUnrecoverableErrorDuringAdd(WriteException); + return; } - /** - * Specific resolve conflicts happened when multiple bookies failures in same ensemble. - * - *

Resolving the version conflicts between local ledgerMetadata and zk - * ledgerMetadata. This will do the following: - *

    - *
  • - * check whether ledgerMetadata state matches of local and zk
  • - *
  • - * if the zk ledgerMetadata still contains the failed bookie, then - * update zookeeper with the newBookie otherwise send write request
  • - *
- *

- */ - private boolean resolveConflict(LedgerMetadata newMeta) { - LedgerMetadata metadata = getLedgerMetadata(); - if (LOG.isDebugEnabled()) { - LOG.debug("[EnsembleChange-L{}-{}] : resolving conflicts - local metadata = \n {} \n," - + " zk metadata = \n {} \n", ledgerId, ensembleChangeIdx, metadata, newMeta); - } - // make sure the ledger isn't closed by other ones. - if (metadata.getState() != newMeta.getState()) { - if (LOG.isDebugEnabled()) { - LOG.info("[EnsembleChange-L{}-{}] : resolving conflicts but state changed," - + " local metadata = \n {} \n, zk metadata = \n {} \n", - ledgerId, ensembleChangeIdx, metadata, newMeta); - } - return false; - } - - // We should check number of ensembles since there are two kinds of metadata conflicts: - // - Case 1: Multiple bookies involved in ensemble change. - // Number of ensembles should be same in this case. - // - Case 2: Recovery (Auto/Manually) replaced ensemble and ensemble changed. - // The metadata changed due to ensemble change would have one more ensemble - // than the metadata changed by recovery. - int diff = newMeta.getEnsembles().size() - metadata.getEnsembles().size(); - if (0 != diff) { - if (LOG.isDebugEnabled()) { - LOG.debug("[EnsembleChange-L{}-{}] : resolving conflicts but ensembles have {} differences," - + " local metadata = \n {} \n, zk metadata = \n {} \n", - ledgerId, ensembleChangeIdx, diff, metadata, newMeta); - } - if (-1 == diff) { - // Case 1: metadata is changed by other ones (e.g. Recovery) - return updateMetadataIfPossible(metadata, newMeta); - } - return false; - } - - // - // Case 2: - // - // If the failed the bookie is still existed in the metadata (in zookeeper), it means that - // the ensemble change of the failed bookie is failed due to metadata conflicts. so try to - // update the ensemble change metadata again. Otherwise, it means that the ensemble change - // is already succeed, unset the success and re-adding entries. - if (!areFailedBookiesReplaced(newMeta, ensembleInfo)) { - // If the in-memory data doesn't contains the failed bookie, it means the ensemble change - // didn't finish, so try to resolve conflicts with the metadata read from zookeeper and - // update ensemble changed metadata again. - if (areFailedBookiesReplaced(metadata, ensembleInfo)) { - return updateMetadataIfPossible(metadata, newMeta); - } - } else { - ensembleChangeCounter.inc(); - // We've successfully changed an ensemble - // the failed bookie has been replaced - int newBlockAddCompletions = blockAddCompletions.decrementAndGet(); - unsetSuccessAndSendWriteRequest(ensembleInfo.newEnsemble, ensembleInfo.replacedBookies); - if (LOG.isDebugEnabled()) { - LOG.info("[EnsembleChange-L{}-{}] : resolved conflicts, block add complectiosn {} => {}.", - ledgerId, ensembleChangeIdx, curBlockAddCompletions, newBlockAddCompletions); - } - } - return true; - } + if (LOG.isDebugEnabled()) { + LOG.debug("{} Replacing {} in {}", logContext, failedBookies, origEnsemble); + } + + AtomicInteger attempts = new AtomicInteger(0); + new MetadataUpdateLoop( + clientCtx.getLedgerManager(), getId(), + this::getLedgerMetadata, + (metadata) -> !metadata.isClosed() && !metadata.isInRecovery() + && failedBookies.entrySet().stream().anyMatch( + (e) -> metadata.getLastEnsembleValue().get(e.getKey()).equals(e.getValue())), + (metadata) -> { + attempts.incrementAndGet(); + + List currentEnsemble = getCurrentEnsemble(); + List newEnsemble = EnsembleUtils.replaceBookiesInEnsemble( + clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext); + Long lastEnsembleKey = metadata.getLastEnsembleKey(); + LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata); + long newEnsembleStartEntry = getLastAddConfirmed() + 1; + checkState(lastEnsembleKey <= newEnsembleStartEntry, + "New ensemble must either replace the last ensemble, or add a new one"); + if (LOG.isDebugEnabled()) { + LOG.debug("{}[attempt:{}] changing ensemble from: {} to: {} starting at entry: {}", + logContext, attempts.get(), currentEnsemble, newEnsemble, newEnsembleStartEntry); + } - /** - * Check whether all the failed bookies are replaced. - * - * @param newMeta - * new ledger metadata - * @param ensembleInfo - * ensemble info used for ensemble change. - * @return true if all failed bookies are replaced, false otherwise - */ - private boolean areFailedBookiesReplaced(LedgerMetadata newMeta, EnsembleInfo ensembleInfo) { - boolean replaced = true; - for (Integer replacedBookieIdx : ensembleInfo.replacedBookies) { - BookieSocketAddress failedBookieAddr = ensembleInfo.failedBookies.get(replacedBookieIdx); - BookieSocketAddress replacedBookieAddr = newMeta.getEnsembles() - .lastEntry().getValue().get(replacedBookieIdx); - replaced &= !Objects.equal(replacedBookieAddr, failedBookieAddr); - } - return replaced; - } + if (lastEnsembleKey.equals(newEnsembleStartEntry)) { + return builder.replaceEnsembleEntry(newEnsembleStartEntry, newEnsemble).build(); + } else { + return builder.newEnsembleEntry(newEnsembleStartEntry, newEnsemble).build(); + } + }, + this::setLedgerMetadata) + .run().whenCompleteAsync((metadata, ex) -> { + if (ex != null) { + LOG.warn("{}[attempt:{}] Exception changing ensemble", logContext, attempts.get(), ex); + handleUnrecoverableErrorDuringAdd(BKException.getExceptionCode(ex, WriteException)); + } else if (metadata.isClosed()) { + if (LOG.isDebugEnabled()) { + LOG.debug("{}[attempt:{}] Metadata closed during attempt to replace bookie." + + " Another client must have recovered the ledger.", logContext, attempts.get()); + } + handleUnrecoverableErrorDuringAdd(BKException.Code.LedgerClosedException); + } else if (metadata.isInRecovery()) { + if (LOG.isDebugEnabled()) { + LOG.debug("{}[attempt:{}] Metadata marked as in-recovery during attempt to replace bookie." + + " Another client must be recovering the ledger.", logContext, attempts.get()); + } - private boolean updateMetadataIfPossible(LedgerMetadata metadata, LedgerMetadata newMeta) { - // if the local metadata is newer than zookeeper metadata, it means that metadata is updated - // again when it was trying re-reading the metatada, re-kick the reread again - if (metadata.isNewerThan(newMeta)) { - if (LOG.isDebugEnabled()) { - LOG.debug("[EnsembleChange-L{}-{}] : reread metadata because local metadata is newer.", - new Object[]{ledgerId, ensembleChangeIdx}); - } - rereadMetadata(this); - return true; - } - // make sure the metadata doesn't changed by other ones. - if (metadata.isConflictWith(newMeta)) { - if (LOG.isDebugEnabled()) { - LOG.debug("[EnsembleChange-L{}-{}] : metadata is conflicted, local metadata = \n {} \n," - + " zk metadata = \n {} \n", ledgerId, ensembleChangeIdx, metadata, newMeta); - } - return false; - } - if (LOG.isDebugEnabled()) { - LOG.info("[EnsembleChange-L{}-{}] : resolved ledger metadata conflict and writing to zookeeper," - + " local meta data is \n {} \n, zk meta data is \n {}.", - ledgerId, ensembleChangeIdx, metadata, newMeta); - } - // update znode version - metadata.setVersion(newMeta.getVersion()); - // merge ensemble infos from new meta except last ensemble - // since they might be modified by recovery tool. - metadata.mergeEnsembles(newMeta.getEnsembles()); - writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, curBlockAddCompletions, - ensembleChangeIdx)); - return true; - } + handleUnrecoverableErrorDuringAdd(BKException.Code.LedgerFencedException); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("{}[attempt:{}] Success updating metadata.", logContext, attempts.get()); + } - @Override - public String toString() { - return String.format("ReReadLedgerMetadata(%d)", ledgerId); - } + synchronized (metadataLock) { + if (!delayedWriteFailedBookies.isEmpty()) { + Map toReplace = new HashMap<>(delayedWriteFailedBookies); + delayedWriteFailedBookies.clear(); + + ensembleChangeLoop(origEnsemble, toReplace); + } else { + List newEnsemble = getCurrentEnsemble(); + Set replaced = EnsembleUtils.diffEnsemble(origEnsemble, newEnsemble); + LOG.info("New Ensemble: {} for ledger: {}", newEnsemble, ledgerId); + unsetSuccessAndSendWriteRequest(newEnsemble, replaced); + changingEnsemble = false; + } + } + } + }, clientCtx.getMainWorkerPool().chooseThread(ledgerId)); } void unsetSuccessAndSendWriteRequest(List ensemble, final Set bookies) { 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 c1d38490bce..bd8beae48bf 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 @@ -222,12 +222,12 @@ protected void doAsyncAddEntry(final PendingAddOp op) { // synchronized on this to ensure that // the ledger isn't closed between checking and // updating lastAddPushed - if (getLedgerMetadata().isClosed()) { - wasClosed = true; - } else { + if (isHandleWritable()) { long currentLength = addToLength(op.payload.readableBytes()); op.setLedgerLength(currentLength); pendingAddOps.add(op); + } else { + wasClosed = true; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index ffb0d6aa7bc..8c5e3b8515c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -19,6 +19,7 @@ import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; @@ -148,9 +149,14 @@ public LedgerMetadata(int ensembleSize, this.writeQuorumSize = writeQuorumSize; this.ackQuorumSize = ackQuorumSize; this.state = state; - lastEntryId.ifPresent((eid) -> this.lastEntryId = eid); + if (lastEntryId.isPresent()) { + this.lastEntryId = lastEntryId.get(); + } else { + this.lastEntryId = LedgerHandle.INVALID_ENTRY_ID; + } length.ifPresent((l) -> this.length = l); setEnsembles(ensembles); + if (state != LedgerMetadataFormat.State.CLOSED) { currentEnsemble = this.ensembles.lastEntry().getValue(); } @@ -788,11 +794,13 @@ Set getBookiesInThisLedger() { return bookies; } - java.util.Optional getLastEnsembleKey() { - if (ensembles.size() > 0) { - return java.util.Optional.of(ensembles.lastKey()); - } else { - return java.util.Optional.empty(); - } + List getLastEnsembleValue() { + checkState(!ensembles.isEmpty(), "Metadata should never be created with no ensembles"); + return ensembles.lastEntry().getValue(); + } + + Long getLastEnsembleKey() { + checkState(!ensembles.isEmpty(), "Metadata should never be created with no ensembles"); + return ensembles.lastKey(); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index a9d83b04757..ae78d5f2652 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -102,6 +102,19 @@ LedgerMetadataBuilder withEnsembleSize(int ensembleSize) { return this; } + LedgerMetadataBuilder withWriteQuorumSize(int writeQuorumSize) { + checkArgument(ensembleSize >= writeQuorumSize, "Write quorum must be less or equal to ensemble size"); + checkArgument(writeQuorumSize >= ackQuorumSize, "Write quorum must be greater or equal to ack quorum"); + this.writeQuorumSize = writeQuorumSize; + return this; + } + + LedgerMetadataBuilder withAckQuorumSize(int ackQuorumSize) { + checkArgument(writeQuorumSize >= ackQuorumSize, "Ack quorum must be less or equal to write quorum"); + this.ackQuorumSize = ackQuorumSize; + return this; + } + LedgerMetadataBuilder newEnsembleEntry(long firstEntry, List ensemble) { checkArgument(ensemble.size() == ensembleSize, "Size of passed in ensemble must match the ensembleSize of the builder"); 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 89bf0b85a73..ad2f7ae4f07 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 @@ -251,10 +251,8 @@ public void safeRun() { // We are about to send. Check if we need to make an ensemble change // becasue of delayed write errors - Map delayedWriteFailedBookies = lh.getDelayedWriteFailedBookies(); - if (!delayedWriteFailedBookies.isEmpty()) { - lh.handleDelayedWriteBookieFailure(); - } + lh.maybeHandleDelayedWriteBookieFailure(); + // Iterate over set and trigger the sendWriteRequests DistributionSchedule.WriteSet writeSet = lh.distributionSchedule.getWriteSet(entryId); @@ -293,7 +291,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre clientCtx.getClientStats().getAddOpUrCounter().inc(); if (!clientCtx.getConf().disableEnsembleChangeFeature.isAvailable() && !clientCtx.getConf().delayEnsembleChange) { - lh.getDelayedWriteFailedBookies().putIfAbsent(bookieIndex, addr); + lh.notifyWriteFailed(bookieIndex, addr); } } // even the add operation is completed, but because we don't reset completed flag back to false when diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java index e2c9a44a3c4..e4794de575f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java @@ -20,18 +20,14 @@ */ package org.apache.bookkeeper.client; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; import java.security.GeneralSecurityException; -import java.util.ArrayList; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NavigableMap; -import java.util.Optional; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; @@ -209,64 +205,6 @@ public void readLastConfirmedComplete(int rc, long lastConfirmed, Object ctx) { }, ctx); } - List replaceBookiesInEnsemble(LedgerMetadata metadata, - List oldEnsemble, - Map failedBookies) - throws BKException.BKNotEnoughBookiesException { - List newEnsemble = new ArrayList<>(oldEnsemble); - - int ensembleSize = metadata.getEnsembleSize(); - int writeQ = metadata.getWriteQuorumSize(); - int ackQ = metadata.getAckQuorumSize(); - Map customMetadata = metadata.getCustomMetadata(); - - Set exclude = new HashSet<>(failedBookies.values()); - - int replaced = 0; - for (Map.Entry entry : failedBookies.entrySet()) { - int idx = entry.getKey(); - BookieSocketAddress addr = entry.getValue(); - if (LOG.isDebugEnabled()) { - LOG.debug("[EnsembleChange-L{}] replacing bookie: {} index: {}", getId(), addr, idx); - } - - if (!newEnsemble.get(idx).equals(addr)) { - if (LOG.isDebugEnabled()) { - LOG.debug("[EnsembleChange-L{}] Not changing failed bookie {} at index {}, already changed to {}", - getId(), addr, idx, newEnsemble.get(idx)); - } - continue; - } - try { - BookieSocketAddress newBookie = clientCtx.getBookieWatcher().replaceBookie( - ensembleSize, writeQ, ackQ, customMetadata, newEnsemble, idx, exclude); - newEnsemble.set(idx, newBookie); - - replaced++; - } catch (BKException.BKNotEnoughBookiesException e) { - // if there is no bookie replaced, we throw not enough bookie exception - if (replaced <= 0) { - throw e; - } else { - break; - } - } - } - return newEnsemble; - } - - private static Set diffEnsemble(List e1, - List e2) { - checkArgument(e1.size() == e2.size(), "Ensembles must be of same size"); - Set diff = new HashSet<>(); - for (int i = 0; i < e1.size(); i++) { - if (!e1.get(i).equals(e2.get(i))) { - diff.add(i); - } - } - return diff; - } - /** * For a read only ledger handle, this method will only ever be called during recovery, * when we are reading forward from LAC and writing back those entries. As such, @@ -276,21 +214,19 @@ private static Set diffEnsemble(List e1, */ @Override void handleBookieFailure(final Map failedBookies) { - blockAddCompletions.incrementAndGet(); - // handleBookieFailure should always run in the ordered executor thread for this // ledger, so this synchronized should be unnecessary, but putting it here now // just in case (can be removed when we validate threads) synchronized (metadataLock) { + String logContext = String.format("[RecoveryEnsembleChange(ledger:%d)]", ledgerId); + long lac = getLastAddConfirmed(); LedgerMetadata metadata = getLedgerMetadata(); List currentEnsemble = getCurrentEnsemble(); try { - List newEnsemble = replaceBookiesInEnsemble(metadata, currentEnsemble, - failedBookies); - - Set replaced = diffEnsemble(currentEnsemble, newEnsemble); - blockAddCompletions.decrementAndGet(); + List newEnsemble = EnsembleUtils.replaceBookiesInEnsemble( + clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext); + Set replaced = EnsembleUtils.diffEnsemble(currentEnsemble, newEnsemble); if (!replaced.isEmpty()) { newEnsemblesFromRecovery.put(lac + 1, newEnsemble); unsetSuccessAndSendWriteRequest(newEnsemble, replaced); @@ -378,16 +314,14 @@ CompletableFuture closeRecovered() { (metadata) -> metadata.isInRecovery(), (metadata) -> { LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata); - Optional lastEnsembleKey = metadata.getLastEnsembleKey(); - checkState(lastEnsembleKey.isPresent(), - "Metadata shouldn't have been created without at least one ensemble"); + Long lastEnsembleKey = metadata.getLastEnsembleKey(); synchronized (metadataLock) { newEnsemblesFromRecovery.entrySet().forEach( (e) -> { - checkState(e.getKey() >= lastEnsembleKey.get(), + checkState(e.getKey() >= lastEnsembleKey, "Once a ledger is in recovery, noone can add ensembles without closing"); // Occurs when a bookie need to be replaced at very start of recovery - if (lastEnsembleKey.get().equals(e.getKey())) { + if (lastEnsembleKey.equals(e.getKey())) { builder.replaceEnsembleEntry(e.getKey(), e.getValue()); } else { builder.newEnsembleEntry(e.getKey(), e.getValue()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteHandle.java index edad5f46022..95a1765d0a4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteHandle.java @@ -141,6 +141,14 @@ default long append(byte[] data, int offset, int length) throws BKException, Int * entry of the ledger is. Once the ledger has been closed, all reads from the * ledger will return the same set of entries. * + *

The close operation can error if it finds conflicting metadata when it + * tries to write to the metadata store. On close, the metadata state is set to + * closed and lastEntry and length of the ledger are fixed in the metadata. A + * conflict occurs if the metadata in the metadata store has a different value for + * the lastEntry or length. If another process has updated the metadata, setting it + * to closed, but have fixed the lastEntry and length to the same values as this + * process is trying to write, the operation completes successfully. + * * @return an handle to access the result of the operation */ @Override @@ -152,6 +160,14 @@ default long append(byte[] data, int offset, int length) throws BKException, Int *

Closing a ledger will ensure that all clients agree on what the last * entry of the ledger is. Once the ledger has been closed, all reads from the * ledger will return the same set of entries. + * + *

The close operation can error if it finds conflicting metadata when it + * tries to write to the metadata store. On close, the metadata state is set to + * closed and lastEntry and length of the ledger are fixed in the metadata. A + * conflict occurs if the metadata in the metadata store has a different value for + * the lastEntry or length. If another process has updated the metadata, setting it + * to closed, but have fixed the lastEntry and length to the same values as this + * process is trying to write, the operation completes successfully. */ @Override default void close() throws BKException, InterruptedException { 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 0471e50b1ea..7be404d0d17 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 @@ -536,12 +536,9 @@ public void testReadAfterLastAddConfirmed() throws Exception { } } - try { - writeLh.close(); - fail("should not be able to close the first LedgerHandler as a recovery has been performed"); - } catch (BKException.BKMetadataVersionException expected) { - } - + // should still be able to close as long as recovery closed the ledger + // with the same last entryId and length as in the write handle. + writeLh.close(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java index bb3e5532f7d..5add60f62c5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java @@ -17,11 +17,15 @@ */ package org.apache.bookkeeper.client; +import static java.nio.charset.StandardCharsets.UTF_8; + import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import java.security.GeneralSecurityException; +import java.util.function.Function; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.util.ByteBufList; @@ -30,6 +34,8 @@ * Client utilities. */ public class ClientUtil { + public static final byte[] PASSWD = "foobar".getBytes(UTF_8); + public static ByteBuf generatePacket(long ledgerId, long entryId, long lastAddConfirmed, long length, byte[] data) throws GeneralSecurityException { return generatePacket(ledgerId, entryId, lastAddConfirmed, length, data, 0, data.length); @@ -49,4 +55,21 @@ public static boolean isLedgerOpen(LedgerHandle handle) { return !handle.getLedgerMetadata().isClosed(); } + public static LedgerMetadata setupLedger(ClientContext clientCtx, long ledgerId, + LedgerMetadataBuilder builder) throws Exception { + LedgerMetadata md = builder.withPassword(PASSWD).build(); + GenericCallbackFuture mdPromise = new GenericCallbackFuture<>(); + clientCtx.getLedgerManager().createLedgerMetadata(ledgerId, md, mdPromise); + return mdPromise.get(); + } + + public static LedgerMetadata transformMetadata(ClientContext clientCtx, long ledgerId, + Function transform) + throws Exception { + GenericCallbackFuture readPromise = new GenericCallbackFuture<>(); + GenericCallbackFuture writePromise = new GenericCallbackFuture<>(); + clientCtx.getLedgerManager().readLedgerMetadata(ledgerId, readPromise); + clientCtx.getLedgerManager().writeLedgerMetadata(ledgerId, transform.apply(readPromise.get()), writePromise); + return writePromise.get(); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/DeferredSyncTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/DeferredSyncTest.java index 95dec9c21a9..996c902ffe7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/DeferredSyncTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/DeferredSyncTest.java @@ -19,6 +19,7 @@ import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -213,7 +214,7 @@ public void testForbiddenEnsembleChange() throws Exception { // expected } LedgerHandle lh = (LedgerHandle) wh; - assertTrue(lh.getDelayedWriteFailedBookies().isEmpty()); + assertFalse(lh.hasDelayedWriteFailedBookies()); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java new file mode 100644 index 00000000000..dbcd8baef7c --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java @@ -0,0 +1,444 @@ +/* + * + * 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.client; + +import static org.apache.bookkeeper.util.TestUtils.assertEventuallyTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import com.google.common.collect.Lists; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import org.apache.bookkeeper.client.api.WriteFlag; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.net.BookieSocketAddress; + +import org.junit.Assert; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Ledger recovery tests using mocks rather than a real cluster. + */ +public class HandleFailuresTest { + private static final Logger log = LoggerFactory.getLogger(LedgerRecovery2Test.class); + + private static final BookieSocketAddress b1 = new BookieSocketAddress("b1", 3181); + private static final BookieSocketAddress b2 = new BookieSocketAddress("b2", 3181); + private static final BookieSocketAddress b3 = new BookieSocketAddress("b3", 3181); + private static final BookieSocketAddress b4 = new BookieSocketAddress("b4", 3181); + private static final BookieSocketAddress b5 = new BookieSocketAddress("b5", 3181); + + @Test + public void testChangeTriggeredOneTimeForOneFailure() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create().newEnsembleEntry( + 0L, Lists.newArrayList(b1, b2, b3))); + + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + clientCtx.getMockBookieClient().errorBookies(b1); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + lh.appendAsync("entry1".getBytes()); + lh.appendAsync("entry2".getBytes()); + lh.appendAsync("entry3".getBytes()); + lh.appendAsync("entry4".getBytes()); + lh.appendAsync("entry5".getBytes()).get(); + + verify(clientCtx.getLedgerManager(), times(1)).writeLedgerMetadata(anyLong(), any(), any()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); + } + + @Test + public void testSecondFailureOccursWhileFirstBeingHandled() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + + clientCtx.getMockRegistrationClient().addBookies(b4, b5).get(); + CompletableFuture b2blocker = new CompletableFuture<>(); + clientCtx.getMockBookieClient().setPreWriteHook( + (bookie, ledgerId, entryId) -> { + if (bookie.equals(b1)) { + return FutureUtils.exception(new BKException.BKWriteException()); + } else if (bookie.equals(b2)) { + return b2blocker; + } else { + return FutureUtils.value(null); + } + }); + CompletableFuture metadataNotifier = new CompletableFuture<>(); + CompletableFuture metadataBlocker = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook( + (ledgerId, metadata) -> { + metadataNotifier.complete(null); + return metadataBlocker; + }); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + lh.appendAsync("entry1".getBytes()); + lh.appendAsync("entry2".getBytes()); + lh.appendAsync("entry3".getBytes()); + lh.appendAsync("entry4".getBytes()); + CompletableFuture future = lh.appendAsync("entry5".getBytes()); + + metadataNotifier.get(); // wait for first metadata write to occur + b2blocker.completeExceptionally(new BKException.BKWriteException()); // make b2 requests fail + metadataBlocker.complete(null); + + future.get(); + verify(clientCtx.getLedgerManager(), times(2)).writeLedgerMetadata(anyLong(), any(), any()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b3)); + Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b4)); + Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b5)); + } + + @Test + public void testHandlingFailuresOneBookieFailsImmediately() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + clientCtx.getMockBookieClient().errorBookies(b1); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + lh.append("entry1".getBytes()); + lh.close(); + + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); + } + + @Test + public void testHandlingFailuresOneBookieFailsAfterOneEntry() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + lh.append("entry1".getBytes()); + clientCtx.getMockBookieClient().errorBookies(b1); + lh.append("entry2".getBytes()); + lh.close(); + + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b4, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), 1L); + } + + @Test + public void testHandlingFailuresMultipleBookieFailImmediatelyNotEnoughToReplace() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + clientCtx.getMockBookieClient().errorBookies(b1, b2); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + try { + lh.append("entry1".getBytes()); + Assert.fail("Shouldn't have been able to add"); + } catch (BKException.BKNotEnoughBookiesException bke) { + // correct behaviour + assertEventuallyTrue("Failure to add should trigger ledger closure", + () -> lh.getLedgerMetadata().isClosed()); + Assert.assertEquals("Ledger should be empty", + lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID); + Assert.assertEquals("Should be only one ensemble", lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals("Ensemble shouldn't have changed", lh.getLedgerMetadata().getEnsembles().get(0L), + Lists.newArrayList(b1, b2, b3)); + } + } + + @Test + public void testHandlingFailuresMultipleBookieFailAfterOneEntryNotEnoughToReplace() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + lh.append("entry1".getBytes()); + + clientCtx.getMockBookieClient().errorBookies(b1, b2); + + try { + lh.append("entry2".getBytes()); + Assert.fail("Shouldn't have been able to add"); + } catch (BKException.BKNotEnoughBookiesException bke) { + // correct behaviour + assertEventuallyTrue("Failure to add should trigger ledger closure", + () -> lh.getLedgerMetadata().isClosed()); + Assert.assertEquals("Ledger should be empty", lh.getLedgerMetadata().getLastEntryId(), 0L); + Assert.assertEquals("Should be only one ensemble", lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals("Ensemble shouldn't have changed", lh.getLedgerMetadata().getEnsembles().get(0L), + Lists.newArrayList(b1, b2, b3)); + } + } + + @Test + public void testClientClosesWhileFailureHandlerInProgress() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + clientCtx.getMockBookieClient().errorBookies(b2); + + CompletableFuture changeInProgress = new CompletableFuture<>(); + CompletableFuture blockEnsembleChange = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + if (metadata.getEnsembles().get(0L).get(1).equals(b4)) { // block the write trying to replace b2 with b4 + changeInProgress.complete(null); + return blockEnsembleChange; + } else { + return FutureUtils.value(null); + } + }); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + CompletableFuture future = lh.appendAsync("entry1".getBytes()); + changeInProgress.get(); + + lh.close(); + + blockEnsembleChange.complete(null); // allow ensemble change to continue + try { + future.get(); + Assert.fail("Add shouldn't have succeeded"); + } catch (ExecutionException ee) { + Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerClosedException.class); + } + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID); + } + + @Test + public void testMetadataSetToClosedDuringFailureHandler() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + clientCtx.getMockBookieClient().errorBookies(b2); + + CompletableFuture changeInProgress = new CompletableFuture<>(); + CompletableFuture blockEnsembleChange = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + if (metadata.getEnsembles().get(0L).get(1).equals(b4)) { // block the write trying to replace b2 with b4 + changeInProgress.complete(null); + return blockEnsembleChange; + } else { + return FutureUtils.value(null); + } + }); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + CompletableFuture future = lh.appendAsync("entry1".getBytes()); + changeInProgress.get(); + + ClientUtil.transformMetadata(clientCtx, 10L, + (metadata) -> LedgerMetadataBuilder.from(metadata).closingAt(1234L, 10L).build()); + + blockEnsembleChange.complete(null); // allow ensemble change to continue + try { + future.get(); + Assert.fail("Add shouldn't have succeeded"); + } catch (ExecutionException ee) { + Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerClosedException.class); + } + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), 1234L); + } + + @Test + public void testMetadataSetToInRecoveryDuringFailureHandler() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + clientCtx.getMockBookieClient().errorBookies(b2); + + CompletableFuture changeInProgress = new CompletableFuture<>(); + CompletableFuture blockEnsembleChange = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + if (metadata.getEnsembles().get(0L).get(1).equals(b4)) { // block the write trying to replace b2 with b4 + changeInProgress.complete(null); + return blockEnsembleChange; + } else { + return FutureUtils.value(null); + } + }); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + CompletableFuture future = lh.appendAsync("entry1".getBytes()); + changeInProgress.get(); + + ClientUtil.transformMetadata(clientCtx, 10L, + (metadata) -> LedgerMetadataBuilder.from(metadata).withInRecoveryState().build()); + + blockEnsembleChange.complete(null); // allow ensemble change to continue + try { + future.get(); + Assert.fail("Add shouldn't have succeeded"); + } catch (ExecutionException ee) { + Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerFencedException.class); + } + Assert.assertFalse(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + } + + @Test + public void testOldEnsembleChangedDuringFailureHandler() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + lh.append("entry1".getBytes()); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + clientCtx.getMockBookieClient().errorBookies(b3); + lh.append("entry2".getBytes()); + + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); + + + CompletableFuture changeInProgress = new CompletableFuture<>(); + CompletableFuture blockEnsembleChange = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + // block the write trying to replace b1 with b5 + if (metadata.getEnsembles().size() > 2 + && metadata.getEnsembles().get(2L).get(0).equals(b5)) { + changeInProgress.complete(null); + return blockEnsembleChange; + } else { + return FutureUtils.value(null); + } + }); + + clientCtx.getMockRegistrationClient().addBookies(b5).get(); + clientCtx.getMockBookieClient().errorBookies(b1); + + CompletableFuture future = lh.appendAsync("entry3".getBytes()); + changeInProgress.get(); + + ClientUtil.transformMetadata(clientCtx, 10L, + (metadata) -> LedgerMetadataBuilder.from(metadata).replaceEnsembleEntry( + 0L, Lists.newArrayList(b4, b2, b5)).build()); + + blockEnsembleChange.complete(null); // allow ensemble change to continue + future.get(); + + Assert.assertFalse(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 3); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b4, b2, b5)); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(2L), Lists.newArrayList(b5, b2, b4)); + } + + @Test + public void testNoAddsAreCompletedWhileFailureHandlingInProgress() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + clientCtx.getMockBookieClient().errorBookies(b3); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + lh.append("entry1".getBytes()); + + CompletableFuture changeInProgress = new CompletableFuture<>(); + CompletableFuture blockEnsembleChange = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + // block the write trying to replace b3 with b4 + if (metadata.getEnsembles().get(1L).get(2).equals(b4)) { + changeInProgress.complete(null); + return blockEnsembleChange; + } else { + return FutureUtils.value(null); + } + }); + + CompletableFuture future = lh.appendAsync("entry2".getBytes()); + changeInProgress.get(); + try { + future.get(1, TimeUnit.SECONDS); + Assert.fail("Shouldn't complete"); + } catch (TimeoutException te) { + } + blockEnsembleChange.complete(null); + future.get(); + + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java new file mode 100644 index 00000000000..0194e48a6c4 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java @@ -0,0 +1,269 @@ +/* + * 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.client; + +import com.google.common.collect.Lists; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.bookkeeper.client.api.WriteFlag; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.net.BookieSocketAddress; + +import org.junit.Assert; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Ledger recovery tests using mocks rather than a real cluster. + */ +public class LedgerClose2Test { + private static final Logger log = LoggerFactory.getLogger(LedgerRecovery2Test.class); + + private static final BookieSocketAddress b1 = new BookieSocketAddress("b1", 3181); + private static final BookieSocketAddress b2 = new BookieSocketAddress("b2", 3181); + private static final BookieSocketAddress b3 = new BookieSocketAddress("b3", 3181); + private static final BookieSocketAddress b4 = new BookieSocketAddress("b4", 3181); + private static final BookieSocketAddress b5 = new BookieSocketAddress("b5", 3181); + + @Test + public void testTryAddAfterCloseHasBeenCalled() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + + for (int i = 0; i < 1000; i++) { + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, i, + LedgerMetadataBuilder.create().newEnsembleEntry( + 0L, Lists.newArrayList(b1, b2, b3))); + LedgerHandle lh = new LedgerHandle(clientCtx, i, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + CompletableFuture closeFuture = lh.closeAsync(); + try { + long eid = lh.append("entry".getBytes()); + + // if it succeeds, it should be in final ledge + closeFuture.get(); + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), eid); + } catch (BKException.BKLedgerClosedException bke) { + closeFuture.get(); + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID); + } + } + } + + @Test + public void testMetadataChangedDuringClose() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + lh.append("entry1".getBytes()); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + clientCtx.getMockBookieClient().errorBookies(b3); + lh.append("entry2".getBytes()); + + CompletableFuture closeInProgress = new CompletableFuture<>(); + CompletableFuture blockClose = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + // block the write trying to replace b3 with b4 + if (metadata.isClosed()) { + closeInProgress.complete(null); + return blockClose; + } else { + return FutureUtils.value(null); + } + }); + CompletableFuture closeFuture = lh.closeAsync(); + closeInProgress.get(); + + ClientUtil.transformMetadata(clientCtx, 10L, + (metadata) -> LedgerMetadataBuilder.from(metadata).replaceEnsembleEntry( + 0L, Lists.newArrayList(b4, b2, b5)).build()); + + blockClose.complete(null); + closeFuture.get(); + + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b4, b2, b5)); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); + Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), 1L); + } + + @Test + public void testMetadataCloseWithCorrectLengthDuringClose() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + long lac = lh.append("entry1".getBytes()); + long length = lh.getLength(); + + CompletableFuture closeInProgress = new CompletableFuture<>(); + CompletableFuture blockClose = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + // block the write trying to do the first close + if (!closeInProgress.isDone() && metadata.isClosed()) { + closeInProgress.complete(null); + return blockClose; + } else { + return FutureUtils.value(null); + } + }); + CompletableFuture closeFuture = lh.closeAsync(); + closeInProgress.get(); + + ClientUtil.transformMetadata(clientCtx, 10L, + (metadata) -> LedgerMetadataBuilder.from(metadata).closingAt(lac, length).build()); + + blockClose.complete(null); + closeFuture.get(); + + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), lac); + Assert.assertEquals(lh.getLedgerMetadata().getLength(), length); + } + + @Test + public void testMetadataCloseWithDifferentLengthDuringClose() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + long lac = lh.append("entry1".getBytes()); + long length = lh.getLength(); + + CompletableFuture closeInProgress = new CompletableFuture<>(); + CompletableFuture blockClose = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + // block the write trying to do the first close + if (!closeInProgress.isDone() && metadata.isClosed()) { + closeInProgress.complete(null); + return blockClose; + } else { + return FutureUtils.value(null); + } + }); + CompletableFuture closeFuture = lh.closeAsync(); + closeInProgress.get(); + + /* close with different length. can happen in cases where there's a write outstanding */ + ClientUtil.transformMetadata(clientCtx, 10L, + (metadata) -> LedgerMetadataBuilder.from(metadata).closingAt(lac + 1, length + 100).build()); + + blockClose.complete(null); + try { + closeFuture.get(); + Assert.fail("Close should fail. Ledger has been closed in a state we don't know how to untangle"); + } catch (ExecutionException ee) { + Assert.assertEquals(ee.getCause().getClass(), BKException.BKMetadataVersionException.class); + } + } + + @Test + public void testMetadataCloseMarkedInRecoveryWhileClosing() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + long lac = lh.append("entry1".getBytes()); + long length = lh.getLength(); + + CompletableFuture closeInProgress = new CompletableFuture<>(); + CompletableFuture blockClose = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + // block the write trying to do the first close + if (metadata.isClosed()) { + closeInProgress.complete(null); + return blockClose; + } else { + return FutureUtils.value(null); + } + }); + CompletableFuture closeFuture = lh.closeAsync(); + closeInProgress.get(); + + ClientUtil.transformMetadata(clientCtx, 10L, + (metadata) -> LedgerMetadataBuilder.from(metadata).withInRecoveryState().build()); + + blockClose.complete(null); + + closeFuture.get(); // should override in recovery, since this handle knows what it has written + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), lac); + Assert.assertEquals(lh.getLedgerMetadata().getLength(), length); + } + + @Test + public void testCloseWhileAddInProgress() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + // block all entry writes from completing + CompletableFuture writesHittingBookies = new CompletableFuture<>(); + clientCtx.getMockBookieClient().setPreWriteHook((bookie, ledgerId, entryId) -> { + writesHittingBookies.complete(null); + return new CompletableFuture(); + }); + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + CompletableFuture future = lh.appendAsync("entry1".getBytes()); + writesHittingBookies.get(); + + lh.close(); + try { + future.get(); + Assert.fail("That write shouldn't have succeeded"); + } catch (ExecutionException ee) { + Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerClosedException.class); + } + Assert.assertTrue(lh.getLedgerMetadata().isClosed()); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID); + Assert.assertEquals(lh.getLedgerMetadata().getLength(), 0); + } +} + diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java index 535b97a4b59..89b008836e0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MdcContextTest.java @@ -172,7 +172,9 @@ public void testAddWithEnsembleChange() throws Exception { lh.addEntry(1, entry); assertLogWithMdc("ledger_add_entry", "Could not connect to bookie"); assertLogWithMdc("ledger_add_entry", "Failed to write entry"); - assertLogWithMdc("ledger_add_entry", "New Ensemble"); + //commented out until we figure out a way to preserve MDC through a call out + //to another thread pool + //assertLogWithMdc("ledger_add_entry", "New Ensemble"); } @Test @@ -197,7 +199,7 @@ public void testAddFailsWithReadOnlyBookie() throws Exception { assertLogWithMdc("ledger_add_entry", "Error writing entry:0 to ledger:0"); assertLogWithMdc("ledger_add_entry", "Add for failed on bookie"); assertLogWithMdc("ledger_add_entry", "Failed to find 1 bookies"); - assertLogWithMdc("ledger_add_entry", "Could not get additional bookie to remake ensemble, closing ledger: 0"); + assertLogWithMdc("ledger_add_entry", "Closing ledger 0 due to NotEnoughBookiesException"); } @Test diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java index f114cbf70f3..4a4599f04b0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java @@ -22,6 +22,7 @@ import static com.google.common.base.Charsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.FEATURE_DISABLE_ENSEMBLE_CHANGE; +import static org.apache.bookkeeper.util.TestUtils.assertEventuallyTrue; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -164,8 +165,9 @@ public void run() { } else { assertTrue("Should fail adding entries when enable ensemble change again.", failTest.get()); - assertTrue("Ledger should be closed when enable ensemble change again.", - lh.getLedgerMetadata().isClosed()); + // The ledger close occurs in the background, so assert that it happens eventually + assertEventuallyTrue("Ledger should be closed when enable ensemble change again.", + () -> lh.getLedgerMetadata().isClosed()); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java index d05f864b538..48f638c3334 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java @@ -38,11 +38,14 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Test read last confirmed long by polling. */ @RunWith(Parameterized.class) public class TestReadLastConfirmedLongPoll extends BookKeeperClusterTestCase { + private static final Logger log = LoggerFactory.getLogger(TestReadLastConfirmedLongPoll.class); final DigestType digestType; public TestReadLastConfirmedLongPoll(Class storageClass) { @@ -153,7 +156,7 @@ public void testReadLACLongPollWhenSomeBookiesDown() throws Exception { ServerConfiguration[] confs = new ServerConfiguration[numEntries - 1]; for (int j = 0; j < numEntries - 1; j++) { int idx = (i + 1 + j) % numEntries; - confs[j] = killBookie(lh.getCurrentEnsemble().get(idx)); + confs[j] = killBookie(lh.getLedgerMetadata().getLastEnsembleValue().get(idx)); } final AtomicBoolean entryAsExpected = new AtomicBoolean(false); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index 4586e09842e..ea04192f88f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -21,18 +21,16 @@ import com.google.common.base.Optional; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.function.Consumer; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerMetadata; - +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; @@ -51,12 +49,17 @@ public class MockLedgerManager implements LedgerManager { static final Logger LOG = LoggerFactory.getLogger(MockLedgerManager.class); - boolean stallingWrites = false; - final List> stalledWrites = new ArrayList<>(); + /** + * Hook for injecting errors or delays. + */ + public interface Hook { + CompletableFuture runHook(long ledgerId, LedgerMetadata metadata); + } final Map> metadataMap; final ExecutorService executor; final boolean ownsExecutor; + private Hook preWriteHook = (ledgerId, metadata) -> FutureUtils.value(null); public MockLedgerManager() { this(new HashMap<>(), @@ -83,23 +86,8 @@ private LedgerMetadata readMetadata(long ledgerId) throws Exception { } } - public void stallWrites() throws Exception { - synchronized (this) { - stallingWrites = true; - } - } - - public void releaseStalledWrites(int rc) { - List> toRelease; - synchronized (this) { - stallingWrites = false; - toRelease = new ArrayList<>(stalledWrites); - stalledWrites.clear(); - } - - executor.execute(() -> { - toRelease.forEach(w -> w.accept(rc)); - }); + public void setPreWriteHook(Hook hook) { + this.preWriteHook = hook; } public void executeCallback(Runnable r) { @@ -147,42 +135,35 @@ public void readLedgerMetadata(long ledgerId, GenericCallback cb @Override public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback cb) { - Runnable write = () -> { - try { - LedgerMetadata oldMetadata = readMetadata(ledgerId); - if (oldMetadata == null) { - executeCallback(() -> cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null)); - } else if (!oldMetadata.getVersion().equals(metadata.getVersion())) { - executeCallback(() -> cb.operationComplete(BKException.Code.MetadataVersionException, null)); - } else { - LongVersion oldVersion = (LongVersion) oldMetadata.getVersion(); - metadataMap.put(ledgerId, Pair.of(new LongVersion(oldVersion.getLongVersion() + 1), - metadata.serialize())); - LedgerMetadata readBack = readMetadata(ledgerId); - executeCallback(() -> cb.operationComplete(BKException.Code.OK, readBack)); - } - } catch (Exception e) { - LOG.error("Error writing metadata", e); - executeCallback(() -> cb.operationComplete(BKException.Code.MetaStoreException, null)); - } - }; - - synchronized (this) { - if (stallingWrites) { - LOG.info("[L{}, stallId={}] Stalling write of metadata", ledgerId, System.identityHashCode(write)); - stalledWrites.add((rc) -> { - LOG.info("[L{}, stallid={}] Unstalled write", ledgerId, System.identityHashCode(write)); - - if (rc == BKException.Code.OK) { - write.run(); + preWriteHook.runHook(ledgerId, metadata) + .thenComposeAsync((ignore) -> { + try { + LedgerMetadata oldMetadata = readMetadata(ledgerId); + if (oldMetadata == null) { + return FutureUtils.exception(new BKException.BKNoSuchLedgerExistsException()); + } else if (!oldMetadata.getVersion().equals(metadata.getVersion())) { + return FutureUtils.exception(new BKException.BKMetadataVersionException()); } else { - executeCallback(() -> cb.operationComplete(rc, null)); + LongVersion oldVersion = (LongVersion) oldMetadata.getVersion(); + metadataMap.put(ledgerId, Pair.of(new LongVersion(oldVersion.getLongVersion() + 1), + metadata.serialize())); + LedgerMetadata readBack = readMetadata(ledgerId); + return FutureUtils.value(readBack); } - }); - } else { - executor.execute(write); - } - } + } catch (Exception e) { + LOG.error("Error writing metadata", e); + return FutureUtils.exception(e); + } + }, executor) + .whenComplete((res, ex) -> { + if (ex != null) { + executeCallback(() -> cb.operationComplete( + BKException.getExceptionCode(ex, BKException.Code.MetaStoreException), + null)); + } else { + executeCallback(() -> cb.operationComplete(BKException.Code.OK, res)); + } + }); } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConditionalSetTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConditionalSetTest.java index b6d1153d16e..285d39d5db0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConditionalSetTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConditionalSetTest.java @@ -20,8 +20,6 @@ */ package org.apache.bookkeeper.test; -import static org.junit.Assert.fail; - import java.io.IOException; import org.apache.bookkeeper.client.BKException; @@ -92,15 +90,9 @@ public void testConditionalSet() throws IOException, InterruptedException, LOG.debug("Opened the ledger already"); /* - * Writer tries to close the ledger, and if should fail. + * Writer tries to close the ledger, and it should succeed as recovery closed + * the ledger already, but with the correct LAC and length */ - try { - lhWrite.close(); - fail("Should have received an exception when trying to close the ledger."); - } catch (BKException e) { - /* - * Correctly failed to close the ledger - */ - } + lhWrite.close(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java index 26b24482067..352560775c0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java @@ -24,13 +24,16 @@ import java.io.File; import java.util.HashSet; import java.util.Set; - import java.util.concurrent.TimeUnit; +import java.util.function.BooleanSupplier; + import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.ReadHandle; +import org.junit.Assert; + /** * Test utilities. */ @@ -77,4 +80,16 @@ public static long waitUntilExplicitLacUpdated(LedgerHandle rh, long newLac) thr return lac; } + public static void assertEventuallyTrue(String description, BooleanSupplier predicate) throws Exception { + assertEventuallyTrue(description, predicate, 10, TimeUnit.SECONDS); + } + + public static void assertEventuallyTrue(String description, BooleanSupplier predicate, + long duration, TimeUnit unit) throws Exception { + long iterations = unit.toMillis(duration) / 100; + for (int i = 0; i < iterations && !predicate.getAsBoolean(); i++) { + Thread.sleep(100); + } + Assert.assertTrue(description, predicate.getAsBoolean()); + } } From 27be45ec93471b9a75834a9cc9e1cb8120c9bcd7 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Sun, 28 Oct 2018 13:54:53 +0100 Subject: [PATCH 0090/1642] Remove LedgerMetadata#getEnsembles in favour of #getAllEnsembles They do the same thing except that #getEnsembles returns a TreeMap while #getAllEnsembles returns a NavigableMap. LedgerMetadata#getAllEnsembles is part of the api interface. Master issue: #281 Reviewers: Enrico Olivelli , Sijie Guo This closes #1759 from ivankelly/remove-get-ensembles --- .../apache/bookkeeper/bookie/BookieShell.java | 3 +- .../ScanAndCompareGarbageCollector.java | 2 +- .../bookkeeper/client/BookKeeperAdmin.java | 16 ++-- .../bookkeeper/client/LedgerChecker.java | 2 +- .../bookkeeper/client/LedgerFragment.java | 2 +- .../client/LedgerFragmentReplicator.java | 4 +- .../bookkeeper/client/LedgerHandle.java | 4 +- .../bookkeeper/client/LedgerMetadata.java | 11 --- .../client/LedgerMetadataBuilder.java | 2 +- .../bookkeeper/client/UpdateLedgerOp.java | 4 +- .../replication/BookieLedgerIndexer.java | 2 +- .../replication/ReplicationWorker.java | 2 +- .../bookie/TestGcOverreplicatedLedger.java | 4 +- .../bookkeeper/client/BookieRecoveryTest.java | 12 +-- .../client/BookieWriteLedgerTest.java | 18 ++--- .../bookkeeper/client/HandleFailuresTest.java | 77 ++++++++++--------- .../bookkeeper/client/LedgerClose2Test.java | 18 ++--- .../client/LedgerRecovery2Test.java | 18 ++--- .../client/TestDelayEnsembleChange.java | 28 +++---- .../client/TestDisableEnsembleChange.java | 10 +-- .../apache/bookkeeper/client/TestFencing.java | 2 +- .../bookkeeper/client/TestLedgerChecker.java | 26 +++---- .../client/TestLedgerFragmentReplication.java | 12 +-- .../client/TestMaxEnsembleChangeNum.java | 2 +- .../bookkeeper/client/TestSequenceRead.java | 2 +- .../client/TestSpeculativeRead.java | 18 ++--- .../bookkeeper/client/UpdateLedgerOpTest.java | 2 +- .../client/api/BookKeeperBuildersTest.java | 4 +- .../AuditorPeriodicBookieCheckTest.java | 2 +- .../replication/AuditorPeriodicCheckTest.java | 8 +- .../replication/BookieAutoRecoveryTest.java | 30 ++++---- ...estAutoRecoveryAlongWithBookieServers.java | 4 +- .../replication/TestReplicationWorker.java | 29 +++---- .../server/http/TestHttpService.java | 2 +- .../org/apache/bookkeeper/tls/TestTLS.java | 4 +- .../bookkeeper/client/LedgerReader.java | 2 +- .../TestCompatRecoveryNoPassword.groovy | 2 +- 37 files changed, 185 insertions(+), 205 deletions(-) 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 2829a730d3e..18ce8c43924 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 @@ -589,7 +589,8 @@ private int bkQuery(BookKeeperAdmin bkAdmin, Set bookieAddr private Map inspectLedger(LedgerMetadata metadata, Set bookiesToInspect) { Map numBookiesToReplacePerEnsemble = new TreeMap(); - for (Map.Entry> ensemble : metadata.getEnsembles().entrySet()) { + for (Map.Entry> ensemble : + metadata.getAllEnsembles().entrySet()) { List bookieList = ensemble.getValue(); System.out.print(ensemble.getKey() + ":\t"); int numBookiesToReplace = 0; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index 137c07cd55e..33f10e901a4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -248,7 +248,7 @@ public void operationComplete(int rc, LedgerMetadata ledgerMetadata) { return; } SortedMap> ensembles = - ledgerMetadata.getEnsembles(); + ledgerMetadata.getAllEnsembles(); for (List ensemble : ensembles.values()) { // check if this bookie is supposed to have this ledger if (ensemble.contains(selfBookieAddress)) { 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 e5b59143548..f637889be7f 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 @@ -818,8 +818,8 @@ public void processResult(int rc, String path, Object ctx) { */ Map ledgerFragmentsRange = new HashMap(); Long curEntryId = null; - for (Map.Entry> entry : lh.getLedgerMetadata().getEnsembles() - .entrySet()) { + for (Map.Entry> entry : + lh.getLedgerMetadata().getAllEnsembles().entrySet()) { if (curEntryId != null) { ledgerFragmentsRange.put(curEntryId, entry.getKey() - 1); } @@ -864,7 +864,7 @@ public void processResult(int rc, String path, Object ctx) { */ for (final Long startEntryId : ledgerFragmentsToRecover) { Long endEntryId = ledgerFragmentsRange.get(startEntryId); - List ensemble = lh.getLedgerMetadata().getEnsembles().get(startEntryId); + List ensemble = lh.getLedgerMetadata().getAllEnsembles().get(startEntryId); // Get bookies to replace Map targetBookieAddresses; try { @@ -1090,11 +1090,11 @@ private static Map getReplacementBooki private static boolean containBookiesInLastEnsemble(LedgerMetadata lm, Set bookies) { - if (lm.getEnsembles().size() <= 0) { + if (lm.getAllEnsembles().size() <= 0) { return false; } - Long lastKey = lm.getEnsembles().lastKey(); - List lastEnsemble = lm.getEnsembles().get(lastKey); + Long lastKey = lm.getAllEnsembles().lastKey(); + List lastEnsemble = lm.getAllEnsembles().get(lastKey); return containBookies(lastEnsemble, bookies); } @@ -1546,7 +1546,7 @@ public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieS public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieSocketAddress bookieAddress, LedgerMetadata ledgerMetadata) { - Collection> ensemblesOfSegments = ledgerMetadata.getEnsembles().values(); + Collection> ensemblesOfSegments = ledgerMetadata.getAllEnsembles().values(); Iterator> ensemblesOfSegmentsIterator = ensemblesOfSegments.iterator(); List ensemble; int segmentNo = 0; @@ -1568,7 +1568,7 @@ private static boolean areEntriesOfSegmentStoredInTheBookie(LedgerMetadata ledge int writeQuorumSize = ledgerMetadata.getWriteQuorumSize(); List>> segments = - new LinkedList<>(ledgerMetadata.getEnsembles().entrySet()); + new LinkedList<>(ledgerMetadata.getAllEnsembles().entrySet()); boolean lastSegment = (segmentNo == (segments.size() - 1)); 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 cbd99767764..b9745a9371c 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 @@ -323,7 +323,7 @@ public void checkLedger(final LedgerHandle lh, Long curEntryId = null; List curEnsemble = null; for (Map.Entry> e : lh - .getLedgerMetadata().getEnsembles().entrySet()) { + .getLedgerMetadata().getAllEnsembles().entrySet()) { if (curEntryId != null) { Set bookieIndexes = new HashSet(); for (int i = 0; i < curEnsemble.size(); i++) { 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 49b8de26690..e3224fdea99 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 @@ -51,7 +51,7 @@ public class LedgerFragment { this.ensemble = lh.getLedgerMetadata().getEnsemble(firstEntryId); this.schedule = lh.getDistributionSchedule(); SortedMap> ensembles = lh - .getLedgerMetadata().getEnsembles(); + .getLedgerMetadata().getAllEnsembles(); this.isLedgerClosed = lh.getLedgerMetadata().isClosed() || !ensemble.equals(ensembles.get(ensembles.lastKey())); } 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 dc7302c46f1..76a9178db61 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 @@ -378,12 +378,12 @@ private static void updateEnsembleInfo( lh::getLedgerMetadata, (metadata) -> { // returns true if any of old bookies exist in ensemble - List ensemble = metadata.getEnsembles().get(fragmentStartId); + List ensemble = metadata.getAllEnsembles().get(fragmentStartId); return oldBookie2NewBookie.keySet().stream().anyMatch(ensemble::contains); }, (currentMetadata) -> { // replace all old bookies with new bookies in ensemble - List newEnsemble = currentMetadata.getEnsembles().get(fragmentStartId) + List newEnsemble = currentMetadata.getAllEnsembles().get(fragmentStartId) .stream().map((bookie) -> oldBookie2NewBookie.getOrDefault(bookie, bookie)) .collect(Collectors.toList()); return LedgerMetadataBuilder.from(currentMetadata) 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 a340cc7f6c6..6125a3d1b92 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 @@ -344,7 +344,7 @@ public Map getCustomMetadata() { * @return the count of fragments */ public synchronized long getNumFragments() { - return getLedgerMetadata().getEnsembles().size(); + return getLedgerMetadata().getAllEnsembles().size(); } /** @@ -354,7 +354,7 @@ public synchronized long getNumFragments() { * @return count of unique bookies */ public synchronized long getNumBookies() { - Map> m = getLedgerMetadata().getEnsembles(); + Map> m = getLedgerMetadata().getAllEnsembles(); Set s = Sets.newHashSet(); for (List aList : m.values()) { s.addAll(aList); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 8c5e3b8515c..02e0ede38d0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -217,17 +217,6 @@ private LedgerMetadata() { this.hasPassword = false; } - /** - * Get the Map of bookie ensembles for the various ledger fragments - * that make up the ledger. - * - * @return SortedMap of Ledger Fragments and the corresponding - * bookie ensembles that store the entries. - */ - public TreeMap> getEnsembles() { - return ensembles; - } - @Override public NavigableMap> getAllEnsembles() { return ensembles; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index ae78d5f2652..b1b9dbdd3ac 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -74,7 +74,7 @@ static LedgerMetadataBuilder from(LedgerMetadata other) { builder.length = Optional.of(length); } - builder.ensembles.putAll(other.getEnsembles()); + builder.ensembles.putAll(other.getAllEnsembles()); builder.digestType = other.getDigestType(); if (other.hasPassword()) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java index befd4f37248..37f8b096d6c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java @@ -102,7 +102,7 @@ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final lm, ledgerId, ref::get, (metadata) -> { - return metadata.getEnsembles().values().stream() + return metadata.getAllEnsembles().values().stream() .flatMap(Collection::stream) .filter(b -> b.equals(oldBookieId)) .count() > 0; @@ -161,7 +161,7 @@ private static LedgerMetadata replaceBookieInEnsembles(LedgerMetadata metadata, BookieSocketAddress oldBookieId, BookieSocketAddress newBookieId) { LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata); - for (Map.Entry> e : metadata.getEnsembles().entrySet()) { + for (Map.Entry> e : metadata.getAllEnsembles().entrySet()) { List newEnsemble = e.getValue().stream() .map(b -> b.equals(oldBookieId) ? newBookieId : b) .collect(Collectors.toList()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java index 027081a08f6..80bc1a5c9c4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java @@ -74,7 +74,7 @@ public void operationComplete(int rc, LedgerMetadata ledgerMetadata) { if (rc == BKException.Code.OK) { for (Map.Entry> ensemble : ledgerMetadata - .getEnsembles().entrySet()) { + .getAllEnsembles().entrySet()) { for (BookieSocketAddress bookie : ensemble .getValue()) { putLedger(bookie2ledgersMap, 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 e3f9ac679d3..6339810b04f 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 @@ -364,7 +364,7 @@ private boolean isLastSegmentOpenAndMissingBookies(LedgerHandle lh) throws BKExc return false; } - SortedMap> ensembles = admin.getLedgerMetadata(lh).getEnsembles(); + SortedMap> ensembles = admin.getLedgerMetadata(lh).getAllEnsembles(); List finalEnsemble = ensembles.get(ensembles.lastKey()); Collection available = admin.getAvailableBookies(); for (BookieSocketAddress b : finalEnsemble) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java index 9feb9df1025..1796319988b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java @@ -148,7 +148,7 @@ public void operationComplete(int rc, LedgerMetadata result) { Assert.fail("No ledger metadata found"); } BookieSocketAddress address = null; - SortedMap> ensembleMap = newLedgerMetadata.get().getEnsembles(); + SortedMap> ensembleMap = newLedgerMetadata.get().getAllEnsembles(); for (List ensemble : ensembleMap.values()) { address = ensemble.get(0); } @@ -235,7 +235,7 @@ private BookieSocketAddress getBookieNotInEnsemble(LedgerMetadata ledgerMetadata for (BookieServer bk : bs) { allAddresses.add(bk.getLocalAddress()); } - SortedMap> ensembles = ledgerMetadata.getEnsembles(); + SortedMap> ensembles = ledgerMetadata.getAllEnsembles(); for (List fragmentEnsembles : ensembles.values()) { allAddresses.removeAll(fragmentEnsembles); } 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 b1ad88abfb1..e657195e44c 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 @@ -506,7 +506,7 @@ long await() throws InterruptedException { private boolean verifyFullyReplicated(LedgerHandle lh, long untilEntry) throws Exception { LedgerMetadata md = getLedgerMetadata(lh); - Map> ensembles = md.getEnsembles(); + Map> ensembles = md.getAllEnsembles(); HashMap ranges = new HashMap(); ArrayList keyList = Collections.list( @@ -586,7 +586,7 @@ private boolean findDupesInEnsembles(List lhs) throws Exception { long numDupes = 0; for (LedgerHandle lh : lhs) { LedgerMetadata md = getLedgerMetadata(lh); - for (Map.Entry> e : md.getEnsembles().entrySet()) { + for (Map.Entry> e : md.getAllEnsembles().entrySet()) { HashSet set = new HashSet(); long fragment = e.getKey(); @@ -619,7 +619,7 @@ public void testBookieRecoveryOnClosedLedgers() throws Exception { closeLedgers(lhs); // Shutdown last bookie server in last ensemble - List lastEnsemble = lhs.get(0).getLedgerMetadata().getEnsembles() + List lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles() .entrySet().iterator().next().getValue(); BookieSocketAddress bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); killBookie(bookieToKill); @@ -648,7 +648,7 @@ public void testBookieRecoveryOnOpenedLedgers() throws Exception { writeEntriestoLedgers(numMsgs, 0, lhs); // Shutdown the first bookie server - List lastEnsemble = lhs.get(0).getLedgerMetadata().getEnsembles() + List lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles() .entrySet().iterator().next().getValue(); BookieSocketAddress bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); killBookie(bookieToKill); @@ -684,7 +684,7 @@ public void testBookieRecoveryOnInRecoveryLedger() throws Exception { writeEntriestoLedgers(numMsgs, 0, lhs); // Shutdown the first bookie server - List lastEnsemble = lhs.get(0).getLedgerMetadata().getEnsembles() + List lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles() .entrySet().iterator().next().getValue(); // removed bookie BookieSocketAddress bookieToKill = lastEnsemble.get(0); @@ -728,7 +728,7 @@ public void testBookieRecoveryOnInRecoveryLedger() throws Exception { for (LedgerHandle newLh : newLhs) { // first ensemble should contains bookieToKill2 and not contain bookieToKill Map.Entry> entry = - newLh.getLedgerMetadata().getEnsembles().entrySet().iterator().next(); + newLh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next(); assertFalse(entry.getValue().contains(bookieToKill)); assertTrue(entry.getValue().contains(bookieToKill2)); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java index 82a87c3d934..52360faa2d6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java @@ -151,7 +151,7 @@ public void testWithMultipleBookieFailuresInLastEnsemble() throws Exception { // Shutdown three bookies in the last ensemble and continue writing List ensemble = lh.getLedgerMetadata() - .getEnsembles().entrySet().iterator().next().getValue(); + .getAllEnsembles().entrySet().iterator().next().getValue(); killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); killBookie(ensemble.get(2)); @@ -196,7 +196,7 @@ public void testWriteAndReadStats() throws Exception { CountDownLatch sleepLatch1 = new CountDownLatch(1); CountDownLatch sleepLatch2 = new CountDownLatch(1); List ensemble = lh.getLedgerMetadata() - .getEnsembles().entrySet().iterator().next().getValue(); + .getAllEnsembles().entrySet().iterator().next().getValue(); sleepBookie(ensemble.get(0), sleepLatch1); @@ -389,7 +389,7 @@ public void testLedgerCreateAdv() throws Exception { startNewBookie(); // Shutdown one bookie in the last ensemble and continue writing - List ensemble = lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() .getValue(); killBookie(ensemble.get(0)); @@ -528,7 +528,7 @@ public void testLedgerCreateAdvWithLedgerId() throws Exception { startNewBookie(); // Shutdown one bookie in the last ensemble and continue writing - List ensemble = lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() .getValue(); killBookie(ensemble.get(0)); @@ -847,7 +847,7 @@ public void testAsyncWritesWithMultipleFailuresInLastEnsemble() // Shutdown three bookies in the last ensemble and continue writing List ensemble = lh.getLedgerMetadata() - .getEnsembles().entrySet().iterator().next().getValue(); + .getAllEnsembles().entrySet().iterator().next().getValue(); killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); killBookie(ensemble.get(2)); @@ -919,7 +919,7 @@ public void testLedgerCreateAdvWithAsyncWritesWithBookieFailures() throws Except } // Start One more bookie and shutdown one from last ensemble before reading startNewBookie(); - List ensemble = lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() .getValue(); killBookie(ensemble.get(0)); @@ -989,7 +989,7 @@ public void testLedgerHandleAdvOutOfOrderWriteAndFrocedEnsembleChange() throws E CountDownLatch sleepLatch1 = new CountDownLatch(1); List ensemble; - ensemble = lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next().getValue(); + ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next().getValue(); // Put all 3 bookies to sleep and start 3 new ones sleepBookie(ensemble.get(0), sleepLatch1); @@ -1073,7 +1073,7 @@ public void testLedgerCreateAdvWithRandomAsyncWritesWithBookieFailuresBetweenWri if (j == numEntriesToWrite / 2) { // Start One more bookie and shutdown one from last ensemble at half-way startNewBookie(); - List ensemble = lh.getLedgerMetadata().getEnsembles().entrySet() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet() .iterator().next().getValue(); killBookie(ensemble.get(0)); } @@ -1142,7 +1142,7 @@ public void testLedgerCreateAdvWithRandomAsyncWritesWithBookieFailures() throws } // Start One more bookie and shutdown one from last ensemble before reading startNewBookie(); - List ensemble = lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() .getValue(); killBookie(ensemble.get(0)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java index dbcd8baef7c..ce266809876 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java @@ -74,8 +74,8 @@ public void testChangeTriggeredOneTimeForOneFailure() throws Exception { lh.appendAsync("entry5".getBytes()).get(); verify(clientCtx.getLedgerManager(), times(1)).writeLedgerMetadata(anyLong(), any(), any()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); } @Test @@ -120,10 +120,10 @@ public void testSecondFailureOccursWhileFirstBeingHandled() throws Exception { future.get(); verify(clientCtx.getLedgerManager(), times(2)).writeLedgerMetadata(anyLong(), any(), any()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b3)); - Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b4)); - Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b5)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b3)); + Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b4)); + Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b5)); } @Test @@ -142,8 +142,8 @@ public void testHandlingFailuresOneBookieFailsImmediately() throws Exception { lh.close(); Assert.assertTrue(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); } @Test @@ -163,9 +163,9 @@ public void testHandlingFailuresOneBookieFailsAfterOneEntry() throws Exception { lh.close(); Assert.assertTrue(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b4, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(1L), Lists.newArrayList(b4, b2, b3)); Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), 1L); } @@ -189,8 +189,8 @@ public void testHandlingFailuresMultipleBookieFailImmediatelyNotEnoughToReplace( () -> lh.getLedgerMetadata().isClosed()); Assert.assertEquals("Ledger should be empty", lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID); - Assert.assertEquals("Should be only one ensemble", lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals("Ensemble shouldn't have changed", lh.getLedgerMetadata().getEnsembles().get(0L), + Assert.assertEquals("Should be only one ensemble", lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals("Ensemble shouldn't have changed", lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); } } @@ -217,8 +217,8 @@ public void testHandlingFailuresMultipleBookieFailAfterOneEntryNotEnoughToReplac assertEventuallyTrue("Failure to add should trigger ledger closure", () -> lh.getLedgerMetadata().isClosed()); Assert.assertEquals("Ledger should be empty", lh.getLedgerMetadata().getLastEntryId(), 0L); - Assert.assertEquals("Should be only one ensemble", lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals("Ensemble shouldn't have changed", lh.getLedgerMetadata().getEnsembles().get(0L), + Assert.assertEquals("Should be only one ensemble", lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals("Ensemble shouldn't have changed", lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); } } @@ -236,7 +236,8 @@ public void testClientClosesWhileFailureHandlerInProgress() throws Exception { CompletableFuture changeInProgress = new CompletableFuture<>(); CompletableFuture blockEnsembleChange = new CompletableFuture<>(); clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { - if (metadata.getEnsembles().get(0L).get(1).equals(b4)) { // block the write trying to replace b2 with b4 + // block the write trying to replace b2 with b4 + if (metadata.getAllEnsembles().get(0L).get(1).equals(b4)) { changeInProgress.complete(null); return blockEnsembleChange; } else { @@ -259,8 +260,8 @@ public void testClientClosesWhileFailureHandlerInProgress() throws Exception { Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerClosedException.class); } Assert.assertTrue(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID); } @@ -277,7 +278,8 @@ public void testMetadataSetToClosedDuringFailureHandler() throws Exception { CompletableFuture changeInProgress = new CompletableFuture<>(); CompletableFuture blockEnsembleChange = new CompletableFuture<>(); clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { - if (metadata.getEnsembles().get(0L).get(1).equals(b4)) { // block the write trying to replace b2 with b4 + if (metadata.getAllEnsembles().get(0L).get(1).equals(b4)) { + // block the write trying to replace b2 with b4 changeInProgress.complete(null); return blockEnsembleChange; } else { @@ -301,8 +303,8 @@ public void testMetadataSetToClosedDuringFailureHandler() throws Exception { Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerClosedException.class); } Assert.assertTrue(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), 1234L); } @@ -319,7 +321,8 @@ public void testMetadataSetToInRecoveryDuringFailureHandler() throws Exception { CompletableFuture changeInProgress = new CompletableFuture<>(); CompletableFuture blockEnsembleChange = new CompletableFuture<>(); clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { - if (metadata.getEnsembles().get(0L).get(1).equals(b4)) { // block the write trying to replace b2 with b4 + if (metadata.getAllEnsembles().get(0L).get(1).equals(b4)) { + // block the write trying to replace b2 with b4 changeInProgress.complete(null); return blockEnsembleChange; } else { @@ -343,8 +346,8 @@ public void testMetadataSetToInRecoveryDuringFailureHandler() throws Exception { Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerFencedException.class); } Assert.assertFalse(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); } @Test @@ -362,17 +365,17 @@ public void testOldEnsembleChangedDuringFailureHandler() throws Exception { clientCtx.getMockBookieClient().errorBookies(b3); lh.append("entry2".getBytes()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); CompletableFuture changeInProgress = new CompletableFuture<>(); CompletableFuture blockEnsembleChange = new CompletableFuture<>(); clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { // block the write trying to replace b1 with b5 - if (metadata.getEnsembles().size() > 2 - && metadata.getEnsembles().get(2L).get(0).equals(b5)) { + if (metadata.getAllEnsembles().size() > 2 + && metadata.getAllEnsembles().get(2L).get(0).equals(b5)) { changeInProgress.complete(null); return blockEnsembleChange; } else { @@ -394,10 +397,10 @@ public void testOldEnsembleChangedDuringFailureHandler() throws Exception { future.get(); Assert.assertFalse(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 3); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b4, b2, b5)); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(2L), Lists.newArrayList(b5, b2, b4)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 3); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b4, b2, b5)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(2L), Lists.newArrayList(b5, b2, b4)); } @Test @@ -419,7 +422,7 @@ public void testNoAddsAreCompletedWhileFailureHandlingInProgress() throws Except CompletableFuture blockEnsembleChange = new CompletableFuture<>(); clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { // block the write trying to replace b3 with b4 - if (metadata.getEnsembles().get(1L).get(2).equals(b4)) { + if (metadata.getAllEnsembles().get(1L).get(2).equals(b4)) { changeInProgress.complete(null); return blockEnsembleChange; } else { @@ -437,8 +440,8 @@ public void testNoAddsAreCompletedWhileFailureHandlingInProgress() throws Except blockEnsembleChange.complete(null); future.get(); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java index 0194e48a6c4..f2db821dfdd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java @@ -106,9 +106,9 @@ public void testMetadataChangedDuringClose() throws Exception { closeFuture.get(); Assert.assertTrue(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b4, b2, b5)); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b4, b2, b5)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), 1L); } @@ -147,8 +147,8 @@ public void testMetadataCloseWithCorrectLengthDuringClose() throws Exception { closeFuture.get(); Assert.assertTrue(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), lac); Assert.assertEquals(lh.getLedgerMetadata().getLength(), length); } @@ -228,8 +228,8 @@ public void testMetadataCloseMarkedInRecoveryWhileClosing() throws Exception { closeFuture.get(); // should override in recovery, since this handle knows what it has written Assert.assertTrue(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), lac); Assert.assertEquals(lh.getLedgerMetadata().getLength(), length); } @@ -260,8 +260,8 @@ public void testCloseWhileAddInProgress() throws Exception { Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerClosedException.class); } Assert.assertTrue(lh.getLedgerMetadata().isClosed()); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID); Assert.assertEquals(lh.getLedgerMetadata().getLength(), 0); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java index 9a7cc7fa880..2dba8c3212c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java @@ -209,8 +209,8 @@ public void testRecoveryBookieFailedAtStart() throws Exception { lh.recover(recoveryPromise, null, false); recoveryPromise.get(); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b4, b3)); } @@ -238,10 +238,10 @@ public void testRecoveryOneBookieFailsDuring() throws Exception { lh.recover(recoveryPromise, null, false); recoveryPromise.get(); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 2); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(0L), + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 2); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().get(1L), + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(1L), Lists.newArrayList(b1, b4, b3)); Assert.assertEquals(lh.getLastAddConfirmed(), 1L); } @@ -269,10 +269,10 @@ public void testRecoveryTwoBookiesFailOnSameEntry() throws Exception { lh.recover(recoveryPromise, null, false); recoveryPromise.get(); - Assert.assertEquals(lh.getLedgerMetadata().getEnsembles().size(), 1); - Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b3)); - Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b4)); - Assert.assertTrue(lh.getLedgerMetadata().getEnsembles().get(0L).contains(b5)); + Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); + Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b3)); + Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b4)); + Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b5)); Assert.assertEquals(lh.getLastAddConfirmed(), 0L); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java index e8da86e0f72..c76a75c3cfd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java @@ -155,7 +155,7 @@ public void testNotChangeEnsembleIfNotBrokenAckQuorum() throws Exception { // ensure there is no ensemble changed assertEquals("There should be no ensemble change if delaying ensemble change is enabled.", - 1, lh.getLedgerMetadata().getEnsembles().size()); + 1, lh.getLedgerMetadata().getAllEnsembles().size()); bsConfs.add(conf0); bs.add(startBookie(conf0)); @@ -168,7 +168,7 @@ public void testNotChangeEnsembleIfNotBrokenAckQuorum() throws Exception { // ensure there is no ensemble changed assertEquals("There should be no ensemble change if delaying ensemble change is enabled.", - 1, lh.getLedgerMetadata().getEnsembles().size()); + 1, lh.getLedgerMetadata().getAllEnsembles().size()); // check entries verifyEntries(lh, 0, numEntries, 5, 0); @@ -192,7 +192,7 @@ public void testChangeEnsembleIfBrokenAckQuorum() throws Exception { lh.addEntry(data); } - for (BookieSocketAddress addr : lh.getLedgerMetadata().getEnsembles().get(0L)) { + for (BookieSocketAddress addr : lh.getLedgerMetadata().getAllEnsembles().get(0L)) { assertTrue( LEDGER_ENSEMBLE_BOOKIE_DISTRIBUTION + " should be > 0 for " + addr, bkc.getTestStatsProvider().getCounter( @@ -221,7 +221,7 @@ public void testChangeEnsembleIfBrokenAckQuorum() throws Exception { // ensure there is no ensemble changed assertEquals("There should be no ensemble change if delaying ensemble change is enabled.", - 1, lh.getLedgerMetadata().getEnsembles().size()); + 1, lh.getLedgerMetadata().getAllEnsembles().size()); assertTrue( "Stats should not have captured an ensemble change", bkc.getTestStatsProvider().getOpStatsLogger( @@ -238,7 +238,7 @@ public void testChangeEnsembleIfBrokenAckQuorum() throws Exception { // ensure there is no ensemble changed assertEquals("There should be no ensemble change if delaying ensemble change is enabled.", - 1, lh.getLedgerMetadata().getEnsembles().size()); + 1, lh.getLedgerMetadata().getAllEnsembles().size()); logger.info("Kill bookie 2 and write another {} entries.", numEntries); @@ -250,7 +250,7 @@ public void testChangeEnsembleIfBrokenAckQuorum() throws Exception { // ensemble change should kick in assertEquals("There should be ensemble change if ack quorum couldn't be formed.", - 2, lh.getLedgerMetadata().getEnsembles().size()); + 2, lh.getLedgerMetadata().getAllEnsembles().size()); assertTrue( "Stats should have captured an ensemble change", bkc.getTestStatsProvider().getOpStatsLogger( @@ -278,7 +278,7 @@ public void testChangeEnsembleIfBrokenAckQuorum() throws Exception { // ensure there is no ensemble changed assertEquals("There should be no ensemble change if delaying ensemble change is enabled.", - 2, lh.getLedgerMetadata().getEnsembles().size()); + 2, lh.getLedgerMetadata().getAllEnsembles().size()); // check entries verifyEntries(lh, 0, numEntries, 5, 0); @@ -316,7 +316,7 @@ public void testEnsembleChangeWithNotEnoughBookies() throws Exception { // ensure there is ensemble changed assertEquals("There should be ensemble change if ack quorum is broken.", - 2, lh.getLedgerMetadata().getEnsembles().size()); + 2, lh.getLedgerMetadata().getAllEnsembles().size()); bsConfs.add(conf0); bs.add(startBookie(conf0)); @@ -331,7 +331,7 @@ public void testEnsembleChangeWithNotEnoughBookies() throws Exception { // ensure there is no ensemble changed assertEquals("There should be no ensemble change after adding failed bookies back.", - 2, lh.getLedgerMetadata().getEnsembles().size()); + 2, lh.getLedgerMetadata().getAllEnsembles().size()); // check entries verifyEntries(lh, 0, numEntries, 5, 0); @@ -372,7 +372,7 @@ public void testEnsembleChangeWithMoreBookieFailures() throws Exception { // ensure there is no ensemble changed assertEquals("There should be ensemble change if breaking ack quorum.", - 2, lh.getLedgerMetadata().getEnsembles().size()); + 2, lh.getLedgerMetadata().getAllEnsembles().size()); for (ServerConfiguration conf : confs) { bsConfs.add(conf); @@ -386,7 +386,7 @@ public void testEnsembleChangeWithMoreBookieFailures() throws Exception { // ensure there is no ensemble changed assertEquals("There should not be ensemble changed if delaying ensemble change is enabled.", - 2, lh.getLedgerMetadata().getEnsembles().size()); + 2, lh.getLedgerMetadata().getAllEnsembles().size()); // check entries verifyEntries(lh, 0, numEntries, 5, 0); @@ -414,7 +414,7 @@ public void testChangeEnsembleIfBookieReadOnly() throws Exception { // ensure there is no ensemble changed assertEquals("The ensemble should change when a bookie is readonly even if we delay ensemble change.", - 2, lh.getLedgerMetadata().getEnsembles().size()); + 2, lh.getLedgerMetadata().getAllEnsembles().size()); } @@ -438,7 +438,7 @@ public void testChangeEnsembleSecondBookieReadOnly() throws Exception { } assertEquals("There should be ensemble change if delaying ensemble change is enabled.", - 1, lh.getLedgerMetadata().getEnsembles().size()); + 1, lh.getLedgerMetadata().getAllEnsembles().size()); // kill two bookies, but we still have 3 bookies for the ack quorum. setBookieToReadOnly(readOnlyBookie); @@ -449,7 +449,7 @@ public void testChangeEnsembleSecondBookieReadOnly() throws Exception { // ensure there is no ensemble changed assertEquals("The ensemble should change when a bookie is readonly even if we delay ensemble change.", - 2, lh.getLedgerMetadata().getEnsembles().size()); + 2, lh.getLedgerMetadata().getAllEnsembles().size()); assertEquals(3, lh.getCurrentEnsemble().size()); assertFalse(lh.getCurrentEnsemble().contains(failedBookie)); assertFalse(lh.getCurrentEnsemble().contains(readOnlyBookie)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java index 4a4599f04b0..000765c2adc 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java @@ -88,9 +88,9 @@ void disableEnsembleChangeTest(boolean startNewBookie) throws Exception { final AtomicBoolean failTest = new AtomicBoolean(false); final byte[] entry = "test-disable-ensemble-change".getBytes(UTF_8); - assertEquals(1, lh.getLedgerMetadata().getEnsembles().size()); + assertEquals(1, lh.getLedgerMetadata().getAllEnsembles().size()); ArrayList ensembleBeforeFailure = - new ArrayList<>(lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next().getValue()); + new ArrayList<>(lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next().getValue()); final RateLimiter rateLimiter = RateLimiter.create(10); @@ -120,9 +120,9 @@ public void run() { // check the ensemble after failure assertEquals("No new ensemble should be added when disable ensemble change.", - 1, lh.getLedgerMetadata().getEnsembles().size()); + 1, lh.getLedgerMetadata().getAllEnsembles().size()); ArrayList ensembleAfterFailure = - new ArrayList<>(lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next().getValue()); + new ArrayList<>(lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next().getValue()); assertArrayEquals(ensembleBeforeFailure.toArray(new BookieSocketAddress[ensembleBeforeFailure.size()]), ensembleAfterFailure.toArray(new BookieSocketAddress[ensembleAfterFailure.size()])); @@ -161,7 +161,7 @@ public void run() { assertFalse("Ledger should be closed when enable ensemble change again.", lh.getLedgerMetadata().isClosed()); assertEquals("New ensemble should be added when enable ensemble change again.", - 2, lh.getLedgerMetadata().getEnsembles().size()); + 2, lh.getLedgerMetadata().getAllEnsembles().size()); } else { assertTrue("Should fail adding entries when enable ensemble change again.", failTest.get()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java index d0787d5157b..cfe9979bc9c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java @@ -351,7 +351,7 @@ public void testFencingWithHungBookie() throws Exception { } CountDownLatch sleepLatch = new CountDownLatch(1); - sleepBookie(writelh.getLedgerMetadata().getEnsembles().get(0L).get(1), sleepLatch); + sleepBookie(writelh.getLedgerMetadata().getAllEnsembles().get(0L).get(1), sleepLatch); LedgerHandle readlh = bkc.openLedger(writelh.getId(), digestType, "testPasswd".getBytes()); 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 16a09585370..9e3162645f9 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 @@ -78,7 +78,7 @@ public void testChecker() throws Exception { for (int i = 0; i < 10; i++) { lh.addEntry(TEST_LEDGER_ENTRY_DATA); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getEnsembles() + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles() .get(0L).get(0); LOG.info("Killing {}", replicaToKill); killBookie(replicaToKill); @@ -98,7 +98,7 @@ public void testChecker() throws Exception { result.iterator().next().getAddresses().contains(replicaToKill)); BookieSocketAddress replicaToKill2 = lh.getLedgerMetadata() - .getEnsembles().get(0L).get(1); + .getAllEnsembles().get(0L).get(1); LOG.info("Killing {}", replicaToKill2); killBookie(replicaToKill2); @@ -138,7 +138,7 @@ public void testShouldNotGetTheFragmentIfThereIsNoMissedEntry() // Kill the 3rd BK from ensemble. List firstEnsemble = lh.getLedgerMetadata() - .getEnsembles().get(0L); + .getAllEnsembles().get(0L); BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(2); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); @@ -147,13 +147,13 @@ public void testShouldNotGetTheFragmentIfThereIsNoMissedEntry() startNewBookie(); LOG.info("Ensembles after first entry :" - + lh.getLedgerMetadata().getEnsembles()); + + lh.getLedgerMetadata().getAllEnsembles()); // Adding one more entry. Here enseble should be reformed. lh.addEntry(TEST_LEDGER_ENTRY_DATA); LOG.info("Ensembles after second entry :" - + lh.getLedgerMetadata().getEnsembles()); + + lh.getLedgerMetadata().getAllEnsembles()); Set result = getUnderReplicatedFragments(lh); @@ -181,7 +181,7 @@ public void testShouldGetTwoFrgamentsIfTwoBookiesFailedInSameEnsemble() lh.addEntry(TEST_LEDGER_ENTRY_DATA); List firstEnsemble = lh.getLedgerMetadata() - .getEnsembles().get(0L); + .getAllEnsembles().get(0L); BookieSocketAddress firstBookieFromEnsemble = firstEnsemble.get(0); killBookie(firstEnsemble, firstBookieFromEnsemble); @@ -214,7 +214,7 @@ public void testShouldNotGetAnyFragmentIfNoLedgerPresent() TEST_LEDGER_PASSWORD); List firstEnsemble = lh.getLedgerMetadata() - .getEnsembles().get(0L); + .getAllEnsembles().get(0L); BookieSocketAddress firstBookieFromEnsemble = firstEnsemble.get(0); killBookie(firstBookieFromEnsemble); startNewBookie(); @@ -259,7 +259,7 @@ public void testShouldGetFailedEnsembleNumberOfFgmntsIfEnsembleBookiesFailedOnNe // Kill all three bookies List firstEnsemble = lh.getLedgerMetadata() - .getEnsembles().get(0L); + .getAllEnsembles().get(0L); for (BookieSocketAddress bkAddr : firstEnsemble) { killBookie(firstEnsemble, bkAddr); } @@ -302,7 +302,7 @@ public void testShouldNotGetAnyFragmentWithEmptyLedger() throws Exception { public void testShouldGet2FragmentsWithEmptyLedgerButBookiesDead() throws Exception { LedgerHandle lh = bkc.createLedger(3, 2, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembles().get(0L)) { + for (BookieSocketAddress b : lh.getLedgerMetadata().getAllEnsembles().get(0L)) { killBookie(b); } Set result = getUnderReplicatedFragments(lh); @@ -322,7 +322,7 @@ public void testShouldGetOneFragmentWithSingleEntryOpenedLedger() throws Excepti TEST_LEDGER_PASSWORD); lh.addEntry(TEST_LEDGER_ENTRY_DATA); List firstEnsemble = lh.getLedgerMetadata() - .getEnsembles().get(0L); + .getAllEnsembles().get(0L); BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(0); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); @@ -356,7 +356,7 @@ public void testSingleEntryAfterEnsembleChange() throws Exception { lh.addEntry(TEST_LEDGER_ENTRY_DATA); } List firstEnsemble = lh.getLedgerMetadata() - .getEnsembles().get(0L); + .getAllEnsembles().get(0L); DistributionSchedule.WriteSet writeSet = lh.getDistributionSchedule().getWriteSet(lh.getLastAddPushed()); BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(writeSet.get(0)); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" @@ -401,7 +401,7 @@ public void testClosedEmptyLedger() throws Exception { LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); List firstEnsemble = lh.getLedgerMetadata() - .getEnsembles().get(0L); + .getAllEnsembles().get(0L); lh.close(); BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(0); @@ -428,7 +428,7 @@ public void testClosedSingleEntryLedger() throws Exception { LedgerHandle lh = bkc.createLedger(3, 2, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); List firstEnsemble = lh.getLedgerMetadata() - .getEnsembles().get(0L); + .getAllEnsembles().get(0L); lh.addEntry(TEST_LEDGER_ENTRY_DATA); lh.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 9782ca7e13f..6af5ed9d9a9 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 @@ -87,7 +87,7 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getEnsembles() + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles() .get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); @@ -112,7 +112,7 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() // Killing all bookies except newly replicated bookie SortedMap> allBookiesBeforeReplication = lh - .getLedgerMetadata().getEnsembles(); + .getLedgerMetadata().getAllEnsembles(); for (Entry> entry : allBookiesBeforeReplication.entrySet()) { List bookies = entry.getValue(); for (BookieSocketAddress bookie : bookies) { @@ -141,7 +141,7 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getEnsembles() + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles() .get(0L).get(0); startNewBookie(); @@ -154,7 +154,7 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() } BookieSocketAddress replicaToKill2 = lh.getLedgerMetadata() - .getEnsembles().get(0L).get(1); + .getAllEnsembles().get(0L).get(1); BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); LOG.info("New Bookie addr : {}", newBkAddr); @@ -200,7 +200,7 @@ public void testReplicateLFShouldReturnFalseIfTheReplicationFails() } // Kill the first Bookie - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getEnsembles() + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles() .get(0L).get(0); killBookie(replicaToKill); LOG.info("Killed Bookie =" + replicaToKill); @@ -210,7 +210,7 @@ public void testReplicateLFShouldReturnFalseIfTheReplicationFails() lh.addEntry(data); } // Kill the second Bookie - replicaToKill = lh.getLedgerMetadata().getEnsembles().get(0L).get(0); + replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); killBookie(replicaToKill); LOG.info("Killed Bookie =" + replicaToKill); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestMaxEnsembleChangeNum.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestMaxEnsembleChangeNum.java index d3e810c13ff..3015bef64a0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestMaxEnsembleChangeNum.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestMaxEnsembleChangeNum.java @@ -56,7 +56,7 @@ public void testChangeEnsembleMaxNumWithWriter() throws Exception { writer.append(ByteBuffer.wrap(data)); } assertEquals("There should be zero ensemble change", - 1, getLedgerMetadata(lId).getEnsembles().size()); + 1, getLedgerMetadata(lId).getAllEnsembles().size()); simulateEnsembleChangeWithWriter(changeNum, numEntries, writer); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java index 283cf759161..7bfba0c4075 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java @@ -55,7 +55,7 @@ public TestSequenceRead() { private LedgerHandle createLedgerWithDuplicatedBookies() throws Exception { final LedgerHandle lh = bkc.createLedger(3, 3, 3, digestType, passwd); // introduce duplicated bookies in an ensemble. - SortedMap> ensembles = lh.getLedgerMetadata().getEnsembles(); + SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); TreeMap> newEnsembles = new TreeMap<>(); for (Map.Entry> entry : ensembles.entrySet()) { List newList = new ArrayList(entry.getValue().size()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java index 0ae13ba546f..510dd3fac67 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java @@ -138,7 +138,7 @@ public void testSpeculativeRead() throws Exception { // sleep second bookie CountDownLatch sleepLatch = new CountDownLatch(1); - BookieSocketAddress second = lnospec.getLedgerMetadata().getEnsembles().get(0L).get(1); + BookieSocketAddress second = lnospec.getLedgerMetadata().getAllEnsembles().get(0L).get(1); sleepBookie(second, sleepLatch); try { @@ -194,9 +194,9 @@ public void testSpeculativeReadMultipleReplicasDown() throws Exception { // sleep bookie 1, 2 & 4 CountDownLatch sleepLatch = new CountDownLatch(1); - sleepBookie(l.getLedgerMetadata().getEnsembles().get(0L).get(1), sleepLatch); - sleepBookie(l.getLedgerMetadata().getEnsembles().get(0L).get(2), sleepLatch); - sleepBookie(l.getLedgerMetadata().getEnsembles().get(0L).get(4), sleepLatch); + sleepBookie(l.getLedgerMetadata().getAllEnsembles().get(0L).get(1), sleepLatch); + sleepBookie(l.getLedgerMetadata().getAllEnsembles().get(0L).get(2), sleepLatch); + sleepBookie(l.getLedgerMetadata().getAllEnsembles().get(0L).get(4), sleepLatch); try { // read first entry, should complete faster than timeout @@ -218,8 +218,8 @@ public void testSpeculativeReadMultipleReplicasDown() throws Exception { // bookies 1 & 2 should be registered as slow bookies because of speculative reads Set expectedSlowBookies = new HashSet<>(); - expectedSlowBookies.add(l.getLedgerMetadata().getEnsembles().get(0L).get(1)); - expectedSlowBookies.add(l.getLedgerMetadata().getEnsembles().get(0L).get(2)); + expectedSlowBookies.add(l.getLedgerMetadata().getAllEnsembles().get(0L).get(1)); + expectedSlowBookies.add(l.getLedgerMetadata().getAllEnsembles().get(0L).get(2)); assertEquals(((RackawareEnsemblePlacementPolicy) bkspec.getPlacementPolicy()).slowBookies.asMap().keySet(), expectedSlowBookies); @@ -268,8 +268,8 @@ public void testSpeculativeReadFirstReadCompleteIsOk() throws Exception { // sleep bookies CountDownLatch sleepLatch0 = new CountDownLatch(1); CountDownLatch sleepLatch1 = new CountDownLatch(1); - sleepBookie(l.getLedgerMetadata().getEnsembles().get(0L).get(0), sleepLatch0); - sleepBookie(l.getLedgerMetadata().getEnsembles().get(0L).get(1), sleepLatch1); + sleepBookie(l.getLedgerMetadata().getAllEnsembles().get(0L).get(0), sleepLatch0); + sleepBookie(l.getLedgerMetadata().getAllEnsembles().get(0L).get(1), sleepLatch1); try { // read goes to first bookie, spec read timeout occurs, @@ -328,7 +328,7 @@ public void testSpeculativeReadScheduling() throws Exception { LedgerHandle l = bkspec.openLedger(id, digestType, passwd); - List ensemble = l.getLedgerMetadata().getEnsembles().get(0L); + List ensemble = l.getLedgerMetadata().getAllEnsembles().get(0L); BitSet allHosts = new BitSet(ensemble.size()); for (int i = 0; i < ensemble.size(); i++) { allHosts.set(i, true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java index 91628e7e8d7..739abea78b4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java @@ -239,7 +239,7 @@ public void addComplete(int rccb, LedgerHandle lh, long entryId, Object ctx) { lh.close(); LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes()); final LedgerMetadata ledgerMetadata = openLedger.getLedgerMetadata(); - assertEquals("Failed to reform ensemble!", 2, ledgerMetadata.getEnsembles().size()); + assertEquals("Failed to reform ensemble!", 2, ledgerMetadata.getAllEnsembles().size()); ensemble = ledgerMetadata.getEnsemble(0); assertTrue("Failed to update the ledger metadata to use bookie host name", ensemble.contains(toBookieAddr)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java index 65ea441a86a..58b75ba87a0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java @@ -335,7 +335,7 @@ public void testOpenLedgerNoRecovery() throws Exception { writeQuorumSize, ackQuorumSize, password, customMetadata); registerMockLedgerMetadata(ledgerId, ledgerMetadata); - ledgerMetadata.getEnsembles().values().forEach(bookieAddressList -> { + ledgerMetadata.getAllEnsembles().values().forEach(bookieAddressList -> { bookieAddressList.forEach(bookieAddress -> { registerMockEntryForRead(ledgerId, BookieProtocol.LAST_ADD_CONFIRMED, bookieAddress, entryData, -1); registerMockEntryForRead(ledgerId, 0, bookieAddress, entryData, -1); @@ -356,7 +356,7 @@ public void testOpenLedgerRecovery() throws Exception { writeQuorumSize, ackQuorumSize, password, customMetadata); registerMockLedgerMetadata(ledgerId, ledgerMetadata); - ledgerMetadata.getEnsembles().values().forEach(bookieAddressList -> { + ledgerMetadata.getAllEnsembles().values().forEach(bookieAddressList -> { bookieAddressList.forEach(bookieAddress -> { registerMockEntryForRead(ledgerId, BookieProtocol.LAST_ADD_CONFIRMED, bookieAddress, entryData, -1); registerMockEntryForRead(ledgerId, 0, bookieAddress, entryData, -1); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java index e53b3fffa44..81bfc3d5f38 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java @@ -96,7 +96,7 @@ public void testPeriodicBookieCheckInterval() throws Exception { LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); LedgerMetadata md = LedgerHandleAdapter.getLedgerMetadata(lh); - List ensemble = new ArrayList<>(md.getEnsembles().get(0L)); + List ensemble = new ArrayList<>(md.getAllEnsembles().get(0L)); ensemble.set(0, new BookieSocketAddress("1.1.1.1", 1000)); md.updateEnsemble(0L, ensemble); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 52580c56b19..23b33788f84 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -480,9 +480,9 @@ void setLatch(CountDownLatch latch) { private BookieSocketAddress replaceBookieWithWriteFailingBookie(LedgerHandle lh) throws Exception { int bookieIdx = -1; - Long entryId = LedgerHandleAdapter.getLedgerMetadata(lh).getEnsembles().firstKey(); + Long entryId = LedgerHandleAdapter.getLedgerMetadata(lh).getAllEnsembles().firstKey(); List curEnsemble = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(entryId); + .getLedgerMetadata(lh).getAllEnsembles().get(entryId); // Identify a bookie in the current ledger ensemble to be replaced BookieSocketAddress replacedBookie = null; @@ -577,8 +577,8 @@ public void testFailedWriteRecovery() throws Exception { // check that ensemble has changed and the bookie that rejected writes has // been replaced in the ensemble LedgerHandle newLh = bkc.openLedger(lh.getId(), DigestType.CRC32, "passwd".getBytes()); - for (Map.Entry> e : LedgerHandleAdapter.getLedgerMetadata(newLh). - getEnsembles().entrySet()) { + for (Map.Entry> e : + newLh.getLedgerMetadata().getAllEnsembles().entrySet()) { List ensemble = e.getValue(); assertFalse("Ensemble hasn't been updated", ensemble.contains(replacedBookie)); } 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 4929b0ecc27..07b937550e0 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 @@ -151,7 +151,7 @@ public void testOpenLedgers() throws Exception { LedgerHandle lh = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); + .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(0); final String urLedgerZNode = getUrLedgerZNode(lh); ledgerReplicaIndex = getReplicaIndexInLedger(lh, replicaToKillAddr); @@ -200,7 +200,7 @@ public void testClosedLedgers() throws Exception { LedgerHandle lhandle = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(lhandle).getEnsembles().get(0L).get(0); + .getLedgerMetadata(lhandle).getAllEnsembles().get(0L).get(0); CountDownLatch latch = new CountDownLatch(listOfLedgerHandle.size()); for (LedgerHandle lh : listOfLedgerHandle) { @@ -262,7 +262,7 @@ public void testStopWhileReplicationInProgress() throws Exception { closeLedgers(listOfLedgerHandle); LedgerHandle handle = listOfLedgerHandle.get(0); BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(handle).getEnsembles().get(0L).get(0); + .getLedgerMetadata(handle).getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie:" + replicaToKillAddr); // Each ledger, there will be two events : create urLedger and after @@ -341,11 +341,11 @@ public void testNoSuchLedgerExists() throws Exception { watchUrLedgerNode(getUrLedgerZNode(lh), latch)); } BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(listOfLedgerHandle.get(0)).getEnsembles() + .getLedgerMetadata(listOfLedgerHandle.get(0)).getAllEnsembles() .get(0L).get(0); killBookie(replicaToKillAddr); replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(listOfLedgerHandle.get(0)).getEnsembles() + .getLedgerMetadata(listOfLedgerHandle.get(0)).getAllEnsembles() .get(0L).get(0); killBookie(replicaToKillAddr); // waiting to publish urLedger znode by Auditor @@ -384,9 +384,9 @@ public void testEmptyLedgerLosesQuorumEventually() throws Exception { watchUrLedgerNode(urZNode, latch); BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(2); + .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(2); LOG.info("Killing last bookie, {}, in ensemble {}", replicaToKill, - LedgerHandleAdapter.getLedgerMetadata(lh).getEnsembles().get(0L)); + LedgerHandleAdapter.getLedgerMetadata(lh).getAllEnsembles().get(0L)); killBookie(replicaToKill); getAuditor(10, TimeUnit.SECONDS).submitAuditTask().get(); // ensure auditor runs @@ -399,9 +399,9 @@ public void testEmptyLedgerLosesQuorumEventually() throws Exception { } replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(1); + .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(1); LOG.info("Killing second bookie, {}, in ensemble {}", replicaToKill, - LedgerHandleAdapter.getLedgerMetadata(lh).getEnsembles().get(0L)); + LedgerHandleAdapter.getLedgerMetadata(lh).getAllEnsembles().get(0L)); killBookie(replicaToKill); getAuditor(10, TimeUnit.SECONDS).submitAuditTask().get(); // ensure auditor runs @@ -443,8 +443,7 @@ public void testLedgerMetadataContainsIpAddressAsBookieID() List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); LedgerHandle lh = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - final SortedMap> ensembles = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles(); + final SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); final List bkAddresses = ensembles.get(0L); BookieSocketAddress replicaToKillAddr = bkAddresses.get(0); for (BookieSocketAddress bookieSocketAddress : bkAddresses) { @@ -522,8 +521,7 @@ public void testLedgerMetadataContainsHostNameAsBookieID() List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); LedgerHandle lh = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - final SortedMap> ensembles = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles(); + final SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); final List bkAddresses = ensembles.get(0L); BookieSocketAddress replicaToKillAddr = bkAddresses.get(0); for (BookieSocketAddress bookieSocketAddress : bkAddresses) { @@ -577,8 +575,7 @@ public void testLedgerMetadataContainsHostNameAsBookieID() } private int getReplicaIndexInLedger(LedgerHandle lh, BookieSocketAddress replicaToKill) { - SortedMap> ensembles = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles(); + SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); int ledgerReplicaIndex = -1; for (BookieSocketAddress addr : ensembles.get(0L)) { ++ledgerReplicaIndex; @@ -595,8 +592,7 @@ private void verifyLedgerEnsembleMetadataAfterReplication( LedgerHandle openLedger = bkc .openLedger(lh.getId(), digestType, PASSWD); - BookieSocketAddress inetSocketAddress = LedgerHandleAdapter - .getLedgerMetadata(openLedger).getEnsembles().get(0L) + BookieSocketAddress inetSocketAddress = openLedger.getLedgerMetadata().getAllEnsembles().get(0L) .get(ledgerReplicaIndex); assertEquals("Rereplication has been failed and ledgerReplicaIndex :" + ledgerReplicaIndex, newBookieServer.getLocalAddress(), diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java index ca767842fd7..81319997085 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java @@ -75,7 +75,7 @@ public void testAutoRecoveryAlongWithBookieServers() throws Exception { } lh.close(); BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); + .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(0); killBookie(replicaToKill); @@ -88,7 +88,7 @@ public void testAutoRecoveryAlongWithBookieServers() throws Exception { // Killing all bookies except newly replicated bookie for (Entry> entry : - lh.getLedgerMetadata().getEnsembles().entrySet()) { + lh.getLedgerMetadata().getAllEnsembles().entrySet()) { List bookies = entry.getValue(); for (BookieSocketAddress bookie : bookies) { if (bookie.equals(newBkAddr)) { 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 a4b280a3b7f..9df734646f0 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 @@ -38,7 +38,6 @@ import org.apache.bookkeeper.client.ClientUtil; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerHandleAdapter; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManagerFactory; @@ -139,8 +138,7 @@ public void testRWShouldReplicateFragmentsToTargetBookie() throws Exception { for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); killBookie(replicaToKill); @@ -188,8 +186,7 @@ public void testRWShouldRetryUntilThereAreEnoughBksAvailableForReplication() lh.addEntry(data); } lh.close(); - BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); ServerConfiguration killedBookieConfig = killBookie(replicaToKill); @@ -238,8 +235,7 @@ public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplicatio lh.addEntry(data); } lh.close(); - BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); ServerConfiguration killedBookieConfig = killBookie(replicaToKill); @@ -295,8 +291,7 @@ public void testRWShouldCleanTheLedgerFromUnderReplicationIfLedgerAlreadyDeleted lh.addEntry(data); } lh.close(); - BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); killBookie(replicaToKill); @@ -330,8 +325,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() for (int i = 0; i < 10; i++) { lh1.addEntry(data); } - BookieSocketAddress replicaToKillFromFirstLedger = LedgerHandleAdapter - .getLedgerMetadata(lh1).getEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKillFromFirstLedger = lh1.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKillFromFirstLedger); @@ -342,8 +336,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() for (int i = 0; i < 10; i++) { lh2.addEntry(data); } - BookieSocketAddress replicaToKillFromSecondLedger = LedgerHandleAdapter - .getLedgerMetadata(lh2).getEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKillFromSecondLedger = lh2.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKillFromSecondLedger); @@ -402,8 +395,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); killBookie(replicaToKill); @@ -470,7 +462,7 @@ public void testBookiesNotAvailableScenarioForReplicationWorker() throws Excepti // kill all bookies for (int i = 0; i < ensembleSize; i++) { - bookiesKilled[i] = LedgerHandleAdapter.getLedgerMetadata(lh).getEnsembles().get(0L).get(i); + bookiesKilled[i] = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(i); killedBookiesConfig[i] = getBkConf(bookiesKilled[i]); LOG.info("Killing Bookie", bookiesKilled[i]); killBookie(bookiesKilled[i]); @@ -589,8 +581,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); killBookie(replicaToKill); @@ -683,7 +674,7 @@ private void killAllBookies(LedgerHandle lh, BookieSocketAddress excludeBK) throws Exception { // Killing all bookies except newly replicated bookie for (Entry> entry : - lh.getLedgerMetadata().getEnsembles().entrySet()) { + lh.getLedgerMetadata().getAllEnsembles().entrySet()) { List bookies = entry.getValue(); for (BookieSocketAddress bookie : bookies) { if (bookie.equals(excludeBK)) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index d328bd4ac0b..e9feada8b12 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -666,7 +666,7 @@ private void testListUnderReplicatedLedgerService(LedgerManagerFactory mFactory) LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, "passwd".getBytes()); LedgerMetadata md = LedgerHandleAdapter.getLedgerMetadata(lh); - List ensemble = new ArrayList<>(md.getEnsembles().get(0L)); + List ensemble = new ArrayList<>(md.getAllEnsembles().get(0L)); ensemble.set(0, new BookieSocketAddress("1.1.1.1", 1000)); md.updateEnsemble(0L, ensemble); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java index 8dafe8bc08b..5d92979437c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java @@ -435,8 +435,8 @@ public void testTLSClientButOnlyFewTLSServers() throws Exception { ClientConfiguration clientConf = new ClientConfiguration(baseClientConf); LedgerMetadata metadata = testClient(clientConf, 2); - assertTrue(metadata.getEnsembles().size() > 0); - Collection> ensembles = metadata.getEnsembles().values(); + assertTrue(metadata.getAllEnsembles().size() > 0); + Collection> ensembles = metadata.getAllEnsembles().values(); for (List bookies : ensembles) { for (BookieSocketAddress bookieAddress : bookies) { int port = bookieAddress.getPort(); diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java index 33881103241..c06bf848ac2 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java @@ -83,7 +83,7 @@ public LedgerReader(BookKeeper bkc) { } public static SortedMap> bookiesForLedger(final LedgerHandle lh) { - return lh.getLedgerMetadata().getEnsembles(); + return lh.getLedgerMetadata().getAllEnsembles(); } public void readEntriesFromAllBookies(final LedgerHandle lh, long eid, diff --git a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy index 14243b9d424..0818b2aa544 100644 --- a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy +++ b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy @@ -104,7 +104,7 @@ class TestCompatRecoveryNoPassword { long untilEntry) throws Exception { LedgerMetadata md = getLedgerMetadata(bookkeeper, lh.getId()) - def ensembles = md.getEnsembles() + def ensembles = md.getAllEnsembles() def ranges = new HashMap() def keyList = new ArrayList(ensembles.keySet()) From 7e9d3de0b246589d3d661d87e27d9a411fe8ff7a Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Sun, 28 Oct 2018 05:56:47 -0700 Subject: [PATCH 0091/1642] Don't cache Bookie hostname DNS resolution forever ### Motivation `BookieSocketAddress` is resolving the bookie DNS name in its constructor and then using the already resolved `InetSocketAddress` instance. If the IP of a bookie changes, the BK client will continue to use the old IP address. ### Changes Construct a new `InetSocketAddress` each time `getSocketAddress()` gets called (eg: each time we attempt to make a new connection) so that we're making sure to get the right IP. I cannot think of a good way to add unit test for this at this point, suggestions are welcome. I think this should be included in a patch release as well 4.7.3 or 4.8.1 Reviewers: Andrey Yegorov , Enrico Olivelli , Sijie Guo , Ivan Kelly This closes #1762 from merlimat/fix-dns --- .../apache/bookkeeper/net/BookieSocketAddress.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java index 4e8f3246c66..6d562e3a671 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java @@ -39,13 +39,10 @@ public class BookieSocketAddress { private final String hostname; private final int port; - private final InetSocketAddress socketAddress; - // Constructor that takes in both a port. public BookieSocketAddress(String hostname, int port) { this.hostname = hostname; this.port = port; - socketAddress = new InetSocketAddress(hostname, port); } // Constructor from a String "serialized" version of this class. @@ -60,7 +57,6 @@ public BookieSocketAddress(String addr) throws UnknownHostException { } catch (NumberFormatException nfe) { throw new UnknownHostException(addr); } - socketAddress = new InetSocketAddress(hostname, port); } // Public getters @@ -74,7 +70,12 @@ public int getPort() { // Method to return an InetSocketAddress for the regular port. public InetSocketAddress getSocketAddress() { - return socketAddress; + // Return each time a new instance of the InetSocketAddress because the hostname + // gets resolved in its constructor and then cached forever. + // If we keep using the same InetSocketAddress instance, if bookies are advertising + // hostnames and the IP change, the BK client will keep forever to try to connect + // to the old IP. + return new InetSocketAddress(hostname, port); } /** From c92575186df1a2301869d896be4305c7ada9269a Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sun, 28 Oct 2018 21:05:28 +0800 Subject: [PATCH 0092/1642] [MERGE SCRIPT] remove authors section Descriptions of the changes in this PR: *Motivation* We are using github api to merge pull requests now. So the authors section is not correct, remove that section and rely on github merge api. *Changes* remove authors section in the commit log. Reviewers: Enrico Olivelli This closes #1758 from sijie/fix_merge_scripts --- dev/bk-merge-pr.py | 14 -------------- dev/bk-merge-pr3.py | 14 -------------- 2 files changed, 28 deletions(-) diff --git a/dev/bk-merge-pr.py b/dev/bk-merge-pr.py index 4dec84d6d15..e48b119dc8c 100755 --- a/dev/bk-merge-pr.py +++ b/dev/bk-merge-pr.py @@ -152,16 +152,6 @@ def merge_pr(pr_num, target_ref, title, body, default_pr_reviewers, pr_repo_desc pr_branch_name = "%s_MERGE_PR_%s" % (TEMP_BRANCH_PREFIX, pr_num) run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) - commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, - '--pretty=format:%an <%ae>']).split("\n") - distinct_authors = sorted(set(commit_authors), - key=lambda x: commit_authors.count(x), reverse=True) - primary_author = raw_input( - "Enter primary author in the format of \"name \" [%s]: " % - distinct_authors[0]) - if primary_author == "": - primary_author = distinct_authors[0] - reviewers = raw_input("Enter reviewers [%s]: " % default_pr_reviewers).strip() if reviewers == '': reviewers = default_pr_reviewers @@ -185,10 +175,6 @@ def merge_pr(pr_num, target_ref, title, body, default_pr_reviewers, pr_repo_desc # to people every time someone creates a public fork of the project. merge_message_flags += [body.replace("@", "")] - authors = "\n".join(["Author: %s" % a for a in distinct_authors]) - - merge_message_flags += [authors] - if (reviewers != ""): merge_message_flags += ["Reviewers: %s" % reviewers] diff --git a/dev/bk-merge-pr3.py b/dev/bk-merge-pr3.py index 4f755f9cfb9..069358fdb79 100755 --- a/dev/bk-merge-pr3.py +++ b/dev/bk-merge-pr3.py @@ -153,16 +153,6 @@ def merge_pr(pr_num, target_ref, title, body, default_pr_reviewers, pr_repo_desc pr_branch_name = "%s_MERGE_PR_%s" % (TEMP_BRANCH_PREFIX, pr_num) run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) - commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, - '--pretty=format:%an <%ae>']).split("\n") - distinct_authors = sorted(set(commit_authors), - key=lambda x: commit_authors.count(x), reverse=True) - primary_author = input( - "Enter primary author in the format of \"name \" [%s]: " % - distinct_authors[0]) - if primary_author == "": - primary_author = distinct_authors[0] - reviewers = input("Enter reviewers [%s]: " % default_pr_reviewers).strip() if reviewers == '': reviewers = default_pr_reviewers @@ -186,10 +176,6 @@ def merge_pr(pr_num, target_ref, title, body, default_pr_reviewers, pr_repo_desc # to people every time someone creates a public fork of the project. merge_message_flags += [body.replace("@", "")] - authors = "\n".join(["Author: %s" % a for a in distinct_authors]) - - merge_message_flags += [authors] - if (reviewers != ""): merge_message_flags += ["Reviewers: %s" % reviewers] From fa94c931337866622dcfa64626e5f61aeaf93ea0 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Sun, 28 Oct 2018 14:06:18 +0100 Subject: [PATCH 0093/1642] Kill LedgerMetadata#isConflictWith It doesn't make sense anymore, as local copies of the metadata are never modified, only updated with the latest version from the metadata store. In effect, this logic has been broken out to the places where we try to update the metadata store copy. Each time we try to update the metadata store, we ensure that the update we are applying still makes sense with regard to the copy of the metadata we are updating. Master issue: #281 Reviewers: Sijie Guo , Enrico Olivelli This closes #1760 from ivankelly/kill-is-conflict-with --- .../bookkeeper/client/LedgerMetadata.java | 61 ------------------- .../bookkeeper/client/LedgerMetadataTest.java | 56 ----------------- .../client/UpdateLedgerCmdTest.java | 3 - .../bookkeeper/client/UpdateLedgerOpTest.java | 9 --- 4 files changed, 129 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 02e0ede38d0..d8e1c1be372 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -34,7 +34,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -672,66 +671,6 @@ public static boolean areByteArrayValMapsEqual(Map first, Map keyIter = ensembles.keySet().iterator(); - Iterator newMetaKeyIter = newMeta.ensembles.keySet().iterator(); - for (int i = 0; i < newMeta.ensembles.size(); i++) { - Long curKey = keyIter.next(); - Long newMetaKey = newMetaKeyIter.next(); - if (!curKey.equals(newMetaKey)) { - return true; - } - } - } - return false; - } - @Override public String toString() { return toStringRepresentation(true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java index dc947dde722..5f802cb6049 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java @@ -99,62 +99,6 @@ public void testStoreSystemtimeAsLedgerCtimeDisabled() assertFalse(format.hasCtime()); } - @Test - public void testIsConflictWithStoreSystemtimeAsLedgerCtimeDisabled() { - LedgerMetadata lm1 = new LedgerMetadata( - 3, - 3, - 2, - DigestType.CRC32, - passwd, - Collections.emptyMap(), - false); - LedgerMetadata lm2 = new LedgerMetadata(lm1); - - lm1.setCtime(1L); - lm2.setCtime(2L); - assertFalse(lm1.isConflictWith(lm2)); - } - - @Test - public void testIsConflictWithStoreSystemtimeAsLedgerCtimeEnabled() { - LedgerMetadata lm1 = new LedgerMetadata( - 3, - 3, - 2, - DigestType.CRC32, - passwd, - Collections.emptyMap(), - true); - LedgerMetadata lm2 = new LedgerMetadata(lm1); - - lm1.setCtime(1L); - lm2.setCtime(2L); - assertTrue(lm1.isConflictWith(lm2)); - } - - @Test - public void testIsConflictWithDifferentStoreSystemtimeAsLedgerCtimeFlags() { - LedgerMetadata lm1 = new LedgerMetadata( - 3, - 3, - 2, - DigestType.CRC32, - passwd, - Collections.emptyMap(), - true); - LedgerMetadata lm2 = new LedgerMetadata( - 3, - 3, - 2, - DigestType.CRC32, - passwd, - Collections.emptyMap(), - false); - - assertTrue(lm1.isConflictWith(lm2)); - } - @Test public void testToString() { LedgerMetadata lm1 = new LedgerMetadata( diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java index 6eca0de22f1..fa184972da8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java @@ -97,9 +97,6 @@ private int getUpdatedLedgersCount(BookKeeper bk, List ledgers, Bo List ensemble; int updatedLedgersCount = 0; for (LedgerHandle lh : ledgers) { - // ledger#close() would hit BadVersion exception as rename - // increments cversion. But LedgerMetadata#isConflictWith() - // gracefully handles this conflicts. lh.close(); LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes()); ensemble = openLedger.getLedgerMetadata().getEnsemble(0); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java index 739abea78b4..ad5c450ceb1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java @@ -110,9 +110,6 @@ public void testManyLedgers(boolean useShortHostName) throws Exception { updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, Integer.MIN_VALUE, progressable); for (LedgerHandle lh : ledgers) { - // ledger#close() would hit BadVersion exception as rename - // increments cversion. But LedgerMetadata#isConflictWith() - // gracefully handles this conflicts. lh.close(); LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes()); ensemble = openLedger.getLedgerMetadata().getEnsemble(0); @@ -218,9 +215,6 @@ public void testChangeEnsembleAfterRenaming(boolean useShortHostName) throws Exc bsConfs.add(serverConf1); bs.add(startBookie(serverConf1)); - // ledger#asyncAddEntry() would hit BadVersion exception as rename incr - // cversion. But LedgerMetadata#isConflictWith() gracefully handles - // this conflicts. final CountDownLatch latch = new CountDownLatch(1); final AtomicInteger rc = new AtomicInteger(BKException.Code.OK); lh.asyncAddEntry("foobar".getBytes(), new AddCallback() { @@ -304,9 +298,6 @@ private int getUpdatedLedgersCount(BookKeeper bk, List ledgers, Bo List ensemble; int updatedLedgersCount = 0; for (LedgerHandle lh : ledgers) { - // ledger#close() would hit BadVersion exception as rename - // increments cversion. But LedgerMetadata#isConflictWith() - // gracefully handles this conflicts. lh.close(); LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes()); ensemble = openLedger.getLedgerMetadata().getEnsemble(0); From 2418007d1d0686e5e0aafd634d011b0c4eb435bd Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Sun, 28 Oct 2018 06:30:23 -0700 Subject: [PATCH 0094/1642] [WEBSITE] Fix broken link to ledger-api#example-application Link should be to a section in same page Reviewers: Enrico Olivelli , Sijie Guo This closes #1764 from merlimat/fix-docs-link --- site/docs/4.7.2/api/ledger-api.md | 2 +- site/docs/4.8.0/api/ledger-api.md | 4 ++-- site/docs/latest/api/ledger-api.md | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/site/docs/4.7.2/api/ledger-api.md b/site/docs/4.7.2/api/ledger-api.md index d30c4c8e4cb..c22daeb121a 100644 --- a/site/docs/4.7.2/api/ledger-api.md +++ b/site/docs/4.7.2/api/ledger-api.md @@ -8,7 +8,7 @@ The ledger API is a lower-level API for BookKeeper that enables you to interact 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. +> For a more in-depth tutorial that involves a real use case for BookKeeper, see the [Example application](#example-application) guide. ## Installation diff --git a/site/docs/4.8.0/api/ledger-api.md b/site/docs/4.8.0/api/ledger-api.md index 38573610113..2b44d47e995 100644 --- a/site/docs/4.8.0/api/ledger-api.md +++ b/site/docs/4.8.0/api/ledger-api.md @@ -8,7 +8,7 @@ The ledger API is a lower-level API for BookKeeper that enables you to interact 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. +> For a more in-depth tutorial that involves a real use case for BookKeeper, see the [Example application](#example-application) guide. ## Installation @@ -837,5 +837,5 @@ WriteHandle wh = bk.newCreateLedgerOp() wh.force().get(); // wait for fsync, make data available to readers and to the replicator wh.close(); // seal the ledger - + ``` diff --git a/site/docs/latest/api/ledger-api.md b/site/docs/latest/api/ledger-api.md index b6cb0f02a95..22dacb59d6c 100644 --- a/site/docs/latest/api/ledger-api.md +++ b/site/docs/latest/api/ledger-api.md @@ -8,7 +8,7 @@ The ledger API is a lower-level API for BookKeeper that enables you to interact 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. +> For a more in-depth tutorial that involves a real use case for BookKeeper, see the [Example application](#example-application) guide. ## Installation @@ -837,5 +837,5 @@ WriteHandle wh = bk.newCreateLedgerOp() wh.force().get(); // wait for fsync, make data available to readers and to the replicator wh.close(); // seal the ledger - + ``` From 85aa0c467b9330d6b1124f71bc51bbbca7824991 Mon Sep 17 00:00:00 2001 From: Qi Wang <42832902+codingwangqi@users.noreply.github.com> Date: Tue, 30 Oct 2018 09:53:19 +0800 Subject: [PATCH 0095/1642] Issue #1752: Etcd docker based tests are running on wrong jobs on CI Descriptions of the changes in this PR: *Motivation* Fixes #1752 Etcd metadata driver is using testcontainer to run Etcd in containers for integration tests. However this job is running as part normal build, which can cause container image being deleted due to concurrent builds. *Changes* Move etcd metadata driver tests to be as part of `integrationTests` Master Issue: #1752 Reviewers: Enrico Olivelli , Sijie Guo This closes #1766 from codingwangqi/move_etcd_to_integration_tests, closes #1752 --- ...okkeeper_precommit_integrationtests.groovy | 9 ++ .test-infra/scripts/pre-docker-tests.sh | 1 + metadata-drivers/etcd/pom.xml | 121 ++++++++++-------- 3 files changed, 81 insertions(+), 50 deletions(-) diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy index 10958ee308e..7a169cb9da3 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy @@ -60,6 +60,15 @@ freeStyleJob('bookkeeper_precommit_integrationtests') { properties(skipTests: true, interactiveMode: false) } + // Run metadata driver tests + maven { + // Set Maven parameters. + common_job_properties.setMavenConfig(delegate) + rootPOM('metadata-drivers/pom.xml') + goals('-B test -DintegrationTests') + } + + // Run all integration tests maven { // Set Maven parameters. common_job_properties.setMavenConfig(delegate) diff --git a/.test-infra/scripts/pre-docker-tests.sh b/.test-infra/scripts/pre-docker-tests.sh index 3e92513e1fd..1edb48c0d7b 100755 --- a/.test-infra/scripts/pre-docker-tests.sh +++ b/.test-infra/scripts/pre-docker-tests.sh @@ -29,3 +29,4 @@ docker system prune -f # clean up any dangling networks from previous runs docker network prune -f --filter name=testnetwork_* docker system events > docker.debug-info & echo $! > docker-log.pid +docker pull quay.io/coreos/etcd:v3.3 diff --git a/metadata-drivers/etcd/pom.xml b/metadata-drivers/etcd/pom.xml index 0d9e0dd164d..8e3b3390546 100644 --- a/metadata-drivers/etcd/pom.xml +++ b/metadata-drivers/etcd/pom.xml @@ -18,55 +18,76 @@ - - org.apache.bookkeeper.metadata.drivers - metadata-drivers-parent - 4.9.0-SNAPSHOT - .. - - 4.0.0 + org.apache.bookkeeper.metadata.drivers - metadata-stores-etcd - Apache BookKeeper :: Metadata Drivers:: Etcd - - - org.apache.bookkeeper - bookkeeper-server - ${project.version} - - - com.coreos - jetcd-core - ${etcd.version} - - - io.grpc - * - - - - - io.grpc - grpc-all - - - org.testcontainers - testcontainers - test - - - org.apache.bookkeeper - bookkeeper-common - ${project.version} - test-jar - test - - - org.apache.bookkeeper - bookkeeper-server - ${project.version} - test-jar - test - - + metadata-drivers-parent + 4.9.0-SNAPSHOT + .. + + 4.0.0 + org.apache.bookkeeper.metadata.drivers + metadata-stores-etcd + Apache BookKeeper :: Metadata Drivers:: Etcd + + + org.apache.bookkeeper + bookkeeper-server + ${project.version} + + + com.coreos + jetcd-core + ${etcd.version} + + + io.grpc + * + + + + + io.grpc + grpc-all + + + org.testcontainers + testcontainers + test + + + org.apache.bookkeeper + bookkeeper-common + ${project.version} + test-jar + test + + + org.apache.bookkeeper + bookkeeper-server + ${project.version} + test-jar + test + + + + + integrationTests + + + integrationTests + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + false + + + + + + From f0880aa8924a6e82348990d8ca4c80b3310c273a Mon Sep 17 00:00:00 2001 From: Samuel Just Date: Tue, 30 Oct 2018 11:26:50 -0700 Subject: [PATCH 0096/1642] ISSUE #1757: prevent race between flush and delete from recreating index IndexPersistencManager.flushLedgerHandle can race with delete by obtaining a FileInfo just prior to delete and then proceeding to rewrite the file info resurrecting it. FileInfo provides a convenient point of synchronization for avoinding this race. FileInfo.moveLedgerIndexFile, FileInfo.flushHeader, and FileInfo.delete() are synchronized already, so this patch simply adds a deleted flag to the FileInfo object to indicate that the FileInfo has become invalid. checkOpen is called in every method and will now throw FileInfoDeleted if delete has been called. IndexPersistenceManager can catch it and deal with it appropriately in flush (which generally means moving onto the next ledger). This patch also eliminates ledgersToFlush and ledgersFlushing. Their purpose appears to be to allow delete to avoid flushing a ledger which has been selected for flushing but not flushed yet avoiding the above race. It's not sufficient, however, because IndexInMemPageMgr calls IndexPersistenceManager.flushLedgerHeader, which can obtain a FileInfo for the ledger prior to the deletion and then call relocateIndexFileAndFlushHeader afterwards. Also, if the purpose was to avoid concurrent calls into flushSpecificLedger on the same ledger, it's wrong because of the following sequence: t0: thread 0 calls flushOneOrMoreLedgers t1: thread 0 place ledger 10 into ledgersFlushing and completes flushSpecificLedger t2: thread 2 performs a write to ledger 10 t3: thread 1 calls flushOneOrMoreLedgers skipping ledger 10 t4: thread 0 releases ledger 10 from ledgersFlushing t5: thread 1 completes flushOneOrMoreLedgers Although thread 1 begins to flush after the write to ledger 10, it won't capture the write rendering the flush incorrect. I don't think it's actually worth avoiding overlapping flushes here because both FileInfo and LedgerEntryPage track dirty state. As such, it seems simpler to just get rid of them. This patch also adds a more agressive version of testFlushDeleteRace to test the new behavior. Testing with multiple flushers turned up a bug with LedgerEntryPage.getPageToWrite where didn't return a buffer with independent read pointers, so this patch addresses that as well. (bug W-5549455) (rev cguttapalem) Signed-off-by: Samuel Just (cherry picked from commit 7b5ac3d5e76ac4df618764cafe80aef2994703ec) Author: Reviewers: Enrico Olivelli , Sijie Guo This closes #1769 from athanatos/forupstream/wip-1757, closes #1757 --- .../apache/bookkeeper/bookie/FileInfo.java | 17 ++++ .../bookie/FileInfoBackingCache.java | 2 + .../bookkeeper/bookie/IndexInMemPageMgr.java | 22 +---- .../bookie/IndexPersistenceMgr.java | 16 +++- .../bookkeeper/bookie/LedgerEntryPage.java | 3 +- .../bookkeeper/bookie/LedgerCacheTest.java | 83 +++++++++++++------ 6 files changed, 95 insertions(+), 48 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java index a5ddacf0ff6..656674f5e03 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java @@ -101,6 +101,8 @@ class FileInfo extends Watchable { // this FileInfo Header Version int headerVersion; + private boolean deleted; + public FileInfo(File lf, byte[] masterKey, int fileInfoVersionToWrite) throws IOException { super(WATCHER_RECYCLER); @@ -108,6 +110,7 @@ public FileInfo(File lf, byte[] masterKey, int fileInfoVersionToWrite) throws IO this.masterKey = masterKey; mode = "rw"; this.headerVersion = fileInfoVersionToWrite; + this.deleted = false; } synchronized Long getLastAddConfirmed() { @@ -257,6 +260,16 @@ public synchronized void readHeader() throws IOException { } } + public synchronized boolean isDeleted() { + return deleted; + } + + public static class FileInfoDeletedException extends IOException { + FileInfoDeletedException() { + super("FileInfo already deleted"); + } + } + @VisibleForTesting void checkOpen(boolean create) throws IOException { checkOpen(create, false); @@ -264,6 +277,9 @@ void checkOpen(boolean create) throws IOException { private synchronized void checkOpen(boolean create, boolean openBeforeClose) throws IOException { + if (deleted) { + throw new FileInfoDeletedException(); + } if (fc != null) { return; } @@ -540,6 +556,7 @@ public synchronized byte[] getMasterKey() throws IOException { } public synchronized boolean delete() { + deleted = true; return lf.delete(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfoBackingCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfoBackingCache.java index 6beba6a744d..078292fb811 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfoBackingCache.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfoBackingCache.java @@ -49,6 +49,8 @@ private static CachedFileInfo tryRetainFileInfo(CachedFileInfo fi) throws IOExce boolean retained = fi.tryRetain(); if (!retained) { throw new IOException("FileInfo " + fi + " is already marked dead"); + } else if (fi.isDeleted()) { + throw new Bookie.NoLedgerException(fi.ledgerId); } return fi; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java index 66e97f79423..0cf5cc93f87 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java @@ -40,7 +40,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -345,12 +344,6 @@ public void onSetDirty(LedgerEntryPage lep) { // flush and read pages private final IndexPersistenceMgr indexPersistenceManager; - /** - * the list of potentially dirty ledgers. - */ - private final ConcurrentLinkedQueue ledgersToFlush = new ConcurrentLinkedQueue(); - private final ConcurrentSkipListSet ledgersFlushing = new ConcurrentSkipListSet(); - // Stats private final Counter ledgerCacheHitCounter; private final Counter ledgerCacheMissCounter; @@ -504,7 +497,6 @@ private LedgerEntryPage grabLedgerEntryPage(long ledger, long pageEntry) throws void removePagesForLedger(long ledgerId) { pageMapAndList.removeEntriesForALedger(ledgerId); - ledgersToFlush.remove(ledgerId); } long getLastEntryInMem(long ledgerId) { @@ -542,18 +534,12 @@ private LedgerEntryPage grabCleanPage(long ledger, long entry) throws IOExceptio } void flushOneOrMoreLedgers(boolean doAll) throws IOException { - if (ledgersToFlush.isEmpty()) { - ledgersToFlush.addAll(pageMapAndList.getActiveLedgers()); - } - Long potentiallyDirtyLedger; - while (null != (potentiallyDirtyLedger = ledgersToFlush.poll())) { - if (!ledgersFlushing.add(potentiallyDirtyLedger)) { - continue; - } + List ledgersToFlush = new ArrayList<>(pageMapAndList.getActiveLedgers()); + for (Long potentiallyDirtyLedger : ledgersToFlush) { try { flushSpecificLedger(potentiallyDirtyLedger); - } finally { - ledgersFlushing.remove(potentiallyDirtyLedger); + } catch (Bookie.NoLedgerException e) { + continue; } if (!doAll) { break; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java index 83cb88fcc31..11292d1688a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java @@ -525,7 +525,12 @@ private File getLedgerDirForLedger(FileInfo fi) { private void moveLedgerIndexFile(Long l, FileInfo fi) throws NoWritableLedgerDirException, IOException { File newLedgerIndexFile = getNewLedgerIndexFile(l, getLedgerDirForLedger(fi)); - fi.moveToNewLocation(newLedgerIndexFile, fi.getSizeSinceLastwrite()); + try { + fi.moveToNewLocation(newLedgerIndexFile, fi.getSizeSinceLastwrite()); + } catch (FileInfo.FileInfoDeletedException fileInfoDeleted) { + // File concurrently deleted + throw new Bookie.NoLedgerException(l); + } } void flushLedgerHeader(long ledger) throws IOException { @@ -599,7 +604,7 @@ public int compare(LedgerEntryPage o1, LedgerEntryPage o2) { private void writeBuffers(Long ledger, List entries, FileInfo fi, - int start, int count) throws IOException { + int start, int count) throws IOException, Bookie.NoLedgerException { if (LOG.isTraceEnabled()) { LOG.trace("Writing {} buffers of {}", count, Long.toHexString(ledger)); } @@ -616,7 +621,12 @@ private void writeBuffers(Long ledger, } long totalWritten = 0; while (buffs[buffs.length - 1].remaining() > 0) { - long rc = fi.write(buffs, entries.get(start + 0).getFirstEntryPosition()); + long rc = 0; + try { + rc = fi.write(buffs, entries.get(start + 0).getFirstEntryPosition()); + } catch (FileInfo.FileInfoDeletedException e) { + throw new Bookie.NoLedgerException(ledger); + } if (rc <= 0) { throw new IOException("Short write to ledger " + ledger + " rc = " + rc); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java index a9cef723061..be8755959cc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java @@ -207,7 +207,8 @@ public void readPage(FileInfo fi) throws IOException { public ByteBuffer getPageToWrite() { checkPage(); page.clear(); - return page; + // Different callers to this method should be able to reasonably expect independent read pointers + return page.duplicate(); } long getLedger() { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java index 1a01299a6cd..232173507b6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java @@ -379,24 +379,33 @@ public void testSyncThreadNPE() throws IOException { } /** - * Race where a flush would fail because a garbage collection occurred at - * the wrong time. + * Test for race between delete and flush. * {@link https://issues.apache.org/jira/browse/BOOKKEEPER-604} + * {@link https://github.com/apache/bookkeeper/issues/1757} */ @Test public void testFlushDeleteRace() throws Exception { newLedgerCache(); final AtomicInteger rc = new AtomicInteger(0); - final LinkedBlockingQueue ledgerQ = new LinkedBlockingQueue(1); + final LinkedBlockingQueue ledgerQ = new LinkedBlockingQueue<>(100); final byte[] masterKey = "masterKey".getBytes(); + final long numLedgers = 1000; + final int numFlushers = 10; + final int numDeleters = 10; + final AtomicBoolean running = new AtomicBoolean(true); Thread newLedgerThread = new Thread() { public void run() { try { - for (int i = 0; i < 1000 && rc.get() == 0; i++) { + for (long i = 0; i < numLedgers && rc.get() == 0; i++) { ledgerCache.setMasterKey(i, masterKey); - ledgerQ.put((long) i); + + ledgerCache.putEntryOffset(i, 1, 0); + ledgerQ.put(i); + } + for (int i = 0; i < numDeleters; ++i) { + ledgerQ.put(-1L); } - } catch (Exception e) { + } catch (Throwable e) { rc.set(-1); LOG.error("Exception in new ledger thread", e); } @@ -404,51 +413,73 @@ public void run() { }; newLedgerThread.start(); - Thread flushThread = new Thread() { + Thread[] flushThreads = new Thread[numFlushers]; + for (int i = 0; i < numFlushers; ++i) { + Thread flushThread = new Thread() { public void run() { try { - while (true) { - Long id = ledgerQ.peek(); - if (id == null) { - continue; - } - LOG.info("Put entry for {}", id); - try { - ledgerCache.putEntryOffset((long) id, 1, 0); - } catch (Bookie.NoLedgerException nle) { - //ignore - } + while (running.get()) { ledgerCache.flushLedger(true); } - } catch (Exception e) { + } catch (Throwable e) { rc.set(-1); LOG.error("Exception in flush thread", e); } + LOG.error("Shutting down flush thread"); } }; - flushThread.start(); + flushThread.start(); + flushThreads[i] = flushThread; + } - Thread deleteThread = new Thread() { + Thread[] deleteThreads = new Thread[numDeleters]; + for (int i = 0; i < numDeleters; ++i) { + Thread deleteThread = new Thread() { public void run() { try { while (true) { long id = ledgerQ.take(); + if (id == -1L) { + break; + } LOG.info("Deleting {}", id); ledgerCache.deleteLedger(id); } - } catch (Exception e) { + } catch (Throwable e) { rc.set(-1); LOG.error("Exception in delete thread", e); } } }; - deleteThread.start(); + deleteThread.start(); + deleteThreads[i] = deleteThread; + } newLedgerThread.join(); - assertEquals("Should have been no errors", rc.get(), 0); - deleteThread.interrupt(); - flushThread.interrupt(); + for (Thread deleteThread : deleteThreads) { + deleteThread.join(); + } + + running.set(false); + for (Thread flushThread : flushThreads) { + flushThread.join(); + } + + assertEquals("Should have been no errors", rc.get(), 0); + for (long i = 0L; i < numLedgers; ++i) { + boolean gotError = false; + try { + LOG.error("Checking {}", i); + ledgerCache.getEntryOffset(i, 0); + } catch (NoLedgerException e) { + gotError = true; + } + if (!gotError) { + LOG.error("Ledger {} is still around", i); + fail("Found ledger " + i + ", which should have been removed"); + } + } } // Mock SortedLedgerStorage to simulate flush failure (Dependency Fault Injection) From d85ea4edf9150b5d6778935404fd007882714fa4 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 30 Oct 2018 11:46:31 -0700 Subject: [PATCH 0097/1642] [TESTS] Fix integration test TestCompatUpgradeYahooCustom.testUpgradeYahooCustom Descriptions of the changes in this PR: *Motivation* apache/bookkeeper#1749 fixes bash script issue but it introduced `set -e` which will fail `bin/bookkeeper` if grepping configuration file failed. *Changes* `set -e` is useful to fail fast if anything fails in the bash script. This change is to mask exit code of `grep` operation, since the bash script already handle the grep result. *Tests* Manually verified the change works. And the existing TestCompatUpgradeYahooCustom.testUpgradeYahooCustom integration tests would also verify if the change fixes the problem or not. Reviewers: Enrico Olivelli , Ivan Kelly This closes #1772 from sijie/fix_common_scripts --- bin/bookkeeper | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index a207ff73290..70b6ccd1444 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -37,7 +37,8 @@ fi # check the configuration to see if table service is enabled or not. if [ -z "${ENABLE_TABLE_SERVICE}" ]; then - TABLE_SERVICE_SETTING=$(grep StreamStorageLifecycleComponent ${BOOKIE_CONF_TO_CHECK}) + # mask exit code if the configuration file doesn't contain `StreamStorageLifecycleComponent` + TABLE_SERVICE_SETTING=$(grep StreamStorageLifecycleComponent ${BOOKIE_CONF_TO_CHECK} | cat) if [[ "${TABLE_SERVICE_SETTING}" =~ ^extraServerComponents.* ]]; then ENABLE_TABLE_SERVICE="true" fi From bc5bd1b9cbba5270649660d7ee5ec3107c0765d6 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 30 Oct 2018 11:47:54 -0700 Subject: [PATCH 0098/1642] [TABLE SERVICE] Fix python client can't read keys written by java clients Descriptions of the changes in this PR: *Motivation* Python client is a thin table service client. So it relies on storage container intercepting the requests to attach routing header. It attaches the routing header if the requests are needed to route to remote containers; however for the local containers it doesn't attach the right routing header. *Changes* Fix `StorageContainerImpl` to use proxy routing interceptor to intercept requests to attach routing header. *Tests* Add integration tests in python client & add test for python clients reading key/value pairs written by java clients. Reviewers: Jia Zhai This closes #1771 from sijie/remove_python23_incompatible_changes --- pom.xml | 1 + .../clients/python/bookkeeper/admin/client.py | 2 - .../python/bookkeeper/admin/namespace.py | 21 +++- .../python/bookkeeper/admin/namespaces.py | 16 ++- .../python/bookkeeper/common/constants.py | 22 ++++ .../python/bookkeeper/common/exceptions.py | 3 +- stream/clients/python/bookkeeper/kv/client.py | 2 - stream/clients/python/bookkeeper/kv/table.py | 23 ++++ stream/clients/python/nox.py | 27 +++++ stream/clients/python/noxfile.py | 27 +++++ stream/clients/python/scripts/docker_build.sh | 34 ++++++ .../scripts/docker_integration_tests.sh | 29 +++++ .../python/scripts/run_integration_tests.sh | 41 +++++++ stream/clients/python/setup.py | 2 +- .../tests/integration/bookkeeper/__init__.py | 11 ++ .../bookkeeper/admin/test_admin_client.py | 81 +++++++++++++ .../integration/bookkeeper/kv/test_client.py | 111 ++++++++++++++++++ .../storage/impl/sc/StorageContainerImpl.java | 3 + .../impl/service/RangeStoreServiceImpl.java | 47 ++++++++ .../current-version-image/Dockerfile | 8 +- .../current-version-image/pom.xml | 66 +++++++---- .../scripts/install-python-client.sh | 24 ++++ 22 files changed, 567 insertions(+), 34 deletions(-) create mode 100755 stream/clients/python/scripts/docker_build.sh create mode 100755 stream/clients/python/scripts/docker_integration_tests.sh create mode 100755 stream/clients/python/scripts/run_integration_tests.sh create mode 100644 stream/clients/python/tests/integration/bookkeeper/__init__.py create mode 100644 stream/clients/python/tests/integration/bookkeeper/admin/test_admin_client.py create mode 100644 stream/clients/python/tests/integration/bookkeeper/kv/test_client.py create mode 100644 tests/docker-images/current-version-image/scripts/install-python-client.sh diff --git a/pom.xml b/pom.xml index c13d4878ce6..95aaf73f7c6 100644 --- a/pom.xml +++ b/pom.xml @@ -169,6 +169,7 @@ 2.7 4.3.0 1.3.7 + 1.6.0 1.6 0.8.0 1.8 diff --git a/stream/clients/python/bookkeeper/admin/client.py b/stream/clients/python/bookkeeper/admin/client.py index fdb82d2699a..80de76d7b67 100644 --- a/stream/clients/python/bookkeeper/admin/client.py +++ b/stream/clients/python/bookkeeper/admin/client.py @@ -16,7 +16,6 @@ import grpc import logging -import pkg_resources from bookkeeper import types from bookkeeper.admin.namespace import Namespace @@ -24,7 +23,6 @@ from bookkeeper.common.service_uri import ServiceURI from bookkeeper.proto.storage_pb2_grpc import RootRangeServiceStub -__version__ = pkg_resources.get_distribution('bookkeeper').version __logger__ = logging.getLogger("bookkeeper.admin.Client") diff --git a/stream/clients/python/bookkeeper/admin/namespace.py b/stream/clients/python/bookkeeper/admin/namespace.py index 5c3910b1ce6..631673a5e9b 100644 --- a/stream/clients/python/bookkeeper/admin/namespace.py +++ b/stream/clients/python/bookkeeper/admin/namespace.py @@ -14,9 +14,12 @@ from __future__ import absolute_import -from bookkeeper.common.constants import __DEFAULT_STREAM_CONF__ +from bookkeeper.common.constants import __DEFAULT_TABLE_CONF__ from bookkeeper.common.constants import __ROOT_RANGE_METADATA__ from bookkeeper.common.exceptions import from_root_range_rpc_response +from bookkeeper.common.exceptions import InternalServerError +from bookkeeper.common.exceptions import StreamExistsError +from bookkeeper.common.exceptions import StreamNotFoundError from bookkeeper.common.method import wrap_method from bookkeeper.common.retry import Retry from bookkeeper.common.timeout import ExponentialTimeout @@ -38,7 +41,7 @@ def __init__(self, root_range_service, namespace): self.__delete_with_retries__ =\ wrap_method(self.__delete_stream__, self.__default_retry__) - def create(self, stream_name, stream_config=__DEFAULT_STREAM_CONF__): + def create(self, stream_name, stream_config=__DEFAULT_TABLE_CONF__): return self.__create_with_retries__(stream_name, stream_config) def __create_stream__(self, stream_name, stream_config): @@ -51,8 +54,18 @@ def __create_stream__(self, stream_name, stream_config): request=create_stream_req, metadata=__ROOT_RANGE_METADATA__ ) - create_stream_resp = from_root_range_rpc_response(create_stream_resp) - return create_stream_resp.stream_props + try: + create_stream_resp = from_root_range_rpc_response(create_stream_resp) + return create_stream_resp.stream_props + except InternalServerError as ise: + # currently if a stream exists, it also throws + # internal server error + try: + self.get(stream_name=stream_name) + raise StreamExistsError("stream '%s' already exists at namespace '%s'" + % (stream_name, self.__namespace__)) + except StreamNotFoundError: + raise ise def get(self, stream_name): return self.__get_with_retries__(stream_name) diff --git a/stream/clients/python/bookkeeper/admin/namespaces.py b/stream/clients/python/bookkeeper/admin/namespaces.py index 3df8cbd57d3..d0e66352ffa 100644 --- a/stream/clients/python/bookkeeper/admin/namespaces.py +++ b/stream/clients/python/bookkeeper/admin/namespaces.py @@ -17,6 +17,9 @@ from bookkeeper.common.constants import __DEFAULT_NS_CONF__ from bookkeeper.common.constants import __ROOT_RANGE_METADATA__ from bookkeeper.common.exceptions import from_root_range_rpc_response +from bookkeeper.common.exceptions import InternalServerError +from bookkeeper.common.exceptions import NamespaceExistsError +from bookkeeper.common.exceptions import NamespaceNotFoundError from bookkeeper.common.method import wrap_method from bookkeeper.common.retry import Retry from bookkeeper.common.timeout import ExponentialTimeout @@ -48,8 +51,17 @@ def __create_ns__(self, namespace, namespace_config): request=create_ns_req, metadata=__ROOT_RANGE_METADATA__ ) - create_ns_resp = from_root_range_rpc_response(create_ns_resp) - return create_ns_resp.ns_props + try: + create_ns_resp = from_root_range_rpc_response(create_ns_resp) + return create_ns_resp.ns_props + except InternalServerError as ise: + # currently if a namespace exists, it also throws + # internal server error. + try: + self.get(namespace=namespace) + raise NamespaceExistsError("namespace '%s' already exists" % namespace) + except NamespaceNotFoundError: + raise ise def get(self, namespace): return self.__get_with_retries__(namespace) diff --git a/stream/clients/python/bookkeeper/common/constants.py b/stream/clients/python/bookkeeper/common/constants.py index b21f4bcceb0..8c72791ff0d 100644 --- a/stream/clients/python/bookkeeper/common/constants.py +++ b/stream/clients/python/bookkeeper/common/constants.py @@ -20,6 +20,28 @@ ('bk-rt-sc-id-bin', util.to_bytes(__ROOT_RANGE_ID__, 8, "big")) ] __DEFAULT_STREAM_CONF__ = stream_pb2.StreamConfiguration( + key_type=stream_pb2.RangeKeyType.values()[0], + min_num_ranges=24, + initial_num_ranges=4, + split_policy=stream_pb2.SplitPolicy( + type=stream_pb2.SplitPolicyType.values()[0], + fixed_range_policy=stream_pb2.FixedRangeSplitPolicy( + num_ranges=2 + ) + ), + rolling_policy=stream_pb2.SegmentRollingPolicy( + size_policy=stream_pb2.SizeBasedSegmentRollingPolicy( + max_segment_size=128*1024*1024 + ) + ), + retention_policy=stream_pb2.RetentionPolicy( + time_policy=stream_pb2.TimeBasedRetentionPolicy( + retention_minutes=-1 + ) + ), + storage_type=stream_pb2.StorageType.values()[0] +) +__DEFAULT_TABLE_CONF__ = stream_pb2.StreamConfiguration( key_type=stream_pb2.RangeKeyType.values()[0], min_num_ranges=24, initial_num_ranges=4, diff --git a/stream/clients/python/bookkeeper/common/exceptions.py b/stream/clients/python/bookkeeper/common/exceptions.py index 782a579d5b7..531b79a88d9 100644 --- a/stream/clients/python/bookkeeper/common/exceptions.py +++ b/stream/clients/python/bookkeeper/common/exceptions.py @@ -98,7 +98,8 @@ def __init__(self, message, errors=(), response=None): self._response = response def __str__(self): - return '{} {}'.format(self.code, self.message) + return 'grpc_status_code = {}, bk_status_code = {} : {}'\ + .format(self.grpc_status_code, self.bk_status_code, self.message) @property def errors(self): diff --git a/stream/clients/python/bookkeeper/kv/client.py b/stream/clients/python/bookkeeper/kv/client.py index 3de12919905..e942fa634fa 100644 --- a/stream/clients/python/bookkeeper/kv/client.py +++ b/stream/clients/python/bookkeeper/kv/client.py @@ -16,7 +16,6 @@ import grpc import logging -import pkg_resources from bookkeeper import types from bookkeeper.admin.namespace import Namespace @@ -24,7 +23,6 @@ from bookkeeper.kv.table import Table from bookkeeper.proto.storage_pb2_grpc import RootRangeServiceStub -__version__ = pkg_resources.get_distribution('bookkeeper').version __logger__ = logging.getLogger("bookkeeper.kv.Client") diff --git a/stream/clients/python/bookkeeper/kv/table.py b/stream/clients/python/bookkeeper/kv/table.py index 605ab5d525a..c5be5cad5ef 100644 --- a/stream/clients/python/bookkeeper/kv/table.py +++ b/stream/clients/python/bookkeeper/kv/table.py @@ -40,6 +40,8 @@ def __init__(self, channel, stream_props): wrap_method(self.__do_get__, self.__default_retry__) self.__del_with_retries__ =\ wrap_method(self.__do_del__, self.__default_retry__) + self.__incr_with_retries__ =\ + wrap_method(self.__do_incr__, self.__default_retry__) __logger__.info("initialized table instance with properties : %s", stream_props) @@ -79,6 +81,27 @@ def __do_put__(self, key, value, routing_header, grpc_metadata): ) from_table_rpc_response(put_resp) + def incr_str(self, key_str, amount): + key = key_str.encode('utf-8') + return self.incr(key, amount) + + def incr(self, key, amount): + metadata = self.__make_routing_metadata__(key) + header = self.__make_routing_header__(key) + return self.__incr_with_retries__(key, amount, header, metadata) + + def __do_incr__(self, key, amount, routing_header, grpc_metadata): + incr_req = kv_rpc_pb2.IncrementRequest( + key=key, + amount=amount, + header=routing_header + ) + incr_resp = self.__table_service__.Increment( + request=incr_req, + metadata=grpc_metadata + ) + from_table_rpc_response(incr_resp) + def get_str(self, key_str): key = key_str.encode('utf-8') return self.get(key) diff --git a/stream/clients/python/nox.py b/stream/clients/python/nox.py index 5966201c4a0..6bcb2d9f77d 100644 --- a/stream/clients/python/nox.py +++ b/stream/clients/python/nox.py @@ -45,6 +45,33 @@ def default(session): *session.posargs ) +@nox.session +def integration(session): + """Default integration test session. + This is intended to be run **without** an interpreter set, so + that the current ``python`` (on the ``PATH``) or the version of + Python corresponding to the ``nox`` binary the ``PATH`` can + run the tests. + """ + # Install all test dependencies, then install local packages in-place. + session.install('pytest', 'pytest-cov') + for local_dep in LOCAL_DEPS: + session.install('-e', local_dep) + session.install('-e', '.') + + # Run py.test against the unit tests. + session.run( + 'py.test', + '--quiet', + '--cov-append', + '--cov-report=', + '--cov=bookkeeper', + '--cov-config=.coveragerc', + os.path.join('tests', 'integration'), + *session.posargs + ) + + @nox.session def lint(session): diff --git a/stream/clients/python/noxfile.py b/stream/clients/python/noxfile.py index 3027c9212ca..612afc1742d 100644 --- a/stream/clients/python/noxfile.py +++ b/stream/clients/python/noxfile.py @@ -52,6 +52,33 @@ def unit(session): default(session) +@nox.session(python=[os.environ['PY_VERSION']]) +def integration(session): + """Default integration test session. + This is intended to be run **without** an interpreter set, so + that the current ``python`` (on the ``PATH``) or the version of + Python corresponding to the ``nox`` binary the ``PATH`` can + run the tests. + """ + # Install all test dependencies, then install local packages in-place. + session.install('pytest', 'pytest-cov') + for local_dep in LOCAL_DEPS: + session.install('-e', local_dep) + session.install('-e', '.') + + # Run py.test against the unit tests. + session.run( + 'py.test', + '--quiet', + '--cov-append', + '--cov-report=', + '--cov=bookkeeper', + '--cov-config=.coveragerc', + os.path.join('tests', 'integration'), + *session.posargs + ) + + @nox.session def lint(session): """Run linters. diff --git a/stream/clients/python/scripts/docker_build.sh b/stream/clients/python/scripts/docker_build.sh new file mode 100755 index 00000000000..f3bd918f805 --- /dev/null +++ b/stream/clients/python/scripts/docker_build.sh @@ -0,0 +1,34 @@ +#!/bin/bash + +# 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. + +set -e -x -u + +SCRIPT_DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +PY_VERSION=${PY_VERSION:-"3.7"} + +COMMANDS=`cat < start() { channel = InProcessChannelBuilder.forName(containerName) .usePlaintext() .directExecutor() + // attach routing header interceptor + .intercept(new RoutingHeaderProxyInterceptor()) .build(); return FutureUtils.value(StorageContainerImpl.this); } catch (IOException e) { diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java index e3b2d3991df..5fe1612f657 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java @@ -41,6 +41,7 @@ import org.apache.bookkeeper.stream.proto.kv.rpc.PutResponse; import org.apache.bookkeeper.stream.proto.kv.rpc.RangeRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.RangeResponse; +import org.apache.bookkeeper.stream.proto.kv.rpc.ResponseHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.RoutingHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TxnRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.TxnResponse; @@ -58,6 +59,7 @@ import org.apache.bookkeeper.stream.proto.storage.GetNamespaceResponse; import org.apache.bookkeeper.stream.proto.storage.GetStreamRequest; import org.apache.bookkeeper.stream.proto.storage.GetStreamResponse; +import org.apache.bookkeeper.stream.proto.storage.StatusCode; import org.apache.bookkeeper.stream.protocol.RangeId; import org.apache.bookkeeper.stream.protocol.util.StorageContainerPlacementPolicy; import org.apache.bookkeeper.stream.storage.api.kv.TableStore; @@ -242,6 +244,15 @@ public CompletableFuture getActiveRanges(GetActiveRange public CompletableFuture range(RangeRequest request) { RoutingHeader header = request.getHeader(); + if (header.getRangeId() <= 0L) { + return CompletableFuture.completedFuture(RangeResponse.newBuilder() + .setHeader(ResponseHeader.newBuilder() + .setCode(StatusCode.BAD_REQUEST) + .setRoutingHeader(request.getHeader()) + .build()) + .build()); + } + RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); TableStore store = tableStoreCache.getTableStore(rid); if (null != store) { @@ -256,6 +267,15 @@ public CompletableFuture range(RangeRequest request) { public CompletableFuture put(PutRequest request) { RoutingHeader header = request.getHeader(); + if (header.getRangeId() <= 0L) { + return CompletableFuture.completedFuture(PutResponse.newBuilder() + .setHeader(ResponseHeader.newBuilder() + .setCode(StatusCode.BAD_REQUEST) + .setRoutingHeader(request.getHeader()) + .build()) + .build()); + } + RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); TableStore store = tableStoreCache.getTableStore(rid); if (null != store) { @@ -270,6 +290,15 @@ public CompletableFuture put(PutRequest request) { public CompletableFuture delete(DeleteRangeRequest request) { RoutingHeader header = request.getHeader(); + if (header.getRangeId() <= 0L) { + return CompletableFuture.completedFuture(DeleteRangeResponse.newBuilder() + .setHeader(ResponseHeader.newBuilder() + .setCode(StatusCode.BAD_REQUEST) + .setRoutingHeader(request.getHeader()) + .build()) + .build()); + } + RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); TableStore store = tableStoreCache.getTableStore(rid); if (null != store) { @@ -284,6 +313,15 @@ public CompletableFuture delete(DeleteRangeRequest request) public CompletableFuture txn(TxnRequest request) { RoutingHeader header = request.getHeader(); + if (header.getRangeId() <= 0L) { + return CompletableFuture.completedFuture(TxnResponse.newBuilder() + .setHeader(ResponseHeader.newBuilder() + .setCode(StatusCode.BAD_REQUEST) + .setRoutingHeader(request.getHeader()) + .build()) + .build()); + } + RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); TableStore store = tableStoreCache.getTableStore(rid); if (null != store) { @@ -298,6 +336,15 @@ public CompletableFuture txn(TxnRequest request) { public CompletableFuture incr(IncrementRequest request) { RoutingHeader header = request.getHeader(); + if (header.getRangeId() <= 0L) { + return CompletableFuture.completedFuture(IncrementResponse.newBuilder() + .setHeader(ResponseHeader.newBuilder() + .setCode(StatusCode.BAD_REQUEST) + .setRoutingHeader(request.getHeader()) + .build()) + .build()); + } + RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); TableStore store = tableStoreCache.getTableStore(rid); if (null != store) { diff --git a/tests/docker-images/current-version-image/Dockerfile b/tests/docker-images/current-version-image/Dockerfile index c174b6a318f..7d998a46def 100644 --- a/tests/docker-images/current-version-image/Dockerfile +++ b/tests/docker-images/current-version-image/Dockerfile @@ -35,7 +35,8 @@ ENV JAVA_HOME=/usr/lib/jvm/jre-1.8.0 # prepare utils RUN set -x \ && adduser "${BK_USER}" \ - && yum install -y java-1.8.0-openjdk-headless wget bash python sudo netcat \ + && yum install -y epel-release \ + && yum install -y java-1.8.0-openjdk-headless wget bash python-pip python-devel sudo netcat gcc gcc-c++ \ && mkdir -pv /opt \ && cd /opt \ && yum clean all @@ -47,8 +48,13 @@ RUN mv /opt/${PKG_NAME} /opt/bookkeeper WORKDIR /opt/bookkeeper COPY target/scripts /opt/bookkeeper/scripts +COPY scripts/install-python-client.sh /opt/bookkeeper/scripts RUN chmod +x -R /opt/bookkeeper/scripts/ +# copy the python client +ADD target/bookkeeper-client/ /opt/bookkeeper/bookkeeper-client +RUN /opt/bookkeeper/scripts/install-python-client.sh + ENTRYPOINT [ "/bin/bash", "/opt/bookkeeper/scripts/entrypoint.sh" ] CMD ["bookie"] diff --git a/tests/docker-images/current-version-image/pom.xml b/tests/docker-images/current-version-image/pom.xml index 0d2c95da5ce..edffc648143 100644 --- a/tests/docker-images/current-version-image/pom.xml +++ b/tests/docker-images/current-version-image/pom.xml @@ -46,6 +46,51 @@ + + + org.codehaus.mojo + exec-maven-plugin + ${exec-maven-plugin.version} + + + build-python-client + generate-resources + + exec + + + ${project.basedir}/target + ${project.basedir}/../../../stream/clients/python/scripts/docker_build.sh + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + ${maven-antrun-plugin.version} + + + generate-resources + + run + + + + copy python wheel file + + + copying docker scripts + + + + + + + com.spotify dockerfile-maven-plugin @@ -97,27 +142,6 @@ - - - org.apache.maven.plugins - maven-antrun-plugin - ${maven-antrun-plugin.version} - - - generate-resources - - run - - - - copying docker scripts - - - - - - - diff --git a/tests/docker-images/current-version-image/scripts/install-python-client.sh b/tests/docker-images/current-version-image/scripts/install-python-client.sh new file mode 100644 index 00000000000..2719b335ac1 --- /dev/null +++ b/tests/docker-images/current-version-image/scripts/install-python-client.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +# +# 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. +# + +set -x + +WHEEL_FILE=`ls /opt/bookkeeper/bookkeeper-client/*.whl` +pip install ${WHEEL_FILE} From 64ecc98c9d23f144745ee7f5d26ee23b841cdab4 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 31 Oct 2018 01:18:57 -0700 Subject: [PATCH 0099/1642] Reduce stack traces in logs for common cases ### Motivation For common "error" conditions the exception message is the only important bit while the stack traces are just adding clutter the logs files. Reviewers: Andrey Yegorov , Sijie Guo This closes #1776 from merlimat/master --- .../org/apache/bookkeeper/bookie/Journal.java | 4 ++-- .../proto/PerChannelBookieClient.java | 23 ++++++++++++++++--- 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 0db23c63453..25680925f9a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -513,7 +513,7 @@ public void run() { running = false; } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.error("ForceWrite thread interrupted", e); + LOG.info("ForceWrite thread interrupted"); // close is idempotent if (null != req) { req.shouldClose = true; @@ -1142,7 +1142,7 @@ public void run() { LOG.error("I/O exception in Journal thread!", ioe); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); - LOG.warn("Journal exits when shutting down", ie); + LOG.info("Journal exits when shutting down"); } finally { // There could be packets queued for forceWrite on this logFile // That is fine as this exception is going to anyway take down the 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 79e4de912bc..071b8395bf7 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 @@ -49,6 +49,7 @@ import io.netty.channel.epoll.EpollSocketChannel; import io.netty.channel.local.LocalChannel; import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.channel.unix.Errors.NativeIoException; import io.netty.handler.codec.CorruptedFrameException; import io.netty.handler.codec.DecoderException; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; @@ -62,6 +63,7 @@ import java.io.IOException; import java.net.SocketAddress; +import java.net.UnknownHostException; import java.security.cert.Certificate; import java.util.ArrayDeque; import java.util.ArrayList; @@ -1137,7 +1139,13 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E } if (cause instanceof IOException) { - LOG.warn("Exception caught on:{} cause:", ctx.channel(), cause); + if (cause instanceof NativeIoException) { + // Stack trace is not very interesting for native IO exceptio, the important part is in + // the exception message + LOG.warn("Exception caught on:{} cause: {}", ctx.channel(), cause.getMessage()); + } else { + LOG.warn("Exception caught on:{} cause:", ctx.channel(), cause); + } ctx.close(); return; } @@ -2254,8 +2262,17 @@ public void operationComplete(ChannelFuture future) throws Exception { closeChannel(future.channel()); return; // pendingOps should have been completed when other channel connected } else { - LOG.error("Could not connect to bookie: {}/{}, current state {} : ", - future.channel(), addr, state, future.cause()); + Throwable cause = future.cause(); + if (cause instanceof UnknownHostException || cause instanceof NativeIoException) { + // Don't log stack trace for common errors + LOG.warn("Could not connect to bookie: {}/{}, current state {} : {}", + future.channel(), addr, state, future.cause().getMessage()); + } else { + // Regular exceptions, include stack trace + LOG.error("Could not connect to bookie: {}/{}, current state {} : ", + future.channel(), addr, state, future.cause()); + } + rc = BKException.Code.BookieHandleNotAvailableException; closeChannel(future.channel()); channel = null; From c85d3dd154817606babacfcf3ed30e0f2f2d0f2b Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 2 Nov 2018 09:59:38 -0700 Subject: [PATCH 0100/1642] [TESTS] Issue #1752: disable etcd tests in normal builds Descriptions of the changes in this PR: *Motivation* apache/bookkeeper#1766 attempts to disable etcd tests in normal builds. however it only added the `integrationTests` profile but didn't disable tests in normal builds. *Changes* Disable etcd tests in normal builds Master Issue: #1752 Reviewers: Enrico Olivelli This closes #1779 from sijie/run_etcd_tests_on_integrationtests, closes #1752 --- metadata-drivers/etcd/pom.xml | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/metadata-drivers/etcd/pom.xml b/metadata-drivers/etcd/pom.xml index 8e3b3390546..abc9f6e5d92 100644 --- a/metadata-drivers/etcd/pom.xml +++ b/metadata-drivers/etcd/pom.xml @@ -69,6 +69,24 @@ test + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + true + + ${project.version} + ${project.build.directory} + + + + + + integrationTests From 1480a2d31818a4312d2c576f472bbe49f73bfb15 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 2 Nov 2018 14:41:09 -0700 Subject: [PATCH 0101/1642] [TABLE SERVICE] add default port to service hosts Descriptions of the changes in this PR: *Motivation* Currently we have to add `4181` in the service uri in order to make it work. Ideally it would be great that the system can fill the default port if port is missing. *Changes* add default port when parsing `ServiceURI` Reviewers: Enrico Olivelli This closes #1778 from sijie/add_default_port --- .../bookkeeper/common/net/ServiceURI.java | 19 +++++++++++++++---- .../bookkeeper/common/net/ServiceURITest.java | 16 ++++++++++++++-- 2 files changed, 29 insertions(+), 6 deletions(-) diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java index 4e2b158df40..3d419f3ccfc 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java @@ -27,6 +27,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.net.URI; import java.util.List; +import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.EqualsAndHashCode; import lombok.Getter; @@ -122,6 +123,7 @@ public class ServiceURI { * Service string for bookkeeper service. */ public static final String SERVICE_BK = "bk"; + public static final int SERVICE_BK_PORT = 4181; /** * The default local bk service uri. @@ -160,7 +162,7 @@ public static ServiceURI create(String uriStr) { public static ServiceURI create(URI uri) { checkNotNull(uri, "service uri instance is null"); - String serviceName = null; + String serviceName; String[] serviceInfos = new String[0]; String scheme = uri.getScheme(); if (null != scheme) { @@ -175,6 +177,8 @@ public static ServiceURI create(URI uri) { serviceName = schemeParts[0]; serviceInfos = new String[schemeParts.length - 1]; System.arraycopy(schemeParts, 1, serviceInfos, 0, serviceInfos.length); + } else { + serviceName = null; } String userAndHostInformation = uri.getAuthority(); @@ -192,7 +196,10 @@ public static ServiceURI create(URI uri) { serviceUser = null; serviceHosts = splitter.splitToList(userAndHostInformation); } - serviceHosts.forEach(host -> validateHostName(host)); + serviceHosts = serviceHosts + .stream() + .map(host -> validateHostName(serviceName, host)) + .collect(Collectors.toList()); String servicePath = uri.getPath(); checkArgument(null != servicePath, @@ -207,7 +214,7 @@ public static ServiceURI create(URI uri) { uri); } - private static void validateHostName(String hostname) { + private static String validateHostName(String serviceName, String hostname) { String[] parts = hostname.split(":"); if (parts.length >= 3) { throw new IllegalArgumentException("Invalid hostname : " + hostname); @@ -217,8 +224,12 @@ private static void validateHostName(String hostname) { } catch (NumberFormatException nfe) { throw new IllegalArgumentException("Invalid hostname : " + hostname); } + return hostname; + } else if (parts.length == 1 && serviceName.toLowerCase().equals(SERVICE_BK)) { + return hostname + ":" + SERVICE_BK_PORT; + } else { + return hostname; } - } private final String serviceName; diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/net/ServiceURITest.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/net/ServiceURITest.java index 9982c43a08b..2ec83df32c0 100644 --- a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/net/ServiceURITest.java +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/net/ServiceURITest.java @@ -154,7 +154,19 @@ public void testMultipleHostsWithoutPorts() { "bk", new String[0], null, - new String[] { "host1", "host2", "host3" }, + new String[] { "host1:4181", "host2:4181", "host3:4181" }, + "/path/to/namespace"); + } + + @Test + public void testMultipleHostsMixedPorts() { + String serviceUri = "bk://host1:3181,host2,host3:2181/path/to/namespace"; + assertServiceUri( + serviceUri, + "bk", + new String[0], + null, + new String[] { "host1:3181", "host2:4181", "host3:2181" }, "/path/to/namespace"); } @@ -166,7 +178,7 @@ public void testMultipleHostsMixed() { "bk", new String[0], null, - new String[] { "host1:2181", "host2", "host3:2181" }, + new String[] { "host1:2181", "host2:4181", "host3:2181" }, "/path/to/namespace"); } From 84d0727f54c2e937465224d678877a3c7a752a16 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 2 Nov 2018 17:47:37 -0700 Subject: [PATCH 0102/1642] [TABLE SERVICE] [PYTHON] add default port to service hosts Descriptions of the changes in this PR: *Motivation* Similar as #1778, add default port to service hosts if port is missing. *Changes* add default port to service hosts if they miss ports Related Issue: #1778 Reviewers: Jia Zhai This closes #1780 from sijie/fix_python_service_uri --- .../python/bookkeeper/common/service_uri.py | 18 +++--- .../clients/python/bookkeeper/common/util.py | 8 +++ .../bookkeeper/common/test_service_uri.py | 55 +++++++++++++++++++ .../tests/unit/bookkeeper/common/test_util.py | 21 +++++++ 4 files changed, 94 insertions(+), 8 deletions(-) create mode 100644 stream/clients/python/tests/unit/bookkeeper/common/test_service_uri.py create mode 100644 stream/clients/python/tests/unit/bookkeeper/common/test_util.py diff --git a/stream/clients/python/bookkeeper/common/service_uri.py b/stream/clients/python/bookkeeper/common/service_uri.py index ee8cd93070d..7cf782086b0 100644 --- a/stream/clients/python/bookkeeper/common/service_uri.py +++ b/stream/clients/python/bookkeeper/common/service_uri.py @@ -10,14 +10,13 @@ # See the License for the specific language governing permissions and # limitations under the License. -import sys +from bookkeeper.common.util import __PYTHON3__ +from bookkeeper.common.util import new_hostname_with_port -if sys.version_info[0] < 3: - USE_PYTHON3 = False - from urlparse import urlparse -else: - USE_PYTHON3 = True +if __PYTHON3__: from urllib.parse import urlparse +else: + from urlparse import urlparse class ServiceURI(object): @@ -27,5 +26,8 @@ def __init__(self, service_uri): self.service_name = self.uri.scheme self.service_user = self.uri.username self.service_path = self.uri.path - self.service_location = self.uri.netloc - self.service_hosts = self.uri.netloc.split(',') + if __PYTHON3__: + self.service_hosts = list(map(lambda x: new_hostname_with_port(x), self.uri.netloc.split(','))) + else: + self.service_hosts = map(lambda x: new_hostname_with_port(x), self.uri.netloc.split(',')) + self.service_location = ','.join(self.service_hosts) diff --git a/stream/clients/python/bookkeeper/common/util.py b/stream/clients/python/bookkeeper/common/util.py index 54f11d9621d..cb47176406e 100644 --- a/stream/clients/python/bookkeeper/common/util.py +++ b/stream/clients/python/bookkeeper/common/util.py @@ -24,3 +24,11 @@ def to_bytes(n, length, endianess='big'): h = '%x' % n s = ('0'*(len(h) % 2) + h).zfill(length*2).decode('hex') return s if endianess == 'big' else s[::-1] + + +def new_hostname_with_port(hostname, default_port=4181): + host_parts = hostname.split(':') + if len(host_parts) > 1: + return hostname + else: + return "%s:%d" % (hostname, default_port) diff --git a/stream/clients/python/tests/unit/bookkeeper/common/test_service_uri.py b/stream/clients/python/tests/unit/bookkeeper/common/test_service_uri.py new file mode 100644 index 00000000000..bcb0ca876a5 --- /dev/null +++ b/stream/clients/python/tests/unit/bookkeeper/common/test_service_uri.py @@ -0,0 +1,55 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +from bookkeeper.common.service_uri import ServiceURI + + +def test_service_uri_one_host_without_port(): + uri = ServiceURI("bk://127.0.0.1") + assert "bk" == uri.service_name + assert uri.service_user is None + assert "127.0.0.1:4181" == uri.service_location + assert ["127.0.0.1:4181"] == uri.service_hosts + + +def test_service_uri_one_host_with_port(): + uri = ServiceURI("bk://127.0.0.1:3181") + assert "bk" == uri.service_name + assert uri.service_user is None + assert "127.0.0.1:3181" == uri.service_location + assert ["127.0.0.1:3181"] == uri.service_hosts + + +def test_service_uri_multiple_hosts_with_port(): + uri = ServiceURI("bk://127.0.0.1:3181,127.0.0.2:4181,127.0.0.3:5181") + assert "bk" == uri.service_name + assert uri.service_user is None + assert "127.0.0.1:3181,127.0.0.2:4181,127.0.0.3:5181" == uri.service_location + assert ["127.0.0.1:3181", "127.0.0.2:4181", "127.0.0.3:5181"] == uri.service_hosts + + +def test_service_uri_multiple_hosts_without_port(): + uri = ServiceURI("bk://127.0.0.1,127.0.0.2,127.0.0.3") + assert "bk" == uri.service_name + assert uri.service_user is None + assert "127.0.0.1:4181,127.0.0.2:4181,127.0.0.3:4181" == uri.service_location + assert ["127.0.0.1:4181", "127.0.0.2:4181", "127.0.0.3:4181"] == uri.service_hosts + + +def test_service_uri_multiple_hosts_mixed_with_and_without_port(): + uri = ServiceURI("bk://127.0.0.1:3181,127.0.0.2,127.0.0.3:5181") + assert "bk" == uri.service_name + assert uri.service_user is None + assert "127.0.0.1:3181,127.0.0.2:4181,127.0.0.3:5181" == uri.service_location + assert ["127.0.0.1:3181", "127.0.0.2:4181", "127.0.0.3:5181"] == uri.service_hosts diff --git a/stream/clients/python/tests/unit/bookkeeper/common/test_util.py b/stream/clients/python/tests/unit/bookkeeper/common/test_util.py new file mode 100644 index 00000000000..214d3f19d1b --- /dev/null +++ b/stream/clients/python/tests/unit/bookkeeper/common/test_util.py @@ -0,0 +1,21 @@ +# -*- coding: utf-8 -*- +# +# Licensed 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 +# +# https://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. + +from bookkeeper.common import util + + +def test_new_hostname_with_port(): + assert "127.0.0.1:3181" == util.new_hostname_with_port("127.0.0.1:3181") + assert "127.0.0.1:4181" == util.new_hostname_with_port("127.0.0.1") + assert "127.0.0.1:2181" == util.new_hostname_with_port("127.0.0.1", 2181) From a490f488c71f66600de3f9a2483c3a69772a1fdb Mon Sep 17 00:00:00 2001 From: Manjiri Tapaswi Date: Sat, 3 Nov 2018 10:41:19 -0700 Subject: [PATCH 0103/1642] ISSUE #1767: security vulnerabilities in 3rd party dependencies Descriptions of the changes in this PR: Upgraded jline and jackson to remove security vulnerabilities mentioned in #1767 ### Motivation Remove security vulnerabilities mentioned in #1767 ### Changes Upgraded jline and jackson 3rd party dependencies Master Issue: #1767 Reviewers: Enrico Olivelli This closes #1777 from mptap/fix-1767-remove-vulnerabilities, closes #1767 --- bookkeeper-dist/src/assemble/bin-all.xml | 2 +- bookkeeper-dist/src/assemble/bin-server.xml | 2 +- .../src/main/resources/LICENSE-all.bin.txt | 24 +++++++++---------- .../src/main/resources/LICENSE-server.bin.txt | 12 +++++----- .../deps/{jline-0.9.94 => jline-2.11}/LICENSE | 4 +++- pom.xml | 18 +++++++++++++- 6 files changed, 40 insertions(+), 22 deletions(-) rename bookkeeper-dist/src/main/resources/deps/{jline-0.9.94 => jline-2.11}/LICENSE (92%) diff --git a/bookkeeper-dist/src/assemble/bin-all.xml b/bookkeeper-dist/src/assemble/bin-all.xml index 7b047c44682..c1d8b439a85 100644 --- a/bookkeeper-dist/src/assemble/bin-all.xml +++ b/bookkeeper-dist/src/assemble/bin-all.xml @@ -61,7 +61,7 @@ netty-4.1.22.Final/* paranamer-2.8/LICENSE.txt protobuf-3.0.0/LICENSE - jline-0.9.94/LICENSE + jline-2.11/LICENSE protobuf-3.5.1/LICENSE scala-library-2.11.7/LICENSE.md scala-parser-combinators_2.11-1.0.4/LICENSE.md diff --git a/bookkeeper-dist/src/assemble/bin-server.xml b/bookkeeper-dist/src/assemble/bin-server.xml index aa7d1b89c8c..46054c730fa 100644 --- a/bookkeeper-dist/src/assemble/bin-server.xml +++ b/bookkeeper-dist/src/assemble/bin-server.xml @@ -53,7 +53,7 @@ javax.servlet-api-3.1.0/CDDL+GPL-1.1 netty-4.1.22.Final/* protobuf-3.0.0/LICENSE - jline-0.9.94/LICENSE + jline-2.11/LICENSE protobuf-3.5.1/LICENSE slf4j-1.7.25/LICENSE.txt diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index 0b181f1e20f..748ca2497d0 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -205,11 +205,11 @@ The following bundled 3rd party jars are distributed under the Apache Software License, Version 2. -- lib/com.fasterxml.jackson.core-jackson-annotations-2.8.9.jar [1] -- lib/com.fasterxml.jackson.core-jackson-core-2.8.9.jar [2] -- lib/com.fasterxml.jackson.core-jackson-databind-2.8.9.jar [3] -- lib/com.fasterxml.jackson.module-jackson-module-paranamer-2.8.4.jar [4] -- lib/com.fasterxml.jackson.module-jackson-module-scala_2.11-2.8.4.jar [5] +- lib/com.fasterxml.jackson.core-jackson-annotations-2.9.7.jar [1] +- lib/com.fasterxml.jackson.core-jackson-core-2.9.7.jar [2] +- lib/com.fasterxml.jackson.core-jackson-databind-2.9.7.jar [3] +- lib/com.fasterxml.jackson.module-jackson-module-paranamer-2.9.7.jar [4] +- lib/com.fasterxml.jackson.module-jackson-module-scala_2.11-2.9.7.jar [5] - lib/com.github.ben-manes.caffeine-caffeine-2.3.4.jar [9] - lib/com.google.guava-guava-21.0.jar [6] - lib/commons-cli-commons-cli-1.2.jar [7] @@ -300,11 +300,11 @@ Apache Software License, Version 2. - lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [48] - lib/org.apache.yetus-audience-annotations-0.5.0.jar [49] -[1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9 -[2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9 -[3] Source available at https://github.com/FasterXML/jackson-databind/tree/jackson-databind-2.8.9 -[4] Source available at https://github.com/FasterXML/jackson-modules-base/tree/jackson-modules-base-2.8.4 -[5] Source available at https://github.com/FasterXML/jackson-module-scala/tree/f9e099 +[1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.9.7 +[2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.9.7 +[3] Source available at https://github.com/FasterXML/jackson-databind/tree/jackson-databind-2.9.7 +[4] Source available at https://github.com/FasterXML/jackson-modules-base/tree/jackson-modules-base-2.9.7 +[5] Source available at https://github.com/FasterXML/jackson-module-scala/tree/jackson-module-scala-2.9.7 [6] Source available at https://github.com/google/guava/tree/v21.0 [7] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-cli.git;a=tag;h=bc8f0e [8] Source available at http://svn.apache.org/viewvc/commons/proper/codec/tags/1_6/ @@ -550,8 +550,8 @@ Bundled as Source available at https://github.com/google/google-auth-library-java/tree/0.9.0 ------------------------------------------------------------------------------------ This product bundles the JLine Library, which is available under a "2-clause BSD" -license. For details, see deps/jline-0.9.94/LICENSE +license. For details, see deps/jline-2.11/LICENSE Bundled as - - lib/jline-jline-0.9.94.jar + - lib/jline-jline-2.11.jar diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index 6b630b4ccde..bd023c87518 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -205,9 +205,9 @@ The following bundled 3rd party jars are distributed under the Apache Software License, Version 2. -- lib/com.fasterxml.jackson.core-jackson-annotations-2.8.9.jar [1] -- lib/com.fasterxml.jackson.core-jackson-core-2.8.9.jar [2] -- lib/com.fasterxml.jackson.core-jackson-databind-2.8.9.jar [3] +- lib/com.fasterxml.jackson.core-jackson-annotations-2.9.7.jar [1] +- lib/com.fasterxml.jackson.core-jackson-core-2.9.7.jar [2] +- lib/com.fasterxml.jackson.core-jackson-databind-2.9.7.jar [3] - lib/com.google.guava-guava-21.0.jar [4] - lib/commons-cli-commons-cli-1.2.jar [5] - lib/commons-codec-commons-codec-1.6.jar [6] @@ -267,7 +267,7 @@ Apache Software License, Version 2. [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9 -[3] Source available at https://github.com/FasterXML/jackson-databind/tree/jackson-databind-2.8.9 +[3] Source available at https://github.com/FasterXML/jackson-databind/tree/jackson-databind-2.9.7 [4] Source available at https://github.com/google/guava/tree/v21.0 [5] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-cli.git;a=tag;h=bc8f0e [6] Source available at http://svn.apache.org/viewvc/commons/proper/codec/tags/1_6/ @@ -438,7 +438,7 @@ Bundled as Source available at https://github.com/google/google-auth-library-java/tree/0.9.0 ------------------------------------------------------------------------------------ This product bundles the JLine Library, which is available under a "2-clause BSD" -license. For details, see deps/jline-0.9.94/LICENSE +license. For details, see deps/jline-2.11/LICENSE Bundled as - - lib/jline-jline-0.9.94.jar + - lib/jline-jline-2.11.jar diff --git a/bookkeeper-dist/src/main/resources/deps/jline-0.9.94/LICENSE b/bookkeeper-dist/src/main/resources/deps/jline-2.11/LICENSE similarity index 92% rename from bookkeeper-dist/src/main/resources/deps/jline-0.9.94/LICENSE rename to bookkeeper-dist/src/main/resources/deps/jline-2.11/LICENSE index 246f54f7365..9ef434ebc2f 100644 --- a/bookkeeper-dist/src/main/resources/deps/jline-0.9.94/LICENSE +++ b/bookkeeper-dist/src/main/resources/deps/jline-2.11/LICENSE @@ -1,6 +1,8 @@ -Copyright (c) 2002-2006, Marc Prud'hommeaux +Copyright (c) 2002-2012, the original author or authors. All rights reserved. +http://www.opensource.org/licenses/bsd-license.php + Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: diff --git a/pom.xml b/pom.xml index 95aaf73f7c6..8d43ecda2e6 100644 --- a/pom.xml +++ b/pom.xml @@ -133,10 +133,11 @@ 2.7.3 1.3 2.1.10 - 2.8.9 + 2.9.7 1.9.11 1.48 9.4.5.v20170502 + 2.11 1.19 2.8.2 3.2.7 @@ -226,6 +227,11 @@ freebuilder ${freebuilder.version} + + jline + jline + ${jline.version} + @@ -321,6 +327,16 @@ jackson-annotations ${jackson.version} + + com.fasterxml.jackson.module + jackson-module-paranamer + ${jackson.version} + + + com.fasterxml.jackson.module + jackson-module-scala_2.11 + ${jackson.version} + org.codehaus.jackson From 9d6d3caad354f56b3eabd0bf31be4c3c72606492 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Sat, 3 Nov 2018 10:42:49 -0700 Subject: [PATCH 0104/1642] Use default metric registry in Prometheus exporter ### Motivation Currently the Prometheus exporter is using a private metrics register. This doesn't work well for BK client where we want to expose in the same registry of application, or when we want to interact with other tools that use the default static registry (eg: log4j, jetty integrations for prometheus). Reviewers: Enrico Olivelli , Sijie Guo This closes #1765 from merlimat/prometheus --- .../prometheus/PrometheusMetricsProvider.java | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java index 645a686a4ee..df8279e68b0 100644 --- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java +++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java @@ -22,7 +22,7 @@ //CHECKSTYLE.OFF: IllegalImport import io.netty.util.internal.PlatformDependent; //CHECKSTYLE.ON: IllegalImport - +import io.prometheus.client.Collector; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Gauge; import io.prometheus.client.Gauge.Child; @@ -81,7 +81,7 @@ public class PrometheusMetricsProvider implements StatsProvider { final ConcurrentMap opStats = new ConcurrentSkipListMap<>(); public PrometheusMetricsProvider() { - this(new CollectorRegistry()); + this(CollectorRegistry.defaultRegistry); } public PrometheusMetricsProvider(CollectorRegistry registry) { @@ -128,25 +128,25 @@ public void start(Configuration conf) { } // Include standard JVM stats - new StandardExports().register(registry); - new MemoryPoolsExports().register(registry); - new GarbageCollectorExports().register(registry); - new ThreadExports().register(registry); + registerMetrics(new StandardExports()); + registerMetrics(new MemoryPoolsExports()); + registerMetrics(new GarbageCollectorExports()); + registerMetrics(new ThreadExports()); // Add direct memory allocated through unsafe - Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { + registerMetrics(Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { @Override public double get() { return directMemoryUsage != null ? directMemoryUsage.longValue() : Double.NaN; } - }).register(registry); + })); - Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { + registerMetrics(Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { @Override public double get() { return PlatformDependent.maxDirectMemory(); } - }).register(registry); + })); executor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("metrics")); @@ -191,6 +191,18 @@ void rotateLatencyCollection() { }); } + private void registerMetrics(Collector collector) { + try { + collector.register(registry); + } catch (Exception e) { + // Ignore if these were already registered + if (log.isDebugEnabled()) { + log.debug("Failed to register Prometheus collector exports", e); + } + } + } + + private static final Logger log = LoggerFactory.getLogger(PrometheusMetricsProvider.class); /* From c24465d40226bb7e93ae116e2501dc3db568e6f6 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 6 Nov 2018 12:22:23 -0800 Subject: [PATCH 0105/1642] Upgrade to Netty 4.1.31 and use individual dependencies ### Motivation * Upgrade to latest Netty version which brings in perf improvements and some new features (eg: https://github.com/netty/netty/pull/8267) * Broke down the dependencies from `netty-all` into individual components, as discussed at https://github.com/apache/bookkeeper/pull/1755#discussion_r228449352 Reviewers: Ivan Kelly , Enrico Olivelli , Sijie Guo This closes #1784 from merlimat/netty-4.1.31 --- bookkeeper-dist/src/assemble/bin-all.xml | 17 +----- bookkeeper-dist/src/assemble/bin-server.xml | 16 +----- .../src/main/resources/LICENSE-all.bin.txt | 56 ++++++++++-------- .../src/main/resources/LICENSE-server.bin.txt | 56 ++++++++++-------- .../src/main/resources/NOTICE-all.bin.txt | 18 +++++- .../src/main/resources/NOTICE-server.bin.txt | 17 +++++- .../netty-4.1.22.Final/LICENSE.webbit.txt | 37 ------------ .../LICENSE.base64.txt | 0 .../LICENSE.jbzip2.txt | 0 .../LICENSE.jfastlz.txt | 0 .../LICENSE.jsr166y.txt | 0 .../LICENSE.libdivsufsort.txt | 0 .../LICENSE.protobuf.txt | 0 .../LICENSE.slf4j.txt | 0 bookkeeper-server/pom.xml | 20 ++----- .../prometheus-metrics-provider/pom.xml | 2 +- microbenchmarks/pom.xml | 4 -- pom.xml | 57 ++++++++++++++++++- stream/distributedlog/core/pom.xml | 11 ---- 19 files changed, 157 insertions(+), 154 deletions(-) delete mode 100644 bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.webbit.txt rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.22.Final => netty-4.1.31.Final}/LICENSE.base64.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.22.Final => netty-4.1.31.Final}/LICENSE.jbzip2.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.22.Final => netty-4.1.31.Final}/LICENSE.jfastlz.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.22.Final => netty-4.1.31.Final}/LICENSE.jsr166y.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.22.Final => netty-4.1.31.Final}/LICENSE.libdivsufsort.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.22.Final => netty-4.1.31.Final}/LICENSE.protobuf.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.22.Final => netty-4.1.31.Final}/LICENSE.slf4j.txt (100%) diff --git a/bookkeeper-dist/src/assemble/bin-all.xml b/bookkeeper-dist/src/assemble/bin-all.xml index c1d8b439a85..9e11c8ef939 100644 --- a/bookkeeper-dist/src/assemble/bin-all.xml +++ b/bookkeeper-dist/src/assemble/bin-all.xml @@ -58,7 +58,7 @@ javax.servlet-api-3.1.0/CDDL+GPL-1.1 jsr-305/LICENSE netty-3.10.1.Final/* - netty-4.1.22.Final/* + netty-4.1.31.Final/* paranamer-2.8/LICENSE.txt protobuf-3.0.0/LICENSE jline-2.11/LICENSE @@ -99,21 +99,6 @@ ${artifact.groupId}-${artifact.artifactId}-${artifact.version}${dashClassifier?}.${artifact.extension} - - io.netty:netty-buffer - io.netty:netty-codec - io.netty:netty-codec-dns - io.netty:netty-codec-http - io.netty:netty-codec-http2 - io.netty:netty-codec-socks - io.netty:netty-common - io.netty:netty-handler - io.netty:netty-handler-proxy - io.netty:netty-resolver - io.netty:netty-resolver-dns - io.netty:netty-tcnative-boringssl-static - io.netty:netty-transport-native-epoll - io.netty:netty-transport org.apache.bookkeeper:stream-storage-common org.apache.bookkeeper:stream-storage-proto diff --git a/bookkeeper-dist/src/assemble/bin-server.xml b/bookkeeper-dist/src/assemble/bin-server.xml index 46054c730fa..ee88fe0d179 100644 --- a/bookkeeper-dist/src/assemble/bin-server.xml +++ b/bookkeeper-dist/src/assemble/bin-server.xml @@ -51,7 +51,7 @@ google-auth-library-credentials-0.9.0/LICENSE javax.servlet-api-3.1.0/CDDL+GPL-1.1 - netty-4.1.22.Final/* + netty-4.1.31.Final/* protobuf-3.0.0/LICENSE jline-2.11/LICENSE protobuf-3.5.1/LICENSE @@ -89,20 +89,6 @@ ${artifact.groupId}-${artifact.artifactId}-${artifact.version}${dashClassifier?}.${artifact.extension} com.google.code.findbugs:jsr305 - - io.netty:netty-buffer - io.netty:netty-codec - io.netty:netty-codec-dns - io.netty:netty-codec-http - io.netty:netty-codec-http2 - io.netty:netty-codec-socks - io.netty:netty-common - io.netty:netty-handler - io.netty:netty-handler-proxy - io.netty:netty-resolver - io.netty:netty-resolver-dns - io.netty:netty-tcnative-boringssl-static - io.netty:netty-transport org.apache.bookkeeper:stream-storage-common org.apache.bookkeeper:stream-storage-proto diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index 748ca2497d0..175050d9ad4 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -250,7 +250,22 @@ Apache Software License, Version 2. - lib/io.dropwizard.metrics-metrics-graphite-3.1.0.jar [21] - lib/io.dropwizard.metrics-metrics-jvm-3.1.0.jar [21] - lib/io.netty-netty-3.10.1.Final.jar [22] -- lib/io.netty-netty-all-4.1.22.Final.jar [23] +- lib/io.netty-netty-buffer-4.1.31.Final.jar [23] +- lib/io.netty-netty-codec-4.1.31.Final.jar [23] +- lib/io.netty-netty-codec-dns-4.1.31.Final.jar [23] +- lib/io.netty-netty-codec-http-4.1.31.Final.jar [23] +- lib/io.netty-netty-codec-http2-4.1.31.Final.jar [23] +- lib/io.netty-netty-codec-socks-4.1.31.Final.jar [23] +- lib/io.netty-netty-common-4.1.31.Final.jar [23] +- lib/io.netty-netty-handler-4.1.31.Final.jar [23] +- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar [23] +- lib/io.netty-netty-resolver-4.1.31.Final.jar [23] +- lib/io.netty-netty-resolver-dns-4.1.31.Final.jar [23] +- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar [23] +- lib/io.netty-netty-transport-4.1.31.Final.jar [23] +- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar [23] +- lib/io.netty-netty-transport-native-epoll-4.1.31.Final-linux-x86_64.jar [23] +- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar [23] - lib/io.prometheus-simpleclient-0.0.21.jar [24] - lib/io.prometheus-simpleclient_common-0.0.21.jar [24] - lib/io.prometheus-simpleclient_hotspot-0.0.21.jar [24] @@ -385,71 +400,63 @@ WebSocket and HTTP server: * https://github.com/joewalnes/webbit ------------------------------------------------------------------------------------ -lib/io.netty-netty-all-4.1.22.Final.jar bundles some 3rd party dependencies +lib/io.netty-netty-codec-4.1.31.Final.jar bundles some 3rd party dependencies -lib/io.netty-netty-all-4.1.22.Final.jar contains the extensions to Java Collections Framework which has +lib/io.netty-netty-codec-4.1.31.Final.jar contains the extensions to Java Collections Framework which has been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.jsr166y.txt (Public Domain) + * deps/netty-4.1.31.Final/LICENSE.jsr166y.txt (Public Domain) * HOMEPAGE: * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified version of Robert Harder's Public Domain +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified version of Robert Harder's Public Domain Base64 Encoder and Decoder, which can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.base64.txt (Public Domain) + * deps/netty-4.1.31.Final/LICENSE.base64.txt (Public Domain) * HOMEPAGE: * http://iharder.sourceforge.net/current/java/base64/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'Webbit', an event based -WebSocket and HTTP server, which can be obtained at: - - * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.webbit.txt (BSD License) - * HOMEPAGE: - * https://github.com/joewalnes/webbit - -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'SLF4J', a simple logging +lib/io.netty-netty-common-4.1.31.Final.jar contains a modified portion of 'SLF4J', a simple logging facade for Java, which can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.slf4j.txt (MIT License) + * deps/netty-4.1.31.Final/LICENSE.slf4j.txt (MIT License) * HOMEPAGE: * http://www.slf4j.org/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression and decompression library written by Matthew J. Francis. It can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.jbzip2.txt (MIT License) + * deps/netty-4.1.31.Final/LICENSE.jbzip2.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jbzip2/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct the suffix array and the Burrows-Wheeler transformed string for any input string of a constant-size alphabet written by Yuta Mori. It can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.libdivsufsort.txt (MIT License) + * deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt (MIT License) * HOMEPAGE: * https://github.com/y-256/libdivsufsort -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression and decompression library written by William Kinney. It can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.jfastlz.txt (MIT License) + * deps/netty-4.1.31.Final/LICENSE.jfastlz.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jfastlz/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data interchange format, which can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.protobuf.txt (New BSD License) + * deps/netty-4.1.31.Final/LICENSE.protobuf.txt (New BSD License) * HOMEPAGE: * https://github.com/google/protobuf @@ -554,4 +561,3 @@ license. For details, see deps/jline-2.11/LICENSE Bundled as - lib/jline-jline-2.11.jar - diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index bd023c87518..62b4f19d4d1 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -215,7 +215,21 @@ Apache Software License, Version 2. - lib/commons-io-commons-io-2.4.jar [8] - lib/commons-lang-commons-lang-2.6.jar [9] - lib/commons-logging-commons-logging-1.1.1.jar [10] -- lib/io.netty-netty-all-4.1.22.Final.jar [11] +- lib/io.netty-netty-buffer-4.1.31.Final.jar [11] +- lib/io.netty-netty-codec-4.1.31.Final.jar [11] +- lib/io.netty-netty-codec-dns-4.1.31.Final.jar [11] +- lib/io.netty-netty-codec-http-4.1.31.Final.jar [11] +- lib/io.netty-netty-codec-http2-4.1.31.Final.jar [11] +- lib/io.netty-netty-codec-socks-4.1.31.Final.jar [11] +- lib/io.netty-netty-common-4.1.31.Final.jar [11] +- lib/io.netty-netty-handler-4.1.31.Final.jar [11] +- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar [11] +- lib/io.netty-netty-resolver-4.1.31.Final.jar [11] +- lib/io.netty-netty-resolver-dns-4.1.31.Final.jar [11] +- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar [11] +- lib/io.netty-netty-transport-4.1.31.Final.jar [11] +- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar [11] +- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar [11] - lib/io.prometheus-simpleclient-0.0.21.jar [12] - lib/io.prometheus-simpleclient_common-0.0.21.jar [12] - lib/io.prometheus-simpleclient_hotspot-0.0.21.jar [12] @@ -275,7 +289,7 @@ Apache Software License, Version 2. [8] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-io.git;a=tag;h=603579 [9] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=tag;h=375459 [10] Source available at http://svn.apache.org/viewvc/commons/proper/logging/tags/commons-logging-1.1.1/ -[11] Source available at https://github.com/netty/netty/tree/netty-4.1.22.Final +[11] Source available at https://github.com/netty/netty/tree/netty-4.1.31.Final [12] Source available at https://github.com/prometheus/client_java/tree/parent-0.0.21 [13] Source available at https://github.com/vert-x3/vertx-auth/tree/3.4.1 [14] Source available at https://github.com/eclipse/vert.x/tree/3.4.1 @@ -303,71 +317,63 @@ Apache Software License, Version 2. [36] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 ------------------------------------------------------------------------------------ -lib/io.netty-netty-all-4.1.22.Final.jar bundles some 3rd party dependencies +lib/io.netty-netty-codec-4.1.31.Final.jar bundles some 3rd party dependencies -lib/io.netty-netty-all-4.1.22.Final.jar contains the extensions to Java Collections Framework which has +lib/io.netty-netty-codec-4.1.31.Final.jar contains the extensions to Java Collections Framework which has been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.jsr166y.txt (Public Domain) + * deps/netty-4.1.31.Final/LICENSE.jsr166y.txt (Public Domain) * HOMEPAGE: * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified version of Robert Harder's Public Domain +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified version of Robert Harder's Public Domain Base64 Encoder and Decoder, which can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.base64.txt (Public Domain) + * deps/netty-4.1.31.Final/LICENSE.base64.txt (Public Domain) * HOMEPAGE: * http://iharder.sourceforge.net/current/java/base64/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'Webbit', an event based -WebSocket and HTTP server, which can be obtained at: - - * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.webbit.txt (BSD License) - * HOMEPAGE: - * https://github.com/joewalnes/webbit - -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'SLF4J', a simple logging +lib/io.netty-netty-common-4.1.31.Final.jar contains a modified portion of 'SLF4J', a simple logging facade for Java, which can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.slf4j.txt (MIT License) + * deps/netty-4.1.31.Final/LICENSE.slf4j.txt (MIT License) * HOMEPAGE: * http://www.slf4j.org/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression and decompression library written by Matthew J. Francis. It can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.jbzip2.txt (MIT License) + * deps/netty-4.1.31.Final/LICENSE.jbzip2.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jbzip2/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct the suffix array and the Burrows-Wheeler transformed string for any input string of a constant-size alphabet written by Yuta Mori. It can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.libdivsufsort.txt (MIT License) + * deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt (MIT License) * HOMEPAGE: * https://github.com/y-256/libdivsufsort -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression and decompression library written by William Kinney. It can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.jfastlz.txt (MIT License) + * deps/netty-4.1.31.Final/LICENSE.jfastlz.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jfastlz/ -lib/io.netty-netty-all-4.1.22.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data interchange format, which can be obtained at: * LICENSE: - * deps/netty-4.1.22.Final/LICENSE.protobuf.txt (New BSD License) + * deps/netty-4.1.31.Final/LICENSE.protobuf.txt (New BSD License) * HOMEPAGE: * https://github.com/google/protobuf diff --git a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt index dea8a3f9ae2..c9a025560c4 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt @@ -45,7 +45,23 @@ License for the specific language governing permissions and limitations under the License. ------------------------------------------------------------------------------------ -- lib/io.netty-netty-all-4.1.22.Final.jar +- lib/io.netty-netty-buffer-4.1.31.Final.jar +- lib/io.netty-netty-codec-4.1.31.Final.jar +- lib/io.netty-netty-codec-dns-4.1.31.Final.jar +- lib/io.netty-netty-codec-http-4.1.31.Final.jar +- lib/io.netty-netty-codec-http2-4.1.31.Final.jar +- lib/io.netty-netty-codec-socks-4.1.31.Final.jar +- lib/io.netty-netty-common-4.1.31.Final.jar +- lib/io.netty-netty-handler-4.1.31.Final.jar +- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar +- lib/io.netty-netty-resolver-4.1.31.Final.jar +- lib/io.netty-netty-resolver-dns-4.1.31.Final.jar +- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar +- lib/io.netty-netty-transport-4.1.31.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.31.Final-linux-x86_64.jar +- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar + The Netty Project ================= diff --git a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt index d2f4f403f99..c2dcf1d75d2 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt @@ -5,7 +5,22 @@ This product includes software developed at The Apache Software Foundation (http://www.apache.org/). ------------------------------------------------------------------------------------ -- lib/io.netty-netty-all-4.1.22.Final.jar +- lib/io.netty-netty-buffer-4.1.31.Final.jar +- lib/io.netty-netty-codec-4.1.31.Final.jar +- lib/io.netty-netty-codec-dns-4.1.31.Final.jar +- lib/io.netty-netty-codec-http-4.1.31.Final.jar +- lib/io.netty-netty-codec-http2-4.1.31.Final.jar +- lib/io.netty-netty-codec-socks-4.1.31.Final.jar +- lib/io.netty-netty-common-4.1.31.Final.jar +- lib/io.netty-netty-handler-4.1.31.Final.jar +- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar +- lib/io.netty-netty-resolver-4.1.31.Final.jar +- lib/io.netty-netty-resolver-dns-4.1.31.Final.jar +- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar +- lib/io.netty-netty-transport-4.1.31.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar +- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar + The Netty Project ================= diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.webbit.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.webbit.txt deleted file mode 100644 index 05ae225fa31..00000000000 --- a/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.webbit.txt +++ /dev/null @@ -1,37 +0,0 @@ -(BSD License: http://www.opensource.org/licenses/bsd-license) - -Copyright (c) 2011, Joe Walnes, Aslak Hellesøy and contributors -All rights reserved. - -Redistribution and use in source and binary forms, with or -without modification, are permitted provided that the -following conditions are met: - -* Redistributions of source code must retain the above - copyright notice, this list of conditions and the - following disclaimer. - -* Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the - following disclaimer in the documentation and/or other - materials provided with the distribution. - -* Neither the name of the Webbit nor the names of - its contributors may be used to endorse or promote products - derived from this software without specific prior written - permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND -CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, -INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF -MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR -CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, -INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE -GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR -BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT -OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE -POSSIBILITY OF SUCH DAMAGE. diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.base64.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.base64.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.base64.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.base64.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.jbzip2.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jbzip2.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.jbzip2.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jbzip2.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.jfastlz.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jfastlz.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.jfastlz.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jfastlz.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.jsr166y.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jsr166y.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.jsr166y.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jsr166y.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.libdivsufsort.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.libdivsufsort.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.protobuf.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.protobuf.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.protobuf.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.protobuf.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.slf4j.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.slf4j.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.22.Final/LICENSE.slf4j.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.slf4j.txt diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index e093d8051b0..10b5ea41486 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -29,13 +29,6 @@ org.apache.bookkeeper bookkeeper-common ${project.parent.version} - - - - io.netty - netty-common - - org.apache.bookkeeper @@ -61,7 +54,11 @@ io.netty - netty-all + netty-handler + + + io.netty + netty-transport-native-epoll io.netty @@ -76,13 +73,6 @@ org.apache.bookkeeper circe-checksum ${project.version} - - - - io.netty - netty-buffer - - commons-cli diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml b/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml index 92b35c61ba7..9f212e52408 100644 --- a/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml +++ b/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml @@ -50,7 +50,7 @@ io.netty - netty-all + netty-common diff --git a/microbenchmarks/pom.xml b/microbenchmarks/pom.xml index a0841bb5419..0f34b319d16 100644 --- a/microbenchmarks/pom.xml +++ b/microbenchmarks/pom.xml @@ -38,10 +38,6 @@ jmh-generator-annprocess provided - - io.netty - netty-all - org.slf4j slf4j-api diff --git a/pom.xml b/pom.xml index 8d43ecda2e6..25624b24040 100644 --- a/pom.xml +++ b/pom.xml @@ -147,8 +147,8 @@ 1.16.22 1.3.0 2.22.0 - 4.1.22.Final - 2.0.7.Final + 4.1.31.Final + 2.0.19.Final 9.1.3 2.0.0-beta.5 0.0.21 @@ -371,7 +371,58 @@ io.netty - netty-all + netty-transport + ${netty.version} + + + io.netty + netty-handler + ${netty.version} + + + io.netty + netty-transport-native-epoll + ${netty.version} + + + io.netty + netty-transport-native-epoll + ${netty.version} + linux-x86_64 + + + io.netty + netty-codec-dns + ${netty.version} + + + io.netty + netty-codec-http + ${netty.version} + + + io.netty + netty-codec-http2 + ${netty.version} + + + io.netty + netty-codec-socks + ${netty.version} + + + io.netty + netty-handler-proxy + ${netty.version} + + + io.netty + netty-resolver + ${netty.version} + + + io.netty + netty-resolver-dns ${netty.version} diff --git a/stream/distributedlog/core/pom.xml b/stream/distributedlog/core/pom.xml index 5617f2c4c1e..582197585f3 100644 --- a/stream/distributedlog/core/pom.xml +++ b/stream/distributedlog/core/pom.xml @@ -29,17 +29,6 @@ org.apache.distributedlog distributedlog-protocol ${project.parent.version} - - - - io.netty - netty-buffer - - - io.netty - netty-common - - org.apache.zookeeper From b7ef2ceb1e9b8b72f5f3c44fa10dd848f66468cb Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 8 Nov 2018 08:49:32 -0800 Subject: [PATCH 0106/1642] Cache InetSocketAddress if hostname is IPAddress Descriptions of the changes in this PR: - in BookieSocketAddress if IPAddress is hostname then it is okay to cache InetSocketAddress, since the canonicalhostname of the node dont change. Reviewers: Sijie Guo , Enrico Olivelli , Matteo Merli , Andrey Yegorov This closes #1789 from reddycharan/bookieaddressonlywhenhostname --- .../bookkeeper/net/BookieSocketAddress.java | 37 +++++++++++--- .../net/BookieSocketAddressTest.java | 50 +++++++++++++++++++ 2 files changed, 81 insertions(+), 6 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieSocketAddressTest.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java index 6d562e3a671..cd0a20cfa4d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java @@ -22,10 +22,12 @@ import static org.apache.bookkeeper.util.BookKeeperConstants.COLON; +import com.google.common.net.InetAddresses; import io.netty.channel.local.LocalAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; +import java.util.Optional; /** * This is a data wrapper class that is an InetSocketAddress, it would use the hostname @@ -38,11 +40,24 @@ public class BookieSocketAddress { // Member fields that make up this class. private final String hostname; private final int port; + private final Optional socketAddress; // Constructor that takes in both a port. public BookieSocketAddress(String hostname, int port) { this.hostname = hostname; this.port = port; + /* + * if ipaddress is used for bookieid then lets cache InetSocketAddress + * otherwise not cache it. If Hostname is used for bookieid, then it is + * ok for node to change its ipaddress. But if ipaddress is used for + * bookieid then it is invalid scenario if node's ipaddress changes and + * nodes HostName is considered static. + */ + if (InetAddresses.isInetAddress(hostname)) { + socketAddress = Optional.of(new InetSocketAddress(hostname, port)); + } else { + socketAddress = Optional.empty(); + } } // Constructor from a String "serialized" version of this class. @@ -57,8 +72,15 @@ public BookieSocketAddress(String addr) throws UnknownHostException { } catch (NumberFormatException nfe) { throw new UnknownHostException(addr); } + if (InetAddresses.isInetAddress(hostname)) { + socketAddress = Optional.of(new InetSocketAddress(hostname, port)); + } else { + socketAddress = Optional.empty(); + } } + + // Public getters public String getHostName() { return hostname; @@ -70,12 +92,15 @@ public int getPort() { // Method to return an InetSocketAddress for the regular port. public InetSocketAddress getSocketAddress() { - // Return each time a new instance of the InetSocketAddress because the hostname - // gets resolved in its constructor and then cached forever. - // If we keep using the same InetSocketAddress instance, if bookies are advertising - // hostnames and the IP change, the BK client will keep forever to try to connect - // to the old IP. - return new InetSocketAddress(hostname, port); + /* + * Return each time a new instance of the InetSocketAddress if hostname + * is used as bookieid. If we keep using the same InetSocketAddress + * instance, if bookies are advertising hostnames and the IP change, the + * BK client will keep forever to try to connect to the old IP. + */ + return socketAddress.orElseGet(() -> { + return new InetSocketAddress(hostname, port); + }); } /** diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieSocketAddressTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieSocketAddressTest.java new file mode 100644 index 00000000000..1da5643d998 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieSocketAddressTest.java @@ -0,0 +1,50 @@ +/** + * + * 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.net; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.net.InetSocketAddress; +import org.junit.Test; + +/** + * Tests for BookieSocketAddress getSocketAddress cache logic. + */ + +public class BookieSocketAddressTest { + + @Test + public void testHostnameBookieId() throws Exception { + BookieSocketAddress hostnameAddress = new BookieSocketAddress("localhost", 3181); + InetSocketAddress inetSocketAddress1 = hostnameAddress.getSocketAddress(); + InetSocketAddress inetSocketAddress2 = hostnameAddress.getSocketAddress(); + assertFalse("InetSocketAddress should be recreated", inetSocketAddress1 == inetSocketAddress2); + } + + @Test + public void testIPAddressBookieId() throws Exception { + BookieSocketAddress ipAddress = new BookieSocketAddress("127.0.0.1", 3181); + InetSocketAddress inetSocketAddress1 = ipAddress.getSocketAddress(); + InetSocketAddress inetSocketAddress2 = ipAddress.getSocketAddress(); + assertTrue("InetSocketAddress should be cached", inetSocketAddress1 == inetSocketAddress2); + } +} From af3a22a39380f7c4a8c3593b26b1293ce377bd67 Mon Sep 17 00:00:00 2001 From: Nicolas Michael Date: Thu, 8 Nov 2018 08:51:10 -0800 Subject: [PATCH 0107/1642] Issue #1791: Read Submission should bypass OSE Threads ### Motivation Profiling of our Bookkeeper Client code for read requests shows that client threads spend half of their time in dispatching requests to OrderedExecutors (just the dispatch itself, not the execution inside OSE): 54% of their CPU time is spent in OrderedExecutor.executeOrdered() (called by LedgerHandle.readEntriesInternalAsync()). The high time spend in request submission to OSE is largely caused by Linux scheduling cost, that is the cost of dispatching the OSE thread to CPU: 42% of total time (3/4th of executeOrdered() time), threads spend in Unsafe.unpark(), which is essentially Linux scheduling/dispatching of another thread. ### Changes This change executes read submissions (PendingReadOp) on read-only ledger handles directly inside the client thread instead of submitting them to Ordered Executors. Tests with a prototype have shown significant improvements in both overall CPU consumption as well as read latency. The additional work client threads have to do (the dispatch of the read requests to netty) is roughly the same as the (saved) dispatch cost to OSE, so the change turns out to be neutral for CPU consumption of client threads. In some experiments, the savings even exceed the additional work, and client threads consume less cpu even though they "do more". It also frees up lots of resources in OSE threads. Since it eliminates one context-switch in read submission and also avoids serialization of reads to the same ledger (or ledgers hashing to the same OSE), it also reduces read latency. For a mixed read-write workload (14,000 reads/sec on read-only ledgers, 4,000 writes/sec on another set of ledgers), this change has reduced CPU consumption of OSE threads by 25%, kept CPU consumption of client (and Netty) threads the same, and yielded a 6% improvement of read latency (as measured by BK Client). Master Issue: #1791: Read Submission should bypass OSE Threads Reviewers: Enrico Olivelli , Andrey Yegorov , Sijie Guo , Matteo Merli This closes #1792 from nicmichael/DirectRead, closes #1791 --- .../java/org/apache/bookkeeper/client/LedgerHandle.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) 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 6125a3d1b92..12e00e84fb7 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 @@ -840,7 +840,14 @@ CompletableFuture readEntriesInternalAsync(long firstEntry, ws.recycle(); } - clientCtx.getMainWorkerPool().executeOrdered(ledgerId, op); + if (isHandleWritable()) { + // Ledger handle in read/write mode: submit to OSE for ordered execution. + clientCtx.getMainWorkerPool().executeOrdered(ledgerId, op); + } else { + // Read-only ledger handle: bypass OSE and execute read directly in client thread. + // This avoids a context-switch to OSE thread and thus reduces latency. + op.run(); + } } else { op.future().completeExceptionally(BKException.create(ClientClosedException)); } From 8c74e857fa26d6d64dc7a66a6d848634dc428f46 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 8 Nov 2018 21:47:12 -0800 Subject: [PATCH 0108/1642] [TOOLS] add cookie related commands Descriptions of the changes in this PR: *Motivation* In some use cases, you need cookie related tools to create/delete/update/get cookie when handling production issues. Currently bookkeeper doesn't provide such commands. *Changes* Add cookie related commands - create - delete - get - update - generate Reviewers: Enrico Olivelli , Jia Zhai , Matteo Merli This closes #1794 from sijie/add_cookie_commands --- .../bookkeeper/bookie/BookieException.java | 23 ++ .../apache/bookkeeper/bookie/BookieShell.java | 44 +++- .../org/apache/bookkeeper/bookie/Cookie.java | 7 +- .../discover/ZKRegistrationManager.java | 5 + .../cli/commands/cookie/CookieCommand.java | 124 +++++++++++ .../commands/cookie/CreateCookieCommand.java | 102 +++++++++ .../commands/cookie/DeleteCookieCommand.java | 91 ++++++++ .../cookie/GenerateCookieCommand.java | 126 +++++++++++ .../cli/commands/cookie/GetCookieCommand.java | 101 +++++++++ .../commands/cookie/UpdateCookieCommand.java | 102 +++++++++ .../cli/commands/cookie/package-info.java | 23 ++ .../tools/cli/helpers/BookieShellCommand.java | 62 ++++++ .../tools/cli/helpers/ClientCommand.java | 2 +- .../cli/commands/CookieCommandGroup.java | 54 +++++ ...he.bookkeeper.tools.framework.CommandGroup | 1 + .../cookie/CreateCookieCommandTest.java | 180 ++++++++++++++++ .../cookie/DeleteCookieCommandTest.java | 135 ++++++++++++ .../cookie/GenerateCookieCommandTest.java | 198 ++++++++++++++++++ .../commands/cookie/GetCookieCommandTest.java | 146 +++++++++++++ .../cookie/UpdateCookieCommandTest.java | 180 ++++++++++++++++ .../cli/helpers/BookieShellCommandTest.java | 65 ++++++ .../cli/helpers/CookieCommandTestBase.java | 91 ++++++++ .../bookkeeper/tools/common/BKCommand.java | 14 ++ 23 files changed, 1869 insertions(+), 7 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CookieCommand.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommand.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommand.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommand.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommand.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/package-info.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommand.java create mode 100644 tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java create mode 100644 tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java create mode 100644 tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java create mode 100644 tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java create mode 100644 tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java create mode 100644 tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java create mode 100644 tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommandTest.java create mode 100644 tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CookieCommandTestBase.java 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 3d84148d1cf..83002ce252b 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 @@ -63,6 +63,8 @@ public static BookieException create(int code) { return new DiskPartitionDuplicationException(); case Code.CookieNotFoundException: return new CookieNotFoundException(); + case Code.CookieExistsException: + return new CookieExistException(); case Code.MetadataStoreException: return new MetadataStoreException(); case Code.UnknownBookieIdException: @@ -88,6 +90,7 @@ public interface Code { int MetadataStoreException = -106; int UnknownBookieIdException = -107; int OperationRejectedException = -108; + int CookieExistsException = -109; } public int getCode() { @@ -118,6 +121,9 @@ public String getMessage(int code) { case Code.CookieNotFoundException: err = "Cookie not found"; break; + case Code.CookieExistsException: + err = "Cookie already exists"; + break; case Code.MetadataStoreException: err = "Error performing metadata operations"; break; @@ -231,6 +237,23 @@ public CookieNotFoundException(Throwable cause) { } } + /** + * Signal that cookie already exists when creating a new cookie. + */ + public static class CookieExistException extends BookieException { + public CookieExistException() { + this(""); + } + + public CookieExistException(String reason) { + super(Code.CookieExistsException, reason); + } + + public CookieExistException(Throwable cause) { + super(Code.CookieExistsException, cause); + } + } + /** * Signals that an exception occurs on upgrading a bookie. */ 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 18ce8c43924..e0e79c2de1f 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 @@ -92,6 +92,7 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.client.UpdateLedgerOp; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -115,6 +116,11 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.DeleteCookieCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.GenerateCookieCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.GetCookieCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.UpdateCookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.DiskChecker; @@ -191,6 +197,14 @@ public class BookieShell implements Tool { static final String CMD_CONVERT_TO_INTERLEAVED_STORAGE = "convert-to-interleaved-storage"; static final String CMD_REBUILD_DB_LEDGER_LOCATIONS_INDEX = "rebuild-db-ledger-locations-index"; static final String CMD_REGENERATE_INTERLEAVED_STORAGE_INDEX_FILE = "regenerate-interleaved-storage-index-file"; + + // cookie commands + static final String CMD_CREATE_COOKIE = "cookie_create"; + static final String CMD_DELETE_COOKIE = "cookie_delete"; + static final String CMD_UPDATE_COOKIE = "cookie_update"; + static final String CMD_GET_COOKIE = "cookie_get"; + static final String CMD_GENERATE_COOKIE = "cookie_generate"; + static final String CMD_HELP = "help"; final ServerConfiguration bkConf = new ServerConfiguration(); @@ -214,9 +228,15 @@ public BookieShell(LedgerIdFormatter ledgeridFormatter, EntryFormatter entryForm this.entryFormatter = entryFormatter; } - interface Command { + /** + * BookieShell command. + */ + @Private + public interface Command { int runCmd(String[] args) throws Exception; + String description(); + void printUsage(); } @@ -235,6 +255,11 @@ abstract class MyCommand implements Command { this.cmdName = cmdName; } + public String description() { + // we used the string returned by `getUsage` as description in showing the list of commands + return getUsage(); + } + @Override public int runCmd(String[] args) throws Exception { try { @@ -2881,7 +2906,7 @@ int runCmd(CommandLine cmdLine) throws Exception { } } - final Map commands = new HashMap(); + final Map commands = new HashMap<>(); { commands.put(CMD_METAFORMAT, new MetaFormatCmd()); @@ -2918,6 +2943,17 @@ int runCmd(CommandLine cmdLine) throws Exception { commands.put(CMD_HELP, new HelpCmd()); commands.put(CMD_LOSTBOOKIERECOVERYDELAY, new LostBookieRecoveryDelayCmd()); commands.put(CMD_TRIGGERAUDIT, new TriggerAuditCmd()); + // cookie related commands + commands.put(CMD_CREATE_COOKIE, + new CreateCookieCommand().asShellCommand(CMD_CREATE_COOKIE, bkConf)); + commands.put(CMD_DELETE_COOKIE, + new DeleteCookieCommand().asShellCommand(CMD_DELETE_COOKIE, bkConf)); + commands.put(CMD_UPDATE_COOKIE, + new UpdateCookieCommand().asShellCommand(CMD_UPDATE_COOKIE, bkConf)); + commands.put(CMD_GET_COOKIE, + new GetCookieCommand().asShellCommand(CMD_GET_COOKIE, bkConf)); + commands.put(CMD_GENERATE_COOKIE, + new GenerateCookieCommand().asShellCommand(CMD_GENERATE_COOKIE, bkConf)); } @Override @@ -2939,8 +2975,8 @@ private void printShellUsage() { + "[-entryformat ] [-conf configuration] "); System.err.println("where command is one of:"); List commandNames = new ArrayList(); - for (MyCommand c : commands.values()) { - commandNames.add(" " + c.getUsage()); + for (Command c : commands.values()) { + commandNames.add(" " + c.description()); } Collections.sort(commandNames); for (String s : commandNames) { 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 4a9f6f6c79e..7d4175cd6f6 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 @@ -212,8 +212,11 @@ private static Builder parse(BufferedReader reader) throws IOException { public void writeToDirectory(File directory) throws IOException { File versionFile = new File(directory, - BookKeeperConstants.VERSION_FILENAME); + BookKeeperConstants.VERSION_FILENAME); + writeToFile(versionFile); + } + public void writeToFile (File versionFile) throws IOException { FileOutputStream fos = new FileOutputStream(versionFile); BufferedWriter bw = null; try { @@ -386,7 +389,7 @@ public boolean isBookieHostCreatedFromIp() throws IOException { * Cookie builder. */ public static class Builder { - private int layoutVersion = 0; + private int layoutVersion = CURRENT_COOKIE_LAYOUT_VERSION; private String bookieHost = null; private String journalDirs = null; private String ledgerDirs = null; 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 index eab9e4f1864..a2a3e7c62a8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java @@ -37,6 +37,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.BookieException.BookieIllegalOpException; +import org.apache.bookkeeper.bookie.BookieException.CookieExistException; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; import org.apache.bookkeeper.client.BKException; @@ -327,6 +328,10 @@ public void writeCookie(String bookieId, } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new MetadataStoreException("Interrupted writing cookie for bookie " + bookieId, ie); + } catch (NoNodeException nne) { + throw new CookieNotFoundException(bookieId); + } catch (NodeExistsException nee) { + throw new CookieExistException(bookieId); } catch (KeeperException e) { throw new MetadataStoreException("Failed to write cookie for bookie " + bookieId); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CookieCommand.java new file mode 100644 index 00000000000..fe2c5318df0 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CookieCommand.java @@ -0,0 +1,124 @@ +/* + * 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.tools.cli.commands.cookie; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.net.UnknownHostException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Paths; +import java.util.concurrent.ExecutionException; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieShellCommand; +import org.apache.bookkeeper.tools.common.BKCommand; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.commons.configuration.CompositeConfiguration; + +/** + * This is a mixin for cookie related commands to extends. + */ +@Slf4j +abstract class CookieCommand + extends BKCommand { + + protected CookieCommand(CliSpec spec) { + super(spec); + } + + @Override + protected boolean apply(ServiceURI serviceURI, + CompositeConfiguration conf, + BKFlags globalFlags, + CookieFlagsT cmdFlags) { + ServerConfiguration serverConf = new ServerConfiguration(); + serverConf.loadConf(conf); + + if (null != serviceURI) { + serverConf.setMetadataServiceUri(serviceURI.getUri().toString()); + } + + try { + return MetadataDrivers.runFunctionWithRegistrationManager(serverConf, registrationManager -> { + try { + apply(registrationManager, cmdFlags); + return true; + } catch (Exception e) { + throw new UncheckedExecutionException(e); + } + }); + } catch (MetadataException | ExecutionException | UncheckedExecutionException e) { + Throwable cause = e; + if (!(e instanceof MetadataException) && null != e.getCause()) { + cause = e.getCause(); + } + spec.console().println("Failed to process cookie command '" + name() + "'"); + cause.printStackTrace(spec.console()); + return false; + } + } + + protected String getBookieId(CookieFlagsT cmdFlags) throws UnknownHostException { + checkArgument( + cmdFlags.arguments.size() == 1, + "No bookie id or more bookie ids is specified"); + + String bookieId = cmdFlags.arguments.get(0); + try { + new BookieSocketAddress(bookieId); + } catch (UnknownHostException nhe) { + spec.console() + .println("Invalid bookie id '" + + bookieId + "'is used to create cookie." + + " Bookie id should be in the format of ':'"); + throw nhe; + } + return bookieId; + } + + protected byte[] readCookieDataFromFile(String cookieFile) throws IOException { + try { + return Files.readAllBytes(Paths.get(cookieFile)); + } catch (NoSuchFileException nfe) { + spec.console() + .println("Cookie file '" + cookieFile + "' doesn't exist."); + throw nfe; + } + } + + + protected abstract void apply(RegistrationManager rm, CookieFlagsT cmdFlags) + throws Exception; + + public org.apache.bookkeeper.bookie.BookieShell.Command asShellCommand(String shellCmdName, + CompositeConfiguration conf) { + return new BookieShellCommand<>(shellCmdName, this, conf); + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommand.java new file mode 100644 index 00000000000..430fed9dce8 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommand.java @@ -0,0 +1,102 @@ +/* + * 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.tools.cli.commands.cookie; + +import com.beust.jcommander.Parameter; +import java.io.PrintStream; +import lombok.Setter; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.BookieException.CookieExistException; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand.Flags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * A command that create cookie. + */ +@Slf4j +public class CreateCookieCommand extends CookieCommand { + + private static final String NAME = "create"; + private static final String DESC = "Create a cookie for a given bookie"; + + /** + * Flags to create a cookie for a given bookie. + */ + @Accessors(fluent = true) + @Setter + public static class Flags extends CliFlags { + + @Parameter( + names = { "-cf", "--cookie-file" }, + description = "The file to be uploaded as cookie", + required = true) + private String cookieFile; + + } + + public CreateCookieCommand() { + this(new Flags()); + } + + protected CreateCookieCommand(PrintStream console) { + this(new Flags(), console); + } + + public CreateCookieCommand(Flags flags) { + this(flags, System.out); + } + + private CreateCookieCommand(Flags flags, PrintStream console) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .withConsole(console) + .withArgumentsUsage("") + .build()); + } + + @Override + protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { + String bookieId = getBookieId(cmdFlags); + + byte[] data = readCookieDataFromFile(cmdFlags.cookieFile); + Versioned cookie = new Versioned<>(data, Version.NEW); + try { + rm.writeCookie(bookieId, cookie); + } catch (CookieExistException cee) { + spec.console() + .println("Cookie already exist for bookie '" + bookieId + "'"); + throw cee; + } catch (BookieException be) { + spec.console() + .println("Exception on creating cookie for bookie '" + bookieId + "'"); + be.printStackTrace(spec.console()); + throw be; + } + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommand.java new file mode 100644 index 00000000000..4c42615e8b6 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommand.java @@ -0,0 +1,91 @@ +/* + * 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.tools.cli.commands.cookie; + +import java.io.PrintStream; +import lombok.Setter; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.tools.cli.commands.cookie.DeleteCookieCommand.Flags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.versioning.LongVersion; + +/** + * A command that deletes cookie. + */ +@Slf4j +public class DeleteCookieCommand extends CookieCommand { + + private static final String NAME = "delete"; + private static final String DESC = "Delete a cookie for a given bookie"; + + /** + * Flags to delete a cookie for a given bookie. + */ + @Accessors(fluent = true) + @Setter + public static class Flags extends CliFlags { + } + + public DeleteCookieCommand() { + this(new Flags()); + } + + DeleteCookieCommand(PrintStream console) { + this(new Flags(), console); + } + + public DeleteCookieCommand(Flags flags) { + this(flags, System.out); + } + + private DeleteCookieCommand(Flags flags, PrintStream console) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .withConsole(console) + .withArgumentsUsage("") + .build()); + } + + @Override + protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { + String bookieId = getBookieId(cmdFlags); + + try { + rm.removeCookie(bookieId, new LongVersion(-1)); + } catch (CookieNotFoundException cee) { + spec.console() + .println("Cookie not found for bookie '" + bookieId + "'"); + throw cee; + } catch (BookieException be) { + spec.console() + .println("Exception on deleting cookie for bookie '" + bookieId + "'"); + be.printStackTrace(spec.console()); + throw be; + } + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java new file mode 100644 index 00000000000..daa3c435aad --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java @@ -0,0 +1,126 @@ +/* + * 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.tools.cli.commands.cookie; + +import com.beust.jcommander.Parameter; +import java.io.File; +import java.io.PrintStream; +import lombok.Setter; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.bookie.Cookie.Builder; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.tools.cli.commands.cookie.GenerateCookieCommand.Flags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.commons.lang3.StringUtils; + +/** + * A command that generate cookie. + */ +@Slf4j +public class GenerateCookieCommand extends CookieCommand { + + private static final String NAME = "generate"; + private static final String DESC = "Generate a cookie for a given bookie"; + + /** + * Flags to generate a cookie for a given bookie. + */ + @Accessors(fluent = true) + @Setter + public static class Flags extends CliFlags { + + @Parameter( + names = { "-j", "--journal-dirs" }, + description = "The journal directories used by this bookie", + required = true) + private String journalDirs; + + @Parameter( + names = { "-l", "--ledger-dirs" }, + description = "The ledger directories used by this bookie", + required = true) + private String ledgerDirs; + + @Parameter( + names = { "-i", "--instance-id" }, + description = "The instance id of the cluster that this bookie belongs to." + + " If omitted, it will used the instance id of the cluster that this cli connects to.") + private String instanceId = null; + + @Parameter( + names = { "-o", "--output-file" }, + description = "The output file to save the generated cookie.", + required = true) + private String outputFile; + + } + + public GenerateCookieCommand() { + this(new Flags()); + } + + GenerateCookieCommand(PrintStream console) { + this(new Flags(), console); + } + + public GenerateCookieCommand(Flags flags) { + this(flags, System.out); + } + + private GenerateCookieCommand(Flags flags, PrintStream console) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .withConsole(console) + .withArgumentsUsage("") + .build()); + } + + @Override + protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { + String bookieId = getBookieId(cmdFlags); + + String instanceId; + if (null == cmdFlags.instanceId) { + instanceId = rm.getClusterInstanceId(); + } else { + instanceId = cmdFlags.instanceId; + } + + Builder builder = Cookie.newBuilder(); + builder.setBookieHost(bookieId); + if (StringUtils.isEmpty(instanceId)) { + builder.setInstanceId(null); + } else { + builder.setInstanceId(instanceId); + } + builder.setJournalDirs(cmdFlags.journalDirs); + builder.setLedgerDirs(cmdFlags.ledgerDirs); + + Cookie cookie = builder.build(); + cookie.writeToFile(new File(cmdFlags.outputFile)); + spec.console().println("Successfully saved the generated cookie to " + cmdFlags.outputFile); + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommand.java new file mode 100644 index 00000000000..76f5f7c0099 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommand.java @@ -0,0 +1,101 @@ +/* + * 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.tools.cli.commands.cookie; + +import java.io.PrintStream; +import lombok.Setter; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.commands.cookie.GetCookieCommand.Flags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * A command that deletes cookie. + */ +@Slf4j +public class GetCookieCommand extends CookieCommand { + + private static final String NAME = "get"; + private static final String DESC = "Retrieve a cookie for a given bookie"; + + /** + * Flags to delete a cookie for a given bookie. + */ + @Accessors(fluent = true) + @Setter + public static class Flags extends CliFlags { + } + + public GetCookieCommand() { + this(new Flags()); + } + + GetCookieCommand(PrintStream console) { + this(new Flags(), console); + } + + public GetCookieCommand(Flags flags) { + this(flags, System.out); + } + + private GetCookieCommand(Flags flags, PrintStream console) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .withConsole(console) + .withArgumentsUsage("") + .build()); + } + + @Override + protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { + String bookieId = getBookieId(cmdFlags); + + try { + Versioned cookie = Cookie.readFromRegistrationManager( + rm, new BookieSocketAddress(bookieId) + ); + spec.console().println("Cookie for bookie '" + bookieId + "' is:"); + spec.console().println("---"); + spec.console().println( + cookie.getValue() + ); + spec.console().println("---"); + } catch (CookieNotFoundException cee) { + spec.console() + .println("Cookie not found for bookie '" + bookieId + "'"); + throw cee; + } catch (BookieException be) { + spec.console() + .println("Exception on getting cookie for bookie '" + bookieId + "'"); + be.printStackTrace(spec.console()); + throw be; + } + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommand.java new file mode 100644 index 00000000000..77e5f05857b --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommand.java @@ -0,0 +1,102 @@ +/* + * 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.tools.cli.commands.cookie; + +import com.beust.jcommander.Parameter; +import java.io.PrintStream; +import lombok.Setter; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.tools.cli.commands.cookie.UpdateCookieCommand.Flags; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * A command that updates cookie. + */ +@Slf4j +public class UpdateCookieCommand extends CookieCommand { + + private static final String NAME = "update"; + private static final String DESC = "Update a cookie for a given bookie"; + + /** + * Flags to create a cookie for a given bookie. + */ + @Accessors(fluent = true) + @Setter + public static class Flags extends CliFlags { + + @Parameter( + names = { "-cf", "--cookie-file" }, + description = "The file to be uploaded as cookie", + required = true) + private String cookieFile; + + } + + public UpdateCookieCommand() { + this(new Flags()); + } + + UpdateCookieCommand(PrintStream console) { + this(new Flags(), console); + } + + public UpdateCookieCommand(Flags flags) { + this(flags, System.out); + } + + private UpdateCookieCommand(Flags flags, PrintStream console) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .withConsole(console) + .withArgumentsUsage("") + .build()); + } + + @Override + protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { + String bookieId = getBookieId(cmdFlags); + + byte[] data = readCookieDataFromFile(cmdFlags.cookieFile); + Versioned cookie = new Versioned<>(data, new LongVersion(-1L)); + try { + rm.writeCookie(bookieId, cookie); + } catch (CookieNotFoundException cnfe) { + spec.console() + .println("Cookie not found for bookie '" + bookieId + "' to update"); + throw cnfe; + } catch (BookieException be) { + spec.console() + .println("Exception on updating cookie for bookie '" + bookieId + "'"); + be.printStackTrace(spec.console()); + throw be; + } + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/package-info.java new file mode 100644 index 00000000000..248c49ad7a0 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/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. + */ + +/** + * Cookie related cli commands. + */ +package org.apache.bookkeeper.tools.cli.commands.cookie; \ No newline at end of file diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommand.java new file mode 100644 index 00000000000..4010e607eb8 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommand.java @@ -0,0 +1,62 @@ +/* + * 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.tools.cli.helpers; + +import org.apache.bookkeeper.bookie.BookieShell.Command; +import org.apache.bookkeeper.tools.common.BKCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.commons.configuration.CompositeConfiguration; + +/** + * This is a util class that converts new cli command to old shell command. + */ +public class BookieShellCommand implements Command { + + protected final String shellCmdName; + protected final BKCommand bkCmd; + protected final CompositeConfiguration conf; + + public BookieShellCommand(String shellCmdName, + BKCommand bkCmd, + CompositeConfiguration conf) { + this.shellCmdName = shellCmdName; + this.bkCmd = bkCmd; + this.conf = conf; + } + + @Override + public int runCmd(String[] args) throws Exception { + return bkCmd.apply( + shellCmdName, + conf, + args + ); + } + + @Override + public String description() { + return shellCmdName + " [options]"; + } + + @Override + public void printUsage() { + bkCmd.usage(); + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommand.java index ae807dfb657..45cb40e43a3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommand.java @@ -66,7 +66,7 @@ protected boolean apply(ClientConfiguration conf, run(bk, cmdFlags); return true; } catch (Exception e) { - log.error("Faild to process command '{}'", name(), e); + log.error("Failed to process command '{}'", name(), e); return false; } } diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java new file mode 100644 index 00000000000..3dded4e5cb5 --- /dev/null +++ b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java @@ -0,0 +1,54 @@ +/* + * 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.tools.cli.commands; + +import org.apache.bookkeeper.tools.cli.BKCtl; +import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.DeleteCookieCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.GenerateCookieCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.GetCookieCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.UpdateCookieCommand; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliCommandGroup; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Commands that operates cookies. + */ +public class CookieCommandGroup extends CliCommandGroup { + + private static final String NAME = "cookie"; + private static final String DESC = "Commands on operating cookies"; + + private static final CliSpec spec = CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withParent(BKCtl.NAME) + .addCommand(new CreateCookieCommand()) + .addCommand(new DeleteCookieCommand()) + .addCommand(new GetCookieCommand()) + .addCommand(new UpdateCookieCommand()) + .addCommand(new GenerateCookieCommand()) + .build(); + + public CookieCommandGroup() { + super(spec); + } +} diff --git a/tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup b/tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup index 44fc194f59f..0147eca5e6b 100644 --- a/tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup +++ b/tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup @@ -18,6 +18,7 @@ org.apache.bookkeeper.tools.cli.commands.BookieCommandGroup org.apache.bookkeeper.tools.cli.commands.BookiesCommandGroup +org.apache.bookkeeper.tools.cli.commands.CookieCommandGroup org.apache.bookkeeper.tools.cli.commands.LedgerCommandGroup org.apache.bookkeeper.stream.cli.ClusterCommandGroup org.apache.bookkeeper.stream.cli.NamespaceCommandGroup diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java new file mode 100644 index 00000000000..fcfae0be770 --- /dev/null +++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java @@ -0,0 +1,180 @@ +/* + * 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.tools.cli.commands.cookie; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.PrintStream; +import java.nio.file.Files; +import java.nio.file.Paths; +import org.apache.bookkeeper.bookie.BookieException.CookieExistException; +import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.tools.cli.helpers.CookieCommandTestBase; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.versioning.Versioned; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test {@link CreateCookieCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ MetadataDrivers.class }) +public class CreateCookieCommandTest extends CookieCommandTestBase { + + @Rule + public final TemporaryFolder testFolder = new TemporaryFolder(); + + private final ByteArrayOutputStream output = new ByteArrayOutputStream(); + private final PrintStream console = new PrintStream(output); + + private boolean runCommand(String[] args) { + CreateCookieCommand createCmd = new CreateCookieCommand(console); + BKFlags bkFlags = new BKFlags(); + bkFlags.serviceUri = "zk://127.0.0.1"; + return createCmd.apply(bkFlags, args); + } + + private String getConsoleOutput() { + return new String(output.toByteArray(), UTF_8); + } + + /** + * Run a command without providing bookie id. + */ + @Test + public void testMissingBookieId() { + assertFalse(runCommand(new String[] {})); + String consoleOutput = getConsoleOutput(); + assertBookieIdMissing(consoleOutput); + } + + private void assertPrintUsage(String consoleOutput) { + assertPrintUsage(consoleOutput, "create [flags] "); + } + + /** + * Run a command without cookie file. + */ + @Test + public void testMissingCookieFileOption() { + assertFalse(runCommand(new String[] { BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertOptionMissing(consoleOutput, "-cf, --cookie-file"); + assertPrintUsage(consoleOutput); + } + + /** + * Run a command with invalid bookie id. + */ + @Test + public void testInvalidBookieId() { + assertFalse(runCommand(new String[] { "-cf", "test-cookie-file", INVALID_BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertInvalidBookieId(consoleOutput, INVALID_BOOKIE_ID); + } + + /** + * Run a command with a non-existent cookie file. + */ + @Test + public void testCreateCookieFromNonExistentCookieFile() { + String file = "/path/to/non-existent-cookie-file"; + assertFalse(runCommand(new String[] { "-cf", file, BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertCookieFileNotExists(consoleOutput, file); + } + + /** + * A successful run. + */ + @SuppressWarnings("unchecked") + @Test + public void testCreateCookieFromExistentCookieFile() throws Exception { + File file = testFolder.newFile("test-cookie-file"); + byte[] content = "test-create-cookie".getBytes(UTF_8); + Files.write(Paths.get(file.toURI()), content); + String fileName = file.getPath(); + assertTrue(runCommand(new String[] { "-cf", fileName, BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue(consoleOutput, consoleOutput.isEmpty()); + verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + } + + /** + * Run a command to create cookie on an existing cookie. + */ + @SuppressWarnings("unchecked") + @Test + public void testCreateAlreadyExistedCookie() throws Exception { + doThrow(new CookieExistException()) + .when(rm).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + + File file = testFolder.newFile("test-cookie-file"); + byte[] content = "test-create-cookie".getBytes(UTF_8); + Files.write(Paths.get(file.toURI()), content); + String fileName = file.getPath(); + assertFalse(runCommand(new String[] { "-cf", fileName, BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue( + consoleOutput, + consoleOutput.contains("Cookie already exist for bookie '" + BOOKIE_ID + "'")); + verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + } + + /** + * Run a command to create cookie when exception is thrown. + */ + @SuppressWarnings("unchecked") + @Test + public void testCreateCookieException() throws Exception { + doThrow(new OperationRejectedException()) + .when(rm).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + + File file = testFolder.newFile("test-cookie-file"); + byte[] content = "test-create-cookie".getBytes(UTF_8); + Files.write(Paths.get(file.toURI()), content); + String fileName = file.getPath(); + assertFalse(runCommand(new String[] { "-cf", fileName, BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue( + consoleOutput, + consoleOutput.contains("Exception on creating cookie for bookie '" + BOOKIE_ID + "'")); + assertTrue( + consoleOutput, + consoleOutput.contains(OperationRejectedException.class.getName())); + verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + } + +} diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java new file mode 100644 index 00000000000..01ad58d42bb --- /dev/null +++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java @@ -0,0 +1,135 @@ +/* + * 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.tools.cli.commands.cookie; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; +import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.tools.cli.helpers.CookieCommandTestBase; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.versioning.LongVersion; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test {@link DeleteCookieCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ MetadataDrivers.class }) +public class DeleteCookieCommandTest extends CookieCommandTestBase { + + @Rule + public final TemporaryFolder testFolder = new TemporaryFolder(); + + private final ByteArrayOutputStream output = new ByteArrayOutputStream(); + private final PrintStream console = new PrintStream(output); + + private boolean runCommand(String[] args) { + DeleteCookieCommand deleteCmd = new DeleteCookieCommand(console); + BKFlags bkFlags = new BKFlags(); + bkFlags.serviceUri = "zk://127.0.0.1"; + return deleteCmd.apply(bkFlags, args); + } + + private String getConsoleOutput() { + return new String(output.toByteArray(), UTF_8); + } + + /** + * Run a command without providing bookie id. + */ + @Test + public void testMissingBookieId() { + assertFalse(runCommand(new String[] {})); + String consoleOutput = getConsoleOutput(); + assertBookieIdMissing(consoleOutput); + } + + /** + * Run a command with invalid bookie id. + */ + @Test + public void testInvalidBookieId() { + assertFalse(runCommand(new String[] { INVALID_BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertInvalidBookieId(consoleOutput, INVALID_BOOKIE_ID); + } + + /** + * A successful run. + */ + @Test + public void testDeleteCookieFromExistentCookieFile() throws Exception { + assertTrue(runCommand(new String[] { BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue(consoleOutput, consoleOutput.isEmpty()); + verify(rm, times(1)).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + } + + /** + * Run a command to delete cookie on an non-existent cookie. + */ + @Test + public void testDeleteNonExistedCookie() throws Exception { + doThrow(new CookieNotFoundException()) + .when(rm).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + + assertFalse(runCommand(new String[] { BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue( + consoleOutput, + consoleOutput.contains("Cookie not found for bookie '" + BOOKIE_ID + "'")); + verify(rm, times(1)).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + } + + /** + * Run a command to delete cookie when exception is thrown. + */ + @Test + public void testDeleteCookieException() throws Exception { + doThrow(new OperationRejectedException()) + .when(rm).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + + assertFalse(runCommand(new String[] { BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue( + consoleOutput, + consoleOutput.contains("Exception on deleting cookie for bookie '" + BOOKIE_ID + "'")); + assertTrue( + consoleOutput, + consoleOutput.contains(OperationRejectedException.class.getName())); + verify(rm, times(1)).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + } + +} diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java new file mode 100644 index 00000000000..6e2729450c4 --- /dev/null +++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java @@ -0,0 +1,198 @@ +/* + * 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.tools.cli.commands.cookie; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.PrintStream; +import java.nio.file.Files; +import java.nio.file.Paths; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.tools.cli.helpers.CookieCommandTestBase; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test {@link GetCookieCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ MetadataDrivers.class }) +public class GenerateCookieCommandTest extends CookieCommandTestBase { + + @Rule + public final TemporaryFolder testFolder = new TemporaryFolder(); + + private final ByteArrayOutputStream output = new ByteArrayOutputStream(); + private final PrintStream console = new PrintStream(output); + + private boolean runCommand(String[] args) { + GenerateCookieCommand getCmd = new GenerateCookieCommand(console); + BKFlags bkFlags = new BKFlags(); + bkFlags.serviceUri = "zk://127.0.0.1"; + return getCmd.apply(bkFlags, args); + } + + private String getConsoleOutput() { + return new String(output.toByteArray(), UTF_8); + } + + /** + * Run a command without providing bookie id. + */ + @Test + public void testMissingBookieId() { + assertFalse(runCommand(new String[] {})); + String consoleOutput = getConsoleOutput(); + assertBookieIdMissing(consoleOutput); + } + + /** + * Run a command with invalid bookie id. + */ + @Test + public void testInvalidBookieId() { + assertFalse(runCommand(new String[] { + "-j", "/path/to/journal", + "-l", "/path/to/ledgers", + "-o", "/path/to/cookie-file", + INVALID_BOOKIE_ID + })); + String consoleOutput = getConsoleOutput(); + assertInvalidBookieId(consoleOutput, INVALID_BOOKIE_ID); + } + + /** + * Run a command without journal dirs. + */ + @Test + public void testMissingJournalDir() { + assertFalse(runCommand(new String[] { "-l", "/path/to/ledgers", "-o", "/path/to/cookie-file", BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertOptionMissing(consoleOutput, "-j, --journal-dirs"); + } + + /** + * Run a command without ledger dirs. + */ + @Test + public void testMissingLedgerDirs() { + assertFalse(runCommand(new String[] { "-j", "/path/to/journal", "-o", "/path/to/cookie-file", BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertOptionMissing(consoleOutput, "-l, --ledger-dirs"); + } + + /** + * Run a command without output file. + */ + @Test + public void testMissingOutputFile() { + assertFalse(runCommand(new String[] { "-j", "/path/to/journal", "-l", "/path/to/ledgers", BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertOptionMissing(consoleOutput, "-o, --output-file"); + } + + /** + * A successful run without instance id. + */ + @Test + public void testGenerateCookieWithoutInstanceId() throws Exception { + File cookieFile = testFolder.newFile("cookie-without-instance-id"); + String journalDir = "/path/to/journal"; + String ledgersDir = "/path/to/ledgers"; + String instanceId = "test-instance-id"; + + Cookie cookie = Cookie.newBuilder() + .setBookieHost(BOOKIE_ID) + .setInstanceId(instanceId) + .setJournalDirs(journalDir) + .setLedgerDirs(ledgersDir) + .build(); + + when(rm.getClusterInstanceId()).thenReturn(instanceId); + assertTrue( + getConsoleOutput(), + runCommand(new String[] { + "-l", ledgersDir, + "-j", journalDir, + "-o", cookieFile.getPath(), + BOOKIE_ID + })); + String consoleOutput = getConsoleOutput(); + assertTrue(consoleOutput, consoleOutput.contains( + "Successfully saved the generated cookie to " + cookieFile.getPath() + )); + verify(rm, times(1)).getClusterInstanceId(); + + byte[] data = Files.readAllBytes(Paths.get(cookieFile.getPath())); + assertArrayEquals(cookie.toString().getBytes(UTF_8), data); + } + + /** + * A successful run with instance id. + */ + @Test + public void testGenerateCookieWithInstanceId() throws Exception { + File cookieFile = testFolder.newFile("cookie-with-instance-id"); + String journalDir = "/path/to/journal"; + String ledgersDir = "/path/to/ledgers"; + String instanceId = "test-instance-id"; + + Cookie cookie = Cookie.newBuilder() + .setBookieHost(BOOKIE_ID) + .setInstanceId(instanceId) + .setJournalDirs(journalDir) + .setLedgerDirs(ledgersDir) + .build(); + + when(rm.getClusterInstanceId()).thenReturn(instanceId); + assertTrue( + getConsoleOutput(), + runCommand(new String[] { + "-l", ledgersDir, + "-j", journalDir, + "-o", cookieFile.getPath(), + "-i", instanceId, + BOOKIE_ID + })); + String consoleOutput = getConsoleOutput(); + assertTrue(consoleOutput, consoleOutput.contains( + "Successfully saved the generated cookie to " + cookieFile.getPath() + )); + verify(rm, times(0)).getClusterInstanceId(); + + byte[] data = Files.readAllBytes(Paths.get(cookieFile.getPath())); + assertArrayEquals(cookie.toString().getBytes(UTF_8), data); + } + +} diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java new file mode 100644 index 00000000000..ef45a05d359 --- /dev/null +++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java @@ -0,0 +1,146 @@ +/* + * 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.tools.cli.commands.cookie; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; +import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.tools.cli.helpers.CookieCommandTestBase; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Versioned; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test {@link GetCookieCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ MetadataDrivers.class }) +public class GetCookieCommandTest extends CookieCommandTestBase { + + @Rule + public final TemporaryFolder testFolder = new TemporaryFolder(); + + private final ByteArrayOutputStream output = new ByteArrayOutputStream(); + private final PrintStream console = new PrintStream(output); + + private boolean runCommand(String[] args) { + GetCookieCommand getCmd = new GetCookieCommand(console); + BKFlags bkFlags = new BKFlags(); + bkFlags.serviceUri = "zk://127.0.0.1"; + return getCmd.apply(bkFlags, args); + } + + private String getConsoleOutput() { + return new String(output.toByteArray(), UTF_8); + } + + /** + * Run a command without providing bookie id. + */ + @Test + public void testMissingBookieId() { + assertFalse(runCommand(new String[] {})); + String consoleOutput = getConsoleOutput(); + assertBookieIdMissing(consoleOutput); + } + + /** + * Run a command with invalid bookie id. + */ + @Test + public void testInvalidBookieId() { + assertFalse(runCommand(new String[] { INVALID_BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertInvalidBookieId(consoleOutput, INVALID_BOOKIE_ID); + } + + /** + * A successful run. + */ + @Test + public void testGetCookieFromExistentCookieFile() throws Exception { + Cookie cookie = Cookie.newBuilder() + .setBookieHost(BOOKIE_ID) + .setInstanceId("test-instance-id") + .setJournalDirs("/path/to/journal/dir") + .setLedgerDirs("/path/to/ledger/dirs") + .build(); + when(rm.readCookie(eq(BOOKIE_ID))) + .thenReturn(new Versioned<>(cookie.toString().getBytes(UTF_8), new LongVersion(-1L))); + assertTrue(runCommand(new String[] { BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue(consoleOutput, consoleOutput.contains(cookie.toString())); + verify(rm, times(1)).readCookie(eq(BOOKIE_ID)); + } + + /** + * Run a command to get cookie on an non-existent cookie. + */ + @Test + public void testGetNonExistedCookie() throws Exception { + doThrow(new CookieNotFoundException()) + .when(rm).readCookie(eq(BOOKIE_ID)); + + assertFalse(runCommand(new String[] { BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue( + consoleOutput, + consoleOutput.contains("Cookie not found for bookie '" + BOOKIE_ID + "'")); + verify(rm, times(1)).readCookie(eq(BOOKIE_ID)); + } + + /** + * Run a command to get cookie when exception is thrown. + */ + @Test + public void testGetCookieException() throws Exception { + doThrow(new OperationRejectedException()) + .when(rm).readCookie(eq(BOOKIE_ID)); + + assertFalse(runCommand(new String[] { BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue( + consoleOutput, + consoleOutput.contains("Exception on getting cookie for bookie '" + BOOKIE_ID + "'")); + assertTrue( + consoleOutput, + consoleOutput.contains(OperationRejectedException.class.getName())); + verify(rm, times(1)).readCookie(eq(BOOKIE_ID)); + } + +} diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java new file mode 100644 index 00000000000..308cd2aee1b --- /dev/null +++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java @@ -0,0 +1,180 @@ +/* + * 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.tools.cli.commands.cookie; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.PrintStream; +import java.nio.file.Files; +import java.nio.file.Paths; +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; +import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.tools.cli.helpers.CookieCommandTestBase; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.versioning.Versioned; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test {@link UpdateCookieCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ MetadataDrivers.class }) +public class UpdateCookieCommandTest extends CookieCommandTestBase { + + @Rule + public final TemporaryFolder testFolder = new TemporaryFolder(); + + private final ByteArrayOutputStream output = new ByteArrayOutputStream(); + private final PrintStream console = new PrintStream(output); + + private boolean runCommand(String[] args) { + UpdateCookieCommand updateCmd = new UpdateCookieCommand(console); + BKFlags bkFlags = new BKFlags(); + bkFlags.serviceUri = "zk://127.0.0.1"; + return updateCmd.apply(bkFlags, args); + } + + private String getConsoleOutput() { + return new String(output.toByteArray(), UTF_8); + } + + /** + * Run a command without providing bookie id. + */ + @Test + public void testMissingBookieId() { + assertFalse(runCommand(new String[] {})); + String consoleOutput = getConsoleOutput(); + assertBookieIdMissing(consoleOutput); + } + + private void assertPrintUsage(String consoleOutput) { + assertPrintUsage(consoleOutput, "update [flags] "); + } + + /** + * Run a command without cookie file. + */ + @Test + public void testMissingCookieFileOption() { + assertFalse(runCommand(new String[] { BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertOptionMissing(consoleOutput, "-cf, --cookie-file"); + assertPrintUsage(consoleOutput); + } + + /** + * Run a command with invalid bookie id. + */ + @Test + public void testInvalidBookieId() { + assertFalse(runCommand(new String[] { "-cf", "test-cookie-file", INVALID_BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertInvalidBookieId(consoleOutput, INVALID_BOOKIE_ID); + } + + /** + * Run a command with a non-existent cookie file. + */ + @Test + public void testUpdateCookieFromNonExistentCookieFile() { + String file = "/path/to/non-existent-cookie-file"; + assertFalse(runCommand(new String[] { "-cf", file, BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertCookieFileNotExists(consoleOutput, file); + } + + /** + * A successful run. + */ + @SuppressWarnings("unchecked") + @Test + public void testUpdateCookieFromExistentCookieFile() throws Exception { + File file = testFolder.newFile("test-cookie-file"); + byte[] content = "test-update-cookie".getBytes(UTF_8); + Files.write(Paths.get(file.toURI()), content); + String fileName = file.getPath(); + assertTrue(runCommand(new String[] { "-cf", fileName, BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue(consoleOutput, consoleOutput.isEmpty()); + verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + } + + /** + * Run a command to update cookie on an non-existent cookie. + */ + @SuppressWarnings("unchecked") + @Test + public void testUpdateNonExistedCookie() throws Exception { + doThrow(new CookieNotFoundException()) + .when(rm).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + + File file = testFolder.newFile("test-cookie-file"); + byte[] content = "test-update-cookie".getBytes(UTF_8); + Files.write(Paths.get(file.toURI()), content); + String fileName = file.getPath(); + assertFalse(runCommand(new String[] { "-cf", fileName, BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue( + consoleOutput, + consoleOutput.contains("Cookie not found for bookie '" + BOOKIE_ID + "'")); + verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + } + + /** + * Run a command to update cookie when exception is thrown. + */ + @SuppressWarnings("unchecked") + @Test + public void testUpdateCookieException() throws Exception { + doThrow(new OperationRejectedException()) + .when(rm).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + + File file = testFolder.newFile("test-cookie-file"); + byte[] content = "test-update-cookie".getBytes(UTF_8); + Files.write(Paths.get(file.toURI()), content); + String fileName = file.getPath(); + assertFalse(runCommand(new String[] { "-cf", fileName, BOOKIE_ID })); + String consoleOutput = getConsoleOutput(); + assertTrue( + consoleOutput, + consoleOutput.contains("Exception on updating cookie for bookie '" + BOOKIE_ID + "'")); + assertTrue( + consoleOutput, + consoleOutput.contains(OperationRejectedException.class.getName())); + verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + } + +} diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommandTest.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommandTest.java new file mode 100644 index 00000000000..9597cdeef21 --- /dev/null +++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommandTest.java @@ -0,0 +1,65 @@ +/* + * 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.tools.cli.helpers; + +import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.same; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import org.apache.bookkeeper.tools.common.BKCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.commons.configuration.CompositeConfiguration; +import org.junit.Test; + +/** + * Unit test {@link BookieShellCommand}. + */ +public class BookieShellCommandTest { + + @SuppressWarnings("unchecked") + @Test + public void testShellCommand() throws Exception { + BKCommand command = mock(BKCommand.class); + String shellCommandName = "test-shell-command"; + CompositeConfiguration conf = new CompositeConfiguration(); + BookieShellCommand shellCommand = new BookieShellCommand<>( + shellCommandName, + command, + conf); + + // test `description` + assertEquals( + shellCommandName + " [options]", + shellCommand.description()); + + // test `printUsage` + shellCommand.printUsage(); + verify(command, times(1)).usage(); + + // test `runCmd` + String[] args = new String[] { "arg-1", "arg-2" }; + shellCommand.runCmd(args); + verify(command, times(1)) + .apply(same(shellCommandName), same(conf), same(args)); + } + +} diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CookieCommandTestBase.java b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CookieCommandTestBase.java new file mode 100644 index 00000000000..f6f66b6e2fa --- /dev/null +++ b/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CookieCommandTestBase.java @@ -0,0 +1,91 @@ +/* + * 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.tools.cli.helpers; + +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; + +import java.util.function.Function; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.junit.Before; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * A test base for testing cookie commands. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ MetadataDrivers.class }) +public class CookieCommandTestBase extends CommandTestBase { + + protected static final String INVALID_BOOKIE_ID = "127.0.0.1"; + protected static final String BOOKIE_ID = "127.0.0.1:3181"; + + protected RegistrationManager rm; + + @Before + public void setup() throws Exception { + PowerMockito.mockStatic(MetadataDrivers.class); + this.rm = mock(RegistrationManager.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function func = invocationOnMock.getArgument(1); + func.apply(rm); + return true; + }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", + any(ServerConfiguration.class), any(Function.class)); + } + + protected void assertBookieIdMissing(String consoleOutput) { + assertTrue( + consoleOutput, + consoleOutput.contains("No bookie id or more bookie ids is specified") + ); + } + + protected void assertInvalidBookieId(String consoleOutput, String bookieId) { + assertTrue( + consoleOutput, + consoleOutput.contains("Invalid bookie id '" + bookieId + "'")); + } + + protected void assertOptionMissing(String consoleOutput, String option) { + assertTrue( + consoleOutput, + consoleOutput.contains("The following option is required: " + option)); + } + + protected void assertPrintUsage(String consoleOutput, String usage) { + assertTrue( + consoleOutput, + consoleOutput.contains("Usage: " + usage)); + } + + protected void assertCookieFileNotExists(String consoleOutput, String cookieFile) { + assertTrue( + consoleOutput, + consoleOutput.contains("Cookie file '" + cookieFile + "' doesn't exist.")); + } + +} diff --git a/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/BKCommand.java b/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/BKCommand.java index bf591abf449..35b2cbd9b54 100644 --- a/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/BKCommand.java +++ b/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/BKCommand.java @@ -23,6 +23,7 @@ import java.net.URL; import java.nio.file.Paths; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; import org.apache.bookkeeper.common.net.ServiceURI; import org.apache.bookkeeper.tools.framework.Cli; import org.apache.bookkeeper.tools.framework.CliCommand; @@ -50,6 +51,19 @@ public Boolean apply(BKFlags globalFlags, String[] args) { return 0 == Cli.runCli(newSpec, args); } + /** + * Made this as public for allowing old bookie shell use new cli command. + * This should be removed once we get rid of the old bookie shell. + */ + @Private + public int apply(String commandName, CompositeConfiguration conf, String[] args) { + CliSpec newSpec = CliSpec.newBuilder(spec) + .withName(commandName) + .withRunFunc(cmdFlags -> apply(null, conf, new BKFlags(), cmdFlags)) + .build(); + return Cli.runCli(newSpec, args); + } + protected boolean apply(BKFlags bkFlags, CommandFlagsT cmdFlags) { ServiceURI serviceURI = null; From e7e2c89d00bfa2345c03e726b96cdd571a59dbeb Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 9 Nov 2018 16:32:37 -0800 Subject: [PATCH 0109/1642] [TOOLS] improve bkctl help message Descriptions of the changes in this PR: *Motivation* Currently `bin/bktcl help` will output all command groups all together. It is a bit hard to tell what command groups to use, especially some commands are used for table service. *Changes* Introduce `category` in the cli spec to define which category that a command group belongs to. So when it prints out the help message, it can use that information to categorize the command groups together to provide better user experience. *Results* ``` $ bin/bkctl help bkctl interacts and operates Apache BookKeeper clusters Usage: bkctl [flags] [command group] [commands] Infrastructure commands : bookie Commands on operating a single bookie bookies Commands on operating a cluster of bookies cluster Commands on administrating bookkeeper clusters Ledger service commands : ledger Commands on interacting with ledgers Table service commands : namespace Commands on operating namespaces table Commands on interacting with tables tables Commands on operating tables Other commands : help Display help information about it ``` Reviewers: Enrico Olivelli , Matteo Merli This closes #1793 from sijie/add_groups --- .../cli/commands/BookieCommandGroup.java | 3 ++ .../cli/commands/BookiesCommandGroup.java | 3 ++ .../cli/commands/LedgerCommandGroup.java | 3 ++ .../tools/common/BKCommandCategories.java | 36 +++++++++++++ .../tools/framework/CliCommand.java | 5 ++ .../bookkeeper/tools/framework/CliSpec.java | 17 +++++- .../bookkeeper/tools/framework/Command.java | 9 ++++ .../tools/framework/CommandUtils.java | 53 ++++++++++++++++--- .../stream/cli/ClusterCommandGroup.java | 3 ++ .../stream/cli/NamespaceCommandGroup.java | 3 ++ .../stream/cli/TableAdminCommandGroup.java | 3 ++ .../stream/cli/TableCommandGroup.java | 3 ++ 12 files changed, 134 insertions(+), 7 deletions(-) create mode 100644 tools/framework/src/main/java/org/apache/bookkeeper/tools/common/BKCommandCategories.java diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index ec4fbc5f607..2a5cf1fddbe 100644 --- a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -18,6 +18,8 @@ */ package org.apache.bookkeeper.tools.cli.commands; +import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; + import org.apache.bookkeeper.tools.cli.BKCtl; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.common.BKFlags; @@ -36,6 +38,7 @@ public class BookieCommandGroup extends CliCommandGroup { .withName(NAME) .withDescription(DESC) .withParent(BKCtl.NAME) + .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new LastMarkCommand()) .build(); diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java index 1fff60f8f60..6b86de70296 100644 --- a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java +++ b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java @@ -18,6 +18,8 @@ */ package org.apache.bookkeeper.tools.cli.commands; +import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; + import org.apache.bookkeeper.tools.cli.BKCtl; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.common.BKFlags; @@ -36,6 +38,7 @@ public class BookiesCommandGroup extends CliCommandGroup { .withName(NAME) .withDescription(DESC) .withParent(BKCtl.NAME) + .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new ListBookiesCommand()) .build(); diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java index 4236c4918ce..c5286565724 100644 --- a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java +++ b/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java @@ -18,6 +18,8 @@ */ package org.apache.bookkeeper.tools.cli.commands; +import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_LEDGER_SERVICE; + import org.apache.bookkeeper.tools.cli.BKCtl; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; @@ -36,6 +38,7 @@ public class LedgerCommandGroup extends CliCommandGroup { .withName(NAME) .withDescription(DESC) .withParent(BKCtl.NAME) + .withCategory(CATEGORY_LEDGER_SERVICE) .addCommand(new SimpleTestCommand()) .build(); diff --git a/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/BKCommandCategories.java b/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/BKCommandCategories.java new file mode 100644 index 00000000000..35aa3bd97c2 --- /dev/null +++ b/tools/framework/src/main/java/org/apache/bookkeeper/tools/common/BKCommandCategories.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.tools.common; + +/** + * Classes to keep a list of command categories. + */ +public final class BKCommandCategories { + + // commands that operate cluster and nodes + public static final String CATEGORY_INFRA_SERVICE = "Infrastructure commands"; + + // commands that operate ledger service + public static final String CATEGORY_LEDGER_SERVICE = "Ledger service commands"; + + // commands that operate table service + public static final String CATEGORY_TABLE_SERVICE = "Table service commands"; + +} diff --git a/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CliCommand.java b/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CliCommand.java index ab7b955d1cd..cf68acfeddf 100644 --- a/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CliCommand.java +++ b/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CliCommand.java @@ -41,6 +41,11 @@ public void setParent(String parent) { .build(); } + @Override + public String category() { + return spec.category(); + } + @Override public String name() { return name; diff --git a/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CliSpec.java b/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CliSpec.java index bbae3f581ae..c708bf131f9 100644 --- a/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CliSpec.java +++ b/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CliSpec.java @@ -67,10 +67,12 @@ public static class Builder { private PrintStream console = System.out; private boolean isCommandGroup = false; private String argumentsUsage = ""; + private String category = ""; private Builder() {} private Builder(CliSpec spec) { + this.category = spec.category; this.name = spec.name; this.parent = spec.parent; this.usage = spec.usage; @@ -85,6 +87,11 @@ private Builder(CliSpec spec) { this.isCommandGroup = spec.isCommandGroup; } + public Builder withCategory(String category) { + this.category = category; + return this; + } + public Builder withName(String name) { this.name = name; return this; @@ -142,6 +149,7 @@ public Builder setCommandGroup(boolean enabled) { public CliSpec build() { return new CliSpec<>( + category, name, parent, usage, @@ -158,6 +166,7 @@ public CliSpec build() { } + private final String category; private final String name; private final String parent; private final String usage; @@ -171,7 +180,8 @@ public CliSpec build() { // whether the cli spec is for a command group. private final boolean isCommandGroup; - private CliSpec(String name, + private CliSpec(String category, + String name, String parent, String usage, String argumentsUsage, @@ -182,6 +192,7 @@ private CliSpec(String name, Function runFunc, PrintStream console, boolean isCommandGroup) { + this.category = category; this.name = name; this.parent = parent; this.usage = usage; @@ -195,6 +206,10 @@ private CliSpec(String name, this.isCommandGroup = isCommandGroup; } + public String category() { + return category; + } + public String name() { return name; } diff --git a/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/Command.java b/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/Command.java index 186b29fad1b..16814322960 100644 --- a/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/Command.java +++ b/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/Command.java @@ -32,6 +32,15 @@ default boolean hidden() { return false; } + /** + * Return category of this command belongs to. + * + * @return category name + */ + default String category() { + return ""; + } + /** * Return command name. * diff --git a/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CommandUtils.java b/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CommandUtils.java index 77c6b48ae61..4698be9aa6d 100644 --- a/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CommandUtils.java +++ b/tools/framework/src/main/java/org/apache/bookkeeper/tools/framework/CommandUtils.java @@ -27,7 +27,9 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.stream.IntStream; +import org.apache.commons.lang.StringUtils; /** * Utils to process a commander. @@ -164,9 +166,6 @@ public static void printAvailableCommands(Map commands, return; } - printer.println("Commands:"); - printer.println(); - int longestCommandName = commands .keySet() .stream() @@ -174,17 +173,44 @@ public static void printAvailableCommands(Map commands, .max() .orElse(0); + // group the commands by category + Map> categorizedCommands = new TreeMap<>(); for (Map.Entry commandEntry : commands.entrySet()) { if ("help".equals(commandEntry.getKey())) { - // don't print help message along with available other commands + // don't add help message along with other commands continue; } - printCommand(printer, commandEntry.getKey(), commandEntry.getValue(), longestCommandName); + String category = commandEntry.getValue().category(); + Map subCommands = categorizedCommands.get(category); + if (null == subCommands) { + subCommands = new TreeMap<>(); + categorizedCommands.put(category, subCommands); + } + subCommands.put(commandEntry.getKey(), commandEntry.getValue()); + } + + // there is only one category, print all of them under `Commands`. + if (categorizedCommands.size() <= 1) { + printer.println("Commands:"); + printer.println(); + } + + for (Map.Entry> categoryEntry : categorizedCommands.entrySet()) { + String category = categoryEntry.getKey(); + printCategoryHeader(printer, category); + Map subCommands = categoryEntry.getValue(); + for (Map.Entry commandEntry : subCommands.entrySet()) { + printCommand(printer, commandEntry.getKey(), commandEntry.getValue(), longestCommandName); + } + printer.println(); } Command helpCmd = commands.get("help"); if (null != helpCmd) { - printer.println(); + if (categorizedCommands.size() > 1) { + // if commands has been categorized, put help + printCategoryHeader(printer, "Other commands"); + } printCommand(printer, "help", helpCmd, longestCommandName); } @@ -215,4 +241,19 @@ private static void printCommand(PrintStream printer, command.description()); } + private static void printCategoryHeader(PrintStream printer, + String category) { + if (StringUtils.isEmpty(category)) { + return; + } + + printIndent(printer, 0); + printDescription( + printer, + 0, + 0, + category + " :"); + printer.println(); + } + } diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/ClusterCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/ClusterCommandGroup.java index 375bc6610ec..303275fc2df 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/ClusterCommandGroup.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/ClusterCommandGroup.java @@ -17,6 +17,8 @@ */ package org.apache.bookkeeper.stream.cli; +import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; + import org.apache.bookkeeper.stream.cli.commands.cluster.InitClusterCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -34,6 +36,7 @@ public class ClusterCommandGroup extends CliCommandGroup { .withName(NAME) .withDescription(DESC) .withParent("bkctl") + .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new InitClusterCommand()) .build(); diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java index 1300bf2515d..7263bf72b3f 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java @@ -17,6 +17,8 @@ */ package org.apache.bookkeeper.stream.cli; +import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_TABLE_SERVICE; + import org.apache.bookkeeper.stream.cli.commands.namespace.CreateNamespaceCommand; import org.apache.bookkeeper.stream.cli.commands.namespace.DeleteNamespaceCommand; import org.apache.bookkeeper.stream.cli.commands.namespace.GetNamespaceCommand; @@ -36,6 +38,7 @@ public class NamespaceCommandGroup extends CliCommandGroup { .withName(NAME) .withDescription(DESC) .withParent("bkctl") + .withCategory(CATEGORY_TABLE_SERVICE) .addCommand(new CreateNamespaceCommand()) .addCommand(new GetNamespaceCommand()) .addCommand(new DeleteNamespaceCommand()) diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java index bce4fc58713..8a57c956aba 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java @@ -17,6 +17,8 @@ */ package org.apache.bookkeeper.stream.cli; +import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_TABLE_SERVICE; + import org.apache.bookkeeper.stream.cli.commands.table.CreateTableCommand; import org.apache.bookkeeper.stream.cli.commands.table.DeleteTableCommand; import org.apache.bookkeeper.stream.cli.commands.table.GetTableCommand; @@ -36,6 +38,7 @@ public class TableAdminCommandGroup extends CliCommandGroup { .withName(NAME) .withDescription(DESC) .withParent("bkctl") + .withCategory(CATEGORY_TABLE_SERVICE) .addCommand(new CreateTableCommand()) .addCommand(new GetTableCommand()) .addCommand(new DeleteTableCommand()) diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java index f7d96b7ab5d..31bff70f555 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java @@ -17,6 +17,8 @@ */ package org.apache.bookkeeper.stream.cli; +import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_TABLE_SERVICE; + import org.apache.bookkeeper.stream.cli.commands.table.DelCommand; import org.apache.bookkeeper.stream.cli.commands.table.GetCommand; import org.apache.bookkeeper.stream.cli.commands.table.IncrementCommand; @@ -37,6 +39,7 @@ public class TableCommandGroup extends CliCommandGroup { .withName(NAME) .withDescription(DESC) .withParent("bkctl") + .withCategory(CATEGORY_TABLE_SERVICE) .addCommand(new PutCommand()) .addCommand(new GetCommand()) .addCommand(new IncrementCommand()) From 5c9ba3a261c3494831533e055bba3f706937e197 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sat, 10 Nov 2018 01:34:57 -0800 Subject: [PATCH 0110/1642] [BUILD] Ignore `versionsBackup` file generated by `mvn versions:set` Descriptions of the changes in this PR: *Motivation* `mvn versions:set` will generate files suffixed with `.versionsBackup` Reviewers: Enrico Olivelli , Matteo Merli This closes #1801 from sijie/exclude_versionsbackup --- .gitignore | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index dcdcb75890a..d8778b2a75a 100644 --- a/.gitignore +++ b/.gitignore @@ -32,3 +32,6 @@ data/ # Pid files **/*.pid + +# Exclude versionBackup file (generated by `mvn versions:set`) +**/*.versionsBackup From 2a83438e91c37f00fda29e187efa36e1915e3f89 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Sun, 11 Nov 2018 10:42:32 -0800 Subject: [PATCH 0111/1642] Fixed Auth with v2 protocol ### Motivation BK auth framework is currently broken when using v2 protocol. ### Changes * Fixed auth when using V2 protocol * Made sure a client with authentication enabled can talk to a bookie without authentication. This is required in any case when enabling/disabling authentication on a live cluster. * Run all auth tests against both v2 and v3 protocol. This should be included in 4.7.2 to give a path to upgrade. cc/ rdhabalia Reviewers: Enrico Olivelli , Sijie Guo This closes #1805 from merlimat/fix-v2-auth --- .../apache/bookkeeper/proto/AuthHandler.java | 58 +++++++++++++++---- .../proto/BookieRequestProcessor.java | 11 ++++ .../proto/PerChannelBookieClient.java | 2 +- .../org/apache/bookkeeper/auth/TestAuth.java | 40 ++++++++++++- 4 files changed, 99 insertions(+), 12 deletions(-) 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 1b1f60fc592..a7ac452aea3 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 @@ -226,16 +226,19 @@ static class ClientSideHandler extends ChannelDuplexHandler { final Queue waitingForAuth = new ConcurrentLinkedQueue<>(); final ClientConnectionPeer connectionPeer; + private final boolean isUsingV2Protocol; + public ClientAuthProvider getAuthProvider() { return authProvider; } ClientSideHandler(ClientAuthProvider.Factory authProviderFactory, AtomicLong transactionIdGenerator, - ClientConnectionPeer connectionPeer) { + ClientConnectionPeer connectionPeer, boolean isUsingV2Protocol) { this.authProviderFactory = authProviderFactory; this.transactionIdGenerator = transactionIdGenerator; this.connectionPeer = connectionPeer; authProvider = null; + this.isUsingV2Protocol = isUsingV2Protocol; } @Override @@ -279,7 +282,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (AUTHENTICATION_DISABLED_PLUGIN_NAME.equals(am.getAuthPluginName())){ SocketAddress remote = ctx.channel().remoteAddress(); LOG.info("Authentication is not enabled." - + "Considering this client {0} authenticated", remote); + + "Considering this client {} authenticated", remote); AuthHandshakeCompleteCallback cb = new AuthHandshakeCompleteCallback(ctx); cb.operationComplete(BKException.Code.OK, null); return; @@ -296,6 +299,33 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception break; } } + } else if (msg instanceof BookieProtocol.Response) { + BookieProtocol.Response resp = (BookieProtocol.Response) msg; + switch (resp.opCode) { + case BookieProtocol.AUTH: + if (resp.errorCode != BookieProtocol.EOK) { + authenticationError(ctx, resp.errorCode); + } else { + BookkeeperProtocol.AuthMessage am = ((BookieProtocol.AuthResponse) resp).authMessage; + if (AUTHENTICATION_DISABLED_PLUGIN_NAME.equals(am.getAuthPluginName())) { + SocketAddress remote = ctx.channel().remoteAddress(); + LOG.info("Authentication is not enabled." + + "Considering this client {} authenticated", remote); + AuthHandshakeCompleteCallback cb = new AuthHandshakeCompleteCallback(ctx); + cb.operationComplete(BKException.Code.OK, null); + return; + } + byte[] payload = am.getPayload().toByteArray(); + authProvider.process(AuthToken.wrap(payload), new AuthRequestCallback(ctx, + authProviderFactory.getPluginName())); + } + break; + default: + LOG.warn("dropping received message {} from bookie {}", msg, ctx.channel()); + // else just drop the message, we're not authenticated so nothing should be coming + // through + break; + } } } @@ -319,7 +349,7 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) } else if (msg instanceof BookieProtocol.Request) { // let auth messages through, queue the rest BookieProtocol.Request req = (BookieProtocol.Request) msg; - if (BookkeeperProtocol.OperationType.AUTH.getNumber() == req.getOpCode()) { + if (BookieProtocol.AUTH == req.getOpCode()) { super.write(ctx, msg, promise); super.flush(ctx); } else { @@ -356,16 +386,24 @@ public void operationComplete(int rc, AuthToken newam) { authenticationError(ctx, rc); return; } + AuthMessage message = AuthMessage.newBuilder().setAuthPluginName(pluginName) .setPayload(ByteString.copyFrom(newam.getData())).build(); - BookkeeperProtocol.BKPacketHeader header = BookkeeperProtocol.BKPacketHeader.newBuilder() - .setVersion(BookkeeperProtocol.ProtocolVersion.VERSION_THREE) - .setOperation(BookkeeperProtocol.OperationType.AUTH).setTxnId(newTxnId()).build(); - BookkeeperProtocol.Request.Builder builder = BookkeeperProtocol.Request.newBuilder().setHeader(header) - .setAuthRequest(message); - - channel.writeAndFlush(builder.build()); + if (isUsingV2Protocol) { + channel.writeAndFlush( + new BookieProtocol.AuthRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, message), + channel.voidPromise()); + } else { + // V3 protocol + BookkeeperProtocol.BKPacketHeader header = BookkeeperProtocol.BKPacketHeader.newBuilder() + .setVersion(BookkeeperProtocol.ProtocolVersion.VERSION_THREE) + .setOperation(BookkeeperProtocol.OperationType.AUTH).setTxnId(newTxnId()).build(); + BookkeeperProtocol.Request.Builder builder = BookkeeperProtocol.Request.newBuilder() + .setHeader(header) + .setAuthRequest(message); + channel.writeAndFlush(builder.build()); + } } } 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 5a50bf69291..78b35ec6cd2 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 @@ -498,6 +498,17 @@ public void processRequest(Object msg, Channel c) { checkArgument(r instanceof BookieProtocol.ReadRequest); processReadRequest((BookieProtocol.ReadRequest) r, c); break; + case BookieProtocol.AUTH: + LOG.info("Ignoring auth operation from client {}", c.remoteAddress()); + BookkeeperProtocol.AuthMessage message = BookkeeperProtocol.AuthMessage + .newBuilder() + .setAuthPluginName(AuthProviderFactoryFactory.AUTHENTICATION_DISABLED_PLUGIN_NAME) + .setPayload(ByteString.copyFrom(AuthToken.NULL.getData())) + .build(); + + c.writeAndFlush(new BookieProtocol.AuthResponse( + BookieProtocol.CURRENT_PROTOCOL_VERSION, message)); + break; default: LOG.error("Unknown op type {}, sending error", r.getOpCode()); c.writeAndFlush(ResponseBuilder.buildErrorResponse(BookieProtocol.EBADREQ, r)); 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 071b8395bf7..917f6f3323e 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 @@ -462,7 +462,7 @@ protected void initChannel(Channel ch) throws Exception { "bookieProtoDecoder", new BookieProtoEncoding.ResponseDecoder(extRegistry, useV2WireProtocol)); pipeline.addLast("authHandler", new AuthHandler.ClientSideHandler(authProviderFactory, txnIdGenerator, - connectionPeer)); + connectionPeer, useV2WireProtocol)); pipeline.addLast("mainhandler", PerChannelBookieClient.this); } }); 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 cca97e8d404..bf7d84b9546 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 @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collection; import java.util.Enumeration; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -45,12 +46,16 @@ import org.apache.bookkeeper.proto.ClientConnectionPeer; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Test authentication. */ +@RunWith(Parameterized.class) public class TestAuth extends BookKeeperClusterTestCase { static final Logger LOG = LoggerFactory.getLogger(TestAuth.class); public static final String TEST_AUTH_PROVIDER_PLUGIN_NAME = "TestAuthProviderPlugin"; @@ -61,8 +66,29 @@ public class TestAuth extends BookKeeperClusterTestCase { private static final byte[] FAILURE_RESPONSE = {2}; private static final byte[] PAYLOAD_MESSAGE = {3}; - public TestAuth() { + enum ProtocolVersion { + ProtocolV2, ProtocolV3 + } + + @Parameters + public static Collection configs() { + return Arrays.asList(new Object[][] { + { ProtocolVersion.ProtocolV2 }, + { ProtocolVersion.ProtocolV3 }, + }); + } + + private final ProtocolVersion protocolVersion; + + public TestAuth(ProtocolVersion protocolVersion) { super(0); // start them later when auth providers are configured + this.protocolVersion = protocolVersion; + } + + protected ClientConfiguration newClientConfiguration() { + ClientConfiguration conf = super.newClientConfiguration(); + conf.setUseV2WireProtocol(protocolVersion == ProtocolVersion.ProtocolV2); + return conf; } // we pass in ledgerId because the method may throw exceptions @@ -136,6 +162,13 @@ public void testSingleMessageAuth() throws Exception { @Test public void testCloseMethodCalledOnAuthProvider() throws Exception { + LogCloseCallsBookieAuthProviderFactory.closeCountersOnFactory.set(0); + LogCloseCallsBookieAuthProviderFactory.closeCountersOnConnections.set(0); + LogCloseCallsBookieAuthProviderFactory.initCountersOnFactory.set(0); + LogCloseCallsBookieAuthProviderFactory.initCountersOnConnections.set(0); + LogCloseCallsClientAuthProviderFactory.initCountersOnFactory.set(0); + LogCloseCallsClientAuthProviderFactory.closeCountersOnFactory.set(0); + ServerConfiguration bookieConf = newServerConfiguration(); bookieConf.setBookieAuthProviderFactoryClass( LogCloseCallsBookieAuthProviderFactory.class.getName()); @@ -272,6 +305,11 @@ public void testDifferentPluginFailure() throws Exception { } catch (BKException.BKUnauthorizedAccessException bke) { // bookie should have sent a negative response before // breaking the conneciton + assertEquals(ProtocolVersion.ProtocolV3, protocolVersion); + } catch (BKException.BKNotEnoughBookiesException nebe) { + // With V2 we don't get the authorization error, but rather just + // fail to write to bookies. + assertEquals(ProtocolVersion.ProtocolV2, protocolVersion); } assertFalse(ledgerId.get() == -1); assertEquals("Shouldn't have entry", 0, entryCount(ledgerId.get(), bookieConf, clientConf)); From 24c34ae3f4bafa3ed1682bf39044e2b1cb723e3d Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Sun, 11 Nov 2018 13:28:58 -0800 Subject: [PATCH 0112/1642] Fix indentation in BP-34 doc Descriptions of the changes in this PR: - just indentation fix Reviewers: Enrico Olivelli , Sijie Guo This closes #1803 from reddycharan/bp34fix --- site/bps/BP-34-cluster-metadata-checker.md | 34 +++++++++++----------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/site/bps/BP-34-cluster-metadata-checker.md b/site/bps/BP-34-cluster-metadata-checker.md index 29916d18d33..1992d4cab70 100644 --- a/site/bps/BP-34-cluster-metadata-checker.md +++ b/site/bps/BP-34-cluster-metadata-checker.md @@ -14,10 +14,10 @@ Ideally for having complete confidence on the date in the cluster, it is needed ### Proposed Changes Intention of this new checker is to validate following things - - ledger placement policy : Ensemble of each segment in Ledger should adhere to LedgerPlacementPolicy - - durability contract : Every entry has WQ number of replicas and entries are replicated according to RoundRobinDistributionSchedule - - progress in handling under replication : No ledger is marked underreplicated for more than acceptable time - - availability of bookies of the ensemble of ledgers : If Auditor fails to get response from a Bookie, then that Bookie shouldn’t be registered to metadata server and Auditor should be aware of it unavailability or if it is a transient error in getting response from Bookie then subsequent calls to that Bookie should succeed. + - ledger placement policy : Ensemble of each segment in Ledger should adhere to LedgerPlacementPolicy + - durability contract : Every entry has WQ number of replicas and entries are replicated according to RoundRobinDistributionSchedule + - progress in handling under replication : No ledger is marked underreplicated for more than acceptable time + - availability of bookies of the ensemble of ledgers : If Auditor fails to get response from a Bookie, then that Bookie shouldn’t be registered to metadata server and Auditor should be aware of it unavailability or if it is a transient error in getting response from Bookie then subsequent calls to that Bookie should succeed. Roles and Responsibilities of the cluster metadata checker - Police the durability contract and report violations. Its job is to make sure that the metadata server(zk) and the storage servers (bookies) are in sync. Simply put, check if bookies agree with the metadata server metadata and if not, raise an alert. @@ -25,19 +25,19 @@ Roles and Responsibilities of the cluster metadata checker - The Metadata Scrutiny needs to be light weighted esp., on Bookie and must run regularly giving the confidence that the cluster is in good state. High Level Logic - - Things would get complicated analyzing ledgers which are not closed because of several reasons, viz., unable to know lastEntryId by reading ZK metadata, possibility of change in ensemble because of write failure to a bookie, and other subtleties in dealing with last unclosed segment of the ledger. So for the sake of simplicity this checker should be limited to ledgers which are write closed/fenced. - - This durability check for each ledger will be run as a processor in ledgerManager.asyncProcessLedgers and it would ignore ledgers which are still open for write. - - first step is to check if this ledger is marked underreplicated already. If it is marked underreplicated for more than acceptable time then report it as violation otherwise skip this underreplicated ledger for this iteration of durability check. Since there is no point in further analyzing this ledger if it is already marked under replicated. - - get the ledger metadata of the ledger from the metadata server - - make sure that the ensemble of the ledger segments is in agreement with ledgerplacement policy. Any violation should be reported. - - get the info about available entries of the ledger from the bookies of the ensemble. Bookie is expected to return list of entries it contains for a given ledger - - Have to make sure that Bookies contain all the entries it is supposed to contain according to the RoundRobinDistributionSchedule and each entry has writequorum number of copies. Any violation should be reported. - - If there is any failure in trying to get info. from Bookie of the ensembles of the ledger, then add this ledger to potentially faulty ledgers list (but don't report it yet.) - - (in previous steps, in case of any violation or bookie read error, before reporting violation, check if the ledger is marked underreplicated. If it is marked underreplicated then ignore this ledger for this iteration. If it is not marked underreplicated, then get the ledgermetadata of this ledger onemore time. Check if it is any different from the ledgermetadata we got initially then instead of reporting the violation, redo the analysis for this ledger because apparently something had changed in the metadata (esp. with ensemble) and hence it is better to reevaluate instead of false alarm.) - - if there are potentially faulty ledgers because of unavailable/unreachable bookies, then schedule a new durability check task with time delay just for the potentially faulty ledgers. Even after subsequent delayed checks, if Auditor failed to get response from bookies then make sure that Bookie isn’t registered to metadata server and Auditor is aware of it unavailability, if not then report the violation. - - Auditor is going to use existing mechanisms/frameworks to report the violations - bookkeeper-stats statslogger/counters and complementing information in logs. - - It makes sense to group all the durability violations found in a scrutiny run according to the categories and report the aggregated count for each category after the end of the scrutiny run. - - before reporting these violations, each violation should be logged with complete information, so that it can be used to understand what went wrong. + - Things would get complicated analyzing ledgers which are not closed because of several reasons, viz., unable to know lastEntryId by reading ZK metadata, possibility of change in ensemble because of write failure to a bookie, and other subtleties in dealing with last unclosed segment of the ledger. So for the sake of simplicity this checker should be limited to ledgers which are write closed/fenced. + - This durability check for each ledger will be run as a processor in ledgerManager.asyncProcessLedgers and it would ignore ledgers which are still open for write. + - first step is to check if this ledger is marked underreplicated already. If it is marked underreplicated for more than acceptable time then report it as violation otherwise skip this underreplicated ledger for this iteration of durability check. Since there is no point in further analyzing this ledger if it is already marked under replicated. + - get the ledger metadata of the ledger from the metadata server + - make sure that the ensemble of the ledger segments is in agreement with ledgerplacement policy. Any violation should be reported. + - get the info about available entries of the ledger from the bookies of the ensemble. Bookie is expected to return list of entries it contains for a given ledger + - Have to make sure that Bookies contain all the entries it is supposed to contain according to the RoundRobinDistributionSchedule and each entry has writequorum number of copies. Any violation should be reported. + - If there is any failure in trying to get info. from Bookie of the ensembles of the ledger, then add this ledger to potentially faulty ledgers list (but don't report it yet.) + - (in previous steps, in case of any violation or bookie read error, before reporting violation, check if the ledger is marked underreplicated. If it is marked underreplicated then ignore this ledger for this iteration. If it is not marked underreplicated, then get the ledgermetadata of this ledger onemore time. Check if it is any different from the ledgermetadata we got initially then instead of reporting the violation, redo the analysis for this ledger because apparently something had changed in the metadata (esp. with ensemble) and hence it is better to reevaluate instead of false alarm.) + - if there are potentially faulty ledgers because of unavailable/unreachable bookies, then schedule a new durability check task with time delay just for the potentially faulty ledgers. Even after subsequent delayed checks, if Auditor failed to get response from bookies then make sure that Bookie isn’t registered to metadata server and Auditor is aware of it unavailability, if not then report the violation. + - Auditor is going to use existing mechanisms/frameworks to report the violations - bookkeeper-stats statslogger/counters and complementing information in logs. + - It makes sense to group all the durability violations found in a scrutiny run according to the categories and report the aggregated count for each category after the end of the scrutiny run. + - before reporting these violations, each violation should be logged with complete information, so that it can be used to understand what went wrong. ### Public Interfaces From 21fe90ba181a1fb27554719364ec495f4c72db6b Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Sun, 11 Nov 2018 13:30:15 -0800 Subject: [PATCH 0113/1642] Fix bugs in DefaultEnsemblePlacementPolicy Descriptions of the changes in this PR: - bookieInfoMap is not initialized and newEnsemble will throws BKNotEnoughBookiesException if diskWeightBasedPlacement is enabled - add test coverage for DefaultEnsemblePlacementPolicy with diskWeightBasedPlacement enabled Reviewers: Sijie Guo , Andrey Yegorov This closes #1788 from reddycharan/defaultplacementfix --- .../client/DefaultEnsemblePlacementPolicy.java | 5 +++++ .../client/GenericEnsemblePlacementPolicyTest.java | 14 +++++++++++++- 2 files changed, 18 insertions(+), 1 deletion(-) 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 28efe661545..917d18f7178 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 @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -58,6 +59,7 @@ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy { private final ReentrantReadWriteLock rwLock; DefaultEnsemblePlacementPolicy() { + bookieInfoMap = new HashMap(); rwLock = new ReentrantReadWriteLock(); } @@ -92,6 +94,9 @@ public ArrayList newEnsemble(int ensembleSize, int quorumSi } newBookies.add(b); --ensembleSize; + if (ensembleSize == 0) { + return newBookies; + } } } finally { rwLock.readLock().unlock(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java index bf369cf08ba..8fbb009d240 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java @@ -24,6 +24,8 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -33,10 +35,14 @@ import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; /** * Testing a generic ensemble placement policy. */ +@RunWith(Parameterized.class) public class GenericEnsemblePlacementPolicyTest extends BookKeeperClusterTestCase { private BookKeeper.DigestType digestType = BookKeeper.DigestType.CRC32; @@ -46,9 +52,15 @@ public class GenericEnsemblePlacementPolicyTest extends BookKeeperClusterTestCas private static List> customMetadataOnNewEnsembleStack = new ArrayList<>(); private static List> customMetadataOnReplaceBookieStack = new ArrayList<>(); - public GenericEnsemblePlacementPolicyTest() { + @Parameters + public static Collection getDiskWeightBasedPlacementEnabled() { + return Arrays.asList(new Object[][] { { false }, { true } }); + } + + public GenericEnsemblePlacementPolicyTest(boolean diskWeightBasedPlacementEnabled) { super(0); baseClientConf.setEnsemblePlacementPolicy(CustomEnsemblePlacementPolicy.class); + baseClientConf.setDiskWeightBasedPlacementEnabled(diskWeightBasedPlacementEnabled); } /** From fa4b852605f355b5676a6301721d850700040bc0 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 12 Nov 2018 04:42:59 -0800 Subject: [PATCH 0114/1642] ISSUE #1799: bkctl is broken with default build options Descriptions of the changes in this PR: *Motivation* `bkctl` is designed in a modularized way for extensibility. it loads command groups via ServiceLoader. However current build profile doesn't leverage this extensibility. Instead it hardcodes all the commands in one service load file. So if `bkctl` is built without `-Dstream`, it will broke. *Changes* - move `ledger` related commands to `tools/ledger` - generate the service load file by concating `commands` files from each module's resources directory. *Results* bkctl works for all build profiles. - without `-Dstream` ``` $ bin/bkctl bkctl interacts and operates Apache BookKeeper clusters Usage: bkctl [flags] [command group] [commands] Commands: bookie Commands on operating a single bookie bookies Commands on operating a cluster of bookies cookie Commands on operating cookies ledger Commands on interacting with ledgers help Display help information about it Flags: -c, --conf Configuration file -n, --namespace Namespace scope to run commands (only valid for table service for now) -u, --service-uri Service Uri -h, --help Display help information Use "bkctl [command] --help" or "bkctl help [command]" for more information about a command ``` - with `-Dstream` ``` $ bin/bkctl bkctl interacts and operates Apache BookKeeper clusters Usage: bkctl [flags] [command group] [commands] Commands: bookie Commands on operating a single bookie bookies Commands on operating a cluster of bookies cluster Commands on administrating bookkeeper clusters cookie Commands on operating cookies ledger Commands on interacting with ledgers namespace Commands on operating namespaces table Commands on interacting with tables tables Commands on operating tables help Display help information about it Flags: -c, --conf Configuration file -n, --namespace Namespace scope to run commands (only valid for table service for now) -u, --service-uri Service Uri -h, --help Display help information Use "bkctl [command] --help" or "bkctl help [command]" for more information about a command ``` Master Issue: #1799 Reviewers: Ivan Kelly , Enrico Olivelli This closes #1800 from sijie/refactor_bkctl, closes #1799 --- .gitignore | 3 + .../latest/getting-started/installation.md | 10 +- tools/all/pom.xml | 111 ++++++++++++------ tools/ledger/pom.xml | 57 +++++++++ .../apache/bookkeeper/tools/cli/BKCtl.java | 0 .../cli/commands/BookieCommandGroup.java | 0 .../cli/commands/BookiesCommandGroup.java | 0 .../cli/commands/CookieCommandGroup.java | 0 .../cli/commands/LedgerCommandGroup.java | 0 .../tools/cli/commands/package-info.java | 0 .../bookkeeper/tools/cli/package-info.java | 0 .../src/main/resources/commands} | 4 - .../commands/bookie/LastMarkCommandTest.java | 0 .../bookies/ListBookiesCommandTest.java | 0 .../client/SimpleTestCommandTest.java | 0 .../cookie/CreateCookieCommandTest.java | 0 .../cookie/DeleteCookieCommandTest.java | 0 .../cookie/GenerateCookieCommandTest.java | 0 .../commands/cookie/GetCookieCommandTest.java | 0 .../cookie/UpdateCookieCommandTest.java | 0 .../cli/helpers/BookieCommandTestBase.java | 0 .../cli/helpers/BookieShellCommandTest.java | 0 .../tools/cli/helpers/ClientCommandTest.java | 0 .../cli/helpers/ClientCommandTestBase.java | 0 .../tools/cli/helpers/CommandTestBase.java | 0 .../cli/helpers/CookieCommandTestBase.java | 0 .../cli/helpers/DiscoveryCommandTest.java | 0 .../cli/helpers/DiscoveryCommandTestBase.java | 0 .../src/test/resources/log4j.properties | 0 tools/pom.xml | 1 + tools/stream/src/main/resources/commands | 22 ++++ 31 files changed, 164 insertions(+), 44 deletions(-) create mode 100644 tools/ledger/pom.xml rename tools/{all => ledger}/src/main/java/org/apache/bookkeeper/tools/cli/BKCtl.java (100%) rename tools/{all => ledger}/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java (100%) rename tools/{all => ledger}/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java (100%) rename tools/{all => ledger}/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java (100%) rename tools/{all => ledger}/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java (100%) rename tools/{all => ledger}/src/main/java/org/apache/bookkeeper/tools/cli/commands/package-info.java (100%) rename tools/{all => ledger}/src/main/java/org/apache/bookkeeper/tools/cli/package-info.java (100%) rename tools/{all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup => ledger/src/main/resources/commands} (82%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommandTestBase.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CookieCommandTestBase.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java (100%) rename tools/{all => ledger}/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTestBase.java (100%) rename tools/{all => ledger}/src/test/resources/log4j.properties (100%) create mode 100644 tools/stream/src/main/resources/commands diff --git a/.gitignore b/.gitignore index d8778b2a75a..86f2415acf0 100644 --- a/.gitignore +++ b/.gitignore @@ -33,5 +33,8 @@ data/ # Pid files **/*.pid +# files are generated under following directories +tools/all/src/main/resources + # Exclude versionBackup file (generated by `mvn versions:set`) **/*.versionsBackup diff --git a/site/docs/latest/getting-started/installation.md b/site/docs/latest/getting-started/installation.md index 9986cd8e043..7e92b471581 100644 --- a/site/docs/latest/getting-started/installation.md +++ b/site/docs/latest/getting-started/installation.md @@ -44,6 +44,12 @@ Once you have the BookKeeper on your local machine, either by [downloading](#dow $ mvn package ``` +Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. + +```shell +$ mvn package -Dstream +``` + > You can skip tests by adding the `-DskipTests` flag when running `mvn package`. ### Useful Maven commands @@ -60,7 +66,9 @@ Command | Action `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 +`mvn -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin + +> You can enable `table service` by adding the `-Dstream` flag when running above commands. ## Package directory diff --git a/tools/all/pom.xml b/tools/all/pom.xml index 685c2e1103b..9a29f6099f8 100644 --- a/tools/all/pom.xml +++ b/tools/all/pom.xml @@ -1,20 +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. + --> 4.0.0 @@ -27,33 +29,37 @@ org.apache.bookkeeper - bookkeeper-tools-framework + bookkeeper-tools-ledger ${project.version} - - org.apache.bookkeeper - bookkeeper-server - ${project.parent.version} - - - org.slf4j - slf4j-log4j12 - runtime - - - org.apache.bookkeeper - buildtools - ${project.parent.version} - test - - - org.apache.bookkeeper - bookkeeper-server - test-jar - ${project.parent.version} - test - + + + + org.apache.maven.plugins + maven-antrun-plugin + ${maven-antrun-plugin.version} + + + append-ledger-commands + generate-resources + + run + + + + + + + + + + + + + + stream @@ -70,6 +76,33 @@ ${project.version} + + + + org.apache.maven.plugins + maven-antrun-plugin + ${maven-antrun-plugin.version} + + + append-stream-commands + generate-resources + + run + + + + + + + + + + + + + + diff --git a/tools/ledger/pom.xml b/tools/ledger/pom.xml new file mode 100644 index 00000000000..6c5182b211e --- /dev/null +++ b/tools/ledger/pom.xml @@ -0,0 +1,57 @@ + + + + 4.0.0 + + bookkeeper-tools-parent + org.apache.bookkeeper + 4.9.0-SNAPSHOT + + bookkeeper-tools-ledger + Apache BookKeeper :: Tools :: Ledger + + + org.apache.bookkeeper + bookkeeper-tools-framework + ${project.version} + + + org.apache.bookkeeper + bookkeeper-server + ${project.parent.version} + + + org.slf4j + slf4j-log4j12 + runtime + + + org.apache.bookkeeper + buildtools + ${project.parent.version} + test + + + org.apache.bookkeeper + bookkeeper-server + test-jar + ${project.parent.version} + test + + + diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/BKCtl.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/BKCtl.java similarity index 100% rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/BKCtl.java rename to tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/BKCtl.java diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java similarity index 100% rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java rename to tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java similarity index 100% rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java rename to tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java similarity index 100% rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java rename to tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java similarity index 100% rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java rename to tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/package-info.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/package-info.java similarity index 100% rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/commands/package-info.java rename to tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/package-info.java diff --git a/tools/all/src/main/java/org/apache/bookkeeper/tools/cli/package-info.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/package-info.java similarity index 100% rename from tools/all/src/main/java/org/apache/bookkeeper/tools/cli/package-info.java rename to tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/package-info.java diff --git a/tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup b/tools/ledger/src/main/resources/commands similarity index 82% rename from tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup rename to tools/ledger/src/main/resources/commands index 0147eca5e6b..7ea146b7825 100644 --- a/tools/all/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup +++ b/tools/ledger/src/main/resources/commands @@ -20,7 +20,3 @@ org.apache.bookkeeper.tools.cli.commands.BookieCommandGroup org.apache.bookkeeper.tools.cli.commands.BookiesCommandGroup org.apache.bookkeeper.tools.cli.commands.CookieCommandGroup org.apache.bookkeeper.tools.cli.commands.LedgerCommandGroup -org.apache.bookkeeper.stream.cli.ClusterCommandGroup -org.apache.bookkeeper.stream.cli.NamespaceCommandGroup -org.apache.bookkeeper.stream.cli.TableAdminCommandGroup -org.apache.bookkeeper.stream.cli.TableCommandGroup diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/SimpleTestCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommandTestBase.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommandTestBase.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommandTestBase.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieCommandTestBase.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/BookieShellCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/ClientCommandTestBase.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CommandTestBase.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CookieCommandTestBase.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CookieCommandTestBase.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CookieCommandTestBase.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/CookieCommandTestBase.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTest.java diff --git a/tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTestBase.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTestBase.java similarity index 100% rename from tools/all/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTestBase.java rename to tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/helpers/DiscoveryCommandTestBase.java diff --git a/tools/all/src/test/resources/log4j.properties b/tools/ledger/src/test/resources/log4j.properties similarity index 100% rename from tools/all/src/test/resources/log4j.properties rename to tools/ledger/src/test/resources/log4j.properties diff --git a/tools/pom.xml b/tools/pom.xml index 2981bbb0221..c538a6a9e0f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,6 +27,7 @@ pom framework + ledger all diff --git a/tools/stream/src/main/resources/commands b/tools/stream/src/main/resources/commands new file mode 100644 index 00000000000..070efbfe44c --- /dev/null +++ b/tools/stream/src/main/resources/commands @@ -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. +# + +org.apache.bookkeeper.stream.cli.ClusterCommandGroup +org.apache.bookkeeper.stream.cli.NamespaceCommandGroup +org.apache.bookkeeper.stream.cli.TableAdminCommandGroup +org.apache.bookkeeper.stream.cli.TableCommandGroup From 5cc3fdd570cc3b0950ac1cb1c7d7921845ee49c9 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 12 Nov 2018 04:54:34 -0800 Subject: [PATCH 0115/1642] [tools] package bookkeeper tools into a separated distribution MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Descriptions of the changes in this PR: *Motivation* Server distribution contains a lot of stuffs that doesn’t need to run bkctl and bookkeeper shell, such as stats providers, http servers and many other dependencies. At many environments, you want to deliver a set of tools to your customers to interact with or operate bookkeeper clusters. They don’t care about server side stuffs and ask them to use a server package to run a tool makes things confused. *changes* Create a new ‘bkctl’ package that only contains bookkeeper shell and new cli. Reviewers: Charan Reddy Guttapalem , Enrico Olivelli , Matteo Merli This closes #1798 from sijie/package_bookkeeper_tool --- .travis_scripts/build.sh | 1 + bin/common.sh | 2 +- bookkeeper-dist/bkctl/pom.xml | 88 ++++ bookkeeper-dist/pom.xml | 1 + bookkeeper-dist/src/assemble/bkctl.xml | 116 ++++++ .../src/main/resources/LICENSE-bkctl.bin.txt | 385 ++++++++++++++++++ .../src/main/resources/NOTICE-bkctl.bin.txt | 97 +++++ 7 files changed, 689 insertions(+), 1 deletion(-) create mode 100644 bookkeeper-dist/bkctl/pom.xml create mode 100644 bookkeeper-dist/src/assemble/bkctl.xml create mode 100644 bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt create mode 100644 bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt diff --git a/.travis_scripts/build.sh b/.travis_scripts/build.sh index 49e33a9aea9..dcfc3e70634 100755 --- a/.travis_scripts/build.sh +++ b/.travis_scripts/build.sh @@ -30,6 +30,7 @@ else mvn --batch-mode clean apache-rat:check compile spotbugs:check install -DskipTests -Dstream $BK_HOME/dev/check-binary-license ./bookkeeper-dist/all/target/bookkeeper-all-*-bin.tar.gz; $BK_HOME/dev/check-binary-license ./bookkeeper-dist/server/target/bookkeeper-server-*-bin.tar.gz; + $BK_HOME/dev/check-binary-license ./bookkeeper-dist/bkctl/target/bkctl-*-bin.tar.gz; if [ "$DLOG_MODIFIED" == "true" ]; then cd $BK_HOME/stream/distributedlog mvn --batch-mode clean package -Ddistributedlog diff --git a/bin/common.sh b/bin/common.sh index d54b1d52ce7..73de28f7b8a 100755 --- a/bin/common.sh +++ b/bin/common.sh @@ -117,7 +117,7 @@ CLI_GC_LOGGING_OPTS=${CLI_GC_LOGGING_OPTS:-"${DEFAULT_CLI_GC_LOGGING_OPTS}"} find_module_jar_at() { DIR=$1 MODULE=$2 - REGEX="^${MODULE}-[0-9\\.]*(-SNAPSHOT)?.jar$" + REGEX="^${MODULE}-[0-9\\.]*((-[a-zA-Z]*(-[0-9]*)?)|(-SNAPSHOT))?.jar$" if [ -d ${DIR} ]; then cd ${DIR} for f in *.jar; do diff --git a/bookkeeper-dist/bkctl/pom.xml b/bookkeeper-dist/bkctl/pom.xml new file mode 100644 index 00000000000..e581865d71b --- /dev/null +++ b/bookkeeper-dist/bkctl/pom.xml @@ -0,0 +1,88 @@ + + + 4.0.0 + + bookkeeper-dist + org.apache.bookkeeper + 4.9.0-SNAPSHOT + .. + + + bkctl + jar + Apache BookKeeper :: Dist (Bkctl) + + + + + org.apache.bookkeeper + bookkeeper-tools + ${project.version} + + + org.rocksdb + rocksdbjni + + + + + + + org.slf4j + slf4j-log4j12 + + + + + + + maven-assembly-plugin + ${maven-assembly-plugin.version} + + bkctl-${project.version} + true + + ../src/assemble/bkctl.xml + + posix + + + + package + + single + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + ${maven-deploy-plugin.version} + + true + + + + + diff --git a/bookkeeper-dist/pom.xml b/bookkeeper-dist/pom.xml index 7ae8c387f59..b81fe60d426 100644 --- a/bookkeeper-dist/pom.xml +++ b/bookkeeper-dist/pom.xml @@ -28,6 +28,7 @@ all server + bkctl UTF-8 diff --git a/bookkeeper-dist/src/assemble/bkctl.xml b/bookkeeper-dist/src/assemble/bkctl.xml new file mode 100644 index 00000000000..439c344d700 --- /dev/null +++ b/bookkeeper-dist/src/assemble/bkctl.xml @@ -0,0 +1,116 @@ + + + bin + + tar.gz + + true + + + target + + + ${project.artifactId}-${project.version}.jar + + + + ../../conf + conf + + bk_cli_env.sh + bk_server.conf + bkenv.sh + jaas_example.conf + log4j.cli.properties + log4j.shell.properties + nettyenv.sh + + + + ../../bin + 755 + bin + + bkctl + bkperf + bookkeeper + common.sh + + + + 644 + + ${basedir}/*.txt + + + + ../src/main/resources/deps + deps + + google-auth-library-credentials-0.9.0/LICENSE + netty-4.1.31.Final/* + protobuf-3.0.0/LICENSE + jline-2.11/LICENSE + protobuf-3.5.1/LICENSE + slf4j-1.7.25/LICENSE.txt + + 644 + + + + + ../../README.md + + 644 + + + ../src/main/resources/LICENSE-bkctl.bin.txt + + LICENSE + 644 + + + ../src/main/resources/NOTICE-bkctl.bin.txt + + NOTICE + 644 + + + + + lib + false + runtime + false + + ${artifact.groupId}-${artifact.artifactId}-${artifact.version}${dashClassifier?}.${artifact.extension} + + com.google.code.findbugs:jsr305 + + org.apache.bookkeeper:stream-storage-common + org.apache.bookkeeper:stream-storage-proto + org.apache.bookkeeper:stream-storage-api + org.apache.bookkeeper:stream-storage-java-client-base + + org.rocksdb:rocksdbjni + + + + diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt new file mode 100644 index 00000000000..486a35ab1db --- /dev/null +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -0,0 +1,385 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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 following bundled 3rd party jars are distributed under the +Apache Software License, Version 2. + +- lib/com.fasterxml.jackson.core-jackson-annotations-2.9.7.jar [1] +- lib/com.fasterxml.jackson.core-jackson-core-2.9.7.jar [2] +- lib/com.fasterxml.jackson.core-jackson-databind-2.9.7.jar [3] +- lib/com.google.guava-guava-21.0.jar [4] +- lib/commons-cli-commons-cli-1.2.jar [5] +- lib/commons-codec-commons-codec-1.6.jar [6] +- lib/commons-configuration-commons-configuration-1.10.jar [7] +- lib/commons-io-commons-io-2.4.jar [8] +- lib/commons-lang-commons-lang-2.6.jar [9] +- lib/commons-logging-commons-logging-1.1.1.jar [10] +- lib/io.netty-netty-buffer-4.1.31.Final.jar [11] +- lib/io.netty-netty-codec-4.1.31.Final.jar [11] +- lib/io.netty-netty-codec-http-4.1.31.Final.jar [11] +- lib/io.netty-netty-codec-http2-4.1.31.Final.jar [11] +- lib/io.netty-netty-codec-socks-4.1.31.Final.jar [11] +- lib/io.netty-netty-common-4.1.31.Final.jar [11] +- lib/io.netty-netty-handler-4.1.31.Final.jar [11] +- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar [11] +- lib/io.netty-netty-resolver-4.1.31.Final.jar [11] +- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar [11] +- lib/io.netty-netty-transport-4.1.31.Final.jar [11] +- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar [11] +- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar [11] +- lib/log4j-log4j-1.2.17.jar [16] +- lib/net.java.dev.jna-jna-3.2.7.jar [17] +- lib/org.apache.commons-commons-collections4-4.1.jar [18] +- lib/org.apache.commons-commons-lang3-3.6.jar [19] +- lib/org.apache.zookeeper-zookeeper-3.4.13.jar [20] +- lib/com.beust-jcommander-1.48.jar [23] +- lib/net.jpountz.lz4-lz4-1.3.0.jar [25] +- lib/com.google.api.grpc-proto-google-common-protos-1.0.0.jar [27] +- lib/com.google.code.gson-gson-2.7.jar [28] +- lib/io.opencensus-opencensus-api-0.11.0.jar [29] +- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.11.0.jar [29] +- lib/com.squareup.okhttp-okhttp-2.5.0.jar [30] +- lib/com.squareup.okio-okio-1.13.0.jar [31] +- lib/io.grpc-grpc-all-1.12.0.jar [32] +- lib/io.grpc-grpc-auth-1.12.0.jar [32] +- lib/io.grpc-grpc-context-1.12.0.jar [32] +- lib/io.grpc-grpc-core-1.12.0.jar [32] +- lib/io.grpc-grpc-netty-1.12.0.jar [32] +- lib/io.grpc-grpc-okhttp-1.12.0.jar [32] +- lib/io.grpc-grpc-protobuf-1.12.0.jar [32] +- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar [32] +- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar [32] +- lib/io.grpc-grpc-stub-1.12.0.jar [32] +- lib/io.grpc-grpc-testing-1.12.0.jar [32] +- lib/org.apache.curator-curator-client-4.0.1.jar [33] +- lib/org.apache.curator-curator-framework-4.0.1.jar [33] +- lib/org.apache.curator-curator-recipes-4.0.1.jar [33] +- lib/org.inferred-freebuilder-1.14.9.jar [34] +- lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [35] +- lib/org.apache.yetus-audience-annotations-0.5.0.jar [36] + +[1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9 +[2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9 +[3] Source available at https://github.com/FasterXML/jackson-databind/tree/jackson-databind-2.9.7 +[4] Source available at https://github.com/google/guava/tree/v21.0 +[5] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-cli.git;a=tag;h=bc8f0e +[6] Source available at http://svn.apache.org/viewvc/commons/proper/codec/tags/1_6/ +[7] Source available at http://svn.apache.org/viewvc/commons/proper/configuration/tags/CONFIGURATION_1_10/ +[8] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-io.git;a=tag;h=603579 +[9] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=tag;h=375459 +[10] Source available at http://svn.apache.org/viewvc/commons/proper/logging/tags/commons-logging-1.1.1/ +[11] Source available at https://github.com/netty/netty/tree/netty-4.1.31.Final +[16] Source available at http://logging.apache.org/log4j/1.2/download.html +[17] Source available at https://github.com/java-native-access/jna/tree/3.2.7 +[18] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad +[19] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=shortlog;h=refs/tags/LANG_3_6 +[20] Source available at https://github.com/apache/zookeeper/tree/release-3.4.13 +[23] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48 +[25] Source available at https://github.com/lz4/lz4-java/tree/1.3.0 +[27] Source available at https://github.com/googleapis/googleapis +[28] Source available at https://github.com/google/gson/tree/gson-parent-2.7 +[29] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.11.0 +[30] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 +[31] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 +[32] Source available at https://github.com/grpc/grpc-java/tree/v1.12.0 +[33] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 +[34] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 +[35] Source available at https://github.com/google/error-prone/tree/v2.1.2 +[36] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 + +------------------------------------------------------------------------------------ +lib/io.netty-netty-codec-4.1.31.Final.jar bundles some 3rd party dependencies + +lib/io.netty-netty-codec-4.1.31.Final.jar contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * deps/netty-4.1.31.Final/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * deps/netty-4.1.31.Final/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +lib/io.netty-netty-common-4.1.31.Final.jar contains a modified portion of 'SLF4J', a simple logging +facade for Java, which can be obtained at: + + * LICENSE: + * deps/netty-4.1.31.Final/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression +and decompression library written by Matthew J. Francis. It can be obtained at: + + * LICENSE: + * deps/netty-4.1.31.Final/LICENSE.jbzip2.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jbzip2/ + +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct +the suffix array and the Burrows-Wheeler transformed string for any input string of +a constant-size alphabet written by Yuta Mori. It can be obtained at: + + * LICENSE: + * deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt (MIT License) + * HOMEPAGE: + * https://github.com/y-256/libdivsufsort + +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression +and decompression library written by William Kinney. It can be obtained at: + + * LICENSE: + * deps/netty-4.1.31.Final/LICENSE.jfastlz.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jfastlz/ + +lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data +interchange format, which can be obtained at: + + * LICENSE: + * deps/netty-4.1.31.Final/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * https://github.com/google/protobuf + +------------------------------------------------------------------------------------ +This product bundles Google Protocol Buffers, which is available under a "3-clause BSD" +license. + +Bundled as + - lib/com.google.protobuf-protobuf-java-3.5.1.jar + - lib/com.google.protobuf-protobuf-java-util-3.5.1.jar +Source available at https://github.com/google/protobuf/tree/v3.5.1 +For details, see deps/protobuf-3.5.1/LICENSE. + +Bundled as lib/com.google.protobuf.nano-protobuf-javanano-3.0.0-alpha-5.jar +Source available at https://github.com/google/protobuf/tree/3.0.0-pre +For details, see deps/protobuf-3.0.0/LICENSE. +------------------------------------------------------------------------------------ +This product bundles Simple Logging Facade for Java, which is available under a +MIT license. For details, see deps/slf4j-1.7.25/LICENSE.txt. + +Bundled as + - lib/org.slf4j-slf4j-api-1.7.25.jar + - lib/org.slf4j-slf4j-log4j12-1.7.25.jar +Source available at https://github.com/qos-ch/slf4j/tree/v_1.7.25 +------------------------------------------------------------------------------------ +This product bundles the Google Auth Library, which is available under a "3-clause BSD" +license. For details, see deps/google-auth-library-credentials-0.9.0/LICENSE + +Bundled as + - lib/com.google.auth-google-auth-library-credentials-0.9.0.jar +Source available at https://github.com/google/google-auth-library-java/tree/0.9.0 +------------------------------------------------------------------------------------ +This product bundles the JLine Library, which is available under a "2-clause BSD" +license. For details, see deps/jline-2.11/LICENSE + +Bundled as + - lib/jline-jline-2.11.jar diff --git a/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt new file mode 100644 index 00000000000..680a02dca82 --- /dev/null +++ b/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt @@ -0,0 +1,97 @@ +Apache BookKeeper +Copyright 2011-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +------------------------------------------------------------------------------------ +- lib/io.netty-netty-buffer-4.1.31.Final.jar +- lib/io.netty-netty-codec-4.1.31.Final.jar +- lib/io.netty-netty-codec-http-4.1.31.Final.jar +- lib/io.netty-netty-codec-http2-4.1.31.Final.jar +- lib/io.netty-netty-codec-socks-4.1.31.Final.jar +- lib/io.netty-netty-common-4.1.31.Final.jar +- lib/io.netty-netty-handler-4.1.31.Final.jar +- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar +- lib/io.netty-netty-resolver-4.1.31.Final.jar +- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar +- lib/io.netty-netty-transport-4.1.31.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar +- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar + + + The Netty Project + ================= + +Please visit the Netty web site for more information: + + * http://netty.io/ + +Copyright 2014 The Netty Project + +The Netty Project 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. + +------------------------------------------------------------------------------------ +- lib/com.beust-jcommander-1.48.jar + +Copyright 2010 Cedric Beust cedric@beust.com + +------------------------------------------------------------------------------------ +- lib/io.grpc-grpc-all-1.12.0.jar +- lib/io.grpc-grpc-auth-1.12.0.jar +- lib/io.grpc-grpc-context-1.12.0.jar +- lib/io.grpc-grpc-core-1.12.0.jar +- lib/io.grpc-grpc-netty-1.12.0.jar +- lib/io.grpc-grpc-okhttp-1.12.0.jar +- lib/io.grpc-grpc-protobuf-1.12.0.jar +- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar +- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar +- lib/io.grpc-grpc-stub-1.12.0.jar +- lib/io.grpc-grpc-testing-1.12.0.jar + +Copyright 2014, gRPC Authors All rights reserved. + +Licensed 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. + +This product contains a modified portion of 'OkHttp', an open source +HTTP & SPDY client for Android and Java applications, which can be obtained +at: + + * LICENSE: + * okhttp/third_party/okhttp/LICENSE (Apache License 2.0) + * HOMEPAGE: + * https://github.com/square/okhttp + * LOCATION_IN_GRPC: + * okhttp/third_party/okhttp + +This product contains a modified portion of 'Netty', an open source +networking library, which can be obtained at: + + * LICENSE: + * netty/third_party/netty/LICENSE.txt (Apache License 2.0) + * HOMEPAGE: + * https://netty.io + * LOCATION_IN_GRPC: + * netty/third_party/netty + +------------------------------------------------------------------------------------ From 6aa56a72df02f941acd182cca134ecc859d1dde7 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Mon, 12 Nov 2018 21:11:59 +0100 Subject: [PATCH 0116/1642] Move version out of LedgerMetadata The version is a property of the metadata store and not of the LedgerMetadata object itself. Putting it in the LedgerMetadata forces that LedgerMetadata to be mutable, as it needs to be updated before being sent to the LedgerManager. This change moves version out of LedgerMetadata and modifies the LedgerManager to use Versioned. This is a large change, but almost purely mechanical. TestHttpService has been modified to remove a preexisting flake in a test which had to be modified for this change. Master issue: #281 Reviewers: Enrico Olivelli , Sijie Guo This closes #1797 from ivankelly/kill-set-version --- .../apache/bookkeeper/bookie/BookieShell.java | 57 ++++---- .../ScanAndCompareGarbageCollector.java | 11 +- .../apache/bookkeeper/client/BookKeeper.java | 7 +- .../bookkeeper/client/BookKeeperAdmin.java | 15 ++- .../bookkeeper/client/LedgerCreateOp.java | 9 +- .../client/LedgerFragmentReplicator.java | 2 +- .../bookkeeper/client/LedgerHandle.java | 49 ++++--- .../bookkeeper/client/LedgerHandleAdv.java | 3 +- .../bookkeeper/client/LedgerMetadata.java | 46 +------ .../client/LedgerMetadataBuilder.java | 46 ++++--- .../bookkeeper/client/LedgerOpenOp.java | 17 ++- .../bookkeeper/client/MetadataUpdateLoop.java | 35 ++--- .../client/ReadOnlyLedgerHandle.java | 30 ++--- .../bookkeeper/client/UpdateLedgerOp.java | 7 +- .../meta/AbstractZkLedgerManager.java | 38 +++--- .../bookkeeper/meta/CleanupLedgerManager.java | 16 ++- .../apache/bookkeeper/meta/LedgerManager.java | 11 +- .../meta/MSLedgerManagerFactory.java | 43 +++--- .../proto/BookkeeperInternalCallbacks.java | 3 +- .../replication/BookieLedgerIndexer.java | 39 +++--- .../http/service/ListLedgerService.java | 7 +- .../bookkeeper/bookie/CompactionTest.java | 8 +- .../bookie/TestGcOverreplicatedLedger.java | 19 +-- .../bookkeeper/client/BookieRecoveryTest.java | 7 +- .../apache/bookkeeper/client/ClientUtil.java | 39 ++++-- .../bookkeeper/client/HandleFailuresTest.java | 31 ++--- .../bookkeeper/client/LedgerClose2Test.java | 16 +-- .../client/LedgerRecovery2Test.java | 37 +++-- .../client/MetadataUpdateLoopTest.java | 127 ++++++++++-------- .../client/MockBookKeeperTestCase.java | 15 ++- .../bookkeeper/client/MockLedgerHandle.java | 6 +- .../client/ParallelLedgerRecoveryTest.java | 31 ++--- .../client/TestLedgerFragmentReplication.java | 6 +- .../bookkeeper/client/TestSequenceRead.java | 57 ++------ .../client/TestWatchEnsembleChange.java | 7 +- .../meta/AbstractZkLedgerManagerTest.java | 97 +++++-------- .../apache/bookkeeper/meta/GcLedgersTest.java | 9 +- .../meta/LedgerManagerIteratorTest.java | 5 +- .../bookkeeper/meta/MockLedgerManager.java | 23 ++-- .../AuditorPeriodicBookieCheckTest.java | 32 ++--- .../server/http/TestHttpService.java | 35 ++--- .../metadata/etcd/EtcdLedgerManager.java | 32 ++--- .../metadata/etcd/LedgerMetadataConsumer.java | 6 +- .../metadata/etcd/EtcdLedgerManagerTest.java | 75 +++++------ .../TestCompatRecoveryNoPassword.groovy | 5 +- 45 files changed, 574 insertions(+), 642 deletions(-) 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 e0e79c2de1f..df07d61522f 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 @@ -1030,33 +1030,30 @@ public void process(Long ledgerId, VoidCallback cb) { printLedgerMetadata(ledgerId, null, false); cb.processResult(BKException.Code.OK, null, null); } else { - GenericCallback gencb = new GenericCallback() { - @Override - public void operationComplete(int rc, LedgerMetadata ledgerMetadata) { - if (rc == BKException.Code.OK) { - if ((bookieAddress == null) - || BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie(ledgerId, - bookieAddress, ledgerMetadata)) { - /* - * the print method has to be in - * synchronized scope, otherwise - * output of printLedgerMetadata - * could interleave since this - * callback for different - * ledgers can happen in - * different threads. - */ - synchronized (BookieShell.this) { - printLedgerMetadata(ledgerId, ledgerMetadata, printMeta); - } + GenericCallback> gencb = (rc, ledgerMetadata) -> { + if (rc == BKException.Code.OK) { + if ((bookieAddress == null) + || BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie( + ledgerId, bookieAddress, ledgerMetadata.getValue())) { + /* + * the print method has to be in + * synchronized scope, otherwise + * output of printLedgerMetadata + * could interleave since this + * callback for different + * ledgers can happen in + * different threads. + */ + synchronized (BookieShell.this) { + printLedgerMetadata(ledgerId, ledgerMetadata.getValue(), printMeta); } - } else if (rc == BKException.Code.NoSuchLedgerExistsException) { - rc = BKException.Code.OK; - } else { - LOG.error("Unable to read the ledger: " + ledgerId + " information"); } - cb.processResult(rc, null, null); + } else if (rc == BKException.Code.NoSuchLedgerExistsException) { + rc = BKException.Code.OK; + } else { + LOG.error("Unable to read the ledger: " + ledgerId + " information"); } + cb.processResult(rc, null, null); }; ledgerManager.readLedgerMetadata(ledgerId, gencb); } @@ -1138,21 +1135,21 @@ public int runCmd(CommandLine cmdLine) throws Exception { runFunctionWithLedgerManagerFactory(bkConf, mFactory -> { try (LedgerManager m = mFactory.newLedgerManager()) { if (cmdLine.hasOption("dumptofile")) { - GenericCallbackFuture cb = new GenericCallbackFuture<>(); + GenericCallbackFuture> cb = new GenericCallbackFuture<>(); m.readLedgerMetadata(lid, cb); Files.write(FileSystems.getDefault().getPath(cmdLine.getOptionValue("dumptofile")), - cb.join().serialize()); + cb.join().getValue().serialize()); } else if (cmdLine.hasOption("restorefromfile")) { byte[] serialized = Files.readAllBytes( FileSystems.getDefault().getPath(cmdLine.getOptionValue("restorefromfile"))); - LedgerMetadata md = LedgerMetadata.parseConfig(serialized, Version.NEW, Optional.absent()); - GenericCallbackFuture cb = new GenericCallbackFuture<>(); + LedgerMetadata md = LedgerMetadata.parseConfig(serialized, Optional.absent()); + GenericCallbackFuture> cb = new GenericCallbackFuture<>(); m.createLedgerMetadata(lid, md, cb); cb.join(); } else { - GenericCallbackFuture cb = new GenericCallbackFuture<>(); + GenericCallbackFuture> cb = new GenericCallbackFuture<>(); m.readLedgerMetadata(lid, cb); - printLedgerMetadata(lid, cb.get(), true); + printLedgerMetadata(lid, cb.get().getValue(), true); } } catch (Exception e) { throw new UncheckedExecutionException(e); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index 33f10e901a4..19d630ff75d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -49,6 +49,7 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.ZkUtils; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; @@ -183,7 +184,7 @@ public void gc(GarbageCleaner garbageCleaner) { if (verifyMetadataOnGc) { CountDownLatch latch = new CountDownLatch(1); final AtomicInteger metaRC = new AtomicInteger(0); - ledgerManager.readLedgerMetadata(bkLid, (int rc, LedgerMetadata x) -> { + ledgerManager.readLedgerMetadata(bkLid, (rc, ignore) -> { metaRC.set(rc); latch.countDown(); }); @@ -236,19 +237,19 @@ private Set removeOverReplicatedledgers(Set bkActiveledgers, final G ZkLedgerUnderreplicationManager.acquireUnderreplicatedLedgerLock(zk, zkLedgersRootPath, ledgerId, zkAcls); semaphore.acquire(); - ledgerManager.readLedgerMetadata(ledgerId, new GenericCallback() { + ledgerManager.readLedgerMetadata(ledgerId, new GenericCallback>() { @Override - public void operationComplete(int rc, LedgerMetadata ledgerMetadata) { + public void operationComplete(int rc, Versioned ledgerMetadata) { if (rc == BKException.Code.OK) { // do not delete a ledger that is not closed, since the ensemble might change again and // include the current bookie while we are deleting it - if (!ledgerMetadata.isClosed()) { + if (!ledgerMetadata.getValue().isClosed()) { release(); return; } SortedMap> ensembles = - ledgerMetadata.getAllEnsembles(); + ledgerMetadata.getValue().getAllEnsembles(); for (List ensemble : ensembles.values()) { // check if this bookie is supposed to have this ledger if (ensemble.contains(selfBookieAddress)) { 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 d434ffa39e5..4d0fb32ffce 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 @@ -79,6 +79,7 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.bookkeeper.util.SafeRunnable; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.lang.SystemUtils; import org.apache.zookeeper.KeeperException; @@ -1299,11 +1300,11 @@ public void deleteLedger(long lId) throws InterruptedException, BKException { * @param cb callback method */ public void asyncIsClosed(long lId, final IsClosedCallback cb, final Object ctx){ - ledgerManager.readLedgerMetadata(lId, new GenericCallback(){ + ledgerManager.readLedgerMetadata(lId, new GenericCallback>(){ @Override - public void operationComplete(int rc, LedgerMetadata lm){ + public void operationComplete(int rc, Versioned lm){ if (rc == BKException.Code.OK) { - cb.isClosedComplete(rc, lm.isClosed(), ctx); + cb.isClosedComplete(rc, lm.getValue().isClosed(), ctx); } else { cb.isClosedComplete(rc, false, ctx); } 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 f637889be7f..5aa20867c47 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 @@ -82,6 +82,7 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -490,9 +491,9 @@ public void asyncGetLedgersContainBookies(final Set bookies bkc.getLedgerManager().asyncProcessLedgers(new Processor() { @Override public void process(final Long lid, final AsyncCallback.VoidCallback cb) { - bkc.getLedgerManager().readLedgerMetadata(lid, new GenericCallback() { + bkc.getLedgerManager().readLedgerMetadata(lid, new GenericCallback>() { @Override - public void operationComplete(int rc, LedgerMetadata metadata) { + public void operationComplete(int rc, Versioned metadata) { if (BKException.Code.NoSuchLedgerExistsException == rc) { // the ledger was deleted during this iteration. cb.processResult(BKException.Code.OK, null, null); @@ -501,11 +502,11 @@ public void operationComplete(int rc, LedgerMetadata metadata) { cb.processResult(rc, null, null); return; } - Set bookiesInLedger = metadata.getBookiesInThisLedger(); + Set bookiesInLedger = metadata.getValue().getBookiesInThisLedger(); Sets.SetView intersection = Sets.intersection(bookiesInLedger, bookies); if (!intersection.isEmpty()) { - ledgers.put(lid, metadata); + ledgers.put(lid, metadata.getValue()); } cb.processResult(BKException.Code.OK, null, null); } @@ -1637,7 +1638,7 @@ private static boolean areEntriesOfSegmentStoredInTheBookie(LedgerMetadata ledge } static class ReadMetadataCallback extends AbstractFuture - implements GenericCallback { + implements GenericCallback> { final long ledgerId; ReadMetadataCallback(long ledgerId) { @@ -1648,11 +1649,11 @@ long getLedgerId() { return ledgerId; } - public void operationComplete(int rc, LedgerMetadata result) { + public void operationComplete(int rc, Versioned result) { if (rc != 0) { setException(BKException.create(rc)); } else { - set(result); + set(result.getValue()); } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java index e39b2a19b00..b55e7ba9ca3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java @@ -46,6 +46,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.MathUtils; +import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +54,7 @@ * Encapsulates asynchronous ledger create operation. * */ -class LedgerCreateOp implements GenericCallback { +class LedgerCreateOp implements GenericCallback> { static final Logger LOG = LoggerFactory.getLogger(LedgerCreateOp.class); @@ -187,7 +188,7 @@ public void initiateAdv(final long ledgerId) { * Callback when created ledger. */ @Override - public void operationComplete(int rc, LedgerMetadata writtenMetadata) { + public void operationComplete(int rc, Versioned writtenMetadata) { if (this.generateLedgerId && (BKException.Code.LedgerExistException == rc)) { // retry to generate a new ledger id generateLedgerIdAndCreateLedger(); @@ -199,9 +200,9 @@ public void operationComplete(int rc, LedgerMetadata writtenMetadata) { try { if (adv) { - lh = new LedgerHandleAdv(bk.getClientCtx(), ledgerId, metadata, digestType, passwd, writeFlags); + lh = new LedgerHandleAdv(bk.getClientCtx(), ledgerId, writtenMetadata, digestType, passwd, writeFlags); } else { - lh = new LedgerHandle(bk.getClientCtx(), ledgerId, metadata, digestType, passwd, writeFlags); + lh = new LedgerHandle(bk.getClientCtx(), ledgerId, writtenMetadata, digestType, passwd, writeFlags); } } catch (GeneralSecurityException e) { LOG.error("Security exception while creating ledger: " + ledgerId, e); 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 76a9178db61..28929cf9a8d 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 @@ -375,7 +375,7 @@ private static void updateEnsembleInfo( MetadataUpdateLoop updateLoop = new MetadataUpdateLoop( ledgerManager, lh.getId(), - lh::getLedgerMetadata, + lh::getVersionedLedgerMetadata, (metadata) -> { // returns true if any of old bookies exist in ensemble List ensemble = metadata.getAllEnsembles().get(fragmentStartId); 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 12e00e84fb7..dedae6e0cff 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 @@ -75,12 +75,12 @@ import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.SafeRunnable; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.collections4.IteratorUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,7 +95,7 @@ public class LedgerHandle implements WriteHandle { final ClientContext clientCtx; final byte[] ledgerKey; - private LedgerMetadata metadata; + private Versioned versionedMetadata; final long ledgerId; long lastAddPushed; @@ -156,15 +156,17 @@ private enum HandleState { private final OpStatsLogger clientChannelWriteWaitStats; LedgerHandle(ClientContext clientCtx, - long ledgerId, LedgerMetadata metadata, + long ledgerId, Versioned versionedMetadata, BookKeeper.DigestType digestType, byte[] password, EnumSet writeFlags) throws GeneralSecurityException, NumberFormatException { this.clientCtx = clientCtx; - this.metadata = metadata; + this.versionedMetadata = versionedMetadata; this.pendingAddOps = new ConcurrentLinkedQueue(); this.writeFlags = writeFlags; + + LedgerMetadata metadata = versionedMetadata.getValue(); if (metadata.isClosed()) { lastAddConfirmed = lastAddPushed = metadata.getLastEntryId(); length = metadata.getLength(); @@ -190,7 +192,9 @@ private enum HandleState { // password, so that the bookie can avoid processing the keys for each entry this.ledgerKey = DigestManager.generateMasterKey(password); distributionSchedule = new RoundRobinDistributionSchedule( - metadata.getWriteQuorumSize(), metadata.getAckQuorumSize(), metadata.getEnsembleSize()); + metadata.getWriteQuorumSize(), + metadata.getAckQuorumSize(), + metadata.getEnsembleSize()); this.bookieFailureHistory = CacheBuilder.newBuilder() .expireAfterWrite(clientCtx.getConf().bookieFailureHistoryExpirationMSec, TimeUnit.MILLISECONDS) .build(new CacheLoader() { @@ -310,14 +314,19 @@ public byte[] getLedgerKey() { */ @Override public LedgerMetadata getLedgerMetadata() { - return metadata; + return versionedMetadata.getValue(); + } + + Versioned getVersionedLedgerMetadata() { + return versionedMetadata; } - boolean setLedgerMetadata(LedgerMetadata expected, LedgerMetadata newMetadata) { + boolean setLedgerMetadata(Versioned expected, Versioned newMetadata) { synchronized (this) { // ensure that we only update the metadata if it is the object we expect it to be - if (metadata == expected) { - metadata = newMetadata; + if (versionedMetadata == expected) { + versionedMetadata = newMetadata; + LedgerMetadata metadata = versionedMetadata.getValue(); if (metadata.isClosed()) { lastAddConfirmed = lastAddPushed = metadata.getLastEntryId(); length = metadata.getLength(); @@ -419,14 +428,6 @@ BookiesHealthInfo getBookiesHealthInfo() { return bookiesHealthInfo; } - void writeLedgerConfig(GenericCallback writeCb) { - if (LOG.isDebugEnabled()) { - LOG.debug("Writing metadata to ledger manager: {}, {}", this.ledgerId, getLedgerMetadata().getVersion()); - } - - clientCtx.getLedgerManager().writeLedgerMetadata(ledgerId, getLedgerMetadata(), writeCb); - } - /** * {@inheritDoc} */ @@ -532,7 +533,7 @@ public void safeRun() { tearDownWriteHandleState(); new MetadataUpdateLoop( clientCtx.getLedgerManager(), getId(), - LedgerHandle.this::getLedgerMetadata, + LedgerHandle.this::getVersionedLedgerMetadata, (metadata) -> { if (metadata.isClosed()) { /* If the ledger has been closed with the same lastEntry @@ -1851,7 +1852,7 @@ void ensembleChangeLoop(List origEnsemble, Map !metadata.isClosed() && !metadata.isInRecovery() && failedBookies.entrySet().stream().anyMatch( (e) -> metadata.getLastEnsembleValue().get(e.getKey()).equals(e.getValue())), @@ -1882,13 +1883,13 @@ void ensembleChangeLoop(List origEnsemble, Map ensemble, final S } } - void rereadMetadata(final GenericCallback cb) { - clientCtx.getLedgerManager().readLedgerMetadata(ledgerId, cb); - } - void registerOperationFailureOnBookie(BookieSocketAddress bookie, long entryId) { if (clientCtx.getConf().enableBookieFailureTracking) { bookieFailureHistory.put(bookie, entryId); @@ -1965,6 +1962,6 @@ List getCurrentEnsemble() { // Getting current ensemble from the metadata is only a temporary // thing until metadata is immutable. At that point, current ensemble // becomes a property of the LedgerHandle itself. - return metadata.getCurrentEnsemble(); + return versionedMetadata.getValue().getCurrentEnsemble(); } } 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 bd8beae48bf..6f38b8e678d 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 @@ -37,6 +37,7 @@ import org.apache.bookkeeper.client.api.WriteAdvHandle; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.util.SafeRunnable; +import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,7 +57,7 @@ public int compare(PendingAddOp o1, PendingAddOp o2) { } LedgerHandleAdv(ClientContext clientCtx, - long ledgerId, LedgerMetadata metadata, + long ledgerId, Versioned metadata, BookKeeper.DigestType digestType, byte[] password, EnumSet writeFlags) throws GeneralSecurityException, NumberFormatException { super(clientCtx, ledgerId, metadata, digestType, password, writeFlags); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index d8e1c1be372..620a7980198 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -45,7 +45,6 @@ import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; -import org.apache.bookkeeper.versioning.Version; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,7 +84,6 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe private LedgerMetadataFormat.State state; private TreeMap> ensembles = new TreeMap<>(); private List currentEnsemble; - volatile Version version = Version.NEW; private boolean hasPassword = false; private LedgerMetadataFormat.DigestType digestType; @@ -140,8 +138,7 @@ public LedgerMetadata(int ensembleSize, DigestType digestType, java.util.Optional password, java.util.Optional ctime, - Map customMetadata, - Version version) { + Map customMetadata) { checkArgument(ensembles.size() > 0, "There must be at least one ensemble in the ledger"); this.ensembleSize = ensembleSize; @@ -174,7 +171,6 @@ public LedgerMetadata(int ensembleSize, }); this.customMetadata.putAll(customMetadata); - this.version = version; } /** @@ -197,7 +193,6 @@ public LedgerMetadata(int ensembleSize, int writeQuorumSize, int ackQuorumSize, this.lastEntryId = other.lastEntryId; this.metadataFormatVersion = other.metadataFormatVersion; this.state = other.state; - this.version = other.version; this.hasPassword = other.hasPassword; this.digestType = other.digestType; this.ctime = other.ctime; @@ -472,17 +467,14 @@ private byte[] serializeVersion1() { * * @param bytes * byte array to parse - * @param version - * version of the ledger metadata * @param msCtime * metadata store creation time, used for legacy ledgers * @return LedgerConfig * @throws IOException * if the given byte[] cannot be parsed */ - public static LedgerMetadata parseConfig(byte[] bytes, Version version, Optional msCtime) throws IOException { + public static LedgerMetadata parseConfig(byte[] bytes, Optional msCtime) throws IOException { LedgerMetadata lc = new LedgerMetadata(); - lc.version = version; String config = new String(bytes, UTF_8); @@ -608,37 +600,6 @@ static LedgerMetadata parseVersion1Config(LedgerMetadata lc, return lc; } - /** - * Updates the version of this metadata. - * - * @param v Version - */ - public void setVersion(Version v) { - this.version = v; - } - - /** - * Returns the last version. - * - * @return version - */ - public Version getVersion() { - return this.version; - } - - /** - * Is the metadata newer than given newMeta. - * - * @param newMeta the metadata to compare - * @return true if this is newer than newMeta, false otherwise - */ - boolean isNewerThan(LedgerMetadata newMeta) { - if (null == version) { - return false; - } - return Version.Occurred.AFTER == version.compare(newMeta.version); - } - /** * Routine to compare two {@code Map}; Since the values in the map are {@code byte[]}, we can't use * {@code Map.equals}. @@ -689,8 +650,7 @@ public String toSafeString() { private String toStringRepresentation(boolean withPassword) { StringBuilder sb = new StringBuilder(); - sb.append("(meta:").append(new String(serialize(withPassword), UTF_8)).append(", version:").append(version) - .append(")"); + sb.append("(meta:").append(new String(serialize(withPassword), UTF_8)).append(")"); return sb.toString(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index b1b9dbdd3ac..8206438b715 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import java.util.Arrays; @@ -30,11 +31,18 @@ import java.util.TreeMap; import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; +import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; -import org.apache.bookkeeper.versioning.Version; -class LedgerMetadataBuilder { +/** + * Builder for building LedgerMetadata objects. + */ +@LimitedPrivate +@Unstable +@VisibleForTesting +public class LedgerMetadataBuilder { private int ensembleSize = 3; private int writeQuorumSize = 3; private int ackQuorumSize = 2; @@ -51,13 +59,11 @@ class LedgerMetadataBuilder { private Optional ctime = Optional.empty(); private Map customMetadata = Collections.emptyMap(); - private Version version = Version.NEW; - - static LedgerMetadataBuilder create() { + public static LedgerMetadataBuilder create() { return new LedgerMetadataBuilder(); } - static LedgerMetadataBuilder from(LedgerMetadata other) { + public static LedgerMetadataBuilder from(LedgerMetadata other) { LedgerMetadataBuilder builder = new LedgerMetadataBuilder(); builder.ensembleSize = other.getEnsembleSize(); builder.writeQuorumSize = other.getWriteQuorumSize(); @@ -86,36 +92,39 @@ static LedgerMetadataBuilder from(LedgerMetadata other) { } builder.customMetadata = ImmutableMap.copyOf(other.getCustomMetadata()); - builder.version = other.getVersion(); - return builder; } - LedgerMetadataBuilder withPassword(byte[] password) { + public LedgerMetadataBuilder withPassword(byte[] password) { this.password = Optional.of(Arrays.copyOf(password, password.length)); return this; } - LedgerMetadataBuilder withEnsembleSize(int ensembleSize) { + public LedgerMetadataBuilder withDigestType(DigestType digestType) { + this.digestType = digestType; + return this; + } + + public LedgerMetadataBuilder withEnsembleSize(int ensembleSize) { checkState(ensembles.size() == 0, "Can only set ensemble size before adding ensembles to the builder"); this.ensembleSize = ensembleSize; return this; } - LedgerMetadataBuilder withWriteQuorumSize(int writeQuorumSize) { + public LedgerMetadataBuilder withWriteQuorumSize(int writeQuorumSize) { checkArgument(ensembleSize >= writeQuorumSize, "Write quorum must be less or equal to ensemble size"); checkArgument(writeQuorumSize >= ackQuorumSize, "Write quorum must be greater or equal to ack quorum"); this.writeQuorumSize = writeQuorumSize; return this; } - LedgerMetadataBuilder withAckQuorumSize(int ackQuorumSize) { + public LedgerMetadataBuilder withAckQuorumSize(int ackQuorumSize) { checkArgument(writeQuorumSize >= ackQuorumSize, "Ack quorum must be less or equal to write quorum"); this.ackQuorumSize = ackQuorumSize; return this; } - LedgerMetadataBuilder newEnsembleEntry(long firstEntry, List ensemble) { + public LedgerMetadataBuilder newEnsembleEntry(long firstEntry, List ensemble) { checkArgument(ensemble.size() == ensembleSize, "Size of passed in ensemble must match the ensembleSize of the builder"); checkArgument(ensembles.isEmpty() || firstEntry > ensembles.lastKey(), @@ -124,7 +133,7 @@ LedgerMetadataBuilder newEnsembleEntry(long firstEntry, List ensemble) { + public LedgerMetadataBuilder replaceEnsembleEntry(long firstEntry, List ensemble) { checkArgument(ensemble.size() == ensembleSize, "Size of passed in ensemble must match the ensembleSize of the builder"); checkArgument(ensembles.containsKey(firstEntry), @@ -133,23 +142,22 @@ LedgerMetadataBuilder replaceEnsembleEntry(long firstEntry, List { +class LedgerOpenOp implements GenericCallback> { static final Logger LOG = LoggerFactory.getLogger(LedgerOpenOp.class); final BookKeeper bk; @@ -125,21 +126,25 @@ public void initiateWithoutRecovery() { * Implements Open Ledger Callback. */ @Override - public void operationComplete(int rc, LedgerMetadata metadata) { + public void operationComplete(int rc, Versioned versionedMetadata) { if (BKException.Code.OK != rc) { // open ledger failed. openComplete(rc, null); return; } + LedgerMetadata metadata = versionedMetadata.getValue(); final byte[] passwd; // we should use digest type from metadata *ONLY* when: // 1) digest type is stored in metadata // 2) `autodetection` is enabled - DigestType digestType = enableDigestAutodetection && metadata.hasPassword() - ? fromApiDigestType(metadata.getDigestType()) - : suggestedDigestType; + DigestType digestType; + if (enableDigestAutodetection && metadata.hasPassword()) { + digestType = fromApiDigestType(metadata.getDigestType()); + } else { + digestType = suggestedDigestType; + } /* For an administrative open, the default passwords * are read from the configuration, but if the metadata @@ -168,7 +173,7 @@ public void operationComplete(int rc, LedgerMetadata metadata) { // get the ledger metadata back try { - lh = new ReadOnlyLedgerHandle(bk.getClientCtx(), ledgerId, metadata, digestType, + lh = new ReadOnlyLedgerHandle(bk.getClientCtx(), ledgerId, versionedMetadata, digestType, passwd, !doRecovery); } catch (GeneralSecurityException e) { LOG.error("Security exception while opening ledger: " + ledgerId, e); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java index 9cd2afd1c9a..821a4dac174 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +51,7 @@ class MetadataUpdateLoop { private final LedgerManager lm; private final long ledgerId; - private final Supplier currentLocalValue; + private final Supplier> currentLocalValue; private final NeedsUpdatePredicate needsTransformation; private final MetadataTransform transform; private final LocalValueUpdater updateLocalValue; @@ -69,7 +70,7 @@ interface MetadataTransform { } interface LocalValueUpdater { - boolean updateValue(LedgerMetadata oldValue, LedgerMetadata newValue); + boolean updateValue(Versioned oldValue, Versioned newValue); } /** @@ -88,7 +89,7 @@ interface LocalValueUpdater { */ MetadataUpdateLoop(LedgerManager lm, long ledgerId, - Supplier currentLocalValue, + Supplier> currentLocalValue, NeedsUpdatePredicate needsTransformation, MetadataTransform transform, LocalValueUpdater updateLocalValue) { @@ -103,22 +104,23 @@ interface LocalValueUpdater { ledgerId, System.identityHashCode(this)); } - CompletableFuture run() { - CompletableFuture promise = new CompletableFuture<>(); + CompletableFuture> run() { + CompletableFuture> promise = new CompletableFuture<>(); writeLoop(currentLocalValue.get(), promise); return promise; } - private void writeLoop(LedgerMetadata currentLocal, CompletableFuture promise) { + private void writeLoop(Versioned currentLocal, + CompletableFuture> promise) { LOG.debug("{} starting write loop iteration, attempt {}", logContext, WRITE_LOOP_COUNT_UPDATER.incrementAndGet(this)); try { - if (needsTransformation.needsUpdate(currentLocal)) { - LedgerMetadata transformed = transform.transform(currentLocal); + if (needsTransformation.needsUpdate(currentLocal.getValue())) { + LedgerMetadata transformed = transform.transform(currentLocal.getValue()); - writeToStore(ledgerId, transformed) + writeToStore(ledgerId, transformed, currentLocal.getVersion()) .whenComplete((writtenMetadata, ex) -> { if (ex == null) { if (updateLocalValue.updateValue(currentLocal, writtenMetadata)) { @@ -153,16 +155,16 @@ private void writeLoop(LedgerMetadata currentLocal, CompletableFuture updateLocalValueFromStore(long ledgerId) { - CompletableFuture promise = new CompletableFuture<>(); + private CompletableFuture> updateLocalValueFromStore(long ledgerId) { + CompletableFuture> promise = new CompletableFuture<>(); readLoop(ledgerId, promise); return promise; } - private void readLoop(long ledgerId, CompletableFuture promise) { - LedgerMetadata current = currentLocalValue.get(); + private void readLoop(long ledgerId, CompletableFuture> promise) { + Versioned current = currentLocalValue.get(); lm.readLedgerMetadata(ledgerId, (rc, read) -> { @@ -185,10 +187,11 @@ private void readLoop(long ledgerId, CompletableFuture promise) }); } - private CompletableFuture writeToStore(long ledgerId, LedgerMetadata toWrite) { - CompletableFuture promise = new CompletableFuture<>(); + private CompletableFuture> writeToStore(long ledgerId, LedgerMetadata toWrite, + Version currentVersion) { + CompletableFuture> promise = new CompletableFuture<>(); - lm.writeLedgerMetadata(ledgerId, toWrite, + lm.writeLedgerMetadata(ledgerId, toWrite, currentVersion, (rc, written) -> { if (rc != BKException.Code.OK) { promise.completeExceptionally(BKException.create(rc)); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java index e4794de575f..9c2901ccee1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java @@ -45,6 +45,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.TimedGenericCallback; import org.apache.bookkeeper.util.SafeRunnable; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,25 +64,22 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListene class MetadataUpdater extends SafeRunnable { - final LedgerMetadata newMetadata; + final Versioned newMetadata; - MetadataUpdater(LedgerMetadata metadata) { + MetadataUpdater(Versioned metadata) { this.newMetadata = metadata; } @Override public void safeRun() { while (true) { - LedgerMetadata currentMetadata = getLedgerMetadata(); + Versioned currentMetadata = getVersionedLedgerMetadata(); Version.Occurred occurred = currentMetadata.getVersion().compare(newMetadata.getVersion()); if (Version.Occurred.BEFORE == occurred) { - LOG.info("Updated ledger metadata for ledger {} to {}.", ledgerId, newMetadata.toSafeString()); synchronized (ReadOnlyLedgerHandle.this) { - if (newMetadata.isClosed()) { - ReadOnlyLedgerHandle.this.lastAddConfirmed = newMetadata.getLastEntryId(); - ReadOnlyLedgerHandle.this.length = newMetadata.getLength(); - } if (setLedgerMetadata(currentMetadata, newMetadata)) { + LOG.info("Updated ledger metadata for ledger {} to {}, version {}.", + ledgerId, newMetadata.getValue().toSafeString(), newMetadata.getVersion()); break; } } @@ -98,7 +96,7 @@ public String toString() { } ReadOnlyLedgerHandle(ClientContext clientCtx, - long ledgerId, LedgerMetadata metadata, + long ledgerId, Versioned metadata, BookKeeper.DigestType digestType, byte[] password, boolean watch) throws GeneralSecurityException, NumberFormatException { @@ -147,7 +145,7 @@ public void asyncAddEntry(final byte[] data, final int offset, final int length, } @Override - public void onChanged(long lid, LedgerMetadata newMetadata) { + public void onChanged(long lid, Versioned newMetadata) { if (LOG.isDebugEnabled()) { LOG.debug("Received ledger metadata update on {} : {}", lid, newMetadata); } @@ -157,7 +155,7 @@ public void onChanged(long lid, LedgerMetadata newMetadata) { if (null == newMetadata) { return; } - LedgerMetadata currentMetadata = getLedgerMetadata(); + Versioned currentMetadata = getVersionedLedgerMetadata(); Version.Occurred occurred = currentMetadata.getVersion().compare(newMetadata.getVersion()); if (LOG.isDebugEnabled()) { LOG.debug("Try to update metadata from {} to {} : {}", @@ -276,13 +274,13 @@ void recover(GenericCallback finalCb, } new MetadataUpdateLoop( clientCtx.getLedgerManager(), getId(), - this::getLedgerMetadata, + this::getVersionedLedgerMetadata, needsUpdate, (metadata) -> LedgerMetadataBuilder.from(metadata).withInRecoveryState().build(), this::setLedgerMetadata) .run() .thenCompose((metadata) -> { - if (metadata.isClosed()) { + if (metadata.getValue().isClosed()) { return CompletableFuture.completedFuture(ReadOnlyLedgerHandle.this); } else { return new LedgerRecoveryOp(ReadOnlyLedgerHandle.this, clientCtx) @@ -301,16 +299,16 @@ void recover(GenericCallback finalCb, }); } - CompletableFuture closeRecovered() { + CompletableFuture> closeRecovered() { long lac, len; synchronized (this) { lac = lastAddConfirmed; len = length; } LOG.info("Closing recovered ledger {} at entry {}", getId(), lac); - CompletableFuture f = new MetadataUpdateLoop( + CompletableFuture> f = new MetadataUpdateLoop( clientCtx.getLedgerManager(), getId(), - this::getLedgerMetadata, + this::getVersionedLedgerMetadata, (metadata) -> metadata.isInRecovery(), (metadata) -> { LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java index 37f8b096d6c..fec6e58af4b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java @@ -38,6 +38,7 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; +import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,10 +95,10 @@ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final final long ledgerId = ledgerItr.next(); issuedLedgerCnt.incrementAndGet(); - GenericCallbackFuture readPromise = new GenericCallbackFuture<>(); + GenericCallbackFuture> readPromise = new GenericCallbackFuture<>(); lm.readLedgerMetadata(ledgerId, readPromise); - CompletableFuture writePromise = readPromise.thenCompose((readMetadata) -> { - AtomicReference ref = new AtomicReference<>(readMetadata); + CompletableFuture> writePromise = readPromise.thenCompose((readMetadata) -> { + AtomicReference> ref = new AtomicReference<>(readMetadata); return new MetadataUpdateLoop( lm, ledgerId, ref::get, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index ccfbb52ecfb..b0af1ba8498 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -46,6 +46,7 @@ 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.AsyncCallback; import org.apache.zookeeper.AsyncCallback.DataCallback; import org.apache.zookeeper.AsyncCallback.StatCallback; @@ -85,7 +86,7 @@ public abstract class AbstractZkLedgerManager implements LedgerManager, Watcher /** * ReadLedgerMetadataTask class. */ - protected class ReadLedgerMetadataTask implements Runnable, GenericCallback { + protected class ReadLedgerMetadataTask implements Runnable, GenericCallback> { final long ledgerId; @@ -108,7 +109,7 @@ public void run() { } @Override - public void operationComplete(int rc, final LedgerMetadata result) { + public void operationComplete(int rc, final Versioned result) { if (BKException.Code.OK == rc) { final Set listenerSet = listeners.get(ledgerId); if (null != listenerSet) { @@ -245,15 +246,13 @@ public void process(WatchedEvent event) { @Override public void createLedgerMetadata(final long ledgerId, final LedgerMetadata metadata, - final GenericCallback ledgerCb) { + final GenericCallback> ledgerCb) { String ledgerPath = getLedgerPath(ledgerId); StringCallback scb = new StringCallback() { @Override public void processResult(int rc, String path, Object ctx, String name) { if (rc == Code.OK.intValue()) { - // update version - metadata.setVersion(new LongVersion(0)); - ledgerCb.operationComplete(BKException.Code.OK, metadata); + ledgerCb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, new LongVersion(0))); } else if (rc == Code.NODEEXISTS.intValue()) { LOG.warn("Failed to create ledger metadata for {} which already exist", ledgerId); ledgerCb.operationComplete(BKException.Code.LedgerExistException, null); @@ -381,11 +380,11 @@ public void unregisterLedgerMetadataListener(long ledgerId, LedgerMetadataListen } @Override - public void readLedgerMetadata(final long ledgerId, final GenericCallback readCb) { + public void readLedgerMetadata(final long ledgerId, final GenericCallback> readCb) { readLedgerMetadata(ledgerId, readCb, null); } - protected void readLedgerMetadata(final long ledgerId, final GenericCallback readCb, + protected void readLedgerMetadata(final long ledgerId, final GenericCallback> readCb, Watcher watcher) { zk.getData(getLedgerPath(ledgerId), watcher, new DataCallback() { @Override @@ -409,29 +408,28 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta readCb.operationComplete(BKException.Code.ZKException, null); return; } - LedgerMetadata metadata; + try { - metadata = LedgerMetadata.parseConfig(data, new LongVersion(stat.getVersion()), - Optional.of(stat.getCtime())); + LongVersion version = new LongVersion(stat.getVersion()); + LedgerMetadata metadata = LedgerMetadata.parseConfig(data, Optional.of(stat.getCtime())); + readCb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, version)); } catch (IOException e) { LOG.error("Could not parse ledger metadata for ledger: " + ledgerId, e); readCb.operationComplete(BKException.Code.ZKException, null); - return; } - readCb.operationComplete(BKException.Code.OK, metadata); + } }, null); } @Override - public void writeLedgerMetadata(final long ledgerId, final LedgerMetadata metadata, - final GenericCallback cb) { - Version v = metadata.getVersion(); - if (!(v instanceof LongVersion)) { + public void writeLedgerMetadata(final long ledgerId, final LedgerMetadata metadata, final Version currentVersion, + final GenericCallback> cb) { + if (!(currentVersion instanceof LongVersion)) { cb.operationComplete(BKException.Code.MetadataVersionException, null); return; } - final LongVersion zv = (LongVersion) v; + final LongVersion zv = (LongVersion) currentVersion; zk.setData(getLedgerPath(ledgerId), metadata.serialize(), (int) zv.getLongVersion(), new StatCallback() { @@ -441,8 +439,8 @@ public void processResult(int rc, String path, Object ctx, Stat stat) { cb.operationComplete(BKException.Code.MetadataVersionException, null); } else if (KeeperException.Code.OK.intValue() == rc) { // update metadata version - metadata.setVersion(zv.setLongVersion(stat.getVersion())); - cb.operationComplete(BKException.Code.OK, metadata); + cb.operationComplete(BKException.Code.OK, + new Versioned<>(metadata, new LongVersion(stat.getVersion()))); } else if (KeeperException.Code.NONODE.intValue() == rc) { LOG.warn("Ledger node does not exist in ZooKeeper: ledgerId={}", ledgerId); cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java index 36f9d8c7412..6a87cf2823c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback; /** @@ -109,14 +110,14 @@ private GenericCallback removeCallback(GenericCallback callback) { @Override public void createLedgerMetadata(long lid, LedgerMetadata metadata, - GenericCallback cb) { + GenericCallback> cb) { closeLock.readLock().lock(); try { if (closed) { cb.operationComplete(BKException.Code.ClientClosedException, null); return; } - underlying.createLedgerMetadata(lid, metadata, new CleanupGenericCallback(cb)); + underlying.createLedgerMetadata(lid, metadata, new CleanupGenericCallback<>(cb)); } finally { closeLock.readLock().unlock(); } @@ -140,14 +141,14 @@ public void removeLedgerMetadata(long ledgerId, Version version, @Override public void readLedgerMetadata(long ledgerId, - GenericCallback readCb) { + GenericCallback> readCb) { closeLock.readLock().lock(); try { if (closed) { readCb.operationComplete(BKException.Code.ClientClosedException, null); return; } - underlying.readLedgerMetadata(ledgerId, new CleanupGenericCallback(readCb)); + underlying.readLedgerMetadata(ledgerId, new CleanupGenericCallback<>(readCb)); } finally { closeLock.readLock().unlock(); } @@ -155,15 +156,16 @@ public void readLedgerMetadata(long ledgerId, @Override public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - GenericCallback cb) { + Version currentVersion, + GenericCallback> cb) { closeLock.readLock().lock(); try { if (closed) { cb.operationComplete(BKException.Code.ClientClosedException, null); return; } - underlying.writeLedgerMetadata(ledgerId, metadata, - new CleanupGenericCallback(cb)); + underlying.writeLedgerMetadata(ledgerId, metadata, currentVersion, + new CleanupGenericCallback<>(cb)); } finally { closeLock.readLock().unlock(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java index 84b9cb69ab0..1ad2eb1c646 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback; /** @@ -55,7 +56,7 @@ public interface LedgerManager extends Closeable { * for other issue * */ - void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback cb); + void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback> cb); /** * Remove a specified ledger metadata by ledgerId and version. @@ -86,7 +87,7 @@ public interface LedgerManager extends Closeable { *
  • {@link BKException.Code.ZKException} for other issue
  • * */ - void readLedgerMetadata(long ledgerId, GenericCallback readCb); + void readLedgerMetadata(long ledgerId, GenericCallback> readCb); /** * Write ledger metadata. @@ -95,6 +96,8 @@ public interface LedgerManager extends Closeable { * Ledger Id * @param metadata * Ledger Metadata to write + * @param currentVersion + * The version of the metadata we expect to be overwriting. * @param cb * Callback when finished writing ledger metadata, returning the written metadata. * Return code:
      @@ -103,7 +106,9 @@ public interface LedgerManager extends Closeable { *
    • {@link BKException.Code.ZKException} for other issue
    • *
    */ - void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback cb); + void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion, + GenericCallback> cb); /** * Register the ledger metadata listener on ledgerId. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index 5b28a0be487..d8fc4f2b5fc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -224,7 +224,7 @@ static class MsLedgerManager implements LedgerManager, MetastoreWatcher { // callbacks ScheduledExecutorService scheduler; - protected class ReadLedgerMetadataTask implements Runnable, GenericCallback { + protected class ReadLedgerMetadataTask implements Runnable, GenericCallback> { final long ledgerId; @@ -247,12 +247,12 @@ public void run() { } @Override - public void operationComplete(int rc, final LedgerMetadata result) { + public void operationComplete(int rc, final Versioned result) { if (BKException.Code.OK == rc) { final Set listenerSet = listeners.get(ledgerId); if (null != listenerSet) { if (LOG.isDebugEnabled()) { - LOG.debug("Ledger metadata is changed for {} : {}.", ledgerId, result); + LOG.debug("Ledger metadata is changed for {} : {}.", ledgerId, result.getValue()); } scheduler.submit(new Runnable() { @Override @@ -376,7 +376,7 @@ public void close() { @Override public void createLedgerMetadata(final long lid, final LedgerMetadata metadata, - final GenericCallback ledgerCb) { + final GenericCallback> ledgerCb) { MetastoreCallback msCallback = new MetastoreCallback() { @Override public void complete(int rc, Version version, Object ctx) { @@ -391,9 +391,7 @@ public void complete(int rc, Version version, Object ctx) { if (LOG.isDebugEnabled()) { LOG.debug("Create ledger {} with version {} successfully.", lid, version); } - // update version - metadata.setVersion(version); - ledgerCb.operationComplete(BKException.Code.OK, metadata); + ledgerCb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, version)); } }; @@ -423,7 +421,7 @@ public void complete(int rc, Void value, Object ctx) { } @Override - public void readLedgerMetadata(final long ledgerId, final GenericCallback readCb) { + public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { final String key = ledgerId2Key(ledgerId); MetastoreCallback> msCallback = new MetastoreCallback>() { @Override @@ -440,54 +438,49 @@ public void complete(int rc, Versioned value, Object ctx) { readCb.operationComplete(BKException.Code.MetaStoreException, null); return; } - LedgerMetadata metadata; try { - metadata = LedgerMetadata.parseConfig(value.getValue().getField(META_FIELD), - value.getVersion(), Optional.absent()); + LedgerMetadata metadata = LedgerMetadata.parseConfig( + value.getValue().getField(META_FIELD), Optional.absent()); + readCb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, value.getVersion())); } catch (IOException e) { LOG.error("Could not parse ledger metadata for ledger " + ledgerId + " : ", e); readCb.operationComplete(BKException.Code.MetaStoreException, null); - return; } - readCb.operationComplete(BKException.Code.OK, metadata); } }; ledgerTable.get(key, this, msCallback, ALL_FIELDS); } @Override - public void writeLedgerMetadata(final long ledgerId, final LedgerMetadata metadata, - final GenericCallback cb) { + public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion, + final GenericCallback> cb) { Value data = new Value().setField(META_FIELD, metadata.serialize()); if (LOG.isDebugEnabled()) { - LOG.debug("Writing ledger {} metadata, version {}", new Object[] { ledgerId, metadata.getVersion() }); + LOG.debug("Writing ledger {} metadata, version {}", new Object[] { ledgerId, currentVersion }); } final String key = ledgerId2Key(ledgerId); MetastoreCallback msCallback = new MetastoreCallback() { @Override public void complete(int rc, Version version, Object ctx) { - int bkRc; if (MSException.Code.BadVersion.getCode() == rc) { LOG.info("Bad version provided to updat metadata for ledger {}", ledgerId); - bkRc = BKException.Code.MetadataVersionException; + cb.operationComplete(BKException.Code.MetadataVersionException, null); } else if (MSException.Code.NoKey.getCode() == rc) { LOG.warn("Ledger {} doesn't exist when writing its ledger metadata.", ledgerId); - bkRc = BKException.Code.NoSuchLedgerExistsException; + cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); } else if (MSException.Code.OK.getCode() == rc) { - metadata.setVersion(version); - bkRc = BKException.Code.OK; + cb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, version)); } else { LOG.warn("Conditional update ledger metadata failed: ", MSException.create(MSException.Code.get(rc), "Failed to put key " + key)); - bkRc = BKException.Code.MetaStoreException; + cb.operationComplete(BKException.Code.MetaStoreException, null); } - - cb.operationComplete(bkRc, metadata); } }; - ledgerTable.put(key, data, metadata.getVersion(), msCallback, null); + ledgerTable.put(key, data, currentVersion, msCallback, null); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java index d5ed5aeb8d6..8412e06f42d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java @@ -38,6 +38,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.MathUtils; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +69,7 @@ public interface LedgerMetadataListener { * @param metadata * new ledger metadata. */ - void onChanged(long ledgerId, LedgerMetadata metadata); + void onChanged(long ledgerId, Versioned metadata); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java index 80bc1a5c9c4..5227176b5bc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java @@ -33,6 +33,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.ReplicationException.BKAuditException; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,32 +67,24 @@ public Map> getBookieToLedgerIndex() Processor ledgerProcessor = new Processor() { @Override - public void process(final Long ledgerId, - final AsyncCallback.VoidCallback iterCallback) { - GenericCallback genericCallback = new GenericCallback() { - @Override - public void operationComplete(int rc, - LedgerMetadata ledgerMetadata) { - if (rc == BKException.Code.OK) { - for (Map.Entry> ensemble : ledgerMetadata - .getAllEnsembles().entrySet()) { - for (BookieSocketAddress bookie : ensemble - .getValue()) { - putLedger(bookie2ledgersMap, - bookie.toString(), - ledgerId); - } + public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { + GenericCallback> genericCallback = (rc, ledgerMetadata) -> { + if (rc == BKException.Code.OK) { + for (Map.Entry> ensemble + : ledgerMetadata.getValue().getAllEnsembles().entrySet()) { + for (BookieSocketAddress bookie : ensemble.getValue()) { + putLedger(bookie2ledgersMap, bookie.toString(), ledgerId); } - } else if (rc == BKException.Code.NoSuchLedgerExistsException) { - LOG.info("Ignoring replication of already deleted ledger {}", - ledgerId); - rc = BKException.Code.OK; - } else { - LOG.warn("Unable to read the ledger:" + ledgerId - + " information"); } - iterCallback.processResult(rc, null, null); + } else if (rc == BKException.Code.NoSuchLedgerExistsException) { + LOG.info("Ignoring replication of already deleted ledger {}", + ledgerId); + rc = BKException.Code.OK; + } else { + LOG.warn("Unable to read the ledger:" + ledgerId + + " information"); } + iterCallback.processResult(rc, null, null); }; ledgerManager.readLedgerMetadata(ledgerId, genericCallback); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java index efd00e7341f..928b33a2667 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java @@ -39,6 +39,7 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; +import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +69,7 @@ public ListLedgerService(ServerConfiguration conf, BookieServer bookieServer) { * Callback for reading ledger metadata. */ public static class ReadLedgerMetadataCallback extends AbstractFuture - implements BookkeeperInternalCallbacks.GenericCallback { + implements BookkeeperInternalCallbacks.GenericCallback> { final long ledgerId; ReadLedgerMetadataCallback(long ledgerId) { @@ -79,11 +80,11 @@ long getLedgerId() { return ledgerId; } - public void operationComplete(int rc, LedgerMetadata result) { + public void operationComplete(int rc, Versioned result) { if (rc != 0) { setException(BKException.create(rc)); } else { - set(result); + set(result.getValue()); } } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java index dbf09d98143..6e7b23da2f6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java @@ -73,6 +73,7 @@ import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.TestUtils; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback; import org.junit.Before; import org.junit.Test; @@ -918,7 +919,7 @@ private LedgerManager getLedgerManager(final Set ledgers) { LedgerManager manager = new LedgerManager() { @Override public void createLedgerMetadata(long lid, LedgerMetadata metadata, - GenericCallback cb) { + GenericCallback> cb) { unsupported(); } @Override @@ -927,12 +928,13 @@ public void removeLedgerMetadata(long ledgerId, Version version, unsupported(); } @Override - public void readLedgerMetadata(long ledgerId, GenericCallback readCb) { + public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { unsupported(); } @Override public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - GenericCallback cb) { + Version currentVersion, + GenericCallback> cb) { unsupported(); } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java index 1796319988b..2fd686dee92 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java @@ -48,6 +48,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.SnapshotMap; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.ZooDefs; import org.junit.Assert; import org.junit.Before; @@ -86,12 +87,12 @@ public void testGcOverreplicatedLedger() throws Exception { final AtomicReference newLedgerMetadata = new AtomicReference<>(null); final CountDownLatch latch = new CountDownLatch(1); - ledgerManager.readLedgerMetadata(lh.getId(), new GenericCallback() { + ledgerManager.readLedgerMetadata(lh.getId(), new GenericCallback>() { @Override - public void operationComplete(int rc, LedgerMetadata result) { + public void operationComplete(int rc, Versioned result) { if (rc == BKException.Code.OK) { - newLedgerMetadata.set(result); + newLedgerMetadata.set(result.getValue()); } latch.countDown(); } @@ -133,12 +134,12 @@ public void testNoGcOfLedger() throws Exception { final AtomicReference newLedgerMetadata = new AtomicReference<>(null); final CountDownLatch latch = new CountDownLatch(1); - ledgerManager.readLedgerMetadata(lh.getId(), new GenericCallback() { + ledgerManager.readLedgerMetadata(lh.getId(), new GenericCallback>() { @Override - public void operationComplete(int rc, LedgerMetadata result) { + public void operationComplete(int rc, Versioned result) { if (rc == BKException.Code.OK) { - newLedgerMetadata.set(result); + newLedgerMetadata.set(result.getValue()); } latch.countDown(); } @@ -184,12 +185,12 @@ public void testNoGcIfLedgerBeingReplicated() throws Exception { final AtomicReference newLedgerMetadata = new AtomicReference<>(null); final CountDownLatch latch = new CountDownLatch(1); - ledgerManager.readLedgerMetadata(lh.getId(), new GenericCallback() { + ledgerManager.readLedgerMetadata(lh.getId(), new GenericCallback>() { @Override - public void operationComplete(int rc, LedgerMetadata result) { + public void operationComplete(int rc, Versioned result) { if (rc == BKException.Code.OK) { - newLedgerMetadata.set(result); + newLedgerMetadata.set(result.getValue()); } latch.countDown(); } 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 e657195e44c..d9bd8882b35 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 @@ -49,6 +49,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.versioning.Versioned; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -559,13 +560,13 @@ public SyncLedgerMetaObject() { private LedgerMetadata getLedgerMetadata(LedgerHandle lh) throws Exception { final SyncLedgerMetaObject syncObj = new SyncLedgerMetaObject(); - bkc.getLedgerManager().readLedgerMetadata(lh.getId(), new GenericCallback() { + bkc.getLedgerManager().readLedgerMetadata(lh.getId(), new GenericCallback>() { @Override - public void operationComplete(int rc, LedgerMetadata result) { + public void operationComplete(int rc, Versioned result) { synchronized (syncObj) { syncObj.rc = rc; - syncObj.meta = result; + syncObj.meta = result.getValue(); syncObj.value = true; syncObj.notify(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java index 5add60f62c5..e00ec5b954a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java @@ -25,15 +25,19 @@ import java.security.GeneralSecurityException; import java.util.function.Function; +import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.util.ByteBufList; +import org.apache.bookkeeper.versioning.Versioned; /** * Client utilities. */ public class ClientUtil { + public static final org.apache.bookkeeper.client.api.DigestType DIGEST_TYPE = + org.apache.bookkeeper.client.api.DigestType.CRC32C; public static final byte[] PASSWD = "foobar".getBytes(UTF_8); public static ByteBuf generatePacket(long ledgerId, long entryId, long lastAddConfirmed, @@ -55,21 +59,34 @@ public static boolean isLedgerOpen(LedgerHandle handle) { return !handle.getLedgerMetadata().isClosed(); } - public static LedgerMetadata setupLedger(ClientContext clientCtx, long ledgerId, - LedgerMetadataBuilder builder) throws Exception { - LedgerMetadata md = builder.withPassword(PASSWD).build(); - GenericCallbackFuture mdPromise = new GenericCallbackFuture<>(); - clientCtx.getLedgerManager().createLedgerMetadata(ledgerId, md, mdPromise); + public static Versioned setupLedger(ClientContext clientCtx, long ledgerId, + LedgerMetadataBuilder builder) throws Exception { + return setupLedger(clientCtx.getLedgerManager(), ledgerId, builder); + } + + public static Versioned setupLedger(LedgerManager ledgerManager, long ledgerId, + LedgerMetadataBuilder builder) throws Exception { + LedgerMetadata md = builder.withPassword(PASSWD).withDigestType(DIGEST_TYPE).build(); + GenericCallbackFuture> mdPromise = new GenericCallbackFuture<>(); + ledgerManager.createLedgerMetadata(ledgerId, md, mdPromise); return mdPromise.get(); } - public static LedgerMetadata transformMetadata(ClientContext clientCtx, long ledgerId, - Function transform) + public static Versioned transformMetadata(ClientContext clientCtx, long ledgerId, + Function transform) + throws Exception { + return transformMetadata(clientCtx.getLedgerManager(), ledgerId, transform); + } + + public static Versioned transformMetadata(LedgerManager ledgerManager, long ledgerId, + Function transform) throws Exception { - GenericCallbackFuture readPromise = new GenericCallbackFuture<>(); - GenericCallbackFuture writePromise = new GenericCallbackFuture<>(); - clientCtx.getLedgerManager().readLedgerMetadata(ledgerId, readPromise); - clientCtx.getLedgerManager().writeLedgerMetadata(ledgerId, transform.apply(readPromise.get()), writePromise); + GenericCallbackFuture> readPromise = new GenericCallbackFuture<>(); + GenericCallbackFuture> writePromise = new GenericCallbackFuture<>(); + ledgerManager.readLedgerMetadata(ledgerId, readPromise); + ledgerManager.writeLedgerMetadata(ledgerId, transform.apply(readPromise.get().getValue()), + readPromise.get().getVersion(), writePromise); return writePromise.get(); } + } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java index ce266809876..3a547b9f071 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java @@ -36,6 +36,7 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.versioning.Versioned; import org.junit.Assert; import org.junit.Test; @@ -58,7 +59,7 @@ public class HandleFailuresTest { @Test public void testChangeTriggeredOneTimeForOneFailure() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create().newEnsembleEntry( 0L, Lists.newArrayList(b1, b2, b3))); @@ -73,7 +74,7 @@ public void testChangeTriggeredOneTimeForOneFailure() throws Exception { lh.appendAsync("entry4".getBytes()); lh.appendAsync("entry5".getBytes()).get(); - verify(clientCtx.getLedgerManager(), times(1)).writeLedgerMetadata(anyLong(), any(), any()); + verify(clientCtx.getLedgerManager(), times(1)).writeLedgerMetadata(anyLong(), any(), any(), any()); Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); } @@ -81,7 +82,7 @@ public void testChangeTriggeredOneTimeForOneFailure() throws Exception { @Test public void testSecondFailureOccursWhileFirstBeingHandled() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -119,7 +120,7 @@ public void testSecondFailureOccursWhileFirstBeingHandled() throws Exception { metadataBlocker.complete(null); future.get(); - verify(clientCtx.getLedgerManager(), times(2)).writeLedgerMetadata(anyLong(), any(), any()); + verify(clientCtx.getLedgerManager(), times(2)).writeLedgerMetadata(anyLong(), any(), any(), any()); Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b3)); Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b4)); @@ -129,7 +130,7 @@ public void testSecondFailureOccursWhileFirstBeingHandled() throws Exception { @Test public void testHandlingFailuresOneBookieFailsImmediately() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -149,7 +150,7 @@ public void testHandlingFailuresOneBookieFailsImmediately() throws Exception { @Test public void testHandlingFailuresOneBookieFailsAfterOneEntry() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -172,7 +173,7 @@ public void testHandlingFailuresOneBookieFailsAfterOneEntry() throws Exception { @Test public void testHandlingFailuresMultipleBookieFailImmediatelyNotEnoughToReplace() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -198,7 +199,7 @@ public void testHandlingFailuresMultipleBookieFailImmediatelyNotEnoughToReplace( @Test public void testHandlingFailuresMultipleBookieFailAfterOneEntryNotEnoughToReplace() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -226,7 +227,7 @@ public void testHandlingFailuresMultipleBookieFailAfterOneEntryNotEnoughToReplac @Test public void testClientClosesWhileFailureHandlerInProgress() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -267,8 +268,8 @@ public void testClientClosesWhileFailureHandlerInProgress() throws Exception { @Test public void testMetadataSetToClosedDuringFailureHandler() throws Exception { - MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + MockClientContext clientCtx = MockClientContext.create(); + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -310,8 +311,8 @@ public void testMetadataSetToClosedDuringFailureHandler() throws Exception { @Test public void testMetadataSetToInRecoveryDuringFailureHandler() throws Exception { - MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + MockClientContext clientCtx = MockClientContext.create(); + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -353,7 +354,7 @@ public void testMetadataSetToInRecoveryDuringFailureHandler() throws Exception { @Test public void testOldEnsembleChangedDuringFailureHandler() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -406,7 +407,7 @@ public void testOldEnsembleChangedDuringFailureHandler() throws Exception { @Test public void testNoAddsAreCompletedWhileFailureHandlingInProgress() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java index f2db821dfdd..93846bb68f5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java @@ -23,6 +23,7 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.versioning.Versioned; import org.junit.Assert; import org.junit.Test; @@ -47,9 +48,8 @@ public void testTryAddAfterCloseHasBeenCalled() throws Exception { MockClientContext clientCtx = MockClientContext.create(); for (int i = 0; i < 1000; i++) { - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, i, - LedgerMetadataBuilder.create().newEnsembleEntry( - 0L, Lists.newArrayList(b1, b2, b3))); + Versioned md = ClientUtil.setupLedger(clientCtx, i, + LedgerMetadataBuilder.create().newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); LedgerHandle lh = new LedgerHandle(clientCtx, i, md, BookKeeper.DigestType.CRC32C, ClientUtil.PASSWD, WriteFlag.NONE); CompletableFuture closeFuture = lh.closeAsync(); @@ -71,7 +71,7 @@ public void testTryAddAfterCloseHasBeenCalled() throws Exception { @Test public void testMetadataChangedDuringClose() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -115,7 +115,7 @@ public void testMetadataChangedDuringClose() throws Exception { @Test public void testMetadataCloseWithCorrectLengthDuringClose() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -156,7 +156,7 @@ public void testMetadataCloseWithCorrectLengthDuringClose() throws Exception { @Test public void testMetadataCloseWithDifferentLengthDuringClose() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -197,7 +197,7 @@ public void testMetadataCloseWithDifferentLengthDuringClose() throws Exception { @Test public void testMetadataCloseMarkedInRecoveryWhileClosing() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); @@ -237,7 +237,7 @@ public void testMetadataCloseMarkedInRecoveryWhileClosing() throws Exception { @Test public void testCloseWhileAddInProgress() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = ClientUtil.setupLedger(clientCtx, 10L, + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java index 2dba8c3212c..4f483a6df5a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; +import org.apache.bookkeeper.versioning.Versioned; import org.junit.Assert; import org.junit.Test; @@ -47,12 +48,12 @@ public class LedgerRecovery2Test { private static final BookieSocketAddress b4 = new BookieSocketAddress("b4", 3181); private static final BookieSocketAddress b5 = new BookieSocketAddress("b5", 3181); - private static LedgerMetadata setupLedger(ClientContext clientCtx, long ledgerId, + private static Versioned setupLedger(ClientContext clientCtx, long ledgerId, List bookies) throws Exception { LedgerMetadata md = LedgerMetadataBuilder.create() .withPassword(PASSWD) .newEnsembleEntry(0, bookies).build(); - GenericCallbackFuture mdPromise = new GenericCallbackFuture<>(); + GenericCallbackFuture> mdPromise = new GenericCallbackFuture<>(); clientCtx.getLedgerManager().createLedgerMetadata(1L, md, mdPromise); return mdPromise.get(); } @@ -61,7 +62,7 @@ private static LedgerMetadata setupLedger(ClientContext clientCtx, long ledgerId public void testCantRecoverAllDown() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = setupLedger(clientCtx, 1L, Lists.newArrayList(b1, b2, b3)); + Versioned md = setupLedger(clientCtx, 1L, Lists.newArrayList(b1, b2, b3)); clientCtx.getMockBookieClient().errorBookies(b1, b2, b3); @@ -81,7 +82,7 @@ public void testCantRecoverAllDown() throws Exception { public void testCanReadLacButCantWrite() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + Versioned md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); clientCtx.getMockBookieClient().seedEntries(b1, 1L, 0L, -1L); clientCtx.getMockBookieClient().setPreWriteHook( @@ -103,7 +104,7 @@ public void testCanReadLacButCantWrite() throws Exception { public void testMetadataClosedDuringRecovery() throws Exception { MockClientContext clientCtx = MockClientContext.create(); - LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + Versioned md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); CompletableFuture writingBack = new CompletableFuture<>(); CompletableFuture blocker = new CompletableFuture<>(); @@ -123,12 +124,8 @@ public void testMetadataClosedDuringRecovery() throws Exception { writingBack.get(10, TimeUnit.SECONDS); - GenericCallbackFuture readPromise = new GenericCallbackFuture<>(); - clientCtx.getLedgerManager().readLedgerMetadata(1L, readPromise); - LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(readPromise.get()); - GenericCallbackFuture writePromise = new GenericCallbackFuture<>(); - clientCtx.getLedgerManager().writeLedgerMetadata(1L, builder.closingAt(-1, 0).build(), writePromise); - writePromise.get(); + ClientUtil.transformMetadata(clientCtx, 1L, + (metadata) -> LedgerMetadataBuilder.from(metadata).closingAt(-1, 0).build()); // allow recovery to continue blocker.complete(null); @@ -144,7 +141,7 @@ public void testNewEnsembleAddedDuringRecovery() throws Exception { MockClientContext clientCtx = MockClientContext.create(); clientCtx.getMockRegistrationClient().addBookies(b4).get(); - LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + Versioned md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); CompletableFuture writingBack = new CompletableFuture<>(); CompletableFuture blocker = new CompletableFuture<>(); @@ -169,13 +166,9 @@ public void testNewEnsembleAddedDuringRecovery() throws Exception { writingBack.get(10, TimeUnit.SECONDS); - GenericCallbackFuture readPromise = new GenericCallbackFuture<>(); - clientCtx.getLedgerManager().readLedgerMetadata(1L, readPromise); - LedgerMetadata newMeta = LedgerMetadataBuilder.from(readPromise.get()) - .newEnsembleEntry(1L, Lists.newArrayList(b1, b2, b4)).build(); - GenericCallbackFuture writePromise = new GenericCallbackFuture<>(); - clientCtx.getLedgerManager().writeLedgerMetadata(1L, newMeta, writePromise); - writePromise.get(); + ClientUtil.transformMetadata(clientCtx, 1L, + (metadata) -> LedgerMetadataBuilder.from(metadata).newEnsembleEntry(1L, Lists.newArrayList(b1, b2, b4)) + .build()); // allow recovery to continue failing.completeExceptionally(new BKException.BKWriteException()); @@ -194,7 +187,7 @@ public void testRecoveryBookieFailedAtStart() throws Exception { MockClientContext clientCtx = MockClientContext.create(); clientCtx.getMockRegistrationClient().addBookies(b4).get(); - LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + Versioned md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); CompletableFuture writingBack = new CompletableFuture<>(); CompletableFuture blocker = new CompletableFuture<>(); @@ -219,7 +212,7 @@ public void testRecoveryOneBookieFailsDuring() throws Exception { MockClientContext clientCtx = MockClientContext.create(); clientCtx.getMockRegistrationClient().addBookies(b4).get(); - LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + Versioned md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); clientCtx.getMockBookieClient().seedEntries(b1, 1L, 0L, -1L); clientCtx.getMockBookieClient().seedEntries(b3, 1L, 1L, -1L); clientCtx.getMockBookieClient().setPreWriteHook( @@ -251,7 +244,7 @@ public void testRecoveryTwoBookiesFailOnSameEntry() throws Exception { MockClientContext clientCtx = MockClientContext.create(); clientCtx.getMockRegistrationClient().addBookies(b4, b5).get(); - LedgerMetadata md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); + Versioned md = setupLedger(clientCtx, 1, Lists.newArrayList(b1, b2, b3)); clientCtx.getMockBookieClient().seedEntries(b1, 1L, 0L, -1L); clientCtx.getMockBookieClient().setPreWriteHook( (bookie, ledgerId, entryId) -> { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java index e70a7d31e1f..7154d94239f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java @@ -39,12 +39,16 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import lombok.AllArgsConstructor; +import lombok.Data; + import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.MockLedgerManager; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.lang3.tuple.Triple; import org.junit.Assert; @@ -72,12 +76,12 @@ public void testBasicUpdate() throws Exception { new BookieSocketAddress("0.0.0.2:3181"), new BookieSocketAddress("0.0.0.3:3181"), new BookieSocketAddress("0.0.0.4:3181"))).build(); - GenericCallbackFuture promise = new GenericCallbackFuture<>(); + GenericCallbackFuture> promise = new GenericCallbackFuture<>(); long ledgerId = 1234L; lm.createLedgerMetadata(ledgerId, initMeta, promise); - LedgerMetadata writtenMetadata = promise.get(); + Versioned writtenMetadata = promise.get(); - AtomicReference reference = new AtomicReference<>(writtenMetadata); + AtomicReference> reference = new AtomicReference<>(writtenMetadata); BookieSocketAddress newAddress = new BookieSocketAddress("0.0.0.5:3181"); MetadataUpdateLoop loop = new MetadataUpdateLoop( @@ -94,7 +98,7 @@ public void testBasicUpdate() throws Exception { loop.run().get(); Assert.assertNotEquals(reference.get(), writtenMetadata); - Assert.assertEquals(reference.get().getEnsemble(0L).get(0), newAddress); + Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), newAddress); } } @@ -115,12 +119,12 @@ public void testConflictOnWrite() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); - GenericCallbackFuture promise = new GenericCallbackFuture<>(); + GenericCallbackFuture> promise = new GenericCallbackFuture<>(); lm.createLedgerMetadata(ledgerId, initMeta, promise); - LedgerMetadata writtenMetadata = promise.get(); + Versioned writtenMetadata = promise.get(); - AtomicReference reference1 = new AtomicReference<>(writtenMetadata); - CompletableFuture loop1 = new MetadataUpdateLoop( + AtomicReference> reference1 = new AtomicReference<>(writtenMetadata); + CompletableFuture> loop1 = new MetadataUpdateLoop( lm, ledgerId, reference1::get, @@ -132,8 +136,8 @@ public void testConflictOnWrite() throws Exception { }, reference1::compareAndSet).run(); - AtomicReference reference2 = new AtomicReference<>(writtenMetadata); - CompletableFuture loop2 = new MetadataUpdateLoop( + AtomicReference> reference2 = new AtomicReference<>(writtenMetadata); + CompletableFuture> loop2 = new MetadataUpdateLoop( lm, ledgerId, reference2::get, @@ -147,21 +151,21 @@ public void testConflictOnWrite() throws Exception { lm.releaseWrites(); - LedgerMetadata l1meta = loop1.get(); - LedgerMetadata l2meta = loop2.get(); + Versioned l1meta = loop1.get(); + Versioned l2meta = loop2.get(); Assert.assertEquals(l1meta, reference1.get()); Assert.assertEquals(l2meta, reference2.get()); Assert.assertEquals(l1meta.getVersion().compare(l2meta.getVersion()), Version.Occurred.BEFORE); - Assert.assertEquals(l1meta.getEnsemble(0L).get(0), b2); - Assert.assertEquals(l1meta.getEnsemble(0L).get(1), b1); + Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(0), b2); + Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(1), b1); - Assert.assertEquals(l2meta.getEnsemble(0L).get(0), b2); - Assert.assertEquals(l2meta.getEnsemble(0L).get(1), b3); + Assert.assertEquals(l2meta.getValue().getEnsemble(0L).get(0), b2); + Assert.assertEquals(l2meta.getValue().getEnsemble(0L).get(1), b3); - verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any()); + verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any(), any()); } } @@ -182,13 +186,13 @@ public void testConflictOnWriteBothWritingSame() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); - GenericCallbackFuture promise = new GenericCallbackFuture<>(); + GenericCallbackFuture> promise = new GenericCallbackFuture<>(); lm.createLedgerMetadata(ledgerId, initMeta, promise); - LedgerMetadata writtenMetadata = promise.get(); + Versioned writtenMetadata = promise.get(); - AtomicReference reference = new AtomicReference<>(writtenMetadata); + AtomicReference> reference = new AtomicReference<>(writtenMetadata); - CompletableFuture loop1 = new MetadataUpdateLoop( + CompletableFuture> loop1 = new MetadataUpdateLoop( lm, ledgerId, reference::get, @@ -199,7 +203,7 @@ public void testConflictOnWriteBothWritingSame() throws Exception { return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, reference::compareAndSet).run(); - CompletableFuture loop2 = new MetadataUpdateLoop( + CompletableFuture> loop2 = new MetadataUpdateLoop( lm, ledgerId, reference::get, @@ -216,10 +220,10 @@ public void testConflictOnWriteBothWritingSame() throws Exception { Assert.assertEquals(loop1.get(), loop2.get()); Assert.assertEquals(loop1.get(), reference.get()); - Assert.assertEquals(reference.get().getEnsemble(0L).get(0), b2); - Assert.assertEquals(reference.get().getEnsemble(0L).get(1), b1); + Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), b2); + Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(1), b1); - verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any()); + verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any(), any()); } } @@ -238,13 +242,13 @@ public void testConflictOnLocalUpdate() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); - GenericCallbackFuture promise = new GenericCallbackFuture<>(); + GenericCallbackFuture> promise = new GenericCallbackFuture<>(); lm.createLedgerMetadata(ledgerId, initMeta, promise); - LedgerMetadata writtenMetadata = promise.get(); + Versioned writtenMetadata = promise.get(); - AtomicReference reference = new AtomicReference<>(writtenMetadata); + AtomicReference> reference = new AtomicReference<>(writtenMetadata); - CompletableFuture loop1 = new MetadataUpdateLoop( + CompletableFuture> loop1 = new MetadataUpdateLoop( lm, ledgerId, reference::get, @@ -257,7 +261,7 @@ public void testConflictOnLocalUpdate() throws Exception { reference::compareAndSet).run(); lm.waitForWriteCount(1); - CompletableFuture loop2 = new MetadataUpdateLoop( + CompletableFuture> loop2 = new MetadataUpdateLoop( lm, ledgerId, reference::get, @@ -274,10 +278,10 @@ public void testConflictOnLocalUpdate() throws Exception { Assert.assertEquals(loop1.get(), reference.get()); - Assert.assertEquals(reference.get().getEnsemble(0L).get(0), b2); - Assert.assertEquals(reference.get().getEnsemble(0L).get(1), b3); + Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), b2); + Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(1), b3); - verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any()); + verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any(), any()); } } @@ -306,17 +310,17 @@ public void testHammer() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(ensembleSize) .newEnsembleEntry(0L, initialEnsemble).build(); - GenericCallbackFuture promise = new GenericCallbackFuture<>(); + GenericCallbackFuture> promise = new GenericCallbackFuture<>(); lm.createLedgerMetadata(ledgerId, initMeta, promise); - LedgerMetadata writtenMetadata = promise.get(); + Versioned writtenMetadata = promise.get(); - AtomicReference reference = new AtomicReference<>(writtenMetadata); + AtomicReference> reference = new AtomicReference<>(writtenMetadata); List replacementBookies = IntStream.range(0, ensembleSize) .mapToObj((i) -> address(String.format("0.0.%d.1:3181", i))) .collect(Collectors.toList()); - List> loops = IntStream.range(0, ensembleSize) + List>> loops = IntStream.range(0, ensembleSize) .mapToObj((i) -> new MetadataUpdateLoop( lm, ledgerId, @@ -332,7 +336,7 @@ public void testHammer() throws Exception { loops.forEach((l) -> l.join()); - Assert.assertEquals(reference.get().getEnsemble(0L), replacementBookies); + Assert.assertEquals(reference.get().getValue().getEnsemble(0L), replacementBookies); } } @@ -351,19 +355,19 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(1) .newEnsembleEntry(0L, Lists.newArrayList(b0)).build(); - GenericCallbackFuture promise = new GenericCallbackFuture<>(); + GenericCallbackFuture> promise = new GenericCallbackFuture<>(); lm.createLedgerMetadata(ledgerId, initMeta, promise); - LedgerMetadata writtenMetadata = promise.get(); + Versioned writtenMetadata = promise.get(); - AtomicReference reference = new AtomicReference<>(writtenMetadata); - CompletableFuture loop1 = new MetadataUpdateLoop( + AtomicReference> reference = new AtomicReference<>(writtenMetadata); + CompletableFuture> loop1 = new MetadataUpdateLoop( lm, ledgerId, reference::get, (currentMetadata) -> !currentMetadata.isClosed(), (currentMetadata) -> LedgerMetadataBuilder.from(currentMetadata).closingAt(10L, 100L).build(), reference::compareAndSet).run(); - CompletableFuture loop2 = new MetadataUpdateLoop( + CompletableFuture> loop2 = new MetadataUpdateLoop( lm, ledgerId, reference::get, @@ -382,7 +386,7 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { reference::compareAndSet).run(); lm.releaseWrites(); - LedgerMetadata l1meta = loop1.get(); + Versioned l1meta = loop1.get(); try { loop2.get(); Assert.fail("Update loop should have failed"); @@ -390,10 +394,10 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerClosedException.class); } Assert.assertEquals(l1meta, reference.get()); - Assert.assertEquals(l1meta.getEnsemble(0L).get(0), b0); - Assert.assertTrue(l1meta.isClosed()); + Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(0), b0); + Assert.assertTrue(l1meta.getValue().isClosed()); - verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any()); + verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any(), any()); } } @@ -416,7 +420,8 @@ public void close() { static class DeferCallbacksMockLedgerManager extends MockLedgerManager { int writeCount = 0; final int numToDefer; - List, Integer, LedgerMetadata>> deferred = Lists.newArrayList(); + List>, Integer, Versioned>> deferred = + Lists.newArrayList(); DeferCallbacksMockLedgerManager(int numToDefer) { this.numToDefer = numToDefer; @@ -434,8 +439,9 @@ synchronized void waitForWriteCount(int count) throws Exception { @Override public synchronized void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - GenericCallback cb) { - super.writeLedgerMetadata(ledgerId, metadata, + Version currentVersion, + GenericCallback> cb) { + super.writeLedgerMetadata(ledgerId, metadata, currentVersion, (rc, written) -> { synchronized (DeferCallbacksMockLedgerManager.this) { if (writeCount++ < numToDefer) { @@ -451,9 +457,18 @@ public synchronized void writeLedgerMetadata(long ledgerId, LedgerMetadata metad }; } + @Data + @AllArgsConstructor + static class DeferredUpdate { + final long ledgerId; + final LedgerMetadata metadata; + final Version currentVersion; + final GenericCallback> cb; + } + static class BlockableMockLedgerManager extends MockLedgerManager { boolean blocking = false; - List>> reqs = Lists.newArrayList(); + List reqs = Lists.newArrayList(); synchronized void blockWrites() { blocking = true; @@ -461,16 +476,18 @@ synchronized void blockWrites() { synchronized void releaseWrites() { blocking = false; - reqs.forEach((r) -> super.writeLedgerMetadata(r.getLeft(), r.getMiddle(), r.getRight())); + reqs.forEach((r) -> super.writeLedgerMetadata(r.getLedgerId(), r.getMetadata(), + r.getCurrentVersion(), r.getCb())); } @Override public synchronized void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - GenericCallback cb) { + Version currentVersion, + GenericCallback> cb) { if (blocking) { - reqs.add(Triple.of(ledgerId, metadata, cb)); + reqs.add(new DeferredUpdate(ledgerId, metadata, currentVersion, cb)); } else { - super.writeLedgerMetadata(ledgerId, metadata, cb); + super.writeLedgerMetadata(ledgerId, metadata, currentVersion, cb); } }; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java index c290a7e01be..7d71837a5c6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java @@ -65,6 +65,9 @@ import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.ByteBufList; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.junit.After; import org.junit.Before; import org.mockito.invocation.InvocationOnMock; @@ -368,7 +371,8 @@ private void setupReadLedgerMetadata() { if (ledgerMetadata == null) { cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); } else { - cb.operationComplete(BKException.Code.OK, new LedgerMetadata(ledgerMetadata)); + cb.operationComplete(BKException.Code.OK, + new Versioned<>(new LedgerMetadata(ledgerMetadata), new LongVersion(1))); } }); return null; @@ -421,7 +425,7 @@ private void setupCreateLedgerMetadata() { executor.executeOrdered(ledgerId, () -> { LedgerMetadata ledgerMetadata = (LedgerMetadata) args[1]; mockLedgerMetadataRegistry.put(ledgerId, new LedgerMetadata(ledgerMetadata)); - cb.operationComplete(BKException.Code.OK, null); + cb.operationComplete(BKException.Code.OK, new Versioned<>(ledgerMetadata, new LongVersion(1))); }); return null; }).when(ledgerManager).createLedgerMetadata(anyLong(), any(), any()); @@ -433,14 +437,15 @@ private void setupWriteLedgerMetadata() { Object[] args = invocation.getArguments(); Long ledgerId = (Long) args[0]; LedgerMetadata metadata = (LedgerMetadata) args[1]; - BookkeeperInternalCallbacks.GenericCallback cb = (BookkeeperInternalCallbacks.GenericCallback) args[2]; + Version currentVersion = (Version) args[2]; + BookkeeperInternalCallbacks.GenericCallback cb = (BookkeeperInternalCallbacks.GenericCallback) args[3]; executor.executeOrdered(ledgerId, () -> { LedgerMetadata newMetadata = LedgerMetadataBuilder.from(metadata).build(); mockLedgerMetadataRegistry.put(ledgerId, newMetadata); - cb.operationComplete(BKException.Code.OK, newMetadata); + cb.operationComplete(BKException.Code.OK, new Versioned<>(newMetadata, new LongVersion(1234))); }); return null; - }).when(ledgerManager).writeLedgerMetadata(anyLong(), any(), any()); + }).when(ledgerManager).writeLedgerMetadata(anyLong(), any(), any(), any()); } @SuppressWarnings("unchecked") diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java index 47350208dc8..c2235c25fcc 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java @@ -41,6 +41,8 @@ import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,8 +62,8 @@ public class MockLedgerHandle extends LedgerHandle { MockLedgerHandle(MockBookKeeper bk, long id, DigestType digest, byte[] passwd) throws GeneralSecurityException { super(bk.getClientCtx(), id, - new LedgerMetadata(3, 3, 2, DigestType.MAC, "".getBytes()), DigestType.MAC, "".getBytes(), - WriteFlag.NONE); + new Versioned<>(new LedgerMetadata(3, 3, 2, DigestType.MAC, "".getBytes()), new LongVersion(0L)), + DigestType.MAC, "".getBytes(), WriteFlag.NONE); this.bk = bk; this.id = id; this.digest = digest; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index 9bdbfacce30..fab13edbac6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -63,6 +63,7 @@ import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback.VoidCallback; import org.apache.zookeeper.KeeperException; import org.junit.After; @@ -93,7 +94,8 @@ void setLatch(CountDownLatch waitLatch) { } @Override - public void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback cb) { + public void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, + GenericCallback> cb) { lm.createLedgerMetadata(ledgerId, metadata, cb); } @@ -103,7 +105,7 @@ public void removeLedgerMetadata(long ledgerId, Version version, GenericCallback } @Override - public void readLedgerMetadata(long ledgerId, GenericCallback readCb) { + public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { lm.readLedgerMetadata(ledgerId, readCb); } @@ -113,8 +115,9 @@ public LedgerRangeIterator getLedgerRanges() { } @Override - public void writeLedgerMetadata(final long ledgerId, final LedgerMetadata metadata, - final GenericCallback cb) { + public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion, + GenericCallback> cb) { final CountDownLatch cdl = waitLatch; if (null != cdl) { executorService.submit(new Runnable() { @@ -126,11 +129,11 @@ public void run() { Thread.currentThread().interrupt(); LOG.error("Interrupted on waiting latch : ", e); } - lm.writeLedgerMetadata(ledgerId, metadata, cb); + lm.writeLedgerMetadata(ledgerId, metadata, currentVersion, cb); } }); } else { - lm.writeLedgerMetadata(ledgerId, metadata, cb); + lm.writeLedgerMetadata(ledgerId, metadata, currentVersion, cb); } } @@ -364,19 +367,11 @@ public void operationComplete(int rc, Void result) { LedgerHandle newRecoverLh = newBk.openLedgerNoRecovery(lh.getId(), digestType, "".getBytes()); assertEquals(BookieProtocol.INVALID_ENTRY_ID, newRecoverLh.getLastAddPushed()); assertEquals(BookieProtocol.INVALID_ENTRY_ID, newRecoverLh.getLastAddConfirmed()); + // mark the ledger as in recovery to update version. - newRecoverLh.getLedgerMetadata().markLedgerInRecovery(); - final CountDownLatch updateLatch = new CountDownLatch(1); - final AtomicInteger updateResult = new AtomicInteger(0x12345); - newRecoverLh.writeLedgerConfig(new GenericCallback() { - @Override - public void operationComplete(int rc, LedgerMetadata result) { - updateResult.set(rc); - updateLatch.countDown(); - } - }); - updateLatch.await(); - assertEquals(BKException.Code.OK, updateResult.get()); + ClientUtil.transformMetadata(newBk.getClientCtx(), newRecoverLh.getId(), + (metadata) -> LedgerMetadataBuilder.from(metadata).withInRecoveryState().build()); + newRecoverLh.close(); LOG.info("Updated ledger manager {}.", newRecoverLh.getLedgerMetadata()); } 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 6af5ed9d9a9..f407c3571e3 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 @@ -37,6 +37,8 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Versioned; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -244,7 +246,9 @@ public boolean isClosed() { return true; } }; - LedgerHandle lh = new LedgerHandle(bkc.getClientCtx(), 0, metadata, TEST_DIGEST_TYPE, + LedgerHandle lh = new LedgerHandle(bkc.getClientCtx(), 0, + new Versioned<>(metadata, new LongVersion(0L)), + TEST_DIGEST_TYPE, TEST_PSSWD, WriteFlag.NONE); testSplitIntoSubFragments(10, 21, -1, 1, lh); testSplitIntoSubFragments(10, 21, 20, 1, lh); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java index 7bfba0c4075..e51368fd38a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java @@ -22,16 +22,9 @@ import static org.junit.Assert.assertEquals; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.concurrent.CountDownLatch; +import com.google.common.collect.Lists; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; import org.slf4j.Logger; @@ -44,53 +37,29 @@ public class TestSequenceRead extends BookKeeperClusterTestCase { private static final Logger logger = LoggerFactory.getLogger(TestSequenceRead.class); - final DigestType digestType; - final byte[] passwd = "sequence-read".getBytes(); - public TestSequenceRead() { super(5); - this.digestType = DigestType.CRC32; } - private LedgerHandle createLedgerWithDuplicatedBookies() throws Exception { - final LedgerHandle lh = bkc.createLedger(3, 3, 3, digestType, passwd); + private long createLedgerWithDuplicatedBookies() throws Exception { + long ledgerId = 12345L; // introduce duplicated bookies in an ensemble. - SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); - TreeMap> newEnsembles = new TreeMap<>(); - for (Map.Entry> entry : ensembles.entrySet()) { - List newList = new ArrayList(entry.getValue().size()); - BookieSocketAddress firstBookie = entry.getValue().get(0); - for (BookieSocketAddress ignored : entry.getValue()) { - newList.add(firstBookie); - } - newEnsembles.put(entry.getKey(), newList); - } - lh.getLedgerMetadata().setEnsembles(newEnsembles); - // update the ledger metadata with duplicated bookies - final CountDownLatch latch = new CountDownLatch(1); - bkc.getLedgerManager().writeLedgerMetadata(lh.getId(), lh.getLedgerMetadata(), - new BookkeeperInternalCallbacks.GenericCallback() { - @Override - public void operationComplete(int rc, LedgerMetadata result) { - if (BKException.Code.OK == rc) { - latch.countDown(); - } else { - logger.error("Error on writing ledger metadata for ledger {} : ", lh.getId(), - BKException.getMessage(rc)); - } - } - }); - latch.await(); - logger.info("Update ledger metadata with duplicated bookies for ledger {}.", lh.getId()); - return lh; + LedgerMetadataBuilder builder = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(getBookie(0), getBookie(0), getBookie(0))); + ClientUtil.setupLedger(bkc.getLedgerManager(), ledgerId, builder); + + logger.info("Update ledger metadata with duplicated bookies for ledger {}.", ledgerId); + return ledgerId; } @Test public void testSequenceReadOnDuplicatedBookies() throws Exception { - final LedgerHandle lh = createLedgerWithDuplicatedBookies(); + final long ledgerId = createLedgerWithDuplicatedBookies(); // should be able to open the ledger even it has duplicated bookies - final LedgerHandle readLh = bkc.openLedger(lh.getId(), digestType, passwd); + final LedgerHandle readLh = bkc.openLedger( + ledgerId, DigestType.fromApiDigestType(ClientUtil.DIGEST_TYPE), ClientUtil.PASSWD); assertEquals(LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed()); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java index 456839a6320..43acabde5a5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java @@ -44,6 +44,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -134,10 +135,10 @@ private void testWatchMetadataRemoval(LedgerManagerFactory factory) throws Excep @Override public void operationComplete(int rc, final Long lid) { manager.createLedgerMetadata(lid, new LedgerMetadata(4, 2, 2, digestType, "fpj was here".getBytes()), - new BookkeeperInternalCallbacks.GenericCallback(){ + new BookkeeperInternalCallbacks.GenericCallback>(){ @Override - public void operationComplete(int rc, LedgerMetadata result) { + public void operationComplete(int rc, Versioned result) { bbLedgerId.putLong(lid); bbLedgerId.flip(); createLatch.countDown(); @@ -154,7 +155,7 @@ public void operationComplete(int rc, LedgerMetadata result) { new LedgerMetadataListener() { @Override - public void onChanged(long ledgerId, LedgerMetadata metadata) { + public void onChanged(long ledgerId, Versioned metadata) { assertEquals(ledgerId, createdLid); assertEquals(metadata, null); removeLatch.countDown(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index b2cf68d25c3..290af7bdad0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -59,6 +59,7 @@ 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.MockZooKeeperTestCase; import org.apache.zookeeper.AsyncCallback.DataCallback; import org.apache.zookeeper.AsyncCallback.StatCallback; @@ -154,13 +155,10 @@ public void testCreateLedgerMetadataSuccess() throws Exception { KeeperException.Code.OK.intValue(), ledgerStr ); - assertEquals(Version.NEW, metadata.getVersion()); - - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); ledgerManager.createLedgerMetadata(ledgerId, metadata, callbackFuture); - callbackFuture.get(); - assertEquals(new LongVersion(0), metadata.getVersion()); + assertEquals(new LongVersion(0), callbackFuture.get().getVersion()); } @Test @@ -171,9 +169,7 @@ public void testCreateLedgerMetadataNodeExists() throws Exception { ledgerStr, CreateMode.PERSISTENT, KeeperException.Code.NODEEXISTS.intValue(), null); - assertEquals(Version.NEW, metadata.getVersion()); - - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); ledgerManager.createLedgerMetadata(ledgerId, metadata, callbackFuture); try { result(callbackFuture); @@ -183,9 +179,6 @@ public void testCreateLedgerMetadataNodeExists() throws Exception { BKException bke = (BKException) e; assertEquals(Code.LedgerExistException, bke.getCode()); } - - // creation failed, so metadata should not be modified - assertEquals(Version.NEW, metadata.getVersion()); } @Test @@ -196,9 +189,7 @@ public void testCreateLedgerMetadataException() throws Exception { ledgerStr, CreateMode.PERSISTENT, KeeperException.Code.CONNECTIONLOSS.intValue(), null); - assertEquals(Version.NEW, metadata.getVersion()); - - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); ledgerManager.createLedgerMetadata(ledgerId, metadata, callbackFuture); try { result(callbackFuture); @@ -208,9 +199,6 @@ public void testCreateLedgerMetadataException() throws Exception { BKException bke = (BKException) e; assertEquals(Code.ZKException, bke.getCode()); } - - // creation failed, so metadata should not be modified - assertEquals(Version.NEW, metadata.getVersion()); } @Test @@ -355,7 +343,6 @@ public void testReadLedgerMetadataSuccess() throws Exception { long ledgerId = System.currentTimeMillis(); String ledgerStr = String.valueOf(ledgerId); - metadata.setVersion(new LongVersion(1234L)); Stat stat = mock(Stat.class); when(stat.getVersion()).thenReturn(1234); when(stat.getCtime()).thenReturn(metadata.getCtime()); @@ -363,10 +350,11 @@ public void testReadLedgerMetadataSuccess() throws Exception { ledgerStr, false, KeeperException.Code.OK.intValue(), metadata.serialize(), stat); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); - LedgerMetadata readMetadata = result(callbackFuture); - assertEquals(metadata, readMetadata); + Versioned readMetadata = result(callbackFuture); + assertEquals(metadata, readMetadata.getValue()); + assertEquals(new LongVersion(1234), readMetadata.getVersion()); verify(mockZk, times(1)) .getData(eq(ledgerStr), eq(null), any(DataCallback.class), any()); @@ -381,7 +369,7 @@ public void testReadLedgerMetadataNoNode() throws Exception { ledgerStr, false, KeeperException.Code.NONODE.intValue(), null, null); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); try { result(callbackFuture); @@ -403,7 +391,7 @@ public void testReadLedgerMetadataException() throws Exception { ledgerStr, false, KeeperException.Code.CONNECTIONLOSS.intValue(), null, null); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); try { result(callbackFuture); @@ -425,7 +413,7 @@ public void testReadLedgerMetadataStatMissing() throws Exception { ledgerStr, false, KeeperException.Code.OK.intValue(), metadata.serialize(), null); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); try { result(callbackFuture); @@ -443,7 +431,6 @@ public void testReadLedgerMetadataDataCorrupted() throws Exception { long ledgerId = System.currentTimeMillis(); String ledgerStr = String.valueOf(ledgerId); - metadata.setVersion(new LongVersion(1234L)); Stat stat = mock(Stat.class); when(stat.getVersion()).thenReturn(1234); when(stat.getCtime()).thenReturn(metadata.getCtime()); @@ -451,7 +438,7 @@ public void testReadLedgerMetadataDataCorrupted() throws Exception { ledgerStr, false, KeeperException.Code.OK.intValue(), new byte[0], stat); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); try { result(callbackFuture); @@ -469,7 +456,6 @@ public void testWriteLedgerMetadataSuccess() throws Exception { long ledgerId = System.currentTimeMillis(); String ledgerStr = String.valueOf(ledgerId); - metadata.setVersion(new LongVersion(1234L)); Stat stat = mock(Stat.class); when(stat.getVersion()).thenReturn(1235); when(stat.getCtime()).thenReturn(metadata.getCtime()); @@ -477,13 +463,10 @@ public void testWriteLedgerMetadataSuccess() throws Exception { ledgerStr, metadata.serialize(), 1234, KeeperException.Code.OK.intValue(), stat); - assertEquals(new LongVersion(1234L), metadata.getVersion()); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); + ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L), callbackFuture); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.writeLedgerMetadata(ledgerId, metadata, callbackFuture); - result(callbackFuture); - - assertEquals(new LongVersion(1235L), metadata.getVersion()); + assertEquals(new LongVersion(1235L), result(callbackFuture).getVersion()); verify(mockZk, times(1)) .setData(eq(ledgerStr), any(byte[].class), eq(1234), any(StatCallback.class), any()); @@ -494,15 +477,12 @@ public void testWriteLedgerMetadataBadVersion() throws Exception { long ledgerId = System.currentTimeMillis(); String ledgerStr = String.valueOf(ledgerId); - metadata.setVersion(new LongVersion(1234L)); mockZkSetData( ledgerStr, metadata.serialize(), 1234, KeeperException.Code.BADVERSION.intValue(), null); - assertEquals(new LongVersion(1234L), metadata.getVersion()); - - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.writeLedgerMetadata(ledgerId, metadata, callbackFuture); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); + ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L), callbackFuture); try { result(callbackFuture); fail("Should fail on writing ledger metadata if encountering bad version"); @@ -510,9 +490,6 @@ public void testWriteLedgerMetadataBadVersion() throws Exception { assertEquals(Code.MetadataVersionException, bke.getCode()); } - // version remain unchanged - assertEquals(new LongVersion(1234L), metadata.getVersion()); - verify(mockZk, times(1)) .setData(eq(ledgerStr), any(byte[].class), eq(1234), any(StatCallback.class), any()); } @@ -522,15 +499,12 @@ public void testWriteLedgerMetadataException() throws Exception { long ledgerId = System.currentTimeMillis(); String ledgerStr = String.valueOf(ledgerId); - metadata.setVersion(new LongVersion(1234L)); mockZkSetData( ledgerStr, metadata.serialize(), 1234, KeeperException.Code.CONNECTIONLOSS.intValue(), null); - assertEquals(new LongVersion(1234L), metadata.getVersion()); - - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.writeLedgerMetadata(ledgerId, metadata, callbackFuture); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); + ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L), callbackFuture); try { result(callbackFuture); fail("Should fail on writing ledger metadata if encountering zookeeper exceptions"); @@ -538,8 +512,6 @@ public void testWriteLedgerMetadataException() throws Exception { assertEquals(Code.ZKException, bke.getCode()); } - // version remain unchanged - assertEquals(new LongVersion(1234L), metadata.getVersion()); verify(mockZk, times(1)) .setData(eq(ledgerStr), any(byte[].class), eq(1234), any(StatCallback.class), any()); @@ -560,10 +532,8 @@ public void testWriteLedgerMetadataInvalidVersion() throws Exception { private void testWriteLedgerMetadataInvalidVersion(Version invalidVersion) throws Exception { long ledgerId = System.currentTimeMillis(); - metadata.setVersion(invalidVersion); - - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.writeLedgerMetadata(ledgerId, metadata, callbackFuture); + GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); + ledgerManager.writeLedgerMetadata(ledgerId, metadata, invalidVersion, callbackFuture); try { result(callbackFuture); fail("Should fail on writing ledger metadata if an invalid version is provided."); @@ -581,9 +551,8 @@ public void testLedgerMetadataListener() throws Exception { String ledgerStr = String.valueOf(ledgerId); LinkedBlockingQueue changes = new LinkedBlockingQueue<>(); - LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(metadata); + LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(metadata.getValue()); - metadata.setVersion(new LongVersion(1234L)); Stat stat = mock(Stat.class); when(stat.getVersion()).thenReturn(1234); when(stat.getCtime()).thenReturn(metadata.getCtime()); @@ -606,7 +575,6 @@ public void testLedgerMetadataListener() throws Exception { Watcher registeredWatcher1 = watcherSet1.stream().findFirst().get(); // mock get data to return an updated metadata - metadata.setVersion(new LongVersion(1235L)); when(stat.getVersion()).thenReturn(1235); mockZkGetData( ledgerStr, true, @@ -644,9 +612,9 @@ public void testLedgerMetadataListenerOnLedgerDeleted() throws Exception { String ledgerStr = String.valueOf(ledgerId); LinkedBlockingQueue> changes = new LinkedBlockingQueue<>(); - LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(Optional.ofNullable(metadata)); + LedgerMetadataListener listener = + (ledgerId1, metadata) -> changes.add(Optional.ofNullable(metadata != null ? metadata.getValue() : null)); - metadata.setVersion(new LongVersion(1234L)); Stat stat = mock(Stat.class); when(stat.getVersion()).thenReturn(1234); when(stat.getCtime()).thenReturn(metadata.getCtime()); @@ -695,9 +663,10 @@ public void testLedgerMetadataListenerOnLedgerDeletedEvent() throws Exception { String ledgerStr = String.valueOf(ledgerId); LinkedBlockingQueue> changes = new LinkedBlockingQueue<>(); - LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(Optional.ofNullable(metadata)); + LedgerMetadataListener listener = + (ledgerId1, metadata) -> changes.add( + Optional.ofNullable(metadata != null ? metadata.getValue() : null)); - metadata.setVersion(new LongVersion(1234L)); Stat stat = mock(Stat.class); when(stat.getVersion()).thenReturn(1234); when(stat.getCtime()).thenReturn(metadata.getCtime()); @@ -737,9 +706,8 @@ public void testLedgerMetadataListenerOnRetries() throws Exception { String ledgerStr = String.valueOf(ledgerId); LinkedBlockingQueue changes = new LinkedBlockingQueue<>(); - LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(metadata); + LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(metadata.getValue()); - metadata.setVersion(new LongVersion(1234L)); Stat stat = mock(Stat.class); when(stat.getVersion()).thenReturn(1234); when(stat.getCtime()).thenReturn(metadata.getCtime()); @@ -786,9 +754,8 @@ public void testLedgerMetadataListenerOnSessionExpired() throws Exception { String ledgerStr = String.valueOf(ledgerId); LinkedBlockingQueue changes = new LinkedBlockingQueue<>(); - LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(metadata); + LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(metadata.getValue()); - metadata.setVersion(new LongVersion(1234L)); Stat stat = mock(Stat.class); when(stat.getVersion()).thenReturn(1234); when(stat.getCtime()).thenReturn(metadata.getCtime()); @@ -835,9 +802,8 @@ public void testUnregisterLedgerMetadataListener() throws Exception { String ledgerStr = String.valueOf(ledgerId); LinkedBlockingQueue changes = new LinkedBlockingQueue<>(); - LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(metadata); + LedgerMetadataListener listener = (ledgerId1, metadata) -> changes.add(metadata.getValue()); - metadata.setVersion(new LongVersion(1234L)); Stat stat = mock(Stat.class); when(stat.getVersion()).thenReturn(1234); when(stat.getCtime()).thenReturn(metadata.getCtime()); @@ -861,7 +827,6 @@ public void testUnregisterLedgerMetadataListener() throws Exception { Watcher registeredWatcher1 = watcherSet1.stream().findFirst().get(); // mock get data to return an updated metadata - metadata.setVersion(new LongVersion(1235L)); when(stat.getVersion()).thenReturn(1235); mockZkGetData( ledgerStr, true, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 1459500c2f7..10192cbeda0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -72,6 +72,7 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,9 +108,9 @@ public void operationComplete(int rc, final Long ledgerId) { getLedgerManager().createLedgerMetadata(ledgerId, new LedgerMetadata(1, 1, 1, DigestType.MAC, "".getBytes()), - new GenericCallback() { + new GenericCallback>() { @Override - public void operationComplete(int rc, LedgerMetadata writtenMetadata) { + public void operationComplete(int rc, Versioned writtenMetadata) { if (rc == BKException.Code.OK) { activeLedgers.put(ledgerId, true); createdLedgers.add(ledgerId); @@ -505,7 +506,7 @@ public void testGcLedgersIfReadLedgerMetadataSaysNoSuchLedger() throws Exception LedgerManager mockLedgerManager = new CleanupLedgerManager(getLedgerManager()) { @Override - public void readLedgerMetadata(long ledgerId, GenericCallback readCb) { + public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { readCb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); } }; @@ -547,7 +548,7 @@ public void testGcLedgersIfReadLedgerMetadataFailsForDeletedLedgers() throws Exc LedgerManager mockLedgerManager = new CleanupLedgerManager(getLedgerManager()) { @Override - public void readLedgerMetadata(long ledgerId, GenericCallback readCb) { + public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { readCb.operationComplete(BKException.Code.ZKException, null); } }; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java index ac8223bf00b..477afe49374 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java @@ -53,6 +53,7 @@ import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs; import org.junit.After; @@ -87,7 +88,7 @@ public void throwAsyncErrors() throws Throwable { } } - class RCCheckCB implements GenericCallback { + class RCCheckCB implements GenericCallback> { private final String opType; private final CountDownLatch latch; private final Optional rcExpected; @@ -101,7 +102,7 @@ public RCCheckCB(String opType, CountDownLatch latch, Optional rcExpect } @Override - public void operationComplete(int rc, LedgerMetadata writtenMetadata) { + public void operationComplete(int rc, Versioned writtenMetadata) { safeWrapper(() -> { try { rcExpected.map((Integer expected) -> { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index ea04192f88f..075b8f1b9a8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -37,6 +37,7 @@ import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.lang3.tuple.Pair; import org.apache.zookeeper.AsyncCallback; @@ -77,12 +78,12 @@ public MockLedgerManager newClient() { return new MockLedgerManager(metadataMap, executor, false); } - private LedgerMetadata readMetadata(long ledgerId) throws Exception { + private Versioned readMetadata(long ledgerId) throws Exception { Pair pair = metadataMap.get(ledgerId); if (pair == null) { return null; } else { - return LedgerMetadata.parseConfig(pair.getRight(), pair.getLeft(), Optional.absent()); + return new Versioned<>(LedgerMetadata.parseConfig(pair.getRight(), Optional.absent()), pair.getLeft()); } } @@ -95,14 +96,15 @@ public void executeCallback(Runnable r) { } @Override - public void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback cb) { + public void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, + GenericCallback> cb) { executor.submit(() -> { if (metadataMap.containsKey(ledgerId)) { executeCallback(() -> cb.operationComplete(BKException.Code.LedgerExistException, null)); } else { metadataMap.put(ledgerId, Pair.of(new LongVersion(0L), metadata.serialize())); try { - LedgerMetadata readBack = readMetadata(ledgerId); + Versioned readBack = readMetadata(ledgerId); executeCallback(() -> cb.operationComplete(BKException.Code.OK, readBack)); } catch (Exception e) { LOG.error("Error reading back written metadata", e); @@ -116,10 +118,10 @@ public void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, Generic public void removeLedgerMetadata(long ledgerId, Version version, GenericCallback cb) {} @Override - public void readLedgerMetadata(long ledgerId, GenericCallback cb) { + public void readLedgerMetadata(long ledgerId, GenericCallback> cb) { executor.submit(() -> { try { - LedgerMetadata metadata = readMetadata(ledgerId); + Versioned metadata = readMetadata(ledgerId); if (metadata == null) { executeCallback( () -> cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null)); @@ -134,20 +136,21 @@ public void readLedgerMetadata(long ledgerId, GenericCallback cb } @Override - public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback cb) { + public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion, GenericCallback> cb) { preWriteHook.runHook(ledgerId, metadata) .thenComposeAsync((ignore) -> { try { - LedgerMetadata oldMetadata = readMetadata(ledgerId); + Versioned oldMetadata = readMetadata(ledgerId); if (oldMetadata == null) { return FutureUtils.exception(new BKException.BKNoSuchLedgerExistsException()); - } else if (!oldMetadata.getVersion().equals(metadata.getVersion())) { + } else if (!oldMetadata.getVersion().equals(currentVersion)) { return FutureUtils.exception(new BKException.BKMetadataVersionException()); } else { LongVersion oldVersion = (LongVersion) oldMetadata.getVersion(); metadataMap.put(ledgerId, Pair.of(new LongVersion(oldVersion.getLongVersion() + 1), metadata.serialize())); - LedgerMetadata readBack = readMetadata(ledgerId); + Versioned readBack = readMetadata(ledgerId); return FutureUtils.value(readBack); } } catch (Exception e) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java index 81bfc3d5f38..06df18b7550 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java @@ -23,20 +23,17 @@ import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; import static org.junit.Assert.assertEquals; +import com.google.common.collect.Lists; import com.google.common.util.concurrent.UncheckedExecutionException; -import java.util.ArrayList; -import java.util.List; + import lombok.Cleanup; -import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerHandleAdapter; -import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.ClientUtil; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.After; import org.junit.Before; @@ -93,18 +90,13 @@ public void testPeriodicBookieCheckInterval() throws Exception { try (LedgerManager ledgerManager = mFactory.newLedgerManager()) { @Cleanup final LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); - - LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); - LedgerMetadata md = LedgerHandleAdapter.getLedgerMetadata(lh); - List ensemble = new ArrayList<>(md.getAllEnsembles().get(0L)); - ensemble.set(0, new BookieSocketAddress("1.1.1.1", 1000)); - md.updateEnsemble(0L, ensemble); - - GenericCallbackFuture cb = - new GenericCallbackFuture(); - ledgerManager.writeLedgerMetadata(lh.getId(), md, cb); - cb.get(); - + long ledgerId = 12345L; + ClientUtil.setupLedger(bkc.getLedgerManager(), ledgerId, + LedgerMetadataBuilder.create().withEnsembleSize(3) + .withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList( + new BookieSocketAddress("1.1.1.1", 1000), + getBookie(0), getBookie(1)))); long underReplicatedLedger = -1; for (int i = 0; i < 10; i++) { underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); @@ -113,7 +105,7 @@ public void testPeriodicBookieCheckInterval() throws Exception { } Thread.sleep(CHECK_INTERVAL * 1000); } - assertEquals("Ledger should be under replicated", lh.getId(), underReplicatedLedger); + assertEquals("Ledger should be under replicated", ledgerId, underReplicatedLedger); } catch (Exception e) { throw new UncheckedExecutionException(e.getMessage(), e); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index e9feada8b12..079e08f422f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -23,20 +23,19 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.UncheckedExecutionException; import java.io.File; -import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.Future; import lombok.Cleanup; import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.ClientUtil; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerHandleAdapter; -import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.common.util.JsonUtil; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; @@ -48,9 +47,9 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; + import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -664,25 +663,27 @@ private void testListUnderReplicatedLedgerService(LedgerManagerFactory mFactory) @Cleanup final LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); - LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, "passwd".getBytes()); - LedgerMetadata md = LedgerHandleAdapter.getLedgerMetadata(lh); - List ensemble = new ArrayList<>(md.getAllEnsembles().get(0L)); - ensemble.set(0, new BookieSocketAddress("1.1.1.1", 1000)); - md.updateEnsemble(0L, ensemble); - - GenericCallbackFuture cb = - new GenericCallbackFuture(); - ledgerManager.writeLedgerMetadata(lh.getId(), md, cb); - cb.get(); - + // 192.0.2.0/24 is reserved TEST-NET range + LedgerMetadataBuilder metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1000), + getBookie(0), + getBookie(1))); + ClientUtil.setupLedger(ledgerManager, 1L, metadata); + + // wait for up to two minutes to complete. + // if the metadata was created just before checkAllLedgers ran, then we need to wait for the timeout long underReplicatedLedger = -1; - for (int i = 0; i < 10; i++) { + for (int i = 0; i < 120; i++) { underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); if (underReplicatedLedger != -1) { + LOG.info("Underreplicated ledgers found, breaking out of loop"); break; } Thread.sleep(1000); } + assertTrue(underReplicatedLedger != -1); HttpServiceRequest request2 = new HttpServiceRequest(null, HttpServer.Method.GET, null); HttpServiceResponse response2 = listUnderReplicatedLedgerService.handle(request2); diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index 1dc44398452..7df7038f96c 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -54,6 +54,7 @@ import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback.VoidCallback; /** @@ -66,7 +67,6 @@ class EtcdLedgerManager implements LedgerManager { try { return LedgerMetadata.parseConfig( bs.getBytes(), - Version.ANY, Optional.absent() ); } catch (IOException ioe) { @@ -105,7 +105,7 @@ ValueStream getLedgerMetadataStream(long ledgerId) { @Override public void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, - GenericCallback cb) { + GenericCallback> cb) { String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); log.info("Create ledger metadata under key {}", ledgerKey); @@ -135,8 +135,8 @@ public void createLedgerMetadata(long ledgerId, cb.operationComplete(Code.LedgerExistException, null); } } else { - metadata.setVersion(new LongVersion(resp.getHeader().getRevision())); - cb.operationComplete(Code.OK, metadata); + cb.operationComplete(Code.OK, new Versioned<>(metadata, + new LongVersion(resp.getHeader().getRevision()))); } }) .exceptionally(cause -> { @@ -211,7 +211,7 @@ public void removeLedgerMetadata(long ledgerId, } @Override - public void readLedgerMetadata(long ledgerId, GenericCallback readCb) { + public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); log.info("read ledger metadata under key {}", ledgerKey); @@ -220,14 +220,10 @@ public void readLedgerMetadata(long ledgerId, GenericCallback re if (getResp.getCount() > 0) { KeyValue kv = getResp.getKvs().get(0); byte[] data = kv.getValue().getBytes(); - LedgerMetadata metadata; try { - metadata = LedgerMetadata.parseConfig( - data, - new LongVersion(kv.getModRevision()), - Optional.absent() - ); - readCb.operationComplete(Code.OK, metadata); + LedgerMetadata metadata = LedgerMetadata.parseConfig(data, Optional.absent()); + readCb.operationComplete( + Code.OK, new Versioned<>(metadata, new LongVersion(kv.getModRevision()))); } catch (IOException ioe) { log.error("Could not parse ledger metadata for ledger : {}", ledgerId, ioe); readCb.operationComplete(Code.MetaStoreException, null); @@ -244,13 +240,13 @@ public void readLedgerMetadata(long ledgerId, GenericCallback re } @Override - public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback cb) { - Version v = metadata.getVersion(); - if (Version.NEW == v || !(v instanceof LongVersion)) { + public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion, GenericCallback> cb) { + if (Version.NEW == currentVersion || !(currentVersion instanceof LongVersion)) { cb.operationComplete(Code.MetadataVersionException, null); return; } - final LongVersion lv = (LongVersion) v; + final LongVersion lv = (LongVersion) currentVersion; String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); kvClient.txn() @@ -268,8 +264,8 @@ public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericC .commit() .thenAccept(resp -> { if (resp.isSucceeded()) { - metadata.setVersion(new LongVersion(resp.getHeader().getRevision())); - cb.operationComplete(Code.OK, metadata); + cb.operationComplete( + Code.OK, new Versioned<>(metadata, new LongVersion(resp.getHeader().getRevision()))); } else { GetResponse getResp = resp.getGetResponses().get(0); if (getResp.getCount() > 0) { diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java index a201b253811..d466b340d7a 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java @@ -61,12 +61,10 @@ public boolean equals(Object obj) { @Override public void accept(Versioned ledgerMetadataVersioned) { - LedgerMetadata lm = ledgerMetadataVersioned.getValue(); - if (null == lm) { + if (null == ledgerMetadataVersioned.getValue()) { onDeletedConsumer.accept(ledgerId); } else { - lm.setVersion(ledgerMetadataVersioned.getVersion()); - listener.onChanged(ledgerId, lm); + listener.onChanged(ledgerId, ledgerMetadataVersioned); } } } diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index 226fd48a830..e44678034d6 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -50,6 +50,7 @@ import org.apache.bookkeeper.client.BKException.Code; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; @@ -60,6 +61,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.lang.RandomStringUtils; import org.junit.After; import org.junit.Before; @@ -102,7 +104,7 @@ public void testLedgerCRUD() throws Exception { // ledger doesn't exist: read - GenericCallbackFuture readFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> readFuture = new GenericCallbackFuture<>(); lm.readLedgerMetadata(ledgerId, readFuture); try { result(readFuture); @@ -124,9 +126,8 @@ public void testLedgerCRUD() throws Exception { // ledger doesn't exist : write - GenericCallbackFuture writeFuture = new GenericCallbackFuture<>(); - metadata.setVersion(new LongVersion(999L)); - lm.writeLedgerMetadata(ledgerId, metadata, writeFuture); + GenericCallbackFuture> writeFuture = new GenericCallbackFuture<>(); + lm.writeLedgerMetadata(ledgerId, metadata, new LongVersion(999L), writeFuture); try { result(deleteFuture); fail("Should fail on updating ledger metadata if the ledger doesn't exist"); @@ -136,10 +137,10 @@ public void testLedgerCRUD() throws Exception { // ledger doesn't exist : create - GenericCallbackFuture createFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> createFuture = new GenericCallbackFuture<>(); lm.createLedgerMetadata(ledgerId, metadata, createFuture); - LedgerMetadata writtenMetadata = result(createFuture); - assertSame(metadata, writtenMetadata); + Versioned writtenMetadata = result(createFuture); + assertSame(metadata, writtenMetadata.getValue()); Version version = writtenMetadata.getVersion(); assertNotNull(version); assertTrue(version instanceof LongVersion); @@ -161,14 +162,12 @@ public void testLedgerCRUD() throws Exception { readFuture = new GenericCallbackFuture<>(); lm.readLedgerMetadata(ledgerId, readFuture); - LedgerMetadata readMetadata = result(readFuture); - assertEquals(metadata, readMetadata); + Versioned readMetadata = result(readFuture); + assertEquals(metadata, readMetadata.getValue()); // ledger exists: update metadata with wrong version - - readMetadata.setVersion(new LongVersion(Long.MAX_VALUE)); writeFuture = new GenericCallbackFuture<>(); - lm.writeLedgerMetadata(ledgerId, readMetadata, writeFuture); + lm.writeLedgerMetadata(ledgerId, readMetadata.getValue(), new LongVersion(Long.MAX_VALUE), writeFuture); try { result(writeFuture); fail("Should fail to write metadata using a wrong version"); @@ -178,7 +177,7 @@ public void testLedgerCRUD() throws Exception { readFuture = new GenericCallbackFuture<>(); lm.readLedgerMetadata(ledgerId, readFuture); readMetadata = result(readFuture); - assertEquals(metadata, readMetadata); + assertEquals(metadata, readMetadata.getValue()); // ledger exists: delete metadata with wrong version @@ -193,13 +192,13 @@ public void testLedgerCRUD() throws Exception { readFuture = new GenericCallbackFuture<>(); lm.readLedgerMetadata(ledgerId, readFuture); readMetadata = result(readFuture); - assertEquals(metadata, readMetadata); + assertEquals(metadata, readMetadata.getValue()); // ledger exists: update metadata with the right version LongVersion curVersion = (LongVersion) readMetadata.getVersion(); writeFuture = new GenericCallbackFuture<>(); - lm.writeLedgerMetadata(ledgerId, readMetadata, writeFuture); + lm.writeLedgerMetadata(ledgerId, readMetadata.getValue(), curVersion, writeFuture); writtenMetadata = result(writeFuture); LongVersion newVersion = (LongVersion) writtenMetadata.getVersion(); assertTrue(curVersion.getLongVersion() < newVersion.getLongVersion()); @@ -266,9 +265,9 @@ public void testLedgerRangeIterator() throws Exception { } private void createNumLedgers(int numLedgers) throws Exception { - List> createFutures = new ArrayList<>(numLedgers); + List>> createFutures = new ArrayList<>(numLedgers); for (int i = 0; i < numLedgers; i++) { - GenericCallbackFuture createFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> createFuture = new GenericCallbackFuture<>(); createFutures.add(createFuture); LedgerMetadata metadata = new LedgerMetadata( 3, 3, 2, @@ -291,15 +290,15 @@ public void testRegisterLedgerMetadataListener() throws Exception { "test-password".getBytes(UTF_8) ); metadata.addEnsemble(0L, createNumBookies(3)); - GenericCallbackFuture createFuture = new GenericCallbackFuture<>(); + GenericCallbackFuture> createFuture = new GenericCallbackFuture<>(); lm.createLedgerMetadata(ledgerId, metadata, createFuture); result(createFuture); - LedgerMetadata readMetadata = readLedgerMetadata(ledgerId); - log.info("Create ledger metadata : {}", readMetadata); + Versioned readMetadata = readLedgerMetadata(ledgerId); + log.info("Create ledger metadata : {}", readMetadata.getValue()); // register first listener - LinkedBlockingQueue metadataQueue1 = new LinkedBlockingQueue<>(); + LinkedBlockingQueue> metadataQueue1 = new LinkedBlockingQueue<>(); LedgerMetadataListener listener1 = (lid, m) -> { log.info("[listener1] Received ledger {} metadata : {}", lid, m); metadataQueue1.add(m); @@ -307,7 +306,7 @@ public void testRegisterLedgerMetadataListener() throws Exception { log.info("Registered first listener for ledger {}", ledgerId); lm.registerLedgerMetadataListener(ledgerId, listener1); // we should receive a metadata notification when a ledger is created - LedgerMetadata notifiedMetadata = metadataQueue1.take(); + Versioned notifiedMetadata = metadataQueue1.take(); assertEquals(readMetadata, notifiedMetadata); ValueStream lms = lm.getLedgerMetadataStream(ledgerId); assertNotNull(lms.waitUntilWatched()); @@ -315,21 +314,21 @@ public void testRegisterLedgerMetadataListener() throws Exception { // register second listener - LinkedBlockingQueue metadataQueue2 = new LinkedBlockingQueue<>(); + LinkedBlockingQueue> metadataQueue2 = new LinkedBlockingQueue<>(); LedgerMetadataListener listener2 = (lid, m) -> { log.info("[listener2] Received ledger {} metadata : {}", lid, m); metadataQueue2.add(m); }; log.info("Registered second listener for ledger {}", ledgerId); lm.registerLedgerMetadataListener(ledgerId, listener2); - LedgerMetadata notifiedMetadata2 = metadataQueue2.take(); + Versioned notifiedMetadata2 = metadataQueue2.take(); assertEquals(readMetadata, notifiedMetadata2); assertNotNull(lm.getLedgerMetadataStream(ledgerId)); // update the metadata - metadata.setVersion(notifiedMetadata.getVersion()); - metadata.addEnsemble(10L, createNumBookies(3)); - writeLedgerMetadata(ledgerId, metadata); + writeLedgerMetadata(ledgerId, + LedgerMetadataBuilder.from(metadata).newEnsembleEntry(10L, createNumBookies(3)).build(), + notifiedMetadata.getVersion()); readMetadata = readLedgerMetadata(ledgerId); assertEquals(readMetadata, metadataQueue1.take()); assertEquals(readMetadata, metadataQueue2.take()); @@ -344,9 +343,9 @@ public void testRegisterLedgerMetadataListener() throws Exception { assertEquals(1, lms.getNumConsumers()); // update the metadata again - metadata.setVersion(readMetadata.getVersion()); - metadata.addEnsemble(20L, createNumBookies(3)); - writeLedgerMetadata(ledgerId, metadata); + writeLedgerMetadata(ledgerId, + LedgerMetadataBuilder.from(metadata).newEnsembleEntry(20L, createNumBookies(3)).build(), + readMetadata.getVersion()); readMetadata = readLedgerMetadata(ledgerId); assertEquals(readMetadata, metadataQueue1.take()); assertNull(metadataQueue2.poll()); @@ -360,9 +359,9 @@ public void testRegisterLedgerMetadataListener() throws Exception { assertEquals(0, lms.getNumConsumers()); // update the metadata again - metadata.setVersion(readMetadata.getVersion()); - metadata.addEnsemble(30L, createNumBookies(3)); - writeLedgerMetadata(ledgerId, metadata); + writeLedgerMetadata(ledgerId, + LedgerMetadataBuilder.from(metadata).newEnsembleEntry(30L, createNumBookies(3)).build(), + readMetadata.getVersion()); readMetadata = readLedgerMetadata(ledgerId); assertNull(metadataQueue1.poll()); assertNull(metadataQueue2.poll()); @@ -386,15 +385,15 @@ public void testRegisterLedgerMetadataListener() throws Exception { assertNull(metadataQueue2.poll()); } - LedgerMetadata readLedgerMetadata(long lid) throws Exception { - GenericCallbackFuture readFuture = new GenericCallbackFuture<>(); + Versioned readLedgerMetadata(long lid) throws Exception { + GenericCallbackFuture> readFuture = new GenericCallbackFuture<>(); lm.readLedgerMetadata(lid, readFuture); return result(readFuture); } - void writeLedgerMetadata(long lid, LedgerMetadata metadata) throws Exception { - GenericCallbackFuture writeFuture = new GenericCallbackFuture<>(); - lm.writeLedgerMetadata(lid, metadata, writeFuture); + void writeLedgerMetadata(long lid, LedgerMetadata metadata, Version version) throws Exception { + GenericCallbackFuture> writeFuture = new GenericCallbackFuture<>(); + lm.writeLedgerMetadata(lid, metadata, version, writeFuture); result(writeFuture); } diff --git a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy index 0818b2aa544..a4858e9a8d7 100644 --- a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy +++ b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy @@ -38,6 +38,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback import org.apache.bookkeeper.tests.integration.utils.BookKeeperClusterUtils import org.apache.bookkeeper.tests.integration.utils.DockerUtils import org.apache.bookkeeper.tests.integration.utils.MavenClassLoader +import org.apache.bookkeeper.versioning.Versioned import org.jboss.arquillian.junit.Arquillian import org.jboss.arquillian.test.api.ArquillianResource @@ -58,7 +59,7 @@ class TestCompatRecoveryNoPassword { DockerClient docker private LedgerMetadata getLedgerMetadata(BookKeeper bookkeeper, long ledgerId) throws Exception { - CompletableFuture future = new CompletableFuture() + CompletableFuture> future = new CompletableFuture<>() bookkeeper.getLedgerManager().readLedgerMetadata( ledgerId, { rc, result -> if (rc != BKException.Code.OK) { @@ -67,7 +68,7 @@ class TestCompatRecoveryNoPassword { future.complete(result) } }) - return future.get() + return future.get().getValue() } private static class ReplicationVerificationCallback implements ReadEntryCallback { From d89480861e0d778c7df773a5042212f855090b8f Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 13 Nov 2018 00:53:33 -0800 Subject: [PATCH 0117/1642] BP-36: Stats documentation annotation Descriptions of the changes in this PR: *Motivation* A common ask from people using bookkeeper is how they can monitor bookies and bookkeeper clients, what kind of metrics that bookkeeper exposes and what are the important metrics. Currently bookkeeper doesn't provide any metrics page for guiding people on monitoring bookkeeper services. In order to help people on this, we need to provide a few documentation pages about metrics. However if we just write such pages, those pages can quickly get out-of-dated when code is changed. The proposal here is to seek a programming way for generating metrics related pages. Master Issue: #1785 Reviewers: Sijie Guo , Jia Zhai , Enrico Olivelli , Andrey Yegorov This closes #1786 from sijie/bp_stats_generator --- .../BP-36-stats-documentation-annotation.md | 157 ++++++++++++++++++ site/community/bookkeeper_proposals.md | 5 +- 2 files changed, 160 insertions(+), 2 deletions(-) create mode 100644 site/bps/BP-36-stats-documentation-annotation.md diff --git a/site/bps/BP-36-stats-documentation-annotation.md b/site/bps/BP-36-stats-documentation-annotation.md new file mode 100644 index 00000000000..c8905ddc8fc --- /dev/null +++ b/site/bps/BP-36-stats-documentation-annotation.md @@ -0,0 +1,157 @@ +--- +title: "BP-36: Stats documentation annotation" +issue: https://github.com/apache/bookkeeper/ +state: "Accepted" +release: "4.9.0" +--- + +### Motivation + +A common ask from people using bookkeeper is how they can monitor bookies and bookkeeper clients, what kind of metrics that bookkeeper exposes +and what are the important metrics. Currently bookkeeper doesn't provide any metrics page for guiding people on monitoring bookkeeper services. + +In order to help people on this, we need to provide a few documentation pages about metrics. However if we just write such pages, those pages +can quickly get out-of-dated when code is changed. The proposal here is to seek a programming way for generating metrics related pages. + +### Public Interfaces + +Introduced a `StatsDoc` annotation. + +```bash +/** + * Documenting the stats. + */ +@Retention(RetentionPolicy.RUNTIME) +@Documented +public @interface StatsDoc { + + /** + * The name of the category to group stats together. + * + * @return name of the stats category. + */ + String category() default ""; + + /** + * The scope of this stats. + * + * @return scope of this stats + */ + String scope() default ""; + + /** + * The name of this stats + * + * @return name of this stats + */ + String name(); + + /** + * The help message of this stats + * + * @return help message of this stats + */ + String help(); + + /** + * The parent metric name. + * + *

    It can used for analyzing the relationships + * between the metrics, especially for the latency metrics. + * + * @return the parent metric name + */ + default String parent() { return ""; } + + /** + * The metric name of an operation that happens + * after the operation of this metric. + * + *

    similar as {@link #parent()}, it can be used for analyzing + * the dependencies between metrics. + * + * @return the metric name of an operation that happens after the operation of this metric. + */ + default String happensAfter() { return ""; } + + +} +``` + +The `StatsDoc` annotation provides a way to annotate metrics we added to bookkeeper. + +- category: which category that the metric belongs to. e.g. server, or client. +- scope: the scope of the metric. e.g. `bookie` scope. +- name: the name of the metric. +- help: the description of the metric. + +### Proposed Changes + +In addition to the `StatsDoc` annotation, bookkeeper should provide a tool for generating the metrics yaml file +for documenting all annotated metrics. + +```bash +bin/stats-doc-gen +``` + +Example output: + +```yaml +"server": + "bookie_BOOKIE_READ_ENTRY_BYTES": + "description": |- + bytes stats of ReadEntry on a bookie + "type": |- + OPSTATS + "bookie_WRITE_BYTES": + "description": |- + total bytes written to a bookie + "type": |- + COUNTER + "bookie_BOOKIE_ADD_ENTRY": + "description": |- + operations stats of AddEntry on a bookie + "type": |- + OPSTATS + "bookie_BOOKIE_RECOVERY_ADD_ENTRY": + "description": |- + operation stats of RecoveryAddEntry on a bookie + "type": |- + OPSTATS + "bookie_BOOKIE_ADD_ENTRY_BYTES": + "description": |- + bytes stats of AddEntry on a bookie + "type": |- + OPSTATS + "bookie_BOOKIE_FORCE_LEDGER": + "description": |- + total force operations occurred on a bookie + "type": |- + COUNTER + "bookie_READ_BYTES": + "description": |- + total bytes read from a bookie + "type": |- + COUNTER + "bookie_BOOKIE_READ_ENTRY": + "description": |- + operation stats of ReadEntry on a bookie + "type": |- + OPSTATS +``` + +### Compatibility, Deprecation, and Migration Plan + +It is a new feature, which doesn't have any compatibility impacts. + +There is nothing deprecated. + +There is nothing to migrate. + +### Test Plan + +Existing testing is good enough to cover code changes. No new tests are needed. + +### Rejected Alternatives + +Alternatively, we have to manually maintain the metrics page and update each time when a new metric is added. diff --git a/site/community/bookkeeper_proposals.md b/site/community/bookkeeper_proposals.md index 94bd6a5e00c..58244087bb4 100644 --- a/site/community/bookkeeper_proposals.md +++ b/site/community/bookkeeper_proposals.md @@ -85,7 +85,7 @@ using Google Doc. This section lists all the _bookkeeper proposals_ made to BookKeeper. -*Next Proposal Number: 36* +*Next Proposal Number: 37* ### Inprogress @@ -106,7 +106,8 @@ Proposal | State [BP-32: Advisory (optimistic) write close](../../bps/BP-32-advisory-write-close) | Accepted [BP-33: Move releasing docker images out of main repo](../../bps/BP-33-building-official-docker-imags) | Draft [BP-34: Cluster Metadata Checker](../../bps/BP-34-cluster-metadata-checker) | Accepted -[BP-35: 128 bits support](../../bps/BP-35-128-bits-support) | Under Discussion +[BP-35: 128 bits support](../../bps/BP-35-128-bits-support) | Accepted +[BP-36: Stats documentation annotation](../../bps/BP-36-stats-documentation-annotation) | Accepted ### Adopted From d683ccb00f2685d6028d9565dc3944a00d97e8d0 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 13 Nov 2018 00:54:56 -0800 Subject: [PATCH 0118/1642] [STATS] introduce `StatsDoc` annotation for better documenting metrics exposed by bookkeeper Descriptions of the changes in this PR: ### Motivation A common ask from people using bookkeeper is how they can monitor bookies and bookkeeper clients, what kind of metrics that bookkeeper exposes and what are the important metrics. Currently bookkeeper doesn't provide any metrics page for guiding people on monitoring bookkeeper services. In order to help people on this, we need to provide a few documentation pages about metrics. However if we just write such pages, those pages can quickly get out-of-dated when code is changed. ### Changes - Introduce an annotation `StatsDoc` for annotating the counters/gauges/opstats in the source code. - Provide a tool to generate the stats and their documentation into a yaml file. The yaml file will be used by the website for rendering a metrics reference page. ### Results ``` "server": "bookie_BOOKIE_READ_ENTRY_BYTES": "description": |- bytes stats of ReadEntry on a bookie "type": |- OPSTATS "bookie_WRITE_BYTES": "description": |- total bytes written to a bookie "type": |- COUNTER "bookie_BOOKIE_ADD_ENTRY": "description": |- operations stats of AddEntry on a bookie "type": |- OPSTATS "bookie_BOOKIE_RECOVERY_ADD_ENTRY": "description": |- operation stats of RecoveryAddEntry on a bookie "type": |- OPSTATS "bookie_BOOKIE_ADD_ENTRY_BYTES": "description": |- bytes stats of AddEntry on a bookie "type": |- OPSTATS "bookie_BOOKIE_FORCE_LEDGER": "description": |- total force operations occurred on a bookie "type": |- COUNTER "bookie_READ_BYTES": "description": |- total bytes read from a bookie "type": |- COUNTER "bookie_BOOKIE_READ_ENTRY": "description": |- operation stats of ReadEntry on a bookie "type": |- OPSTATS ``` Master Issue: #1786 Reviewers: Ivan Kelly , Jia Zhai This closes #1787 from sijie/stats_generator --- .../bookkeeper/test/TestStatsProvider.java | 12 + .../bookie/BookKeeperServerStats.java | 2 + .../org/apache/bookkeeper/bookie/Bookie.java | 62 ++-- .../bookkeeper/bookie/stats/BookieStats.java | 84 +++++ .../bookkeeper/bookie/stats/package-info.java | 23 ++ .../stats/CodahaleMetricsProvider.java | 11 + .../codahale/CodahaleMetricsProvider.java | 11 + .../prometheus/PrometheusMetricsProvider.java | 21 +- .../twitter/finagle/FinagleStatsProvider.java | 5 + .../twitter/ostrich/OstrichProvider.java | 5 + .../twitter/science/TwitterStatsProvider.java | 11 + .../stats/CachingStatsProvider.java | 5 + .../org/apache/bookkeeper/stats/Stats.java | 4 + .../bookkeeper/stats/StatsProvider.java | 11 + .../stats/annotations/StatsDoc.java | 62 ++++ .../stats/annotations/package-info.java | 23 ++ dev/stats-doc-gen | 66 ++++ pom.xml | 18 ++ stats/pom.xml | 39 +++ stats/utils/pom.xml | 55 ++++ .../stats/utils/StatsDocGenerator.java | 298 ++++++++++++++++++ .../bookkeeper/stats/utils/package-info.java | 23 ++ 22 files changed, 806 insertions(+), 45 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/package-info.java create mode 100644 bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/StatsDoc.java create mode 100644 bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/package-info.java create mode 100755 dev/stats-doc-gen create mode 100644 stats/pom.xml create mode 100644 stats/utils/pom.xml create mode 100644 stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/StatsDocGenerator.java create mode 100644 stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/package-info.java diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java index aca154ec0c2..bef0dfeee68 100644 --- a/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java @@ -31,6 +31,7 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.commons.configuration.Configuration; +import org.apache.commons.lang.StringUtils; /** * Simple in-memory stat provider for use in unit tests. @@ -261,4 +262,15 @@ private void registerGauge(String name, Gauge gauge) { private void unregisterGauge(String name, Gauge gauge) { gaugeMap.remove(name, gauge); } + + @Override + public String getStatsName(String... statsComponents) { + if (statsComponents.length == 0) { + return ""; + } else if (statsComponents[0].isEmpty()) { + return StringUtils.join(statsComponents, '.', 1, statsComponents.length); + } else { + return StringUtils.join(statsComponents, '.'); + } + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java index e048bd732ee..736d34122c7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java @@ -25,6 +25,8 @@ */ public interface BookKeeperServerStats { + String CATEGORY_SERVER = "server"; + String SERVER_SCOPE = "bookkeeper_server"; String BOOKIE_SCOPE = "bookie"; 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 3795b37b91b..d0db80e26ff 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,18 +21,9 @@ package org.apache.bookkeeper.bookie; -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_FORCE_LEDGER; -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.JOURNAL_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_BYTES; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_BYTES; -import static org.apache.bookkeeper.bookie.Bookie.METAENTRY_ID_FENCE_KEY; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; @@ -71,6 +62,7 @@ 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.bookie.stats.BookieStats; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationManager; @@ -82,9 +74,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNS; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; -import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.DiskChecker; @@ -142,16 +132,7 @@ public class Bookie extends BookieCriticalThread { // Expose Stats final StatsLogger statsLogger; - private final Counter writeBytes; - private final Counter readBytes; - private final Counter forceLedgerOps; - // Bookie Operation Latency Stats - private final OpStatsLogger addEntryStats; - private final OpStatsLogger recoveryAddEntryStats; - private final OpStatsLogger readEntryStats; - // Bookie Operation Bytes Stats - private final OpStatsLogger addBytesStats; - private final OpStatsLogger readBytesStats; + private final BookieStats bookieStats; /** * Exception is thrown when no such a ledger is found in this bookie. @@ -744,14 +725,7 @@ public void start() { handles = new HandleFactoryImpl(ledgerStorage); // Expose Stats - writeBytes = statsLogger.getCounter(WRITE_BYTES); - readBytes = statsLogger.getCounter(READ_BYTES); - forceLedgerOps = statsLogger.getCounter(BOOKIE_FORCE_LEDGER); - addEntryStats = statsLogger.getOpStatsLogger(BOOKIE_ADD_ENTRY); - recoveryAddEntryStats = statsLogger.getOpStatsLogger(BOOKIE_RECOVERY_ADD_ENTRY); - readEntryStats = statsLogger.getOpStatsLogger(BOOKIE_READ_ENTRY); - addBytesStats = statsLogger.getOpStatsLogger(BOOKIE_ADD_ENTRY_BYTES); - readBytesStats = statsLogger.getOpStatsLogger(BOOKIE_READ_ENTRY_BYTES); + this.bookieStats = new BookieStats(statsLogger); } StateManager initializeStateManager() throws IOException { @@ -1163,7 +1137,7 @@ private void addEntryInternal(LedgerDescriptor handle, ByteBuf entry, long ledgerId = handle.getLedgerId(); long entryId = handle.addEntry(entry); - writeBytes.add(entry.readableBytes()); + bookieStats.getWriteBytes().add(entry.readableBytes()); // journal `addEntry` should happen after the entry is added to ledger storage. // otherwise the journal entry can potentially be rolled before the ledger is created in ledger storage. @@ -1213,11 +1187,11 @@ public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] } finally { long elapsedNanos = MathUtils.elapsedNanos(requestNanos); if (success) { - recoveryAddEntryStats.registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS); - addBytesStats.registerSuccessfulValue(entrySize); + bookieStats.getRecoveryAddEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS); + bookieStats.getAddBytesStats().registerSuccessfulValue(entrySize); } else { - recoveryAddEntryStats.registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS); - addBytesStats.registerFailedValue(entrySize); + bookieStats.getRecoveryAddEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS); + bookieStats.getAddBytesStats().registerFailedValue(entrySize); } entry.release(); @@ -1272,7 +1246,7 @@ public void forceLedger(long ledgerId, WriteCallback cb, } Journal journal = getJournal(ledgerId); journal.forceLedger(ledgerId, cb, ctx); - forceLedgerOps.inc(); + bookieStats.getForceLedgerOps().inc(); } /** @@ -1301,11 +1275,11 @@ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Obj } finally { long elapsedNanos = MathUtils.elapsedNanos(requestNanos); if (success) { - addEntryStats.registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS); - addBytesStats.registerSuccessfulValue(entrySize); + bookieStats.getAddEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS); + bookieStats.getAddBytesStats().registerSuccessfulValue(entrySize); } else { - addEntryStats.registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS); - addBytesStats.registerFailedValue(entrySize); + bookieStats.getAddEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS); + bookieStats.getAddBytesStats().registerFailedValue(entrySize); } entry.release(); @@ -1355,17 +1329,17 @@ public ByteBuf readEntry(long ledgerId, long entryId) LOG.trace("Reading {}@{}", entryId, ledgerId); } ByteBuf entry = handle.readEntry(entryId); - readBytes.add(entry.readableBytes()); + bookieStats.getReadBytes().add(entry.readableBytes()); success = true; return entry; } finally { long elapsedNanos = MathUtils.elapsedNanos(requestNanos); if (success) { - readEntryStats.registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS); - readBytesStats.registerSuccessfulValue(entrySize); + bookieStats.getReadEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS); + bookieStats.getReadBytesStats().registerSuccessfulValue(entrySize); } else { - readEntryStats.registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS); - readBytesStats.registerFailedValue(entrySize); + bookieStats.getReadEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS); + bookieStats.getReadEntryStats().registerFailedValue(entrySize); } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java new file mode 100644 index 00000000000..72921d72971 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java @@ -0,0 +1,84 @@ +/* + * 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.stats; + +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_FORCE_LEDGER; +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.CATEGORY_SERVER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_BYTES; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_BYTES; + +import lombok.Getter; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for bookie related stats. + */ +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "Bookie related stats" +) +@Getter +public class BookieStats { + + // Expose Stats + final StatsLogger statsLogger; + @StatsDoc(name = WRITE_BYTES, help = "total bytes written to a bookie") + private final Counter writeBytes; + @StatsDoc(name = READ_BYTES, help = "total bytes read from a bookie") + private final Counter readBytes; + @StatsDoc(name = BOOKIE_FORCE_LEDGER, help = "total force operations occurred on a bookie") + private final Counter forceLedgerOps; + // Bookie Operation Latency Stats + @StatsDoc(name = BOOKIE_ADD_ENTRY, help = "operations stats of AddEntry on a bookie") + private final OpStatsLogger addEntryStats; + @StatsDoc(name = BOOKIE_RECOVERY_ADD_ENTRY, help = "operation stats of RecoveryAddEntry on a bookie") + private final OpStatsLogger recoveryAddEntryStats; + @StatsDoc(name = BOOKIE_READ_ENTRY, help = "operation stats of ReadEntry on a bookie") + private final OpStatsLogger readEntryStats; + // Bookie Operation Bytes Stats + @StatsDoc(name = BOOKIE_ADD_ENTRY_BYTES, help = "bytes stats of AddEntry on a bookie") + private final OpStatsLogger addBytesStats; + @StatsDoc(name = BOOKIE_READ_ENTRY_BYTES, help = "bytes stats of ReadEntry on a bookie") + private final OpStatsLogger readBytesStats; + + public BookieStats(StatsLogger statsLogger) { + this.statsLogger = statsLogger; + writeBytes = statsLogger.getCounter(WRITE_BYTES); + readBytes = statsLogger.getCounter(READ_BYTES); + forceLedgerOps = statsLogger.getCounter(BOOKIE_FORCE_LEDGER); + addEntryStats = statsLogger.getOpStatsLogger(BOOKIE_ADD_ENTRY); + recoveryAddEntryStats = statsLogger.getOpStatsLogger(BOOKIE_RECOVERY_ADD_ENTRY); + readEntryStats = statsLogger.getOpStatsLogger(BOOKIE_READ_ENTRY); + addBytesStats = statsLogger.getOpStatsLogger(BOOKIE_ADD_ENTRY_BYTES); + readBytesStats = statsLogger.getOpStatsLogger(BOOKIE_READ_ENTRY_BYTES); + } + + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/package-info.java new file mode 100644 index 00000000000..99261760435 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/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. + */ + +/** + * Package of the classes for defining bookie stats. + */ +package org.apache.bookkeeper.bookie.stats; \ No newline at end of file diff --git a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsProvider.java b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsProvider.java index 01658c7652e..bbf7bcd0524 100644 --- a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsProvider.java +++ b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsProvider.java @@ -140,4 +140,15 @@ public StatsLogger getStatsLogger(String name) { initIfNecessary(); return new CodahaleStatsLogger(getMetrics(), name); } + + @Override + public String getStatsName(String... statsComponents) { + if (statsComponents.length == 0) { + return ""; + } + String baseName = statsComponents[0]; + String[] names = new String[statsComponents.length - 1]; + System.arraycopy(statsComponents, 1, names, 0, names.length); + return MetricRegistry.name(baseName, names); + } } diff --git a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/CodahaleMetricsProvider.java b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/CodahaleMetricsProvider.java index f4ca952f259..1bd5b185efa 100644 --- a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/CodahaleMetricsProvider.java +++ b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/CodahaleMetricsProvider.java @@ -141,4 +141,15 @@ public StatsLogger getStatsLogger(String name) { initIfNecessary(); return new CodahaleStatsLogger(getMetrics(), name); } + + @Override + public String getStatsName(String... statsComponents) { + if (statsComponents.length == 0) { + return ""; + } + String baseName = statsComponents[0]; + String[] names = new String[statsComponents.length - 1]; + System.arraycopy(statsComponents, 1, names, 0, names.length); + return MetricRegistry.name(baseName, names); + } } diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java index df8279e68b0..0b4c0c2111e 100644 --- a/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java +++ b/bookkeeper-stats-providers/prometheus-metrics-provider/src/main/java/org/apache/bookkeeper/stats/prometheus/PrometheusMetricsProvider.java @@ -46,6 +46,7 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.commons.configuration.Configuration; +import org.apache.commons.lang.StringUtils; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; @@ -101,6 +102,19 @@ public void stop() { public StatsLogger getStatsLogger(String scope) { return new PrometheusStatsLogger(PrometheusMetricsProvider.this, scope); } + + @Override + public String getStatsName(String... statsComponents) { + String completeName; + if (statsComponents.length == 0) { + return ""; + } else if (statsComponents[0].isEmpty()) { + completeName = StringUtils.join(statsComponents, '_', 1, statsComponents.length); + } else { + completeName = StringUtils.join(statsComponents, '_'); + } + return Collector.sanitizeMetricName(completeName); + } }); } @@ -184,6 +198,11 @@ public void writeAllMetrics(Writer writer) throws IOException { opStats.forEach((name, opStatLogger) -> PrometheusTextFormatUtil.writeOpStat(writer, name, opStatLogger)); } + @Override + public String getStatsName(String... statsComponents) { + return cachingStatsProvider.getStatsName(statsComponents); + } + @VisibleForTesting void rotateLatencyCollection() { opStats.forEach((name, metric) -> { @@ -222,4 +241,4 @@ private void registerMetrics(Collector collector) { directMemoryUsage = tmpDirectMemoryUsage; } -} \ No newline at end of file +} diff --git a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsProvider.java b/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsProvider.java index aff129d9846..a3b569cc5b5 100644 --- a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsProvider.java +++ b/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsProvider.java @@ -64,4 +64,9 @@ public void stop() { /* no-op */ } public StatsLogger getStatsLogger(final String scope) { return this.cachingStatsProvider.getStatsLogger(scope); } + + @Override + public String getStatsName(String... statsComponents) { + return cachingStatsProvider.getStatsName(statsComponents); + } } diff --git a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichProvider.java b/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichProvider.java index 1cc7bf6958f..173b20022c1 100644 --- a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichProvider.java +++ b/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichProvider.java @@ -118,4 +118,9 @@ public void stop() { public StatsLogger getStatsLogger(String scope) { return cachingStatsProvider.getStatsLogger(scope); } + + @Override + public String getStatsName(String... statsComponents) { + return cachingStatsProvider.getStatsName(statsComponents); + } } diff --git a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsProvider.java b/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsProvider.java index 75c2842c986..2a410c03b03 100644 --- a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsProvider.java +++ b/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsProvider.java @@ -20,6 +20,7 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.commons.configuration.Configuration; +import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,6 +54,11 @@ public void stop() { public StatsLogger getStatsLogger(String scope) { return new TwitterStatsLoggerImpl(scope); } + + @Override + public String getStatsName(String... statsComponents) { + return StringUtils.join(statsComponents, '_').toLowerCase(); + } }); } @@ -85,4 +91,9 @@ public void stop() { public StatsLogger getStatsLogger(String name) { return this.cachingStatsProvider.getStatsLogger(name); } + + @Override + public String getStatsName(String... statsComponents) { + return this.cachingStatsProvider.getStatsName(statsComponents); + } } diff --git a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/CachingStatsProvider.java b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/CachingStatsProvider.java index 4d38e9eef7c..e3fa3aa69cc 100644 --- a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/CachingStatsProvider.java +++ b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/CachingStatsProvider.java @@ -57,4 +57,9 @@ public StatsLogger getStatsLogger(String scope) { } return statsLogger; } + + @Override + public String getStatsName(String... statsComponents) { + return underlying.getStatsName(statsComponents); + } } diff --git a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/Stats.java b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/Stats.java index a8115b352c6..a3799b08e5f 100644 --- a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/Stats.java +++ b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/Stats.java @@ -37,6 +37,10 @@ public class Stats { public static void loadStatsProvider(Configuration conf) { String className = conf.getString(STATS_PROVIDER_CLASS); + loadStatsProvider(className); + } + + public static void loadStatsProvider(String className) { if (className != null) { try { Class cls = Class.forName(className); diff --git a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java index b6e34600d75..0bb236a6e5e 100644 --- a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java +++ b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/StatsProvider.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.io.Writer; import org.apache.commons.configuration.Configuration; +import org.apache.commons.lang.StringUtils; /** * Provider to provide stats logger for different scopes. @@ -53,4 +54,14 @@ default void writeAllMetrics(Writer writer) throws IOException { * @return stats logger for the given scope */ StatsLogger getStatsLogger(String scope); + + /** + * Return the fully qualified stats name comprised of given statsComponents. + * + * @param statsComponents stats components to comprise the fully qualified stats name + * @return the fully qualified stats name + */ + default String getStatsName(String...statsComponents) { + return StringUtils.join(statsComponents, '/'); + } } diff --git a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/StatsDoc.java b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/StatsDoc.java new file mode 100644 index 00000000000..97f487a69a1 --- /dev/null +++ b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/StatsDoc.java @@ -0,0 +1,62 @@ +/* + * 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.stats.annotations; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Documenting the stats. + */ +@Retention(RetentionPolicy.RUNTIME) +@Documented +public @interface StatsDoc { + + /** + * The name of the category to group stats together. + * + * @return name of the stats category. + */ + String category() default ""; + + /** + * The scope of this stats. + * + * @return scope of this stats + */ + String scope() default ""; + + /** + * The name of this stats. + * + * @return name of this stats + */ + String name(); + + /** + * The help message of this stats. + * + * @return help message of this stats + */ + String help(); + + +} diff --git a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/package-info.java b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/package-info.java new file mode 100644 index 00000000000..b8daf752023 --- /dev/null +++ b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/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. + */ + +/** + * Annotations for bookkeeper stats api. + */ +package org.apache.bookkeeper.stats.annotations; \ No newline at end of file diff --git a/dev/stats-doc-gen b/dev/stats-doc-gen new file mode 100755 index 00000000000..3a20aef8a59 --- /dev/null +++ b/dev/stats-doc-gen @@ -0,0 +1,66 @@ +#!/usr/bin/env bash +# +#/** +# * 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. +# */ + +# Stats Documentation Generator + +BINDIR=`dirname "$0"` +BK_HOME=`cd ${BINDIR}/..;pwd` + +source ${BK_HOME}/bin/common.sh +source ${BK_HOME}/conf/bk_cli_env.sh + +CLI_MODULE_PATH=stats/utils +CLI_MODULE_NAME="(org.apache.bookkeeper.stats-)?bookkeeper-stats-utils" +CLI_MODULE_HOME=${BK_HOME}/${CLI_MODULE_PATH} + +# find the module jar +CLI_JAR=$(find_module_jar ${CLI_MODULE_PATH} ${CLI_MODULE_NAME}) + +# set up the classpath +CLI_CLASSPATH=$(set_module_classpath ${CLI_MODULE_PATH}) + +DEFAULT_LOG_CONF=${BK_HOME}/conf/log4j.cli.properties +if [ -z "${CLI_LOG_CONF}" ]; then + CLI_LOG_CONF=${DEFAULT_LOG_CONF} +fi +CLI_LOG_DIR=${CLI_LOG_DIR:-"$BK_HOME/logs"} +CLI_LOG_FILE=${CLI_LOG_FILE:-"stats-doc-gen.log"} +CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,ROLLINGFILE"} + +# add all dependencies in the classpath +ALL_MODULE_PATH=bookkeeper-dist/all +ALL_MODULE_CLASSPATH=$(set_module_classpath ${ALL_MODULE_PATH}) + +# Configure the classpath +CLI_CLASSPATH="$CLI_JAR:$CLI_CLASSPATH:$CLI_EXTRA_CLASSPATH:$ALL_MODULE_CLASSPATH" +CLI_CLASSPATH="`dirname $CLI_LOG_CONF`:$CLI_CLASSPATH" + +# Build the OPTs +BOOKIE_OPTS=$(build_bookie_opts) +GC_OPTS=$(build_cli_jvm_opts ${CLI_LOG_DIR} "stats-doc-gen-gc.log") +NETTY_OPTS=$(build_netty_opts) +LOGGING_OPTS=$(build_cli_logging_opts ${CLI_LOG_CONF} ${CLI_LOG_DIR} ${CLI_LOG_FILE} ${CLI_ROOT_LOGGER}) + +OPTS="${OPTS} -cp ${CLI_CLASSPATH} ${BOOKIE_OPTS} ${GC_OPTS} ${NETTY_OPTS} ${LOGGING_OPTS} ${CLI_EXTRA_OPTS}" + +#Change to BK_HOME to support relative paths +cd "$BK_HOME" +echo "running stats-doc-gen, logging to ${CLI_LOG_DIR}/${CLI_LOG_FILE}" +exec ${JAVA} ${OPTS} org.apache.bookkeeper.stats.utils.StatsDocGenerator $@ diff --git a/pom.xml b/pom.xml index 25624b24040..5c6d9eb6100 100644 --- a/pom.xml +++ b/pom.xml @@ -55,6 +55,8 @@ circe-checksum bookkeeper-common bookkeeper-common-allocator + stats + bookkeeper-stats bookkeeper-proto bookkeeper-server @@ -156,9 +158,11 @@ 3.5.1 3.5.1-1 1.12.0 + 0.9.11 5.13.1 3.0.1 1.7.25 + 1.23 3.1.1 1.3.2 1.8.3 @@ -297,6 +301,13 @@ ${commons-lang3.version} + + + org.reflections + reflections + ${reflections.version} + + net.jpountz.lz4 @@ -311,6 +322,13 @@ ${jna.version} + + + org.yaml + snakeyaml + ${snakeyaml.version} + + com.fasterxml.jackson.core diff --git a/stats/pom.xml b/stats/pom.xml new file mode 100644 index 00000000000..c699ed4e17c --- /dev/null +++ b/stats/pom.xml @@ -0,0 +1,39 @@ + + + + 4.0.0 + + org.apache.bookkeeper + bookkeeper + 4.9.0-SNAPSHOT + .. + + pom + org.apache.bookkeeper.stats + bookkeeper-stats-parent + Apache BookKeeper :: Stats :: Parent + + + utils + + + + diff --git a/stats/utils/pom.xml b/stats/utils/pom.xml new file mode 100644 index 00000000000..9be19cfcbb4 --- /dev/null +++ b/stats/utils/pom.xml @@ -0,0 +1,55 @@ + + + + 4.0.0 + + bookkeeper-stats-parent + org.apache.bookkeeper.stats + 4.9.0-SNAPSHOT + .. + + org.apache.bookkeeper.stats + bookkeeper-stats-utils + Apache BookKeeper :: Stats :: Utils + + + org.apache.bookkeeper.stats + bookkeeper-stats-api + ${project.version} + + + org.reflections + reflections + + + org.yaml + snakeyaml + + + com.beust + jcommander + + + com.fasterxml.jackson.core + jackson-annotations + + + diff --git a/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/StatsDocGenerator.java b/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/StatsDocGenerator.java new file mode 100644 index 00000000000..db2e13f7f0f --- /dev/null +++ b/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/StatsDocGenerator.java @@ -0,0 +1,298 @@ +/* + * 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.stats.utils; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.google.common.base.Predicate; +import com.google.common.base.Strings; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.Writer; +import java.lang.reflect.Field; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.Stats; +import org.apache.bookkeeper.stats.StatsProvider; +import org.apache.bookkeeper.stats.annotations.StatsDoc; +import org.reflections.Reflections; +import org.reflections.util.ConfigurationBuilder; +import org.reflections.util.FilterBuilder; +import org.yaml.snakeyaml.DumperOptions; +import org.yaml.snakeyaml.DumperOptions.FlowStyle; +import org.yaml.snakeyaml.DumperOptions.ScalarStyle; +import org.yaml.snakeyaml.Yaml; + +/** + * Generator stats documentation. + */ +@Slf4j +public class StatsDocGenerator { + + enum StatsType { + COUNTER, + GAUGE, + OPSTATS + } + + @AllArgsConstructor + @Data + static class StatsDocEntry { + private String name; + private StatsType type; + private String description; + + public Map properties() { + Map properties = new TreeMap<>(); + properties.put("type", type.name()); + properties.put("description", description); + return properties; + } + } + + private static Reflections newReflections(String packagePrefix) { + List urls = new ArrayList<>(); + ClassLoader[] classLoaders = new ClassLoader[] { + StatsDocGenerator.class.getClassLoader(), + Thread.currentThread().getContextClassLoader() + }; + for (int i = 0; i < classLoaders.length; i++) { + if (classLoaders[i] instanceof URLClassLoader) { + urls.addAll(Arrays.asList(((URLClassLoader) classLoaders[i]).getURLs())); + } else { + throw new RuntimeException("ClassLoader '" + classLoaders[i] + " is not an instance of URLClassLoader"); + } + } + Predicate filters = new FilterBuilder() + .includePackage(packagePrefix); + ConfigurationBuilder confBuilder = new ConfigurationBuilder(); + confBuilder.filterInputsBy(filters); + confBuilder.setUrls(urls); + return new Reflections(confBuilder); + } + + private final String packagePrefix; + private final Reflections reflections; + private final StatsProvider statsProvider; + private final NavigableMap> docEntries = new TreeMap<>(); + + public StatsDocGenerator(String packagePrefix, + StatsProvider provider) { + this.packagePrefix = packagePrefix; + this.reflections = newReflections(packagePrefix); + this.statsProvider = provider; + } + + public void generate(String filename) throws Exception { + log.info("Processing classes under package {}", packagePrefix); + // get all classes annotated with `StatsDoc` + Set> annotatedClasses = reflections.getTypesAnnotatedWith(StatsDoc.class); + log.info("Retrieve all `StatsDoc` annotated classes : {}", annotatedClasses); + + for (Class annotatedClass : annotatedClasses) { + generateDocForAnnotatedClass(annotatedClass); + } + log.info("Successfully processed classes under package {}", packagePrefix); + log.info("Writing stats doc to file {}", filename); + writeDoc(filename); + log.info("Successfully wrote stats doc to file {}", filename); + } + + private void generateDocForAnnotatedClass(Class annotatedClass) { + StatsDoc scopeStatsDoc = annotatedClass.getDeclaredAnnotation(StatsDoc.class); + if (scopeStatsDoc == null) { + return; + } + + log.info("Processing StatsDoc annotated class {} : {}", annotatedClass, scopeStatsDoc); + + Field[] fields = annotatedClass.getDeclaredFields(); + for (Field field : fields) { + StatsDoc fieldStatsDoc = field.getDeclaredAnnotation(StatsDoc.class); + if (null == fieldStatsDoc) { + // it is not a `StatsDoc` annotated field + continue; + } + generateDocForAnnotatedField(scopeStatsDoc, fieldStatsDoc, field); + } + + log.info("Successfully processed StatsDoc annotated class {}.", annotatedClass); + } + + private NavigableMap getCategoryMap(String category) { + NavigableMap categoryMap = docEntries.get(category); + if (null == categoryMap) { + categoryMap = new TreeMap<>(); + docEntries.put(category, categoryMap); + } + return categoryMap; + } + + private void generateDocForAnnotatedField(StatsDoc scopedStatsDoc, StatsDoc fieldStatsDoc, Field field) { + NavigableMap categoryMap = getCategoryMap(scopedStatsDoc.category()); + + String statsName = + statsProvider.getStatsName(scopedStatsDoc.scope(), scopedStatsDoc.name(), fieldStatsDoc.name()); + StatsType statsType; + if (Counter.class.isAssignableFrom(field.getType())) { + statsType = StatsType.COUNTER; + } else if (Gauge.class.isAssignableFrom(field.getType())) { + statsType = StatsType.GAUGE; + } else if (OpStatsLogger.class.isAssignableFrom(field.getType())) { + statsType = StatsType.OPSTATS; + } else { + throw new IllegalArgumentException("Unknown stats field '" + field.getName() + + "' is annotated with `StatsDoc`: " + field.getType()); + } + + String helpDesc = fieldStatsDoc.help(); + StatsDocEntry docEntry = new StatsDocEntry(statsName, statsType, helpDesc); + categoryMap.put(statsName, docEntry); + } + + private void writeDoc(String file) throws IOException { + DumperOptions options = new DumperOptions(); + options.setDefaultFlowStyle(FlowStyle.BLOCK); + options.setDefaultScalarStyle(ScalarStyle.LITERAL); + Yaml yaml = new Yaml(options); + Writer writer; + if (Strings.isNullOrEmpty(file)) { + writer = new OutputStreamWriter(System.out, UTF_8); + } else { + writer = new OutputStreamWriter(new FileOutputStream(file), UTF_8); + } + try { + Map>> docs = docEntries.entrySet() + .stream() + .collect(Collectors.toMap( + e -> e.getKey(), + e -> e.getValue().entrySet() + .stream() + .collect(Collectors.toMap( + e1 -> e1.getKey(), + e1 -> e1.getValue().properties() + )) + )); + yaml.dump(docs, writer); + writer.flush(); + } finally { + writer.close(); + } + } + + /** + * Args for stats generator. + */ + private static class MainArgs { + + @Parameter( + names = { + "-p", "--package" + }, + description = "Package prefix of the classes to generate stats doc") + String packagePrefix = "org.apache.bookkeeper"; + + @Parameter( + names = { + "-sp", "--stats-provider" + }, + description = "The stats provider used for generating stats doc") + String statsProviderClass = "prometheus"; + + @Parameter( + names = { + "-o", "--output-yaml-file" + }, + description = "The output yaml file to dump stats docs." + + " If omitted, the output goes to stdout." + ) + String yamlFile = null; + + @Parameter( + names = { + "-h", "--help" + }, + description = "Show this help message") + boolean help = false; + + } + + public static void main(String[] args) throws Exception { + MainArgs mainArgs = new MainArgs(); + + JCommander commander = new JCommander(); + try { + commander.setProgramName("stats-doc-gen"); + commander.addObject(mainArgs); + commander.parse(args); + if (mainArgs.help) { + commander.usage(); + Runtime.getRuntime().exit(0); + return; + } + } catch (Exception e) { + commander.usage(); + Runtime.getRuntime().exit(-1); + return; + } + + Stats.loadStatsProvider(getStatsProviderClass(mainArgs.statsProviderClass)); + StatsProvider provider = Stats.get(); + + StatsDocGenerator docGen = new StatsDocGenerator( + mainArgs.packagePrefix, + provider + ); + docGen.generate(mainArgs.yamlFile); + } + + private static String getStatsProviderClass(String providerClass) { + switch (providerClass.toLowerCase()) { + case "ostrich": + return "org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider"; + case "prometheus": + return "org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider"; + case "finagle": + return "org.apache.bookkeeper.stats.twitter.finagle.FinagleStatsProvider"; + case "codahale": + return "org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider"; + default: + return providerClass; + } + } + +} diff --git a/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/package-info.java b/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/package-info.java new file mode 100644 index 00000000000..c0b248fa1af --- /dev/null +++ b/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/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. + */ + +/** + * Utilities for bookkeeper stats api. + */ +package org.apache.bookkeeper.stats.utils; \ No newline at end of file From 16f6dd92c9e381900ac6977c86418bdacf7ff4f5 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 13 Nov 2018 10:35:24 -0800 Subject: [PATCH 0119/1642] Added module to enable CPU affinity ### Motivation This is part of a set of changes aimed at reducing latency in BK at the expense of other aspects (eg: max throughput). While not intended to be used as default settings, they might be good to have whenever the latency becomes critical. Pinning a thread to a particular CPU will ensure no other process will execute on that CPU reducing all scheduler induced context switches that will cause latency jittery. A given thread that wants to get pinned to a CPU just needs to call: ```java CpuAffinity.acquireCore(); ``` It's called `acquireCore()` because it will also disable hyper-threading on the pinned cpu. Subsequent PRs will use this module to have the option to pin critical threads to available CPUs. ### Changes * Added JNI module to call `sched_setaffinity()` to pin a thread to a particular CPU * Automatically discover available isolated CPUs * Acquire file-based locks to allow multiple processes on same machine to acquire CPUs independently. Reviewers: Ivan Kelly , Enrico Olivelli , Sijie Guo This closes #1641 from merlimat/cpu-affinity --- .../scurrilous/circe/utils/NativeUtils.java | 15 +- cpu-affinity/pom.xml | 181 +++++ .../src/main/affinity/cpp/affinity_jni.c | 96 +++ cpu-affinity/src/main/assembly/assembly.xml | 62 ++ .../common/util/affinity/CpuAffinity.java | 51 ++ .../util/affinity/impl/CpuAffinityImpl.java | 218 ++++++ .../util/affinity/impl/CpuAffinityJni.java | 37 + .../affinity/impl/IsolatedProcessors.java | 102 +++ .../util/affinity/impl/NativeUtils.java | 86 +++ .../util/affinity/impl/ProcessorsInfo.java | 99 +++ .../util/affinity/impl/package-info.java | 22 + .../common/util/affinity/package-info.java | 22 + .../util/affinity/impl/CpuInfoTest.java | 53 ++ .../affinity/impl/IsolatedProcessorsTest.java | 47 ++ .../src/test/resources/proc_cpuinfo.txt | 648 ++++++++++++++++++ pom.xml | 3 +- 16 files changed, 1735 insertions(+), 7 deletions(-) create mode 100644 cpu-affinity/pom.xml create mode 100644 cpu-affinity/src/main/affinity/cpp/affinity_jni.c create mode 100644 cpu-affinity/src/main/assembly/assembly.xml create mode 100644 cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/CpuAffinity.java create mode 100644 cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityImpl.java create mode 100644 cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityJni.java create mode 100644 cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java create mode 100644 cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/NativeUtils.java create mode 100644 cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java create mode 100644 cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/package-info.java create mode 100644 cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/package-info.java create mode 100644 cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/CpuInfoTest.java create mode 100644 cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessorsTest.java create mode 100644 cpu-affinity/src/test/resources/proc_cpuinfo.txt diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java b/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java index 8776092d583..05ace295801 100644 --- a/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java +++ b/circe-checksum/src/main/java/com/scurrilous/circe/utils/NativeUtils.java @@ -37,7 +37,7 @@ public class NativeUtils { /** * loads given library from the this jar. ie: this jar contains: /lib/pulsar-checksum.jnilib - * + * * @param path * : absolute path of the library in the jar
    * if this jar contains: /lib/pulsar-checksum.jnilib then provide the same absolute path as input @@ -67,13 +67,16 @@ public static void loadLibraryFromJar(String path) throws Exception { throw new FileNotFoundException("Couldn't find file into jar " + path); } - OutputStream out = new FileOutputStream(temp); try { - while ((read = input.read(buffer)) != -1) { - out.write(buffer, 0, read); + OutputStream out = new FileOutputStream(temp); + try { + while ((read = input.read(buffer)) != -1) { + out.write(buffer, 0, read); + } + } finally { + out.close(); } } finally { - out.close(); input.close(); } @@ -87,7 +90,7 @@ public static void loadLibraryFromJar(String path) throws Exception { /** * Returns jni library extension based on OS specification. Maven-nar generates jni library based on different OS : * http://mark.donszelmann.org/maven-nar-plugin/aol.html (jni.extension) - * + * * @return */ public static String libType() { diff --git a/cpu-affinity/pom.xml b/cpu-affinity/pom.xml new file mode 100644 index 00000000000..f58a07991ef --- /dev/null +++ b/cpu-affinity/pom.xml @@ -0,0 +1,181 @@ + + + 4.0.0 + + org.apache.bookkeeper + bookkeeper + 4.9.0-SNAPSHOT + .. + + + cpu-affinity + nar + Apache BookKeeper :: CPU Affinity Library + CPU Affinity Library + + + dynamic + + + + + com.google.guava + guava + + + org.apache.commons + commons-lang3 + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + -Xlint:deprecation + -Xlint:unchecked + + -Xpkginfo:always + + + + + com.github.maven-nar + nar-maven-plugin + ${nar-maven-plugin.version} + true + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/main/assembly/assembly.xml + + false + posix + + + + make-assembly + package + + single + + + + + + org.apache.rat + apache-rat-plugin + + + + **/src/test/resources/proc_cpuinfo.txt + + + + + + + + + mac + + + Mac OS X + + + + + + com.github.maven-nar + nar-maven-plugin + ${nar-maven-plugin.version} + true + + ${nar.runtime} + cpu-affinity + + + jni + org.apache.bookkeeper.utils.affinity + + + + ${nar.cpp.optionSet} + false + false + full + + + + + + + + + Linux + + + Linux + + + + + + com.github.maven-nar + nar-maven-plugin + ${nar-maven-plugin.version} + true + + ${nar.runtime} + cpu-affinity + + + jni + org.apache.bookkeeper.utils.affinity + + + + ${nar.cpp.optionSet} + false + false + full + + + rt + + + + + + + + + diff --git a/cpu-affinity/src/main/affinity/cpp/affinity_jni.c b/cpu-affinity/src/main/affinity/cpp/affinity_jni.c new file mode 100644 index 00000000000..a0aedf6466c --- /dev/null +++ b/cpu-affinity/src/main/affinity/cpp/affinity_jni.c @@ -0,0 +1,96 @@ +/** + * + * 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. + * + */ + +// Use different error code to differentiate non-implemented error +static const int NOT_IMPLEMENTED = -2; + +#ifdef __linux__ +#define _GNU_SOURCE +#include +#include +#include + +static int set_affinity(int cpuid) { + cpu_set_t cpus; + CPU_ZERO(&cpus); + CPU_SET((size_t)cpuid, &cpus); + int threadId = (int)syscall(SYS_gettid); + return sched_setaffinity(threadId, sizeof(cpu_set_t), &cpus); +} + +static const int IS_AVAILABLE = 1; + +#else + +static int set_affinity(int cpuid) { return NOT_IMPLEMENTED; } + +static const int IS_AVAILABLE = 0; + +#endif + +#include +#include +#include +#include + +#include + +/* + * Class: org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni + * Method: isRoot + * Signature: ()Z + */ +JNIEXPORT jboolean JNICALL +Java_org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni_isRoot(JNIEnv *env, jclass cls) { + return getuid() == 0; +} + +/* + * Class: org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni + * Method: isAvailable + * Signature: ()Z + */ +JNIEXPORT jboolean JNICALL +Java_org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni_isAvailable(JNIEnv *env, jclass cls) { + return IS_AVAILABLE == 1; +} + +/* + * Class: org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni + * Method: setAffinity + * Signature: (I)V + */ +JNIEXPORT void JNICALL Java_org_apache_bookkeeper_common_util_affinity_impl_CpuAffinityJni_setAffinity( + JNIEnv *env, jclass cls, jint cpuid) { + int res = set_affinity(cpuid); + + if (res == 0) { + // Success + return; + } else if (res == NOT_IMPLEMENTED) { + (*env)->ThrowNew(env, (*env)->FindClass(env, "java/lang/Exception"), "CPU affinity not implemented"); + } else { + // Error in sched_setaffinity, get message from errno + char buffer[1024]; + strerror_r(errno, buffer, sizeof(buffer)); + (*env)->ThrowNew(env, (*env)->FindClass(env, "java/lang/Exception"), buffer); + } +} diff --git a/cpu-affinity/src/main/assembly/assembly.xml b/cpu-affinity/src/main/assembly/assembly.xml new file mode 100644 index 00000000000..377b97ff54e --- /dev/null +++ b/cpu-affinity/src/main/assembly/assembly.xml @@ -0,0 +1,62 @@ + + + + all + + jar + + + false + + + ${project.build.directory}/nar/${project.artifactId}-${project.version}-${os.arch}-MacOSX-gpp-jni/lib/${os.arch}-MacOSX-gpp/jni + + lib + + lib* + + + + ${project.build.directory}/nar/${project.artifactId}-${project.version}-${os.arch}-Linux-gpp-jni/lib/${os.arch}-Linux-gpp/jni + + lib + + lib* + + + + ${project.build.directory}/nar/${project.artifactId}-${project.version}-${os.arch}-${os.name}-gpp-jni/lib/${os.arch}-${os.name}-gpp/jni + + lib + + lib* + + + + ${project.build.directory}/classes + + + **/* + + + + diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/CpuAffinity.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/CpuAffinity.java new file mode 100644 index 00000000000..d0dab942582 --- /dev/null +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/CpuAffinity.java @@ -0,0 +1,51 @@ +/** + * + * 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.common.util.affinity; + +import lombok.experimental.UtilityClass; + +import org.apache.bookkeeper.common.util.affinity.impl.CpuAffinityImpl; + +/** + * Utilities for enabling thread to CPU affinity. + */ +@UtilityClass +public class CpuAffinity { + + /** + * Acquire ownership of one CPU core for the current thread. + * + *

    Notes: + * + *

      + *
    1. This method will only consider CPUs that are "isolated" by the OS. Eg: boot the kernel with + * isolcpus=2,3,6,7 parameter + *
    2. + *
    3. This method will disable hyper-threading on the owned core + *
    4. Once a thread successfully acquires a CPU, ownership will be retained, even if the thread exits, for as long + * as the JVM process is alive. + *
    + */ + public static void acquireCore() { + CpuAffinityImpl.acquireCore(); + } +} diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityImpl.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityImpl.java new file mode 100644 index 00000000000..f3c80a98b1c --- /dev/null +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityImpl.java @@ -0,0 +1,218 @@ +/** + * + * 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.common.util.affinity.impl; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + +import lombok.experimental.UtilityClass; +import lombok.extern.slf4j.Slf4j; + +/** + * Implementation of CPU Affinity functionality. + */ +@UtilityClass +@Slf4j +public class CpuAffinityImpl { + + private static boolean isInitialized = false; + private static boolean isSupported; + + // Id of CPU cores acquired by this process + private static final SortedSet acquiredProcessors = new TreeSet<>(); + + // Ids of processors that were isolated by Linux at boot time. This is the set + // of processors that can acquired by this or other processes. + private static SortedSet isolatedProcessors = null; + + private static ProcessorsInfo processorsInfo = null; + + public static synchronized void acquireCore() { + if (!isInitialized) { + init(); + } + + if (!isSupported) { + throw new RuntimeException("CPU Affinity not supported in current environment"); + } + + if (!CpuAffinityJni.isRoot()) { + throw new RuntimeException("CPU Affinity can only be set if the process is running as root"); + } + + try { + int cpu = pickAvailableCpu(); + CpuAffinityJni.setAffinity(cpu); + + log.info("Thread {} has successfully acquired ownership of cpu {}", Thread.currentThread().getName(), cpu); + } catch (IOException e) { + throw new RuntimeException("Failed to acquire CPU core: " + e.getMessage()); + } + } + + private static final String LOCK_FILE_PREFIX = Paths.get(System.getProperty("java.io.tmpdir"), "cpu-lock-") + .toString(); + + /** + * Other than the cores acquired by this process, there might be other processes on the same host trying to acquire + * the available cores. + * + *

    We use file-locks to ensure that other processes are aware of which CPUs are taken and that these locks are + * automatically released if the process crashes. + */ + private static synchronized int pickAvailableCpu() throws IOException { + if (isolatedProcessors == null) { + isolatedProcessors = IsolatedProcessors.get(); + } + for (int isolatedCpu : isolatedProcessors) { + if (log.isDebugEnabled()) { + log.debug("Checking CPU {}", isolatedCpu); + } + if (acquiredProcessors.contains(isolatedCpu)) { + if (log.isDebugEnabled()) { + log.debug("Ignoring CPU {} since it's already acquired", isolatedCpu); + } + continue; + } + + if (tryAcquireCpu(isolatedCpu)) { + if (log.isDebugEnabled()) { + log.debug("Using CPU {}", isolatedCpu); + } + return isolatedCpu; + } + } + + throw new RuntimeException( + "There is no available isolated CPU to acquire for thread " + Thread.currentThread().getName()); + } + + private static boolean tryAcquireCpu(int targetCpu) throws IOException { + // First, acquire lock on all the cpus that share the same core as target cpu + if (processorsInfo == null) { + processorsInfo = ProcessorsInfo.parseCpuInfo(); + } + + Set cpusToAcquire = processorsInfo.getCpusOnSameCore(targetCpu); + List acquiredCpus = new ArrayList<>(); + + for (int cpu : cpusToAcquire) { + Closeable lock = tryAcquireFileLock(cpu); + if (lock == null) { + if (log.isDebugEnabled()) { + log.debug("Failed to acquire lock on CPU {}", cpu); + } + + // Failed to acquire one cpu, release the rest that were already locked + for (Closeable l : acquiredCpus) { + l.close(); + } + + return false; + } else { + acquiredCpus.add(lock); + } + } + + // At this point, we have ownership of all required cpus + // Make sure the requested CPU is enabled and that all other CPUs on the same core are disabled, so that + // hyper-threading will not affect latency. + for (int cpu : cpusToAcquire) { + if (cpu == targetCpu) { + IsolatedProcessors.enableCpu(cpu); + } else { + IsolatedProcessors.disableCpu(cpu); + } + + acquiredProcessors.add(cpu); + } + return true; + } + + /** + * Try to acquire a lock on a particular cpu. + * + * @return null if the lock was not available + * @return a {@link Closeable} lock object if the lock was acquired + */ + private static Closeable tryAcquireFileLock(int cpu) throws IOException { + String lockPath = LOCK_FILE_PREFIX + cpu; + + RandomAccessFile file = null; + FileChannel channel = null; + FileLock lock = null; + + try { + file = new RandomAccessFile(new File(lockPath), "rw"); + channel = file.getChannel(); + lock = channel.tryLock(); + + if (lock == null) { + return null; + } else { + final FileLock finalLock = lock; + final FileChannel finalChannel = channel; + final RandomAccessFile finalFile = file; + return () -> { + // Closable object + finalLock.close(); + finalChannel.close(); + finalFile.close(); + }; + } + } finally { + if (lock == null) { + if (channel != null) { + channel.close(); + } + + if (file != null) { + file.close(); + } + } + } + } + + private static void init() { + try { + // Since this feature is only available in Linux, there's no point + // in checking for MacOS jnilib or Windows dll extensions + NativeUtils.loadLibraryFromJar("/lib/libcpu-affinity.so"); + isSupported = CpuAffinityJni.isAvailable(); + } catch (final Exception | UnsatisfiedLinkError e) { + log.warn("Unable to load CPU affinity library: {}", e.getMessage(), e); + isSupported = false; + } finally { + isInitialized = true; + } + } + +} diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityJni.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityJni.java new file mode 100644 index 00000000000..30ee325bca4 --- /dev/null +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/CpuAffinityJni.java @@ -0,0 +1,37 @@ +/** + * + * 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.common.util.affinity.impl; + +import java.io.IOException; + +import lombok.experimental.UtilityClass; + +/** + * JNI wrapper of native functions for CPU affinity. + */ +@UtilityClass +public class CpuAffinityJni { + static native boolean isRoot(); + + static native boolean isAvailable(); + + static native void setAffinity(int cpuId) throws IOException; +} diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java new file mode 100644 index 00000000000..fd9cbbca55d --- /dev/null +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java @@ -0,0 +1,102 @@ +/** + * + * 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.common.util.affinity.impl; + +import com.google.common.base.Charsets; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.SortedSet; +import java.util.TreeSet; + +import lombok.experimental.UtilityClass; +import lombok.extern.slf4j.Slf4j; + +import org.apache.commons.lang3.StringUtils; + +@UtilityClass +@Slf4j +class IsolatedProcessors { + + private static final Charset ENCODING = Charsets.US_ASCII; + + private static final String ISOLATED_CPU_PATH = "/sys/devices/system/cpu/isolated"; + + @SuppressFBWarnings("DMI_HARDCODED_ABSOLUTE_FILENAME") + static SortedSet get() throws IOException { + return parseProcessorRange(new String(Files.readAllBytes(Paths.get(ISOLATED_CPU_PATH)), ENCODING)); + } + + static SortedSet parseProcessorRange(String range) { + SortedSet processors = new TreeSet<>(); + + for (String part : StringUtils.trim(range).split(",")) { + if (part.contains("-")) { + // This is a range, eg: 1-5 with both edges included + String[] parts = part.split("-"); + int first = Integer.parseInt(parts[0]); + int last = Integer.parseInt(parts[1]); + + for (int i = first; i <= last; i++) { + processors.add(i); + } + } else if (!part.isEmpty()) { + processors.add(Integer.parseInt(part)); + } + } + + return processors; + } + + /** + * Instruct Linux to disable a particular CPU. This is used to disable hyper-threading on a particular core, by + * shutting down the cpu that shares the same core. + */ + static void disableCpu(int cpu) throws IOException { + changeCpuStatus(cpu, false); + } + + static void enableCpu(int cpu) throws IOException { + changeCpuStatus(cpu, true); + } + + /** + * Instruct Linux to disable a particular CPU. This is used to disable hyper-threading on a particular core, by + * shutting down the cpu that shares the same core. + */ + private static void changeCpuStatus(int cpu, boolean enable) throws IOException { + Path cpuPath = Paths.get(String.format("/sys/devices/system/cpu/cpu%d/online", cpu)); + + boolean currentState = Integer + .parseInt(StringUtils.trim(new String(Files.readAllBytes(cpuPath), ENCODING))) != 0; + + if (currentState != enable) { + Files.write(cpuPath, (enable ? "1\n" : "0\n").getBytes(ENCODING), StandardOpenOption.TRUNCATE_EXISTING); + log.info("{} CPU {}", enable ? "Enabled" : "Disabled", cpu); + } + } +} diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/NativeUtils.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/NativeUtils.java new file mode 100644 index 00000000000..67c8679ab12 --- /dev/null +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/NativeUtils.java @@ -0,0 +1,86 @@ +/** + * + * 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.common.util.affinity.impl; + +import static com.google.common.base.Preconditions.checkArgument; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import lombok.experimental.UtilityClass; + +/** + * Utility class to load jni library from inside a JAR. + */ +@UtilityClass +public class NativeUtils { + /** + * loads given library from the this jar. ie: this jar contains: /lib/pulsar-checksum.jnilib + * + * @param path + * : absolute path of the library in the jar
    + * if this jar contains: /lib/pulsar-checksum.jnilib then provide the same absolute path as input + * @throws Exception + */ + @SuppressFBWarnings( + value = "OBL_UNSATISFIED_OBLIGATION", + justification = "work around for java 9: https://github.com/spotbugs/spotbugs/issues/493") + public static void loadLibraryFromJar(String path) throws Exception { + checkArgument(path.startsWith("/"), "absolute path must start with /"); + + String[] parts = path.split("/"); + String filename = (parts.length > 0) ? parts[parts.length - 1] : null; + + File dir = File.createTempFile("native", ""); + if (!(dir.mkdir())) { + throw new IOException("Failed to create temp directory " + dir.getAbsolutePath()); + } + dir.deleteOnExit(); + File temp = new File(dir, filename); + temp.deleteOnExit(); + + byte[] buffer = new byte[1024]; + int read; + + try (InputStream input = NativeUtils.class.getResourceAsStream(path); + OutputStream out = new FileOutputStream(temp)) { + if (input == null) { + throw new FileNotFoundException("Couldn't find file into jar " + path); + } + + while ((read = input.read(buffer)) != -1) { + out.write(buffer, 0, read); + } + } + + if (!temp.exists()) { + throw new FileNotFoundException("Failed to copy file from jar at " + temp.getAbsolutePath()); + } + + System.load(temp.getAbsolutePath()); + } +} diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java new file mode 100644 index 00000000000..5f4449bafcd --- /dev/null +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java @@ -0,0 +1,99 @@ +/** + * + * 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.common.util.affinity.impl; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.common.base.Charsets; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; + +import org.apache.commons.lang3.StringUtils; + +/** + * Discover the list of processors from /proc/cpuinfo. + */ +class ProcessorsInfo { + + private static final Charset ENCODING = Charsets.US_ASCII; + + /** + * Given one cpu id, return all CPUs that are sharing the same core. + */ + public Set getCpusOnSameCore(int targetCpu) { + Set result = new TreeSet<>(); + int targetCore = cpus.get(targetCpu); + + cpus.forEach((cpu, core) -> { + if (core == targetCore) { + result.add(cpu); + } + }); + + return result; + } + + private final Map cpus = new TreeMap<>(); + + static ProcessorsInfo parseCpuInfo(String cpuInfoString) { + ProcessorsInfo pi = new ProcessorsInfo(); + + for (String cpu : cpuInfoString.split("\n\n")) { + int cpuId = -1; + int coreId = -1; + + for (String line : cpu.split("\n")) { + String[] parts = line.split(":", 2); + String key = StringUtils.trim(parts[0]); + String value = StringUtils.trim(parts[1]); + + if (key.equals("core id")) { + coreId = Integer.parseInt(value); + } else if (key.equals("processor")) { + cpuId = Integer.parseInt(value); + } else { + // ignore + } + } + + checkArgument(cpuId >= 0); + checkArgument(coreId >= 0); + pi.cpus.put(cpuId, coreId); + } + + return pi; + } + + static ProcessorsInfo parseCpuInfo() { + try { + return parseCpuInfo(new String(Files.readAllBytes(Paths.get("/proc/cpuinfo")), ENCODING)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/package-info.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/package-info.java new file mode 100644 index 00000000000..35da8164662 --- /dev/null +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/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. + */ + +/** + * Implementation of util functions for CPU affinity. + */ +package org.apache.bookkeeper.common.util.affinity.impl; \ No newline at end of file diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/package-info.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/package-info.java new file mode 100644 index 00000000000..af4f4b4add1 --- /dev/null +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/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. + */ + +/** + * Util functions for CPU affinity. + */ +package org.apache.bookkeeper.common.util.affinity; \ No newline at end of file diff --git a/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/CpuInfoTest.java b/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/CpuInfoTest.java new file mode 100644 index 00000000000..6f8db0f1733 --- /dev/null +++ b/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/CpuInfoTest.java @@ -0,0 +1,53 @@ +/** + * + * 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.common.util.affinity.impl; + +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Sets; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.util.stream.Collectors; + +import org.junit.Test; + +/** + * Tests for CpuInfo class. + */ +public class CpuInfoTest { + + @Test + public void testParseCpuInfo() throws Exception { + try (BufferedReader r = new BufferedReader( + new InputStreamReader(CpuInfoTest.class.getResourceAsStream("/proc_cpuinfo.txt")))) { + String text = r.lines().collect(Collectors.joining("\n")); + + ProcessorsInfo pi = ProcessorsInfo.parseCpuInfo(text); + + assertEquals(Sets.newHashSet(0, 12), pi.getCpusOnSameCore(0)); + assertEquals(Sets.newHashSet(0, 12), pi.getCpusOnSameCore(12)); + + assertEquals(Sets.newHashSet(8, 20), pi.getCpusOnSameCore(8)); + assertEquals(Sets.newHashSet(8, 20), pi.getCpusOnSameCore(20)); + } + } +} diff --git a/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessorsTest.java b/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessorsTest.java new file mode 100644 index 00000000000..5bb68e48fee --- /dev/null +++ b/cpu-affinity/src/test/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessorsTest.java @@ -0,0 +1,47 @@ +/** + * + * 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.common.util.affinity.impl; + +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Sets; + +import org.junit.Test; + +/** + * Tests for {@link IsolatedProcessors}. + */ +public class IsolatedProcessorsTest { + + @Test + public void testParseProcessors() throws Exception { + assertEquals(Sets.newHashSet(), IsolatedProcessors.parseProcessorRange("")); + + assertEquals(Sets.newHashSet(1, 3, 4, 5, 6, 7), IsolatedProcessors.parseProcessorRange("1,3-7")); + + assertEquals(Sets.newHashSet(1), IsolatedProcessors.parseProcessorRange("1")); + assertEquals(Sets.newHashSet(1, 3), IsolatedProcessors.parseProcessorRange("1,3")); + assertEquals(Sets.newHashSet(1, 3, 4, 5, 6, 7, 10, 11, 12, 13), + IsolatedProcessors.parseProcessorRange("1,3-7,10-13")); + + assertEquals(Sets.newHashSet(1, 3, 4, 5, 6, 7), IsolatedProcessors.parseProcessorRange("1,3-7\n")); + } +} diff --git a/cpu-affinity/src/test/resources/proc_cpuinfo.txt b/cpu-affinity/src/test/resources/proc_cpuinfo.txt new file mode 100644 index 00000000000..9ffe3e3a00d --- /dev/null +++ b/cpu-affinity/src/test/resources/proc_cpuinfo.txt @@ -0,0 +1,648 @@ +processor : 0 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 0 +cpu cores : 12 +apicid : 0 +initial apicid : 0 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 1 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 1 +cpu cores : 12 +apicid : 2 +initial apicid : 2 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 2 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 2 +cpu cores : 12 +apicid : 4 +initial apicid : 4 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 3 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 3 +cpu cores : 12 +apicid : 6 +initial apicid : 6 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 4 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 4 +cpu cores : 12 +apicid : 8 +initial apicid : 8 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 5 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 5 +cpu cores : 12 +apicid : 10 +initial apicid : 10 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 6 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 8 +cpu cores : 12 +apicid : 16 +initial apicid : 16 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 7 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 9 +cpu cores : 12 +apicid : 18 +initial apicid : 18 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 8 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 10 +cpu cores : 12 +apicid : 20 +initial apicid : 20 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 9 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 11 +cpu cores : 12 +apicid : 22 +initial apicid : 22 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 10 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 12 +cpu cores : 12 +apicid : 24 +initial apicid : 24 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 11 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 13 +cpu cores : 12 +apicid : 26 +initial apicid : 26 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 12 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 0 +cpu cores : 12 +apicid : 1 +initial apicid : 1 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 13 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 1 +cpu cores : 12 +apicid : 3 +initial apicid : 3 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 14 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 2 +cpu cores : 12 +apicid : 5 +initial apicid : 5 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 15 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 3 +cpu cores : 12 +apicid : 7 +initial apicid : 7 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 16 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 4 +cpu cores : 12 +apicid : 9 +initial apicid : 9 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 17 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 5 +cpu cores : 12 +apicid : 11 +initial apicid : 11 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 18 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 8 +cpu cores : 12 +apicid : 17 +initial apicid : 17 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 19 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 9 +cpu cores : 12 +apicid : 19 +initial apicid : 19 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 20 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 10 +cpu cores : 12 +apicid : 21 +initial apicid : 21 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 21 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 11 +cpu cores : 12 +apicid : 23 +initial apicid : 23 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 22 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 12 +cpu cores : 12 +apicid : 25 +initial apicid : 25 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + +processor : 23 +vendor_id : GenuineIntel +cpu family : 6 +model : 79 +model name : Intel(R) Xeon(R) CPU E5-2687W v4 @ 3.00GHz +stepping : 1 +microcode : 0xb00002a +cpu MHz : 3199.951 +cache size : 30720 KB +physical id : 0 +siblings : 24 +core id : 13 +cpu cores : 12 +apicid : 27 +initial apicid : 27 +fpu : yes +fpu_exception : yes +cpuid level : 20 +wp : yes +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx smx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid dca sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch epb invpcid_single ibrs ibpb stibp kaiser tpr_shadow vnmi flexpriority ept vpid fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm cqm rdseed adx smap intel_pt xsaveopt cqm_llc cqm_occup_llc cqm_mbm_total cqm_mbm_local dtherm ida arat pln pts +bugs : cpu_meltdown spectre_v1 spectre_v2 spec_store_bypass +bogomips : 5986.05 +clflush size : 64 +cache_alignment : 64 +address sizes : 46 bits physical, 48 bits virtual +power management: + diff --git a/pom.xml b/pom.xml index 5c6d9eb6100..9903f5a8afe 100644 --- a/pom.xml +++ b/pom.xml @@ -65,6 +65,7 @@ bookkeeper-http stream/distributedlog tools + cpu-affinity metadata-drivers bookkeeper-dist shaded @@ -190,7 +191,7 @@ 3.1.0 2.2.1 2.21.0 - 3.1.0 + 3.5.2 1.4.1.Final 0.5.0 6.19 From fb5491a6ec644f6ca00bcba41a0601359d604c40 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 14 Nov 2018 15:12:57 -0800 Subject: [PATCH 0120/1642] Added BlockingQueue implementation based on JCtools ### Motivation Add a `BlockingQueue` implementation that is suitable for low latency and low contention. Key points: * Optimized for multiple producers and single consumer * When waiting for entries, the thread is blocked with busy wait to avoid context switch. (This will be used in subsequent PRs to optionally enable it) Reviewers: Enrico Olivelli , Sijie Guo This closes #1682 from merlimat/blocking-queue --- bookkeeper-common/pom.xml | 4 + .../common/collections/BlockingMpscQueue.java | 160 ++++++++++++++++++ .../common/collections/BusyWait.java | 65 +++++++ .../collections/BlockingMpscQueueTest.java | 112 ++++++++++++ .../src/main/resources/LICENSE-all.bin.txt | 2 + .../src/main/resources/LICENSE-bkctl.bin.txt | 2 + .../src/main/resources/LICENSE-server.bin.txt | 2 + .../org/apache/bookkeeper/bookie/Bookie.java | 11 +- .../org/apache/bookkeeper/bookie/Journal.java | 12 +- .../bookie/LedgerDescriptorImpl.java | 29 ++-- .../bookkeeper/proto/WriteLacProcessorV3.java | 5 + .../client/BookKeeperCloseTest.java | 4 +- .../client/ParallelLedgerRecoveryTest.java | 4 +- pom.xml | 8 + 14 files changed, 394 insertions(+), 26 deletions(-) create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/BlockingMpscQueue.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/BusyWait.java create mode 100644 bookkeeper-common/src/test/java/org/apache/bookkeeper/common/collections/BlockingMpscQueueTest.java diff --git a/bookkeeper-common/pom.xml b/bookkeeper-common/pom.xml index 4b8b7e1fa67..89463353310 100644 --- a/bookkeeper-common/pom.xml +++ b/bookkeeper-common/pom.xml @@ -50,6 +50,10 @@ com.fasterxml.jackson.core jackson-annotations + + org.jctools + jctools-core + com.google.code.findbugs jsr305 diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/BlockingMpscQueue.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/BlockingMpscQueue.java new file mode 100644 index 00000000000..619c0edd607 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/BlockingMpscQueue.java @@ -0,0 +1,160 @@ +/* + * 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.common.collections; + +import java.util.Collection; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.jctools.queues.MpscArrayQueue; + +/** + * Blocking queue optimized for multiple producers and single consumer. + */ +public class BlockingMpscQueue extends MpscArrayQueue implements BlockingQueue { + + public BlockingMpscQueue(int size) { + super(size); + } + + @Override + public void put(T e) throws InterruptedException { + while (!this.relaxedOffer(e)) { + // Do busy-spin loop + if (Thread.interrupted()) { + throw new InterruptedException(); + } + } + } + + @Override + public boolean offer(T e, long timeout, TimeUnit unit) throws InterruptedException { + long absoluteEndTime = System.nanoTime() + unit.toNanos(timeout); + + while (!this.relaxedOffer(e)) { + // Do busy-spin loop + + if (System.nanoTime() > absoluteEndTime) { + return false; + } + + if (Thread.interrupted()) { + throw new InterruptedException(); + } + } + + return true; + } + + @Override + public T take() throws InterruptedException { + int idleCounter = 0; + while (true) { + T item = relaxedPoll(); + if (item == null) { + if (Thread.interrupted()) { + throw new InterruptedException(); + } + + idleCounter = WAIT_STRATEGY.idle(idleCounter); + continue; + } + + + return item; + } + } + + @Override + public T poll(long timeout, TimeUnit unit) throws InterruptedException { + long absoluteEndTime = System.nanoTime() + unit.toNanos(timeout); + + int idleCounter = 0; + while (true) { + T item = relaxedPoll(); + if (item == null) { + if (Thread.interrupted()) { + throw new InterruptedException(); + } + + if (System.nanoTime() > absoluteEndTime) { + return null; + } else { + idleCounter = WAIT_STRATEGY.idle(idleCounter); + continue; + } + } + + return item; + } + } + + @Override + public int remainingCapacity() { + return capacity() - size(); + } + + @Override + public int drainTo(Collection c) { + int initialSize = c.size(); + + final DrainStrategy ds = new DrainStrategy(); + drain(c::add, ds, ds); + return c.size() - initialSize; + } + + @Override + public int drainTo(Collection c, int maxElements) { + return drain(c::add, maxElements); + } + + /** + * Wait strategy combined with exit condition, for draining the queue. + */ + private static final class DrainStrategy implements WaitStrategy, ExitCondition { + + boolean reachedEnd = false; + + @Override + public boolean keepRunning() { + return !reachedEnd; + } + + @Override + public int idle(int idleCounter) { + reachedEnd = true; + return idleCounter; + } + + } + + /** + * Waiting strategy that starts with busy loop and gradually falls back to sleeping if no items are available. + */ + private static final WaitStrategy SPIN_STRATEGY = new WaitStrategy() { + + @Override + public int idle(int idleCounter) { + BusyWait.onSpinWait(); + return idleCounter + 1; + } + }; + + private static final WaitStrategy WAIT_STRATEGY = SPIN_STRATEGY; +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/BusyWait.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/BusyWait.java new file mode 100644 index 00000000000..b44a9f68044 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/BusyWait.java @@ -0,0 +1,65 @@ +/* + * 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.common.collections; + +import java.lang.invoke.MethodHandle; +import java.lang.invoke.MethodHandles; +import java.lang.invoke.MethodType; + +import lombok.experimental.UtilityClass; +import lombok.extern.slf4j.Slf4j; + +/** + * Utility class to use "Thread.onSpinWait()" when available. + */ +@UtilityClass +@Slf4j +public class BusyWait { + + /** + * If available (Java 9+), use intrinsic {@link Thread#onSpinWait} which will + * reduce CPU consumption during the wait, otherwise fallback to regular + * spinning. + */ + public static void onSpinWait() { + if (ON_SPIN_WAIT != null) { + try { + ON_SPIN_WAIT.invokeExact(); + } catch (Throwable t) { + // Ignore + } + } + } + + private static final MethodHandle ON_SPIN_WAIT; + + static { + MethodHandle handle = null; + try { + handle = MethodHandles.lookup().findStatic(Thread.class, "onSpinWait", MethodType.methodType(void.class)); + } catch (Throwable t) { + // Ignore + if (log.isDebugEnabled()) { + log.debug("Unable to use 'onSpinWait' from JVM", t); + } + } + + ON_SPIN_WAIT = handle; + } +} diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/collections/BlockingMpscQueueTest.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/collections/BlockingMpscQueueTest.java new file mode 100644 index 00000000000..b778001757a --- /dev/null +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/collections/BlockingMpscQueueTest.java @@ -0,0 +1,112 @@ +/* + * 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.common.collections; + +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 java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.junit.Test; + +/** + * Unit tests for {@link BlockingMpscQueue}. + */ +public class BlockingMpscQueueTest { + + @Test + public void basicTest() throws Exception { + final int size = 15; + BlockingQueue queue = new BlockingMpscQueue<>(size); + + for (int i = 0; i < size; i++) { + queue.put(i); + + assertEquals(size - i, queue.remainingCapacity()); + } + + assertEquals(size, queue.size()); + + for (int i = 0; i < size; i++) { + Integer n = queue.take(); + assertTrue(n != null); + } + + assertEquals(0, queue.size()); + + Integer res = queue.poll(100, TimeUnit.MILLISECONDS); + assertNull(res); + } + + @Test + public void testOffer() throws Exception { + final int size = 16; + BlockingQueue queue = new BlockingMpscQueue<>(size); + + for (int i = 0; i < size; i++) { + assertTrue(queue.offer(1, 100, TimeUnit.MILLISECONDS)); + } + + assertEquals(size, queue.size()); + + assertFalse(queue.offer(1, 100, TimeUnit.MILLISECONDS)); + assertEquals(size, queue.size()); + } + + @Test + public void testDrain() throws Exception { + final int size = 10; + BlockingQueue queue = new BlockingMpscQueue<>(size); + + for (int i = 0; i < size; i++) { + queue.put(i); + } + + List list = new ArrayList<>(size); + queue.drainTo(list); + + assertEquals(size, list.size()); + + assertEquals(0, queue.size()); + + Integer res = queue.poll(100, TimeUnit.MILLISECONDS); + assertNull(res); + } + + @Test + public void testDrainWithLimit() throws Exception { + final int size = 10; + BlockingQueue queue = new BlockingMpscQueue<>(size); + + for (int i = 0; i < size; i++) { + queue.put(i); + } + + List list = new ArrayList<>(); + queue.drainTo(list, 5); + assertEquals(5, list.size()); + + assertEquals(5, queue.size()); + } +} diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index 175050d9ad4..a1863d11912 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -314,6 +314,7 @@ Apache Software License, Version 2. - lib/org.inferred-freebuilder-1.14.9.jar [47] - lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [48] - lib/org.apache.yetus-audience-annotations-0.5.0.jar [49] +- lib/org.jctools-jctools-core-2.1.2.jar [50] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.9.7 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.9.7 @@ -363,6 +364,7 @@ Apache Software License, Version 2. [47] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 [48] Source available at https://github.com/google/error-prone/tree/v2.1.2 [49] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 +[50] Source available at https://github.com/JCTools/JCTools/tree/v2.1.2 ------------------------------------------------------------------------------------ diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt index 486a35ab1db..692e9a0e5cd 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -258,6 +258,7 @@ Apache Software License, Version 2. - lib/org.inferred-freebuilder-1.14.9.jar [34] - lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [35] - lib/org.apache.yetus-audience-annotations-0.5.0.jar [36] +- lib/org.jctools-jctools-core-2.1.2.jar [37] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9 @@ -287,6 +288,7 @@ Apache Software License, Version 2. [34] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 [35] Source available at https://github.com/google/error-prone/tree/v2.1.2 [36] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 +[37] Source available at https://github.com/JCTools/JCTools/tree/v2.1.2 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-4.1.31.Final.jar bundles some 3rd party dependencies diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index 62b4f19d4d1..a8b5b884752 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -278,6 +278,7 @@ Apache Software License, Version 2. - lib/org.inferred-freebuilder-1.14.9.jar [34] - lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [35] - lib/org.apache.yetus-audience-annotations-0.5.0.jar [36] +- lib/org.jctools-jctools-core-2.1.2.jar [37] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.8.9 @@ -315,6 +316,7 @@ Apache Software License, Version 2. [34] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 [35] Source available at https://github.com/google/error-prone/tree/v2.1.2 [36] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 +[37] Source available at https://github.com/JCTools/JCTools/tree/v2.1.2 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-4.1.31.Final.jar bundles some 3rd party dependencies 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 d0db80e26ff..cd0b2f745c2 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 @@ -1133,7 +1133,7 @@ private Journal getJournal(long ledgerId) { */ private void addEntryInternal(LedgerDescriptor handle, ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey) - throws IOException, BookieException { + throws IOException, BookieException, InterruptedException { long ledgerId = handle.getLedgerId(); long entryId = handle.addEntry(entry); @@ -1170,7 +1170,7 @@ private void addEntryInternal(LedgerDescriptor handle, ByteBuf entry, * is not exposed to users. */ public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] masterKey) - throws IOException, BookieException { + throws IOException, BookieException, InterruptedException { long requestNanos = MathUtils.nowInNano(); boolean success = false; int entrySize = 0; @@ -1208,7 +1208,7 @@ static ByteBuf createExplicitLACEntry(long ledgerId, ByteBuf explicitLac) { } public void setExplicitLac(ByteBuf entry, WriteCallback writeCallback, Object ctx, byte[] masterKey) - throws IOException, BookieException { + throws IOException, InterruptedException, BookieException { try { long ledgerId = entry.getLong(entry.readerIndex()); LedgerDescriptor handle = handles.getHandle(ledgerId, masterKey); @@ -1254,7 +1254,7 @@ public void forceLedger(long ledgerId, WriteCallback cb, * @throws BookieException.LedgerFencedException if the ledger is fenced */ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey) - throws IOException, BookieException.LedgerFencedException, BookieException { + throws IOException, BookieException.LedgerFencedException, BookieException, InterruptedException { long requestNanos = MathUtils.nowInNano(); boolean success = false; int entrySize = 0; @@ -1313,7 +1313,8 @@ public SettableFuture getResult() { * This method is idempotent. Once a ledger is fenced, it can * never be unfenced. Fencing a fenced ledger has no effect. */ - public SettableFuture fenceLedger(long ledgerId, byte[] masterKey) throws IOException, BookieException { + public SettableFuture fenceLedger(long ledgerId, byte[] masterKey) + throws IOException, BookieException { LedgerDescriptor handle = handles.getHandle(ledgerId, masterKey); return handle.fenceAndLogInJournal(getJournal(ledgerId)); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 25680925f9a..57f05c274f6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -870,27 +870,29 @@ public boolean accept(long journalId) { } } - public void logAddEntry(ByteBuffer entry, boolean ackBeforeSync, WriteCallback cb, Object ctx) { + public void logAddEntry(ByteBuffer entry, boolean ackBeforeSync, WriteCallback cb, Object ctx) + throws InterruptedException { logAddEntry(Unpooled.wrappedBuffer(entry), ackBeforeSync, cb, ctx); } /** * record an add entry operation in journal. */ - public void logAddEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx) { + public void logAddEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx) + throws InterruptedException { long ledgerId = entry.getLong(entry.readerIndex() + 0); long entryId = entry.getLong(entry.readerIndex() + 8); logAddEntry(ledgerId, entryId, entry, ackBeforeSync, cb, ctx); } @VisibleForTesting - void logAddEntry(long ledgerId, long entryId, ByteBuf entry, - boolean ackBeforeSync, WriteCallback cb, Object ctx) { + void logAddEntry(long ledgerId, long entryId, ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx) + throws InterruptedException { //Retain entry until it gets written to journal entry.retain(); journalQueueSize.inc(); - queue.add(QueueEntry.create( + queue.put(QueueEntry.create( entry, ackBeforeSync, ledgerId, entryId, cb, ctx, MathUtils.nowInNano(), journalAddEntryStats, journalQueueSize)); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java index a0f34eab65d..ee91ed0f0dd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java @@ -122,18 +122,23 @@ private SettableFuture logFenceEntryInJournal(Journal journal) { result = logFenceResult = SettableFuture.create(); } ByteBuf entry = createLedgerFenceEntry(ledgerId); - journal.logAddEntry(entry, false /* ackBeforeSync */, (rc, ledgerId, entryId, addr, ctx) -> { - if (LOG.isDebugEnabled()) { - LOG.debug("Record fenced state for ledger {} in journal with rc {}", - ledgerId, BKException.codeLogger(rc)); - } - if (rc == 0) { - fenceEntryPersisted.compareAndSet(false, true); - result.set(true); - } else { - result.set(false); - } - }, null); + try { + journal.logAddEntry(entry, false /* ackBeforeSync */, (rc, ledgerId, entryId, addr, ctx) -> { + if (LOG.isDebugEnabled()) { + LOG.debug("Record fenced state for ledger {} in journal with rc {}", + ledgerId, BKException.codeLogger(rc)); + } + if (rc == 0) { + fenceEntryPersisted.compareAndSet(false, true); + result.set(true); + } else { + result.set(false); + } + }, null); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + result.setException(e); + } return result; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java index 2f018ff9610..7e42a7320af 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java @@ -111,6 +111,11 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre logger.error("Error saving lac {} for ledger:{}", lac, ledgerId, e); status = StatusCode.EIO; + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("Interrupted while saving lac {} for ledger:{}", + lac, ledgerId, e); + status = StatusCode.EIO; } catch (BookieException e) { logger.error("Unauthorized access to ledger:{} while adding lac:{}", ledgerId, lac, e); 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 850fe5d7ef2..eda68c866b1 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 @@ -77,7 +77,7 @@ private void restartBookieSlow() throws Exception{ @Override public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] masterKey) - throws IOException, BookieException { + throws IOException, BookieException, InterruptedException { try { Thread.sleep(5000); } catch (InterruptedException ie) { @@ -91,7 +91,7 @@ public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, @Override public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey) - throws IOException, BookieException { + throws IOException, BookieException, InterruptedException { try { Thread.sleep(5000); } catch (InterruptedException ie) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index fab13edbac6..5ba73c6733e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -515,8 +515,8 @@ public DelayResponseBookie(ServerConfiguration conf) } @Override - public void addEntry(ByteBuf entry, boolean ackBeforeSync, final WriteCallback cb, - Object ctx, byte[] masterKey) throws IOException, BookieException { + public void addEntry(ByteBuf entry, boolean ackBeforeSync, final WriteCallback cb, Object ctx, byte[] masterKey) + throws IOException, BookieException, InterruptedException { super.addEntry(entry, ackBeforeSync, new WriteCallback() { @Override public void writeComplete(int rc, long ledgerId, long entryId, diff --git a/pom.xml b/pom.xml index 9903f5a8afe..8f151b85042 100644 --- a/pom.xml +++ b/pom.xml @@ -170,6 +170,7 @@ 1.29.0 3.4.1 3.4.13 + 2.1.2 0.12 2.7 @@ -584,6 +585,13 @@ ${jetty.version} + + + org.jctools + jctools-core + ${jctools.version} + + From 72bed29b356c60422e5e9fceee0a1c4d286d6983 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 16 Nov 2018 23:09:31 +0100 Subject: [PATCH 0121/1642] Fix cpu-affinity module build on jdk10+ Port tricks from circe-checksum to cpu-affinity module in order to make the build pass on JDK11 Reviewers: Sijie Guo , Matteo Merli This closes #1816 from eolivelli/fix/jdk11-cpu-affinity-build --- cpu-affinity/pom.xml | 48 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/cpu-affinity/pom.xml b/cpu-affinity/pom.xml index f58a07991ef..1439259f66c 100644 --- a/cpu-affinity/pom.xml +++ b/cpu-affinity/pom.xml @@ -73,6 +73,7 @@ org.apache.maven.plugins maven-assembly-plugin + ${maven-assembly-plugin.version} src/main/assembly/assembly.xml @@ -104,6 +105,53 @@ + + + jdk-without-javah + + [10,) + + + + + com.github.maven-nar + nar-maven-plugin + ${nar-maven-plugin.version} + true + + + + default-nar-javah + none + + + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven-compiler-plugin.version} + + 1.8 + 1.8 + + + + -Xlint:deprecation + -Xlint:unchecked + + -Xpkginfo:always + + -h + ${project.build.directory}/nar/javah-include + + + + + + mac From 3699ff5ee3f061edfc5bb197654c2887df7cce79 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 16 Nov 2018 14:10:26 -0800 Subject: [PATCH 0122/1642] [BOOKIE] Fix sorted ledger storage rotating entry log files too frequent Descriptions of the changes in this PR: *Motivation* A strong behavior was observed when using sorted ledger storage with single entry log manager on production: "the entry log files are rotated very frequently and small entry log files are produced". The problem was introduced due to #1410. At current entry logger, when a new entry log file is created, EntryLogger will notify its listeners that a new entry log file is rotated via [`onRotateEntryLog`](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerBase.java#L152). Before the change in #1410, `SortedLedgerStorage` inherits from `InterleavedLedgerStorage`. So when a new entry log file is rotated, `SortedLedgerStorage` is notified. However after the change in #1410, `SortedLedgerStorage` doesn't inherits `InterleavedLedgerStorage` anymore. Instead, the relationship is changed to composition. `SortedLedgerStorage` is composed using an interleaved ledger storage. So the entrylog listener contract was broken. `SortedLedgerStorage` will not receive any `onRotateEntryLog` notification any more. *Changes* When `SortedLedgerStorage` initializes, it passes its own entry log listener down to the interleaved ledger storage. So entry logger can notify the right person for entry log rotations. *Tests* Existing tests should cover most of the case. Looking for how to add new test cases. Reviewers: Enrico Olivelli , Charan Reddy Guttapalem , Andrey Yegorov This closes #1807 from sijie/fix_rotation_behavior --- .../bookie/EntryLogManagerBase.java | 14 +++++++++++ .../EntryLogManagerForEntryLogPerLedger.java | 6 +++-- .../EntryLogManagerForSingleEntryLog.java | 12 ++++++---- .../bookie/InterleavedLedgerStorage.java | 23 ++++++++++++++++++- .../bookie/SortedLedgerStorage.java | 5 +++- 5 files changed, 52 insertions(+), 8 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerBase.java index 701fb7b2e8e..f9c6d97cfd8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerBase.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerBase.java @@ -21,6 +21,9 @@ package org.apache.bookkeeper.bookie; +import static org.apache.bookkeeper.bookie.EntryLogger.UNASSIGNED_LEDGERID; + +import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.util.concurrent.FastThreadLocal; @@ -131,7 +134,18 @@ void flushLogChannel(BufferedLogChannel logChannel, boolean forceMetadata) throw * Creates a new log file. This method should be guarded by a lock, * so callers of this method should be in right scope of the lock. */ + @VisibleForTesting void createNewLog(long ledgerId) throws IOException { + createNewLog(ledgerId, ""); + } + + void createNewLog(long ledgerId, String reason) throws IOException { + if (ledgerId != UNASSIGNED_LEDGERID) { + log.info("Creating a new entry log file for ledger '{}' {}", ledgerId, reason); + } else { + log.info("Creating a new entry log file {}", reason); + } + BufferedLogChannel logChannel = getCurrentLogForLedger(ledgerId); // first tried to create a new log channel. add current log channel to ToFlush list only when // there is a new log channel. it would prevent that a log channel is referenced by both diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java index 39ed60cea57..8093b53b157 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java @@ -484,7 +484,7 @@ public boolean commitEntryMemTableFlush() throws IOException { try { if (reachEntryLogLimit(currentLog, 0L)) { log.info("Rolling entry logger since it reached size limitation for ledger: {}", ledgerId); - createNewLog(ledgerId); + createNewLog(ledgerId, "after entry log file is rotated"); } } finally { lock.unlock(); @@ -640,7 +640,9 @@ BufferedLogChannel getCurrentLogForLedgerForAddEntry(long ledgerId, int entrySiz if (logChannel != null) { logChannel.flushAndForceWriteIfRegularFlush(false); } - createNewLog(ledgerId); + createNewLog(ledgerId, + ": diskFull = " + diskFull + ", allDisksFull = " + allDisksFull + + ", reachEntryLogLimit = " + reachEntryLogLimit + ", logChannel = " + logChannel); } return getCurrentLogForLedger(ledgerId); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForSingleEntryLog.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForSingleEntryLog.java index 3e552d0fca9..72c818a30c7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForSingleEntryLog.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForSingleEntryLog.java @@ -92,7 +92,7 @@ synchronized BufferedLogChannel getCurrentLogForLedgerForAddEntry(long ledgerId, boolean rollLog) throws IOException { if (null == activeLogChannel) { // log channel can be null because the file is deferred to be created - createNewLog(UNASSIGNED_LEDGERID); + createNewLog(UNASSIGNED_LEDGERID, "because current active log channel has not initialized yet"); } boolean reachEntryLogLimit = rollLog ? reachEntryLogLimit(activeLogChannel, entrySize) @@ -103,7 +103,8 @@ synchronized BufferedLogChannel getCurrentLogForLedgerForAddEntry(long ledgerId, if (activeLogChannel != null) { activeLogChannel.flushAndForceWriteIfRegularFlush(false); } - createNewLog(UNASSIGNED_LEDGERID); + createNewLog(UNASSIGNED_LEDGERID, + ": createNewLog = " + createNewLog + ", reachEntryLogLimit = " + reachEntryLogLimit); // Reset the flag if (createNewLog) { shouldCreateNewEntryLog.set(false); @@ -238,7 +239,9 @@ public boolean commitEntryMemTableFlush() throws IOException { */ if (reachEntryLogLimit(activeLogChannel, 0L) || logIdAfterFlush != logIdBeforeFlush) { log.info("Rolling entry logger since it reached size limitation"); - createNewLog(UNASSIGNED_LEDGERID); + createNewLog(UNASSIGNED_LEDGERID, + "due to reaching log limit after flushing memtable : logIdBeforeFlush = " + + logIdBeforeFlush + ", logIdAfterFlush = " + logIdAfterFlush); return true; } return false; @@ -251,7 +254,8 @@ public void prepareSortedLedgerStorageCheckpoint(long numBytesFlushed) throws IO // it means bytes might live at current active entry log, we need // roll current entry log and then issue checkpoint to underlying // interleaved ledger storage. - createNewLog(UNASSIGNED_LEDGERID); + createNewLog(UNASSIGNED_LEDGERID, + "due to preparing checkpoint : numBytesFlushed = " + numBytesFlushed); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java index 08e7f4ef914..8ab6517b489 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java @@ -98,11 +98,32 @@ public void initialize(ServerConfiguration conf, Checkpointer checkpointer, StatsLogger statsLogger) throws IOException { + initializeWithEntryLogListener( + conf, + ledgerManager, + ledgerDirsManager, + indexDirsManager, + stateManager, + checkpointSource, + checkpointer, + this, + statsLogger); + } + + void initializeWithEntryLogListener(ServerConfiguration conf, + LedgerManager ledgerManager, + LedgerDirsManager ledgerDirsManager, + LedgerDirsManager indexDirsManager, + StateManager stateManager, + CheckpointSource checkpointSource, + Checkpointer checkpointer, + EntryLogListener entryLogListener, + StatsLogger statsLogger) throws IOException { checkNotNull(checkpointSource, "invalid null checkpoint source"); checkNotNull(checkpointer, "invalid null checkpointer"); this.checkpointSource = checkpointSource; this.checkpointer = checkpointer; - entryLogger = new EntryLogger(conf, ledgerDirsManager, this, statsLogger.scope(ENTRYLOGGER_SCOPE)); + entryLogger = new EntryLogger(conf, ledgerDirsManager, entryLogListener, statsLogger.scope(ENTRYLOGGER_SCOPE)); ledgerCache = new LedgerCacheImpl(conf, activeLedgers, null == indexDirsManager ? ledgerDirsManager : indexDirsManager, statsLogger); gcThread = new GarbageCollectorThread(conf, ledgerManager, this, statsLogger.scope("gc")); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java index 5c4f75a22e2..e4b137ffbb5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java @@ -72,7 +72,7 @@ public void initialize(ServerConfiguration conf, StatsLogger statsLogger) throws IOException { - interleavedLedgerStorage.initialize( + interleavedLedgerStorage.initializeWithEntryLogListener( conf, ledgerManager, ledgerDirsManager, @@ -80,6 +80,9 @@ public void initialize(ServerConfiguration conf, stateManager, checkpointSource, checkpointer, + // uses sorted ledger storage's own entry log listener + // since it manages entry log rotations and checkpoints. + this, statsLogger); if (conf.isEntryLogPerLedgerEnabled()) { From 7ecf93f8c4f75974d4f3f85acf4826d97d22ec09 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Sat, 17 Nov 2018 09:11:31 -0800 Subject: [PATCH 0123/1642] Introduce condensed encoding format for availabilityOfEntriesOfLedger Descriptions of the changes in this PR: - this condensed encoding format will take fewer bytes than initially proposed bit vector. Reviewers: Sijie Guo , Enrico Olivelli This closes #1814 from reddycharan/updatebpdoc --- site/bps/BP-34-cluster-metadata-checker.md | 61 ++++++++++++++++++++-- 1 file changed, 58 insertions(+), 3 deletions(-) diff --git a/site/bps/BP-34-cluster-metadata-checker.md b/site/bps/BP-34-cluster-metadata-checker.md index 1992d4cab70..d022fb08b91 100644 --- a/site/bps/BP-34-cluster-metadata-checker.md +++ b/site/bps/BP-34-cluster-metadata-checker.md @@ -51,13 +51,68 @@ message GetListOfEntriesOfALedgerRequest { message GetListOfEntriesOfALedgerResponse { required StatusCode status = 1; required int64 ledgerId = 2; - optional bytes availabilityOfEntriesOfLedger = 3; // treated as array of bits indicating availability of entry at that particular index location + optional bytes availabilityOfEntriesOfLedger = 3; // explained below } ``` -For the sake of future extensibility it would be helpful to add version info (and possibly some metadata in the future) at the beginning of the 'availabilityOfEntriesOfLedger'. So the first 64 bytes will be considered reserved space, with the first byte specifying the version for now and the rest of the bytes in the reserved space will be 0's. +For ‘availabilityOfEntriesOfLedger’ we can use following condensed encoding format, which helps in reducing the number of bytes needed to represent the list of entries. -Here Bookie is expected to attain this information just from just LedgerCache (Index Files) - IndexPersistenceMgr and IndexInMemPageMgr but it doesn’t actually check the availability of this entry in Entrylogger. Since the intention of this checker is limited to do just metadata validation at cluster level. +(note: following representation is used just for understanding purpose, but this is not protobuf (or any other) representation) + +``` +AvailabilityOfEntriesOfLedger { + OrderedCollection sequenceGroup; +} + +SequenceGroup { + long firstSequenceStart; + long lastSequenceStart; + int sequenceSize; + int sequencePeriod; +} +``` + +Nomenclature: + + - Continuous entries are grouped as a ’Sequence’. + - Number of continuous entries in a ‘Sequence’ is called ‘sequenceSize’. + - Gap between Consecutive sequences is called ‘sequencePeriod’. + - Consecutive sequences with same sequenceSize and same sequencePeriod in between consecutive sequences are grouped as a SequenceGroup. + - ‘firstSequenceStart’ is the first entry in the first sequence of the SequenceGroup. + - ‘lastSequenceStart’ is the first entry in the last sequence of the SequenceGroup. + - Ordered collection of such SequenceGroups will represent entries of a ledger residing in a bookie. + +(in the best case scenario there will be only one SequenceGroup, ‘sequencePeriod’ will be ensembleSize and ‘sequenceSize’ will be writeQuorumSize of the ledger). + +for example, + +example 1 (best case scenario): + +1, 2, 4, 5, 7, 8, 10, 11 + +in this case (1, 2), (4, 5), (7, 8), (10, 11) are sequences and in this scenario there happens to be just one SequenceGroup, which can be represented like + +{ firstSequenceStart - 1, lastSequenceStart - 10, sequenceSize - 2, sequencePeriod - 3 } + +example 2 (an entry is missing): + +1, 2, 3, 6, 7, 8, 11, 13, 16, 17, 18, 21, 22 + +(entry 12 is missing and in the last sequence there are only 2 entries 21, 22) +in this case (1, 2, 3), (6, 7, 8), (11), (13), (16, 17, 18), (21, 22) are the sequences +so the sequence groups are + +{ firstSequenceStart - 1, lastSequenceStart - 6, sequenceSize - 3, sequencePeriod - 5 }, { firstSequenceStart - 11, lastSequenceStart - 13, sequenceSize - 1, sequencePeriod - 2 }, { firstSequenceStart - 16, lastSequenceStart - 16, sequenceSize - 3, sequencePeriod - 0 }, { firstSequenceStart - 21, lastSequenceStart - 21, sequenceSize - 2, sequencePeriod - 0 } + +As you can notice to represent a SequenceGroup, two long values and two int values are needed, so each SequenceGroup can be represented with (2 * 8 + 2 * 4 = 24 bytes). + +In the ‘availabilityOfEntriesOfLedger’ byte array, for the sake of future extensibility it would be helpful to have reserved space for metadata at the beginning. So the first 64 bytes will be used for metadata, with the first four bytes specifying the int version number, next four bytes specifying the number of entries for now and the rest of the bytes in the reserved space will be 0's. The encoded format will be represented after the first 64 bytes. The ordered collection of SequenceGroups will be appended sequentially to this byte array, with each SequenceGroup taking 24 bytes. + +So for a ledger having thousands of entries, this condensed encoded format would need one or two SequenceGroups (in the best case, with no holes and no overreplication) 24/48 bytes, which would be much less than what is needed to represent using bit vector (array of bits indicating availability of entry at that particular index location) + +Any encoded format needs encoder and decoder at the sending/receiving ends of the channel, so the encoding/decoding logic should be handled optimally from computation and memory perspective. + +Here Bookie is expected to just attain index information (say from LedgerCache (Index Files) - IndexPersistenceMgr and IndexInMemPageMgr, unflushed entries in EntryMemTable in SortedLedgerStorage case and from rocksdb database in DBLedgerStorage case) but it doesn’t actually check the availability of this entry in Entrylogger. Since the intention of this checker is limited to do just metadata validation at cluster level. ### Compatibility, Deprecation, and Migration Plan From e5a591371651469a9d589fb1dc93b891d67cf70b Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Mon, 19 Nov 2018 11:26:39 +0100 Subject: [PATCH 0124/1642] Rollback snakeyaml version (Fixes IT tests) The stats gen (#1787) change introduced a version of snakeyaml which is incompatible with arquillian-cube (between 1.19 and 1.23 they remove some methods). This was picked up by the integration tests, but overriden to submit. This change pins the snakeyaml version at 1.19, thereby allowing the integration tests to run again. Reviewers: Enrico Olivelli This closes #1817 from ivankelly/fix-it --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8f151b85042..125f2e2458a 100644 --- a/pom.xml +++ b/pom.xml @@ -163,7 +163,7 @@ 5.13.1 3.0.1 1.7.25 - 1.23 + 1.19 3.1.1 1.3.2 1.8.3 From 62057d12c53d43ed903bd9628a0eda2af763b78d Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Mon, 19 Nov 2018 15:23:36 +0100 Subject: [PATCH 0125/1642] Change LedgerManager to use CompletableFuture All new uses of LedgerManager are creating GenericCallbackFutures to pass in as the callback. This patch cuts out the middleman, and makes LedgerManager return CompletableFutures itself. The patch only touches the CRUD operations, and not LedgerManager#asyncProcessLedgers, which I may tackle in a later PR. Master issue: #281 Reviewers: Sijie Guo This closes #1809 from ivankelly/metastore-futures --- .../apache/bookkeeper/bookie/BookieShell.java | 87 ++++---- .../ScanAndCompareGarbageCollector.java | 94 ++++---- .../apache/bookkeeper/client/BKException.java | 16 +- .../apache/bookkeeper/client/BookKeeper.java | 15 +- .../bookkeeper/client/BookKeeperAdmin.java | 64 ++---- .../bookkeeper/client/LedgerCreateOp.java | 73 ++++--- .../bookkeeper/client/LedgerDeleteOp.java | 27 +-- .../bookkeeper/client/LedgerOpenOp.java | 23 +- .../bookkeeper/client/MetadataUpdateLoop.java | 54 ++--- .../bookkeeper/client/UpdateLedgerOp.java | 6 +- .../meta/AbstractZkLedgerManager.java | 106 +++++---- .../bookkeeper/meta/CleanupLedgerManager.java | 61 ++++-- .../apache/bookkeeper/meta/LedgerManager.java | 54 ++--- .../meta/MSLedgerManagerFactory.java | 79 +++---- .../replication/BookieLedgerIndexer.java | 49 ++--- .../http/service/GetLedgerMetaService.java | 5 +- .../http/service/ListLedgerService.java | 55 ++--- .../bookkeeper/bookie/CompactionTest.java | 22 +- .../bookie/TestGcOverreplicatedLedger.java | 61 +----- .../bookkeeper/client/BookieRecoveryTest.java | 26 +-- .../apache/bookkeeper/client/ClientUtil.java | 14 +- .../bookkeeper/client/HandleFailuresTest.java | 4 +- .../client/LedgerRecovery2Test.java | 4 +- .../client/MetadataUpdateLoopTest.java | 112 +++++----- .../client/MockBookKeeperTestCase.java | 52 ++--- .../client/ParallelLedgerRecoveryTest.java | 33 +-- .../client/TestWatchEnsembleChange.java | 38 ++-- .../meta/AbstractZkLedgerManagerTest.java | 75 ++----- .../apache/bookkeeper/meta/GcLedgersTest.java | 62 ++---- .../meta/LedgerManagerIteratorTest.java | 204 +++++------------- .../bookkeeper/meta/MockLedgerManager.java | 42 ++-- .../replication/AuditorLedgerCheckerTest.java | 14 +- .../metadata/etcd/EtcdLedgerManager.java | 91 ++++---- .../metadata/etcd/EtcdLedgerManagerTest.java | 120 +++-------- .../TestCompatRecoveryNoPassword.groovy | 11 +- 35 files changed, 732 insertions(+), 1121 deletions(-) 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 df07d61522f..a4a437c6993 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 @@ -105,8 +105,6 @@ import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.proto.BookieProtocol; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.replication.ReplicationException; @@ -1024,41 +1022,45 @@ public int runCmd(CommandLine cmdLine) throws Exception { final CountDownLatch processDone = new CountDownLatch(1); Processor ledgerProcessor = new Processor() { - @Override - public void process(Long ledgerId, VoidCallback cb) { - if (!printMeta && (bookieAddress == null)) { - printLedgerMetadata(ledgerId, null, false); - cb.processResult(BKException.Code.OK, null, null); - } else { - GenericCallback> gencb = (rc, ledgerMetadata) -> { - if (rc == BKException.Code.OK) { - if ((bookieAddress == null) - || BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie( - ledgerId, bookieAddress, ledgerMetadata.getValue())) { - /* - * the print method has to be in - * synchronized scope, otherwise - * output of printLedgerMetadata - * could interleave since this - * callback for different - * ledgers can happen in - * different threads. - */ - synchronized (BookieShell.this) { - printLedgerMetadata(ledgerId, ledgerMetadata.getValue(), printMeta); - } - } - } else if (rc == BKException.Code.NoSuchLedgerExistsException) { - rc = BKException.Code.OK; - } else { - LOG.error("Unable to read the ledger: " + ledgerId + " information"); - } - cb.processResult(rc, null, null); - }; - ledgerManager.readLedgerMetadata(ledgerId, gencb); + @Override + public void process(Long ledgerId, VoidCallback cb) { + if (!printMeta && (bookieAddress == null)) { + printLedgerMetadata(ledgerId, null, false); + cb.processResult(BKException.Code.OK, null, null); + } else { + ledgerManager.readLedgerMetadata(ledgerId).whenComplete( + (metadata, exception) -> { + if (exception == null) { + if ((bookieAddress == null) + || BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie( + ledgerId, bookieAddress, metadata.getValue())) { + /* + * the print method has to be in + * synchronized scope, otherwise + * output of printLedgerMetadata + * could interleave since this + * callback for different + * ledgers can happen in + * different threads. + */ + synchronized (BookieShell.this) { + printLedgerMetadata(ledgerId, metadata.getValue(), + printMeta); + } + } + cb.processResult(BKException.Code.OK, null, null); + } else if (BKException.getExceptionCode(exception) + == BKException.Code.NoSuchLedgerExistsException) { + cb.processResult(BKException.Code.OK, null, null); + } else { + LOG.error("Unable to read the ledger: {} information", ledgerId); + cb.processResult(BKException.getExceptionCode(exception), + null, null); + } + }); + } } - } - }; + }; ledgerManager.asyncProcessLedgers(ledgerProcessor, new AsyncCallback.VoidCallback() { @Override @@ -1135,21 +1137,16 @@ public int runCmd(CommandLine cmdLine) throws Exception { runFunctionWithLedgerManagerFactory(bkConf, mFactory -> { try (LedgerManager m = mFactory.newLedgerManager()) { if (cmdLine.hasOption("dumptofile")) { - GenericCallbackFuture> cb = new GenericCallbackFuture<>(); - m.readLedgerMetadata(lid, cb); + Versioned md = m.readLedgerMetadata(lid).join(); Files.write(FileSystems.getDefault().getPath(cmdLine.getOptionValue("dumptofile")), - cb.join().getValue().serialize()); + md.getValue().serialize()); } else if (cmdLine.hasOption("restorefromfile")) { byte[] serialized = Files.readAllBytes( FileSystems.getDefault().getPath(cmdLine.getOptionValue("restorefromfile"))); LedgerMetadata md = LedgerMetadata.parseConfig(serialized, Optional.absent()); - GenericCallbackFuture> cb = new GenericCallbackFuture<>(); - m.createLedgerMetadata(lid, md, cb); - cb.join(); + m.createLedgerMetadata(lid, md).join(); } else { - GenericCallbackFuture> cb = new GenericCallbackFuture<>(); - m.readLedgerMetadata(lid, cb); - printLedgerMetadata(lid, cb.get().getValue(), true); + printLedgerMetadata(lid, m.readLedgerMetadata(lid).get().getValue(), true); } } catch (Exception e) { throw new UncheckedExecutionException(e); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index 19d630ff75d..d4c8bc2b9fe 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -23,6 +23,7 @@ import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ACTIVE_LEDGER_COUNT; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.DELETED_LEDGER_COUNT; +import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import com.google.common.collect.Sets; import java.io.IOException; @@ -33,9 +34,7 @@ import java.util.TreeSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; -import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; @@ -43,13 +42,11 @@ import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.ZkUtils; -import org.apache.bookkeeper.versioning.Versioned; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; @@ -182,18 +179,15 @@ public void gc(GarbageCleaner garbageCleaner) { for (Long bkLid : subBkActiveLedgers) { if (!ledgersInMetadata.contains(bkLid)) { if (verifyMetadataOnGc) { - CountDownLatch latch = new CountDownLatch(1); - final AtomicInteger metaRC = new AtomicInteger(0); - ledgerManager.readLedgerMetadata(bkLid, (rc, ignore) -> { - metaRC.set(rc); - latch.countDown(); - }); - latch.await(); - if (metaRC.get() != BKException.Code.NoSuchLedgerExistsException) { - LOG.warn( - "Ledger {} Missing in metadata list, but ledgerManager returned rc: {}.", - bkLid, - metaRC.get()); + int rc = BKException.Code.OK; + try { + result(ledgerManager.readLedgerMetadata(bkLid)); + } catch (BKException e) { + rc = e.getCode(); + } + if (rc != BKException.Code.NoSuchLedgerExistsException) { + LOG.warn("Ledger {} Missing in metadata list, but ledgerManager returned rc: {}.", + bkLid, rc); continue; } } @@ -237,44 +231,40 @@ private Set removeOverReplicatedledgers(Set bkActiveledgers, final G ZkLedgerUnderreplicationManager.acquireUnderreplicatedLedgerLock(zk, zkLedgersRootPath, ledgerId, zkAcls); semaphore.acquire(); - ledgerManager.readLedgerMetadata(ledgerId, new GenericCallback>() { - - @Override - public void operationComplete(int rc, Versioned ledgerMetadata) { - if (rc == BKException.Code.OK) { - // do not delete a ledger that is not closed, since the ensemble might change again and - // include the current bookie while we are deleting it - if (!ledgerMetadata.getValue().isClosed()) { - release(); - return; - } - SortedMap> ensembles = - ledgerMetadata.getValue().getAllEnsembles(); - for (List ensemble : ensembles.values()) { - // check if this bookie is supposed to have this ledger - if (ensemble.contains(selfBookieAddress)) { - release(); - return; + ledgerManager.readLedgerMetadata(ledgerId) + .whenComplete((metadata, exception) -> { + try { + if (exception == null) { + // do not delete a ledger that is not closed, since the ensemble might + // change again and include the current bookie while we are deleting it + if (!metadata.getValue().isClosed()) { + return; + } + SortedMap> ensembles = + metadata.getValue().getAllEnsembles(); + for (List ensemble : ensembles.values()) { + // check if this bookie is supposed to have this ledger + if (ensemble.contains(selfBookieAddress)) { + return; + } + } + // this bookie is not supposed to have this ledger, + // thus we can delete this ledger now + overReplicatedLedgers.add(ledgerId); + garbageCleaner.clean(ledgerId); + } + } finally { + semaphore.release(); + latch.countDown(); + try { + ZkLedgerUnderreplicationManager.releaseUnderreplicatedLedgerLock( + zk, zkLedgersRootPath, ledgerId); + } catch (Throwable t) { + LOG.error("Exception when removing underreplicated lock for ledger {}", + ledgerId, t); } } - // this bookie is not supposed to have this ledger, thus we can delete this ledger now - overReplicatedLedgers.add(ledgerId); - garbageCleaner.clean(ledgerId); - } - release(); - } - - private void release() { - semaphore.release(); - latch.countDown(); - try { - ZkLedgerUnderreplicationManager.releaseUnderreplicatedLedgerLock(zk, zkLedgersRootPath, - ledgerId); - } catch (Throwable t) { - LOG.error("Exception when removing underreplicated lock for ledger {}", ledgerId, t); - } - } - }); + }); } catch (Throwable t) { LOG.error("Exception when iterating through the ledgers to check for over-replication", t); latch.countDown(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java index d6304927e5e..912fb1ee4c2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java @@ -437,9 +437,13 @@ public BKLedgerIdOverflowException() { /** * Extract an exception code from an BKException, or use a default if it's another type. + * The throwable is null, assume that no exception took place and return + * {@link BKException.Code.OK}. */ public static int getExceptionCode(Throwable t, int defaultCode) { - if (t instanceof BKException) { + if (t == null) { + return BKException.Code.OK; + } else if (t instanceof BKException) { return ((BKException) t).getCode(); } else if (t.getCause() != null) { return getExceptionCode(t.getCause(), defaultCode); @@ -447,4 +451,14 @@ public static int getExceptionCode(Throwable t, int defaultCode) { return defaultCode; } } + + /** + * Extract an exception code from an BKException, or default to unexpected exception if throwable + * is not a BKException. + * + * @see #getExceptionCode(Throwable,int) + */ + public static int getExceptionCode(Throwable t) { + return getExceptionCode(t, Code.UnexpectedConditionException); + } } 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 4d0fb32ffce..14760a8879e 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 @@ -73,13 +73,11 @@ import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.DataFormats; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.bookkeeper.util.SafeRunnable; -import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.lang.SystemUtils; import org.apache.zookeeper.KeeperException; @@ -1300,16 +1298,13 @@ public void deleteLedger(long lId) throws InterruptedException, BKException { * @param cb callback method */ public void asyncIsClosed(long lId, final IsClosedCallback cb, final Object ctx){ - ledgerManager.readLedgerMetadata(lId, new GenericCallback>(){ - @Override - public void operationComplete(int rc, Versioned lm){ - if (rc == BKException.Code.OK) { - cb.isClosedComplete(rc, lm.getValue().isClosed(), ctx); + ledgerManager.readLedgerMetadata(lId).whenComplete((metadata, exception) -> { + if (exception == null) { + cb.isClosedComplete(BKException.Code.OK, metadata.getValue().isClosed(), ctx); } else { - cb.isClosedComplete(rc, false, ctx); + cb.isClosedComplete(BKException.getExceptionCode(exception), false, ctx); } - } - }); + }); } /** 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 5aa20867c47..3f7cee45e80 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 @@ -27,7 +27,6 @@ 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 com.google.common.util.concurrent.UncheckedExecutionException; import java.io.File; @@ -82,7 +81,6 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.IOUtils; -import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -491,26 +489,25 @@ public void asyncGetLedgersContainBookies(final Set bookies bkc.getLedgerManager().asyncProcessLedgers(new Processor() { @Override public void process(final Long lid, final AsyncCallback.VoidCallback cb) { - bkc.getLedgerManager().readLedgerMetadata(lid, new GenericCallback>() { - @Override - public void operationComplete(int rc, Versioned metadata) { - if (BKException.Code.NoSuchLedgerExistsException == rc) { - // the ledger was deleted during this iteration. - cb.processResult(BKException.Code.OK, null, null); - return; - } else if (BKException.Code.OK != rc) { - cb.processResult(rc, null, null); - return; - } - Set bookiesInLedger = metadata.getValue().getBookiesInThisLedger(); - Sets.SetView intersection = + bkc.getLedgerManager().readLedgerMetadata(lid) + .whenComplete((metadata, exception) -> { + if (BKException.getExceptionCode(exception) + == BKException.Code.NoSuchLedgerExistsException) { + // the ledger was deleted during this iteration. + cb.processResult(BKException.Code.OK, null, null); + return; + } else if (exception != null) { + cb.processResult(BKException.getExceptionCode(exception), null, null); + return; + } + Set bookiesInLedger = metadata.getValue().getBookiesInThisLedger(); + Sets.SetView intersection = Sets.intersection(bookiesInLedger, bookies); - if (!intersection.isEmpty()) { - ledgers.put(lid, metadata.getValue()); - } - cb.processResult(BKException.Code.OK, null, null); - } - }); + if (!intersection.isEmpty()) { + ledgers.put(lid, metadata.getValue()); + } + cb.processResult(BKException.Code.OK, null, null); + }); } }, new AsyncCallback.VoidCallback() { @Override @@ -1525,10 +1522,8 @@ private void waitForLedgersToBeReplicated(Collection ledgers, BookieSocket public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieSocketAddress bookieAddress, LedgerManager ledgerManager) { - ReadMetadataCallback cb = new ReadMetadataCallback(ledgerId); - ledgerManager.readLedgerMetadata(ledgerId, cb); try { - LedgerMetadata ledgerMetadata = cb.get(); + LedgerMetadata ledgerMetadata = ledgerManager.readLedgerMetadata(ledgerId).get().getValue(); return areEntriesOfLedgerStoredInTheBookie(ledgerId, bookieAddress, ledgerMetadata); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -1636,25 +1631,4 @@ private static boolean areEntriesOfSegmentStoredInTheBookie(LedgerMetadata ledge } return firstStoredEntryId != LedgerHandle.INVALID_ENTRY_ID; } - - static class ReadMetadataCallback extends AbstractFuture - implements GenericCallback> { - final long ledgerId; - - ReadMetadataCallback(long ledgerId) { - this.ledgerId = ledgerId; - } - - long getLedgerId() { - return ledgerId; - } - - public void operationComplete(int rc, Versioned result) { - if (rc != 0) { - setException(BKException.create(rc)); - } else { - set(result.getValue()); - } - } - } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java index b55e7ba9ca3..16a6e9547c7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java @@ -54,7 +54,7 @@ * Encapsulates asynchronous ledger create operation. * */ -class LedgerCreateOp implements GenericCallback> { +class LedgerCreateOp { static final Logger LOG = LoggerFactory.getLogger(LedgerCreateOp.class); @@ -151,7 +151,8 @@ public void initiate() { generateLedgerIdAndCreateLedger(); } else { // Create ledger with supplied ledgerId - bk.getLedgerManager().createLedgerMetadata(ledgerId, metadata, LedgerCreateOp.this); + bk.getLedgerManager().createLedgerMetadata(ledgerId, metadata) + .whenComplete((written, exception) -> metadataCallback(written, exception)); } } @@ -167,7 +168,8 @@ public void operationComplete(int rc, Long ledgerId) { } LedgerCreateOp.this.ledgerId = ledgerId; // create a ledger with metadata - bk.getLedgerManager().createLedgerMetadata(ledgerId, metadata, LedgerCreateOp.this); + bk.getLedgerManager().createLedgerMetadata(ledgerId, metadata) + .whenComplete((written, exception) -> metadataCallback(written, exception)); } }); } @@ -185,44 +187,45 @@ public void initiateAdv(final long ledgerId) { } /** - * Callback when created ledger. + * Callback when metadata store has responded. */ - @Override - public void operationComplete(int rc, Versioned writtenMetadata) { - if (this.generateLedgerId && (BKException.Code.LedgerExistException == rc)) { - // retry to generate a new ledger id - generateLedgerIdAndCreateLedger(); - return; - } else if (BKException.Code.OK != rc) { - createComplete(rc, null); - return; - } - - try { - if (adv) { - lh = new LedgerHandleAdv(bk.getClientCtx(), ledgerId, writtenMetadata, digestType, passwd, writeFlags); + private void metadataCallback(Versioned writtenMetadata, Throwable exception) { + if (exception != null) { + if (this.generateLedgerId + && (BKException.getExceptionCode(exception) == BKException.Code.LedgerExistException)) { + // retry to generate a new ledger id + generateLedgerIdAndCreateLedger(); } else { - lh = new LedgerHandle(bk.getClientCtx(), ledgerId, writtenMetadata, digestType, passwd, writeFlags); + createComplete(BKException.getExceptionCode(exception), null); + } + } else { + try { + if (adv) { + lh = new LedgerHandleAdv(bk.getClientCtx(), ledgerId, writtenMetadata, + digestType, passwd, writeFlags); + } else { + lh = new LedgerHandle(bk.getClientCtx(), ledgerId, writtenMetadata, digestType, passwd, writeFlags); + } + } catch (GeneralSecurityException e) { + LOG.error("Security exception while creating ledger: " + ledgerId, e); + createComplete(BKException.Code.DigestNotInitializedException, null); + return; + } catch (NumberFormatException e) { + LOG.error("Incorrectly entered parameter throttle: " + bk.getConf().getThrottleValue(), e); + createComplete(BKException.Code.IncorrectParameterException, null); + return; } - } catch (GeneralSecurityException e) { - LOG.error("Security exception while creating ledger: " + ledgerId, e); - createComplete(BKException.Code.DigestNotInitializedException, null); - return; - } catch (NumberFormatException e) { - LOG.error("Incorrectly entered parameter throttle: " + bk.getConf().getThrottleValue(), e); - createComplete(BKException.Code.IncorrectParameterException, null); - return; - } - List curEns = lh.getLedgerMetadata().getEnsemble(0L); - LOG.info("Ensemble: {} for ledger: {}", curEns, lh.getId()); + List curEns = lh.getLedgerMetadata().getEnsemble(0L); + LOG.info("Ensemble: {} for ledger: {}", curEns, lh.getId()); - for (BookieSocketAddress bsa : curEns) { - clientStats.getEnsembleBookieDistributionCounter(bsa.toString()).inc(); - } + for (BookieSocketAddress bsa : curEns) { + clientStats.getEnsembleBookieDistributionCounter(bsa.toString()).inc(); + } - // return the ledger handle back - createComplete(BKException.Code.OK, lh); + // return the ledger handle back + createComplete(BKException.Code.OK, lh); + } } private void createComplete(int rc, LedgerHandle lh) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerDeleteOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerDeleteOp.java index f2461d50898..89837ae5207 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerDeleteOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerDeleteOp.java @@ -30,7 +30,6 @@ import org.apache.bookkeeper.client.api.DeleteBuilder; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.MathUtils; -import org.apache.bookkeeper.util.OrderedGenericCallback; import org.apache.bookkeeper.versioning.Version; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,7 +38,7 @@ * Encapsulates asynchronous ledger delete operation. * */ -class LedgerDeleteOp extends OrderedGenericCallback { +class LedgerDeleteOp { static final Logger LOG = LoggerFactory.getLogger(LedgerDeleteOp.class); @@ -64,7 +63,6 @@ class LedgerDeleteOp extends OrderedGenericCallback { */ LedgerDeleteOp(BookKeeper bk, BookKeeperClientStats clientStats, long ledgerId, DeleteCallback cb, Object ctx) { - super(bk.getMainWorkerPool(), ledgerId); this.bk = bk; this.ledgerId = ledgerId; this.cb = cb; @@ -79,20 +77,15 @@ class LedgerDeleteOp extends OrderedGenericCallback { public void initiate() { // Asynchronously delete the ledger from meta manager // When this completes, it will invoke the callback method below. - bk.getLedgerManager().removeLedgerMetadata(ledgerId, Version.ANY, this); - } - - /** - * Implements Delete Callback. - */ - @Override - public void safeOperationComplete(int rc, Void result) { - if (BKException.Code.OK != rc) { - deleteOpLogger.registerFailedEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); - } else { - deleteOpLogger.registerSuccessfulEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); - } - cb.deleteComplete(rc, this.ctx); + bk.getLedgerManager().removeLedgerMetadata(ledgerId, Version.ANY) + .whenCompleteAsync((ignore, exception) -> { + if (exception != null) { + deleteOpLogger.registerFailedEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); + } else { + deleteOpLogger.registerSuccessfulEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); + } + cb.deleteComplete(BKException.getExceptionCode(exception), this.ctx); + }, bk.getMainWorkerPool().chooseThread(ledgerId)); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java index 16d7bd59b3c..c2dc33a6c63 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java @@ -36,7 +36,6 @@ import org.apache.bookkeeper.client.api.BKException.Code; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.client.impl.OpenBuilderBase; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.OrderedGenericCallback; @@ -48,7 +47,7 @@ * Encapsulates the ledger open operation. * */ -class LedgerOpenOp implements GenericCallback> { +class LedgerOpenOp { static final Logger LOG = LoggerFactory.getLogger(LedgerOpenOp.class); final BookKeeper bk; @@ -111,7 +110,14 @@ public void initiate() { /** * Asynchronously read the ledger metadata node. */ - bk.getLedgerManager().readLedgerMetadata(ledgerId, this); + bk.getLedgerManager().readLedgerMetadata(ledgerId) + .whenComplete((metadata, exception) -> { + if (exception != null) { + openComplete(BKException.getExceptionCode(exception), null); + } else { + openWithMetadata(metadata); + } + }); } /** @@ -122,16 +128,7 @@ public void initiateWithoutRecovery() { initiate(); } - /** - * Implements Open Ledger Callback. - */ - @Override - public void operationComplete(int rc, Versioned versionedMetadata) { - if (BKException.Code.OK != rc) { - // open ledger failed. - openComplete(rc, null); - return; - } + private void openWithMetadata(Versioned versionedMetadata) { LedgerMetadata metadata = versionedMetadata.getValue(); final byte[] passwd; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java index 821a4dac174..435ff69f62d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java @@ -120,7 +120,7 @@ private void writeLoop(Versioned currentLocal, if (needsTransformation.needsUpdate(currentLocal.getValue())) { LedgerMetadata transformed = transform.transform(currentLocal.getValue()); - writeToStore(ledgerId, transformed, currentLocal.getVersion()) + lm.writeLedgerMetadata(ledgerId, transformed, currentLocal.getVersion()) .whenComplete((writtenMetadata, ex) -> { if (ex == null) { if (updateLocalValue.updateValue(currentLocal, writtenMetadata)) { @@ -166,39 +166,23 @@ private CompletableFuture> updateLocalValueFromStore(l private void readLoop(long ledgerId, CompletableFuture> promise) { Versioned current = currentLocalValue.get(); - lm.readLedgerMetadata(ledgerId, - (rc, read) -> { - if (rc != BKException.Code.OK) { - LOG.error("{} Failed to read metadata from store, rc = {}", - logContext, rc); - promise.completeExceptionally(BKException.create(rc)); - } else if (current.getVersion().compare(read.getVersion()) - == Version.Occurred.CONCURRENTLY) { - // no update needed, these are the same in the immutable world - promise.complete(current); - } else if (updateLocalValue.updateValue(current, read)) { - // updated local value successfully - promise.complete(read); - } else { - // local value changed while we were reading, - // look at new value, and try to read again - readLoop(ledgerId, promise); - } - }); - } - - private CompletableFuture> writeToStore(long ledgerId, LedgerMetadata toWrite, - Version currentVersion) { - CompletableFuture> promise = new CompletableFuture<>(); - - lm.writeLedgerMetadata(ledgerId, toWrite, currentVersion, - (rc, written) -> { - if (rc != BKException.Code.OK) { - promise.completeExceptionally(BKException.create(rc)); - } else { - promise.complete(written); - } - }); - return promise; + lm.readLedgerMetadata(ledgerId).whenComplete( + (read, exception) -> { + if (exception != null) { + LOG.error("{} Failed to read metadata from store", + logContext, exception); + promise.completeExceptionally(exception); + } else if (current.getVersion().compare(read.getVersion()) == Version.Occurred.CONCURRENTLY) { + // no update needed, these are the same in the immutable world + promise.complete(current); + } else if (updateLocalValue.updateValue(current, read)) { + // updated local value successfully + promise.complete(read); + } else { + // local value changed while we were reading, + // look at new value, and try to read again + readLoop(ledgerId, promise); + } + }); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java index fec6e58af4b..98c544e9a26 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java @@ -37,7 +37,6 @@ import org.apache.bookkeeper.bookie.BookieShell.UpdateLedgerNotifier; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,9 +94,8 @@ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final final long ledgerId = ledgerItr.next(); issuedLedgerCnt.incrementAndGet(); - GenericCallbackFuture> readPromise = new GenericCallbackFuture<>(); - lm.readLedgerMetadata(ledgerId, readPromise); - CompletableFuture> writePromise = readPromise.thenCompose((readMetadata) -> { + CompletableFuture> writePromise = lm.readLedgerMetadata(ledgerId) + .thenCompose((readMetadata) -> { AtomicReference> ref = new AtomicReference<>(readMetadata); return new MetadataUpdateLoop( lm, ledgerId, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index b0af1ba8498..a23c3302427 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -28,6 +28,7 @@ import java.util.NavigableSet; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executors; @@ -36,6 +37,7 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; @@ -86,7 +88,7 @@ public abstract class AbstractZkLedgerManager implements LedgerManager, Watcher /** * ReadLedgerMetadataTask class. */ - protected class ReadLedgerMetadataTask implements Runnable, GenericCallback> { + protected class ReadLedgerMetadataTask implements Runnable { final long ledgerId; @@ -100,7 +102,8 @@ public void run() { if (LOG.isDebugEnabled()) { LOG.debug("Re-read ledger metadata for {}.", ledgerId); } - readLedgerMetadata(ledgerId, this, AbstractZkLedgerManager.this); + readLedgerMetadata(ledgerId, AbstractZkLedgerManager.this) + .whenComplete((metadata, exception) -> handleMetadata(metadata, exception)); } else { if (LOG.isDebugEnabled()) { LOG.debug("Ledger metadata listener for ledger {} is already removed.", ledgerId); @@ -108,26 +111,22 @@ public void run() { } } - @Override - public void operationComplete(int rc, final Versioned result) { - if (BKException.Code.OK == rc) { + private void handleMetadata(Versioned result, Throwable exception) { + if (exception == null) { final Set listenerSet = listeners.get(ledgerId); if (null != listenerSet) { if (LOG.isDebugEnabled()) { LOG.debug("Ledger metadata is changed for {} : {}.", ledgerId, result); } - scheduler.submit(new Runnable() { - @Override - public void run() { + scheduler.submit(() -> { synchronized (listenerSet) { for (LedgerMetadataListener listener : listenerSet) { listener.onChanged(ledgerId, result); } } - } - }); + }); } - } else if (BKException.Code.NoSuchLedgerExistsException == rc) { + } else if (BKException.getExceptionCode(exception) == BKException.Code.NoSuchLedgerExistsException) { // the ledger is removed, do nothing Set listenerSet = listeners.remove(ledgerId); if (null != listenerSet) { @@ -144,7 +143,8 @@ public void run() { } } } else { - LOG.warn("Failed on read ledger metadata of ledger {} : {}", ledgerId, rc); + LOG.warn("Failed on read ledger metadata of ledger {}: {}", + ledgerId, BKException.getExceptionCode(exception)); scheduler.schedule(this, ZK_CONNECT_BACKOFF_MS, TimeUnit.MILLISECONDS); } } @@ -245,53 +245,43 @@ public void process(WatchedEvent event) { } @Override - public void createLedgerMetadata(final long ledgerId, final LedgerMetadata metadata, - final GenericCallback> ledgerCb) { + public CompletableFuture> createLedgerMetadata(long ledgerId, LedgerMetadata metadata) { + CompletableFuture> promise = new CompletableFuture<>(); String ledgerPath = getLedgerPath(ledgerId); StringCallback scb = new StringCallback() { @Override public void processResult(int rc, String path, Object ctx, String name) { if (rc == Code.OK.intValue()) { - ledgerCb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, new LongVersion(0))); + promise.complete(new Versioned<>(metadata, new LongVersion(0))); } else if (rc == Code.NODEEXISTS.intValue()) { LOG.warn("Failed to create ledger metadata for {} which already exist", ledgerId); - ledgerCb.operationComplete(BKException.Code.LedgerExistException, null); + promise.completeExceptionally(new BKException.BKLedgerExistException()); } else { LOG.error("Could not create node for ledger {}", ledgerId, KeeperException.create(Code.get(rc), path)); - ledgerCb.operationComplete(BKException.Code.ZKException, null); + promise.completeExceptionally(new BKException.ZKException()); } } }; List zkAcls = ZkUtils.getACLs(conf); ZkUtils.asyncCreateFullPathOptimistic(zk, ledgerPath, metadata.serialize(), zkAcls, CreateMode.PERSISTENT, scb, null); + return promise; } - /** - * Removes ledger metadata from ZooKeeper and deletes its parent znodes - * recursively if they dont have anymore children. - * - * @param ledgerId - * ledger identifier - * @param version - * local version of metadata znode - * @param cb - * callback object - */ @Override - public void removeLedgerMetadata(final long ledgerId, final Version version, - final GenericCallback cb) { + public CompletableFuture removeLedgerMetadata(final long ledgerId, final Version version) { + CompletableFuture promise = new CompletableFuture<>(); int znodeVersion = -1; if (Version.NEW == version) { LOG.error("Request to delete ledger {} metadata with version set to the initial one", ledgerId); - cb.operationComplete(BKException.Code.MetadataVersionException, (Void) null); - return; + promise.completeExceptionally(new BKException.BKMetadataVersionException()); + return promise; } else if (Version.ANY != version) { if (!(version instanceof LongVersion)) { LOG.info("Not an instance of ZKVersion: {}", ledgerId); - cb.operationComplete(BKException.Code.MetadataVersionException, (Void) null); - return; + promise.completeExceptionally(new BKException.BKMetadataVersionException()); + return promise; } else { znodeVersion = (int) ((LongVersion) version).getLongVersion(); } @@ -300,10 +290,9 @@ public void removeLedgerMetadata(final long ledgerId, final Version version, VoidCallback callbackForDelete = new VoidCallback() { @Override public void processResult(int rc, String path, Object ctx) { - int bkRc; if (rc == KeeperException.Code.NONODE.intValue()) { LOG.warn("Ledger node does not exist in ZooKeeper: ledgerId={}", ledgerId); - bkRc = BKException.Code.NoSuchLedgerExistsException; + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); } else if (rc == KeeperException.Code.OK.intValue()) { // removed listener on ledgerId Set listenerSet = listeners.remove(ledgerId); @@ -319,11 +308,10 @@ public void processResult(int rc, String path, Object ctx) { ledgerId); } } - bkRc = BKException.Code.OK; + FutureUtils.complete(promise, null); } else { - bkRc = BKException.Code.ZKException; + promise.completeExceptionally(new BKException.ZKException()); } - cb.operationComplete(bkRc, (Void) null); } }; String ledgerZnodePath = getLedgerPath(ledgerId); @@ -337,6 +325,7 @@ public void processResult(int rc, String path, Object ctx) { } else { zk.delete(ledgerZnodePath, znodeVersion, callbackForDelete, null); } + return promise; } @Override @@ -380,12 +369,12 @@ public void unregisterLedgerMetadataListener(long ledgerId, LedgerMetadataListen } @Override - public void readLedgerMetadata(final long ledgerId, final GenericCallback> readCb) { - readLedgerMetadata(ledgerId, readCb, null); + public CompletableFuture> readLedgerMetadata(long ledgerId) { + return readLedgerMetadata(ledgerId, null); } - protected void readLedgerMetadata(final long ledgerId, final GenericCallback> readCb, - Watcher watcher) { + protected CompletableFuture> readLedgerMetadata(long ledgerId, Watcher watcher) { + CompletableFuture> promise = new CompletableFuture<>(); zk.getData(getLedgerPath(ledgerId), watcher, new DataCallback() { @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { @@ -394,40 +383,41 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta LOG.debug("No such ledger: " + ledgerId, KeeperException.create(KeeperException.Code.get(rc), path)); } - readCb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); return; } if (rc != KeeperException.Code.OK.intValue()) { LOG.error("Could not read metadata for ledger: " + ledgerId, KeeperException.create(KeeperException.Code.get(rc), path)); - readCb.operationComplete(BKException.Code.ZKException, null); + promise.completeExceptionally(new BKException.ZKException()); return; } if (stat == null) { LOG.error("Could not parse ledger metadata for ledger: {}. Stat object is null", ledgerId); - readCb.operationComplete(BKException.Code.ZKException, null); + promise.completeExceptionally(new BKException.ZKException()); return; } try { LongVersion version = new LongVersion(stat.getVersion()); LedgerMetadata metadata = LedgerMetadata.parseConfig(data, Optional.of(stat.getCtime())); - readCb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, version)); + promise.complete(new Versioned<>(metadata, version)); } catch (IOException e) { LOG.error("Could not parse ledger metadata for ledger: " + ledgerId, e); - readCb.operationComplete(BKException.Code.ZKException, null); + promise.completeExceptionally(new BKException.ZKException()); } - } }, null); + return promise; } @Override - public void writeLedgerMetadata(final long ledgerId, final LedgerMetadata metadata, final Version currentVersion, - final GenericCallback> cb) { + public CompletableFuture> writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion) { + CompletableFuture> promise = new CompletableFuture<>(); if (!(currentVersion instanceof LongVersion)) { - cb.operationComplete(BKException.Code.MetadataVersionException, null); - return; + promise.completeExceptionally(new BKException.BKMetadataVersionException()); + return promise; } final LongVersion zv = (LongVersion) currentVersion; zk.setData(getLedgerPath(ledgerId), @@ -436,20 +426,20 @@ public void writeLedgerMetadata(final long ledgerId, final LedgerMetadata metada @Override public void processResult(int rc, String path, Object ctx, Stat stat) { if (KeeperException.Code.BADVERSION.intValue() == rc) { - cb.operationComplete(BKException.Code.MetadataVersionException, null); + promise.completeExceptionally(new BKException.BKMetadataVersionException()); } else if (KeeperException.Code.OK.intValue() == rc) { // update metadata version - cb.operationComplete(BKException.Code.OK, - new Versioned<>(metadata, new LongVersion(stat.getVersion()))); + promise.complete(new Versioned<>(metadata, new LongVersion(stat.getVersion()))); } else if (KeeperException.Code.NONODE.intValue() == rc) { LOG.warn("Ledger node does not exist in ZooKeeper: ledgerId={}", ledgerId); - cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); } else { LOG.warn("Conditional update ledger metadata failed: {}", KeeperException.Code.get(rc)); - cb.operationComplete(BKException.Code.ZKException, null); + promise.completeExceptionally(new BKException.ZKException()); } } }, null); + return promise; } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java index 6a87cf2823c..311cb0aa3e6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java @@ -22,12 +22,14 @@ import java.io.IOException; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; @@ -80,6 +82,7 @@ public LedgerRange next() throws IOException { new ConcurrentHashMap(); private boolean closed = false; private final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock(); + private final Set> futures = ConcurrentHashMap.newKeySet(); public CleanupLedgerManager(LedgerManager lm) { this.underlying = lm; @@ -108,64 +111,69 @@ private GenericCallback removeCallback(GenericCallback callback) { return callbacks.remove(callback); } + private void recordPromise(CompletableFuture promise) { + futures.add(promise); + promise.thenRun(() -> futures.remove(promise)); + } + @Override - public void createLedgerMetadata(long lid, LedgerMetadata metadata, - GenericCallback> cb) { + public CompletableFuture> createLedgerMetadata(long lid, LedgerMetadata metadata) { closeLock.readLock().lock(); try { if (closed) { - cb.operationComplete(BKException.Code.ClientClosedException, null); - return; + return closedPromise(); + } else { + CompletableFuture> promise = underlying.createLedgerMetadata(lid, metadata); + recordPromise(promise); + return promise; } - underlying.createLedgerMetadata(lid, metadata, new CleanupGenericCallback<>(cb)); } finally { closeLock.readLock().unlock(); } } @Override - public void removeLedgerMetadata(long ledgerId, Version version, - GenericCallback vb) { + public CompletableFuture removeLedgerMetadata(long ledgerId, Version version) { closeLock.readLock().lock(); try { if (closed) { - vb.operationComplete(BKException.Code.ClientClosedException, null); - return; + return closedPromise(); } - underlying.removeLedgerMetadata(ledgerId, version, - new CleanupGenericCallback(vb)); + CompletableFuture promise = underlying.removeLedgerMetadata(ledgerId, version); + recordPromise(promise); + return promise; } finally { closeLock.readLock().unlock(); } } @Override - public void readLedgerMetadata(long ledgerId, - GenericCallback> readCb) { + public CompletableFuture> readLedgerMetadata(long ledgerId) { closeLock.readLock().lock(); try { if (closed) { - readCb.operationComplete(BKException.Code.ClientClosedException, null); - return; + return closedPromise(); } - underlying.readLedgerMetadata(ledgerId, new CleanupGenericCallback<>(readCb)); + CompletableFuture> promise = underlying.readLedgerMetadata(ledgerId); + recordPromise(promise); + return promise; } finally { closeLock.readLock().unlock(); } } @Override - public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - Version currentVersion, - GenericCallback> cb) { + public CompletableFuture> writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion) { closeLock.readLock().lock(); try { if (closed) { - cb.operationComplete(BKException.Code.ClientClosedException, null); - return; + return closedPromise(); } - underlying.writeLedgerMetadata(ledgerId, metadata, currentVersion, - new CleanupGenericCallback<>(cb)); + CompletableFuture> promise = + underlying.writeLedgerMetadata(ledgerId, metadata, currentVersion); + recordPromise(promise); + return promise; } finally { closeLock.readLock().unlock(); } @@ -234,6 +242,13 @@ public void close() throws IOException { callback.operationComplete(BKException.Code.ClientClosedException, null); } } + BKException exception = new BKException.BKClientClosedException(); + futures.forEach((f) -> f.completeExceptionally(exception)); + futures.clear(); underlying.close(); } + + private static CompletableFuture closedPromise() { + return FutureUtils.exception(new BKException.BKClientClosedException()); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java index 1ad2eb1c646..8bcda063ad1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java @@ -22,9 +22,9 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.LedgerMetadata; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.versioning.Version; @@ -47,16 +47,13 @@ public interface LedgerManager extends Closeable { * Ledger id provided to be created * @param metadata * Metadata provided when creating the new ledger - * @param cb - * Callback when creating a new ledger, returning the written metadata. - * Return code:

      - *
    • {@link BKException.Code.OK} if success
    • - *
    • {@link BKException.Code.LedgerExistException} if given ledger id exist
    • - *
    • {@link BKException.Code.ZKException}/{@link BKException.Code.MetaStoreException} - * for other issue
    • + * @return Future which, when completed returns the metadata of the newly created ledger. + * Completed with an exception:
        + *
      • {@link BKException.BKLedgerExistException} if given ledger id exist
      • + *
      • {@link BKException.BKZKException}/{@link BKException.BKMetaStoreException} for other issues
      • *
      */ - void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, GenericCallback> cb); + CompletableFuture> createLedgerMetadata(long ledgerId, LedgerMetadata metadata); /** * Remove a specified ledger metadata by ledgerId and version. @@ -65,29 +62,27 @@ public interface LedgerManager extends Closeable { * Ledger Id * @param version * Ledger metadata version - * @param cb - * Callback when remove ledger metadata. Return code:
        - *
      • {@link BKException.Code.OK} if success
      • - *
      • {@link BKException.Code.MetadataVersionException} if version doesn't match
      • - *
      • {@link BKException.Code.NoSuchLedgerExistsException} if ledger not exist
      • - *
      • {@link BKException.Code.ZKException} for other issue
      • + * @return Future which, when completed, denotes that the ledger metadata has been removed. + * Completed with an exception:
          + *
        • {@link BKException.BKMetadataVersionException} if version doesn't match
        • + *
        • {@link BKException.BKNoSuchLedgerExistsException} if ledger not exist
        • + *
        • {@link BKException.ZKException} for other issues
        • *
        */ - void removeLedgerMetadata(long ledgerId, Version version, GenericCallback cb); + CompletableFuture removeLedgerMetadata(long ledgerId, Version version); /** * Read ledger metadata of a specified ledger. * * @param ledgerId * Ledger Id - * @param readCb - * Callback when read ledger metadata. Return code:
          - *
        • {@link BKException.Code.OK} if success
        • - *
        • {@link BKException.Code.NoSuchLedgerExistsException} if ledger not exist
        • - *
        • {@link BKException.Code.ZKException} for other issue
        • + * @return Future which, when completed, contains the requested versioned metadata. + * Completed with an exception::
            + *
          • {@link BKException.BKNoSuchLedgerExistsException} if ledger not exist
          • + *
          • {@link BKException.ZKException} for other issues
          • *
          */ - void readLedgerMetadata(long ledgerId, GenericCallback> readCb); + CompletableFuture> readLedgerMetadata(long ledgerId); /** * Write ledger metadata. @@ -98,17 +93,14 @@ public interface LedgerManager extends Closeable { * Ledger Metadata to write * @param currentVersion * The version of the metadata we expect to be overwriting. - * @param cb - * Callback when finished writing ledger metadata, returning the written metadata. - * Return code:
            - *
          • {@link BKException.Code.OK} if success
          • - *
          • {@link BKException.Code.MetadataVersionException} if version in metadata doesn't match
          • - *
          • {@link BKException.Code.ZKException} for other issue
          • + * @return Future which, when completed, contains the newly written metadata. + * Comleted with an exceptione:
              + *
            • {@link BKException.BKMetadataVersionException} if version in metadata doesn't match
            • + *
            • {@link BKException.ZKException} for other issue
            • *
            */ - void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - Version currentVersion, - GenericCallback> cb); + CompletableFuture> writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion); /** * Register the ledger metadata listener on ledgerId. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index d8fc4f2b5fc..694ae64d3f3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -30,6 +30,7 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; @@ -39,6 +40,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.metastore.MSException; @@ -55,7 +57,6 @@ import org.apache.bookkeeper.metastore.MetastoreUtils; import org.apache.bookkeeper.metastore.MetastoreWatcher; import org.apache.bookkeeper.metastore.Value; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.ReplicationException; @@ -224,7 +225,7 @@ static class MsLedgerManager implements LedgerManager, MetastoreWatcher { // callbacks ScheduledExecutorService scheduler; - protected class ReadLedgerMetadataTask implements Runnable, GenericCallback> { + protected class ReadLedgerMetadataTask implements Runnable { final long ledgerId; @@ -238,7 +239,8 @@ public void run() { if (LOG.isDebugEnabled()) { LOG.debug("Re-read ledger metadata for {}.", ledgerId); } - readLedgerMetadata(ledgerId, ReadLedgerMetadataTask.this); + readLedgerMetadata(ledgerId).whenComplete( + (metadata, exception) -> handleMetadata(metadata, exception)); } else { if (LOG.isDebugEnabled()) { LOG.debug("Ledger metadata listener for ledger {} is already removed.", ledgerId); @@ -246,26 +248,22 @@ public void run() { } } - @Override - public void operationComplete(int rc, final Versioned result) { - if (BKException.Code.OK == rc) { + private void handleMetadata(Versioned metadata, Throwable exception) { + if (exception == null) { final Set listenerSet = listeners.get(ledgerId); if (null != listenerSet) { if (LOG.isDebugEnabled()) { - LOG.debug("Ledger metadata is changed for {} : {}.", ledgerId, result.getValue()); + LOG.debug("Ledger metadata is changed for {} : {}.", ledgerId, metadata.getValue()); } - scheduler.submit(new Runnable() { - @Override - public void run() { + scheduler.submit(() -> { synchronized (listenerSet) { for (LedgerMetadataListener listener : listenerSet) { - listener.onChanged(ledgerId, result); + listener.onChanged(ledgerId, metadata); } } - } - }); + }); } - } else if (BKException.Code.NoSuchLedgerExistsException == rc) { + } else if (BKException.getExceptionCode(exception) == BKException.Code.NoSuchLedgerExistsException) { // the ledger is removed, do nothing Set listenerSet = listeners.remove(ledgerId); if (null != listenerSet) { @@ -275,7 +273,8 @@ public void run() { } } } else { - LOG.warn("Failed on read ledger metadata of ledger {} : {}", ledgerId, rc); + LOG.warn("Failed on read ledger metadata of ledger {}: {}", + ledgerId, BKException.getExceptionCode(exception)); scheduler.schedule(this, MS_CONNECT_BACKOFF_MS, TimeUnit.MILLISECONDS); } } @@ -375,112 +374,116 @@ public void close() { } @Override - public void createLedgerMetadata(final long lid, final LedgerMetadata metadata, - final GenericCallback> ledgerCb) { + public CompletableFuture> createLedgerMetadata(long lid, LedgerMetadata metadata) { + CompletableFuture> promise = new CompletableFuture<>(); MetastoreCallback msCallback = new MetastoreCallback() { @Override public void complete(int rc, Version version, Object ctx) { if (MSException.Code.BadVersion.getCode() == rc) { - ledgerCb.operationComplete(BKException.Code.MetadataVersionException, null); + promise.completeExceptionally(new BKException.BKMetadataVersionException()); return; } if (MSException.Code.OK.getCode() != rc) { - ledgerCb.operationComplete(BKException.Code.MetaStoreException, null); + promise.completeExceptionally(new BKException.MetaStoreException()); return; } if (LOG.isDebugEnabled()) { LOG.debug("Create ledger {} with version {} successfully.", lid, version); } - ledgerCb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, version)); + promise.complete(new Versioned<>(metadata, version)); } }; ledgerTable.put(ledgerId2Key(lid), new Value().setField(META_FIELD, metadata.serialize()), Version.NEW, msCallback, null); + return promise; } @Override - public void removeLedgerMetadata(final long ledgerId, final Version version, - final GenericCallback cb) { + public CompletableFuture removeLedgerMetadata(final long ledgerId, final Version version) { + CompletableFuture promise = new CompletableFuture<>(); MetastoreCallback msCallback = new MetastoreCallback() { @Override public void complete(int rc, Void value, Object ctx) { int bkRc; if (MSException.Code.NoKey.getCode() == rc) { LOG.warn("Ledger entry does not exist in meta table: ledgerId={}", ledgerId); - bkRc = BKException.Code.NoSuchLedgerExistsException; + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); } else if (MSException.Code.OK.getCode() == rc) { - bkRc = BKException.Code.OK; + FutureUtils.complete(promise, null); } else { - bkRc = BKException.Code.MetaStoreException; + promise.completeExceptionally(new BKException.MetaStoreException()); } - cb.operationComplete(bkRc, (Void) null); } }; ledgerTable.remove(ledgerId2Key(ledgerId), version, msCallback, null); + return promise; } @Override - public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { + public CompletableFuture> readLedgerMetadata(long ledgerId) { final String key = ledgerId2Key(ledgerId); + CompletableFuture> promise = new CompletableFuture<>(); MetastoreCallback> msCallback = new MetastoreCallback>() { @Override public void complete(int rc, Versioned value, Object ctx) { if (MSException.Code.NoKey.getCode() == rc) { LOG.error("No ledger metadata found for ledger " + ledgerId + " : ", MSException.create(MSException.Code.get(rc), "No key " + key + " found.")); - readCb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); return; } if (MSException.Code.OK.getCode() != rc) { LOG.error("Could not read metadata for ledger " + ledgerId + " : ", MSException.create(MSException.Code.get(rc), "Failed to get key " + key)); - readCb.operationComplete(BKException.Code.MetaStoreException, null); + promise.completeExceptionally(new BKException.MetaStoreException()); return; } try { LedgerMetadata metadata = LedgerMetadata.parseConfig( value.getValue().getField(META_FIELD), Optional.absent()); - readCb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, value.getVersion())); + promise.complete(new Versioned<>(metadata, value.getVersion())); } catch (IOException e) { LOG.error("Could not parse ledger metadata for ledger " + ledgerId + " : ", e); - readCb.operationComplete(BKException.Code.MetaStoreException, null); + promise.completeExceptionally(new BKException.MetaStoreException()); } } }; ledgerTable.get(key, this, msCallback, ALL_FIELDS); + return promise; } @Override - public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - Version currentVersion, - final GenericCallback> cb) { + public CompletableFuture> writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion) { Value data = new Value().setField(META_FIELD, metadata.serialize()); if (LOG.isDebugEnabled()) { LOG.debug("Writing ledger {} metadata, version {}", new Object[] { ledgerId, currentVersion }); } + CompletableFuture> promise = new CompletableFuture<>(); final String key = ledgerId2Key(ledgerId); MetastoreCallback msCallback = new MetastoreCallback() { @Override public void complete(int rc, Version version, Object ctx) { if (MSException.Code.BadVersion.getCode() == rc) { LOG.info("Bad version provided to updat metadata for ledger {}", ledgerId); - cb.operationComplete(BKException.Code.MetadataVersionException, null); + promise.completeExceptionally(new BKException.BKMetadataVersionException()); } else if (MSException.Code.NoKey.getCode() == rc) { LOG.warn("Ledger {} doesn't exist when writing its ledger metadata.", ledgerId); - cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); } else if (MSException.Code.OK.getCode() == rc) { - cb.operationComplete(BKException.Code.OK, new Versioned<>(metadata, version)); + promise.complete(new Versioned<>(metadata, version)); } else { LOG.warn("Conditional update ledger metadata failed: ", MSException.create(MSException.Code.get(rc), "Failed to put key " + key)); - cb.operationComplete(BKException.Code.MetaStoreException, null); + promise.completeExceptionally(new BKException.MetaStoreException()); } } }; ledgerTable.put(key, data, currentVersion, msCallback, null); + return promise; } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java index 5227176b5bc..c09a7702c63 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java @@ -27,13 +27,10 @@ import java.util.concurrent.CountDownLatch; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.ReplicationException.BKAuditException; -import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,29 +63,29 @@ public Map> getBookieToLedgerIndex() final CountDownLatch ledgerCollectorLatch = new CountDownLatch(1); Processor ledgerProcessor = new Processor() { - @Override - public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { - GenericCallback> genericCallback = (rc, ledgerMetadata) -> { - if (rc == BKException.Code.OK) { - for (Map.Entry> ensemble - : ledgerMetadata.getValue().getAllEnsembles().entrySet()) { - for (BookieSocketAddress bookie : ensemble.getValue()) { - putLedger(bookie2ledgersMap, bookie.toString(), ledgerId); - } - } - } else if (rc == BKException.Code.NoSuchLedgerExistsException) { - LOG.info("Ignoring replication of already deleted ledger {}", - ledgerId); - rc = BKException.Code.OK; - } else { - LOG.warn("Unable to read the ledger:" + ledgerId - + " information"); - } - iterCallback.processResult(rc, null, null); - }; - ledgerManager.readLedgerMetadata(ledgerId, genericCallback); - } - }; + @Override + public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { + ledgerManager.readLedgerMetadata(ledgerId).whenComplete( + (metadata, exception) -> { + if (exception == null) { + for (Map.Entry> ensemble + : metadata.getValue().getAllEnsembles().entrySet()) { + for (BookieSocketAddress bookie : ensemble.getValue()) { + putLedger(bookie2ledgersMap, bookie.toString(), ledgerId); + } + } + iterCallback.processResult(BKException.Code.OK, null, null); + } else if (BKException.getExceptionCode(exception) + == BKException.Code.NoSuchLedgerExistsException) { + LOG.info("Ignoring replication of already deleted ledger {}", ledgerId); + iterCallback.processResult(BKException.Code.OK, null, null); + } else { + LOG.warn("Unable to read the ledger: {} information", ledgerId); + iterCallback.processResult(BKException.getExceptionCode(exception), null, null); + } + }); + } + }; // Reading the result after processing all the ledgers final List resultCode = new ArrayList(1); ledgerManager.asyncProcessLedgers(ledgerProcessor, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java index 43b49943f41..1a924e5d5ee 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java @@ -65,10 +65,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { // output Map output = Maps.newHashMap(); - ListLedgerService.ReadLedgerMetadataCallback cb = - new ListLedgerService.ReadLedgerMetadataCallback(ledgerId); - manager.readLedgerMetadata(ledgerId, cb); - LedgerMetadata md = cb.get(); + LedgerMetadata md = manager.readLedgerMetadata(ledgerId).get().getValue(); output.put(ledgerId.toString(), new String(md.serialize(), UTF_8)); manager.close(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java index 928b33a2667..0d10fcdac0b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java @@ -21,13 +21,10 @@ import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkNotNull; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.util.concurrent.AbstractFuture; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; -import org.apache.bookkeeper.client.BKException; +import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.common.util.JsonUtil; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -38,7 +35,6 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,33 +61,11 @@ public ListLedgerService(ServerConfiguration conf, BookieServer bookieServer) { // Number of LedgerMetadata contains in each page static final int LIST_LEDGER_BATCH_SIZE = 100; - /** - * Callback for reading ledger metadata. - */ - public static class ReadLedgerMetadataCallback extends AbstractFuture - implements BookkeeperInternalCallbacks.GenericCallback> { - final long ledgerId; - - ReadLedgerMetadataCallback(long ledgerId) { - this.ledgerId = ledgerId; - } - - long getLedgerId() { - return ledgerId; - } - - public void operationComplete(int rc, Versioned result) { - if (rc != 0) { - setException(BKException.create(rc)); - } else { - set(result.getValue()); - } - } - } - static void keepLedgerMetadata(ReadLedgerMetadataCallback cb, LinkedHashMap output) + static void keepLedgerMetadata(long ledgerId, CompletableFuture> future, + LinkedHashMap output) throws Exception { - LedgerMetadata md = cb.get(); - output.put(Long.valueOf(cb.getLedgerId()).toString(), new String(md.serialize(), UTF_8)); + LedgerMetadata md = future.get().getValue(); + output.put(Long.valueOf(ledgerId).toString(), new String(md.serialize(), UTF_8)); } @Override @@ -117,7 +91,8 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { // output LinkedHashMap output = Maps.newLinkedHashMap(); // futures for readLedgerMetadata for each page. - List futures = Lists.newArrayListWithExpectedSize(LIST_LEDGER_BATCH_SIZE); + Map>> futures = + new LinkedHashMap<>(LIST_LEDGER_BATCH_SIZE); if (printMeta) { int ledgerIndex = 0; @@ -137,22 +112,20 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { ledgerIndex++; if (endLedgerIndex == 0 // no actual page parameter provided || (ledgerIndex >= startLedgerIndex && ledgerIndex <= endLedgerIndex)) { - ReadLedgerMetadataCallback cb = new ReadLedgerMetadataCallback(lid); - manager.readLedgerMetadata(lid, cb); - futures.add(cb); + futures.put(lid, manager.readLedgerMetadata(lid)); } } if (futures.size() >= LIST_LEDGER_BATCH_SIZE) { - while (!futures.isEmpty()) { - ReadLedgerMetadataCallback cb = futures.remove(0); - keepLedgerMetadata(cb, output); + for (Map.Entry> > e : futures.entrySet()) { + keepLedgerMetadata(e.getKey(), e.getValue(), output); } + futures.clear(); } } - while (!futures.isEmpty()) { - ReadLedgerMetadataCallback cb = futures.remove(0); - keepLedgerMetadata(cb, output); + for (Map.Entry> > e : futures.entrySet()) { + keepLedgerMetadata(e.getKey(), e.getValue(), output); } + futures.clear(); } else { while (iter.hasNext()) { LedgerManager.LedgerRange r = iter.next(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java index 6e7b23da2f6..e0c9e2f7d93 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java @@ -48,6 +48,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -61,7 +62,6 @@ import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.proto.checksum.DigestManager; @@ -918,24 +918,27 @@ public void checkpointComplete(CheckpointSource.Checkpoint checkpoint, boolean c private LedgerManager getLedgerManager(final Set ledgers) { LedgerManager manager = new LedgerManager() { @Override - public void createLedgerMetadata(long lid, LedgerMetadata metadata, - GenericCallback> cb) { + public CompletableFuture> createLedgerMetadata(long lid, + LedgerMetadata metadata) { unsupported(); + return null; } @Override - public void removeLedgerMetadata(long ledgerId, Version version, - GenericCallback vb) { + public CompletableFuture removeLedgerMetadata(long ledgerId, Version version) { unsupported(); + return null; } @Override - public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { + public CompletableFuture> readLedgerMetadata(long ledgerId) { unsupported(); + return null; } @Override - public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - Version currentVersion, - GenericCallback> cb) { + public CompletableFuture> writeLedgerMetadata(long ledgerId, + LedgerMetadata metadata, + Version currentVersion) { unsupported(); + return null; } @Override public void asyncProcessLedgers(Processor processor, @@ -960,6 +963,7 @@ void unsupported() { LOG.error("Unsupported operation called", new Exception()); throw new RuntimeException("Unsupported op"); } + @Override public LedgerRangeIterator getLedgerRanges() { final AtomicBoolean hasnext = new AtomicBoolean(true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java index 2fd686dee92..8020f1fa27f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java @@ -29,11 +29,8 @@ import java.util.Collection; import java.util.List; import java.util.SortedMap; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import org.apache.bookkeeper.bookie.GarbageCollector.GarbageCleaner; -import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.LedgerMetadata; @@ -45,10 +42,8 @@ import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.SnapshotMap; -import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.ZooDefs; import org.junit.Assert; import org.junit.Before; @@ -85,23 +80,9 @@ public void testGcOverreplicatedLedger() throws Exception { LedgerHandle lh = bkc.createLedger(2, 2, DigestType.MAC, "".getBytes()); activeLedgers.put(lh.getId(), true); - final AtomicReference newLedgerMetadata = new AtomicReference<>(null); - final CountDownLatch latch = new CountDownLatch(1); - ledgerManager.readLedgerMetadata(lh.getId(), new GenericCallback>() { + LedgerMetadata newLedgerMetadata = ledgerManager.readLedgerMetadata(lh.getId()).get().getValue(); - @Override - public void operationComplete(int rc, Versioned result) { - if (rc == BKException.Code.OK) { - newLedgerMetadata.set(result.getValue()); - } - latch.countDown(); - } - }); - latch.await(); - if (newLedgerMetadata.get() == null) { - Assert.fail("No ledger metadata found"); - } - BookieSocketAddress bookieNotInEnsemble = getBookieNotInEnsemble(newLedgerMetadata.get()); + BookieSocketAddress bookieNotInEnsemble = getBookieNotInEnsemble(newLedgerMetadata); ServerConfiguration bkConf = getBkConf(bookieNotInEnsemble); bkConf.setGcOverreplicatedLedgerWaitTime(10, TimeUnit.MILLISECONDS); @@ -132,24 +113,9 @@ public void testNoGcOfLedger() throws Exception { LedgerHandle lh = bkc.createLedger(2, 2, DigestType.MAC, "".getBytes()); activeLedgers.put(lh.getId(), true); - final AtomicReference newLedgerMetadata = new AtomicReference<>(null); - final CountDownLatch latch = new CountDownLatch(1); - ledgerManager.readLedgerMetadata(lh.getId(), new GenericCallback>() { - - @Override - public void operationComplete(int rc, Versioned result) { - if (rc == BKException.Code.OK) { - newLedgerMetadata.set(result.getValue()); - } - latch.countDown(); - } - }); - latch.await(); - if (newLedgerMetadata.get() == null) { - Assert.fail("No ledger metadata found"); - } + LedgerMetadata newLedgerMetadata = ledgerManager.readLedgerMetadata(lh.getId()).get().getValue(); BookieSocketAddress address = null; - SortedMap> ensembleMap = newLedgerMetadata.get().getAllEnsembles(); + SortedMap> ensembleMap = newLedgerMetadata.getAllEnsembles(); for (List ensemble : ensembleMap.values()) { address = ensemble.get(0); } @@ -183,23 +149,8 @@ public void testNoGcIfLedgerBeingReplicated() throws Exception { LedgerHandle lh = bkc.createLedger(2, 2, DigestType.MAC, "".getBytes()); activeLedgers.put(lh.getId(), true); - final AtomicReference newLedgerMetadata = new AtomicReference<>(null); - final CountDownLatch latch = new CountDownLatch(1); - ledgerManager.readLedgerMetadata(lh.getId(), new GenericCallback>() { - - @Override - public void operationComplete(int rc, Versioned result) { - if (rc == BKException.Code.OK) { - newLedgerMetadata.set(result.getValue()); - } - latch.countDown(); - } - }); - latch.await(); - if (newLedgerMetadata.get() == null) { - Assert.fail("No ledger metadata found"); - } - BookieSocketAddress bookieNotInEnsemble = getBookieNotInEnsemble(newLedgerMetadata.get()); + LedgerMetadata newLedgerMetadata = ledgerManager.readLedgerMetadata(lh.getId()).get().getValue(); + BookieSocketAddress bookieNotInEnsemble = getBookieNotInEnsemble(newLedgerMetadata); ServerConfiguration bkConf = getBkConf(bookieNotInEnsemble); bkConf.setGcOverreplicatedLedgerWaitTime(10, TimeUnit.MILLISECONDS); 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 d9bd8882b35..506f5122b8c 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 @@ -20,7 +20,6 @@ */ package org.apache.bookkeeper.client; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -46,10 +45,8 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.versioning.Versioned; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -559,28 +556,7 @@ public SyncLedgerMetaObject() { } private LedgerMetadata getLedgerMetadata(LedgerHandle lh) throws Exception { - final SyncLedgerMetaObject syncObj = new SyncLedgerMetaObject(); - bkc.getLedgerManager().readLedgerMetadata(lh.getId(), new GenericCallback>() { - - @Override - public void operationComplete(int rc, Versioned result) { - synchronized (syncObj) { - syncObj.rc = rc; - syncObj.meta = result.getValue(); - syncObj.value = true; - syncObj.notify(); - } - } - - }); - - synchronized (syncObj) { - while (!syncObj.value) { - syncObj.wait(); - } - } - assertEquals(BKException.Code.OK, syncObj.rc); - return syncObj.meta; + return bkc.getLedgerManager().readLedgerMetadata(lh.getId()).get().getValue(); } private boolean findDupesInEnsembles(List lhs) throws Exception { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java index e00ec5b954a..f56c350fb70 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java @@ -26,7 +26,6 @@ import java.util.function.Function; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.util.ByteBufList; @@ -67,9 +66,7 @@ public static Versioned setupLedger(ClientContext clientCtx, lon public static Versioned setupLedger(LedgerManager ledgerManager, long ledgerId, LedgerMetadataBuilder builder) throws Exception { LedgerMetadata md = builder.withPassword(PASSWD).withDigestType(DIGEST_TYPE).build(); - GenericCallbackFuture> mdPromise = new GenericCallbackFuture<>(); - ledgerManager.createLedgerMetadata(ledgerId, md, mdPromise); - return mdPromise.get(); + return ledgerManager.createLedgerMetadata(ledgerId, md).get(); } public static Versioned transformMetadata(ClientContext clientCtx, long ledgerId, @@ -81,12 +78,9 @@ public static Versioned transformMetadata(ClientContext clientCt public static Versioned transformMetadata(LedgerManager ledgerManager, long ledgerId, Function transform) throws Exception { - GenericCallbackFuture> readPromise = new GenericCallbackFuture<>(); - GenericCallbackFuture> writePromise = new GenericCallbackFuture<>(); - ledgerManager.readLedgerMetadata(ledgerId, readPromise); - ledgerManager.writeLedgerMetadata(ledgerId, transform.apply(readPromise.get().getValue()), - readPromise.get().getVersion(), writePromise); - return writePromise.get(); + Versioned current = ledgerManager.readLedgerMetadata(ledgerId).get(); + return ledgerManager.writeLedgerMetadata(ledgerId, transform.apply(current.getValue()), + current.getVersion()).get(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java index 3a547b9f071..e4b48d91624 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java @@ -74,7 +74,7 @@ public void testChangeTriggeredOneTimeForOneFailure() throws Exception { lh.appendAsync("entry4".getBytes()); lh.appendAsync("entry5".getBytes()).get(); - verify(clientCtx.getLedgerManager(), times(1)).writeLedgerMetadata(anyLong(), any(), any(), any()); + verify(clientCtx.getLedgerManager(), times(1)).writeLedgerMetadata(anyLong(), any(), any()); Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); } @@ -120,7 +120,7 @@ public void testSecondFailureOccursWhileFirstBeingHandled() throws Exception { metadataBlocker.complete(null); future.get(); - verify(clientCtx.getLedgerManager(), times(2)).writeLedgerMetadata(anyLong(), any(), any(), any()); + verify(clientCtx.getLedgerManager(), times(2)).writeLedgerMetadata(anyLong(), any(), any()); Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().size(), 1); Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b3)); Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b4)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java index 4f483a6df5a..2a1050e49b0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java @@ -53,9 +53,7 @@ private static Versioned setupLedger(ClientContext clientCtx, lo LedgerMetadata md = LedgerMetadataBuilder.create() .withPassword(PASSWD) .newEnsembleEntry(0, bookies).build(); - GenericCallbackFuture> mdPromise = new GenericCallbackFuture<>(); - clientCtx.getLedgerManager().createLedgerMetadata(1L, md, mdPromise); - return mdPromise.get(); + return clientCtx.getLedgerManager().createLedgerMetadata(1L, md).get(); } @Test diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java index 7154d94239f..23476855ed3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java @@ -45,8 +45,6 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.MockLedgerManager; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.lang3.tuple.Triple; @@ -76,10 +74,8 @@ public void testBasicUpdate() throws Exception { new BookieSocketAddress("0.0.0.2:3181"), new BookieSocketAddress("0.0.0.3:3181"), new BookieSocketAddress("0.0.0.4:3181"))).build(); - GenericCallbackFuture> promise = new GenericCallbackFuture<>(); long ledgerId = 1234L; - lm.createLedgerMetadata(ledgerId, initMeta, promise); - Versioned writtenMetadata = promise.get(); + Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); @@ -119,9 +115,8 @@ public void testConflictOnWrite() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); - GenericCallbackFuture> promise = new GenericCallbackFuture<>(); - lm.createLedgerMetadata(ledgerId, initMeta, promise); - Versioned writtenMetadata = promise.get(); + Versioned writtenMetadata = + lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference1 = new AtomicReference<>(writtenMetadata); CompletableFuture> loop1 = new MetadataUpdateLoop( @@ -165,7 +160,7 @@ public void testConflictOnWrite() throws Exception { Assert.assertEquals(l2meta.getValue().getEnsemble(0L).get(0), b2); Assert.assertEquals(l2meta.getValue().getEnsemble(0L).get(1), b3); - verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any(), any()); + verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any()); } } @@ -186,10 +181,7 @@ public void testConflictOnWriteBothWritingSame() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); - GenericCallbackFuture> promise = new GenericCallbackFuture<>(); - lm.createLedgerMetadata(ledgerId, initMeta, promise); - Versioned writtenMetadata = promise.get(); - + Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); CompletableFuture> loop1 = new MetadataUpdateLoop( @@ -223,7 +215,7 @@ public void testConflictOnWriteBothWritingSame() throws Exception { Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), b2); Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(1), b1); - verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any(), any()); + verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any()); } } @@ -242,10 +234,7 @@ public void testConflictOnLocalUpdate() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) .newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); - GenericCallbackFuture> promise = new GenericCallbackFuture<>(); - lm.createLedgerMetadata(ledgerId, initMeta, promise); - Versioned writtenMetadata = promise.get(); - + Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); CompletableFuture> loop1 = new MetadataUpdateLoop( @@ -281,7 +270,7 @@ public void testConflictOnLocalUpdate() throws Exception { Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), b2); Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(1), b3); - verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any(), any()); + verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any()); } } @@ -310,9 +299,7 @@ public void testHammer() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(ensembleSize) .newEnsembleEntry(0L, initialEnsemble).build(); - GenericCallbackFuture> promise = new GenericCallbackFuture<>(); - lm.createLedgerMetadata(ledgerId, initMeta, promise); - Versioned writtenMetadata = promise.get(); + Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); @@ -355,9 +342,7 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(1) .newEnsembleEntry(0L, Lists.newArrayList(b0)).build(); - GenericCallbackFuture> promise = new GenericCallbackFuture<>(); - lm.createLedgerMetadata(ledgerId, initMeta, promise); - Versioned writtenMetadata = promise.get(); + Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); CompletableFuture> loop1 = new MetadataUpdateLoop( @@ -397,7 +382,7 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(0), b0); Assert.assertTrue(l1meta.getValue().isClosed()); - verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any(), any()); + verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any()); } } @@ -420,7 +405,7 @@ public void close() { static class DeferCallbacksMockLedgerManager extends MockLedgerManager { int writeCount = 0; final int numToDefer; - List>, Integer, Versioned>> deferred = + List>, Versioned, Throwable>> deferred = Lists.newArrayList(); DeferCallbacksMockLedgerManager(int numToDefer) { @@ -428,7 +413,14 @@ static class DeferCallbacksMockLedgerManager extends MockLedgerManager { } synchronized void runDeferred() { - deferred.forEach((d) -> d.getLeft().operationComplete(d.getMiddle(), d.getRight())); + deferred.forEach((d) -> { + Throwable t = d.getRight(); + if (t != null) { + d.getLeft().completeExceptionally(t); + } else { + d.getLeft().complete(d.getMiddle()); + } + }); } synchronized void waitForWriteCount(int count) throws Exception { @@ -438,32 +430,38 @@ synchronized void waitForWriteCount(int count) throws Exception { } @Override - public synchronized void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - Version currentVersion, - GenericCallback> cb) { - super.writeLedgerMetadata(ledgerId, metadata, currentVersion, - (rc, written) -> { - synchronized (DeferCallbacksMockLedgerManager.this) { - if (writeCount++ < numToDefer) { - LOG.info("Added aaaaato deferals"); - deferred.add(Triple.of(cb, rc, written)); - } else { - LOG.info("Completing {}", numToDefer); - cb.operationComplete(rc, written); - } - DeferCallbacksMockLedgerManager.this.notifyAll(); - } - }); + public synchronized CompletableFuture> writeLedgerMetadata( + long ledgerId, LedgerMetadata metadata, + Version currentVersion) { + CompletableFuture> promise = new CompletableFuture<>(); + super.writeLedgerMetadata(ledgerId, metadata, currentVersion) + .whenComplete((written, exception) -> { + synchronized (DeferCallbacksMockLedgerManager.this) { + if (writeCount++ < numToDefer) { + LOG.info("Added to deferals"); + deferred.add(Triple.of(promise, written, exception)); + } else { + LOG.info("Completing {}", numToDefer); + if (exception != null) { + promise.completeExceptionally(exception); + } else { + promise.complete(written); + } + } + DeferCallbacksMockLedgerManager.this.notifyAll(); + } + }); + return promise; }; } @Data @AllArgsConstructor static class DeferredUpdate { + final CompletableFuture> promise; final long ledgerId; final LedgerMetadata metadata; final Version currentVersion; - final GenericCallback> cb; } static class BlockableMockLedgerManager extends MockLedgerManager { @@ -476,18 +474,28 @@ synchronized void blockWrites() { synchronized void releaseWrites() { blocking = false; - reqs.forEach((r) -> super.writeLedgerMetadata(r.getLedgerId(), r.getMetadata(), - r.getCurrentVersion(), r.getCb())); + reqs.forEach((r) -> { + super.writeLedgerMetadata(r.getLedgerId(), r.getMetadata(), + r.getCurrentVersion()) + .whenComplete((written, exception) -> { + if (exception != null) { + r.getPromise().completeExceptionally(exception); + } else { + r.getPromise().complete(written); + } + }); + }); } @Override - public synchronized void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - Version currentVersion, - GenericCallback> cb) { + public synchronized CompletableFuture> writeLedgerMetadata( + long ledgerId, LedgerMetadata metadata, Version currentVersion) { if (blocking) { - reqs.add(new DeferredUpdate(ledgerId, metadata, currentVersion, cb)); + CompletableFuture> promise = new CompletableFuture<>(); + reqs.add(new DeferredUpdate(promise, ledgerId, metadata, currentVersion)); + return promise; } else { - super.writeLedgerMetadata(ledgerId, metadata, currentVersion, cb); + return super.writeLedgerMetadata(ledgerId, metadata, currentVersion); } }; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java index 7d71837a5c6..e80807869aa 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java @@ -41,6 +41,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListSet; @@ -365,18 +366,17 @@ private void setupReadLedgerMetadata() { doAnswer(invocation -> { Object[] args = invocation.getArguments(); Long ledgerId = (Long) args[0]; + CompletableFuture> promise = new CompletableFuture<>(); executor.executeOrdered(ledgerId, () -> { - BookkeeperInternalCallbacks.GenericCallback cb = (BookkeeperInternalCallbacks.GenericCallback) args[1]; LedgerMetadata ledgerMetadata = mockLedgerMetadataRegistry.get(ledgerId); if (ledgerMetadata == null) { - cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); } else { - cb.operationComplete(BKException.Code.OK, - new Versioned<>(new LedgerMetadata(ledgerMetadata), new LongVersion(1))); + promise.complete(new Versioned<>(new LedgerMetadata(ledgerMetadata), new LongVersion(1))); } }); - return null; - }).when(ledgerManager).readLedgerMetadata(anyLong(), any()); + return promise; + }).when(ledgerManager).readLedgerMetadata(anyLong()); } @SuppressWarnings("unchecked") @@ -384,16 +384,16 @@ private void setupRemoveLedgerMetadata() { doAnswer(invocation -> { Object[] args = invocation.getArguments(); Long ledgerId = (Long) args[0]; + CompletableFuture promise = new CompletableFuture<>(); executor.executeOrdered(ledgerId, () -> { - BookkeeperInternalCallbacks.GenericCallback cb = (BookkeeperInternalCallbacks.GenericCallback) args[2]; - if (mockLedgerMetadataRegistry.remove(ledgerId) != null) { - cb.operationComplete(BKException.Code.OK, null); - } else { - cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); - } - }); - return null; - }).when(ledgerManager).removeLedgerMetadata(anyLong(), any(), any()); + if (mockLedgerMetadataRegistry.remove(ledgerId) != null) { + promise.complete(null); + } else { + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + } + }); + return promise; + }).when(ledgerManager).removeLedgerMetadata(anyLong(), any()); } private void setupRegisterLedgerMetadataListener() { @@ -420,15 +420,17 @@ private void setupLedgerIdGenerator() { private void setupCreateLedgerMetadata() { doAnswer(invocation -> { Object[] args = invocation.getArguments(); - BookkeeperInternalCallbacks.GenericCallback cb = (BookkeeperInternalCallbacks.GenericCallback) args[2]; Long ledgerId = (Long) args[0]; + + CompletableFuture> promise = new CompletableFuture<>(); executor.executeOrdered(ledgerId, () -> { - LedgerMetadata ledgerMetadata = (LedgerMetadata) args[1]; - mockLedgerMetadataRegistry.put(ledgerId, new LedgerMetadata(ledgerMetadata)); - cb.operationComplete(BKException.Code.OK, new Versioned<>(ledgerMetadata, new LongVersion(1))); + + LedgerMetadata ledgerMetadata = (LedgerMetadata) args[1]; + mockLedgerMetadataRegistry.put(ledgerId, new LedgerMetadata(ledgerMetadata)); + promise.complete(new Versioned<>(ledgerMetadata, new LongVersion(1))); }); - return null; - }).when(ledgerManager).createLedgerMetadata(anyLong(), any(), any()); + return promise; + }).when(ledgerManager).createLedgerMetadata(anyLong(), any()); } @SuppressWarnings("unchecked") @@ -438,14 +440,14 @@ private void setupWriteLedgerMetadata() { Long ledgerId = (Long) args[0]; LedgerMetadata metadata = (LedgerMetadata) args[1]; Version currentVersion = (Version) args[2]; - BookkeeperInternalCallbacks.GenericCallback cb = (BookkeeperInternalCallbacks.GenericCallback) args[3]; + CompletableFuture> promise = new CompletableFuture<>(); executor.executeOrdered(ledgerId, () -> { LedgerMetadata newMetadata = LedgerMetadataBuilder.from(metadata).build(); mockLedgerMetadataRegistry.put(ledgerId, newMetadata); - cb.operationComplete(BKException.Code.OK, new Versioned<>(newMetadata, new LongVersion(1234))); + promise.complete(new Versioned<>(newMetadata, new LongVersion(1234))); }); - return null; - }).when(ledgerManager).writeLedgerMetadata(anyLong(), any(), any(), any()); + return promise; + }).when(ledgerManager).writeLedgerMetadata(anyLong(), any(), any()); } @SuppressWarnings("unchecked") diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index 5ba73c6733e..9682eb62db7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.util.Enumeration; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -94,19 +95,19 @@ void setLatch(CountDownLatch waitLatch) { } @Override - public void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, - GenericCallback> cb) { - lm.createLedgerMetadata(ledgerId, metadata, cb); + public CompletableFuture> createLedgerMetadata( + long ledgerId, LedgerMetadata metadata) { + return lm.createLedgerMetadata(ledgerId, metadata); } @Override - public void removeLedgerMetadata(long ledgerId, Version version, GenericCallback cb) { - lm.removeLedgerMetadata(ledgerId, version, cb); + public CompletableFuture removeLedgerMetadata(long ledgerId, Version version) { + return lm.removeLedgerMetadata(ledgerId, version); } @Override - public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { - lm.readLedgerMetadata(ledgerId, readCb); + public CompletableFuture> readLedgerMetadata(long ledgerId) { + return lm.readLedgerMetadata(ledgerId); } @Override @@ -115,11 +116,11 @@ public LedgerRangeIterator getLedgerRanges() { } @Override - public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - Version currentVersion, - GenericCallback> cb) { + public CompletableFuture> writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion) { final CountDownLatch cdl = waitLatch; if (null != cdl) { + CompletableFuture> promise = new CompletableFuture<>(); executorService.submit(new Runnable() { @Override public void run() { @@ -129,11 +130,19 @@ public void run() { Thread.currentThread().interrupt(); LOG.error("Interrupted on waiting latch : ", e); } - lm.writeLedgerMetadata(ledgerId, metadata, currentVersion, cb); + lm.writeLedgerMetadata(ledgerId, metadata, currentVersion) + .whenComplete((metadata, exception) -> { + if (exception != null) { + promise.completeExceptionally(exception); + } else { + promise.complete(metadata); + } + }); } }); + return promise; } else { - lm.writeLedgerMetadata(ledgerId, metadata, currentVersion, cb); + return lm.writeLedgerMetadata(ledgerId, metadata, currentVersion); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java index 43acabde5a5..38411ac9b3a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java @@ -39,7 +39,6 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LongHierarchicalLedgerManagerFactory; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -132,21 +131,17 @@ private void testWatchMetadataRemoval(LedgerManagerFactory factory) throws Excep final CountDownLatch removeLatch = new CountDownLatch(1); idGenerator.generateLedgerId(new GenericCallback() { - @Override - public void operationComplete(int rc, final Long lid) { - manager.createLedgerMetadata(lid, new LedgerMetadata(4, 2, 2, digestType, "fpj was here".getBytes()), - new BookkeeperInternalCallbacks.GenericCallback>(){ - - @Override - public void operationComplete(int rc, Versioned result) { - bbLedgerId.putLong(lid); - bbLedgerId.flip(); - createLatch.countDown(); - } - }); - - } - }); + @Override + public void operationComplete(int rc, final Long lid) { + LedgerMetadata metadata = new LedgerMetadata(4, 2, 2, digestType, "fpj was here".getBytes()); + manager.createLedgerMetadata(lid, metadata) + .whenComplete((result, exception) -> { + bbLedgerId.putLong(lid); + bbLedgerId.flip(); + createLatch.countDown(); + }); + } + }); assertTrue(createLatch.await(2000, TimeUnit.MILLISECONDS)); final long createdLid = bbLedgerId.getLong(); @@ -162,14 +157,7 @@ public void onChanged(long ledgerId, Versioned metadata) { } }); - manager.removeLedgerMetadata(createdLid, Version.ANY, - new BookkeeperInternalCallbacks.GenericCallback() { - - @Override - public void operationComplete(int rc, Void result) { - assertEquals(rc, BKException.Code.OK); - } - }); - assertTrue(removeLatch.await(2000, TimeUnit.MILLISECONDS)); + manager.removeLedgerMetadata(createdLid, Version.ANY).get(2, TimeUnit.SECONDS); + assertTrue(removeLatch.await(2, TimeUnit.SECONDS)); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 290af7bdad0..71b229dd23a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -54,7 +54,6 @@ import org.apache.bookkeeper.common.testing.executors.MockExecutorController; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.LongVersion; @@ -155,10 +154,9 @@ public void testCreateLedgerMetadataSuccess() throws Exception { KeeperException.Code.OK.intValue(), ledgerStr ); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.createLedgerMetadata(ledgerId, metadata, callbackFuture); + Versioned result = ledgerManager.createLedgerMetadata(ledgerId, metadata).get(); - assertEquals(new LongVersion(0), callbackFuture.get().getVersion()); + assertEquals(new LongVersion(0), result.getVersion()); } @Test @@ -169,10 +167,8 @@ public void testCreateLedgerMetadataNodeExists() throws Exception { ledgerStr, CreateMode.PERSISTENT, KeeperException.Code.NODEEXISTS.intValue(), null); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.createLedgerMetadata(ledgerId, metadata, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.createLedgerMetadata(ledgerId, metadata)); fail("Should fail to create ledger metadata if the ledger already exists"); } catch (Exception e) { assertTrue(e instanceof BKException); @@ -189,10 +185,8 @@ public void testCreateLedgerMetadataException() throws Exception { ledgerStr, CreateMode.PERSISTENT, KeeperException.Code.CONNECTIONLOSS.intValue(), null); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.createLedgerMetadata(ledgerId, metadata, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.createLedgerMetadata(ledgerId, metadata)); fail("Should fail to create ledger metadata when encountering zookeeper exception"); } catch (Exception e) { assertTrue(e instanceof BKException); @@ -211,9 +205,7 @@ public void testRemoveLedgerMetadataSuccess() throws Exception { ledgerStr, (int) version.getLongVersion(), KeeperException.Code.OK.intValue()); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.removeLedgerMetadata(ledgerId, version, callbackFuture); - result(callbackFuture); + ledgerManager.removeLedgerMetadata(ledgerId, version).get(); verify(mockZk, times(1)) .delete(eq(ledgerStr), eq(1234), any(VoidCallback.class), eq(null)); @@ -228,9 +220,7 @@ public void testRemoveLedgerMetadataVersionAny() throws Exception { ledgerStr, -1, KeeperException.Code.OK.intValue()); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.removeLedgerMetadata(ledgerId, Version.ANY, callbackFuture); - result(callbackFuture); + ledgerManager.removeLedgerMetadata(ledgerId, Version.ANY).get(); verify(mockZk, times(1)) .delete(eq(ledgerStr), eq(-1), any(VoidCallback.class), eq(null)); @@ -250,10 +240,8 @@ public void testRemoveLedgerMetadataUnknownVersionType() throws Exception { private void testRemoveLedgerMetadataInvalidVersion(Version version) throws Exception { long ledgerId = System.currentTimeMillis(); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.removeLedgerMetadata(ledgerId, version, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.removeLedgerMetadata(ledgerId, version)); fail("Should fail to remove metadata if version is " + Version.NEW); } catch (BKException bke) { assertEquals(Code.MetadataVersionException, bke.getCode()); @@ -270,10 +258,8 @@ public void testRemoveLedgerMetadataNoNode() throws Exception { ledgerStr, (int) version.getLongVersion(), KeeperException.Code.NONODE.intValue()); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.removeLedgerMetadata(ledgerId, version, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.removeLedgerMetadata(ledgerId, version)); fail("Should fail to remove metadata if no such ledger exists"); } catch (BKException bke) { assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); @@ -293,10 +279,8 @@ public void testRemoveLedgerMetadataException() throws Exception { ledgerStr, (int) version.getLongVersion(), KeeperException.Code.CONNECTIONLOSS.intValue()); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.removeLedgerMetadata(ledgerId, version, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.removeLedgerMetadata(ledgerId, version)); fail("Should fail to remove metadata if no such ledger exists"); } catch (BKException bke) { assertEquals(Code.ZKException, bke.getCode()); @@ -327,9 +311,7 @@ private void testRemoveLedgerMetadataHierarchicalLedgerManager(AbstractZkLedgerM ledgerStr, (int) version.getLongVersion(), KeeperException.Code.OK.intValue()); - GenericCallbackFuture callbackFuture = new GenericCallbackFuture<>(); - lm.removeLedgerMetadata(ledgerId, version, callbackFuture); - result(callbackFuture); + lm.removeLedgerMetadata(ledgerId, version).get(); PowerMockito.verifyStatic( ZkUtils.class, times(1)); @@ -350,9 +332,7 @@ public void testReadLedgerMetadataSuccess() throws Exception { ledgerStr, false, KeeperException.Code.OK.intValue(), metadata.serialize(), stat); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); - Versioned readMetadata = result(callbackFuture); + Versioned readMetadata = result(ledgerManager.readLedgerMetadata(ledgerId)); assertEquals(metadata, readMetadata.getValue()); assertEquals(new LongVersion(1234), readMetadata.getVersion()); @@ -369,10 +349,8 @@ public void testReadLedgerMetadataNoNode() throws Exception { ledgerStr, false, KeeperException.Code.NONODE.intValue(), null, null); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.readLedgerMetadata(ledgerId)); fail("Should fail on reading ledger metadata if a ledger doesn't exist"); } catch (BKException bke) { assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); @@ -391,10 +369,8 @@ public void testReadLedgerMetadataException() throws Exception { ledgerStr, false, KeeperException.Code.CONNECTIONLOSS.intValue(), null, null); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.readLedgerMetadata(ledgerId)); fail("Should fail on reading ledger metadata if a ledger doesn't exist"); } catch (BKException bke) { assertEquals(Code.ZKException, bke.getCode()); @@ -413,10 +389,8 @@ public void testReadLedgerMetadataStatMissing() throws Exception { ledgerStr, false, KeeperException.Code.OK.intValue(), metadata.serialize(), null); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.readLedgerMetadata(ledgerId)); fail("Should fail on reading ledger metadata if a ledger doesn't exist"); } catch (BKException bke) { assertEquals(Code.ZKException, bke.getCode()); @@ -438,10 +412,8 @@ public void testReadLedgerMetadataDataCorrupted() throws Exception { ledgerStr, false, KeeperException.Code.OK.intValue(), new byte[0], stat); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.readLedgerMetadata(ledgerId, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.readLedgerMetadata(ledgerId)); fail("Should fail on reading ledger metadata if a ledger doesn't exist"); } catch (BKException bke) { assertEquals(Code.ZKException, bke.getCode()); @@ -463,10 +435,9 @@ public void testWriteLedgerMetadataSuccess() throws Exception { ledgerStr, metadata.serialize(), 1234, KeeperException.Code.OK.intValue(), stat); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L), callbackFuture); + Version v = ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L)).get().getVersion(); - assertEquals(new LongVersion(1235L), result(callbackFuture).getVersion()); + assertEquals(new LongVersion(1235L), v); verify(mockZk, times(1)) .setData(eq(ledgerStr), any(byte[].class), eq(1234), any(StatCallback.class), any()); @@ -481,10 +452,8 @@ public void testWriteLedgerMetadataBadVersion() throws Exception { ledgerStr, metadata.serialize(), 1234, KeeperException.Code.BADVERSION.intValue(), null); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L), callbackFuture); try { - result(callbackFuture); + result(ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L))); fail("Should fail on writing ledger metadata if encountering bad version"); } catch (BKException bke) { assertEquals(Code.MetadataVersionException, bke.getCode()); @@ -503,10 +472,8 @@ public void testWriteLedgerMetadataException() throws Exception { ledgerStr, metadata.serialize(), 1234, KeeperException.Code.CONNECTIONLOSS.intValue(), null); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L), callbackFuture); try { - result(callbackFuture); + result(ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L))); fail("Should fail on writing ledger metadata if encountering zookeeper exceptions"); } catch (BKException bke) { assertEquals(Code.ZKException, bke.getCode()); @@ -532,10 +499,8 @@ public void testWriteLedgerMetadataInvalidVersion() throws Exception { private void testWriteLedgerMetadataInvalidVersion(Version invalidVersion) throws Exception { long ledgerId = System.currentTimeMillis(); - GenericCallbackFuture> callbackFuture = new GenericCallbackFuture<>(); - ledgerManager.writeLedgerMetadata(ledgerId, metadata, invalidVersion, callbackFuture); try { - result(callbackFuture); + result(ledgerManager.writeLedgerMetadata(ledgerId, metadata, invalidVersion)); fail("Should fail on writing ledger metadata if an invalid version is provided."); } catch (BKException bke) { assertEquals(Code.MetadataVersionException, bke.getCode()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 10192cbeda0..baf3d007ac8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -44,6 +44,7 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -107,19 +108,16 @@ public void operationComplete(int rc, final Long ledgerId) { } getLedgerManager().createLedgerMetadata(ledgerId, - new LedgerMetadata(1, 1, 1, DigestType.MAC, "".getBytes()), - new GenericCallback>() { - @Override - public void operationComplete(int rc, Versioned writtenMetadata) { - if (rc == BKException.Code.OK) { - activeLedgers.put(ledgerId, true); - createdLedgers.add(ledgerId); - } - synchronized (expected) { - int num = expected.decrementAndGet(); - if (num == 0) { - expected.notify(); - } + new LedgerMetadata(1, 1, 1, DigestType.MAC, "".getBytes())) + .whenComplete((result, exception) -> { + if (exception == null) { + activeLedgers.put(ledgerId, true); + createdLedgers.add(ledgerId); + } + synchronized (expected) { + int num = expected.decrementAndGet(); + if (num == 0) { + expected.notify(); } } }); @@ -138,18 +136,7 @@ public void operationComplete(int rc, Versioned writtenMetadata) } private void removeLedger(long ledgerId) throws Exception { - final AtomicInteger rc = new AtomicInteger(0); - final CountDownLatch latch = new CountDownLatch(1); - getLedgerManager().removeLedgerMetadata(ledgerId, Version.ANY, - new GenericCallback() { - @Override - public void operationComplete(int rc2, Void result) { - rc.set(rc2); - latch.countDown(); - } - }); - assertTrue(latch.await(10, TimeUnit.SECONDS)); - assertEquals("Remove should have succeeded for ledgerId: " + ledgerId, 0, rc.get()); + getLedgerManager().removeLedgerMetadata(ledgerId, Version.ANY).get(10, TimeUnit.SECONDS); } @Test @@ -170,18 +157,7 @@ public void testGarbageCollectLedgers() throws Exception { // random remove several ledgers for (int i = 0; i < numRemovedLedgers; i++) { long ledgerId = tmpList.get(i); - synchronized (removedLedgers) { - getLedgerManager().removeLedgerMetadata(ledgerId, Version.ANY, - new GenericCallback() { - @Override - public void operationComplete(int rc, Void result) { - synchronized (removedLedgers) { - removedLedgers.notify(); - } - } - }); - removedLedgers.wait(); - } + getLedgerManager().removeLedgerMetadata(ledgerId, Version.ANY).get(); removedLedgers.add(ledgerId); createdLedgers.remove(ledgerId); } @@ -504,10 +480,12 @@ public void testGcLedgersIfReadLedgerMetadataSaysNoSuchLedger() throws Exception createLedgers(numLedgers, createdLedgers); + CompletableFuture> errorFuture = new CompletableFuture<>(); + errorFuture.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); LedgerManager mockLedgerManager = new CleanupLedgerManager(getLedgerManager()) { @Override - public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { - readCb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null); + public CompletableFuture> readLedgerMetadata(long ledgerId) { + return errorFuture; } }; @@ -546,10 +524,12 @@ public void testGcLedgersIfReadLedgerMetadataFailsForDeletedLedgers() throws Exc createLedgers(numLedgers, createdLedgers); + CompletableFuture> errorFuture = new CompletableFuture<>(); + errorFuture.completeExceptionally(new BKException.ZKException()); LedgerManager mockLedgerManager = new CleanupLedgerManager(getLedgerManager()) { @Override - public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { - readCb.operationComplete(BKException.Code.ZKException, null); + public CompletableFuture> readLedgerMetadata(long ledgerId) { + return errorFuture; } }; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java index 477afe49374..90fb2ee3a71 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java @@ -25,22 +25,21 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Optional; -import java.util.Queue; import java.util.Random; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentSkipListSet; 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.AtomicInteger; @@ -49,14 +48,11 @@ import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.Version; -import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs; -import org.junit.After; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -69,100 +65,15 @@ public LedgerManagerIteratorTest(Class lmFactory super(lmFactoryCls); } - final Queue exceptions = new ConcurrentLinkedQueue<>(); - - Runnable safeWrapper(Runnable r) { - return () -> { - try { - r.run(); - } catch (Throwable e) { - exceptions.add(e); - } - }; - } - - @After - public void throwAsyncErrors() throws Throwable { - while (exceptions.peek() != null) { - throw exceptions.remove(); - } - } - - class RCCheckCB implements GenericCallback> { - private final String opType; - private final CountDownLatch latch; - private final Optional rcExpected; - private final long ledgerId; - - public RCCheckCB(String opType, CountDownLatch latch, Optional rcExpected, long ledgerId) { - this.opType = opType; - this.latch = latch; - this.rcExpected = rcExpected; - this.ledgerId = ledgerId; - } - - @Override - public void operationComplete(int rc, Versioned writtenMetadata) { - safeWrapper(() -> { - try { - rcExpected.map((Integer expected) -> { - assertEquals( - "Incorrect rc on ledger: " + ledgerId + ", op type: " + opType, - expected.longValue(), rc); - return null; - }); - } finally { - latch.countDown(); - } - }).run(); - } - } - - class VoidRCCheckCB implements GenericCallback { - private final String opType; - private final CountDownLatch latch; - private final Optional rcExpected; - private final long ledgerId; - - public VoidRCCheckCB(String opType, CountDownLatch latch, Optional rcExpected, long ledgerId) { - this.opType = opType; - this.latch = latch; - this.rcExpected = rcExpected; - this.ledgerId = ledgerId; - } - - @Override - public void operationComplete(int rc, Void result) { - safeWrapper(() -> { - try { - rcExpected.map((Integer expected) -> { - assertEquals( - "Incorrect rc on ledger: " + ledgerId + ", op type: " + opType, - expected.longValue(), rc); - return null; - }); - } finally { - latch.countDown(); - } - }).run(); - } - } - /** * Remove ledger using lm syncronously. * * @param lm * @param ledgerId - * @param rcExpected return value expected, -1 to ignore * @throws InterruptedException */ - void removeLedger(LedgerManager lm, Long ledgerId, Optional rcExpected) throws Throwable { - CountDownLatch latch = new CountDownLatch(1); - lm.removeLedgerMetadata( - ledgerId, Version.ANY, new VoidRCCheckCB("removeLedger", latch, rcExpected, ledgerId)); - latch.await(); - throwAsyncErrors(); - + void removeLedger(LedgerManager lm, Long ledgerId) throws Exception { + lm.removeLedgerMetadata(ledgerId, Version.ANY).get(); } /** @@ -170,18 +81,13 @@ void removeLedger(LedgerManager lm, Long ledgerId, Optional rcExpected) * * @param lm * @param ledgerId - * @param rcExpected return value expected, -1 to ignore * @throws InterruptedException */ - void createLedger(LedgerManager lm, Long ledgerId, Optional rcExpected) throws Throwable { + void createLedger(LedgerManager lm, Long ledgerId) throws Exception { LedgerMetadata meta = new LedgerMetadata( 3, 3, 2, BookKeeper.DigestType.CRC32, "passwd".getBytes()); - CountDownLatch latch = new CountDownLatch(1); - lm.createLedgerMetadata( - ledgerId, meta, new RCCheckCB("createLedger", latch, rcExpected, ledgerId)); - latch.await(); - throwAsyncErrors(); + lm.createLedgerMetadata(ledgerId, meta).get(); } static Set ledgerRangeToSet(LedgerRangeIterator lri) throws IOException { @@ -232,7 +138,7 @@ public void testSingleLedger() throws Throwable { LedgerManager lm = getLedgerManager(); long id = 2020202; - createLedger(lm, id, Optional.of(BKException.Code.OK)); + createLedger(lm, id); LedgerRangeIterator lri = lm.getLedgerRanges(); assertNotNull(lri); @@ -250,7 +156,7 @@ public void testTwoLedgers() throws Throwable { Set ids = new TreeSet<>(Arrays.asList(101010101L, 2020340302L)); for (Long id: ids) { - createLedger(lm, id, Optional.of(BKException.Code.OK)); + createLedger(lm, id); } LedgerRangeIterator lri = lm.getLedgerRanges(); @@ -268,7 +174,7 @@ public void testSeveralContiguousLedgers() throws Throwable { Set ids = new TreeSet<>(); for (long i = 0; i < 2000; ++i) { - createLedger(lm, i, Optional.of(BKException.Code.OK)); + createLedger(lm, i); ids.add(i); } @@ -312,7 +218,7 @@ public void testRemovalOfNodeJustTraversed() throws Throwable { ids.addAll(toRemove); ids.addAll(mustHave); for (Long id: ids) { - createLedger(lm, id, Optional.of(BKException.Code.OK)); + createLedger(lm, id); } Set found = new TreeSet<>(); @@ -323,7 +229,7 @@ public void testRemovalOfNodeJustTraversed() throws Throwable { if (lr.getLedgers().contains(first)) { for (long id: toRemove) { - removeLedger(lm, id, Optional.of(BKException.Code.OK)); + removeLedger(lm, id); } toRemove.clear(); } @@ -349,7 +255,7 @@ public void validateEmptyL4PathSkipped() throws Throwable { 6334994393848474732L, 7349370101927398483L)); for (Long id: ids) { - createLedger(lm, id, Optional.of(BKException.Code.OK)); + createLedger(lm, id); } String paths[] = { @@ -396,7 +302,7 @@ public void testWithSeveralIncompletePaths() throws Throwable { 6334994393848474732L, 7349370101927398483L)); for (Long id: ids) { - createLedger(lm, id, Optional.of(BKException.Code.OK)); + createLedger(lm, id); } String paths[] = { @@ -439,56 +345,45 @@ public void checkConcurrentModifications() throws Throwable { if (!longRange) { lid %= 1000000; } - createLedger(lm, lid, Optional.of(BKException.Code.OK)); + createLedger(lm, lid); mustExist.add(lid); } final long start = MathUtils.nowInNano(); final CountDownLatch latch = new CountDownLatch(1); - ArrayList threads = new ArrayList<>(); + ArrayList> futures = new ArrayList<>(); + ExecutorService executor = Executors.newCachedThreadPool(); final ConcurrentSkipListSet createdLedgers = new ConcurrentSkipListSet<>(); for (int i = 0; i < numWriters; ++i) { - Thread thread = new Thread(safeWrapper(() -> { - LedgerManager writerLM = getIndependentLedgerManager(); - Random writerRNG = new Random(rng.nextLong()); - try { + Future f = executor.submit(() -> { + LedgerManager writerLM = getIndependentLedgerManager(); + Random writerRNG = new Random(rng.nextLong()); + latch.await(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - fail("Checker interrupted"); - } - while (MathUtils.elapsedNanos(start) < runtime) { - long candidate = 0; - do { - candidate = Math.abs(writerRNG.nextLong()); - if (!longRange) { - candidate %= 1000000; - } - } while (mustExist.contains(candidate) || !createdLedgers.add(candidate)); - try { - createLedger(writerLM, candidate, Optional.empty()); - removeLedger(writerLM, candidate, Optional.empty()); - } catch (Throwable e) { - fail("Got exception thrashing store: " + e.toString()); + + while (MathUtils.elapsedNanos(start) < runtime) { + long candidate = 0; + do { + candidate = Math.abs(writerRNG.nextLong()); + if (!longRange) { + candidate %= 1000000; + } + } while (mustExist.contains(candidate) || !createdLedgers.add(candidate)); + + createLedger(writerLM, candidate); + removeLedger(writerLM, candidate); } - } - })); - thread.start(); - threads.add(thread); + return null; + }); + futures.add(f); } for (int i = 0; i < numCheckers; ++i) { - Thread thread = new Thread(safeWrapper(() -> { - LedgerManager checkerLM = getIndependentLedgerManager(); - try { + Future f = executor.submit(() -> { + LedgerManager checkerLM = getIndependentLedgerManager(); latch.await(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - fail("Checker interrupted"); - e.printStackTrace(); - } - while (MathUtils.elapsedNanos(start) < runtime) { - try { + + while (MathUtils.elapsedNanos(start) < runtime) { LedgerRangeIterator lri = checkerLM.getLedgerRanges(); Set returnedIds = ledgerRangeToSet(lri); for (long id: mustExist) { @@ -499,20 +394,17 @@ public void checkConcurrentModifications() throws Throwable { for (long id: mustExist) { assertTrue(ledgersReadAsync.contains(id)); } - } catch (IOException | InterruptedException e) { - e.printStackTrace(); - fail("Got exception scanning ledgers: " + e.toString()); } - } - })); - thread.start(); - threads.add(thread); + return null; + }); + futures.add(f); } latch.countDown(); - for (Thread thread: threads) { - thread.join(); + for (Future f : futures) { + f.get(); } + executor.shutdownNow(); } @SuppressWarnings("deprecation") @@ -560,7 +452,7 @@ public void testLedgerManagerFormat() throws Throwable { ids.add(1234567891234L); } for (Long id : ids) { - createLedger(lm, id, Optional.of(BKException.Code.OK)); + createLedger(lm, id); } // create some invalid nodes under zkLedgersRootPath @@ -607,7 +499,7 @@ public void hierarchicalLedgerManagerAsyncProcessLedgersTest() throws Throwable Set ledgerIds = new TreeSet<>(Arrays.asList(1234L, 123456789123456789L)); for (Long ledgerId : ledgerIds) { - createLedger(lm, ledgerId, Optional.of(BKException.Code.OK)); + createLedger(lm, ledgerId); } Set ledgersReadThroughIterator = ledgerRangeToSet(lri); assertEquals("Comparing LedgersIds read through Iterator", ledgerIds, ledgersReadThroughIterator); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index 075b8f1b9a8..111040722ac 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -25,13 +25,13 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; @@ -96,48 +96,54 @@ public void executeCallback(Runnable r) { } @Override - public void createLedgerMetadata(long ledgerId, LedgerMetadata metadata, - GenericCallback> cb) { + public CompletableFuture> createLedgerMetadata(long ledgerId, LedgerMetadata metadata) { + CompletableFuture> promise = new CompletableFuture<>(); executor.submit(() -> { if (metadataMap.containsKey(ledgerId)) { - executeCallback(() -> cb.operationComplete(BKException.Code.LedgerExistException, null)); + executeCallback(() -> promise.completeExceptionally(new BKException.BKLedgerExistException())); } else { metadataMap.put(ledgerId, Pair.of(new LongVersion(0L), metadata.serialize())); try { Versioned readBack = readMetadata(ledgerId); - executeCallback(() -> cb.operationComplete(BKException.Code.OK, readBack)); + executeCallback(() -> promise.complete(readBack)); } catch (Exception e) { LOG.error("Error reading back written metadata", e); - executeCallback(() -> cb.operationComplete(BKException.Code.MetaStoreException, null)); + executeCallback(() -> promise.completeExceptionally(new BKException.MetaStoreException())); } } }); + return promise; } @Override - public void removeLedgerMetadata(long ledgerId, Version version, GenericCallback cb) {} + public CompletableFuture removeLedgerMetadata(long ledgerId, Version version) { + return CompletableFuture.completedFuture(null); + } @Override - public void readLedgerMetadata(long ledgerId, GenericCallback> cb) { + public CompletableFuture> readLedgerMetadata(long ledgerId) { + CompletableFuture> promise = new CompletableFuture<>(); executor.submit(() -> { try { Versioned metadata = readMetadata(ledgerId); if (metadata == null) { - executeCallback( - () -> cb.operationComplete(BKException.Code.NoSuchLedgerExistsException, null)); + executeCallback(() -> promise.completeExceptionally( + new BKException.BKNoSuchLedgerExistsException())); } else { - executeCallback(() -> cb.operationComplete(BKException.Code.OK, metadata)); + executeCallback(() -> promise.complete(metadata)); } } catch (Exception e) { LOG.error("Error reading metadata", e); - executeCallback(() -> cb.operationComplete(BKException.Code.MetaStoreException, null)); + executeCallback(() -> promise.completeExceptionally(new BKException.MetaStoreException())); } }); + return promise; } @Override - public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - Version currentVersion, GenericCallback> cb) { + public CompletableFuture> writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion) { + CompletableFuture> promise = new CompletableFuture<>(); preWriteHook.runHook(ledgerId, metadata) .thenComposeAsync((ignore) -> { try { @@ -160,13 +166,13 @@ public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, }, executor) .whenComplete((res, ex) -> { if (ex != null) { - executeCallback(() -> cb.operationComplete( - BKException.getExceptionCode(ex, BKException.Code.MetaStoreException), - null)); + Throwable cause = (ex instanceof CompletionException) ? ex.getCause() : ex; + executeCallback(() -> promise.completeExceptionally(cause)); } else { - executeCallback(() -> cb.operationComplete(BKException.Code.OK, res)); + executeCallback(() -> promise.complete(res)); } }); + return promise; } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index 342c443add6..83d1dc0da45 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -66,7 +66,6 @@ import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.commons.lang.mutable.MutableInt; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; @@ -553,22 +552,11 @@ public void testTriggerAuditorWithNoPendingAuditTask() throws Exception { ensemble.add(new BookieSocketAddress("88.88.88.88:8888")); metadata.addEnsemble(0, ensemble); - MutableInt ledgerCreateRC = new MutableInt(-1); - CountDownLatch latch = new CountDownLatch(1); long ledgerId = (Math.abs(rand.nextLong())) % 100000000; try (LedgerManager lm = driver.getLedgerManagerFactory().newLedgerManager()) { - lm.createLedgerMetadata(ledgerId, metadata, - (rc, result) -> { - ledgerCreateRC.setValue(rc); - latch.countDown(); - }); + lm.createLedgerMetadata(ledgerId, metadata).get(2000, TimeUnit.MILLISECONDS); } - - Assert.assertTrue("Ledger creation should complete within 2 secs", - latch.await(2000, TimeUnit.MILLISECONDS)); - Assert.assertEquals("LedgerCreate should succeed and return OK rc value", BKException.Code.OK, - ledgerCreateRC.getValue()); ledgerList.add(ledgerId); } diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index 7df7038f96c..8bd6684ca19 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -37,18 +37,18 @@ import java.util.List; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.function.Consumer; import java.util.function.Function; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.client.BKException.Code; +import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.metadata.etcd.helpers.KeyIterator; import org.apache.bookkeeper.metadata.etcd.helpers.KeyStream; import org.apache.bookkeeper.metadata.etcd.helpers.ValueStream; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap; @@ -103,9 +103,9 @@ ValueStream getLedgerMetadataStream(long ledgerId) { } @Override - public void createLedgerMetadata(long ledgerId, - LedgerMetadata metadata, - GenericCallback> cb) { + public CompletableFuture> createLedgerMetadata(long ledgerId, + LedgerMetadata metadata) { + CompletableFuture> promise = new CompletableFuture<>(); String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); log.info("Create ledger metadata under key {}", ledgerKey); @@ -129,36 +129,36 @@ public void createLedgerMetadata(long ledgerId, GetResponse getResp = resp.getGetResponses().get(0); if (getResp.getCount() <= 0) { // key doesn't exist but we fail to put the key - cb.operationComplete(Code.UnexpectedConditionException, null); + promise.completeExceptionally(new BKException.BKUnexpectedConditionException()); } else { // key exists - cb.operationComplete(Code.LedgerExistException, null); + promise.completeExceptionally(new BKException.BKLedgerExistException()); } } else { - cb.operationComplete(Code.OK, new Versioned<>(metadata, - new LongVersion(resp.getHeader().getRevision()))); + promise.complete(new Versioned<>(metadata, + new LongVersion(resp.getHeader().getRevision()))); } }) .exceptionally(cause -> { - cb.operationComplete(Code.MetaStoreException, null); - return null; - }); + promise.completeExceptionally(new BKException.MetaStoreException()); + return null; + }); + return promise; } @Override - public void removeLedgerMetadata(long ledgerId, - Version version, - GenericCallback cb) { + public CompletableFuture removeLedgerMetadata(long ledgerId, Version version) { + CompletableFuture promise = new CompletableFuture<>(); long revision = -0xabcd; if (Version.NEW == version) { log.error("Request to delete ledger {} metadata with version set to the initial one", ledgerId); - cb.operationComplete(Code.MetadataVersionException, null); - return; + promise.completeExceptionally(new BKException.BKMetadataVersionException()); + return promise; } else if (Version.ANY != version) { if (!(version instanceof LongVersion)) { log.info("Not an instance of LongVersion : {}", ledgerId); - cb.operationComplete(Code.MetadataVersionException, null); - return; + promise.completeExceptionally(new BKException.BKMetadataVersionException()); + return promise; } else { revision = ((LongVersion) version).getLongVersion(); } @@ -192,26 +192,28 @@ public void removeLedgerMetadata(long ledgerId, .commit() .thenAccept(txnResp -> { if (txnResp.isSucceeded()) { - cb.operationComplete(Code.OK, null); + promise.complete(null); } else { GetResponse getResp = txnResp.getGetResponses().get(0); if (getResp.getCount() > 0) { // fail to delete the ledger - cb.operationComplete(Code.MetadataVersionException, null); + promise.completeExceptionally(new BKException.BKMetadataVersionException()); } else { log.warn("Deleting ledger {} failed due to : ledger key {} doesn't exist", ledgerId, ledgerKey); - cb.operationComplete(Code.NoSuchLedgerExistsException, null); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); } } }) .exceptionally(cause -> { - cb.operationComplete(Code.MetaStoreException, null); - return null; - }); + promise.completeExceptionally(new BKException.MetaStoreException()); + return null; + }); + return promise; } @Override - public void readLedgerMetadata(long ledgerId, GenericCallback> readCb) { + public CompletableFuture> readLedgerMetadata(long ledgerId) { + CompletableFuture> promise = new CompletableFuture<>(); String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); log.info("read ledger metadata under key {}", ledgerKey); @@ -222,29 +224,30 @@ public void readLedgerMetadata(long ledgerId, GenericCallback(metadata, new LongVersion(kv.getModRevision()))); + promise.complete(new Versioned<>(metadata, new LongVersion(kv.getModRevision()))); } catch (IOException ioe) { log.error("Could not parse ledger metadata for ledger : {}", ledgerId, ioe); - readCb.operationComplete(Code.MetaStoreException, null); + promise.completeExceptionally(new BKException.MetaStoreException()); return; } } else { - readCb.operationComplete(Code.NoSuchLedgerExistsException, null); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); } }) .exceptionally(cause -> { - readCb.operationComplete(Code.MetaStoreException, null); - return null; - }); + promise.completeExceptionally(new BKException.MetaStoreException()); + return null; + }); + return promise; } @Override - public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, - Version currentVersion, GenericCallback> cb) { + public CompletableFuture> writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, + Version currentVersion) { + CompletableFuture> promise = new CompletableFuture<>(); if (Version.NEW == currentVersion || !(currentVersion instanceof LongVersion)) { - cb.operationComplete(Code.MetadataVersionException, null); - return; + promise.completeExceptionally(new BKException.BKMetadataVersionException()); + return promise; } final LongVersion lv = (LongVersion) currentVersion; String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); @@ -264,24 +267,24 @@ public void writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, .commit() .thenAccept(resp -> { if (resp.isSucceeded()) { - cb.operationComplete( - Code.OK, new Versioned<>(metadata, new LongVersion(resp.getHeader().getRevision()))); + promise.complete(new Versioned<>(metadata, new LongVersion(resp.getHeader().getRevision()))); } else { GetResponse getResp = resp.getGetResponses().get(0); if (getResp.getCount() > 0) { log.warn("Conditional update ledger metadata failed :" + " expected version = {}, actual version = {}", getResp.getKvs().get(0).getModRevision(), lv); - cb.operationComplete(Code.MetadataVersionException, null); + promise.completeExceptionally(new BKException.BKMetadataVersionException()); } else { - cb.operationComplete(Code.NoSuchLedgerExistsException, null); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); } } }) .exceptionally(cause -> { - cb.operationComplete(Code.MetaStoreException, null); - return null; - }); + promise.completeExceptionally(new BKException.MetaStoreException()); + return null; + }); + return promise; } private LedgerMetadataConsumer listenerToConsumer(long ledgerId, diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index e44678034d6..4e214837761 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -57,7 +57,6 @@ import org.apache.bookkeeper.metadata.etcd.helpers.ValueStream; import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; @@ -103,43 +102,31 @@ public void testLedgerCRUD() throws Exception { ); // ledger doesn't exist: read - - GenericCallbackFuture> readFuture = new GenericCallbackFuture<>(); - lm.readLedgerMetadata(ledgerId, readFuture); try { - result(readFuture); + result(lm.readLedgerMetadata(ledgerId)); fail("Should fail on reading ledger metadata if the ledger doesn't exist"); } catch (BKException bke) { assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); } // ledger doesn't exist : delete - - GenericCallbackFuture deleteFuture = new GenericCallbackFuture<>(); - lm.removeLedgerMetadata(ledgerId, new LongVersion(999L), deleteFuture); try { - result(deleteFuture); + result(lm.removeLedgerMetadata(ledgerId, new LongVersion(999L))); fail("Should fail on deleting ledger metadata if the ledger doesn't exist"); } catch (BKException bke) { assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); } // ledger doesn't exist : write - - GenericCallbackFuture> writeFuture = new GenericCallbackFuture<>(); - lm.writeLedgerMetadata(ledgerId, metadata, new LongVersion(999L), writeFuture); try { - result(deleteFuture); + result(lm.writeLedgerMetadata(ledgerId, metadata, new LongVersion(999L))); fail("Should fail on updating ledger metadata if the ledger doesn't exist"); } catch (BKException bke) { assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); } // ledger doesn't exist : create - - GenericCallbackFuture> createFuture = new GenericCallbackFuture<>(); - lm.createLedgerMetadata(ledgerId, metadata, createFuture); - Versioned writtenMetadata = result(createFuture); + Versioned writtenMetadata = result(lm.createLedgerMetadata(ledgerId, metadata)); assertSame(metadata, writtenMetadata.getValue()); Version version = writtenMetadata.getVersion(); assertNotNull(version); @@ -149,73 +136,52 @@ public void testLedgerCRUD() throws Exception { // ledger exists : create // attempt to create the ledger again will result in exception `LedgerExistsException` - createFuture = new GenericCallbackFuture<>(); try { - lm.createLedgerMetadata(ledgerId, metadata, createFuture); - result(createFuture); + result(lm.createLedgerMetadata(ledgerId, metadata)); fail("Should fail on creating ledger metadata if the ledger already exists"); } catch (BKException bke) { assertEquals(Code.LedgerExistException, bke.getCode()); } // ledger exists: get - - readFuture = new GenericCallbackFuture<>(); - lm.readLedgerMetadata(ledgerId, readFuture); - Versioned readMetadata = result(readFuture); + Versioned readMetadata = result(lm.readLedgerMetadata(ledgerId)); assertEquals(metadata, readMetadata.getValue()); // ledger exists: update metadata with wrong version - writeFuture = new GenericCallbackFuture<>(); - lm.writeLedgerMetadata(ledgerId, readMetadata.getValue(), new LongVersion(Long.MAX_VALUE), writeFuture); try { - result(writeFuture); + result(lm.writeLedgerMetadata(ledgerId, readMetadata.getValue(), new LongVersion(Long.MAX_VALUE))); fail("Should fail to write metadata using a wrong version"); } catch (BKException bke) { assertEquals(Code.MetadataVersionException, bke.getCode()); } - readFuture = new GenericCallbackFuture<>(); - lm.readLedgerMetadata(ledgerId, readFuture); - readMetadata = result(readFuture); + readMetadata = result(lm.readLedgerMetadata(ledgerId)); assertEquals(metadata, readMetadata.getValue()); // ledger exists: delete metadata with wrong version - - deleteFuture = new GenericCallbackFuture<>(); - lm.removeLedgerMetadata(ledgerId, new LongVersion(Long.MAX_VALUE), deleteFuture); try { - result(deleteFuture); + result(lm.removeLedgerMetadata(ledgerId, new LongVersion(Long.MAX_VALUE))); fail("Should fail to delete metadata using a wrong version"); } catch (BKException bke) { assertEquals(Code.MetadataVersionException, bke.getCode()); } - readFuture = new GenericCallbackFuture<>(); - lm.readLedgerMetadata(ledgerId, readFuture); - readMetadata = result(readFuture); + + readMetadata = result(lm.readLedgerMetadata(ledgerId)); assertEquals(metadata, readMetadata.getValue()); // ledger exists: update metadata with the right version LongVersion curVersion = (LongVersion) readMetadata.getVersion(); - writeFuture = new GenericCallbackFuture<>(); - lm.writeLedgerMetadata(ledgerId, readMetadata.getValue(), curVersion, writeFuture); - writtenMetadata = result(writeFuture); + writtenMetadata = result(lm.writeLedgerMetadata(ledgerId, readMetadata.getValue(), curVersion)); LongVersion newVersion = (LongVersion) writtenMetadata.getVersion(); assertTrue(curVersion.getLongVersion() < newVersion.getLongVersion()); - readFuture = new GenericCallbackFuture<>(); - lm.readLedgerMetadata(ledgerId, readFuture); - readMetadata = result(readFuture); + + readMetadata = result(lm.readLedgerMetadata(ledgerId)); assertEquals(writtenMetadata, readMetadata); // ledger exists: delete metadata with the right version - - deleteFuture = new GenericCallbackFuture<>(); - lm.removeLedgerMetadata(ledgerId, newVersion, deleteFuture); - result(deleteFuture); - readFuture = new GenericCallbackFuture<>(); + result(lm.removeLedgerMetadata(ledgerId, newVersion)); try { - lm.readLedgerMetadata(ledgerId, readFuture); - result(readFuture); + result(lm.readLedgerMetadata(ledgerId)); fail("Should fail to read ledger if it is deleted"); } catch (BKException bke) { assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); @@ -267,14 +233,12 @@ public void testLedgerRangeIterator() throws Exception { private void createNumLedgers(int numLedgers) throws Exception { List>> createFutures = new ArrayList<>(numLedgers); for (int i = 0; i < numLedgers; i++) { - GenericCallbackFuture> createFuture = new GenericCallbackFuture<>(); - createFutures.add(createFuture); LedgerMetadata metadata = new LedgerMetadata( 3, 3, 2, DigestType.CRC32C, "test-password".getBytes(UTF_8) ); - lm.createLedgerMetadata(i, metadata, createFuture); + createFutures.add(lm.createLedgerMetadata(i, metadata)); } FutureUtils.result(FutureUtils.collect(createFutures)); } @@ -290,10 +254,8 @@ public void testRegisterLedgerMetadataListener() throws Exception { "test-password".getBytes(UTF_8) ); metadata.addEnsemble(0L, createNumBookies(3)); - GenericCallbackFuture> createFuture = new GenericCallbackFuture<>(); - lm.createLedgerMetadata(ledgerId, metadata, createFuture); - result(createFuture); - Versioned readMetadata = readLedgerMetadata(ledgerId); + result(lm.createLedgerMetadata(ledgerId, metadata)); + Versioned readMetadata = lm.readLedgerMetadata(ledgerId).get(); log.info("Create ledger metadata : {}", readMetadata.getValue()); // register first listener @@ -326,10 +288,10 @@ public void testRegisterLedgerMetadataListener() throws Exception { assertNotNull(lm.getLedgerMetadataStream(ledgerId)); // update the metadata - writeLedgerMetadata(ledgerId, - LedgerMetadataBuilder.from(metadata).newEnsembleEntry(10L, createNumBookies(3)).build(), - notifiedMetadata.getVersion()); - readMetadata = readLedgerMetadata(ledgerId); + lm.writeLedgerMetadata(ledgerId, + LedgerMetadataBuilder.from(metadata).newEnsembleEntry(10L, createNumBookies(3)).build(), + notifiedMetadata.getVersion()).get(); + readMetadata = lm.readLedgerMetadata(ledgerId).get(); assertEquals(readMetadata, metadataQueue1.take()); assertEquals(readMetadata, metadataQueue2.take()); lms = lm.getLedgerMetadataStream(ledgerId); @@ -343,10 +305,10 @@ public void testRegisterLedgerMetadataListener() throws Exception { assertEquals(1, lms.getNumConsumers()); // update the metadata again - writeLedgerMetadata(ledgerId, - LedgerMetadataBuilder.from(metadata).newEnsembleEntry(20L, createNumBookies(3)).build(), - readMetadata.getVersion()); - readMetadata = readLedgerMetadata(ledgerId); + lm.writeLedgerMetadata(ledgerId, + LedgerMetadataBuilder.from(metadata).newEnsembleEntry(20L, createNumBookies(3)).build(), + readMetadata.getVersion()).get(); + readMetadata = lm.readLedgerMetadata(ledgerId).get(); assertEquals(readMetadata, metadataQueue1.take()); assertNull(metadataQueue2.poll()); @@ -359,10 +321,10 @@ public void testRegisterLedgerMetadataListener() throws Exception { assertEquals(0, lms.getNumConsumers()); // update the metadata again - writeLedgerMetadata(ledgerId, - LedgerMetadataBuilder.from(metadata).newEnsembleEntry(30L, createNumBookies(3)).build(), - readMetadata.getVersion()); - readMetadata = readLedgerMetadata(ledgerId); + lm.writeLedgerMetadata(ledgerId, + LedgerMetadataBuilder.from(metadata).newEnsembleEntry(30L, createNumBookies(3)).build(), + readMetadata.getVersion()).get(); + readMetadata = lm.readLedgerMetadata(ledgerId).get(); assertNull(metadataQueue1.poll()); assertNull(metadataQueue2.poll()); @@ -375,7 +337,7 @@ public void testRegisterLedgerMetadataListener() throws Exception { assertEquals(1, lms.getNumConsumers()); // delete the ledger - removeLedgerMetadata(ledgerId, readMetadata.getVersion()); + lm.removeLedgerMetadata(ledgerId, readMetadata.getVersion()).get(); // the listener will eventually be removed while (lm.getLedgerMetadataStream(ledgerId) != null) { TimeUnit.MILLISECONDS.sleep(100); @@ -385,24 +347,6 @@ public void testRegisterLedgerMetadataListener() throws Exception { assertNull(metadataQueue2.poll()); } - Versioned readLedgerMetadata(long lid) throws Exception { - GenericCallbackFuture> readFuture = new GenericCallbackFuture<>(); - lm.readLedgerMetadata(lid, readFuture); - return result(readFuture); - } - - void writeLedgerMetadata(long lid, LedgerMetadata metadata, Version version) throws Exception { - GenericCallbackFuture> writeFuture = new GenericCallbackFuture<>(); - lm.writeLedgerMetadata(lid, metadata, version, writeFuture); - result(writeFuture); - } - - void removeLedgerMetadata(long lid, Version version) throws Exception { - GenericCallbackFuture deleteFuture = new GenericCallbackFuture<>(); - lm.removeLedgerMetadata(lid, version, deleteFuture); - result(deleteFuture); - } - static List createNumBookies(int numBookies) { return IntStream.range(0, numBookies) .mapToObj(idx -> new BookieSocketAddress("127.0.0.1", 3181 + idx)) diff --git a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy index a4858e9a8d7..7f229e79074 100644 --- a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy +++ b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy @@ -59,16 +59,7 @@ class TestCompatRecoveryNoPassword { DockerClient docker private LedgerMetadata getLedgerMetadata(BookKeeper bookkeeper, long ledgerId) throws Exception { - CompletableFuture> future = new CompletableFuture<>() - bookkeeper.getLedgerManager().readLedgerMetadata( - ledgerId, { rc, result -> - if (rc != BKException.Code.OK) { - future.completeExceptionally(BKException.create(rc)) - } else { - future.complete(result) - } - }) - return future.get().getValue() + return bookkeeper.getLedgerManager().readLedgerMetadata(ledgerId).get().getValue() } private static class ReplicationVerificationCallback implements ReadEntryCallback { From 1b85ce353fcc5e38cbebcd2f9c63c9022b9af347 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Wed, 21 Nov 2018 18:24:59 +0100 Subject: [PATCH 0126/1642] AbstractZkLedgerManager should catch all exceptions from parseConfig And always complete the request if an exception does occur, rather than throwing it out to the higher level, and hanging the request forever. Reviewers: Enrico Olivelli This closes #1824 from ivankelly/catch-all-exception-zk --- .../org/apache/bookkeeper/meta/AbstractZkLedgerManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index a23c3302427..b5bea645b9b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -402,8 +402,8 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta LongVersion version = new LongVersion(stat.getVersion()); LedgerMetadata metadata = LedgerMetadata.parseConfig(data, Optional.of(stat.getCtime())); promise.complete(new Versioned<>(metadata, version)); - } catch (IOException e) { - LOG.error("Could not parse ledger metadata for ledger: " + ledgerId, e); + } catch (Throwable t) { + LOG.error("Could not parse ledger metadata for ledger: {}", ledgerId, t); promise.completeExceptionally(new BKException.ZKException()); } } From cb9068be7741f114b23c6f29d1db348a0dbac2b5 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 22 Nov 2018 02:42:45 +0100 Subject: [PATCH 0127/1642] LedgerMetadata#parseConfig uses Metadata Builder parseConfig is one of the main places that mutates member fields of LedgerMetadata. This change removes these mutates to make it use LedgerMetadataBuilder instead. Master issue: #281 Reviewers: Sijie Guo This closes #1825 from ivankelly/parse-builder --- .../apache/bookkeeper/bookie/BookieShell.java | 4 +- .../bookkeeper/client/LedgerMetadata.java | 174 ++++++++++-------- .../client/LedgerMetadataBuilder.java | 43 ++++- .../meta/AbstractZkLedgerManager.java | 2 +- .../meta/MSLedgerManagerFactory.java | 4 +- .../client/MetadataUpdateLoopTest.java | 9 +- .../client/ParallelLedgerRecoveryTest.java | 3 +- .../meta/AbstractZkLedgerManagerTest.java | 24 ++- .../apache/bookkeeper/meta/GcLedgersTest.java | 13 +- .../bookkeeper/meta/MockLedgerManager.java | 5 +- .../metadata/etcd/EtcdLedgerManager.java | 6 +- .../metadata/etcd/EtcdLedgerManagerTest.java | 16 +- 12 files changed, 186 insertions(+), 117 deletions(-) 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 a4a437c6993..12d85e2eb98 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 @@ -25,7 +25,6 @@ import static org.apache.bookkeeper.tools.cli.helpers.CommandHelpers.getBookieSocketAddrStringRepresentation; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.UncheckedExecutionException; @@ -63,6 +62,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -1143,7 +1143,7 @@ public int runCmd(CommandLine cmdLine) throws Exception { } else if (cmdLine.hasOption("restorefromfile")) { byte[] serialized = Files.readAllBytes( FileSystems.getDefault().getPath(cmdLine.getOptionValue("restorefromfile"))); - LedgerMetadata md = LedgerMetadata.parseConfig(serialized, Optional.absent()); + LedgerMetadata md = LedgerMetadata.parseConfig(serialized, Optional.empty()); m.createLedgerMetadata(lid, md).join(); } else { printLedgerMetadata(lid, m.readLedgerMetadata(lid).get().getValue(), true); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 620a7980198..90f0f136eab 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -22,7 +22,6 @@ import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.protobuf.ByteString; @@ -38,9 +37,11 @@ import java.util.Map; import java.util.Map.Entry; import java.util.NavigableMap; +import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; +import java.util.stream.Collectors; import lombok.EqualsAndHashCode; import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -72,14 +73,13 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe public static final String VERSION_KEY = "BookieMetadataFormatVersion"; private int metadataFormatVersion = 0; - private int ensembleSize; private int writeQuorumSize; private int ackQuorumSize; private long length; private long lastEntryId; private long ctime; - boolean storeSystemtimeAsLedgerCreationTime = false; + boolean storeCtime; // non-private so builder can access for copy private LedgerMetadataFormat.State state; private TreeMap> ensembles = new TreeMap<>(); @@ -103,12 +103,13 @@ public LedgerMetadata(int ensembleSize, this.ackQuorumSize = ackQuorumSize; if (storeSystemtimeAsLedgerCreationTime) { this.ctime = System.currentTimeMillis(); + this.storeCtime = storeSystemtimeAsLedgerCreationTime; } else { // if client disables storing its system time as ledger creation time, there should be no ctime at this // moment. this.ctime = -1L; + this.storeCtime = false; } - this.storeSystemtimeAsLedgerCreationTime = storeSystemtimeAsLedgerCreationTime; /* * It is set in PendingReadOp.readEntryComplete, and @@ -128,19 +129,22 @@ public LedgerMetadata(int ensembleSize, } } - LedgerMetadata(int ensembleSize, + LedgerMetadata(int metadataFormatVersion, + int ensembleSize, int writeQuorumSize, int ackQuorumSize, LedgerMetadataFormat.State state, - java.util.Optional lastEntryId, - java.util.Optional length, + Optional lastEntryId, + Optional length, Map> ensembles, DigestType digestType, - java.util.Optional password, - java.util.Optional ctime, + Optional password, + long ctime, + boolean storeCtime, Map customMetadata) { checkArgument(ensembles.size() > 0, "There must be at least one ensemble in the ledger"); + this.metadataFormatVersion = metadataFormatVersion; this.ensembleSize = ensembleSize; this.writeQuorumSize = writeQuorumSize; this.ackQuorumSize = ackQuorumSize; @@ -165,10 +169,8 @@ public LedgerMetadata(int ensembleSize, this.hasPassword = true; }); - ctime.ifPresent((c) -> { - this.ctime = c; - this.storeSystemtimeAsLedgerCreationTime = true; - }); + this.ctime = ctime; + this.storeCtime = storeCtime; this.customMetadata.putAll(customMetadata); } @@ -196,7 +198,7 @@ public LedgerMetadata(int ensembleSize, int writeQuorumSize, int ackQuorumSize, this.hasPassword = other.hasPassword; this.digestType = other.digestType; this.ctime = other.ctime; - this.storeSystemtimeAsLedgerCreationTime = other.storeSystemtimeAsLedgerCreationTime; + this.storeCtime = other.storeCtime; this.password = new byte[other.password.length]; System.arraycopy(other.password, 0, this.password, 0, other.password.length); // copy the ensembles @@ -243,11 +245,6 @@ public long getCtime() { return ctime; } - @VisibleForTesting - void setCtime(long ctime) { - this.ctime = ctime; - } - /** * In versions 4.1.0 and below, the digest type and password were not * stored in the metadata. @@ -260,7 +257,11 @@ boolean hasPassword() { @VisibleForTesting public byte[] getPassword() { - return Arrays.copyOf(password, password.length); + if (!hasPassword()) { + return new byte[0]; + } else { + return Arrays.copyOf(password, password.length); + } } @Override @@ -382,7 +383,7 @@ LedgerMetadataFormat buildProtoFormat(boolean withPassword) { .setEnsembleSize(ensembleSize).setLength(length) .setState(state).setLastEntryId(lastEntryId); - if (storeSystemtimeAsLedgerCreationTime) { + if (storeCtime) { builder.setCtime(ctime); } @@ -467,15 +468,14 @@ private byte[] serializeVersion1() { * * @param bytes * byte array to parse - * @param msCtime + * @param metadataStoreCtime * metadata store creation time, used for legacy ledgers * @return LedgerConfig * @throws IOException * if the given byte[] cannot be parsed */ - public static LedgerMetadata parseConfig(byte[] bytes, Optional msCtime) throws IOException { - LedgerMetadata lc = new LedgerMetadata(); - + public static LedgerMetadata parseConfig(byte[] bytes, + Optional metadataStoreCtime) throws IOException { String config = new String(bytes, UTF_8); if (LOG.isDebugEnabled()) { @@ -486,89 +486,94 @@ public static LedgerMetadata parseConfig(byte[] bytes, Optional msCtime) t if (versionLine == null) { throw new IOException("Invalid metadata. Content missing"); } + final int metadataFormatVersion; if (versionLine.startsWith(VERSION_KEY)) { String parts[] = versionLine.split(tSplitter); - lc.metadataFormatVersion = Integer.parseInt(parts[1]); + metadataFormatVersion = Integer.parseInt(parts[1]); } else { // if no version is set, take it to be version 1 // as the parsing is the same as what we had before // we introduce versions - lc.metadataFormatVersion = 1; + metadataFormatVersion = 1; // reset the reader reader.close(); reader = new BufferedReader(new StringReader(config)); } - if (lc.metadataFormatVersion < LOWEST_COMPAT_METADATA_FORMAT_VERSION - || lc.metadataFormatVersion > CURRENT_METADATA_FORMAT_VERSION) { - throw new IOException("Metadata version not compatible. Expected between " - + LOWEST_COMPAT_METADATA_FORMAT_VERSION + " and " + CURRENT_METADATA_FORMAT_VERSION - + ", but got " + lc.metadataFormatVersion); + if (metadataFormatVersion < LOWEST_COMPAT_METADATA_FORMAT_VERSION + || metadataFormatVersion > CURRENT_METADATA_FORMAT_VERSION) { + throw new IOException( + String.format("Metadata version not compatible. Expected between %d and %d, but got %d", + LOWEST_COMPAT_METADATA_FORMAT_VERSION, CURRENT_METADATA_FORMAT_VERSION, + metadataFormatVersion)); } - if (lc.metadataFormatVersion == 1) { - return parseVersion1Config(lc, reader); + if (metadataFormatVersion == 1) { + return parseVersion1Config(reader); } + LedgerMetadataBuilder builder = LedgerMetadataBuilder.create() + .withMetadataFormatVersion(metadataFormatVersion); + // remaining size is total minus the length of the version line and '\n' char[] configBuffer = new char[config.length() - (versionLine.length() + 1)]; if (configBuffer.length != reader.read(configBuffer, 0, configBuffer.length)) { throw new IOException("Invalid metadata buffer"); } - LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder(); + LedgerMetadataFormat.Builder formatBuilder = LedgerMetadataFormat.newBuilder(); + TextFormat.merge((CharSequence) CharBuffer.wrap(configBuffer), formatBuilder); + LedgerMetadataFormat data = formatBuilder.build(); - TextFormat.merge((CharSequence) CharBuffer.wrap(configBuffer), builder); - LedgerMetadataFormat data = builder.build(); - lc.writeQuorumSize = data.getQuorumSize(); - if (data.hasCtime()) { - lc.ctime = data.getCtime(); - lc.storeSystemtimeAsLedgerCreationTime = true; - } else if (msCtime.isPresent()) { - lc.ctime = msCtime.get(); - lc.storeSystemtimeAsLedgerCreationTime = false; - } + builder.withEnsembleSize(data.getEnsembleSize()); + builder.withWriteQuorumSize(data.getQuorumSize()); if (data.hasAckQuorumSize()) { - lc.ackQuorumSize = data.getAckQuorumSize(); + builder.withAckQuorumSize(data.getAckQuorumSize()); } else { - lc.ackQuorumSize = lc.writeQuorumSize; + builder.withAckQuorumSize(data.getQuorumSize()); + } + + if (data.hasCtime()) { + builder.withCreationTime(data.getCtime()).storingCreationTime(true); + } else if (metadataStoreCtime.isPresent()) { + builder.withCreationTime(metadataStoreCtime.get()).storingCreationTime(false); } - lc.ensembleSize = data.getEnsembleSize(); - lc.length = data.getLength(); - lc.state = data.getState(); - lc.lastEntryId = data.getLastEntryId(); + if (data.getState() == LedgerMetadataFormat.State.IN_RECOVERY) { + builder.withInRecoveryState(); + } else if (data.getState() == LedgerMetadataFormat.State.CLOSED) { + builder.closingAt(data.getLastEntryId(), data.getLength()); + } if (data.hasPassword()) { - lc.digestType = data.getDigestType(); - lc.password = data.getPassword().toByteArray(); - lc.hasPassword = true; + builder.withPassword(data.getPassword().toByteArray()) + .withDigestType(protoToApiDigestType(data.getDigestType())); } for (LedgerMetadataFormat.Segment s : data.getSegmentList()) { - ArrayList addrs = new ArrayList(); - for (String member : s.getEnsembleMemberList()) { - addrs.add(new BookieSocketAddress(member)); + List addrs = new ArrayList<>(); + for (String addr : s.getEnsembleMemberList()) { + addrs.add(new BookieSocketAddress(addr)); } - lc.addEnsemble(s.getFirstEntryId(), addrs); + builder.newEnsembleEntry(s.getFirstEntryId(), addrs); } if (data.getCustomMetadataCount() > 0) { - List cMetadataList = data.getCustomMetadataList(); - lc.customMetadata = Maps.newHashMap(); - for (LedgerMetadataFormat.cMetadataMapEntry ent : cMetadataList) { - lc.customMetadata.put(ent.getKey(), ent.getValue().toByteArray()); - } + builder.withCustomMetadata(data.getCustomMetadataList().stream().collect( + Collectors.toMap(e -> e.getKey(), + e -> e.getValue().toByteArray()))); } - return lc; + return builder.build(); } - static LedgerMetadata parseVersion1Config(LedgerMetadata lc, - BufferedReader reader) throws IOException { + static LedgerMetadata parseVersion1Config(BufferedReader reader) throws IOException { + LedgerMetadataBuilder builder = LedgerMetadataBuilder.create().withMetadataFormatVersion(1); try { - lc.writeQuorumSize = lc.ackQuorumSize = Integer.parseInt(reader.readLine()); - lc.ensembleSize = Integer.parseInt(reader.readLine()); - lc.length = Long.parseLong(reader.readLine()); + int quorumSize = Integer.parseInt(reader.readLine()); + int ensembleSize = Integer.parseInt(reader.readLine()); + long length = Long.parseLong(reader.readLine()); + + builder.withEnsembleSize(ensembleSize).withWriteQuorumSize(quorumSize).withAckQuorumSize(quorumSize); String line = reader.readLine(); while (line != null) { @@ -577,27 +582,25 @@ static LedgerMetadata parseVersion1Config(LedgerMetadata lc, if (parts[1].equals(closed)) { Long l = Long.parseLong(parts[0]); if (l == IN_RECOVERY) { - lc.state = LedgerMetadataFormat.State.IN_RECOVERY; + builder.withInRecoveryState(); } else { - lc.state = LedgerMetadataFormat.State.CLOSED; - lc.lastEntryId = l; + builder.closingAt(l, length); } break; - } else { - lc.state = LedgerMetadataFormat.State.OPEN; } ArrayList addrs = new ArrayList(); for (int j = 1; j < parts.length; j++) { addrs.add(new BookieSocketAddress(parts[j])); } - lc.addEnsemble(Long.parseLong(parts[0]), addrs); + builder.newEnsembleEntry(Long.parseLong(parts[0]), addrs); + line = reader.readLine(); } + return builder.build(); } catch (NumberFormatException e) { throw new IOException(e); } - return lc; } /** @@ -691,4 +694,23 @@ Long getLastEnsembleKey() { checkState(!ensembles.isEmpty(), "Metadata should never be created with no ensembles"); return ensembles.lastKey(); } + + int getMetadataFormatVersion() { + return metadataFormatVersion; + } + + private static DigestType protoToApiDigestType(LedgerMetadataFormat.DigestType digestType) { + switch (digestType) { + case HMAC: + return DigestType.MAC; + case CRC32: + return DigestType.CRC32; + case CRC32C: + return DigestType.CRC32C; + case DUMMY: + return DigestType.DUMMY; + default: + throw new IllegalArgumentException("Unable to convert digest type " + digestType); + } + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index 8206438b715..74ec7179892 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -43,6 +43,7 @@ @Unstable @VisibleForTesting public class LedgerMetadataBuilder { + private int metadataFormatVersion = LedgerMetadata.CURRENT_METADATA_FORMAT_VERSION; private int ensembleSize = 3; private int writeQuorumSize = 3; private int ackQuorumSize = 2; @@ -56,7 +57,8 @@ public class LedgerMetadataBuilder { private DigestType digestType = DigestType.CRC32C; private Optional password = Optional.empty(); - private Optional ctime = Optional.empty(); + private long ctime = -1; + private boolean storeCtime = false; private Map customMetadata = Collections.emptyMap(); public static LedgerMetadataBuilder create() { @@ -65,6 +67,7 @@ public static LedgerMetadataBuilder create() { public static LedgerMetadataBuilder from(LedgerMetadata other) { LedgerMetadataBuilder builder = new LedgerMetadataBuilder(); + builder.metadataFormatVersion = other.getMetadataFormatVersion(); builder.ensembleSize = other.getEnsembleSize(); builder.writeQuorumSize = other.getWriteQuorumSize(); builder.ackQuorumSize = other.getAckQuorumSize(); @@ -87,14 +90,19 @@ public static LedgerMetadataBuilder from(LedgerMetadata other) { builder.password = Optional.of(other.getPassword()); } - if (other.storeSystemtimeAsLedgerCreationTime) { - builder.ctime = Optional.of(other.getCtime()); - } + builder.ctime = other.getCtime(); + builder.storeCtime = other.storeCtime; + builder.customMetadata = ImmutableMap.copyOf(other.getCustomMetadata()); return builder; } + public LedgerMetadataBuilder withMetadataFormatVersion(int version) { + this.metadataFormatVersion = version; + return this; + } + public LedgerMetadataBuilder withPassword(byte[] password) { this.password = Optional.of(Arrays.copyOf(password, password.length)); return this; @@ -112,14 +120,11 @@ public LedgerMetadataBuilder withEnsembleSize(int ensembleSize) { } public LedgerMetadataBuilder withWriteQuorumSize(int writeQuorumSize) { - checkArgument(ensembleSize >= writeQuorumSize, "Write quorum must be less or equal to ensemble size"); - checkArgument(writeQuorumSize >= ackQuorumSize, "Write quorum must be greater or equal to ack quorum"); this.writeQuorumSize = writeQuorumSize; return this; } public LedgerMetadataBuilder withAckQuorumSize(int ackQuorumSize) { - checkArgument(writeQuorumSize >= ackQuorumSize, "Ack quorum must be less or equal to write quorum"); this.ackQuorumSize = ackQuorumSize; return this; } @@ -154,10 +159,30 @@ public LedgerMetadataBuilder closingAt(long lastEntryId, long length) { return this; } + public LedgerMetadataBuilder withCustomMetadata(Map customMetadata) { + this.customMetadata = ImmutableMap.copyOf(customMetadata); + return this; + } + + public LedgerMetadataBuilder withCreationTime(long ctime) { + this.ctime = ctime; + return this; + } + + public LedgerMetadataBuilder storingCreationTime(boolean storing) { + this.storeCtime = storing; + return this; + } + public LedgerMetadata build() { - return new LedgerMetadata(ensembleSize, writeQuorumSize, ackQuorumSize, + checkArgument(ensembleSize >= writeQuorumSize, "Write quorum must be less or equal to ensemble size"); + checkArgument(writeQuorumSize >= ackQuorumSize, "Write quorum must be greater or equal to ack quorum"); + + return new LedgerMetadata(metadataFormatVersion, + ensembleSize, writeQuorumSize, ackQuorumSize, state, lastEntryId, length, ensembles, - digestType, password, ctime, customMetadata); + digestType, password, ctime, storeCtime, + customMetadata); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index b5bea645b9b..306cf50d052 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -18,7 +18,6 @@ package org.apache.bookkeeper.meta; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import io.netty.util.concurrent.DefaultThreadFactory; @@ -26,6 +25,7 @@ import java.util.HashSet; import java.util.List; import java.util.NavigableSet; +import java.util.Optional; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index 694ae64d3f3..35658dde08a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -21,12 +21,12 @@ import static org.apache.bookkeeper.metastore.MetastoreTable.ALL_FIELDS; import static org.apache.bookkeeper.metastore.MetastoreTable.NON_FIELDS; -import com.google.common.base.Optional; import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; @@ -441,7 +441,7 @@ public void complete(int rc, Versioned value, Object ctx) { } try { LedgerMetadata metadata = LedgerMetadata.parseConfig( - value.getValue().getField(META_FIELD), Optional.absent()); + value.getValue().getField(META_FIELD), Optional.empty()); promise.complete(new Versioned<>(metadata, value.getVersion())); } catch (IOException e) { LOG.error("Could not parse ledger metadata for ledger " + ledgerId + " : ", e); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java index 23476855ed3..843ec9b701c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java @@ -114,7 +114,7 @@ public void testConflictOnWrite() throws Exception { BookieSocketAddress b3 = new BookieSocketAddress("0.0.0.3:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) - .newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); + .withWriteQuorumSize(2).newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); @@ -180,8 +180,8 @@ public void testConflictOnWriteBothWritingSame() throws Exception { BookieSocketAddress b2 = new BookieSocketAddress("0.0.0.2:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) - .newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); - Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); + .withWriteQuorumSize(2).newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); + Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); CompletableFuture> loop1 = new MetadataUpdateLoop( @@ -233,7 +233,7 @@ public void testConflictOnLocalUpdate() throws Exception { BookieSocketAddress b3 = new BookieSocketAddress("0.0.0.3:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) - .newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); + .withWriteQuorumSize(2).newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); @@ -341,6 +341,7 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { BookieSocketAddress b1 = new BookieSocketAddress("0.0.0.1:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(1) + .withWriteQuorumSize(1).withAckQuorumSize(1) .newEnsembleEntry(0L, Lists.newArrayList(b0)).build(); Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index 9682eb62db7..9571d1e7fb4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -673,7 +673,8 @@ public void operationComplete(int rc, Void result) { final AtomicInteger rcHolder = new AtomicInteger(-1234); final CountDownLatch doneLatch = new CountDownLatch(1); - new ReadLastConfirmedOp(readLh, bkc.getBookieClient(), readLh.getCurrentEnsemble(), + new ReadLastConfirmedOp(readLh, bkc.getBookieClient(), + readLh.getLedgerMetadata().getAllEnsembles().lastEntry().getValue(), new ReadLastConfirmedOp.LastConfirmedDataCallback() { @Override public void readLastConfirmedDataComplete(int rc, DigestManager.RecoveryData data) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 71b229dd23a..fb5638581f2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -39,8 +39,9 @@ import static org.mockito.Mockito.when; import static org.mockito.Mockito.withSettings; +import com.google.common.collect.Lists; import java.time.Duration; -import java.util.Collections; +import java.util.List; import java.util.Optional; import java.util.Set; import java.util.concurrent.Executors; @@ -49,11 +50,12 @@ import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.Code; -import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.common.testing.executors.MockExecutorController; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.LongVersion; @@ -111,12 +113,18 @@ public void setup() throws Exception { withSettings() .useConstructor(conf, mockZk) .defaultAnswer(CALLS_REAL_METHODS)); - this.metadata = new LedgerMetadata( - 5, 3, 3, - DigestType.CRC32, - new byte[0], - Collections.emptyMap(), - false); + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 3181), + new BookieSocketAddress("192.0.2.2", 3181), + new BookieSocketAddress("192.0.2.3", 3181), + new BookieSocketAddress("192.0.2.4", 3181), + new BookieSocketAddress("192.0.2.5", 3181)); + this.metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(5) + .withWriteQuorumSize(3) + .withAckQuorumSize(3) + .newEnsembleEntry(0L, ensemble) + .withCreationTime(12345L).build(); doAnswer(invocationOnMock -> { long ledgerId = invocationOnMock.getArgument(0); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index baf3d007ac8..182ffa117aa 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -30,6 +30,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.ArrayList; @@ -62,12 +63,13 @@ import org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector; import org.apache.bookkeeper.bookie.StateManager; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -93,6 +95,8 @@ public GcLedgersTest(Class lmFactoryCls) { */ private void createLedgers(int numLedgers, final Set createdLedgers) throws IOException { final AtomicInteger expected = new AtomicInteger(numLedgers); + List ensemble = Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234)); + for (int i = 0; i < numLedgers; i++) { getLedgerIdGenerator().generateLedgerId(new GenericCallback() { @Override @@ -107,8 +111,11 @@ public void operationComplete(int rc, final Long ledgerId) { return; } - getLedgerManager().createLedgerMetadata(ledgerId, - new LedgerMetadata(1, 1, 1, DigestType.MAC, "".getBytes())) + LedgerMetadata md = LedgerMetadataBuilder.create() + .withEnsembleSize(1).withWriteQuorumSize(1).withAckQuorumSize(1) + .newEnsembleEntry(0L, ensemble).build(); + + getLedgerManager().createLedgerMetadata(ledgerId, md) .whenComplete((result, exception) -> { if (exception == null) { activeLedgers.put(ledgerId, true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index 111040722ac..a8b7e86ec16 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -19,10 +19,9 @@ */ package org.apache.bookkeeper.meta; -import com.google.common.base.Optional; - import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -83,7 +82,7 @@ private Versioned readMetadata(long ledgerId) throws Exception { if (pair == null) { return null; } else { - return new Versioned<>(LedgerMetadata.parseConfig(pair.getRight(), Optional.absent()), pair.getLeft()); + return new Versioned<>(LedgerMetadata.parseConfig(pair.getRight(), Optional.empty()), pair.getLeft()); } } diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index 8bd6684ca19..8c183abcbfd 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -31,10 +31,10 @@ import com.coreos.jetcd.options.DeleteOption; import com.coreos.jetcd.options.GetOption; import com.coreos.jetcd.options.PutOption; -import com.google.common.base.Optional; import com.google.common.collect.Sets; import java.io.IOException; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -67,7 +67,7 @@ class EtcdLedgerManager implements LedgerManager { try { return LedgerMetadata.parseConfig( bs.getBytes(), - Optional.absent() + Optional.empty() ); } catch (IOException ioe) { log.error("Could not parse ledger metadata : {}", bs.toStringUtf8(), ioe); @@ -223,7 +223,7 @@ public CompletableFuture> readLedgerMetadata(long ledg KeyValue kv = getResp.getKvs().get(0); byte[] data = kv.getValue().getBytes(); try { - LedgerMetadata metadata = LedgerMetadata.parseConfig(data, Optional.absent()); + LedgerMetadata metadata = LedgerMetadata.parseConfig(data, Optional.empty()); promise.complete(new Versioned<>(metadata, new LongVersion(kv.getModRevision()))); } catch (IOException ioe) { log.error("Could not parse ledger metadata for ledger : {}", ledgerId, ioe); diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index 4e214837761..da331655b83 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -37,6 +37,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -95,11 +96,16 @@ public void tearDown() throws Exception { @Test public void testLedgerCRUD() throws Exception { long ledgerId = System.currentTimeMillis(); - LedgerMetadata metadata = new LedgerMetadata( - 3, 3, 2, - DigestType.CRC32C, - "test-password".getBytes(UTF_8) - ); + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234)); + LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .withPassword("test-password".getBytes(UTF_8)) + .withDigestType(DigestType.CRC32C.toApiDigestType()) + .newEnsembleEntry(0L, ensemble) + .build(); // ledger doesn't exist: read try { From 940865d44b282e146bcde13a1027579096f0ee28 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 22 Nov 2018 11:15:35 +0000 Subject: [PATCH 0128/1642] Fix flake in AuditorPeriodicBookieCheckTest The flake was occurring because the bookie checker was trying to connect to a TCP endpoint that didn't respond (1.1.1.1) and was waiting for the whole tcp connect timeout (10sec), while the test was waiting 10 seconds for the success condition to occur. The success condition often occurred just after the test fails. The solution was to bring the connect timeout for the client down to 500ms, giving the auditor plenty of time to register the underreplicated ledger. I guess this used to pass in the past because maybe 1.1.1.1 was actually rejecting packets instead of dropping them, and these changed when cloudflare took over the address. I changed the endpoint to 192.0.2.0/24 range (TEST-NET) to avoid a similar scenario in the future. Reviewers: Enrico Olivelli , Sijie Guo This closes #1826 from ivankelly/auditor-bookie-flake --- .../bookkeeper/replication/AuditorPeriodicBookieCheckTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java index 06df18b7550..c503f12a1f8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java @@ -66,6 +66,7 @@ public void setUp() throws Exception { ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); conf.setAuditorPeriodicBookieCheckInterval(CHECK_INTERVAL); conf.setMetadataServiceUri(metadataServiceUri); + conf.setProperty("clientConnectTimeoutMillis", 500); String addr = bs.get(0).getLocalAddress().toString(); auditorElector = new AuditorElector(addr, conf); @@ -95,7 +96,7 @@ public void testPeriodicBookieCheckInterval() throws Exception { LedgerMetadataBuilder.create().withEnsembleSize(3) .withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList( - new BookieSocketAddress("1.1.1.1", 1000), + new BookieSocketAddress("192.0.2.1", 1000), getBookie(0), getBookie(1)))); long underReplicatedLedger = -1; for (int i = 0; i < 10; i++) { From e2960d776a24aa21e8a0a08b91d7ed58950342b4 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 22 Nov 2018 15:23:30 +0000 Subject: [PATCH 0129/1642] Remove LedgerMetadata constructors except one for builder LedgerMetadata shouldn't generally be created outside the client, except for testing. In the case where it is, the builder should be used to construct it. I've left the copy constructor for now, as I don't feel comfortable removing it until all fields are final, which will occur soon. Master issue: #281 Reviewers: Enrico Olivelli This closes #1827 from ivankelly/kill-constructors --- .../apache/bookkeeper/client/BookKeeper.java | 11 ++- .../bookkeeper/client/LedgerCreateOp.java | 48 +++++------ .../bookkeeper/client/LedgerMetadata.java | 52 ------------ .../bookkeeper/client/LedgerMetadataTest.java | 82 +++++++++---------- .../bookkeeper/client/MockLedgerHandle.java | 14 +++- .../ReadLastConfirmedAndEntryOpTest.java | 8 +- .../client/TestLedgerFragmentReplication.java | 22 ++--- .../client/TestWatchEnsembleChange.java | 11 ++- .../client/api/BookKeeperBuildersTest.java | 20 ++--- .../meta/LedgerManagerIteratorTest.java | 16 +++- .../replication/AuditorLedgerCheckerTest.java | 9 +- .../metadata/etcd/EtcdLedgerManagerTest.java | 21 +++-- 12 files changed, 150 insertions(+), 164 deletions(-) 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 14760a8879e..72edf51461a 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 @@ -32,6 +32,7 @@ import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; import java.net.URI; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -743,7 +744,8 @@ public void asyncCreateLedger(final int ensSize, final int writeQuorumSize, final DigestType digestType, final byte[] passwd, final CreateCallback cb, final Object ctx) { - asyncCreateLedger(ensSize, writeQuorumSize, writeQuorumSize, digestType, passwd, cb, ctx, null); + asyncCreateLedger(ensSize, writeQuorumSize, writeQuorumSize, + digestType, passwd, cb, ctx, Collections.emptyMap()); } /** @@ -832,7 +834,7 @@ public LedgerHandle createLedger(DigestType digestType, byte passwd[]) public LedgerHandle createLedger(int ensSize, int qSize, DigestType digestType, byte passwd[]) throws InterruptedException, BKException { - return createLedger(ensSize, qSize, qSize, digestType, passwd, null); + return createLedger(ensSize, qSize, qSize, digestType, passwd, Collections.emptyMap()); } /** @@ -852,7 +854,7 @@ public LedgerHandle createLedger(int ensSize, int qSize, public LedgerHandle createLedger(int ensSize, int writeQuorumSize, int ackQuorumSize, DigestType digestType, byte passwd[]) throws InterruptedException, BKException { - return createLedger(ensSize, writeQuorumSize, ackQuorumSize, digestType, passwd, null); + return createLedger(ensSize, writeQuorumSize, ackQuorumSize, digestType, passwd, Collections.emptyMap()); } /** @@ -906,7 +908,8 @@ public LedgerHandle createLedger(int ensSize, int writeQuorumSize, int ackQuorum public LedgerHandle createLedgerAdv(int ensSize, int writeQuorumSize, int ackQuorumSize, DigestType digestType, byte passwd[]) throws InterruptedException, BKException { - return createLedgerAdv(ensSize, writeQuorumSize, ackQuorumSize, digestType, passwd, null); + return createLedgerAdv(ensSize, writeQuorumSize, ackQuorumSize, + digestType, passwd, Collections.emptyMap()); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java index 16a6e9547c7..d95665c035e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java @@ -59,10 +59,14 @@ class LedgerCreateOp { static final Logger LOG = LoggerFactory.getLogger(LedgerCreateOp.class); final CreateCallback cb; - final LedgerMetadata metadata; + LedgerMetadata metadata; LedgerHandle lh; long ledgerId = -1L; final Object ctx; + final int ensembleSize; + final int writeQuorumSize; + final int ackQuorumSize; + final Map customMetadata; final byte[] passwd; final BookKeeper bk; final DigestType digestType; @@ -102,15 +106,11 @@ class LedgerCreateOp { EnumSet writeFlags, BookKeeperClientStats clientStats) { this.bk = bk; - this.metadata = new LedgerMetadata( - ensembleSize, - writeQuorumSize, - ackQuorumSize, - digestType, - passwd, - customMetadata, - bk.getConf().getStoreSystemtimeAsLedgerCreationTime()); + this.ensembleSize = ensembleSize; + this.writeQuorumSize = writeQuorumSize; + this.ackQuorumSize = ackQuorumSize; this.digestType = digestType; + this.customMetadata = customMetadata; this.writeFlags = writeFlags; this.passwd = passwd; this.cb = cb; @@ -124,29 +124,29 @@ class LedgerCreateOp { * Initiates the operation. */ public void initiate() { - // allocate ensemble first - - /* - * Adding bookies to ledger handle - */ + LedgerMetadataBuilder metadataBuilder = LedgerMetadataBuilder.create() + .withEnsembleSize(ensembleSize).withWriteQuorumSize(writeQuorumSize).withAckQuorumSize(ackQuorumSize) + .withDigestType(digestType.toApiDigestType()).withPassword(passwd); + if (customMetadata != null) { + metadataBuilder.withCustomMetadata(customMetadata); + } + if (bk.getConf().getStoreSystemtimeAsLedgerCreationTime()) { + metadataBuilder.withCreationTime(System.currentTimeMillis()); + } - List ensemble; + // select bookies for first ensemble try { - ensemble = bk.getBookieWatcher() - .newEnsemble(metadata.getEnsembleSize(), - metadata.getWriteQuorumSize(), - metadata.getAckQuorumSize(), - metadata.getCustomMetadata()); + List ensemble = bk.getBookieWatcher() + .newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata); + metadataBuilder.newEnsembleEntry(0L, ensemble); } catch (BKNotEnoughBookiesException e) { LOG.error("Not enough bookies to create ledger"); createComplete(e.getCode(), null); return; } - /* - * Add ensemble to the configuration - */ - metadata.addEnsemble(0L, ensemble); + + this.metadata = metadataBuilder.build(); if (this.generateLedgerId) { generateLedgerIdAndCreateLedger(); } else { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 90f0f136eab..5fd5f06eeed 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -91,44 +91,6 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe private Map customMetadata = Maps.newHashMap(); - public LedgerMetadata(int ensembleSize, - int writeQuorumSize, - int ackQuorumSize, - BookKeeper.DigestType digestType, - byte[] password, - Map customMetadata, - boolean storeSystemtimeAsLedgerCreationTime) { - this.ensembleSize = ensembleSize; - this.writeQuorumSize = writeQuorumSize; - this.ackQuorumSize = ackQuorumSize; - if (storeSystemtimeAsLedgerCreationTime) { - this.ctime = System.currentTimeMillis(); - this.storeCtime = storeSystemtimeAsLedgerCreationTime; - } else { - // if client disables storing its system time as ledger creation time, there should be no ctime at this - // moment. - this.ctime = -1L; - this.storeCtime = false; - } - - /* - * It is set in PendingReadOp.readEntryComplete, and - * we read it in LedgerRecoveryOp.readComplete. - */ - this.length = 0; - this.state = LedgerMetadataFormat.State.OPEN; - this.lastEntryId = LedgerHandle.INVALID_ENTRY_ID; - this.metadataFormatVersion = CURRENT_METADATA_FORMAT_VERSION; - - this.digestType = digestType.equals(BookKeeper.DigestType.MAC) - ? LedgerMetadataFormat.DigestType.HMAC : LedgerMetadataFormat.DigestType.valueOf(digestType.toString()); - this.password = Arrays.copyOf(password, password.length); - this.hasPassword = true; - if (customMetadata != null) { - this.customMetadata = customMetadata; - } - } - LedgerMetadata(int metadataFormatVersion, int ensembleSize, int writeQuorumSize, @@ -175,15 +137,6 @@ public LedgerMetadata(int ensembleSize, this.customMetadata.putAll(customMetadata); } - /** - * Used for testing purpose only. - */ - @VisibleForTesting - public LedgerMetadata(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - BookKeeper.DigestType digestType, byte[] password) { - this(ensembleSize, writeQuorumSize, ackQuorumSize, digestType, password, null, false); - } - /** * Copy Constructor. */ @@ -208,11 +161,6 @@ public LedgerMetadata(int ensembleSize, int writeQuorumSize, int ackQuorumSize, this.customMetadata = other.customMetadata; } - private LedgerMetadata() { - this(0, 0, 0, BookKeeper.DigestType.MAC, new byte[] {}); - this.hasPassword = false; - } - @Override public NavigableMap> getAllEnsembles() { return ensembles; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java index 5f802cb6049..38e5260c8ab 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java @@ -23,11 +23,12 @@ 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 com.google.common.collect.Lists; import java.util.Collections; -import java.util.NoSuchElementException; +import java.util.List; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; import org.junit.Test; @@ -41,14 +42,15 @@ public class LedgerMetadataTest { @Test public void testGetters() { - org.apache.bookkeeper.client.api.LedgerMetadata metadata = new LedgerMetadata( - 3, - 2, - 1, - DigestType.CRC32, - passwd, - Collections.emptyMap(), - false); + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234)); + org.apache.bookkeeper.client.api.LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) + .withDigestType(DigestType.CRC32.toApiDigestType()).withPassword(passwd) + .newEnsembleEntry(0L, ensemble) + .build(); assertEquals(3, metadata.getEnsembleSize()); assertEquals(2, metadata.getWriteQuorumSize()); @@ -59,27 +61,23 @@ public void testGetters() { assertEquals(-1L, metadata.getLastEntryId()); assertEquals(0, metadata.getLength()); assertFalse(metadata.isClosed()); - assertTrue(metadata.getAllEnsembles().isEmpty()); - - try { - metadata.getEnsembleAt(99L); - fail("Should fail to retrieve ensemble if ensembles is empty"); - } catch (NoSuchElementException e) { - // expected - } + assertEquals(1, metadata.getAllEnsembles().size()); + assertEquals(ensemble, metadata.getAllEnsembles().get(0L)); + assertEquals(ensemble, metadata.getEnsembleAt(99L)); } @Test public void testStoreSystemtimeAsLedgerCtimeEnabled() throws Exception { - LedgerMetadata lm = new LedgerMetadata( - 3, - 3, - 2, - DigestType.CRC32, - passwd, - Collections.emptyMap(), - true); + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234)); + LedgerMetadata lm = LedgerMetadataBuilder.create() + .newEnsembleEntry(0L, ensemble) + .withCreationTime(System.currentTimeMillis()) + .storingCreationTime(true) + .build(); LedgerMetadataFormat format = lm.buildProtoFormat(); assertTrue(format.hasCtime()); } @@ -87,28 +85,28 @@ public void testStoreSystemtimeAsLedgerCtimeEnabled() @Test public void testStoreSystemtimeAsLedgerCtimeDisabled() throws Exception { - LedgerMetadata lm = new LedgerMetadata( - 3, - 3, - 2, - DigestType.CRC32, - passwd, - Collections.emptyMap(), - false); + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234)); + LedgerMetadata lm = LedgerMetadataBuilder.create() + .newEnsembleEntry(0L, ensemble).build(); + LedgerMetadataFormat format = lm.buildProtoFormat(); assertFalse(format.hasCtime()); } @Test public void testToString() { - LedgerMetadata lm1 = new LedgerMetadata( - 3, - 3, - 2, - DigestType.CRC32, - passwd, - Collections.emptyMap(), - true); + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234)); + + LedgerMetadata lm1 = LedgerMetadataBuilder.create() + .withDigestType(DigestType.CRC32.toApiDigestType()) + .withPassword(passwd) + .newEnsembleEntry(0L, ensemble).build(); assertTrue("toString should contain 'password' field", lm1.toString().contains("password")); assertTrue("toString should contain password value", lm1.toString().contains(passwdStr)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java index c2235c25fcc..b7cb9e14a34 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java @@ -28,6 +28,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Enumeration; +import java.util.List; import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.RejectedExecutionException; @@ -41,6 +42,7 @@ import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; @@ -62,7 +64,7 @@ public class MockLedgerHandle extends LedgerHandle { MockLedgerHandle(MockBookKeeper bk, long id, DigestType digest, byte[] passwd) throws GeneralSecurityException { super(bk.getClientCtx(), id, - new Versioned<>(new LedgerMetadata(3, 3, 2, DigestType.MAC, "".getBytes()), new LongVersion(0L)), + new Versioned<>(createMetadata(), new LongVersion(0L)), DigestType.MAC, "".getBytes(), WriteFlag.NONE); this.bk = bk; this.id = id; @@ -265,6 +267,16 @@ public CompletableFuture readLastAddConfirmedAndEntryAsyn return readHandle.readLastAddConfirmedAndEntryAsync(entryId, timeOutInMillis, parallel); } + private static LedgerMetadata createMetadata() { + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234)); + return LedgerMetadataBuilder.create() + .newEnsembleEntry(0L, ensemble) + .build(); + } + private static final Logger log = LoggerFactory.getLogger(MockLedgerHandle.class); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java index 3d9c3943b3f..252c1b9d589 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java @@ -99,13 +99,15 @@ public void setup() throws Exception { internalConf = ClientInternalConf.fromConfig(conf); // metadata - this.ledgerMetadata = - new LedgerMetadata(3, 3, 2, DigestType.CRC32, new byte[0]); ArrayList ensemble = new ArrayList<>(3); for (int i = 0; i < 3; i++) { ensemble.add(new BookieSocketAddress("127.0.0.1", 3181 + i)); } - this.ledgerMetadata.addEnsemble(0L, ensemble); + this.ledgerMetadata = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(2) + .withDigestType(DigestType.CRC32.toApiDigestType()) + .withPassword(new byte[0]) + .newEnsembleEntry(0L, ensemble).build(); this.distributionSchedule = new RoundRobinDistributionSchedule(3, 2, 3); // schedulers this.scheduler = Executors.newSingleThreadScheduledExecutor(); 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 f407c3571e3..3503e91aa39 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 @@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.collect.Lists; import com.google.common.collect.Sets; import java.util.Enumeration; @@ -234,18 +235,17 @@ public void testReplicateLFShouldReturnFalseIfTheReplicationFails() @Test public void testSplitIntoSubFragmentsWithDifferentFragmentBoundaries() throws Exception { - LedgerMetadata metadata = new LedgerMetadata(3, 3, 3, TEST_DIGEST_TYPE, - TEST_PSSWD) { - @Override - List getEnsemble(long entryId) { - return null; - } + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234)); + LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .withPassword(TEST_PSSWD).withDigestType(TEST_DIGEST_TYPE.toApiDigestType()) + .closingAt(-1, 0) + .newEnsembleEntry(0L, ensemble) + .build(); - @Override - public boolean isClosed() { - return true; - } - }; LedgerHandle lh = new LedgerHandle(bkc.getClientCtx(), 0, new Versioned<>(metadata, new LongVersion(0L)), TEST_DIGEST_TYPE, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java index 38411ac9b3a..25906662d70 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Lists; import com.google.common.util.concurrent.UncheckedExecutionException; import java.nio.ByteBuffer; import java.util.Arrays; @@ -130,10 +131,18 @@ private void testWatchMetadataRemoval(LedgerManagerFactory factory) throws Excep final CountDownLatch createLatch = new CountDownLatch(1); final CountDownLatch removeLatch = new CountDownLatch(1); + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234), + new BookieSocketAddress("192.0.2.4", 1234)); idGenerator.generateLedgerId(new GenericCallback() { @Override public void operationComplete(int rc, final Long lid) { - LedgerMetadata metadata = new LedgerMetadata(4, 2, 2, digestType, "fpj was here".getBytes()); + LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(4).withWriteQuorumSize(2) + .withAckQuorumSize(2) + .newEnsembleEntry(0L, ensemble).build(); manager.createLedgerMetadata(lid, metadata) .whenComplete((result, exception) -> { bbLedgerId.putLong(lid); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java index 58b75ba87a0..f593642778c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.client.MockBookKeeperTestCase; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.proto.BookieProtocol; @@ -405,16 +406,15 @@ public void testDeleteLedgerBookKeeperClosed() throws Exception { protected LedgerMetadata generateLedgerMetadata(int ensembleSize, int writeQuorumSize, int ackQuorumSize, byte[] password, Map customMetadata) { - LedgerMetadata ledgerMetadata = new LedgerMetadata( - ensembleSize, - writeQuorumSize, - ackQuorumSize, - BookKeeper.DigestType.CRC32, - password, - customMetadata, - true); - ledgerMetadata.addEnsemble(0, generateNewEnsemble(ensembleSize)); - return ledgerMetadata; + return LedgerMetadataBuilder.create() + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .withDigestType(BookKeeper.DigestType.CRC32.toApiDigestType()) + .withPassword(password) + .withCustomMetadata(customMetadata) + .withCreationTime(System.currentTimeMillis()) + .newEnsembleEntry(0, generateNewEnsemble(ensembleSize)).build(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java index 90fb2ee3a71..73f8d39e10c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java @@ -26,6 +26,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Lists; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -46,8 +47,10 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.Version; @@ -84,9 +87,16 @@ void removeLedger(LedgerManager lm, Long ledgerId) throws Exception { * @throws InterruptedException */ void createLedger(LedgerManager lm, Long ledgerId) throws Exception { - LedgerMetadata meta = new LedgerMetadata( - 3, 3, 2, - BookKeeper.DigestType.CRC32, "passwd".getBytes()); + List ensemble = Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234)); + LedgerMetadata meta = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .withDigestType(BookKeeper.DigestType.CRC32.toApiDigestType()) + .withPassword("passwd".getBytes()) + .newEnsembleEntry(0L, ensemble) + .build(); lm.createLedgerMetadata(ledgerId, meta).get(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index 83d1dc0da45..9db91887a63 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -52,6 +52,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; @@ -545,12 +546,16 @@ public void testTriggerAuditorWithNoPendingAuditTask() throws Exception { int numofledgers = 5; Random rand = new Random(); for (int i = 0; i < numofledgers; i++) { - LedgerMetadata metadata = new LedgerMetadata(3, 2, 2, DigestType.CRC32, "passwd".getBytes()); ArrayList ensemble = new ArrayList(); ensemble.add(new BookieSocketAddress("99.99.99.99:9999")); ensemble.add(new BookieSocketAddress("11.11.11.11:1111")); ensemble.add(new BookieSocketAddress("88.88.88.88:8888")); - metadata.addEnsemble(0, ensemble); + + LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(2) + .withDigestType(DigestType.CRC32.toApiDigestType()) + .withPassword("passwd".getBytes()) + .newEnsembleEntry(0L, ensemble).build(); long ledgerId = (Math.abs(rand.nextLong())) % 100000000; diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index da331655b83..95da4ffe774 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -239,11 +239,11 @@ public void testLedgerRangeIterator() throws Exception { private void createNumLedgers(int numLedgers) throws Exception { List>> createFutures = new ArrayList<>(numLedgers); for (int i = 0; i < numLedgers; i++) { - LedgerMetadata metadata = new LedgerMetadata( - 3, 3, 2, - DigestType.CRC32C, - "test-password".getBytes(UTF_8) - ); + LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .withDigestType(DigestType.CRC32C.toApiDigestType()) + .withPassword("test-password".getBytes(UTF_8)) + .newEnsembleEntry(0L, createNumBookies(3)).build(); createFutures.add(lm.createLedgerMetadata(i, metadata)); } FutureUtils.result(FutureUtils.collect(createFutures)); @@ -254,12 +254,11 @@ public void testRegisterLedgerMetadataListener() throws Exception { long ledgerId = System.currentTimeMillis(); // create a ledger metadata - LedgerMetadata metadata = new LedgerMetadata( - 3, 3, 2, - DigestType.CRC32C, - "test-password".getBytes(UTF_8) - ); - metadata.addEnsemble(0L, createNumBookies(3)); + LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .withDigestType(DigestType.CRC32C.toApiDigestType()) + .withPassword("test-password".getBytes(UTF_8)) + .newEnsembleEntry(0L, createNumBookies(3)).build(); result(lm.createLedgerMetadata(ledgerId, metadata)); Versioned readMetadata = lm.readLedgerMetadata(ledgerId).get(); log.info("Create ledger metadata : {}", readMetadata.getValue()); From 02e1130d03d8d796c3d95b047f38aae5994e4aa3 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 22 Nov 2018 17:45:48 +0000 Subject: [PATCH 0130/1642] Make most LedgerMetadata fields final Make most fields in LedgerMetadata final. The ones which I have not changed in this patch require a more involved change, so they'll come in separate PRs. Master issue: #281 Reviewers: Enrico Olivelli This closes #1828 from ivankelly/mostly-final --- .../bookkeeper/client/LedgerMetadata.java | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 5fd5f06eeed..026cc973cfb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -72,22 +72,22 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe public static final int CURRENT_METADATA_FORMAT_VERSION = 2; public static final String VERSION_KEY = "BookieMetadataFormatVersion"; - private int metadataFormatVersion = 0; - private int ensembleSize; - private int writeQuorumSize; - private int ackQuorumSize; + private final int metadataFormatVersion; + private final int ensembleSize; + private final int writeQuorumSize; + private final int ackQuorumSize; private long length; private long lastEntryId; - private long ctime; - boolean storeCtime; // non-private so builder can access for copy + private final long ctime; + final boolean storeCtime; // non-private so builder can access for copy private LedgerMetadataFormat.State state; private TreeMap> ensembles = new TreeMap<>(); private List currentEnsemble; - private boolean hasPassword = false; - private LedgerMetadataFormat.DigestType digestType; - private byte[] password; + private final boolean hasPassword; // IKTODO other things should be optionals instead + private final LedgerMetadataFormat.DigestType digestType; + private final byte[] password; private Map customMetadata = Maps.newHashMap(); @@ -126,11 +126,13 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe this.digestType = digestType.equals(DigestType.MAC) ? LedgerMetadataFormat.DigestType.HMAC : LedgerMetadataFormat.DigestType.valueOf(digestType.toString()); - password.ifPresent((pw) -> { - this.password = pw; - this.hasPassword = true; - }); - + if (password.isPresent()) { + this.password = password.get(); + this.hasPassword = true; + } else { + this.password = null; + this.hasPassword = false; + } this.ctime = ctime; this.storeCtime = storeCtime; From 557e40fc289befe4ccbcd9c9204a180d08cf3c5b Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 22 Nov 2018 11:18:42 -0800 Subject: [PATCH 0131/1642] Remove duplication logic for 'minNumRacksPerWriteQuorum' handling in RackawareEnsemblePlacementPolicyImpl Descriptions of the changes in this PR: With this change https://github.com/apache/bookkeeper/commit/9ba4c4e0d8be770e03110a958fb8b75a65ae0f59 we introduced 'minNumRacksPerWriteQuorum' config option. To handle that, logic is added in RackawareEnsemblePlacementPolicyImpl.newEnsembleInternal and RackQuorumCoverageSet. But in retrospect, this is kind of duplicate logic and changes made in RackawareEnsemblePlacementPolicyImpl.newEnsembleInternal is not really needed, because anyhow 'apply' of predicate (RRTopologyAwareCoverageEnsemble) is called, which calls 'apply' method on all the concerned RackQuorumCoverageSet. Reviewers: Sijie Guo This closes #1810 from reddycharan/revertrackawarechanges --- .../RackawareEnsemblePlacementPolicyImpl.java | 59 +------------------ .../TestRackawareEnsemblePlacementPolicy.java | 10 ++-- 2 files changed, 8 insertions(+), 61 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 1d587d07693..9c0feb7ee37 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 @@ -554,9 +554,7 @@ protected List newEnsembleInternal( } return addrs; } - // pick nodes by racks, to ensure there is at least write quorum number of racks. - int idx = 0; - String[] racks = new String[ensembleSize]; + for (int i = 0; i < ensembleSize; i++) { String curRack; if (null == prevNode) { @@ -566,64 +564,11 @@ protected List newEnsembleInternal( curRack = localNode.getNetworkLocation(); } } else { - StringBuilder sb = new StringBuilder(); - sb.append("~"); - - if (writeQuorumSize > 1) { - /* - * RackAwareEnsemblePlacementPolicy should try to select - * bookies from atleast - * minNumRacksPerWriteQuorumForThisEnsemble number of - * different racks for a write quorum. So in a - * WriteQuorum, bookies should be from - * minNumRacksPerWriteQuorumForThisEnsemble number of - * racks. So we would add racks of - * (minNumRacksPerWriteQuorumForThisEnsemble-1) - * neighbours (both sides) to the exclusion list - * (~curRack). - */ - for (int j = 1; j < minNumRacksPerWriteQuorumForThisEnsemble; j++) { - int nextIndex = i + j; - if (nextIndex >= ensembleSize) { - nextIndex %= ensembleSize; - } - /* - * if racks[nextIndex] is null, then it means bookie - * is not yet selected for ensemble at 'nextIndex' - * index. - */ - if (racks[nextIndex] != null) { - if (!((sb.length() == 1) && (sb.charAt(0) == '~'))) { - sb.append(NetworkTopologyImpl.NODE_SEPARATOR); - } - sb.append(racks[nextIndex]); - } - } - - for (int j = 1; j < minNumRacksPerWriteQuorumForThisEnsemble; j++) { - int nextIndex = i - j; - if (nextIndex < 0) { - nextIndex += ensembleSize; - } - /* - * if racks[nextIndex] is null, then it means bookie - * is not yet selected for ensemble at 'nextIndex' - * index. - */ - if (racks[nextIndex] != null) { - if (!((sb.length() == 1) && (sb.charAt(0) == '~'))) { - sb.append(NetworkTopologyImpl.NODE_SEPARATOR); - } - sb.append(racks[nextIndex]); - } - } - } - curRack = sb.toString(); + curRack = "~" + prevNode.getNetworkLocation(); } boolean firstBookieInTheEnsemble = (null == prevNode); prevNode = selectFromNetworkLocation(curRack, excludeNodes, ensemble, ensemble, !enforceMinNumRacksPerWriteQuorum || firstBookieInTheEnsemble); - racks[i] = prevNode.getNetworkLocation(); } List bookieList = ensemble.toList(); if (ensembleSize != bookieList.size()) { 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 dd95b7a388d..d80104456cd 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 @@ -1200,12 +1200,12 @@ public void testNewEnsembleWithMultipleRacks() throws Exception { int acqQuorumSize = 2; List ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, null, new HashSet<>()); - int numCovered = getNumCoveredWriteQuorums(ensemble, 2, conf.getMinNumRacksPerWriteQuorum()); + int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); ensembleSize = 4; List ensemble2 = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, null, new HashSet<>()); - numCovered = getNumCoveredWriteQuorums(ensemble2, 2, conf.getMinNumRacksPerWriteQuorum()); + numCovered = getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -1319,12 +1319,14 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { int ackQuorumSize = 2; List ensemble1 = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); - assertEquals(ensembleSize, getNumCoveredWriteQuorums(ensemble1, 2, conf.getMinNumRacksPerWriteQuorum())); + assertEquals(ensembleSize, + getNumCoveredWriteQuorums(ensemble1, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); ensembleSize = 4; writeQuorumSize = 4; List ensemble2 = repp.newEnsemble(ensembleSize, writeQuorumSize, 2, null, new HashSet<>()); - assertEquals(ensembleSize, getNumCoveredWriteQuorums(ensemble2, 2, conf.getMinNumRacksPerWriteQuorum())); + assertEquals(ensembleSize, + getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); } From 14ae95f235faffd76d3365233ec90ea62e2aacc0 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 22 Nov 2018 19:20:42 +0000 Subject: [PATCH 0132/1642] Move utility method from LedgerMetadata to test The method is only used in the test, so it should live there. Master issue: #281 Reviewers: Sijie Guo , Enrico Olivelli This closes #1829 from ivankelly/move-junk --- .../bookkeeper/client/LedgerMetadata.java | 32 ----------------- .../client/BookieWriteLedgerTest.java | 35 ++++++++++++++++++- 2 files changed, 34 insertions(+), 33 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 026cc973cfb..7b40e734651 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -553,38 +553,6 @@ static LedgerMetadata parseVersion1Config(BufferedReader reader) throws IOExcept } } - /** - * Routine to compare two {@code Map}; Since the values in the map are {@code byte[]}, we can't use - * {@code Map.equals}. - * @param first - * The first map - * @param second - * The second map to compare with - * @return true if the 2 maps contain the exact set of {@code } pairs. - */ - public static boolean areByteArrayValMapsEqual(Map first, Map second) { - if (first == null && second == null) { - return true; - } - - // above check confirms that both are not null; - // if one is null the other isn't; so they must - // be different - if (first == null || second == null) { - return false; - } - - if (first.size() != second.size()) { - return false; - } - for (Map.Entry entry : first.entrySet()) { - if (!Arrays.equals(entry.getValue(), second.get(entry.getKey()))) { - return false; - } - } - return true; - } - @Override public String toString() { return toStringRepresentation(true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java index 52360faa2d6..a3fe0706f8e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java @@ -39,6 +39,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Enumeration; import java.util.HashMap; import java.util.List; @@ -588,12 +589,44 @@ public void testLedgerCreateWithCustomMetadata() throws Exception { lh = bkc.openLedger(ledgerId, digestType, ledgerPassword); Map outputCustomMetadataMap = lh.getCustomMetadata(); assertTrue("Can't retrieve proper Custom Data", - LedgerMetadata.areByteArrayValMapsEqual(inputCustomMetadataMap, outputCustomMetadataMap)); + areByteArrayValMapsEqual(inputCustomMetadataMap, outputCustomMetadataMap)); lh.close(); bkc.deleteLedger(ledgerId); } } + /** + * Routine to compare two {@code Map}; Since the values in the map are {@code byte[]}, we can't use + * {@code Map.equals}. + * @param first + * The first map + * @param second + * The second map to compare with + * @return true if the 2 maps contain the exact set of {@code } pairs. + */ + public static boolean areByteArrayValMapsEqual(Map first, Map second) { + if (first == null && second == null) { + return true; + } + + // above check confirms that both are not null; + // if one is null the other isn't; so they must + // be different + if (first == null || second == null) { + return false; + } + + if (first.size() != second.size()) { + return false; + } + for (Map.Entry entry : first.entrySet()) { + if (!Arrays.equals(entry.getValue(), second.get(entry.getKey()))) { + return false; + } + } + return true; + } + /* * Verify the functionality of Advanced Ledger which accepts ledgerId as * input and returns LedgerHandleAdv. LedgerHandleAdv takes entryId for From 175185540a0fd21eb2e82208e8b126d624042150 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 22 Nov 2018 16:03:15 -0800 Subject: [PATCH 0133/1642] [SCRIPTS] improve bookkeeper script to better handle standalone command Descriptions of the changes in this PR: *Motivation* `bin/bookkeeper standalone` is broken when a release package is built without including `stream` profile. *Changes* `standalone` is a service component including everything. so fail the command if it can't locate corresponding components. Related Issues: #1822 Reviewers: Ivan Kelly , Enrico Olivelli This closes #1823 from sijie/refactor_bookkeeper_scripts --- bin/bookkeeper | 34 ++++++++++++-------- bin/common.sh | 86 ++++++++++++++++++++++++++++++++++++++++++++++++-- 2 files changed, 104 insertions(+), 16 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index 70b6ccd1444..ddaae2286e8 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -35,21 +35,17 @@ else BOOKIE_CONF_TO_CHECK=${BOOKIE_CONF} fi -# check the configuration to see if table service is enabled or not. -if [ -z "${ENABLE_TABLE_SERVICE}" ]; then - # mask exit code if the configuration file doesn't contain `StreamStorageLifecycleComponent` - TABLE_SERVICE_SETTING=$(grep StreamStorageLifecycleComponent ${BOOKIE_CONF_TO_CHECK} | cat) - if [[ "${TABLE_SERVICE_SETTING}" =~ ^extraServerComponents.* ]]; then - ENABLE_TABLE_SERVICE="true" - fi -fi +FIND_TABLE_SERVICE_RESULT=$(find_table_service ${BOOKIE_CONF_TO_CHECK} $1) -if [ \( "x$1" == "xstandalone" \) -o \( "x${ENABLE_TABLE_SERVICE}" == "xtrue" \) ]; then +if [ "x${FIND_TABLE_SERVICE_RESULT}" == "xtrue" ]; then BOOKIE_MODULE_PATH=stream/server - BOOKIE_MODULE_NAME="(org.apache.bookkeeper-)?stream-storage-server" -else + BOOKIE_MODULE_NAME=${TABLE_SERVICE_MODULE_NAME} +elif [ "x${FIND_TABLE_SERVICE_RESULT}" == "xfalse" ]; then BOOKIE_MODULE_PATH=bookkeeper-server - BOOKIE_MODULE_NAME="(org.apache.bookkeeper-)?bookkeeper-server" + BOOKIE_MODULE_NAME=${BOOKIE_SERVER_MODULE_NAME} +else + echo ${FIND_TABLE_SERVICE_RESULT} + exit 1 fi # find the module jar @@ -62,13 +58,25 @@ bookkeeper_help() { cat < where command is one of: + +[service commands] + bookie Run a bookie server autorecovery Run AutoRecovery service daemon + zookeeper Run zookeeper server + +[development commands] + localbookie Run a test ensemble of bookies locally standalone Run a standalone cluster (with all service components) locally + +[tooling commands] + upgrade Upgrade bookie filesystem shell Run shell for admin commands - zookeeper Run zookeeper server + +[other commands] + help This help message or command is the full name of a class with a defined main() method. diff --git a/bin/common.sh b/bin/common.sh index 73de28f7b8a..f92c0b9629b 100755 --- a/bin/common.sh +++ b/bin/common.sh @@ -114,6 +114,20 @@ CLI_MEM_OPTS=${CLI_MEM_OPTS:-"-Xms${CLI_MIN_HEAP_MEMORY} -Xmx${CLI_MAX_HEAP_MEMO CLI_GC_OPTS=${CLI_GC_OPTS:-"${DEFAULT_CLI_GC_OPTS}"} CLI_GC_LOGGING_OPTS=${CLI_GC_LOGGING_OPTS:-"${DEFAULT_CLI_GC_LOGGING_OPTS}"} +# module names +BOOKIE_SERVER_MODULE_NAME="(org.apache.bookkeeper-)?bookkeeper-server" +TABLE_SERVICE_MODULE_NAME="(org.apache.bookkeeper-)?stream-storage-server" + +is_released_binary() { + if [ -d ${BK_HOME}/lib ]; then + echo "true" + return + else + echo "false" + return + fi +} + find_module_jar_at() { DIR=$1 MODULE=$2 @@ -129,6 +143,21 @@ find_module_jar_at() { fi } +find_module_release_jar() { + MODULE_NAME=$1 + RELEASE_JAR=$(find_module_jar_at ${BK_HOME} ${MODULE_NAME}) + if [ -n "${RELEASE_JAR}" ]; then + MODULE_JAR=${RELEASE_JAR} + else + RELEASE_JAR=$(find_module_jar_at ${BK_HOME}/lib ${MODULE_NAME}) + if [ -n "${RELEASE_JAR}" ]; then + MODULE_JAR=${RELEASE_JAR} + fi + fi + echo ${RELEASE_JAR} + return +} + find_module_jar() { MODULE_PATH=$1 MODULE_NAME=$2 @@ -146,10 +175,13 @@ find_module_jar() { BUILT_JAR=$(find_module_jar_at ${BK_HOME}/${MODULE_PATH}/target ${MODULE_NAME}) if [ -z "${BUILT_JAR}" ]; then echo "Couldn't find module '${MODULE_NAME}' jar." >&2 - read -p "Do you want me to run \`mvn package -DskiptTests\` for you ? " answer + read -p "Do you want me to run \`mvn package -DskipTests -Dstream\` for you ? (y|n) " answer case "${answer:0:1}" in y|Y ) - mvn package -DskipTests -Dstream + mkdir -p ${BK_HOME}/logs + output="${BK_HOME}/logs/build.out" + echo "see output at ${output} for the progress ..." >&2 + mvn package -DskipTests -Dstream &> ${output} ;; * ) exit 1 @@ -182,8 +214,12 @@ add_maven_deps_to_classpath() { # and cache it. Save the file into our target dir so a mvn clean will get # clean it up and force us create a new one. f="${BK_HOME}/${MODULE_PATH}/target/cached_classpath.txt" + output="${BK_HOME}/${MODULE_PATH}/target/build_classpath.out" if [ ! -f ${f} ]; then - ${MVN} -f "${BK_HOME}/${MODULE_PATH}/pom.xml" -Dstream dependency:build-classpath -Dmdep.outputFile="target/cached_classpath.txt" &> /dev/null + echo "the classpath of module '${MODULE_PATH}' is not found, generating it ..." >&2 + echo "see output at ${output} for the progress ..." >&2 + ${MVN} -f "${BK_HOME}/${MODULE_PATH}/pom.xml" -Dstream dependency:build-classpath -Dmdep.outputFile="target/cached_classpath.txt" &> ${output} + echo "the classpath of module '${MODULE_PATH}' is generated at '${f}'." >&2 fi } @@ -249,3 +285,47 @@ build_cli_logging_opts() { build_bookie_opts() { echo "-Djava.net.preferIPv4Stack=true" } + +find_table_service() { + BOOKIE_CONF_TO_CHECK=$1 + SERVICE_COMMAND=$2 + + # check if it is a released binary + IS_RELEASED_BINARY=$(is_released_binary) + + # check if table service is released + TABLE_SERVICE_RELEASED="true" + if [ "x${IS_RELEASED_BINARY}" == "xtrue" ]; then + TABLE_SERVICE_RELEASE_JAR=$(find_module_release_jar ${TABLE_SERVICE_MODULE_NAME}) + if [ "x${TABLE_SERVICE_RELEASE_JAR}" == "x" ]; then + TABLE_SERVICE_RELEASED="false" + fi + fi + + # check the configuration to see if table service is enabled or not. + if [ -z "${ENABLE_TABLE_SERVICE}" ]; then + # mask exit code if the configuration file doesn't contain `StreamStorageLifecycleComponent` + TABLE_SERVICE_SETTING=$(grep StreamStorageLifecycleComponent ${BOOKIE_CONF_TO_CHECK} | cat) + if [[ "${TABLE_SERVICE_SETTING}" =~ ^extraServerComponents.* ]]; then + if [ "x${TABLE_SERVICE_RELEASED}" == "xfalse" ]; then + echo "The release binary is built without table service. Please disable \`StreamStorageLifecycleComponent\` in your bookie configuration at '${BOOKIE_CONF_TO_CHECK}'." + return + fi + ENABLE_TABLE_SERVICE="true" + fi + fi + + # standalone only run + if [ \( "x${SERVICE_COMMAND}" == "xstandalone" \) -a \( "x${TABLE_SERVICE_RELEASED}" == "xfalse" \) ]; then + echo "The release binary is built without table service. Use \`localbookie \` instead of \`standalone\` for local development." + return + fi + + if [ \( "x${SERVICE_COMMAND}" == "xstandalone" \) -o \( "x${ENABLE_TABLE_SERVICE}" == "xtrue" \) ]; then + echo "true" + return + else + echo "false" + return + fi +} From 4d0bc5250f800ecfd38d95bd6ebd7830c4b5b37e Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 23 Nov 2018 00:04:44 +0000 Subject: [PATCH 0134/1642] Remove LedgerMetadata methods modifying ensembles Remove any methods in LedgerMetadata that modify the ensemble. With this change the ensembles are 100% immutable. Master issue: #281 Reviewers: Enrico Olivelli , Sijie Guo This closes #1830 from ivankelly/no-mod-ensemble --- .../bookkeeper/client/LedgerMetadata.java | 68 ++++++------------- 1 file changed, 22 insertions(+), 46 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 7b40e734651..bf5e9f5df6d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -32,6 +32,7 @@ import java.nio.CharBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -76,14 +77,15 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe private final int ensembleSize; private final int writeQuorumSize; private final int ackQuorumSize; + + private LedgerMetadataFormat.State state; private long length; private long lastEntryId; private final long ctime; final boolean storeCtime; // non-private so builder can access for copy - private LedgerMetadataFormat.State state; - private TreeMap> ensembles = new TreeMap<>(); - private List currentEnsemble; + private final NavigableMap> ensembles; + private final ImmutableList currentEnsemble; private final boolean hasPassword; // IKTODO other things should be optionals instead private final LedgerMetadataFormat.DigestType digestType; @@ -117,10 +119,17 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe this.lastEntryId = LedgerHandle.INVALID_ENTRY_ID; } length.ifPresent((l) -> this.length = l); - setEnsembles(ensembles); + + this.ensembles = Collections.unmodifiableNavigableMap( + ensembles.entrySet().stream().collect(TreeMap::new, + (m, e) -> m.put(e.getKey(), + ImmutableList.copyOf(e.getValue())), + TreeMap::putAll)); if (state != LedgerMetadataFormat.State.CLOSED) { currentEnsemble = this.ensembles.lastEntry().getValue(); + } else { + currentEnsemble = null; } this.digestType = digestType.equals(DigestType.MAC) @@ -156,9 +165,15 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe this.storeCtime = other.storeCtime; this.password = new byte[other.password.length]; System.arraycopy(other.password, 0, this.password, 0, other.password.length); - // copy the ensembles - for (Entry> entry : other.ensembles.entrySet()) { - this.addEnsemble(entry.getKey(), entry.getValue()); + this.ensembles = Collections.unmodifiableNavigableMap( + other.ensembles.entrySet().stream().collect(TreeMap::new, + (m, e) -> m.put(e.getKey(), + ImmutableList.copyOf(e.getValue())), + TreeMap::putAll)); + if (state != LedgerMetadataFormat.State.CLOSED) { + currentEnsemble = this.ensembles.lastEntry().getValue(); + } else { + currentEnsemble = null; } this.customMetadata = other.customMetadata; } @@ -168,13 +183,6 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe return ensembles; } - void setEnsembles(Map> newEnsembles) { - this.ensembles = newEnsembles.entrySet().stream() - .collect(TreeMap::new, - (m, e) -> m.put(e.getKey(), ImmutableList.copyOf(e.getValue())), - TreeMap::putAll); - } - @Override public int getEnsembleSize() { return ensembleSize; @@ -270,22 +278,10 @@ void close(long entryId) { state = LedgerMetadataFormat.State.CLOSED; } - public void addEnsemble(long startEntryId, List ensemble) { - checkArgument(ensembles.isEmpty() || startEntryId >= ensembles.lastKey()); - - ensembles.put(startEntryId, ImmutableList.copyOf(ensemble)); - currentEnsemble = ensemble; - } - List getCurrentEnsemble() { return currentEnsemble; } - public void updateEnsemble(long startEntryId, List ensemble) { - checkArgument(ensembles.containsKey(startEntryId)); - ensembles.put(startEntryId, ImmutableList.copyOf(ensemble)); - } - List getEnsemble(long entryId) { // the head map cannot be empty, since we insert an ensemble for // entry-id 0, right when we start @@ -575,26 +571,6 @@ private String toStringRepresentation(boolean withPassword) { return sb.toString(); } - void mergeEnsembles(SortedMap> newEnsembles) { - // allow new metadata to be one ensemble less than current metadata - // since ensemble change might kick in when recovery changed metadata - int diff = ensembles.size() - newEnsembles.size(); - if (0 != diff && 1 != diff) { - return; - } - int i = 0; - for (Entry> entry : newEnsembles.entrySet()) { - ++i; - if (ensembles.size() != i) { - // we should use last ensemble from current metadata - // not the new metadata read from zookeeper - long key = entry.getKey(); - List ensemble = entry.getValue(); - ensembles.put(key, ImmutableList.copyOf(ensemble)); - } - } - } - Set getBookiesInThisLedger() { Set bookies = new HashSet(); for (List ensemble : ensembles.values()) { From a809a59d2085e279b27a9b004feccb25f34e4ecc Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 23 Nov 2018 14:32:31 +0000 Subject: [PATCH 0135/1642] Make LedgerMetadata fields for closing immutable Master issue: #281 Reviewers: Enrico Olivelli , Sijie Guo This closes #1831 from ivankelly/close-immut --- .../bookkeeper/client/LedgerMetadata.java | 39 +++++++------------ .../client/LedgerMetadataBuilder.java | 11 ++---- .../bookkeeper/client/LedgerRecoveryTest.java | 6 +-- 3 files changed, 18 insertions(+), 38 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index bf5e9f5df6d..4f6707757cb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -78,9 +78,9 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe private final int writeQuorumSize; private final int ackQuorumSize; - private LedgerMetadataFormat.State state; - private long length; - private long lastEntryId; + private final LedgerMetadataFormat.State state; + private final long length; + private final long lastEntryId; private final long ctime; final boolean storeCtime; // non-private so builder can access for copy @@ -107,18 +107,22 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe boolean storeCtime, Map customMetadata) { checkArgument(ensembles.size() > 0, "There must be at least one ensemble in the ledger"); + if (state == LedgerMetadataFormat.State.CLOSED) { + checkArgument(length.isPresent(), "Closed ledger must have a length"); + checkArgument(lastEntryId.isPresent(), "Closed ledger must have a last entry"); + } else { + checkArgument(!length.isPresent(), "Non-closed ledger must not have a length"); + checkArgument(!lastEntryId.isPresent(), "Non-closed ledger must not have a last entry"); + } this.metadataFormatVersion = metadataFormatVersion; this.ensembleSize = ensembleSize; this.writeQuorumSize = writeQuorumSize; this.ackQuorumSize = ackQuorumSize; this.state = state; - if (lastEntryId.isPresent()) { - this.lastEntryId = lastEntryId.get(); - } else { - this.lastEntryId = LedgerHandle.INVALID_ENTRY_ID; - } - length.ifPresent((l) -> this.length = l); + + this.lastEntryId = lastEntryId.orElse(LedgerHandle.INVALID_ENTRY_ID); + this.length = length.orElse(0L); this.ensembles = Collections.unmodifiableNavigableMap( ensembles.entrySet().stream().collect(TreeMap::new, @@ -248,10 +252,6 @@ public long getLength() { return length; } - void setLength(long length) { - this.length = length; - } - @Override public boolean isClosed() { return state == LedgerMetadataFormat.State.CLOSED; @@ -265,19 +265,6 @@ public LedgerMetadataFormat.State getState() { return state; } - void setState(LedgerMetadataFormat.State state) { - this.state = state; - } - - void markLedgerInRecovery() { - state = LedgerMetadataFormat.State.IN_RECOVERY; - } - - void close(long entryId) { - lastEntryId = entryId; - state = LedgerMetadataFormat.State.CLOSED; - } - List getCurrentEnsemble() { return currentEnsemble; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index 74ec7179892..4a42c70d81f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -73,14 +73,9 @@ public static LedgerMetadataBuilder from(LedgerMetadata other) { builder.ackQuorumSize = other.getAckQuorumSize(); builder.state = other.getState(); - - long lastEntryId = other.getLastEntryId(); - if (lastEntryId != LedgerHandle.INVALID_ENTRY_ID) { - builder.lastEntryId = Optional.of(lastEntryId); - } - long length = other.getLength(); - if (length > 0) { - builder.length = Optional.of(length); + if (builder.state == LedgerMetadataFormat.State.CLOSED) { + builder.lastEntryId = Optional.of(other.getLastEntryId()); + builder.length = Optional.of(other.getLength()); } builder.ensembles.putAll(other.getAllEnsembles()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java index 6a21da2d61c..9a927b68852 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java @@ -84,10 +84,8 @@ private void testInternal(int numEntries) throws Exception { /* * Check if has recovered properly. */ - assertTrue("Has not recovered correctly: " + afterlh.getLastAddConfirmed(), - afterlh.getLastAddConfirmed() == numEntries - 1); - assertTrue("Has not set the length correctly: " + afterlh.getLength() + ", " + length, - afterlh.getLength() == length); + assertEquals("Has not recovered correctly", numEntries - 1, afterlh.getLastAddConfirmed()); + assertEquals("Has not set the length correctly", length, afterlh.getLength()); } @Test From 6a6f46bf45dcb8ee05f1efec62a8e73ec32e6db0 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 23 Nov 2018 15:57:35 +0100 Subject: [PATCH 0136/1642] Prepare website structure for 4.8.1 release Signed-off-by: eolivelli Descriptions of the changes in this PR: ### Motivation Prepare the wesite for 4.8.1 release ### Changes Create the structure for docs for 4.8.1 Reviewers: Sijie Guo This closes #1790 from eolivelli/fix/site-481-structure --- site/_config.yml | 2 +- site/docs/4.8.1/admin/autorecovery.md | 128 +++ site/docs/4.8.1/admin/bookies.md | 180 ++++ site/docs/4.8.1/admin/geo-replication.md | 22 + site/docs/4.8.1/admin/http.md | 407 +++++++++ site/docs/4.8.1/admin/metrics.md | 41 + site/docs/4.8.1/admin/perf.md | 3 + site/docs/4.8.1/admin/placement.md | 3 + site/docs/4.8.1/admin/upgrade.md | 181 ++++ site/docs/4.8.1/api/distributedlog-api.md | 395 ++++++++ site/docs/4.8.1/api/ledger-adv-api.md | 111 +++ site/docs/4.8.1/api/ledger-api.md | 841 ++++++++++++++++++ site/docs/4.8.1/api/overview.md | 17 + site/docs/4.8.1/deployment/dcos.md | 142 +++ site/docs/4.8.1/deployment/kubernetes.md | 181 ++++ site/docs/4.8.1/deployment/manual.md | 56 ++ site/docs/4.8.1/development/codebase.md | 3 + site/docs/4.8.1/development/protocol.md | 148 +++ site/docs/4.8.1/getting-started/concepts.md | 202 +++++ .../4.8.1/getting-started/installation.md | 74 ++ .../docs/4.8.1/getting-started/run-locally.md | 16 + site/docs/4.8.1/overview/overview.md | 60 ++ site/docs/4.8.1/overview/releaseNotes.md | 68 ++ .../4.8.1/overview/releaseNotesTemplate.md | 17 + site/docs/4.8.1/reference/cli.md | 10 + site/docs/4.8.1/reference/config.md | 9 + site/docs/4.8.1/reference/metrics.md | 3 + site/docs/4.8.1/security/overview.md | 21 + site/docs/4.8.1/security/sasl.md | 202 +++++ site/docs/4.8.1/security/tls.md | 210 +++++ site/docs/4.8.1/security/zookeeper.md | 41 + 31 files changed, 3793 insertions(+), 1 deletion(-) create mode 100644 site/docs/4.8.1/admin/autorecovery.md create mode 100644 site/docs/4.8.1/admin/bookies.md create mode 100644 site/docs/4.8.1/admin/geo-replication.md create mode 100644 site/docs/4.8.1/admin/http.md create mode 100644 site/docs/4.8.1/admin/metrics.md create mode 100644 site/docs/4.8.1/admin/perf.md create mode 100644 site/docs/4.8.1/admin/placement.md create mode 100644 site/docs/4.8.1/admin/upgrade.md create mode 100644 site/docs/4.8.1/api/distributedlog-api.md create mode 100644 site/docs/4.8.1/api/ledger-adv-api.md create mode 100644 site/docs/4.8.1/api/ledger-api.md create mode 100644 site/docs/4.8.1/api/overview.md create mode 100644 site/docs/4.8.1/deployment/dcos.md create mode 100644 site/docs/4.8.1/deployment/kubernetes.md create mode 100644 site/docs/4.8.1/deployment/manual.md create mode 100644 site/docs/4.8.1/development/codebase.md create mode 100644 site/docs/4.8.1/development/protocol.md create mode 100644 site/docs/4.8.1/getting-started/concepts.md create mode 100644 site/docs/4.8.1/getting-started/installation.md create mode 100644 site/docs/4.8.1/getting-started/run-locally.md create mode 100644 site/docs/4.8.1/overview/overview.md create mode 100644 site/docs/4.8.1/overview/releaseNotes.md create mode 100644 site/docs/4.8.1/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.8.1/reference/cli.md create mode 100644 site/docs/4.8.1/reference/config.md create mode 100644 site/docs/4.8.1/reference/metrics.md create mode 100644 site/docs/4.8.1/security/overview.md create mode 100644 site/docs/4.8.1/security/sasl.md create mode 100644 site/docs/4.8.1/security/tls.md create mode 100644 site/docs/4.8.1/security/zookeeper.md diff --git a/site/_config.yml b/site/_config.yml index 2b871aa8e4c..28920e6ca76 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -8,8 +8,8 @@ destination: local-generated twitter_url: https://twitter.com/asfbookkeeper versions: -# [next_version_placeholder] - "4.8.0" +# [next_version_placeholder] - "4.7.2" - "4.7.1" - "4.7.0" diff --git a/site/docs/4.8.1/admin/autorecovery.md b/site/docs/4.8.1/admin/autorecovery.md new file mode 100644 index 00000000000..b1dd078f9b2 --- /dev/null +++ b/site/docs/4.8.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: + +* the `shell recover` option +* 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 shell recover \ + 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 shell recover \ + 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.8.1/admin/bookies.md b/site/docs/4.8.1/admin/bookies.md new file mode 100644 index 00000000000..1b0427dae3c --- /dev/null +++ b/site/docs/4.8.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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + \ + \ + + ``` + + 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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + 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.8.1/admin/geo-replication.md b/site/docs/4.8.1/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.8.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.8.1/admin/http.md b/site/docs/4.8.1/admin/http.md new file mode 100644 index 00000000000..dc647449621 --- /dev/null +++ b/site/docs/4.8.1/admin/http.md @@ -0,0 +1,407 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 4 components: +1. Heartbeat: heartbeat for a specific bookie. +1. Config: doing the server configuration for a specific bookie. +1. Ledger: HTTP endpoints related to ledgers. +1. Bookie: HTTP endpoints related to bookies. +1. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Config + +### Endpoint: /metrics +1. Method: GET + * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.8.1/admin/metrics.md b/site/docs/4.8.1/admin/metrics.md new file mode 100644 index 00000000000..142df3dcd2d --- /dev/null +++ b/site/docs/4.8.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.prometheus.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.8.1/admin/perf.md b/site/docs/4.8.1/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.8.1/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.8.1/admin/placement.md b/site/docs/4.8.1/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.8.1/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.8.1/admin/upgrade.md b/site/docs/4.8.1/admin/upgrade.md new file mode 100644 index 00000000000..6825949dd77 --- /dev/null +++ b/site/docs/4.8.1/admin/upgrade.md @@ -0,0 +1,181 @@ +--- +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.7.x to 4.8.1 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.7.x to 4.8.1. + +In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. +(Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it). +Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. +But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. + + +### 4.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.8.1/api/distributedlog-api.md b/site/docs/4.8.1/api/distributedlog-api.md new file mode 100644 index 00000000000..8a1ea830422 --- /dev/null +++ b/site/docs/4.8.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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.8.1/api/ledger-adv-api.md b/site/docs/4.8.1/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.8.1/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.8.1/api/ledger-api.md b/site/docs/4.8.1/api/ledger-api.md new file mode 100644 index 00000000000..8147d39c9a9 --- /dev/null +++ b/site/docs/4.8.1/api/ledger-api.md @@ -0,0 +1,841 @@ +--- +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.8.1 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.8.1 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.8.1' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.8.1' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` + +### Relaxing Durability + +In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. +In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will +be always readable by other clients. + +On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after +writing the entry to SO buffers without waiting for an fsync. +In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. +Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. + +In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble ackknowledge the call after +performing an fsync to the disk which is storing the journal. +This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. + +The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. +In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. + + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete + + +wh.force().get(); // wait for fsync, make data available to readers and to the replicator + +wh.close(); // seal the ledger + +``` diff --git a/site/docs/4.8.1/api/overview.md b/site/docs/4.8.1/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.8.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.8.1/deployment/dcos.md b/site/docs/4.8.1/deployment/dcos.md new file mode 100644 index 00000000000..113bfda18a0 --- /dev/null +++ b/site/docs/4.8.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.8.1] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.8.1/deployment/kubernetes.md b/site/docs/4.8.1/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.8.1/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.8.1/deployment/manual.md b/site/docs/4.8.1/deployment/manual.md new file mode 100644 index 00000000000..daafd5556f5 --- /dev/null +++ b/site/docs/4.8.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.8.1/development/codebase.md b/site/docs/4.8.1/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.8.1/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.8.1/development/protocol.md b/site/docs/4.8.1/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.8.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.8.1/getting-started/concepts.md b/site/docs/4.8.1/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.8.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.8.1/getting-started/installation.md b/site/docs/4.8.1/getting-started/installation.md new file mode 100644 index 00000000000..9986cd8e043 --- /dev/null +++ b/site/docs/4.8.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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.8.1/getting-started/run-locally.md b/site/docs/4.8.1/getting-started/run-locally.md new file mode 100644 index 00000000000..edbfab9fda6 --- /dev/null +++ b/site/docs/4.8.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 +$ bookkeeper-server/bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.8.1/overview/overview.md b/site/docs/4.8.1/overview/overview.md new file mode 100644 index 00000000000..1d1f71b30a5 --- /dev/null +++ b/site/docs/4.8.1/overview/overview.md @@ -0,0 +1,60 @@ +--- +title: Apache BookKeeper™ 4.8.1 +--- + + +This documentation is for Apache BookKeeper™ version 4.8.1. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.8.1 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.8.1/overview/releaseNotes.md b/site/docs/4.8.1/overview/releaseNotes.md new file mode 100644 index 00000000000..c73db64703c --- /dev/null +++ b/site/docs/4.8.1/overview/releaseNotes.md @@ -0,0 +1,68 @@ +--- +title: Apache BookKeeper 4.8.1 Release Notes +--- + +This is the 13th release of Apache BookKeeper! + +The 4.8.1 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.7.0. +It is a new big milestone in Apache BookKeeper community, +this release include great new features, like Relaxed Durability, Stream Storage service and Multiple active Entrylogs. + +Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.8.1](../../admin/upgrade). The technical details of +this release are summarized below. + +## Highlights + +The main features in 4.8.1 are around following areas: + +- Durability +- ExplicitLAC feature +- New Table Storage Service +- Bug Fixes + + +### New WriteFlag DEFERRED_SYNC + +The writer may ask for temporary relaxed durability writes, that is to receive early acknowledge from Bookies, before an fsync() on Journal. +Together with this new flag we introduced the new WriteHandle#force() API, this this API the writer is able to request an explicit guarantee of durability to the Bookies +it is mostly like and explicit fsync() on a file system. + +See [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) and [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) for reference + +### New behaviour for Netty ByteBuf reference count management + +All the client side APIs which take ByteBufs now will have the net effect of decrementing by 1 the refcount. +This is consistent with general contract of Netty. +It is expected that the client passes the ownership of the ByteBuf to BookKeeper client. + +### Multiple Active Entrylogs + +It is now possible on the Bookie to have multiple active entry loggers, +this new feature will help with compaction performance and some specific workloads. + +See [Multiple active entrylogs](https://github.com/apache/bookkeeper/issues/570) + +### Table Storage Service + +From this version we are providing the a table (key/value) service embedded in Bookies. + +See [BP-30: BookKeeper Table Service](https://github.com/apache/bookkeeper/issues/1205) + +### Make ExplicitLAC persistent + +ExplicitLAC was contributed from Salesforce in 4.5.0 release, but in the first release +it was a beft-effort in-memory mechanism. Now you can configure Bookies to store durably ExplicitLAC. + +See [Make ExplicitLAC persistent](https://github.com/apache/bookkeeper/issues/1527) + +### Ensemble change on Delayed Write Failure + +We are handling more gracefully the case of a failure of a Bookie in spite of a succeeded write. +If you are writing with Ack Quorum = 2 and Write Quorum = 3, writes will succeeed even if 1 of 3 Bookies fail, +now BookKeeper will trigger an *ensemble change* and replace the failed bookie earlier. + +See [Ensemble change on Delayed Write Failure](https://github.com/apache/bookkeeper/issues/1390) + +## Full list of changes + +- [https://github.com/apache/bookkeeper/milestone/4](https://github.com/apache/bookkeeper/milestone/4?closed=1) diff --git a/site/docs/4.8.1/overview/releaseNotesTemplate.md b/site/docs/4.8.1/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..d1061738ab2 --- /dev/null +++ b/site/docs/4.8.1/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.8.1 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.8.1. The technical details of this release are summarized +below. + +## Highlights + +[List the highlights] + +## Details + +[list to issues list] + diff --git a/site/docs/4.8.1/reference/cli.md b/site/docs/4.8.1/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.8.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.8.1/reference/config.md b/site/docs/4.8.1/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.8.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.8.1/reference/metrics.md b/site/docs/4.8.1/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.8.1/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.8.1/security/overview.md b/site/docs/4.8.1/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.8.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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.8.1/security/sasl.md b/site/docs/4.8.1/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.8.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.8.1/security/tls.md b/site/docs/4.8.1/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.8.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.8.1/security/zookeeper.md b/site/docs/4.8.1/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.8.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 872db06a716b1477ccf93ebcae2274477ceb28b6 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 23 Nov 2018 16:44:35 +0000 Subject: [PATCH 0137/1642] Remove LedgerMetadata copy constructor A copy constructor makes no sense when it's not possible to mutate the object. Master issue: #281 Reviewers: Enrico Olivelli This closes #1832 from ivankelly/kill-copy --- .../bookkeeper/client/LedgerMetadata.java | 30 ------------------- .../bookkeeper/client/LedgerRecoveryOp.java | 2 +- .../client/MockBookKeeperTestCase.java | 4 +-- 3 files changed, 3 insertions(+), 33 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 4f6707757cb..6094df26ae3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -152,36 +152,6 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe this.customMetadata.putAll(customMetadata); } - /** - * Copy Constructor. - */ - LedgerMetadata(LedgerMetadata other) { - this.ensembleSize = other.ensembleSize; - this.writeQuorumSize = other.writeQuorumSize; - this.ackQuorumSize = other.ackQuorumSize; - this.length = other.length; - this.lastEntryId = other.lastEntryId; - this.metadataFormatVersion = other.metadataFormatVersion; - this.state = other.state; - this.hasPassword = other.hasPassword; - this.digestType = other.digestType; - this.ctime = other.ctime; - this.storeCtime = other.storeCtime; - this.password = new byte[other.password.length]; - System.arraycopy(other.password, 0, this.password, 0, other.password.length); - this.ensembles = Collections.unmodifiableNavigableMap( - other.ensembles.entrySet().stream().collect(TreeMap::new, - (m, e) -> m.put(e.getKey(), - ImmutableList.copyOf(e.getValue())), - TreeMap::putAll)); - if (state != LedgerMetadataFormat.State.CLOSED) { - currentEnsemble = this.ensembles.lastEntry().getValue(); - } else { - currentEnsemble = null; - } - this.customMetadata = other.customMetadata; - } - @Override public NavigableMap> getAllEnsembles() { return ensembles; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java index 923c4df2025..bc2985a9577 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java @@ -104,7 +104,7 @@ public void readLastConfirmedDataComplete(int rc, RecoveryData data) { } // keep a copy of ledger metadata before proceeding // ledger recovery - metadataForRecovery = new LedgerMetadata(lh.getLedgerMetadata()); + metadataForRecovery = lh.getLedgerMetadata(); doRecoveryRead(); } else if (rc == BKException.Code.UnauthorizedAccessException) { submitCallback(rc); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java index e80807869aa..a88f3561088 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java @@ -372,7 +372,7 @@ private void setupReadLedgerMetadata() { if (ledgerMetadata == null) { promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); } else { - promise.complete(new Versioned<>(new LedgerMetadata(ledgerMetadata), new LongVersion(1))); + promise.complete(new Versioned<>(ledgerMetadata, new LongVersion(1))); } }); return promise; @@ -426,7 +426,7 @@ private void setupCreateLedgerMetadata() { executor.executeOrdered(ledgerId, () -> { LedgerMetadata ledgerMetadata = (LedgerMetadata) args[1]; - mockLedgerMetadataRegistry.put(ledgerId, new LedgerMetadata(ledgerMetadata)); + mockLedgerMetadataRegistry.put(ledgerId, ledgerMetadata); promise.complete(new Versioned<>(ledgerMetadata, new LongVersion(1))); }); return promise; From fd8b3ae39331b4605c6ae2d2ca3b8c1ed8303f26 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 23 Nov 2018 18:50:47 +0000 Subject: [PATCH 0138/1642] Make custom metadata final Master issue: #281 Reviewers: Enrico Olivelli This closes #1833 from ivankelly/immut-cust --- .../org/apache/bookkeeper/client/LedgerMetadata.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 6094df26ae3..514f1af27f7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -23,7 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Maps; +import com.google.common.collect.ImmutableMap; import com.google.protobuf.ByteString; import com.google.protobuf.TextFormat; import java.io.BufferedReader; @@ -87,11 +87,11 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe private final NavigableMap> ensembles; private final ImmutableList currentEnsemble; - private final boolean hasPassword; // IKTODO other things should be optionals instead + private final boolean hasPassword; private final LedgerMetadataFormat.DigestType digestType; private final byte[] password; - private Map customMetadata = Maps.newHashMap(); + private final Map customMetadata; LedgerMetadata(int metadataFormatVersion, int ensembleSize, @@ -149,7 +149,7 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe this.ctime = ctime; this.storeCtime = storeCtime; - this.customMetadata.putAll(customMetadata); + this.customMetadata = ImmutableMap.copyOf(customMetadata); } @Override @@ -272,10 +272,6 @@ public Map getCustomMetadata() { return this.customMetadata; } - void setCustomMetadata(Map customMetadata) { - this.customMetadata = customMetadata; - } - LedgerMetadataFormat buildProtoFormat() { return buildProtoFormat(true); } From 60f322efef45720e27b724c18b75846cb1437f3a Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sat, 24 Nov 2018 21:29:12 -0800 Subject: [PATCH 0139/1642] Remove MathUtils.now() Descriptions of the changes in this PR: *Motivation* `MathUtils.now()` using System.nanoTime doesn't provide accurate time. It is used for measuring pupose. However since `nanoTime` can have numeric overflow, so when it is used for converting into milliseconds, it is misleading and error-prone. A lot of places using `MathUtis.now` can all replaced with `System.currentTimeMillis()`. It leads to unknown behavior on compactions. examples is shown in the following figure. wechatimg180 example: this graph shows the major_compaction_count. major compaction is supposed to run every day. but in the graph, there is no major compaction occuring in 4 days. *Changes* Remove `MathUtils.now()` and replace it with `System.currentTimeMillis()`. Reviewers: Jia Zhai , Enrico Olivelli This closes #1837 from sijie/remove_mathutils_now --- .../apache/bookkeeper/common/util/MathUtils.java | 15 --------------- .../bookkeeper/common/util/TestMathUtils.java | 8 -------- .../java/org/apache/bookkeeper/bookie/Bookie.java | 8 ++++---- .../bookkeeper/bookie/GarbageCollectorThread.java | 8 ++++---- .../bookie/ScanAndCompareGarbageCollector.java | 7 +++---- .../org/apache/bookkeeper/bookie/SyncThread.java | 7 +++---- .../bookkeeper/net/StabilizeNetworkTopology.java | 7 +++---- .../apache/bookkeeper/util/LocalBookKeeper.java | 4 ++-- .../java/org/apache/bookkeeper/util/Shell.java | 4 ++-- .../apache/bookkeeper/bookie/CompactionTest.java | 7 +++---- 10 files changed, 24 insertions(+), 51 deletions(-) diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/MathUtils.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/MathUtils.java index 94999a49ec4..43d5d778bd1 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/MathUtils.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/MathUtils.java @@ -40,21 +40,6 @@ public static int findNextPositivePowerOfTwo(final int value) { return 1 << (32 - Integer.numberOfLeadingZeros(value - 1)); } - /** - * Current time from some arbitrary time base in the past, counting in - * milliseconds, and not affected by settimeofday or similar system clock - * changes. This is appropriate to use when computing how much longer to - * wait for an interval to expire. - * - *

            NOTE: only use it for measuring. - * http://docs.oracle.com/javase/1.5.0/docs/api/java/lang/System.html#nanoTime%28%29 - * - * @return current time in milliseconds. - */ - public static long now() { - return System.nanoTime() / NANOSECONDS_PER_MILLISECOND; - } - /** * Current time from some arbitrary time base in the past, counting in * nanoseconds, and not affected by settimeofday or similar system clock diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestMathUtils.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestMathUtils.java index 3bd58c21939..fc2318b9392 100644 --- a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestMathUtils.java +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestMathUtils.java @@ -19,13 +19,11 @@ package org.apache.bookkeeper.common.util; import static org.apache.bookkeeper.common.util.MathUtils.findNextPositivePowerOfTwo; -import static org.apache.bookkeeper.common.util.MathUtils.now; import static org.apache.bookkeeper.common.util.MathUtils.nowInNano; import static org.apache.bookkeeper.common.util.MathUtils.signSafeMod; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import java.util.concurrent.TimeUnit; import org.junit.Test; /** @@ -46,12 +44,6 @@ public void testFindNextPositivePowerOfTwo() { assertEquals(16384, findNextPositivePowerOfTwo(12345)); } - @Test - public void testNow() { - long nowInMillis = now(); - assertTrue(TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) >= nowInMillis); - } - @Test public void testNowInNanos() { long nowInNanos = nowInNano(); 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 cd0b2f745c2..f91bf285b4f 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 @@ -733,7 +733,7 @@ StateManager initializeStateManager() throws IOException { } void readJournal() throws IOException, BookieException { - long startTs = MathUtils.now(); + long startTs = System.currentTimeMillis(); JournalScanner scanner = new JournalScanner() { @Override public void process(int journalVersion, long offset, ByteBuffer recBuff) throws IOException { @@ -823,7 +823,7 @@ public void process(int journalVersion, long offset, ByteBuffer recBuff) throws for (Journal journal : journals) { journal.replay(scanner); } - long elapsedTs = MathUtils.now() - startTs; + long elapsedTs = System.currentTimeMillis() - startTs; LOG.info("Finished replaying journal in {} ms.", elapsedTs); } @@ -1492,7 +1492,7 @@ public static void main(String[] args) Bookie b = new Bookie(new ServerConfiguration()); b.start(); CounterCallback cb = new CounterCallback(); - long start = MathUtils.now(); + long start = System.currentTimeMillis(); for (int i = 0; i < 100000; i++) { ByteBuf buff = Unpooled.buffer(1024); buff.writeLong(1); @@ -1501,7 +1501,7 @@ public static void main(String[] args) b.addEntry(buff, false /* ackBeforeSync */, cb, null, new byte[0]); } cb.waitZero(); - long end = MathUtils.now(); + long end = System.currentTimeMillis(); System.out.println("Took " + (end - start) + "ms"); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java index 4b1c834443a..ad752debdaf 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java @@ -255,7 +255,7 @@ public Long getSample() { LOG.info("Major Compaction : enabled=" + enableMajorCompaction + ", threshold=" + majorCompactionThreshold + ", interval=" + majorCompactionInterval); - lastMinorCompactionTime = lastMajorCompactionTime = MathUtils.now(); + lastMinorCompactionTime = lastMajorCompactionTime = System.currentTimeMillis(); } public void enableForceGC() { @@ -364,13 +364,13 @@ public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMin LOG.info("Disk full, suspend minor compaction to slow down filling disk."); } - long curTime = MathUtils.now(); + long curTime = System.currentTimeMillis(); if (enableMajorCompaction && (!suspendMajor) && (force || curTime - lastMajorCompactionTime > majorCompactionInterval)) { // enter major compaction LOG.info("Enter major compaction, suspendMajor {}", suspendMajor); doCompactEntryLogs(majorCompactionThreshold); - lastMajorCompactionTime = MathUtils.now(); + lastMajorCompactionTime = System.currentTimeMillis(); // and also move minor compaction time lastMinorCompactionTime = lastMajorCompactionTime; majorCompactionCounter.inc(); @@ -379,7 +379,7 @@ public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMin // enter minor compaction LOG.info("Enter minor compaction, suspendMinor {}", suspendMinor); doCompactEntryLogs(minorCompactionThreshold); - lastMinorCompactionTime = MathUtils.now(); + lastMinorCompactionTime = System.currentTimeMillis(); minorCompactionCounter.inc(); } this.gcThreadRuntime.registerSuccessfulEvent( diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index d4c8bc2b9fe..6af5d15b027 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -45,7 +45,6 @@ import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.zookeeper.KeeperException; @@ -96,7 +95,7 @@ public ScanAndCompareGarbageCollector(LedgerManager ledgerManager, CompactableLe this.conf = conf; this.selfBookieAddress = Bookie.getBookieAddress(conf); this.gcOverReplicatedLedgerIntervalMillis = conf.getGcOverreplicatedLedgerWaitTimeMillis(); - this.lastOverReplicatedLedgerGcTimeMillis = MathUtils.now(); + this.lastOverReplicatedLedgerGcTimeMillis = System.currentTimeMillis(); if (gcOverReplicatedLedgerIntervalMillis > 0) { this.enableGcOverReplicatedLedger = true; } @@ -137,7 +136,7 @@ public void gc(GarbageCleaner garbageCleaner) { Long.MAX_VALUE)); this.activeLedgerCounter = bkActiveLedgers.size(); - long curTime = MathUtils.now(); + long curTime = System.currentTimeMillis(); boolean checkOverreplicatedLedgers = (enableGcOverReplicatedLedger && curTime - lastOverReplicatedLedgerGcTimeMillis > gcOverReplicatedLedgerIntervalMillis); if (checkOverreplicatedLedgers) { @@ -150,7 +149,7 @@ public void gc(GarbageCleaner garbageCleaner) { } else { LOG.info("Removed over-replicated ledgers: {}", overReplicatedLedgers); } - lastOverReplicatedLedgerGcTimeMillis = MathUtils.now(); + lastOverReplicatedLedgerGcTimeMillis = System.currentTimeMillis(); } // Iterate over all the ledger on the metadata store 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 a7c3a7a5632..00288dc27d6 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 @@ -36,7 +36,6 @@ import org.apache.bookkeeper.bookie.LedgerDirsManager.LedgerDirsListener; import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.util.MathUtils; /** * SyncThread is a background thread which help checkpointing ledger storage @@ -205,11 +204,11 @@ void shutdown() throws InterruptedException { requestFlush(); executor.shutdown(); - long start = MathUtils.now(); + long start = System.currentTimeMillis(); while (!executor.awaitTermination(5, TimeUnit.MINUTES)) { - long now = MathUtils.now(); + long now = System.currentTimeMillis(); log.info("SyncThread taking a long time to shutdown. Has taken {}" - + " seconds so far", now - start); + + " milliseconds so far", now - start); } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/StabilizeNetworkTopology.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/StabilizeNetworkTopology.java index 0bac3bfd4a9..5c244f248fa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/StabilizeNetworkTopology.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/StabilizeNetworkTopology.java @@ -26,7 +26,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.util.MathUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,7 +41,7 @@ static class NodeStatus { boolean tentativeToRemove; NodeStatus() { - this.lastPresentTime = MathUtils.now(); + this.lastPresentTime = System.currentTimeMillis(); } synchronized boolean isTentativeToRemove() { @@ -52,7 +51,7 @@ synchronized boolean isTentativeToRemove() { synchronized NodeStatus updateStatus(boolean tentativeToRemove) { this.tentativeToRemove = tentativeToRemove; if (!this.tentativeToRemove) { - this.lastPresentTime = MathUtils.now(); + this.lastPresentTime = System.currentTimeMillis(); } return this; } @@ -88,7 +87,7 @@ public void run(Timeout timeout) throws Exception { // no status of this node, remove this node from topology impl.remove(node); } else if (status.isTentativeToRemove()) { - long millisSinceLastSeen = MathUtils.now() - status.getLastPresentTime(); + long millisSinceLastSeen = System.currentTimeMillis() - status.getLastPresentTime(); if (millisSinceLastSeen >= stabilizePeriodMillis) { logger.info("Node {} (seen @ {}) becomes stale for {} ms, remove it from the topology.", node, status.getLastPresentTime(), millisSinceLastSeen); 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 9555474e069..e431e8e5e70 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 @@ -492,7 +492,7 @@ private static void usage() { } public static boolean waitForServerUp(String hp, long timeout) { - long start = MathUtils.now(); + long start = System.currentTimeMillis(); String split[] = hp.split(":"); String host = split[0]; int port = Integer.parseInt(split[1]); @@ -524,7 +524,7 @@ public static boolean waitForServerUp(String hp, long timeout) { LOG.info("server " + hp + " not up " + e); } - if (MathUtils.now() > start + timeout) { + if (System.currentTimeMillis() > start + timeout) { break; } try { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java index 9e37614a749..60c7648e0a3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java @@ -93,7 +93,7 @@ protected void setWorkingDirectory(File dir) { * Check to see if a command needs to be executed and execute if needed. */ protected void run() throws IOException { - if (lastTime + interval > MathUtils.now()) { + if (lastTime + interval > System.currentTimeMillis()) { return; } exitCode = 0; // reset for next run @@ -199,7 +199,7 @@ public void run() { LOG.warn("Error while closing the error stream", ioe); } process.destroy(); - lastTime = MathUtils.now(); + lastTime = System.currentTimeMillis(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java index e0c9e2f7d93..e35c30973f0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java @@ -70,7 +70,6 @@ import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.util.DiskChecker; import org.apache.bookkeeper.util.HardLink; -import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.TestUtils; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; @@ -268,7 +267,7 @@ public void checkpointComplete(Checkpoint checkPoint, boolean compact) Checkpointer.NULL, NullStatsLogger.INSTANCE); storage.start(); - long startTime = MathUtils.now(); + long startTime = System.currentTimeMillis(); storage.gcThread.enableForceGC(); storage.gcThread.triggerGC().get(); //major storage.gcThread.triggerGC().get(); //minor @@ -1174,7 +1173,7 @@ public void checkpointComplete(Checkpoint checkPoint, boolean compact) storage.gcThread.suspendMajorGC(); Thread.sleep(1000); - long startTime = MathUtils.now(); + long startTime = System.currentTimeMillis(); majorCompactions = stats.getCounter("storage.gc." + MAJOR_COMPACTION_COUNT).get().intValue(); Thread.sleep(conf.getMajorCompactionInterval() * 1000 + conf.getGcWaitTime()); @@ -1194,7 +1193,7 @@ public void checkpointComplete(Checkpoint checkPoint, boolean compact) storage.gcThread.suspendMinorGC(); Thread.sleep(1000); - startTime = MathUtils.now(); + startTime = System.currentTimeMillis(); minorCompactions = stats.getCounter("storage.gc." + MINOR_COMPACTION_COUNT).get().intValue(); Thread.sleep(conf.getMajorCompactionInterval() * 1000 + conf.getGcWaitTime()); From d4e489bb252cfb1a50171eec6135301937d57fde Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 26 Nov 2018 09:51:38 +0100 Subject: [PATCH 0140/1642] Release notes for 4.8.1 Release notes for 4.8.1 The commit to review is d09296c589b8a188937833ca51a4b24ea337a901 Reviewers: Jia Zhai , Matteo Merli , Sijie Guo This closes #1795 from eolivelli/fix/481-release-notes --- site/_config.yml | 3 +- site/docs/4.8.1/overview/releaseNotes.md | 61 +++++------------------- site/releases.md | 8 ++++ 3 files changed, 22 insertions(+), 50 deletions(-) diff --git a/site/_config.yml b/site/_config.yml index 28920e6ca76..2512864fda3 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -8,8 +8,9 @@ destination: local-generated twitter_url: https://twitter.com/asfbookkeeper versions: -- "4.8.0" # [next_version_placeholder] +- "4.8.1" +- "4.8.0" - "4.7.2" - "4.7.1" - "4.7.0" diff --git a/site/docs/4.8.1/overview/releaseNotes.md b/site/docs/4.8.1/overview/releaseNotes.md index c73db64703c..7434f47c84e 100644 --- a/site/docs/4.8.1/overview/releaseNotes.md +++ b/site/docs/4.8.1/overview/releaseNotes.md @@ -2,67 +2,30 @@ title: Apache BookKeeper 4.8.1 Release Notes --- -This is the 13th release of Apache BookKeeper! +This is the 14th release of Apache BookKeeper! -The 4.8.1 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.7.0. -It is a new big milestone in Apache BookKeeper community, -this release include great new features, like Relaxed Durability, Stream Storage service and Multiple active Entrylogs. +The 4.8.1 release is a bugfix release which fixes a bunch of issues reported from users of 4.8.0. -Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.8.1](../../admin/upgrade). The technical details of -this release are summarized below. +Apache BookKeeper users who are using 4.8.0 are encouraged to upgrade to 4.8.1. The technical details of this release are summarized +below. ## Highlights -The main features in 4.8.1 are around following areas: +- Use default metrics registry in Prometheus exporter, see [apache/bookkeeper#1765](https://github.com/apache/bookkeeper/pull/1765) -- Durability -- ExplicitLAC feature -- New Table Storage Service -- Bug Fixes +- Don't cache Bookie hostname DNS resolution forever, see [apache/bookkeeper#1762](https://github.com/apache/bookkeeper/pull/1762) +- Reduce stack traces in logs for common cases, see [apache/bookkeeper#1762](https://github.com/apache/bookkeeper/pull/1776) -### New WriteFlag DEFERRED_SYNC +- Ledger deletion racing with flush can cause a ledger index to be resurrected, see [apache/bookkeeper#1757](https://github.com/apache/bookkeeper/pull/1757) -The writer may ask for temporary relaxed durability writes, that is to receive early acknowledge from Bookies, before an fsync() on Journal. -Together with this new flag we introduced the new WriteHandle#force() API, this this API the writer is able to request an explicit guarantee of durability to the Bookies -it is mostly like and explicit fsync() on a file system. +- EntryMemTable.newEntry retains reference to passed ByteBuffer array, can cause corruption on journal replay, see [apache/bookkeeper#1737](https://github.com/apache/bookkeeper/pull/1737) -See [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) and [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) for reference -### New behaviour for Netty ByteBuf reference count management +### Dependency Changes -All the client side APIs which take ByteBufs now will have the net effect of decrementing by 1 the refcount. -This is consistent with general contract of Netty. -It is expected that the client passes the ownership of the ByteBuf to BookKeeper client. - -### Multiple Active Entrylogs - -It is now possible on the Bookie to have multiple active entry loggers, -this new feature will help with compaction performance and some specific workloads. - -See [Multiple active entrylogs](https://github.com/apache/bookkeeper/issues/570) - -### Table Storage Service - -From this version we are providing the a table (key/value) service embedded in Bookies. - -See [BP-30: BookKeeper Table Service](https://github.com/apache/bookkeeper/issues/1205) - -### Make ExplicitLAC persistent - -ExplicitLAC was contributed from Salesforce in 4.5.0 release, but in the first release -it was a beft-effort in-memory mechanism. Now you can configure Bookies to store durably ExplicitLAC. - -See [Make ExplicitLAC persistent](https://github.com/apache/bookkeeper/issues/1527) - -### Ensemble change on Delayed Write Failure - -We are handling more gracefully the case of a failure of a Bookie in spite of a succeeded write. -If you are writing with Ack Quorum = 2 and Write Quorum = 3, writes will succeeed even if 1 of 3 Bookies fail, -now BookKeeper will trigger an *ensemble change* and replace the failed bookie earlier. - -See [Ensemble change on Delayed Write Failure](https://github.com/apache/bookkeeper/issues/1390) +There is no dependecy upgrade from 4.8.0. ## Full list of changes -- [https://github.com/apache/bookkeeper/milestone/4](https://github.com/apache/bookkeeper/milestone/4?closed=1) +- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.1+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.1+is%3Aclosed) diff --git a/site/releases.md b/site/releases.md index d7888fb8fce..72ac0c1cfd7 100644 --- a/site/releases.md +++ b/site/releases.md @@ -64,6 +64,14 @@ Client Guide | API docs ## News +### 22 November, 2018 Release 4.8.1 available + +This is the 14th release of Apache BookKeeper ! + +The 4.8.1 release is a bugfix release which fixes a bunch of issues reported from users of 4.8.0. + +See [BookKeeper 4.8.1 Release Notes](../docs/4.8.1/overview/releaseNotes) for details. + ### 26 September, 2018 Release 4.8.0 available This is the 13th release of Apache BookKeeper ! From a1758e38fb2cb291d23f22d5facebbc936d892e9 Mon Sep 17 00:00:00 2001 From: Jia Zhai Date: Tue, 27 Nov 2018 18:16:19 +0800 Subject: [PATCH 0141/1642] Add rest endpoint trigger_gc to trigger GC on Bookie Descriptions of the changes in this PR: Add rest endpoint trigger_gc to trigger GC on Bookie ### Motivation Some times user would like to trigger GC manually instead of waiting to the timeout or disk full. Reviewers: Enrico Olivelli , Sijie Guo This closes #1838 from jiazhai/rest_force_gc --- .../apache/bookkeeper/http/HttpRouter.java | 2 + .../apache/bookkeeper/http/HttpServer.java | 1 + .../bookie/GarbageCollectorThread.java | 7 ++ .../bookie/InterleavedLedgerStorage.java | 5 ++ .../bookkeeper/bookie/LedgerStorage.java | 6 ++ .../bookie/SortedLedgerStorage.java | 5 ++ .../bookie/storage/ldb/DbLedgerStorage.java | 4 ++ .../ldb/SingleDirectoryDbLedgerStorage.java | 5 ++ .../server/http/BKHttpServiceProvider.java | 3 + .../server/http/service/TriggerGCService.java | 71 +++++++++++++++++++ .../server/http/TestHttpService.java | 38 ++++++++++ site/docs/latest/admin/http.md | 11 +++ 12 files changed, 158 insertions(+) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerGCService.java diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java index a856c720700..5384df6520e 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java @@ -45,6 +45,7 @@ public abstract class HttpRouter { public static final String LAST_LOG_MARK = "/api/v1/bookie/last_log_mark"; public static final String LIST_DISK_FILE = "/api/v1/bookie/list_disk_file"; public static final String EXPAND_STORAGE = "/api/v1/bookie/expand_storage"; + public static final String GC = "/api/v1/bookie/gc"; // autorecovery public static final String RECOVERY_BOOKIE = "/api/v1/autorecovery/bookie"; public static final String LIST_UNDER_REPLICATED_LEDGER = "/api/v1/autorecovery/list_under_replicated_ledger"; @@ -73,6 +74,7 @@ public HttpRouter(AbstractHttpHandlerFactory handlerFactory) { this.endpointHandlers.put(LAST_LOG_MARK, handlerFactory.newHandler(HttpServer.ApiType.LAST_LOG_MARK)); this.endpointHandlers.put(LIST_DISK_FILE, handlerFactory.newHandler(HttpServer.ApiType.LIST_DISK_FILE)); this.endpointHandlers.put(EXPAND_STORAGE, handlerFactory.newHandler(HttpServer.ApiType.EXPAND_STORAGE)); + this.endpointHandlers.put(GC, handlerFactory.newHandler(HttpServer.ApiType.GC)); // autorecovery this.endpointHandlers.put(RECOVERY_BOOKIE, handlerFactory.newHandler(HttpServer.ApiType.RECOVERY_BOOKIE)); diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java index 30e4d05ae0c..5e9f5090274 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java @@ -77,6 +77,7 @@ enum ApiType { LAST_LOG_MARK, LIST_DISK_FILE, EXPAND_STORAGE, + GC, // autorecovery RECOVERY_BOOKIE, LIST_UNDER_REPLICATED_LEDGER, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java index ad752debdaf..c4d9414f1bc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java @@ -382,6 +382,13 @@ public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMin lastMinorCompactionTime = System.currentTimeMillis(); minorCompactionCounter.inc(); } + + if (force) { + if (forceGarbageCollection.compareAndSet(true, false)) { + LOG.info("{} Set forceGarbageCollection to false after force GC to make it forceGC-able again.", Thread + .currentThread().getName()); + } + } this.gcThreadRuntime.registerSuccessfulEvent( MathUtils.nowInNano() - threadStart, TimeUnit.NANOSECONDS); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java index 8ab6517b489..d7d4977950d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java @@ -191,6 +191,11 @@ public void diskJustWritable(File disk) { }; } + @Override + public void forceGC() { + gcThread.enableForceGC(); + } + @Override public void start() { gcThread.start(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java index 34e32b9499d..ef64ff986c2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java @@ -175,4 +175,10 @@ default LedgerStorage getUnderlyingLedgerStorage() { return this; } + /** + * Force trigger Garbage Collection. + */ + default void forceGC() { + return; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java index e4b137ffbb5..dd7b373375a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java @@ -327,4 +327,9 @@ public void flushEntriesLocationsIndex() throws IOException { public LedgerStorage getUnderlyingLedgerStorage() { return interleavedLedgerStorage; } + + @Override + public void forceGC() { + interleavedLedgerStorage.forceGC(); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index 8753363699f..831de531d91 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -348,4 +348,8 @@ public static void readLedgerIndexEntries(long ledgerId, ServerConfiguration ser } } + @Override + public void forceGC() { + ledgerStorageList.stream().forEach(SingleDirectoryDbLedgerStorage::forceGC); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index e31b44e8147..27b4214e1b5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -263,6 +263,11 @@ public void start() { gcThread.start(); } + @Override + public void forceGC() { + gcThread.enableForceGC(); + } + @Override public void shutdown() throws InterruptedException { try { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java index 052b50ecf62..102b0eb57ef 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java @@ -53,6 +53,7 @@ import org.apache.bookkeeper.server.http.service.ReadLedgerEntryService; import org.apache.bookkeeper.server.http.service.RecoveryBookieService; import org.apache.bookkeeper.server.http.service.TriggerAuditService; +import org.apache.bookkeeper.server.http.service.TriggerGCService; import org.apache.bookkeeper.server.http.service.WhoIsAuditorService; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; @@ -208,6 +209,8 @@ public HttpEndpointService provideHttpEndpointService(ApiType type) { return new ListDiskFilesService(configuration); case EXPAND_STORAGE: return new ExpandStorageService(configuration); + case GC: + return new TriggerGCService(configuration, bookieServer); // autorecovery case RECOVERY_BOOKIE: diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerGCService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerGCService.java new file mode 100644 index 00000000000..185965afad8 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerGCService.java @@ -0,0 +1,71 @@ +/* + * 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.server.http.service; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.bookkeeper.common.util.JsonUtil; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.http.HttpServer; +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.proto.BookieServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * HttpEndpointService that handle force trigger GC requests. + * + *

            The PUT method will force trigger GC on current bookie, and make GC run at backend. + */ +public class TriggerGCService implements HttpEndpointService { + + static final Logger LOG = LoggerFactory.getLogger(TriggerGCService.class); + + protected ServerConfiguration conf; + protected BookieServer bookieServer; + + public TriggerGCService(ServerConfiguration conf, BookieServer bookieServer) { + checkNotNull(conf); + checkNotNull(bookieServer); + this.conf = conf; + this.bookieServer = bookieServer; + } + + @Override + public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { + HttpServiceResponse response = new HttpServiceResponse(); + // PUT + if (HttpServer.Method.PUT == request.getMethod()) { + bookieServer.getBookie().getLedgerStorage().forceGC(); + + String output = "Triggered GC on BookieServer: " + bookieServer.toString(); + String jsonResponse = JsonUtil.toJson(output); + LOG.debug("output body:" + jsonResponse); + response.setBody(jsonResponse); + response.setCode(HttpServer.StatusCode.OK); + return response; + } else { + response.setCode(HttpServer.StatusCode.NOT_FOUND); + response.setBody("Not found method. Should be PUT method to trigger GC."); + return response; + } + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index 079e08f422f..3f587c75336 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -741,4 +741,42 @@ public void testDecommissionService() throws Exception { stopAuditorElector(); } + /** + * Create ledgers, then test Delete Ledger service. + */ + @Test + public void testTriggerGCService() throws Exception { + baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + BookKeeper.DigestType digestType = BookKeeper.DigestType.CRC32; + int numLedgers = 4; + int numMsgs = 100; + LedgerHandle[] lh = new LedgerHandle[numLedgers]; + // create ledgers + for (int i = 0; i < numLedgers; i++) { + lh[i] = bkc.createLedger(digestType, "".getBytes()); + } + String content = "Apache BookKeeper is cool!"; + // add entries + for (int i = 0; i < numMsgs; i++) { + for (int j = 0; j < numLedgers; j++) { + lh[j].addEntry(content.getBytes()); + } + } + // close ledgers + for (int i = 0; i < numLedgers; i++) { + lh[i].close(); + } + HttpEndpointService triggerGCService = bkHttpServiceProvider + .provideHttpEndpointService(HttpServer.ApiType.GC); + + //1, GET, should return NOT_FOUND + HttpServiceRequest request1 = new HttpServiceRequest(null, HttpServer.Method.GET, null); + HttpServiceResponse response1 = triggerGCService.handle(request1); + assertEquals(HttpServer.StatusCode.NOT_FOUND.getValue(), response1.getStatusCode()); + + //2, PUT, should return OK + HttpServiceRequest request2 = new HttpServiceRequest(null, HttpServer.Method.PUT, null); + HttpServiceResponse response2 = triggerGCService.handle(request2); + assertEquals(HttpServer.StatusCode.OK.getValue(), response2.getStatusCode()); + } } diff --git a/site/docs/latest/admin/http.md b/site/docs/latest/admin/http.md index dc647449621..7c1abea49f2 100644 --- a/site/docs/latest/admin/http.md +++ b/site/docs/latest/admin/http.md @@ -271,6 +271,17 @@ Currently all the HTTP endpoints could be divided into these 4 components: |403 | Permission denied | |404 | Not found | +### Endpoint: /api/v1/bookie/gc +1. Method: PUT + * Description: trigger gc for this bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + ## Auto recovery ### Endpoint: /api/v1/autorecovery/bookie/ From b0fe6077753252e5df0641ad9598a753105463b7 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 27 Nov 2018 22:58:12 -0800 Subject: [PATCH 0142/1642] [UTILS] Move ReflectionUtils to bookkeeper-common module Descriptions of the changes in this PR: *Motivation* RefelectionUtils contains helpers for loading classes from relections. It is useful across the project. However the class is in `bookkeeper-server` module only. *Changes* Move this class to `bookkeeper-common` module Reviewers: Jia Zhai This closes #1841 from sijie/move_reflection_utils --- .../org/apache/bookkeeper/common}/util/ReflectionUtils.java | 4 +--- .../apache/bookkeeper/common}/util/TestReflectionUtils.java | 4 ++-- .../apache/bookkeeper/auth/AuthProviderFactoryFactory.java | 3 +-- .../org/apache/bookkeeper/bookie/LedgerStorageFactory.java | 2 +- .../main/java/org/apache/bookkeeper/client/BookKeeper.java | 2 +- .../client/RackawareEnsemblePlacementPolicyImpl.java | 2 +- .../org/apache/bookkeeper/conf/AbstractConfiguration.java | 2 +- .../java/org/apache/bookkeeper/conf/ClientConfiguration.java | 2 +- .../java/org/apache/bookkeeper/conf/ServerConfiguration.java | 2 +- .../bookkeeper/meta/AbstractZkLedgerManagerFactory.java | 2 +- .../main/java/org/apache/bookkeeper/meta/MetadataDrivers.java | 2 +- .../org/apache/bookkeeper/metastore/MetastoreFactory.java | 2 +- .../bookkeeper/server/component/ServerLifecycleComponent.java | 2 +- .../bookkeeper/server/service/StatsProviderService.java | 2 +- .../apache/bookkeeper/tls/SecurityProviderFactoryFactory.java | 2 +- .../main/java/org/apache/bookkeeper/util/EntryFormatter.java | 1 + .../java/org/apache/bookkeeper/util/LedgerIdFormatter.java | 1 + .../apache/distributedlog/DistributedLogConfiguration.java | 2 +- .../distributedlog/namespace/NamespaceDriverManager.java | 2 +- .../src/main/java/org/apache/distributedlog/tools/Tool.java | 2 +- .../apache/distributedlog/tools/TestDistributedLogTool.java | 2 +- .../tests/shaded/DistributedLogCoreShadedJarTest.java | 2 +- 22 files changed, 23 insertions(+), 24 deletions(-) rename {bookkeeper-server/src/main/java/org/apache/bookkeeper => bookkeeper-common/src/main/java/org/apache/bookkeeper/common}/util/ReflectionUtils.java (99%) rename {bookkeeper-server/src/test/java/org/apache/bookkeeper => bookkeeper-common/src/test/java/org/apache/bookkeeper/common}/util/TestReflectionUtils.java (95%) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ReflectionUtils.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/ReflectionUtils.java similarity index 99% rename from bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ReflectionUtils.java rename to bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/ReflectionUtils.java index 6297651c633..64cf34b93dd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ReflectionUtils.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/ReflectionUtils.java @@ -1,5 +1,4 @@ /* - * * 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 @@ -16,9 +15,8 @@ * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. - * */ -package org.apache.bookkeeper.util; +package org.apache.bookkeeper.common.util; import java.lang.reflect.Constructor; import java.util.Map; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestReflectionUtils.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestReflectionUtils.java similarity index 95% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestReflectionUtils.java rename to bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestReflectionUtils.java index 61712397e3d..dd1535fa2ce 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestReflectionUtils.java +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/TestReflectionUtils.java @@ -17,9 +17,9 @@ * under the License. */ -package org.apache.bookkeeper.util; +package org.apache.bookkeeper.common.util; -import static org.apache.bookkeeper.util.ReflectionUtils.forName; +import static org.apache.bookkeeper.common.util.ReflectionUtils.forName; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/AuthProviderFactoryFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/AuthProviderFactoryFactory.java index c5906fe1fc6..b121a056596 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/AuthProviderFactoryFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/AuthProviderFactoryFactory.java @@ -22,12 +22,11 @@ import java.io.IOException; import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.proto.BookieConnectionPeer; import org.apache.bookkeeper.proto.ClientConnectionPeer; -import org.apache.bookkeeper.util.ReflectionUtils; - /** * A factory to manage the authentication provider factories. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorageFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorageFactory.java index 6a9d0c44a1e..82d7aad666d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorageFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorageFactory.java @@ -22,7 +22,7 @@ import java.io.IOException; -import org.apache.bookkeeper.util.ReflectionUtils; +import org.apache.bookkeeper.common.util.ReflectionUtils; /** * A factory that creates {@link LedgerStorage} by reflection. 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 72edf51461a..a3c40079723 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 @@ -58,6 +58,7 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; @@ -77,7 +78,6 @@ import org.apache.bookkeeper.proto.DataFormats; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.bookkeeper.util.SafeRunnable; import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.lang.SystemUtils; 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 9c0feb7ee37..155229a6c65 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 @@ -48,6 +48,7 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.Configurable; import org.apache.bookkeeper.feature.FeatureProvider; @@ -62,7 +63,6 @@ import org.apache.bookkeeper.net.StabilizeNetworkTopology; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.commons.collections4.CollectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 a3c47abda7a..e395cb38253 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 @@ -30,6 +30,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.util.JsonUtil; import org.apache.bookkeeper.common.util.JsonUtil.ParseJsonException; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory; import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory; @@ -37,7 +38,6 @@ import org.apache.bookkeeper.meta.LongHierarchicalLedgerManagerFactory; import org.apache.bookkeeper.util.EntryFormatter; import org.apache.bookkeeper.util.LedgerIdFormatter; -import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.bookkeeper.util.StringEntryFormatter; import org.apache.commons.configuration.CompositeConfiguration; import org.apache.commons.configuration.ConfigurationException; 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 c902db823bb..86ec44457b6 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 @@ -26,10 +26,10 @@ import org.apache.bookkeeper.client.EnsemblePlacementPolicy; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; +import org.apache.bookkeeper.common.util.ReflectionUtils; 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; 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 e9c551e3dc4..afcfadcedfa 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 @@ -24,12 +24,12 @@ import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; import org.apache.bookkeeper.bookie.LedgerStorage; import org.apache.bookkeeper.bookie.SortedLedgerStorage; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.discover.ZKRegistrationManager; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.bookkeeper.util.BookKeeperConstants; -import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.commons.configuration.ConfigurationException; /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java index 42ec7344b5f..72bc3e841e6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java @@ -22,10 +22,10 @@ import java.util.List; import java.util.Objects; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.meta.LayoutManager.LedgerLayoutExistsException; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.commons.configuration.ConfigurationException; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZKUtil; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java index bab6d8423bb..44c25df5b53 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java @@ -37,13 +37,13 @@ import lombok.NoArgsConstructor; import lombok.ToString; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.exceptions.Code; import org.apache.bookkeeper.meta.exceptions.MetadataException; import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.lang3.StringUtils; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreFactory.java index 92f2eadcca9..847d2091521 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreFactory.java @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.metastore; -import org.apache.bookkeeper.util.ReflectionUtils; +import org.apache.bookkeeper.common.util.ReflectionUtils; /** * Metastore Factory. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/ServerLifecycleComponent.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/ServerLifecycleComponent.java index 281bb7d756d..cc8ee6ef7bf 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/ServerLifecycleComponent.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/component/ServerLifecycleComponent.java @@ -26,9 +26,9 @@ import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.common.component.LifecycleComponent; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.server.conf.BookieConfiguration; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.util.ReflectionUtils; /** * A {@link LifecycleComponent} that runs on a bookie server. It can be loaded via reflections. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/StatsProviderService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/StatsProviderService.java index ab5d894bfaa..ad734af7ce5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/StatsProviderService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/StatsProviderService.java @@ -19,11 +19,11 @@ package org.apache.bookkeeper.server.service; import java.io.IOException; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.server.component.ServerLifecycleComponent; import org.apache.bookkeeper.server.conf.BookieConfiguration; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsProvider; -import org.apache.bookkeeper.util.ReflectionUtils; /** * A {@link org.apache.bookkeeper.common.component.LifecycleComponent} that runs stats provider. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityProviderFactoryFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityProviderFactoryFactory.java index a6dad0b0059..6eb2c941586 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityProviderFactoryFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityProviderFactoryFactory.java @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.tls; -import org.apache.bookkeeper.util.ReflectionUtils; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EntryFormatter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EntryFormatter.java index c24e375d71f..4f2a3c393dd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EntryFormatter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EntryFormatter.java @@ -21,6 +21,7 @@ package org.apache.bookkeeper.util; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.AbstractConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java index f037f6d6d5e..8f676ffb39c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LedgerIdFormatter.java @@ -23,6 +23,7 @@ import java.util.UUID; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.AbstractConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java index 4052f4d19bf..7a71ba85dd3 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java @@ -25,11 +25,11 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.Set; +import org.apache.bookkeeper.common.util.ReflectionUtils; 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.bookkeeper.util.ReflectionUtils; import org.apache.commons.configuration.CompositeConfiguration; import org.apache.commons.configuration.Configuration; import org.apache.commons.configuration.ConfigurationException; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java index b35de8fe7fe..67b54a93fd6 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java @@ -26,7 +26,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import org.apache.bookkeeper.util.ReflectionUtils; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.commons.lang.StringUtils; import org.apache.distributedlog.DistributedLogConstants; import org.apache.distributedlog.impl.BKNamespaceDriver; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/Tool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/Tool.java index bc5603ccf09..31395ca4050 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/Tool.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/Tool.java @@ -19,7 +19,7 @@ import java.util.Map; import java.util.TreeMap; -import org.apache.bookkeeper.util.ReflectionUtils; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.HelpFormatter; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java index 753f23aa7c1..553e0172f65 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java @@ -23,7 +23,7 @@ import java.net.URI; import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; -import org.apache.bookkeeper.util.ReflectionUtils; +import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.distributedlog.DLMTestUtil; import org.apache.distributedlog.DLSN; import org.apache.distributedlog.DistributedLogConfiguration; diff --git a/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java b/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java index c4519dfdf99..6432c303a28 100644 --- a/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java +++ b/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java @@ -30,6 +30,7 @@ import static org.mockito.Mockito.when; +import dlshade.org.apache.bookkeeper.common.util.ReflectionUtils; import dlshade.org.apache.bookkeeper.conf.AbstractConfiguration; import dlshade.org.apache.bookkeeper.conf.ServerConfiguration; import dlshade.org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory; @@ -37,7 +38,6 @@ import dlshade.org.apache.bookkeeper.meta.LayoutManager; import dlshade.org.apache.bookkeeper.meta.LedgerLayout; import dlshade.org.apache.bookkeeper.meta.LedgerManagerFactory; -import dlshade.org.apache.bookkeeper.util.ReflectionUtils; import java.io.IOException; import org.junit.Test; import org.junit.runner.RunWith; From bfdb5bf02dfd6d32ffe647cbc9ace0a058f6e16a Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 28 Nov 2018 09:19:52 +0100 Subject: [PATCH 0143/1642] Use ${sha1} instead of ${ghprbActualCommit} for Pre-commit Jobs Descriptions of the changes in this PR: ### Motivation According to the documentation of the GitHub Pull Request plugin: - ${sha1} means a commit which is the result of merging the PR branch with the target branch - ${ghprbActualCommit} is directly the PR branch So currently we are not testing the PR branch against the target branch, but only the branch "as it is" see: https://www.google.com/url?sa=t&rct=j&q=&esrc=s&source=web&cd=4&ved=2ahUKEwje0PjpnuDeAhUF2SwKHXN4BgYQFjADegQIAxAB&url=https%3A%2F%2Fwiki.jenkins.io%2Fdisplay%2FJENKINS%2FGitHub%2Bpull%2Brequest%2Bbuilder%2Bplugin&usg=AOvVaw2XvPL5ynRYnqVEWqpe9cs1 ### Changes Use ${sha1} instead of ${ghprbActualCommit} for Pre-commit Jobs Reviewers: Ivan Kelly , Sijie Guo This closes #1818 from eolivelli/fix/jenskin-use-sha1 --- .../jenkins/job_bookkeeper_precommit_bookie_tests.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_client_tests.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_integrationtests.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_java8.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_java9.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_remaining_tests.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_replication_tests.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_validation.groovy | 2 +- 9 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy index f0e9ec085e4..8f8e5ae5017 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy @@ -35,7 +35,7 @@ freeStyleJob('bookkeeper_precommit_bookie_tests') { 'JDK 1.8 (latest)', 200, 'ubuntu', - '${ghprbActualCommit}') + '${sha1}') // Sets that this is a PreCommit job. common_job_properties.setPreCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy index b09607c51bd..9240099ae59 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy @@ -35,7 +35,7 @@ freeStyleJob('bookkeeper_precommit_client_tests') { 'JDK 1.8 (latest)', 200, 'ubuntu', - '${ghprbActualCommit}') + '${sha1}') // Sets that this is a PreCommit job. common_job_properties.setPreCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy index 7a169cb9da3..cda3d9be027 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy @@ -32,7 +32,7 @@ freeStyleJob('bookkeeper_precommit_integrationtests') { 'JDK 1.8 (latest)', 200, 'ubuntu', - '${ghprbActualCommit}') + '${sha1}') throttleConcurrentBuilds { // limit builds to 1 per node to avoid conflicts on building docker images diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy index 67d7646432e..6a888fdbe06 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy @@ -44,7 +44,7 @@ mavenJob('bookkeeper_precommit_pullrequest_java8') { 'JDK 1.8 (latest)', 200, 'ubuntu', - '${ghprbActualCommit}') + '${sha1}') // Sets that this is a PreCommit job. common_job_properties.setPreCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy index 43cb02b69f0..6b02857bb94 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy @@ -44,7 +44,7 @@ mavenJob('bookkeeper_precommit_pullrequest_java9') { 'JDK 1.9 (latest)', 200, 'ubuntu', - '${ghprbActualCommit}') + '${sha1}') // Sets that this is a PreCommit job. common_job_properties.setPreCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy index d57605bdf96..f3f560393da 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy @@ -39,7 +39,7 @@ freeStyleJob('bookkeeper_precommit_remaining_tests') { 'JDK 1.8 (latest)', 200, 'ubuntu', - '${ghprbActualCommit}') + '${sha1}') // Sets that this is a PreCommit job. common_job_properties.setPreCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy index 61893bac041..21342800582 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy @@ -35,7 +35,7 @@ freeStyleJob('bookkeeper_precommit_replication_tests') { 'JDK 1.8 (latest)', 200, 'ubuntu', - '${ghprbActualCommit}') + '${sha1}') // Sets that this is a PreCommit job. common_job_properties.setPreCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy index 1adc19e89a4..af0038d4cf2 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy @@ -35,7 +35,7 @@ freeStyleJob('bookkeeper_precommit_tls_tests') { 'JDK 1.8 (latest)', 200, 'ubuntu', - '${ghprbActualCommit}') + '${sha1}') // Sets that this is a PreCommit job. common_job_properties.setPreCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy index 57a2791ce5b..e9336e070fc 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy @@ -35,7 +35,7 @@ mavenJob('bookkeeper_precommit_pullrequest_validation') { 'JDK 1.8 (latest)', 200, 'ubuntu', - '${ghprbActualCommit}') + '${sha1}') // Sets that this is a PreCommit job. common_job_properties.setPreCommit( From 1efb12795bd43d6af23404aadf3ca77fe2de06dd Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Wed, 28 Nov 2018 16:31:13 +0100 Subject: [PATCH 0144/1642] Only publish suspect ledgers if they have missing fragments This is a fix for a flake in AuditorPeriodicCheckTest#testIndexCorruption. Auditor#checkAllLedgers() runs a check on all ledgers, passing ProcessLostFragmentsCb as a callback to LedgerChecker#checkLedger for each one. LedgerChecker#checkLedger triggers the callback on completion, regardless of whether there are fragments missing on not. Previously, ProcessLostFragments was not checking if there were lost fragments before publishing the ledger as suspected in zookeeper. The flake triggered as there were always two ledgers that existed when the check occurred, both would be reported as suspected, and it was random which would be returned while polling for underreplicated ledgers. The fix is to check that something is actually underreplicated before publishing. Reviewers: Sijie Guo This closes #1834 from ivankelly/auditor-periodic-flake --- .../main/java/org/apache/bookkeeper/replication/Auditor.java | 5 +++++ 1 file changed, 5 insertions(+) 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 1fcd0f88877..acf0c09c1bd 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 @@ -647,6 +647,11 @@ public void operationComplete(int rc, Set fragments) { for (LedgerFragment f : fragments) { bookies.addAll(f.getAddresses()); } + if (bookies.isEmpty()) { + // no missing fragments + callback.processResult(Code.OK, null, null); + return; + } publishSuspectedLedgersAsync( bookies.stream().map(BookieSocketAddress::toString).collect(Collectors.toList()), Sets.newHashSet(lh.getId()) From e73b1feb9f6536868e913c2ae04c2cc2af19788f Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 28 Nov 2018 12:07:29 -0800 Subject: [PATCH 0145/1642] [LEDGER STORAGE] DbLedgerStorage should do periodical flush Descriptions of the changes in this PR: *Motivation* DbLedgerStorage doesn't drive checkpoint itself. so currently DbLedgerStorage flush only happens either when write-cache is full or entry log file rotated. The correctness is still maintained. However the behavior is different from original yahoo behavior. *Changes* Revert the behavior back to original periodical flush Reviewers: Matteo Merli , Jia Zhai This closes #1843 from sijie/dbledgerstorage_sync --- .../org/apache/bookkeeper/bookie/Bookie.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) 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 f91bf285b4f..ffb92ed2d76 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 @@ -63,6 +63,7 @@ import org.apache.bookkeeper.bookie.LedgerDirsManager.LedgerDirsListener; import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException; import org.apache.bookkeeper.bookie.stats.BookieStats; +import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationManager; @@ -681,14 +682,22 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger) LOG.info("Using ledger storage: {}", ledgerStorageClass); ledgerStorage = LedgerStorageFactory.createLedgerStorage(ledgerStorageClass); + boolean isDbLedgerStorage = ledgerStorage instanceof DbLedgerStorage; + /* * with this change https://github.com/apache/bookkeeper/pull/677, - * LedgerStorage drives the checkpoint logic. But with multiple entry - * logs, checkpoint logic based on a entry log is not possible, hence it - * needs to be timebased recurring thing and it is driven by SyncThread. - * SyncThread.start does that and it is started in Bookie.start method. + * LedgerStorage drives the checkpoint logic. + * + *

            There are two exceptions: + * + * 1) with multiple entry logs, checkpoint logic based on a entry log is + * not possible, hence it needs to be timebased recurring thing and + * it is driven by SyncThread. SyncThread.start does that and it is + * started in Bookie.start method. + * + * 2) DbLedgerStorage */ - if (entryLogPerLedgerEnabled) { + if (entryLogPerLedgerEnabled || isDbLedgerStorage) { syncThread = new SyncThread(conf, getLedgerDirsListener(), ledgerStorage, checkpointSource) { @Override public void startCheckpoint(Checkpoint checkpoint) { From 9c2a017b4bd1a762771a31c1781382538b56c950 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 28 Nov 2018 18:10:38 -0800 Subject: [PATCH 0146/1642] [RELEASE] fix docker mount volume issue Descriptions of the changes in this PR: * Motivation #1674 introduced a regression for running release scripts on mac. the destination should be "/home/${USER}" rather than ${LOCAL_HOME}. because the docker os is linux. Otherwise the gpg keys can not propagated correctly from host machine to docker. * Changes Use `/home/${USER}` as the destination path Reviewers: Jia Zhai This closes #1845 from sijie/fix_release_scripts --- dev/release/000-run-docker.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dev/release/000-run-docker.sh b/dev/release/000-run-docker.sh index d63e2221b3b..259b46e9902 100755 --- a/dev/release/000-run-docker.sh +++ b/dev/release/000-run-docker.sh @@ -41,12 +41,10 @@ if [ "$(uname -s)" == "Linux" ]; then USER_NAME=${SUDO_USER:=$USER} USER_ID=$(id -u "${USER_NAME}") GROUP_ID=$(id -g "${USER_NAME}") - LOCAL_HOME="/home/${USER_NAME}" else # boot2docker uid and gid USER_NAME=$USER USER_ID=1000 GROUP_ID=50 - LOCAL_HOME="/Users/${USER_NAME}" fi docker build -t "${IMAGE_NAME}-${USER_NAME}" - < Date: Wed, 28 Nov 2018 18:41:34 -0800 Subject: [PATCH 0147/1642] [STATS] Add @StatsDoc annotation for bookie server request stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document the request stats for bookkeeper server. *Changes* - add `parent` and `happensAfter` in StatsDoc - convert bookie request stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Jia Zhai This closes #1839 from sijie/bp36_add_parent_and_happensafter --- .../bookkeeper/bookie/stats/BookieStats.java | 14 +- .../proto/BookieRequestProcessor.java | 185 ++-------- .../proto/ForceLedgerProcessorV3.java | 11 +- .../proto/GetBookieInfoProcessorV3.java | 10 +- .../proto/LongPollReadEntryProcessorV3.java | 6 +- .../bookkeeper/proto/PacketProcessorBase.java | 2 +- .../proto/PacketProcessorBaseV3.java | 6 +- .../bookkeeper/proto/ReadEntryProcessor.java | 13 +- .../proto/ReadEntryProcessorV3.java | 20 +- .../bookkeeper/proto/ReadLacProcessorV3.java | 10 +- .../apache/bookkeeper/proto/RequestStats.java | 342 ++++++++++++++++++ .../bookkeeper/proto/WriteEntryProcessor.java | 18 +- .../proto/WriteEntryProcessorV3.java | 12 +- .../bookkeeper/proto/WriteLacProcessorV3.java | 19 +- .../proto/ForceLedgerProcessorV3Test.java | 5 +- .../proto/WriteEntryProcessorTest.java | 5 +- .../proto/WriteEntryProcessorV3Test.java | 7 +- .../stats/annotations/StatsDoc.java | 20 + 18 files changed, 470 insertions(+), 235 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestStats.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java index 72921d72971..5e033e9e9f3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java @@ -19,6 +19,7 @@ package org.apache.bookkeeper.bookie.stats; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY; 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_FORCE_LEDGER; @@ -28,6 +29,7 @@ import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_BYTES; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_BYTES; import lombok.Getter; @@ -56,11 +58,19 @@ public class BookieStats { @StatsDoc(name = BOOKIE_FORCE_LEDGER, help = "total force operations occurred on a bookie") private final Counter forceLedgerOps; // Bookie Operation Latency Stats - @StatsDoc(name = BOOKIE_ADD_ENTRY, help = "operations stats of AddEntry on a bookie") + @StatsDoc( + name = BOOKIE_ADD_ENTRY, + help = "operations stats of AddEntry on a bookie", + parent = ADD_ENTRY + ) private final OpStatsLogger addEntryStats; @StatsDoc(name = BOOKIE_RECOVERY_ADD_ENTRY, help = "operation stats of RecoveryAddEntry on a bookie") private final OpStatsLogger recoveryAddEntryStats; - @StatsDoc(name = BOOKIE_READ_ENTRY, help = "operation stats of ReadEntry on a bookie") + @StatsDoc( + name = BOOKIE_READ_ENTRY, + help = "operation stats of ReadEntry on a bookie", + parent = READ_ENTRY + ) private final OpStatsLogger readEntryStats; // Bookie Operation Bytes Stats @StatsDoc(name = BOOKIE_ADD_ENTRY_BYTES, help = "bytes stats of AddEntry on a bookie") 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 78b35ec6cd2..b883f74a21f 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 @@ -21,34 +21,6 @@ package org.apache.bookkeeper.proto; import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY_BLOCKED; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY_BLOCKED_WAIT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY_IN_PROGRESS; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY_REQUEST; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CHANNEL_WRITE; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FORCE_LEDGER; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FORCE_LEDGER_REQUEST; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.GET_BOOKIE_INFO; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.GET_BOOKIE_INFO_REQUEST; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_BLOCKED; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_BLOCKED_WAIT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_FENCE_READ; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_FENCE_REQUEST; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_FENCE_WAIT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_IN_PROGRESS; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_LONG_POLL_PRE_WAIT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_LONG_POLL_READ; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_LONG_POLL_REQUEST; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_LONG_POLL_WAIT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_REQUEST; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_SCHEDULING_DELAY; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_LAC; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_LAC_REQUEST; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_LAST_ENTRY_NOENTRY_ERROR; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_LAC; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_LAC_REQUEST; import static org.apache.bookkeeper.proto.RequestUtils.hasFlag; import com.google.common.annotations.VisibleForTesting; @@ -68,7 +40,6 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import lombok.AccessLevel; @@ -80,9 +51,6 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.processor.RequestProcessor; -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.Gauge; -import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.tls.SecurityHandlerFactory; @@ -146,37 +114,8 @@ public class BookieRequestProcessor implements RequestProcessor { // Expose Stats private final BKStats bkStats = BKStats.getInstance(); private final boolean statsEnabled; - private final OpStatsLogger addRequestStats; - private final OpStatsLogger addEntryStats; - final OpStatsLogger readRequestStats; - final OpStatsLogger readEntryStats; - final OpStatsLogger forceLedgerStats; - final OpStatsLogger forceLedgerRequestStats; - final OpStatsLogger fenceReadRequestStats; - final OpStatsLogger fenceReadEntryStats; - final OpStatsLogger fenceReadWaitStats; - final OpStatsLogger readEntrySchedulingDelayStats; - final OpStatsLogger longPollPreWaitStats; - final OpStatsLogger longPollWaitStats; - final OpStatsLogger longPollReadStats; - final OpStatsLogger longPollReadRequestStats; - final Counter readLastEntryNoEntryErrorCounter; - final OpStatsLogger writeLacRequestStats; - final OpStatsLogger writeLacStats; - final OpStatsLogger readLacRequestStats; - final OpStatsLogger readLacStats; - final OpStatsLogger getBookieInfoRequestStats; - final OpStatsLogger getBookieInfoStats; - final OpStatsLogger channelWriteStats; - final OpStatsLogger addEntryBlockedStats; - final OpStatsLogger readEntryBlockedStats; - - final AtomicInteger addsInProgress = new AtomicInteger(0); - final AtomicInteger maxAddsInProgress = new AtomicInteger(0); - final AtomicInteger addsBlocked = new AtomicInteger(0); - final AtomicInteger readsInProgress = new AtomicInteger(0); - final AtomicInteger readsBlocked = new AtomicInteger(0); - final AtomicInteger maxReadsInProgress = new AtomicInteger(0); + + private final RequestStats requestStats; final Semaphore addsSemaphore; final Semaphore readsSemaphore; @@ -248,86 +187,13 @@ public BookieRequestProcessor(ServerConfiguration serverCfg, Bookie bookie, // Expose Stats this.statsEnabled = serverCfg.isStatisticsEnabled(); - this.addEntryStats = statsLogger.getOpStatsLogger(ADD_ENTRY); - this.addRequestStats = statsLogger.getOpStatsLogger(ADD_ENTRY_REQUEST); - this.readEntryStats = statsLogger.getOpStatsLogger(READ_ENTRY); - this.forceLedgerStats = statsLogger.getOpStatsLogger(FORCE_LEDGER); - this.forceLedgerRequestStats = statsLogger.getOpStatsLogger(FORCE_LEDGER_REQUEST); - this.readRequestStats = statsLogger.getOpStatsLogger(READ_ENTRY_REQUEST); - this.fenceReadEntryStats = statsLogger.getOpStatsLogger(READ_ENTRY_FENCE_READ); - this.fenceReadRequestStats = statsLogger.getOpStatsLogger(READ_ENTRY_FENCE_REQUEST); - this.fenceReadWaitStats = statsLogger.getOpStatsLogger(READ_ENTRY_FENCE_WAIT); - this.readEntrySchedulingDelayStats = statsLogger.getOpStatsLogger(READ_ENTRY_SCHEDULING_DELAY); - this.longPollPreWaitStats = statsLogger.getOpStatsLogger(READ_ENTRY_LONG_POLL_PRE_WAIT); - this.longPollWaitStats = statsLogger.getOpStatsLogger(READ_ENTRY_LONG_POLL_WAIT); - this.longPollReadStats = statsLogger.getOpStatsLogger(READ_ENTRY_LONG_POLL_READ); - this.longPollReadRequestStats = statsLogger.getOpStatsLogger(READ_ENTRY_LONG_POLL_REQUEST); - this.readLastEntryNoEntryErrorCounter = statsLogger.getCounter(READ_LAST_ENTRY_NOENTRY_ERROR); - this.writeLacStats = statsLogger.getOpStatsLogger(WRITE_LAC); - this.writeLacRequestStats = statsLogger.getOpStatsLogger(WRITE_LAC_REQUEST); - this.readLacStats = statsLogger.getOpStatsLogger(READ_LAC); - this.readLacRequestStats = statsLogger.getOpStatsLogger(READ_LAC_REQUEST); - this.getBookieInfoStats = statsLogger.getOpStatsLogger(GET_BOOKIE_INFO); - this.getBookieInfoRequestStats = statsLogger.getOpStatsLogger(GET_BOOKIE_INFO_REQUEST); - this.channelWriteStats = statsLogger.getOpStatsLogger(CHANNEL_WRITE); - - this.addEntryBlockedStats = statsLogger.getOpStatsLogger(ADD_ENTRY_BLOCKED_WAIT); - this.readEntryBlockedStats = statsLogger.getOpStatsLogger(READ_ENTRY_BLOCKED_WAIT); + this.requestStats = new RequestStats(statsLogger); int maxAdds = serverCfg.getMaxAddsInProgressLimit(); addsSemaphore = maxAdds > 0 ? new Semaphore(maxAdds, true) : null; int maxReads = serverCfg.getMaxReadsInProgressLimit(); readsSemaphore = maxReads > 0 ? new Semaphore(maxReads, true) : null; - - statsLogger.registerGauge(ADD_ENTRY_IN_PROGRESS, new Gauge() { - @Override - public Number getDefaultValue() { - return 0; - } - - @Override - public Number getSample() { - return addsInProgress; - } - }); - - statsLogger.registerGauge(ADD_ENTRY_BLOCKED, new Gauge() { - @Override - public Number getDefaultValue() { - return 0; - } - - @Override - public Number getSample() { - return addsBlocked; - } - }); - - statsLogger.registerGauge(READ_ENTRY_IN_PROGRESS, new Gauge() { - @Override - public Number getDefaultValue() { - return 0; - } - - @Override - public Number getSample() { - return readsInProgress; - } - }); - - statsLogger.registerGauge(READ_ENTRY_BLOCKED, new Gauge() { - @Override - public Number getDefaultValue() { - return 0; - } - - @Override - public Number getSample() { - return readsBlocked; - } - }); - } protected void onAddRequestStart(Channel channel) { @@ -336,21 +202,19 @@ protected void onAddRequestStart(Channel channel) { final long throttlingStartTimeNanos = MathUtils.nowInNano(); channel.config().setAutoRead(false); LOG.info("Too many add requests in progress, disabling autoread on channel {}", channel); - addsBlocked.incrementAndGet(); + requestStats.blockAddRequest(); addsSemaphore.acquireUninterruptibly(); channel.config().setAutoRead(true); final long delayNanos = MathUtils.elapsedNanos(throttlingStartTimeNanos); LOG.info("Re-enabled autoread on channel {} after AddRequest delay of {} nanos", channel, delayNanos); - addEntryBlockedStats.registerSuccessfulEvent(delayNanos, TimeUnit.NANOSECONDS); - addsBlocked.decrementAndGet(); + requestStats.unblockAddRequest(delayNanos); } } - final int curr = addsInProgress.incrementAndGet(); - maxAddsInProgress.accumulateAndGet(curr, Integer::max); + requestStats.trackAddRequest(); } protected void onAddRequestFinish() { - addsInProgress.decrementAndGet(); + requestStats.untrackAddRequest(); if (addsSemaphore != null) { addsSemaphore.release(); } @@ -362,21 +226,19 @@ protected void onReadRequestStart(Channel channel) { final long throttlingStartTimeNanos = MathUtils.nowInNano(); channel.config().setAutoRead(false); LOG.info("Too many read requests in progress, disabling autoread on channel {}", channel); - readsBlocked.incrementAndGet(); + requestStats.blockReadRequest(); readsSemaphore.acquireUninterruptibly(); channel.config().setAutoRead(true); final long delayNanos = MathUtils.elapsedNanos(throttlingStartTimeNanos); LOG.info("Re-enabled autoread on channel {} after ReadRequest delay of {} nanos", channel, delayNanos); - readEntryBlockedStats.registerSuccessfulEvent(delayNanos, TimeUnit.NANOSECONDS); - readsBlocked.decrementAndGet(); + requestStats.unblockReadRequest(delayNanos); } } - final int curr = readsInProgress.incrementAndGet(); - maxReadsInProgress.accumulateAndGet(curr, Integer::max); + requestStats.trackReadRequest(); } protected void onReadRequestFinish() { - readsInProgress.decrementAndGet(); + requestStats.untrackReadRequest(); if (readsSemaphore != null) { readsSemaphore.release(); } @@ -384,12 +246,12 @@ protected void onReadRequestFinish() { @VisibleForTesting int maxAddsInProgressCount() { - return maxAddsInProgress.get(); + return requestStats.maxAddsInProgressCount(); } @VisibleForTesting int maxReadsInProgressCount() { - return maxReadsInProgress.get(); + return requestStats.maxReadsInProgressCount(); } @Override @@ -576,7 +438,7 @@ private void processAddRequestV3(final BookkeeperProtocol.Request r, final Chann .setStatus(addResponse.getStatus()) .setAddResponse(addResponse); BookkeeperProtocol.Response resp = response.build(); - write.sendResponse(addResponse.getStatus(), resp, addRequestStats); + write.sendResponse(addResponse.getStatus(), resp, requestStats.getAddRequestStats()); } } } @@ -610,7 +472,10 @@ private void processForceLedgerRequestV3(final BookkeeperProtocol.Request r, fin .setStatus(forceLedgerResponse.getStatus()) .setForceLedgerResponse(forceLedgerResponse); BookkeeperProtocol.Response resp = response.build(); - forceLedger.sendResponse(forceLedgerResponse.getStatus(), resp, forceLedgerRequestStats); + forceLedger.sendResponse( + forceLedgerResponse.getStatus(), + resp, + requestStats.getForceLedgerRequestStats()); } } } @@ -660,7 +525,7 @@ private void processReadRequestV3(final BookkeeperProtocol.Request r, final Chan .setStatus(readResponse.getStatus()) .setReadResponse(readResponse); BookkeeperProtocol.Response resp = response.build(); - read.sendResponse(readResponse.getStatus(), resp, readRequestStats); + read.sendResponse(readResponse.getStatus(), resp, requestStats.getReadRequestStats()); } } } @@ -740,8 +605,10 @@ private void processAddRequest(final BookieProtocol.ParsedAddRequest r, final Ch r.entryId); } - write.sendResponse(BookieProtocol.ETOOMANYREQUESTS, - ResponseBuilder.buildErrorResponse(BookieProtocol.ETOOMANYREQUESTS, r), addRequestStats); + write.sendResponse( + BookieProtocol.ETOOMANYREQUESTS, + ResponseBuilder.buildErrorResponse(BookieProtocol.ETOOMANYREQUESTS, r), + requestStats.getAddRequestStats()); } } } @@ -770,8 +637,10 @@ private void processReadRequest(final BookieProtocol.ReadRequest r, final Channe r.entryId); } - read.sendResponse(BookieProtocol.ETOOMANYREQUESTS, - ResponseBuilder.buildErrorResponse(BookieProtocol.ETOOMANYREQUESTS, r), readRequestStats); + read.sendResponse( + BookieProtocol.ETOOMANYREQUESTS, + ResponseBuilder.buildErrorResponse(BookieProtocol.ETOOMANYREQUESTS, r), + requestStats.getReadRequestStats()); } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java index 0c8ef01fa87..f8891722a9e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java @@ -67,11 +67,11 @@ private ForceLedgerResponse getForceLedgerResponse() { "ledgerId must be {} but was {}", ledgerId, ledgerId1); if (BookieProtocol.EOK == rc) { - requestProcessor.getForceLedgerStats() + requestProcessor.getRequestStats().getForceLedgerStats() .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } else { - requestProcessor.getForceLedgerStats() + requestProcessor.getRequestStats().getForceLedgerStats() .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } @@ -94,7 +94,7 @@ private ForceLedgerResponse getForceLedgerResponse() { .setStatus(forceLedgerResponse.getStatus()) .setForceLedgerResponse(forceLedgerResponse); Response resp = response.build(); - sendResponse(status, resp, requestProcessor.getForceLedgerRequestStats()); + sendResponse(status, resp, requestProcessor.getRequestStats().getForceLedgerRequestStats()); }; StatusCode status = null; try { @@ -124,7 +124,10 @@ public void safeRun() { .setStatus(forceLedgerResponse.getStatus()) .setForceLedgerResponse(forceLedgerResponse); Response resp = response.build(); - sendResponse(forceLedgerResponse.getStatus(), resp, requestProcessor.getForceLedgerRequestStats()); + sendResponse( + forceLedgerResponse.getStatus(), + resp, + requestProcessor.getRequestStats().getForceLedgerRequestStats()); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java index d964957488b..fe315f62cae 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java @@ -54,8 +54,8 @@ private GetBookieInfoResponse getGetBookieInfoResponse() { if (!isVersionCompatible()) { getBookieInfoResponse.setStatus(StatusCode.EBADVERSION); - requestProcessor.getGetBookieInfoStats().registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getGetBookieInfoStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); return getBookieInfoResponse.build(); } @@ -80,8 +80,8 @@ private GetBookieInfoResponse getGetBookieInfoResponse() { } getBookieInfoResponse.setStatus(status); - requestProcessor.getGetBookieInfoStats().registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getGetBookieInfoStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); return getBookieInfoResponse.build(); } @@ -98,6 +98,6 @@ private void sendResponse(GetBookieInfoResponse getBookieInfoResponse) { .setGetBookieInfoResponse(getBookieInfoResponse); sendResponse(response.getStatus(), response.build(), - requestProcessor.getGetBookieInfoRequestStats()); + requestProcessor.getRequestStats().getGetBookieInfoRequestStats()); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java index fdbbd353043..6f25d684b6c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java @@ -101,7 +101,7 @@ protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, try { return super.readEntry(readResponseBuilder, entryId, true, startTimeSw); } catch (Bookie.NoEntryException e) { - requestProcessor.readLastEntryNoEntryErrorCounter.inc(); + requestProcessor.getRequestStats().getReadLastEntryNoEntryErrorCounter().inc(); logger.info( "No entry found while piggyback reading entry {} from ledger {} : previous lac = {}", entryId, ledgerId, previousLAC); @@ -153,7 +153,7 @@ private ReadResponse getLongPollReadResponse() { return buildErrorResponse(StatusCode.EIO, startTimeSw); } - registerSuccessfulEvent(requestProcessor.longPollPreWaitStats, startTimeSw); + registerSuccessfulEvent(requestProcessor.getRequestStats().getLongPollPreWaitStats(), startTimeSw); lastPhaseStartTime.reset().start(); if (watched) { @@ -213,7 +213,7 @@ private synchronized void scheduleDeferredRead(boolean timeout) { expirationTimerTask.cancel(); } - registerEvent(timeout, requestProcessor.longPollWaitStats, lastPhaseStartTime); + registerEvent(timeout, requestProcessor.getRequestStats().getLongPollWaitStats(), lastPhaseStartTime); lastPhaseStartTime.reset().start(); } } 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 b7dee2d4a8d..54368a0c07b 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 @@ -79,7 +79,7 @@ public void safeRun() { if (!isVersionCompatible()) { sendResponse(BookieProtocol.EBADVERSION, ResponseBuilder.buildErrorResponse(BookieProtocol.EBADVERSION, request), - requestProcessor.readRequestStats); + requestProcessor.getRequestStats().getReadRequestStats()); return; } processPacket(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java index 7dc29a38bf1..15765a252b2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java @@ -79,7 +79,7 @@ protected void sendResponse(StatusCode code, Object response, OpStatsLogger stat if (!channel.isWritable()) { LOGGER.warn("cannot write response to non-writable channel {} for request {}", channel, StringUtils.requestToString(request)); - requestProcessor.getChannelWriteStats() + requestProcessor.getRequestStats().getChannelWriteStats() .registerFailedEvent(MathUtils.elapsedNanos(writeNanos), TimeUnit.NANOSECONDS); statsLogger.registerFailedEvent(MathUtils.elapsedNanos(enqueueNanos), TimeUnit.NANOSECONDS); return; @@ -93,10 +93,10 @@ protected void sendResponse(StatusCode code, Object response, OpStatsLogger stat public void operationComplete(ChannelFuture future) throws Exception { long writeElapsedNanos = MathUtils.elapsedNanos(writeNanos); if (!future.isSuccess()) { - requestProcessor.getChannelWriteStats() + requestProcessor.getRequestStats().getChannelWriteStats() .registerFailedEvent(writeElapsedNanos, TimeUnit.NANOSECONDS); } else { - requestProcessor.getChannelWriteStats() + requestProcessor.getRequestStats().getChannelWriteStats() .registerSuccessfulEvent(writeElapsedNanos, TimeUnit.NANOSECONDS); } if (StatusCode.EOK == code) { 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 edeb8a674f7..6566c7b4643 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 @@ -20,7 +20,6 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; import io.netty.util.Recycler; -import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; import java.io.IOException; @@ -129,17 +128,17 @@ protected void processPacket() { LOG.trace("Read entry rc = {} for {}", errorCode, request); } if (errorCode == BookieProtocol.EOK) { - requestProcessor.readEntryStats.registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getReadEntryStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); sendResponse(errorCode, ResponseBuilder.buildReadResponse(data, request), - requestProcessor.readRequestStats); + requestProcessor.getRequestStats().getReadRequestStats()); } else { ReferenceCountUtil.release(data); - requestProcessor.readEntryStats.registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getReadEntryStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); sendResponse(errorCode, ResponseBuilder.buildErrorResponse(errorCode, request), - requestProcessor.readRequestStats); + requestProcessor.getRequestStats().getReadRequestStats()); } recycle(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java index e7e56533317..88b76627071 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java @@ -71,14 +71,14 @@ public ReadEntryProcessorV3(Request request, this.ledgerId = readRequest.getLedgerId(); this.entryId = readRequest.getEntryId(); if (RequestUtils.isFenceRequest(this.readRequest)) { - this.readStats = requestProcessor.fenceReadEntryStats; - this.reqStats = requestProcessor.fenceReadRequestStats; + this.readStats = requestProcessor.getRequestStats().getFenceReadEntryStats(); + this.reqStats = requestProcessor.getRequestStats().getFenceReadRequestStats(); } else if (readRequest.hasPreviousLAC()) { - this.readStats = requestProcessor.longPollReadStats; - this.reqStats = requestProcessor.longPollReadRequestStats; + this.readStats = requestProcessor.getRequestStats().getLongPollReadStats(); + this.reqStats = requestProcessor.getRequestStats().getLongPollReadRequestStats(); } else { - this.readStats = requestProcessor.readEntryStats; - this.reqStats = requestProcessor.readRequestStats; + this.readStats = requestProcessor.getRequestStats().getReadEntryStats(); + this.reqStats = requestProcessor.getRequestStats().getReadRequestStats(); } this.fenceThreadPool = fenceThreadPool; @@ -246,7 +246,7 @@ protected ReadResponse getReadResponse() { @Override public void safeRun() { - requestProcessor.readEntrySchedulingDelayStats.registerSuccessfulEvent( + requestProcessor.getRequestStats().getReadEntrySchedulingDelayStats().registerSuccessfulEvent( MathUtils.elapsedNanos(enqueueNanos), TimeUnit.NANOSECONDS); if (!isVersionCompatible()) { @@ -275,11 +275,11 @@ private void getFenceResponse(ReadResponse.Builder readResponse, StatusCode status; if (!fenceResult) { status = StatusCode.EIO; - registerFailedEvent(requestProcessor.fenceReadWaitStats, lastPhaseStartTime); + registerFailedEvent(requestProcessor.getRequestStats().getFenceReadWaitStats(), lastPhaseStartTime); } else { status = StatusCode.EOK; readResponse.setBody(ByteString.copyFrom(entryBody.nioBuffer())); - registerSuccessfulEvent(requestProcessor.fenceReadWaitStats, lastPhaseStartTime); + registerSuccessfulEvent(requestProcessor.getRequestStats().getFenceReadWaitStats(), lastPhaseStartTime); } if (null != entryBody) { @@ -296,7 +296,7 @@ private void sendFenceResponse(ReadResponse.Builder readResponse, // build the fence read response getFenceResponse(readResponse, entryBody, fenceResult); // register fence read stat - registerEvent(!fenceResult, requestProcessor.fenceReadEntryStats, startTimeSw); + registerEvent(!fenceResult, requestProcessor.getRequestStats().getFenceReadEntryStats(), startTimeSw); // send the fence read response sendResponse(readResponse.build()); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java index 898ddb0413b..a3bc31118c1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java @@ -102,11 +102,11 @@ private ReadLacResponse getReadLacResponse() { } if (status == StatusCode.EOK) { - requestProcessor.readLacStats.registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getReadLacStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } else { - requestProcessor.readLacStats.registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getReadLacStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } // Finally set the status and return readLacResponse.setStatus(status); @@ -126,6 +126,6 @@ private void sendResponse(ReadLacResponse readLacResponse) { .setReadLacResponse(readLacResponse); sendResponse(response.getStatus(), response.build(), - requestProcessor.readLacRequestStats); + requestProcessor.getRequestStats().getReadLacRequestStats()); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestStats.java new file mode 100644 index 00000000000..1799e660d77 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestStats.java @@ -0,0 +1,342 @@ +/* + * 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.proto; + +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY_BLOCKED; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY_BLOCKED_WAIT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY_IN_PROGRESS; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY_REQUEST; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CHANNEL_WRITE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FORCE_LEDGER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FORCE_LEDGER_REQUEST; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.GET_BOOKIE_INFO; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.GET_BOOKIE_INFO_REQUEST; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_BLOCKED; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_BLOCKED_WAIT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_FENCE_READ; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_FENCE_REQUEST; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_FENCE_WAIT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_IN_PROGRESS; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_LONG_POLL_PRE_WAIT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_LONG_POLL_READ; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_LONG_POLL_REQUEST; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_LONG_POLL_WAIT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_REQUEST; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_SCHEDULING_DELAY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_LAC; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_LAC_REQUEST; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_LAST_ENTRY_NOENTRY_ERROR; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SERVER_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_LAC; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_LAC_REQUEST; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Getter; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for request related stats. + */ +@StatsDoc( + name = SERVER_SCOPE, + category = CATEGORY_SERVER, + help = "Bookie request stats" +) +@Getter +public class RequestStats { + + final AtomicInteger addsInProgress = new AtomicInteger(0); + final AtomicInteger maxAddsInProgress = new AtomicInteger(0); + final AtomicInteger addsBlocked = new AtomicInteger(0); + final AtomicInteger readsInProgress = new AtomicInteger(0); + final AtomicInteger readsBlocked = new AtomicInteger(0); + final AtomicInteger maxReadsInProgress = new AtomicInteger(0); + + @StatsDoc( + name = ADD_ENTRY_REQUEST, + help = "request stats of AddEntry on a bookie" + ) + private final OpStatsLogger addRequestStats; + @StatsDoc( + name = ADD_ENTRY, + help = "operation stats of AddEntry on a bookie", + parent = ADD_ENTRY_REQUEST + ) + private final OpStatsLogger addEntryStats; + @StatsDoc( + name = READ_ENTRY_REQUEST, + help = "request stats of ReadEntry on a bookie" + ) + final OpStatsLogger readRequestStats; + @StatsDoc( + name = READ_ENTRY, + help = "operation stats of ReadEntry on a bookie", + parent = READ_ENTRY_REQUEST + ) + final OpStatsLogger readEntryStats; + @StatsDoc( + name = FORCE_LEDGER, + help = "operation stats of ForceLedger on a bookie", + parent = FORCE_LEDGER_REQUEST + ) + final OpStatsLogger forceLedgerStats; + @StatsDoc( + name = FORCE_LEDGER_REQUEST, + help = "request stats of ForceLedger on a bookie" + ) + final OpStatsLogger forceLedgerRequestStats; + @StatsDoc( + name = READ_ENTRY_FENCE_REQUEST, + help = "request stats of FenceRead on a bookie" + ) + final OpStatsLogger fenceReadRequestStats; + @StatsDoc( + name = READ_ENTRY_FENCE_READ, + help = "operation stats of FenceRead on a bookie", + parent = READ_ENTRY_FENCE_REQUEST, + happensAfter = READ_ENTRY_FENCE_WAIT + ) + final OpStatsLogger fenceReadEntryStats; + @StatsDoc( + name = READ_ENTRY_FENCE_WAIT, + help = "operation stats of FenceReadWait on a bookie", + parent = READ_ENTRY_FENCE_REQUEST + ) + final OpStatsLogger fenceReadWaitStats; + @StatsDoc( + name = READ_ENTRY_SCHEDULING_DELAY, + help = "operation stats of ReadEntry scheduling delays on a bookie" + ) + final OpStatsLogger readEntrySchedulingDelayStats; + @StatsDoc( + name = READ_ENTRY_LONG_POLL_PRE_WAIT, + help = "operation stats of LongPoll Reads pre wait time on a bookie", + parent = READ_ENTRY_LONG_POLL_REQUEST + ) + final OpStatsLogger longPollPreWaitStats; + @StatsDoc( + name = READ_ENTRY_LONG_POLL_WAIT, + help = "operation stats of LongPoll Reads wait time on a bookie", + happensAfter = READ_ENTRY_LONG_POLL_PRE_WAIT, + parent = READ_ENTRY_LONG_POLL_REQUEST + ) + final OpStatsLogger longPollWaitStats; + @StatsDoc( + name = READ_ENTRY_LONG_POLL_READ, + help = "operation stats of LongPoll Reads on a bookie", + happensAfter = READ_ENTRY_LONG_POLL_WAIT, + parent = READ_ENTRY_LONG_POLL_REQUEST + ) + final OpStatsLogger longPollReadStats; + @StatsDoc( + name = READ_ENTRY_LONG_POLL_REQUEST, + help = "request stats of LongPoll Reads on a bookie" + ) + final OpStatsLogger longPollReadRequestStats; + @StatsDoc( + name = READ_LAST_ENTRY_NOENTRY_ERROR, + help = "total NOENTRY errors of reading last entry on a bookie" + ) + final Counter readLastEntryNoEntryErrorCounter; + @StatsDoc( + name = WRITE_LAC_REQUEST, + help = "request stats of WriteLac on a bookie" + ) + final OpStatsLogger writeLacRequestStats; + @StatsDoc( + name = WRITE_LAC, + help = "operation stats of WriteLac on a bookie", + parent = WRITE_LAC_REQUEST + ) + final OpStatsLogger writeLacStats; + @StatsDoc( + name = READ_LAC_REQUEST, + help = "request stats of ReadLac on a bookie" + ) + final OpStatsLogger readLacRequestStats; + @StatsDoc( + name = READ_LAC, + help = "operation stats of ReadLac on a bookie", + parent = READ_LAC_REQUEST + ) + final OpStatsLogger readLacStats; + @StatsDoc( + name = GET_BOOKIE_INFO_REQUEST, + help = "request stats of GetBookieInfo on a bookie" + ) + final OpStatsLogger getBookieInfoRequestStats; + @StatsDoc( + name = GET_BOOKIE_INFO, + help = "operation stats of GetBookieInfo on a bookie" + ) + final OpStatsLogger getBookieInfoStats; + @StatsDoc( + name = CHANNEL_WRITE, + help = "channel write stats on a bookie" + ) + final OpStatsLogger channelWriteStats; + @StatsDoc( + name = ADD_ENTRY_BLOCKED, + help = "operation stats of AddEntry blocked on a bookie" + ) + final OpStatsLogger addEntryBlockedStats; + @StatsDoc( + name = READ_ENTRY_BLOCKED, + help = "operation stats of ReadEntry blocked on a bookie" + ) + final OpStatsLogger readEntryBlockedStats; + + public RequestStats(StatsLogger statsLogger) { + this.addEntryStats = statsLogger.getOpStatsLogger(ADD_ENTRY); + this.addRequestStats = statsLogger.getOpStatsLogger(ADD_ENTRY_REQUEST); + this.readEntryStats = statsLogger.getOpStatsLogger(READ_ENTRY); + this.forceLedgerStats = statsLogger.getOpStatsLogger(FORCE_LEDGER); + this.forceLedgerRequestStats = statsLogger.getOpStatsLogger(FORCE_LEDGER_REQUEST); + this.readRequestStats = statsLogger.getOpStatsLogger(READ_ENTRY_REQUEST); + this.fenceReadEntryStats = statsLogger.getOpStatsLogger(READ_ENTRY_FENCE_READ); + this.fenceReadRequestStats = statsLogger.getOpStatsLogger(READ_ENTRY_FENCE_REQUEST); + this.fenceReadWaitStats = statsLogger.getOpStatsLogger(READ_ENTRY_FENCE_WAIT); + this.readEntrySchedulingDelayStats = statsLogger.getOpStatsLogger(READ_ENTRY_SCHEDULING_DELAY); + this.longPollPreWaitStats = statsLogger.getOpStatsLogger(READ_ENTRY_LONG_POLL_PRE_WAIT); + this.longPollWaitStats = statsLogger.getOpStatsLogger(READ_ENTRY_LONG_POLL_WAIT); + this.longPollReadStats = statsLogger.getOpStatsLogger(READ_ENTRY_LONG_POLL_READ); + this.longPollReadRequestStats = statsLogger.getOpStatsLogger(READ_ENTRY_LONG_POLL_REQUEST); + this.readLastEntryNoEntryErrorCounter = statsLogger.getCounter(READ_LAST_ENTRY_NOENTRY_ERROR); + this.writeLacStats = statsLogger.getOpStatsLogger(WRITE_LAC); + this.writeLacRequestStats = statsLogger.getOpStatsLogger(WRITE_LAC_REQUEST); + this.readLacStats = statsLogger.getOpStatsLogger(READ_LAC); + this.readLacRequestStats = statsLogger.getOpStatsLogger(READ_LAC_REQUEST); + this.getBookieInfoStats = statsLogger.getOpStatsLogger(GET_BOOKIE_INFO); + this.getBookieInfoRequestStats = statsLogger.getOpStatsLogger(GET_BOOKIE_INFO_REQUEST); + this.channelWriteStats = statsLogger.getOpStatsLogger(CHANNEL_WRITE); + + this.addEntryBlockedStats = statsLogger.getOpStatsLogger(ADD_ENTRY_BLOCKED_WAIT); + this.readEntryBlockedStats = statsLogger.getOpStatsLogger(READ_ENTRY_BLOCKED_WAIT); + + statsLogger.registerGauge(ADD_ENTRY_IN_PROGRESS, new Gauge() { + @Override + public Number getDefaultValue() { + return 0; + } + + @Override + public Number getSample() { + return addsInProgress; + } + }); + + statsLogger.registerGauge(ADD_ENTRY_BLOCKED, new Gauge() { + @Override + public Number getDefaultValue() { + return 0; + } + + @Override + public Number getSample() { + return addsBlocked; + } + }); + + statsLogger.registerGauge(READ_ENTRY_IN_PROGRESS, new Gauge() { + @Override + public Number getDefaultValue() { + return 0; + } + + @Override + public Number getSample() { + return readsInProgress; + } + }); + + statsLogger.registerGauge(READ_ENTRY_BLOCKED, new Gauge() { + @Override + public Number getDefaultValue() { + return 0; + } + + @Override + public Number getSample() { + return readsBlocked; + } + }); + } + + // + // Add requests + // + + void blockAddRequest() { + addsBlocked.incrementAndGet(); + } + + void unblockAddRequest(long delayNanos) { + addEntryBlockedStats.registerSuccessfulEvent(delayNanos, TimeUnit.NANOSECONDS); + addsBlocked.decrementAndGet(); + } + + void trackAddRequest() { + final int curr = addsInProgress.incrementAndGet(); + maxAddsInProgress.accumulateAndGet(curr, Integer::max); + } + + void untrackAddRequest() { + addsInProgress.decrementAndGet(); + } + + int maxAddsInProgressCount() { + return maxAddsInProgress.get(); + } + + // + // Read requests + // + + void blockReadRequest() { + readsBlocked.incrementAndGet(); + } + + void unblockReadRequest(long delayNanos) { + readEntryBlockedStats.registerSuccessfulEvent(delayNanos, TimeUnit.NANOSECONDS); + readsBlocked.decrementAndGet(); + } + + void trackReadRequest() { + final int curr = readsInProgress.incrementAndGet(); + maxReadsInProgress.accumulateAndGet(curr, Integer::max); + } + + void untrackReadRequest() { + readsInProgress.decrementAndGet(); + } + + int maxReadsInProgressCount() { + return maxReadsInProgress.get(); + } + +} 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 f5af75ac4a6..70db7ce4919 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 @@ -63,7 +63,7 @@ protected void processPacket() { + " so rejecting the request from the client!"); sendResponse(BookieProtocol.EREADONLY, ResponseBuilder.buildErrorResponse(BookieProtocol.EREADONLY, request), - requestProcessor.getAddRequestStats()); + requestProcessor.getRequestStats().getAddRequestStats()); request.release(); request.recycle(); return; @@ -104,11 +104,11 @@ protected void processPacket() { } if (rc != BookieProtocol.EOK) { - requestProcessor.getAddEntryStats().registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getAddEntryStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); sendResponse(rc, ResponseBuilder.buildErrorResponse(rc, request), - requestProcessor.getAddRequestStats()); + requestProcessor.getRequestStats().getAddRequestStats()); request.recycle(); } } @@ -117,15 +117,15 @@ protected void processPacket() { public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { if (BookieProtocol.EOK == rc) { - requestProcessor.getAddEntryStats().registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getAddEntryStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } else { - requestProcessor.getAddEntryStats().registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getAddEntryStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } sendResponse(rc, ResponseBuilder.buildAddResponse(request), - requestProcessor.getAddRequestStats()); + requestProcessor.getRequestStats().getAddRequestStats()); request.recycle(); recycle(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java index 7747e5c0e1a..c8ea0672104 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java @@ -80,11 +80,11 @@ private AddResponse getAddResponse() { public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { if (BookieProtocol.EOK == rc) { - requestProcessor.getAddEntryStats().registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getAddEntryStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } else { - requestProcessor.getAddEntryStats().registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getAddEntryStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } StatusCode status; @@ -105,7 +105,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, .setStatus(addResponse.getStatus()) .setAddResponse(addResponse); Response resp = response.build(); - sendResponse(status, resp, requestProcessor.getAddRequestStats()); + sendResponse(status, resp, requestProcessor.getRequestStats().getAddRequestStats()); } }; final EnumSet writeFlags; @@ -171,7 +171,7 @@ public void safeRun() { .setAddResponse(addResponse); Response resp = response.build(); sendResponse(addResponse.getStatus(), resp, - requestProcessor.getAddRequestStats()); + requestProcessor.getRequestStats().getAddRequestStats()); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java index 7e42a7320af..691102bae12 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java @@ -71,11 +71,11 @@ private WriteLacResponse getWriteLacResponse() { @Override public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { if (BookieProtocol.EOK == rc) { - requestProcessor.writeLacStats.registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getWriteLacStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } else { - requestProcessor.writeLacStats.registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getWriteLacStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } StatusCode status; @@ -96,7 +96,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre .setStatus(writeLacResponse.getStatus()) .setWriteLacResponse(writeLacResponse); Response resp = response.build(); - sendResponse(status, resp, requestProcessor.writeLacRequestStats); + sendResponse(status, resp, requestProcessor.getRequestStats().getWriteLacRequestStats()); } }; @@ -130,8 +130,8 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre // If everything is okay, we return null so that the calling function // dosn't return a response back to the caller. if (!status.equals(StatusCode.EOK)) { - requestProcessor.writeLacStats.registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), - TimeUnit.NANOSECONDS); + requestProcessor.getRequestStats().getWriteLacStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); writeLacResponse.setStatus(status); return writeLacResponse.build(); } @@ -147,7 +147,10 @@ public void safeRun() { .setStatus(writeLacResponse.getStatus()) .setWriteLacResponse(writeLacResponse); Response resp = response.build(); - sendResponse(writeLacResponse.getStatus(), resp, requestProcessor.writeLacRequestStats); + sendResponse( + writeLacResponse.getStatus(), + resp, + requestProcessor.getRequestStats().getWriteLacRequestStats()); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java index 37d4647343d..bab83fb3267 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java @@ -75,10 +75,7 @@ public void setup() { requestProcessor = mock(BookieRequestProcessor.class); when(requestProcessor.getBookie()).thenReturn(bookie); when(requestProcessor.getWaitTimeoutOnBackpressureMillis()).thenReturn(-1L); - when(requestProcessor.getForceLedgerStats()) - .thenReturn(NullStatsLogger.INSTANCE.getOpStatsLogger("force_ledger")); - when(requestProcessor.getForceLedgerRequestStats()) - .thenReturn(NullStatsLogger.INSTANCE.getOpStatsLogger("force_ledger_request")); + when(requestProcessor.getRequestStats()).thenReturn(new RequestStats(NullStatsLogger.INSTANCE)); processor = new ForceLedgerProcessorV3( request, channel, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorTest.java index 5901c2f5823..bbcffea08cd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorTest.java @@ -67,10 +67,7 @@ public void setup() { bookie = mock(Bookie.class); requestProcessor = mock(BookieRequestProcessor.class); when(requestProcessor.getBookie()).thenReturn(bookie); - when(requestProcessor.getAddEntryStats()) - .thenReturn(NullStatsLogger.INSTANCE.getOpStatsLogger("add_entry")); - when(requestProcessor.getAddRequestStats()) - .thenReturn(NullStatsLogger.INSTANCE.getOpStatsLogger("add_requests")); + when(requestProcessor.getRequestStats()).thenReturn(new RequestStats(NullStatsLogger.INSTANCE)); processor = WriteEntryProcessor.create( request, channel, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java index df7b1532b63..292dc519ca9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java @@ -80,12 +80,7 @@ public void setup() { requestProcessor = mock(BookieRequestProcessor.class); when(requestProcessor.getBookie()).thenReturn(bookie); when(requestProcessor.getWaitTimeoutOnBackpressureMillis()).thenReturn(-1L); - when(requestProcessor.getAddEntryStats()) - .thenReturn(NullStatsLogger.INSTANCE.getOpStatsLogger("add_entry")); - when(requestProcessor.getAddRequestStats()) - .thenReturn(NullStatsLogger.INSTANCE.getOpStatsLogger("add_requests")); - when(requestProcessor.getChannelWriteStats()) - .thenReturn(NullStatsLogger.INSTANCE.getOpStatsLogger("CHANNEL_WRITE")); + when(requestProcessor.getRequestStats()).thenReturn(new RequestStats(NullStatsLogger.INSTANCE)); processor = new WriteEntryProcessorV3( request, channel, diff --git a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/StatsDoc.java b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/StatsDoc.java index 97f487a69a1..d2ca8c60bed 100644 --- a/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/StatsDoc.java +++ b/bookkeeper-stats/src/main/java/org/apache/bookkeeper/stats/annotations/StatsDoc.java @@ -58,5 +58,25 @@ */ String help(); + /** + * The parent metric name. + * + *

            It can used for analyzing the relationships + * between the metrics, especially for the latency metrics. + * + * @return the parent metric name + */ + String parent() default ""; + + /** + * The metric name of an operation that happens + * after the operation of this metric. + * + *

            similar as {@link #parent()}, it can be used for analyzing + * the dependencies between metrics. + * + * @return the metric name of an operation that happens after the operation of this metric. + */ + String happensAfter() default ""; } From 973ea774586763f0aadce5bca8297fd440cae722 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 29 Nov 2018 10:07:00 +0100 Subject: [PATCH 0148/1642] Rename LedgerMetadataBuilder#closingAt to parameter order (#1835) * Rename LedgerMetadataBuilder#closingAt to clarify parameter order The two parameters are long, which can be confusing. Master issue: #281 --- .../org/apache/bookkeeper/client/LedgerHandle.java | 3 ++- .../org/apache/bookkeeper/client/LedgerMetadata.java | 4 ++-- .../bookkeeper/client/LedgerMetadataBuilder.java | 12 ++++++++++-- .../bookkeeper/client/ReadOnlyLedgerHandle.java | 2 +- .../apache/bookkeeper/client/HandleFailuresTest.java | 3 ++- .../apache/bookkeeper/client/LedgerClose2Test.java | 6 ++++-- .../bookkeeper/client/LedgerRecovery2Test.java | 3 ++- .../bookkeeper/client/MetadataUpdateLoopTest.java | 5 ++++- .../client/TestLedgerFragmentReplication.java | 2 +- 9 files changed, 28 insertions(+), 12 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 dedae6e0cff..9af32713080 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 @@ -557,7 +557,8 @@ public void safeRun() { }, (metadata) -> { return LedgerMetadataBuilder.from(metadata) - .closingAt(lastEntry, finalLength).build(); + .withClosedState().withLastEntryId(lastEntry) + .withLength(finalLength).build(); }, LedgerHandle.this::setLedgerMetadata) .run().whenComplete((metadata, ex) -> { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 514f1af27f7..894a7b1d456 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -441,7 +441,7 @@ public static LedgerMetadata parseConfig(byte[] bytes, if (data.getState() == LedgerMetadataFormat.State.IN_RECOVERY) { builder.withInRecoveryState(); } else if (data.getState() == LedgerMetadataFormat.State.CLOSED) { - builder.closingAt(data.getLastEntryId(), data.getLength()); + builder.withClosedState().withLastEntryId(data.getLastEntryId()).withLength(data.getLength()); } if (data.hasPassword()) { @@ -483,7 +483,7 @@ static LedgerMetadata parseVersion1Config(BufferedReader reader) throws IOExcept if (l == IN_RECOVERY) { builder.withInRecoveryState(); } else { - builder.closingAt(l, length); + builder.withClosedState().withLastEntryId(l).withLength(length); } break; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index 4a42c70d81f..9034e18703a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -147,10 +147,18 @@ public LedgerMetadataBuilder withInRecoveryState() { return this; } - public LedgerMetadataBuilder closingAt(long lastEntryId, long length) { + public LedgerMetadataBuilder withClosedState() { + this.state = LedgerMetadataFormat.State.CLOSED; + return this; + } + + public LedgerMetadataBuilder withLastEntryId(long lastEntryId) { this.lastEntryId = Optional.of(lastEntryId); + return this; + } + + public LedgerMetadataBuilder withLength(long length) { this.length = Optional.of(length); - this.state = LedgerMetadataFormat.State.CLOSED; return this; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java index 9c2901ccee1..59ddd5651b6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java @@ -326,7 +326,7 @@ CompletableFuture> closeRecovered() { } }); } - return builder.closingAt(lac, len).build(); + return builder.withClosedState().withLastEntryId(lac).withLength(len).build(); }, this::setLedgerMetadata).run(); f.thenRun(() -> { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java index e4b48d91624..2f75e12a839 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java @@ -294,7 +294,8 @@ public void testMetadataSetToClosedDuringFailureHandler() throws Exception { changeInProgress.get(); ClientUtil.transformMetadata(clientCtx, 10L, - (metadata) -> LedgerMetadataBuilder.from(metadata).closingAt(1234L, 10L).build()); + (metadata) -> LedgerMetadataBuilder.from(metadata) + .withClosedState().withLastEntryId(1234L).withLength(10L).build()); blockEnsembleChange.complete(null); // allow ensemble change to continue try { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java index 93846bb68f5..541af0a54dc 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java @@ -141,7 +141,8 @@ public void testMetadataCloseWithCorrectLengthDuringClose() throws Exception { closeInProgress.get(); ClientUtil.transformMetadata(clientCtx, 10L, - (metadata) -> LedgerMetadataBuilder.from(metadata).closingAt(lac, length).build()); + (metadata) -> LedgerMetadataBuilder.from(metadata) + .withClosedState().withLastEntryId(lac).withLength(length).build()); blockClose.complete(null); closeFuture.get(); @@ -183,7 +184,8 @@ public void testMetadataCloseWithDifferentLengthDuringClose() throws Exception { /* close with different length. can happen in cases where there's a write outstanding */ ClientUtil.transformMetadata(clientCtx, 10L, - (metadata) -> LedgerMetadataBuilder.from(metadata).closingAt(lac + 1, length + 100).build()); + (metadata) -> LedgerMetadataBuilder.from(metadata) + .withClosedState().withLastEntryId(lac + 1).withLength(length + 100).build()); blockClose.complete(null); try { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java index 2a1050e49b0..82c1a4b0bb4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java @@ -123,7 +123,8 @@ public void testMetadataClosedDuringRecovery() throws Exception { writingBack.get(10, TimeUnit.SECONDS); ClientUtil.transformMetadata(clientCtx, 1L, - (metadata) -> LedgerMetadataBuilder.from(metadata).closingAt(-1, 0).build()); + (metadata) -> LedgerMetadataBuilder.from(metadata) + .withClosedState().withLastEntryId(-1).withLength(0).build()); // allow recovery to continue blocker.complete(null); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java index 843ec9b701c..2544a20ef0e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java @@ -351,7 +351,10 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { ledgerId, reference::get, (currentMetadata) -> !currentMetadata.isClosed(), - (currentMetadata) -> LedgerMetadataBuilder.from(currentMetadata).closingAt(10L, 100L).build(), + (currentMetadata) -> { + return LedgerMetadataBuilder.from(currentMetadata) + .withClosedState().withLastEntryId(10L).withLength(100L).build(); + }, reference::compareAndSet).run(); CompletableFuture> loop2 = new MetadataUpdateLoop( lm, 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 3503e91aa39..4b8d06c3127 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 @@ -242,7 +242,7 @@ public void testSplitIntoSubFragmentsWithDifferentFragmentBoundaries() LedgerMetadata metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .withPassword(TEST_PSSWD).withDigestType(TEST_DIGEST_TYPE.toApiDigestType()) - .closingAt(-1, 0) + .withClosedState().withLastEntryId(-1).withLength(0) .newEnsembleEntry(0L, ensemble) .build(); From ca4327cd2e2c3eb980cd424d6791b2105d723d94 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 29 Nov 2018 22:51:05 -0800 Subject: [PATCH 0149/1642] [CI] increase timeout of website build to 120 mins Descriptions of the changes in this PR: *Motivation* Since we have more releases, the build time for website is increasing to beyond 30 min. Increase it to 120 mins for now Reviewers: Enrico Olivelli , Jia Zhai This closes #1844 from sijie/change_website_buildtime --- .test-infra/jenkins/common_job_properties.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.test-infra/jenkins/common_job_properties.groovy b/.test-infra/jenkins/common_job_properties.groovy index 999f9db509a..acb615f32f3 100644 --- a/.test-infra/jenkins/common_job_properties.groovy +++ b/.test-infra/jenkins/common_job_properties.groovy @@ -35,7 +35,7 @@ class common_job_properties { 'https://gitbox.apache.org/repos/asf/bookkeeper.git', branch, 'git-websites', - 30) + 120) } // Sets common top-level job properties for main repository jobs. From 87c0abc5919e97329ab9d50dae3d9981ab21e756 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 30 Nov 2018 00:27:23 -0800 Subject: [PATCH 0150/1642] Set default sizes of DbLedgerStorage read and write cache to be proportional to JVM direct memory ### Motivation To simplify Bookie configuration when using `DbLedgerStorage`, set the memory size defaults for WriteCache, ReadCache and RocksDB block cache to be pegged to the available direct memory configured in the JVM. User can always configure specific values and override this behavior. Reviewers: Enrico Olivelli , Sijie Guo This closes #1813 from merlimat/auto-conf-db-storage-mem --- .../bookie/storage/ldb/DbLedgerStorage.java | 17 ++++++++++++----- .../storage/ldb/KeyValueStorageRocksDB.java | 9 ++++++++- .../bookie/storage/ldb/DbLedgerStorageTest.java | 2 ++ .../bookkeeper/conf/TestBKConfiguration.java | 3 +++ 4 files changed, 25 insertions(+), 6 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index 831de531d91..aad42ad361e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -28,6 +28,9 @@ import io.netty.buffer.ByteBuf; import io.netty.util.concurrent.DefaultThreadFactory; +//CHECKSTYLE.OFF: IllegalImport +import io.netty.util.internal.PlatformDependent; +//CHECKSTYLE.ON: IllegalImport import java.io.File; import java.io.IOException; @@ -66,17 +69,17 @@ @Slf4j public class DbLedgerStorage implements LedgerStorage { - static final String WRITE_CACHE_MAX_SIZE_MB = "dbStorage_writeCacheMaxSizeMb"; + public static final String WRITE_CACHE_MAX_SIZE_MB = "dbStorage_writeCacheMaxSizeMb"; - static final String READ_AHEAD_CACHE_MAX_SIZE_MB = "dbStorage_readAheadCacheMaxSizeMb"; + public static final String READ_AHEAD_CACHE_MAX_SIZE_MB = "dbStorage_readAheadCacheMaxSizeMb"; static final String MAX_THROTTLE_TIME_MILLIS = "dbStorage_maxThrottleTimeMs"; - private static final long DEFAULT_WRITE_CACHE_MAX_SIZE_MB = 16; - private static final long DEFAULT_READ_CACHE_MAX_SIZE_MB = 16; - private static final int MB = 1024 * 1024; + private static final long DEFAULT_WRITE_CACHE_MAX_SIZE_MB = (long) (0.25 * PlatformDependent.maxDirectMemory()) + / MB; + private static final long DEFAULT_READ_CACHE_MAX_SIZE_MB = (long) (0.25 * PlatformDependent.maxDirectMemory()) / MB; private int numberOfDirs; private List ledgerStorageList; @@ -97,6 +100,10 @@ public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, Le log.info(" - Write cache size: {} MB", writeCacheMaxSize / MB); log.info(" - Read Cache: {} MB", readCacheMaxSize / MB); + if (readCacheMaxSize + writeCacheMaxSize > PlatformDependent.maxDirectMemory()) { + throw new IOException("Read and write cache sizes exceed the configured max direct memory size"); + } + long perDirectoryWriteCacheSize = writeCacheMaxSize / numberOfDirs; long perDirectoryReadCacheSize = readCacheMaxSize / numberOfDirs; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java index a343b59a814..eff40e8a8df 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java @@ -24,6 +24,10 @@ import com.google.common.primitives.UnsignedBytes; +//CHECKSTYLE.OFF: IllegalImport +import io.netty.util.internal.PlatformDependent; +//CHECKSTYLE.ON: IllegalImport + import java.io.IOException; import java.util.Comparator; import java.util.Map.Entry; @@ -97,13 +101,16 @@ public KeyValueStorageRocksDB(String path, DbConfigType dbConfigType, ServerConf options.setCreateIfMissing(true); if (dbConfigType == DbConfigType.Huge) { + // Set default RocksDB block-cache size to 10% of direct mem, unless override + long defaultRocksDBBlockCacheSizeBytes = PlatformDependent.maxDirectMemory() / 10; + long writeBufferSizeMB = conf.getInt(ROCKSDB_WRITE_BUFFER_SIZE_MB, 64); long sstSizeMB = conf.getInt(ROCKSDB_SST_SIZE_MB, 64); int numLevels = conf.getInt(ROCKSDB_NUM_LEVELS, -1); int numFilesInLevel0 = conf.getInt(ROCKSDB_NUM_FILES_IN_LEVEL0, 4); long maxSizeInLevel1MB = conf.getLong(ROCKSDB_MAX_SIZE_IN_LEVEL1_MB, 256); int blockSize = conf.getInt(ROCKSDB_BLOCK_SIZE, 64 * 1024); - long blockCacheSize = conf.getLong(ROCKSDB_BLOCK_CACHE_SIZE, 256 * 1024 * 1024); + long blockCacheSize = conf.getLong(ROCKSDB_BLOCK_CACHE_SIZE, defaultRocksDBBlockCacheSizeBytes); int bloomFilterBitsPerKey = conf.getInt(ROCKSDB_BLOOM_FILTERS_BITS_PER_KEY, 10); boolean lz4CompressionEnabled = conf.getBoolean(ROCKSDB_LZ4_COMPRESSION_ENABLED, true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java index fb0a97a08cf..3a3e982ef95 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java @@ -245,6 +245,8 @@ public void doubleDirectory() throws Exception { File secondDir = new File(tmpDir, "dir2"); ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); conf.setGcWaitTime(gcWaitTime); + conf.setProperty(DbLedgerStorage.WRITE_CACHE_MAX_SIZE_MB, 4); + conf.setProperty(DbLedgerStorage.READ_AHEAD_CACHE_MAX_SIZE_MB, 4); conf.setLedgerStorageClass(DbLedgerStorage.class.getName()); conf.setLedgerDirNames(new String[] { firstDir.getCanonicalPath(), secondDir.getCanonicalPath() }); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java index aba12a748ce..9073b01ccc9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.Enumeration; +import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,6 +54,8 @@ public static ServerConfiguration newServerConfiguration() { confReturn.setGcWaitTime(1000); confReturn.setDiskUsageThreshold(0.999f); confReturn.setDiskUsageWarnThreshold(0.99f); + confReturn.setProperty(DbLedgerStorage.WRITE_CACHE_MAX_SIZE_MB, 4); + confReturn.setProperty(DbLedgerStorage.READ_AHEAD_CACHE_MAX_SIZE_MB, 4); setLoopbackInterfaceAndAllowLoopback(confReturn); return confReturn; } From a805c2bbb3e509731dc83fb2aba575fb5edbbbf3 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 30 Nov 2018 11:25:40 +0100 Subject: [PATCH 0151/1642] Move serialization code out of LedgerMetadata This PR moves the serialization code out of LedgerMetadata so that it can later be adapted to run different serialization code depending on the environment. Notable non-refactor changes: - LedgerMetadata#toString no longer uses #serialize because it's no longer available. Instead it uses the ToString helper from guava. byte[] fields are now base64 encoded. - There's a new state enum and getter in api.LedgerMetadata. This is so that LedgerMetadataFormat can be removed from client.LedgerMetadata. Master issue: #723 Reviewers: Sijie Guo This closes #1848 from ivankelly/refactor-md-serde --- .../apache/bookkeeper/bookie/BookieShell.java | 8 +- .../bookkeeper/client/LedgerMetadata.java | 336 +++-------------- .../client/LedgerMetadataBuilder.java | 13 +- .../bookkeeper/client/api/LedgerMetadata.java | 25 ++ .../meta/AbstractZkLedgerManager.java | 8 +- .../bookkeeper/meta/LedgerMetadataSerDe.java | 338 ++++++++++++++++++ .../meta/MSLedgerManagerFactory.java | 9 +- .../http/service/GetLedgerMetaService.java | 6 +- .../http/service/ListLedgerService.java | 10 +- .../bookkeeper/client/LedgerMetadataTest.java | 19 +- .../meta/AbstractZkLedgerManagerTest.java | 29 +- .../bookkeeper/meta/MockLedgerManager.java | 8 +- .../server/http/TestHttpService.java | 3 +- .../metadata/etcd/EtcdLedgerManager.java | 15 +- 14 files changed, 482 insertions(+), 345 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java 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 12d85e2eb98..c58fa834415 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 @@ -98,6 +98,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; @@ -1105,7 +1106,7 @@ Options getOptions() { void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { System.out.println("ledgerID: " + ledgerIdFormatter.formatLedgerId(ledgerId)); if (printMeta) { - System.out.println(new String(md.serialize(), UTF_8)); + System.out.println(new String(new LedgerMetadataSerDe().serialize(md), UTF_8)); } } @@ -1114,6 +1115,7 @@ void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { */ class LedgerMetadataCmd extends MyCommand { Options lOpts = new Options(); + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); LedgerMetadataCmd() { super(CMD_LEDGERMETADATA); @@ -1139,11 +1141,11 @@ public int runCmd(CommandLine cmdLine) throws Exception { if (cmdLine.hasOption("dumptofile")) { Versioned md = m.readLedgerMetadata(lid).join(); Files.write(FileSystems.getDefault().getPath(cmdLine.getOptionValue("dumptofile")), - md.getValue().serialize()); + serDe.serialize(md.getValue())); } else if (cmdLine.hasOption("restorefromfile")) { byte[] serialized = Files.readAllBytes( FileSystems.getDefault().getPath(cmdLine.getOptionValue("restorefromfile"))); - LedgerMetadata md = LedgerMetadata.parseConfig(serialized, Optional.empty()); + LedgerMetadata md = serDe.parseConfig(serialized, Optional.empty()); m.createLedgerMetadata(lid, md).join(); } else { printLedgerMetadata(lid, m.readLedgerMetadata(lid).get().getValue(), true); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 894a7b1d456..631761285a2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -17,26 +17,19 @@ */ package org.apache.bookkeeper.client; -import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.protobuf.ByteString; -import com.google.protobuf.TextFormat; -import java.io.BufferedReader; -import java.io.IOException; -import java.io.StringReader; -import java.nio.CharBuffer; -import java.util.ArrayList; import java.util.Arrays; +import java.util.Base64; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.NavigableMap; import java.util.Optional; import java.util.Set; @@ -45,8 +38,8 @@ import java.util.stream.Collectors; import lombok.EqualsAndHashCode; import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata.State; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,25 +53,12 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMetadata { static final Logger LOG = LoggerFactory.getLogger(LedgerMetadata.class); - private static final String closed = "CLOSED"; - private static final String lSplitter = "\n"; - private static final String tSplitter = "\t"; - - // can't use -1 for NOTCLOSED because that is reserved for a closed, empty - // ledger - private static final int NOTCLOSED = -101; - private static final int IN_RECOVERY = -102; - - public static final int LOWEST_COMPAT_METADATA_FORMAT_VERSION = 0; - public static final int CURRENT_METADATA_FORMAT_VERSION = 2; - public static final String VERSION_KEY = "BookieMetadataFormatVersion"; - private final int metadataFormatVersion; private final int ensembleSize; private final int writeQuorumSize; private final int ackQuorumSize; - private final LedgerMetadataFormat.State state; + private final State state; private final long length; private final long lastEntryId; private final long ctime; @@ -88,7 +68,7 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe private final ImmutableList currentEnsemble; private final boolean hasPassword; - private final LedgerMetadataFormat.DigestType digestType; + private final DigestType digestType; private final byte[] password; private final Map customMetadata; @@ -97,7 +77,7 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe int ensembleSize, int writeQuorumSize, int ackQuorumSize, - LedgerMetadataFormat.State state, + State state, Optional lastEntryId, Optional length, Map> ensembles, @@ -107,7 +87,7 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe boolean storeCtime, Map customMetadata) { checkArgument(ensembles.size() > 0, "There must be at least one ensemble in the ledger"); - if (state == LedgerMetadataFormat.State.CLOSED) { + if (state == State.CLOSED) { checkArgument(length.isPresent(), "Closed ledger must have a length"); checkArgument(lastEntryId.isPresent(), "Closed ledger must have a last entry"); } else { @@ -130,14 +110,13 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe ImmutableList.copyOf(e.getValue())), TreeMap::putAll)); - if (state != LedgerMetadataFormat.State.CLOSED) { + if (state != State.CLOSED) { currentEnsemble = this.ensembles.lastEntry().getValue(); } else { currentEnsemble = null; } - this.digestType = digestType.equals(DigestType.MAC) - ? LedgerMetadataFormat.DigestType.HMAC : LedgerMetadataFormat.DigestType.valueOf(digestType.toString()); + this.digestType = digestType; if (password.isPresent()) { this.password = password.get(); @@ -183,7 +162,7 @@ public long getCtime() { * * @return whether the password has been stored in the metadata */ - boolean hasPassword() { + public boolean hasPassword() { return hasPassword; } @@ -198,18 +177,7 @@ public byte[] getPassword() { @Override public DigestType getDigestType() { - switch (digestType) { - case HMAC: - return DigestType.MAC; - case CRC32: - return DigestType.CRC32; - case CRC32C: - return DigestType.CRC32C; - case DUMMY: - return DigestType.DUMMY; - default: - throw new IllegalArgumentException("Unable to convert digest type " + digestType); - } + return digestType; } @Override @@ -224,14 +192,15 @@ public long getLength() { @Override public boolean isClosed() { - return state == LedgerMetadataFormat.State.CLOSED; + return state == State.CLOSED; } public boolean isInRecovery() { - return state == LedgerMetadataFormat.State.IN_RECOVERY; + return state == State.IN_RECOVERY; } - public LedgerMetadataFormat.State getState() { + @Override + public State getState() { return state; } @@ -272,236 +241,6 @@ public Map getCustomMetadata() { return this.customMetadata; } - LedgerMetadataFormat buildProtoFormat() { - return buildProtoFormat(true); - } - - LedgerMetadataFormat buildProtoFormat(boolean withPassword) { - LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder(); - builder.setQuorumSize(writeQuorumSize).setAckQuorumSize(ackQuorumSize) - .setEnsembleSize(ensembleSize).setLength(length) - .setState(state).setLastEntryId(lastEntryId); - - if (storeCtime) { - builder.setCtime(ctime); - } - - if (hasPassword) { - builder.setDigestType(digestType); - if (withPassword) { - builder.setPassword(ByteString.copyFrom(password)); - } - } - - if (customMetadata != null) { - LedgerMetadataFormat.cMetadataMapEntry.Builder cMetadataBuilder = - LedgerMetadataFormat.cMetadataMapEntry.newBuilder(); - for (Map.Entry entry : customMetadata.entrySet()) { - cMetadataBuilder.setKey(entry.getKey()).setValue(ByteString.copyFrom(entry.getValue())); - builder.addCustomMetadata(cMetadataBuilder.build()); - } - } - - for (Map.Entry> entry : ensembles.entrySet()) { - LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder(); - segmentBuilder.setFirstEntryId(entry.getKey()); - for (BookieSocketAddress addr : entry.getValue()) { - segmentBuilder.addEnsembleMember(addr.toString()); - } - builder.addSegment(segmentBuilder.build()); - } - return builder.build(); - } - - /** - * Generates a byte array of this object. - * - * @return the metadata serialized into a byte array - */ - public byte[] serialize() { - return serialize(true); - } - - public byte[] serialize(boolean withPassword) { - if (metadataFormatVersion == 1) { - return serializeVersion1(); - } - - StringBuilder s = new StringBuilder(); - s.append(VERSION_KEY).append(tSplitter).append(CURRENT_METADATA_FORMAT_VERSION).append(lSplitter); - s.append(TextFormat.printToString(buildProtoFormat(withPassword))); - if (LOG.isDebugEnabled()) { - LOG.debug("Serialized config: {}", s); - } - return s.toString().getBytes(UTF_8); - } - - private byte[] serializeVersion1() { - StringBuilder s = new StringBuilder(); - s.append(VERSION_KEY).append(tSplitter).append(metadataFormatVersion).append(lSplitter); - s.append(writeQuorumSize).append(lSplitter).append(ensembleSize).append(lSplitter).append(length); - - for (Map.Entry> entry : ensembles.entrySet()) { - s.append(lSplitter).append(entry.getKey()); - for (BookieSocketAddress addr : entry.getValue()) { - s.append(tSplitter); - s.append(addr.toString()); - } - } - - if (state == LedgerMetadataFormat.State.IN_RECOVERY) { - s.append(lSplitter).append(IN_RECOVERY).append(tSplitter).append(closed); - } else if (state == LedgerMetadataFormat.State.CLOSED) { - s.append(lSplitter).append(getLastEntryId()).append(tSplitter).append(closed); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Serialized config: {}", s); - } - - return s.toString().getBytes(UTF_8); - } - - /** - * Parses a given byte array and transforms into a LedgerConfig object. - * - * @param bytes - * byte array to parse - * @param metadataStoreCtime - * metadata store creation time, used for legacy ledgers - * @return LedgerConfig - * @throws IOException - * if the given byte[] cannot be parsed - */ - public static LedgerMetadata parseConfig(byte[] bytes, - Optional metadataStoreCtime) throws IOException { - String config = new String(bytes, UTF_8); - - if (LOG.isDebugEnabled()) { - LOG.debug("Parsing Config: {}", config); - } - BufferedReader reader = new BufferedReader(new StringReader(config)); - String versionLine = reader.readLine(); - if (versionLine == null) { - throw new IOException("Invalid metadata. Content missing"); - } - final int metadataFormatVersion; - if (versionLine.startsWith(VERSION_KEY)) { - String parts[] = versionLine.split(tSplitter); - metadataFormatVersion = Integer.parseInt(parts[1]); - } else { - // if no version is set, take it to be version 1 - // as the parsing is the same as what we had before - // we introduce versions - metadataFormatVersion = 1; - // reset the reader - reader.close(); - reader = new BufferedReader(new StringReader(config)); - } - - if (metadataFormatVersion < LOWEST_COMPAT_METADATA_FORMAT_VERSION - || metadataFormatVersion > CURRENT_METADATA_FORMAT_VERSION) { - throw new IOException( - String.format("Metadata version not compatible. Expected between %d and %d, but got %d", - LOWEST_COMPAT_METADATA_FORMAT_VERSION, CURRENT_METADATA_FORMAT_VERSION, - metadataFormatVersion)); - } - - if (metadataFormatVersion == 1) { - return parseVersion1Config(reader); - } - - LedgerMetadataBuilder builder = LedgerMetadataBuilder.create() - .withMetadataFormatVersion(metadataFormatVersion); - - // remaining size is total minus the length of the version line and '\n' - char[] configBuffer = new char[config.length() - (versionLine.length() + 1)]; - if (configBuffer.length != reader.read(configBuffer, 0, configBuffer.length)) { - throw new IOException("Invalid metadata buffer"); - } - - LedgerMetadataFormat.Builder formatBuilder = LedgerMetadataFormat.newBuilder(); - TextFormat.merge((CharSequence) CharBuffer.wrap(configBuffer), formatBuilder); - LedgerMetadataFormat data = formatBuilder.build(); - - builder.withEnsembleSize(data.getEnsembleSize()); - builder.withWriteQuorumSize(data.getQuorumSize()); - if (data.hasAckQuorumSize()) { - builder.withAckQuorumSize(data.getAckQuorumSize()); - } else { - builder.withAckQuorumSize(data.getQuorumSize()); - } - - if (data.hasCtime()) { - builder.withCreationTime(data.getCtime()).storingCreationTime(true); - } else if (metadataStoreCtime.isPresent()) { - builder.withCreationTime(metadataStoreCtime.get()).storingCreationTime(false); - } - - if (data.getState() == LedgerMetadataFormat.State.IN_RECOVERY) { - builder.withInRecoveryState(); - } else if (data.getState() == LedgerMetadataFormat.State.CLOSED) { - builder.withClosedState().withLastEntryId(data.getLastEntryId()).withLength(data.getLength()); - } - - if (data.hasPassword()) { - builder.withPassword(data.getPassword().toByteArray()) - .withDigestType(protoToApiDigestType(data.getDigestType())); - } - - for (LedgerMetadataFormat.Segment s : data.getSegmentList()) { - List addrs = new ArrayList<>(); - for (String addr : s.getEnsembleMemberList()) { - addrs.add(new BookieSocketAddress(addr)); - } - builder.newEnsembleEntry(s.getFirstEntryId(), addrs); - } - - if (data.getCustomMetadataCount() > 0) { - builder.withCustomMetadata(data.getCustomMetadataList().stream().collect( - Collectors.toMap(e -> e.getKey(), - e -> e.getValue().toByteArray()))); - } - return builder.build(); - } - - static LedgerMetadata parseVersion1Config(BufferedReader reader) throws IOException { - LedgerMetadataBuilder builder = LedgerMetadataBuilder.create().withMetadataFormatVersion(1); - try { - int quorumSize = Integer.parseInt(reader.readLine()); - int ensembleSize = Integer.parseInt(reader.readLine()); - long length = Long.parseLong(reader.readLine()); - - builder.withEnsembleSize(ensembleSize).withWriteQuorumSize(quorumSize).withAckQuorumSize(quorumSize); - - String line = reader.readLine(); - while (line != null) { - String parts[] = line.split(tSplitter); - - if (parts[1].equals(closed)) { - Long l = Long.parseLong(parts[0]); - if (l == IN_RECOVERY) { - builder.withInRecoveryState(); - } else { - builder.withClosedState().withLastEntryId(l).withLength(length); - } - break; - } - - ArrayList addrs = new ArrayList(); - for (int j = 1; j < parts.length; j++) { - addrs.add(new BookieSocketAddress(parts[j])); - } - builder.newEnsembleEntry(Long.parseLong(parts[0]), addrs); - - line = reader.readLine(); - } - return builder.build(); - } catch (NumberFormatException e) { - throw new IOException(e); - } - } - @Override public String toString() { return toStringRepresentation(true); @@ -519,9 +258,30 @@ public String toSafeString() { } private String toStringRepresentation(boolean withPassword) { - StringBuilder sb = new StringBuilder(); - sb.append("(meta:").append(new String(serialize(withPassword), UTF_8)).append(")"); - return sb.toString(); + MoreObjects.ToStringHelper helper = MoreObjects.toStringHelper("LedgerMetadata"); + helper.add("formatVersion", metadataFormatVersion) + .add("ensembleSize", ensembleSize) + .add("writeQuorumSize", writeQuorumSize) + .add("ackQuorumSize", ackQuorumSize) + .add("state", state); + if (state == State.CLOSED) { + helper.add("length", length) + .add("lastEntryId", lastEntryId); + } + if (hasPassword()) { + helper.add("digestType", digestType); + if (withPassword) { + helper.add("password", "base64:" + Base64.getEncoder().encodeToString(password)); + } else { + helper.add("password", "OMITTED"); + } + } + helper.add("ensembles", ensembles.toString()); + helper.add("customMetadata", + customMetadata.entrySet().stream().collect( + Collectors.toMap(e -> e.getKey(), + e -> "base64:" + Base64.getEncoder().encodeToString(e.getValue())))); + return helper.toString(); } Set getBookiesInThisLedger() { @@ -542,22 +302,12 @@ Long getLastEnsembleKey() { return ensembles.lastKey(); } - int getMetadataFormatVersion() { + public int getMetadataFormatVersion() { return metadataFormatVersion; } - private static DigestType protoToApiDigestType(LedgerMetadataFormat.DigestType digestType) { - switch (digestType) { - case HMAC: - return DigestType.MAC; - case CRC32: - return DigestType.CRC32; - case CRC32C: - return DigestType.CRC32C; - case DUMMY: - return DigestType.DUMMY; - default: - throw new IllegalArgumentException("Unable to convert digest type " + digestType); - } + // temporarily method, until storeCtime is removed from the metadata object itself + public boolean shouldStoreCtime() { + return storeCtime; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index 9034e18703a..88f50893008 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import static org.apache.bookkeeper.meta.LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; @@ -31,10 +32,10 @@ import java.util.TreeMap; import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata.State; import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; /** * Builder for building LedgerMetadata objects. @@ -43,12 +44,12 @@ @Unstable @VisibleForTesting public class LedgerMetadataBuilder { - private int metadataFormatVersion = LedgerMetadata.CURRENT_METADATA_FORMAT_VERSION; + private int metadataFormatVersion = CURRENT_METADATA_FORMAT_VERSION; private int ensembleSize = 3; private int writeQuorumSize = 3; private int ackQuorumSize = 2; - private LedgerMetadataFormat.State state = LedgerMetadataFormat.State.OPEN; + private State state = State.OPEN; private Optional lastEntryId = Optional.empty(); private Optional length = Optional.empty(); @@ -73,7 +74,7 @@ public static LedgerMetadataBuilder from(LedgerMetadata other) { builder.ackQuorumSize = other.getAckQuorumSize(); builder.state = other.getState(); - if (builder.state == LedgerMetadataFormat.State.CLOSED) { + if (builder.state == State.CLOSED) { builder.lastEntryId = Optional.of(other.getLastEntryId()); builder.length = Optional.of(other.getLength()); } @@ -143,12 +144,12 @@ public LedgerMetadataBuilder replaceEnsembleEntry(long firstEntry, List> getAllEnsembles(); + /** + * Returns the state of the metadata. + * + * @return the state of the metadata. + */ + State getState(); + /** + * Possible metadata states. + */ + enum State { + /** The ledger is open. New entry may be added to it. */ + OPEN, + + /** A reader has tried to, or may be trying to recover the ledger. + The writer may be able to add new entries if fencing hasn't already occurred, + but any attempt to change ensemble will fail and the write will be forced to + close the ledger. + */ + IN_RECOVERY, + + /** The ledger is closed. No new entries may be added to it. + The length and lastEntryId are fixed. Ensembles may change, but only for rereplication. + */ + CLOSED; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 306cf50d052..2bd9b13900a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -75,6 +75,7 @@ public abstract class AbstractZkLedgerManager implements LedgerManager, Watcher @VisibleForTesting static final int ZK_CONNECT_BACKOFF_MS = 200; + private final LedgerMetadataSerDe serDe; protected final AbstractConfiguration conf; protected final ZooKeeper zk; protected final String ledgerRootPath; @@ -159,6 +160,7 @@ private void handleMetadata(Versioned result, Throwable exceptio * ZooKeeper Client Handle */ protected AbstractZkLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { + this.serDe = new LedgerMetadataSerDe(); this.conf = conf; this.zk = zk; this.ledgerRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf); @@ -264,7 +266,7 @@ public void processResult(int rc, String path, Object ctx, String name) { } }; List zkAcls = ZkUtils.getACLs(conf); - ZkUtils.asyncCreateFullPathOptimistic(zk, ledgerPath, metadata.serialize(), zkAcls, + ZkUtils.asyncCreateFullPathOptimistic(zk, ledgerPath, serDe.serialize(metadata), zkAcls, CreateMode.PERSISTENT, scb, null); return promise; } @@ -400,7 +402,7 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta try { LongVersion version = new LongVersion(stat.getVersion()); - LedgerMetadata metadata = LedgerMetadata.parseConfig(data, Optional.of(stat.getCtime())); + LedgerMetadata metadata = serDe.parseConfig(data, Optional.of(stat.getCtime())); promise.complete(new Versioned<>(metadata, version)); } catch (Throwable t) { LOG.error("Could not parse ledger metadata for ledger: {}", ledgerId, t); @@ -421,7 +423,7 @@ public CompletableFuture> writeLedgerMetadata(long led } final LongVersion zv = (LongVersion) currentVersion; zk.setData(getLedgerPath(ledgerId), - metadata.serialize(), (int) zv.getLongVersion(), + serDe.serialize(metadata), (int) zv.getLongVersion(), new StatCallback() { @Override public void processResult(int rc, String path, Object ctx, Stat stat) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java new file mode 100644 index 00000000000..26b616b6eca --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -0,0 +1,338 @@ +/** + * 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.meta; + +import static com.google.common.base.Charsets.UTF_8; +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ByteString; +import com.google.protobuf.TextFormat; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.StringReader; +import java.nio.CharBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata.State; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Serialization and deserialization for LedgerMetadata. + */ +public class LedgerMetadataSerDe { + private static final Logger log = LoggerFactory.getLogger(LedgerMetadataSerDe.class); + + public static final int CURRENT_METADATA_FORMAT_VERSION = 2; + private static final int LOWEST_COMPAT_METADATA_FORMAT_VERSION = 0; + + // for pulling the version + private static final String VERSION_KEY = "BookieMetadataFormatVersion"; + private static final String LINE_SPLITTER = "\n"; + private static final String FIELD_SPLITTER = "\t"; + + // old V1 constants + private static final String V1_CLOSED_TAG = "CLOSED"; + private static final int V1_IN_RECOVERY_ENTRY_ID = -102; + + public byte[] serialize(LedgerMetadata metadata) { + if (metadata.getMetadataFormatVersion() == 1) { + return serializeVersion1(metadata); + } + + StringBuilder s = new StringBuilder(); + s.append(VERSION_KEY).append(FIELD_SPLITTER) + .append(CURRENT_METADATA_FORMAT_VERSION).append(LINE_SPLITTER); + s.append(TextFormat.printToString(buildProtoFormat(metadata))); + if (log.isDebugEnabled()) { + log.debug("Serialized config: {}", s); + } + return s.toString().getBytes(UTF_8); + } + + private byte[] serializeVersion1(LedgerMetadata metadata) { + StringBuilder s = new StringBuilder(); + s.append(VERSION_KEY).append(FIELD_SPLITTER) + .append(metadata.getMetadataFormatVersion()).append(LINE_SPLITTER); + s.append(metadata.getWriteQuorumSize()).append(LINE_SPLITTER) + .append(metadata.getEnsembleSize()).append(LINE_SPLITTER).append(metadata.getLength()); + + for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { + s.append(LINE_SPLITTER).append(entry.getKey()); + for (BookieSocketAddress addr : entry.getValue()) { + s.append(FIELD_SPLITTER); + s.append(addr.toString()); + } + } + + if (metadata.getState() == State.IN_RECOVERY) { + s.append(LINE_SPLITTER).append(V1_IN_RECOVERY_ENTRY_ID) + .append(FIELD_SPLITTER).append(V1_CLOSED_TAG); + } else if (metadata.getState() == State.CLOSED) { + s.append(LINE_SPLITTER).append(metadata.getLastEntryId()) + .append(FIELD_SPLITTER).append(V1_CLOSED_TAG); + } else { + checkArgument(metadata.getState() == State.OPEN, + String.format("Unknown state %s for V1 serialization", metadata.getState())); + } + + if (log.isDebugEnabled()) { + log.debug("Serialized config: {}", s); + } + + return s.toString().getBytes(UTF_8); + } + + @VisibleForTesting + public LedgerMetadataFormat buildProtoFormat(LedgerMetadata metadata) { + LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder(); + builder.setQuorumSize(metadata.getWriteQuorumSize()) + .setAckQuorumSize(metadata.getAckQuorumSize()) + .setEnsembleSize(metadata.getEnsembleSize()) + .setLength(metadata.getLength()) + .setLastEntryId(metadata.getLastEntryId()); + + switch (metadata.getState()) { + case CLOSED: + builder.setState(LedgerMetadataFormat.State.CLOSED); + break; + case IN_RECOVERY: + builder.setState(LedgerMetadataFormat.State.IN_RECOVERY); + break; + case OPEN: + builder.setState(LedgerMetadataFormat.State.OPEN); + break; + default: + checkArgument(false, + String.format("Unknown state %s for protobuf serialization", metadata.getState())); + break; + } + + if (metadata.shouldStoreCtime()) { + builder.setCtime(metadata.getCtime()); + } + + if (metadata.hasPassword()) { + builder.setDigestType(apiToProtoDigestType(metadata.getDigestType())) + .setPassword(ByteString.copyFrom(metadata.getPassword())); + } + + Map customMetadata = metadata.getCustomMetadata(); + if (customMetadata.size() > 0) { + LedgerMetadataFormat.cMetadataMapEntry.Builder cMetadataBuilder = + LedgerMetadataFormat.cMetadataMapEntry.newBuilder(); + for (Map.Entry entry : customMetadata.entrySet()) { + cMetadataBuilder.setKey(entry.getKey()).setValue(ByteString.copyFrom(entry.getValue())); + builder.addCustomMetadata(cMetadataBuilder.build()); + } + } + + for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { + LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder(); + segmentBuilder.setFirstEntryId(entry.getKey()); + for (BookieSocketAddress addr : entry.getValue()) { + segmentBuilder.addEnsembleMember(addr.toString()); + } + builder.addSegment(segmentBuilder.build()); + } + return builder.build(); + } + + + /** + * Parses a given byte array and transforms into a LedgerConfig object. + * + * @param bytes + * byte array to parse + * @param metadataStoreCtime + * metadata store creation time, used for legacy ledgers + * @return LedgerConfig + * @throws IOException + * if the given byte[] cannot be parsed + */ + public LedgerMetadata parseConfig(byte[] bytes, + Optional metadataStoreCtime) throws IOException { + String config = new String(bytes, UTF_8); + + if (log.isDebugEnabled()) { + log.debug("Parsing Config: {}", config); + } + BufferedReader reader = new BufferedReader(new StringReader(config)); + String versionLine = reader.readLine(); + if (versionLine == null) { + throw new IOException("Invalid metadata. Content missing"); + } + final int metadataFormatVersion; + if (versionLine.startsWith(VERSION_KEY)) { + String parts[] = versionLine.split(FIELD_SPLITTER); + metadataFormatVersion = Integer.parseInt(parts[1]); + } else { + // if no version is set, take it to be version 1 + // as the parsing is the same as what we had before + // we introduce versions + metadataFormatVersion = 1; + // reset the reader + reader.close(); + reader = new BufferedReader(new StringReader(config)); + } + + if (metadataFormatVersion < LOWEST_COMPAT_METADATA_FORMAT_VERSION + || metadataFormatVersion > CURRENT_METADATA_FORMAT_VERSION) { + throw new IOException( + String.format("Metadata version not compatible. Expected between %d and %d, but got %d", + LOWEST_COMPAT_METADATA_FORMAT_VERSION, CURRENT_METADATA_FORMAT_VERSION, + metadataFormatVersion)); + } + + if (metadataFormatVersion == 1) { + return parseVersion1Config(reader); + } + + LedgerMetadataBuilder builder = LedgerMetadataBuilder.create() + .withMetadataFormatVersion(metadataFormatVersion); + + // remaining size is total minus the length of the version line and '\n' + char[] configBuffer = new char[config.length() - (versionLine.length() + 1)]; + if (configBuffer.length != reader.read(configBuffer, 0, configBuffer.length)) { + throw new IOException("Invalid metadata buffer"); + } + + LedgerMetadataFormat.Builder formatBuilder = LedgerMetadataFormat.newBuilder(); + TextFormat.merge((CharSequence) CharBuffer.wrap(configBuffer), formatBuilder); + LedgerMetadataFormat data = formatBuilder.build(); + + builder.withEnsembleSize(data.getEnsembleSize()); + builder.withWriteQuorumSize(data.getQuorumSize()); + if (data.hasAckQuorumSize()) { + builder.withAckQuorumSize(data.getAckQuorumSize()); + } else { + builder.withAckQuorumSize(data.getQuorumSize()); + } + + if (data.hasCtime()) { + builder.withCreationTime(data.getCtime()).storingCreationTime(true); + } else if (metadataStoreCtime.isPresent()) { + builder.withCreationTime(metadataStoreCtime.get()).storingCreationTime(false); + } + + if (data.getState() == LedgerMetadataFormat.State.IN_RECOVERY) { + builder.withInRecoveryState(); + } else if (data.getState() == LedgerMetadataFormat.State.CLOSED) { + builder.withClosedState().withLastEntryId(data.getLastEntryId()).withLength(data.getLength()); + } + + if (data.hasPassword()) { + builder.withPassword(data.getPassword().toByteArray()) + .withDigestType(protoToApiDigestType(data.getDigestType())); + } + + for (LedgerMetadataFormat.Segment s : data.getSegmentList()) { + List addrs = new ArrayList<>(); + for (String addr : s.getEnsembleMemberList()) { + addrs.add(new BookieSocketAddress(addr)); + } + builder.newEnsembleEntry(s.getFirstEntryId(), addrs); + } + + if (data.getCustomMetadataCount() > 0) { + builder.withCustomMetadata(data.getCustomMetadataList().stream().collect( + Collectors.toMap(e -> e.getKey(), + e -> e.getValue().toByteArray()))); + } + return builder.build(); + } + + static LedgerMetadata parseVersion1Config(BufferedReader reader) throws IOException { + LedgerMetadataBuilder builder = LedgerMetadataBuilder.create().withMetadataFormatVersion(1); + try { + int quorumSize = Integer.parseInt(reader.readLine()); + int ensembleSize = Integer.parseInt(reader.readLine()); + long length = Long.parseLong(reader.readLine()); + + builder.withEnsembleSize(ensembleSize).withWriteQuorumSize(quorumSize).withAckQuorumSize(quorumSize); + + String line = reader.readLine(); + while (line != null) { + String parts[] = line.split(FIELD_SPLITTER); + + if (parts[1].equals(V1_CLOSED_TAG)) { + Long l = Long.parseLong(parts[0]); + if (l == V1_IN_RECOVERY_ENTRY_ID) { + builder.withInRecoveryState(); + } else { + builder.withClosedState().withLastEntryId(l).withLength(length); + } + break; + } + + ArrayList addrs = new ArrayList(); + for (int j = 1; j < parts.length; j++) { + addrs.add(new BookieSocketAddress(parts[j])); + } + builder.newEnsembleEntry(Long.parseLong(parts[0]), addrs); + + line = reader.readLine(); + } + return builder.build(); + } catch (NumberFormatException e) { + throw new IOException(e); + } + } + + private static LedgerMetadataFormat.DigestType apiToProtoDigestType(DigestType digestType) { + switch (digestType) { + case MAC: + return LedgerMetadataFormat.DigestType.HMAC; + case CRC32: + return LedgerMetadataFormat.DigestType.CRC32; + case CRC32C: + return LedgerMetadataFormat.DigestType.CRC32C; + case DUMMY: + return LedgerMetadataFormat.DigestType.DUMMY; + default: + throw new IllegalArgumentException("Unable to convert digest type " + digestType); + } + } + + private static DigestType protoToApiDigestType(LedgerMetadataFormat.DigestType digestType) { + switch (digestType) { + case HMAC: + return DigestType.MAC; + case CRC32: + return DigestType.CRC32; + case CRC32C: + return DigestType.CRC32C; + case DUMMY: + return DigestType.DUMMY; + default: + throw new IllegalArgumentException("Unable to convert digest type " + digestType); + } + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index 35658dde08a..7e28217d02b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -210,7 +210,7 @@ public LedgerIdGenerator newLedgerIdGenerator() { static class MsLedgerManager implements LedgerManager, MetastoreWatcher { final ZooKeeper zk; final AbstractConfiguration conf; - + private final LedgerMetadataSerDe serDe; final MetaStore metastore; final MetastoreScannableTable ledgerTable; final int maxEntriesPerScan; @@ -284,6 +284,7 @@ private void handleMetadata(Versioned metadata, Throwable except this.conf = conf; this.zk = zk; this.metastore = metastore; + this.serDe = new LedgerMetadataSerDe(); try { ledgerTable = metastore.createScannableTable(TABLE_NAME); @@ -394,7 +395,7 @@ public void complete(int rc, Version version, Object ctx) { } }; - ledgerTable.put(ledgerId2Key(lid), new Value().setField(META_FIELD, metadata.serialize()), + ledgerTable.put(ledgerId2Key(lid), new Value().setField(META_FIELD, serDe.serialize(metadata)), Version.NEW, msCallback, null); return promise; } @@ -440,7 +441,7 @@ public void complete(int rc, Versioned value, Object ctx) { return; } try { - LedgerMetadata metadata = LedgerMetadata.parseConfig( + LedgerMetadata metadata = serDe.parseConfig( value.getValue().getField(META_FIELD), Optional.empty()); promise.complete(new Versioned<>(metadata, value.getVersion())); } catch (IOException e) { @@ -456,7 +457,7 @@ public void complete(int rc, Versioned value, Object ctx) { @Override public CompletableFuture> writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, Version currentVersion) { - Value data = new Value().setField(META_FIELD, metadata.serialize()); + Value data = new Value().setField(META_FIELD, serDe.serialize(metadata)); if (LOG.isDebugEnabled()) { LOG.debug("Writing ledger {} metadata, version {}", new Object[] { ledgerId, currentVersion }); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java index 1a924e5d5ee..4225c081215 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.http.service.HttpServiceResponse; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.proto.BookieServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,10 +47,13 @@ public class GetLedgerMetaService implements HttpEndpointService { protected ServerConfiguration conf; protected BookieServer bookieServer; + private final LedgerMetadataSerDe serDe; + public GetLedgerMetaService(ServerConfiguration conf, BookieServer bookieServer) { checkNotNull(conf); this.conf = conf; this.bookieServer = bookieServer; + this.serDe = new LedgerMetadataSerDe(); } @Override @@ -66,7 +70,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { // output Map output = Maps.newHashMap(); LedgerMetadata md = manager.readLedgerMetadata(ledgerId).get().getValue(); - output.put(ledgerId.toString(), new String(md.serialize(), UTF_8)); + output.put(ledgerId.toString(), new String(serDe.serialize(md), UTF_8)); manager.close(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java index 0d10fcdac0b..1df1b36ca08 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.http.service.HttpServiceResponse; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; @@ -51,21 +52,24 @@ public class ListLedgerService implements HttpEndpointService { protected ServerConfiguration conf; protected BookieServer bookieServer; + private final LedgerMetadataSerDe serDe; public ListLedgerService(ServerConfiguration conf, BookieServer bookieServer) { checkNotNull(conf); this.conf = conf; this.bookieServer = bookieServer; + this.serDe = new LedgerMetadataSerDe(); + } // Number of LedgerMetadata contains in each page static final int LIST_LEDGER_BATCH_SIZE = 100; - static void keepLedgerMetadata(long ledgerId, CompletableFuture> future, - LinkedHashMap output) + private void keepLedgerMetadata(long ledgerId, CompletableFuture> future, + LinkedHashMap output) throws Exception { LedgerMetadata md = future.get().getValue(); - output.put(Long.valueOf(ledgerId).toString(), new String(md.serialize(), UTF_8)); + output.put(Long.valueOf(ledgerId).toString(), new String(serDe.serialize(md), UTF_8)); } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java index 38e5260c8ab..80e1c759806 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java @@ -25,9 +25,11 @@ import static org.junit.Assert.assertTrue; import com.google.common.collect.Lists; +import java.util.Base64; import java.util.Collections; import java.util.List; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; import org.junit.Test; @@ -37,8 +39,7 @@ */ public class LedgerMetadataTest { - private static final String passwdStr = "testPasswd"; - private static final byte[] passwd = passwdStr.getBytes(UTF_8); + private static final byte[] passwd = "testPasswd".getBytes(UTF_8); @Test public void testGetters() { @@ -78,7 +79,7 @@ public void testStoreSystemtimeAsLedgerCtimeEnabled() .withCreationTime(System.currentTimeMillis()) .storingCreationTime(true) .build(); - LedgerMetadataFormat format = lm.buildProtoFormat(); + LedgerMetadataFormat format = new LedgerMetadataSerDe().buildProtoFormat(lm); assertTrue(format.hasCtime()); } @@ -92,7 +93,7 @@ public void testStoreSystemtimeAsLedgerCtimeDisabled() LedgerMetadata lm = LedgerMetadataBuilder.create() .newEnsembleEntry(0L, ensemble).build(); - LedgerMetadataFormat format = lm.buildProtoFormat(); + LedgerMetadataFormat format = new LedgerMetadataSerDe().buildProtoFormat(lm); assertFalse(format.hasCtime()); } @@ -106,11 +107,11 @@ public void testToString() { LedgerMetadata lm1 = LedgerMetadataBuilder.create() .withDigestType(DigestType.CRC32.toApiDigestType()) .withPassword(passwd) - .newEnsembleEntry(0L, ensemble).build(); + .newEnsembleEntry(0L, ensemble) + .build(); - assertTrue("toString should contain 'password' field", lm1.toString().contains("password")); - assertTrue("toString should contain password value", lm1.toString().contains(passwdStr)); - assertFalse("toSafeString should not contain 'password' field", lm1.toSafeString().contains("password")); - assertFalse("toSafeString should not contain password value", lm1.toSafeString().contains(passwdStr)); + assertTrue("toString should contain password value", + lm1.toString().contains(Base64.getEncoder().encodeToString(passwd))); + assertTrue("toSafeString should not contain password value", lm1.toSafeString().contains("OMITTED")); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index fb5638581f2..6e37fee51ce 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -91,6 +91,7 @@ public class AbstractZkLedgerManagerTest extends MockZooKeeperTestCase { private ScheduledExecutorService scheduler; private MockExecutorController schedulerController; private LedgerMetadata metadata; + private LedgerMetadataSerDe serDe; @Before public void setup() throws Exception { @@ -140,6 +141,8 @@ public void setup() throws Exception { assertSame(mockZk, ledgerManager.zk); assertSame(conf, ledgerManager.conf); assertSame(scheduler, ledgerManager.scheduler); + + this.serDe = new LedgerMetadataSerDe(); } @After @@ -338,7 +341,7 @@ public void testReadLedgerMetadataSuccess() throws Exception { when(stat.getCtime()).thenReturn(metadata.getCtime()); mockZkGetData( ledgerStr, false, - KeeperException.Code.OK.intValue(), metadata.serialize(), stat); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); Versioned readMetadata = result(ledgerManager.readLedgerMetadata(ledgerId)); assertEquals(metadata, readMetadata.getValue()); @@ -395,7 +398,7 @@ public void testReadLedgerMetadataStatMissing() throws Exception { mockZkGetData( ledgerStr, false, - KeeperException.Code.OK.intValue(), metadata.serialize(), null); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), null); try { result(ledgerManager.readLedgerMetadata(ledgerId)); @@ -440,7 +443,7 @@ public void testWriteLedgerMetadataSuccess() throws Exception { when(stat.getVersion()).thenReturn(1235); when(stat.getCtime()).thenReturn(metadata.getCtime()); mockZkSetData( - ledgerStr, metadata.serialize(), 1234, + ledgerStr, serDe.serialize(metadata), 1234, KeeperException.Code.OK.intValue(), stat); Version v = ledgerManager.writeLedgerMetadata(ledgerId, metadata, new LongVersion(1234L)).get().getVersion(); @@ -457,7 +460,7 @@ public void testWriteLedgerMetadataBadVersion() throws Exception { String ledgerStr = String.valueOf(ledgerId); mockZkSetData( - ledgerStr, metadata.serialize(), 1234, + ledgerStr, serDe.serialize(metadata), 1234, KeeperException.Code.BADVERSION.intValue(), null); try { @@ -477,7 +480,7 @@ public void testWriteLedgerMetadataException() throws Exception { String ledgerStr = String.valueOf(ledgerId); mockZkSetData( - ledgerStr, metadata.serialize(), 1234, + ledgerStr, serDe.serialize(metadata), 1234, KeeperException.Code.CONNECTIONLOSS.intValue(), null); try { @@ -531,7 +534,7 @@ public void testLedgerMetadataListener() throws Exception { when(stat.getCtime()).thenReturn(metadata.getCtime()); mockZkGetData( ledgerStr, true, - KeeperException.Code.OK.intValue(), metadata.serialize(), stat); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); ledgerManager.registerLedgerMetadataListener(ledgerId, listener); @@ -551,7 +554,7 @@ public void testLedgerMetadataListener() throws Exception { when(stat.getVersion()).thenReturn(1235); mockZkGetData( ledgerStr, true, - KeeperException.Code.OK.intValue(), metadata.serialize(), stat); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); // notify the watcher event notifyWatchedEvent( @@ -593,7 +596,7 @@ public void testLedgerMetadataListenerOnLedgerDeleted() throws Exception { when(stat.getCtime()).thenReturn(metadata.getCtime()); mockZkGetData( ledgerStr, true, - KeeperException.Code.OK.intValue(), metadata.serialize(), stat); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); ledgerManager.registerLedgerMetadataListener(ledgerId, listener); assertTrue(ledgerManager.listeners.containsKey(ledgerId)); @@ -645,7 +648,7 @@ public void testLedgerMetadataListenerOnLedgerDeletedEvent() throws Exception { when(stat.getCtime()).thenReturn(metadata.getCtime()); mockZkGetData( ledgerStr, true, - KeeperException.Code.OK.intValue(), metadata.serialize(), stat); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); ledgerManager.registerLedgerMetadataListener(ledgerId, listener); assertTrue(ledgerManager.listeners.containsKey(ledgerId)); @@ -707,7 +710,7 @@ public void testLedgerMetadataListenerOnRetries() throws Exception { // mock get data to return a valid response mockZkGetData( ledgerStr, true, - KeeperException.Code.OK.intValue(), metadata.serialize(), stat); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); schedulerController.advance(Duration.ofMillis(ZK_CONNECT_BACKOFF_MS)); @@ -734,7 +737,7 @@ public void testLedgerMetadataListenerOnSessionExpired() throws Exception { when(stat.getCtime()).thenReturn(metadata.getCtime()); mockZkGetData( ledgerStr, true, - KeeperException.Code.OK.intValue(), metadata.serialize(), stat); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); ledgerManager.registerLedgerMetadataListener(ledgerId, listener); @@ -782,7 +785,7 @@ public void testUnregisterLedgerMetadataListener() throws Exception { when(stat.getCtime()).thenReturn(metadata.getCtime()); mockZkGetData( ledgerStr, true, - KeeperException.Code.OK.intValue(), metadata.serialize(), stat); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); ledgerManager.registerLedgerMetadataListener(ledgerId, listener); assertTrue(ledgerManager.listeners.containsKey(ledgerId)); @@ -803,7 +806,7 @@ public void testUnregisterLedgerMetadataListener() throws Exception { when(stat.getVersion()).thenReturn(1235); mockZkGetData( ledgerStr, true, - KeeperException.Code.OK.intValue(), metadata.serialize(), stat); + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); // unregister the listener ledgerManager.unregisterLedgerMetadataListener(ledgerId, listener); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index a8b7e86ec16..ff0126a920b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -59,6 +59,7 @@ public interface Hook { final Map> metadataMap; final ExecutorService executor; final boolean ownsExecutor; + final LedgerMetadataSerDe serDe; private Hook preWriteHook = (ledgerId, metadata) -> FutureUtils.value(null); public MockLedgerManager() { @@ -71,6 +72,7 @@ private MockLedgerManager(Map> metadataMap, this.metadataMap = metadataMap; this.executor = executor; this.ownsExecutor = ownsExecutor; + this.serDe = new LedgerMetadataSerDe(); } public MockLedgerManager newClient() { @@ -82,7 +84,7 @@ private Versioned readMetadata(long ledgerId) throws Exception { if (pair == null) { return null; } else { - return new Versioned<>(LedgerMetadata.parseConfig(pair.getRight(), Optional.empty()), pair.getLeft()); + return new Versioned<>(serDe.parseConfig(pair.getRight(), Optional.empty()), pair.getLeft()); } } @@ -101,7 +103,7 @@ public CompletableFuture> createLedgerMetadata(long le if (metadataMap.containsKey(ledgerId)) { executeCallback(() -> promise.completeExceptionally(new BKException.BKLedgerExistException())); } else { - metadataMap.put(ledgerId, Pair.of(new LongVersion(0L), metadata.serialize())); + metadataMap.put(ledgerId, Pair.of(new LongVersion(0L), serDe.serialize(metadata))); try { Versioned readBack = readMetadata(ledgerId); executeCallback(() -> promise.complete(readBack)); @@ -154,7 +156,7 @@ public CompletableFuture> writeLedgerMetadata(long led } else { LongVersion oldVersion = (LongVersion) oldMetadata.getVersion(); metadataMap.put(ledgerId, Pair.of(new LongVersion(oldVersion.getLongVersion() + 1), - metadata.serialize())); + serDe.serialize(metadata))); Versioned readBack = readMetadata(ledgerId); return FutureUtils.value(readBack); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index 3f587c75336..c4d441dbe89 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -45,6 +45,7 @@ import org.apache.bookkeeper.http.service.HttpServiceResponse; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.replication.AuditorElector; @@ -386,7 +387,7 @@ public void testGetLedgerMetaService() throws Exception { assertEquals(1, respBody.size()); // verify LedgerMetadata content is equal assertTrue(respBody.get(ledgerId.toString()).toString() - .equals(new String(lh[0].getLedgerMetadata().serialize()))); + .equals(new String(new LedgerMetadataSerDe().serialize(lh[0].getLedgerMetadata())))); } @Test diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index 8c183abcbfd..5155fad8b17 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -46,6 +46,7 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.metadata.etcd.helpers.KeyIterator; import org.apache.bookkeeper.metadata.etcd.helpers.KeyStream; import org.apache.bookkeeper.metadata.etcd.helpers.ValueStream; @@ -63,9 +64,10 @@ @Slf4j class EtcdLedgerManager implements LedgerManager { - private static final Function LEDGER_METADATA_FUNCTION = bs -> { + private final LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + private final Function ledgerMetadataFunction = bs -> { try { - return LedgerMetadata.parseConfig( + return serDe.parseConfig( bs.getBytes(), Optional.empty() ); @@ -84,6 +86,7 @@ class EtcdLedgerManager implements LedgerManager { new ConcurrentLongHashMap<>(); private final ConcurrentMap listeners = new ConcurrentHashMap<>(); + private volatile boolean closed = false; EtcdLedgerManager(Client client, @@ -121,7 +124,7 @@ public CompletableFuture> createLedgerMetadata(long le .build())) .Else(com.coreos.jetcd.op.Op.put( ledgerKeyBs, - ByteSequence.fromBytes(metadata.serialize()), + ByteSequence.fromBytes(serDe.serialize(metadata)), PutOption.DEFAULT)) .commit() .thenAccept(resp -> { @@ -223,7 +226,7 @@ public CompletableFuture> readLedgerMetadata(long ledg KeyValue kv = getResp.getKvs().get(0); byte[] data = kv.getValue().getBytes(); try { - LedgerMetadata metadata = LedgerMetadata.parseConfig(data, Optional.empty()); + LedgerMetadata metadata = serDe.parseConfig(data, Optional.empty()); promise.complete(new Versioned<>(metadata, new LongVersion(kv.getModRevision()))); } catch (IOException ioe) { log.error("Could not parse ledger metadata for ledger : {}", ledgerId, ioe); @@ -259,7 +262,7 @@ public CompletableFuture> writeLedgerMetadata(long led CmpTarget.modRevision(lv.getLongVersion()))) .Then(com.coreos.jetcd.op.Op.put( ledgerKeyBs, - ByteSequence.fromBytes(metadata.serialize()), + ByteSequence.fromBytes(serDe.serialize(metadata)), PutOption.DEFAULT)) .Else(com.coreos.jetcd.op.Op.get( ledgerKeyBs, @@ -307,7 +310,7 @@ public void registerLedgerMetadataListener(long ledgerId, LedgerMetadataListener ledgerId, (lid) -> new ValueStream<>( client, watchClient, - LEDGER_METADATA_FUNCTION, + ledgerMetadataFunction, ByteSequence.fromString(EtcdUtils.getLedgerKey(scope, ledgerId))) ); LedgerMetadataConsumer lmConsumer = listenerToConsumer(ledgerId, listener, From 1a8082663129b85f0eafaaf24a9c2a0c616d55e1 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Sat, 1 Dec 2018 01:06:29 +0100 Subject: [PATCH 0152/1642] Let OS choose port for vertx test The previous implementation was searching for a open port by repeatedly trying to startServer on ports starting at 8080. However, after the first attempt fails, the Vertx instance in VertxHttpServer is broken and the test hangs. 8080 is a very common port to have stuff running on, so these hangs have happened to be repeatedly. This fix is to allow the OS to choose the port, by specifying 0 as the listening port and querying afterwards. Issue: #1821 Reviewers: Sijie Guo , Enrico Olivelli This closes #1853 from ivankelly/vertx-8080 --- .../http/vertx/VertxHttpServer.java | 10 ++++++-- .../http/vertx/TestVertxHttpServer.java | 25 ++++--------------- 2 files changed, 13 insertions(+), 22 deletions(-) diff --git a/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java b/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java index 26c07e84b69..15d1039ca3a 100644 --- a/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java +++ b/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java @@ -46,11 +46,16 @@ public class VertxHttpServer implements HttpServer { private Vertx vertx; private boolean isRunning; private HttpServiceProvider httpServiceProvider; + private int listeningPort = -1; public VertxHttpServer() { this.vertx = Vertx.vertx(); } + int getListeningPort() { + return listeningPort; + } + @Override public void initialize(HttpServiceProvider httpServiceProvider) { this.httpServiceProvider = httpServiceProvider; @@ -58,7 +63,7 @@ public void initialize(HttpServiceProvider httpServiceProvider) { @Override public boolean startServer(int port) { - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture> future = new CompletableFuture<>(); VertxHttpHandlerFactory handlerFactory = new VertxHttpHandlerFactory(httpServiceProvider); Router router = Router.router(vertx); HttpRouter requestRouter = new HttpRouter(handlerFactory) { @@ -76,9 +81,10 @@ public void start() throws Exception { } }); try { - AsyncResult asyncResult = future.get(); + AsyncResult asyncResult = future.get(); if (asyncResult.succeeded()) { LOG.info("Vertx Http server started successfully"); + listeningPort = asyncResult.result().actualPort(); isRunning = true; return true; } else { diff --git a/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java b/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java index 59fcfd80921..1c547d3ad6f 100644 --- a/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java +++ b/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java @@ -21,6 +21,7 @@ package org.apache.bookkeeper.http.vertx; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import java.io.BufferedReader; import java.io.IOException; @@ -39,19 +40,13 @@ * Unit test {@link VertxHttpServer}. */ public class TestVertxHttpServer { - - private int port = 8080; - @Test public void testStartBasicHttpServer() throws Exception { VertxHttpServer httpServer = new VertxHttpServer(); HttpServiceProvider httpServiceProvider = NullHttpServiceProvider.getInstance(); httpServer.initialize(httpServiceProvider); - int port = getNextPort(); - while (!httpServer.startServer(port)) { - httpServer.stopServer(); - port = getNextPort(); - } + assertTrue(httpServer.startServer(0)); + int port = httpServer.getListeningPort(); HttpResponse httpResponse = sendGet(getUrl(port, HttpRouter.HEARTBEAT)); assertEquals(HttpServer.StatusCode.OK.getValue(), httpResponse.responseCode); assertEquals(HeartbeatService.HEARTBEAT.trim(), httpResponse.responseBody.trim()); @@ -63,11 +58,8 @@ public void testStartMetricsServiceOnRouterPath() throws Exception { VertxHttpServer httpServer = new VertxHttpServer(); HttpServiceProvider httpServiceProvider = NullHttpServiceProvider.getInstance(); httpServer.initialize(httpServiceProvider); - int port = getNextPort(); - while (!httpServer.startServer(port)) { - httpServer.stopServer(); - port = getNextPort(); - } + assertTrue(httpServer.startServer(0)); + int port = httpServer.getListeningPort(); HttpResponse httpResponse = sendGet(getUrl(port, HttpRouter.METRICS)); assertEquals(HttpServer.StatusCode.OK.getValue(), httpResponse.responseCode); httpServer.stopServer(); @@ -109,11 +101,4 @@ public HttpResponse(int responseCode, String responseBody) { this.responseBody = responseBody; } } - - private int getNextPort() throws Exception { - if (port > 65535) { - throw new Exception("No port available"); - } - return port++; - } } From ed00022a97751bcaa34b63a5141a4fa9f5d152d4 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 30 Nov 2018 16:29:01 -0800 Subject: [PATCH 0153/1642] [TABLE SERVICE] add a bookie registration based grpc name resolver Descriptions of the changes in this PR: *Motivation* Table service is using grpc name resolver to figure out the alive servers serving table service. Currently it is using default dns name resolver. In the context of Pulsar, since brokers talk to bookies via zookeeper service discovery, it is making sense for brokers talk to table service via zookeeper service discovery. So this PR is to add a bookie registration based grpc name resolver. *Changes* Implement a bookie-registration library based grpc name resolver. Reviewers: Jia Zhai This closes #1842 from sijie/add_zookeeper_resolver --- stream/bk-grpc-name-resolver/pom.xml | 70 ++++++++ .../resolver/BKRegistrationNameResolver.java | 133 +++++++++++++++ .../BKRegistrationNameResolverProvider.java | 84 ++++++++++ .../grpc/resolver/package-info.java | 23 +++ .../BKRegistrationNameResolverTest.java | 155 ++++++++++++++++++ .../grpc/resolver/GrpcChannelsTest.java | 44 +++++ .../clients/utils/GrpcChannels.java | 19 ++- .../clients/utils/GrpcChannelsTest.java | 70 ++++++++ .../resolver/NameResolverFactoryProvider.java | 37 +++++ .../resolver/NameResolverProviderFactory.java | 64 ++++++++ .../resolver/ServiceNameResolverProvider.java | 37 +---- stream/pom.xml | 1 + 12 files changed, 702 insertions(+), 35 deletions(-) create mode 100644 stream/bk-grpc-name-resolver/pom.xml create mode 100644 stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolver.java create mode 100644 stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverProvider.java create mode 100644 stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/package-info.java create mode 100644 stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java create mode 100644 stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/GrpcChannelsTest.java create mode 100644 stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/utils/GrpcChannelsTest.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/resolver/NameResolverFactoryProvider.java create mode 100644 stream/common/src/main/java/org/apache/bookkeeper/common/resolver/NameResolverProviderFactory.java diff --git a/stream/bk-grpc-name-resolver/pom.xml b/stream/bk-grpc-name-resolver/pom.xml new file mode 100644 index 00000000000..312adfbf1dd --- /dev/null +++ b/stream/bk-grpc-name-resolver/pom.xml @@ -0,0 +1,70 @@ + + + + 4.0.0 + + stream-storage-parent + org.apache.bookkeeper + 4.9.0-SNAPSHOT + .. + + org.apache.bookkeeper + bk-grpc-name-resolver + Apache BookKeeper :: Stream Storage :: Common :: BK Grpc Name Resolver + + + org.apache.bookkeeper + stream-storage-common + ${project.version} + + + org.apache.bookkeeper + bookkeeper-server + ${project.version} + + + org.apache.bookkeeper + stream-storage-java-client-base + ${project.version} + test + + + org.apache.bookkeeper + bookkeeper-common + ${project.version} + test-jar + test + + + org.apache.zookeeper + zookeeper + test-jar + test + + + org.apache.bookkeeper + bookkeeper-server + ${project.version} + test-jar + test + + + diff --git a/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolver.java b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolver.java new file mode 100644 index 00000000000..cc25978079b --- /dev/null +++ b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolver.java @@ -0,0 +1,133 @@ +/* + * 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.grpc.resolver; + +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.grpc.Attributes; +import io.grpc.EquivalentAddressGroup; +import io.grpc.NameResolver; +import java.net.URI; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.meta.MetadataClientDriver; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.stats.NullStatsLogger; + +/** + * A {@link NameResolver} implementation based on bookkeeper {@link org.apache.bookkeeper.discover.RegistrationClient}. + */ +class BKRegistrationNameResolver extends NameResolver { + + private final MetadataClientDriver clientDriver; + private final URI serviceURI; + private final ScheduledExecutorService executor; + + private Listener listener; + private boolean shutdown; + private boolean resolving; + + BKRegistrationNameResolver(MetadataClientDriver clientDriver, + URI serviceURI) { + this.clientDriver = clientDriver; + this.serviceURI = serviceURI; + this.executor = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("registration-name-resolver").build()); + } + + @Override + public String getServiceAuthority() { + return serviceURI.getAuthority(); + } + + @Override + public synchronized void start(Listener listener) { + checkState(null == this.listener, "Resolver already started"); + this.listener = Objects.requireNonNull(listener, "Listener is null"); + + ClientConfiguration conf = new ClientConfiguration(); + conf.setMetadataServiceUri(serviceURI.toString()); + + try { + clientDriver.initialize(conf, executor, NullStatsLogger.INSTANCE, Optional.empty()); + } catch (MetadataException e) { + throw new RuntimeException("Failed to initialize registration client driver at " + serviceURI, e); + } + + resolve(); + } + + private synchronized void resolve() { + if (resolving || shutdown) { + return; + } + resolving = true; + this.clientDriver.getRegistrationClient().watchWritableBookies(bookies -> { + Listener savedListener; + synchronized (this) { + savedListener = listener; + } + savedListener.onAddresses( + hostsToEquivalentAddressGroups(bookies.getValue()), + Attributes.EMPTY + ); + }).whenComplete((ignored, cause) -> { + try { + if (null != cause) { + resolve(); + } + } finally { + synchronized (this) { + resolving = false; + } + } + }); + } + + private static List hostsToEquivalentAddressGroups(Set bookies) { + return bookies.stream() + .map(addr -> new EquivalentAddressGroup( + Collections.singletonList(addr.getSocketAddress()), + Attributes.EMPTY + )) + .collect(Collectors.toList()); + } + + @Override + public void shutdown() { + synchronized (this) { + if (shutdown) { + return; + } + shutdown = true; + } + executor.shutdown(); + clientDriver.close(); + } +} diff --git a/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverProvider.java b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverProvider.java new file mode 100644 index 00000000000..1c61de72aea --- /dev/null +++ b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverProvider.java @@ -0,0 +1,84 @@ +/* + * 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.grpc.resolver; + +import com.google.common.collect.Lists; +import io.grpc.Attributes; +import io.grpc.NameResolver; +import io.grpc.NameResolverProvider; +import java.net.URI; +import javax.annotation.Nullable; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.common.resolver.NameResolverFactoryProvider; +import org.apache.bookkeeper.common.resolver.NameResolverProviderFactory; +import org.apache.bookkeeper.meta.MetadataClientDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; + +/** + * An implementation of {@link NameResolverProvider} that provides {@link io.grpc.NameResolver}s + * to resolve servers registered using bookkeeper registration library. + */ +@Slf4j +public class BKRegistrationNameResolverProvider extends NameResolverFactoryProvider { + + @Override + protected boolean isAvailable() { + return true; + } + + @Override + protected int priority() { + return 100; + } + + @Nullable + @Override + public NameResolver newNameResolver(URI targetUri, Attributes params) { + ServiceURI serviceURI; + try { + serviceURI = ServiceURI.create(targetUri); + } catch (NullPointerException | IllegalArgumentException e) { + // invalid uri here, so return null to allow grpc to use other name resolvers + log.info("BKRegistrationNameResolverProvider doesn't know how to resolve {} : cause {}", + targetUri, e.getMessage()); + return null; + } + + MetadataClientDriver clientDriver; + try { + clientDriver = MetadataDrivers.getClientDriver(serviceURI.getUri()); + return new BKRegistrationNameResolver(clientDriver, serviceURI.getUri()); + } catch (IllegalArgumentException iae) { + log.error("Unknown service uri : {}", serviceURI, iae); + return null; + } + } + + @Override + public String getDefaultScheme() { + return ServiceURI.SERVICE_ZK; + } + + @Override + public NameResolver.Factory toFactory() { + return new NameResolverProviderFactory(Lists.newArrayList(this)); + } +} diff --git a/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/package-info.java b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/package-info.java new file mode 100644 index 00000000000..5e82b08fa61 --- /dev/null +++ b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/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 collection of grpc resolovers. + */ +package org.apache.bookkeeper.grpc.resolver; \ No newline at end of file diff --git a/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java new file mode 100644 index 00000000000..726c8b00b97 --- /dev/null +++ b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java @@ -0,0 +1,155 @@ +/* + * 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.grpc.resolver; + +import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; +import static org.junit.Assert.assertEquals; + +import io.grpc.Attributes; +import io.grpc.EquivalentAddressGroup; +import io.grpc.NameResolver; +import io.grpc.NameResolver.Listener; +import io.grpc.Status; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.net.URI; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.Collectors; +import lombok.Cleanup; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.ZooDefs.Ids; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * Unit test {@link BKRegistrationNameResolver}. + */ +public class BKRegistrationNameResolverTest extends BookKeeperClusterTestCase { + + private static final String ROOT_PATH = "/resolver-test"; + private static final String SERVERS_PATH = ROOT_PATH + "/servers"; + + private final BKRegistrationNameResolverProvider resolverProvider; + + private MetadataBookieDriver bookieDriver; + private URI serviceUri; + + public BKRegistrationNameResolverTest() { + super(0); + this.resolverProvider = new BKRegistrationNameResolverProvider(); + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + + zkc.transaction() + .create(ROOT_PATH, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT) + .create(SERVERS_PATH, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT) + .create(SERVERS_PATH + "/" + AVAILABLE_NODE, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT) + .commit(); + + serviceUri = URI.create("zk://" + zkUtil.getZooKeeperConnectString() + SERVERS_PATH); + + + ServerConfiguration serverConf = new ServerConfiguration(); + serverConf.setMetadataServiceUri(serviceUri.toString()); + bookieDriver = MetadataDrivers.getBookieDriver(serviceUri); + bookieDriver.initialize(serverConf, () -> {}, NullStatsLogger.INSTANCE); + } + + @After + @Override + public void tearDown() throws Exception { + bookieDriver.close(); + + super.tearDown(); + } + + @Test + public void testNameResolver() throws Exception { + int numServers = 3; + + Set addressSet = new HashSet<>(); + for (int i = 0; i < numServers; i++) { + InetSocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i); + addressSet.add(address); + bookieDriver.getRegistrationManager().registerBookie( + "127.0.0.1:" + (3181 + i), false + ); + } + + LinkedBlockingQueue> notifications = new LinkedBlockingQueue<>(); + + + @Cleanup("shutdown") + NameResolver resolver = resolverProvider.newNameResolver(serviceUri, Attributes.EMPTY); + resolver.start(new Listener() { + @Override + public void onAddresses(List servers, Attributes attributes) { + notifications.add(servers); + } + + @Override + public void onError(Status error) { + + } + }); + + List groups = notifications.take(); + assertEquals(numServers, groups.size()); + + Set receivedSet = groups.stream() + .map(group -> group.getAddresses().get(0)) + .collect(Collectors.toSet()); + assertEquals(addressSet, receivedSet); + + // add 3 more servers + + for (int i = numServers; i < 2 * numServers; i++) { + InetSocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i); + addressSet.add(address); + bookieDriver.getRegistrationManager().registerBookie( + "127.0.0.1:" + (3181 + i), false + ); + } + + List notification = notifications.take(); + while (notification.size() < 2 * numServers) { + notification = notifications.take(); + } + assertEquals(2 * numServers, notification.size()); + receivedSet = notification.stream() + .map(group -> group.getAddresses().get(0)) + .collect(Collectors.toSet()); + assertEquals(addressSet, receivedSet); + } + +} diff --git a/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/GrpcChannelsTest.java b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/GrpcChannelsTest.java new file mode 100644 index 00000000000..9df1bf1deb8 --- /dev/null +++ b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/GrpcChannelsTest.java @@ -0,0 +1,44 @@ +/* + * 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.grpc.resolver; + +import static org.junit.Assert.assertTrue; + +import io.grpc.ManagedChannelBuilder; +import io.grpc.netty.NettyChannelBuilder; +import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.clients.utils.GrpcChannels; +import org.junit.Test; + +/** + * Unit test {@link org.apache.bookkeeper.clients.utils.GrpcChannels} with registration based name resolver. + */ +public class GrpcChannelsTest { + + @Test + public void testZKServiceUri() { + String serviceUri = "zk://127.0.0.1/stream/servers"; + ManagedChannelBuilder builder = GrpcChannels.createChannelBuilder( + serviceUri, + StorageClientSettings.newBuilder().serviceUri(serviceUri).build()); + assertTrue(builder instanceof NettyChannelBuilder); + } + +} diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcChannels.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcChannels.java index 8588359f090..f05caec6736 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcChannels.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcChannels.java @@ -23,7 +23,9 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.common.resolver.NameResolverFactoryProvider; import org.apache.bookkeeper.common.resolver.ServiceNameResolverProvider; +import org.apache.bookkeeper.common.util.ReflectionUtils; /** * Utils to create grpc channels. @@ -32,6 +34,8 @@ public final class GrpcChannels { private static final String BACKEND_INPROCESS = "inprocess"; + private static final String BK_REG_NAME_RESOLVER_PROVIDER = + "org.apache.bookkeeper.grpc.resolver.BKRegistrationNameResolverProvider"; private GrpcChannels() {} @@ -51,9 +55,22 @@ public static ManagedChannelBuilder createChannelBuilder(String serviceUri, // this is an inprocess service, so build an inprocess channel. String serviceName = uri.getServiceHosts()[0]; builder = InProcessChannelBuilder.forName(serviceName).directExecutor(); - } else { + } else if (null == uri.getServiceName() || ServiceURI.SERVICE_BK.equals(uri.getServiceName())) { builder = ManagedChannelBuilder.forTarget(serviceUri) .nameResolverFactory(new ServiceNameResolverProvider().toFactory()); + } else { + NameResolverFactoryProvider provider; + try { + provider = ReflectionUtils.newInstance( + BK_REG_NAME_RESOLVER_PROVIDER, + NameResolverFactoryProvider.class); + } catch (RuntimeException re) { + log.error("It seems that you don't have `bk-grpc-name-resolver` in your class path." + + " Please make sure you include it as your application's dependency."); + throw re; + } + builder = ManagedChannelBuilder.forTarget(serviceUri) + .nameResolverFactory(provider.toFactory()); } if (settings.usePlaintext()) { builder = builder.usePlaintext(); diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/utils/GrpcChannelsTest.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/utils/GrpcChannelsTest.java new file mode 100644 index 00000000000..3e3e81e98e8 --- /dev/null +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/utils/GrpcChannelsTest.java @@ -0,0 +1,70 @@ +/* + * 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.clients.utils; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import io.grpc.ManagedChannelBuilder; +import io.grpc.inprocess.InProcessChannelBuilder; +import io.grpc.netty.NettyChannelBuilder; +import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.junit.Test; + +/** + * Unit test {@link GrpcChannels}. + */ +public class GrpcChannelsTest { + + @Test + public void testInprocessServiceUri() { + String serviceUri = "bk+inprocess://service"; + ManagedChannelBuilder builder = GrpcChannels.createChannelBuilder( + serviceUri, + StorageClientSettings.newBuilder().serviceUri(serviceUri).build() + ); + assertTrue(builder instanceof InProcessChannelBuilder); + } + + @Test + public void testBKServiceUri() { + String serviceUri = "bk://127.0.0.1"; + ManagedChannelBuilder builder = GrpcChannels.createChannelBuilder( + serviceUri, + StorageClientSettings.newBuilder().serviceUri(serviceUri).build() + ); + assertTrue(builder instanceof NettyChannelBuilder); + } + + @Test + public void testZKServiceUri() { + String serviceUri = "zk://127.0.0.1/stream/servers"; + try { + GrpcChannels.createChannelBuilder( + serviceUri, + StorageClientSettings.newBuilder().serviceUri(serviceUri).build() + ); + fail("Should fail to create grpc channel because `bk-grpc-name-resolver` is not in the classpath"); + } catch (RuntimeException re) { + assertTrue(re.getCause() instanceof ClassNotFoundException); + } + } + +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/NameResolverFactoryProvider.java b/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/NameResolverFactoryProvider.java new file mode 100644 index 00000000000..c8edeaee4af --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/NameResolverFactoryProvider.java @@ -0,0 +1,37 @@ +/* + * 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.common.resolver; + +import io.grpc.NameResolver; +import io.grpc.NameResolverProvider; + +/** + * A {@link NameResolverProvider} that provides method to convert back to {@link NameResolver.Factory}. + */ +public abstract class NameResolverFactoryProvider extends NameResolverProvider { + + /** + * Convert the provider to a {@link NameResolver.Factory}. + * + * @return the name resolver factory. + */ + public abstract NameResolver.Factory toFactory(); + +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/NameResolverProviderFactory.java b/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/NameResolverProviderFactory.java new file mode 100644 index 00000000000..571ef6f7f98 --- /dev/null +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/NameResolverProviderFactory.java @@ -0,0 +1,64 @@ +/* + * 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.common.resolver; + +import static com.google.common.base.Preconditions.checkState; + +import io.grpc.Attributes; +import io.grpc.NameResolver; +import io.grpc.NameResolverProvider; +import java.net.URI; +import java.util.List; + +/** + * A {@link io.grpc.NameResolverProvider} based {@link NameResolver.Factory}. + */ +public class NameResolverProviderFactory extends NameResolver.Factory { + + private final List providers; + + public NameResolverProviderFactory(List providers) { + this.providers = providers; + } + + @Override + public NameResolver newNameResolver(URI targetUri, Attributes params) { + checkForProviders(); + for (NameResolverProvider provider : providers) { + NameResolver resolver = provider.newNameResolver(targetUri, params); + if (resolver != null) { + return resolver; + } + } + return null; + } + + @Override + public String getDefaultScheme() { + checkForProviders(); + return providers.get(0).getDefaultScheme(); + } + + private void checkForProviders() { + checkState(!providers.isEmpty(), + "No NameResolverProviders found. Please check your configuration"); + } + +} diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/ServiceNameResolverProvider.java b/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/ServiceNameResolverProvider.java index 9e11adf537b..05bf7b5b6d6 100644 --- a/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/ServiceNameResolverProvider.java +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/ServiceNameResolverProvider.java @@ -18,8 +18,6 @@ package org.apache.bookkeeper.common.resolver; -import static com.google.common.base.Preconditions.checkState; - import com.google.common.base.Function; import com.google.common.collect.Lists; import io.grpc.Attributes; @@ -41,7 +39,7 @@ * to resolve {@link org.apache.bookkeeper.common.net.ServiceURI}. */ @Slf4j -public final class ServiceNameResolverProvider extends NameResolverProvider { +public final class ServiceNameResolverProvider extends NameResolverFactoryProvider { private final DnsNameResolverProvider dnsProvider; private final Resource executorResource; @@ -127,38 +125,9 @@ public String getDefaultScheme() { return ServiceURI.SERVICE_BK; } + @Override public NameResolver.Factory toFactory() { - return new NameResolverFactory(Lists.newArrayList(this)); + return new NameResolverProviderFactory(Lists.newArrayList(this)); } - private static class NameResolverFactory extends NameResolver.Factory { - private final List providers; - - public NameResolverFactory(List providers) { - this.providers = providers; - } - - @Override - public NameResolver newNameResolver(URI targetUri, Attributes params) { - checkForProviders(); - for (NameResolverProvider provider : providers) { - NameResolver resolver = provider.newNameResolver(targetUri, params); - if (resolver != null) { - return resolver; - } - } - return null; - } - - @Override - public String getDefaultScheme() { - checkForProviders(); - return providers.get(0).getDefaultScheme(); - } - - private void checkForProviders() { - checkState(!providers.isEmpty(), - "No NameResolverProviders found. Please check your configuration"); - } - } } diff --git a/stream/pom.xml b/stream/pom.xml index f4866816fbe..e8a04916cb3 100644 --- a/stream/pom.xml +++ b/stream/pom.xml @@ -38,6 +38,7 @@ clients storage server + bk-grpc-name-resolver From 0cd12641f883eea15e34937b2a9ec24656aa9966 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Sat, 1 Dec 2018 01:29:53 +0100 Subject: [PATCH 0154/1642] LedgerMetadata should require digest and password together There's never a case when setting the digest and password where you only have one, but not the other. The patch adds validation in LedgerMetadata that if you provide one, you provide the other. The only case where you don't provide these is when using pre-4.1.0 (i think) metadata. Master issue: #281 Reviewers: Sijie Guo , Enrico Olivelli This closes #1836 from ivankelly/builder-pw --- .../apache/bookkeeper/client/LedgerMetadata.java | 15 +++++++++++---- .../bookkeeper/client/LedgerMetadataBuilder.java | 6 +++--- .../bookkeeper/client/LedgerRecovery2Test.java | 3 ++- .../client/ReadLastConfirmedAndEntryOpTest.java | 2 +- .../client/api/BookKeeperBuildersTest.java | 2 +- .../meta/LedgerManagerIteratorTest.java | 2 +- .../replication/AuditorLedgerCheckerTest.java | 2 +- .../metadata/etcd/EtcdLedgerManagerTest.java | 10 +++++----- 8 files changed, 25 insertions(+), 17 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 631761285a2..fa32f67b33e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -81,7 +81,7 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe Optional lastEntryId, Optional length, Map> ensembles, - DigestType digestType, + Optional digestType, Optional password, long ctime, boolean storeCtime, @@ -94,6 +94,9 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe checkArgument(!length.isPresent(), "Non-closed ledger must not have a length"); checkArgument(!lastEntryId.isPresent(), "Non-closed ledger must not have a last entry"); } + checkArgument((digestType.isPresent() && password.isPresent()) + || (!digestType.isPresent() && !password.isPresent()), + "Either both password and digest type must be set, or neither"); this.metadataFormatVersion = metadataFormatVersion; this.ensembleSize = ensembleSize; @@ -116,14 +119,14 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe currentEnsemble = null; } - this.digestType = digestType; - if (password.isPresent()) { this.password = password.get(); + this.digestType = digestType.get(); this.hasPassword = true; } else { this.password = null; this.hasPassword = false; + this.digestType = null; } this.ctime = ctime; this.storeCtime = storeCtime; @@ -177,7 +180,11 @@ public byte[] getPassword() { @Override public DigestType getDigestType() { - return digestType; + if (!hasPassword()) { + return null; + } else { + return digestType; + } } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index 88f50893008..c5a329cc33c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -55,7 +55,7 @@ public class LedgerMetadataBuilder { private TreeMap> ensembles = new TreeMap<>(); - private DigestType digestType = DigestType.CRC32C; + private Optional digestType = Optional.empty(); private Optional password = Optional.empty(); private long ctime = -1; @@ -81,9 +81,9 @@ public static LedgerMetadataBuilder from(LedgerMetadata other) { builder.ensembles.putAll(other.getAllEnsembles()); - builder.digestType = other.getDigestType(); if (other.hasPassword()) { builder.password = Optional.of(other.getPassword()); + builder.digestType = Optional.of(other.getDigestType()); } builder.ctime = other.getCtime(); @@ -105,7 +105,7 @@ public LedgerMetadataBuilder withPassword(byte[] password) { } public LedgerMetadataBuilder withDigestType(DigestType digestType) { - this.digestType = digestType; + this.digestType = Optional.of(digestType); return this; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java index 82c1a4b0bb4..d9c5d35e2d9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java @@ -25,6 +25,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; @@ -51,7 +52,7 @@ public class LedgerRecovery2Test { private static Versioned setupLedger(ClientContext clientCtx, long ledgerId, List bookies) throws Exception { LedgerMetadata md = LedgerMetadataBuilder.create() - .withPassword(PASSWD) + .withPassword(PASSWD).withDigestType(DigestType.CRC32C) .newEnsembleEntry(0, bookies).build(); return clientCtx.getLedgerManager().createLedgerMetadata(1L, md).get(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java index 252c1b9d589..527ce8d39c7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java @@ -105,8 +105,8 @@ public void setup() throws Exception { } this.ledgerMetadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(2) - .withDigestType(DigestType.CRC32.toApiDigestType()) .withPassword(new byte[0]) + .withDigestType(DigestType.CRC32.toApiDigestType()) .newEnsembleEntry(0L, ensemble).build(); this.distributionSchedule = new RoundRobinDistributionSchedule(3, 2, 3); // schedulers diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java index f593642778c..0316b7f2694 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java @@ -410,8 +410,8 @@ protected LedgerMetadata generateLedgerMetadata(int ensembleSize, .withEnsembleSize(ensembleSize) .withWriteQuorumSize(writeQuorumSize) .withAckQuorumSize(ackQuorumSize) - .withDigestType(BookKeeper.DigestType.CRC32.toApiDigestType()) .withPassword(password) + .withDigestType(BookKeeper.DigestType.CRC32.toApiDigestType()) .withCustomMetadata(customMetadata) .withCreationTime(System.currentTimeMillis()) .newEnsembleEntry(0, generateNewEnsemble(ensembleSize)).build(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java index 73f8d39e10c..03e13585e85 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java @@ -93,8 +93,8 @@ void createLedger(LedgerManager lm, Long ledgerId) throws Exception { new BookieSocketAddress("192.0.2.3", 1234)); LedgerMetadata meta = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) - .withDigestType(BookKeeper.DigestType.CRC32.toApiDigestType()) .withPassword("passwd".getBytes()) + .withDigestType(BookKeeper.DigestType.CRC32.toApiDigestType()) .newEnsembleEntry(0L, ensemble) .build(); lm.createLedgerMetadata(ledgerId, meta).get(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index 9db91887a63..ef57d6bddbf 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -553,8 +553,8 @@ public void testTriggerAuditorWithNoPendingAuditTask() throws Exception { LedgerMetadata metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(2) - .withDigestType(DigestType.CRC32.toApiDigestType()) .withPassword("passwd".getBytes()) + .withDigestType(DigestType.CRC32.toApiDigestType()) .newEnsembleEntry(0L, ensemble).build(); long ledgerId = (Math.abs(rand.nextLong())) % 100000000; diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index 95da4ffe774..a7df94f14ce 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -241,8 +241,8 @@ private void createNumLedgers(int numLedgers) throws Exception { for (int i = 0; i < numLedgers; i++) { LedgerMetadata metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) - .withDigestType(DigestType.CRC32C.toApiDigestType()) .withPassword("test-password".getBytes(UTF_8)) + .withDigestType(DigestType.CRC32C.toApiDigestType()) .newEnsembleEntry(0L, createNumBookies(3)).build(); createFutures.add(lm.createLedgerMetadata(i, metadata)); } @@ -255,10 +255,10 @@ public void testRegisterLedgerMetadataListener() throws Exception { // create a ledger metadata LedgerMetadata metadata = LedgerMetadataBuilder.create() - .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) - .withDigestType(DigestType.CRC32C.toApiDigestType()) - .withPassword("test-password".getBytes(UTF_8)) - .newEnsembleEntry(0L, createNumBookies(3)).build(); + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) + .withPassword("test-password".getBytes(UTF_8)) + .withDigestType(DigestType.CRC32C.toApiDigestType()) + .newEnsembleEntry(0L, createNumBookies(3)).build(); result(lm.createLedgerMetadata(ledgerId, metadata)); Versioned readMetadata = lm.readLedgerMetadata(ledgerId).get(); log.info("Create ledger metadata : {}", readMetadata.getValue()); From e76803c67204c646b8cbf6eb701a20e4a5806cfc Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Mon, 3 Dec 2018 13:37:03 +0100 Subject: [PATCH 0155/1642] Users of LedgerMetadata should use the api interface All users to LedgerMetadata should use api.LedgerMetadata rather than client.LedgerMetadata. Some methods have been promoted to the interface to allow this. Other methods have been moved out to a utility class that acts purely on api.LedgerMetadata. client.LedgerMetadata has been renamed to LedgerMetadataImpl. Master issue: #723 Reviewers: Enrico Olivelli , Sijie Guo This closes #1852 from ivankelly/kill-store-ctime --- .../apache/bookkeeper/bookie/BookieShell.java | 2 +- .../apache/bookkeeper/client/BookKeeper.java | 3 +- .../bookkeeper/client/BookKeeperAdmin.java | 6 +- .../bookkeeper/client/EnsembleUtils.java | 1 + .../bookkeeper/client/LedgerCreateOp.java | 3 +- .../bookkeeper/client/LedgerFragment.java | 2 +- .../bookkeeper/client/LedgerHandle.java | 14 +-- .../bookkeeper/client/LedgerHandleAdv.java | 1 + .../client/LedgerMetadataBuilder.java | 15 +-- ...rMetadata.java => LedgerMetadataImpl.java} | 95 +++++-------------- .../client/LedgerMetadataUtils.java | 79 +++++++++++++++ .../bookkeeper/client/LedgerOpenOp.java | 1 + .../bookkeeper/client/LedgerRecoveryOp.java | 1 + .../bookkeeper/client/MetadataUpdateLoop.java | 1 + .../bookkeeper/client/PendingReadOp.java | 5 +- .../client/ReadLastConfirmedAndEntryOp.java | 1 + .../client/ReadOnlyLedgerHandle.java | 9 +- .../bookkeeper/client/UpdateLedgerOp.java | 1 + .../bookkeeper/client/api/LedgerMetadata.java | 31 ++++++ .../conf/AbstractConfiguration.java | 1 + .../bookkeeper/conf/ClientConfiguration.java | 3 - .../meta/AbstractZkLedgerManager.java | 2 +- .../bookkeeper/meta/CleanupLedgerManager.java | 2 +- .../apache/bookkeeper/meta/LedgerManager.java | 2 +- .../bookkeeper/meta/LedgerMetadataSerDe.java | 6 +- .../meta/MSLedgerManagerFactory.java | 2 +- .../proto/BookkeeperInternalCallbacks.java | 2 +- .../replication/ReplicationWorker.java | 2 +- .../http/service/GetLedgerMetaService.java | 2 +- .../http/service/ListLedgerService.java | 2 +- .../bookkeeper/bookie/BookieShellTest.java | 2 +- .../bookkeeper/bookie/CompactionTest.java | 2 +- .../bookie/TestGcOverreplicatedLedger.java | 2 +- ...rDiskSpaceWeightedLedgerPlacementTest.java | 18 ++-- .../bookkeeper/client/BookieRecoveryTest.java | 5 +- .../apache/bookkeeper/client/ClientUtil.java | 1 + .../GenericEnsemblePlacementPolicyTest.java | 2 +- .../bookkeeper/client/HandleFailuresTest.java | 1 + .../bookkeeper/client/LedgerClose2Test.java | 1 + .../client/LedgerHandleAdapter.java | 7 -- .../bookkeeper/client/LedgerMetadataTest.java | 1 + .../client/LedgerRecovery2Test.java | 1 + .../client/MetadataUpdateLoopTest.java | 57 +++++------ .../client/MockBookKeeperTestCase.java | 1 + .../bookkeeper/client/MockLedgerHandle.java | 1 + .../client/ParallelLedgerRecoveryTest.java | 1 + .../ReadLastConfirmedAndEntryOpTest.java | 1 + .../client/TestBookieHealthCheck.java | 14 +-- .../client/TestDelayEnsembleChange.java | 9 +- .../apache/bookkeeper/client/TestFencing.java | 4 +- .../client/TestGetBookieInfoTimeout.java | 2 +- .../client/TestLedgerFragmentReplication.java | 1 + .../bookkeeper/client/TestParallelRead.java | 6 +- .../client/TestReadEntryListener.java | 4 +- .../client/TestReadLastConfirmedLongPoll.java | 2 +- .../bookkeeper/client/TestReadTimeout.java | 6 +- .../client/TestWatchEnsembleChange.java | 1 + .../client/UpdateLedgerCmdTest.java | 2 +- .../bookkeeper/client/UpdateLedgerOpTest.java | 17 ++-- .../client/api/BookKeeperBuildersTest.java | 1 - .../meta/AbstractZkLedgerManagerTest.java | 2 +- .../apache/bookkeeper/meta/GcLedgersTest.java | 2 +- .../meta/LedgerManagerIteratorTest.java | 2 +- .../bookkeeper/meta/MockLedgerManager.java | 2 +- .../replication/AuditorLedgerCheckerTest.java | 2 +- .../replication/AuditorPeriodicCheckTest.java | 6 +- .../replication/BookieAutoRecoveryTest.java | 32 +++---- ...estAutoRecoveryAlongWithBookieServers.java | 4 +- .../org/apache/bookkeeper/tls/TestTLS.java | 2 +- .../metadata/etcd/EtcdLedgerManager.java | 2 +- .../metadata/etcd/LedgerMetadataConsumer.java | 2 +- .../metadata/etcd/EtcdLedgerManagerTest.java | 2 +- .../bookkeeper/client/BookKeeperAccessor.java | 4 - .../bookkeeper/client/LedgerReader.java | 4 +- .../distributedlog/TestAsyncReaderWriter.java | 7 +- .../TestCompatRecoveryNoPassword.groovy | 2 +- 76 files changed, 308 insertions(+), 239 deletions(-) rename bookkeeper-server/src/main/java/org/apache/bookkeeper/client/{LedgerMetadata.java => LedgerMetadataImpl.java} (76%) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java 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 c58fa834415..9108ef36d24 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 @@ -90,8 +90,8 @@ import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.client.UpdateLedgerOp; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; 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 a3c40079723..b8f703dc23a 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 @@ -1041,8 +1041,7 @@ public LedgerHandle createLedgerAdv(final long ledgerId, throw BKException.create(BKException.Code.UnexpectedConditionException); } - LOG.info("Ensemble: {} for ledger: {}", lh.getLedgerMetadata().getEnsemble(0L), - lh.getId()); + LOG.info("Ensemble: {} for ledger: {}", lh.getLedgerMetadata().getEnsembleAt(0L), lh.getId()); return lh; } 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 3f7cee45e80..3f837dbaf70 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 @@ -60,6 +60,7 @@ import org.apache.bookkeeper.client.LedgerFragmentReplicator.SingleFragmentCallback; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncOpenCallback; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncReadCallback; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; @@ -500,7 +501,8 @@ public void process(final Long lid, final AsyncCallback.VoidCallback cb) { cb.processResult(BKException.getExceptionCode(exception), null, null); return; } - Set bookiesInLedger = metadata.getValue().getBookiesInThisLedger(); + Set bookiesInLedger = + LedgerMetadataUtils.getBookiesInThisLedger(metadata.getValue()); Sets.SetView intersection = Sets.intersection(bookiesInLedger, bookies); if (!intersection.isEmpty()) { @@ -739,7 +741,7 @@ public void openComplete(int rc, final LedgerHandle lh, Object ctx) { } LedgerMetadata lm = lh.getLedgerMetadata(); - if (skipOpenLedgers && !lm.isClosed() && !lm.isInRecovery()) { + if (skipOpenLedgers && lm.getState() == LedgerMetadata.State.OPEN) { LOG.info("Skip recovering open ledger {}.", lId); try { lh.close(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java index e4ab118f7dd..6995a069e17 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.net.BookieSocketAddress; import org.slf4j.Logger; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java index d95665c035e..eaa316360c0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java @@ -38,6 +38,7 @@ import org.apache.bookkeeper.client.SyncCallbackUtils.SyncCreateCallback; import org.apache.bookkeeper.client.api.CreateAdvBuilder; import org.apache.bookkeeper.client.api.CreateBuilder; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteAdvHandle; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.api.WriteHandle; @@ -216,7 +217,7 @@ private void metadataCallback(Versioned writtenMetadata, Throwab return; } - List curEns = lh.getLedgerMetadata().getEnsemble(0L); + List curEns = lh.getLedgerMetadata().getEnsembleAt(0L); LOG.info("Ensemble: {} for ledger: {}", curEns, lh.getId()); for (BookieSocketAddress bsa : curEns) { 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 e3224fdea99..382fe4e6b7c 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 @@ -48,7 +48,7 @@ public class LedgerFragment { this.firstEntryId = firstEntryId; this.lastKnownEntryId = lastKnownEntryId; this.bookieIndexes = bookieIndexes; - this.ensemble = lh.getLedgerMetadata().getEnsemble(firstEntryId); + this.ensemble = lh.getLedgerMetadata().getEnsembleAt(firstEntryId); this.schedule = lh.getDistributionSchedule(); SortedMap> ensembles = lh .getLedgerMetadata().getAllEnsembles(); 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 9af32713080..075788bed3e 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 @@ -67,6 +67,7 @@ import org.apache.bookkeeper.client.api.BKException.Code; import org.apache.bookkeeper.client.api.LastConfirmedAndEntry; import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.api.WriteHandle; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; @@ -1688,7 +1689,7 @@ public long readExplicitLastConfirmed() throws InterruptedException, BKException // close the ledger and send fails to all the adds in the pipeline void handleUnrecoverableErrorDuringAdd(int rc) { - if (getLedgerMetadata().isInRecovery()) { + if (getLedgerMetadata().getState() == LedgerMetadata.State.IN_RECOVERY) { // we should not close ledger if ledger is recovery mode // otherwise we may lose entry. errorOutPendingAdds(rc); @@ -1854,16 +1855,17 @@ void ensembleChangeLoop(List origEnsemble, Map !metadata.isClosed() && !metadata.isInRecovery() + (metadata) -> metadata.getState() == LedgerMetadata.State.OPEN && failedBookies.entrySet().stream().anyMatch( - (e) -> metadata.getLastEnsembleValue().get(e.getKey()).equals(e.getValue())), + e -> LedgerMetadataUtils.getLastEnsembleValue(metadata) + .get(e.getKey()).equals(e.getValue())), (metadata) -> { attempts.incrementAndGet(); List currentEnsemble = getCurrentEnsemble(); List newEnsemble = EnsembleUtils.replaceBookiesInEnsemble( clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext); - Long lastEnsembleKey = metadata.getLastEnsembleKey(); + Long lastEnsembleKey = LedgerMetadataUtils.getLastEnsembleKey(metadata); LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata); long newEnsembleStartEntry = getLastAddConfirmed() + 1; checkState(lastEnsembleKey <= newEnsembleStartEntry, @@ -1890,7 +1892,7 @@ void ensembleChangeLoop(List origEnsemble, Map getCurrentEnsemble() { // Getting current ensemble from the metadata is only a temporary // thing until metadata is immutable. At that point, current ensemble // becomes a property of the LedgerHandle itself. - return versionedMetadata.getValue().getCurrentEnsemble(); + return LedgerMetadataUtils.getCurrentEnsemble(versionedMetadata.getValue()); } } 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 6f38b8e678d..14317d0514a 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 @@ -34,6 +34,7 @@ import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.AsyncCallback.AddCallbackWithLatency; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncAddCallback; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteAdvHandle; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.util.SafeRunnable; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index c5a329cc33c..0c80315f4da 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -32,6 +32,7 @@ import java.util.TreeMap; import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.LedgerMetadata.State; import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable; @@ -87,7 +88,9 @@ public static LedgerMetadataBuilder from(LedgerMetadata other) { } builder.ctime = other.getCtime(); - builder.storeCtime = other.storeCtime; + + /** Hack to get around fact that ctime was never versioned correctly */ + builder.storeCtime = LedgerMetadataUtils.shouldStoreCtime(other); builder.customMetadata = ImmutableMap.copyOf(other.getCustomMetadata()); @@ -182,11 +185,11 @@ public LedgerMetadata build() { checkArgument(ensembleSize >= writeQuorumSize, "Write quorum must be less or equal to ensemble size"); checkArgument(writeQuorumSize >= ackQuorumSize, "Write quorum must be greater or equal to ack quorum"); - return new LedgerMetadata(metadataFormatVersion, - ensembleSize, writeQuorumSize, ackQuorumSize, - state, lastEntryId, length, ensembles, - digestType, password, ctime, storeCtime, - customMetadata); + return new LedgerMetadataImpl(metadataFormatVersion, + ensembleSize, writeQuorumSize, ackQuorumSize, + state, lastEntryId, length, ensembles, + digestType, password, ctime, storeCtime, + customMetadata); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java similarity index 76% rename from bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java rename to bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java index fa32f67b33e..c0fcadd07ed 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java @@ -18,26 +18,22 @@ package org.apache.bookkeeper.client; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.util.Arrays; import java.util.Base64; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.Optional; -import java.util.Set; -import java.util.SortedMap; import java.util.TreeMap; import java.util.stream.Collectors; import lombok.EqualsAndHashCode; import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.LedgerMetadata.State; import org.apache.bookkeeper.net.BookieSocketAddress; import org.slf4j.Logger; @@ -50,8 +46,8 @@ *

            It provides parsing and serialization methods of such metadata. */ @EqualsAndHashCode -public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMetadata { - static final Logger LOG = LoggerFactory.getLogger(LedgerMetadata.class); +class LedgerMetadataImpl implements LedgerMetadata { + static final Logger LOG = LoggerFactory.getLogger(LedgerMetadataImpl.class); private final int metadataFormatVersion; private final int ensembleSize; @@ -73,19 +69,19 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe private final Map customMetadata; - LedgerMetadata(int metadataFormatVersion, - int ensembleSize, - int writeQuorumSize, - int ackQuorumSize, - State state, - Optional lastEntryId, - Optional length, - Map> ensembles, - Optional digestType, - Optional password, - long ctime, - boolean storeCtime, - Map customMetadata) { + LedgerMetadataImpl(int metadataFormatVersion, + int ensembleSize, + int writeQuorumSize, + int ackQuorumSize, + State state, + Optional lastEntryId, + Optional length, + Map> ensembles, + Optional digestType, + Optional password, + long ctime, + boolean storeCtime, + Map customMetadata) { checkArgument(ensembles.size() > 0, "There must be at least one ensemble in the ledger"); if (state == State.CLOSED) { checkArgument(length.isPresent(), "Closed ledger must have a length"); @@ -165,11 +161,12 @@ public long getCtime() { * * @return whether the password has been stored in the metadata */ + @Override public boolean hasPassword() { return hasPassword; } - @VisibleForTesting + @Override public byte[] getPassword() { if (!hasPassword()) { return new byte[0]; @@ -202,47 +199,18 @@ public boolean isClosed() { return state == State.CLOSED; } - public boolean isInRecovery() { - return state == State.IN_RECOVERY; - } - @Override public State getState() { return state; } - List getCurrentEnsemble() { - return currentEnsemble; - } - - List getEnsemble(long entryId) { + @Override + public List getEnsembleAt(long entryId) { // the head map cannot be empty, since we insert an ensemble for // entry-id 0, right when we start return ensembles.get(ensembles.headMap(entryId + 1).lastKey()); } - @Override - public List getEnsembleAt(long entryId) { - return getEnsemble(entryId); - } - - /** - * the entry id greater than the given entry-id at which the next ensemble change takes - * place. - * - * @param entryId - * @return the entry id of the next ensemble change (-1 if no further ensemble changes) - */ - long getNextEnsembleChange(long entryId) { - SortedMap> tailMap = ensembles.tailMap(entryId + 1); - - if (tailMap.isEmpty()) { - return -1; - } else { - return tailMap.firstKey(); - } - } - @Override public Map getCustomMetadata() { return this.customMetadata; @@ -260,6 +228,7 @@ public String toString() { * @return a string representation of the object without password field in * it. */ + @Override public String toSafeString() { return toStringRepresentation(false); } @@ -291,30 +260,12 @@ private String toStringRepresentation(boolean withPassword) { return helper.toString(); } - Set getBookiesInThisLedger() { - Set bookies = new HashSet(); - for (List ensemble : ensembles.values()) { - bookies.addAll(ensemble); - } - return bookies; - } - - List getLastEnsembleValue() { - checkState(!ensembles.isEmpty(), "Metadata should never be created with no ensembles"); - return ensembles.lastEntry().getValue(); - } - - Long getLastEnsembleKey() { - checkState(!ensembles.isEmpty(), "Metadata should never be created with no ensembles"); - return ensembles.lastKey(); - } - + @Override public int getMetadataFormatVersion() { return metadataFormatVersion; } - // temporarily method, until storeCtime is removed from the metadata object itself - public boolean shouldStoreCtime() { + boolean shouldStoreCtime() { return storeCtime; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java new file mode 100644 index 00000000000..07d73409613 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java @@ -0,0 +1,79 @@ +/** + * 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.client; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.SortedMap; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utilities for working with ledger metadata. + */ +public class LedgerMetadataUtils { + static final Logger LOG = LoggerFactory.getLogger(LedgerMetadataUtils.class); + + static List getCurrentEnsemble(LedgerMetadata metadata) { + return getLastEnsembleValue(metadata); + } + + /** + * the entry id greater than the given entry-id at which the next ensemble change takes + * place. + * + * @param entryId + * @return the entry id of the next ensemble change (-1 if no further ensemble changes) + */ + static long getNextEnsembleChange(LedgerMetadata metadata, long entryId) { + SortedMap> tailMap = metadata.getAllEnsembles().tailMap(entryId + 1); + + if (tailMap.isEmpty()) { + return -1; + } else { + return tailMap.firstKey(); + } + } + + static Set getBookiesInThisLedger(LedgerMetadata metadata) { + Set bookies = new HashSet(); + for (List ensemble : metadata.getAllEnsembles().values()) { + bookies.addAll(ensemble); + } + return bookies; + } + + static List getLastEnsembleValue(LedgerMetadata metadata) { + checkArgument(!metadata.getAllEnsembles().isEmpty(), "Metadata should never be created with no ensembles"); + return metadata.getAllEnsembles().lastEntry().getValue(); + } + + static Long getLastEnsembleKey(LedgerMetadata metadata) { + checkArgument(!metadata.getAllEnsembles().isEmpty(), "Metadata should never be created with no ensembles"); + return metadata.getAllEnsembles().lastKey(); + } + + public static boolean shouldStoreCtime(LedgerMetadata metadata) { + return metadata instanceof LedgerMetadataImpl && ((LedgerMetadataImpl) metadata).shouldStoreCtime(); + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java index c2dc33a6c63..2193b3ef9fb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncOpenCallback; import org.apache.bookkeeper.client.api.BKException.Code; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.client.impl.OpenBuilderBase; import org.apache.bookkeeper.stats.OpStatsLogger; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java index bc2985a9577..2dd6ea27990 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java @@ -22,6 +22,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener; import org.apache.bookkeeper.proto.checksum.DigestManager.RecoveryData; import org.slf4j.Logger; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java index 435ff69f62d..5c8cb14e652 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.function.Supplier; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; 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 a66d889a895..aa89eaa7f9e 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 @@ -35,6 +35,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.bookkeeper.client.BKException.BKDigestMatchException; import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.impl.LedgerEntriesImpl; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; import org.apache.bookkeeper.common.util.SafeRunnable; @@ -506,8 +507,8 @@ void initiate() { List ensemble = null; do { if (i == nextEnsembleChange) { - ensemble = getLedgerMetadata().getEnsemble(i); - nextEnsembleChange = getLedgerMetadata().getNextEnsembleChange(i); + ensemble = getLedgerMetadata().getEnsembleAt(i); + nextEnsembleChange = LedgerMetadataUtils.getNextEnsembleChange(getLedgerMetadata(), i); } LedgerEntryRequest entry; if (parallelRead) { 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 cb9de32111f..ed1bece65f2 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 @@ -28,6 +28,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java index 59ddd5651b6..72930f7acee 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java @@ -37,6 +37,7 @@ import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; import org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; @@ -266,11 +267,11 @@ void recover(GenericCallback finalCb, clientCtx.getClientStats().getRecoverOpLogger()); MetadataUpdateLoop.NeedsUpdatePredicate needsUpdate = - (metadata) -> !(metadata.isClosed() || metadata.isInRecovery()); + (metadata) -> metadata.getState() == LedgerMetadata.State.OPEN; if (forceRecovery) { // in the force recovery case, we want to update the metadata // to IN_RECOVERY, even if the ledger is already closed - needsUpdate = (metadata) -> !metadata.isInRecovery(); + needsUpdate = (metadata) -> metadata.getState() != LedgerMetadata.State.IN_RECOVERY; } new MetadataUpdateLoop( clientCtx.getLedgerManager(), getId(), @@ -309,10 +310,10 @@ CompletableFuture> closeRecovered() { CompletableFuture> f = new MetadataUpdateLoop( clientCtx.getLedgerManager(), getId(), this::getVersionedLedgerMetadata, - (metadata) -> metadata.isInRecovery(), + (metadata) -> metadata.getState() == LedgerMetadata.State.IN_RECOVERY, (metadata) -> { LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata); - Long lastEnsembleKey = metadata.getLastEnsembleKey(); + Long lastEnsembleKey = LedgerMetadataUtils.getLastEnsembleKey(metadata); synchronized (metadataLock) { newEnsemblesFromRecovery.entrySet().forEach( (e) -> { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java index 98c544e9a26..6fce7e1fc38 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java @@ -35,6 +35,7 @@ import java.util.stream.Collectors; import org.apache.bookkeeper.bookie.BookieShell.UpdateLedgerNotifier; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.versioning.Versioned; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java index de86832a88f..2ce19404d09 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java @@ -74,8 +74,25 @@ public interface LedgerMetadata { */ long getLength(); + /** + * Whether the metadata contains the password and digest type for the ledger. + * Ledgers created with version 4.1.0 clients or older do not have this information. + * + * @return true if the metadata contains the password and digest type, false otherwise. + */ + boolean hasPassword(); + + /** + * Get the password for the ledger. + * For ledgers created with version 4.1.0 or older, an empty byte array is returned. + * + * @return the password for the ledger. + */ + byte[] getPassword(); + /** * Returns the digest type used by this ledger. + * May return null if the ledger was created with version 4.1.0 or below. * * @return the digest type used by this ledger. */ @@ -143,4 +160,18 @@ enum State { */ CLOSED; } + + /** + * Similar to #toString(), but omits the password of the ledger, so that it is safe to log the output. + * + * @return a string representation of the metadata, omitting the password. + */ + String toSafeString(); + + /** + * Get the format version which should be used to serialize the metadata. + * + * @return the format version. + */ + int getMetadataFormatVersion(); } 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 e395cb38253..7f00d09adb5 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 @@ -84,6 +84,7 @@ public abstract class AbstractConfiguration protected static final String REREPLICATION_ENTRY_BATCH_SIZE = "rereplicationEntryBatchSize"; protected static final String STORE_SYSTEMTIME_AS_LEDGER_UNDERREPLICATED_MARK_TIME = "storeSystemTimeAsLedgerUnderreplicatedMarkTime"; + protected static final String STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME = "storeSystemTimeAsLedgerCreationTime"; // Metastore settings, only being used when LEDGER_MANAGER_FACTORY_CLASS is MSLedgerManagerFactory protected static final String METASTORE_IMPL_CLASS = "metastoreImplClass"; 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 86ec44457b6..7d0d319882b 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 @@ -154,9 +154,6 @@ public class ClientConfiguration extends AbstractConfiguration ensembleAtFirstEntry = lh.getLedgerMetadata().getEnsemble(lId); + List ensembleAtFirstEntry = lh.getLedgerMetadata().getEnsembleAt(lId); assertEquals(2, ensembleAtFirstEntry.size()); killBookie(ensembleAtFirstEntry.get(0)); lh.addEntry(value); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java index 2f75e12a839..9f95a94fbfe 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java @@ -33,6 +33,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.net.BookieSocketAddress; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java index 541af0a54dc..45b27135935 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java @@ -20,6 +20,7 @@ import com.google.common.collect.Lists; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.net.BookieSocketAddress; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java index 7098f4ee475..9450e512f16 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java @@ -29,13 +29,6 @@ */ public class LedgerHandleAdapter { - /** - * Get the ledger handle. - */ - public static LedgerMetadata getLedgerMetadata(LedgerHandle lh) { - return lh.getLedgerMetadata(); - } - public static ByteBufList toSend(LedgerHandle lh, long entryId, ByteBuf data) { return lh.getDigestManager().computeDigestAndPackageForSending(entryId, lh.getLastAddConfirmed(), lh.addToLength(data.readableBytes()), data); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java index 80e1c759806..a9ffc2d4221 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java @@ -29,6 +29,7 @@ import java.util.Collections; import java.util.List; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java index d9c5d35e2d9..2e17836bd54 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java @@ -26,6 +26,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java index 2544a20ef0e..ffacb21f05d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java @@ -42,6 +42,7 @@ import lombok.AllArgsConstructor; import lombok.Data; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.MockLedgerManager; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -86,7 +87,7 @@ public void testBasicUpdate() throws Exception { reference::get, (currentMetadata) -> true, (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, newAddress); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -94,7 +95,7 @@ public void testBasicUpdate() throws Exception { loop.run().get(); Assert.assertNotEquals(reference.get(), writtenMetadata); - Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), newAddress); + Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(0), newAddress); } } @@ -123,9 +124,9 @@ public void testConflictOnWrite() throws Exception { lm, ledgerId, reference1::get, - (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b0), + (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b2); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -136,9 +137,9 @@ public void testConflictOnWrite() throws Exception { lm, ledgerId, reference2::get, - (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b1), + (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b1), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(1, b3); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -154,11 +155,11 @@ public void testConflictOnWrite() throws Exception { Assert.assertEquals(l1meta.getVersion().compare(l2meta.getVersion()), Version.Occurred.BEFORE); - Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(0), b2); - Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(1), b1); + Assert.assertEquals(l1meta.getValue().getEnsembleAt(0L).get(0), b2); + Assert.assertEquals(l1meta.getValue().getEnsembleAt(0L).get(1), b1); - Assert.assertEquals(l2meta.getValue().getEnsemble(0L).get(0), b2); - Assert.assertEquals(l2meta.getValue().getEnsemble(0L).get(1), b3); + Assert.assertEquals(l2meta.getValue().getEnsembleAt(0L).get(0), b2); + Assert.assertEquals(l2meta.getValue().getEnsembleAt(0L).get(1), b3); verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any()); } @@ -188,9 +189,9 @@ public void testConflictOnWriteBothWritingSame() throws Exception { lm, ledgerId, reference::get, - (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b0), + (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b2); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -199,9 +200,9 @@ public void testConflictOnWriteBothWritingSame() throws Exception { lm, ledgerId, reference::get, - (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b0), + (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b2); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -212,8 +213,8 @@ public void testConflictOnWriteBothWritingSame() throws Exception { Assert.assertEquals(loop1.get(), loop2.get()); Assert.assertEquals(loop1.get(), reference.get()); - Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), b2); - Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(1), b1); + Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(0), b2); + Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(1), b1); verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any()); } @@ -241,9 +242,9 @@ public void testConflictOnLocalUpdate() throws Exception { lm, ledgerId, reference::get, - (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b0), + (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b2); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -254,9 +255,9 @@ public void testConflictOnLocalUpdate() throws Exception { lm, ledgerId, reference::get, - (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b1), + (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b1), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(1, b3); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -267,8 +268,8 @@ public void testConflictOnLocalUpdate() throws Exception { Assert.assertEquals(loop1.get(), reference.get()); - Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), b2); - Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(1), b3); + Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(0), b2); + Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(1), b3); verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any()); } @@ -312,9 +313,9 @@ public void testHammer() throws Exception { lm, ledgerId, reference::get, - (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(initialEnsemble.get(i)), + (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(initialEnsemble.get(i)), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(i, replacementBookies.get(i)); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -323,7 +324,7 @@ public void testHammer() throws Exception { loops.forEach((l) -> l.join()); - Assert.assertEquals(reference.get().getValue().getEnsemble(0L), replacementBookies); + Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L), replacementBookies); } } @@ -364,11 +365,11 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { if (currentMetadata.isClosed()) { throw new BKException.BKLedgerClosedException(); } else { - return currentMetadata.getEnsemble(0L).contains(b0); + return currentMetadata.getEnsembleAt(0L).contains(b0); } }, (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b1); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -383,7 +384,7 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerClosedException.class); } Assert.assertEquals(l1meta, reference.get()); - Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(0), b0); + Assert.assertEquals(l1meta.getValue().getEnsembleAt(0L).get(0), b0); Assert.assertTrue(l1meta.getValue().isClosed()); verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java index a88f3561088..6d6e4d15a26 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java @@ -53,6 +53,7 @@ import org.apache.bookkeeper.client.BKException.Code; import org.apache.bookkeeper.client.api.CreateBuilder; import org.apache.bookkeeper.client.api.DeleteBuilder; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.OpenBuilder; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.common.util.OrderedScheduler; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java index b7cb9e14a34..3aaeedb56ac 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java @@ -39,6 +39,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.api.LastConfirmedAndEntry; import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index 9571d1e7fb4..ab966fb4ae2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -43,6 +43,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java index 527ce8d39c7..a1032a16e0d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java @@ -47,6 +47,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.LastConfirmedAndEntryCallback; import org.apache.bookkeeper.client.api.LastConfirmedAndEntry; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.impl.LastConfirmedAndEntryImpl; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; import org.apache.bookkeeper.common.concurrent.FutureUtils; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java index ad35450b4ad..cbee711624a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java @@ -55,7 +55,7 @@ public void testBkQuarantine() throws Exception { lh.addEntry(msg); } - BookieSocketAddress bookieToQuarantine = lh.getLedgerMetadata().getEnsemble(numEntries).get(0); + BookieSocketAddress bookieToQuarantine = lh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); sleepBookie(bookieToQuarantine, baseClientConf.getAddEntryTimeout() * 2).await(); byte[] tempMsg = "temp-msg".getBytes(); @@ -79,12 +79,12 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { // the bookie to be left out of the ensemble should always be the quarantined bookie LedgerHandle lh1 = bkc.createLedger(2, 2, 2, BookKeeper.DigestType.CRC32, new byte[] {}); LedgerHandle lh2 = bkc.createLedger(3, 3, 3, BookKeeper.DigestType.CRC32, new byte[] {}); - Assert.assertFalse(lh1.getLedgerMetadata().getEnsemble(0).contains(bookieToQuarantine)); - Assert.assertFalse(lh2.getLedgerMetadata().getEnsemble(0).contains(bookieToQuarantine)); + Assert.assertFalse(lh1.getLedgerMetadata().getEnsembleAt(0).contains(bookieToQuarantine)); + Assert.assertFalse(lh2.getLedgerMetadata().getEnsembleAt(0).contains(bookieToQuarantine)); // the quarantined bookie can still be in the ensemble if we do not have enough healthy bookies LedgerHandle lh3 = bkc.createLedger(4, 4, 4, BookKeeper.DigestType.CRC32, new byte[] {}); - Assert.assertTrue(lh3.getLedgerMetadata().getEnsemble(0).contains(bookieToQuarantine)); + Assert.assertTrue(lh3.getLedgerMetadata().getEnsembleAt(0).contains(bookieToQuarantine)); // make sure faulty bookie is out of quarantine Thread.sleep(baseClientConf.getBookieQuarantineTimeSeconds() * 1000); @@ -97,7 +97,7 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { public void testNoQuarantineOnBkRestart() throws Exception { final LedgerHandle lh = bkc.createLedger(2, 2, 2, BookKeeper.DigestType.CRC32, new byte[] {}); final int numEntries = 20; - BookieSocketAddress bookieToRestart = lh.getLedgerMetadata().getEnsemble(0).get(0); + BookieSocketAddress bookieToRestart = lh.getLedgerMetadata().getEnsembleAt(0).get(0); // we add entries on a separate thread so that we can restart a bookie on the current thread Thread addEntryThread = new Thread() { @@ -132,8 +132,8 @@ public void testNoQuarantineOnExpectedBkErrors() throws Exception { byte[] msg = ("msg-" + i).getBytes(); lh.addEntry(msg); } - BookieSocketAddress bookie1 = lh.getLedgerMetadata().getEnsemble(0).get(0); - BookieSocketAddress bookie2 = lh.getLedgerMetadata().getEnsemble(0).get(1); + BookieSocketAddress bookie1 = lh.getLedgerMetadata().getEnsembleAt(0).get(0); + BookieSocketAddress bookie2 = lh.getLedgerMetadata().getEnsembleAt(0).get(1); try { // we read an entry that is not added lh.readEntries(10, 10); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java index c76a75c3cfd..f8c5be5876a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java @@ -37,6 +37,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; @@ -103,7 +104,7 @@ private void verifyEntries(LedgerHandle lh, long startEntry, long untilEntry, LedgerMetadata md = lh.getLedgerMetadata(); for (long eid = startEntry; eid < untilEntry; eid++) { - List addresses = md.getEnsemble(eid); + List addresses = md.getEnsembleAt(eid); VerificationCallback callback = new VerificationCallback(addresses.size()); for (BookieSocketAddress addr : addresses) { bkc.getBookieClient().readEntry(addr, lh.getId(), eid, @@ -121,7 +122,7 @@ private void verifyEntriesRange(LedgerHandle lh, long startEntry, long untilEntr LedgerMetadata md = lh.getLedgerMetadata(); for (long eid = startEntry; eid < untilEntry; eid++) { - List addresses = md.getEnsemble(eid); + List addresses = md.getEnsembleAt(eid); VerificationCallback callback = new VerificationCallback(addresses.size()); for (BookieSocketAddress addr : addresses) { bkc.getBookieClient().readEntry(addr, lh.getId(), eid, @@ -257,8 +258,8 @@ public void testChangeEnsembleIfBrokenAckQuorum() throws Exception { CLIENT_SCOPE + "." + WATCHER_SCOPE + "." + REPLACE_BOOKIE_TIME) .getSuccessCount() > 0); - List firstFragment = lh.getLedgerMetadata().getEnsemble(0); - List secondFragment = lh.getLedgerMetadata().getEnsemble(3 * numEntries); + List firstFragment = lh.getLedgerMetadata().getEnsembleAt(0); + List secondFragment = lh.getLedgerMetadata().getEnsembleAt(3 * numEntries); assertFalse(firstFragment.get(0).equals(secondFragment.get(0))); assertFalse(firstFragment.get(1).equals(secondFragment.get(1))); assertFalse(firstFragment.get(2).equals(secondFragment.get(2))); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java index cfe9979bc9c..aa2dd6b9b07 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java @@ -266,7 +266,7 @@ public void testFencingInteractionWithBookieRecovery() throws Exception { writelh.addEntry(tmp.getBytes()); } - BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0); + BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); killBookie(bookieToKill); // write entries to change ensemble @@ -318,7 +318,7 @@ public void testFencingInteractionWithBookieRecovery2() throws Exception { LedgerHandle readlh = bkc.openLedger(writelh.getId(), digestType, "testPasswd".getBytes()); // should be fenced by now - BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0); + BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); killBookie(bookieToKill); admin.recoverBookieData(bookieToKill); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java index 6291413614e..4ec5992a391 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java @@ -100,7 +100,7 @@ public void testGetBookieInfoTimeout() throws Exception { ClientConfiguration cConf = new ClientConfiguration(); cConf.setGetBookieInfoTimeout(2); - final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsemble(0).get(0); + final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsembleAt(0).get(0); int sleeptime = cConf.getBookieInfoTimeout() * 3; CountDownLatch latch = sleepBookie(bookieToSleep, sleeptime); latch.await(); 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 4b8d06c3127..68580edd9bb 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 @@ -34,6 +34,7 @@ import java.util.concurrent.CountDownLatch; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java index 68fd29c942d..84edc62eb95 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java @@ -156,7 +156,7 @@ public void testFailParallelRecoveryReadMissingEntryImmediately() throws Excepti LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - List ensemble = lh.getLedgerMetadata().getEnsemble(10); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(10); CountDownLatch latch1 = new CountDownLatch(1); CountDownLatch latch2 = new CountDownLatch(1); // sleep two bookie @@ -187,7 +187,7 @@ public void testParallelReadWithFailedBookies() throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - List ensemble = lh.getLedgerMetadata().getEnsemble(5); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(5); // kill two bookies killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); @@ -223,7 +223,7 @@ public void testParallelReadFailureWithFailedBookies() throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - List ensemble = lh.getLedgerMetadata().getEnsemble(5); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(5); // kill two bookies killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java index 7f28cc3e078..2ef72ff7dd2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java @@ -231,7 +231,7 @@ private void readWithFailedBookiesTest(boolean parallelRead) throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); List ensemble = - lh.getLedgerMetadata().getEnsemble(5); + lh.getLedgerMetadata().getEnsembleAt(5); // kill two bookies killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); @@ -270,7 +270,7 @@ private void readFailureWithFailedBookiesTest(boolean parallelRead) throws Excep LedgerHandle lh = bkc.openLedger(id, digestType, passwd); List ensemble = - lh.getLedgerMetadata().getEnsemble(5); + lh.getLedgerMetadata().getEnsembleAt(5); // kill bookies killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java index 48f638c3334..c4ec8f7a95a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java @@ -156,7 +156,7 @@ public void testReadLACLongPollWhenSomeBookiesDown() throws Exception { ServerConfiguration[] confs = new ServerConfiguration[numEntries - 1]; for (int j = 0; j < numEntries - 1; j++) { int idx = (i + 1 + j) % numEntries; - confs[j] = killBookie(lh.getLedgerMetadata().getLastEnsembleValue().get(idx)); + confs[j] = killBookie(LedgerMetadataUtils.getLastEnsembleValue(lh.getLedgerMetadata()).get(idx)); } final AtomicBoolean entryAsExpected = new AtomicBoolean(false); 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 d244218faf9..1e2f17e9a67 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 @@ -62,9 +62,9 @@ public void testReadTimeout() throws Exception { } Set beforeSet = new HashSet(); - beforeSet.addAll(writelh.getLedgerMetadata().getEnsemble(numEntries)); + beforeSet.addAll(writelh.getLedgerMetadata().getEnsembleAt(numEntries)); - final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0); + final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); int sleeptime = baseClientConf.getReadTimeout() * 3; CountDownLatch latch = sleepBookie(bookieToSleep, sleeptime); latch.await(); @@ -80,7 +80,7 @@ public void addComplete(int rc, LedgerHandle lh, Assert.assertTrue("Write request did not finish", completed.get()); Set afterSet = new HashSet(); - afterSet.addAll(writelh.getLedgerMetadata().getEnsemble(numEntries + 1)); + afterSet.addAll(writelh.getLedgerMetadata().getEnsembleAt(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/client/TestWatchEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java index 25906662d70..5c6a8c722da 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java @@ -34,6 +34,7 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerIdGenerator; import org.apache.bookkeeper.meta.LedgerManager; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java index fa184972da8..4e8d892721e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java @@ -99,7 +99,7 @@ private int getUpdatedLedgersCount(BookKeeper bk, List ledgers, Bo for (LedgerHandle lh : ledgers) { lh.close(); LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes()); - ensemble = openLedger.getLedgerMetadata().getEnsemble(0); + ensemble = openLedger.getLedgerMetadata().getEnsembleAt(0); if (ensemble.contains(toBookieAddr)) { updatedLedgersCount++; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java index ad5c450ceb1..c2b0d75704e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java @@ -35,6 +35,7 @@ import org.apache.bookkeeper.bookie.BookieShell.UpdateLedgerNotifier; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; @@ -98,7 +99,7 @@ public void testManyLedgers(boolean useShortHostName) throws Exception { ledgers.add(createLedgerWithEntries(bk, 0)); } - List ensemble = lh1.getLedgerMetadata().getEnsemble(0); + List ensemble = lh1.getLedgerMetadata().getEnsembleAt(0); BookieSocketAddress curBookieAddr = ensemble.get(0); baseConf.setUseHostNameAsBookieID(true); @@ -112,7 +113,7 @@ public void testManyLedgers(boolean useShortHostName) throws Exception { for (LedgerHandle lh : ledgers) { lh.close(); LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes()); - ensemble = openLedger.getLedgerMetadata().getEnsemble(0); + ensemble = openLedger.getLedgerMetadata().getEnsembleAt(0); assertTrue("Failed to update the ledger metadata to use bookie host name", ensemble.contains(toBookieAddr)); assertFalse("Failed to update the ledger metadata to use bookie host name", @@ -137,7 +138,7 @@ public void testLimitLessThanTotalLedgers() throws Exception { ledgers.add(createLedgerWithEntries(bk, 0)); } - List ensemble = lh1.getLedgerMetadata().getEnsemble(0); + List ensemble = lh1.getLedgerMetadata().getEnsembleAt(0); BookieSocketAddress curBookieAddr = ensemble.get(0); baseConf.setUseHostNameAsBookieID(true); @@ -193,7 +194,7 @@ public void testChangeEnsembleAfterRenaming(boolean useShortHostName) throws Exc LedgerHandle lh = createLedgerWithEntries(bk, 100); BookieServer bookieServer = bs.get(0); - List ensemble = lh.getLedgerMetadata().getEnsemble(0); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(0); BookieSocketAddress curBookieAddr = null; for (BookieSocketAddress bookieSocketAddress : ensemble) { if (bookieServer.getLocalAddress().equals(bookieSocketAddress)) { @@ -234,7 +235,7 @@ public void addComplete(int rccb, LedgerHandle lh, long entryId, Object ctx) { LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes()); final LedgerMetadata ledgerMetadata = openLedger.getLedgerMetadata(); assertEquals("Failed to reform ensemble!", 2, ledgerMetadata.getAllEnsembles().size()); - ensemble = ledgerMetadata.getEnsemble(0); + ensemble = ledgerMetadata.getEnsembleAt(0); assertTrue("Failed to update the ledger metadata to use bookie host name", ensemble.contains(toBookieAddr)); } @@ -273,7 +274,7 @@ public void addComplete(int rccb, LedgerHandle lh, long entryId, Object ctx) { } }; th.start(); - List ensemble = lh.getLedgerMetadata().getEnsemble(0); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(0); BookieSocketAddress curBookieAddr = ensemble.get(0); BookieSocketAddress toBookieAddr = new BookieSocketAddress("localhost:" + curBookieAddr.getPort()); UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin); @@ -287,7 +288,7 @@ public void addComplete(int rccb, LedgerHandle lh, long entryId, Object ctx) { } lh.close(); LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes()); - ensemble = openLedger.getLedgerMetadata().getEnsemble(0); + ensemble = openLedger.getLedgerMetadata().getEnsembleAt(0); assertTrue("Failed to update the ledger metadata to use bookie host name", ensemble.contains(toBookieAddr)); } @@ -300,7 +301,7 @@ private int getUpdatedLedgersCount(BookKeeper bk, List ledgers, Bo for (LedgerHandle lh : ledgers) { lh.close(); LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes()); - ensemble = openLedger.getLedgerMetadata().getEnsemble(0); + ensemble = openLedger.getLedgerMetadata().getEnsembleAt(0); if (ensemble.contains(toBookieAddr)) { updatedLedgersCount++; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java index 0316b7f2694..85cb727140f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java @@ -33,7 +33,6 @@ import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.client.MockBookKeeperTestCase; import org.apache.bookkeeper.conf.ClientConfiguration; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 6e37fee51ce..37f55b3e6e6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -50,8 +50,8 @@ import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.Code; -import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.testing.executors.MockExecutorController; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 182ffa117aa..d2735657e16 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -63,8 +63,8 @@ import org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector; import org.apache.bookkeeper.bookie.StateManager; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java index 03e13585e85..bdb29029766 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java @@ -46,8 +46,8 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; -import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index ff0126a920b..398bb07d712 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -29,7 +29,7 @@ import java.util.concurrent.Executors; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index ef57d6bddbf..f25aa88180b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -51,8 +51,8 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 23b33788f84..fd97da31fe1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -55,7 +55,6 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerHandleAdapter; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; @@ -480,9 +479,8 @@ void setLatch(CountDownLatch latch) { private BookieSocketAddress replaceBookieWithWriteFailingBookie(LedgerHandle lh) throws Exception { int bookieIdx = -1; - Long entryId = LedgerHandleAdapter.getLedgerMetadata(lh).getAllEnsembles().firstKey(); - List curEnsemble = LedgerHandleAdapter - .getLedgerMetadata(lh).getAllEnsembles().get(entryId); + Long entryId = lh.getLedgerMetadata().getAllEnsembles().firstKey(); + List curEnsemble = lh.getLedgerMetadata().getAllEnsembles().get(entryId); // Identify a bookie in the current ledger ensemble to be replaced BookieSocketAddress replacedBookie = null; 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 07b937550e0..45cd2eb7e18 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 @@ -36,7 +36,6 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.BookKeeperTestClient; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerHandleAdapter; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; @@ -150,8 +149,7 @@ public void testOpenLedgers() throws Exception { List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); LedgerHandle lh = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKillAddr = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); final String urLedgerZNode = getUrLedgerZNode(lh); ledgerReplicaIndex = getReplicaIndexInLedger(lh, replicaToKillAddr); @@ -199,8 +197,7 @@ public void testClosedLedgers() throws Exception { closeLedgers(listOfLedgerHandle); LedgerHandle lhandle = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(lhandle).getAllEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKillAddr = lhandle.getLedgerMetadata().getAllEnsembles().get(0L).get(0); CountDownLatch latch = new CountDownLatch(listOfLedgerHandle.size()); for (LedgerHandle lh : listOfLedgerHandle) { @@ -261,8 +258,7 @@ public void testStopWhileReplicationInProgress() throws Exception { numberOfLedgers, 5); closeLedgers(listOfLedgerHandle); LedgerHandle handle = listOfLedgerHandle.get(0); - BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(handle).getAllEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKillAddr = handle.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie:" + replicaToKillAddr); // Each ledger, there will be two events : create urLedger and after @@ -340,13 +336,13 @@ public void testNoSuchLedgerExists() throws Exception { assertNull("UrLedger already exists!", watchUrLedgerNode(getUrLedgerZNode(lh), latch)); } - BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(listOfLedgerHandle.get(0)).getAllEnsembles() - .get(0L).get(0); + BookieSocketAddress replicaToKillAddr = listOfLedgerHandle.get(0) + .getLedgerMetadata().getAllEnsembles() + .get(0L).get(0); killBookie(replicaToKillAddr); - replicaToKillAddr = LedgerHandleAdapter - .getLedgerMetadata(listOfLedgerHandle.get(0)).getAllEnsembles() - .get(0L).get(0); + replicaToKillAddr = listOfLedgerHandle.get(0) + .getLedgerMetadata().getAllEnsembles() + .get(0L).get(0); killBookie(replicaToKillAddr); // waiting to publish urLedger znode by Auditor latch.await(); @@ -383,10 +379,9 @@ public void testEmptyLedgerLosesQuorumEventually() throws Exception { String urZNode = getUrLedgerZNode(lh); watchUrLedgerNode(urZNode, latch); - BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(2); + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(2); LOG.info("Killing last bookie, {}, in ensemble {}", replicaToKill, - LedgerHandleAdapter.getLedgerMetadata(lh).getAllEnsembles().get(0L)); + lh.getLedgerMetadata().getAllEnsembles().get(0L)); killBookie(replicaToKill); getAuditor(10, TimeUnit.SECONDS).submitAuditTask().get(); // ensure auditor runs @@ -398,10 +393,9 @@ public void testEmptyLedgerLosesQuorumEventually() throws Exception { assertTrue("Should be marked as replicated", latch.await(10, TimeUnit.SECONDS)); } - replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(1); + replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(1); LOG.info("Killing second bookie, {}, in ensemble {}", replicaToKill, - LedgerHandleAdapter.getLedgerMetadata(lh).getAllEnsembles().get(0L)); + lh.getLedgerMetadata().getAllEnsembles().get(0L)); killBookie(replicaToKill); getAuditor(10, TimeUnit.SECONDS).submitAuditTask().get(); // ensure auditor runs diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java index 81319997085..0e6f9276c14 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java @@ -29,7 +29,6 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerHandleAdapter; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -74,8 +73,7 @@ public void testAutoRecoveryAlongWithBookieServers() throws Exception { lh.addEntry(testData); } lh.close(); - BookieSocketAddress replicaToKill = LedgerHandleAdapter - .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(0); + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); killBookie(replicaToKill); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java index 5d92979437c..306a7f6a6b7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java @@ -48,7 +48,7 @@ import org.apache.bookkeeper.client.BookKeeperTestClient; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index 5155fad8b17..d571bf8057e 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -44,7 +44,7 @@ import java.util.function.Function; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.metadata.etcd.helpers.KeyIterator; diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java index d466b340d7a..292d9736372 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java @@ -16,7 +16,7 @@ import java.util.Objects; import java.util.function.Consumer; -import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.versioning.Versioned; diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index a7df94f14ce..984224c6b8b 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -50,8 +50,8 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.Code; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java index 3152a74c563..2ca0268c3a8 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java +++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java @@ -37,8 +37,4 @@ public static void forceRecoverLedger(LedgerHandle lh, "Recovery can only run on ReadOnlyLedgerHandle"); ((ReadOnlyLedgerHandle) lh).recover(cb, null, true); } - - public static LedgerMetadata getLedgerMetadata(LedgerHandle lh) { - return lh.getLedgerMetadata(); - } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java index c06bf848ac2..b7a6ba9120a 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java @@ -119,7 +119,7 @@ public void readEntryComplete(int rc, long lid, long eid, ByteBuf buffer, Object } }; - List ensemble = lh.getLedgerMetadata().getEnsemble(eid); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(eid); for (int i = 0; i < writeSet.size(); i++) { int idx = writeSet.get(i); clientCtx.getBookieClient().readEntry(ensemble.get(idx), lh.getId(), eid, readEntryCallback, @@ -224,7 +224,7 @@ public void readLacs(final LedgerHandle lh, long eid, } }; - List ensemble = lh.getLedgerMetadata().getEnsemble(eid); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(eid); for (int i = 0; i < writeSet.size(); i++) { int idx = writeSet.get(i); clientCtx.getBookieClient().readEntry(ensemble.get(idx), lh.getId(), eid, readEntryCallback, diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java index 75e22acfd30..b04c27b9bbc 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java @@ -42,9 +42,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import org.apache.bookkeeper.client.BookKeeper; -import org.apache.bookkeeper.client.BookKeeperAccessor; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.LedgerMetadata; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.feature.FixedValueFeature; @@ -1992,7 +1991,7 @@ public void testCreateLogStreamWithDifferentReplicationFactor() throws Exception long ledgerId = segments.get(0).getLogSegmentId(); LedgerHandle lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get() .openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8)); - LedgerMetadata metadata = BookKeeperAccessor.getLedgerMetadata(lh); + LedgerMetadata metadata = lh.getLedgerMetadata(); assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT, metadata.getEnsembleSize()); lh.close(); Utils.close(writer); @@ -2011,7 +2010,7 @@ public void testCreateLogStreamWithDifferentReplicationFactor() throws Exception ledgerId = segments.get(0).getLogSegmentId(); lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get() .openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8)); - metadata = BookKeeperAccessor.getLedgerMetadata(lh); + metadata = lh.getLedgerMetadata(); assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT - 1, metadata.getEnsembleSize()); lh.close(); Utils.close(writer); diff --git a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy index 7f229e79074..b7987a8356d 100644 --- a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy +++ b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy @@ -30,7 +30,7 @@ import org.apache.bookkeeper.client.BKException import org.apache.bookkeeper.client.BookKeeper import org.apache.bookkeeper.client.BookKeeperAdmin import org.apache.bookkeeper.client.LedgerHandle -import org.apache.bookkeeper.client.LedgerMetadata +import org.apache.bookkeeper.client.api.LedgerMetadata import org.apache.bookkeeper.conf.ClientConfiguration import org.apache.bookkeeper.net.BookieSocketAddress import org.apache.bookkeeper.proto.BookieProtocol From 4594baca1eaec519dd56b41d3f1451e9cae7dd01 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 3 Dec 2018 16:07:16 +0100 Subject: [PATCH 0156/1642] Mark 4.8.1 as latest release Descriptions of the changes in this PR: Change website configuration so that the downloads make suggest 4.8.1 and 4.8.0 Reviewers: Sijie Guo This closes #1855 from eolivelli/fix/mark-481-latest --- site/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/_config.yml b/site/_config.yml index 2512864fda3..d747fc4518a 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -33,7 +33,7 @@ archived_versions: - "4.1.0" - "4.0.0" latest_version: "4.9.0-SNAPSHOT" -latest_release: "4.8.0" +latest_release: "4.8.1" stable_release: "4.7.2" distributedlog_version: "4.7.2" From 66e368d72d255620f4ad49e4ab0aec51ddf34189 Mon Sep 17 00:00:00 2001 From: snow4young <1723262513@qq.com> Date: Tue, 4 Dec 2018 03:41:01 +0800 Subject: [PATCH 0157/1642] [TABLE SERVICE] remove extra code Descriptions of the changes in this PR: review code and found extra code. ### Motivation clean code ### Changes clean extra code Master Issue: # Reviewers: Sijie Guo , Enrico Olivelli This closes #1856 from xujianhai/master --- .../bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java | 1 - 1 file changed, 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java index eff40e8a8df..c06a3cc8916 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java @@ -117,7 +117,6 @@ public KeyValueStorageRocksDB(String path, DbConfigType dbConfigType, ServerConf if (lz4CompressionEnabled) { options.setCompressionType(CompressionType.LZ4_COMPRESSION); } - options.setCompressionType(CompressionType.LZ4_COMPRESSION); options.setWriteBufferSize(writeBufferSizeMB * 1024 * 1024); options.setMaxWriteBufferNumber(4); if (numLevels > 0) { From 7a5c5e4cf7bdce1e2220fc1155965d4bb4a5a25f Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Tue, 4 Dec 2018 12:03:14 +0100 Subject: [PATCH 0158/1642] Add max ledger metadata format version to layout The max ledger metadata format version is the maximum format version that will be used to write ledger metadata. By setting it in the ledger layout it becomes a cluster-wide configuration which is initialized along with the cluster. Any cluster initialized with the current code will end up with version 2. For this cluster serde will only ever serialize with up to version 2 of the ledger metadata format, so all clients that understand version 2 will continue to work, even as the software version increases and new metadata formats become available (such as the binary metadata format coming soon). Currently there is no handling in LedgerMetadataSerDe, because we don't have a new version, but when there is, it will use ``` Math.min(maxLedgerMetadataFormatVersion, metadata.getMetadataFormatVersion()) ``` to decide which serialization method to use. Reviewers: Enrico Olivelli , Sijie Guo This closes #1858 from ivankelly/format-version --- .../apache/bookkeeper/bookie/BookieShell.java | 7 ++- .../conf/AbstractConfiguration.java | 24 +++++++++++ .../meta/AbstractZkLedgerManager.java | 2 +- .../meta/AbstractZkLedgerManagerFactory.java | 1 + .../apache/bookkeeper/meta/LedgerLayout.java | 43 +++++++++++++++---- .../bookkeeper/meta/LedgerMetadataSerDe.java | 6 +++ .../meta/MSLedgerManagerFactory.java | 2 +- .../http/service/GetLedgerMetaService.java | 2 +- .../http/service/ListLedgerService.java | 2 +- .../bookkeeper/client/LedgerMetadataTest.java | 6 ++- .../meta/AbstractZkLedgerManagerTest.java | 2 +- .../bookkeeper/meta/MockLedgerManager.java | 2 +- .../bookkeeper/meta/TestLedgerLayout.java | 43 +++++++++++++++++++ .../server/http/TestHttpService.java | 3 +- .../metadata/etcd/EtcdLedgerManager.java | 30 +++++++------ .../etcd/EtcdLedgerManagerFactory.java | 4 +- .../metadata/etcd/EtcdLedgerManagerTest.java | 3 +- 17 files changed, 147 insertions(+), 35 deletions(-) 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 9108ef36d24..9c6180b63e4 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 @@ -1106,7 +1106,7 @@ Options getOptions() { void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { System.out.println("ledgerID: " + ledgerIdFormatter.formatLedgerId(ledgerId)); if (printMeta) { - System.out.println(new String(new LedgerMetadataSerDe().serialize(md), UTF_8)); + System.out.println(md.toString()); } } @@ -1115,7 +1115,10 @@ void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { */ class LedgerMetadataCmd extends MyCommand { Options lOpts = new Options(); - LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + // the max version won't actually take effect as this tool + // never creates new metadata (there'll already be a format version in the existing metadata) + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe( + LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); LedgerMetadataCmd() { super(CMD_LEDGERMETADATA); 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 7f00d09adb5..3057bfe65b6 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 @@ -85,6 +85,7 @@ public abstract class AbstractConfiguration protected static final String STORE_SYSTEMTIME_AS_LEDGER_UNDERREPLICATED_MARK_TIME = "storeSystemTimeAsLedgerUnderreplicatedMarkTime"; protected static final String STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME = "storeSystemTimeAsLedgerCreationTime"; + protected static final String MAX_LEDGER_METADATA_FORMAT_VERSION = "maxLedgerMetadataFormatVersion"; // Metastore settings, only being used when LEDGER_MANAGER_FACTORY_CLASS is MSLedgerManagerFactory protected static final String METASTORE_IMPL_CLASS = "metastoreImplClass"; @@ -855,6 +856,29 @@ public boolean getStoreSystemTimeAsLedgerUnderreplicatedMarkTime() { return getBoolean(STORE_SYSTEMTIME_AS_LEDGER_UNDERREPLICATED_MARK_TIME, true); } + /** + * Set the maximum format version which should be used when serializing ledger metadata. + * Setting the maximum format allows us to ensure that all clients running against a cluster + * will be able to read back all written metadata. + * Normally this is set when loading the layout from the metadata store. Users should not + * set this directly. + */ + public T setMaxLedgerMetadataFormatVersion(int maxVersion) { + setProperty(MAX_LEDGER_METADATA_FORMAT_VERSION, maxVersion); + return getThis(); + } + + /** + * Get the maximum format version which should be used when serializing ledger metadata. + * The default is 2, as that was the current version when this functionallity was introduced. + * + * @see #setMaxLedgerMetadataFormatVersion(int) + * @return the maximum format version with which to serialize metadata. + */ + public int getMaxLedgerMetadataFormatVersion() { + return getInteger(MAX_LEDGER_METADATA_FORMAT_VERSION, 2); + } + /** * Whether to preserve MDC for tasks in Executor. * diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 5dbdd06791d..97413b7bffb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -160,7 +160,7 @@ private void handleMetadata(Versioned result, Throwable exceptio * ZooKeeper Client Handle */ protected AbstractZkLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { - this.serDe = new LedgerMetadataSerDe(); + this.serDe = new LedgerMetadataSerDe(conf.getMaxLedgerMetadataFormatVersion()); this.conf = conf; this.zk = zk; this.ledgerRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java index 72bc3e841e6..4bb2fb141d3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java @@ -178,6 +178,7 @@ public static LedgerManagerFactory newLedgerManagerFactory( if (log.isDebugEnabled()) { log.debug("read ledger layout {}", layout); } + conf.setMaxLedgerMetadataFormatVersion(layout.getMaxLedgerMetadataFormatVersion()); // there is existing layout, we need to look into the layout. // handle pre V2 layout diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java index eb5b705c16b..c39e6b6feae 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java @@ -39,8 +39,14 @@ public class LedgerLayout { // version of ledger layout metadata public static final int LAYOUT_FORMAT_VERSION = 2; - private static final String splitter = ":"; - private static final String lSplitter = "\n"; + private static final String FIELD_SPLITTER = ":"; + private static final String LINE_SPLITTER = "\n"; + + // For version 2 and below, max ledger metadata format wasn't stored in the layout + // so assume 2 if it is missing. + private static final int DEFAULT_MAX_LEDGER_METADATA_FORMAT_VERSION = 2; + private static final String MAX_LEDGER_METADATA_FORMAT_VERSION_FIELD = + "MAX_LEDGER_METADATA_FORMAT_VERSION"; // ledger manager factory class private final String managerFactoryClass; @@ -50,6 +56,9 @@ public class LedgerLayout { // layout version of how to store layout information private final int layoutFormatVersion; + // maximum format version that can be used for storing ledger metadata + private final int maxLedgerMetadataFormatVersion; + /** * Ledger Layout Constructor. * @@ -59,13 +68,17 @@ public class LedgerLayout { * Ledger Manager Version */ public LedgerLayout(String managerFactoryCls, int managerVersion) { - this(managerFactoryCls, managerVersion, LAYOUT_FORMAT_VERSION); + this(managerFactoryCls, managerVersion, + LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION, + LAYOUT_FORMAT_VERSION); } LedgerLayout(String managerFactoryCls, int managerVersion, + int maxLedgerMetadataFormatVersion, int layoutVersion) { this.managerFactoryClass = managerFactoryCls; this.managerVersion = managerVersion; + this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; this.layoutFormatVersion = layoutVersion; } @@ -76,8 +89,11 @@ public LedgerLayout(String managerFactoryCls, int managerVersion) { */ public byte[] serialize() throws IOException { String s = - new StringBuilder().append(layoutFormatVersion).append(lSplitter) - .append(managerFactoryClass).append(splitter).append(managerVersion).toString(); + new StringBuilder().append(layoutFormatVersion).append(LINE_SPLITTER) + .append(managerFactoryClass).append(FIELD_SPLITTER).append(managerVersion).append(LINE_SPLITTER) + .append(MAX_LEDGER_METADATA_FORMAT_VERSION_FIELD).append(FIELD_SPLITTER) + .append(maxLedgerMetadataFormatVersion) + .toString(); if (log.isDebugEnabled()) { log.debug("Serialized layout info: {}", s); @@ -100,7 +116,7 @@ public static LedgerLayout parseLayout(byte[] bytes) throws IOException { log.debug("Parsing Layout: {}", layout); } - String lines[] = layout.split(lSplitter); + String lines[] = layout.split(LINE_SPLITTER); try { int layoutFormatVersion = Integer.parseInt(lines[0]); @@ -113,7 +129,7 @@ public static LedgerLayout parseLayout(byte[] bytes) throws IOException { throw new IOException("Ledger manager and its version absent from layout: " + layout); } - String[] parts = lines[1].split(splitter); + String[] parts = lines[1].split(FIELD_SPLITTER); if (parts.length != 2) { throw new IOException("Invalid Ledger Manager defined in layout : " + layout); } @@ -121,7 +137,18 @@ public static LedgerLayout parseLayout(byte[] bytes) throws IOException { String managerFactoryCls = parts[0]; // ledger manager version int managerVersion = Integer.parseInt(parts[1]); - return new LedgerLayout(managerFactoryCls, managerVersion, layoutFormatVersion); + + int maxLedgerMetadataFormatVersion = DEFAULT_MAX_LEDGER_METADATA_FORMAT_VERSION; + if (lines.length >= 3) { + String[] metadataFormatParts = lines[2].split(FIELD_SPLITTER); + if (metadataFormatParts.length != 2 + || !metadataFormatParts[0].equals(MAX_LEDGER_METADATA_FORMAT_VERSION_FIELD)) { + throw new IOException("Invalid field for max ledger metadata format:" + lines[2]); + } + maxLedgerMetadataFormatVersion = Integer.parseInt(metadataFormatParts[1]); + } + return new LedgerLayout(managerFactoryCls, managerVersion, + maxLedgerMetadataFormatVersion, layoutFormatVersion); } catch (NumberFormatException e) { throw new IOException("Could not parse layout '" + layout + "'", e); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java index 6020a3bf4f1..4653af4f309 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -63,6 +63,12 @@ public class LedgerMetadataSerDe { private static final String V1_CLOSED_TAG = "CLOSED"; private static final int V1_IN_RECOVERY_ENTRY_ID = -102; + private final int maxLedgerMetadataFormatVersion; + + public LedgerMetadataSerDe(int maxLedgerMetadataFormatVersion) { + this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; + } + public byte[] serialize(LedgerMetadata metadata) { if (metadata.getMetadataFormatVersion() == 1) { return serializeVersion1(metadata); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index fc876324452..38c6816dd6b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -284,7 +284,7 @@ private void handleMetadata(Versioned metadata, Throwable except this.conf = conf; this.zk = zk; this.metastore = metastore; - this.serDe = new LedgerMetadataSerDe(); + this.serDe = new LedgerMetadataSerDe(conf.getMaxLedgerMetadataFormatVersion()); try { ledgerTable = metastore.createScannableTable(TABLE_NAME); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java index 7fcaeda810d..537b50ce468 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java @@ -53,7 +53,7 @@ public GetLedgerMetaService(ServerConfiguration conf, BookieServer bookieServer) checkNotNull(conf); this.conf = conf; this.bookieServer = bookieServer; - this.serDe = new LedgerMetadataSerDe(); + this.serDe = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java index f553b6431b9..13022e4f2d7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java @@ -58,7 +58,7 @@ public ListLedgerService(ServerConfiguration conf, BookieServer bookieServer) { checkNotNull(conf); this.conf = conf; this.bookieServer = bookieServer; - this.serDe = new LedgerMetadataSerDe(); + this.serDe = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java index a9ffc2d4221..d9141e64616 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java @@ -80,7 +80,8 @@ public void testStoreSystemtimeAsLedgerCtimeEnabled() .withCreationTime(System.currentTimeMillis()) .storingCreationTime(true) .build(); - LedgerMetadataFormat format = new LedgerMetadataSerDe().buildProtoFormat(lm); + LedgerMetadataFormat format = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION) + .buildProtoFormat(lm); assertTrue(format.hasCtime()); } @@ -94,7 +95,8 @@ public void testStoreSystemtimeAsLedgerCtimeDisabled() LedgerMetadata lm = LedgerMetadataBuilder.create() .newEnsembleEntry(0L, ensemble).build(); - LedgerMetadataFormat format = new LedgerMetadataSerDe().buildProtoFormat(lm); + LedgerMetadataFormat format = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION) + .buildProtoFormat(lm); assertFalse(format.hasCtime()); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 37f55b3e6e6..20a8d5134ac 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -142,7 +142,7 @@ public void setup() throws Exception { assertSame(conf, ledgerManager.conf); assertSame(scheduler, ledgerManager.scheduler); - this.serDe = new LedgerMetadataSerDe(); + this.serDe = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } @After diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index 398bb07d712..a73adeb4eee 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -72,7 +72,7 @@ private MockLedgerManager(Map> metadataMap, this.metadataMap = metadataMap; this.executor = executor; this.ownsExecutor = ownsExecutor; - this.serDe = new LedgerMetadataSerDe(); + this.serDe = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } public MockLedgerManager newClient() { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java index 62315fe2e81..16b30bdf071 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java @@ -18,8 +18,12 @@ */ package org.apache.bookkeeper.meta; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.fail; + +import java.io.IOException; import org.junit.Test; @@ -63,4 +67,43 @@ public void testGetters() { hierarchical1.getLayoutFormatVersion()); } + @Test + public void testParseNoMaxLedgerMetadataFormatVersion() throws Exception { + LedgerLayout layout = LedgerLayout.parseLayout("1\nblahblahLM:3".getBytes(UTF_8)); + + assertEquals(layout.getMaxLedgerMetadataFormatVersion(), 2); + } + + @Test + public void testParseWithMaxLedgerMetadataFormatVersion() throws Exception { + LedgerLayout layout = LedgerLayout.parseLayout( + "1\nblahblahLM:3\nMAX_LEDGER_METADATA_FORMAT_VERSION:123".getBytes(UTF_8)); + + assertEquals(layout.getMaxLedgerMetadataFormatVersion(), 123); + } + + @Test + public void testCorruptMaxLedgerLayout() throws Exception { + try { + LedgerLayout.parseLayout("1\nblahblahLM:3\nMAXXX_LEDGER_METADATA_FORMAT_VERSION:123".getBytes(UTF_8)); + fail("Shouldn't have been able to parse"); + } catch (IOException ioe) { + // expected + } + + try { + LedgerLayout.parseLayout("1\nblahblahLM:3\nMAXXX_LEDGER_METADATA_FORMAT_VERSION:blah".getBytes(UTF_8)); + fail("Shouldn't have been able to parse"); + } catch (IOException ioe) { + // expected + } + } + + @Test + public void testMoreFieldsAdded() throws Exception { + LedgerLayout layout = LedgerLayout.parseLayout( + "1\nblahblahLM:3\nMAX_LEDGER_METADATA_FORMAT_VERSION:123\nFOO:BAR".getBytes(UTF_8)); + + assertEquals(layout.getMaxLedgerMetadataFormatVersion(), 123); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index c4d441dbe89..e88d29d3a68 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -387,7 +387,8 @@ public void testGetLedgerMetaService() throws Exception { assertEquals(1, respBody.size()); // verify LedgerMetadata content is equal assertTrue(respBody.get(ledgerId.toString()).toString() - .equals(new String(new LedgerMetadataSerDe().serialize(lh[0].getLedgerMetadata())))); + .equals(new String(new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION) + .serialize(lh[0].getLedgerMetadata())))); } @Test diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index d571bf8057e..e20a9ba146f 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -45,6 +45,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.metadata.etcd.helpers.KeyIterator; @@ -64,19 +65,8 @@ @Slf4j class EtcdLedgerManager implements LedgerManager { - private final LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); - private final Function ledgerMetadataFunction = bs -> { - try { - return serDe.parseConfig( - bs.getBytes(), - Optional.empty() - ); - } catch (IOException ioe) { - log.error("Could not parse ledger metadata : {}", bs.toStringUtf8(), ioe); - throw new RuntimeException( - "Could not parse ledger metadata : " + bs.toStringUtf8(), ioe); - } - }; + private final LedgerMetadataSerDe serDe; + private final Function ledgerMetadataFunction; private final String scope; private final Client client; @@ -89,12 +79,24 @@ class EtcdLedgerManager implements LedgerManager { private volatile boolean closed = false; - EtcdLedgerManager(Client client, + EtcdLedgerManager(AbstractConfiguration conf, + Client client, String scope) { this.client = client; this.kvClient = client.getKVClient(); this.scope = scope; this.watchClient = new EtcdWatchClient(client); + this.serDe = new LedgerMetadataSerDe(conf.getMaxLedgerMetadataFormatVersion()); + + this.ledgerMetadataFunction = bs -> { + try { + return serDe.parseConfig(bs.getBytes(), Optional.empty()); + } catch (IOException ioe) { + log.error("Could not parse ledger metadata : {}", bs.toStringUtf8(), ioe); + throw new RuntimeException( + "Could not parse ledger metadata : " + bs.toStringUtf8(), ioe); + } + }; } private boolean isClosed() { diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java index dc7d1517b2b..9c19df07ddb 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java @@ -42,6 +42,7 @@ class EtcdLedgerManagerFactory implements LedgerManagerFactory { private String scope; private Client client; + private AbstractConfiguration conf; @Override public int getCurrentVersion() { @@ -66,6 +67,7 @@ public LedgerManagerFactory initialize(AbstractConfiguration conf, throw new IOException("Invalid metadata service uri", e); } this.client = etcdLayoutManager.getClient(); + this.conf = conf; return this; } @@ -82,7 +84,7 @@ public LedgerIdGenerator newLedgerIdGenerator() { @Override public LedgerManager newLedgerManager() { - return new EtcdLedgerManager(client, scope); + return new EtcdLedgerManager(conf, client, scope); } @Override diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index 984224c6b8b..27dc84c24ac 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -53,6 +53,7 @@ import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; import org.apache.bookkeeper.metadata.etcd.helpers.ValueStream; @@ -81,7 +82,7 @@ public class EtcdLedgerManagerTest extends EtcdTestBase { public void setUp() throws Exception { super.setUp(); this.scope = RandomStringUtils.randomAlphabetic(8); - this.lm = new EtcdLedgerManager(etcdClient, scope); + this.lm = new EtcdLedgerManager(new ClientConfiguration(), etcdClient, scope); } @Override From 974eb05f2fe3ff9e3d9cc0e2c1c7a5d9133e0a87 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 4 Dec 2018 10:53:41 -0800 Subject: [PATCH 0159/1642] remove misleading comment Descriptions of the changes in this PR: *Motivation* ParallelRead is actually used internally at ledger recovery. It is used at production. *Changes* Removed misleading comment. Reviewers: Enrico Olivelli , Jia Zhai This closes #1671 from sijie/remove_misleading_comment --- .../main/java/org/apache/bookkeeper/client/PendingReadOp.java | 1 - 1 file changed, 1 deletion(-) 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 aa89eaa7f9e..e8206602e93 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 @@ -487,7 +487,6 @@ public ScheduledFuture getSpeculativeTask() { return speculativeTask; } - // I don't think this is ever used in production code -Ivan PendingReadOp parallelRead(boolean enabled) { this.parallelRead = enabled; return this; From 3970e6aff52ff021320df188c1c05dc82e6ef940 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Tue, 4 Dec 2018 10:55:15 -0800 Subject: [PATCH 0160/1642] [RELEASE NOTES] Release Notes for 4.7.3 Reviewers: Enrico Olivelli This closes #1850 from sijie/release_notes_473 --- site/docs/4.7.3/overview/releaseNotes.md | 34 ++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 site/docs/4.7.3/overview/releaseNotes.md diff --git a/site/docs/4.7.3/overview/releaseNotes.md b/site/docs/4.7.3/overview/releaseNotes.md new file mode 100644 index 00000000000..0e574540ed5 --- /dev/null +++ b/site/docs/4.7.3/overview/releaseNotes.md @@ -0,0 +1,34 @@ +--- +title: Apache BookKeeper 4.7.3 Release Notes +--- + +This is the 16th release of Apache BookKeeper! + +The 4.7.3 release is a bugfix release which fixes a bunch of issues reported from users of 4.7.2. + +Apache BookKeeper users who are using 4.7.2 are encouraged to upgrade to 4.7.3. The technical details of this release are summarized +below. + +## Highlights + +- Cancel Scheduled SpeculativeReads, see [apache/bookkeeper#1665](https://github.com/apache/bookkeeper/pull/1665) + +- IllegalReferenceCountException at closing EntryLogManagerForSingleEntryLog, see [apache/bookkeeper#1703](https://github.com/apache/bookkeeper/issues/1703) + +- EntryMemTable.newEntry retains reference to passed ByteBuffer array can cause corruption on journal replay, see [apache/bookkeeper#1737](https://github.com/apache/bookkeeper/issues/1737) + +- Ledger deletion racing with flush can cause a ledger index to be resurrected, see [apache/bookkeeper#1757](https://github.com/apache/bookkeeper/issues/1757) + +- Don't cache Bookie hostname DNS resolution forever, see [apache/bookkeeper#1762](https://github.com/apache/bookkeeper/pull/1762) + +- Use default metric registry in Prometheus export, see [apache/bookkeeper#1765](https://github.com/apache/bookkeeper/pull/1765) + +- Fix Auth with v2 protocol, see [apache/bookkeeper#1805](https://github.com/apache/bookkeeper/pull/1805) + +- Remove MathUtils.now to address compaction scheduling deplay issues, see [apache/bookkeeper#1837](https://github.com/apache/bookkeeper/pull/1837) + +- DbLedgerStorage should do periodical flush, see [apache/bookkeeper#1843](https://github.com/apache/bookkeeper/pull/1843) + +## Full list of changes + +- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.3+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.3+is%3Aclosed) From 141d28ff8074aacf12531022b54fdba7bea13a64 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 4 Dec 2018 20:28:53 +0100 Subject: [PATCH 0161/1642] Update BookKeeper version in Docker file for 4.8.1 release Descriptions of the changes in this PR: Change version from 4.8.0 in 4.8.1 on Dockerfile in branch-4.8 ### Motivation Distribute new Docker images for 4.8.1 release Reviewers: Sijie Guo This closes #1851 from eolivelli/fix/docker-4.8.1 Reviewers: Sijie Guo This closes #1854 from eolivelli/fix/pick-481-docker --- docker/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/Dockerfile b/docker/Dockerfile index deb00eacc9b..e63ab771d50 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -20,7 +20,7 @@ FROM centos:7 MAINTAINER Apache BookKeeper -ARG BK_VERSION=4.8.0 +ARG BK_VERSION=4.8.1 ARG DISTRO_NAME=bookkeeper-server-${BK_VERSION}-bin ARG GPG_KEY=A615D22C From 296bde8b25ceedd5706595bafb4c4a04b1319ac1 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 5 Dec 2018 12:35:18 -0800 Subject: [PATCH 0162/1642] [RELEASE] [WEBSITE] Add documentation for release 4.7.3 *Motivation* Add documentation for release 4.7.3 Descriptions of the changes in this PR: Reviewers: Matteo Merli This closes #1859 from sijie/website_473 --- site/_config.yml | 5 +- site/docs/4.7.3/admin/autorecovery.md | 128 +++ site/docs/4.7.3/admin/bookies.md | 180 ++++ site/docs/4.7.3/admin/geo-replication.md | 22 + site/docs/4.7.3/admin/http.md | 394 +++++++++ site/docs/4.7.3/admin/metrics.md | 41 + site/docs/4.7.3/admin/perf.md | 3 + site/docs/4.7.3/admin/placement.md | 3 + site/docs/4.7.3/admin/upgrade.md | 183 ++++ site/docs/4.7.3/api/distributedlog-api.md | 395 +++++++++ site/docs/4.7.3/api/ledger-adv-api.md | 111 +++ site/docs/4.7.3/api/ledger-api.md | 802 ++++++++++++++++++ site/docs/4.7.3/api/overview.md | 17 + site/docs/4.7.3/deployment/dcos.md | 142 ++++ site/docs/4.7.3/deployment/kubernetes.md | 181 ++++ site/docs/4.7.3/deployment/manual.md | 56 ++ site/docs/4.7.3/development/codebase.md | 3 + site/docs/4.7.3/development/protocol.md | 148 ++++ site/docs/4.7.3/getting-started/concepts.md | 202 +++++ .../4.7.3/getting-started/installation.md | 74 ++ .../docs/4.7.3/getting-started/run-locally.md | 16 + site/docs/4.7.3/overview/overview.md | 58 ++ .../4.7.3/overview/releaseNotesTemplate.md | 17 + site/docs/4.7.3/reference/cli.md | 10 + site/docs/4.7.3/reference/config.md | 9 + site/docs/4.7.3/reference/metrics.md | 3 + site/docs/4.7.3/security/overview.md | 21 + site/docs/4.7.3/security/sasl.md | 202 +++++ site/docs/4.7.3/security/tls.md | 210 +++++ site/docs/4.7.3/security/zookeeper.md | 41 + site/releases.md | 8 + 31 files changed, 3683 insertions(+), 2 deletions(-) create mode 100644 site/docs/4.7.3/admin/autorecovery.md create mode 100644 site/docs/4.7.3/admin/bookies.md create mode 100644 site/docs/4.7.3/admin/geo-replication.md create mode 100644 site/docs/4.7.3/admin/http.md create mode 100644 site/docs/4.7.3/admin/metrics.md create mode 100644 site/docs/4.7.3/admin/perf.md create mode 100644 site/docs/4.7.3/admin/placement.md create mode 100644 site/docs/4.7.3/admin/upgrade.md create mode 100644 site/docs/4.7.3/api/distributedlog-api.md create mode 100644 site/docs/4.7.3/api/ledger-adv-api.md create mode 100644 site/docs/4.7.3/api/ledger-api.md create mode 100644 site/docs/4.7.3/api/overview.md create mode 100644 site/docs/4.7.3/deployment/dcos.md create mode 100644 site/docs/4.7.3/deployment/kubernetes.md create mode 100644 site/docs/4.7.3/deployment/manual.md create mode 100644 site/docs/4.7.3/development/codebase.md create mode 100644 site/docs/4.7.3/development/protocol.md create mode 100644 site/docs/4.7.3/getting-started/concepts.md create mode 100644 site/docs/4.7.3/getting-started/installation.md create mode 100644 site/docs/4.7.3/getting-started/run-locally.md create mode 100644 site/docs/4.7.3/overview/overview.md create mode 100644 site/docs/4.7.3/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.7.3/reference/cli.md create mode 100644 site/docs/4.7.3/reference/config.md create mode 100644 site/docs/4.7.3/reference/metrics.md create mode 100644 site/docs/4.7.3/security/overview.md create mode 100644 site/docs/4.7.3/security/sasl.md create mode 100644 site/docs/4.7.3/security/tls.md create mode 100644 site/docs/4.7.3/security/zookeeper.md diff --git a/site/_config.yml b/site/_config.yml index d747fc4518a..3e4c3aa89e6 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -11,6 +11,7 @@ versions: # [next_version_placeholder] - "4.8.1" - "4.8.0" +- "4.7.3" - "4.7.2" - "4.7.1" - "4.7.0" @@ -34,8 +35,8 @@ archived_versions: - "4.0.0" latest_version: "4.9.0-SNAPSHOT" latest_release: "4.8.1" -stable_release: "4.7.2" -distributedlog_version: "4.7.2" +stable_release: "4.7.3" +distributedlog_version: "4.7.3" defaults: - scope: diff --git a/site/docs/4.7.3/admin/autorecovery.md b/site/docs/4.7.3/admin/autorecovery.md new file mode 100644 index 00000000000..b1dd078f9b2 --- /dev/null +++ b/site/docs/4.7.3/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: + +* the `shell recover` option +* 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 shell recover \ + 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 shell recover \ + 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.7.3/admin/bookies.md b/site/docs/4.7.3/admin/bookies.md new file mode 100644 index 00000000000..1b0427dae3c --- /dev/null +++ b/site/docs/4.7.3/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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + \ + \ + + ``` + + 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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + 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.7.3/admin/geo-replication.md b/site/docs/4.7.3/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.7.3/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.7.3/admin/http.md b/site/docs/4.7.3/admin/http.md new file mode 100644 index 00000000000..0097adc62b8 --- /dev/null +++ b/site/docs/4.7.3/admin/http.md @@ -0,0 +1,394 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 4 components: +1. Heartbeat: heartbeat for a specific bookie. +1. Config: doing the server configuration for a specific bookie. +1. Ledger: HTTP endpoints related to ledgers. +1. Bookie: HTTP endpoints related to bookies. +1. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.7.3/admin/metrics.md b/site/docs/4.7.3/admin/metrics.md new file mode 100644 index 00000000000..142df3dcd2d --- /dev/null +++ b/site/docs/4.7.3/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.prometheus.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.7.3/admin/perf.md b/site/docs/4.7.3/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.7.3/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.7.3/admin/placement.md b/site/docs/4.7.3/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.7.3/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.7.3/admin/upgrade.md b/site/docs/4.7.3/admin/upgrade.md new file mode 100644 index 00000000000..f35b8a43904 --- /dev/null +++ b/site/docs/4.7.3/admin/upgrade.md @@ -0,0 +1,183 @@ +--- +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.7.2 to 4.7.3 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.3. So you can follow the general upgrade sequence to upgrade from 4.7.2 to 4.7.3. + +### 4.7.1 to 4.7.2 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.2. So you can follow the general upgrade sequence to upgrade from 4.7.1 to 4.7.2. + +### 4.7.0 to 4.7.1 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.1. So you can follow the general upgrade sequence to upgrade from 4.7.0 to 4.7.1. + +### 4.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.7.3/api/distributedlog-api.md b/site/docs/4.7.3/api/distributedlog-api.md new file mode 100644 index 00000000000..27293028609 --- /dev/null +++ b/site/docs/4.7.3/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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.7.3/api/ledger-adv-api.md b/site/docs/4.7.3/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.7.3/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.7.3/api/ledger-api.md b/site/docs/4.7.3/api/ledger-api.md new file mode 100644 index 00000000000..5d69cb1b0fa --- /dev/null +++ b/site/docs/4.7.3/api/ledger-api.md @@ -0,0 +1,802 @@ +--- +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.7.3 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.7.3 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.7.3' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.7.3' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for +guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` diff --git a/site/docs/4.7.3/api/overview.md b/site/docs/4.7.3/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.7.3/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.7.3/deployment/dcos.md b/site/docs/4.7.3/deployment/dcos.md new file mode 100644 index 00000000000..69d22c888ab --- /dev/null +++ b/site/docs/4.7.3/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.7.3] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.7.3/deployment/kubernetes.md b/site/docs/4.7.3/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.7.3/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.7.3/deployment/manual.md b/site/docs/4.7.3/deployment/manual.md new file mode 100644 index 00000000000..daafd5556f5 --- /dev/null +++ b/site/docs/4.7.3/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.7.3/development/codebase.md b/site/docs/4.7.3/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.7.3/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.7.3/development/protocol.md b/site/docs/4.7.3/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.7.3/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.7.3/getting-started/concepts.md b/site/docs/4.7.3/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.7.3/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.7.3/getting-started/installation.md b/site/docs/4.7.3/getting-started/installation.md new file mode 100644 index 00000000000..9986cd8e043 --- /dev/null +++ b/site/docs/4.7.3/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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.7.3/getting-started/run-locally.md b/site/docs/4.7.3/getting-started/run-locally.md new file mode 100644 index 00000000000..edbfab9fda6 --- /dev/null +++ b/site/docs/4.7.3/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 +$ bookkeeper-server/bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.7.3/overview/overview.md b/site/docs/4.7.3/overview/overview.md new file mode 100644 index 00000000000..ccca3a48ece --- /dev/null +++ b/site/docs/4.7.3/overview/overview.md @@ -0,0 +1,58 @@ +--- +title: Apache BookKeeper™ 4.7.3 +--- + + +This documentation is for Apache BookKeeper™ version 4.7.3. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.7.3 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.7.3/overview/releaseNotesTemplate.md b/site/docs/4.7.3/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..9cec1b73542 --- /dev/null +++ b/site/docs/4.7.3/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.7.3 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.7.3. The technical details of this release are summarized +below. + +## Highlights + +[List the highlights] + +## Details + +[list to issues list] + diff --git a/site/docs/4.7.3/reference/cli.md b/site/docs/4.7.3/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.7.3/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.7.3/reference/config.md b/site/docs/4.7.3/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.7.3/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.7.3/reference/metrics.md b/site/docs/4.7.3/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.7.3/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.7.3/security/overview.md b/site/docs/4.7.3/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.7.3/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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.7.3/security/sasl.md b/site/docs/4.7.3/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.7.3/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.7.3/security/tls.md b/site/docs/4.7.3/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.7.3/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.7.3/security/zookeeper.md b/site/docs/4.7.3/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.7.3/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) diff --git a/site/releases.md b/site/releases.md index 72ac0c1cfd7..b89c1063127 100644 --- a/site/releases.md +++ b/site/releases.md @@ -64,6 +64,14 @@ Client Guide | API docs ## News +### 04 December, 2018 Release 4.7.3 available + +This is the 15th release of Apache BookKeeper! + +The 4.7.3 release is a bugfix release which fixes a bunch of issues reported from users of 4.7.2. + +See [BookKeeper 4.7.3 Release Notes](../docs/4.7.3/overview/releaseNotes) for details. + ### 22 November, 2018 Release 4.8.1 available This is the 14th release of Apache BookKeeper ! From bd00dab32559d3a86a58d560b4426b84f9f38d05 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 6 Dec 2018 05:16:50 -0800 Subject: [PATCH 0163/1642] Fix selectFromNetworkLocation in RackawareEnsemblePlacementPolicyImpl Descriptions of the changes in this PR: Since beginning, selectFromNetworkLocation(excludeRacks, excludeBookies,..) method kind of ignores predicate/ensemble passed to that method https://github.com/apache/bookkeeper/blob/branch-4.7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L694. This was kind of ok, because so far effectively this method is called from only one place - https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L612, which passes TruePredicate.INSTANCE and EnsembleForReplacementWithNoConstraints.INSTANCE. But it is not ideal to ignore those parameters in selectFromNetworkLocation(excludeRacks, excludeBookies,..), from future usage perspective. So passing the received predicate and ensemble to the underlying calls. Reviewers: Sijie Guo , Samuel Just This closes #1862 from reddycharan/fixrackaware --- .../RackawareEnsemblePlacementPolicyImpl.java | 3 +- .../TestRackawareEnsemblePlacementPolicy.java | 100 ++++++++++++++++++ 2 files changed, 101 insertions(+), 2 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 155229a6c65..578d49e7ed9 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 @@ -722,8 +722,7 @@ public BookieNode selectFromNetworkLocation(Set excludeRacks, } try { - return selectRandomInternal(knownNodes, 1, fullExclusionBookiesList, TruePredicate.INSTANCE, - EnsembleForReplacementWithNoConstraints.INSTANCE).get(0); + return selectRandomInternal(knownNodes, 1, fullExclusionBookiesList, predicate, ensemble).get(0); } catch (BKNotEnoughBookiesException e) { if (!fallbackToRandom) { LOG.error( 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 d80104456cd..e75e93fa864 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 @@ -41,6 +41,7 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; +import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.EnsembleForReplacementWithNoConstraints; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.TruePredicate; @@ -1176,6 +1177,105 @@ public void testSelectBookieFromNetworkLocAndExcludingRacks() throws Exception { || rackLocationNames[2].equals(bookieNode.getNetworkLocation())); } + @Test + public void testSelectBookieByExcludingRacksAndBookies() throws Exception { + repp.uninitalize(); + + int minNumRacksPerWriteQuorum = 4; + ClientConfiguration clientConf = new ClientConfiguration(conf); + clientConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorum); + // set enforceMinNumRacksPerWriteQuorum + clientConf.setEnforceMinNumRacksPerWriteQuorum(true); + /* + * Durability is enforced + * + * When durability is being enforced; we must not violate the predicate + * even when selecting a random bookie; as durability guarantee is not + * best effort; correctness is implied by it + */ + repp = new RackawareEnsemblePlacementPolicy(true); + repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE); + repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); + + int numOfRacks = 3; + int numOfBookiesPerRack = 5; + String[] rackLocationNames = new String[numOfRacks]; + List bookieSocketAddresses = new ArrayList(); + Map bookieRackMap = new HashMap(); + BookieSocketAddress bookieAddress; + + for (int i = 0; i < numOfRacks; i++) { + rackLocationNames[i] = "/default-region/r" + i; + for (int j = 0; j < numOfBookiesPerRack; j++) { + int index = i * numOfBookiesPerRack + j; + bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181); + StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), rackLocationNames[i]); + bookieSocketAddresses.add(bookieAddress); + bookieRackMap.put(bookieAddress, rackLocationNames[i]); + } + } + + repp.onClusterChanged(new HashSet(bookieSocketAddresses), + new HashSet()); + + Set excludeBookiesOfRackR0 = new HashSet(); + for (int i = 0; i < numOfBookiesPerRack; i++) { + excludeBookiesOfRackR0.add(bookieSocketAddresses.get(i)); + } + + Set excludeBookieNodesOfRackR0 = repp.convertBookiesToNodes(excludeBookiesOfRackR0); + + Set excludeRackR1 = new HashSet(); + excludeRackR1.add(rackLocationNames[1]); + + BookieNode nodeSelected; + nodeSelected = repp.selectFromNetworkLocation(excludeRackR1, excludeBookieNodesOfRackR0, TruePredicate.INSTANCE, + EnsembleForReplacementWithNoConstraints.INSTANCE, false); + assertEquals("BookieNode should be from Rack2", rackLocationNames[2], nodeSelected.getNetworkLocation()); + + try { + /* + * durability is enforced, so false predicate will reject all + * bookies. + */ + repp.selectFromNetworkLocation(excludeRackR1, excludeBookieNodesOfRackR0, (candidate, chosenBookies) -> { + return false; + }, EnsembleForReplacementWithNoConstraints.INSTANCE, false); + fail("Should get not enough bookies exception since we are using false predicate"); + } catch (BKNotEnoughBookiesException bnebe) { + // this is expected + } + + try { + /* + * Using ensemble which rejects all the nodes. + */ + repp.selectFromNetworkLocation(excludeRackR1, excludeBookieNodesOfRackR0, TruePredicate.INSTANCE, + new Ensemble() { + + @Override + public boolean addNode(BookieNode node) { + return false; + } + + @Override + public List toList() { + return null; + } + + @Override + public boolean validate() { + return false; + } + + }, false); + fail("Should get not enough bookies exception since ensemble rejects all the nodes"); + } catch (BKNotEnoughBookiesException bnebe) { + // this is expected + } + } + @Test public void testNewEnsembleWithMultipleRacks() throws Exception { BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.1", 3181); From 52c036446c10fbb91c2adb5094c5a582e4c10f0c Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 6 Dec 2018 14:20:22 +0100 Subject: [PATCH 0164/1642] changingEnsemble should be negated before calling unset success If the first pending add op is completed, but does not have the replaced bookie in its write set, callbacks are triggered straight away. Previously this would then hang forever, as the changingEnsemble would be true. This patch sets changingEnsemble to false before calling unsetSuccessAndSendWriteRequest so that if callbacks are triggered straight away, they can actually complete. It also moves the call to unsetSuccessAndSendWriteRequest outside of the metadataLock so that the callbacks don't run inside the lock. Reviewers: Sijie Guo This closes #1857 from ivankelly/block-order --- .../bookkeeper/client/LedgerHandle.java | 11 +++-- .../bookkeeper/client/HandleFailuresTest.java | 46 +++++++++++++++++++ 2 files changed, 54 insertions(+), 3 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 075788bed3e..6f0af27e179 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 @@ -1904,6 +1904,8 @@ void ensembleChangeLoop(List origEnsemble, Map newEnsemble = null; + Set replaced = null; synchronized (metadataLock) { if (!delayedWriteFailedBookies.isEmpty()) { Map toReplace = new HashMap<>(delayedWriteFailedBookies); @@ -1911,13 +1913,16 @@ void ensembleChangeLoop(List origEnsemble, Map newEnsemble = getCurrentEnsemble(); - Set replaced = EnsembleUtils.diffEnsemble(origEnsemble, newEnsemble); + newEnsemble = getCurrentEnsemble(); + replaced = EnsembleUtils.diffEnsemble(origEnsemble, newEnsemble); LOG.info("New Ensemble: {} for ledger: {}", newEnsemble, ledgerId); - unsetSuccessAndSendWriteRequest(newEnsemble, replaced); + changingEnsemble = false; } } + if (newEnsemble != null) { // unsetSuccess outside of lock + unsetSuccessAndSendWriteRequest(newEnsemble, replaced); + } } }, clientCtx.getMainWorkerPool().chooseThread(ledgerId)); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java index 9f95a94fbfe..1b8220c65d4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java @@ -447,4 +447,50 @@ public void testNoAddsAreCompletedWhileFailureHandlingInProgress() throws Except Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); } + + @Test + public void testHandleFailureBookieNotInWriteSet() throws Exception { + MockClientContext clientCtx = MockClientContext.create(); + Versioned md = ClientUtil.setupLedger(clientCtx, 10L, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + clientCtx.getMockRegistrationClient().addBookies(b4).get(); + + CompletableFuture b1Delay = new CompletableFuture<>(); + // Delay the first write to b1, then error it + clientCtx.getMockBookieClient().setPreWriteHook((bookie, ledgerId, entryId) -> { + if (bookie.equals(b1)) { + return b1Delay; + } else { + return FutureUtils.value(null); + } + }); + + CompletableFuture changeInProgress = new CompletableFuture<>(); + CompletableFuture blockEnsembleChange = new CompletableFuture<>(); + clientCtx.getMockLedgerManager().setPreWriteHook((ledgerId, metadata) -> { + changeInProgress.complete(null); + return blockEnsembleChange; + }); + + LedgerHandle lh = new LedgerHandle(clientCtx, 10L, md, BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + log.info("b2 should be enough to complete first add"); + lh.append("entry1".getBytes()); + + log.info("when b1 completes with failure, handleFailures should kick off"); + b1Delay.completeExceptionally(new BKException.BKWriteException()); + + log.info("write second entry, should have enough bookies, but blocks completion on failure handling"); + CompletableFuture e2 = lh.appendAsync("entry2".getBytes()); + changeInProgress.get(); + assertEventuallyTrue("e2 should eventually complete", () -> lh.pendingAddOps.peek().completed); + Assert.assertFalse("e2 shouldn't be completed to client", e2.isDone()); + blockEnsembleChange.complete(null); // allow ensemble change to continue + + log.info("e2 should complete"); + e2.get(10, TimeUnit.SECONDS); + } + } From 8e6cea6bd1d2ba77311e9248e8304084a05f14a8 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 6 Dec 2018 21:02:41 +0100 Subject: [PATCH 0165/1642] Ledger manager factories initialized with max metadata version In dce4fd4 the max metadata format version was put in the configuration which was then used to create ledger managers. This is not safe though, as it there are multiple paths which through which ledger manager factories are created and some may not see this configuration modification. For example, on a new cluster where there is no preexisting layout, the layout isn't written until after the ledger manager factory has been created. This patch changes LedgerManagerFactory#initialize to explicitly require that the max ledger metadata format is specified in the call. Master issue: #723 Reviewers: Enrico Olivelli This closes #1865 from ivankelly/explicit-max --- .../conf/AbstractConfiguration.java | 24 ------------------- .../AbstractHierarchicalLedgerManager.java | 5 ++-- .../meta/AbstractZkLedgerManager.java | 5 ++-- .../meta/AbstractZkLedgerManagerFactory.java | 16 +++++++------ .../bookkeeper/meta/FlatLedgerManager.java | 5 ++-- .../meta/FlatLedgerManagerFactory.java | 7 ++++-- .../meta/HierarchicalLedgerManager.java | 9 +++---- .../HierarchicalLedgerManagerFactory.java | 2 +- .../bookkeeper/meta/LedgerManagerFactory.java | 5 +++- .../meta/LegacyHierarchicalLedgerManager.java | 5 ++-- ...egacyHierarchicalLedgerManagerFactory.java | 7 ++++-- .../meta/LongHierarchicalLedgerManager.java | 5 ++-- .../LongHierarchicalLedgerManagerFactory.java | 2 +- .../meta/MSLedgerManagerFactory.java | 12 ++++++---- .../client/ParallelLedgerRecoveryTest.java | 7 ++++-- .../meta/AbstractZkLedgerManagerTest.java | 12 ++++++---- .../metadata/etcd/EtcdLedgerManager.java | 9 ++++--- .../etcd/EtcdLedgerManagerFactory.java | 9 +++---- .../metadata/etcd/EtcdMetadataDriverBase.java | 4 +++- .../metadata/etcd/EtcdLedgerManagerTest.java | 5 ++-- .../DistributedLogCoreShadedJarTest.java | 4 ++-- 21 files changed, 82 insertions(+), 77 deletions(-) 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 3057bfe65b6..7f00d09adb5 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 @@ -85,7 +85,6 @@ public abstract class AbstractConfiguration protected static final String STORE_SYSTEMTIME_AS_LEDGER_UNDERREPLICATED_MARK_TIME = "storeSystemTimeAsLedgerUnderreplicatedMarkTime"; protected static final String STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME = "storeSystemTimeAsLedgerCreationTime"; - protected static final String MAX_LEDGER_METADATA_FORMAT_VERSION = "maxLedgerMetadataFormatVersion"; // Metastore settings, only being used when LEDGER_MANAGER_FACTORY_CLASS is MSLedgerManagerFactory protected static final String METASTORE_IMPL_CLASS = "metastoreImplClass"; @@ -856,29 +855,6 @@ public boolean getStoreSystemTimeAsLedgerUnderreplicatedMarkTime() { return getBoolean(STORE_SYSTEMTIME_AS_LEDGER_UNDERREPLICATED_MARK_TIME, true); } - /** - * Set the maximum format version which should be used when serializing ledger metadata. - * Setting the maximum format allows us to ensure that all clients running against a cluster - * will be able to read back all written metadata. - * Normally this is set when loading the layout from the metadata store. Users should not - * set this directly. - */ - public T setMaxLedgerMetadataFormatVersion(int maxVersion) { - setProperty(MAX_LEDGER_METADATA_FORMAT_VERSION, maxVersion); - return getThis(); - } - - /** - * Get the maximum format version which should be used when serializing ledger metadata. - * The default is 2, as that was the current version when this functionallity was introduced. - * - * @see #setMaxLedgerMetadataFormatVersion(int) - * @return the maximum format version with which to serialize metadata. - */ - public int getMaxLedgerMetadataFormatVersion() { - return getInteger(MAX_LEDGER_METADATA_FORMAT_VERSION, 2); - } - /** * Whether to preserve MDC for tasks in Executor. * diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java index 1aa8e151d2c..1a0ce2eb5f5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java @@ -49,8 +49,9 @@ public abstract class AbstractHierarchicalLedgerManager extends AbstractZkLedger * @param zk * ZooKeeper Client Handle */ - public AbstractHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { - super(conf, zk); + public AbstractHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk, + int maxLedgerMetadataFormatVersion) { + super(conf, zk, maxLedgerMetadataFormatVersion); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 97413b7bffb..1c901542c32 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -159,8 +159,9 @@ private void handleMetadata(Versioned result, Throwable exceptio * @param zk * ZooKeeper Client Handle */ - protected AbstractZkLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { - this.serDe = new LedgerMetadataSerDe(conf.getMaxLedgerMetadataFormatVersion()); + protected AbstractZkLedgerManager(AbstractConfiguration conf, ZooKeeper zk, + int maxLedgerMetadataFormatVersion) { + this.serDe = new LedgerMetadataSerDe(maxLedgerMetadataFormatVersion); this.conf = conf; this.zk = zk; this.ledgerRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java index 4bb2fb141d3..be95a53886c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java @@ -160,8 +160,8 @@ public static LedgerManagerFactory newLedgerManagerFactory( // if layoutManager is null, return the default ledger manager if (layoutManager == null) { - return new FlatLedgerManagerFactory() - .initialize(conf, null, FlatLedgerManagerFactory.CUR_VERSION); + return new FlatLedgerManagerFactory().initialize(conf, null, + FlatLedgerManagerFactory.CUR_VERSION, LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } LedgerManagerFactory lmFactory; @@ -172,13 +172,13 @@ public static LedgerManagerFactory newLedgerManagerFactory( if (layout == null) { // no existing layout lmFactory = createNewLMFactory(conf, layoutManager, factoryClass); - return lmFactory - .initialize(conf, layoutManager, lmFactory.getCurrentVersion()); + return lmFactory.initialize(conf, layoutManager, + lmFactory.getCurrentVersion(), + LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } if (log.isDebugEnabled()) { log.debug("read ledger layout {}", layout); } - conf.setMaxLedgerMetadataFormatVersion(layout.getMaxLedgerMetadataFormatVersion()); // there is existing layout, we need to look into the layout. // handle pre V2 layout @@ -198,7 +198,8 @@ public static LedgerManagerFactory newLedgerManagerFactory( } else { throw new IOException("Unknown ledger manager type: " + lmType); } - return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion()); + return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion(), + layout.getMaxLedgerMetadataFormatVersion()); } // handle V2 layout case @@ -228,7 +229,8 @@ public static LedgerManagerFactory newLedgerManagerFactory( } // instantiate a factory lmFactory = ReflectionUtils.newInstance(factoryClass); - return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion()); + return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion(), + layout.getMaxLedgerMetadataFormatVersion()); } private static String normalizedLedgerManagerFactoryClassName(String factoryClass, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java index 7ee2e2289ea..115ffe3f8c3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java @@ -55,8 +55,9 @@ class FlatLedgerManager extends AbstractZkLedgerManager { * ZooKeeper Client Handle * @throws IOException when version is not compatible */ - public FlatLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { - super(conf, zk); + public FlatLedgerManager(AbstractConfiguration conf, ZooKeeper zk, + int maxLedgerMetadataFormatVersion) { + super(conf, zk, maxLedgerMetadataFormatVersion); ledgerPrefix = ledgerRootPath + "/" + StringUtils.LEDGER_NODE_PREFIX; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java index 19ac418eeae..5c92ba12d93 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java @@ -41,6 +41,7 @@ public class FlatLedgerManagerFactory extends AbstractZkLedgerManagerFactory { public static final int CUR_VERSION = 1; AbstractConfiguration conf; + private int maxLedgerMetadataFormatVersion; @Override public int getCurrentVersion() { @@ -50,7 +51,8 @@ public int getCurrentVersion() { @Override public LedgerManagerFactory initialize(final AbstractConfiguration conf, final LayoutManager layoutManager, - final int factoryVersion) + final int factoryVersion, + int maxLedgerMetadataFormatVersion) throws IOException { checkArgument(layoutManager == null || layoutManager instanceof ZkLayoutManager); @@ -61,6 +63,7 @@ public LedgerManagerFactory initialize(final AbstractConfiguration conf, this.conf = conf; this.zk = layoutManager == null ? null : ((ZkLayoutManager) layoutManager).getZk(); + this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; return this; } @@ -79,7 +82,7 @@ public LedgerIdGenerator newLedgerIdGenerator() { @Override public LedgerManager newLedgerManager() { - return new FlatLedgerManager(conf, zk); + return new FlatLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java index 946ed2a5d32..c4f211a1ab2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java @@ -46,10 +46,11 @@ class HierarchicalLedgerManager extends AbstractHierarchicalLedgerManager { LegacyHierarchicalLedgerManager legacyLM; LongHierarchicalLedgerManager longLM; - public HierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { - super(conf, zk); - legacyLM = new LegacyHierarchicalLedgerManager(conf, zk); - longLM = new LongHierarchicalLedgerManager (conf, zk); + public HierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk, + int maxLedgerMetadataFormatVersion) { + super(conf, zk, maxLedgerMetadataFormatVersion); + legacyLM = new LegacyHierarchicalLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); + longLM = new LongHierarchicalLedgerManager (conf, zk, maxLedgerMetadataFormatVersion); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManagerFactory.java index 4a9d6cfbf6e..c4f0f4f4d62 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManagerFactory.java @@ -42,6 +42,6 @@ public LedgerIdGenerator newLedgerIdGenerator() { @Override public LedgerManager newLedgerManager() { - return new HierarchicalLedgerManager(conf, zk); + return new HierarchicalLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java index 80d3a6526f9..eb4e0f446a6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java @@ -43,12 +43,15 @@ public interface LedgerManagerFactory extends AutoCloseable { * Layout manager used for initialize ledger manager factory * @param factoryVersion * What version used to initialize factory. + * @param maxLedgerMetadataFormatVersion + * Maximum format version for ledger metadata. * @return ledger manager factory instance * @throws IOException when fail to initialize the factory. */ LedgerManagerFactory initialize(AbstractConfiguration conf, LayoutManager layoutManager, - int factoryVersion) + int factoryVersion, + int maxLedgerMetadataFormatVersion) throws IOException; /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java index 76ecc9d68de..b5065a872a0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java @@ -69,8 +69,9 @@ protected StringBuilder initialValue() { * @param zk * ZooKeeper Client Handle */ - public LegacyHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { - super(conf, zk); + public LegacyHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk, + int maxLedgerMetadataFormatVersion) { + super(conf, zk, maxLedgerMetadataFormatVersion); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManagerFactory.java index 91579735b05..bd5c57927c2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManagerFactory.java @@ -38,6 +38,7 @@ public class LegacyHierarchicalLedgerManagerFactory extends AbstractZkLedgerMana public static final int CUR_VERSION = 1; AbstractConfiguration conf; + int maxLedgerMetadataFormatVersion; @Override public int getCurrentVersion() { @@ -47,7 +48,8 @@ public int getCurrentVersion() { @Override public LedgerManagerFactory initialize(final AbstractConfiguration conf, final LayoutManager lm, - final int factoryVersion) + final int factoryVersion, + int maxLedgerMetadataFormatVersion) throws IOException { checkArgument(lm instanceof ZkLayoutManager); @@ -58,6 +60,7 @@ public LedgerManagerFactory initialize(final AbstractConfiguration conf, + factoryVersion); } this.conf = conf; + this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; this.zk = zkLayoutManager.getZk(); return this; } @@ -80,7 +83,7 @@ public LedgerIdGenerator newLedgerIdGenerator() { @Override public LedgerManager newLedgerManager() { - return new LegacyHierarchicalLedgerManager(conf, zk); + return new LegacyHierarchicalLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java index 2e69e90a5c1..e464ed04a18 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java @@ -70,8 +70,9 @@ class LongHierarchicalLedgerManager extends AbstractHierarchicalLedgerManager { * @param zk * ZooKeeper Client Handle */ - public LongHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { - super(conf, zk); + public LongHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk, + int maxLedgerMetadataFormatVersion) { + super(conf, zk, maxLedgerMetadataFormatVersion); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManagerFactory.java index 93ad9ddc681..4c06cc48269 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManagerFactory.java @@ -26,7 +26,7 @@ public class LongHierarchicalLedgerManagerFactory extends HierarchicalLedgerMana @Override public LedgerManager newLedgerManager() { - return new LongHierarchicalLedgerManager(conf, zk); + return new LongHierarchicalLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index 38c6816dd6b..e10f2743c68 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -96,6 +96,7 @@ public class MSLedgerManagerFactory extends AbstractZkLedgerManagerFactory { public static final String META_FIELD = ".META"; AbstractConfiguration conf; + private int maxLedgerMetadataFormatVersion; MetaStore metastore; @Override @@ -106,7 +107,8 @@ public int getCurrentVersion() { @Override public LedgerManagerFactory initialize(final AbstractConfiguration conf, final LayoutManager layoutManager, - final int factoryVersion) throws IOException { + final int factoryVersion, + int maxLedgerMetadataFormatVersion) throws IOException { checkArgument(layoutManager instanceof ZkLayoutManager); ZkLayoutManager zkLayoutManager = (ZkLayoutManager) layoutManager; @@ -115,6 +117,7 @@ public LedgerManagerFactory initialize(final AbstractConfiguration conf, } this.conf = conf; this.zk = zkLayoutManager.getZk(); + this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; // load metadata store String msName = conf.getMetastoreImplClass(); @@ -280,11 +283,12 @@ private void handleMetadata(Versioned metadata, Throwable except } } - MsLedgerManager(final AbstractConfiguration conf, final ZooKeeper zk, final MetaStore metastore) { + MsLedgerManager(final AbstractConfiguration conf, final ZooKeeper zk, final MetaStore metastore, + int maxLedgerMetadataFormatVersion) { this.conf = conf; this.zk = zk; this.metastore = metastore; - this.serDe = new LedgerMetadataSerDe(conf.getMaxLedgerMetadataFormatVersion()); + this.serDe = new LedgerMetadataSerDe(maxLedgerMetadataFormatVersion); try { ledgerTable = metastore.createScannableTable(TABLE_NAME); @@ -651,7 +655,7 @@ public static boolean isSpecialZnode(String znode) { @Override public LedgerManager newLedgerManager() { - return new MsLedgerManager(conf, zk, metastore); + return new MsLedgerManager(conf, zk, metastore, maxLedgerMetadataFormatVersion); } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index ab966fb4ae2..1f9656b94d6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -50,6 +50,7 @@ import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.exceptions.Code; import org.apache.bookkeeper.meta.exceptions.MetadataException; @@ -184,7 +185,8 @@ public synchronized LedgerManagerFactory getLedgerManagerFactory() throws Metada if (null == lmFactory) { try { lmFactory = new TestLedgerManagerFactory() - .initialize(conf, layoutManager, TestLedgerManagerFactory.CUR_VERSION); + .initialize(conf, layoutManager, TestLedgerManagerFactory.CUR_VERSION, + LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } catch (IOException e) { throw new MetadataException(Code.METADATA_SERVICE_ERROR, e); } @@ -200,7 +202,8 @@ public synchronized LedgerManagerFactory getLedgerManagerFactory() throws Metada if (null == lmFactory) { try { lmFactory = new TestLedgerManagerFactory() - .initialize(conf, layoutManager, TestLedgerManagerFactory.CUR_VERSION); + .initialize(conf, layoutManager, TestLedgerManagerFactory.CUR_VERSION, + LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } catch (IOException e) { throw new MetadataException(Code.METADATA_SERVICE_ERROR, e); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 20a8d5134ac..620c3b2e004 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -110,9 +110,9 @@ public void setup() throws Exception { this.conf = new ClientConfiguration(); this.ledgerManager = mock( - AbstractZkLedgerManager.class, - withSettings() - .useConstructor(conf, mockZk) + AbstractZkLedgerManager.class, + withSettings() + .useConstructor(conf, mockZk, LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION) .defaultAnswer(CALLS_REAL_METHODS)); List ensemble = Lists.newArrayList( new BookieSocketAddress("192.0.2.1", 3181), @@ -303,13 +303,15 @@ public void testRemoveLedgerMetadataException() throws Exception { @Test public void testRemoveLedgerMetadataHierarchical() throws Exception { - HierarchicalLedgerManager hlm = new HierarchicalLedgerManager(conf, mockZk); + HierarchicalLedgerManager hlm = new HierarchicalLedgerManager(conf, mockZk, + LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); testRemoveLedgerMetadataHierarchicalLedgerManager(hlm); } @Test public void testRemoveLedgerMetadataLongHierarchical() throws Exception { - LongHierarchicalLedgerManager hlm = new LongHierarchicalLedgerManager(conf, mockZk); + LongHierarchicalLedgerManager hlm = new LongHierarchicalLedgerManager(conf, mockZk, + LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); testRemoveLedgerMetadataHierarchicalLedgerManager(hlm); } diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index e20a9ba146f..ffbbc08df0c 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -45,7 +45,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.api.LedgerMetadata; -import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.metadata.etcd.helpers.KeyIterator; @@ -79,14 +78,14 @@ class EtcdLedgerManager implements LedgerManager { private volatile boolean closed = false; - EtcdLedgerManager(AbstractConfiguration conf, - Client client, - String scope) { + EtcdLedgerManager(Client client, + String scope, + int maxLedgerMetadataFormatVersion) { this.client = client; this.kvClient = client.getKVClient(); this.scope = scope; this.watchClient = new EtcdWatchClient(client); - this.serDe = new LedgerMetadataSerDe(conf.getMaxLedgerMetadataFormatVersion()); + this.serDe = new LedgerMetadataSerDe(maxLedgerMetadataFormatVersion); this.ledgerMetadataFunction = bs -> { try { diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java index 9c19df07ddb..0bc16d815cd 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java @@ -42,7 +42,7 @@ class EtcdLedgerManagerFactory implements LedgerManagerFactory { private String scope; private Client client; - private AbstractConfiguration conf; + private int maxLedgerMetadataFormatVersion; @Override public int getCurrentVersion() { @@ -52,7 +52,8 @@ public int getCurrentVersion() { @Override public LedgerManagerFactory initialize(AbstractConfiguration conf, LayoutManager layoutManager, - int factoryVersion) throws IOException { + int factoryVersion, + int maxLedgerMetadataFormatVersion) throws IOException { checkArgument(layoutManager instanceof EtcdLayoutManager); EtcdLayoutManager etcdLayoutManager = (EtcdLayoutManager) layoutManager; @@ -67,7 +68,7 @@ public LedgerManagerFactory initialize(AbstractConfiguration conf, throw new IOException("Invalid metadata service uri", e); } this.client = etcdLayoutManager.getClient(); - this.conf = conf; + this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; return this; } @@ -84,7 +85,7 @@ public LedgerIdGenerator newLedgerIdGenerator() { @Override public LedgerManager newLedgerManager() { - return new EtcdLedgerManager(conf, client, scope); + return new EtcdLedgerManager(client, scope, maxLedgerMetadataFormatVersion); } @Override diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java index 47b034c00ef..cbd6f321430 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java @@ -27,6 +27,7 @@ import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.meta.LayoutManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.exceptions.Code; import org.apache.bookkeeper.meta.exceptions.MetadataException; import org.apache.bookkeeper.stats.StatsLogger; @@ -106,7 +107,8 @@ public synchronized LedgerManagerFactory getLedgerManagerFactory() if (null == lmFactory) { try { lmFactory = new EtcdLedgerManagerFactory(); - lmFactory.initialize(conf, layoutManager, EtcdLedgerManagerFactory.VERSION); + lmFactory.initialize(conf, layoutManager, EtcdLedgerManagerFactory.VERSION, + LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } catch (IOException ioe) { throw new MetadataException( Code.METADATA_SERVICE_ERROR, "Failed to initialize ledger manager factory", ioe); diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index 27dc84c24ac..30877f7d5a5 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -53,9 +53,9 @@ import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.metadata.etcd.helpers.ValueStream; import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -82,7 +82,8 @@ public class EtcdLedgerManagerTest extends EtcdTestBase { public void setUp() throws Exception { super.setUp(); this.scope = RandomStringUtils.randomAlphabetic(8); - this.lm = new EtcdLedgerManager(new ClientConfiguration(), etcdClient, scope); + this.lm = new EtcdLedgerManager(etcdClient, scope, + LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); } @Override diff --git a/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java b/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java index 6432c303a28..26230ec7a0c 100644 --- a/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java +++ b/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java @@ -179,7 +179,7 @@ private void testShadeLedgerManagerFactoryAllowed(String factoryClassName, when(manager.readLedgerLayout()).thenReturn(layout); LedgerManagerFactory factory = mock(LedgerManagerFactory.class); - when(factory.initialize(any(AbstractConfiguration.class), same(manager), anyInt())) + when(factory.initialize(any(AbstractConfiguration.class), same(manager), anyInt(), anyInt())) .thenReturn(factory); PowerMockito.mockStatic(ReflectionUtils.class); when(ReflectionUtils.newInstance(any(Class.class))) @@ -191,7 +191,7 @@ private void testShadeLedgerManagerFactoryAllowed(String factoryClassName, if (allowShaded) { assertSame(factory, result); verify(factory, times(1)) - .initialize(any(AbstractConfiguration.class), same(manager), anyInt()); + .initialize(any(AbstractConfiguration.class), same(manager), anyInt(), anyInt()); } else { fail("Should fail to instantiate ledger manager factory if allowShaded is false"); } From 36d60f11a2951dca674d216e466bb52a3952a44f Mon Sep 17 00:00:00 2001 From: Samuel Just Date: Tue, 11 Dec 2018 12:43:25 -0800 Subject: [PATCH 0166/1642] ISSUE #1770: Add local checker for Sorted/InterleavedLedgerStorage The main goal of this patch is the ScrubberService LifecycleComponent which runs in the background periodically verifying the internal consistency of the LedgerStorage. To get that to work: - LedgerStorage now has a localConsistencyCheck call with implementations in Interleaved and Sorted. - In service of that implementation, LedgerCache gains an interface for iterating safely over the entries of a ledger with a way of handling concurrently modified or deleted ledgers with corresponding modifications to LedgerEntryPage for detecting deletion. - EntryLogger has been refactored to permit checking the correctness (and throwing a descriptive exception in case of a problem) of an entry without actually reading it for use within localConsistencyCheck. - The two mechanisms for iterating over a ledger's entries in BookieShell have both been replaced with the new single implementation (InterleavedLedgerStorageTest now has a test checking the output of the affected command.) - Misc changes to *LogCompactor to support tests in InterleavedLedgerStorageTest. Because the consistency check needs to run in the background and hold LEP instances potentially for a relatively long time, a delete may overlap with the scan of an LEP page. As part of this patch, IndexInMemPageMgr and LedgerEntryPage now permit an LEP to be marked deleted and not added back to the set of free pages until released. This patch also adds an option to run the checker on startup (defaults to false). (bug W-5188823) (bug W-5153309) (rev cguttapalem) Signed-off-by: Samuel Just Reviewers: Enrico Olivelli , Sijie Guo This closes #1819 from athanatos/forupstream/wip-1770, closes #1770 --- .../bookie/AbstractLogCompactor.java | 13 +- .../bookie/BookKeeperServerStats.java | 4 + .../org/apache/bookkeeper/bookie/Bookie.java | 88 ++++- .../apache/bookkeeper/bookie/BookieShell.java | 314 +++++++----------- .../bookie/BufferedChannelBase.java | 8 +- .../bookie/BufferedReadChannel.java | 2 +- .../bookkeeper/bookie/EntryLogCompactor.java | 15 +- .../apache/bookkeeper/bookie/EntryLogger.java | 165 +++++++-- .../bookie/GarbageCollectorThread.java | 17 +- .../bookkeeper/bookie/IndexInMemPageMgr.java | 117 +++++-- .../bookie/IndexPersistenceMgr.java | 20 +- .../bookie/InterleavedLedgerStorage.java | 109 ++++++ .../InterleavedStorageRegenerateIndexOp.java | 8 + .../apache/bookkeeper/bookie/LedgerCache.java | 54 ++- .../bookkeeper/bookie/LedgerCacheImpl.java | 10 + .../bookkeeper/bookie/LedgerEntryPage.java | 44 ++- .../bookkeeper/bookie/LedgerStorage.java | 43 +++ .../bookie/ReadOnlyEntryLogger.java | 4 +- .../bookkeeper/bookie/ScrubberStats.java | 33 ++ .../bookie/SortedLedgerStorage.java | 8 + .../TransactionalEntryLogCompactor.java | 17 +- .../bookie/storage/ldb/DbLedgerStorage.java | 6 +- .../ldb/SingleDirectoryDbLedgerStorage.java | 33 +- .../bookkeeper/conf/ServerConfiguration.java | 57 ++++ .../org/apache/bookkeeper/server/Main.java | 9 + .../server/service/ScrubberService.java | 145 ++++++++ .../bookkeeper/bookie/CompactionTest.java | 11 +- .../bookie/InterleavedLedgerStorageTest.java | 303 +++++++++++++++++ 28 files changed, 1340 insertions(+), 317 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScrubberStats.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/ScrubberService.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/AbstractLogCompactor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/AbstractLogCompactor.java index 8f190a3649c..57ec8978cc0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/AbstractLogCompactor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/AbstractLogCompactor.java @@ -32,12 +32,17 @@ public abstract class AbstractLogCompactor { protected final ServerConfiguration conf; protected final Throttler throttler; - protected final GarbageCollectorThread gcThread; - public AbstractLogCompactor(GarbageCollectorThread gcThread) { - this.gcThread = gcThread; - this.conf = gcThread.conf; + interface LogRemovalListener { + void removeEntryLog(long logToRemove); + } + + protected final LogRemovalListener logRemovalListener; + + public AbstractLogCompactor(ServerConfiguration conf, LogRemovalListener logRemovalListener) { + this.conf = conf; this.throttler = new Throttler(conf); + this.logRemovalListener = logRemovalListener; } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java index 736d34122c7..69d0eda8661 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java @@ -108,6 +108,10 @@ public interface BookKeeperServerStats { // Ledger Storage Stats String STORAGE_GET_OFFSET = "STORAGE_GET_OFFSET"; String STORAGE_GET_ENTRY = "STORAGE_GET_ENTRY"; + + // Ledger Storage Scrub Stats + String STORAGE_SCRUB_PAGES_SCANNED = "STORAGE_SCRUB_PAGES_SCANNED"; + // Ledger Cache Stats String LEDGER_CACHE_READ_PAGE = "LEDGER_CACHE_READ_PAGE"; // SkipList Stats 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 ffb92ed2d76..fd352d8e71c 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 @@ -47,6 +47,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -602,6 +603,66 @@ public Bookie(ServerConfiguration conf) this(conf, NullStatsLogger.INSTANCE); } + private static LedgerStorage buildLedgerStorage(ServerConfiguration conf) throws IOException { + // Instantiate the ledger storage implementation + String ledgerStorageClass = conf.getLedgerStorageClass(); + LOG.info("Using ledger storage: {}", ledgerStorageClass); + return LedgerStorageFactory.createLedgerStorage(ledgerStorageClass); + } + + /** + * Initialize LedgerStorage instance without checkpointing for use within the shell + * and other RO users. ledgerStorage must not have already been initialized. + * + *

            The caller is responsible for disposing of the ledgerStorage object. + * + * @param conf Bookie config. + * @param ledgerStorage Instance to initialize. + * @return Passed ledgerStorage instance + * @throws IOException + */ + static LedgerStorage mountLedgerStorageOffline( + ServerConfiguration conf, + LedgerStorage ledgerStorage) throws IOException { + StatsLogger statsLogger = NullStatsLogger.INSTANCE; + DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()); + + LedgerDirsManager ledgerDirsManager = createLedgerDirsManager( + conf, diskChecker, statsLogger.scope(LD_LEDGER_SCOPE)); + LedgerDirsManager indexDirsManager = createIndexDirsManager( + conf, diskChecker, statsLogger.scope(LD_INDEX_SCOPE), ledgerDirsManager); + + if (null == ledgerStorage) { + ledgerStorage = buildLedgerStorage(conf); + } + + CheckpointSource checkpointSource = new CheckpointSource() { + @Override + public Checkpoint newCheckpoint() { + return Checkpoint.MAX; + } + + @Override + public void checkpointComplete(Checkpoint checkpoint, boolean compact) + throws IOException { + } + }; + + Checkpointer checkpointer = Checkpointer.NULL; + + ledgerStorage.initialize( + conf, + null, + ledgerDirsManager, + indexDirsManager, + null, + checkpointSource, + checkpointer, + statsLogger); + + return ledgerStorage; + } + public Bookie(ServerConfiguration conf, StatsLogger statsLogger) throws IOException, InterruptedException, BookieException { super("Bookie-" + conf.getBookiePort()); @@ -677,10 +738,7 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger) this.entryLogPerLedgerEnabled = conf.isEntryLogPerLedgerEnabled(); CheckpointSource checkpointSource = new CheckpointSourceList(journals); - // Instantiate the ledger storage implementation - String ledgerStorageClass = conf.getLedgerStorageClass(); - LOG.info("Using ledger storage: {}", ledgerStorageClass); - ledgerStorage = LedgerStorageFactory.createLedgerStorage(ledgerStorageClass); + ledgerStorage = buildLedgerStorage(conf); boolean isDbLedgerStorage = ledgerStorage instanceof DbLedgerStorage; @@ -871,7 +929,29 @@ public synchronized void start() { } catch (ExecutionException e) { LOG.error("Error on executing a fully flush after replaying journals."); shutdown(ExitCode.BOOKIE_EXCEPTION); + return; + } + + if (conf.isLocalConsistencyCheckOnStartup()) { + LOG.info("Running local consistency check on startup prior to accepting IO."); + List errors = null; + try { + errors = ledgerStorage.localConsistencyCheck(Optional.empty()); + } catch (IOException e) { + LOG.error("Got a fatal exception while checking store", e); + shutdown(ExitCode.BOOKIE_EXCEPTION); + return; + } + if (errors != null && errors.size() > 0) { + LOG.error("Bookie failed local consistency check:"); + for (LedgerStorage.DetectedInconsistency error : errors) { + LOG.error("Ledger {}, entry {}: ", error.getLedgerId(), error.getEntryId(), error.getException()); + } + shutdown(ExitCode.BOOKIE_EXCEPTION); + return; + } } + LOG.info("Finished reading journal, starting bookie"); 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 9c6180b63e4..083a66159c4 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 @@ -26,7 +26,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.util.concurrent.UncheckedExecutionException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; @@ -35,7 +34,6 @@ import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.Serializable; import java.math.RoundingMode; @@ -144,6 +142,7 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.mutable.MutableBoolean; +import org.apache.commons.lang.mutable.MutableLong; import org.apache.commons.lang3.ArrayUtils; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.AsyncCallback.VoidCallback; @@ -205,6 +204,7 @@ public class BookieShell implements Tool { static final String CMD_GENERATE_COOKIE = "cookie_generate"; static final String CMD_HELP = "help"; + static final String CMD_LOCALCONSISTENCYCHECK = "localconsistencycheck"; final ServerConfiguration bkConf = new ServerConfiguration(); File[] indexDirectories; @@ -239,6 +239,14 @@ public interface Command { void printUsage(); } + void printInfoLine(String s) { + System.out.println(s); + } + + void printErrorLine(String s) { + System.err.println(s); + } + abstract class MyCommand implements Command { abstract Options getOptions(); @@ -717,7 +725,7 @@ class LedgerCmd extends MyCommand { public int runCmd(CommandLine cmdLine) throws Exception { String[] leftArgs = cmdLine.getArgs(); if (leftArgs.length <= 0) { - System.err.println("ERROR: missing ledger id"); + printErrorLine("ERROR: missing ledger id"); printUsage(); return -1; } @@ -730,7 +738,7 @@ public int runCmd(CommandLine cmdLine) throws Exception { try { ledgerId = ledgerIdFormatter.readLedgerId(leftArgs[0]); } catch (IllegalArgumentException iae) { - System.err.println("ERROR: invalid ledger id " + leftArgs[0]); + printErrorLine("ERROR: invalid ledger id " + leftArgs[0]); printUsage(); return -1; } @@ -739,19 +747,69 @@ public int runCmd(CommandLine cmdLine) throws Exception { // dump ledger info try { DbLedgerStorage.readLedgerIndexEntries(ledgerId, bkConf, - (currentEntry, entryLogId, position) -> System.out.println( + (currentEntry, entryLogId, position) -> printInfoLine( "entry " + currentEntry + "\t:\t(log: " + entryLogId + ", pos: " + position + ")")); } catch (IOException e) { System.err.printf("ERROR: initializing dbLedgerStorage %s", e.getMessage()); return -1; } - } else { + } else if ((bkConf.getLedgerStorageClass().equals(SortedLedgerStorage.class.getName()) + || bkConf.getLedgerStorageClass().equals(InterleavedLedgerStorage.class.getName()))) { + ServerConfiguration conf = new ServerConfiguration(bkConf); + InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); + Bookie.mountLedgerStorageOffline(conf, interleavedStorage); + if (printMeta) { // print meta - readLedgerMeta(ledgerId); + printInfoLine("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " ====="); + LedgerCache.LedgerIndexMetadata meta = interleavedStorage.readLedgerIndexMetadata(ledgerId); + printInfoLine("master key : " + meta.getMasterKeyHex()); + + long size = meta.size; + if (size % 8 == 0) { + printInfoLine("size : " + size); + } else { + printInfoLine("size : " + size + + " (not aligned with 8, may be corrupted or under flushing now)"); + } + + printInfoLine("entries : " + (size / 8)); + printInfoLine("isFenced : " + meta.fenced); + } + + try { + // dump ledger info + printInfoLine("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " ====="); + for (LedgerCache.PageEntries page : interleavedStorage.getIndexEntries(ledgerId)) { + final MutableLong curEntry = new MutableLong(page.getFirstEntry()); + try (LedgerEntryPage lep = page.getLEP()){ + lep.getEntries((entry, offset) -> { + while (curEntry.longValue() < entry) { + printInfoLine("entry " + curEntry + "\t:\tN/A"); + curEntry.increment(); + } + long entryLogId = offset >> 32L; + long pos = offset & 0xffffffffL; + printInfoLine("entry " + curEntry + "\t:\t(log:" + entryLogId + ", pos: " + pos + ")"); + curEntry.increment(); + return true; + }); + } catch (IOException ie) { + printInfoLine("Failed to read index page @ " + page.getFirstEntry() + + ", the index file may be corrupted : " + + ie.getMessage()); + return 1; + } + + while (curEntry.longValue() < page.getLastEntry()) { + printInfoLine("entry " + curEntry + "\t:\tN/A"); + curEntry.increment(); + } + } + } catch (IOException ie) { + LOG.error("Failed to read index page"); + return 1; } - // dump ledger info - readLedgerIndexEntries(ledgerId); } return 0; @@ -1178,6 +1236,51 @@ Options getOptions() { } } + /** + * Print the metadata for a ledger. + */ + class LocalConsistencyCheck extends MyCommand { + Options lOpts = new Options(); + + LocalConsistencyCheck() { + super(CMD_LOCALCONSISTENCYCHECK); + } + + @Override + public int runCmd(CommandLine cmdLine) throws Exception { + LOG.info("=== Performing local consistency check ==="); + ServerConfiguration conf = new ServerConfiguration(bkConf); + LedgerStorage ledgerStorage = Bookie.mountLedgerStorageOffline(conf, null); + List errors = ledgerStorage.localConsistencyCheck( + java.util.Optional.empty()); + if (errors.size() > 0) { + LOG.info("=== Check returned errors: ==="); + for (LedgerStorage.DetectedInconsistency error : errors) { + LOG.error("Ledger {}, entry {}: ", error.getLedgerId(), error.getEntryId(), error.getException()); + } + return 1; + } else { + LOG.info("=== Check passed ==="); + return 0; + } + } + + @Override + String getDescription() { + return "Validate Ledger Storage internal metadata"; + } + + @Override + String getUsage() { + return "localconsistencycheck"; + } + + @Override + Options getOptions() { + return lOpts; + } + } + /** * Simple test to create a ledger and write to it. */ @@ -2626,41 +2729,12 @@ String getUsage() { int runCmd(CommandLine cmdLine) throws Exception { LOG.info("=== Converting to DbLedgerStorage ==="); ServerConfiguration conf = new ServerConfiguration(bkConf); - LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(bkConf, bkConf.getLedgerDirs(), - new DiskChecker(bkConf.getDiskUsageThreshold(), bkConf.getDiskUsageWarnThreshold())); - LedgerDirsManager ledgerIndexManager = new LedgerDirsManager(bkConf, bkConf.getLedgerDirs(), - new DiskChecker(bkConf.getDiskUsageThreshold(), bkConf.getDiskUsageWarnThreshold())); InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); - DbLedgerStorage dbStorage = new DbLedgerStorage(); + Bookie.mountLedgerStorageOffline(conf, interleavedStorage); - CheckpointSource checkpointSource = new CheckpointSource() { - @Override - public Checkpoint newCheckpoint() { - return Checkpoint.MAX; - } - - @Override - public void checkpointComplete(Checkpoint checkpoint, boolean compact) - throws IOException { - } - }; - Checkpointer checkpointer = new Checkpointer() { - @Override - public void startCheckpoint(Checkpoint checkpoint) { - // No-op - } - - @Override - public void start() { - // no-op - } - }; - - interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager, - null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE); - dbStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager, null, - checkpointSource, checkpointer, NullStatsLogger.INSTANCE); + DbLedgerStorage dbStorage = new DbLedgerStorage(); + Bookie.mountLedgerStorageOffline(conf, dbStorage); int convertedLedgers = 0; for (long ledgerId : interleavedStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)) { @@ -2668,13 +2742,13 @@ public void start() { LOG.debug("Converting ledger {}", ledgerIdFormatter.formatLedgerId(ledgerId)); } - FileInfo fi = getFileInfo(ledgerId); + LedgerCache.LedgerIndexMetadata fi = interleavedStorage.readLedgerIndexMetadata(ledgerId); - Iterable> entries = getLedgerIndexEntries(ledgerId); + LedgerCache.PageEntriesIterable pages = interleavedStorage.getIndexEntries(ledgerId); - long numberOfEntries = dbStorage.addLedgerToIndex(ledgerId, fi.isFenced(), fi.getMasterKey(), entries); + long numberOfEntries = dbStorage.addLedgerToIndex(ledgerId, fi.fenced, fi.masterKey, pages); if (LOG.isDebugEnabled()) { - LOG.debug(" -- done. fenced={} entries={}", fi.isFenced(), numberOfEntries); + LOG.debug(" -- done. fenced={} entries={}", fi.fenced, numberOfEntries); } // Remove index from old storage @@ -2921,6 +2995,7 @@ int runCmd(CommandLine cmdLine) throws Exception { commands.put(CMD_WHOISAUDITOR, new WhoIsAuditorCmd()); commands.put(CMD_WHATISINSTANCEID, new WhatIsInstanceId()); commands.put(CMD_LEDGERMETADATA, new LedgerMetadataCmd()); + commands.put(CMD_LOCALCONSISTENCYCHECK, new LocalConsistencyCheck()); commands.put(CMD_SIMPLETEST, new SimpleTestCmd()); commands.put(CMD_BOOKIESANITYTEST, new BookieSanityTestCmd()); commands.put(CMD_READLOG, new ReadLogCmd()); @@ -3123,23 +3198,6 @@ private File getLedgerFile(long ledgerId) { return lf; } - /** - * Get FileInfo for a specified ledger. - * - * @param ledgerId Ledger Id - * @return read only file info instance - */ - ReadOnlyFileInfo getFileInfo(long ledgerId) throws IOException { - File ledgerFile = getLedgerFile(ledgerId); - if (null == ledgerFile) { - throw new FileNotFoundException("No index file found for ledger " + ledgerId - + ". It may be not flushed yet."); - } - ReadOnlyFileInfo fi = new ReadOnlyFileInfo(ledgerFile, null); - fi.readHeader(); - return fi; - } - private synchronized void initEntryLogger() throws IOException { if (null == entryLogger) { // provide read only entry logger @@ -3185,77 +3243,6 @@ protected void scanJournal(Journal journal, long journalId, JournalScanner scann /// Bookie Shell Commands /// - /** - * Read ledger meta. - * - * @param ledgerId Ledger Id - */ - protected void readLedgerMeta(long ledgerId) throws Exception { - System.out.println("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " ====="); - FileInfo fi = getFileInfo(ledgerId); - byte[] masterKey = fi.getMasterKey(); - if (null == masterKey) { - System.out.println("master key : NULL"); - } else { - System.out.println("master key : " + bytes2Hex(fi.getMasterKey())); - } - long size = fi.size(); - if (size % 8 == 0) { - System.out.println("size : " + size); - } else { - System.out.println("size : " + size + " (not aligned with 8, may be corrupted or under flushing now)"); - } - System.out.println("entries : " + (size / 8)); - System.out.println("isFenced : " + fi.isFenced()); - } - - /** - * Read ledger index entries. - * - * @param ledgerId Ledger Id - * @throws IOException - */ - protected void readLedgerIndexEntries(long ledgerId) throws IOException { - System.out.println("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " ====="); - FileInfo fi = getFileInfo(ledgerId); - long size = fi.size(); - System.out.println("size : " + size); - long curSize = 0; - long curEntry = 0; - LedgerEntryPage lep = new LedgerEntryPage(pageSize, entriesPerPage); - lep.usePage(); - try { - while (curSize < size) { - lep.setLedgerAndFirstEntry(ledgerId, curEntry); - lep.readPage(fi); - - // process a page - for (int i = 0; i < entriesPerPage; i++) { - long offset = lep.getOffset(i * 8); - if (0 == offset) { - System.out.println("entry " + curEntry + "\t:\tN/A"); - } else { - long entryLogId = offset >> 32L; - long pos = offset & 0xffffffffL; - System.out.println("entry " + curEntry + "\t:\t(log:" + entryLogId + ", pos: " + pos + ")"); - } - ++curEntry; - } - - curSize += pageSize; - } - } catch (IOException ie) { - LOG.error("Failed to read index page : ", ie); - if (curSize + pageSize < size) { - System.out.println("Failed to read index page @ " + curSize + ", the index file may be corrupted : " - + ie.getMessage()); - } else { - System.out.println("Failed to read last index page @ " + curSize + ", the index file may be corrupted " - + "or last index page is not fully flushed yet : " + ie.getMessage()); - } - } - } - protected void printEntryLogMetadata(long logId) throws IOException { LOG.info("Print entryLogMetadata of entrylog {} ({}.log)", logId, Long.toHexString(logId)); initEntryLogger(); @@ -3266,67 +3253,6 @@ protected void printEntryLogMetadata(long logId) throws IOException { }); } - /** - * Get an iterable over pages of entries and locations for a ledger. - * - * @param ledgerId - * @return - * @throws IOException - */ - protected Iterable> getLedgerIndexEntries(final long ledgerId) throws IOException { - final FileInfo fi = getFileInfo(ledgerId); - final long size = fi.size(); - - final LedgerEntryPage lep = new LedgerEntryPage(pageSize, entriesPerPage); - lep.usePage(); - - final Iterator> iterator = new Iterator>() { - long curSize = 0; - long curEntry = 0; - - @Override - public boolean hasNext() { - return curSize < size; - } - - @Override - public SortedMap next() { - SortedMap entries = Maps.newTreeMap(); - lep.setLedgerAndFirstEntry(ledgerId, curEntry); - try { - lep.readPage(fi); - } catch (IOException e) { - throw new RuntimeException(e); - } - - // process a page - for (int i = 0; i < entriesPerPage; i++) { - long offset = lep.getOffset(i * 8); - if (offset != 0) { - entries.put(curEntry, offset); - } - ++curEntry; - } - - curSize += pageSize; - return entries; - } - - @Override - public void remove() { - throw new RuntimeException("Cannot remove"); - } - - }; - - return new Iterable>() { - @Override - public Iterator> iterator() { - return iterator; - } - }; - } - /** * Scan over an entry log file. * diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannelBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannelBase.java index cfaee56b41d..c982ba0866d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannelBase.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannelBase.java @@ -25,6 +25,12 @@ * to buffer the input and output data. This class is a base class for wrapping the {@link FileChannel}. */ public abstract class BufferedChannelBase { + static class BufferedChannelClosedException extends IOException { + BufferedChannelClosedException() { + super("Attempting to access a file channel that has already been closed"); + } + } + protected final FileChannel fileChannel; protected BufferedChannelBase(FileChannel fc) { @@ -36,7 +42,7 @@ protected FileChannel validateAndGetFileChannel() throws IOException { // guarantee that once a log file has been closed and possibly deleted during garbage // collection, attempts will not be made to read from it if (!fileChannel.isOpen()) { - throw new IOException("Attempting to access a file channel that has already been closed"); + throw new BufferedChannelClosedException(); } return fileChannel; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedReadChannel.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedReadChannel.java index 96dea6f670c..04e58f018b8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedReadChannel.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedReadChannel.java @@ -45,7 +45,7 @@ public class BufferedReadChannel extends BufferedChannelBase { long invocationCount = 0; long cacheHitCount = 0; - public BufferedReadChannel(FileChannel fileChannel, int readCapacity) throws IOException { + public BufferedReadChannel(FileChannel fileChannel, int readCapacity) { super(fileChannel); this.readCapacity = readCapacity; this.readBuffer = Unpooled.buffer(readCapacity); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogCompactor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogCompactor.java index a5e2c3fc210..98f4960549f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogCompactor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogCompactor.java @@ -27,6 +27,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.bookkeeper.conf.ServerConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,11 +44,15 @@ public class EntryLogCompactor extends AbstractLogCompactor { final CompactableLedgerStorage ledgerStorage; private final int maxOutstandingRequests; - public EntryLogCompactor(GarbageCollectorThread gcThread) { - super(gcThread); + public EntryLogCompactor( + ServerConfiguration conf, + EntryLogger entryLogger, + CompactableLedgerStorage ledgerStorage, + LogRemovalListener logRemover) { + super(conf, logRemover); this.maxOutstandingRequests = conf.getCompactionMaxOutstandingRequests(); - this.entryLogger = gcThread.getEntryLogger(); - this.ledgerStorage = gcThread.getLedgerStorage(); + this.entryLogger = entryLogger; + this.ledgerStorage = ledgerStorage; } @Override @@ -57,7 +62,7 @@ public boolean compact(EntryLogMetadata entryLogMeta) { scannerFactory.newScanner(entryLogMeta)); scannerFactory.flush(); LOG.info("Removing entry log {} after compaction", entryLogMeta.getEntryLogId()); - gcThread.removeEntryLog(entryLogMeta.getEntryLogId()); + logRemovalListener.removeEntryLog(entryLogMeta.getEntryLogId()); } catch (LedgerDirsManager.NoWritableLedgerDirException nwlde) { LOG.warn("No writable ledger directory available, aborting compaction", nwlde); return false; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index ddf255a3b3b..f5913b334de 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -43,6 +43,7 @@ import java.io.InputStreamReader; import java.io.RandomAccessFile; import java.nio.ByteBuffer; +import java.nio.channels.AsynchronousCloseException; import java.nio.channels.FileChannel; import java.util.ArrayList; import java.util.Collections; @@ -58,6 +59,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.util.DiskChecker; import org.apache.bookkeeper.util.IOUtils; import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap; import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap.BiConsumerLong; @@ -323,6 +325,11 @@ interface EntryLogListener { void onRotateEntryLog(); } + public EntryLogger(ServerConfiguration conf) throws IOException { + this(conf, new LedgerDirsManager(conf, conf.getLedgerDirs(), + new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()))); + } + /** * Create an EntryLogger that stores it's log files in the given directories. */ @@ -616,7 +623,8 @@ public long addEntry(long ledger, ByteBuf entry, boolean rollLog) throws IOExcep private final FastThreadLocal sizeBuffer = new FastThreadLocal() { @Override protected ByteBuf initialValue() throws Exception { - return Unpooled.buffer(4); + // Max usage is size (4 bytes) + ledgerId (8 bytes) + entryid (8 bytes) + return Unpooled.buffer(4 + 8 + 8); } }; @@ -693,10 +701,85 @@ static long logIdForOffset(long offset) { - public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) - throws IOException, Bookie.NoEntryException { - long entryLogId = logIdForOffset(location); - long pos = location & 0xffffffffL; + /** + * Exception type for representing lookup errors. Useful for disambiguating different error + * conditions for reporting purposes. + */ + static class EntryLookupException extends Exception { + EntryLookupException(String message) { + super(message); + } + + /** + * Represents case where log file is missing. + */ + static class MissingLogFileException extends EntryLookupException { + MissingLogFileException(long ledgerId, long entryId, long entryLogId, long pos) { + super(String.format("Missing entryLog %d for ledgerId %d, entry %d at offset %d", + entryLogId, + ledgerId, + entryId, + pos)); + } + } + + /** + * Represents case where entry log is present, but does not contain the specified entry. + */ + static class MissingEntryException extends EntryLookupException { + MissingEntryException(long ledgerId, long entryId, long entryLogId, long pos) { + super(String.format("pos %d (entry %d for ledgerId %d) past end of entryLog %d", + pos, + entryId, + ledgerId, + entryLogId)); + } + } + + /** + * Represents case where log is present, but encoded entry length header is invalid. + */ + static class InvalidEntryLengthException extends EntryLookupException { + InvalidEntryLengthException(long ledgerId, long entryId, long entryLogId, long pos) { + super(String.format("Invalid entry length at pos %d (entry %d for ledgerId %d) for entryLog %d", + pos, + entryId, + ledgerId, + entryLogId)); + } + } + + /** + * Represents case where the entry at pos is wrong. + */ + static class WrongEntryException extends EntryLookupException { + WrongEntryException(long foundEntryId, long foundLedgerId, long ledgerId, + long entryId, long entryLogId, long pos) { + super(String.format( + "Found entry %d, ledger %d at pos %d entryLog %d, should have found entry %d for ledgerId %d", + foundEntryId, + foundLedgerId, + pos, + entryLogId, + entryId, + ledgerId)); + } + } + } + + private static class EntryLogEntry { + final int entrySize; + final BufferedReadChannel fc; + + EntryLogEntry(int entrySize, BufferedReadChannel fc) { + this.entrySize = entrySize; + this.fc = fc; + } + } + + private EntryLogEntry getFCForEntryInternal( + long ledgerId, long entryId, long entryLogId, long pos) + throws EntryLookupException, IOException { ByteBuf sizeBuff = sizeBuffer.get(); sizeBuff.clear(); pos -= 4; // we want to get the ledgerId and length to check @@ -704,15 +787,15 @@ public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) try { fc = getChannelForLogId(entryLogId); } catch (FileNotFoundException e) { - FileNotFoundException newe = new FileNotFoundException(e.getMessage() + " for " + ledgerId - + " with location " + location); - newe.setStackTrace(e.getStackTrace()); - throw newe; + throw new EntryLookupException.MissingLogFileException(ledgerId, entryId, entryLogId, pos); } - if (readFromLogChannel(entryLogId, fc, sizeBuff, pos) != sizeBuff.capacity()) { - throw new Bookie.NoEntryException("Short read from entrylog " + entryLogId, - ledgerId, entryId); + try { + if (readFromLogChannel(entryLogId, fc, sizeBuff, pos) != sizeBuff.capacity()) { + throw new EntryLookupException.MissingEntryException(ledgerId, entryId, entryLogId, pos); + } + } catch (BufferedChannelBase.BufferedChannelClosedException | AsynchronousCloseException e) { + throw new EntryLookupException.MissingLogFileException(ledgerId, entryId, entryLogId, pos); } pos += 4; int entrySize = sizeBuff.readInt(); @@ -724,12 +807,42 @@ public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) } if (entrySize < MIN_SANE_ENTRY_SIZE) { LOG.error("Read invalid entry length {}", entrySize); - throw new IOException("Invalid entry length " + entrySize); + throw new EntryLookupException.InvalidEntryLengthException(ledgerId, entryId, entryLogId, pos); + } + + long thisLedgerId = sizeBuff.getLong(4); + long thisEntryId = sizeBuff.getLong(12); + if (thisLedgerId != ledgerId || thisEntryId != entryId) { + throw new EntryLookupException.WrongEntryException( + thisEntryId, thisLedgerId, ledgerId, entryId, entryLogId, pos); } + return new EntryLogEntry(entrySize, fc); + } + + void checkEntry(long ledgerId, long entryId, long location) throws EntryLookupException, IOException { + long entryLogId = logIdForOffset(location); + long pos = location & 0xffffffffL; + getFCForEntryInternal(ledgerId, entryId, entryLogId, pos); + } - ByteBuf data = PooledByteBufAllocator.DEFAULT.directBuffer(entrySize, entrySize); - int rc = readFromLogChannel(entryLogId, fc, data, pos); - if (rc != entrySize) { + public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) + throws IOException, Bookie.NoEntryException { + long entryLogId = logIdForOffset(location); + long pos = location & 0xffffffffL; + + final EntryLogEntry entry; + try { + entry = getFCForEntryInternal(ledgerId, entryId, entryLogId, pos); + } catch (EntryLookupException.MissingEntryException entryLookupError) { + throw new Bookie.NoEntryException("Short read from entrylog " + entryLogId, + ledgerId, entryId); + } catch (EntryLookupException e) { + throw new IOException(e.toString()); + } + + ByteBuf data = PooledByteBufAllocator.DEFAULT.directBuffer(entry.entrySize, entry.entrySize); + int rc = readFromLogChannel(entryLogId, entry.fc, data, pos); + if (rc != entry.entrySize) { // Note that throwing NoEntryException here instead of IOException is not // without risk. If all bookies in a quorum throw this same exception // the client will assume that it has reached the end of the ledger. @@ -740,31 +853,15 @@ public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) data.release(); throw new Bookie.NoEntryException("Short read for " + ledgerId + "@" + entryId + " in " + entryLogId + "@" - + pos + "(" + rc + "!=" + entrySize + ")", ledgerId, entryId); + + pos + "(" + rc + "!=" + entry.entrySize + ")", ledgerId, entryId); } - data.writerIndex(entrySize); + data.writerIndex(entry.entrySize); return data; } public ByteBuf readEntry(long ledgerId, long entryId, long location) throws IOException, Bookie.NoEntryException { - long entryLogId = logIdForOffset(location); - long pos = location & 0xffffffffL; - ByteBuf data = internalReadEntry(ledgerId, entryId, location); - long thisLedgerId = data.getLong(0); - if (thisLedgerId != ledgerId) { - data.release(); - throw new IOException("problem found in " + entryLogId + "@" + entryId + " at position + " + pos - + " entry belongs to " + thisLedgerId + " not " + ledgerId); - } - long thisEntryId = data.getLong(8); - if (thisEntryId != entryId) { - data.release(); - throw new IOException("problem found in " + entryLogId + "@" + entryId + " at position + " + pos - + " entry is " + thisEntryId + " not " + entryId); - } - return data; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java index c4d9414f1bc..369883cd2e4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java @@ -149,7 +149,7 @@ public GarbageCollectorThread(ServerConfiguration conf, LedgerManager ledgerManager, final CompactableLedgerStorage ledgerStorage, StatsLogger statsLogger, - ScheduledExecutorService gcExecutor) + ScheduledExecutorService gcExecutor) throws IOException { this.gcExecutor = gcExecutor; this.conf = conf; @@ -210,10 +210,17 @@ public Long getSample() { majorCompactionThreshold = conf.getMajorCompactionThreshold(); majorCompactionInterval = conf.getMajorCompactionInterval() * SECOND; isForceGCAllowWhenNoSpace = conf.getIsForceGCAllowWhenNoSpace(); + + AbstractLogCompactor.LogRemovalListener remover = new AbstractLogCompactor.LogRemovalListener() { + @Override + public void removeEntryLog(long logToRemove) { + GarbageCollectorThread.this.removeEntryLog(logToRemove); + } + }; if (conf.getUseTransactionalCompaction()) { - this.compactor = new TransactionalEntryLogCompactor(this); + this.compactor = new TransactionalEntryLogCompactor(conf, entryLogger, ledgerStorage, remover); } else { - this.compactor = new EntryLogCompactor(this); + this.compactor = new EntryLogCompactor(conf, entryLogger, ledgerStorage, remover); } if (minorCompactionInterval > 0 && minorCompactionThreshold > 0) { @@ -591,10 +598,6 @@ protected Map extractMetaFromEntryLogs(Map lPages = pages.remove(ledgerId); if (null != lPages) { for (Map.Entry pageEntry: lPages.entrySet()) { - long entryId = pageEntry.getKey(); - synchronized (lruCleanPageMap) { - lruCleanPageMap.remove(new EntryKey(ledgerId, entryId)); - } - LedgerEntryPage lep = pageEntry.getValue(); - // Cannot imagine under what circumstances we would have a null entry here - // Just being safe - if (null != lep) { - if (lep.inUse()) { - illegalStateDeleteCounter.inc(); - } - listOfFreePages.add(lep); - } + lep.usePage(); + lep.markDeleted(); + lep.releasePage(); } - } } @@ -318,7 +308,11 @@ public void onSetInUse(LedgerEntryPage lep) { @Override public void onResetInUse(LedgerEntryPage lep) { - addToCleanPagesList(lep); + if (!lep.isDeleted()) { + addToCleanPagesList(lep); + } else { + addToListOfFreePages(lep); + } } @Override @@ -396,24 +390,10 @@ public int getPageSize() { return pageSize; } - /** - * @return entries per page used in ledger cache - */ - public int getEntriesPerPage() { - return entriesPerPage; - } - - /** - * @return page limitation in ledger cache - */ - public int getPageLimit() { - return pageLimit; - } - /** * @return number of page used in ledger cache */ - public int getNumUsedPages() { + private int getNumUsedPages() { return pageCount.get(); } @@ -427,7 +407,7 @@ public int getNumUsedPages() { * @return ledger entry page * @throws IOException */ - public LedgerEntryPage getLedgerEntryPage(long ledger, + LedgerEntryPage getLedgerEntryPage(long ledger, long pageEntry) throws IOException { LedgerEntryPage lep = getLedgerEntryPageFromCache(ledger, pageEntry, false); if (lep == null) { @@ -616,4 +596,79 @@ long getEntryOffset(long ledger, long entry) throws IOException { } } } + + /** + * Represents a page of the index. + */ + private class PageEntriesImpl implements LedgerCache.PageEntries { + final long ledgerId; + final long initEntry; + + PageEntriesImpl(long ledgerId, long initEntry) { + this.ledgerId = ledgerId; + this.initEntry = initEntry; + } + + public LedgerEntryPage getLEP() throws IOException { + return getLedgerEntryPage(ledgerId, initEntry); + } + + public long getFirstEntry() { + return initEntry; + } + + public long getLastEntry() { + return initEntry + entriesPerPage; + } + } + + /** + * Iterable over index pages -- returns PageEntries rather than individual + * entries because getEntries() above needs to be able to throw an IOException. + */ + private class PageEntriesIterableImpl implements LedgerCache.PageEntriesIterable { + final long ledgerId; + final FileInfoBackingCache.CachedFileInfo fi; + final long totalEntries; + + long curEntry = 0; + + PageEntriesIterableImpl(long ledgerId) throws IOException { + this.ledgerId = ledgerId; + this.fi = indexPersistenceManager.getFileInfo(ledgerId, null); + this.totalEntries = max(entriesPerPage * (fi.size() / pageSize), getLastEntryInMem(ledgerId)); + } + + @Override + public Iterator iterator() { + return new Iterator() { + @Override + public boolean hasNext() { + return curEntry < totalEntries; + } + + @Override + public LedgerCache.PageEntries next() { + LedgerCache.PageEntries next = new PageEntriesImpl(ledgerId, curEntry); + curEntry += entriesPerPage; + return next; + } + }; + } + + @Override + public void close() { + fi.release(); + } + } + + /** + * Return iterator over pages for mapping entries to entry loggers. + * @param ledgerId + * @return Iterator over pages + * @throws IOException + */ + public LedgerCache.PageEntriesIterable listEntries(long ledgerId) throws IOException { + return new PageEntriesIterableImpl(ledgerId); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java index 11292d1688a..a622feacca5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java @@ -240,7 +240,7 @@ CachedFileInfo getFileInfo(final Long ledger, final byte masterKey[]) throws IOE if (ee.getCause() instanceof IOException) { throw (IOException) ee.getCause(); } else { - throw new IOException("Failed to load file info for ledger " + ledger, ee); + throw new LedgerCache.NoIndexForLedger("Failed to load file info for ledger " + ledger, ee); } } finally { pendingGetFileInfoCounter.dec(); @@ -715,4 +715,22 @@ long getPersistEntryBeyondInMem(long ledgerId, long lastEntryInMem) throws IOExc return lastEntry; } + /** + * Read ledger meta. + * @param ledgerId Ledger Id + */ + public LedgerCache.LedgerIndexMetadata readLedgerIndexMetadata(long ledgerId) throws IOException { + CachedFileInfo fi = null; + try { + fi = getFileInfo(ledgerId, null); + return new LedgerCache.LedgerIndexMetadata( + fi.getMasterKey(), + fi.size(), + fi.isFenced()); + } finally { + if (fi != null) { + fi.release(); + } + } + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java index d7d4977950d..4a96ece6a46 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java @@ -25,18 +25,24 @@ import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ENTRYLOGGER_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_GET_ENTRY; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_GET_OFFSET; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_SCRUB_PAGES_SCANNED; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.RateLimiter; import io.netty.buffer.ByteBuf; import java.io.File; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.Optional; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import lombok.Cleanup; import org.apache.bookkeeper.bookie.Bookie.NoLedgerException; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.bookie.EntryLogger.EntryLogListener; @@ -49,6 +55,8 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.SnapshotMap; +import org.apache.commons.lang.mutable.MutableBoolean; +import org.apache.commons.lang.mutable.MutableLong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,6 +90,7 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry // Expose Stats private OpStatsLogger getOffsetStats; private OpStatsLogger getEntryStats; + private OpStatsLogger pageScanStats; @VisibleForTesting public InterleavedLedgerStorage() { @@ -131,6 +140,7 @@ void initializeWithEntryLogListener(ServerConfiguration conf, // Expose Stats getOffsetStats = statsLogger.getOpStatsLogger(STORAGE_GET_OFFSET); getEntryStats = statsLogger.getOpStatsLogger(STORAGE_GET_ENTRY); + pageScanStats = statsLogger.getOpStatsLogger(STORAGE_SCRUB_PAGES_SCANNED); } private LedgerDirsListener getLedgerDirsListener() { @@ -466,4 +476,103 @@ public void onRotateEntryLog() { Checkpoint checkpoint = checkpointSource.newCheckpoint(); checkpointer.startCheckpoint(checkpoint); } + + /** + * Return iterable for index entries for ledgerId. + * @param ledgerId ledger to scan + * @return Iterator + */ + public LedgerCache.PageEntriesIterable getIndexEntries(long ledgerId) throws IOException { + return ledgerCache.listEntries(ledgerId); + } + + /** + * Read implementation metadata for index file. + * @param ledgerId + * @return Implementation metadata + * @throws IOException + */ + public LedgerCache.LedgerIndexMetadata readLedgerIndexMetadata(long ledgerId) throws IOException { + return ledgerCache.readLedgerIndexMetadata(ledgerId); + } + + @Override + public List localConsistencyCheck(Optional rateLimiter) throws IOException { + long checkStart = MathUtils.nowInNano(); + LOG.info("Starting localConsistencyCheck"); + long checkedLedgers = 0; + long checkedPages = 0; + final MutableLong checkedEntries = new MutableLong(0); + final MutableLong pageRetries = new MutableLong(0); + NavigableMap bkActiveLedgersSnapshot = activeLedgers.snapshot(); + final List errors = new ArrayList<>(); + for (Long ledger : bkActiveLedgersSnapshot.keySet()) { + try (LedgerCache.PageEntriesIterable pages = ledgerCache.listEntries(ledger)) { + for (LedgerCache.PageEntries page : pages) { + @Cleanup LedgerEntryPage lep = page.getLEP(); + MutableBoolean retry = new MutableBoolean(false); + do { + int version = lep.getVersion(); + + MutableBoolean success = new MutableBoolean(true); + long start = MathUtils.nowInNano(); + lep.getEntries((entry, offset) -> { + rateLimiter.ifPresent(RateLimiter::acquire); + + try { + entryLogger.checkEntry(ledger, entry, offset); + checkedEntries.increment(); + } catch (EntryLogger.EntryLookupException e) { + if (version != lep.getVersion()) { + pageRetries.increment(); + if (lep.isDeleted()) { + LOG.debug("localConsistencyCheck: ledger {} deleted", ledger); + } else { + LOG.debug("localConsistencyCheck: concurrent modification, retrying"); + retry.setValue(true); + } + return false; + } else { + errors.add(new DetectedInconsistency(ledger, entry, e)); + LOG.error("Got error: ", e); + } + success.setValue(false); + } + return true; + }); + + if (success.booleanValue()) { + pageScanStats.registerSuccessfulEvent( + MathUtils.elapsedNanos(start), TimeUnit.NANOSECONDS); + } else { + pageScanStats.registerFailedEvent( + MathUtils.elapsedNanos(start), TimeUnit.NANOSECONDS); + } + } while (retry.booleanValue()); + checkedPages++; + } + } catch (NoLedgerException | FileInfo.FileInfoDeletedException e) { + if (activeLedgers.containsKey(ledger)) { + LOG.error("Cannot find ledger {}, should exist, exception is ", ledger, e); + errors.add(new DetectedInconsistency(ledger, -1, e)); + } else { + LOG.debug("ledger {} deleted since snapshot taken", ledger); + } + } catch (Exception e) { + throw new IOException("Got other exception in localConsistencyCheck", e); + } + checkedLedgers++; + } + LOG.info( + "Finished localConsistencyCheck, took {}s to scan {} ledgers, {} pages, " + + "{} entries with {} retries, {} errors", + TimeUnit.NANOSECONDS.toSeconds(MathUtils.elapsedNanos(checkStart)), + checkedLedgers, + checkedPages, + checkedEntries.longValue(), + pageRetries.longValue(), + errors.size()); + + return errors; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java index de48eafe5d4..f5e7c17baa7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java @@ -226,5 +226,13 @@ public void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException { public ByteBuf getExplicitLac(long ledgerId) { throw new UnsupportedOperationException(); } + @Override + public PageEntriesIterable listEntries(long ledgerId) throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public LedgerIndexMetadata readLedgerIndexMetadata(long ledgerId) throws IOException { + throw new UnsupportedOperationException(); + } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java index 14d48255d5a..86984b7ac94 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java @@ -21,6 +21,8 @@ package org.apache.bookkeeper.bookie; +import static org.apache.bookkeeper.bookie.BookieShell.bytes2Hex; + import io.netty.buffer.ByteBuf; import java.io.Closeable; import java.io.IOException; @@ -31,7 +33,7 @@ * an entry log file. It does user level caching to more efficiently manage disk * head scheduling. */ -interface LedgerCache extends Closeable { +public interface LedgerCache extends Closeable { boolean setFenced(long ledgerId) throws IOException; boolean isFenced(long ledgerId) throws IOException; @@ -56,4 +58,54 @@ boolean waitForLastAddConfirmedUpdate(long ledgerId, void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException; ByteBuf getExplicitLac(long ledgerId); + + /** + * Specific exception to encode the case where the index is not present. + */ + class NoIndexForLedger extends IOException { + NoIndexForLedger(String reason, Exception cause) { + super(reason, cause); + } + } + + /** + * Represents a page of the index. + */ + interface PageEntries { + LedgerEntryPage getLEP() throws IOException; + long getFirstEntry(); + long getLastEntry(); + } + + /** + * Iterable over index pages -- returns PageEntries rather than individual + * entries because getEntries() above needs to be able to throw an IOException. + */ + interface PageEntriesIterable extends AutoCloseable, Iterable {} + + PageEntriesIterable listEntries(long ledgerId) throws IOException; + + /** + * Represents summary of ledger metadata. + */ + class LedgerIndexMetadata { + public final byte[] masterKey; + public final long size; + public final boolean fenced; + LedgerIndexMetadata(byte[] masterKey, long size, boolean fenced) { + this.masterKey = masterKey; + this.size = size; + this.fenced = fenced; + } + + public String getMasterKeyHex() { + if (null == masterKey) { + return "NULL"; + } else { + return bytes2Hex(masterKey); + } + } + } + + LedgerIndexMetadata readLedgerIndexMetadata(long ledgerId) throws IOException; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java index 1db7d47041e..119a4f46a89 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java @@ -169,4 +169,14 @@ public boolean ledgerExists(long ledgerId) throws IOException { public void close() throws IOException { indexPersistenceManager.close(); } + + @Override + public PageEntriesIterable listEntries(long ledgerId) throws IOException { + return indexPageManager.listEntries(ledgerId); + } + + @Override + public LedgerIndexMetadata readLedgerIndexMetadata(long ledgerId) throws IOException { + return indexPersistenceManager.readLedgerIndexMetadata(ledgerId); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java index be8755959cc..c272e7ca0bf 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java @@ -33,7 +33,7 @@ * This is a page in the LedgerCache. It holds the locations * (entrylogfile, offset) for entry ids. */ -public class LedgerEntryPage { +public class LedgerEntryPage implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(LedgerEntryPage.class); @@ -47,6 +47,7 @@ public class LedgerEntryPage { private final AtomicInteger version = new AtomicInteger(0); private volatile int last = -1; // Last update position private final LEPStateChangeCallback callback; + private boolean deleted; public static int getIndexEntrySize() { return indexEntrySize; @@ -75,11 +76,20 @@ public void resetPage() { entryKey = new EntryKey(-1, BookieProtocol.INVALID_ENTRY_ID); clean = true; useCount.set(0); + deleted = false; if (null != this.callback) { callback.onResetInUse(this); } } + public void markDeleted() { + deleted = true; + version.incrementAndGet(); + } + + public boolean isDeleted() { + return deleted; + } @Override public String toString() { @@ -215,7 +225,7 @@ long getLedger() { return entryKey.getLedgerId(); } - int getVersion() { + public int getVersion() { return version.get(); } @@ -262,4 +272,34 @@ public long getLastEntry() { return index >= 0 ? (index + entryKey.getEntryId()) : 0; } } + + /** + * Interface for getEntries to propagate entry, pos pairs. + */ + public interface EntryVisitor { + boolean visit(long entry, long pos) throws Exception; + } + + /** + * Iterates over non-empty entry mappings. + * + * @param vis Consumer for entry position pairs. + * @throws Exception + */ + public void getEntries(EntryVisitor vis) throws Exception { + // process a page + for (int i = 0; i < entriesPerPage; i++) { + long offset = getOffset(i * 8); + if (offset != 0) { + if (!vis.visit(getFirstEntry() + i, offset)) { + return; + } + } + } + } + + @Override + public void close() throws Exception { + releasePage(); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java index ef64ff986c2..7a98fc797b6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java @@ -21,8 +21,12 @@ package org.apache.bookkeeper.bookie; +import com.google.common.util.concurrent.RateLimiter; import io.netty.buffer.ByteBuf; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -181,4 +185,43 @@ default LedgerStorage getUnderlyingLedgerStorage() { default void forceGC() { return; } + + /** + * Class for describing location of a generic inconsistency. Implementations should + * ensure that detail is populated with an exception which adequately describes the + * nature of the problem. + */ + class DetectedInconsistency { + private long ledgerId; + private long entryId; + private Exception detail; + + DetectedInconsistency(long ledgerId, long entryId, Exception detail) { + this.ledgerId = ledgerId; + this.entryId = entryId; + this.detail = detail; + } + + public long getLedgerId() { + return ledgerId; + } + + public long getEntryId() { + return entryId; + } + + public Exception getException() { + return detail; + } + } + + /** + * Performs internal check of local storage logging any inconsistencies. + * @param rateLimiter Provide to rate of entry checking. null for unlimited. + * @return List of inconsistencies detected + * @throws IOException + */ + default List localConsistencyCheck(Optional rateLimiter) throws IOException { + return new ArrayList<>(); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyEntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyEntryLogger.java index 3a07ec4e805..2a683dcefa5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyEntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyEntryLogger.java @@ -25,7 +25,6 @@ import java.nio.ByteBuffer; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.util.DiskChecker; /** * Read Only Entry Logger. @@ -33,8 +32,7 @@ public class ReadOnlyEntryLogger extends EntryLogger { public ReadOnlyEntryLogger(ServerConfiguration conf) throws IOException { - super(conf, new LedgerDirsManager(conf, conf.getLedgerDirs(), - new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()))); + super(conf); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScrubberStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScrubberStats.java new file mode 100644 index 00000000000..f22c784c6fd --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScrubberStats.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. + * + */ + +package org.apache.bookkeeper.bookie; + +/** + * Stats associated with the consistency checker. + */ +public class ScrubberStats { + public static final String SCOPE = "scrubber"; + + public static final String RUN_DURATION = "runTime"; + public static final String DETECTED_SCRUB_ERRORS = "detectedScrubErrors"; + public static final String DETECTED_FATAL_SCRUB_ERRORS = "detectedFatalScrubErrors"; +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java index dd7b373375a..dd07d755504 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java @@ -21,9 +21,12 @@ package org.apache.bookkeeper.bookie; import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.RateLimiter; import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.buffer.ByteBuf; import java.io.IOException; +import java.util.List; +import java.util.Optional; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -332,4 +335,9 @@ public LedgerStorage getUnderlyingLedgerStorage() { public void forceGC() { interleavedLedgerStorage.forceGC(); } + + @Override + public List localConsistencyCheck(Optional rateLimiter) throws IOException { + return interleavedLedgerStorage.localConsistencyCheck(rateLimiter); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java index 51a2cdda1d9..74a30a63a56 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java @@ -29,6 +29,7 @@ import java.util.List; import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner; +import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.util.HardLink; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,10 +54,14 @@ public class TransactionalEntryLogCompactor extends AbstractLogCompactor { // flushed compaction log file suffix static final String COMPACTED_SUFFIX = ".compacted"; - public TransactionalEntryLogCompactor(GarbageCollectorThread gcThread) { - super(gcThread); - this.entryLogger = gcThread.getEntryLogger(); - this.ledgerStorage = gcThread.getLedgerStorage(); + public TransactionalEntryLogCompactor( + ServerConfiguration conf, + EntryLogger entryLogger, + CompactableLedgerStorage ledgerStorage, + LogRemovalListener logRemover) { + super(conf, logRemover); + this.entryLogger = entryLogger; + this.ledgerStorage = ledgerStorage; } /** @@ -201,7 +206,7 @@ boolean complete() { if (offsets.isEmpty()) { // no valid entries is compacted, delete entry log file LOG.info("No valid entry is found in entry log after scan, removing entry log now."); - gcThread.removeEntryLog(metadata.getEntryLogId()); + logRemovalListener.removeEntryLog(metadata.getEntryLogId()); entryLogger.removeCurCompactionLog(); return false; } @@ -335,7 +340,7 @@ boolean complete() { String compactedFilename = compactedLogFile.getName(); String oldEntryLogFilename = compactedFilename.substring(compactedFilename.indexOf(".log") + 5); long entryLogId = EntryLogger.fileName2LogId(oldEntryLogFilename); - gcThread.removeEntryLog(entryLogId); + logRemovalListener.removeEntryLog(entryLogId); } return true; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index aad42ad361e..4c5fd33197c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -36,7 +36,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.SortedMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -47,6 +46,7 @@ import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.bookie.Checkpointer; import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification; +import org.apache.bookkeeper.bookie.LedgerCache; import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.LedgerStorage; import org.apache.bookkeeper.bookie.StateManager; @@ -275,8 +275,8 @@ public ByteBuf getExplicitLac(long ledgerId) { } public long addLedgerToIndex(long ledgerId, boolean isFenced, byte[] masterKey, - Iterable> entries) throws Exception { - return getLedgerSorage(ledgerId).addLedgerToIndex(ledgerId, isFenced, masterKey, entries); + LedgerCache.PageEntriesIterable pages) throws Exception { + return getLedgerSorage(ledgerId).addLedgerToIndex(ledgerId, isFenced, masterKey, pages); } public long getLastEntryInLedger(long ledgerId) throws IOException { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 27b4214e1b5..2e3c5562aba 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -30,14 +30,12 @@ import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; -import java.util.SortedMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.StampedLock; @@ -53,7 +51,9 @@ import org.apache.bookkeeper.bookie.EntryLogger; import org.apache.bookkeeper.bookie.GarbageCollectorThread; import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification; +import org.apache.bookkeeper.bookie.LedgerCache; import org.apache.bookkeeper.bookie.LedgerDirsManager; +import org.apache.bookkeeper.bookie.LedgerEntryPage; import org.apache.bookkeeper.bookie.StateManager; import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageDataFormats.LedgerData; import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorage.Batch; @@ -67,6 +67,7 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap; +import org.apache.commons.lang.mutable.MutableLong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -859,35 +860,33 @@ public void flushEntriesLocationsIndex() throws IOException { * * @param ledgerId * the ledger id - * @param entries - * a map of entryId -> location + * @param pages + * Iterator over index pages from Indexed * @return the number of */ public long addLedgerToIndex(long ledgerId, boolean isFenced, byte[] masterKey, - Iterable> entries) throws Exception { + LedgerCache.PageEntriesIterable pages) throws Exception { LedgerData ledgerData = LedgerData.newBuilder().setExists(true).setFenced(isFenced) .setMasterKey(ByteString.copyFrom(masterKey)).build(); ledgerIndex.set(ledgerId, ledgerData); - AtomicLong numberOfEntries = new AtomicLong(); + MutableLong numberOfEntries = new MutableLong(); // Iterate over all the entries pages Batch batch = entryLocationIndex.newBatch(); - entries.forEach(map -> { - map.forEach((entryId, location) -> { - try { + for (LedgerCache.PageEntries page: pages) { + try (LedgerEntryPage lep = page.getLEP()) { + lep.getEntries((entryId, location) -> { entryLocationIndex.addLocation(batch, ledgerId, entryId, location); - } catch (IOException e) { - throw new RuntimeException(e); - } - - numberOfEntries.incrementAndGet(); - }); - }); + numberOfEntries.increment(); + return true; + }); + } + } batch.flush(); batch.close(); - return numberOfEntries.get(); + return numberOfEntries.longValue(); } @Override 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 afcfadcedfa..2a77e91cde0 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 @@ -55,6 +55,9 @@ public class ServerConfiguration extends AbstractConfiguration 0; + } + + /** + * Get local scrub interval. + * + * @return Number of seconds between scrubs, <= 0 for disabled. + */ + public long getLocalScrubPeriod() { + return this.getLong(LOCAL_SCRUB_PERIOD, 0); + } + + /** + * Set local scrub period in seconds (<= 0 for disabled). Scrub will be scheduled at delays + * chosen from the interval (.5 * interval, 1.5 * interval) + */ + public void setLocalScrubPeriod(long period) { + this.setProperty(LOCAL_SCRUB_PERIOD, period); + } + + /** + * Get local scrub rate limit (entries/second). + * + * @return Max number of entries to scrub per second, 0 for disabled. + */ + public double getLocalScrubRateLimit() { + return this.getDouble(LOCAL_SCRUB_RATE_LIMIT, 60); + } + + /** + * Get local scrub rate limit (entries/second). + * + * @param scrubRateLimit Max number of entries per second to scan. + */ + public void setLocalScrubRateLimit(double scrubRateLimit) { + this.setProperty(LOCAL_SCRUB_RATE_LIMIT, scrubRateLimit); + } + /** * Get flush interval. Default value is 10 second. It isn't useful to decrease * this value, since ledger storage only checkpoints when an entry logger file @@ -3093,4 +3143,11 @@ public ServerConfiguration setEntryLogPerLedgerCounterLimitsMultFactor( Integer.toString(entryLogPerLedgerCounterLimitsMultFactor)); return this; } + + /** + * True if a local consistency check should be performed on startup. + */ + public boolean isLocalConsistencyCheckOnStartup() { + return this.getBoolean(LOCAL_CONSISTENCY_CHECK_ON_STARTUP, 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 b991d31b896..ec163f81fb4 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 @@ -28,6 +28,7 @@ import java.util.concurrent.ExecutionException; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.ExitCode; +import org.apache.bookkeeper.bookie.ScrubberStats; import org.apache.bookkeeper.common.component.ComponentStarter; import org.apache.bookkeeper.common.component.LifecycleComponent; import org.apache.bookkeeper.common.component.LifecycleComponentStack; @@ -39,6 +40,7 @@ import org.apache.bookkeeper.server.service.AutoRecoveryService; import org.apache.bookkeeper.server.service.BookieService; import org.apache.bookkeeper.server.service.HttpService; +import org.apache.bookkeeper.server.service.ScrubberService; import org.apache.bookkeeper.server.service.StatsProviderService; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.cli.BasicParser; @@ -302,6 +304,13 @@ public static LifecycleComponentStack buildBookieServer(BookieConfiguration conf serverBuilder.addComponent(bookieService); log.info("Load lifecycle component : {}", BookieService.class.getName()); + if (conf.getServerConf().isLocalScrubEnabled()) { + serverBuilder.addComponent( + new ScrubberService( + rootStatsLogger.scope(ScrubberStats.SCOPE), + conf, bookieService.getServer().getBookie().getLedgerStorage())); + } + // 3. build auto recovery if (conf.getServerConf().isAutoRecoveryDaemonEnabled()) { AutoRecoveryService autoRecoveryService = diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/ScrubberService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/ScrubberService.java new file mode 100644 index 00000000000..4c027a61401 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/ScrubberService.java @@ -0,0 +1,145 @@ +/** + * + * 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.server.service; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.bookie.ScrubberStats.DETECTED_FATAL_SCRUB_ERRORS; +import static org.apache.bookkeeper.bookie.ScrubberStats.DETECTED_SCRUB_ERRORS; +import static org.apache.bookkeeper.bookie.ScrubberStats.RUN_DURATION; + +import com.google.common.util.concurrent.RateLimiter; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.bookkeeper.bookie.ExitCode; +import org.apache.bookkeeper.bookie.LedgerStorage; +import org.apache.bookkeeper.server.component.ServerLifecycleComponent; +import org.apache.bookkeeper.server.conf.BookieConfiguration; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.util.MathUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link org.apache.bookkeeper.common.component.LifecycleComponent} that runs the scrubber background service. + */ +public class ScrubberService extends ServerLifecycleComponent { + private static final Logger LOG = LoggerFactory.getLogger(ScrubberService.class); + + private static final String NAME = "scrubber"; + private final ScheduledExecutorService executor; + private final Random rng = new Random(); + private final long scrubPeriod; + private final Optional scrubRateLimiter; + private final AtomicBoolean stop = new AtomicBoolean(false); + private final LedgerStorage ledgerStorage; + + private final OpStatsLogger scrubCounter; + private final Counter errorCounter; + private final Counter fatalErrorCounter; + + public ScrubberService( + StatsLogger logger, + BookieConfiguration conf, + LedgerStorage ledgerStorage) { + super(NAME, conf, logger); + this.executor = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("ScrubThread")); + + this.scrubPeriod = conf.getServerConf().getLocalScrubPeriod(); + checkArgument( + scrubPeriod > 0, + "localScrubInterval must be > 0 for ScrubberService to be used"); + + double rateLimit = conf.getServerConf().getLocalScrubRateLimit(); + this.scrubRateLimiter = rateLimit == 0 ? Optional.empty() : Optional.of(RateLimiter.create(rateLimit)); + + this.ledgerStorage = ledgerStorage; + + this.scrubCounter = logger.getOpStatsLogger(RUN_DURATION); + this.errorCounter = logger.getCounter(DETECTED_SCRUB_ERRORS); + this.fatalErrorCounter = logger.getCounter(DETECTED_FATAL_SCRUB_ERRORS); + } + + private long getNextPeriodMS() { + return (long) (((double) scrubPeriod) * (1.5 - rng.nextDouble()) * 1000); + } + + private void doSchedule() { + executor.schedule( + this::run, + getNextPeriodMS(), + TimeUnit.MILLISECONDS); + + } + + private void run() { + boolean success = false; + long start = MathUtils.nowInNano(); + try { + List errors = ledgerStorage.localConsistencyCheck(scrubRateLimiter); + if (errors.size() > 0) { + errorCounter.add(errors.size()); + LOG.error("Found inconsistency during localConsistencyCheck:"); + for (LedgerStorage.DetectedInconsistency error : errors) { + LOG.error("Ledger {}, entry {}: ", error.getLedgerId(), error.getEntryId(), error.getException()); + } + } + success = true; + } catch (IOException e) { + fatalErrorCounter.inc(); + LOG.error("Got fatal exception {} running localConsistencyCheck", e.toString()); + } + if (success) { + scrubCounter.registerSuccessfulEvent(MathUtils.elapsedNanos(start), TimeUnit.NANOSECONDS); + } else { + scrubCounter.registerFailedEvent(MathUtils.elapsedNanos(start), TimeUnit.NANOSECONDS); + Runtime.getRuntime().exit(ExitCode.BOOKIE_EXCEPTION); + } + if (!stop.get()) { + doSchedule(); + } + } + + @Override + protected void doStart() { + doSchedule(); + } + + @Override + protected void doStop() { + stop.set(true); + executor.shutdown(); + } + + @Override + protected void doClose() throws IOException { + // no-op + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java index c110eeb99d2..bbe9c10edcd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java @@ -1383,7 +1383,12 @@ private Set findCompactedEntryLogFiles() { private static class MockTransactionalEntryLogCompactor extends TransactionalEntryLogCompactor { public MockTransactionalEntryLogCompactor(GarbageCollectorThread gcThread) { - super(gcThread); + super(gcThread.conf, + gcThread.entryLogger, + gcThread.ledgerStorage, + (long entry) -> { + gcThread.removeEntryLog(entry); + }); } synchronized void compactWithIndexFlushFailure(EntryLogMetadata metadata) { @@ -1405,7 +1410,7 @@ synchronized void compactWithIndexFlushFailure(EntryLogMetadata metadata) { LOG.info("Compaction for {} end in PartialFlushIndexPhase.", metadata.getEntryLogId()); return; } - gcThread.removeEntryLog(metadata.getEntryLogId()); + logRemovalListener.removeEntryLog(metadata.getEntryLogId()); LOG.info("Compacted entry log : {}.", metadata.getEntryLogId()); } @@ -1428,7 +1433,7 @@ synchronized void compactWithLogFlushFailure(EntryLogMetadata metadata) { LOG.info("Compaction for entry log {} end in UpdateIndexPhase.", metadata.getEntryLogId()); return; } - gcThread.removeEntryLog(metadata.getEntryLogId()); + logRemovalListener.removeEntryLog(metadata.getEntryLogId()); LOG.info("Compacted entry log : {}.", metadata.getEntryLogId()); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java new file mode 100644 index 00000000000..fd4f314c4ef --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java @@ -0,0 +1,303 @@ +/** + * + * 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 org.junit.Assert.assertEquals; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.util.DiskChecker; +import org.apache.bookkeeper.util.EntryFormatter; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.apache.commons.lang.mutable.MutableLong; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * Test for InterleavedLedgerStorage. + */ +public class InterleavedLedgerStorageTest { + + CheckpointSource checkpointSource = new CheckpointSource() { + @Override + public Checkpoint newCheckpoint() { + return Checkpoint.MAX; + } + + @Override + public void checkpointComplete(Checkpoint checkpoint, boolean compact) throws IOException { + } + }; + + Checkpointer checkpointer = new Checkpointer() { + @Override + public void startCheckpoint(Checkpoint checkpoint) { + // No-op + } + + @Override + public void start() { + // no-op + } + }; + + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + LedgerDirsManager ledgerDirsManager; + InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); + final long numWrites = 2000; + final long entriesPerWrite = 2; + + @Before + public void setUp() throws Exception { + File tmpDir = File.createTempFile("bkTest", ".dir"); + tmpDir.delete(); + tmpDir.mkdir(); + File curDir = Bookie.getCurrentDirectory(tmpDir); + Bookie.checkDirectoryStructure(curDir); + + conf = TestBKConfiguration.newServerConfiguration(); + conf.setLedgerDirNames(new String[] { tmpDir.toString() }); + conf.setEntryLogSizeLimit(2048); + ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), + new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); + + interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, + null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE); + + // Insert some ledger & entries in the interleaved storage + for (long entryId = 0; entryId < numWrites; entryId++) { + for (long ledgerId = 0; ledgerId < 5; ledgerId++) { + if (entryId == 0) { + interleavedStorage.setMasterKey(ledgerId, ("ledger-" + ledgerId).getBytes()); + interleavedStorage.setFenced(ledgerId); + } + ByteBuf entry = Unpooled.buffer(128); + entry.writeLong(ledgerId); + entry.writeLong(entryId * entriesPerWrite); + entry.writeBytes(("entry-" + entryId).getBytes()); + + interleavedStorage.addEntry(entry); + } + } + } + + @Test + public void testIndexEntryIterator() throws Exception { + try (LedgerCache.PageEntriesIterable pages = interleavedStorage.getIndexEntries(0)) { + MutableLong curEntry = new MutableLong(0); + for (LedgerCache.PageEntries page : pages) { + try (LedgerEntryPage lep = page.getLEP()) { + lep.getEntries((entry, offset) -> { + Assert.assertEquals(curEntry.longValue(), entry); + Assert.assertNotEquals(0, offset); + curEntry.setValue(entriesPerWrite + entry); + return true; + }); + } + } + Assert.assertEquals(entriesPerWrite * numWrites, curEntry.longValue()); + } + } + + @Test + public void testConsistencyCheckConcurrentModification() throws Exception { + AtomicBoolean done = new AtomicBoolean(false); + EntryLogger entryLogger = interleavedStorage.getEntryLogger(); + List asyncErrors = new ArrayList<>(); + Thread mutator = new Thread(() -> { + EntryLogCompactor compactor = new EntryLogCompactor( + conf, + entryLogger, + interleavedStorage, + entryLogger::removeEntryLog); + long next = 0; + while (!done.get()) { + try { + compactor.compact(entryLogger.getEntryLogMetadata(next)); + next++; + } catch (IOException e) { + asyncErrors.add(e); + break; + } + } + }); + mutator.start(); + + for (int i = 0; i < 100; ++i) { + assert interleavedStorage.localConsistencyCheck(Optional.empty()).size() == 0; + Thread.sleep(10); + } + + done.set(true); + mutator.join(); + for (Exception e: asyncErrors) { + throw e; + } + } + + + @Test + public void testConsistencyMissingEntry() throws Exception { + // set 1, 1 to nonsense + interleavedStorage.ledgerCache.putEntryOffset(1, 1, 0xFFFFFFFFFFFFFFFFL); + + List errors = interleavedStorage.localConsistencyCheck(Optional.empty()); + Assert.assertEquals(1, errors.size()); + LedgerStorage.DetectedInconsistency inconsistency = errors.remove(0); + Assert.assertEquals(1, inconsistency.getEntryId()); + Assert.assertEquals(1, inconsistency.getLedgerId()); + } + + @Test + public void testWrongEntry() throws Exception { + // set 1, 1 to nonsense + interleavedStorage.ledgerCache.putEntryOffset( + 1, + 1, + interleavedStorage.ledgerCache.getEntryOffset(0, 0)); + + List errors = interleavedStorage.localConsistencyCheck(Optional.empty()); + Assert.assertEquals(1, errors.size()); + LedgerStorage.DetectedInconsistency inconsistency = errors.remove(0); + Assert.assertEquals(1, inconsistency.getEntryId()); + Assert.assertEquals(1, inconsistency.getLedgerId()); + } + + @Test + public void testShellCommands() throws Exception { + interleavedStorage.flush(); + interleavedStorage.shutdown(); + final Pattern entryPattern = Pattern.compile( + "entry (?\\d+)\t:\t((?N/A)|\\(log:(?\\d+), pos: (?\\d+)\\))"); + + class Metadata { + final Pattern keyPattern = Pattern.compile("master key +: ([0-9a-f])"); + final Pattern sizePattern = Pattern.compile("size +: (\\d+)"); + final Pattern entriesPattern = Pattern.compile("entries +: (\\d+)"); + final Pattern isFencedPattern = Pattern.compile("isFenced +: (\\w+)"); + + public String masterKey; + public long size = -1; + public long entries = -1; + public boolean foundFenced = false; + + void check(String s) { + Matcher keyMatcher = keyPattern.matcher(s); + if (keyMatcher.matches()) { + masterKey = keyMatcher.group(1); + return; + } + + Matcher sizeMatcher = sizePattern.matcher(s); + if (sizeMatcher.matches()) { + size = Long.valueOf(sizeMatcher.group(1)); + return; + } + + Matcher entriesMatcher = entriesPattern.matcher(s); + if (entriesMatcher.matches()) { + entries = Long.valueOf(entriesMatcher.group(1)); + return; + } + + Matcher isFencedMatcher = isFencedPattern.matcher(s); + if (isFencedMatcher.matches()) { + Assert.assertEquals("true", isFencedMatcher.group(1)); + foundFenced = true; + return; + } + } + + void validate(long foundEntries) { + Assert.assertTrue(entries >= numWrites * entriesPerWrite); + Assert.assertEquals(entries, foundEntries); + Assert.assertTrue(foundFenced); + Assert.assertNotEquals(-1, size); + } + } + final Metadata foundMetadata = new Metadata(); + + AtomicLong curEntry = new AtomicLong(0); + AtomicLong someEntryLogger = new AtomicLong(-1); + BookieShell shell = new BookieShell( + LedgerIdFormatter.LONG_LEDGERID_FORMATTER, EntryFormatter.STRING_FORMATTER) { + @Override + void printInfoLine(String s) { + Matcher matcher = entryPattern.matcher(s); + System.out.println(s); + if (matcher.matches()) { + assertEquals(Long.toString(curEntry.get()), matcher.group("entry")); + + if (matcher.group("na") == null) { + String logId = matcher.group("logid"); + Assert.assertNotEquals(matcher.group("logid"), null); + Assert.assertNotEquals(matcher.group("pos"), null); + Assert.assertTrue((curEntry.get() % entriesPerWrite) == 0); + Assert.assertTrue(curEntry.get() <= numWrites * entriesPerWrite); + if (someEntryLogger.get() == -1) { + someEntryLogger.set(Long.valueOf(logId)); + } + } else { + Assert.assertEquals(matcher.group("logid"), null); + Assert.assertEquals(matcher.group("pos"), null); + Assert.assertTrue(((curEntry.get() % entriesPerWrite) != 0) + || ((curEntry.get() >= (entriesPerWrite * numWrites)))); + } + curEntry.incrementAndGet(); + } else { + foundMetadata.check(s); + } + } + }; + shell.setConf(conf); + int res = shell.run(new String[] { "ledger", "-m", "0" }); + Assert.assertEquals(0, res); + Assert.assertTrue(curEntry.get() >= numWrites * entriesPerWrite); + foundMetadata.validate(curEntry.get()); + + // Should pass consistency checker + res = shell.run(new String[] { "localconsistencycheck" }); + Assert.assertEquals(0, res); + + + // Remove a logger + EntryLogger entryLogger = new EntryLogger(conf); + entryLogger.removeEntryLog(someEntryLogger.get()); + + // Should fail consistency checker + res = shell.run(new String[] { "localconsistencycheck" }); + Assert.assertEquals(1, res); + } +} From 753ff93b0bdd3a3055aa7f5f4d9111c4af84909e Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 12 Dec 2018 09:13:19 +0800 Subject: [PATCH 0167/1642] [CI] Update precommit job captions with trigger phrase Descriptions of the changes in this PR: *Motivation* Sometime people don't know how to retrigger a bookkeeper precommit job. *Changes* This PR is adding the trigger phrase to the job caption. so people will know how to retrigger the jobs in the PR itself. Reviewers: Enrico Olivelli This closes #1881 from sijie/update_jenkins_for_self_instructions --- .../jenkins/job_bookkeeper_precommit_bookie_tests.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_client_tests.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_integrationtests.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_java8.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_java9.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_remaining_tests.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_replication_tests.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_validation.groovy | 2 +- 9 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy index 8f8e5ae5017..f25cdd5004f 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy @@ -40,7 +40,7 @@ freeStyleJob('bookkeeper_precommit_bookie_tests') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'Bookie Tests', + 'Bookie Tests (trigger via `run bookkeeper-server bookie tests`)', '.*(re)?run bookkeeper-server (bookie )?tests.*', '.*\\[x\\] \\[skip bookkeeper-server (bookie )?tests\\].*', true) diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy index 9240099ae59..03e26365eab 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy @@ -40,7 +40,7 @@ freeStyleJob('bookkeeper_precommit_client_tests') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'Client Tests', + 'Client Tests (trigger via `run bookkeeper-server client tests`)', '.*(re)?run bookkeeper-server (client )?tests.*', '.*\\[x\\] \\[skip bookkeeper-server (client )?tests\\].*', true) diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy index cda3d9be027..0fa3ddfbfdb 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy @@ -42,7 +42,7 @@ freeStyleJob('bookkeeper_precommit_integrationtests') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'Integration Tests', + 'Integration Tests (trigger via `run integration tests`)', '.*(re)?run integration tests.*', '.*\\[x\\] \\[skip integration tests\\].*') diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy index 6a888fdbe06..19b6b956680 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy @@ -49,7 +49,7 @@ mavenJob('bookkeeper_precommit_pullrequest_java8') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'Build (Java 8)', + 'Build (Java 8) (trigger via `rebuild java8`)', '.*(re)?build java8.*', '.*\\[x\\] \\[skip build java8\\].*') diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy index 6b02857bb94..268d829d5d5 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy @@ -49,7 +49,7 @@ mavenJob('bookkeeper_precommit_pullrequest_java9') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'Build (Java 9)', + 'Build (Java 9) (trigger via `rebuild java9`)', '.*(re)?build java9.*', '.*\\[x\\] \\[skip build java9\\].*') diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy index f3f560393da..d713145e1a0 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy @@ -44,7 +44,7 @@ freeStyleJob('bookkeeper_precommit_remaining_tests') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'All Other Tests', + 'All Other Tests (trigger via `run bookkeeper-server remaining tests`)', '.*(re)?run bookkeeper-server (remaining )?tests.*', '.*\\[x\\] \\[skip bookkeeper-server (remaining )?tests\\].*', true) diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy index 21342800582..8d4a2ff12a3 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy @@ -40,7 +40,7 @@ freeStyleJob('bookkeeper_precommit_replication_tests') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'Replication Tests', + 'Replication Tests (trigger via `run bookkeeper-server replication tests`)', '.*(re)?run bookkeeper-server (replication )?tests.*', '.*\\[x\\] \\[skip bookkeeper-server (replication )?tests\\].*', true) diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy index af0038d4cf2..071bd1114a1 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy @@ -40,7 +40,7 @@ freeStyleJob('bookkeeper_precommit_tls_tests') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'TLS Tests', + 'TLS Tests (trigger via `run bookkeeper-server tls tests`)', '.*(re)?run bookkeeper-server (tls )?tests.*', '.*\\[x\\] \\[skip bookkeeper-server (tls )?tests\\].*', true) diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy index e9336e070fc..e7e75aada62 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy @@ -40,7 +40,7 @@ mavenJob('bookkeeper_precommit_pullrequest_validation') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'PR Validation', + 'PR Validation (trigger via `run pr validation`)', '.*(re)?run pr validation.*', '', true) From 465557b29971a561eb34a46fac3ef727c3818ca6 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 12 Dec 2018 09:14:54 +0800 Subject: [PATCH 0168/1642] [STATS] [DOC] Add @StatsDoc annotation for entrylogger stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document the entrylogger stats. *Changes* - convert entrylogger stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Enrico Olivelli , Jia Zhai This closes #1871 from sijie/document_ledgerstorage_stats --- .../EntryLogManagerForEntryLogPerLedger.java | 29 +++++++++++++++++++ .../apache/bookkeeper/bookie/EntryLogger.java | 4 --- 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java index 8093b53b157..5e123dce0ec 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java @@ -21,6 +21,8 @@ package org.apache.bookkeeper.bookie; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ENTRYLOGGER_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ENTRYLOGS_PER_LEDGER; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.NUM_LEDGERS_HAVING_MULTIPLE_ENTRYLOGS; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.NUM_OF_WRITE_ACTIVE_LEDGERS; @@ -61,6 +63,7 @@ import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.bookkeeper.util.IOUtils; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap; @@ -115,11 +118,37 @@ private void setEntryLogWithDirInfo(BufferedLogChannelWithDirInfo entryLogWithDi } } + @StatsDoc( + name = ENTRYLOGGER_SCOPE, + category = CATEGORY_SERVER, + help = "EntryLogger related stats" + ) class EntryLogsPerLedgerCounter { + + @StatsDoc( + name = NUM_OF_WRITE_ACTIVE_LEDGERS, + help = "Number of write active ledgers" + ) private final Counter numOfWriteActiveLedgers; + @StatsDoc( + name = NUM_OF_WRITE_LEDGERS_REMOVED_CACHE_EXPIRY, + help = "Number of write ledgers removed after cache expiry" + ) private final Counter numOfWriteLedgersRemovedCacheExpiry; + @StatsDoc( + name = NUM_OF_WRITE_LEDGERS_REMOVED_CACHE_MAXSIZE, + help = "Number of write ledgers removed due to reach max cache size" + ) private final Counter numOfWriteLedgersRemovedCacheMaxSize; + @StatsDoc( + name = NUM_LEDGERS_HAVING_MULTIPLE_ENTRYLOGS, + help = "Number of ledgers having multiple entry logs" + ) private final Counter numLedgersHavingMultipleEntrylogs; + @StatsDoc( + name = ENTRYLOGS_PER_LEDGER, + help = "The distribution of number of entry logs per ledger" + ) private final OpStatsLogger entryLogsPerLedger; /* * ledgerIdEntryLogCounterCacheMap cache will be used to store count of diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index f5913b334de..6b41b6c958b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -82,9 +82,6 @@ public class EntryLogger { @VisibleForTesting static final int UNINITIALIZED_LOG_ID = -0xDEAD; - // Expose Stats - private final StatsLogger statsLogger; - static class BufferedLogChannel extends BufferedChannel { private final long logId; private final EntryLogMetadata entryLogMetadata; @@ -375,7 +372,6 @@ public EntryLogger(ServerConfiguration conf, this.recentlyCreatedEntryLogsStatus = new RecentEntryLogsStatus(logId + 1); this.entryLoggerAllocator = new EntryLoggerAllocator(conf, ledgerDirsManager, recentlyCreatedEntryLogsStatus, logId); - this.statsLogger = statsLogger; if (entryLogPerLedgerEnabled) { this.entryLogManager = new EntryLogManagerForEntryLogPerLedger(conf, ledgerDirsManager, entryLoggerAllocator, listeners, recentlyCreatedEntryLogsStatus, statsLogger); From 9d024c46f0da38227a25375df8abbe746457a13e Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 12 Dec 2018 09:19:29 +0800 Subject: [PATCH 0169/1642] [STATS] [DOC] Add @StatsDoc annotation for bookie state manager stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document bookie state manager stats. *Changes* - convert bookie state manager stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Enrico Olivelli This closes #1876 from sijie/bookie_state_stats --- .../bookkeeper/bookie/BookieStateManager.java | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java index 370e06d8207..54d6993e0ac 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java @@ -21,6 +21,8 @@ package org.apache.bookkeeper.bookie; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SERVER_STATUS; import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -42,6 +44,7 @@ import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.bookkeeper.util.DiskChecker; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,6 +53,11 @@ * An implementation of StateManager. */ @Slf4j +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "Bookie state manager related stats" +) public class BookieStateManager implements StateManager { private static final Logger LOG = LoggerFactory.getLogger(BookieStateManager.class); private final ServerConfiguration conf; @@ -73,7 +81,11 @@ public class BookieStateManager implements StateManager { private ShutdownHandler shutdownHandler; private final Supplier rm; // Expose Stats - private final StatsLogger statsLogger; + @StatsDoc( + name = SERVER_STATUS, + help = "Bookie status (1: up, 0: readonly, -1: unregistered)" + ) + private final Gauge serverStatusGauge; public BookieStateManager(ServerConfiguration conf, StatsLogger statsLogger, @@ -98,13 +110,12 @@ public BookieStateManager(ServerConfiguration conf, List statusDirs, Supplier bookieIdSupplier) throws IOException { this.conf = conf; - this.statsLogger = statsLogger; this.rm = rm; this.statusDirs = statusDirs; // ZK ephemeral node for this Bookie. this.bookieId = bookieIdSupplier.get(); // 1 : up, 0 : readonly, -1 : unregistered - statsLogger.registerGauge(SERVER_STATUS, new Gauge() { + this.serverStatusGauge = new Gauge() { @Override public Number getDefaultValue() { return 0; @@ -120,7 +131,8 @@ public Number getSample() { return 1; } } - }); + }; + statsLogger.registerGauge(SERVER_STATUS, serverStatusGauge); } private boolean isRegistrationManagerDisabled() { From 71af09899913f66889509d5a125e0b27315042b8 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 12 Dec 2018 09:20:52 +0800 Subject: [PATCH 0170/1642] [STATS] [DOC] Add @StatsDoc annotation for interleaved ledger storage stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document interleaved ledger storage stats. *Changes* - convert interleaved ledger storage stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Enrico Olivelli This closes #1877 from sijie/interleaved_stats --- .../bookie/InterleavedLedgerStorage.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java index 4a96ece6a46..0a0e60da37e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java @@ -22,6 +22,9 @@ package org.apache.bookkeeper.bookie; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_READ_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ENTRYLOGGER_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_GET_ENTRY; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_GET_OFFSET; @@ -53,6 +56,7 @@ import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.SnapshotMap; import org.apache.commons.lang.mutable.MutableBoolean; @@ -66,6 +70,11 @@ *

            This ledger storage implementation stores all entries in a single * file and maintains an index file for each ledger. */ +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "Bookie related stats" +) public class InterleavedLedgerStorage implements CompactableLedgerStorage, EntryLogListener { private static final Logger LOG = LoggerFactory.getLogger(InterleavedLedgerStorage.class); @@ -88,7 +97,18 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry private final AtomicBoolean somethingWritten = new AtomicBoolean(false); // Expose Stats + @StatsDoc( + name = STORAGE_GET_OFFSET, + help = "Operation stats of getting offset from ledger cache", + parent = BOOKIE_READ_ENTRY + ) private OpStatsLogger getOffsetStats; + @StatsDoc( + name = STORAGE_GET_ENTRY, + help = "Operation stats of getting entry from entry logger", + parent = BOOKIE_READ_ENTRY, + happensAfter = STORAGE_GET_OFFSET + ) private OpStatsLogger getEntryStats; private OpStatsLogger pageScanStats; From 12eca7cc73b837d9f01608941fb993e074311cb4 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 12 Dec 2018 09:23:07 +0800 Subject: [PATCH 0171/1642] [STATS] [DOC] Add @StatsDoc annotation for bookkeeper client stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document bookkeeper client stats. *Changes* - convert bookkeeper client stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Enrico Olivelli This closes #1878 from sijie/client_stats --- .../client/BookKeeperClientStats.java | 117 +------- .../bookkeeper/client/BookieWatcherImpl.java | 16 ++ .../client/LedgerFragmentReplicator.java | 35 ++- .../RackawareEnsemblePlacementPolicyImpl.java | 28 +- .../impl/BookKeeperClientStatsImpl.java | 259 ++++++++++++++++++ .../proto/PerChannelBookieClient.java | 97 +++++++ 6 files changed, 430 insertions(+), 122 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperClientStatsImpl.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java index 2fa6753b105..8b358b6ead1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java @@ -21,6 +21,7 @@ package org.apache.bookkeeper.client; +import org.apache.bookkeeper.client.impl.BookKeeperClientStatsImpl; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.OpStatsLogger; @@ -30,6 +31,8 @@ * List of constants for defining client stats names. */ public interface BookKeeperClientStats { + String CATEGORY_CLIENT = "client"; + String CLIENT_SCOPE = "bookkeeper_client"; // Metadata Operations @@ -114,119 +117,7 @@ public interface BookKeeperClientStats { void registerPendingAddsGauge(Gauge gauge); static BookKeeperClientStats newInstance(StatsLogger stats) { - OpStatsLogger createOpLogger = stats.getOpStatsLogger(CREATE_OP); - OpStatsLogger deleteOpLogger = stats.getOpStatsLogger(DELETE_OP); - OpStatsLogger openOpLogger = stats.getOpStatsLogger(OPEN_OP); - OpStatsLogger recoverOpLogger = stats.getOpStatsLogger(RECOVER_OP); - OpStatsLogger readOpLogger = stats.getOpStatsLogger(READ_OP); - Counter readOpDmCounter = stats.getCounter(READ_OP_DM); - OpStatsLogger readLacAndEntryOpLogger = stats.getOpStatsLogger(READ_LAST_CONFIRMED_AND_ENTRY); - OpStatsLogger readLacAndEntryRespLogger = stats.getOpStatsLogger(READ_LAST_CONFIRMED_AND_ENTRY_RESPONSE); - OpStatsLogger addOpLogger = stats.getOpStatsLogger(ADD_OP); - OpStatsLogger forceOpLogger = stats.getOpStatsLogger(FORCE_OP); - Counter addOpUrCounter = stats.getCounter(ADD_OP_UR); - OpStatsLogger writeLacOpLogger = stats.getOpStatsLogger(WRITE_LAC_OP); - OpStatsLogger readLacOpLogger = stats.getOpStatsLogger(READ_LAC_OP); - OpStatsLogger recoverAddEntriesStats = stats.getOpStatsLogger(LEDGER_RECOVER_ADD_ENTRIES); - OpStatsLogger recoverReadEntriesStats = stats.getOpStatsLogger(LEDGER_RECOVER_READ_ENTRIES); - - Counter ensembleChangeCounter = stats.getCounter(ENSEMBLE_CHANGES); - Counter lacUpdateHitsCounter = stats.getCounter(LAC_UPDATE_HITS); - Counter lacUpdateMissesCounter = stats.getCounter(LAC_UPDATE_MISSES); - OpStatsLogger clientChannelWriteWaitStats = stats.getOpStatsLogger(CLIENT_CHANNEL_WRITE_WAIT); - - Counter speculativeReadCounter = stats.getCounter(SPECULATIVE_READ_COUNT); - - return new BookKeeperClientStats() { - @Override - public OpStatsLogger getCreateOpLogger() { - return createOpLogger; - } - @Override - public OpStatsLogger getOpenOpLogger() { - return openOpLogger; - } - @Override - public OpStatsLogger getDeleteOpLogger() { - return deleteOpLogger; - } - @Override - public OpStatsLogger getRecoverOpLogger() { - return recoverOpLogger; - } - @Override - public OpStatsLogger getReadOpLogger() { - return readOpLogger; - } - @Override - public OpStatsLogger getReadLacAndEntryOpLogger() { - return readLacAndEntryOpLogger; - } - @Override - public OpStatsLogger getReadLacAndEntryRespLogger() { - return readLacAndEntryRespLogger; - } - @Override - public OpStatsLogger getAddOpLogger() { - return addOpLogger; - } - @Override - public OpStatsLogger getForceOpLogger() { - return forceOpLogger; - } - @Override - public OpStatsLogger getWriteLacOpLogger() { - return writeLacOpLogger; - } - @Override - public OpStatsLogger getReadLacOpLogger() { - return readLacOpLogger; - } - @Override - public OpStatsLogger getRecoverAddCountLogger() { - return recoverAddEntriesStats; - } - @Override - public OpStatsLogger getRecoverReadCountLogger() { - return recoverReadEntriesStats; - } - @Override - public Counter getReadOpDmCounter() { - return readOpDmCounter; - } - @Override - public Counter getAddOpUrCounter() { - return addOpUrCounter; - } - @Override - public Counter getSpeculativeReadCounter() { - return speculativeReadCounter; - } - @Override - public Counter getEnsembleChangeCounter() { - return ensembleChangeCounter; - } - @Override - public Counter getLacUpdateHitsCounter() { - return lacUpdateHitsCounter; - } - @Override - public Counter getLacUpdateMissesCounter() { - return lacUpdateMissesCounter; - } - @Override - public OpStatsLogger getClientChannelWriteWaitLogger() { - return clientChannelWriteWaitStats; - } - @Override - public Counter getEnsembleBookieDistributionCounter(String bookie) { - return stats.getCounter(LEDGER_ENSEMBLE_BOOKIE_DISTRIBUTION + "-" + bookie); - } - @Override - public void registerPendingAddsGauge(Gauge gauge) { - stats.registerGauge(PENDING_ADDS, gauge); - } - }; + return new BookKeeperClientStatsImpl(stats); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index 2941a41fa1f..8b14b77174a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -19,6 +19,8 @@ import static org.apache.bookkeeper.bookie.BookKeeperServerStats.NEW_ENSEMBLE_TIME; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.REPLACE_BOOKIE_TIME; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WATCHER_SCOPE; +import static org.apache.bookkeeper.client.BookKeeperClientStats.CREATE_OP; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; @@ -43,6 +45,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; /** * This class is responsible for maintaining a consistent view of what bookies @@ -51,6 +54,10 @@ * replacement * */ +@StatsDoc( + name = WATCHER_SCOPE, + help = "Bookie watcher related stats" +) @Slf4j class BookieWatcherImpl implements BookieWatcher { @@ -70,7 +77,16 @@ class BookieWatcherImpl implements BookieWatcher { private final ClientConfiguration conf; private final RegistrationClient registrationClient; private final EnsemblePlacementPolicy placementPolicy; + @StatsDoc( + name = NEW_ENSEMBLE_TIME, + help = "operation stats of new ensembles", + parent = CREATE_OP + ) private final OpStatsLogger newEnsembleTimer; + @StatsDoc( + name = REPLACE_BOOKIE_TIME, + help = "operation stats of replacing bookie in an ensemble" + ) private final OpStatsLogger replaceBookieTimer; // Bookies that will not be preferred to be chosen in a new ensemble 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 28929cf9a8d..765dec026e5 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 @@ -20,6 +20,11 @@ package org.apache.bookkeeper.client; import static org.apache.bookkeeper.client.LedgerHandle.INVALID_ENTRY_ID; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_BYTES_READ; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_BYTES_WRITTEN; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_READ; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_WRITTEN; +import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_WORKER_SCOPE; import io.netty.buffer.Unpooled; @@ -41,11 +46,11 @@ import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.MultiCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; -import org.apache.bookkeeper.replication.ReplicationStats; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.bookkeeper.util.ByteBufList; import org.apache.zookeeper.AsyncCallback; import org.slf4j.Logger; @@ -55,23 +60,43 @@ * This is the helper class for replicating the fragments from one bookie to * another. */ +@StatsDoc( + name = REPLICATION_WORKER_SCOPE, + help = "Ledger fragment replicator related stats" +) public class LedgerFragmentReplicator { // BookKeeper instance private BookKeeper bkc; private StatsLogger statsLogger; + @StatsDoc( + name = NUM_ENTRIES_READ, + help = "Number of entries read by the replicator" + ) private final Counter numEntriesRead; + @StatsDoc( + name = NUM_BYTES_READ, + help = "The distribution of size of entries read by the replicator" + ) private final OpStatsLogger numBytesRead; + @StatsDoc( + name = NUM_ENTRIES_WRITTEN, + help = "Number of entries written by the replicator" + ) private final Counter numEntriesWritten; + @StatsDoc( + name = NUM_BYTES_WRITTEN, + help = "The distribution of size of entries written by the replicator" + ) private final OpStatsLogger numBytesWritten; public LedgerFragmentReplicator(BookKeeper bkc, StatsLogger statsLogger) { this.bkc = bkc; this.statsLogger = statsLogger; - numEntriesRead = this.statsLogger.getCounter(ReplicationStats.NUM_ENTRIES_READ); - numBytesRead = this.statsLogger.getOpStatsLogger(ReplicationStats.NUM_BYTES_READ); - numEntriesWritten = this.statsLogger.getCounter(ReplicationStats.NUM_ENTRIES_WRITTEN); - numBytesWritten = this.statsLogger.getOpStatsLogger(ReplicationStats.NUM_BYTES_WRITTEN); + numEntriesRead = this.statsLogger.getCounter(NUM_ENTRIES_READ); + numBytesRead = this.statsLogger.getOpStatsLogger(NUM_BYTES_READ); + numEntriesWritten = this.statsLogger.getCounter(NUM_ENTRIES_WRITTEN); + numBytesWritten = this.statsLogger.getOpStatsLogger(NUM_BYTES_WRITTEN); } public LedgerFragmentReplicator(BookKeeper bkc) { 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 578d49e7ed9..7578c416ee1 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 @@ -18,6 +18,10 @@ package org.apache.bookkeeper.client; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_JOINED; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_LEFT; +import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE; +import static org.apache.bookkeeper.client.BookKeeperClientStats.READ_REQUESTS_REORDERED; import static org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy.UNKNOWN_REGION; import com.google.common.cache.Cache; @@ -44,7 +48,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Supplier; -import org.apache.bookkeeper.bookie.BookKeeperServerStats; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; @@ -63,6 +66,7 @@ import org.apache.bookkeeper.net.StabilizeNetworkTopology; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.commons.collections4.CollectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,6 +76,10 @@ * *

            Make most of the class and methods as protected, so it could be extended to implement other algorithms. */ +@StatsDoc( + name = CLIENT_SCOPE, + help = "BookKeeper client stats" +) public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsemblePlacementPolicy { static final Logger LOG = LoggerFactory.getLogger(RackawareEnsemblePlacementPolicyImpl.class); @@ -204,8 +212,20 @@ public void reloadCachedMappings() { // looks like these only assigned in the same thread as constructor, immediately after constructor; // no need to make volatile protected StatsLogger statsLogger = null; + @StatsDoc( + name = BOOKIES_JOINED, + help = "The distribution of number of bookies joined the cluster on each network topology change" + ) protected OpStatsLogger bookiesJoinedCounter = null; + @StatsDoc( + name = BOOKIES_LEFT, + help = "The distribution of number of bookies left the cluster on each network topology change" + ) protected OpStatsLogger bookiesLeftCounter = null; + @StatsDoc( + name = READ_REQUESTS_REORDERED, + help = "The distribution of number of bookies reordered on each read request" + ) protected OpStatsLogger readReorderedCounter = null; private String defaultRack = NetworkTopology.DEFAULT_RACK; @@ -244,9 +264,9 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns StatsLogger statsLogger) { checkNotNull(statsLogger, "statsLogger should not be null, use NullStatsLogger instead."); this.statsLogger = statsLogger; - this.bookiesJoinedCounter = statsLogger.getOpStatsLogger(BookKeeperServerStats.BOOKIES_JOINED); - this.bookiesLeftCounter = statsLogger.getOpStatsLogger(BookKeeperServerStats.BOOKIES_LEFT); - this.readReorderedCounter = statsLogger.getOpStatsLogger(BookKeeperClientStats.READ_REQUESTS_REORDERED); + this.bookiesJoinedCounter = statsLogger.getOpStatsLogger(BOOKIES_JOINED); + this.bookiesLeftCounter = statsLogger.getOpStatsLogger(BOOKIES_LEFT); + this.readReorderedCounter = statsLogger.getOpStatsLogger(READ_REQUESTS_REORDERED); this.reorderReadsRandom = reorderReadsRandom; this.stabilizePeriodSeconds = stabilizePeriodSeconds; this.reorderThresholdPendingRequests = reorderThresholdPendingRequests; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperClientStatsImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperClientStatsImpl.java new file mode 100644 index 00000000000..a29621a0d00 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperClientStatsImpl.java @@ -0,0 +1,259 @@ +/* + * 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.client.impl; + +import static org.apache.bookkeeper.client.BookKeeperClientStats.CATEGORY_CLIENT; +import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE; + +import org.apache.bookkeeper.client.BookKeeperClientStats; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * The default implementation of {@link BookKeeperClientStats}. + */ +@StatsDoc( + name = CLIENT_SCOPE, + category = CATEGORY_CLIENT, + help = "BookKeeper client stats" +) +public class BookKeeperClientStatsImpl implements BookKeeperClientStats { + private final StatsLogger stats; + @StatsDoc( + name = CREATE_OP, + help = "operation stats of creating ledgers" + ) + private final OpStatsLogger createOpLogger; + @StatsDoc( + name = DELETE_OP, + help = "operation stats of deleting ledgers" + ) + private final OpStatsLogger deleteOpLogger; + @StatsDoc( + name = OPEN_OP, + help = "operation stats of opening ledgers" + ) + private final OpStatsLogger openOpLogger; + @StatsDoc( + name = RECOVER_OP, + help = "operation stats of recovering ledgers" + ) + private final OpStatsLogger recoverOpLogger; + @StatsDoc( + name = READ_OP, + help = "operation stats of reading entries requests" + ) + private final OpStatsLogger readOpLogger; + @StatsDoc( + name = READ_OP_DM, + help = "the number of read entries hitting DigestMismatch errors" + ) + private final Counter readOpDmCounter; + @StatsDoc( + name = READ_LAST_CONFIRMED_AND_ENTRY, + help = "operation stats of read_last_confirmed_and_entry requests" + ) + private final OpStatsLogger readLacAndEntryOpLogger; + @StatsDoc( + name = READ_LAST_CONFIRMED_AND_ENTRY_RESPONSE, + help = "operation stats of read_last_confirmed_and_entry responses" + ) + private final OpStatsLogger readLacAndEntryRespLogger; + @StatsDoc( + name = ADD_OP, + help = "operation stats of adding entries requests" + ) + private final OpStatsLogger addOpLogger; + @StatsDoc( + name = FORCE_OP, + help = "operation stats of force requests" + ) + private final OpStatsLogger forceOpLogger; + @StatsDoc( + name = ADD_OP_UR, + help = "the number of add entries under replication" + ) + private final Counter addOpUrCounter; + @StatsDoc( + name = WRITE_LAC_OP, + help = "operation stats of write_lac requests" + ) + private final OpStatsLogger writeLacOpLogger; + @StatsDoc( + name = READ_LAC_OP, + help = "operation stats of read_lac requests" + ) + private final OpStatsLogger readLacOpLogger; + @StatsDoc( + name = LEDGER_RECOVER_ADD_ENTRIES, + help = "the distribution of entries written in ledger recovery requests" + ) + private final OpStatsLogger recoverAddEntriesStats; + @StatsDoc( + name = LEDGER_RECOVER_READ_ENTRIES, + help = "the distribution of entries read in ledger recovery requests" + ) + private final OpStatsLogger recoverReadEntriesStats; + + @StatsDoc( + name = ENSEMBLE_CHANGES, + help = "The number of ensemble changes" + ) + private final Counter ensembleChangeCounter; + @StatsDoc( + name = LAC_UPDATE_HITS, + help = "The number of successful lac updates on piggybacked responses" + ) + private final Counter lacUpdateHitsCounter; + @StatsDoc( + name = LAC_UPDATE_MISSES, + help = "The number of unsuccessful lac updates on piggybacked responses" + ) + private final Counter lacUpdateMissesCounter; + @StatsDoc( + name = CLIENT_CHANNEL_WRITE_WAIT, + help = " The latency distribution of waiting time on channel being writable" + ) + private final OpStatsLogger clientChannelWriteWaitStats; + @StatsDoc( + name = SPECULATIVE_READ_COUNT, + help = "The number of speculative read requests" + ) + private final Counter speculativeReadCounter; + + + public BookKeeperClientStatsImpl(StatsLogger stats) { + this.stats = stats; + this.createOpLogger = stats.getOpStatsLogger(CREATE_OP); + this.deleteOpLogger = stats.getOpStatsLogger(DELETE_OP); + this.openOpLogger = stats.getOpStatsLogger(OPEN_OP); + this.recoverOpLogger = stats.getOpStatsLogger(RECOVER_OP); + this.readOpLogger = stats.getOpStatsLogger(READ_OP); + this.readOpDmCounter = stats.getCounter(READ_OP_DM); + this.readLacAndEntryOpLogger = stats.getOpStatsLogger(READ_LAST_CONFIRMED_AND_ENTRY); + this.readLacAndEntryRespLogger = stats.getOpStatsLogger(READ_LAST_CONFIRMED_AND_ENTRY_RESPONSE); + this.addOpLogger = stats.getOpStatsLogger(ADD_OP); + this.forceOpLogger = stats.getOpStatsLogger(FORCE_OP); + this.addOpUrCounter = stats.getCounter(ADD_OP_UR); + this.writeLacOpLogger = stats.getOpStatsLogger(WRITE_LAC_OP); + this.readLacOpLogger = stats.getOpStatsLogger(READ_LAC_OP); + this.recoverAddEntriesStats = stats.getOpStatsLogger(LEDGER_RECOVER_ADD_ENTRIES); + this.recoverReadEntriesStats = stats.getOpStatsLogger(LEDGER_RECOVER_READ_ENTRIES); + + this.ensembleChangeCounter = stats.getCounter(ENSEMBLE_CHANGES); + this.lacUpdateHitsCounter = stats.getCounter(LAC_UPDATE_HITS); + this.lacUpdateMissesCounter = stats.getCounter(LAC_UPDATE_MISSES); + this.clientChannelWriteWaitStats = stats.getOpStatsLogger(CLIENT_CHANNEL_WRITE_WAIT); + + speculativeReadCounter = stats.getCounter(SPECULATIVE_READ_COUNT); + } + + @Override + public OpStatsLogger getCreateOpLogger() { + return createOpLogger; + } + @Override + public OpStatsLogger getOpenOpLogger() { + return openOpLogger; + } + @Override + public OpStatsLogger getDeleteOpLogger() { + return deleteOpLogger; + } + @Override + public OpStatsLogger getRecoverOpLogger() { + return recoverOpLogger; + } + @Override + public OpStatsLogger getReadOpLogger() { + return readOpLogger; + } + @Override + public OpStatsLogger getReadLacAndEntryOpLogger() { + return readLacAndEntryOpLogger; + } + @Override + public OpStatsLogger getReadLacAndEntryRespLogger() { + return readLacAndEntryRespLogger; + } + @Override + public OpStatsLogger getAddOpLogger() { + return addOpLogger; + } + @Override + public OpStatsLogger getForceOpLogger() { + return forceOpLogger; + } + @Override + public OpStatsLogger getWriteLacOpLogger() { + return writeLacOpLogger; + } + @Override + public OpStatsLogger getReadLacOpLogger() { + return readLacOpLogger; + } + @Override + public OpStatsLogger getRecoverAddCountLogger() { + return recoverAddEntriesStats; + } + @Override + public OpStatsLogger getRecoverReadCountLogger() { + return recoverReadEntriesStats; + } + @Override + public Counter getReadOpDmCounter() { + return readOpDmCounter; + } + @Override + public Counter getAddOpUrCounter() { + return addOpUrCounter; + } + @Override + public Counter getSpeculativeReadCounter() { + return speculativeReadCounter; + } + @Override + public Counter getEnsembleChangeCounter() { + return ensembleChangeCounter; + } + @Override + public Counter getLacUpdateHitsCounter() { + return lacUpdateHitsCounter; + } + @Override + public Counter getLacUpdateMissesCounter() { + return lacUpdateMissesCounter; + } + @Override + public OpStatsLogger getClientChannelWriteWaitLogger() { + return clientChannelWriteWaitStats; + } + @Override + public Counter getEnsembleBookieDistributionCounter(String bookie) { + return stats.getCounter(LEDGER_ENSEMBLE_BOOKIE_DISTRIBUTION + "-" + bookie); + } + @Override + public void registerPendingAddsGauge(Gauge gauge) { + stats.registerGauge(PENDING_ADDS, gauge); + } +} 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 917f6f3323e..6839afc8f25 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 @@ -125,6 +125,7 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.tls.SecurityHandlerFactory; import org.apache.bookkeeper.tls.SecurityHandlerFactory.NodeType; @@ -142,6 +143,10 @@ * This class manages all details of connection to a particular bookie. It also * has reconnect logic if a connection to a bookie fails. */ +@StatsDoc( + name = BookKeeperClientStats.CHANNEL_SCOPE, + help = "Per channel bookie client stats" +) @Sharable public class PerChannelBookieClient extends ChannelInboundHandlerAdapter { @@ -177,29 +182,121 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter { new SynchronizedHashMultiMap<>(); private final StatsLogger statsLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_READ_OP, + help = "channel stats of read entries requests" + ) private final OpStatsLogger readEntryOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_TIMEOUT_READ, + help = "timeout stats of read entries requests" + ) private final OpStatsLogger readTimeoutOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_ADD_OP, + help = "channel stats of add entries requests" + ) private final OpStatsLogger addEntryOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_WRITE_LAC_OP, + help = "channel stats of write_lac requests" + ) private final OpStatsLogger writeLacOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_FORCE_OP, + help = "channel stats of force requests" + ) private final OpStatsLogger forceLedgerOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_READ_LAC_OP, + help = "channel stats of read_lac requests" + ) private final OpStatsLogger readLacOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_TIMEOUT_ADD, + help = "timeout stats of add entries requests" + ) private final OpStatsLogger addTimeoutOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_TIMEOUT_WRITE_LAC, + help = "timeout stats of write_lac requests" + ) private final OpStatsLogger writeLacTimeoutOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_TIMEOUT_FORCE, + help = "timeout stats of force requests" + ) private final OpStatsLogger forceLedgerTimeoutOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_TIMEOUT_READ_LAC, + help = "timeout stats of read_lac requests" + ) private final OpStatsLogger readLacTimeoutOpLogger; + @StatsDoc( + name = BookKeeperClientStats.GET_BOOKIE_INFO_OP, + help = "channel stats of get_bookie_info requests" + ) private final OpStatsLogger getBookieInfoOpLogger; + @StatsDoc( + name = BookKeeperClientStats.TIMEOUT_GET_BOOKIE_INFO, + help = "timeout stats of get_bookie_info requests" + ) private final OpStatsLogger getBookieInfoTimeoutOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_START_TLS_OP, + help = "channel stats of start_tls requests" + ) private final OpStatsLogger startTLSOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CHANNEL_TIMEOUT_START_TLS_OP, + help = "timeout stats of start_tls requests" + ) private final OpStatsLogger startTLSTimeoutOpLogger; + @StatsDoc( + name = BookKeeperClientStats.CLIENT_CONNECT_TIMER, + help = "channel stats of connect requests" + ) private final OpStatsLogger connectTimer; + @StatsDoc( + name = BookKeeperClientStats.NETTY_EXCEPTION_CNT, + help = "the number of exceptions received from this channel" + ) private final Counter exceptionCounter; + @StatsDoc( + name = BookKeeperClientStats.ADD_OP_OUTSTANDING, + help = "the number of outstanding add_entry requests" + ) private final Counter addEntryOutstanding; + @StatsDoc( + name = BookKeeperClientStats.READ_OP_OUTSTANDING, + help = "the number of outstanding add_entry requests" + ) private final Counter readEntryOutstanding; /* collect stats on all Ops that flows through netty pipeline */ + @StatsDoc( + name = BookKeeperClientStats.NETTY_OPS, + help = "channel stats for all operations flowing through netty pipeline" + ) private final OpStatsLogger nettyOpLogger; + @StatsDoc( + name = BookKeeperClientStats.ACTIVE_NON_TLS_CHANNEL_COUNTER, + help = "the number of active non-tls channels" + ) private final Counter activeNonTlsChannelCounter; + @StatsDoc( + name = BookKeeperClientStats.ACTIVE_TLS_CHANNEL_COUNTER, + help = "the number of active tls channels" + ) private final Counter activeTlsChannelCounter; + @StatsDoc( + name = BookKeeperClientStats.FAILED_CONNECTION_COUNTER, + help = "the number of failed connections" + ) private final Counter failedConnectionCounter; + @StatsDoc( + name = BookKeeperClientStats.FAILED_TLS_HANDSHAKE_COUNTER, + help = "the number of failed tls handshakes" + ) private final Counter failedTlsHandshakeCounter; private final boolean useV2WireProtocol; From 8ea469ddf865e74fda782303723fcf28f2e914ad Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 13 Dec 2018 22:41:34 +0800 Subject: [PATCH 0172/1642] [STATS] [DOC] Add @StatsDoc annotation for memtable stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document the memtable stats. *Changes* - convert memtable stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Jia Zhai , Enrico Olivelli This closes #1873 from sijie/memtable_stats --- .../bookkeeper/bookie/EntryMemTable.java | 56 +++++------- .../EntryMemTableWithParallelFlusher.java | 2 +- .../bookie/stats/EntryMemTableStats.java | 91 +++++++++++++++++++ 3 files changed, 115 insertions(+), 34 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/EntryMemTableStats.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java index 0a95fe92158..a27e54eb2d7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java @@ -19,13 +19,6 @@ package org.apache.bookkeeper.bookie; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_FLUSH_BYTES; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_GET_ENTRY; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_PUT_ENTRY; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_SNAPSHOT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_THROTTLING; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_THROTTLING_LATENCY; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.concurrent.ConcurrentSkipListMap; @@ -36,9 +29,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.bookkeeper.bookie.Bookie.NoLedgerException; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; +import org.apache.bookkeeper.bookie.stats.EntryMemTableStats; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.slf4j.Logger; @@ -117,12 +109,7 @@ private EntrySkipList newSkipList() { } // Stats - private final OpStatsLogger snapshotStats; - private final OpStatsLogger putEntryStats; - private final OpStatsLogger getEntryStats; - final Counter flushBytesCounter; - private final Counter throttlingCounter; - private final OpStatsLogger throttlingStats; + protected final EntryMemTableStats memTableStats; /** * Constructor. @@ -150,12 +137,7 @@ public EntryMemTable(final ServerConfiguration conf, final CheckpointSource sour this.skipListSemaphore = new Semaphore((int) skipListSizeLimit * 2); // Stats - this.snapshotStats = statsLogger.getOpStatsLogger(SKIP_LIST_SNAPSHOT); - this.putEntryStats = statsLogger.getOpStatsLogger(SKIP_LIST_PUT_ENTRY); - this.getEntryStats = statsLogger.getOpStatsLogger(SKIP_LIST_GET_ENTRY); - this.flushBytesCounter = statsLogger.getCounter(SKIP_LIST_FLUSH_BYTES); - this.throttlingCounter = statsLogger.getCounter(SKIP_LIST_THROTTLING); - this.throttlingStats = statsLogger.getOpStatsLogger(SKIP_LIST_THROTTLING_LATENCY); + this.memTableStats = new EntryMemTableStats(statsLogger); } void dump() { @@ -203,9 +185,11 @@ Checkpoint snapshot(Checkpoint oldCp) throws IOException { } if (null != cp) { - snapshotStats.registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + memTableStats.getSnapshotStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } else { - snapshotStats.registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + memTableStats.getSnapshotStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } } return cp; @@ -272,7 +256,7 @@ long flushSnapshot(final SkipListFlusher flusher, Checkpoint checkpoint) throws } } } - flushBytesCounter.add(size); + memTableStats.getFlushBytesCounter().add(size); clearSnapshot(keyValues); } } @@ -322,11 +306,11 @@ public long addEntry(long ledgerId, long entryId, final ByteBuffer entry, final final int len = entry.remaining(); if (!skipListSemaphore.tryAcquire(len)) { - throttlingCounter.inc(); + memTableStats.getThrottlingCounter().inc(); final long throttlingStartTimeNanos = MathUtils.nowInNano(); skipListSemaphore.acquireUninterruptibly(len); - throttlingStats.registerSuccessfulEvent(MathUtils.elapsedNanos(throttlingStartTimeNanos), - TimeUnit.NANOSECONDS); + memTableStats.getThrottlingStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(throttlingStartTimeNanos), TimeUnit.NANOSECONDS); } this.lock.readLock().lock(); @@ -340,9 +324,11 @@ public long addEntry(long ledgerId, long entryId, final ByteBuffer entry, final return size; } finally { if (success) { - putEntryStats.registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + memTableStats.getPutEntryStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } else { - putEntryStats.registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + memTableStats.getPutEntryStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } } } @@ -406,9 +392,11 @@ public EntryKeyValue getEntry(long ledgerId, long entryId) throws IOException { } finally { this.lock.readLock().unlock(); if (success) { - getEntryStats.registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + memTableStats.getGetEntryStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } else { - getEntryStats.registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + memTableStats.getGetEntryStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } } @@ -435,9 +423,11 @@ public EntryKeyValue getLastEntry(long ledgerId) throws IOException { } finally { this.lock.readLock().unlock(); if (success) { - getEntryStats.registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + memTableStats.getGetEntryStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } else { - getEntryStats.registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + memTableStats.getGetEntryStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTableWithParallelFlusher.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTableWithParallelFlusher.java index 4f2cf022923..d092f2064ce 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTableWithParallelFlusher.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTableWithParallelFlusher.java @@ -139,7 +139,7 @@ public void safeRun() { throw new IOException("Failed to complete the flushSnapshotByParallelizing", exceptionWhileFlushingParallelly.get()); } - flushBytesCounter.add(flushedSize.get()); + memTableStats.getFlushBytesCounter().add(flushedSize.get()); clearSnapshot(keyValues); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/EntryMemTableStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/EntryMemTableStats.java new file mode 100644 index 00000000000..62840f9c13c --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/EntryMemTableStats.java @@ -0,0 +1,91 @@ +/* + * 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.stats; + +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_ADD_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_FLUSH_BYTES; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_GET_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_PUT_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_SNAPSHOT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_THROTTLING; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SKIP_LIST_THROTTLING_LATENCY; + +import lombok.Getter; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for memtable related stats. + */ +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "EntryMemTable related stats" +) +@Getter +public class EntryMemTableStats { + + @StatsDoc( + name = SKIP_LIST_SNAPSHOT, + help = "operation stats of taking memtable snapshots" + ) + private final OpStatsLogger snapshotStats; + @StatsDoc( + name = SKIP_LIST_PUT_ENTRY, + help = "operation stats of putting entries to memtable", + parent = BOOKIE_ADD_ENTRY + ) + private final OpStatsLogger putEntryStats; + @StatsDoc( + name = SKIP_LIST_GET_ENTRY, + help = "operation stats of getting entries from memtable", + parent = BOOKIE_ADD_ENTRY + ) + private final OpStatsLogger getEntryStats; + @StatsDoc( + name = SKIP_LIST_FLUSH_BYTES, + help = "The number of bytes flushed from memtable to entry log files" + ) + private final Counter flushBytesCounter; + @StatsDoc( + name = SKIP_LIST_THROTTLING, + help = "The number of requests throttled due to memtables are full" + ) + private final Counter throttlingCounter; + @StatsDoc( + name = SKIP_LIST_THROTTLING_LATENCY, + help = "The distribution of request throttled duration" + ) + private final OpStatsLogger throttlingStats; + + public EntryMemTableStats(StatsLogger statsLogger) { + this.snapshotStats = statsLogger.getOpStatsLogger(SKIP_LIST_SNAPSHOT); + this.putEntryStats = statsLogger.getOpStatsLogger(SKIP_LIST_PUT_ENTRY); + this.getEntryStats = statsLogger.getOpStatsLogger(SKIP_LIST_GET_ENTRY); + this.flushBytesCounter = statsLogger.getCounter(SKIP_LIST_FLUSH_BYTES); + this.throttlingCounter = statsLogger.getCounter(SKIP_LIST_THROTTLING); + this.throttlingStats = statsLogger.getOpStatsLogger(SKIP_LIST_THROTTLING_LATENCY); + } + +} From 69a240a0ab2d5829c2748a50eacaddec08c037da Mon Sep 17 00:00:00 2001 From: Jia Zhai Date: Thu, 13 Dec 2018 22:45:09 +0800 Subject: [PATCH 0173/1642] Add get method for rest endpoint gc Descriptions of the changes in this PR: Add get method for rest endpoint gc to get force triggered GC running status on Bookie. true -- force triggered GC is running on Bookie. false -- not running. ### Motivation This is base on PR #1838, and in the review comments sijie is suggested to add get methods. ### Changes Add get method for rest endpoint gc and unit test. Reviewers: Sijie Guo This closes #1840 from jiazhai/rest_gc_get --- .../apache/bookkeeper/http/HttpRouter.java | 2 + .../apache/bookkeeper/http/HttpServer.java | 2 + .../bookie/GarbageCollectionStatus.java | 47 ++++++++++ .../bookie/GarbageCollectorThread.java | 24 ++++++ .../bookie/InterleavedLedgerStorage.java | 31 ++++--- .../bookkeeper/bookie/LedgerStorage.java | 21 +++++ .../bookie/SortedLedgerStorage.java | 10 +++ .../bookie/storage/ldb/DbLedgerStorage.java | 13 +++ .../ldb/SingleDirectoryDbLedgerStorage.java | 13 +++ .../server/http/BKHttpServiceProvider.java | 3 + .../server/http/service/GCDetailsService.java | 85 +++++++++++++++++++ .../server/http/service/TriggerGCService.java | 25 +++++- .../server/http/TestHttpService.java | 52 ++++++++++-- site/docs/latest/admin/http.md | 39 +++++++++ 14 files changed, 349 insertions(+), 18 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectionStatus.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GCDetailsService.java diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java index 5384df6520e..b4251d7327c 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java @@ -46,6 +46,7 @@ public abstract class HttpRouter { public static final String LIST_DISK_FILE = "/api/v1/bookie/list_disk_file"; public static final String EXPAND_STORAGE = "/api/v1/bookie/expand_storage"; public static final String GC = "/api/v1/bookie/gc"; + public static final String GC_DETAILS = "/api/v1/bookie/gc_details"; // autorecovery public static final String RECOVERY_BOOKIE = "/api/v1/autorecovery/bookie"; public static final String LIST_UNDER_REPLICATED_LEDGER = "/api/v1/autorecovery/list_under_replicated_ledger"; @@ -75,6 +76,7 @@ public HttpRouter(AbstractHttpHandlerFactory handlerFactory) { this.endpointHandlers.put(LIST_DISK_FILE, handlerFactory.newHandler(HttpServer.ApiType.LIST_DISK_FILE)); this.endpointHandlers.put(EXPAND_STORAGE, handlerFactory.newHandler(HttpServer.ApiType.EXPAND_STORAGE)); this.endpointHandlers.put(GC, handlerFactory.newHandler(HttpServer.ApiType.GC)); + this.endpointHandlers.put(GC_DETAILS, handlerFactory.newHandler(HttpServer.ApiType.GC_DETAILS)); // autorecovery this.endpointHandlers.put(RECOVERY_BOOKIE, handlerFactory.newHandler(HttpServer.ApiType.RECOVERY_BOOKIE)); diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java index 5e9f5090274..0922b56d90d 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java @@ -78,6 +78,8 @@ enum ApiType { LIST_DISK_FILE, EXPAND_STORAGE, GC, + GC_DETAILS, + // autorecovery RECOVERY_BOOKIE, LIST_UNDER_REPLICATED_LEDGER, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectionStatus.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectionStatus.java new file mode 100644 index 00000000000..e1728ddfa49 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectionStatus.java @@ -0,0 +1,47 @@ +/** + * + * 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 lombok.Builder; +import lombok.Getter; +import lombok.Setter; + +/** + * This is the garbage collection thread status. + * It includes what phase GarbageCollection (major/minor), gc counters, last gc time, etc. + */ +@Setter +@Getter +@Builder +public class GarbageCollectionStatus { + // whether the GC thread is in force GC. + private boolean forceCompacting; + // whether the GC thread is in major compacting. + private boolean majorCompacting; + // whether the GC thread is in minor compacting. + private boolean minorCompacting; + + private long lastMajorCompactionTime; + private long lastMinorCompactionTime; + private long majorCompactionCounter; + private long minorCompactionCounter; +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java index 369883cd2e4..ca02651e879 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java @@ -108,6 +108,10 @@ public class GarbageCollectorThread extends SafeRunnable { // to reduce the risk getting entry log corrupted final AtomicBoolean compacting = new AtomicBoolean(false); + // use to get the compacting status + final AtomicBoolean minorCompacting = new AtomicBoolean(false); + final AtomicBoolean majorCompacting = new AtomicBoolean(false); + volatile boolean running = true; // track the last scanned successfully log id @@ -298,6 +302,10 @@ Future triggerGC() { }); } + public boolean isInForceGC() { + return forceGarbageCollection.get(); + } + public void suspendMajorGC() { if (suspendMajorCompaction.compareAndSet(false, true)) { LOG.info("Suspend Major Compaction triggered by thread: {}", Thread.currentThread().getName()); @@ -376,18 +384,22 @@ public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMin && (force || curTime - lastMajorCompactionTime > majorCompactionInterval)) { // enter major compaction LOG.info("Enter major compaction, suspendMajor {}", suspendMajor); + majorCompacting.set(true); doCompactEntryLogs(majorCompactionThreshold); lastMajorCompactionTime = System.currentTimeMillis(); // and also move minor compaction time lastMinorCompactionTime = lastMajorCompactionTime; majorCompactionCounter.inc(); + majorCompacting.set(false); } else if (enableMinorCompaction && (!suspendMinor) && (force || curTime - lastMinorCompactionTime > minorCompactionInterval)) { // enter minor compaction LOG.info("Enter minor compaction, suspendMinor {}", suspendMinor); + minorCompacting.set(true); doCompactEntryLogs(minorCompactionThreshold); lastMinorCompactionTime = System.currentTimeMillis(); minorCompactionCounter.inc(); + minorCompacting.set(false); } if (force) { @@ -601,4 +613,16 @@ protected Map extractMetaFromEntryLogs(Map localConsistencyCheck(Optional r if (success.booleanValue()) { pageScanStats.registerSuccessfulEvent( - MathUtils.elapsedNanos(start), TimeUnit.NANOSECONDS); + MathUtils.elapsedNanos(start), TimeUnit.NANOSECONDS); } else { pageScanStats.registerFailedEvent( - MathUtils.elapsedNanos(start), TimeUnit.NANOSECONDS); + MathUtils.elapsedNanos(start), TimeUnit.NANOSECONDS); } } while (retry.booleanValue()); checkedPages++; @@ -584,15 +590,20 @@ public List localConsistencyCheck(Optional r checkedLedgers++; } LOG.info( - "Finished localConsistencyCheck, took {}s to scan {} ledgers, {} pages, " - + "{} entries with {} retries, {} errors", - TimeUnit.NANOSECONDS.toSeconds(MathUtils.elapsedNanos(checkStart)), - checkedLedgers, - checkedPages, - checkedEntries.longValue(), - pageRetries.longValue(), - errors.size()); + "Finished localConsistencyCheck, took {}s to scan {} ledgers, {} pages, " + + "{} entries with {} retries, {} errors", + TimeUnit.NANOSECONDS.toSeconds(MathUtils.elapsedNanos(checkStart)), + checkedLedgers, + checkedPages, + checkedEntries.longValue(), + pageRetries.longValue(), + errors.size()); return errors; } + + @Override + public List getGarbageCollectionStatus() { + return Collections.singletonList(gcThread.getGarbageCollectionStatus()); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java index 7a98fc797b6..111b8c213fc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java @@ -25,6 +25,7 @@ import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Optional; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; @@ -224,4 +225,24 @@ public Exception getException() { default List localConsistencyCheck(Optional rateLimiter) throws IOException { return new ArrayList<>(); } + + /** + * Whether force triggered Garbage Collection is running or not. + * + * @return + * true -- force triggered Garbage Collection is running, + * false -- force triggered Garbage Collection is not running + */ + default boolean isInForceGC() { + return false; + } + + + /** + * Get Garbage Collection status. + * Since DbLedgerStorage is a list of storage instances, we should return a list. + */ + default List getGarbageCollectionStatus() { + return Collections.emptyList(); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java index dd07d755504..5e4dbadd2a2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java @@ -340,4 +340,14 @@ public void forceGC() { public List localConsistencyCheck(Optional rateLimiter) throws IOException { return interleavedLedgerStorage.localConsistencyCheck(rateLimiter); } + + @Override + public boolean isInForceGC() { + return interleavedLedgerStorage.isInForceGC(); + } + + @Override + public List getGarbageCollectionStatus() { + return interleavedLedgerStorage.getGarbageCollectionStatus(); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index 4c5fd33197c..070656573dc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -39,12 +39,14 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.CheckpointSource; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.bookie.Checkpointer; +import org.apache.bookkeeper.bookie.GarbageCollectionStatus; import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification; import org.apache.bookkeeper.bookie.LedgerCache; import org.apache.bookkeeper.bookie.LedgerDirsManager; @@ -359,4 +361,15 @@ public static void readLedgerIndexEntries(long ledgerId, ServerConfiguration ser public void forceGC() { ledgerStorageList.stream().forEach(SingleDirectoryDbLedgerStorage::forceGC); } + + @Override + public boolean isInForceGC() { + return ledgerStorageList.stream().anyMatch(SingleDirectoryDbLedgerStorage::isInForceGC); + } + + @Override + public List getGarbageCollectionStatus() { + return ledgerStorageList.stream() + .map(single -> single.getGarbageCollectionStatus().get(0)).collect(Collectors.toList()); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 2e3c5562aba..58504ab6b74 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -30,6 +30,8 @@ import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; +import java.util.Collections; +import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -49,6 +51,7 @@ import org.apache.bookkeeper.bookie.CompactableLedgerStorage; import org.apache.bookkeeper.bookie.EntryLocation; import org.apache.bookkeeper.bookie.EntryLogger; +import org.apache.bookkeeper.bookie.GarbageCollectionStatus; import org.apache.bookkeeper.bookie.GarbageCollectorThread; import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification; import org.apache.bookkeeper.bookie.LedgerCache; @@ -269,6 +272,11 @@ public void forceGC() { gcThread.enableForceGC(); } + @Override + public boolean isInForceGC() { + return gcThread.isInForceGC(); + } + @Override public void shutdown() throws InterruptedException { try { @@ -922,6 +930,11 @@ long getReadCacheCount() { return readCache.count(); } + @Override + public List getGarbageCollectionStatus() { + return Collections.singletonList(gcThread.getGarbageCollectionStatus()); + } + /** * Interface which process ledger logger. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java index 102b0eb57ef..662777cadd5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java @@ -41,6 +41,7 @@ import org.apache.bookkeeper.server.http.service.DecommissionService; import org.apache.bookkeeper.server.http.service.DeleteLedgerService; import org.apache.bookkeeper.server.http.service.ExpandStorageService; +import org.apache.bookkeeper.server.http.service.GCDetailsService; import org.apache.bookkeeper.server.http.service.GetLastLogMarkService; import org.apache.bookkeeper.server.http.service.GetLedgerMetaService; import org.apache.bookkeeper.server.http.service.ListBookieInfoService; @@ -211,6 +212,8 @@ public HttpEndpointService provideHttpEndpointService(ApiType type) { return new ExpandStorageService(configuration); case GC: return new TriggerGCService(configuration, bookieServer); + case GC_DETAILS: + return new GCDetailsService(configuration, bookieServer); // autorecovery case RECOVERY_BOOKIE: diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GCDetailsService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GCDetailsService.java new file mode 100644 index 00000000000..29b59e271ba --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GCDetailsService.java @@ -0,0 +1,85 @@ +/* + * 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.server.http.service; + +import static com.google.common.base.Preconditions.checkNotNull; + +import java.util.List; +import org.apache.bookkeeper.bookie.GarbageCollectionStatus; +import org.apache.bookkeeper.common.util.JsonUtil; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.http.HttpServer; +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.proto.BookieServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * HttpEndpointService that handle get garbage collection details service. + * + *

            Get Garbage Collection status, the output would be like: + * [ { + * "forceCompacting" : false, + * "majorCompacting" : false, + * "minorCompacting" : false, + * "lastMajorCompactionTime" : 1544578144944, + * "lastMinorCompactionTime" : 1544578144944, + * "majorCompactionCounter" : 1, + * "minorCompactionCounter" : 0 + * } ] + */ +public class GCDetailsService implements HttpEndpointService { + + static final Logger LOG = LoggerFactory.getLogger(GCDetailsService.class); + + protected ServerConfiguration conf; + protected BookieServer bookieServer; + + public GCDetailsService(ServerConfiguration conf, BookieServer bookieServer) { + checkNotNull(conf); + checkNotNull(bookieServer); + this.conf = conf; + this.bookieServer = bookieServer; + } + + @Override + public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { + HttpServiceResponse response = new HttpServiceResponse(); + + if (HttpServer.Method.GET == request.getMethod()) { + List details = bookieServer.getBookie() + .getLedgerStorage().getGarbageCollectionStatus(); + + String jsonResponse = JsonUtil.toJson(details); + if (LOG.isDebugEnabled()) { + LOG.debug("output body:" + jsonResponse); + } + response.setBody(jsonResponse); + response.setCode(HttpServer.StatusCode.OK); + return response; + } else { + response.setCode(HttpServer.StatusCode.NOT_FOUND); + response.setBody("Only support GET method to retrieve GC details." + + " If you want to trigger gc, send a POST to gc endpoint."); + return response; + } + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerGCService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerGCService.java index 185965afad8..ebf9ea9c347 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerGCService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerGCService.java @@ -27,6 +27,7 @@ import org.apache.bookkeeper.http.service.HttpServiceRequest; import org.apache.bookkeeper.http.service.HttpServiceResponse; import org.apache.bookkeeper.proto.BookieServer; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +35,12 @@ * HttpEndpointService that handle force trigger GC requests. * *

            The PUT method will force trigger GC on current bookie, and make GC run at backend. + * + *

            The GET method will get the force triggered GC running or not. + * Output would be like: + * { + * "is_in_force_gc" : "false" + * } */ public class TriggerGCService implements HttpEndpointService { @@ -52,19 +59,31 @@ public TriggerGCService(ServerConfiguration conf, BookieServer bookieServer) { @Override public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { HttpServiceResponse response = new HttpServiceResponse(); - // PUT + if (HttpServer.Method.PUT == request.getMethod()) { bookieServer.getBookie().getLedgerStorage().forceGC(); String output = "Triggered GC on BookieServer: " + bookieServer.toString(); String jsonResponse = JsonUtil.toJson(output); - LOG.debug("output body:" + jsonResponse); + if (LOG.isDebugEnabled()) { + LOG.debug("output body:" + jsonResponse); + } + response.setBody(jsonResponse); + response.setCode(HttpServer.StatusCode.OK); + return response; + } else if (HttpServer.Method.GET == request.getMethod()) { + Boolean isInForceGC = bookieServer.getBookie().getLedgerStorage().isInForceGC(); + Pair output = Pair.of("is_in_force_gc", isInForceGC.toString()); + String jsonResponse = JsonUtil.toJson(output); + if (LOG.isDebugEnabled()) { + LOG.debug("output body:" + jsonResponse); + } response.setBody(jsonResponse); response.setCode(HttpServer.StatusCode.OK); return response; } else { response.setCode(HttpServer.StatusCode.NOT_FOUND); - response.setBody("Not found method. Should be PUT method to trigger GC."); + response.setBody("Not found method. Should be PUT to trigger GC, Or GET to get Force GC state."); return response; } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index e88d29d3a68..da25d14d5c0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -743,9 +743,6 @@ public void testDecommissionService() throws Exception { stopAuditorElector(); } - /** - * Create ledgers, then test Delete Ledger service. - */ @Test public void testTriggerGCService() throws Exception { baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); @@ -771,14 +768,59 @@ public void testTriggerGCService() throws Exception { HttpEndpointService triggerGCService = bkHttpServiceProvider .provideHttpEndpointService(HttpServer.ApiType.GC); - //1, GET, should return NOT_FOUND + //1, GET, should return OK HttpServiceRequest request1 = new HttpServiceRequest(null, HttpServer.Method.GET, null); HttpServiceResponse response1 = triggerGCService.handle(request1); - assertEquals(HttpServer.StatusCode.NOT_FOUND.getValue(), response1.getStatusCode()); + assertEquals(HttpServer.StatusCode.OK.getValue(), response1.getStatusCode()); + assertTrue(response1.getBody().contains("\"is_in_force_gc\" : \"false\"")); //2, PUT, should return OK HttpServiceRequest request2 = new HttpServiceRequest(null, HttpServer.Method.PUT, null); HttpServiceResponse response2 = triggerGCService.handle(request2); assertEquals(HttpServer.StatusCode.OK.getValue(), response2.getStatusCode()); } + + @Test + public void testGCDetailsService() throws Exception { + baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + BookKeeper.DigestType digestType = BookKeeper.DigestType.CRC32; + int numLedgers = 4; + int numMsgs = 100; + LedgerHandle[] lh = new LedgerHandle[numLedgers]; + // create ledgers + for (int i = 0; i < numLedgers; i++) { + lh[i] = bkc.createLedger(digestType, "".getBytes()); + } + String content = "This is test for GC details service!"; + // add entries + for (int i = 0; i < numMsgs; i++) { + for (int j = 0; j < numLedgers; j++) { + lh[j].addEntry(content.getBytes()); + } + } + // close ledgers + for (int i = 0; i < numLedgers; i++) { + lh[i].close(); + } + HttpEndpointService gcDetailsService = bkHttpServiceProvider + .provideHttpEndpointService(HttpServer.ApiType.GC_DETAILS); + + // force trigger a GC + HttpEndpointService triggerGCService = bkHttpServiceProvider + .provideHttpEndpointService(HttpServer.ApiType.GC); + HttpServiceRequest request0 = new HttpServiceRequest(null, HttpServer.Method.PUT, null); + HttpServiceResponse response0 = triggerGCService.handle(request0); + assertEquals(HttpServer.StatusCode.OK.getValue(), response0.getStatusCode()); + + //1, GET, should return OK + HttpServiceRequest request1 = new HttpServiceRequest(null, HttpServer.Method.GET, null); + HttpServiceResponse response1 = gcDetailsService.handle(request1); + assertEquals(HttpServer.StatusCode.OK.getValue(), response1.getStatusCode()); + LOG.info("Get response: {}", response1.getBody()); + + //2, PUT, should return NOT_FOUND + HttpServiceRequest request3 = new HttpServiceRequest(null, HttpServer.Method.PUT, null); + HttpServiceResponse response3 = gcDetailsService.handle(request3); + assertEquals(HttpServer.StatusCode.NOT_FOUND.getValue(), response3.getStatusCode()); + } } diff --git a/site/docs/latest/admin/http.md b/site/docs/latest/admin/http.md index 7c1abea49f2..270dcb3a16f 100644 --- a/site/docs/latest/admin/http.md +++ b/site/docs/latest/admin/http.md @@ -282,6 +282,45 @@ Currently all the HTTP endpoints could be divided into these 4 components: |403 | Permission denied | |404 | Not found | +1. Method: GET + * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + { + "is_in_force_gc" : "false" + } + ``` + +### Endpoint: /api/v1/bookie/gc_details +1. Method: GET + * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + [ { + "forceCompacting" : false, + "majorCompacting" : false, + "minorCompacting" : false, + "lastMajorCompactionTime" : 1544578144944, + "lastMinorCompactionTime" : 1544578144944, + "majorCompactionCounter" : 1, + "minorCompactionCounter" : 0 + } ] + ``` + ## Auto recovery ### Endpoint: /api/v1/autorecovery/bookie/ From 27d4c046f7e774a5b74d5c8744bc9e4642a6bab9 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 13 Dec 2018 22:49:02 +0800 Subject: [PATCH 0174/1642] [STATS] [DOC] Add @StatsDoc annotation for db ledger storage stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document the db ledger storage stats. *Changes* - convert db ledger storage stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Jia Zhai , Enrico Olivelli This closes #1874 from sijie/ldb_stats --- .../bookie/storage/ldb/DbLedgerStorage.java | 57 +---- .../storage/ldb/DbLedgerStorageStats.java | 204 ++++++++++++++++++ .../storage/ldb/EntryLocationIndex.java | 22 +- .../storage/ldb/EntryLocationIndexStats.java | 66 ++++++ .../storage/ldb/LedgerMetadataIndex.java | 22 +- .../storage/ldb/LedgerMetadataIndexStats.java | 66 ++++++ .../ldb/SingleDirectoryDbLedgerStorage.java | 129 +++-------- 7 files changed, 385 insertions(+), 181 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageStats.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndexStats.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndexStats.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index 070656573dc..ccabaed15d5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -58,7 +58,6 @@ import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.DiskChecker; @@ -87,6 +86,7 @@ public class DbLedgerStorage implements LedgerStorage { // Keep 1 single Bookie GC thread so the the compactions from multiple individual directories are serialized private ScheduledExecutorService gcExecutor; + private DbLedgerStorageStats stats; @Override public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, @@ -123,7 +123,13 @@ public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, Le perDirectoryReadCacheSize)); } - registerStats(statsLogger); + this.stats = new DbLedgerStorageStats( + statsLogger, + () -> ledgerStorageList.stream().mapToLong(SingleDirectoryDbLedgerStorage::getWriteCacheSize).sum(), + () -> ledgerStorageList.stream().mapToLong(SingleDirectoryDbLedgerStorage::getWriteCacheCount).sum(), + () -> ledgerStorageList.stream().mapToLong(SingleDirectoryDbLedgerStorage::getReadCacheSize).sum(), + () -> ledgerStorageList.stream().mapToLong(SingleDirectoryDbLedgerStorage::getReadCacheCount).sum() + ); } @VisibleForTesting @@ -136,53 +142,6 @@ protected SingleDirectoryDbLedgerStorage newSingleDirectoryDbLedgerStorage(Serve stateManager, checkpointSource, checkpointer, statsLogger, gcExecutor, writeCacheSize, readCacheSize); } - public void registerStats(StatsLogger stats) { - stats.registerGauge("write-cache-size", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return ledgerStorageList.stream().mapToLong(SingleDirectoryDbLedgerStorage::getWriteCacheSize).sum(); - } - }); - stats.registerGauge("write-cache-count", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return ledgerStorageList.stream().mapToLong(SingleDirectoryDbLedgerStorage::getWriteCacheCount).sum(); - } - }); - stats.registerGauge("read-cache-size", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return ledgerStorageList.stream().mapToLong(SingleDirectoryDbLedgerStorage::getReadCacheSize).sum(); - } - }); - stats.registerGauge("read-cache-count", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return ledgerStorageList.stream().mapToLong(SingleDirectoryDbLedgerStorage::getReadCacheCount).sum(); - } - }); - } - @Override public void start() { ledgerStorageList.forEach(LedgerStorage::start); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageStats.java new file mode 100644 index 00000000000..bc99c609d25 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageStats.java @@ -0,0 +1,204 @@ +/* + * 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.storage.ldb; + +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_ADD_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; + +import java.util.function.Supplier; +import lombok.Getter; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for db ledger storage stats. + */ +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "DbLedgerStorage related stats" +) +@Getter +class DbLedgerStorageStats { + + private static final String ADD_ENTRY = "add-entry"; + private static final String READ_ENTRY = "read-entry"; + private static final String READ_CACHE_HITS = "read-cache-hits"; + private static final String READ_CACHE_MISSES = "read-cache-misses"; + private static final String READAHEAD_BATCH_COUNT = "readahead-batch-count"; + private static final String READAHEAD_BATCH_SIZE = "readahead-batch-size"; + private static final String FLUSH = "flush"; + private static final String FLUSH_SIZE = "flush-size"; + private static final String THROTTLED_WRITE_REQUESTS = "throttled-write-requests"; + private static final String REJECTED_WRITE_REQUESTS = "rejected-write-requests"; + private static final String WRITE_CACHE_SIZE = "write-cache-size"; + private static final String WRITE_CACHE_COUNT = "write-cache-count"; + private static final String READ_CACHE_SIZE = "read-cache-size"; + private static final String READ_CACHE_COUNT = "read-cache-count"; + + @StatsDoc( + name = ADD_ENTRY, + help = "operation stats of adding entries to db ledger storage", + parent = BOOKIE_ADD_ENTRY + ) + private final OpStatsLogger addEntryStats; + @StatsDoc( + name = READ_ENTRY, + help = "operation stats of reading entries from db ledger storage", + parent = BOOKIE_ADD_ENTRY + ) + private final OpStatsLogger readEntryStats; + @StatsDoc( + name = READ_CACHE_HITS, + help = "operation stats of read cache hits", + parent = READ_ENTRY + ) + private final OpStatsLogger readCacheHitStats; + @StatsDoc( + name = READ_CACHE_MISSES, + help = "operation stats of read cache misses", + parent = READ_ENTRY + ) + private final OpStatsLogger readCacheMissStats; + @StatsDoc( + name = READAHEAD_BATCH_COUNT, + help = "the distribution of num of entries to read in one readahead batch" + ) + private final OpStatsLogger readAheadBatchCountStats; + @StatsDoc( + name = READAHEAD_BATCH_COUNT, + help = "the distribution of num of bytes to read in one readahead batch" + ) + private final OpStatsLogger readAheadBatchSizeStats; + @StatsDoc( + name = FLUSH_SIZE, + help = "operation stats of flushing write cache to entry log files" + ) + private final OpStatsLogger flushStats; + @StatsDoc( + name = FLUSH_SIZE, + help = "the distribution of number of bytes flushed from write cache to entry log files" + ) + private final OpStatsLogger flushSizeStats; + @StatsDoc( + name = THROTTLED_WRITE_REQUESTS, + help = "The number of requests throttled due to write cache is full" + ) + private final Counter throttledWriteRequests; + @StatsDoc( + name = REJECTED_WRITE_REQUESTS, + help = "The number of requests rejected due to write cache is full" + ) + private final Counter rejectedWriteRequests; + + @StatsDoc( + name = WRITE_CACHE_SIZE, + help = "Current number of bytes in write cache" + ) + private final Gauge writeCacheSizeGauge; + @StatsDoc( + name = WRITE_CACHE_COUNT, + help = "Current number of entries in write cache" + ) + private final Gauge writeCacheCountGauge; + @StatsDoc( + name = READ_CACHE_SIZE, + help = "Current number of bytes in read cache" + ) + private final Gauge readCacheSizeGauge; + @StatsDoc( + name = READ_CACHE_COUNT, + help = "Current number of entries in read cache" + ) + private final Gauge readCacheCountGauge; + + DbLedgerStorageStats(StatsLogger stats, + Supplier writeCacheSizeSupplier, + Supplier writeCacheCountSupplier, + Supplier readCacheSizeSupplier, + Supplier readCacheCountSupplier) { + addEntryStats = stats.getOpStatsLogger(ADD_ENTRY); + readEntryStats = stats.getOpStatsLogger(READ_ENTRY); + readCacheHitStats = stats.getOpStatsLogger(READ_CACHE_HITS); + readCacheMissStats = stats.getOpStatsLogger(READ_CACHE_MISSES); + readAheadBatchCountStats = stats.getOpStatsLogger(READAHEAD_BATCH_COUNT); + readAheadBatchSizeStats = stats.getOpStatsLogger(READAHEAD_BATCH_SIZE); + flushStats = stats.getOpStatsLogger(FLUSH); + flushSizeStats = stats.getOpStatsLogger(FLUSH_SIZE); + + throttledWriteRequests = stats.getCounter(THROTTLED_WRITE_REQUESTS); + rejectedWriteRequests = stats.getCounter(REJECTED_WRITE_REQUESTS); + + writeCacheSizeGauge = new Gauge() { + @Override + public Long getDefaultValue() { + return 0L; + } + + @Override + public Long getSample() { + return writeCacheSizeSupplier.get(); + } + }; + stats.registerGauge(WRITE_CACHE_SIZE, writeCacheSizeGauge); + writeCacheCountGauge = new Gauge() { + @Override + public Long getDefaultValue() { + return 0L; + } + + @Override + public Long getSample() { + return writeCacheCountSupplier.get(); + } + }; + stats.registerGauge(WRITE_CACHE_COUNT, writeCacheCountGauge); + readCacheSizeGauge = new Gauge() { + @Override + public Long getDefaultValue() { + return 0L; + } + + @Override + public Long getSample() { + return readCacheSizeSupplier.get(); + } + }; + stats.registerGauge(READ_CACHE_SIZE, readCacheSizeGauge); + readCacheCountGauge = new Gauge() { + + @Override + public Long getDefaultValue() { + return 0L; + } + + @Override + public Long getSample() { + return readCacheCountSupplier.get(); + } + }; + stats.registerGauge(READ_CACHE_COUNT, readCacheCountGauge); + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java index 21b87e2be96..5673883b919 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java @@ -34,7 +34,6 @@ import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorage.Batch; import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.collections.ConcurrentLongHashSet; import org.slf4j.Logger; @@ -51,33 +50,22 @@ public class EntryLocationIndex implements Closeable { private final KeyValueStorage locationsDb; private final ConcurrentLongHashSet deletedLedgers = new ConcurrentLongHashSet(); - private StatsLogger stats; + private final EntryLocationIndexStats stats; public EntryLocationIndex(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath, StatsLogger stats) throws IOException { String locationsDbPath = FileSystems.getDefault().getPath(basePath, "locations").toFile().toString(); locationsDb = storageFactory.newKeyValueStorage(locationsDbPath, DbConfigType.Huge, conf); - this.stats = stats; - registerStats(); - } - - public void registerStats() { - stats.registerGauge("entries-count", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { + this.stats = new EntryLocationIndexStats( + stats, + () -> { try { return locationsDb.count(); } catch (IOException e) { return -1L; } - } - }); + }); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndexStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndexStats.java new file mode 100644 index 00000000000..dd87f7b92bb --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndexStats.java @@ -0,0 +1,66 @@ +/* + * 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.storage.ldb; + +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; + +import java.util.function.Supplier; +import lombok.Getter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for ledger metadata index stats. + */ +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "Entry location index stats" +) +@Getter +class EntryLocationIndexStats { + + private static final String ENTRIES_COUNT = "entries-count"; + + @StatsDoc( + name = ENTRIES_COUNT, + help = "Current number of entries" + ) + private final Gauge entriesCountGauge; + + EntryLocationIndexStats(StatsLogger statsLogger, + Supplier entriesCountSupplier) { + entriesCountGauge = new Gauge() { + @Override + public Long getDefaultValue() { + return 0L; + } + + @Override + public Long getSample() { + return entriesCountSupplier.get(); + } + }; + statsLogger.registerGauge(ENTRIES_COUNT, entriesCountGauge); + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java index 04bf32dba71..8383db4388f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java @@ -39,7 +39,6 @@ import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorage.CloseableIterator; import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap; import org.slf4j.Logger; @@ -56,7 +55,7 @@ public class LedgerMetadataIndex implements Closeable { private final AtomicInteger ledgersCount; private final KeyValueStorage ledgersDb; - private StatsLogger stats; + private final LedgerMetadataIndexStats stats; // Holds ledger modifications applied in memory map, and pending to be flushed on db private final ConcurrentLinkedQueue> pendingLedgersUpdates; @@ -89,22 +88,9 @@ public LedgerMetadataIndex(ServerConfiguration conf, KeyValueStorageFactory stor this.pendingLedgersUpdates = new ConcurrentLinkedQueue>(); this.pendingDeletedLedgers = new ConcurrentLinkedQueue(); - this.stats = stats; - registerStats(); - } - - public void registerStats() { - stats.registerGauge("ledgers-count", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return (long) ledgersCount.get(); - } - }); + this.stats = new LedgerMetadataIndexStats( + stats, + () -> (long) ledgersCount.get()); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndexStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndexStats.java new file mode 100644 index 00000000000..a46e38b650c --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndexStats.java @@ -0,0 +1,66 @@ +/* + * 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.storage.ldb; + +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; + +import java.util.function.Supplier; +import lombok.Getter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for ledger metadata index stats. + */ +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "Ledger metadata index stats" +) +@Getter +class LedgerMetadataIndexStats { + + private static final String LEDGERS_COUNT = "ledgers-count"; + + @StatsDoc( + name = LEDGERS_COUNT, + help = "Current number of ledgers" + ) + private final Gauge ledgersCountGauge; + + LedgerMetadataIndexStats(StatsLogger statsLogger, + Supplier ledgersCountSupplier) { + ledgersCountGauge = new Gauge() { + @Override + public Long getDefaultValue() { + return 0L; + } + + @Override + public Long getSample() { + return ledgersCountSupplier.get(); + } + }; + statsLogger.registerGauge(LEDGERS_COUNT, ledgersCountGauge); + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 58504ab6b74..197014aa93b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -64,8 +64,6 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.proto.BookieProtocol; -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.MathUtils; @@ -124,19 +122,7 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage private final long maxThrottleTimeNanos; - private final StatsLogger stats; - - private final OpStatsLogger addEntryStats; - private final OpStatsLogger readEntryStats; - private final OpStatsLogger readCacheHitStats; - private final OpStatsLogger readCacheMissStats; - private final OpStatsLogger readAheadBatchCountStats; - private final OpStatsLogger readAheadBatchSizeStats; - private final OpStatsLogger flushStats; - private final OpStatsLogger flushSizeStats; - - private final Counter throttledWriteRequests; - private final Counter rejectedWriteRequests; + private final DbLedgerStorageStats dbLedgerStorageStats; static final String READ_AHEAD_CACHE_BATCH_SIZE = "dbStorage_readAheadCacheBatchSize"; private static final int DEFAULT_READ_AHEAD_CACHE_BATCH_SIZE = 100; @@ -169,10 +155,8 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le readCache = new ReadCache(readCacheMaxSize); - this.stats = statsLogger; - - ledgerIndex = new LedgerMetadataIndex(conf, KeyValueStorageRocksDB.factory, baseDir, stats); - entryLocationIndex = new EntryLocationIndex(conf, KeyValueStorageRocksDB.factory, baseDir, stats); + ledgerIndex = new LedgerMetadataIndex(conf, KeyValueStorageRocksDB.factory, baseDir, statsLogger); + entryLocationIndex = new EntryLocationIndex(conf, KeyValueStorageRocksDB.factory, baseDir, statsLogger); transientLedgerInfoCache = new ConcurrentLongHashMap<>(16 * 1024, Runtime.getRuntime().availableProcessors() * 2); @@ -183,62 +167,13 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le entryLogger = new EntryLogger(conf, ledgerDirsManager, null, statsLogger); gcThread = new GarbageCollectorThread(conf, ledgerManager, this, statsLogger); - stats.registerGauge("write-cache-size", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return writeCache.size() + writeCacheBeingFlushed.size(); - } - }); - stats.registerGauge("write-cache-count", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return writeCache.count() + writeCacheBeingFlushed.count(); - } - }); - stats.registerGauge("read-cache-size", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return readCache.size(); - } - }); - stats.registerGauge("read-cache-count", new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return readCache.count(); - } - }); - - addEntryStats = stats.getOpStatsLogger("add-entry"); - readEntryStats = stats.getOpStatsLogger("read-entry"); - readCacheHitStats = stats.getOpStatsLogger("read-cache-hits"); - readCacheMissStats = stats.getOpStatsLogger("read-cache-misses"); - readAheadBatchCountStats = stats.getOpStatsLogger("readahead-batch-count"); - readAheadBatchSizeStats = stats.getOpStatsLogger("readahead-batch-size"); - flushStats = stats.getOpStatsLogger("flush"); - flushSizeStats = stats.getOpStatsLogger("flush-size"); - - throttledWriteRequests = stats.getCounter("throttled-write-requests"); - rejectedWriteRequests = stats.getCounter("rejected-write-requests"); + dbLedgerStorageStats = new DbLedgerStorageStats( + statsLogger, + () -> writeCache.size() + writeCacheBeingFlushed.size(), + () -> writeCache.count() + writeCacheBeingFlushed.count(), + () -> readCache.size(), + () -> readCache.count() + ); } @Override @@ -393,7 +328,7 @@ public long addEntry(ByteBuf entry) throws IOException, BookieException { // after successfully insert the entry, update LAC and notify the watchers updateCachedLacIfNeeded(ledgerId, lac); - recordSuccessfulEvent(addEntryStats, startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getAddEntryStats(), startTime); return entryId; } @@ -414,7 +349,7 @@ private void triggerFlushAndAddEntry(long ledgerId, long entryId, ByteBuf entry) }); } - throttledWriteRequests.inc(); + dbLedgerStorageStats.getThrottledWriteRequests().inc(); long absoluteTimeoutNanos = System.nanoTime() + maxThrottleTimeNanos; while (System.nanoTime() < absoluteTimeoutNanos) { @@ -438,7 +373,7 @@ private void triggerFlushAndAddEntry(long ledgerId, long entryId, ByteBuf entry) } // Timeout expired and we weren't able to insert in write cache - rejectedWriteRequests.inc(); + dbLedgerStorageStats.getRejectedWriteRequests().inc(); throw new OperationRejectedException(); } @@ -473,24 +408,24 @@ public ByteBuf getEntry(long ledgerId, long entryId) throws IOException { // First try to read from the write cache of recent entries ByteBuf entry = localWriteCache.get(ledgerId, entryId); if (entry != null) { - recordSuccessfulEvent(readCacheHitStats, startTime); - recordSuccessfulEvent(readEntryStats, startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadCacheHitStats(), startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadEntryStats(), startTime); return entry; } // If there's a flush going on, the entry might be in the flush buffer entry = localWriteCacheBeingFlushed.get(ledgerId, entryId); if (entry != null) { - recordSuccessfulEvent(readCacheHitStats, startTime); - recordSuccessfulEvent(readEntryStats, startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadCacheHitStats(), startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadEntryStats(), startTime); return entry; } // Try reading from read-ahead cache entry = readCache.get(ledgerId, entryId); if (entry != null) { - recordSuccessfulEvent(readCacheHitStats, startTime); - recordSuccessfulEvent(readEntryStats, startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadCacheHitStats(), startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadEntryStats(), startTime); return entry; } @@ -503,7 +438,7 @@ public ByteBuf getEntry(long ledgerId, long entryId) throws IOException { } entry = entryLogger.readEntry(ledgerId, entryId, entryLocation); } catch (NoEntryException e) { - recordFailedEvent(readEntryStats, startTime); + recordFailedEvent(dbLedgerStorageStats.getReadEntryStats(), startTime); throw e; } @@ -513,8 +448,8 @@ public ByteBuf getEntry(long ledgerId, long entryId) throws IOException { long nextEntryLocation = entryLocation + 4 /* size header */ + entry.readableBytes(); fillReadAheadCache(ledgerId, entryId + 1, nextEntryLocation); - recordSuccessfulEvent(readCacheMissStats, startTime); - recordSuccessfulEvent(readEntryStats, startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadCacheMissStats(), startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadEntryStats(), startTime); return entry; } @@ -551,8 +486,8 @@ private void fillReadAheadCache(long orginalLedgerId, long firstEntryId, long fi } } - readAheadBatchCountStats.registerSuccessfulValue(count); - readAheadBatchSizeStats.registerSuccessfulValue(size); + dbLedgerStorageStats.getReadAheadBatchCountStats().registerSuccessfulValue(count); + dbLedgerStorageStats.getReadAheadBatchSizeStats().registerSuccessfulValue(size); } catch (Exception e) { if (log.isDebugEnabled()) { log.debug("Exception during read ahead for ledger: {}: e", orginalLedgerId, e); @@ -578,8 +513,8 @@ public ByteBuf getLastEntry(long ledgerId) throws IOException { } } - recordSuccessfulEvent(readCacheHitStats, startTime); - recordSuccessfulEvent(readEntryStats, startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadCacheHitStats(), startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadEntryStats(), startTime); return entry; } @@ -595,8 +530,8 @@ public ByteBuf getLastEntry(long ledgerId) throws IOException { } } - recordSuccessfulEvent(readCacheHitStats, startTime); - recordSuccessfulEvent(readEntryStats, startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadCacheHitStats(), startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadEntryStats(), startTime); return entry; } } finally { @@ -612,8 +547,8 @@ public ByteBuf getLastEntry(long ledgerId) throws IOException { long entryLocation = entryLocationIndex.getLocation(ledgerId, lastEntryId); ByteBuf content = entryLogger.readEntry(ledgerId, lastEntryId, entryLocation); - recordSuccessfulEvent(readCacheMissStats, startTime); - recordSuccessfulEvent(readEntryStats, startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadCacheMissStats(), startTime); + recordSuccessfulEvent(dbLedgerStorageStats.getReadEntryStats(), startTime); return content; } @@ -702,8 +637,8 @@ public void checkpoint(Checkpoint checkpoint) throws IOException { log.debug("Flushing done time {} s -- Written {} MB/s", flushTimeSeconds, flushThroughput); } - recordSuccessfulEvent(flushStats, startTime); - flushSizeStats.registerSuccessfulValue(sizeToFlush); + recordSuccessfulEvent(dbLedgerStorageStats.getFlushStats(), startTime); + dbLedgerStorageStats.getFlushSizeStats().registerSuccessfulValue(sizeToFlush); } catch (IOException e) { // Leave IOExecption as it is throw e; From cdaf29baf1ac906d03355f159a59d8cb1ca7077e Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 13 Dec 2018 22:50:44 +0800 Subject: [PATCH 0175/1642] [STATS] [DOC] Add @StatsDoc annotation for bookkeeper autorecovery stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document bookkeeper autorecovery stats. *Changes* - convert bookkeeper autorecovery stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Jia Zhai This closes #1879 from sijie/replication_stats --- .../bookkeeper/replication/Auditor.java | 64 ++++++++++++++++++- .../replication/AuditorElector.java | 10 +++ .../replication/ReplicationWorker.java | 26 +++++++- 3 files changed, 96 insertions(+), 4 deletions(-) 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 acf0c09c1bd..89883b05285 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,18 @@ */ package org.apache.bookkeeper.replication; +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; +import static org.apache.bookkeeper.replication.ReplicationStats.AUDIT_BOOKIES_TIME; +import static org.apache.bookkeeper.replication.ReplicationStats.BOOKIE_TO_LEDGERS_MAP_CREATION_TIME; +import static org.apache.bookkeeper.replication.ReplicationStats.CHECK_ALL_LEDGERS_TIME; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_BOOKIES_PER_LEDGER; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_BOOKIE_AUDITS_DELAYED; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_DELAYED_BOOKIE_AUDITS_DELAYES_CANCELLED; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_FRAGMENTS_PER_LEDGER; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_CHECKED; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_UNDER_REPLICATED_LEDGERS; +import static org.apache.bookkeeper.replication.ReplicationStats.URL_PUBLISH_TIME_FOR_LOST_BOOKIE; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import com.google.common.collect.Lists; @@ -61,6 +73,7 @@ import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.commons.collections4.CollectionUtils; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; @@ -76,6 +89,10 @@ * *

            TODO: eliminate the direct usage of zookeeper here {@link https://github.com/apache/bookkeeper/issues/1332} */ +@StatsDoc( + name = AUDITOR_SCOPE, + help = "Auditor related stats" +) public class Auditor implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(Auditor.class); private final ServerConfiguration conf; @@ -88,20 +105,61 @@ public class Auditor implements AutoCloseable { private final ScheduledExecutorService executor; private List knownBookies = new ArrayList(); private final String bookieIdentifier; + private volatile Future auditTask; + private Set bookiesToBeAudited = Sets.newHashSet(); + private volatile int lostBookieRecoveryDelayBeforeChange; + private final StatsLogger statsLogger; + @StatsDoc( + name = NUM_UNDER_REPLICATED_LEDGERS, + help = "the distribution of num under_replicated ledgers on each auditor run" + ) private final OpStatsLogger numUnderReplicatedLedger; + @StatsDoc( + name = URL_PUBLISH_TIME_FOR_LOST_BOOKIE, + help = "the latency distribution of publishing under replicated ledgers for lost bookies" + ) private final OpStatsLogger uRLPublishTimeForLostBookies; + @StatsDoc( + name = BOOKIE_TO_LEDGERS_MAP_CREATION_TIME, + help = "the latency distribution of creating bookies-to-ledgers map" + ) private final OpStatsLogger bookieToLedgersMapCreationTime; + @StatsDoc( + name = CHECK_ALL_LEDGERS_TIME, + help = "the latency distribution of checking all ledgers" + ) private final OpStatsLogger checkAllLedgersTime; + @StatsDoc( + name = AUDIT_BOOKIES_TIME, + help = "the latency distribution of auditing all the bookies" + ) private final OpStatsLogger auditBookiesTime; + @StatsDoc( + name = NUM_LEDGERS_CHECKED, + help = "the number of ledgers checked by the auditor" + ) private final Counter numLedgersChecked; + @StatsDoc( + name = NUM_FRAGMENTS_PER_LEDGER, + help = "the distribution of number of fragments per ledger" + ) private final OpStatsLogger numFragmentsPerLedger; + @StatsDoc( + name = NUM_BOOKIES_PER_LEDGER, + help = "the distribution of number of bookies per ledger" + ) private final OpStatsLogger numBookiesPerLedger; + @StatsDoc( + name = NUM_BOOKIE_AUDITS_DELAYED, + help = "the number of bookie-audits delayed" + ) private final Counter numBookieAuditsDelayed; + @StatsDoc( + name = NUM_DELAYED_BOOKIE_AUDITS_DELAYES_CANCELLED, + help = "the number of delayed-bookie-audits cancelled" + ) private final Counter numDelayedBookieAuditsCancelled; - private volatile Future auditTask; - private Set bookiesToBeAudited = Sets.newHashSet(); - private volatile int lostBookieRecoveryDelayBeforeChange; static BookKeeper createBookKeeperClient(ServerConfiguration conf) throws InterruptedException, IOException { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java index 14c5c538562..543aaac57de 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java @@ -21,6 +21,7 @@ package org.apache.bookkeeper.replication; import static com.google.common.base.Charsets.UTF_8; +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; import static org.apache.bookkeeper.replication.ReplicationStats.ELECTION_ATTEMPTS; import com.google.common.annotations.VisibleForTesting; @@ -48,6 +49,7 @@ import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.ZkUtils; import org.apache.commons.lang.StringUtils; @@ -71,6 +73,10 @@ * will be elected as Auditor. All the other bookies will be watching on their * predecessor znode according to the ephemeral sequence numbers. */ +@StatsDoc( + name = AUDITOR_SCOPE, + help = "Auditor related stats" +) public class AuditorElector { private static final Logger LOG = LoggerFactory .getLogger(AuditorElector.class); @@ -98,6 +104,10 @@ public class AuditorElector { private AtomicBoolean running = new AtomicBoolean(false); // Expose Stats + @StatsDoc( + name = ELECTION_ATTEMPTS, + help = "The number of auditor election attempts" + ) private final Counter electionAttempts; private final StatsLogger statsLogger; 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 eeaa96bc9ab..74d20816dbd 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 @@ -22,6 +22,7 @@ import static org.apache.bookkeeper.replication.ReplicationStats.NUM_DEFER_LEDGER_LOCK_RELEASE_OF_FAILED_LEDGER; 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.REPLICATION_WORKER_SCOPE; import static org.apache.bookkeeper.replication.ReplicationStats.REREPLICATE_OP; import com.google.common.base.Stopwatch; @@ -64,6 +65,7 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,6 +74,10 @@ * ReplicationWorker will take the fragments one by one from * ZKLedgerUnderreplicationManager and replicates to it. */ +@StatsDoc( + name = REPLICATION_WORKER_SCOPE, + help = "replication worker related stats" +) public class ReplicationWorker implements Runnable { private static final Logger LOG = LoggerFactory .getLogger(ReplicationWorker.class); @@ -93,8 +99,25 @@ public class ReplicationWorker implements Runnable { // Expose Stats private final StatsLogger statsLogger; + @StatsDoc( + name = REPLICATE_EXCEPTION, + help = "replication related exceptions" + ) + private final StatsLogger exceptionLogger; + @StatsDoc( + name = REREPLICATE_OP, + help = "operation stats of re-replicating ledgers" + ) private final OpStatsLogger rereplicateOpStats; + @StatsDoc( + name = NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED, + help = "the number of ledgers re-replicated" + ) private final Counter numLedgersReplicated; + @StatsDoc( + name = NUM_DEFER_LEDGER_LOCK_RELEASE_OF_FAILED_LEDGER, + help = "the number of defer-ledger-lock-releases of failed ledgers" + ) private final Counter numDeferLedgerLockReleaseOfFailedLedger; private final Map exceptionCounters; final LoadingCache replicationFailedLedgers; @@ -164,6 +187,7 @@ public AtomicInteger load(Long key) throws Exception { // Expose Stats this.statsLogger = statsLogger; + this.exceptionLogger = statsLogger.scope(REPLICATE_EXCEPTION); this.rereplicateOpStats = this.statsLogger.getOpStatsLogger(REREPLICATE_OP); this.numLedgersReplicated = this.statsLogger.getCounter(NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED); this.numDeferLedgerLockReleaseOfFailedLedger = this.statsLogger @@ -561,7 +585,7 @@ 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); + counter = this.exceptionLogger.getCounter(name); this.exceptionCounters.put(name, counter); } return counter; From cf546bdd1666b8e15c49d2514052c940b9130465 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 14 Dec 2018 09:28:26 +0800 Subject: [PATCH 0176/1642] [STATS] [DOC] Add @StatsDoc annotation for journal stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document the journal stats. *Changes* - convert journal stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Jia Zhai , Enrico Olivelli This closes #1870 from sijie/statsdoc_journal --- .../org/apache/bookkeeper/bookie/Journal.java | 103 ++++------ .../bookkeeper/bookie/stats/JournalStats.java | 190 ++++++++++++++++++ .../bookie/BookieJournalForceTest.java | 4 +- 3 files changed, 232 insertions(+), 65 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/JournalStats.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 57f05c274f6..a868536169f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -46,6 +46,7 @@ import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException; +import org.apache.bookkeeper.bookie.stats.JournalStats; import org.apache.bookkeeper.common.collections.RecyclableArrayList; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; @@ -363,8 +364,9 @@ public class ForceWriteRequest { private long enqueueTime; public int process(boolean shouldForceWrite) throws IOException { - forceWriteQueueSize.dec(); - fwEnqueueTimeStats.registerSuccessfulEvent(MathUtils.elapsedNanos(enqueueTime), TimeUnit.NANOSECONDS); + journalStats.getForceWriteQueueSize().dec(); + journalStats.getFwEnqueueTimeStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(enqueueTime), TimeUnit.NANOSECONDS); if (isMarker) { return 0; @@ -374,7 +376,8 @@ public int process(boolean shouldForceWrite) throws IOException { if (shouldForceWrite) { long startTime = MathUtils.nowInNano(); this.logFile.forceWrite(false); - journalSyncStats.registerSuccessfulEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); + journalStats.getJournalSyncStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); } lastLogMark.setCurLogMark(this.logId, this.lastFlushedPosition); @@ -384,7 +387,7 @@ public int process(boolean shouldForceWrite) throws IOException { if (qe != null) { cbThreadPool.execute(qe); } - journalCbQueueSize.inc(); + journalStats.getJournalCbQueueSize().inc(); } return forceWriteWaiters.size(); @@ -438,7 +441,7 @@ private ForceWriteRequest createForceWriteRequest(JournalChannel logFile, req.shouldClose = shouldClose; req.isMarker = isMarker; req.enqueueTime = MathUtils.nowInNano(); - forceWriteQueueSize.inc(); + journalStats.getForceWriteQueueSize().inc(); return req; } @@ -490,7 +493,8 @@ public void run() { // the last force write and then reset the counter so we can accumulate // requests in the write we are about to issue if (numReqInLastForceWrite > 0) { - forceWriteGroupingCountStats.registerSuccessfulValue(numReqInLastForceWrite); + journalStats.getForceWriteGroupingCountStats() + .registerSuccessfulValue(numReqInLastForceWrite); numReqInLastForceWrite = 0; } } @@ -614,24 +618,7 @@ static void writePaddingBytes(JournalChannel jc, ByteBuf paddingBuffer, int jour private final LedgerDirsManager ledgerDirsManager; // Expose Stats - private final OpStatsLogger journalAddEntryStats; - private final OpStatsLogger journalForceLedgerStats; - private final OpStatsLogger journalSyncStats; - private final OpStatsLogger fwEnqueueTimeStats; - private final OpStatsLogger journalCreationStats; - private final OpStatsLogger journalFlushStats; - private final OpStatsLogger journalProcessTimeStats; - private final OpStatsLogger journalQueueStats; - private final OpStatsLogger forceWriteGroupingCountStats; - private final OpStatsLogger forceWriteBatchEntriesStats; - private final OpStatsLogger forceWriteBatchBytesStats; - private final Counter journalQueueSize; - private final Counter forceWriteQueueSize; - private final Counter journalCbQueueSize; - private final Counter flushMaxWaitCounter; - private final Counter flushMaxOutstandingBytesCounter; - private final Counter flushEmptyQueueCounter; - private final Counter journalWriteBytes; + private final JournalStats journalStats; public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf, LedgerDirsManager ledgerDirsManager) { @@ -679,27 +666,11 @@ public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf } // Expose Stats - journalAddEntryStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_ADD_ENTRY); - journalForceLedgerStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_LEDGER); - journalSyncStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_SYNC); - fwEnqueueTimeStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_WRITE_ENQUEUE); - journalCreationStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_CREATION_LATENCY); - journalFlushStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FLUSH_LATENCY); - journalQueueStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_QUEUE_LATENCY); - journalProcessTimeStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_PROCESS_TIME_LATENCY); - forceWriteGroupingCountStats = - statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_WRITE_GROUPING_COUNT); - forceWriteBatchEntriesStats = - statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_WRITE_BATCH_ENTRIES); - forceWriteBatchBytesStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_WRITE_BATCH_BYTES); - journalQueueSize = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_QUEUE_SIZE); - forceWriteQueueSize = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_FORCE_WRITE_QUEUE_SIZE); - journalCbQueueSize = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_CB_QUEUE_SIZE); - flushMaxWaitCounter = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_NUM_FLUSH_MAX_WAIT); - flushMaxOutstandingBytesCounter = - statsLogger.getCounter(BookKeeperServerStats.JOURNAL_NUM_FLUSH_MAX_OUTSTANDING_BYTES); - flushEmptyQueueCounter = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_NUM_FLUSH_EMPTY_QUEUE); - journalWriteBytes = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_WRITE_BYTES); + this.journalStats = new JournalStats(statsLogger); + } + + JournalStats getJournalStats() { + return this.journalStats; } public File getJournalDirectory() { @@ -891,18 +862,20 @@ void logAddEntry(long ledgerId, long entryId, ByteBuf entry, boolean ackBeforeSy //Retain entry until it gets written to journal entry.retain(); - journalQueueSize.inc(); + journalStats.getJournalQueueSize().inc(); queue.put(QueueEntry.create( entry, ackBeforeSync, ledgerId, entryId, cb, ctx, MathUtils.nowInNano(), - journalAddEntryStats, journalQueueSize)); + journalStats.getJournalAddEntryStats(), + journalStats.getJournalQueueSize())); } void forceLedger(long ledgerId, WriteCallback cb, Object ctx) { - journalQueueSize.inc(); + journalStats.getJournalQueueSize().inc(); queue.add(QueueEntry.create( null, false /* ackBeforeSync */, ledgerId, Bookie.METAENTRY_ID_FORCE_LEDGER, cb, ctx, MathUtils.nowInNano(), - journalForceLedgerStats, journalQueueSize)); + journalStats.getJournalForceLedgerStats(), + journalStats.getJournalQueueSize())); } /** @@ -968,7 +941,7 @@ public void run() { journalAlignmentSize, removePagesFromCache, journalFormatVersionToWrite, getBufferedChannelBuilder()); - journalCreationStats.registerSuccessfulEvent( + journalStats.getJournalCreationStats().registerSuccessfulEvent( journalCreationWatcher.stop().elapsed(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS); bc = logFile.getBufferedChannel(); @@ -978,15 +951,15 @@ public void run() { if (qe == null) { if (dequeueStartTime != 0) { - journalProcessTimeStats.registerSuccessfulEvent(MathUtils.elapsedNanos(dequeueStartTime), - TimeUnit.NANOSECONDS); + journalStats.getJournalProcessTimeStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(dequeueStartTime), TimeUnit.NANOSECONDS); } if (numEntriesToFlush == 0) { qe = queue.take(); dequeueStartTime = MathUtils.nowInNano(); - journalQueueStats.registerSuccessfulEvent(MathUtils.elapsedNanos(qe.enqueueTime), - TimeUnit.NANOSECONDS); + journalStats.getJournalQueueStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(qe.enqueueTime), TimeUnit.NANOSECONDS); } else { long pollWaitTimeNanos = maxGroupWaitInNanos - MathUtils.elapsedNanos(toFlush.get(0).enqueueTime); @@ -997,8 +970,8 @@ public void run() { dequeueStartTime = MathUtils.nowInNano(); if (qe != null) { - journalQueueStats.registerSuccessfulEvent(MathUtils.elapsedNanos(qe.enqueueTime), - TimeUnit.NANOSECONDS); + journalStats.getJournalQueueStats() + .registerSuccessfulEvent(MathUtils.elapsedNanos(qe.enqueueTime), TimeUnit.NANOSECONDS); } boolean shouldFlush = false; @@ -1017,20 +990,20 @@ public void run() { // b) limit the number of entries to group groupWhenTimeout = false; shouldFlush = true; - flushMaxWaitCounter.inc(); + journalStats.getFlushMaxWaitCounter().inc(); } else if (qe != null && ((bufferedEntriesThreshold > 0 && toFlush.size() > bufferedEntriesThreshold) || (bc.position() > lastFlushPosition + bufferedWritesThreshold))) { // 2. If we have buffered more than the buffWriteThreshold or bufferedEntriesThreshold shouldFlush = true; - flushMaxOutstandingBytesCounter.inc(); + journalStats.getFlushMaxOutstandingBytesCounter().inc(); } else if (qe == null) { // We should get here only if we flushWhenQueueEmpty is true else we would wait // for timeout that would put is past the maxWait threshold // 3. If the queue is empty i.e. no benefit of grouping. This happens when we have one // publish at a time - common case in tests. shouldFlush = true; - flushEmptyQueueCounter.inc(); + journalStats.getFlushEmptyQueueCounter().inc(); } // toFlush is non null and not empty so should be safe to access getFirst @@ -1051,7 +1024,7 @@ public void run() { } lastFlushPosition = bc.position(); - journalFlushStats.registerSuccessfulEvent( + journalStats.getJournalFlushStats().registerSuccessfulEvent( journalFlushWatcher.stop().elapsed(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS); // Trace the lifetime of entries through persistence @@ -1064,8 +1037,10 @@ public void run() { } } - forceWriteBatchEntriesStats.registerSuccessfulValue(numEntriesToFlush); - forceWriteBatchBytesStats.registerSuccessfulValue(batchSize); + journalStats.getForceWriteBatchEntriesStats() + .registerSuccessfulValue(numEntriesToFlush); + journalStats.getForceWriteBatchBytesStats() + .registerSuccessfulValue(batchSize); boolean shouldRolloverJournal = (lastFlushPosition > maxJournalSize); if (syncData) { @@ -1120,8 +1095,8 @@ public void run() { qe.entry.release(); } else if (qe.entryId != Bookie.METAENTRY_ID_FORCE_LEDGER) { int entrySize = qe.entry.readableBytes(); - journalWriteBytes.add(entrySize); - journalQueueSize.dec(); + journalStats.getJournalWriteBytes().add(entrySize); + journalStats.getJournalQueueSize().dec(); batchSize += (4 + entrySize); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/JournalStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/JournalStats.java new file mode 100644 index 00000000000..7c7f7e48a67 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/JournalStats.java @@ -0,0 +1,190 @@ +/* + * 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.stats; + +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ADD_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FORCE_LEDGER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_ADD_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_CB_QUEUE_SIZE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_CREATION_LATENCY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_FLUSH_LATENCY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_FORCE_LEDGER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_FORCE_WRITE_BATCH_BYTES; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_FORCE_WRITE_BATCH_ENTRIES; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_FORCE_WRITE_ENQUEUE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_FORCE_WRITE_GROUPING_COUNT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_FORCE_WRITE_QUEUE_SIZE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_NUM_FLUSH_EMPTY_QUEUE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_NUM_FLUSH_MAX_OUTSTANDING_BYTES; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_NUM_FLUSH_MAX_WAIT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_PROCESS_TIME_LATENCY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_QUEUE_LATENCY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_QUEUE_SIZE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_SYNC; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_WRITE_BYTES; + +import lombok.Getter; +import org.apache.bookkeeper.bookie.BookKeeperServerStats; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for journal related stats. + */ +@StatsDoc( + name = JOURNAL_SCOPE, + category = CATEGORY_SERVER, + help = "Journal related stats" +) +@Getter +public class JournalStats { + + @StatsDoc( + name = JOURNAL_ADD_ENTRY, + help = "operation stats of recording addEntry requests in the journal", + parent = ADD_ENTRY + ) + private final OpStatsLogger journalAddEntryStats; + @StatsDoc( + name = JOURNAL_FORCE_LEDGER, + help = "operation stats of recording forceLedger requests in the journal", + parent = FORCE_LEDGER + ) + private final OpStatsLogger journalForceLedgerStats; + @StatsDoc( + name = JOURNAL_SYNC, + help = "operation stats of syncing data to journal disks", + parent = JOURNAL_ADD_ENTRY, + happensAfter = JOURNAL_FORCE_WRITE_ENQUEUE + ) + private final OpStatsLogger journalSyncStats; + @StatsDoc( + name = JOURNAL_FORCE_WRITE_ENQUEUE, + help = "operation stats of enqueueing force write requests to force write queue", + parent = JOURNAL_ADD_ENTRY, + happensAfter = JOURNAL_PROCESS_TIME_LATENCY + ) + private final OpStatsLogger fwEnqueueTimeStats; + @StatsDoc( + name = JOURNAL_CREATION_LATENCY, + help = "operation stats of creating journal files", + parent = JOURNAL_PROCESS_TIME_LATENCY + ) + private final OpStatsLogger journalCreationStats; + @StatsDoc( + name = JOURNAL_FLUSH_LATENCY, + help = "operation stats of flushing data from memory to filesystem (but not yet fsyncing to disks)", + parent = JOURNAL_PROCESS_TIME_LATENCY, + happensAfter = JOURNAL_CREATION_LATENCY + ) + private final OpStatsLogger journalFlushStats; + @StatsDoc( + name = JOURNAL_PROCESS_TIME_LATENCY, + help = "operation stats of processing requests in a journal (from dequeue an item to finish processing it)", + parent = JOURNAL_ADD_ENTRY, + happensAfter = JOURNAL_QUEUE_LATENCY + ) + private final OpStatsLogger journalProcessTimeStats; + @StatsDoc( + name = JOURNAL_QUEUE_LATENCY, + help = "operation stats of enqueuing requests to a journal", + parent = JOURNAL_ADD_ENTRY + ) + private final OpStatsLogger journalQueueStats; + @StatsDoc( + name = JOURNAL_FORCE_WRITE_GROUPING_COUNT, + help = "The distribution of number of force write requests grouped in a force write" + ) + private final OpStatsLogger forceWriteGroupingCountStats; + @StatsDoc( + name = JOURNAL_FORCE_WRITE_BATCH_ENTRIES, + help = "The distribution of number of entries grouped together into a force write request" + ) + private final OpStatsLogger forceWriteBatchEntriesStats; + @StatsDoc( + name = JOURNAL_FORCE_WRITE_BATCH_BYTES, + help = "The distribution of number of bytes grouped together into a force write request" + ) + private final OpStatsLogger forceWriteBatchBytesStats; + @StatsDoc( + name = JOURNAL_QUEUE_SIZE, + help = "The journal queue size" + ) + private final Counter journalQueueSize; + @StatsDoc( + name = JOURNAL_FORCE_WRITE_QUEUE_SIZE, + help = "The force write queue size" + ) + private final Counter forceWriteQueueSize; + @StatsDoc( + name = JOURNAL_CB_QUEUE_SIZE, + help = "The journal callback queue size" + ) + private final Counter journalCbQueueSize; + @StatsDoc( + name = JOURNAL_NUM_FLUSH_MAX_WAIT, + help = "The number of journal flushes triggered by MAX_WAIT time" + ) + private final Counter flushMaxWaitCounter; + @StatsDoc( + name = JOURNAL_NUM_FLUSH_MAX_OUTSTANDING_BYTES, + help = "The number of journal flushes triggered by MAX_OUTSTANDING_BYTES" + ) + private final Counter flushMaxOutstandingBytesCounter; + @StatsDoc( + name = JOURNAL_NUM_FLUSH_EMPTY_QUEUE, + help = "The number of journal flushes triggered when journal queue becomes empty" + ) + private final Counter flushEmptyQueueCounter; + @StatsDoc( + name = JOURNAL_WRITE_BYTES, + help = "The number of bytes appended to the journal" + ) + private final Counter journalWriteBytes; + + public JournalStats(StatsLogger statsLogger) { + journalAddEntryStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_ADD_ENTRY); + journalForceLedgerStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_LEDGER); + journalSyncStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_SYNC); + fwEnqueueTimeStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_WRITE_ENQUEUE); + journalCreationStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_CREATION_LATENCY); + journalFlushStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FLUSH_LATENCY); + journalQueueStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_QUEUE_LATENCY); + journalProcessTimeStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_PROCESS_TIME_LATENCY); + forceWriteGroupingCountStats = + statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_WRITE_GROUPING_COUNT); + forceWriteBatchEntriesStats = + statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_WRITE_BATCH_ENTRIES); + forceWriteBatchBytesStats = statsLogger.getOpStatsLogger(BookKeeperServerStats.JOURNAL_FORCE_WRITE_BATCH_BYTES); + journalQueueSize = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_QUEUE_SIZE); + forceWriteQueueSize = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_FORCE_WRITE_QUEUE_SIZE); + journalCbQueueSize = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_CB_QUEUE_SIZE); + flushMaxWaitCounter = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_NUM_FLUSH_MAX_WAIT); + flushMaxOutstandingBytesCounter = + statsLogger.getCounter(BookKeeperServerStats.JOURNAL_NUM_FLUSH_MAX_OUTSTANDING_BYTES); + flushEmptyQueueCounter = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_NUM_FLUSH_EMPTY_QUEUE); + journalWriteBytes = statsLogger.getCounter(BookKeeperServerStats.JOURNAL_WRITE_BYTES); + } + +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java index 76daa7d56ad..3ac507c8e98 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java @@ -42,6 +42,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.Journal.ForceWriteRequest; import org.apache.bookkeeper.bookie.Journal.LastLogMark; +import org.apache.bookkeeper.bookie.stats.JournalStats; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -210,10 +211,11 @@ public void testAckBeforeSyncWithJournalBufferedEntriesThreshold() throws Except CountDownLatch forceWriteThreadSuspendedLatch = new CountDownLatch(1); enableForceWriteThreadSuspension(forceWriteThreadSuspendedLatch, journal); + JournalStats journalStats = journal.getJournalStats(); TestStatsProvider testStatsProvider = new TestStatsProvider(); Counter flushMaxOutstandingBytesCounter = testStatsProvider.getStatsLogger("test") .getCounter("flushMaxOutstandingBytesCounter"); - Whitebox.setInternalState(journal, "flushMaxOutstandingBytesCounter", flushMaxOutstandingBytesCounter); + Whitebox.setInternalState(journalStats, "flushMaxOutstandingBytesCounter", flushMaxOutstandingBytesCounter); journal.start(); From 1aa981f0286ac72e736039644d58f6b77b4b35c3 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 14 Dec 2018 09:43:33 +0800 Subject: [PATCH 0177/1642] BP-37: Improve configuration management for better documentation Descriptions of the changes in this PR: *Motivation* One common task in developing bookkeeper is to make sure all the configuration settings are well documented, and the configuration file we ship in each release is in-sync with the code itself. However maintaining things in-sync is non-trivial. This proposal is exploring a new way to manage configuration settings for better documentation. Master Issue: #1867 Reviewers: Matteo Merli , Enrico Olivelli , Jia Zhai This closes #1868 from sijie/bp37_conf_documentation --- site/bps/BP-37-conf-documentation.md | 222 +++++++++++++++++++++++++ site/community/bookkeeper_proposals.md | 3 +- 2 files changed, 224 insertions(+), 1 deletion(-) create mode 100644 site/bps/BP-37-conf-documentation.md diff --git a/site/bps/BP-37-conf-documentation.md b/site/bps/BP-37-conf-documentation.md new file mode 100644 index 00000000000..4c9d0d9388f --- /dev/null +++ b/site/bps/BP-37-conf-documentation.md @@ -0,0 +1,222 @@ +--- +title: "BP-37: Improve configuration management for better documentation" +issue: https://github.com/apache/bookkeeper/1867 +state: "Accepted" +release: "4.9.0" +--- + +### Motivation + +One common task in developing bookkeeper is to make sure all the configuration +settings are well documented, and the configuration file we ship in each release +is in-sync with the code itself. + +However maintaining things in-sync is non-trivial. This proposal is exploring +a new way to manage configuration settings for better documentation. + +### Public Interfaces + +1. Introduced `ConfigKey` for defining a configuration key. A configuration key + will include informations, such as required/optional, deprecated, documentation + and etc. + +```java +public class ConfigKey { + /** + * Flag indicates whether the setting is required. + */ + @Default + private boolean required = false; + + /** + * Name of the configuration setting. + */ + private String name; + + /** + * Type of the configuration setting. + */ + @Default + private Type type = Type.STRING; + + /** + * Description of the configuration setting. + */ + @Default + private String description = ""; + + /** + * Documentation of the configuration setting. + */ + @Default + private String documentation = ""; + + /** + * Default value as a string representation. + */ + @Default + private Object defaultValue = null; + + /** + * The list of options for this setting. + */ + @Default + private List optionValues = Collections.emptyList(); + + /** + * The validator used for validating configuration value. + */ + @Default + private Validator validator = NullValidator.of(); + + /** + * The key-group to group settings together. + */ + @Default + private ConfigKeyGroup group = ConfigKeyGroup.DEFAULT; + + /** + * The order of the setting in the key-group. + */ + @Default + private int orderInGroup = Integer.MIN_VALUE; + + /** + * The list of settings dependents on this setting. + */ + @Default + private List dependents = Collections.emptyList(); + + /** + * Whether this setting is deprecated or not. + */ + @Default + private boolean deprecated = false; + + /** + * The config key that deprecates this key. + */ + @Default + private String deprecatedByConfigKey = ""; + + /** + * The version when this settings was deprecated. + */ + @Default + private String deprecatedSince = ""; + + /** + * The version when this setting was introduced. + */ + @Default + private String since = ""; +} +``` + +2. Introduced `ConfigKeyGroup` for grouping configuration keys together. + +```java +public class ConfigKeyGroup { + /** + * Name of the key group. + */ + private String name; + + /** + * Description of the key group. + */ + @Default + private String description = ""; + + /** + * The list of sub key-groups of this key group. + */ + @Default + private List children = Collections.emptyList(); + + /** + * The order of the key-group in a configuration. + */ + @Default + private int order = Integer.MIN_VALUE; +} +``` + +### Proposed Changes + +Besides introducing `ConfigKey` and `ConfigKeyGroup`, this BP will also introduce a class +`ConfigDef` - it defines the keys for a configuration. + +The `ConfigDef` will be generated via `ConfigDef.of(Configuration.class)`. It will retrieve +all the static fields of `ConfigKey` defined in the configuration class and build the configuration +definition. + +The `ConfigDef` will also provide a `save` method for saving the configuration definition +as a configuration file. + +### Example + +Following is an example how to use `ConfigKey` and `ConfigKeyGroup` to organize +configuration settings. + +```java +// Ledger Storage Settings + +private static final ConfigKeyGroup GROUP_LEDGER_STORAGE = ConfigKeyGroup.builder("ledgerstorage") + .description("Ledger Storage related settings") + .order(10) // place a place holder here + .build(); + +protected static final String LEDGER_STORAGE_CLASS = "ledgerStorageClass"; +protected static final ConfigKey LEDGER_STORAGE_CLASS_KEY = ConfigKey.builder(LEDGER_STORAGE_CLASS) + .type(Type.CLASS) + .description("Ledger storage implementation class") + .defaultValue(SortedLedgerStorage.class.getName()) + .optionValues(Lists.newArrayList( + InterleavedLedgerStorage.class.getName(), + SortedLedgerStorage.class.getName(), + DbLedgerStorage.class.getName() + )) + .validator(ClassValidator.of(LedgerStorage.class)) + .group(GROUP_LEDGER_STORAGE) + .build(); +``` + +Example on how to generate the `ConfigDef` and use the configuration definition to +validate if a configuration instance is valid. + +```java +// generate config def +ConfigDef configDef = ConfigDef.of(ServerConfiguration.class); +try { + configDef.validate(this); +} catch (ConfigException e) { + throw new ConfigurationException(e.getMessage(), e.getCause()); +} +``` + +Example on how to save the configuration definition to a configuration file. + +```java +ConfigDef configDef = ConfigDef.of(TestConfig2.class); +String savedConf; +try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + configDef.save(baos); + savedConf = baos.toString(); +} +``` + +### Compatibility, Deprecation, and Migration Plan + +It only changes the way how we organize configuration settings and how we document them. +It doesn't change the public interfaces for existing configuration. So there is nothing +to deprecate and migrate. + +### Test Plan + +Existing testing is good enough to cover code changes. No new tests are needed. + +### Rejected Alternatives + +Alternatively, we have to manually maintain the configuration files and update each time +when a new configuration setting is added. diff --git a/site/community/bookkeeper_proposals.md b/site/community/bookkeeper_proposals.md index 58244087bb4..c15e1d26d5c 100644 --- a/site/community/bookkeeper_proposals.md +++ b/site/community/bookkeeper_proposals.md @@ -85,7 +85,7 @@ using Google Doc. This section lists all the _bookkeeper proposals_ made to BookKeeper. -*Next Proposal Number: 37* +*Next Proposal Number: 38* ### Inprogress @@ -108,6 +108,7 @@ Proposal | State [BP-34: Cluster Metadata Checker](../../bps/BP-34-cluster-metadata-checker) | Accepted [BP-35: 128 bits support](../../bps/BP-35-128-bits-support) | Accepted [BP-36: Stats documentation annotation](../../bps/BP-36-stats-documentation-annotation) | Accepted +[BP-37: Improve configuration management for better documentation](../../bps/BP-37-conf-documentation) | Accepted ### Adopted From 57b833eeefb58e711284474d6b9f8d5a66bd8391 Mon Sep 17 00:00:00 2001 From: Pasha Kuznetsov Date: Thu, 13 Dec 2018 17:45:11 -0800 Subject: [PATCH 0178/1642] Issue #1884: (@W-5697664@) dir_*_usage stats are reported as 0 ... for read-only bookies after a restart ### Motivation Fixing the Issue #1884: When a read-only bookie is restarted it keeps reporting `dir_*_usage` stats as `0` until it becomes writable again. This is caused by the `LedgerDirsMonitor.check` only updating `diskUsages` if there are any writable dirs, or if the total usage goes below the low water mark, otherwise relying on previously filled values which are `0` after a bookie is restarted. ### Changes * Change the `LedgerDirsMonitor.check` to update `diskUsages` even when there are no writable dirs. * Add new `testLedgerDirsMonitorStartReadOnly` testing this scenario. * Simplify previous tests checking read-only since `diskUsages` are now updated regardless if a bookie is in read-only mode. jvrao reddycharan Reviewers: Sijie Guo , Charan Reddy Guttapalem This closes #1885 from pasha-kuznetsov/issue-1884-dir-usage-ro-restart, closes #1884 --- .../bookkeeper/bookie/LedgerDirsMonitor.java | 65 ++++++++-------- .../bookie/TestLedgerDirsManager.java | 76 ++++++++++++++----- 2 files changed, 90 insertions(+), 51 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java index fedebb74222..d0033669320 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java @@ -122,43 +122,46 @@ private void check() { } List fullfilledDirs = new ArrayList(ldm.getFullFilledLedgerDirs()); - boolean hasWritableLedgerDirs = ldm.hasWritableLedgerDirs(); - float totalDiskUsage = 0; + boolean makeWritable = ldm.hasWritableLedgerDirs(); - // When bookie is in READONLY mode .i.e there are no writableLedgerDirs: - // - Check if the total disk usage is below DiskLowWaterMarkUsageThreshold. - // - If So, walk through the entire list of fullfilledDirs and add them back to writableLedgerDirs list if - // their usage is < conf.getDiskUsageThreshold. + // When bookie is in READONLY mode, i.e there are no writableLedgerDirs: + // - Update fullfilledDirs disk usage. + // - If the total disk usage is below DiskLowWaterMarkUsageThreshold + // add fullfilledDirs back to writableLedgerDirs list if their usage is < conf.getDiskUsageThreshold. try { - if (hasWritableLedgerDirs - || (totalDiskUsage = diskChecker.getTotalDiskUsage(ldm.getAllLedgerDirs())) < conf - .getDiskLowWaterMarkUsageThreshold()) { - // Check all full-filled disk space usage - for (File dir : fullfilledDirs) { - try { - diskUsages.put(dir, diskChecker.checkDir(dir)); + if (!makeWritable) { + float totalDiskUsage = diskChecker.getTotalDiskUsage(ldm.getAllLedgerDirs()); + if (totalDiskUsage < conf.getDiskLowWaterMarkUsageThreshold()) { + makeWritable = true; + } else { + LOG.debug( + "Current TotalDiskUsage: {} is greater than LWMThreshold: {}." + + " So not adding any filledDir to WritableDirsList", + totalDiskUsage, conf.getDiskLowWaterMarkUsageThreshold()); + } + } + // Update all full-filled disk space usage + for (File dir : fullfilledDirs) { + try { + diskUsages.put(dir, diskChecker.checkDir(dir)); + if (makeWritable) { ldm.addToWritableDirs(dir, true); - } catch (DiskErrorException e) { - // Notify disk failure to all the listeners - for (LedgerDirsListener listener : ldm.getListeners()) { - listener.diskFailed(dir); - } - } catch (DiskWarnThresholdException e) { - diskUsages.put(dir, e.getUsage()); - // the full-filled dir become writable but still - // above - // warn threshold + } + } catch (DiskErrorException e) { + // Notify disk failure to all the listeners + for (LedgerDirsListener listener : ldm.getListeners()) { + listener.diskFailed(dir); + } + } catch (DiskWarnThresholdException e) { + diskUsages.put(dir, e.getUsage()); + // the full-filled dir become writable but still above the warn threshold + if (makeWritable) { ldm.addToWritableDirs(dir, false); - } catch (DiskOutOfSpaceException e) { - // the full-filled dir is still full-filled - diskUsages.put(dir, e.getUsage()); } + } catch (DiskOutOfSpaceException e) { + // the full-filled dir is still full-filled + diskUsages.put(dir, e.getUsage()); } - } else { - LOG.debug( - "Current TotalDiskUsage: {} is greater than LWMThreshold: {}." - + " So not adding any filledDir to WritableDirsList", - totalDiskUsage, conf.getDiskLowWaterMarkUsageThreshold()); } } catch (IOException ioe) { LOG.error("Got IOException while monitoring Dirs", ioe); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java index b8555ca999e..9d166a4ebf5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java @@ -21,8 +21,6 @@ package org.apache.bookkeeper.bookie; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -243,7 +241,6 @@ private void testLedgerDirsMonitorDuringTransition(boolean highPriorityWritesAll @Test public void testLedgerDirsMonitorHandlingLowWaterMark() throws Exception { - ledgerMonitor.shutdown(); final float warn = 0.90f; @@ -370,13 +367,13 @@ public void testLedgerDirsMonitorHandlingWithMultipleLedgerDirectories() throws // should goto readwrite setUsageAndThenVerify(curDir1, lwm - 0.17f, curDir2, nospace + 0.03f, mockDiskChecker, mockLedgerDirsListener, false); - assertTrue("Only one LedgerDir should be writable", dirsManager.getWritableLedgerDirs().size() == 1); + assertEquals("Only one LedgerDir should be writable", 1, dirsManager.getWritableLedgerDirs().size()); // bring both the dirs below lwm // should still be readwrite setUsageAndThenVerify(curDir1, lwm - 0.03f, curDir2, lwm - 0.02f, mockDiskChecker, mockLedgerDirsListener, false); - assertTrue("Both the LedgerDirs should be writable", dirsManager.getWritableLedgerDirs().size() == 2); + assertEquals("Both the LedgerDirs should be writable", 2, dirsManager.getWritableLedgerDirs().size()); // bring both the dirs above lwm but < threshold // should still be readwrite @@ -384,6 +381,52 @@ public void testLedgerDirsMonitorHandlingWithMultipleLedgerDirectories() throws false); } + @Test + public void testLedgerDirsMonitorStartReadOnly() throws Exception { + ledgerMonitor.shutdown(); + + final float nospace = 0.90f; + final float lwm = 0.80f; + + File tmpDir1 = createTempDir("bkTest", ".dir"); + File curDir1 = Bookie.getCurrentDirectory(tmpDir1); + Bookie.checkDirectoryStructure(curDir1); + + File tmpDir2 = createTempDir("bkTest", ".dir"); + File curDir2 = Bookie.getCurrentDirectory(tmpDir2); + Bookie.checkDirectoryStructure(curDir2); + + conf.setDiskUsageThreshold(nospace); + conf.setDiskLowWaterMarkUsageThreshold(lwm); + conf.setDiskUsageWarnThreshold(nospace); + conf.setLedgerDirNames(new String[] { tmpDir1.toString(), tmpDir2.toString() }); + + // Both disks are out of space at the start. + HashMap usageMap = new HashMap<>(); + usageMap.put(curDir1, nospace + 0.05f); + usageMap.put(curDir2, nospace + 0.05f); + + mockDiskChecker = new MockDiskChecker(nospace, warnThreshold); + mockDiskChecker.setUsageMap(usageMap); + dirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), + new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), + statsLogger); + + ledgerMonitor = new LedgerDirsMonitor(conf, mockDiskChecker, dirsManager); + try { + ledgerMonitor.init(); + fail("NoWritableLedgerDirException expected"); + } catch (NoWritableLedgerDirException exception) { + // ok + } + final MockLedgerDirsListener mockLedgerDirsListener = new MockLedgerDirsListener(); + dirsManager.addLedgerDirsListener(mockLedgerDirsListener); + ledgerMonitor.start(); + + Thread.sleep((diskCheckInterval * 2) + 100); + verifyUsage(curDir1, nospace + 0.05f, curDir2, nospace + 0.05f, mockLedgerDirsListener, true); + } + private void setUsageAndThenVerify(File dir1, float dir1Usage, File dir2, float dir2Usage, MockDiskChecker mockDiskChecker, MockLedgerDirsListener mockLedgerDirsListener, boolean verifyReadOnly) throws InterruptedException { @@ -391,26 +434,19 @@ private void setUsageAndThenVerify(File dir1, float dir1Usage, File dir2, float usageMap.put(dir1, dir1Usage); usageMap.put(dir2, dir2Usage); mockDiskChecker.setUsageMap(usageMap); + verifyUsage(dir1, dir1Usage, dir2, dir2Usage, mockLedgerDirsListener, verifyReadOnly); + } + + private void verifyUsage(File dir1, float dir1Usage, File dir2, float dir2Usage, + MockLedgerDirsListener mockLedgerDirsListener, boolean verifyReadOnly) { executorController.advance(Duration.ofMillis(diskCheckInterval)); float sample1 = getGauge(dir1.getParent()).getSample().floatValue(); float sample2 = getGauge(dir2.getParent()).getSample().floatValue(); - if (verifyReadOnly) { - assertTrue(mockLedgerDirsListener.readOnly); - - // LedgerDirsMonitor stops updating diskUsages when the bookie is in the readonly mode, - // so the stats will reflect an older value at the time when the bookie became readonly - assertThat(sample1, greaterThan(90f)); - assertThat(sample1, lessThan(100f)); - assertThat(sample2, greaterThan(90f)); - assertThat(sample2, lessThan(100f)); - } else { - assertFalse(mockLedgerDirsListener.readOnly); - - assertThat(sample1, equalTo(dir1Usage * 100f)); - assertThat(sample2, equalTo(dir2Usage * 100f)); - } + assertEquals(mockLedgerDirsListener.readOnly, verifyReadOnly); + assertThat(sample1, equalTo(dir1Usage * 100f)); + assertThat(sample2, equalTo(dir2Usage * 100f)); } private Gauge getGauge(String path) { From 5bcd11bc15ae32f0dd42fda6d4bdcff818730ad7 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sun, 16 Dec 2018 16:20:35 +0800 Subject: [PATCH 0179/1642] [STATS] [DOC] Add @StatsDoc annotation for garbage collector stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document gc stats. *Changes* - convert gc stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Jia Zhai , Enrico Olivelli This closes #1875 from sijie/gc_stats --- .../bookie/GarbageCollectorThread.java | 81 +++------- .../ScanAndCompareGarbageCollector.java | 23 +-- .../bookie/stats/GarbageCollectorStats.java | 151 ++++++++++++++++++ .../apache/bookkeeper/meta/GcLedgersTest.java | 26 +-- 4 files changed, 181 insertions(+), 100 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/GarbageCollectorStats.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java index ca02651e879..006592646c0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java @@ -21,14 +21,6 @@ package org.apache.bookkeeper.bookie; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ACTIVE_ENTRY_LOG_COUNT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ACTIVE_ENTRY_LOG_SPACE_BYTES; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.MAJOR_COMPACTION_COUNT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.MINOR_COMPACTION_COUNT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.RECLAIMED_COMPACTION_SPACE_BYTES; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.RECLAIMED_DELETION_SPACE_BYTES; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.THREAD_RUNTIME; - import com.google.common.annotations.VisibleForTesting; import io.netty.util.concurrent.DefaultThreadFactory; @@ -46,11 +38,9 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.bookie.GarbageCollector.GarbageCleaner; +import org.apache.bookkeeper.bookie.stats.GarbageCollectorStats; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.Gauge; -import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.SafeRunnable; @@ -92,12 +82,7 @@ public class GarbageCollectorThread extends SafeRunnable { final AbstractLogCompactor compactor; // Stats loggers for garbage collection operations - final StatsLogger statsLogger; - private final Counter minorCompactionCounter; - private final Counter majorCompactionCounter; - private final Counter reclaimedSpaceViaDeletes; - private final Counter reclaimedSpaceViaCompaction; - private final OpStatsLogger gcThreadRuntime; + private final GarbageCollectorStats gcStats; private volatile long totalEntryLogSize; private volatile int numActiveEntryLogs; @@ -124,7 +109,7 @@ public class GarbageCollectorThread extends SafeRunnable { // Boolean to disable minor compaction, when disk is full final AtomicBoolean suspendMinorCompaction = new AtomicBoolean(false); - final GarbageCollector garbageCollector; + final ScanAndCompareGarbageCollector garbageCollector; final GarbageCleaner garbageCleaner; final ServerConfiguration conf; @@ -162,52 +147,28 @@ public GarbageCollectorThread(ServerConfiguration conf, this.ledgerStorage = ledgerStorage; this.gcWaitTime = conf.getGcWaitTime(); + this.numActiveEntryLogs = 0; + this.totalEntryLogSize = 0L; + this.garbageCollector = new ScanAndCompareGarbageCollector(ledgerManager, ledgerStorage, conf, statsLogger); + this.gcStats = new GarbageCollectorStats( + statsLogger, + () -> numActiveEntryLogs, + () -> totalEntryLogSize, + () -> garbageCollector.getNumActiveLedgers() + ); + this.garbageCleaner = ledgerId -> { try { if (LOG.isDebugEnabled()) { LOG.debug("delete ledger : " + ledgerId); } + gcStats.getDeletedLedgerCounter().inc(); ledgerStorage.deleteLedger(ledgerId); } catch (IOException e) { LOG.error("Exception when deleting the ledger index file on the Bookie: ", e); } }; - // Stat state initialization - this.statsLogger = statsLogger; - - this.minorCompactionCounter = statsLogger.getCounter(MINOR_COMPACTION_COUNT); - this.majorCompactionCounter = statsLogger.getCounter(MAJOR_COMPACTION_COUNT); - this.reclaimedSpaceViaCompaction = statsLogger.getCounter(RECLAIMED_COMPACTION_SPACE_BYTES); - this.reclaimedSpaceViaDeletes = statsLogger.getCounter(RECLAIMED_DELETION_SPACE_BYTES); - this.gcThreadRuntime = statsLogger.getOpStatsLogger(THREAD_RUNTIME); - this.numActiveEntryLogs = 0; - statsLogger.registerGauge(ACTIVE_ENTRY_LOG_COUNT, new Gauge() { - @Override - public Integer getDefaultValue() { - return 0; - } - - @Override - public Integer getSample() { - return numActiveEntryLogs; - } - }); - this.totalEntryLogSize = 0L; - statsLogger.registerGauge(ACTIVE_ENTRY_LOG_SPACE_BYTES, new Gauge() { - @Override - public Long getDefaultValue() { - return 0L; - } - - @Override - public Long getSample() { - return totalEntryLogSize; - } - }); - - this.garbageCollector = new ScanAndCompareGarbageCollector(ledgerManager, ledgerStorage, conf, statsLogger); - // compaction parameters minorCompactionThreshold = conf.getMinorCompactionThreshold(); minorCompactionInterval = conf.getMinorCompactionInterval() * SECOND; @@ -389,7 +350,7 @@ public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMin lastMajorCompactionTime = System.currentTimeMillis(); // and also move minor compaction time lastMinorCompactionTime = lastMajorCompactionTime; - majorCompactionCounter.inc(); + gcStats.getMajorCompactionCounter().inc(); majorCompacting.set(false); } else if (enableMinorCompaction && (!suspendMinor) && (force || curTime - lastMinorCompactionTime > minorCompactionInterval)) { @@ -398,7 +359,7 @@ public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMin minorCompacting.set(true); doCompactEntryLogs(minorCompactionThreshold); lastMinorCompactionTime = System.currentTimeMillis(); - minorCompactionCounter.inc(); + gcStats.getMinorCompactionCounter().inc(); minorCompacting.set(false); } @@ -408,7 +369,7 @@ public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMin .currentThread().getName()); } } - this.gcThreadRuntime.registerSuccessfulEvent( + gcStats.getGcThreadRuntime().registerSuccessfulEvent( MathUtils.nowInNano() - threadStart, TimeUnit.NANOSECONDS); } @@ -434,7 +395,7 @@ private void doGcEntryLogs() { // We can remove this entry log file now. LOG.info("Deleting entryLogId " + entryLogId + " as it has no active ledgers!"); removeEntryLog(entryLogId); - this.reclaimedSpaceViaDeletes.add(meta.getTotalSize()); + gcStats.getReclaimedSpaceViaDeletes().add(meta.getTotalSize()); } totalEntryLogSizeAcc.getAndAdd(meta.getRemainingSize()); @@ -492,7 +453,7 @@ void doCompactEntryLogs(double threshold) { long priorRemainingSize = meta.getRemainingSize(); compactEntryLog(meta); - this.reclaimedSpaceViaCompaction.add(meta.getTotalSize() - priorRemainingSize); + gcStats.getReclaimedSpaceViaCompaction().add(meta.getTotalSize() - priorRemainingSize); if (!running) { // if gc thread is not running, stop compaction return; @@ -621,8 +582,8 @@ public GarbageCollectionStatus getGarbageCollectionStatus() { .minorCompacting(minorCompacting.get()) .lastMajorCompactionTime(lastMajorCompactionTime) .lastMinorCompactionTime(lastMinorCompactionTime) - .majorCompactionCounter(majorCompactionCounter.get()) - .minorCompactionCounter(minorCompactionCounter.get()) + .majorCompactionCounter(gcStats.getMajorCompactionCounter().get()) + .minorCompactionCounter(gcStats.getMinorCompactionCounter().get()) .build(); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index 6af5d15b027..24c5c97e9cc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -21,8 +21,6 @@ package org.apache.bookkeeper.bookie; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ACTIVE_LEDGER_COUNT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.DELETED_LEDGER_COUNT; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import com.google.common.collect.Sets; @@ -42,8 +40,6 @@ import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; @@ -69,7 +65,7 @@ * *

            TODO: eliminate the direct usage of zookeeper here {@link https://github.com/apache/bookkeeper/issues/1331} */ -public class ScanAndCompareGarbageCollector implements GarbageCollector{ +public class ScanAndCompareGarbageCollector implements GarbageCollector { static final Logger LOG = LoggerFactory.getLogger(ScanAndCompareGarbageCollector.class); static final int MAX_CONCURRENT_ZK_REQUESTS = 1000; @@ -86,7 +82,6 @@ public class ScanAndCompareGarbageCollector implements GarbageCollector{ private final String zkLedgersRootPath; private final boolean verifyMetadataOnGc; private int activeLedgerCounter; - private Counter deletedLedgerCounter; public ScanAndCompareGarbageCollector(LedgerManager ledgerManager, CompactableLedgerStorage ledgerStorage, ServerConfiguration conf, StatsLogger statsLogger) throws IOException { @@ -106,20 +101,11 @@ public ScanAndCompareGarbageCollector(LedgerManager ledgerManager, CompactableLe verifyMetadataOnGc = conf.getVerifyMetadataOnGC(); - this.deletedLedgerCounter = statsLogger.getCounter(DELETED_LEDGER_COUNT); - this.activeLedgerCounter = 0; - statsLogger.registerGauge(ACTIVE_LEDGER_COUNT, new Gauge() { - @Override - public Integer getDefaultValue() { - return 0; - } + } - @Override - public Integer getSample() { - return activeLedgerCounter; - } - }); + public int getNumActiveLedgers() { + return activeLedgerCounter; } @Override @@ -190,7 +176,6 @@ public void gc(GarbageCleaner garbageCleaner) { continue; } } - deletedLedgerCounter.inc(); garbageCleaner.clean(bkLid); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/GarbageCollectorStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/GarbageCollectorStats.java new file mode 100644 index 00000000000..1c9475608f4 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/GarbageCollectorStats.java @@ -0,0 +1,151 @@ +/* + * 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.stats; + +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ACTIVE_ENTRY_LOG_COUNT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ACTIVE_ENTRY_LOG_SPACE_BYTES; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ACTIVE_LEDGER_COUNT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.DELETED_LEDGER_COUNT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.MAJOR_COMPACTION_COUNT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.MINOR_COMPACTION_COUNT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.RECLAIMED_COMPACTION_SPACE_BYTES; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.RECLAIMED_DELETION_SPACE_BYTES; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.THREAD_RUNTIME; + +import java.util.function.Supplier; +import lombok.Getter; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for gc stats. + */ +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "Garbage Collector related stats" +) +@Getter +public class GarbageCollectorStats { + + final StatsLogger statsLogger; + @StatsDoc( + name = MINOR_COMPACTION_COUNT, + help = "Number of minor compactions" + ) + private final Counter minorCompactionCounter; + @StatsDoc( + name = MAJOR_COMPACTION_COUNT, + help = "Number of major compactions" + ) + private final Counter majorCompactionCounter; + @StatsDoc( + name = RECLAIMED_DELETION_SPACE_BYTES, + help = "Number of disk space bytes reclaimed via deleting entry log files" + ) + private final Counter reclaimedSpaceViaDeletes; + @StatsDoc( + name = RECLAIMED_COMPACTION_SPACE_BYTES, + help = "Number of disk space bytes reclaimed via compacting entry log files" + ) + private final Counter reclaimedSpaceViaCompaction; + @StatsDoc( + name = DELETED_LEDGER_COUNT, + help = "Number of ledgers deleted by garbage collection" + ) + private final Counter deletedLedgerCounter; + @StatsDoc( + name = THREAD_RUNTIME, + help = "Operation stats of garbage collections" + ) + private final OpStatsLogger gcThreadRuntime; + @StatsDoc( + name = ACTIVE_ENTRY_LOG_COUNT, + help = "Current number of active entry log files" + ) + private final Gauge activeEntryLogCountGauge; + @StatsDoc( + name = ACTIVE_ENTRY_LOG_SPACE_BYTES, + help = "Current number of active entry log space bytes" + ) + private final Gauge activeEntryLogSpaceBytesGauge; + @StatsDoc( + name = ACTIVE_LEDGER_COUNT, + help = "Current number of active ledgers" + ) + private final Gauge activeLedgerCountGauge; + + public GarbageCollectorStats(StatsLogger statsLogger, + Supplier activeEntryLogCountSupplier, + Supplier activeEntryLogSpaceBytesSupplier, + Supplier activeLedgerCountSupplier) { + this.statsLogger = statsLogger; + + this.minorCompactionCounter = statsLogger.getCounter(MINOR_COMPACTION_COUNT); + this.majorCompactionCounter = statsLogger.getCounter(MAJOR_COMPACTION_COUNT); + this.reclaimedSpaceViaCompaction = statsLogger.getCounter(RECLAIMED_COMPACTION_SPACE_BYTES); + this.reclaimedSpaceViaDeletes = statsLogger.getCounter(RECLAIMED_DELETION_SPACE_BYTES); + this.gcThreadRuntime = statsLogger.getOpStatsLogger(THREAD_RUNTIME); + this.deletedLedgerCounter = statsLogger.getCounter(DELETED_LEDGER_COUNT); + + this.activeEntryLogCountGauge = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + return activeEntryLogCountSupplier.get(); + } + }; + statsLogger.registerGauge(ACTIVE_ENTRY_LOG_COUNT, activeEntryLogCountGauge); + this.activeEntryLogSpaceBytesGauge = new Gauge() { + @Override + public Long getDefaultValue() { + return 0L; + } + + @Override + public Long getSample() { + return activeEntryLogSpaceBytesSupplier.get(); + } + }; + statsLogger.registerGauge(ACTIVE_ENTRY_LOG_SPACE_BYTES, activeEntryLogSpaceBytesGauge); + this.activeLedgerCountGauge = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + return activeLedgerCountSupplier.get(); + } + }; + statsLogger.registerGauge(ACTIVE_LEDGER_COUNT, activeLedgerCountGauge); + } + +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index d2735657e16..828956d4f35 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -21,8 +21,6 @@ package org.apache.bookkeeper.meta; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ACTIVE_LEDGER_COUNT; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.DELETED_LEDGER_COUNT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -173,7 +171,7 @@ public void testGarbageCollectLedgers() throws Exception { final CountDownLatch endLatch = new CountDownLatch(2); final CompactableLedgerStorage mockLedgerStorage = new MockLedgerStorage(); TestStatsProvider stats = new TestStatsProvider(); - final GarbageCollector garbageCollector = new ScanAndCompareGarbageCollector(getLedgerManager(), + final ScanAndCompareGarbageCollector garbageCollector = new ScanAndCompareGarbageCollector(getLedgerManager(), mockLedgerStorage, baseConf, stats.getStatsLogger("gc")); Thread gcThread = new Thread() { @Override @@ -235,12 +233,9 @@ public void run() { for (Long ledger : createdLedgers) { assertTrue(activeLedgers.containsKey(ledger)); } - assertTrue( - "Wrong DELETED_LEDGER_COUNT", - stats.getCounter("gc." + DELETED_LEDGER_COUNT).get() == removedLedgers.size()); assertTrue( "Wrong ACTIVE_LEDGER_COUNT", - stats.getGauge("gc." + ACTIVE_LEDGER_COUNT).getSample().intValue() == createdLedgers.size()); + garbageCollector.getNumActiveLedgers() == createdLedgers.size()); } @Test @@ -253,7 +248,7 @@ public void testGcLedgersOutsideRange() throws Exception { MockLedgerStorage mockLedgerStorage = new MockLedgerStorage(); TestStatsProvider stats = new TestStatsProvider(); - final GarbageCollector garbageCollector = new ScanAndCompareGarbageCollector(getLedgerManager(), + final ScanAndCompareGarbageCollector garbageCollector = new ScanAndCompareGarbageCollector(getLedgerManager(), mockLedgerStorage, baseConf, stats.getStatsLogger("gc")); GarbageCollector.GarbageCleaner cleaner = new GarbageCollector.GarbageCleaner() { @Override @@ -271,37 +266,26 @@ public void clean(long ledgerId) { garbageCollector.gc(cleaner); assertNull("Should have cleaned nothing", cleaned.poll()); - assertTrue( - "Wrong DELETED_LEDGER_COUNT", - stats.getCounter("gc." + DELETED_LEDGER_COUNT).get() == 0); assertTrue( "Wrong ACTIVE_LEDGER_COUNT", - stats.getGauge( - "gc." + ACTIVE_LEDGER_COUNT).getSample().intValue() == numLedgers); + garbageCollector.getNumActiveLedgers() == numLedgers); long last = createdLedgers.last(); removeLedger(last); garbageCollector.gc(cleaner); assertNotNull("Should have cleaned something", cleaned.peek()); assertEquals("Should have cleaned last ledger" + last, (long) last, (long) cleaned.poll()); - assertTrue( - "Wrong DELETED_LEDGER_COUNT", - stats.getCounter("gc." + DELETED_LEDGER_COUNT).get() == 1); long first = createdLedgers.first(); removeLedger(first); garbageCollector.gc(cleaner); assertNotNull("Should have cleaned something", cleaned.peek()); assertEquals("Should have cleaned first ledger" + first, (long) first, (long) cleaned.poll()); - assertTrue( - "Wrong DELETED_LEDGER_COUNT", - stats.getCounter("gc." + DELETED_LEDGER_COUNT).get() == 2); garbageCollector.gc(cleaner); assertTrue( "Wrong ACTIVE_LEDGER_COUNT", - stats.getGauge( - "gc." + ACTIVE_LEDGER_COUNT).getSample().intValue() == (numLedgers - 2)); + garbageCollector.getNumActiveLedgers() == (numLedgers - 2)); } From f5ca2f14730a42f587c321db0d0a8af3f15427c6 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 17 Dec 2018 21:18:49 +0800 Subject: [PATCH 0180/1642] [STATS] [DOC] Add @StatsDoc annotation for ledger cache stats Descriptions of the changes in this PR: *Motivation* As part of [BP-36](https://github.com/apache/bookkeeper/issues/1785), this PR is to document the ledger cache stats. *Changes* - convert ledger cache stats to use StatsDoc for documenting metrics Master Issue: #1785 Reviewers: Enrico Olivelli , Jia Zhai This closes #1872 from sijie/ledgercache_stats --- .../bookkeeper/bookie/IndexInMemPageMgr.java | 11 +- .../bookie/IndexPersistenceMgr.java | 47 ++------ .../bookie/stats/IndexInMemPageMgrStats.java | 60 +++++++++++ .../stats/IndexPersistenceMgrStats.java | 101 ++++++++++++++++++ 4 files changed, 175 insertions(+), 44 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/IndexInMemPageMgrStats.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/IndexPersistenceMgrStats.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java index b2362943244..1e7d432ae83 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java @@ -21,8 +21,6 @@ package org.apache.bookkeeper.bookie; import static java.lang.Long.max; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.INDEX_INMEM_ILLEGAL_STATE_DELETE; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.INDEX_INMEM_ILLEGAL_STATE_RESET; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LEDGER_CACHE_HIT; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LEDGER_CACHE_MISS; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LEDGER_CACHE_READ_PAGE; @@ -43,6 +41,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.bookkeeper.bookie.stats.IndexInMemPageMgrStats; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.Gauge; @@ -64,16 +63,14 @@ private static class InMemPageCollection implements LEPStateChangeCallback { final ConcurrentLinkedQueue listOfFreePages; // Stats - final Counter illegalStateResetCounter; - final Counter illegalStateDeleteCounter; + private final IndexInMemPageMgrStats inMemPageMgrStats; public InMemPageCollection(StatsLogger statsLogger) { pages = new ConcurrentHashMap<>(); lruCleanPageMap = Collections.synchronizedMap(new LinkedHashMap(16, 0.75f, true)); listOfFreePages = new ConcurrentLinkedQueue(); - illegalStateResetCounter = statsLogger.getCounter(INDEX_INMEM_ILLEGAL_STATE_RESET); - illegalStateDeleteCounter = statsLogger.getCounter(INDEX_INMEM_ILLEGAL_STATE_DELETE); + inMemPageMgrStats = new IndexInMemPageMgrStats(statsLogger); } /** @@ -294,7 +291,7 @@ LedgerEntryPage grabCleanPage(long ledgerId, long firstEntry) { public void addToListOfFreePages(LedgerEntryPage lep) { if ((null == lep) || lep.inUse()) { - illegalStateResetCounter.inc(); + inMemPageMgrStats.getIllegalStateResetCounter().inc(); } if (null != lep) { listOfFreePages.add(lep); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java index a622feacca5..8d8f5cfa399 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java @@ -20,11 +20,6 @@ */ package org.apache.bookkeeper.bookie; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LEDGER_CACHE_NUM_EVICTED_LEDGERS; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.PENDING_GET_FILE_INFO; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_FILE_INFO_CACHE_SIZE; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_FILE_INFO_CACHE_SIZE; - import com.google.common.annotations.VisibleForTesting; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; @@ -43,10 +38,9 @@ import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.bookie.FileInfoBackingCache.CachedFileInfo; import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException; +import org.apache.bookkeeper.bookie.stats.IndexPersistenceMgrStats; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.SnapshotMap; import org.slf4j.Logger; @@ -89,9 +83,7 @@ public static final String getLedgerName(long ledgerId) { final SnapshotMap activeLedgers; final LedgerDirsManager ledgerDirsManager; - // Stats - private final Counter evictedLedgersCounter; - private final Counter pendingGetFileInfoCounter; + private final IndexPersistenceMgrStats persistenceMgrStats; public IndexPersistenceMgr(int pageSize, int entriesPerPage, @@ -127,30 +119,11 @@ public IndexPersistenceMgr(int pageSize, fileInfoEvictionListener); // Expose Stats - evictedLedgersCounter = statsLogger.getCounter(LEDGER_CACHE_NUM_EVICTED_LEDGERS); - pendingGetFileInfoCounter = statsLogger.getCounter(PENDING_GET_FILE_INFO); - statsLogger.registerGauge(WRITE_FILE_INFO_CACHE_SIZE, new Gauge() { - @Override - public Number getDefaultValue() { - return 0; - } - - @Override - public Number getSample() { - return writeFileInfoCache.size(); - } - }); - statsLogger.registerGauge(READ_FILE_INFO_CACHE_SIZE, new Gauge() { - @Override - public Number getDefaultValue() { - return 0; - } - - @Override - public Number getSample() { - return readFileInfoCache.size(); - } - }); + persistenceMgrStats = new IndexPersistenceMgrStats( + statsLogger, + () -> writeFileInfoCache.size(), + () -> readFileInfoCache.size() + ); } private static Cache buildCache(int concurrencyLevel, @@ -192,7 +165,7 @@ private void handleLedgerEviction(RemovalNotification noti return; } if (notification.wasEvicted()) { - evictedLedgersCounter.inc(); + persistenceMgrStats.getEvictedLedgersCounter().inc(); } fileInfo.release(); } @@ -207,7 +180,7 @@ private void handleLedgerEviction(RemovalNotification noti CachedFileInfo getFileInfo(final Long ledger, final byte masterKey[]) throws IOException { try { CachedFileInfo fi; - pendingGetFileInfoCounter.inc(); + persistenceMgrStats.getPendingGetFileInfoCounter().inc(); Callable loader = () -> { CachedFileInfo fileInfo = fileInfoBackingCache.loadFileInfo(ledger, masterKey); activeLedgers.put(ledger, true); @@ -243,7 +216,7 @@ CachedFileInfo getFileInfo(final Long ledger, final byte masterKey[]) throws IOE throw new LedgerCache.NoIndexForLedger("Failed to load file info for ledger " + ledger, ee); } } finally { - pendingGetFileInfoCounter.dec(); + persistenceMgrStats.getPendingGetFileInfoCounter().dec(); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/IndexInMemPageMgrStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/IndexInMemPageMgrStats.java new file mode 100644 index 00000000000..7d2b1914a9d --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/IndexInMemPageMgrStats.java @@ -0,0 +1,60 @@ +/* + * 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.stats; + +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.INDEX_INMEM_ILLEGAL_STATE_DELETE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.INDEX_INMEM_ILLEGAL_STATE_RESET; + +import lombok.Getter; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for {@link org.apache.bookkeeper.bookie.IndexInMemPageMgr} stats. + */ +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "Index InMemPage Manager related stats" +) +@Getter +public class IndexInMemPageMgrStats { + + // Stats + @StatsDoc( + name = INDEX_INMEM_ILLEGAL_STATE_RESET, + help = "The number of index pages detected as in illegal state when resetting" + ) + private final Counter illegalStateResetCounter; + @StatsDoc( + name = INDEX_INMEM_ILLEGAL_STATE_DELETE, + help = "The number of index pages detected as in illegal state when deleting" + ) + private final Counter illegalStateDeleteCounter; + + public IndexInMemPageMgrStats(StatsLogger statsLogger) { + illegalStateResetCounter = statsLogger.getCounter(INDEX_INMEM_ILLEGAL_STATE_RESET); + illegalStateDeleteCounter = statsLogger.getCounter(INDEX_INMEM_ILLEGAL_STATE_DELETE); + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/IndexPersistenceMgrStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/IndexPersistenceMgrStats.java new file mode 100644 index 00000000000..3c06b5bc100 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/IndexPersistenceMgrStats.java @@ -0,0 +1,101 @@ +/* + * 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.stats; + +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LEDGER_CACHE_NUM_EVICTED_LEDGERS; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.PENDING_GET_FILE_INFO; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_FILE_INFO_CACHE_SIZE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_FILE_INFO_CACHE_SIZE; + +import java.util.function.Supplier; +import lombok.Getter; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +/** + * A umbrella class for {@link org.apache.bookkeeper.bookie.IndexPersistenceMgr} stats. + */ +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "Index Persistence Manager related stats" +) +@Getter +public class IndexPersistenceMgrStats { + + // Stats + @StatsDoc( + name = LEDGER_CACHE_NUM_EVICTED_LEDGERS, + help = "Number of ledgers evicted from ledger caches" + ) + private final Counter evictedLedgersCounter; + @StatsDoc( + name = PENDING_GET_FILE_INFO, + help = "Number of pending get-file-info requests" + ) + private final Counter pendingGetFileInfoCounter; + @StatsDoc( + name = WRITE_FILE_INFO_CACHE_SIZE, + help = "Current write file info cache size" + ) + private final Gauge writeFileInfoCacheSizeGauge; + @StatsDoc( + name = READ_FILE_INFO_CACHE_SIZE, + help = "Current read file info cache size" + ) + private final Gauge readFileInfoCacheSizeGauge; + + public IndexPersistenceMgrStats(StatsLogger statsLogger, + Supplier writeFileInfoCacheSizeSupplier, + Supplier readFileInfoCacheSizeSupplier) { + evictedLedgersCounter = statsLogger.getCounter(LEDGER_CACHE_NUM_EVICTED_LEDGERS); + pendingGetFileInfoCounter = statsLogger.getCounter(PENDING_GET_FILE_INFO); + writeFileInfoCacheSizeGauge = new Gauge() { + @Override + public Number getDefaultValue() { + return 0; + } + + @Override + public Number getSample() { + return writeFileInfoCacheSizeSupplier.get(); + } + }; + statsLogger.registerGauge(WRITE_FILE_INFO_CACHE_SIZE, writeFileInfoCacheSizeGauge); + readFileInfoCacheSizeGauge = new Gauge() { + @Override + public Number getDefaultValue() { + return 0; + } + + @Override + public Number getSample() { + return readFileInfoCacheSizeSupplier.get(); + } + }; + statsLogger.registerGauge(READ_FILE_INFO_CACHE_SIZE, readFileInfoCacheSizeGauge); + } + + +} From 486375c32368390267d723dbb7b34081448cab8a Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Wed, 19 Dec 2018 03:47:12 -0800 Subject: [PATCH 0181/1642] Bring back statslogger to BookKeeper client in ReplicationWorker process. Descriptions of the changes in this PR: - https://github.com/apache/bookkeeper/commit/2837f6257baf15dc9dd9eb4bcac34596b442be33 had inadvertently removed StatsLogger to BookKeeper client instance in ReplicationWorker process. So restore StatsLogger in BookKeeper client object. - Also introduce new config called - 'limitStatsLogging', which would be used to limit statslogging as and when needed. - currently this config is used to limit the stats from PCBC. Because if AR process is running in each Bookie node, then for each AR there will be n number of PCBCs and totally it comes out to n^2 PCBCs in the cluster. Which is unmanageable from stats collector perspective. So this config value can be set to true in AR config. Reviewers: Sijie Guo This closes #1888 from reddycharan/bringbackmetricsforarprocess --- .../conf/AbstractConfiguration.java | 26 +++++++++++++++++++ .../bookkeeper/proto/BookieClientImpl.java | 8 ++++-- .../bookkeeper/replication/Auditor.java | 9 +++++-- site/_data/config/bk_server.yaml | 3 +++ 4 files changed, 42 insertions(+), 4 deletions(-) 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 7f00d09adb5..f12200fcaff 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 @@ -153,6 +153,9 @@ public abstract class AbstractConfiguration // enforce minimum number of racks per write quorum public static final String ENFORCE_MIN_NUM_RACKS_PER_WRITE_QUORUM = "enforceMinNumRacksPerWriteQuorum"; + // option to limit stats logging + public static final String LIMIT_STATS_LOGGING = "limitStatsLogging"; + protected AbstractConfiguration() { super(); if (READ_SYSTEM_PROPERTIES) { @@ -875,6 +878,29 @@ public T setPreserveMdcForTaskExecution(boolean enabled) { setProperty(PRESERVE_MDC_FOR_TASK_EXECUTION, enabled); return getThis(); } + + /** + * Return the flag indicating whether to limit stats logging. + * + * @return + * the boolean flag indicating whether to limit stats logging + */ + public boolean getLimitStatsLogging() { + return getBoolean(LIMIT_STATS_LOGGING, false); + } + + /** + * Sets flag to limit the stats logging. + * + * @param limitStatsLogging + * flag to limit the stats logging. + * @return configuration. + */ + public T setLimitStatsLogging(boolean limitStatsLogging) { + setProperty(LIMIT_STATS_LOGGING, limitStatsLogging); + return getThis(); + } + /** * Trickery to allow inheritance with fluent style. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index 50dd85fd83e..18f48a2ab89 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -171,8 +171,12 @@ public long getNumPendingRequests(BookieSocketAddress address, long ledgerId) { @Override public PerChannelBookieClient create(BookieSocketAddress address, PerChannelBookieClientPool pcbcPool, SecurityHandlerFactory shFactory) throws SecurityException { - return new PerChannelBookieClient(conf, executor, eventLoopGroup, address, statsLogger, - authProviderFactory, registry, pcbcPool, shFactory); + StatsLogger statsLoggerForPCBC = statsLogger; + if (conf.getLimitStatsLogging()) { + statsLoggerForPCBC = NullStatsLogger.INSTANCE; + } + return new PerChannelBookieClient(conf, executor, eventLoopGroup, address, statsLoggerForPCBC, + authProviderFactory, registry, pcbcPool, shFactory); } public PerChannelBookieClientPool lookupClient(BookieSocketAddress addr) { 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 89883b05285..870cea3bde0 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 @@ -71,6 +71,7 @@ import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; @@ -161,12 +162,16 @@ public class Auditor implements AutoCloseable { ) private final Counter numDelayedBookieAuditsCancelled; - static BookKeeper createBookKeeperClient(ServerConfiguration conf) + static BookKeeper createBookKeeperClient(ServerConfiguration conf) throws InterruptedException, IOException { + return createBookKeeperClient(conf, NullStatsLogger.INSTANCE); + } + + static BookKeeper createBookKeeperClient(ServerConfiguration conf, StatsLogger statsLogger) throws InterruptedException, IOException { ClientConfiguration clientConfiguration = new ClientConfiguration(conf); clientConfiguration.setClientRole(ClientConfiguration.CLIENT_ROLE_SYSTEM); try { - return BookKeeper.forConfig(clientConfiguration).build(); + return BookKeeper.forConfig(clientConfiguration).statsLogger(statsLogger).build(); } catch (BKException e) { throw new IOException("Failed to create bookkeeper client", e); } diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index b60bfbf9930..a7115985df6 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -538,6 +538,9 @@ groups: - Twitter Ostrich : org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider - Twitter Science : org.apache.bookkeeper.stats.twitter.science.TwitterStatsProvider default: org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider + - param: limitStatsLogging + description: option to limit stats logging + default: 'false' - name: Prometheus Metrics Provider Settings params: From d8ff9c3c67903c6996aba4a1bccf3764a8185930 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sat, 22 Dec 2018 07:40:11 +0800 Subject: [PATCH 0182/1642] Introduce a configuration framework for better organizing and documentating configuration settings Descriptions of the changes in this PR: ### Motivation One common task in developing bookkeeper is to make sure all the configuration settings are well documented, and the configuration file we ship in each release is in-sync with the code itself. However maintaining things in-sync is non-trivial. This proposal is exploring a new way to manage configuration settings for better documentation. ### Changes - Introduce `ConfigKey` for defining a configuration setting key in a configuration - Introduce `ConfigKeyGroup` for grouping configuration settings together - Introduce `ConfigDef` for generating the configuration definition for a given configuration - Add a `save` method for saving a configuration definition into a configuration file Master Issue: #1867 Reviewers: Jia Zhai , Enrico Olivelli This closes #1869 from sijie/config_defs --- .../bookkeeper/common/conf/ConfigDef.java | 327 +++++++++++++++ .../common/conf/ConfigException.java | 47 +++ .../bookkeeper/common/conf/ConfigKey.java | 371 ++++++++++++++++++ .../common/conf/ConfigKeyGroup.java | 111 ++++++ .../apache/bookkeeper/common/conf/Type.java | 50 +++ .../bookkeeper/common/conf/Validator.java | 48 +++ .../conf/validators/ClassValidator.java | 81 ++++ .../common/conf/validators/NullValidator.java | 46 +++ .../conf/validators/RangeValidator.java | 96 +++++ .../common/conf/validators/package-info.java | 23 ++ .../bookkeeper/common/conf/ConfigDefTest.java | 300 ++++++++++++++ .../common/conf/ConfigKeyGroupTest.java | 59 +++ .../bookkeeper/common/conf/ConfigKeyTest.java | 336 ++++++++++++++++ .../conf/validators/ClassValidatorTest.java | 61 +++ .../conf/validators/RangeValidatorTest.java | 58 +++ .../src/test/resources/test_conf_2.conf | 130 ++++++ .../bookkeeper/conf/ServerConfiguration.java | 72 +++- .../SortedLedgerStorageCheckpointTest.java | 2 +- .../conf/TestServerConfiguration.java | 41 ++ pom.xml | 3 + 20 files changed, 2251 insertions(+), 11 deletions(-) create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigDef.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigException.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigKey.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigKeyGroup.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/Type.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/Validator.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/ClassValidator.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/NullValidator.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/package-info.java create mode 100644 bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigDefTest.java create mode 100644 bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigKeyGroupTest.java create mode 100644 bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigKeyTest.java create mode 100644 bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/validators/ClassValidatorTest.java create mode 100644 bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/validators/RangeValidatorTest.java create mode 100644 bookkeeper-common/src/test/resources/test_conf_2.conf diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigDef.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigDef.java new file mode 100644 index 00000000000..6e37ebcd061 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigDef.java @@ -0,0 +1,327 @@ +/* + * 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.common.conf; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.collect.Sets; +import java.io.IOException; +import java.io.OutputStream; +import java.io.PrintStream; +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.configuration.Configuration; +import org.apache.commons.lang.StringUtils; + +/** + * A definition of a configuration instance. + */ +@Slf4j +@Getter +public class ConfigDef { + + /** + * Builder to build a configuration definition. + */ + public static class Builder { + + private final Set groups = new TreeSet<>(ConfigKeyGroup.ORDERING); + private final Map> settings = new HashMap<>(); + + private Builder() {} + + /** + * Add the config key group to the builder. + * + * @param group config key group + * @return builder to build this configuration def + */ + public Builder withConfigKeyGroup(ConfigKeyGroup group) { + groups.add(group); + return this; + } + + /** + * Add the config key to the builder. + * + * @param key the key to add to the builder. + * @return builder to build this configuration def + */ + public Builder withConfigKey(ConfigKey key) { + ConfigKeyGroup group = key.group(); + Set keys; + String groupName; + if (null == group) { + groupName = ""; + } else { + groupName = group.name(); + groups.add(group); + } + keys = settings.computeIfAbsent(groupName, name -> new TreeSet<>(ConfigKey.ORDERING)); + keys.add(key); + return this; + } + + public ConfigDef build() { + checkArgument( + Sets.difference( + groups.stream().map(group -> group.name()).collect(Collectors.toSet()), + settings.keySet() + ).isEmpty(), + "Configuration Key Groups doesn't match with keys"); + return new ConfigDef(groups, settings); + } + + } + + /** + * Create a builder to build a config def. + * + * @return builder to build a config def. + */ + public static Builder builder() { + return new Builder(); + } + + private final Set groups; + private final Map> settings; + private final Map keys; + + private ConfigDef(Set groups, + Map> settings) { + this.groups = groups; + this.settings = settings; + this.keys = settings.values() + .stream() + .flatMap(keys -> keys.stream()) + .collect(Collectors.toSet()) + .stream() + .collect(Collectors.toMap( + key -> key.name(), + key -> key + )); + } + + /** + * Validate if the provided conf is a valid configuration of this configuration definition. + * + * @param conf the configuration to validate + */ + public void validate(Configuration conf) throws ConfigException { + for (ConfigKey key : keys.values()) { + key.validate(conf); + } + } + + /** + * Build the config definitation of a config class. + * + * @param configClass config class + * @return config definition. + */ + @SuppressWarnings("unchecked") + public static ConfigDef of(Class configClass) { + ConfigDef.Builder builder = ConfigDef.builder(); + + Field[] fields = configClass.getDeclaredFields(); + for (Field field : fields) { + if (Modifier.isStatic(field.getModifiers()) && field.getType().equals(ConfigKey.class)) { + field.setAccessible(true); + try { + builder.withConfigKey((ConfigKey) field.get(null)); + } catch (IllegalAccessException e) { + log.error("Illegal to access {}#{}", configClass.getSimpleName(), field.getName(), e); + } + } + } + + return builder.build(); + } + + // + // Methods to save the configuration to an {@link OutputStream} + // + + private static final int MAX_COLUMN_SIZE = 80; + private static final String COMMENT_PREFIX = "# "; + + public void save(Path path) throws IOException { + try (OutputStream stream = Files.newOutputStream( + path, StandardOpenOption.WRITE, StandardOpenOption.TRUNCATE_EXISTING)) { + save(stream); + } + } + + public void save(OutputStream os) throws IOException { + try (PrintStream ps = new PrintStream(os, false, UTF_8.name())) { + save(ps); + ps.flush(); + } + } + + private void writeNSharps(PrintStream stream, int num) { + IntStream.range(0, num).forEach(ignored -> stream.print("#")); + } + + private void writeConfigKeyGroup(PrintStream stream, ConfigKeyGroup group) { + int maxLength = Math.min( + group.description().length() + COMMENT_PREFIX.length(), + MAX_COLUMN_SIZE + ); + // "###########" + writeNSharps(stream, maxLength); + stream.println(); + // "# Settings of `` + writeSentence(stream, COMMENT_PREFIX, "Settings of `" + group.name() + "`"); + stream.println("#"); + // "# " + writeSentence(stream, COMMENT_PREFIX, group.description()); + // "###########" + writeNSharps(stream, maxLength); + stream.println(); + } + + private void writeConfigKey(PrintStream stream, + ConfigKey key) { + // "# " + // "#" + if (StringUtils.isNotBlank(key.description())) { + writeSentence(stream, COMMENT_PREFIX, key.description()); + stream.println("#"); + } + // "# " + // "#" + if (StringUtils.isNotBlank(key.documentation())) { + writeSentence(stream, COMMENT_PREFIX, key.documentation()); + stream.println("#"); + } + // "# type: , required" + writeSentence( + stream, + COMMENT_PREFIX, + "TYPE: " + key.type() + ", " + (key.required() ? "required" : "optional")); + if (null != key.validator() && StringUtils.isNotBlank(key.validator().documentation())) { + writeSentence( + stream, COMMENT_PREFIX, + "@constraints : " + key.validator().documentation() + ); + } + if (!key.optionValues().isEmpty()) { + writeSentence( + stream, COMMENT_PREFIX, "@options :" + ); + key.optionValues().forEach(value -> { + writeSentence( + stream, COMMENT_PREFIX, " " + value + ); + }); + } + // "#" + // "# @Since" + if (StringUtils.isNotBlank(key.since())) { + stream.println("#"); + writeSentence(stream, COMMENT_PREFIX, + "@since " + key.since() + ""); + } + // "#" + // "# @Deprecated" + if (key.deprecated()) { + stream.println("#"); + writeSentence(stream, COMMENT_PREFIX, getDeprecationDescription(key)); + } + // = + stream.print(key.name()); + stream.print("="); + if (null != key.defaultValue()) { + stream.print(key.defaultValue()); + } + stream.println(); + } + + private String getDeprecationDescription(ConfigKey key) { + StringBuilder sb = new StringBuilder(); + sb.append("@deprecated"); + if (StringUtils.isNotBlank(key.deprecatedSince())) { + sb.append(" since `") + .append(key.deprecatedSince()) + .append("`"); + } + if (StringUtils.isNotBlank(key.deprecatedByConfigKey())) { + sb.append(" in favor of using `") + .append(key.deprecatedByConfigKey()) + .append("`"); + } + return sb.toString(); + } + + private void writeSentence(PrintStream stream, + String prefix, + String sentence) { + int max = MAX_COLUMN_SIZE; + String[] words = sentence.split(" "); + int i = 0; + stream.print(prefix); + int current = prefix.length(); + while (i < words.length) { + String word = words[i]; + if (word.length() > max || current + word.length() <= max) { + if (i != 0) { + stream.print(" "); + } + stream.print(word); + current += (word.length() + 1); + } else { + stream.println(); + stream.print(prefix); + stream.print(word); + current = prefix.length() + word.length(); + } + ++i; + } + stream.println(); + } + + private void save(PrintStream stream) { + for (ConfigKeyGroup group : groups) { + writeConfigKeyGroup(stream, group); + stream.println(); + Set groupKeys = settings.getOrDefault(group.name(), Collections.emptySet()); + groupKeys.forEach(key -> { + writeConfigKey(stream, key); + stream.println(); + }); + } + } + + +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigException.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigException.java new file mode 100644 index 00000000000..a0534a6b2de --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigException.java @@ -0,0 +1,47 @@ +/* + * 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.common.conf; + +/** + * Exception thrown for configuration errors. + */ +public class ConfigException extends Exception { + + private static final long serialVersionUID = -7842276571881795108L; + + /** + * Construct a config exception with provided error. + * + * @param error error message + */ + public ConfigException(String error) { + super(error); + } + + /** + * Construct a config exception with provided error and reason. + * + * @param error error message + * @param cause error cause + */ + public ConfigException(String error, Throwable cause) { + super(error, cause); + } +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigKey.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigKey.java new file mode 100644 index 00000000000..b2bb47a92b9 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigKey.java @@ -0,0 +1,371 @@ +/* + * 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.common.conf; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import lombok.Builder; +import lombok.Builder.Default; +import lombok.Data; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; +import org.apache.bookkeeper.common.conf.validators.NullValidator; +import org.apache.bookkeeper.common.util.ReflectionUtils; +import org.apache.commons.configuration.Configuration; +import org.apache.commons.configuration.ConfigurationException; + +/** + * A configuration key in a configuration. + */ +@Data +@Builder(builderMethodName = "internalBuilder") +@Accessors(fluent = true) +@Public +@Slf4j +public class ConfigKey { + + public static final Comparator ORDERING = (o1, o2) -> { + int ret = Integer.compare(o1.orderInGroup, o2.orderInGroup); + if (ret == 0) { + return o1.name().compareTo(o2.name()); + } else { + return ret; + } + }; + + /** + * Build a config key of name. + * + * @param name config key name + * @return config key builder + */ + public static ConfigKeyBuilder builder(String name) { + return internalBuilder().name(name); + } + + /** + * Flag indicates whether the setting is required. + */ + @Default + private boolean required = false; + + /** + * Name of the configuration setting. + */ + private String name; + + /** + * Type of the configuration setting. + */ + @Default + private Type type = Type.STRING; + + /** + * Description of the configuration setting. + */ + @Default + private String description = ""; + + /** + * Documentation of the configuration setting. + */ + @Default + private String documentation = ""; + + /** + * Default value as a string representation. + */ + @Default + private Object defaultValue = null; + + private String defaultValueAsString() { + if (null == defaultValue) { + return null; + } else if (defaultValue instanceof String) { + return (String) defaultValue; + } else if (defaultValue instanceof Class) { + return ((Class) defaultValue).getName(); + } else { + return defaultValue.toString(); + } + } + + /** + * The list of options for this setting. + */ + @Default + private List optionValues = Collections.emptyList(); + + /** + * The validator used for validating configuration value. + */ + @Default + private Validator validator = NullValidator.of(); + + /** + * The key-group to group settings together. + */ + @Default + private ConfigKeyGroup group = ConfigKeyGroup.DEFAULT; + + /** + * The order of the setting in the key-group. + */ + @Default + private int orderInGroup = Integer.MIN_VALUE; + + /** + * The list of settings dependents on this setting. + */ + @Default + private List dependents = Collections.emptyList(); + + /** + * Whether this setting is deprecated or not. + */ + @Default + private boolean deprecated = false; + + /** + * The config key that deprecates this key. + */ + @Default + private String deprecatedByConfigKey = ""; + + /** + * The version when this settings was deprecated. + */ + @Default + private String deprecatedSince = ""; + + /** + * The version when this setting was introduced. + */ + @Default + private String since = ""; + + /** + * {@inheritDoc} + */ + @Override + public boolean equals(Object o) { + if (!(o instanceof ConfigKey)) { + return false; + } + ConfigKey other = (ConfigKey) o; + return Objects.equals(name, other.name); + } + + /** + * {@inheritDoc} + */ + @Override + public int hashCode() { + return name.hashCode(); + } + + /** + * Validate the setting is valid in the provided config conf. + * + * @param conf configuration to test + */ + public void validate(Configuration conf) throws ConfigException { + if (conf.containsKey(name()) && validator() != null) { + Object value = get(conf); + if (!validator().validate(name(), value)) { + throw new ConfigException("Invalid setting of '" + name() + + "' found the configuration: value = '" + value + "', requirement = '" + validator + "'"); + } + } else if (required()) { // missing config on a required field + throw new ConfigException( + "Setting '" + name() + "' is required but missing in the configuration"); + } + } + + /** + * Update the setting name in the configuration conf with the provided value. + * + * @param conf configuration to set + * @param value value of the setting + */ + public void set(Configuration conf, Object value) { + if (!type().validator().validate(name(), value)) { + throw new IllegalArgumentException( + "Invalid value '" + value + "' to set on setting '" + name() + "': expected type = " + type); + } + + if (null != validator() && !validator().validate(name(), value)) { + throw new IllegalArgumentException( + "Invalid value '" + value + "' to set on setting '" + name() + "': required '" + validator() + "'"); + } + + if (value instanceof Class) { + conf.setProperty(name(), ((Class) value).getName()); + } else { + conf.setProperty(name(), value); + } + } + + /** + * Retrieve the setting from the configuration conf as a {@link Long} value. + * + * @param conf configuration to retrieve the setting + * @return the value as a long number + */ + public long getLong(Configuration conf) { + checkArgument(type() == Type.LONG, "'" + name() + "' is NOT a LONG numeric setting"); + return conf.getLong(name(), (Long) defaultValue()); + } + + /** + * Retrieve the setting from the configuration conf as a {@link Integer} value. + * + * @param conf configuration to retrieve the setting + * @return the value as an integer number + */ + public int getInt(Configuration conf) { + checkArgument(type() == Type.INT, "'" + name() + "' is NOT a INT numeric setting"); + return conf.getInt(name(), (Integer) defaultValue()); + } + + /** + * Retrieve the setting from the configuration conf as a {@link Short} value. + * + * @param conf configuration to retrieve the setting + * @return the value as a short number + */ + public short getShort(Configuration conf) { + checkArgument(type() == Type.SHORT, "'" + name() + "' is NOT a SHORT numeric setting"); + return conf.getShort(name(), (Short) defaultValue()); + } + + /** + * Retrieve the setting from the configuration conf as a {@link Boolean} value. + * + * @param conf configuration to retrieve the setting + * @return the value as a boolean flag + */ + public boolean getBoolean(Configuration conf) { + checkArgument(type() == Type.BOOLEAN, "'" + name() + "' is NOT a BOOL numeric setting"); + return conf.getBoolean(name(), (Boolean) defaultValue()); + } + + /** + * Retrieve the setting from the configuration conf as a {@link Double} value. + * + * @param conf configuration to retrieve the setting + * @return the value as a double number + */ + public double getDouble(Configuration conf) { + checkArgument(type() == Type.DOUBLE, "'" + name() + "' is NOT a DOUBLE numeric setting"); + return conf.getDouble(name(), (Double) defaultValue()); + } + + /** + * Retrieve the setting from the configuration conf as a {@link String} value. + * + * @param conf configuration to retrieve the setting + * @return the value as a string. + */ + public String getString(Configuration conf) { + return conf.getString(name(), defaultValueAsString()); + } + + /** + * Retrieve the setting from the configuration conf as a {@link Class} value. + * + * @param conf configuration to retrieve the setting + * @return the value as a class + */ + @SuppressWarnings("unchecked") + public Class getClass(Configuration conf, Class interfaceCls) { + checkArgument(type() == Type.CLASS, "'" + name() + "' is NOT a CLASS setting"); + try { + Class defaultClass = (Class) defaultValue(); + return ReflectionUtils.getClass(conf, name(), defaultClass, interfaceCls, getClass().getClassLoader()); + } catch (ConfigurationException e) { + throw new IllegalArgumentException("Invalid class is set to setting '" + name() + "': ", e); + } + } + + /** + * Retrieve the setting from the configuration conf as a {@link Class} value. + * + * @param conf configuration to retrieve the setting + * @return the value as a class + */ + @SuppressWarnings("unchecked") + public Class getClass(Configuration conf) { + checkArgument(type() == Type.CLASS, "'" + name() + "' is NOT a CLASS setting"); + try { + Class defaultClass = (Class) defaultValue(); + return ReflectionUtils.getClass(conf, name(), defaultClass, getClass().getClassLoader()); + } catch (ConfigurationException e) { + throw new IllegalArgumentException("Invalid class is set to setting '" + name() + "': ", e); + } + } + + /** + * Retrieve the setting from the configuration conf as a {@link Class} value. + * + * @param conf configuration to retrieve the setting + * @return the value as list of values + */ + @SuppressWarnings("unchecked") + public List getList(Configuration conf) { + checkArgument(type() == Type.LIST, "'" + name() + "' is NOT a LIST setting"); + List list = (List) defaultValue(); + if (null == list) { + list = Collections.emptyList(); + } + return conf.getList(name(), list); + } + + /** + * Retrieve the setting value from the provided conf. + * + * @return the setting value + */ + public Object get(Configuration conf) { + switch (type()) { + case LONG: + return getLong(conf); + case INT: + return getInt(conf); + case SHORT: + return getShort(conf); + case DOUBLE: + return getDouble(conf); + case BOOLEAN: + return getBoolean(conf); + case LIST: + return getList(conf); + case CLASS: + return getClass(conf); + default: + return getString(conf); + } + } +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigKeyGroup.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigKeyGroup.java new file mode 100644 index 00000000000..833e907b38e --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/ConfigKeyGroup.java @@ -0,0 +1,111 @@ +/* + * 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.common.conf; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import lombok.Builder; +import lombok.Builder.Default; +import lombok.Data; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; + +/** + * Define a group of configuration settings. + */ +@Data +@Accessors(fluent = true) +@Builder(builderMethodName = "internalBuilder") +@Public +public class ConfigKeyGroup { + + /** + * Ordering the key groups in a configuration. + */ + public static final Comparator ORDERING = (o1, o2) -> { + int ret = Integer.compare(o1.order, o2.order); + if (0 == ret) { + return o1.name().compareTo(o2.name()); + } else { + return ret; + } + }; + + /** + * Create a config key group of name. + * + * @param name key group name + * @return key group builder + */ + public static ConfigKeyGroupBuilder builder(String name) { + return internalBuilder().name(name); + } + + /** + * The default key group. + */ + public static final ConfigKeyGroup DEFAULT = builder("").build(); + + /** + * Name of the key group. + */ + private String name; + + /** + * Description of the key group. + */ + @Default + private String description = ""; + + /** + * The list of sub key-groups of this key group. + */ + @Default + private List children = Collections.emptyList(); + + /** + * The order of the key-group in a configuration. + */ + @Default + private int order = Integer.MIN_VALUE; + + /** + * {@inheritDoc} + */ + @Override + public boolean equals(Object o) { + if (!(o instanceof ConfigKeyGroup)) { + return false; + } + ConfigKeyGroup other = (ConfigKeyGroup) o; + return Objects.equals(name, other.name); + } + + /** + * {@inheritDoc} + */ + @Override + public int hashCode() { + return name.hashCode(); + } + +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/Type.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/Type.java new file mode 100644 index 00000000000..c48e94c8b97 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/Type.java @@ -0,0 +1,50 @@ +/* + * 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.common.conf; + +import java.util.List; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; + +/** + * Config key types. + */ +@Public +public enum Type { + + BOOLEAN((name, value) -> value instanceof Boolean), + STRING((name, value) -> value instanceof String), + INT((name, value) -> value instanceof Integer), + SHORT((name, value) -> value instanceof Short), + LONG((name, value) -> value instanceof Long), + DOUBLE((name, value) -> value instanceof Double), + LIST((name, value) -> value instanceof List), + CLASS((name, value) -> value instanceof Class || value instanceof String); + + private Validator validator; + + Type(Validator validator) { + this.validator = validator; + } + + public Validator validator() { + return validator; + } + +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/Validator.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/Validator.java new file mode 100644 index 00000000000..249ad310416 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/Validator.java @@ -0,0 +1,48 @@ +/* + * 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.common.conf; + +import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; + +/** + * Validator that validates configuration settings. + */ +@Public +public interface Validator { + + /** + * Validates the configuration value. + * + * @param name name of the configuration setting + * @param value value of the configuration setting + * @return true if it is a valid value, otherwise false. + */ + boolean validate(String name, Object value); + + /** + * Return the documentation for a given validator. + * + * @return the documentation for a given validator + */ + default String documentation() { + return ""; + } + +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/ClassValidator.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/ClassValidator.java new file mode 100644 index 00000000000..dcd5f41cbdc --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/ClassValidator.java @@ -0,0 +1,81 @@ +/* + * 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.common.conf.validators; + +import lombok.Data; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.conf.Validator; +import org.apache.bookkeeper.common.util.ReflectionUtils; + +/** + * Validator that validates a configuration setting is returning a given type of class. + */ +@Slf4j +@Data +public class ClassValidator implements Validator { + + /** + * Create a validator to validate if a setting is returning a class that extends from + * interfaceClass. + * + * @param interfaceClass interface class + * @return the validator that expects a setting return a class that extends from interfaceClass + */ + public static ClassValidator of(Class interfaceClass) { + return new ClassValidator<>(interfaceClass); + } + + private final Class interfaceClass; + + @Override + public boolean validate(String name, Object value) { + if (value instanceof String) { + try { + ReflectionUtils.forName((String) value, interfaceClass); + return true; + } catch (RuntimeException re) { + log.warn("Setting value of '{}' is not '{}' : {}", + name, interfaceClass.getName(), value, re); + return false; + } + } else if (value instanceof Class) { + Class cls = (Class) value; + if (!interfaceClass.isAssignableFrom(cls)) { + log.warn("Setting value of '{}' is not '{}' : {}", + name, interfaceClass.getName(), cls.getName()); + return false; + } else { + return true; + } + } else { + return false; + } + } + + @Override + public String toString() { + return "Class extends " + interfaceClass.getName(); + } + + @Override + public String documentation() { + return "class extends `" + interfaceClass.getName() + "`"; + } +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/NullValidator.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/NullValidator.java new file mode 100644 index 00000000000..1d384dfb73a --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/NullValidator.java @@ -0,0 +1,46 @@ +/* + * 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.common.conf.validators; + +import org.apache.bookkeeper.common.conf.Validator; + +/** + * A validator that does nothing. + */ +public class NullValidator implements Validator { + + /** + * Return the instance of NullValidator. + * + * @return the instance of NullValidator. + */ + public static NullValidator of() { + return INSTANCE; + } + + private static final NullValidator INSTANCE = new NullValidator(); + + private NullValidator() {} + + @Override + public boolean validate(String name, Object value) { + return true; + } +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java new file mode 100644 index 00000000000..2dbadf4e28c --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java @@ -0,0 +1,96 @@ +/* + * 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.common.conf.validators; + +import lombok.Data; +import org.apache.bookkeeper.common.conf.Validator; + +/** + * Validator that validates a configuration value is in a numeric range. + */ +@Data +public class RangeValidator implements Validator { + + /** + * A numeric range that checks the lower bound. + * + * @param min the minimum acceptable value + * @return a numeric range that checks the lower bound + */ + public static RangeValidator atLeast(Number min) { + return new RangeValidator(min, null); + } + + /** + * A numeric range that checks the upper bound. + * + * @param max the maximum acceptable value + * @return a numeric range that checks the upper bound + */ + public static RangeValidator atMost(Number max) { + return new RangeValidator(null, max); + } + + /** + * A numeric range that checks both lower and upper bounds. + * + * @param min the minimum acceptable value + * @param max the maximum acceptable value + * @return a numeric range that checks both lower and upper bounds + */ + public static RangeValidator between(Number min, Number max) { + return new RangeValidator(min, max); + } + + private final Number min; + private final Number max; + + @Override + public boolean validate(String name, Object value) { + if (value instanceof Number) { + Number n = (Number) value; + if (min != null && n.doubleValue() < min.doubleValue()) { + return false; + } else if (max != null && n.doubleValue() > max.doubleValue()) { + return false; + } else { + return true; + } + } else { + return false; + } + } + + @Override + public String toString() { + if (null == min) { + return "[... , " + max + "]"; + } else if (null == max) { + return "[" + min + ", ...]"; + } else { + return "[" + min + ", " + max + "]"; + } + } + + @Override + public String documentation() { + return toString(); + } +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/package-info.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/package-info.java new file mode 100644 index 00000000000..e4c141a0f93 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/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 collection of validators that validate configuration settings. + */ +package org.apache.bookkeeper.common.conf.validators; \ No newline at end of file diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigDefTest.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigDefTest.java new file mode 100644 index 00000000000..7ba3e71ccae --- /dev/null +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigDefTest.java @@ -0,0 +1,300 @@ +/* + * 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.common.conf; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import com.google.common.collect.Lists; +import com.google.common.io.ByteStreams; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; +import java.util.Set; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.conf.validators.ClassValidator; +import org.apache.bookkeeper.common.conf.validators.RangeValidator; +import org.junit.Test; + +/** + * Unit test {@link ConfigDef}. + */ +@Slf4j +public class ConfigDefTest { + + private static class TestConfig { + + private static final ConfigKeyGroup group1 = ConfigKeyGroup.builder("group1") + .description("Group 1 Settings") + .order(1) + .build(); + + private static final ConfigKey key11 = ConfigKey.builder("key11") + .type(Type.LONG) + .group(group1) + .validator(RangeValidator.atLeast(1000)) + .build(); + + private static final ConfigKeyGroup group2 = ConfigKeyGroup.builder("group2") + .description("Group 2 Settings") + .order(2) + .build(); + + private static final ConfigKey key21 = ConfigKey.builder("key21") + .type(Type.LONG) + .group(group2) + .validator(RangeValidator.atMost(1000)) + .orderInGroup(2) + .build(); + + private static final ConfigKey key22 = ConfigKey.builder("key22") + .type(Type.STRING) + .group(group2) + .validator(ClassValidator.of(Runnable.class)) + .orderInGroup(1) + .build(); + + } + + private static class TestConfig2 { + + private static final ConfigKeyGroup emptyGroup = ConfigKeyGroup.builder("empty_group") + .description("Empty Group Settings") + .order(1) + .build(); + + private static final ConfigKeyGroup group1 = ConfigKeyGroup.builder("group1") + .description("This is a very long description : Lorem ipsum dolor sit amet," + + " consectetur adipiscing elit. Maecenas bibendum ac felis id commodo." + + " Etiam mauris purus, fringilla id tempus in, mollis vel orci. Duis" + + " ultricies at erat eget iaculis.") + .order(2) + .build(); + + private static final ConfigKey intKey = ConfigKey.builder("int_key") + .type(Type.INT) + .description("it is an int key") + .group(group1) + .validator(RangeValidator.atLeast(1000)) + .build(); + + private static final ConfigKey longKey = ConfigKey.builder("long_key") + .type(Type.LONG) + .description("it is a long key") + .group(group1) + .validator(RangeValidator.atMost(1000)) + .build(); + + private static final ConfigKey shortKey = ConfigKey.builder("short_key") + .type(Type.SHORT) + .description("it is a short key") + .group(group1) + .validator(RangeValidator.between(500, 1000)) + .build(); + + private static final ConfigKey doubleKey = ConfigKey.builder("double_key") + .type(Type.DOUBLE) + .description("it is a double key") + .group(group1) + .validator(RangeValidator.between(1234.0f, 5678.0f)) + .build(); + + private static final ConfigKey boolKey = ConfigKey.builder("bool_key") + .type(Type.BOOLEAN) + .description("it is a bool key") + .group(group1) + .build(); + + private static final ConfigKey classKey = ConfigKey.builder("class_key") + .type(Type.CLASS) + .description("it is a class key") + .validator(ClassValidator.of(Runnable.class)) + .group(group1) + .build(); + + private static final ConfigKey listKey = ConfigKey.builder("list_key") + .type(Type.LIST) + .description("it is a list key") + .group(group1) + .build(); + + private static final ConfigKey stringKey = ConfigKey.builder("string_key") + .type(Type.STRING) + .description("it is a string key") + .group(group1) + .build(); + + private static final ConfigKeyGroup group2 = ConfigKeyGroup.builder("group2") + .description("This group has short description") + .order(3) + .build(); + + private static final ConfigKey keyWithSince = ConfigKey.builder("key_with_since") + .type(Type.STRING) + .description("it is a string key with since") + .since("4.7.0") + .group(group2) + .orderInGroup(10) + .build(); + + private static final ConfigKey keyWithDocumentation = ConfigKey.builder("key_with_short_documentation") + .type(Type.STRING) + .description("it is a string key with documentation") + .documentation("it has a short documentation") + .group(group2) + .orderInGroup(9) + .build(); + + private static final ConfigKey keyWithLongDocumentation = + ConfigKey.builder("key_long_short_documentation") + .type(Type.STRING) + .description("it is a string key with documentation") + .documentation("it has a long documentation : Lorem ipsum dolor sit amet," + + " consectetur adipiscing elit. Maecenas bibendum ac felis id commodo." + + " Etiam mauris purus, fringilla id tempus in, mollis vel orci. Duis" + + " ultricies at erat eget iaculis.") + .group(group2) + .orderInGroup(8) + .build(); + + private static final ConfigKey keyWithDefaultValue = ConfigKey.builder("key_with_default_value") + .type(Type.STRING) + .description("it is a string key with default value") + .defaultValue("this-is-a-test-value") + .group(group2) + .orderInGroup(7) + .build(); + + private static final ConfigKey keyWithOptionalValues = ConfigKey.builder("key_with_optional_values") + .type(Type.STRING) + .description("it is a string key with optional values") + .defaultValue("this-is-a-default-value") + .optionValues(Lists.newArrayList( + "item1", "item2", "item3", "item3" + )) + .group(group2) + .orderInGroup(6) + .build(); + + private static final ConfigKey deprecatedKey = ConfigKey.builder("deprecated_key") + .type(Type.STRING) + .deprecated(true) + .description("it is a deprecated key") + .group(group2) + .orderInGroup(5) + .build(); + + private static final ConfigKey deprecatedKeyWithSince = ConfigKey.builder("deprecated_key_with_since") + .type(Type.STRING) + .deprecated(true) + .deprecatedSince("4.3.0") + .description("it is a deprecated key with since") + .group(group2) + .orderInGroup(4) + .build(); + + private static final ConfigKey deprecatedKeyWithReplacedKey = + ConfigKey.builder("deprecated_key_with_replaced_key") + .type(Type.STRING) + .deprecated(true) + .deprecatedByConfigKey("key_with_optional_values") + .description("it is a deprecated key with replaced key") + .group(group2) + .orderInGroup(3) + .build(); + + private static final ConfigKey deprecatedKeyWithSinceAndReplacedKey = + ConfigKey.builder("deprecated_key_with_since_and_replaced_key") + .type(Type.STRING) + .deprecated(true) + .deprecatedSince("4.3.0") + .deprecatedByConfigKey("key_with_optional_values") + .description("it is a deprecated key with since and replaced key") + .group(group2) + .orderInGroup(2) + .build(); + + private static final ConfigKey requiredKey = ConfigKey.builder("required_key") + .type(Type.STRING) + .required(true) + .description("it is a required key") + .group(group2) + .orderInGroup(1) + .build(); + + } + + @Test + public void testBuildConfigDef() { + ConfigDef configDef = ConfigDef.of(TestConfig.class); + assertEquals(2, configDef.getGroups().size()); + + Iterator grpIter = configDef.getGroups().iterator(); + + // iterate over group 1 + assertTrue(grpIter.hasNext()); + ConfigKeyGroup group1 = grpIter.next(); + assertSame(TestConfig.group1, group1); + Set keys = configDef.getSettings().get(group1.name()); + assertNotNull(keys); + assertEquals(1, keys.size()); + assertEquals(TestConfig.key11, keys.iterator().next()); + + // iterate over group 2 + assertTrue(grpIter.hasNext()); + ConfigKeyGroup group2 = grpIter.next(); + assertSame(TestConfig.group2, group2); + keys = configDef.getSettings().get(group2.name()); + assertNotNull(keys); + assertEquals(2, keys.size()); + Iterator keyIter = keys.iterator(); + assertEquals(TestConfig.key22, keyIter.next()); + assertEquals(TestConfig.key21, keyIter.next()); + assertFalse(keyIter.hasNext()); + + // no more group + assertFalse(grpIter.hasNext()); + } + + @Test + public void testSaveConfigDef() throws IOException { + byte[] confData; + try (InputStream is = this.getClass().getClassLoader().getResourceAsStream("test_conf_2.conf")) { + confData = new byte[is.available()]; + ByteStreams.readFully(is, confData); + } + + ConfigDef configDef = ConfigDef.of(TestConfig2.class); + String readConf; + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + configDef.save(baos); + readConf = baos.toString(); + log.info("\n{}", readConf); + } + + assertEquals(new String(confData, UTF_8), readConf); + } + +} diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigKeyGroupTest.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigKeyGroupTest.java new file mode 100644 index 00000000000..a8abefa7b3d --- /dev/null +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigKeyGroupTest.java @@ -0,0 +1,59 @@ +/* + * 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.common.conf; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; + +/** + * Unit test {@link ConfigKeyGroup}. + */ +public class ConfigKeyGroupTest { + + @Test + public void testEquals() { + ConfigKeyGroup grp1 = ConfigKeyGroup.builder("group1") + .description("test group 1") + .build(); + ConfigKeyGroup anotherGrp1 = ConfigKeyGroup.builder("group1") + .description("test another group 1") + .build(); + + assertEquals(grp1, anotherGrp1); + } + + @Test + public void testOrdering() { + ConfigKeyGroup grp10 = ConfigKeyGroup.builder("group1") + .order(0) + .build(); + ConfigKeyGroup grp20 = ConfigKeyGroup.builder("group2") + .order(0) + .build(); + ConfigKeyGroup grp21 = ConfigKeyGroup.builder("group2") + .order(1) + .build(); + + assertTrue(ConfigKeyGroup.ORDERING.compare(grp10, grp20) < 0); + assertTrue(ConfigKeyGroup.ORDERING.compare(grp20, grp21) < 0); + } + +} diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigKeyTest.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigKeyTest.java new file mode 100644 index 00000000000..858a615db54 --- /dev/null +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/ConfigKeyTest.java @@ -0,0 +1,336 @@ +/* + * 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.common.conf; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.Lists; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Function; +import org.apache.commons.configuration.CompositeConfiguration; +import org.apache.commons.configuration.Configuration; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; + +/** + * Unit test {@link ConfigKey}. + */ +public class ConfigKeyTest { + + /** + * Test Function A. + */ + private static class TestFunctionA implements Function { + + @Override + public String apply(String s) { + return s + "!"; + } + } + + /** + * Test Function B. + */ + private static class TestFunctionB implements Function { + + @Override + public String apply(String s) { + return s + "!"; + } + } + + /** + * Test Function C. + */ + private static class TestFunctionC implements Function { + + @Override + public String apply(String s) { + return s + "!"; + } + } + + @Rule + public final TestName runtime = new TestName(); + + @Test + public void testValidateRequiredField() { + String keyName = runtime.getMethodName(); + Configuration conf = new ConcurrentConfiguration(); + ConfigKey key = ConfigKey.builder(keyName) + .required(true) + .build(); + + try { + key.validate(conf); + fail("Required key should exist in the configuration"); + } catch (ConfigException ce) { + // expected + } + } + + @Test + public void testValidateFieldSuccess() throws ConfigException { + String keyName = runtime.getMethodName(); + Validator validator = mock(Validator.class); + when(validator.validate(anyString(), any())).thenReturn(true); + Configuration conf = new ConcurrentConfiguration(); + conf.setProperty(keyName, "test-value"); + ConfigKey key = ConfigKey.builder(keyName) + .validator(validator) + .build(); + + key.validate(conf); + verify(validator, times(1)).validate(eq(keyName), eq("test-value")); + } + + @Test + public void testValidateFieldFailure() { + String keyName = runtime.getMethodName(); + Validator validator = mock(Validator.class); + when(validator.validate(anyString(), any())).thenReturn(false); + Configuration conf = new ConcurrentConfiguration(); + conf.setProperty(keyName, "test-value"); + ConfigKey key = ConfigKey.builder(keyName) + .validator(validator) + .build(); + + try { + key.validate(conf); + fail("Should fail validation if validator#validate returns false"); + } catch (ConfigException ce) { + // expected + } + verify(validator, times(1)).validate(eq(keyName), eq("test-value")); + } + + @Test + public void testGetLong() { + String keyName = runtime.getMethodName(); + long defaultValue = System.currentTimeMillis(); + ConfigKey key = ConfigKey.builder(keyName) + .required(true) + .type(Type.LONG) + .defaultValue(defaultValue) + .build(); + + Configuration conf = new ConcurrentConfiguration(); + + // get default value + assertEquals(defaultValue, key.getLong(conf)); + assertEquals(defaultValue, key.get(conf)); + + // set value + long newValue = System.currentTimeMillis() * 2; + key.set(conf, newValue); + assertEquals(newValue, key.getLong(conf)); + assertEquals(newValue, key.get(conf)); + } + + @Test + public void testGetInt() { + String keyName = runtime.getMethodName(); + int defaultValue = ThreadLocalRandom.current().nextInt(10000); + ConfigKey key = ConfigKey.builder(keyName) + .required(true) + .type(Type.INT) + .defaultValue(defaultValue) + .build(); + + Configuration conf = new ConcurrentConfiguration(); + + // get default value + assertEquals(defaultValue, key.getInt(conf)); + assertEquals(defaultValue, key.get(conf)); + + // set value + int newValue = defaultValue * 2; + key.set(conf, newValue); + assertEquals(newValue, key.getInt(conf)); + assertEquals(newValue, key.get(conf)); + } + + @Test + public void testGetShort() { + String keyName = runtime.getMethodName(); + short defaultValue = (short) ThreadLocalRandom.current().nextInt(10000); + ConfigKey key = ConfigKey.builder(keyName) + .required(true) + .type(Type.SHORT) + .defaultValue(defaultValue) + .build(); + + Configuration conf = new ConcurrentConfiguration(); + + // get default value + assertEquals(defaultValue, key.getShort(conf)); + assertEquals(defaultValue, key.get(conf)); + + // set value + short newValue = (short) (defaultValue * 2); + key.set(conf, newValue); + assertEquals(newValue, key.getShort(conf)); + assertEquals(newValue, key.get(conf)); + } + + @Test + public void testGetDouble() { + String keyName = runtime.getMethodName(); + double defaultValue = ThreadLocalRandom.current().nextDouble(10000.0f); + ConfigKey key = ConfigKey.builder(keyName) + .required(true) + .type(Type.DOUBLE) + .defaultValue(defaultValue) + .build(); + + Configuration conf = new ConcurrentConfiguration(); + + // get default value + assertEquals(defaultValue, key.getDouble(conf), 0.0001); + assertEquals(defaultValue, key.get(conf)); + + // set value + double newValue = (defaultValue * 2); + key.set(conf, newValue); + assertEquals(newValue, key.getDouble(conf), 0.0001); + assertEquals(newValue, key.get(conf)); + } + + @Test + public void testGetBoolean() { + String keyName = runtime.getMethodName(); + boolean defaultValue = ThreadLocalRandom.current().nextBoolean(); + ConfigKey key = ConfigKey.builder(keyName) + .required(true) + .type(Type.BOOLEAN) + .defaultValue(defaultValue) + .build(); + + Configuration conf = new ConcurrentConfiguration(); + + // get default value + assertEquals(defaultValue, key.getBoolean(conf)); + assertEquals(defaultValue, key.get(conf)); + + // set value + boolean newValue = !defaultValue; + key.set(conf, newValue); + assertEquals(newValue, key.getBoolean(conf)); + assertEquals(newValue, key.get(conf)); + } + + @Test + public void testGetList() { + String keyName = runtime.getMethodName(); + List defaultList = Lists.newArrayList( + "item1", "item2", "item3" + ); + ConfigKey key = ConfigKey.builder(keyName) + .required(true) + .type(Type.LIST) + .defaultValue(defaultList) + .build(); + + Configuration conf = new CompositeConfiguration(); + + // get default value + assertEquals(defaultList, key.getList(conf)); + assertEquals(defaultList, key.get(conf)); + + // set value + List newList = Lists.newArrayList( + "item4", "item5", "item6" + ); + key.set(conf, newList); + assertEquals(newList, key.getList(conf)); + assertEquals(newList, key.get(conf)); + + // set string value + newList = Lists.newArrayList( + "item7", "item8", "item9" + ); + conf.setProperty(key.name(), "item7,item8,item9"); + assertEquals(newList, key.getList(conf)); + assertEquals(newList, key.get(conf)); + } + + @Test + public void testGetClass() { + String keyName = runtime.getMethodName(); + Class defaultClass = TestFunctionA.class; + ConfigKey key = ConfigKey.builder(keyName) + .required(true) + .type(Type.CLASS) + .defaultValue(defaultClass) + .build(); + + Configuration conf = new CompositeConfiguration(); + + // get default value + assertEquals(defaultClass, key.getClass(conf)); + assertEquals(defaultClass, key.get(conf)); + + // set value + Class newClass = TestFunctionB.class; + key.set(conf, newClass); + assertEquals(newClass, key.getClass(conf)); + assertEquals(newClass, key.get(conf)); + + // set string value + String newClassName = TestFunctionC.class.getName(); + conf.setProperty(key.name(), newClassName); + assertEquals(TestFunctionC.class, key.getClass(conf)); + assertEquals(TestFunctionC.class, key.get(conf)); + } + + @Test + public void testGetString() { + String keyName = runtime.getMethodName(); + String defaultValue = "default-string-value"; + ConfigKey key = ConfigKey.builder(keyName) + .required(true) + .type(Type.STRING) + .defaultValue(defaultValue) + .build(); + + Configuration conf = new CompositeConfiguration(); + + // get default value + assertEquals(defaultValue, key.getString(conf)); + assertEquals(defaultValue, key.get(conf)); + + // set value + String newValue = "new-string-value"; + key.set(conf, newValue); + assertEquals(newValue, key.getString(conf)); + assertEquals(newValue, key.get(conf)); + } + +} diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/validators/ClassValidatorTest.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/validators/ClassValidatorTest.java new file mode 100644 index 00000000000..bfb7971b45b --- /dev/null +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/validators/ClassValidatorTest.java @@ -0,0 +1,61 @@ +/* + * 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.common.conf.validators; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.function.Function; +import org.junit.Test; + +/** + * Unit test for {@link ClassValidator}. + */ +public class ClassValidatorTest { + + private static class TestFunction implements Function { + + @Override + public String apply(String s) { + return s + "!"; + } + } + + @Test + public void testValidateStrings() { + ClassValidator validator = ClassValidator.of(Function.class); + assertTrue(validator.validate("test-valid-classname", TestFunction.class.getName())); + assertFalse(validator.validate("test-invalid-classname", "unknown")); + } + + @Test + public void testValidateClass() { + ClassValidator validator = ClassValidator.of(Function.class); + assertTrue(validator.validate("test-valid-class", TestFunction.class)); + assertFalse(validator.validate("test-invalid-class", Integer.class)); + } + + @Test + public void testValidateWrongType() { + ClassValidator validator = ClassValidator.of(Function.class); + assertFalse(validator.validate("test-invalid-type", 12345)); + } + +} diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/validators/RangeValidatorTest.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/validators/RangeValidatorTest.java new file mode 100644 index 00000000000..b8725957c2a --- /dev/null +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/conf/validators/RangeValidatorTest.java @@ -0,0 +1,58 @@ +/* + * 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.common.conf.validators; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; + +/** + * Unit test {@link RangeValidator} validator. + */ +public class RangeValidatorTest { + + @Test + public void testAtLeastRangeValidator() { + RangeValidator range = RangeValidator.atLeast(1234L); + assertTrue(range.validate("test-0", 1235L)); + assertTrue(range.validate("test-1", 1234L)); + assertFalse(range.validate("test-2", 1233L)); + } + + @Test + public void testAtMostRangeValidator() { + RangeValidator range = RangeValidator.atMost(1234L); + assertFalse(range.validate("test-0", 1235L)); + assertTrue(range.validate("test-1", 1234L)); + assertTrue(range.validate("test-2", 1233L)); + } + + @Test + public void testBetweenRangeValidator() { + RangeValidator range = RangeValidator.between(1230L, 1240L); + assertTrue(range.validate("test-0", 1230L)); + assertTrue(range.validate("test-1", 1235L)); + assertTrue(range.validate("test-2", 1240L)); + assertFalse(range.validate("test-3", 1229L)); + assertFalse(range.validate("test-4", 1241L)); + } + +} diff --git a/bookkeeper-common/src/test/resources/test_conf_2.conf b/bookkeeper-common/src/test/resources/test_conf_2.conf new file mode 100644 index 00000000000..ca6f7bb2dbc --- /dev/null +++ b/bookkeeper-common/src/test/resources/test_conf_2.conf @@ -0,0 +1,130 @@ +################################################################################ +# Settings of `group1` +# +# This is a very long description : Lorem ipsum dolor sit amet, consectetur +# adipiscing elit. Maecenas bibendum ac felis id commodo. Etiam mauris purus, +# fringilla id tempus in, mollis vel orci. Duis ultricies at erat eget iaculis. +################################################################################ + +# it is a bool key +# +# TYPE: BOOLEAN, optional +bool_key= + +# it is a class key +# +# TYPE: CLASS, optional +# @constraints : class extends `java.lang.Runnable` +class_key= + +# it is a double key +# +# TYPE: DOUBLE, optional +# @constraints : [1234.0, 5678.0] +double_key= + +# it is an int key +# +# TYPE: INT, optional +# @constraints : [1000, ...] +int_key= + +# it is a list key +# +# TYPE: LIST, optional +list_key= + +# it is a long key +# +# TYPE: LONG, optional +# @constraints : [... , 1000] +long_key= + +# it is a short key +# +# TYPE: SHORT, optional +# @constraints : [500, 1000] +short_key= + +# it is a string key +# +# TYPE: STRING, optional +string_key= + +################################## +# Settings of `group2` +# +# This group has short description +################################## + +# it is a required key +# +# TYPE: STRING, required +required_key= + +# it is a deprecated key with since and replaced key +# +# TYPE: STRING, optional +# +# @deprecated since `4.3.0` in favor of using `key_with_optional_values` +deprecated_key_with_since_and_replaced_key= + +# it is a deprecated key with replaced key +# +# TYPE: STRING, optional +# +# @deprecated in favor of using `key_with_optional_values` +deprecated_key_with_replaced_key= + +# it is a deprecated key with since +# +# TYPE: STRING, optional +# +# @deprecated since `4.3.0` +deprecated_key_with_since= + +# it is a deprecated key +# +# TYPE: STRING, optional +# +# @deprecated +deprecated_key= + +# it is a string key with optional values +# +# TYPE: STRING, optional +# @options : +# item1 +# item2 +# item3 +# item3 +key_with_optional_values=this-is-a-default-value + +# it is a string key with default value +# +# TYPE: STRING, optional +key_with_default_value=this-is-a-test-value + +# it is a string key with documentation +# +# it has a long documentation : Lorem ipsum dolor sit amet, consectetur +# adipiscing elit. Maecenas bibendum ac felis id commodo. Etiam mauris purus, +# fringilla id tempus in, mollis vel orci. Duis ultricies at erat eget iaculis. +# +# TYPE: STRING, optional +key_long_short_documentation= + +# it is a string key with documentation +# +# it has a short documentation +# +# TYPE: STRING, optional +key_with_short_documentation= + +# it is a string key with since +# +# TYPE: STRING, optional +# +# @since 4.7.0 +key_with_since= + 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 2a77e91cde0..1d3865189f2 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 @@ -17,28 +17,77 @@ */ package org.apache.bookkeeper.conf; +import static org.apache.bookkeeper.util.BookKeeperConstants.MAX_LOG_SIZE_LIMIT; + import com.google.common.annotations.Beta; import com.google.common.base.Strings; +import com.google.common.collect.Lists; import java.io.File; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; import org.apache.bookkeeper.bookie.LedgerStorage; import org.apache.bookkeeper.bookie.SortedLedgerStorage; +import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; +import org.apache.bookkeeper.common.conf.ConfigDef; +import org.apache.bookkeeper.common.conf.ConfigException; +import org.apache.bookkeeper.common.conf.ConfigKey; +import org.apache.bookkeeper.common.conf.ConfigKeyGroup; +import org.apache.bookkeeper.common.conf.Type; +import org.apache.bookkeeper.common.conf.validators.ClassValidator; +import org.apache.bookkeeper.common.conf.validators.RangeValidator; import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.discover.ZKRegistrationManager; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; -import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.commons.configuration.ConfigurationException; /** * Configuration manages server-side settings. */ public class ServerConfiguration extends AbstractConfiguration { + + // Ledger Storage Settings + + private static final ConfigKeyGroup GROUP_LEDGER_STORAGE = ConfigKeyGroup.builder("ledgerstorage") + .description("Ledger Storage related settings") + .order(10) // place a place holder here + .build(); + + protected static final String LEDGER_STORAGE_CLASS = "ledgerStorageClass"; + protected static final ConfigKey LEDGER_STORAGE_CLASS_KEY = ConfigKey.builder(LEDGER_STORAGE_CLASS) + .type(Type.CLASS) + .description("Ledger storage implementation class") + .defaultValue(SortedLedgerStorage.class) + .optionValues(Lists.newArrayList( + InterleavedLedgerStorage.class.getName(), + SortedLedgerStorage.class.getName(), + DbLedgerStorage.class.getName() + )) + .validator(ClassValidator.of(LedgerStorage.class)) + .group(GROUP_LEDGER_STORAGE) + .build(); + // Entry Log Parameters + + private static final ConfigKeyGroup GROUP_LEDGER_STORAGE_ENTRY_LOGGER = ConfigKeyGroup.builder("entrylogger") + .description("EntryLogger related settings") + .order(11) + .build(); + protected static final String ENTRY_LOG_SIZE_LIMIT = "logSizeLimit"; + protected static final ConfigKey ENTRY_LOG_SIZE_LIMIT_KEY = ConfigKey.builder(ENTRY_LOG_SIZE_LIMIT) + .type(Type.LONG) + .description("Max file size of entry logger, in bytes") + .documentation("A new entry log file will be created when the old one reaches this file size limitation") + .defaultValue(MAX_LOG_SIZE_LIMIT) + .validator(RangeValidator.between(0, MAX_LOG_SIZE_LIMIT)) + .group(GROUP_LEDGER_STORAGE_ENTRY_LOGGER) + .build(); + protected static final String ENTRY_LOG_FILE_PREALLOCATION_ENABLED = "entryLogFilePreallocationEnabled"; + + protected static final String MINOR_COMPACTION_INTERVAL = "minorCompactionInterval"; protected static final String MINOR_COMPACTION_THRESHOLD = "minorCompactionThreshold"; protected static final String MAJOR_COMPACTION_INTERVAL = "majorCompactionInterval"; @@ -166,7 +215,6 @@ public class ServerConfiguration extends AbstractConfiguration * @throws ConfigurationException */ public void validate() throws ConfigurationException { + // generate config def + ConfigDef configDef = ConfigDef.of(ServerConfiguration.class); + try { + configDef.validate(this); + } catch (ConfigException e) { + throw new ConfigurationException(e.getMessage(), e.getCause()); + } + if (getSkipListArenaChunkSize() < getSkipListArenaMaxAllocSize()) { throw new ConfigurationException("Arena max allocation size should be smaller than the chunk size."); } @@ -2532,10 +2588,6 @@ public void validate() throws ConfigurationException { if (getJournalAlignmentSize() > getJournalPreAllocSizeMB() * 1024 * 1024) { throw new ConfigurationException("Invalid preallocation size : " + getJournalPreAllocSizeMB() + " MB"); } - if (getEntryLogSizeLimit() > BookKeeperConstants.MAX_LOG_SIZE_LIMIT) { - throw new ConfigurationException("Entry log file size should not be larger than " - + BookKeeperConstants.MAX_LOG_SIZE_LIMIT); - } if (0 == getBookiePort() && !getAllowEphemeralPorts()) { throw new ConfigurationException("Invalid port specified, using ephemeral ports accidentally?"); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java index 322cdd08990..44f20e6f54c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java @@ -99,7 +99,7 @@ public void checkpointComplete(Checkpoint checkpoint, boolean compact) public SortedLedgerStorageCheckpointTest() { super(); - conf.setEntryLogSizeLimit(1); + conf.setEntryLogSizeLimit(1024); conf.setEntryLogFilePreAllocationEnabled(false); this.checkpoints = new LinkedBlockingQueue<>(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestServerConfiguration.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestServerConfiguration.java index 424202d7fe5..fb139f5f29f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestServerConfiguration.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestServerConfiguration.java @@ -22,8 +22,10 @@ package org.apache.bookkeeper.conf; import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import org.apache.commons.configuration.ConfigurationException; import org.junit.Before; @@ -111,4 +113,43 @@ public void testValidityOfJournalAndFileInfoVersions() throws ConfigurationExcep conf.setFileInfoFormatVersionToWrite(1); conf.validate(); } + + @Test + public void testEntryLogSizeLimit() throws ConfigurationException { + ServerConfiguration conf = new ServerConfiguration(); + try { + conf.setEntryLogSizeLimit(-1); + fail("should fail setEntryLogSizeLimit since `logSizeLimit` is too small"); + } catch (IllegalArgumentException iae) { + // expected + } + try { + conf.setProperty("logSizeLimit", "-1"); + conf.validate(); + fail("Invalid configuration since `logSizeLimit` is too small"); + } catch (ConfigurationException ce) { + // expected + } + + try { + conf.setEntryLogSizeLimit(2 * 1024 * 1024 * 1024L - 1); + fail("Should fail setEntryLogSizeLimit size `logSizeLimit` is too large"); + } catch (IllegalArgumentException iae) { + // expected + } + try { + conf.validate(); + fail("Invalid configuration since `logSizeLimit` is too large"); + } catch (ConfigurationException ce) { + // expected + } + + conf.setEntryLogSizeLimit(512 * 1024 * 1024); + conf.validate(); + assertEquals(512 * 1024 * 1024, conf.getEntryLogSizeLimit()); + + conf.setEntryLogSizeLimit(1073741824); + conf.validate(); + assertEquals(1073741824, conf.getEntryLogSizeLimit()); + } } diff --git a/pom.xml b/pom.xml index 125f2e2458a..c0c2c1ab2b5 100644 --- a/pom.xml +++ b/pom.xml @@ -996,6 +996,9 @@ **/__pycache__/** **/bookkeeper.egg-info/** **/pip-selfcheck.json + + + **/test_conf_2.conf true From ee8b41fdb66e6dae6a22cdacf7601db67fb1b960 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Sat, 22 Dec 2018 00:41:22 +0100 Subject: [PATCH 0183/1642] Multiple calls to LedgerHandle#close should wait on actual close The previous behaviour was to complete successfully immediately if close had already been called on the handle. This allowed for potential consistency violations, as the caller could believe that the ledger was closed, when in fact the close operation was still in progress and could still potentially fail. Issue: #1712 Reviewers: Sijie Guo This closes #1889 from ivankelly/double-close-anomoly --- .../bookkeeper/client/LedgerHandle.java | 22 +++++++----- .../bookkeeper/client/LedgerClose2Test.java | 36 +++++++++++++++++++ 2 files changed, 50 insertions(+), 8 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 6f0af27e179..157a1b84cac 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 @@ -106,6 +106,7 @@ private enum HandleState { }; private HandleState handleState = HandleState.OPEN; + private final CompletableFuture closePromise = new CompletableFuture<>(); /** * Last entryId which has been confirmed to be written durably to the bookies. @@ -508,6 +509,16 @@ public void safeRun() { final long lastEntry; final long finalLength; + closePromise.whenComplete((ignore, ex) -> { + if (ex != null) { + cb.closeComplete( + BKException.getExceptionCode(ex, BKException.Code.UnexpectedConditionException), + LedgerHandle.this, ctx); + } else { + cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx); + } + }); + synchronized (LedgerHandle.this) { prevHandleState = handleState; @@ -524,9 +535,7 @@ public void safeRun() { // running under any bk locks. errorOutPendingAdds(rc, pendingAdds); - if (prevHandleState == HandleState.CLOSED) { - cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx); - } else { + if (prevHandleState != HandleState.CLOSED) { if (LOG.isDebugEnabled()) { LOG.debug("Closing ledger: {} at entryId {} with {} bytes", getId(), lastEntry, finalLength); } @@ -564,12 +573,9 @@ public void safeRun() { LedgerHandle.this::setLedgerMetadata) .run().whenComplete((metadata, ex) -> { if (ex != null) { - cb.closeComplete( - BKException.getExceptionCode( - ex, BKException.Code.UnexpectedConditionException), - LedgerHandle.this, ctx); + closePromise.completeExceptionally(ex); } else { - cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx); + FutureUtils.complete(closePromise, null); } }); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java index 45b27135935..612b36e27e4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java @@ -268,5 +268,41 @@ public void testCloseWhileAddInProgress() throws Exception { Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID); Assert.assertEquals(lh.getLedgerMetadata().getLength(), 0); } + + @Test + public void testDoubleCloseOnHandle() throws Exception { + long ledgerId = 123L; + MockClientContext clientCtx = MockClientContext.create(); + + Versioned md = ClientUtil.setupLedger(clientCtx, ledgerId, + LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3))); + + CompletableFuture metadataPromise = new CompletableFuture<>(); + CompletableFuture clientPromise = new CompletableFuture<>(); + + LedgerHandle writer = new LedgerHandle(clientCtx, ledgerId, md, + BookKeeper.DigestType.CRC32C, + ClientUtil.PASSWD, WriteFlag.NONE); + long eid1 = writer.append("entry1".getBytes()); + + log.info("block writes from completing on bookies and metadata"); + clientCtx.getMockBookieClient().setPostWriteHook((bookie, lid, eid) -> clientPromise); + clientCtx.getMockLedgerManager().setPreWriteHook((lid, metadata) -> metadataPromise); + + log.info("try to add another entry, it will block"); + writer.appendAsync("entry2".getBytes()); + + log.info("attempt one close, should block forever"); + CompletableFuture firstClose = writer.closeAsync(); + + log.info("attempt second close, should not finish before first one"); + CompletableFuture secondClose = writer.closeAsync(); + + Thread.sleep(500); // give it a chance to complete, the request jumps around threads + Assert.assertFalse(firstClose.isDone()); + Assert.assertFalse(secondClose.isDone()); + } } From 1ca8ba949e34a28f68ca7c1abf3b6c792c96cb13 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Sat, 22 Dec 2018 00:44:36 +0100 Subject: [PATCH 0184/1642] Revert maxLedgerMetadataFormatVersion changes in layout There is ongoing discussions about how to do this, so I'm reverting this change for now to allow 4.9 release to proceed. The change modifies the contents of the layout znode, so once they're in an official release they cannot be removed without breaking BC. The reverted changes are: dd684b Ledger manager factories initialized with max metadata version dce4fd Add max ledger metadata format version to layout Master issue: #723 Reviewers: Sijie Guo , Enrico Olivelli This closes #1890 from ivankelly/revert-max-format --- .../apache/bookkeeper/bookie/BookieShell.java | 7 +-- .../AbstractHierarchicalLedgerManager.java | 5 +-- .../meta/AbstractZkLedgerManager.java | 5 +-- .../meta/AbstractZkLedgerManagerFactory.java | 15 +++---- .../bookkeeper/meta/FlatLedgerManager.java | 5 +-- .../meta/FlatLedgerManagerFactory.java | 7 +-- .../meta/HierarchicalLedgerManager.java | 9 ++-- .../HierarchicalLedgerManagerFactory.java | 2 +- .../apache/bookkeeper/meta/LedgerLayout.java | 43 ++++--------------- .../bookkeeper/meta/LedgerManagerFactory.java | 5 +-- .../bookkeeper/meta/LedgerMetadataSerDe.java | 6 --- .../meta/LegacyHierarchicalLedgerManager.java | 5 +-- ...egacyHierarchicalLedgerManagerFactory.java | 7 +-- .../meta/LongHierarchicalLedgerManager.java | 5 +-- .../LongHierarchicalLedgerManagerFactory.java | 2 +- .../meta/MSLedgerManagerFactory.java | 12 ++---- .../http/service/GetLedgerMetaService.java | 2 +- .../http/service/ListLedgerService.java | 2 +- .../bookkeeper/client/LedgerMetadataTest.java | 6 +-- .../client/ParallelLedgerRecoveryTest.java | 7 +-- .../meta/AbstractZkLedgerManagerTest.java | 14 +++--- .../bookkeeper/meta/MockLedgerManager.java | 2 +- .../bookkeeper/meta/TestLedgerLayout.java | 43 ------------------- .../server/http/TestHttpService.java | 3 +- .../metadata/etcd/EtcdLedgerManager.java | 29 ++++++------- .../etcd/EtcdLedgerManagerFactory.java | 7 +-- .../metadata/etcd/EtcdMetadataDriverBase.java | 4 +- .../metadata/etcd/EtcdLedgerManagerTest.java | 4 +- .../DistributedLogCoreShadedJarTest.java | 4 +- 29 files changed, 75 insertions(+), 192 deletions(-) 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 083a66159c4..32e288b7e3b 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 @@ -1164,7 +1164,7 @@ Options getOptions() { void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { System.out.println("ledgerID: " + ledgerIdFormatter.formatLedgerId(ledgerId)); if (printMeta) { - System.out.println(md.toString()); + System.out.println(new String(new LedgerMetadataSerDe().serialize(md), UTF_8)); } } @@ -1173,10 +1173,7 @@ void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { */ class LedgerMetadataCmd extends MyCommand { Options lOpts = new Options(); - // the max version won't actually take effect as this tool - // never creates new metadata (there'll already be a format version in the existing metadata) - LedgerMetadataSerDe serDe = new LedgerMetadataSerDe( - LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); LedgerMetadataCmd() { super(CMD_LEDGERMETADATA); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java index 1a0ce2eb5f5..1aa8e151d2c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java @@ -49,9 +49,8 @@ public abstract class AbstractHierarchicalLedgerManager extends AbstractZkLedger * @param zk * ZooKeeper Client Handle */ - public AbstractHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk, - int maxLedgerMetadataFormatVersion) { - super(conf, zk, maxLedgerMetadataFormatVersion); + public AbstractHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { + super(conf, zk); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 1c901542c32..5dbdd06791d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -159,9 +159,8 @@ private void handleMetadata(Versioned result, Throwable exceptio * @param zk * ZooKeeper Client Handle */ - protected AbstractZkLedgerManager(AbstractConfiguration conf, ZooKeeper zk, - int maxLedgerMetadataFormatVersion) { - this.serDe = new LedgerMetadataSerDe(maxLedgerMetadataFormatVersion); + protected AbstractZkLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { + this.serDe = new LedgerMetadataSerDe(); this.conf = conf; this.zk = zk; this.ledgerRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java index be95a53886c..72bc3e841e6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java @@ -160,8 +160,8 @@ public static LedgerManagerFactory newLedgerManagerFactory( // if layoutManager is null, return the default ledger manager if (layoutManager == null) { - return new FlatLedgerManagerFactory().initialize(conf, null, - FlatLedgerManagerFactory.CUR_VERSION, LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + return new FlatLedgerManagerFactory() + .initialize(conf, null, FlatLedgerManagerFactory.CUR_VERSION); } LedgerManagerFactory lmFactory; @@ -172,9 +172,8 @@ public static LedgerManagerFactory newLedgerManagerFactory( if (layout == null) { // no existing layout lmFactory = createNewLMFactory(conf, layoutManager, factoryClass); - return lmFactory.initialize(conf, layoutManager, - lmFactory.getCurrentVersion(), - LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + return lmFactory + .initialize(conf, layoutManager, lmFactory.getCurrentVersion()); } if (log.isDebugEnabled()) { log.debug("read ledger layout {}", layout); @@ -198,8 +197,7 @@ public static LedgerManagerFactory newLedgerManagerFactory( } else { throw new IOException("Unknown ledger manager type: " + lmType); } - return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion(), - layout.getMaxLedgerMetadataFormatVersion()); + return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion()); } // handle V2 layout case @@ -229,8 +227,7 @@ public static LedgerManagerFactory newLedgerManagerFactory( } // instantiate a factory lmFactory = ReflectionUtils.newInstance(factoryClass); - return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion(), - layout.getMaxLedgerMetadataFormatVersion()); + return lmFactory.initialize(conf, layoutManager, layout.getManagerVersion()); } private static String normalizedLedgerManagerFactoryClassName(String factoryClass, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java index 115ffe3f8c3..7ee2e2289ea 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java @@ -55,9 +55,8 @@ class FlatLedgerManager extends AbstractZkLedgerManager { * ZooKeeper Client Handle * @throws IOException when version is not compatible */ - public FlatLedgerManager(AbstractConfiguration conf, ZooKeeper zk, - int maxLedgerMetadataFormatVersion) { - super(conf, zk, maxLedgerMetadataFormatVersion); + public FlatLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { + super(conf, zk); ledgerPrefix = ledgerRootPath + "/" + StringUtils.LEDGER_NODE_PREFIX; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java index 5c92ba12d93..19ac418eeae 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManagerFactory.java @@ -41,7 +41,6 @@ public class FlatLedgerManagerFactory extends AbstractZkLedgerManagerFactory { public static final int CUR_VERSION = 1; AbstractConfiguration conf; - private int maxLedgerMetadataFormatVersion; @Override public int getCurrentVersion() { @@ -51,8 +50,7 @@ public int getCurrentVersion() { @Override public LedgerManagerFactory initialize(final AbstractConfiguration conf, final LayoutManager layoutManager, - final int factoryVersion, - int maxLedgerMetadataFormatVersion) + final int factoryVersion) throws IOException { checkArgument(layoutManager == null || layoutManager instanceof ZkLayoutManager); @@ -63,7 +61,6 @@ public LedgerManagerFactory initialize(final AbstractConfiguration conf, this.conf = conf; this.zk = layoutManager == null ? null : ((ZkLayoutManager) layoutManager).getZk(); - this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; return this; } @@ -82,7 +79,7 @@ public LedgerIdGenerator newLedgerIdGenerator() { @Override public LedgerManager newLedgerManager() { - return new FlatLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); + return new FlatLedgerManager(conf, zk); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java index c4f211a1ab2..946ed2a5d32 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java @@ -46,11 +46,10 @@ class HierarchicalLedgerManager extends AbstractHierarchicalLedgerManager { LegacyHierarchicalLedgerManager legacyLM; LongHierarchicalLedgerManager longLM; - public HierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk, - int maxLedgerMetadataFormatVersion) { - super(conf, zk, maxLedgerMetadataFormatVersion); - legacyLM = new LegacyHierarchicalLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); - longLM = new LongHierarchicalLedgerManager (conf, zk, maxLedgerMetadataFormatVersion); + public HierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { + super(conf, zk); + legacyLM = new LegacyHierarchicalLedgerManager(conf, zk); + longLM = new LongHierarchicalLedgerManager (conf, zk); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManagerFactory.java index c4f0f4f4d62..4a9d6cfbf6e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManagerFactory.java @@ -42,6 +42,6 @@ public LedgerIdGenerator newLedgerIdGenerator() { @Override public LedgerManager newLedgerManager() { - return new HierarchicalLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); + return new HierarchicalLedgerManager(conf, zk); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java index c39e6b6feae..eb5b705c16b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java @@ -39,14 +39,8 @@ public class LedgerLayout { // version of ledger layout metadata public static final int LAYOUT_FORMAT_VERSION = 2; - private static final String FIELD_SPLITTER = ":"; - private static final String LINE_SPLITTER = "\n"; - - // For version 2 and below, max ledger metadata format wasn't stored in the layout - // so assume 2 if it is missing. - private static final int DEFAULT_MAX_LEDGER_METADATA_FORMAT_VERSION = 2; - private static final String MAX_LEDGER_METADATA_FORMAT_VERSION_FIELD = - "MAX_LEDGER_METADATA_FORMAT_VERSION"; + private static final String splitter = ":"; + private static final String lSplitter = "\n"; // ledger manager factory class private final String managerFactoryClass; @@ -56,9 +50,6 @@ public class LedgerLayout { // layout version of how to store layout information private final int layoutFormatVersion; - // maximum format version that can be used for storing ledger metadata - private final int maxLedgerMetadataFormatVersion; - /** * Ledger Layout Constructor. * @@ -68,17 +59,13 @@ public class LedgerLayout { * Ledger Manager Version */ public LedgerLayout(String managerFactoryCls, int managerVersion) { - this(managerFactoryCls, managerVersion, - LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION, - LAYOUT_FORMAT_VERSION); + this(managerFactoryCls, managerVersion, LAYOUT_FORMAT_VERSION); } LedgerLayout(String managerFactoryCls, int managerVersion, - int maxLedgerMetadataFormatVersion, int layoutVersion) { this.managerFactoryClass = managerFactoryCls; this.managerVersion = managerVersion; - this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; this.layoutFormatVersion = layoutVersion; } @@ -89,11 +76,8 @@ public LedgerLayout(String managerFactoryCls, int managerVersion) { */ public byte[] serialize() throws IOException { String s = - new StringBuilder().append(layoutFormatVersion).append(LINE_SPLITTER) - .append(managerFactoryClass).append(FIELD_SPLITTER).append(managerVersion).append(LINE_SPLITTER) - .append(MAX_LEDGER_METADATA_FORMAT_VERSION_FIELD).append(FIELD_SPLITTER) - .append(maxLedgerMetadataFormatVersion) - .toString(); + new StringBuilder().append(layoutFormatVersion).append(lSplitter) + .append(managerFactoryClass).append(splitter).append(managerVersion).toString(); if (log.isDebugEnabled()) { log.debug("Serialized layout info: {}", s); @@ -116,7 +100,7 @@ public static LedgerLayout parseLayout(byte[] bytes) throws IOException { log.debug("Parsing Layout: {}", layout); } - String lines[] = layout.split(LINE_SPLITTER); + String lines[] = layout.split(lSplitter); try { int layoutFormatVersion = Integer.parseInt(lines[0]); @@ -129,7 +113,7 @@ public static LedgerLayout parseLayout(byte[] bytes) throws IOException { throw new IOException("Ledger manager and its version absent from layout: " + layout); } - String[] parts = lines[1].split(FIELD_SPLITTER); + String[] parts = lines[1].split(splitter); if (parts.length != 2) { throw new IOException("Invalid Ledger Manager defined in layout : " + layout); } @@ -137,18 +121,7 @@ public static LedgerLayout parseLayout(byte[] bytes) throws IOException { String managerFactoryCls = parts[0]; // ledger manager version int managerVersion = Integer.parseInt(parts[1]); - - int maxLedgerMetadataFormatVersion = DEFAULT_MAX_LEDGER_METADATA_FORMAT_VERSION; - if (lines.length >= 3) { - String[] metadataFormatParts = lines[2].split(FIELD_SPLITTER); - if (metadataFormatParts.length != 2 - || !metadataFormatParts[0].equals(MAX_LEDGER_METADATA_FORMAT_VERSION_FIELD)) { - throw new IOException("Invalid field for max ledger metadata format:" + lines[2]); - } - maxLedgerMetadataFormatVersion = Integer.parseInt(metadataFormatParts[1]); - } - return new LedgerLayout(managerFactoryCls, managerVersion, - maxLedgerMetadataFormatVersion, layoutFormatVersion); + return new LedgerLayout(managerFactoryCls, managerVersion, layoutFormatVersion); } catch (NumberFormatException e) { throw new IOException("Could not parse layout '" + layout + "'", e); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java index eb4e0f446a6..80d3a6526f9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManagerFactory.java @@ -43,15 +43,12 @@ public interface LedgerManagerFactory extends AutoCloseable { * Layout manager used for initialize ledger manager factory * @param factoryVersion * What version used to initialize factory. - * @param maxLedgerMetadataFormatVersion - * Maximum format version for ledger metadata. * @return ledger manager factory instance * @throws IOException when fail to initialize the factory. */ LedgerManagerFactory initialize(AbstractConfiguration conf, LayoutManager layoutManager, - int factoryVersion, - int maxLedgerMetadataFormatVersion) + int factoryVersion) throws IOException; /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java index 4653af4f309..6020a3bf4f1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -63,12 +63,6 @@ public class LedgerMetadataSerDe { private static final String V1_CLOSED_TAG = "CLOSED"; private static final int V1_IN_RECOVERY_ENTRY_ID = -102; - private final int maxLedgerMetadataFormatVersion; - - public LedgerMetadataSerDe(int maxLedgerMetadataFormatVersion) { - this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; - } - public byte[] serialize(LedgerMetadata metadata) { if (metadata.getMetadataFormatVersion() == 1) { return serializeVersion1(metadata); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java index b5065a872a0..76ecc9d68de 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java @@ -69,9 +69,8 @@ protected StringBuilder initialValue() { * @param zk * ZooKeeper Client Handle */ - public LegacyHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk, - int maxLedgerMetadataFormatVersion) { - super(conf, zk, maxLedgerMetadataFormatVersion); + public LegacyHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { + super(conf, zk); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManagerFactory.java index bd5c57927c2..91579735b05 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManagerFactory.java @@ -38,7 +38,6 @@ public class LegacyHierarchicalLedgerManagerFactory extends AbstractZkLedgerMana public static final int CUR_VERSION = 1; AbstractConfiguration conf; - int maxLedgerMetadataFormatVersion; @Override public int getCurrentVersion() { @@ -48,8 +47,7 @@ public int getCurrentVersion() { @Override public LedgerManagerFactory initialize(final AbstractConfiguration conf, final LayoutManager lm, - final int factoryVersion, - int maxLedgerMetadataFormatVersion) + final int factoryVersion) throws IOException { checkArgument(lm instanceof ZkLayoutManager); @@ -60,7 +58,6 @@ public LedgerManagerFactory initialize(final AbstractConfiguration conf, + factoryVersion); } this.conf = conf; - this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; this.zk = zkLayoutManager.getZk(); return this; } @@ -83,7 +80,7 @@ public LedgerIdGenerator newLedgerIdGenerator() { @Override public LedgerManager newLedgerManager() { - return new LegacyHierarchicalLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); + return new LegacyHierarchicalLedgerManager(conf, zk); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java index e464ed04a18..2e69e90a5c1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java @@ -70,9 +70,8 @@ class LongHierarchicalLedgerManager extends AbstractHierarchicalLedgerManager { * @param zk * ZooKeeper Client Handle */ - public LongHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk, - int maxLedgerMetadataFormatVersion) { - super(conf, zk, maxLedgerMetadataFormatVersion); + public LongHierarchicalLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { + super(conf, zk); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManagerFactory.java index 4c06cc48269..93ad9ddc681 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManagerFactory.java @@ -26,7 +26,7 @@ public class LongHierarchicalLedgerManagerFactory extends HierarchicalLedgerMana @Override public LedgerManager newLedgerManager() { - return new LongHierarchicalLedgerManager(conf, zk, maxLedgerMetadataFormatVersion); + return new LongHierarchicalLedgerManager(conf, zk); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index e10f2743c68..fc876324452 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -96,7 +96,6 @@ public class MSLedgerManagerFactory extends AbstractZkLedgerManagerFactory { public static final String META_FIELD = ".META"; AbstractConfiguration conf; - private int maxLedgerMetadataFormatVersion; MetaStore metastore; @Override @@ -107,8 +106,7 @@ public int getCurrentVersion() { @Override public LedgerManagerFactory initialize(final AbstractConfiguration conf, final LayoutManager layoutManager, - final int factoryVersion, - int maxLedgerMetadataFormatVersion) throws IOException { + final int factoryVersion) throws IOException { checkArgument(layoutManager instanceof ZkLayoutManager); ZkLayoutManager zkLayoutManager = (ZkLayoutManager) layoutManager; @@ -117,7 +115,6 @@ public LedgerManagerFactory initialize(final AbstractConfiguration conf, } this.conf = conf; this.zk = zkLayoutManager.getZk(); - this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; // load metadata store String msName = conf.getMetastoreImplClass(); @@ -283,12 +280,11 @@ private void handleMetadata(Versioned metadata, Throwable except } } - MsLedgerManager(final AbstractConfiguration conf, final ZooKeeper zk, final MetaStore metastore, - int maxLedgerMetadataFormatVersion) { + MsLedgerManager(final AbstractConfiguration conf, final ZooKeeper zk, final MetaStore metastore) { this.conf = conf; this.zk = zk; this.metastore = metastore; - this.serDe = new LedgerMetadataSerDe(maxLedgerMetadataFormatVersion); + this.serDe = new LedgerMetadataSerDe(); try { ledgerTable = metastore.createScannableTable(TABLE_NAME); @@ -655,7 +651,7 @@ public static boolean isSpecialZnode(String znode) { @Override public LedgerManager newLedgerManager() { - return new MsLedgerManager(conf, zk, metastore, maxLedgerMetadataFormatVersion); + return new MsLedgerManager(conf, zk, metastore); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java index 537b50ce468..7fcaeda810d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java @@ -53,7 +53,7 @@ public GetLedgerMetaService(ServerConfiguration conf, BookieServer bookieServer) checkNotNull(conf); this.conf = conf; this.bookieServer = bookieServer; - this.serDe = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + this.serDe = new LedgerMetadataSerDe(); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java index 13022e4f2d7..f553b6431b9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java @@ -58,7 +58,7 @@ public ListLedgerService(ServerConfiguration conf, BookieServer bookieServer) { checkNotNull(conf); this.conf = conf; this.bookieServer = bookieServer; - this.serDe = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + this.serDe = new LedgerMetadataSerDe(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java index d9141e64616..a9ffc2d4221 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java @@ -80,8 +80,7 @@ public void testStoreSystemtimeAsLedgerCtimeEnabled() .withCreationTime(System.currentTimeMillis()) .storingCreationTime(true) .build(); - LedgerMetadataFormat format = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION) - .buildProtoFormat(lm); + LedgerMetadataFormat format = new LedgerMetadataSerDe().buildProtoFormat(lm); assertTrue(format.hasCtime()); } @@ -95,8 +94,7 @@ public void testStoreSystemtimeAsLedgerCtimeDisabled() LedgerMetadata lm = LedgerMetadataBuilder.create() .newEnsembleEntry(0L, ensemble).build(); - LedgerMetadataFormat format = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION) - .buildProtoFormat(lm); + LedgerMetadataFormat format = new LedgerMetadataSerDe().buildProtoFormat(lm); assertFalse(format.hasCtime()); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index 1f9656b94d6..ab966fb4ae2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -50,7 +50,6 @@ import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; -import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.exceptions.Code; import org.apache.bookkeeper.meta.exceptions.MetadataException; @@ -185,8 +184,7 @@ public synchronized LedgerManagerFactory getLedgerManagerFactory() throws Metada if (null == lmFactory) { try { lmFactory = new TestLedgerManagerFactory() - .initialize(conf, layoutManager, TestLedgerManagerFactory.CUR_VERSION, - LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + .initialize(conf, layoutManager, TestLedgerManagerFactory.CUR_VERSION); } catch (IOException e) { throw new MetadataException(Code.METADATA_SERVICE_ERROR, e); } @@ -202,8 +200,7 @@ public synchronized LedgerManagerFactory getLedgerManagerFactory() throws Metada if (null == lmFactory) { try { lmFactory = new TestLedgerManagerFactory() - .initialize(conf, layoutManager, TestLedgerManagerFactory.CUR_VERSION, - LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + .initialize(conf, layoutManager, TestLedgerManagerFactory.CUR_VERSION); } catch (IOException e) { throw new MetadataException(Code.METADATA_SERVICE_ERROR, e); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 620c3b2e004..37f55b3e6e6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -110,9 +110,9 @@ public void setup() throws Exception { this.conf = new ClientConfiguration(); this.ledgerManager = mock( - AbstractZkLedgerManager.class, - withSettings() - .useConstructor(conf, mockZk, LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION) + AbstractZkLedgerManager.class, + withSettings() + .useConstructor(conf, mockZk) .defaultAnswer(CALLS_REAL_METHODS)); List ensemble = Lists.newArrayList( new BookieSocketAddress("192.0.2.1", 3181), @@ -142,7 +142,7 @@ public void setup() throws Exception { assertSame(conf, ledgerManager.conf); assertSame(scheduler, ledgerManager.scheduler); - this.serDe = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + this.serDe = new LedgerMetadataSerDe(); } @After @@ -303,15 +303,13 @@ public void testRemoveLedgerMetadataException() throws Exception { @Test public void testRemoveLedgerMetadataHierarchical() throws Exception { - HierarchicalLedgerManager hlm = new HierarchicalLedgerManager(conf, mockZk, - LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + HierarchicalLedgerManager hlm = new HierarchicalLedgerManager(conf, mockZk); testRemoveLedgerMetadataHierarchicalLedgerManager(hlm); } @Test public void testRemoveLedgerMetadataLongHierarchical() throws Exception { - LongHierarchicalLedgerManager hlm = new LongHierarchicalLedgerManager(conf, mockZk, - LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + LongHierarchicalLedgerManager hlm = new LongHierarchicalLedgerManager(conf, mockZk); testRemoveLedgerMetadataHierarchicalLedgerManager(hlm); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index a73adeb4eee..398bb07d712 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -72,7 +72,7 @@ private MockLedgerManager(Map> metadataMap, this.metadataMap = metadataMap; this.executor = executor; this.ownsExecutor = ownsExecutor; - this.serDe = new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + this.serDe = new LedgerMetadataSerDe(); } public MockLedgerManager newClient() { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java index 16b30bdf071..62315fe2e81 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerLayout.java @@ -18,12 +18,8 @@ */ package org.apache.bookkeeper.meta; -import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.fail; - -import java.io.IOException; import org.junit.Test; @@ -67,43 +63,4 @@ public void testGetters() { hierarchical1.getLayoutFormatVersion()); } - @Test - public void testParseNoMaxLedgerMetadataFormatVersion() throws Exception { - LedgerLayout layout = LedgerLayout.parseLayout("1\nblahblahLM:3".getBytes(UTF_8)); - - assertEquals(layout.getMaxLedgerMetadataFormatVersion(), 2); - } - - @Test - public void testParseWithMaxLedgerMetadataFormatVersion() throws Exception { - LedgerLayout layout = LedgerLayout.parseLayout( - "1\nblahblahLM:3\nMAX_LEDGER_METADATA_FORMAT_VERSION:123".getBytes(UTF_8)); - - assertEquals(layout.getMaxLedgerMetadataFormatVersion(), 123); - } - - @Test - public void testCorruptMaxLedgerLayout() throws Exception { - try { - LedgerLayout.parseLayout("1\nblahblahLM:3\nMAXXX_LEDGER_METADATA_FORMAT_VERSION:123".getBytes(UTF_8)); - fail("Shouldn't have been able to parse"); - } catch (IOException ioe) { - // expected - } - - try { - LedgerLayout.parseLayout("1\nblahblahLM:3\nMAXXX_LEDGER_METADATA_FORMAT_VERSION:blah".getBytes(UTF_8)); - fail("Shouldn't have been able to parse"); - } catch (IOException ioe) { - // expected - } - } - - @Test - public void testMoreFieldsAdded() throws Exception { - LedgerLayout layout = LedgerLayout.parseLayout( - "1\nblahblahLM:3\nMAX_LEDGER_METADATA_FORMAT_VERSION:123\nFOO:BAR".getBytes(UTF_8)); - - assertEquals(layout.getMaxLedgerMetadataFormatVersion(), 123); - } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index da25d14d5c0..3d948c33d10 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -387,8 +387,7 @@ public void testGetLedgerMetaService() throws Exception { assertEquals(1, respBody.size()); // verify LedgerMetadata content is equal assertTrue(respBody.get(ledgerId.toString()).toString() - .equals(new String(new LedgerMetadataSerDe(LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION) - .serialize(lh[0].getLedgerMetadata())))); + .equals(new String(new LedgerMetadataSerDe().serialize(lh[0].getLedgerMetadata())))); } @Test diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index ffbbc08df0c..d571bf8057e 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -64,8 +64,19 @@ @Slf4j class EtcdLedgerManager implements LedgerManager { - private final LedgerMetadataSerDe serDe; - private final Function ledgerMetadataFunction; + private final LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + private final Function ledgerMetadataFunction = bs -> { + try { + return serDe.parseConfig( + bs.getBytes(), + Optional.empty() + ); + } catch (IOException ioe) { + log.error("Could not parse ledger metadata : {}", bs.toStringUtf8(), ioe); + throw new RuntimeException( + "Could not parse ledger metadata : " + bs.toStringUtf8(), ioe); + } + }; private final String scope; private final Client client; @@ -79,23 +90,11 @@ class EtcdLedgerManager implements LedgerManager { private volatile boolean closed = false; EtcdLedgerManager(Client client, - String scope, - int maxLedgerMetadataFormatVersion) { + String scope) { this.client = client; this.kvClient = client.getKVClient(); this.scope = scope; this.watchClient = new EtcdWatchClient(client); - this.serDe = new LedgerMetadataSerDe(maxLedgerMetadataFormatVersion); - - this.ledgerMetadataFunction = bs -> { - try { - return serDe.parseConfig(bs.getBytes(), Optional.empty()); - } catch (IOException ioe) { - log.error("Could not parse ledger metadata : {}", bs.toStringUtf8(), ioe); - throw new RuntimeException( - "Could not parse ledger metadata : " + bs.toStringUtf8(), ioe); - } - }; } private boolean isClosed() { diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java index 0bc16d815cd..dc7d1517b2b 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerFactory.java @@ -42,7 +42,6 @@ class EtcdLedgerManagerFactory implements LedgerManagerFactory { private String scope; private Client client; - private int maxLedgerMetadataFormatVersion; @Override public int getCurrentVersion() { @@ -52,8 +51,7 @@ public int getCurrentVersion() { @Override public LedgerManagerFactory initialize(AbstractConfiguration conf, LayoutManager layoutManager, - int factoryVersion, - int maxLedgerMetadataFormatVersion) throws IOException { + int factoryVersion) throws IOException { checkArgument(layoutManager instanceof EtcdLayoutManager); EtcdLayoutManager etcdLayoutManager = (EtcdLayoutManager) layoutManager; @@ -68,7 +66,6 @@ public LedgerManagerFactory initialize(AbstractConfiguration conf, throw new IOException("Invalid metadata service uri", e); } this.client = etcdLayoutManager.getClient(); - this.maxLedgerMetadataFormatVersion = maxLedgerMetadataFormatVersion; return this; } @@ -85,7 +82,7 @@ public LedgerIdGenerator newLedgerIdGenerator() { @Override public LedgerManager newLedgerManager() { - return new EtcdLedgerManager(client, scope, maxLedgerMetadataFormatVersion); + return new EtcdLedgerManager(client, scope); } @Override diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java index cbd6f321430..47b034c00ef 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdMetadataDriverBase.java @@ -27,7 +27,6 @@ import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.meta.LayoutManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; -import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.exceptions.Code; import org.apache.bookkeeper.meta.exceptions.MetadataException; import org.apache.bookkeeper.stats.StatsLogger; @@ -107,8 +106,7 @@ public synchronized LedgerManagerFactory getLedgerManagerFactory() if (null == lmFactory) { try { lmFactory = new EtcdLedgerManagerFactory(); - lmFactory.initialize(conf, layoutManager, EtcdLedgerManagerFactory.VERSION, - LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + lmFactory.initialize(conf, layoutManager, EtcdLedgerManagerFactory.VERSION); } catch (IOException ioe) { throw new MetadataException( Code.METADATA_SERVICE_ERROR, "Failed to initialize ledger manager factory", ioe); diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index 30877f7d5a5..984224c6b8b 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -55,7 +55,6 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; -import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.metadata.etcd.helpers.ValueStream; import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -82,8 +81,7 @@ public class EtcdLedgerManagerTest extends EtcdTestBase { public void setUp() throws Exception { super.setUp(); this.scope = RandomStringUtils.randomAlphabetic(8); - this.lm = new EtcdLedgerManager(etcdClient, scope, - LedgerMetadataSerDe.CURRENT_METADATA_FORMAT_VERSION); + this.lm = new EtcdLedgerManager(etcdClient, scope); } @Override diff --git a/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java b/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java index 26230ec7a0c..6432c303a28 100644 --- a/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java +++ b/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java @@ -179,7 +179,7 @@ private void testShadeLedgerManagerFactoryAllowed(String factoryClassName, when(manager.readLedgerLayout()).thenReturn(layout); LedgerManagerFactory factory = mock(LedgerManagerFactory.class); - when(factory.initialize(any(AbstractConfiguration.class), same(manager), anyInt(), anyInt())) + when(factory.initialize(any(AbstractConfiguration.class), same(manager), anyInt())) .thenReturn(factory); PowerMockito.mockStatic(ReflectionUtils.class); when(ReflectionUtils.newInstance(any(Class.class))) @@ -191,7 +191,7 @@ private void testShadeLedgerManagerFactoryAllowed(String factoryClassName, if (allowShaded) { assertSame(factory, result); verify(factory, times(1)) - .initialize(any(AbstractConfiguration.class), same(manager), anyInt(), anyInt()); + .initialize(any(AbstractConfiguration.class), same(manager), anyInt()); } else { fail("Should fail to instantiate ledger manager factory if allowShaded is false"); } From 83d182906a3590fa33a6e8f2f00108ea3d846c2e Mon Sep 17 00:00:00 2001 From: Samuel Just Date: Thu, 3 Jan 2019 16:41:37 -0800 Subject: [PATCH 0185/1642] ISSUE #1892: ILS: reset retry in consistency check loop This patch additionally: - modifies InterleavedLedgerStorageTest.java to test with and without entryLogPerLedger - refactors the test a bit to ensure that the gc calls really do race with the right part of the checker - addresses a few other more cosmetic errors (bug W-5721713) Signed-off-by: Samuel Just Reviewers: Enrico Olivelli This closes #1894 from athanatos/forupstream/wip-1892, closes #1892 --- .../bookie/BookKeeperServerStats.java | 1 + .../apache/bookkeeper/bookie/BookieShell.java | 2 +- .../apache/bookkeeper/bookie/EntryLogger.java | 10 +- .../bookie/InterleavedLedgerStorage.java | 31 +++- .../bookie/InterleavedLedgerStorageTest.java | 132 +++++++++++++++--- 5 files changed, 152 insertions(+), 24 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java index 69d0eda8661..cdafd15a991 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java @@ -111,6 +111,7 @@ public interface BookKeeperServerStats { // Ledger Storage Scrub Stats String STORAGE_SCRUB_PAGES_SCANNED = "STORAGE_SCRUB_PAGES_SCANNED"; + String STORAGE_SCRUB_PAGE_RETRIES = "STORAGE_SCRUB_PAGE_RETRIES"; // Ledger Cache Stats String LEDGER_CACHE_READ_PAGE = "LEDGER_CACHE_READ_PAGE"; 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 32e288b7e3b..f53195a3fae 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 @@ -1234,7 +1234,7 @@ Options getOptions() { } /** - * Print the metadata for a ledger. + * Check local storage for inconsistencies. */ class LocalConsistencyCheck extends MyCommand { Options lOpts = new Options(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index 6b41b6c958b..af82620bf76 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -696,6 +696,10 @@ static long logIdForOffset(long offset) { } + static long posForOffset(long location) { + return location & 0xffffffffL; + } + /** * Exception type for representing lookup errors. Useful for disambiguating different error @@ -778,7 +782,7 @@ private EntryLogEntry getFCForEntryInternal( throws EntryLookupException, IOException { ByteBuf sizeBuff = sizeBuffer.get(); sizeBuff.clear(); - pos -= 4; // we want to get the ledgerId and length to check + pos -= 4; // we want to get the entrySize as well as the ledgerId and entryId BufferedReadChannel fc; try { fc = getChannelForLogId(entryLogId); @@ -817,14 +821,14 @@ private EntryLogEntry getFCForEntryInternal( void checkEntry(long ledgerId, long entryId, long location) throws EntryLookupException, IOException { long entryLogId = logIdForOffset(location); - long pos = location & 0xffffffffL; + long pos = posForOffset(location); getFCForEntryInternal(ledgerId, entryId, entryLogId, pos); } public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) throws IOException, Bookie.NoEntryException { long entryLogId = logIdForOffset(location); - long pos = location & 0xffffffffL; + long pos = posForOffset(location); final EntryLogEntry entry; try { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java index 6078d6924ae..d1287c31927 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java @@ -29,6 +29,7 @@ import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_GET_ENTRY; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_GET_OFFSET; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_SCRUB_PAGES_SCANNED; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_SCRUB_PAGE_RETRIES; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; @@ -55,6 +56,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.proto.BookieProtocol; +import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; @@ -112,6 +114,7 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry ) private OpStatsLogger getEntryStats; private OpStatsLogger pageScanStats; + private Counter retryCounter; @VisibleForTesting public InterleavedLedgerStorage() { @@ -149,11 +152,34 @@ void initializeWithEntryLogListener(ServerConfiguration conf, Checkpointer checkpointer, EntryLogListener entryLogListener, StatsLogger statsLogger) throws IOException { + initializeWithEntryLogger( + conf, + ledgerManager, + ledgerDirsManager, + indexDirsManager, + stateManager, + checkpointSource, + checkpointer, + new EntryLogger(conf, ledgerDirsManager, entryLogListener, statsLogger.scope(ENTRYLOGGER_SCOPE)), + statsLogger); + } + + @VisibleForTesting + public void initializeWithEntryLogger(ServerConfiguration conf, + LedgerManager ledgerManager, + LedgerDirsManager ledgerDirsManager, + LedgerDirsManager indexDirsManager, + StateManager stateManager, + CheckpointSource checkpointSource, + Checkpointer checkpointer, + EntryLogger entryLogger, + StatsLogger statsLogger) throws IOException { checkNotNull(checkpointSource, "invalid null checkpoint source"); checkNotNull(checkpointer, "invalid null checkpointer"); + this.entryLogger = entryLogger; + this.entryLogger.addListener(this); this.checkpointSource = checkpointSource; this.checkpointer = checkpointer; - entryLogger = new EntryLogger(conf, ledgerDirsManager, entryLogListener, statsLogger.scope(ENTRYLOGGER_SCOPE)); ledgerCache = new LedgerCacheImpl(conf, activeLedgers, null == indexDirsManager ? ledgerDirsManager : indexDirsManager, statsLogger); gcThread = new GarbageCollectorThread(conf, ledgerManager, this, statsLogger.scope("gc")); @@ -162,6 +188,7 @@ void initializeWithEntryLogListener(ServerConfiguration conf, getOffsetStats = statsLogger.getOpStatsLogger(STORAGE_GET_OFFSET); getEntryStats = statsLogger.getOpStatsLogger(STORAGE_GET_ENTRY); pageScanStats = statsLogger.getOpStatsLogger(STORAGE_SCRUB_PAGES_SCANNED); + retryCounter = statsLogger.getCounter(STORAGE_SCRUB_PAGE_RETRIES); } private LedgerDirsListener getLedgerDirsListener() { @@ -538,6 +565,7 @@ public List localConsistencyCheck(Optional r @Cleanup LedgerEntryPage lep = page.getLEP(); MutableBoolean retry = new MutableBoolean(false); do { + retry.setValue(false); int version = lep.getVersion(); MutableBoolean success = new MutableBoolean(true); @@ -556,6 +584,7 @@ public List localConsistencyCheck(Optional r } else { LOG.debug("localConsistencyCheck: concurrent modification, retrying"); retry.setValue(true); + retryCounter.inc(); } return false; } else { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java index fd4f314c4ef..4fde8e74a6a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java @@ -20,6 +20,8 @@ */ package org.apache.bookkeeper.bookie; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_SCRUB_PAGE_RETRIES; import static org.junit.Assert.assertEquals; import io.netty.buffer.ByteBuf; @@ -27,9 +29,11 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Optional; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicLong; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -37,18 +41,37 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.util.DiskChecker; import org.apache.bookkeeper.util.EntryFormatter; import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.apache.commons.lang.mutable.MutableInt; import org.apache.commons.lang.mutable.MutableLong; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Test for InterleavedLedgerStorage. */ +@RunWith(Parameterized.class) public class InterleavedLedgerStorageTest { + private static final Logger LOG = LoggerFactory.getLogger(InterleavedLedgerStorageTest.class); + + @Parameterized.Parameters + public static Iterable elplSetting() { + return Arrays.asList(true, false); + } + + public InterleavedLedgerStorageTest(boolean elplSetting) { + conf.setEntryLogSizeLimit(2048); + conf.setEntryLogPerLedgerEnabled(elplSetting); + } CheckpointSource checkpointSource = new CheckpointSource() { @Override @@ -73,8 +96,41 @@ public void start() { } }; + static class TestableEntryLogger extends EntryLogger { + public interface CheckEntryListener { + void accept(long ledgerId, + long entryId, + long entryLogId, + long pos); + } + volatile CheckEntryListener testPoint; + + public TestableEntryLogger( + ServerConfiguration conf, + LedgerDirsManager ledgerDirsManager, + EntryLogListener listener, + StatsLogger statsLogger) throws IOException { + super(conf, ledgerDirsManager, listener, statsLogger); + } + + void setCheckEntryTestPoint(CheckEntryListener testPoint) throws InterruptedException { + this.testPoint = testPoint; + } + + @Override + void checkEntry(long ledgerId, long entryId, long location) throws EntryLookupException, IOException { + CheckEntryListener runBefore = testPoint; + if (runBefore != null) { + runBefore.accept(ledgerId, entryId, logIdForOffset(location), posForOffset(location)); + } + super.checkEntry(ledgerId, entryId, location); + } + } + + TestStatsProvider statsProvider = new TestStatsProvider(); ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); LedgerDirsManager ledgerDirsManager; + TestableEntryLogger entryLogger; InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); final long numWrites = 2000; final long entriesPerWrite = 2; @@ -87,14 +143,16 @@ public void setUp() throws Exception { File curDir = Bookie.getCurrentDirectory(tmpDir); Bookie.checkDirectoryStructure(curDir); - conf = TestBKConfiguration.newServerConfiguration(); - conf.setLedgerDirNames(new String[] { tmpDir.toString() }); - conf.setEntryLogSizeLimit(2048); + conf.setLedgerDirNames(new String[]{tmpDir.toString()}); ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); - interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, - null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE); + entryLogger = new TestableEntryLogger( + conf, ledgerDirsManager, null, NullStatsLogger.INSTANCE); + interleavedStorage.initializeWithEntryLogger( + conf, null, ledgerDirsManager, ledgerDirsManager, + null, checkpointSource, checkpointer, entryLogger, + statsProvider.getStatsLogger(BOOKIE_SCOPE)); // Insert some ledger & entries in the interleaved storage for (long entryId = 0; entryId < numWrites; entryId++) { @@ -132,41 +190,77 @@ public void testIndexEntryIterator() throws Exception { } @Test - public void testConsistencyCheckConcurrentModification() throws Exception { - AtomicBoolean done = new AtomicBoolean(false); - EntryLogger entryLogger = interleavedStorage.getEntryLogger(); - List asyncErrors = new ArrayList<>(); + public void testConsistencyCheckConcurrentGC() throws Exception { + final long signalDone = -1; + final List asyncErrors = new ArrayList<>(); + final LinkedBlockingQueue toCompact = new LinkedBlockingQueue<>(); + final Semaphore awaitingCompaction = new Semaphore(0); + + interleavedStorage.flush(); + final long lastLogId = entryLogger.getLeastUnflushedLogId(); + + final MutableInt counter = new MutableInt(0); + entryLogger.setCheckEntryTestPoint((ledgerId, entryId, entryLogId, pos) -> { + if (entryLogId < lastLogId) { + if (counter.intValue() % 100 == 0) { + try { + toCompact.put(entryLogId); + awaitingCompaction.acquire(); + } catch (InterruptedException e) { + asyncErrors.add(e); + } + } + counter.increment(); + } + }); + Thread mutator = new Thread(() -> { EntryLogCompactor compactor = new EntryLogCompactor( conf, entryLogger, interleavedStorage, entryLogger::removeEntryLog); - long next = 0; - while (!done.get()) { + while (true) { + Long next = null; try { + next = toCompact.take(); + if (next == null || next == signalDone) { + break; + } compactor.compact(entryLogger.getEntryLogMetadata(next)); - next++; - } catch (IOException e) { + } catch (BufferedChannelBase.BufferedChannelClosedException e) { + // next was already removed, ignore + } catch (Exception e) { asyncErrors.add(e); break; + } finally { + if (next != null) { + awaitingCompaction.release(); + } } } }); mutator.start(); - for (int i = 0; i < 100; ++i) { - assert interleavedStorage.localConsistencyCheck(Optional.empty()).size() == 0; - Thread.sleep(10); + List inconsistencies = interleavedStorage.localConsistencyCheck( + Optional.empty()); + for (LedgerStorage.DetectedInconsistency e: inconsistencies) { + LOG.error("Found: {}", e); } + Assert.assertEquals(0, inconsistencies.size()); - done.set(true); + toCompact.offer(signalDone); mutator.join(); for (Exception e: asyncErrors) { throw e; } - } + if (!conf.isEntryLogPerLedgerEnabled()) { + Assert.assertNotEquals( + 0, + statsProvider.getCounter(BOOKIE_SCOPE + "." + STORAGE_SCRUB_PAGE_RETRIES).get().longValue()); + } + } @Test public void testConsistencyMissingEntry() throws Exception { From b8ffcf9d85105c9e239a01d40d62e7e863a012b3 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 7 Jan 2019 15:41:52 -0800 Subject: [PATCH 0186/1642] Allow to configure BK for low latency busy-wait settings ### Motivation Added `enableBusyWait` configuration option to turn on CPU Affinity and busy wait on sockets and queues. Reviewers: Sijie Guo This closes #1812 from merlimat/configure-busy-poll --- bookkeeper-common/pom.xml | 5 ++ .../common/util/OrderedExecutor.java | 44 ++++++++-- .../common/util/OrderedScheduler.java | 3 +- .../org/apache/bookkeeper/bookie/Journal.java | 35 +++++++- .../apache/bookkeeper/client/BookKeeper.java | 32 ++----- .../conf/AbstractConfiguration.java | 38 +++++++++ .../bookkeeper/conf/ServerConfiguration.java | 22 +++++ .../bookkeeper/proto/BookieNettyServer.java | 58 ++++++++----- .../apache/bookkeeper/util/EventLoopUtil.java | 85 +++++++++++++++++++ conf/bk_server.conf | 9 ++ shaded/bookkeeper-server-shaded/pom.xml | 1 + shaded/distributedlog-core-shaded/pom.xml | 1 + site/_data/config/bk_server.yaml | 10 +++ 13 files changed, 289 insertions(+), 54 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EventLoopUtil.java diff --git a/bookkeeper-common/pom.xml b/bookkeeper-common/pom.xml index 89463353310..d2825a0988e 100644 --- a/bookkeeper-common/pom.xml +++ b/bookkeeper-common/pom.xml @@ -30,6 +30,11 @@ bookkeeper-stats-api ${project.parent.version} + + org.apache.bookkeeper + cpu-affinity + ${project.parent.version} + com.google.guava guava diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java index 220e1af9c02..a645d64646b 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; import java.util.Random; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -44,6 +45,8 @@ import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.collections.BlockingMpscQueue; +import org.apache.bookkeeper.common.util.affinity.CpuAffinity; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; @@ -68,6 +71,7 @@ @Slf4j public class OrderedExecutor implements ExecutorService { public static final int NO_TASK_LIMIT = -1; + private static final int DEFAULT_MAX_ARRAY_QUEUE_SIZE = 10_000; protected static final long WARN_TIME_MICRO_SEC_DEFAULT = TimeUnit.SECONDS.toMicros(1); final String name; @@ -80,7 +84,7 @@ public class OrderedExecutor implements ExecutorService { final boolean preserveMdcForTaskExecution; final long warnTimeMicroSec; final int maxTasksInQueue; - + final boolean enableBusyWait; public static Builder newBuilder() { return new Builder(); @@ -98,7 +102,7 @@ public OrderedExecutor build() { } return new OrderedExecutor(name, numThreads, threadFactory, statsLogger, traceTaskExecution, preserveMdcForTaskExecution, - warnTimeMicroSec, maxTasksInQueue); + warnTimeMicroSec, maxTasksInQueue, enableBusyWait); } } @@ -114,6 +118,7 @@ public abstract static class AbstractBuilder { protected boolean preserveMdcForTaskExecution = false; protected long warnTimeMicroSec = WARN_TIME_MICRO_SEC_DEFAULT; protected int maxTasksInQueue = NO_TASK_LIMIT; + protected boolean enableBusyWait = false; public AbstractBuilder name(String name) { this.name = name; @@ -155,6 +160,11 @@ public AbstractBuilder traceTaskWarnTimeMicroSec(long warnTimeMicroSec) { return this; } + public AbstractBuilder enableBusyWait(boolean enableBusyWait) { + this.enableBusyWait = enableBusyWait; + return this; + } + @SuppressWarnings("unchecked") public T build() { if (null == threadFactory) { @@ -168,7 +178,8 @@ public T build() { traceTaskExecution, preserveMdcForTaskExecution, warnTimeMicroSec, - maxTasksInQueue); + maxTasksInQueue, + enableBusyWait); } } @@ -277,7 +288,15 @@ public T call() throws Exception { } protected ThreadPoolExecutor createSingleThreadExecutor(ThreadFactory factory) { - return new ThreadPoolExecutor(1, 1, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(), factory); + BlockingQueue queue; + if (enableBusyWait) { + // Use queue with busy-wait polling strategy + queue = new BlockingMpscQueue<>(maxTasksInQueue > 0 ? maxTasksInQueue : DEFAULT_MAX_ARRAY_QUEUE_SIZE); + } else { + // By default, use regular JDK LinkedBlockingQueue + queue = new LinkedBlockingQueue<>(); + } + return new ThreadPoolExecutor(1, 1, 0L, TimeUnit.MILLISECONDS, queue, factory); } protected ExecutorService getBoundedExecutor(ThreadPoolExecutor executor) { @@ -361,12 +380,14 @@ public Future submit(Runnable task, T result) { */ protected OrderedExecutor(String baseName, int numThreads, ThreadFactory threadFactory, StatsLogger statsLogger, boolean traceTaskExecution, - boolean preserveMdcForTaskExecution, long warnTimeMicroSec, int maxTasksInQueue) { + boolean preserveMdcForTaskExecution, long warnTimeMicroSec, int maxTasksInQueue, + boolean enableBusyWait) { checkArgument(numThreads > 0); checkArgument(!StringUtils.isBlank(baseName)); this.maxTasksInQueue = maxTasksInQueue; this.warnTimeMicroSec = warnTimeMicroSec; + this.enableBusyWait = enableBusyWait; name = baseName; threads = new ExecutorService[numThreads]; threadIds = new long[numThreads]; @@ -374,12 +395,25 @@ protected OrderedExecutor(String baseName, int numThreads, ThreadFactory threadF ThreadPoolExecutor thread = createSingleThreadExecutor( new ThreadFactoryBuilder().setNameFormat(name + "-" + getClass().getSimpleName() + "-" + i + "-%d") .setThreadFactory(threadFactory).build()); + threads[i] = addExecutorDecorators(getBoundedExecutor(thread)); final int idx = i; try { threads[idx].submit(() -> { threadIds[idx] = Thread.currentThread().getId(); + + if (enableBusyWait) { + // Try to acquire 1 CPU core to the executor thread. If it fails we + // are just logging the error and continuing, falling back to + // non-isolated CPUs. + try { + CpuAffinity.acquireCore(); + } catch (Throwable t) { + log.warn("Failed to acquire CPU core for thread {}", Thread.currentThread().getName(), + t.getMessage(), t); + } + } }).get(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java index 3c1fef5f797..979b55ded9d 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java @@ -114,10 +114,9 @@ private OrderedScheduler(String baseName, long warnTimeMicroSec, int maxTasksInQueue) { super(baseName, numThreads, threadFactory, statsLogger, traceTaskExecution, - preserveMdcForTaskExecution, warnTimeMicroSec, maxTasksInQueue); + preserveMdcForTaskExecution, warnTimeMicroSec, maxTasksInQueue, false /* enableBusyWait */); } - @Override protected ScheduledThreadPoolExecutor createSingleThreadExecutor(ThreadFactory factory) { return new ScheduledThreadPoolExecutor(1, factory); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index a868536169f..dd47da8bdef 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -40,6 +40,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -47,7 +48,9 @@ import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException; import org.apache.bookkeeper.bookie.stats.JournalStats; +import org.apache.bookkeeper.common.collections.BlockingMpscQueue; import org.apache.bookkeeper.common.collections.RecyclableArrayList; +import org.apache.bookkeeper.common.util.affinity.CpuAffinity; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.stats.Counter; @@ -56,7 +59,6 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.IOUtils; import org.apache.bookkeeper.util.MathUtils; -import org.apache.bookkeeper.util.collections.GrowableArrayBlockingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -472,6 +474,15 @@ public ForceWriteThread(Thread threadToNotifyOnEx, boolean enableGroupForceWrite @Override public void run() { LOG.info("ForceWrite Thread started"); + + if (conf.isBusyWaitEnabled()) { + try { + CpuAffinity.acquireCore(); + } catch (Exception e) { + LOG.warn("Unable to acquire CPU core for Journal ForceWrite thread: {}", e.getMessage(), e); + } + } + boolean shouldForceWrite = true; int numReqInLastForceWrite = 0; while (running) { @@ -611,8 +622,8 @@ static void writePaddingBytes(JournalChannel jc, ByteBuf paddingBuffer, int jour private final ExecutorService cbThreadPool; // journal entry queue to commit - final BlockingQueue queue = new GrowableArrayBlockingQueue(); - final BlockingQueue forceWriteRequests = new GrowableArrayBlockingQueue(); + final BlockingQueue queue; + final BlockingQueue forceWriteRequests; volatile boolean running = true; private final LedgerDirsManager ledgerDirsManager; @@ -628,6 +639,16 @@ public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf, LedgerDirsManager ledgerDirsManager, StatsLogger statsLogger) { super("BookieJournal-" + conf.getBookiePort()); + + if (conf.isBusyWaitEnabled()) { + // To achieve lower latency, use busy-wait blocking queue implementation + queue = new BlockingMpscQueue<>(conf.getJournalQueueSize()); + forceWriteRequests = new BlockingMpscQueue<>(conf.getJournalQueueSize()); + } else { + queue = new ArrayBlockingQueue<>(conf.getJournalQueueSize()); + forceWriteRequests = new ArrayBlockingQueue<>(conf.getJournalQueueSize()); + } + this.ledgerDirsManager = ledgerDirsManager; this.conf = conf; this.journalDirectory = journalDirectory; @@ -906,6 +927,14 @@ public int getJournalQueueLength() { public void run() { LOG.info("Starting journal on {}", journalDirectory); + if (conf.isBusyWaitEnabled()) { + try { + CpuAffinity.acquireCore(); + } catch (Exception e) { + LOG.warn("Unable to acquire CPU core for Journal thread: {}", e.getMessage(), e); + } + } + RecyclableArrayList toFlush = entryListRecycler.newInstance(); int numEntriesToFlush = 0; ByteBuf lenBuff = Unpooled.buffer(4); 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 b8f703dc23a..0acf16fd75f 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 @@ -25,11 +25,11 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; + import io.netty.channel.EventLoopGroup; -import io.netty.channel.epoll.EpollEventLoopGroup; -import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.HashedWheelTimer; import io.netty.util.concurrent.DefaultThreadFactory; + import java.io.IOException; import java.net.URI; import java.util.Collections; @@ -39,9 +39,9 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; 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.IsClosedCallback; @@ -78,9 +78,9 @@ import org.apache.bookkeeper.proto.DataFormats; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.util.EventLoopUtil; import org.apache.bookkeeper.util.SafeRunnable; import org.apache.commons.configuration.ConfigurationException; -import org.apache.commons.lang.SystemUtils; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; @@ -404,6 +404,7 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo .traceTaskExecution(conf.getEnableTaskExecutionStats()) .preserveMdcForTaskExecution(conf.getPreserveMdcForTaskExecution()) .traceTaskWarnTimeMicroSec(conf.getTaskExecutionWarnTimeMicros()) + .enableBusyWait(conf.isBusyWaitEnabled()) .build(); // initialize stats logger @@ -434,7 +435,8 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo // initialize event loop group if (null == eventLoopGroup) { - this.eventLoopGroup = getDefaultEventLoopGroup(conf); + this.eventLoopGroup = EventLoopUtil.getClientEventLoopGroup(conf, + new DefaultThreadFactory("bookkeeper-io")); this.ownEventLoopGroup = true; } else { this.eventLoopGroup = eventLoopGroup; @@ -601,8 +603,7 @@ BookieWatcher getBookieWatcher() { return bookieWatcher; } - @VisibleForTesting - OrderedExecutor getMainWorkerPool() { + public OrderedExecutor getMainWorkerPool() { return mainWorkerPool; } @@ -1282,7 +1283,6 @@ public void asyncDeleteLedger(final long lId, final DeleteCallback cb, final Obj * @throws InterruptedException * @throws BKException */ - @SuppressWarnings("unchecked") public void deleteLedger(long lId) throws InterruptedException, BKException { CompletableFuture future = new CompletableFuture<>(); SyncDeleteCallback result = new SyncDeleteCallback(future); @@ -1406,22 +1406,6 @@ public void close() throws BKException, InterruptedException { this.metadataDriver.close(); } - static EventLoopGroup getDefaultEventLoopGroup(ClientConfiguration conf) { - ThreadFactory threadFactory = new DefaultThreadFactory("bookkeeper-io"); - final int numThreads = conf.getNumIOThreads(); - - if (SystemUtils.IS_OS_LINUX) { - try { - return new EpollEventLoopGroup(numThreads, threadFactory); - } catch (Throwable t) { - LOG.warn("Could not use Netty Epoll event loop for bookie server: {}", t.getMessage()); - return new NioEventLoopGroup(numThreads, threadFactory); - } - } else { - return new NioEventLoopGroup(numThreads, threadFactory); - } - } - @Override public CreateBuilder newCreateLedgerOp() { return new LedgerCreateOp.CreateBuilderImpl(this); 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 f12200fcaff..65d702b3b51 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 @@ -86,6 +86,8 @@ public abstract class AbstractConfiguration "storeSystemTimeAsLedgerUnderreplicatedMarkTime"; protected static final String STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME = "storeSystemTimeAsLedgerCreationTime"; + protected static final String ENABLE_BUSY_WAIT = "enableBusyWait"; + // Metastore settings, only being used when LEDGER_MANAGER_FACTORY_CLASS is MSLedgerManagerFactory protected static final String METASTORE_IMPL_CLASS = "metastoreImplClass"; protected static final String METASTORE_MAX_ENTRIES_PER_SCAN = "metastoreMaxEntriesPerScan"; @@ -879,6 +881,42 @@ public T setPreserveMdcForTaskExecution(boolean enabled) { return getThis(); } + /** + * Return whether the busy-wait is enabled for BookKeeper and Netty IO threads. + * + *

            Default is false + * + * @return the value of the option + */ + public boolean isBusyWaitEnabled() { + return getBoolean(ENABLE_BUSY_WAIT, false); + } + + /** + * Option to enable busy-wait settings. + * + *

            Default is false. + * + *

            WARNING: This option will enable spin-waiting on executors and IO threads + * in order to reduce latency during context switches. The spinning will + * consume 100% CPU even when bookie is not doing any work. It is + * recommended to reduce the number of threads in the main workers pool + * ({@link ClientConfiguration#setNumWorkerThreads(int)}) and Netty event + * loop {@link ClientConfiguration#setNumIOThreads(int)} to only have few + * CPU cores busy. + *

            + * + * @param busyWaitEanbled + * if enabled, use spin-waiting strategy to reduce latency in + * context switches + * + * @see #isBusyWaitEnabled() + */ + public T setBusyWaitEnabled(boolean busyWaitEanbled) { + setProperty(ENABLE_BUSY_WAIT, busyWaitEanbled); + return getThis(); + } + /** * Return the flag indicating whether to limit stats logging. * 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 1d3865189f2..64f6c4b1020 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 @@ -134,6 +134,7 @@ public class ServerConfiguration extends AbstractConfiguration newTaskQueue(int maxPendingTasks) { + if (conf.isBusyWaitEnabled()) { + return new BlockingMpscQueue<>(Math.min(maxPendingTasks, 10_000)); + } else { + return super.newTaskQueue(maxPendingTasks); + } + } + }; + } + }; + + // Enable CPU affinity on IO threads + if (conf.isBusyWaitEnabled()) { + for (int i = 0; i < conf.getServerNumIOThreads(); i++) { + jvmEventLoopGroup.next().submit(() -> { + try { + CpuAffinity.acquireCore(); + } catch (Throwable t) { + LOG.warn("Failed to acquire CPU core for thread {}", Thread.currentThread().getName(), + t.getMessage(), t); + } + }); + } + } + allChannels = new CleanupChannelGroup(jvmEventLoopGroup); } else { jvmEventLoopGroup = null; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EventLoopUtil.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EventLoopUtil.java new file mode 100644 index 00000000000..32cb2281ea2 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/EventLoopUtil.java @@ -0,0 +1,85 @@ +/** + * 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.util; + +import io.netty.channel.EventLoopGroup; +import io.netty.channel.SelectStrategy; +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; + +import java.util.concurrent.ThreadFactory; + +import lombok.experimental.UtilityClass; +import lombok.extern.slf4j.Slf4j; + +import org.apache.bookkeeper.common.util.affinity.CpuAffinity; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.commons.lang.SystemUtils; + + +/** + * Utility class to initialize Netty event loops. + */ +@Slf4j +@UtilityClass +public class EventLoopUtil { + public static EventLoopGroup getClientEventLoopGroup(ClientConfiguration conf, ThreadFactory threadFactory) { + return getEventLoopGroup(threadFactory, conf.getNumIOThreads(), conf.isBusyWaitEnabled()); + } + + public static EventLoopGroup getServerEventLoopGroup(ServerConfiguration conf, ThreadFactory threadFactory) { + return getEventLoopGroup(threadFactory, conf.getServerNumIOThreads(), conf.isBusyWaitEnabled()); + } + + private static EventLoopGroup getEventLoopGroup(ThreadFactory threadFactory, + int numThreads, boolean enableBusyWait) { + if (!SystemUtils.IS_OS_LINUX) { + return new NioEventLoopGroup(numThreads, threadFactory); + } + + try { + if (!enableBusyWait) { + // Regular Epoll based event loop + return new EpollEventLoopGroup(numThreads, threadFactory); + } + + // With low latency setting, put the Netty event loop on busy-wait loop to reduce cost of + // context switches + EpollEventLoopGroup eventLoopGroup = new EpollEventLoopGroup(numThreads, threadFactory, + () -> (selectSupplier, hasTasks) -> SelectStrategy.BUSY_WAIT); + + // Enable CPU affinity on IO threads + for (int i = 0; i < numThreads; i++) { + eventLoopGroup.next().submit(() -> { + try { + CpuAffinity.acquireCore(); + } catch (Throwable t) { + log.warn("Failed to acquire CPU core for thread {}", Thread.currentThread().getName(), + t.getMessage(), t); + } + }); + } + + return eventLoopGroup; + } catch (ExceptionInInitializerError | NoClassDefFoundError | UnsatisfiedLinkError e) { + log.warn("Could not use Netty Epoll event loop: {}", e.getMessage()); + return new NioEventLoopGroup(numThreads, threadFactory); + } + } +} diff --git a/conf/bk_server.conf b/conf/bk_server.conf index df6c9916ca3..23012dc4dd8 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -133,6 +133,12 @@ extraServerComponents= # avoid the executor queue to grow indefinitely # maxPendingAddRequestsPerThread=10000 +# Option to enable busy-wait settings. Default is false. +# WARNING: This option will enable spin-waiting on executors and IO threads in order to reduce latency during +# context switches. The spinning will consume 100% CPU even when bookie is not doing any work. It is recommended to +# reduce the number of threads in the main workers pool and Netty event loop to only have few CPU cores busy. +# enableBusyWait=false + ############################################################################# ## Long poll request parameter settings ############################################################################# @@ -332,6 +338,9 @@ journalDirectories=/tmp/bk-txn # If we should flush the journal when journal queue is empty # journalFlushWhenQueueEmpty=false +# Set the size of the journal queue. +# journalQueueSize=10000 + ############################################################################# ## Ledger storage settings ############################################################################# diff --git a/shaded/bookkeeper-server-shaded/pom.xml b/shaded/bookkeeper-server-shaded/pom.xml index 820dc01a189..a39227bea10 100644 --- a/shaded/bookkeeper-server-shaded/pom.xml +++ b/shaded/bookkeeper-server-shaded/pom.xml @@ -66,6 +66,7 @@ com.google.guava:guava com.google.protobuf:protobuf-java org.apache.bookkeeper:bookkeeper-common + org.apache.bookkeeper:cpu-affinity org.apache.bookkeeper:bookkeeper-tools-framework org.apache.bookkeeper:bookkeeper-proto org.apache.bookkeeper:bookkeeper-server diff --git a/shaded/distributedlog-core-shaded/pom.xml b/shaded/distributedlog-core-shaded/pom.xml index 4f5c9f2c5c5..e5ef59f80b3 100644 --- a/shaded/distributedlog-core-shaded/pom.xml +++ b/shaded/distributedlog-core-shaded/pom.xml @@ -87,6 +87,7 @@ net.java.dev.jna:jna net.jpountz.lz4:lz4 org.apache.bookkeeper:bookkeeper-common + org.apache.bookkeeper:cpu-affinity org.apache.bookkeeper:bookkeeper-tools-framework org.apache.bookkeeper:bookkeeper-proto org.apache.bookkeeper:bookkeeper-server diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index a7115985df6..a42e12eb2b7 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -71,6 +71,13 @@ groups: - param: maxPendingReadRequestsPerThread description: If add worker threads are enabled, limit the number of pending requests, to avoid the executor queue to grow indefinitely. If zero or negative, the number of pending requests is unlimited. default: 10000 + - param: enableBusyWait + description: | + Option to enable busy-wait settings. Default is false. + WARNING: This option will enable spin-waiting on executors and IO threads in order to reduce latency during + context switches. The spinning will consume 100% CPU even when bookie is not doing any work. It is recommended to + reduce the number of threads in the main workers pool and Netty event loop to only have few CPU cores busy. + default: false - name: Long poll settings params: @@ -248,6 +255,9 @@ groups: - param: journalFlushWhenQueueEmpty description: If we should flush the journal when journal queue is empty. default: 'false' + - param: journalQueueSize + description: Set the size of the journal queue. + default: 10000 - name: Ledger storage settings params: From 369847476f65d4f7c8c2cbb8495479f392c592ca Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 8 Jan 2019 12:16:09 -0800 Subject: [PATCH 0187/1642] Configure Netty allocator in bookie and client ### Motivation This is based on #1754. Adding the code to configure and use the allocator wrapper in bookie and client. (I'll rebase once the first PR is merged) Reviewers: Ivan Kelly , Enrico Olivelli , Sijie Guo This closes #1755 from merlimat/use-allocator --- .../bookkeeper/benchmark/BenchBookie.java | 4 +- .../allocator/impl/ByteBufAllocatorImpl.java | 68 +++++++++---- .../impl/ByteBufAllocatorBuilderTest.java | 12 +-- bookkeeper-server/pom.xml | 5 + .../org/apache/bookkeeper/bookie/Bookie.java | 28 ++++-- .../apache/bookkeeper/bookie/BookieShell.java | 10 +- .../bookkeeper/bookie/BufferedChannel.java | 15 +-- .../apache/bookkeeper/bookie/EntryLogger.java | 25 ++--- .../bookie/EntryLoggerAllocator.java | 8 +- .../bookie/InterleavedLedgerStorage.java | 14 ++- .../org/apache/bookkeeper/bookie/Journal.java | 15 ++- .../bookkeeper/bookie/LedgerStorage.java | 5 +- .../bookkeeper/bookie/ReadOnlyBookie.java | 6 +- .../bookie/SlowBufferedChannel.java | 10 +- .../bookie/SortedLedgerStorage.java | 8 +- .../bookie/storage/ldb/DbLedgerStorage.java | 10 +- .../bookie/storage/ldb/ReadCache.java | 10 +- .../ldb/SingleDirectoryDbLedgerStorage.java | 15 +-- .../bookie/storage/ldb/WriteCache.java | 11 ++- .../apache/bookkeeper/client/BookKeeper.java | 49 +++++++-- .../bookkeeper/client/ClientContext.java | 3 + .../bookkeeper/client/LedgerHandle.java | 2 +- .../client/api/BookKeeperBuilder.java | 11 +++ .../client/impl/BookKeeperBuilderImpl.java | 7 ++ .../conf/AbstractConfiguration.java | 99 +++++++++++++++++++ .../bookkeeper/conf/ClientConfiguration.java | 8 ++ .../bookkeeper/proto/BookieClientImpl.java | 20 ++-- .../bookkeeper/proto/BookieNettyServer.java | 9 +- .../proto/BookieRequestProcessor.java | 11 ++- .../apache/bookkeeper/proto/BookieServer.java | 36 +++++-- .../proto/PerChannelBookieClient.java | 20 ++-- .../proto/checksum/CRC32CDigestManager.java | 7 +- .../proto/checksum/CRC32DigestManager.java | 5 +- .../proto/checksum/DigestManager.java | 31 +++--- .../proto/checksum/DummyDigestManager.java | 5 +- .../proto/checksum/MacDigestManager.java | 5 +- .../tls/SecurityHandlerFactory.java | 3 +- .../bookkeeper/tls/TLSContextFactory.java | 10 +- .../apache/bookkeeper/util/ByteBufList.java | 3 +- .../bookie/BookieInitializationTest.java | 9 +- .../bookie/BufferedChannelTest.java | 6 +- .../bookkeeper/bookie/CompactionTest.java | 26 +++-- .../bookkeeper/bookie/CreateNewLogTest.java | 8 +- .../bookkeeper/bookie/EntryLogTest.java | 6 +- .../bookie/IndexPersistenceMgrTest.java | 4 +- .../bookie/InterleavedLedgerStorageTest.java | 4 +- .../bookkeeper/bookie/LedgerCacheTest.java | 7 +- .../bookkeeper/bookie/LedgerStorageTest.java | 8 +- .../bookie/SlowInterleavedLedgerStorage.java | 10 +- .../SortedLedgerStorageCheckpointTest.java | 5 +- .../bookkeeper/bookie/StateManagerTest.java | 6 +- .../bookkeeper/bookie/TestSyncThread.java | 5 +- .../storage/ldb/ConversionRollbackTest.java | 5 +- .../bookie/storage/ldb/ConversionTest.java | 8 +- .../ldb/DbLedgerStorageWriteCacheTest.java | 11 ++- .../ldb/LocationsIndexRebuildTest.java | 5 +- .../bookie/storage/ldb/ReadCacheTest.java | 7 +- .../bookie/storage/ldb/WriteCacheTest.java | 23 +++-- .../bookkeeper/client/BookKeeperTest.java | 1 + .../client/BookKeeperTestClient.java | 4 +- .../apache/bookkeeper/client/ClientUtil.java | 4 +- .../client/MockBookKeeperTestCase.java | 10 +- .../bookkeeper/client/MockClientContext.java | 15 +++ .../ReadLastConfirmedAndEntryOpTest.java | 4 +- .../client/TestGetBookieInfoTimeout.java | 4 +- .../bookkeeper/conf/TestBKConfiguration.java | 2 + .../apache/bookkeeper/meta/GcLedgersTest.java | 6 +- .../meta/LedgerManagerTestCase.java | 6 +- .../proto/BookieBackpressureTest.java | 10 +- .../bookkeeper/proto/MockBookieClient.java | 4 +- .../proto/TestBookieRequestProcessor.java | 9 +- .../test/BookKeeperClusterTestCase.java | 9 +- .../bookkeeper/test/BookieClientTest.java | 13 +-- conf/bk_server.conf | 50 ++++++++++ .../proto/checksum/DigestTypeBenchmark.java | 7 +- shaded/bookkeeper-server-shaded/pom.xml | 3 +- shaded/distributedlog-core-shaded/pom.xml | 3 +- site/_data/config/bk_server.yaml | 59 ++++++++++- 78 files changed, 763 insertions(+), 246 deletions(-) diff --git a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java index 216aff9890b..94776f0443c 100644 --- a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java +++ b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java @@ -20,6 +20,7 @@ package org.apache.bookkeeper.benchmark; import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.channel.EventLoopGroup; import io.netty.channel.epoll.EpollEventLoopGroup; @@ -175,7 +176,8 @@ public static void main(String[] args) new DefaultThreadFactory("BookKeeperClientScheduler")); ClientConfiguration conf = new ClientConfiguration(); - BookieClient bc = new BookieClientImpl(conf, eventLoop, executor, scheduler, NullStatsLogger.INSTANCE); + BookieClient bc = new BookieClientImpl(conf, eventLoop, PooledByteBufAllocator.DEFAULT, executor, scheduler, + NullStatsLogger.INSTANCE); LatencyCallback lc = new LatencyCallback(); ThroughputCallback tc = new ThroughputCallback(); diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java index 35441659a05..1889eb9b057 100644 --- a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorImpl.java @@ -39,6 +39,10 @@ public class ByteBufAllocatorImpl extends AbstractByteBufAllocator implements By private static final Logger log = LoggerFactory.getLogger(ByteBufAllocatorImpl.class); + // Same as AbstractByteBufAllocator, but copied here since it's not visible + private static final int DEFAULT_INITIAL_CAPACITY = 256; + private static final int DEFAULT_MAX_CAPACITY = Integer.MAX_VALUE; + private final ByteBufAllocator pooledAllocator; private final ByteBufAllocator unpooledAllocator; private final PoolingPolicy poolingPolicy; @@ -63,16 +67,22 @@ public class ByteBufAllocatorImpl extends AbstractByteBufAllocator implements By if (poolingPolicy == PoolingPolicy.PooledDirect) { if (pooledAllocator == null) { - this.pooledAllocator = new PooledByteBufAllocator( - true /* preferDirect */, - poolingConcurrency /* nHeapArena */, - poolingConcurrency /* nDirectArena */, - PooledByteBufAllocator.defaultPageSize(), - PooledByteBufAllocator.defaultMaxOrder(), - PooledByteBufAllocator.defaultTinyCacheSize(), - PooledByteBufAllocator.defaultSmallCacheSize(), - PooledByteBufAllocator.defaultNormalCacheSize(), - PooledByteBufAllocator.defaultUseCacheForAllThreads()); + if (poolingConcurrency == PooledByteBufAllocator.defaultNumDirectArena()) { + // If all the parameters are the same as in the default Netty pool, + // just reuse the static instance as the underlying allocator. + this.pooledAllocator = PooledByteBufAllocator.DEFAULT; + } else { + this.pooledAllocator = new PooledByteBufAllocator( + true /* preferDirect */, + poolingConcurrency /* nHeapArena */, + poolingConcurrency /* nDirectArena */, + PooledByteBufAllocator.defaultPageSize(), + PooledByteBufAllocator.defaultMaxOrder(), + PooledByteBufAllocator.defaultTinyCacheSize(), + PooledByteBufAllocator.defaultSmallCacheSize(), + PooledByteBufAllocator.defaultNormalCacheSize(), + PooledByteBufAllocator.defaultUseCacheForAllThreads()); + } } else { this.pooledAllocator = pooledAllocator; } @@ -109,6 +119,25 @@ public class ByteBufAllocatorImpl extends AbstractByteBufAllocator implements By } } + @Override + public ByteBuf buffer() { + return buffer(DEFAULT_INITIAL_CAPACITY); + } + + @Override + public ByteBuf buffer(int initialCapacity) { + return buffer(initialCapacity, DEFAULT_MAX_CAPACITY); + } + + @Override + public ByteBuf buffer(int initialCapacity, int maxCapacity) { + if (poolingPolicy == PoolingPolicy.PooledDirect) { + return newDirectBuffer(initialCapacity, maxCapacity, true /* can fallback to heap if needed */); + } else { + return newHeapBuffer(initialCapacity, maxCapacity); + } + } + @Override protected ByteBuf newHeapBuffer(int initialCapacity, int maxCapacity) { try { @@ -125,30 +154,33 @@ protected ByteBuf newHeapBuffer(int initialCapacity, int maxCapacity) { @Override protected ByteBuf newDirectBuffer(int initialCapacity, int maxCapacity) { + // If caller asked specifically for a direct buffer, we cannot fallback to heap + return newDirectBuffer(initialCapacity, maxCapacity, false); + } + + private ByteBuf newDirectBuffer(int initialCapacity, int maxCapacity, boolean canFallbackToHeap) { if (poolingPolicy == PoolingPolicy.PooledDirect) { try { return pooledAllocator.directBuffer(initialCapacity, maxCapacity); } catch (OutOfMemoryError e) { - switch (outOfMemoryPolicy) { - case ThrowException: - outOfMemoryListener.accept(e); - throw e; - - case FallbackToHeap: + if (canFallbackToHeap && outOfMemoryPolicy == OutOfMemoryPolicy.FallbackToHeap) { try { return unpooledAllocator.heapBuffer(initialCapacity, maxCapacity); } catch (OutOfMemoryError e2) { outOfMemoryListener.accept(e2); throw e2; } + } else { + // ThrowException + outOfMemoryListener.accept(e); + throw e; } - return null; } } else { // Unpooled heap buffer. Force heap buffers because unpooled direct // buffers have very high overhead of allocation/reclaiming try { - return unpooledAllocator.heapBuffer(initialCapacity, maxCapacity); + return unpooledAllocator.directBuffer(initialCapacity, maxCapacity); } catch (OutOfMemoryError e) { outOfMemoryListener.accept(e); throw e; diff --git a/bookkeeper-common-allocator/src/test/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderTest.java b/bookkeeper-common-allocator/src/test/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderTest.java index 8ff66c33171..662dd832c47 100644 --- a/bookkeeper-common-allocator/src/test/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderTest.java +++ b/bookkeeper-common-allocator/src/test/java/org/apache/bookkeeper/common/allocator/impl/ByteBufAllocatorBuilderTest.java @@ -145,10 +145,10 @@ public void testOomWithFallbackAndNoMoreHeap() { } @Test - public void testOomUnpooled() { + public void testOomUnpooledDirect() { ByteBufAllocator heapAlloc = mock(ByteBufAllocator.class); - OutOfMemoryError noHeapError = new OutOfMemoryError("no more heap"); - when(heapAlloc.heapBuffer(anyInt(), anyInt())).thenThrow(noHeapError); + OutOfMemoryError noMemError = new OutOfMemoryError("no more direct mem"); + when(heapAlloc.directBuffer(anyInt(), anyInt())).thenThrow(noMemError); AtomicReference receivedException = new AtomicReference<>(); @@ -166,11 +166,11 @@ public void testOomUnpooled() { fail("Should have thrown exception"); } catch (OutOfMemoryError e) { // Expected - assertEquals(noHeapError, e); + assertEquals(noMemError, e); } // Ensure the notification was triggered even when exception is thrown - assertEquals(noHeapError, receivedException.get()); + assertEquals(noMemError, receivedException.get()); } @Test @@ -214,7 +214,7 @@ public void testUnpooled() { ByteBuf buf2 = alloc.directBuffer(); assertEquals(UnpooledByteBufAllocator.DEFAULT, buf2.alloc()); - assertTrue(buf2.hasArray()); + assertFalse(buf2.hasArray()); } @Test diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index 10b5ea41486..c1b684eea60 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -30,6 +30,11 @@ bookkeeper-common ${project.parent.version}
            + + org.apache.bookkeeper + bookkeeper-common-allocator + ${project.parent.version} + org.apache.bookkeeper bookkeeper-proto 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 fd352d8e71c..a0acd31c43d 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 @@ -28,9 +28,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.util.concurrent.SettableFuture; + import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.File; import java.io.FileNotFoundException; import java.io.FilenameFilter; @@ -53,6 +57,7 @@ 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; @@ -136,6 +141,8 @@ public class Bookie extends BookieCriticalThread { final StatsLogger statsLogger; private final BookieStats bookieStats; + private final ByteBufAllocator allocator; + /** * Exception is thrown when no such a ledger is found in this bookie. */ @@ -600,7 +607,7 @@ public static File[] getCurrentDirectories(File[] dirs) { public Bookie(ServerConfiguration conf) throws IOException, InterruptedException, BookieException { - this(conf, NullStatsLogger.INSTANCE); + this(conf, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT); } private static LedgerStorage buildLedgerStorage(ServerConfiguration conf) throws IOException { @@ -658,12 +665,13 @@ public void checkpointComplete(Checkpoint checkpoint, boolean compact) null, checkpointSource, checkpointer, - statsLogger); + statsLogger, + UnpooledByteBufAllocator.DEFAULT); return ledgerStorage; } - public Bookie(ServerConfiguration conf, StatsLogger statsLogger) + public Bookie(ServerConfiguration conf, StatsLogger statsLogger, ByteBufAllocator allocator) throws IOException, InterruptedException, BookieException { super("Bookie-" + conf.getBookiePort()); this.statsLogger = statsLogger; @@ -676,6 +684,7 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger) this.ledgerDirsManager = createLedgerDirsManager(conf, diskChecker, statsLogger.scope(LD_LEDGER_SCOPE)); this.indexDirsManager = createIndexDirsManager(conf, diskChecker, statsLogger.scope(LD_INDEX_SCOPE), this.ledgerDirsManager); + this.allocator = allocator; // instantiate zookeeper client to initialize ledger manager this.metadataDriver = instantiateMetadataDriver(conf); @@ -732,7 +741,7 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger) journals = Lists.newArrayList(); for (int i = 0; i < journalDirectories.size(); i++) { journals.add(new Journal(i, journalDirectories.get(i), - conf, ledgerDirsManager, statsLogger.scope(JOURNAL_SCOPE))); + conf, ledgerDirsManager, statsLogger.scope(JOURNAL_SCOPE), allocator)); } this.entryLogPerLedgerEnabled = conf.isEntryLogPerLedgerEnabled(); @@ -786,7 +795,8 @@ public void start() { stateManager, checkpointSource, syncThread, - statsLogger); + statsLogger, + allocator); handles = new HandleFactoryImpl(ledgerStorage); @@ -1287,8 +1297,8 @@ public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] } } - static ByteBuf createExplicitLACEntry(long ledgerId, ByteBuf explicitLac) { - ByteBuf bb = PooledByteBufAllocator.DEFAULT.directBuffer(8 + 8 + 4 + explicitLac.capacity()); + private ByteBuf createExplicitLACEntry(long ledgerId, ByteBuf explicitLac) { + ByteBuf bb = allocator.directBuffer(8 + 8 + 4 + explicitLac.capacity()); bb.writeLong(ledgerId); bb.writeLong(METAENTRY_ID_LEDGER_EXPLICITLAC); bb.writeInt(explicitLac.capacity()); @@ -1485,6 +1495,10 @@ public synchronized void waitZero() throws InterruptedException { } } + public ByteBufAllocator getAllocator() { + return allocator; + } + /** * Format the bookie server data. * 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 f53195a3fae..dc9137e6a1a 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 @@ -29,7 +29,9 @@ import com.google.common.util.concurrent.UncheckedExecutionException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; +import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.concurrent.DefaultThreadFactory; @@ -917,8 +919,8 @@ int runCmd(CommandLine cmdLine) throws Exception { ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor( new DefaultThreadFactory("BookKeeperClientSchedulerPool")); - BookieClient bookieClient = new BookieClientImpl(conf, eventLoopGroup, executor, - scheduler, NullStatsLogger.INSTANCE); + BookieClient bookieClient = new BookieClientImpl(conf, eventLoopGroup, + UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE); LongStream.range(firstEntry, lastEntry).forEach(entryId -> { CompletableFuture future = new CompletableFuture<>(); @@ -2825,9 +2827,9 @@ public void start() { }; dbStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager, null, - checkpointSource, checkpointer, NullStatsLogger.INSTANCE); + checkpointSource, checkpointer, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT); interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager, - null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE); + null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT); LedgerCache interleavedLedgerCache = interleavedStorage.ledgerCache; int convertedLedgers = 0; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java index 633c5400dab..31fb2035ea9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java @@ -66,23 +66,24 @@ public class BufferedChannel extends BufferedReadChannel implements Closeable { private boolean closed = false; // make constructor to be public for unit test - public BufferedChannel(FileChannel fc, int capacity) throws IOException { + public BufferedChannel(ByteBufAllocator allocator, FileChannel fc, int capacity) throws IOException { // Use the same capacity for read and write buffers. - this(fc, capacity, 0L); + this(allocator, fc, capacity, 0L); } - public BufferedChannel(FileChannel fc, int capacity, long unpersistedBytesBound) throws IOException { + public BufferedChannel(ByteBufAllocator allocator, FileChannel fc, int capacity, long unpersistedBytesBound) + throws IOException { // Use the same capacity for read and write buffers. - this(fc, capacity, capacity, unpersistedBytesBound); + this(allocator, fc, capacity, capacity, unpersistedBytesBound); } - public BufferedChannel(FileChannel fc, int writeCapacity, int readCapacity, long unpersistedBytesBound) - throws IOException { + public BufferedChannel(ByteBufAllocator allocator, FileChannel fc, int writeCapacity, int readCapacity, + long unpersistedBytesBound) throws IOException { super(fc, readCapacity); this.writeCapacity = writeCapacity; this.position = new AtomicLong(fc.position()); this.writeBufferStartPosition.set(position.get()); - this.writeBuffer = ByteBufAllocator.DEFAULT.directBuffer(writeCapacity); + this.writeBuffer = allocator.directBuffer(writeCapacity); this.unpersistedBytes = new AtomicLong(0); this.unpersistedBytesBound = unpersistedBytesBound; this.doRegularFlushes = unpersistedBytesBound > 0; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index af82620bf76..1389370a461 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -88,9 +88,9 @@ static class BufferedLogChannel extends BufferedChannel { private final File logFile; private long ledgerIdAssigned = UNASSIGNED_LEDGERID; - public BufferedLogChannel(FileChannel fc, int writeCapacity, int readCapacity, long logId, File logFile, - long unpersistedBytesBound) throws IOException { - super(fc, writeCapacity, readCapacity, unpersistedBytesBound); + public BufferedLogChannel(ByteBufAllocator allocator, FileChannel fc, int writeCapacity, int readCapacity, + long logId, File logFile, long unpersistedBytesBound) throws IOException { + super(allocator, fc, writeCapacity, readCapacity, unpersistedBytesBound); this.logId = logId; this.entryLogMetadata = new EntryLogMetadata(logId); this.logFile = logFile; @@ -283,6 +283,8 @@ private static class Header { private final int maxSaneEntrySize; + private final ByteBufAllocator allocator; + final ServerConfiguration conf; /** * Scan entries in a entry log file. @@ -332,15 +334,16 @@ public EntryLogger(ServerConfiguration conf) throws IOException { */ public EntryLogger(ServerConfiguration conf, LedgerDirsManager ledgerDirsManager) throws IOException { - this(conf, ledgerDirsManager, null, NullStatsLogger.INSTANCE); + this(conf, ledgerDirsManager, null, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT); } public EntryLogger(ServerConfiguration conf, - LedgerDirsManager ledgerDirsManager, EntryLogListener listener, StatsLogger statsLogger) - throws IOException { + LedgerDirsManager ledgerDirsManager, EntryLogListener listener, StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { //We reserve 500 bytes as overhead for the protocol. This is not 100% accurate // but the protocol varies so an exact value is difficult to determine this.maxSaneEntrySize = conf.getNettyMaxFrameSizeBytes() - 500; + this.allocator = allocator; this.ledgerDirsManager = ledgerDirsManager; this.conf = conf; entryLogPerLedgerEnabled = conf.isEntryLogPerLedgerEnabled(); @@ -371,7 +374,7 @@ public EntryLogger(ServerConfiguration conf, } this.recentlyCreatedEntryLogsStatus = new RecentEntryLogsStatus(logId + 1); this.entryLoggerAllocator = new EntryLoggerAllocator(conf, ledgerDirsManager, recentlyCreatedEntryLogsStatus, - logId); + logId, allocator); if (entryLogPerLedgerEnabled) { this.entryLogManager = new EntryLogManagerForEntryLogPerLedger(conf, ledgerDirsManager, entryLoggerAllocator, listeners, recentlyCreatedEntryLogsStatus, statsLogger); @@ -840,7 +843,7 @@ public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) throw new IOException(e.toString()); } - ByteBuf data = PooledByteBufAllocator.DEFAULT.directBuffer(entry.entrySize, entry.entrySize); + ByteBuf data = allocator.buffer(entry.entrySize, entry.entrySize); int rc = readFromLogChannel(entryLogId, entry.fc, data, pos); if (rc != entry.entrySize) { // Note that throwing NoEntryException here instead of IOException is not @@ -872,7 +875,7 @@ private Header getHeaderForLogId(long entryLogId) throws IOException { BufferedReadChannel bc = getChannelForLogId(entryLogId); // Allocate buffer to read (version, ledgersMapOffset, ledgerCount) - ByteBuf headers = PooledByteBufAllocator.DEFAULT.directBuffer(LOGFILE_HEADER_SIZE); + ByteBuf headers = allocator.directBuffer(LOGFILE_HEADER_SIZE); try { bc.read(headers, 0); @@ -988,7 +991,7 @@ public void scanEntryLog(long entryLogId, EntryLogScanner scanner) throws IOExce long pos = LOGFILE_HEADER_SIZE; // Start with a reasonably sized buffer size - ByteBuf data = PooledByteBufAllocator.DEFAULT.directBuffer(1024 * 1024); + ByteBuf data = allocator.directBuffer(1024 * 1024); try { @@ -1070,7 +1073,7 @@ EntryLogMetadata extractEntryLogMetadataFromIndex(long entryLogId) throws IOExce EntryLogMetadata meta = new EntryLogMetadata(entryLogId); final int maxMapSize = LEDGERS_MAP_HEADER_SIZE + LEDGERS_MAP_ENTRY_SIZE * LEDGERS_MAP_MAX_BATCH_SIZE; - ByteBuf ledgersMap = ByteBufAllocator.DEFAULT.directBuffer(maxMapSize); + ByteBuf ledgersMap = allocator.directBuffer(maxMapSize); try { while (offset < bc.size()) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java index 3ddd8e2e738..1c32b55de03 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java @@ -25,6 +25,7 @@ import static org.apache.bookkeeper.bookie.TransactionalEntryLogCompactor.COMPACTING_SUFFIX; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import java.io.BufferedWriter; @@ -61,11 +62,14 @@ class EntryLoggerAllocator { private final Object createCompactionLogLock = new Object(); private final EntryLogger.RecentEntryLogsStatus recentlyCreatedEntryLogsStatus; private final boolean entryLogPreAllocationEnabled; + private final ByteBufAllocator byteBufAllocator; final ByteBuf logfileHeader = Unpooled.buffer(EntryLogger.LOGFILE_HEADER_SIZE); EntryLoggerAllocator(ServerConfiguration conf, LedgerDirsManager ledgerDirsManager, - EntryLogger.RecentEntryLogsStatus recentlyCreatedEntryLogsStatus, long logId) { + EntryLogger.RecentEntryLogsStatus recentlyCreatedEntryLogsStatus, long logId, + ByteBufAllocator byteBufAllocator) { this.conf = conf; + this.byteBufAllocator = byteBufAllocator; this.ledgerDirsManager = ledgerDirsManager; this.preallocatedLogId = logId; this.recentlyCreatedEntryLogsStatus = recentlyCreatedEntryLogsStatus; @@ -161,7 +165,7 @@ private synchronized BufferedLogChannel allocateNewLog(File dirForNextEntryLog, File newLogFile = new File(dirForNextEntryLog, logFileName); FileChannel channel = new RandomAccessFile(newLogFile, "rw").getChannel(); - BufferedLogChannel logChannel = new BufferedLogChannel(channel, conf.getWriteBufferBytes(), + BufferedLogChannel logChannel = new BufferedLogChannel(byteBufAllocator, channel, conf.getWriteBufferBytes(), conf.getReadBufferBytes(), preallocatedLogId, newLogFile, conf.getFlushIntervalInBytes()); logfileHeader.readerIndex(0); logChannel.write(logfileHeader); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java index d1287c31927..3b5bf0114df 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java @@ -35,6 +35,8 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.RateLimiter; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; + import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -129,7 +131,8 @@ public void initialize(ServerConfiguration conf, StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, - StatsLogger statsLogger) + StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { initializeWithEntryLogListener( conf, @@ -140,7 +143,8 @@ public void initialize(ServerConfiguration conf, checkpointSource, checkpointer, this, - statsLogger); + statsLogger, + allocator); } void initializeWithEntryLogListener(ServerConfiguration conf, @@ -151,7 +155,8 @@ void initializeWithEntryLogListener(ServerConfiguration conf, CheckpointSource checkpointSource, Checkpointer checkpointer, EntryLogListener entryLogListener, - StatsLogger statsLogger) throws IOException { + StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { initializeWithEntryLogger( conf, ledgerManager, @@ -160,7 +165,8 @@ void initializeWithEntryLogListener(ServerConfiguration conf, stateManager, checkpointSource, checkpointer, - new EntryLogger(conf, ledgerDirsManager, entryLogListener, statsLogger.scope(ENTRYLOGGER_SCOPE)), + new EntryLogger(conf, ledgerDirsManager, entryLogListener, statsLogger.scope(ENTRYLOGGER_SCOPE), + allocator), statsLogger); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index dd47da8bdef..0a78fcada8d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -26,7 +26,9 @@ import com.google.common.util.concurrent.MoreExecutors; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; import io.netty.util.concurrent.DefaultThreadFactory; @@ -85,8 +87,8 @@ private interface JournalIdFilter { */ @FunctionalInterface public interface BufferedChannelBuilder { - BufferedChannelBuilder DEFAULT_BCBUILDER = - (FileChannel fc, int capacity) -> new BufferedChannel(fc, capacity); + BufferedChannelBuilder DEFAULT_BCBUILDER = (FileChannel fc, + int capacity) -> new BufferedChannel(UnpooledByteBufAllocator.DEFAULT, fc, capacity); BufferedChannel create(FileChannel fc, int capacity) throws IOException; } @@ -627,18 +629,21 @@ static void writePaddingBytes(JournalChannel jc, ByteBuf paddingBuffer, int jour volatile boolean running = true; private final LedgerDirsManager ledgerDirsManager; + private final ByteBufAllocator allocator; // Expose Stats private final JournalStats journalStats; public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf, LedgerDirsManager ledgerDirsManager) { - this(journalIndex, journalDirectory, conf, ledgerDirsManager, NullStatsLogger.INSTANCE); + this(journalIndex, journalDirectory, conf, ledgerDirsManager, NullStatsLogger.INSTANCE, + UnpooledByteBufAllocator.DEFAULT); } public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf, - LedgerDirsManager ledgerDirsManager, StatsLogger statsLogger) { + LedgerDirsManager ledgerDirsManager, StatsLogger statsLogger, ByteBufAllocator allocator) { super("BookieJournal-" + conf.getBookiePort()); + this.allocator = allocator; if (conf.isBusyWaitEnabled()) { // To achieve lower latency, use busy-wait blocking queue implementation @@ -1161,7 +1166,7 @@ public void run() { } public BufferedChannelBuilder getBufferedChannelBuilder() { - return BufferedChannelBuilder.DEFAULT_BCBUILDER; + return (FileChannel fc, int capacity) -> new BufferedChannel(allocator, fc, capacity); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java index 111b8c213fc..1353e8b528e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java @@ -23,6 +23,8 @@ import com.google.common.util.concurrent.RateLimiter; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; + import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -53,7 +55,8 @@ void initialize(ServerConfiguration conf, StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, - StatsLogger statsLogger) + StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException; /** 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 5125c077fe2..cb6e9f09817 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 @@ -21,6 +21,8 @@ package org.apache.bookkeeper.bookie; +import io.netty.buffer.ByteBufAllocator; + import java.io.IOException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.stats.StatsLogger; @@ -38,9 +40,9 @@ public class ReadOnlyBookie extends Bookie { private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyBookie.class); - public ReadOnlyBookie(ServerConfiguration conf, StatsLogger statsLogger) + public ReadOnlyBookie(ServerConfiguration conf, StatsLogger statsLogger, ByteBufAllocator allocator) throws IOException, KeeperException, InterruptedException, BookieException { - super(conf, statsLogger); + super(conf, statsLogger, allocator); if (conf.isReadOnlyModeEnabled()) { stateManager.forceToReadOnly(); } else { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SlowBufferedChannel.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SlowBufferedChannel.java index 9fdc34ca7d6..e9731a25908 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SlowBufferedChannel.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SlowBufferedChannel.java @@ -22,6 +22,7 @@ */ import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import java.io.IOException; import java.nio.channels.FileChannel; @@ -36,12 +37,13 @@ public class SlowBufferedChannel extends BufferedChannel { public volatile long addDelay = 0; public volatile long flushDelay = 0; - public SlowBufferedChannel(FileChannel fc, int capacity) throws IOException { - super(fc, capacity); + public SlowBufferedChannel(ByteBufAllocator allocator, FileChannel fc, int capacity) throws IOException { + super(allocator, fc, capacity); } - public SlowBufferedChannel(FileChannel fc, int writeCapacity, int readCapacity) throws IOException { - super(fc, writeCapacity, readCapacity); + public SlowBufferedChannel(ByteBufAllocator allocator, FileChannel fc, int writeCapacity, int readCapacity) + throws IOException { + super(allocator, fc, writeCapacity, readCapacity); } public void setAddDelay(long delay) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java index 5e4dbadd2a2..77653db9646 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java @@ -24,6 +24,8 @@ import com.google.common.util.concurrent.RateLimiter; import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; + import java.io.IOException; import java.util.List; import java.util.Optional; @@ -72,7 +74,8 @@ public void initialize(ServerConfiguration conf, StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, - StatsLogger statsLogger) + StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { interleavedLedgerStorage.initializeWithEntryLogListener( @@ -86,7 +89,8 @@ public void initialize(ServerConfiguration conf, // uses sorted ledger storage's own entry log listener // since it manages entry log rotations and checkpoints. this, - statsLogger); + statsLogger, + allocator); if (conf.isEntryLogPerLedgerEnabled()) { this.memTable = new EntryMemTableWithParallelFlusher(conf, checkpointSource, statsLogger); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index ccabaed15d5..287f4526f0f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -27,6 +27,7 @@ import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.util.concurrent.DefaultThreadFactory; //CHECKSTYLE.OFF: IllegalImport import io.netty.util.internal.PlatformDependent; @@ -63,7 +64,6 @@ import org.apache.bookkeeper.util.DiskChecker; - /** * Implementation of LedgerStorage that uses RocksDB to keep the indexes for entries stored in EntryLogs. */ @@ -88,13 +88,16 @@ public class DbLedgerStorage implements LedgerStorage { private ScheduledExecutorService gcExecutor; private DbLedgerStorageStats stats; + protected ByteBufAllocator allocator; + @Override public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StateManager stateManager, CheckpointSource checkpointSource, - Checkpointer checkpointer, StatsLogger statsLogger) throws IOException { + Checkpointer checkpointer, StatsLogger statsLogger, ByteBufAllocator allocator) throws IOException { long writeCacheMaxSize = conf.getLong(WRITE_CACHE_MAX_SIZE_MB, DEFAULT_WRITE_CACHE_MAX_SIZE_MB) * MB; long readCacheMaxSize = conf.getLong(READ_AHEAD_CACHE_MAX_SIZE_MB, DEFAULT_READ_CACHE_MAX_SIZE_MB) * MB; + this.allocator = allocator; this.numberOfDirs = ledgerDirsManager.getAllLedgerDirs().size(); log.info("Started Db Ledger Storage"); @@ -139,7 +142,8 @@ protected SingleDirectoryDbLedgerStorage newSingleDirectoryDbLedgerStorage(Serve StatsLogger statsLogger, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize) throws IOException { return new SingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager, - stateManager, checkpointSource, checkpointer, statsLogger, gcExecutor, writeCacheSize, readCacheSize); + stateManager, checkpointSource, checkpointer, statsLogger, allocator, gcExecutor, writeCacheSize, + readCacheSize); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java index b14478fccd6..986c741a010 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java @@ -57,13 +57,15 @@ public class ReadCache implements Closeable { private final int segmentSize; + private ByteBufAllocator allocator; private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - public ReadCache(long maxCacheSize) { - this(maxCacheSize, DEFAULT_MAX_SEGMENT_SIZE); + public ReadCache(ByteBufAllocator allocator, long maxCacheSize) { + this(allocator, maxCacheSize, DEFAULT_MAX_SEGMENT_SIZE); } - public ReadCache(long maxCacheSize, int maxSegmentSize) { + public ReadCache(ByteBufAllocator allocator, long maxCacheSize, int maxSegmentSize) { + this.allocator = allocator; int segmentsCount = Math.max(2, (int) (maxCacheSize / maxSegmentSize)); segmentSize = (int) (maxCacheSize / segmentsCount); @@ -140,7 +142,7 @@ public ByteBuf get(long ledgerId, long entryId) { int entryOffset = (int) res.first; int entryLen = (int) res.second; - ByteBuf entry = ByteBufAllocator.DEFAULT.directBuffer(entryLen, entryLen); + ByteBuf entry = allocator.directBuffer(entryLen, entryLen); entry.writeBytes(cacheSegments.get(segmentIdx), entryOffset, entryLen); return entry; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 197014aa93b..3289b3d2f1b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -27,6 +27,7 @@ import com.google.protobuf.ByteString; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; @@ -132,7 +133,8 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, StatsLogger statsLogger, - ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize) throws IOException { + ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize) + throws IOException { checkArgument(ledgerDirsManager.getAllLedgerDirs().size() == 1, "Db implementation only allows for one storage dir"); @@ -141,8 +143,8 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le log.info("Creating single directory db ledger storage on {}", baseDir); this.writeCacheMaxSize = writeCacheSize; - this.writeCache = new WriteCache(writeCacheMaxSize / 2); - this.writeCacheBeingFlushed = new WriteCache(writeCacheMaxSize / 2); + this.writeCache = new WriteCache(allocator, writeCacheMaxSize / 2); + this.writeCacheBeingFlushed = new WriteCache(allocator, writeCacheMaxSize / 2); this.checkpointSource = checkpointSource; @@ -153,7 +155,7 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le DEFAULT_MAX_THROTTLE_TIME_MILLIS); maxThrottleTimeNanos = TimeUnit.MILLISECONDS.toNanos(maxThrottleTimeMillis); - readCache = new ReadCache(readCacheMaxSize); + readCache = new ReadCache(allocator, readCacheMaxSize); ledgerIndex = new LedgerMetadataIndex(conf, KeyValueStorageRocksDB.factory, baseDir, statsLogger); entryLocationIndex = new EntryLocationIndex(conf, KeyValueStorageRocksDB.factory, baseDir, statsLogger); @@ -164,7 +166,7 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le TransientLedgerInfo.LEDGER_INFO_CACHING_TIME_MINUTES, TransientLedgerInfo.LEDGER_INFO_CACHING_TIME_MINUTES, TimeUnit.MINUTES); - entryLogger = new EntryLogger(conf, ledgerDirsManager, null, statsLogger); + entryLogger = new EntryLogger(conf, ledgerDirsManager, null, statsLogger, allocator); gcThread = new GarbageCollectorThread(conf, ledgerManager, this, statsLogger); dbLedgerStorageStats = new DbLedgerStorageStats( @@ -179,7 +181,8 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le @Override public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StateManager stateManager, CheckpointSource checkpointSource, - Checkpointer checkpointer, StatsLogger statsLogger) throws IOException { + Checkpointer checkpointer, StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { /// Initialized in constructor } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java index 08ffe6732dc..ac58e8eacac 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java @@ -80,17 +80,20 @@ public interface EntryConsumer { private final ConcurrentLongHashSet deletedLedgers = new ConcurrentLongHashSet(); - public WriteCache(long maxCacheSize) { + private final ByteBufAllocator allocator; + + public WriteCache(ByteBufAllocator allocator, long maxCacheSize) { // Default maxSegmentSize set to 1Gb - this(maxCacheSize, 1 * 1024 * 1024 * 1024); + this(allocator, maxCacheSize, 1 * 1024 * 1024 * 1024); } - public WriteCache(long maxCacheSize, int maxSegmentSize) { + public WriteCache(ByteBufAllocator allocator, long maxCacheSize, int maxSegmentSize) { checkArgument(maxSegmentSize > 0); long alignedMaxSegmentSize = alignToPowerOfTwo(maxSegmentSize); checkArgument(maxSegmentSize == alignedMaxSegmentSize, "Max segment size needs to be in form of 2^n"); + this.allocator = allocator; this.maxCacheSize = maxCacheSize; this.maxSegmentSize = (int) maxSegmentSize; this.segmentOffsetMask = maxSegmentSize - 1; @@ -185,7 +188,7 @@ public ByteBuf get(long ledgerId, long entryId) { long offset = result.first; int size = (int) result.second; - ByteBuf entry = ByteBufAllocator.DEFAULT.buffer(size, size); + ByteBuf entry = allocator.buffer(size, size); int localOffset = (int) (offset & segmentOffsetMask); int segmentIdx = (int) (offset >>> segmentOffsetBits); 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 0acf16fd75f..94591afc803 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 @@ -26,6 +26,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; import io.netty.util.concurrent.DefaultThreadFactory; @@ -56,6 +58,7 @@ import org.apache.bookkeeper.client.api.DeleteBuilder; import org.apache.bookkeeper.client.api.OpenBuilder; import org.apache.bookkeeper.client.api.WriteFlag; +import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.common.util.ReflectionUtils; @@ -99,10 +102,11 @@ */ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper { - static final Logger LOG = LoggerFactory.getLogger(BookKeeper.class); + private static final Logger LOG = LoggerFactory.getLogger(BookKeeper.class); final EventLoopGroup eventLoopGroup; + private final ByteBufAllocator allocator; // The stats logger for this client. private final StatsLogger statsLogger; @@ -149,6 +153,7 @@ public static class Builder { ZooKeeper zk = null; EventLoopGroup eventLoopGroup = null; + ByteBufAllocator allocator = null; StatsLogger statsLogger = NullStatsLogger.INSTANCE; DNSToSwitchMapping dnsResolver = null; HashedWheelTimer requestTimer = null; @@ -213,6 +218,18 @@ public Builder eventLoopGroup(EventLoopGroup f) { return this; } + /** + * Configure the bookkeeper client with a provided {@link ByteBufAllocator}. + * + * @param allocator an external {@link ByteBufAllocator} to use by the bookkeeper client. + * @return client builder. + * @since 4.9 + */ + public Builder allocator(ByteBufAllocator allocator) { + this.allocator = allocator; + return this; + } + /** * Configure the bookkeeper client with a provided {@link ZooKeeper} client. * @@ -276,7 +293,8 @@ public Builder featureProvider(FeatureProvider featureProvider) { public BookKeeper build() throws IOException, InterruptedException, BKException { checkNotNull(statsLogger, "No stats logger provided"); - return new BookKeeper(conf, zk, eventLoopGroup, statsLogger, dnsResolver, requestTimer, featureProvider); + return new BookKeeper(conf, zk, eventLoopGroup, allocator, statsLogger, dnsResolver, requestTimer, + featureProvider); } } @@ -313,7 +331,7 @@ public BookKeeper(String servers) throws IOException, InterruptedException, */ public BookKeeper(final ClientConfiguration conf) throws IOException, InterruptedException, BKException { - this(conf, null, null, NullStatsLogger.INSTANCE, + this(conf, null, null, null, NullStatsLogger.INSTANCE, null, null, null); } @@ -347,7 +365,7 @@ private static EventLoopGroup validateEventLoopGroup(EventLoopGroup eventLoopGro */ public BookKeeper(ClientConfiguration conf, ZooKeeper zk) throws IOException, InterruptedException, BKException { - this(conf, validateZooKeeper(zk), null, NullStatsLogger.INSTANCE, null, null, null); + this(conf, validateZooKeeper(zk), null, null, NullStatsLogger.INSTANCE, null, null, null); } /** @@ -369,17 +387,19 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk) */ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLoopGroup) throws IOException, InterruptedException, BKException { - this(conf, validateZooKeeper(zk), validateEventLoopGroup(eventLoopGroup), NullStatsLogger.INSTANCE, + this(conf, validateZooKeeper(zk), validateEventLoopGroup(eventLoopGroup), null, NullStatsLogger.INSTANCE, null, null, null); } /** * Constructor for use with the builder. Other constructors also use it. */ + @SuppressWarnings("deprecation") @VisibleForTesting BookKeeper(ClientConfiguration conf, ZooKeeper zkc, EventLoopGroup eventLoopGroup, + ByteBufAllocator byteBufAllocator, StatsLogger rootStatsLogger, DNSToSwitchMapping dnsResolver, HashedWheelTimer requestTimer, @@ -443,8 +463,19 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo this.ownEventLoopGroup = false; } + if (byteBufAllocator != null) { + this.allocator = byteBufAllocator; + } else { + this.allocator = ByteBufAllocatorBuilder.create() + .poolingPolicy(conf.getAllocatorPoolingPolicy()) + .poolingConcurrency(conf.getAllocatorPoolingConcurrency()) + .outOfMemoryPolicy(conf.getAllocatorOutOfMemoryPolicy()) + .leakDetectionPolicy(conf.getAllocatorLeakDetectionPolicy()) + .build(); + } + // initialize bookie client - this.bookieClient = new BookieClientImpl(conf, this.eventLoopGroup, this.mainWorkerPool, + this.bookieClient = new BookieClientImpl(conf, this.eventLoopGroup, this.allocator, this.mainWorkerPool, scheduler, rootStatsLogger); if (null == requestTimer) { @@ -517,6 +548,7 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo bookieWatcher = null; bookieInfoScheduler = null; bookieClient = null; + allocator = UnpooledByteBufAllocator.DEFAULT; } private EnsemblePlacementPolicy initializeEnsemblePlacementPolicy(ClientConfiguration conf, @@ -1466,6 +1498,11 @@ public BookKeeperClientStats getClientStats() { public boolean isClientClosed() { return BookKeeper.this.isClosed(); } + + @Override + public ByteBufAllocator getByteBufAllocator() { + return allocator; + } }; ClientContext getClientCtx() { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientContext.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientContext.java index d8803d0ea6d..da3abde2a8b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientContext.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientContext.java @@ -20,6 +20,8 @@ */ package org.apache.bookkeeper.client; +import io.netty.buffer.ByteBufAllocator; + import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.meta.LedgerManager; @@ -37,6 +39,7 @@ interface ClientContext { BookieWatcher getBookieWatcher(); EnsemblePlacementPolicy getPlacementPolicy(); BookieClient getBookieClient(); + ByteBufAllocator getByteBufAllocator(); OrderedExecutor getMainWorkerPool(); OrderedScheduler getScheduler(); BookKeeperClientStats getClientStats(); 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 157a1b84cac..bd8ec68314c 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 @@ -188,7 +188,7 @@ private enum HandleState { } macManager = DigestManager.instantiate(ledgerId, password, BookKeeper.DigestType.toProtoDigestType(digestType), - clientCtx.getConf().useV2WireProtocol); + clientCtx.getByteBufAllocator(), clientCtx.getConf().useV2WireProtocol); // If the password is empty, pass the same random ledger key which is generated by the hash of the empty // password, so that the bookie can avoid processing the keys for each entry diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BookKeeperBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BookKeeperBuilder.java index 0e147dd3011..ea30dc5177c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BookKeeperBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BookKeeperBuilder.java @@ -20,9 +20,11 @@ */ package org.apache.bookkeeper.client.api; +import io.netty.buffer.ByteBufAllocator; import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; import java.io.IOException; + import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable; import org.apache.bookkeeper.feature.FeatureProvider; @@ -47,6 +49,15 @@ public interface BookKeeperBuilder { */ BookKeeperBuilder eventLoopGroup(EventLoopGroup eventLoopGroup); + /** + * Configure the bookkeeper client with a provided {@link ByteBufAllocator}. + * + * @param allocator an external {@link ByteBufAllocator} to use by the bookkeeper client. + * @return client builder. + * @since 4.9 + */ + BookKeeperBuilder allocator(ByteBufAllocator allocator); + /** * Configure the bookkeeper client with a provided {@link StatsLogger}. * 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 3a07d1bc55d..6373ace3ed2 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,6 +20,7 @@ */ package org.apache.bookkeeper.client.impl; +import io.netty.buffer.ByteBufAllocator; import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; import java.io.IOException; @@ -50,6 +51,12 @@ public BookKeeperBuilder eventLoopGroup(EventLoopGroup eventLoopGroup) { return this; } + @Override + public BookKeeperBuilder allocator(ByteBufAllocator allocator) { + builder.allocator(allocator); + return this; + } + @Override public BookKeeperBuilder statsLogger(StatsLogger statsLogger) { builder.statsLogger(statsLogger); 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 65d702b3b51..36cfa637e97 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 @@ -28,6 +28,10 @@ import javax.net.ssl.SSLEngine; import lombok.extern.slf4j.Slf4j; + +import org.apache.bookkeeper.common.allocator.LeakDetectionPolicy; +import org.apache.bookkeeper.common.allocator.OutOfMemoryPolicy; +import org.apache.bookkeeper.common.allocator.PoolingPolicy; import org.apache.bookkeeper.common.util.JsonUtil; import org.apache.bookkeeper.common.util.JsonUtil.ParseJsonException; import org.apache.bookkeeper.common.util.ReflectionUtils; @@ -155,6 +159,12 @@ public abstract class AbstractConfiguration // enforce minimum number of racks per write quorum public static final String ENFORCE_MIN_NUM_RACKS_PER_WRITE_QUORUM = "enforceMinNumRacksPerWriteQuorum"; + // Allocator configuration + protected static final String ALLOCATOR_POOLING_POLICY = "allocatorPoolingPolicy"; + protected static final String ALLOCATOR_POOLING_CONCURRENCY = "allocatorPoolingConcurrency"; + protected static final String ALLOCATOR_OOM_POLICY = "allocatorOutOfMemoryPolicy"; + protected static final String ALLOCATOR_LEAK_DETECTION_POLICY = "allocatorLeakDetectionPolicy"; + // option to limit stats logging public static final String LIMIT_STATS_LOGGING = "limitStatsLogging"; @@ -881,6 +891,95 @@ public T setPreserveMdcForTaskExecution(boolean enabled) { return getThis(); } + /** + * @return the configured pooling policy for the allocator. + */ + public PoolingPolicy getAllocatorPoolingPolicy() { + return PoolingPolicy.valueOf(this.getString(ALLOCATOR_POOLING_POLICY, PoolingPolicy.PooledDirect.toString())); + } + + /** + * Define the memory pooling policy. + * + *

            Default is {@link PoolingPolicy#PooledDirect} + * + * @param poolingPolicy + * the memory pooling policy + * @return configuration object. + */ + public T setAllocatorPoolingPolicy(PoolingPolicy poolingPolicy) { + this.setProperty(ALLOCATOR_POOLING_POLICY, poolingPolicy.toString()); + return getThis(); + } + + /** + * @return the configured pooling concurrency for the allocator. + */ + public int getAllocatorPoolingConcurrency() { + return this.getInteger(ALLOCATOR_POOLING_CONCURRENCY, 2 * Runtime.getRuntime().availableProcessors()); + } + + /** + * Controls the amount of concurrency for the memory pool. + * + *

            Default is to have a number of allocator arenas equals to 2 * CPUS. + * + *

            Decreasing this number will reduce the amount of memory overhead, at the + * expense of increased allocation contention. + * + * @param concurrency + * the concurrency level to use for the allocator pool + * @return configuration object. + */ + public T setAllocatorPoolingConcurrenncy(int concurrency) { + this.setProperty(ALLOCATOR_POOLING_POLICY, concurrency); + return getThis(); + } + + /** + * @return the configured ouf of memory policy for the allocator. + */ + public OutOfMemoryPolicy getAllocatorOutOfMemoryPolicy() { + return OutOfMemoryPolicy + .valueOf(this.getString(ALLOCATOR_OOM_POLICY, OutOfMemoryPolicy.FallbackToHeap.toString())); + } + + /** + * Define the memory allocator out of memory policy. + * + *

            Default is {@link OutOfMemoryPolicy#FallbackToHeap} + * + * @param oomPolicy + * the "out-of-memory" policy for the memory allocator + * @return configuration object. + */ + public T setAllocatorOutOfMemoryPolicy(OutOfMemoryPolicy oomPolicy) { + this.setProperty(ALLOCATOR_OOM_POLICY, oomPolicy.toString()); + return getThis(); + } + + /** + * Return the configured leak detection policy for the allocator. + */ + public LeakDetectionPolicy getAllocatorLeakDetectionPolicy() { + return LeakDetectionPolicy + .valueOf(this.getString(ALLOCATOR_LEAK_DETECTION_POLICY, LeakDetectionPolicy.Disabled.toString())); + } + + /** + * Enable the leak detection for the allocator. + * + *

            Default is {@link LeakDetectionPolicy#Disabled} + * + * @param leakDetectionPolicy + * the leak detection policy for the memory allocator + * @return configuration object. + */ + public T setAllocatorLeakDetectionPolicy(LeakDetectionPolicy leakDetectionPolicy) { + this.setProperty(ALLOCATOR_LEAK_DETECTION_POLICY, leakDetectionPolicy.toString()); + return getThis(); + } + /** * Return whether the busy-wait is enabled for BookKeeper and Netty IO threads. * 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 7d0d319882b..3b390d48e41 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 @@ -21,11 +21,14 @@ import static org.apache.bookkeeper.util.BookKeeperConstants.FEATURE_DISABLE_ENSEMBLE_CHANGE; import io.netty.buffer.ByteBuf; + import java.util.concurrent.TimeUnit; + import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.EnsemblePlacementPolicy; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; +import org.apache.bookkeeper.client.api.BookKeeperBuilder; import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.discover.ZKRegistrationClient; @@ -1772,8 +1775,11 @@ public ClientConfiguration setMaxAllowedEnsembleChanges(int num) { /** * Option to use Netty Pooled ByteBufs. * + * @deprecated see {@link BookKeeperBuilder#allocator(io.netty.buffer.ByteBufAllocator)} + * * @return the value of the option */ + @Deprecated public boolean isNettyUsePooledBuffers() { return getBoolean(NETTY_USE_POOLED_BUFFERS, true); } @@ -1785,6 +1791,8 @@ public boolean isNettyUsePooledBuffers() { * @param enabled * if enabled BookKeeper will use default Pooled Netty Buffer allocator * + * @deprecated see {@link BookKeeperBuilder#allocator(io.netty.buffer.ByteBufAllocator)} + * * @see #setUseV2WireProtocol(boolean) * @see ByteBuf#release() * @see LedgerHandle#readEntries(long, long) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index 18f48a2ab89..83428214a5d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -27,6 +27,7 @@ import com.google.protobuf.ExtensionRegistry; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; @@ -76,11 +77,12 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFactory { static final Logger LOG = LoggerFactory.getLogger(BookieClient.class); - OrderedExecutor executor; - ScheduledExecutorService scheduler; - ScheduledFuture timeoutFuture; + private final OrderedExecutor executor; + private final ScheduledExecutorService scheduler; + private final ScheduledFuture timeoutFuture; - EventLoopGroup eventLoopGroup; + private final EventLoopGroup eventLoopGroup; + private final ByteBufAllocator allocator; final ConcurrentHashMap channels = new ConcurrentHashMap(); @@ -96,10 +98,12 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFac private final long bookieErrorThresholdPerInterval; public BookieClientImpl(ClientConfiguration conf, EventLoopGroup eventLoopGroup, + ByteBufAllocator allocator, OrderedExecutor executor, ScheduledExecutorService scheduler, StatsLogger statsLogger) throws IOException { this.conf = conf; this.eventLoopGroup = eventLoopGroup; + this.allocator = allocator; this.executor = executor; this.closed = false; this.closeLock = new ReentrantReadWriteLock(); @@ -120,6 +124,8 @@ public BookieClientImpl(ClientConfiguration conf, EventLoopGroup eventLoopGroup, conf.getTimeoutMonitorIntervalSec(), conf.getTimeoutMonitorIntervalSec(), TimeUnit.SECONDS); + } else { + this.timeoutFuture = null; } } @@ -175,7 +181,7 @@ public PerChannelBookieClient create(BookieSocketAddress address, PerChannelBook if (conf.getLimitStatsLogging()) { statsLoggerForPCBC = NullStatsLogger.INSTANCE; } - return new PerChannelBookieClient(conf, executor, eventLoopGroup, address, statsLoggerForPCBC, + return new PerChannelBookieClient(conf, executor, eventLoopGroup, allocator, address, statsLoggerForPCBC, authProviderFactory, registry, pcbcPool, shFactory); } @@ -596,8 +602,8 @@ public void writeComplete(int rc, long ledger, long entry, BookieSocketAddress a .build(); ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor( new DefaultThreadFactory("BookKeeperClientScheduler")); - BookieClientImpl bc = new BookieClientImpl(new ClientConfiguration(), eventLoopGroup, executor, - scheduler, NullStatsLogger.INSTANCE); + BookieClientImpl bc = new BookieClientImpl(new ClientConfiguration(), eventLoopGroup, + null, executor, scheduler, NullStatsLogger.INSTANCE); BookieSocketAddress addr = new BookieSocketAddress(args[0], Integer.parseInt(args[1])); for (int i = 0; i < 100000; i++) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java index add453728bb..5b5c288123a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java @@ -24,6 +24,7 @@ import com.google.protobuf.ExtensionRegistry; import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.PooledByteBufAllocator; import io.netty.channel.AdaptiveRecvByteBufAllocator; import io.netty.channel.Channel; @@ -107,8 +108,11 @@ class BookieNettyServer { final BookieAuthProvider.Factory authProviderFactory; final ExtensionRegistry registry = ExtensionRegistry.newInstance(); - BookieNettyServer(ServerConfiguration conf, RequestProcessor processor) + private final ByteBufAllocator allocator; + + BookieNettyServer(ServerConfiguration conf, RequestProcessor processor, ByteBufAllocator allocator) throws IOException, KeeperException, InterruptedException, BookieException { + this.allocator = allocator; this.maxFrameSize = conf.getNettyMaxFrameSizeBytes(); this.conf = conf; this.requestProcessor = processor; @@ -296,7 +300,8 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { private void listenOn(InetSocketAddress address, BookieSocketAddress bookieAddress) throws InterruptedException { if (!conf.isDisableServerSocketBind()) { ServerBootstrap bootstrap = new ServerBootstrap(); - bootstrap.childOption(ChannelOption.ALLOCATOR, new PooledByteBufAllocator(true)); + bootstrap.option(ChannelOption.ALLOCATOR, allocator); + bootstrap.childOption(ChannelOption.ALLOCATOR, allocator); bootstrap.group(eventLoopGroup, eventLoopGroup); bootstrap.childOption(ChannelOption.TCP_NODELAY, conf.getServerTcpNoDelay()); bootstrap.childOption(ChannelOption.SO_LINGER, conf.getServerSockLinger()); 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 b883f74a21f..7d5e2e613ed 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 @@ -29,6 +29,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.ByteString; +import io.netty.buffer.ByteBufAllocator; import io.netty.channel.Channel; import io.netty.handler.ssl.SslHandler; import io.netty.util.HashedWheelTimer; @@ -44,6 +45,7 @@ import lombok.AccessLevel; import lombok.Getter; + import org.apache.bookkeeper.auth.AuthProviderFactoryFactory; import org.apache.bookkeeper.auth.AuthToken; import org.apache.bookkeeper.bookie.Bookie; @@ -124,9 +126,12 @@ public class BookieRequestProcessor implements RequestProcessor { final Optional> blacklistedChannels; final Consumer onResponseTimeout; - public BookieRequestProcessor(ServerConfiguration serverCfg, Bookie bookie, - StatsLogger statsLogger, SecurityHandlerFactory shFactory) throws SecurityException { + private final ByteBufAllocator allocator; + + public BookieRequestProcessor(ServerConfiguration serverCfg, Bookie bookie, StatsLogger statsLogger, + SecurityHandlerFactory shFactory, ByteBufAllocator allocator) throws SecurityException { this.serverCfg = serverCfg; + this.allocator = allocator; this.waitTimeoutOnBackpressureMillis = serverCfg.getWaitTimeoutOnResponseBackpressureMillis(); this.preserveMdcForTaskExecution = serverCfg.getPreserveMdcForTaskExecution(); this.bookie = bookie; @@ -158,7 +163,7 @@ public BookieRequestProcessor(ServerConfiguration serverCfg, Bookie bookie, OrderedExecutor.NO_TASK_LIMIT, statsLogger); this.shFactory = shFactory; if (shFactory != null) { - shFactory.init(NodeType.Server, serverCfg); + shFactory.init(NodeType.Server, serverCfg, allocator); } this.requestTimer = new HashedWheelTimer( 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 27e8ab2eef5..b8c1adcef06 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 @@ -25,6 +25,8 @@ import static org.apache.bookkeeper.conf.AbstractConfiguration.PERMITTED_STARTUP_USERS; import com.google.common.annotations.VisibleForTesting; +import io.netty.buffer.ByteBufAllocator; + import java.io.IOException; import java.lang.Thread.UncaughtExceptionHandler; import java.net.UnknownHostException; @@ -38,6 +40,7 @@ import org.apache.bookkeeper.bookie.ExitCode; import org.apache.bookkeeper.bookie.ReadOnlyBookie; import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder; import org.apache.bookkeeper.common.util.JsonUtil.ParseJsonException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -62,7 +65,7 @@ public class BookieServer { final ServerConfiguration conf; BookieNettyServer nettyServer; private volatile boolean running = false; - Bookie bookie; + private final Bookie bookie; DeathWatcher deathWatcher; private static final Logger LOG = LoggerFactory.getLogger(BookieServer.class); @@ -96,10 +99,11 @@ public BookieServer(ServerConfiguration conf, StatsLogger statsLogger) LOG.error("Got ParseJsonException while converting Config to JSONString", pe); } + ByteBufAllocator allocator = getAllocator(conf); this.statsLogger = statsLogger; - this.nettyServer = new BookieNettyServer(this.conf, null); + this.nettyServer = new BookieNettyServer(this.conf, null, allocator); try { - this.bookie = newBookie(conf); + this.bookie = newBookie(conf, allocator); } catch (IOException | KeeperException | InterruptedException | BookieException e) { // interrupted on constructing a bookie this.nettyServer.shutdown(); @@ -110,7 +114,7 @@ public BookieServer(ServerConfiguration conf, StatsLogger statsLogger) shFactory = SecurityProviderFactoryFactory .getSecurityProviderFactory(conf.getTLSProviderFactoryClass()); this.requestProcessor = new BookieRequestProcessor(conf, bookie, - statsLogger.scope(SERVER_SCOPE), shFactory); + statsLogger.scope(SERVER_SCOPE), shFactory, bookie.getAllocator()); this.nettyServer.setRequestProcessor(this.requestProcessor); } @@ -126,11 +130,11 @@ public void setExceptionHandler(UncaughtExceptionHandler exceptionHandler) { this.uncaughtExceptionHandler = exceptionHandler; } - protected Bookie newBookie(ServerConfiguration conf) + protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) throws IOException, KeeperException, InterruptedException, BookieException { return conf.isForceReadOnlyBookie() - ? new ReadOnlyBookie(conf, statsLogger.scope(BOOKIE_SCOPE)) - : new Bookie(conf, statsLogger.scope(BOOKIE_SCOPE)); + ? new ReadOnlyBookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator) + : new Bookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator); } public void start() throws IOException, UnavailableException, InterruptedException, BKException { @@ -285,6 +289,24 @@ public void run() { } } + private ByteBufAllocator getAllocator(ServerConfiguration conf) { + return ByteBufAllocatorBuilder.create() + .poolingPolicy(conf.getAllocatorPoolingPolicy()) + .poolingConcurrency(conf.getAllocatorPoolingConcurrency()) + .outOfMemoryPolicy(conf.getAllocatorOutOfMemoryPolicy()) + .outOfMemoryListener((ex) -> { + try { + LOG.error("Unable to allocate memory, exiting bookie", ex); + } finally { + if (uncaughtExceptionHandler != null) { + uncaughtExceptionHandler.uncaughtException(Thread.currentThread(), ex); + } + } + }) + .leakDetectionPolicy(conf.getAllocatorLeakDetectionPolicy()) + .build(); + } + /** * Legacy Method to run bookie server. */ 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 6839afc8f25..d411e9880d2 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 @@ -29,7 +29,6 @@ import io.netty.bootstrap.Bootstrap; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.Channel; @@ -166,6 +165,7 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter { final BookieSocketAddress addr; final EventLoopGroup eventLoopGroup; + final ByteBufAllocator allocator; final OrderedExecutor executor; final long addEntryTimeoutNanos; final long readEntryTimeoutNanos; @@ -345,12 +345,14 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor StatsLogger parentStatsLogger, ClientAuthProvider.Factory authProviderFactory, ExtensionRegistry extRegistry, PerChannelBookieClientPool pcbcPool) throws SecurityException { - this(conf, executor, eventLoopGroup, addr, NullStatsLogger.INSTANCE, + this(conf, executor, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, addr, NullStatsLogger.INSTANCE, authProviderFactory, extRegistry, pcbcPool, null); } public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor, - EventLoopGroup eventLoopGroup, BookieSocketAddress addr, + EventLoopGroup eventLoopGroup, + ByteBufAllocator allocator, + BookieSocketAddress addr, StatsLogger parentStatsLogger, ClientAuthProvider.Factory authProviderFactory, ExtensionRegistry extRegistry, PerChannelBookieClientPool pcbcPool, @@ -364,6 +366,7 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor } else { this.eventLoopGroup = eventLoopGroup; } + this.allocator = allocator; this.state = ConnectionState.DISCONNECTED; this.addEntryTimeoutNanos = TimeUnit.SECONDS.toNanos(conf.getAddEntryTimeout()); this.readEntryTimeoutNanos = TimeUnit.SECONDS.toNanos(conf.getReadEntryTimeout()); @@ -376,7 +379,7 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor this.extRegistry = extRegistry; this.shFactory = shFactory; if (shFactory != null) { - shFactory.init(NodeType.Client, conf); + shFactory.init(NodeType.Client, conf, allocator); } StringBuilder nameBuilder = new StringBuilder(); @@ -515,14 +518,7 @@ protected ChannelFuture connect() { bootstrap.channel(NioSocketChannel.class); } - ByteBufAllocator allocator; - if (this.conf.isNettyUsePooledBuffers()) { - allocator = PooledByteBufAllocator.DEFAULT; - } else { - allocator = UnpooledByteBufAllocator.DEFAULT; - } - - bootstrap.option(ChannelOption.ALLOCATOR, allocator); + bootstrap.option(ChannelOption.ALLOCATOR, this.allocator); bootstrap.option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.getClientConnectTimeoutMillis()); bootstrap.option(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark( conf.getClientWriteBufferLowWaterMark(), conf.getClientWriteBufferHighWaterMark())); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32CDigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32CDigestManager.java index d6d1949b47f..59dec3a2355 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32CDigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32CDigestManager.java @@ -22,6 +22,7 @@ import com.scurrilous.circe.crc.Sse42Crc32C; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.util.concurrent.FastThreadLocal; import lombok.extern.slf4j.Slf4j; @@ -38,10 +39,10 @@ protected MutableInt initialValue() throws Exception { } }; - public CRC32CDigestManager(long ledgerId, boolean useV2Protocol) { - super(ledgerId, useV2Protocol); + public CRC32CDigestManager(long ledgerId, boolean useV2Protocol, ByteBufAllocator allocator) { + super(ledgerId, useV2Protocol, allocator); if (!Sse42Crc32C.isSupported()) { - log.error("Sse42Crc32C is not supported, will use less slower CRC32C implementation."); + log.error("Sse42Crc32C is not supported, will use a slower CRC32C implementation."); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32DigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32DigestManager.java index b71ab596262..d06bc8030cd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32DigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32DigestManager.java @@ -19,6 +19,7 @@ */ import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.util.concurrent.FastThreadLocal; /** @@ -46,8 +47,8 @@ protected CRC32Digest initialValue() { } }; - public CRC32DigestManager(long ledgerId, boolean useV2Protocol) { - super(ledgerId, useV2Protocol); + public CRC32DigestManager(long ledgerId, boolean useV2Protocol, ByteBufAllocator allocator) { + super(ledgerId, useV2Protocol, allocator); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java index 19286374540..2dabf828216 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.proto.checksum; import io.netty.buffer.ByteBuf; -import io.netty.buffer.PooledByteBufAllocator; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.util.ReferenceCountUtil; @@ -47,6 +47,7 @@ public abstract class DigestManager { final long ledgerId; final boolean useV2Protocol; + private final ByteBufAllocator allocator; abstract int getMacCodeLength(); @@ -60,28 +61,24 @@ void update(byte[] data) { final int macCodeLength; - public DigestManager(long ledgerId, boolean useV2Protocol) { + public DigestManager(long ledgerId, boolean useV2Protocol, ByteBufAllocator allocator) { this.ledgerId = ledgerId; this.useV2Protocol = useV2Protocol; - macCodeLength = getMacCodeLength(); - } - - public static DigestManager instantiate(long ledgerId, byte[] passwd, DigestType digestType) - throws GeneralSecurityException { - return instantiate(ledgerId, passwd, digestType, false); + this.macCodeLength = getMacCodeLength(); + this.allocator = allocator; } public static DigestManager instantiate(long ledgerId, byte[] passwd, DigestType digestType, - boolean useV2Protocol) throws GeneralSecurityException { + ByteBufAllocator allocator, boolean useV2Protocol) throws GeneralSecurityException { switch(digestType) { case HMAC: - return new MacDigestManager(ledgerId, passwd, useV2Protocol); + return new MacDigestManager(ledgerId, passwd, useV2Protocol, allocator); case CRC32: - return new CRC32DigestManager(ledgerId, useV2Protocol); + return new CRC32DigestManager(ledgerId, useV2Protocol, allocator); case CRC32C: - return new CRC32CDigestManager(ledgerId, useV2Protocol); + return new CRC32CDigestManager(ledgerId, useV2Protocol, allocator); case DUMMY: - return new DummyDigestManager(ledgerId, useV2Protocol); + return new DummyDigestManager(ledgerId, useV2Protocol, allocator); default: throw new GeneralSecurityException("Unknown checksum type: " + digestType); } @@ -106,7 +103,7 @@ public ByteBufList computeDigestAndPackageForSending(long entryId, long lastAddC /* * For V2 protocol, use pooled direct ByteBuf's to avoid object allocation in DigestManager. */ - ByteBuf headersBuffer = PooledByteBufAllocator.DEFAULT.buffer(METADATA_LENGTH + macCodeLength); + ByteBuf headersBuffer = allocator.buffer(METADATA_LENGTH + macCodeLength); headersBuffer.writeLong(ledgerId); headersBuffer.writeLong(entryId); headersBuffer.writeLong(lastAddConfirmed); @@ -149,7 +146,7 @@ public ByteBufList computeDigestAndPackageForSending(long entryId, long lastAddC public ByteBufList computeDigestAndPackageForSendingLac(long lac) { ByteBuf headersBuffer; if (this.useV2Protocol) { - headersBuffer = PooledByteBufAllocator.DEFAULT.buffer(LAC_METADATA_LENGTH + macCodeLength); + headersBuffer = allocator.buffer(LAC_METADATA_LENGTH + macCodeLength); } else { headersBuffer = Unpooled.buffer(LAC_METADATA_LENGTH + macCodeLength); } @@ -185,7 +182,7 @@ private void verifyDigest(long entryId, ByteBuf dataReceived, boolean skipEntryI int offset = METADATA_LENGTH + macCodeLength; update(dataReceived.slice(offset, dataReceived.readableBytes() - offset)); - ByteBuf digest = PooledByteBufAllocator.DEFAULT.buffer(macCodeLength); + ByteBuf digest = allocator.buffer(macCodeLength); populateValueAndReset(digest); try { @@ -225,7 +222,7 @@ public long verifyDigestAndReturnLac(ByteBuf dataReceived) throws BKDigestMatchE update(dataReceived.slice(0, LAC_METADATA_LENGTH)); - ByteBuf digest = PooledByteBufAllocator.DEFAULT.buffer(macCodeLength); + ByteBuf digest = allocator.buffer(macCodeLength); try { populateValueAndReset(digest); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DummyDigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DummyDigestManager.java index 1b771f0785b..aeb0d5b21a7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DummyDigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DummyDigestManager.java @@ -21,14 +21,15 @@ package org.apache.bookkeeper.proto.checksum; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; /** * This class provides a noop digest implementation. */ public class DummyDigestManager extends DigestManager { - public DummyDigestManager(long ledgerId, boolean useV2Protocol) { - super(ledgerId, useV2Protocol); + public DummyDigestManager(long ledgerId, boolean useV2Protocol, ByteBufAllocator allocator) { + super(ledgerId, useV2Protocol, allocator); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java index e71c077eabc..92b93f1450c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java @@ -21,6 +21,7 @@ import static com.google.common.base.Charsets.UTF_8; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import java.security.GeneralSecurityException; import java.security.MessageDigest; @@ -71,9 +72,9 @@ protected Mac initialValue() { } }; - public MacDigestManager(long ledgerId, byte[] passwd, boolean useV2Protocol) + public MacDigestManager(long ledgerId, byte[] passwd, boolean useV2Protocol, ByteBufAllocator allocator) throws GeneralSecurityException { - super(ledgerId, useV2Protocol); + super(ledgerId, useV2Protocol, allocator); this.passwd = Arrays.copyOf(passwd, passwd.length); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityHandlerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityHandlerFactory.java index 59be8847dd9..5b43744fd1e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityHandlerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityHandlerFactory.java @@ -17,6 +17,7 @@ */ package org.apache.bookkeeper.tls; +import io.netty.buffer.ByteBufAllocator; import io.netty.handler.ssl.SslHandler; import org.apache.bookkeeper.conf.AbstractConfiguration; @@ -37,7 +38,7 @@ enum NodeType { String getHandlerName(); - void init(NodeType type, AbstractConfiguration conf) throws SecurityException; + void init(NodeType type, AbstractConfiguration conf, ByteBufAllocator allocator) throws SecurityException; SslHandler newTLSHandler(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java index 17aea85fafa..32e24264046 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java @@ -20,7 +20,8 @@ import com.google.common.base.Strings; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import io.netty.buffer.PooledByteBufAllocator; + +import io.netty.buffer.ByteBufAllocator; import io.netty.handler.ssl.ClientAuth; import io.netty.handler.ssl.OpenSsl; import io.netty.handler.ssl.SslContext; @@ -79,6 +80,7 @@ public String toString() { private String[] protocols; private String[] ciphers; private SslContext sslContext; + private ByteBufAllocator allocator; private String getPasswordFromFile(String path) throws IOException { byte[] pwd; @@ -350,7 +352,9 @@ private void createServerContext(AbstractConfiguration conf) throws SecurityExce } @Override - public synchronized void init(NodeType type, AbstractConfiguration conf) throws SecurityException { + public synchronized void init(NodeType type, AbstractConfiguration conf, ByteBufAllocator allocator) + throws SecurityException { + this.allocator = allocator; final String enabledProtocols; final String enabledCiphers; @@ -397,7 +401,7 @@ public synchronized void init(NodeType type, AbstractConfiguration conf) throws @Override public SslHandler newTLSHandler() { - SslHandler sslHandler = sslContext.newHandler(PooledByteBufAllocator.DEFAULT); + SslHandler sslHandler = sslContext.newHandler(allocator); if (protocols != null && protocols.length != 0) { sslHandler.engine().setEnabledProtocols(protocols); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java index f9e4cffce09..355cf3f307b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java @@ -23,7 +23,6 @@ import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; -import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandler.Sharable; import io.netty.channel.ChannelHandlerContext; @@ -307,7 +306,7 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) if (prependSize) { // Prepend the frame size before writing the buffer list, so that we only have 1 single size // header - ByteBuf sizeBuffer = PooledByteBufAllocator.DEFAULT.directBuffer(4, 4); + ByteBuf sizeBuffer = ctx.alloc().directBuffer(4, 4); sizeBuffer.writeInt(b.readableBytes()); ctx.write(sizeBuffer, ctx.voidPromise()); } 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 530969bf68d..7dde6fdf0a2 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 @@ -34,6 +34,8 @@ import static org.mockito.Mockito.mock; import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.UnpooledByteBufAllocator; import java.io.BufferedReader; import java.io.BufferedWriter; @@ -153,7 +155,8 @@ public void testExitCodeZK_REG_FAIL() throws Exception { // simulating ZooKeeper exception by assigning a closed zk client to bk BookieServer bkServer = new BookieServer(conf) { - protected Bookie newBookie(ServerConfiguration conf) + @Override + protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) throws IOException, KeeperException, InterruptedException, BookieException { Bookie bookie = new Bookie(conf); @@ -708,7 +711,7 @@ public MockBookieServer(ServerConfiguration conf) throws IOException, KeeperExce } @Override - protected Bookie newBookie(ServerConfiguration conf) + protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) throws IOException, KeeperException, InterruptedException, BookieException { return new MockBookieWithNoopShutdown(conf, NullStatsLogger.INSTANCE); } @@ -717,7 +720,7 @@ protected Bookie newBookie(ServerConfiguration conf) class MockBookieWithNoopShutdown extends Bookie { public MockBookieWithNoopShutdown(ServerConfiguration conf, StatsLogger statsLogger) throws IOException, KeeperException, InterruptedException, BookieException { - super(conf, statsLogger); + super(conf, statsLogger, UnpooledByteBufAllocator.DEFAULT); } // making Bookie Shutdown no-op. Ideally for this testcase we need to diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BufferedChannelTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BufferedChannelTest.java index 86f3a8643f0..c98663dd631 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BufferedChannelTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BufferedChannelTest.java @@ -23,6 +23,8 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.File; import java.io.RandomAccessFile; import java.nio.channels.FileChannel; @@ -76,8 +78,8 @@ public void testBufferedChannel(int byteBufLength, int numOfWrites, int unpersis newLogFile.deleteOnExit(); FileChannel fileChannel = new RandomAccessFile(newLogFile, "rw").getChannel(); - BufferedChannel logChannel = new BufferedChannel(fileChannel, INTERNAL_BUFFER_WRITE_CAPACITY, - INTERNAL_BUFFER_READ_CAPACITY, unpersistedBytesBound); + BufferedChannel logChannel = new BufferedChannel(UnpooledByteBufAllocator.DEFAULT, fileChannel, + INTERNAL_BUFFER_WRITE_CAPACITY, INTERNAL_BUFFER_READ_CAPACITY, unpersistedBytesBound); ByteBuf dataBuf = generateEntry(byteBufLength); dataBuf.markReaderIndex(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java index bbe9c10edcd..556556e348d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java @@ -37,6 +37,7 @@ import com.google.common.util.concurrent.UncheckedExecutionException; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import java.io.File; import java.io.IOException; @@ -265,7 +266,8 @@ public void checkpointComplete(Checkpoint checkPoint, boolean compact) null, cp, Checkpointer.NULL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, + UnpooledByteBufAllocator.DEFAULT); storage.start(); long startTime = System.currentTimeMillis(); storage.gcThread.enableForceGC(); @@ -616,7 +618,8 @@ public void testCompactionPersistence() throws Exception { Bookie newbookie = new Bookie(newBookieConf); DigestManager digestManager = DigestManager.instantiate(ledgerId, passwdBytes, - BookKeeper.DigestType.toProtoDigestType(digestType), baseClientConf.getUseV2WireProtocol()); + BookKeeper.DigestType.toProtoDigestType(digestType), UnpooledByteBufAllocator.DEFAULT, + baseClientConf.getUseV2WireProtocol()); for (long entryId = 0; entryId <= lastAddConfirmed; entryId++) { ByteBuf readEntryBufWithChecksum = newbookie.readEntry(ledgerId, entryId); @@ -860,7 +863,8 @@ public void checkpointComplete(CheckpointSource.Checkpoint checkpoint, boolean c null, checkpointSource, Checkpointer.NULL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, + UnpooledByteBufAllocator.DEFAULT); ledgers.add(1L); ledgers.add(2L); ledgers.add(3L); @@ -885,7 +889,8 @@ public void checkpointComplete(CheckpointSource.Checkpoint checkpoint, boolean c dirs, dirs, null, checkpointSource, Checkpointer.NULL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, + UnpooledByteBufAllocator.DEFAULT); storage.start(); for (int i = 0; i < 10; i++) { if (!log0.exists()) { @@ -910,7 +915,8 @@ public void checkpointComplete(CheckpointSource.Checkpoint checkpoint, boolean c null, checkpointSource, Checkpointer.NULL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, + UnpooledByteBufAllocator.DEFAULT); storage.getEntry(1, 1); // entry should exist } @@ -1021,7 +1027,8 @@ public void checkpointComplete(Checkpoint checkpoint, null, checkpointSource, Checkpointer.NULL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, + UnpooledByteBufAllocator.DEFAULT); double threshold = 0.1; // shouldn't throw exception @@ -1063,7 +1070,7 @@ public void checkpointComplete(Checkpoint checkpoint, }; InterleavedLedgerStorage storage = new InterleavedLedgerStorage(); storage.initialize(conf, manager, dirs, dirs, null, checkpointSource, - Checkpointer.NULL, NullStatsLogger.INSTANCE); + Checkpointer.NULL, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT); for (long ledger = 0; ledger <= 10; ledger++) { ledgers.add(ledger); @@ -1081,7 +1088,7 @@ public void checkpointComplete(Checkpoint checkpoint, storage = new InterleavedLedgerStorage(); storage.initialize(conf, manager, dirs, dirs, null, checkpointSource, - Checkpointer.NULL, NullStatsLogger.INSTANCE); + Checkpointer.NULL, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT); long startingEntriesCount = storage.gcThread.entryLogger.getLeastUnflushedLogId() - storage.gcThread.scannedLogId; @@ -1158,7 +1165,8 @@ public void checkpointComplete(Checkpoint checkPoint, boolean compact) null, cp, Checkpointer.NULL, - stats.getStatsLogger("storage")); + stats.getStatsLogger("storage"), + UnpooledByteBufAllocator.DEFAULT); storage.start(); int majorCompactions = stats.getCounter("storage.gc." + MAJOR_COMPACTION_COUNT).get().intValue(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java index f5d4edce967..0b24db23047 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java @@ -22,6 +22,8 @@ import com.google.common.util.concurrent.MoreExecutors; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.File; import java.io.IOException; import java.util.HashMap; @@ -606,7 +608,8 @@ public void testEntryLogManagerMetrics() throws Exception { conf.setEntryLogPerLedgerCounterLimitsMultFactor(entryLogPerLedgerCounterLimitsMultFactor); LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); - EntryLogger entryLogger = new EntryLogger(conf, ledgerDirsManager, null, statsLogger); + EntryLogger entryLogger = new EntryLogger(conf, ledgerDirsManager, null, statsLogger, + UnpooledByteBufAllocator.DEFAULT); EntryLogManagerForEntryLogPerLedger entrylogManager = (EntryLogManagerForEntryLogPerLedger) entryLogger .getEntryLogManager(); // set same thread executor for entryLoggerAllocator's allocatorExecutor @@ -731,7 +734,8 @@ public void testEntryLogManagerMetricsFromExpiryAspect() throws Exception { conf.setEntryLogPerLedgerCounterLimitsMultFactor(entryLogPerLedgerCounterLimitsMultFactor); LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); - EntryLogger entryLogger = new EntryLogger(conf, ledgerDirsManager, null, statsLogger); + EntryLogger entryLogger = new EntryLogger(conf, ledgerDirsManager, null, statsLogger, + UnpooledByteBufAllocator.DEFAULT); EntryLogManagerForEntryLogPerLedger entrylogManager = (EntryLogManagerForEntryLogPerLedger) entryLogger .getEntryLogManager(); // set same thread executor for entryLoggerAllocator's allocatorExecutor diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java index 9062261c1f2..e1f35820da4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java @@ -30,6 +30,8 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -963,8 +965,8 @@ private EntryLogger.BufferedLogChannel createDummyBufferedLogChannel(EntryLogger File tmpFile = File.createTempFile("entrylog", logid + ""); tmpFile.deleteOnExit(); FileChannel fc = new RandomAccessFile(tmpFile, "rw").getChannel(); - EntryLogger.BufferedLogChannel logChannel = new BufferedLogChannel(fc, 10, 10, logid, tmpFile, - servConf.getFlushIntervalInBytes()); + EntryLogger.BufferedLogChannel logChannel = new BufferedLogChannel(UnpooledByteBufAllocator.DEFAULT, fc, 10, 10, + logid, tmpFile, servConf.getFlushIntervalInBytes()); return logChannel; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java index 70f2a0ecb07..909e646b2c7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java @@ -29,6 +29,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; +import io.netty.buffer.UnpooledByteBufAllocator; import java.io.File; import java.io.IOException; @@ -342,7 +343,8 @@ void validateFileInfo(IndexPersistenceMgr indexPersistenceMgr, long ledgerId, in preCreateFileInfoForLedger(ledgerId, headerVersion); DigestManager digestManager = DigestManager.instantiate(ledgerId, masterKey, - BookKeeper.DigestType.toProtoDigestType(digestType), getUseV2WireProtocol); + BookKeeper.DigestType.toProtoDigestType(digestType), UnpooledByteBufAllocator.DEFAULT, + getUseV2WireProtocol); CachedFileInfo fileInfo = indexPersistenceMgr.getFileInfo(ledgerId, masterKey); fileInfo.readHeader(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java index 4fde8e74a6a..c12ed9126e5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java @@ -26,6 +26,8 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -110,7 +112,7 @@ public TestableEntryLogger( LedgerDirsManager ledgerDirsManager, EntryLogListener listener, StatsLogger statsLogger) throws IOException { - super(conf, ledgerDirsManager, listener, statsLogger); + super(conf, ledgerDirsManager, listener, statsLogger, UnpooledByteBufAllocator.DEFAULT); } void setCheckEntryTestPoint(CheckEntryListener testPoint) throws InterruptedException { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java index 232173507b6..a606f9bd756 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java @@ -28,6 +28,7 @@ import static org.junit.Assert.fail; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import java.io.File; @@ -522,7 +523,8 @@ public void initialize(ServerConfiguration conf, StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, - StatsLogger statsLogger) throws IOException { + StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { super.initialize( conf, ledgerManager, @@ -531,7 +533,8 @@ public void initialize(ServerConfiguration conf, stateManager, checkpointSource, checkpointer, - statsLogger); + statsLogger, + allocator); if (this.memTable instanceof EntryMemTableWithParallelFlusher) { this.memTable = new EntryMemTableWithParallelFlusher(conf, checkpointSource, statsLogger) { @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java index 697d7c0e88d..d028f70a644 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java @@ -23,6 +23,8 @@ import static org.junit.Assert.assertEquals; import io.netty.buffer.ByteBuf; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -145,7 +147,8 @@ public void testExplicitLacWriteToJournal(int journalFormatVersionToWrite, int f if ((journalFormatVersionToWrite >= 6) && (fileInfoFormatVersionToWrite >= 1)) { DigestManager digestManager = DigestManager.instantiate(ledgerId, passwdBytes, - BookKeeper.DigestType.toProtoDigestType(digestType), confWithExplicitLAC.getUseV2WireProtocol()); + BookKeeper.DigestType.toProtoDigestType(digestType), UnpooledByteBufAllocator.DEFAULT, + confWithExplicitLAC.getUseV2WireProtocol()); long explicitLacPersistedInJournal = digestManager.verifyDigestAndReturnLac(explicitLacBuf); assertEquals("explicitLac persisted in journal", (numOfEntries - 1), explicitLacPersistedInJournal); } else { @@ -226,7 +229,8 @@ public void testExplicitLacWriteToFileInfo(int journalFormatVersionToWrite, int if ((journalFormatVersionToWrite >= 6) && (fileInfoFormatVersionToWrite >= 1)) { DigestManager digestManager = DigestManager.instantiate(ledgerId, passwdBytes, - BookKeeper.DigestType.toProtoDigestType(digestType), confWithExplicitLAC.getUseV2WireProtocol()); + BookKeeper.DigestType.toProtoDigestType(digestType), UnpooledByteBufAllocator.DEFAULT, + confWithExplicitLAC.getUseV2WireProtocol()); long explicitLacReadFromFileInfo = digestManager.verifyDigestAndReturnLac(explicitLacBufReadFromFileInfo); assertEquals("explicitLac persisted in FileInfo", (numOfEntries - 1), explicitLacReadFromFileInfo); } else { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java index 645af9c498e..0b429ad0905 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java @@ -22,6 +22,9 @@ */ import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.IOException; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -48,7 +51,7 @@ public static class SlowEntryLogger extends EntryLogger { public SlowEntryLogger(ServerConfiguration conf, LedgerDirsManager ledgerDirsManager, EntryLogListener listener, StatsLogger statsLogger) throws IOException { - super(conf, ledgerDirsManager, listener, statsLogger); + super(conf, ledgerDirsManager, listener, statsLogger, UnpooledByteBufAllocator.DEFAULT); } public SlowEntryLogger setAddDelay(long delay) { @@ -110,10 +113,11 @@ public void initialize(ServerConfiguration conf, StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, - StatsLogger statsLogger) + StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { super.initialize(conf, ledgerManager, ledgerDirsManager, indexDirsManager, - stateManager, checkpointSource, checkpointer, statsLogger); + stateManager, checkpointSource, checkpointer, statsLogger, allocator); // do not want to add these to config class, reading throw "raw" interface long getDelay = conf.getLong(PROP_SLOW_STORAGE_GET_DELAY, 0); long addDelay = conf.getLong(PROP_SLOW_STORAGE_ADD_DELAY, 0); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java index 44f20e6f54c..48d10380355 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java @@ -26,6 +26,8 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.IOException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; @@ -141,7 +143,8 @@ public void start() { null, checkpointSrc, checkpointer, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, + UnpooledByteBufAllocator.DEFAULT); } @After diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java index 8da2ffaea93..71658e7ab7d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java @@ -22,6 +22,9 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; + +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.File; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; @@ -152,7 +155,8 @@ public void testReadOnlyBookieTransitions() throws Exception{ .setJournalDirName(tmpDir.toString()) .setMetadataServiceUri(zkUtil.getMetadataServiceUri()) .setForceReadOnlyBookie(true); - ReadOnlyBookie readOnlyBookie = new ReadOnlyBookie(readOnlyConf, NullStatsLogger.INSTANCE); + ReadOnlyBookie readOnlyBookie = new ReadOnlyBookie(readOnlyConf, NullStatsLogger.INSTANCE, + UnpooledByteBufAllocator.DEFAULT); readOnlyBookie.start(); assertTrue(readOnlyBookie.isRunning()); assertTrue(readOnlyBookie.isReadOnly()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java index d9fa8cc67e4..707eb817ee4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java @@ -26,6 +26,8 @@ import static org.junit.Assert.fail; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; + import java.io.IOException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -273,7 +275,8 @@ public void initialize( StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, - StatsLogger statsLogger) + StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionRollbackTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionRollbackTest.java index bfd7a4d712f..5c2463369a9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionRollbackTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionRollbackTest.java @@ -25,6 +25,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import java.io.File; import java.io.IOException; @@ -94,7 +95,7 @@ public void convertFromDbStorageToInterleaved() throws Exception { DbLedgerStorage dbStorage = new DbLedgerStorage(); dbStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource, checkpointer, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT); // Insert some ledger & entries in the dbStorage for (long ledgerId = 0; ledgerId < 5; ledgerId++) { @@ -124,7 +125,7 @@ public void convertFromDbStorageToInterleaved() throws Exception { // Verify that interleaved storage index has the same entries InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, - null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE); + null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT); Set ledgers = Sets.newTreeSet(interleavedStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)); Assert.assertEquals(Sets.newTreeSet(Lists.newArrayList(0L, 1L, 2L, 3L, 4L)), ledgers); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionTest.java index b2afe4c6d78..780b8ec6fe9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ConversionTest.java @@ -25,6 +25,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import java.io.File; import java.io.IOException; @@ -91,7 +92,7 @@ public void test() throws Exception { InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, - null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE); + null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT); // Insert some ledger & entries in the interleaved storage for (long ledgerId = 0; ledgerId < 5; ledgerId++) { @@ -121,11 +122,12 @@ public void test() throws Exception { // Verify that db index has the same entries DbLedgerStorage dbStorage = new DbLedgerStorage(); dbStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, - null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE); + null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT); interleavedStorage = new InterleavedLedgerStorage(); interleavedStorage.initialize(conf, null, ledgerDirsManager, - ledgerDirsManager, null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE); + ledgerDirsManager, null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, + UnpooledByteBufAllocator.DEFAULT); Set ledgers = Sets.newTreeSet(dbStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)); Assert.assertEquals(Sets.newTreeSet(Lists.newArrayList(0L, 1L, 2L, 3L, 4L)), ledgers); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java index 7a45ee764a0..e5feef30482 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.fail; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import java.io.File; @@ -58,10 +59,11 @@ private static class MockedDbLedgerStorage extends DbLedgerStorage { protected SingleDirectoryDbLedgerStorage newSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, - StatsLogger statsLogger, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize) + StatsLogger statsLogger, ScheduledExecutorService gcExecutor, + long writeCacheSize, long readCacheSize) throws IOException { return new MockedSingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager, - stateManager, checkpointSource, checkpointer, statsLogger, gcExecutor, writeCacheSize, + stateManager, checkpointSource, checkpointer, statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize); } @@ -69,9 +71,10 @@ private static class MockedSingleDirectoryDbLedgerStorage extends SingleDirector public MockedSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, StatsLogger statsLogger, - ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize) throws IOException { + ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize, + long readCacheSize) throws IOException { super(conf, ledgerManager, ledgerDirsManager, indexDirsManager, stateManager, checkpointSource, - checkpointer, statsLogger, gcExecutor, writeCacheSize, readCacheSize); + checkpointer, statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize); } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildTest.java index 629a238f283..7bdbcd518b2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildTest.java @@ -27,6 +27,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import java.io.File; import java.io.IOException; @@ -92,7 +93,7 @@ public void test() throws Exception { DbLedgerStorage ledgerStorage = new DbLedgerStorage(); ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource, checkpointer, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT); // Insert some ledger & entries in the storage for (long ledgerId = 0; ledgerId < 5; ledgerId++) { @@ -122,7 +123,7 @@ public void test() throws Exception { // Verify that db index has the same entries ledgerStorage = new DbLedgerStorage(); ledgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource, checkpointer, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT); Set ledgers = Sets.newTreeSet(ledgerStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)); Assert.assertEquals(Sets.newTreeSet(Lists.newArrayList(0L, 1L, 2L, 3L, 4L)), ledgers); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCacheTest.java index 42e509963d4..337140c2dd9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCacheTest.java @@ -25,6 +25,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import org.junit.Test; @@ -35,7 +36,7 @@ public class ReadCacheTest { @Test public void simple() { - ReadCache cache = new ReadCache(10 * 1024); + ReadCache cache = new ReadCache(UnpooledByteBufAllocator.DEFAULT, 10 * 1024); assertEquals(0, cache.count()); assertEquals(0, cache.size()); @@ -72,7 +73,7 @@ public void simple() { @Test public void emptyCache() { - ReadCache cache = new ReadCache(10 * 1024); + ReadCache cache = new ReadCache(UnpooledByteBufAllocator.DEFAULT, 10 * 1024); assertEquals(0, cache.count()); assertEquals(0, cache.size()); @@ -84,7 +85,7 @@ public void emptyCache() { @Test public void multipleSegments() { // Test with multiple smaller segments - ReadCache cache = new ReadCache(10 * 1024, 2 * 1024); + ReadCache cache = new ReadCache(UnpooledByteBufAllocator.DEFAULT, 10 * 1024, 2 * 1024); assertEquals(0, cache.count()); assertEquals(0, cache.size()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCacheTest.java index f8b2bba4463..5726bbbf8eb 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCacheTest.java @@ -26,9 +26,10 @@ import static org.junit.Assert.assertTrue; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.ByteBufUtil; -import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import java.nio.charset.Charset; import java.util.concurrent.BrokenBarrierException; @@ -46,11 +47,13 @@ */ public class WriteCacheTest { + private static final ByteBufAllocator allocator = UnpooledByteBufAllocator.DEFAULT; + @Test public void simple() throws Exception { - WriteCache cache = new WriteCache(10 * 1024); + WriteCache cache = new WriteCache(allocator, 10 * 1024); - ByteBuf entry1 = PooledByteBufAllocator.DEFAULT.buffer(1024); + ByteBuf entry1 = allocator.buffer(1024); ByteBufUtil.writeUtf8(entry1, "entry-1"); entry1.writerIndex(entry1.capacity()); @@ -87,9 +90,9 @@ public void cacheFull() throws Exception { int entrySize = 1024; int entriesCount = cacheSize / entrySize; - WriteCache cache = new WriteCache(cacheSize); + WriteCache cache = new WriteCache(allocator, cacheSize); - ByteBuf entry = PooledByteBufAllocator.DEFAULT.buffer(entrySize); + ByteBuf entry = allocator.buffer(entrySize); entry.writerIndex(entry.capacity()); for (int i = 0; i < entriesCount; i++) { @@ -125,7 +128,7 @@ public void cacheFull() throws Exception { @Test public void testMultipleSegments() { // Create cache with max size 1Mb and each segment is 16Kb - WriteCache cache = new WriteCache(1024 * 1024, 16 * 1024); + WriteCache cache = new WriteCache(allocator, 1024 * 1024, 16 * 1024); ByteBuf entry = Unpooled.buffer(1024); entry.writerIndex(entry.capacity()); @@ -142,7 +145,7 @@ public void testMultipleSegments() { @Test public void testEmptyCache() { - WriteCache cache = new WriteCache(1024 * 1024, 16 * 1024); + WriteCache cache = new WriteCache(allocator, 1024 * 1024, 16 * 1024); assertEquals(0, cache.count()); assertEquals(0, cache.size()); @@ -160,7 +163,7 @@ public void testEmptyCache() { @Test public void testMultipleWriters() throws Exception { // Create cache with max size 1Mb and each segment is 16Kb - WriteCache cache = new WriteCache(10 * 1024 * 1024, 16 * 1024); + WriteCache cache = new WriteCache(allocator, 10 * 1024 * 1024, 16 * 1024); ExecutorService executor = Executors.newCachedThreadPool(); @@ -220,7 +223,7 @@ public void testMultipleWriters() throws Exception { @Test public void testLedgerDeletion() { - WriteCache cache = new WriteCache(1024 * 1024, 16 * 1024); + WriteCache cache = new WriteCache(allocator, 1024 * 1024, 16 * 1024); ByteBuf entry = Unpooled.buffer(1024); entry.writerIndex(entry.capacity()); @@ -265,7 +268,7 @@ public void testLedgerDeletion() { @Test public void testWriteReadsInMultipleSegments() { // Create cache with max size 4 KB and each segment is 128 bytes - WriteCache cache = new WriteCache(4 * 1024, 128); + WriteCache cache = new WriteCache(allocator, 4 * 1024, 128); for (int i = 0; i < 48; i++) { boolean inserted = cache.put(1, i, Unpooled.wrappedBuffer(("test-" + i).getBytes())); 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 7be404d0d17..cb45ba77e8d 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 @@ -587,6 +587,7 @@ public void testReadWriteWithV2WireProtocol() throws Exception { } } + @SuppressWarnings("deprecation") @Test public void testReadEntryReleaseByteBufs() throws Exception { ClientConfiguration confWriter = new ClientConfiguration(); 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 253f0d9091c..e1d32af4638 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 @@ -20,6 +20,8 @@ */ package org.apache.bookkeeper.client; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.io.IOException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; @@ -45,7 +47,7 @@ public class BookKeeperTestClient extends BookKeeper { public BookKeeperTestClient(ClientConfiguration conf, TestStatsProvider statsProvider) throws IOException, InterruptedException, BKException { - super(conf, null, null, + super(conf, null, null, new UnpooledByteBufAllocator(false), statsProvider == null ? NullStatsLogger.INSTANCE : statsProvider.getStatsLogger(""), null, null, null); this.statsProvider = statsProvider; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java index 68a68463bb6..a6b873ed08a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java @@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import java.security.GeneralSecurityException; import java.util.function.Function; @@ -47,7 +48,8 @@ public static ByteBuf generatePacket(long ledgerId, long entryId, long lastAddCo public static ByteBuf generatePacket(long ledgerId, long entryId, long lastAddConfirmed, long length, byte[] data, int offset, int len) throws GeneralSecurityException { - DigestManager dm = DigestManager.instantiate(ledgerId, new byte[2], DigestType.CRC32); + DigestManager dm = DigestManager.instantiate(ledgerId, new byte[2], DigestType.CRC32, + UnpooledByteBufAllocator.DEFAULT, true); return ByteBufList.coalesce(dm.computeDigestAndPackageForSending(entryId, lastAddConfirmed, length, Unpooled.wrappedBuffer(data, offset, len))); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java index 6d6e4d15a26..69b56352240 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java @@ -31,7 +31,9 @@ import static org.mockito.Mockito.when; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import java.security.GeneralSecurityException; import java.util.ArrayList; @@ -210,6 +212,11 @@ public BookKeeperClientStats getClientStats() { public boolean isClientClosed() { return bk.isClosed(); } + + @Override + public ByteBufAllocator getByteBufAllocator() { + return UnpooledByteBufAllocator.DEFAULT; + } }; when(bk.getClientCtx()).thenReturn(clientCtx); when(bk.getLedgerManager()).thenReturn(ledgerManager); @@ -241,7 +248,8 @@ private DigestManager getDigestType(long ledgerId) throws GeneralSecurityExcepti metadata.getPassword(), org.apache.bookkeeper.client.BookKeeper.DigestType.toProtoDigestType( org.apache.bookkeeper.client.BookKeeper.DigestType.fromApiDigestType( - metadata.getDigestType()))); + metadata.getDigestType())), + UnpooledByteBufAllocator.DEFAULT, false); } @After diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java index f36c00885bf..f0be8d02a3b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java @@ -22,6 +22,9 @@ import static com.google.common.base.Preconditions.checkState; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.util.function.BooleanSupplier; import org.apache.bookkeeper.common.util.OrderedExecutor; @@ -50,6 +53,7 @@ public class MockClientContext implements ClientContext { private BookKeeperClientStats clientStats; private BooleanSupplier isClientClosed; private MockRegistrationClient regClient; + private ByteBufAllocator allocator; static MockClientContext create() throws Exception { ClientConfiguration conf = new ClientConfiguration(); @@ -67,6 +71,7 @@ static MockClientContext create() throws Exception { .setPlacementPolicy(placementPolicy) .setRegistrationClient(regClient) .setBookieClient(new MockBookieClient(scheduler)) + .setByteBufAllocator(UnpooledByteBufAllocator.DEFAULT) .setMainWorkerPool(scheduler) .setScheduler(scheduler) .setClientStats(BookKeeperClientStats.newInstance(NullStatsLogger.INSTANCE)) @@ -83,6 +88,7 @@ static MockClientContext copyOf(ClientContext other) { .setMainWorkerPool(other.getMainWorkerPool()) .setScheduler(other.getScheduler()) .setClientStats(other.getClientStats()) + .setByteBufAllocator(other.getByteBufAllocator()) .setIsClientClosed(other::isClientClosed); } @@ -151,6 +157,11 @@ public MockClientContext setRegistrationClient(MockRegistrationClient regClient) return this; } + public MockClientContext setByteBufAllocator(ByteBufAllocator allocator) { + this.allocator = allocator; + return this; + } + private static T maybeSpy(T orig) { if (Mockito.mockingDetails(orig).isSpy()) { return orig; @@ -204,4 +215,8 @@ public boolean isClientClosed() { return isClientClosed.getAsBoolean(); } + @Override + public ByteBufAllocator getByteBufAllocator() { + return allocator; + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java index a1032a16e0d..e1c203d2b96 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java @@ -32,6 +32,8 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -132,7 +134,7 @@ public void setup() throws Exception { when(mockLh.getCurrentEnsemble()).thenReturn(ensemble); when(mockLh.getLedgerMetadata()).thenReturn(ledgerMetadata); when(mockLh.getDistributionSchedule()).thenReturn(distributionSchedule); - digestManager = new DummyDigestManager(LEDGERID, false); + digestManager = new DummyDigestManager(LEDGERID, false, UnpooledByteBufAllocator.DEFAULT); when(mockLh.getDigestManager()).thenReturn(digestManager); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java index 4ec5992a391..39c615a63a4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue; +import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.concurrent.DefaultThreadFactory; @@ -108,7 +109,8 @@ public void testGetBookieInfoTimeout() throws Exception { // try to get bookie info from the sleeping bookie. It should fail with timeout error BookieSocketAddress addr = new BookieSocketAddress(bookieToSleep.getSocketAddress().getHostString(), bookieToSleep.getPort()); - BookieClient bc = new BookieClientImpl(cConf, eventLoopGroup, executor, scheduler, NullStatsLogger.INSTANCE); + BookieClient bc = new BookieClientImpl(cConf, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, executor, + scheduler, NullStatsLogger.INSTANCE); long flags = BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE | BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java index 9073b01ccc9..f993e206427 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/TestBKConfiguration.java @@ -27,6 +27,7 @@ import java.util.Enumeration; import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; +import org.apache.bookkeeper.common.allocator.PoolingPolicy; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,6 +55,7 @@ public static ServerConfiguration newServerConfiguration() { confReturn.setGcWaitTime(1000); confReturn.setDiskUsageThreshold(0.999f); confReturn.setDiskUsageWarnThreshold(0.99f); + confReturn.setAllocatorPoolingPolicy(PoolingPolicy.UnpooledHeap); confReturn.setProperty(DbLedgerStorage.WRITE_CACHE_MAX_SIZE_MB, 4); confReturn.setProperty(DbLedgerStorage.READ_AHEAD_CACHE_MAX_SIZE_MB, 4); setLoopbackInterfaceAndAllowLoopback(confReturn); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 828956d4f35..31bd406d489 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -30,6 +30,8 @@ import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; + import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -48,6 +50,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; + import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.CheckpointSource; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; @@ -566,7 +569,8 @@ public void initialize( StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, - StatsLogger statsLogger) throws IOException { + StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java index 3aa2b8ac562..cf9b3ddb6c2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java @@ -22,6 +22,8 @@ package org.apache.bookkeeper.meta; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; + import java.io.IOException; import java.net.URI; import java.util.Arrays; @@ -29,6 +31,7 @@ import java.util.Map; import java.util.NavigableMap; import java.util.Optional; + import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.CheckpointSource; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; @@ -173,7 +176,8 @@ public void initialize( StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, - StatsLogger statsLogger) throws IOException { + StatsLogger statsLogger, + ByteBufAllocator allocator) throws IOException { } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieBackpressureTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieBackpressureTest.java index 5454a702dfe..4b13ebe94ce 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieBackpressureTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieBackpressureTest.java @@ -25,6 +25,8 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; +import io.netty.buffer.UnpooledByteBufAllocator; + import java.lang.reflect.Field; import java.nio.channels.FileChannel; import java.util.Enumeration; @@ -130,7 +132,7 @@ private void mockJournal(Bookie bookie, long getDelay, long addDelay, long flush for (int i = 0; i < journals.size(); i++) { Journal mock = spy(journals.get(i)); when(mock.getBufferedChannelBuilder()).thenReturn((FileChannel fc, int capacity) -> { - SlowBufferedChannel sbc = new SlowBufferedChannel(fc, capacity); + SlowBufferedChannel sbc = new SlowBufferedChannel(UnpooledByteBufAllocator.DEFAULT, fc, capacity); sbc.setAddDelay(addDelay); sbc.setGetDelay(getDelay); sbc.setFlushDelay(flushDelay); @@ -306,7 +308,7 @@ private BookieRequestProcessor generateDataAndDoReads(final int bkId) throws Exc BookieServer bks = bs.get(bkId); bks.shutdown(); bks = new BookieServer(bsConfs.get(bkId)); - mockJournal(bks.bookie, getDelay, addDelay, flushDelay); + mockJournal(bks.getBookie(), getDelay, addDelay, flushDelay); bks.start(); bs.set(bkId, bks); @@ -347,7 +349,7 @@ private void doWritesNoBackpressure(final int bkId) throws Exception { BookieServer bks = bs.get(bkId); bks.shutdown(); bks = new BookieServer(bsConfs.get(bkId)); - mockJournal(bks.bookie, getDelay, addDelay, flushDelay); + mockJournal(bks.getBookie(), getDelay, addDelay, flushDelay); bks.start(); bs.set(bkId, bks); @@ -392,7 +394,7 @@ private void doWritesWithBackpressure(final int bkId) throws Exception { BookieServer bks = bs.get(bkId); bks.shutdown(); bks = new BookieServer(bsConfs.get(bkId)); - mockJournal(bks.bookie, getDelay, addDelay, flushDelay); + mockJournal(bks.getBookie(), getDelay, addDelay, flushDelay); bks.start(); bs.set(bkId, bks); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java index 04119d51543..2c349a01185 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java @@ -24,6 +24,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import java.util.Collections; import java.util.EnumSet; @@ -109,7 +110,8 @@ public void removeErrors(BookieSocketAddress... bookies) { } public void seedEntries(BookieSocketAddress bookie, long ledgerId, long entryId, long lac) throws Exception { - DigestManager digestManager = DigestManager.instantiate(ledgerId, new byte[0], DigestType.CRC32C); + DigestManager digestManager = DigestManager.instantiate(ledgerId, new byte[0], DigestType.CRC32C, + UnpooledByteBufAllocator.DEFAULT, false); ByteBuf entry = ByteBufList.coalesce(digestManager.computeDigestAndPackageForSending( entryId, lac, 0, Unpooled.buffer(10))); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java index 4ebe01cfccb..4cf8a7cd64f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java @@ -29,6 +29,9 @@ import static org.mockito.Mockito.mock; import com.google.protobuf.ByteString; + +import io.netty.buffer.UnpooledByteBufAllocator; + import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.proto.BookkeeperProtocol.AddRequest; @@ -54,7 +57,7 @@ public void testConstructLongPollThreads() throws Exception { // long poll threads == read threads ServerConfiguration conf = new ServerConfiguration(); try (BookieRequestProcessor processor = new BookieRequestProcessor( - conf, mock(Bookie.class), NullStatsLogger.INSTANCE, null)) { + conf, mock(Bookie.class), NullStatsLogger.INSTANCE, null, UnpooledByteBufAllocator.DEFAULT)) { assertSame(processor.getReadThreadPool(), processor.getLongPollThreadPool()); } @@ -62,7 +65,7 @@ conf, mock(Bookie.class), NullStatsLogger.INSTANCE, null)) { conf = new ServerConfiguration(); conf.setNumReadWorkerThreads(0); try (BookieRequestProcessor processor = new BookieRequestProcessor( - conf, mock(Bookie.class), NullStatsLogger.INSTANCE, null)) { + conf, mock(Bookie.class), NullStatsLogger.INSTANCE, null, UnpooledByteBufAllocator.DEFAULT)) { assertNull(processor.getReadThreadPool()); assertNotNull(processor.getLongPollThreadPool()); } @@ -72,7 +75,7 @@ conf, mock(Bookie.class), NullStatsLogger.INSTANCE, null)) { conf.setNumReadWorkerThreads(2); conf.setNumLongPollWorkerThreads(2); try (BookieRequestProcessor processor = new BookieRequestProcessor( - conf, mock(Bookie.class), NullStatsLogger.INSTANCE, null)) { + conf, mock(Bookie.class), NullStatsLogger.INSTANCE, null, UnpooledByteBufAllocator.DEFAULT)) { assertNotNull(processor.getReadThreadPool()); assertNotNull(processor.getLongPollThreadPool()); assertNotSame(processor.getReadThreadPool(), processor.getLongPollThreadPool()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index b1feef059a5..97b7488b78d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -21,10 +21,14 @@ package org.apache.bookkeeper.test; + + import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.junit.Assert.assertTrue; import com.google.common.base.Stopwatch; +import io.netty.buffer.ByteBufAllocator; + import java.io.File; import java.io.IOException; import java.net.UnknownHostException; @@ -41,6 +45,7 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.client.BookKeeperTestClient; +import org.apache.bookkeeper.common.allocator.PoolingPolicy; import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -237,6 +242,7 @@ protected void stopZKCluster() throws Exception { protected void startBKCluster(String metadataServiceUri) throws Exception { baseConf.setMetadataServiceUri(metadataServiceUri); baseClientConf.setMetadataServiceUri(metadataServiceUri); + baseClientConf.setAllocatorPoolingPolicy(PoolingPolicy.UnpooledHeap); if (numBookies > 0) { bkc = new BookKeeperTestClient(baseClientConf, new TestStatsProvider()); } @@ -303,6 +309,7 @@ protected ServerConfiguration newServerConfiguration(int port, File journalDir, } conf.setLedgerDirNames(ledgerDirNames); conf.setEnableTaskExecutionStats(true); + conf.setAllocatorPoolingPolicy(PoolingPolicy.UnpooledHeap); return conf; } @@ -677,7 +684,7 @@ protected BookieServer startBookie(ServerConfiguration conf, final Bookie b) TestStatsProvider provider = new TestStatsProvider(); BookieServer server = new BookieServer(conf, provider.getStatsLogger("")) { @Override - protected Bookie newBookie(ServerConfiguration conf) { + protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) { return b; } }; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java index c6cc72bcd8a..d38a178b779 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java @@ -25,6 +25,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.concurrent.DefaultThreadFactory; @@ -157,8 +158,8 @@ public void testWriteGaps() throws Exception { BookieSocketAddress addr = bs.getLocalAddress(); ResultStruct arc = new ResultStruct(); - BookieClient bc = new BookieClientImpl(new ClientConfiguration(), eventLoopGroup, executor, - scheduler, NullStatsLogger.INSTANCE); + BookieClient bc = new BookieClientImpl(new ClientConfiguration(), eventLoopGroup, + UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE); ByteBufList bb = createByteBuffer(1, 1, 1); bc.addEntry(addr, 1, passwd, 1, bb, wrcb, arc, BookieProtocol.FLAG_NONE, false, WriteFlag.NONE); synchronized (arc) { @@ -258,8 +259,8 @@ private ByteBufList createByteBuffer(int i, long lid, long eid) { public void testNoLedger() throws Exception { ResultStruct arc = new ResultStruct(); BookieSocketAddress addr = bs.getLocalAddress(); - BookieClient bc = new BookieClientImpl(new ClientConfiguration(), eventLoopGroup, executor, - scheduler, NullStatsLogger.INSTANCE); + BookieClient bc = new BookieClientImpl(new ClientConfiguration(), eventLoopGroup, + UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE); synchronized (arc) { bc.readEntry(addr, 2, 13, recb, arc, BookieProtocol.FLAG_NONE); arc.wait(1000); @@ -270,8 +271,8 @@ public void testNoLedger() throws Exception { @Test public void testGetBookieInfo() throws IOException, InterruptedException { BookieSocketAddress addr = bs.getLocalAddress(); - BookieClient bc = new BookieClientImpl(new ClientConfiguration(), new NioEventLoopGroup(), executor, - scheduler, NullStatsLogger.INSTANCE); + BookieClient bc = new BookieClientImpl(new ClientConfiguration(), new NioEventLoopGroup(), + UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE); long flags = BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE | BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; diff --git a/conf/bk_server.conf b/conf/bk_server.conf index 23012dc4dd8..41798a6df4b 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -965,3 +965,53 @@ storage.serve.readonly.tables=false # the cluster controller schedule interval, in milliseconds. default is 30 seconds. storage.cluster.controller.schedule.interval.ms=30000 + + +############################################################################# +## Netty Allocator Settings +############################################################################# + +# Define the memory pooling policy. +# Available options are: +# - PooledDirect: Use Direct memory for all buffers and pool the memory. +# Direct memory will avoid the overhead of JVM GC and most +# memory copies when reading and writing to socket channel. +# Pooling will add memory space overhead due to the fact that +# there will be fragmentation in the allocator and that threads +# will keep a portion of memory as thread-local to avoid +# contention when possible. +# - UnpooledHeap: Allocate memory from JVM heap without any pooling. +# This option has the least overhead in terms of memory usage +# since the memory will be automatically reclaimed by the +# JVM GC but might impose a performance penalty at high +# throughput. +# Default is: PooledDirect +# allocatorPoolingPolicy=PooledDirect + +# Controls the amount of concurrency for the memory pool. +# Default is to have a number of allocator arenas equals to 2 * CPUS. +# Decreasing this number will reduce the amount of memory overhead, at the +# expense of increased allocation contention. +# allocatorPoolingConcurrency=8 + +# Define the memory allocator out of memory policy. +# Available options are: +# - FallbackToHeap: If it's not possible to allocate a buffer from direct memory, +# fallback to allocate an unpooled buffer from JVM heap. +# This will help absorb memory allocation spikes because the heap +# allocations will naturally slow down the process and will result +# if full GC cleanup if the Heap itself is full. +# - ThrowException: Throw regular OOM exception without taking addition actions. +# Default is: FallbackToHeap +# allocatorOutOfMemoryPolicy=FallbackToHeap + +# Available options are: +# - Disabled: No leak detection and no overhead. +# - Simple: Instruments 1% of the allocated buffer to track for leaks. +# - Advanced: Instruments 1% of the allocated buffer to track for leaks, reporting +# stack traces of places where the buffer was used. +# - Paranoid: Instruments 100% of the allocated buffer to track for leaks, reporting +# stack traces of places where the buffer was used. Introduce very +# significant overhead. +# Default is: Disabled +# allocatorLeakDetectionPolicy=Disabled diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java index 3995ea8875c..2319b2ea3eb 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java @@ -24,6 +24,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; @@ -106,13 +107,13 @@ public static class MyState { public void doSetup() throws Exception { final byte[] password = "password".getBytes("UTF-8"); crc32 = DigestManager.instantiate(ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE), - password, DigestType.CRC32); + password, DigestType.CRC32, PooledByteBufAllocator.DEFAULT, true); crc32c = DigestManager.instantiate(ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE), - password, DigestType.CRC32C); + password, DigestType.CRC32C, PooledByteBufAllocator.DEFAULT, true); mac = DigestManager.instantiate(ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE), - password, DigestType.HMAC); + password, DigestType.HMAC, PooledByteBufAllocator.DEFAULT, true); digestBuf = Unpooled.buffer(getDigestManager(digest).getMacCodeLength()); diff --git a/shaded/bookkeeper-server-shaded/pom.xml b/shaded/bookkeeper-server-shaded/pom.xml index a39227bea10..85c76beba46 100644 --- a/shaded/bookkeeper-server-shaded/pom.xml +++ b/shaded/bookkeeper-server-shaded/pom.xml @@ -66,6 +66,7 @@ com.google.guava:guava com.google.protobuf:protobuf-java org.apache.bookkeeper:bookkeeper-common + org.apache.bookkeeper:bookkeeper-common-allocator org.apache.bookkeeper:cpu-affinity org.apache.bookkeeper:bookkeeper-tools-framework org.apache.bookkeeper:bookkeeper-proto @@ -83,7 +84,7 @@ - + org.codehaus.mojo license-maven-plugin diff --git a/shaded/distributedlog-core-shaded/pom.xml b/shaded/distributedlog-core-shaded/pom.xml index e5ef59f80b3..85ad8f0b5cd 100644 --- a/shaded/distributedlog-core-shaded/pom.xml +++ b/shaded/distributedlog-core-shaded/pom.xml @@ -87,6 +87,7 @@ net.java.dev.jna:jna net.jpountz.lz4:lz4 org.apache.bookkeeper:bookkeeper-common + org.apache.bookkeeper:bookkeeper-common-allocator org.apache.bookkeeper:cpu-affinity org.apache.bookkeeper:bookkeeper-tools-framework org.apache.bookkeeper:bookkeeper-proto @@ -207,7 +208,7 @@ - + org.codehaus.mojo license-maven-plugin diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index a42e12eb2b7..d83a8f1024a 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -137,7 +137,7 @@ groups: default: 8080 - name: Security settings - params: + params: - param: bookieAuthProviderFactoryClass description: The bookie authentication provider factory class name. If this is null, no authentication will take place. default: null @@ -207,7 +207,7 @@ groups: 5: expanding header to 512 and padding writes to align sector size configured by `journalAlignmentSize` 6: persisting explicitLac is introduced - By default, it is `6`. + By default, it is `6`. If you'd like to disable persisting ExplicitLac, you can set this config to < `6` and also fileInfoFormatVersionToWrite should be set to 0. If there is mismatch then the serverconfig is considered invalid. You can disable `padding-writes` by setting journal version back to `4`. This feature is available in 4.5.0 and onward versions. default: 6 @@ -618,7 +618,7 @@ groups: - param: ensemblePlacementPolicy description: | The ensemble placement policy used for finding bookie for re-replicating entries. - + Options: - org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy - org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy @@ -678,4 +678,55 @@ groups: description: The time to backoff when replication worker encounters exceptions on replicating a ledger, in milliseconds. default: 5000 - +- name: Memory allocator settings + params: + - param: allocatorPoolingPolicy + description: | + Define the memory pooling policy. + + Available options are: + - PooledDirect: Use Direct memory for all buffers and pool the memory. + Direct memory will avoid the overhead of JVM GC and most + memory copies when reading and writing to socket channel. + Pooling will add memory space overhead due to the fact that + there will be fragmentation in the allocator and that threads + will keep a portion of memory as thread-local to avoid + contention when possible. + - UnpooledHeap: Allocate memory from JVM heap without any pooling. + This option has the least overhead in terms of memory usage + since the memory will be automatically reclaimed by the + JVM GC but might impose a performance penalty at high + throughput. + default: PooledDirect + - param: allocatorPoolingConcurrency + description: | + Controls the amount of concurrency for the memory pool. + Default is to have a number of allocator arenas equals to 2 * CPUS. + Decreasing this number will reduce the amount of memory overhead, at the + expense of increased allocation contention. + default: 2 * CPUS + - param: allocatorOutOfMemoryPolicy + description: | + Define the memory allocator out of memory policy. + + Available options are: + - FallbackToHeap: If it's not possible to allocate a buffer from direct memory, + fallback to allocate an unpooled buffer from JVM heap. + This will help absorb memory allocation spikes because the heap + allocations will naturally slow down the process and will result + if full GC cleanup if the Heap itself is full. + - ThrowException: Throw regular OOM exception without taking addition actions. + default: FallbackToHeap + - param: allocatorLeakDetectionPolicy + description: | + Define the memory allocator leak detection policy. + + Available options are: + - Disabled: No leak detection and no overhead. + - Simple: Instruments 1% of the allocated buffer to track for leaks. + - Advanced: Instruments 1% of the allocated buffer to track for leaks, reporting + stack traces of places where the buffer was used. + - Paranoid: Instruments 100% of the allocated buffer to track for leaks, reporting + stack traces of places where the buffer was used. Introduce very + significant overhead. + default: Disabled From 29485fc4ef5ca170d88e2cfc9c447a11d6b24e9d Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 8 Jan 2019 16:12:48 -0800 Subject: [PATCH 0188/1642] Allow to configure sticky reads ### Motivation Currently the BK client is issuing the read requests in round-robin fashion across all the bookies in the write set. One issue with this approach is that it's not taking full advantage of the read-ahead cache, either explicit (like in `DbLedgerStorage`) or implicit (by reading data through Linux page cache which will do some prefetching). With `e=2`, `w=2`, when we read `e-0` from `bookie-1` and `e-1` from `bookie-2`, we fail to take advantage of the fact that `bookie-1` will have already `e-1` in memory. Effectively with `e-2`, `w-2` the disk read IO will be doubled, compared to the amount of data served to BK clients. The larger the quorum, the bigger will be overhead (eg: `e=5`, `w=5` will lead to 5x reads from disk). ### Changes Added a BK client flag for "sticky reads". When reading from a ledger that has `E=W` (every bookie has all the entries), the sticky reads will direct all read request to 1 single bookie in the ensemble. Reviewers: Enrico Olivelli , Sijie Guo This closes #1808 from merlimat/single-bookie-per-ledger-read-master --- .../bookkeeper/client/ClientInternalConf.java | 2 + .../client/EnsemblePlacementPolicy.java | 35 +++ .../bookkeeper/client/LedgerHandle.java | 65 ++++++ .../bookkeeper/client/PendingReadOp.java | 9 +- .../bookkeeper/conf/ClientConfiguration.java | 29 +++ .../bookie/BookieStickyReadsTest.java | 213 ++++++++++++++++++ 6 files changed, 350 insertions(+), 3 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStickyReadsTest.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java index ac56a1fbfb8..da79108deb5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java @@ -41,6 +41,7 @@ class ClientInternalConf { final long addEntryQuorumTimeoutNanos; final boolean enableParallelRecoveryRead; final boolean enableReorderReadSequence; + final boolean enableStickyReads; final int recoveryReadBatchSize; final int throttleValue; final int bookieFailureHistoryExpirationMSec; @@ -80,6 +81,7 @@ private ClientInternalConf(ClientConfiguration conf, this.timeoutMonitorIntervalSec = conf.getTimeoutMonitorIntervalSec(); this.enableBookieFailureTracking = conf.getEnableBookieFailureTracking(); this.useV2WireProtocol = conf.getUseV2WireProtocol(); + this.enableStickyReads = conf.isStickyReadsEnabled(); if (conf.getFirstSpeculativeReadTimeout() > 0) { this.readSpeculativeRequestPolicy = 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 e185964d36b..23932a3d7df 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 @@ -22,9 +22,12 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.DistributionSchedule.WriteSet; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -351,4 +354,36 @@ DistributionSchedule.WriteSet reorderReadLACSequence( */ default void updateBookieInfo(Map bookieInfoMap) { } + + /** + * Select one bookie to the "sticky" bookie where all reads for a particular + * ledger will be directed to. + * + *

            The default implementation will pick a bookie randomly from the ensemble. + * Other placement policies will be able to do better decisions based on + * additional informations (eg: rack or region awareness). + * + * @param metadata + * the {@link LedgerMetadata} object + * @param currentStickyBookieIndex + * if we are changing the sticky bookie after a read failure, the + * current sticky bookie is passed in so that we will avoid + * choosing it again + * @return the index, within the ensemble of the bookie chosen as the sticky + * bookie + * + * @since 4.9 + */ + default int getStickyReadBookieIndex(LedgerMetadata metadata, Optional currentStickyBookieIndex) { + if (!currentStickyBookieIndex.isPresent()) { + // Pick one bookie randomly from the current ensemble as the initial + // "sticky bookie" + return ThreadLocalRandom.current().nextInt() % metadata.getEnsembleSize(); + } else { + // When choosing a new sticky bookie index (eg: after the current + // one has read failures), by default we pick the next one in the + // ensemble, to avoid picking up the same one again. + return (currentStickyBookieIndex.get() + 1) % metadata.getEnsembleSize(); + } + } } 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 bd8ec68314c..7e21f975230 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 @@ -42,6 +42,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Queue; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -58,6 +59,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback; import org.apache.bookkeeper.client.BKException.BKIncorrectParameterException; import org.apache.bookkeeper.client.BKException.BKReadException; +import org.apache.bookkeeper.client.DistributionSchedule.WriteSet; import org.apache.bookkeeper.client.SyncCallbackUtils.FutureReadLastConfirmed; import org.apache.bookkeeper.client.SyncCallbackUtils.FutureReadLastConfirmedAndEntry; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncAddCallback; @@ -120,6 +122,18 @@ private enum HandleState { */ volatile long pendingAddsSequenceHead; + /** + * If bookie sticky reads are enabled, this will contain the index of the bookie + * selected as "sticky" for this ledger. The bookie is chosen at random when the + * LedgerHandle is created. + * + *

            In case of failures, the bookie index will be updated (to the next bookie in + * the ensemble) to avoid continuing to attempt to read from a failed bookie. + * + *

            If the index is -1, it means the sticky reads are disabled. + */ + private int stickyBookieIndex; + long length; final DigestManager macManager; final DistributionSchedule distributionSchedule; @@ -181,6 +195,13 @@ private enum HandleState { this.ledgerId = ledgerId; + if (clientCtx.getConf().enableStickyReads + && getLedgerMetadata().getEnsembleSize() == getLedgerMetadata().getWriteQuorumSize()) { + stickyBookieIndex = clientCtx.getPlacementPolicy().getStickyReadBookieIndex(metadata, Optional.empty()); + } else { + stickyBookieIndex = -1; + } + if (clientCtx.getConf().throttleValue > 0) { this.throttler = RateLimiter.create(clientCtx.getConf().throttleValue); } else { @@ -237,6 +258,21 @@ public Integer getSample() { initializeWriteHandleState(); } + /** + * Notify the LedgerHandle that a read operation was failed on a particular bookie. + */ + void recordReadErrorOnBookie(int bookieIndex) { + // If sticky bookie reads are enabled, switch the sticky bookie to the + // next bookie in the ensemble so that we avoid to keep reading from the + // same failed bookie + if (stickyBookieIndex != -1) { + // This will be idempotent when we have multiple read errors on the + // same bookie. The net result is that we just go to the next bookie + stickyBookieIndex = clientCtx.getPlacementPolicy().getStickyReadBookieIndex(getLedgerMetadata(), + Optional.of(bookieIndex)); + } + } + protected void initializeWriteHandleState() { if (clientCtx.getConf().explicitLacInterval > 0) { explicitLacFlushPolicy = new ExplicitLacFlushPolicy.ExplicitLacFlushPolicyImpl( @@ -1978,4 +2014,33 @@ List getCurrentEnsemble() { // becomes a property of the LedgerHandle itself. return LedgerMetadataUtils.getCurrentEnsemble(versionedMetadata.getValue()); } + + /** + * Return a {@link WriteSet} suitable for reading a particular entry. + * This will include all bookies that are cotna + */ + WriteSet getWriteSetForReadOperation(long entryId) { + if (stickyBookieIndex != -1) { + // When sticky reads are enabled we want to make sure to take + // advantage of read-ahead (or, anyway, from efficiencies in + // reading sequential data from disk through the page cache). + // For this, all the entries that a given bookie prefetches, + // should read from that bookie. + // For example, with e=2, w=2, a=2 we would have + // B-1 B-2 + // e-0 X X + // e-1 X X + // e-2 X X + // + // In this case we want all the requests to be issued to B-1 (by + // preference), so that cache hits will be maximized. + // + // We can only enable sticky reads if the ensemble==writeQuorum + // otherwise the same bookie will not have all the entries + // stored + return distributionSchedule.getWriteSet(stickyBookieIndex); + } else { + return distributionSchedule.getWriteSet(entryId); + } + } } 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 e8206602e93..65a3d76f9ab 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 @@ -98,12 +98,13 @@ abstract class LedgerEntryRequest implements SpeculativeRequestExecutor, AutoClo this.eId = eId; if (clientCtx.getConf().enableReorderReadSequence) { - writeSet = clientCtx.getPlacementPolicy().reorderReadSequence( + writeSet = clientCtx.getPlacementPolicy() + .reorderReadSequence( ensemble, lh.getBookiesHealthInfo(), - lh.distributionSchedule.getWriteSet(eId)); + lh.getWriteSetForReadOperation(eId)); } else { - writeSet = lh.distributionSchedule.getWriteSet(eId); + writeSet = lh.getWriteSetForReadOperation(eId); } } @@ -209,6 +210,8 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress errMsg, lh.ledgerId, eId, host); } } + + lh.recordReadErrorOnBookie(bookieIndex); } /** 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 3b390d48e41..29c6820dcf1 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 @@ -114,6 +114,7 @@ public class ClientConfiguration extends AbstractConfigurationIf this flag is enabled, the client will use one single bookie (by + * preference) to read all entries for a ledger. + * + *

            Having all the read to one bookie will increase the chances that + * a read request will be fullfilled by Bookie read cache (or OS file + * system cache) when doing sequential reads. + * + * @param enabled the flag to enable/disable sticky reads. + * @return client configuration instance. + */ + public ClientConfiguration setStickyReadsEnabled(boolean enabled) { + setProperty(STICKY_READS_ENABLED, enabled); + return this; + } + /** * Get Ensemble Placement Policy Class. * diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStickyReadsTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStickyReadsTest.java new file mode 100644 index 00000000000..a23b0e5bcd8 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStickyReadsTest.java @@ -0,0 +1,213 @@ +/* + * + * 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 org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.beust.jcommander.internal.Lists; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; + +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.client.api.WriteHandle; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.junit.Test; + +/** + * Tests of the main BookKeeper client. + */ +@Slf4j +public class BookieStickyReadsTest extends BookKeeperClusterTestCase { + + private static final int NUM_BOOKIES = 3; + + private static final String READ_ENTRY_REQUEST_METRIC = "bookkeeper_server.READ_ENTRY_REQUEST"; + + public BookieStickyReadsTest() { + super(NUM_BOOKIES); + } + + @Test + public void testNormalReads() throws Exception { + ClientConfiguration conf = new ClientConfiguration(baseClientConf); + + // Default should already be set to false + // conf.setStickyReadsEnabled(false); + + writeAndReadEntries(conf, 3, 3, 3); + + // All bookies should have received at least some read request + getBookieReadRequestStats().values().forEach(readRequests -> assertTrue(readRequests > 0)); + } + + @Test + public void testStickyFlagWithStriping() throws Exception { + ClientConfiguration conf = new ClientConfiguration(baseClientConf); + conf.setStickyReadsEnabled(true); + + writeAndReadEntries(conf, 3, 2, 2); + + // All bookies should have received at least some read request since we + // don't enable sticky reads when striping is enabled + getBookieReadRequestStats().values().forEach(readRequests -> assertTrue(readRequests > 0)); + } + + @Test + public void stickyReadsWithNoFailures() throws Exception { + ClientConfiguration conf = new ClientConfiguration(baseClientConf); + conf.setStickyReadsEnabled(true); + + writeAndReadEntries(conf, 3, 3, 3); + + // All read requests should have been made to a single bookie + Map stats = getBookieReadRequestStats(); + boolean foundBookieWithRequests = false; + for (long readRequests : stats.values()) { + if (readRequests > 0) { + assertFalse("Another bookie already had received requests", foundBookieWithRequests); + foundBookieWithRequests = true; + } + } + } + + @Test + public void stickyReadsWithFailures() throws Exception { + ClientConfiguration conf = new ClientConfiguration(baseClientConf); + conf.setStickyReadsEnabled(true); + + @Cleanup + BookKeeper bkc = new BookKeeper(conf); + + final int n = 10; + long ledgerId; + + try (WriteHandle wh = bkc.newCreateLedgerOp() + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(3) + .withPassword("".getBytes()) + .execute() + .join()) { + ledgerId = wh.getId(); + + for (int i = 0; i < n; i++) { + wh.append(("entry-" + i).getBytes()); + } + } + + @Cleanup + ReadHandle rh = bkc.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword("".getBytes()) + .execute() + .join(); + + // Read 1 entry and detect which bookie was being used + @Cleanup + LedgerEntries entry0 = rh.read(0, 0); + assertArrayEquals("entry-0".getBytes(), entry0.getEntry(0).getEntryBytes()); + + // All read requests should have been made to a single bookie + int bookieWithRequests = -1; + for (int i = 0; i < NUM_BOOKIES; i++) { + long requests = getStatsProvider(i).getOpStatsLogger(READ_ENTRY_REQUEST_METRIC) + .getSuccessCount(); + + log.info("Bookie {} --- requests: {}", i, requests); + + if (requests > 0) { + assertTrue("Another bookie already had received requests", bookieWithRequests == -1); + bookieWithRequests = i; + } + } + + // Suspend the sticky bookie. Reads should now go to a different sticky + // bookie + bs.get(bookieWithRequests).suspendProcessing(); + + for (int i = 0; i < n; i++) { + @Cleanup + LedgerEntries entries = rh.read(i, i); + + assertArrayEquals(("entry-" + i).getBytes(), entries.getEntry(i).getEntryBytes()); + } + + // At this point, we should have 1 bookie with 1 request (the initial + // request), and a second bookie with 10 requests. The 3rd bookie should + // have no requests + List requestCounts = Lists.newArrayList(getBookieReadRequestStats().values()); + Collections.sort(requestCounts); + + assertEquals(0, requestCounts.get(0).longValue()); + assertEquals(1, requestCounts.get(1).longValue()); + assertEquals(10, requestCounts.get(2).longValue()); + } + + private Map getBookieReadRequestStats() throws Exception { + Map stats = new TreeMap<>(); + for (int i = 0; i < NUM_BOOKIES; i++) { + stats.put(i, getStatsProvider(i).getOpStatsLogger(READ_ENTRY_REQUEST_METRIC) + .getSuccessCount()); + } + + return stats; + } + + private void writeAndReadEntries(ClientConfiguration conf, int ensembleSize, int writeQuorum, int ackQuorum) + throws Exception { + @Cleanup + BookKeeper bkc = new BookKeeper(conf); + + @Cleanup + WriteHandle wh = bkc.newCreateLedgerOp() + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorum) + .withAckQuorumSize(ackQuorum) + .withPassword("".getBytes()) + .execute() + .join(); + + final int n = 10; + + for (int i = 0; i < n; i++) { + wh.append(("entry-" + i).getBytes()); + } + + for (int i = 0; i < n; i++) { + @Cleanup + LedgerEntries entries = wh.read(i, i); + + assertArrayEquals(("entry-" + i).getBytes(), entries.getEntry(i).getEntryBytes()); + } + } +} From 525e527aa0f40baf3f82293d435b8afb36a81f4e Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Sat, 12 Jan 2019 18:04:35 -0800 Subject: [PATCH 0189/1642] Enhance EnsemblePlacementPolicy and DNSResolverDecorator to log relevant metrics. Descriptions of the changes in this PR: Make changes to EnsemblePlacementPolicy so that it would return boolean value indicating if the return value of newEnsemble and replaceBookie are strictly adhering to corresponding PlacementPolicy or it fell back to random. Similarly DNSResolverDecorator should log a metric when it was unable to resolve rack info and it is using default rack. Reviewers: Samuel Just , Sijie Guo , Venkateswararao Jujjuri (JV) This closes #1883 from reddycharan/enhanceplacementpolicy --- .../bookie/BookKeeperServerStats.java | 2 + .../LocalBookieEnsemblePlacementPolicy.java | 19 +- .../bookkeeper/client/BookKeeperAdmin.java | 16 +- .../bookkeeper/client/BookieWatcherImpl.java | 71 +++++-- .../DefaultEnsemblePlacementPolicy.java | 29 ++- .../client/EnsemblePlacementPolicy.java | 50 +++-- ...ITopologyAwareEnsemblePlacementPolicy.java | 3 +- .../RackawareEnsemblePlacementPolicy.java | 15 +- .../RackawareEnsemblePlacementPolicyImpl.java | 100 ++++++++-- .../RegionAwareEnsemblePlacementPolicy.java | 42 ++++- .../bookkeeper/net/ScriptBasedMapping.java | 8 +- .../GenericEnsemblePlacementPolicyTest.java | 7 +- .../TestRackawareEnsemblePlacementPolicy.java | 177 +++++++++++++----- ...areEnsemblePlacementPolicyUsingScript.java | 45 +++-- ...estRackawarePolicyNotificationUpdates.java | 9 +- ...estRegionAwareEnsemblePlacementPolicy.java | 108 ++++++----- 16 files changed, 515 insertions(+), 186 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java index cdafd15a991..b58514b6709 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java @@ -69,6 +69,8 @@ public interface BookKeeperServerStats { String WATCHER_SCOPE = "bookie_watcher"; String REPLACE_BOOKIE_TIME = "REPLACE_BOOKIE_TIME"; String NEW_ENSEMBLE_TIME = "NEW_ENSEMBLE_TIME"; + String FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER = "FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER"; + String ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER = "ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER"; // Bookie Operations String BOOKIE_ADD_ENTRY = "BOOKIE_ADD_ENTRY"; 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 46978ea5c65..7b7cc46bcfd 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 @@ -36,6 +36,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,9 +81,9 @@ public Set onClusterChanged(Set writab } @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - java.util.Map customMetadata, Set currentEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies) + public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + java.util.Map customMetadata, List currentEnsemble, + BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { throw new BKNotEnoughBookiesException(); } @@ -109,18 +110,24 @@ public DistributionSchedule.WriteSet reorderReadLACSequence( } @Override - public List newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - java.util.Map customMetadata, Set excludeBookies) + public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, java.util.Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { if (ensembleSize > 1) { throw new IllegalArgumentException("Local ensemble policy can only return 1 bookie"); } - return Lists.newArrayList(bookieAddress); + return Pair.of(Lists.newArrayList(bookieAddress), true); } @Override public void updateBookieInfo(Map bookieToFreeSpaceMap) { return; } + + @Override + public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, + int ackQuorumSize) { + return true; + } } 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 3f837dbaf70..7b492922f15 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 @@ -35,7 +35,6 @@ 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; @@ -82,6 +81,7 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.IOUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -994,15 +994,25 @@ private Map getReplacementBookiesByIndexes( // allocate bookies for (Integer bookieIndex : bookieIndexesToRereplicate) { BookieSocketAddress oldBookie = ensemble.get(bookieIndex); - BookieSocketAddress newBookie = + Pair replaceBookieResponse = bkc.getPlacementPolicy().replaceBookie( lh.getLedgerMetadata().getEnsembleSize(), lh.getLedgerMetadata().getWriteQuorumSize(), lh.getLedgerMetadata().getAckQuorumSize(), lh.getLedgerMetadata().getCustomMetadata(), - new HashSet<>(ensemble), + ensemble, oldBookie, bookiesToExclude); + BookieSocketAddress newBookie = replaceBookieResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + if (!isEnsembleAdheringToPlacementPolicy) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "replaceBookie for bookie: {} in ensemble: {} " + + "is not adhering to placement policy and chose {}", + oldBookie, ensemble, newBookie); + } + } targetBookieAddresses.put(bookieIndex, newBookie); bookiesToExclude.add(newBookie); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index 8b14b77174a..74c1df984b2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -17,6 +17,7 @@ */ package org.apache.bookkeeper.client; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.NEW_ENSEMBLE_TIME; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.REPLACE_BOOKIE_TIME; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WATCHER_SCOPE; @@ -35,6 +36,8 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import lombok.extern.slf4j.Slf4j; + +import org.apache.bookkeeper.bookie.BookKeeperServerStats; import org.apache.bookkeeper.client.BKException.BKInterruptedException; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BKException.MetaStoreException; @@ -43,9 +46,11 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; +import org.apache.commons.lang3.tuple.Pair; /** * This class is responsible for maintaining a consistent view of what bookies @@ -88,6 +93,12 @@ class BookieWatcherImpl implements BookieWatcher { help = "operation stats of replacing bookie in an ensemble" ) private final OpStatsLogger replaceBookieTimer; + @StatsDoc( + name = ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER, + help = "total number of newEnsemble/replaceBookie operations failed to adhere" + + " EnsemblePlacementPolicy" + ) + private final Counter ensembleNotAdheringToPlacementPolicy; // Bookies that will not be preferred to be chosen in a new ensemble final Cache quarantinedBookies; @@ -117,6 +128,8 @@ public void onRemoval(RemovalNotification bookie) }).build(); this.newEnsembleTimer = statsLogger.getOpStatsLogger(NEW_ENSEMBLE_TIME); this.replaceBookieTimer = statsLogger.getOpStatsLogger(REPLACE_BOOKIE_TIME); + this.ensembleNotAdheringToPlacementPolicy = statsLogger + .getCounter(BookKeeperServerStats.ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER); } @Override @@ -213,19 +226,34 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi int ackQuorumSize, Map customMetadata) throws BKNotEnoughBookiesException { long startTime = MathUtils.nowInNano(); + Pair, Boolean> newEnsembleResponse; List socketAddresses; + boolean isEnsembleAdheringToPlacementPolicy = false; try { - socketAddresses = placementPolicy.newEnsemble(ensembleSize, - writeQuorumSize, ackQuorumSize, customMetadata, new HashSet( - quarantinedBookies.asMap().keySet())); + Set quarantinedBookiesSet = quarantinedBookies.asMap().keySet(); + newEnsembleResponse = placementPolicy.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, + customMetadata, new HashSet(quarantinedBookiesSet)); + socketAddresses = newEnsembleResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.getRight(); + if (!isEnsembleAdheringToPlacementPolicy) { + ensembleNotAdheringToPlacementPolicy.inc(); + log.warn("New ensemble: {} is not adhering to Placement Policy. quarantinedBookies: {}", + socketAddresses, quarantinedBookiesSet); + } // we try to only get from the healthy bookies first newEnsembleTimer.registerSuccessfulEvent(MathUtils.nowInNano() - startTime, TimeUnit.NANOSECONDS); } catch (BKNotEnoughBookiesException e) { if (log.isDebugEnabled()) { log.debug("Not enough healthy bookies available, using quarantined bookies"); } - socketAddresses = placementPolicy.newEnsemble( + newEnsembleResponse = placementPolicy.newEnsemble( ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, new HashSet<>()); + socketAddresses = newEnsembleResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.getRight(); + if (!isEnsembleAdheringToPlacementPolicy) { + ensembleNotAdheringToPlacementPolicy.inc(); + log.warn("New ensemble: {} is not adhering to Placement Policy", socketAddresses); + } newEnsembleTimer.registerFailedEvent(MathUtils.nowInNano() - startTime, TimeUnit.NANOSECONDS); } return socketAddresses; @@ -239,22 +267,43 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, throws BKNotEnoughBookiesException { long startTime = MathUtils.nowInNano(); BookieSocketAddress addr = existingBookies.get(bookieIdx); + Pair replaceBookieResponse; BookieSocketAddress socketAddress; + boolean isEnsembleAdheringToPlacementPolicy = false; try { // we exclude the quarantined bookies also first - Set existingAndQuarantinedBookies = new HashSet(existingBookies); - existingAndQuarantinedBookies.addAll(quarantinedBookies.asMap().keySet()); - socketAddress = placementPolicy.replaceBookie( + Set excludedBookiesAndQuarantinedBookies = new HashSet( + excludeBookies); + Set quarantinedBookiesSet = quarantinedBookies.asMap().keySet(); + excludedBookiesAndQuarantinedBookies.addAll(quarantinedBookiesSet); + replaceBookieResponse = placementPolicy.replaceBookie( ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, - existingAndQuarantinedBookies, addr, excludeBookies); + existingBookies, addr, excludedBookiesAndQuarantinedBookies); + socketAddress = replaceBookieResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + if (!isEnsembleAdheringToPlacementPolicy) { + ensembleNotAdheringToPlacementPolicy.inc(); + log.warn( + "replaceBookie for bookie: {} in ensemble: {} is not adhering to placement policy and" + + " chose {}. excludedBookies {} and quarantinedBookies {}", + addr, existingBookies, socketAddress, excludeBookies, quarantinedBookiesSet); + } replaceBookieTimer.registerSuccessfulEvent(MathUtils.nowInNano() - startTime, TimeUnit.NANOSECONDS); } catch (BKNotEnoughBookiesException e) { if (log.isDebugEnabled()) { log.debug("Not enough healthy bookies available, using quarantined bookies"); } - socketAddress = placementPolicy.replaceBookie( - ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, - new HashSet(existingBookies), addr, excludeBookies); + replaceBookieResponse = placementPolicy.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, + customMetadata, existingBookies, addr, excludeBookies); + socketAddress = replaceBookieResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + if (!isEnsembleAdheringToPlacementPolicy) { + ensembleNotAdheringToPlacementPolicy.inc(); + log.warn( + "replaceBookie for bookie: {} in ensemble: {} is not adhering to placement policy and" + + " chose {}. excludedBookies {}", + addr, existingBookies, socketAddress, excludeBookies); + } replaceBookieTimer.registerFailedEvent(MathUtils.nowInNano() - startTime, TimeUnit.NANOSECONDS); } return socketAddress; 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 917d18f7178..dddbe1cb40d 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 @@ -39,6 +39,7 @@ import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,12 +65,12 @@ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy { } @Override - public ArrayList newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, + public Pair, Boolean> newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { ArrayList newBookies = new ArrayList(ensembleSize); if (ensembleSize <= 0) { - return newBookies; + return Pair.of(newBookies, false); } List allBookies; rwLock.readLock().lock(); @@ -95,7 +96,8 @@ public ArrayList newEnsemble(int ensembleSize, int quorumSi newBookies.add(b); --ensembleSize; if (ensembleSize == 0) { - return newBookies; + return Pair.of(newBookies, + isEnsembleAdheringToPlacementPolicy(newBookies, quorumSize, ackQuorumSize)); } } } finally { @@ -110,7 +112,8 @@ public ArrayList newEnsemble(int ensembleSize, int quorumSi newBookies.add(bookie); --ensembleSize; if (ensembleSize == 0) { - return newBookies; + return Pair.of(newBookies, + isEnsembleAdheringToPlacementPolicy(newBookies, quorumSize, ackQuorumSize)); } } } @@ -118,13 +121,17 @@ public ArrayList newEnsemble(int ensembleSize, int quorumSi } @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, Set currentEnsemble, + public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { excludeBookies.addAll(currentEnsemble); - ArrayList addresses = newEnsemble(1, 1, 1, customMetadata, excludeBookies); - return addresses.get(0); + List addresses = newEnsemble(1, 1, 1, customMetadata, excludeBookies).getLeft(); + + BookieSocketAddress candidateAddr = addresses.get(0); + List newEnsemble = new ArrayList(currentEnsemble); + newEnsemble.set(currentEnsemble.indexOf(bookieToReplace), candidateAddr); + return Pair.of(candidateAddr, isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); } @Override @@ -210,4 +217,10 @@ public void updateBookieInfo(Map bookieInfoMap) public void uninitalize() { // do nothing } + + @Override + public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, + int ackQuorumSize) { + return true; + } } 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 23932a3d7df..00bac8e3c41 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 @@ -35,6 +35,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.commons.lang3.tuple.Pair; /** * {@link EnsemblePlacementPolicy} encapsulates the algorithm that bookkeeper client uses to select a number of bookies @@ -263,12 +264,12 @@ Set onClusterChanged(Set writableBooki * @throws BKNotEnoughBookiesException if not enough bookies available. * @return the List<org.apache.bookkeeper.net.BookieSocketAddress> */ - List newEnsemble(int ensembleSize, - int writeQuorumSize, - int ackQuorumSize, - Map customMetadata, - Set excludeBookies) - throws BKNotEnoughBookiesException; + Pair, Boolean> newEnsemble(int ensembleSize, + int writeQuorumSize, + int ackQuorumSize, + Map customMetadata, + Set excludeBookies) + throws BKNotEnoughBookiesException; /** * Choose a new bookie to replace bookieToReplace. If no bookie available in the cluster, @@ -287,14 +288,14 @@ List newEnsemble(int ensembleSize, * @throws BKNotEnoughBookiesException * @return the org.apache.bookkeeper.net.BookieSocketAddress */ - BookieSocketAddress replaceBookie(int ensembleSize, - int writeQuorumSize, - int ackQuorumSize, - Map customMetadata, - Set currentEnsemble, - BookieSocketAddress bookieToReplace, - Set excludeBookies) - throws BKNotEnoughBookiesException; + Pair replaceBookie(int ensembleSize, + int writeQuorumSize, + int ackQuorumSize, + Map customMetadata, + List currentEnsemble, + BookieSocketAddress bookieToReplace, + Set excludeBookies) + throws BKNotEnoughBookiesException; /** * Register a bookie as slow so that it is tried after available and read-only bookies. @@ -386,4 +387,25 @@ default int getStickyReadBookieIndex(LedgerMetadata metadata, Optional return (currentStickyBookieIndex.get() + 1) % metadata.getEnsembleSize(); } } + + /** + * returns true if the Ensemble is strictly adhering to placement policy, + * like in the case of RackawareEnsemblePlacementPolicy, bookies in the + * writeset are from 'minNumRacksPerWriteQuorum' number of racks. And in the + * case of RegionawareEnsemblePlacementPolicy, check for + * minimumRegionsForDurability, reppRegionsToWrite, rack distribution within + * a region and other parameters of RegionAwareEnsemblePlacementPolicy. + * + * @param ensembleList + * list of BookieSocketAddress of bookies in the ensemble + * @param writeQuorumSize + * writeQuorumSize of the ensemble + * @param ackQuorumSize + * ackQuorumSize of the ensemble + * @return + */ + default boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, + int ackQuorumSize) { + return false; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java index 7c9e07cd10a..254f5359c20 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.common.annotation.InterfaceStability; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.Node; +import org.apache.commons.lang3.tuple.Pair; /** * Interface for topology aware ensemble placement policy. @@ -93,7 +94,7 @@ interface Ensemble { * @return list of bookies forming the ensemble * @throws BKException.BKNotEnoughBookiesException */ - List newEnsemble( + Pair, Boolean> newEnsemble( int ensembleSize, int writeQuorumSize, int ackQuorumSize, 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 1fd7580c816..8054d97187b 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 @@ -18,19 +18,16 @@ package org.apache.bookkeeper.client; import io.netty.util.HashedWheelTimer; - import java.util.List; import java.util.Map; import java.util.Set; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; -import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble; -import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate; -import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.commons.lang3.tuple.Pair; /** * A placement policy implementation use rack information for placing ensembles. @@ -95,8 +92,8 @@ public Set onClusterChanged(Set writab } @Override - public List newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, Set excludeBookies) + public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { try { return super.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, excludeBookies); @@ -110,8 +107,8 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi } @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, Set currentEnsemble, + public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { try { @@ -146,7 +143,7 @@ public DistributionSchedule.WriteSet reorderReadLACSequence( } @Override - public List newEnsemble(int ensembleSize, + public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Set excludeBookies, 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 7578c416ee1..6db7de8f727 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 @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_JOINED; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_LEFT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER; import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE; import static org.apache.bookkeeper.client.BookKeeperClientStats.READ_REQUESTS_REORDERED; import static org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy.UNKNOWN_REGION; @@ -35,6 +36,7 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -64,10 +66,12 @@ import org.apache.bookkeeper.net.NodeBase; import org.apache.bookkeeper.net.ScriptBasedMapping; import org.apache.bookkeeper.net.StabilizeNetworkTopology; +import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -146,12 +150,19 @@ static class DNSResolverDecorator implements DNSToSwitchMapping { final Supplier defaultRackSupplier; final DNSToSwitchMapping resolver; - - DNSResolverDecorator(DNSToSwitchMapping resolver, Supplier defaultRackSupplier) { + @StatsDoc( + name = FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER, + help = "total number of times Resolver failed to resolve rack information of a node" + ) + final Counter failedToResolveNetworkLocationCounter; + + DNSResolverDecorator(DNSToSwitchMapping resolver, Supplier defaultRackSupplier, + Counter failedToResolveNetworkLocationCounter) { checkNotNull(resolver, "Resolver cannot be null"); checkNotNull(defaultRackSupplier, "defaultRackSupplier should not be null"); this.defaultRackSupplier = defaultRackSupplier; this.resolver = resolver; + this.failedToResolveNetworkLocationCounter = failedToResolveNetworkLocationCounter; } public List resolve(List names) { @@ -167,6 +178,7 @@ public List resolve(List names) { if (rNames.get(i) == null) { LOG.warn("Failed to resolve network location for {}, using default rack for it : {}.", names.get(i), defaultRack); + failedToResolveNetworkLocationCounter.inc(); rNames.set(i, defaultRack); } } @@ -178,6 +190,7 @@ public List resolve(List names) { rNames = new ArrayList<>(names.size()); for (int i = 0; i < names.size(); ++i) { + failedToResolveNetworkLocationCounter.inc(); rNames.add(defaultRack); } return rNames; @@ -227,6 +240,7 @@ public void reloadCachedMappings() { help = "The distribution of number of bookies reordered on each read request" ) protected OpStatsLogger readReorderedCounter = null; + protected Counter failedToResolveNetworkLocationCounter = null; private String defaultRack = NetworkTopology.DEFAULT_RACK; @@ -267,10 +281,13 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns this.bookiesJoinedCounter = statsLogger.getOpStatsLogger(BOOKIES_JOINED); this.bookiesLeftCounter = statsLogger.getOpStatsLogger(BOOKIES_LEFT); this.readReorderedCounter = statsLogger.getOpStatsLogger(READ_REQUESTS_REORDERED); + this.failedToResolveNetworkLocationCounter = statsLogger + .getCounter(FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER); this.reorderReadsRandom = reorderReadsRandom; this.stabilizePeriodSeconds = stabilizePeriodSeconds; this.reorderThresholdPendingRequests = reorderThresholdPendingRequests; - this.dnsResolver = new DNSResolverDecorator(dnsResolver, () -> this.getDefaultRack()); + this.dnsResolver = new DNSResolverDecorator(dnsResolver, () -> this.getDefaultRack(), + failedToResolveNetworkLocationCounter); this.timer = timer; this.minNumRacksPerWriteQuorum = minNumRacksPerWriteQuorum; this.enforceMinNumRacksPerWriteQuorum = enforceMinNumRacksPerWriteQuorum; @@ -342,9 +359,18 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, ((RackChangeNotifier) dnsResolver).registerRackChangeListener(this); } } catch (RuntimeException re) { - LOG.info("Failed to initialize DNS Resolver {}, used default subnet resolver : {}", - dnsResolverName, re, re.getMessage()); - dnsResolver = new DefaultResolver(() -> this.getDefaultRack()); + if (!enforceMinNumRacksPerWriteQuorum) { + LOG.info("Failed to initialize DNS Resolver {}, used default subnet resolver : {}", dnsResolverName, + re, re.getMessage()); + dnsResolver = new DefaultResolver(() -> this.getDefaultRack()); + } else { + /* + * if minNumRacksPerWriteQuorum is enforced, then it + * shouldn't continue in the case of failure to create + * dnsResolver. + */ + throw re; + } } } slowBookies = CacheBuilder.newBuilder() @@ -479,7 +505,7 @@ public void handleBookiesThatJoined(Set joinedBookies) { } } - protected Set convertBookiesToNodes(Set excludeBookies) { + protected Set convertBookiesToNodes(Collection excludeBookies) { Set nodes = new HashSet(); for (BookieSocketAddress addr : excludeBookies) { BookieNode bn = knownBookies.get(addr); @@ -500,13 +526,13 @@ private static Set getNetworkLocations(Set bookieNodes) { } @Override - public List newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, Set excludeBookies) + public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { return newEnsembleInternal(ensembleSize, writeQuorumSize, excludeBookies, null, null); } - protected List newEnsembleInternal(int ensembleSize, + protected Pair, Boolean> newEnsembleInternal(int ensembleSize, int writeQuorumSize, Set excludeBookies, Ensemble parentEnsemble, @@ -522,7 +548,7 @@ protected List newEnsembleInternal(int ensembleSize, } @Override - public List newEnsemble(int ensembleSize, + public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Set excludeBookies, @@ -538,7 +564,7 @@ public List newEnsemble(int ensembleSize, parentPredicate); } - protected List newEnsembleInternal( + protected Pair, Boolean> newEnsembleInternal( int ensembleSize, int writeQuorumSize, int ackQuorumSize, @@ -572,7 +598,7 @@ protected List newEnsembleInternal( for (BookieNode bn : bns) { addrs.add(bn.getAddr()); } - return addrs; + return Pair.of(addrs, false); } for (int i = 0; i < ensembleSize; i++) { @@ -596,15 +622,15 @@ protected List newEnsembleInternal( ensembleSize, bookieList); throw new BKNotEnoughBookiesException(); } - return bookieList; + return Pair.of(bookieList, isEnsembleAdheringToPlacementPolicy(bookieList, writeQuorumSize, ackQuorumSize)); } finally { rwLock.readLock().unlock(); } } @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, Set currentEnsemble, + public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { rwLock.readLock().lock(); @@ -639,7 +665,19 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, if (LOG.isDebugEnabled()) { LOG.debug("Bookie {} is chosen to replace bookie {}.", candidate, bn); } - return candidate.getAddr(); + BookieSocketAddress candidateAddr = candidate.getAddr(); + List newEnsemble = new ArrayList(currentEnsemble); + if (currentEnsemble.isEmpty()) { + /* + * in testing code there are test cases which would pass empty + * currentEnsemble + */ + newEnsemble.add(candidateAddr); + } else { + newEnsemble.set(currentEnsemble.indexOf(bookieToReplace), candidateAddr); + } + return Pair.of(candidateAddr, + isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); } finally { rwLock.readLock().unlock(); } @@ -1206,4 +1244,32 @@ static void shuffleWithMask(DistributionSchedule.WriteSet writeSet, } } } + + @Override + public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, + int ackQuorumSize) { + int ensembleSize = ensembleList.size(); + int minNumRacksPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, minNumRacksPerWriteQuorum); + HashSet racksOrRegionsInQuorum = new HashSet(); + BookieSocketAddress bookie; + for (int i = 0; i < ensembleList.size(); i++) { + racksOrRegionsInQuorum.clear(); + for (int j = 0; j < writeQuorumSize; j++) { + bookie = ensembleList.get((i + j) % ensembleSize); + try { + racksOrRegionsInQuorum.add(knownBookies.get(bookie).getNetworkLocation()); + } catch (Exception e) { + /* + * any issue/exception in analyzing whether ensemble is strictly adhering to + * placement policy should be swallowed. + */ + LOG.warn("Received exception while trying to get network location of bookie: {}", bookie, e); + } + } + if (racksOrRegionsInQuorum.size() < minNumRacksPerWriteQuorumForThisEnsemble) { + return false; + } + } + return true; + } } 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 c52e0fee852..1bd4b75eac4 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 @@ -225,8 +225,8 @@ protected List selectRandomFromRegions(Set availableRegions, @Override - public List newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, Set excludeBookies) + public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { int effectiveMinRegionsForDurability = disableDurabilityFeature.isAvailable() ? 1 : minRegionsForDurability; @@ -279,7 +279,7 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi for (BookieNode bn : bns) { addrs.add(bn.getAddr()); } - return addrs; + return Pair.of(addrs, isEnsembleAdheringToPlacementPolicy(addrs, writeQuorumSize, ackQuorumSize)); } // Single region, fall back to RackAwareEnsemblePlacement @@ -347,7 +347,7 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi try { List allocated = policyWithinRegion.newEnsemble(newEnsembleSize, newWriteQuorumSize, newWriteQuorumSize, excludeBookies, tempEnsemble, - tempEnsemble); + tempEnsemble).getLeft(); ensemble = tempEnsemble; remainingEnsemble -= addToEnsembleSize; remainingWriteQuorum -= addToWriteQuorum; @@ -407,15 +407,17 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi throw new BKException.BKNotEnoughBookiesException(); } LOG.info("Bookies allocated successfully {}", ensemble); - return ensemble.toList(); + List ensembleList = ensemble.toList(); + return Pair.of(ensembleList, + isEnsembleAdheringToPlacementPolicy(ensembleList, writeQuorumSize, ackQuorumSize)); } finally { rwLock.readLock().unlock(); } } @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, Set currentEnsemble, + public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { rwLock.readLock().lock(); @@ -469,7 +471,19 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, if (LOG.isDebugEnabled()) { LOG.debug("Bookie {} is chosen to replace bookie {}.", candidate, bookieNodeToReplace); } - return candidate.getAddr(); + BookieSocketAddress candidateAddr = candidate.getAddr(); + List newEnsemble = new ArrayList(currentEnsemble); + if (currentEnsemble.isEmpty()) { + /* + * in testing code there are test cases which would pass empty + * currentEnsemble + */ + newEnsemble.add(candidateAddr); + } else { + newEnsemble.set(currentEnsemble.indexOf(bookieToReplace), candidateAddr); + } + return Pair.of(candidateAddr, + isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); } finally { rwLock.readLock().unlock(); } @@ -550,4 +564,16 @@ public final DistributionSchedule.WriteSet reorderReadLACSequence( finalList.addMissingIndices(ensemble.size()); return finalList; } + + @Override + public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, + int ackQuorumSize) { + /** + * TODO: have to implement actual logic for this method for + * RegionAwareEnsemblePlacementPolicy. For now return true value. + * + * - https://github.com/apache/bookkeeper/issues/1898 + */ + return true; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java index 8c8350c3fc7..230f66d411d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java @@ -25,8 +25,8 @@ import org.apache.bookkeeper.util.Shell.ShellCommandExecutor; import org.apache.commons.configuration.Configuration; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class implements the {@link DNSToSwitchMapping} interface using a @@ -129,7 +129,7 @@ public void setConf(Configuration conf) { private static final class RawScriptBasedMapping extends AbstractDNSToSwitchMapping { private String scriptName; private int maxArgs; //max hostnames per call of the script - private static final Log LOG = LogFactory.getLog(ScriptBasedMapping.class); + private static final Logger LOG = LoggerFactory.getLogger(ScriptBasedMapping.class); /** * Set the configuration and extract the configuration parameters of interest. @@ -233,7 +233,7 @@ private String runResolveCommand(List args) { s.execute(); allOutput.append(s.getOutput()).append(" "); } catch (Exception e) { - LOG.warn("Exception running " + s, e); + LOG.warn("Exception running: {} Exception message: {}", s, e.getMessage()); return null; } loopCount++; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java index bb55d0c6d0e..205c5f4cb99 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java @@ -33,6 +33,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.commons.lang3.tuple.Pair; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -69,8 +70,8 @@ public GenericEnsemblePlacementPolicyTest(boolean diskWeightBasedPlacementEnable public static final class CustomEnsemblePlacementPolicy extends DefaultEnsemblePlacementPolicy { @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map customMetadata, Set currentEnsemble, + public Pair replaceBookie(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { new Exception("replaceBookie " + ensembleSize + "," + customMetadata).printStackTrace(); @@ -81,7 +82,7 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, } @Override - public ArrayList newEnsemble(int ensembleSize, int quorumSize, + public Pair, Boolean> newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { assertNotNull(customMetadata); 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 e75e93fa864..e6cd07bcc7b 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 @@ -52,6 +52,7 @@ import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,6 +70,7 @@ public class TestRackawareEnsemblePlacementPolicy extends TestCase { ClientConfiguration conf = new ClientConfiguration(); BookieSocketAddress addr1, addr2, addr3, addr4; io.netty.util.HashedWheelTimer timer; + final int minNumRacksPerWriteQuorumConfValue = 2; @Override protected void setUp() throws Exception { @@ -80,6 +82,7 @@ protected void setUp() throws Exception { StaticDNSResolver.addNodeToRack("localhost", NetworkTopology.DEFAULT_REGION_AND_RACK); LOG.info("Set up static DNS Resolver."); conf.setProperty(REPP_DNS_RESOLVER_CLASS, StaticDNSResolver.class.getName()); + conf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); addr1 = new BookieSocketAddress("127.0.0.2", 3181); addr2 = new BookieSocketAddress("127.0.0.3", 3181); addr3 = new BookieSocketAddress("127.0.0.4", 3181); @@ -564,8 +567,12 @@ public void testReplaceBookieWithEnoughBookiesInSameRack() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new HashSet<>(), addr2, new HashSet<>()); + Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, new HashSet<>()); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); assertEquals(addr3, replacedBookie); + assertTrue(isEnsembleAdheringToPlacementPolicy); } @Test @@ -589,10 +596,13 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRack() throws Exception // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new HashSet<>(), addr2, excludedAddrs); - + Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, excludedAddrs); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); assertFalse(addr1.equals(replacedBookie)); assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); + assertTrue(isEnsembleAdheringToPlacementPolicy); } @Test @@ -619,7 +629,7 @@ public void testReplaceBookieWithNotEnoughBookies() throws Exception { excludedAddrs.add(addr3); excludedAddrs.add(addr4); try { - repp.replaceBookie(1, 1, 1, null, new HashSet(), addr2, excludedAddrs); + repp.replaceBookie(1, 1, 1, null, new ArrayList(), addr2, excludedAddrs); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not enou @@ -628,7 +638,7 @@ public void testReplaceBookieWithNotEnoughBookies() throws Exception { @Test public void testReplaceBookieWithEnoughBookiesInSameRackAsEnsemble() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.1", 3181); + BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.5", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.2", 3181); BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.3", 3181); BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.4", 3181); @@ -645,15 +655,18 @@ public void testReplaceBookieWithEnoughBookiesInSameRackAsEnsemble() throws Exce addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set ensembleBookies = new HashSet(); + List ensembleBookies = new ArrayList(); ensembleBookies.add(addr2); ensembleBookies.add(addr4); - BookieSocketAddress replacedBookie = repp.replaceBookie( + Pair replaceBookieResponse = repp.replaceBookie( 1, 1, 1 , null, ensembleBookies, addr4, new HashSet<>()); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); assertEquals(addr1, replacedBookie); + assertTrue(isEnsembleAdheringToPlacementPolicy); } @Test @@ -670,10 +683,18 @@ public void testNewEnsembleWithSingleRack() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); + Pair, Boolean> ensembleResponse; + ensembleResponse = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); + List ensemble = ensembleResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); assertEquals(0, getNumCoveredWriteQuorums(ensemble, 2, conf.getMinNumRacksPerWriteQuorum())); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()); + assertFalse(isEnsembleAdheringToPlacementPolicy); + Pair, Boolean> ensembleResponse2; + ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()); + List ensemble2 = ensembleResponse2.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.getRight(); assertEquals(0, getNumCoveredWriteQuorums(ensemble2, 2, conf.getMinNumRacksPerWriteQuorum())); + assertFalse(isEnsembleAdheringToPlacementPolicy2); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); } @@ -703,17 +724,19 @@ public void testSingleRackWithEnforceMinNumRacks() throws Exception { addrs.add(addr3); addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); - + Pair, Boolean> ensembleResponse; List ensemble; try { - ensemble = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); + ensembleResponse = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); + ensemble = ensembleResponse.getLeft(); fail("Should get not enough bookies exception since there is only one rack."); } catch (BKNotEnoughBookiesException bnebe) { } try { - ensemble = repp.newEnsemble(3, 2, 2, new HashSet<>(), EnsembleForReplacementWithNoConstraints.INSTANCE, - TruePredicate.INSTANCE); + ensembleResponse = repp.newEnsemble(3, 2, 2, new HashSet<>(), + EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); + ensemble = ensembleResponse.getLeft(); fail("Should get not enough bookies exception since there is only one rack."); } catch (BKNotEnoughBookiesException bnebe) { } @@ -766,19 +789,27 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { * and there are enough bookies in 3 racks, this newEnsemble calls should * succeed. */ + Pair, Boolean> ensembleResponse; List ensemble; + boolean isEnsembleAdheringToPlacementPolicy; int ensembleSize = numOfRacks * numOfBookiesPerRack; int writeQuorumSize = numOfRacks; int ackQuorumSize = numOfRacks; - ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); + ensemble = ensembleResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + assertTrue(isEnsembleAdheringToPlacementPolicy); - ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(), + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(), EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); + ensemble = ensembleResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + assertTrue(isEnsembleAdheringToPlacementPolicy); } @Test @@ -822,16 +853,24 @@ public void testNewEnsembleWithSufficientRacksAndEnforceMinNumRacks() throws Exc * ensembleSizes (as long as there are enough number of bookies in each * rack). */ + Pair, Boolean> ensembleResponse; List ensemble; + boolean isEnsembleAdheringToPlacementPolicy; for (int ensembleSize = effectiveMinNumRacksPerWriteQuorum; ensembleSize < 40; ensembleSize++) { - ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); + ensemble = ensembleResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + assertTrue(isEnsembleAdheringToPlacementPolicy); - ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(), + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(), EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); + ensemble = ensembleResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + assertTrue(isEnsembleAdheringToPlacementPolicy); } } @@ -873,12 +912,14 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * and there are enough bookies in 3 racks, this newEnsemble call should * succeed. */ + Pair, Boolean> ensembleResponse; List ensemble; int ensembleSize = numOfRacks * numOfBookiesPerRack; int writeQuorumSize = numOfRacks; int ackQuorumSize = numOfRacks; - ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); + ensemble = ensembleResponse.getLeft(); BookieSocketAddress bookieInEnsembleToBeReplaced = ensemble.get(7); // get rack of some other bookie @@ -895,7 +936,7 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { repp.onClusterChanged(bookieSocketAddresses, new HashSet()); try { repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, - new HashSet(ensemble), bookieInEnsembleToBeReplaced, new HashSet<>()); + ensemble, bookieInEnsembleToBeReplaced, new HashSet<>()); fail("Should get not enough bookies exception since there are no more bookies in rack" + "of 'bookieInEnsembleToReplace'" + "and new bookie added belongs to the rack of some other bookie in the ensemble"); @@ -917,16 +958,22 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * this replaceBookie should succeed, because a new bookie is added to a * new rack. */ - BookieSocketAddress replacedBookieAddress = repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, - null, new HashSet(ensemble), bookieInEnsembleToBeReplaced, new HashSet<>()); + Pair replaceBookieResponse; + BookieSocketAddress replacedBookieAddress; + boolean isEnsembleAdheringToPlacementPolicy; + replaceBookieResponse = repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, ensemble, + bookieInEnsembleToBeReplaced, new HashSet<>()); + replacedBookieAddress = replaceBookieResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); assertEquals("It should be newBookieAddress2", newBookieAddress2, replacedBookieAddress); + assertTrue(isEnsembleAdheringToPlacementPolicy); Set bookiesToExclude = new HashSet<>(); bookiesToExclude.add(newBookieAddress2); repp.onClusterChanged(bookieSocketAddresses, new HashSet()); try { - repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, - new HashSet(ensemble), bookieInEnsembleToBeReplaced, bookiesToExclude); + repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, ensemble, + bookieInEnsembleToBeReplaced, bookiesToExclude); fail("Should get not enough bookies exception since the only available bookie to replace" + "is added to excludedBookies list"); } catch (BKNotEnoughBookiesException bnebe) { @@ -949,9 +996,12 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * replaced, so we should be able to replacebookie though * newBookieAddress2 is added to excluded bookies list. */ - replacedBookieAddress = repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, - new HashSet(ensemble), bookieInEnsembleToBeReplaced, bookiesToExclude); + replaceBookieResponse = repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, + ensemble, bookieInEnsembleToBeReplaced, bookiesToExclude); + replacedBookieAddress = replaceBookieResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); assertEquals("It should be newBookieAddress3", newBookieAddress3, replacedBookieAddress); + assertTrue(isEnsembleAdheringToPlacementPolicy); } @Test @@ -1298,15 +1348,21 @@ public void testNewEnsembleWithMultipleRacks() throws Exception { int ensembleSize = 3; int writeQuorumSize = 2; int acqQuorumSize = 2; - List ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, - null, new HashSet<>()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, + acqQuorumSize, null, new HashSet<>()); + List ensemble = ensembleResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); + assertFalse(isEnsembleAdheringToPlacementPolicy); ensembleSize = 4; - List ensemble2 = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, - null, new HashSet<>()); + Pair, Boolean> ensembleResponse2 = repp.newEnsemble(ensembleSize, writeQuorumSize, + acqQuorumSize, null, new HashSet<>()); + List ensemble2 = ensembleResponse2.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.getRight(); numCovered = getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); + assertFalse(isEnsembleAdheringToPlacementPolicy2); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); } @@ -1375,11 +1431,13 @@ void validateNumOfWriteQuorumsCoveredInEnsembleCreation(Set repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); repp.onClusterChanged(addrs, new HashSet()); - - List ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, writeQuorumSize, null, - new HashSet<>()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, + writeQuorumSize, null, new HashSet<>()); + List ensemble = ensembleResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, minNumRacksPerWriteQuorum); assertEquals("minimum number of racks covered for writequorum ensemble: " + ensemble, ensembleSize, numCovered); + assertTrue(isEnsembleAdheringToPlacementPolicy); } @Test @@ -1417,16 +1475,22 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { int ensembleSize = 3; int writeQuorumSize = 3; int ackQuorumSize = 2; - List ensemble1 = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, - null, new HashSet<>()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, + ackQuorumSize, null, new HashSet<>()); + List ensemble1 = ensembleResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy1 = ensembleResponse.getRight(); assertEquals(ensembleSize, getNumCoveredWriteQuorums(ensemble1, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); + assertTrue(isEnsembleAdheringToPlacementPolicy1); ensembleSize = 4; writeQuorumSize = 4; - List ensemble2 = repp.newEnsemble(ensembleSize, writeQuorumSize, 2, null, - new HashSet<>()); + Pair, Boolean> ensembleResponse2 = repp.newEnsemble(ensembleSize, writeQuorumSize, + 2, null, new HashSet<>()); + List ensemble2 = ensembleResponse2.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.getRight(); assertEquals(ensembleSize, getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); + assertTrue(isEnsembleAdheringToPlacementPolicy2); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); } @@ -1497,11 +1561,16 @@ public void testWeightedPlacementAndReplaceBookieWithEnoughBookiesInSameRack() t selectionCounts.put(addr3, 0L); selectionCounts.put(addr4, 0L); int numTries = 50000; + Pair replaceBookieResponse; + boolean isEnsembleAdheringToPlacementPolicy; BookieSocketAddress replacedBookie; for (int i = 0; i < numTries; i++) { // replace node under r2 - replacedBookie = repp.replaceBookie(1, 1, 1, null, new HashSet<>(), addr2, new HashSet<>()); + replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); + replacedBookie = replaceBookieResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); assertTrue("replaced : " + replacedBookie, addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); + assertTrue(isEnsembleAdheringToPlacementPolicy); selectionCounts.put(replacedBookie, selectionCounts.get(replacedBookie) + 1); } double observedMultiple = ((double) selectionCounts.get(addr4) / (double) selectionCounts.get(addr3)); @@ -1557,14 +1626,19 @@ public void testWeightedPlacementAndReplaceBookieWithoutEnoughBookiesInSameRack( selectionCounts.put(addr3, 0L); selectionCounts.put(addr4, 0L); int numTries = 50000; + Pair replaceBookieResponse; BookieSocketAddress replacedBookie; + boolean isEnsembleAdheringToPlacementPolicy; for (int i = 0; i < numTries; i++) { // addr2 is on /r2 and this is the only one on this rack. So the replacement // will come from other racks. However, the weight should be honored in such // selections as well - replacedBookie = repp.replaceBookie(1, 1, 1, null, new HashSet<>(), addr2, new HashSet<>()); + replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); + replacedBookie = replaceBookieResponse.getLeft(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); assertTrue(addr0.equals(replacedBookie) || addr1.equals(replacedBookie) || addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); + assertTrue(isEnsembleAdheringToPlacementPolicy); selectionCounts.put(replacedBookie, selectionCounts.get(replacedBookie) + 1); } /* @@ -1656,6 +1730,7 @@ public void testWeightedPlacementAndNewEnsembleWithEnoughBookiesInSameRack() thr int numTries = 10000; Set excludeList = new HashSet(); + Pair, Boolean> ensembleResponse; List ensemble; int ensembleSize = 3; int writeQuorumSize = 2; @@ -1664,7 +1739,8 @@ public void testWeightedPlacementAndNewEnsembleWithEnoughBookiesInSameRack() thr // addr2 is on /r2 and this is the only one on this rack. So the replacement // will come from other racks. However, the weight should be honored in such // selections as well - ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, null, excludeList); + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, null, excludeList); + ensemble = ensembleResponse.getLeft(); assertTrue( "Rackaware selection not happening " + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()), @@ -1726,21 +1802,23 @@ public void testWeightedPlacementAndNewEnsembleWithoutEnoughBookies() throws Exc bookieInfoMap.put(addr5, new BookieInfo(1000L, 1000L)); repp.updateBookieInfo(bookieInfoMap); - - List ensemble = new ArrayList(); + Pair, Boolean> ensembleResponse; + List ensemble; Set excludeList = new HashSet(); try { excludeList.add(addr1); excludeList.add(addr2); excludeList.add(addr3); excludeList.add(addr4); - ensemble = repp.newEnsemble(3, 2, 2, null, excludeList); + ensembleResponse = repp.newEnsemble(3, 2, 2, null, excludeList); + ensemble = ensembleResponse.getLeft(); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies" + ensemble); } catch (BKNotEnoughBookiesException e) { // this is expected } try { - ensemble = repp.newEnsemble(1, 1, 1, null, excludeList); + ensembleResponse = repp.newEnsemble(1, 1, 1, null, excludeList); + ensemble = ensembleResponse.getLeft(); } catch (BKNotEnoughBookiesException e) { fail("Should not throw BKNotEnoughBookiesException when there are enough bookies for the ensemble"); } @@ -1824,13 +1902,20 @@ public void testPlacementOnStabilizeNetworkTopology() throws Exception { // we will never use addr4 even it is in the stabilized network topology for (int i = 0; i < 5; i++) { - List ensemble = - repp.newEnsemble(3, 3, 3, null, new HashSet()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, + new HashSet()); + List ensemble = ensembleResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); assertFalse(ensemble.contains(addr4)); + assertFalse(isEnsembleAdheringToPlacementPolicy); } // we could still use addr4 for urgent allocation if it is just bookie flapping - List ensemble = repp.newEnsemble(4, 4, 4, null, new HashSet()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(4, 2, 2, null, + new HashSet()); + List ensemble = ensembleResponse.getLeft(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + assertFalse(isEnsembleAdheringToPlacementPolicy); assertTrue(ensemble.contains(addr4)); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java index d9f253507e5..e0fd2bdffe2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java @@ -30,6 +30,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.util.HashedWheelTimer; + +import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Optional; @@ -44,6 +46,7 @@ import org.apache.bookkeeper.net.ScriptBasedMapping; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.Shell; +import org.apache.commons.lang3.tuple.Pair; import org.junit.After; import org.junit.Assume; import org.junit.Before; @@ -111,7 +114,9 @@ public void testReplaceBookieWithEnoughBookiesInSameRack() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new HashSet<>(), addr2, new HashSet<>()); + Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, new HashSet<>()); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); assertEquals(addr3, replacedBookie); } @@ -133,7 +138,9 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRack() throws Exception // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new HashSet<>(), addr2, excludedAddrs); + Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, excludedAddrs); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); assertFalse(addr1.equals(replacedBookie)); assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); @@ -160,7 +167,7 @@ public void testReplaceBookieWithNotEnoughBookies() throws Exception { excludedAddrs.add(addr3); excludedAddrs.add(addr4); try { - repp.replaceBookie(1, 1, 1, null, new HashSet(), addr2, excludedAddrs); + repp.replaceBookie(1, 1, 1, null, new ArrayList(), addr2, excludedAddrs); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not BKNotEnoughBookiesException @@ -197,7 +204,9 @@ public void testReplaceBookieWithScriptMappingError() throws Exception { // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new HashSet<>(), addr2, excludedAddrs); + Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, excludedAddrs); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); assertFalse(addr1.equals(replacedBookie)); assertFalse(addr2.equals(replacedBookie)); @@ -235,7 +244,9 @@ public void testReplaceBookieWithScriptMappingError2() throws Exception { // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new HashSet<>(), addr2, excludedAddrs); + Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, excludedAddrs); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); assertFalse(addr1.equals(replacedBookie)); assertFalse(addr2.equals(replacedBookie)); @@ -257,9 +268,13 @@ public void testNewEnsembleWithSingleRack() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, + new HashSet<>()); + List ensemble = ensembleResponse.getLeft(); assertEquals(0, getNumCoveredWriteQuorums(ensemble, 2)); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()); + Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, + new HashSet<>()); + List ensemble2 = ensembleResponse2.getLeft(); assertEquals(0, getNumCoveredWriteQuorums(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -281,10 +296,14 @@ public void testNewEnsembleWithMultipleRacks() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, + new HashSet<>()); + List ensemble = ensembleResponse.getLeft(); int numCovered = getNumCoveredWriteQuorums(ensemble, 2); assertTrue(numCovered == 2); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()); + Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, + new HashSet<>()); + List ensemble2 = ensembleResponse2.getLeft(); numCovered = getNumCoveredWriteQuorums(ensemble2, 2); assertTrue(numCovered == 2); } catch (BKNotEnoughBookiesException bnebe) { @@ -315,9 +334,13 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { addrs.add(addr8); repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble1 = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); + Pair, Boolean> ensembleResponse1 = repp.newEnsemble(3, 2, 2, null, + new HashSet<>()); + List ensemble1 = ensembleResponse1.getLeft(); assertEquals(3, getNumCoveredWriteQuorums(ensemble1, 2)); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()); + Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, + new HashSet<>()); + List ensemble2 = ensembleResponse2.getLeft(); assertEquals(4, getNumCoveredWriteQuorums(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception."); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java index 7dc1d39d1d4..3192d049c12 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java @@ -42,6 +42,7 @@ import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,8 +105,9 @@ public void testNotifyRackChange() throws Exception { int ensembleSize = 3; int writeQuorumSize = 2; int acqQuorumSize = 2; - List ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, - Collections.emptyMap(), Collections.emptySet()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, + acqQuorumSize, Collections.emptyMap(), Collections.emptySet()); + List ensemble = ensembleResponse.getLeft(); int numCovered = TestRackawareEnsemblePlacementPolicy.getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); @@ -118,8 +120,9 @@ public void testNotifyRackChange() throws Exception { StaticDNSResolver.changeRack(bookieAddressList, rackList); numOfAvailableRacks = numOfAvailableRacks + 1; acqQuorumSize = 1; - ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, Collections.emptyMap(), + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, Collections.emptyMap(), Collections.emptySet()); + ensemble = ensembleResponse.getLeft(); assertEquals(3, TestRackawareEnsemblePlacementPolicy.getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); assertTrue(ensemble.contains(addr1)); 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 e541230e35b..8e4f10d5b3b 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 @@ -51,6 +51,7 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -423,8 +424,9 @@ public void testReplaceBookieWithEnoughBookiesInSameRegion() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new HashSet(), - addr2, new HashSet()); + Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, + new ArrayList(), addr2, new HashSet()); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); assertEquals(addr3, replacedBookie); } @@ -449,8 +451,9 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRegion() throws Excepti // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, - new HashSet(), addr2, excludedAddrs); + Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, + new ArrayList(), addr2, excludedAddrs); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); assertFalse(addr1.equals(replacedBookie)); assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); @@ -475,7 +478,9 @@ public void testNewEnsembleBookieWithNotEnoughBookies() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - List list = repp.newEnsemble(5, 5, 3, null, new HashSet()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(5, 5, 3, null, + new HashSet()); + List list = ensembleResponse.getLeft(); LOG.info("Ensemble : {}", list); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { @@ -507,7 +512,7 @@ public void testReplaceBookieWithNotEnoughBookies() throws Exception { excludedAddrs.add(addr3); excludedAddrs.add(addr4); try { - repp.replaceBookie(1, 1, 1, null, new HashSet(), addr2, excludedAddrs); + repp.replaceBookie(1, 1, 1, null, new ArrayList(), addr2, excludedAddrs); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not enou @@ -536,11 +541,13 @@ public void testNewEnsembleWithSingleRegion() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(3, 2, 2, null, + Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, new HashSet()); + List ensemble = ensembleResponse.getLeft(); assertEquals(0, getNumCoveredRegionsInWriteQuorum(ensemble, 2)); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, + Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, new HashSet()); + List ensemble2 = ensembleResponse2.getLeft(); assertEquals(0, getNumCoveredRegionsInWriteQuorum(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -569,8 +576,9 @@ public void testNewEnsembleWithMultipleRegions() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(3, 2, 2, null, + Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, new HashSet()); + List ensemble = ensembleResponse.getLeft(); int numCovered = getNumCoveredRegionsInWriteQuorum(ensemble, 2); assertTrue(numCovered >= 1); assertTrue(numCovered < 3); @@ -578,8 +586,9 @@ public void testNewEnsembleWithMultipleRegions() throws Exception { fail("Should not get not enough bookies exception even there is only one rack."); } try { - List ensemble2 = repp.newEnsemble(4, 2, 2, null, + Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, new HashSet()); + List ensemble2 = ensembleResponse2.getLeft(); int numCovered = getNumCoveredRegionsInWriteQuorum(ensemble2, 2); assertTrue(numCovered >= 1 && numCovered < 3); } catch (BKNotEnoughBookiesException bnebe) { @@ -618,11 +627,13 @@ public void testNewEnsembleWithEnoughRegions() throws Exception { addrs.add(addr8); repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble1 = repp.newEnsemble(3, 2, 2, null, + Pair, Boolean> ensembleResponse1 = repp.newEnsemble(3, 2, 2, null, new HashSet()); + List ensemble1 = ensembleResponse1.getLeft(); assertEquals(3, getNumCoveredRegionsInWriteQuorum(ensemble1, 2)); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, + Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, new HashSet()); + List ensemble2 = ensembleResponse2.getLeft(); assertEquals(4, getNumCoveredRegionsInWriteQuorum(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -669,23 +680,27 @@ public void testNewEnsembleWithThreeRegions() throws Exception { addrs.add(addr10); repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(6, 6, 4, null, + Pair, Boolean> ensembleResponse = repp.newEnsemble(6, 6, 4, null, new HashSet()); + List ensemble = ensembleResponse.getLeft(); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 6); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensemble = repp.newEnsemble(7, 7, 4, null, new HashSet()); + ensembleResponse = repp.newEnsemble(7, 7, 4, null, new HashSet()); + ensemble = ensembleResponse.getLeft(); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 7); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensemble = repp.newEnsemble(8, 8, 5, null, new HashSet()); + ensembleResponse = repp.newEnsemble(8, 8, 5, null, new HashSet()); + ensemble = ensembleResponse.getLeft(); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 8); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensemble = repp.newEnsemble(9, 9, 5, null, new HashSet()); + ensembleResponse = repp.newEnsemble(9, 9, 5, null, new HashSet()); + ensemble = ensembleResponse.getLeft(); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 9); @@ -738,8 +753,9 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { repp.onClusterChanged(addrs, new HashSet()); try { ((SettableFeature) featureProvider.scope("region1").getFeature("disallowBookies")).set(true); - List ensemble = repp.newEnsemble(6, 6, 4, null, + Pair, Boolean> ensembleResponse = repp.newEnsemble(6, 6, 4, null, new HashSet()); + List ensemble = ensembleResponse.getLeft(); assertEquals(2, getNumRegionsInEnsemble(ensemble)); assert(ensemble.contains(addr1)); assert(ensemble.contains(addr3)); @@ -753,16 +769,16 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { } try { ((SettableFeature) featureProvider.scope("region2").getFeature("disallowBookies")).set(true); - List ensemble = repp.newEnsemble(6, 6, 4, null, - new HashSet()); + repp.newEnsemble(6, 6, 4, null, new HashSet()); fail("Should get not enough bookies exception even there is only one region with insufficient bookies."); } catch (BKNotEnoughBookiesException bnebe) { // Expected } try { ((SettableFeature) featureProvider.scope("region2").getFeature("disallowBookies")).set(false); - List ensemble = repp.newEnsemble(6, 6, 4, null, + Pair, Boolean> ensembleResponse = repp.newEnsemble(6, 6, 4, null, new HashSet()); + List ensemble = ensembleResponse.getLeft(); assert(ensemble.contains(addr1)); assert(ensemble.contains(addr3)); assert(ensemble.contains(addr4)); @@ -835,8 +851,9 @@ public void testNewEnsembleWithFiveRegions() throws Exception { repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(10, 10, 10, null, + Pair, Boolean> ensembleResponse = repp.newEnsemble(10, 10, 10, null, new HashSet()); + List ensemble = ensembleResponse.getLeft(); assert(ensemble.size() == 10); assertEquals(5, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -847,7 +864,9 @@ public void testNewEnsembleWithFiveRegions() throws Exception { try { Set excludedAddrs = new HashSet(); excludedAddrs.add(addr10); - List ensemble = repp.newEnsemble(10, 10, 10, null, excludedAddrs); + Pair, Boolean> ensembleResponse = repp.newEnsemble(10, 10, 10, null, + excludedAddrs); + List ensemble = ensembleResponse.getLeft(); assert(ensemble.contains(addr11) && ensemble.contains(addr12)); assert(ensemble.size() == 10); assertEquals(5, getNumRegionsInEnsemble(ensemble)); @@ -937,9 +956,11 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole ackQuorum = 5; } + Pair, Boolean> ensembleResponse; List ensemble; try { - ensemble = repp.newEnsemble(6, 6, ackQuorum, null, new HashSet()); + ensembleResponse = repp.newEnsemble(6, 6, ackQuorum, null, new HashSet()); + ensemble = ensembleResponse.getLeft(); assert(ensemble.size() == 6); assertEquals(3, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -960,9 +981,9 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole Set excludedAddrs = new HashSet(); for (BookieSocketAddress addr: region2Bookies) { if (ensemble.contains(addr)) { - BookieSocketAddress replacedBookie = repp.replaceBookie( - 6, 6, ackQuorum, null, - new HashSet<>(ensemble), addr, excludedAddrs); + Pair replaceBookieResponse = repp.replaceBookie(6, 6, ackQuorum, null, + ensemble, addr, excludedAddrs); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); ensemble.remove(addr); ensemble.add(replacedBookie); } @@ -986,9 +1007,9 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole Set excludedAddrs = new HashSet(); try { - BookieSocketAddress replacedBookie = repp.replaceBookie( - 6, 6, ackQuorum, null, - new HashSet<>(ensemble), bookieToReplace, excludedAddrs); + Pair replaceBookieResponse = repp.replaceBookie(6, 6, ackQuorum, null, + ensemble, bookieToReplace, excludedAddrs); + BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); assert (replacedBookie.equals(replacedBookieExpected)); assertEquals(3, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -997,9 +1018,7 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole excludedAddrs.add(replacedBookieExpected); try { - BookieSocketAddress replacedBookie = repp.replaceBookie( - 6, 6, ackQuorum, null, - new HashSet<>(ensemble), bookieToReplace, excludedAddrs); + repp.replaceBookie(6, 6, ackQuorum, null, ensemble, bookieToReplace, excludedAddrs); if (minDurability > 1 && !disableDurabilityFeature.isAvailable()) { fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } @@ -1073,9 +1092,11 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di .set(true); } + Pair, Boolean> ensembleResponse; List ensemble; try { - ensemble = repp.newEnsemble(6, 6, 4, null, new HashSet()); + ensembleResponse = repp.newEnsemble(6, 6, 4, null, new HashSet()); + ensemble = ensembleResponse.getLeft(); assert(ensemble.size() == 6); } catch (BKNotEnoughBookiesException bnebe) { LOG.error("BKNotEnoughBookiesException", bnebe); @@ -1086,9 +1107,7 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di Set excludedAddrs = new HashSet(); try { - repp.replaceBookie( - 6, 6, 4, null, - new HashSet<>(ensemble), addr4, excludedAddrs); + repp.replaceBookie(6, 6, 4, null, ensemble, ensemble.get(2), excludedAddrs); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); } @@ -1141,8 +1160,8 @@ public void testNewEnsembleFailWithFiveRegions() throws Exception { excludedAddrs.add(addr10); excludedAddrs.add(addr9); try { - List list = repp.newEnsemble(5, 5, 5, null, excludedAddrs); - LOG.info("Ensemble : {}", list); + Pair, Boolean> list = repp.newEnsemble(5, 5, 5, null, excludedAddrs); + LOG.info("Ensemble : {}", list.getLeft()); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not enou @@ -1201,8 +1220,9 @@ public void testBasicReorderReadLACSequenceWithLocalRegion() throws Exception { private void basicReorderReadSequenceWithLocalRegionTest(String myRegion, boolean isReadLAC) throws Exception { prepareNetworkTopologyForReorderTests(myRegion); - - List ensemble = repp.newEnsemble(9, 9, 5, null, new HashSet()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(9, 9, 5, null, + new HashSet()); + List ensemble = ensembleResponse.getLeft(); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); @@ -1258,7 +1278,9 @@ public void testBasicReorderReadLACSequenceWithRemoteRegion() throws Exception { private void basicReorderReadSequenceWithRemoteRegionTest(String myRegion, boolean isReadLAC) throws Exception { prepareNetworkTopologyForReorderTests(myRegion); - List ensemble = repp.newEnsemble(9, 9, 5, null, new HashSet()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(9, 9, 5, null, + new HashSet()); + List ensemble = ensembleResponse.getLeft(); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); @@ -1328,7 +1350,9 @@ private void reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(boolean isR prepareNetworkTopologyForReorderTests(myRegion); - List ensemble = repp.newEnsemble(9, 9, 5, null, new HashSet()); + Pair, Boolean> ensembleResponse = repp.newEnsemble(9, 9, 5, null, + new HashSet()); + List ensemble = ensembleResponse.getLeft(); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); From 9936fdcffa6d12b217a7ae686f6329e008b7fd25 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 16 Jan 2019 13:24:12 +0100 Subject: [PATCH 0190/1642] Make Spotbugs pass on JDK11 Descriptions of the changes in this PR: - upgrade spotbugs to 3.1.8 - disable spotbugs on JDK11 (due to for https://github.com/spotbugs/spotbugs/issues/756) ### Motivation This change is on the way to give full "official" support do JDK11 ### Changes - upgrade spotbugs to 3.1.8 - fix new spotbugs issues - disable spotbugs on JDK11 (due to for https://github.com/spotbugs/spotbugs/issues/756) Reviewers: Sijie Guo This closes #1849 from eolivelli/fix/ci-java11 --- .../collections/RecyclableArrayList.java | 11 ++++++++++ pom.xml | 22 +++++++++++++++++-- 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/RecyclableArrayList.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/RecyclableArrayList.java index 4915d7749ac..a4932c0412c 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/RecyclableArrayList.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/RecyclableArrayList.java @@ -64,4 +64,15 @@ public void recycle() { handle.recycle(this); } } + + @Override + public boolean equals(Object obj) { + return super.equals(obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } + } diff --git a/pom.xml b/pom.xml index c0c2c1ab2b5..e8715f7ed13 100644 --- a/pom.xml +++ b/pom.xml @@ -164,7 +164,7 @@ 3.0.1 1.7.25 1.19 - 3.1.1 + 3.1.8 1.3.2 1.8.3 1.29.0 @@ -196,7 +196,7 @@ 1.4.1.Final 0.5.0 6.19 - 3.1.0-RC6 + 3.1.8 @@ -1100,5 +1100,23 @@ stream + + + jdk11-no-spotbugs + + [11,) + + + + + com.github.spotbugs + spotbugs-maven-plugin + + true + + + + + From e7a365604751ba0884e0fef12a7fc95f234e008e Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 17 Jan 2019 01:23:33 +0100 Subject: [PATCH 0191/1642] Move Jobs from Obsolete JDK9 to JDK11 Descriptions of the changes in this PR: - create new jobs for JDK 11 - disable spotbugs on JDK 11 on CI - drop jdk9 and jdk10 from Travis Reviewers: Sijie Guo This closes #1903 from eolivelli/fix/ci-jdk11-part1 --- ...ob_bookkeeper_postcommit_master_java11.groovy} | 6 +++--- ...ovy => job_bookkeeper_precommit_java11.groovy} | 15 ++++++++------- .travis.yml | 5 ----- 3 files changed, 11 insertions(+), 15 deletions(-) rename .test-infra/jenkins/{job_bookkeeper_postcommit_master_java9.groovy => job_bookkeeper_postcommit_master_java11.groovy} (89%) rename .test-infra/jenkins/{job_bookkeeper_precommit_java9.groovy => job_bookkeeper_precommit_java11.groovy} (83%) diff --git a/.test-infra/jenkins/job_bookkeeper_postcommit_master_java9.groovy b/.test-infra/jenkins/job_bookkeeper_postcommit_master_java11.groovy similarity index 89% rename from .test-infra/jenkins/job_bookkeeper_postcommit_master_java9.groovy rename to .test-infra/jenkins/job_bookkeeper_postcommit_master_java11.groovy index 6669104c93a..25423ad7837 100644 --- a/.test-infra/jenkins/job_bookkeeper_postcommit_master_java9.groovy +++ b/.test-infra/jenkins/job_bookkeeper_postcommit_master_java11.groovy @@ -19,15 +19,15 @@ import common_job_properties // This job runs the Java postcommit tests on Java 9 -mavenJob('bookkeeper_postcommit_master_java9') { - description('Runs nightly build for bookkeeper in Java 9.') +mavenJob('bookkeeper_postcommit_master_java11') { + description('Runs nightly build for bookkeeper in Java 11.') // clean up the workspace before build wrappers { preBuildCleanup() } // Set common parameters. common_job_properties.setTopLevelMainJobProperties( - delegate, 'master', 'JDK 1.9 (latest)') + delegate, 'master', 'JDK 11 (latest)') // Sets that this is a PostCommit job. common_job_properties.setPostCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_java11.groovy similarity index 83% rename from .test-infra/jenkins/job_bookkeeper_precommit_java9.groovy rename to .test-infra/jenkins/job_bookkeeper_precommit_java11.groovy index 268d829d5d5..7089d49a806 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_java9.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_java11.groovy @@ -19,8 +19,8 @@ import common_job_properties // This is the Java precommit which runs a maven install, and the current set of precommit tests. -mavenJob('bookkeeper_precommit_pullrequest_java9') { - description('precommit verification for pull requests of Apache BookKeeper in Java 9.') +mavenJob('bookkeeper_precommit_pullrequest_java11') { + description('precommit verification for pull requests of Apache BookKeeper in Java 11.') // clean up the workspace before build wrappers { preBuildCleanup() } @@ -41,7 +41,7 @@ mavenJob('bookkeeper_precommit_pullrequest_java9') { common_job_properties.setTopLevelMainJobProperties( delegate, 'master', - 'JDK 1.9 (latest)', + 'JDK 11 (latest)', 200, 'ubuntu', '${sha1}') @@ -49,13 +49,14 @@ mavenJob('bookkeeper_precommit_pullrequest_java9') { // Sets that this is a PreCommit job. common_job_properties.setPreCommit( delegate, - 'Build (Java 9) (trigger via `rebuild java9`)', - '.*(re)?build java9.*', - '.*\\[x\\] \\[skip build java9\\].*') + 'Build (Java 11) (trigger via `rebuild java11`)', + '.*(re)?build java11.*', + '.*\\[x\\] \\[skip build java11\\].*') // Set Maven parameters. common_job_properties.setMavenConfig(delegate) // Maven build project - goals('clean package spotbugs:check -Dstream -DskipBookKeeperServerTests') + // Spotbugs disabled on JDK11 due to https://github.com/spotbugs/spotbugs/issues/756 + goals('clean package -Dstream -DskipBookKeeperServerTests') } diff --git a/.travis.yml b/.travis.yml index ad53fd5f29f..c9b09d17502 100644 --- a/.travis.yml +++ b/.travis.yml @@ -32,11 +32,6 @@ matrix: osx_image: xcode9.2 - os: linux env: CUSTOM_JDK="oraclejdk8" - # disabled oraclejdk9 since it has unknown failures on https://github.com/apache/bookkeeper/pull/1683 - #- os: linux - # env: CUSTOM_JDK="oraclejdk9" - - os: linux - env: CUSTOM_JDK="oraclejdk10" - os: linux env: CUSTOM_JDK="oraclejdk11" - os: osx From 75b8f15198d6f514f48060b9859628056281f98b Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Wed, 16 Jan 2019 16:26:44 -0800 Subject: [PATCH 0192/1642] Binary metadata format This patch adds a binary metadata format and bumps the metadata format version to 3. The contents of the binary metadata is the same as the contents of the text format for now. The difference is that the binary is more compact, and the fields can be added to the metadata when using the binary format, which isn't possible with the text format. With the text format, parsing with a client that didn't recognise the new field would fail. For now, the text format (version 2) is still used by default. We will provide a tool to allow administrators to bump to version 3. Some tests have been modified to provide digest and password to the builder. All protobuf metadata in released versions has had digest and password (first protobuf metadata was in release-4.2.0). So if new metadata is created or read with version 2, it will have this two fields set. Master issue: #723 Reviewers: Enrico Olivelli , Matteo Merli , Sijie Guo This closes #1866 from ivankelly/binary-metadata --- .../apache/bookkeeper/bookie/BookieShell.java | 2 +- .../apache/bookkeeper/client/BKException.java | 15 + .../bookkeeper/client/api/BKException.java | 9 + .../meta/AbstractZkLedgerManager.java | 22 +- .../bookkeeper/meta/LedgerMetadataSerDe.java | 422 ++++++++++++------ .../meta/MSLedgerManagerFactory.java | 26 +- .../bookkeeper/client/LedgerMetadataTest.java | 32 -- .../client/MetadataUpdateLoopTest.java | 7 + .../bookkeeper/client/MockLedgerHandle.java | 8 +- .../client/TestWatchEnsembleChange.java | 1 + .../meta/AbstractZkLedgerManagerTest.java | 2 + .../apache/bookkeeper/meta/GcLedgersTest.java | 3 + .../bookkeeper/meta/MockLedgerManager.java | 2 +- .../meta/TestLedgerMetadataSerDe.java | 184 ++++++++ .../metadata/etcd/EtcdLedgerManager.java | 23 +- 15 files changed, 579 insertions(+), 179 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java 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 dc9137e6a1a..9df7f49e115 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 @@ -1166,7 +1166,7 @@ Options getOptions() { void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { System.out.println("ledgerID: " + ledgerIdFormatter.formatLedgerId(ledgerId)); if (printMeta) { - System.out.println(new String(new LedgerMetadataSerDe().serialize(md), UTF_8)); + System.out.println(md.toString()); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java index 912fb1ee4c2..b21d74504c4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java @@ -122,6 +122,8 @@ public static BKException create(int code) { return new BKLedgerIdOverflowException(); case Code.SecurityException: return new BKSecurityException(); + case Code.MetadataSerializationException: + return new BKMetadataSerializationException(); default: return new BKUnexpectedConditionException(); } @@ -435,6 +437,19 @@ public BKLedgerIdOverflowException() { } } + /** + * Bookkeeper metadata serialization exception. + */ + public static class BKMetadataSerializationException extends BKException { + public BKMetadataSerializationException() { + super(Code.MetadataSerializationException); + } + + public BKMetadataSerializationException(Throwable cause) { + super(Code.MetadataSerializationException, cause); + } + } + /** * Extract an exception code from an BKException, or use a default if it's another type. * The throwable is null, assume that no exception took place and return diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java index cc28158938a..058a9a744d6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java @@ -167,6 +167,8 @@ public static String getMessage(int code) { return "Bookie operation timeout"; case Code.SecurityException: return "Failed to establish a secure connection"; + case Code.MetadataSerializationException: + return "Failed to serialize metadata"; default: return "Unexpected condition"; } @@ -271,6 +273,13 @@ public interface Code { */ int LedgerIdOverflowException = -106; + /** + * Failure to serialize metadata. + * + * @since 4.9 + */ + int MetadataSerializationException = -107; + /** * Generic exception code used to propagate in replication pipeline. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 5dbdd06791d..96c2f0f026b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -265,9 +265,17 @@ public void processResult(int rc, String path, Object ctx, String name) { } } }; + final byte[] data; + try { + data = serDe.serialize(metadata); + } catch (IOException ioe) { + promise.completeExceptionally(new BKException.BKMetadataSerializationException(ioe)); + return promise; + } + List zkAcls = ZkUtils.getACLs(conf); - ZkUtils.asyncCreateFullPathOptimistic(zk, ledgerPath, serDe.serialize(metadata), zkAcls, - CreateMode.PERSISTENT, scb, null); + ZkUtils.asyncCreateFullPathOptimistic(zk, ledgerPath, data, zkAcls, + CreateMode.PERSISTENT, scb, null); return promise; } @@ -422,8 +430,16 @@ public CompletableFuture> writeLedgerMetadata(long led return promise; } final LongVersion zv = (LongVersion) currentVersion; + + final byte[] data; + try { + data = serDe.serialize(metadata); + } catch (IOException ioe) { + promise.completeExceptionally(new BKException.BKMetadataSerializationException(ioe)); + return promise; + } zk.setData(getLedgerPath(ledgerId), - serDe.serialize(metadata), (int) zv.getLongVersion(), + data, (int) zv.getLongVersion(), new StatCallback() { @Override public void processResult(int rc, String path, Object ctx, Stat stat) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java index 6020a3bf4f1..aab72fbc0cf 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -19,16 +19,23 @@ import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; -import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; import com.google.protobuf.TextFormat; import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.StringReader; -import java.nio.CharBuffer; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.io.UnsupportedEncodingException; import java.util.ArrayList; +import java.util.Base64; import java.util.List; import java.util.Map; import java.util.Optional; @@ -51,122 +58,268 @@ public class LedgerMetadataSerDe { private static final Logger log = LoggerFactory.getLogger(LedgerMetadataSerDe.class); - public static final int CURRENT_METADATA_FORMAT_VERSION = 2; - private static final int LOWEST_COMPAT_METADATA_FORMAT_VERSION = 0; + /** + * Text based manual serialization. + * Available from v4.0.x onwards. + */ + public static final int METADATA_FORMAT_VERSION_1 = 1; + + /** + * Protobuf based, serialized using TextFormat. + * Available from v4.2.x onwards. + * Can contain ctime or not, but if it contains ctime it can only be parse by v4.4.x onwards. + */ + public static final int METADATA_FORMAT_VERSION_2 = 2; + + /** + * Protobuf based, serialized in binary format. + * Available from v4.9.x onwards. + */ + public static final int METADATA_FORMAT_VERSION_3 = 3; + + public static final int MAXIMUM_METADATA_FORMAT_VERSION = METADATA_FORMAT_VERSION_3; + public static final int CURRENT_METADATA_FORMAT_VERSION = METADATA_FORMAT_VERSION_2; + private static final int LOWEST_COMPAT_METADATA_FORMAT_VERSION = METADATA_FORMAT_VERSION_1; // for pulling the version - private static final String VERSION_KEY = "BookieMetadataFormatVersion"; + private static final int MAX_VERSION_DIGITS = 10; + private static final byte[] VERSION_KEY_BYTES = "BookieMetadataFormatVersion\t".getBytes(UTF_8); private static final String LINE_SPLITTER = "\n"; + private static final byte[] LINE_SPLITTER_BYTES = LINE_SPLITTER.getBytes(UTF_8); private static final String FIELD_SPLITTER = "\t"; // old V1 constants private static final String V1_CLOSED_TAG = "CLOSED"; private static final int V1_IN_RECOVERY_ENTRY_ID = -102; - public byte[] serialize(LedgerMetadata metadata) { - if (metadata.getMetadataFormatVersion() == 1) { - return serializeVersion1(metadata); - } - - StringBuilder s = new StringBuilder(); - s.append(VERSION_KEY).append(FIELD_SPLITTER) - .append(CURRENT_METADATA_FORMAT_VERSION).append(LINE_SPLITTER); - s.append(TextFormat.printToString(buildProtoFormat(metadata))); - if (log.isDebugEnabled()) { - log.debug("Serialized config: {}", s); - } - return s.toString().getBytes(UTF_8); + private static void writeHeader(OutputStream os, int version) throws IOException { + os.write(VERSION_KEY_BYTES); + os.write(String.valueOf(version).getBytes(UTF_8)); + os.write(LINE_SPLITTER_BYTES); } - private byte[] serializeVersion1(LedgerMetadata metadata) { - StringBuilder s = new StringBuilder(); - s.append(VERSION_KEY).append(FIELD_SPLITTER) - .append(metadata.getMetadataFormatVersion()).append(LINE_SPLITTER); - s.append(metadata.getWriteQuorumSize()).append(LINE_SPLITTER) - .append(metadata.getEnsembleSize()).append(LINE_SPLITTER).append(metadata.getLength()); - - for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { - s.append(LINE_SPLITTER).append(entry.getKey()); - for (BookieSocketAddress addr : entry.getValue()) { - s.append(FIELD_SPLITTER); - s.append(addr.toString()); - } - } + private static int readHeader(InputStream is) throws IOException { + checkState(LINE_SPLITTER_BYTES.length == 1, "LINE_SPLITTER must be single byte"); - if (metadata.getState() == State.IN_RECOVERY) { - s.append(LINE_SPLITTER).append(V1_IN_RECOVERY_ENTRY_ID) - .append(FIELD_SPLITTER).append(V1_CLOSED_TAG); - } else if (metadata.getState() == State.CLOSED) { - s.append(LINE_SPLITTER).append(metadata.getLastEntryId()) - .append(FIELD_SPLITTER).append(V1_CLOSED_TAG); - } else { - checkArgument(metadata.getState() == State.OPEN, - String.format("Unknown state %s for V1 serialization", metadata.getState())); + for (int i = 0; i < VERSION_KEY_BYTES.length; i++) { + int b = is.read(); + if (b < 0 || ((byte) b) != VERSION_KEY_BYTES[i]) { + throw new IOException("Ledger metadata header corrupt at index " + i); + } } - - if (log.isDebugEnabled()) { - log.debug("Serialized config: {}", s); + byte[] versionBuf = new byte[MAX_VERSION_DIGITS]; + int i = 0; + while (i < MAX_VERSION_DIGITS) { + int b = is.read(); + if (b == LINE_SPLITTER_BYTES[0]) { + String versionStr = new String(versionBuf, 0, i, UTF_8); + try { + return Integer.parseInt(versionStr); + } catch (NumberFormatException nfe) { + throw new IOException("Unable to parse version number from " + versionStr); + } + } else if (b < 0) { + break; + } else { + versionBuf[i++] = (byte) b; + } } - - return s.toString().getBytes(UTF_8); + throw new IOException("Unable to find end of version number, metadata appears corrupt"); } - @VisibleForTesting - public LedgerMetadataFormat buildProtoFormat(LedgerMetadata metadata) { - LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder(); - builder.setQuorumSize(metadata.getWriteQuorumSize()) - .setAckQuorumSize(metadata.getAckQuorumSize()) - .setEnsembleSize(metadata.getEnsembleSize()) - .setLength(metadata.getLength()) - .setLastEntryId(metadata.getLastEntryId()); - - switch (metadata.getState()) { - case CLOSED: - builder.setState(LedgerMetadataFormat.State.CLOSED); + public byte[] serialize(LedgerMetadata metadata) throws IOException { + int formatVersion = metadata.getMetadataFormatVersion(); + final byte[] serialized; + switch (formatVersion) { + case METADATA_FORMAT_VERSION_3: + serialized = serializeVersion3(metadata); break; - case IN_RECOVERY: - builder.setState(LedgerMetadataFormat.State.IN_RECOVERY); + case METADATA_FORMAT_VERSION_2: + serialized = serializeVersion2(metadata); break; - case OPEN: - builder.setState(LedgerMetadataFormat.State.OPEN); + case METADATA_FORMAT_VERSION_1: + serialized = serializeVersion1(metadata); break; default: - checkArgument(false, - String.format("Unknown state %s for protobuf serialization", metadata.getState())); - break; + throw new IllegalArgumentException("Invalid format version " + formatVersion); } + if (log.isDebugEnabled()) { + String serializedStr; + if (formatVersion > METADATA_FORMAT_VERSION_2) { + serializedStr = Base64.getEncoder().encodeToString(serialized); + } else { + serializedStr = new String(serialized, UTF_8); + } + log.debug("Serialized with format {}: {}", formatVersion, serializedStr); + } + return serialized; + } + + private static byte[] serializeVersion3(LedgerMetadata metadata) throws IOException { + try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { + writeHeader(os, METADATA_FORMAT_VERSION_3); + LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder(); + builder.setQuorumSize(metadata.getWriteQuorumSize()) + .setAckQuorumSize(metadata.getAckQuorumSize()) + .setEnsembleSize(metadata.getEnsembleSize()) + .setLength(metadata.getLength()) + .setLastEntryId(metadata.getLastEntryId()); + + switch (metadata.getState()) { + case CLOSED: + builder.setState(LedgerMetadataFormat.State.CLOSED); + break; + case IN_RECOVERY: + builder.setState(LedgerMetadataFormat.State.IN_RECOVERY); + break; + case OPEN: + builder.setState(LedgerMetadataFormat.State.OPEN); + break; + default: + checkArgument(false, + String.format("Unknown state %s for protobuf serialization", metadata.getState())); + break; + } - /** Hack to get around fact that ctime was never versioned correctly */ - if (LedgerMetadataUtils.shouldStoreCtime(metadata)) { builder.setCtime(metadata.getCtime()); - } + builder.setDigestType(apiToProtoDigestType(metadata.getDigestType())); + + serializePassword(metadata.getPassword(), builder); + + Map customMetadata = metadata.getCustomMetadata(); + if (customMetadata.size() > 0) { + LedgerMetadataFormat.cMetadataMapEntry.Builder cMetadataBuilder = + LedgerMetadataFormat.cMetadataMapEntry.newBuilder(); + for (Map.Entry entry : customMetadata.entrySet()) { + cMetadataBuilder.setKey(entry.getKey()).setValue(ByteString.copyFrom(entry.getValue())); + builder.addCustomMetadata(cMetadataBuilder.build()); + } + } - if (metadata.hasPassword()) { - builder.setDigestType(apiToProtoDigestType(metadata.getDigestType())) - .setPassword(ByteString.copyFrom(metadata.getPassword())); + for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { + LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder(); + segmentBuilder.setFirstEntryId(entry.getKey()); + for (BookieSocketAddress addr : entry.getValue()) { + segmentBuilder.addEnsembleMember(addr.toString()); + } + builder.addSegment(segmentBuilder.build()); + } + + builder.build().writeDelimitedTo(os); + return os.toByteArray(); } + } - Map customMetadata = metadata.getCustomMetadata(); - if (customMetadata.size() > 0) { - LedgerMetadataFormat.cMetadataMapEntry.Builder cMetadataBuilder = - LedgerMetadataFormat.cMetadataMapEntry.newBuilder(); - for (Map.Entry entry : customMetadata.entrySet()) { - cMetadataBuilder.setKey(entry.getKey()).setValue(ByteString.copyFrom(entry.getValue())); - builder.addCustomMetadata(cMetadataBuilder.build()); + private static byte[] serializeVersion2(LedgerMetadata metadata) throws IOException { + try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { + writeHeader(os, METADATA_FORMAT_VERSION_2); + try (PrintWriter writer = new PrintWriter(new OutputStreamWriter(os, UTF_8.name()))) { + /*********************************************************************** + * WARNING: Do not modify to add fields. + * This code is purposefully duplicated, as version 2 does not support adding + * fields, and if this code was shared with version 3, it would be easy to + * accidently add new fields and create BC issues. + **********************************************************************/ + LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder(); + builder.setQuorumSize(metadata.getWriteQuorumSize()) + .setAckQuorumSize(metadata.getAckQuorumSize()) + .setEnsembleSize(metadata.getEnsembleSize()) + .setLength(metadata.getLength()) + .setLastEntryId(metadata.getLastEntryId()); + + switch (metadata.getState()) { + case CLOSED: + builder.setState(LedgerMetadataFormat.State.CLOSED); + break; + case IN_RECOVERY: + builder.setState(LedgerMetadataFormat.State.IN_RECOVERY); + break; + case OPEN: + builder.setState(LedgerMetadataFormat.State.OPEN); + break; + default: + checkArgument(false, + String.format("Unknown state %s for protobuf serialization", metadata.getState())); + break; + } + + /** Hack to get around fact that ctime was never versioned correctly */ + if (LedgerMetadataUtils.shouldStoreCtime(metadata)) { + builder.setCtime(metadata.getCtime()); + } + + builder.setDigestType(apiToProtoDigestType(metadata.getDigestType())); + serializePassword(metadata.getPassword(), builder); + + Map customMetadata = metadata.getCustomMetadata(); + if (customMetadata.size() > 0) { + LedgerMetadataFormat.cMetadataMapEntry.Builder cMetadataBuilder = + LedgerMetadataFormat.cMetadataMapEntry.newBuilder(); + for (Map.Entry entry : customMetadata.entrySet()) { + cMetadataBuilder.setKey(entry.getKey()).setValue(ByteString.copyFrom(entry.getValue())); + builder.addCustomMetadata(cMetadataBuilder.build()); + } + } + + for (Map.Entry> entry : + metadata.getAllEnsembles().entrySet()) { + LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder(); + segmentBuilder.setFirstEntryId(entry.getKey()); + for (BookieSocketAddress addr : entry.getValue()) { + segmentBuilder.addEnsembleMember(addr.toString()); + } + builder.addSegment(segmentBuilder.build()); + } + + TextFormat.print(builder.build(), writer); + writer.flush(); } + return os.toByteArray(); } + } + + private static byte[] serializeVersion1(LedgerMetadata metadata) throws IOException { + try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { + writeHeader(os, METADATA_FORMAT_VERSION_1); + + try (PrintWriter writer = new PrintWriter(new OutputStreamWriter(os, UTF_8.name()))) { + writer.append(String.valueOf(metadata.getWriteQuorumSize())).append(LINE_SPLITTER); + writer.append(String.valueOf(metadata.getEnsembleSize())).append(LINE_SPLITTER); + writer.append(String.valueOf(metadata.getLength())).append(LINE_SPLITTER); - for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { - LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder(); - segmentBuilder.setFirstEntryId(entry.getKey()); - for (BookieSocketAddress addr : entry.getValue()) { - segmentBuilder.addEnsembleMember(addr.toString()); + for (Map.Entry> entry : + metadata.getAllEnsembles().entrySet()) { + writer.append(String.valueOf(entry.getKey())); + for (BookieSocketAddress addr : entry.getValue()) { + writer.append(FIELD_SPLITTER).append(addr.toString()); + } + writer.append(LINE_SPLITTER); + } + + if (metadata.getState() == State.IN_RECOVERY) { + writer.append(String.valueOf(V1_IN_RECOVERY_ENTRY_ID)).append(FIELD_SPLITTER).append(V1_CLOSED_TAG); + } else if (metadata.getState() == State.CLOSED) { + writer.append(String.valueOf(metadata.getLastEntryId())) + .append(FIELD_SPLITTER).append(V1_CLOSED_TAG); + } else { + checkArgument(metadata.getState() == State.OPEN, + String.format("Unknown state %s for V1 serialization", metadata.getState())); + } + writer.flush(); + } catch (UnsupportedEncodingException uee) { + throw new RuntimeException("UTF_8 should be supported everywhere"); } - builder.addSegment(segmentBuilder.build()); + return os.toByteArray(); } - return builder.build(); } + private static void serializePassword(byte[] password, LedgerMetadataFormat.Builder builder) { + if (password == null || password.length == 0) { + builder.setPassword(ByteString.EMPTY); + } else { + builder.setPassword(ByteString.copyFrom(password)); + } + } /** * Parses a given byte array and transforms into a LedgerConfig object. @@ -181,55 +334,65 @@ public LedgerMetadataFormat buildProtoFormat(LedgerMetadata metadata) { */ public LedgerMetadata parseConfig(byte[] bytes, Optional metadataStoreCtime) throws IOException { - String config = new String(bytes, UTF_8); - if (log.isDebugEnabled()) { - log.debug("Parsing Config: {}", config); - } - BufferedReader reader = new BufferedReader(new StringReader(config)); - String versionLine = reader.readLine(); - if (versionLine == null) { - throw new IOException("Invalid metadata. Content missing"); - } - final int metadataFormatVersion; - if (versionLine.startsWith(VERSION_KEY)) { - String parts[] = versionLine.split(FIELD_SPLITTER); - metadataFormatVersion = Integer.parseInt(parts[1]); - } else { - // if no version is set, take it to be version 1 - // as the parsing is the same as what we had before - // we introduce versions - metadataFormatVersion = 1; - // reset the reader - reader.close(); - reader = new BufferedReader(new StringReader(config)); - } - - if (metadataFormatVersion < LOWEST_COMPAT_METADATA_FORMAT_VERSION - || metadataFormatVersion > CURRENT_METADATA_FORMAT_VERSION) { - throw new IOException( - String.format("Metadata version not compatible. Expected between %d and %d, but got %d", - LOWEST_COMPAT_METADATA_FORMAT_VERSION, CURRENT_METADATA_FORMAT_VERSION, - metadataFormatVersion)); + log.debug("Deserializing {}", Base64.getEncoder().encodeToString(bytes)); } + try (ByteArrayInputStream is = new ByteArrayInputStream(bytes)) { + int metadataFormatVersion = readHeader(is); + if (log.isDebugEnabled()) { + String contentStr = ""; + if (metadataFormatVersion <= METADATA_FORMAT_VERSION_2) { + contentStr = ", content: " + new String(bytes, UTF_8); + } + log.debug("Format version {} detected{}", metadataFormatVersion, contentStr); + } - if (metadataFormatVersion == 1) { - return parseVersion1Config(reader); + switch (metadataFormatVersion) { + case METADATA_FORMAT_VERSION_3: + return parseVersion3Config(is); + case METADATA_FORMAT_VERSION_2: + return parseVersion2Config(is, metadataStoreCtime); + case METADATA_FORMAT_VERSION_1: + return parseVersion1Config(is); + default: + throw new IOException( + String.format("Metadata version not compatible. Expected between %d and %d, but got %d", + LOWEST_COMPAT_METADATA_FORMAT_VERSION, CURRENT_METADATA_FORMAT_VERSION, + metadataFormatVersion)); + } } + } + private static LedgerMetadata parseVersion3Config(InputStream is) throws IOException { LedgerMetadataBuilder builder = LedgerMetadataBuilder.create() - .withMetadataFormatVersion(metadataFormatVersion); + .withMetadataFormatVersion(METADATA_FORMAT_VERSION_3); + LedgerMetadataFormat.Builder formatBuilder = LedgerMetadataFormat.newBuilder(); + formatBuilder.mergeDelimitedFrom(is); + decodeFormat(formatBuilder.build(), builder); + return builder.build(); + } - // remaining size is total minus the length of the version line and '\n' - char[] configBuffer = new char[config.length() - (versionLine.length() + 1)]; - if (configBuffer.length != reader.read(configBuffer, 0, configBuffer.length)) { - throw new IOException("Invalid metadata buffer"); - } + private static LedgerMetadata parseVersion2Config(InputStream is, Optional metadataStoreCtime) + throws IOException { + LedgerMetadataBuilder builder = LedgerMetadataBuilder.create() + .withMetadataFormatVersion(METADATA_FORMAT_VERSION_2); LedgerMetadataFormat.Builder formatBuilder = LedgerMetadataFormat.newBuilder(); - TextFormat.merge((CharSequence) CharBuffer.wrap(configBuffer), formatBuilder); + try (InputStreamReader reader = new InputStreamReader(is, UTF_8.name())) { + TextFormat.merge(reader, formatBuilder); + } LedgerMetadataFormat data = formatBuilder.build(); + decodeFormat(data, builder); + if (data.hasCtime()) { + // 'storingCreationTime' is only ever taken into account for serializing version 2 + builder.storingCreationTime(true); + } else if (metadataStoreCtime.isPresent()) { + builder.withCreationTime(metadataStoreCtime.get()).storingCreationTime(false); + } + return builder.build(); + } + private static void decodeFormat(LedgerMetadataFormat data, LedgerMetadataBuilder builder) throws IOException { builder.withEnsembleSize(data.getEnsembleSize()); builder.withWriteQuorumSize(data.getQuorumSize()); if (data.hasAckQuorumSize()) { @@ -239,9 +402,7 @@ public LedgerMetadata parseConfig(byte[] bytes, } if (data.hasCtime()) { - builder.withCreationTime(data.getCtime()).storingCreationTime(true); - } else if (metadataStoreCtime.isPresent()) { - builder.withCreationTime(metadataStoreCtime.get()).storingCreationTime(false); + builder.withCreationTime(data.getCtime()); } if (data.getState() == LedgerMetadataFormat.State.IN_RECOVERY) { @@ -268,12 +429,11 @@ public LedgerMetadata parseConfig(byte[] bytes, Collectors.toMap(e -> e.getKey(), e -> e.getValue().toByteArray()))); } - return builder.build(); } - static LedgerMetadata parseVersion1Config(BufferedReader reader) throws IOException { - LedgerMetadataBuilder builder = LedgerMetadataBuilder.create().withMetadataFormatVersion(1); - try { + private static LedgerMetadata parseVersion1Config(InputStream is) throws IOException { + try (BufferedReader reader = new BufferedReader(new InputStreamReader(is, UTF_8.name()))) { + LedgerMetadataBuilder builder = LedgerMetadataBuilder.create().withMetadataFormatVersion(1); int quorumSize = Integer.parseInt(reader.readLine()); int ensembleSize = Integer.parseInt(reader.readLine()); long length = Long.parseLong(reader.readLine()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index fc876324452..0834147a8c5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -395,8 +395,15 @@ public void complete(int rc, Version version, Object ctx) { } }; - ledgerTable.put(ledgerId2Key(lid), new Value().setField(META_FIELD, serDe.serialize(metadata)), - Version.NEW, msCallback, null); + final byte[] bytes; + try { + bytes = serDe.serialize(metadata); + } catch (IOException ioe) { + promise.completeExceptionally(new BKException.BKMetadataSerializationException(ioe)); + return promise; + } + ledgerTable.put(ledgerId2Key(lid), new Value().setField(META_FIELD, bytes), + Version.NEW, msCallback, null); return promise; } @@ -413,7 +420,7 @@ public void complete(int rc, Void value, Object ctx) { } else if (MSException.Code.OK.getCode() == rc) { FutureUtils.complete(promise, null); } else { - promise.completeExceptionally(new BKException.MetaStoreException()); + promise.completeExceptionally(new BKException.BKMetadataSerializationException()); } } }; @@ -457,13 +464,22 @@ public void complete(int rc, Versioned value, Object ctx) { @Override public CompletableFuture> writeLedgerMetadata(long ledgerId, LedgerMetadata metadata, Version currentVersion) { - Value data = new Value().setField(META_FIELD, serDe.serialize(metadata)); + + CompletableFuture> promise = new CompletableFuture<>(); + final byte[] bytes; + try { + bytes = serDe.serialize(metadata); + } catch (IOException ioe) { + promise.completeExceptionally(new BKException.MetaStoreException(ioe)); + return promise; + } + + Value data = new Value().setField(META_FIELD, bytes); if (LOG.isDebugEnabled()) { LOG.debug("Writing ledger {} metadata, version {}", new Object[] { ledgerId, currentVersion }); } - CompletableFuture> promise = new CompletableFuture<>(); final String key = ledgerId2Key(ledgerId); MetastoreCallback msCallback = new MetastoreCallback() { @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java index a9ffc2d4221..6fee301e70d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java @@ -30,9 +30,7 @@ import java.util.List; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; -import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; import org.junit.Test; /** @@ -68,36 +66,6 @@ public void testGetters() { assertEquals(ensemble, metadata.getEnsembleAt(99L)); } - @Test - public void testStoreSystemtimeAsLedgerCtimeEnabled() - throws Exception { - List ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234)); - LedgerMetadata lm = LedgerMetadataBuilder.create() - .newEnsembleEntry(0L, ensemble) - .withCreationTime(System.currentTimeMillis()) - .storingCreationTime(true) - .build(); - LedgerMetadataFormat format = new LedgerMetadataSerDe().buildProtoFormat(lm); - assertTrue(format.hasCtime()); - } - - @Test - public void testStoreSystemtimeAsLedgerCtimeDisabled() - throws Exception { - List ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234)); - LedgerMetadata lm = LedgerMetadataBuilder.create() - .newEnsembleEntry(0L, ensemble).build(); - - LedgerMetadataFormat format = new LedgerMetadataSerDe().buildProtoFormat(lm); - assertFalse(format.hasCtime()); - } - @Test public void testToString() { List ensemble = Lists.newArrayList( diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java index ffacb21f05d..b96859b217a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java @@ -42,6 +42,7 @@ import lombok.AllArgsConstructor; import lombok.Data; +import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.MockLedgerManager; @@ -69,6 +70,7 @@ public class MetadataUpdateLoopTest { public void testBasicUpdate() throws Exception { try (LedgerManager lm = new MockLedgerManager()) { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(5) + .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .newEnsembleEntry(0L, Lists.newArrayList( new BookieSocketAddress("0.0.0.0:3181"), new BookieSocketAddress("0.0.0.1:3181"), @@ -115,6 +117,7 @@ public void testConflictOnWrite() throws Exception { BookieSocketAddress b3 = new BookieSocketAddress("0.0.0.3:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) + .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .withWriteQuorumSize(2).newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); @@ -181,6 +184,7 @@ public void testConflictOnWriteBothWritingSame() throws Exception { BookieSocketAddress b2 = new BookieSocketAddress("0.0.0.2:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) + .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .withWriteQuorumSize(2).newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); @@ -234,6 +238,7 @@ public void testConflictOnLocalUpdate() throws Exception { BookieSocketAddress b3 = new BookieSocketAddress("0.0.0.3:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) + .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .withWriteQuorumSize(2).newEnsembleEntry(0L, Lists.newArrayList(b0, b1)).build(); Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); @@ -299,6 +304,7 @@ public void testHammer() throws Exception { .collect(Collectors.toList()); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(ensembleSize) + .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .newEnsembleEntry(0L, initialEnsemble).build(); Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); @@ -342,6 +348,7 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { BookieSocketAddress b1 = new BookieSocketAddress("0.0.0.1:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(1) + .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .withWriteQuorumSize(1).withAckQuorumSize(1) .newEnsembleEntry(0L, Lists.newArrayList(b0)).build(); Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java index 3aaeedb56ac..9d68ec8a145 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java @@ -65,8 +65,8 @@ public class MockLedgerHandle extends LedgerHandle { MockLedgerHandle(MockBookKeeper bk, long id, DigestType digest, byte[] passwd) throws GeneralSecurityException { super(bk.getClientCtx(), id, - new Versioned<>(createMetadata(), new LongVersion(0L)), - DigestType.MAC, "".getBytes(), WriteFlag.NONE); + new Versioned<>(createMetadata(digest, passwd), new LongVersion(0L)), + digest, passwd, WriteFlag.NONE); this.bk = bk; this.id = id; this.digest = digest; @@ -268,12 +268,14 @@ public CompletableFuture readLastAddConfirmedAndEntryAsyn return readHandle.readLastAddConfirmedAndEntryAsync(entryId, timeOutInMillis, parallel); } - private static LedgerMetadata createMetadata() { + private static LedgerMetadata createMetadata(DigestType digest, byte[] passwd) { List ensemble = Lists.newArrayList( new BookieSocketAddress("192.0.2.1", 1234), new BookieSocketAddress("192.0.2.2", 1234), new BookieSocketAddress("192.0.2.3", 1234)); return LedgerMetadataBuilder.create() + .withDigestType(digest.toApiDigestType()) + .withPassword(passwd) .newEnsembleEntry(0L, ensemble) .build(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java index 5c6a8c722da..26a265b6a8f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java @@ -141,6 +141,7 @@ private void testWatchMetadataRemoval(LedgerManagerFactory factory) throws Excep @Override public void operationComplete(int rc, final Long lid) { LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withDigestType(digestType.toApiDigestType()).withPassword(new byte[0]) .withEnsembleSize(4).withWriteQuorumSize(2) .withAckQuorumSize(2) .newEnsembleEntry(0L, ensemble).build(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 37f55b3e6e6..5776af363b3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -51,6 +51,7 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.Code; import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.testing.executors.MockExecutorController; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -121,6 +122,7 @@ public void setup() throws Exception { new BookieSocketAddress("192.0.2.4", 3181), new BookieSocketAddress("192.0.2.5", 3181)); this.metadata = LedgerMetadataBuilder.create() + .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .withEnsembleSize(5) .withWriteQuorumSize(3) .withAckQuorumSize(3) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 31bd406d489..933c11763f0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -65,6 +65,7 @@ import org.apache.bookkeeper.bookie.StateManager; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -113,6 +114,8 @@ public void operationComplete(int rc, final Long ledgerId) { } LedgerMetadata md = LedgerMetadataBuilder.create() + .withDigestType(DigestType.CRC32C) + .withPassword(new byte[0]) .withEnsembleSize(1).withWriteQuorumSize(1).withAckQuorumSize(1) .newEnsembleEntry(0L, ensemble).build(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index 398bb07d712..f5cbe3a7e65 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -103,8 +103,8 @@ public CompletableFuture> createLedgerMetadata(long le if (metadataMap.containsKey(ledgerId)) { executeCallback(() -> promise.completeExceptionally(new BKException.BKLedgerExistException())); } else { - metadataMap.put(ledgerId, Pair.of(new LongVersion(0L), serDe.serialize(metadata))); try { + metadataMap.put(ledgerId, Pair.of(new LongVersion(0L), serDe.serialize(metadata))); Versioned readBack = readMetadata(ledgerId); executeCallback(() -> promise.complete(readBack)); } catch (Exception e) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java new file mode 100644 index 00000000000..d9ce5a2cd50 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java @@ -0,0 +1,184 @@ +/** + * 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.meta; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.collect.Lists; +import java.io.IOException; +import java.util.Base64; +import java.util.Optional; +import java.util.Random; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.junit.Assert; +import org.junit.Test; + +/** + * Test Ledger Metadata serialization and deserialization. + */ +public class TestLedgerMetadataSerDe { + // as used in 4.0.x & 4.1.x + private static final String version1 = + "Qm9va2llTWV0YWRhdGFGb3JtYXRWZXJzaW9uCTEKMgozCjAKMAkxOTIuMC4yLjE6MTIzNAkxOTIu" + + "MC4yLjI6MTIzNAkxOTIuMC4yLjM6MTIzNAotMTAyCUNMT1NFRA=="; + + // as used in 4.2.x & 4.3.x (text protobuf based metadata, password and digest introduced) + private static final String version2 = + "Qm9va2llTWV0YWRhdGFGb3JtYXRWZXJzaW9uCTIKcXVvcnVtU2l6ZTogMgplbnNlbWJsZVNpemU6I" + + "DMKbGVuZ3RoOiAwCmxhc3RFbnRyeUlkOiAtMQpzdGF0ZTogSU5fUkVDT1ZFUlkKc2VnbWVudCB7" + + "CiAgZW5zZW1ibGVNZW1iZXI6ICIxOTIuMC4yLjE6MTIzNCIKICBlbnNlbWJsZU1lbWJlcjogIjE" + + "5Mi4wLjIuMjoxMjM0IgogIGVuc2VtYmxlTWVtYmVyOiAiMTkyLjAuMi4zOjEyMzQiCiAgZmlyc3" + + "RFbnRyeUlkOiAwCn0KZGlnZXN0VHlwZTogQ1JDMzIKcGFzc3dvcmQ6ICJwYXNzd2QiCmFja1F1b" + + "3J1bVNpemU6IDIK"; + + // version 2 + ctime, as used in 4.4.x to 4.8.x (ctime is optional from 4.6.x onwards) + private static final String version2ctime = + "Qm9va2llTWV0YWRhdGFGb3JtYXRWZXJzaW9uCTIKcXVvcnVtU2l6ZTogMgplbnNlbWJsZVNpemU6I" + + "DMKbGVuZ3RoOiAwCmxhc3RFbnRyeUlkOiAtMQpzdGF0ZTogSU5fUkVDT1ZFUlkKc2VnbWVudCB7" + + "CiAgZW5zZW1ibGVNZW1iZXI6ICIxOTIuMC4yLjE6MTIzNCIKICBlbnNlbWJsZU1lbWJlcjogIjE" + + "5Mi4wLjIuMjoxMjM0IgogIGVuc2VtYmxlTWVtYmVyOiAiMTkyLjAuMi4zOjEyMzQiCiAgZmlyc3" + + "RFbnRyeUlkOiAwCn0KZGlnZXN0VHlwZTogQ1JDMzIKcGFzc3dvcmQ6ICJwYXNzd2QiCmFja1F1b" + + "3J1bVNpemU6IDIKY3RpbWU6IDE1NDQwMDIzODMwNzUK"; + + // version 3, since 4.9.x, protobuf binary format + private static final String version3 = + "Qm9va2llTWV0YWRhdGFGb3JtYXRWZXJzaW9uCTMKXggCEAMYACD///////////8BKAEyMgoOMTkyL" + + "jAuMi4xOjMxODEKDjE5Mi4wLjIuMjozMTgxCg4xOTIuMC4yLjM6MzE4MRAAOANCBmZvb2JhckgB" + + "UP///////////wE="; + + private static void testDecodeEncode(String encoded) throws Exception { + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + LedgerMetadata md = serDe.parseConfig(Base64.getDecoder().decode(encoded), Optional.empty()); + String reserialized = Base64.getEncoder().encodeToString(serDe.serialize(md)); + + Assert.assertEquals(encoded, reserialized); + } + + @Test + public void testVersion1SerDe() throws Exception { + testDecodeEncode(version1); + } + + @Test + public void testVersion2SerDe() throws Exception { + testDecodeEncode(version2); + } + + @Test + public void testVersion2CtimeSerDe() throws Exception { + testDecodeEncode(version2ctime); + } + + @Test + public void testVersion3SerDe() throws Exception { + testDecodeEncode(version3); + } + + @Test(expected = IOException.class) + public void testJunkSerDe() throws Exception { + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + String junk = ""; + serDe.parseConfig(junk.getBytes(UTF_8), Optional.empty()); + } + + @Test(expected = IOException.class) + public void testJunk2SerDe() throws Exception { + byte[] randomBytes = new byte[1000]; + new Random().nextBytes(randomBytes); + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + serDe.parseConfig(randomBytes, Optional.empty()); + } + + @Test(expected = IOException.class) + public void testJunkVersionSerDe() throws Exception { + byte[] junkVersion = "BookieMetadataFormatVersion\tfoobar\nblahblah".getBytes(UTF_8); + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + serDe.parseConfig(junkVersion, Optional.empty()); + } + + @Test(expected = IOException.class) + public void testVeryLongVersionSerDe() throws Exception { + byte[] veryLongVersion = "BookieMetadataFormatVersion\t123456789123456789\nblahblah".getBytes(UTF_8); + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + serDe.parseConfig(veryLongVersion, Optional.empty()); + } + + @Test + public void testPeggedToV2SerDe() throws Exception { + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) + .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) + .newEnsembleEntry(0L, Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 3181), + new BookieSocketAddress("192.0.2.2", 3181), + new BookieSocketAddress("192.0.2.3", 3181))) + .build(); + byte[] encoded = serDe.serialize(metadata); + + LedgerMetadata decoded = serDe.parseConfig(encoded, Optional.empty()); + Assert.assertEquals(2, decoded.getMetadataFormatVersion()); + } + + @Test + public void testStoreSystemtimeAsLedgerCtimeEnabledWithVersion2() + throws Exception { + LedgerMetadata lm = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) + .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) + .newEnsembleEntry(0L, Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234))) + .withCreationTime(123456L) + .storingCreationTime(true) + .build(); + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + byte[] serialized = serDe.serialize(lm); + LedgerMetadata deserialized = serDe.parseConfig(serialized, Optional.of(654321L)); + Assert.assertEquals(deserialized.getCtime(), 123456L); + + // give it another round + LedgerMetadata deserialized2 = serDe.parseConfig(serDe.serialize(deserialized), Optional.of(98765L)); + Assert.assertEquals(deserialized2.getCtime(), 123456L); + } + + @Test + public void testStoreSystemtimeAsLedgerCtimeDisabledWithVersion2() + throws Exception { + LedgerMetadata lm = LedgerMetadataBuilder.create() + .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) + .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) + .newEnsembleEntry(0L, Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1234), + new BookieSocketAddress("192.0.2.2", 1234), + new BookieSocketAddress("192.0.2.3", 1234))) + .build(); + + LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + byte[] serialized = serDe.serialize(lm); + LedgerMetadata deserialized = serDe.parseConfig(serialized, Optional.of(654321L)); + Assert.assertEquals(deserialized.getCtime(), 654321L); + + // give it another round + LedgerMetadata deserialized2 = serDe.parseConfig(serDe.serialize(deserialized), Optional.of(98765L)); + Assert.assertEquals(deserialized2.getCtime(), 98765L); + } +} diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index d571bf8057e..4988a810fca 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -110,8 +110,16 @@ public CompletableFuture> createLedgerMetadata(long le LedgerMetadata metadata) { CompletableFuture> promise = new CompletableFuture<>(); String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); - ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); log.info("Create ledger metadata under key {}", ledgerKey); + + ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); + final ByteSequence valueBs; + try { + valueBs = ByteSequence.fromBytes(serDe.serialize(metadata)); + } catch (IOException ioe) { + promise.completeExceptionally(new BKException.BKMetadataSerializationException(ioe)); + return promise; + } kvClient.txn() .If(new Cmp( ledgerKeyBs, @@ -124,7 +132,7 @@ public CompletableFuture> createLedgerMetadata(long le .build())) .Else(com.coreos.jetcd.op.Op.put( ledgerKeyBs, - ByteSequence.fromBytes(serDe.serialize(metadata)), + valueBs, PutOption.DEFAULT)) .commit() .thenAccept(resp -> { @@ -255,6 +263,15 @@ public CompletableFuture> writeLedgerMetadata(long led final LongVersion lv = (LongVersion) currentVersion; String ledgerKey = EtcdUtils.getLedgerKey(scope, ledgerId); ByteSequence ledgerKeyBs = ByteSequence.fromString(ledgerKey); + + final ByteSequence valueBs; + try { + valueBs = ByteSequence.fromBytes(serDe.serialize(metadata)); + } catch (IOException ioe) { + promise.completeExceptionally(new BKException.BKMetadataSerializationException(ioe)); + return promise; + } + kvClient.txn() .If(new Cmp( ledgerKeyBs, @@ -262,7 +279,7 @@ public CompletableFuture> writeLedgerMetadata(long led CmpTarget.modRevision(lv.getLongVersion()))) .Then(com.coreos.jetcd.op.Op.put( ledgerKeyBs, - ByteSequence.fromBytes(serDe.serialize(metadata)), + valueBs, PutOption.DEFAULT)) .Else(com.coreos.jetcd.op.Op.get( ledgerKeyBs, From 0525c8fb8d30ff0da374160d4f5094953ffee3e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Herv=C3=A9=20Boutemy?= Date: Thu, 17 Jan 2019 12:34:26 +0100 Subject: [PATCH 0193/1642] replace default source-release-assembly by bookkeeper-dist fix for #1895 Reviewers: Enrico Olivelli , Sijie Guo This closes #1904 from hboutemy/master --- bookkeeper-dist/pom.xml | 9 --------- pom.xml | 20 ++++++++++++++++++++ 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/bookkeeper-dist/pom.xml b/bookkeeper-dist/pom.xml index b81fe60d426..6bc7b1e8967 100644 --- a/bookkeeper-dist/pom.xml +++ b/bookkeeper-dist/pom.xml @@ -65,15 +65,6 @@ - - - org.apache.maven.plugins - maven-deploy-plugin - ${maven-deploy-plugin.version} - - true - - diff --git a/pom.xml b/pom.xml index e8715f7ed13..cf355666d00 100644 --- a/pom.xml +++ b/pom.xml @@ -1118,5 +1118,25 @@ + + + apache-release + + + + maven-assembly-plugin + + + source-release-assembly + + + true + + + + + + + From 526954264c9f1801e23a629265d42c412fd53311 Mon Sep 17 00:00:00 2001 From: "Huang,Dongfa" Date: Mon, 21 Jan 2019 16:54:59 +0800 Subject: [PATCH 0194/1642] Small fix wrong nodesUninitialized count when checkCovered Descriptions of the changes in this PR: ### Motivation Since count `nodesUninitialized` is always 0, there is no wait if we haven't seen any OK responses and there are still nodes not heard from ### Changes Correct nodesUninitialized count and add a related testcase Reviewers: Enrico Olivelli , Sijie Guo This closes #1900 from huangdongfa/fix-error-nodesUninitialized --- .../client/RoundRobinDistributionSchedule.java | 5 +++-- .../client/RoundRobinDistributionScheduleTest.java | 10 ++++++++++ .../bookkeeper/replication/BookieAutoRecoveryTest.java | 2 +- 3 files changed, 14 insertions(+), 3 deletions(-) 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 d07940823b9..a6506afe72d 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 @@ -380,8 +380,9 @@ public synchronized boolean checkCovered() { } else if (covered[nodeIndex] != BKException.Code.NoSuchEntryException && covered[nodeIndex] != BKException.Code.NoSuchLedgerExistsException) { nodesNotCovered++; - } else if (covered[nodeIndex] == BKException.Code.UNINITIALIZED) { - nodesUninitialized++; + if (covered[nodeIndex] == BKException.Code.UNINITIALIZED) { + nodesUninitialized++; + } } } // if we haven't seen any OK responses and there are still nodes not heard from, 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 b78f1adf090..b78897a4cb8 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 @@ -73,6 +73,16 @@ public void testCoverageSets() { } } assertEquals("Should be no errors", 0, errors); + + RoundRobinDistributionSchedule schedule = new RoundRobinDistributionSchedule( + 5, 3, 5); + DistributionSchedule.QuorumCoverageSet covSet = schedule.getCoverageSet(); + covSet.addBookie(0, BKException.Code.NoSuchLedgerExistsException); + covSet.addBookie(1, BKException.Code.NoSuchEntryException); + covSet.addBookie(2, BKException.Code.NoSuchLedgerExistsException); + covSet.addBookie(3, BKException.Code.UNINITIALIZED); + covSet.addBookie(4, BKException.Code.UNINITIALIZED); + assertFalse(covSet.checkCovered()); } /** 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 45cd2eb7e18..0a6d264f643 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 @@ -390,7 +390,7 @@ public void testEmptyLedgerLosesQuorumEventually() throws Exception { latch = new CountDownLatch(1); Stat s = watchUrLedgerNode(urZNode, latch); // should be marked as replicated if (s != null) { - assertTrue("Should be marked as replicated", latch.await(10, TimeUnit.SECONDS)); + assertTrue("Should be marked as replicated", latch.await(15, TimeUnit.SECONDS)); } replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(1); From aeb83797ca27db5e429f137d739dc9e20f7a4f2b Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Mon, 21 Jan 2019 00:56:22 -0800 Subject: [PATCH 0195/1642] HTTP GetMetaService returns JSON rather than calling #serialize Once the binary ledger metadata serializing is available, the output of GetMetaService would no longer be understandable. In preparation for this, make GetMetaService use output JSON, rather than directly calling #serialize. Master issue: #723 Reviewers: Enrico Olivelli , Sijie Guo This closes #1864 from ivankelly/get-to-string --- .../org/apache/bookkeeper/client/LedgerCreateOp.java | 2 +- .../apache/bookkeeper/net/BookieSocketAddress.java | 3 +++ .../server/http/service/GetLedgerMetaService.java | 5 ++--- .../bookkeeper/server/http/TestHttpService.java | 12 ++++++++---- 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java index eaa316360c0..9e9a2c4add9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java @@ -132,7 +132,7 @@ public void initiate() { metadataBuilder.withCustomMetadata(customMetadata); } if (bk.getConf().getStoreSystemtimeAsLedgerCreationTime()) { - metadataBuilder.withCreationTime(System.currentTimeMillis()); + metadataBuilder.withCreationTime(System.currentTimeMillis()).storingCreationTime(true); } // select bookies for first ensemble diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java index cd0a20cfa4d..e9fc0f57e13 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java @@ -22,6 +22,7 @@ import static org.apache.bookkeeper.util.BookKeeperConstants.COLON; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.net.InetAddresses; import io.netty.channel.local.LocalAddress; @@ -91,6 +92,7 @@ public int getPort() { } // Method to return an InetSocketAddress for the regular port. + @JsonIgnore public InetSocketAddress getSocketAddress() { /* * Return each time a new instance of the InetSocketAddress if hostname @@ -106,6 +108,7 @@ public InetSocketAddress getSocketAddress() { /** * Maps the socketAddress to a "local" address. */ + @JsonIgnore public LocalAddress getLocalAddress() { // for local address, we just need "port" to differentiate different addresses. return new LocalAddress("" + port); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java index 7fcaeda810d..451bf08ca71 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java @@ -18,7 +18,6 @@ */ package org.apache.bookkeeper.server.http.service; -import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.collect.Maps; @@ -68,9 +67,9 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { LedgerManager manager = mFactory.newLedgerManager(); // output - Map output = Maps.newHashMap(); + Map output = Maps.newHashMap(); LedgerMetadata md = manager.readLedgerMetadata(ledgerId).get().getValue(); - output.put(ledgerId.toString(), new String(serDe.serialize(md), UTF_8)); + output.put(ledgerId.toString(), md); manager.close(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index 3d948c33d10..ee6b278a0a3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -45,7 +45,6 @@ import org.apache.bookkeeper.http.service.HttpServiceResponse; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; -import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.replication.AuditorElector; @@ -83,6 +82,7 @@ public TestHttpService() { public void setUp() throws Exception { super.setUp(); baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + baseClientConf.setStoreSystemtimeAsLedgerCreationTime(true); this.bkHttpServiceProvider = new BKHttpServiceProvider.Builder() .setBookieServer(bs.get(numberOfBookies - 1)) .setServerConfiguration(baseConf) @@ -383,11 +383,15 @@ public void testGetLedgerMetaService() throws Exception { HttpServiceResponse response2 = getLedgerMetaService.handle(request2); assertEquals(HttpServer.StatusCode.OK.getValue(), response2.getStatusCode()); @SuppressWarnings("unchecked") - HashMap respBody = JsonUtil.fromJson(response2.getBody(), HashMap.class); + HashMap respBody = JsonUtil.fromJson(response2.getBody(), HashMap.class); assertEquals(1, respBody.size()); + @SuppressWarnings("unchecked") + HashMap expected = JsonUtil.fromJson(JsonUtil.toJson(lh[0].getLedgerMetadata()), HashMap.class); + @SuppressWarnings("unchecked") + HashMap actual = (HashMap) respBody.get(ledgerId.toString()); + // verify LedgerMetadata content is equal - assertTrue(respBody.get(ledgerId.toString()).toString() - .equals(new String(new LedgerMetadataSerDe().serialize(lh[0].getLedgerMetadata())))); + assertTrue(Maps.difference(expected, actual).areEqual()); } @Test From 162cca653c03ee59c46ea1e336a8028deb540677 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 21 Jan 2019 01:02:29 -0800 Subject: [PATCH 0196/1642] [maven-release-plugin] prepare branch branch-4.9 --- cpu-affinity/pom.xml | 5 +---- metadata-drivers/etcd/pom.xml | 4 +--- metadata-drivers/pom.xml | 4 +--- pom.xml | 2 +- stats/pom.xml | 3 +-- stats/utils/pom.xml | 3 +-- tests/docker-images/current-version-image/pom.xml | 8 +++----- tools/all/pom.xml | 6 ++---- 8 files changed, 11 insertions(+), 24 deletions(-) diff --git a/cpu-affinity/pom.xml b/cpu-affinity/pom.xml index 1439259f66c..4dbe39b23da 100644 --- a/cpu-affinity/pom.xml +++ b/cpu-affinity/pom.xml @@ -14,10 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/metadata-drivers/etcd/pom.xml b/metadata-drivers/etcd/pom.xml index abc9f6e5d92..706bcfa5bde 100644 --- a/metadata-drivers/etcd/pom.xml +++ b/metadata-drivers/etcd/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + org.apache.bookkeeper.metadata.drivers metadata-drivers-parent diff --git a/metadata-drivers/pom.xml b/metadata-drivers/pom.xml index 9cc47ca894d..ab3ebd98550 100644 --- a/metadata-drivers/pom.xml +++ b/metadata-drivers/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + bookkeeper org.apache.bookkeeper diff --git a/pom.xml b/pom.xml index cf355666d00..7f0bc00074a 100644 --- a/pom.xml +++ b/pom.xml @@ -40,7 +40,7 @@ scm:git:https://github.com/apache/bookkeeper.git scm:git:https://github.com/apache/bookkeeper.git https://github.com/apache/bookkeeper - HEAD + branch-4.9 JIRA diff --git a/stats/pom.xml b/stats/pom.xml index c699ed4e17c..dac6a8a06f7 100644 --- a/stats/pom.xml +++ b/stats/pom.xml @@ -17,8 +17,7 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/stats/utils/pom.xml b/stats/utils/pom.xml index 9be19cfcbb4..8861fb066b1 100644 --- a/stats/utils/pom.xml +++ b/stats/utils/pom.xml @@ -17,8 +17,7 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + 4.0.0 bookkeeper-stats-parent diff --git a/tests/docker-images/current-version-image/pom.xml b/tests/docker-images/current-version-image/pom.xml index edffc648143..25edb2e7350 100644 --- a/tests/docker-images/current-version-image/pom.xml +++ b/tests/docker-images/current-version-image/pom.xml @@ -79,13 +79,11 @@ copy python wheel file - - + + copying docker scripts - + diff --git a/tools/all/pom.xml b/tools/all/pom.xml index 9a29f6099f8..a7c3c01485d 100644 --- a/tools/all/pom.xml +++ b/tools/all/pom.xml @@ -48,8 +48,7 @@ - + @@ -91,8 +90,7 @@ - + From 34c917fd6520982aae9b0bff0433d6c8ca36e191 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 21 Jan 2019 01:02:39 -0800 Subject: [PATCH 0197/1642] [maven-release-plugin] prepare for next development iteration --- bookkeeper-benchmark/pom.xml | 2 +- bookkeeper-common-allocator/pom.xml | 2 +- bookkeeper-common/pom.xml | 2 +- bookkeeper-dist/all/pom.xml | 2 +- bookkeeper-dist/bkctl/pom.xml | 2 +- bookkeeper-dist/pom.xml | 2 +- bookkeeper-dist/server/pom.xml | 2 +- bookkeeper-http/http-server/pom.xml | 2 +- bookkeeper-http/pom.xml | 2 +- bookkeeper-http/twitter-http-server/pom.xml | 2 +- bookkeeper-http/vertx-http-server/pom.xml | 2 +- bookkeeper-proto/pom.xml | 2 +- bookkeeper-server/pom.xml | 2 +- bookkeeper-stats-providers/codahale-metrics-provider/pom.xml | 2 +- bookkeeper-stats-providers/pom.xml | 2 +- .../prometheus-metrics-provider/pom.xml | 2 +- bookkeeper-stats-providers/twitter-finagle-provider/pom.xml | 2 +- bookkeeper-stats-providers/twitter-ostrich-provider/pom.xml | 2 +- bookkeeper-stats-providers/twitter-science-provider/pom.xml | 2 +- bookkeeper-stats/pom.xml | 2 +- buildtools/pom.xml | 4 ++-- circe-checksum/pom.xml | 2 +- cpu-affinity/pom.xml | 2 +- metadata-drivers/etcd/pom.xml | 2 +- metadata-drivers/pom.xml | 2 +- microbenchmarks/pom.xml | 2 +- pom.xml | 4 ++-- shaded/bookkeeper-server-shaded/pom.xml | 2 +- shaded/bookkeeper-server-tests-shaded/pom.xml | 2 +- shaded/distributedlog-core-shaded/pom.xml | 2 +- shaded/pom.xml | 2 +- stats/pom.xml | 2 +- stats/utils/pom.xml | 2 +- stream/distributedlog/common/pom.xml | 2 +- stream/distributedlog/core/pom.xml | 2 +- stream/distributedlog/io/dlfs/pom.xml | 2 +- stream/distributedlog/io/pom.xml | 2 +- stream/distributedlog/pom.xml | 2 +- stream/distributedlog/protocol/pom.xml | 2 +- tests/backward-compat/current-server-old-clients/pom.xml | 2 +- tests/backward-compat/hierarchical-ledger-manager/pom.xml | 2 +- tests/backward-compat/hostname-bookieid/pom.xml | 2 +- tests/backward-compat/old-cookie-new-cluster/pom.xml | 2 +- tests/backward-compat/pom.xml | 2 +- tests/backward-compat/recovery-no-password/pom.xml | 2 +- tests/backward-compat/upgrade-direct/pom.xml | 2 +- tests/backward-compat/upgrade/pom.xml | 2 +- tests/backward-compat/yahoo-custom-version/pom.xml | 2 +- tests/docker-images/all-released-versions-image/pom.xml | 2 +- tests/docker-images/all-versions-image/pom.xml | 2 +- tests/docker-images/current-version-image/pom.xml | 2 +- tests/docker-images/pom.xml | 2 +- tests/integration-tests-base-groovy/pom.xml | 2 +- tests/integration-tests-base/pom.xml | 2 +- tests/integration-tests-topologies/pom.xml | 2 +- tests/integration-tests-utils/pom.xml | 2 +- tests/integration/pom.xml | 2 +- tests/integration/smoke/pom.xml | 2 +- tests/integration/standalone/pom.xml | 2 +- tests/pom.xml | 2 +- tests/scripts/pom.xml | 2 +- tests/shaded/bookkeeper-server-shaded-test/pom.xml | 2 +- tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml | 2 +- tests/shaded/distributedlog-core-shaded-test/pom.xml | 2 +- tests/shaded/pom.xml | 2 +- tools/all/pom.xml | 2 +- tools/framework/pom.xml | 2 +- tools/ledger/pom.xml | 2 +- tools/pom.xml | 2 +- 69 files changed, 71 insertions(+), 71 deletions(-) diff --git a/bookkeeper-benchmark/pom.xml b/bookkeeper-benchmark/pom.xml index 9b86140719b..aa147db71ad 100644 --- a/bookkeeper-benchmark/pom.xml +++ b/bookkeeper-benchmark/pom.xml @@ -21,7 +21,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper bookkeeper-benchmark diff --git a/bookkeeper-common-allocator/pom.xml b/bookkeeper-common-allocator/pom.xml index a98889923ee..f56e8768dad 100644 --- a/bookkeeper-common-allocator/pom.xml +++ b/bookkeeper-common-allocator/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper-common-allocator Apache BookKeeper :: Common :: Allocator diff --git a/bookkeeper-common/pom.xml b/bookkeeper-common/pom.xml index d2825a0988e..506271c952a 100644 --- a/bookkeeper-common/pom.xml +++ b/bookkeeper-common/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper-common Apache BookKeeper :: Common diff --git a/bookkeeper-dist/all/pom.xml b/bookkeeper-dist/all/pom.xml index 79e9808596c..1e1dd9569b2 100644 --- a/bookkeeper-dist/all/pom.xml +++ b/bookkeeper-dist/all/pom.xml @@ -23,7 +23,7 @@ bookkeeper-dist org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. diff --git a/bookkeeper-dist/bkctl/pom.xml b/bookkeeper-dist/bkctl/pom.xml index e581865d71b..88764f14919 100644 --- a/bookkeeper-dist/bkctl/pom.xml +++ b/bookkeeper-dist/bkctl/pom.xml @@ -23,7 +23,7 @@ bookkeeper-dist org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. diff --git a/bookkeeper-dist/pom.xml b/bookkeeper-dist/pom.xml index 6bc7b1e8967..7cce6b7bd82 100644 --- a/bookkeeper-dist/pom.xml +++ b/bookkeeper-dist/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT 4.0.0 bookkeeper-dist diff --git a/bookkeeper-dist/server/pom.xml b/bookkeeper-dist/server/pom.xml index 186a652c1b5..ce95d7b0cf6 100644 --- a/bookkeeper-dist/server/pom.xml +++ b/bookkeeper-dist/server/pom.xml @@ -23,7 +23,7 @@ bookkeeper-dist org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. diff --git a/bookkeeper-http/http-server/pom.xml b/bookkeeper-http/http-server/pom.xml index ec99dfd954c..d708669a8de 100644 --- a/bookkeeper-http/http-server/pom.xml +++ b/bookkeeper-http/http-server/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. 4.0.0 diff --git a/bookkeeper-http/pom.xml b/bookkeeper-http/pom.xml index 0daa352ef0d..0c7c2afcc0e 100644 --- a/bookkeeper-http/pom.xml +++ b/bookkeeper-http/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.http diff --git a/bookkeeper-http/twitter-http-server/pom.xml b/bookkeeper-http/twitter-http-server/pom.xml index 0a214061aad..f60d740ae2b 100644 --- a/bookkeeper-http/twitter-http-server/pom.xml +++ b/bookkeeper-http/twitter-http-server/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. 4.0.0 diff --git a/bookkeeper-http/vertx-http-server/pom.xml b/bookkeeper-http/vertx-http-server/pom.xml index 0b3c9c23ae8..121ed6c1526 100644 --- a/bookkeeper-http/vertx-http-server/pom.xml +++ b/bookkeeper-http/vertx-http-server/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. 4.0.0 diff --git a/bookkeeper-proto/pom.xml b/bookkeeper-proto/pom.xml index f9520d8ba1b..f57fea8eceb 100644 --- a/bookkeeper-proto/pom.xml +++ b/bookkeeper-proto/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper-proto Apache BookKeeper :: Protocols diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index c1b684eea60..a2d853865b9 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper-server Apache BookKeeper :: Server diff --git a/bookkeeper-stats-providers/codahale-metrics-provider/pom.xml b/bookkeeper-stats-providers/codahale-metrics-provider/pom.xml index cab53fefad0..1f213635a29 100644 --- a/bookkeeper-stats-providers/codahale-metrics-provider/pom.xml +++ b/bookkeeper-stats-providers/codahale-metrics-provider/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. org.apache.bookkeeper.stats diff --git a/bookkeeper-stats-providers/pom.xml b/bookkeeper-stats-providers/pom.xml index e64f32cfb87..b36d36961b0 100644 --- a/bookkeeper-stats-providers/pom.xml +++ b/bookkeeper-stats-providers/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT 4.0.0 bookkeeper-stats-providers diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml b/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml index 9f212e52408..bbe182af9ac 100644 --- a/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml +++ b/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. org.apache.bookkeeper.stats diff --git a/bookkeeper-stats-providers/twitter-finagle-provider/pom.xml b/bookkeeper-stats-providers/twitter-finagle-provider/pom.xml index 90dcb818faa..1c2e4a3df22 100644 --- a/bookkeeper-stats-providers/twitter-finagle-provider/pom.xml +++ b/bookkeeper-stats-providers/twitter-finagle-provider/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. org.apache.bookkeeper.stats diff --git a/bookkeeper-stats-providers/twitter-ostrich-provider/pom.xml b/bookkeeper-stats-providers/twitter-ostrich-provider/pom.xml index f1662e28223..694ffc4b3ca 100644 --- a/bookkeeper-stats-providers/twitter-ostrich-provider/pom.xml +++ b/bookkeeper-stats-providers/twitter-ostrich-provider/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. org.apache.bookkeeper.stats diff --git a/bookkeeper-stats-providers/twitter-science-provider/pom.xml b/bookkeeper-stats-providers/twitter-science-provider/pom.xml index 278a5b5262c..e766fe06bd7 100644 --- a/bookkeeper-stats-providers/twitter-science-provider/pom.xml +++ b/bookkeeper-stats-providers/twitter-science-provider/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. org.apache.bookkeeper.stats diff --git a/bookkeeper-stats/pom.xml b/bookkeeper-stats/pom.xml index 0d4a9b20e66..561ecdc1e20 100644 --- a/bookkeeper-stats/pom.xml +++ b/bookkeeper-stats/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.stats bookkeeper-stats-api diff --git a/buildtools/pom.xml b/buildtools/pom.xml index ea4665b63bd..b82dd655267 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -20,9 +20,9 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT buildtools Apache BookKeeper :: Build Tools - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT diff --git a/circe-checksum/pom.xml b/circe-checksum/pom.xml index cb2ebdb8eff..0e6e8f4a1fd 100644 --- a/circe-checksum/pom.xml +++ b/circe-checksum/pom.xml @@ -24,7 +24,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. diff --git a/cpu-affinity/pom.xml b/cpu-affinity/pom.xml index 4dbe39b23da..8431355365e 100644 --- a/cpu-affinity/pom.xml +++ b/cpu-affinity/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. diff --git a/metadata-drivers/etcd/pom.xml b/metadata-drivers/etcd/pom.xml index 706bcfa5bde..9d92db76e8f 100644 --- a/metadata-drivers/etcd/pom.xml +++ b/metadata-drivers/etcd/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper.metadata.drivers metadata-drivers-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. 4.0.0 diff --git a/metadata-drivers/pom.xml b/metadata-drivers/pom.xml index ab3ebd98550..5a2753440ba 100644 --- a/metadata-drivers/pom.xml +++ b/metadata-drivers/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.metadata.drivers diff --git a/microbenchmarks/pom.xml b/microbenchmarks/pom.xml index 0f34b319d16..8ec203cbec2 100644 --- a/microbenchmarks/pom.xml +++ b/microbenchmarks/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT microbenchmarks Apache BookKeeper :: microbenchmarks diff --git a/pom.xml b/pom.xml index 7f0bc00074a..935f0f44877 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ 4.0.0 org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper pom Apache BookKeeper :: Parent @@ -40,7 +40,7 @@ scm:git:https://github.com/apache/bookkeeper.git scm:git:https://github.com/apache/bookkeeper.git https://github.com/apache/bookkeeper - branch-4.9 + HEAD JIRA diff --git a/shaded/bookkeeper-server-shaded/pom.xml b/shaded/bookkeeper-server-shaded/pom.xml index 85c76beba46..df158a42ba4 100644 --- a/shaded/bookkeeper-server-shaded/pom.xml +++ b/shaded/bookkeeper-server-shaded/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper shaded-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. bookkeeper-server-shaded diff --git a/shaded/bookkeeper-server-tests-shaded/pom.xml b/shaded/bookkeeper-server-tests-shaded/pom.xml index 3689415d8bb..45c21ee06f5 100644 --- a/shaded/bookkeeper-server-tests-shaded/pom.xml +++ b/shaded/bookkeeper-server-tests-shaded/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper shaded-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. bookkeeper-server-tests-shaded diff --git a/shaded/distributedlog-core-shaded/pom.xml b/shaded/distributedlog-core-shaded/pom.xml index 85ad8f0b5cd..c7206b91dc9 100644 --- a/shaded/distributedlog-core-shaded/pom.xml +++ b/shaded/distributedlog-core-shaded/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper shaded-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.distributedlog diff --git a/shaded/pom.xml b/shaded/pom.xml index fca99f2afb7..56d1af1c50f 100644 --- a/shaded/pom.xml +++ b/shaded/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper shaded-parent diff --git a/stats/pom.xml b/stats/pom.xml index dac6a8a06f7..0d81cf7ab96 100644 --- a/stats/pom.xml +++ b/stats/pom.xml @@ -22,7 +22,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. pom diff --git a/stats/utils/pom.xml b/stats/utils/pom.xml index 8861fb066b1..15e87c3706b 100644 --- a/stats/utils/pom.xml +++ b/stats/utils/pom.xml @@ -22,7 +22,7 @@ bookkeeper-stats-parent org.apache.bookkeeper.stats - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.bookkeeper.stats diff --git a/stream/distributedlog/common/pom.xml b/stream/distributedlog/common/pom.xml index a8d19db6f48..ea71a8f67fd 100644 --- a/stream/distributedlog/common/pom.xml +++ b/stream/distributedlog/common/pom.xml @@ -20,7 +20,7 @@ org.apache.distributedlog distributedlog - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT distributedlog-common Apache BookKeeper :: DistributedLog :: Common diff --git a/stream/distributedlog/core/pom.xml b/stream/distributedlog/core/pom.xml index 582197585f3..6dced1fe0e7 100644 --- a/stream/distributedlog/core/pom.xml +++ b/stream/distributedlog/core/pom.xml @@ -20,7 +20,7 @@ org.apache.distributedlog distributedlog - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT distributedlog-core Apache BookKeeper :: DistributedLog :: Core Library diff --git a/stream/distributedlog/io/dlfs/pom.xml b/stream/distributedlog/io/dlfs/pom.xml index bb08a4655ad..6188d5172f6 100644 --- a/stream/distributedlog/io/dlfs/pom.xml +++ b/stream/distributedlog/io/dlfs/pom.xml @@ -20,7 +20,7 @@ distributedlog org.apache.distributedlog - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. org.apache.distributedlog diff --git a/stream/distributedlog/io/pom.xml b/stream/distributedlog/io/pom.xml index 7d18e3dc4bf..8fcf5198fa1 100644 --- a/stream/distributedlog/io/pom.xml +++ b/stream/distributedlog/io/pom.xml @@ -19,7 +19,7 @@ org.apache.distributedlog distributedlog - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT 4.0.0 distributedlog-io diff --git a/stream/distributedlog/pom.xml b/stream/distributedlog/pom.xml index 8cf8fcf6d0c..b3d421c2d89 100644 --- a/stream/distributedlog/pom.xml +++ b/stream/distributedlog/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../.. org.apache.distributedlog diff --git a/stream/distributedlog/protocol/pom.xml b/stream/distributedlog/protocol/pom.xml index 9c49081aa2b..1df5ae4e187 100644 --- a/stream/distributedlog/protocol/pom.xml +++ b/stream/distributedlog/protocol/pom.xml @@ -20,7 +20,7 @@ org.apache.distributedlog distributedlog - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT distributedlog-protocol Apache BookKeeper :: DistributedLog :: Protocol diff --git a/tests/backward-compat/current-server-old-clients/pom.xml b/tests/backward-compat/current-server-old-clients/pom.xml index ee708c42ade..19751252344 100644 --- a/tests/backward-compat/current-server-old-clients/pom.xml +++ b/tests/backward-compat/current-server-old-clients/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.backward-compat tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/hierarchical-ledger-manager/pom.xml b/tests/backward-compat/hierarchical-ledger-manager/pom.xml index 393804d4abe..9b68d41c12a 100644 --- a/tests/backward-compat/hierarchical-ledger-manager/pom.xml +++ b/tests/backward-compat/hierarchical-ledger-manager/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.backward-compat tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/hostname-bookieid/pom.xml b/tests/backward-compat/hostname-bookieid/pom.xml index a475c1f2ac6..f286ce16598 100644 --- a/tests/backward-compat/hostname-bookieid/pom.xml +++ b/tests/backward-compat/hostname-bookieid/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.backward-compat tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/old-cookie-new-cluster/pom.xml b/tests/backward-compat/old-cookie-new-cluster/pom.xml index 58ae6d099e8..9fdeacedb0e 100644 --- a/tests/backward-compat/old-cookie-new-cluster/pom.xml +++ b/tests/backward-compat/old-cookie-new-cluster/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.backward-compat tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/pom.xml b/tests/backward-compat/pom.xml index d5d7d6b875e..1e6f1fd13eb 100644 --- a/tests/backward-compat/pom.xml +++ b/tests/backward-compat/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests integration-tests-base-groovy - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../integration-tests-base-groovy org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/recovery-no-password/pom.xml b/tests/backward-compat/recovery-no-password/pom.xml index 6c7575ffa47..5472afcf68d 100644 --- a/tests/backward-compat/recovery-no-password/pom.xml +++ b/tests/backward-compat/recovery-no-password/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.backward-compat tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/upgrade-direct/pom.xml b/tests/backward-compat/upgrade-direct/pom.xml index 2a41b91b2bb..240c66cd7a7 100644 --- a/tests/backward-compat/upgrade-direct/pom.xml +++ b/tests/backward-compat/upgrade-direct/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.backward-compat tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/upgrade/pom.xml b/tests/backward-compat/upgrade/pom.xml index c1f7e9cdfe6..3a86f8122d9 100644 --- a/tests/backward-compat/upgrade/pom.xml +++ b/tests/backward-compat/upgrade/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.backward-compat tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/yahoo-custom-version/pom.xml b/tests/backward-compat/yahoo-custom-version/pom.xml index 1fe5c8f2a22..f8aca74d57f 100644 --- a/tests/backward-compat/yahoo-custom-version/pom.xml +++ b/tests/backward-compat/yahoo-custom-version/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.backward-compat tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.backward-compat diff --git a/tests/docker-images/all-released-versions-image/pom.xml b/tests/docker-images/all-released-versions-image/pom.xml index 9d1f9293c94..2d516a77b85 100644 --- a/tests/docker-images/all-released-versions-image/pom.xml +++ b/tests/docker-images/all-released-versions-image/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper.tests docker-images - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.tests diff --git a/tests/docker-images/all-versions-image/pom.xml b/tests/docker-images/all-versions-image/pom.xml index cf8e380e9e4..6f162bc0fbe 100644 --- a/tests/docker-images/all-versions-image/pom.xml +++ b/tests/docker-images/all-versions-image/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper.tests docker-images - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.tests diff --git a/tests/docker-images/current-version-image/pom.xml b/tests/docker-images/current-version-image/pom.xml index 25edb2e7350..77a7ff79785 100644 --- a/tests/docker-images/current-version-image/pom.xml +++ b/tests/docker-images/current-version-image/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper.tests docker-images - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.tests diff --git a/tests/docker-images/pom.xml b/tests/docker-images/pom.xml index c44e57109b5..d5cf4482d65 100644 --- a/tests/docker-images/pom.xml +++ b/tests/docker-images/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests docker-images diff --git a/tests/integration-tests-base-groovy/pom.xml b/tests/integration-tests-base-groovy/pom.xml index 14eadf3e1f4..ac0167b5845 100644 --- a/tests/integration-tests-base-groovy/pom.xml +++ b/tests/integration-tests-base-groovy/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests integration-tests-base - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT ../integration-tests-base diff --git a/tests/integration-tests-base/pom.xml b/tests/integration-tests-base/pom.xml index 13532433241..b178f19de9a 100644 --- a/tests/integration-tests-base/pom.xml +++ b/tests/integration-tests-base/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests diff --git a/tests/integration-tests-topologies/pom.xml b/tests/integration-tests-topologies/pom.xml index 766effba3fb..5b239398ac1 100644 --- a/tests/integration-tests-topologies/pom.xml +++ b/tests/integration-tests-topologies/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests diff --git a/tests/integration-tests-utils/pom.xml b/tests/integration-tests-utils/pom.xml index 07c68fc186e..1324c0a10ea 100644 --- a/tests/integration-tests-utils/pom.xml +++ b/tests/integration-tests-utils/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index 5ae2e34dfb3..8b0073b9787 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.bookkeeper.tests.integration diff --git a/tests/integration/smoke/pom.xml b/tests/integration/smoke/pom.xml index 96b2b80b80f..61e77122839 100644 --- a/tests/integration/smoke/pom.xml +++ b/tests/integration/smoke/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.integration tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.integration diff --git a/tests/integration/standalone/pom.xml b/tests/integration/standalone/pom.xml index 51aeff23850..88f0a521fb8 100644 --- a/tests/integration/standalone/pom.xml +++ b/tests/integration/standalone/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests.integration tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.integration diff --git a/tests/pom.xml b/tests/pom.xml index 16086809ab6..76d1eed5de3 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests tests-parent diff --git a/tests/scripts/pom.xml b/tests/scripts/pom.xml index 0c514b0e496..538a56709cf 100644 --- a/tests/scripts/pom.xml +++ b/tests/scripts/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests diff --git a/tests/shaded/bookkeeper-server-shaded-test/pom.xml b/tests/shaded/bookkeeper-server-shaded-test/pom.xml index d37a4674118..959dc9eb8bb 100644 --- a/tests/shaded/bookkeeper-server-shaded-test/pom.xml +++ b/tests/shaded/bookkeeper-server-shaded-test/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.shaded shaded-tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. bookkeeper-server-shaded-test diff --git a/tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml b/tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml index 1498edf7dc6..eb8bac68da9 100644 --- a/tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml +++ b/tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.shaded shaded-tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. bookkeeper-server-tests-shaded-test diff --git a/tests/shaded/distributedlog-core-shaded-test/pom.xml b/tests/shaded/distributedlog-core-shaded-test/pom.xml index 56d2e46e95b..ef781d82710 100644 --- a/tests/shaded/distributedlog-core-shaded-test/pom.xml +++ b/tests/shaded/distributedlog-core-shaded-test/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.shaded shaded-tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. distributedlog-core-shaded-test diff --git a/tests/shaded/pom.xml b/tests/shaded/pom.xml index 9f19eb48e42..6a2d7f43eba 100644 --- a/tests/shaded/pom.xml +++ b/tests/shaded/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.bookkeeper.tests.shaded diff --git a/tools/all/pom.xml b/tools/all/pom.xml index a7c3c01485d..5eabacd5ee4 100644 --- a/tools/all/pom.xml +++ b/tools/all/pom.xml @@ -22,7 +22,7 @@ bookkeeper-tools-parent org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper-tools Apache BookKeeper :: Tools diff --git a/tools/framework/pom.xml b/tools/framework/pom.xml index e27dfd28db1..cf3c6f53c0b 100644 --- a/tools/framework/pom.xml +++ b/tools/framework/pom.xml @@ -20,7 +20,7 @@ bookkeeper-tools-parent org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper-tools-framework Apache BookKeeper :: Tools :: Framework diff --git a/tools/ledger/pom.xml b/tools/ledger/pom.xml index 6c5182b211e..24dcb38ebaf 100644 --- a/tools/ledger/pom.xml +++ b/tools/ledger/pom.xml @@ -20,7 +20,7 @@ bookkeeper-tools-parent org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper-tools-ledger Apache BookKeeper :: Tools :: Ledger diff --git a/tools/pom.xml b/tools/pom.xml index c538a6a9e0f..1ace10a6498 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper-tools-parent Apache BookKeeper :: Tools :: Parent From 5f5e0ecf3220905c9db1390025bb7e46f6fecd2d Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 21 Jan 2019 18:31:37 -0800 Subject: [PATCH 0198/1642] [BUILD] Fix master build after release 4.9.0 Descriptions of the changes in this PR: *Motivation* when creating branch-4.9, maven release plugin doesn't bump the version for stream modules. *Modifications* run `mvn versions:set` to manually fix the version for all modules Reviewers: Jia Zhai This closes #1909 from sijie/fix_master_build --- stream/api/pom.xml | 2 +- stream/bk-grpc-name-resolver/pom.xml | 2 +- stream/clients/java/all/pom.xml | 2 +- stream/clients/java/base/pom.xml | 2 +- stream/clients/java/kv/pom.xml | 2 +- stream/clients/java/pom.xml | 2 +- stream/clients/pom.xml | 2 +- stream/common/pom.xml | 2 +- stream/pom.xml | 2 +- stream/proto/pom.xml | 2 +- stream/server/pom.xml | 2 +- stream/statelib/pom.xml | 2 +- stream/storage/api/pom.xml | 2 +- stream/storage/impl/pom.xml | 2 +- stream/storage/pom.xml | 2 +- stream/tests-common/pom.xml | 2 +- tests/integration/cluster/pom.xml | 2 +- tools/perf/pom.xml | 2 +- tools/stream/pom.xml | 2 +- 19 files changed, 19 insertions(+), 19 deletions(-) diff --git a/stream/api/pom.xml b/stream/api/pom.xml index 705702f433b..c095a6919a7 100644 --- a/stream/api/pom.xml +++ b/stream/api/pom.xml @@ -21,7 +21,7 @@ stream-storage-parent org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/bk-grpc-name-resolver/pom.xml b/stream/bk-grpc-name-resolver/pom.xml index 312adfbf1dd..0c4e530a20a 100644 --- a/stream/bk-grpc-name-resolver/pom.xml +++ b/stream/bk-grpc-name-resolver/pom.xml @@ -23,7 +23,7 @@ stream-storage-parent org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/clients/java/all/pom.xml b/stream/clients/java/all/pom.xml index 503557e08f0..14c5cddf24e 100644 --- a/stream/clients/java/all/pom.xml +++ b/stream/clients/java/all/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-java-client-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT stream-storage-java-client Apache BookKeeper :: Stream Storage :: Clients :: Java Client diff --git a/stream/clients/java/base/pom.xml b/stream/clients/java/base/pom.xml index b3f70064993..ec36a5f7abd 100644 --- a/stream/clients/java/base/pom.xml +++ b/stream/clients/java/base/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-java-client-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT stream-storage-java-client-base Apache BookKeeper :: Stream Storage :: Clients :: Java Client :: Base diff --git a/stream/clients/java/kv/pom.xml b/stream/clients/java/kv/pom.xml index b2f312fb36a..23a203584d2 100644 --- a/stream/clients/java/kv/pom.xml +++ b/stream/clients/java/kv/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-java-client-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT stream-storage-java-kv-client Apache BookKeeper :: Stream Storage :: Clients :: Java Client :: KV diff --git a/stream/clients/java/pom.xml b/stream/clients/java/pom.xml index f167c422720..176960cfdba 100644 --- a/stream/clients/java/pom.xml +++ b/stream/clients/java/pom.xml @@ -22,7 +22,7 @@ org.apache.bookkeeper stream-storage-clients-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. stream-storage-java-client-parent diff --git a/stream/clients/pom.xml b/stream/clients/pom.xml index 0c049546fe9..263cda38c34 100644 --- a/stream/clients/pom.xml +++ b/stream/clients/pom.xml @@ -22,7 +22,7 @@ org.apache.bookkeeper stream-storage-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. stream-storage-clients-parent diff --git a/stream/common/pom.xml b/stream/common/pom.xml index c5d0339fe70..2d3ec60c9b2 100644 --- a/stream/common/pom.xml +++ b/stream/common/pom.xml @@ -21,7 +21,7 @@ stream-storage-parent org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/pom.xml b/stream/pom.xml index e8a04916cb3..adcc4ca45bd 100644 --- a/stream/pom.xml +++ b/stream/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. pom diff --git a/stream/proto/pom.xml b/stream/proto/pom.xml index 7c3e86fdfde..747432cecba 100644 --- a/stream/proto/pom.xml +++ b/stream/proto/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/server/pom.xml b/stream/server/pom.xml index 3ec5a5dc08b..fef2555e608 100644 --- a/stream/server/pom.xml +++ b/stream/server/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT stream-storage-server Apache BookKeeper :: Stream Storage :: Server diff --git a/stream/statelib/pom.xml b/stream/statelib/pom.xml index 230ead2f81e..d92de5761dd 100644 --- a/stream/statelib/pom.xml +++ b/stream/statelib/pom.xml @@ -22,7 +22,7 @@ stream-storage-parent org.apache.bookkeeper - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/storage/api/pom.xml b/stream/storage/api/pom.xml index 67df3e1f4d7..d1c16d38b08 100644 --- a/stream/storage/api/pom.xml +++ b/stream/storage/api/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-service-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. stream-storage-service-api diff --git a/stream/storage/impl/pom.xml b/stream/storage/impl/pom.xml index a2ce3e0de99..723f865cbb8 100644 --- a/stream/storage/impl/pom.xml +++ b/stream/storage/impl/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-service-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. stream-storage-service-impl diff --git a/stream/storage/pom.xml b/stream/storage/pom.xml index 3a600cb9bc3..76c524ffa91 100644 --- a/stream/storage/pom.xml +++ b/stream/storage/pom.xml @@ -22,7 +22,7 @@ org.apache.bookkeeper stream-storage-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. stream-storage-service-parent diff --git a/stream/tests-common/pom.xml b/stream/tests-common/pom.xml index 6f760251c55..4fd1340eb63 100644 --- a/stream/tests-common/pom.xml +++ b/stream/tests-common/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT .. org.apache.bookkeeper.tests diff --git a/tests/integration/cluster/pom.xml b/tests/integration/cluster/pom.xml index 2e79a09ec48..dc22a10262e 100644 --- a/tests/integration/cluster/pom.xml +++ b/tests/integration/cluster/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests.integration tests-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT org.apache.bookkeeper.tests.integration diff --git a/tools/perf/pom.xml b/tools/perf/pom.xml index 3142063a2b6..ddbbf37030c 100644 --- a/tools/perf/pom.xml +++ b/tools/perf/pom.xml @@ -18,7 +18,7 @@ org.apache.bookkeeper bookkeeper-tools-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT bookkeeper-perf Apache BookKeeper :: Tools :: Perf diff --git a/tools/stream/pom.xml b/tools/stream/pom.xml index 540e941234c..5a9d3face1e 100644 --- a/tools/stream/pom.xml +++ b/tools/stream/pom.xml @@ -22,7 +22,7 @@ org.apache.bookkeeper bookkeeper-tools-parent - 4.9.0-SNAPSHOT + 4.10.0-SNAPSHOT stream-storage-cli Apache BookKeeper :: Tools :: Stream From 0cd5c9dd85022ae8394111bd710976ec41f02ba6 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 21 Jan 2019 18:39:11 -0800 Subject: [PATCH 0199/1642] [RELEASE] Release table service python client Descriptions of the changes in this PR: *Motivation* Update the table service python client version to release version *Modifications* Bump the version to 4.9.0 Reviewers: Enrico Olivelli This closes #1906 from sijie/release_python_client --- stream/clients/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index 1dfbbe47c3f..9dd171f04f8 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -19,7 +19,7 @@ name = 'apache-bookkeeper-client' description = 'Apache BookKeeper client library' -version = '4.9.0-alpha-2' +version = '4.9.0' # Should be one of: # 'Development Status :: 3 - Alpha' # 'Development Status :: 4 - Beta' From b3a323083e26e527610540e052b4d34446efceb0 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 21 Jan 2019 18:47:58 -0800 Subject: [PATCH 0200/1642] [RELEASE] add bkctl package to release scripts Descriptions of the changes in this PR: *Motivation* starting from bookkeeper 4.9.0, a separate tool package `bkctl` is released. *Modifications* add bkctl into release scripts Reviewers: Enrico Olivelli This closes #1907 from sijie/include_bkctl_package --- dev/release/004-stage-packages.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dev/release/004-stage-packages.sh b/dev/release/004-stage-packages.sh index 23941342f9a..e2e5bbaf7c5 100755 --- a/dev/release/004-stage-packages.sh +++ b/dev/release/004-stage-packages.sh @@ -50,6 +50,8 @@ cp ${SRC_DIR}/bookkeeper-dist/server/target/bookkeeper-server-${VERSION}-bin.tar cp ${SRC_DIR}/bookkeeper-dist/server/target/bookkeeper-server-${VERSION}-bin.tar.gz.asc ${DEST_DIR}/bookkeeper-server-${VERSION}-bin.tar.gz.asc cp ${SRC_DIR}/bookkeeper-dist/all/target/bookkeeper-all-${VERSION}-bin.tar.gz ${DEST_DIR}/bookkeeper-all-${VERSION}-bin.tar.gz cp ${SRC_DIR}/bookkeeper-dist/all/target/bookkeeper-all-${VERSION}-bin.tar.gz.asc ${DEST_DIR}/bookkeeper-all-${VERSION}-bin.tar.gz.asc +cp ${SRC_DIR}/bookkeeper-dist/bkctl/target/bkctl-${VERSION}-bin.tar.gz ${DEST_DIR}/bkctl-${VERSION}-bin.tar.gz +cp ${SRC_DIR}/bookkeeper-dist/bkctl/target/bkctl-${VERSION}-bin.tar.gz.asc ${DEST_DIR}/bkctl-${VERSION}-bin.tar.gz.asc echo "Copied packages." echo "Generating sha512 files ..." @@ -57,6 +59,7 @@ cd ${DEST_DIR} shasum -a 512 bookkeeper-${VERSION}-src.tar.gz > bookkeeper-${VERSION}-src.tar.gz.sha512 shasum -a 512 bookkeeper-server-${VERSION}-bin.tar.gz > bookkeeper-server-${VERSION}-bin.tar.gz.sha512 shasum -a 512 bookkeeper-all-${VERSION}-bin.tar.gz > bookkeeper-all-${VERSION}-bin.tar.gz.sha512 +shasum -a 512 bkctl-${VERSION}-bin.tar.gz > bkctl-${VERSION}-bin.tar.gz.sha512 echo "Generated sha512 files." cd ${DIST_DEV_DIR}/bookkeeper From 60cf102ff283cfdf6c22f168dbb75532c55e1edb Mon Sep 17 00:00:00 2001 From: eolivelli Date: Tue, 22 Jan 2019 22:01:09 +0100 Subject: [PATCH 0201/1642] fix/pair --- .../bookkeeper/common/collections/Pair.java | 69 +++++++++++++++++++ .../LocalBookieEnsemblePlacementPolicy.java | 2 +- .../bookkeeper/client/BookKeeperAdmin.java | 2 +- .../bookkeeper/client/BookieWatcherImpl.java | 2 +- .../DefaultEnsemblePlacementPolicy.java | 2 +- .../client/EnsemblePlacementPolicy.java | 2 +- ...ITopologyAwareEnsemblePlacementPolicy.java | 2 +- .../RackawareEnsemblePlacementPolicy.java | 2 +- .../RackawareEnsemblePlacementPolicyImpl.java | 2 +- .../RegionAwareEnsemblePlacementPolicy.java | 2 +- .../GenericEnsemblePlacementPolicyTest.java | 2 +- .../TestRackawareEnsemblePlacementPolicy.java | 2 +- ...estRackawarePolicyNotificationUpdates.java | 2 +- ...estRegionAwareEnsemblePlacementPolicy.java | 2 +- 14 files changed, 82 insertions(+), 13 deletions(-) create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/Pair.java diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/Pair.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/Pair.java new file mode 100644 index 00000000000..020c0dc9611 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/Pair.java @@ -0,0 +1,69 @@ +/* + * 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.common.collections; + +import java.io.Serializable; +import lombok.EqualsAndHashCode; + +/** + * An immutable pair of objects. + */ +@EqualsAndHashCode +public final class Pair implements Serializable { + + private static final long serialVersionUID = 2343L; + + private final K left; + private final V right; + + /** + * Creates an immutable pair. + * + * @param left the left object + * @param right the right object + * @return a new Pair of objects + */ + public static Pair of(K left, V right) { + return new Pair<>(left, right); + } + + private Pair(K left, V right) { + this.left = left; + this.right = right; + } + + /** + * Gets the left object. + * + * @return the left object + */ + public K getLeft() { + return left; + } + + /** + * Gets the right object. + * + * @return the right object + */ + public V getRight() { + return right; + } + +} 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 7b7cc46bcfd..8816225f7d1 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 @@ -30,13 +30,13 @@ import org.apache.bookkeeper.client.BookiesHealthInfo; import org.apache.bookkeeper.client.DistributionSchedule; import org.apache.bookkeeper.client.EnsemblePlacementPolicy; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 7b492922f15..9da23d0e5f4 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 @@ -60,6 +60,7 @@ import org.apache.bookkeeper.client.SyncCallbackUtils.SyncOpenCallback; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncReadCallback; import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; @@ -81,7 +82,6 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.IOUtils; -import org.apache.commons.lang3.tuple.Pair; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index 74c1df984b2..50ca8f2cd5e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -41,6 +41,7 @@ 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.collections.Pair; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -50,7 +51,6 @@ import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; -import org.apache.commons.lang3.tuple.Pair; /** * This class is responsible for maintaining a consistent view of what bookies 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 dddbe1cb40d..a9eb2214712 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 @@ -33,13 +33,13 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 00bac8e3c41..d192528befb 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 @@ -30,12 +30,12 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.commons.lang3.tuple.Pair; /** * {@link EnsemblePlacementPolicy} encapsulates the algorithm that bookkeeper client uses to select a number of bookies diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java index 254f5359c20..7acd3b7b4a2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java @@ -26,9 +26,9 @@ import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.Node; -import org.apache.commons.lang3.tuple.Pair; /** * Interface for topology aware ensemble placement policy. 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 8054d97187b..55f1a07981a 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 @@ -23,11 +23,11 @@ import java.util.Set; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.commons.lang3.tuple.Pair; /** * A placement policy implementation use rack information for placing ensembles. 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 6db7de8f727..e571d30781e 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 @@ -53,6 +53,7 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.Configurable; @@ -71,7 +72,6 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 1bd4b75eac4..74cd282095f 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 @@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.FeatureProvider; @@ -39,7 +40,6 @@ import org.apache.bookkeeper.net.NodeBase; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java index 205c5f4cb99..b1eca1fd923 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java @@ -31,9 +31,9 @@ import java.util.Map; import java.util.Set; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.commons.lang3.tuple.Pair; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; 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 e6cd07bcc7b..1038c0b0102 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 @@ -45,6 +45,7 @@ import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.EnsembleForReplacementWithNoConstraints; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.TruePredicate; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; @@ -52,7 +53,6 @@ import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; -import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java index 3192d049c12..7d7b39f4423 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java @@ -36,13 +36,13 @@ import junit.framework.TestCase; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; -import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 8e4f10d5b3b..f8e29b98430 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 @@ -41,6 +41,7 @@ import java.util.concurrent.TimeUnit; import junit.framework.TestCase; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; +import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.feature.SettableFeature; @@ -51,7 +52,6 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.StaticDNSResolver; -import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From a9ad0b826e900d8b794e23c69db7a4cd260b77b7 Mon Sep 17 00:00:00 2001 From: eolivelli Date: Tue, 22 Jan 2019 22:01:59 +0100 Subject: [PATCH 0202/1642] Revert "fix/pair" This reverts commit 615f96701a461160d6d246e5387668b0e3406609. --- .../bookkeeper/common/collections/Pair.java | 69 ------------------- .../LocalBookieEnsemblePlacementPolicy.java | 2 +- .../bookkeeper/client/BookKeeperAdmin.java | 2 +- .../bookkeeper/client/BookieWatcherImpl.java | 2 +- .../DefaultEnsemblePlacementPolicy.java | 2 +- .../client/EnsemblePlacementPolicy.java | 2 +- ...ITopologyAwareEnsemblePlacementPolicy.java | 2 +- .../RackawareEnsemblePlacementPolicy.java | 2 +- .../RackawareEnsemblePlacementPolicyImpl.java | 2 +- .../RegionAwareEnsemblePlacementPolicy.java | 2 +- .../GenericEnsemblePlacementPolicyTest.java | 2 +- .../TestRackawareEnsemblePlacementPolicy.java | 2 +- ...estRackawarePolicyNotificationUpdates.java | 2 +- ...estRegionAwareEnsemblePlacementPolicy.java | 2 +- 14 files changed, 13 insertions(+), 82 deletions(-) delete mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/Pair.java diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/Pair.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/Pair.java deleted file mode 100644 index 020c0dc9611..00000000000 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/collections/Pair.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.common.collections; - -import java.io.Serializable; -import lombok.EqualsAndHashCode; - -/** - * An immutable pair of objects. - */ -@EqualsAndHashCode -public final class Pair implements Serializable { - - private static final long serialVersionUID = 2343L; - - private final K left; - private final V right; - - /** - * Creates an immutable pair. - * - * @param left the left object - * @param right the right object - * @return a new Pair of objects - */ - public static Pair of(K left, V right) { - return new Pair<>(left, right); - } - - private Pair(K left, V right) { - this.left = left; - this.right = right; - } - - /** - * Gets the left object. - * - * @return the left object - */ - public K getLeft() { - return left; - } - - /** - * Gets the right object. - * - * @return the right object - */ - public V getRight() { - return right; - } - -} 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 8816225f7d1..7b7cc46bcfd 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 @@ -30,13 +30,13 @@ import org.apache.bookkeeper.client.BookiesHealthInfo; import org.apache.bookkeeper.client.DistributionSchedule; import org.apache.bookkeeper.client.EnsemblePlacementPolicy; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 9da23d0e5f4..7b492922f15 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 @@ -60,7 +60,6 @@ import org.apache.bookkeeper.client.SyncCallbackUtils.SyncOpenCallback; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncReadCallback; import org.apache.bookkeeper.client.api.LedgerMetadata; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; @@ -82,6 +81,7 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.IOUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index 50ca8f2cd5e..74c1df984b2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -41,7 +41,6 @@ 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.collections.Pair; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -51,6 +50,7 @@ import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; +import org.apache.commons.lang3.tuple.Pair; /** * This class is responsible for maintaining a consistent view of what bookies 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 a9eb2214712..dddbe1cb40d 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 @@ -33,13 +33,13 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 d192528befb..00bac8e3c41 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 @@ -30,12 +30,12 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.commons.lang3.tuple.Pair; /** * {@link EnsemblePlacementPolicy} encapsulates the algorithm that bookkeeper client uses to select a number of bookies diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java index 7acd3b7b4a2..254f5359c20 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java @@ -26,9 +26,9 @@ import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.Node; +import org.apache.commons.lang3.tuple.Pair; /** * Interface for topology aware ensemble placement policy. 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 55f1a07981a..8054d97187b 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 @@ -23,11 +23,11 @@ import java.util.Set; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.commons.lang3.tuple.Pair; /** * A placement policy implementation use rack information for placing ensembles. 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 e571d30781e..6db7de8f727 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 @@ -53,7 +53,6 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.Configurable; @@ -72,6 +71,7 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 74cd282095f..1bd4b75eac4 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 @@ -29,7 +29,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.FeatureProvider; @@ -40,6 +39,7 @@ import org.apache.bookkeeper.net.NodeBase; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java index b1eca1fd923..205c5f4cb99 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java @@ -31,9 +31,9 @@ import java.util.Map; import java.util.Set; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.commons.lang3.tuple.Pair; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; 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 1038c0b0102..e6cd07bcc7b 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 @@ -45,7 +45,6 @@ import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.EnsembleForReplacementWithNoConstraints; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.TruePredicate; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; @@ -53,6 +52,7 @@ import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java index 7d7b39f4423..3192d049c12 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java @@ -36,13 +36,13 @@ import junit.framework.TestCase; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 f8e29b98430..8e4f10d5b3b 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 @@ -41,7 +41,6 @@ import java.util.concurrent.TimeUnit; import junit.framework.TestCase; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; -import org.apache.bookkeeper.common.collections.Pair; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.feature.SettableFeature; @@ -52,6 +51,7 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From e718c772e504e820975e0946fef273a523285889 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 24 Jan 2019 20:07:36 -0800 Subject: [PATCH 0203/1642] [TABLE SERVICE] bump python client version to 4.10.0-SNAPSHOT Descriptions of the changes in this PR: *Modifications* Bump the development of python client to 4.10.0-SNAPSHOT Reviewers: Enrico Olivelli This closes #1911 from sijie/bump_python_client_development --- stream/clients/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index 9dd171f04f8..dbbf4435bac 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -19,7 +19,7 @@ name = 'apache-bookkeeper-client' description = 'Apache BookKeeper client library' -version = '4.9.0' +version = '4.10.0-SNAPSHOT' # Should be one of: # 'Development Status :: 3 - Alpha' # 'Development Status :: 4 - Beta' From 5a07a22d8dd2c4dc1dc5e0076db4b99e8bfffbd3 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 25 Jan 2019 05:09:10 +0100 Subject: [PATCH 0204/1642] Add semantically meaningful return values to placement policy Currently newEnsemble and replaceBookie in EnsemblePlacementPolicy return a apache commons Pair<> with the second argument being a boolean to denote whether the placement conforms strictly to the policy. From calling code, the meaning of this second value is unclear. This patch replaces Pair<> with an PlacementResult object, in which the strict conformity argument is clearly labels. This will also allow extension in the future to return more metadata about particular placements. Also, we shouldn't put third party library classes in interfaces. Issue: #1914 Reviewers: Enrico Olivelli , Charan Reddy Guttapalem , Sijie Guo This closes #1916 from ivankelly/meaningful-placement-res --- .../LocalBookieEnsemblePlacementPolicy.java | 7 +- .../bookkeeper/client/BookKeeperAdmin.java | 7 +- .../bookkeeper/client/BookieWatcherImpl.java | 21 ++- .../DefaultEnsemblePlacementPolicy.java | 16 +- .../client/EnsemblePlacementPolicy.java | 54 ++++-- ...ITopologyAwareEnsemblePlacementPolicy.java | 3 +- .../RackawareEnsemblePlacementPolicy.java | 7 +- .../RackawareEnsemblePlacementPolicyImpl.java | 37 ++-- .../RegionAwareEnsemblePlacementPolicy.java | 14 +- .../GenericEnsemblePlacementPolicyTest.java | 5 +- .../TestRackawareEnsemblePlacementPolicy.java | 160 +++++++++--------- ...areEnsemblePlacementPolicyUsingScript.java | 51 +++--- ...estRackawarePolicyNotificationUpdates.java | 12 +- ...estRegionAwareEnsemblePlacementPolicy.java | 116 +++++-------- 14 files changed, 245 insertions(+), 265 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 7b7cc46bcfd..db88bc70f82 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 @@ -36,7 +36,6 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -81,7 +80,7 @@ public Set onClusterChanged(Set writab } @Override - public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, java.util.Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { @@ -110,14 +109,14 @@ public DistributionSchedule.WriteSet reorderReadLACSequence( } @Override - public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, java.util.Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { if (ensembleSize > 1) { throw new IllegalArgumentException("Local ensemble policy can only return 1 bookie"); } - return Pair.of(Lists.newArrayList(bookieAddress), true); + return PlacementResult.of(Lists.newArrayList(bookieAddress), true); } @Override 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 7b492922f15..1b4f476b24a 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 @@ -81,7 +81,6 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.IOUtils; -import org.apache.commons.lang3.tuple.Pair; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -994,7 +993,7 @@ private Map getReplacementBookiesByIndexes( // allocate bookies for (Integer bookieIndex : bookieIndexesToRereplicate) { BookieSocketAddress oldBookie = ensemble.get(bookieIndex); - Pair replaceBookieResponse = + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = bkc.getPlacementPolicy().replaceBookie( lh.getLedgerMetadata().getEnsembleSize(), lh.getLedgerMetadata().getWriteQuorumSize(), @@ -1003,8 +1002,8 @@ private Map getReplacementBookiesByIndexes( ensemble, oldBookie, bookiesToExclude); - BookieSocketAddress newBookie = replaceBookieResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + BookieSocketAddress newBookie = replaceBookieResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); if (!isEnsembleAdheringToPlacementPolicy) { if (LOG.isDebugEnabled()) { LOG.debug( diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index 74c1df984b2..f4dd9a907ad 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -50,7 +50,6 @@ import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; -import org.apache.commons.lang3.tuple.Pair; /** * This class is responsible for maintaining a consistent view of what bookies @@ -226,15 +225,15 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi int ackQuorumSize, Map customMetadata) throws BKNotEnoughBookiesException { long startTime = MathUtils.nowInNano(); - Pair, Boolean> newEnsembleResponse; + EnsemblePlacementPolicy.PlacementResult> newEnsembleResponse; List socketAddresses; boolean isEnsembleAdheringToPlacementPolicy = false; try { Set quarantinedBookiesSet = quarantinedBookies.asMap().keySet(); newEnsembleResponse = placementPolicy.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, new HashSet(quarantinedBookiesSet)); - socketAddresses = newEnsembleResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.getRight(); + socketAddresses = newEnsembleResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.isStrictlyAdheringToPolicy(); if (!isEnsembleAdheringToPlacementPolicy) { ensembleNotAdheringToPlacementPolicy.inc(); log.warn("New ensemble: {} is not adhering to Placement Policy. quarantinedBookies: {}", @@ -248,8 +247,8 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi } newEnsembleResponse = placementPolicy.newEnsemble( ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, new HashSet<>()); - socketAddresses = newEnsembleResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.getRight(); + socketAddresses = newEnsembleResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.isStrictlyAdheringToPolicy(); if (!isEnsembleAdheringToPlacementPolicy) { ensembleNotAdheringToPlacementPolicy.inc(); log.warn("New ensemble: {} is not adhering to Placement Policy", socketAddresses); @@ -267,7 +266,7 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, throws BKNotEnoughBookiesException { long startTime = MathUtils.nowInNano(); BookieSocketAddress addr = existingBookies.get(bookieIdx); - Pair replaceBookieResponse; + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; BookieSocketAddress socketAddress; boolean isEnsembleAdheringToPlacementPolicy = false; try { @@ -279,8 +278,8 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, replaceBookieResponse = placementPolicy.replaceBookie( ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, existingBookies, addr, excludedBookiesAndQuarantinedBookies); - socketAddress = replaceBookieResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + socketAddress = replaceBookieResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); if (!isEnsembleAdheringToPlacementPolicy) { ensembleNotAdheringToPlacementPolicy.inc(); log.warn( @@ -295,8 +294,8 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, } replaceBookieResponse = placementPolicy.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, existingBookies, addr, excludeBookies); - socketAddress = replaceBookieResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + socketAddress = replaceBookieResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); if (!isEnsembleAdheringToPlacementPolicy) { ensembleNotAdheringToPlacementPolicy.inc(); log.warn( 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 dddbe1cb40d..c130b5d4944 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 @@ -39,7 +39,6 @@ import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,12 +64,12 @@ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy { } @Override - public Pair, Boolean> newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, + public PlacementResult> newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { ArrayList newBookies = new ArrayList(ensembleSize); if (ensembleSize <= 0) { - return Pair.of(newBookies, false); + return PlacementResult.of(newBookies, false); } List allBookies; rwLock.readLock().lock(); @@ -96,7 +95,7 @@ public Pair, Boolean> newEnsemble(int ensembleSize, in newBookies.add(b); --ensembleSize; if (ensembleSize == 0) { - return Pair.of(newBookies, + return PlacementResult.of(newBookies, isEnsembleAdheringToPlacementPolicy(newBookies, quorumSize, ackQuorumSize)); } } @@ -112,7 +111,7 @@ public Pair, Boolean> newEnsemble(int ensembleSize, in newBookies.add(bookie); --ensembleSize; if (ensembleSize == 0) { - return Pair.of(newBookies, + return PlacementResult.of(newBookies, isEnsembleAdheringToPlacementPolicy(newBookies, quorumSize, ackQuorumSize)); } } @@ -121,17 +120,18 @@ public Pair, Boolean> newEnsemble(int ensembleSize, in } @Override - public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { excludeBookies.addAll(currentEnsemble); - List addresses = newEnsemble(1, 1, 1, customMetadata, excludeBookies).getLeft(); + List addresses = newEnsemble(1, 1, 1, customMetadata, excludeBookies).getResult(); BookieSocketAddress candidateAddr = addresses.get(0); List newEnsemble = new ArrayList(currentEnsemble); newEnsemble.set(currentEnsemble.indexOf(bookieToReplace), candidateAddr); - return Pair.of(candidateAddr, isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); + return PlacementResult.of(candidateAddr, + isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); } @Override 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 00bac8e3c41..fad3f9224c1 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 @@ -35,7 +35,6 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.commons.lang3.tuple.Pair; /** * {@link EnsemblePlacementPolicy} encapsulates the algorithm that bookkeeper client uses to select a number of bookies @@ -262,13 +261,13 @@ Set onClusterChanged(Set writableBooki * provides in {@link BookKeeper#createLedger(int, int, int, BookKeeper.DigestType, byte[])} * @param excludeBookies Bookies that should not be considered as targets. * @throws BKNotEnoughBookiesException if not enough bookies available. - * @return the List<org.apache.bookkeeper.net.BookieSocketAddress> + * @return a placement result containing list of bookie addresses for the ensemble. */ - Pair, Boolean> newEnsemble(int ensembleSize, - int writeQuorumSize, - int ackQuorumSize, - Map customMetadata, - Set excludeBookies) + PlacementResult> newEnsemble(int ensembleSize, + int writeQuorumSize, + int ackQuorumSize, + Map customMetadata, + Set excludeBookies) throws BKNotEnoughBookiesException; /** @@ -286,15 +285,15 @@ Pair, Boolean> newEnsemble(int ensembleSize, * @param bookieToReplace bookie to replace * @param excludeBookies bookies that should not be considered as candidate. * @throws BKNotEnoughBookiesException - * @return the org.apache.bookkeeper.net.BookieSocketAddress + * @return a placement result containing the new bookie address. */ - Pair replaceBookie(int ensembleSize, - int writeQuorumSize, - int ackQuorumSize, - Map customMetadata, - List currentEnsemble, - BookieSocketAddress bookieToReplace, - Set excludeBookies) + PlacementResult replaceBookie(int ensembleSize, + int writeQuorumSize, + int ackQuorumSize, + Map customMetadata, + List currentEnsemble, + BookieSocketAddress bookieToReplace, + Set excludeBookies) throws BKNotEnoughBookiesException; /** @@ -408,4 +407,29 @@ default boolean isEnsembleAdheringToPlacementPolicy(List en int ackQuorumSize) { return false; } + + /** + * Result of a placement calculation against a placement policy. + */ + final class PlacementResult { + private final T result; + private final boolean adhering; + + public static PlacementResult of(T result, boolean adhering) { + return new PlacementResult<>(result, adhering); + } + + private PlacementResult(T result, boolean adhering) { + this.result = result; + this.adhering = adhering; + } + + public T getResult() { + return result; + } + + public boolean isStrictlyAdheringToPolicy() { + return adhering; + } + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java index 254f5359c20..b196236af9b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java @@ -28,7 +28,6 @@ import org.apache.bookkeeper.common.annotation.InterfaceStability; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.Node; -import org.apache.commons.lang3.tuple.Pair; /** * Interface for topology aware ensemble placement policy. @@ -94,7 +93,7 @@ interface Ensemble { * @return list of bookies forming the ensemble * @throws BKException.BKNotEnoughBookiesException */ - Pair, Boolean> newEnsemble( + PlacementResult> newEnsemble( int ensembleSize, int writeQuorumSize, int ackQuorumSize, 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 8054d97187b..a7cdce4cb62 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 @@ -27,7 +27,6 @@ import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.commons.lang3.tuple.Pair; /** * A placement policy implementation use rack information for placing ensembles. @@ -92,7 +91,7 @@ public Set onClusterChanged(Set writab } @Override - public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { try { @@ -107,7 +106,7 @@ public Pair, Boolean> newEnsemble(int ensembleSize, in } @Override - public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { @@ -143,7 +142,7 @@ public DistributionSchedule.WriteSet reorderReadLACSequence( } @Override - public Pair, Boolean> newEnsemble(int ensembleSize, + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Set excludeBookies, 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 6db7de8f727..02cf3c3851c 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 @@ -71,7 +71,6 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -526,17 +525,17 @@ private static Set getNetworkLocations(Set bookieNodes) { } @Override - public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { return newEnsembleInternal(ensembleSize, writeQuorumSize, excludeBookies, null, null); } - protected Pair, Boolean> newEnsembleInternal(int ensembleSize, - int writeQuorumSize, - Set excludeBookies, - Ensemble parentEnsemble, - Predicate parentPredicate) + protected PlacementResult> newEnsembleInternal(int ensembleSize, + int writeQuorumSize, + Set excludeBookies, + Ensemble parentEnsemble, + Predicate parentPredicate) throws BKNotEnoughBookiesException { return newEnsembleInternal( ensembleSize, @@ -548,12 +547,12 @@ protected Pair, Boolean> newEnsembleInternal(int ensem } @Override - public Pair, Boolean> newEnsemble(int ensembleSize, - int writeQuorumSize, - int ackQuorumSize, - Set excludeBookies, - Ensemble parentEnsemble, - Predicate parentPredicate) + public PlacementResult> newEnsemble(int ensembleSize, + int writeQuorumSize, + int ackQuorumSize, + Set excludeBookies, + Ensemble parentEnsemble, + Predicate parentPredicate) throws BKNotEnoughBookiesException { return newEnsembleInternal( ensembleSize, @@ -564,7 +563,7 @@ public Pair, Boolean> newEnsemble(int ensembleSize, parentPredicate); } - protected Pair, Boolean> newEnsembleInternal( + protected PlacementResult> newEnsembleInternal( int ensembleSize, int writeQuorumSize, int ackQuorumSize, @@ -598,7 +597,7 @@ protected Pair, Boolean> newEnsembleInternal( for (BookieNode bn : bns) { addrs.add(bn.getAddr()); } - return Pair.of(addrs, false); + return PlacementResult.of(addrs, false); } for (int i = 0; i < ensembleSize; i++) { @@ -622,14 +621,16 @@ protected Pair, Boolean> newEnsembleInternal( ensembleSize, bookieList); throw new BKNotEnoughBookiesException(); } - return Pair.of(bookieList, isEnsembleAdheringToPlacementPolicy(bookieList, writeQuorumSize, ackQuorumSize)); + return PlacementResult.of(bookieList, + isEnsembleAdheringToPlacementPolicy( + bookieList, writeQuorumSize, ackQuorumSize)); } finally { rwLock.readLock().unlock(); } } @Override - public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { @@ -676,7 +677,7 @@ public Pair replaceBookie(int ensembleSize, int wr } else { newEnsemble.set(currentEnsemble.indexOf(bookieToReplace), candidateAddr); } - return Pair.of(candidateAddr, + return PlacementResult.of(candidateAddr, isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); } finally { rwLock.readLock().unlock(); 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 1bd4b75eac4..f91e9ef0f41 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 @@ -225,7 +225,7 @@ protected List selectRandomFromRegions(Set availableRegions, @Override - public Pair, Boolean> newEnsemble(int ensembleSize, int writeQuorumSize, + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { @@ -279,7 +279,9 @@ public Pair, Boolean> newEnsemble(int ensembleSize, in for (BookieNode bn : bns) { addrs.add(bn.getAddr()); } - return Pair.of(addrs, isEnsembleAdheringToPlacementPolicy(addrs, writeQuorumSize, ackQuorumSize)); + return PlacementResult.of(addrs, + isEnsembleAdheringToPlacementPolicy( + addrs, writeQuorumSize, ackQuorumSize)); } // Single region, fall back to RackAwareEnsemblePlacement @@ -347,7 +349,7 @@ public Pair, Boolean> newEnsemble(int ensembleSize, in try { List allocated = policyWithinRegion.newEnsemble(newEnsembleSize, newWriteQuorumSize, newWriteQuorumSize, excludeBookies, tempEnsemble, - tempEnsemble).getLeft(); + tempEnsemble).getResult(); ensemble = tempEnsemble; remainingEnsemble -= addToEnsembleSize; remainingWriteQuorum -= addToWriteQuorum; @@ -408,7 +410,7 @@ public Pair, Boolean> newEnsemble(int ensembleSize, in } LOG.info("Bookies allocated successfully {}", ensemble); List ensembleList = ensemble.toList(); - return Pair.of(ensembleList, + return PlacementResult.of(ensembleList, isEnsembleAdheringToPlacementPolicy(ensembleList, writeQuorumSize, ackQuorumSize)); } finally { rwLock.readLock().unlock(); @@ -416,7 +418,7 @@ public Pair, Boolean> newEnsemble(int ensembleSize, in } @Override - public Pair replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { @@ -482,7 +484,7 @@ public Pair replaceBookie(int ensembleSize, int wr } else { newEnsemble.set(currentEnsemble.indexOf(bookieToReplace), candidateAddr); } - return Pair.of(candidateAddr, + return PlacementResult.of(candidateAddr, isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); } finally { rwLock.readLock().unlock(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java index 205c5f4cb99..828a431920a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java @@ -33,7 +33,6 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.commons.lang3.tuple.Pair; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -70,7 +69,7 @@ public GenericEnsemblePlacementPolicyTest(boolean diskWeightBasedPlacementEnable public static final class CustomEnsemblePlacementPolicy extends DefaultEnsemblePlacementPolicy { @Override - public Pair replaceBookie(int ensembleSize, int writeQuorumSize, + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, List currentEnsemble, BookieSocketAddress bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { @@ -82,7 +81,7 @@ public Pair replaceBookie(int ensembleSize, int wr } @Override - public Pair, Boolean> newEnsemble(int ensembleSize, int quorumSize, + public PlacementResult> newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { assertNotNull(customMetadata); 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 e6cd07bcc7b..a55b5607412 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 @@ -52,7 +52,6 @@ import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; -import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -567,10 +566,10 @@ public void testReplaceBookieWithEnoughBookiesInSameRack() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, new HashSet<>()); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = + repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); + BookieSocketAddress replacedBookie = replaceBookieResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); assertEquals(addr3, replacedBookie); assertTrue(isEnsembleAdheringToPlacementPolicy); } @@ -596,10 +595,10 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRack() throws Exception // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, excludedAddrs); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = + repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, excludedAddrs); + BookieSocketAddress replacedBookie = replaceBookieResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); assertFalse(addr1.equals(replacedBookie)); assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); assertTrue(isEnsembleAdheringToPlacementPolicy); @@ -658,13 +657,13 @@ public void testReplaceBookieWithEnoughBookiesInSameRackAsEnsemble() throws Exce List ensembleBookies = new ArrayList(); ensembleBookies.add(addr2); ensembleBookies.add(addr4); - Pair replaceBookieResponse = repp.replaceBookie( + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = repp.replaceBookie( 1, 1, 1 , null, ensembleBookies, addr4, new HashSet<>()); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + BookieSocketAddress replacedBookie = replaceBookieResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); assertEquals(addr1, replacedBookie); assertTrue(isEnsembleAdheringToPlacementPolicy); } @@ -683,16 +682,16 @@ public void testNewEnsembleWithSingleRack() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; ensembleResponse = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); - List ensemble = ensembleResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + List ensemble = ensembleResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); assertEquals(0, getNumCoveredWriteQuorums(ensemble, 2, conf.getMinNumRacksPerWriteQuorum())); assertFalse(isEnsembleAdheringToPlacementPolicy); - Pair, Boolean> ensembleResponse2; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse2; ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()); - List ensemble2 = ensembleResponse2.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.getRight(); + List ensemble2 = ensembleResponse2.getResult(); + boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isStrictlyAdheringToPolicy(); assertEquals(0, getNumCoveredWriteQuorums(ensemble2, 2, conf.getMinNumRacksPerWriteQuorum())); assertFalse(isEnsembleAdheringToPlacementPolicy2); } catch (BKNotEnoughBookiesException bnebe) { @@ -724,19 +723,16 @@ public void testSingleRackWithEnforceMinNumRacks() throws Exception { addrs.add(addr3); addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); - Pair, Boolean> ensembleResponse; List ensemble; try { - ensembleResponse = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); - ensemble = ensembleResponse.getLeft(); + ensemble = repp.newEnsemble(3, 2, 2, null, new HashSet<>()).getResult(); fail("Should get not enough bookies exception since there is only one rack."); } catch (BKNotEnoughBookiesException bnebe) { } try { - ensembleResponse = repp.newEnsemble(3, 2, 2, new HashSet<>(), - EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); - ensemble = ensembleResponse.getLeft(); + ensemble = repp.newEnsemble(3, 2, 2, new HashSet<>(), + EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE).getResult(); fail("Should get not enough bookies exception since there is only one rack."); } catch (BKNotEnoughBookiesException bnebe) { } @@ -789,7 +785,7 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { * and there are enough bookies in 3 racks, this newEnsemble calls should * succeed. */ - Pair, Boolean> ensembleResponse; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; List ensemble; boolean isEnsembleAdheringToPlacementPolicy; int ensembleSize = numOfRacks * numOfBookiesPerRack; @@ -797,16 +793,16 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { int ackQuorumSize = numOfRacks; ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); - ensemble = ensembleResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + ensemble = ensembleResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); assertTrue(isEnsembleAdheringToPlacementPolicy); ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(), EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); - ensemble = ensembleResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + ensemble = ensembleResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); assertTrue(isEnsembleAdheringToPlacementPolicy); @@ -853,21 +849,21 @@ public void testNewEnsembleWithSufficientRacksAndEnforceMinNumRacks() throws Exc * ensembleSizes (as long as there are enough number of bookies in each * rack). */ - Pair, Boolean> ensembleResponse; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; List ensemble; boolean isEnsembleAdheringToPlacementPolicy; for (int ensembleSize = effectiveMinNumRacksPerWriteQuorum; ensembleSize < 40; ensembleSize++) { ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); - ensemble = ensembleResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + ensemble = ensembleResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); assertTrue(isEnsembleAdheringToPlacementPolicy); ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(), EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); - ensemble = ensembleResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + ensemble = ensembleResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); assertTrue(isEnsembleAdheringToPlacementPolicy); @@ -912,14 +908,14 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * and there are enough bookies in 3 racks, this newEnsemble call should * succeed. */ - Pair, Boolean> ensembleResponse; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; List ensemble; int ensembleSize = numOfRacks * numOfBookiesPerRack; int writeQuorumSize = numOfRacks; int ackQuorumSize = numOfRacks; ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); - ensemble = ensembleResponse.getLeft(); + ensemble = ensembleResponse.getResult(); BookieSocketAddress bookieInEnsembleToBeReplaced = ensemble.get(7); // get rack of some other bookie @@ -958,13 +954,13 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * this replaceBookie should succeed, because a new bookie is added to a * new rack. */ - Pair replaceBookieResponse; + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; BookieSocketAddress replacedBookieAddress; boolean isEnsembleAdheringToPlacementPolicy; replaceBookieResponse = repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, ensemble, bookieInEnsembleToBeReplaced, new HashSet<>()); - replacedBookieAddress = replaceBookieResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + replacedBookieAddress = replaceBookieResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); assertEquals("It should be newBookieAddress2", newBookieAddress2, replacedBookieAddress); assertTrue(isEnsembleAdheringToPlacementPolicy); @@ -998,8 +994,8 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { */ replaceBookieResponse = repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, ensemble, bookieInEnsembleToBeReplaced, bookiesToExclude); - replacedBookieAddress = replaceBookieResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + replacedBookieAddress = replaceBookieResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); assertEquals("It should be newBookieAddress3", newBookieAddress3, replacedBookieAddress); assertTrue(isEnsembleAdheringToPlacementPolicy); } @@ -1348,18 +1344,20 @@ public void testNewEnsembleWithMultipleRacks() throws Exception { int ensembleSize = 3; int writeQuorumSize = 2; int acqQuorumSize = 2; - Pair, Boolean> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, - acqQuorumSize, null, new HashSet<>()); - List ensemble = ensembleResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + repp.newEnsemble(ensembleSize, writeQuorumSize, + acqQuorumSize, null, new HashSet<>()); + List ensemble = ensembleResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); assertFalse(isEnsembleAdheringToPlacementPolicy); ensembleSize = 4; - Pair, Boolean> ensembleResponse2 = repp.newEnsemble(ensembleSize, writeQuorumSize, - acqQuorumSize, null, new HashSet<>()); - List ensemble2 = ensembleResponse2.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.getRight(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse2 = + repp.newEnsemble(ensembleSize, writeQuorumSize, + acqQuorumSize, null, new HashSet<>()); + List ensemble2 = ensembleResponse2.getResult(); + boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isStrictlyAdheringToPolicy(); numCovered = getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); assertFalse(isEnsembleAdheringToPlacementPolicy2); @@ -1431,10 +1429,11 @@ void validateNumOfWriteQuorumsCoveredInEnsembleCreation(Set repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); repp.onClusterChanged(addrs, new HashSet()); - Pair, Boolean> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, - writeQuorumSize, null, new HashSet<>()); - List ensemble = ensembleResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + repp.newEnsemble(ensembleSize, writeQuorumSize, + writeQuorumSize, null, new HashSet<>()); + List ensemble = ensembleResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, minNumRacksPerWriteQuorum); assertEquals("minimum number of racks covered for writequorum ensemble: " + ensemble, ensembleSize, numCovered); assertTrue(isEnsembleAdheringToPlacementPolicy); @@ -1475,19 +1474,20 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { int ensembleSize = 3; int writeQuorumSize = 3; int ackQuorumSize = 2; - Pair, Boolean> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, - ackQuorumSize, null, new HashSet<>()); - List ensemble1 = ensembleResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy1 = ensembleResponse.getRight(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + repp.newEnsemble(ensembleSize, writeQuorumSize, + ackQuorumSize, null, new HashSet<>()); + List ensemble1 = ensembleResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy1 = ensembleResponse.isStrictlyAdheringToPolicy(); assertEquals(ensembleSize, getNumCoveredWriteQuorums(ensemble1, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); assertTrue(isEnsembleAdheringToPlacementPolicy1); ensembleSize = 4; writeQuorumSize = 4; - Pair, Boolean> ensembleResponse2 = repp.newEnsemble(ensembleSize, writeQuorumSize, - 2, null, new HashSet<>()); - List ensemble2 = ensembleResponse2.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.getRight(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse2 = + repp.newEnsemble(ensembleSize, writeQuorumSize, 2, null, new HashSet<>()); + List ensemble2 = ensembleResponse2.getResult(); + boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isStrictlyAdheringToPolicy(); assertEquals(ensembleSize, getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); assertTrue(isEnsembleAdheringToPlacementPolicy2); @@ -1561,14 +1561,14 @@ public void testWeightedPlacementAndReplaceBookieWithEnoughBookiesInSameRack() t selectionCounts.put(addr3, 0L); selectionCounts.put(addr4, 0L); int numTries = 50000; - Pair replaceBookieResponse; + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; boolean isEnsembleAdheringToPlacementPolicy; BookieSocketAddress replacedBookie; for (int i = 0; i < numTries; i++) { // replace node under r2 replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); - replacedBookie = replaceBookieResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + replacedBookie = replaceBookieResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); assertTrue("replaced : " + replacedBookie, addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); assertTrue(isEnsembleAdheringToPlacementPolicy); selectionCounts.put(replacedBookie, selectionCounts.get(replacedBookie) + 1); @@ -1626,7 +1626,7 @@ public void testWeightedPlacementAndReplaceBookieWithoutEnoughBookiesInSameRack( selectionCounts.put(addr3, 0L); selectionCounts.put(addr4, 0L); int numTries = 50000; - Pair replaceBookieResponse; + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; BookieSocketAddress replacedBookie; boolean isEnsembleAdheringToPlacementPolicy; for (int i = 0; i < numTries; i++) { @@ -1634,8 +1634,8 @@ public void testWeightedPlacementAndReplaceBookieWithoutEnoughBookiesInSameRack( // will come from other racks. However, the weight should be honored in such // selections as well replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); - replacedBookie = replaceBookieResponse.getLeft(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getRight(); + replacedBookie = replaceBookieResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); assertTrue(addr0.equals(replacedBookie) || addr1.equals(replacedBookie) || addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); assertTrue(isEnsembleAdheringToPlacementPolicy); @@ -1730,7 +1730,7 @@ public void testWeightedPlacementAndNewEnsembleWithEnoughBookiesInSameRack() thr int numTries = 10000; Set excludeList = new HashSet(); - Pair, Boolean> ensembleResponse; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; List ensemble; int ensembleSize = 3; int writeQuorumSize = 2; @@ -1740,7 +1740,7 @@ public void testWeightedPlacementAndNewEnsembleWithEnoughBookiesInSameRack() thr // will come from other racks. However, the weight should be honored in such // selections as well ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, null, excludeList); - ensemble = ensembleResponse.getLeft(); + ensemble = ensembleResponse.getResult(); assertTrue( "Rackaware selection not happening " + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()), @@ -1802,7 +1802,7 @@ public void testWeightedPlacementAndNewEnsembleWithoutEnoughBookies() throws Exc bookieInfoMap.put(addr5, new BookieInfo(1000L, 1000L)); repp.updateBookieInfo(bookieInfoMap); - Pair, Boolean> ensembleResponse; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; List ensemble; Set excludeList = new HashSet(); try { @@ -1811,14 +1811,14 @@ public void testWeightedPlacementAndNewEnsembleWithoutEnoughBookies() throws Exc excludeList.add(addr3); excludeList.add(addr4); ensembleResponse = repp.newEnsemble(3, 2, 2, null, excludeList); - ensemble = ensembleResponse.getLeft(); + ensemble = ensembleResponse.getResult(); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies" + ensemble); } catch (BKNotEnoughBookiesException e) { // this is expected } try { ensembleResponse = repp.newEnsemble(1, 1, 1, null, excludeList); - ensemble = ensembleResponse.getLeft(); + ensemble = ensembleResponse.getResult(); } catch (BKNotEnoughBookiesException e) { fail("Should not throw BKNotEnoughBookiesException when there are enough bookies for the ensemble"); } @@ -1902,19 +1902,19 @@ public void testPlacementOnStabilizeNetworkTopology() throws Exception { // we will never use addr4 even it is in the stabilized network topology for (int i = 0; i < 5; i++) { - Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + repp.newEnsemble(3, 2, 2, null, new HashSet()); + List ensemble = ensembleResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); assertFalse(ensemble.contains(addr4)); assertFalse(isEnsembleAdheringToPlacementPolicy); } // we could still use addr4 for urgent allocation if it is just bookie flapping - Pair, Boolean> ensembleResponse = repp.newEnsemble(4, 2, 2, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.getRight(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + repp.newEnsemble(4, 2, 2, null, new HashSet()); + List ensemble = ensembleResponse.getResult(); + boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); assertFalse(isEnsembleAdheringToPlacementPolicy); assertTrue(ensemble.contains(addr4)); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java index e0fd2bdffe2..2aff5d88201 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java @@ -46,7 +46,6 @@ import org.apache.bookkeeper.net.ScriptBasedMapping; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.Shell; -import org.apache.commons.lang3.tuple.Pair; import org.junit.After; import org.junit.Assume; import org.junit.Before; @@ -114,9 +113,8 @@ public void testReplaceBookieWithEnoughBookiesInSameRack() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, new HashSet<>()); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, new HashSet<>()).getResult(); assertEquals(addr3, replacedBookie); } @@ -138,9 +136,8 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRack() throws Exception // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, excludedAddrs); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, excludedAddrs).getResult(); assertFalse(addr1.equals(replacedBookie)); assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); @@ -204,9 +201,8 @@ public void testReplaceBookieWithScriptMappingError() throws Exception { // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, excludedAddrs); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, excludedAddrs).getResult(); assertFalse(addr1.equals(replacedBookie)); assertFalse(addr2.equals(replacedBookie)); @@ -244,9 +240,8 @@ public void testReplaceBookieWithScriptMappingError2() throws Exception { // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, excludedAddrs); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2, excludedAddrs).getResult(); assertFalse(addr1.equals(replacedBookie)); assertFalse(addr2.equals(replacedBookie)); @@ -268,13 +263,11 @@ public void testNewEnsembleWithSingleRack() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, - new HashSet<>()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(3, 2, 2, null, + new HashSet<>()).getResult(); assertEquals(0, getNumCoveredWriteQuorums(ensemble, 2)); - Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, - new HashSet<>()); - List ensemble2 = ensembleResponse2.getLeft(); + List ensemble2 = repp.newEnsemble(4, 2, 2, null, + new HashSet<>()).getResult(); assertEquals(0, getNumCoveredWriteQuorums(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -296,14 +289,12 @@ public void testNewEnsembleWithMultipleRacks() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, - new HashSet<>()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(3, 2, 2, null, + new HashSet<>()).getResult(); int numCovered = getNumCoveredWriteQuorums(ensemble, 2); assertTrue(numCovered == 2); - Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, - new HashSet<>()); - List ensemble2 = ensembleResponse2.getLeft(); + List ensemble2 = repp.newEnsemble(4, 2, 2, null, + new HashSet<>()).getResult(); numCovered = getNumCoveredWriteQuorums(ensemble2, 2); assertTrue(numCovered == 2); } catch (BKNotEnoughBookiesException bnebe) { @@ -334,13 +325,11 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { addrs.add(addr8); repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse1 = repp.newEnsemble(3, 2, 2, null, - new HashSet<>()); - List ensemble1 = ensembleResponse1.getLeft(); + List ensemble1 = repp.newEnsemble(3, 2, 2, null, + new HashSet<>()).getResult(); assertEquals(3, getNumCoveredWriteQuorums(ensemble1, 2)); - Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, - new HashSet<>()); - List ensemble2 = ensembleResponse2.getLeft(); + List ensemble2 = repp.newEnsemble(4, 2, 2, null, + new HashSet<>()).getResult(); assertEquals(4, getNumCoveredWriteQuorums(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception."); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java index 3192d049c12..b206e649665 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java @@ -42,7 +42,7 @@ import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; -import org.apache.commons.lang3.tuple.Pair; + import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -105,9 +105,8 @@ public void testNotifyRackChange() throws Exception { int ensembleSize = 3; int writeQuorumSize = 2; int acqQuorumSize = 2; - Pair, Boolean> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, - acqQuorumSize, Collections.emptyMap(), Collections.emptySet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, + acqQuorumSize, Collections.emptyMap(), Collections.emptySet()).getResult(); int numCovered = TestRackawareEnsemblePlacementPolicy.getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); @@ -120,9 +119,8 @@ public void testNotifyRackChange() throws Exception { StaticDNSResolver.changeRack(bookieAddressList, rackList); numOfAvailableRacks = numOfAvailableRacks + 1; acqQuorumSize = 1; - ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, Collections.emptyMap(), - Collections.emptySet()); - ensemble = ensembleResponse.getLeft(); + ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, Collections.emptyMap(), + Collections.emptySet()).getResult(); assertEquals(3, TestRackawareEnsemblePlacementPolicy.getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); assertTrue(ensemble.contains(addr1)); 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 8e4f10d5b3b..a7f42aa35d1 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 @@ -51,7 +51,6 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.StaticDNSResolver; -import org.apache.commons.lang3.tuple.Pair; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -424,9 +423,8 @@ public void testReplaceBookieWithEnoughBookiesInSameRegion() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, - new ArrayList(), addr2, new HashSet()); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, + new ArrayList(), addr2, new HashSet()).getResult(); assertEquals(addr3, replacedBookie); } @@ -451,9 +449,8 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRegion() throws Excepti // replace node under r2 Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - Pair replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, - new ArrayList(), addr2, excludedAddrs); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, + new ArrayList(), addr2, excludedAddrs).getResult(); assertFalse(addr1.equals(replacedBookie)); assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); @@ -478,9 +475,8 @@ public void testNewEnsembleBookieWithNotEnoughBookies() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse = repp.newEnsemble(5, 5, 3, null, - new HashSet()); - List list = ensembleResponse.getLeft(); + List list = repp.newEnsemble(5, 5, 3, null, + new HashSet()).getResult(); LOG.info("Ensemble : {}", list); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { @@ -541,13 +537,11 @@ public void testNewEnsembleWithSingleRegion() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(3, 2, 2, null, + new HashSet()).getResult(); assertEquals(0, getNumCoveredRegionsInWriteQuorum(ensemble, 2)); - Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, - new HashSet()); - List ensemble2 = ensembleResponse2.getLeft(); + List ensemble2 = repp.newEnsemble(4, 2, 2, null, + new HashSet()).getResult(); assertEquals(0, getNumCoveredRegionsInWriteQuorum(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -576,9 +570,8 @@ public void testNewEnsembleWithMultipleRegions() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse = repp.newEnsemble(3, 2, 2, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(3, 2, 2, null, + new HashSet()).getResult(); int numCovered = getNumCoveredRegionsInWriteQuorum(ensemble, 2); assertTrue(numCovered >= 1); assertTrue(numCovered < 3); @@ -586,9 +579,8 @@ public void testNewEnsembleWithMultipleRegions() throws Exception { fail("Should not get not enough bookies exception even there is only one rack."); } try { - Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, - new HashSet()); - List ensemble2 = ensembleResponse2.getLeft(); + List ensemble2 = repp.newEnsemble(4, 2, 2, null, + new HashSet()).getResult(); int numCovered = getNumCoveredRegionsInWriteQuorum(ensemble2, 2); assertTrue(numCovered >= 1 && numCovered < 3); } catch (BKNotEnoughBookiesException bnebe) { @@ -627,13 +619,11 @@ public void testNewEnsembleWithEnoughRegions() throws Exception { addrs.add(addr8); repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse1 = repp.newEnsemble(3, 2, 2, null, - new HashSet()); - List ensemble1 = ensembleResponse1.getLeft(); + List ensemble1 = repp.newEnsemble(3, 2, 2, null, + new HashSet()).getResult(); assertEquals(3, getNumCoveredRegionsInWriteQuorum(ensemble1, 2)); - Pair, Boolean> ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, - new HashSet()); - List ensemble2 = ensembleResponse2.getLeft(); + List ensemble2 = repp.newEnsemble(4, 2, 2, null, + new HashSet()).getResult(); assertEquals(4, getNumCoveredRegionsInWriteQuorum(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -680,27 +670,23 @@ public void testNewEnsembleWithThreeRegions() throws Exception { addrs.add(addr10); repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse = repp.newEnsemble(6, 6, 4, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(6, 6, 4, null, + new HashSet()).getResult(); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 6); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensembleResponse = repp.newEnsemble(7, 7, 4, null, new HashSet()); - ensemble = ensembleResponse.getLeft(); + ensemble = repp.newEnsemble(7, 7, 4, null, new HashSet()).getResult(); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 7); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensembleResponse = repp.newEnsemble(8, 8, 5, null, new HashSet()); - ensemble = ensembleResponse.getLeft(); + ensemble = repp.newEnsemble(8, 8, 5, null, new HashSet()).getResult(); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 8); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensembleResponse = repp.newEnsemble(9, 9, 5, null, new HashSet()); - ensemble = ensembleResponse.getLeft(); + ensemble = repp.newEnsemble(9, 9, 5, null, new HashSet()).getResult(); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 9); @@ -753,9 +739,8 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { repp.onClusterChanged(addrs, new HashSet()); try { ((SettableFeature) featureProvider.scope("region1").getFeature("disallowBookies")).set(true); - Pair, Boolean> ensembleResponse = repp.newEnsemble(6, 6, 4, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(6, 6, 4, null, + new HashSet()).getResult(); assertEquals(2, getNumRegionsInEnsemble(ensemble)); assert(ensemble.contains(addr1)); assert(ensemble.contains(addr3)); @@ -776,9 +761,8 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { } try { ((SettableFeature) featureProvider.scope("region2").getFeature("disallowBookies")).set(false); - Pair, Boolean> ensembleResponse = repp.newEnsemble(6, 6, 4, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(6, 6, 4, null, + new HashSet()).getResult(); assert(ensemble.contains(addr1)); assert(ensemble.contains(addr3)); assert(ensemble.contains(addr4)); @@ -851,9 +835,8 @@ public void testNewEnsembleWithFiveRegions() throws Exception { repp.onClusterChanged(addrs, new HashSet()); try { - Pair, Boolean> ensembleResponse = repp.newEnsemble(10, 10, 10, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(10, 10, 10, null, + new HashSet()).getResult(); assert(ensemble.size() == 10); assertEquals(5, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -864,9 +847,8 @@ public void testNewEnsembleWithFiveRegions() throws Exception { try { Set excludedAddrs = new HashSet(); excludedAddrs.add(addr10); - Pair, Boolean> ensembleResponse = repp.newEnsemble(10, 10, 10, null, - excludedAddrs); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(10, 10, 10, null, + excludedAddrs).getResult(); assert(ensemble.contains(addr11) && ensemble.contains(addr12)); assert(ensemble.size() == 10); assertEquals(5, getNumRegionsInEnsemble(ensemble)); @@ -956,11 +938,9 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole ackQuorum = 5; } - Pair, Boolean> ensembleResponse; List ensemble; try { - ensembleResponse = repp.newEnsemble(6, 6, ackQuorum, null, new HashSet()); - ensemble = ensembleResponse.getLeft(); + ensemble = repp.newEnsemble(6, 6, ackQuorum, null, new HashSet()).getResult(); assert(ensemble.size() == 6); assertEquals(3, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -981,9 +961,8 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole Set excludedAddrs = new HashSet(); for (BookieSocketAddress addr: region2Bookies) { if (ensemble.contains(addr)) { - Pair replaceBookieResponse = repp.replaceBookie(6, 6, ackQuorum, null, - ensemble, addr, excludedAddrs); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + BookieSocketAddress replacedBookie = repp.replaceBookie(6, 6, ackQuorum, null, + ensemble, addr, excludedAddrs).getResult(); ensemble.remove(addr); ensemble.add(replacedBookie); } @@ -1007,9 +986,8 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole Set excludedAddrs = new HashSet(); try { - Pair replaceBookieResponse = repp.replaceBookie(6, 6, ackQuorum, null, - ensemble, bookieToReplace, excludedAddrs); - BookieSocketAddress replacedBookie = replaceBookieResponse.getLeft(); + BookieSocketAddress replacedBookie = repp.replaceBookie(6, 6, ackQuorum, null, + ensemble, bookieToReplace, excludedAddrs).getResult(); assert (replacedBookie.equals(replacedBookieExpected)); assertEquals(3, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -1092,11 +1070,9 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di .set(true); } - Pair, Boolean> ensembleResponse; List ensemble; try { - ensembleResponse = repp.newEnsemble(6, 6, 4, null, new HashSet()); - ensemble = ensembleResponse.getLeft(); + ensemble = repp.newEnsemble(6, 6, 4, null, new HashSet()).getResult(); assert(ensemble.size() == 6); } catch (BKNotEnoughBookiesException bnebe) { LOG.error("BKNotEnoughBookiesException", bnebe); @@ -1160,8 +1136,7 @@ public void testNewEnsembleFailWithFiveRegions() throws Exception { excludedAddrs.add(addr10); excludedAddrs.add(addr9); try { - Pair, Boolean> list = repp.newEnsemble(5, 5, 5, null, excludedAddrs); - LOG.info("Ensemble : {}", list.getLeft()); + LOG.info("Ensemble : {}", repp.newEnsemble(5, 5, 5, null, excludedAddrs).getResult()); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not enou @@ -1220,9 +1195,8 @@ public void testBasicReorderReadLACSequenceWithLocalRegion() throws Exception { private void basicReorderReadSequenceWithLocalRegionTest(String myRegion, boolean isReadLAC) throws Exception { prepareNetworkTopologyForReorderTests(myRegion); - Pair, Boolean> ensembleResponse = repp.newEnsemble(9, 9, 5, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(9, 9, 5, null, + new HashSet()).getResult(); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); @@ -1278,9 +1252,8 @@ public void testBasicReorderReadLACSequenceWithRemoteRegion() throws Exception { private void basicReorderReadSequenceWithRemoteRegionTest(String myRegion, boolean isReadLAC) throws Exception { prepareNetworkTopologyForReorderTests(myRegion); - Pair, Boolean> ensembleResponse = repp.newEnsemble(9, 9, 5, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(9, 9, 5, null, + new HashSet()).getResult(); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); @@ -1350,9 +1323,8 @@ private void reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(boolean isR prepareNetworkTopologyForReorderTests(myRegion); - Pair, Boolean> ensembleResponse = repp.newEnsemble(9, 9, 5, null, - new HashSet()); - List ensemble = ensembleResponse.getLeft(); + List ensemble = repp.newEnsemble(9, 9, 5, null, + new HashSet()).getResult(); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); From 0b087de518a28b344fc31c26d7413fc84cbf39d6 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 31 Jan 2019 02:17:09 -0800 Subject: [PATCH 0205/1642] Ignore usage of localNode in PlacementPolicy. Descriptions of the changes in this PR: 'ignoreLocalNodeInPlacementPolicy' specifies whether to ignore localnode in the internal logic of placement policy. If it is not possible or useful to use Bookkeeper client node's (or AutoReplicator) rack/region info. for placement policy then it is better to ignore localnode instead of false alarming with error/warn log lines and metrics. It is not valid to expect rack mapping information to be available for Bookkeeper client and ReplicationWorker nodes. So if rackmapping info. is not available then it is better to ignore creation of BookieNode by resolving rack information for the client localnode. Otherwise it would log unnecessary warn/error log lines and rack resolution failure metrics. Reviewers: Sijie Guo , Jia Zhai , Enrico Olivelli This closes #1917 from reddycharan/ignorelocalnodeinplacementpolicy --- .../RackawareEnsemblePlacementPolicy.java | 8 +++++--- .../RackawareEnsemblePlacementPolicyImpl.java | 19 +++++++++++++------ .../RegionAwareEnsemblePlacementPolicy.java | 6 ++++-- .../conf/AbstractConfiguration.java | 17 +++++++++++++++++ site/_data/config/bk_server.yaml | 4 ++++ 5 files changed, 43 insertions(+), 11 deletions(-) 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 a7cdce4cb62..49e9df53da7 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 @@ -55,18 +55,20 @@ protected RackawareEnsemblePlacementPolicy initialize(DNSToSwitchMapping dnsReso int maxWeightMultiple, int minNumRacksPerWriteQuorum, boolean enforceMinNumRacksPerWriteQuorum, + boolean ignoreLocalNodeInPlacementPolicy, StatsLogger statsLogger) { if (stabilizePeriodSeconds > 0) { super.initialize(dnsResolver, timer, reorderReadsRandom, 0, reorderThresholdPendingRequests, isWeighted, - maxWeightMultiple, minNumRacksPerWriteQuorum, enforceMinNumRacksPerWriteQuorum, statsLogger); + maxWeightMultiple, minNumRacksPerWriteQuorum, enforceMinNumRacksPerWriteQuorum, + ignoreLocalNodeInPlacementPolicy, statsLogger); slave = new RackawareEnsemblePlacementPolicyImpl(enforceDurability); slave.initialize(dnsResolver, timer, reorderReadsRandom, stabilizePeriodSeconds, reorderThresholdPendingRequests, isWeighted, maxWeightMultiple, minNumRacksPerWriteQuorum, - enforceMinNumRacksPerWriteQuorum, statsLogger); + enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy, statsLogger); } else { super.initialize(dnsResolver, timer, reorderReadsRandom, stabilizePeriodSeconds, reorderThresholdPendingRequests, isWeighted, maxWeightMultiple, minNumRacksPerWriteQuorum, - enforceMinNumRacksPerWriteQuorum, statsLogger); + enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy, statsLogger); slave = null; } return this; 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 02cf3c3851c..304db72745c 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 @@ -93,6 +93,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP protected int minNumRacksPerWriteQuorum; protected boolean enforceMinNumRacksPerWriteQuorum; + protected boolean ignoreLocalNodeInPlacementPolicy; public static final String REPP_DNS_RESOLVER_CLASS = "reppDnsResolverClass"; public static final String REPP_RANDOM_READ_REORDERING = "ensembleRandomReadReordering"; @@ -274,6 +275,7 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns int maxWeightMultiple, int minNumRacksPerWriteQuorum, boolean enforceMinNumRacksPerWriteQuorum, + boolean ignoreLocalNodeInPlacementPolicy, StatsLogger statsLogger) { checkNotNull(statsLogger, "statsLogger should not be null, use NullStatsLogger instead."); this.statsLogger = statsLogger; @@ -290,6 +292,7 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns this.timer = timer; this.minNumRacksPerWriteQuorum = minNumRacksPerWriteQuorum; this.enforceMinNumRacksPerWriteQuorum = enforceMinNumRacksPerWriteQuorum; + this.ignoreLocalNodeInPlacementPolicy = ignoreLocalNodeInPlacementPolicy; // create the network topology if (stabilizePeriodSeconds > 0) { @@ -298,12 +301,15 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns this.topology = new NetworkTopologyImpl(); } - BookieNode bn; - try { - bn = createBookieNode(new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), 0)); - } catch (UnknownHostException e) { - LOG.error("Failed to get local host address : ", e); - bn = null; + BookieNode bn = null; + if (!ignoreLocalNodeInPlacementPolicy) { + try { + bn = createBookieNode(new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), 0)); + } catch (UnknownHostException e) { + LOG.error("Failed to get local host address : ", e); + } + } else { + LOG.info("Ignoring LocalNode in Placementpolicy"); } localNode = bn; LOG.info("Initialize rackaware ensemble placement policy @ {} @ {} : {}.", @@ -390,6 +396,7 @@ public Long load(BookieSocketAddress key) throws Exception { conf.getBookieMaxWeightMultipleForWeightBasedPlacement(), conf.getMinNumRacksPerWriteQuorum(), conf.getEnforceMinNumRacksPerWriteQuorum(), + conf.getIgnoreLocalNodeInPlacementPolicy(), statsLogger); } 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 f91e9ef0f41..cd80fdf3f49 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 @@ -131,7 +131,8 @@ public void handleBookiesThatJoined(Set joinedBookies) { perRegionPlacement.put(region, new RackawareEnsemblePlacementPolicy() .initialize(dnsResolver, timer, this.reorderReadsRandom, this.stabilizePeriodSeconds, this.reorderThresholdPendingRequests, this.isWeighted, this.maxWeightMultiple, - this.minNumRacksPerWriteQuorum, this.enforceMinNumRacksPerWriteQuorum, statsLogger) + this.minNumRacksPerWriteQuorum, this.enforceMinNumRacksPerWriteQuorum, + this.ignoreLocalNodeInPlacementPolicy, statsLogger) .withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK)); } @@ -180,7 +181,8 @@ public RegionAwareEnsemblePlacementPolicy initialize(ClientConfiguration conf, perRegionPlacement.put(region, new RackawareEnsemblePlacementPolicy(true) .initialize(dnsResolver, timer, this.reorderReadsRandom, this.stabilizePeriodSeconds, this.reorderThresholdPendingRequests, this.isWeighted, this.maxWeightMultiple, - this.minNumRacksPerWriteQuorum, this.enforceMinNumRacksPerWriteQuorum, statsLogger) + this.minNumRacksPerWriteQuorum, this.enforceMinNumRacksPerWriteQuorum, + this.ignoreLocalNodeInPlacementPolicy, statsLogger) .withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK)); } minRegionsForDurability = conf.getInt(REPP_MINIMUM_REGIONS_FOR_DURABILITY, 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 36cfa637e97..87c1f415461 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 @@ -159,6 +159,9 @@ public abstract class AbstractConfiguration // enforce minimum number of racks per write quorum public static final String ENFORCE_MIN_NUM_RACKS_PER_WRITE_QUORUM = "enforceMinNumRacksPerWriteQuorum"; + // ignore usage of local node in the internal logic of placement policy + public static final String IGNORE_LOCAL_NODE_IN_PLACEMENT_POLICY = "ignoreLocalNodeInPlacementPolicy"; + // Allocator configuration protected static final String ALLOCATOR_POOLING_POLICY = "allocatorPoolingPolicy"; protected static final String ALLOCATOR_POOLING_CONCURRENCY = "allocatorPoolingConcurrency"; @@ -843,6 +846,20 @@ public boolean getEnforceMinNumRacksPerWriteQuorum() { return getBoolean(ENFORCE_MIN_NUM_RACKS_PER_WRITE_QUORUM, false); } + /** + * Sets the flag to ignore usage of localnode in placement policy. + */ + public void setIgnoreLocalNodeInPlacementPolicy(boolean ignoreLocalNodeInPlacementPolicy) { + setProperty(IGNORE_LOCAL_NODE_IN_PLACEMENT_POLICY, ignoreLocalNodeInPlacementPolicy); + } + + /** + * Whether to ignore localnode in placementpolicy. + */ + public boolean getIgnoreLocalNodeInPlacementPolicy() { + return getBoolean(IGNORE_LOCAL_NODE_IN_PLACEMENT_POLICY, false); + } + /** * Enable the Auditor to use system time as underreplicated ledger mark * time. diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index d83a8f1024a..f74da27e2e0 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -640,6 +640,10 @@ groups: - param: enforceMinNumRacksPerWriteQuorum description: | 'enforceMinNumRacksPerWriteQuorum' enforces RackawareEnsemblePlacementPolicy to pick bookies from 'minNumRacksPerWriteQuorum' racks for a writeQuorum. If it cann't find bookie then it would throw BKNotEnoughBookiesException instead of picking random one. + - param: ignoreLocalNodeInPlacementPolicy + description: | + 'ignoreLocalNodeInPlacementPolicy' specifies whether to ignore localnode in the internal logic of placement policy. If it is not possible or useful to use Bookkeeper client node's (or AutoReplicator) rack/region info. for placement policy then it is better to ignore localnode instead of false alarming with log lines and metrics. + default: false - name: AutoRecovery auditor settings params: From b348e330c7c0a23b513bae6000d5ce1c17fdd260 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 1 Feb 2019 23:52:12 +0800 Subject: [PATCH 0206/1642] [RELEASE] Update website to include documentation for 4.9.0 Reviewers: Jia Zhai This closes #1928 from sijie/update_website_490 --- site/_config.yml | 5 +- site/docs/4.9.0/admin/autorecovery.md | 128 +++ site/docs/4.9.0/admin/bookies.md | 180 ++++ site/docs/4.9.0/admin/geo-replication.md | 22 + site/docs/4.9.0/admin/http.md | 457 ++++++++++ site/docs/4.9.0/admin/metrics.md | 41 + site/docs/4.9.0/admin/perf.md | 3 + site/docs/4.9.0/admin/placement.md | 3 + site/docs/4.9.0/admin/upgrade.md | 175 ++++ site/docs/4.9.0/api/distributedlog-api.md | 395 ++++++++ site/docs/4.9.0/api/ledger-adv-api.md | 111 +++ site/docs/4.9.0/api/ledger-api.md | 841 ++++++++++++++++++ site/docs/4.9.0/api/overview.md | 17 + site/docs/4.9.0/deployment/dcos.md | 142 +++ site/docs/4.9.0/deployment/kubernetes.md | 181 ++++ site/docs/4.9.0/deployment/manual.md | 56 ++ site/docs/4.9.0/development/codebase.md | 3 + site/docs/4.9.0/development/protocol.md | 148 +++ site/docs/4.9.0/getting-started/concepts.md | 202 +++++ .../4.9.0/getting-started/installation.md | 82 ++ .../docs/4.9.0/getting-started/run-locally.md | 16 + site/docs/4.9.0/overview/overview.md | 60 ++ site/docs/4.9.0/overview/releaseNotes.md | 17 + .../4.9.0/overview/releaseNotesTemplate.md | 17 + site/docs/4.9.0/reference/cli.md | 10 + site/docs/4.9.0/reference/config.md | 9 + site/docs/4.9.0/reference/metrics.md | 3 + site/docs/4.9.0/security/overview.md | 21 + site/docs/4.9.0/security/sasl.md | 202 +++++ site/docs/4.9.0/security/tls.md | 210 +++++ site/docs/4.9.0/security/zookeeper.md | 41 + site/docs/latest/overview/overview.md | 2 +- site/docs/latest/overview/releaseNotes.md | 4 +- .../latest/overview/releaseNotesTemplate.md | 2 +- 34 files changed, 3800 insertions(+), 6 deletions(-) create mode 100644 site/docs/4.9.0/admin/autorecovery.md create mode 100644 site/docs/4.9.0/admin/bookies.md create mode 100644 site/docs/4.9.0/admin/geo-replication.md create mode 100644 site/docs/4.9.0/admin/http.md create mode 100644 site/docs/4.9.0/admin/metrics.md create mode 100644 site/docs/4.9.0/admin/perf.md create mode 100644 site/docs/4.9.0/admin/placement.md create mode 100644 site/docs/4.9.0/admin/upgrade.md create mode 100644 site/docs/4.9.0/api/distributedlog-api.md create mode 100644 site/docs/4.9.0/api/ledger-adv-api.md create mode 100644 site/docs/4.9.0/api/ledger-api.md create mode 100644 site/docs/4.9.0/api/overview.md create mode 100644 site/docs/4.9.0/deployment/dcos.md create mode 100644 site/docs/4.9.0/deployment/kubernetes.md create mode 100644 site/docs/4.9.0/deployment/manual.md create mode 100644 site/docs/4.9.0/development/codebase.md create mode 100644 site/docs/4.9.0/development/protocol.md create mode 100644 site/docs/4.9.0/getting-started/concepts.md create mode 100644 site/docs/4.9.0/getting-started/installation.md create mode 100644 site/docs/4.9.0/getting-started/run-locally.md create mode 100644 site/docs/4.9.0/overview/overview.md create mode 100644 site/docs/4.9.0/overview/releaseNotes.md create mode 100644 site/docs/4.9.0/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.9.0/reference/cli.md create mode 100644 site/docs/4.9.0/reference/config.md create mode 100644 site/docs/4.9.0/reference/metrics.md create mode 100644 site/docs/4.9.0/security/overview.md create mode 100644 site/docs/4.9.0/security/sasl.md create mode 100644 site/docs/4.9.0/security/tls.md create mode 100644 site/docs/4.9.0/security/zookeeper.md diff --git a/site/_config.yml b/site/_config.yml index 3e4c3aa89e6..1ccfb677875 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -9,6 +9,7 @@ twitter_url: https://twitter.com/asfbookkeeper versions: # [next_version_placeholder] +- "4.9.0" - "4.8.1" - "4.8.0" - "4.7.3" @@ -33,8 +34,8 @@ archived_versions: - "4.2.0" - "4.1.0" - "4.0.0" -latest_version: "4.9.0-SNAPSHOT" -latest_release: "4.8.1" +latest_version: "4.10.0-SNAPSHOT" +latest_release: "4.9.0" stable_release: "4.7.3" distributedlog_version: "4.7.3" diff --git a/site/docs/4.9.0/admin/autorecovery.md b/site/docs/4.9.0/admin/autorecovery.md new file mode 100644 index 00000000000..b1dd078f9b2 --- /dev/null +++ b/site/docs/4.9.0/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: + +* the `shell recover` option +* 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 shell recover \ + 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 shell recover \ + 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.9.0/admin/bookies.md b/site/docs/4.9.0/admin/bookies.md new file mode 100644 index 00000000000..1b0427dae3c --- /dev/null +++ b/site/docs/4.9.0/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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + \ + \ + + ``` + + 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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + 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.9.0/admin/geo-replication.md b/site/docs/4.9.0/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.9.0/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.9.0/admin/http.md b/site/docs/4.9.0/admin/http.md new file mode 100644 index 00000000000..270dcb3a16f --- /dev/null +++ b/site/docs/4.9.0/admin/http.md @@ -0,0 +1,457 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 4 components: +1. Heartbeat: heartbeat for a specific bookie. +1. Config: doing the server configuration for a specific bookie. +1. Ledger: HTTP endpoints related to ledgers. +1. Bookie: HTTP endpoints related to bookies. +1. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Config + +### Endpoint: /metrics +1. Method: GET + * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/bookie/gc +1. Method: PUT + * Description: trigger gc for this bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: GET + * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + { + "is_in_force_gc" : "false" + } + ``` + +### Endpoint: /api/v1/bookie/gc_details +1. Method: GET + * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + [ { + "forceCompacting" : false, + "majorCompacting" : false, + "minorCompacting" : false, + "lastMajorCompactionTime" : 1544578144944, + "lastMinorCompactionTime" : 1544578144944, + "majorCompactionCounter" : 1, + "minorCompactionCounter" : 0 + } ] + ``` + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.9.0/admin/metrics.md b/site/docs/4.9.0/admin/metrics.md new file mode 100644 index 00000000000..142df3dcd2d --- /dev/null +++ b/site/docs/4.9.0/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.prometheus.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.9.0/admin/perf.md b/site/docs/4.9.0/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.9.0/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.9.0/admin/placement.md b/site/docs/4.9.0/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.9.0/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.9.0/admin/upgrade.md b/site/docs/4.9.0/admin/upgrade.md new file mode 100644 index 00000000000..57c65208131 --- /dev/null +++ b/site/docs/4.9.0/admin/upgrade.md @@ -0,0 +1,175 @@ +--- +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.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.7.x to 4.8.X upgrade + +In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.9.0/api/distributedlog-api.md b/site/docs/4.9.0/api/distributedlog-api.md new file mode 100644 index 00000000000..5b1ebfb9d62 --- /dev/null +++ b/site/docs/4.9.0/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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.9.0/api/ledger-adv-api.md b/site/docs/4.9.0/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.9.0/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.9.0/api/ledger-api.md b/site/docs/4.9.0/api/ledger-api.md new file mode 100644 index 00000000000..15d282945b0 --- /dev/null +++ b/site/docs/4.9.0/api/ledger-api.md @@ -0,0 +1,841 @@ +--- +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.9.0 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.9.0 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.9.0' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.9.0' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` + +### Relaxing Durability + +In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. +In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will +be always readable by other clients. + +On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after +writing the entry to SO buffers without waiting for an fsync. +In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. +Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. + +In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble ackknowledge the call after +performing an fsync to the disk which is storing the journal. +This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. + +The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. +In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. + + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete + + +wh.force().get(); // wait for fsync, make data available to readers and to the replicator + +wh.close(); // seal the ledger + +``` diff --git a/site/docs/4.9.0/api/overview.md b/site/docs/4.9.0/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.9.0/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.9.0/deployment/dcos.md b/site/docs/4.9.0/deployment/dcos.md new file mode 100644 index 00000000000..1373b01628f --- /dev/null +++ b/site/docs/4.9.0/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.9.0] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.9.0/deployment/kubernetes.md b/site/docs/4.9.0/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.9.0/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.9.0/deployment/manual.md b/site/docs/4.9.0/deployment/manual.md new file mode 100644 index 00000000000..daafd5556f5 --- /dev/null +++ b/site/docs/4.9.0/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.9.0/development/codebase.md b/site/docs/4.9.0/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.9.0/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.9.0/development/protocol.md b/site/docs/4.9.0/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.9.0/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.9.0/getting-started/concepts.md b/site/docs/4.9.0/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.9.0/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.9.0/getting-started/installation.md b/site/docs/4.9.0/getting-started/installation.md new file mode 100644 index 00000000000..7e92b471581 --- /dev/null +++ b/site/docs/4.9.0/getting-started/installation.md @@ -0,0 +1,82 @@ +--- +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 +``` + +Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. + +```shell +$ mvn package -Dstream +``` + +> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin + +> You can enable `table service` by adding the `-Dstream` flag when running above commands. + +## 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.9.0/getting-started/run-locally.md b/site/docs/4.9.0/getting-started/run-locally.md new file mode 100644 index 00000000000..edbfab9fda6 --- /dev/null +++ b/site/docs/4.9.0/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 +$ bookkeeper-server/bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.9.0/overview/overview.md b/site/docs/4.9.0/overview/overview.md new file mode 100644 index 00000000000..dc219e73489 --- /dev/null +++ b/site/docs/4.9.0/overview/overview.md @@ -0,0 +1,60 @@ +--- +title: Apache BookKeeper™ 4.9.0 +--- + + +This documentation is for Apache BookKeeper™ version 4.9.0. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.9.0 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.9.0/overview/releaseNotes.md b/site/docs/4.9.0/overview/releaseNotes.md new file mode 100644 index 00000000000..28a667b2e26 --- /dev/null +++ b/site/docs/4.9.0/overview/releaseNotes.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.9.0 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.9.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.9.0/overview/releaseNotesTemplate.md b/site/docs/4.9.0/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..28a667b2e26 --- /dev/null +++ b/site/docs/4.9.0/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.9.0 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.9.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.9.0/reference/cli.md b/site/docs/4.9.0/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.9.0/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.9.0/reference/config.md b/site/docs/4.9.0/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.9.0/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.9.0/reference/metrics.md b/site/docs/4.9.0/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.9.0/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.9.0/security/overview.md b/site/docs/4.9.0/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.9.0/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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.9.0/security/sasl.md b/site/docs/4.9.0/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.9.0/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.9.0/security/tls.md b/site/docs/4.9.0/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.9.0/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.9.0/security/zookeeper.md b/site/docs/4.9.0/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.9.0/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) diff --git a/site/docs/latest/overview/overview.md b/site/docs/latest/overview/overview.md index 66452dc96d0..189520d8a76 100644 --- a/site/docs/latest/overview/overview.md +++ b/site/docs/latest/overview/overview.md @@ -1,5 +1,5 @@ --- -title: Apache BookKeeper™ 4.9.0-SNAPSHOT +title: Apache BookKeeper™ 4.10.0-SNAPSHOT --- +Proposal PR - #abc \ No newline at end of file diff --git a/.github/ISSUE_TEMPLATE/bug_report.md b/.github/ISSUE_TEMPLATE/bug_report.md new file mode 100644 index 00000000000..3021fbfe02c --- /dev/null +++ b/.github/ISSUE_TEMPLATE/bug_report.md @@ -0,0 +1,34 @@ +--- +name: Bug report +about: Create a bug report to help us improve +title: '' +labels: type/bug +assignees: '' + +--- + +**BUG REPORT** + +***Describe the bug*** + +A clear and concise description of what the bug is. + +***To Reproduce*** + +Steps to reproduce the behavior: +1. Go to '...' +2. Click on '....' +3. Scroll down to '....' +4. See error + +***Expected behavior*** + +A clear and concise description of what you expected to happen. + +***Screenshots*** + +If applicable, add screenshots to help explain your problem. + +***Additional context*** + +Add any other context about the problem here. diff --git a/.github/ISSUE_TEMPLATE/feature_request.md b/.github/ISSUE_TEMPLATE/feature_request.md new file mode 100644 index 00000000000..931a2e4bcca --- /dev/null +++ b/.github/ISSUE_TEMPLATE/feature_request.md @@ -0,0 +1,18 @@ +--- +name: Feature request +about: Suggest an idea for this project +title: '' +labels: type/feature +assignees: '' + +--- + +**FEATURE REQUEST** + +1. Please describe the feature you are requesting. + +2. Indicate the importance of this issue to you (blocker, must-have, should-have, nice-to-have). + Are you currently using any workarounds to address this issue? + +3. Provide any additional detail on your proposed use case for this feature. + diff --git a/.github/ISSUE_TEMPLATE/question.md b/.github/ISSUE_TEMPLATE/question.md new file mode 100644 index 00000000000..afa05872e83 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/question.md @@ -0,0 +1,14 @@ +--- +name: Question +about: Ask the community for answering questions of BookKeeper +labels: type/question +assignees: '' + +--- + +**QUESTION** + +Have you checked our documentation at http://bookkeeper.apache.org/? +If you could not find an answer there, please consider asking your question +in our community mailing list at user@bookkeeper.apache.org, as it would benefit +other members of our community. diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 064f988e857..1e1991bd86e 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -26,7 +26,7 @@ Master Issue: # > - [ ] [skip bookkeeper-server remaining tests]: skip testing all other tests in bookkeeper-server module. > - [ ] [skip integration tests]: skip docker based integration tests. if you make java code changes, you shouldn't skip integration tests. > - [ ] [skip build java8]: skip build on java8. *ONLY* skip this when *ONLY* changing files under documentation under `site`. -> - [ ] [skip build java9]: skip build on java9. *ONLY* skip this when *ONLY* changing files under documentation under `site`. +> - [ ] [skip build java11]: skip build on java11. *ONLY* skip this when *ONLY* changing files under documentation under `site`. > --- > --- From 35db5af6ec16a151d6f28e77e754bda01b512eb8 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sat, 9 Feb 2019 09:33:56 +0800 Subject: [PATCH 0211/1642] Bump Netty and GRPC version Descriptions of the changes in this PR: *Motivation* When we bumped Netty's version, we didn't bump the gRPC version. So the gRPC version has become very old. *Modifications* Bump both gRPC and Netty version to make them aligned with each other. Reviewers: Enrico Olivelli This closes #1934 from sijie/bump_grpc_version --- bookkeeper-dist/src/assemble/bin-all.xml | 6 +- bookkeeper-dist/src/assemble/bin-server.xml | 4 +- bookkeeper-dist/src/assemble/bkctl.xml | 4 +- .../src/main/resources/LICENSE-all.bin.txt | 103 ++--- .../src/main/resources/LICENSE-bkctl.bin.txt | 97 ++-- .../src/main/resources/LICENSE-server.bin.txt | 101 +++-- .../src/main/resources/NOTICE-all.bin.txt | 54 +-- .../src/main/resources/NOTICE-bkctl.bin.txt | 48 +- .../src/main/resources/NOTICE-server.bin.txt | 52 +-- .../deps/checker-compat-qual-2.5.2/LICENSE | 413 ++++++++++++++++++ .../LICENSE.base64.txt | 0 .../LICENSE.jbzip2.txt | 0 .../LICENSE.jfastlz.txt | 0 .../LICENSE.jsr166y.txt | 0 .../LICENSE.libdivsufsort.txt | 0 .../LICENSE.protobuf.txt | 0 .../LICENSE.slf4j.txt | 0 pom.xml | 6 +- .../bookkeeper/clients/utils/GrpcUtils.java | 9 +- 19 files changed, 664 insertions(+), 233 deletions(-) create mode 100644 bookkeeper-dist/src/main/resources/deps/checker-compat-qual-2.5.2/LICENSE rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.31.Final => netty-4.1.32.Final}/LICENSE.base64.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.31.Final => netty-4.1.32.Final}/LICENSE.jbzip2.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.31.Final => netty-4.1.32.Final}/LICENSE.jfastlz.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.31.Final => netty-4.1.32.Final}/LICENSE.jsr166y.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.31.Final => netty-4.1.32.Final}/LICENSE.libdivsufsort.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.31.Final => netty-4.1.32.Final}/LICENSE.protobuf.txt (100%) rename bookkeeper-dist/src/main/resources/deps/{netty-4.1.31.Final => netty-4.1.32.Final}/LICENSE.slf4j.txt (100%) diff --git a/bookkeeper-dist/src/assemble/bin-all.xml b/bookkeeper-dist/src/assemble/bin-all.xml index 9e11c8ef939..6e638eda756 100644 --- a/bookkeeper-dist/src/assemble/bin-all.xml +++ b/bookkeeper-dist/src/assemble/bin-all.xml @@ -54,14 +54,15 @@ ../src/main/resources/deps deps + checker-compat-qual-2.5.2/LICENSE google-auth-library-credentials-0.9.0/LICENSE javax.servlet-api-3.1.0/CDDL+GPL-1.1 + jline-2.11/LICENSE jsr-305/LICENSE netty-3.10.1.Final/* - netty-4.1.31.Final/* + netty-4.1.32.Final/* paranamer-2.8/LICENSE.txt protobuf-3.0.0/LICENSE - jline-2.11/LICENSE protobuf-3.5.1/LICENSE scala-library-2.11.7/LICENSE.md scala-parser-combinators_2.11-1.0.4/LICENSE.md @@ -105,6 +106,7 @@ org.apache.bookkeeper:stream-storage-api org.apache.bookkeeper:stream-storage-java-client-base org.apache.bookkeeper:stream-storage-java-kv-client + org.codehaus.mojo:animal-sniffer-annotations diff --git a/bookkeeper-dist/src/assemble/bin-server.xml b/bookkeeper-dist/src/assemble/bin-server.xml index ee88fe0d179..57cf40bcc71 100644 --- a/bookkeeper-dist/src/assemble/bin-server.xml +++ b/bookkeeper-dist/src/assemble/bin-server.xml @@ -49,9 +49,10 @@ ../src/main/resources/deps deps + checker-compat-qual-2.5.2/LICENSE google-auth-library-credentials-0.9.0/LICENSE javax.servlet-api-3.1.0/CDDL+GPL-1.1 - netty-4.1.31.Final/* + netty-4.1.32.Final/* protobuf-3.0.0/LICENSE jline-2.11/LICENSE protobuf-3.5.1/LICENSE @@ -95,6 +96,7 @@ org.apache.bookkeeper:stream-storage-api org.apache.bookkeeper:stream-storage-java-client-base org.apache.bookkeeper:stream-storage-java-kv-client + org.codehaus.mojo:animal-sniffer-annotations diff --git a/bookkeeper-dist/src/assemble/bkctl.xml b/bookkeeper-dist/src/assemble/bkctl.xml index 439c344d700..4f2d92fa994 100644 --- a/bookkeeper-dist/src/assemble/bkctl.xml +++ b/bookkeeper-dist/src/assemble/bkctl.xml @@ -64,8 +64,9 @@ ../src/main/resources/deps deps + checker-compat-qual-2.5.2/LICENSE google-auth-library-credentials-0.9.0/LICENSE - netty-4.1.31.Final/* + netty-4.1.32.Final/* protobuf-3.0.0/LICENSE jline-2.11/LICENSE protobuf-3.5.1/LICENSE @@ -108,6 +109,7 @@ org.apache.bookkeeper:stream-storage-proto org.apache.bookkeeper:stream-storage-api org.apache.bookkeeper:stream-storage-java-client-base + org.codehaus.mojo:animal-sniffer-annotations org.rocksdb:rocksdbjni diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index a1863d11912..fc52da9b103 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -250,22 +250,22 @@ Apache Software License, Version 2. - lib/io.dropwizard.metrics-metrics-graphite-3.1.0.jar [21] - lib/io.dropwizard.metrics-metrics-jvm-3.1.0.jar [21] - lib/io.netty-netty-3.10.1.Final.jar [22] -- lib/io.netty-netty-buffer-4.1.31.Final.jar [23] -- lib/io.netty-netty-codec-4.1.31.Final.jar [23] -- lib/io.netty-netty-codec-dns-4.1.31.Final.jar [23] -- lib/io.netty-netty-codec-http-4.1.31.Final.jar [23] -- lib/io.netty-netty-codec-http2-4.1.31.Final.jar [23] -- lib/io.netty-netty-codec-socks-4.1.31.Final.jar [23] -- lib/io.netty-netty-common-4.1.31.Final.jar [23] -- lib/io.netty-netty-handler-4.1.31.Final.jar [23] -- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar [23] -- lib/io.netty-netty-resolver-4.1.31.Final.jar [23] -- lib/io.netty-netty-resolver-dns-4.1.31.Final.jar [23] -- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar [23] -- lib/io.netty-netty-transport-4.1.31.Final.jar [23] -- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar [23] -- lib/io.netty-netty-transport-native-epoll-4.1.31.Final-linux-x86_64.jar [23] -- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar [23] +- lib/io.netty-netty-buffer-4.1.32.Final.jar [23] +- lib/io.netty-netty-codec-4.1.32.Final.jar [23] +- lib/io.netty-netty-codec-dns-4.1.32.Final.jar [23] +- lib/io.netty-netty-codec-http-4.1.32.Final.jar [23] +- lib/io.netty-netty-codec-http2-4.1.32.Final.jar [23] +- lib/io.netty-netty-codec-socks-4.1.32.Final.jar [23] +- lib/io.netty-netty-common-4.1.32.Final.jar [23] +- lib/io.netty-netty-handler-4.1.32.Final.jar [23] +- lib/io.netty-netty-handler-proxy-4.1.32.Final.jar [23] +- lib/io.netty-netty-resolver-4.1.32.Final.jar [23] +- lib/io.netty-netty-resolver-dns-4.1.32.Final.jar [23] +- lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar [23] +- lib/io.netty-netty-transport-4.1.32.Final.jar [23] +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar [23] +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final-linux-x86_64.jar [23] +- lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar [23] - lib/io.prometheus-simpleclient-0.0.21.jar [24] - lib/io.prometheus-simpleclient_common-0.0.21.jar [24] - lib/io.prometheus-simpleclient_hotspot-0.0.21.jar [24] @@ -291,23 +291,23 @@ Apache Software License, Version 2. - lib/net.jpountz.lz4-lz4-1.3.0.jar [38] - lib/org.codehaus.jackson-jackson-core-asl-1.9.11.jar [39] - lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [39] -- lib/com.google.api.grpc-proto-google-common-protos-1.0.0.jar [40] +- lib/com.google.api.grpc-proto-google-common-protos-1.12.0.jar [40] - lib/com.google.code.gson-gson-2.7.jar [41] -- lib/io.opencensus-opencensus-api-0.11.0.jar [42] -- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.11.0.jar [42] +- lib/io.opencensus-opencensus-api-0.18.0.jar [42] +- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.18.0.jar [42] - lib/com.squareup.okhttp-okhttp-2.5.0.jar [43] - lib/com.squareup.okio-okio-1.13.0.jar [44] -- lib/io.grpc-grpc-all-1.12.0.jar [45] -- lib/io.grpc-grpc-auth-1.12.0.jar [45] -- lib/io.grpc-grpc-context-1.12.0.jar [45] -- lib/io.grpc-grpc-core-1.12.0.jar [45] -- lib/io.grpc-grpc-netty-1.12.0.jar [45] -- lib/io.grpc-grpc-okhttp-1.12.0.jar [45] -- lib/io.grpc-grpc-protobuf-1.12.0.jar [45] -- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar [45] -- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar [45] -- lib/io.grpc-grpc-stub-1.12.0.jar [45] -- lib/io.grpc-grpc-testing-1.12.0.jar [45] +- lib/io.grpc-grpc-all-1.18.0.jar [45] +- lib/io.grpc-grpc-auth-1.18.0.jar [45] +- lib/io.grpc-grpc-context-1.18.0.jar [45] +- lib/io.grpc-grpc-core-1.18.0.jar [45] +- lib/io.grpc-grpc-netty-1.18.0.jar [45] +- lib/io.grpc-grpc-okhttp-1.18.0.jar [45] +- lib/io.grpc-grpc-protobuf-1.18.0.jar [45] +- lib/io.grpc-grpc-protobuf-lite-1.18.0.jar [45] +- lib/io.grpc-grpc-protobuf-nano-1.18.0.jar [45] +- lib/io.grpc-grpc-stub-1.18.0.jar [45] +- lib/io.grpc-grpc-testing-1.18.0.jar [45] - lib/org.apache.curator-curator-client-4.0.1.jar [46] - lib/org.apache.curator-curator-framework-4.0.1.jar [46] - lib/org.apache.curator-curator-recipes-4.0.1.jar [46] @@ -338,7 +338,7 @@ Apache Software License, Version 2. [20] Source available at https://github.com/twitter/util/tree/util-6.43.0 [21] Source available at https://github.com/dropwizard/metrics/tree/v3.1.0 [22] Source available at https://bintray.com/netty/downloads/download_file?file_path=netty-3.10.1.Final-dist.tar.bz2 -[23] Source available at https://github.com/netty/netty/tree/netty-4.1.22.Final +[23] Source available at https://github.com/netty/netty/tree/netty-4.1.32.Final [24] Source available at https://github.com/prometheus/client_java/tree/parent-0.0.21 [25] Source available at https://github.com/vert-x3/vertx-auth/tree/3.4.1 [26] Source available at https://github.com/eclipse/vert.x/tree/3.4.1 @@ -356,10 +356,10 @@ Apache Software License, Version 2. [39] Source available at https://github.com/codehaus/jackson/tree/d1c26ec538a8701cff7548009254fd6183b71873 [40] Source available at https://github.com/googleapis/googleapis [41] Source available at https://github.com/google/gson/tree/gson-parent-2.7 -[42] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.11.0 +[42] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.18.0 [43] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 [44] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 -[45] Source available at https://github.com/grpc/grpc-java/tree/v1.12.0 +[45] Source available at https://github.com/grpc/grpc-java/tree/v1.18.0 [46] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 [47] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 [48] Source available at https://github.com/google/error-prone/tree/v2.1.2 @@ -402,63 +402,63 @@ WebSocket and HTTP server: * https://github.com/joewalnes/webbit ------------------------------------------------------------------------------------ -lib/io.netty-netty-codec-4.1.31.Final.jar bundles some 3rd party dependencies +lib/io.netty-netty-codec-4.1.32.Final.jar bundles some 3rd party dependencies -lib/io.netty-netty-codec-4.1.31.Final.jar contains the extensions to Java Collections Framework which has +lib/io.netty-netty-codec-4.1.32.Final.jar contains the extensions to Java Collections Framework which has been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.jsr166y.txt (Public Domain) + * deps/netty-4.1.32.Final/LICENSE.jsr166y.txt (Public Domain) * HOMEPAGE: * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified version of Robert Harder's Public Domain +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified version of Robert Harder's Public Domain Base64 Encoder and Decoder, which can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.base64.txt (Public Domain) + * deps/netty-4.1.32.Final/LICENSE.base64.txt (Public Domain) * HOMEPAGE: * http://iharder.sourceforge.net/current/java/base64/ -lib/io.netty-netty-common-4.1.31.Final.jar contains a modified portion of 'SLF4J', a simple logging +lib/io.netty-netty-common-4.1.32.Final.jar contains a modified portion of 'SLF4J', a simple logging facade for Java, which can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.slf4j.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.slf4j.txt (MIT License) * HOMEPAGE: * http://www.slf4j.org/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression and decompression library written by Matthew J. Francis. It can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.jbzip2.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.jbzip2.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jbzip2/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct the suffix array and the Burrows-Wheeler transformed string for any input string of a constant-size alphabet written by Yuta Mori. It can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.libdivsufsort.txt (MIT License) * HOMEPAGE: * https://github.com/y-256/libdivsufsort -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression and decompression library written by William Kinney. It can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.jfastlz.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.jfastlz.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jfastlz/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data interchange format, which can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.protobuf.txt (New BSD License) + * deps/netty-4.1.32.Final/LICENSE.protobuf.txt (New BSD License) * HOMEPAGE: * https://github.com/google/protobuf @@ -505,7 +505,6 @@ license. Bundled as - lib/com.google.protobuf-protobuf-java-3.5.1.jar - - lib/com.google.protobuf-protobuf-java-util-3.5.1.jar Source available at https://github.com/google/protobuf/tree/v3.5.1 For details, see deps/protobuf-3.5.1/LICENSE. @@ -563,3 +562,9 @@ license. For details, see deps/jline-2.11/LICENSE Bundled as - lib/jline-jline-2.11.jar +------------------------------------------------------------------------------------ +This product uses the annotations from The Checker Framework, which are licensed under +MIT License. For details, see deps/checker-compat-qual-2.5.2/LICENSE + +Bundles as + - lib/org.checkerframework-checker-compat-qual-2.5.2.jar diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt index 692e9a0e5cd..aedc7e3e41d 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -215,19 +215,19 @@ Apache Software License, Version 2. - lib/commons-io-commons-io-2.4.jar [8] - lib/commons-lang-commons-lang-2.6.jar [9] - lib/commons-logging-commons-logging-1.1.1.jar [10] -- lib/io.netty-netty-buffer-4.1.31.Final.jar [11] -- lib/io.netty-netty-codec-4.1.31.Final.jar [11] -- lib/io.netty-netty-codec-http-4.1.31.Final.jar [11] -- lib/io.netty-netty-codec-http2-4.1.31.Final.jar [11] -- lib/io.netty-netty-codec-socks-4.1.31.Final.jar [11] -- lib/io.netty-netty-common-4.1.31.Final.jar [11] -- lib/io.netty-netty-handler-4.1.31.Final.jar [11] -- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar [11] -- lib/io.netty-netty-resolver-4.1.31.Final.jar [11] -- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar [11] -- lib/io.netty-netty-transport-4.1.31.Final.jar [11] -- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar [11] -- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar [11] +- lib/io.netty-netty-buffer-4.1.32.Final.jar [11] +- lib/io.netty-netty-codec-4.1.32.Final.jar [11] +- lib/io.netty-netty-codec-http-4.1.32.Final.jar [11] +- lib/io.netty-netty-codec-http2-4.1.32.Final.jar [11] +- lib/io.netty-netty-codec-socks-4.1.32.Final.jar [11] +- lib/io.netty-netty-common-4.1.32.Final.jar [11] +- lib/io.netty-netty-handler-4.1.32.Final.jar [11] +- lib/io.netty-netty-handler-proxy-4.1.32.Final.jar [11] +- lib/io.netty-netty-resolver-4.1.32.Final.jar [11] +- lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar [11] +- lib/io.netty-netty-transport-4.1.32.Final.jar [11] +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar [11] +- lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar [11] - lib/log4j-log4j-1.2.17.jar [16] - lib/net.java.dev.jna-jna-3.2.7.jar [17] - lib/org.apache.commons-commons-collections4-4.1.jar [18] @@ -235,23 +235,23 @@ Apache Software License, Version 2. - lib/org.apache.zookeeper-zookeeper-3.4.13.jar [20] - lib/com.beust-jcommander-1.48.jar [23] - lib/net.jpountz.lz4-lz4-1.3.0.jar [25] -- lib/com.google.api.grpc-proto-google-common-protos-1.0.0.jar [27] +- lib/com.google.api.grpc-proto-google-common-protos-1.12.0.jar [27] - lib/com.google.code.gson-gson-2.7.jar [28] -- lib/io.opencensus-opencensus-api-0.11.0.jar [29] -- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.11.0.jar [29] +- lib/io.opencensus-opencensus-api-0.18.0.jar [29] +- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.18.0.jar [29] - lib/com.squareup.okhttp-okhttp-2.5.0.jar [30] - lib/com.squareup.okio-okio-1.13.0.jar [31] -- lib/io.grpc-grpc-all-1.12.0.jar [32] -- lib/io.grpc-grpc-auth-1.12.0.jar [32] -- lib/io.grpc-grpc-context-1.12.0.jar [32] -- lib/io.grpc-grpc-core-1.12.0.jar [32] -- lib/io.grpc-grpc-netty-1.12.0.jar [32] -- lib/io.grpc-grpc-okhttp-1.12.0.jar [32] -- lib/io.grpc-grpc-protobuf-1.12.0.jar [32] -- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar [32] -- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar [32] -- lib/io.grpc-grpc-stub-1.12.0.jar [32] -- lib/io.grpc-grpc-testing-1.12.0.jar [32] +- lib/io.grpc-grpc-all-1.18.0.jar [32] +- lib/io.grpc-grpc-auth-1.18.0.jar [32] +- lib/io.grpc-grpc-context-1.18.0.jar [32] +- lib/io.grpc-grpc-core-1.18.0.jar [32] +- lib/io.grpc-grpc-netty-1.18.0.jar [32] +- lib/io.grpc-grpc-okhttp-1.18.0.jar [32] +- lib/io.grpc-grpc-protobuf-1.18.0.jar [32] +- lib/io.grpc-grpc-protobuf-lite-1.18.0.jar [32] +- lib/io.grpc-grpc-protobuf-nano-1.18.0.jar [32] +- lib/io.grpc-grpc-stub-1.18.0.jar [32] +- lib/io.grpc-grpc-testing-1.18.0.jar [32] - lib/org.apache.curator-curator-client-4.0.1.jar [33] - lib/org.apache.curator-curator-framework-4.0.1.jar [33] - lib/org.apache.curator-curator-recipes-4.0.1.jar [33] @@ -270,7 +270,7 @@ Apache Software License, Version 2. [8] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-io.git;a=tag;h=603579 [9] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=tag;h=375459 [10] Source available at http://svn.apache.org/viewvc/commons/proper/logging/tags/commons-logging-1.1.1/ -[11] Source available at https://github.com/netty/netty/tree/netty-4.1.31.Final +[11] Source available at https://github.com/netty/netty/tree/netty-4.1.32.Final [16] Source available at http://logging.apache.org/log4j/1.2/download.html [17] Source available at https://github.com/java-native-access/jna/tree/3.2.7 [18] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad @@ -280,10 +280,10 @@ Apache Software License, Version 2. [25] Source available at https://github.com/lz4/lz4-java/tree/1.3.0 [27] Source available at https://github.com/googleapis/googleapis [28] Source available at https://github.com/google/gson/tree/gson-parent-2.7 -[29] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.11.0 +[29] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.18.0 [30] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 [31] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 -[32] Source available at https://github.com/grpc/grpc-java/tree/v1.12.0 +[32] Source available at https://github.com/grpc/grpc-java/tree/v1.18.0 [33] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 [34] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 [35] Source available at https://github.com/google/error-prone/tree/v2.1.2 @@ -291,63 +291,63 @@ Apache Software License, Version 2. [37] Source available at https://github.com/JCTools/JCTools/tree/v2.1.2 ------------------------------------------------------------------------------------ -lib/io.netty-netty-codec-4.1.31.Final.jar bundles some 3rd party dependencies +lib/io.netty-netty-codec-4.1.32.Final.jar bundles some 3rd party dependencies -lib/io.netty-netty-codec-4.1.31.Final.jar contains the extensions to Java Collections Framework which has +lib/io.netty-netty-codec-4.1.32.Final.jar contains the extensions to Java Collections Framework which has been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.jsr166y.txt (Public Domain) + * deps/netty-4.1.32.Final/LICENSE.jsr166y.txt (Public Domain) * HOMEPAGE: * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified version of Robert Harder's Public Domain +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified version of Robert Harder's Public Domain Base64 Encoder and Decoder, which can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.base64.txt (Public Domain) + * deps/netty-4.1.32.Final/LICENSE.base64.txt (Public Domain) * HOMEPAGE: * http://iharder.sourceforge.net/current/java/base64/ -lib/io.netty-netty-common-4.1.31.Final.jar contains a modified portion of 'SLF4J', a simple logging +lib/io.netty-netty-common-4.1.32.Final.jar contains a modified portion of 'SLF4J', a simple logging facade for Java, which can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.slf4j.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.slf4j.txt (MIT License) * HOMEPAGE: * http://www.slf4j.org/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression and decompression library written by Matthew J. Francis. It can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.jbzip2.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.jbzip2.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jbzip2/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct the suffix array and the Burrows-Wheeler transformed string for any input string of a constant-size alphabet written by Yuta Mori. It can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.libdivsufsort.txt (MIT License) * HOMEPAGE: * https://github.com/y-256/libdivsufsort -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression and decompression library written by William Kinney. It can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.jfastlz.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.jfastlz.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jfastlz/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data interchange format, which can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.protobuf.txt (New BSD License) + * deps/netty-4.1.32.Final/LICENSE.protobuf.txt (New BSD License) * HOMEPAGE: * https://github.com/google/protobuf @@ -357,7 +357,6 @@ license. Bundled as - lib/com.google.protobuf-protobuf-java-3.5.1.jar - - lib/com.google.protobuf-protobuf-java-util-3.5.1.jar Source available at https://github.com/google/protobuf/tree/v3.5.1 For details, see deps/protobuf-3.5.1/LICENSE. @@ -385,3 +384,9 @@ license. For details, see deps/jline-2.11/LICENSE Bundled as - lib/jline-jline-2.11.jar +------------------------------------------------------------------------------------ +This product uses the annotations from The Checker Framework, which are licensed under +MIT License. For details, see deps/checker-compat-qual-2.5.2/LICENSE + +Bundles as + - lib/org.checkerframework-checker-compat-qual-2.5.2.jar diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index a8b5b884752..44c1e470786 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -215,21 +215,21 @@ Apache Software License, Version 2. - lib/commons-io-commons-io-2.4.jar [8] - lib/commons-lang-commons-lang-2.6.jar [9] - lib/commons-logging-commons-logging-1.1.1.jar [10] -- lib/io.netty-netty-buffer-4.1.31.Final.jar [11] -- lib/io.netty-netty-codec-4.1.31.Final.jar [11] -- lib/io.netty-netty-codec-dns-4.1.31.Final.jar [11] -- lib/io.netty-netty-codec-http-4.1.31.Final.jar [11] -- lib/io.netty-netty-codec-http2-4.1.31.Final.jar [11] -- lib/io.netty-netty-codec-socks-4.1.31.Final.jar [11] -- lib/io.netty-netty-common-4.1.31.Final.jar [11] -- lib/io.netty-netty-handler-4.1.31.Final.jar [11] -- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar [11] -- lib/io.netty-netty-resolver-4.1.31.Final.jar [11] -- lib/io.netty-netty-resolver-dns-4.1.31.Final.jar [11] -- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar [11] -- lib/io.netty-netty-transport-4.1.31.Final.jar [11] -- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar [11] -- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar [11] +- lib/io.netty-netty-buffer-4.1.32.Final.jar [11] +- lib/io.netty-netty-codec-4.1.32.Final.jar [11] +- lib/io.netty-netty-codec-dns-4.1.32.Final.jar [11] +- lib/io.netty-netty-codec-http-4.1.32.Final.jar [11] +- lib/io.netty-netty-codec-http2-4.1.32.Final.jar [11] +- lib/io.netty-netty-codec-socks-4.1.32.Final.jar [11] +- lib/io.netty-netty-common-4.1.32.Final.jar [11] +- lib/io.netty-netty-handler-4.1.32.Final.jar [11] +- lib/io.netty-netty-handler-proxy-4.1.32.Final.jar [11] +- lib/io.netty-netty-resolver-4.1.32.Final.jar [11] +- lib/io.netty-netty-resolver-dns-4.1.32.Final.jar [11] +- lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar [11] +- lib/io.netty-netty-transport-4.1.32.Final.jar [11] +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar [11] +- lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar [11] - lib/io.prometheus-simpleclient-0.0.21.jar [12] - lib/io.prometheus-simpleclient_common-0.0.21.jar [12] - lib/io.prometheus-simpleclient_hotspot-0.0.21.jar [12] @@ -255,23 +255,23 @@ Apache Software License, Version 2. - lib/net.jpountz.lz4-lz4-1.3.0.jar [25] - lib/org.codehaus.jackson-jackson-core-asl-1.9.11.jar [26] - lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [26] -- lib/com.google.api.grpc-proto-google-common-protos-1.0.0.jar [27] +- lib/com.google.api.grpc-proto-google-common-protos-1.12.0.jar [27] - lib/com.google.code.gson-gson-2.7.jar [28] -- lib/io.opencensus-opencensus-api-0.11.0.jar [29] -- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.11.0.jar [29] +- lib/io.opencensus-opencensus-api-0.18.0.jar [29] +- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.18.0.jar [29] - lib/com.squareup.okhttp-okhttp-2.5.0.jar [30] - lib/com.squareup.okio-okio-1.13.0.jar [31] -- lib/io.grpc-grpc-all-1.12.0.jar [32] -- lib/io.grpc-grpc-auth-1.12.0.jar [32] -- lib/io.grpc-grpc-context-1.12.0.jar [32] -- lib/io.grpc-grpc-core-1.12.0.jar [32] -- lib/io.grpc-grpc-netty-1.12.0.jar [32] -- lib/io.grpc-grpc-okhttp-1.12.0.jar [32] -- lib/io.grpc-grpc-protobuf-1.12.0.jar [32] -- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar [32] -- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar [32] -- lib/io.grpc-grpc-stub-1.12.0.jar [32] -- lib/io.grpc-grpc-testing-1.12.0.jar [32] +- lib/io.grpc-grpc-all-1.18.0.jar [32] +- lib/io.grpc-grpc-auth-1.18.0.jar [32] +- lib/io.grpc-grpc-context-1.18.0.jar [32] +- lib/io.grpc-grpc-core-1.18.0.jar [32] +- lib/io.grpc-grpc-netty-1.18.0.jar [32] +- lib/io.grpc-grpc-okhttp-1.18.0.jar [32] +- lib/io.grpc-grpc-protobuf-1.18.0.jar [32] +- lib/io.grpc-grpc-protobuf-lite-1.18.0.jar [32] +- lib/io.grpc-grpc-protobuf-nano-1.18.0.jar [32] +- lib/io.grpc-grpc-stub-1.18.0.jar [32] +- lib/io.grpc-grpc-testing-1.18.0.jar [32] - lib/org.apache.curator-curator-client-4.0.1.jar [33] - lib/org.apache.curator-curator-framework-4.0.1.jar [33] - lib/org.apache.curator-curator-recipes-4.0.1.jar [33] @@ -290,7 +290,7 @@ Apache Software License, Version 2. [8] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-io.git;a=tag;h=603579 [9] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=tag;h=375459 [10] Source available at http://svn.apache.org/viewvc/commons/proper/logging/tags/commons-logging-1.1.1/ -[11] Source available at https://github.com/netty/netty/tree/netty-4.1.31.Final +[11] Source available at https://github.com/netty/netty/tree/netty-4.1.32.Final [12] Source available at https://github.com/prometheus/client_java/tree/parent-0.0.21 [13] Source available at https://github.com/vert-x3/vertx-auth/tree/3.4.1 [14] Source available at https://github.com/eclipse/vert.x/tree/3.4.1 @@ -308,10 +308,10 @@ Apache Software License, Version 2. [26] Source available at https://github.com/codehaus/jackson/tree/d1c26ec538a8701cff7548009254fd6183b71873 [27] Source available at https://github.com/googleapis/googleapis [28] Source available at https://github.com/google/gson/tree/gson-parent-2.7 -[29] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.11.0 +[29] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.18.0 [30] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 [31] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 -[32] Source available at https://github.com/grpc/grpc-java/tree/v1.12.0 +[32] Source available at https://github.com/grpc/grpc-java/tree/v1.18.0 [33] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 [34] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 [35] Source available at https://github.com/google/error-prone/tree/v2.1.2 @@ -319,63 +319,63 @@ Apache Software License, Version 2. [37] Source available at https://github.com/JCTools/JCTools/tree/v2.1.2 ------------------------------------------------------------------------------------ -lib/io.netty-netty-codec-4.1.31.Final.jar bundles some 3rd party dependencies +lib/io.netty-netty-codec-4.1.32.Final.jar bundles some 3rd party dependencies -lib/io.netty-netty-codec-4.1.31.Final.jar contains the extensions to Java Collections Framework which has +lib/io.netty-netty-codec-4.1.32.Final.jar contains the extensions to Java Collections Framework which has been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.jsr166y.txt (Public Domain) + * deps/netty-4.1.32.Final/LICENSE.jsr166y.txt (Public Domain) * HOMEPAGE: * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified version of Robert Harder's Public Domain +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified version of Robert Harder's Public Domain Base64 Encoder and Decoder, which can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.base64.txt (Public Domain) + * deps/netty-4.1.32.Final/LICENSE.base64.txt (Public Domain) * HOMEPAGE: * http://iharder.sourceforge.net/current/java/base64/ -lib/io.netty-netty-common-4.1.31.Final.jar contains a modified portion of 'SLF4J', a simple logging +lib/io.netty-netty-common-4.1.32.Final.jar contains a modified portion of 'SLF4J', a simple logging facade for Java, which can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.slf4j.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.slf4j.txt (MIT License) * HOMEPAGE: * http://www.slf4j.org/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of 'jbzip2', a Java bzip2 compression and decompression library written by Matthew J. Francis. It can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.jbzip2.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.jbzip2.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jbzip2/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of 'libdivsufsort', a C API library to construct the suffix array and the Burrows-Wheeler transformed string for any input string of a constant-size alphabet written by Yuta Mori. It can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.libdivsufsort.txt (MIT License) * HOMEPAGE: * https://github.com/y-256/libdivsufsort -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of 'jfastlz', a Java port of FastLZ compression and decompression library written by William Kinney. It can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.jfastlz.txt (MIT License) + * deps/netty-4.1.32.Final/LICENSE.jfastlz.txt (MIT License) * HOMEPAGE: * https://code.google.com/p/jfastlz/ -lib/io.netty-netty-codec-4.1.31.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data +lib/io.netty-netty-codec-4.1.32.Final.jar contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data interchange format, which can be obtained at: * LICENSE: - * deps/netty-4.1.31.Final/LICENSE.protobuf.txt (New BSD License) + * deps/netty-4.1.32.Final/LICENSE.protobuf.txt (New BSD License) * HOMEPAGE: * https://github.com/google/protobuf @@ -416,7 +416,6 @@ license. Bundled as - lib/com.google.protobuf-protobuf-java-3.5.1.jar - - lib/com.google.protobuf-protobuf-java-util-3.5.1.jar Source available at https://github.com/google/protobuf/tree/v3.5.1 For details, see deps/protobuf-3.5.1/LICENSE. @@ -450,3 +449,9 @@ license. For details, see deps/jline-2.11/LICENSE Bundled as - lib/jline-jline-2.11.jar +------------------------------------------------------------------------------------ +This product uses the annotations from The Checker Framework, which are licensed under +MIT License. For details, see deps/checker-compat-qual-2.5.2/LICENSE + +Bundles as + - lib/org.checkerframework-checker-compat-qual-2.5.2.jar diff --git a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt index c9a025560c4..6916fda9a0c 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt @@ -45,22 +45,22 @@ License for the specific language governing permissions and limitations under the License. ------------------------------------------------------------------------------------ -- lib/io.netty-netty-buffer-4.1.31.Final.jar -- lib/io.netty-netty-codec-4.1.31.Final.jar -- lib/io.netty-netty-codec-dns-4.1.31.Final.jar -- lib/io.netty-netty-codec-http-4.1.31.Final.jar -- lib/io.netty-netty-codec-http2-4.1.31.Final.jar -- lib/io.netty-netty-codec-socks-4.1.31.Final.jar -- lib/io.netty-netty-common-4.1.31.Final.jar -- lib/io.netty-netty-handler-4.1.31.Final.jar -- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar -- lib/io.netty-netty-resolver-4.1.31.Final.jar -- lib/io.netty-netty-resolver-dns-4.1.31.Final.jar -- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar -- lib/io.netty-netty-transport-4.1.31.Final.jar -- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar -- lib/io.netty-netty-transport-native-epoll-4.1.31.Final-linux-x86_64.jar -- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar +- lib/io.netty-netty-buffer-4.1.32.Final.jar +- lib/io.netty-netty-codec-4.1.32.Final.jar +- lib/io.netty-netty-codec-dns-4.1.32.Final.jar +- lib/io.netty-netty-codec-http-4.1.32.Final.jar +- lib/io.netty-netty-codec-http2-4.1.32.Final.jar +- lib/io.netty-netty-codec-socks-4.1.32.Final.jar +- lib/io.netty-netty-common-4.1.32.Final.jar +- lib/io.netty-netty-handler-4.1.32.Final.jar +- lib/io.netty-netty-handler-proxy-4.1.32.Final.jar +- lib/io.netty-netty-resolver-4.1.32.Final.jar +- lib/io.netty-netty-resolver-dns-4.1.32.Final.jar +- lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar +- lib/io.netty-netty-transport-4.1.32.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final-linux-x86_64.jar +- lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar The Netty Project @@ -143,17 +143,17 @@ granted provided that the copyright notice appears in all copies. Copyright 2010 Cedric Beust cedric@beust.com ------------------------------------------------------------------------------------ -- lib/io.grpc-grpc-all-1.12.0.jar -- lib/io.grpc-grpc-auth-1.12.0.jar -- lib/io.grpc-grpc-context-1.12.0.jar -- lib/io.grpc-grpc-core-1.12.0.jar -- lib/io.grpc-grpc-netty-1.12.0.jar -- lib/io.grpc-grpc-okhttp-1.12.0.jar -- lib/io.grpc-grpc-protobuf-1.12.0.jar -- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar -- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar -- lib/io.grpc-grpc-stub-1.12.0.jar -- lib/io.grpc-grpc-testing-1.12.0.jar +- lib/io.grpc-grpc-all-1.18.0.jar +- lib/io.grpc-grpc-auth-1.18.0.jar +- lib/io.grpc-grpc-context-1.18.0.jar +- lib/io.grpc-grpc-core-1.18.0.jar +- lib/io.grpc-grpc-netty-1.18.0.jar +- lib/io.grpc-grpc-okhttp-1.18.0.jar +- lib/io.grpc-grpc-protobuf-1.18.0.jar +- lib/io.grpc-grpc-protobuf-lite-1.18.0.jar +- lib/io.grpc-grpc-protobuf-nano-1.18.0.jar +- lib/io.grpc-grpc-stub-1.18.0.jar +- lib/io.grpc-grpc-testing-1.18.0.jar Copyright 2014, gRPC Authors All rights reserved. diff --git a/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt index 680a02dca82..d7f93303f70 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt @@ -5,19 +5,19 @@ This product includes software developed at The Apache Software Foundation (http://www.apache.org/). ------------------------------------------------------------------------------------ -- lib/io.netty-netty-buffer-4.1.31.Final.jar -- lib/io.netty-netty-codec-4.1.31.Final.jar -- lib/io.netty-netty-codec-http-4.1.31.Final.jar -- lib/io.netty-netty-codec-http2-4.1.31.Final.jar -- lib/io.netty-netty-codec-socks-4.1.31.Final.jar -- lib/io.netty-netty-common-4.1.31.Final.jar -- lib/io.netty-netty-handler-4.1.31.Final.jar -- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar -- lib/io.netty-netty-resolver-4.1.31.Final.jar -- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar -- lib/io.netty-netty-transport-4.1.31.Final.jar -- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar -- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar +- lib/io.netty-netty-buffer-4.1.32.Final.jar +- lib/io.netty-netty-codec-4.1.32.Final.jar +- lib/io.netty-netty-codec-http-4.1.32.Final.jar +- lib/io.netty-netty-codec-http2-4.1.32.Final.jar +- lib/io.netty-netty-codec-socks-4.1.32.Final.jar +- lib/io.netty-netty-common-4.1.32.Final.jar +- lib/io.netty-netty-handler-4.1.32.Final.jar +- lib/io.netty-netty-handler-proxy-4.1.32.Final.jar +- lib/io.netty-netty-resolver-4.1.32.Final.jar +- lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar +- lib/io.netty-netty-transport-4.1.32.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar +- lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar The Netty Project @@ -47,17 +47,17 @@ under the License. Copyright 2010 Cedric Beust cedric@beust.com ------------------------------------------------------------------------------------ -- lib/io.grpc-grpc-all-1.12.0.jar -- lib/io.grpc-grpc-auth-1.12.0.jar -- lib/io.grpc-grpc-context-1.12.0.jar -- lib/io.grpc-grpc-core-1.12.0.jar -- lib/io.grpc-grpc-netty-1.12.0.jar -- lib/io.grpc-grpc-okhttp-1.12.0.jar -- lib/io.grpc-grpc-protobuf-1.12.0.jar -- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar -- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar -- lib/io.grpc-grpc-stub-1.12.0.jar -- lib/io.grpc-grpc-testing-1.12.0.jar +- lib/io.grpc-grpc-all-1.18.0.jar +- lib/io.grpc-grpc-auth-1.18.0.jar +- lib/io.grpc-grpc-context-1.18.0.jar +- lib/io.grpc-grpc-core-1.18.0.jar +- lib/io.grpc-grpc-netty-1.18.0.jar +- lib/io.grpc-grpc-okhttp-1.18.0.jar +- lib/io.grpc-grpc-protobuf-1.18.0.jar +- lib/io.grpc-grpc-protobuf-lite-1.18.0.jar +- lib/io.grpc-grpc-protobuf-nano-1.18.0.jar +- lib/io.grpc-grpc-stub-1.18.0.jar +- lib/io.grpc-grpc-testing-1.18.0.jar Copyright 2014, gRPC Authors All rights reserved. diff --git a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt index c2dcf1d75d2..cc2060e871a 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt @@ -5,21 +5,21 @@ This product includes software developed at The Apache Software Foundation (http://www.apache.org/). ------------------------------------------------------------------------------------ -- lib/io.netty-netty-buffer-4.1.31.Final.jar -- lib/io.netty-netty-codec-4.1.31.Final.jar -- lib/io.netty-netty-codec-dns-4.1.31.Final.jar -- lib/io.netty-netty-codec-http-4.1.31.Final.jar -- lib/io.netty-netty-codec-http2-4.1.31.Final.jar -- lib/io.netty-netty-codec-socks-4.1.31.Final.jar -- lib/io.netty-netty-common-4.1.31.Final.jar -- lib/io.netty-netty-handler-4.1.31.Final.jar -- lib/io.netty-netty-handler-proxy-4.1.31.Final.jar -- lib/io.netty-netty-resolver-4.1.31.Final.jar -- lib/io.netty-netty-resolver-dns-4.1.31.Final.jar -- lib/io.netty-netty-tcnative-boringssl-static-2.0.19.Final.jar -- lib/io.netty-netty-transport-4.1.31.Final.jar -- lib/io.netty-netty-transport-native-epoll-4.1.31.Final.jar -- lib/io.netty-netty-transport-native-unix-common-4.1.31.Final.jar +- lib/io.netty-netty-buffer-4.1.32.Final.jar +- lib/io.netty-netty-codec-4.1.32.Final.jar +- lib/io.netty-netty-codec-dns-4.1.32.Final.jar +- lib/io.netty-netty-codec-http-4.1.32.Final.jar +- lib/io.netty-netty-codec-http2-4.1.32.Final.jar +- lib/io.netty-netty-codec-socks-4.1.32.Final.jar +- lib/io.netty-netty-common-4.1.32.Final.jar +- lib/io.netty-netty-handler-4.1.32.Final.jar +- lib/io.netty-netty-handler-proxy-4.1.32.Final.jar +- lib/io.netty-netty-resolver-4.1.32.Final.jar +- lib/io.netty-netty-resolver-dns-4.1.32.Final.jar +- lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar +- lib/io.netty-netty-transport-4.1.32.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar +- lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar The Netty Project @@ -102,17 +102,17 @@ granted provided that the copyright notice appears in all copies. Copyright 2010 Cedric Beust cedric@beust.com ------------------------------------------------------------------------------------ -- lib/io.grpc-grpc-all-1.12.0.jar -- lib/io.grpc-grpc-auth-1.12.0.jar -- lib/io.grpc-grpc-context-1.12.0.jar -- lib/io.grpc-grpc-core-1.12.0.jar -- lib/io.grpc-grpc-netty-1.12.0.jar -- lib/io.grpc-grpc-okhttp-1.12.0.jar -- lib/io.grpc-grpc-protobuf-1.12.0.jar -- lib/io.grpc-grpc-protobuf-lite-1.12.0.jar -- lib/io.grpc-grpc-protobuf-nano-1.12.0.jar -- lib/io.grpc-grpc-stub-1.12.0.jar -- lib/io.grpc-grpc-testing-1.12.0.jar +- lib/io.grpc-grpc-all-1.18.0.jar +- lib/io.grpc-grpc-auth-1.18.0.jar +- lib/io.grpc-grpc-context-1.18.0.jar +- lib/io.grpc-grpc-core-1.18.0.jar +- lib/io.grpc-grpc-netty-1.18.0.jar +- lib/io.grpc-grpc-okhttp-1.18.0.jar +- lib/io.grpc-grpc-protobuf-1.18.0.jar +- lib/io.grpc-grpc-protobuf-lite-1.18.0.jar +- lib/io.grpc-grpc-protobuf-nano-1.18.0.jar +- lib/io.grpc-grpc-stub-1.18.0.jar +- lib/io.grpc-grpc-testing-1.18.0.jar Copyright 2014, gRPC Authors All rights reserved. diff --git a/bookkeeper-dist/src/main/resources/deps/checker-compat-qual-2.5.2/LICENSE b/bookkeeper-dist/src/main/resources/deps/checker-compat-qual-2.5.2/LICENSE new file mode 100644 index 00000000000..7d677403f06 --- /dev/null +++ b/bookkeeper-dist/src/main/resources/deps/checker-compat-qual-2.5.2/LICENSE @@ -0,0 +1,413 @@ +The Checker Framework +Copyright 2004-present by the Checker Framework developers + + +Most of the Checker Framework is licensed under the GNU General Public +License, version 2 (GPL2), with the classpath exception. The text of this +license appears below. This is the same license used for OpenJDK. + +A few parts of the Checker Framework have more permissive licenses. + + * The annotations are licensed under the MIT License. (The text of this + license appears below.) More specifically, all the parts of the Checker + Framework that you might want to include with your own program use the + MIT License. This is the checker-qual.jar file and all the files that + appear in it: every file in a qual/ directory, plus utility files such + as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc. + In addition, the cleanroom implementations of third-party annotations, + which the Checker Framework recognizes as aliases for its own + annotations, are licensed under the MIT License. + +Some external libraries that are included with the Checker Framework have +different licenses. + + * javaparser is dual licensed under the LGPL or the Apache license -- you + may use it under whichever one you want. (The javaparser source code + contains a file with the text of the GPL, but it is not clear why, since + javaparser does not use the GPL.) See file stubparser/LICENSE + and the source code of all its files. + + * JUnit is licensed under the Common Public License v1.0 (see + http://www.junit.org/license), with parts (Hamcrest) licensed under the + BSD License (see http://hamcrest.org/JavaHamcrest/). + + * Libraries in plume-lib (https://github.com/plume-lib/) are licensed + under the MIT License. + +The Checker Framework includes annotations for the JDK in directory +checker/jdk/, and for some other libraries. Each annotated library uses +the same license as the unannotated version of the library. + +=========================================================================== + +The GNU General Public License (GPL) + +Version 2, June 1991 + +Copyright (C) 1989, 1991 Free Software Foundation, Inc. +59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + +Everyone is permitted to copy and distribute verbatim copies of this license +document, but changing it is not allowed. + +Preamble + +The licenses for most software are designed to take away your freedom to share +and change it. By contrast, the GNU General Public License is intended to +guarantee your freedom to share and change free software--to make sure the +software is free for all its users. This General Public License applies to +most of the Free Software Foundation's software and to any other program whose +authors commit to using it. (Some other Free Software Foundation software is +covered by the GNU Library General Public License instead.) You can apply it to +your programs, too. + +When we speak of free software, we are referring to freedom, not price. Our +General Public Licenses are designed to make sure that you have the freedom to +distribute copies of free software (and charge for this service if you wish), +that you receive source code or can get it if you want it, that you can change +the software or use pieces of it in new free programs; and that you know you +can do these things. + +To protect your rights, we need to make restrictions that forbid anyone to deny +you these rights or to ask you to surrender the rights. These restrictions +translate to certain responsibilities for you if you distribute copies of the +software, or if you modify it. + +For example, if you distribute copies of such a program, whether gratis or for +a fee, you must give the recipients all the rights that you have. You must +make sure that they, too, receive or can get the source code. And you must +show them these terms so they know their rights. + +We protect your rights with two steps: (1) copyright the software, and (2) +offer you this license which gives you legal permission to copy, distribute +and/or modify the software. + +Also, for each author's protection and ours, we want to make certain that +everyone understands that there is no warranty for this free software. If the +software is modified by someone else and passed on, we want its recipients to +know that what they have is not the original, so that any problems introduced +by others will not reflect on the original authors' reputations. + +Finally, any free program is threatened constantly by software patents. We +wish to avoid the danger that redistributors of a free program will +individually obtain patent licenses, in effect making the program proprietary. +To prevent this, we have made it clear that any patent must be licensed for +everyone's free use or not licensed at all. + +The precise terms and conditions for copying, distribution and modification +follow. + +TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + +0. This License applies to any program or other work which contains a notice +placed by the copyright holder saying it may be distributed under the terms of +this General Public License. The "Program", below, refers to any such program +or work, and a "work based on the Program" means either the Program or any +derivative work under copyright law: that is to say, a work containing the +Program or a portion of it, either verbatim or with modifications and/or +translated into another language. (Hereinafter, translation is included +without limitation in the term "modification".) Each licensee is addressed as +"you". + +Activities other than copying, distribution and modification are not covered by +this License; they are outside its scope. The act of running the Program is +not restricted, and the output from the Program is covered only if its contents +constitute a work based on the Program (independent of having been made by +running the Program). Whether that is true depends on what the Program does. + +1. You may copy and distribute verbatim copies of the Program's source code as +you receive it, in any medium, provided that you conspicuously and +appropriately publish on each copy an appropriate copyright notice and +disclaimer of warranty; keep intact all the notices that refer to this License +and to the absence of any warranty; and give any other recipients of the +Program a copy of this License along with the Program. + +You may charge a fee for the physical act of transferring a copy, and you may +at your option offer warranty protection in exchange for a fee. + +2. You may modify your copy or copies of the Program or any portion of it, thus +forming a work based on the Program, and copy and distribute such modifications +or work under the terms of Section 1 above, provided that you also meet all of +these conditions: + + a) You must cause the modified files to carry prominent notices stating + that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in whole or + in part contains or is derived from the Program or any part thereof, to be + licensed as a whole at no charge to all third parties under the terms of + this License. + + c) If the modified program normally reads commands interactively when run, + you must cause it, when started running for such interactive use in the + most ordinary way, to print or display an announcement including an + appropriate copyright notice and a notice that there is no warranty (or + else, saying that you provide a warranty) and that users may redistribute + the program under these conditions, and telling the user how to view a copy + of this License. (Exception: if the Program itself is interactive but does + not normally print such an announcement, your work based on the Program is + not required to print an announcement.) + +These requirements apply to the modified work as a whole. If identifiable +sections of that work are not derived from the Program, and can be reasonably +considered independent and separate works in themselves, then this License, and +its terms, do not apply to those sections when you distribute them as separate +works. But when you distribute the same sections as part of a whole which is a +work based on the Program, the distribution of the whole must be on the terms +of this License, whose permissions for other licensees extend to the entire +whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest your +rights to work written entirely by you; rather, the intent is to exercise the +right to control the distribution of derivative or collective works based on +the Program. + +In addition, mere aggregation of another work not based on the Program with the +Program (or with a work based on the Program) on a volume of a storage or +distribution medium does not bring the other work under the scope of this +License. + +3. You may copy and distribute the Program (or a work based on it, under +Section 2) in object code or executable form under the terms of Sections 1 and +2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable source + code, which must be distributed under the terms of Sections 1 and 2 above + on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three years, to + give any third party, for a charge no more than your cost of physically + performing source distribution, a complete machine-readable copy of the + corresponding source code, to be distributed under the terms of Sections 1 + and 2 above on a medium customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer to + distribute corresponding source code. (This alternative is allowed only + for noncommercial distribution and only if you received the program in + object code or executable form with such an offer, in accord with + Subsection b above.) + +The source code for a work means the preferred form of the work for making +modifications to it. For an executable work, complete source code means all +the source code for all modules it contains, plus any associated interface +definition files, plus the scripts used to control compilation and installation +of the executable. However, as a special exception, the source code +distributed need not include anything that is normally distributed (in either +source or binary form) with the major components (compiler, kernel, and so on) +of the operating system on which the executable runs, unless that component +itself accompanies the executable. + +If distribution of executable or object code is made by offering access to copy +from a designated place, then offering equivalent access to copy the source +code from the same place counts as distribution of the source code, even though +third parties are not compelled to copy the source along with the object code. + +4. You may not copy, modify, sublicense, or distribute the Program except as +expressly provided under this License. Any attempt otherwise to copy, modify, +sublicense or distribute the Program is void, and will automatically terminate +your rights under this License. However, parties who have received copies, or +rights, from you under this License will not have their licenses terminated so +long as such parties remain in full compliance. + +5. You are not required to accept this License, since you have not signed it. +However, nothing else grants you permission to modify or distribute the Program +or its derivative works. These actions are prohibited by law if you do not +accept this License. Therefore, by modifying or distributing the Program (or +any work based on the Program), you indicate your acceptance of this License to +do so, and all its terms and conditions for copying, distributing or modifying +the Program or works based on it. + +6. Each time you redistribute the Program (or any work based on the Program), +the recipient automatically receives a license from the original licensor to +copy, distribute or modify the Program subject to these terms and conditions. +You may not impose any further restrictions on the recipients' exercise of the +rights granted herein. You are not responsible for enforcing compliance by +third parties to this License. + +7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), conditions +are imposed on you (whether by court order, agreement or otherwise) that +contradict the conditions of this License, they do not excuse you from the +conditions of this License. If you cannot distribute so as to satisfy +simultaneously your obligations under this License and any other pertinent +obligations, then as a consequence you may not distribute the Program at all. +For example, if a patent license would not permit royalty-free redistribution +of the Program by all those who receive copies directly or indirectly through +you, then the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Program. + +If any portion of this section is held invalid or unenforceable under any +particular circumstance, the balance of the section is intended to apply and +the section as a whole is intended to apply in other circumstances. + +It is not the purpose of this section to induce you to infringe any patents or +other property right claims or to contest validity of any such claims; this +section has the sole purpose of protecting the integrity of the free software +distribution system, which is implemented by public license practices. Many +people have made generous contributions to the wide range of software +distributed through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing to +distribute software through any other system and a licensee cannot impose that +choice. + +This section is intended to make thoroughly clear what is believed to be a +consequence of the rest of this License. + +8. If the distribution and/or use of the Program is restricted in certain +countries either by patents or by copyrighted interfaces, the original +copyright holder who places the Program under this License may add an explicit +geographical distribution limitation excluding those countries, so that +distribution is permitted only in or among countries not thus excluded. In +such case, this License incorporates the limitation as if written in the body +of this License. + +9. The Free Software Foundation may publish revised and/or new versions of the +General Public License from time to time. Such new versions will be similar in +spirit to the present version, but may differ in detail to address new problems +or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any later +version", you have the option of following the terms and conditions either of +that version or of any later version published by the Free Software Foundation. +If the Program does not specify a version number of this License, you may +choose any version ever published by the Free Software Foundation. + +10. If you wish to incorporate parts of the Program into other free programs +whose distribution conditions are different, write to the author to ask for +permission. For software which is copyrighted by the Free Software Foundation, +write to the Free Software Foundation; we sometimes make exceptions for this. +Our decision will be guided by the two goals of preserving the free status of +all derivatives of our free software and of promoting the sharing and reuse of +software generally. + +NO WARRANTY + +11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR +THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE +STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE +PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND +PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, +YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION. + +12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL +ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE +PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY +GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR +INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA +BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A +FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER +OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +END OF TERMS AND CONDITIONS + +How to Apply These Terms to Your New Programs + +If you develop a new program, and you want it to be of the greatest possible +use to the public, the best way to achieve this is to make it free software +which everyone can redistribute and change under these terms. + +To do so, attach the following notices to the program. It is safest to attach +them to the start of each source file to most effectively convey the exclusion +of warranty; and each file should have at least the "copyright" line and a +pointer to where the full notice is found. + + One line to give the program's name and a brief idea of what it does. + + Copyright (C) + + This program is free software; you can redistribute it and/or modify it + under the terms of the GNU General Public License as published by the Free + Software Foundation; either version 2 of the License, or (at your option) + any later version. + + This program is distributed in the hope that it will be useful, but WITHOUT + ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for + more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., 59 + Temple Place, Suite 330, Boston, MA 02111-1307 USA + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this when it +starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author Gnomovision comes + with ABSOLUTELY NO WARRANTY; for details type 'show w'. This is free + software, and you are welcome to redistribute it under certain conditions; + type 'show c' for details. + +The hypothetical commands 'show w' and 'show c' should show the appropriate +parts of the General Public License. Of course, the commands you use may be +called something other than 'show w' and 'show c'; they could even be +mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your school, +if any, to sign a "copyright disclaimer" for the program, if necessary. Here +is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the program + 'Gnomovision' (which makes passes at compilers) written by James Hacker. + + signature of Ty Coon, 1 April 1989 + + Ty Coon, President of Vice + +This General Public License does not permit incorporating your program into +proprietary programs. If your program is a subroutine library, you may +consider it more useful to permit linking proprietary applications with the +library. If this is what you want to do, use the GNU Library General Public +License instead of this License. + + +"CLASSPATH" EXCEPTION TO THE GPL + +Certain source files distributed by Oracle America and/or its affiliates are +subject to the following clarification and special exception to the GPL, but +only where Oracle has expressly included in the particular source file's header +the words "Oracle designates this particular file as subject to the "Classpath" +exception as provided by Oracle in the LICENSE file that accompanied this code." + + Linking this library statically or dynamically with other modules is making + a combined work based on this library. Thus, the terms and conditions of + the GNU General Public License cover the whole combination. + + As a special exception, the copyright holders of this library give you + permission to link this library with independent modules to produce an + executable, regardless of the license terms of these independent modules, + and to copy and distribute the resulting executable under terms of your + choice, provided that you also meet, for each linked independent module, + the terms and conditions of the license of that module. An independent + module is a module which is not derived from or based on this library. If + you modify this library, you may extend this exception to your version of + the library, but you are not obligated to do so. If you do not wish to do + so, delete this exception statement from your version. + +=========================================================================== + +MIT License: + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +=========================================================================== diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.base64.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.base64.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.base64.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.base64.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jbzip2.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.jbzip2.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jbzip2.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.jbzip2.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jfastlz.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.jfastlz.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jfastlz.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.jfastlz.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jsr166y.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.jsr166y.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.jsr166y.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.jsr166y.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.libdivsufsort.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.libdivsufsort.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.libdivsufsort.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.protobuf.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.protobuf.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.protobuf.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.protobuf.txt diff --git a/bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.slf4j.txt b/bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.slf4j.txt similarity index 100% rename from bookkeeper-dist/src/main/resources/deps/netty-4.1.31.Final/LICENSE.slf4j.txt rename to bookkeeper-dist/src/main/resources/deps/netty-4.1.32.Final/LICENSE.slf4j.txt diff --git a/pom.xml b/pom.xml index 935f0f44877..fddb3badc83 100644 --- a/pom.xml +++ b/pom.xml @@ -130,7 +130,7 @@ 1.14.9 3.0.2 2.1.2 - 1.12.0 + 1.18.0 21.0 1.1.1 2.7.3 @@ -150,8 +150,8 @@ 1.16.22 1.3.0 2.22.0 - 4.1.31.Final - 2.0.19.Final + 4.1.32.Final + 2.0.20.Final 9.1.3 2.0.0-beta.5 0.0.21 diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcUtils.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcUtils.java index e1d396a9bb2..02a202fa9e9 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcUtils.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcUtils.java @@ -20,10 +20,8 @@ import static org.apache.bookkeeper.clients.utils.ClientConstants.TOKEN; -import io.grpc.Attributes; -import io.grpc.CallCredentials; +import io.grpc.CallCredentials2; import io.grpc.Metadata; -import io.grpc.MethodDescriptor; import io.grpc.stub.AbstractStub; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -50,10 +48,9 @@ public static > T configureGrpcStub(T stub, Optional tokenKey = Metadata.Key.of(TOKEN, Metadata.ASCII_STRING_MARSHALLER); metadata.put(tokenKey, t); - CallCredentials callCredentials = new CallCredentials() { + CallCredentials2 callCredentials = new CallCredentials2() { @Override - public void applyRequestMetadata(MethodDescriptor method, - Attributes attrs, + public void applyRequestMetadata(RequestInfo requestInfo, Executor appExecutor, MetadataApplier applier) { applier.apply(metadata); From 249e3a117aecaa779db7363c2e7ec2249b26161d Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 8 Feb 2019 17:36:30 -0800 Subject: [PATCH 0212/1642] fix invalid formatter xml ### Motivation `formatter.xml` is an invalid xml and eclipse fails to load it. Reviewers: Enrico Olivelli , Sijie Guo This closes #1939 from rdhabalia/format_xml --- buildtools/src/main/resources/ide/eclipse/formatter.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildtools/src/main/resources/ide/eclipse/formatter.xml b/buildtools/src/main/resources/ide/eclipse/formatter.xml index b8648e0702a..01b73dcd996 100644 --- a/buildtools/src/main/resources/ide/eclipse/formatter.xml +++ b/buildtools/src/main/resources/ide/eclipse/formatter.xml @@ -1,3 +1,4 @@ + - From 08a13b433f6393f1840a35b315f91a1aa0cf8bb4 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sat, 9 Feb 2019 09:38:26 +0800 Subject: [PATCH 0213/1642] [WEBSITE] Update release schedule for 4.10.0 Reviewers: Jia Zhai This closes #1932 from sijie/update_release_schedule --- site/community/releases.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/site/community/releases.md b/site/community/releases.md index ca8f409bc8e..557f7ce2598 100644 --- a/site/community/releases.md +++ b/site/community/releases.md @@ -15,19 +15,19 @@ Apache BookKeeper community makes a feture release every 3 month. ### Feature Release Window -The next feature release is `4.8.0`. The release window is the following: +The next feature release is `4.10.0`. The release window is the following: | **Date** | **Event** | -| April 13, 2017 | Merge window opens on master branch | -| July 12, 2018 | Major feature should be in, Cut release branch | -| July 19, 2018 | Minor feature should be in, Stabilize release branch | -| July 26, 2018 - August 2, 2018 | Code freeze, Only accept fixes for blocking issues, Rolling out release candidates | +| February 1, 2019 | Merge window opens on master branch | +| March 12, 2019 | Major feature should be in, Cut release branch | +| March 19, 2019 | Minor feature should be in, Stabilize release branch | +| March 26, 2019 | Code freeze, Only accept fixes for blocking issues, Rolling out release candidates | ## Release Schedule -- **4.7.0**: November 2017 - April 2018 -- **4.8.0**: April 2018 - July 2018 -- **4.9.0**: July 2018 - October 2018 -- **4.10.0**: October 2018 - January 2019 +- **4.10.0**: February 2019 - March 2019 +- **4.11.0**: April 2019 - May 2019 +- **4.12.0**: June 2019 - July 2019 +- **4.13.0**: August 2019 - September 2019 From 102d595d6c8608f07c25d84ab8a5a74de365377e Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 13 Feb 2019 10:31:51 +0800 Subject: [PATCH 0214/1642] [DOC] Add python client release instructions to release guide Reviewers: Jia Zhai , Enrico Olivelli This closes #1942 from sijie/update_instructions_on_publish_pythonclient --- site/community/release_guide.md | 56 +++++++++++++++++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/site/community/release_guide.md b/site/community/release_guide.md index 70aefe09567..bb5834dda15 100644 --- a/site/community/release_guide.md +++ b/site/community/release_guide.md @@ -134,6 +134,17 @@ Configure access to the [Apache Nexus repository](http://repository.apache.org/) +#### Create an account on PyPi + +Since 4.9.0 we are releasing a python client for table service during release process. In order to publishing +a python package to PyPi, you need to [create an account](https://pypi.org/account/register/) there. After +you create the account successfully, you also need to add the account as a maintainer +for [bookkeeper-client](https://pypi.org/project/apache-bookkeeper-client/) project. You can checkout who +are the maintainers at the project page and ask them for adding your account as the maintainer. + +You can also read the instructions on [how to upload packages to PyPi](https://twine.readthedocs.io/en/latest/) +if you are interested in learning more details. + ### Create a new version in Github When contributors resolve an issue in GitHub, they are tagging it with a release that will contain their changes. With the release currently underway, new issues should be resolved against a subsequent future release. Therefore, you should create a release item for this subsequent release, as follows: @@ -153,6 +164,13 @@ The list of release-blocking issues is available at the [milestones page](https: * If the issue has not been resolved and it is acceptable to defer this until the next release, update the `Milestone` field to the new milestone you just created. Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate. * If the issue has not been resolved and it is not acceptable to release until it is fixed, the release cannot proceed. Instead, work with the BookKeeper community to resolve the issue. +### Change Python Client Version + +Before cutting a release, you need to update the python client version in +[setup.py](https://github.com/apache/bookkeeper/blob/master/stream/clients/python/setup.py#L22) +from `SNAPSHOT` version to a release version and get the change merge to master. For example, +in release 4.10.0, you need to change the version from `4.10.0-SNAPSHOT` to `4.10.0`. + ### Review Release Notes in Github > Github does not automatically generates Release Notes based on the `Milestone` field applied to issues. @@ -556,6 +574,36 @@ done 2. Once the new docker image is built, update BC tests to include new docker image. Example: [release-4.7.0](https://github.com/apache/bookkeeper/pull/1352) +### Release Python Client + +Make sure you have installed [`pip`](https://pypi.org/project/pip/) and +[`twine`](https://twine.readthedocs.io/en/latest/). + +- Install Pip + ```bash + brew install pip + ``` + +- Install Twine + ```bash + pip install twine + ``` + +After install `twine`, make sure `twine` exist in your PATH before releasing python client. + +```bash +twine --version +``` + +Now, you are ready to publish the python client. + +```bash +cd stream/clients/python +./publish.sh +``` + +Check the PyPi project package to make sure the python client is uploaded to https://pypi.org/project/apache-bookkeeper-client/ . + ### Advance version on release branch > only do this for minor release @@ -573,6 +621,14 @@ Then you have to create a PR and submit it for review. Example PR: [release-4.7.0](https://github.com/apache/bookkeeper/pull/1350) +### Advance python client version + +If you are doing a major release, you need to update the python client version to next major development version in master +and next minor development version in the branch. For example, if you are doing 4.9.0 release, you need to bump the version +in master to `4.10.0-SNAPSHOT`, and the version in `branch-4.9` to `4.9.1-SNAPSHOT`. + +If you are only doing a minor release, you just need to update the version in release branch. For example, if you are doing +4.9.1 release, you need to bump the version in `branch-4.9` to `4.9.2-SNAPSHOT`. ### Mark the version as released in Github From ccbbb5509a9342ad51fb2f010ac0c16f345ac753 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sat, 16 Feb 2019 06:59:54 -0800 Subject: [PATCH 0215/1642] [BK-GC] Fix GC thread gets blocked ### Motivation It addresses below thread-stuck while performing gc in bookie. ``` Thread 3363: (state = BLOCKED) - java.lang.Object.wait(long) bci=0 (Compiled frame; information may be imprecise) - java.lang.Object.wait() bci=2, line=502 (Compiled frame) - org.apache.bookkeeper.util.ZkUtils.getChildrenInSingleNode(org.apache.zookeeper.ZooKeeper, java.lang.String) bci=34, line=243 (Compiled frame) - org.apache.bookkeeper.meta.LongHierarchicalLedgerManager$LongHierarchicalLedgerRangeIterator.getChildrenAt(java.lang.String) bci=8, line=165 (Compiled fr ame) - org.apache.bookkeeper.meta.LongHierarchicalLedgerManager$LongHierarchicalLedgerRangeIterator$LeafIterator.(org.apache.bookkeeper.meta.LongHierarchic alLedgerManager$LongHierarchicalLedgerRangeIterator, java.lang.String) bci=11, line=187 (Compiled frame) - org.apache.bookkeeper.meta.LongHierarchicalLedgerManager$LongHierarchicalLedgerRangeIterator$InnerIterator.advance() bci=137, line=261 (Compiled frame) - org.apache.bookkeeper.meta.LongHierarchicalLedgerManager$LongHierarchicalLedgerRangeIterator$InnerIterator.next() bci=28, line=281 (Compiled frame) - org.apache.bookkeeper.meta.LongHierarchicalLedgerManager$LongHierarchicalLedgerRangeIterator$InnerIterator.next() bci=4, line=278 (Compiled frame) - org.apache.bookkeeper.meta.LongHierarchicalLedgerManager$LongHierarchicalLedgerRangeIterator$InnerIterator.next() bci=4, line=278 (Compiled frame) - org.apache.bookkeeper.meta.LongHierarchicalLedgerManager$LongHierarchicalLedgerRangeIterator$InnerIterator.next() bci=4, line=278 (Compiled frame) - org.apache.bookkeeper.meta.LongHierarchicalLedgerManager$LongHierarchicalLedgerRangeIterator.next() bci=8, line=304 (Compiled frame) - org.apache.bookkeeper.meta.HierarchicalLedgerManager$HierarchicalLedgerRangeIterator.next() bci=26, line=117 (Compiled frame) - org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.gc(org.apache.bookkeeper.bookie.GarbageCollector$GarbageCleaner) bci=195, line=168 (Compiled frame) - org.apache.bookkeeper.bookie.GarbageCollectorThread.doGcLedgers() bci=8, line=393 (Compiled frame) - org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(boolean, boolean, boolean) bci=39, line=355 (Compiled frame) - org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun() bci=28, line=333 (Compiled frame) - org.apache.bookkeeper.common.util.SafeRunnable.run() bci=1, line=36 (Compiled frame) - java.util.concurrent.Executors$RunnableAdapter.call() bci=4, line=511 (Compiled frame) - java.util.concurrent.FutureTask.runAndReset() bci=47, line=308 (Compiled frame) - java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask) bci=1, line=180 (Compiled frame) - java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run() bci=37, line=294 (Compiled frame) - java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker) bci=95, line=1142 (Compiled frame) - java.util.concurrent.ThreadPoolExecutor$Worker.run() bci=5, line=617 (Interpreted frame) - io.netty.util.concurrent.FastThreadLocalRunnable.run() bci=4, line=30 (Interpreted frame) - java.lang.Thread.run() bci=11, line=748 (Interpreted frame) ``` ### Changes add time-out to zk operation to avoid GC thread blocking. Reviewers: Enrico Olivelli , Sijie Guo This closes #1937 from rdhabalia/gc_dead --- .../ScanAndCompareGarbageCollector.java | 3 ++- .../bookkeeper/client/BookKeeperAdmin.java | 2 +- .../bookkeeper/meta/CleanupLedgerManager.java | 4 ++-- .../bookkeeper/meta/FlatLedgerManager.java | 5 +++-- .../meta/HierarchicalLedgerManager.java | 6 +++--- .../apache/bookkeeper/meta/LedgerManager.java | 5 ++++- .../meta/LegacyHierarchicalLedgerManager.java | 11 +++++++--- .../meta/LongHierarchicalLedgerManager.java | 11 ++++++---- .../meta/MSLedgerManagerFactory.java | 2 +- .../http/service/ListLedgerService.java | 2 +- .../org/apache/bookkeeper/util/ZkUtils.java | 10 ++++++++-- .../bookkeeper/bookie/CompactionTest.java | 2 +- .../client/ParallelLedgerRecoveryTest.java | 4 ++-- .../apache/bookkeeper/meta/GcLedgersTest.java | 6 +++--- .../meta/LedgerManagerIteratorTest.java | 20 +++++++++---------- .../bookkeeper/meta/MockLedgerManager.java | 2 +- .../metadata/etcd/EtcdLedgerManager.java | 2 +- .../metadata/etcd/EtcdLedgerManagerTest.java | 2 +- 18 files changed, 59 insertions(+), 40 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index 24c5c97e9cc..b1e77229c13 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -139,7 +139,8 @@ public void gc(GarbageCleaner garbageCleaner) { } // Iterate over all the ledger on the metadata store - LedgerRangeIterator ledgerRangeIterator = ledgerManager.getLedgerRanges(); + long zkOpTimeout = this.conf.getZkTimeout() * 2; + LedgerRangeIterator ledgerRangeIterator = ledgerManager.getLedgerRanges(zkOpTimeout); Set ledgersInMetadata = null; long start; long end = -1; 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 37b59d189f5..9303ffdff8c 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 @@ -1320,7 +1320,7 @@ private static boolean validateDirectoriesAreEmpty(File[] dirs, String typeOfDir */ public Iterable listLedgers() throws IOException { - final LedgerRangeIterator iterator = bkc.getLedgerManager().getLedgerRanges(); + final LedgerRangeIterator iterator = bkc.getLedgerManager().getLedgerRanges(0); return new Iterable() { public Iterator iterator() { return new Iterator() { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java index 25c5aca1c6e..3c7ddc51d30 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java @@ -210,13 +210,13 @@ public void processResult(int rc, String path, Object ctx) { } @Override - public LedgerRangeIterator getLedgerRanges() { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { closeLock.readLock().lock(); try { if (closed) { return new ClosedLedgerRangeIterator(); } - return underlying.getLedgerRanges(); + return underlying.getLedgerRanges(zkOpTimeoutSec); } finally { closeLock.readLock().unlock(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java index 7ee2e2289ea..8c7f05f9f7f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java @@ -89,7 +89,7 @@ public void asyncProcessLedgers(final Processor processor, } @Override - public LedgerRangeIterator getLedgerRanges() { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeOutSec) { return new LedgerRangeIterator() { // single iterator, can visit only one time boolean nextCalled = false; @@ -103,7 +103,8 @@ private synchronized void preload() throws IOException { try { zkActiveLedgers = ledgerListToSet( - ZkUtils.getChildrenInSingleNode(zk, ledgerRootPath), ledgerRootPath); + ZkUtils.getChildrenInSingleNode(zk, ledgerRootPath, zkOpTimeOutSec), + ledgerRootPath); nextRange = new LedgerRange(zkActiveLedgers); } catch (KeeperException.NoNodeException e) { throw new IOException("Path does not exist: " + ledgerRootPath, e); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java index 946ed2a5d32..079fcfacd25 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java @@ -87,9 +87,9 @@ protected long getLedgerId(String ledgerPath) throws IOException { } @Override - public LedgerRangeIterator getLedgerRanges() { - LedgerRangeIterator legacyLedgerRangeIterator = legacyLM.getLedgerRanges(); - LedgerRangeIterator longLedgerRangeIterator = longLM.getLedgerRanges(); + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { + LedgerRangeIterator legacyLedgerRangeIterator = legacyLM.getLedgerRanges(zkOpTimeoutSec); + LedgerRangeIterator longLedgerRangeIterator = longLM.getLedgerRanges(zkOpTimeoutSec); return new HierarchicalLedgerRangeIterator(legacyLedgerRangeIterator, longLedgerRangeIterator); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java index cc7630b10b8..039ff7ccbde 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java @@ -149,9 +149,12 @@ void asyncProcessLedgers(Processor processor, AsyncCallback.VoidCallback f /** * Loop to scan a range of metadata from metadata storage. * + * @param zkOpTimeOutSec + * Iterator considers timeout while fetching ledger-range from + * zk. * @return will return a iterator of the Ranges */ - LedgerRangeIterator getLedgerRanges(); + LedgerRangeIterator getLedgerRanges(long zkOpTimeOutSec); /** * Used to represent the Ledgers range returned from the diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java index 76ecc9d68de..1a63407e7ad 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java @@ -153,8 +153,8 @@ protected String getLedgerParentNodeRegex() { } @Override - public LedgerRangeIterator getLedgerRanges() { - return new LegacyHierarchicalLedgerRangeIterator(); + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { + return new LegacyHierarchicalLedgerRangeIterator(zkOpTimeoutSec); } /** @@ -166,6 +166,11 @@ private class LegacyHierarchicalLedgerRangeIterator implements LedgerRangeIterat private String curL1Nodes = ""; private boolean iteratorDone = false; private LedgerRange nextRange = null; + private final long zkOpTimeoutSec; + + public LegacyHierarchicalLedgerRangeIterator(long zkOpTimeoutSec) { + this.zkOpTimeoutSec = zkOpTimeoutSec; + } /** * Iterate next level1 znode. @@ -261,7 +266,7 @@ LedgerRange getLedgerRangeByLevel(final String level1, final String level2) String nodePath = nodeBuilder.toString(); List ledgerNodes = null; try { - ledgerNodes = ZkUtils.getChildrenInSingleNode(zk, nodePath); + ledgerNodes = ZkUtils.getChildrenInSingleNode(zk, nodePath, zkOpTimeoutSec); } catch (KeeperException.NoNodeException e) { /* If the node doesn't exist, we must have raced with a recursive node removal, just * return an empty list. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java index 2e69e90a5c1..95f8e48bde8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java @@ -139,8 +139,8 @@ public void process(String lNode, VoidCallback cb) { } @Override - public LedgerRangeIterator getLedgerRanges() { - return new LongHierarchicalLedgerRangeIterator(); + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { + return new LongHierarchicalLedgerRangeIterator(zkOpTimeoutSec); } @@ -149,6 +149,7 @@ public LedgerRangeIterator getLedgerRanges() { */ private class LongHierarchicalLedgerRangeIterator implements LedgerRangeIterator { LedgerRangeIterator rootIterator; + final long zkOpTimeoutSec; /** * Returns all children with path as a parent. If path is non-existent, @@ -162,7 +163,7 @@ private class LongHierarchicalLedgerRangeIterator implements LedgerRangeIterator */ List getChildrenAt(String path) throws IOException { try { - List children = ZkUtils.getChildrenInSingleNode(zk, path); + List children = ZkUtils.getChildrenInSingleNode(zk, path, zkOpTimeoutSec); Collections.sort(children); return children; } catch (KeeperException.NoNodeException e) { @@ -284,7 +285,9 @@ public LedgerRange next() throws IOException { } } - private LongHierarchicalLedgerRangeIterator() {} + private LongHierarchicalLedgerRangeIterator(long zkOpTimeoutSec) { + this.zkOpTimeoutSec = zkOpTimeoutSec; + } private void bootstrap() throws IOException { if (rootIterator == null) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index 0834147a8c5..266db3a3fbc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -641,7 +641,7 @@ public LedgerRange next() throws IOException { } @Override - public LedgerRangeIterator getLedgerRanges() { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { return new MSLedgerRangeIterator(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java index f553b6431b9..1683fd4a887 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java @@ -90,7 +90,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { LedgerManagerFactory mFactory = bookieServer.getBookie().getLedgerManagerFactory(); LedgerManager manager = mFactory.newLedgerManager(); - LedgerManager.LedgerRangeIterator iter = manager.getLedgerRanges(); + LedgerManager.LedgerRangeIterator iter = manager.getLedgerRanges(0); // output LinkedHashMap output = Maps.newLinkedHashMap(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java index 5ba4a855496..23d0b421c3e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.conf.AbstractConfiguration; @@ -221,7 +222,7 @@ private static class GetChildrenCtx { * @throws InterruptedException * @throws IOException */ - public static List getChildrenInSingleNode(final ZooKeeper zk, final String node) + public static List getChildrenInSingleNode(final ZooKeeper zk, final String node, long timeOutSec) throws InterruptedException, IOException, KeeperException.NoNodeException { final GetChildrenCtx ctx = new GetChildrenCtx(); getChildrenInSingleNode(zk, node, new GenericCallback>() { @@ -240,7 +241,12 @@ public void operationComplete(int rc, List ledgers) { synchronized (ctx) { while (!ctx.done) { - ctx.wait(); + try { + ctx.wait(timeOutSec > 0 ? TimeUnit.SECONDS.toMillis(timeOutSec) : 0); + } catch (InterruptedException e) { + ctx.rc = Code.OPERATIONTIMEOUT.intValue(); + ctx.done = true; + } } } if (Code.NONODE.intValue() == ctx.rc) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java index 556556e348d..3d7d2c4aa1a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java @@ -970,7 +970,7 @@ void unsupported() { } @Override - public LedgerRangeIterator getLedgerRanges() { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { final AtomicBoolean hasnext = new AtomicBoolean(true); return new LedgerManager.LedgerRangeIterator() { @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index ab966fb4ae2..e6c3c222928 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -112,8 +112,8 @@ public CompletableFuture> readLedgerMetadata(long ledg } @Override - public LedgerRangeIterator getLedgerRanges() { - return lm.getLedgerRanges(); + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { + return lm.getLedgerRanges(zkOpTimeoutSec); } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 933c11763f0..abc66144a72 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -316,7 +316,7 @@ public void clean(long ledgerId) { }; SortedSet scannedLedgers = new TreeSet(); - LedgerRangeIterator iterator = getLedgerManager().getLedgerRanges(); + LedgerRangeIterator iterator = getLedgerManager().getLedgerRanges(0); while (iterator.hasNext()) { LedgerRange ledgerRange = iterator.next(); scannedLedgers.addAll(ledgerRange.getLedgers()); @@ -422,7 +422,7 @@ public void testGcLedgersIfLedgerManagerIteratorFails() throws Exception { LedgerManager mockLedgerManager = new CleanupLedgerManager(getLedgerManager()) { @Override - public LedgerRangeIterator getLedgerRanges() { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeout) { return new LedgerRangeIterator() { @Override public LedgerRange next() throws IOException { @@ -552,7 +552,7 @@ public void clean(long ledgerId) { public void validateLedgerRangeIterator(SortedSet createdLedgers) throws IOException { SortedSet scannedLedgers = new TreeSet(); - LedgerRangeIterator iterator = getLedgerManager().getLedgerRanges(); + LedgerRangeIterator iterator = getLedgerManager().getLedgerRanges(0); while (iterator.hasNext()) { LedgerRange ledgerRange = iterator.next(); scannedLedgers.addAll(ledgerRange.getLedgers()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java index bdb29029766..1593e4bab98 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java @@ -134,7 +134,7 @@ static Set getLedgerIdsByUsingAsyncProcessLedgers(LedgerManager lm) throws @Test public void testIterateNoLedgers() throws Exception { LedgerManager lm = getLedgerManager(); - LedgerRangeIterator lri = lm.getLedgerRanges(); + LedgerRangeIterator lri = lm.getLedgerRanges(0); assertNotNull(lri); if (lri.hasNext()) { lri.next(); @@ -150,7 +150,7 @@ public void testSingleLedger() throws Throwable { long id = 2020202; createLedger(lm, id); - LedgerRangeIterator lri = lm.getLedgerRanges(); + LedgerRangeIterator lri = lm.getLedgerRanges(0); assertNotNull(lri); Set lids = ledgerRangeToSet(lri); assertEquals(lids.size(), 1); @@ -169,7 +169,7 @@ public void testTwoLedgers() throws Throwable { createLedger(lm, id); } - LedgerRangeIterator lri = lm.getLedgerRanges(); + LedgerRangeIterator lri = lm.getLedgerRanges(0); assertNotNull(lri); Set returnedIds = ledgerRangeToSet(lri); assertEquals(ids, returnedIds); @@ -188,7 +188,7 @@ public void testSeveralContiguousLedgers() throws Throwable { ids.add(i); } - LedgerRangeIterator lri = lm.getLedgerRanges(); + LedgerRangeIterator lri = lm.getLedgerRanges(0); assertNotNull(lri); Set returnedIds = ledgerRangeToSet(lri); assertEquals(ids, returnedIds); @@ -232,7 +232,7 @@ public void testRemovalOfNodeJustTraversed() throws Throwable { } Set found = new TreeSet<>(); - LedgerRangeIterator lri = lm.getLedgerRanges(); + LedgerRangeIterator lri = lm.getLedgerRanges(0); while (lri.hasNext()) { LedgerManager.LedgerRange lr = lri.next(); found.addAll(lr.getLedgers()); @@ -279,7 +279,7 @@ public void validateEmptyL4PathSkipped() throws Throwable { path, "data".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } - LedgerRangeIterator lri = lm.getLedgerRanges(); + LedgerRangeIterator lri = lm.getLedgerRanges(0); assertNotNull(lri); Set returnedIds = ledgerRangeToSet(lri); assertEquals(ids, returnedIds); @@ -287,7 +287,7 @@ public void validateEmptyL4PathSkipped() throws Throwable { Set ledgersReadAsync = getLedgerIdsByUsingAsyncProcessLedgers(lm); assertEquals("Comparing LedgersIds read asynchronously", ids, ledgersReadAsync); - lri = lm.getLedgerRanges(); + lri = lm.getLedgerRanges(0); int emptyRanges = 0; while (lri.hasNext()) { if (lri.next().getLedgers().isEmpty()) { @@ -329,7 +329,7 @@ public void testWithSeveralIncompletePaths() throws Throwable { path, "data".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } - LedgerRangeIterator lri = lm.getLedgerRanges(); + LedgerRangeIterator lri = lm.getLedgerRanges(0); assertNotNull(lri); Set returnedIds = ledgerRangeToSet(lri); assertEquals(ids, returnedIds); @@ -394,7 +394,7 @@ public void checkConcurrentModifications() throws Throwable { latch.await(); while (MathUtils.elapsedNanos(start) < runtime) { - LedgerRangeIterator lri = checkerLM.getLedgerRanges(); + LedgerRangeIterator lri = checkerLM.getLedgerRanges(0); Set returnedIds = ledgerRangeToSet(lri); for (long id: mustExist) { assertTrue(returnedIds.contains(id)); @@ -505,7 +505,7 @@ public void testLedgerManagerFormat() throws Throwable { public void hierarchicalLedgerManagerAsyncProcessLedgersTest() throws Throwable { Assume.assumeTrue(baseConf.getLedgerManagerFactoryClass().equals(HierarchicalLedgerManagerFactory.class)); LedgerManager lm = getLedgerManager(); - LedgerRangeIterator lri = lm.getLedgerRanges(); + LedgerRangeIterator lri = lm.getLedgerRanges(0); Set ledgerIds = new TreeSet<>(Arrays.asList(1234L, 123456789123456789L)); for (Long ledgerId : ledgerIds) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index f5cbe3a7e65..0f818b66c9e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -188,7 +188,7 @@ public void asyncProcessLedgers(Processor processor, AsyncCallback.VoidCal } @Override - public LedgerRangeIterator getLedgerRanges() { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { return null; } diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index 4988a810fca..a40a4bc383b 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -420,7 +420,7 @@ private void processLedgers(KeyStream ks, } @Override - public LedgerRangeIterator getLedgerRanges() { + public LedgerRangeIterator getLedgerRanges(long opTimeOutSec) { KeyStream ks = new KeyStream<>( kvClient, ByteSequence.fromString(EtcdUtils.getLedgerKey(scope, 0L)), diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java index 984224c6b8b..992d24f09be 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java @@ -225,7 +225,7 @@ public void testLedgerRangeIterator() throws Exception { createNumLedgers(numLedgers); long nextLedgerId = 0L; - LedgerRangeIterator iter = lm.getLedgerRanges(); + LedgerRangeIterator iter = lm.getLedgerRanges(0); while (iter.hasNext()) { LedgerRange lr = iter.next(); for (Long lid : lr.getLedgers()) { From d2550b1e005cbbc52796d678b2e4f356b9569eea Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Wed, 20 Feb 2019 02:24:03 -0800 Subject: [PATCH 0216/1642] Filter empty string for networkTopologyScriptFileName Descriptions of the changes in this PR: - filter empty string for networkTopologyScriptFileName. Reviewers: Enrico Olivelli , Sijie Guo This closes #1952 from reddycharan/ignoreemptystring --- .../bookkeeper/net/AbstractDNSToSwitchMapping.java | 4 +++- .../org/apache/bookkeeper/net/ScriptBasedMapping.java | 11 +++++++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java index 84e9bd47670..a19cc62b4b9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java @@ -23,6 +23,7 @@ import org.apache.bookkeeper.conf.Configurable; import org.apache.commons.configuration.Configuration; +import org.apache.commons.lang.StringUtils; /** * This is a base class for DNS to Switch mappings. @@ -117,7 +118,8 @@ public String dumpTopology() { } protected boolean isSingleSwitchByScriptPolicy() { - return conf != null && conf.getString(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null; + return conf != null + && (!StringUtils.isNotBlank(conf.getString(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY))); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java index 230f66d411d..0ef92ef63f3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.util.Shell.ShellCommandExecutor; import org.apache.commons.configuration.Configuration; +import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -139,8 +140,14 @@ private static final class RawScriptBasedMapping extends AbstractDNSToSwitchMapp public void setConf(Configuration conf) { super.setConf(conf); if (conf != null) { - scriptName = conf.getString(SCRIPT_FILENAME_KEY); - maxArgs = conf.getInt(SCRIPT_ARG_COUNT_KEY, DEFAULT_ARG_COUNT); + String scriptNameConfValue = conf.getString(SCRIPT_FILENAME_KEY); + if (StringUtils.isNotBlank(scriptNameConfValue)) { + scriptName = scriptNameConfValue; + maxArgs = conf.getInt(SCRIPT_ARG_COUNT_KEY, DEFAULT_ARG_COUNT); + } else { + scriptName = null; + maxArgs = 0; + } } else { scriptName = null; maxArgs = 0; From 8c4b60fb31a6945c55b3c985c1a4b7e221f407f8 Mon Sep 17 00:00:00 2001 From: Samuel Just Date: Wed, 20 Feb 2019 02:43:29 -0800 Subject: [PATCH 0217/1642] BOOKKEEPER-1919: putEntryOffset translate FileInfoDeletedException IndexInMemPageMgr should translate FileInfoDeletedException into NoLedgerException as expected by users like InterleavedLedgerStorage.updateEntriesLocations and EntryMemTable.flushSnapshot. Signed-off-by: Samuel Just Reviewers: Enrico Olivelli , Sijie Guo This closes #1950 from athanatos/forupstream/wip-1919 --- .../bookkeeper/bookie/IndexInMemPageMgr.java | 2 + .../bookkeeper/bookie/LedgerCacheTest.java | 122 ++++++++++++++++++ 2 files changed, 124 insertions(+) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java index 1e7d432ae83..0b77dd771cf 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java @@ -571,6 +571,8 @@ void putEntryOffset(long ledger, long entry, long offset) throws IOException { lep = getLedgerEntryPage(ledger, pageEntry); assert lep != null; lep.setOffset(offset, offsetInPage * LedgerEntryPage.getIndexEntrySize()); + } catch (FileInfo.FileInfoDeletedException e) { + throw new Bookie.NoLedgerException(ledger); } finally { if (null != lep) { lep.releasePage(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java index a606f9bd756..405b93a8cce 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java @@ -379,6 +379,128 @@ public void testSyncThreadNPE() throws IOException { } } + + /** + * Test for race between putEntryOffset and flush. + * {@link https://github.com/apache/bookkeeper/issues/1919} + */ + @Test + public void testPutEntryOffsetDeleteRace() throws Exception { + newLedgerCache(); + final AtomicInteger rc = new AtomicInteger(0); + final LinkedBlockingQueue putQ = new LinkedBlockingQueue<>(100); + final LinkedBlockingQueue deleteQ = new LinkedBlockingQueue<>(100); + final byte[] masterKey = "masterKey".getBytes(); + final long numLedgers = 1000; + final int numPutters = 10; + final int numDeleters = 10; + final AtomicBoolean running = new AtomicBoolean(true); + Thread newLedgerThread = new Thread() { + public void run() { + try { + for (long i = 0; i < numLedgers && rc.get() == 0; i++) { + ledgerCache.setMasterKey(i, masterKey); + + ledgerCache.putEntryOffset(i, 1, 0); + deleteQ.put(i); + putQ.put(i); + } + for (int i = 0; i < numPutters; ++i) { + putQ.put(-1L); + } + for (int i = 0; i < numDeleters; ++i) { + deleteQ.put(-1L); + } + } catch (Throwable e) { + rc.set(-1); + LOG.error("Exception in new ledger thread", e); + } + } + }; + newLedgerThread.start(); + + Thread[] flushThreads = new Thread[numPutters]; + for (int i = 0; i < numPutters; ++i) { + Thread flushThread = new Thread() { + public void run() { + try { + while (true) { + long id = putQ.take(); + if (id == -1L) { + break; + } + LOG.info("Putting {}", id); + try { + ledgerCache.putEntryOffset(id, 2, 0); + ledgerCache.deleteLedger(id); + } catch (NoLedgerException e) { + // No problem + } + } + } catch (Throwable e) { + rc.set(-1); + LOG.error("Exception in put thread", e); + } + } + }; + flushThread.start(); + flushThreads[i] = flushThread; + } + + Thread[] deleteThreads = new Thread[numDeleters]; + for (int i = 0; i < numDeleters; ++i) { + Thread deleteThread = new Thread() { + public void run() { + try { + while (true) { + long id = deleteQ.take(); + if (id == -1L) { + break; + } + LOG.info("Deleting {}", id); + try { + ledgerCache.deleteLedger(id); + } catch (NoLedgerException e) { + // No problem + } + } + } catch (Throwable e) { + rc.set(-1); + LOG.error("Exception in delete thread", e); + } + } + }; + deleteThread.start(); + deleteThreads[i] = deleteThread; + } + + newLedgerThread.join(); + + for (Thread deleteThread : deleteThreads) { + deleteThread.join(); + } + + running.set(false); + for (Thread flushThread : flushThreads) { + flushThread.join(); + } + + assertEquals("Should have been no errors", rc.get(), 0); + for (long i = 0L; i < numLedgers; ++i) { + boolean gotError = false; + try { + LOG.error("Checking {}", i); + ledgerCache.getEntryOffset(i, 0); + } catch (NoLedgerException e) { + gotError = true; + } + if (!gotError) { + LOG.error("Ledger {} is still around", i); + fail("Found ledger " + i + ", which should have been removed"); + } + } + } + /** * Test for race between delete and flush. * {@link https://issues.apache.org/jira/browse/BOOKKEEPER-604} From 039a9bf5fdb1b9781aa238341a4d94a02781d248 Mon Sep 17 00:00:00 2001 From: Like Date: Wed, 20 Feb 2019 20:11:00 +0800 Subject: [PATCH 0218/1642] [DOC] Fix command path in documents Closes #1556 Reviewers: Enrico Olivelli , Sijie Guo This closes #1948 from liketic/fix-cmd-dir --- docker/README.md | 6 ++--- site/_data/cli/bookkeeper-daemon.yaml | 2 +- site/_data/cli/bookkeeper.yaml | 2 +- site/_includes/shell.html | 2 +- site/docs/latest/admin/autorecovery.md | 10 ++++---- site/docs/latest/admin/bookies.md | 24 +++++++++---------- site/docs/latest/api/ledger-api.md | 2 +- site/docs/latest/deployment/manual.md | 4 ++-- .../latest/getting-started/run-locally.md | 2 +- 9 files changed, 27 insertions(+), 27 deletions(-) diff --git a/docker/README.md b/docker/README.md index 6cd337f27bd..66fa081271a 100644 --- a/docker/README.md +++ b/docker/README.md @@ -230,10 +230,10 @@ Usually we could config files bk_server.conf, bkenv.sh, log4j.properties, and lo Be careful where you put the transaction log (journal). A dedicated transaction log device is key to consistent good performance. Putting the log on a busy device will adversely effect performance. -Here is some useful and graceful command the could be used to replace the default command, once you want to delete the cookeis and do auto recovery: +Here is some useful and graceful command the could be used to replace the default command, once you want to delete the cookies and do auto recovery: ``` -/bookkeeper/bookkeeper-server/bin/bookkeeper shell bookieformat -nonInteractive -force -deleteCookie -/bookkeeper/bookkeeper-server/bin/bookkeeper autorecovery +/bookkeeper/bin/bookkeeper shell bookieformat -nonInteractive -force -deleteCookie +/bookkeeper/bin/bookkeeper autorecovery ``` Use them, and replace the default [CMD] when you wanted to do things other than start a bookie. diff --git a/site/_data/cli/bookkeeper-daemon.yaml b/site/_data/cli/bookkeeper-daemon.yaml index 86318ed6608..97fc029e25b 100644 --- a/site/_data/cli/bookkeeper-daemon.yaml +++ b/site/_data/cli/bookkeeper-daemon.yaml @@ -1,5 +1,5 @@ description: Manages bookies. -root_path: bookkeeper-server/bin +root_path: bin commands: - name: bookie description: Runs the bookie server. diff --git a/site/_data/cli/bookkeeper.yaml b/site/_data/cli/bookkeeper.yaml index 0b9bbd45153..cc4b2eca5d5 100644 --- a/site/_data/cli/bookkeeper.yaml +++ b/site/_data/cli/bookkeeper.yaml @@ -1,5 +1,5 @@ description: Manages bookies. -root_path: bookkeeper-server/bin +root_path: bin commands: - name: bookie description: Starts up a bookie. diff --git a/site/_includes/shell.html b/site/_includes/shell.html index ef7caf0f8a7..5e0927f66e3 100644 --- a/site/_includes/shell.html +++ b/site/_includes/shell.html @@ -8,7 +8,7 @@

            {{ command.name }}

            Usage
            ```shell -$ bookkeeper-server/bin/bookkeeper shell {{ command.name }}{% if command.argument %} \ +$ bin/bookkeeper shell {{ command.name }}{% if command.argument %} \ {{ command.argument }}{% endif %}{% if command.options %} \ {% endif %} ``` diff --git a/site/docs/latest/admin/autorecovery.md b/site/docs/latest/admin/autorecovery.md index b1dd078f9b2..e64dbac6abe 100644 --- a/site/docs/latest/admin/autorecovery.md +++ b/site/docs/latest/admin/autorecovery.md @@ -18,7 +18,7 @@ You can manually recover failed bookies using the [`bookkeeper`](../../reference Here's an example: ```bash -$ bookkeeper-server/bin/bookkeeper shell recover \ +$ bin/bookkeeper shell recover \ zk1.example.com:2181 \ # IP and port for ZooKeeper 192.168.1.10:3181 # IP and port for the failed bookie ``` @@ -26,7 +26,7 @@ $ bookkeeper-server/bin/bookkeeper shell recover \ If you wish, you can also specify which bookie you'd like to rereplicate to. Here's an example: ```bash -$ bookkeeper-server/bin/bookkeeper shell recover \ +$ bin/bookkeeper shell recover \ 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 @@ -58,7 +58,7 @@ AutoRecovery can be run in two ways: 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 +$ 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. @@ -78,13 +78,13 @@ You can disable AutoRecovery at any time, for example during maintenance. Disabl You can disable AutoRecover using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: ```bash -$ bookkeeper-server/bin/bookkeeper shell autorecovery -disable +$ 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 +$ bin/bookkeeper shell autorecovery -enable ``` ## AutoRecovery architecture diff --git a/site/docs/latest/admin/bookies.md b/site/docs/latest/admin/bookies.md index 1b0427dae3c..68e8a7df7e2 100644 --- a/site/docs/latest/admin/bookies.md +++ b/site/docs/latest/admin/bookies.md @@ -31,13 +31,13 @@ You can run bookies either in the foreground or in the background, using [nohup] 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 +$ 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 +$ bin/bookkeeper-daemon.sh start bookie ``` ### Local bookies @@ -47,7 +47,7 @@ The instructions above showed you how to run bookies intended for production use This would spin up a local ensemble of 6 bookies: ```shell -$ bookkeeper-server/bin/bookkeeper localbookie 6 +$ bin/bookkeeper localbookie 6 ``` > When you run a local bookie ensemble, all bookies run in a single JVM process. @@ -75,7 +75,7 @@ To enable logging for a bookie, create a `log4j.properties` file and point the ` ```shell $ export BOOKIE_LOG_CONF=/some/path/log4j.properties -$ bookkeeper-server/bin/bookkeeper bookie +$ bin/bookkeeper bookie ``` ## Upgrading @@ -99,25 +99,25 @@ Flag | Action A standard upgrade pattern is to run an upgrade... ```shell -$ bookkeeper-server/bin/bookkeeper upgrade --upgrade +$ 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 +$ bin/bookkeeper upgrade --finalize ``` ...and then restart the server: ```shell -$ bookkeeper-server/bin/bookkeeper bookie +$ bin/bookkeeper bookie ``` If something has gone wrong, you can always perform a rollback: ```shell -$ bookkeeper-server/bin/bookkeeper upgrade --rollback +$ bin/bookkeeper upgrade --rollback ``` ## Formatting @@ -127,13 +127,13 @@ You can format bookie metadata in ZooKeeper using the [`metaformat`](../../refer 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 +$ 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 +$ bin/bookkeeper shell bookieformat ``` > The `-force` and `-nonInteractive` flags are also available for the `bookieformat` command. @@ -162,7 +162,7 @@ If the change was the result of an accidental configuration change, the change c 1. Run the following command to re-replicate the data: ```bash - $ bookkeeper-server/bin/bookkeeper shell recover \ + $ bin/bookkeeper shell recover \ \ \ @@ -171,7 +171,7 @@ If the change was the result of an accidental configuration change, the change c 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ + $ bin/bookkeeper shell recover \ zk1.example.com \ 192.168.1.10:3181 \ 192.168.1.10:3181 diff --git a/site/docs/latest/api/ledger-api.md b/site/docs/latest/api/ledger-api.md index 22dacb59d6c..38f436d1cf8 100644 --- a/site/docs/latest/api/ledger-api.md +++ b/site/docs/latest/api/ledger-api.md @@ -302,7 +302,7 @@ Before you start, you will need to have a BookKeeper cluster running locally on To start up a cluster consisting of six {% pop bookies %} locally: ```shell -$ bookkeeper-server/bin/bookkeeper localbookie 6 +$ bin/bookkeeper localbookie 6 ``` You can specify a different number of bookies if you'd like. diff --git a/site/docs/latest/deployment/manual.md b/site/docs/latest/deployment/manual.md index daafd5556f5..49800da1822 100644 --- a/site/docs/latest/deployment/manual.md +++ b/site/docs/latest/deployment/manual.md @@ -26,7 +26,7 @@ zkServers=100.0.0.1:2181,100.0.0.2:2181,100.0.0.3:2181 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 +$ 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). @@ -40,7 +40,7 @@ $ bookkeeper-server/bin/bookkeeper bookie 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 +$ bin/bookkeeper shell metaformat ``` You can run in the formatting diff --git a/site/docs/latest/getting-started/run-locally.md b/site/docs/latest/getting-started/run-locally.md index edbfab9fda6..520140317d9 100644 --- a/site/docs/latest/getting-started/run-locally.md +++ b/site/docs/latest/getting-started/run-locally.md @@ -10,7 +10,7 @@ toc_disable: true This would start up an ensemble with 10 bookies: ```shell -$ bookkeeper-server/bin/bookkeeper localbookie 10 +$ bin/bookkeeper localbookie 10 ``` > When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. From 32462d9186426799bf1b0124ef2bd0fc51ca1776 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 21 Feb 2019 12:41:14 -0800 Subject: [PATCH 0219/1642] Exclude defaultrack bookies when enforceMinNumRacksPerWriteQuorum is enabled Descriptions of the changes in this PR: - enforceMinNumRacksPerWriteQuorum is meant to be used for strict placement policy. So when it is enabled, bookies which belong to default faultzone/rack (because of failure in resolving network location) should be excluded from bookie selection. - add gauge for number of bookies in default faultzone/rack. It will be helpful to create alerts based on this gauge. - add gauge for number of ledgers found not adhering to strict placement policy in Auditor's placement policy check. This gauge will be more helpful in creating alert instead of using monotonously increasing alert. Reviewers: Enrico Olivelli , Pasha Kuznetsov , Sijie Guo , Venkateswararao Jujjuri (JV) This closes #1941 from reddycharan/ignoredefaultzoneandfixmetrics --- .../bookkeeper/test/TestStatsProvider.java | 2 +- .../client/BookKeeperClientStats.java | 3 + .../client/EnsemblePlacementPolicy.java | 6 + .../RackawareEnsemblePlacementPolicyImpl.java | 121 ++++++--- .../RegionAwareEnsemblePlacementPolicy.java | 25 +- .../TopologyAwareEnsemblePlacementPolicy.java | 2 - .../bookkeeper/net/NetworkTopology.java | 11 + .../bookkeeper/net/NetworkTopologyImpl.java | 12 +- .../net/StabilizeNetworkTopology.java | 6 + .../bookkeeper/replication/Auditor.java | 61 ++++- .../replication/ReplicationStats.java | 3 +- .../TestRackawareEnsemblePlacementPolicy.java | 248 ++++++++++++++++-- .../AuditorPlacementPolicyCheckTest.java | 28 +- 13 files changed, 427 insertions(+), 101 deletions(-) diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java index bef0dfeee68..624791cdd5a 100644 --- a/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java @@ -177,7 +177,7 @@ public Counter getCounter(String name) { } public Gauge getGauge(String name) { - return gaugeMap.get(path); + return gaugeMap.get(getSubPath(name)); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java index 8b358b6ead1..cdfde67693d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java @@ -93,6 +93,9 @@ public interface BookKeeperClientStats { String FAILED_CONNECTION_COUNTER = "FAILED_CONNECTION_COUNTER"; String FAILED_TLS_HANDSHAKE_COUNTER = "FAILED_TLS_HANDSHAKE_COUNTER"; + // placementpolicy stats + String NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK = "NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK"; + OpStatsLogger getCreateOpLogger(); OpStatsLogger getOpenOpLogger(); OpStatsLogger getDeleteOpLogger(); 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 fad3f9224c1..7dc81112531 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 @@ -251,6 +251,9 @@ Set onClusterChanged(Set writableBooki *

            {@code customMetadata} is the same user defined data that user provides * when {@link BookKeeper#createLedger(int, int, int, BookKeeper.DigestType, byte[], Map)}. * + *

            If 'enforceMinNumRacksPerWriteQuorum' config is enabled then the bookies belonging to default + * faultzone (rack) will be excluded while selecting bookies. + * * @param ensembleSize * Ensemble Size * @param writeQuorumSize @@ -274,6 +277,9 @@ PlacementResult> newEnsemble(int ensembleSize, * Choose a new bookie to replace bookieToReplace. If no bookie available in the cluster, * {@link BKNotEnoughBookiesException} is thrown. * + *

            If 'enforceMinNumRacksPerWriteQuorum' config is enabled then the bookies belonging to default + * faultzone (rack) will be excluded while selecting bookies. + * * @param ensembleSize * the value of ensembleSize * @param writeQuorumSize 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 304db72745c..ef4341ae5b3 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 @@ -22,6 +22,7 @@ import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_LEFT; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER; import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE; +import static org.apache.bookkeeper.client.BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK; import static org.apache.bookkeeper.client.BookKeeperClientStats.READ_REQUESTS_REORDERED; import static org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy.UNKNOWN_REGION; @@ -67,6 +68,7 @@ import org.apache.bookkeeper.net.ScriptBasedMapping; import org.apache.bookkeeper.net.StabilizeNetworkTopology; import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; @@ -114,11 +116,6 @@ static class DefaultResolver implements DNSToSwitchMapping { final Supplier defaultRackSupplier; - // for backwards compat - public DefaultResolver() { - this(() -> NetworkTopology.DEFAULT_REGION_AND_RACK); - } - public DefaultResolver(Supplier defaultRackSupplier) { checkNotNull(defaultRackSupplier, "defaultRackSupplier should not be null"); this.defaultRackSupplier = defaultRackSupplier; @@ -240,7 +237,16 @@ public void reloadCachedMappings() { help = "The distribution of number of bookies reordered on each read request" ) protected OpStatsLogger readReorderedCounter = null; + @StatsDoc( + name = FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER, + help = "Counter for number of times DNSResolverDecorator failed to resolve Network Location" + ) protected Counter failedToResolveNetworkLocationCounter = null; + @StatsDoc( + name = NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK, + help = "Gauge for the number of writable Bookies in default rack" + ) + protected Gauge numWritableBookiesInDefaultRack; private String defaultRack = NetworkTopology.DEFAULT_RACK; @@ -282,8 +288,24 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns this.bookiesJoinedCounter = statsLogger.getOpStatsLogger(BOOKIES_JOINED); this.bookiesLeftCounter = statsLogger.getOpStatsLogger(BOOKIES_LEFT); this.readReorderedCounter = statsLogger.getOpStatsLogger(READ_REQUESTS_REORDERED); - this.failedToResolveNetworkLocationCounter = statsLogger - .getCounter(FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER); + this.failedToResolveNetworkLocationCounter = statsLogger.getCounter(FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER); + this.numWritableBookiesInDefaultRack = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + rwLock.readLock().lock(); + try { + return topology.countNumOfAvailableNodes(getDefaultRack(), Collections.emptySet()); + } finally { + rwLock.readLock().unlock(); + } + } + }; + this.statsLogger.registerGauge(NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK, numWritableBookiesInDefaultRack); this.reorderReadsRandom = reorderReadsRandom; this.stabilizePeriodSeconds = stabilizePeriodSeconds; this.reorderThresholdPendingRequests = reorderThresholdPendingRequests; @@ -417,7 +439,9 @@ public void onBookieRackChange(List bookieAddressList) { if (node != null) { // refresh the rack info if its a known bookie topology.remove(node); - topology.add(createBookieNode(bookieAddress)); + BookieNode newNode = createBookieNode(bookieAddress); + topology.add(newNode); + knownBookies.put(bookieAddress, newNode); } } } finally { @@ -455,6 +479,9 @@ public Set onClusterChanged(Set writab } } + /* + * this method should be called in writelock scope of 'rwLock' + */ @Override public void handleBookiesThatLeft(Set leftBookies) { for (BookieSocketAddress addr : leftBookies) { @@ -483,6 +510,9 @@ public void handleBookiesThatLeft(Set leftBookies) { } } + /* + * this method should be called in writelock scope of 'rwLock' + */ @Override public void handleBookiesThatJoined(Set joinedBookies) { // node joined @@ -531,26 +561,53 @@ private static Set getNetworkLocations(Set bookieNodes) { return networkLocs; } + /* + * this method should be called in readlock scope of 'rwLock' + */ + protected Set addDefaultRackBookiesIfMinNumRacksIsEnforced( + Set excludeBookies) { + Set comprehensiveExclusionBookiesSet; + if (enforceMinNumRacksPerWriteQuorum) { + Set bookiesInDefaultRack = null; + Set defaultRackLeaves = topology.getLeaves(getDefaultRack()); + for (Node node : defaultRackLeaves) { + if (node instanceof BookieNode) { + if (bookiesInDefaultRack == null) { + bookiesInDefaultRack = new HashSet(excludeBookies); + } + bookiesInDefaultRack.add(((BookieNode) node).getAddr()); + } else { + LOG.error("found non-BookieNode: {} as leaf of defaultrack: {}", node, getDefaultRack()); + } + } + if ((bookiesInDefaultRack == null) || bookiesInDefaultRack.isEmpty()) { + comprehensiveExclusionBookiesSet = excludeBookies; + } else { + comprehensiveExclusionBookiesSet = new HashSet(excludeBookies); + comprehensiveExclusionBookiesSet.addAll(bookiesInDefaultRack); + LOG.info("enforceMinNumRacksPerWriteQuorum is enabled, so Excluding bookies of defaultRack: {}", + bookiesInDefaultRack); + } + } else { + comprehensiveExclusionBookiesSet = excludeBookies; + } + return comprehensiveExclusionBookiesSet; + } + @Override public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { - return newEnsembleInternal(ensembleSize, writeQuorumSize, excludeBookies, null, null); - } - - protected PlacementResult> newEnsembleInternal(int ensembleSize, - int writeQuorumSize, - Set excludeBookies, - Ensemble parentEnsemble, - Predicate parentPredicate) - throws BKNotEnoughBookiesException { - return newEnsembleInternal( - ensembleSize, - writeQuorumSize, - writeQuorumSize, - excludeBookies, - parentEnsemble, - parentPredicate); + rwLock.readLock().lock(); + try { + Set comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced( + excludeBookies); + PlacementResult> newEnsembleResult = newEnsembleInternal(ensembleSize, + writeQuorumSize, ackQuorumSize, comprehensiveExclusionBookiesSet, null, null); + return newEnsembleResult; + } finally { + rwLock.readLock().unlock(); + } } @Override @@ -643,6 +700,7 @@ public PlacementResult replaceBookie(int ensembleSize, int throws BKNotEnoughBookiesException { rwLock.readLock().lock(); try { + excludeBookies = addDefaultRackBookiesIfMinNumRacksIsEnforced(excludeBookies); excludeBookies.addAll(currentEnsemble); BookieNode bn = knownBookies.get(bookieToReplace); if (null == bn) { @@ -1253,28 +1311,31 @@ static void shuffleWithMask(DistributionSchedule.WriteSet writeSet, } } + // this method should be called in readlock scope of 'rwlock' @Override public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, int ackQuorumSize) { int ensembleSize = ensembleList.size(); int minNumRacksPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, minNumRacksPerWriteQuorum); - HashSet racksOrRegionsInQuorum = new HashSet(); + HashSet racksInQuorum = new HashSet(); BookieSocketAddress bookie; for (int i = 0; i < ensembleList.size(); i++) { - racksOrRegionsInQuorum.clear(); + racksInQuorum.clear(); for (int j = 0; j < writeQuorumSize; j++) { bookie = ensembleList.get((i + j) % ensembleSize); try { - racksOrRegionsInQuorum.add(knownBookies.get(bookie).getNetworkLocation()); + racksInQuorum.add(knownBookies.get(bookie).getNetworkLocation()); } catch (Exception e) { /* - * any issue/exception in analyzing whether ensemble is strictly adhering to - * placement policy should be swallowed. + * any issue/exception in analyzing whether ensemble is + * strictly adhering to placement policy should be + * swallowed. */ LOG.warn("Received exception while trying to get network location of bookie: {}", bookie, e); } } - if (racksOrRegionsInQuorum.size() < minNumRacksPerWriteQuorumForThisEnsemble) { + if ((racksInQuorum.size() < minNumRacksPerWriteQuorumForThisEnsemble) + || (enforceMinNumRacksPerWriteQuorum && racksInQuorum.contains(getDefaultRack()))) { return false; } } 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 cd80fdf3f49..399e8aaf322 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 @@ -228,7 +228,7 @@ protected List selectRandomFromRegions(Set availableRegions, @Override public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map customMetadata, Set excludeBookies) + int ackQuorumSize, Map customMetadata, Set excludedBookies) throws BKException.BKNotEnoughBookiesException { int effectiveMinRegionsForDurability = disableDurabilityFeature.isAvailable() ? 1 : minRegionsForDurability; @@ -257,7 +257,9 @@ public PlacementResult> newEnsemble(int ensembleSize, rwLock.readLock().lock(); try { - Set excludeNodes = convertBookiesToNodes(excludeBookies); + Set comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced( + excludedBookies); + Set excludeNodes = convertBookiesToNodes(comprehensiveExclusionBookiesSet); Set availableRegions = new HashSet(); for (String region: perRegionPlacement.keySet()) { if ((null == disallowBookiePlacementInRegionFeatureName) @@ -294,8 +296,8 @@ public PlacementResult> newEnsemble(int ensembleSize, effectiveMinRegionsForDurability, minNumRacksPerWriteQuorum); TopologyAwareEnsemblePlacementPolicy nextPolicy = perRegionPlacement.get( availableRegions.iterator().next()); - return nextPolicy.newEnsemble(ensembleSize, writeQuorumSize, writeQuorumSize, excludeBookies, ensemble, - ensemble); + return nextPolicy.newEnsemble(ensembleSize, writeQuorumSize, writeQuorumSize, + comprehensiveExclusionBookiesSet, ensemble, ensemble); } int remainingEnsemble = ensembleSize; @@ -349,9 +351,10 @@ public PlacementResult> newEnsemble(int ensembleSize, int newEnsembleSize = currentAllocation.getLeft() + addToEnsembleSize; int newWriteQuorumSize = currentAllocation.getRight() + addToWriteQuorum; try { - List allocated = policyWithinRegion.newEnsemble(newEnsembleSize, - newWriteQuorumSize, newWriteQuorumSize, excludeBookies, tempEnsemble, - tempEnsemble).getResult(); + List allocated = policyWithinRegion + .newEnsemble(newEnsembleSize, newWriteQuorumSize, newWriteQuorumSize, + comprehensiveExclusionBookiesSet, tempEnsemble, tempEnsemble) + .getResult(); ensemble = tempEnsemble; remainingEnsemble -= addToEnsembleSize; remainingWriteQuorum -= addToWriteQuorum; @@ -379,12 +382,12 @@ public PlacementResult> newEnsemble(int ensembleSize, if (regionsReachedMaxAllocation.contains(region)) { if (currentAllocation.getLeft() > 0) { LOG.info("Allocating {} bookies in region {} : ensemble {} exclude {}", - currentAllocation.getLeft(), region, excludeBookies, ensemble); + currentAllocation.getLeft(), region, comprehensiveExclusionBookiesSet, ensemble); policyWithinRegion.newEnsemble( currentAllocation.getLeft(), currentAllocation.getRight(), currentAllocation.getRight(), - excludeBookies, + comprehensiveExclusionBookiesSet, ensemble, ensemble); LOG.info("Allocated {} bookies in region {} : {}", @@ -428,7 +431,9 @@ public PlacementResult replaceBookie(int ensembleSize, int try { boolean enforceDurability = enforceDurabilityInReplace && !disableDurabilityFeature.isAvailable(); int effectiveMinRegionsForDurability = enforceDurability ? minRegionsForDurability : 1; - Set excludeNodes = convertBookiesToNodes(excludeBookies); + Set comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced( + excludeBookies); + Set excludeNodes = convertBookiesToNodes(comprehensiveExclusionBookiesSet); RRTopologyAwareCoverageEnsemble ensemble = new RRTopologyAwareCoverageEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, 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 19cb5050710..7fa75550498 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 @@ -78,7 +78,6 @@ public boolean validate() { } protected static class BookieNode extends NodeBase { - private final BookieSocketAddress addr; // identifier of a bookie node. BookieNode(BookieSocketAddress addr, String networkLoc) { @@ -108,7 +107,6 @@ public boolean equals(Object obj) { public String toString() { return String.format("", name); } - } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopology.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopology.java index a6bcf77a7c8..d2d37ea52c7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopology.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopology.java @@ -17,6 +17,7 @@ */ package org.apache.bookkeeper.net; +import java.util.Collection; import java.util.Set; /** @@ -76,4 +77,14 @@ public interface NetworkTopology { */ Set getLeaves(String loc); + /** + * Return the number of leaves in scope but not in excludedNodes. + * + *

            If scope starts with ~, return the number of nodes that are not + * in scope and excludedNodes; + * @param scope a path string that may start with ~ + * @param excludedNodes a list of nodes + * @return number of available nodes + */ + int countNumOfAvailableNodes(String scope, Collection excludedNodes); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java index d6756f8cc9d..dcf4cade3d0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java @@ -789,15 +789,7 @@ public Set getLeaves(String scope) { } } - /** - * Return the number of leaves in scope but not in excludedNodes. - * - *

            If scope starts with ~, return the number of nodes that are not - * in scope and excludedNodes; - * @param scope a path string that may start with ~ - * @param excludedNodes a list of nodes - * @return number of available nodes - */ + @Override public int countNumOfAvailableNodes(String scope, Collection excludedNodes) { boolean isExcluded = false; if (scope.startsWith("~")) { @@ -815,7 +807,7 @@ public int countNumOfAvailableNodes(String scope, Collection excludedNodes } } Node n = getNode(scope); - int scopeNodeCount = 1; + int scopeNodeCount = 0; if (n instanceof InnerNode) { scopeNodeCount = ((InnerNode) n).getNumOfLeaves(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/StabilizeNetworkTopology.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/StabilizeNetworkTopology.java index 5c244f248fa..df80bf9dbfd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/StabilizeNetworkTopology.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/StabilizeNetworkTopology.java @@ -21,6 +21,7 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; +import java.util.Collection; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -151,4 +152,9 @@ public int getNumOfRacks() { public Set getLeaves(String loc) { return impl.getLeaves(loc); } + + @Override + public int countNumOfAvailableNodes(String scope, Collection excludedNodes) { + return impl.countNumOfAvailableNodes(scope, excludedNodes); + } } 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 dd791be27d4..ad588490c8b 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,7 +20,6 @@ */ package org.apache.bookkeeper.replication; - import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; import static org.apache.bookkeeper.replication.ReplicationStats.AUDIT_BOOKIES_TIME; import static org.apache.bookkeeper.replication.ReplicationStats.BOOKIE_TO_LEDGERS_MAP_CREATION_TIME; @@ -30,9 +29,8 @@ import static org.apache.bookkeeper.replication.ReplicationStats.NUM_DELAYED_BOOKIE_AUDITS_DELAYES_CANCELLED; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_FRAGMENTS_PER_LEDGER; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_CHECKED; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_UNDER_REPLICATED_LEDGERS; -import static org.apache.bookkeeper.replication.ReplicationStats. - PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER; import static org.apache.bookkeeper.replication.ReplicationStats.PLACEMENT_POLICY_CHECK_TIME; import static org.apache.bookkeeper.replication.ReplicationStats.URL_PUBLISH_TIME_FOR_LOST_BOOKIE; @@ -55,6 +53,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.bookkeeper.client.BKException; @@ -79,6 +78,7 @@ import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -117,6 +117,8 @@ public class Auditor implements AutoCloseable { private volatile Future auditTask; private Set bookiesToBeAudited = Sets.newHashSet(); private volatile int lostBookieRecoveryDelayBeforeChange; + private final AtomicInteger ledgersNotAdheringToPlacementPolicyGuageValue; + private final AtomicInteger numOfLedgersFoundInPlacementPolicyCheck; private final StatsLogger statsLogger; @StatsDoc( @@ -175,11 +177,10 @@ public class Auditor implements AutoCloseable { ) private final Counter numDelayedBookieAuditsCancelled; @StatsDoc( - name = PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER, - help = "total number of " - + "ledgers failed to adhere to EnsemblePlacementPolicy found in PLACEMENT POLICY check" + name = NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY, + help = "Gauge for number of ledgers not adhering to placement policy found in placement policy check" ) - private final Counter placementPolicyCheckEnsembleNotAdheringToPlacementPolicy; + private final Gauge numLedgersNotAdheringToPlacementPolicy; static BookKeeper createBookKeeperClient(ServerConfiguration conf) throws InterruptedException, IOException { return createBookKeeperClient(conf, NullStatsLogger.INSTANCE); @@ -230,6 +231,8 @@ public Auditor(final String bookieIdentifier, this.conf = conf; this.bookieIdentifier = bookieIdentifier; this.statsLogger = statsLogger; + this.numOfLedgersFoundInPlacementPolicyCheck = new AtomicInteger(0); + this.ledgersNotAdheringToPlacementPolicyGuageValue = new AtomicInteger(0); numUnderReplicatedLedger = this.statsLogger.getOpStatsLogger(ReplicationStats.NUM_UNDER_REPLICATED_LEDGERS); uRLPublishTimeForLostBookies = this.statsLogger @@ -245,8 +248,20 @@ public Auditor(final String bookieIdentifier, numBookieAuditsDelayed = this.statsLogger.getCounter(ReplicationStats.NUM_BOOKIE_AUDITS_DELAYED); numDelayedBookieAuditsCancelled = this.statsLogger .getCounter(ReplicationStats.NUM_DELAYED_BOOKIE_AUDITS_DELAYES_CANCELLED); - placementPolicyCheckEnsembleNotAdheringToPlacementPolicy = statsLogger - .getCounter(ReplicationStats.PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER); + numLedgersNotAdheringToPlacementPolicy = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + return ledgersNotAdheringToPlacementPolicyGuageValue.get(); + } + }; + this.statsLogger.registerGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY, + numLedgersNotAdheringToPlacementPolicy); + this.bkc = bkc; this.ownBkc = ownBkc; initialize(conf, bkc); @@ -612,12 +627,33 @@ public void run() { Stopwatch stopwatch = Stopwatch.createStarted(); LOG.info("Starting PlacementPolicyCheck"); placementPolicyCheck(); + int numOfLedgersFoundInPlacementPolicyCheckValue = numOfLedgersFoundInPlacementPolicyCheck + .get(); long placementPolicyCheckDuration = stopwatch.stop().elapsed(TimeUnit.MILLISECONDS); - LOG.info("Completed placementPolicyCheck in {} milliSeconds", placementPolicyCheckDuration); + LOG.info( + "Completed placementPolicyCheck in {} milliSeconds." + + " numOfLedgersNotAdheringToPlacementPolicy {}", + placementPolicyCheckDuration, numOfLedgersFoundInPlacementPolicyCheckValue); + ledgersNotAdheringToPlacementPolicyGuageValue + .set(numOfLedgersFoundInPlacementPolicyCheckValue); placementPolicyCheckTime.registerSuccessfulEvent(placementPolicyCheckDuration, TimeUnit.MILLISECONDS); } catch (BKAuditException e) { - LOG.error("BKAuditException running periodic placementPolicy check", e); + int numOfLedgersFoundInPlacementPolicyCheckValue = numOfLedgersFoundInPlacementPolicyCheck + .get(); + if (numOfLedgersFoundInPlacementPolicyCheckValue > 0) { + /* + * Though there is BKAuditException while doing + * placementPolicyCheck, it found few ledgers not + * adhering to placement policy. So reporting it. + */ + ledgersNotAdheringToPlacementPolicyGuageValue + .set(numOfLedgersFoundInPlacementPolicyCheckValue); + } + LOG.error( + "BKAuditException running periodic placementPolicy check." + + "numOfLedgersNotAdheringToPlacementPolicy {}", + numOfLedgersFoundInPlacementPolicyCheckValue, e); } } }, initialDelay, interval, TimeUnit.SECONDS); @@ -893,6 +929,7 @@ void checkAllLedgers() throws BKException, IOException, InterruptedException, Ke void placementPolicyCheck() throws BKAuditException { final CountDownLatch placementPolicyCheckLatch = new CountDownLatch(1); + this.numOfLedgersFoundInPlacementPolicyCheck.set(0); Processor ledgerProcessor = new Processor() { @Override public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { @@ -920,7 +957,7 @@ public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { } } if (foundSegmentNotAdheringToPlacementPolicy) { - placementPolicyCheckEnsembleNotAdheringToPlacementPolicy.inc(); + numOfLedgersFoundInPlacementPolicyCheck.incrementAndGet(); } } else { if (LOG.isDebugEnabled()) { 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 e9b89995452..c13bc56cc23 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 @@ -40,8 +40,7 @@ public interface ReplicationStats { String NUM_LEDGERS_CHECKED = "NUM_LEDGERS_CHECKED"; String NUM_BOOKIE_AUDITS_DELAYED = "NUM_BOOKIE_AUDITS_DELAYED"; String NUM_DELAYED_BOOKIE_AUDITS_DELAYES_CANCELLED = "NUM_DELAYED_BOOKIE_AUDITS_CANCELLED"; - String PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER = - "PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER"; + String NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY = "NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY"; String REPLICATION_WORKER_SCOPE = "replication_worker"; String REREPLICATE_OP = "rereplicate"; 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 a55b5607412..26fa23ff566 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 @@ -29,6 +29,7 @@ import java.net.InetAddress; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -50,8 +51,12 @@ import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.net.Node; +import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.collections4.CollectionUtils; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -740,17 +745,22 @@ public void testSingleRackWithEnforceMinNumRacks() throws Exception { @Test public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { + String defaultRackForThisTest = NetworkTopology.DEFAULT_REGION_AND_RACK; repp.uninitalize(); + updateMyRack(defaultRackForThisTest); int minNumRacksPerWriteQuorum = 4; ClientConfiguration clientConf = new ClientConfiguration(conf); clientConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorum); // set enforceMinNumRacksPerWriteQuorum clientConf.setEnforceMinNumRacksPerWriteQuorum(true); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); - repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); + repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, statsLogger); + repp.withDefaultRack(defaultRackForThisTest); + Gauge numBookiesInDefaultRackGauge = statsLogger + .getGauge(BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK); int numOfRacks = 3; int numOfBookiesPerRack = 5; @@ -764,26 +774,41 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { } } - repp.onClusterChanged(new HashSet(Arrays.asList(bookieSocketAddresses)), - new HashSet()); + int numOfBookiesInDefaultRack = 5; + BookieSocketAddress[] bookieSocketAddressesInDefaultRack = new BookieSocketAddress[numOfBookiesInDefaultRack]; + for (int i = 0; i < numOfBookiesInDefaultRack; i++) { + bookieSocketAddressesInDefaultRack[i] = new BookieSocketAddress("128.0.0." + (100 + i), 3181); + StaticDNSResolver.addNodeToRack(bookieSocketAddressesInDefaultRack[i].getHostName(), + defaultRackForThisTest); + } + + List nonDefaultRackBookiesList = Arrays.asList(bookieSocketAddresses); + List defaultRackBookiesList = Arrays.asList(bookieSocketAddressesInDefaultRack); + Set writableBookies = new HashSet(nonDefaultRackBookiesList); + writableBookies.addAll(defaultRackBookiesList); + repp.onClusterChanged(writableBookies, new HashSet()); + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, + numBookiesInDefaultRackGauge.getSample()); try { + // this newEnsemble call will exclude default rack bookies repp.newEnsemble(8, 4, 4, null, new HashSet<>()); - fail("Should get not enough bookies exception since there are only 3 racks"); + fail("Should get not enough bookies exception since there are only 3 non-default racks"); } catch (BKNotEnoughBookiesException bnebe) { } try { - repp.newEnsemble(8, 4, 4, new HashSet<>(), + repp.newEnsemble(8, 4, 4, new HashSet<>(defaultRackBookiesList), EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); - fail("Should get not enough bookies exception since there are only 3 racks"); + fail("Should get not enough bookies exception since there are only 3 non-default racks" + + " and defaultrack bookies are excluded"); } catch (BKNotEnoughBookiesException bnebe) { } /* * Though minNumRacksPerWriteQuorum is set to 4, since writeQuorum is 3 - * and there are enough bookies in 3 racks, this newEnsemble calls should - * succeed. + * and there are enough bookies in 3 racks, this newEnsemble calls + * should succeed. */ EnsemblePlacementPolicy.PlacementResult> ensembleResponse; List ensemble; @@ -799,8 +824,9 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); assertTrue(isEnsembleAdheringToPlacementPolicy); - ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(), - EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, + new HashSet<>(defaultRackBookiesList), EnsembleForReplacementWithNoConstraints.INSTANCE, + TruePredicate.INSTANCE); ensemble = ensembleResponse.getResult(); isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, @@ -872,17 +898,23 @@ public void testNewEnsembleWithSufficientRacksAndEnforceMinNumRacks() throws Exc @Test public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { + String defaultRackForThisTest = NetworkTopology.DEFAULT_REGION_AND_RACK; repp.uninitalize(); + updateMyRack(defaultRackForThisTest); int minNumRacksPerWriteQuorum = 4; ClientConfiguration clientConf = new ClientConfiguration(conf); clientConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorum); // set enforceMinNumRacksPerWriteQuorum clientConf.setEnforceMinNumRacksPerWriteQuorum(true); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); repp = new RackawareEnsemblePlacementPolicy(); repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); - repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); + statsLogger); + repp.withDefaultRack(defaultRackForThisTest); + Gauge numBookiesInDefaultRackGauge = statsLogger + .getGauge(BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK); int numOfRacks = 3; int numOfBookiesPerRack = 5; @@ -900,8 +932,25 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { bookieRackMap.put(bookieAddress, rack); } } + /* + * bookies in this default rack should not be returned for replacebookie + * response. + */ + int numOfBookiesInDefaultRack = 5; + BookieSocketAddress[] bookieSocketAddressesInDefaultRack = new BookieSocketAddress[numOfBookiesInDefaultRack]; + for (int i = 0; i < numOfBookiesInDefaultRack; i++) { + bookieSocketAddressesInDefaultRack[i] = new BookieSocketAddress("127.0.0." + (i + 100), 3181); + StaticDNSResolver.addNodeToRack(bookieSocketAddressesInDefaultRack[i].getHostName(), + defaultRackForThisTest); + } - repp.onClusterChanged(bookieSocketAddresses, new HashSet()); + Set nonDefaultRackBookiesList = bookieSocketAddresses; + List defaultRackBookiesList = Arrays.asList(bookieSocketAddressesInDefaultRack); + Set writableBookies = new HashSet(nonDefaultRackBookiesList); + writableBookies.addAll(defaultRackBookiesList); + repp.onClusterChanged(writableBookies, new HashSet()); + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, + numBookiesInDefaultRackGauge.getSample()); /* * Though minNumRacksPerWriteQuorum is set to 4, since writeQuorum is 3 @@ -927,9 +976,12 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { */ StaticDNSResolver.addNodeToRack(newBookieAddress1.getHostName(), rackOfOtherBookieInEnsemble); bookieSocketAddresses.add(newBookieAddress1); + writableBookies.add(newBookieAddress1); bookieRackMap.put(newBookieAddress1, rackOfOtherBookieInEnsemble); - repp.onClusterChanged(bookieSocketAddresses, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, + numBookiesInDefaultRackGauge.getSample()); try { repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, ensemble, bookieInEnsembleToBeReplaced, new HashSet<>()); @@ -947,9 +999,12 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { */ StaticDNSResolver.addNodeToRack(newBookieAddress2.getHostName(), newRack); bookieSocketAddresses.add(newBookieAddress2); + writableBookies.add(newBookieAddress2); bookieRackMap.put(newBookieAddress2, newRack); - repp.onClusterChanged(bookieSocketAddresses, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, + numBookiesInDefaultRackGauge.getSample()); /* * this replaceBookie should succeed, because a new bookie is added to a * new rack. @@ -966,7 +1021,9 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { Set bookiesToExclude = new HashSet<>(); bookiesToExclude.add(newBookieAddress2); - repp.onClusterChanged(bookieSocketAddresses, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, + numBookiesInDefaultRackGauge.getSample()); try { repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, ensemble, bookieInEnsembleToBeReplaced, bookiesToExclude); @@ -984,9 +1041,12 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { */ StaticDNSResolver.addNodeToRack(newBookieAddress3.getHostName(), rackOfBookieToBeReplaced); bookieSocketAddresses.add(newBookieAddress3); + writableBookies.add(newBookieAddress3); bookieRackMap.put(newBookieAddress3, rackOfBookieToBeReplaced); - repp.onClusterChanged(bookieSocketAddresses, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, + numBookiesInDefaultRackGauge.getSample()); /* * here we have added new bookie to the rack of the bookie to be * replaced, so we should be able to replacebookie though @@ -2023,4 +2083,154 @@ public void testShuffleWithMask() { assertTrue(shuffleOccurred); } + @Test + public void testNumBookiesInDefaultRackGauge() throws Exception { + String defaultRackForThisTest = NetworkTopology.DEFAULT_REGION_AND_RACK; + repp.uninitalize(); + updateMyRack(defaultRackForThisTest); + + // Update cluster + BookieSocketAddress newAddr1 = new BookieSocketAddress("127.0.0.100", 3181); + BookieSocketAddress newAddr2 = new BookieSocketAddress("127.0.0.101", 3181); + BookieSocketAddress newAddr3 = new BookieSocketAddress("127.0.0.102", 3181); + BookieSocketAddress newAddr4 = new BookieSocketAddress("127.0.0.103", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(newAddr1.getHostName(), defaultRackForThisTest); + StaticDNSResolver.addNodeToRack(newAddr2.getHostName(), "/default-region/r2"); + StaticDNSResolver.addNodeToRack(newAddr3.getHostName(), "/default-region/r3"); + StaticDNSResolver.addNodeToRack(newAddr4.getHostName(), defaultRackForThisTest); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); + + repp = new RackawareEnsemblePlacementPolicy(); + repp.initialize(conf, Optional. empty(), timer, DISABLE_ALL, statsLogger); + repp.withDefaultRack(defaultRackForThisTest); + + Gauge numBookiesInDefaultRackGauge = statsLogger + .getGauge(BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK); + + Set writeableBookies = new HashSet(); + writeableBookies.add(newAddr1); + writeableBookies.add(newAddr2); + Set readOnlyBookies = new HashSet(); + readOnlyBookies.add(newAddr3); + readOnlyBookies.add(newAddr4); + repp.onClusterChanged(writeableBookies, readOnlyBookies); + // only writable bookie - newAddr1 in default rack + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", 1, numBookiesInDefaultRackGauge.getSample()); + + readOnlyBookies.remove(newAddr4); + writeableBookies.add(newAddr4); + repp.onClusterChanged(writeableBookies, readOnlyBookies); + // newAddr4 is also added to writable bookie so 2 writable bookies - + // newAddr1 and newAddr4 + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", 2, numBookiesInDefaultRackGauge.getSample()); + + // newAddr4 rack is changed and it is not in default anymore + StaticDNSResolver.changeRack(Arrays.asList(newAddr4), Arrays.asList("/default-region/r4")); + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", 1, numBookiesInDefaultRackGauge.getSample()); + + writeableBookies.clear(); + // writeableBookies is empty so 0 writable bookies in default rack + repp.onClusterChanged(writeableBookies, readOnlyBookies); + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", 0, numBookiesInDefaultRackGauge.getSample()); + + StaticDNSResolver.changeRack(Arrays.asList(newAddr1), Arrays.asList("/default-region/r2")); + readOnlyBookies.clear(); + writeableBookies.add(newAddr1); + writeableBookies.add(newAddr2); + writeableBookies.add(newAddr3); + writeableBookies.add(newAddr4); + repp.onClusterChanged(writeableBookies, readOnlyBookies); + // newAddr1 rack is changed and it is not in default anymore. So no + // bookies in default rack anymore + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", 0, numBookiesInDefaultRackGauge.getSample()); + } + + @Test + public void testNewEnsembleExcludesDefaultRackBookiesEnforceMinNumRacks() throws Exception { + String defaultRackForThisTest = NetworkTopology.DEFAULT_REGION_AND_RACK; + repp.uninitalize(); + updateMyRack(defaultRackForThisTest); + int minNumRacksPerWriteQuorum = 4; + ClientConfiguration clientConf = new ClientConfiguration(conf); + clientConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorum); + // set enforceMinNumRacksPerWriteQuorum + clientConf.setEnforceMinNumRacksPerWriteQuorum(true); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); + + repp = new RackawareEnsemblePlacementPolicy(); + repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, statsLogger); + repp.withDefaultRack(defaultRackForThisTest); + Gauge numBookiesInDefaultRackGauge = statsLogger + .getGauge(BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK); + + int writeQuorumSize = 3; + int ackQuorumSize = 3; + int effectiveMinNumRacksPerWriteQuorum = Math.min(minNumRacksPerWriteQuorum, writeQuorumSize); + + int numOfRacks = 2 * effectiveMinNumRacksPerWriteQuorum - 1; + int numOfBookiesPerRack = 20; + BookieSocketAddress[] bookieSocketAddresses = new BookieSocketAddress[numOfRacks * numOfBookiesPerRack]; + + for (int i = 0; i < numOfRacks; i++) { + for (int j = 0; j < numOfBookiesPerRack; j++) { + int index = i * numOfBookiesPerRack + j; + bookieSocketAddresses[index] = new BookieSocketAddress("128.0.0." + index, 3181); + StaticDNSResolver.addNodeToRack(bookieSocketAddresses[index].getHostName(), "/default-region/r" + i); + } + } + + int numOfBookiesInDefaultRack = 10; + BookieSocketAddress[] bookieSocketAddressesInDefaultRack = new BookieSocketAddress[numOfBookiesInDefaultRack]; + for (int i = 0; i < numOfBookiesInDefaultRack; i++) { + bookieSocketAddressesInDefaultRack[i] = new BookieSocketAddress("127.0.0." + (i + 100), 3181); + StaticDNSResolver.addNodeToRack(bookieSocketAddressesInDefaultRack[i].getHostName(), + defaultRackForThisTest); + } + + Set writableBookies = new HashSet( + Arrays.asList(bookieSocketAddresses)); + writableBookies.addAll(Arrays.asList(bookieSocketAddressesInDefaultRack)); + repp.onClusterChanged(writableBookies, new HashSet()); + assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, + numBookiesInDefaultRackGauge.getSample()); + + /* + * in this scenario we have enough number of racks (2 * + * effectiveMinNumRacksPerWriteQuorum - 1) and more number of bookies in + * each rack. So we should be able to create ensemble for all + * ensembleSizes (as long as there are enough number of bookies in each + * rack). + * + * Since minNumRacksPerWriteQuorum is enforced, it shouldn't select node + * from default rack. + */ + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; + List ensemble; + boolean isEnsembleAdheringToPlacementPolicy; + for (int ensembleSize = effectiveMinNumRacksPerWriteQuorum; ensembleSize < 40; ensembleSize++) { + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); + ensemble = ensembleResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + assertEquals("Number of writeQuorum sets covered", ensembleSize, + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + assertTrue(isEnsembleAdheringToPlacementPolicy); + + ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); + ensemble = ensembleResponse.getResult(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + assertEquals("Number of writeQuorum sets covered", ensembleSize, + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + assertTrue(isEnsembleAdheringToPlacementPolicy); + Collection bookiesOfDefaultRackInEnsemble = CollectionUtils + .intersection(Arrays.asList(bookieSocketAddressesInDefaultRack), ensemble); + assertTrue("Ensemble is not supposed to contain bookies from default rack, but ensemble contains - " + + bookiesOfDefaultRackInEnsemble, bookiesOfDefaultRackInEnsemble.isEmpty()); + } + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java index 68b5df2bf99..69134a3a626 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -52,10 +52,10 @@ import org.apache.bookkeeper.replication.AuditorPeriodicCheckTest.TestAuditor; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; -import org.apache.bookkeeper.test.TestStatsProvider.TestCounter; import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; import org.apache.bookkeeper.util.StaticDNSResolver; @@ -182,14 +182,14 @@ public void testPlacementPolicyCheckWithBookiesFromDifferentRacks() throws Excep MutableObject auditorRef = new MutableObject(); try { TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); - TestCounter placementPolicyCheckEnsembleNotAdheringToPlacementPolicy = (TestCounter) statsLogger.getCounter( - ReplicationStats.PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); /* * since all of the bookies are in different racks, there shouldn't be any ledger not adhering * to placement policy. */ - assertEquals("PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER SuccessCount", 0L, - placementPolicyCheckEnsembleNotAdheringToPlacementPolicy.get().longValue()); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", 0, + ledgersNotAdheringToPlacementPolicyGuage.getSample()); } finally { Auditor auditor = auditorRef.getValue(); if (auditor != null) { @@ -263,11 +263,10 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPlacementPolicy() th MutableObject auditorRef = new MutableObject(); try { TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); - TestCounter placementPolicyCheckEnsembleNotAdheringToPlacementPolicy = (TestCounter) statsLogger.getCounter( - ReplicationStats.PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER); - assertEquals("PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER SuccessCount", - (long) numOfLedgersNotAdheringToPlacementPolicy, - placementPolicyCheckEnsembleNotAdheringToPlacementPolicy.get().longValue()); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", + numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); } finally { Auditor auditor = auditorRef.getValue(); if (auditor != null) { @@ -362,11 +361,10 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPolicyWithMultipleSe MutableObject auditorRef = new MutableObject(); try { TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); - TestCounter placementPolicyCheckEnsembleNotAdheringToPlacementPolicy = (TestCounter) statsLogger.getCounter( - ReplicationStats.PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER); - assertEquals("PLACEMENT_POLICY_CHECK_ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER SuccessCount", - (long) numOfLedgersNotAdheringToPlacementPolicy, - placementPolicyCheckEnsembleNotAdheringToPlacementPolicy.get().longValue()); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY gauge value", + numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); } finally { Auditor auditor = auditorRef.getValue(); if (auditor != null) { From 7e7c5b8f92e3c97209f1f6a4ff6057f177e00b7e Mon Sep 17 00:00:00 2001 From: Like Date: Mon, 25 Feb 2019 13:35:00 +0800 Subject: [PATCH 0220/1642] Fix obsoleted proposals document link Closes #1531 Reviewers: Sijie Guo This closes #1959 from liketic/fix-link --- site/community/contributing.md | 2 +- site/community/issue-report.md | 2 +- site/community/meeting.md | 2 +- site/community/releases.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/site/community/contributing.md b/site/community/contributing.md index 49792b8c45a..7bf513b3c6a 100644 --- a/site/community/contributing.md +++ b/site/community/contributing.md @@ -54,7 +54,7 @@ Slack channels are great for quick questions or discussions on specialized topic To avoid potential frustration during the code review cycle, we encourage you to clearly scope and design non-trivial contributions with the BookKeeper community before you start coding. -We are using [BookKeeper Proposals](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BookKeeper+Proposals) for managing major changes to BookKeeper. +We are using [BookKeeper Proposals](http://bookkeeper.apache.org/community/bookkeeper_proposals/) for managing major changes to BookKeeper. ## Code diff --git a/site/community/issue-report.md b/site/community/issue-report.md index 21ff3696558..f7f5912e79f 100644 --- a/site/community/issue-report.md +++ b/site/community/issue-report.md @@ -28,7 +28,7 @@ Here is an very useful artical [How to report bugs effectively]( http://www.chia - Please describe the feature you are requesting. - Indicate the importance of this issue to you (_blocker_, _must-have_, _should-have_, _nice-to-have_). Are you currently using any workarounds to address this issue? - Provide any additional detail on your proposed use case for this feature. -- If it is a [BookKeeper Proposal](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BookKeeper+Proposals), please label this issue as `BP`. +- If it is a [BookKeeper Proposal](http://bookkeeper.apache.org/community/bookkeeper_proposals/), please label this issue as `BP`. #### If it is a **BUG REPORT** diff --git a/site/community/meeting.md b/site/community/meeting.md index 8936023de73..6bcd049c6b2 100644 --- a/site/community/meeting.md +++ b/site/community/meeting.md @@ -6,7 +6,7 @@ The community meeting runs bi-weekly on Thursday 8am - 9am PST. The meeting link The meeting is typically comprised of 3 parts: -- Discuss [BookKeeper Proposals](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BookKeeper+Proposals). +- Discuss [BookKeeper Proposals](http://bookkeeper.apache.org/community/bookkeeper_proposals/). - Review and address concerns for any open pull requests. - Open discussion. diff --git a/site/community/releases.md b/site/community/releases.md index 557f7ce2598..df556656d46 100644 --- a/site/community/releases.md +++ b/site/community/releases.md @@ -7,7 +7,7 @@ title: Release Management Apache BookKeeper community makes a feture release every 3 month. - A month before the release date, the release manager will cut branches and also publish a list of features that will be included in the release. These features will typically - be [BookKeeper Proposals](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BookKeeper+Proposals), but not always. + be [BookKeeper Proposals](http://bookkeeper.apache.org/community/bookkeeper_proposals/), but not always. - Another week will be left for *minor* features to get in, but at this point the community will start efforts to stabilize the release branch and contribute mostly tests and fixes. - Two weeks before the release date, the bookkeeper community will announce code-freeze and start rolling out release candidates, after which only fixes for blocking bugs will be merged. From 3fd9c82d9c664160a8b83e6ec801a3280d6268ce Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Sun, 24 Feb 2019 21:36:16 -0800 Subject: [PATCH 0221/1642] Log error (if any) in Shell.runCommand Descriptions of the changes in this PR: - Log error (if any) in Shell.runCommand even in the case of successful execution Reviewers: Enrico Olivelli , Sijie Guo This closes #1956 from reddycharan/printshellerror --- .../src/main/java/org/apache/bookkeeper/util/Shell.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java index 60c7648e0a3..8067a608968 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java @@ -176,6 +176,11 @@ public void run() { //taken care in finally block if (exitCode != 0) { throw new ExitCodeException(exitCode, errMsg.toString()); + } else { + String errMsgStr = errMsg.toString(); + if (!errMsgStr.isEmpty()) { + LOG.error(errMsgStr); + } } } catch (InterruptedException ie) { Thread.currentThread().interrupt(); From b65bd00067da07c825de3bbd90d096c2bda993cc Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sun, 24 Feb 2019 21:40:19 -0800 Subject: [PATCH 0222/1642] [BOOKIE-HTTP-SERVER] Fix: broken vertx rest endpoints ### Motivation Right now, vertx-http-server is not serving any rest endpoint except get because vertx server doesn't add put/post/delete http-methods into routing rules. ### Modification Add put/post/delete http-methods into routing rules. Reviewers: Sijie Guo This closes #1953 from rdhabalia/http_ep --- .../java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java b/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java index 15d1039ca3a..6d0bd63d7ac 100644 --- a/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java +++ b/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java @@ -70,6 +70,9 @@ public boolean startServer(int port) { @Override public void bindHandler(String endpoint, VertxAbstractHandler handler) { router.get(endpoint).handler(handler); + router.put(endpoint).handler(handler); + router.post(endpoint).handler(handler); + router.delete(endpoint).handler(handler); } }; requestRouter.bindAll(); From f3845d540ca7eac70746cba8765ff942d38dc208 Mon Sep 17 00:00:00 2001 From: Kieran Gorman Date: Mon, 25 Feb 2019 09:20:34 +0000 Subject: [PATCH 0223/1642] [DOC] popover for 'striped' as well as 'striping' Adds a popover term for 'striped' to match 'striping'. ### Motivation screen shot 2019-02-19 at 14 08 07 The first "link" text has no pop-over, but the second does. ### Changes Just copies the pop-over for striping. It seems inelegant vs. allowing multiple pop-over terms to refer to the same HTML fragment rather than being 1-1, but also is a straightforward and unobtrusive change. Master Issue: (no issue) Reviewers: Enrico Olivelli , Sijie Guo This closes #1951 from kjgorman/striped-popover --- site/_data/popovers.yaml | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/site/_data/popovers.yaml b/site/_data/popovers.yaml index 8b5347c95d7..1c4f5731510 100644 --- a/site/_data/popovers.yaml +++ b/site/_data/popovers.yaml @@ -12,11 +12,16 @@ - term: striping d: | Striping is the process of distributing BookKeeper ledgers to sub-groups of bookies rather than to all bookies in a BookKeeper ensemble. - + + Striping is essential to ensuring fast performance. +- term: striped + d: | + Striping is the process of distributing BookKeeper ledgers to sub-groups of bookies rather than to all bookies in a BookKeeper ensemble. + Striping is essential to ensuring fast performance. - term: journal d: A journal file stores BookKeeper transaction logs. - term: fencing d: When a reader forces a ledger to close, preventing any further entries from being written to the ledger. - term: record - d: A record is a sequence of bytes (plus some metadata) written to a BookKeeper ledger. Records are also known as entries. \ No newline at end of file + d: A record is a sequence of bytes (plus some metadata) written to a BookKeeper ledger. Records are also known as entries. From 41d2505661bcacc6f34daaa90a57fc09db7d6f64 Mon Sep 17 00:00:00 2001 From: "yong.zhang" Date: Wed, 27 Feb 2019 15:21:49 +0800 Subject: [PATCH 0224/1642] Fix unable download twitter dependencies --- Fixes #1962 *Motivation* There are some twitter dependencies can't download from repo *Modifications* - remove some using in StatsLoggerBenchmark.java - update some config in pom.xml Descriptions of the changes in this PR: You can add -Dtwitter to choose add twitter dependencies. ### Motivation (Explain: why you're making that change, what is the problem you're trying to solve) ### Changes (Describe: what changes you have made) Master Issue: #1962 Reviewers: Enrico Olivelli , Jia Zhai , Sijie Guo This closes #1966 from zymap/master --- bookkeeper-server/pom.xml | 20 ------------------ bookkeeper-stats-providers/pom.xml | 21 ++++++++++++++++--- microbenchmarks/pom.xml | 10 --------- .../stats/StatsLoggerBenchmark.java | 4 ---- 4 files changed, 18 insertions(+), 37 deletions(-) diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index a2d853865b9..1b131969f7f 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -241,26 +241,6 @@ - - twitter-science-provider - - - org.apache.bookkeeper.stats - twitter-science-provider - ${project.parent.version} - - - - - codahale-metrics-provider - - - org.apache.bookkeeper.stats - codahale-metrics-provider - ${project.parent.version} - - - twitter-http-server diff --git a/bookkeeper-stats-providers/pom.xml b/bookkeeper-stats-providers/pom.xml index b36d36961b0..75af272d56a 100644 --- a/bookkeeper-stats-providers/pom.xml +++ b/bookkeeper-stats-providers/pom.xml @@ -26,10 +26,25 @@ pom Apache BookKeeper :: Stats Providers - twitter-finagle-provider - twitter-science-provider - twitter-ostrich-provider codahale-metrics-provider prometheus-metrics-provider + + + + twitter + + + twitter + + + + + twitter-finagle-provider + twitter-science-provider + twitter-ostrich-provider + + + + diff --git a/microbenchmarks/pom.xml b/microbenchmarks/pom.xml index 8ec203cbec2..8aa66a7f98d 100644 --- a/microbenchmarks/pom.xml +++ b/microbenchmarks/pom.xml @@ -63,16 +63,6 @@ codahale-metrics-provider ${project.version} - - org.apache.bookkeeper.stats - twitter-science-provider - ${project.version} - - - org.apache.bookkeeper.stats - twitter-ostrich-provider - ${project.version} - diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/StatsLoggerBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/StatsLoggerBenchmark.java index 0a2cd063800..3986ffadd83 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/StatsLoggerBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/StatsLoggerBenchmark.java @@ -30,8 +30,6 @@ import org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider; import org.apache.bookkeeper.stats.codahale.FastCodahaleMetricsProvider; import org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider; -import org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider; -import org.apache.bookkeeper.stats.twitter.science.TwitterStatsProvider; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Level; @@ -60,8 +58,6 @@ public class StatsLoggerBenchmark { providers.put("Prometheus", PrometheusMetricsProvider::new); providers.put("Codahale", CodahaleMetricsProvider::new); providers.put("FastCodahale", FastCodahaleMetricsProvider::new); - providers.put("Twitter", TwitterStatsProvider::new); - providers.put("Ostrich", OstrichProvider::new); } @State(Scope.Benchmark) From 923fc6e33ca1e985717a8c8800a29791cede48fb Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 26 Feb 2019 23:37:46 -0800 Subject: [PATCH 0225/1642] Add http method test for vertx-http server ### Motivation As discussed at #1953, adding test to verify all http-method works for vertx-http-server. Reviewers: Enrico Olivelli , Sijie Guo This closes #1963 from rdhabalia/gc_test --- .../http/vertx/TestVertxHttpServer.java | 26 +++++++++++++++---- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java b/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java index 1c547d3ad6f..c6cc13301db 100644 --- a/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java +++ b/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java @@ -47,7 +47,7 @@ public void testStartBasicHttpServer() throws Exception { httpServer.initialize(httpServiceProvider); assertTrue(httpServer.startServer(0)); int port = httpServer.getListeningPort(); - HttpResponse httpResponse = sendGet(getUrl(port, HttpRouter.HEARTBEAT)); + HttpResponse httpResponse = send(getUrl(port, HttpRouter.HEARTBEAT), HttpServer.Method.GET); assertEquals(HttpServer.StatusCode.OK.getValue(), httpResponse.responseCode); assertEquals(HeartbeatService.HEARTBEAT.trim(), httpResponse.responseBody.trim()); httpServer.stopServer(); @@ -60,17 +60,33 @@ public void testStartMetricsServiceOnRouterPath() throws Exception { httpServer.initialize(httpServiceProvider); assertTrue(httpServer.startServer(0)); int port = httpServer.getListeningPort(); - HttpResponse httpResponse = sendGet(getUrl(port, HttpRouter.METRICS)); + HttpResponse httpResponse = send(getUrl(port, HttpRouter.METRICS), HttpServer.Method.GET); assertEquals(HttpServer.StatusCode.OK.getValue(), httpResponse.responseCode); httpServer.stopServer(); } - // HTTP GET request - private HttpResponse sendGet(String url) throws IOException { + @Test + public void testHttpMethods() throws Exception { + VertxHttpServer httpServer = new VertxHttpServer(); + HttpServiceProvider httpServiceProvider = NullHttpServiceProvider.getInstance(); + httpServer.initialize(httpServiceProvider); + assertTrue(httpServer.startServer(0)); + int port = httpServer.getListeningPort(); + HttpResponse httpResponse = send(getUrl(port, HttpRouter.GC), HttpServer.Method.GET); + assertEquals(HttpServer.StatusCode.OK.getValue(), httpResponse.responseCode); + httpResponse = send(getUrl(port, HttpRouter.GC), HttpServer.Method.POST); + assertEquals(HttpServer.StatusCode.OK.getValue(), httpResponse.responseCode); + httpResponse = send(getUrl(port, HttpRouter.GC), HttpServer.Method.PUT); + assertEquals(HttpServer.StatusCode.OK.getValue(), httpResponse.responseCode); + httpServer.stopServer(); + } + + // HTTP request + private HttpResponse send(String url, HttpServer.Method method) throws IOException { URL obj = new URL(url); HttpURLConnection con = (HttpURLConnection) obj.openConnection(); // optional, default is GET - con.setRequestMethod("GET"); + con.setRequestMethod(method.toString()); int responseCode = con.getResponseCode(); StringBuilder response = new StringBuilder(); BufferedReader in = null; From 28c885cb61f6341ec9845b6e1a15e7a9a11624a2 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Tue, 26 Feb 2019 23:50:45 -0800 Subject: [PATCH 0226/1642] Use conf value in RackawareEnsemblePlacementPolicyImpl.initialize method Descriptions of the changes in this PR: - Use conf value in RackawareEnsemblePlacementPolicyImpl.initialize method, instead of uninitialized enforceMinNumRacksPerWriteQuorum variable Reviewers: Enrico Olivelli , Sijie Guo This closes #1964 from reddycharan/fixexceptioncondition --- .../RackawareEnsemblePlacementPolicyImpl.java | 2 +- ...areEnsemblePlacementPolicyUsingScript.java | 45 +++++++++++++++++++ 2 files changed, 46 insertions(+), 1 deletion(-) 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 ef4341ae5b3..4cbe25bc9f8 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 @@ -386,7 +386,7 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, ((RackChangeNotifier) dnsResolver).registerRackChangeListener(this); } } catch (RuntimeException re) { - if (!enforceMinNumRacksPerWriteQuorum) { + if (!conf.getEnforceMinNumRacksPerWriteQuorum()) { LOG.info("Failed to initialize DNS Resolver {}, used default subnet resolver : {}", dnsResolverName, re, re.getMessage()); dnsResolver = new DefaultResolver(() -> this.getDefaultRack()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java index 2aff5d88201..fc3caaf2966 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java @@ -358,6 +358,51 @@ public void testRemoveBookieFromCluster() { repp.onClusterChanged(addrs, new HashSet()); } + @Test + public void testNetworkTopologyScriptFileNameIsEmpty() throws Exception { + ignoreTestIfItIsWindowsOS(); + repp.uninitalize(); + + ClientConfiguration newConf = new ClientConfiguration(); + newConf.setProperty(REPP_DNS_RESOLVER_CLASS, ScriptBasedMapping.class.getName()); + newConf.setProperty(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, ""); + newConf.setEnforceMinNumRacksPerWriteQuorum(false); + timer = new HashedWheelTimer(new ThreadFactoryBuilder().setNameFormat("TestTimer-%d").build(), + newConf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS, newConf.getTimeoutTimerNumTicks()); + + repp = new RackawareEnsemblePlacementPolicy(); + try { + repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE); + } catch (RuntimeException re) { + fail("EnforceMinNumRacksPerWriteQuorum is not set, so repp.initialize should succeed even if" + + " networkTopologyScriptFileName is empty"); + } + repp.uninitalize(); + + newConf.setEnforceMinNumRacksPerWriteQuorum(true); + repp = new RackawareEnsemblePlacementPolicy(); + try { + repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE); + fail("EnforceMinNumRacksPerWriteQuorum is set, so repp.initialize should fail if" + + " networkTopologyScriptFileName is empty"); + } catch (RuntimeException re) { + } + repp.uninitalize(); + + newConf.setProperty(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, + "src/test/resources/networkmappingscript.sh"); + try { + repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE); + } catch (RuntimeException re) { + fail("EnforceMinNumRacksPerWriteQuorum is set and networkTopologyScriptFileName is not empty," + + " so it should succeed"); + } + repp.uninitalize(); + } + private int getNumCoveredWriteQuorums(List ensemble, int writeQuorumSize) throws Exception { int ensembleSize = ensemble.size(); From ac5f509af10bb434c39554fd193ffa345947db0a Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 28 Feb 2019 23:13:21 +0800 Subject: [PATCH 0227/1642] [PYTHON] Fix python client version Descriptions of the changes in this PR: *Motivation* pip doesn't work with '-SNAPSHOT'. Use `-alpha-0` instead of `-SNAPSHOT` for python. *Modifications* - update setup.py file - update release guide Reviewers: Jia Zhai This closes #1954 from sijie/fix_python_version --- site/community/release_guide.md | 7 ++++--- stream/clients/python/setup.py | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/site/community/release_guide.md b/site/community/release_guide.md index bb5834dda15..c8a05a3a83f 100644 --- a/site/community/release_guide.md +++ b/site/community/release_guide.md @@ -169,7 +169,7 @@ The list of release-blocking issues is available at the [milestones page](https: Before cutting a release, you need to update the python client version in [setup.py](https://github.com/apache/bookkeeper/blob/master/stream/clients/python/setup.py#L22) from `SNAPSHOT` version to a release version and get the change merge to master. For example, -in release 4.10.0, you need to change the version from `4.10.0-SNAPSHOT` to `4.10.0`. +in release 4.10.0, you need to change the version from `4.10.0-alpha-0` to `4.10.0`. ### Review Release Notes in Github @@ -625,10 +625,11 @@ Example PR: [release-4.7.0](https://github.com/apache/bookkeeper/pull/1350) If you are doing a major release, you need to update the python client version to next major development version in master and next minor development version in the branch. For example, if you are doing 4.9.0 release, you need to bump the version -in master to `4.10.0-SNAPSHOT`, and the version in `branch-4.9` to `4.9.1-SNAPSHOT`. +in master to `4.10.0-alpha-0` (NOTE: we are using `alpha-0` as `SNAPSHOT`, otherwise pypi doesn't work), and the version in +`branch-4.9` to `4.9.1-alpha-0`. If you are only doing a minor release, you just need to update the version in release branch. For example, if you are doing -4.9.1 release, you need to bump the version in `branch-4.9` to `4.9.2-SNAPSHOT`. +4.9.1 release, you need to bump the version in `branch-4.9` to `4.9.2-alpha-0`. ### Mark the version as released in Github diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index dbbf4435bac..fb8a696566a 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -19,7 +19,7 @@ name = 'apache-bookkeeper-client' description = 'Apache BookKeeper client library' -version = '4.10.0-SNAPSHOT' +version = '4.10.0-alpha-0' # Should be one of: # 'Development Status :: 3 - Alpha' # 'Development Status :: 4 - Beta' From be6f2b50314c5a3450728082df0919732e381146 Mon Sep 17 00:00:00 2001 From: Like Date: Sat, 2 Mar 2019 23:20:53 +0800 Subject: [PATCH 0228/1642] Use automatic resource management to close streams We can use automatic resource management introduced in java 7, which makes code shorter. Reviewers: Sijie Guo This closes #1958 from liketic/use-resource --- .../bookkeeper/bookie/BookieStatus.java | 11 ++-------- .../org/apache/bookkeeper/bookie/Cookie.java | 11 ++-------- .../apache/bookkeeper/bookie/EntryLogger.java | 20 +++---------------- 3 files changed, 7 insertions(+), 35 deletions(-) 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 index 49aa66ed237..ecdf82e9d64 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStatus.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStatus.java @@ -123,16 +123,9 @@ synchronized void writeToDirectories(List directories) { * @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)); + try (FileOutputStream fos = new FileOutputStream(file); + BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(fos, UTF_8))) { bw.write(body); - } finally { - if (bw != null) { - bw.close(); - } - fos.close(); } } 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 7d4175cd6f6..1ddc5abd7ed 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 @@ -217,16 +217,9 @@ public void writeToDirectory(File directory) throws IOException { } public void writeToFile (File versionFile) throws IOException { - FileOutputStream fos = new FileOutputStream(versionFile); - BufferedWriter bw = null; - try { - bw = new BufferedWriter(new OutputStreamWriter(fos, UTF_8)); + try (FileOutputStream fos = new FileOutputStream(versionFile); + BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(fos, UTF_8))) { bw.write(toString()); - } finally { - if (bw != null) { - bw.close(); - } - fos.close(); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index 1389370a461..55ac6111b34 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -35,7 +35,6 @@ import io.netty.util.concurrent.FastThreadLocal; import java.io.BufferedReader; import java.io.File; -import java.io.FileFilter; import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.FilenameFilter; @@ -547,12 +546,7 @@ private long getLastLogId(File dir) { return id; } // read failed, scan the ledger directories to find biggest log id - File[] logFiles = dir.listFiles(new FileFilter() { - @Override - public boolean accept(File file) { - return file.getName().endsWith(".log"); - } - }); + File[] logFiles = dir.listFiles(file -> file.getName().endsWith(".log")); List logs = new ArrayList(); if (logFiles != null) { for (File lf : logFiles) { @@ -579,19 +573,11 @@ private long readLastLogId(File f) { } catch (FileNotFoundException e) { return INVALID_LID; } - BufferedReader br = new BufferedReader(new InputStreamReader(fis, UTF_8)); - try { + try (BufferedReader br = new BufferedReader(new InputStreamReader(fis, UTF_8))) { String lastIdString = br.readLine(); return Long.parseLong(lastIdString, 16); - } catch (IOException e) { + } catch (IOException | NumberFormatException e) { return INVALID_LID; - } catch (NumberFormatException e) { - return INVALID_LID; - } finally { - try { - br.close(); - } catch (IOException e) { - } } } From 26a335bd3ea3d4f74f634929f1de53b0af6ac762 Mon Sep 17 00:00:00 2001 From: Like Date: Sun, 3 Mar 2019 23:14:59 +0800 Subject: [PATCH 0229/1642] Use the same thread to monitor ledger and index directories Closes #1655 Reviewers: Charan Reddy Guttapalem , Sijie Guo This closes #1957 from liketic/reduce-monitor-threads --- .../org/apache/bookkeeper/bookie/Bookie.java | 79 ++++--------------- .../bookkeeper/bookie/LedgerDirsMonitor.java | 29 ++++--- .../bookie/BookieInitializationTest.java | 4 +- .../bookie/BookieStorageThresholdTest.java | 19 +++-- .../bookkeeper/bookie/CompactionTest.java | 2 +- .../bookie/IndexPersistenceMgrTest.java | 4 +- .../bookie/TestLedgerDirsManager.java | 11 +-- 7 files changed, 57 insertions(+), 91 deletions(-) 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 a0acd31c43d..16b63121a9c 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 @@ -94,7 +94,6 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.tuple.Pair; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -123,10 +122,8 @@ public class Bookie extends BookieCriticalThread { static final long METAENTRY_ID_LEDGER_EXPLICITLAC = -0x8000; private final LedgerDirsManager ledgerDirsManager; - private LedgerDirsManager indexDirsManager; - - LedgerDirsMonitor ledgerMonitor; - LedgerDirsMonitor idxMonitor; + private final LedgerDirsManager indexDirsManager; + LedgerDirsMonitor dirsMonitor; // Registration Manager for managing registration protected final MetadataBookieDriver metadataDriver; @@ -232,7 +229,7 @@ public boolean accept(File dir, String name) { private void checkEnvironment(MetadataBookieDriver metadataDriver) throws BookieException, IOException { List allLedgerDirs = new ArrayList(ledgerDirsManager.getAllLedgerDirs().size() - + indexDirsManager.getAllLedgerDirs().size()); + + indexDirsManager.getAllLedgerDirs().size()); allLedgerDirs.addAll(ledgerDirsManager.getAllLedgerDirs()); if (indexDirsManager != ledgerDirsManager) { allLedgerDirs.addAll(indexDirsManager.getAllLedgerDirs()); @@ -705,12 +702,17 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger, ByteBufAllocato stateManager = initializeStateManager(); // register shutdown handler using trigger mode stateManager.setShutdownHandler(exitCode -> triggerBookieShutdown(exitCode)); - // Initialise ledgerDirMonitor. This would look through all the + // Initialise dirsMonitor. This would look through all the // configured directories. When disk errors or all the ledger // directories are full, would throws exception and fail bookie startup. - this.ledgerMonitor = new LedgerDirsMonitor(conf, diskChecker, ledgerDirsManager); + List dirsManagers = new ArrayList<>(); + dirsManagers.add(ledgerDirsManager); + if (indexDirsManager != ledgerDirsManager) { + dirsManagers.add(indexDirsManager); + } + this.dirsMonitor = new LedgerDirsMonitor(conf, diskChecker, dirsManagers); try { - this.ledgerMonitor.init(); + this.dirsMonitor.init(); } catch (NoWritableLedgerDirException nle) { // start in read-only mode if no writable dirs and read-only allowed if (!conf.isReadOnlyModeEnabled()) { @@ -720,23 +722,6 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger, ByteBufAllocato } } - if (ledgerDirsManager == indexDirsManager) { - this.idxMonitor = this.ledgerMonitor; - } else { - this.idxMonitor = new LedgerDirsMonitor(conf, diskChecker, indexDirsManager); - try { - this.idxMonitor.init(); - } catch (NoWritableLedgerDirException nle) { - // start in read-only mode if no writable dirs and read-only allowed - if (!conf.isReadOnlyModeEnabled()) { - throw nle; - } else { - this.stateManager.transitionToReadOnlyMode(); - } - } - } - - // instantiate the journals journals = Lists.newArrayList(); for (int i = 0; i < journalDirectories.size(); i++) { @@ -912,22 +897,15 @@ public synchronized void start() { journalDirectories.stream().map(File::getName).collect(Collectors.joining(", "))); } //Start DiskChecker thread - ledgerMonitor.start(); - if (indexDirsManager != ledgerDirsManager) { - idxMonitor.start(); - } + dirsMonitor.start(); // replay journals try { readJournal(); - } catch (IOException ioe) { + } catch (IOException | BookieException ioe) { LOG.error("Exception while replaying journals, shutting down", ioe); shutdown(ExitCode.BOOKIE_EXCEPTION); return; - } catch (BookieException be) { - LOG.error("Exception while replaying journals, shutting down", be); - shutdown(ExitCode.BOOKIE_EXCEPTION); - return; } // Do a fully flush after journal replay @@ -1184,11 +1162,7 @@ synchronized int shutdown(int exitCode) { } //Shutdown disk checker - ledgerMonitor.shutdown(); - if (indexDirsManager != ledgerDirsManager) { - idxMonitor.shutdown(); - } - + dirsMonitor.shutdown(); } // Shutdown the ZK client if (metadataDriver != null) { @@ -1350,10 +1324,9 @@ public void forceLedger(long ledgerId, WriteCallback cb, /** * Add entry to a ledger. - * @throws BookieException.LedgerFencedException if the ledger is fenced */ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx, byte[] masterKey) - throws IOException, BookieException.LedgerFencedException, BookieException, InterruptedException { + throws IOException, BookieException, InterruptedException { long requestNanos = MathUtils.nowInNano(); boolean success = false; int entrySize = 0; @@ -1385,26 +1358,6 @@ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Obj } } - static class FutureWriteCallback implements WriteCallback { - - SettableFuture result = SettableFuture.create(); - - @Override - public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { - if (LOG.isDebugEnabled()) { - LOG.debug("Finished writing entry {} @ ledger {} for {} : {}", - entryId, ledgerId, addr, rc); - } - - result.set(0 == rc); - } - - public SettableFuture getResult() { - return result; - } - } - /** * Fences a ledger. From this point on, clients will be unable to * write to this ledger. Only recoveryAddEntry will be @@ -1591,7 +1544,7 @@ private static boolean cleanDir(File dir) { * @throws InterruptedException */ public static void main(String[] args) - throws IOException, InterruptedException, BookieException, KeeperException { + throws IOException, InterruptedException, BookieException { Bookie b = new Bookie(new ServerConfiguration()); b.start(); CounterCallback cb = new CounterCallback(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java index d0033669320..2b7c90152d1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java @@ -50,25 +50,24 @@ class LedgerDirsMonitor { private final int interval; private final ServerConfiguration conf; - private final ConcurrentMap diskUsages; private final DiskChecker diskChecker; - private final LedgerDirsManager ldm; + private final List dirsManagers; private long minUsableSizeForHighPriorityWrites; private ScheduledExecutorService executor; private ScheduledFuture checkTask; public LedgerDirsMonitor(final ServerConfiguration conf, final DiskChecker diskChecker, - final LedgerDirsManager ldm) { + final List dirsManagers) { this.interval = conf.getDiskCheckInterval(); this.minUsableSizeForHighPriorityWrites = conf.getMinUsableSizeForHighPriorityWrites(); this.conf = conf; this.diskChecker = diskChecker; - this.diskUsages = ldm.getDiskUsages(); - this.ldm = ldm; + this.dirsManagers = dirsManagers; } - private void check() { + private void check(final LedgerDirsManager ldm) { + final ConcurrentMap diskUsages = ldm.getDiskUsages(); try { List writableDirs = ldm.getWritableLedgerDirs(); // Check all writable dirs disk space usage. @@ -171,6 +170,10 @@ private void check() { } } + private void check() { + dirsManagers.forEach(this::check); + } + /** * Sweep through all the directories to check disk errors or disk full. * @@ -181,7 +184,7 @@ private void check() { * less space than threshold */ public void init() throws DiskErrorException, NoWritableLedgerDirException { - checkDirs(ldm.getWritableLedgerDirs()); + checkDirs(); } // start the daemon for disk monitoring @@ -191,7 +194,7 @@ public void start() { .setNameFormat("LedgerDirsMonitorThread") .setDaemon(true) .build()); - this.checkTask = this.executor.scheduleAtFixedRate(() -> check(), interval, interval, TimeUnit.MILLISECONDS); + this.checkTask = this.executor.scheduleAtFixedRate(this::check, interval, interval, TimeUnit.MILLISECONDS); } // shutdown disk monitoring daemon @@ -207,9 +210,15 @@ public void shutdown() { } } - public void checkDirs(List writableDirs) + private void checkDirs() throws NoWritableLedgerDirException, DiskErrorException { + for (LedgerDirsManager dirsManager : dirsManagers) { + checkDirs(dirsManager); + } + } + + private void checkDirs(final LedgerDirsManager ldm) throws DiskErrorException, NoWritableLedgerDirException { - for (File dir : writableDirs) { + for (File dir : ldm.getWritableLedgerDirs()) { try { diskChecker.checkDir(dir); } catch (DiskWarnThresholdException 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 7dde6fdf0a2..c4ed03135bd 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 @@ -50,6 +50,7 @@ import java.net.URLConnection; import java.security.AccessControlException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Random; @@ -812,7 +813,8 @@ public void testWithDiskError() throws Exception { LedgerDirsManager ldm = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); LedgerDirsMonitor ledgerMonitor = new LedgerDirsMonitor(conf, - new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), ldm); + new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), + Collections.singletonList(ldm)); ledgerMonitor.init(); fail("should throw exception"); } catch (Exception e) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStorageThresholdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStorageThresholdTest.java index 6f7d1c9887f..0bd2a9736c6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStorageThresholdTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStorageThresholdTest.java @@ -26,6 +26,7 @@ import static org.junit.Assert.fail; import java.io.File; +import java.util.Collections; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -157,9 +158,9 @@ public void testStorageThresholdCompaction() throws Exception { bs.add(server); bsConfs.add(conf); Bookie bookie = server.getBookie(); - // since we are going to set dependency injected ledgermonitor, so we need to shutdown - // the ledgermonitor which was created as part of the initialization of Bookie - bookie.ledgerMonitor.shutdown(); + // since we are going to set dependency injected dirsMonitor, so we need to shutdown + // the dirsMonitor which was created as part of the initialization of Bookie + bookie.dirsMonitor.shutdown(); LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager(); @@ -183,13 +184,11 @@ public void diskFull(File disk) { // Dependency Injected class ThresholdTestDiskChecker thresholdTestDiskChecker = new ThresholdTestDiskChecker( baseConf.getDiskUsageThreshold(), baseConf.getDiskUsageWarnThreshold()); - LedgerDirsMonitor ledgerDirsMonitor = new LedgerDirsMonitor(baseConf, thresholdTestDiskChecker, - ledgerDirsManager); - // set the ledgermonitor and idxmonitor and initiate/start it - bookie.ledgerMonitor = ledgerDirsMonitor; - bookie.idxMonitor = ledgerDirsMonitor; - bookie.ledgerMonitor.init(); - bookie.ledgerMonitor.start(); + bookie.dirsMonitor = new LedgerDirsMonitor(baseConf, thresholdTestDiskChecker, + Collections.singletonList(ledgerDirsManager)); + // set the dirsMonitor and initiate/start it + bookie.dirsMonitor.init(); + bookie.dirsMonitor.start(); // create ledgers and add fragments LedgerHandle[] lhs = prepareData(3); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java index 3d7d2c4aa1a..0d6e5d64775 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java @@ -451,7 +451,7 @@ public void testMinorCompactionWithNoWritableLedgerDirsButIsForceGCAllowWhenNoSp for (BookieServer bookieServer : bs) { Bookie bookie = bookieServer.getBookie(); bookie.ledgerStorage.flush(); - bookie.ledgerMonitor.shutdown(); + bookie.dirsMonitor.shutdown(); LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager(); List ledgerDirs = ledgerDirsManager.getAllLedgerDirs(); // Major and Minor compaction are not disabled even though discs are full. Check LedgerDirsListener of diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java index 909e646b2c7..5b31b7183a0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java @@ -38,6 +38,7 @@ import java.nio.channels.FileChannel; import java.security.GeneralSecurityException; import java.util.Arrays; +import java.util.Collections; import org.apache.bookkeeper.bookie.FileInfoBackingCache.CachedFileInfo; import org.apache.bookkeeper.client.BookKeeper; @@ -82,7 +83,8 @@ public void setUp() throws Exception { ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); ledgerMonitor = new LedgerDirsMonitor(conf, - new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), ledgerDirsManager); + new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), + Collections.singletonList(ledgerDirsManager)); ledgerMonitor.init(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java index 9d166a4ebf5..e0a4838e43c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java @@ -32,6 +32,7 @@ import java.io.IOException; import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -112,7 +113,7 @@ public void setUp() throws Exception { dirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), statsLogger); ledgerMonitor = new LedgerDirsMonitor(conf, - mockDiskChecker, dirsManager); + mockDiskChecker, Collections.singletonList(dirsManager)); ledgerMonitor.init(); } @@ -215,7 +216,7 @@ private void testLedgerDirsMonitorDuringTransition(boolean highPriorityWritesAll conf.setMinUsableSizeForHighPriorityWrites(curDir.getUsableSpace() + 1024); dirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), statsLogger); - ledgerMonitor = new LedgerDirsMonitor(conf, mockDiskChecker, dirsManager); + ledgerMonitor = new LedgerDirsMonitor(conf, mockDiskChecker, Collections.singletonList(dirsManager)); ledgerMonitor.init(); } @@ -257,7 +258,7 @@ public void testLedgerDirsMonitorHandlingLowWaterMark() throws Exception { mockDiskChecker = new MockDiskChecker(nospace, warnThreshold); dirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); - ledgerMonitor = new LedgerDirsMonitor(conf, mockDiskChecker, dirsManager); + ledgerMonitor = new LedgerDirsMonitor(conf, mockDiskChecker, Collections.singletonList(dirsManager)); ledgerMonitor.init(); final MockLedgerDirsListener mockLedgerDirsListener = new MockLedgerDirsListener(); dirsManager.addLedgerDirsListener(mockLedgerDirsListener); @@ -321,7 +322,7 @@ public void testLedgerDirsMonitorHandlingWithMultipleLedgerDirectories() throws dirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), statsLogger); - ledgerMonitor = new LedgerDirsMonitor(conf, mockDiskChecker, dirsManager); + ledgerMonitor = new LedgerDirsMonitor(conf, mockDiskChecker, Collections.singletonList(dirsManager)); usageMap = new HashMap(); usageMap.put(curDir1, 0.1f); usageMap.put(curDir2, 0.1f); @@ -412,7 +413,7 @@ public void testLedgerDirsMonitorStartReadOnly() throws Exception { new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), statsLogger); - ledgerMonitor = new LedgerDirsMonitor(conf, mockDiskChecker, dirsManager); + ledgerMonitor = new LedgerDirsMonitor(conf, mockDiskChecker, Collections.singletonList(dirsManager)); try { ledgerMonitor.init(); fail("NoWritableLedgerDirException expected"); From a5f142938823f4fcd01cdea3eae1f3f9581efc2c Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Tue, 5 Mar 2019 08:49:23 +0100 Subject: [PATCH 0230/1642] [DLOG] Avoid double read in readahead There was a bug in the ReadAheadReader whereby, if it was in a paused state and the last segment it had read had been closed, then when a writer started writing new segments, the ReadAheadReader would issue a double read. This would end up triggering an error when it got to the end of the segment as the issued one of the issued reads would then be cancelled when the segment was closed. This fix adds an explicit state machine to the reader and moves all state transitions to run on the ordered executor, so only one read can ever be issued at a time. Reviewers: Enrico Olivelli , Sijie Guo This closes #1973 from ivankelly/dlog-reader-crash-master --- .../distributedlog/ReadAheadEntryReader.java | 121 ++++++++++++------ .../distributedlog/TestCancelledRead.java | 76 +++++++++++ 2 files changed, 157 insertions(+), 40 deletions(-) create mode 100644 stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java index c9bca44400d..ccd42ebaec4 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java @@ -67,6 +67,15 @@ class ReadAheadEntryReader implements private static final Logger logger = LoggerFactory.getLogger(ReadAheadEntryReader.class); + private enum State { + IDLE, + READING, + PAUSED, + CLOSED, + ERROR + } + private State state = State.IDLE; + // // Static Functions // @@ -460,6 +469,11 @@ public CompletableFuture asyncClose() { } private void unsafeAsyncClose(CompletableFuture closePromise) { + if (logger.isDebugEnabled()) { + logger.debug("[{}][state:{}] Closing read ahead", streamName, state); + } + state = State.CLOSED; + List> closeFutures = Lists.newArrayListWithExpectedSize( segmentReaders.size() + segmentReadersToClose.size() + 1); if (null != currentSegmentReader) { @@ -512,6 +526,16 @@ void setLastException(IOException cause) { } // the exception is set and notify the state change notifyStateChangeOnFailure(cause); + + orderedSubmit(new CloseableRunnable() { + @Override + public void safeRun() { + if (logger.isDebugEnabled()) { + logger.debug("[{}][state:{}] Read ahead errored", streamName, state); + } + state = State.ERROR; + } + }); } void checkLastException() throws IOException { @@ -571,15 +595,15 @@ public void onSuccess(List entries) { } // notify on data available notifyStateChangeOnSuccess(); - if (entryQueue.size() >= maxCachedEntries) { - pauseReadAheadOnCacheFull(); - } else { - scheduleReadNext(); - } + + completeRead(); + scheduleRead(); } @Override public void onFailure(Throwable cause) { + completeRead(); + if (cause instanceof EndOfLogSegmentException) { // we reach end of the log segment moveToNextLogSegment(); @@ -592,24 +616,6 @@ public void onFailure(Throwable cause) { } } - private synchronized void invokeReadAhead() { - if (readAheadPaused) { - scheduleReadNext(); - readAheadPaused = false; - } - } - - private synchronized void pauseReadAheadOnCacheFull() { - this.readAheadPaused = true; - if (!isCacheFull()) { - invokeReadAhead(); - } - } - - private synchronized void pauseReadAheadOnNoMoreLogSegments() { - this.readAheadPaused = true; - } - // // Cache Related Methods // @@ -630,7 +636,7 @@ public Entry.Reader getNextReadAheadEntry(long waitTime, TimeUnit waitTimeUnit) } finally { // resume readahead if the cache becomes empty if (null != entry && !isCacheFull()) { - invokeReadAhead(); + scheduleRead(); } } } @@ -784,7 +790,7 @@ private void unsafeReinitializeLogSegments(List segments) { unsafeMoveToNextLogSegment(); } // resume readahead if necessary - invokeReadAhead(); + scheduleRead(); } /** @@ -839,7 +845,9 @@ private void unsafeInitializeLogSegments(List segments) { currentSegmentReader.openReader(); currentSegmentReader.startRead(); currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber(); - unsafeReadNext(currentSegmentReader); + + scheduleRead(); + if (!segmentReaders.isEmpty()) { for (SegmentReader reader : segmentReaders) { reader.openReader(); @@ -921,7 +929,6 @@ private void unsafeMoveToNextLogSegment() { currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber(); nextSegmentReader = null; // start reading - unsafeReadNext(currentSegmentReader); unsafePrefetchNextSegment(true); hasSegmentToRead = true; } else { @@ -931,7 +938,6 @@ private void unsafeMoveToNextLogSegment() { logger.debug("move to read segment {}", currentSegmentReader.getSegment()); currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber(); nextSegmentReader = null; - unsafeReadNext(currentSegmentReader); unsafePrefetchNextSegment(true); hasSegmentToRead = true; } @@ -942,25 +948,60 @@ private void unsafeMoveToNextLogSegment() { readHandler.getFullyQualifiedName()); isCatchingUp = false; } - pauseReadAheadOnNoMoreLogSegments(); } + + scheduleRead(); } - void scheduleReadNext() { + void completeRead() { orderedSubmit(new CloseableRunnable() { - @Override - public void safeRun() { - if (null == currentSegmentReader) { - pauseReadAheadOnNoMoreLogSegments(); - return; + @Override + public void safeRun() { + if (logger.isDebugEnabled()) { + logger.debug("[{}][state:{}] Read completed", streamName, state); + } + if (state == State.READING) { + state = State.IDLE; + } } - unsafeReadNext(currentSegmentReader); - } - }); + }); } - private void unsafeReadNext(SegmentReader reader) { - reader.readNext().whenComplete(this); + void scheduleRead() { + orderedSubmit(new CloseableRunnable() { + @Override + public void safeRun() { + + boolean cacheFull = isCacheFull(); + SegmentReader reader = currentSegmentReader; + boolean hasMoreSegments = reader != null; + if (logger.isDebugEnabled()) { + logger.debug("[{}][state:{}] scheduling read, cacheFull {}, hasMoreSegments {}", + streamName, state, cacheFull, hasMoreSegments); + } + switch (state) { + case IDLE: + if (cacheFull || !hasMoreSegments) { + state = State.PAUSED; + } else { + reader.readNext().whenComplete(ReadAheadEntryReader.this); + state = State.READING; + } + break; + case PAUSED: + if (!cacheFull && hasMoreSegments) { + reader.readNext().whenComplete(ReadAheadEntryReader.this); + state = State.READING; + } + break; + case READING: + case ERROR: + case CLOSED: + // do nothing + break; + } + } + }); } @Override diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java new file mode 100644 index 00000000000..40cf6326884 --- /dev/null +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java @@ -0,0 +1,76 @@ +/** + * 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.distributedlog; + +import static org.junit.Assert.assertNotNull; + +import java.util.concurrent.CompletableFuture; +import org.apache.distributedlog.api.LogWriter; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test Cases for RollLogSegments. + */ +public class TestCancelledRead extends TestDistributedLogBase { + private static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class); + + @Test(timeout = 600000) + public void testWritingAndTailing() throws Exception { + String name = "writing-and-tailing"; + DistributedLogConfiguration conf = new DistributedLogConfiguration() + .setReadAheadWaitTime(5000) + .setOutputBufferSize(0) + .setCreateStreamIfNotExists(true) + .setImmediateFlushEnabled(true) + .setFailFastOnStreamNotReady(true) + .setPeriodicFlushFrequencyMilliSeconds(0) + .setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE) + .setEnableReadAhead(false) + .setLogSegmentRollingIntervalMinutes(0); + + CompletableFuture f = new CompletableFuture<>(); + long entryId = 0; + + try (BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name); + LogWriter writer = dlm.startLogSegmentNonPartitioned()) { + entryId++; + writer.write(DLMTestUtil.getLogRecordInstance(entryId, 100000)); + } + + try (BKDistributedLogManager dlmReader = (BKDistributedLogManager) createNewDLM(conf, name)) { + BKAsyncLogReader reader = (BKAsyncLogReader) dlmReader.getAsyncLogReader(DLSN.InitialDLSN); + + assertNotNull(reader.readNext().get()); + + conf.setMaxLogSegmentBytes(1000); + try (BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name); + LogWriter writer = dlm.startLogSegmentNonPartitioned()) { + for (int i = 0; i < 100; i++) { + entryId++; + writer.write(DLMTestUtil.getLogRecordInstance(entryId, 100)); + + assertNotNull(reader.readNext().get()); + } + } finally { + reader.asyncClose().get(); + } + } + } +} From 3fc0ec3ff54a0059bb2ee1d230300b11515660c5 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Tue, 5 Mar 2019 10:27:04 -0800 Subject: [PATCH 0231/1642] Add validateConf to AbstractDNSToSwitchMapping Descriptions of the changes in this PR: - when setConf of AbstractDNSToSwitchMapping is called it should do sanity checking of the conf/env. and throw RuntimeException if things are not valid. - For RawScriptBasedMapping.validateConf, try executing the script with no arguments for sanity check purpose. Here it is expected that running script with no arguments would do sanity check of the script and the env. (there are 2 commits in this PR, but this PR is meant for the second commit and there is other PR for the first commit) Reviewers: Sijie Guo This closes #1965 from reddycharan/sanitycheckmappingconf --- .../RackawareEnsemblePlacementPolicyImpl.java | 4 +- .../net/AbstractDNSToSwitchMapping.java | 7 +++ .../bookkeeper/net/ScriptBasedMapping.java | 35 +++++++++-- ...areEnsemblePlacementPolicyUsingScript.java | 58 +++++++++++++++++++ 4 files changed, 96 insertions(+), 8 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 4cbe25bc9f8..ed35d490a70 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 @@ -387,8 +387,8 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, } } catch (RuntimeException re) { if (!conf.getEnforceMinNumRacksPerWriteQuorum()) { - LOG.info("Failed to initialize DNS Resolver {}, used default subnet resolver : {}", dnsResolverName, - re, re.getMessage()); + LOG.error("Failed to initialize DNS Resolver {}, used default subnet resolver : {}", + dnsResolverName, re, re.getMessage()); dnsResolver = new DefaultResolver(() -> this.getDefaultRack()); } else { /* diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java index a19cc62b4b9..77e64c0b83b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java @@ -65,6 +65,7 @@ public Configuration getConf() { public void setConf(Configuration conf) { this.conf = conf; + validateConf(); } /** @@ -138,4 +139,10 @@ public static boolean isMappingSingleSwitch(DNSToSwitchMapping mapping) { && ((AbstractDNSToSwitchMapping) mapping).isSingleSwitch(); } + /** + * when setConf is called it should do sanity checking of the conf/env. and + * throw RuntimeException if things are not valid. + */ + protected void validateConf() { + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java index 0ef92ef63f3..890bcfcde64 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java @@ -132,13 +132,19 @@ private static final class RawScriptBasedMapping extends AbstractDNSToSwitchMapp private int maxArgs; //max hostnames per call of the script private static final Logger LOG = LoggerFactory.getLogger(ScriptBasedMapping.class); - /** - * Set the configuration and extract the configuration parameters of interest. - * @param conf the new configuration + /* + * extract 'scriptName' and 'maxArgs' parameters from the conf and throw + * RuntimeException if 'scriptName' is null. Also for sanity check + * purpose try executing the script with no arguments. Here it is + * expected that running script with no arguments would do sanity check + * of the script and the env, and return successfully if script and env. + * are valid. If sanity check of the script with no argument fails then + * throw RuntimeException. + * */ @Override - public void setConf(Configuration conf) { - super.setConf(conf); + protected void validateConf() { + Configuration conf = getConf(); if (conf != null) { String scriptNameConfValue = conf.getString(SCRIPT_FILENAME_KEY); if (StringUtils.isNotBlank(scriptNameConfValue)) { @@ -154,7 +160,24 @@ public void setConf(Configuration conf) { } if (null == scriptName) { - throw new RuntimeException("No network topology script is found when using script based DNS resolver."); + throw new RuntimeException("No network topology script is found when using script" + + " based DNS resolver."); + } else { + File dir = null; + String userDir; + if ((userDir = System.getProperty("user.dir")) != null) { + dir = new File(userDir); + } + String[] execString = { this.scriptName }; + ShellCommandExecutor s = new ShellCommandExecutor(execString, dir); + try { + s.execute(); + } catch (Exception e) { + LOG.error("Conf validation failed. Got exception for sanity check of script: " + this.scriptName, + e); + throw new RuntimeException( + "Conf validation failed. Got exception for sanity check of script: " + this.scriptName, e); + } } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java index fc3caaf2966..f90f45d0f91 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java @@ -403,6 +403,64 @@ public void testNetworkTopologyScriptFileNameIsEmpty() throws Exception { repp.uninitalize(); } + @Test + public void testIfValidateConfFails() throws Exception { + ignoreTestIfItIsWindowsOS(); + repp.uninitalize(); + + ClientConfiguration newConf = new ClientConfiguration(); + newConf.setProperty(REPP_DNS_RESOLVER_CLASS, ScriptBasedMapping.class.getName()); + /* + * this script, exits with error value if no argument is passed to it. + * So mapping.validateConf will fail. + */ + newConf.setProperty(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, + "src/test/resources/networkmappingscriptwithargs.sh"); + timer = new HashedWheelTimer(new ThreadFactoryBuilder().setNameFormat("TestTimer-%d").build(), + newConf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS, newConf.getTimeoutTimerNumTicks()); + + repp = new RackawareEnsemblePlacementPolicy(); + repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + + repp.uninitalize(); + repp = new RackawareEnsemblePlacementPolicy(); + try { + repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE); + } catch (RuntimeException re) { + fail("EnforceMinNumRacksPerWriteQuorum is not set, so repp.initialize should succeed" + + " even if mapping.validateConf fails"); + } + + newConf.setEnforceMinNumRacksPerWriteQuorum(true); + repp.uninitalize(); + repp = new RackawareEnsemblePlacementPolicy(); + try { + repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE); + fail("EnforceMinNumRacksPerWriteQuorum is set, so repp.initialize should fail" + + " if mapping.validateConf fails"); + } catch (RuntimeException re) { + + } + + /* + * this script returns successfully even if no argument is passed to it. + * So mapping.validateConf will succeed. + */ + newConf.setProperty(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, + "src/test/resources/networkmappingscript.sh"); + repp.uninitalize(); + repp = new RackawareEnsemblePlacementPolicy(); + try { + repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE); + } catch (RuntimeException re) { + fail("EnforceMinNumRacksPerWriteQuorum is set, and mapping.validateConf succeeds." + + " So repp.initialize should succeed"); + } + } + private int getNumCoveredWriteQuorums(List ensemble, int writeQuorumSize) throws Exception { int ensembleSize = ensemble.size(); From 338a1447d34c1da33b749999f502486628bd2718 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Sun, 10 Mar 2019 18:32:27 +0800 Subject: [PATCH 0232/1642] Migrate command `bookieinfo` to bkctl ### Motivation Migrate command `bookieinfo` to bkctl. ### Changes You can execute as `bin/bkctl bookie info`. Reviewers: Sijie Guo This closes #1972 from zymap/yongzhang/command_info --- .../apache/bookkeeper/bookie/BookieShell.java | 44 +------- .../cli/commands/bookies/InfoCommand.java | 103 ++++++++++++++++++ .../cli/commands/BookiesCommandGroup.java | 2 + .../cli/commands/bookies/InfoCommandTest.java | 97 +++++++++++++++++ 4 files changed, 205 insertions(+), 41 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommandTest.java 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 9df7f49e115..33f9f9a48d0 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 @@ -38,7 +38,6 @@ import java.io.File; import java.io.IOException; import java.io.Serializable; -import java.math.RoundingMode; import java.net.URI; import java.nio.ByteBuffer; import java.nio.file.FileSystems; @@ -47,7 +46,6 @@ import java.nio.file.Paths; import java.nio.file.attribute.BasicFileAttributes; import java.nio.file.attribute.FileTime; -import java.text.DecimalFormat; import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; @@ -87,7 +85,6 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.BookKeeperAdmin; -import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.UpdateLedgerOp; @@ -113,6 +110,7 @@ import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; @@ -2538,46 +2536,10 @@ Options getOptions() { return lOpts; } - String getReadable(long val) { - String unit[] = {"", "KB", "MB", "GB", "TB"}; - int cnt = 0; - double d = val; - while (d >= 1000 && cnt < unit.length - 1) { - d = d / 1000; - cnt++; - } - DecimalFormat df = new DecimalFormat("#.###"); - df.setRoundingMode(RoundingMode.DOWN); - return cnt > 0 ? "(" + df.format(d) + unit[cnt] + ")" : unit[cnt]; - } - @Override public int runCmd(CommandLine cmdLine) throws Exception { - ClientConfiguration clientConf = new ClientConfiguration(bkConf); - clientConf.setDiskWeightBasedPlacementEnabled(true); - BookKeeper bk = new BookKeeper(clientConf); - - Map map = bk.getBookieInfo(); - if (map.size() == 0) { - System.out.println("Failed to retrieve bookie information from any of the bookies"); - bk.close(); - return 0; - } - - System.out.println("Free disk space info:"); - long totalFree = 0, total = 0; - for (Map.Entry e : map.entrySet()) { - BookieInfo bInfo = e.getValue(); - BookieSocketAddress bookieId = e.getKey(); - System.out.println(getBookieSocketAddrStringRepresentation(bookieId) + ":\tFree: " - + bInfo.getFreeDiskSpace() + getReadable(bInfo.getFreeDiskSpace()) + "\tTotal: " - + bInfo.getTotalDiskSpace() + getReadable(bInfo.getTotalDiskSpace())); - totalFree += bInfo.getFreeDiskSpace(); - total += bInfo.getTotalDiskSpace(); - } - System.out.println("Total free disk space in the cluster:\t" + totalFree + getReadable(totalFree)); - System.out.println("Total disk capacity in the cluster:\t" + total + getReadable(total)); - bk.close(); + InfoCommand cmd = new InfoCommand(); + cmd.apply(bkConf, new CliFlags()); return 0; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java new file mode 100644 index 00000000000..2e49d3926e2 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java @@ -0,0 +1,103 @@ +/* + * 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.tools.cli.commands.bookies; + +import java.io.IOException; +import java.math.RoundingMode; +import java.text.DecimalFormat; +import java.util.Map; + +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.cli.helpers.CommandHelpers; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + + +/** + * A bookie command to retrieve bookie info. + */ +public class InfoCommand extends BookieCommand { + + private static final String NAME = "info"; + private static final String DESC = "Retrieve bookie info such as free and total disk space."; + + public InfoCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withFlags(new CliFlags()) + .withDescription(DESC) + .build()); + } + + String getReadable(long val) { + String unit[] = {"", "KB", "MB", "GB", "TB"}; + int cnt = 0; + double d = val; + while (d >= 1000 && cnt < unit.length - 1) { + d = d / 1000; + cnt++; + } + DecimalFormat df = new DecimalFormat("#.###"); + df.setRoundingMode(RoundingMode.DOWN); + return cnt > 0 ? "(" + df.format(d) + unit[cnt] + ")" : unit[cnt]; + } + + + @Override + public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { + + ClientConfiguration clientConf = new ClientConfiguration(conf); + clientConf.setDiskWeightBasedPlacementEnabled(true); + try (BookKeeper bk = new BookKeeper(clientConf)) { + Map map = bk.getBookieInfo(); + if (map.size() == 0) { + System.out.println("Failed to retrieve bookie information from any of the bookies"); + bk.close(); + return true; + } + + System.out.println("Free disk space info:"); + long totalFree = 0, total = 0; + for (Map.Entry e : map.entrySet()) { + BookieInfo bInfo = e.getValue(); + BookieSocketAddress bookieId = e.getKey(); + System.out.println(CommandHelpers.getBookieSocketAddrStringRepresentation(bookieId) + + ":\tFree: " + bInfo.getFreeDiskSpace() + getReadable(bInfo.getFreeDiskSpace()) + + "\tTotal: " + bInfo.getTotalDiskSpace() + getReadable(bInfo.getTotalDiskSpace())); + totalFree += bInfo.getFreeDiskSpace(); + total += bInfo.getTotalDiskSpace(); + } + + System.out.println("Total free disk space in the cluster:\t" + totalFree + getReadable(totalFree)); + System.out.println("Total disk capacity in the cluster:\t" + total + getReadable(total)); + bk.close(); + + return true; + } catch (IOException | InterruptedException | BKException e) { + e.printStackTrace(); + } + return true; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java index 6b86de70296..b299b5c40c1 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java @@ -21,6 +21,7 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; import org.apache.bookkeeper.tools.cli.BKCtl; +import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -40,6 +41,7 @@ public class BookiesCommandGroup extends CliCommandGroup { .withParent(BKCtl.NAME) .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new ListBookiesCommand()) + .addCommand(new InfoCommand()) .build(); public BookiesCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommandTest.java new file mode 100644 index 00000000000..05575cbecdc --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommandTest.java @@ -0,0 +1,97 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import java.util.HashMap; +import java.util.Map; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookieInfoReader; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test of {@link InfoCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({InfoCommand.class}) +public class InfoCommandTest extends BookieCommandTestBase { + + private BookieSocketAddress bookieId; + private BookieInfoReader.BookieInfo bInfo; + private BookKeeper bk; + private Map map = new HashMap<>(); + + public InfoCommandTest() { + super(1, 0); + } + + @Before + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class) + .withNoArguments() + .thenReturn(conf); + + PowerMockito.whenNew(ClientConfiguration.class) + .withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(conf)) + .thenReturn(mock(ClientConfiguration.class)); + + this.bk = mock(BookKeeper.class); + PowerMockito.whenNew(BookKeeper.class) + .withParameterTypes(ClientConfiguration.class) + .withArguments(any(ClientConfiguration.class)) + .thenReturn(bk); + + this.bookieId = new BookieSocketAddress("localhost", 9999); + this.bInfo = mock(BookieInfoReader.BookieInfo.class); + map.put(bookieId, bInfo); + when(bk.getBookieInfo()).thenReturn(map); + } + + @Test + public void testCommand() throws Exception { + InfoCommand cmd = new InfoCommand(); + cmd.apply(bkFlags, new String[]{""}); + + PowerMockito.verifyNew(ClientConfiguration.class, times(1)) + .withArguments(eq(conf)); + PowerMockito.verifyNew(BookKeeper.class, times(1)) + .withArguments(any(ClientConfiguration.class)); + + verify(bk, times(1)).getBookieInfo(); + verify(bInfo, times(1 * 3)).getFreeDiskSpace(); + verify(bInfo, times(1 * 3)).getTotalDiskSpace(); + } +} From f2af9a63c17e92760430e0112dfdff2ddb517181 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 11 Mar 2019 09:57:35 +0800 Subject: [PATCH 0233/1642] Migrate command `bookieformat` Descriptions of the changes in this PR: Migrate command `bookieformat`. ### Motivation #1974 ### Changes - Add command `bookieformat` to bkctl - Replace command in shell Reviewers: Jia Zhai , Sijie Guo This closes #1975 from zymap/command-bookieformat and squashes the following commits: 90e8a1dde [Yong Zhang] Add doc e079383ac [Yong Zhang] Fix the way deal with exception eb78fe74f [Yong Zhang] Add unit test for `commandformat` a3bafe6d9 [Yong Zhang] Migrate command `bookieformate` to bkctl --- .../apache/bookkeeper/bookie/BookieShell.java | 26 ++-- .../cli/commands/bookie/FormatCommand.java | 109 ++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../commands/bookie/FormatCommandTest.java | 136 ++++++++++++++++++ 4 files changed, 256 insertions(+), 17 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommandTest.java 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 33f9f9a48d0..628690e6969 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 @@ -109,6 +109,7 @@ import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; @@ -456,23 +457,14 @@ String getUsage() { int runCmd(CommandLine cmdLine) throws Exception { boolean interactive = (!cmdLine.hasOption("n")); boolean force = cmdLine.hasOption("f"); - - ServerConfiguration conf = new ServerConfiguration(bkConf); - boolean result = Bookie.format(conf, interactive, force); - // delete cookie - if (cmdLine.hasOption("d")) { - runFunctionWithRegistrationManager(bkConf, rm -> { - try { - Versioned cookie = Cookie.readFromRegistrationManager(rm, conf); - cookie.getValue().deleteFromRegistrationManager(rm, conf, cookie.getVersion()); - } catch (CookieNotFoundException nne) { - LOG.warn("No cookie to remove : ", nne); - } catch (BookieException be) { - throw new UncheckedExecutionException(be.getMessage(), be); - } - return null; - }); - } + boolean deletecookie = cmdLine.hasOption("d"); + + FormatCommand.Flags flags = new FormatCommand.Flags() + .nonInteractive(interactive) + .force(force) + .deleteCookie(deletecookie); + FormatCommand command = new FormatCommand(flags); + boolean result = command.apply(bkConf, flags); return (result) ? 0 : 1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java new file mode 100644 index 00000000000..f213c1741eb --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java @@ -0,0 +1,109 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.concurrent.ExecutionException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.versioning.Versioned; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to format the current server contents. + */ +public class FormatCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(FormatCommand.class); + + private static final String NAME = "format"; + private static final String DESC = "Format the current server contents."; + + public FormatCommand() { + this(new Flags()); + } + + public FormatCommand(Flags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for format bookie command. + */ + @Accessors(fluent = true) + @Setter + public static class Flags extends CliFlags { + + @Parameter(names = {"-n", "--noninteractive"}, + description = "Whether to confirm if old data exists?") + private boolean nonInteractive; + + @Parameter(names = {"-f", "--force"}, + description = "If [noninteractive] is specified, then whether" + + "to force delete the old data without prompt?") + private boolean force; + + @Parameter(names = {"-d", "--deletecookie"}, + description = "Delete its cookie on metadata store.") + private boolean deleteCookie; + + } + + @Override + public boolean apply(ServerConfiguration conf, Flags cmdFlags) { + + ServerConfiguration bfconf = new ServerConfiguration(conf); + boolean result = Bookie.format(bfconf, cmdFlags.nonInteractive, cmdFlags.force); + + // delete cookie + if (cmdFlags.deleteCookie) { + try { + runFunctionWithRegistrationManager(conf, rm -> { + + try { + Versioned cookie = Cookie.readFromRegistrationManager(rm, bfconf); + cookie.getValue().deleteFromRegistrationManager(rm, bfconf, cookie.getVersion()); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + + return null; + }); + } catch (MetadataException | ExecutionException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + return result; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 2a5cf1fddbe..cfa37e37e1d 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -21,6 +21,7 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; import org.apache.bookkeeper.tools.cli.BKCtl; +import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -40,6 +41,7 @@ public class BookieCommandGroup extends CliCommandGroup { .withParent(BKCtl.NAME) .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new LastMarkCommand()) + .addCommand(new FormatCommand()) .build(); public BookieCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommandTest.java new file mode 100644 index 00000000000..f8fa98450ba --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommandTest.java @@ -0,0 +1,136 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.function.Function; + +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test {@link FormatCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ FormatCommand.class, MetadataDrivers.class, Cookie.class }) +public class FormatCommandTest extends BookieCommandTestBase { + + public FormatCommandTest() { + super(3, 0); + } + + @SuppressWarnings("unchecked") + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(conf)).thenReturn(conf); + + PowerMockito.mockStatic(MetadataDrivers.class); + RegistrationManager rm = mock(RegistrationManager.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function func = invocationOnMock.getArgument(1); + func.apply(rm); + return true; + }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", any(ServerConfiguration.class), + any(Function.class)); + + Versioned cookie = mock(Versioned.class); + PowerMockito.whenNew(Versioned.class).withParameterTypes(Object.class, Version.class) + .withArguments(any(Cookie.class), eq(new LongVersion(1L))).thenReturn(cookie); + + PowerMockito.mockStatic(Cookie.class); + when(Cookie.readFromRegistrationManager(rm, conf)).thenReturn((Versioned) cookie); + + when(cookie.getValue()).thenReturn(mock(Cookie.class)); + } + + /** + * Test different type of command flags. + */ + @Test + public void testNonInteraction() { + testCommand("-n"); + } + + @Test + public void testNonInteractionLongArgs() { + testCommand("--noninteractive"); + } + + @Test + public void testForce() { + testCommand("-f"); + } + + @Test + public void testForceLongArgs() { + testCommand("--force"); + } + + @Test + public void testDeleteCookie() { + testCommand("-d"); + } + + @Test + public void testDeleteCookieLongArgs() { + testCommand("--deletecookie"); + } + + @Test + public void testAllCommand() { + testCommand("-n", "-f", "-d"); + } + + @Test + public void testAllCommandLongArgs() { + testCommand("--noninteractive", "--force", "--deletecookie"); + } + + private void testCommand(String... args) { + FormatCommand cmd = new FormatCommand(); + try { + assertTrue(cmd.apply(bkFlags, args)); + } catch (Exception e) { + fail("Should not throw any exception here"); + } + } + +} From 3ada127d831c307beec66e159c2a8ea208aaa4dc Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 13 Mar 2019 00:30:43 +0800 Subject: [PATCH 0234/1642] Issue #1977: Migrate command 'bookieinit' Descriptions of the changes in this PR: Migrate command `bookieinit` from shell to bkctl ### Motivation Issue #1977 ### Changes - Replace command `bookieinit` in shell Reviewers: Sijie Guo This closes #1978 from zymap/command-bookieinit, closes #1977 --- .../apache/bookkeeper/bookie/BookieShell.java | 4 +- .../cli/commands/bookie/InitCommand.java | 54 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../cli/commands/bookie/InitCommandTest.java | 57 +++++++++++++++++++ 4 files changed, 116 insertions(+), 1 deletion(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/InitCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/InitCommandTest.java 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 628690e6969..4a75d56005f 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 @@ -110,6 +110,7 @@ import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; @@ -498,7 +499,8 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { ServerConfiguration conf = new ServerConfiguration(bkConf); - boolean result = BookKeeperAdmin.initBookie(conf); + InitCommand initCommand = new InitCommand(); + boolean result = initCommand.apply(conf, new CliFlags()); return (result) ? 0 : 1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/InitCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/InitCommand.java new file mode 100644 index 00000000000..f18b4750c47 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/InitCommand.java @@ -0,0 +1,54 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * A command to initialize new bookie. + */ +public class InitCommand extends BookieCommand { + + private static final String NAME = "init"; + private static final String DESC = "Initialize new bookie."; + + public InitCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new CliFlags()) + .build()); + } + + @Override + public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { + + boolean result = false; + try { + result = BookKeeperAdmin.initBookie(conf); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + return result; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index cfa37e37e1d..568075918d3 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -22,6 +22,7 @@ import org.apache.bookkeeper.tools.cli.BKCtl; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -41,6 +42,7 @@ public class BookieCommandGroup extends CliCommandGroup { .withParent(BKCtl.NAME) .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new LastMarkCommand()) + .addCommand(new InitCommand()) .addCommand(new FormatCommand()) .build(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/InitCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/InitCommandTest.java new file mode 100644 index 00000000000..8d5e7fd1c4d --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/InitCommandTest.java @@ -0,0 +1,57 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.junit.Assert.fail; + +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link InitCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({BookKeeperAdmin.class}) +public class InitCommandTest extends BookieCommandTestBase { + + public InitCommandTest() { + super(3, 0); + } + + public void setup() throws Exception { + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.mockStatic(BookKeeperAdmin.class); + PowerMockito.when(BookKeeperAdmin.initBookie(conf)).thenReturn(true); + } + + @Test + public void testInitCommand() { + InitCommand initCommand = new InitCommand(); + try { + initCommand.apply(bkFlags, new String[] { "" }); + } catch (Exception e) { + fail("Should not throw any exception here."); + } + } +} From b122d9095985dfe266e50179296b600c997b001f Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 14 Mar 2019 06:19:23 +0100 Subject: [PATCH 0235/1642] Tool to search and replace bookie ids in ledger metadata To use: ``` bin/bkctl bookieid searchreplace --from --to ``` To be used in cases where the DNS name of the bookie has to change, and you don't want all the data to have to be moved by autorecovery. Reviewers: Enrico Olivelli , Jia Zhai This closes #1968 from ivankelly/bk-replace-bookieid --- .../bookkeeper/tests/integration/TestCLI.java | 49 +++++++ .../tests/integration/TestSmoke.java | 8 +- .../cli/commands/BookieIdCommandGroup.java | 48 +++++++ .../SearchReplaceBookieIdCommand.java | 130 ++++++++++++++++++ .../cli/commands/bookieid/package-info.java | 22 +++ tools/ledger/src/main/resources/commands | 1 + 6 files changed, 254 insertions(+), 4 deletions(-) create mode 100644 tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieIdCommandGroup.java create mode 100644 tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/SearchReplaceBookieIdCommand.java create mode 100644 tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/package-info.java diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java index f31beef8adb..781d28e8aa8 100644 --- a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java +++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java @@ -19,9 +19,15 @@ package org.apache.bookkeeper.tests.integration; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import com.github.dockerjava.api.DockerClient; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.WriteHandle; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.tests.integration.utils.BookKeeperClusterUtils; import org.apache.bookkeeper.tests.integration.utils.DockerUtils; import org.jboss.arquillian.junit.Arquillian; @@ -102,4 +108,47 @@ public void test003_ListRWBookies() throws Exception { ).contains("ReadWrite Bookies :")); } + @Test + public void test004_SearchReplaceBookieId() throws Exception { + String zookeeper = BookKeeperClusterUtils.zookeeperConnectString(docker); + + String bookie = BookKeeperClusterUtils.getAnyBookie(); + int numEntries = 100; + try (BookKeeper bk = new BookKeeper(zookeeper)) { + long ledgerId; + BookieSocketAddress toReplace; + BookieSocketAddress replaceWith = new BookieSocketAddress("192.0.2.1:3181"); + try (WriteHandle writelh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32C).withPassword(TestSmoke.PASSWD) + .withEnsembleSize(1).withWriteQuorumSize(1).withAckQuorumSize(1).execute().get()) { + ledgerId = writelh.getId(); + toReplace = writelh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + for (int i = 0; i < numEntries; i++) { + writelh.append(("entry-" + i).getBytes()); + } + } + + TestSmoke.readEntries(bk, ledgerId, numEntries); + + DockerUtils.runCommand(docker, bookie, + bkctl, + "bookieid", "searchreplace", + "--from", toReplace.toString(), + "--to", replaceWith.toString()); + + try { + TestSmoke.readEntries(bk, ledgerId, numEntries); + fail("Shouldn't be able to read, as bookie id is rubbish"); + } catch (BKException.BKBookieHandleNotAvailableException e) { + // expected + } + + DockerUtils.runCommand(docker, bookie, + bkctl, + "bookieid", "searchreplace", + "--from", replaceWith.toString(), + "--to", toReplace.toString()); + TestSmoke.readEntries(bk, ledgerId, numEntries); + } + } } diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestSmoke.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestSmoke.java index 2662b18f986..1efa63d3f28 100644 --- a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestSmoke.java +++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestSmoke.java @@ -57,7 +57,7 @@ @RunWith(Arquillian.class) @FixMethodOrder(MethodSorters.NAME_ASCENDING) public class TestSmoke { - private static final byte[] PASSWD = "foobar".getBytes(); + static final byte[] PASSWD = "foobar".getBytes(); @ArquillianResource DockerClient docker; @@ -112,9 +112,9 @@ public void test002_ReadWriteAdv() throws Exception { } } - private static void readEntries(BookKeeper bk, - long ledgerId, - int numExpectedEntries) throws Exception { + static void readEntries(BookKeeper bk, + long ledgerId, + int numExpectedEntries) throws Exception { try (LedgerHandle readlh = bk.openLedger(ledgerId, BookKeeper.DigestType.CRC32C, PASSWD)) { long lac = readlh.getLastAddConfirmed(); int i = 0; diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieIdCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieIdCommandGroup.java new file mode 100644 index 00000000000..f230d096329 --- /dev/null +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieIdCommandGroup.java @@ -0,0 +1,48 @@ +/* + * 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.tools.cli.commands; + +import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; + +import org.apache.bookkeeper.tools.cli.BKCtl; +import org.apache.bookkeeper.tools.cli.commands.bookieid.SearchReplaceBookieIdCommand; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliCommandGroup; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Commands that operate on bookie IDs. + */ +public class BookieIdCommandGroup extends CliCommandGroup { + + private static final String NAME = "bookieid"; + private static final String DESC = "Commands operating on bookie ids"; + + private static final CliSpec spec = CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withParent(BKCtl.NAME) + .withCategory(CATEGORY_INFRA_SERVICE) + .addCommand(new SearchReplaceBookieIdCommand()) + .build(); + + public BookieIdCommandGroup() { + super(spec); + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/SearchReplaceBookieIdCommand.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/SearchReplaceBookieIdCommand.java new file mode 100644 index 00000000000..d2486b44584 --- /dev/null +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/SearchReplaceBookieIdCommand.java @@ -0,0 +1,130 @@ +/* + * 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.tools.cli.commands.bookieid; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.RateLimiter; + +import java.util.ArrayList; +import java.util.List; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.BookKeeper; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.ClientCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * Search and replace a bookie id in ledger metadata. + */ +public class SearchReplaceBookieIdCommand extends ClientCommand { + + private static final String NAME = "searchreplace"; + private static final String DESC = "Search all ledgers for a bookie ID and replace"; + + /** + * Flags for replace bookie id. + */ + @Accessors(fluent = true) + @Setter + public static class Flags extends CliFlags { + + @Parameter(names = { "-f", "--from" }, description = "Bookie ID to search for", required = true) + private String from; + @Parameter(names = { "-t", "--to" }, description = "Bookie ID to replace with", required = true) + private String to; + @Parameter(names = { "-m", "--max" }, description = "Maximum number of replacements to make") + private long max = Long.MAX_VALUE; + @Parameter(names = { "-r", "--rate" }, description = "Rate limit (updates per second)") + private int rate = Integer.MAX_VALUE; + @Parameter(names = { "--dry-run" }, description = "Don't actually write anything") + private boolean dryRun = false; + @Parameter(names = { "-v", "--verbose" }, description = "Verbose output") + private boolean verbose = false; + } + + public SearchReplaceBookieIdCommand() { + this(new Flags()); + } + + public SearchReplaceBookieIdCommand(Flags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + @Override + protected void run(BookKeeper bk, Flags flags) throws Exception { + try (BookKeeperAdmin admin = new BookKeeperAdmin((org.apache.bookkeeper.client.BookKeeper) bk)) { + LedgerManager ledgerManager = ((org.apache.bookkeeper.client.BookKeeper) bk).getLedgerManager(); + long i = 0; + + BookieSocketAddress fromAddr = new BookieSocketAddress(flags.from); + BookieSocketAddress toAddr = new BookieSocketAddress(flags.to); + System.out.println(String.format("Replacing bookie id %s with %s in metadata", fromAddr, toAddr)); + RateLimiter limiter = RateLimiter.create(flags.rate); + for (Long lid : admin.listLedgers()) { + Versioned md = ledgerManager.readLedgerMetadata(lid).get(); + if (md.getValue().getAllEnsembles().entrySet() + .stream().anyMatch(e -> e.getValue().contains(fromAddr))) { + limiter.acquire(); + + LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(md.getValue()); + md.getValue().getAllEnsembles().entrySet().stream() + .filter(e -> e.getValue().contains(fromAddr)) + .forEach(e -> { + List ensemble = new ArrayList<>(e.getValue()); + ensemble.replaceAll((a) -> { + if (a.equals(fromAddr)) { + return toAddr; + } else { + return a; + } + }); + builder.replaceEnsembleEntry(e.getKey(), ensemble); + }); + LedgerMetadata newMeta = builder.build(); + if (flags.verbose) { + System.out.println("Replacing ledger " + lid + " metadata ..."); + System.out.println(md.getValue().toSafeString()); + System.out.println("with ..."); + System.out.println(newMeta.toSafeString()); + } + i++; + if (!flags.dryRun) { + ledgerManager.writeLedgerMetadata(lid, newMeta, md.getVersion()).get(); + } + } + if (i >= flags.max) { + System.out.println("Max number of ledgers processed, exiting"); + break; + } + } + System.out.println("Replaced bookie ID in " + i + " ledgers"); + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/package-info.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/package-info.java new file mode 100644 index 00000000000..131d6725006 --- /dev/null +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/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. + */ + +/** + * CLI commands for working with Bookie IDs. + */ +package org.apache.bookkeeper.tools.cli.commands.bookieid; diff --git a/tools/ledger/src/main/resources/commands b/tools/ledger/src/main/resources/commands index 7ea146b7825..c9af7a127cf 100644 --- a/tools/ledger/src/main/resources/commands +++ b/tools/ledger/src/main/resources/commands @@ -17,6 +17,7 @@ # org.apache.bookkeeper.tools.cli.commands.BookieCommandGroup +org.apache.bookkeeper.tools.cli.commands.BookieIdCommandGroup org.apache.bookkeeper.tools.cli.commands.BookiesCommandGroup org.apache.bookkeeper.tools.cli.commands.CookieCommandGroup org.apache.bookkeeper.tools.cli.commands.LedgerCommandGroup From 94b65a9e17f808cd6e77e01523c795bcf1fd9a5d Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 15 Mar 2019 14:56:33 +0800 Subject: [PATCH 0236/1642] Issue #1980: Migrate command `ledger` from shell to bkctl Descriptions of the changes in this PR: Migrate command `ledger` from shell to bkctl ### Motivation #1980 ### Changes - Add new implement for `ledger` Reviewers: Jia Zhai , Sijie Guo This closes #1981 from zymap/command-ledger, closes #1980 --- .../org/apache/bookkeeper/bookie/Bookie.java | 5 +- .../apache/bookkeeper/bookie/BookieShell.java | 97 +-------- .../cli/commands/bookie/LedgerCommand.java | 202 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../commands/bookie/LedgerCommandTest.java | 166 ++++++++++++++ 5 files changed, 380 insertions(+), 92 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.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 16b63121a9c..44c950978de 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 @@ -625,9 +625,8 @@ private static LedgerStorage buildLedgerStorage(ServerConfiguration conf) throws * @return Passed ledgerStorage instance * @throws IOException */ - static LedgerStorage mountLedgerStorageOffline( - ServerConfiguration conf, - LedgerStorage ledgerStorage) throws IOException { + public static LedgerStorage mountLedgerStorageOffline(ServerConfiguration conf, LedgerStorage ledgerStorage) + throws IOException { StatsLogger statsLogger = NullStatsLogger.INSTANCE; DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()); 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 4a75d56005f..026c900c283 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 @@ -112,6 +112,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; @@ -144,7 +145,6 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.mutable.MutableBoolean; -import org.apache.commons.lang.mutable.MutableLong; import org.apache.commons.lang3.ArrayUtils; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.AsyncCallback.VoidCallback; @@ -717,96 +717,15 @@ class LedgerCmd extends MyCommand { @Override public int runCmd(CommandLine cmdLine) throws Exception { - String[] leftArgs = cmdLine.getArgs(); - if (leftArgs.length <= 0) { - printErrorLine("ERROR: missing ledger id"); - printUsage(); - return -1; - } - - boolean printMeta = false; + LedgerCommand cmd = new LedgerCommand(ledgerIdFormatter); + cmd.setPrint(BookieShell.this::printInfoLine); + LedgerCommand.LedgerFlags flags = new LedgerCommand.LedgerFlags(); if (cmdLine.hasOption("m")) { - printMeta = true; - } - long ledgerId; - try { - ledgerId = ledgerIdFormatter.readLedgerId(leftArgs[0]); - } catch (IllegalArgumentException iae) { - printErrorLine("ERROR: invalid ledger id " + leftArgs[0]); - printUsage(); - return -1; + flags.meta(true); } - - if (bkConf.getLedgerStorageClass().equals(DbLedgerStorage.class.getName())) { - // dump ledger info - try { - DbLedgerStorage.readLedgerIndexEntries(ledgerId, bkConf, - (currentEntry, entryLogId, position) -> printInfoLine( - "entry " + currentEntry + "\t:\t(log: " + entryLogId + ", pos: " + position + ")")); - } catch (IOException e) { - System.err.printf("ERROR: initializing dbLedgerStorage %s", e.getMessage()); - return -1; - } - } else if ((bkConf.getLedgerStorageClass().equals(SortedLedgerStorage.class.getName()) - || bkConf.getLedgerStorageClass().equals(InterleavedLedgerStorage.class.getName()))) { - ServerConfiguration conf = new ServerConfiguration(bkConf); - InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); - Bookie.mountLedgerStorageOffline(conf, interleavedStorage); - - if (printMeta) { - // print meta - printInfoLine("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " ====="); - LedgerCache.LedgerIndexMetadata meta = interleavedStorage.readLedgerIndexMetadata(ledgerId); - printInfoLine("master key : " + meta.getMasterKeyHex()); - - long size = meta.size; - if (size % 8 == 0) { - printInfoLine("size : " + size); - } else { - printInfoLine("size : " + size - + " (not aligned with 8, may be corrupted or under flushing now)"); - } - - printInfoLine("entries : " + (size / 8)); - printInfoLine("isFenced : " + meta.fenced); - } - - try { - // dump ledger info - printInfoLine("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " ====="); - for (LedgerCache.PageEntries page : interleavedStorage.getIndexEntries(ledgerId)) { - final MutableLong curEntry = new MutableLong(page.getFirstEntry()); - try (LedgerEntryPage lep = page.getLEP()){ - lep.getEntries((entry, offset) -> { - while (curEntry.longValue() < entry) { - printInfoLine("entry " + curEntry + "\t:\tN/A"); - curEntry.increment(); - } - long entryLogId = offset >> 32L; - long pos = offset & 0xffffffffL; - printInfoLine("entry " + curEntry + "\t:\t(log:" + entryLogId + ", pos: " + pos + ")"); - curEntry.increment(); - return true; - }); - } catch (IOException ie) { - printInfoLine("Failed to read index page @ " + page.getFirstEntry() - + ", the index file may be corrupted : " - + ie.getMessage()); - return 1; - } - - while (curEntry.longValue() < page.getLastEntry()) { - printInfoLine("entry " + curEntry + "\t:\tN/A"); - curEntry.increment(); - } - } - } catch (IOException ie) { - LOG.error("Failed to read index page"); - return 1; - } - } - - return 0; + flags.ledgerId(Long.parseLong(cmdLine.getArgs()[0])); + boolean result = cmd.apply(bkConf, flags); + return (result) ? 0 : 1; } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java new file mode 100644 index 00000000000..faaefcb82a9 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java @@ -0,0 +1,202 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.util.function.Consumer; + +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; +import org.apache.bookkeeper.bookie.LedgerCache; +import org.apache.bookkeeper.bookie.LedgerEntryPage; +import org.apache.bookkeeper.bookie.SortedLedgerStorage; +import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.apache.commons.lang.mutable.MutableLong; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to dump ledger index entries into readable format. + */ +public class LedgerCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(LedgerCommand.class); + + private static final String NAME = "ledger"; + private static final String DESC = "Dump ledger index entries into readable format"; + + private LedgerIdFormatter ledgerIdFormatter; + + private Consumer print = this::printInfoLine; + + public void setPrint(Consumer print) { + this.print = print; + } + + public LedgerCommand() { + this(new LedgerFlags()); + } + + public LedgerCommand(LedgerIdFormatter ledgerIdFormatter) { + this(new LedgerFlags()); + this.ledgerIdFormatter = ledgerIdFormatter; + } + + private LedgerCommand(LedgerFlags flags) { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build()); + } + + /** + * Flags for ledger command. + */ + @Accessors(fluent = true) + @Setter + public static class LedgerFlags extends CliFlags { + + @Parameter(names = { "-id", "--ledgerId" }, description = "Specific ledger id", required = true) + private long ledgerId; + + @Parameter(names = { "-m", "--meta" }, description = "Print meta information") + private boolean meta; + + @Parameter(names = { "-l", "--ledgeridformatter" }, description = "Set ledger id formatter") + private String ledgerIdFormatter = ""; + } + + @Override + public boolean apply(ServerConfiguration conf, LedgerFlags cmdFlags) { + initLedgerIdFormatter(conf, cmdFlags); + long ledgerId = cmdFlags.ledgerId; + if (conf.getLedgerStorageClass().equals(DbLedgerStorage.class.getName())) { + // dump ledger info + if (!dumpLedgerInfo(ledgerId, conf)) { + return false; + } + } else if (conf.getLedgerStorageClass().equals(SortedLedgerStorage.class.getName()) + || conf.getLedgerStorageClass().equals(InterleavedLedgerStorage.class.getName())) { + ServerConfiguration tConf = new ServerConfiguration(conf); + InterleavedLedgerStorage interleavedLedgerStorage = new InterleavedLedgerStorage(); + try { + Bookie.mountLedgerStorageOffline(tConf, interleavedLedgerStorage); + } catch (IOException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + + if (cmdFlags.meta) { + // print meta + printMeta(ledgerId, interleavedLedgerStorage); + } + + try { + print.accept("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " ====="); + for (LedgerCache.PageEntries page : interleavedLedgerStorage.getIndexEntries(ledgerId)) { + if (printPageEntries(page)) { + return true; + } + } + } catch (IOException e) { + LOG.error("Failed to read index page"); + return true; + } + + } + return true; + } + + private void initLedgerIdFormatter(ServerConfiguration conf, LedgerFlags flags) { + if (flags.ledgerIdFormatter.equals("")) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } else { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(flags.ledgerIdFormatter, conf); + } + } + + private boolean dumpLedgerInfo(long ledgerId, ServerConfiguration conf) { + try { + DbLedgerStorage.readLedgerIndexEntries(ledgerId, conf, (currentEntry, entryLodId, position) -> System.out + .println("entry " + currentEntry + "\t:\t(log: " + entryLodId + ", pos: " + position + ")")); + } catch (IOException e) { + System.err.printf("ERROR: initializing dbLedgerStorage %s", e.getMessage()); + return false; + } + return true; + } + + private void printMeta(long ledgerId, InterleavedLedgerStorage interleavedLedgerStorage) { + print.accept("===== LEDGER: " + ledgerIdFormatter.formatLedgerId(ledgerId) + " ====="); + try { + LedgerCache.LedgerIndexMetadata meta = interleavedLedgerStorage.readLedgerIndexMetadata(ledgerId); + print.accept("master key : " + meta.getMasterKeyHex()); + long size = meta.size; + if (size % 8 == 0) { + print.accept("size : " + size); + } else { + print.accept("size : " + size + "(not aligned with 8, may be corrupted or under flushing now)"); + } + + print.accept("entries : " + (size / 8)); + print.accept("isFenced : " + meta.fenced); + } catch (IOException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean printPageEntries(LedgerCache.PageEntries page) { + final MutableLong curEntry = new MutableLong(page.getFirstEntry()); + try (LedgerEntryPage lep = page.getLEP()) { + lep.getEntries((entry, offset) -> { + while (curEntry.longValue() < entry) { + print.accept("entry " + curEntry + "\t:\tN/A"); + curEntry.increment(); + } + long entryLogId = offset >> 32L; + long pos = offset & 0xffffffffL; + print.accept("entry " + curEntry + "\t:\t(log:" + entryLogId + ", pos: " + pos + ")"); + curEntry.increment(); + return true; + }); + } catch (Exception e) { + print.accept( + "Failed to read index page @ " + page.getFirstEntry() + ", the index file may be corrupted : " + e + .getMessage()); + return true; + } + + while (curEntry.longValue() < page.getLastEntry()) { + print.accept("entry " + curEntry + "\t:\tN/A"); + curEntry.increment(); + } + + return false; + } + + + private void printInfoLine(String mes) { + System.out.println(mes); + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 568075918d3..e7daee6e149 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -24,6 +24,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -44,6 +45,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new LastMarkCommand()) .addCommand(new InitCommand()) .addCommand(new FormatCommand()) + .addCommand(new LedgerCommand()) .build(); public BookieCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java new file mode 100644 index 00000000000..1023458fef0 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommandTest.java @@ -0,0 +1,166 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.verify; +import static org.mockito.internal.verification.VerificationModeFactory.times; + +import java.util.Iterator; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; +import org.apache.bookkeeper.bookie.LedgerCache; +import org.apache.bookkeeper.bookie.LedgerEntryPage; +import org.apache.bookkeeper.bookie.LedgerStorage; +import org.apache.bookkeeper.bookie.SortedLedgerStorage; +import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; +import org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link LedgerCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ DbLedgerStorage.class, SortedLedgerStorage.class, InterleavedLedgerStorage.class, Bookie.class, + LedgerStorage.class, LedgerCache.PageEntries.class, LedgerCache.PageEntriesIterable.class, LedgerCommand.class, + LedgerCache.LedgerIndexMetadata.class }) +public class LedgerCommandTest extends BookieCommandTestBase { + + private LedgerCache.LedgerIndexMetadata metadata; + private ServerConfiguration tConf; + + public LedgerCommandTest() { + super(3, 0); + } + + public void setup() throws Exception { + super.setup(); + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + PowerMockito.mockStatic(DbLedgerStorage.class); + PowerMockito.doAnswer(invocationOnMock -> { + SingleDirectoryDbLedgerStorage.LedgerLoggerProcessor p = invocationOnMock.getArgument(2); + p.process(1L, 1L, 1L); + return true; + }).when(DbLedgerStorage.class, "readLedgerIndexEntries", anyLong(), any(ServerConfiguration.class), + any(SingleDirectoryDbLedgerStorage.LedgerLoggerProcessor.class)); + PowerMockito.when(DbLedgerStorage.class.getName()) + .thenReturn("org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage"); + + tConf = PowerMockito.mock(ServerConfiguration.class); + PowerMockito.whenNew(ServerConfiguration.class).withArguments(ServerConfiguration.class) + .thenReturn(tConf); + + InterleavedLedgerStorage interleavedLedgerStorage = PowerMockito.mock(InterleavedLedgerStorage.class); + PowerMockito.whenNew(InterleavedLedgerStorage.class).withNoArguments().thenReturn(interleavedLedgerStorage); + + PowerMockito.mockStatic(Bookie.class); + PowerMockito.when(Bookie.mountLedgerStorageOffline(eq(tConf), eq(interleavedLedgerStorage))) + .thenReturn(PowerMockito.mock(LedgerStorage.class)); + + LedgerCache.PageEntries e = PowerMockito.mock(LedgerCache.PageEntries.class); + LedgerCache.PageEntriesIterable i = PowerMockito.mock(LedgerCache.PageEntriesIterable.class); + PowerMockito.when(interleavedLedgerStorage.getIndexEntries(anyLong())).thenReturn(i); + PowerMockito.when(i.iterator()).thenReturn(getPageIterator(e)); + LedgerEntryPage lep = PowerMockito.mock(LedgerEntryPage.class); + PowerMockito.when(e.getLEP()).thenReturn(lep); + + metadata = PowerMockito.mock(LedgerCache.LedgerIndexMetadata.class); + PowerMockito.when(interleavedLedgerStorage.readLedgerIndexMetadata(anyLong())).thenReturn(metadata); + PowerMockito.when(metadata.getMasterKeyHex()).thenReturn(""); + } + + public Iterator getPageIterator(LedgerCache.PageEntries page) { + Iterator i = new Iterator() { + int i = 0; + + @Override + public boolean hasNext() { + if (i < 2) { + i++; + return true; + } + return false; + } + + @Override + public LedgerCache.PageEntries next() { + return page; + } + }; + return i; + } + + // Test without ledger id + @Test + public void testWithoutLedgerId() { + testLedgerCommand(""); + } + + // test ledger command without args + @Test + public void testNoArguments() { + testLedgerCommand("-id", "1"); + } + + @Test + public void testWithMeta() throws Exception { + LedgerCommand cmd = new LedgerCommand(); + cmd.apply(bkFlags, new String[] { "-id", "1", "-m" }); + + PowerMockito.verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(conf)); + PowerMockito.verifyNew(InterleavedLedgerStorage.class, times(1)).withNoArguments(); + + verify(metadata, times(1)).getMasterKeyHex(); + } + + @Test + public void testDbLedgerStorage() throws Exception { + conf.setLedgerStorageClass("org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage"); + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + LedgerCommand cmd = new LedgerCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[]{"-id", "1"})); + } + + // test update formatter by flag + @Test + public void testFormatterFlag() { + testLedgerCommand("-id", "1", "-l", "hex"); + } + + public void testLedgerCommand(String... args) { + LedgerCommand ledgerCommand = new LedgerCommand(); + try { + ledgerCommand.apply(bkFlags, args); + } catch (IllegalArgumentException iae) { + if (!iae.getMessage().equals("No ledger id is specified")) { + Assert.fail("exception is not expect ! "); + } + } + } +} From c13f60165ee4912052f0e7ad28de785ed1e3f8f6 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 18 Mar 2019 16:11:27 +0800 Subject: [PATCH 0237/1642] Issue #1982: Migrate command `bookiesanity` Descriptions of the changes in this PR: - migrate command `bookiesanity` ### Changes - use command by `bkctl bookie sanity` Master Issue: #1982 Reviewers: Jia Zhai , Sijie Guo This closes #1983 from zymap/command-bookiesanity, closes #1982 --- .../apache/bookkeeper/bookie/BookieShell.java | 63 +------ .../commands/bookie/SanityTestCommand.java | 139 +++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../bookie/SanityTestCommandTest.java | 164 ++++++++++++++++++ 4 files changed, 310 insertions(+), 58 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/SanityTestCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/SanityTestCommandTest.java 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 026c900c283..3d78c334a92 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 @@ -52,7 +52,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.Enumeration; import java.util.Formatter; import java.util.HashMap; import java.util.HashSet; @@ -83,7 +82,6 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.MetaStoreException; import org.apache.bookkeeper.client.BookKeeper; -import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; @@ -113,6 +111,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; @@ -1269,62 +1268,10 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { - int numberOfEntries = getOptionIntValue(cmdLine, "entries", 10); - int timeoutSecs = getOptionIntValue(cmdLine, "timeout", 1); - - ClientConfiguration conf = new ClientConfiguration(); - conf.addConfiguration(bkConf); - conf.setEnsemblePlacementPolicy(LocalBookieEnsemblePlacementPolicy.class); - conf.setAddEntryTimeout(timeoutSecs); - conf.setReadEntryTimeout(timeoutSecs); - - BookKeeper bk = new BookKeeper(conf); - LedgerHandle lh = null; - try { - lh = bk.createLedger(1, 1, DigestType.MAC, new byte[0]); - LOG.info("Created ledger {}", lh.getId()); - - for (int i = 0; i < numberOfEntries; i++) { - String content = "entry-" + i; - lh.addEntry(content.getBytes(UTF_8)); - } - - LOG.info("Written {} entries in ledger {}", numberOfEntries, lh.getId()); - - // Reopen the ledger and read entries - lh = bk.openLedger(lh.getId(), DigestType.MAC, new byte[0]); - if (lh.getLastAddConfirmed() != (numberOfEntries - 1)) { - throw new Exception("Invalid last entry found on ledger. expecting: " + (numberOfEntries - 1) - + " -- found: " + lh.getLastAddConfirmed()); - } - - Enumeration entries = lh.readEntries(0, numberOfEntries - 1); - int i = 0; - while (entries.hasMoreElements()) { - LedgerEntry entry = entries.nextElement(); - String actualMsg = new String(entry.getEntry(), UTF_8); - String expectedMsg = "entry-" + (i++); - if (!expectedMsg.equals(actualMsg)) { - throw new Exception("Failed validation of received message - Expected: " + expectedMsg - + ", Actual: " + actualMsg); - } - } - - LOG.info("Read {} entries from ledger {}", entries, lh.getId()); - } catch (Exception e) { - LOG.warn("Error in bookie sanity test", e); - return -1; - } finally { - if (lh != null) { - bk.deleteLedger(lh.getId()); - LOG.info("Deleted ledger {}", lh.getId()); - } - - bk.close(); - } - - LOG.info("Bookie sanity test succeeded"); - return 0; + SanityTestCommand command = new SanityTestCommand(); + SanityTestCommand.SanityFlags flags = new SanityTestCommand.SanityFlags(); + boolean result = command.apply(bkConf, flags); + return (result) ? 0 : -1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/SanityTestCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/SanityTestCommand.java new file mode 100644 index 00000000000..c1a81fd7436 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/SanityTestCommand.java @@ -0,0 +1,139 @@ +/* + * 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.tools.cli.commands.bookie; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.Enumeration; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.LocalBookieEnsemblePlacementPolicy; +import org.apache.bookkeeper.client.BookKeeper; +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.tools.cli.commands.bookie.SanityTestCommand.SanityFlags; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A bookie command to sanity test for local bookie. + */ +public class SanityTestCommand extends BookieCommand { + + private static final Logger LOG = LoggerFactory.getLogger(SanityTestCommand.class); + private static final String NAME = "sanitytest"; + private static final String DESC = "Sanity test for local bookie. " + + "Create ledger and write/reads entries on local bookie."; + + public SanityTestCommand() { + this(new SanityFlags()); + } + + public SanityTestCommand(SanityFlags flags) { + super(CliSpec.newBuilder().withFlags(flags).withName(NAME).withDescription(DESC).build()); + } + + /** + * Flags for sanity command. + */ + @Accessors(fluent = true) + @Setter + public static class SanityFlags extends CliFlags{ + + @Parameter(names = {"-e", "--entries"}, description = "Total entries to be added for the test (default 10)") + private int entries = 10; + + @Parameter(names = { "-t", + "--timeout" }, description = "Timeout for write/read operations in seconds (default 1)") + private int timeout = 1; + + } + + @Override + public boolean apply(ServerConfiguration conf, SanityFlags cmdFlags) { + try { + return handle(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean handle(ServerConfiguration conf, SanityFlags cmdFlags) throws Exception { + ClientConfiguration clientConf = new ClientConfiguration(); + clientConf.addConfiguration(conf); + clientConf.setEnsemblePlacementPolicy(LocalBookieEnsemblePlacementPolicy.class); + clientConf.setAddEntryTimeout(cmdFlags.timeout); + clientConf.setReadEntryTimeout(cmdFlags.timeout); + + BookKeeper bk = new BookKeeper(clientConf); + LedgerHandle lh = null; + try { + lh = bk.createLedger(1, 1, BookKeeper.DigestType.MAC, new byte[0]); + LOG.info("Create ledger {}", lh.getId()); + + for (int i = 0; i < cmdFlags.entries; i++) { + String content = "entry-" + i; + lh.addEntry(content.getBytes(UTF_8)); + } + + LOG.info("Written {} entries in ledger {}", cmdFlags.entries, lh.getId()); + + // Reopen the ledger and read entries + lh = bk.openLedger(lh.getId(), BookKeeper.DigestType.MAC, new byte[0]); + if (lh.getLastAddConfirmed() != (cmdFlags.entries - 1)) { + throw new Exception("Invalid last entry found on ledger. expecting: " + (cmdFlags.entries - 1) + + " -- found: " + lh.getLastAddConfirmed()); + } + + Enumeration entries = lh.readEntries(0, cmdFlags.entries - 1); + int i = 0; + while (entries.hasMoreElements()) { + LedgerEntry entry = entries.nextElement(); + String actualMsg = new String(entry.getEntry(), UTF_8); + String expectedMsg = "entry-" + (i++); + if (!expectedMsg.equals(actualMsg)) { + throw new Exception("Failed validation of received message - Expected: " + expectedMsg + + ", Actual: " + actualMsg); + } + } + + LOG.info("Read {} entries from ledger {}", i, lh.getId()); + } catch (Exception e) { + LOG.warn("Error in bookie sanity test", e); + return false; + } finally { + if (lh != null) { + bk.deleteLedger(lh.getId()); + LOG.info("Deleted ledger {}", lh.getId()); + } + + bk.close(); + } + + LOG.info("Bookie sanity test succeeded"); + return true; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index e7daee6e149..940d482f2d1 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -45,6 +46,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new LastMarkCommand()) .addCommand(new InitCommand()) .addCommand(new FormatCommand()) + .addCommand(new SanityTestCommand()) .addCommand(new LedgerCommand()) .build(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/SanityTestCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/SanityTestCommandTest.java new file mode 100644 index 00000000000..fb596efaf8c --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/SanityTestCommandTest.java @@ -0,0 +1,164 @@ +/* + * 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.tools.cli.commands.bookie; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static junit.framework.TestCase.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.Enumeration; +import java.util.Vector; +import org.apache.bookkeeper.bookie.LocalBookieEnsemblePlacementPolicy; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.LedgerEntry; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.commons.configuration.Configuration; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Test for sanity command. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ SanityTestCommand.class, LedgerEntry.class}) +public class SanityTestCommandTest extends BookieCommandTestBase { + + private ClientConfiguration clientConf; + private BookKeeper bk; + private LedgerHandle lh; + + public SanityTestCommandTest() { + super(3, 1); + } + + @Override + public void setup() throws Exception { + super.setup(); + + clientConf = mock(ClientConfiguration.class); + PowerMockito.whenNew(ClientConfiguration.class).withNoArguments().thenReturn(clientConf); + + bk = mock(BookKeeper.class); + lh = mock(LedgerHandle.class); + PowerMockito.whenNew(BookKeeper.class).withParameterTypes(ClientConfiguration.class) + .withArguments(any(ClientConfiguration.class)).thenReturn(bk); + when(bk.createLedger(anyInt(), anyInt(), any(BookKeeper.DigestType.class), eq(new byte[0]))).thenReturn(lh); + when(bk.openLedger(anyLong(), any(BookKeeper.DigestType.class), eq(new byte[0]))).thenReturn(lh); + when(lh.getLastAddConfirmed()).thenReturn(9L); + Enumeration entryEnumeration = getEntry(); + when(lh.readEntries(anyLong(), anyLong())).thenReturn(entryEnumeration); + when(lh.getId()).thenReturn(1L); + + } + + private Enumeration getEntry() { + Vector entries = new Vector<>(); + for (int i = 0; i < 10; i++) { + LedgerEntry ledgerEntry = mock(LedgerEntry.class); + String payload = "entry-" + i; + when(ledgerEntry.getEntry()).thenReturn(payload.getBytes(UTF_8)); + entries.add(ledgerEntry); + } + return entries.elements(); + } + + @Test + public void testDefaultArgs() { + testSanityCommand(""); + } + + @Test + public void testEntriesShortArgs() { + when(lh.getLastAddConfirmed()).thenReturn(0L); + testSanityCommand("-e", "1"); + verifyFunc(); + } + + @Test + public void testEntriesLongArgs() { + when(lh.getLastAddConfirmed()).thenReturn(0L); + testSanityCommand("--entries", "1"); + verifyFunc(); + } + + private void verifyFunc() { + try { + verify(clientConf, times(1)).setAddEntryTimeout(1); + verify(clientConf, times(1)).setReadEntryTimeout(1); + verify(lh, times(1)).addEntry(any()); + verify(lh, times(1)).readEntries(0, 0); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + @Test + public void testTimeoutShortArgs() { + testSanityCommand("-t", "10"); + } + + @Test + public void testTimeoutLongArgs() { + testSanityCommand("--timeout", "10"); + } + + private void verifyTimeout() { + verify(clientConf, times(1)).setAddEntryTimeout(10); + verify(clientConf, times(1)).setReadEntryTimeout(10); + try { + verify(lh, times(10)).addEntry(any()); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + public void testSanityCommand(String... args) { + SanityTestCommand cmd = new SanityTestCommand(); + assertTrue(cmd.apply(bkFlags, args)); + try { + verifyNew(ClientConfiguration.class, times(1)).withNoArguments(); + verify(clientConf, times(1)).addConfiguration(any(Configuration.class)); + verify(clientConf, times(1)).setEnsemblePlacementPolicy(LocalBookieEnsemblePlacementPolicy.class); + + verifyNew(BookKeeper.class).withArguments(clientConf); + verify(bk, times(1)).createLedger(1, 1, BookKeeper.DigestType.MAC, new byte[0]); + verify(lh, times(6)).getId(); + verify(bk, times(1)).openLedger(anyLong(), eq(BookKeeper.DigestType.MAC), eq(new byte[0])); + verify(lh, times(1)).getLastAddConfirmed(); + verify(bk, times(1)).deleteLedger(anyLong()); + verify(bk, times(1)).close(); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } +} From a8f979c0948b6b76f901dfac56f84353f2da2f20 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 19 Mar 2019 23:38:40 +0800 Subject: [PATCH 0238/1642] Issue #1985: Migrate command `convert-to-db-storage` *Motivation* - Use bkctl to run command `convert-to-db-storage` *Modifications* - #1985 - Add command in `bookieGroup` Reviewers: Sijie Guo This closes #1986 from zymap/command-ctdb, closes #1985 --- .../apache/bookkeeper/bookie/BookieShell.java | 41 +----- .../bookie/InterleavedLedgerStorage.java | 2 + .../bookie/ConvertToDBStorageCommand.java | 128 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../bookie/ConvertToDBStorageCommandTest.java | 124 +++++++++++++++++ 5 files changed, 261 insertions(+), 36 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommandTest.java 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 3d78c334a92..ded7d142959 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 @@ -107,6 +107,7 @@ import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; @@ -2548,42 +2549,10 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { - LOG.info("=== Converting to DbLedgerStorage ==="); - ServerConfiguration conf = new ServerConfiguration(bkConf); - - InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); - Bookie.mountLedgerStorageOffline(conf, interleavedStorage); - - DbLedgerStorage dbStorage = new DbLedgerStorage(); - Bookie.mountLedgerStorageOffline(conf, dbStorage); - - int convertedLedgers = 0; - for (long ledgerId : interleavedStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Converting ledger {}", ledgerIdFormatter.formatLedgerId(ledgerId)); - } - - LedgerCache.LedgerIndexMetadata fi = interleavedStorage.readLedgerIndexMetadata(ledgerId); - - LedgerCache.PageEntriesIterable pages = interleavedStorage.getIndexEntries(ledgerId); - - long numberOfEntries = dbStorage.addLedgerToIndex(ledgerId, fi.fenced, fi.masterKey, pages); - if (LOG.isDebugEnabled()) { - LOG.debug(" -- done. fenced={} entries={}", fi.fenced, numberOfEntries); - } - - // Remove index from old storage - interleavedStorage.deleteLedger(ledgerId); - - if (++convertedLedgers % 1000 == 0) { - LOG.info("Converted {} ledgers", convertedLedgers); - } - } - - dbStorage.shutdown(); - interleavedStorage.shutdown(); - - LOG.info("---- Done Converting ----"); + ConvertToDBStorageCommand cmd = new ConvertToDBStorageCommand(); + ConvertToDBStorageCommand.CTDBFlags flags = new ConvertToDBStorageCommand.CTDBFlags(); + cmd.setLedgerIdFormatter(ledgerIdFormatter); + cmd.apply(bkConf, flags); return 0; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java index 3b5bf0114df..59ea9ec0374 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java @@ -50,6 +50,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; +import lombok.Getter; import org.apache.bookkeeper.bookie.Bookie.NoLedgerException; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.bookie.EntryLogger.EntryLogListener; @@ -84,6 +85,7 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry private static final Logger LOG = LoggerFactory.getLogger(InterleavedLedgerStorage.class); EntryLogger entryLogger; + @Getter LedgerCache ledgerCache; protected CheckpointSource checkpointSource; protected Checkpointer checkpointer; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommand.java new file mode 100644 index 00000000000..dc48b6c277f --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommand.java @@ -0,0 +1,128 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; +import org.apache.bookkeeper.bookie.LedgerCache; +import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A command to convert bookie indexes from InterleavedStorage to DbLedgerStorage format. + */ +public class ConvertToDBStorageCommand extends BookieCommand { + + private static final Logger LOG = LoggerFactory.getLogger(ConvertToDBStorageCommand.class); + private static final String NAME = "converttodbstorage"; + private static final String DESC = "Convert bookie indexes from InterleavedStorage to DbLedgerStorage format"; + private static final String NOT_INIT = "default formatter"; + + @Setter + private LedgerIdFormatter ledgerIdFormatter; + + public ConvertToDBStorageCommand() { + this(new CTDBFlags()); + } + public ConvertToDBStorageCommand(CTDBFlags flags) { + super(CliSpec.newBuilder().withFlags(flags).withName(NAME).withDescription(DESC).build()); + } + + /** + * Flags for this command. + */ + @Accessors(fluent = true) + @Setter + public static class CTDBFlags extends CliFlags { + @Parameter(names = { "-l", "--ledgeridformatter" }, description = "Set ledger id formatter") + private String ledgerIdFormatter = NOT_INIT; + } + + @Override + public boolean apply(ServerConfiguration conf, CTDBFlags cmdFlags) { + initLedgerIdFormatter(conf, cmdFlags); + try { + return handle(conf); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean handle(ServerConfiguration conf) throws Exception { + LOG.info("=== Converting to DbLedgerStorage ==="); + ServerConfiguration bkConf = new ServerConfiguration(conf); + + InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); + Bookie.mountLedgerStorageOffline(bkConf, interleavedStorage); + + DbLedgerStorage dbStorage = new DbLedgerStorage(); + Bookie.mountLedgerStorageOffline(bkConf, dbStorage); + + int convertedLedgers = 0; + for (long ledgerId : interleavedStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Converting ledger {}", ledgerIdFormatter.formatLedgerId(ledgerId)); + } + + LedgerCache.LedgerIndexMetadata fi = interleavedStorage.readLedgerIndexMetadata(ledgerId); + + LedgerCache.PageEntriesIterable pages = interleavedStorage.getIndexEntries(ledgerId); + + long numberOfEntries = dbStorage.addLedgerToIndex(ledgerId, fi.fenced, fi.masterKey, pages); + if (LOG.isDebugEnabled()) { + LOG.debug(" -- done. fenced={} entries={}", fi.fenced, numberOfEntries); + } + + // Remove index from old storage + interleavedStorage.deleteLedger(ledgerId); + + if (++convertedLedgers % 1000 == 0) { + LOG.info("Converted {} ledgers", convertedLedgers); + } + } + + dbStorage.shutdown(); + interleavedStorage.shutdown(); + + LOG.info("---- Done Converting ----"); + return true; + } + + private void initLedgerIdFormatter(ServerConfiguration conf, CTDBFlags flags) { + if (this.ledgerIdFormatter != null) { + return; + } + if (flags.ledgerIdFormatter.equals(NOT_INIT)) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } else { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(flags.ledgerIdFormatter, conf); + } + } + +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 940d482f2d1..86129a4f7cf 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -21,6 +21,7 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; import org.apache.bookkeeper.tools.cli.BKCtl; +import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; @@ -48,6 +49,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new FormatCommand()) .addCommand(new SanityTestCommand()) .addCommand(new LedgerCommand()) + .addCommand(new ConvertToDBStorageCommand()) .build(); public BookieCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommandTest.java new file mode 100644 index 00000000000..cc370873af6 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToDBStorageCommandTest.java @@ -0,0 +1,124 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.doNothing; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; +import static org.powermock.api.mockito.PowerMockito.whenNew; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.Iterator; +import java.util.Vector; +import java.util.stream.LongStream; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; +import org.apache.bookkeeper.bookie.LedgerCache; +import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link ConvertToDBStorageCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ Bookie.class, ConvertToDBStorageCommand.class }) +public class ConvertToDBStorageCommandTest extends BookieCommandTestBase { + + private InterleavedLedgerStorage interleavedLedgerStorage; + private DbLedgerStorage dbStorage; + private LedgerCache.LedgerIndexMetadata metadata; + private LedgerCache.PageEntriesIterable entries; + + public ConvertToDBStorageCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class).withArguments(conf) + .thenReturn(conf); + + interleavedLedgerStorage = mock(InterleavedLedgerStorage.class); + whenNew(InterleavedLedgerStorage.class).withNoArguments().thenReturn(interleavedLedgerStorage); + doNothing().when(interleavedLedgerStorage).shutdown(); + when(interleavedLedgerStorage.getActiveLedgersInRange(anyLong(), anyLong())).thenReturn(this::getLedgerId); + metadata = mock(LedgerCache.LedgerIndexMetadata.class); + when(interleavedLedgerStorage.readLedgerIndexMetadata(anyLong())).thenReturn(metadata); + entries = mock(LedgerCache.PageEntriesIterable.class); + when(interleavedLedgerStorage.getIndexEntries(anyLong())).thenReturn(entries); + + dbStorage = mock(DbLedgerStorage.class); + whenNew(DbLedgerStorage.class).withNoArguments().thenReturn(dbStorage); + doNothing().when(dbStorage).shutdown(); + when(dbStorage.addLedgerToIndex(anyLong(), anyBoolean(), eq(new byte[0]), + any(LedgerCache.PageEntriesIterable.class))).thenReturn(1L); + + PowerMockito.mockStatic(Bookie.class); + PowerMockito.when(Bookie.mountLedgerStorageOffline(eq(conf), eq(interleavedLedgerStorage))) + .thenReturn(PowerMockito.mock(InterleavedLedgerStorage.class)); + PowerMockito.when(Bookie.mountLedgerStorageOffline(eq(conf), eq(dbStorage))).thenReturn(dbStorage); + } + + private Iterator getLedgerId() { + Vector longs = new Vector<>(); + LongStream.range(0L, 10L).forEach(longs::add); + return longs.iterator(); + } + + @Test + public void testCTDB() { + ConvertToDBStorageCommand cmd = new ConvertToDBStorageCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "" })); + + try { + verifyNew(ServerConfiguration.class, times(1)).withArguments(conf); + verifyNew(InterleavedLedgerStorage.class, times(1)).withNoArguments(); + verifyNew(DbLedgerStorage.class, times(1)).withNoArguments(); + + verify(interleavedLedgerStorage, times(10)).readLedgerIndexMetadata(anyLong()); + verify(interleavedLedgerStorage, times(10)).getIndexEntries(anyLong()); + verify(dbStorage, times(10)) + .addLedgerToIndex(anyLong(), anyBoolean(), any(), any(LedgerCache.PageEntriesIterable.class)); + verify(interleavedLedgerStorage, times(10)).deleteLedger(anyLong()); + + verify(dbStorage, times(1)).shutdown(); + verify(interleavedLedgerStorage, times(1)).shutdown(); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } +} From 3a1ac7c26361f15365b324ce0d4bcff5df8d7d46 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Wed, 20 Mar 2019 10:19:22 +0100 Subject: [PATCH 0239/1642] Release notes for v4.8.2 Reviewers: Enrico Olivelli , Jia Zhai This closes #1976 from ivankelly/v4.8.2-relnotes --- site/docs/4.8.2/overview/releaseNotes.md | 48 ++++++++++++++++++++++++ 1 file changed, 48 insertions(+) create mode 100644 site/docs/4.8.2/overview/releaseNotes.md diff --git a/site/docs/4.8.2/overview/releaseNotes.md b/site/docs/4.8.2/overview/releaseNotes.md new file mode 100644 index 00000000000..5907b074d81 --- /dev/null +++ b/site/docs/4.8.2/overview/releaseNotes.md @@ -0,0 +1,48 @@ +--- +title: Apache BookKeeper 4.8.2 Release Notes +--- + +This is the 17th release of Apache BookKeeper! + +The 4.8.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.8.1. + +Apache BookKeeper users who are using 4.8.1 are encouraged to upgrade to 4.8.2. The technical details of this release are summarized +below. + +## Highlights + +- [DLOG] Avoid double read in readahead, see [apache/bookkeeper#1973](https://github.com/apache/bookkeeper/pull/1973) + +- Small fix wrong nodesUninitialized count when checkCovered, see [apache/bookkeeper#1900](https://github.com/apache/bookkeeper/pull/1900) + +- Handle double bookie failures, see [apache/bookkeeper#1886](https://github.com/apache/bookkeeper/pull/1886) + +- dir_\*_usage stats are reported as 0, see [apache/bookkeeper#1884](https://github.com/apache/bookkeeper/pull/1884) + +- Fix selectFromNetworkLocation in RackawareEnsemblePlacementPolicyImpl, see [apache/bookkeeper#1862](https://github.com/apache/bookkeeper/pull/1862) + +- DbLedgerStorage should do periodical flush, see [apache/bookkeeper#1842](https://github.com/apache/bookkeeper/pull/1842) + +- Add rest endpoint trigger_gc to trigger GC on Bookie, see [apache/bookkeeper#1838](https://github.com/apache/bookkeeper/pull/1838) + +- Fix sorted ledger storage rotating entry log files too frequent, see [apache/bookkeeper#1807](https://github.com/apache/bookkeeper/pull/1807) + +- Fixed Auth with v2 protocol, see [apache/bookkeeper#1805](https://github.com/apache/bookkeeper/pull/1805) + +- [tools] add cookie related commands, see [apache/bookkeeper#1974](https://github.com/apache/bookkeeper/pull/1794) + +- [tools] improve bkctl help message, see [apache/bookkeeper#1793](https://github.com/apache/bookkeeper/pull/1793) + +- Read Submission should bypass OSE Threads, see [apache/bookkeeper#1791](https://github.com/apache/bookkeeper/pull/1791) + +- Cache InetSocketAddress if hostname is IPAddress, see [apache/bookkeeper#1789](https://github.com/apache/bookkeeper/pull/1789) + +- Fix bugs in DefaultEnsemblePlacementPolicy, see [apache/bookkeeper#1788](https://github.com/apache/bookkeeper/pull/1788) + +### Dependency Changes + +There is no dependency upgrade from 4.8.1. + +## Full list of changes + +- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.2+is%3Aclosed) From 2ba00d548b983142b883416595f251254f3ca132 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Sun, 24 Mar 2019 19:43:27 -0700 Subject: [PATCH 0240/1642] In DbLedgerStorage use default values when config key is present but empty ### Motivation Currently setting the `dbStorage_writeCacheMaxSizeMb=` with empty value is making the DbLedgerStorage initialize to fail since the empty string is being parsed as long. Instead, we should just apply the default value as in the case where the config key is not there. Reviewers: Enrico Olivelli , Jia Zhai , Sijie Guo This closes #1996 from merlimat/db-storage-config --- .../bookie/storage/ldb/DbLedgerStorage.java | 23 ++++++++++++++++--- .../storage/ldb/KeyValueStorageRocksDB.java | 3 ++- .../ldb/DbLedgerStorageBookieTest.java | 6 +++++ 3 files changed, 28 insertions(+), 4 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index 287f4526f0f..e1183490177 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -62,6 +62,7 @@ import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.DiskChecker; +import org.apache.commons.lang3.StringUtils; /** @@ -80,7 +81,8 @@ public class DbLedgerStorage implements LedgerStorage { private static final long DEFAULT_WRITE_CACHE_MAX_SIZE_MB = (long) (0.25 * PlatformDependent.maxDirectMemory()) / MB; - private static final long DEFAULT_READ_CACHE_MAX_SIZE_MB = (long) (0.25 * PlatformDependent.maxDirectMemory()) / MB; + private static final long DEFAULT_READ_CACHE_MAX_SIZE_MB = (long) (0.25 * PlatformDependent.maxDirectMemory()) + / MB; private int numberOfDirs; private List ledgerStorageList; @@ -94,8 +96,10 @@ public class DbLedgerStorage implements LedgerStorage { public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StateManager stateManager, CheckpointSource checkpointSource, Checkpointer checkpointer, StatsLogger statsLogger, ByteBufAllocator allocator) throws IOException { - long writeCacheMaxSize = conf.getLong(WRITE_CACHE_MAX_SIZE_MB, DEFAULT_WRITE_CACHE_MAX_SIZE_MB) * MB; - long readCacheMaxSize = conf.getLong(READ_AHEAD_CACHE_MAX_SIZE_MB, DEFAULT_READ_CACHE_MAX_SIZE_MB) * MB; + long writeCacheMaxSize = getLongVariableOrDefault(conf, WRITE_CACHE_MAX_SIZE_MB, + DEFAULT_WRITE_CACHE_MAX_SIZE_MB) * MB; + long readCacheMaxSize = getLongVariableOrDefault(conf, READ_AHEAD_CACHE_MAX_SIZE_MB, + DEFAULT_READ_CACHE_MAX_SIZE_MB) * MB; this.allocator = allocator; this.numberOfDirs = ledgerDirsManager.getAllLedgerDirs().size(); @@ -335,4 +339,17 @@ public List getGarbageCollectionStatus() { return ledgerStorageList.stream() .map(single -> single.getGarbageCollectionStatus().get(0)).collect(Collectors.toList()); } + + static long getLongVariableOrDefault(ServerConfiguration conf, String keyName, long defaultValue) { + Object obj = conf.getProperty(keyName); + if (obj instanceof Number) { + return ((Number) obj).longValue(); + } else if (obj == null) { + return defaultValue; + } else if (StringUtils.isEmpty(conf.getString(keyName))) { + return defaultValue; + } else { + return conf.getLong(keyName); + } + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java index c06a3cc8916..88411ced3e7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java @@ -103,6 +103,8 @@ public KeyValueStorageRocksDB(String path, DbConfigType dbConfigType, ServerConf if (dbConfigType == DbConfigType.Huge) { // Set default RocksDB block-cache size to 10% of direct mem, unless override long defaultRocksDBBlockCacheSizeBytes = PlatformDependent.maxDirectMemory() / 10; + long blockCacheSize = DbLedgerStorage.getLongVariableOrDefault(conf, ROCKSDB_BLOCK_CACHE_SIZE, + defaultRocksDBBlockCacheSizeBytes); long writeBufferSizeMB = conf.getInt(ROCKSDB_WRITE_BUFFER_SIZE_MB, 64); long sstSizeMB = conf.getInt(ROCKSDB_SST_SIZE_MB, 64); @@ -110,7 +112,6 @@ public KeyValueStorageRocksDB(String path, DbConfigType dbConfigType, ServerConf int numFilesInLevel0 = conf.getInt(ROCKSDB_NUM_FILES_IN_LEVEL0, 4); long maxSizeInLevel1MB = conf.getLong(ROCKSDB_MAX_SIZE_IN_LEVEL1_MB, 256); int blockSize = conf.getInt(ROCKSDB_BLOCK_SIZE, 64 * 1024); - long blockCacheSize = conf.getLong(ROCKSDB_BLOCK_CACHE_SIZE, defaultRocksDBBlockCacheSizeBytes); int bloomFilterBitsPerKey = conf.getInt(ROCKSDB_BLOOM_FILTERS_BITS_PER_KEY, 10); boolean lz4CompressionEnabled = conf.getBoolean(ROCKSDB_LZ4_COMPRESSION_ENABLED, true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageBookieTest.java index cfc74474174..11c3ef3b6e1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageBookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageBookieTest.java @@ -43,6 +43,12 @@ public DbLedgerStorageBookieTest() { baseConf.setLedgerStorageClass(DbLedgerStorage.class.getName()); baseConf.setFlushInterval(60000); baseConf.setGcWaitTime(60000); + + // Leave it empty to pickup default + baseConf.setProperty(DbLedgerStorage.WRITE_CACHE_MAX_SIZE_MB, ""); + + // Configure explicitely with a int object + baseConf.setProperty(DbLedgerStorage.READ_AHEAD_CACHE_MAX_SIZE_MB, 16); } @Test From 0e6bf6db8e276972c6e0cb8a74c30e4491cead5d Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 25 Mar 2019 10:50:13 +0800 Subject: [PATCH 0241/1642] Issue #1987: Migrate command `convert-to-interleaved-storage` Descriptions of the changes in this PR: - Migrate command `convert-to-interleaved-storage` ### Motivation - #1987 - Use `bkctl` to run command `convert-to-interleaved-storage` ### Changes - Add command in `bookiegroup` Reviewers: Jia Zhai , Sijie Guo This closes #1988 from zymap/command-ctis, closes #1987 --- .../apache/bookkeeper/bookie/BookieShell.java | 94 +-------- .../ConvertToInterleavedStorageCommand.java | 183 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 3 +- ...onvertToInterleavedStorageCommandTest.java | 167 ++++++++++++++++ 4 files changed, 358 insertions(+), 89 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommandTest.java 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 ded7d142959..8ac9c85f1c8 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 @@ -29,7 +29,6 @@ import com.google.common.util.concurrent.UncheckedExecutionException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; -import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.EventLoopGroup; @@ -74,10 +73,8 @@ import java.util.stream.LongStream; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; -import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner; import org.apache.bookkeeper.bookie.Journal.JournalScanner; -import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.MetaStoreException; @@ -108,6 +105,7 @@ import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; @@ -2070,12 +2068,12 @@ private int updateBookieIdInCookie(final String bookieId, final boolean useHostn for (File dir : ledgerDirectories) { newCookie.writeToDirectory(dir); } - LOG.info("Updated cookie file present in ledgerDirectories {}", ledgerDirectories); + LOG.info("Updated cookie file present in ledgerDirectories {}", (Object) ledgerDirectories); if (ledgerDirectories != indexDirectories) { for (File dir : indexDirectories) { newCookie.writeToDirectory(dir); } - LOG.info("Updated cookie file present in indexDirectories {}", indexDirectories); + LOG.info("Updated cookie file present in indexDirectories {}", (Object) indexDirectories); } } // writes newcookie to zookeeper @@ -2584,89 +2582,9 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { - LOG.info("=== Converting DbLedgerStorage ==="); - ServerConfiguration conf = new ServerConfiguration(bkConf); - LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(bkConf, bkConf.getLedgerDirs(), - new DiskChecker(bkConf.getDiskUsageThreshold(), bkConf.getDiskUsageWarnThreshold())); - LedgerDirsManager ledgerIndexManager = new LedgerDirsManager(bkConf, bkConf.getLedgerDirs(), - new DiskChecker(bkConf.getDiskUsageThreshold(), bkConf.getDiskUsageWarnThreshold())); - - DbLedgerStorage dbStorage = new DbLedgerStorage(); - InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); - - CheckpointSource checkpointSource = new CheckpointSource() { - @Override - public Checkpoint newCheckpoint() { - return Checkpoint.MAX; - } - - @Override - public void checkpointComplete(Checkpoint checkpoint, boolean compact) - throws IOException { - } - }; - Checkpointer checkpointer = new Checkpointer() { - @Override - public void startCheckpoint(Checkpoint checkpoint) { - // No-op - } - - @Override - public void start() { - // no-op - } - }; - - dbStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager, null, - checkpointSource, checkpointer, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT); - interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager, - null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT); - LedgerCache interleavedLedgerCache = interleavedStorage.ledgerCache; - - int convertedLedgers = 0; - for (long ledgerId : dbStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Converting ledger {}", ledgerIdFormatter.formatLedgerId(ledgerId)); - } - - interleavedStorage.setMasterKey(ledgerId, dbStorage.readMasterKey(ledgerId)); - if (dbStorage.isFenced(ledgerId)) { - interleavedStorage.setFenced(ledgerId); - } - - long lastEntryInLedger = dbStorage.getLastEntryInLedger(ledgerId); - for (long entryId = 0; entryId <= lastEntryInLedger; entryId++) { - try { - long location = dbStorage.getLocation(ledgerId, entryId); - if (location != 0L) { - interleavedLedgerCache.putEntryOffset(ledgerId, entryId, location); - } - } catch (Bookie.NoEntryException e) { - // Ignore entry - } - } - - if (++convertedLedgers % 1000 == 0) { - LOG.info("Converted {} ledgers", convertedLedgers); - } - } - - dbStorage.shutdown(); - - interleavedLedgerCache.flushLedger(true); - interleavedStorage.flush(); - interleavedStorage.shutdown(); - - String baseDir = ledgerDirsManager.getAllLedgerDirs().get(0).toString(); - - // Rename databases and keep backup - Files.move(FileSystems.getDefault().getPath(baseDir, "ledgers"), - FileSystems.getDefault().getPath(baseDir, "ledgers.backup")); - - Files.move(FileSystems.getDefault().getPath(baseDir, "locations"), - FileSystems.getDefault().getPath(baseDir, "locations.backup")); - - LOG.info("---- Done Converting {} ledgers ----", convertedLedgers); + ConvertToInterleavedStorageCommand cmd = new ConvertToInterleavedStorageCommand(); + ConvertToInterleavedStorageCommand.CTISFlags flags = new ConvertToInterleavedStorageCommand.CTISFlags(); + cmd.apply(bkConf, flags); return 0; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommand.java new file mode 100644 index 00000000000..8b1183c43c5 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommand.java @@ -0,0 +1,183 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import io.netty.buffer.PooledByteBufAllocator; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.CheckpointSource; +import org.apache.bookkeeper.bookie.Checkpointer; +import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; +import org.apache.bookkeeper.bookie.LedgerCache; +import org.apache.bookkeeper.bookie.LedgerDirsManager; +import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.DiskChecker; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A command to convert bookie indexes from DbLedgerStorage to InterleavedStorage format. + */ +public class ConvertToInterleavedStorageCommand extends BookieCommand { + + private static final Logger LOG = LoggerFactory.getLogger(ConvertToInterleavedStorageCommand.class); + private static final String NAME = "converttointerleavedstorage"; + private static final String DESC = "Convert bookie indexes from DbLedgerStorage to InterleavedStorage format"; + private static final String NOT_INIT = "default formatter"; + + @Setter + private LedgerIdFormatter ledgerIdFormatter; + + public ConvertToInterleavedStorageCommand() { + this(new CTISFlags()); + } + + public ConvertToInterleavedStorageCommand(CTISFlags flags) { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build()); + } + + /** + * Flags for this command. + */ + @Accessors(fluent = true) + public static class CTISFlags extends CliFlags{ + + @Parameter(names = { "-l", "--ledgeridformatter" }, description = "Set ledger id formatter") + private String ledgerIdFormatter = NOT_INIT; + + } + + @Override + public boolean apply(ServerConfiguration conf, CTISFlags cmdFlags) { + initLedgerIdFormatter(conf, cmdFlags); + try { + return handle(conf); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean handle(ServerConfiguration bkConf) throws Exception { + LOG.info("=== Converting DbLedgerStorage ==="); + ServerConfiguration conf = new ServerConfiguration(bkConf); + LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(bkConf, bkConf.getLedgerDirs(), + new DiskChecker(bkConf.getDiskUsageThreshold(), bkConf.getDiskUsageWarnThreshold())); + LedgerDirsManager ledgerIndexManager = new LedgerDirsManager(bkConf, bkConf.getLedgerDirs(), + new DiskChecker(bkConf.getDiskUsageThreshold(), bkConf.getDiskUsageWarnThreshold())); + + DbLedgerStorage dbStorage = new DbLedgerStorage(); + InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); + + CheckpointSource checkpointSource = new CheckpointSource() { + @Override + public Checkpoint newCheckpoint() { + return Checkpoint.MAX; + } + + @Override + public void checkpointComplete(Checkpoint checkpoint, boolean compact) {} + }; + Checkpointer checkpointer = new Checkpointer() { + @Override + public void startCheckpoint(CheckpointSource.Checkpoint checkpoint) { + // No-op + } + + @Override + public void start() { + // no-op + } + }; + + dbStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager, null, + checkpointSource, checkpointer, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT); + interleavedStorage.initialize(conf, null, ledgerDirsManager, ledgerIndexManager, + null, checkpointSource, checkpointer, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT); + LedgerCache interleavedLedgerCache = interleavedStorage.getLedgerCache(); + + int convertedLedgers = 0; + for (long ledgerId : dbStorage.getActiveLedgersInRange(0, Long.MAX_VALUE)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Converting ledger {}", ledgerIdFormatter.formatLedgerId(ledgerId)); + } + + interleavedStorage.setMasterKey(ledgerId, dbStorage.readMasterKey(ledgerId)); + if (dbStorage.isFenced(ledgerId)) { + interleavedStorage.setFenced(ledgerId); + } + + long lastEntryInLedger = dbStorage.getLastEntryInLedger(ledgerId); + for (long entryId = 0; entryId <= lastEntryInLedger; entryId++) { + try { + long location = dbStorage.getLocation(ledgerId, entryId); + if (location != 0L) { + interleavedLedgerCache.putEntryOffset(ledgerId, entryId, location); + } + } catch (Bookie.NoEntryException e) { + // Ignore entry + } + } + + if (++convertedLedgers % 1000 == 0) { + LOG.info("Converted {} ledgers", convertedLedgers); + } + } + + dbStorage.shutdown(); + + interleavedLedgerCache.flushLedger(true); + interleavedStorage.flush(); + interleavedStorage.shutdown(); + + String baseDir = ledgerDirsManager.getAllLedgerDirs().get(0).toString(); + + // Rename databases and keep backup + Files.move(FileSystems.getDefault().getPath(baseDir, "ledgers"), + FileSystems.getDefault().getPath(baseDir, "ledgers.backup")); + + Files.move(FileSystems.getDefault().getPath(baseDir, "locations"), + FileSystems.getDefault().getPath(baseDir, "locations.backup")); + + LOG.info("---- Done Converting {} ledgers ----", convertedLedgers); + return true; + } + + private void initLedgerIdFormatter(ServerConfiguration conf, CTISFlags flags) { + if (this.ledgerIdFormatter != null) { + return; + } + if (flags.ledgerIdFormatter.equals(NOT_INIT)) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } else { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(flags.ledgerIdFormatter, conf); + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 86129a4f7cf..9b2277220b9 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -22,6 +22,7 @@ import org.apache.bookkeeper.tools.cli.BKCtl; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; @@ -50,7 +51,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new SanityTestCommand()) .addCommand(new LedgerCommand()) .addCommand(new ConvertToDBStorageCommand()) - .build(); + .addCommand(new ConvertToInterleavedStorageCommand()).build(); public BookieCommandGroup() { super(spec); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommandTest.java new file mode 100644 index 00000000000..6849ed1dff5 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ConvertToInterleavedStorageCommandTest.java @@ -0,0 +1,167 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.doNothing; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; +import static org.powermock.api.mockito.PowerMockito.whenNew; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import io.netty.buffer.PooledByteBufAllocator; +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Vector; +import java.util.stream.LongStream; +import org.apache.bookkeeper.bookie.CheckpointSource; +import org.apache.bookkeeper.bookie.Checkpointer; +import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; +import org.apache.bookkeeper.bookie.LedgerCache; +import org.apache.bookkeeper.bookie.LedgerDirsManager; +import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.util.DiskChecker; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link ConvertToInterleavedStorageCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ ConvertToInterleavedStorageCommand.class, LedgerCache.class }) +public class ConvertToInterleavedStorageCommandTest extends BookieCommandTestBase { + + private LedgerDirsManager ledgerDirsManager; + private DbLedgerStorage dbStorage; + private InterleavedLedgerStorage interleavedLedgerStorage; + private LedgerCache interleavedLedgerCache; + + @Rule + private TemporaryFolder folder = new TemporaryFolder(); + + public ConvertToInterleavedStorageCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + createTmpFile(); + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(conf)).thenReturn(conf); + + DiskChecker diskChecker = mock(DiskChecker.class); + whenNew(DiskChecker.class).withArguments(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()) + .thenReturn(diskChecker); + + ledgerDirsManager = mock(LedgerDirsManager.class); + whenNew(LedgerDirsManager.class).withParameterTypes(ServerConfiguration.class, File[].class, DiskChecker.class) + .withArguments(conf, conf.getLedgerDirs(), diskChecker).thenReturn(ledgerDirsManager); + when(ledgerDirsManager.getAllLedgerDirs()).thenReturn(getFileList()); + + dbStorage = mock(DbLedgerStorage.class); + whenNew(DbLedgerStorage.class).withNoArguments().thenReturn(dbStorage); + when(dbStorage.getActiveLedgersInRange(anyLong(), anyLong())).thenReturn(this::getLedgerId); + + interleavedLedgerCache = mock(LedgerCache.class); + doNothing().when(interleavedLedgerCache).flushLedger(anyBoolean()); + + interleavedLedgerStorage = mock(InterleavedLedgerStorage.class); + whenNew(InterleavedLedgerStorage.class).withNoArguments().thenReturn(interleavedLedgerStorage); + doNothing().when(interleavedLedgerStorage).flush(); + doNothing().when(interleavedLedgerStorage).shutdown(); + when(interleavedLedgerStorage.getLedgerCache()).thenReturn(interleavedLedgerCache); + } + + private Iterator getLedgerId() { + Vector longs = new Vector<>(); + LongStream.range(0L, 10L).forEach(longs::add); + return longs.iterator(); + } + + private List getFileList() { + List files = new LinkedList<>(); + files.add(folder.getRoot()); + return files; + } + + private void createTmpFile() { + try { + folder.newFile("ledgers"); + folder.newFile("locations"); + System.out.println(folder.getRoot().getAbsolutePath()); + } catch (IOException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + @Test + public void testConvertToInterleavedStorageCommand() { + ConvertToInterleavedStorageCommand cmd = new ConvertToInterleavedStorageCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "" })); + + try { + verifyNew(ServerConfiguration.class).withArguments(eq(conf)); + verifyNew(LedgerDirsManager.class, times(2)) + .withArguments(eq(conf), any(File[].class), any(DiskChecker.class)); + verifyNew(DbLedgerStorage.class, times(1)).withNoArguments(); + verifyNew(InterleavedLedgerStorage.class, times(1)).withNoArguments(); + + verify(dbStorage, times(1)).initialize(eq(conf), eq(null), any(LedgerDirsManager.class), + any(LedgerDirsManager.class), eq(null), any(CheckpointSource.class), + any(Checkpointer.class), eq(NullStatsLogger.INSTANCE), eq(PooledByteBufAllocator.DEFAULT)); + verify(interleavedLedgerStorage, times(1)) + .initialize(eq(conf), eq(null), any(LedgerDirsManager.class), + any(LedgerDirsManager.class), eq(null), any(CheckpointSource.class), + any(Checkpointer.class), eq(NullStatsLogger.INSTANCE), eq(PooledByteBufAllocator.DEFAULT)); + verify(dbStorage, times(1)).getActiveLedgersInRange(anyLong(), anyLong()); + verify(dbStorage, times(10)).readMasterKey(anyLong()); + verify(interleavedLedgerStorage, times(10)).setMasterKey(anyLong(), any()); + verify(dbStorage, times(10)).getLastEntryInLedger(anyLong()); + verify(dbStorage, times(10)).getLocation(anyLong(), anyLong()); + verify(dbStorage, times(1)).shutdown(); + verify(interleavedLedgerCache, times(1)).flushLedger(true); + verify(interleavedLedgerStorage, times(1)).flush(); + verify(interleavedLedgerStorage, times(1)).shutdown(); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } +} From 0d4aa179e89baec176f73e19440e01307c0059c4 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 25 Mar 2019 11:14:16 +0800 Subject: [PATCH 0242/1642] Migrate command `listfilesondisk` Descriptions of the changes in this PR: - Replace command `listfilesondisk` ### Motivation - Use `bkctl` to run command `listfilesondisk` - #1989 ### Changes - Add command in `bkctl` Reviewers: Jia Zhai , Sijie Guo This closes #1990 from zymap/command-listfilesondisc and squashes the following commits: 922e2b70a [Sijie Guo] Merge branch 'master' into command-listfilesondisc 8d72a00bd [Yong Zhang] Migrate command `listfilesondisk` --- .../apache/bookkeeper/bookie/BookieShell.java | 36 +----- .../bookie/ListFilesOnDiscCommand.java | 104 ++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../bookie/ListFilesOnDiscCommandTest.java | 117 ++++++++++++++++++ 4 files changed, 228 insertions(+), 31 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListFilesOnDiscCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListFilesOnDiscCommandTest.java 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 8ac9c85f1c8..fa52bc7140b 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 @@ -110,6 +110,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; @@ -1601,38 +1602,11 @@ public int runCmd(CommandLine cmdLine) throws Exception { boolean journal = cmdLine.hasOption("txn"); boolean entrylog = cmdLine.hasOption("log"); boolean index = cmdLine.hasOption("idx"); - boolean all = false; - if (!journal && !entrylog && !index && !all) { - all = true; - } - - if (all || journal) { - File[] journalDirs = bkConf.getJournalDirs(); - List journalFiles = listFilesAndSort(journalDirs, "txn"); - System.out.println("--------- Printing the list of Journal Files ---------"); - for (File journalFile : journalFiles) { - System.out.println(journalFile.getCanonicalPath()); - } - System.out.println(); - } - if (all || entrylog) { - File[] ledgerDirs = bkConf.getLedgerDirs(); - List ledgerFiles = listFilesAndSort(ledgerDirs, "log"); - System.out.println("--------- Printing the list of EntryLog/Ledger Files ---------"); - for (File ledgerFile : ledgerFiles) { - System.out.println(ledgerFile.getCanonicalPath()); - } - System.out.println(); - } - if (all || index) { - File[] indexDirs = (bkConf.getIndexDirs() == null) ? bkConf.getLedgerDirs() : bkConf.getIndexDirs(); - List indexFiles = listFilesAndSort(indexDirs, "idx"); - System.out.println("--------- Printing the list of Index Files ---------"); - for (File indexFile : indexFiles) { - System.out.println(indexFile.getCanonicalPath()); - } - } + ListFilesOnDiscCommand.LFODFlags flags = new ListFilesOnDiscCommand.LFODFlags().journal(journal) + .entrylog(entrylog).index(index); + ListFilesOnDiscCommand cmd = new ListFilesOnDiscCommand(flags); + cmd.apply(bkConf, flags); return 0; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListFilesOnDiscCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListFilesOnDiscCommand.java new file mode 100644 index 00000000000..35d694bdc8c --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListFilesOnDiscCommand.java @@ -0,0 +1,104 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.File; +import java.io.IOException; +import java.util.List; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.BookieShell; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Command to list the files in JournalDirectory/LedgerDirectories/IndexDirectories. + */ +public class ListFilesOnDiscCommand extends BookieCommand { + + private static final String NAME = "listfilesondisc"; + private static final String DESC = "List the files in JournalDirectory/LedgerDirectories/IndexDirectories."; + + public ListFilesOnDiscCommand() { + this(new LFODFlags()); + } + + public ListFilesOnDiscCommand(LFODFlags flags) { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build()); + } + + /** + * Flags for list files on disc command. + */ + @Accessors(fluent = true) + @Setter + public static class LFODFlags extends CliFlags { + @Parameter(names = {"-txn", "--journal"}, description = "Print list of Journal Files") + private boolean journal; + + @Parameter(names = {"-log", "--entrylog"}, description = "Print list of EntryLog Files") + private boolean entrylog; + + @Parameter(names = {"-idx", "--index"}, description = "Print list of Index Files") + private boolean index; + } + + @Override + public boolean apply(ServerConfiguration conf, LFODFlags cmdFlags) { + try { + return handler(conf, cmdFlags); + } catch (IOException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean handler(ServerConfiguration conf, LFODFlags cmd) throws IOException { + if (cmd.journal) { + File[] journalDirs = conf.getJournalDirs(); + List journalFiles = BookieShell.listFilesAndSort(journalDirs, "txn"); + System.out.println("--------- Printing the list of Journal Files ---------"); + for (File journalFile : journalFiles) { + System.out.println(journalFile.getCanonicalPath()); + } + System.out.println(); + } + if (cmd.entrylog) { + File[] ledgerDirs = conf.getLedgerDirs(); + List ledgerFiles = BookieShell.listFilesAndSort(ledgerDirs, "log"); + System.out.println("--------- Printing the list of EntryLog/Ledger Files ---------"); + for (File ledgerFile : ledgerFiles) { + System.out.println(ledgerFile.getCanonicalPath()); + } + System.out.println(); + } + if (cmd.index) { + File[] indexDirs = (conf.getIndexDirs() == null) ? conf.getLedgerDirs() : conf.getIndexDirs(); + List indexFiles = BookieShell.listFilesAndSort(indexDirs, "idx"); + System.out.println("--------- Printing the list of Index Files ---------"); + for (File indexFile : indexFiles) { + System.out.println(indexFile.getCanonicalPath()); + } + } + return true; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 9b2277220b9..9390d2e6932 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -27,6 +27,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -50,6 +51,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new FormatCommand()) .addCommand(new SanityTestCommand()) .addCommand(new LedgerCommand()) + .addCommand(new ListFilesOnDiscCommand()) .addCommand(new ConvertToDBStorageCommand()) .addCommand(new ConvertToInterleavedStorageCommand()).build(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListFilesOnDiscCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListFilesOnDiscCommandTest.java new file mode 100644 index 00000000000..e9528faf864 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListFilesOnDiscCommandTest.java @@ -0,0 +1,117 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.powermock.api.mockito.PowerMockito.whenNew; + +import java.io.File; +import java.io.IOException; +import org.apache.bookkeeper.bookie.BookieShell; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + + +/** + * Unit test for {@link ListFilesOnDiscCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ ListFilesOnDiscCommand.class }) +public class ListFilesOnDiscCommandTest extends BookieCommandTestBase { + + @Rule + private TemporaryFolder folder = new TemporaryFolder(); + + public ListFilesOnDiscCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + createTmpDirs(); + whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + } + + private void createTmpDirs() throws IOException { + File journals = folder.newFolder("journals"); + conf.setJournalDirsName(new String[] { journals.getAbsolutePath() }); + journals.mkdir(); + File ledgers = folder.newFolder("ledgers"); + conf.setLedgerDirNames(new String[] { ledgers.getAbsolutePath() }); + ledgers.mkdir(); + File index = folder.newFolder("index"); + conf.setIndexDirName(new String[] { index.getAbsolutePath() }); + index.mkdir(); + + for (int i = 0; i < 10; i++) { + File.createTempFile("journal-" + i, ".txn", journals); + File.createTempFile("ledger-" + i, ".log", ledgers); + File.createTempFile("index-" + i, ".idx", index); + } + System.out.println("over"); + } + + @Test + public void testListJournalCommand() { + testCommand("-txn"); + Assert.assertEquals(10, BookieShell.listFilesAndSort(conf.getJournalDirs(), "txn").size()); + } + + @Test + public void testListJournalLongCommand() { + testCommand("--journal"); + Assert.assertEquals(10, BookieShell.listFilesAndSort(conf.getJournalDirs(), "txn").size()); + } + + @Test + public void testListEntryLogCommand() { + testCommand("-log"); + Assert.assertEquals(10, BookieShell.listFilesAndSort(conf.getLedgerDirs(), "log").size()); + } + + @Test + public void testListEntryLogLongCommand() { + testCommand("--entrylog"); + Assert.assertEquals(10, BookieShell.listFilesAndSort(conf.getLedgerDirs(), "log").size()); + } + + @Test + public void testListIndexCommand() { + testCommand("-idx"); + Assert.assertEquals(10, BookieShell.listFilesAndSort(conf.getIndexDirs(), "idx").size()); + } + + @Test + public void testListIndexLongCommand() { + testCommand("--index"); + Assert.assertEquals(10, BookieShell.listFilesAndSort(conf.getIndexDirs(), "idx").size()); + } + + private void testCommand(String... args) { + ListFilesOnDiscCommand cmd = new ListFilesOnDiscCommand(); + Assert.assertTrue(cmd.apply(bkFlags, args)); + } +} From dd08fa7f1a66bcf9320878f8312842ec9b716755 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Tue, 26 Mar 2019 15:27:23 +0100 Subject: [PATCH 0243/1642] [RELEASE] Update website to include documentation for 4.8.2 Reviewers: Enrico Olivelli This closes #1993 from ivankelly/482site --- site/_config.yml | 5 +- site/docs/4.8.2/admin/autorecovery.md | 128 +++ site/docs/4.8.2/admin/bookies.md | 180 ++++ site/docs/4.8.2/admin/geo-replication.md | 22 + site/docs/4.8.2/admin/http.md | 418 +++++++++ site/docs/4.8.2/admin/metrics.md | 41 + site/docs/4.8.2/admin/perf.md | 3 + site/docs/4.8.2/admin/placement.md | 3 + site/docs/4.8.2/admin/upgrade.md | 175 ++++ site/docs/4.8.2/api/distributedlog-api.md | 395 ++++++++ site/docs/4.8.2/api/ledger-adv-api.md | 111 +++ site/docs/4.8.2/api/ledger-api.md | 841 ++++++++++++++++++ site/docs/4.8.2/api/overview.md | 17 + site/docs/4.8.2/deployment/dcos.md | 142 +++ site/docs/4.8.2/deployment/kubernetes.md | 181 ++++ site/docs/4.8.2/deployment/manual.md | 56 ++ site/docs/4.8.2/development/codebase.md | 3 + site/docs/4.8.2/development/protocol.md | 148 +++ site/docs/4.8.2/getting-started/concepts.md | 202 +++++ .../4.8.2/getting-started/installation.md | 82 ++ .../docs/4.8.2/getting-started/run-locally.md | 16 + site/docs/4.8.2/overview/overview.md | 60 ++ site/docs/4.8.2/overview/releaseNotes.md | 1 + .../4.8.2/overview/releaseNotesTemplate.md | 17 + site/docs/4.8.2/reference/cli.md | 10 + site/docs/4.8.2/reference/config.md | 9 + site/docs/4.8.2/reference/metrics.md | 3 + site/docs/4.8.2/security/overview.md | 21 + site/docs/4.8.2/security/sasl.md | 202 +++++ site/docs/4.8.2/security/tls.md | 210 +++++ site/docs/4.8.2/security/zookeeper.md | 41 + site/releases.md | 8 + 32 files changed, 3749 insertions(+), 2 deletions(-) create mode 100644 site/docs/4.8.2/admin/autorecovery.md create mode 100644 site/docs/4.8.2/admin/bookies.md create mode 100644 site/docs/4.8.2/admin/geo-replication.md create mode 100644 site/docs/4.8.2/admin/http.md create mode 100644 site/docs/4.8.2/admin/metrics.md create mode 100644 site/docs/4.8.2/admin/perf.md create mode 100644 site/docs/4.8.2/admin/placement.md create mode 100644 site/docs/4.8.2/admin/upgrade.md create mode 100644 site/docs/4.8.2/api/distributedlog-api.md create mode 100644 site/docs/4.8.2/api/ledger-adv-api.md create mode 100644 site/docs/4.8.2/api/ledger-api.md create mode 100644 site/docs/4.8.2/api/overview.md create mode 100644 site/docs/4.8.2/deployment/dcos.md create mode 100644 site/docs/4.8.2/deployment/kubernetes.md create mode 100644 site/docs/4.8.2/deployment/manual.md create mode 100644 site/docs/4.8.2/development/codebase.md create mode 100644 site/docs/4.8.2/development/protocol.md create mode 100644 site/docs/4.8.2/getting-started/concepts.md create mode 100644 site/docs/4.8.2/getting-started/installation.md create mode 100644 site/docs/4.8.2/getting-started/run-locally.md create mode 100644 site/docs/4.8.2/overview/overview.md create mode 100644 site/docs/4.8.2/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.8.2/reference/cli.md create mode 100644 site/docs/4.8.2/reference/config.md create mode 100644 site/docs/4.8.2/reference/metrics.md create mode 100644 site/docs/4.8.2/security/overview.md create mode 100644 site/docs/4.8.2/security/sasl.md create mode 100644 site/docs/4.8.2/security/tls.md create mode 100644 site/docs/4.8.2/security/zookeeper.md diff --git a/site/_config.yml b/site/_config.yml index 1ccfb677875..8227fc1c615 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -10,6 +10,7 @@ twitter_url: https://twitter.com/asfbookkeeper versions: # [next_version_placeholder] - "4.9.0" +- "4.8.2" - "4.8.1" - "4.8.0" - "4.7.3" @@ -36,8 +37,8 @@ archived_versions: - "4.0.0" latest_version: "4.10.0-SNAPSHOT" latest_release: "4.9.0" -stable_release: "4.7.3" -distributedlog_version: "4.7.3" +stable_release: "4.8.2" +distributedlog_version: "4.8.2" defaults: - scope: diff --git a/site/docs/4.8.2/admin/autorecovery.md b/site/docs/4.8.2/admin/autorecovery.md new file mode 100644 index 00000000000..b1dd078f9b2 --- /dev/null +++ b/site/docs/4.8.2/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: + +* the `shell recover` option +* 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 shell recover \ + 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 shell recover \ + 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.8.2/admin/bookies.md b/site/docs/4.8.2/admin/bookies.md new file mode 100644 index 00000000000..1b0427dae3c --- /dev/null +++ b/site/docs/4.8.2/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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + \ + \ + + ``` + + 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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + 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.8.2/admin/geo-replication.md b/site/docs/4.8.2/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.8.2/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.8.2/admin/http.md b/site/docs/4.8.2/admin/http.md new file mode 100644 index 00000000000..7c1abea49f2 --- /dev/null +++ b/site/docs/4.8.2/admin/http.md @@ -0,0 +1,418 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 4 components: +1. Heartbeat: heartbeat for a specific bookie. +1. Config: doing the server configuration for a specific bookie. +1. Ledger: HTTP endpoints related to ledgers. +1. Bookie: HTTP endpoints related to bookies. +1. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Config + +### Endpoint: /metrics +1. Method: GET + * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/bookie/gc +1. Method: PUT + * Description: trigger gc for this bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.8.2/admin/metrics.md b/site/docs/4.8.2/admin/metrics.md new file mode 100644 index 00000000000..142df3dcd2d --- /dev/null +++ b/site/docs/4.8.2/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.prometheus.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.8.2/admin/perf.md b/site/docs/4.8.2/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.8.2/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.8.2/admin/placement.md b/site/docs/4.8.2/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.8.2/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.8.2/admin/upgrade.md b/site/docs/4.8.2/admin/upgrade.md new file mode 100644 index 00000000000..57c65208131 --- /dev/null +++ b/site/docs/4.8.2/admin/upgrade.md @@ -0,0 +1,175 @@ +--- +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.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.7.x to 4.8.X upgrade + +In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.8.2/api/distributedlog-api.md b/site/docs/4.8.2/api/distributedlog-api.md new file mode 100644 index 00000000000..f073b291048 --- /dev/null +++ b/site/docs/4.8.2/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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.8.2/api/ledger-adv-api.md b/site/docs/4.8.2/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.8.2/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.8.2/api/ledger-api.md b/site/docs/4.8.2/api/ledger-api.md new file mode 100644 index 00000000000..b6cb0f02a95 --- /dev/null +++ b/site/docs/4.8.2/api/ledger-api.md @@ -0,0 +1,841 @@ +--- +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 + +{{ site.latest_version }} + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +{{ site.latest_version }} + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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: '{{ site.latest_version }}' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:{{ site.latest_version }}' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` + +### Relaxing Durability + +In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. +In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will +be always readable by other clients. + +On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after +writing the entry to SO buffers without waiting for an fsync. +In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. +Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. + +In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble ackknowledge the call after +performing an fsync to the disk which is storing the journal. +This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. + +The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. +In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. + + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete + + +wh.force().get(); // wait for fsync, make data available to readers and to the replicator + +wh.close(); // seal the ledger + +``` diff --git a/site/docs/4.8.2/api/overview.md b/site/docs/4.8.2/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.8.2/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.8.2/deployment/dcos.md b/site/docs/4.8.2/deployment/dcos.md new file mode 100644 index 00000000000..dab023deb7a --- /dev/null +++ b/site/docs/4.8.2/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 [{{ site.latest_version }}] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.8.2/deployment/kubernetes.md b/site/docs/4.8.2/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.8.2/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.8.2/deployment/manual.md b/site/docs/4.8.2/deployment/manual.md new file mode 100644 index 00000000000..daafd5556f5 --- /dev/null +++ b/site/docs/4.8.2/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.8.2/development/codebase.md b/site/docs/4.8.2/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.8.2/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.8.2/development/protocol.md b/site/docs/4.8.2/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.8.2/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.8.2/getting-started/concepts.md b/site/docs/4.8.2/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.8.2/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.8.2/getting-started/installation.md b/site/docs/4.8.2/getting-started/installation.md new file mode 100644 index 00000000000..7e92b471581 --- /dev/null +++ b/site/docs/4.8.2/getting-started/installation.md @@ -0,0 +1,82 @@ +--- +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 +``` + +Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. + +```shell +$ mvn package -Dstream +``` + +> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin + +> You can enable `table service` by adding the `-Dstream` flag when running above commands. + +## 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.8.2/getting-started/run-locally.md b/site/docs/4.8.2/getting-started/run-locally.md new file mode 100644 index 00000000000..edbfab9fda6 --- /dev/null +++ b/site/docs/4.8.2/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 +$ bookkeeper-server/bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.8.2/overview/overview.md b/site/docs/4.8.2/overview/overview.md new file mode 100644 index 00000000000..808693803bc --- /dev/null +++ b/site/docs/4.8.2/overview/overview.md @@ -0,0 +1,60 @@ +--- +title: Apache BookKeeper™ 4.8.2 +--- + + +This documentation is for Apache BookKeeper™ version {{ site.latest_version }}. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper {{ site.latest_version }} Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.8.2/overview/releaseNotes.md b/site/docs/4.8.2/overview/releaseNotes.md index 5907b074d81..6a76e9ac1f8 100644 --- a/site/docs/4.8.2/overview/releaseNotes.md +++ b/site/docs/4.8.2/overview/releaseNotes.md @@ -46,3 +46,4 @@ There is no dependency upgrade from 4.8.1. ## Full list of changes - [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.2+is%3Aclosed) + diff --git a/site/docs/4.8.2/overview/releaseNotesTemplate.md b/site/docs/4.8.2/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..4dedbb4b486 --- /dev/null +++ b/site/docs/4.8.2/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.8.2 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to {{ site.latest_version }}. The technical details of this release are summarized +below. + +## Highlights + +[List the highlights] + +## Details + +[list to issues list] + diff --git a/site/docs/4.8.2/reference/cli.md b/site/docs/4.8.2/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.8.2/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.8.2/reference/config.md b/site/docs/4.8.2/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.8.2/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.8.2/reference/metrics.md b/site/docs/4.8.2/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.8.2/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.8.2/security/overview.md b/site/docs/4.8.2/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.8.2/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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.8.2/security/sasl.md b/site/docs/4.8.2/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.8.2/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.8.2/security/tls.md b/site/docs/4.8.2/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.8.2/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.8.2/security/zookeeper.md b/site/docs/4.8.2/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.8.2/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) diff --git a/site/releases.md b/site/releases.md index 718af9256f2..447e2e19896 100644 --- a/site/releases.md +++ b/site/releases.md @@ -64,6 +64,14 @@ Client Guide | API docs ## News +### 19 March, 2019 Release 4.8.2 available + +This is the 17th release of Apache BookKeeper! + +The 4.8.2 release is a bugfix release which fixes a couple of issues reported from users of 4.8.1. + +See [BookKeeper 4.8.2 Release Notes](../docs/4.8.2/overview/releaseNotes) for details. + ### 31 January, 2019 Release 4.9.0 available This is the 16th release of Apache BookKeeper! From c9e0854f21deb341669dbb18d4ad8a0d418b5e37 Mon Sep 17 00:00:00 2001 From: Boyang Jerry Peng Date: Tue, 26 Mar 2019 09:33:54 -0500 Subject: [PATCH 0244/1642] fix potential NPE when releasing entry that is null Descriptions of the changes in this PR: ### Motivation A interrupt exception can occur during the poll operation of the blocking and cause a NPE to be thrown ### Changes Check if entry is null before trying to release it Reviewers: Ivan Kelly , Sijie Guo , Matteo Merli This closes #1998 from jerrypeng/fix_NPE --- .../impl/logsegment/BKLogSegmentEntryReader.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java index 813e9742a30..f414bf45555 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; +import io.netty.util.ReferenceCountUtil; import java.io.IOException; import java.util.ArrayList; import java.util.Enumeration; @@ -800,7 +801,7 @@ private void readEntriesFromReadAheadCache(PendingReadRequest nextRequest) { return; } } finally { - removedEntry.release(); + ReferenceCountUtil.safeRelease(removedEntry); } } else if (skipBrokenEntries && BKException.Code.DigestMatchException == entry.getRc()) { // skip this entry and move forward From 9b4f88113f360241d0ca0442e8f09f7d45c662b2 Mon Sep 17 00:00:00 2001 From: Like Date: Wed, 27 Mar 2019 13:11:17 +0800 Subject: [PATCH 0245/1642] Fix error message for unrecognized number-of-bookies Closes #1925 Reviewers: Enrico Olivelli , Sijie Guo This closes #1997 from liketic/fix-issue-1925 --- .../apache/bookkeeper/util/LocalBookKeeper.java | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) 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 e431e8e5e70..97884a6454d 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 @@ -428,25 +428,32 @@ private void serializeLocalBookieConfig(ServerConfiguration localBookieConfig, S Iterator keys = localBookieConfig.getKeys(); try (PrintWriter writer = new PrintWriter(localBookieConfFile, "UTF-8")) { while (keys.hasNext()) { - String key = keys.next().toString(); + String key = keys.next(); String[] values = localBookieConfig.getStringArray(key); StringBuilder concatenatedValue = new StringBuilder(values[0]); for (int i = 1; i < values.length; i++) { - concatenatedValue.append("," + values[i]); + concatenatedValue.append(",").append(values[i]); } writer.println(key + "=" + concatenatedValue.toString()); } } } - public static void main(String[] args) throws Exception, SecurityException { + public static void main(String[] args) { try { if (args.length < 1) { usage(); System.exit(-1); } - int numBookies = Integer.parseInt(args[0]); + int numBookies = 0; + try { + numBookies = Integer.parseInt(args[0]); + } catch (NumberFormatException nfe) { + LOG.error("Unrecognized number-of-bookies: {}", args[0]); + usage(); + System.exit(-1); + } ServerConfiguration conf = new ServerConfiguration(); conf.setAllowLoopback(true); From 23d9b02b6ef9268013ee901d8e0b1629e5f34d75 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 27 Mar 2019 23:35:48 +0800 Subject: [PATCH 0246/1642] Fix when met unexpect entry id crashed Fixes #2001 *Motivation* When met a unexpect entry id, db indexes failed cleanup. *Modifications* Catch the exception and ignore it. Master Issue: #2001 Reviewers: Jia Zhai , Matteo Merli This closes #2002 from zymap/fixnoentryexception --- .../storage/ldb/EntryLocationIndex.java | 10 +++++- .../storage/ldb/EntryLocationIndexTest.java | 32 +++++++++++++++++++ 2 files changed, 41 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java index 5673883b919..6b01c503fff 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java @@ -216,7 +216,15 @@ public void removeOffsetFromDeletedLedgers() throws IOException { } long firstEntryId = ArrayUtil.getLong(firstKeyRes.getKey(), 8); - long lastEntryId = getLastEntryInLedgerInternal(ledgerId); + long lastEntryId; + try { + lastEntryId = getLastEntryInLedgerInternal(ledgerId); + } catch (Bookie.NoEntryException nee) { + if (log.isDebugEnabled()) { + log.debug("No last entry id found for ledger {}", ledgerId); + } + continue; + } if (log.isDebugEnabled()) { log.debug("Deleting index for ledger {} entries ({} -> {})", ledgerId, firstEntryId, lastEntryId); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndexTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndexTest.java index 6e83ffd58a4..f1cefed1615 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndexTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndexTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals; import java.io.File; +import java.io.IOException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -108,4 +109,35 @@ public void addLedgerAfterDeleteTest() throws Exception { idx.close(); } + + // test non exist entry + @Test + public void testDeleteSpecialEntry() throws IOException { + File tmpDir = File.createTempFile("bkTest", ".dir"); + tmpDir.delete(); + tmpDir.mkdir(); + tmpDir.deleteOnExit(); + + EntryLocationIndex idx = new EntryLocationIndex(serverConfiguration, KeyValueStorageRocksDB.factory, + tmpDir.getAbsolutePath(), NullStatsLogger.INSTANCE); + + // Add some dummy indexes + idx.addLocation(40312, -1, 1); + idx.addLocation(40313, 10, 2); + idx.addLocation(40320, 0, 3); + + // Add more indexes in a different batch + idx.addLocation(40313, 11, 5); + idx.addLocation(40313, 12, 6); + idx.addLocation(40320, 1, 7); + + // delete a non exist entry + idx.delete(40312); + idx.removeOffsetFromDeletedLedgers(); + + // another delete entry operation shouldn't effected + idx.delete(40313); + idx.removeOffsetFromDeletedLedgers(); + assertEquals(0, idx.getLocation(40312, 10)); + } } From 9a0157c4feaedd920399242f3920f8415600625a Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 29 Mar 2019 12:44:24 +0800 Subject: [PATCH 0247/1642] Migrate command `readjournal` Descriptions of the changes in this PR: - Use `bkctl` execute `readjournal` command ### Motivation - Migrate from bookieshelll Reviewers: Jia Zhai , Sijie Guo This closes #2000 from zymap/command-readjournal --- .../org/apache/bookkeeper/bookie/Bookie.java | 4 +- .../apache/bookkeeper/bookie/BookieShell.java | 196 ++-------------- .../apache/bookkeeper/bookie/LedgerCache.java | 2 +- .../tools/cli/commands/bookie/FormatUtil.java | 98 ++++++++ .../commands/bookie/ReadJournalCommand.java | 213 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 5 +- .../bookie/ReadJournalCommandTest.java | 128 +++++++++++ 7 files changed, 466 insertions(+), 180 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatUtil.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommandTest.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 44c950978de..06ccd588b48 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 @@ -116,8 +116,8 @@ public class Bookie extends BookieCriticalThread { final HandleFactory handles; final boolean entryLogPerLedgerEnabled; - static final long METAENTRY_ID_LEDGER_KEY = -0x1000; - static final long METAENTRY_ID_FENCE_KEY = -0x2000; + public static final long METAENTRY_ID_LEDGER_KEY = -0x1000; + public static final long METAENTRY_ID_FENCE_KEY = -0x2000; public static final long METAENTRY_ID_FORCE_LEDGER = -0x4000; static final long METAENTRY_ID_LEDGER_EXPLICITLAC = -0x8000; 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 fa52bc7140b..8c448a11cd7 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 @@ -29,7 +29,6 @@ import com.google.common.util.concurrent.UncheckedExecutionException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; -import io.netty.buffer.Unpooled; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; @@ -38,7 +37,6 @@ import java.io.IOException; import java.io.Serializable; import java.net.URI; -import java.nio.ByteBuffer; import java.nio.file.FileSystems; import java.nio.file.Files; import java.nio.file.Path; @@ -51,7 +49,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.Formatter; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -71,10 +68,10 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.LongStream; + 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; import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.MetaStoreException; @@ -107,10 +104,12 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.FormatUtil; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; @@ -122,7 +121,6 @@ import org.apache.bookkeeper.tools.cli.commands.cookie.UpdateCookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.util.BookKeeperConstants; -import org.apache.bookkeeper.util.DiskChecker; import org.apache.bookkeeper.util.EntryFormatter; import org.apache.bookkeeper.util.IOUtils; import org.apache.bookkeeper.util.LedgerIdFormatter; @@ -1437,55 +1435,25 @@ public int runCmd(CommandLine cmdLine) throws Exception { return -1; } + long journalId = -1L; + String filename = ""; + try { + journalId = Long.parseLong(leftArgs[0]); + } catch (NumberFormatException nfe) { + filename = leftArgs[0]; + } + boolean printMsg = false; if (cmdLine.hasOption("m")) { printMsg = true; } - Journal journal = null; - if (getJournals().size() > 1) { - if (!cmdLine.hasOption("dir")) { - System.err.println("ERROR: invalid or missing journal directory"); - printUsage(); - return -1; - } - - File journalDirectory = new File(cmdLine.getOptionValue("dir")); - for (Journal j : getJournals()) { - if (j.getJournalDirectory().equals(journalDirectory)) { - journal = j; - break; - } - } - - if (journal == null) { - System.err.println("ERROR: journal directory not found"); - printUsage(); - return -1; - } - } else { - journal = getJournals().get(0); - } - - long journalId; - try { - journalId = Long.parseLong(leftArgs[0]); - } catch (NumberFormatException nfe) { - // not a journal id - File f = new File(leftArgs[0]); - String name = f.getName(); - if (!name.endsWith(".txn")) { - // not a journal file - System.err.println("ERROR: invalid journal file name " + leftArgs[0]); - printUsage(); - return -1; - } - String idString = name.split("\\.")[0]; - journalId = Long.parseLong(idString, 16); - } - // scan journal - scanJournal(journal, journalId, printMsg); - return 0; + ReadJournalCommand.ReadJournalFlags flags = new ReadJournalCommand.ReadJournalFlags().msg(printMsg) + .fileName(filename).journalId(journalId) + .dir(cmdLine.getOptionValue("dir")); + ReadJournalCommand cmd = new ReadJournalCommand(ledgerIdFormatter, entryFormatter); + boolean result = cmd.apply(bkConf, flags); + return result ? 0 : -1; } @Override @@ -1626,7 +1594,6 @@ Options getOptions() { } } - /** * Command to print help message. */ @@ -1803,7 +1770,6 @@ int runCmd(CommandLine cmdLine) throws Exception { } } - /** * Print which node has the auditor lock. */ @@ -2493,7 +2459,6 @@ public interface UpdateLedgerNotifier { void progress(long updated, long issued); } - /** * Convert bookie indexes from InterleavedStorage to DbLedgerStorage format. */ @@ -2898,29 +2863,6 @@ protected void scanEntryLog(long logId, EntryLogScanner scanner) throws IOExcept entryLogger.scanEntryLog(logId, scanner); } - private synchronized List getJournals() throws IOException { - if (null == journals) { - journals = Lists.newArrayListWithCapacity(bkConf.getJournalDirs().length); - int idx = 0; - for (File journalDir : bkConf.getJournalDirs()) { - journals.add(new Journal(idx++, new File(journalDir, BookKeeperConstants.CURRENT_DIR), bkConf, - new LedgerDirsManager(bkConf, bkConf.getLedgerDirs(), - new DiskChecker(bkConf.getDiskUsageThreshold(), bkConf.getDiskUsageWarnThreshold())))); - } - } - return journals; - } - - /** - * Scan journal file. - * - * @param journalId Journal File Id - * @param scanner Journal File Scanner - */ - protected void scanJournal(Journal journal, long journalId, JournalScanner scanner) throws IOException { - journal.scanJournal(journalId, 0L, scanner); - } - /// /// Bookie Shell Commands /// @@ -2953,7 +2895,7 @@ public boolean accept(long ledgerId) { @Override public void process(long ledgerId, long startPos, ByteBuf entry) { - formatEntry(startPos, entry, printMsg); + FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter); } }); } @@ -2985,7 +2927,7 @@ public void process(long candidateLedgerId, long startPos, ByteBuf entry) { if ((candidateLedgerId == entrysLedgerId) && (candidateLedgerId == ledgerId) && ((entrysEntryId == entryId) || (entryId == -1))) { entryFound.setValue(true); - formatEntry(startPos, entry, printMsg); + FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter); } } }); @@ -3034,7 +2976,7 @@ public void process(long ledgerId, long entryStartPos, ByteBuf entry) { */ long entryEndPos = entryStartPos + entrySize + 4 - 1; if (((rangeEndPos == -1) || (entryStartPos <= rangeEndPos)) && (rangeStartPos <= entryEndPos)) { - formatEntry(entryStartPos, entry, printMsg); + FormatUtil.formatEntry(entryStartPos, entry, printMsg, ledgerIdFormatter, entryFormatter); entryFound.setValue(true); } } @@ -3049,40 +2991,6 @@ public void process(long ledgerId, long entryStartPos, ByteBuf entry) { } } - /** - * Scan a journal file. - * - * @param journalId Journal File Id - * @param printMsg Whether printing the entry data. - */ - protected void scanJournal(Journal journal, long journalId, final boolean printMsg) throws Exception { - System.out.println("Scan journal " + journalId + " (" + Long.toHexString(journalId) + ".txn)"); - scanJournal(journal, journalId, new JournalScanner() { - boolean printJournalVersion = false; - - @Override - public void process(int journalVersion, long offset, ByteBuffer entry) throws IOException { - if (!printJournalVersion) { - System.out.println("Journal Version : " + journalVersion); - printJournalVersion = true; - } - formatEntry(offset, Unpooled.wrappedBuffer(entry), printMsg); - } - }); - } - - /** - * Print last log mark. - */ - protected void printLastLogMark() throws IOException { - for (Journal journal : getJournals()) { - LogMark lastLogMark = journal.getLastLogMark().getCurMark(); - System.out.println("LastLogMark: Journal Id - " + lastLogMark.getLogFileId() + "(" - + Long.toHexString(lastLogMark.getLogFileId()) + ".txn), Pos - " - + lastLogMark.getLogFileOffset()); - } - } - /** * Format the entry into a readable format. * @@ -3102,70 +3010,6 @@ private void formatEntry(LedgerEntry entry, boolean printMsg) { } } - /** - * Format the message into a readable format. - * - * @param pos - * File offset of the message stored in entry log file - * @param recBuff - * Entry Data - * @param printMsg - * Whether printing the message body - */ - private void formatEntry(long pos, ByteBuf recBuff, boolean printMsg) { - int entrySize = recBuff.readableBytes(); - long ledgerId = recBuff.readLong(); - long entryId = recBuff.readLong(); - - System.out.println("--------- Lid=" + ledgerIdFormatter.formatLedgerId(ledgerId) + ", Eid=" + entryId - + ", ByteOffset=" + pos + ", EntrySize=" + entrySize + " ---------"); - if (entryId == Bookie.METAENTRY_ID_LEDGER_KEY) { - int masterKeyLen = recBuff.readInt(); - byte[] masterKey = new byte[masterKeyLen]; - recBuff.readBytes(masterKey); - System.out.println("Type: META"); - System.out.println("MasterKey: " + bytes2Hex(masterKey)); - System.out.println(); - return; - } - if (entryId == Bookie.METAENTRY_ID_FENCE_KEY) { - System.out.println("Type: META"); - System.out.println("Fenced"); - System.out.println(); - return; - } - // process a data entry - long lastAddConfirmed = recBuff.readLong(); - System.out.println("Type: DATA"); - System.out.println("LastConfirmed: " + lastAddConfirmed); - if (!printMsg) { - System.out.println(); - return; - } - // skip digest checking - recBuff.skipBytes(8); - System.out.println("Data:"); - System.out.println(); - try { - byte[] ret = new byte[recBuff.readableBytes()]; - recBuff.readBytes(ret); - entryFormatter.formatEntry(ret); - } catch (Exception e) { - System.out.println("N/A. Corrupted."); - } - System.out.println(); - } - - static String bytes2Hex(byte[] data) { - StringBuilder sb = new StringBuilder(data.length * 2); - Formatter formatter = new Formatter(sb); - for (byte b : data) { - formatter.format("%02x", b); - } - formatter.close(); - return sb.toString(); - } - private static int getOptionIntValue(CommandLine cmdLine, String option, int defaultVal) { if (cmdLine.hasOption(option)) { String val = cmdLine.getOptionValue(option); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java index 86984b7ac94..993bd0cd007 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java @@ -21,7 +21,7 @@ package org.apache.bookkeeper.bookie; -import static org.apache.bookkeeper.bookie.BookieShell.bytes2Hex; +import static org.apache.bookkeeper.tools.cli.commands.bookie.FormatUtil.bytes2Hex; import io.netty.buffer.ByteBuf; import java.io.Closeable; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatUtil.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatUtil.java new file mode 100644 index 00000000000..d786c7ee72a --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatUtil.java @@ -0,0 +1,98 @@ +/* + * 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.tools.cli.commands.bookie; + +import io.netty.buffer.ByteBuf; +import java.util.Formatter; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.util.EntryFormatter; +import org.apache.bookkeeper.util.LedgerIdFormatter; + +/** + * .Provide to format message. + */ +public class FormatUtil { + + /** + * Format the message into a readable format. + * @param pos + * File offset of the message stored in entry log file + * @param recBuff + * Entry Data + * @param printMsg + * Whether printing the message body + * @param ledgerIdFormatter + * @param entryFormatter + */ + public static void formatEntry(long pos, ByteBuf recBuff, boolean printMsg, LedgerIdFormatter ledgerIdFormatter, + EntryFormatter entryFormatter) { + int entrySize = recBuff.readableBytes(); + long ledgerId = recBuff.readLong(); + long entryId = recBuff.readLong(); + + System.out.println( + "--------- Lid=" + ledgerIdFormatter.formatLedgerId(ledgerId) + ", Eid=" + entryId + ", ByteOffset=" + pos + + ", EntrySize=" + entrySize + " ---------"); + if (entryId == Bookie.METAENTRY_ID_LEDGER_KEY) { + int masterKeyLen = recBuff.readInt(); + byte[] masterKey = new byte[masterKeyLen]; + recBuff.readBytes(masterKey); + System.out.println("Type: META"); + System.out.println("MasterKey: " + bytes2Hex(masterKey)); + System.out.println(); + return; + } + if (entryId == Bookie.METAENTRY_ID_FENCE_KEY) { + System.out.println("Type: META"); + System.out.println("Fenced"); + System.out.println(); + return; + } + // process a data entry + long lastAddConfirmed = recBuff.readLong(); + System.out.println("Type: DATA"); + System.out.println("LastConfirmed: " + lastAddConfirmed); + if (!printMsg) { + System.out.println(); + return; + } + // skip digest checking + recBuff.skipBytes(8); + System.out.println("Data:"); + System.out.println(); + try { + byte[] ret = new byte[recBuff.readableBytes()]; + recBuff.readBytes(ret); + entryFormatter.formatEntry(ret); + } catch (Exception e) { + System.out.println("N/A. Corrupted."); + } + System.out.println(); + } + + public static String bytes2Hex(byte[] data) { + StringBuilder sb = new StringBuilder(data.length * 2); + Formatter formatter = new Formatter(sb); + for (byte b : data) { + formatter.format("%02x", b); + } + formatter.close(); + return sb.toString(); + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommand.java new file mode 100644 index 00000000000..672925745d1 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommand.java @@ -0,0 +1,213 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.UncheckedExecutionException; +import io.netty.buffer.Unpooled; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.Journal; +import org.apache.bookkeeper.bookie.LedgerDirsManager; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.BookKeeperConstants; +import org.apache.bookkeeper.util.DiskChecker; +import org.apache.bookkeeper.util.EntryFormatter; +import org.apache.bookkeeper.util.LedgerIdFormatter; + +/** + * Command to scan a journal file and format the entries into readable format. + */ +public class ReadJournalCommand extends BookieCommand { + + private static final String NAME = "readjournal"; + private static final String DESC = "Scan a journal file and format the entries into readable format."; + private static final long DEFAULT_JOURNALID = -1L; + private static final String DEFAULT = ""; + private LedgerIdFormatter ledgerIdFormatter; + private EntryFormatter entryFormatter; + + List journals = null; + + public ReadJournalCommand() { + this(new ReadJournalFlags()); + } + + public ReadJournalCommand(LedgerIdFormatter idFormatter, EntryFormatter entryFormatter) { + this(new ReadJournalFlags()); + this.ledgerIdFormatter = idFormatter; + this.entryFormatter = entryFormatter; + } + + ReadJournalCommand(ReadJournalFlags flags) { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build()); + } + + /** + * Flag for read journal command. + */ + @Accessors(fluent = true) + @Setter + public static class ReadJournalFlags extends CliFlags { + + @Parameter(names = {"-m", "--msg"}, description = "Print message body") + private boolean msg; + + @Parameter(names = { "-d", "--dir" }, description = "Journal directory (needed if more than one journal " + + "configured)") + private String dir = DEFAULT; + + @Parameter(names = {"-id", "--journalid"}, description = "Journal Id") + private long journalId = DEFAULT_JOURNALID; + + @Parameter(names = {"-f", "--filename"}, description = "Journal file name") + private String fileName = DEFAULT; + + @Parameter(names = {"-l", "--ledgerIdFormatter"}, description = "Set ledger id formatter") + private String ledgerIdFormatter = DEFAULT; + + @Parameter(names = {"-e", "--entryformatter"}, description = "set entry formatter") + private String entryFormatter = DEFAULT; + + } + + @Override + public boolean apply(ServerConfiguration conf, ReadJournalFlags cmdFlags) { + initTools(conf, cmdFlags); + if (!checkArgs(cmdFlags)) { + return false; + } + try { + return handler(conf, cmdFlags); + } catch (IOException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private void initTools(ServerConfiguration conf, ReadJournalFlags flags) { + if (!flags.ledgerIdFormatter.equals(DEFAULT)) { + ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(flags.ledgerIdFormatter, conf); + } else { + ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } + + if (!flags.entryFormatter.equals(DEFAULT)) { + entryFormatter = EntryFormatter.newEntryFormatter(flags.entryFormatter, conf); + } else { + entryFormatter = EntryFormatter.newEntryFormatter(conf); + } + } + private boolean handler(ServerConfiguration conf, ReadJournalFlags cmd) throws IOException { + Journal journal = null; + if (getJournals(conf).size() > 1) { + if (cmd.dir.equals(DEFAULT)) { + System.err.println("ERROR: invalid or missing journal directory"); + usage(); + return false; + } + File journalDirectory = new File(cmd.dir); + for (Journal j : getJournals(conf)) { + if (j.getJournalDirectory().equals(journalDirectory)) { + journal = j; + break; + } + } + + if (journal == null) { + System.err.println("ERROR: journal directory not found"); + usage(); + return false; + } + } else { + journal = getJournals(conf).get(0); + } + + long journalId = cmd.journalId; + if (cmd.journalId == DEFAULT_JOURNALID && !cmd.fileName.equals(DEFAULT)) { + File f = new File(cmd.fileName); + String name = f.getName(); + if (!name.endsWith(".txn")) { + System.err.println("ERROR: invalid journal file name " + cmd.fileName); + usage(); + return false; + } + String idString = name.split("\\.")[0]; + journalId = Long.parseLong(idString); + } + scanJournal(journal, journalId, cmd.msg); + return true; + } + + private boolean checkArgs(ReadJournalFlags flags) { + if ((flags.fileName.equals(DEFAULT) && flags.journalId == DEFAULT_JOURNALID)) { + System.out.println("ERROR: You should figure jounalId or journal filename"); + return false; + } + + return true; + } + + private synchronized List getJournals(ServerConfiguration conf) { + if (null == journals) { + journals = Lists.newArrayListWithCapacity(conf.getJournalDirs().length); + int idx = 0; + for (File journalDir : conf.getJournalDirs()) { + journals.add(new Journal(idx++, new File(journalDir, BookKeeperConstants.CURRENT_DIR), conf, + new LedgerDirsManager(conf, conf.getLedgerDirs(), + new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())))); + } + } + return journals; + } + + /** + * Scan a journal file. + * + * @param journalId Journal File Id + * @param printMsg Whether printing the entry data. + */ + private void scanJournal(Journal journal, long journalId, final boolean printMsg) throws IOException { + System.out.println("Scan journal " + journalId + " (" + Long.toHexString(journalId) + ".txn)"); + scanJournal(journal, journalId, new Journal.JournalScanner() { + boolean printJournalVersion = false; + + @Override + public void process(int journalVersion, long offset, ByteBuffer entry) throws IOException { + if (!printJournalVersion) { + System.out.println("Journal Version : " + journalVersion); + printJournalVersion = true; + } + FormatUtil + .formatEntry(offset, Unpooled.wrappedBuffer(entry), printMsg, ledgerIdFormatter, entryFormatter); + } + }); + } + + private void scanJournal(Journal journal, long journalId, Journal.JournalScanner scanner) throws IOException { + journal.scanJournal(journalId, 0L, scanner); + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 9390d2e6932..c8c781764b2 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -53,7 +54,9 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new LedgerCommand()) .addCommand(new ListFilesOnDiscCommand()) .addCommand(new ConvertToDBStorageCommand()) - .addCommand(new ConvertToInterleavedStorageCommand()).build(); + .addCommand(new ConvertToInterleavedStorageCommand()) + .addCommand(new ReadJournalCommand()) + .build(); public BookieCommandGroup() { super(spec); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommandTest.java new file mode 100644 index 00000000000..d78519a963b --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadJournalCommandTest.java @@ -0,0 +1,128 @@ +/* + * 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.tools.cli.commands.bookie; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; +import static org.powermock.api.mockito.PowerMockito.whenNew; + +import java.io.File; +import org.apache.bookkeeper.bookie.Journal; +import org.apache.bookkeeper.bookie.LedgerDirsManager; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.util.DiskChecker; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + + +/** + * Unit test for read journal command. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ ReadJournalCommand.class, Journal.class, LedgerDirsManager.class, + DiskChecker.class }) +public class ReadJournalCommandTest extends BookieCommandTestBase { + + private Journal journal; + + @Rule + private TemporaryFolder folder = new TemporaryFolder(); + + public ReadJournalCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + DiskChecker checker = mock(DiskChecker.class); + whenNew(DiskChecker.class).withArguments(eq(conf.getDiskUsageThreshold()), eq(conf.getDiskUsageWarnThreshold())) + .thenReturn(checker); + LedgerDirsManager ledgerDirsManager = mock(LedgerDirsManager.class); + whenNew(LedgerDirsManager.class).withArguments(eq(conf), eq(conf.getLedgerDirs()), eq(checker)) + .thenReturn(ledgerDirsManager); + journal = mock(Journal.class); + whenNew(Journal.class).withArguments(anyInt(), any(File.class), eq(conf), eq(ledgerDirsManager)) + .thenReturn(journal); + when(journal.getJournalDirectory()).thenReturn(conf.getJournalDirs()[0]); + } + + @Test + public void testWithJournalId() throws Exception { + conf.setJournalDirsName(new String[] { folder.getRoot().getAbsolutePath() }); + when(journal.getJournalDirectory()).thenReturn(new File("")); + testCommand("-id", "1"); + verifyNew(Journal.class, times(1)) + .withArguments(anyInt(), any(File.class), eq(conf), any(LedgerDirsManager.class)); + verifyNew(LedgerDirsManager.class, times(1)) + .withArguments(eq(conf), eq(conf.getLedgerDirs()), any(DiskChecker.class)); + verifyNew(DiskChecker.class, times(1)) + .withArguments(eq(conf.getDiskUsageThreshold()), eq(conf.getDiskUsageWarnThreshold())); + } + + @Test + public void testWithFilename() throws Exception { + conf.setJournalDirsName(new String[] { folder.getRoot().getAbsolutePath() }); + when(journal.getJournalDirectory()).thenReturn(new File("")); + File file = folder.newFile("1.txn"); + testCommand("-f", file.getAbsolutePath(), "-m"); + verifyNew(Journal.class, times(1)) + .withArguments(anyInt(), any(File.class), eq(conf), any(LedgerDirsManager.class)); + verifyNew(LedgerDirsManager.class, times(1)) + .withArguments(eq(conf), eq(conf.getLedgerDirs()), any(DiskChecker.class)); + verifyNew(DiskChecker.class, times(1)) + .withArguments(eq(conf.getDiskUsageThreshold()), eq(conf.getDiskUsageWarnThreshold())); + } + + @Test + public void testWithMsg() throws Exception { + testCommand("-id", "1", "-d", conf.getJournalDirs()[0].getAbsolutePath()); + verifyNew(Journal.class, times(3)) + .withArguments(anyInt(), any(File.class), eq(conf), any(LedgerDirsManager.class)); + verifyNew(LedgerDirsManager.class, times(3)) + .withArguments(eq(conf), eq(conf.getLedgerDirs()), any(DiskChecker.class)); + verifyNew(DiskChecker.class, times(3)) + .withArguments(eq(conf.getDiskUsageThreshold()), eq(conf.getDiskUsageWarnThreshold())); + verify(journal, times(1)).getJournalDirectory(); + } + + public void testCommand(String... args) throws Exception { + ReadJournalCommand command = new ReadJournalCommand(); + Assert.assertTrue(command.apply(bkFlags, args)); + } + + @Test + public void testWithoutArgs() { + ReadJournalCommand command = new ReadJournalCommand(); + Assert.assertFalse(command.apply(bkFlags, new String[] { "" })); + } +} From a01b766b318020138f970ce53d005cc4683aca74 Mon Sep 17 00:00:00 2001 From: Like Date: Fri, 29 Mar 2019 12:45:33 +0800 Subject: [PATCH 0248/1642] Fix IDE complain as there are multi choices for error code There are two ```Code``` in ```org.apache.bookkeeper.client. BKException```: The first is https://github.com/apache/bookkeeper/blob/f26a4cae0e9205ad391c6d4d79f2937871864c28/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java#L136 The second is https://github.com/apache/bookkeeper/blob/f26a4cae0e9205ad391c6d4d79f2937871864c28/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java#L180 BKException's subclasses cannot auto choice which ```Code```. Reviewers: Sijie Guo This closes #1999 from liketic/fix-code --- .../apache/bookkeeper/client/BKException.java | 72 +++++++++---------- 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java index b21d74504c4..781a07b4bf0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java @@ -141,7 +141,7 @@ public interface Code extends org.apache.bookkeeper.client.api.BKException.Code */ public static class BKSecurityException extends BKException { public BKSecurityException() { - super(Code.SecurityException); + super(BKException.Code.SecurityException); } } @@ -150,7 +150,7 @@ public BKSecurityException() { */ public static class BKReadException extends BKException { public BKReadException() { - super(Code.ReadException); + super(BKException.Code.ReadException); } } @@ -159,7 +159,7 @@ public BKReadException() { */ public static class BKNoSuchEntryException extends BKException { public BKNoSuchEntryException() { - super(Code.NoSuchEntryException); + super(BKException.Code.NoSuchEntryException); } } @@ -168,7 +168,7 @@ public BKNoSuchEntryException() { */ public static class BKQuorumException extends BKException { public BKQuorumException() { - super(Code.QuorumException); + super(BKException.Code.QuorumException); } } @@ -177,7 +177,7 @@ public BKQuorumException() { */ public static class BKBookieException extends BKException { public BKBookieException() { - super(Code.NoBookieAvailableException); + super(BKException.Code.NoBookieAvailableException); } } @@ -186,7 +186,7 @@ public BKBookieException() { */ public static class BKDigestNotInitializedException extends BKException { public BKDigestNotInitializedException() { - super(Code.DigestNotInitializedException); + super(BKException.Code.DigestNotInitializedException); } } @@ -195,7 +195,7 @@ public BKDigestNotInitializedException() { */ public static class BKDigestMatchException extends BKException { public BKDigestMatchException() { - super(Code.DigestMatchException); + super(BKException.Code.DigestMatchException); } } @@ -204,7 +204,7 @@ public BKDigestMatchException() { */ public static class BKIllegalOpException extends BKException { public BKIllegalOpException() { - super(Code.IllegalOpException); + super(BKException.Code.IllegalOpException); } } @@ -213,7 +213,7 @@ public BKIllegalOpException() { */ public static class BKAddEntryQuorumTimeoutException extends BKException { public BKAddEntryQuorumTimeoutException() { - super(Code.AddEntryQuorumTimeoutException); + super(BKException.Code.AddEntryQuorumTimeoutException); } } @@ -222,7 +222,7 @@ public BKAddEntryQuorumTimeoutException() { */ public static class BKDuplicateEntryIdException extends BKException { public BKDuplicateEntryIdException() { - super(Code.DuplicateEntryIdException); + super(BKException.Code.DuplicateEntryIdException); } } @@ -231,7 +231,7 @@ public BKDuplicateEntryIdException() { */ public static class BKUnexpectedConditionException extends BKException { public BKUnexpectedConditionException() { - super(Code.UnexpectedConditionException); + super(BKException.Code.UnexpectedConditionException); } } @@ -240,7 +240,7 @@ public BKUnexpectedConditionException() { */ public static class BKNotEnoughBookiesException extends BKException { public BKNotEnoughBookiesException() { - super(Code.NotEnoughBookiesException); + super(BKException.Code.NotEnoughBookiesException); } } @@ -249,7 +249,7 @@ public BKNotEnoughBookiesException() { */ public static class BKWriteException extends BKException { public BKWriteException() { - super(Code.WriteException); + super(BKException.Code.WriteException); } } @@ -258,7 +258,7 @@ public BKWriteException() { */ public static class BKProtocolVersionException extends BKException { public BKProtocolVersionException() { - super(Code.ProtocolVersionException); + super(BKException.Code.ProtocolVersionException); } } @@ -267,7 +267,7 @@ public BKProtocolVersionException() { */ public static class BKMetadataVersionException extends BKException { public BKMetadataVersionException() { - super(Code.MetadataVersionException); + super(BKException.Code.MetadataVersionException); } } @@ -276,7 +276,7 @@ public BKMetadataVersionException() { */ public static class BKNoSuchLedgerExistsException extends BKException { public BKNoSuchLedgerExistsException() { - super(Code.NoSuchLedgerExistsException); + super(BKException.Code.NoSuchLedgerExistsException); } } @@ -285,7 +285,7 @@ public BKNoSuchLedgerExistsException() { */ public static class BKBookieHandleNotAvailableException extends BKException { public BKBookieHandleNotAvailableException() { - super(Code.BookieHandleNotAvailableException); + super(BKException.Code.BookieHandleNotAvailableException); } } @@ -294,7 +294,7 @@ public BKBookieHandleNotAvailableException() { */ public static class ZKException extends BKException { public ZKException() { - super(Code.ZKException); + super(BKException.Code.ZKException); } } @@ -303,11 +303,11 @@ public ZKException() { */ public static class MetaStoreException extends BKException { public MetaStoreException() { - super(Code.MetaStoreException); + super(BKException.Code.MetaStoreException); } public MetaStoreException(Throwable cause) { - super(Code.MetaStoreException, cause); + super(BKException.Code.MetaStoreException, cause); } } @@ -316,7 +316,7 @@ public MetaStoreException(Throwable cause) { */ public static class BKLedgerExistException extends BKException { public BKLedgerExistException() { - super(Code.LedgerExistException); + super(BKException.Code.LedgerExistException); } } @@ -325,7 +325,7 @@ public BKLedgerExistException() { */ public static class BKLedgerRecoveryException extends BKException { public BKLedgerRecoveryException() { - super(Code.LedgerRecoveryException); + super(BKException.Code.LedgerRecoveryException); } } @@ -334,7 +334,7 @@ public BKLedgerRecoveryException() { */ public static class BKLedgerClosedException extends BKException { public BKLedgerClosedException() { - super(Code.LedgerClosedException); + super(BKException.Code.LedgerClosedException); } } @@ -343,7 +343,7 @@ public BKLedgerClosedException() { */ public static class BKIncorrectParameterException extends BKException { public BKIncorrectParameterException() { - super(Code.IncorrectParameterException); + super(BKException.Code.IncorrectParameterException); } } @@ -352,7 +352,7 @@ public BKIncorrectParameterException() { */ public static class BKInterruptedException extends BKException { public BKInterruptedException() { - super(Code.InterruptedException); + super(BKException.Code.InterruptedException); } } @@ -361,7 +361,7 @@ public BKInterruptedException() { */ public static class BKLedgerFencedException extends BKException { public BKLedgerFencedException() { - super(Code.LedgerFencedException); + super(BKException.Code.LedgerFencedException); } } @@ -370,7 +370,7 @@ public BKLedgerFencedException() { */ public static class BKUnauthorizedAccessException extends BKException { public BKUnauthorizedAccessException() { - super(Code.UnauthorizedAccessException); + super(BKException.Code.UnauthorizedAccessException); } } @@ -379,7 +379,7 @@ public BKUnauthorizedAccessException() { */ public static class BKUnclosedFragmentException extends BKException { public BKUnclosedFragmentException() { - super(Code.UnclosedFragmentException); + super(BKException.Code.UnclosedFragmentException); } } @@ -388,7 +388,7 @@ public BKUnclosedFragmentException() { */ public static class BKWriteOnReadOnlyBookieException extends BKException { public BKWriteOnReadOnlyBookieException() { - super(Code.WriteOnReadOnlyBookieException); + super(BKException.Code.WriteOnReadOnlyBookieException); } } @@ -397,7 +397,7 @@ public BKWriteOnReadOnlyBookieException() { */ public static class BKTooManyRequestsException extends BKException { public BKTooManyRequestsException() { - super(Code.TooManyRequestsException); + super(BKException.Code.TooManyRequestsException); } } @@ -406,7 +406,7 @@ public BKTooManyRequestsException() { */ public static class BKReplicationException extends BKException { public BKReplicationException() { - super(Code.ReplicationException); + super(BKException.Code.ReplicationException); } } @@ -415,7 +415,7 @@ public BKReplicationException() { */ public static class BKClientClosedException extends BKException { public BKClientClosedException() { - super(Code.ClientClosedException); + super(BKException.Code.ClientClosedException); } } @@ -424,7 +424,7 @@ public BKClientClosedException() { */ public static class BKTimeoutException extends BKException { public BKTimeoutException() { - super(Code.TimeoutException); + super(BKException.Code.TimeoutException); } } @@ -433,7 +433,7 @@ public BKTimeoutException() { */ public static class BKLedgerIdOverflowException extends BKException { public BKLedgerIdOverflowException() { - super(Code.LedgerIdOverflowException); + super(BKException.Code.LedgerIdOverflowException); } } @@ -442,11 +442,11 @@ public BKLedgerIdOverflowException() { */ public static class BKMetadataSerializationException extends BKException { public BKMetadataSerializationException() { - super(Code.MetadataSerializationException); + super(BKException.Code.MetadataSerializationException); } public BKMetadataSerializationException(Throwable cause) { - super(Code.MetadataSerializationException, cause); + super(BKException.Code.MetadataSerializationException, cause); } } From bd20160e3317f5b3104db7b04cf9a3bc2a5c2dc1 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Mon, 1 Apr 2019 14:19:49 +0200 Subject: [PATCH 0249/1642] Docker autobuild hook This hook will allow docker autobuild to build from the offical tag and removes the need to update the dockerfile after the release has been made. It also adds a build arg, DISTRO_URL which can be used for testing release candidates. ``` docker build --build-arg BK_VERSION=4.9.1 --build-arg DISTRO_URL= . ``` Reviewers: Enrico Olivelli , Sijie Guo This closes #2017 from ivankelly/docker-tag2 --- docker/Dockerfile | 12 +++++----- docker/hooks/build | 3 +++ site/community/release_guide.md | 39 ++++++++++++++------------------- 3 files changed, 25 insertions(+), 29 deletions(-) create mode 100755 docker/hooks/build diff --git a/docker/Dockerfile b/docker/Dockerfile index d8639d8594d..ff1396b5c8f 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -22,7 +22,7 @@ MAINTAINER Apache BookKeeper ARG BK_VERSION=4.9.0 ARG DISTRO_NAME=bookkeeper-server-${BK_VERSION}-bin -ARG GPG_KEY=FD74402C +ARG DISTRO_URL=https://archive.apache.org/dist/bookkeeper/bookkeeper-${BK_VERSION}/${DISTRO_NAME}.tar.gz ENV BOOKIE_PORT=3181 EXPOSE $BOOKIE_PORT @@ -30,18 +30,18 @@ ENV BK_USER=bookkeeper ENV BK_HOME=/opt/bookkeeper ENV JAVA_HOME=/usr/lib/jvm/jre-1.8.0 - # Download Apache Bookkeeper, untar and clean up RUN set -x \ && adduser "${BK_USER}" \ && yum install -y java-1.8.0-openjdk-headless wget bash python sudo \ && mkdir -pv /opt \ && cd /opt \ - && wget -q "https://archive.apache.org/dist/bookkeeper/bookkeeper-${BK_VERSION}/${DISTRO_NAME}.tar.gz" \ - && wget -q "https://archive.apache.org/dist/bookkeeper/bookkeeper-${BK_VERSION}/${DISTRO_NAME}.tar.gz.asc" \ - && wget -q "https://archive.apache.org/dist/bookkeeper/bookkeeper-${BK_VERSION}/${DISTRO_NAME}.tar.gz.sha512" \ + && wget -q "${DISTRO_URL}" \ + && wget -q "${DISTRO_URL}.asc" \ + && wget -q "${DISTRO_URL}.sha512" \ && sha512sum -c ${DISTRO_NAME}.tar.gz.sha512 \ - && gpg --keyserver ha.pool.sks-keyservers.net --recv-key "$GPG_KEY" \ + && wget https://dist.apache.org/repos/dist/release/bookkeeper/KEYS \ + && gpg --import KEYS \ && gpg --batch --verify "$DISTRO_NAME.tar.gz.asc" "$DISTRO_NAME.tar.gz" \ && tar -xzf "$DISTRO_NAME.tar.gz" \ && mv bookkeeper-server-${BK_VERSION}/ /opt/bookkeeper/ \ diff --git a/docker/hooks/build b/docker/hooks/build new file mode 100755 index 00000000000..3c479e2b7be --- /dev/null +++ b/docker/hooks/build @@ -0,0 +1,3 @@ +#!/bin/bash + +docker build --build-arg BK_VERSION=$DOCKER_TAG -t $IMAGE_NAME . \ No newline at end of file diff --git a/site/community/release_guide.md b/site/community/release_guide.md index c8a05a3a83f..ef2ef09ec1d 100644 --- a/site/community/release_guide.md +++ b/site/community/release_guide.md @@ -472,14 +472,25 @@ Copy the source release from the `dev` repository to the `release` repository at 2. Merge the Release Notes pull request and make sure the Release Notes is updated. -### Update Dockerfile +### Git tag -> NOTE: The dockerfile PR should only be merged after the release package is showed up under https://archive.apache.org/dist/bookkeeper/ +> NOTE: Only create the release tag after the release package is showed up under https://archive.apache.org/dist/bookkeeper/ as creating the tag triggers a docker autobuild which needs the package to exist. If you forget to do so, the build will fail. In this case you can delete the tag from github and push it again. -1. Update the `BK_VERSION` and `GPG_KEY` in `docker/Dockerfile` (e.g. [Pull Request 436](https://github.com/apache/bookkeeper/pull/436) ), - send a pull request for review and get an approval from the community. +Create and push a new signed for the released version by copying the tag for the final release tag, as follows -2. Once the pull request is approved, merge this pull request into master and make sure it is cherry-picked into corresponding branch. +```shell +git tag -s "${TAG}" "${RC_TAG}" +git push apache "${TAG}" +``` + +Remove rc tags: + +```shell +for num in $(seq 0 ${RC_NUM}); do + git tag -d "v${VERSION}-rc${num}" + git push apache :"v${VERSION}-rc${num}" +done +``` ### Update DC/OS BookKeeper package @@ -548,24 +559,6 @@ It is easy if only version need be bump. $ git commit -m "new bookkeeper version" ``` -### Git tag - -Create and push a new signed for the released version by copying the tag for the final release tag, as follows - -```shell -git tag -s "${TAG}" "${RC_TAG}" -git push apache "${TAG}" -``` - -Remove rc tags: - -```shell -for num in $(seq 0 ${RC_NUM}); do - git tag -d "v${VERSION}-rc${num}" - git push apache :"v${VERSION}-rc${num}" -done -``` - ### Verify Docker Image > After release tag is created, it will automatically trigger docker auto build. From a5d8235742148d39be529357a07cea1e9d80e25d Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 2 Apr 2019 00:10:39 +0800 Subject: [PATCH 0250/1642] Migrate command `listledgers` Descriptions of the changes in this PR: Using `bkctl` run command `listledgers` ### Motivation #2004 Reviewers: Jia Zhai , Sijie Guo This closes #2005 from zymap/command-listledgers --- .../apache/bookkeeper/bookie/BookieShell.java | 79 +------ .../commands/bookie/ListLedgersCommand.java | 195 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../bookie/ListLedgersCommandTest.java | 105 ++++++++++ 4 files changed, 307 insertions(+), 74 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommandTest.java 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 8c448a11cd7..9746356f58a 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 @@ -60,11 +60,9 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.LongStream; @@ -95,7 +93,6 @@ import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.proto.BookieProtocol; -import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.replication.ReplicationException; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -109,6 +106,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; @@ -143,8 +141,6 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.commons.lang3.ArrayUtils; -import org.apache.zookeeper.AsyncCallback; -import org.apache.zookeeper.AsyncCallback.VoidCallback; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; @@ -981,76 +977,11 @@ class ListLedgersCmd extends MyCommand { public int runCmd(CommandLine cmdLine) throws Exception { final boolean printMeta = cmdLine.hasOption("m"); final String bookieidToBePartOfEnsemble = cmdLine.getOptionValue("bookieid"); - final BookieSocketAddress bookieAddress = StringUtils.isBlank(bookieidToBePartOfEnsemble) ? null - : new BookieSocketAddress(bookieidToBePartOfEnsemble); - runFunctionWithLedgerManagerFactory(bkConf, mFactory -> { - try (LedgerManager ledgerManager = mFactory.newLedgerManager()) { - - final AtomicInteger returnCode = new AtomicInteger(BKException.Code.OK); - final CountDownLatch processDone = new CountDownLatch(1); - - Processor ledgerProcessor = new Processor() { - @Override - public void process(Long ledgerId, VoidCallback cb) { - if (!printMeta && (bookieAddress == null)) { - printLedgerMetadata(ledgerId, null, false); - cb.processResult(BKException.Code.OK, null, null); - } else { - ledgerManager.readLedgerMetadata(ledgerId).whenComplete( - (metadata, exception) -> { - if (exception == null) { - if ((bookieAddress == null) - || BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie( - ledgerId, bookieAddress, metadata.getValue())) { - /* - * the print method has to be in - * synchronized scope, otherwise - * output of printLedgerMetadata - * could interleave since this - * callback for different - * ledgers can happen in - * different threads. - */ - synchronized (BookieShell.this) { - printLedgerMetadata(ledgerId, metadata.getValue(), - printMeta); - } - } - cb.processResult(BKException.Code.OK, null, null); - } else if (BKException.getExceptionCode(exception) - == BKException.Code.NoSuchLedgerExistsException) { - cb.processResult(BKException.Code.OK, null, null); - } else { - LOG.error("Unable to read the ledger: {} information", ledgerId); - cb.processResult(BKException.getExceptionCode(exception), - null, null); - } - }); - } - } - }; - - ledgerManager.asyncProcessLedgers(ledgerProcessor, new AsyncCallback.VoidCallback() { - @Override - public void processResult(int rc, String s, Object obj) { - returnCode.set(rc); - processDone.countDown(); - } - }, null, BKException.Code.OK, BKException.Code.ReadException); - - processDone.await(); - if (returnCode.get() != BKException.Code.OK) { - LOG.error("Received error return value while processing ledgers: {}", returnCode.get()); - throw BKException.create(returnCode.get()); - } - - } catch (Exception ioe) { - LOG.error("Received Exception while processing ledgers", ioe); - throw new UncheckedExecutionException(ioe); - } - return null; - }); + ListLedgersCommand.ListLedgersFlags flags = new ListLedgersCommand.ListLedgersFlags() + .bookieId(bookieidToBePartOfEnsemble).meta(printMeta); + ListLedgersCommand cmd = new ListLedgersCommand(ledgerIdFormatter); + cmd.apply(bkConf, flags); return 0; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommand.java new file mode 100644 index 00000000000..542216f2727 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommand.java @@ -0,0 +1,195 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.net.UnknownHostException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; +import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand.ListLedgersFlags; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command for list all ledgers on the cluster. + */ +public class ListLedgersCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(ListLedgersCommand.class); + + private static final String NAME = "listledgers"; + private static final String DESC = "List all ledgers on the cluster (this may take a long time)."; + private static final String DEFAULT = ""; + + private LedgerIdFormatter ledgerIdFormatter; + + public ListLedgersCommand() { + this(new ListLedgersFlags()); + } + + public ListLedgersCommand(LedgerIdFormatter ledgerIdFormatter) { + this(new ListLedgersFlags()); + this.ledgerIdFormatter = ledgerIdFormatter; + } + + public ListLedgersCommand(ListLedgersFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for ListLedgers command. + */ + @Accessors(fluent = true) + @Setter + public static class ListLedgersFlags extends CliFlags{ + + @Parameter(names = {"-m", "--meta"}, description = "Print metadata") + private boolean meta; + + @Parameter(names = { "-id", "--bookieid" }, description = "List ledgers residing in this bookie") + private String bookieId; + + @Parameter(names = { "-l", "--ledgerIdFormatter" }, description = "Set ledger id formatter") + private String ledgerIdFormatter = DEFAULT; + } + + @Override + public boolean apply(ServerConfiguration conf, ListLedgersFlags cmdFlags) { + initLedgerFrommat(conf, cmdFlags); + try { + handler(conf, cmdFlags); + } catch (UnknownHostException e) { + System.err.println("Bookie id error"); + return false; + } catch (MetadataException | ExecutionException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + + return true; + } + + private void initLedgerFrommat(ServerConfiguration conf, ListLedgersFlags cmdFlags) { + if (ledgerIdFormatter != null) { + return; + } + if (!cmdFlags.ledgerIdFormatter.equals(DEFAULT)) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(cmdFlags.ledgerIdFormatter, conf); + } else { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } + } + + public boolean handler(ServerConfiguration conf, ListLedgersFlags flags) + throws UnknownHostException, MetadataException, ExecutionException { + + final BookieSocketAddress bookieAddress = StringUtils.isBlank(flags.bookieId) ? null : + new BookieSocketAddress(flags.bookieId); + + runFunctionWithLedgerManagerFactory(conf, mFactory -> { + try (LedgerManager ledgerManager = mFactory.newLedgerManager()) { + + final AtomicInteger returnCode = new AtomicInteger(BKException.Code.OK); + final CountDownLatch processDone = new CountDownLatch(1); + + BookkeeperInternalCallbacks.Processor ledgerProcessor = (ledgerId, cb) -> { + if (!flags.meta && (bookieAddress == null)) { + printLedgerMetadata(ledgerId, null, false); + cb.processResult(BKException.Code.OK, null, null); + } else { + ledgerManager.readLedgerMetadata(ledgerId).whenComplete((metadata, exception) -> { + if (exception == null) { + if ((bookieAddress == null) + || BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie + (ledgerId, bookieAddress, metadata.getValue())) { + /* + * the print method has to be in + * synchronized scope, otherwise + * output of printLedgerMetadata + * could interleave since this + * callback for different + * ledgers can happen in + * different threads. + */ + synchronized (ListLedgersCommand.this) { + printLedgerMetadata(ledgerId, metadata.getValue(), flags.meta); + } + } + cb.processResult(BKException.Code.OK, null, null); + } else if (BKException.getExceptionCode(exception) + == BKException.Code.NoSuchLedgerExistsException) { + cb.processResult(BKException.Code.OK, null, null); + } else { + LOG.error("Unable to read the ledger: {} information", ledgerId); + cb.processResult(BKException.getExceptionCode(exception), null, null); + } + }); + } + }; + + ledgerManager.asyncProcessLedgers(ledgerProcessor, (rc, s, obj) -> { + returnCode.set(rc); + processDone.countDown(); + }, null, BKException.Code.OK, BKException.Code.ReadException); + + processDone.await(); + if (returnCode.get() != BKException.Code.OK) { + LOG.error("Received error return value while processing ledgers: {}", returnCode.get()); + throw BKException.create(returnCode.get()); + } + + } catch (Exception ioe) { + LOG.error("Received Exception while processing ledgers", ioe); + throw new UncheckedExecutionException(ioe); + } + return null; + }); + + return true; + } + + private void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { + System.out.println("ledgerID: " + ledgerIdFormatter.formatLedgerId(ledgerId)); + if (printMeta) { + System.out.println(md.toString()); + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index c8c781764b2..496788961dd 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; @@ -54,6 +55,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new LedgerCommand()) .addCommand(new ListFilesOnDiscCommand()) .addCommand(new ConvertToDBStorageCommand()) + .addCommand(new ListLedgersCommand()) .addCommand(new ConvertToInterleavedStorageCommand()) .addCommand(new ReadJournalCommand()) .build(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommandTest.java new file mode 100644 index 00000000000..9470023f9f1 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommandTest.java @@ -0,0 +1,105 @@ +/* + * 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.tools.cli.commands.bookie; + + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.concurrent.CountDownLatch; +import java.util.function.Function; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.zookeeper.AsyncCallback; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for ListLedgers command. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ ListLedgersCommand.class, MetadataDrivers.class, BookkeeperInternalCallbacks.class, + CountDownLatch.class }) +public class ListLedgersCommandTest extends BookieCommandTestBase { + + public ListLedgersCommandTest() { + super(3, 3); + } + + @SuppressWarnings("unchecked") + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + BookieSocketAddress bookieAddress = mock(BookieSocketAddress.class); + PowerMockito.whenNew(BookieSocketAddress.class).withParameterTypes(String.class).withArguments(anyString()) + .thenReturn(bookieAddress); + + PowerMockito.mockStatic(MetadataDrivers.class); + LedgerManagerFactory mFactory = mock(LedgerManagerFactory.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function function = invocationOnMock.getArgument(1); + function.apply(mFactory); + return true; + }).when(MetadataDrivers.class, "runFunctionWithLedgerManagerFactory", any(ServerConfiguration.class), + any(Function.class)); + + CountDownLatch processDone = mock(CountDownLatch.class); + PowerMockito.whenNew(CountDownLatch.class).withArguments(anyInt()) + .thenReturn(processDone); + + LedgerManager ledgerManager = mock(LedgerManager.class); + when(mFactory.newLedgerManager()).thenReturn(ledgerManager); + + AsyncCallback.VoidCallback callback = mock(AsyncCallback.VoidCallback.class); + PowerMockito.doAnswer(invocationOnMock -> { + processDone.countDown(); + return null; + }).when(callback).processResult(anyInt(), anyString(), any()); + } + + @Test + public void testWithoutBookieId() { + testCommand(""); + } + + @Test + public void testWithBookieId() { + testCommand("-id", "1"); + } + + private void testCommand(String... args) { + ListLedgersCommand command = new ListLedgersCommand(); + Assert.assertTrue(command.apply(bkFlags, args)); + } +} From f54a0f30556b9e58ca29cdb5ba992855a100797a Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 2 Apr 2019 00:18:42 +0800 Subject: [PATCH 0251/1642] Migrate command `metadataformat` Descriptions of the changes in this PR: #2031 Reviewers: Sijie Guo This closes #2032 from zymap/command-metaformat --- .../apache/bookkeeper/bookie/BookieShell.java | 8 +- .../commands/bookies/MetaFormatCommand.java | 74 +++++++++++++++++++ .../cli/commands/BookiesCommandGroup.java | 2 + .../bookies/MetaFormatCommandTest.java | 60 +++++++++++++++ 4 files changed, 142 insertions(+), 2 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/MetaFormatCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/MetaFormatCommandTest.java 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 9746356f58a..487a5a3f656 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 @@ -111,6 +111,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.DeleteCookieCommand; @@ -318,8 +319,11 @@ int runCmd(CommandLine cmdLine) throws Exception { boolean interactive = (!cmdLine.hasOption("n")); boolean force = cmdLine.hasOption("f"); - boolean result = BookKeeperAdmin.format(bkConf, interactive, force); - return (result) ? 0 : 1; + MetaFormatCommand cmd = new MetaFormatCommand(); + MetaFormatCommand.MetaFormatFlags flags = new MetaFormatCommand.MetaFormatFlags() + .interactive(interactive).force(force); + boolean result = cmd.apply(bkConf, flags); + return result ? 0 : 1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/MetaFormatCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/MetaFormatCommand.java new file mode 100644 index 00000000000..8ec247b662d --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/MetaFormatCommand.java @@ -0,0 +1,74 @@ +/* + * 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.tools.cli.commands.bookies; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Format the bookkeeper metadata present in zookeeper. + */ +public class MetaFormatCommand extends BookieCommand { + + private static final String NAME = "metaformat"; + private static final String DESC = "Format bookkeeper metadata in zookeeper."; + + public MetaFormatCommand() { + this(new MetaFormatFlags()); + } + + private MetaFormatCommand(MetaFormatFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for command meta format. + */ + @Accessors(fluent = true) + @Setter + public static class MetaFormatFlags extends CliFlags { + + @Parameter(names = { "-n", "nonInteractive" }, description = "Whether to confirm old data exists..?") + private boolean interactive; + + @Parameter(names = {"-f", "--force"}, + description = "If [nonInteractive] is specified, then whether to force delete the old data without prompt.") + private boolean force; + } + + @Override + public boolean apply(ServerConfiguration conf, MetaFormatFlags flags) { + try { + return BookKeeperAdmin.format(conf, flags.interactive, flags.force); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java index b299b5c40c1..a9e6fbbfdc1 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java @@ -23,6 +23,7 @@ import org.apache.bookkeeper.tools.cli.BKCtl; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -42,6 +43,7 @@ public class BookiesCommandGroup extends CliCommandGroup { .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new ListBookiesCommand()) .addCommand(new InfoCommand()) + .addCommand(new MetaFormatCommand()) .build(); public BookiesCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/MetaFormatCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/MetaFormatCommandTest.java new file mode 100644 index 00000000000..f8075d6a361 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/MetaFormatCommandTest.java @@ -0,0 +1,60 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; + +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link MetaFormatCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ MetaFormatCommand.class, BookKeeperAdmin.class }) +public class MetaFormatCommandTest extends BookieCommandTestBase { + + public MetaFormatCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.mockStatic(BookKeeperAdmin.class); + PowerMockito.when(BookKeeperAdmin.class, "format", any(ServerConfiguration.class), anyBoolean(), anyBoolean()) + .thenReturn(true); + + } + + @Test + public void testCommand() { + MetaFormatCommand cmd = new MetaFormatCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "" })); + } +} From e7aabe54ec60fb349ca8607019b01ff929c3fb3e Mon Sep 17 00:00:00 2001 From: Arvin Date: Tue, 2 Apr 2019 00:27:42 +0800 Subject: [PATCH 0252/1642] ISSUE #2020: close db properly to avoid open RocksDB failure at the second time Descriptions of the changes in this PR: ### Motivation If not releasing resources of failed/closed asyncStore, new creating of the same store identifier will fail, mainly caused by RocksDBException, like #2020 shows. ### Changes add scStores to factory's instance variable at the `addstore` method of `MVCCStoreFactoryImpl` class; release store when open fail; Descriptions of the changes in this PR: Master Issue: #2020 Reviewers: Jia Zhai , Sijie Guo This closes #2022 from ArvinDevel/issue2020, closes #2020 --- .../impl/sc/StorageContainerRegistryImpl.java | 5 +++++ .../storage/impl/store/MVCCStoreFactoryImpl.java | 16 +++++++++++++++- 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerRegistryImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerRegistryImpl.java index 160d9a86470..4208b1a0644 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerRegistryImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerRegistryImpl.java @@ -101,6 +101,11 @@ private CompletableFuture unsafeStartStorageContainer(long scI } else { log.warn("Fail to de-register StorageContainer ('{}') when failed to start", scId, cause); } + log.info("Release resources hold by StorageContainer ('{}') during de-register", scId); + newStorageContainer.stop().exceptionally(throwable -> { + log.error("Stop StorageContainer ('{}') fail during de-register", scId); + return null; + }); } else { log.info("Successfully started registered StorageContainer ('{}').", scId); } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImpl.java index 6f01d4b599b..e3d0a553692 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImpl.java @@ -130,6 +130,7 @@ private synchronized void addStore(long scId, long streamId, long rangeId, Map> scStores = stores.get(scId); if (null == scStores) { scStores = Maps.newHashMap(); + stores.putIfAbsent(scId, scStores); } RangeId rid = RangeId.of(streamId, rangeId); MVCCAsyncStore oldStore = scStores.get(rid); @@ -207,7 +208,18 @@ CompletableFuture> newStore(long scId, long strea .isReadonly(serveReadOnlyTable) .build(); - return store.init(spec).thenApply(ignored -> { + return store.init(spec).whenComplete((ignored, throwable) -> { + // since the store has not been added, so can't release its resources during close sc + if (null != throwable) { + log.info("Clearing resources hold by stream({})/range({}) at storage container ({}) ", + streamId, rangeId, scId); + store.closeAsync().whenComplete((i, t) -> { + if (null != t) { + log.error("Clear resources hold by {} fail", store.name()); + } + }); + } + }).thenApply(ignored -> { log.info("Successfully initialize stream({})/range({}) at storage container ({})", streamId, rangeId, scId); addStore(scId, streamId, rangeId, store); @@ -222,11 +234,13 @@ public CompletableFuture closeStores(long scId) { scStores = stores.remove(scId); } if (null == scStores) { + log.info("scStores for {} on store factory is null, return directly", scId); return FutureUtils.Void(); } List> closeFutures = Lists.newArrayList(); for (MVCCAsyncStore store : scStores.values()) { + log.info("Closing {} of sc {}", store.name(), scId); closeFutures.add(store.closeAsync()); } From c0b1cb5949e1c23501567270e54f25851c685a62 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 2 Apr 2019 16:32:23 +0800 Subject: [PATCH 0253/1642] Migrate command `deleteledger` Descriptions of the changes in this PR: #2025 Reviewers: Sijie Guo This closes #2026 from zymap/command-deleteledger --- .../apache/bookkeeper/bookie/BookieShell.java | 28 +--- .../commands/client/DeleteLedgerCommand.java | 130 ++++++++++++++++++ .../cli/commands/LedgerCommandGroup.java | 2 + .../client/DeleteLedgerCommandTest.java | 97 +++++++++++++ 4 files changed, 234 insertions(+), 23 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommandTest.java 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 487a5a3f656..8ec97c74780 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 @@ -112,6 +112,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; +import org.apache.bookkeeper.tools.cli.commands.client.DeleteLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.DeleteCookieCommand; @@ -2199,31 +2200,12 @@ class DeleteLedgerCmd extends MyCommand { @Override public int runCmd(CommandLine cmdLine) throws Exception { final long lid = getOptionLedgerIdValue(cmdLine, "ledgerid", -1); - if (lid == -1) { - System.err.println("Must specify a ledger id"); - return -1; - } boolean force = cmdLine.hasOption("f"); - boolean confirm = false; - if (!force) { - confirm = IOUtils.confirmPrompt( - "Are you sure to delete Ledger : " + ledgerIdFormatter.formatLedgerId(lid) + "?"); - } - - BookKeeper bk = null; - try { - if (force || confirm) { - ClientConfiguration conf = new ClientConfiguration(); - conf.addConfiguration(bkConf); - bk = new BookKeeper(conf); - bk.deleteLedger(lid); - } - } finally { - if (bk != null) { - bk.close(); - } - } + DeleteLedgerCommand cmd = new DeleteLedgerCommand(ledgerIdFormatter); + DeleteLedgerCommand.DeleteLedgerFlags flags = new DeleteLedgerCommand.DeleteLedgerFlags() + .ledgerId(lid).force(force); + cmd.apply(bkConf, flags); return 0; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java new file mode 100644 index 00000000000..70ffee6b6ff --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java @@ -0,0 +1,130 @@ +/* + * 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.tools.cli.commands.client; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.util.LedgerIdFormatter; + +/** + * Command to delete a given ledger. + */ +public class DeleteLedgerCommand extends BookieCommand { + + private static final String NAME = "deleteledger"; + private static final String DESC = "Delete a ledger."; + private static final String DEFAULT = ""; + + private LedgerIdFormatter ledgerIdFormatter; + + public DeleteLedgerCommand() { + this(new DeleteLedgerFlags()); + } + + public DeleteLedgerCommand(LedgerIdFormatter ledgerIdFormatter) { + this(new DeleteLedgerFlags()); + this.ledgerIdFormatter = ledgerIdFormatter; + } + + private DeleteLedgerCommand(DeleteLedgerFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for delete ledger command. + */ + @Accessors(fluent = true) + @Setter + public static class DeleteLedgerFlags extends CliFlags { + + @Parameter(names = { "-l", "--ledgerid" }, description = "Ledger ID", required = true) + private long ledgerId; + + @Parameter(names = { "-f", "--force" }, description = "Whether to force delete the Ledger without prompt..?") + private boolean force; + + @Parameter(names = { "-lf", "--ledgeridformatter" }, description = "Set ledger id formatter") + private String ledgerIdFormatter = DEFAULT; + + } + + @Override + public boolean apply(ServerConfiguration conf, DeleteLedgerFlags cmdFlags) { + initLedgerIdFormatter(conf, cmdFlags); + try { + return deleteLedger(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private void initLedgerIdFormatter(ServerConfiguration conf, DeleteLedgerFlags flags) { + if (null == ledgerIdFormatter && !flags.ledgerIdFormatter.equals(DEFAULT)) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(flags.ledgerIdFormatter, conf); + } else if (null == ledgerIdFormatter && flags.ledgerIdFormatter.equals(DEFAULT)) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } + } + + private boolean deleteLedger(ServerConfiguration conf, DeleteLedgerFlags flags) + throws IOException, BKException, InterruptedException { + + if (flags.ledgerId < 0) { + System.err.println("Ledger id error."); + return false; + } + + boolean confirm = false; + if (!flags.force) { + confirm = IOUtils.confirmPrompt( + "Are your sure to delete Ledger : " + ledgerIdFormatter.formatLedgerId(flags.ledgerId) + "?"); + } + + BookKeeper bookKeeper = null; + try { + if (flags.force || confirm) { + ClientConfiguration configuration = new ClientConfiguration(); + configuration.addConfiguration(conf); + bookKeeper = new BookKeeper(configuration); + bookKeeper.deleteLedger(flags.ledgerId); + } + } finally { + if (bookKeeper != null) { + bookKeeper.close(); + } + } + + return true; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java index c5286565724..86734bc58dd 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java @@ -21,6 +21,7 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_LEDGER_SERVICE; import org.apache.bookkeeper.tools.cli.BKCtl; +import org.apache.bookkeeper.tools.cli.commands.client.DeleteLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -40,6 +41,7 @@ public class LedgerCommandGroup extends CliCommandGroup { .withParent(BKCtl.NAME) .withCategory(CATEGORY_LEDGER_SERVICE) .addCommand(new SimpleTestCommand()) + .addCommand(new DeleteLedgerCommand()) .build(); public LedgerCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommandTest.java new file mode 100644 index 00000000000..b95725a14a2 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommandTest.java @@ -0,0 +1,97 @@ +/* + * 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.tools.cli.commands.client; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.util.IOUtils; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link DeleteLedgerCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ DeleteLedgerCommand.class, BookKeeper.class, IOUtils.class, ClientConfiguration.class }) +public class DeleteLedgerCommandTest extends BookieCommandTestBase { + + private BookKeeper bookKeeper; + private ClientConfiguration clientConf; + + public DeleteLedgerCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + + this.clientConf = mock(ClientConfiguration.class); + PowerMockito.whenNew(ClientConfiguration.class).withNoArguments().thenReturn(clientConf); + PowerMockito.doNothing().when(clientConf).addConfiguration(eq(conf)); + + this.bookKeeper = mock(BookKeeper.class); + PowerMockito.whenNew(BookKeeper.class).withParameterTypes(ClientConfiguration.class) + .withArguments(eq(clientConf)).thenReturn(bookKeeper); + PowerMockito.doNothing().when(bookKeeper).deleteLedger(anyLong()); + PowerMockito.doNothing().when(bookKeeper).close(); + + PowerMockito.mockStatic(IOUtils.class); + + } + + @Test + public void testCommandWithoutForce() throws Exception { + PowerMockito.when(IOUtils.class, "confirmPrompt", anyString()).thenReturn(false); + + DeleteLedgerCommand cmd = new DeleteLedgerCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-l", "1" })); + + PowerMockito.verifyNew(ClientConfiguration.class, never()).withNoArguments(); + verify(clientConf, never()).addConfiguration(conf); + PowerMockito.verifyNew(BookKeeper.class, never()).withArguments(eq(clientConf)); + verify(bookKeeper, never()).deleteLedger(1); + } + + @Test + public void testCommandWithForce() throws Exception { + DeleteLedgerCommand cmd = new DeleteLedgerCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-l", "1", "-f" })); + + PowerMockito.verifyNew(ClientConfiguration.class, times(1)).withNoArguments(); + verify(clientConf, times(1)).addConfiguration(any(ServerConfiguration.class)); + PowerMockito.verifyNew(BookKeeper.class, times(1)).withArguments(eq(clientConf)); + verify(bookKeeper, times(1)).deleteLedger(1); + } + +} From 80b28a3c7383d826debcd7c1cfca6005cd6c45ec Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 2 Apr 2019 16:39:03 +0800 Subject: [PATCH 0254/1642] Migrate command `lostbookierecoverydelay` Descriptions of the changes in this PR: - Using `bkctl` run command `lostbookierecoverydelay` ### Motivation #2015 Reviewers: Sijie Guo This closes #2016 from zymap/command-lostbookierecoverydelay --- .../apache/bookkeeper/bookie/BookieShell.java | 33 ++--- .../LostBookieRecoveryDelayCommand.java | 119 ++++++++++++++++++ .../commands/autorecovery/package-info.java | 23 ++++ .../commands/AutoRecoveryCommandGroup.java | 47 +++++++ tools/ledger/src/main/resources/commands | 1 + .../LostBookieRecoveryDelayCommandTets.java | 101 +++++++++++++++ 6 files changed, 301 insertions(+), 23 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommand.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java create mode 100644 tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommandTets.java 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 8ec97c74780..a656f38da29 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 @@ -98,6 +98,7 @@ import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; @@ -1679,30 +1680,16 @@ String getUsage() { int runCmd(CommandLine cmdLine) throws Exception { boolean getter = cmdLine.hasOption("g"); boolean setter = cmdLine.hasOption("s"); - - if ((!getter && !setter) || (getter && setter)) { - LOG.error("One and only one of -get and -set must be specified"); - printUsage(); - return 1; + int set = 0; + if (setter) { + set = Integer.parseInt(cmdLine.getOptionValue("set")); } - ClientConfiguration adminConf = new ClientConfiguration(bkConf); - BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); - try { - if (getter) { - int lostBookieRecoveryDelay = admin.getLostBookieRecoveryDelay(); - LOG.info("LostBookieRecoveryDelay value in ZK: {}", String.valueOf(lostBookieRecoveryDelay)); - } else { - int lostBookieRecoveryDelay = Integer.parseInt(cmdLine.getOptionValue("set")); - admin.setLostBookieRecoveryDelay(lostBookieRecoveryDelay); - LOG.info("Successfully set LostBookieRecoveryDelay value in ZK: {}", - String.valueOf(lostBookieRecoveryDelay)); - } - } finally { - if (admin != null) { - admin.close(); - } - } - return 0; + + LostBookieRecoveryDelayCommand.LBRDFlags flags = new LostBookieRecoveryDelayCommand.LBRDFlags() + .get(getter).set(set); + LostBookieRecoveryDelayCommand cmd = new LostBookieRecoveryDelayCommand(); + boolean result = cmd.apply(bkConf, flags); + return result ? 0 : 1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommand.java new file mode 100644 index 00000000000..12a4564a790 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommand.java @@ -0,0 +1,119 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.replication.ReplicationException; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to Setter and Getter for LostBookieRecoveryDelay value (in seconds) in metadata store. + */ +public class LostBookieRecoveryDelayCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(LostBookieRecoveryDelayCommand.class); + + private static final String NAME = "lostbookierecoverydelay"; + private static final String DESC = + "Setter and Getter for LostBookieRecoveryDelay value (in seconds) in metadata store"; + + private static final int DEFAULT = 0; + + public LostBookieRecoveryDelayCommand() { + this(new LBRDFlags()); + } + + private LostBookieRecoveryDelayCommand(LBRDFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for command LostBookieRecoveryDelay. + */ + @Accessors(fluent = true) + @Setter + public static class LBRDFlags extends CliFlags{ + + @Parameter(names = { "-g", "--get" }, description = "Get LostBookieRecoveryDelay value (in seconds)") + private boolean get; + + @Parameter(names = { "-s", "--set" }, description = "Set LostBookieRecoveryDelay value (in seconds)") + private int set = DEFAULT; + + } + + @Override + public boolean apply(ServerConfiguration conf, LBRDFlags cmdFlags) { + try { + return handler(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + public boolean handler(ServerConfiguration conf, LBRDFlags flags) + throws InterruptedException, BKException, IOException, ReplicationException.UnavailableException, + ReplicationException.CompatibilityException, KeeperException { + boolean getter = flags.get; + boolean setter = false; + if (flags.set != DEFAULT) { + setter = true; + } + + if ((!getter && !setter) || (getter && setter)) { + LOG.error("One and only one of -get and -set must be specified"); + return false; + } + ClientConfiguration adminConf = new ClientConfiguration(conf); + BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); + try { + if (getter) { + int lostBookieRecoveryDelay = admin.getLostBookieRecoveryDelay(); + LOG.info("LostBookieRecoveryDelay value in ZK: {}", String.valueOf(lostBookieRecoveryDelay)); + } else { + int lostBookieRecoveryDelay = flags.set; + admin.setLostBookieRecoveryDelay(lostBookieRecoveryDelay); + LOG.info("Successfully set LostBookieRecoveryDelay value in ZK: {}", + String.valueOf(lostBookieRecoveryDelay)); + } + } finally { + if (admin != null) { + admin.close(); + } + } + return true; + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java new file mode 100644 index 00000000000..fa1bbbff0a1 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/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. + */ + +/** + * This package provides all toggle commands. + */ +package org.apache.bookkeeper.tools.cli.commands.autorecovery; \ No newline at end of file diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java new file mode 100644 index 00000000000..d59412b3aa7 --- /dev/null +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java @@ -0,0 +1,47 @@ +/* + * 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.tools.cli.commands; + +import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; + +import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliCommandGroup; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Commands on some specific operation. + */ +public class AutoRecoveryCommandGroup extends CliCommandGroup { + + private static final String NAME = "autorecovery"; + private static final String DESC = "Command on some specific operation."; + + private static final CliSpec spec = CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withCategory(CATEGORY_INFRA_SERVICE) + .addCommand(new LostBookieRecoveryDelayCommand()) + .build(); + + public AutoRecoveryCommandGroup() { + super(spec); + } +} + diff --git a/tools/ledger/src/main/resources/commands b/tools/ledger/src/main/resources/commands index c9af7a127cf..c4d6412c55f 100644 --- a/tools/ledger/src/main/resources/commands +++ b/tools/ledger/src/main/resources/commands @@ -21,3 +21,4 @@ org.apache.bookkeeper.tools.cli.commands.BookieIdCommandGroup org.apache.bookkeeper.tools.cli.commands.BookiesCommandGroup org.apache.bookkeeper.tools.cli.commands.CookieCommandGroup org.apache.bookkeeper.tools.cli.commands.LedgerCommandGroup +org.apache.bookkeeper.tools.cli.commands.AutoRecoveryCommandGroup \ No newline at end of file diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommandTets.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommandTets.java new file mode 100644 index 00000000000..861c523b4ed --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommandTets.java @@ -0,0 +1,101 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.doNothing; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; + +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link LostBookieRecoveryDelayCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ LostBookieRecoveryDelayCommand.class, ClientConfiguration.class, BookKeeperAdmin.class }) +public class LostBookieRecoveryDelayCommandTets extends BookieCommandTestBase { + + private ClientConfiguration clientConfiguration; + private BookKeeperAdmin admin; + + public LostBookieRecoveryDelayCommandTets() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + clientConfiguration = mock(ClientConfiguration.class); + PowerMockito.whenNew(ClientConfiguration.class).withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(conf)).thenReturn(clientConfiguration); + + admin = mock(BookKeeperAdmin.class); + PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class) + .withArguments(eq(clientConfiguration)).thenReturn(admin); + + when(admin.getLostBookieRecoveryDelay()).thenReturn(1); + doNothing().when(admin).setLostBookieRecoveryDelay(anyInt()); + } + + @Test + public void testWithoutArgs() { + LostBookieRecoveryDelayCommand cmd = new LostBookieRecoveryDelayCommand(); + Assert.assertFalse(cmd.apply(bkFlags, new String[] { "" })); + } + + @Test + public void testWithSet() throws Exception { + testCommand("-s", "1"); + verifyNew(ClientConfiguration.class, times(1)).withArguments(conf); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(clientConfiguration); + verify(admin, times(1)).setLostBookieRecoveryDelay(1); + } + + @Test + public void testWithGet() throws Exception { + testCommand("-g"); + verifyNew(ClientConfiguration.class, times(1)).withArguments(conf); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(clientConfiguration); + verify(admin, times(1)).getLostBookieRecoveryDelay(); + } + + + private void testCommand(String... args) { + LostBookieRecoveryDelayCommand cmd = new LostBookieRecoveryDelayCommand(); + Assert.assertTrue(cmd.apply(bkFlags, args)); + } +} From d2c9289fc4c4e9b62678bfd878f3c8d3ea43778f Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 3 Apr 2019 02:21:14 +0800 Subject: [PATCH 0255/1642] Migrate command `readlogmetadata` Descriptions of the changes in this PR: #2038 Reviewers: Sijie Guo This closes #2039 from zymap/command-readlogmetadata --- .../apache/bookkeeper/bookie/BookieShell.java | 20 +-- .../bookkeeper/bookie/EntryLogMetadata.java | 2 +- .../bookie/ReadLogMetadataCommand.java | 145 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../bookie/ReadLogMetadataCommandTest.java | 88 +++++++++++ 5 files changed, 242 insertions(+), 15 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommandTest.java 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 a656f38da29..72546e4fe7c 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 @@ -109,6 +109,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; @@ -1306,6 +1307,8 @@ class ReadLogMetadataCmd extends MyCommand { @Override public int runCmd(CommandLine cmdLine) throws Exception { + ReadLogMetadataCommand cmd = new ReadLogMetadataCommand(ledgerIdFormatter); + ReadLogMetadataCommand.ReadLogMetadataFlags flags = new ReadLogMetadataCommand.ReadLogMetadataFlags(); String[] leftArgs = cmdLine.getArgs(); if (leftArgs.length <= 0) { LOG.error("ERROR: missing entry log id or entry log file name"); @@ -1316,22 +1319,11 @@ public int runCmd(CommandLine cmdLine) throws Exception { long logId; try { logId = Long.parseLong(leftArgs[0]); + flags.logId(logId); } catch (NumberFormatException nfe) { - // not a entry log id - File f = new File(leftArgs[0]); - String name = f.getName(); - if (!name.endsWith(".log")) { - // not a log file - LOG.error("ERROR: invalid entry log file name " + leftArgs[0]); - printUsage(); - return -1; - } - String idString = name.split("\\.")[0]; - logId = Long.parseLong(idString, 16); + flags.logFilename(leftArgs[0]); } - - printEntryLogMetadata(logId); - + cmd.apply(bkConf, flags); return 0; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogMetadata.java index ad6d87d24c2..17cf58fb52c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogMetadata.java @@ -74,7 +74,7 @@ public long getRemainingSize() { return remainingSize; } - ConcurrentLongLongHashMap getLedgersMap() { + public ConcurrentLongLongHashMap getLedgersMap() { return ledgersMap; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommand.java new file mode 100644 index 00000000000..4b73742e538 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommand.java @@ -0,0 +1,145 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.File; +import java.io.IOException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.EntryLogger; +import org.apache.bookkeeper.bookie.ReadOnlyEntryLogger; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand.ReadLogMetadataFlags; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to print metadata of entry log. + */ +public class ReadLogMetadataCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(ReadLogMetadataFlags.class); + + private static final String NAME = "readlogmetadata"; + private static final String DESC = "Prints entrylog's metadata"; + + private static final long DEFAULT_LOGID = -1L; + private static final String DEFAULT_FILENAME = ""; + private static final String DEFAULT = ""; + + private LedgerIdFormatter ledgerIdFormatter; + + EntryLogger entryLogger = null; + + public ReadLogMetadataCommand() { + this(new ReadLogMetadataFlags()); + } + + public ReadLogMetadataCommand(LedgerIdFormatter ledgerIdFormatter) { + this(new ReadLogMetadataFlags()); + this.ledgerIdFormatter = ledgerIdFormatter; + } + + private ReadLogMetadataCommand(ReadLogMetadataFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for read log metadata command. + */ + @Accessors(fluent = true) + @Setter + public static class ReadLogMetadataFlags extends CliFlags { + + @Parameter(names = { "-l", "--logid" }, description = "Entry log id") + private long logId; + + @Parameter(names = { "-f", "--filename" }, description = "Entry log filename") + private String logFilename; + + @Parameter(names = { "-lf", "--ledgeridformatter" }, description = "Set ledger id formatter") + private String ledgerIdFormatter = DEFAULT; + + } + + @Override + public boolean apply(ServerConfiguration conf, ReadLogMetadataFlags cmdFlags) { + if (!cmdFlags.ledgerIdFormatter.equals(DEFAULT) && ledgerIdFormatter == null) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(cmdFlags.ledgerIdFormatter, conf); + } else if (ledgerIdFormatter == null) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } + if (cmdFlags.logId == DEFAULT_LOGID && cmdFlags.logFilename.equals(DEFAULT_FILENAME)) { + System.err.println("Missing entry log id or entry log file name"); + return false; + } + try { + return readLogMetadata(conf, cmdFlags); + } catch (IOException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + public boolean readLogMetadata(ServerConfiguration conf, ReadLogMetadataFlags flags) throws IOException { + long logid = DEFAULT_LOGID; + if (flags.logId != DEFAULT_LOGID) { + logid = flags.logId; + } else if (!flags.logFilename.equals(DEFAULT_FILENAME)) { + File f = new File(flags.logFilename); + String name = f.getName(); + if (!name.endsWith(".log")) { + LOG.error("ERROR: invalid entry log file name " + flags.logFilename); + return false; + } + String idString = name.split("\\.")[0]; + logid = Long.parseLong(idString, 16); + } + + printEntryLogMetadata(conf, logid); + return true; + } + + private void printEntryLogMetadata(ServerConfiguration conf, long logId) throws IOException { + LOG.info("Print entryLogMetadata of entrylog {} ({}.log)", logId, Long.toHexString(logId)); + initEntryLogger(conf); + EntryLogMetadata entryLogMetadata = entryLogger.getEntryLogMetadata(logId); + entryLogMetadata.getLedgersMap().forEach((ledgerId, size) -> { + LOG.info("--------- Lid={}, TotalSizeOfEntriesOfLedger={} ---------", + ledgerIdFormatter.formatLedgerId(ledgerId), size); + }); + } + + private synchronized void initEntryLogger(ServerConfiguration conf) throws IOException { + // provide read only entry logger + if (null == entryLogger) { + entryLogger = new ReadOnlyEntryLogger(conf); + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 496788961dd..98480ef0436 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -30,6 +30,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -58,6 +59,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new ListLedgersCommand()) .addCommand(new ConvertToInterleavedStorageCommand()) .addCommand(new ReadJournalCommand()) + .addCommand(new ReadLogMetadataCommand()) .build(); public BookieCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommandTest.java new file mode 100644 index 00000000000..983c9b132fa --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommandTest.java @@ -0,0 +1,88 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; + +import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.ReadOnlyEntryLogger; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link ReadLogMetadataCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ ReadLogMetadataCommand.class }) +public class ReadLogMetadataCommandTest extends BookieCommandTestBase { + + @Mock + private ReadOnlyEntryLogger entryLogger; + + @Mock + private EntryLogMetadata entryLogMetadata; + + public ReadLogMetadataCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + PowerMockito.whenNew(ReadOnlyEntryLogger.class).withArguments(eq(conf)).thenReturn(entryLogger); + when(entryLogger.getEntryLogMetadata(anyLong())).thenReturn(entryLogMetadata); + + ConcurrentLongLongHashMap map = new ConcurrentLongLongHashMap(); + map.put(1, 1); + when(entryLogMetadata.getLedgersMap()).thenReturn(map); + + } + + @Test + public void testWithoutFlags() { + ReadLogMetadataCommand cmd = new ReadLogMetadataCommand(); + Assert.assertFalse(cmd.apply(bkFlags, new String[] {"-l", "-1", "-f", ""})); + } + + @Test + public void commandTest() throws Exception { + ReadLogMetadataCommand cmd = new ReadLogMetadataCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-l", "1" })); + verifyNew(ReadOnlyEntryLogger.class, times(1)).withArguments(eq(conf)); + verify(entryLogger, times(1)).getEntryLogMetadata(anyLong()); + verify(entryLogMetadata, times(1)).getLedgersMap(); + } +} + From c4293e33a5aa196c7f1d18ae9ac23d40f40e4c1c Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 3 Apr 2019 02:33:16 +0800 Subject: [PATCH 0256/1642] Migrate command `initnewcluster` Descriptions of the changes in this PR: #2046 Reviewers: Sijie Guo This closes #2047 from zymap/command-initnewcluster --- .../apache/bookkeeper/bookie/BookieShell.java | 4 +- .../cli/commands/bookies/InitCommand.java | 61 +++++++++++++++++++ .../cli/commands/BookiesCommandGroup.java | 2 + .../cli/commands/bookies/InitCommandTest.java | 58 ++++++++++++++++++ 4 files changed, 124 insertions(+), 1 deletion(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InitCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InitCommandTest.java 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 72546e4fe7c..5ad8d4feba8 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 @@ -365,7 +365,9 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { - boolean result = BookKeeperAdmin.initNewCluster(bkConf); + org.apache.bookkeeper.tools.cli.commands.bookies.InitCommand initCommand = + new org.apache.bookkeeper.tools.cli.commands.bookies.InitCommand(); + boolean result = initCommand.apply(bkConf, new CliFlags()); return (result) ? 0 : 1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InitCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InitCommand.java new file mode 100644 index 00000000000..8846f548387 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InitCommand.java @@ -0,0 +1,61 @@ +/* + * 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.tools.cli.commands.bookies; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Intializes new cluster by creating required znodes for the cluster. If + * ledgersrootpath is already existing then it will error out. If for any + * reason it errors out while creating znodes for the cluster, then before + * running initnewcluster again, try nuking existing cluster by running + * nukeexistingcluster. This is required because ledgersrootpath znode would + * be created after verifying that it doesn't exist, hence during next retry + * of initnewcluster it would complain saying that ledgersrootpath is + * already existing. + */ +public class InitCommand extends BookieCommand { + + private static final String NAME = "init"; + private static final String DESC = + "Initializes a new bookkeeper cluster. If initnewcluster fails then try nuking " + + "existing cluster by running nukeexistingcluster before running initnewcluster again"; + + public InitCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new CliFlags()) + .build()); + } + + @Override + public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { + try { + return BookKeeperAdmin.initNewCluster(conf); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java index a9e6fbbfdc1..e211b133f6f 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java @@ -22,6 +22,7 @@ import org.apache.bookkeeper.tools.cli.BKCtl; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; import org.apache.bookkeeper.tools.common.BKFlags; @@ -44,6 +45,7 @@ public class BookiesCommandGroup extends CliCommandGroup { .addCommand(new ListBookiesCommand()) .addCommand(new InfoCommand()) .addCommand(new MetaFormatCommand()) + .addCommand(new InitCommand()) .build(); public BookiesCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InitCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InitCommandTest.java new file mode 100644 index 00000000000..7c0f9baeb13 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InitCommandTest.java @@ -0,0 +1,58 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.mockito.ArgumentMatchers.eq; + +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link InitCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ InitCommand.class, BookKeeperAdmin.class }) +public class InitCommandTest extends BookieCommandTestBase { + + public InitCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.mockStatic(BookKeeperAdmin.class); + PowerMockito.when(BookKeeperAdmin.initNewCluster(eq(conf))).thenReturn(true); + } + + @Test + public void testCommand() { + InitCommand initCommand = new InitCommand(); + Assert.assertTrue(initCommand.apply(bkFlags, new String[] {})); + } +} From 49c0596c4ae9d85031bbebf3e11896eb612b6b94 Mon Sep 17 00:00:00 2001 From: bd2019us <47727869+bd2019us@users.noreply.github.com> Date: Tue, 2 Apr 2019 13:35:23 -0500 Subject: [PATCH 0257/1642] ISSUE #2023: change cached thread pool to fixed thread pool Descriptions of the changes in this PR: change newCachedThreadPool() to newFixedThreadPool(int) ### Motivation newFixedThreadPool(int) can be freely configured with the total number of threads, while cached thread pool may cause OutOfMemoryError when there are too many threads need to created. Reviewers: Enrico Olivelli , Sijie Guo This closes #2024 from bd2019us/patch, closes #2023 --- .../java/org/apache/bookkeeper/tools/perf/table/PerfClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/PerfClient.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/PerfClient.java index 4b3442700cf..835f816c655 100644 --- a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/PerfClient.java +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/table/PerfClient.java @@ -324,7 +324,7 @@ private void runBenchmarkTasks() throws Exception { final CountDownLatch latch = new CountDownLatch(tasks.size()); @Cleanup("shutdown") - ExecutorService executor = Executors.newCachedThreadPool(); + ExecutorService executor = Executors.newFixedThreadPool(tasks.size()); for (BenchmarkTask task : tasks) { executor.submit(() -> { try { From 7b29955a7689a1a19719a99c386bcf6556be48c8 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 3 Apr 2019 14:15:23 +0800 Subject: [PATCH 0258/1642] Migrate command `listunderreplicated` Descriptions of the changes in this PR: - Using `bkctl` run command `listunderreplicated` ### Motivation #2013 Reviewers: Jia Zhai , Sijie Guo This closes #2014 from zymap/command-listunderreplicated --- .../apache/bookkeeper/bookie/BookieShell.java | 64 +------ .../ListUnderReplicatedCommand.java | 176 ++++++++++++++++++ .../commands/autorecovery/package-info.java | 2 +- .../commands/AutoRecoveryCommandGroup.java | 5 +- .../ListUnderReplicatedCommandTest.java | 160 ++++++++++++++++ 5 files changed, 348 insertions(+), 59 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ListUnderReplicatedCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ListUnderReplicatedCommandTest.java 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 5ad8d4feba8..6f8f1c30c73 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 @@ -63,7 +63,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.LongStream; @@ -87,7 +86,6 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; -import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieClient; @@ -95,9 +93,8 @@ import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.replication.ReplicationException; -import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; -import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; @@ -912,58 +909,13 @@ int runCmd(CommandLine cmdLine) throws Exception { final boolean printMissingReplica = cmdLine.hasOption("printmissingreplica"); final boolean printReplicationWorkerId = cmdLine.hasOption("printreplicationworkerid"); - final Predicate> predicate; - if (!StringUtils.isBlank(includingBookieId) && !StringUtils.isBlank(excludingBookieId)) { - predicate = replicasList -> (replicasList.contains(includingBookieId) - && !replicasList.contains(excludingBookieId)); - } else if (!StringUtils.isBlank(includingBookieId)) { - predicate = replicasList -> replicasList.contains(includingBookieId); - } else if (!StringUtils.isBlank(excludingBookieId)) { - predicate = replicasList -> !replicasList.contains(excludingBookieId); - } else { - predicate = null; - } - - runFunctionWithLedgerManagerFactory(bkConf, mFactory -> { - LedgerUnderreplicationManager underreplicationManager; - try { - underreplicationManager = mFactory.newLedgerUnderreplicationManager(); - } catch (KeeperException | CompatibilityException e) { - throw new UncheckedExecutionException("Failed to new ledger underreplicated manager", e); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new UncheckedExecutionException("Interrupted on newing ledger underreplicated manager", e); - } - Iterator iter = underreplicationManager.listLedgersToRereplicate(predicate); - while (iter.hasNext()) { - UnderreplicatedLedger underreplicatedLedger = iter.next(); - long urLedgerId = underreplicatedLedger.getLedgerId(); - System.out.println(ledgerIdFormatter.formatLedgerId(urLedgerId)); - long ctime = underreplicatedLedger.getCtime(); - if (ctime != UnderreplicatedLedger.UNASSIGNED_CTIME) { - System.out.println("\tCtime : " + ctime); - } - if (printMissingReplica) { - underreplicatedLedger.getReplicaList().forEach((missingReplica) -> { - System.out.println("\tMissingReplica : " + missingReplica); - }); - } - if (printReplicationWorkerId) { - try { - String replicationWorkerId = underreplicationManager - .getReplicationWorkerIdRereplicatingLedger(urLedgerId); - if (replicationWorkerId != null) { - System.out.println("\tReplicationWorkerId : " + replicationWorkerId); - } - } catch (UnavailableException e) { - LOG.error("Failed to get ReplicationWorkerId rereplicating ledger {} -- {}", urLedgerId, - e.getMessage()); - } - } - } - return null; - }); - + ListUnderReplicatedCommand.LURFlags flags = new ListUnderReplicatedCommand.LURFlags() + .missingReplica(includingBookieId) + .excludingMissingReplica(excludingBookieId) + .printMissingReplica(printMissingReplica) + .printReplicationWorkerId(printReplicationWorkerId); + ListUnderReplicatedCommand cmd = new ListUnderReplicatedCommand(ledgerIdFormatter); + cmd.apply(bkConf, flags); return 0; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ListUnderReplicatedCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ListUnderReplicatedCommand.java new file mode 100644 index 00000000000..1142d31c66e --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ListUnderReplicatedCommand.java @@ -0,0 +1,176 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.function.Predicate; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.UnderreplicatedLedger; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.replication.ReplicationException; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.apache.commons.lang.StringUtils; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to listing under replicated ledgers. + */ +public class ListUnderReplicatedCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(ListUnderReplicatedCommand.class); + + private static final String NAME = "listunderreplicated"; + private static final String DESC = "List ledgers marked as underreplicated, with oprional options to specify " + + "missingreplica (BookieId) and to exclude missingreplica."; + private static final String DEFAULT = ""; + + private LedgerIdFormatter ledgerIdFormatter; + + public ListUnderReplicatedCommand() { + this(new LURFlags()); + } + + public ListUnderReplicatedCommand(LedgerIdFormatter ledgerIdFormatter) { + this(); + this.ledgerIdFormatter = ledgerIdFormatter; + } + + private ListUnderReplicatedCommand(LURFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for list under replicated command. + */ + @Accessors(fluent = true) + @Setter + public static class LURFlags extends CliFlags{ + + @Parameter(names = { "-pmr", "--printmissingreplica" }, description = "Whether to print missingreplicas list?") + private boolean printMissingReplica; + + @Parameter(names = { "-prw", + "--printreplicationworkerid" }, description = "Whether wo print replicationworkerid?") + private boolean printReplicationWorkerId; + + @Parameter(names = { "-mr", "--missingreplica" }, description = "Bookie Id of missing replica") + private String missingReplica = DEFAULT; + + @Parameter(names = { "-emr", "--excludingmissingreplica" }, description = "Bookie Id of missing replica to " + + "ignore") + private String excludingMissingReplica = DEFAULT; + + @Parameter(names = {"-l", "--ledgeridformatter"}, description = "Set ledger id formatter") + private String ledgerIdFormatter = DEFAULT; + } + + @Override + public boolean apply(ServerConfiguration conf, LURFlags cmdFlags) { + if (!cmdFlags.ledgerIdFormatter.equals(DEFAULT) && ledgerIdFormatter == null) { + ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(cmdFlags.ledgerIdFormatter, conf); + } else if (ledgerIdFormatter == null) { + ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } + try { + return handler(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + public boolean handler(ServerConfiguration bkConf, LURFlags flags) throws MetadataException, ExecutionException { + final String includingBookieId = flags.missingReplica; + final String excludingBookieId = flags.excludingMissingReplica; + final boolean printMissingReplica = flags.printMissingReplica; + final boolean printReplicationWorkerId = flags.printReplicationWorkerId; + + final Predicate> predicate; + if (!StringUtils.isBlank(includingBookieId) && !StringUtils.isBlank(excludingBookieId)) { + predicate = replicasList -> (replicasList.contains(includingBookieId) + && !replicasList.contains(excludingBookieId)); + } else if (!StringUtils.isBlank(includingBookieId)) { + predicate = replicasList -> replicasList.contains(includingBookieId); + } else if (!StringUtils.isBlank(excludingBookieId)) { + predicate = replicasList -> !replicasList.contains(excludingBookieId); + } else { + predicate = null; + } + + runFunctionWithLedgerManagerFactory(bkConf, mFactory -> { + LedgerUnderreplicationManager underreplicationManager; + try { + underreplicationManager = mFactory.newLedgerUnderreplicationManager(); + } catch (KeeperException | ReplicationException.CompatibilityException e) { + throw new UncheckedExecutionException("Failed to new ledger underreplicated manager", e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedExecutionException("Interrupted on newing ledger underreplicated manager", e); + } + Iterator iter = underreplicationManager.listLedgersToRereplicate(predicate); + while (iter.hasNext()) { + UnderreplicatedLedger underreplicatedLedger = iter.next(); + long urLedgerId = underreplicatedLedger.getLedgerId(); + System.out.println(ledgerIdFormatter.formatLedgerId(urLedgerId)); + long ctime = underreplicatedLedger.getCtime(); + if (ctime != UnderreplicatedLedger.UNASSIGNED_CTIME) { + System.out.println("\tCtime : " + ctime); + } + if (printMissingReplica) { + underreplicatedLedger.getReplicaList().forEach((missingReplica) -> { + System.out.println("\tMissingReplica : " + missingReplica); + }); + } + if (printReplicationWorkerId) { + try { + String replicationWorkerId = underreplicationManager + .getReplicationWorkerIdRereplicatingLedger(urLedgerId); + if (replicationWorkerId != null) { + System.out.println("\tReplicationWorkerId : " + replicationWorkerId); + } + } catch (ReplicationException.UnavailableException e) { + LOG.error("Failed to get ReplicationWorkerId rereplicating ledger {} -- {}", urLedgerId, + e.getMessage()); + } + } + } + return null; + }); + return true; + } + +} + diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java index fa1bbbff0a1..9294bf9ba3a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java @@ -18,6 +18,6 @@ */ /** - * This package provides all toggle commands. + * This package provides all autorecovery commands. */ package org.apache.bookkeeper.tools.cli.commands.autorecovery; \ No newline at end of file diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java index d59412b3aa7..9cee7ff40a0 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java @@ -20,6 +20,7 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -37,11 +38,11 @@ public class AutoRecoveryCommandGroup extends CliCommandGroup { .withName(NAME) .withDescription(DESC) .withCategory(CATEGORY_INFRA_SERVICE) + .addCommand(new ListUnderReplicatedCommand()) .addCommand(new LostBookieRecoveryDelayCommand()) .build(); public AutoRecoveryCommandGroup() { super(spec); } -} - +} \ No newline at end of file diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ListUnderReplicatedCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ListUnderReplicatedCommandTest.java new file mode 100644 index 00000000000..5fefe585026 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ListUnderReplicatedCommandTest.java @@ -0,0 +1,160 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.when; + +import java.util.ArrayList; +import java.util.Vector; +import java.util.function.Function; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.UnderreplicatedLedger; +import org.apache.bookkeeper.replication.ReplicationException; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.zookeeper.KeeperException; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link ListUnderReplicatedCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ListUnderReplicatedCommand.class, MetadataDrivers.class, UnderreplicatedLedger.class }) +public class ListUnderReplicatedCommandTest extends BookieCommandTestBase { + + private UnderreplicatedLedger ledger; + private LedgerManagerFactory factory; + private LedgerUnderreplicationManager underreplicationManager; + + public ListUnderReplicatedCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + PowerMockito.mockStatic(MetadataDrivers.class); + factory = mock(LedgerManagerFactory.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function function = invocationOnMock.getArgument(1); + function.apply(factory); + return true; + }).when(MetadataDrivers.class, "runFunctionWithLedgerManagerFactory", any(ServerConfiguration.class), + any(Function.class)); + + underreplicationManager = mock(LedgerUnderreplicationManager.class); + when(factory.newLedgerUnderreplicationManager()).thenReturn(underreplicationManager); + + ledger = mock(UnderreplicatedLedger.class); + when(ledger.getLedgerId()).thenReturn(1L); + when(ledger.getCtime()).thenReturn(1L); + + Vector ledgers = new Vector<>(); + ledgers.add(ledger); + + when(underreplicationManager.listLedgersToRereplicate(any())).thenReturn(ledgers.iterator()); + + } + + @Test + public void testWithoutArgs() + throws InterruptedException, ReplicationException.CompatibilityException, KeeperException { + testCommand(""); + verify(factory, times(1)).newLedgerUnderreplicationManager(); + verify(underreplicationManager, times(1)).listLedgersToRereplicate(any()); + verify(ledger, times(1)).getLedgerId(); + verify(ledger, times(1)).getCtime(); + } + + @Test + public void testMissingReplica() + throws InterruptedException, ReplicationException.CompatibilityException, KeeperException { + testCommand("-mr", ""); + verify(factory, times(1)).newLedgerUnderreplicationManager(); + verify(underreplicationManager, times(1)).listLedgersToRereplicate(any()); + verify(ledger, times(1)).getLedgerId(); + verify(ledger, times(1)).getCtime(); + } + + @Test + public void testExcludingMissingReplica() + throws InterruptedException, ReplicationException.CompatibilityException, KeeperException { + testCommand("-emr", ""); + verify(factory, times(1)).newLedgerUnderreplicationManager(); + verify(underreplicationManager, times(1)).listLedgersToRereplicate(any()); + verify(ledger, times(1)).getLedgerId(); + verify(ledger, times(1)).getCtime(); + } + + @Test + public void testPrintMissingReplica() + throws InterruptedException, ReplicationException.CompatibilityException, KeeperException { + + ArrayList list = new ArrayList<>(); + list.add("replica"); + + when(ledger.getReplicaList()).thenReturn(list); + testCommand("-pmr"); + verify(factory, times(1)).newLedgerUnderreplicationManager(); + verify(underreplicationManager, times(1)).listLedgersToRereplicate(any()); + verify(ledger, times(1)).getLedgerId(); + verify(ledger, times(1)).getCtime(); + verify(ledger, times(1)).getReplicaList(); + } + + @Test + public void testPrintReplicationWorkerId() throws ReplicationException.UnavailableException, InterruptedException, + ReplicationException.CompatibilityException, KeeperException { + when(underreplicationManager.getReplicationWorkerIdRereplicatingLedger(1L)).thenReturn("test"); + + testCommand("-prw"); + verify(factory, times(1)).newLedgerUnderreplicationManager(); + verify(underreplicationManager, times(1)).listLedgersToRereplicate(any()); + verify(ledger, times(1)).getLedgerId(); + verify(ledger, times(1)).getCtime(); + verify(underreplicationManager, times(1)).getReplicationWorkerIdRereplicatingLedger(1L); + } + + @Test + public void testCommand1() { + ListUnderReplicatedCommand cmd = new ListUnderReplicatedCommand(); + cmd.apply(bkFlags, new String[] { "" }); + } + + private void testCommand(String... args) { + ListUnderReplicatedCommand cmd = new ListUnderReplicatedCommand(); + Assert.assertTrue(cmd.apply(bkFlags, args)); + } + +} + From e809e824272a46f30944fd86249aec2d100b3ff7 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 3 Apr 2019 14:16:23 +0800 Subject: [PATCH 0259/1642] Migrate command `nukeexistingcluster` Descriptions of the changes in this PR: #2029 Reviewers: Sijie Guo This closes #2030 from zymap/command-nukeexistingcluster --- .../apache/bookkeeper/bookie/BookieShell.java | 20 ++-- .../bookies/NukeExistingClusterCommand.java | 92 +++++++++++++++++++ .../cli/commands/BookiesCommandGroup.java | 2 + .../NukeExistingClusterCommandTest.java | 75 +++++++++++++++ 4 files changed, 176 insertions(+), 13 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/NukeExistingClusterCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/NukeExistingClusterCommandTest.java 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 6f8f1c30c73..725faa71e40 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 @@ -111,6 +111,8 @@ import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.NukeExistingClusterCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.NukeExistingClusterCommand.NukeExistingClusterFlags; import org.apache.bookkeeper.tools.cli.commands.client.DeleteLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; @@ -405,19 +407,11 @@ int runCmd(CommandLine cmdLine) throws Exception { String zkledgersrootpath = cmdLine.getOptionValue("zkledgersrootpath"); String instanceid = cmdLine.getOptionValue("instanceid"); - /* - * for NukeExistingCluster command 'zkledgersrootpath' should be provided and either force option or - * instanceid should be provided. - */ - if ((zkledgersrootpath == null) || (force == (instanceid != null))) { - LOG.error( - "zkledgersrootpath should be specified and either force option " - + "or instanceid should be specified (but not both)"); - printUsage(); - return -1; - } - - boolean result = BookKeeperAdmin.nukeExistingCluster(bkConf, zkledgersrootpath, instanceid, force); + NukeExistingClusterCommand cmd = new NukeExistingClusterCommand(); + NukeExistingClusterFlags flags = new NukeExistingClusterFlags().force(force) + .zkLedgersRootPath(zkledgersrootpath) + .instandId(instanceid); + boolean result = cmd.apply(bkConf, flags); return (result) ? 0 : 1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/NukeExistingClusterCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/NukeExistingClusterCommand.java new file mode 100644 index 00000000000..4e46ccbeb8f --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/NukeExistingClusterCommand.java @@ -0,0 +1,92 @@ +/* + * 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.tools.cli.commands.bookies; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Nuke bookkeeper metadata of existing cluster in zookeeper. + */ +public class NukeExistingClusterCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(NukeExistingClusterCommand.class); + + private static final String NAME = "nukeexistingcluster"; + private static final String DESC = "Nuke bookkeeper cluster by deleting metadata."; + + public NukeExistingClusterCommand() { + this(new NukeExistingClusterFlags()); + } + + private NukeExistingClusterCommand(NukeExistingClusterFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for nuke existing cluster command. + */ + @Accessors(fluent = true) + @Setter + public static class NukeExistingClusterFlags extends CliFlags { + + @Parameter(names = {"-f", "--force"}, + description = "If instance id is not specified, then whether to force nuke " + + "the metadata without " + "validating instance id") + private boolean force; + + @Parameter(names = {"-p", "--zkledgersrootpath"}, description = "zookeeper ledgers root path", required = true) + private String zkLedgersRootPath; + + @Parameter(names = {"-i", "--instanceid"}, description = "instance id") + private String instandId; + + } + + @Override + public boolean apply(ServerConfiguration conf, NukeExistingClusterFlags cmdFlags) { + /* + * for NukeExistingCluster command 'zkledgersrootpath' should be provided and either force option or + * instanceid should be provided. + */ + if (cmdFlags.force == (cmdFlags.instandId != null)) { + LOG.error("Either force option or instanceid should be specified (but no both)"); + return false; + } + try { + return BookKeeperAdmin.nukeExistingCluster(conf, cmdFlags.zkLedgersRootPath, + cmdFlags.instandId, cmdFlags.force); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java index e211b133f6f..710a2fe3772 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookies.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.NukeExistingClusterCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -44,6 +45,7 @@ public class BookiesCommandGroup extends CliCommandGroup { .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new ListBookiesCommand()) .addCommand(new InfoCommand()) + .addCommand(new NukeExistingClusterCommand()) .addCommand(new MetaFormatCommand()) .addCommand(new InitCommand()) .build(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/NukeExistingClusterCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/NukeExistingClusterCommandTest.java new file mode 100644 index 00000000000..a6ebd9e9a83 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/NukeExistingClusterCommandTest.java @@ -0,0 +1,75 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; + +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link NukeExistingClusterCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ NukeExistingClusterCommand.class, BookKeeperAdmin.class }) +public class NukeExistingClusterCommandTest extends BookieCommandTestBase { + + public NukeExistingClusterCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + +// PowerMockito.mockStatic(BookKeeperAdmin.classs); + } + + @Test + public void commandWithoutFlags() throws Exception { + NukeExistingClusterCommand cmd = new NukeExistingClusterCommand(); + Assert.assertFalse(cmd.apply(bkFlags, new String[] { "" })); + } + + @Test + public void commandWithoutForceAndInstanceId() { + NukeExistingClusterCommand cmd = new NukeExistingClusterCommand(); + Assert.assertFalse(cmd.apply(bkFlags, new String[] { "-p", "" })); + } + + @Test + public void testCommand() throws Exception { + PowerMockito.mockStatic(BookKeeperAdmin.class); + PowerMockito.when( + BookKeeperAdmin.nukeExistingCluster(any(ServerConfiguration.class), anyString(), anyString(), anyBoolean())) + .thenReturn(true); + + NukeExistingClusterCommand cmd = new NukeExistingClusterCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-p", "", "-i", "1" })); + } +} From a7693df0c29b8a8a22842da26325ba1cbcf4cdb0 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 3 Apr 2019 14:17:45 +0800 Subject: [PATCH 0260/1642] Migrate command `readlog` Descriptions of the changes in this PR: #2044 Reviewers: Sijie Guo This closes #2045 from zymap/command-readlog --- .../apache/bookkeeper/bookie/BookieShell.java | 191 +---------- .../cli/commands/bookie/ReadLogCommand.java | 296 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../commands/bookie/ReadLogCommandTest.java | 86 +++++ 4 files changed, 396 insertions(+), 179 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommandTest.java 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 725faa71e40..0d8dea5b6d5 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 @@ -27,7 +27,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.util.concurrent.UncheckedExecutionException; -import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.EventLoopGroup; @@ -68,7 +67,6 @@ 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.storage.ldb.LocationsIndexRebuildOp; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.MetaStoreException; @@ -99,13 +97,13 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; -import org.apache.bookkeeper.tools.cli.commands.bookie.FormatUtil; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; @@ -142,7 +140,6 @@ import org.apache.commons.configuration.PropertiesConfiguration; import org.apache.commons.io.FileUtils; import org.apache.commons.lang.StringUtils; -import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.commons.lang3.ArrayUtils; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; @@ -1182,6 +1179,8 @@ public int runCmd(CommandLine cmdLine) throws Exception { printUsage(); return -1; } + ReadLogCommand cmd = new ReadLogCommand(ledgerIdFormatter, entryFormatter); + ReadLogCommand.ReadLogFlags flags = new ReadLogCommand.ReadLogFlags(); boolean printMsg = false; if (cmdLine.hasOption("m")) { @@ -1190,40 +1189,22 @@ public int runCmd(CommandLine cmdLine) throws Exception { long logId; try { logId = Long.parseLong(leftArgs[0]); + flags.entryLogId(logId); } catch (NumberFormatException nfe) { // not a entry log id - File f = new File(leftArgs[0]); - String name = f.getName(); - if (!name.endsWith(".log")) { - // not a log file - System.err.println("ERROR: invalid entry log file name " + leftArgs[0]); - printUsage(); - return -1; - } - String idString = name.split("\\.")[0]; - logId = Long.parseLong(idString, 16); + flags.filename(leftArgs[0]); } - final long lId = getOptionLedgerIdValue(cmdLine, "ledgerid", -1); final long eId = getOptionLongValue(cmdLine, "entryid", -1); final long startpos = getOptionLongValue(cmdLine, "startpos", -1); final long endpos = getOptionLongValue(cmdLine, "endpos", -1); - - // scan entry log - if (startpos != -1) { - if ((endpos != -1) && (endpos < startpos)) { - System.err - .println("ERROR: StartPosition of the range should be lesser than or equal to EndPosition"); - return -1; - } - scanEntryLogForPositionRange(logId, startpos, endpos, printMsg); - } else if (lId != -1) { - scanEntryLogForSpecificEntry(logId, lId, eId, printMsg); - } else { - scanEntryLog(logId, printMsg); - } - - return 0; + flags.endPos(endpos); + flags.startPos(startpos); + flags.entryId(eId); + flags.ledgerId(lId); + flags.msg(printMsg); + boolean result = cmd.apply(bkConf, flags); + return (result) ? 0 : -1; } @Override @@ -2665,30 +2646,6 @@ public static void main(String argv[]) throws Exception { System.exit(res); } - /// - /// Bookie File Operations - /// - - /** - * Get the ledger file of a specified ledger. - * - * @param ledgerId Ledger Id - * - * @return file object. - */ - private File getLedgerFile(long ledgerId) { - String ledgerName = IndexPersistenceMgr.getLedgerName(ledgerId); - File lf = null; - for (File d : indexDirectories) { - lf = new File(d, ledgerName); - if (lf.exists()) { - break; - } - lf = null; - } - return lf; - } - private synchronized void initEntryLogger() throws IOException { if (null == entryLogger) { // provide read only entry logger @@ -2696,17 +2653,6 @@ private synchronized void initEntryLogger() throws IOException { } } - /** - * Scan over entry log. - * - * @param logId Entry Log Id - * @param scanner Entry Log Scanner - */ - protected void scanEntryLog(long logId, EntryLogScanner scanner) throws IOException { - initEntryLogger(); - entryLogger.scanEntryLog(logId, scanner); - } - /// /// Bookie Shell Commands /// @@ -2721,119 +2667,6 @@ protected void printEntryLogMetadata(long logId) throws IOException { }); } - /** - * Scan over an entry log file. - * - * @param logId - * Entry Log File id. - * @param printMsg - * Whether printing the entry data. - */ - protected void scanEntryLog(long logId, final boolean printMsg) throws Exception { - System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)"); - scanEntryLog(logId, new EntryLogScanner() { - @Override - public boolean accept(long ledgerId) { - return true; - } - - @Override - public void process(long ledgerId, long startPos, ByteBuf entry) { - FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter); - } - }); - } - - /** - * Scan over an entry log file for a particular entry. - * - * @param logId Entry Log File id. - * @param ledgerId id of the ledger - * @param entryId entryId of the ledger we are looking for (-1 for all of the entries of the ledger) - * @param printMsg Whether printing the entry data. - * @throws Exception - */ - protected void scanEntryLogForSpecificEntry(long logId, final long ledgerId, final long entryId, - final boolean printMsg) throws Exception { - System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for LedgerId " - + ledgerId + ((entryId == -1) ? "" : " for EntryId " + entryId)); - final MutableBoolean entryFound = new MutableBoolean(false); - scanEntryLog(logId, new EntryLogScanner() { - @Override - public boolean accept(long candidateLedgerId) { - return ((candidateLedgerId == ledgerId) && ((!entryFound.booleanValue()) || (entryId == -1))); - } - - @Override - public void process(long candidateLedgerId, long startPos, ByteBuf entry) { - long entrysLedgerId = entry.getLong(entry.readerIndex()); - long entrysEntryId = entry.getLong(entry.readerIndex() + 8); - if ((candidateLedgerId == entrysLedgerId) && (candidateLedgerId == ledgerId) - && ((entrysEntryId == entryId) || (entryId == -1))) { - entryFound.setValue(true); - FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter); - } - } - }); - if (!entryFound.booleanValue()) { - System.out.println("LedgerId " + ledgerId + ((entryId == -1) ? "" : " EntryId " + entryId) - + " is not available in the entry log " + logId + " (" + Long.toHexString(logId) + ".log)"); - } - } - - /** - * Scan over an entry log file for entries in the given position range. - * - * @param logId Entry Log File id. - * @param rangeStartPos Start position of the entry we are looking for - * @param rangeEndPos End position of the entry we are looking for (-1 for till the end of the entrylog) - * @param printMsg Whether printing the entry data. - * @throws Exception - */ - protected void scanEntryLogForPositionRange(long logId, final long rangeStartPos, final long rangeEndPos, - final boolean printMsg) throws Exception { - System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for PositionRange: " - + rangeStartPos + " - " + rangeEndPos); - final MutableBoolean entryFound = new MutableBoolean(false); - scanEntryLog(logId, new EntryLogScanner() { - private MutableBoolean stopScanning = new MutableBoolean(false); - - @Override - public boolean accept(long ledgerId) { - return !stopScanning.booleanValue(); - } - - @Override - public void process(long ledgerId, long entryStartPos, ByteBuf entry) { - if (!stopScanning.booleanValue()) { - if ((rangeEndPos != -1) && (entryStartPos > rangeEndPos)) { - stopScanning.setValue(true); - } else { - int entrySize = entry.readableBytes(); - /** - * entrySize of an entry (inclusive of payload and - * header) value is stored as int value in log file, but - * it is not counted in the entrySize, hence for calculating - * the end position of the entry we need to add additional - * 4 (intsize of entrySize). Please check - * EntryLogger.scanEntryLog. - */ - long entryEndPos = entryStartPos + entrySize + 4 - 1; - if (((rangeEndPos == -1) || (entryStartPos <= rangeEndPos)) && (rangeStartPos <= entryEndPos)) { - FormatUtil.formatEntry(entryStartPos, entry, printMsg, ledgerIdFormatter, entryFormatter); - entryFound.setValue(true); - } - } - } - } - }); - if (!entryFound.booleanValue()) { - System.out.println("Entry log " + logId + " (" + Long.toHexString(logId) - + ".log) doesn't has any entry in the range " + rangeStartPos + " - " + rangeEndPos - + ". Probably the position range, you have provided is lesser than the LOGFILE_HEADER_SIZE (1024) " - + "or greater than the current log filesize."); - } - } /** * Format the entry into a readable format. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java new file mode 100644 index 00000000000..eae71ca9239 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java @@ -0,0 +1,296 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import io.netty.buffer.ByteBuf; +import java.io.File; +import java.io.IOException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.EntryLogger; +import org.apache.bookkeeper.bookie.ReadOnlyEntryLogger; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.EntryFormatter; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.apache.commons.lang.mutable.MutableBoolean; + +/** + * Command to read entry log files. + */ +public class ReadLogCommand extends BookieCommand { + + private static final String NAME = "readlog"; + private static final String DESC = "Scan an entry file and format the entries into readable format."; + + private EntryLogger entryLogger; + private EntryFormatter entryFormatter; + private LedgerIdFormatter ledgerIdFormatter; + + public ReadLogCommand() { + this(new ReadLogFlags()); + } + + public ReadLogCommand(LedgerIdFormatter ledgerIdFormatter, EntryFormatter entryFormatter) { + this(new ReadLogFlags()); + this.ledgerIdFormatter = ledgerIdFormatter; + this.entryFormatter = entryFormatter; + } + private ReadLogCommand(ReadLogFlags flags) { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build()); + } + + /** + * Flags for read log command. + */ + @Accessors(fluent = true) + @Setter + public static class ReadLogFlags extends CliFlags { + + @Parameter(names = { "-m", "msg" }, description = "Print message body") + private boolean msg; + + @Parameter(names = { "-l", "--ledgerid" }, description = "Ledger ID") + private long ledgerId = -1; + + @Parameter(names = { "-e", "--entryid" }, description = "Entry ID") + private long entryId = -1; + + @Parameter(names = { "-sp", "--startpos" }, description = "Start Position") + private long startPos = -1; + + @Parameter(names = { "-ep", "--endpos" }, description = "End Position") + private long endPos = -1; + + @Parameter(names = { "-f", "--filename" }, description = "Entry log filename") + private String filename; + + @Parameter(names = { "-li", "--entrylogid" }, description = "Entry log id") + private long entryLogId = -1; + + @Parameter(names = {"-lf", "--ledgerIdFormatter"}, description = "Set ledger id formatter") + private String ledgerIdFormatter; + + @Parameter(names = {"-ef", "--entryformatter"}, description = "set entry formatter") + private String entryFormatter; + } + + @Override + public boolean apply(ServerConfiguration conf, ReadLogFlags cmdFlags) { + + if (cmdFlags.ledgerIdFormatter != null && this.ledgerIdFormatter == null) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(cmdFlags.ledgerIdFormatter, conf); + } else if (this.ledgerIdFormatter == null) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } + + if (cmdFlags.entryFormatter != null && this.entryFormatter == null) { + this.entryFormatter = EntryFormatter.newEntryFormatter(cmdFlags.entryFormatter, conf); + } else if (this.entryFormatter == null) { + this.entryFormatter = EntryFormatter.newEntryFormatter(conf); + } + + if (cmdFlags.entryLogId == -1 && cmdFlags.filename == null) { + System.err.println("Missing entry log id or entry log file name"); + usage(); + return false; + } + try { + return readLog(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean readLog(ServerConfiguration conf, ReadLogFlags flags) throws Exception { + long logId = flags.entryLogId; + if (logId == -1 && flags.filename != null) { + File f = new File(flags.filename); + String name = f.getName(); + if (!name.endsWith(".log")) { + System.err.println("Invalid entry log file name " + flags.filename); + usage(); + return false; + } + String idString = name.split("\\.")[0]; + logId = Long.parseLong(idString, 16); + } + + final long lId = flags.ledgerId; + final long eId = flags.entryId; + final long startpos = flags.startPos; + final long endpos = flags.endPos; + + // scan entry log + if (startpos != -1) { + if ((endpos != -1) && (endpos < startpos)) { + System.err + .println("ERROR: StartPosition of the range should be lesser than or equal to EndPosition"); + return false; + } + scanEntryLogForPositionRange(conf, logId, startpos, endpos, flags.msg); + } else if (lId != -1) { + scanEntryLogForSpecificEntry(conf, logId, lId, eId, flags.msg); + } else { + scanEntryLog(conf, logId, flags.msg); + } + return true; + } + + /** + * Scan over an entry log file for entries in the given position range. + * + * @param logId Entry Log File id. + * @param rangeStartPos Start position of the entry we are looking for + * @param rangeEndPos End position of the entry we are looking for (-1 for till the end of the entrylog) + * @param printMsg Whether printing the entry data. + * @throws Exception + */ + private void scanEntryLogForPositionRange(ServerConfiguration conf, long logId, final long rangeStartPos, + final long rangeEndPos, + final boolean printMsg) throws Exception { + System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for PositionRange: " + + rangeStartPos + " - " + rangeEndPos); + final MutableBoolean entryFound = new MutableBoolean(false); + scanEntryLog(conf, logId, new EntryLogger.EntryLogScanner() { + private MutableBoolean stopScanning = new MutableBoolean(false); + + @Override + public boolean accept(long ledgerId) { + return !stopScanning.booleanValue(); + } + + @Override + public void process(long ledgerId, long entryStartPos, ByteBuf entry) throws IOException { + if (!stopScanning.booleanValue()) { + if ((rangeEndPos != -1) && (entryStartPos > rangeEndPos)) { + stopScanning.setValue(true); + } else { + int entrySize = entry.readableBytes(); + /** + * entrySize of an entry (inclusive of payload and + * header) value is stored as int value in log file, but + * it is not counted in the entrySize, hence for calculating + * the end position of the entry we need to add additional + * 4 (intsize of entrySize). Please check + * EntryLogger.scanEntryLog. + */ + long entryEndPos = entryStartPos + entrySize + 4 - 1; + if (((rangeEndPos == -1) || (entryStartPos <= rangeEndPos)) && (rangeStartPos <= entryEndPos)) { + FormatUtil.formatEntry(entryStartPos, entry, printMsg, ledgerIdFormatter, entryFormatter); + entryFound.setValue(true); + } + } + } + } + }); + if (!entryFound.booleanValue()) { + System.out.println( + "Entry log " + logId + " (" + Long.toHexString(logId) + ".log) doesn't has any entry in the range " + + rangeStartPos + " - " + rangeEndPos + + ". Probably the position range, you have provided is lesser than the LOGFILE_HEADER_SIZE (1024) " + + "or greater than the current log filesize."); + } + } + + /** + * Scan over entry log. + * + * @param logId Entry Log Id + * @param scanner Entry Log Scanner + */ + private void scanEntryLog(ServerConfiguration conf, long logId, EntryLogger.EntryLogScanner scanner) + throws IOException { + initEntryLogger(conf); + entryLogger.scanEntryLog(logId, scanner); + } + + private synchronized void initEntryLogger(ServerConfiguration conf) throws IOException { + if (null == entryLogger) { + // provide read only entry logger + entryLogger = new ReadOnlyEntryLogger(conf); + } + } + + /** + * Scan over an entry log file for a particular entry. + * + * @param logId Entry Log File id. + * @param ledgerId id of the ledger + * @param entryId entryId of the ledger we are looking for (-1 for all of the entries of the ledger) + * @param printMsg Whether printing the entry data. + * @throws Exception + */ + private void scanEntryLogForSpecificEntry(ServerConfiguration conf, long logId, final long ledgerId, + final long entryId, + final boolean printMsg) throws Exception { + System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for LedgerId " + + ledgerId + ((entryId == -1) ? "" : " for EntryId " + entryId)); + final MutableBoolean entryFound = new MutableBoolean(false); + scanEntryLog(conf, logId, new EntryLogger.EntryLogScanner() { + @Override + public boolean accept(long candidateLedgerId) { + return ((candidateLedgerId == ledgerId) && ((!entryFound.booleanValue()) || (entryId == -1))); + } + + @Override + public void process(long candidateLedgerId, long startPos, ByteBuf entry) { + long entrysLedgerId = entry.getLong(entry.readerIndex()); + long entrysEntryId = entry.getLong(entry.readerIndex() + 8); + if ((candidateLedgerId == entrysLedgerId) && (candidateLedgerId == ledgerId) + && ((entrysEntryId == entryId) || (entryId == -1))) { + entryFound.setValue(true); + FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter); + } + } + }); + if (!entryFound.booleanValue()) { + System.out.println("LedgerId " + ledgerId + ((entryId == -1) ? "" : " EntryId " + entryId) + + " is not available in the entry log " + logId + " (" + Long.toHexString(logId) + + ".log)"); + } + } + + /** + * Scan over an entry log file. + * + * @param logId + * Entry Log File id. + * @param printMsg + * Whether printing the entry data. + */ + private void scanEntryLog(ServerConfiguration conf, long logId, final boolean printMsg) throws Exception { + System.out.println("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)"); + scanEntryLog(conf, logId, new EntryLogger.EntryLogScanner() { + @Override + public boolean accept(long ledgerId) { + return true; + } + + @Override + public void process(long ledgerId, long startPos, ByteBuf entry) { + FormatUtil.formatEntry(startPos, entry, printMsg, ledgerIdFormatter, entryFormatter); + } + }); + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 98480ef0436..2970c8c5e3a 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -30,6 +30,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; @@ -59,6 +60,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new ListLedgersCommand()) .addCommand(new ConvertToInterleavedStorageCommand()) .addCommand(new ReadJournalCommand()) + .addCommand(new ReadLogCommand()) .addCommand(new ReadLogMetadataCommand()) .build(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommandTest.java new file mode 100644 index 00000000000..7f18fcbbcb9 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommandTest.java @@ -0,0 +1,86 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.powermock.api.mockito.PowerMockito.doNothing; + +import org.apache.bookkeeper.bookie.EntryLogger; +import org.apache.bookkeeper.bookie.ReadOnlyEntryLogger; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link ReadLogCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ ReadLogCommand.class }) +public class ReadLogCommandTest extends BookieCommandTestBase { + + @Mock + private ReadOnlyEntryLogger entryLogger; + + @Mock + private EntryLogger.EntryLogScanner entryLogScanner; + + public ReadLogCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.whenNew(ReadOnlyEntryLogger.class).withArguments(eq(conf)).thenReturn(entryLogger); + doNothing().when(entryLogger).scanEntryLog(anyLong(), eq(entryLogScanner)); + } + + @Test + public void testWithoutAnyFlags() { + ReadLogCommand cmd = new ReadLogCommand(); + Assert.assertFalse(cmd.apply(bkFlags, new String[] {})); + } + + @Test + public void testWithEntryId() { + ReadLogCommand cmd = new ReadLogCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-li", "1" })); + } + + @Test + public void testWithEntryFilename() { + ReadLogCommand cmd = new ReadLogCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-f", "1.log" })); + } + + @Test + public void testWithErrorPos() { + ReadLogCommand cmd = new ReadLogCommand(); + Assert.assertFalse(cmd.apply(bkFlags, new String[] { "-sp", "1", "-ep", "0", "-li", "1" })); + } +} From e80fb5f9215205e638e7a5c1817c8c282b0f41cb Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 3 Apr 2019 14:18:52 +0800 Subject: [PATCH 0261/1642] Migrate command `decommission` Descriptions of the changes in this PR: #2049 Reviewers: Sijie Guo This closes #2050 from zymap/command-decommissionbookie --- .../apache/bookkeeper/bookie/BookieShell.java | 41 +----- .../commands/bookies/DecommissionCommand.java | 126 ++++++++++++++++ .../cli/commands/BookiesCommandGroup.java | 2 + .../bookies/DecommissionCommandTest.java | 135 ++++++++++++++++++ 4 files changed, 270 insertions(+), 34 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java 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 0d8dea5b6d5..870d31b45af 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 @@ -106,6 +106,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.DecommissionCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; @@ -2240,40 +2241,12 @@ Options getOptions() { @Override public int runCmd(CommandLine cmdLine) throws Exception { - ClientConfiguration adminConf = new ClientConfiguration(bkConf); - BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); - try { - final String remoteBookieidToDecommission = cmdLine.getOptionValue("bookieid"); - final BookieSocketAddress bookieAddressToDecommission = (StringUtils - .isBlank(remoteBookieidToDecommission) ? Bookie.getBookieAddress(bkConf) - : new BookieSocketAddress(remoteBookieidToDecommission)); - admin.decommissionBookie(bookieAddressToDecommission); - LOG.info("The ledgers stored in the given decommissioning bookie: {} are properly replicated", - bookieAddressToDecommission); - runFunctionWithRegistrationManager(bkConf, rm -> { - try { - Versioned cookie = Cookie.readFromRegistrationManager(rm, bookieAddressToDecommission); - cookie.getValue().deleteFromRegistrationManager(rm, bookieAddressToDecommission, - cookie.getVersion()); - } catch (CookieNotFoundException nne) { - LOG.warn("No cookie to remove for the decommissioning bookie: {}, it could be deleted already", - bookieAddressToDecommission, nne); - } catch (BookieException be) { - throw new UncheckedExecutionException(be.getMessage(), be); - } - return 0; - }); - LOG.info("Cookie of the decommissioned bookie: {} is deleted successfully", - bookieAddressToDecommission); - return 0; - } catch (Exception e) { - LOG.error("Received exception in DecommissionBookieCmd ", e); - return -1; - } finally { - if (admin != null) { - admin.close(); - } - } + DecommissionCommand cmd = new DecommissionCommand(); + DecommissionCommand.DecommissionFlags flags = new DecommissionCommand.DecommissionFlags(); + final String remoteBookieidToDecommission = cmdLine.getOptionValue("bookieid"); + flags.remoteBookieIdToDecommission(remoteBookieidToDecommission); + boolean result = cmd.apply(bkConf, flags); + return (result) ? 0 : -1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java new file mode 100644 index 00000000000..c7aa537fedf --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java @@ -0,0 +1,126 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.versioning.Versioned; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to trigger AuditTask by resetting lostBookieRecoveryDelay and + * then make sure the ledgers stored in the bookie are properly replicated + * and Cookie of the decommissioned bookie should be deleted from metadata + * server. + */ +public class DecommissionCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(DecommissionCommand.class); + + private static final String NAME = "decommission"; + private static final String DESC = + "Force trigger the Audittask and make sure all the ledgers stored in the decommissioning bookie" + + " are replicated and cookie of the decommissioned bookie is deleted from metadata server."; + + public DecommissionCommand() { + this(new DecommissionFlags()); + } + + private DecommissionCommand(DecommissionFlags flags) { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build()); + } + + /** + * Flags for decommission command. + */ + @Accessors(fluent = true) + @Setter + public static class DecommissionFlags extends CliFlags { + + @Parameter(names = { "-b", "--bookieid" }, description = "Decommission a remote bookie") + private String remoteBookieIdToDecommission; + + } + + @Override + public boolean apply(ServerConfiguration conf, DecommissionFlags cmdFlags) { + try { + return decommission(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean decommission(ServerConfiguration conf, DecommissionFlags flags) + throws BKException, InterruptedException, IOException { + ClientConfiguration adminConf = new ClientConfiguration(conf); + BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); + try { + final String remoteBookieidToDecommission = flags.remoteBookieIdToDecommission; + final BookieSocketAddress bookieAddressToDecommission = (StringUtils.isBlank(remoteBookieidToDecommission) + ? Bookie.getBookieAddress(conf) + : new BookieSocketAddress( + remoteBookieidToDecommission)); + admin.decommissionBookie(bookieAddressToDecommission); + LOG.info("The ledgers stored in the given decommissioning bookie: {} are properly replicated", + bookieAddressToDecommission); + runFunctionWithRegistrationManager(conf, rm -> { + try { + Versioned cookie = Cookie.readFromRegistrationManager(rm, bookieAddressToDecommission); + cookie.getValue().deleteFromRegistrationManager(rm, bookieAddressToDecommission, + cookie.getVersion()); + } catch (BookieException.CookieNotFoundException nne) { + LOG.warn("No cookie to remove for the decommissioning bookie: {}, it could be deleted already", + bookieAddressToDecommission, nne); + } catch (BookieException be) { + throw new UncheckedExecutionException(be.getMessage(), be); + } + return true; + }); + LOG.info("Cookie of the decommissioned bookie: {} is deleted successfully", + bookieAddressToDecommission); + return true; + } catch (Exception e) { + LOG.error("Received exception in DecommissionBookieCmd ", e); + return false; + } finally { + if (admin != null) { + admin.close(); + } + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java index 710a2fe3772..1f4d4cf1f79 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java @@ -21,6 +21,7 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; import org.apache.bookkeeper.tools.cli.BKCtl; +import org.apache.bookkeeper.tools.cli.commands.bookies.DecommissionCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; @@ -47,6 +48,7 @@ public class BookiesCommandGroup extends CliCommandGroup { .addCommand(new InfoCommand()) .addCommand(new NukeExistingClusterCommand()) .addCommand(new MetaFormatCommand()) + .addCommand(new DecommissionCommand()) .addCommand(new InitCommand()) .build(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java new file mode 100644 index 00000000000..7ce5a026e45 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java @@ -0,0 +1,135 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; + +import java.util.function.Function; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.Cookie; +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.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link DecommissionCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ DecommissionCommand.class, Bookie.class, MetadataDrivers.class, Cookie.class }) +public class DecommissionCommandTest extends BookieCommandTestBase { + + @Mock + private ClientConfiguration clientConfiguration; + + @Mock + private BookKeeperAdmin bookKeeperAdmin; + + @Mock + private BookieSocketAddress bookieSocketAddress; + + @Mock + private Versioned cookieVersioned; + + @Mock + private Cookie cookie; + + @Mock + private Version version; + public DecommissionCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.whenNew(ClientConfiguration.class).withArguments(eq(conf)).thenReturn(clientConfiguration); + PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class) + .withArguments(eq(clientConfiguration)).thenReturn(bookKeeperAdmin); + PowerMockito.whenNew(BookieSocketAddress.class).withArguments(anyString()).thenReturn(bookieSocketAddress); + PowerMockito.mockStatic(Bookie.class); + PowerMockito.when(Bookie.getBookieAddress(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress); + PowerMockito.doNothing().when(bookKeeperAdmin).decommissionBookie(eq(bookieSocketAddress)); + + RegistrationManager registrationManager = mock(RegistrationManager.class); + PowerMockito.mockStatic(MetadataDrivers.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function f = invocationOnMock.getArgument(1); + f.apply(registrationManager); + return true; + }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", any(ServerConfiguration.class), + any(Function.class)); + + PowerMockito.mockStatic(Cookie.class); + PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress))) + .thenReturn(cookieVersioned); + when(cookieVersioned.getValue()).thenReturn(cookie); + when(cookieVersioned.getVersion()).thenReturn(version); + PowerMockito.doNothing().when(cookie) + .deleteFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress), eq(version)); + } + + @Test + public void testWithoutBookieId() throws Exception { + DecommissionCommand cmd = new DecommissionCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "" })); + + verifyNew(ClientConfiguration.class, times(1)).withArguments(eq(conf)); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration)); + verifyNew(BookieSocketAddress.class, never()).withArguments(anyString()); + verify(bookKeeperAdmin, times(1)).decommissionBookie(eq(bookieSocketAddress)); + verify(cookieVersioned, times(1)).getValue(); + verify(cookieVersioned, times(1)).getVersion(); + } + + @Test + public void testWithBookieId() throws Exception { + DecommissionCommand cmd = new DecommissionCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-b", "1" })); + + verifyNew(ClientConfiguration.class, times(1)).withArguments(eq(conf)); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration)); + verifyNew(BookieSocketAddress.class, times(1)).withArguments(anyString()); + verify(bookKeeperAdmin, times(1)).decommissionBookie(eq(bookieSocketAddress)); + verify(cookieVersioned, times(1)).getValue(); + verify(cookieVersioned, times(1)).getVersion(); + } +} From fad61de3f9f4eb2d2044d3f075eeab7ca9b10c0c Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 3 Apr 2019 21:30:16 +0800 Subject: [PATCH 0262/1642] Migrate command `readledger` Descriptions of the changes in this PR: #2040 - Using `bkctl` run `readledger` ``` Read a range of entries from a ledger. Usage: bkctl bookie readledger [flags] Flags: -b, --bookie Only read from a specific bookie -ef, --entryformatter Set entry formatter -fe, --firstentryid First Entry ID -r, --force-recovery Ensure the ledger is properly closed before reading -le, --lastentryid Last Entry ID -l, --ledgerid Ledger ID -lf, --ledgeridformatter Set ledger id formatter -m, --msg Print message body -h, --help Display help information ``` Reviewers: Jia Zhai , Sijie Guo This closes #2041 from zymap/command-readledger and squashes the following commits: 56b0a4581 [Yong Zhang] Remove unused import 30dafa85f [Yong Zhang] Merge branch 'master' into command-readledger bfbd6b023 [Yong Zhang] Migrate command `decommission` d40b8b69f [Yong Zhang] Migrate command `readlog` 95d145a15 [Yong Zhang] Migrate command `nukeexistingcluster` e2b1dc7f3 [Yong Zhang] Migrate command `listunderreplicated` c465c4761 [Yong Zhang] Remove unused import 0988e12c7 [bd2019us] ISSUE #2023: change cached thread pool to fixed thread pool 6a6d7bbd9 [Yong Zhang] Migrate command `initnewcluster` 931df8c2c [Sijie Guo] Merge branch 'master' into command-readledger c391fe58d [Yong Zhang] Migrate command `readlogmetadata` 120d67737 [Yong Zhang] Migrate command `lostbookierecoverydelay` bf66235e5 [Yong Zhang] Migrate command `deleteledger` 87e6644f2 [Yong Zhang] Fix some conflict 5ae05f0d2 [Yong Zhang] Migrate command `readledger` 751e55fa4 [Arvin] ISSUE #2020: close db properly to avoid open RocksDB failure at the second time 138a7ae85 [Yong Zhang] Migrate command `metadataformat` b043d1694 [Yong Zhang] Migrate command `listledgers` 4573285db [Ivan Kelly] Docker autobuild hook e3d807a32 [Like] Fix IDE complain as there are multi choices for error code 9524a9f4a [Yong Zhang] Migrate command `readjournal` 6c3f33f55 [Yong Zhang] Fix when met unexpect entry id crashed e35a108c7 [Like] Fix error message for unrecognized number-of-bookies 5902ee27b [Boyang Jerry Peng] fix potential NPE when releasing entry that is null 6aa73ce05 [Ivan Kelly] [RELEASE] Update website to include documentation for 4.8.2 1448d12aa [Yong Zhang] Migrate command `listfilesondisk` 4de598379 [Yong Zhang] Issue #1987: Migrate command `convert-to-interleaved-storage` 468743e7e [Matteo Merli] In DbLedgerStorage use default values when config key is present but empty f26a4cae0 [Ivan Kelly] Release notes for v4.8.2 ec2636cd2 [Yong Zhang] Issue #1985: Migrate command `convert-to-db-storage` 8cc7239ac [Yong Zhang] Issue #1982: Migrate command `bookiesanity` fa90f0185 [Yong Zhang] Issue #1980: Migrate command `ledger` from shell to bkctl --- .../apache/bookkeeper/bookie/BookieShell.java | 104 +------- .../commands/bookie/ReadLedgerCommand.java | 244 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../bookie/ReadLedgerCommandTest.java | 169 ++++++++++++ 4 files changed, 428 insertions(+), 91 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommandTest.java 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 870d31b45af..98f276c75dc 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 @@ -27,11 +27,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.util.concurrent.UncheckedExecutionException; -import io.netty.buffer.ByteBufUtil; -import io.netty.buffer.UnpooledByteBufAllocator; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.util.concurrent.DefaultThreadFactory; import java.io.File; import java.io.IOException; import java.io.Serializable; @@ -50,7 +45,6 @@ import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -58,13 +52,8 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import java.util.stream.LongStream; - import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp; @@ -73,11 +62,9 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.LedgerEntry; -import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.UpdateLedgerOp; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; -import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationManager; @@ -86,12 +73,8 @@ import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookieClient; -import org.apache.bookkeeper.proto.BookieClientImpl; -import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.replication.ReplicationException; -import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; @@ -103,6 +86,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; @@ -772,91 +756,29 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { final long ledgerId = getOptionLedgerIdValue(cmdLine, "ledgerid", -1); - if (ledgerId == -1) { - System.err.println("Must specify a ledger id"); - return -1; - } - final long firstEntry = getOptionLongValue(cmdLine, "firstentryid", 0); long lastEntry = getOptionLongValue(cmdLine, "lastentryid", -1); boolean printMsg = cmdLine.hasOption("m"); boolean forceRecovery = cmdLine.hasOption("r"); final BookieSocketAddress bookie; + String bookieAddress; if (cmdLine.hasOption("b")) { // A particular bookie was specified - bookie = new BookieSocketAddress(cmdLine.getOptionValue("b")); + bookieAddress = cmdLine.getOptionValue("b"); } else { - bookie = null; - } - - ClientConfiguration conf = new ClientConfiguration(); - conf.addConfiguration(bkConf); - - try (BookKeeperAdmin bk = new BookKeeperAdmin(conf)) { - if (forceRecovery) { - // Force the opening of the ledger to trigger recovery - try (LedgerHandle lh = bk.openLedger(ledgerId)) { - if (lastEntry == -1 || lastEntry > lh.getLastAddConfirmed()) { - lastEntry = lh.getLastAddConfirmed(); - } - } - } - - if (bookie == null) { - // No bookie was specified, use normal bk client - Iterator entries = bk.readEntries(ledgerId, firstEntry, lastEntry).iterator(); - while (entries.hasNext()) { - LedgerEntry entry = entries.next(); - formatEntry(entry, printMsg); - } - } else { - // Use BookieClient to target a specific bookie - EventLoopGroup eventLoopGroup = new NioEventLoopGroup(); - OrderedExecutor executor = OrderedExecutor.newBuilder() - .numThreads(1) - .name("BookieClientScheduler") - .build(); - - ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor( - new DefaultThreadFactory("BookKeeperClientSchedulerPool")); - - BookieClient bookieClient = new BookieClientImpl(conf, eventLoopGroup, - UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE); - - LongStream.range(firstEntry, lastEntry).forEach(entryId -> { - CompletableFuture future = new CompletableFuture<>(); - - bookieClient.readEntry(bookie, ledgerId, entryId, - (rc, ledgerId1, entryId1, buffer, ctx) -> { - if (rc != BKException.Code.OK) { - LOG.error("Failed to read entry {} -- {}", entryId1, BKException.getMessage(rc)); - future.completeExceptionally(BKException.create(rc)); - return; - } - - System.out.println("--------- Lid=" + ledgerIdFormatter.formatLedgerId(ledgerId) - + ", Eid=" + entryId + " ---------"); - if (printMsg) { - System.out.println("Data: " + ByteBufUtil.prettyHexDump(buffer)); - } - - future.complete(null); - }, null, BookieProtocol.FLAG_NONE); - - try { - future.get(); - } catch (Exception e) { - LOG.error("Error future.get while reading entries from ledger {}", ledgerId, e); - } - }); - - eventLoopGroup.shutdownGracefully(); - executor.shutdown(); - bookieClient.close(); - } + bookieAddress = null; } + ReadLedgerCommand cmd = new ReadLedgerCommand(entryFormatter, ledgerIdFormatter); + ReadLedgerCommand.ReadLedgerFlags flags = new ReadLedgerCommand.ReadLedgerFlags(); + flags.bookieAddresss(bookieAddress); + flags.firstEntryId(firstEntry); + flags.forceRecovery(forceRecovery); + flags.lastEntryId(lastEntry); + flags.ledgerId(ledgerId); + flags.msg(printMsg); + cmd.apply(bkConf, flags); return 0; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommand.java new file mode 100644 index 00000000000..98c1b11cf11 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommand.java @@ -0,0 +1,244 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import io.netty.buffer.ByteBufUtil; +import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.LongStream; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.LedgerEntry; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookieClient; +import org.apache.bookkeeper.proto.BookieClientImpl; +import org.apache.bookkeeper.proto.BookieProtocol; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.EntryFormatter; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to read ledger entries. + */ +public class ReadLedgerCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(ReadLedgerCommand.class); + + private static final String NAME = "readledger"; + private static final String DESC = "Read a range of entries from a ledger."; + + EntryFormatter entryFormatter; + LedgerIdFormatter ledgerIdFormatter; + + public ReadLedgerCommand() { + this(new ReadLedgerFlags()); + } + + public ReadLedgerCommand(EntryFormatter entryFormatter, LedgerIdFormatter ledgerIdFormatter) { + this(new ReadLedgerFlags()); + this.ledgerIdFormatter = ledgerIdFormatter; + this.entryFormatter = entryFormatter; + } + + private ReadLedgerCommand(ReadLedgerFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for read ledger command. + */ + @Accessors(fluent = true) + @Setter + public static class ReadLedgerFlags extends CliFlags { + + @Parameter(names = { "-m", "--msg" }, description = "Print message body") + private boolean msg; + + @Parameter(names = { "-l", "--ledgerid" }, description = "Ledger ID") + private long ledgerId = -1; + + @Parameter(names = { "-fe", "--firstentryid" }, description = "First Entry ID") + private long firstEntryId = -1; + + @Parameter(names = { "-le", "--lastentryid" }, description = "Last Entry ID") + private long lastEntryId = -1; + + @Parameter(names = { "-r", "--force-recovery" }, + description = "Ensure the ledger is properly closed before reading") + private boolean forceRecovery; + + @Parameter(names = { "-b", "--bookie" }, description = "Only read from a specific bookie") + private String bookieAddresss; + + @Parameter(names = { "-lf", "--ledgeridformatter" }, description = "Set ledger id formatter") + private String ledgerIdFormatter; + + @Parameter(names = { "-ef", "--entryformatter" }, description = "Set entry formatter") + private String entryFormatter; + } + + @Override + public boolean apply(ServerConfiguration conf, ReadLedgerFlags cmdFlags) { + if (cmdFlags.ledgerIdFormatter != null && ledgerIdFormatter == null) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(cmdFlags.ledgerIdFormatter, conf); + } else if (ledgerIdFormatter == null) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } + + if (cmdFlags.entryFormatter != null && entryFormatter == null) { + this.entryFormatter = EntryFormatter.newEntryFormatter(cmdFlags.entryFormatter, conf); + } else if (entryFormatter == null) { + this.entryFormatter = EntryFormatter.newEntryFormatter(conf); + } + + try { + return readledger(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean readledger(ServerConfiguration serverConf, ReadLedgerFlags flags) + throws InterruptedException, BKException, IOException { + + long lastEntry = flags.lastEntryId; + + final BookieSocketAddress bookie; + if (flags.bookieAddresss != null) { + // A particular bookie was specified + bookie = new BookieSocketAddress(flags.bookieAddresss); + } else { + bookie = null; + } + + ClientConfiguration conf = new ClientConfiguration(); + conf.addConfiguration(serverConf); + + try (BookKeeperAdmin bk = new BookKeeperAdmin(conf)) { + if (flags.forceRecovery) { + // Force the opening of the ledger to trigger recovery + try (LedgerHandle lh = bk.openLedger(flags.ledgerId)) { + if (lastEntry == -1 || lastEntry > lh.getLastAddConfirmed()) { + lastEntry = lh.getLastAddConfirmed(); + } + } + } + + if (bookie == null) { + // No bookie was specified, use normal bk client + Iterator entries = bk.readEntries(flags.ledgerId, flags.firstEntryId, lastEntry) + .iterator(); + while (entries.hasNext()) { + LedgerEntry entry = entries.next(); + formatEntry(entry, flags.msg); + } + } else { + // Use BookieClient to target a specific bookie + EventLoopGroup eventLoopGroup = new NioEventLoopGroup(); + OrderedExecutor executor = OrderedExecutor.newBuilder() + .numThreads(1) + .name("BookieClientScheduler") + .build(); + + ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("BookKeeperClientSchedulerPool")); + + BookieClient bookieClient = new BookieClientImpl(conf, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, + executor, scheduler, NullStatsLogger.INSTANCE); + + LongStream.range(flags.firstEntryId, lastEntry).forEach(entryId -> { + CompletableFuture future = new CompletableFuture<>(); + + bookieClient.readEntry(bookie, flags.ledgerId, entryId, + (rc, ledgerId1, entryId1, buffer, ctx) -> { + if (rc != BKException.Code.OK) { + LOG.error("Failed to read entry {} -- {}", entryId1, + BKException.getMessage(rc)); + future.completeExceptionally(BKException.create(rc)); + return; + } + + System.out.println( + "--------- Lid=" + ledgerIdFormatter.formatLedgerId(flags.ledgerId) + + ", Eid=" + entryId + " ---------"); + if (flags.msg) { + System.out.println("Data: " + ByteBufUtil.prettyHexDump(buffer)); + } + + future.complete(null); + }, null, BookieProtocol.FLAG_NONE); + + try { + future.get(); + } catch (Exception e) { + LOG.error("Error future.get while reading entries from ledger {}", flags.ledgerId, e); + } + }); + + eventLoopGroup.shutdownGracefully(); + executor.shutdown(); + bookieClient.close(); + } + } + return true; + } + + /** + * Format the entry into a readable format. + * + * @param entry + * ledgerentry to print + * @param printMsg + * Whether printing the message body + */ + private void formatEntry(LedgerEntry entry, boolean printMsg) { + long ledgerId = entry.getLedgerId(); + long entryId = entry.getEntryId(); + long entrySize = entry.getLength(); + System.out.println("--------- Lid=" + ledgerIdFormatter.formatLedgerId(ledgerId) + ", Eid=" + entryId + + ", EntrySize=" + entrySize + " ---------"); + if (printMsg) { + entryFormatter.formatEntry(entry.getEntry()); + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 2970c8c5e3a..51e579f027c 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -30,6 +30,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; @@ -60,6 +61,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new ListLedgersCommand()) .addCommand(new ConvertToInterleavedStorageCommand()) .addCommand(new ReadJournalCommand()) + .addCommand(new ReadLedgerCommand()) .addCommand(new ReadLogCommand()) .addCommand(new ReadLogMetadataCommand()) .build(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommandTest.java new file mode 100644 index 00000000000..7eb651d0472 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommandTest.java @@ -0,0 +1,169 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; + +import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.LedgerEntry; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookieClientImpl; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link ReadLedgerCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ ReadLedgerCommand.class, BookKeeperAdmin.class, BookieSocketAddress.class, ClientConfiguration.class, + LedgerHandle.class, LedgerEntry.class, OrderedExecutor.class }) +public class ReadLedgerCommandTest extends BookieCommandTestBase { + + @Mock + private BookieSocketAddress bookieSocketAddress; + + @Mock + private ClientConfiguration clientConfiguration; + + @Mock + private BookKeeperAdmin bookKeeperAdmin; + + @Mock + private LedgerHandle ledgerHandle; + + @Mock + private LedgerEntry entry; + + @Mock + private NioEventLoopGroup nioEventLoopGroup; + + @Mock + private OrderedExecutor orderedExecutor; + + @Mock + private ScheduledExecutorService scheduledExecutorService; + + @Mock + private DefaultThreadFactory defaultThreadFactory; + + @Mock + private BookieClientImpl bookieClient; + + public ReadLedgerCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.whenNew(BookieSocketAddress.class).withArguments(anyString()).thenReturn(bookieSocketAddress); + PowerMockito.whenNew(ClientConfiguration.class).withNoArguments().thenReturn(clientConfiguration); + PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class) + .withArguments(eq(clientConfiguration)).thenReturn(bookKeeperAdmin); + + when(bookKeeperAdmin.openLedger(anyLong())).thenReturn(ledgerHandle); + when(ledgerHandle.getLastAddConfirmed()).thenReturn(1L); + + List entries = new LinkedList<>(); + entries.add(entry); + when(entry.getLedgerId()).thenReturn(1L); + when(entry.getEntryId()).thenReturn(1L); + when(entry.getLength()).thenReturn(1L); + + when(bookKeeperAdmin.readEntries(anyLong(), anyLong(), anyLong())).thenReturn(entries); + + PowerMockito.whenNew(NioEventLoopGroup.class).withNoArguments().thenReturn(nioEventLoopGroup); + + PowerMockito.mockStatic(OrderedExecutor.class); + OrderedExecutor.Builder builder = mock(OrderedExecutor.Builder.class); + when(OrderedExecutor.newBuilder()).thenReturn(builder); + when(builder.numThreads(anyInt())).thenCallRealMethod(); + when(builder.name(anyString())).thenCallRealMethod(); + when(builder.build()).thenReturn(orderedExecutor); + + PowerMockito.mockStatic(Executors.class); + PowerMockito.whenNew(DefaultThreadFactory.class).withArguments(anyString()).thenReturn(defaultThreadFactory); + when(Executors.newSingleThreadScheduledExecutor(eq(defaultThreadFactory))).thenReturn(scheduledExecutorService); + + PowerMockito.whenNew(BookieClientImpl.class) + .withArguments(eq(clientConfiguration), eq(nioEventLoopGroup), eq(UnpooledByteBufAllocator.DEFAULT), + eq(orderedExecutor), eq(scheduledExecutorService), eq(NullStatsLogger.INSTANCE)) + .thenReturn(bookieClient); + + + } + + @Test + public void testWithoutBookieAddress() throws Exception { + ReadLedgerCommand cmd = new ReadLedgerCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-r" })); + verifyNew(ClientConfiguration.class, times(1)).withNoArguments(); + verify(clientConfiguration, times(1)).addConfiguration(eq(conf)); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration)); + verify(bookKeeperAdmin, times(1)).openLedger(anyLong()); + verify(ledgerHandle, times(1)).getLastAddConfirmed(); + verify(bookKeeperAdmin, times(1)).readEntries(anyLong(), anyLong(), anyLong()); + verify(entry, times(1)).getLedgerId(); + verify(entry, times(1)).getEntryId(); + verify(entry, times(1)).getLength(); + } + + @Test + public void testWithBookieAddress() throws Exception { + ReadLedgerCommand cmd = new ReadLedgerCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-b", "localhost:9000" })); + verifyNew(NioEventLoopGroup.class, times(1)).withNoArguments(); + verifyNew(DefaultThreadFactory.class, times(1)).withArguments(anyString()); + verifyNew(BookieClientImpl.class, times(1)) + .withArguments(eq(clientConfiguration), eq(nioEventLoopGroup), eq(UnpooledByteBufAllocator.DEFAULT), + eq(orderedExecutor), eq(scheduledExecutorService), eq(NullStatsLogger.INSTANCE)); + verify(nioEventLoopGroup, times(1)).shutdownGracefully(); + verify(orderedExecutor, times(1)).shutdown(); + verify(bookieClient, times(1)).close(); + } + +} From ef1d04b0af856180964a153eb9288840a5154796 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 3 Apr 2019 21:40:06 +0800 Subject: [PATCH 0263/1642] Migrate command `localconsistencycheck` Descriptions of the changes in this PR: #2042 Using bkctl run command localconsistencycheck ``` Validate Ledger Storage internal metadata Usage: bkctl bookie localconsistencycheck [flags] Flags: -h, --help Display help information ``` Reviewers: Jia Zhai , Sijie Guo This closes #2043 from zymap/command-localconsistencycheck and squashes the following commits: c31f07a00 [Sijie Guo] Merge branch 'master' into command-localconsistencycheck c391fe58d [Yong Zhang] Migrate command `readlogmetadata` 120d67737 [Yong Zhang] Migrate command `lostbookierecoverydelay` bf66235e5 [Yong Zhang] Migrate command `deleteledger` c9bb4a42c [Yong Zhang] Migrate command `localconsistencycheck` 751e55fa4 [Arvin] ISSUE #2020: close db properly to avoid open RocksDB failure at the second time 138a7ae85 [Yong Zhang] Migrate command `metadataformat` b043d1694 [Yong Zhang] Migrate command `listledgers` 4573285db [Ivan Kelly] Docker autobuild hook e3d807a32 [Like] Fix IDE complain as there are multi choices for error code 9524a9f4a [Yong Zhang] Migrate command `readjournal` 6c3f33f55 [Yong Zhang] Fix when met unexpect entry id crashed e35a108c7 [Like] Fix error message for unrecognized number-of-bookies 5902ee27b [Boyang Jerry Peng] fix potential NPE when releasing entry that is null 6aa73ce05 [Ivan Kelly] [RELEASE] Update website to include documentation for 4.8.2 1448d12aa [Yong Zhang] Migrate command `listfilesondisk` 4de598379 [Yong Zhang] Issue #1987: Migrate command `convert-to-interleaved-storage` 468743e7e [Matteo Merli] In DbLedgerStorage use default values when config key is present but empty f26a4cae0 [Ivan Kelly] Release notes for v4.8.2 ec2636cd2 [Yong Zhang] Issue #1985: Migrate command `convert-to-db-storage` 8cc7239ac [Yong Zhang] Issue #1982: Migrate command `bookiesanity` fa90f0185 [Yong Zhang] Issue #1980: Migrate command `ledger` from shell to bkctl --- .../apache/bookkeeper/bookie/BookieShell.java | 19 +---- .../bookie/LocalConsistencyCheckCommand.java | 77 +++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + .../LocalConsistencyCheckCommandTest.java | 77 +++++++++++++++++++ 4 files changed, 160 insertions(+), 15 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommandTest.java 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 98f276c75dc..9d2bba6f591 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 @@ -85,6 +85,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.LocalConsistencyCheckCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; @@ -961,21 +962,9 @@ class LocalConsistencyCheck extends MyCommand { @Override public int runCmd(CommandLine cmdLine) throws Exception { - LOG.info("=== Performing local consistency check ==="); - ServerConfiguration conf = new ServerConfiguration(bkConf); - LedgerStorage ledgerStorage = Bookie.mountLedgerStorageOffline(conf, null); - List errors = ledgerStorage.localConsistencyCheck( - java.util.Optional.empty()); - if (errors.size() > 0) { - LOG.info("=== Check returned errors: ==="); - for (LedgerStorage.DetectedInconsistency error : errors) { - LOG.error("Ledger {}, entry {}: ", error.getLedgerId(), error.getEntryId(), error.getException()); - } - return 1; - } else { - LOG.info("=== Check passed ==="); - return 0; - } + LocalConsistencyCheckCommand cmd = new LocalConsistencyCheckCommand(); + boolean result = cmd.apply(bkConf, new CliFlags()); + return (result) ? 0 : 1; } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommand.java new file mode 100644 index 00000000000..60a94bcf03b --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommand.java @@ -0,0 +1,77 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.util.List; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.LedgerStorage; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to check local storage for inconsistencies. + */ +public class LocalConsistencyCheckCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(LocalConsistencyCheckCommand.class); + + private static final String NAME = "localconsistencycheck"; + private static final String DESC = "Validate Ledger Storage internal metadata"; + + public LocalConsistencyCheckCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new CliFlags()) + .build()); + } + + @Override + public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { + try { + return check(conf); + } catch (IOException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean check(ServerConfiguration conf) throws IOException { + LOG.info("=== Performing local consistency check ==="); + ServerConfiguration serverConfiguration = new ServerConfiguration(conf); + LedgerStorage ledgerStorage = Bookie.mountLedgerStorageOffline(serverConfiguration, null); + List errors = ledgerStorage.localConsistencyCheck( + java.util.Optional.empty()); + if (errors.size() > 0) { + LOG.info("=== Check returned errors: ==="); + for (LedgerStorage.DetectedInconsistency error : errors) { + LOG.error("Ledger {}, entry {}: ", error.getLedgerId(), error.getEntryId(), error.getException()); + } + return false; + } else { + LOG.info("=== Check passed ==="); + return true; + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 51e579f027c..95cd459af11 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -29,6 +29,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.LocalConsistencyCheckCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; @@ -64,6 +65,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new ReadLedgerCommand()) .addCommand(new ReadLogCommand()) .addCommand(new ReadLogMetadataCommand()) + .addCommand(new LocalConsistencyCheckCommand()) .build(); public BookieCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommandTest.java new file mode 100644 index 00000000000..3d32f9b017e --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/LocalConsistencyCheckCommandTest.java @@ -0,0 +1,77 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.verifyNew; + +import java.util.ArrayList; +import java.util.List; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.LedgerStorage; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link LocalConsistencyCheckCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ LocalConsistencyCheckCommand.class, Bookie.class }) +public class LocalConsistencyCheckCommandTest extends BookieCommandTestBase { + + @Mock + private ServerConfiguration serverConfiguration; + + @Mock + private LedgerStorage ledgerStorage; + + public LocalConsistencyCheckCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.whenNew(ServerConfiguration.class).withArguments(eq(conf)).thenReturn(serverConfiguration); + PowerMockito.mockStatic(Bookie.class); + PowerMockito.when(Bookie.mountLedgerStorageOffline(eq(serverConfiguration), eq(null))) + .thenReturn(ledgerStorage); + List errors = new ArrayList<>(); + PowerMockito.when(ledgerStorage.localConsistencyCheck(eq(java.util.Optional.empty()))).thenReturn(errors); + } + + @Test + public void testCommand() throws Exception { + LocalConsistencyCheckCommand cmd = new LocalConsistencyCheckCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] {})); + verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(conf)); + verify(ledgerStorage, times(1)).localConsistencyCheck(eq(java.util.Optional.empty())); + } +} From 43efa29d6052419d0566b6a0d06b1156c079dd5a Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 8 Apr 2019 12:29:48 +0800 Subject: [PATCH 0264/1642] Migrate command `recover` Descriptions of the changes in this PR: #2055 Reviewers: Jia Zhai , Sijie Guo This closes #2056 from zymap/command-recover --- .../apache/bookkeeper/bookie/BookieShell.java | 165 +---------- .../cli/commands/bookies/RecoverCommand.java | 271 ++++++++++++++++++ .../bookkeeper/bookie/BookieShellTest.java | 3 +- .../cli/commands/BookiesCommandGroup.java | 2 + .../commands/bookies/RecoverCommandTest.java | 200 +++++++++++++ 5 files changed, 487 insertions(+), 154 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommandTest.java 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 9d2bba6f591..44469037420 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 @@ -44,21 +44,16 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp; -import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.BKException.MetaStoreException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.LedgerEntry; @@ -67,7 +62,6 @@ import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; @@ -97,6 +91,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.NukeExistingClusterCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.NukeExistingClusterCommand.NukeExistingClusterFlags; +import org.apache.bookkeeper.tools.cli.commands.bookies.RecoverCommand; import org.apache.bookkeeper.tools.cli.commands.client.DeleteLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; @@ -530,154 +525,18 @@ int runCmd(CommandLine cmdLine) throws Exception { Long ledgerId = getOptionLedgerIdValue(cmdLine, "ledger", -1); - // Get bookies list - final String[] bookieStrs = args[0].split(","); - final Set bookieAddrs = new HashSet<>(); - for (String bookieStr : bookieStrs) { - final String bookieStrParts[] = bookieStr.split(":"); - if (bookieStrParts.length != 2) { - System.err.println("BookieSrcs has invalid bookie address format (host:port expected) : " - + bookieStr); - return -1; - } - bookieAddrs.add(new BookieSocketAddress(bookieStrParts[0], - Integer.parseInt(bookieStrParts[1]))); - } - - if (!force) { - System.err.println("Bookies : " + bookieAddrs); - if (!IOUtils.confirmPrompt("Are you sure to recover them : (Y/N)")) { - System.err.println("Give up!"); - return -1; - } - } - - LOG.info("Constructing admin"); - ClientConfiguration adminConf = new ClientConfiguration(bkConf); - BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); - LOG.info("Construct admin : {}", admin); - try { - if (query) { - return bkQuery(admin, bookieAddrs); - } - if (-1 != ledgerId) { - return bkRecoveryLedger(admin, ledgerId, bookieAddrs, dryrun, skipOpenLedgers, removeCookies); - } - return bkRecovery(admin, bookieAddrs, dryrun, skipOpenLedgers, removeCookies); - } finally { - admin.close(); - } - } - - private int bkQuery(BookKeeperAdmin bkAdmin, Set bookieAddrs) - throws InterruptedException, BKException { - SortedMap ledgersContainBookies = - bkAdmin.getLedgersContainBookies(bookieAddrs); - System.err.println("NOTE: Bookies in inspection list are marked with '*'."); - for (Map.Entry ledger : ledgersContainBookies.entrySet()) { - System.out.println("ledger " + ledger.getKey() + " : " + ledger.getValue().getState()); - Map numBookiesToReplacePerEnsemble = - inspectLedger(ledger.getValue(), bookieAddrs); - System.out.print("summary: ["); - for (Map.Entry entry : numBookiesToReplacePerEnsemble.entrySet()) { - System.out.print(entry.getKey() + "=" + entry.getValue() + ", "); - } - System.out.println("]"); - System.out.println(); - } - System.err.println("Done"); - return 0; - } - - private Map inspectLedger(LedgerMetadata metadata, Set bookiesToInspect) { - Map numBookiesToReplacePerEnsemble = new TreeMap(); - for (Map.Entry> ensemble : - metadata.getAllEnsembles().entrySet()) { - List bookieList = ensemble.getValue(); - System.out.print(ensemble.getKey() + ":\t"); - int numBookiesToReplace = 0; - for (BookieSocketAddress bookie : bookieList) { - System.out.print(bookie); - if (bookiesToInspect.contains(bookie)) { - System.out.print("*"); - ++numBookiesToReplace; - } else { - System.out.print(" "); - } - System.out.print(" "); - } - System.out.println(); - numBookiesToReplacePerEnsemble.put(ensemble.getKey(), numBookiesToReplace); - } - return numBookiesToReplacePerEnsemble; - } - - private int bkRecoveryLedger(BookKeeperAdmin bkAdmin, - long lid, - Set bookieAddrs, - boolean dryrun, - boolean skipOpenLedgers, - boolean removeCookies) - throws InterruptedException, BKException, KeeperException { - bkAdmin.recoverBookieData(lid, bookieAddrs, dryrun, skipOpenLedgers); - if (removeCookies) { - deleteCookies(bkAdmin.getConf(), bookieAddrs); - } - return 0; - } - - private int bkRecovery(BookKeeperAdmin bkAdmin, - Set bookieAddrs, - boolean dryrun, - boolean skipOpenLedgers, - boolean removeCookies) - throws InterruptedException, BKException, KeeperException { - bkAdmin.recoverBookieData(bookieAddrs, dryrun, skipOpenLedgers); - if (removeCookies) { - deleteCookies(bkAdmin.getConf(), bookieAddrs); - } - return 0; - } - - private void deleteCookies(ClientConfiguration conf, - Set bookieAddrs) throws BKException { - ServerConfiguration serverConf = new ServerConfiguration(conf); - try { - runFunctionWithRegistrationManager(serverConf, rm -> { - try { - for (BookieSocketAddress addr : bookieAddrs) { - deleteCookie(rm, addr); - } - } catch (Exception e) { - throw new UncheckedExecutionException(e); - } - return null; - }); - } catch (Exception e) { - Throwable cause = e; - if (e instanceof UncheckedExecutionException) { - cause = e.getCause(); - } - if (cause instanceof BKException) { - throw (BKException) cause; - } else { - BKException bke = new MetaStoreException(); - bke.initCause(bke); - throw bke; - } - } - } - - private void deleteCookie(RegistrationManager rm, - BookieSocketAddress bookieSrc) throws BookieException { - try { - 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); - } + RecoverCommand cmd = new RecoverCommand(); + RecoverCommand.RecoverFlags flags = new RecoverCommand.RecoverFlags(); + flags.bookieAddress(args[0]); + flags.deleteCookie(removeCookies); + flags.dryRun(dryrun); + flags.force(force); + flags.ledger(ledgerId); + flags.skipOpenLedgers(skipOpenLedgers); + flags.query(query); + boolean result = cmd.apply(bkConf, flags); + return (result) ? 0 : -1; } - } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java new file mode 100644 index 00000000000..ba4acf53113 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java @@ -0,0 +1,271 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.versioning.Versioned; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to ledger data recovery for failed bookie. + */ +public class RecoverCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(RecoverCommand.class); + + private static final String NAME = "recover"; + private static final String DESC = "Recover the ledger data for failed bookie"; + + private static final long DEFAULT_ID = -1L; + + public RecoverCommand() { + this(new RecoverFlags()); + } + + private RecoverCommand(RecoverFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for recover command. + */ + @Accessors(fluent = true) + @Setter + public static class RecoverFlags extends CliFlags{ + + @Parameter(names = { "-l", "--ledger" }, description = "Recover a specific ledger") + private long ledger = DEFAULT_ID; + + @Parameter(names = { "-f", "--force" }, description = "Force recovery without confirmation") + private boolean force; + + @Parameter(names = { "-q", "--query" }, description = "Query the ledgers that contain given bookies") + private boolean query; + + @Parameter(names = { "-dr", "--drarun" }, description = "Printing the recovery plan w/o doing actual recovery") + private boolean dryRun; + + @Parameter(names = {"-sk", "--skipopenledgers"}, description = "Skip recovering open ledgers") + private boolean skipOpenLedgers; + + @Parameter(names = { "-d", "--deletecookie" }, description = "Delete cookie node for the bookie") + private boolean deleteCookie; + + @Parameter(names = { "-bs", "--bokiesrc" }, description = "Bookie address") + private String bookieAddress; + } + + @Override + public boolean apply(ServerConfiguration conf, RecoverFlags cmdFlags) { + try { + return recover(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean recover(ServerConfiguration conf, RecoverFlags flags) + throws IOException, BKException, InterruptedException, KeeperException { + boolean query = flags.query; + boolean dryrun = flags.dryRun; + boolean force = flags.force; + boolean skipOpenLedgers = flags.skipOpenLedgers; + boolean removeCookies = !dryrun && flags.deleteCookie; + + Long ledgerId = flags.ledger; + + // Get bookies list + final String[] bookieStrs = flags.bookieAddress.split(","); + final Set bookieAddrs = new HashSet<>(); + for (String bookieStr : bookieStrs) { + final String bookieStrParts[] = bookieStr.split(":"); + if (bookieStrParts.length != 2) { + System.err.println("BookieSrcs has invalid bookie address format (host:port expected) : " + + bookieStr); + return false; + } + bookieAddrs.add(new BookieSocketAddress(bookieStrParts[0], + Integer.parseInt(bookieStrParts[1]))); + } + + if (!force) { + System.err.println("Bookies : " + bookieAddrs); + if (!IOUtils.confirmPrompt("Are you sure to recover them : (Y/N)")) { + System.err.println("Give up!"); + return false; + } + } + + LOG.info("Constructing admin"); + ClientConfiguration adminConf = new ClientConfiguration(conf); + BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); + LOG.info("Construct admin : {}", admin); + try { + if (query) { + return bkQuery(admin, bookieAddrs); + } + if (DEFAULT_ID != ledgerId) { + return bkRecoveryLedger(admin, ledgerId, bookieAddrs, dryrun, skipOpenLedgers, removeCookies); + } + return bkRecovery(admin, bookieAddrs, dryrun, skipOpenLedgers, removeCookies); + } finally { + admin.close(); + } + } + + private boolean bkQuery(BookKeeperAdmin bkAdmin, Set bookieAddrs) + throws InterruptedException, BKException { + SortedMap ledgersContainBookies = + bkAdmin.getLedgersContainBookies(bookieAddrs); + System.err.println("NOTE: Bookies in inspection list are marked with '*'."); + for (Map.Entry ledger : ledgersContainBookies.entrySet()) { + System.out.println("ledger " + ledger.getKey() + " : " + ledger.getValue().getState()); + Map numBookiesToReplacePerEnsemble = + inspectLedger(ledger.getValue(), bookieAddrs); + System.out.print("summary: ["); + for (Map.Entry entry : numBookiesToReplacePerEnsemble.entrySet()) { + System.out.print(entry.getKey() + "=" + entry.getValue() + ", "); + } + System.out.println("]"); + System.out.println(); + } + System.err.println("Done"); + return true; + } + + private Map inspectLedger(LedgerMetadata metadata, Set bookiesToInspect) { + Map numBookiesToReplacePerEnsemble = new TreeMap(); + for (Map.Entry> ensemble : + metadata.getAllEnsembles().entrySet()) { + List bookieList = ensemble.getValue(); + System.out.print(ensemble.getKey() + ":\t"); + int numBookiesToReplace = 0; + for (BookieSocketAddress bookie : bookieList) { + System.out.print(bookie); + if (bookiesToInspect.contains(bookie)) { + System.out.print("*"); + ++numBookiesToReplace; + } else { + System.out.print(" "); + } + System.out.print(" "); + } + System.out.println(); + numBookiesToReplacePerEnsemble.put(ensemble.getKey(), numBookiesToReplace); + } + return numBookiesToReplacePerEnsemble; + } + + private boolean bkRecoveryLedger(BookKeeperAdmin bkAdmin, + long lid, + Set bookieAddrs, + boolean dryrun, + boolean skipOpenLedgers, + boolean removeCookies) + throws InterruptedException, BKException { + bkAdmin.recoverBookieData(lid, bookieAddrs, dryrun, skipOpenLedgers); + if (removeCookies) { + deleteCookies(bkAdmin.getConf(), bookieAddrs); + } + return true; + } + + private void deleteCookies(ClientConfiguration conf, + Set bookieAddrs) throws BKException { + ServerConfiguration serverConf = new ServerConfiguration(conf); + try { + runFunctionWithRegistrationManager(serverConf, rm -> { + try { + for (BookieSocketAddress addr : bookieAddrs) { + deleteCookie(rm, addr); + } + } catch (Exception e) { + throw new UncheckedExecutionException(e); + } + return null; + }); + } catch (Exception e) { + Throwable cause = e; + if (e instanceof UncheckedExecutionException) { + cause = e.getCause(); + } + if (cause instanceof BKException) { + throw (BKException) cause; + } else { + BKException bke = new BKException.MetaStoreException(); + bke.initCause(bke); + throw bke; + } + } + + } + + private void deleteCookie(RegistrationManager rm, BookieSocketAddress bookieSrc) throws BookieException { + try { + Versioned cookie = Cookie.readFromRegistrationManager(rm, bookieSrc); + cookie.getValue().deleteFromRegistrationManager(rm, bookieSrc, cookie.getVersion()); + } catch (BookieException.CookieNotFoundException nne) { + LOG.warn("No cookie to remove for {} : ", bookieSrc, nne); + } + } + + private boolean bkRecovery(BookKeeperAdmin bkAdmin, + Set bookieAddrs, + boolean dryrun, + boolean skipOpenLedgers, + boolean removeCookies) + throws InterruptedException, BKException { + bkAdmin.recoverBookieData(bookieAddrs, dryrun, skipOpenLedgers); + if (removeCookies) { + deleteCookies(bkAdmin.getConf(), bookieAddrs); + } + return true; + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java index 386746651ab..9fcb2fb546a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java @@ -51,6 +51,7 @@ import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.RecoverCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.util.EntryFormatter; @@ -73,7 +74,7 @@ * Unit test for {@link BookieShell}. */ @RunWith(PowerMockRunner.class) -@PrepareForTest({ BookieShell.class, MetadataDrivers.class }) +@PrepareForTest({ BookieShell.class, MetadataDrivers.class, RecoverCommand.class }) public class BookieShellTest { private ClientConfiguration clientConf; diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java index 1f4d4cf1f79..cb161f59782 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java @@ -27,6 +27,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.NukeExistingClusterCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.RecoverCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -50,6 +51,7 @@ public class BookiesCommandGroup extends CliCommandGroup { .addCommand(new MetaFormatCommand()) .addCommand(new DecommissionCommand()) .addCommand(new InitCommand()) + .addCommand(new RecoverCommand()) .build(); public BookiesCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommandTest.java new file mode 100644 index 00000000000..12f336b7950 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommandTest.java @@ -0,0 +1,200 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.verify; +import static org.mockito.internal.verification.VerificationModeFactory.times; +import static org.powermock.api.mockito.PowerMockito.doNothing; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; + +import com.google.common.collect.ImmutableList; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.function.Function; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link RecoverCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ RecoverCommand.class, MetadataDrivers.class, Cookie.class }) +public class RecoverCommandTest extends BookieCommandTestBase { + + @Mock + private BookieSocketAddress bookieSocketAddress; + + @Mock + private ClientConfiguration clientConfiguration; + + @Mock + private BookKeeperAdmin bookKeeperAdmin; + + @Mock + private LedgerMetadata ledgerMetadata; + + @Mock + private ServerConfiguration serverConfiguration; + + @Mock + private RegistrationManager registrationManager; + + @Mock + private Versioned cookieVersioned; + + public RecoverCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(clientConfiguration)).thenReturn(conf); + PowerMockito.whenNew(BookieSocketAddress.class).withArguments(anyString(), anyInt()) + .thenReturn(bookieSocketAddress); + PowerMockito.whenNew(ClientConfiguration.class).withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(conf)).thenReturn(clientConfiguration); + PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class) + .withArguments(eq(clientConfiguration)).thenReturn(bookKeeperAdmin); + + mockBkQuery(); + mockDeleteCookie(); + mockDeleteCookies(); + mockBkRecovery(); + + } + + private void mockBkQuery() throws BKException, InterruptedException { + SortedMap ledgerMetadataSortedMap = new TreeMap<>(); + ledgerMetadataSortedMap.put(1L, ledgerMetadata); + when(bookKeeperAdmin.getLedgersContainBookies(any())).thenReturn(ledgerMetadataSortedMap); + ArrayList arrayList = new ArrayList<>(); + arrayList.add(bookieSocketAddress); + Map> map = new HashMap<>(); + map.put(1L, arrayList); + NavigableMap> navigableMap = Collections.unmodifiableNavigableMap( + map.entrySet().stream() + .collect(TreeMap::new, (m, e) -> m.put(e.getKey(), ImmutableList.copyOf(e.getValue())), + TreeMap::putAll)); + doReturn(navigableMap).when(ledgerMetadata).getAllEnsembles(); + } + + + + private void mockDeleteCookies() throws Exception { + PowerMockito.mockStatic(MetadataDrivers.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function function = invocationOnMock.getArgument(1); + function.apply(registrationManager); + return null; + }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", any(ServerConfiguration.class), + any(Function.class)); + } + + private void mockDeleteCookie() throws BookieException { + PowerMockito.mockStatic(Cookie.class); + when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress))) + .thenReturn(cookieVersioned); + Cookie cookie = mock(Cookie.class); + when(cookieVersioned.getValue()).thenReturn(cookie); + Version version = mock(Version.class); + when(cookieVersioned.getVersion()).thenReturn(version); + doNothing().when(cookie) + .deleteFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress), eq(version)); + } + + private void mockBkRecovery() throws BKException, InterruptedException { + doNothing().when(bookKeeperAdmin).recoverBookieData(any(), anyBoolean(), anyBoolean()); + when(bookKeeperAdmin.getConf()).thenReturn(clientConfiguration); + } + + @Test + public void testBookieListCheck() { + RecoverCommand cmd = new RecoverCommand(); + Assert.assertFalse(cmd.apply(bkFlags, new String[] { "-bs", "127.0.0.1:8000,8001" })); + } + + @Test + public void testQuery() throws Exception { + RecoverCommand cmd = new RecoverCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-q", "-bs", "127.0.0.1:8000", "-f" })); + verifyNew(ClientConfiguration.class, times(1)).withArguments(eq(conf)); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration)); + verify(bookKeeperAdmin, times(1)).getLedgersContainBookies(any()); + } + + @Test + public void testLedgerId() throws Exception { + RecoverCommand cmd = new RecoverCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-bs", "127.0.0.1:8000", "-f", "-l", "1" })); + verifyNew(ClientConfiguration.class, times(1)).withArguments(eq(conf)); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration)); + verify(bookKeeperAdmin, times(1)) + .recoverBookieData(anyLong(), any(), anyBoolean(), anyBoolean()); + } + + @Test + public void testWithLedgerIdAndRemoveCookies() throws Exception { + RecoverCommand cmd = new RecoverCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-bs", "127.0.0.1:8000", "-f", "-l", "1", "-d" })); + verifyNew(ClientConfiguration.class, times(1)).withArguments(eq(conf)); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration)); + verify(bookKeeperAdmin, times(1)).recoverBookieData(anyLong(), any(), anyBoolean(), anyBoolean()); + verify(bookKeeperAdmin, times(1)).getConf(); + verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(clientConfiguration)); + verify(cookieVersioned, times(1)).getValue(); + } +} From e626211429947cbaa9665f723ea4f4ed99cd52fa Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 8 Apr 2019 22:15:35 +0200 Subject: [PATCH 0265/1642] Release notes for 4.9.1 Descriptions of the changes in this PR: - Release notes for Apache BookKeeper 4.9.1 - Add Apache BookKeeper docs for 4.9.1 to the website Reviewers: Sijie Guo This closes #1992 from eolivelli/fix/releasenotest491 --- site/_config.yml | 3 +- site/docs/4.9.1/admin/autorecovery.md | 128 +++ site/docs/4.9.1/admin/bookies.md | 180 ++++ site/docs/4.9.1/admin/geo-replication.md | 22 + site/docs/4.9.1/admin/http.md | 457 ++++++++++ site/docs/4.9.1/admin/metrics.md | 41 + site/docs/4.9.1/admin/perf.md | 3 + site/docs/4.9.1/admin/placement.md | 3 + site/docs/4.9.1/admin/upgrade.md | 175 ++++ site/docs/4.9.1/api/distributedlog-api.md | 395 ++++++++ site/docs/4.9.1/api/ledger-adv-api.md | 111 +++ site/docs/4.9.1/api/ledger-api.md | 841 ++++++++++++++++++ site/docs/4.9.1/api/overview.md | 17 + site/docs/4.9.1/deployment/dcos.md | 142 +++ site/docs/4.9.1/deployment/kubernetes.md | 181 ++++ site/docs/4.9.1/deployment/manual.md | 56 ++ site/docs/4.9.1/development/codebase.md | 3 + site/docs/4.9.1/development/protocol.md | 148 +++ site/docs/4.9.1/getting-started/concepts.md | 202 +++++ .../4.9.1/getting-started/installation.md | 82 ++ .../docs/4.9.1/getting-started/run-locally.md | 16 + site/docs/4.9.1/overview/overview.md | 60 ++ site/docs/4.9.1/overview/releaseNotes.md | 24 + .../4.9.1/overview/releaseNotesTemplate.md | 17 + site/docs/4.9.1/reference/cli.md | 10 + site/docs/4.9.1/reference/config.md | 9 + site/docs/4.9.1/reference/metrics.md | 3 + site/docs/4.9.1/security/overview.md | 21 + site/docs/4.9.1/security/sasl.md | 202 +++++ site/docs/4.9.1/security/tls.md | 210 +++++ site/docs/4.9.1/security/zookeeper.md | 41 + site/releases.md | 8 + 32 files changed, 3810 insertions(+), 1 deletion(-) create mode 100644 site/docs/4.9.1/admin/autorecovery.md create mode 100644 site/docs/4.9.1/admin/bookies.md create mode 100644 site/docs/4.9.1/admin/geo-replication.md create mode 100644 site/docs/4.9.1/admin/http.md create mode 100644 site/docs/4.9.1/admin/metrics.md create mode 100644 site/docs/4.9.1/admin/perf.md create mode 100644 site/docs/4.9.1/admin/placement.md create mode 100644 site/docs/4.9.1/admin/upgrade.md create mode 100644 site/docs/4.9.1/api/distributedlog-api.md create mode 100644 site/docs/4.9.1/api/ledger-adv-api.md create mode 100644 site/docs/4.9.1/api/ledger-api.md create mode 100644 site/docs/4.9.1/api/overview.md create mode 100644 site/docs/4.9.1/deployment/dcos.md create mode 100644 site/docs/4.9.1/deployment/kubernetes.md create mode 100644 site/docs/4.9.1/deployment/manual.md create mode 100644 site/docs/4.9.1/development/codebase.md create mode 100644 site/docs/4.9.1/development/protocol.md create mode 100644 site/docs/4.9.1/getting-started/concepts.md create mode 100644 site/docs/4.9.1/getting-started/installation.md create mode 100644 site/docs/4.9.1/getting-started/run-locally.md create mode 100644 site/docs/4.9.1/overview/overview.md create mode 100644 site/docs/4.9.1/overview/releaseNotes.md create mode 100644 site/docs/4.9.1/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.9.1/reference/cli.md create mode 100644 site/docs/4.9.1/reference/config.md create mode 100644 site/docs/4.9.1/reference/metrics.md create mode 100644 site/docs/4.9.1/security/overview.md create mode 100644 site/docs/4.9.1/security/sasl.md create mode 100644 site/docs/4.9.1/security/tls.md create mode 100644 site/docs/4.9.1/security/zookeeper.md diff --git a/site/_config.yml b/site/_config.yml index 8227fc1c615..5097719c125 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -8,6 +8,7 @@ destination: local-generated twitter_url: https://twitter.com/asfbookkeeper versions: +- "4.9.1" # [next_version_placeholder] - "4.9.0" - "4.8.2" @@ -36,7 +37,7 @@ archived_versions: - "4.1.0" - "4.0.0" latest_version: "4.10.0-SNAPSHOT" -latest_release: "4.9.0" +latest_release: "4.9.1" stable_release: "4.8.2" distributedlog_version: "4.8.2" diff --git a/site/docs/4.9.1/admin/autorecovery.md b/site/docs/4.9.1/admin/autorecovery.md new file mode 100644 index 00000000000..b1dd078f9b2 --- /dev/null +++ b/site/docs/4.9.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: + +* the `shell recover` option +* 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 shell recover \ + 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 shell recover \ + 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.9.1/admin/bookies.md b/site/docs/4.9.1/admin/bookies.md new file mode 100644 index 00000000000..1b0427dae3c --- /dev/null +++ b/site/docs/4.9.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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + \ + \ + + ``` + + 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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + 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.9.1/admin/geo-replication.md b/site/docs/4.9.1/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.9.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.9.1/admin/http.md b/site/docs/4.9.1/admin/http.md new file mode 100644 index 00000000000..270dcb3a16f --- /dev/null +++ b/site/docs/4.9.1/admin/http.md @@ -0,0 +1,457 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 4 components: +1. Heartbeat: heartbeat for a specific bookie. +1. Config: doing the server configuration for a specific bookie. +1. Ledger: HTTP endpoints related to ledgers. +1. Bookie: HTTP endpoints related to bookies. +1. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Config + +### Endpoint: /metrics +1. Method: GET + * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/bookie/gc +1. Method: PUT + * Description: trigger gc for this bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: GET + * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + { + "is_in_force_gc" : "false" + } + ``` + +### Endpoint: /api/v1/bookie/gc_details +1. Method: GET + * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + [ { + "forceCompacting" : false, + "majorCompacting" : false, + "minorCompacting" : false, + "lastMajorCompactionTime" : 1544578144944, + "lastMinorCompactionTime" : 1544578144944, + "majorCompactionCounter" : 1, + "minorCompactionCounter" : 0 + } ] + ``` + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.9.1/admin/metrics.md b/site/docs/4.9.1/admin/metrics.md new file mode 100644 index 00000000000..142df3dcd2d --- /dev/null +++ b/site/docs/4.9.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.prometheus.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.9.1/admin/perf.md b/site/docs/4.9.1/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.9.1/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.9.1/admin/placement.md b/site/docs/4.9.1/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.9.1/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.9.1/admin/upgrade.md b/site/docs/4.9.1/admin/upgrade.md new file mode 100644 index 00000000000..57c65208131 --- /dev/null +++ b/site/docs/4.9.1/admin/upgrade.md @@ -0,0 +1,175 @@ +--- +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.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.7.x to 4.8.X upgrade + +In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.9.1/api/distributedlog-api.md b/site/docs/4.9.1/api/distributedlog-api.md new file mode 100644 index 00000000000..cce2cdb9f45 --- /dev/null +++ b/site/docs/4.9.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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.9.1/api/ledger-adv-api.md b/site/docs/4.9.1/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.9.1/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.9.1/api/ledger-api.md b/site/docs/4.9.1/api/ledger-api.md new file mode 100644 index 00000000000..08984696f91 --- /dev/null +++ b/site/docs/4.9.1/api/ledger-api.md @@ -0,0 +1,841 @@ +--- +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.9.1 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.9.1 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.9.1' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.9.1' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` + +### Relaxing Durability + +In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. +In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will +be always readable by other clients. + +On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after +writing the entry to SO buffers without waiting for an fsync. +In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. +Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. + +In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble ackknowledge the call after +performing an fsync to the disk which is storing the journal. +This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. + +The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. +In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. + + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete + + +wh.force().get(); // wait for fsync, make data available to readers and to the replicator + +wh.close(); // seal the ledger + +``` diff --git a/site/docs/4.9.1/api/overview.md b/site/docs/4.9.1/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.9.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.9.1/deployment/dcos.md b/site/docs/4.9.1/deployment/dcos.md new file mode 100644 index 00000000000..9b91dd5f209 --- /dev/null +++ b/site/docs/4.9.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.9.1] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.9.1/deployment/kubernetes.md b/site/docs/4.9.1/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.9.1/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.9.1/deployment/manual.md b/site/docs/4.9.1/deployment/manual.md new file mode 100644 index 00000000000..daafd5556f5 --- /dev/null +++ b/site/docs/4.9.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.9.1/development/codebase.md b/site/docs/4.9.1/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.9.1/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.9.1/development/protocol.md b/site/docs/4.9.1/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.9.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.9.1/getting-started/concepts.md b/site/docs/4.9.1/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.9.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.9.1/getting-started/installation.md b/site/docs/4.9.1/getting-started/installation.md new file mode 100644 index 00000000000..7e92b471581 --- /dev/null +++ b/site/docs/4.9.1/getting-started/installation.md @@ -0,0 +1,82 @@ +--- +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 +``` + +Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. + +```shell +$ mvn package -Dstream +``` + +> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin + +> You can enable `table service` by adding the `-Dstream` flag when running above commands. + +## 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.9.1/getting-started/run-locally.md b/site/docs/4.9.1/getting-started/run-locally.md new file mode 100644 index 00000000000..edbfab9fda6 --- /dev/null +++ b/site/docs/4.9.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 +$ bookkeeper-server/bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.9.1/overview/overview.md b/site/docs/4.9.1/overview/overview.md new file mode 100644 index 00000000000..291e26f688b --- /dev/null +++ b/site/docs/4.9.1/overview/overview.md @@ -0,0 +1,60 @@ +--- +title: Apache BookKeeper™ 4.9.1 +--- + + +This documentation is for Apache BookKeeper™ version 4.9.1. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.9.1 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.9.1/overview/releaseNotes.md b/site/docs/4.9.1/overview/releaseNotes.md new file mode 100644 index 00000000000..b97ecb9aac7 --- /dev/null +++ b/site/docs/4.9.1/overview/releaseNotes.md @@ -0,0 +1,24 @@ +--- +title: Apache BookKeeper 4.9.1 Release Notes +--- + +This is the 18th release of Apache BookKeeper! + +The 4.9.1 release incorporates a few critical bug fixes, since previous major release, 4.9.0. + +Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.9.1](../../admin/upgrade). The technical details of +this release are summarized below. + +### Dependencies Changes + +No dependency change. + +### Bug Fixes + +- [Issue #1973: [DLOG] Avoid double read in readahead](https://github.com/apache/bookkeeper/pull/1973) +- [Issue #1952: Filter empty string for networkTopologyScriptFileName](https://github.com/apache/bookkeeper/pull/1952) +- [Issue #1950: putEntryOffset translate FileInfoDeletedException](https://github.com/apache/bookkeeper/pull/1950) + +## Full list of changes + +- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.1+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.1+is%3Aclosed) diff --git a/site/docs/4.9.1/overview/releaseNotesTemplate.md b/site/docs/4.9.1/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..3f7a0398638 --- /dev/null +++ b/site/docs/4.9.1/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.9.1 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.9.1. The technical details of this release are summarized +below. + +## Highlights + +[List the highlights] + +## Details + +[list to issues list] + diff --git a/site/docs/4.9.1/reference/cli.md b/site/docs/4.9.1/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.9.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.9.1/reference/config.md b/site/docs/4.9.1/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.9.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.9.1/reference/metrics.md b/site/docs/4.9.1/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.9.1/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.9.1/security/overview.md b/site/docs/4.9.1/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.9.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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.9.1/security/sasl.md b/site/docs/4.9.1/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.9.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.9.1/security/tls.md b/site/docs/4.9.1/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.9.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.9.1/security/zookeeper.md b/site/docs/4.9.1/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.9.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) diff --git a/site/releases.md b/site/releases.md index 447e2e19896..1f25b2a08b4 100644 --- a/site/releases.md +++ b/site/releases.md @@ -64,6 +64,14 @@ Client Guide | API docs ## News +### 7 April, 2019 Release 4.9.1 available + +This is the 18th release of Apache BookKeeper ! + +The 4.9.1 release is a bugfix release which fixes a couple of issues reported from users of 4.9.0. + +See [BookKeeper 4.9.1 Release Notes](../docs/4.9.1/overview/releaseNotes) for details. + ### 19 March, 2019 Release 4.8.2 available This is the 17th release of Apache BookKeeper! From 006a25efb1702e8c013bd1c2738e5cef5625595a Mon Sep 17 00:00:00 2001 From: "Dongfa,Huang" Date: Tue, 9 Apr 2019 16:16:44 +0800 Subject: [PATCH 0266/1642] Avoid useless verify if LedgerEntryRequest completed MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Avoid useless verify if LedgerEntryRequest completed Change-Id: Ifda2a6e218c49105a5627be69566ea2ce4a57699 Descriptions of the changes in this PR: Print misleading logs when the SpeculativeRequestExecutionPolicy is turned on: 2019-04-03 18:30:49,839 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: -1 , actual: 602 2019-04-03 18:30:49,839 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: -1 , actual: 606 2019-04-03 18:30:49,839 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: -1 , actual: 610 2019-04-03 18:30:49,839 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: -1 , actual: 614 2019-04-03 18:30:49,843 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 644 , actual: 622 2019-04-03 18:30:49,843 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 640 , actual: 626 2019-04-03 18:30:49,843 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 656 , actual: 630 2019-04-03 18:30:49,843 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 652 , actual: 634 2019-04-03 18:30:49,843 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 648 , actual: 638 2019-04-03 18:30:49,846 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 660 , actual: 642 2019-04-03 18:30:49,846 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 675 , actual: 646 2019-04-03 18:30:49,846 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 671 , actual: 650 2019-04-03 18:30:49,846 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 667 , actual: 654 2019-04-03 18:30:49,846 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 663 , actual: 658 2019-04-03 18:30:49,848 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: -1 , actual: 670 2019-04-03 18:30:49,849 ERROR org.apache.bookkeeper.client.DigestManager: Entry-id mismatch in authenticated message, expected: 692 , actual: 662 ### Motivation Avoid useless verify and redundant logging(caused by the use of recycled `entryImpl`) if LedgerEntryRequest completed ### Changes Return immediately if completed Master Issue: # Reviewers: Sijie Guo This closes #2061 from huangdongfa/master --- .../main/java/org/apache/bookkeeper/client/PendingReadOp.java | 3 +++ 1 file changed, 3 insertions(+) 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 65a3d76f9ab..22c0fc15b38 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 @@ -131,6 +131,9 @@ public void close() { */ boolean complete(int bookieIndex, BookieSocketAddress host, final ByteBuf buffer) { ByteBuf content; + if (isComplete()) { + return false; + } try { content = lh.macManager.verifyDigestAndReturnData(eId, buffer); } catch (BKDigestMatchException e) { From fee768ece708e6b4fdbb0335a00e6dfb97f7e2c9 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 9 Apr 2019 20:00:26 +0800 Subject: [PATCH 0267/1642] Migrate command `regenerate-interleaved-storage-index-file` Descriptions of the changes in this PR: #2033 ``` Regenerate an interleaved storage index file, from available entrylogger files. Usage: bkctl bookie regenerate-interleaved-storage-index-file [flags] Flags: -b, --b64password The password in base64 encoding, for cases where the password is not UTF-8. -d, --dryrun Process the entryLogger, but don't write anthing. -l, --ledgerids Ledger(s) whose index needs to be regenerated. Multiple can be specified, comma separated. -p, --password The bookie stores the password in the index file, so we need it to regenerate.This must match the value in the ledger metadata. -h, --help Display help information ``` Reviewers: Sijie Guo This closes #2035 from zymap/command-RISIF --- .../apache/bookkeeper/bookie/BookieShell.java | 29 ++-- ...ateInterleavedStorageIndexFileCommand.java | 133 ++++++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + ...nterleavedStorageIndexFileCommandTest.java | 85 +++++++++++ 4 files changed, 232 insertions(+), 17 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/RegenerateInterleavedStorageIndexFileCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/RegenerateInterleavedStorageIndexFileCommandTest.java 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 44469037420..4de4aa7c12b 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 @@ -18,7 +18,6 @@ package org.apache.bookkeeper.bookie; -import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithMetadataBookieDriver; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; @@ -39,7 +38,6 @@ import java.nio.file.attribute.FileTime; import java.util.ArrayList; import java.util.Arrays; -import java.util.Base64; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -48,7 +46,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; @@ -84,6 +81,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.RegenerateInterleavedStorageIndexFileCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.DecommissionCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; @@ -2173,23 +2171,20 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { - byte[] password; + RegenerateInterleavedStorageIndexFileCommand cmd = new RegenerateInterleavedStorageIndexFileCommand(); + RegenerateInterleavedStorageIndexFileCommand.RISIFFlags + flags = new RegenerateInterleavedStorageIndexFileCommand.RISIFFlags(); + List ledgerIds = Arrays.stream(cmdLine.getOptionValues("ledgerIds")).map((id) -> Long.parseLong(id)) + .collect(Collectors.toList()); + boolean dryRun = cmdLine.hasOption("dryRun"); + flags.ledgerIds(ledgerIds); if (cmdLine.hasOption("password")) { - password = cmdLine.getOptionValue("password").getBytes(UTF_8); + flags.password(cmdLine.getOptionValue("password")); } else if (cmdLine.hasOption("b64password")) { - password = Base64.getDecoder().decode(cmdLine.getOptionValue("b64password")); - } else { - LOG.error("The password must be specified to regenerate the index file."); - return 1; + flags.b64Password(cmdLine.getOptionValue("b64password")); } - Set ledgerIds = Arrays.stream(cmdLine.getOptionValues("ledgerIds")) - .map((id) -> Long.parseLong(id)).collect(Collectors.toSet()); - boolean dryRun = cmdLine.hasOption("dryRun"); - - LOG.info("=== Rebuilding index file for {} ===", ledgerIds); - ServerConfiguration conf = new ServerConfiguration(bkConf); - new InterleavedStorageRegenerateIndexOp(conf, ledgerIds, password).initiate(dryRun); - LOG.info("-- Done rebuilding index file for {} --", ledgerIds); + flags.dryRun(dryRun); + cmd.apply(bkConf, flags); return 0; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/RegenerateInterleavedStorageIndexFileCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/RegenerateInterleavedStorageIndexFileCommand.java new file mode 100644 index 00000000000..bbf933fda86 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/RegenerateInterleavedStorageIndexFileCommand.java @@ -0,0 +1,133 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.converters.CommaParameterSplitter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.security.NoSuchAlgorithmException; +import java.util.Base64; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.InterleavedStorageRegenerateIndexOp; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to regenerate an index file for interleaved storage. + */ +public class RegenerateInterleavedStorageIndexFileCommand + extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(RegenerateInterleavedStorageIndexFileCommand.class); + + private static final String NAME = "regenerate-interleaved-storage-index-file"; + private static final String DESC = + "Regenerate an interleaved storage index file, from available entrylogger " + "files."; + private static final String DEFAULT = ""; + + public RegenerateInterleavedStorageIndexFileCommand() { + this(new RISIFFlags()); + } + + private RegenerateInterleavedStorageIndexFileCommand(RISIFFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for regenerate interleaved storage index file command. + */ + @Accessors(fluent = true) + @Setter + public static class RISIFFlags extends CliFlags { + + @Parameter(names = { "-p", "--password" }, + description = "The bookie stores the password in the index file, so we need it to regenerate." + + "This must match the value in the ledger metadata.") + private String password = DEFAULT; + + @Parameter(names = { "-b", "--b64password" }, + description = "The password in base64 encoding, for cases where the password is not UTF-8.") + private String b64Password = DEFAULT; + + @Parameter(names = { "-d", "--dryrun" }, description = "Process the entryLogger, but don't write anthing.") + private boolean dryRun; + + @Parameter(names = { "-l", "--ledgerids" }, + description = "Ledger(s) whose index needs to be regenerated. Multiple can be specified, comma separated.", + splitter = CommaParameterSplitter.class) + private List ledgerIds; + + } + + @Override + public boolean apply(ServerConfiguration conf, RISIFFlags cmdFlags) { + try { + return generate(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean generate(ServerConfiguration conf, RISIFFlags flags) throws NoSuchAlgorithmException, IOException { + validateFlags(flags); + byte[] password; + if (!flags.password.equals(DEFAULT)) { + password = flags.password.getBytes(StandardCharsets.UTF_8); + } else if (!flags.b64Password.equals(DEFAULT)) { + password = Base64.getDecoder().decode(flags.b64Password); + } else { + LOG.error("The password must be specified to regenerate the index file"); + return false; + } + + Set ledgerIds = flags.ledgerIds.stream().collect(Collectors.toSet()); + + LOG.info("=== Rebuilding index file for {} ===", ledgerIds); + ServerConfiguration serverConfiguration = new ServerConfiguration(conf); + InterleavedStorageRegenerateIndexOp i = new InterleavedStorageRegenerateIndexOp(serverConfiguration, ledgerIds, + password); + i.initiate(flags.dryRun); + + LOG.info("-- Done rebuilding index file for {} --", ledgerIds); + return true; + } + + private void validateFlags(RISIFFlags flags) { + if (flags.password == null) { + flags.password = DEFAULT; + } + if (flags.b64Password == null) { + flags.b64Password = DEFAULT; + } + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 95cd459af11..9e94cd28c67 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.RegenerateInterleavedStorageIndexFileCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -66,6 +67,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new ReadLogCommand()) .addCommand(new ReadLogMetadataCommand()) .addCommand(new LocalConsistencyCheckCommand()) + .addCommand(new RegenerateInterleavedStorageIndexFileCommand()) .build(); public BookieCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/RegenerateInterleavedStorageIndexFileCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/RegenerateInterleavedStorageIndexFileCommandTest.java new file mode 100644 index 00000000000..1ade885fd92 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/RegenerateInterleavedStorageIndexFileCommandTest.java @@ -0,0 +1,85 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.util.Arrays; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.bookkeeper.bookie.InterleavedStorageRegenerateIndexOp; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link RegenerateInterleavedStorageIndexFileCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ RegenerateInterleavedStorageIndexFileCommand.class, InterleavedStorageRegenerateIndexOp.class, + ServerConfiguration.class }) +public class RegenerateInterleavedStorageIndexFileCommandTest extends BookieCommandTestBase { + + @Mock + private ServerConfiguration serverConfiguration; + + @Mock + private InterleavedStorageRegenerateIndexOp op; + + public RegenerateInterleavedStorageIndexFileCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.whenNew(ServerConfiguration.class).withArguments(eq(conf)).thenReturn(serverConfiguration); + } + + @Test + public void testCommand() throws Exception { + String ledgerIds = "1,2,3"; + String password = "12345"; + Set ledgerIdsSet = Arrays.stream(ledgerIds.split(",")).map((id) -> Long.parseLong(id)) + .collect(Collectors.toSet()); + byte[] bytes = password.getBytes(); + PowerMockito.whenNew(InterleavedStorageRegenerateIndexOp.class) + .withArguments(eq(serverConfiguration), eq(ledgerIdsSet), eq(bytes)).thenReturn(op); + PowerMockito.doNothing().when(op).initiate(anyBoolean()); + + RegenerateInterleavedStorageIndexFileCommand cmd = new RegenerateInterleavedStorageIndexFileCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-p", password, "-l", ledgerIds })); + PowerMockito.verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(conf)); + PowerMockito.verifyNew(InterleavedStorageRegenerateIndexOp.class, times(1)) + .withArguments(eq(serverConfiguration), eq(ledgerIdsSet), eq(bytes)); + verify(op, times(1)).initiate(anyBoolean()); + } +} + From 80730f556f81396ec16fa026d8d377cdbe513599 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 10 Apr 2019 09:42:04 +0800 Subject: [PATCH 0268/1642] Migrate command `updateledgers` Descriptions of the changes in this PR: #2057 ``` Update bookie id in ledgers (this may take a long time). Usage: bkctl ledger update [flags] Flags: -host, --hostname Expects configuration useHostNameAsBookieID=true as the optin value -l, --limit Maximum number of ledgers of ledgers to update (default: no limit) -p, --printprogress Print messages on every configured seconds if verbose turned on (default: 10 secs) -v, --verbose Print status of the ledger updation (default: false) -s, updatepersec Number of ledgers updating per second (default: 5 per sec) -h, --help Display help information ``` Reviewers: Jia Zhai , Sijie Guo This closes #2058 from zymap/command-updateledgers --- .../apache/bookkeeper/bookie/BookieShell.java | 62 ++----- .../commands/bookie/FlipBookieIdCommand.java | 165 ++++++++++++++++++ .../commands/client/DeleteLedgerCommand.java | 2 +- .../cli/commands/BookieCommandGroup.java | 2 + .../bookie/FlipBookieIdCommandTest.java | 108 ++++++++++++ 5 files changed, 288 insertions(+), 51 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java 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 4de4aa7c12b..f3f83a35bb2 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 @@ -46,15 +46,12 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp; -import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.LedgerEntry; -import org.apache.bookkeeper.client.UpdateLedgerOp; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -70,6 +67,7 @@ import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.FlipBookieIdCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; @@ -102,7 +100,6 @@ import org.apache.bookkeeper.util.EntryFormatter; import org.apache.bookkeeper.util.IOUtils; import org.apache.bookkeeper.util.LedgerIdFormatter; -import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.Tool; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; @@ -1788,6 +1785,9 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { + FlipBookieIdCommand cmd = new FlipBookieIdCommand(); + FlipBookieIdCommand.FlipBookieIdFlags flags = new FlipBookieIdCommand.FlipBookieIdFlags(); + final String bookieId = cmdLine.getOptionValue("bookieId"); if (StringUtils.isBlank(bookieId)) { LOG.error("Invalid argument list!"); @@ -1800,23 +1800,8 @@ int runCmd(CommandLine cmdLine) throws Exception { return -1; } boolean useHostName = getOptionalValue(bookieId, "hostname"); - if (!bkConf.getUseHostNameAsBookieID() && useHostName) { - LOG.error("Expects configuration useHostNameAsBookieID=true as the option value passed is 'hostname'"); - return -1; - } else if (bkConf.getUseHostNameAsBookieID() && !useHostName) { - LOG.error("Expects configuration useHostNameAsBookieID=false as the option value passed is 'ip'"); - return -1; - } final int rate = getOptionIntValue(cmdLine, "updatespersec", 5); - if (rate <= 0) { - LOG.error("Invalid updatespersec {}, should be > 0", rate); - return -1; - } final int limit = getOptionIntValue(cmdLine, "limit", Integer.MIN_VALUE); - if (limit <= 0 && limit != Integer.MIN_VALUE) { - LOG.error("Invalid limit {}, should be > 0", limit); - return -1; - } final boolean verbose = getOptionBooleanValue(cmdLine, "verbose", false); final long printprogress; if (!verbose) { @@ -1828,37 +1813,14 @@ int runCmd(CommandLine cmdLine) throws Exception { // defaulting to 10 seconds printprogress = getOptionLongValue(cmdLine, "printprogress", 10); } - final ClientConfiguration conf = new ClientConfiguration(); - conf.addConfiguration(bkConf); - final BookKeeper bk = new BookKeeper(conf); - final BookKeeperAdmin admin = new BookKeeperAdmin(conf); - final UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, admin); - final ServerConfiguration serverConf = new ServerConfiguration(bkConf); - final BookieSocketAddress newBookieId = Bookie.getBookieAddress(serverConf); - serverConf.setUseHostNameAsBookieID(!useHostName); - final BookieSocketAddress oldBookieId = Bookie.getBookieAddress(serverConf); - - UpdateLedgerNotifier progressable = new UpdateLedgerNotifier() { - long lastReport = System.nanoTime(); - - @Override - public void progress(long updated, long issued) { - if (printprogress <= 0) { - return; // disabled - } - if (TimeUnit.MILLISECONDS.toSeconds(MathUtils.elapsedMSec(lastReport)) >= printprogress) { - LOG.info("Number of ledgers issued={}, updated={}", issued, updated); - lastReport = MathUtils.nowInNano(); - } - } - }; - try { - updateLedgerOp.updateBookieIdInLedgers(oldBookieId, newBookieId, rate, limit, progressable); - } catch (IOException e) { - LOG.error("Failed to update ledger metadata", e); - return -1; - } - return 0; + flags.hostname(useHostName); + flags.printProgress(printprogress); + flags.limit(limit); + flags.updatePerSec(rate); + flags.verbose(verbose); + + boolean result = cmd.apply(bkConf, flags); + return (result) ? 0 : -1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java new file mode 100644 index 00000000000..852379f63af --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java @@ -0,0 +1,165 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieShell; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.UpdateLedgerOp; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.MathUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to update ledger command. + */ +public class FlipBookieIdCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(FlipBookieIdCommand.class); + + private static final String NAME = "flip-bookie-id"; + private static final String DESC = "Update bookie id in ledgers (this may take a long time)."; + + public FlipBookieIdCommand() { + this(new FlipBookieIdFlags()); + } + + private FlipBookieIdCommand(FlipBookieIdFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for update ledger command. + */ + @Accessors(fluent = true) + @Setter + public static class FlipBookieIdFlags extends CliFlags { + + @Parameter(names = { "-host", "--hostname" }, + description = "Expects configuration useHostNameAsBookieID=true as the option value (default: ip address)") + private boolean hostname; + + @Parameter(names = { "-s", "--updatepersec" }, + description = "Number of ledgers updating per second (default: 5 per sec)") + private int updatePerSec = 5; + + @Parameter(names = {"-l", "--limit"}, + description = "Maximum number of ledgers of ledgers to update (default: no limit)") + private int limit = Integer.MIN_VALUE; + + @Parameter(names = { "-v", "--verbose" }, description = "Print status of the ledger updation (default: false)") + private boolean verbose; + + @Parameter(names = { "-p", "--printprogress" }, + description = "Print messages on every configured seconds if verbose turned on (default: 10 secs)") + private long printProgress = 10; + } + + @Override + public boolean apply(ServerConfiguration conf, FlipBookieIdFlags cmdFlags) { + try { + return updateLedger(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean updateLedger(ServerConfiguration conf, FlipBookieIdFlags flags) + throws InterruptedException, BKException, IOException { + + if (!conf.getUseHostNameAsBookieID() && flags.hostname) { + LOG.error("Expects configuration useHostNameAsBookieID=true as the option value"); + return false; + } else if (conf.getUseHostNameAsBookieID() && !flags.hostname) { + LOG.error("Expects configuration useHostNameAsBookieID=false as the option value'"); + return false; + } + + final int rate = flags.updatePerSec; + if (rate <= 0) { + LOG.error("Invalid updatespersec {}, should be > 0", rate); + return false; + } + + final int limit = flags.limit; + if (limit <= 0 && limit != Integer.MIN_VALUE) { + LOG.error("Invalid limit {}, should be > 0", limit); + return false; + } + + final long printProgress; + if (flags.verbose) { + printProgress = 10; + } else { + printProgress = flags.printProgress; + } + + final ClientConfiguration clientConfiguration = new ClientConfiguration(); + clientConfiguration.addConfiguration(conf); + final BookKeeper bk = new BookKeeper(clientConfiguration); + final BookKeeperAdmin admin = new BookKeeperAdmin(bk); + final UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, admin); + final ServerConfiguration serverConfiguration = new ServerConfiguration(conf); + final BookieSocketAddress newBookieId = Bookie.getBookieAddress(serverConfiguration); + serverConfiguration.setUseHostNameAsBookieID(!flags.hostname); + final BookieSocketAddress oldBookieId = Bookie.getBookieAddress(serverConfiguration); + + BookieShell.UpdateLedgerNotifier progressable = new BookieShell.UpdateLedgerNotifier() { + long lastReport = System.nanoTime(); + + @Override + public void progress(long updated, long issued) { + if (printProgress <= 0) { + return; // disabled + } + if (TimeUnit.MILLISECONDS.toSeconds(MathUtils.elapsedMSec(lastReport)) >= printProgress) { + LOG.info("Number of ledgers issued={}, updated={}", issued, updated); + lastReport = MathUtils.nowInNano(); + } + } + }; + + try { + updateLedgerOp.updateBookieIdInLedgers(oldBookieId, newBookieId, rate, limit, progressable); + } catch (IOException e) { + LOG.error("Failed to update ledger metadata", e); + return false; + } + + return true; + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java index 70ffee6b6ff..410f47ba34b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java @@ -38,7 +38,7 @@ */ public class DeleteLedgerCommand extends BookieCommand { - private static final String NAME = "deleteledger"; + private static final String NAME = "delete"; private static final String DESC = "Delete a ledger."; private static final String DEFAULT = ""; diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 9e94cd28c67..d7eca51a91a 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -23,6 +23,7 @@ import org.apache.bookkeeper.tools.cli.BKCtl; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.FlipBookieIdCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; @@ -67,6 +68,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new ReadLogCommand()) .addCommand(new ReadLogMetadataCommand()) .addCommand(new LocalConsistencyCheckCommand()) + .addCommand(new FlipBookieIdCommand()) .addCommand(new RegenerateInterleavedStorageIndexFileCommand()) .build(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java new file mode 100644 index 00000000000..a57a7584b8e --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.verify; +import static org.mockito.internal.verification.VerificationModeFactory.times; +import static org.powermock.api.mockito.PowerMockito.verifyNew; + +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.UpdateLedgerOp; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link FlipBookieIdCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ FlipBookieIdCommand.class, Bookie.class, UpdateLedgerOp.class }) +public class FlipBookieIdCommandTest extends BookieCommandTestBase { + + @Mock + private ClientConfiguration clientConfiguration; + + @Mock + private BookKeeper bookKeeper; + + @Mock + private BookKeeperAdmin bookKeeperAdmin; + + @Mock + private UpdateLedgerOp updateLedgerOp; + + @Mock + private ServerConfiguration serverConfiguration; + + @Mock + private BookieSocketAddress bookieSocketAddress; + + public FlipBookieIdCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.whenNew(ClientConfiguration.class).withNoArguments().thenReturn(clientConfiguration); + PowerMockito.whenNew(BookKeeper.class).withParameterTypes(ClientConfiguration.class) + .withArguments(eq(clientConfiguration)).thenReturn(bookKeeper); + PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(BookKeeper.class).withArguments(eq(bookKeeper)) + .thenReturn(bookKeeperAdmin); + PowerMockito.whenNew(UpdateLedgerOp.class).withArguments(eq(bookKeeper), eq(bookKeeperAdmin)) + .thenReturn(updateLedgerOp); + PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(conf)).thenReturn(serverConfiguration); + PowerMockito.mockStatic(Bookie.class); + PowerMockito.when(Bookie.getBookieAddress(eq(serverConfiguration))).thenReturn(bookieSocketAddress); + } + + @Test + public void testCommand() throws Exception { + FlipBookieIdCommand cmd = new FlipBookieIdCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "" })); + verifyNew(ClientConfiguration.class, times(1)).withNoArguments(); + verify(clientConfiguration, times(1)).addConfiguration(eq(conf)); + verifyNew(BookKeeper.class, times(1)).withArguments(eq(clientConfiguration)); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(bookKeeper)); + verifyNew(UpdateLedgerOp.class, times(1)).withArguments(eq(bookKeeper), eq(bookKeeperAdmin)); + verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(conf)); + verify(serverConfiguration, times(1)).setUseHostNameAsBookieID(anyBoolean()); + verify(updateLedgerOp, times(1)) + .updateBookieIdInLedgers(eq(bookieSocketAddress), eq(bookieSocketAddress), anyInt(), anyInt(), any()); + } + +} From 9203b4f51f21df5735a2efd8952f94211c9e045e Mon Sep 17 00:00:00 2001 From: Nicolas Michael Date: Tue, 9 Apr 2019 18:44:52 -0700 Subject: [PATCH 0269/1642] ISSUE #2053: Bugfix for Percentile Calculation in FastCodahale Timer Implementation This bugfix for the FastCodahale timer implementation ensures that percentiles provided by a FastSnapshot are calculated correctly even if the total count of events (provided by FastTimer) is out of sync with the recorded events in the percentile buckets. ### Motivation FastCodahale Timer implementation may miscalculate percentiles if snapshots of values are slightly out of sync, and if only few events have been recorded. FastCodahale Timers use fine-grained locking and are meant to tolerate that (some) values change while being recorded or while snapshots are created. Currently, the total count of requests is not synchronized with the number of requests recorded in percentile buckets. If a snapshot is created while the total count of the timer has been incremented beyond the sum of values in the percentile buckets, the percentile calculation may produce wrong values. For example, if 3 percentile values have been recorded, but the overall count is 4, then the percentile calculation would be based on 4 values. This becomes most obvious if a percentile > .75 (e.g. p95) is being calculated. For this, the implementation will try to find 0.95 * 4 values, which is more than the 3 values recorded in the buckets. Since no bucket fulfills the criteria, the bound of the last (overflow) bucket will be returned, i.e. Long.MAX_VALUE. ### Changes FastSnapshots now bases the percentile calculation on the sum of values in the percentile buckets rather than a count provided by the caller (i.e. FastTimer). This ensures that percentiles are calculated correctly without the need of having all counters fully synchronized. Master Issue: #2053 Reviewers: Jia Zhai , Sijie Guo This closes #2054 from nicmichael/fast-codahale-bugfix, closes #2053 --- .../stats/codahale/FastSnapshot.java | 19 +++++++++++++-- .../stats/codahale/FastTimerTest.java | 23 +++++++++++++++++++ 2 files changed, 40 insertions(+), 2 deletions(-) diff --git a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/FastSnapshot.java b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/FastSnapshot.java index ee6466ea807..f6ceb8834d2 100644 --- a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/FastSnapshot.java +++ b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/FastSnapshot.java @@ -32,6 +32,7 @@ public class FastSnapshot extends Snapshot { private final long max; private final long sum; private final long cnt; + private final long pcnt; private final long[] values; @SuppressFBWarnings( @@ -43,18 +44,19 @@ public FastSnapshot(FastTimer timer, long min, long max, long sum, long cnt, lon this.max = max; this.sum = sum; this.cnt = cnt; + this.pcnt = values != null ? sumOf(values) : 0; this.values = values; } @Override public double getValue(double quantile) { - if (cnt == 0 || values == null) { + if (pcnt == 0 || values == null) { return 0; } long qcnt = 0; for (int i = 0; i < values.length; i++) { qcnt += values[i]; - if (((double) qcnt) / ((double) cnt) > quantile) { + if (((double) qcnt) / ((double) pcnt) > quantile) { return timer.getBucketBound(i); } } @@ -105,4 +107,17 @@ public void dump(OutputStream output) { // values in this snapshot represent percentile buckets, but not discrete values } + /** + * Calculates the sum of values of an array. + * @param a an array of values + * @return the sum of all array values + */ + private long sumOf(long[] a) { + long sum = 0; + for (long x : a) { + sum += x; + } + return sum; + } + } \ No newline at end of file diff --git a/bookkeeper-stats-providers/codahale-metrics-provider/src/test/java/org/apache/bookkeeper/stats/codahale/FastTimerTest.java b/bookkeeper-stats-providers/codahale-metrics-provider/src/test/java/org/apache/bookkeeper/stats/codahale/FastTimerTest.java index 34ae5c3157d..b3a744fbdf5 100644 --- a/bookkeeper-stats-providers/codahale-metrics-provider/src/test/java/org/apache/bookkeeper/stats/codahale/FastTimerTest.java +++ b/bookkeeper-stats-providers/codahale-metrics-provider/src/test/java/org/apache/bookkeeper/stats/codahale/FastTimerTest.java @@ -216,4 +216,27 @@ public void testTimerNoBuckets() { assertEquals("FastSnapshot.getMean()", 10, (int) Math.round(s.getMean() / 1000000)); } + @Test + public void testSnapshotOutOfSync() { + FastTimer t = getMockedFastTimer(1, FastTimer.Buckets.fine); + t.update(t.getBucketBound(0) - 1, TimeUnit.NANOSECONDS); // add value to 1st bucket + t.update(t.getBucketBound(1) - 1, TimeUnit.NANOSECONDS); // add value to 2nd bucket + t.update(t.getBucketBound(2) - 1, TimeUnit.NANOSECONDS); // add value to 3rd bucket + incSec(); // advance mocked time to next second + Snapshot s1 = t.getSnapshot(); + long[] buckets = new long[t.getNumberOfBuckets()]; + buckets[0] = 1; + buckets[1] = 1; + buckets[2] = 1; + Snapshot s2 = new FastSnapshot(t, + t.getBucketBound(0) - 1, + t.getBucketBound(2) - 1, + t.getBucketBound(0) + t.getBucketBound(1) + t.getBucketBound(2) + 3, + 4, // count (4) is out of sync with number of recorded events in buckets (3) + buckets); + assertEquals("FastSnapshot.getMin()", s1.getMin(), s2.getMin()); + assertEquals("FastSnapshot.getMax()", s1.getMax(), s2.getMax()); + assertEquals("FastSnapshot.getValue(0.95)", (long) s1.getValue(0.95), (long) s2.getValue(0.95)); + } + } From 24d69a280e28c137fea949fabed1e8832f59dc61 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 10 Apr 2019 19:18:17 +0800 Subject: [PATCH 0270/1642] Migrate command `rebuild-db-ledger-locations-index` Descriptions of the changes in this PR: #2036 Reviewers: Sijie Guo This closes #2034 from zymap/command-RDBLLI --- .../apache/bookkeeper/bookie/BookieShell.java | 8 +-- .../RebuildDBLedgerLocationsIndexCommand.java | 56 +++++++++++++++ .../cli/commands/BookieCommandGroup.java | 2 + ...uildDBLedgerLocationsIndexCommandTest.java | 71 +++++++++++++++++++ 4 files changed, 132 insertions(+), 5 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/RebuildDBLedgerLocationsIndexCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/RebuildDBLedgerLocationsIndexCommandTest.java 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 f3f83a35bb2..f87eb0dc8e0 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 @@ -49,7 +49,6 @@ import java.util.stream.Collectors; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; -import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp; import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.api.LedgerMetadata; @@ -79,6 +78,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.RebuildDBLedgerLocationsIndexCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.RegenerateInterleavedStorageIndexFileCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.DecommissionCommand; @@ -2083,10 +2083,8 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { - LOG.info("=== Rebuilding bookie index ==="); - ServerConfiguration conf = new ServerConfiguration(bkConf); - new LocationsIndexRebuildOp(conf).initiate(); - LOG.info("-- Done rebuilding bookie index --"); + RebuildDBLedgerLocationsIndexCommand cmd = new RebuildDBLedgerLocationsIndexCommand(); + cmd.apply(bkConf, new CliFlags()); return 0; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/RebuildDBLedgerLocationsIndexCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/RebuildDBLedgerLocationsIndexCommand.java new file mode 100644 index 00000000000..fda5945dbb2 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/RebuildDBLedgerLocationsIndexCommand.java @@ -0,0 +1,56 @@ +/* + * 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.tools.cli.commands.bookie; + +import java.io.IOException; +import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to rebuild DBLedgerStorage locations index. + */ +public class RebuildDBLedgerLocationsIndexCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(RebuildDBLedgerLocationsIndexCommand.class); + + private static final String NAME = "rebuild-db-ledger-locations-index"; + private static final String DESC = "Rbuild DBLedgerStorage locations index by scanning the entry logs"; + + public RebuildDBLedgerLocationsIndexCommand() { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(new CliFlags()).build()); + } + + @Override + public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { + LOG.info("=== Rebuilding bookie index ==="); + ServerConfiguration serverConfiguration = new ServerConfiguration(conf); + try { + new LocationsIndexRebuildOp(serverConfiguration).initiate(); + } catch (IOException e) { + e.printStackTrace(); + } + LOG.info("-- Done rebuilding bookie index --"); + return true; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index d7eca51a91a..02ccf47a7a0 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -35,6 +35,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.RebuildDBLedgerLocationsIndexCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.RegenerateInterleavedStorageIndexFileCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; @@ -64,6 +65,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new ListLedgersCommand()) .addCommand(new ConvertToInterleavedStorageCommand()) .addCommand(new ReadJournalCommand()) + .addCommand(new RebuildDBLedgerLocationsIndexCommand()) .addCommand(new ReadLedgerCommand()) .addCommand(new ReadLogCommand()) .addCommand(new ReadLogMetadataCommand()) diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/RebuildDBLedgerLocationsIndexCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/RebuildDBLedgerLocationsIndexCommandTest.java new file mode 100644 index 00000000000..658d156ec24 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/RebuildDBLedgerLocationsIndexCommandTest.java @@ -0,0 +1,71 @@ +/* + * 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.tools.cli.commands.bookie; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link RebuildDBLedgerLocationsIndexCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ RebuildDBLedgerLocationsIndexCommand.class }) +public class RebuildDBLedgerLocationsIndexCommandTest extends BookieCommandTestBase { + + @Mock + private LocationsIndexRebuildOp locationsIndexRebuildOp; + + public RebuildDBLedgerLocationsIndexCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(conf)).thenReturn(conf); + + PowerMockito.whenNew(LocationsIndexRebuildOp.class).withParameterTypes(ServerConfiguration.class) + .withArguments(eq(conf)).thenReturn(locationsIndexRebuildOp); + PowerMockito.doNothing().when(locationsIndexRebuildOp).initiate(); + } + + @Test + public void testCommand() throws Exception { + RebuildDBLedgerLocationsIndexCommand command = new RebuildDBLedgerLocationsIndexCommand(); + Assert.assertTrue(command.apply(bkFlags, new String[] { "" })); + + PowerMockito.verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(conf)); + PowerMockito.verifyNew(LocationsIndexRebuildOp.class, times(1)).withArguments(eq(conf)); + verify(locationsIndexRebuildOp, times(1)).initiate(); + } +} From 7bae459036fc5034c685de5f11102ad2d22bbb22 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 11 Apr 2019 10:19:47 +0800 Subject: [PATCH 0271/1642] Migrate command `Whatisinstanceid` Descriptions of the changes in this PR: #2027 Reviewers: Enrico Olivelli , Sijie Guo This closes #2028 from zymap/whatisinstanceid --- .../apache/bookkeeper/bookie/BookieShell.java | 14 +--- .../commands/bookies/InstanceIdCommand.java | 65 +++++++++++++++++ .../cli/commands/BookiesCommandGroup.java | 2 + .../bookies/InstanceIdCommandTest.java | 69 +++++++++++++++++++ 4 files changed, 139 insertions(+), 11 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InstanceIdCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InstanceIdCommandTest.java 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 f87eb0dc8e0..2177aa6ba15 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 @@ -83,6 +83,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.DecommissionCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.InstanceIdCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.NukeExistingClusterCommand; @@ -1458,17 +1459,8 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { - runFunctionWithRegistrationManager(bkConf, rm -> { - String readInstanceId = null; - try { - readInstanceId = rm.getClusterInstanceId(); - } catch (BookieException e) { - throw new UncheckedExecutionException(e); - } - LOG.info("Metadata Service Uri: {} InstanceId: {}", - bkConf.getMetadataServiceUriUnchecked(), readInstanceId); - return null; - }); + InstanceIdCommand cmd = new InstanceIdCommand(); + cmd.apply(bkConf, new CliFlags()); return 0; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InstanceIdCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InstanceIdCommand.java new file mode 100644 index 00000000000..6fd60e35f08 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InstanceIdCommand.java @@ -0,0 +1,65 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to print instance id of the cluster. + */ +public class InstanceIdCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(InstanceIdCommand.class); + + private static final String NAME = "instanceid"; + private static final String DESC = "Print the instanceid of the cluster"; + + public InstanceIdCommand() { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(new CliFlags()).build()); + } + + @Override + public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { + try { + runFunctionWithRegistrationManager(conf, rm -> { + String readInstanceId = null; + try { + readInstanceId = rm.getClusterInstanceId(); + } catch (BookieException e) { + throw new UncheckedExecutionException(e); + } + LOG.info("Metadata Service Uri: {} InstanceId: {}", + conf.getMetadataServiceUriUnchecked(), readInstanceId); + return null; + }); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + return true; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java index cb161f59782..c7a44d4afd5 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java @@ -24,6 +24,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookies.DecommissionCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InitCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.InstanceIdCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.NukeExistingClusterCommand; @@ -52,6 +53,7 @@ public class BookiesCommandGroup extends CliCommandGroup { .addCommand(new DecommissionCommand()) .addCommand(new InitCommand()) .addCommand(new RecoverCommand()) + .addCommand(new InstanceIdCommand()) .build(); public BookiesCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InstanceIdCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InstanceIdCommandTest.java new file mode 100644 index 00000000000..5f2e716514f --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InstanceIdCommandTest.java @@ -0,0 +1,69 @@ +/* + * 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.tools.cli.commands.bookies; + +import static org.mockito.ArgumentMatchers.any; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.when; + +import java.util.function.Function; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link InstanceIdCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ InstanceIdCommand.class, MetadataDrivers.class, RegistrationManager.class }) +public class InstanceIdCommandTest extends BookieCommandTestBase { + + public InstanceIdCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.mockStatic(MetadataDrivers.class); + + RegistrationManager manager = mock(RegistrationManager.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function function = invocationOnMock.getArgument(1); + function.apply(manager); + return null; + }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", + any(ServerConfiguration.class), any(Function.class)); + when(manager.getClusterInstanceId()).thenReturn(""); + } + + @Test + public void testCommand() { + InstanceIdCommand cmd = new InstanceIdCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] {})); + } +} From 6bf1ff82c6623d1d0f76dbaa9d5db8669482424a Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 15 Apr 2019 14:00:56 +0800 Subject: [PATCH 0272/1642] Migrate command `whoisauditor` Descriptions of the changes in this PR: - Using `bkctl` run command `whoisauditor` ### Motivation #2007 Reviewers: Sijie Guo , Jia Zhai This closes #2008 from zymap/command-whoisauditor --- .../apache/bookkeeper/bookie/BookieShell.java | 32 ++----- .../autorecovery/WhoIsAuditorCommand.java | 90 +++++++++++++++++++ .../commands/AutoRecoveryCommandGroup.java | 16 ++-- tools/ledger/src/main/resources/commands | 2 +- .../autorecovery/WhoIsAuditorCommandTest.java | 88 ++++++++++++++++++ 5 files changed, 191 insertions(+), 37 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommandTest.java 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 2177aa6ba15..3f307f58e8b 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 @@ -21,7 +21,6 @@ import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithMetadataBookieDriver; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; -import static org.apache.bookkeeper.tools.cli.helpers.CommandHelpers.getBookieSocketAddrStringRepresentation; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; @@ -29,7 +28,6 @@ import java.io.File; import java.io.IOException; import java.io.Serializable; -import java.net.URI; import java.nio.file.FileSystems; import java.nio.file.Files; import java.nio.file.Path; @@ -58,12 +56,11 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; -import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.replication.ReplicationException; import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.WhoIsAuditorCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.FlipBookieIdCommand; @@ -104,7 +101,6 @@ import org.apache.bookkeeper.util.Tool; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.HelpFormatter; @@ -119,7 +115,6 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.ArrayUtils; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1408,27 +1403,10 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { - ZooKeeper zk = null; - try { - String metadataServiceUri = bkConf.getMetadataServiceUri(); - String zkServers = ZKMetadataDriverBase.getZKServersFromServiceUri(URI.create(metadataServiceUri)); - zk = ZooKeeperClient.newBuilder() - .connectString(zkServers) - .sessionTimeoutMs(bkConf.getZkTimeout()) - .build(); - BookieSocketAddress bookieId = AuditorElector.getCurrentAuditor(bkConf, zk); - if (bookieId == null) { - LOG.info("No auditor elected"); - return -1; - } - LOG.info("Auditor: " + getBookieSocketAddrStringRepresentation(bookieId)); - } finally { - if (zk != null) { - zk.close(); - } - } - - return 0; + CliFlags flags = new CliFlags(); + WhoIsAuditorCommand cmd = new WhoIsAuditorCommand(); + boolean result = cmd.apply(bkConf, flags); + return result ? 0 : -1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommand.java new file mode 100644 index 00000000000..9653db8c469 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommand.java @@ -0,0 +1,90 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import static org.apache.bookkeeper.tools.cli.helpers.CommandHelpers.getBookieSocketAddrStringRepresentation; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.net.URI; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.replication.AuditorElector; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.commons.configuration.ConfigurationException; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to print which node has the auditor lock. + */ +public class WhoIsAuditorCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(WhoIsAuditorCommand.class); + + private static final String NAME = "whoisauditor"; + private static final String DESC = "Print the node which holds the auditor lock."; + + public WhoIsAuditorCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new CliFlags()) + .build()); + } + + @Override + public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { + try { + return getAuditor(conf); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + public boolean getAuditor(ServerConfiguration conf) + throws ConfigurationException, InterruptedException, IOException, KeeperException { + ZooKeeper zk = null; + try { + String metadataServiceUri = conf.getMetadataServiceUri(); + String zkServers = ZKMetadataDriverBase.getZKServersFromServiceUri(URI.create(metadataServiceUri)); + zk = ZooKeeperClient.newBuilder() + .connectString(zkServers) + .sessionTimeoutMs(conf.getZkTimeout()) + .build(); + BookieSocketAddress bookieId = AuditorElector.getCurrentAuditor(conf, zk); + if (bookieId == null) { + LOG.info("No auditor elected"); + return false; + } + LOG.info("Auditor: " + getBookieSocketAddrStringRepresentation(bookieId)); + } finally { + if (zk != null) { + zk.close(); + } + } + return true; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java index 9cee7ff40a0..814976a1b48 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java @@ -20,8 +20,7 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; -import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; -import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.WhoIsAuditorCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -35,14 +34,13 @@ public class AutoRecoveryCommandGroup extends CliCommandGroup { private static final String DESC = "Command on some specific operation."; private static final CliSpec spec = CliSpec.newBuilder() - .withName(NAME) - .withDescription(DESC) - .withCategory(CATEGORY_INFRA_SERVICE) - .addCommand(new ListUnderReplicatedCommand()) - .addCommand(new LostBookieRecoveryDelayCommand()) - .build(); + .withName(NAME) + .withDescription(DESC) + .withCategory(CATEGORY_INFRA_SERVICE) + .addCommand(new WhoIsAuditorCommand()) + .build(); public AutoRecoveryCommandGroup() { super(spec); } -} \ No newline at end of file +} diff --git a/tools/ledger/src/main/resources/commands b/tools/ledger/src/main/resources/commands index c4d6412c55f..d81ba5aae06 100644 --- a/tools/ledger/src/main/resources/commands +++ b/tools/ledger/src/main/resources/commands @@ -21,4 +21,4 @@ org.apache.bookkeeper.tools.cli.commands.BookieIdCommandGroup org.apache.bookkeeper.tools.cli.commands.BookiesCommandGroup org.apache.bookkeeper.tools.cli.commands.CookieCommandGroup org.apache.bookkeeper.tools.cli.commands.LedgerCommandGroup -org.apache.bookkeeper.tools.cli.commands.AutoRecoveryCommandGroup \ No newline at end of file +org.apache.bookkeeper.tools.cli.commands.AutoRecoveryCommandGroup diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommandTest.java new file mode 100644 index 00000000000..b4f248a233f --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommandTest.java @@ -0,0 +1,88 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.when; + +import java.net.URI; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.replication.AuditorElector; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.tools.cli.helpers.CommandHelpers; +import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link WhoIsAuditorCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ WhoIsAuditorCommand.class, ZKMetadataDriverBase.class, ZooKeeperClient.class, AuditorElector.class, + CommandHelpers.class +}) +public class WhoIsAuditorCommandTest extends BookieCommandTestBase { + + public WhoIsAuditorCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + PowerMockito.mockStatic(ZKMetadataDriverBase.class); + PowerMockito.when(ZKMetadataDriverBase.getZKServersFromServiceUri(eq(URI.create(conf.getMetadataServiceUri())))) + .thenReturn(""); + + ZooKeeperClient zk = mock(ZooKeeperClient.class); + ZooKeeperClient.Builder builder = mock(ZooKeeperClient.Builder.class); + PowerMockito.mockStatic(ZooKeeperClient.class); + PowerMockito.when(ZooKeeperClient.newBuilder()).thenReturn(builder); + when(builder.connectString(anyString())).thenReturn(builder); + when(builder.sessionTimeoutMs(anyInt())).thenReturn(builder); + when(builder.build()).thenReturn(zk); + + BookieSocketAddress bookieId = mock(BookieSocketAddress.class); + + PowerMockito.mockStatic(AuditorElector.class); + PowerMockito.when(AuditorElector.getCurrentAuditor(eq(conf), eq(zk))) + .thenReturn(bookieId); + + PowerMockito.mockStatic(CommandHelpers.class); + PowerMockito.when(CommandHelpers.getBookieSocketAddrStringRepresentation(eq(bookieId))).thenReturn(""); + } + + @Test + public void testCommand() { + WhoIsAuditorCommand cmd = new WhoIsAuditorCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "" })); + } +} From 73ef5f91f1ded19d2cb4a62ade5f0917211b06b9 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 16 Apr 2019 16:48:32 +0800 Subject: [PATCH 0273/1642] Migrate command `autorecovery` Descriptions of the changes in this PR: - Using `bkctl` run command `autorecovery` ### Motivation #2009 Reviewers: Jia Zhai , Sijie Guo This closes #2010 from zymap/command-autorecovery and squashes the following commits: 6fcd96904 [Yong Zhang] Fix validation cd383f372 [Yong Zhang] Rename package efb573221 [Yong Zhang] Rename command 89da2852e [Yong Zhang] Fix error in bookieshell a037501ee [Yong Zhang] Rename args 4bb36b0b3 [Yong Zhang] Fix imports postion e41a742d3 [Yong Zhang] Add unit test for command `autorecovery` b0c91f704 [Yong Zhang] Rename file cb06f66de [Yong Zhang] Migrate command `autorecovery` ed008f278 [Yong Zhang] Migrate command `whoisauditor` 5b8e0971a [Yong Zhang] Migrate command `Whatisinstanceid` 90c79444d [Yong Zhang] Migrate command `rebuild-db-ledger-locations-index` 848f8527f [Nicolas Michael] ISSUE #2053: Bugfix for Percentile Calculation in FastCodahale Timer Implementation 06f2b6f50 [Yong Zhang] Migrate command `updateledgers` 7ad5849b1 [Yong Zhang] Migrate command `regenerate-interleaved-storage-index-file` d4dbb6bfb [Dongfa,Huang] Avoid useless verify if LedgerEntryRequest completed 5c150f283 [Enrico Olivelli] Release notes for 4.9.1 1246826ba [Yong Zhang] Migrate command `recover` 1d4cc71fd [Yong Zhang] Migrate command `localconsistencycheck` 67f83620e [Yong Zhang] Migrate command `readledger` bfbd6b023 [Yong Zhang] Migrate command `decommission` d40b8b69f [Yong Zhang] Migrate command `readlog` 95d145a15 [Yong Zhang] Migrate command `nukeexistingcluster` e2b1dc7f3 [Yong Zhang] Migrate command `listunderreplicated` 0988e12c7 [bd2019us] ISSUE #2023: change cached thread pool to fixed thread pool 6a6d7bbd9 [Yong Zhang] Migrate command `initnewcluster` c391fe58d [Yong Zhang] Migrate command `readlogmetadata` 120d67737 [Yong Zhang] Migrate command `lostbookierecoverydelay` bf66235e5 [Yong Zhang] Migrate command `deleteledger` 751e55fa4 [Arvin] ISSUE #2020: close db properly to avoid open RocksDB failure at the second time 138a7ae85 [Yong Zhang] Migrate command `metadataformat` b043d1694 [Yong Zhang] Migrate command `listledgers` 4573285db [Ivan Kelly] Docker autobuild hook e3d807a32 [Like] Fix IDE complain as there are multi choices for error code 9524a9f4a [Yong Zhang] Migrate command `readjournal` 6c3f33f55 [Yong Zhang] Fix when met unexpect entry id crashed e35a108c7 [Like] Fix error message for unrecognized number-of-bookies 5902ee27b [Boyang Jerry Peng] fix potential NPE when releasing entry that is null 6aa73ce05 [Ivan Kelly] [RELEASE] Update website to include documentation for 4.8.2 1448d12aa [Yong Zhang] Migrate command `listfilesondisk` 4de598379 [Yong Zhang] Issue #1987: Migrate command `convert-to-interleaved-storage` 468743e7e [Matteo Merli] In DbLedgerStorage use default values when config key is present but empty f26a4cae0 [Ivan Kelly] Release notes for v4.8.2 ec2636cd2 [Yong Zhang] Issue #1985: Migrate command `convert-to-db-storage` 8cc7239ac [Yong Zhang] Issue #1982: Migrate command `bookiesanity` fa90f0185 [Yong Zhang] Issue #1980: Migrate command `ledger` from shell to bkctl --- .../apache/bookkeeper/bookie/BookieShell.java | 45 +------ .../commands/autorecovery/ToggleCommand.java | 121 +++++++++++++++++ .../commands/AutoRecoveryCommandGroup.java | 2 + .../autorecovery/AutoRecoveryCommandTest.java | 123 ++++++++++++++++++ 4 files changed, 251 insertions(+), 40 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ToggleCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/AutoRecoveryCommandTest.java 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 3f307f58e8b..eff70edd3ee 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 @@ -55,11 +55,10 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerMetadataSerDe; -import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.replication.ReplicationException; import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.ToggleCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.WhoIsAuditorCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; @@ -114,7 +113,6 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.ArrayUtils; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1290,43 +1288,10 @@ int runCmd(CommandLine cmdLine) throws Exception { boolean disable = cmdLine.hasOption("d"); boolean enable = cmdLine.hasOption("e"); - if (enable && disable) { - LOG.error("Only one of -enable and -disable can be specified"); - printUsage(); - return 1; - } - - runFunctionWithLedgerManagerFactory(bkConf, mFactory -> { - try { - try (LedgerUnderreplicationManager underreplicationManager = - mFactory.newLedgerUnderreplicationManager()) { - if (!enable && !disable) { - boolean enabled = underreplicationManager.isLedgerReplicationEnabled(); - System.out.println("Autorecovery is " + (enabled ? "enabled." : "disabled.")); - } else if (enable) { - if (underreplicationManager.isLedgerReplicationEnabled()) { - LOG.warn("Autorecovery already enabled. Doing nothing"); - } else { - LOG.info("Enabling autorecovery"); - underreplicationManager.enableLedgerReplication(); - } - } else { - if (!underreplicationManager.isLedgerReplicationEnabled()) { - LOG.warn("Autorecovery already disabled. Doing nothing"); - } else { - LOG.info("Disabling autorecovery"); - underreplicationManager.disableLedgerReplication(); - } - } - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new UncheckedExecutionException(e); - } catch (KeeperException | ReplicationException e) { - throw new UncheckedExecutionException(e); - } - return null; - }); + ToggleCommand.AutoRecoveryFlags flags = new ToggleCommand.AutoRecoveryFlags() + .enable(enable).status(!disable && !enable); + ToggleCommand cmd = new ToggleCommand(); + cmd.apply(bkConf, flags); return 0; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ToggleCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ToggleCommand.java new file mode 100644 index 00000000000..c1457153c71 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/ToggleCommand.java @@ -0,0 +1,121 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.concurrent.ExecutionException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.replication.ReplicationException; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Command to enable or disable auto recovery in the cluster. + */ +public class ToggleCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(ToggleCommand.class); + + private static final String NAME = "toggle"; + private static final String DESC = "Enable or disable auto recovery in the cluster. Default is disable."; + + public ToggleCommand() { + this(new AutoRecoveryFlags()); + } + + private ToggleCommand(AutoRecoveryFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME).withDescription(DESC) + .withFlags(flags).build()); + } + + /** + * Flags for auto recovery command. + */ + @Accessors(fluent = true) + @Setter + public static class AutoRecoveryFlags extends CliFlags { + + @Parameter(names = { "-e", "--enable" }, description = "Enable or disable auto recovery of under replicated " + + "ledgers.") + private boolean enable; + + @Parameter(names = {"-s", "--status"}, description = "Check the auto recovery status.") + private boolean status; + + } + + @Override + public boolean apply(ServerConfiguration conf, AutoRecoveryFlags cmdFlags) { + try { + return handler(conf, cmdFlags); + } catch (MetadataException | ExecutionException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean handler(ServerConfiguration conf, AutoRecoveryFlags flags) + throws MetadataException, ExecutionException { + MetadataDrivers.runFunctionWithLedgerManagerFactory(conf, mFactory -> { + try { + try (LedgerUnderreplicationManager underreplicationManager = mFactory + .newLedgerUnderreplicationManager()) { + if (flags.status) { + System.out.println("Autorecovery is " + (underreplicationManager.isLedgerReplicationEnabled() + ? "enabled." : "disabled.")); + return null; + } + if (flags.enable) { + if (underreplicationManager.isLedgerReplicationEnabled()) { + LOG.warn("Autorecovery already enabled. Doing nothing"); + } else { + LOG.info("Enabling autorecovery"); + underreplicationManager.enableLedgerReplication(); + } + } else { + if (!underreplicationManager.isLedgerReplicationEnabled()) { + LOG.warn("Autorecovery already disabled. Doing nothing"); + } else { + LOG.info("Disabling autorecovery"); + underreplicationManager.disableLedgerReplication(); + } + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedExecutionException(e); + } catch (KeeperException | ReplicationException e) { + throw new UncheckedExecutionException(e); + } + return null; + }); + return true; + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java index 814976a1b48..aa4d7f4a8a5 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java @@ -20,6 +20,7 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.ToggleCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.WhoIsAuditorCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -38,6 +39,7 @@ public class AutoRecoveryCommandGroup extends CliCommandGroup { .withDescription(DESC) .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new WhoIsAuditorCommand()) + .addCommand(new ToggleCommand()) .build(); public AutoRecoveryCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/AutoRecoveryCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/AutoRecoveryCommandTest.java new file mode 100644 index 00000000000..50c68d7e78d --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/AutoRecoveryCommandTest.java @@ -0,0 +1,123 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.when; + +import java.util.function.Function; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.replication.ReplicationException; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.zookeeper.KeeperException; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + + +/** + * Unit test for {@link ToggleCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ ToggleCommand.class, MetadataDrivers.class }) +public class AutoRecoveryCommandTest extends BookieCommandTestBase { + + private LedgerManagerFactory ledgerManagerFactory; + private LedgerUnderreplicationManager ledgerUnderreplicationManager; + + public AutoRecoveryCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + ledgerManagerFactory = mock(LedgerManagerFactory.class); + + PowerMockito.mockStatic(MetadataDrivers.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function function = invocationOnMock.getArgument(1); + function.apply(ledgerManagerFactory); + return true; + }).when(MetadataDrivers.class, "runFunctionWithLedgerManagerFactory", any(ServerConfiguration.class), + any(Function.class)); + + ledgerUnderreplicationManager = mock(LedgerUnderreplicationManager.class); + when(ledgerManagerFactory.newLedgerUnderreplicationManager()).thenReturn(ledgerUnderreplicationManager); + } + + @Test + public void testWithEnable() + throws InterruptedException, ReplicationException.CompatibilityException, KeeperException, + ReplicationException.UnavailableException { + testCommand("-e"); + verify(ledgerManagerFactory, times(1)).newLedgerUnderreplicationManager(); + verify(ledgerUnderreplicationManager, times(1)).isLedgerReplicationEnabled(); + } + + @Test + public void testWithEnableLongArgs() throws ReplicationException.UnavailableException { + when(ledgerUnderreplicationManager.isLedgerReplicationEnabled()).thenReturn(false); + testCommand("--enable"); + verify(ledgerUnderreplicationManager, times(1)).enableLedgerReplication(); + } + + @Test + public void testWithLook() + throws InterruptedException, ReplicationException.CompatibilityException, KeeperException, + ReplicationException.UnavailableException { + testCommand("s"); + verify(ledgerManagerFactory, times(1)).newLedgerUnderreplicationManager(); + verify(ledgerUnderreplicationManager, times(1)).isLedgerReplicationEnabled(); + } + + @Test + public void testWithNoArgs() + throws InterruptedException, ReplicationException.CompatibilityException, KeeperException, + ReplicationException.UnavailableException { + testCommand(""); + verify(ledgerManagerFactory, times(1)).newLedgerUnderreplicationManager(); + verify(ledgerUnderreplicationManager, times(1)).isLedgerReplicationEnabled(); + } + + @Test + public void testWithNoArgsDisable() throws ReplicationException.UnavailableException { + when(ledgerUnderreplicationManager.isLedgerReplicationEnabled()).thenReturn(true); + testCommand(""); + verify(ledgerUnderreplicationManager, times(1)).isLedgerReplicationEnabled(); + verify(ledgerUnderreplicationManager, times(1)).disableLedgerReplication(); + } + + private void testCommand(String... args) { + ToggleCommand cmd = new ToggleCommand(); + Assert.assertTrue(cmd.apply(bkFlags, args)); + } +} From 991240c79fb5982fd995dcce58e2c19eeb7f9995 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 17 Apr 2019 14:32:24 +0800 Subject: [PATCH 0274/1642] Migrate command `triggeraudit` Descriptions of the changes in this PR: - Using `bkctl` run command `triggeraudit` ### Motivation #2011 Reviewers: Sijie Guo , Jia Zhai This closes #2012 from zymap/command-triggeraudit and squashes the following commits: 5a4b496ec [Yong Zhang] Fix conflict --- 236b7d6e5 [Yong Zhang] Fix validation b039b637e [Yong Zhang] Rename file 8bea52522 [Yong Zhang] Remove unused imports d39d1e886 [Yong Zhang] Migrate command `triggeraudit` 60d993edf [Yong Zhang] Migrate command `autorecovery` ed008f278 [Yong Zhang] Migrate command `whoisauditor` 5b8e0971a [Yong Zhang] Migrate command `Whatisinstanceid` 90c79444d [Yong Zhang] Migrate command `rebuild-db-ledger-locations-index` 848f8527f [Nicolas Michael] ISSUE #2053: Bugfix for Percentile Calculation in FastCodahale Timer Implementation 06f2b6f50 [Yong Zhang] Migrate command `updateledgers` 7ad5849b1 [Yong Zhang] Migrate command `regenerate-interleaved-storage-index-file` d4dbb6bfb [Dongfa,Huang] Avoid useless verify if LedgerEntryRequest completed 5c150f283 [Enrico Olivelli] Release notes for 4.9.1 1246826ba [Yong Zhang] Migrate command `recover` 1d4cc71fd [Yong Zhang] Migrate command `localconsistencycheck` 67f83620e [Yong Zhang] Migrate command `readledger` bfbd6b023 [Yong Zhang] Migrate command `decommission` d40b8b69f [Yong Zhang] Migrate command `readlog` 95d145a15 [Yong Zhang] Migrate command `nukeexistingcluster` e2b1dc7f3 [Yong Zhang] Migrate command `listunderreplicated` 0988e12c7 [bd2019us] ISSUE #2023: change cached thread pool to fixed thread pool 6a6d7bbd9 [Yong Zhang] Migrate command `initnewcluster` c391fe58d [Yong Zhang] Migrate command `readlogmetadata` 120d67737 [Yong Zhang] Migrate command `lostbookierecoverydelay` bf66235e5 [Yong Zhang] Migrate command `deleteledger` 751e55fa4 [Arvin] ISSUE #2020: close db properly to avoid open RocksDB failure at the second time 138a7ae85 [Yong Zhang] Migrate command `metadataformat` b043d1694 [Yong Zhang] Migrate command `listledgers` 4573285db [Ivan Kelly] Docker autobuild hook e3d807a32 [Like] Fix IDE complain as there are multi choices for error code 9524a9f4a [Yong Zhang] Migrate command `readjournal` 6c3f33f55 [Yong Zhang] Fix when met unexpect entry id crashed e35a108c7 [Like] Fix error message for unrecognized number-of-bookies 5902ee27b [Boyang Jerry Peng] fix potential NPE when releasing entry that is null 6aa73ce05 [Ivan Kelly] [RELEASE] Update website to include documentation for 4.8.2 1448d12aa [Yong Zhang] Migrate command `listfilesondisk` 4de598379 [Yong Zhang] Issue #1987: Migrate command `convert-to-interleaved-storage` 468743e7e [Matteo Merli] In DbLedgerStorage use default values when config key is present but empty f26a4cae0 [Ivan Kelly] Release notes for v4.8.2 ec2636cd2 [Yong Zhang] Issue #1985: Migrate command `convert-to-db-storage` 8cc7239ac [Yong Zhang] Issue #1982: Migrate command `bookiesanity` fa90f0185 [Yong Zhang] Issue #1980: Migrate command `ledger` from shell to bkctl --- .../apache/bookkeeper/bookie/BookieShell.java | 14 +--- .../autorecovery/TriggerAuditCommand.java | 68 ++++++++++++++++ .../commands/autorecovery/package-info.java | 19 ++--- .../commands/AutoRecoveryCommandGroup.java | 7 ++ .../autorecovery/TriggerAuditCommandTest.java | 81 +++++++++++++++++++ 5 files changed, 166 insertions(+), 23 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/TriggerAuditCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/TriggerAuditCommandTest.java 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 eff70edd3ee..700ce114ea5 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,11 +47,9 @@ import java.util.stream.Collectors; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; -import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; -import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerMetadataSerDe; @@ -59,6 +57,7 @@ import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.ToggleCommand; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.TriggerAuditCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.WhoIsAuditorCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand; @@ -1861,15 +1860,8 @@ Options getOptions() { @Override public int runCmd(CommandLine cmdLine) throws Exception { - ClientConfiguration adminConf = new ClientConfiguration(bkConf); - BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); - try { - admin.triggerAudit(); - } finally { - if (admin != null) { - admin.close(); - } - } + TriggerAuditCommand cmd = new TriggerAuditCommand(); + cmd.apply(bkConf, new CliFlags()); return 0; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/TriggerAuditCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/TriggerAuditCommand.java new file mode 100644 index 00000000000..c645a5c9972 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/TriggerAuditCommand.java @@ -0,0 +1,68 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import com.google.common.util.concurrent.UncheckedExecutionException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; + +/** + * Command to trigger AuditTask by resetting lostBookieRecoveryDelay to its current value. + */ +public class TriggerAuditCommand extends BookieCommand { + + private static final String NAME = "triggeraudit"; + private static final String DESC = "Force trigger the Audit by resetting the lostBookieRecoveryDelay."; + + public TriggerAuditCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new CliFlags()) + .build()); + } + + @Override + public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { + try { + return handler(conf); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + public boolean handler(ServerConfiguration configuration) throws Exception { + ClientConfiguration adminConf = new ClientConfiguration(configuration); + BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); + + try { + admin.triggerAudit(); + } finally { + if (admin != null) { + admin.close(); + } + } + + return true; + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java index 9294bf9ba3a..acfe982e304 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -7,17 +7,12 @@ * "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 + * 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. - */ - -/** - * This package provides all autorecovery commands. + * 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.tools.cli.commands.autorecovery; \ No newline at end of file diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java index aa4d7f4a8a5..7cc88ea7dfd 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java @@ -20,7 +20,10 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.ToggleCommand; +import org.apache.bookkeeper.tools.cli.commands.autorecovery.TriggerAuditCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.WhoIsAuditorCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -39,7 +42,11 @@ public class AutoRecoveryCommandGroup extends CliCommandGroup { .withDescription(DESC) .withCategory(CATEGORY_INFRA_SERVICE) .addCommand(new WhoIsAuditorCommand()) + .addCommand(new TriggerAuditCommand()) .addCommand(new ToggleCommand()) + .addCommand(new TriggerAuditCommand()) + .addCommand(new ListUnderReplicatedCommand()) + .addCommand(new LostBookieRecoveryDelayCommand()) .build(); public AutoRecoveryCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/TriggerAuditCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/TriggerAuditCommandTest.java new file mode 100644 index 00000000000..b60eaf67ffc --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/TriggerAuditCommandTest.java @@ -0,0 +1,81 @@ +/* + * 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.tools.cli.commands.autorecovery; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.doNothing; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyNew; + +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link TriggerAuditCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({TriggerAuditCommand.class}) +public class TriggerAuditCommandTest extends BookieCommandTestBase { + + private ClientConfiguration clientConfiguration; + private BookKeeperAdmin admin; + + public TriggerAuditCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + super.setup(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + + clientConfiguration = mock(ClientConfiguration.class); + PowerMockito.whenNew(ClientConfiguration.class).withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(conf)).thenReturn(clientConfiguration); + + admin = mock(BookKeeperAdmin.class); + PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class) + .withArguments(eq(clientConfiguration)).thenReturn(admin); + + doNothing().when(admin).triggerAudit(); + } + + @Test + public void testCommand() throws Exception { + TriggerAuditCommand cmd = new TriggerAuditCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "" })); + + verifyNew(ClientConfiguration.class, times(1)).withArguments(conf); + verifyNew(BookKeeperAdmin.class, times(1)).withArguments(clientConfiguration); + + verify(admin, times(1)).triggerAudit(); + } +} From e39150556cd8f26d48f7bdf357277bdc24922d51 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 19 Apr 2019 08:30:26 +0800 Subject: [PATCH 0275/1642] Migrate command `updatecookie` Descriptions of the changes in this PR: #2059 ### Description Provide some admin operation about cookie ``` Command to update cookie Usage: bkctl cookie admin [flags] Flags: -d, --delete Delete cookie both locally and in zooKeeper -e, --expandstorage Add new empty ledger/index directories -f, --force Force delete cookie -host, --hostname Expects config useHostNameAsBookieID=true as the option value -l, --list List paths of all the cookies present locally and on zooKeeper -h, --help Display help information ``` ### Changes Update command `updatecookie` to `admin` Reviewers: Jia Zhai This closes #2060 from zymap/command-updatecookie --- .../apache/bookkeeper/bookie/BookieShell.java | 226 +----------- .../cli/commands/cookie/AdminCommand.java | 326 ++++++++++++++++++ .../cli/commands/CookieCommandGroup.java | 2 + .../cli/commands/cookie/AdminCommandTest.java | 218 ++++++++++++ 4 files changed, 559 insertions(+), 213 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java 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 700ce114ea5..02a7391c444 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 @@ -19,11 +19,8 @@ package org.apache.bookkeeper.bookie; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; -import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithMetadataBookieDriver; -import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; import com.google.common.util.concurrent.UncheckedExecutionException; import java.io.File; import java.io.IOException; @@ -40,13 +37,10 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; -import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; -import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; @@ -86,18 +80,16 @@ import org.apache.bookkeeper.tools.cli.commands.bookies.RecoverCommand; import org.apache.bookkeeper.tools.cli.commands.client.DeleteLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; +import org.apache.bookkeeper.tools.cli.commands.cookie.AdminCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.DeleteCookieCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.GenerateCookieCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.GetCookieCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.UpdateCookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; -import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.EntryFormatter; -import org.apache.bookkeeper.util.IOUtils; import org.apache.bookkeeper.util.LedgerIdFormatter; import org.apache.bookkeeper.util.Tool; -import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; @@ -111,7 +103,6 @@ import org.apache.commons.configuration.PropertiesConfiguration; import org.apache.commons.io.FileUtils; import org.apache.commons.lang.StringUtils; -import org.apache.commons.lang3.ArrayUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1453,7 +1444,8 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) throws Exception { - int retValue = -1; + AdminCommand cmd = new AdminCommand(); + AdminCommand.AdminFlags flags = new AdminCommand.AdminFlags(); Option[] options = cmdLine.getOptions(); if (options.length != 1) { LOG.error("Invalid command!"); @@ -1474,214 +1466,22 @@ int runCmd(CommandLine cmdLine) throws Exception { return -1; } boolean useHostName = getOptionalValue(bookieId, HOSTNAME); - if (!bkConf.getUseHostNameAsBookieID() && useHostName) { - LOG.error( - "Expects config useHostNameAsBookieID=true as the option value passed is 'hostname'"); - return -1; - } else if (bkConf.getUseHostNameAsBookieID() && !useHostName) { - LOG.error("Expects configuration useHostNameAsBookieID=false as the option value passed is 'ip'"); - return -1; - } - retValue = updateBookieIdInCookie(bookieId, useHostName); - } else if (thisCommandOption.getLongOpt().equals(EXPANDSTORAGE)) { - bkConf.setAllowStorageExpansion(true); - return expandStorage(); - } else if (thisCommandOption.getLongOpt().equals(LIST)) { - return listOrDeleteCookies(false, false); - } else if (thisCommandOption.getLongOpt().equals(DELETE)) { + flags.hostname(useHostName); + flags.ip(!useHostName); + } + flags.expandstorage(thisCommandOption.getLongOpt().equals(EXPANDSTORAGE)); + flags.list(thisCommandOption.getLongOpt().equals(LIST)); + flags.delete(thisCommandOption.getLongOpt().equals(DELETE)); + if (thisCommandOption.getLongOpt().equals(DELETE)) { boolean force = false; String optionValue = thisCommandOption.getValue(); if (!StringUtils.isEmpty(optionValue) && optionValue.equals(FORCE)) { force = true; } - return listOrDeleteCookies(true, force); - } else { - LOG.error("Invalid command!"); - this.printUsage(); - return -1; + flags.force(force); } - return retValue; - } - - private int updateBookieIdInCookie(final String bookieId, final boolean useHostname) - throws Exception { - return runFunctionWithRegistrationManager(bkConf, rm -> { - try { - ServerConfiguration conf = new ServerConfiguration(bkConf); - String newBookieId = Bookie.getBookieAddress(conf).toString(); - // read oldcookie - Versioned oldCookie = null; - try { - conf.setUseHostNameAsBookieID(!useHostname); - oldCookie = Cookie.readFromRegistrationManager(rm, conf); - } catch (CookieNotFoundException nne) { - LOG.error("Either cookie already updated with UseHostNameAsBookieID={} or no cookie exists!", - useHostname, nne); - return -1; - } - Cookie newCookie = Cookie.newBuilder(oldCookie.getValue()).setBookieHost(newBookieId).build(); - boolean hasCookieUpdatedInDirs = verifyCookie(newCookie, journalDirectories[0]); - for (File dir : ledgerDirectories) { - hasCookieUpdatedInDirs &= verifyCookie(newCookie, dir); - } - if (indexDirectories != ledgerDirectories) { - for (File dir : indexDirectories) { - hasCookieUpdatedInDirs &= verifyCookie(newCookie, dir); - } - } - - if (hasCookieUpdatedInDirs) { - try { - conf.setUseHostNameAsBookieID(useHostname); - Cookie.readFromRegistrationManager(rm, conf); - // since newcookie exists, just do cleanup of oldcookie and return - conf.setUseHostNameAsBookieID(!useHostname); - oldCookie.getValue().deleteFromRegistrationManager(rm, conf, oldCookie.getVersion()); - return 0; - } catch (CookieNotFoundException nne) { - if (LOG.isDebugEnabled()) { - LOG.debug("Ignoring, cookie will be written to zookeeper"); - } - } - } else { - // writes newcookie to local dirs - for (File journalDirectory : journalDirectories) { - newCookie.writeToDirectory(journalDirectory); - LOG.info("Updated cookie file present in journalDirectory {}", journalDirectory); - } - for (File dir : ledgerDirectories) { - newCookie.writeToDirectory(dir); - } - LOG.info("Updated cookie file present in ledgerDirectories {}", (Object) ledgerDirectories); - if (ledgerDirectories != indexDirectories) { - for (File dir : indexDirectories) { - newCookie.writeToDirectory(dir); - } - LOG.info("Updated cookie file present in indexDirectories {}", (Object) indexDirectories); - } - } - // writes newcookie to zookeeper - conf.setUseHostNameAsBookieID(useHostname); - newCookie.writeToRegistrationManager(rm, conf, Version.NEW); - - // delete oldcookie - conf.setUseHostNameAsBookieID(!useHostname); - oldCookie.getValue().deleteFromRegistrationManager(rm, conf, oldCookie.getVersion()); - return 0; - } catch (IOException | BookieException ioe) { - LOG.error("IOException during cookie updation!", ioe); - return -1; - } - }); - } - - private int expandStorage() throws Exception { - return runFunctionWithMetadataBookieDriver(bkConf, driver -> { - List allLedgerDirs = Lists.newArrayList(); - allLedgerDirs.addAll(Arrays.asList(ledgerDirectories)); - if (indexDirectories != ledgerDirectories) { - allLedgerDirs.addAll(Arrays.asList(indexDirectories)); - } - - try { - Bookie.checkEnvironmentWithStorageExpansion( - bkConf, driver, Arrays.asList(journalDirectories), allLedgerDirs); - return 0; - } catch (BookieException e) { - LOG.error("Exception while updating cookie for storage expansion", e); - return -1; - } - }); - } - - private boolean verifyCookie(Cookie oldCookie, File dir) throws IOException { - try { - Cookie cookie = Cookie.readFromDirectory(dir); - cookie.verify(oldCookie); - } catch (InvalidCookieException e) { - return false; - } - return true; - } - - private int listOrDeleteCookies(boolean delete, boolean force) throws Exception { - BookieSocketAddress bookieAddress = Bookie.getBookieAddress(bkConf); - File[] journalDirs = bkConf.getJournalDirs(); - File[] ledgerDirs = bkConf.getLedgerDirs(); - File[] indexDirs = bkConf.getIndexDirs(); - File[] allDirs = ArrayUtils.addAll(journalDirs, ledgerDirs); - if (indexDirs != null) { - allDirs = ArrayUtils.addAll(allDirs, indexDirs); - } - - File[] allCurDirs = Bookie.getCurrentDirectories(allDirs); - List allVersionFiles = new LinkedList(); - File versionFile; - for (File curDir : allCurDirs) { - versionFile = new File(curDir, BookKeeperConstants.VERSION_FILENAME); - if (versionFile.exists()) { - allVersionFiles.add(versionFile); - } - } - - if (!allVersionFiles.isEmpty()) { - if (delete) { - boolean confirm = force; - if (!confirm) { - confirm = IOUtils.confirmPrompt("Are you sure you want to delete Cookies locally?"); - } - if (confirm) { - for (File verFile : allVersionFiles) { - if (!verFile.delete()) { - LOG.error( - "Failed to delete Local cookie file {}. So aborting deletecookie of Bookie: {}", - verFile, bookieAddress); - return -1; - } - } - LOG.info("Deleted Local Cookies of Bookie: {}", bookieAddress); - } else { - LOG.info("Skipping deleting local Cookies of Bookie: {}", bookieAddress); - } - } else { - LOG.info("Listing local Cookie Files of Bookie: {}", bookieAddress); - for (File verFile : allVersionFiles) { - LOG.info(verFile.getCanonicalPath()); - } - } - } else { - LOG.info("No local cookies for Bookie: {}", bookieAddress); - } - - return runFunctionWithRegistrationManager(bkConf, rm -> { - try { - Versioned cookie = null; - try { - cookie = Cookie.readFromRegistrationManager(rm, bookieAddress); - } catch (CookieNotFoundException nne) { - LOG.info("No cookie for {} in metadata store", bookieAddress); - return 0; - } - - if (delete) { - boolean confirm = force; - if (!confirm) { - confirm = IOUtils.confirmPrompt( - "Are you sure you want to delete Cookies from metadata store?"); - } - - if (confirm) { - cookie.getValue().deleteFromRegistrationManager(rm, bkConf, cookie.getVersion()); - LOG.info("Deleted Cookie from metadata store for Bookie: {}", bookieAddress); - } else { - LOG.info("Skipping deleting cookie from metadata store for Bookie: {}", bookieAddress); - } - } - } catch (BookieException | IOException e) { - return -1; - } - return 0; - }); + boolean result = cmd.apply(bkConf, flags); + return (result) ? 0 : -1; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java new file mode 100644 index 00000000000..dce11ce1dfe --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java @@ -0,0 +1,326 @@ +/* + * 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.tools.cli.commands.cookie; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithMetadataBookieDriver; +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; + +import com.beust.jcommander.Parameter; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.BookKeeperConstants; +import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; +import org.apache.commons.lang3.ArrayUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to update cookie. + */ +public class AdminCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(AdminCommand.class); + + private static final String NAME = "admin"; + private static final String DESC = "Command to update cookie"; + + private File[] journalDirectories; + private File[] ledgerDirectories; + private File[] indexDirectories; + + public AdminCommand() { + this(new AdminFlags()); + } + + private AdminCommand(AdminFlags flags) { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build()); + } + + /** + * Flags for admin command. + */ + @Accessors(fluent = true) + @Setter + public static class AdminFlags extends CliFlags { + + @Parameter(names = { "-host", + "--hostname" }, description = "Expects config useHostNameAsBookieID=true as the option value") + private boolean hostname; + + @Parameter(names = { "-p", "-ip" }, + description = "Expects config useHostNameAsBookieID=false as the option value") + private boolean ip; + + @Parameter(names = { "-e", "--expandstorage" }, description = "Add new empty ledger/index directories") + private boolean expandstorage; + + @Parameter(names = { "-l", "--list" }, description = "List paths of all the cookies present locally and on " + + "zooKeeper") + private boolean list; + + @Parameter(names = { "-d", "--delete" }, description = "Delete cookie both locally and in zooKeeper") + private boolean delete; + + @Parameter(names = {"-f", "--force"}, description = "Force delete cookie") + private boolean force; + + } + + @Override + public boolean apply(ServerConfiguration conf, AdminFlags cmdFlags) { + initDirectory(conf); + try { + return update(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private void initDirectory(ServerConfiguration bkConf) { + this.journalDirectories = Bookie.getCurrentDirectories(bkConf.getJournalDirs()); + this.ledgerDirectories = Bookie.getCurrentDirectories(bkConf.getLedgerDirs()); + if (null == bkConf.getIndexDirs()) { + this.indexDirectories = this.ledgerDirectories; + } else { + this.indexDirectories = Bookie.getCurrentDirectories(bkConf.getIndexDirs()); + } + } + + private boolean update(ServerConfiguration conf, AdminFlags flags) throws Exception { + boolean useHostName = flags.hostname; + if (flags.hostname || flags.ip) { + if (!conf.getUseHostNameAsBookieID() && useHostName) { + LOG.error("Expects configuration useHostNameAsBookieID=true as the option value"); + return false; + } else if (conf.getUseHostNameAsBookieID() && !useHostName) { + LOG.error("Expects configuration useHostNameAsBookieID=false as the option value"); + return false; + } + return updateBookieIdInCookie(conf, flags.hostname); + } else if (flags.expandstorage) { + conf.setAllowStorageExpansion(true); + return expandStorage(conf); + } else if (flags.list) { + return listOrDeleteCookies(conf, false, false); + } else if (flags.delete) { + return listOrDeleteCookies(conf, true, flags.force); + } else { + LOG.error("Invalid command !"); + usage(); + return false; + } + } + + private boolean updateBookieIdInCookie(ServerConfiguration bkConf, final boolean useHostname) + throws Exception { + return runFunctionWithRegistrationManager(bkConf, rm -> { + try { + ServerConfiguration conf = new ServerConfiguration(bkConf); + String newBookieId = Bookie.getBookieAddress(conf).toString(); + // read oldcookie + Versioned oldCookie = null; + try { + conf.setUseHostNameAsBookieID(!useHostname); + oldCookie = Cookie.readFromRegistrationManager(rm, conf); + } catch (BookieException.CookieNotFoundException nne) { + LOG.error("Either cookie already updated with UseHostNameAsBookieID={} or no cookie exists!", + useHostname, nne); + return false; + } + Cookie newCookie = Cookie.newBuilder(oldCookie.getValue()).setBookieHost(newBookieId).build(); + + boolean hasCookieUpdatedInDirs = verifyCookie(newCookie, journalDirectories[0]); + for (File dir : ledgerDirectories) { + hasCookieUpdatedInDirs &= verifyCookie(newCookie, dir); + } + if (indexDirectories != ledgerDirectories) { + for (File dir : indexDirectories) { + hasCookieUpdatedInDirs &= verifyCookie(newCookie, dir); + } + } + + if (hasCookieUpdatedInDirs) { + try { + conf.setUseHostNameAsBookieID(useHostname); + Cookie.readFromRegistrationManager(rm, conf); + // since newcookie exists, just do cleanup of oldcookie and return + conf.setUseHostNameAsBookieID(!useHostname); + oldCookie.getValue().deleteFromRegistrationManager(rm, conf, oldCookie.getVersion()); + return true; + } catch (BookieException.CookieNotFoundException nne) { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring, cookie will be written to zookeeper"); + } + } + } else { + // writes newcookie to local dirs + for (File journalDirectory : journalDirectories) { + newCookie.writeToDirectory(journalDirectory); + LOG.info("Updated cookie file present in journalDirectory {}", journalDirectory); + } + for (File dir : ledgerDirectories) { + newCookie.writeToDirectory(dir); + } + LOG.info("Updated cookie file present in ledgerDirectories {}", (Object) ledgerDirectories); + if (ledgerDirectories != indexDirectories) { + for (File dir : indexDirectories) { + newCookie.writeToDirectory(dir); + } + LOG.info("Updated cookie file present in indexDirectories {}", (Object) indexDirectories); + } + } + // writes newcookie to zookeeper + conf.setUseHostNameAsBookieID(useHostname); + newCookie.writeToRegistrationManager(rm, conf, Version.NEW); + + // delete oldcookie + conf.setUseHostNameAsBookieID(!useHostname); + oldCookie.getValue().deleteFromRegistrationManager(rm, conf, oldCookie.getVersion()); + return true; + } catch (IOException | BookieException ioe) { + LOG.error("IOException during cookie updation!", ioe); + return false; + } + }); + } + + private boolean verifyCookie(Cookie oldCookie, File dir) throws IOException { + try { + Cookie cookie = Cookie.readFromDirectory(dir); + cookie.verify(oldCookie); + } catch (BookieException.InvalidCookieException e) { + return false; + } + return true; + } + + private boolean expandStorage(ServerConfiguration bkConf) throws Exception { + return runFunctionWithMetadataBookieDriver(bkConf, driver -> { + List allLedgerDirs = Lists.newArrayList(); + allLedgerDirs.addAll(Arrays.asList(ledgerDirectories)); + if (indexDirectories != ledgerDirectories) { + allLedgerDirs.addAll(Arrays.asList(indexDirectories)); + } + + try { + Bookie.checkEnvironmentWithStorageExpansion(bkConf, driver, Arrays.asList(journalDirectories), + allLedgerDirs); + return true; + } catch (BookieException e) { + LOG.error("Exception while updating cookie for storage expansion", e); + return false; + } + }); + } + + private boolean listOrDeleteCookies(ServerConfiguration bkConf, boolean delete, boolean force) throws Exception { + BookieSocketAddress bookieAddress = Bookie.getBookieAddress(bkConf); + File[] journalDirs = bkConf.getJournalDirs(); + File[] ledgerDirs = bkConf.getLedgerDirs(); + File[] indexDirs = bkConf.getIndexDirs(); + File[] allDirs = ArrayUtils.addAll(journalDirs, ledgerDirs); + if (indexDirs != null) { + allDirs = ArrayUtils.addAll(allDirs, indexDirs); + } + + File[] allCurDirs = Bookie.getCurrentDirectories(allDirs); + List allVersionFiles = new LinkedList(); + File versionFile; + for (File curDir : allCurDirs) { + versionFile = new File(curDir, BookKeeperConstants.VERSION_FILENAME); + if (versionFile.exists()) { + allVersionFiles.add(versionFile); + } + } + + if (!allVersionFiles.isEmpty()) { + if (delete) { + boolean confirm = force; + if (!confirm) { + confirm = IOUtils.confirmPrompt("Are you sure you want to delete Cookies locally?"); + } + if (confirm) { + for (File verFile : allVersionFiles) { + if (!verFile.delete()) { + LOG.error("Failed to delete Local cookie file {}. So aborting deletecookie of Bookie: {}", + verFile, bookieAddress); + return false; + } + } + LOG.info("Deleted Local Cookies of Bookie: {}", bookieAddress); + } else { + LOG.info("Skipping deleting local Cookies of Bookie: {}", bookieAddress); + } + } else { + LOG.info("Listing local Cookie Files of Bookie: {}", bookieAddress); + for (File verFile : allVersionFiles) { + LOG.info(verFile.getCanonicalPath()); + } + } + } else { + LOG.info("No local cookies for Bookie: {}", bookieAddress); + } + + return runFunctionWithRegistrationManager(bkConf, rm -> { + try { + Versioned cookie = null; + try { + cookie = Cookie.readFromRegistrationManager(rm, bookieAddress); + } catch (BookieException.CookieNotFoundException nne) { + LOG.info("No cookie for {} in metadata store", bookieAddress); + return true; + } + + if (delete) { + boolean confirm = force; + if (!confirm) { + confirm = IOUtils.confirmPrompt("Are you sure you want to delete Cookies from metadata store?"); + } + + if (confirm) { + cookie.getValue().deleteFromRegistrationManager(rm, bkConf, cookie.getVersion()); + LOG.info("Deleted Cookie from metadata store for Bookie: {}", bookieAddress); + } else { + LOG.info("Skipping deleting cookie from metadata store for Bookie: {}", bookieAddress); + } + } + } catch (BookieException | IOException e) { + return false; + } + return true; + }); + } +} diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java index 3dded4e5cb5..83dfc26b60a 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/CookieCommandGroup.java @@ -20,6 +20,7 @@ package org.apache.bookkeeper.tools.cli.commands; import org.apache.bookkeeper.tools.cli.BKCtl; +import org.apache.bookkeeper.tools.cli.commands.cookie.AdminCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.DeleteCookieCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.GenerateCookieCommand; @@ -46,6 +47,7 @@ public class CookieCommandGroup extends CliCommandGroup { .addCommand(new GetCookieCommand()) .addCommand(new UpdateCookieCommand()) .addCommand(new GenerateCookieCommand()) + .addCommand(new AdminCommand()) .build(); public CookieCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java new file mode 100644 index 00000000000..9f39867981c --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java @@ -0,0 +1,218 @@ +/* + * 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.tools.cli.commands.cookie; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.verify; +import static org.mockito.internal.verification.VerificationModeFactory.times; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyNew; +import static org.powermock.api.mockito.PowerMockito.when; + +import java.io.File; +import java.io.IOException; +import java.net.UnknownHostException; +import java.util.function.Function; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.Cookie; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.util.BookKeeperConstants; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link AdminCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ AdminCommand.class, MetadataDrivers.class, Cookie.class, Bookie.class, RegistrationManager.class }) +public class AdminCommandTest extends BookieCommandTestBase { + + @Mock + private ServerConfiguration serverConfiguration; + + @Mock + private Versioned cookieVersioned; + + @Mock + private Cookie cookie; + + @Mock + private BookieSocketAddress bookieSocketAddress; + + public AdminCommandTest() throws IOException { + super(3, 3); + } + + public void createIndex() throws IOException { + String[] indexDirs = new String[3]; + for (int i = 0; i < indexDirs.length; i++) { + File dir = this.testDir.newFile(); + dir.mkdirs(); + indexDirs[i] = dir.getAbsolutePath(); + } + this.conf.setIndexDirName(indexDirs); + + } + @Override + public void setup() throws Exception { + super.setup(); + createIndex(); + + PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); + PowerMockito.mockStatic(MetadataDrivers.class); + PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class) + .withArguments(eq(conf)).thenReturn(serverConfiguration); + PowerMockito.mockStatic(Cookie.class); + PowerMockito.mockStatic(MetadataDrivers.class); + PowerMockito.mockStatic(Bookie.class); + + mockUpdateBookieIdInCookie(); + mockVerifyCookie(); + mockInitDirecotory(); + mockExpandStorage(); + mockListOrDeleteCookies(); + + } + + private void mockInitDirecotory() throws IOException { + File[] files = new File[1]; + files[0] = testDir.getRoot(); + testDir.newFile(BookKeeperConstants.VERSION_FILENAME); + PowerMockito.when(Bookie.getCurrentDirectories(any())).thenReturn(files); + } + + private void mockUpdateBookieIdInCookie() throws Exception { + RegistrationManager registrationManager = mock(RegistrationManager.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function f = invocationOnMock.getArgument(1); + f.apply(registrationManager); + return true; + }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", any(ServerConfiguration.class), + any(Function.class)); + + PowerMockito.when(Bookie.getBookieAddress(eq(serverConfiguration))).thenReturn(bookieSocketAddress); + when(bookieSocketAddress.toString()).thenReturn("1"); + PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(serverConfiguration))) + .thenReturn(cookieVersioned); + PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress))) + .thenReturn(cookieVersioned); + when(cookieVersioned.getValue()).thenReturn(cookie); + Cookie.Builder builder = mock(Cookie.Builder.class); + PowerMockito.when(Cookie.newBuilder(eq(cookie))).thenReturn(builder); + PowerMockito.when(builder.setBookieHost(anyString())).thenReturn(builder); + when(builder.build()).thenReturn(cookie); + + PowerMockito.when(serverConfiguration.setUseHostNameAsBookieID(anyBoolean())).thenReturn(serverConfiguration); + PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(serverConfiguration))) + .thenReturn(cookieVersioned); + + Version version = mock(Version.class); + when(cookieVersioned.getVersion()).thenReturn(version); + when(cookieVersioned.getValue()).thenReturn(cookie); + doNothing().when(cookie) + .deleteFromRegistrationManager(eq(registrationManager), eq(serverConfiguration), eq(version)); + + doNothing().when(cookie).writeToDirectory(any(File.class)); + doNothing().when(cookie) + .writeToRegistrationManager(eq(registrationManager), eq(serverConfiguration), eq(Version.NEW)); + + doNothing().when(cookie) + .deleteFromRegistrationManager(eq(registrationManager), any(ServerConfiguration.class), eq(version)); + } + + private void mockVerifyCookie() throws IOException, BookieException.InvalidCookieException { + PowerMockito.when(Cookie.readFromDirectory(any(File.class))).thenReturn(cookie); + doNothing().when(cookie).verify(any(Cookie.class)); + } + + private void mockExpandStorage() throws Exception { + MetadataBookieDriver metadataBookieDriver = mock(MetadataBookieDriver.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function f = invocationOnMock.getArgument(1); + f.apply(metadataBookieDriver); + return true; + }).when(MetadataDrivers.class, "runFunctionWithMetadataBookieDriver", any(ServerConfiguration.class), + any(Function.class)); + PowerMockito.doNothing() + .when(Bookie.class, "checkEnvironmentWithStorageExpansion", any(ServerConfiguration.class), + eq(metadataBookieDriver), any(), any()); + } + + private void mockListOrDeleteCookies() throws UnknownHostException { + + when(Bookie.getBookieAddress(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress); + } + + @Test + public void testWithoutAnyFlags() { + AdminCommand cmd = new AdminCommand(); + Assert.assertFalse(cmd.apply(bkFlags, new String[] {""})); + } + + @Test + public void testWithHostName() throws Exception { + conf.setUseHostNameAsBookieID(true); + testCommand("-host"); + verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(conf)); + verify(serverConfiguration, times(3)).setUseHostNameAsBookieID(anyBoolean()); + + verify(cookie, times(2)).verify(any(Cookie.class)); + } + + @Test + public void testWithExpand() { + testCommand("-e"); + } + + @Test + public void testWithList() { + testCommand("-l"); + } + + @Test + public void testWithDelete() throws BookieException { + testCommand("-d", "-f"); + verify(cookie, times(1)) + .deleteFromRegistrationManager(any(RegistrationManager.class), any(ServerConfiguration.class), + any(Version.class)); + } + + private void testCommand(String... args) { + AdminCommand cmd = new AdminCommand(); + Assert.assertTrue(cmd.apply(bkFlags, args)); + } +} From 79aa7f27fbc5e051f6b57fafc96a6dfa848de521 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 19 Apr 2019 02:32:02 +0200 Subject: [PATCH 0276/1642] Wait for LAC update even if ledger fenced Previous behaviour was to return straight away. However, LAC can change when the ledger is fenced and there is no guarantee that a fenced ledger will turn into a closed ledger (fencing client may crash), which would cause the client tailing with longpoll LAC to go into a tight loop. Reviewers: Enrico Olivelli , Jia Zhai , Matteo Merli , Venkateswararao Jujjuri (JV) , Sijie Guo This closes #2052 from ivankelly/wait-on-fenced --- .../src/main/java/org/apache/bookkeeper/bookie/FileInfo.java | 3 +-- .../bookkeeper/bookie/storage/ldb/TransientLedgerInfo.java | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java index 656674f5e03..f9bcd285e3b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java @@ -139,8 +139,7 @@ long setLastAddConfirmed(long lac) { synchronized boolean waitForLastAddConfirmedUpdate(long previousLAC, Watcher watcher) { - if ((null != lac && lac > previousLAC) - || isClosed || ((stateBits & STATE_FENCED_BIT) == STATE_FENCED_BIT)) { + if ((null != lac && lac > previousLAC) || isClosed) { if (LOG.isTraceEnabled()) { LOG.trace("Wait For LAC {} , {}", this.lac, previousLAC); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/TransientLedgerInfo.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/TransientLedgerInfo.java index 27d63e8dbfd..91f3fbb52d3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/TransientLedgerInfo.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/TransientLedgerInfo.java @@ -91,7 +91,7 @@ long setLastAddConfirmed(long lac) { synchronized boolean waitForLastAddConfirmedUpdate(long previousLAC, Watcher watcher) throws IOException { lastAccessed = System.currentTimeMillis(); - if ((lac != NOT_ASSIGNED_LAC && lac > previousLAC) || isClosed || ledgerIndex.get(ledgerId).getFenced()) { + if ((lac != NOT_ASSIGNED_LAC && lac > previousLAC) || isClosed) { return false; } @@ -153,4 +153,4 @@ public void close() { notifyWatchers(Long.MAX_VALUE); } -} \ No newline at end of file +} From f91b06402c857184b5fa3bfd4d878c0d260e6e5c Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 19 Apr 2019 02:40:07 +0200 Subject: [PATCH 0277/1642] k Otherwise the watch objects will accumulate and eventually cause an OOM on the bookie, if LAC doesn't progress. Reviewers: Enrico Olivelli , Matteo Merli , Venkateswararao Jujjuri (JV) This closes #2051 from ivankelly/tme-oom --- .../org/apache/bookkeeper/bookie/Bookie.java | 7 ++ .../apache/bookkeeper/bookie/FileInfo.java | 4 + .../bookie/IndexPersistenceMgr.java | 13 ++ .../bookie/InterleavedLedgerStorage.java | 6 + .../InterleavedStorageRegenerateIndexOp.java | 6 + .../apache/bookkeeper/bookie/LedgerCache.java | 2 + .../bookkeeper/bookie/LedgerCacheImpl.java | 7 ++ .../bookkeeper/bookie/LedgerDescriptor.java | 2 + .../bookie/LedgerDescriptorImpl.java | 5 + .../bookkeeper/bookie/LedgerStorage.java | 10 ++ .../bookie/SortedLedgerStorage.java | 7 ++ .../bookie/storage/ldb/DbLedgerStorage.java | 7 ++ .../ldb/SingleDirectoryDbLedgerStorage.java | 7 ++ .../storage/ldb/TransientLedgerInfo.java | 5 + .../proto/LongPollReadEntryProcessorV3.java | 9 +- .../proto/ReadEntryProcessorV3.java | 4 +- .../bookkeeper/bookie/TestSyncThread.java | 6 + .../apache/bookkeeper/meta/GcLedgersTest.java | 6 + .../meta/LedgerManagerTestCase.java | 6 + .../LongPollReadEntryProcessorV3Test.java | 114 ++++++++++++++++++ 20 files changed, 227 insertions(+), 6 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.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 06ccd588b48..e253fdcee6d 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 @@ -1409,6 +1409,13 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, return handle.waitForLastAddConfirmedUpdate(previousLAC, watcher); } + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId); + handle.cancelWaitForLastAddConfirmedUpdate(watcher); + } + @VisibleForTesting public LedgerStorage getLedgerStorage() { return ledgerStorage; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java index f9bcd285e3b..e8571be6fdc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java @@ -150,6 +150,10 @@ synchronized boolean waitForLastAddConfirmedUpdate(long previousLAC, return true; } + synchronized void cancelWaitForLastAddConfirmedUpdate(Watcher watcher) { + deleteWatcher(watcher); + } + public boolean isClosed() { return isClosed; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java index 8d8f5cfa399..b728df38454 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java @@ -374,6 +374,19 @@ boolean waitForLastAddConfirmedUpdate(long ledgerId, } } + void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) throws IOException { + CachedFileInfo fi = null; + try { + fi = getFileInfo(ledgerId, null); + fi.cancelWaitForLastAddConfirmedUpdate(watcher); + } finally { + if (null != fi) { + fi.release(); + } + } + } + long updateLastAddConfirmed(long ledgerId, long lac) throws IOException { CachedFileInfo fi = null; try { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java index 59ea9ec0374..becb16bd1ea 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java @@ -352,6 +352,12 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, return ledgerCache.waitForLastAddConfirmedUpdate(ledgerId, previousLAC, watcher); } + @Override + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + ledgerCache.cancelWaitForLastAddConfirmedUpdate(ledgerId, watcher); + } @Override public long addEntry(ByteBuf entry) throws IOException { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java index f5e7c17baa7..880d3c190be 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java @@ -217,6 +217,12 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, throw new UnsupportedOperationException(); } @Override + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + throw new UnsupportedOperationException(); + } + @Override public void deleteLedger(long ledgerId) throws IOException { } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java index 993bd0cd007..cae8bb42eac 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java @@ -53,6 +53,8 @@ public interface LedgerCache extends Closeable { boolean waitForLastAddConfirmedUpdate(long ledgerId, long previousLAC, Watcher watcher) throws IOException; + void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) throws IOException; void deleteLedger(long ledgerId) throws IOException; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java index 119a4f46a89..e6de2f9a43c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java @@ -91,6 +91,13 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, return indexPersistenceManager.waitForLastAddConfirmedUpdate(ledgerId, previousLAC, watcher); } + @Override + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + indexPersistenceManager.cancelWaitForLastAddConfirmedUpdate(ledgerId, watcher); + } + @Override public void putEntryOffset(long ledger, long entry, long offset) throws IOException { indexPageManager.putEntryOffset(ledger, entry, offset); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java index 23840be279c..74bc8b52cd8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java @@ -80,6 +80,8 @@ static ByteBuf createLedgerFenceEntry(Long ledgerId) { abstract boolean waitForLastAddConfirmedUpdate(long previousLAC, Watcher watcher) throws IOException; + abstract void cancelWaitForLastAddConfirmedUpdate(Watcher watcher) + throws IOException; abstract void setExplicitLac(ByteBuf entry) throws IOException; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java index ee91ed0f0dd..563494e83c1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java @@ -168,4 +168,9 @@ boolean waitForLastAddConfirmedUpdate(long previousLAC, Watcher watcher) throws IOException { return ledgerStorage.waitForLastAddConfirmedUpdate(ledgerId, previousLAC, watcher); } + + @Override + void cancelWaitForLastAddConfirmedUpdate(Watcher watcher) throws IOException { + ledgerStorage.cancelWaitForLastAddConfirmedUpdate(ledgerId, watcher); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java index 1353e8b528e..2c47b2888ff 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java @@ -136,6 +136,16 @@ boolean waitForLastAddConfirmedUpdate(long ledgerId, long previousLAC, Watcher watcher) throws IOException; + /** + * Cancel a previous wait for last add confirmed update. + * + * @param ledgerId The ledger being watched. + * @param watcher The watcher to cancel. + * @throws IOException + */ + void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) throws IOException; + /** * Flushes all data in the storage. Once this is called, * add data written to the LedgerStorage up until this point diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java index 77653db9646..23e07163cb2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java @@ -227,6 +227,13 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, return interleavedLedgerStorage.waitForLastAddConfirmedUpdate(ledgerId, previousLAC, watcher); } + @Override + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + interleavedLedgerStorage.cancelWaitForLastAddConfirmedUpdate(ledgerId, watcher); + } + @Override public void checkpoint(final Checkpoint checkpoint) throws IOException { long numBytesFlushed = memTable.flush(this, checkpoint); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index e1183490177..a68cd1cd7b7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -209,6 +209,13 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, long previousLAC, return getLedgerSorage(ledgerId).waitForLastAddConfirmedUpdate(ledgerId, previousLAC, watcher); } + @Override + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + getLedgerSorage(ledgerId).cancelWaitForLastAddConfirmedUpdate(ledgerId, watcher); + } + @Override public void flush() throws IOException { for (LedgerStorage ls : ledgerStorageList) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 3289b3d2f1b..4c4123515f4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -756,6 +756,13 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, long previousLAC, return getOrAddLedgerInfo(ledgerId).waitForLastAddConfirmedUpdate(previousLAC, watcher); } + @Override + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + getOrAddLedgerInfo(ledgerId).cancelWaitForLastAddConfirmedUpdate(watcher); + } + @Override public void setExplicitlac(long ledgerId, ByteBuf lac) throws IOException { getOrAddLedgerInfo(ledgerId).setExplicitLac(lac); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/TransientLedgerInfo.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/TransientLedgerInfo.java index 91f3fbb52d3..17a84624d54 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/TransientLedgerInfo.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/TransientLedgerInfo.java @@ -99,6 +99,11 @@ synchronized boolean waitForLastAddConfirmedUpdate(long previousLAC, return true; } + synchronized void cancelWaitForLastAddConfirmedUpdate(Watcher watcher) + throws IOException { + deleteWatcher(watcher); + } + public ByteBuf getExplicitLac() { ByteBuf retLac = null; synchronized (this) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java index 6f25d684b6c..c04ecc92005 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java @@ -142,7 +142,7 @@ private ReadResponse getLongPollReadResponse() { final boolean watched; try { - watched = requestProcessor.bookie.waitForLastAddConfirmedUpdate(ledgerId, previousLAC, this); + watched = requestProcessor.getBookie().waitForLastAddConfirmedUpdate(ledgerId, previousLAC, this); } catch (Bookie.NoLedgerException e) { logger.info("No ledger found while longpoll reading ledger {}, previous lac = {}.", ledgerId, previousLAC); @@ -163,9 +163,10 @@ private ReadResponse getLongPollReadResponse() { } synchronized (this) { expirationTimerTask = requestTimer.newTimeout(timeout -> { - // When the timeout expires just get whatever is the current - // readLastConfirmed - LongPollReadEntryProcessorV3.this.scheduleDeferredRead(true); + requestProcessor.getBookie().cancelWaitForLastAddConfirmedUpdate(ledgerId, this); + // When the timeout expires just get whatever is the current + // readLastConfirmed + LongPollReadEntryProcessorV3.this.scheduleDeferredRead(true); }, readRequest.getTimeOut(), TimeUnit.MILLISECONDS); } return null; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java index 88b76627071..a8ecc11d7e9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java @@ -174,7 +174,7 @@ protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, boolean readLACPiggyBack, Stopwatch startTimeSw) throws IOException { - ByteBuf entryBody = requestProcessor.bookie.readEntry(ledgerId, entryId); + ByteBuf entryBody = requestProcessor.getBookie().readEntry(ledgerId, entryId); if (null != fenceResult) { handleReadResultForFenceRead(entryBody, readResponseBuilder, entryId, startTimeSw); return null; @@ -184,7 +184,7 @@ protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, if (readLACPiggyBack) { readResponseBuilder.setEntryId(entryId); } else { - long knownLAC = requestProcessor.bookie.readLastAddConfirmed(ledgerId); + long knownLAC = requestProcessor.getBookie().readLastAddConfirmed(ledgerId); readResponseBuilder.setMaxLAC(knownLAC); } registerSuccessfulEvent(readStats, startTimeSw); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java index 707eb817ee4..f57d388d9ad 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java @@ -355,6 +355,12 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, return false; } + @Override + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + } + @Override public void checkpoint(Checkpoint checkpoint) throws IOException { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index abc66144a72..37eacf4d5a7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -678,5 +678,11 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, throws IOException { return false; } + + @Override + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + } } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java index cf9b3ddb6c2..a673d8c7dae 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java @@ -274,6 +274,12 @@ public boolean waitForLastAddConfirmedUpdate(long ledgerId, return false; } + @Override + public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, + Watcher watcher) + throws IOException { + } + @Override public void setExplicitlac(long ledgerId, ByteBuf lac) throws IOException { } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java new file mode 100644 index 00000000000..393d5ddf33a --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java @@ -0,0 +1,114 @@ +/* + * 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.proto; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.protobuf.ByteString; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.util.HashedWheelTimer; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader; +import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; +import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; +import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadRequest; +import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + + + + +/** + * Unit test {@link LongPollReadEntryProcessorV3}. + */ +public class LongPollReadEntryProcessorV3Test { + ExecutorService executor; + HashedWheelTimer timer; + + @Before + public void setup() { + executor = Executors.newSingleThreadExecutor(); + timer = new HashedWheelTimer(); + } + + @After + public void teardown() { + timer.stop(); + executor.shutdownNow(); + } + + @Test + public void testWatchIsCancelledOnTimeout() throws Exception { + Request request = Request.newBuilder() + .setHeader(BKPacketHeader.newBuilder() + .setTxnId(System.currentTimeMillis()) + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.READ_ENTRY) + .build()) + .setReadRequest(ReadRequest.newBuilder() + .setLedgerId(10) + .setEntryId(1) + .setMasterKey(ByteString.copyFrom(new byte[0])) + .setPreviousLAC(0) + .setTimeOut(1) + .build()) + .build(); + + Channel channel = mock(Channel.class); + Bookie bookie = mock(Bookie.class); + + BookieRequestProcessor requestProcessor = mock(BookieRequestProcessor.class); + when(requestProcessor.getBookie()).thenReturn(bookie); + when(requestProcessor.getRequestStats()).thenReturn(new RequestStats(NullStatsLogger.INSTANCE)); + + when(bookie.waitForLastAddConfirmedUpdate(anyLong(), anyLong(), any())) + .thenReturn(true); + when(bookie.readEntry(anyLong(), anyLong())).thenReturn(Unpooled.buffer()); + when(bookie.readLastAddConfirmed(anyLong())).thenReturn(Long.valueOf(1)); + + CompletableFuture cancelFuture = new CompletableFuture<>(); + + doAnswer(invocationOnMock -> { + cancelFuture.complete(null); + return null; + }).when(bookie).cancelWaitForLastAddConfirmedUpdate(anyLong(), any()); + + LongPollReadEntryProcessorV3 processor = new LongPollReadEntryProcessorV3( + request, + channel, + requestProcessor, + executor, executor, timer); + + processor.run(); + + cancelFuture.get(10, TimeUnit.SECONDS); + } +} From 051c4a9ec2a5f2ea2d4adcba0ff0664597f06d30 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 19 Apr 2019 02:43:05 +0200 Subject: [PATCH 0278/1642] Fix typo in overview page for 4.8.2 The release notes link text was saying the latest version rather than 4.8.2. Reviewers: Enrico Olivelli , Sijie Guo This closes #2048 from ivankelly/doc-typo-relnotes --- site/docs/4.8.2/overview/overview.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/docs/4.8.2/overview/overview.md b/site/docs/4.8.2/overview/overview.md index 808693803bc..c3a3fd7a592 100644 --- a/site/docs/4.8.2/overview/overview.md +++ b/site/docs/4.8.2/overview/overview.md @@ -37,7 +37,7 @@ Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Stori Learn more about Apache BookKeeper™ and what it can do for your organization: -- [Apache BookKeeper {{ site.latest_version }} Release Notes](../releaseNotes) +- [Apache BookKeeper 4.8.2 Release Notes](../releaseNotes) - [Java API docs](../../api/javadoc) Or start [using](../../getting-started/installation) Apache BookKeeper today. From 232b7ef4d151d9f898af222923dcae7af512ce42 Mon Sep 17 00:00:00 2001 From: Like Date: Fri, 19 Apr 2019 08:44:54 +0800 Subject: [PATCH 0279/1642] Support asynchronous fence request for V2 ReadEntryProcessor Currently, the ```ReadEntryProcessor``` v2 does not support asynchronous fence request, which wait for 1000 milliseconds to wait it done if it's a fencing request. This pull request attempt to support asynchronous response if a thread pool is provided. Closes #283 Reviewers: Enrico Olivelli , Sijie Guo This closes #2021 from liketic/refactor --- .../proto/BookieRequestProcessor.java | 4 +- .../bookkeeper/proto/ReadEntryProcessor.java | 123 +++++++----- .../proto/ReadEntryProcessorTest.java | 180 ++++++++++++++++++ 3 files changed, 259 insertions(+), 48 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java 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 7d5e2e613ed..02b0f568581 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 @@ -619,7 +619,9 @@ private void processAddRequest(final BookieProtocol.ParsedAddRequest r, final Ch } private void processReadRequest(final BookieProtocol.ReadRequest r, final Channel c) { - ReadEntryProcessor read = ReadEntryProcessor.create(r, c, this); + ExecutorService fenceThreadPool = + null == highPriorityThreadPool ? null : highPriorityThreadPool.chooseThread(c); + ReadEntryProcessor read = ReadEntryProcessor.create(r, c, this, fenceThreadPool); // If it's a high priority read (fencing or as part of recovery process), we want to make sure it // gets executed as fast as possible, so bypass the normal readThreadPool 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 6566c7b4643..a530bd56b11 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 @@ -17,6 +17,10 @@ */ package org.apache.bookkeeper.proto; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; import io.netty.util.Recycler; @@ -24,24 +28,31 @@ import java.io.IOException; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; +import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.proto.BookieProtocol.ReadRequest; +import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + class ReadEntryProcessor extends PacketProcessorBase { private static final Logger LOG = LoggerFactory.getLogger(ReadEntryProcessor.class); - public static ReadEntryProcessor create(ReadRequest request, Channel channel, - BookieRequestProcessor requestProcessor) { + private ExecutorService fenceThreadPool; + + public static ReadEntryProcessor create(ReadRequest request, + Channel channel, + BookieRequestProcessor requestProcessor, + ExecutorService fenceThreadPool) { ReadEntryProcessor rep = RECYCLER.get(); rep.init(request, channel, requestProcessor); + rep.fenceThreadPool = fenceThreadPool; return rep; } @@ -50,55 +61,29 @@ protected void processPacket() { if (LOG.isDebugEnabled()) { LOG.debug("Received new read request: {}", request); } - int errorCode = BookieProtocol.EIO; + int errorCode = BookieProtocol.EOK; long startTimeNanos = MathUtils.nowInNano(); ByteBuf data = null; try { - Future fenceResult = null; + SettableFuture fenceResult = null; if (request.isFencing()) { LOG.warn("Ledger: {} fenced by: {}", request.getLedgerId(), channel.remoteAddress()); if (request.hasMasterKey()) { - fenceResult = requestProcessor.bookie.fenceLedger(request.getLedgerId(), request.getMasterKey()); + fenceResult = requestProcessor.getBookie().fenceLedger(request.getLedgerId(), + request.getMasterKey()); } else { LOG.error("Password not provided, Not safe to fence {}", request.getLedgerId()); throw BookieException.create(BookieException.Code.UnauthorizedAccessException); } } - data = requestProcessor.bookie.readEntry(request.getLedgerId(), request.getEntryId()); + data = requestProcessor.getBookie().readEntry(request.getLedgerId(), request.getEntryId()); if (LOG.isDebugEnabled()) { LOG.debug("##### Read entry ##### {} -- ref-count: {}", data.readableBytes(), data.refCnt()); } - if (null != fenceResult) { - // TODO: {@link https://github.com/apache/bookkeeper/issues/283} - // currently we don't have readCallback to run in separated read - // threads. after BOOKKEEPER-429 is complete, we could improve - // following code to make it not wait here - // - // For now, since we only try to wait after read entry. so writing - // to journal and read entry are executed in different thread - // it would be fine. - try { - Boolean fenced = fenceResult.get(1000, TimeUnit.MILLISECONDS); - if (null == fenced || !fenced) { - // if failed to fence, fail the read request to make it retry. - errorCode = BookieProtocol.EIO; - } else { - errorCode = BookieProtocol.EOK; - } - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - LOG.error("Interrupting fence read entry {}", request, ie); - errorCode = BookieProtocol.EIO; - } catch (ExecutionException ee) { - LOG.error("Failed to fence read entry {}", request, ee); - errorCode = BookieProtocol.EIO; - } catch (TimeoutException te) { - LOG.error("Timeout to fence read entry {}", request, te); - errorCode = BookieProtocol.EIO; - } - } else { - errorCode = BookieProtocol.EOK; + if (fenceResult != null) { + handleReadResultForFenceRead(fenceResult, data, startTimeNanos); + return; } } catch (Bookie.NoLedgerException e) { if (LOG.isDebugEnabled()) { @@ -127,22 +112,66 @@ protected void processPacket() { if (LOG.isTraceEnabled()) { LOG.trace("Read entry rc = {} for {}", errorCode, request); } + sendResponse(data, errorCode, startTimeNanos); + } + + private void sendResponse(ByteBuf data, int errorCode, long startTimeNanos) { + final RequestStats stats = requestProcessor.getRequestStats(); + final OpStatsLogger logger = stats.getReadEntryStats(); + BookieProtocol.Response response; if (errorCode == BookieProtocol.EOK) { - requestProcessor.getRequestStats().getReadEntryStats() - .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); - sendResponse(errorCode, ResponseBuilder.buildReadResponse(data, request), - requestProcessor.getRequestStats().getReadRequestStats()); + logger.registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + response = ResponseBuilder.buildReadResponse(data, request); } else { - ReferenceCountUtil.release(data); - - requestProcessor.getRequestStats().getReadEntryStats() - .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); - sendResponse(errorCode, ResponseBuilder.buildErrorResponse(errorCode, request), - requestProcessor.getRequestStats().getReadRequestStats()); + if (data != null) { + ReferenceCountUtil.release(data); + } + logger.registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + response = ResponseBuilder.buildErrorResponse(errorCode, request); } + sendResponse(errorCode, response, stats.getReadRequestStats()); recycle(); } + private void sendFenceResponse(Boolean result, ByteBuf data, long startTimeNanos) { + final int retCode = result != null && result ? BookieProtocol.EOK : BookieProtocol.EIO; + sendResponse(data, retCode, startTimeNanos); + } + + private void handleReadResultForFenceRead(ListenableFuture fenceResult, + ByteBuf data, + long startTimeNanos) { + if (null != fenceThreadPool) { + Futures.addCallback(fenceResult, new FutureCallback() { + @Override + public void onSuccess(Boolean result) { + sendFenceResponse(result, data, startTimeNanos); + } + + @Override + public void onFailure(Throwable t) { + LOG.error("Error processing fence request", t); + // if failed to fence, fail the read request to make it retry. + sendResponse(data, BookieProtocol.EIO, startTimeNanos); + } + }, fenceThreadPool); + } else { + try { + Boolean fenced = fenceResult.get(1000, TimeUnit.MILLISECONDS); + sendFenceResponse(fenced, data, startTimeNanos); + return; + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + LOG.error("Interrupting fence read entry {}", request, ie); + } catch (ExecutionException ee) { + LOG.error("Failed to fence read entry {}", request, ee.getCause()); + } catch (TimeoutException te) { + LOG.error("Timeout to fence read entry {}", request, te); + } + sendResponse(data, BookieProtocol.EIO, startTimeNanos); + } + } + @Override public String toString() { return String.format("ReadEntry(%d, %d)", request.getLedgerId(), request.getEntryId()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java new file mode 100644 index 00000000000..bf73c10f042 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java @@ -0,0 +1,180 @@ +/* + * 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.proto; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.util.concurrent.SettableFuture; +import io.netty.channel.Channel; +import io.netty.channel.ChannelPromise; +import io.netty.channel.DefaultChannelPromise; + +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.proto.BookieProtocol.ReadRequest; +import org.apache.bookkeeper.proto.BookieProtocol.Response; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.junit.Before; +import org.junit.Test; + +/** + * Unit test {@link ReadEntryProcessor}. + */ +public class ReadEntryProcessorTest { + + private Channel channel; + private BookieRequestProcessor requestProcessor; + private Bookie bookie; + + @Before + public void setup() throws IOException, BookieException { + channel = mock(Channel.class); + bookie = mock(Bookie.class); + requestProcessor = mock(BookieRequestProcessor.class); + when(requestProcessor.getBookie()).thenReturn(bookie); + when(requestProcessor.getWaitTimeoutOnBackpressureMillis()).thenReturn(-1L); + when(requestProcessor.getRequestStats()).thenReturn(new RequestStats(NullStatsLogger.INSTANCE)); + when(channel.voidPromise()).thenReturn(mock(ChannelPromise.class)); + when(channel.writeAndFlush(any())).thenReturn(mock(ChannelPromise.class)); + } + + @Test + public void testSuccessfulAsynchronousFenceRequest() throws Exception { + testAsynchronousRequest(true, BookieProtocol.EOK); + } + + @Test + public void testFailedAsynchronousFenceRequest() throws Exception { + testAsynchronousRequest(false, BookieProtocol.EIO); + } + + private void testAsynchronousRequest(boolean result, int errorCode) throws Exception { + SettableFuture fenceResult = SettableFuture.create(); + when(bookie.fenceLedger(anyLong(), any())).thenReturn(fenceResult); + + ChannelPromise promise = new DefaultChannelPromise(channel); + AtomicReference writtenObject = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocationOnMock -> { + writtenObject.set(invocationOnMock.getArgument(0)); + latch.countDown(); + return promise; + }).when(channel).writeAndFlush(any(Response.class), any()); + + ExecutorService service = Executors.newCachedThreadPool(); + long ledgerId = System.currentTimeMillis(); + ReadRequest request = new ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, + 1, BookieProtocol.FLAG_DO_FENCING, new byte[]{}); + ReadEntryProcessor processor = ReadEntryProcessor.create(request, channel, requestProcessor, service); + processor.run(); + + fenceResult.set(result); + latch.await(); + verify(channel, times(1)).writeAndFlush(any(Response.class), any()); + + assertTrue(writtenObject.get() instanceof Response); + Response response = (Response) writtenObject.get(); + assertEquals(1, response.getEntryId()); + assertEquals(ledgerId, response.getLedgerId()); + assertEquals(BookieProtocol.READENTRY, response.getOpCode()); + assertEquals(errorCode, response.getErrorCode()); + } + + @Test + public void testSuccessfulSynchronousFenceRequest() throws Exception { + testSynchronousRequest(true, BookieProtocol.EOK); + } + + @Test + public void testFailedSynchronousFenceRequest() throws Exception { + testSynchronousRequest(false, BookieProtocol.EIO); + } + + private void testSynchronousRequest(boolean result, int errorCode) throws Exception { + SettableFuture fenceResult = SettableFuture.create(); + when(bookie.fenceLedger(anyLong(), any())).thenReturn(fenceResult); + ChannelPromise promise = new DefaultChannelPromise(channel); + AtomicReference writtenObject = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocationOnMock -> { + writtenObject.set(invocationOnMock.getArgument(0)); + latch.countDown(); + return promise; + }).when(channel).writeAndFlush(any(Response.class), any()); + + long ledgerId = System.currentTimeMillis(); + ReadRequest request = new ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, + 1, BookieProtocol.FLAG_DO_FENCING, new byte[]{}); + ReadEntryProcessor processor = ReadEntryProcessor.create(request, channel, requestProcessor, null); + fenceResult.set(result); + processor.run(); + + latch.await(); + verify(channel, times(1)).writeAndFlush(any(Response.class), any()); + + assertTrue(writtenObject.get() instanceof Response); + Response response = (Response) writtenObject.get(); + assertEquals(1, response.getEntryId()); + assertEquals(ledgerId, response.getLedgerId()); + assertEquals(BookieProtocol.READENTRY, response.getOpCode()); + assertEquals(errorCode, response.getErrorCode()); + } + + @Test + public void testNonFenceRequest() throws Exception { + ChannelPromise promise = new DefaultChannelPromise(channel); + AtomicReference writtenObject = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocationOnMock -> { + writtenObject.set(invocationOnMock.getArgument(0)); + latch.countDown(); + return promise; + }).when(channel).writeAndFlush(any(Response.class), any()); + + long ledgerId = System.currentTimeMillis(); + ReadRequest request = new ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, + 1, (short) 0, new byte[]{}); + ReadEntryProcessor processor = ReadEntryProcessor.create(request, channel, requestProcessor, null); + processor.run(); + + latch.await(); + verify(channel, times(1)).writeAndFlush(any(Response.class), any()); + + assertTrue(writtenObject.get() instanceof Response); + Response response = (Response) writtenObject.get(); + assertEquals(1, response.getEntryId()); + assertEquals(ledgerId, response.getLedgerId()); + assertEquals(BookieProtocol.READENTRY, response.getOpCode()); + assertEquals(BookieProtocol.EOK, response.getErrorCode()); + } +} From 45836dddd7d8f81856dc1aad0141c3be49b6efac Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Fri, 19 Apr 2019 10:11:08 -0700 Subject: [PATCH 0280/1642] ISSUE #1967: make ledger creation and removal robust to zk connectionloss Descriptions of the changes in this PR: The bookkeeper project ZooKeeperClient wrapper for the ZooKeeper client will resend zk node creations and removals upon reconnect after a ConnectionLoss event. In the event that the original succeeded, the resent operation will erroneously return LedgerExistException or NoSuchLedgerExistsException for creation and removal respectively. For removal, this patch limits the operation by allowing it to always succeed if the ledger does not exist in order to make it idempotent. This is appears to be the simplest solution as exclusive removal isn't important. **Note, the above is an actual change to the bk client semantics** For creation, exclusive creation is cleary important for correctness, so this patch adds a creator token field to the LedgerMetdata to disambiguate the above race from a real race. For AbstractZkLedgerManager, this is simply a random long value. There's an oportunity for optimization with the above if exclusive ledger creation failures are expected to be common. You only actually need to perform this check if the operation was really resent. I chose not to go this route yet because it would require messing with the ZooKeeperClient interface to surface that information without burdening other callers. If the client is set to version 2 or older, this field will be ignored and the old behavior will be retained. If the client is version 3 or newer but creation races with an older client, the new client will interpret the nonce to be BLANK and thereby detect the race correctly. Reviewers: Enrico Olivelli , Sijie Guo This closes #2006 from reddycharan/zkretrialrobust, closes #1967 --- .../src/main/proto/DataFormats.proto | 2 + .../client/LedgerMetadataBuilder.java | 9 ++ .../bookkeeper/client/LedgerMetadataImpl.java | 10 ++ .../bookkeeper/client/api/LedgerMetadata.java | 7 + .../meta/AbstractZkLedgerManager.java | 54 ++++++- .../bookkeeper/meta/LedgerMetadataSerDe.java | 8 +- .../bookkeeper/zookeeper/ZooKeeperClient.java | 4 +- .../bookkeeper/client/BookKeeperTest.java | 142 +++++++++++++++++- .../meta/AbstractZkLedgerManagerTest.java | 5 +- .../meta/TestLedgerMetadataSerDe.java | 4 +- .../TestCompatOldClients.groovy | 38 ++++- .../TestCompatUpgradeYahooCustom.groovy | 20 ++- 12 files changed, 284 insertions(+), 19 deletions(-) diff --git a/bookkeeper-proto/src/main/proto/DataFormats.proto b/bookkeeper-proto/src/main/proto/DataFormats.proto index a679248481b..2501affe9b9 100644 --- a/bookkeeper-proto/src/main/proto/DataFormats.proto +++ b/bookkeeper-proto/src/main/proto/DataFormats.proto @@ -60,6 +60,8 @@ message LedgerMetadataFormat { optional bytes value = 2; } repeated cMetadataMapEntry customMetadata = 11; + + optional int64 cToken = 12; } message LedgerRereplicationLayoutFormat { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index 0c80315f4da..e4f75cedfac 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -63,6 +63,9 @@ public class LedgerMetadataBuilder { private boolean storeCtime = false; private Map customMetadata = Collections.emptyMap(); + private static final long BLANK_CTOKEN = 0; + private long cToken = BLANK_CTOKEN; + public static LedgerMetadataBuilder create() { return new LedgerMetadataBuilder(); } @@ -181,6 +184,11 @@ public LedgerMetadataBuilder storingCreationTime(boolean storing) { return this; } + public LedgerMetadataBuilder withCToken(long cToken) { + this.cToken = cToken; + return this; + } + public LedgerMetadata build() { checkArgument(ensembleSize >= writeQuorumSize, "Write quorum must be less or equal to ensemble size"); checkArgument(writeQuorumSize >= ackQuorumSize, "Write quorum must be greater or equal to ack quorum"); @@ -189,6 +197,7 @@ public LedgerMetadata build() { ensembleSize, writeQuorumSize, ackQuorumSize, state, lastEntryId, length, ensembles, digestType, password, ctime, storeCtime, + cToken, customMetadata); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java index c0fcadd07ed..842244ac7e2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java @@ -69,6 +69,8 @@ class LedgerMetadataImpl implements LedgerMetadata { private final Map customMetadata; + private long cToken; + LedgerMetadataImpl(int metadataFormatVersion, int ensembleSize, int writeQuorumSize, @@ -81,6 +83,7 @@ class LedgerMetadataImpl implements LedgerMetadata { Optional password, long ctime, boolean storeCtime, + long cToken, Map customMetadata) { checkArgument(ensembles.size() > 0, "There must be at least one ensemble in the ledger"); if (state == State.CLOSED) { @@ -127,6 +130,8 @@ class LedgerMetadataImpl implements LedgerMetadata { this.ctime = ctime; this.storeCtime = storeCtime; + this.cToken = cToken; + this.customMetadata = ImmutableMap.copyOf(customMetadata); } @@ -268,4 +273,9 @@ public int getMetadataFormatVersion() { boolean shouldStoreCtime() { return storeCtime; } + + @Override + public long getCToken() { + return cToken; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java index 2ce19404d09..de6cdf2c935 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java @@ -174,4 +174,11 @@ enum State { * @return the format version. */ int getMetadataFormatVersion(); + + /** + * Get the unique creator token of the Ledger. + * + * @return the creator token + */ + long getCToken(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 96c2f0f026b..9308bfb8ca7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -33,9 +33,11 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.AbstractConfiguration; @@ -247,8 +249,19 @@ public void process(WatchedEvent event) { } @Override - public CompletableFuture> createLedgerMetadata(long ledgerId, LedgerMetadata metadata) { + public CompletableFuture> createLedgerMetadata(long ledgerId, + LedgerMetadata inputMetadata) { CompletableFuture> promise = new CompletableFuture<>(); + /* + * Create a random number and use it as creator token. + */ + final long cToken = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE); + final LedgerMetadata metadata; + if (inputMetadata.getMetadataFormatVersion() > LedgerMetadataSerDe.METADATA_FORMAT_VERSION_2) { + metadata = LedgerMetadataBuilder.from(inputMetadata).withCToken(cToken).build(); + } else { + metadata = inputMetadata; + } String ledgerPath = getLedgerPath(ledgerId); StringCallback scb = new StringCallback() { @Override @@ -256,8 +269,39 @@ public void processResult(int rc, String path, Object ctx, String name) { if (rc == Code.OK.intValue()) { promise.complete(new Versioned<>(metadata, new LongVersion(0))); } else if (rc == Code.NODEEXISTS.intValue()) { - LOG.warn("Failed to create ledger metadata for {} which already exist", ledgerId); - promise.completeExceptionally(new BKException.BKLedgerExistException()); + LOG.info("Ledger metadata for {} appears to already exist, checking cToken", + ledgerId); + if (metadata.getMetadataFormatVersion() > 2) { + CompletableFuture> readFuture = readLedgerMetadata(ledgerId); + readFuture.handle((readMetadata, exception) -> { + if (exception == null) { + if (readMetadata.getValue().getCToken() == cToken) { + FutureUtils.complete(promise, new Versioned<>(metadata, new LongVersion(0))); + } else { + LOG.warn("Failed to create ledger metadata for {} which already exists", ledgerId); + promise.completeExceptionally(new BKException.BKLedgerExistException()); + } + } else if (exception instanceof KeeperException.NoNodeException) { + // This is a pretty strange case. We tried to create the node, found that it + // already exists, but failed to find it when we reread it. It's possible that + // we successfully created it, got an erroneous NODEEXISTS due to a resend, + // and then it got removed. It's also possible that we actually lost the race + // and then it got removed. I'd argue that returning an error here is the right + // path since recreating it is likely to cause problems. + LOG.warn("Ledger {} appears to have already existed and then been removed, failing" + + " with LedgerExistException"); + promise.completeExceptionally(new BKException.BKLedgerExistException()); + } else { + LOG.error("Could not validate node for ledger {} after LedgerExistsException", ledgerId, + exception); + promise.completeExceptionally(new BKException.ZKException()); + } + return null; + }); + } else { + LOG.warn("Failed to create ledger metadata for {} which already exists", ledgerId); + promise.completeExceptionally(new BKException.BKLedgerExistException()); + } } else { LOG.error("Could not create node for ledger {}", ledgerId, KeeperException.create(Code.get(rc), path)); @@ -301,8 +345,8 @@ public CompletableFuture removeLedgerMetadata(final long ledgerId, final V @Override public void processResult(int rc, String path, Object ctx) { if (rc == KeeperException.Code.NONODE.intValue()) { - LOG.warn("Ledger node does not exist in ZooKeeper: ledgerId={}", ledgerId); - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + LOG.warn("Ledger node does not exist in ZooKeeper: ledgerId={}. Returning success.", ledgerId); + FutureUtils.complete(promise, null); } else if (rc == KeeperException.Code.OK.intValue()) { // removed listener on ledgerId Set listenerSet = listeners.remove(ledgerId); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java index aab72fbc0cf..b34b34bc6d7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -78,7 +78,7 @@ public class LedgerMetadataSerDe { public static final int METADATA_FORMAT_VERSION_3 = 3; public static final int MAXIMUM_METADATA_FORMAT_VERSION = METADATA_FORMAT_VERSION_3; - public static final int CURRENT_METADATA_FORMAT_VERSION = METADATA_FORMAT_VERSION_2; + public static final int CURRENT_METADATA_FORMAT_VERSION = METADATA_FORMAT_VERSION_3; private static final int LOWEST_COMPAT_METADATA_FORMAT_VERSION = METADATA_FORMAT_VERSION_1; // for pulling the version @@ -205,6 +205,8 @@ private static byte[] serializeVersion3(LedgerMetadata metadata) throws IOExcept builder.addSegment(segmentBuilder.build()); } + builder.setCToken(metadata.getCToken()); + builder.build().writeDelimitedTo(os); return os.toByteArray(); } @@ -429,6 +431,10 @@ private static void decodeFormat(LedgerMetadataFormat data, LedgerMetadataBuilde Collectors.toMap(e -> e.getKey(), e -> e.getValue().toByteArray()))); } + + if (data.hasCToken()) { + builder.withCToken(data.getCToken()); + } } private static LedgerMetadata parseVersion1Config(InputStream is) throws IOException { 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 be037f506c5..3d97e6f7b89 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 @@ -271,7 +271,7 @@ public static Builder newBuilder() { return new Builder(); } - ZooKeeperClient(String connectString, + protected ZooKeeperClient(String connectString, int sessionTimeoutMs, ZooKeeperWatcherBase watcherManager, RetryPolicy connectRetryPolicy, @@ -329,7 +329,7 @@ private void closeZkHandle() throws InterruptedException { } } - protected void waitForConnection() throws KeeperException, InterruptedException { + public void waitForConnection() throws KeeperException, InterruptedException { watcherManager.waitForConnection(); } 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 cb45ba77e8d..80d059d6e27 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 @@ -24,6 +24,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -32,10 +33,12 @@ import java.io.IOException; import java.util.Collections; import java.util.Enumeration; +import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; @@ -45,9 +48,22 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.api.WriteHandle; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; +import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase; +import org.apache.zookeeper.AsyncCallback.StringCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.ConnectionLossException; +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.ZooKeeper.States; +import org.apache.zookeeper.data.ACL; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,7 +73,7 @@ */ public class BookKeeperTest extends BookKeeperClusterTestCase { private static final Logger LOG = LoggerFactory.getLogger(BookKeeperTest.class); - + private static final long INVALID_LEDGERID = -1L; private final DigestType digestType; public BookKeeperTest() { @@ -816,4 +832,128 @@ public void testCannotUseForceOnV2Protocol() throws Exception { } } } + + class MockZooKeeperClient extends ZooKeeperClient { + class MockZooKeeper extends ZooKeeper { + public MockZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) + throws IOException { + super(connectString, sessionTimeout, watcher, canBeReadOnly); + } + + @Override + public void create(final String path, byte data[], List acl, CreateMode createMode, StringCallback cb, + Object ctx) { + StringCallback injectedCallback = new StringCallback() { + @Override + public void processResult(int rc, String path, Object ctx, String name) { + /** + * if ledgerIdToInjectFailure matches with the path of + * the node, then throw CONNECTIONLOSS error and then + * reset it to INVALID_LEDGERID. + */ + if (path.contains(ledgerIdToInjectFailure.toString())) { + ledgerIdToInjectFailure.set(INVALID_LEDGERID); + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, name); + } else { + cb.processResult(rc, path, ctx, name); + } + } + }; + super.create(path, data, acl, createMode, injectedCallback, ctx); + } + } + + private final String connectString; + private final int sessionTimeoutMs; + private final ZooKeeperWatcherBase watcherManager; + private final AtomicLong ledgerIdToInjectFailure; + + MockZooKeeperClient(String connectString, int sessionTimeoutMs, ZooKeeperWatcherBase watcher, + AtomicLong ledgerIdToInjectFailure) throws IOException { + /* + * in OperationalRetryPolicy maxRetries is > 0. So in case of any + * RecoverableException scenario, it will retry. + */ + super(connectString, sessionTimeoutMs, watcher, + new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, Integer.MAX_VALUE), + new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 3), + NullStatsLogger.INSTANCE, 1, 0, false); + this.connectString = connectString; + this.sessionTimeoutMs = sessionTimeoutMs; + this.watcherManager = watcher; + this.ledgerIdToInjectFailure = ledgerIdToInjectFailure; + } + + @Override + protected ZooKeeper createZooKeeper() throws IOException { + return new MockZooKeeper(this.connectString, this.sessionTimeoutMs, this.watcherManager, false); + } + } + + @Test + public void testZKConnectionLossForLedgerCreation() throws Exception { + int zkSessionTimeOut = 10000; + AtomicLong ledgerIdToInjectFailure = new AtomicLong(INVALID_LEDGERID); + ZooKeeperWatcherBase zooKeeperWatcherBase = new ZooKeeperWatcherBase(zkSessionTimeOut, + NullStatsLogger.INSTANCE); + MockZooKeeperClient zkFaultInjectionWrapper = new MockZooKeeperClient(zkUtil.getZooKeeperConnectString(), + zkSessionTimeOut, zooKeeperWatcherBase, ledgerIdToInjectFailure); + zkFaultInjectionWrapper.waitForConnection(); + assertEquals("zkFaultInjectionWrapper should be in connected state", States.CONNECTED, + zkFaultInjectionWrapper.getState()); + BookKeeper bk = new BookKeeper(baseClientConf, zkFaultInjectionWrapper); + long oldZkInstanceSessionId = zkFaultInjectionWrapper.getSessionId(); + long ledgerId = 567L; + LedgerHandle lh = bk.createLedgerAdv(ledgerId, 1, 1, 1, DigestType.CRC32, "".getBytes(), null); + lh.close(); + + /* + * trigger Expired event so that MockZooKeeperClient would run + * 'clientCreator' and create new zk handle. In this case it would + * create MockZooKeeper. + */ + zooKeeperWatcherBase.process(new WatchedEvent(EventType.None, KeeperState.Expired, "")); + zkFaultInjectionWrapper.waitForConnection(); + for (int i = 0; i < 10; i++) { + if (zkFaultInjectionWrapper.getState() == States.CONNECTED) { + break; + } + Thread.sleep(200); + } + assertEquals("zkFaultInjectionWrapper should be in connected state", States.CONNECTED, + zkFaultInjectionWrapper.getState()); + assertNotEquals("Session Id of old and new ZK instance should be different", oldZkInstanceSessionId, + zkFaultInjectionWrapper.getSessionId()); + ledgerId++; + ledgerIdToInjectFailure.set(ledgerId); + /** + * ledgerIdToInjectFailure is set to 'ledgerId', so zookeeper.create + * would return CONNECTIONLOSS error for the first time and when it is + * retried, as expected it would return NODEEXISTS error. + * + * AbstractZkLedgerManager.createLedgerMetadata should deal with this + * scenario appropriately. + */ + lh = bk.createLedgerAdv(ledgerId, 1, 1, 1, DigestType.CRC32, "".getBytes(), null); + lh.close(); + assertEquals("injectZnodeCreationNoNodeFailure should have been reset it to INVALID_LEDGERID", INVALID_LEDGERID, + ledgerIdToInjectFailure.get()); + lh = bk.openLedger(ledgerId, DigestType.CRC32, "".getBytes()); + lh.close(); + ledgerId++; + lh = bk.createLedgerAdv(ledgerId, 1, 1, 1, DigestType.CRC32, "".getBytes(), null); + lh.close(); + bk.close(); + } + + @Test + public void testLedgerDeletionIdempotency() throws Exception { + BookKeeper bk = new BookKeeper(baseClientConf); + long ledgerId = 789L; + LedgerHandle lh = bk.createLedgerAdv(ledgerId, 1, 1, 1, DigestType.CRC32, "".getBytes(), null); + lh.close(); + bk.deleteLedger(ledgerId); + bk.deleteLedger(ledgerId); + bk.close(); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 5776af363b3..8e2ec0a4d0e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -273,9 +273,8 @@ public void testRemoveLedgerMetadataNoNode() throws Exception { try { result(ledgerManager.removeLedgerMetadata(ledgerId, version)); - fail("Should fail to remove metadata if no such ledger exists"); } catch (BKException bke) { - assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); + fail("Should succeed"); } verify(mockZk, times(1)) @@ -294,7 +293,7 @@ public void testRemoveLedgerMetadataException() throws Exception { try { result(ledgerManager.removeLedgerMetadata(ledgerId, version)); - fail("Should fail to remove metadata if no such ledger exists"); + fail("Should fail to remove metadata upon ZKException"); } catch (BKException bke) { assertEquals(Code.ZKException, bke.getCode()); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java index d9ce5a2cd50..a0767ba7d8f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java @@ -60,9 +60,9 @@ public class TestLedgerMetadataSerDe { // version 3, since 4.9.x, protobuf binary format private static final String version3 = - "Qm9va2llTWV0YWRhdGFGb3JtYXRWZXJzaW9uCTMKXggCEAMYACD///////////8BKAEyMgoOMTkyL" + "Qm9va2llTWV0YWRhdGFGb3JtYXRWZXJzaW9uCTMKYAgCEAMYACD///////////8BKAEyMgoOMTkyL" + "jAuMi4xOjMxODEKDjE5Mi4wLjIuMjozMTgxCg4xOTIuMC4yLjM6MzE4MRAAOANCBmZvb2JhckgB" - + "UP///////////wE="; + + "UP///////////wFgAA=="; private static void testDecodeEncode(String encoded) throws Exception { LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); diff --git a/tests/backward-compat/current-server-old-clients/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatOldClients.groovy b/tests/backward-compat/current-server-old-clients/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatOldClients.groovy index a7ab4e32062..4fd35d58058 100644 --- a/tests/backward-compat/current-server-old-clients/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatOldClients.groovy +++ b/tests/backward-compat/current-server-old-clients/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatOldClients.groovy @@ -163,11 +163,47 @@ class TestCompatOldClients { } } + private void testReadOpenFailure(String writeVersion, String readerVersion) throws Exception { + String zookeeper = BookKeeperClusterUtils.zookeeperConnectString(docker) + + def writeCL = MavenClassLoader.forBookKeeperVersion(writeVersion) + def writeBK = writeCL.newBookKeeper(zookeeper) + def readCL = MavenClassLoader.forBookKeeperVersion(readerVersion) + def readBK = readCL.newBookKeeper(zookeeper) + try { + def numEntries = 5 + def ledger0 = writeBK.createLedger(3, 2, + writeCL.digestType("CRC32"), + PASSWD) + for (int i = 0; i < numEntries; i++) { + ledger0.addEntry(("foobar" + i).getBytes()) + } + ledger0.close() + + try { + def ledger1 = readBK.openLedger(ledger0.getId(), readCL.digestType("CRC32"), PASSWD) + Assert.fail("For older versions Openledger call is expected to fail with ZKException"); + } catch (Exception exc) { + Assert.assertEquals(exc.getClass().getName(), + "org.apache.bookkeeper.client.BKException\$ZKException") + } + } finally { + readBK.close() + readCL.close() + writeBK.close() + writeCL.close() + } + } + + /** + * Since METADATA_VERSION is upgraded and it is using binary format, the older + * clients which are expecting text format would fail to read ledger metadata. + */ @Test public void testOldClientReadsNewClient() throws Exception { oldClientVersions.each{ def version = it - ThreadReaper.runWithReaper({ testReads(currentVersion, version) }) + ThreadReaper.runWithReaper({ testReadOpenFailure(currentVersion, version) }) } } diff --git a/tests/backward-compat/yahoo-custom-version/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgradeYahooCustom.groovy b/tests/backward-compat/yahoo-custom-version/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgradeYahooCustom.groovy index 18cdec84b22..9762b4bebe1 100644 --- a/tests/backward-compat/yahoo-custom-version/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgradeYahooCustom.groovy +++ b/tests/backward-compat/yahoo-custom-version/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgradeYahooCustom.groovy @@ -170,11 +170,16 @@ class TestCompatUpgradeYahooCustom { openAndVerifyEntries(currentCL, currentBK, ledger4.getId()) assertCantWrite(yahooCL, ledger4) - // yahoo client can fence a bookie created by current client + // Since METADATA_VERSION is upgraded and it is using binary format, the older + // clients which are expecting text format would fail to read ledger metadata. def ledger5 = createAndWrite(currentCL, currentBK) ledgers.add(ledger5.getId()) - openAndVerifyEntries(yahooCL, yahooBK, ledger5.getId()) - assertCantWrite(currentCL, ledger5) + try { + openAndVerifyEntries(yahooCL, yahooBK, ledger5.getId()) + } catch (Exception exc) { + Assert.assertEquals(exc.getClass().getName(), + "org.apache.bookkeeper.client.BKException\$ZKException") + } } finally { currentBK.close() currentCL.close() @@ -203,6 +208,13 @@ class TestCompatUpgradeYahooCustom { "org.apache.bookkeeper.stats.NullStatsProvider") Assert.assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, currentVersion)) - exerciseClients(preUpgradeLedgers) + // Since METADATA_VERSION is upgraded and it is using binary format, the older + // clients which are expecting text format would fail to read ledger metadata. + try { + exerciseClients(preUpgradeLedgers) + } catch (Exception exc) { + Assert.assertEquals(exc.getClass().getName(), + "org.apache.bookkeeper.client.BKException\$ZKException") + } } } From 7a9d0091d241ae970800acd4a9d308dec8c7d8de Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 24 Apr 2019 18:17:04 +0800 Subject: [PATCH 0281/1642] Migrate command `ledgermetadata` Descriptions of the changes in this PR: Using `bkctl` run command `ledgermetadata` ### Motivation #2018 Reviewers: Jia Zhai , Sijie Guo This closes #2019 from zymap/command-ledgermetadata --- .../apache/bookkeeper/bookie/BookieShell.java | 47 ++---- .../client/LedgerMetaDataCommand.java | 140 ++++++++++++++++++ .../meta/AbstractZkLedgerManagerTest.java | 1 + .../meta/TestLedgerMetadataSerDe.java | 8 +- .../cli/commands/LedgerCommandGroup.java | 2 + .../client/LedgerMetaDataCommandTest.java | 136 +++++++++++++++++ 6 files changed, 298 insertions(+), 36 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/LedgerMetaDataCommand.java create mode 100644 tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/LedgerMetaDataCommandTest.java 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 02a7391c444..1caa1e18c74 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 @@ -18,14 +18,10 @@ package org.apache.bookkeeper.bookie; -import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; - import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.UncheckedExecutionException; import java.io.File; import java.io.IOException; import java.io.Serializable; -import java.nio.file.FileSystems; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -39,14 +35,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.meta.LedgerMetadataSerDe; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; @@ -79,6 +72,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookies.NukeExistingClusterCommand.NukeExistingClusterFlags; import org.apache.bookkeeper.tools.cli.commands.bookies.RecoverCommand; import org.apache.bookkeeper.tools.cli.commands.client.DeleteLedgerCommand; +import org.apache.bookkeeper.tools.cli.commands.client.LedgerMetaDataCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.AdminCommand; import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand; @@ -90,7 +84,6 @@ import org.apache.bookkeeper.util.EntryFormatter; import org.apache.bookkeeper.util.LedgerIdFormatter; import org.apache.bookkeeper.util.Tool; -import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.HelpFormatter; @@ -728,7 +721,6 @@ void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { */ class LedgerMetadataCmd extends MyCommand { Options lOpts = new Options(); - LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); LedgerMetadataCmd() { super(CMD_LEDGERMETADATA); @@ -739,36 +731,27 @@ class LedgerMetadataCmd extends MyCommand { @Override public int runCmd(CommandLine cmdLine) throws Exception { - final long lid = getOptionLedgerIdValue(cmdLine, "ledgerid", -1); - if (lid == -1) { + final long ledgerId = getOptionLedgerIdValue(cmdLine, "ledgerid", -1); + if (ledgerId == -1) { System.err.println("Must specify a ledger id"); return -1; } - - if (cmdLine.hasOption("dumptofile") && cmdLine.hasOption("restorefromfile")) { + if (cmdLine.hasOption("dumptofile") && cmdLine.hasOption("restorefromefile")) { System.err.println("Only one of --dumptofile and --restorefromfile can be specified"); return -2; } - runFunctionWithLedgerManagerFactory(bkConf, mFactory -> { - try (LedgerManager m = mFactory.newLedgerManager()) { - if (cmdLine.hasOption("dumptofile")) { - Versioned md = m.readLedgerMetadata(lid).join(); - Files.write(FileSystems.getDefault().getPath(cmdLine.getOptionValue("dumptofile")), - serDe.serialize(md.getValue())); - } else if (cmdLine.hasOption("restorefromfile")) { - byte[] serialized = Files.readAllBytes( - FileSystems.getDefault().getPath(cmdLine.getOptionValue("restorefromfile"))); - LedgerMetadata md = serDe.parseConfig(serialized, Optional.empty()); - m.createLedgerMetadata(lid, md).join(); - } else { - printLedgerMetadata(lid, m.readLedgerMetadata(lid).get().getValue(), true); - } - } catch (Exception e) { - throw new UncheckedExecutionException(e); - } - return null; - }); + LedgerMetaDataCommand.LedgerMetadataFlag flag = new LedgerMetaDataCommand.LedgerMetadataFlag(); + flag.ledgerId(ledgerId); + if (cmdLine.hasOption("dumptofile")) { + flag.dumpToFile(cmdLine.getOptionValue("dumptofile")); + } + if (cmdLine.hasOption("restorefromfile")) { + flag.restoreFromFile(cmdLine.getOptionValue("restorefromfile")); + } + + LedgerMetaDataCommand cmd = new LedgerMetaDataCommand(ledgerIdFormatter); + cmd.apply(bkConf, flag); return 0; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/LedgerMetaDataCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/LedgerMetaDataCommand.java new file mode 100644 index 00000000000..5937a248538 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/LedgerMetaDataCommand.java @@ -0,0 +1,140 @@ +/* + * 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.tools.cli.commands.client; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * Print the metadata for a ledger. + */ +public class LedgerMetaDataCommand extends BookieCommand { + + private static final String NAME = "show"; + private static final String DESC = "Print the metadata for a ledger, or optionally dump to a file."; + private static final String DEFAULT = ""; + private static final long DEFAULT_ID = -1L; + + private LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); + private LedgerIdFormatter ledgerIdFormatter; + + public LedgerMetaDataCommand() { + this(new LedgerMetadataFlag()); + } + + public LedgerMetaDataCommand(LedgerIdFormatter ledgerIdFormatter) { + this(); + this.ledgerIdFormatter = ledgerIdFormatter; + } + + public LedgerMetaDataCommand(LedgerMetadataFlag flag) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flag) + .build()); + } + + /** + * Flags for ledger metadata command. + */ + @Accessors(fluent = true) + @Setter + public static class LedgerMetadataFlag extends CliFlags { + + @Parameter(names = { "-l", "--ledgerid" }, description = "Ledger ID", required = true) + private long ledgerId = DEFAULT_ID; + + @Parameter(names = { "-d", "--dumptofile" }, description = "Dump metadata for ledger, to a file") + private String dumpToFile = DEFAULT; + + @Parameter(names = { "-r", "--restorefromefile" }, description = "Restore metadata for ledger, from a file") + private String restoreFromFile = DEFAULT; + + @Parameter(names = {"-lf", "--ledgeridformatter"}, description = "Set ledger id formatter") + private String ledgerIdFormatter = DEFAULT; + } + + @Override + public boolean apply(ServerConfiguration conf, LedgerMetadataFlag cmdFlags) { + if (!cmdFlags.ledgerIdFormatter.equals(DEFAULT) && ledgerIdFormatter == null) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(cmdFlags.ledgerIdFormatter, conf); + } else if (ledgerIdFormatter == null) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } + try { + return handler(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean handler(ServerConfiguration conf, LedgerMetadataFlag flag) + throws MetadataException, ExecutionException { + if (flag.ledgerId == DEFAULT_ID) { + System.err.println("Must specific a ledger id"); + return false; + } + runFunctionWithLedgerManagerFactory(conf, mFactory -> { + try (LedgerManager m = mFactory.newLedgerManager()) { + if (!flag.dumpToFile.equals(DEFAULT)) { + Versioned md = m.readLedgerMetadata(flag.ledgerId).join(); + Files.write(FileSystems.getDefault().getPath(flag.dumpToFile), + serDe.serialize(md.getValue())); + } else if (!flag.restoreFromFile.equals(DEFAULT)) { + byte[] serialized = Files.readAllBytes( + FileSystems.getDefault().getPath(flag.restoreFromFile)); + LedgerMetadata md = serDe.parseConfig(serialized, Optional.empty()); + m.createLedgerMetadata(flag.ledgerId, md).join(); + } else { + printLedgerMetadata(flag.ledgerId, m.readLedgerMetadata(flag.ledgerId).get().getValue(), true); + } + } catch (Exception e) { + throw new UncheckedExecutionException(e); + } + return null; + }); + return true; + } + + private void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { + System.out.println("ledgerID: " + ledgerIdFormatter.formatLedgerId(ledgerId)); + if (printMeta) { + System.out.println(md.toString()); + } + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 8e2ec0a4d0e..2c0ae4f1aeb 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -124,6 +124,7 @@ public void setup() throws Exception { this.metadata = LedgerMetadataBuilder.create() .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .withEnsembleSize(5) + .withMetadataFormatVersion(2) .withWriteQuorumSize(3) .withAckQuorumSize(3) .newEnsembleEntry(0L, ensemble) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java index a0767ba7d8f..1c9de746c38 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java @@ -124,7 +124,7 @@ public void testVeryLongVersionSerDe() throws Exception { @Test public void testPeggedToV2SerDe() throws Exception { LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); - LedgerMetadata metadata = LedgerMetadataBuilder.create() + LedgerMetadata metadata = LedgerMetadataBuilder.create().withMetadataFormatVersion(2) .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) .newEnsembleEntry(0L, Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 3181), @@ -163,7 +163,7 @@ public void testStoreSystemtimeAsLedgerCtimeEnabledWithVersion2() @Test public void testStoreSystemtimeAsLedgerCtimeDisabledWithVersion2() throws Exception { - LedgerMetadata lm = LedgerMetadataBuilder.create() + LedgerMetadata lm = LedgerMetadataBuilder.create().withMetadataFormatVersion(2) .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) .newEnsembleEntry(0L, Lists.newArrayList( @@ -175,10 +175,10 @@ public void testStoreSystemtimeAsLedgerCtimeDisabledWithVersion2() LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); byte[] serialized = serDe.serialize(lm); LedgerMetadata deserialized = serDe.parseConfig(serialized, Optional.of(654321L)); - Assert.assertEquals(deserialized.getCtime(), 654321L); + Assert.assertEquals(654321L, deserialized.getCtime()); // give it another round LedgerMetadata deserialized2 = serDe.parseConfig(serDe.serialize(deserialized), Optional.of(98765L)); - Assert.assertEquals(deserialized2.getCtime(), 98765L); + Assert.assertEquals(98765L, deserialized2.getCtime()); } } diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java index 86734bc58dd..a133b5db0d5 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/LedgerCommandGroup.java @@ -22,6 +22,7 @@ import org.apache.bookkeeper.tools.cli.BKCtl; import org.apache.bookkeeper.tools.cli.commands.client.DeleteLedgerCommand; +import org.apache.bookkeeper.tools.cli.commands.client.LedgerMetaDataCommand; import org.apache.bookkeeper.tools.cli.commands.client.SimpleTestCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; @@ -42,6 +43,7 @@ public class LedgerCommandGroup extends CliCommandGroup { .withCategory(CATEGORY_LEDGER_SERVICE) .addCommand(new SimpleTestCommand()) .addCommand(new DeleteLedgerCommand()) + .addCommand(new LedgerMetaDataCommand()) .build(); public LedgerCommandGroup() { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/LedgerMetaDataCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/LedgerMetaDataCommandTest.java new file mode 100644 index 00000000000..d4d89c13fe9 --- /dev/null +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/client/LedgerMetaDataCommandTest.java @@ -0,0 +1,136 @@ +/* + * 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.tools.cli.commands.client; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.when; +import static org.powermock.api.mockito.PowerMockito.whenNew; + +import java.io.File; +import java.io.IOException; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerMetadataSerDe; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; +import org.junit.Assert; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** + * Unit test for {@link LedgerMetaDataCommand}. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ LedgerMetaDataCommand.class, MetadataDrivers.class, LedgerMetadata.class, + LedgerManagerFactory.class }) +public class LedgerMetaDataCommandTest extends BookieCommandTestBase { + + private LedgerManager ledgerManager; + private LedgerManagerFactory factory; + private LedgerMetadataSerDe serDe; + + @Mock + private CompletableFuture> future; + + private TemporaryFolder folder = new TemporaryFolder(); + + public LedgerMetaDataCommandTest() { + super(3, 0); + } + + @Override + public void setup() throws Exception { + folder.create(); + + factory = mock(LedgerManagerFactory.class); + + PowerMockito.mockStatic(MetadataDrivers.class); + PowerMockito.doAnswer(invocationOnMock -> { + Function function = invocationOnMock.getArgument(1); + function.apply(factory); + return true; + }).when(MetadataDrivers.class, "runFunctionWithLedgerManagerFactory", any(ServerConfiguration.class), + any(Function.class)); + + ledgerManager = mock(LedgerManager.class); + when(factory.newLedgerManager()).thenReturn(ledgerManager); + + LedgerMetadata ledgerMetadata = mock(LedgerMetadata.class); + when(ledgerMetadata.getMetadataFormatVersion()).thenReturn(1); + Versioned versioned = new Versioned(ledgerMetadata, Version.NEW); + versioned.setValue(ledgerMetadata); + when(future.join()).thenReturn(versioned); + when(ledgerManager.readLedgerMetadata(anyLong())).thenReturn(future); + when(ledgerManager.readLedgerMetadata(anyLong())).thenReturn(future); + when(future.get()).thenReturn(versioned); + + serDe = mock(LedgerMetadataSerDe.class); + whenNew(LedgerMetadataSerDe.class).withNoArguments().thenReturn(serDe); + when(serDe.serialize(eq(ledgerMetadata))).thenReturn(new byte[0]); + when(serDe.parseConfig(eq(new byte[0]), eq(Optional.empty()))).thenReturn(ledgerMetadata); + when(ledgerManager.createLedgerMetadata(anyLong(), eq(ledgerMetadata))).thenReturn(future); + } + + @Test + public void testWithDumpToFile() throws IOException { + File file = folder.newFile("testdump"); + LedgerMetaDataCommand cmd = new LedgerMetaDataCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-l", "1", "-d", file.getAbsolutePath() })); + + verify(ledgerManager, times(1)).readLedgerMetadata(anyLong()); + verify(serDe, times(1)).serialize(any(LedgerMetadata.class)); + } + + @Test + public void testWithRestoreFromFile() throws IOException { + File file = folder.newFile("testrestore"); + LedgerMetaDataCommand cmd = new LedgerMetaDataCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-l", "1", "-r", file.getAbsolutePath() })); + + verify(serDe, times(1)).parseConfig(eq(new byte[0]), eq(Optional.empty())); + verify(ledgerManager, times(1)).createLedgerMetadata(anyLong(), any(LedgerMetadata.class)); + } + + @Test + public void testWithoutArgs() { + LedgerMetaDataCommand cmd = new LedgerMetaDataCommand(); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-l", "1" })); + + verify(factory, times(1)).newLedgerManager(); + verify(ledgerManager, times(1)).readLedgerMetadata(anyLong()); + } + +} From 0073be25cbc1aa46a94daa0ddfb726f926cf0a21 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Sun, 28 Apr 2019 04:06:24 +0200 Subject: [PATCH 0282/1642] DLog Namespace#openLog should declare LogNotFoundException Other methods such as delete do, so openLog should declare it to be consistent and to give users a hint that it can be caught. Reviewers: Enrico Olivelli This closes #2072 from ivankelly/dl-openlog-ex --- .../org/apache/distributedlog/api/namespace/Namespace.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java index 712295d4f2a..1b2a7873e22 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java @@ -128,10 +128,11 @@ void deleteLog(String logName) * name of the log * @return distributedlog manager instance. * @throws InvalidStreamNameException if log name is invalid. + * @throws LogNotFoundException if log doesn't exist. * @throws IOException when encountered issues with backend. */ DistributedLogManager openLog(String logName) - throws InvalidStreamNameException, IOException; + throws InvalidStreamNameException, LogNotFoundException, IOException; /** * Open a log named logName with specific log configurations. @@ -149,13 +150,14 @@ DistributedLogManager openLog(String logName) * dynamic log configuration * @return distributedlog manager instance. * @throws InvalidStreamNameException if log name is invalid. + * @throws LogNotFoundException if log doesn't exist. * @throws IOException when encountered issues with backend. */ DistributedLogManager openLog(String logName, Optional logConf, Optional dynamicLogConf, Optional perStreamStatsLogger) - throws InvalidStreamNameException, IOException; + throws InvalidStreamNameException, LogNotFoundException, IOException; /** * Check whether the log logName exist. From 394f8377b80b62916c4612ad30336503ab39eb76 Mon Sep 17 00:00:00 2001 From: karanmehta93 Date: Mon, 29 Apr 2019 19:28:35 -0700 Subject: [PATCH 0283/1642] =?UTF-8?q?ISSUE=20#2073:=20ReadOnlyBookieTest#t?= =?UTF-8?q?estBookieContinueWritingIfMulti=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …pleLedgersPresent is flaky Descriptions of the changes in this PR: Updated the test to use random port number for bookie startup instead of predefined one. ### Motivation ReadOnlyBookieTest#testBookieContinueWritingIfMultipleLedgersPresent test is flaky due to this reason. ### Changes (Describe: what changes you have made) Master Issue: #2073 Reviewers: Enrico Olivelli , Sijie Guo This closes #2074 from karanmehta93/master, closes #2073 --- .../java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java index 35b2fd36d47..329779ce386 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java @@ -245,7 +245,7 @@ private void startNewBookieWithMultipleLedgerDirs(int numOfLedgerDirs) } ServerConfiguration newConf = newServerConfiguration( - conf.getBookiePort() + 1, + PortManager.nextFreePort(), ledgerDirs[0], ledgerDirs); bsConfs.add(newConf); bs.add(startBookie(newConf)); From 9982c68442fcffdad9c83da1bbed934138d7496f Mon Sep 17 00:00:00 2001 From: mtang01 <49574764+mtang01@users.noreply.github.com> Date: Mon, 29 Apr 2019 19:31:04 -0700 Subject: [PATCH 0284/1642] ISSUE #2067: reduce byte[] allocation in add entry Descriptions of the changes in this PR: This change removes a byte[] copy in DigestManager digest calculation (computeDigestAndPackageForSending) that puts crc header and payload in a continuous buffer. Instead, it uses protobuf ByteString.concat to concatenate header and payload without copy when building protobuf message. ### Motivation In add entry code path, I see lots of byte[] allocated to do digest calculation. It's possible to not allocate byte[]. ### Changes Don't allocate a ByteBuf to copy data. Keep header and data separate, but use ByteString.concat when construct protobuf message. Master Issue: #2067 Reviewers: Enrico Olivelli , Sijie Guo This closes #2068 from mtang01/add_entry_mem, closes #2067 --- .../proto/PerChannelBookieClient.java | 12 +++-- .../proto/checksum/DigestManager.java | 46 ++++++------------- 2 files changed, 21 insertions(+), 37 deletions(-) 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 d411e9880d2..1fdb403c70b 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 @@ -753,13 +753,15 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBuf headerBuilder.setPriority(DEFAULT_HIGH_PRIORITY_VALUE); } - ByteString body; + ByteString body = null; if (toSend.hasArray()) { body = UnsafeByteOperations.unsafeWrap(toSend.array(), toSend.arrayOffset(), toSend.readableBytes()); - } else if (toSend.size() == 1) { - body = UnsafeByteOperations.unsafeWrap(toSend.getBuffer(0).nioBuffer()); } else { - body = UnsafeByteOperations.unsafeWrap(toSend.toArray()); + for (int i = 0; i < toSend.size(); i++) { + ByteString piece = UnsafeByteOperations.unsafeWrap(toSend.getBuffer(i).nioBuffer()); + // use ByteString.concat to avoid byte[] allocation when toSend has multiple ByteBufs + body = (body == null) ? piece : body.concat(piece); + } } AddRequest.Builder addBuilder = AddRequest.newBuilder() .setLedgerId(ledgerId) @@ -1982,6 +1984,7 @@ public void handleV3Response(BookkeeperProtocol.Response response) { } private final Recycler addCompletionRecycler = new Recycler() { + @Override protected AddCompletion newObject(Recycler.Handle handle) { return new AddCompletion(handle); } @@ -2184,6 +2187,7 @@ private long getTxnId() { } private final Recycler v2KeyRecycler = new Recycler() { + @Override protected V2CompletionKey newObject( Recycler.Handle handle) { return new V2CompletionKey(handle); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java index 2dabf828216..034dd6ed77e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java @@ -20,7 +20,6 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; -import io.netty.util.ReferenceCountUtil; import java.security.GeneralSecurityException; import java.security.NoSuchAlgorithmException; @@ -99,41 +98,22 @@ public static byte[] generateMasterKey(byte[] password) throws NoSuchAlgorithmEx */ public ByteBufList computeDigestAndPackageForSending(long entryId, long lastAddConfirmed, long length, ByteBuf data) { + ByteBuf headersBuffer; if (this.useV2Protocol) { - /* - * For V2 protocol, use pooled direct ByteBuf's to avoid object allocation in DigestManager. - */ - ByteBuf headersBuffer = allocator.buffer(METADATA_LENGTH + macCodeLength); - headersBuffer.writeLong(ledgerId); - headersBuffer.writeLong(entryId); - headersBuffer.writeLong(lastAddConfirmed); - headersBuffer.writeLong(length); - - update(headersBuffer); - update(data); - populateValueAndReset(headersBuffer); - - return ByteBufList.get(headersBuffer, data); + headersBuffer = allocator.buffer(METADATA_LENGTH + macCodeLength); } else { - /* - * For V3 protocol, use unpooled heap ByteBuf's (backed by accessible array): The one object - * allocation here saves us later allocations when converting to protobuf ByteString. - */ - ByteBuf sendBuffer = Unpooled.buffer(METADATA_LENGTH + macCodeLength + data.readableBytes()); - sendBuffer.writeLong(ledgerId); - sendBuffer.writeLong(entryId); - sendBuffer.writeLong(lastAddConfirmed); - sendBuffer.writeLong(length); - - update(sendBuffer); - update(data); - populateValueAndReset(sendBuffer); - - sendBuffer.writeBytes(data, data.readerIndex(), data.readableBytes()); - ReferenceCountUtil.release(data); - - return ByteBufList.get(sendBuffer); + headersBuffer = Unpooled.buffer(METADATA_LENGTH + macCodeLength); } + headersBuffer.writeLong(ledgerId); + headersBuffer.writeLong(entryId); + headersBuffer.writeLong(lastAddConfirmed); + headersBuffer.writeLong(length); + + update(headersBuffer); + update(data); + populateValueAndReset(headersBuffer); + + return ByteBufList.get(headersBuffer, data); } /** From bff475d2cca0f5607d5db7fba43fa6ed007d3d3f Mon Sep 17 00:00:00 2001 From: "Venkateswararao Jujjuri (JV)" Date: Tue, 30 Apr 2019 20:20:32 -0700 Subject: [PATCH 0285/1642] Explicit error message if extent is not present on ZK (#2066) * Explicit error message if extent is not present on ZK Client can get no extent error in the following two scenarios. 1. When client attempted to read/open an extent that is not on ZK (metadata server) 2. When client attempted to read a Ledger which is on metadata server, but somehow missing on bookies (Data server) It is quite confusing to get the same error, NoSuchLedgerExists for both these cases. This checkin introduced a new error at the BookKeeper client level, NoSuchLedgerExistsOnMetadataServer if it is not available on ZK. For errors related to Ledger is not available on bookie, I left NoSuchLedgerExists as-is(instead of changing it to NoSuchLedgerExistsOnDataServer) to minimize code changes. This change alone provides enough distinction between these two types of errors. Signed-off-by: Venkateswararao Jujjuri (JV) * Checkstyle fixes * Some more fixes * Fixing DL test fixes --- .../bookie/ScanAndCompareGarbageCollector.java | 2 +- .../java/org/apache/bookkeeper/client/BKException.java | 10 ++++++++++ .../org/apache/bookkeeper/client/BookKeeperAdmin.java | 5 +++-- .../org/apache/bookkeeper/client/LedgerChecker.java | 3 ++- .../org/apache/bookkeeper/client/UpdateLedgerOp.java | 2 +- .../org/apache/bookkeeper/client/api/BKException.java | 7 ++++++- .../apache/bookkeeper/client/impl/OpenBuilderBase.java | 2 +- .../bookkeeper/meta/AbstractZkLedgerManager.java | 7 ++++--- .../java/org/apache/bookkeeper/meta/LedgerManager.java | 4 ++-- .../apache/bookkeeper/meta/MSLedgerManagerFactory.java | 9 +++++---- .../org/apache/bookkeeper/replication/Auditor.java | 6 +++--- .../bookkeeper/replication/BookieLedgerIndexer.java | 2 +- .../bookkeeper/replication/ReplicationWorker.java | 10 +++++----- .../org/apache/bookkeeper/client/MockBookKeeper.java | 8 ++++---- .../bookkeeper/client/MockBookKeeperTestCase.java | 4 ++-- .../bookkeeper/client/api/BookKeeperApiTest.java | 6 +++--- .../bookkeeper/client/api/BookKeeperBuildersTest.java | 6 +++--- .../bookkeeper/meta/AbstractZkLedgerManagerTest.java | 2 +- .../java/org/apache/bookkeeper/meta/GcLedgersTest.java | 7 ++++--- .../apache/distributedlog/bk/TestLedgerAllocator.java | 2 +- .../distributedlog/tools/TestDistributedLogTool.java | 6 +++--- .../tests/integration/TestBookieShellCluster.java | 3 ++- 22 files changed, 67 insertions(+), 46 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index b1e77229c13..247b610aee0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -171,7 +171,7 @@ public void gc(GarbageCleaner garbageCleaner) { } catch (BKException e) { rc = e.getCode(); } - if (rc != BKException.Code.NoSuchLedgerExistsException) { + if (rc != BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { LOG.warn("Ledger {} Missing in metadata list, but ledgerManager returned rc: {}.", bkLid, rc); continue; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java index 781a07b4bf0..a3f03c24157 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java @@ -72,6 +72,8 @@ public static BKException create(int code) { return new BKNotEnoughBookiesException(); case Code.NoSuchLedgerExistsException: return new BKNoSuchLedgerExistsException(); + case Code.NoSuchLedgerExistsOnMetadataServerException: + return new BKNoSuchLedgerExistsOnMetadataServerException(); case Code.BookieHandleNotAvailableException: return new BKBookieHandleNotAvailableException(); case Code.ZKException: @@ -279,6 +281,14 @@ public BKNoSuchLedgerExistsException() { super(BKException.Code.NoSuchLedgerExistsException); } } + /** + * Bookkeeper no such ledger exists on metadata server exception. + */ + public static class BKNoSuchLedgerExistsOnMetadataServerException extends BKException { + public BKNoSuchLedgerExistsOnMetadataServerException() { + super(Code.NoSuchLedgerExistsOnMetadataServerException); + } + } /** * Bookkeeper bookie handle not available exception. 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 9303ffdff8c..6d2319cb57d 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 @@ -492,7 +492,7 @@ public void process(final Long lid, final AsyncCallback.VoidCallback cb) { bkc.getLedgerManager().readLedgerMetadata(lid) .whenComplete((metadata, exception) -> { if (BKException.getExceptionCode(exception) - == BKException.Code.NoSuchLedgerExistsException) { + == BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { // the ledger was deleted during this iteration. cb.processResult(BKException.Code.OK, null, null); return; @@ -1541,7 +1541,8 @@ public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieS throw new RuntimeException(ie); } catch (ExecutionException e) { if (e.getCause() != null - && e.getCause().getClass().equals(BKException.BKNoSuchLedgerExistsException.class)) { + && e.getCause().getClass() + .equals(BKException.BKNoSuchLedgerExistsOnMetadataServerException.class)) { LOG.debug("Ledger: {} has been deleted", ledgerId); return false; } else { 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 b9745a9371c..fe6370b9926 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 @@ -264,7 +264,8 @@ private static class EntryExistsCallback implements ReadEntryCallback { public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf buffer, Object ctx) { - if (BKException.Code.NoSuchEntryException != rc && BKException.Code.NoSuchLedgerExistsException != rc) { + if (BKException.Code.NoSuchEntryException != rc && BKException.Code.NoSuchLedgerExistsException != rc + && BKException.Code.NoSuchLedgerExistsOnMetadataServerException != rc) { entryMayExist.set(true); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java index 6fce7e1fc38..268eff6ca1f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java @@ -116,7 +116,7 @@ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final outstanding.add(writePromise); writePromise.whenComplete((metadata, ex) -> { if (ex != null - && !(ex instanceof BKException.BKNoSuchLedgerExistsException)) { + && !(ex instanceof BKException.BKNoSuchLedgerExistsOnMetadataServerException)) { String error = String.format("Failed to update ledger metadata %s, replacing %s with %s", ledgerId, oldBookieId, newBookieId); LOG.error(error, ex); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java index 058a9a744d6..0ec0997cad1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java @@ -116,7 +116,9 @@ public static String getMessage(int code) { case Code.NotEnoughBookiesException: return "Not enough non-faulty bookies available"; case Code.NoSuchLedgerExistsException: - return "No such ledger exists"; + return "No such ledger exists on Bookies"; + case Code.NoSuchLedgerExistsOnMetadataServerException: + return "No such ledger exists on Metadata Server"; case Code.BookieHandleNotAvailableException: return "Bookie handle is not available"; case Code.ZKException: @@ -242,6 +244,9 @@ public interface Code { int TimeoutException = -23; int SecurityException = -24; + /** No such ledger exists one metadata server. */ + int NoSuchLedgerExistsOnMetadataServerException = -25; + /** * Operation is illegal. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java index c2c4c35e1fa..07e861517d4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java @@ -66,7 +66,7 @@ public OpenBuilder withDigestType(DigestType digestType) { protected int validate() { if (ledgerId < 0) { LOG.error("invalid ledgerId {} < 0", ledgerId); - return Code.NoSuchLedgerExistsException; + return Code.NoSuchLedgerExistsOnMetadataServerException; } return Code.OK; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 9308bfb8ca7..79a4f50c3cc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -129,7 +129,8 @@ private void handleMetadata(Versioned result, Throwable exceptio } }); } - } else if (BKException.getExceptionCode(exception) == BKException.Code.NoSuchLedgerExistsException) { + } else if (BKException.getExceptionCode(exception) + == BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { // the ledger is removed, do nothing Set listenerSet = listeners.remove(ledgerId); if (null != listenerSet) { @@ -437,7 +438,7 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta LOG.debug("No such ledger: " + ledgerId, KeeperException.create(KeeperException.Code.get(rc), path)); } - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException()); return; } if (rc != KeeperException.Code.OK.intValue()) { @@ -494,7 +495,7 @@ public void processResult(int rc, String path, Object ctx, Stat stat) { promise.complete(new Versioned<>(metadata, new LongVersion(stat.getVersion()))); } else if (KeeperException.Code.NONODE.intValue() == rc) { LOG.warn("Ledger node does not exist in ZooKeeper: ledgerId={}", ledgerId); - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException()); } else { LOG.warn("Conditional update ledger metadata failed: {}", KeeperException.Code.get(rc)); promise.completeExceptionally(new BKException.ZKException()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java index 039ff7ccbde..15da14b083b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java @@ -65,7 +65,7 @@ public interface LedgerManager extends Closeable { * @return Future which, when completed, denotes that the ledger metadata has been removed. * Completed with an exception:
              *
            • {@link BKException.BKMetadataVersionException} if version doesn't match
            • - *
            • {@link BKException.BKNoSuchLedgerExistsException} if ledger not exist
            • + *
            • {@link BKException.BKNoSuchLedgerExistsOnMetadataServerException} if ledger not exist
            • *
            • {@link BKException.ZKException} for other issues
            • *
            */ @@ -78,7 +78,7 @@ public interface LedgerManager extends Closeable { * Ledger Id * @return Future which, when completed, contains the requested versioned metadata. * Completed with an exception::
              - *
            • {@link BKException.BKNoSuchLedgerExistsException} if ledger not exist
            • + *
            • {@link BKException.BKNoSuchLedgerExistsOnMetadataServerException} if ledger not exist
            • *
            • {@link BKException.ZKException} for other issues
            • *
            */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index 266db3a3fbc..201a0a3edb9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -263,7 +263,8 @@ private void handleMetadata(Versioned metadata, Throwable except } }); } - } else if (BKException.getExceptionCode(exception) == BKException.Code.NoSuchLedgerExistsException) { + } else if (BKException.getExceptionCode(exception) + == BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { // the ledger is removed, do nothing Set listenerSet = listeners.remove(ledgerId); if (null != listenerSet) { @@ -416,7 +417,7 @@ public void complete(int rc, Void value, Object ctx) { int bkRc; if (MSException.Code.NoKey.getCode() == rc) { LOG.warn("Ledger entry does not exist in meta table: ledgerId={}", ledgerId); - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException()); } else if (MSException.Code.OK.getCode() == rc) { FutureUtils.complete(promise, null); } else { @@ -438,7 +439,7 @@ public void complete(int rc, Versioned value, Object ctx) { if (MSException.Code.NoKey.getCode() == rc) { LOG.error("No ledger metadata found for ledger " + ledgerId + " : ", MSException.create(MSException.Code.get(rc), "No key " + key + " found.")); - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException()); return; } if (MSException.Code.OK.getCode() != rc) { @@ -489,7 +490,7 @@ public void complete(int rc, Version version, Object ctx) { promise.completeExceptionally(new BKException.BKMetadataVersionException()); } else if (MSException.Code.NoKey.getCode() == rc) { LOG.warn("Ledger {} doesn't exist when writing its ledger metadata.", ledgerId); - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException()); } else if (MSException.Code.OK.getCode() == rc) { promise.complete(new Versioned<>(metadata, version)); } else { 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 ad588490c8b..cbf9b45c954 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 @@ -895,7 +895,7 @@ void checkAllLedgers() throws BKException, IOException, InterruptedException, Ke numFragmentsPerLedger.registerSuccessfulValue(lh.getNumFragments()); numBookiesPerLedger.registerSuccessfulValue(lh.getNumBookies()); numLedgersChecked.inc(); - } else if (Code.NoSuchLedgerExistsException == rc) { + } else if (Code.NoSuchLedgerExistsOnMetadataServerException == rc) { if (LOG.isDebugEnabled()) { LOG.debug("Ledger {} was deleted before we could check it", ledgerId); } @@ -966,8 +966,8 @@ public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { } } iterCallback.processResult(BKException.Code.OK, null, null); - } else if (BKException - .getExceptionCode(exception) == BKException.Code.NoSuchLedgerExistsException) { + } else if (BKException.getExceptionCode(exception) + == BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { LOG.debug("Ignoring replication of already deleted ledger {}", ledgerId); iterCallback.processResult(BKException.Code.OK, null, null); } else { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java index c09a7702c63..026fd7262fc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java @@ -76,7 +76,7 @@ public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { } iterCallback.processResult(BKException.Code.OK, null, null); } else if (BKException.getExceptionCode(exception) - == BKException.Code.NoSuchLedgerExistsException) { + == BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { LOG.info("Ignoring replication of already deleted ledger {}", ledgerId); iterCallback.processResult(BKException.Code.OK, null, null); } else { 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 74d20816dbd..1bba2d5ebee 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 @@ -45,7 +45,7 @@ import org.apache.bookkeeper.bookie.BookieThread; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; +import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsOnMetadataServerException; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAdmin; @@ -330,14 +330,14 @@ private boolean rereplicate(long ledgerIdToReplicate) throws InterruptedExceptio return false; } - } catch (BKNoSuchLedgerExistsException e) { + } catch (BKNoSuchLedgerExistsOnMetadataServerException e) { // Ledger might have been deleted by user - LOG.info("BKNoSuchLedgerExistsException while opening " + LOG.info("BKNoSuchLedgerExistsOnMetadataServerException while opening " + "ledger {} for replication. Other clients " + "might have deleted the ledger. " + "So, no harm to continue", ledgerIdToReplicate); underreplicationManager.markLedgerReplicated(ledgerIdToReplicate); - getExceptionCounter("BKNoSuchLedgerExistsException").inc(); + getExceptionCounter("BKNoSuchLedgerExistsOnMetadataServerException").inc(); return false; } catch (BKNotEnoughBookiesException e) { logBKExceptionAndReleaseLedger(e, ledgerIdToReplicate); @@ -456,7 +456,7 @@ public void run() { Thread.currentThread().interrupt(); LOG.info("InterruptedException while fencing the ledger {}" + " for rereplication of postponed ledgers", ledgerId, e); - } catch (BKNoSuchLedgerExistsException bknsle) { + } catch (BKNoSuchLedgerExistsOnMetadataServerException bknsle) { if (LOG.isDebugEnabled()) { LOG.debug("Ledger {} was deleted, safe to continue", ledgerId, bknsle); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java index c65e0702810..81212d1b701 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java @@ -163,7 +163,7 @@ public void asyncOpenLedger(long lId, DigestType digestType, byte[] passwd, Open MockLedgerHandle lh = ledgers.get(lId); if (lh == null) { - cb.openComplete(BKException.Code.NoSuchLedgerExistsException, null, ctx); + cb.openComplete(BKException.Code.NoSuchLedgerExistsOnMetadataServerException, null, ctx); } else if (lh.digest != digestType) { cb.openComplete(BKException.Code.DigestMatchException, null, ctx); } else if (!Arrays.equals(lh.passwd, passwd)) { @@ -190,7 +190,7 @@ public void asyncDeleteLedger(long lId, DeleteCallback cb, Object ctx) { ledgers.remove(lId); cb.deleteComplete(0, ctx); } else { - cb.deleteComplete(BKException.Code.NoSuchLedgerExistsException, ctx); + cb.deleteComplete(BKException.Code.NoSuchLedgerExistsOnMetadataServerException, ctx); } } @@ -203,7 +203,7 @@ public void deleteLedger(long lId) throws InterruptedException, BKException { } if (!ledgers.containsKey(lId)) { - throw BKException.create(BKException.Code.NoSuchLedgerExistsException); + throw BKException.create(BKException.Code.NoSuchLedgerExistsOnMetadataServerException); } ledgers.remove(lId); @@ -238,7 +238,7 @@ public CompletableFuture execute() { MockLedgerHandle lh = ledgers.get(ledgerId); if (lh == null) { - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException()); } else if (lh.digest != DigestType.fromApiDigestType(digestType)) { promise.completeExceptionally(new BKException.BKDigestMatchException()); } else if (!Arrays.equals(lh.passwd, password)) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java index 69b56352240..c5878f1b860 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java @@ -379,7 +379,7 @@ private void setupReadLedgerMetadata() { executor.executeOrdered(ledgerId, () -> { LedgerMetadata ledgerMetadata = mockLedgerMetadataRegistry.get(ledgerId); if (ledgerMetadata == null) { - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException()); } else { promise.complete(new Versioned<>(ledgerMetadata, new LongVersion(1))); } @@ -398,7 +398,7 @@ private void setupRemoveLedgerMetadata() { if (mockLedgerMetadataRegistry.remove(ledgerId) != null) { promise.complete(null); } else { - promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException()); } }); return promise; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperApiTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperApiTest.java index 5429c0dba25..a71002dd98c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperApiTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperApiTest.java @@ -41,7 +41,7 @@ import org.apache.bookkeeper.client.BKException.BKDigestMatchException; import org.apache.bookkeeper.client.BKException.BKDuplicateEntryIdException; import org.apache.bookkeeper.client.BKException.BKLedgerFencedException; -import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; +import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsOnMetadataServerException; import org.apache.bookkeeper.client.BKException.BKUnauthorizedAccessException; import org.apache.bookkeeper.client.MockBookKeeperTestCase; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -339,7 +339,7 @@ public void testOpenLedgerWithRecovery() throws Exception { } } - @Test(expected = BKNoSuchLedgerExistsException.class) + @Test(expected = BKNoSuchLedgerExistsOnMetadataServerException.class) public void testDeleteLedger() throws Exception { long lId; @@ -358,7 +358,7 @@ public void testDeleteLedger() throws Exception { .execute()); } - @Test(expected = BKNoSuchLedgerExistsException.class) + @Test(expected = BKNoSuchLedgerExistsOnMetadataServerException.class) public void testCannotDeleteLedgerTwice() throws Exception { long lId; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java index 85cb727140f..f0e07e29934 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java @@ -30,7 +30,7 @@ import java.util.Map; import org.apache.bookkeeper.client.BKException.BKClientClosedException; import org.apache.bookkeeper.client.BKException.BKIncorrectParameterException; -import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; +import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsOnMetadataServerException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.LedgerMetadataBuilder; @@ -307,12 +307,12 @@ public void testFailCreateAdvLedgerBadFixedLedgerIdNegative() throws Exception { fail("shoud not be able to create a ledger with such specs"); } - @Test(expected = BKNoSuchLedgerExistsException.class) + @Test(expected = BKNoSuchLedgerExistsOnMetadataServerException.class) public void testOpenLedgerNoId() throws Exception { result(newOpenLedgerOp().execute()); } - @Test(expected = BKNoSuchLedgerExistsException.class) + @Test(expected = BKNoSuchLedgerExistsOnMetadataServerException.class) public void testOpenLedgerBadId() throws Exception { result(newOpenLedgerOp() .withPassword(password) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 2c0ae4f1aeb..0dc2b537c0b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -366,7 +366,7 @@ public void testReadLedgerMetadataNoNode() throws Exception { result(ledgerManager.readLedgerMetadata(ledgerId)); fail("Should fail on reading ledger metadata if a ledger doesn't exist"); } catch (BKException bke) { - assertEquals(Code.NoSuchLedgerExistsException, bke.getCode()); + assertEquals(Code.NoSuchLedgerExistsOnMetadataServerException, bke.getCode()); } verify(mockZk, times(1)) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 37eacf4d5a7..10f7048d1d3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -406,7 +406,7 @@ public void clean(long ledgerId) { * * ScanAndCompareGarbageCollector/GC should clean data of ledger only if both the LedgerManager.getLedgerRanges says * that ledger is not existing and also ledgerManager.readLedgerMetadata fails with error - * NoSuchLedgerExistsException. + * NoSuchLedgerExistsOnMetadataServerException. * */ @Test @@ -464,7 +464,7 @@ public void clean(long ledgerId) { * * ScanAndCompareGarbageCollector/GC should clean data of ledger only if both the LedgerManager.getLedgerRanges says * that ledger is not existing and also ledgerManager.readLedgerMetadata fails with error - * NoSuchLedgerExistsException. + * NoSuchLedgerExistsOnMetadataServerException. * */ @Test @@ -508,7 +508,8 @@ public void clean(long ledgerId) { * * ScanAndCompareGarbageCollector/GC should clean data of ledger only if both the LedgerManager.getLedgerRanges says * that ledger is not existing and also ledgerManager.readLedgerMetadata fails with error - * NoSuchLedgerExistsException, but is shouldn't delete if the readLedgerMetadata fails with any other error. + * NoSuchLedgerExistsOnMetadataServerException, but is shouldn't delete if the readLedgerMetadata fails with any + * other error. */ @Test public void testGcLedgersIfReadLedgerMetadataFailsForDeletedLedgers() throws Exception { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java index 1786fe4e126..011562e6ed0 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java @@ -271,7 +271,7 @@ public void testSuccessAllocatorShouldDeleteUnusedledger() throws Exception { try { bkc.get().openLedger(lh1.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes()); fail("LedgerHandle allocated by allocator1 should be deleted."); - } catch (BKException.BKNoSuchLedgerExistsException nslee) { + } catch (BKException.BKNoSuchLedgerExistsOnMetadataServerException nslee) { // as expected } long eid = lh2.addEntry("hello world".getBytes()); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java index 553e0172f65..4e03554b1ab 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java @@ -22,7 +22,7 @@ import static org.junit.Assert.fail; import java.net.URI; -import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; +import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsOnMetadataServerException; import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.distributedlog.DLMTestUtil; import org.apache.distributedlog.DLSN; @@ -203,7 +203,7 @@ public void testToolReadLastConfirmed() throws Exception { // correct functionality. try { cmd.runCmd(); - } catch (BKNoSuchLedgerExistsException ex) { + } catch (BKNoSuchLedgerExistsOnMetadataServerException ex) { } } @@ -214,7 +214,7 @@ public void testToolReadEntriesCommand() throws Exception { cmd.setLedgerId(99999999); try { cmd.runCmd(); - } catch (BKNoSuchLedgerExistsException ex) { + } catch (BKNoSuchLedgerExistsOnMetadataServerException ex) { } } diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java index c68a3e17122..6a6c8792d05 100644 --- a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java +++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestBookieShellCluster.java @@ -211,7 +211,8 @@ public void test102_DumpRestoreMetadata() throws Exception { validateNEntries(bk, ledgerId, numEntries); Assert.fail("Shouldn't have been able to find anything"); } catch (ExecutionException ee) { - Assert.assertEquals(ee.getCause().getClass(), BKException.BKNoSuchLedgerExistsException.class); + Assert.assertEquals(ee.getCause().getClass(), + BKException.BKNoSuchLedgerExistsOnMetadataServerException.class); } log.info("Restore the ledger metadata"); From 4e6bafcec9f7ce702a574a549fd54ceeb8b04eba Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 2 May 2019 11:37:14 -0700 Subject: [PATCH 0286/1642] Use pure python implementation of MurmurHash ### Motivation BK table python client is depending on `mmh3` library for MurmurHash. This library wraps a C based function but there are no binaries published on PyPI. Therefore users need to have a GCC installed in order to install the BK client lib, since it compiles it at install time. GCC is typically not available in Docker containers. ### Modifications Include a pure python implementation of MurmurHash to use if the C based is not found. Notes: * I couldn't find any published pure-python MurmurHash implementations on PyPI * Importing public-domain code is permitted by ASF Reviewers: Ivan Kelly , Enrico Olivelli This closes #2069 from merlimat/mmh3 --- stream/clients/python/bookkeeper/common/router/router.py | 7 ++++++- stream/clients/python/setup.py | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/stream/clients/python/bookkeeper/common/router/router.py b/stream/clients/python/bookkeeper/common/router/router.py index de4f31dc692..f52ef645e54 100644 --- a/stream/clients/python/bookkeeper/common/router/router.py +++ b/stream/clients/python/bookkeeper/common/router/router.py @@ -10,7 +10,12 @@ # See the License for the specific language governing permissions and # limitations under the License. -import mmh3 +try: + # Try with C based implemenation if available + import mmh3 +except ImportError: + # Fallback to pure python + import pymmh3 as mmh3 __SEED__ = 383242705 diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index fb8a696566a..2b205dc86c9 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -33,7 +33,7 @@ 'pytz', 'futures>=3.2.0;python_version<"3.2"', 'grpcio>=1.8.2', - 'mmh3>=2.5.1' + 'pymmh3>=0.0.3' ] extras = { } From 3d7648fbe5949964af13214a1d8043f8a2e306c6 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 2 May 2019 11:42:49 -0700 Subject: [PATCH 0287/1642] ISSUE #1967: make ledger creation and removal robust to zk connectionloss - fix test failures in TestLedgerMetadataSerDe Reviewers: Enrico Olivelli , Sijie Guo This closes #2070 from reddycharan/fixzktestfailures, closes #1967 --- .../bookkeeper/meta/LedgerMetadataSerDe.java | 22 ++++++++++++++----- .../meta/AbstractZkLedgerManagerTest.java | 14 ++++++++++-- .../meta/TestLedgerMetadataSerDe.java | 13 +++++------ 3 files changed, 35 insertions(+), 14 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java index b34b34bc6d7..2ee441abf4e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -181,7 +181,12 @@ private static byte[] serializeVersion3(LedgerMetadata metadata) throws IOExcept break; } - builder.setCtime(metadata.getCtime()); + /** Hack to get around fact that ctime was never versioned correctly */ + if (LedgerMetadataUtils.shouldStoreCtime(metadata)) { + builder.setCtime(metadata.getCtime()); + } + + builder.setDigestType(apiToProtoDigestType(metadata.getDigestType())); serializePassword(metadata.getPassword(), builder); @@ -351,7 +356,7 @@ public LedgerMetadata parseConfig(byte[] bytes, switch (metadataFormatVersion) { case METADATA_FORMAT_VERSION_3: - return parseVersion3Config(is); + return parseVersion3Config(is, metadataStoreCtime); case METADATA_FORMAT_VERSION_2: return parseVersion2Config(is, metadataStoreCtime); case METADATA_FORMAT_VERSION_1: @@ -365,12 +370,19 @@ public LedgerMetadata parseConfig(byte[] bytes, } } - private static LedgerMetadata parseVersion3Config(InputStream is) throws IOException { + private static LedgerMetadata parseVersion3Config(InputStream is, Optional metadataStoreCtime) + throws IOException { LedgerMetadataBuilder builder = LedgerMetadataBuilder.create() - .withMetadataFormatVersion(METADATA_FORMAT_VERSION_3); + .withMetadataFormatVersion(METADATA_FORMAT_VERSION_3); LedgerMetadataFormat.Builder formatBuilder = LedgerMetadataFormat.newBuilder(); formatBuilder.mergeDelimitedFrom(is); - decodeFormat(formatBuilder.build(), builder); + LedgerMetadataFormat data = formatBuilder.build(); + decodeFormat(data, builder); + if (data.hasCtime()) { + builder.storingCreationTime(true); + } else if (metadataStoreCtime.isPresent()) { + builder.withCreationTime(metadataStoreCtime.get()).storingCreationTime(false); + } return builder.build(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index 0dc2b537c0b..b54b7d97413 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -124,7 +124,6 @@ public void setup() throws Exception { this.metadata = LedgerMetadataBuilder.create() .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .withEnsembleSize(5) - .withMetadataFormatVersion(2) .withWriteQuorumSize(3) .withAckQuorumSize(3) .newEnsembleEntry(0L, ensemble) @@ -180,7 +179,18 @@ public void testCreateLedgerMetadataNodeExists() throws Exception { mockZkUtilsAsyncCreateFullPathOptimistic( ledgerStr, CreateMode.PERSISTENT, KeeperException.Code.NODEEXISTS.intValue(), null); - + Stat stat = mock(Stat.class); + when(stat.getVersion()).thenReturn(1234); + when(stat.getCtime()).thenReturn(metadata.getCtime()); + /* + * this is needed because in AbstractZkLedgerManager.readLedgerMetadata + * if MetadataFormatVersion is >2, then for createLedgerMetadata if we + * get NODEEXISTS exception then it will try to read to make sure ledger + * creation is robust to ZK connection loss. Please check Issue #1967. + */ + mockZkGetData( + ledgerStr, false, + KeeperException.Code.OK.intValue(), serDe.serialize(metadata), stat); try { result(ledgerManager.createLedgerMetadata(ledgerId, metadata)); fail("Should fail to create ledger metadata if the ledger already exists"); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java index 1c9de746c38..a159e6cefcd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java @@ -122,9 +122,9 @@ public void testVeryLongVersionSerDe() throws Exception { } @Test - public void testPeggedToV2SerDe() throws Exception { + public void testPeggedToV3SerDe() throws Exception { LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); - LedgerMetadata metadata = LedgerMetadataBuilder.create().withMetadataFormatVersion(2) + LedgerMetadata metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) .newEnsembleEntry(0L, Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 3181), @@ -134,11 +134,11 @@ public void testPeggedToV2SerDe() throws Exception { byte[] encoded = serDe.serialize(metadata); LedgerMetadata decoded = serDe.parseConfig(encoded, Optional.empty()); - Assert.assertEquals(2, decoded.getMetadataFormatVersion()); + Assert.assertEquals(LedgerMetadataSerDe.METADATA_FORMAT_VERSION_3, decoded.getMetadataFormatVersion()); } @Test - public void testStoreSystemtimeAsLedgerCtimeEnabledWithVersion2() + public void testStoreSystemtimeAsLedgerCtimeEnabledWithNewerVersion() throws Exception { LedgerMetadata lm = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) @@ -161,9 +161,9 @@ public void testStoreSystemtimeAsLedgerCtimeEnabledWithVersion2() } @Test - public void testStoreSystemtimeAsLedgerCtimeDisabledWithVersion2() + public void testStoreSystemtimeAsLedgerCtimeDisabledWithNewerVersion() throws Exception { - LedgerMetadata lm = LedgerMetadataBuilder.create().withMetadataFormatVersion(2) + LedgerMetadata lm = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) .newEnsembleEntry(0L, Lists.newArrayList( @@ -171,7 +171,6 @@ public void testStoreSystemtimeAsLedgerCtimeDisabledWithVersion2() new BookieSocketAddress("192.0.2.2", 1234), new BookieSocketAddress("192.0.2.3", 1234))) .build(); - LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); byte[] serialized = serDe.serialize(lm); LedgerMetadata deserialized = serDe.parseConfig(serialized, Optional.of(654321L)); From edd69ca0e9b116ae02d050d14d31c436617c93df Mon Sep 17 00:00:00 2001 From: karanmehta93 Date: Thu, 2 May 2019 12:38:31 -0700 Subject: [PATCH 0288/1642] ISSUE #2075: Bookieshell lastmark command isn't functional, always returning 0-0 Descriptions of the changes in this PR: ### Motivation `lastmark` shell command is to be used while debugging however currently it cannot be used since it returns incorrect value. ### Changes Updated `LedgerDirsManager` object initialization to use correct `ledgerDirs`. Master Issue: #2075 Reviewers: Charan Reddy Guttapalem , Sijie Guo This closes #2076 from karanmehta93/ISSUE-2075, closes #2075 --- .../bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java index 61c1f92657d..f925e1300f4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LastMarkCommand.java @@ -48,7 +48,7 @@ public LastMarkCommand() { @Override public boolean apply(ServerConfiguration conf, CliFlags flags) { LedgerDirsManager dirsManager = new LedgerDirsManager( - conf, conf.getJournalDirs(), + conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); File[] journalDirs = conf.getJournalDirs(); From 172da7eb96970158e56429ac28e77eb0c3eda071 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 3 May 2019 09:29:24 -0700 Subject: [PATCH 0289/1642] Added HTTP handler to expose bookie state ### Motivation Allow to check the state of the bookie through HTTP. The main reason is to allow light-weight probes that can be run frequently to establish whether the bookie is running and meets a minimum readiness criteria. One example is to automatically wait until the bookie has completed the startup sequence. ### Changes Added new HTTP handler that exposes the values from the bookie `StateManager`. Reviewers: Ivan Kelly , Enrico Olivelli , Jia Zhai , Sijie Guo This closes #1995 from merlimat/bookie-state --- .../apache/bookkeeper/http/HttpRouter.java | 4 + .../apache/bookkeeper/http/HttpServer.java | 5 +- .../server/http/BKHttpServiceProvider.java | 9 ++ .../http/service/BookieIsReadyService.java | 61 ++++++++++++ .../http/service/BookieStateService.java | 92 +++++++++++++++++++ .../server/http/TestHttpService.java | 42 ++++++++- site/docs/latest/admin/http.md | 34 +++++++ 7 files changed, 245 insertions(+), 2 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieIsReadyService.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieStateService.java diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java index b4251d7327c..99d5be685f6 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java @@ -47,6 +47,8 @@ public abstract class HttpRouter { public static final String EXPAND_STORAGE = "/api/v1/bookie/expand_storage"; public static final String GC = "/api/v1/bookie/gc"; public static final String GC_DETAILS = "/api/v1/bookie/gc_details"; + public static final String BOOKIE_STATE = "/api/v1/bookie/state"; + public static final String BOOKIE_IS_READY = "/api/v1/bookie/is_ready"; // autorecovery public static final String RECOVERY_BOOKIE = "/api/v1/autorecovery/bookie"; public static final String LIST_UNDER_REPLICATED_LEDGER = "/api/v1/autorecovery/list_under_replicated_ledger"; @@ -77,6 +79,8 @@ public HttpRouter(AbstractHttpHandlerFactory handlerFactory) { this.endpointHandlers.put(EXPAND_STORAGE, handlerFactory.newHandler(HttpServer.ApiType.EXPAND_STORAGE)); this.endpointHandlers.put(GC, handlerFactory.newHandler(HttpServer.ApiType.GC)); this.endpointHandlers.put(GC_DETAILS, handlerFactory.newHandler(HttpServer.ApiType.GC_DETAILS)); + this.endpointHandlers.put(BOOKIE_STATE, handlerFactory.newHandler(HttpServer.ApiType.BOOKIE_STATE)); + this.endpointHandlers.put(BOOKIE_IS_READY, handlerFactory.newHandler(HttpServer.ApiType.BOOKIE_IS_READY)); // autorecovery this.endpointHandlers.put(RECOVERY_BOOKIE, handlerFactory.newHandler(HttpServer.ApiType.RECOVERY_BOOKIE)); diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java index 0922b56d90d..c694a076721 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java @@ -35,7 +35,8 @@ enum StatusCode { REDIRECT(302), FORBIDDEN(403), NOT_FOUND(404), - INTERNAL_ERROR(500); + INTERNAL_ERROR(500), + SERVICE_UNAVAILABLE(503); private int value; @@ -79,6 +80,8 @@ enum ApiType { EXPAND_STORAGE, GC, GC_DETAILS, + BOOKIE_STATE, + BOOKIE_IS_READY, // autorecovery RECOVERY_BOOKIE, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java index 662777cadd5..24795e54e83 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java @@ -19,10 +19,13 @@ package org.apache.bookkeeper.server.http; import com.google.common.util.concurrent.ThreadFactoryBuilder; + import java.io.IOException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; + import lombok.extern.slf4j.Slf4j; + import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeperAdmin; @@ -37,6 +40,8 @@ import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.Auditor; import org.apache.bookkeeper.replication.AutoRecoveryMain; +import org.apache.bookkeeper.server.http.service.BookieIsReadyService; +import org.apache.bookkeeper.server.http.service.BookieStateService; import org.apache.bookkeeper.server.http.service.ConfigurationService; import org.apache.bookkeeper.server.http.service.DecommissionService; import org.apache.bookkeeper.server.http.service.DeleteLedgerService; @@ -214,6 +219,10 @@ public HttpEndpointService provideHttpEndpointService(ApiType type) { return new TriggerGCService(configuration, bookieServer); case GC_DETAILS: return new GCDetailsService(configuration, bookieServer); + case BOOKIE_STATE: + return new BookieStateService(bookieServer.getBookie()); + case BOOKIE_IS_READY: + return new BookieIsReadyService(bookieServer.getBookie()); // autorecovery case RECOVERY_BOOKIE: diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieIsReadyService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieIsReadyService.java new file mode 100644 index 00000000000..04e13b6d415 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieIsReadyService.java @@ -0,0 +1,61 @@ +/* + * 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.server.http.service; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.StateManager; +import org.apache.bookkeeper.http.HttpServer; +import org.apache.bookkeeper.http.service.HttpEndpointService; +import org.apache.bookkeeper.http.service.HttpServiceRequest; +import org.apache.bookkeeper.http.service.HttpServiceResponse; + +/** + * HttpEndpointService that returns 200 if the bookie is ready. + */ +public class BookieIsReadyService implements HttpEndpointService { + + private final Bookie bookie; + + public BookieIsReadyService(Bookie bookie) { + this.bookie = checkNotNull(bookie); + } + + @Override + public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { + HttpServiceResponse response = new HttpServiceResponse(); + + if (HttpServer.Method.GET != request.getMethod()) { + response.setCode(HttpServer.StatusCode.NOT_FOUND); + response.setBody("Only support GET method check if bookie is ready."); + return response; + } + + StateManager sm = bookie.getStateManager(); + if (sm.isRunning() && !sm.isShuttingDown()) { + response.setCode(HttpServer.StatusCode.OK); + response.setBody(""); + } else { + response.setCode(HttpServer.StatusCode.SERVICE_UNAVAILABLE); + response.setBody("Bookie is not fully started yet"); + } + return response; + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieStateService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieStateService.java new file mode 100644 index 00000000000..45ac2c3086a --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieStateService.java @@ -0,0 +1,92 @@ +/* + * 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.server.http.service; + +import static com.google.common.base.Preconditions.checkNotNull; + +import lombok.Data; +import lombok.NoArgsConstructor; + +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.StateManager; +import org.apache.bookkeeper.common.util.JsonUtil; +import org.apache.bookkeeper.http.HttpServer; +import org.apache.bookkeeper.http.service.HttpEndpointService; +import org.apache.bookkeeper.http.service.HttpServiceRequest; +import org.apache.bookkeeper.http.service.HttpServiceResponse; + +/** + * HttpEndpointService that exposes the current state of the bookie. + * + *

            Get the current bookie status: + * + *

            + * 
            + * {
            + *  "running" : true,
            + *  "readOnly" : false,
            + *  "shuttingDown" : false,
            + *  "availableForHighPriorityWrites" : true
            + *}
            + * 
            + * 
            + */ +public class BookieStateService implements HttpEndpointService { + + private final Bookie bookie; + + public BookieStateService(Bookie bookie) { + this.bookie = checkNotNull(bookie); + } + + /** + * POJO definition for the bookie state response. + */ + @Data + @NoArgsConstructor + public static class BookieState { + private boolean running; + private boolean readOnly; + private boolean shuttingDown; + private boolean availableForHighPriorityWrites; + } + + @Override + public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { + HttpServiceResponse response = new HttpServiceResponse(); + + if (HttpServer.Method.GET != request.getMethod()) { + response.setCode(HttpServer.StatusCode.NOT_FOUND); + response.setBody("Only support GET method to retrieve bookie state."); + return response; + } + + StateManager sm = bookie.getStateManager(); + BookieState bs = new BookieState(); + bs.running = sm.isRunning(); + bs.readOnly = sm.isReadOnly(); + bs.shuttingDown = sm.isShuttingDown(); + bs.availableForHighPriorityWrites = sm.isAvailableForHighPriorityWrites(); + + String jsonResponse = JsonUtil.toJson(bs); + response.setBody(jsonResponse); + response.setCode(HttpServer.StatusCode.OK); + return response; + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index ee6b278a0a3..02aa42b864b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -26,12 +26,15 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.UncheckedExecutionException; + import java.io.File; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; import java.util.concurrent.Future; + import lombok.Cleanup; + import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.ClientUtil; import org.apache.bookkeeper.client.LedgerHandle; @@ -48,8 +51,8 @@ import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.replication.AuditorElector; +import org.apache.bookkeeper.server.http.service.BookieStateService.BookieState; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; - import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -826,4 +829,41 @@ public void testGCDetailsService() throws Exception { HttpServiceResponse response3 = gcDetailsService.handle(request3); assertEquals(HttpServer.StatusCode.NOT_FOUND.getValue(), response3.getStatusCode()); } + + @Test + public void testGetBookieState() throws Exception { + HttpEndpointService bookieStateServer = bkHttpServiceProvider + .provideHttpEndpointService(HttpServer.ApiType.BOOKIE_STATE); + + HttpServiceRequest request1 = new HttpServiceRequest(null, HttpServer.Method.GET, null); + HttpServiceResponse response1 = bookieStateServer.handle(request1); + assertEquals(HttpServer.StatusCode.OK.getValue(), response1.getStatusCode()); + + BookieState bs = JsonUtil.fromJson(response1.getBody(), BookieState.class); + assertEquals(true, bs.isRunning()); + assertEquals(false, bs.isReadOnly()); + assertEquals(true, bs.isAvailableForHighPriorityWrites()); + assertEquals(false, bs.isShuttingDown()); + } + + @Test + public void testGetBookieIsReady() throws Exception { + HttpEndpointService bookieStateServer = bkHttpServiceProvider + .provideHttpEndpointService(HttpServer.ApiType.BOOKIE_IS_READY); + + HttpServiceRequest request1 = new HttpServiceRequest(null, HttpServer.Method.GET, null); + HttpServiceResponse response1 = bookieStateServer.handle(request1); + assertEquals(HttpServer.StatusCode.OK.getValue(), response1.getStatusCode()); + + // Try using POST instead of GET + HttpServiceRequest request2 = new HttpServiceRequest(null, HttpServer.Method.POST, null); + HttpServiceResponse response2 = bookieStateServer.handle(request2); + assertEquals(HttpServer.StatusCode.NOT_FOUND.getValue(), response2.getStatusCode()); + + // Simulate bookies shutting down + bs.forEach(bookieServer -> bookieServer.getBookie().getStateManager().forceToShuttingDown()); + HttpServiceRequest request3 = new HttpServiceRequest(null, HttpServer.Method.GET, null); + HttpServiceResponse response3 = bookieStateServer.handle(request3); + assertEquals(HttpServer.StatusCode.SERVICE_UNAVAILABLE.getValue(), response3.getStatusCode()); + } } diff --git a/site/docs/latest/admin/http.md b/site/docs/latest/admin/http.md index 270dcb3a16f..c7fcf0cf5c7 100644 --- a/site/docs/latest/admin/http.md +++ b/site/docs/latest/admin/http.md @@ -321,6 +321,40 @@ Currently all the HTTP endpoints could be divided into these 4 components: } ] ``` +### Endpoint: /api/v1/bookie/state +1. Method: GET + * Description: Exposes the current state of bookie + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + { + "running" : true, + "readOnly" : false, + "shuttingDown" : false, + "availableForHighPriorityWrites" : true + } + ``` + +### Endpoint: /api/v1/bookie/is_ready +1. Method: GET + * Description: Return true if the bookie is ready + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + |503 | Bookie is not ready | + * Body: + + ## Auto recovery ### Endpoint: /api/v1/autorecovery/bookie/ From 871bf7c8c9413000783d1c0a73e7a4649eada9ca Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Sat, 4 May 2019 15:28:01 -0700 Subject: [PATCH 0290/1642] GetListOfEntriesOfLedger implementation Descriptions of the changes in this PR: As described in this BP - https://github.com/apache/bookkeeper/blob/master/site/bps/BP-34-cluster-metadata-checker.md, this request returns list of entries bookie contains for the given ledger in an encoded format. The returned list provides weakly consistent state, of the entries of the ledger. Reviewers: Enrico Olivelli , Sijie Guo This closes #1931 from reddycharan/getlistofentries and squashes the following commits: 467bb8b73 [Charan Reddy Guttapalem] GetListOfEntriesOfLedger implementation 757f99156 [Charan Reddy Guttapalem] GetListOfEntriesOfLedger implementation 4e84bcba0 [Charan Reddy Guttapalem] GetListOfEntriesOfLedger implementation f5655bb29 [Charan Reddy Guttapalem] GetListOfEntriesOfLedger implementation --- .../src/main/proto/BookkeeperProtocol.proto | 13 + .../bookie/BookKeeperServerStats.java | 3 + .../org/apache/bookkeeper/bookie/Bookie.java | 22 + .../bookkeeper/bookie/EntryMemTable.java | 49 ++- .../bookie/InterleavedLedgerStorage.java | 16 +- .../InterleavedStorageRegenerateIndexOp.java | 5 + .../apache/bookkeeper/bookie/LedgerCache.java | 4 + .../bookkeeper/bookie/LedgerCacheImpl.java | 49 +++ .../bookkeeper/bookie/LedgerDescriptor.java | 4 + .../bookie/LedgerDescriptorImpl.java | 6 + .../bookkeeper/bookie/LedgerEntryPage.java | 27 ++ .../bookkeeper/bookie/LedgerStorage.java | 17 + .../bookie/SortedLedgerStorage.java | 11 + .../bookkeeper/bookie/stats/BookieStats.java | 9 + .../bookie/storage/ldb/DbLedgerStorage.java | 8 + .../ldb/SingleDirectoryDbLedgerStorage.java | 7 + .../bookkeeper/client/BookKeeperAdmin.java | 18 +- .../client/BookKeeperClientStats.java | 2 + .../apache/bookkeeper/proto/BookieClient.java | 15 + .../bookkeeper/proto/BookieClientImpl.java | 31 ++ .../proto/BookieRequestProcessor.java | 13 + .../proto/BookkeeperInternalCallbacks.java | 48 +++ .../GetListOfEntriesOfLedgerProcessorV3.java | 110 +++++ .../proto/PerChannelBookieClient.java | 83 ++++ .../apache/bookkeeper/proto/RequestStats.java | 17 + .../util/AvailabilityOfEntriesOfLedger.java | 377 ++++++++++++++++++ .../bookkeeper/util/IteratorUtility.java | 171 ++++++++ .../bookie/InterleavedLedgerStorageTest.java | 56 ++- .../bookkeeper/bookie/LedgerStorageTest.java | 45 +++ .../bookie/SortedLedgerStorageTest.java | 194 +++++++++ .../bookkeeper/bookie/TestEntryMemTable.java | 154 +++++++ .../bookkeeper/bookie/TestSyncThread.java | 6 + .../client/BookKeeperAdminTest.java | 98 +++++ .../apache/bookkeeper/meta/GcLedgersTest.java | 6 + .../meta/LedgerManagerTestCase.java | 6 + .../bookkeeper/proto/MockBookieClient.java | 13 + .../AvailabilityOfEntriesOfLedgerTest.java | 192 +++++++++ .../bookkeeper/util/IteratorUtilityTest.java | 141 +++++++ 38 files changed, 2041 insertions(+), 5 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/util/IteratorUtility.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageTest.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedgerTest.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/util/IteratorUtilityTest.java diff --git a/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto b/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto index f34d56e5919..4f178aaa52c 100644 --- a/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto +++ b/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto @@ -64,6 +64,7 @@ enum OperationType { GET_BOOKIE_INFO = 8; START_TLS = 9; FORCE_LEDGER = 10; + GET_LIST_OF_ENTRIES_OF_LEDGER = 11; } /** @@ -92,6 +93,7 @@ message Request { optional GetBookieInfoRequest getBookieInfoRequest = 105; optional StartTLSRequest startTLSRequest = 106; optional ForceLedgerRequest forceLedgerRequest = 107; + optional GetListOfEntriesOfLedgerRequest getListOfEntriesOfLedgerRequest = 108; // to pass MDC context repeated ContextPair requestContext = 200; } @@ -152,6 +154,10 @@ message GetBookieInfoRequest { optional int64 requested = 1; } +message GetListOfEntriesOfLedgerRequest { + required int64 ledgerId = 1; +} + message Response { required BKPacketHeader header = 1; @@ -167,6 +173,7 @@ message Response { optional GetBookieInfoResponse getBookieInfoResponse = 105; optional StartTLSResponse startTLSResponse = 106; optional ForceLedgerResponse forceLedgerResponse = 107; + optional GetListOfEntriesOfLedgerResponse getListOfEntriesOfLedgerResponse = 108; } message ReadResponse { @@ -213,5 +220,11 @@ message GetBookieInfoResponse { optional int64 freeDiskSpace = 3; } +message GetListOfEntriesOfLedgerResponse { + required StatusCode status = 1; + required int64 ledgerId = 2; + optional bytes availabilityOfEntriesOfLedger = 3; // condensed encoded format representing availability of entries of ledger +} + message StartTLSResponse { } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java index b58514b6709..bdca0481a07 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java @@ -64,6 +64,8 @@ public interface BookKeeperServerStats { String READ_LAC = "READ_LAC"; String GET_BOOKIE_INFO_REQUEST = "GET_BOOKIE_INFO_REQUEST"; String GET_BOOKIE_INFO = "GET_BOOKIE_INFO"; + String GET_LIST_OF_ENTRIES_OF_LEDGER = "GET_LIST_OF_ENTRIES_OF_LEDGER"; + String GET_LIST_OF_ENTRIES_OF_LEDGER_REQUEST = "GET_LIST_OF_ENTRIES_OF_LEDGER_REQUEST"; // Ensemble Stats String WATCHER_SCOPE = "bookie_watcher"; @@ -80,6 +82,7 @@ public interface BookKeeperServerStats { String BOOKIE_READ_LAST_CONFIRMED = "BOOKIE_READ_LAST_CONFIRMED"; String BOOKIE_ADD_ENTRY_BYTES = "BOOKIE_ADD_ENTRY_BYTES"; String BOOKIE_READ_ENTRY_BYTES = "BOOKIE_READ_ENTRY_BYTES"; + String BOOKIE_GET_LIST_OF_ENTRIES_OF_LEDGER = "BOOKIE_GET_LIST_OF_ENTRIES_OF_LEDGER"; String ADD_ENTRY_IN_PROGRESS = "ADD_ENTRY_IN_PROGRESS"; String ADD_ENTRY_BLOCKED = "ADD_ENTRY_BLOCKED"; 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 e253fdcee6d..32adb5d9034 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 @@ -52,6 +52,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.PrimitiveIterator.OfLong; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -1594,4 +1595,25 @@ static LedgerDirsManager createIndexDirsManager(ServerConfiguration conf, DiskCh return new LedgerDirsManager(conf, idxDirs, diskChecker, statsLogger); } } + + public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException, NoLedgerException { + long requestNanos = MathUtils.nowInNano(); + boolean success = false; + try { + LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId); + if (LOG.isTraceEnabled()) { + LOG.trace("GetEntriesOfLedger {}", ledgerId); + } + OfLong entriesOfLedger = handle.getListOfEntriesOfLedger(ledgerId); + success = true; + return entriesOfLedger; + } finally { + long elapsedNanos = MathUtils.elapsedNanos(requestNanos); + if (success) { + bookieStats.getReadEntryStats().registerSuccessfulEvent(elapsedNanos, TimeUnit.NANOSECONDS); + } else { + bookieStats.getReadEntryStats().registerFailedEvent(elapsedNanos, TimeUnit.NANOSECONDS); + } + } + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java index a27e54eb2d7..941906c045d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java @@ -21,17 +21,21 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.PrimitiveIterator; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; + import org.apache.bookkeeper.bookie.Bookie.NoLedgerException; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.bookie.stats.EntryMemTableStats; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.util.IteratorUtility; import org.apache.bookkeeper.util.MathUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,7 +48,6 @@ */ public class EntryMemTable implements AutoCloseable{ private static Logger logger = LoggerFactory.getLogger(Journal.class); - /** * Entry skip list. */ @@ -456,4 +459,48 @@ boolean isEmpty() { public void close() throws Exception { // no-op } + + /* + * returns the primitive long iterator of entries of a ledger available in + * this EntryMemTable. It would be in the ascending order and this Iterator + * is weakly consistent. + */ + PrimitiveIterator.OfLong getListOfEntriesOfLedger(long ledgerId) { + EntryKey thisLedgerFloorEntry = new EntryKey(ledgerId, 0); + EntryKey thisLedgerCeilingEntry = new EntryKey(ledgerId, Long.MAX_VALUE); + Iterator thisLedgerEntriesInKVMap; + Iterator thisLedgerEntriesInSnapshot; + this.lock.readLock().lock(); + try { + /* + * Gets a view of the portion of this map that corresponds to + * entries of this ledger. + * + * Here 'kvmap' is of type 'ConcurrentSkipListMap', so its 'subMap' + * call would return a view of the portion of this map whose keys + * range from fromKey to toKey and it would be of type + * 'ConcurrentNavigableMap'. ConcurrentNavigableMap's 'keySet' would + * return NavigableSet view of the keys contained in this map. This + * view's iterator would be weakly consistent - + * https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ + * package-summary.html#Weakly. + * + * 'weakly consistent' would guarantee 'to traverse elements as they + * existed upon construction exactly once, and may (but are not + * guaranteed to) reflect any modifications subsequent to + * construction.' + * + */ + thisLedgerEntriesInKVMap = this.kvmap.subMap(thisLedgerFloorEntry, thisLedgerCeilingEntry).keySet() + .iterator(); + thisLedgerEntriesInSnapshot = this.snapshot.subMap(thisLedgerFloorEntry, thisLedgerCeilingEntry).keySet() + .iterator(); + } finally { + this.lock.readLock().unlock(); + } + return IteratorUtility.mergeIteratorsForPrimitiveLongIterator(thisLedgerEntriesInKVMap, + thisLedgerEntriesInSnapshot, EntryKey.COMPARATOR, (entryKey) -> { + return entryKey.entryId; + }); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java index becb16bd1ea..5d4ec0e3dd4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java @@ -34,6 +34,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.util.concurrent.RateLimiter; + import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; @@ -45,12 +46,14 @@ import java.util.Map; import java.util.NavigableMap; import java.util.Optional; +import java.util.PrimitiveIterator.OfLong; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; import lombok.Getter; + import org.apache.bookkeeper.bookie.Bookie.NoLedgerException; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.bookie.EntryLogger.EntryLogListener; @@ -65,8 +68,8 @@ import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.SnapshotMap; -import org.apache.commons.lang.mutable.MutableBoolean; -import org.apache.commons.lang.mutable.MutableLong; +import org.apache.commons.lang3.mutable.MutableBoolean; +import org.apache.commons.lang3.mutable.MutableLong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,6 +86,7 @@ ) public class InterleavedLedgerStorage implements CompactableLedgerStorage, EntryLogListener { private static final Logger LOG = LoggerFactory.getLogger(InterleavedLedgerStorage.class); + public static final long INVALID_ENTRYID = -1; EntryLogger entryLogger; @Getter @@ -103,6 +107,8 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry // this indicates that a write has happened since the last flush private final AtomicBoolean somethingWritten = new AtomicBoolean(false); + private int pageSize; + // Expose Stats @StatsDoc( name = STORAGE_GET_OFFSET, @@ -191,6 +197,7 @@ public void initializeWithEntryLogger(ServerConfiguration conf, ledgerCache = new LedgerCacheImpl(conf, activeLedgers, null == indexDirsManager ? ledgerDirsManager : indexDirsManager, statsLogger); gcThread = new GarbageCollectorThread(conf, ledgerManager, this, statsLogger.scope("gc")); + pageSize = conf.getPageSize(); ledgerDirsManager.addLedgerDirsListener(getLedgerDirsListener()); // Expose Stats getOffsetStats = statsLogger.getOpStatsLogger(STORAGE_GET_OFFSET); @@ -649,4 +656,9 @@ public List localConsistencyCheck(Optional r public List getGarbageCollectionStatus() { return Collections.singletonList(gcThread.getGarbageCollectionStatus()); } + + @Override + public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { + return ledgerCache.getEntriesIterator(ledgerId); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java index 880d3c190be..90f5afced69 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java @@ -26,6 +26,7 @@ import java.security.NoSuchAlgorithmException; import java.util.HashMap; import java.util.Map; +import java.util.PrimitiveIterator.OfLong; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -240,5 +241,9 @@ public PageEntriesIterable listEntries(long ledgerId) throws IOException { public LedgerIndexMetadata readLedgerIndexMetadata(long ledgerId) throws IOException { throw new UnsupportedOperationException(); } + @Override + public OfLong getEntriesIterator(long ledgerId) throws IOException { + throw new UnsupportedOperationException(); + } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java index cae8bb42eac..606afb46e73 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java @@ -26,6 +26,8 @@ import io.netty.buffer.ByteBuf; import java.io.Closeable; import java.io.IOException; +import java.util.PrimitiveIterator.OfLong; + import org.apache.bookkeeper.common.util.Watcher; /** @@ -87,6 +89,8 @@ interface PageEntriesIterable extends AutoCloseable, Iterable {} PageEntriesIterable listEntries(long ledgerId) throws IOException; + OfLong getEntriesIterator(long ledgerId) throws IOException; + /** * Represents summary of ledger metadata. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java index e6de2f9a43c..7341ace65c7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java @@ -23,6 +23,11 @@ import io.netty.buffer.ByteBuf; import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.PrimitiveIterator.OfLong; + import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -186,4 +191,48 @@ public PageEntriesIterable listEntries(long ledgerId) throws IOException { public LedgerIndexMetadata readLedgerIndexMetadata(long ledgerId) throws IOException { return indexPersistenceManager.readLedgerIndexMetadata(ledgerId); } + + @Override + public OfLong getEntriesIterator(long ledgerId) throws IOException { + Iterator pageEntriesIteratorNonFinal = null; + try { + pageEntriesIteratorNonFinal = listEntries(ledgerId).iterator(); + } catch (Bookie.NoLedgerException noLedgerException) { + pageEntriesIteratorNonFinal = Collections.emptyIterator(); + } + final Iterator pageEntriesIterator = pageEntriesIteratorNonFinal; + return new OfLong() { + private OfLong entriesInCurrentLEPIterator = null; + { + if (pageEntriesIterator.hasNext()) { + entriesInCurrentLEPIterator = pageEntriesIterator.next().getLEP().getEntriesIterator(); + } + } + + @Override + public boolean hasNext() { + try { + while ((entriesInCurrentLEPIterator != null) && (!entriesInCurrentLEPIterator.hasNext())) { + if (pageEntriesIterator.hasNext()) { + entriesInCurrentLEPIterator = pageEntriesIterator.next().getLEP().getEntriesIterator(); + } else { + entriesInCurrentLEPIterator = null; + } + } + return (entriesInCurrentLEPIterator != null); + } catch (Exception exc) { + throw new RuntimeException( + "Received exception in InterleavedLedgerStorage getEntriesOfLedger hasNext call", exc); + } + } + + @Override + public long nextLong() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return entriesInCurrentLEPIterator.nextLong(); + } + }; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java index 74bc8b52cd8..abb7b34e287 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java @@ -27,6 +27,8 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import java.io.IOException; +import java.util.PrimitiveIterator.OfLong; + import org.apache.bookkeeper.common.util.Watcher; /** @@ -86,4 +88,6 @@ abstract void cancelWaitForLastAddConfirmedUpdate(Watcher watcher) throws IOException { ledgerStorage.cancelWaitForLastAddConfirmedUpdate(ledgerId, watcher); } + + @Override + OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { + return ledgerStorage.getListOfEntriesOfLedger(ledgerId); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java index c272e7ca0bf..98ea9484155 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java @@ -23,6 +23,8 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.NoSuchElementException; +import java.util.PrimitiveIterator.OfLong; import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.util.ZeroBuffer; @@ -298,6 +300,31 @@ public void getEntries(EntryVisitor vis) throws Exception { } } + public OfLong getEntriesIterator() { + return new OfLong() { + long firstEntry = getFirstEntry(); + int curDiffEntry = 0; + + @Override + public boolean hasNext() { + while ((curDiffEntry < entriesPerPage) && (getOffset(curDiffEntry * 8) == 0)) { + curDiffEntry++; + } + return (curDiffEntry != entriesPerPage); + } + + @Override + public long nextLong() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + long nextEntry = firstEntry + curDiffEntry; + curDiffEntry++; + return nextEntry; + } + }; + } + @Override public void close() throws Exception { releasePage(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java index 2c47b2888ff..d371520f0ce 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java @@ -30,6 +30,8 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.PrimitiveIterator; + import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -258,4 +260,19 @@ default boolean isInForceGC() { default List getGarbageCollectionStatus() { return Collections.emptyList(); } + + /** + * Returns the primitive long iterator for entries of the ledger, stored in + * this LedgerStorage. The returned iterator provide weakly consistent state + * of the ledger. It is guaranteed that entries of the ledger added to this + * LedgerStorage by the time this method is called will be available but + * modifications made after method invocation may not be available. + * + * @param ledgerId + * - id of the ledger + * @return the list of entries of the ledger available in this + * ledgerstorage. + * @throws Exception + */ + PrimitiveIterator.OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java index 23e07163cb2..c1ad591037c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java @@ -23,21 +23,25 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.RateLimiter; import com.google.common.util.concurrent.ThreadFactoryBuilder; + import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import java.io.IOException; import java.util.List; import java.util.Optional; +import java.util.PrimitiveIterator; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; + import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.util.IteratorUtility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -361,4 +365,11 @@ public boolean isInForceGC() { public List getGarbageCollectionStatus() { return interleavedLedgerStorage.getGarbageCollectionStatus(); } + + @Override + public PrimitiveIterator.OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { + PrimitiveIterator.OfLong entriesInMemtableItr = memTable.getListOfEntriesOfLedger(ledgerId); + PrimitiveIterator.OfLong entriesFromILSItr = interleavedLedgerStorage.getListOfEntriesOfLedger(ledgerId); + return IteratorUtility.mergePrimitiveLongIterator(entriesInMemtableItr, entriesFromILSItr); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java index 5e033e9e9f3..5df21864eec 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/stats/BookieStats.java @@ -23,11 +23,13 @@ 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_FORCE_LEDGER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_GET_LIST_OF_ENTRIES_OF_LEDGER; 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.CATEGORY_SERVER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.GET_LIST_OF_ENTRIES_OF_LEDGER; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_BYTES; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_BYTES; @@ -72,6 +74,12 @@ public class BookieStats { parent = READ_ENTRY ) private final OpStatsLogger readEntryStats; + @StatsDoc( + name = BOOKIE_GET_LIST_OF_ENTRIES_OF_LEDGER, + help = "operation stats of GetListOfEntriesOfLedger on a bookie", + parent = GET_LIST_OF_ENTRIES_OF_LEDGER + ) + private final OpStatsLogger getListOfEntriesOfLedgerStats; // Bookie Operation Bytes Stats @StatsDoc(name = BOOKIE_ADD_ENTRY_BYTES, help = "bytes stats of AddEntry on a bookie") private final OpStatsLogger addBytesStats; @@ -86,6 +94,7 @@ public BookieStats(StatsLogger statsLogger) { addEntryStats = statsLogger.getOpStatsLogger(BOOKIE_ADD_ENTRY); recoveryAddEntryStats = statsLogger.getOpStatsLogger(BOOKIE_RECOVERY_ADD_ENTRY); readEntryStats = statsLogger.getOpStatsLogger(BOOKIE_READ_ENTRY); + getListOfEntriesOfLedgerStats = statsLogger.getOpStatsLogger(BOOKIE_GET_LIST_OF_ENTRIES_OF_LEDGER); addBytesStats = statsLogger.getOpStatsLogger(BOOKIE_ADD_ENTRY_BYTES); readBytesStats = statsLogger.getOpStatsLogger(BOOKIE_READ_ENTRY_BYTES); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index a68cd1cd7b7..73d254a0cc3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -37,6 +37,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.PrimitiveIterator.OfLong; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -359,4 +360,11 @@ static long getLongVariableOrDefault(ServerConfiguration conf, String keyName, l return conf.getLong(keyName); } } + + @Override + public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { + // check Issue #2078 + throw new UnsupportedOperationException( + "getListOfEntriesOfLedger method is currently unsupported for DbLedgerStorage"); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 4c4123515f4..58ce2beb319 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -33,6 +33,7 @@ import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.PrimitiveIterator.OfLong; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -888,4 +889,10 @@ public interface LedgerLoggerProcessor { } private static final Logger log = LoggerFactory.getLogger(DbLedgerStorage.class); + + @Override + public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { + throw new UnsupportedOperationException( + "getListOfEntriesOfLedger method is currently unsupported for SingleDirectoryDbLedgerStorage"); + } } 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 6d2319cb57d..4a869eee3ba 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 @@ -27,8 +27,8 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; - import com.google.common.util.concurrent.UncheckedExecutionException; + import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -80,6 +80,7 @@ import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.IOUtils; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; @@ -1663,4 +1664,19 @@ public boolean isEnsembleAdheringToPlacementPolicy(List ens return bkc.getPlacementPolicy().isEnsembleAdheringToPlacementPolicy(ensembleBookiesList, writeQuorumSize, ackQuorumSize); } + + /** + * Makes async request for getting list of entries of ledger from a bookie + * and returns Future for the result. + * + * @param address + * BookieSocketAddress of the bookie + * @param ledgerId + * ledgerId + * @return returns Future + */ + public CompletableFuture asyncGetListOfEntriesOfLedger(BookieSocketAddress address, + long ledgerId) { + return bkc.getBookieClient().getListOfEntriesOfLedger(address, ledgerId); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java index cdfde67693d..36c304c5c27 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java @@ -64,6 +64,7 @@ public interface BookKeeperClientStats { String GET_BOOKIE_INFO_OP = "GET_BOOKIE_INFO"; String SPECULATIVE_READ_COUNT = "SPECULATIVE_READ_COUNT"; String READ_REQUESTS_REORDERED = "READ_REQUESTS_REORDERED"; + String GET_LIST_OF_ENTRIES_OF_LEDGER_OP = "GET_LIST_OF_ENTRIES_OF_LEDGER"; // per channel stats String CHANNEL_SCOPE = "per_channel_bookie_client"; @@ -81,6 +82,7 @@ public interface BookKeeperClientStats { String TIMEOUT_GET_BOOKIE_INFO = "TIMEOUT_GET_BOOKIE_INFO"; String CHANNEL_START_TLS_OP = "START_TLS"; String CHANNEL_TIMEOUT_START_TLS_OP = "TIMEOUT_START_TLS"; + String TIMEOUT_GET_LIST_OF_ENTRIES_OF_LEDGER = "TIMEOUT_GET_LIST_OF_ENTRIES_OF_LEDGER"; String NETTY_EXCEPTION_CNT = "NETTY_EXCEPTION_CNT"; String CLIENT_CHANNEL_WRITE_WAIT = "CLIENT_CHANNEL_WRITE_WAIT"; 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 85a4ef9c164..2092a67af41 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 @@ -22,6 +22,7 @@ import java.util.EnumSet; import java.util.List; +import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -31,6 +32,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadLacCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteLacCallback; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.ByteBufList; /** @@ -215,6 +217,19 @@ void readEntryWaitForLACUpdate(BookieSocketAddress address, void getBookieInfo(BookieSocketAddress address, long requested, GetBookieInfoCallback cb, Object ctx); + /** + * Makes async request for getting list of entries of ledger from a bookie + * and returns Future for the result. + * + * @param address + * BookieSocketAddress of the bookie + * @param ledgerId + * ledgerId + * @return returns Future + */ + CompletableFuture getListOfEntriesOfLedger(BookieSocketAddress address, + long ledgerId); + /** * @return whether bookie client object has been closed */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index 83428214a5d..c772a97cdaa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -38,6 +38,7 @@ import java.io.IOException; import java.util.EnumSet; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.RejectedExecutionException; @@ -56,6 +57,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.FutureGetListOfEntriesOfLedger; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; @@ -66,6 +68,7 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.tls.SecurityHandlerFactory; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.ByteBufList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -317,6 +320,33 @@ public void addEntry(final BookieSocketAddress addr, ledgerId); } + @Override + public CompletableFuture getListOfEntriesOfLedger(BookieSocketAddress address, + long ledgerId) { + FutureGetListOfEntriesOfLedger futureResult = new FutureGetListOfEntriesOfLedger(ledgerId); + final PerChannelBookieClientPool client = lookupClient(address); + if (client == null) { + futureResult.getListOfEntriesOfLedgerComplete(getRc(BKException.Code.BookieHandleNotAvailableException), + ledgerId, null); + return futureResult; + } + client.obtain((rc, pcbc) -> { + if (rc != BKException.Code.OK) { + try { + executor.executeOrdered(ledgerId, safeRun(() -> { + futureResult.getListOfEntriesOfLedgerComplete(rc, ledgerId, null); + })); + } catch (RejectedExecutionException re) { + futureResult.getListOfEntriesOfLedgerComplete(getRc(BKException.Code.InterruptedException), + ledgerId, null); + } + } else { + pcbc.getListOfEntriesOfLedger(ledgerId, futureResult); + } + }, ledgerId); + return futureResult; + } + private void completeRead(final int rc, final long ledgerId, final long entryId, @@ -415,6 +445,7 @@ public void recycle() { } } + @Override public void readLac(final BookieSocketAddress addr, final long ledgerId, final ReadLacCallback cb, final Object ctx) { final PerChannelBookieClientPool client = lookupClient(addr); 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 02b0f568581..f46d19c8097 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 @@ -339,6 +339,9 @@ public void processRequest(Object msg, Channel c) { case START_TLS: processStartTLSRequestV3(r, c); break; + case GET_LIST_OF_ENTRIES_OF_LEDGER: + processGetListOfEntriesOfLedgerProcessorV3(r, c); + break; default: LOG.info("Unknown operation type {}", header.getOperation()); BookkeeperProtocol.Response.Builder response = @@ -587,6 +590,16 @@ private void processGetBookieInfoRequestV3(final BookkeeperProtocol.Request r, f } } + private void processGetListOfEntriesOfLedgerProcessorV3(final BookkeeperProtocol.Request r, final Channel c) { + GetListOfEntriesOfLedgerProcessorV3 getListOfEntriesOfLedger = new GetListOfEntriesOfLedgerProcessorV3(r, c, + this); + if (null == readThreadPool) { + getListOfEntriesOfLedger.run(); + } else { + readThreadPool.submit(getListOfEntriesOfLedger); + } + } + private void processAddRequest(final BookieProtocol.ParsedAddRequest r, final Channel c) { WriteEntryProcessor write = WriteEntryProcessor.create(r, c, this); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java index 2ade9e9c299..fdfd379aa70 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java @@ -37,6 +37,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.versioning.Versioned; import org.apache.zookeeper.AsyncCallback; @@ -107,6 +108,53 @@ public interface StartTLSCallback { void startTLSComplete(int rc, Object ctx); } + /** + * A callback interface for GetListOfEntriesOfLedger command. + */ + public interface GetListOfEntriesOfLedgerCallback { + void getListOfEntriesOfLedgerComplete(int rc, long ledgerId, + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger); + } + + /** + * Handle the Response Code and transform it to a BKException. + * + * @param + * @param rc + * @param result + * @param future + */ + public static void finish(int rc, T result, CompletableFuture future) { + if (rc != BKException.Code.OK) { + future.completeExceptionally(BKException.create(rc).fillInStackTrace()); + } else { + future.complete(result); + } + } + + /** + * Future for GetListOfEntriesOfLedger. + */ + public static class FutureGetListOfEntriesOfLedger extends CompletableFuture + implements GetListOfEntriesOfLedgerCallback { + private final long ledgerIdOfTheRequest; + + FutureGetListOfEntriesOfLedger(long ledgerId) { + this.ledgerIdOfTheRequest = ledgerId; + } + + @Override + public void getListOfEntriesOfLedgerComplete(int rc, long ledgerIdOfTheResponse, + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger) { + if ((rc == BKException.Code.OK) && (ledgerIdOfTheRequest != ledgerIdOfTheResponse)) { + LOG.error("For getListOfEntriesOfLedger expected ledgerId in the response: {} actual ledgerId: {}", + ledgerIdOfTheRequest, ledgerIdOfTheResponse); + rc = BKException.Code.ReadException; + } + finish(rc, availabilityOfEntriesOfLedger, this); + } + } + /** * A generic callback interface. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java new file mode 100644 index 00000000000..f78cdf2faf1 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java @@ -0,0 +1,110 @@ +/* + * + * 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.proto; + +import com.google.protobuf.ByteString; +import io.netty.channel.Channel; +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.proto.BookkeeperProtocol.GetListOfEntriesOfLedgerRequest; +import org.apache.bookkeeper.proto.BookkeeperProtocol.GetListOfEntriesOfLedgerResponse; +import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; +import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; +import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; +import org.apache.bookkeeper.util.MathUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A processor class for v3 entries of a ledger packets. + */ +public class GetListOfEntriesOfLedgerProcessorV3 extends PacketProcessorBaseV3 implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(GetListOfEntriesOfLedgerProcessorV3.class); + protected final GetListOfEntriesOfLedgerRequest getListOfEntriesOfLedgerRequest; + protected final long ledgerId; + + public GetListOfEntriesOfLedgerProcessorV3(Request request, Channel channel, + BookieRequestProcessor requestProcessor) { + super(request, channel, requestProcessor); + this.getListOfEntriesOfLedgerRequest = request.getGetListOfEntriesOfLedgerRequest(); + this.ledgerId = getListOfEntriesOfLedgerRequest.getLedgerId(); + } + + private GetListOfEntriesOfLedgerResponse getListOfEntriesOfLedgerResponse() { + long startTimeNanos = MathUtils.nowInNano(); + + GetListOfEntriesOfLedgerResponse.Builder getListOfEntriesOfLedgerResponse = GetListOfEntriesOfLedgerResponse + .newBuilder(); + getListOfEntriesOfLedgerResponse.setLedgerId(ledgerId); + + if (!isVersionCompatible()) { + getListOfEntriesOfLedgerResponse.setStatus(StatusCode.EBADVERSION); + requestProcessor.getRequestStats().getGetListOfEntriesOfLedgerStats() + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + return getListOfEntriesOfLedgerResponse.build(); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Received new getListOfEntriesOfLedger request: {}", request); + } + StatusCode status = StatusCode.EOK; + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = null; + try { + availabilityOfEntriesOfLedger = new AvailabilityOfEntriesOfLedger( + requestProcessor.bookie.getListOfEntriesOfLedger(ledgerId)); + getListOfEntriesOfLedgerResponse.setAvailabilityOfEntriesOfLedger( + ByteString.copyFrom(availabilityOfEntriesOfLedger.serializeStateOfEntriesOfLedger())); + + } catch (Bookie.NoLedgerException e) { + status = StatusCode.ENOLEDGER; + LOG.error("No ledger found while performing getListOfEntriesOfLedger from ledger: {}", ledgerId, e); + } catch (IOException e) { + status = StatusCode.EIO; + LOG.error("IOException while performing getListOfEntriesOfLedger from ledger: {}", ledgerId); + } + + if (status == StatusCode.EOK) { + requestProcessor.getRequestStats().getListOfEntriesOfLedgerStats + .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + } else { + requestProcessor.getRequestStats().getListOfEntriesOfLedgerStats + .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); + } + // Finally set the status and return + getListOfEntriesOfLedgerResponse.setStatus(status); + return getListOfEntriesOfLedgerResponse.build(); + } + + @Override + public void safeRun() { + GetListOfEntriesOfLedgerResponse listOfEntriesOfLedgerResponse = getListOfEntriesOfLedgerResponse(); + Response.Builder response = Response.newBuilder().setHeader(getHeader()) + .setStatus(listOfEntriesOfLedgerResponse.getStatus()) + .setGetListOfEntriesOfLedgerResponse(listOfEntriesOfLedgerResponse); + Response resp = response.build(); + sendResponse(listOfEntriesOfLedgerResponse.getStatus(), resp, + requestProcessor.getRequestStats().getListOfEntriesOfLedgerRequestStats); + } +} 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 1fdb403c70b..2f1659f25de 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 @@ -96,6 +96,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetListOfEntriesOfLedgerCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallbackCtx; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadLacCallback; @@ -109,6 +110,8 @@ import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerResponse; import org.apache.bookkeeper.proto.BookkeeperProtocol.GetBookieInfoRequest; import org.apache.bookkeeper.proto.BookkeeperProtocol.GetBookieInfoResponse; +import org.apache.bookkeeper.proto.BookkeeperProtocol.GetListOfEntriesOfLedgerRequest; +import org.apache.bookkeeper.proto.BookkeeperProtocol.GetListOfEntriesOfLedgerResponse; import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadLacRequest; @@ -128,6 +131,7 @@ import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.tls.SecurityHandlerFactory; import org.apache.bookkeeper.tls.SecurityHandlerFactory.NodeType; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.SafeRunnable; @@ -257,6 +261,8 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter { help = "channel stats of connect requests" ) private final OpStatsLogger connectTimer; + private final OpStatsLogger getListOfEntriesOfLedgerCompletionOpLogger; + private final OpStatsLogger getListOfEntriesOfLedgerCompletionTimeoutOpLogger; @StatsDoc( name = BookKeeperClientStats.NETTY_EXCEPTION_CNT, help = "the number of exceptions received from this channel" @@ -395,6 +401,8 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor forceLedgerOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_FORCE_OP); readLacOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_READ_LAC_OP); getBookieInfoOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.GET_BOOKIE_INFO_OP); + getListOfEntriesOfLedgerCompletionOpLogger = statsLogger + .getOpStatsLogger(BookKeeperClientStats.GET_LIST_OF_ENTRIES_OF_LEDGER_OP); readTimeoutOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_TIMEOUT_READ); addTimeoutOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_TIMEOUT_ADD); writeLacTimeoutOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_TIMEOUT_WRITE_LAC); @@ -403,6 +411,8 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor getBookieInfoTimeoutOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.TIMEOUT_GET_BOOKIE_INFO); startTLSOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_START_TLS_OP); startTLSTimeoutOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_TIMEOUT_START_TLS_OP); + getListOfEntriesOfLedgerCompletionTimeoutOpLogger = statsLogger + .getOpStatsLogger(BookKeeperClientStats.TIMEOUT_GET_LIST_OF_ENTRIES_OF_LEDGER); exceptionCounter = statsLogger.getCounter(BookKeeperClientStats.NETTY_EXCEPTION_CNT); connectTimer = statsLogger.getOpStatsLogger(BookKeeperClientStats.CLIENT_CONNECT_TIMER); addEntryOutstanding = statsLogger.getCounter(BookKeeperClientStats.ADD_OP_OUTSTANDING); @@ -829,6 +839,24 @@ public void readLac(final long ledgerId, ReadLacCallback cb, Object ctx) { writeAndFlush(channel, completionKey, request); } + public void getListOfEntriesOfLedger(final long ledgerId, GetListOfEntriesOfLedgerCallback cb) { + final long txnId = getTxnId(); + final CompletionKey completionKey = new V3CompletionKey(txnId, OperationType.GET_LIST_OF_ENTRIES_OF_LEDGER); + completionObjects.put(completionKey, new GetListOfEntriesOfLedgerCompletion(completionKey, cb, ledgerId)); + + // Build the request. + BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder().setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.GET_LIST_OF_ENTRIES_OF_LEDGER).setTxnId(txnId); + + GetListOfEntriesOfLedgerRequest.Builder getListOfEntriesOfLedgerRequestBuilder = + GetListOfEntriesOfLedgerRequest.newBuilder().setLedgerId(ledgerId); + + final Request getListOfEntriesOfLedgerRequest = Request.newBuilder().setHeader(headerBuilder) + .setGetListOfEntriesOfLedgerRequest(getListOfEntriesOfLedgerRequestBuilder).build(); + + writeAndFlush(channel, completionKey, getListOfEntriesOfLedgerRequest); + } + /** * Long Poll Reads. */ @@ -1983,6 +2011,61 @@ public void handleV3Response(BookkeeperProtocol.Response response) { } } + class GetListOfEntriesOfLedgerCompletion extends CompletionValue { + final GetListOfEntriesOfLedgerCallback cb; + + public GetListOfEntriesOfLedgerCompletion(final CompletionKey key, + final GetListOfEntriesOfLedgerCallback origCallback, final long ledgerId) { + super("GetListOfEntriesOfLedger", null, ledgerId, 0L, getListOfEntriesOfLedgerCompletionOpLogger, + getListOfEntriesOfLedgerCompletionTimeoutOpLogger); + this.cb = new GetListOfEntriesOfLedgerCallback() { + @Override + public void getListOfEntriesOfLedgerComplete(int rc, long ledgerId, + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger) { + logOpResult(rc); + origCallback.getListOfEntriesOfLedgerComplete(rc, ledgerId, availabilityOfEntriesOfLedger); + key.release(); + } + }; + } + + @Override + public void errorOut() { + errorOut(BKException.Code.BookieHandleNotAvailableException); + } + + @Override + public void errorOut(final int rc) { + errorOutAndRunCallback(() -> cb.getListOfEntriesOfLedgerComplete(rc, ledgerId, null)); + } + + @Override + public void handleV3Response(BookkeeperProtocol.Response response) { + GetListOfEntriesOfLedgerResponse getListOfEntriesOfLedgerResponse = response + .getGetListOfEntriesOfLedgerResponse(); + ByteBuf availabilityOfEntriesOfLedgerBuffer = Unpooled.EMPTY_BUFFER; + StatusCode status = response.getStatus() == StatusCode.EOK ? getListOfEntriesOfLedgerResponse.getStatus() + : response.getStatus(); + + if (getListOfEntriesOfLedgerResponse.hasAvailabilityOfEntriesOfLedger()) { + availabilityOfEntriesOfLedgerBuffer = Unpooled.wrappedBuffer( + getListOfEntriesOfLedgerResponse.getAvailabilityOfEntriesOfLedger().asReadOnlyByteBuffer()); + } + + if (LOG.isDebugEnabled()) { + logResponse(status, "ledgerId", ledgerId); + } + + int rc = convertStatus(status, BKException.Code.ReadException); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = null; + if (rc == BKException.Code.OK) { + availabilityOfEntriesOfLedger = new AvailabilityOfEntriesOfLedger( + availabilityOfEntriesOfLedgerBuffer.slice()); + } + cb.getListOfEntriesOfLedgerComplete(rc, ledgerId, availabilityOfEntriesOfLedger); + } + } + private final Recycler addCompletionRecycler = new Recycler() { @Override protected AddCompletion newObject(Recycler.Handle handle) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestStats.java index 1799e660d77..fd91957b233 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestStats.java @@ -30,6 +30,8 @@ import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FORCE_LEDGER_REQUEST; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.GET_BOOKIE_INFO; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.GET_BOOKIE_INFO_REQUEST; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.GET_LIST_OF_ENTRIES_OF_LEDGER; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.GET_LIST_OF_ENTRIES_OF_LEDGER_REQUEST; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_BLOCKED; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_ENTRY_BLOCKED_WAIT; @@ -210,6 +212,17 @@ public class RequestStats { help = "operation stats of ReadEntry blocked on a bookie" ) final OpStatsLogger readEntryBlockedStats; + @StatsDoc( + name = GET_LIST_OF_ENTRIES_OF_LEDGER_REQUEST, + help = "request stats of GetListOfEntriesOfLedger on a bookie" + ) + final OpStatsLogger getListOfEntriesOfLedgerRequestStats; + @StatsDoc( + name = "GET_LIST_OF_ENTRIES_OF_LEDGER", + help = "operation stats of GetListOfEntriesOfLedger", + parent = GET_LIST_OF_ENTRIES_OF_LEDGER_REQUEST + ) + final OpStatsLogger getListOfEntriesOfLedgerStats; public RequestStats(StatsLogger statsLogger) { this.addEntryStats = statsLogger.getOpStatsLogger(ADD_ENTRY); @@ -238,6 +251,10 @@ public RequestStats(StatsLogger statsLogger) { this.addEntryBlockedStats = statsLogger.getOpStatsLogger(ADD_ENTRY_BLOCKED_WAIT); this.readEntryBlockedStats = statsLogger.getOpStatsLogger(READ_ENTRY_BLOCKED_WAIT); + this.getListOfEntriesOfLedgerStats = statsLogger.getOpStatsLogger(GET_LIST_OF_ENTRIES_OF_LEDGER); + this.getListOfEntriesOfLedgerRequestStats = + statsLogger.getOpStatsLogger(GET_LIST_OF_ENTRIES_OF_LEDGER_REQUEST); + statsLogger.registerGauge(ADD_ENTRY_IN_PROGRESS, new Gauge() { @Override public Number getDefaultValue() { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java new file mode 100644 index 00000000000..6042bdb4385 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java @@ -0,0 +1,377 @@ +/** + * 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.util; + +import io.netty.buffer.ByteBuf; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Map.Entry; +import java.util.PrimitiveIterator; +import java.util.TreeMap; + +import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.commons.lang3.mutable.MutableLong; +import org.apache.commons.lang3.mutable.MutableObject; + +/** + * Ordered collection of SequenceGroups will represent entries of the ledger + * residing in a bookie. + * + *

            In the byte array representation of AvailabilityOfEntriesOfLedger, for the + * sake of future extensibility it would be helpful to have reserved space for + * header at the beginning. So the first 64 bytes will be used for header, with + * the first four bytes specifying the int version number, next four bytes + * specifying the number of sequencegroups for now and the rest of the bytes in + * the reserved space will be 0's. The encoded format will be represented after + * the first 64 bytes. The ordered collection of SequenceGroups will be appended + * sequentially to this byte array, with each SequenceGroup taking 24 bytes. + */ +public class AvailabilityOfEntriesOfLedger { + public static final long INVALID_ENTRYID = -1; + + /* + * + * Nomenclature: + * + * - Continuous entries are grouped as a ’Sequence’. - Number of continuous + * entries in a ‘Sequence’ is called ‘sequenceSize’. - Gap between + * Consecutive sequences is called ‘sequencePeriod’. - Consecutive sequences + * with same sequenceSize and same sequencePeriod in between consecutive + * sequences are grouped as a SequenceGroup. - ‘firstSequenceStart’ is the + * first entry in the first sequence of the SequenceGroup. - + * ‘lastSequenceStart’ is the first entry in the last sequence of the + * SequenceGroup. + * + * To represent a SequenceGroup, two long values and two int values are + * needed, so each SequenceGroup can be represented with (2 * 8 + 2 * 4 = 24 + * bytes). + */ + private static class SequenceGroup { + private static final int SEQUENCEGROUP_BYTES = 2 * Long.BYTES + 2 * Integer.BYTES; + private final long firstSequenceStart; + private final int sequenceSize; + private long lastSequenceStart = INVALID_ENTRYID; + private int sequencePeriod; + private boolean isSequenceGroupClosed = false; + private long numOfEntriesInSequenceGroup = 0; + + private SequenceGroup(long firstSequenceStart, int sequenceSize) { + this.firstSequenceStart = firstSequenceStart; + this.lastSequenceStart = firstSequenceStart; + this.sequenceSize = sequenceSize; + this.sequencePeriod = 0; + } + + private SequenceGroup(byte[] serializedSequenceGroup) { + ByteBuffer buffer = ByteBuffer.wrap(serializedSequenceGroup); + firstSequenceStart = buffer.getLong(); + lastSequenceStart = buffer.getLong(); + sequenceSize = buffer.getInt(); + sequencePeriod = buffer.getInt(); + setSequenceGroupClosed(); + } + + private boolean isSequenceGroupClosed() { + return isSequenceGroupClosed; + } + + private void setSequenceGroupClosed() { + this.isSequenceGroupClosed = true; + numOfEntriesInSequenceGroup = (lastSequenceStart - firstSequenceStart) == 0 ? sequenceSize + : (((lastSequenceStart - firstSequenceStart) / sequencePeriod) + 1) * sequenceSize; + } + + private long getNumOfEntriesInSequenceGroup() { + if (!isSequenceGroupClosed()) { + throw new IllegalStateException( + "SequenceGroup is not yet closed, it is illegal to call getNumOfEntriesInSequenceGroup"); + } + return numOfEntriesInSequenceGroup; + } + + private long getLastSequenceStart() { + return lastSequenceStart; + } + + private void setLastSequenceStart(long lastSequenceStart) { + this.lastSequenceStart = lastSequenceStart; + } + + private int getSequencePeriod() { + return sequencePeriod; + } + + private void setSequencePeriod(int sequencePeriod) { + this.sequencePeriod = sequencePeriod; + } + + private long getFirstSequenceStart() { + return firstSequenceStart; + } + + private void serializeSequenceGroup(byte[] byteArrayForSerialization) { + if (!isSequenceGroupClosed()) { + throw new IllegalStateException( + "SequenceGroup is not yet closed, it is illegal to call serializeSequenceGroup"); + } + ByteBuffer buffer = ByteBuffer.wrap(byteArrayForSerialization); + buffer.putLong(firstSequenceStart); + buffer.putLong(lastSequenceStart); + buffer.putInt(sequenceSize); + buffer.putInt(sequencePeriod); + } + + private boolean isEntryAvailable(long entryId) { + if (!isSequenceGroupClosed()) { + throw new IllegalStateException( + "SequenceGroup is not yet closed, it is illegal to call isEntryAvailable"); + } + + if ((entryId >= firstSequenceStart) && (entryId <= (lastSequenceStart + sequenceSize))) { + if (sequencePeriod == 0) { + return ((entryId - firstSequenceStart) < sequenceSize); + } else { + return (((entryId - firstSequenceStart) % sequencePeriod) < sequenceSize); + } + } else { + return false; + } + } + } + + public static final int HEADER_SIZE = 64; + public static final int V0 = 0; + // current version of AvailabilityOfEntriesOfLedger header is V0 + public static final int CURRENT_HEADER_VERSION = V0; + private final TreeMap sortedSequenceGroups = new TreeMap(); + private MutableObject curSequenceGroup = new MutableObject(null); + private MutableLong curSequenceStartEntryId = new MutableLong(INVALID_ENTRYID); + private MutableInt curSequenceSize = new MutableInt(0); + private boolean availabilityOfEntriesOfLedgerClosed = false; + private long totalNumOfAvailableEntries = 0; + + public AvailabilityOfEntriesOfLedger(PrimitiveIterator.OfLong entriesOfLedgerItr) { + while (entriesOfLedgerItr.hasNext()) { + this.addEntryToAvailabileEntriesOfLedger(entriesOfLedgerItr.nextLong()); + } + this.closeStateOfEntriesOfALedger(); + } + + public AvailabilityOfEntriesOfLedger(byte[] serializeStateOfEntriesOfLedger) { + byte[] header = new byte[HEADER_SIZE]; + byte[] serializedSequenceGroupByteArray = new byte[SequenceGroup.SEQUENCEGROUP_BYTES]; + System.arraycopy(serializeStateOfEntriesOfLedger, 0, header, 0, HEADER_SIZE); + + ByteBuffer headerByteBuf = ByteBuffer.wrap(header); + int headerVersion = headerByteBuf.getInt(); + if (headerVersion > CURRENT_HEADER_VERSION) { + throw new IllegalArgumentException("Unsupported Header Version: " + headerVersion); + } + int numOfSequenceGroups = headerByteBuf.getInt(); + SequenceGroup newSequenceGroup; + for (int i = 0; i < numOfSequenceGroups; i++) { + Arrays.fill(serializedSequenceGroupByteArray, (byte) 0); + System.arraycopy(serializeStateOfEntriesOfLedger, HEADER_SIZE + (i * SequenceGroup.SEQUENCEGROUP_BYTES), + serializedSequenceGroupByteArray, 0, SequenceGroup.SEQUENCEGROUP_BYTES); + newSequenceGroup = new SequenceGroup(serializedSequenceGroupByteArray); + sortedSequenceGroups.put(newSequenceGroup.getFirstSequenceStart(), newSequenceGroup); + } + setAvailabilityOfEntriesOfLedgerClosed(); + } + + public AvailabilityOfEntriesOfLedger(ByteBuf byteBuf) { + byte[] header = new byte[HEADER_SIZE]; + byte[] serializedSequenceGroupByteArray = new byte[SequenceGroup.SEQUENCEGROUP_BYTES]; + int readerIndex = byteBuf.readerIndex(); + byteBuf.getBytes(readerIndex, header, 0, HEADER_SIZE); + + ByteBuffer headerByteBuf = ByteBuffer.wrap(header); + int headerVersion = headerByteBuf.getInt(); + if (headerVersion > CURRENT_HEADER_VERSION) { + throw new IllegalArgumentException("Unsupported Header Version: " + headerVersion); + } + int numOfSequenceGroups = headerByteBuf.getInt(); + SequenceGroup newSequenceGroup; + for (int i = 0; i < numOfSequenceGroups; i++) { + Arrays.fill(serializedSequenceGroupByteArray, (byte) 0); + byteBuf.getBytes(readerIndex + HEADER_SIZE + (i * SequenceGroup.SEQUENCEGROUP_BYTES), + serializedSequenceGroupByteArray, 0, SequenceGroup.SEQUENCEGROUP_BYTES); + newSequenceGroup = new SequenceGroup(serializedSequenceGroupByteArray); + sortedSequenceGroups.put(newSequenceGroup.getFirstSequenceStart(), newSequenceGroup); + } + setAvailabilityOfEntriesOfLedgerClosed(); + } + + private void initializeCurSequence(long curSequenceStartEntryIdValue) { + curSequenceStartEntryId.setValue(curSequenceStartEntryIdValue); + curSequenceSize.setValue(1); + } + + private void resetCurSequence() { + curSequenceStartEntryId.setValue(INVALID_ENTRYID); + curSequenceSize.setValue(0); + } + + private boolean isCurSequenceInitialized() { + return curSequenceStartEntryId.longValue() != INVALID_ENTRYID; + } + + private boolean isEntryExistingInCurSequence(long entryId) { + return (curSequenceStartEntryId.longValue() <= entryId) + && (entryId < (curSequenceStartEntryId.longValue() + curSequenceSize.intValue())); + } + + private boolean isEntryAppendableToCurSequence(long entryId) { + return ((curSequenceStartEntryId.longValue() + curSequenceSize.intValue()) == entryId); + } + + private void incrementCurSequenceSize() { + curSequenceSize.increment(); + } + + private void createNewSequenceGroupWithCurSequence() { + SequenceGroup curSequenceGroupValue = curSequenceGroup.getValue(); + curSequenceGroupValue.setSequenceGroupClosed(); + sortedSequenceGroups.put(curSequenceGroupValue.getFirstSequenceStart(), curSequenceGroupValue); + curSequenceGroup.setValue(new SequenceGroup(curSequenceStartEntryId.longValue(), curSequenceSize.intValue())); + } + + private boolean isCurSequenceGroupInitialized() { + return curSequenceGroup.getValue() != null; + } + + private void initializeCurSequenceGroupWithCurSequence() { + curSequenceGroup.setValue(new SequenceGroup(curSequenceStartEntryId.longValue(), curSequenceSize.intValue())); + } + + private boolean doesCurSequenceBelongToCurSequenceGroup() { + long curSequenceStartEntryIdValue = curSequenceStartEntryId.longValue(); + int curSequenceSizeValue = curSequenceSize.intValue(); + boolean belongsToThisSequenceGroup = false; + SequenceGroup curSequenceGroupValue = curSequenceGroup.getValue(); + if ((curSequenceGroupValue.sequenceSize == curSequenceSizeValue) + && ((curSequenceGroupValue.getLastSequenceStart() == INVALID_ENTRYID) || ((curSequenceStartEntryIdValue + - curSequenceGroupValue.getLastSequenceStart()) == curSequenceGroupValue + .getSequencePeriod()))) { + belongsToThisSequenceGroup = true; + } + return belongsToThisSequenceGroup; + } + + private void appendCurSequenceToCurSequenceGroup() { + SequenceGroup curSequenceGroupValue = curSequenceGroup.getValue(); + curSequenceGroupValue.setLastSequenceStart(curSequenceStartEntryId.longValue()); + if (curSequenceGroupValue.getSequencePeriod() == 0) { + curSequenceGroupValue.setSequencePeriod( + ((int) (curSequenceGroupValue.getLastSequenceStart() - curSequenceGroupValue.firstSequenceStart))); + } + } + + private void addCurSequenceToSequenceGroup() { + if (!isCurSequenceGroupInitialized()) { + initializeCurSequenceGroupWithCurSequence(); + } else if (doesCurSequenceBelongToCurSequenceGroup()) { + appendCurSequenceToCurSequenceGroup(); + } else { + createNewSequenceGroupWithCurSequence(); + } + } + + private void addEntryToAvailabileEntriesOfLedger(long entryId) { + if (!isCurSequenceInitialized()) { + initializeCurSequence(entryId); + } else if (isEntryExistingInCurSequence(entryId)) { + /* this entry is already added so do nothing */ + } else if (isEntryAppendableToCurSequence(entryId)) { + incrementCurSequenceSize(); + } else { + addCurSequenceToSequenceGroup(); + initializeCurSequence(entryId); + } + } + + private void closeStateOfEntriesOfALedger() { + if (isCurSequenceInitialized()) { + addCurSequenceToSequenceGroup(); + resetCurSequence(); + } + SequenceGroup curSequenceGroupValue = curSequenceGroup.getValue(); + if (curSequenceGroupValue != null) { + curSequenceGroupValue.setSequenceGroupClosed(); + sortedSequenceGroups.put(curSequenceGroupValue.getFirstSequenceStart(), curSequenceGroupValue); + } + setAvailabilityOfEntriesOfLedgerClosed(); + } + + private boolean isAvailabilityOfEntriesOfLedgerClosed() { + return availabilityOfEntriesOfLedgerClosed; + } + + private void setAvailabilityOfEntriesOfLedgerClosed() { + this.availabilityOfEntriesOfLedgerClosed = true; + for (Entry seqGroupEntry : sortedSequenceGroups.entrySet()) { + totalNumOfAvailableEntries += seqGroupEntry.getValue().getNumOfEntriesInSequenceGroup(); + } + } + + public byte[] serializeStateOfEntriesOfLedger() { + if (!isAvailabilityOfEntriesOfLedgerClosed()) { + throw new IllegalStateException("AvailabilityOfEntriesOfLedger is not yet closed," + + "it is illegal to call serializeStateOfEntriesOfLedger"); + } + byte[] header = new byte[HEADER_SIZE]; + ByteBuffer headerByteBuf = ByteBuffer.wrap(header); + byte[] serializedSequenceGroupByteArray = new byte[SequenceGroup.SEQUENCEGROUP_BYTES]; + byte[] serializedStateByteArray = new byte[HEADER_SIZE + + (sortedSequenceGroups.size() * SequenceGroup.SEQUENCEGROUP_BYTES)]; + final int numOfSequenceGroups = sortedSequenceGroups.size(); + headerByteBuf.putInt(CURRENT_HEADER_VERSION); + headerByteBuf.putInt(numOfSequenceGroups); + System.arraycopy(header, 0, serializedStateByteArray, 0, HEADER_SIZE); + int seqNum = 0; + for (Entry seqGroupEntry : sortedSequenceGroups.entrySet()) { + SequenceGroup seqGroup = seqGroupEntry.getValue(); + Arrays.fill(serializedSequenceGroupByteArray, (byte) 0); + seqGroup.serializeSequenceGroup(serializedSequenceGroupByteArray); + System.arraycopy(serializedSequenceGroupByteArray, 0, serializedStateByteArray, + HEADER_SIZE + ((seqNum++) * SequenceGroup.SEQUENCEGROUP_BYTES), SequenceGroup.SEQUENCEGROUP_BYTES); + } + return serializedStateByteArray; + } + + public boolean isEntryAvailable(long entryId) { + if (!isAvailabilityOfEntriesOfLedgerClosed()) { + throw new IllegalStateException( + "AvailabilityOfEntriesOfLedger is not yet closed, it is illegal to call isEntryAvailable"); + } + Entry seqGroup = sortedSequenceGroups.floorEntry(entryId); + if (seqGroup == null) { + return false; + } + return seqGroup.getValue().isEntryAvailable(entryId); + } + + public long getTotalNumOfAvailableEntries() { + if (!isAvailabilityOfEntriesOfLedgerClosed()) { + throw new IllegalStateException("AvailabilityOfEntriesOfLedger is not yet closed," + + " it is illegal to call getTotalNumOfAvailableEntries"); + } + return totalNumOfAvailableEntries; + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/IteratorUtility.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/IteratorUtility.java new file mode 100644 index 00000000000..701d31a6246 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/IteratorUtility.java @@ -0,0 +1,171 @@ +/** + * Copyright The Apache Software Foundation + * + * 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.util; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.PrimitiveIterator; +import java.util.PrimitiveIterator.OfLong; +import java.util.function.ToLongFunction; + +/** + * Utility class to merge iterators. + */ +public class IteratorUtility { + + private static final long INVALID_ELEMENT = -1; + + /** + * Merges two long primitive sorted iterators and returns merged iterator. + * It expects + * - input iterators to be sorted + * - input iterators to be non-repetitive for merged iterator to be non-repetitive + * It removes duplicates from the input iterators. + * + * @param iter1 + * first primitive oflong input iterator + * @param iter2 + * second primitive oflong input iterator + * @return merged primitive oflong iterator. + */ + public static OfLong mergePrimitiveLongIterator(OfLong iter1, OfLong iter2) { + return new PrimitiveIterator.OfLong() { + private long curIter1Element = INVALID_ELEMENT; + private long curIter2Element = INVALID_ELEMENT; + private boolean hasToPreFetch = true; + + @Override + public boolean hasNext() { + if (hasToPreFetch) { + if (curIter1Element == INVALID_ELEMENT) { + curIter1Element = iter1.hasNext() ? iter1.nextLong() : INVALID_ELEMENT; + } + if (curIter2Element == INVALID_ELEMENT) { + curIter2Element = iter2.hasNext() ? iter2.nextLong() : INVALID_ELEMENT; + } + } + hasToPreFetch = false; + return (curIter1Element != INVALID_ELEMENT || curIter2Element != INVALID_ELEMENT); + } + + @Override + public long nextLong() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + long returnEntryId = INVALID_ELEMENT; + if (curIter1Element != INVALID_ELEMENT && curIter2Element != INVALID_ELEMENT) { + if (curIter1Element == curIter2Element) { + returnEntryId = curIter1Element; + curIter1Element = INVALID_ELEMENT; + curIter2Element = INVALID_ELEMENT; + } else if (curIter1Element < curIter2Element) { + returnEntryId = curIter1Element; + curIter1Element = INVALID_ELEMENT; + } else { + returnEntryId = curIter2Element; + curIter2Element = INVALID_ELEMENT; + } + } else if (curIter1Element != INVALID_ELEMENT) { + returnEntryId = curIter1Element; + curIter1Element = INVALID_ELEMENT; + } else { + returnEntryId = curIter2Element; + curIter2Element = INVALID_ELEMENT; + } + hasToPreFetch = true; + return returnEntryId; + } + }; + } + + /** + * Merges two sorted iterators and returns merged iterator sorted using + * comparator. It uses 'function' to convert T type to long, to return long + * iterator. + * It expects + * - input iterators to be sorted + * - input iterators to be non-repetitive for merged iterator to be non-repetitive + * It removes duplicates from the input iterators. + * + * @param iter1 + * first iterator of type T + * @param iter2 + * second iterator of type T + * @param comparator + * @param function + * @return + */ + public static OfLong mergeIteratorsForPrimitiveLongIterator(Iterator iter1, Iterator iter2, + Comparator comparator, ToLongFunction function) { + return new PrimitiveIterator.OfLong() { + private T curIter1Entry = null; + private T curIter2Entry = null; + private boolean hasToPreFetch = true; + + @Override + public boolean hasNext() { + if (hasToPreFetch) { + if (curIter1Entry == null) { + curIter1Entry = iter1.hasNext() ? iter1.next() : null; + } + if (curIter2Entry == null) { + curIter2Entry = iter2.hasNext() ? iter2.next() : null; + } + } + hasToPreFetch = false; + return (curIter1Entry != null || curIter2Entry != null); + } + + @Override + public long nextLong() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + T returnEntry = null; + if (curIter1Entry != null && curIter2Entry != null) { + int compareValue = comparator.compare(curIter1Entry, curIter2Entry); + if (compareValue == 0) { + returnEntry = curIter1Entry; + curIter1Entry = null; + curIter2Entry = null; + } else if (compareValue < 0) { + returnEntry = curIter1Entry; + curIter1Entry = null; + } else { + returnEntry = curIter2Entry; + curIter2Entry = null; + } + } else if (curIter1Entry != null) { + returnEntry = curIter1Entry; + curIter1Entry = null; + } else { + returnEntry = curIter2Entry; + curIter2Entry = null; + } + hasToPreFetch = true; + return function.applyAsLong(returnEntry); + } + }; + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java index c12ed9126e5..641617305da 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java @@ -23,6 +23,8 @@ import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.STORAGE_SCRUB_PAGE_RETRIES; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -34,11 +36,15 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; +import java.util.PrimitiveIterator.OfLong; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.IntStream; + import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; @@ -135,7 +141,9 @@ void checkEntry(long ledgerId, long entryId, long location) throws EntryLookupEx TestableEntryLogger entryLogger; InterleavedLedgerStorage interleavedStorage = new InterleavedLedgerStorage(); final long numWrites = 2000; + final long moreNumOfWrites = 3000; final long entriesPerWrite = 2; + final long numOfLedgers = 5; @Before public void setUp() throws Exception { @@ -158,7 +166,7 @@ public void setUp() throws Exception { // Insert some ledger & entries in the interleaved storage for (long entryId = 0; entryId < numWrites; entryId++) { - for (long ledgerId = 0; ledgerId < 5; ledgerId++) { + for (long ledgerId = 0; ledgerId < numOfLedgers; ledgerId++) { if (entryId == 0) { interleavedStorage.setMasterKey(ledgerId, ("ledger-" + ledgerId).getBytes()); interleavedStorage.setFenced(ledgerId); @@ -191,6 +199,52 @@ public void testIndexEntryIterator() throws Exception { } } + @Test + public void testGetListOfEntriesOfLedger() throws IOException { + for (long ledgerId = 0; ledgerId < numOfLedgers; ledgerId++) { + OfLong entriesOfLedger = interleavedStorage.getListOfEntriesOfLedger(ledgerId); + ArrayList arrayList = new ArrayList(); + Consumer addMethod = arrayList::add; + entriesOfLedger.forEachRemaining(addMethod); + assertEquals("Number of entries", numWrites, arrayList.size()); + assertTrue("Entries of Ledger", IntStream.range(0, arrayList.size()).allMatch(i -> { + return arrayList.get(i).longValue() == (i * entriesPerWrite); + })); + } + + long nonExistingLedger = 456789L; + OfLong entriesOfLedger = interleavedStorage.getListOfEntriesOfLedger(nonExistingLedger); + assertFalse("There shouldn't be any entry", entriesOfLedger.hasNext()); + } + + @Test + public void testGetListOfEntriesOfLedgerAfterFlush() throws IOException { + interleavedStorage.flush(); + + // Insert some more ledger & entries in the interleaved storage + for (long entryId = numWrites; entryId < moreNumOfWrites; entryId++) { + for (long ledgerId = 0; ledgerId < numOfLedgers; ledgerId++) { + ByteBuf entry = Unpooled.buffer(128); + entry.writeLong(ledgerId); + entry.writeLong(entryId * entriesPerWrite); + entry.writeBytes(("entry-" + entryId).getBytes()); + + interleavedStorage.addEntry(entry); + } + } + + for (long ledgerId = 0; ledgerId < numOfLedgers; ledgerId++) { + OfLong entriesOfLedger = interleavedStorage.getListOfEntriesOfLedger(ledgerId); + ArrayList arrayList = new ArrayList(); + Consumer addMethod = arrayList::add; + entriesOfLedger.forEachRemaining(addMethod); + assertEquals("Number of entries", moreNumOfWrites, arrayList.size()); + assertTrue("Entries of Ledger", IntStream.range(0, arrayList.size()).allMatch(i -> { + return arrayList.get(i).longValue() == (i * entriesPerWrite); + })); + } + } + @Test public void testConsistencyCheckConcurrentGC() throws Exception { final long signalDone = -1; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java index d028f70a644..76244b79fff 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageTest.java @@ -28,7 +28,11 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.util.ArrayList; +import java.util.PrimitiveIterator.OfLong; import java.util.concurrent.CountDownLatch; +import java.util.function.Consumer; +import java.util.stream.IntStream; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; @@ -37,6 +41,7 @@ import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.util.TestUtils; +import org.junit.Assert; import org.junit.Test; /** @@ -277,4 +282,44 @@ ReadOnlyFileInfo getFileInfo(long ledgerId, File[] indexDirectories) throws IOEx fi.readHeader(); return fi; } + + @Test + public void testGetListOfEntriesOfLedger() throws Exception { + ClientConfiguration conf = new ClientConfiguration(); + conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + int numOfBookies = bs.size(); + int numOfEntries = 5; + BookKeeper.DigestType digestType = BookKeeper.DigestType.CRC32; + BookKeeper bkc = new BookKeeper(conf); + LedgerHandle lh = bkc.createLedger(numOfBookies, numOfBookies, digestType, "testPasswd".getBytes()); + long lId = lh.getId(); + for (int i = 0; i < numOfEntries; i++) { + lh.addEntry("000".getBytes()); + } + + ServerConfiguration newBookieConf = new ServerConfiguration(bsConfs.get(0)); + /* + * by reusing bookieServerConfig and setting metadataServiceUri to null + * we can create/start new Bookie instance using the same data + * (journal/ledger/index) of the existing BookeieServer for our testing + * purpose. + */ + newBookieConf.setMetadataServiceUri(null); + Bookie newbookie = new Bookie(newBookieConf); + /* + * since 'newbookie' uses the same data as original Bookie, it should be + * able to read journal of the original bookie. + */ + newbookie.readJournal(); + + OfLong listOfEntriesItr = newbookie.getListOfEntriesOfLedger(lId); + ArrayList arrayList = new ArrayList(); + Consumer addMethod = arrayList::add; + listOfEntriesItr.forEachRemaining(addMethod); + + assertEquals("Num Of Entries", numOfEntries, arrayList.size()); + Assert.assertTrue("Iterator should be sorted", + IntStream.range(0, arrayList.size() - 1).allMatch(k -> arrayList.get(k) <= arrayList.get(k + 1))); + bkc.close(); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageTest.java new file mode 100644 index 00000000000..01383e232b3 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageTest.java @@ -0,0 +1,194 @@ +/** + * + * 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 org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.PrimitiveIterator.OfLong; +import java.util.function.Consumer; +import java.util.stream.IntStream; + +import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.util.DiskChecker; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * Testing SortedLedgerStorage. + */ +@RunWith(Parameterized.class) +public class SortedLedgerStorageTest { + + TestStatsProvider statsProvider = new TestStatsProvider(); + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + LedgerDirsManager ledgerDirsManager; + SortedLedgerStorage sortedLedgerStorage = new SortedLedgerStorage(); + + final long numWrites = 2000; + final long moreNumOfWrites = 3000; + final long entriesPerWrite = 2; + final long numOfLedgers = 5; + + @Parameterized.Parameters + public static Iterable elplSetting() { + return Arrays.asList(true, false); + } + + public SortedLedgerStorageTest(boolean elplSetting) { + conf.setEntryLogSizeLimit(2048); + conf.setEntryLogPerLedgerEnabled(elplSetting); + } + + CheckpointSource checkpointSource = new CheckpointSource() { + @Override + public Checkpoint newCheckpoint() { + return Checkpoint.MAX; + } + + @Override + public void checkpointComplete(Checkpoint checkpoint, boolean compact) throws IOException { + } + }; + + Checkpointer checkpointer = new Checkpointer() { + @Override + public void startCheckpoint(Checkpoint checkpoint) { + // No-op + } + + @Override + public void start() { + // no-op + } + }; + + @Before + public void setUp() throws Exception { + File tmpDir = File.createTempFile("bkTest", ".dir"); + tmpDir.delete(); + tmpDir.mkdir(); + File curDir = Bookie.getCurrentDirectory(tmpDir); + Bookie.checkDirectoryStructure(curDir); + + conf.setLedgerDirNames(new String[] { tmpDir.toString() }); + ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(), + new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); + sortedLedgerStorage.initialize(conf, null, ledgerDirsManager, ledgerDirsManager, null, checkpointSource, + checkpointer, statsProvider.getStatsLogger(BOOKIE_SCOPE), UnpooledByteBufAllocator.DEFAULT); + } + + @Test + public void testGetListOfEntriesOfLedger() throws Exception { + long nonExistingLedgerId = 123456L; + OfLong entriesItr = sortedLedgerStorage.getListOfEntriesOfLedger(nonExistingLedgerId); + assertFalse("There shouldn't be any entries for this ledger", entriesItr.hasNext()); + // Insert some ledger & entries in the interleaved storage + for (long entryId = 0; entryId < numWrites; entryId++) { + for (long ledgerId = 0; ledgerId < numOfLedgers; ledgerId++) { + if (entryId == 0) { + sortedLedgerStorage.setMasterKey(ledgerId, ("ledger-" + ledgerId).getBytes()); + sortedLedgerStorage.setFenced(ledgerId); + } + ByteBuf entry = Unpooled.buffer(128); + entry.writeLong(ledgerId); + entry.writeLong(entryId * entriesPerWrite); + entry.writeBytes(("entry-" + entryId).getBytes()); + + sortedLedgerStorage.addEntry(entry); + } + } + + for (long ledgerId = 0; ledgerId < numOfLedgers; ledgerId++) { + OfLong entriesOfLedger = sortedLedgerStorage.getListOfEntriesOfLedger(ledgerId); + ArrayList arrayList = new ArrayList(); + Consumer addMethod = arrayList::add; + entriesOfLedger.forEachRemaining(addMethod); + assertEquals("Number of entries", numWrites, arrayList.size()); + assertTrue("Entries of Ledger", IntStream.range(0, arrayList.size()).allMatch(i -> { + return arrayList.get(i).longValue() == (i * entriesPerWrite); + })); + } + + nonExistingLedgerId = 456789L; + entriesItr = sortedLedgerStorage.getListOfEntriesOfLedger(nonExistingLedgerId); + assertFalse("There shouldn't be any entry", entriesItr.hasNext()); + } + + @Test + public void testGetListOfEntriesOfLedgerAfterFlush() throws IOException { + // Insert some ledger & entries in the interleaved storage + for (long entryId = 0; entryId < numWrites; entryId++) { + for (long ledgerId = 0; ledgerId < numOfLedgers; ledgerId++) { + if (entryId == 0) { + sortedLedgerStorage.setMasterKey(ledgerId, ("ledger-" + ledgerId).getBytes()); + sortedLedgerStorage.setFenced(ledgerId); + } + ByteBuf entry = Unpooled.buffer(128); + entry.writeLong(ledgerId); + entry.writeLong(entryId * entriesPerWrite); + entry.writeBytes(("entry-" + entryId).getBytes()); + + sortedLedgerStorage.addEntry(entry); + } + } + + sortedLedgerStorage.flush(); + + // Insert some more ledger & entries in the interleaved storage + for (long entryId = numWrites; entryId < moreNumOfWrites; entryId++) { + for (long ledgerId = 0; ledgerId < numOfLedgers; ledgerId++) { + ByteBuf entry = Unpooled.buffer(128); + entry.writeLong(ledgerId); + entry.writeLong(entryId * entriesPerWrite); + entry.writeBytes(("entry-" + entryId).getBytes()); + + sortedLedgerStorage.addEntry(entry); + } + } + + for (long ledgerId = 0; ledgerId < numOfLedgers; ledgerId++) { + OfLong entriesOfLedger = sortedLedgerStorage.getListOfEntriesOfLedger(ledgerId); + ArrayList arrayList = new ArrayList(); + Consumer addMethod = arrayList::add; + entriesOfLedger.forEachRemaining(addMethod); + assertEquals("Number of entries", moreNumOfWrites, arrayList.size()); + assertTrue("Entries of Ledger", IntStream.range(0, arrayList.size()).allMatch(i -> { + return arrayList.get(i).longValue() == (i * entriesPerWrite); + })); + } + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryMemTable.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryMemTable.java index 68e3eeba969..9e6c5595fd2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryMemTable.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryMemTable.java @@ -18,6 +18,8 @@ */ package org.apache.bookkeeper.bookie; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -32,9 +34,14 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.PrimitiveIterator.OfLong; import java.util.Random; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.Bookie.NoLedgerException; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -50,6 +57,7 @@ /** * Test the EntryMemTable class. */ +@Slf4j @RunWith(Parameterized.class) public class TestEntryMemTable implements CacheCallback, SkipListFlusher, CheckpointSource { @@ -297,5 +305,151 @@ public int compareTo(Checkpoint o) { } } + + @Test + public void testGetListOfEntriesOfLedger() throws IOException { + Set flushedKVs = Collections.newSetFromMap(new ConcurrentHashMap()); + KVFLusher flusher = new KVFLusher(flushedKVs); + int numofEntries = 100; + int numOfLedgers = 5; + byte[] data = new byte[10]; + for (long entryId = 1; entryId <= numofEntries; entryId++) { + for (long ledgerId = 1; ledgerId <= numOfLedgers; ledgerId++) { + random.nextBytes(data); + assertTrue(ledgerId + ":" + entryId + " is duplicate in mem-table!", + memTable.addEntry(ledgerId, entryId, ByteBuffer.wrap(data), this) != 0); + } + } + for (long ledgerId = 1; ledgerId <= numOfLedgers; ledgerId++) { + OfLong entriesItr = memTable.getListOfEntriesOfLedger((random.nextInt((int) ledgerId) + 1)); + ArrayList listOfEntries = new ArrayList(); + Consumer addMethod = listOfEntries::add; + entriesItr.forEachRemaining(addMethod); + assertEquals("Number of Entries", numofEntries, listOfEntries.size()); + for (int i = 0; i < numofEntries; i++) { + assertEquals("listOfEntries should be sorted", Long.valueOf(i + 1), listOfEntries.get(i)); + } + } + assertTrue("Snapshot is expected to be empty since snapshot is not done", memTable.snapshot.isEmpty()); + assertTrue("Take snapshot and returned checkpoint should not be empty", memTable.snapshot() != null); + assertFalse("After taking snapshot, snapshot should not be empty ", memTable.snapshot.isEmpty()); + for (long ledgerId = 1; ledgerId <= numOfLedgers; ledgerId++) { + OfLong entriesItr = memTable.getListOfEntriesOfLedger((random.nextInt((int) ledgerId) + 1)); + ArrayList listOfEntries = new ArrayList(); + Consumer addMethod = listOfEntries::add; + entriesItr.forEachRemaining(addMethod); + assertEquals("Number of Entries should be the same even after taking snapshot", numofEntries, + listOfEntries.size()); + for (int i = 0; i < numofEntries; i++) { + assertEquals("listOfEntries should be sorted", Long.valueOf(i + 1), listOfEntries.get(i)); + } + } + + memTable.flush(flusher); + for (long ledgerId = 1; ledgerId <= numOfLedgers; ledgerId++) { + OfLong entriesItr = memTable.getListOfEntriesOfLedger((random.nextInt((int) ledgerId) + 1)); + assertFalse("After flushing there shouldn't be entries in memtable", entriesItr.hasNext()); + } + } + + @Test + public void testGetListOfEntriesOfLedgerFromBothKVMapAndSnapshot() throws IOException { + int numofEntries = 100; + int newNumOfEntries = 200; + int numOfLedgers = 5; + byte[] data = new byte[10]; + for (long entryId = 1; entryId <= numofEntries; entryId++) { + for (long ledgerId = 1; ledgerId <= numOfLedgers; ledgerId++) { + random.nextBytes(data); + assertTrue(ledgerId + ":" + entryId + " is duplicate in mem-table!", + memTable.addEntry(ledgerId, entryId, ByteBuffer.wrap(data), this) != 0); + } + } + + assertTrue("Snapshot is expected to be empty since snapshot is not done", memTable.snapshot.isEmpty()); + assertTrue("Take snapshot and returned checkpoint should not be empty", memTable.snapshot() != null); + assertFalse("After taking snapshot, snapshot should not be empty ", memTable.snapshot.isEmpty()); + + for (long entryId = numofEntries + 1; entryId <= newNumOfEntries; entryId++) { + for (long ledgerId = 1; ledgerId <= numOfLedgers; ledgerId++) { + random.nextBytes(data); + assertTrue(ledgerId + ":" + entryId + " is duplicate in mem-table!", + memTable.addEntry(ledgerId, entryId, ByteBuffer.wrap(data), this) != 0); + } + } + + for (long ledgerId = 1; ledgerId <= numOfLedgers; ledgerId++) { + OfLong entriesItr = memTable.getListOfEntriesOfLedger((random.nextInt((int) ledgerId) + 1)); + ArrayList listOfEntries = new ArrayList(); + Consumer addMethod = listOfEntries::add; + entriesItr.forEachRemaining(addMethod); + assertEquals("Number of Entries should be the same", newNumOfEntries, listOfEntries.size()); + for (int i = 0; i < newNumOfEntries; i++) { + assertEquals("listOfEntries should be sorted", Long.valueOf(i + 1), listOfEntries.get(i)); + } + } + } + + @Test + public void testGetListOfEntriesOfLedgerWhileAddingConcurrently() throws IOException, InterruptedException { + final int numofEntries = 100; + final int newNumOfEntries = 200; + final int concurrentAddOfEntries = 300; + long ledgerId = 5; + byte[] data = new byte[10]; + for (long entryId = 1; entryId <= numofEntries; entryId++) { + random.nextBytes(data); + assertTrue(ledgerId + ":" + entryId + " is duplicate in mem-table!", + memTable.addEntry(ledgerId, entryId, ByteBuffer.wrap(data), this) != 0); + } + + assertTrue("Snapshot is expected to be empty since snapshot is not done", memTable.snapshot.isEmpty()); + assertTrue("Take snapshot and returned checkpoint should not be empty", memTable.snapshot() != null); + assertFalse("After taking snapshot, snapshot should not be empty ", memTable.snapshot.isEmpty()); + + for (long entryId = numofEntries + 1; entryId <= newNumOfEntries; entryId++) { + random.nextBytes(data); + assertTrue(ledgerId + ":" + entryId + " is duplicate in mem-table!", + memTable.addEntry(ledgerId, entryId, ByteBuffer.wrap(data), this) != 0); + } + + AtomicBoolean successfullyAdded = new AtomicBoolean(true); + + Thread threadToAdd = new Thread(new Runnable() { + @Override + public void run() { + try { + for (long entryId = newNumOfEntries + 1; entryId <= concurrentAddOfEntries; entryId++) { + random.nextBytes(data); + boolean thisEntryAddedSuccessfully = (memTable.addEntry(ledgerId, entryId, + ByteBuffer.wrap(data), TestEntryMemTable.this) != 0); + successfullyAdded.set(successfullyAdded.get() && thisEntryAddedSuccessfully); + Thread.sleep(10); + } + } catch (IOException e) { + log.error("Got Unexpected exception while adding entries"); + successfullyAdded.set(false); + } catch (InterruptedException e) { + log.error("Got InterruptedException while waiting"); + successfullyAdded.set(false); + } + } + }); + threadToAdd.start(); + + Thread.sleep(200); + OfLong entriesItr = memTable.getListOfEntriesOfLedger(ledgerId); + ArrayList listOfEntries = new ArrayList(); + while (entriesItr.hasNext()) { + listOfEntries.add(entriesItr.next()); + Thread.sleep(5); + } + threadToAdd.join(5000); + assertTrue("Entries should be added successfully in the spawned thread", successfullyAdded.get()); + + for (int i = 0; i < newNumOfEntries; i++) { + assertEquals("listOfEntries should be sorted", Long.valueOf(i + 1), listOfEntries.get(i)); + } + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java index f57d388d9ad..22535f3d557 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java @@ -29,6 +29,7 @@ import io.netty.buffer.ByteBufAllocator; import java.io.IOException; +import java.util.PrimitiveIterator.OfLong; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -369,6 +370,11 @@ public void checkpoint(Checkpoint checkpoint) @Override public void registerLedgerDeletionListener(LedgerDeletionListener listener) { } + + @Override + public OfLong getListOfEntriesOfLedger(long ledgerId) { + return null; + } } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index 6e8cf324de8..2f419b5e6a7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -24,6 +24,7 @@ import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -33,6 +34,9 @@ import java.util.Iterator; import java.util.List; import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -43,6 +47,7 @@ import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.commons.io.FileUtils; import org.apache.zookeeper.CreateMode; @@ -399,4 +404,97 @@ void initiateNewClusterAndCreateLedgers(ServerConfiguration newConfig, List futureResult = bkAdmin + .asyncGetListOfEntriesOfLedger(bs.get(i).getLocalAddress(), nonExistingLedgerId); + try { + futureResult.get(); + fail("asyncGetListOfEntriesOfLedger is supposed to be failed with NoSuchLedgerExistsException"); + } catch (ExecutionException ee) { + assertTrue(ee.getCause() instanceof BKException); + BKException e = (BKException) ee.getCause(); + assertEquals(e.getCode(), BKException.Code.NoSuchLedgerExistsException); + } + } + } + } + + public void testGetListOfEntriesOfLedger(boolean isLedgerClosed) throws Exception { + ClientConfiguration conf = new ClientConfiguration(); + conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + int numOfEntries = 6; + BookKeeper bkc = new BookKeeper(conf); + LedgerHandle lh = bkc.createLedger(numOfBookies, numOfBookies, digestType, "testPasswd".getBytes()); + long lId = lh.getId(); + for (int i = 0; i < numOfEntries; i++) { + lh.addEntry("000".getBytes()); + } + if (isLedgerClosed) { + lh.close(); + } + try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) { + for (int i = 0; i < bs.size(); i++) { + CompletableFuture futureResult = bkAdmin + .asyncGetListOfEntriesOfLedger(bs.get(i).getLocalAddress(), lId); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = futureResult.get(); + assertEquals("Number of entries", numOfEntries, + availabilityOfEntriesOfLedger.getTotalNumOfAvailableEntries()); + for (int j = 0; j < numOfEntries; j++) { + assertTrue("Entry should be available: " + j, availabilityOfEntriesOfLedger.isEntryAvailable(j)); + } + assertFalse("Entry should not be available: " + numOfEntries, + availabilityOfEntriesOfLedger.isEntryAvailable(numOfEntries)); + } + } + bkc.close(); + } + + @Test + public void testGetListOfEntriesOfLedgerWithJustOneBookieInWriteQuorum() throws Exception { + ClientConfiguration conf = new ClientConfiguration(); + conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + int numOfEntries = 6; + BookKeeper bkc = new BookKeeper(conf); + /* + * in this testsuite there are going to be 2 (numOfBookies) and if + * writeQuorum is 1 then it will stripe entries to those two bookies. + */ + LedgerHandle lh = bkc.createLedger(2, 1, digestType, "testPasswd".getBytes()); + long lId = lh.getId(); + for (int i = 0; i < numOfEntries; i++) { + lh.addEntry("000".getBytes()); + } + + try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) { + for (int i = 0; i < bs.size(); i++) { + CompletableFuture futureResult = bkAdmin + .asyncGetListOfEntriesOfLedger(bs.get(i).getLocalAddress(), lId); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = futureResult.get(); + /* + * since num of bookies in the ensemble is 2 and + * writeQuorum/ackQuorum is 1, it will stripe to these two + * bookies and hence in each bookie there will be only + * numOfEntries/2 entries. + */ + assertEquals("Number of entries", numOfEntries / 2, + availabilityOfEntriesOfLedger.getTotalNumOfAvailableEntries()); + } + } + bkc.close(); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 10f7048d1d3..30121b9182b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -40,6 +40,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.PrimitiveIterator.OfLong; import java.util.Queue; import java.util.Random; import java.util.Set; @@ -685,5 +686,10 @@ public void cancelWaitForLastAddConfirmedUpdate(long ledgerId, Watcher watcher) throws IOException { } + + @Override + public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { + return null; + } } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java index a673d8c7dae..4acc3e7166d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java @@ -31,6 +31,7 @@ import java.util.Map; import java.util.NavigableMap; import java.util.Optional; +import java.util.PrimitiveIterator.OfLong; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.CheckpointSource; @@ -288,5 +289,10 @@ public void setExplicitlac(long ledgerId, ByteBuf lac) throws IOException { public ByteBuf getExplicitLac(long ledgerId) { return null; } + + @Override + public OfLong getListOfEntriesOfLedger(long ledgerId) { + return null; + } } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java index 2c349a01185..1cbddd3abb9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java @@ -42,6 +42,7 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.FutureGetListOfEntriesOfLedger; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadLacCallback; @@ -49,6 +50,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteLacCallback; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; import org.apache.bookkeeper.proto.checksum.DigestManager; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.ByteBufList; import org.slf4j.Logger; @@ -256,6 +258,17 @@ public void getBookieInfo(BookieSocketAddress addr, long requested, })); } + @Override + public CompletableFuture getListOfEntriesOfLedger(BookieSocketAddress address, + long ledgerId) { + FutureGetListOfEntriesOfLedger futureResult = new FutureGetListOfEntriesOfLedger(ledgerId); + executor.executeOrdered(address, safeRun(() -> { + futureResult + .completeExceptionally(BKException.create(BKException.Code.IllegalOpException).fillInStackTrace()); + })); + return futureResult; + } + @Override public boolean isClosed() { return false; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedgerTest.java new file mode 100644 index 00000000000..4368f897b68 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedgerTest.java @@ -0,0 +1,192 @@ +/** + * + * 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.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.PrimitiveIterator; +import java.util.Set; + +import org.junit.Test; + +/** + * Testsuite for AvailabilityOfEntriesOfLedger. + */ +public class AvailabilityOfEntriesOfLedgerTest { + @Test + public void testWithItrConstructor() { + long[][] arrays = { + { 0, 1, 2 }, + { 1, 2}, + { 1, 2, 3, 5, 6, 7, 8 }, + { 0, 1, 5 }, + { 3 }, + { 1, 2, 4, 5, 7, 8 }, + {}, + {0}, + { 1, 2, 3, 5, 6, 11, 12, 13, 14, 15, 16, 17, 100, 1000, 1001, 10000, 20000, 20001 } + }; + for (int i = 0; i < arrays.length; i++) { + long[] tempArray = arrays[i]; + PrimitiveIterator.OfLong primitiveIterator = Arrays.stream(tempArray).iterator(); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = new AvailabilityOfEntriesOfLedger( + primitiveIterator); + assertEquals("Expected total number of entries", tempArray.length, + availabilityOfEntriesOfLedger.getTotalNumOfAvailableEntries()); + for (int j = 0; j < tempArray.length; j++) { + assertTrue(tempArray[j] + " is supposed to be available", + availabilityOfEntriesOfLedger.isEntryAvailable(tempArray[j])); + } + } + } + + @Test + public void testWithItrConstructorWithDuplicates() { + long[][] arrays = { + { 1, 2, 2, 3 }, + { 1, 2, 3, 5, 5, 6, 7, 8, 8 }, + { 1, 1, 5, 5 }, + { 3, 3 }, + { 1, 1, 2, 4, 5, 8, 9, 9, 9, 9, 9 }, + {}, + { 1, 2, 3, 5, 6, 11, 12, 13, 14, 15, 16, 17, 17, 100, 1000, 1000, 1001, 10000, 10000, 20000, 20001 } + }; + for (int i = 0; i < arrays.length; i++) { + long[] tempArray = arrays[i]; + Set tempSet = new HashSet(); + for (int k = 0; k < tempArray.length; k++) { + tempSet.add(tempArray[k]); + } + PrimitiveIterator.OfLong primitiveIterator = Arrays.stream(tempArray).iterator(); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = new AvailabilityOfEntriesOfLedger( + primitiveIterator); + assertEquals("Expected total number of entries", tempSet.size(), + availabilityOfEntriesOfLedger.getTotalNumOfAvailableEntries()); + for (int j = 0; j < tempArray.length; j++) { + assertTrue(tempArray[j] + " is supposed to be available", + availabilityOfEntriesOfLedger.isEntryAvailable(tempArray[j])); + } + } + } + + @Test + public void testSerializeDeserialize() { + long[][] arrays = { + { 0, 1, 2 }, + { 1, 2 }, + { 1, 2, 3, 5, 6, 7, 8 }, + { 0, 1, 5 }, + { 3 }, + { 1, 2, 4, 5, 7, 8 }, + { }, + { 0 }, + { 1, 2, 3, 5, 6, 11, 12, 13, 14, 15, 16, 17, 100, 1000, 1001, 10000, 20000, 20001 } + }; + for (int i = 0; i < arrays.length; i++) { + long[] tempArray = arrays[i]; + PrimitiveIterator.OfLong primitiveIterator = Arrays.stream(tempArray).iterator(); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = new AvailabilityOfEntriesOfLedger( + primitiveIterator); + byte[] serializedState = availabilityOfEntriesOfLedger.serializeStateOfEntriesOfLedger(); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedgerUsingSer = new AvailabilityOfEntriesOfLedger( + serializedState); + assertEquals("Expected total number of entries", tempArray.length, + availabilityOfEntriesOfLedgerUsingSer.getTotalNumOfAvailableEntries()); + for (int j = 0; j < tempArray.length; j++) { + assertTrue(tempArray[j] + " is supposed to be available", + availabilityOfEntriesOfLedgerUsingSer.isEntryAvailable(tempArray[j])); + } + } + } + + @Test + public void testSerializeDeserializeWithItrConstructorWithDuplicates() { + long[][] arrays = { + { 1, 2, 2, 3 }, + { 1, 2, 3, 5, 5, 6, 7, 8, 8 }, + { 1, 1, 5, 5 }, + { 3, 3 }, + { 1, 1, 2, 4, 5, 8, 9, 9, 9, 9, 9 }, + {}, + { 1, 2, 3, 5, 6, 11, 12, 13, 14, 15, 16, 17, 17, 100, 1000, 1000, 1001, 10000, 10000, 20000, 20001 } + }; + for (int i = 0; i < arrays.length; i++) { + long[] tempArray = arrays[i]; + Set tempSet = new HashSet(); + for (int k = 0; k < tempArray.length; k++) { + tempSet.add(tempArray[k]); + } + PrimitiveIterator.OfLong primitiveIterator = Arrays.stream(tempArray).iterator(); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = new AvailabilityOfEntriesOfLedger( + primitiveIterator); + byte[] serializedState = availabilityOfEntriesOfLedger.serializeStateOfEntriesOfLedger(); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedgerUsingSer = new AvailabilityOfEntriesOfLedger( + serializedState); + assertEquals("Expected total number of entries", tempSet.size(), + availabilityOfEntriesOfLedgerUsingSer.getTotalNumOfAvailableEntries()); + for (int j = 0; j < tempArray.length; j++) { + assertTrue(tempArray[j] + " is supposed to be available", + availabilityOfEntriesOfLedgerUsingSer.isEntryAvailable(tempArray[j])); + } + } + } + + @Test + public void testNonExistingEntries() { + long[][] arrays = { + { 0, 1, 2 }, + { 1, 2, 3, 5, 6, 7, 8 }, + { 1, 5 }, + { 3 }, + { 1, 2, 4, 5, 7, 8 }, + {}, + { 1, 2, 3, 5, 6, 11, 12, 13, 14, 15, 16, 17, 100, 1000, 1001, 10000, 20000, 20001 } + }; + /** + * corresponding non-existing entries for 'arrays' + */ + long[][] nonExistingEntries = { + { 3 }, + { 0, 4, 9, 100, 101 }, + { 0, 2, 3, 6, 9 }, + { 0, 1, 2, 4, 5, 6 }, + { 0, 3, 9, 10, 11, 100, 1000 }, + { 0, 1, 2, 3, 4, 5 }, + { 4, 18, 1002, 19999, 20003 } + }; + for (int i = 0; i < arrays.length; i++) { + long[] tempArray = arrays[i]; + long[] nonExistingElementsTempArray = nonExistingEntries[i]; + PrimitiveIterator.OfLong primitiveIterator = Arrays.stream(tempArray).iterator(); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = new AvailabilityOfEntriesOfLedger( + primitiveIterator); + + for (int j = 0; j < nonExistingElementsTempArray.length; j++) { + assertFalse(nonExistingElementsTempArray[j] + " is not supposed to be available", + availabilityOfEntriesOfLedger.isEntryAvailable(nonExistingElementsTempArray[j])); + } + } + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/IteratorUtilityTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/IteratorUtilityTest.java new file mode 100644 index 00000000000..55c1db33569 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/IteratorUtilityTest.java @@ -0,0 +1,141 @@ +/** + * Copyright The Apache Software Foundation + * + * 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.util; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.PrimitiveIterator; +import java.util.function.Consumer; +import java.util.stream.IntStream; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Testsuite for IteratorUtility methods. + */ +public class IteratorUtilityTest { + + @Test + public void testWithPrimitiveItrMerge() { + long[][] arrays = { + { 0, 1, 2 }, + { 0, 1 }, + { 1, 2 }, + { 1, 2, 3, 5, 6, 7, 8 }, + { 1, 2, 3, 5, 6, 7, 8 }, + { 0, 1, 5 }, + { 3 }, + { 1, 2, 4, 5, 7, 8 }, + {}, + {}, + { 0 }, + { 1, 2, 3, 5, 6, 11, 12, 13, 14, 15, 16, 17, 100, 1000, 1001, 10000, 20000, 20001 }, + { 201, 202, 203, 205, 206, 2011, 2012, 2013, 2014, 2015, 2016, 2017, 20100, 201000, + 201001, 2010000, 2020000, 2020001 } + }; + for (int i = 0; i < arrays.length; i++) { + for (int j = i + 1; j < arrays.length; j++) { + long[] tempArray1 = arrays[i]; + long[] tempArray2 = arrays[j]; + HashSet unionSet = new HashSet(); + for (int k = 0; k < tempArray1.length; k++) { + unionSet.add(tempArray1[k]); + } + for (int k = 0; k < tempArray2.length; k++) { + unionSet.add(tempArray2[k]); + } + + PrimitiveIterator.OfLong primitiveIterator1 = Arrays.stream(tempArray1).iterator(); + PrimitiveIterator.OfLong primitiveIterator2 = Arrays.stream(tempArray2).iterator(); + + PrimitiveIterator.OfLong mergedItr = IteratorUtility.mergePrimitiveLongIterator(primitiveIterator1, + primitiveIterator2); + ArrayList mergedArrayList = new ArrayList(); + Consumer addMethod = mergedArrayList::add; + mergedItr.forEachRemaining(addMethod); + int mergedListSize = mergedArrayList.size(); + Assert.assertEquals("Size of the mergedArrayList", unionSet.size(), mergedArrayList.size()); + Assert.assertTrue("mergedArrayList should contain all elements in unionSet", + mergedArrayList.containsAll(unionSet)); + Assert.assertTrue("Merged Iterator should be sorted", IntStream.range(0, mergedListSize - 1) + .allMatch(k -> mergedArrayList.get(k) <= mergedArrayList.get(k + 1))); + Assert.assertTrue("All elements of tempArray1 should be in mergedArrayList", + IntStream.range(0, tempArray1.length).allMatch(k -> mergedArrayList.contains(tempArray1[k]))); + Assert.assertTrue("All elements of tempArray2 should be in mergedArrayList", + IntStream.range(0, tempArray2.length).allMatch(k -> mergedArrayList.contains(tempArray2[k]))); + } + } + } + + @Test + public void testWithItrMerge() { + long[][] arrays = { + { 0, 1, 2 }, + { 0, 1 }, + { 1, 2 }, + { 1, 2, 3, 5, 6, 7, 8 }, + { 1, 2, 3, 5, 6, 7, 8 }, + { 0, 1, 5 }, + { 3 }, + { 1, 2, 4, 5, 7, 8 }, + {}, + {}, + { 0 }, + { 1, 2, 3, 5, 6, 11, 12, 13, 14, 15, 16, 17, 100, 1000, 1001, 10000, 20000, 20001 }, + { 201, 202, 203, 205, 206, 2011, 2012, 2013, 2014, 2015, 2016, 2017, 20100, 201000, + 201001, 2010000, 2020000, 2020001 } + }; + for (int i = 0; i < arrays.length; i++) { + for (int j = i + 1; j < arrays.length; j++) { + long[] tempArray1 = arrays[i]; + ArrayList tempArrayList1 = new ArrayList(); + IntStream.range(0, tempArray1.length).forEach((k) -> tempArrayList1.add(tempArray1[k])); + long[] tempArray2 = arrays[j]; + ArrayList tempArrayList2 = new ArrayList(); + IntStream.range(0, tempArray2.length).forEach((k) -> tempArrayList2.add(tempArray2[k])); + HashSet unionSet = new HashSet(); + unionSet.addAll(tempArrayList1); + unionSet.addAll(tempArrayList2); + + Iterator itr1 = tempArrayList1.iterator(); + Iterator itr2 = tempArrayList2.iterator(); + + Iterator mergedItr = IteratorUtility.mergeIteratorsForPrimitiveLongIterator(itr1, itr2, + Long::compare, (l) -> l); + ArrayList mergedArrayList = new ArrayList(); + Consumer addMethod = mergedArrayList::add; + mergedItr.forEachRemaining(addMethod); + int mergedListSize = mergedArrayList.size(); + Assert.assertEquals("Size of the mergedArrayList", unionSet.size(), mergedArrayList.size()); + Assert.assertTrue("mergedArrayList should contain all elements in unionSet", + mergedArrayList.containsAll(unionSet)); + Assert.assertTrue("Merged Iterator should be sorted", IntStream.range(0, mergedListSize - 1) + .allMatch(k -> mergedArrayList.get(k) <= mergedArrayList.get(k + 1))); + Assert.assertTrue("All elements of tempArray1 should be in mergedArrayList", + IntStream.range(0, tempArray1.length).allMatch(k -> mergedArrayList.contains(tempArray1[k]))); + Assert.assertTrue("All elements of tempArray2 should be in mergedArrayList", + IntStream.range(0, tempArray2.length).allMatch(k -> mergedArrayList.contains(tempArray2[k]))); + } + } + } +} From 7949758abe1cb2091e2c3ad0de3426e51214f6a5 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Fri, 10 May 2019 09:59:17 -0700 Subject: [PATCH 0291/1642] Move common placementpolicy components to TopologyAwareEnsemblePlacementPolicy. Descriptions of the changes in this PR: - Moving components/methods which are common in nature with respect to placementpolicy from RackawareEnsemblePlacementPolicyImpl to TopologyAwareEnsemblePlacementPolicy, so that any new placementpolicy implementation can extend TopologyAwareEnsemblePlacementPolicy and reuse those common components/generic. - This change has no functionality change, it is just reorganizing code. Reviewers: Enrico Olivelli , Sijie Guo This closes #2089 from reddycharan/3azplacement --- .../DefaultEnsemblePlacementPolicy.java | 2 +- .../RackawareEnsemblePlacementPolicyImpl.java | 162 +---------- .../TopologyAwareEnsemblePlacementPolicy.java | 265 +++++++++++++++++- .../client/WeightedRandomSelection.java | 139 +-------- .../client/WeightedRandomSelectionImpl.java | 170 +++++++++++ .../client/TestWeightedRandomSelection.java | 2 +- 6 files changed, 448 insertions(+), 292 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java 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 c130b5d4944..f6bb1af6b3a 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 @@ -195,7 +195,7 @@ public EnsemblePlacementPolicy initialize(ClientConfiguration conf, this.isWeighted = conf.getDiskWeightBasedPlacementEnabled(); if (this.isWeighted) { this.maxWeightMultiple = conf.getBookieMaxWeightMultipleForWeightBasedPlacement(); - this.weightedSelection = new WeightedRandomSelection(this.maxWeightMultiple); + this.weightedSelection = new WeightedRandomSelectionImpl(this.maxWeightMultiple); } return this; } 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 ed35d490a70..be06e13a5a8 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 @@ -46,10 +46,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.function.Supplier; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; @@ -88,10 +85,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsemblePlacementPolicy { static final Logger LOG = LoggerFactory.getLogger(RackawareEnsemblePlacementPolicyImpl.class); - boolean isWeighted; int maxWeightMultiple; - private Map bookieInfoMap = new HashMap(); - private WeightedRandomSelection weightedSelection; protected int minNumRacksPerWriteQuorum; protected boolean enforceMinNumRacksPerWriteQuorum; @@ -112,109 +106,10 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP static final int UNAVAIL_MASK = 0x40 << 24; static final int MASK_BITS = 0xFFF << 20; - static class DefaultResolver implements DNSToSwitchMapping { - - final Supplier defaultRackSupplier; - - public DefaultResolver(Supplier defaultRackSupplier) { - checkNotNull(defaultRackSupplier, "defaultRackSupplier should not be null"); - this.defaultRackSupplier = defaultRackSupplier; - } - - @Override - public List resolve(List names) { - List rNames = new ArrayList(names.size()); - for (@SuppressWarnings("unused") String name : names) { - final String defaultRack = defaultRackSupplier.get(); - checkNotNull(defaultRack, "defaultRack cannot be null"); - rNames.add(defaultRack); - } - return rNames; - } - - @Override - public void reloadCachedMappings() { - // nop - } - - } - - /** - * Decorator for any existing dsn resolver. - * Backfills returned data with appropriate default rack info. - */ - static class DNSResolverDecorator implements DNSToSwitchMapping { - - final Supplier defaultRackSupplier; - final DNSToSwitchMapping resolver; - @StatsDoc( - name = FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER, - help = "total number of times Resolver failed to resolve rack information of a node" - ) - final Counter failedToResolveNetworkLocationCounter; - - DNSResolverDecorator(DNSToSwitchMapping resolver, Supplier defaultRackSupplier, - Counter failedToResolveNetworkLocationCounter) { - checkNotNull(resolver, "Resolver cannot be null"); - checkNotNull(defaultRackSupplier, "defaultRackSupplier should not be null"); - this.defaultRackSupplier = defaultRackSupplier; - this.resolver = resolver; - this.failedToResolveNetworkLocationCounter = failedToResolveNetworkLocationCounter; - } - - public List resolve(List names) { - if (names == null) { - return Collections.emptyList(); - } - final String defaultRack = defaultRackSupplier.get(); - checkNotNull(defaultRack, "Default rack cannot be null"); - - List rNames = resolver.resolve(names); - if (rNames != null && rNames.size() == names.size()) { - 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 : {}.", - names.get(i), defaultRack); - failedToResolveNetworkLocationCounter.inc(); - rNames.set(i, defaultRack); - } - } - return rNames; - } - - LOG.warn("Failed to resolve network location for {}, using default rack for them : {}.", names, - defaultRack); - rNames = new ArrayList<>(names.size()); - - for (int i = 0; i < names.size(); ++i) { - failedToResolveNetworkLocationCounter.inc(); - rNames.add(defaultRack); - } - return rNames; - } - - @Override - public boolean useHostName() { - return resolver.useHostName(); - } - - @Override - public void reloadCachedMappings() { - resolver.reloadCachedMappings(); - } - } - - // for now, we just maintain the writable bookies' topology - protected NetworkTopology topology; - protected DNSToSwitchMapping dnsResolver; protected HashedWheelTimer timer; - protected final Map knownBookies; // Use a loading cache so slow bookies are expired. Use entryId as values. protected Cache slowBookies; protected BookieNode localNode; - protected final ReentrantReadWriteLock rwLock; - // Initialize to empty set - protected ImmutableSet readOnlyBookies = ImmutableSet.of(); protected boolean reorderReadsRandom = false; protected boolean enforceDurability = false; protected int stabilizePeriodSeconds = 0; @@ -222,19 +117,10 @@ public void reloadCachedMappings() { // looks like these only assigned in the same thread as constructor, immediately after constructor; // no need to make volatile protected StatsLogger statsLogger = null; + @StatsDoc( - name = BOOKIES_JOINED, - help = "The distribution of number of bookies joined the cluster on each network topology change" - ) - protected OpStatsLogger bookiesJoinedCounter = null; - @StatsDoc( - name = BOOKIES_LEFT, - help = "The distribution of number of bookies left the cluster on each network topology change" - ) - protected OpStatsLogger bookiesLeftCounter = null; - @StatsDoc( - name = READ_REQUESTS_REORDERED, - help = "The distribution of number of bookies reordered on each read request" + name = READ_REQUESTS_REORDERED, + help = "The distribution of number of bookies reordered on each read request" ) protected OpStatsLogger readReorderedCounter = null; @StatsDoc( @@ -257,9 +143,6 @@ public void reloadCachedMappings() { RackawareEnsemblePlacementPolicyImpl(boolean enforceDurability) { this.enforceDurability = enforceDurability; topology = new NetworkTopologyImpl(); - knownBookies = new HashMap(); - - rwLock = new ReentrantReadWriteLock(); } protected BookieNode createBookieNode(BookieSocketAddress addr) { @@ -341,7 +224,7 @@ public Integer getSample() { this.isWeighted = isWeighted; if (this.isWeighted) { this.maxWeightMultiple = maxWeightMultiple; - this.weightedSelection = new WeightedRandomSelection(this.maxWeightMultiple); + this.weightedSelection = new WeightedRandomSelectionImpl(this.maxWeightMultiple); LOG.info("Weight based placement with max multiple of " + this.maxWeightMultiple); } else { LOG.info("Not weighted"); @@ -553,14 +436,6 @@ protected Set convertBookiesToNodes(Collection exclud return nodes; } - private static Set getNetworkLocations(Set bookieNodes) { - Set networkLocs = new HashSet<>(); - for (Node bookieNode : bookieNodes) { - networkLocs.add(bookieNode.getNetworkLocation()); - } - return networkLocs; - } - /* * this method should be called in readlock scope of 'rwLock' */ @@ -883,7 +758,8 @@ private WeightedRandomSelection prepareForWeightedSelection(List wRSelection = new WeightedRandomSelection(maxWeightMultiple); + WeightedRandomSelection wRSelection = new WeightedRandomSelectionImpl( + maxWeightMultiple); wRSelection.updateMap(rackMap); return wRSelection; } @@ -1000,7 +876,7 @@ protected List selectRandomInternal(List bookiesToSelect rackMap.put(n, new BookieInfo()); } } - wRSelection = new WeightedRandomSelection(this.maxWeightMultiple); + wRSelection = new WeightedRandomSelectionImpl(this.maxWeightMultiple); wRSelection.updateMap(rackMap); } } else { @@ -1287,30 +1163,6 @@ DistributionSchedule.WriteSet reorderReadSequenceWithRegion( 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))); - } - } - } - // this method should be called in readlock scope of 'rwlock' @Override public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, 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 7fa75550498..355632b25e4 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 @@ -17,19 +17,38 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_JOINED; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_LEFT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER; + +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; 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.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Supplier; +import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; +import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.net.NetUtils; import org.apache.bookkeeper.net.NetworkTopology; +import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.net.NodeBase; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,15 +56,34 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements ITopologyAwareEnsemblePlacementPolicy { static final Logger LOG = LoggerFactory.getLogger(TopologyAwareEnsemblePlacementPolicy.class); - protected static class TruePredicate implements Predicate { + protected final Map knownBookies = new HashMap(); + protected final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); + protected Map bookieInfoMap = new HashMap(); + // Initialize to empty set + protected ImmutableSet readOnlyBookies = ImmutableSet.of(); + boolean isWeighted; + protected WeightedRandomSelection weightedSelection; + // for now, we just maintain the writable bookies' topology + protected NetworkTopology topology; + protected DNSToSwitchMapping dnsResolver; + @StatsDoc( + name = BOOKIES_JOINED, + help = "The distribution of number of bookies joined the cluster on each network topology change" + ) + protected OpStatsLogger bookiesJoinedCounter = null; + @StatsDoc( + name = BOOKIES_LEFT, + help = "The distribution of number of bookies left the cluster on each network topology change" + ) + protected OpStatsLogger bookiesLeftCounter = null; + protected static class TruePredicate implements Predicate { public static final TruePredicate INSTANCE = new TruePredicate(); @Override public boolean apply(BookieNode candidate, Ensemble chosenNodes) { return true; } - } protected static class EnsembleForReplacementWithNoConstraints implements Ensemble { @@ -487,6 +525,129 @@ public String toString() { } } + static class DefaultResolver implements DNSToSwitchMapping { + + final Supplier defaultRackSupplier; + + public DefaultResolver(Supplier defaultRackSupplier) { + checkNotNull(defaultRackSupplier, "defaultRackSupplier should not be null"); + this.defaultRackSupplier = defaultRackSupplier; + } + + @Override + public List resolve(List names) { + List rNames = new ArrayList(names.size()); + for (@SuppressWarnings("unused") String name : names) { + final String defaultRack = defaultRackSupplier.get(); + checkNotNull(defaultRack, "defaultRack cannot be null"); + rNames.add(defaultRack); + } + return rNames; + } + + @Override + public void reloadCachedMappings() { + // nop + } + } + + /** + * Decorator for any existing dsn resolver. + * Backfills returned data with appropriate default rack info. + */ + static class DNSResolverDecorator implements DNSToSwitchMapping { + + final Supplier defaultRackSupplier; + final DNSToSwitchMapping resolver; + @StatsDoc( + name = FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER, + help = "total number of times Resolver failed to resolve rack information of a node" + ) + final Counter failedToResolveNetworkLocationCounter; + + DNSResolverDecorator(DNSToSwitchMapping resolver, Supplier defaultRackSupplier, + Counter failedToResolveNetworkLocationCounter) { + checkNotNull(resolver, "Resolver cannot be null"); + checkNotNull(defaultRackSupplier, "defaultRackSupplier should not be null"); + this.defaultRackSupplier = defaultRackSupplier; + this.resolver = resolver; + this.failedToResolveNetworkLocationCounter = failedToResolveNetworkLocationCounter; + } + + public List resolve(List names) { + if (names == null) { + return Collections.emptyList(); + } + final String defaultRack = defaultRackSupplier.get(); + checkNotNull(defaultRack, "Default rack cannot be null"); + + List rNames = resolver.resolve(names); + if (rNames != null && rNames.size() == names.size()) { + 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 : {}.", + names.get(i), defaultRack); + failedToResolveNetworkLocationCounter.inc(); + rNames.set(i, defaultRack); + } + } + return rNames; + } + + LOG.warn("Failed to resolve network location for {}, using default rack for them : {}.", names, + defaultRack); + rNames = new ArrayList<>(names.size()); + + for (int i = 0; i < names.size(); ++i) { + failedToResolveNetworkLocationCounter.inc(); + rNames.add(defaultRack); + } + return rNames; + } + + @Override + public boolean useHostName() { + return resolver.useHostName(); + } + + @Override + public void reloadCachedMappings() { + resolver.reloadCachedMappings(); + } + } + + static Set getNetworkLocations(Set bookieNodes) { + Set networkLocs = new HashSet<>(); + for (Node bookieNode : bookieNodes) { + networkLocs.add(bookieNode.getNetworkLocation()); + } + return networkLocs; + } + + /** + * 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))); + } + } + } + @Override public DistributionSchedule.WriteSet reorderReadSequence( List ensemble, @@ -505,4 +666,104 @@ public DistributionSchedule.WriteSet reorderReadLACSequence( retList.addMissingIndices(ensemble.size()); return retList; } + + @Override + public Set onClusterChanged(Set writableBookies, + Set readOnlyBookies) { + rwLock.writeLock().lock(); + try { + ImmutableSet joinedBookies, leftBookies, deadBookies; + Set oldBookieSet = knownBookies.keySet(); + // left bookies : bookies in known bookies, but not in new writable bookie cluster. + leftBookies = Sets.difference(oldBookieSet, writableBookies).immutableCopy(); + // joined bookies : bookies in new writable bookie cluster, but not in known bookies + joinedBookies = Sets.difference(writableBookies, oldBookieSet).immutableCopy(); + // dead bookies. + deadBookies = Sets.difference(leftBookies, readOnlyBookies).immutableCopy(); + LOG.debug("Cluster changed : left bookies are {}, joined bookies are {}, while dead bookies are {}.", + leftBookies, joinedBookies, deadBookies); + handleBookiesThatLeft(leftBookies); + handleBookiesThatJoined(joinedBookies); + if (this.isWeighted && (leftBookies.size() > 0 || joinedBookies.size() > 0)) { + this.weightedSelection.updateMap(this.bookieInfoMap); + } + if (!readOnlyBookies.isEmpty()) { + this.readOnlyBookies = ImmutableSet.copyOf(readOnlyBookies); + } + + return deadBookies; + } finally { + rwLock.writeLock().unlock(); + } + } + + /* + * this method should be called in writelock scope of 'rwLock' + */ + @Override + public void handleBookiesThatLeft(Set leftBookies) { + for (BookieSocketAddress addr : leftBookies) { + try { + BookieNode node = knownBookies.remove(addr); + if (null != node) { + topology.remove(node); + if (this.isWeighted) { + this.bookieInfoMap.remove(node); + } + + bookiesLeftCounter.registerSuccessfulValue(1L); + + if (LOG.isDebugEnabled()) { + LOG.debug("Cluster changed : bookie {} left from cluster.", addr); + } + } + } catch (Throwable t) { + LOG.error("Unexpected exception while handling leaving bookie {}", addr, t); + if (bookiesLeftCounter != null) { + bookiesLeftCounter.registerFailedValue(1L); + } + // no need to re-throw; we want to process the rest of the bookies + // exception anyways will be caught/logged/suppressed in the ZK's event handler + } + } + } + + /* + * this method should be called in writelock scope of 'rwLock' + */ + @Override + public void handleBookiesThatJoined(Set joinedBookies) { + // node joined + for (BookieSocketAddress addr : joinedBookies) { + try { + BookieNode node = createBookieNode(addr); + topology.add(node); + knownBookies.put(addr, node); + if (this.isWeighted) { + this.bookieInfoMap.putIfAbsent(node, new BookieInfo()); + } + + bookiesJoinedCounter.registerSuccessfulValue(1L); + + if (LOG.isDebugEnabled()) { + LOG.debug("Cluster changed : bookie {} joined the cluster.", addr); + } + } catch (Throwable t) { + // topology.add() throws unchecked exception + LOG.error("Unexpected exception while handling joining bookie {}", addr, t); + + bookiesJoinedCounter.registerFailedValue(1L); + // no need to re-throw; we want to process the rest of the bookies + // exception anyways will be caught/logged/suppressed in the ZK's event handler + } + } + } + + protected BookieNode createBookieNode(BookieSocketAddress addr) { + return new BookieNode(addr, resolveNetworkLocation(addr)); + } + + protected String resolveNetworkLocation(BookieSocketAddress addr) { + return NetUtils.resolveNetworkLocation(dnsResolver, addr.getSocketAddress()); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelection.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelection.java index 040425075eb..8a44174a45d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelection.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelection.java @@ -18,146 +18,19 @@ package org.apache.bookkeeper.client; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; +import java.util.Collection; import java.util.Map; -import java.util.TreeMap; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class WeightedRandomSelection { - static final Logger LOG = LoggerFactory.getLogger(WeightedRandomSelection.class); +interface WeightedRandomSelection { interface WeightedObject { long getWeight(); } - Double randomMax; - int maxProbabilityMultiplier; - Map map; - TreeMap cummulativeMap = new TreeMap(); - ReadWriteLock rwLock = new ReentrantReadWriteLock(true); - - WeightedRandomSelection() { - maxProbabilityMultiplier = -1; - } - - WeightedRandomSelection(int maxMultiplier) { - this.maxProbabilityMultiplier = maxMultiplier; - } - - public void setMaxProbabilityMultiplier(int max) { - this.maxProbabilityMultiplier = max; - } - - void updateMap(Map map) { - // get the sum total of all the values; this will be used to - // calculate the weighted probability later on - Long totalWeight = 0L, min = Long.MAX_VALUE; - List values = new ArrayList(map.values()); - Collections.sort(values, new Comparator() { - public int compare(WeightedObject o1, WeightedObject o2) { - long diff = o1.getWeight() - o2.getWeight(); - if (diff < 0L) { - return -1; - } else if (diff > 0L) { - return 1; - } else { - return 0; - } - } - }); - for (int i = 0; i < values.size(); i++) { - totalWeight += values.get(i).getWeight(); - if (values.get(i).getWeight() != 0 && min > values.get(i).getWeight()) { - min = values.get(i).getWeight(); - } - } - - double median = 0; - if (totalWeight == 0) { - // all the values are zeros; assign a value of 1 to all and the totalWeight equal - // to the size of the values - min = 1L; - median = 1; - totalWeight = (long) values.size(); - } else { - int mid = values.size() / 2; - if ((values.size() % 2) == 1) { - median = values.get(mid).getWeight(); - } else { - median = (double) (values.get(mid - 1).getWeight() + values.get(mid).getWeight()) / 2; - } - } - - double medianWeight, minWeight; - medianWeight = median / (double) totalWeight; - minWeight = (double) min / totalWeight; - if (LOG.isDebugEnabled()) { - LOG.debug("Updating weights map. MediaWeight: {} MinWeight: {}", medianWeight, minWeight); - } + void updateMap(Map map); - double maxWeight = maxProbabilityMultiplier * medianWeight; - Map weightMap = new HashMap(); - for (Map.Entry e : map.entrySet()) { - double weightedProbability; - if (e.getValue().getWeight() > 0) { - weightedProbability = (double) e.getValue().getWeight() / (double) totalWeight; - } else { - weightedProbability = minWeight; - } - if (maxWeight > 0 && weightedProbability > maxWeight) { - weightedProbability = maxWeight; - if (LOG.isDebugEnabled()) { - LOG.debug("Capping the probability to {} for {} Value: {}", - weightedProbability, e.getKey(), e.getValue()); - } - } - weightMap.put(e.getKey(), weightedProbability); - } + T getNextRandom(); - // The probability of picking a bookie randomly is defaultPickProbability - // but we change that priority by looking at the weight that each bookie - // carries. - TreeMap tmpCummulativeMap = new TreeMap(); - Double key = 0.0; - for (Map.Entry e : weightMap.entrySet()) { - tmpCummulativeMap.put(key, e.getKey()); - if (LOG.isDebugEnabled()) { - LOG.debug("Key: {} Value: {} AssignedKey: {} AssignedWeight: {}", - e.getKey(), e.getValue(), key, e.getValue()); - } - key += e.getValue(); - } + T getNextRandom(Collection selectedNodes); - rwLock.writeLock().lock(); - try { - this.map = map; - cummulativeMap = tmpCummulativeMap; - randomMax = key; - } finally { - rwLock.writeLock().unlock(); - } - } - - T getNextRandom() { - rwLock.readLock().lock(); - try { - // pick a random number between 0 and randMax - Double randomNum = randomMax * Math.random(); - // find the nearest key in the map corresponding to the randomNum - Double key = cummulativeMap.floorKey(randomNum); - //LOG.info("Random max: {} CummulativeMap size: {} selected key: {}", randomMax, cummulativeMap.size(), - // key); - return cummulativeMap.get(key); - } finally { - rwLock.readLock().unlock(); - } - } + void setMaxProbabilityMultiplier(int max); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java new file mode 100644 index 00000000000..53130a22675 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java @@ -0,0 +1,170 @@ +/* + * 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.client; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class WeightedRandomSelectionImpl implements WeightedRandomSelection { + static final Logger LOG = LoggerFactory.getLogger(WeightedRandomSelectionImpl.class); + + Double randomMax; + int maxProbabilityMultiplier; + Map map; + TreeMap cummulativeMap = new TreeMap(); + ReadWriteLock rwLock = new ReentrantReadWriteLock(true); + + WeightedRandomSelectionImpl() { + maxProbabilityMultiplier = -1; + } + + WeightedRandomSelectionImpl(int maxMultiplier) { + this.maxProbabilityMultiplier = maxMultiplier; + } + + @Override + public void updateMap(Map map) { + // get the sum total of all the values; this will be used to + // calculate the weighted probability later on + Long totalWeight = 0L, min = Long.MAX_VALUE; + List values = new ArrayList(map.values()); + Collections.sort(values, new Comparator() { + public int compare(WeightedObject o1, WeightedObject o2) { + long diff = o1.getWeight() - o2.getWeight(); + if (diff < 0L) { + return -1; + } else if (diff > 0L) { + return 1; + } else { + return 0; + } + } + }); + for (int i = 0; i < values.size(); i++) { + totalWeight += values.get(i).getWeight(); + if (values.get(i).getWeight() != 0 && min > values.get(i).getWeight()) { + min = values.get(i).getWeight(); + } + } + + double median = 0; + if (totalWeight == 0) { + // all the values are zeros; assign a value of 1 to all and the totalWeight equal + // to the size of the values + min = 1L; + median = 1; + totalWeight = (long) values.size(); + } else { + int mid = values.size() / 2; + if ((values.size() % 2) == 1) { + median = values.get(mid).getWeight(); + } else { + median = (double) (values.get(mid - 1).getWeight() + values.get(mid).getWeight()) / 2; + } + } + + double medianWeight, minWeight; + medianWeight = median / (double) totalWeight; + minWeight = (double) min / totalWeight; + + if (LOG.isDebugEnabled()) { + LOG.debug("Updating weights map. MediaWeight: {} MinWeight: {}", medianWeight, minWeight); + } + + double maxWeight = maxProbabilityMultiplier * medianWeight; + Map weightMap = new HashMap(); + for (Map.Entry e : map.entrySet()) { + double weightedProbability; + if (e.getValue().getWeight() > 0) { + weightedProbability = (double) e.getValue().getWeight() / (double) totalWeight; + } else { + weightedProbability = minWeight; + } + if (maxWeight > 0 && weightedProbability > maxWeight) { + weightedProbability = maxWeight; + if (LOG.isDebugEnabled()) { + LOG.debug("Capping the probability to {} for {} Value: {}", + weightedProbability, e.getKey(), e.getValue()); + } + } + weightMap.put(e.getKey(), weightedProbability); + } + + // The probability of picking a bookie randomly is defaultPickProbability + // but we change that priority by looking at the weight that each bookie + // carries. + TreeMap tmpCummulativeMap = new TreeMap(); + Double key = 0.0; + for (Map.Entry e : weightMap.entrySet()) { + tmpCummulativeMap.put(key, e.getKey()); + if (LOG.isDebugEnabled()) { + LOG.debug("Key: {} Value: {} AssignedKey: {} AssignedWeight: {}", + e.getKey(), e.getValue(), key, e.getValue()); + } + key += e.getValue(); + } + + rwLock.writeLock().lock(); + try { + this.map = map; + cummulativeMap = tmpCummulativeMap; + randomMax = key; + } finally { + rwLock.writeLock().unlock(); + } + } + + @Override + public T getNextRandom() { + rwLock.readLock().lock(); + try { + // pick a random number between 0 and randMax + Double randomNum = randomMax * Math.random(); + // find the nearest key in the map corresponding to the randomNum + Double key = cummulativeMap.floorKey(randomNum); + //LOG.info("Random max: {} CummulativeMap size: {} selected key: {}", randomMax, cummulativeMap.size(), + // key); + return cummulativeMap.get(key); + } finally { + rwLock.readLock().unlock(); + } + } + + @Override + public void setMaxProbabilityMultiplier(int max) { + this.maxProbabilityMultiplier = max; + } + + @Override + public T getNextRandom(Collection selectedNodes) { + throw new UnsupportedOperationException("getNextRandom is not implemented for WeightedRandomSelectionImpl"); + } + +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWeightedRandomSelection.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWeightedRandomSelection.java index fe8f74bfb1f..2aa3c0e89af 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWeightedRandomSelection.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWeightedRandomSelection.java @@ -59,7 +59,7 @@ public long getWeight() { @Before public void setUp() throws Exception { - wRS = new WeightedRandomSelection(); + wRS = new WeightedRandomSelectionImpl(); } @After From 73e39ead53ebcd73d151560465cc83b0fb348b6b Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Mon, 13 May 2019 16:06:41 -0700 Subject: [PATCH 0292/1642] Move common placementpolicy components to TopologyAwareEnsemblePlacementPolicy. Descriptions of the changes in this PR: - Moving components/methods which are common in nature with respect to placementpolicy from RackawareEnsemblePlacementPolicyImpl to TopologyAwareEnsemblePlacementPolicy, so that any new placementpolicy implementation can extend TopologyAwareEnsemblePlacementPolicy and reuse those common components/generic. - Change signature of methods of RackChangeNotifier interface. - Delete duplicate methods in RackawareEnsemblePlacementPolicyImpl and TopologyAwareEnsemblePlacementPolicy (missed removing them in #2089) - Created separate class for BookieNode - This change has no functionality change, it is just reorganizing code. Reviewers: Sijie Guo , Venkateswararao Jujjuri (JV) This closes #2092 from reddycharan/placementpolicyrefactoring --- ...ITopologyAwareEnsemblePlacementPolicy.java | 9 +- .../bookkeeper/client/RackChangeNotifier.java | 8 +- .../RackawareEnsemblePlacementPolicy.java | 3 +- .../RackawareEnsemblePlacementPolicyImpl.java | 135 +----------------- .../RegionAwareEnsemblePlacementPolicy.java | 1 + .../TopologyAwareEnsemblePlacementPolicy.java | 69 +++++---- .../org/apache/bookkeeper/net/BookieNode.java | 53 +++++++ .../TestRackawareEnsemblePlacementPolicy.java | 2 +- .../bookkeeper/util/StaticDNSResolver.java | 8 +- 9 files changed, 110 insertions(+), 178 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java index b196236af9b..d6197c09e8b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java @@ -23,9 +23,9 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble; import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate; -import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; +import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.Node; @@ -185,4 +185,11 @@ T selectFromNetworkLocation(String networkLoc, * bookies that joined. */ void handleBookiesThatJoined(Set joinedBookies); + + /** + * Handle rack change for the bookies. + * + * @param bookieAddressList + */ + void onBookieRackChange(List bookieAddressList); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackChangeNotifier.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackChangeNotifier.java index c66bcaf9762..e9fc3a654d4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackChangeNotifier.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackChangeNotifier.java @@ -18,8 +18,11 @@ package org.apache.bookkeeper.client; +import org.apache.bookkeeper.net.BookieNode; + /** - * Notifier used by the RackawareEnsemblePlacementPolicy to get notified if a rack changes for a bookie. + * Notifier used by the RackawareEnsemblePlacementPolicy to get notified if a + * rack changes for a bookie. */ public interface RackChangeNotifier { @@ -28,5 +31,6 @@ public interface RackChangeNotifier { * * @param rackawarePolicy */ - void registerRackChangeListener(RackawareEnsemblePlacementPolicyImpl rackawarePolicy); + void registerRackChangeListener( + ITopologyAwareEnsemblePlacementPolicy rackawarePolicy); } 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 49e9df53da7..380f556b13e 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 @@ -23,6 +23,7 @@ import java.util.Set; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; +import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.Node; @@ -34,7 +35,7 @@ * @see EnsemblePlacementPolicy */ public class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacementPolicyImpl - implements ITopologyAwareEnsemblePlacementPolicy { + implements ITopologyAwareEnsemblePlacementPolicy { RackawareEnsemblePlacementPolicyImpl slave = null; public RackawareEnsemblePlacementPolicy() { 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 be06e13a5a8..2b1090f9ed6 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 @@ -29,15 +29,12 @@ import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; import io.netty.util.HashedWheelTimer; import java.net.InetAddress; import java.net.UnknownHostException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -55,9 +52,9 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.Configurable; import org.apache.bookkeeper.feature.FeatureProvider; +import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; -import org.apache.bookkeeper.net.NetUtils; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.net.NetworkTopologyImpl; import org.apache.bookkeeper.net.Node; @@ -145,10 +142,6 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP topology = new NetworkTopologyImpl(); } - protected BookieNode createBookieNode(BookieSocketAddress addr) { - return new BookieNode(addr, resolveNetworkLocation(addr)); - } - /** * Initialize the policy. * @@ -310,132 +303,6 @@ public void uninitalize() { // do nothing } - protected String resolveNetworkLocation(BookieSocketAddress addr) { - return NetUtils.resolveNetworkLocation(dnsResolver, addr.getSocketAddress()); - } - - public void onBookieRackChange(List bookieAddressList) { - rwLock.writeLock().lock(); - try { - for (BookieSocketAddress bookieAddress : bookieAddressList) { - BookieNode node = knownBookies.get(bookieAddress); - if (node != null) { - // refresh the rack info if its a known bookie - topology.remove(node); - BookieNode newNode = createBookieNode(bookieAddress); - topology.add(newNode); - knownBookies.put(bookieAddress, newNode); - } - } - } finally { - rwLock.writeLock().unlock(); - } - } - - @Override - public Set onClusterChanged(Set writableBookies, - Set readOnlyBookies) { - rwLock.writeLock().lock(); - try { - ImmutableSet joinedBookies, leftBookies, deadBookies; - Set oldBookieSet = knownBookies.keySet(); - // left bookies : bookies in known bookies, but not in new writable bookie cluster. - leftBookies = Sets.difference(oldBookieSet, writableBookies).immutableCopy(); - // joined bookies : bookies in new writable bookie cluster, but not in known bookies - joinedBookies = Sets.difference(writableBookies, oldBookieSet).immutableCopy(); - // dead bookies. - deadBookies = Sets.difference(leftBookies, readOnlyBookies).immutableCopy(); - LOG.debug("Cluster changed : left bookies are {}, joined bookies are {}, while dead bookies are {}.", - leftBookies, joinedBookies, deadBookies); - handleBookiesThatLeft(leftBookies); - handleBookiesThatJoined(joinedBookies); - if (this.isWeighted && (leftBookies.size() > 0 || joinedBookies.size() > 0)) { - this.weightedSelection.updateMap(this.bookieInfoMap); - } - if (!readOnlyBookies.isEmpty()) { - this.readOnlyBookies = ImmutableSet.copyOf(readOnlyBookies); - } - - return deadBookies; - } finally { - rwLock.writeLock().unlock(); - } - } - - /* - * this method should be called in writelock scope of 'rwLock' - */ - @Override - public void handleBookiesThatLeft(Set leftBookies) { - for (BookieSocketAddress addr : leftBookies) { - try { - BookieNode node = knownBookies.remove(addr); - if (null != node) { - topology.remove(node); - if (this.isWeighted) { - this.bookieInfoMap.remove(node); - } - - bookiesLeftCounter.registerSuccessfulValue(1L); - - if (LOG.isDebugEnabled()) { - LOG.debug("Cluster changed : bookie {} left from cluster.", addr); - } - } - } catch (Throwable t) { - LOG.error("Unexpected exception while handling leaving bookie {}", addr, t); - if (bookiesLeftCounter != null) { - bookiesLeftCounter.registerFailedValue(1L); - } - // no need to re-throw; we want to process the rest of the bookies - // exception anyways will be caught/logged/suppressed in the ZK's event handler - } - } - } - - /* - * this method should be called in writelock scope of 'rwLock' - */ - @Override - public void handleBookiesThatJoined(Set joinedBookies) { - // node joined - for (BookieSocketAddress addr : joinedBookies) { - try { - BookieNode node = createBookieNode(addr); - topology.add(node); - knownBookies.put(addr, node); - if (this.isWeighted) { - this.bookieInfoMap.putIfAbsent(node, new BookieInfo()); - } - - bookiesJoinedCounter.registerSuccessfulValue(1L); - - if (LOG.isDebugEnabled()) { - LOG.debug("Cluster changed : bookie {} joined the cluster.", addr); - } - } catch (Throwable t) { - // topology.add() throws unchecked exception - LOG.error("Unexpected exception while handling joining bookie {}", addr, t); - - bookiesJoinedCounter.registerFailedValue(1L); - // no need to re-throw; we want to process the rest of the bookies - // exception anyways will be caught/logged/suppressed in the ZK's event handler - } - } - } - - protected Set convertBookiesToNodes(Collection excludeBookies) { - Set nodes = new HashSet(); - for (BookieSocketAddress addr : excludeBookies) { - BookieNode bn = knownBookies.get(addr); - if (null == bn) { - bn = createBookieNode(addr); - } - nodes.add(bn); - } - return nodes; - } - /* * this method should be called in readlock scope of 'rwLock' */ 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 399e8aaf322..5ed8dcab015 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 @@ -32,6 +32,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.FeatureProvider; +import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; 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 355632b25e4..27b077d1bd6 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 @@ -28,6 +28,7 @@ import com.google.common.collect.Sets; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -40,6 +41,7 @@ import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; +import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetUtils; @@ -53,7 +55,7 @@ import org.slf4j.LoggerFactory; abstract class TopologyAwareEnsemblePlacementPolicy implements - ITopologyAwareEnsemblePlacementPolicy { + ITopologyAwareEnsemblePlacementPolicy { static final Logger LOG = LoggerFactory.getLogger(TopologyAwareEnsemblePlacementPolicy.class); protected final Map knownBookies = new HashMap(); @@ -115,38 +117,6 @@ public boolean validate() { } - protected static class BookieNode extends NodeBase { - private final BookieSocketAddress addr; // identifier of a bookie node. - - BookieNode(BookieSocketAddress addr, String networkLoc) { - super(addr.toString(), networkLoc); - this.addr = addr; - } - - public BookieSocketAddress getAddr() { - return addr; - } - - @Override - public int hashCode() { - return name.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof BookieNode)) { - return false; - } - BookieNode other = (BookieNode) obj; - return getName().equals(other.getName()); - } - - @Override - public String toString() { - return String.format("", name); - } - } - /** * A predicate checking the rack coverage for write quorum in {@link RoundRobinDistributionSchedule}, * which ensures that a write quorum should be covered by at least two racks. @@ -330,8 +300,6 @@ public void addBookie(BookieNode candidate) { } } - - final int distanceFromLeaves; final int ensembleSize; final int writeQuorumSize; @@ -759,6 +727,25 @@ public void handleBookiesThatJoined(Set joinedBookies) { } } + @Override + public void onBookieRackChange(List bookieAddressList) { + rwLock.writeLock().lock(); + try { + for (BookieSocketAddress bookieAddress : bookieAddressList) { + BookieNode node = knownBookies.get(bookieAddress); + if (node != null) { + // refresh the rack info if its a known bookie + topology.remove(node); + BookieNode newNode = createBookieNode(bookieAddress); + topology.add(newNode); + knownBookies.put(bookieAddress, newNode); + } + } + } finally { + rwLock.writeLock().unlock(); + } + } + protected BookieNode createBookieNode(BookieSocketAddress addr) { return new BookieNode(addr, resolveNetworkLocation(addr)); } @@ -766,4 +753,16 @@ protected BookieNode createBookieNode(BookieSocketAddress addr) { protected String resolveNetworkLocation(BookieSocketAddress addr) { return NetUtils.resolveNetworkLocation(dnsResolver, addr.getSocketAddress()); } + + protected Set convertBookiesToNodes(Collection excludeBookies) { + Set nodes = new HashSet(); + for (BookieSocketAddress addr : excludeBookies) { + BookieNode bn = knownBookies.get(addr); + if (null == bn) { + bn = createBookieNode(addr); + } + nodes.add(bn); + } + return nodes; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java new file mode 100644 index 00000000000..e7d0ef50482 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java @@ -0,0 +1,53 @@ +/** + * 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.net; + +/** + * Bookie Node implementation. + */ +public class BookieNode extends NodeBase { + private final BookieSocketAddress addr; // identifier of a bookie node. + + public BookieNode(BookieSocketAddress addr, String networkLoc) { + super(addr.toString(), networkLoc); + this.addr = addr; + } + + public BookieSocketAddress getAddr() { + return addr; + } + + @Override + public int hashCode() { + return name.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof BookieNode)) { + return false; + } + BookieNode other = (BookieNode) obj; + return getName().equals(other.getName()); + } + + @Override + public String toString() { + return String.format("", name); + } +} \ No newline at end of file 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 26fa23ff566..4a93d4f6fb1 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 @@ -43,10 +43,10 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble; -import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.EnsembleForReplacementWithNoConstraints; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.TruePredicate; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java index d5cb06710b7..b21d8aa683e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java @@ -22,9 +22,10 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy; import org.apache.bookkeeper.client.RackChangeNotifier; -import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl; import org.apache.bookkeeper.net.AbstractDNSToSwitchMapping; +import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; @@ -87,10 +88,10 @@ public void reloadCachedMappings() { // nop } - private static RackawareEnsemblePlacementPolicyImpl rackawarePolicy = null; + private static ITopologyAwareEnsemblePlacementPolicy rackawarePolicy = null; @Override - public void registerRackChangeListener(RackawareEnsemblePlacementPolicyImpl rackawareEnsemblePolicy) { + public void registerRackChangeListener(ITopologyAwareEnsemblePlacementPolicy rackawareEnsemblePolicy) { rackawarePolicy = rackawareEnsemblePolicy; } @@ -101,5 +102,4 @@ public static void changeRack(List bookieAddressList, List< } rackawarePolicy.onBookieRackChange(bookieAddressList); } - } From e3ccdad2779c9a19c4649a0b1d8c6445c9f5de0a Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 14 May 2019 23:35:18 -0700 Subject: [PATCH 0293/1642] [BK-GC] avoid blocking call in gc-thread ### Motivation Right now, we have below 3 issues because of which gc thread gets blocked forever and it can't perform gc-task further. Below issues are mainly related to blocking call while doing zk-operation without timeout. bug-fixes: 1. right now, [GC - ScanAndCompareGarbageCollector](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java#L142) passes timeout in millisecond to [LedgerManager](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java#L166) but it takes it as second and again try to convert it in millis so, 30Kms timeout becomes [30M ms timeout](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java#L245). Sp, fix timeout unit during gc. 2. Right now, GC makes blocking call to get list of children on ledger znode and sometime zk-call back doesn't comeback which blocks the gc-thread forever. However, recently we added the timeout on the [object-waiting-lock](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java#L243-L248) which doesn't work because it's in while loop and `object.wait(timeout)` completes without any exception and GC threads keep running in while loop. 3. add zk-timeout during delete ledgers in bookie else it can also block the GC thread. ### Changes add timeout while bk-gc makes zk-call to verify deleted ledgers. Reviewers: Enrico Olivelli , Sijie Guo , Rajan Dhabalia This closes #1940 from rdhabalia/verify_gc --- .../ScanAndCompareGarbageCollector.java | 23 ++++++++++++++----- .../bookkeeper/meta/CleanupLedgerManager.java | 4 ++-- .../bookkeeper/meta/FlatLedgerManager.java | 4 ++-- .../meta/HierarchicalLedgerManager.java | 6 ++--- .../apache/bookkeeper/meta/LedgerManager.java | 4 ++-- .../meta/LegacyHierarchicalLedgerManager.java | 12 +++++----- .../meta/LongHierarchicalLedgerManager.java | 12 +++++----- .../meta/MSLedgerManagerFactory.java | 2 +- .../org/apache/bookkeeper/util/ZkUtils.java | 12 +++++++--- .../bookkeeper/bookie/CompactionTest.java | 2 +- .../client/ParallelLedgerRecoveryTest.java | 4 ++-- .../bookkeeper/meta/MockLedgerManager.java | 2 +- .../metadata/etcd/EtcdLedgerManager.java | 2 +- 13 files changed, 53 insertions(+), 36 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index 247b610aee0..02e47aec526 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -32,6 +32,9 @@ import java.util.TreeSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; @@ -139,8 +142,9 @@ public void gc(GarbageCleaner garbageCleaner) { } // Iterate over all the ledger on the metadata store - long zkOpTimeout = this.conf.getZkTimeout() * 2; - LedgerRangeIterator ledgerRangeIterator = ledgerManager.getLedgerRanges(zkOpTimeout); + long zkOpTimeoutMs = this.conf.getZkTimeout() * 2; + LedgerRangeIterator ledgerRangeIterator = ledgerManager + .getLedgerRanges(zkOpTimeoutMs); Set ledgersInMetadata = null; long start; long end = -1; @@ -167,13 +171,20 @@ public void gc(GarbageCleaner garbageCleaner) { if (verifyMetadataOnGc) { int rc = BKException.Code.OK; try { - result(ledgerManager.readLedgerMetadata(bkLid)); - } catch (BKException e) { - rc = e.getCode(); + result(ledgerManager.readLedgerMetadata(bkLid), zkOpTimeoutMs, TimeUnit.MILLISECONDS); + } catch (BKException | TimeoutException e) { + if (e instanceof BKException) { + rc = ((BKException) e).getCode(); + } else { + LOG.warn("Time-out while fetching metadata for Ledger {} : {}.", bkLid, + e.getMessage()); + + continue; + } } if (rc != BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { LOG.warn("Ledger {} Missing in metadata list, but ledgerManager returned rc: {}.", - bkLid, rc); + bkLid, rc); continue; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java index 3c7ddc51d30..a3ce432961f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java @@ -210,13 +210,13 @@ public void processResult(int rc, String path, Object ctx) { } @Override - public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { closeLock.readLock().lock(); try { if (closed) { return new ClosedLedgerRangeIterator(); } - return underlying.getLedgerRanges(zkOpTimeoutSec); + return underlying.getLedgerRanges(zkOpTimeoutMs); } finally { closeLock.readLock().unlock(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java index 8c7f05f9f7f..7c89326ffe7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java @@ -89,7 +89,7 @@ public void asyncProcessLedgers(final Processor processor, } @Override - public LedgerRangeIterator getLedgerRanges(long zkOpTimeOutSec) { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { return new LedgerRangeIterator() { // single iterator, can visit only one time boolean nextCalled = false; @@ -103,7 +103,7 @@ private synchronized void preload() throws IOException { try { zkActiveLedgers = ledgerListToSet( - ZkUtils.getChildrenInSingleNode(zk, ledgerRootPath, zkOpTimeOutSec), + ZkUtils.getChildrenInSingleNode(zk, ledgerRootPath, zkOpTimeoutMs), ledgerRootPath); nextRange = new LedgerRange(zkActiveLedgers); } catch (KeeperException.NoNodeException e) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java index 079fcfacd25..9cd6aed92e3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java @@ -87,9 +87,9 @@ protected long getLedgerId(String ledgerPath) throws IOException { } @Override - public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { - LedgerRangeIterator legacyLedgerRangeIterator = legacyLM.getLedgerRanges(zkOpTimeoutSec); - LedgerRangeIterator longLedgerRangeIterator = longLM.getLedgerRanges(zkOpTimeoutSec); + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { + LedgerRangeIterator legacyLedgerRangeIterator = legacyLM.getLedgerRanges(zkOpTimeoutMs); + LedgerRangeIterator longLedgerRangeIterator = longLM.getLedgerRanges(zkOpTimeoutMs); return new HierarchicalLedgerRangeIterator(legacyLedgerRangeIterator, longLedgerRangeIterator); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java index 15da14b083b..56c447d538c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java @@ -149,12 +149,12 @@ void asyncProcessLedgers(Processor processor, AsyncCallback.VoidCallback f /** * Loop to scan a range of metadata from metadata storage. * - * @param zkOpTimeOutSec + * @param zkOpTimeOutMs * Iterator considers timeout while fetching ledger-range from * zk. * @return will return a iterator of the Ranges */ - LedgerRangeIterator getLedgerRanges(long zkOpTimeOutSec); + LedgerRangeIterator getLedgerRanges(long zkOpTimeOutMs); /** * Used to represent the Ledgers range returned from the diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java index 1a63407e7ad..9828719ba5a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LegacyHierarchicalLedgerManager.java @@ -153,8 +153,8 @@ protected String getLedgerParentNodeRegex() { } @Override - public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { - return new LegacyHierarchicalLedgerRangeIterator(zkOpTimeoutSec); + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { + return new LegacyHierarchicalLedgerRangeIterator(zkOpTimeoutMs); } /** @@ -166,10 +166,10 @@ private class LegacyHierarchicalLedgerRangeIterator implements LedgerRangeIterat private String curL1Nodes = ""; private boolean iteratorDone = false; private LedgerRange nextRange = null; - private final long zkOpTimeoutSec; + private final long zkOpTimeoutMs; - public LegacyHierarchicalLedgerRangeIterator(long zkOpTimeoutSec) { - this.zkOpTimeoutSec = zkOpTimeoutSec; + public LegacyHierarchicalLedgerRangeIterator(long zkOpTimeoutMs) { + this.zkOpTimeoutMs = zkOpTimeoutMs; } /** @@ -266,7 +266,7 @@ LedgerRange getLedgerRangeByLevel(final String level1, final String level2) String nodePath = nodeBuilder.toString(); List ledgerNodes = null; try { - ledgerNodes = ZkUtils.getChildrenInSingleNode(zk, nodePath, zkOpTimeoutSec); + ledgerNodes = ZkUtils.getChildrenInSingleNode(zk, nodePath, zkOpTimeoutMs); } catch (KeeperException.NoNodeException e) { /* If the node doesn't exist, we must have raced with a recursive node removal, just * return an empty list. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java index 95f8e48bde8..7ec2f5af1b8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java @@ -139,8 +139,8 @@ public void process(String lNode, VoidCallback cb) { } @Override - public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { - return new LongHierarchicalLedgerRangeIterator(zkOpTimeoutSec); + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { + return new LongHierarchicalLedgerRangeIterator(zkOpTimeoutMs); } @@ -149,7 +149,7 @@ public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { */ private class LongHierarchicalLedgerRangeIterator implements LedgerRangeIterator { LedgerRangeIterator rootIterator; - final long zkOpTimeoutSec; + final long zkOpTimeoutMs; /** * Returns all children with path as a parent. If path is non-existent, @@ -163,7 +163,7 @@ private class LongHierarchicalLedgerRangeIterator implements LedgerRangeIterator */ List getChildrenAt(String path) throws IOException { try { - List children = ZkUtils.getChildrenInSingleNode(zk, path, zkOpTimeoutSec); + List children = ZkUtils.getChildrenInSingleNode(zk, path, zkOpTimeoutMs); Collections.sort(children); return children; } catch (KeeperException.NoNodeException e) { @@ -285,8 +285,8 @@ public LedgerRange next() throws IOException { } } - private LongHierarchicalLedgerRangeIterator(long zkOpTimeoutSec) { - this.zkOpTimeoutSec = zkOpTimeoutSec; + private LongHierarchicalLedgerRangeIterator(long zkOpTimeoutMs) { + this.zkOpTimeoutMs = zkOpTimeoutMs; } private void bootstrap() throws IOException { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index 201a0a3edb9..9b21c876e34 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -642,7 +642,7 @@ public LedgerRange next() throws IOException { } @Override - public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { return new MSLedgerRangeIterator(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java index 23d0b421c3e..cc0612fc77c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java @@ -25,7 +25,6 @@ import java.io.IOException; import java.util.List; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.conf.AbstractConfiguration; @@ -222,7 +221,7 @@ private static class GetChildrenCtx { * @throws InterruptedException * @throws IOException */ - public static List getChildrenInSingleNode(final ZooKeeper zk, final String node, long timeOutSec) + public static List getChildrenInSingleNode(final ZooKeeper zk, final String node, long zkOpTimeoutMs) throws InterruptedException, IOException, KeeperException.NoNodeException { final GetChildrenCtx ctx = new GetChildrenCtx(); getChildrenInSingleNode(zk, node, new GenericCallback>() { @@ -240,13 +239,20 @@ public void operationComplete(int rc, List ledgers) { }); synchronized (ctx) { + long startTime = System.currentTimeMillis(); while (!ctx.done) { try { - ctx.wait(timeOutSec > 0 ? TimeUnit.SECONDS.toMillis(timeOutSec) : 0); + ctx.wait(zkOpTimeoutMs > 0 ? zkOpTimeoutMs : 0); } catch (InterruptedException e) { ctx.rc = Code.OPERATIONTIMEOUT.intValue(); ctx.done = true; } + // timeout the process if get-children response not received + // zkOpTimeoutMs. + if (zkOpTimeoutMs > 0 && (System.currentTimeMillis() - startTime) >= zkOpTimeoutMs) { + ctx.rc = Code.OPERATIONTIMEOUT.intValue(); + ctx.done = true; + } } } if (Code.NONODE.intValue() == ctx.rc) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java index 0d6e5d64775..adef116475c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java @@ -970,7 +970,7 @@ void unsupported() { } @Override - public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { final AtomicBoolean hasnext = new AtomicBoolean(true); return new LedgerManager.LedgerRangeIterator() { @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index e6c3c222928..177cbe38df3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -112,8 +112,8 @@ public CompletableFuture> readLedgerMetadata(long ledg } @Override - public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { - return lm.getLedgerRanges(zkOpTimeoutSec); + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { + return lm.getLedgerRanges(zkOpTimeoutMs); } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java index 0f818b66c9e..952837be37e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java @@ -188,7 +188,7 @@ public void asyncProcessLedgers(Processor processor, AsyncCallback.VoidCal } @Override - public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutSec) { + public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { return null; } diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java index a40a4bc383b..f91f2f7f1f4 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java @@ -420,7 +420,7 @@ private void processLedgers(KeyStream ks, } @Override - public LedgerRangeIterator getLedgerRanges(long opTimeOutSec) { + public LedgerRangeIterator getLedgerRanges(long opTimeOutMs) { KeyStream ks = new KeyStream<>( kvClient, ByteSequence.fromString(EtcdUtils.getLedgerKey(scope, 0L)), From d3442709cc60f85029a7b397ba775229ba1bc879 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 16 May 2019 10:35:08 -0700 Subject: [PATCH 0294/1642] Added release notes for release 4.9.2 ### Motivation Release notes for patch release Reviewers: Enrico Olivelli , Sijie Guo This closes #2088 from merlimat/relnotes-4.9.2 --- site/docs/4.9.2/overview/releaseNotes.md | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 site/docs/4.9.2/overview/releaseNotes.md diff --git a/site/docs/4.9.2/overview/releaseNotes.md b/site/docs/4.9.2/overview/releaseNotes.md new file mode 100644 index 00000000000..ea01f4bf6a3 --- /dev/null +++ b/site/docs/4.9.2/overview/releaseNotes.md @@ -0,0 +1,24 @@ +--- +title: Apache BookKeeper 4.9.2 Release Notes +--- + +The 4.9.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.9.1. + +Apache BookKeeper users who are using 4.9.1 are encouraged to upgrade to 4.9.2. The technical details of this release are summarized +below. + +## Highlights + + - Added HTTP handler to expose bookie state [apache/bookkeeper#1995](https://github.com/apache/bookkeeper/pull/1995) + - Fix DbLedgerStorage encountering unexpected entry id [apache/bookkeeper#2002](https://github.com/apache/bookkeeper/pull/2002) + - Close db properly to avoid open RocksDB failure at the second time [apache/bookkeeper#2022](https://github.com/apache/bookkeeper/pull/2022) + - Cancel LAC watch when longpoll LAC times out [apache/bookkeeper#2051](https://github.com/apache/bookkeeper/pull/2051) + - Wait for LAC update even if ledger fenced [apache/bookkeeper#2052](https://github.com/apache/bookkeeper/pull/2052) + - Bugfix for Percentile Calculation in FastCodahale Timer Implementation [apache/bookkeeper#2054](https://github.com/apache/bookkeeper/pull/2054) + - Use pure python implementation of MurmurHash [apache/bookkeeper#2069](https://github.com/apache/bookkeeper/pull/2069) + - Bookieshell lastmark command isn't functional, always returning 0-0 [apache/bookkeeper#2076](https://github.com/apache/bookkeeper/pull/2076) + + +## Full list of changes + +- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.2+is%3Aclosed) From da29647ea2d19aad7bf904c505ec9640952e8b71 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 17 May 2019 00:39:12 -0700 Subject: [PATCH 0295/1642] [BK-CLIENT] Check empty ledger-parent node while deleting ledger MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### Motivation As discussed at [#4276](https://github.com/apache/pulsar/issues/4276), while deleting ledger, bk-client should check parent node is empty before issuing delete request for parent znode. Reviewers: Enrico Olivelli , Charan Reddy Guttapalem , Matteo Merli This closes #2097 from rdhabalia/led_del and squashes the following commits: f5c0ca36c [rdhabalia] return callback with ok ede5e9403 [rdhabalia] [Bk-Client] Check empty ledger-parent node while deleting ledger d35aa22ad [Charan Reddy Guttapalem] Move common placementpolicy components to TopologyAwareEnsemblePlacementPolicy. b4ca4537b [Charan Reddy Guttapalem] Move common placementpolicy components to TopologyAwareEnsemblePlacementPolicy. aa84c7fdd [Charan Reddy Guttapalem] GetListOfEntriesOfLedger implementation 10859afb7 [Matteo Merli] Added HTTP handler to expose bookie state 707ae5c85 [karanmehta93] ISSUE #2075: Bookieshell lastmark command isn't functional, always returning 0-0 41b39c6ef [Charan Reddy Guttapalem] ISSUE #1967: make ledger creation and removal robust to zk connectionloss 973d2ab0d [Matteo Merli] Use pure python implementation of MurmurHash 9bb7e4b82 [Venkateswararao Jujjuri (JV)] Explicit error message if extent is not present on ZK (#2066) bd699e61d [mtang01] ISSUE #2067: reduce byte[] allocation in add entry 7c62e1232 [karanmehta93] ISSUE #2073: ReadOnlyBookieTest#testBookieContinueWritingIfMulti… 42e77802c [Ivan Kelly] DLog Namespace#openLog should declare LogNotFoundException 86bce12b5 [Yong Zhang] Migrate command `ledgermetadata` 407cb35e5 [Charan Reddy Guttapalem] ISSUE #1967: make ledger creation and removal robust to zk connectionloss eaa601404 [Like] Support asynchronous fence request for V2 ReadEntryProcessor d23b45ec8 [Ivan Kelly] Fix typo in overview page for 4.8.2 44ee320b6 [Ivan Kelly] k 316b71923 [Ivan Kelly] Wait for LAC update even if ledger fenced 066621507 [Yong Zhang] Migrate command `updatecookie` 6f3396801 [Yong Zhang] Migrate command `triggeraudit` 60d993edf [Yong Zhang] Migrate command `autorecovery` ed008f278 [Yong Zhang] Migrate command `whoisauditor` 5b8e0971a [Yong Zhang] Migrate command `Whatisinstanceid` 90c79444d [Yong Zhang] Migrate command `rebuild-db-ledger-locations-index` 848f8527f [Nicolas Michael] ISSUE #2053: Bugfix for Percentile Calculation in FastCodahale Timer Implementation 06f2b6f50 [Yong Zhang] Migrate command `updateledgers` 7ad5849b1 [Yong Zhang] Migrate command `regenerate-interleaved-storage-index-file` d4dbb6bfb [Dongfa,Huang] Avoid useless verify if LedgerEntryRequest completed 5c150f283 [Enrico Olivelli] Release notes for 4.9.1 1246826ba [Yong Zhang] Migrate command `recover` 1d4cc71fd [Yong Zhang] Migrate command `localconsistencycheck` 67f83620e [Yong Zhang] Migrate command `readledger` bfbd6b023 [Yong Zhang] Migrate command `decommission` d40b8b69f [Yong Zhang] Migrate command `readlog` 95d145a15 [Yong Zhang] Migrate command `nukeexistingcluster` e2b1dc7f3 [Yong Zhang] Migrate command `listunderreplicated` 0988e12c7 [bd2019us] ISSUE #2023: change cached thread pool to fixed thread pool 6a6d7bbd9 [Yong Zhang] Migrate command `initnewcluster` c391fe58d [Yong Zhang] Migrate command `readlogmetadata` 120d67737 [Yong Zhang] Migrate command `lostbookierecoverydelay` bf66235e5 [Yong Zhang] Migrate command `deleteledger` 751e55fa4 [Arvin] ISSUE #2020: close db properly to avoid open RocksDB failure at the second time 138a7ae85 [Yong Zhang] Migrate command `metadataformat` b043d1694 [Yong Zhang] Migrate command `listledgers` 4573285db [Ivan Kelly] Docker autobuild hook e3d807a32 [Like] Fix IDE complain as there are multi choices for error code 9524a9f4a [Yong Zhang] Migrate command `readjournal` 6c3f33f55 [Yong Zhang] Fix when met unexpect entry id crashed e35a108c7 [Like] Fix error message for unrecognized number-of-bookies 5902ee27b [Boyang Jerry Peng] fix potential NPE when releasing entry that is null 6aa73ce05 [Ivan Kelly] [RELEASE] Update website to include documentation for 4.8.2 1448d12aa [Yong Zhang] Migrate command `listfilesondisk` 4de598379 [Yong Zhang] Issue #1987: Migrate command `convert-to-interleaved-storage` 468743e7e [Matteo Merli] In DbLedgerStorage use default values when config key is present but empty f26a4cae0 [Ivan Kelly] Release notes for v4.8.2 ec2636cd2 [Yong Zhang] Issue #1985: Migrate command `convert-to-db-storage` 8cc7239ac [Yong Zhang] Issue #1982: Migrate command `bookiesanity` fa90f0185 [Yong Zhang] Issue #1980: Migrate command `ledger` from shell to bkctl --- .../main/java/org/apache/bookkeeper/util/ZkUtils.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java index cc0612fc77c..9d6ba1ab9b0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java @@ -133,8 +133,17 @@ public static void asyncDeleteFullPathOptimistic(final ZooKeeper zk, final Strin public void processResult(int rc, String path, Object ctx) { if (rc == Code.OK.intValue()) { String parent = new File(originalPath).getParent().replace("\\", "/"); - asyncDeleteFullPathOptimistic(zk, parent, -1, callback, leafNodePath); + zk.getData(parent, false, (dRc, dPath, dCtx, data, stat) -> { + if (Code.OK.intValue() == dRc && (stat != null && stat.getNumChildren() == 0)) { + asyncDeleteFullPathOptimistic(zk, parent, -1, callback, leafNodePath); + } else { + // parent node is not empty so, complete the + // callback + callback.processResult(Code.OK.intValue(), path, leafNodePath); + } + }, null); } else { + // parent node deletion fails.. so, complete the callback if (path.equals(leafNodePath)) { callback.processResult(rc, path, leafNodePath); } else { From dbc15d298033cbccd34a32ca398220a13c2cd30f Mon Sep 17 00:00:00 2001 From: karanmehta93 Date: Fri, 17 May 2019 10:38:43 -0700 Subject: [PATCH 0296/1642] Update and flush lastLogMark when replaying journal Descriptions of the changes in this PR: Update `lastLogMark` in memory after replaying each journal Check for writable ledger dirs with `minUsableSizeForEntryLogCreation` to flush the `lastMark` file for bookies in ReadOnlyMode Log line changes ### Motivation Master Issue: #2087 Reviewers: Enrico Olivelli , Charan Reddy Guttapalem , Sijie Guo , Venkateswararao Jujjuri (JV) This closes #2090 from karanmehta93/master and squashes the following commits: f802cf6da [Karan Mehta] Fixed small issue in constuctor chaining in BookKeeperClusterTestCase 407e9f1ff [Karan Mehta] Moved lastLogMark updation logic to Journal.java 585313196 [Karan Mehta] Fix checkstyle errors c3d47014a [Karan Mehta] Fix testJournalLogAddEntryCalledCorrectly test c72b7b55e [Karan Mehta] Addressed nit 5d238f2b7 [Karan Mehta] Refactored code to update lastLogMark only when replaying journal and addressed nits b5515697f [Karan Mehta] Issue #2087 Update and flush lastLogMark when replaying journal d35aa22ad [Charan Reddy Guttapalem] Move common placementpolicy components to TopologyAwareEnsemblePlacementPolicy. --- .../org/apache/bookkeeper/bookie/Bookie.java | 41 +++++- .../org/apache/bookkeeper/bookie/Journal.java | 56 ++------ .../bookie/BookieInitializationTest.java | 122 ++++++++++++++++++ .../bookkeeper/bookie/BookieJournalTest.java | 27 ++-- .../bookie/BookieWriteToJournalTest.java | 16 ++- .../test/BookKeeperClusterTestCase.java | 2 +- .../org/apache/bookkeeper/util/TestUtils.java | 12 ++ 7 files changed, 220 insertions(+), 56 deletions(-) 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 32adb5d9034..467cb037095 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 @@ -883,12 +883,51 @@ public void process(int journalVersion, long offset, ByteBuffer recBuff) throws }; for (Journal journal : journals) { - journal.replay(scanner); + replay(journal, scanner); } long elapsedTs = System.currentTimeMillis() - startTs; LOG.info("Finished replaying journal in {} ms.", elapsedTs); } + /** + * Replay journal files and updates journal's in-memory lastLogMark object. + * + * @param journal Journal object corresponding to a journalDir + * @param scanner Scanner to process replayed entries. + * @throws IOException + */ + private void replay(Journal journal, JournalScanner scanner) throws IOException { + final LogMark markedLog = journal.getLastLogMark().getCurMark(); + List logs = Journal.listJournalIds(journal.getJournalDirectory(), journalId -> { + if (journalId < markedLog.getLogFileId()) { + return false; + } + return true; + }); + // last log mark may be missed due to no sync up before + // validate filtered log ids only when we have markedLogId + if (markedLog.getLogFileId() > 0) { + if (logs.size() == 0 || logs.get(0) != markedLog.getLogFileId()) { + throw new IOException("Recovery log " + markedLog.getLogFileId() + " is missing"); + } + } + + // TODO: When reading in the journal logs that need to be synced, we + // should use BufferedChannels instead to minimize the amount of + // system calls done. + for (Long id : logs) { + long logPosition = 0L; + if (id == markedLog.getLogFileId()) { + logPosition = markedLog.getLogFileOffset(); + } + LOG.info("Replaying journal {} from position {}", id, logPosition); + journal.scanJournal(id, logPosition, scanner); + // Update LastLogMark to Long.MAX_VALUE position after replaying journal + // After LedgerStorage flush, SyncThread should persist this to disk + journal.setLastLogMarkToEof(id); + } + } + @Override public synchronized void start() { setDaemon(true); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 0a78fcada8d..8941b685faf 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -78,7 +78,7 @@ public class Journal extends BookieCriticalThread implements CheckpointSource { /** * Filter to pickup journals. */ - private interface JournalIdFilter { + public interface JournalIdFilter { boolean accept(long journalId); } @@ -196,11 +196,13 @@ void rollLog(LastLogMark lastMark) throws NoWritableLedgerDirException { // which is safe since records before lastMark have been // persisted to disk (both index & entry logger) lastMark.getCurMark().writeLogMark(bb); + if (LOG.isDebugEnabled()) { LOG.debug("RollLog to persist last marked log : {}", lastMark.getCurMark()); } + List writableLedgerDirs = ledgerDirsManager - .getWritableLedgerDirs(); + .getWritableLedgerDirsForNewLog(); for (File dir : writableLedgerDirs) { File file = new File(dir, lastMarkFileName); FileOutputStream fos = null; @@ -707,6 +709,15 @@ public LastLogMark getLastLogMark() { return lastLogMark; } + /** + * Update lastLogMark of the journal, + * Indicates that the file has been processed. + * @param id + */ + void setLastLogMarkToEof(Long id) { + lastLogMark.getCurMark().setLogMark(id, Long.MAX_VALUE); + } + /** * Application tried to schedule a checkpoint. After all the txns added * before checkpoint are persisted, a checkpoint will be returned @@ -826,47 +837,6 @@ public void scanJournal(long journalId, long journalPos, JournalScanner scanner) } } - /** - * Replay journal files. - * - * @param scanner Scanner to process replayed entries. - * @throws IOException - */ - public void replay(JournalScanner scanner) throws IOException { - final LogMark markedLog = lastLogMark.getCurMark(); - List logs = listJournalIds(journalDirectory, new JournalIdFilter() { - @Override - public boolean accept(long journalId) { - if (journalId < markedLog.getLogFileId()) { - return false; - } - return true; - } - }); - // last log mark may be missed due to no sync up before - // validate filtered log ids only when we have markedLogId - if (markedLog.getLogFileId() > 0) { - if (logs.size() == 0 || logs.get(0) != markedLog.getLogFileId()) { - throw new IOException("Recovery log " + markedLog.getLogFileId() + " is missing"); - } - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Try to relay journal logs : {}", logs); - } - // TODO: When reading in the journal logs that need to be synced, we - // should use BufferedChannels instead to minimize the amount of - // system calls done. - for (Long id: logs) { - long logPosition = 0L; - if (id == markedLog.getLogFileId()) { - logPosition = markedLog.getLogFileOffset(); - } - LOG.info("Replaying journal {} from position {}", id, logPosition); - scanJournal(id, logPosition, scanner); - } - } - public void logAddEntry(ByteBuffer entry, boolean ackBeforeSync, WriteCallback cb, Object ctx) throws InterruptedException { logAddEntry(Unpooled.wrappedBuffer(entry), ackBeforeSync, cb, ctx); 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 c4ed03135bd..dbaef35b735 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 @@ -23,6 +23,7 @@ import static com.google.common.base.Charsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.BOOKIE_STATUS_FILENAME; +import static org.apache.bookkeeper.util.TestUtils.countNumOfFiles; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -59,6 +60,7 @@ import java.util.concurrent.ExecutionException; import org.apache.bookkeeper.bookie.BookieException.DiskPartitionDuplicationException; import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; +import org.apache.bookkeeper.bookie.Journal.LastLogMark; import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; @@ -136,6 +138,126 @@ public void tearDown() throws Exception { super.tearDown(); } + @Test + public void testOneJournalReplayForBookieRestartInReadOnlyMode() throws Exception { + testJournalReplayForBookieRestartInReadOnlyMode(1); + } + + @Test + public void testMultipleJournalReplayForBookieRestartInReadOnlyMode() throws Exception { + testJournalReplayForBookieRestartInReadOnlyMode(4); + } + + /** + * Tests that journal replay works correctly when bookie crashes and starts up in RO mode. + */ + private void testJournalReplayForBookieRestartInReadOnlyMode(int numOfJournalDirs) throws Exception { + File tmpLedgerDir = createTempDir("DiskCheck", "test"); + File tmpJournalDir = createTempDir("DiskCheck", "test"); + + String[] journalDirs = new String[numOfJournalDirs]; + for (int i = 0; i < numOfJournalDirs; i++) { + journalDirs[i] = tmpJournalDir.getAbsolutePath() + "/journal-" + i; + } + + final ServerConfiguration conf = newServerConfiguration() + .setJournalDirsName(journalDirs) + .setLedgerDirNames(new String[] { tmpLedgerDir.getPath() }) + .setDiskCheckInterval(1000) + .setLedgerStorageClass(SortedLedgerStorage.class.getName()) + .setAutoRecoveryDaemonEnabled(false) + .setZkTimeout(5000); + + BookieServer server = new MockBookieServer(conf); + server.start(); + + List lastLogMarkList = new ArrayList<>(journalDirs.length); + + for (int i = 0; i < journalDirs.length; i++) { + Journal journal = server.getBookie().journals.get(i); + // LastLogMark should be (0, 0) at the bookie clean start + journal.getLastLogMark().readLog(); + lastLogMarkList.add(journal.getLastLogMark().markLog()); + assertEquals(0, lastLogMarkList.get(i).getCurMark().compare(new LogMark(0, 0))); + } + + ClientConfiguration clientConf = new ClientConfiguration(); + clientConf.setMetadataServiceUri(metadataServiceUri); + BookKeeper bkClient = new BookKeeper(clientConf); + + // Create multiple ledgers for adding entries to multiple journals + for (int i = 0; i < journalDirs.length; i++) { + LedgerHandle lh = bkClient.createLedger(1, 1, 1, DigestType.CRC32, "passwd".getBytes()); + long entryId = -1; + // Ensure that we have non-zero number of entries + long numOfEntries = new Random().nextInt(10) + 3; + for (int j = 0; j < numOfEntries; j++) { + entryId = lh.addEntry("data".getBytes()); + } + assertEquals(entryId, (numOfEntries - 1)); + lh.close(); + } + + for (int i = 0; i < journalDirs.length; i++) { + Journal journal = server.getBookie().journals.get(i); + // In-memory LastLogMark should be updated with every write to journal + assertTrue(journal.getLastLogMark().getCurMark().compare(lastLogMarkList.get(i).getCurMark()) > 0); + lastLogMarkList.set(i, journal.getLastLogMark().markLog()); + } + + // Kill Bookie abruptly before entries are flushed to disk + server.shutdown(); + + conf.setDiskUsageThreshold(0.001f) + .setDiskUsageWarnThreshold(0.0f).setReadOnlyModeEnabled(true).setIsForceGCAllowWhenNoSpace(true) + .setMinUsableSizeForIndexFileCreation(5 * 1024); + + server = new BookieServer(conf); + + for (int i = 0; i < journalDirs.length; i++) { + Journal journal = server.getBookie().journals.get(i); + // LastLogMark should be (0, 0) before bookie restart since bookie crashed before persisting lastMark + assertEquals(0, journal.getLastLogMark().getCurMark().compare(new LogMark(0, 0))); + } + + int numOfRestarts = 3; + // Restart server multiple times to ensure that logs are never replayed and new files are not generated + for (int i = 0; i < numOfRestarts; i++) { + + int txnBefore = countNumOfFiles(conf.getJournalDirs(), "txn"); + int logBefore = countNumOfFiles(conf.getLedgerDirs(), "log"); + int idxBefore = countNumOfFiles(conf.getLedgerDirs(), "idx"); + + server.start(); + + for (int j = 0; j < journalDirs.length; j++) { + Journal journal = server.getBookie().journals.get(j); + assertTrue(journal.getLastLogMark().getCurMark().compare(lastLogMarkList.get(j).getCurMark()) > 0); + lastLogMarkList.set(j, journal.getLastLogMark().markLog()); + } + + server.shutdown(); + + // Every bookie restart initiates a new journal file + // Journals should not be replayed everytime since lastMark gets updated everytime + // New EntryLog files should not be generated. + assertEquals(journalDirs.length, (countNumOfFiles(conf.getJournalDirs(), "txn") - txnBefore)); + + // First restart should replay journal and generate new log/index files + // Subsequent runs should not generate new files (but can delete older ones) + if (i == 0) { + assertTrue((countNumOfFiles(conf.getLedgerDirs(), "log") - logBefore) > 0); + assertTrue((countNumOfFiles(conf.getLedgerDirs(), "idx") - idxBefore) > 0); + } else { + assertTrue((countNumOfFiles(conf.getLedgerDirs(), "log") - logBefore) <= 0); + assertTrue((countNumOfFiles(conf.getLedgerDirs(), "idx") - idxBefore) <= 0); + } + + server = new BookieServer(conf); + } + bkClient.close(); + } + /** * Verify the bookie server exit code. On ZooKeeper exception, should return * exit code ZK_REG_FAIL = 4 diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java index 080ebfebb36..0f1b3271032 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java @@ -38,6 +38,7 @@ import java.util.List; import java.util.Random; +import org.apache.bookkeeper.bookie.Journal.LastLogMark; import org.apache.bookkeeper.client.ClientUtil; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -350,8 +351,7 @@ public void testPreV2Journal() throws Exception { .setLedgerDirNames(new String[] { ledgerDir.getPath() }) .setMetadataServiceUri(null); - Bookie b = new Bookie(conf); - b.readJournal(); + Bookie b = createBookieAndReadJournal(conf); b.readEntry(1, 100); try { @@ -379,8 +379,7 @@ public void testV4Journal() throws Exception { .setLedgerDirNames(new String[] { ledgerDir.getPath() }) .setMetadataServiceUri(null); - Bookie b = new Bookie(conf); - b.readJournal(); + Bookie b = createBookieAndReadJournal(conf); b.readEntry(1, 100); try { @@ -410,8 +409,7 @@ public void testV5Journal() throws Exception { .setLedgerDirNames(new String[] { ledgerDir.getPath() }) .setMetadataServiceUri(null); - Bookie b = new Bookie(conf); - b.readJournal(); + Bookie b = createBookieAndReadJournal(conf); for (int i = 1; i <= 2 * JournalChannel.SECTOR_SIZE; i++) { b.readEntry(1, i); @@ -570,8 +568,7 @@ public void testTruncatedInLenJournal() throws Exception { .setLedgerDirNames(new String[] { ledgerDir.getPath() }) .setMetadataServiceUri(null); - Bookie b = new Bookie(conf); - b.readJournal(); + Bookie b = createBookieAndReadJournal(conf); b.readEntry(1, 99); @@ -614,8 +611,8 @@ public void testTruncatedInEntryJournal() throws Exception { .setLedgerDirNames(new String[] { ledgerDir.getPath() }) .setMetadataServiceUri(null); - Bookie b = new Bookie(conf); - b.readJournal(); + Bookie b = createBookieAndReadJournal(conf); + b.readEntry(1, 99); // still able to read last entry, but it's junk @@ -640,6 +637,16 @@ public void testTruncatedInEntryJournal() throws Exception { } } + private Bookie createBookieAndReadJournal(ServerConfiguration conf) throws Exception { + Bookie b = new Bookie(conf); + for (Journal journal : b.journals) { + LastLogMark lastLogMark = journal.getLastLogMark().markLog(); + b.readJournal(); + assertTrue(journal.getLastLogMark().getCurMark().compare(lastLogMark.getCurMark()) > 0); + } + return b; + } + /** * Test journal replay with SortedLedgerStorage and a very small max * arena size. diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java index 2d197d39fa0..27a5d28618b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java @@ -33,6 +33,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import java.io.File; +import java.io.IOException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -63,6 +64,19 @@ public class BookieWriteToJournalTest { @Rule public TemporaryFolder tempDir = new TemporaryFolder(); + class NoOpJournalReplayBookie extends Bookie { + + public NoOpJournalReplayBookie(ServerConfiguration conf) + throws IOException, InterruptedException, BookieException { + super(conf); + } + + @Override + void readJournal() throws IOException, BookieException { + // Should be no-op since journal objects are mocked + } + } + /** * test that Bookie calls correctly Journal.logAddEntry about "ackBeforeSync" parameter. */ @@ -102,7 +116,7 @@ public void testJournalLogAddEntryCalledCorrectly() throws Exception { whenNew(Journal.class).withAnyArguments().thenReturn(journal); - Bookie b = new Bookie(conf); + Bookie b = new NoOpJournalReplayBookie(conf); b.start(); long ledgerId = 1; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index 97b7488b78d..b8e2426bf96 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -123,7 +123,7 @@ public BookKeeperClusterTestCase(int numBookies) { } public BookKeeperClusterTestCase(int numBookies, int testTimeoutSecs) { - this(numBookies, 1, 120); + this(numBookies, 1, testTimeoutSecs); } public BookKeeperClusterTestCase(int numBookies, int numOfZKNodes, int testTimeoutSecs) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java index 352560775c0..1dc30d4616e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/TestUtils.java @@ -22,6 +22,7 @@ package org.apache.bookkeeper.util; import java.io.File; +import java.util.Collection; import java.util.HashSet; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -32,6 +33,7 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.commons.io.FileUtils; import org.junit.Assert; /** @@ -92,4 +94,14 @@ public static void assertEventuallyTrue(String description, BooleanSupplier pred } Assert.assertTrue(description, predicate.getAsBoolean()); } + + public static int countNumOfFiles(File[] folderNames, String... extensions) { + int count = 0; + for (int i = 0; i < folderNames.length; i++) { + Collection filesCollection = FileUtils.listFiles(folderNames[i], extensions, true); + count += filesCollection.size(); + } + return count; + } + } From 972571b44d71e278f6055d5af70bb9ecf3ae8eab Mon Sep 17 00:00:00 2001 From: hello zepp Date: Mon, 20 May 2019 10:45:35 +0800 Subject: [PATCH 0297/1642] [SITE] modify nettyMaxFrameSizeBytes Description Descriptions of the changes in this PR: Fixes #2079 ### Motivation when the client-side attempt to send more than the default size bytes, it should set up the corresponding parameter `setNettyMaxFrameSizeBytes(int maxSize)` (Explain: why you're making that change, what is the problem you're trying to solve) Master Issue: #2079 Reviewers: Enrico Olivelli , Sijie Guo This closes #2086 from hellozepp/issue-2079 --- site/_data/config/bk_server.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index 328becada7a..172e0a97aae 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -124,7 +124,7 @@ groups: description: The Recv ByteBuf allocator max buf size. default: 1048576 - param: nettyMaxFrameSizeBytes - description: The maximum netty frame size in bytes. Any message received larger than this will be rejected. + description: The maximum netty frame size in bytes. Any message received larger than this will be rejected, so when the client-side attempt to send more than the default size bytes, it should set up the corresponding parameter `setNettyMaxFrameSizeBytes(int maxSize)`, pay attention to the parameter should be less than the value of server-side. default: 5242880 - name: Http server settings From 93a47aa99fe84ae8de8bcd548c87727b02c0d48b Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 20 May 2019 10:51:58 +0800 Subject: [PATCH 0298/1642] Fallback to use v3 protocol for some types of requests if they are not implemented in v2 Descriptions of the changes in this PR: ### Motivation #2071 ### Changes - Add a client pool use v3 wire protocol - Obtain client by version - Currently only support `writeLac` and `readLac` Master Issue: #2071 Reviewers: Enrico Olivelli , Sijie Guo This closes #2085 from zymap/compatible_protocol --- .../bookkeeper/proto/BookieClientImpl.java | 21 +++++-- .../DefaultPerChannelBookieClientPool.java | 62 +++++++++++++++---- .../proto/PerChannelBookieClientFactory.java | 5 +- .../proto/PerChannelBookieClientPool.java | 10 +++ .../bookkeeper/client/ExplicitLacTest.java | 20 ++++++ 5 files changed, 98 insertions(+), 20 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index c772a97cdaa..9cd52e1ebd7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -93,6 +93,8 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFac private final ExtensionRegistry registry; private final ClientConfiguration conf; + private final ClientConfiguration v3Conf; + private final boolean useV3Enforced; private volatile boolean closed; private final ReentrantReadWriteLock closeLock; private final StatsLogger statsLogger; @@ -105,6 +107,9 @@ public BookieClientImpl(ClientConfiguration conf, EventLoopGroup eventLoopGroup, OrderedExecutor executor, ScheduledExecutorService scheduler, StatsLogger statsLogger) throws IOException { this.conf = conf; + this.v3Conf = new ClientConfiguration(conf); + this.v3Conf.setUseV2WireProtocol(false); + this.useV3Enforced = conf.getUseV2WireProtocol(); this.eventLoopGroup = eventLoopGroup; this.allocator = allocator; this.executor = executor; @@ -179,13 +184,17 @@ public long getNumPendingRequests(BookieSocketAddress address, long ledgerId) { @Override public PerChannelBookieClient create(BookieSocketAddress address, PerChannelBookieClientPool pcbcPool, - SecurityHandlerFactory shFactory) throws SecurityException { + SecurityHandlerFactory shFactory, boolean forceUseV3) throws SecurityException { StatsLogger statsLoggerForPCBC = statsLogger; if (conf.getLimitStatsLogging()) { statsLoggerForPCBC = NullStatsLogger.INSTANCE; } - return new PerChannelBookieClient(conf, executor, eventLoopGroup, allocator, address, statsLoggerForPCBC, - authProviderFactory, registry, pcbcPool, shFactory); + ClientConfiguration clientConfiguration = conf; + if (forceUseV3) { + clientConfiguration = v3Conf; + } + return new PerChannelBookieClient(clientConfiguration, executor, eventLoopGroup, allocator, address, + statsLoggerForPCBC, authProviderFactory, registry, pcbcPool, shFactory); } public PerChannelBookieClientPool lookupClient(BookieSocketAddress addr) { @@ -267,7 +276,7 @@ public void writeLac(final BookieSocketAddress addr, final long ledgerId, final } toSend.release(); - }, ledgerId); + }, ledgerId, useV3Enforced); } private void completeAdd(final int rc, @@ -467,7 +476,7 @@ public void readLac(final BookieSocketAddress addr, final long ledgerId, final R } else { pcbc.readLac(ledgerId, cb, ctx); } - }, ledgerId); + }, ledgerId, useV3Enforced); } public void readEntry(BookieSocketAddress addr, long ledgerId, long entryId, @@ -546,7 +555,7 @@ public void getBookieInfo(final BookieSocketAddress addr, final long requested, } else { pcbc.getBookieInfo(requested, cb, ctx); } - }, requested); + }, requested, useV3Enforced); } private void monitorPendingOperations() { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DefaultPerChannelBookieClientPool.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DefaultPerChannelBookieClientPool.java index 04471d5f55e..71193de3f71 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DefaultPerChannelBookieClientPool.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DefaultPerChannelBookieClientPool.java @@ -48,6 +48,7 @@ class DefaultPerChannelBookieClientPool implements PerChannelBookieClientPool, final BookieSocketAddress address; final PerChannelBookieClient[] clients; + final PerChannelBookieClient[] clientsV3Enforced; final ClientConfiguration conf; SecurityHandlerFactory shFactory; @@ -63,12 +64,20 @@ class DefaultPerChannelBookieClientPool implements PerChannelBookieClientPool, this.address = address; this.conf = conf; - this.shFactory = SecurityProviderFactoryFactory - .getSecurityProviderFactory(conf.getTLSProviderFactoryClass()); + this.shFactory = SecurityProviderFactoryFactory.getSecurityProviderFactory(conf.getTLSProviderFactoryClass()); this.clients = new PerChannelBookieClient[coreSize]; for (int i = 0; i < coreSize; i++) { - this.clients[i] = factory.create(address, this, shFactory); + this.clients[i] = factory.create(address, this, shFactory, false); + } + + if (conf.getUseV2WireProtocol()) { + this.clientsV3Enforced = new PerChannelBookieClient[coreSize]; + for (int i = 0; i < coreSize; i++) { + this.clientsV3Enforced[i] = factory.create(address, this, shFactory, true); + } + } else { + this.clientsV3Enforced = this.clients; } } @@ -85,16 +94,31 @@ public void intialize() { } private PerChannelBookieClient getClient(long key) { - if (1 == clients.length) { - return clients[0]; + return getClient(key, false); + } + + private PerChannelBookieClient getClient(long key, PerChannelBookieClient[] pcbc) { + if (1 == pcbc.length) { + return pcbc[0]; } - int idx = MathUtils.signSafeMod(key, clients.length); - return clients[idx]; + int idx = MathUtils.signSafeMod(key, pcbc.length); + return pcbc[idx]; + } + private PerChannelBookieClient getClient(long key, boolean forceUseV3) { + if (forceUseV3) { + return getClient(key, clientsV3Enforced); + } + return getClient(key, clients); } @Override public void obtain(GenericCallback callback, long key) { - getClient(key).connectIfNeededAndDoOp(callback); + obtain(callback, key, false); + } + + @Override + public void obtain(GenericCallback callback, long key, boolean forceUseV3) { + getClient(key, forceUseV3).connectIfNeededAndDoOp(callback); } @Override @@ -106,6 +130,9 @@ public boolean isWritable(long key) { public void checkTimeoutOnPendingOperations() { for (int i = 0; i < clients.length; i++) { clients[i].checkTimeoutOnPendingOperations(); + if (clients != clientsV3Enforced) { + clientsV3Enforced[i].checkTimeoutOnPendingOperations(); + } } } @@ -116,15 +143,21 @@ public void recordError() { @Override public void disconnect(boolean wait) { - for (PerChannelBookieClient pcbc : clients) { - pcbc.disconnect(wait); + for (int i = 0; i < clients.length; i++) { + clients[i].disconnect(); + if (clients != clientsV3Enforced) { + clientsV3Enforced[i].disconnect(); + } } } @Override public void close(boolean wait) { - for (PerChannelBookieClient pcbc : clients) { - pcbc.close(wait); + for (int i = 0; i < clients.length; i++) { + clients[i].close(wait); + if (clients != clientsV3Enforced) { + clientsV3Enforced[i].close(wait); + } } } @@ -134,6 +167,11 @@ public long getNumPendingCompletionRequests() { for (PerChannelBookieClient pcbc : clients) { numPending += pcbc.getNumPendingCompletionRequests(); } + if (clients != clientsV3Enforced) { + for (PerChannelBookieClient pcbc : clientsV3Enforced) { + numPending += pcbc.getNumPendingCompletionRequests(); + } + } return numPending; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientFactory.java index 17abb565827..48797cd968d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientFactory.java @@ -36,6 +36,7 @@ interface PerChannelBookieClientFactory { * @return the client connected to address. * @throws SecurityException */ - PerChannelBookieClient create(BookieSocketAddress address, - PerChannelBookieClientPool pcbcPool, SecurityHandlerFactory shFactory) throws SecurityException; + PerChannelBookieClient create(BookieSocketAddress address, PerChannelBookieClientPool pcbcPool, + SecurityHandlerFactory shFactory, + boolean forceUseV3) throws SecurityException; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientPool.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientPool.java index aa7a5e94492..c7bc0053ccb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientPool.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientPool.java @@ -40,6 +40,16 @@ public interface PerChannelBookieClientPool { */ void obtain(GenericCallback callback, long key); + /** + * Obtain a channel from channel pool by version to execute operations. + * + * @param callback + * callback to return channel from channel pool + * @param forceUseV3 + * whether or not use v3 protocol for connection + */ + void obtain(GenericCallback callback, long key, boolean forceUseV3); + /** * Returns status of a client. * It is suggested to delay/throttle requests to this channel if isWritable is false. diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ExplicitLacTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ExplicitLacTest.java index 5c5c24b31ef..f92cbce3574 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ExplicitLacTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ExplicitLacTest.java @@ -340,5 +340,25 @@ public void testReadHandleWithExplicitLACAndDeferredSync() throws Exception { bkcWithExplicitLAC.close(); } + @Test + public void fallbackV3() throws Exception { + ClientConfiguration v2Conf = new ClientConfiguration(); + v2Conf.setUseV2WireProtocol(true); + v2Conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + v2Conf.setExplictLacInterval(10); + + BookKeeper bookKeeper = new BookKeeper(v2Conf); + LedgerHandle write = (LedgerHandle) bookKeeper.createLedger(1, + 1, + 1, + DigestType.MAC, + "pass".getBytes()); + write.addEntry("test".getBytes()); + TestUtils.waitUntilExplicitLacUpdated(write, 0); + long lac = write.readExplicitLastConfirmed(); + assertEquals(0, lac); + write.close(); + bookKeeper.close(); + } } From 3846fca9c2f6db2d8f671496e4e3a3862662a246 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Sun, 19 May 2019 19:53:36 -0700 Subject: [PATCH 0299/1642] Fix flaky testIOVertexHTTPServerEndpointForARWithPrometheusProvider test Descriptions of the changes in this PR: - in testIOVertexHTTPServerEndpointForARWithPrometheusProvider test it is not correct to assume that Auditor would have been created and started completely when we complete AutoRecoveryMain.start and see it status as started. It would make sure AuditorElector.submitElectionTask has submitted election task but not the completion of Auditor.start. So instead of relying on Auditor metric (NUM_UNDER_REPLICATED_LEDGERS), use ReplicationWorker metric - NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED. Reviewers: Enrico Olivelli , Sijie Guo This closes #2093 from reddycharan/fixflakytest --- .../org/apache/bookkeeper/bookie/BookieInitializationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 dbaef35b735..8e0561b20ee 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 @@ -1318,7 +1318,7 @@ public void testIOVertexHTTPServerEndpointForARWithPrometheusProvider() throws E String metrics = metricsStringBuilder.toString(); // do primitive checks if metrics string contains some stats assertTrue("Metrics should contain basic counters", - metrics.contains(ReplicationStats.NUM_UNDER_REPLICATED_LEDGERS)); + metrics.contains(ReplicationStats.NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED)); // Now, hit the rest endpoint for configs url = new URL("http://localhost:" + nextFreePort + HttpRouter.SERVER_CONFIG); From 0d0ad386264a8d9c88630c363d1e6c4370628099 Mon Sep 17 00:00:00 2001 From: karanmehta93 Date: Fri, 31 May 2019 09:49:13 -0500 Subject: [PATCH 0300/1642] Update lastLogMark to EOF when replaying journal Descriptions of the changes in this PR: ### Motivation The [commit](https://github.com/apache/bookkeeper/commit/36be8362399341022c8de64f9319270726df2cb3) caused integration test failure `test101_RegenerateIndex`, with the exception ``` ```java.io.IOException: Invalid argument at sun.nio.ch.FileDispatcherImpl.read0(Native Method) at sun.nio.ch.FileDispatcherImpl.read(FileDispatcherImpl.java:46) at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223) at sun.nio.ch.IOUtil.read(IOUtil.java:197) at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:159) at org.apache.bookkeeper.bookie.JournalChannel.read(JournalChannel.java:257) at org.apache.bookkeeper.bookie.Journal.fullRead(Journal.java:1171) at org.apache.bookkeeper.bookie.Journal.scanJournal(Journal.java:792) at org.apache.bookkeeper.bookie.Bookie.replay(Bookie.java:924) at org.apache.bookkeeper.bookie.Bookie.readJournal(Bookie.java:886) at org.apache.bookkeeper.bookie.Bookie.start(Bookie.java:943) at org.apache.bookkeeper.proto.BookieServer.start(BookieServer.java:141) at org.apache.bookkeeper.server.service.BookieService.doStart(BookieService.java:58) at org.apache.bookkeeper.common.component.AbstractLifecycleComponent.start(AbstractLifecycleComponent.java:78) at org.apache.bookkeeper.common.component.LifecycleComponentStack.lambda$start$2(LifecycleComponentStack.java:113) at com.google.common.collect.ImmutableList.forEach(ImmutableList.java:408) at org.apache.bookkeeper.common.component.LifecycleComponentStack.start(LifecycleComponentStack.java:113) at org.apache.bookkeeper.common.component.ComponentStarter.startComponent(ComponentStarter.java:80) at org.apache.bookkeeper.server.Main.doMain(Main.java:229) at org.apache.bookkeeper.server.Main.main(Main.java:203) ``` As discussed on slack, it is hard to figure out an exact reason as to why the native JNI call fails with an invalid argument. Hence this PR proposes that the `lastLogMark` is updated to journal EOF instead of an arbitrary LONG.MAX_VALUE. The FileChannel interface defines that the implementors can pass in any long offset and the file handler should return EOF immediately when trying to read it. However it doesn't seem to be working as expected. ### Changes Updated `Journal#setLastLogMark()` method to accept an `scanOffset` instead of constant `LONG.MAX_VALUE`. ivankelly eolivelli Reviewers: Ivan Kelly , Enrico Olivelli This closes #2105 from karanmehta93/master --- .../java/org/apache/bookkeeper/bookie/Bookie.java | 7 ++++--- .../java/org/apache/bookkeeper/bookie/Journal.java | 11 +++++++---- 2 files changed, 11 insertions(+), 7 deletions(-) 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 467cb037095..9bb9e5deaef 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 @@ -921,10 +921,11 @@ private void replay(Journal journal, JournalScanner scanner) throws IOException logPosition = markedLog.getLogFileOffset(); } LOG.info("Replaying journal {} from position {}", id, logPosition); - journal.scanJournal(id, logPosition, scanner); - // Update LastLogMark to Long.MAX_VALUE position after replaying journal + long scanOffset = journal.scanJournal(id, logPosition, scanner); + // Update LastLogMark after completely replaying journal + // scanOffset will point to EOF position // After LedgerStorage flush, SyncThread should persist this to disk - journal.setLastLogMarkToEof(id); + journal.setLastLogMark(id, scanOffset); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 8941b685faf..06fc5e2a98c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -710,12 +710,13 @@ public LastLogMark getLastLogMark() { } /** - * Update lastLogMark of the journal, + * Update lastLogMark of the journal * Indicates that the file has been processed. * @param id + * @param scanOffset */ - void setLastLogMarkToEof(Long id) { - lastLogMark.getCurMark().setLogMark(id, Long.MAX_VALUE); + void setLastLogMark(Long id, long scanOffset) { + lastLogMark.setCurLogMark(id, scanOffset); } /** @@ -769,9 +770,10 @@ public void checkpointComplete(Checkpoint checkpoint, boolean compact) throws IO * @param journalId Journal Log Id * @param journalPos Offset to start scanning * @param scanner Scanner to handle entries + * @return scanOffset - represents the byte till which journal was read * @throws IOException */ - public void scanJournal(long journalId, long journalPos, JournalScanner scanner) + public long scanJournal(long journalId, long journalPos, JournalScanner scanner) throws IOException { JournalChannel recLog; if (journalPos <= 0) { @@ -832,6 +834,7 @@ public void scanJournal(long journalId, long journalPos, JournalScanner scanner) scanner.process(journalVersion, offset, recBuff); } } + return recLog.fc.position(); } finally { recLog.close(); } From 00e38a80be66149e51cc384b691d070e22c62b7c Mon Sep 17 00:00:00 2001 From: Ankit Jain Date: Fri, 31 May 2019 17:24:55 -0700 Subject: [PATCH 0301/1642] Entries must be acknowledged by bookies in multiple fault domains before being acknowledged to client Descriptions of the changes in this PR: Bookkeeper write logic makes sure that there are at least ackQuorumSize number of successful writes before sending ack back to the client. In many cases these may fall into the same fault domain. A mechanism to force bookkeeper to make sure that there are acks from at least minNumRacksPerWriteQuorum number of fault domains and a configuration to enforce this. Signed-off-by: Ankit Jain Master Issue: #2095 Reviewers: Charan Reddy Guttapalem , Enrico Olivelli This closes #2096 from ankit-j/ankit-j/enforceFragmentMultipleFaultDomainWrite and squashes the following commits: c90fd5a3d [Ankit Jain] Addressing review comments 07deae673 [Ankit Jain] Addressing @reddycharan's review comments 14164e291 [Ankit Jain] Fixed spacing error in bk_server.yaml 22c8b3c03 [Ankit Jain] Updated testing. 917ed1c45 [Ankit Jain] Move readLock.unlock to finally block 78e0cd501 [Ankit Jain] Modify test to not use default rack for bookies ca0bc3b8b [Ankit Jain] Entries must be acknowledged by bookies in multiple fault domains before being acknowledged to client d35aa22ad [Charan Reddy Guttapalem] Move common placementpolicy components to TopologyAwareEnsemblePlacementPolicy. --- .../client/BookKeeperClientStats.java | 7 + .../bookkeeper/client/ClientInternalConf.java | 2 + .../client/EnsemblePlacementPolicy.java | 19 +++ .../bookkeeper/client/PendingAddOp.java | 48 +++++- .../RackawareEnsemblePlacementPolicyImpl.java | 29 ++++ .../impl/BookKeeperClientStatsImpl.java | 36 ++++ .../conf/AbstractConfiguration.java | 17 ++ .../bookkeeper/client/BookKeeperTest.java | 156 ++++++++++++++++++ .../TestRackawareEnsemblePlacementPolicy.java | 91 ++++++++++ conf/bk_server.conf | 5 + site/_data/config/bk_server.yaml | 3 + 11 files changed, 410 insertions(+), 3 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java index 36c304c5c27..ed10b9cfe64 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java @@ -97,6 +97,10 @@ public interface BookKeeperClientStats { // placementpolicy stats String NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK = "NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK"; + String WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS = "WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS"; + String WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS_LATENCY = + "WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS_LATENCY"; + String WRITE_TIMED_OUT_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS = "WRITE_TIME_OUT_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS"; OpStatsLogger getCreateOpLogger(); OpStatsLogger getOpenOpLogger(); @@ -119,6 +123,9 @@ public interface BookKeeperClientStats { Counter getLacUpdateHitsCounter(); Counter getLacUpdateMissesCounter(); OpStatsLogger getClientChannelWriteWaitLogger(); + OpStatsLogger getWriteDelayedDueToNotEnoughFaultDomainsLatency(); + Counter getWriteDelayedDueToNotEnoughFaultDomains(); + Counter getWriteTimedOutDueToNotEnoughFaultDomains(); void registerPendingAddsGauge(Gauge gauge); static BookKeeperClientStats newInstance(StatsLogger stats) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java index da79108deb5..85d42c60268 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java @@ -49,6 +49,7 @@ class ClientInternalConf { final long timeoutMonitorIntervalSec; final boolean enableBookieFailureTracking; final boolean useV2WireProtocol; + final boolean enforceMinNumFaultDomainsForWrite; static ClientInternalConf defaultValues() { return fromConfig(new ClientConfiguration()); @@ -82,6 +83,7 @@ private ClientInternalConf(ClientConfiguration conf, this.enableBookieFailureTracking = conf.getEnableBookieFailureTracking(); this.useV2WireProtocol = conf.getUseV2WireProtocol(); this.enableStickyReads = conf.isStickyReadsEnabled(); + this.enforceMinNumFaultDomainsForWrite = conf.getEnforceMinNumFaultDomainsForWrite(); if (conf.getFirstSpeculativeReadTimeout() > 0) { this.readSpeculativeRequestPolicy = 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 7dc81112531..64a4b91bd53 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 @@ -414,6 +414,25 @@ default boolean isEnsembleAdheringToPlacementPolicy(List en return false; } + /** + * Returns true if the bookies that have acknowledged a write adhere to the minimum fault domains as defined in the + * placement policy in use. Ex: In the case of RackawareEnsemblePlacementPolicy, bookies belong to at least + * 'minNumRacksPerWriteQuorum' number of racks. + * + * @param ackedBookies + * list of BookieSocketAddress of bookies that have acknowledged a write. + * @param writeQuorumSize + * writeQuorumSize of the ensemble + * @param ackQuorumSize + * ackQuorumSize of the ensemble + * @return + */ + default boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, + int writeQuorumSize, + int ackQuorumSize) { + return true; + } + /** * Result of a placement calculation against a placement policy. */ 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 ad2f7ae4f07..ff806155e65 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 @@ -30,8 +30,10 @@ import io.netty.util.ReferenceCountUtil; import java.util.EnumSet; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.AsyncCallback.AddCallbackWithLatency; @@ -71,6 +73,8 @@ class PendingAddOp extends SafeRunnable implements WriteCallback { boolean isRecoveryAdd = false; long requestTimeNanos; long qwcLatency; // Quorum Write Completion Latency after response from quorum bookies. + Set addEntrySuccessBookies; + long writeDelayedStartTime; // min fault domains completion latency after response from ack quorum bookies long currentLedgerLength; int pendingWriteRequests; @@ -106,6 +110,13 @@ static PendingAddOp create(LedgerHandle lh, ClientContext clientCtx, op.qwcLatency = 0; op.writeFlags = writeFlags; + if (op.addEntrySuccessBookies == null) { + op.addEntrySuccessBookies = new HashSet<>(); + } else { + op.addEntrySuccessBookies.clear(); + } + op.writeDelayedStartTime = -1; + return op; } @@ -159,6 +170,11 @@ void timeoutQuorumWait() { public void safeRun() { if (completed) { return; + } else if (addEntrySuccessBookies.size() >= lh.getLedgerMetadata().getAckQuorumSize()) { + // If ackQuorum number of bookies have acknowledged the write but still not complete, indicates + // failures due to not having been written to enough fault domains. Increment corresponding + // counter. + clientCtx.getClientStats().getWriteTimedOutDueToNotEnoughFaultDomains().inc(); } lh.handleUnrecoverableErrorDuringAdd(BKException.Code.AddEntryQuorumTimeoutException); } @@ -282,6 +298,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre boolean ackQuorum = false; if (BKException.Code.OK == rc) { ackQuorum = ackSet.completeBookieAndCheck(bookieIndex); + addEntrySuccessBookies.add(ensemble.get(bookieIndex)); } if (completed) { @@ -363,10 +380,33 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre } if (ackQuorum && !completed) { - completed = true; - this.qwcLatency = MathUtils.elapsedNanos(requestTimeNanos); + if (clientCtx.getConf().enforceMinNumFaultDomainsForWrite + && !(clientCtx.getPlacementPolicy() + .areAckedBookiesAdheringToPlacementPolicy(addEntrySuccessBookies, + lh.getLedgerMetadata().getWriteQuorumSize(), + lh.getLedgerMetadata().getAckQuorumSize()))) { + LOG.warn("Write success for entry ID {} delayed, not acknowledged by bookies in enough fault domains", + entryId); + // Increment to indicate write did not complete due to not enough fault domains + clientCtx.getClientStats().getWriteDelayedDueToNotEnoughFaultDomains().inc(); + + // Only do this for the first time. + if (writeDelayedStartTime == -1) { + writeDelayedStartTime = MathUtils.nowInNano(); + } + } else { + completed = true; + this.qwcLatency = MathUtils.elapsedNanos(requestTimeNanos); + + if (writeDelayedStartTime != -1) { + clientCtx.getClientStats() + .getWriteDelayedDueToNotEnoughFaultDomainsLatency() + .registerSuccessfulEvent(MathUtils.elapsedNanos(writeDelayedStartTime), + TimeUnit.NANOSECONDS); + } - sendAddSuccessCallbacks(); + sendAddSuccessCallbacks(); + } } } @@ -478,6 +518,8 @@ private void recyclePendAddOpObject() { hasRun = false; allowFailFast = false; writeFlags = null; + addEntrySuccessBookies.clear(); + writeDelayedStartTime = -1; recyclerHandle.recycle(this); } 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 2b1090f9ed6..86fc3919608 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 @@ -45,6 +45,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; @@ -1060,4 +1061,32 @@ public boolean isEnsembleAdheringToPlacementPolicy(List ens } return true; } + + @Override + public boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, + int writeQuorumSize, + int ackQuorumSize) { + HashSet rackCounter = new HashSet<>(); + int minWriteQuorumNumRacksPerWriteQuorum = Math.min(writeQuorumSize, minNumRacksPerWriteQuorum); + + ReentrantReadWriteLock.ReadLock readLock = rwLock.readLock(); + readLock.lock(); + try { + for (BookieSocketAddress bookie : ackedBookies) { + rackCounter.add(knownBookies.get(bookie).getNetworkLocation()); + } + + // Check to make sure that ensemble is writing to `minNumberOfRacks`'s number of racks at least. + if (LOG.isDebugEnabled()) { + LOG.debug("areAckedBookiesAdheringToPlacementPolicy returning {} because number of racks = {} and " + + "minNumRacksPerWriteQuorum = {}", + rackCounter.size() >= minNumRacksPerWriteQuorum, + rackCounter.size(), + minNumRacksPerWriteQuorum); + } + } finally { + readLock.unlock(); + } + return rackCounter.size() >= minWriteQuorumNumRacksPerWriteQuorum; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperClientStatsImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperClientStatsImpl.java index a29621a0d00..811a4b85b28 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperClientStatsImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperClientStatsImpl.java @@ -141,6 +141,24 @@ public class BookKeeperClientStatsImpl implements BookKeeperClientStats { ) private final Counter speculativeReadCounter; + @StatsDoc( + name = WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS_LATENCY, + help = "The delay in write completion because min number of fault domains was not reached" + ) + private final OpStatsLogger writeDelayedDueToNotEnoughFaultDomainsLatency; + + @StatsDoc( + name = WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS, + help = "The number of times write completion was delayed because min number of fault domains was not reached" + ) + private final Counter writeDelayedDueToNotEnoughFaultDomains; + + @StatsDoc( + name = WRITE_TIMED_OUT_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS, + help = "The number of times write completion timed out because min number of fault domains was not reached" + ) + private final Counter writeTimedOutDueToNotEnoughFaultDomains; + public BookKeeperClientStatsImpl(StatsLogger stats) { this.stats = stats; @@ -166,6 +184,12 @@ public BookKeeperClientStatsImpl(StatsLogger stats) { this.clientChannelWriteWaitStats = stats.getOpStatsLogger(CLIENT_CHANNEL_WRITE_WAIT); speculativeReadCounter = stats.getCounter(SPECULATIVE_READ_COUNT); + + this.writeDelayedDueToNotEnoughFaultDomainsLatency = + stats.getOpStatsLogger(WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS_LATENCY); + this.writeDelayedDueToNotEnoughFaultDomains = stats.getCounter(WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS); + this.writeTimedOutDueToNotEnoughFaultDomains = + stats.getCounter(WRITE_TIMED_OUT_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS); } @Override @@ -253,6 +277,18 @@ public Counter getEnsembleBookieDistributionCounter(String bookie) { return stats.getCounter(LEDGER_ENSEMBLE_BOOKIE_DISTRIBUTION + "-" + bookie); } @Override + public OpStatsLogger getWriteDelayedDueToNotEnoughFaultDomainsLatency() { + return writeDelayedDueToNotEnoughFaultDomainsLatency; + } + @Override + public Counter getWriteDelayedDueToNotEnoughFaultDomains() { + return writeDelayedDueToNotEnoughFaultDomains; + } + @Override + public Counter getWriteTimedOutDueToNotEnoughFaultDomains() { + return writeTimedOutDueToNotEnoughFaultDomains; + } + @Override public void registerPendingAddsGauge(Gauge gauge) { stats.registerGauge(PENDING_ADDS, gauge); } 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 87c1f415461..b666dbce304 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 @@ -159,6 +159,9 @@ public abstract class AbstractConfiguration // enforce minimum number of racks per write quorum public static final String ENFORCE_MIN_NUM_RACKS_PER_WRITE_QUORUM = "enforceMinNumRacksPerWriteQuorum"; + // enforce minimum number of fault domains for write + public static final String ENFORCE_MIN_NUM_FAULT_DOMAINS_FOR_WRITE = "enforceMinNumFaultDomainsForWrite"; + // ignore usage of local node in the internal logic of placement policy public static final String IGNORE_LOCAL_NODE_IN_PLACEMENT_POLICY = "ignoreLocalNodeInPlacementPolicy"; @@ -846,6 +849,20 @@ public boolean getEnforceMinNumRacksPerWriteQuorum() { return getBoolean(ENFORCE_MIN_NUM_RACKS_PER_WRITE_QUORUM, false); } + /** + * Set the flag to enforce minimum number of fault domains for write. + */ + public void setEnforceMinNumFaultDomainsForWrite(boolean enforceMinNumFaultDomainsForWrite) { + setProperty(ENFORCE_MIN_NUM_FAULT_DOMAINS_FOR_WRITE, enforceMinNumFaultDomainsForWrite); + } + + /** + * Get the flag to enforce minimum number of fault domains for write. + */ + public boolean getEnforceMinNumFaultDomainsForWrite() { + return getBoolean(ENFORCE_MIN_NUM_FAULT_DOMAINS_FOR_WRITE, false); + } + /** * Sets the flag to ignore usage of localnode in placement policy. */ 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 80d059d6e27..3c842e6ad0c 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 @@ -20,6 +20,8 @@ */ package org.apache.bookkeeper.client; +import static org.apache.bookkeeper.client.BookKeeperClientStats.WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS; +import static org.apache.bookkeeper.client.BookKeeperClientStats.WRITE_TIMED_OUT_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -31,9 +33,11 @@ import io.netty.util.IllegalReferenceCountException; import java.io.IOException; +import java.net.UnknownHostException; import java.util.Collections; import java.util.Enumeration; import java.util.List; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -48,8 +52,12 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.api.WriteHandle; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase; @@ -956,4 +964,152 @@ public void testLedgerDeletionIdempotency() throws Exception { bk.deleteLedger(ledgerId); bk.close(); } + + /** + * Mock of RackawareEnsemblePlacementPolicy. Overrides areAckedBookiesAdheringToPlacementPolicy to only return true + * when ackedBookies consists of writeQuorumSizeToUseForTesting bookies. + */ + public static class MockRackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacementPolicy { + private int writeQuorumSizeToUseForTesting; + private CountDownLatch conditionFirstInvocationLatch; + + void setWriteQuorumSizeToUseForTesting(int writeQuorumSizeToUseForTesting) { + this.writeQuorumSizeToUseForTesting = writeQuorumSizeToUseForTesting; + } + + void setConditionFirstInvocationLatch(CountDownLatch conditionFirstInvocationLatch) { + this.conditionFirstInvocationLatch = conditionFirstInvocationLatch; + } + + @Override + public boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, + int writeQuorumSize, + int ackQuorumSize) { + conditionFirstInvocationLatch.countDown(); + return ackedBookies.size() == writeQuorumSizeToUseForTesting; + } + } + + /** + * Test to verify that PendingAddOp waits for success condition from areAckedBookiesAdheringToPlacementPolicy + * before returning success to client. Also tests working of WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS and + * WRITE_TIMED_OUT_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS counters. + */ + @Test + public void testEnforceMinNumFaultDomainsForWrite() throws Exception { + byte[] data = "foobar".getBytes(); + byte[] password = "testPasswd".getBytes(); + + startNewBookie(); + startNewBookie(); + + ClientConfiguration conf = new ClientConfiguration(); + conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + conf.setEnsemblePlacementPolicy(MockRackawareEnsemblePlacementPolicy.class); + + conf.setAddEntryTimeout(2); + conf.setAddEntryQuorumTimeout(4); + conf.setEnforceMinNumFaultDomainsForWrite(true); + + TestStatsProvider statsProvider = new TestStatsProvider(); + + // Abnormal values for testing to prevent timeouts + BookKeeperTestClient bk = new BookKeeperTestClient(conf, statsProvider); + StatsLogger statsLogger = bk.getStatsLogger(); + + int ensembleSize = 3; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + + CountDownLatch countDownLatch = new CountDownLatch(1); + MockRackawareEnsemblePlacementPolicy currPlacementPolicy = + (MockRackawareEnsemblePlacementPolicy) bk.getPlacementPolicy(); + currPlacementPolicy.setConditionFirstInvocationLatch(countDownLatch); + currPlacementPolicy.setWriteQuorumSizeToUseForTesting(writeQuorumSize); + + BookieSocketAddress bookieToSleep; + + try (LedgerHandle lh = bk.createLedger(ensembleSize, writeQuorumSize, ackQuorumSize, digestType, password)) { + CountDownLatch sleepLatchCase1 = new CountDownLatch(1); + CountDownLatch sleepLatchCase2 = new CountDownLatch(1); + + // Put all non ensemble bookies to sleep + LOG.info("Putting all non ensemble bookies to sleep."); + for (BookieServer bookieServer : bs) { + try { + if (!lh.getCurrentEnsemble().contains(bookieServer.getLocalAddress())) { + sleepBookie(bookieServer.getLocalAddress(), sleepLatchCase2); + } + } catch (UnknownHostException ignored) {} + } + + Thread writeToLedger = new Thread(() -> { + try { + LOG.info("Initiating write for entry"); + long entryId = lh.addEntry(data); + LOG.info("Wrote entry with entryId = {}", entryId); + } catch (InterruptedException | BKException ignored) { + } + }); + + bookieToSleep = lh.getCurrentEnsemble().get(0); + + LOG.info("Putting picked bookie to sleep"); + sleepBookie(bookieToSleep, sleepLatchCase1); + + assertEquals(statsLogger + .getCounter(WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS) + .get() + .longValue(), 0); + + // Trying to write entry + writeToLedger.start(); + + // Waiting and checking to make sure that write has not succeeded + countDownLatch.await(conf.getAddEntryTimeout(), TimeUnit.SECONDS); + assertEquals("Write succeeded but should not have", -1, lh.lastAddConfirmed); + + // Wake the bookie + sleepLatchCase1.countDown(); + + // Waiting and checking to make sure that write has succeeded + writeToLedger.join(conf.getAddEntryTimeout() * 1000); + assertEquals("Write did not succeed but should have", 0, lh.lastAddConfirmed); + + assertEquals(statsLogger + .getCounter(WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS) + .get() + .longValue(), 1); + + // AddEntry thread for second scenario + Thread writeToLedger2 = new Thread(() -> { + try { + LOG.info("Initiating write for entry"); + long entryId = lh.addEntry(data); + LOG.info("Wrote entry with entryId = {}", entryId); + } catch (InterruptedException | BKException ignored) { + } + }); + + bookieToSleep = lh.getCurrentEnsemble().get(1); + + LOG.info("Putting picked bookie to sleep"); + sleepBookie(bookieToSleep, sleepLatchCase2); + + // Trying to write entry + writeToLedger2.start(); + + // Waiting and checking to make sure that write has failed + writeToLedger2.join((conf.getAddEntryQuorumTimeout() + 2) * 1000); + assertEquals("Write succeeded but should not have", 0, lh.lastAddConfirmed); + + sleepLatchCase2.countDown(); + + assertEquals(statsLogger.getCounter(WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS).get().longValue(), + 2); + + assertEquals(statsLogger.getCounter(WRITE_TIMED_OUT_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS).get().longValue(), + 1); + } + } } 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 4a93d4f6fb1..3fc74839c98 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 @@ -2233,4 +2233,95 @@ public void testNewEnsembleExcludesDefaultRackBookiesEnforceMinNumRacks() throws + bookiesOfDefaultRackInEnsemble, bookiesOfDefaultRackInEnsemble.isEmpty()); } } + + private void testAreAckedBookiesAdheringToPlacementPolicyHelper(int minNumRacksPerWriteQuorumConfValue, + int ensembleSize, + int writeQuorumSize, + int ackQuorumSize, + int numOfBookiesInDefaultRack, + int numOfRacks, + int numOfBookiesPerRack) throws Exception { + String defaultRackForThisTest = NetworkTopology.DEFAULT_REGION_AND_RACK; + repp.uninitalize(); + updateMyRack(defaultRackForThisTest); + + ClientConfiguration conf = new ClientConfiguration(this.conf); + conf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); + + repp = new RackawareEnsemblePlacementPolicy(); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, statsLogger); + repp.withDefaultRack(defaultRackForThisTest); + + List bookieSocketAddressesDefaultRack = new ArrayList<>(); + List bookieSocketAddressesNonDefaultRack = new ArrayList<>(); + Set writableBookies; + Set bookiesForEntry = new HashSet<>(); + + for (int i = 0; i < numOfRacks; i++) { + for (int j = 0; j < numOfBookiesPerRack; j++) { + int index = i * numOfBookiesPerRack + j; + bookieSocketAddressesNonDefaultRack.add(new BookieSocketAddress("128.0.0." + index, 3181)); + StaticDNSResolver.addNodeToRack(bookieSocketAddressesNonDefaultRack.get(index).getHostName(), + "/default-region/r" + i); + } + } + + for (int i = 0; i < numOfBookiesInDefaultRack; i++) { + bookieSocketAddressesDefaultRack.add(new BookieSocketAddress("127.0.0." + (i + 100), 3181)); + StaticDNSResolver.addNodeToRack(bookieSocketAddressesDefaultRack.get(i).getHostName(), + defaultRackForThisTest); + } + + writableBookies = new HashSet<>(bookieSocketAddressesNonDefaultRack); + writableBookies.addAll(bookieSocketAddressesDefaultRack); + repp.onClusterChanged(writableBookies, new HashSet<>()); + + // Case 1 : Bookies in the ensemble from the same rack. + // Manually crafting the ensemble here to create the error case when the check should return false + + List ensemble = new ArrayList<>(bookieSocketAddressesDefaultRack); + for (int entryId = 0; entryId < 10; entryId++) { + DistributionSchedule ds = new RoundRobinDistributionSchedule(writeQuorumSize, ackQuorumSize, ensembleSize); + DistributionSchedule.WriteSet ws = ds.getWriteSet(entryId); + + for (int i = 0; i < ws.size(); i++) { + bookiesForEntry.add(ensemble.get(ws.get(i))); + } + + assertFalse(repp.areAckedBookiesAdheringToPlacementPolicy(bookiesForEntry, writeQuorumSize, ackQuorumSize)); + } + + // Case 2 : Bookies in the ensemble from the different racks + + EnsemblePlacementPolicy.PlacementResult> + ensembleResponse = repp.newEnsemble(ensembleSize, + writeQuorumSize, + ackQuorumSize, + null, + new HashSet<>()); + ensemble = ensembleResponse.getResult(); + for (int entryId = 0; entryId < 10; entryId++) { + DistributionSchedule ds = new RoundRobinDistributionSchedule(writeQuorumSize, ackQuorumSize, ensembleSize); + DistributionSchedule.WriteSet ws = ds.getWriteSet(entryId); + + for (int i = 0; i < ws.size(); i++) { + bookiesForEntry.add(ensemble.get(ws.get(i))); + } + + assertTrue(repp.areAckedBookiesAdheringToPlacementPolicy(bookiesForEntry, writeQuorumSize, ackQuorumSize)); + } + } + + /** + * This tests areAckedBookiesAdheringToPlacementPolicy function in RackawareEnsemblePlacementPolicy. + */ + @Test + public void testAreAckedBookiesAdheringToPlacementPolicy() throws Exception { + testAreAckedBookiesAdheringToPlacementPolicyHelper(2, 7, 3, 2, 7, 3, 3); + testAreAckedBookiesAdheringToPlacementPolicyHelper(4, 6, 3, 2, 6, 3, 3); + testAreAckedBookiesAdheringToPlacementPolicyHelper(5, 7, 5, 3, 7, 5, 2); + } } diff --git a/conf/bk_server.conf b/conf/bk_server.conf index 41798a6df4b..fa77b949a86 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -891,6 +891,11 @@ zkEnableSecurity=false # bookie then it would throw BKNotEnoughBookiesException instead of picking random one. # enforceMinNumRacksPerWriteQuorum=false +# Enforce write being acknowledged by bookies belonging to atleast minimum +# number of fault domains(depending on the placement policy) before being +# acknowledged by bookkeeper. +# enforceMinNumFaultDomainsForWrite=false + ############################################################################# ## Auditor settings ############################################################################# diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index 172e0a97aae..7cccc6d1d09 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -644,6 +644,9 @@ groups: description: | 'ignoreLocalNodeInPlacementPolicy' specifies whether to ignore localnode in the internal logic of placement policy. If it is not possible or useful to use Bookkeeper client node's (or AutoReplicator) rack/region info. for placement policy then it is better to ignore localnode instead of false alarming with log lines and metrics. default: false + - param: enforceMinNumFaultDomainsForWrite + description: | + 'enforceMinNumFaultDomainsForWrite' enforces EnsemblePlacementPolicy to check if a write has made it to bookies in 'minNumRacksPerWriteQuorum' number of fault domains, before acknowledging the write back. - name: AutoRecovery auditor settings params: From 8f7c5dc6739b674bb86a298f92cd613623ef5dde Mon Sep 17 00:00:00 2001 From: zhaohaidao Date: Thu, 6 Jun 2019 15:31:27 +0800 Subject: [PATCH 0302/1642] ISSUE #1927: fix NoNodeException in LocalBookeeper ### Motivation It addresses #1927 where it provides a conf check at the beginning of startLocalBookiesInternal function. ### Changes - Add a conf check at the beginning of startLocalBookiesInternal function where non-default zkLedgersRootPath is not allowed. cc sijie Reviewers: Ivan Kelly , Enrico Olivelli , Sijie Guo This closes #2082 from zhaohaidao/master, closes #1927 --- .../java/org/apache/bookkeeper/util/LocalBookKeeper.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) 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 97884a6454d..d6efeac1af8 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 @@ -134,17 +134,16 @@ private void initializeZookeeper(String zkHost, int zkPort) throws IOException { .connectString(zkHost + ":" + zkPort) .sessionTimeoutMs(zkSessionTimeOut) .build()) { - List multiOps = Lists.newArrayListWithExpectedSize(3); String zkLedgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseConf); - multiOps.add( - Op.create(zkLedgersRootPath, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)); + ZkUtils.createFullPathOptimistic(zkc, zkLedgersRootPath, new byte[0], Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + List multiOps = Lists.newArrayListWithExpectedSize(2); multiOps.add( Op.create(zkLedgersRootPath + "/" + AVAILABLE_NODE, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)); multiOps.add( Op.create(zkLedgersRootPath + "/" + AVAILABLE_NODE + "/" + READONLY, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)); - zkc.multi(multiOps); // No need to create an entry for each requested bookie anymore as the // BookieServers will register themselves with ZooKeeper on startup. @@ -345,7 +344,6 @@ static void startLocalBookiesInternal(ServerConfiguration conf, String zkDataDir, String localBookiesConfigDirName) throws Exception { - conf.setMetadataServiceUri( newMetadataServiceUri( zkHost, @@ -353,7 +351,6 @@ static void startLocalBookiesInternal(ServerConfiguration conf, conf.getLedgerManagerLayoutStringFromFactoryClass(), conf.getZkLedgersRootPath())); LocalBookKeeper lb = new LocalBookKeeper(numBookies, initialBookiePort, conf, localBookiesConfigDirName); - ZooKeeperServerShim zks = null; File zkTmpDir = null; List bkTmpDirs = null; From cf34550b97485ff60cd6e2ff78348d0751effec7 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 6 Jun 2019 01:58:43 -0700 Subject: [PATCH 0303/1642] Update distributedlog library to use NoSuchLedgerExistsOnMetadataServer *Motivation* Issue #2066 introduced `NoSuchLedgerExistsOnMetadataServer`. But the distributedlog library is not updated to reflect to this change. *Modifications* Change to use `NoSuchLedgerExistsOnMetadataServer` in the right place. Reviewers: Ivan Kelly , Enrico Olivelli This closes #2102 from sijie/fix_failure_test --- .../main/java/org/apache/distributedlog/BookKeeperClient.java | 3 ++- .../distributedlog/impl/logsegment/BKLogSegmentEntryStore.java | 3 ++- .../org/apache/distributedlog/tools/DistributedLogTool.java | 2 +- .../org/apache/distributedlog/TestBKDistributedLogManager.java | 2 +- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java index 366178a9d1e..88781353055 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java @@ -27,6 +27,7 @@ import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BKException.Code; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy; @@ -238,7 +239,7 @@ public CompletableFuture deleteLedger(long lid, public void deleteComplete(int rc, Object ctx) { if (BKException.Code.OK == rc) { promise.complete(null); - } else if (BKException.Code.NoSuchLedgerExistsException == rc) { + } else if (Code.NoSuchLedgerExistsOnMetadataServerException == rc) { if (ignoreNonExistentLedger) { promise.complete(null); } else { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java index aaff62e23dd..b164a392455 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java @@ -23,6 +23,7 @@ import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BKException.Code; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -134,7 +135,7 @@ public CompletableFuture deleteLogSegment(LogSegmentMetadata @Override public void deleteComplete(int rc, Object ctx) { DeleteLogSegmentRequest deleteRequest = (DeleteLogSegmentRequest) ctx; - if (BKException.Code.NoSuchLedgerExistsException == rc) { + if (Code.NoSuchLedgerExistsOnMetadataServerException == rc) { logger.warn("No ledger {} found to delete for {}.", deleteRequest.segment.getLogSegmentId(), deleteRequest.segment); } else if (BKException.Code.OK != rc) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java index d19e768dcb4..74489965f71 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java @@ -1277,7 +1277,7 @@ public void run() { if (numLedgersDeleted % 1000 == 0) { System.out.println("Deleted " + numLedgersDeleted + " ledgers."); } - } catch (BKException.BKNoSuchLedgerExistsException e) { + } catch (BKException.BKNoSuchLedgerExistsOnMetadataServerException e) { int numLedgersDeleted = numLedgers.incrementAndGet(); if (numLedgersDeleted % 1000 == 0) { System.out.println("Deleted " + numLedgersDeleted + " ledgers."); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java index 14c751e3aa1..db0689145c5 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java @@ -1237,7 +1237,7 @@ public void testDeleteLog() throws Exception { driver.getReaderBKC().get().openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8)); fail("Should fail to open ledger after we delete the log"); - } catch (BKException.BKNoSuchLedgerExistsException e) { + } catch (BKException.BKNoSuchLedgerExistsOnMetadataServerException e) { // ignore } // delete again should not throw any exception From b2960e33139b94863185840f388396a783cd0828 Mon Sep 17 00:00:00 2001 From: karanmehta93 Date: Fri, 7 Jun 2019 10:25:58 -0500 Subject: [PATCH 0304/1642] Display filesystem and docker info for all jenkins runs Descriptions of the changes in this PR: Changes to shell commands in few scripts ### Motivation It was difficult to reproduce jenkins test failures locally and the cause might be due to differences in underlying file system. These commands will output more information to spot the exact differences. ### Changes Running `df` command with an additional `-T` parameter to output filesystem information Added `docker info` command to print docker details before every test run Reviewers: Enrico Olivelli , Sijie Guo This closes #2107 from karanmehta93/master --- .../jenkins/job_bookkeeper_precommit_bookie_tests.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_client_tests.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_java11.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_java8.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_remaining_tests.groovy | 2 +- .../jenkins/job_bookkeeper_precommit_replication_tests.groovy | 2 +- .test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy | 2 +- .../job_bookkeeper_release_branch_47_integrationtests.groovy | 2 +- .../job_bookkeeper_release_branch_48_integrationtests.groovy | 2 +- .test-infra/scripts/pre-docker-tests.sh | 3 ++- 10 files changed, 11 insertions(+), 10 deletions(-) diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy index f25cdd5004f..a9f4ba0ed91 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy @@ -50,7 +50,7 @@ freeStyleJob('bookkeeper_precommit_bookie_tests') { shell("id") shell("ulimit -a") shell("pwd") - shell("df -h") + shell("df -Th") shell("ps aux") // Build everything diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy index 03e26365eab..9c2c3fba92f 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy @@ -50,7 +50,7 @@ freeStyleJob('bookkeeper_precommit_client_tests') { shell("id") shell("ulimit -a") shell("pwd") - shell("df -h") + shell("df -Th") shell("ps aux") // Build everything diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_java11.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_java11.groovy index 7089d49a806..cf0857c7e1e 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_java11.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_java11.groovy @@ -30,7 +30,7 @@ mavenJob('bookkeeper_precommit_pullrequest_java11') { shell("id") shell("ulimit -a") shell("pwd") - shell("df -h") + shell("df -Th") shell("ps aux") } diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy index 19b6b956680..dcb07c59c38 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy @@ -30,7 +30,7 @@ mavenJob('bookkeeper_precommit_pullrequest_java8') { shell("id") shell("ulimit -a") shell("pwd") - shell("df -h") + shell("df -Th") shell("ps aux") } diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy index d713145e1a0..6e8bf4c38d0 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy @@ -54,7 +54,7 @@ freeStyleJob('bookkeeper_precommit_remaining_tests') { shell("id") shell("ulimit -a") shell("pwd") - shell("df -h") + shell("df -Th") shell("ps aux") // Build everything diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy index 8d4a2ff12a3..a9f356d99c4 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy @@ -50,7 +50,7 @@ freeStyleJob('bookkeeper_precommit_replication_tests') { shell("id") shell("ulimit -a") shell("pwd") - shell("df -h") + shell("df -Th") shell("ps aux") // Build everything diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy index 071bd1114a1..62549d5cce8 100644 --- a/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy @@ -50,7 +50,7 @@ freeStyleJob('bookkeeper_precommit_tls_tests') { shell("id") shell("ulimit -a") shell("pwd") - shell("df -h") + shell("df -Th") shell("ps aux") // Build everything diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_47_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_47_integrationtests.groovy index e6fe9be7889..b0433aa0e94 100644 --- a/.test-infra/jenkins/job_bookkeeper_release_branch_47_integrationtests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_47_integrationtests.groovy @@ -40,7 +40,7 @@ freeStyleJob('bookkeeper_release_branch_47_integrationtests') { shell('id') shell('ulimit -a') shell('pwd') - shell('df -h') + shell('df -Th') shell('ps -eo euser,pid,ppid,pgid,start,pcpu,pmem,cmd') shell('docker network prune -f --filter name=testnetwork_*') // clean up any dangling networks from previous runs shell('docker system events > docker.log & echo $! > docker-log.pid') diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_48_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_48_integrationtests.groovy index 0db8a88c13f..daf4fbd5f64 100644 --- a/.test-infra/jenkins/job_bookkeeper_release_branch_48_integrationtests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_48_integrationtests.groovy @@ -40,7 +40,7 @@ freeStyleJob('bookkeeper_release_branch_48_integrationtests') { shell('id') shell('ulimit -a') shell('pwd') - shell('df -h') + shell('df -Th') shell('ps -eo euser,pid,ppid,pgid,start,pcpu,pmem,cmd') shell('docker network prune -f --filter name=testnetwork_*') // clean up any dangling networks from previous runs shell('docker system events > docker.log & echo $! > docker-log.pid') diff --git a/.test-infra/scripts/pre-docker-tests.sh b/.test-infra/scripts/pre-docker-tests.sh index 1edb48c0d7b..65fac8a5561 100755 --- a/.test-infra/scripts/pre-docker-tests.sh +++ b/.test-infra/scripts/pre-docker-tests.sh @@ -23,8 +23,9 @@ set -ex id ulimit -a pwd -df -h +df -Th ps -eo euser,pid,ppid,pgid,start,pcpu,pmem,cmd +docker info docker system prune -f # clean up any dangling networks from previous runs docker network prune -f --filter name=testnetwork_* From 1ca9521356ef433f16210965609c8fb361516db2 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Fri, 7 Jun 2019 08:27:04 -0700 Subject: [PATCH 0305/1642] Bring back statslogger to BookKeeper client in ReplicationWorker process. Descriptions of the changes in this PR: bd2b16e880d172d4761461fdbf85c1bd19b24e36 is supposed to fix this issue, but it missed passing statsLogger to createBookKeeperClient method. For more info. check desc. of bd2b16e880d172d4761461fdbf85c1bd19b24e36. Reviewers: Enrico Olivelli , Sijie Guo This closes #2108 from reddycharan/enablebookieclientlogger --- .../org/apache/bookkeeper/replication/AutoRecoveryMain.java | 3 ++- .../org/apache/bookkeeper/bookie/BookieInitializationTest.java | 3 +++ 2 files changed, 5 insertions(+), 1 deletion(-) 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 c4954438232..39b41aa6503 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 @@ -36,6 +36,7 @@ import org.apache.bookkeeper.bookie.ExitCode; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperClientStats; import org.apache.bookkeeper.common.component.ComponentStarter; import org.apache.bookkeeper.common.component.LifecycleComponent; import org.apache.bookkeeper.common.component.LifecycleComponentStack; @@ -91,7 +92,7 @@ public AutoRecoveryMain(ServerConfiguration conf, StatsLogger statsLogger) throws IOException, InterruptedException, KeeperException, UnavailableException, CompatibilityException { this.conf = conf; - this.bkc = Auditor.createBookKeeperClient(conf); + this.bkc = Auditor.createBookKeeperClient(conf, statsLogger.scope(BookKeeperClientStats.CLIENT_SCOPE)); MetadataClientDriver metadataClientDriver = bkc.getMetadataClientDriver(); metadataClientDriver.setSessionStateListener(() -> { LOG.error("Client connection to the Metadata server has expired, so shutting down AutoRecoveryMain!"); 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 8e0561b20ee..8ea63e1490e 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 @@ -65,6 +65,7 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.BookKeeperClientStats; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.component.ComponentStarter; import org.apache.bookkeeper.common.component.Lifecycle; @@ -1319,6 +1320,8 @@ public void testIOVertexHTTPServerEndpointForARWithPrometheusProvider() throws E // do primitive checks if metrics string contains some stats assertTrue("Metrics should contain basic counters", metrics.contains(ReplicationStats.NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED)); + assertTrue("Metrics should contain basic counters from BookKeeper client", + metrics.contains(BookKeeperClientStats.CREATE_OP)); // Now, hit the rest endpoint for configs url = new URL("http://localhost:" + nextFreePort + HttpRouter.SERVER_CONFIG); From 2d37286f216d4be9ffae8f35eed90b59422ba4d8 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Mon, 10 Jun 2019 14:57:11 -0700 Subject: [PATCH 0306/1642] Testcase for limitStatsLogging. Descriptions of the changes in this PR: bd2b16e880d172d4761461fdbf85c1bd19b24e36 had introduced 'limitStatsLogging' feature, which would be used to limit statslogging, where it is needed. eg - we would like to suppress logs from PCBC in Auditor/ReplicationWorker process. In this commit I'm adding testcases for this feature. Reviewers: Enrico Olivelli , Sijie Guo This closes #2109 from reddycharan/testlimitstatslogging --- .../proto/PerChannelBookieClient.java | 12 ++++--- .../bookkeeper/test/BookieClientTest.java | 34 ++++++++++++++++--- 2 files changed, 37 insertions(+), 9 deletions(-) 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 2f1659f25de..c151426701f 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 @@ -388,12 +388,8 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor shFactory.init(NodeType.Client, conf, allocator); } - StringBuilder nameBuilder = new StringBuilder(); - nameBuilder.append(addr.getHostName().replace('.', '_').replace('-', '_')) - .append("_").append(addr.getPort()); - this.statsLogger = parentStatsLogger.scope(BookKeeperClientStats.CHANNEL_SCOPE) - .scope(nameBuilder.toString()); + .scope(buildStatsLoggerScopeName(addr)); readEntryOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_READ_OP); addEntryOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_ADD_OP); @@ -493,6 +489,12 @@ public boolean isSecure() { }; } + public static String buildStatsLoggerScopeName(BookieSocketAddress addr) { + StringBuilder nameBuilder = new StringBuilder(); + nameBuilder.append(addr.getHostName().replace('.', '_').replace('-', '_')).append("_").append(addr.getPort()); + return nameBuilder.toString(); + } + private void completeOperation(GenericCallback op, int rc) { //Thread.dumpStack(); closeLock.readLock().lock(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java index d38a178b779..322ca6da772 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java @@ -40,6 +40,7 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.Code; +import org.apache.bookkeeper.client.BookKeeperClientStats; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.util.OrderedExecutor; @@ -55,7 +56,10 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.proto.BookkeeperProtocol; +import org.apache.bookkeeper.proto.PerChannelBookieClient; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.util.IOUtils; import org.junit.After; @@ -269,10 +273,23 @@ public void testNoLedger() throws Exception { } @Test - public void testGetBookieInfo() throws IOException, InterruptedException { + public void testGetBookieInfoWithLimitStatsLogging() throws IOException, InterruptedException { + testGetBookieInfo(true); + } + + @Test + public void testGetBookieInfoWithoutLimitStatsLogging() throws IOException, InterruptedException { + testGetBookieInfo(false); + } + + public void testGetBookieInfo(boolean limitStatsLogging) throws IOException, InterruptedException { BookieSocketAddress addr = bs.getLocalAddress(); - BookieClient bc = new BookieClientImpl(new ClientConfiguration(), new NioEventLoopGroup(), - UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE); + ClientConfiguration clientConf = new ClientConfiguration(); + clientConf.setLimitStatsLogging(limitStatsLogging); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); + BookieClient bc = new BookieClientImpl(clientConf, new NioEventLoopGroup(), UnpooledByteBufAllocator.DEFAULT, + executor, scheduler, statsLogger); long flags = BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE | BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; @@ -281,6 +298,7 @@ class CallbackObj { long requested; long freeDiskSpace, totalDiskCapacity; CountDownLatch latch = new CountDownLatch(1); + CallbackObj(long requested) { this.requested = requested; this.rc = 0; @@ -299,7 +317,7 @@ public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) { obj.freeDiskSpace = bInfo.getFreeDiskSpace(); } if ((obj.requested - & BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE) != 0) { + & BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE) != 0) { obj.totalDiskCapacity = bInfo.getTotalDiskSpace(); } } @@ -313,5 +331,13 @@ public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) { assertTrue("GetBookieInfo failed with error " + obj.rc, obj.rc == Code.OK); assertTrue("GetBookieInfo failed with error " + obj.rc, obj.freeDiskSpace <= obj.totalDiskCapacity); assertTrue("GetBookieInfo failed with error " + obj.rc, obj.totalDiskCapacity > 0); + + TestOpStatsLogger perChannelBookieClientScopeOfThisAddr = (TestOpStatsLogger) statsLogger + .scope(BookKeeperClientStats.CHANNEL_SCOPE) + .scope(PerChannelBookieClient.buildStatsLoggerScopeName(addr)) + .getOpStatsLogger(BookKeeperClientStats.GET_BOOKIE_INFO_OP); + int expectedBookieInfoSuccessCount = (limitStatsLogging) ? 0 : 1; + assertEquals("BookieInfoSuccessCount", expectedBookieInfoSuccessCount, + perChannelBookieClientScopeOfThisAddr.getSuccessCount()); } } From 2654042ee9e53b6a1ebaa64b0fdf95986ccafda7 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 13 Jun 2019 02:22:09 -0700 Subject: [PATCH 0307/1642] Use https for Twitter maven repo ### Motivation Fetch the maven dependencies through HTTPs Reviewers: Ivan Kelly , Enrico Olivelli , Sijie Guo This closes #2100 from merlimat/merlimat-patch-1 --- bookkeeper-stats-providers/twitter-science-provider/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-stats-providers/twitter-science-provider/pom.xml b/bookkeeper-stats-providers/twitter-science-provider/pom.xml index e766fe06bd7..33b31debb1b 100644 --- a/bookkeeper-stats-providers/twitter-science-provider/pom.xml +++ b/bookkeeper-stats-providers/twitter-science-provider/pom.xml @@ -78,7 +78,7 @@ twitter Twitter repo default - http://maven.twttr.com + https://maven.twttr.com From d2835455c6d101ea4068dbb411e67ae21274d5a5 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 20 Jun 2019 14:36:11 +0200 Subject: [PATCH 0308/1642] [DLOG] Delete empty inprogress segment on recovery This addresses an issue, whereby if a user is using getLastTxId() to find the first txnid when starting to write, they will continue to be able to write if the last segment written by the previous writer was empty. If the last segment was empty, then the maxTxId would be higher than the result of getLastTxId(). maxTxId is read from a znode, while getLastTxId() reads the txid of the last persisted record. In the case of an empty inprogress segment, the maxTxId znode was being updated with the expected first transaction id of the segment. This patch addresses the issue with the following changes: 1. The maxTxId znode is only updated when inprogress segment is completed, so it's value always refers to a transaction that exists. 2. On recovery, if the inprogress segment is empty, delete it. There was a TODO comment to do this already there. 3. When generating the sequence number, allow a potential sequence number which is equal to the current max sequence number, as this can be the case where recovery deleted an inprogress empty segment. Reviewers: Enrico Olivelli , Sijie Guo This closes #2099 from ivankelly/txnid-issue --- .../distributedlog/BKLogWriteHandler.java | 65 ++++++---- .../TestDistributedLogBase.java | 6 +- .../org/apache/distributedlog/TestTxnId.java | 118 ++++++++++++++++++ 3 files changed, 165 insertions(+), 24 deletions(-) create mode 100644 stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java index 0ce0a25bb19..3095ec75187 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Comparator; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -145,9 +146,24 @@ public CompletableFuture apply(List segmentList) { segmentList, recoverLogSegmentFunction, scheduler - ).thenApply(GetLastTxIdFunction.INSTANCE); + ).thenApply(removeEmptySegments) + .thenApply(GetLastTxIdFunction.INSTANCE); } }; + private final Function, List> removeEmptySegments = + new Function, List>() { + @Override + public List apply(List segmentList) { + Iterator iter = segmentList.iterator(); + while (iter.hasNext()) { + LogSegmentMetadata segment = iter.next(); + if (segment == null) { + iter.remove(); + } + } + return segmentList; + } + }; // Stats private final StatsLogger perLogStatsLogger; @@ -479,7 +495,11 @@ protected long assignLogSegmentSequenceNumber() throws IOException { // no ledger seqno stored in /ledgers before LOG.info("No max ledger sequence number found while creating log segment {} for {}.", logSegmentSeqNo, getFullyQualifiedName()); - } else if (maxLogSegmentSequenceNo.getSequenceNumber() + 1 != logSegmentSeqNo) { + } else if (maxLogSegmentSequenceNo.getSequenceNumber() + 1 != logSegmentSeqNo // case 1 + && maxLogSegmentSequenceNo.getSequenceNumber() != logSegmentSeqNo) { // case 2 + // case 1 is the common case, where the new log segment is 1 more than the previous + // case 2 can occur when the writer crashes with an empty in progress ledger. This is then deleted + // on recovery, so the next new segment will have a matching sequence number LOG.warn("Unexpected max log segment sequence number {} for {} : list of cached segments = {}", new Object[]{maxLogSegmentSequenceNo.getSequenceNumber(), getFullyQualifiedName(), getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)}); @@ -635,10 +655,6 @@ private void createInprogressLogSegment(Transaction txn, LOG.debug("Try storing max sequence number in startLogSegment {} : {}", inprogressZnodePath, logSegmentSeqNo); storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, logSegmentSeqNo, true); - // Try storing max tx id. - LOG.debug("Try storing MaxTxId in startLogSegment {} {}", inprogressZnodePath, txId); - storeMaxTxId(txn, maxTxId, txId); - txn.execute().whenCompleteAsync(new FutureEventListener() { @Override @@ -1032,22 +1048,29 @@ private CompletableFuture completeLogSegment(LogSegmentMetad return FutureUtils.exception(new IOException("Unrecoverable corruption," + " please check logs.")); } else if (endTxId == DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID) { - // TODO: Empty ledger - Ideally we should just remove it? - endTxId = l.getFirstTxId(); - } + LOG.info("Inprogress segment {} is empty, deleting", l); - CompletableFuture promise = new CompletableFuture(); - doCompleteAndCloseLogSegment( - l.getZNodeName(), - l.getLogSegmentSequenceNumber(), - l.getLogSegmentId(), - l.getFirstTxId(), - endTxId, - recordCount, - lastEntryId, - lastSlotId, - promise); - return promise; + return deleteLogSegment(l).thenApply( + (result) -> { + synchronized (inprogressLSSNs) { + inprogressLSSNs.remove((Long) l.getLogSegmentSequenceNumber()); + } + return null; + }); + } else { + CompletableFuture promise = new CompletableFuture(); + doCompleteAndCloseLogSegment( + l.getZNodeName(), + l.getLogSegmentSequenceNumber(), + l.getLogSegmentId(), + l.getFirstTxId(), + endTxId, + recordCount, + lastEntryId, + lastSlotId, + promise); + return promise; + } } } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java index 903e6843e77..16565a8a4a2 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java @@ -95,12 +95,12 @@ public class TestDistributedLogBase { protected static String zkServers; protected static int zkPort; protected static int numBookies = 3; - private static final List tmpDirs = new ArrayList(); + protected static final List TMP_DIRS = new ArrayList(); @BeforeClass public static void setupCluster() throws Exception { File zkTmpDir = IOUtils.createTempDir("zookeeper", "distrlog"); - tmpDirs.add(zkTmpDir); + TMP_DIRS.add(zkTmpDir); Pair serverAndPort = LocalDLMEmulator.runZookeeperOnAnyPort(zkTmpDir); zks = serverAndPort.getLeft(); zkPort = serverAndPort.getRight(); @@ -125,7 +125,7 @@ public void uncaughtException(Thread t, Throwable e) { public static void teardownCluster() throws Exception { bkutil.teardown(); zks.stop(); - for (File dir : tmpDirs) { + for (File dir : TMP_DIRS) { FileUtils.forceDeleteOnExit(dir); } } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java new file mode 100644 index 00000000000..d5f4115abee --- /dev/null +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java @@ -0,0 +1,118 @@ +/** + * 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.distributedlog; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.stats.NullStatsProvider; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test Cases for RollLogSegments. + */ +public class TestTxnId extends TestDistributedLogBase { + private static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class); + + @Test + public void testRecoveryAfterBookieCrash() throws Exception { + String name = "txnid-after-crash"; + DistributedLogConfiguration conf = new DistributedLogConfiguration() + .setEnsembleSize(5) + .setWriteQuorumSize(5) + .setAckQuorumSize(5) + .setLogSegmentRollingIntervalMinutes(0) + .setLogSegmentRollingConcurrency(-1) + .setMaxLogSegmentBytes(400000); + + long entryId = 0; + List extraBookies = new ArrayList<>(); + try { + extraBookies.add(startExtraBookie()); + extraBookies.add(startExtraBookie()); + + try (BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name); + BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned()) { + writer.write(DLMTestUtil.getLogRecordInstance(1, 100000)).join(); + writer.write(DLMTestUtil.getLogRecordInstance(2, 100000)).join(); + + extraBookies.forEach(b -> b.shutdown()); + + try { + writer.write(DLMTestUtil.getLogRecordInstance(3, 100000)).join(); + Assert.fail("Shouldn't have succeeded"); + } catch (Exception e) { + // expected + } + + writer.write(DLMTestUtil.getLogRecordInstance(4, 100000)).join(); + Assert.fail("Shouldn't be able to write"); + } catch (Exception e) { + // expected + } + + extraBookies.add(startExtraBookie()); + extraBookies.add(startExtraBookie()); + + try (BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name); + BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned()) { + long firstTxid = dlm.getLastTxId() + 1; + for (int i = 0; i < 20; i++) { + logger.info("Writing entry {}", i); + writer.write(DLMTestUtil.getLogRecordInstance(firstTxid + i, 100000)).join(); + Thread.sleep(100); + } + } + } finally { + extraBookies.forEach(b -> b.shutdown()); + } + } + + private BookieServer startExtraBookie() throws Exception { + File journalDir = File.createTempFile("bookie", "journal"); + journalDir.delete(); + journalDir.mkdir(); + TMP_DIRS.add(journalDir); + + File ledgerDir = File.createTempFile("bookie", "ledger"); + ledgerDir.delete(); + ledgerDir.mkdir(); + TMP_DIRS.add(ledgerDir); + + ServerConfiguration conf = new ServerConfiguration(); + conf.setMetadataServiceUri("zk://" + zkServers + "/ledgers"); + conf.setBookiePort(0); + conf.setDiskUsageThreshold(0.99f); + conf.setAllowLoopback(true); + conf.setJournalDirName(journalDir.getPath()); + conf.setLedgerDirNames(new String[] { ledgerDir.getPath() }); + + BookieServer server = new BookieServer(conf, new NullStatsProvider().getStatsLogger("")); + server.start(); + + while (!server.isRunning()) { + Thread.sleep(10); + } + return server; + } +} From 97f0b4d63f9bf09f1ef2ae09169329778215c67a Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 20 Jun 2019 20:19:53 -0700 Subject: [PATCH 0309/1642] Issue #1970: Ensure getStickyReadBookieIndex returns valid bookie index Descriptions of the changes in this PR: Master Issue: #1970 Related Issues: apache/pulsar#3715 apache/pulsar#4526 *Motivation* Fixes #1970 By default bookie uses a random generator to generate a bookie index as the sticky read bookie index. However the random generator will generate negative numbers. Hence it will generate negative bookie indexes in write set and cause ArrayIndexOutOfBoundException when bookkeeper attempts to read entries. *Modifications* Make sure getStickyReadBookieIndex not return negative number. *Verify this change* This problem introduced by a random generator. It is very hard to write a unit test to reproduce this issue. Existing StickyRead tests are good to cover this code change. Reviewers: Enrico Olivelli , Jia Zhai , Yong Zhang This closes #2111 from sijie/fix_sticky_read, closes #1970 --- .../apache/bookkeeper/client/EnsemblePlacementPolicy.java | 5 +++-- .../java/org/apache/bookkeeper/client/LedgerHandle.java | 8 +++++--- 2 files changed, 8 insertions(+), 5 deletions(-) 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 64a4b91bd53..6249ec5a438 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 @@ -30,6 +30,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; +import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -384,12 +385,12 @@ default int getStickyReadBookieIndex(LedgerMetadata metadata, Optional if (!currentStickyBookieIndex.isPresent()) { // Pick one bookie randomly from the current ensemble as the initial // "sticky bookie" - return ThreadLocalRandom.current().nextInt() % metadata.getEnsembleSize(); + return ThreadLocalRandom.current().nextInt(metadata.getEnsembleSize()); } else { // When choosing a new sticky bookie index (eg: after the current // one has read failures), by default we pick the next one in the // ensemble, to avoid picking up the same one again. - return (currentStickyBookieIndex.get() + 1) % metadata.getEnsembleSize(); + return MathUtils.signSafeMod(currentStickyBookieIndex.get() + 1, metadata.getEnsembleSize()); } } 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 7e21f975230..f3abef821b4 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 @@ -95,6 +95,8 @@ public class LedgerHandle implements WriteHandle { static final Logger LOG = LoggerFactory.getLogger(LedgerHandle.class); + private static final int STICKY_READ_BOOKIE_INDEX_UNSET = -1; + final ClientContext clientCtx; final byte[] ledgerKey; @@ -199,7 +201,7 @@ private enum HandleState { && getLedgerMetadata().getEnsembleSize() == getLedgerMetadata().getWriteQuorumSize()) { stickyBookieIndex = clientCtx.getPlacementPolicy().getStickyReadBookieIndex(metadata, Optional.empty()); } else { - stickyBookieIndex = -1; + stickyBookieIndex = STICKY_READ_BOOKIE_INDEX_UNSET; } if (clientCtx.getConf().throttleValue > 0) { @@ -265,7 +267,7 @@ void recordReadErrorOnBookie(int bookieIndex) { // If sticky bookie reads are enabled, switch the sticky bookie to the // next bookie in the ensemble so that we avoid to keep reading from the // same failed bookie - if (stickyBookieIndex != -1) { + if (stickyBookieIndex != STICKY_READ_BOOKIE_INDEX_UNSET) { // This will be idempotent when we have multiple read errors on the // same bookie. The net result is that we just go to the next bookie stickyBookieIndex = clientCtx.getPlacementPolicy().getStickyReadBookieIndex(getLedgerMetadata(), @@ -2020,7 +2022,7 @@ List getCurrentEnsemble() { * This will include all bookies that are cotna */ WriteSet getWriteSetForReadOperation(long entryId) { - if (stickyBookieIndex != -1) { + if (stickyBookieIndex != STICKY_READ_BOOKIE_INDEX_UNSET) { // When sticky reads are enabled we want to make sure to take // advantage of read-ahead (or, anyway, from efficiencies in // reading sequential data from disk through the page cache). From 40dc018e042a49d99fab93a59c4a6ac25e1f4664 Mon Sep 17 00:00:00 2001 From: Hugo Abreu <6619758+hugomiguelabreu@users.noreply.github.com> Date: Fri, 21 Jun 2019 04:22:33 +0100 Subject: [PATCH 0310/1642] Added zookeeper host configuration. Closes #1960 Descriptions of the changes in this PR: Added host IP binding to sandbox. ### Motivation Allowing running the sandbox inside guest machines and be accessible by all system (including other guest machines). When binding 127.0.0.1 other guest machines will experience an error. By allowing to bind the correct IP the namespace will be available and correctly configured for other hosts. ### Changes Added a new argument to the _dlog_ tool allowing to define the host IP to bind ZooKeeper and DistributedLog namespace. Master Issue: #1960 Reviewers: Enrico Olivelli , Sijie Guo This closes #2081 from hugomiguelabreu/master, closes #1960 --- .../apache/distributedlog/LocalDLMEmulator.java | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java index dc014f1b3fa..61bc23105cf 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java @@ -308,13 +308,22 @@ public static Pair runZookeeperOnAnyPort(int baseP public static void main(String[] args) throws Exception { try { if (args.length < 1) { - System.out.println("Usage: LocalDLEmulator "); + System.out.println("Usage: LocalDLEmulator [] "); System.exit(-1); } - final int zkPort = Integer.parseInt(args[0]); + String zkHost = DEFAULT_ZK_HOST; + int zkPort = DEFAULT_ZK_PORT; + if (args.length == 1) { + zkPort = Integer.parseInt(args[0]); + } else { + zkHost = args[0]; + zkPort = Integer.parseInt(args[1]); + } + final File zkDir = IOUtils.createTempDir("distrlog", "zookeeper"); final LocalDLMEmulator localDlm = LocalDLMEmulator.newBuilder() + .zkHost(zkHost) .zkPort(zkPort) .build(); @@ -334,7 +343,7 @@ public void run() { System.out.println(String.format( "DistributedLog Sandbox is running now. You could access distributedlog://%s:%s", - DEFAULT_ZK_HOST, + zkHost, zkPort)); } catch (Exception ex) { System.out.println("Exception occurred running emulator " + ex); From 5c42bf52abd5a61676deca818c79b392dc09da40 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Mon, 24 Jun 2019 23:17:52 -0700 Subject: [PATCH 0311/1642] Implementation of ZoneawareEnsemblePlacementPolicy Descriptions of the changes in this PR: Implementation of ZoneawareEnsemblePlacementPolicy - This is meant to be used in public cloud infrastructure in which Availability zones (AZs) are isolated locations within data center regions from which public cloud services originate and operate. - ZoneawareEnsemblePlacementPolicy is a placement policy implementation which uses zone information for placing ensembles. - In this PlacementPolicy, it is assumed the networktopology/networklocation is going to be two level tree, where first part indicates zoneid and the second part indicates upgradedomainid. - Here upgrade domain is a set of bookies in an AZ that can be brought down together for short term maintenance with no impact to the service. This would help in enabling parallel patching. - Upgrade domain is a logic concept/division and it may be mapped to cloud provider native cluster/grouping like Placement Groups in AWS. Introduce soft PlacementPolicyAdherence - for ZoneawarePlacementPolicy if in a writeset the number of zones are in between minnumofzones and desirednumofzones then it is considered MEETS_SOFT. - if all of the writesets have zones >= desirednumofzones then it is considered MEETS_STRICT - if a writeset contains < minnumofzones then it is considered FAIL - add corresponding metric for PlacementPolicyAdherence.MEETS_SOFT Reviewers: Enrico Olivelli , Sijie Guo , Venkateswararao Jujjuri (JV) This closes #2113 from reddycharan/zoneawareplacementpolicycommunitybranch --- .../LocalBookieEnsemblePlacementPolicy.java | 8 +- .../bookkeeper/client/BookKeeperAdmin.java | 7 +- .../client/BookKeeperClientStats.java | 1 + .../bookkeeper/client/BookieWatcherImpl.java | 21 +- .../DefaultEnsemblePlacementPolicy.java | 8 +- .../DynamicWeightedRandomSelectionImpl.java | 167 ++ .../client/EnsemblePlacementPolicy.java | 56 +- .../RackawareEnsemblePlacementPolicyImpl.java | 36 +- .../RegionAwareEnsemblePlacementPolicy.java | 6 +- .../TopologyAwareEnsemblePlacementPolicy.java | 27 +- .../ZoneawareEnsemblePlacementPolicy.java | 130 ++ .../ZoneawareEnsemblePlacementPolicyImpl.java | 995 ++++++++++++ .../conf/AbstractConfiguration.java | 65 + .../bookkeeper/net/NetworkTopology.java | 3 + .../bookkeeper/replication/Auditor.java | 100 +- .../replication/ReplicationStats.java | 1 + .../TestRackawareEnsemblePlacementPolicy.java | 101 +- .../client/TestWeightedRandomSelection.java | 83 +- .../TestZoneawareEnsemblePlacementPolicy.java | 1339 +++++++++++++++++ .../AuditorPlacementPolicyCheckTest.java | 159 +- site/_data/config/bk_server.yaml | 12 + 21 files changed, 3169 insertions(+), 156 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DynamicWeightedRandomSelectionImpl.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicy.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestZoneawareEnsemblePlacementPolicy.java 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 db88bc70f82..a0d2edb0af9 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 @@ -116,7 +116,7 @@ public PlacementResult> newEnsemble(int ensembleSize, throw new IllegalArgumentException("Local ensemble policy can only return 1 bookie"); } - return PlacementResult.of(Lists.newArrayList(bookieAddress), true); + return PlacementResult.of(Lists.newArrayList(bookieAddress), PlacementPolicyAdherence.MEETS_STRICT); } @Override @@ -125,8 +125,8 @@ public void updateBookieInfo(Map bookieToFreeSp } @Override - public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, - int ackQuorumSize) { - return true; + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + int writeQuorumSize, int ackQuorumSize) { + return PlacementPolicyAdherence.MEETS_STRICT; } } 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 4a869eee3ba..5a5903d40cf 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 @@ -56,6 +56,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; import org.apache.bookkeeper.client.AsyncCallback.RecoverCallback; +import org.apache.bookkeeper.client.EnsemblePlacementPolicy.PlacementPolicyAdherence; import org.apache.bookkeeper.client.LedgerFragmentReplicator.SingleFragmentCallback; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncOpenCallback; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncReadCallback; @@ -1004,8 +1005,8 @@ private Map getReplacementBookiesByIndexes( oldBookie, bookiesToExclude); BookieSocketAddress newBookie = replaceBookieResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); - if (!isEnsembleAdheringToPlacementPolicy) { + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); + if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) { if (LOG.isDebugEnabled()) { LOG.debug( "replaceBookie for bookie: {} in ensemble: {} " @@ -1659,7 +1660,7 @@ private static boolean areEntriesOfSegmentStoredInTheBookie(LedgerMetadata ledge * @return true if the ledger is adhering to * EnsemblePlacementPolicy */ - public boolean isEnsembleAdheringToPlacementPolicy(List ensembleBookiesList, + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleBookiesList, int writeQuorumSize, int ackQuorumSize) { return bkc.getPlacementPolicy().isEnsembleAdheringToPlacementPolicy(ensembleBookiesList, writeQuorumSize, ackQuorumSize); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java index ed10b9cfe64..ddc757ed990 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java @@ -101,6 +101,7 @@ public interface BookKeeperClientStats { String WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS_LATENCY = "WRITE_DELAYED_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS_LATENCY"; String WRITE_TIMED_OUT_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS = "WRITE_TIME_OUT_DUE_TO_NOT_ENOUGH_FAULT_DOMAINS"; + String NUM_WRITABLE_BOOKIES_IN_DEFAULT_FAULTDOMAIN = "NUM_WRITABLE_BOOKIES_IN_DEFAULT_FAULTDOMAIN"; OpStatsLogger getCreateOpLogger(); OpStatsLogger getOpenOpLogger(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index f4dd9a907ad..f26ee61eddd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -41,6 +41,7 @@ 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.client.EnsemblePlacementPolicy.PlacementPolicyAdherence; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -227,14 +228,14 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi long startTime = MathUtils.nowInNano(); EnsemblePlacementPolicy.PlacementResult> newEnsembleResponse; List socketAddresses; - boolean isEnsembleAdheringToPlacementPolicy = false; + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = PlacementPolicyAdherence.FAIL; try { Set quarantinedBookiesSet = quarantinedBookies.asMap().keySet(); newEnsembleResponse = placementPolicy.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, new HashSet(quarantinedBookiesSet)); socketAddresses = newEnsembleResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.isStrictlyAdheringToPolicy(); - if (!isEnsembleAdheringToPlacementPolicy) { + isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.isAdheringToPolicy(); + if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) { ensembleNotAdheringToPlacementPolicy.inc(); log.warn("New ensemble: {} is not adhering to Placement Policy. quarantinedBookies: {}", socketAddresses, quarantinedBookiesSet); @@ -248,8 +249,8 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi newEnsembleResponse = placementPolicy.newEnsemble( ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, new HashSet<>()); socketAddresses = newEnsembleResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.isStrictlyAdheringToPolicy(); - if (!isEnsembleAdheringToPlacementPolicy) { + isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.isAdheringToPolicy(); + if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) { ensembleNotAdheringToPlacementPolicy.inc(); log.warn("New ensemble: {} is not adhering to Placement Policy", socketAddresses); } @@ -268,7 +269,7 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, BookieSocketAddress addr = existingBookies.get(bookieIdx); EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; BookieSocketAddress socketAddress; - boolean isEnsembleAdheringToPlacementPolicy = false; + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = PlacementPolicyAdherence.FAIL; try { // we exclude the quarantined bookies also first Set excludedBookiesAndQuarantinedBookies = new HashSet( @@ -279,8 +280,8 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, existingBookies, addr, excludedBookiesAndQuarantinedBookies); socketAddress = replaceBookieResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); - if (!isEnsembleAdheringToPlacementPolicy) { + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); + if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) { ensembleNotAdheringToPlacementPolicy.inc(); log.warn( "replaceBookie for bookie: {} in ensemble: {} is not adhering to placement policy and" @@ -295,8 +296,8 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, replaceBookieResponse = placementPolicy.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, existingBookies, addr, excludeBookies); socketAddress = replaceBookieResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); - if (!isEnsembleAdheringToPlacementPolicy) { + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); + if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) { ensembleNotAdheringToPlacementPolicy.inc(); log.warn( "replaceBookie for bookie: {} in ensemble: {} is not adhering to placement policy and" 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 f6bb1af6b3a..8569c71c5b9 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 @@ -69,7 +69,7 @@ public PlacementResult> newEnsemble(int ensembleSize, throws BKNotEnoughBookiesException { ArrayList newBookies = new ArrayList(ensembleSize); if (ensembleSize <= 0) { - return PlacementResult.of(newBookies, false); + return PlacementResult.of(newBookies, PlacementPolicyAdherence.FAIL); } List allBookies; rwLock.readLock().lock(); @@ -219,8 +219,8 @@ public void uninitalize() { } @Override - public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, - int ackQuorumSize) { - return true; + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + int writeQuorumSize, int ackQuorumSize) { + return PlacementPolicyAdherence.MEETS_STRICT; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DynamicWeightedRandomSelectionImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DynamicWeightedRandomSelectionImpl.java new file mode 100644 index 00000000000..84c0abedcec --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DynamicWeightedRandomSelectionImpl.java @@ -0,0 +1,167 @@ +/* + * 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.client; + +import com.google.common.math.Quantiles; +import com.google.common.math.Quantiles.ScaleAndIndex; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * DynamicWeightedRandomSelectionImpl class implements both getNextRandom + * overloaded methods. Where getNextRandom() method considers all bookies it + * knows of as candidates, but getNextRandom(Collection selectedNodes) method + * considers only 'selectedNodes' as candidates. + */ +class DynamicWeightedRandomSelectionImpl implements WeightedRandomSelection { + static final Logger LOG = LoggerFactory.getLogger(DynamicWeightedRandomSelectionImpl.class); + + int maxProbabilityMultiplier; + final Map weightMap; + final ReadWriteLock rwLock = new ReentrantReadWriteLock(true); + Random rand; + + DynamicWeightedRandomSelectionImpl() { + this(-1); + } + + DynamicWeightedRandomSelectionImpl(int maxMultiplier) { + this.maxProbabilityMultiplier = maxMultiplier; + this.weightMap = new HashMap(); + rand = new Random(System.currentTimeMillis()); + } + + @Override + public void updateMap(Map updatedMap) { + rwLock.writeLock().lock(); + try { + weightMap.clear(); + weightMap.putAll(updatedMap); + } finally { + rwLock.writeLock().unlock(); + } + } + + @Override + public T getNextRandom() { + rwLock.readLock().lock(); + try { + return getNextRandom(weightMap.keySet()); + } finally { + rwLock.readLock().unlock(); + } + } + + @Override + public T getNextRandom(Collection selectedNodes) { + rwLock.readLock().lock(); + try { + /* + * calculate minWeight and actual total weight. + */ + long minWeight = Long.MAX_VALUE; + long actTotalWeight = 0; + for (T node : selectedNodes) { + long weight = 0; + if ((weightMap.containsKey(node))) { + weight = weightMap.get(node).getWeight(); + } + actTotalWeight += weight; + if (weight > 0 && minWeight > weight) { + minWeight = weight; + } + } + + long medianWeight; + /* + * if actTotalWeight is 0, then assign 1 to minWeight and + * medianWeight. + */ + if (actTotalWeight == 0) { + minWeight = 1L; + medianWeight = 1L; + } else { + /* + * calculate medianWeight. + */ + Function weightFunc = (node) -> { + long weight = 0; + if ((weightMap.containsKey(node))) { + weight = weightMap.get(node).getWeight(); + } + return Long.valueOf(weight); + }; + ArrayList weightList = selectedNodes.stream().map(weightFunc) + .collect(Collectors.toCollection(ArrayList::new)); + ScaleAndIndex median = Quantiles.median(); + medianWeight = (long) median.compute(weightList); + } + + /* + * initialize maxWeight value based on maxProbabilityMultiplier. + */ + long maxWeight = maxProbabilityMultiplier * medianWeight; + + /* + * apply weighted random selection to select an element randomly + * based on weight. + */ + long cumTotalWeight = 0; + T nextRandomNode = null; + for (T node : selectedNodes) { + long weight = 0; + if ((weightMap.containsKey(node))) { + weight = weightMap.get(node).getWeight(); + } + if (weight <= 0) { + weight = minWeight; + } else if (maxWeight > 0 && weight > maxWeight) { + weight = maxWeight; + } + long tmpRandLong = rand.nextLong(); + if (tmpRandLong == Long.MIN_VALUE) { + tmpRandLong++; + } + long randValue = Math.abs(tmpRandLong) % (cumTotalWeight + weight); + if (randValue >= cumTotalWeight) { + nextRandomNode = node; + } + cumTotalWeight += weight; + } + return nextRandomNode; + } finally { + rwLock.readLock().unlock(); + } + } + + @Override + public void setMaxProbabilityMultiplier(int max) { + this.maxProbabilityMultiplier = max; + } +} 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 6249ec5a438..f60f507447c 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 @@ -395,12 +395,17 @@ default int getStickyReadBookieIndex(LedgerMetadata metadata, Optional } /** - * returns true if the Ensemble is strictly adhering to placement policy, - * like in the case of RackawareEnsemblePlacementPolicy, bookies in the - * writeset are from 'minNumRacksPerWriteQuorum' number of racks. And in the - * case of RegionawareEnsemblePlacementPolicy, check for + * returns AdherenceLevel if the Ensemble is strictly/softly/fails adhering + * to placement policy, like in the case of + * RackawareEnsemblePlacementPolicy, bookies in the writeset are from + * 'minNumRacksPerWriteQuorum' number of racks. And in the case of + * RegionawareEnsemblePlacementPolicy, check for * minimumRegionsForDurability, reppRegionsToWrite, rack distribution within - * a region and other parameters of RegionAwareEnsemblePlacementPolicy. + * a region and other parameters of RegionAwareEnsemblePlacementPolicy. In + * ZoneAwareEnsemblePlacementPolicy if bookies in the writeset are from + * 'desiredNumOfZones' then it is considered as MEETS_STRICT if they are + * from 'minNumOfZones' then it is considered as MEETS_SOFT otherwise + * considered as FAIL. * * @param ensembleList * list of BookieSocketAddress of bookies in the ensemble @@ -410,9 +415,9 @@ default int getStickyReadBookieIndex(LedgerMetadata metadata, Optional * ackQuorumSize of the ensemble * @return */ - default boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, - int ackQuorumSize) { - return false; + default PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + int writeQuorumSize, int ackQuorumSize) { + return PlacementPolicyAdherence.FAIL; } /** @@ -434,28 +439,49 @@ default boolean areAckedBookiesAdheringToPlacementPolicy(Set { private final T result; - private final boolean adhering; + private final PlacementPolicyAdherence policyAdherence; - public static PlacementResult of(T result, boolean adhering) { - return new PlacementResult<>(result, adhering); + public static PlacementResult of(T result, PlacementPolicyAdherence policyAdherence) { + return new PlacementResult<>(result, policyAdherence); } - private PlacementResult(T result, boolean adhering) { + private PlacementResult(T result, PlacementPolicyAdherence policyAdherence) { this.result = result; - this.adhering = adhering; + this.policyAdherence = policyAdherence; } public T getResult() { return result; } - public boolean isStrictlyAdheringToPolicy() { - return adhering; + public PlacementPolicyAdherence isAdheringToPolicy() { + return policyAdherence; } } } 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 86fc3919608..13eaddde125 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 @@ -404,7 +404,7 @@ protected PlacementResult> newEnsembleInternal( for (BookieNode bn : bns) { addrs.add(bn.getAddr()); } - return PlacementResult.of(addrs, false); + return PlacementResult.of(addrs, PlacementPolicyAdherence.FAIL); } for (int i = 0; i < ensembleSize; i++) { @@ -492,32 +492,6 @@ public PlacementResult replaceBookie(int ensembleSize, int } } - @Override - public void updateBookieInfo(Map bookieInfoMap) { - if (!isWeighted) { - LOG.info("bookieFreeDiskInfo callback called even without weighted placement policy being used."); - return; - } - List allBookies = new ArrayList(knownBookies.values()); - - // create a new map to reflect the new mapping - Map map = new HashMap(); - for (BookieNode bookie : allBookies) { - if (bookieInfoMap.containsKey(bookie.getAddr())) { - map.put(bookie, bookieInfoMap.get(bookie.getAddr())); - } else { - map.put(bookie, new BookieInfo()); - } - } - rwLock.writeLock().lock(); - try { - this.bookieInfoMap = map; - this.weightedSelection.updateMap(this.bookieInfoMap); - } finally { - rwLock.writeLock().unlock(); - } - } - @Override public BookieNode selectFromNetworkLocation( String networkLoc, @@ -1033,8 +1007,8 @@ DistributionSchedule.WriteSet reorderReadSequenceWithRegion( // this method should be called in readlock scope of 'rwlock' @Override - public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, - int ackQuorumSize) { + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + int writeQuorumSize, int ackQuorumSize) { int ensembleSize = ensembleList.size(); int minNumRacksPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, minNumRacksPerWriteQuorum); HashSet racksInQuorum = new HashSet(); @@ -1056,10 +1030,10 @@ public boolean isEnsembleAdheringToPlacementPolicy(List ens } if ((racksInQuorum.size() < minNumRacksPerWriteQuorumForThisEnsemble) || (enforceMinNumRacksPerWriteQuorum && racksInQuorum.contains(getDefaultRack()))) { - return false; + return PlacementPolicyAdherence.FAIL; } } - return true; + return PlacementPolicyAdherence.MEETS_STRICT; } @Override 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 5ed8dcab015..16bdcfa3862 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 @@ -576,14 +576,14 @@ public final DistributionSchedule.WriteSet reorderReadLACSequence( } @Override - public boolean isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, - int ackQuorumSize) { + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + int writeQuorumSize, int ackQuorumSize) { /** * TODO: have to implement actual logic for this method for * RegionAwareEnsemblePlacementPolicy. For now return true value. * * - https://github.com/apache/bookkeeper/issues/1898 */ - return true; + return PlacementPolicyAdherence.MEETS_STRICT; } } 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 27b077d1bd6..6ad06e445dc 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 @@ -57,7 +57,7 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements ITopologyAwareEnsemblePlacementPolicy { static final Logger LOG = LoggerFactory.getLogger(TopologyAwareEnsemblePlacementPolicy.class); - + public static final String REPP_DNS_RESOLVER_CLASS = "reppDnsResolverClass"; protected final Map knownBookies = new HashMap(); protected final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); protected Map bookieInfoMap = new HashMap(); @@ -746,6 +746,31 @@ public void onBookieRackChange(List bookieAddressList) { } } + @Override + public void updateBookieInfo(Map bookieInfoMap) { + if (!isWeighted) { + LOG.info("bookieFreeDiskInfo callback called even without weighted placement policy being used."); + return; + } + rwLock.writeLock().lock(); + try { + List allBookies = new ArrayList(knownBookies.values()); + // create a new map to reflect the new mapping + Map map = new HashMap(); + for (BookieNode bookie : allBookies) { + if (bookieInfoMap.containsKey(bookie.getAddr())) { + map.put(bookie, bookieInfoMap.get(bookie.getAddr())); + } else { + map.put(bookie, new BookieInfo()); + } + } + this.bookieInfoMap = map; + this.weightedSelection.updateMap(this.bookieInfoMap); + } finally { + rwLock.writeLock().unlock(); + } + } + protected BookieNode createBookieNode(BookieSocketAddress addr) { return new BookieNode(addr, resolveNetworkLocation(addr)); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicy.java new file mode 100644 index 00000000000..265b0ca6441 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicy.java @@ -0,0 +1,130 @@ +/** + * 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.client; + +import io.netty.util.HashedWheelTimer; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.feature.FeatureProvider; +import org.apache.bookkeeper.net.BookieNode; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.stats.StatsLogger; + +/** + * A placement policy implementation use zone information for placing ensembles. + * + * @see EnsemblePlacementPolicy + */ +public class ZoneawareEnsemblePlacementPolicy extends ZoneawareEnsemblePlacementPolicyImpl + implements ITopologyAwareEnsemblePlacementPolicy { + ZoneawareEnsemblePlacementPolicyImpl slave = null; + + public ZoneawareEnsemblePlacementPolicy() { + super(); + } + + @Override + public EnsemblePlacementPolicy initialize(ClientConfiguration conf, + Optional optionalDnsResolver, HashedWheelTimer timer, FeatureProvider featureProvider, + StatsLogger statsLogger) { + if (conf.getNetworkTopologyStabilizePeriodSeconds() > 0) { + ClientConfiguration confClone = new ClientConfiguration(conf); + confClone.setNetworkTopologyStabilizePeriodSeconds(0); + super.initialize(confClone, optionalDnsResolver, timer, featureProvider, statsLogger); + slave = new ZoneawareEnsemblePlacementPolicyImpl(); + slave.initialize(conf, optionalDnsResolver, timer, featureProvider, statsLogger); + } else { + super.initialize(conf, optionalDnsResolver, timer, featureProvider, statsLogger); + slave = null; + } + return this; + } + + @Override + public void uninitalize() { + super.uninitalize(); + if (null != slave) { + slave.uninitalize(); + } + } + + @Override + public Set onClusterChanged(Set writableBookies, + Set readOnlyBookies) { + Set deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies); + if (null != slave) { + deadBookies = slave.onClusterChanged(writableBookies, readOnlyBookies); + } + return deadBookies; + } + + @Override + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) + throws BKException.BKNotEnoughBookiesException { + try { + return super.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, excludeBookies); + } catch (BKException.BKNotEnoughBookiesException bnebe) { + if (slave == null) { + throw bnebe; + } else { + return slave.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, excludeBookies); + } + } + } + + @Override + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, + BookieSocketAddress bookieToReplace, Set excludeBookies) + throws BKException.BKNotEnoughBookiesException { + try { + return super.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, + currentEnsemble, bookieToReplace, excludeBookies); + } catch (BKException.BKNotEnoughBookiesException bnebe) { + if (slave == null) { + throw bnebe; + } else { + return slave.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, + currentEnsemble, bookieToReplace, excludeBookies); + } + } + } + + @Override + public void handleBookiesThatLeft(Set leftBookies) { + super.handleBookiesThatLeft(leftBookies); + if (null != slave) { + slave.handleBookiesThatLeft(leftBookies); + } + } + + @Override + public void handleBookiesThatJoined(Set joinedBookies) { + super.handleBookiesThatJoined(joinedBookies); + if (null != slave) { + slave.handleBookiesThatJoined(joinedBookies); + } + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java new file mode 100644 index 00000000000..dc15c9b2471 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java @@ -0,0 +1,995 @@ +/** + * 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.client; + +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_JOINED; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_LEFT; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER; +import static org.apache.bookkeeper.client.BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_FAULTDOMAIN; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; + +import io.netty.util.HashedWheelTimer; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; +import org.apache.bookkeeper.common.util.ReflectionUtils; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.Configurable; +import org.apache.bookkeeper.feature.FeatureProvider; +import org.apache.bookkeeper.net.BookieNode; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.net.NetworkTopology; +import org.apache.bookkeeper.net.NetworkTopologyImpl; +import org.apache.bookkeeper.net.Node; +import org.apache.bookkeeper.net.NodeBase; +import org.apache.bookkeeper.net.ScriptBasedMapping; +import org.apache.bookkeeper.net.StabilizeNetworkTopology; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Simple zoneaware ensemble placement policy. + */ +public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsemblePlacementPolicy { + + static final Logger LOG = LoggerFactory.getLogger(ZoneawareEnsemblePlacementPolicyImpl.class); + + public static final String UNKNOWN_ZONE = "UnknownZone"; + /* + * this defaultFaultDomain is used as placeholder network location for + * bookies for which network location can't be resolved. In + * ZoneawareEnsemblePlacementPolicyImpl zone is the fault domain and upgrade + * domain is logical concept to enable parallel patching by bringing down + * all the bookies in the upgrade domain. + */ + private String defaultFaultDomain = NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN; + protected ZoneAwareNodeLocation unresolvedNodeLocation = new ZoneAwareNodeLocation( + NetworkTopology.DEFAULT_ZONE, NetworkTopology.DEFAULT_UPGRADEDOMAIN); + private final Random rand; + protected StatsLogger statsLogger = null; + // Use a loading cache so slow bookies are expired. Use entryId as values. + protected Cache slowBookies; + protected BookieNode myNode = null; + protected String myZone = null; + protected boolean reorderReadsRandom = false; + protected int stabilizePeriodSeconds = 0; + protected int reorderThresholdPendingRequests = 0; + protected int maxWeightMultiple; + protected int minNumZonesPerWriteQuorum; + protected int desiredNumZonesPerWriteQuorum; + protected boolean enforceStrictZoneawarePlacement; + protected HashedWheelTimer timer; + protected final ConcurrentMap address2NodePlacement; + + @StatsDoc(name = FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER, help = "Counter for number of times" + + " DNSResolverDecorator failed to resolve Network Location") + protected Counter failedToResolveNetworkLocationCounter = null; + @StatsDoc(name = NUM_WRITABLE_BOOKIES_IN_DEFAULT_FAULTDOMAIN, help = "Gauge for the number of writable" + + " Bookies in default fault domain") + protected Gauge numWritableBookiesInDefaultFaultDomain; + + /** + * Zone and UpgradeDomain pair of a node. + */ + public static class ZoneAwareNodeLocation { + private final String zone; + private final String upgradeDomain; + private final String repString; + + public ZoneAwareNodeLocation(String zone, String upgradeDomain) { + this.zone = zone; + this.upgradeDomain = upgradeDomain; + repString = zone + upgradeDomain; + } + + public String getZone() { + return zone; + } + + public String getUpgradeDomain() { + return upgradeDomain; + } + + @Override + public int hashCode() { + return repString.hashCode(); + } + + @Override + public boolean equals(Object obj) { + return ((obj instanceof ZoneAwareNodeLocation) + && repString.equals(((ZoneAwareNodeLocation) obj).repString)); + } + } + + + ZoneawareEnsemblePlacementPolicyImpl() { + super(); + address2NodePlacement = new ConcurrentHashMap(); + rand = new Random(System.currentTimeMillis()); + } + + protected ZoneAwareNodeLocation getZoneAwareNodeLocation(BookieSocketAddress addr) { + ZoneAwareNodeLocation nodeLocation = address2NodePlacement.get(addr); + if (null == nodeLocation) { + String networkLocation = resolveNetworkLocation(addr); + if (getDefaultFaultDomain().equals(networkLocation)) { + nodeLocation = unresolvedNodeLocation; + } else { + String[] parts = StringUtils.split(NodeBase.normalize(networkLocation), NodeBase.PATH_SEPARATOR); + if (parts.length != 2) { + nodeLocation = unresolvedNodeLocation; + } else { + nodeLocation = new ZoneAwareNodeLocation(NodeBase.PATH_SEPARATOR_STR + parts[0], + NodeBase.PATH_SEPARATOR_STR + parts[1]); + } + } + address2NodePlacement.putIfAbsent(addr, nodeLocation); + } + return nodeLocation; + } + + protected ZoneAwareNodeLocation getZoneAwareNodeLocation(BookieNode node) { + if (null == node || null == node.getAddr()) { + return unresolvedNodeLocation; + } + return getZoneAwareNodeLocation(node.getAddr()); + } + + @Override + public EnsemblePlacementPolicy initialize(ClientConfiguration conf, + Optional optionalDnsResolver, HashedWheelTimer timer, FeatureProvider featureProvider, + StatsLogger statsLogger) { + this.statsLogger = statsLogger; + this.timer = timer; + this.bookiesJoinedCounter = statsLogger.getOpStatsLogger(BOOKIES_JOINED); + this.bookiesLeftCounter = statsLogger.getOpStatsLogger(BOOKIES_LEFT); + this.failedToResolveNetworkLocationCounter = statsLogger.getCounter(FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER); + this.numWritableBookiesInDefaultFaultDomain = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + rwLock.readLock().lock(); + try { + return topology.countNumOfAvailableNodes(getDefaultFaultDomain(), Collections.emptySet()); + } finally { + rwLock.readLock().unlock(); + } + } + }; + this.statsLogger.registerGauge(NUM_WRITABLE_BOOKIES_IN_DEFAULT_FAULTDOMAIN, + numWritableBookiesInDefaultFaultDomain); + this.reorderThresholdPendingRequests = conf.getReorderThresholdPendingRequests(); + this.isWeighted = conf.getDiskWeightBasedPlacementEnabled(); + if (this.isWeighted) { + this.maxWeightMultiple = conf.getBookieMaxWeightMultipleForWeightBasedPlacement(); + this.weightedSelection = new DynamicWeightedRandomSelectionImpl(this.maxWeightMultiple); + LOG.info("Weight based placement with max multiple of {}", this.maxWeightMultiple); + } else { + LOG.info("Not weighted"); + } + this.minNumZonesPerWriteQuorum = conf.getMinNumZonesPerWriteQuorum(); + this.desiredNumZonesPerWriteQuorum = conf.getDesiredNumZonesPerWriteQuorum(); + this.enforceStrictZoneawarePlacement = conf.getEnforceStrictZoneawarePlacement(); + if (minNumZonesPerWriteQuorum > desiredNumZonesPerWriteQuorum) { + LOG.error( + "It is misconfigured, for ZoneawareEnsemblePlacementPolicy, minNumZonesPerWriteQuorum: {} cann't be" + + " greater than desiredNumZonesPerWriteQuorum: {}", + minNumZonesPerWriteQuorum, desiredNumZonesPerWriteQuorum); + throw new IllegalArgumentException("minNumZonesPerWriteQuorum: " + minNumZonesPerWriteQuorum + + " cann't be greater than desiredNumZonesPerWriteQuorum: " + desiredNumZonesPerWriteQuorum); + } + DNSToSwitchMapping actualDNSResolver; + if (optionalDnsResolver.isPresent()) { + actualDNSResolver = optionalDnsResolver.get(); + } else { + String dnsResolverName = conf.getString(REPP_DNS_RESOLVER_CLASS, ScriptBasedMapping.class.getName()); + actualDNSResolver = ReflectionUtils.newInstance(dnsResolverName, DNSToSwitchMapping.class); + if (actualDNSResolver instanceof Configurable) { + ((Configurable) actualDNSResolver).setConf(conf); + } + } + + this.dnsResolver = new DNSResolverDecorator(actualDNSResolver, () -> this.getDefaultFaultDomain(), + failedToResolveNetworkLocationCounter); + this.stabilizePeriodSeconds = conf.getNetworkTopologyStabilizePeriodSeconds(); + // create the network topology + if (stabilizePeriodSeconds > 0) { + this.topology = new StabilizeNetworkTopology(timer, stabilizePeriodSeconds); + } else { + this.topology = new NetworkTopologyImpl(); + } + try { + myNode = createBookieNode(new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), 0)); + myZone = getZoneAwareNodeLocation(myNode).getZone(); + } catch (UnknownHostException e) { + LOG.error("Failed to get local host address : ", e); + throw new RuntimeException(e); + } + LOG.info("Initialized zoneaware ensemble placement policy @ {} @ {} : {}.", myNode, + myNode.getNetworkLocation(), dnsResolver.getClass().getName()); + + slowBookies = CacheBuilder.newBuilder() + .expireAfterWrite(conf.getBookieFailureHistoryExpirationMSec(), TimeUnit.MILLISECONDS) + .build(new CacheLoader() { + @Override + public Long load(BookieSocketAddress key) throws Exception { + return -1L; + } + }); + return this; + } + + public ZoneawareEnsemblePlacementPolicyImpl withDefaultFaultDomain(String defaultFaultDomain) { + checkNotNull(defaultFaultDomain, "Default fault domain cannot be null"); + + String[] parts = StringUtils.split(NodeBase.normalize(defaultFaultDomain), NodeBase.PATH_SEPARATOR); + if (parts.length != 2) { + LOG.error("provided defaultFaultDomain: {} is not valid", defaultFaultDomain); + throw new IllegalArgumentException("invalid defaultFaultDomain"); + } else { + unresolvedNodeLocation = new ZoneAwareNodeLocation(NodeBase.PATH_SEPARATOR_STR + parts[0], + NodeBase.PATH_SEPARATOR_STR + parts[1]); + } + + this.defaultFaultDomain = defaultFaultDomain; + return this; + } + + public String getDefaultFaultDomain() { + return defaultFaultDomain; + } + + @Override + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Set excludeBookies, + org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble parentEnsemble, + org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate parentPredicate) + throws BKNotEnoughBookiesException { + throw new UnsupportedOperationException( + "newEnsemble method with parentEnsemble and parentPredicate is not supported for " + + "ZoneawareEnsemblePlacementPolicyImpl"); + } + + @Override + public BookieNode selectFromNetworkLocation(String networkLoc, Set excludeBookies, + org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate predicate, + org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble ensemble, + boolean fallbackToRandom) throws BKNotEnoughBookiesException { + throw new UnsupportedOperationException( + "selectFromNetworkLocation is not supported for ZoneawareEnsemblePlacementPolicyImpl"); + } + + @Override + public BookieNode selectFromNetworkLocation(Set excludeRacks, Set excludeBookies, + org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate predicate, + org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble ensemble, + boolean fallbackToRandom) throws BKNotEnoughBookiesException { + throw new UnsupportedOperationException( + "selectFromNetworkLocation is not supported for ZoneawareEnsemblePlacementPolicyImpl"); + } + + @Override + public BookieNode selectFromNetworkLocation(String networkLoc, Set excludeRacks, Set excludeBookies, + org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate predicate, + org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble ensemble, + boolean fallbackToRandom) throws BKNotEnoughBookiesException { + throw new UnsupportedOperationException( + "selectFromNetworkLocation is not supported for ZoneawareEnsemblePlacementPolicyImpl"); + } + + @Override + public void uninitalize() { + } + + @Override + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) + throws BKNotEnoughBookiesException { + if (enforceStrictZoneawarePlacement) { + if (ensembleSize % writeQuorumSize != 0) { + /* + * if ensembleSize is not multiple of writeQuorumSize, then the + * write quorums which are wrapped will have bookies from just + * minNumberOfZones though bookies are available from + * desiredNumZones. + * + * lets say for example - desiredZones = 3, minZones = 2, + * ensembleSize = 5, writeQuorumSize = 3, ackQuorumSize = 2 + * + * z1, z2, z3, z1, z2 is a legal ensemble. (lets assume here z1 + * represents a node belonging to zone z1) + * + * the writeQuorum for entry 3 will be z1, z2 and z1, since + * ackQuorumSize is 2, an entry could be written just to two + * bookies that belong to z1. If the zone z1 goes down then the + * entry could potentially be unavailable until the zone z1 has + * come back. + * + * Also, it is not ideal to allow combination which fallsback to + * minZones, when bookies are available from desiredNumZones. + * + * So prohibiting this combination of configuration. + */ + LOG.error("It is illegal for ensembleSize to be not multiple of" + + " writeQuorumSize When StrictZoneawarePlacement is enabled"); + throw new IllegalArgumentException("It is illegal for ensembleSize to be not multiple of" + + " writeQuorumSize When StrictZoneawarePlacement is enabled"); + } + if (writeQuorumSize <= minNumZonesPerWriteQuorum) { + /* + * if we allow writeQuorumSize <= minNumZonesPerWriteQuorum, + * then replaceBookie may fail to find a candidate to replace a + * node when a zone goes down. + * + * lets say for example - desiredZones = 3, minZones = 2, + * ensembleSize = 6, writeQuorumSize = 2, ackQuorumSize = 2 + * + * z1, z2, z3, z1, z2, z3 is a legal ensemble. (lets assume here + * z1 represents a node belonging to zone z1) + * + * Now if Zone z2 goes down, you need to replace Index 1 and 4. + * To replace index 1, you need to find a zone that is not z1 + * and Z3 which is not possible. + * + * So prohibiting this combination of configuration. + */ + LOG.error("It is illegal for writeQuorumSize to be lesser than or equal" + + " to minNumZonesPerWriteQuorum When StrictZoneawarePlacement is enabled"); + throw new IllegalArgumentException("It is illegal for writeQuorumSize to be lesser than or equal" + + " to minNumZonesPerWriteQuorum When StrictZoneawarePlacement is enabled"); + } + } + int desiredNumZonesPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, desiredNumZonesPerWriteQuorum); + List newEnsemble = new ArrayList( + Collections.nCopies(ensembleSize, null)); + rwLock.readLock().lock(); + try { + if (!enforceStrictZoneawarePlacement) { + return createNewEnsembleRandomly(newEnsemble, writeQuorumSize, ackQuorumSize, customMetadata, + excludeBookies); + } + Set comprehensiveExclusionBookiesSet = addDefaultFaultDomainBookies(excludeBookies); + for (int index = 0; index < ensembleSize; index++) { + BookieSocketAddress selectedBookie = setBookieInTheEnsemble(ensembleSize, writeQuorumSize, newEnsemble, + newEnsemble, index, desiredNumZonesPerWriteQuorumForThisEnsemble, + comprehensiveExclusionBookiesSet); + comprehensiveExclusionBookiesSet.add(selectedBookie); + } + return PlacementResult.of(newEnsemble, + isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); + } finally { + rwLock.readLock().unlock(); + } + } + + @Override + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, + BookieSocketAddress bookieToReplace, Set excludeBookies) + throws BKNotEnoughBookiesException { + int bookieToReplaceIndex = currentEnsemble.indexOf(bookieToReplace); + int desiredNumZonesPerWriteQuorumForThisEnsemble = (writeQuorumSize < desiredNumZonesPerWriteQuorum) + ? writeQuorumSize : desiredNumZonesPerWriteQuorum; + List newEnsemble = new ArrayList(currentEnsemble); + rwLock.readLock().lock(); + try { + if (!enforceStrictZoneawarePlacement) { + return selectBookieRandomly(newEnsemble, bookieToReplace, excludeBookies, writeQuorumSize, + ackQuorumSize); + } + Set comprehensiveExclusionBookiesSet = addDefaultFaultDomainBookies(excludeBookies); + comprehensiveExclusionBookiesSet.addAll(currentEnsemble); + BookieSocketAddress candidateAddr = setBookieInTheEnsemble(ensembleSize, writeQuorumSize, currentEnsemble, + newEnsemble, bookieToReplaceIndex, desiredNumZonesPerWriteQuorumForThisEnsemble, + comprehensiveExclusionBookiesSet); + return PlacementResult.of(candidateAddr, + isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); + } finally { + rwLock.readLock().unlock(); + } + } + + private PlacementResult> createNewEnsembleRandomly(List newEnsemble, + int writeQuorumSize, int ackQuorumSize, Map customMetadata, + Set excludeBookies) throws BKNotEnoughBookiesException { + int ensembleSize = newEnsemble.size(); + Set bookiesToConsider = getBookiesToConsider(excludeBookies); + if (bookiesToConsider.size() < newEnsemble.size()) { + LOG.error("Not enough bookies are available to form ensemble of size: {}", newEnsemble.size()); + throw new BKNotEnoughBookiesException(); + } + + for (int i = 0; i < ensembleSize; i++) { + BookieNode candidateNode = selectCandidateNode(bookiesToConsider); + newEnsemble.set(i, candidateNode.getAddr()); + bookiesToConsider.remove(candidateNode); + } + return PlacementResult.of(newEnsemble, + isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); + } + + private PlacementResult selectBookieRandomly(List newEnsemble, + BookieSocketAddress bookieToReplace, Set excludeBookies, int writeQuorumSize, + int ackQuorumSize) throws BKNotEnoughBookiesException { + Set bookiesToExcludeIncludingEnsemble = new HashSet(excludeBookies); + bookiesToExcludeIncludingEnsemble.addAll(newEnsemble); + Set bookiesToConsider = getBookiesToConsider(bookiesToExcludeIncludingEnsemble); + int bookieToReplaceIndex = newEnsemble.indexOf(bookieToReplace); + + if (bookiesToConsider.isEmpty()) { + LOG.error("There is no bookie available to replace a bookie"); + throw new BKNotEnoughBookiesException(); + } + BookieSocketAddress candidateAddr = (selectCandidateNode(bookiesToConsider)).getAddr(); + newEnsemble.set(bookieToReplaceIndex, candidateAddr); + return PlacementResult.of(candidateAddr, + isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); + } + + private Set getBookiesToConsider(Set excludeBookies) { + Set leaves = topology.getLeaves(NodeBase.ROOT); + Set bookiesToConsider = new HashSet(); + BookieNode bookieNode; + for (Node leaf : leaves) { + if (leaf instanceof BookieNode) { + bookieNode = ((BookieNode) leaf); + if (excludeBookies.contains(bookieNode.getAddr())) { + continue; + } + bookiesToConsider.add(bookieNode); + } + } + return bookiesToConsider; + } + + /* + * This method finds the appropriate bookie for newEnsemble by finding + * bookie to replace at bookieToReplaceIndex in the currentEnsemble. + * + * It goes through following filtering process 1) Exclude zones of + * desiredNumZonesPerWriteQuorumForThisEnsemble neighboring nodes 2) Find + * bookies to consider by excluding zones (found from previous step) and + * excluding UDs of the zones to consider. 3) If it can't find eligible + * bookie, then keep reducing the number of neighboring nodes to + * minNumZonesPerWriteQuorum and repeat step 2. 4) If it still can't find + * eligible bookies then find the zones to exclude such that in a writeset + * there will be bookies from atleast minNumZonesPerWriteQuorum zones and + * repeat step 2 5) After getting the list of eligible candidates select a + * node randomly. 6) If step-4 couldn't find eligible candidates then throw + * BKNotEnoughBookiesException. + * + * Example: Ensemble:6 Qw:6 desiredNumZonesPerWriteQuorumForThisEnsemble:3 + * minNumZonesPerWriteQuorum:2 The selection process is as follows: + * + * 1) Find bookies by excluding zones of + * (desiredNumZonesPerWriteQuorumForThisEnsemble -1) neighboring bookies on + * the left and and the right side of the bookieToReplaceIndex. i.e Zones of + * 2 bookies(3-1) on both sides of the index in question will be excluded to + * find bookies. 2) Get the set of zones of the bookies selected above. 3) + * Get the UpgradeDomains to exclude of the each zone selected above to make + * sure bookies of writeSets containing bookieToReplaceIndex are from + * different UD if they belong to same zone. 4) now from the zones selected + * in step 2, apply the filter of UDs to exclude found in previous step and + * get the eligible bookies. 5) If no bookie matches this filter, then + * instead of aiming for unique UDs, fallback to UDs to exclude such that if + * bookies are from same zone in the writeSets containing + * bookieToReplaceIndex then they must be atleast from 2 different UDs. 6) + * now from the zones selected in step 2, apply the filter of UDs to exclude + * found in previous step and get the eligible bookies. 7) If no bookie + * matches this filter, repeat from Step1 to Step6 by decreasing neighboring + * exclude zones from (desiredNumZonesPerWriteQuorumForThisEnsemble - 1), + * which is 2 to (minNumZonesPerWriteQuorum - 1), which is 1 8) If even + * after this, bookies are not found matching the criteria fallback to + * minNumZonesPerWriteQuorum, for this find the zones to exclude such that + * in writesets containing this bookieToReplaceIndex there will be bookies + * from atleast minNumZonesPerWriteQuorum zones, which is 2. 9) Get the set + * of the zones of the bookies by excluding zones selected above. 10) repeat + * Step3 to Step6. 11) After getting the list of eligible candidates select + * a node randomly. 12) If even after Step10 there are no eligible + * candidates then throw BKNotEnoughBookiesException. + */ + private BookieSocketAddress setBookieInTheEnsemble(int ensembleSize, int writeQuorumSize, + List currentEnsemble, List newEnsemble, int bookieToReplaceIndex, + int desiredNumZonesPerWriteQuorumForThisEnsemble, Set excludeBookies) + throws BKNotEnoughBookiesException { + BookieSocketAddress bookieToReplace = currentEnsemble.get(bookieToReplaceIndex); + Set zonesToExclude = null; + Set bookiesToConsiderAfterExcludingZonesAndUDs = null; + for (int numberOfNeighborsToConsider = (desiredNumZonesPerWriteQuorumForThisEnsemble + - 1); numberOfNeighborsToConsider >= (minNumZonesPerWriteQuorum - 1); numberOfNeighborsToConsider--) { + zonesToExclude = getZonesOfNeighboringNodesInEnsemble(currentEnsemble, bookieToReplaceIndex, + (numberOfNeighborsToConsider)); + bookiesToConsiderAfterExcludingZonesAndUDs = getBookiesToConsiderAfterExcludingZonesAndUDs(ensembleSize, + writeQuorumSize, currentEnsemble, bookieToReplaceIndex, excludeBookies, zonesToExclude); + if (!bookiesToConsiderAfterExcludingZonesAndUDs.isEmpty()) { + break; + } + } + if (bookiesToConsiderAfterExcludingZonesAndUDs.isEmpty()) { + zonesToExclude = getZonesToExcludeToMaintainMinZones(currentEnsemble, bookieToReplaceIndex, + writeQuorumSize); + bookiesToConsiderAfterExcludingZonesAndUDs = getBookiesToConsiderAfterExcludingZonesAndUDs(ensembleSize, + writeQuorumSize, currentEnsemble, bookieToReplaceIndex, excludeBookies, zonesToExclude); + } + if (bookiesToConsiderAfterExcludingZonesAndUDs.isEmpty()) { + LOG.error("Not enough bookies are available to replaceBookie : {} in ensemble : {} with excludeBookies {}.", + bookieToReplace, currentEnsemble, excludeBookies); + throw new BKNotEnoughBookiesException(); + } + + BookieSocketAddress candidateAddr = selectCandidateNode(bookiesToConsiderAfterExcludingZonesAndUDs).getAddr(); + newEnsemble.set(bookieToReplaceIndex, candidateAddr); + return candidateAddr; + } + + /* + * this method should be called in readlock scope of 'rwLock'. This method + * returns a new set, by adding excludedBookies and bookies in + * defaultfaultdomain. + */ + protected Set addDefaultFaultDomainBookies(Set excludeBookies) { + Set comprehensiveExclusionBookiesSet = new HashSet(excludeBookies); + Set defaultFaultDomainLeaves = topology.getLeaves(getDefaultFaultDomain()); + for (Node node : defaultFaultDomainLeaves) { + if (node instanceof BookieNode) { + comprehensiveExclusionBookiesSet.add(((BookieNode) node).getAddr()); + } else { + LOG.error("found non-BookieNode: {} as leaf of defaultFaultDomain: {}", node, getDefaultFaultDomain()); + } + } + return comprehensiveExclusionBookiesSet; + } + + /* + * Select bookie randomly from the bookiesToConsiderAfterExcludingUDs set. + * If diskWeightBasedPlacement is enabled then it will select node randomly + * based on node weight. + */ + private BookieNode selectCandidateNode(Set bookiesToConsiderAfterExcludingUDs) { + BookieNode candidate = null; + if (!this.isWeighted) { + int randSelIndex = rand.nextInt(bookiesToConsiderAfterExcludingUDs.size()); + int ind = 0; + for (BookieNode bookieNode : bookiesToConsiderAfterExcludingUDs) { + if (ind == randSelIndex) { + candidate = bookieNode; + break; + } + ind++; + } + } else { + candidate = weightedSelection.getNextRandom(bookiesToConsiderAfterExcludingUDs); + } + return candidate; + } + + private String getExcludedZonesString(Set excludeZones) { + if (excludeZones.isEmpty()) { + return ""; + } + StringBuilder excludedZonesString = new StringBuilder("~"); + boolean firstZone = true; + for (String excludeZone : excludeZones) { + if (!firstZone) { + excludedZonesString.append(NetworkTopologyImpl.NODE_SEPARATOR); + } + excludedZonesString.append(excludeZone); + firstZone = false; + } + return excludedZonesString.toString(); + } + + private Set getBookiesToConsider(String excludedZonesString, Set excludeBookies) { + Set bookiesToConsider = new HashSet(); + Set leaves = topology.getLeaves(excludedZonesString); + for (Node leaf : leaves) { + BookieNode bookieNode = ((BookieNode) leaf); + if (excludeBookies.contains(bookieNode.getAddr())) { + continue; + } + bookiesToConsider.add(bookieNode); + } + return bookiesToConsider; + } + + /* + * For the position of 'bookieToReplaceIndex' in currentEnsemble, get the + * set of bookies eligible by excluding the 'excludeZones' and + * 'excludeBookies'. After excluding excludeZones and excludeBookies, it + * would first try to exclude upgrade domains of neighboring nodes + * (writeset) so the bookie would be from completely new upgrade domain + * of a zone, if a writeset contains bookie from the zone. If Bookie is + * not found matching this criteria, then it will fallback to maintain min + * upgrade domains (two) from a zone, such that if multiple bookies in a + * write quorum are from the same zone then they will be spread across two + * upgrade domains. + */ + private Set getBookiesToConsiderAfterExcludingZonesAndUDs(int ensembleSize, int writeQuorumSize, + List currentEnsemble, int bookieToReplaceIndex, + Set excludeBookies, Set excludeZones) { + Set bookiesToConsiderAfterExcludingZonesAndUDs = new HashSet(); + HashMap> excludingUDsOfZonesToConsider = new HashMap>(); + Set bookiesToConsiderAfterExcludingZones = getBookiesToConsider( + getExcludedZonesString(excludeZones), excludeBookies); + + if (!bookiesToConsiderAfterExcludingZones.isEmpty()) { + Set zonesToConsider = getZonesOfBookies(bookiesToConsiderAfterExcludingZones); + for (String zoneToConsider : zonesToConsider) { + Set upgradeDomainsOfAZoneInNeighboringNodes = getUpgradeDomainsOfAZoneInNeighboringNodes( + currentEnsemble, bookieToReplaceIndex, writeQuorumSize, zoneToConsider); + excludingUDsOfZonesToConsider.put(zoneToConsider, upgradeDomainsOfAZoneInNeighboringNodes); + } + + updateBookiesToConsiderAfterExcludingZonesAndUDs(bookiesToConsiderAfterExcludingZonesAndUDs, + bookiesToConsiderAfterExcludingZones, excludingUDsOfZonesToConsider); + + /* + * If no eligible bookie is found, then instead of aiming for unique + * UDs, fallback to UDs to exclude such that if bookies are from + * same zone in the writeSets containing bookieToReplaceIndex then + * they must be atleast from 2 different UDs + */ + if (bookiesToConsiderAfterExcludingZonesAndUDs.isEmpty()) { + excludingUDsOfZonesToConsider.clear(); + for (String zoneToConsider : zonesToConsider) { + Set udsToExcludeToMaintainMinUDsInWriteQuorums = + getUDsToExcludeToMaintainMinUDsInWriteQuorums(currentEnsemble, bookieToReplaceIndex, + writeQuorumSize, zoneToConsider); + excludingUDsOfZonesToConsider.put(zoneToConsider, udsToExcludeToMaintainMinUDsInWriteQuorums); + } + + updateBookiesToConsiderAfterExcludingZonesAndUDs(bookiesToConsiderAfterExcludingZonesAndUDs, + bookiesToConsiderAfterExcludingZones, excludingUDsOfZonesToConsider); + } + } + return bookiesToConsiderAfterExcludingZonesAndUDs; + } + + /* + * Filter bookies which belong to excludingUDs of zones to consider from + * 'bookiesToConsider' set and add them to + * 'bookiesToConsiderAfterExcludingUDs' set. + */ + private void updateBookiesToConsiderAfterExcludingZonesAndUDs(Set bookiesToConsiderAfterExcludingUDs, + Set bookiesToConsider, HashMap> excludingUDsOfZonesToConsider) { + for (BookieNode bookieToConsider : bookiesToConsider) { + ZoneAwareNodeLocation nodeLocation = getZoneAwareNodeLocation(bookieToConsider); + if (excludingUDsOfZonesToConsider.get(nodeLocation.getZone()).contains(nodeLocation.getUpgradeDomain())) { + continue; + } + bookiesToConsiderAfterExcludingUDs.add(bookieToConsider); + } + } + + /* + * Gets the set of zones of neighboring nodes. + */ + private Set getZonesOfNeighboringNodesInEnsemble(List currentEnsemble, int indexOfNode, + int numOfNeighboringNodes) { + Set zonesOfNeighboringNodes = new HashSet(); + int ensembleSize = currentEnsemble.size(); + for (int i = (-1 * numOfNeighboringNodes); i <= numOfNeighboringNodes; i++) { + if (i == 0) { + continue; + } + int index = (indexOfNode + i + ensembleSize) % ensembleSize; + BookieSocketAddress addrofNode = currentEnsemble.get(index); + if (addrofNode == null) { + continue; + } + String zoneOfNode = getZoneAwareNodeLocation(addrofNode).getZone(); + zonesOfNeighboringNodes.add(zoneOfNode); + } + return zonesOfNeighboringNodes; + } + + /* + * This method returns set of zones to exclude for the position of + * 'indexOfNode', so that writequorums, containing this index, would have + * atleast minNumZonesPerWriteQuorum. + */ + private Set getZonesToExcludeToMaintainMinZones(List currentEnsemble, int indexOfNode, + int writeQuorumSize) { + int ensSize = currentEnsemble.size(); + Set zonesToExclude = new HashSet(); + Set zonesInWriteQuorum = new HashSet(); + for (int i = -(writeQuorumSize - 1); i <= 0; i++) { + zonesInWriteQuorum.clear(); + for (int j = 0; j < writeQuorumSize; j++) { + int indexInEnsemble = (i + j + indexOfNode + ensSize) % ensSize; + if (indexInEnsemble == indexOfNode) { + continue; + } + BookieSocketAddress bookieAddr = currentEnsemble.get(indexInEnsemble); + if (bookieAddr == null) { + continue; + } + ZoneAwareNodeLocation nodeLocation = getZoneAwareNodeLocation(bookieAddr); + zonesInWriteQuorum.add(nodeLocation.getZone()); + } + if (zonesInWriteQuorum.size() <= (minNumZonesPerWriteQuorum - 1)) { + zonesToExclude.addAll(zonesInWriteQuorum); + } + } + return zonesToExclude; + } + + private Set getZonesOfBookies(Collection bookieNodes) { + Set zonesOfBookies = new HashSet(); + for (BookieNode bookieNode : bookieNodes) { + ZoneAwareNodeLocation nodeLocation = getZoneAwareNodeLocation(bookieNode); + zonesOfBookies.add(nodeLocation.getZone()); + } + return zonesOfBookies; + } + + /* + * Gets the set of upgradedomains of neighboring nodes (writeQuorumSize) + * which belong to this 'zone'. + */ + private Set getUpgradeDomainsOfAZoneInNeighboringNodes(List currentEnsemble, + int indexOfNode, int writeQuorumSize, String zone) { + int ensSize = currentEnsemble.size(); + Set upgradeDomainsOfAZoneInNeighboringNodes = new HashSet(); + for (int i = -(writeQuorumSize - 1); i <= (writeQuorumSize - 1); i++) { + if (i == 0) { + continue; + } + int indexInEnsemble = (indexOfNode + i + ensSize) % ensSize; + BookieSocketAddress bookieAddr = currentEnsemble.get(indexInEnsemble); + if (bookieAddr == null) { + continue; + } + ZoneAwareNodeLocation nodeLocation = getZoneAwareNodeLocation(bookieAddr); + if (nodeLocation.getZone().equals(zone)) { + upgradeDomainsOfAZoneInNeighboringNodes.add(nodeLocation.getUpgradeDomain()); + } + } + return upgradeDomainsOfAZoneInNeighboringNodes; + } + + /* + * This method returns set of UpgradeDomains to exclude if a bookie from + * the 'zone' has to be selected for the position of 'indexOfNode', then if + * there are multiple bookies from the 'zone' in a write quorum then they + * will be atleast from minimum of two upgrade domains. + */ + private Set getUDsToExcludeToMaintainMinUDsInWriteQuorums(List currentEnsemble, + int indexOfNode, int writeQuorumSize, String zone) { + int ensSize = currentEnsemble.size(); + Set upgradeDomainsToExclude = new HashSet(); + Set upgradeDomainsOfThisZoneInWriteQuorum = new HashSet(); + for (int i = -(writeQuorumSize - 1); i <= 0; i++) { + upgradeDomainsOfThisZoneInWriteQuorum.clear(); + for (int j = 0; j < writeQuorumSize; j++) { + int indexInEnsemble = (i + j + indexOfNode + ensSize) % ensSize; + if (indexInEnsemble == indexOfNode) { + continue; + } + BookieSocketAddress bookieAddr = currentEnsemble.get(indexInEnsemble); + if (bookieAddr == null) { + continue; + } + ZoneAwareNodeLocation nodeLocation = getZoneAwareNodeLocation(bookieAddr); + if (nodeLocation.getZone().equals(zone)) { + upgradeDomainsOfThisZoneInWriteQuorum.add(nodeLocation.getUpgradeDomain()); + } + } + if (upgradeDomainsOfThisZoneInWriteQuorum.size() == 1) { + upgradeDomainsToExclude.addAll(upgradeDomainsOfThisZoneInWriteQuorum); + } + } + return upgradeDomainsToExclude; + } + + @Override + public void registerSlowBookie(BookieSocketAddress bookieSocketAddress, long entryId) { + // TODO Auto-generated method stub + } + + @Override + public DistributionSchedule.WriteSet reorderReadSequence(List ensemble, + BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { + return writeSet; + } + + @Override + public DistributionSchedule.WriteSet reorderReadLACSequence(List ensemble, + BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { + DistributionSchedule.WriteSet retList = reorderReadSequence(ensemble, bookiesHealthInfo, writeSet); + retList.addMissingIndices(ensemble.size()); + return retList; + } + + /* + * In ZoneAwareEnsemblePlacementPolicy if bookies in the writeset are from + * 'desiredNumOfZones' then it is considered as MEETS_STRICT if they are + * from 'minNumOfZones' then it is considered as MEETS_SOFT otherwise + * considered as FAIL. Also in a writeset if there are multiple bookies from + * the same zone then they are expected to be from different upgrade + * domains. + */ + @Override + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + int writeQuorumSize, int ackQuorumSize) { + PlacementPolicyAdherence placementPolicyAdherence = PlacementPolicyAdherence.MEETS_STRICT; + rwLock.readLock().lock(); + try { + HashMap> bookiesLocationInWriteSet = new HashMap>(); + HashMap numOfBookiesInZones = new HashMap(); + BookieSocketAddress bookieNode; + if (ensembleList.size() % writeQuorumSize != 0) { + placementPolicyAdherence = PlacementPolicyAdherence.FAIL; + if (LOG.isDebugEnabled()) { + LOG.debug( + "For ensemble: {}, ensembleSize: {} is not a multiple of writeQuorumSize: {}", + ensembleList, ensembleList.size(), writeQuorumSize); + } + return placementPolicyAdherence; + } + if (writeQuorumSize <= minNumZonesPerWriteQuorum) { + placementPolicyAdherence = PlacementPolicyAdherence.FAIL; + if (LOG.isDebugEnabled()) { + LOG.debug( + "For ensemble: {}, writeQuorumSize: {} is less than or equal to" + + " minNumZonesPerWriteQuorum: {}", + ensembleList, writeQuorumSize, minNumZonesPerWriteQuorum); + } + return placementPolicyAdherence; + } + int desiredNumZonesPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, desiredNumZonesPerWriteQuorum); + for (int i = 0; i < ensembleList.size(); i++) { + bookiesLocationInWriteSet.clear(); + numOfBookiesInZones.clear(); + for (int j = 0; j < writeQuorumSize; j++) { + int indexOfNode = (i + j) % ensembleList.size(); + bookieNode = ensembleList.get(indexOfNode); + ZoneAwareNodeLocation nodeLocation = getZoneAwareNodeLocation(bookieNode); + if (nodeLocation.equals(unresolvedNodeLocation)) { + placementPolicyAdherence = PlacementPolicyAdherence.FAIL; + if (LOG.isDebugEnabled()) { + LOG.debug("ensemble: {}, contains bookie: {} for which network location is unresolvable", + ensembleList, bookieNode); + } + return placementPolicyAdherence; + } + String zone = nodeLocation.getZone(); + String upgradeDomain = nodeLocation.getUpgradeDomain(); + Set udsOfThisZoneInThisWriteSet = bookiesLocationInWriteSet.get(zone); + if (udsOfThisZoneInThisWriteSet == null) { + udsOfThisZoneInThisWriteSet = new HashSet(); + udsOfThisZoneInThisWriteSet.add(upgradeDomain); + bookiesLocationInWriteSet.put(zone, udsOfThisZoneInThisWriteSet); + numOfBookiesInZones.put(zone, 1); + } else { + udsOfThisZoneInThisWriteSet.add(upgradeDomain); + Integer numOfNodesInAZone = numOfBookiesInZones.get(zone); + numOfBookiesInZones.put(zone, (numOfNodesInAZone + 1)); + } + } + if (numOfBookiesInZones.entrySet().size() < minNumZonesPerWriteQuorum) { + placementPolicyAdherence = PlacementPolicyAdherence.FAIL; + if (LOG.isDebugEnabled()) { + LOG.debug("in ensemble: {}, writeset starting at: {} doesn't contain bookies from" + + " minNumZonesPerWriteQuorum: {}", ensembleList, i, minNumZonesPerWriteQuorum); + } + return placementPolicyAdherence; + } else if (numOfBookiesInZones.entrySet().size() >= desiredNumZonesPerWriteQuorumForThisEnsemble) { + if (!validateMinUDsAreMaintained(numOfBookiesInZones, bookiesLocationInWriteSet)) { + placementPolicyAdherence = PlacementPolicyAdherence.FAIL; + if (LOG.isDebugEnabled()) { + LOG.debug("in ensemble: {}, writeset starting at: {} doesn't maintain min of 2 UDs" + + " when there are multiple bookies from the same zone.", ensembleList, i); + } + return placementPolicyAdherence; + } + } else { + if (!validateMinUDsAreMaintained(numOfBookiesInZones, bookiesLocationInWriteSet)) { + placementPolicyAdherence = PlacementPolicyAdherence.FAIL; + if (LOG.isDebugEnabled()) { + LOG.debug("in ensemble: {}, writeset starting at: {} doesn't maintain min of 2 UDs" + + " when there are multiple bookies from the same zone.", ensembleList, i); + } + return placementPolicyAdherence; + } + if (placementPolicyAdherence == PlacementPolicyAdherence.MEETS_STRICT) { + placementPolicyAdherence = PlacementPolicyAdherence.MEETS_SOFT; + } + } + } + } finally { + rwLock.readLock().unlock(); + } + return placementPolicyAdherence; + } + + private boolean validateMinUDsAreMaintained(HashMap numOfNodesInZones, + HashMap> nodesLocationInWriteSet) { + for (Entry numOfNodesInZone : numOfNodesInZones.entrySet()) { + String zone = numOfNodesInZone.getKey(); + Integer numOfNodesInThisZone = numOfNodesInZone.getValue(); + if (numOfNodesInThisZone > 1) { + Set udsOfThisZone = nodesLocationInWriteSet.get(zone); + if (udsOfThisZone.size() < 2) { + return false; + } + } + } + return true; + } + + @Override + public boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, int writeQuorumSize, + int ackQuorumSize) { + HashSet zonesOfAckedBookies = new HashSet<>(); + int minNumZonesPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, minNumZonesPerWriteQuorum); + boolean areAckedBookiesAdheringToPlacementPolicy = false; + ReentrantReadWriteLock.ReadLock readLock = rwLock.readLock(); + readLock.lock(); + try { + for (BookieSocketAddress ackedBookie : ackedBookies) { + zonesOfAckedBookies.add(getZoneAwareNodeLocation(ackedBookie).getZone()); + } + areAckedBookiesAdheringToPlacementPolicy = ((zonesOfAckedBookies + .size() >= minNumZonesPerWriteQuorumForThisEnsemble) && (ackedBookies.size() >= ackQuorumSize)); + if (LOG.isDebugEnabled()) { + LOG.debug( + "areAckedBookiesAdheringToPlacementPolicy returning {}, because number of ackedBookies = {}," + + " number of Zones of ackedbookies = {}," + + " number of minNumZonesPerWriteQuorumForThisEnsemble = {}", + areAckedBookiesAdheringToPlacementPolicy, ackedBookies.size(), zonesOfAckedBookies.size(), + minNumZonesPerWriteQuorumForThisEnsemble); + } + } finally { + readLock.unlock(); + } + return areAckedBookiesAdheringToPlacementPolicy; + } +} 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 b666dbce304..4886d75bba3 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 @@ -165,6 +165,17 @@ public abstract class AbstractConfiguration // ignore usage of local node in the internal logic of placement policy public static final String IGNORE_LOCAL_NODE_IN_PLACEMENT_POLICY = "ignoreLocalNodeInPlacementPolicy"; + // minimum number of zones per write quorum in ZoneAwarePlacementPolicy + public static final String MIN_NUM_ZONES_PER_WRITE_QUORUM = "minNumZonesPerWriteQuorum"; + + // desired number of zones per write quorum in ZoneAwarePlacementPolicy + public static final String DESIRED_NUM_ZONES_PER_WRITE_QUORUM = "desiredNumZonesPerWriteQuorum"; + + // in ZoneawareEnsemblePlacementPolicy if strict placement is enabled then + // minZones/desiredZones in writeQuorum would be maintained otherwise it + // will pick nodes randomly. + public static final String ENFORCE_STRICT_ZONEAWARE_PLACEMENT = "enforceStrictZoneawarePlacement"; + // Allocator configuration protected static final String ALLOCATOR_POOLING_POLICY = "allocatorPoolingPolicy"; protected static final String ALLOCATOR_POOLING_CONCURRENCY = "allocatorPoolingConcurrency"; @@ -835,6 +846,60 @@ public int getMinNumRacksPerWriteQuorum() { return getInteger(MIN_NUM_RACKS_PER_WRITE_QUORUM, 2); } + /** + * Set the minimum number of zones per write quorum in + * ZoneAwarePlacementPolicy. + */ + public void setMinNumZonesPerWriteQuorum(int minNumZonesPerWriteQuorum) { + setProperty(MIN_NUM_ZONES_PER_WRITE_QUORUM, minNumZonesPerWriteQuorum); + } + + /** + * Get the minimum number of zones per write quorum in + * ZoneAwarePlacementPolicy. + */ + public int getMinNumZonesPerWriteQuorum() { + return getInteger(MIN_NUM_ZONES_PER_WRITE_QUORUM, 2); + } + + /** + * Set the desired number of zones per write quorum in + * ZoneAwarePlacementPolicy. + */ + public void setDesiredNumZonesPerWriteQuorum(int desiredNumZonesPerWriteQuorum) { + setProperty(DESIRED_NUM_ZONES_PER_WRITE_QUORUM, desiredNumZonesPerWriteQuorum); + } + + /** + * Get the desired number of zones per write quorum in + * ZoneAwarePlacementPolicy. + */ + public int getDesiredNumZonesPerWriteQuorum() { + return getInteger(DESIRED_NUM_ZONES_PER_WRITE_QUORUM, 3); + } + + /** + * Set the flag to enforce strict zoneaware placement. + * + *

            in ZoneawareEnsemblePlacementPolicy if strict placement is enabled then + * minZones/desiredZones in writeQuorum would be maintained otherwise it + * will pick nodes randomly. + */ + public void setEnforceStrictZoneawarePlacement(boolean enforceStrictZoneawarePlacement) { + setProperty(ENFORCE_STRICT_ZONEAWARE_PLACEMENT, enforceStrictZoneawarePlacement); + } + + /** + * Get the flag to enforce strict zoneaware placement. + * + *

            in ZoneawareEnsemblePlacementPolicy if strict placement is enabled then + * minZones/desiredZones in writeQuorum would be maintained otherwise it + * will pick nodes randomly. + */ + public boolean getEnforceStrictZoneawarePlacement() { + return getBoolean(ENFORCE_STRICT_ZONEAWARE_PLACEMENT, true); + } + /** * Set the flag to enforce minimum number of racks per write quorum. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopology.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopology.java index d2d37ea52c7..70a1032ce83 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopology.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopology.java @@ -27,6 +27,9 @@ public interface NetworkTopology { String DEFAULT_REGION = "/default-region"; String DEFAULT_RACK = "/default-rack"; + String DEFAULT_ZONE = "/default-zone"; + String DEFAULT_UPGRADEDOMAIN = "/default-upgradedomain"; + String DEFAULT_ZONE_AND_UPGRADEDOMAIN = DEFAULT_ZONE + DEFAULT_UPGRADEDOMAIN; String DEFAULT_REGION_AND_RACK = DEFAULT_REGION + DEFAULT_RACK; /** 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 cbf9b45c954..036f5d51aac 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 @@ -30,6 +30,7 @@ import static org.apache.bookkeeper.replication.ReplicationStats.NUM_FRAGMENTS_PER_LEDGER; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_CHECKED; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_UNDER_REPLICATED_LEDGERS; import static org.apache.bookkeeper.replication.ReplicationStats.PLACEMENT_POLICY_CHECK_TIME; import static org.apache.bookkeeper.replication.ReplicationStats.URL_PUBLISH_TIME_FOR_LOST_BOOKIE; @@ -60,6 +61,7 @@ import org.apache.bookkeeper.client.BKException.Code; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.EnsemblePlacementPolicy.PlacementPolicyAdherence; import org.apache.bookkeeper.client.LedgerChecker; import org.apache.bookkeeper.client.LedgerFragment; import org.apache.bookkeeper.client.LedgerHandle; @@ -118,7 +120,10 @@ public class Auditor implements AutoCloseable { private Set bookiesToBeAudited = Sets.newHashSet(); private volatile int lostBookieRecoveryDelayBeforeChange; private final AtomicInteger ledgersNotAdheringToPlacementPolicyGuageValue; - private final AtomicInteger numOfLedgersFoundInPlacementPolicyCheck; + private final AtomicInteger numOfLedgersFoundNotAdheringInPlacementPolicyCheck; + private final AtomicInteger ledgersSoftlyAdheringToPlacementPolicyGuageValue; + private final AtomicInteger numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck; + private final AtomicInteger numOfClosedLedgersAuditedInPlacementPolicyCheck; private final StatsLogger statsLogger; @StatsDoc( @@ -181,6 +186,11 @@ public class Auditor implements AutoCloseable { help = "Gauge for number of ledgers not adhering to placement policy found in placement policy check" ) private final Gauge numLedgersNotAdheringToPlacementPolicy; + @StatsDoc( + name = NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY, + help = "Gauge for number of ledgers softly adhering to placement policy found in placement policy check" + ) + private final Gauge numLedgersSoftlyAdheringToPlacementPolicy; static BookKeeper createBookKeeperClient(ServerConfiguration conf) throws InterruptedException, IOException { return createBookKeeperClient(conf, NullStatsLogger.INSTANCE); @@ -231,8 +241,11 @@ public Auditor(final String bookieIdentifier, this.conf = conf; this.bookieIdentifier = bookieIdentifier; this.statsLogger = statsLogger; - this.numOfLedgersFoundInPlacementPolicyCheck = new AtomicInteger(0); + this.numOfLedgersFoundNotAdheringInPlacementPolicyCheck = new AtomicInteger(0); this.ledgersNotAdheringToPlacementPolicyGuageValue = new AtomicInteger(0); + this.numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck = new AtomicInteger(0); + this.ledgersSoftlyAdheringToPlacementPolicyGuageValue = new AtomicInteger(0); + this.numOfClosedLedgersAuditedInPlacementPolicyCheck = new AtomicInteger(0); numUnderReplicatedLedger = this.statsLogger.getOpStatsLogger(ReplicationStats.NUM_UNDER_REPLICATED_LEDGERS); uRLPublishTimeForLostBookies = this.statsLogger @@ -261,6 +274,19 @@ public Integer getSample() { }; this.statsLogger.registerGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY, numLedgersNotAdheringToPlacementPolicy); + numLedgersSoftlyAdheringToPlacementPolicy = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + return ledgersSoftlyAdheringToPlacementPolicyGuageValue.get(); + } + }; + this.statsLogger.registerGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY, + numLedgersSoftlyAdheringToPlacementPolicy); this.bkc = bkc; this.ownBkc = ownBkc; @@ -627,20 +653,30 @@ public void run() { Stopwatch stopwatch = Stopwatch.createStarted(); LOG.info("Starting PlacementPolicyCheck"); placementPolicyCheck(); - int numOfLedgersFoundInPlacementPolicyCheckValue = numOfLedgersFoundInPlacementPolicyCheck - .get(); long placementPolicyCheckDuration = stopwatch.stop().elapsed(TimeUnit.MILLISECONDS); + int numOfLedgersFoundNotAdheringInPlacementPolicyCheckValue = + numOfLedgersFoundNotAdheringInPlacementPolicyCheck.get(); + int numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue = + numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck.get(); + int numOfClosedLedgersAuditedInPlacementPolicyCheckValue = + numOfClosedLedgersAuditedInPlacementPolicyCheck.get(); LOG.info( "Completed placementPolicyCheck in {} milliSeconds." - + " numOfLedgersNotAdheringToPlacementPolicy {}", - placementPolicyCheckDuration, numOfLedgersFoundInPlacementPolicyCheckValue); + + " numOfClosedLedgersAuditedInPlacementPolicyCheck {}" + + " numOfLedgersNotAdheringToPlacementPolicy {}" + + " numOfLedgersSoftlyAdheringToPlacementPolicy {}", + placementPolicyCheckDuration, numOfClosedLedgersAuditedInPlacementPolicyCheckValue, + numOfLedgersFoundNotAdheringInPlacementPolicyCheckValue, + numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue); ledgersNotAdheringToPlacementPolicyGuageValue - .set(numOfLedgersFoundInPlacementPolicyCheckValue); + .set(numOfLedgersFoundNotAdheringInPlacementPolicyCheckValue); + ledgersSoftlyAdheringToPlacementPolicyGuageValue + .set(numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue); placementPolicyCheckTime.registerSuccessfulEvent(placementPolicyCheckDuration, TimeUnit.MILLISECONDS); } catch (BKAuditException e) { - int numOfLedgersFoundInPlacementPolicyCheckValue = numOfLedgersFoundInPlacementPolicyCheck - .get(); + int numOfLedgersFoundInPlacementPolicyCheckValue = + numOfLedgersFoundNotAdheringInPlacementPolicyCheck.get(); if (numOfLedgersFoundInPlacementPolicyCheckValue > 0) { /* * Though there is BKAuditException while doing @@ -650,10 +686,25 @@ public void run() { ledgersNotAdheringToPlacementPolicyGuageValue .set(numOfLedgersFoundInPlacementPolicyCheckValue); } + + int numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue = + numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck.get(); + if (numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue > 0) { + /* + * Though there is BKAuditException while doing + * placementPolicyCheck, it found few ledgers softly + * adhering to placement policy. So reporting it. + */ + ledgersSoftlyAdheringToPlacementPolicyGuageValue + .set(numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue); + } + LOG.error( "BKAuditException running periodic placementPolicy check." - + "numOfLedgersNotAdheringToPlacementPolicy {}", - numOfLedgersFoundInPlacementPolicyCheckValue, e); + + "numOfLedgersNotAdheringToPlacementPolicy {}, " + + "numOfLedgersSoftlyAdheringToPlacementPolicy {}", + numOfLedgersFoundInPlacementPolicyCheckValue, + numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue, e); } } }, initialDelay, interval, TimeUnit.SECONDS); @@ -929,7 +980,9 @@ void checkAllLedgers() throws BKException, IOException, InterruptedException, Ke void placementPolicyCheck() throws BKAuditException { final CountDownLatch placementPolicyCheckLatch = new CountDownLatch(1); - this.numOfLedgersFoundInPlacementPolicyCheck.set(0); + this.numOfLedgersFoundNotAdheringInPlacementPolicyCheck.set(0); + this.numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck.set(0); + this.numOfClosedLedgersAuditedInPlacementPolicyCheck.set(0); Processor ledgerProcessor = new Processor() { @Override public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { @@ -940,13 +993,15 @@ public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { int ackQuorumSize = metadata.getAckQuorumSize(); if (metadata.isClosed()) { boolean foundSegmentNotAdheringToPlacementPolicy = false; + boolean foundSegmentSoftlyAdheringToPlacementPolicy = false; for (Map.Entry> ensemble : metadata .getAllEnsembles().entrySet()) { long startEntryIdOfSegment = ensemble.getKey(); List ensembleOfSegment = ensemble.getValue(); - boolean segmentAdheringToPlacementPolicy = admin.isEnsembleAdheringToPlacementPolicy( - ensembleOfSegment, writeQuorumSize, ackQuorumSize); - if (!segmentAdheringToPlacementPolicy) { + PlacementPolicyAdherence segmentAdheringToPlacementPolicy = admin + .isEnsembleAdheringToPlacementPolicy(ensembleOfSegment, writeQuorumSize, + ackQuorumSize); + if (segmentAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) { foundSegmentNotAdheringToPlacementPolicy = true; LOG.warn( "For ledger: {}, Segment starting at entry: {}, with ensemble: {} having " @@ -954,11 +1009,24 @@ public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { + "EnsemblePlacementPolicy", ledgerId, startEntryIdOfSegment, ensembleOfSegment, writeQuorumSize, ackQuorumSize); + } else if (segmentAdheringToPlacementPolicy == PlacementPolicyAdherence.MEETS_SOFT) { + foundSegmentSoftlyAdheringToPlacementPolicy = true; + if (LOG.isDebugEnabled()) { + LOG.debug( + "For ledger: {}, Segment starting at entry: {}, with ensemble: {}" + + " having writeQuorumSize: {} and ackQuorumSize: {} is" + + " softly adhering to EnsemblePlacementPolicy", + ledgerId, startEntryIdOfSegment, ensembleOfSegment, writeQuorumSize, + ackQuorumSize); + } } } if (foundSegmentNotAdheringToPlacementPolicy) { - numOfLedgersFoundInPlacementPolicyCheck.incrementAndGet(); + numOfLedgersFoundNotAdheringInPlacementPolicyCheck.incrementAndGet(); + } else if (foundSegmentSoftlyAdheringToPlacementPolicy) { + numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck.incrementAndGet(); } + numOfClosedLedgersAuditedInPlacementPolicyCheck.incrementAndGet(); } else { if (LOG.isDebugEnabled()) { LOG.debug("Ledger: {} is not yet closed, so skipping the placementPolicy" 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 c13bc56cc23..2f83804a8ee 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 @@ -41,6 +41,7 @@ public interface ReplicationStats { String NUM_BOOKIE_AUDITS_DELAYED = "NUM_BOOKIE_AUDITS_DELAYED"; String NUM_DELAYED_BOOKIE_AUDITS_DELAYES_CANCELLED = "NUM_DELAYED_BOOKIE_AUDITS_CANCELLED"; String NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY = "NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY"; + String NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY = "NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY"; String REPLICATION_WORKER_SCOPE = "replication_worker"; String REREPLICATE_OP = "rereplicate"; 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 3fc74839c98..6017ad92333 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 @@ -42,6 +42,7 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; +import org.apache.bookkeeper.client.EnsemblePlacementPolicy.PlacementPolicyAdherence; import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.EnsembleForReplacementWithNoConstraints; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.TruePredicate; @@ -574,9 +575,9 @@ public void testReplaceBookieWithEnoughBookiesInSameRack() throws Exception { EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); BookieSocketAddress replacedBookie = replaceBookieResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); assertEquals(addr3, replacedBookie); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @Test @@ -603,10 +604,10 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRack() throws Exception EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, excludedAddrs); BookieSocketAddress replacedBookie = replaceBookieResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); assertFalse(addr1.equals(replacedBookie)); assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @Test @@ -668,9 +669,9 @@ public void testReplaceBookieWithEnoughBookiesInSameRackAsEnsemble() throws Exce addr4, new HashSet<>()); BookieSocketAddress replacedBookie = replaceBookieResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); assertEquals(addr1, replacedBookie); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @Test @@ -690,15 +691,15 @@ public void testNewEnsembleWithSingleRack() throws Exception { EnsemblePlacementPolicy.PlacementResult> ensembleResponse; ensembleResponse = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); List ensemble = ensembleResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals(0, getNumCoveredWriteQuorums(ensemble, 2, conf.getMinNumRacksPerWriteQuorum())); - assertFalse(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); EnsemblePlacementPolicy.PlacementResult> ensembleResponse2; ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()); List ensemble2 = ensembleResponse2.getResult(); - boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isAdheringToPolicy(); assertEquals(0, getNumCoveredWriteQuorums(ensemble2, 2, conf.getMinNumRacksPerWriteQuorum())); - assertFalse(isEnsembleAdheringToPlacementPolicy2); + assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); } @@ -812,26 +813,26 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { */ EnsemblePlacementPolicy.PlacementResult> ensembleResponse; List ensemble; - boolean isEnsembleAdheringToPlacementPolicy; + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; int ensembleSize = numOfRacks * numOfBookiesPerRack; int writeQuorumSize = numOfRacks; int ackQuorumSize = numOfRacks; ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); ensemble = ensembleResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(defaultRackBookiesList), EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); ensemble = ensembleResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @Test @@ -877,22 +878,22 @@ public void testNewEnsembleWithSufficientRacksAndEnforceMinNumRacks() throws Exc */ EnsemblePlacementPolicy.PlacementResult> ensembleResponse; List ensemble; - boolean isEnsembleAdheringToPlacementPolicy; + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; for (int ensembleSize = effectiveMinNumRacksPerWriteQuorum; ensembleSize < 40; ensembleSize++) { ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); ensemble = ensembleResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(), EnsembleForReplacementWithNoConstraints.INSTANCE, TruePredicate.INSTANCE); ensemble = ensembleResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } } @@ -1011,13 +1012,13 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { */ EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; BookieSocketAddress replacedBookieAddress; - boolean isEnsembleAdheringToPlacementPolicy; + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; replaceBookieResponse = repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, ensemble, bookieInEnsembleToBeReplaced, new HashSet<>()); replacedBookieAddress = replaceBookieResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); assertEquals("It should be newBookieAddress2", newBookieAddress2, replacedBookieAddress); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); Set bookiesToExclude = new HashSet<>(); bookiesToExclude.add(newBookieAddress2); @@ -1055,9 +1056,9 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { replaceBookieResponse = repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, ensemble, bookieInEnsembleToBeReplaced, bookiesToExclude); replacedBookieAddress = replaceBookieResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); assertEquals("It should be newBookieAddress3", newBookieAddress3, replacedBookieAddress); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @Test @@ -1408,19 +1409,19 @@ public void testNewEnsembleWithMultipleRacks() throws Exception { repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, null, new HashSet<>()); List ensemble = ensembleResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); - assertFalse(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); ensembleSize = 4; EnsemblePlacementPolicy.PlacementResult> ensembleResponse2 = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, null, new HashSet<>()); List ensemble2 = ensembleResponse2.getResult(); - boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isAdheringToPolicy(); numCovered = getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); assertTrue(numCovered >= 1 && numCovered < 3); - assertFalse(isEnsembleAdheringToPlacementPolicy2); + assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy2); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); } @@ -1493,10 +1494,10 @@ void validateNumOfWriteQuorumsCoveredInEnsembleCreation(Set repp.newEnsemble(ensembleSize, writeQuorumSize, writeQuorumSize, null, new HashSet<>()); List ensemble = ensembleResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, minNumRacksPerWriteQuorum); assertEquals("minimum number of racks covered for writequorum ensemble: " + ensemble, ensembleSize, numCovered); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @Test @@ -1538,19 +1539,19 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); List ensemble1 = ensembleResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy1 = ensembleResponse.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy1 = ensembleResponse.isAdheringToPolicy(); assertEquals(ensembleSize, getNumCoveredWriteQuorums(ensemble1, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); - assertTrue(isEnsembleAdheringToPlacementPolicy1); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy1); ensembleSize = 4; writeQuorumSize = 4; EnsemblePlacementPolicy.PlacementResult> ensembleResponse2 = repp.newEnsemble(ensembleSize, writeQuorumSize, 2, null, new HashSet<>()); List ensemble2 = ensembleResponse2.getResult(); - boolean isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isAdheringToPolicy(); assertEquals(ensembleSize, getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); - assertTrue(isEnsembleAdheringToPlacementPolicy2); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy2); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); } @@ -1622,15 +1623,15 @@ public void testWeightedPlacementAndReplaceBookieWithEnoughBookiesInSameRack() t selectionCounts.put(addr4, 0L); int numTries = 50000; EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; - boolean isEnsembleAdheringToPlacementPolicy; + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; BookieSocketAddress replacedBookie; for (int i = 0; i < numTries; i++) { // replace node under r2 replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); replacedBookie = replaceBookieResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); assertTrue("replaced : " + replacedBookie, addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); selectionCounts.put(replacedBookie, selectionCounts.get(replacedBookie) + 1); } double observedMultiple = ((double) selectionCounts.get(addr4) / (double) selectionCounts.get(addr3)); @@ -1688,17 +1689,17 @@ public void testWeightedPlacementAndReplaceBookieWithoutEnoughBookiesInSameRack( int numTries = 50000; EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; BookieSocketAddress replacedBookie; - boolean isEnsembleAdheringToPlacementPolicy; + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; for (int i = 0; i < numTries; i++) { // addr2 is on /r2 and this is the only one on this rack. So the replacement // will come from other racks. However, the weight should be honored in such // selections as well replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); replacedBookie = replaceBookieResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); assertTrue(addr0.equals(replacedBookie) || addr1.equals(replacedBookie) || addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); selectionCounts.put(replacedBookie, selectionCounts.get(replacedBookie) + 1); } /* @@ -1965,17 +1966,17 @@ public void testPlacementOnStabilizeNetworkTopology() throws Exception { EnsemblePlacementPolicy.PlacementResult> ensembleResponse = repp.newEnsemble(3, 2, 2, null, new HashSet()); List ensemble = ensembleResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertFalse(ensemble.contains(addr4)); - assertFalse(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); } // we could still use addr4 for urgent allocation if it is just bookie flapping EnsemblePlacementPolicy.PlacementResult> ensembleResponse = repp.newEnsemble(4, 2, 2, null, new HashSet()); List ensemble = ensembleResponse.getResult(); - boolean isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); - assertFalse(isEnsembleAdheringToPlacementPolicy); + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); + assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); assertTrue(ensemble.contains(addr4)); } @@ -2212,21 +2213,21 @@ public void testNewEnsembleExcludesDefaultRackBookiesEnforceMinNumRacks() throws */ EnsemblePlacementPolicy.PlacementResult> ensembleResponse; List ensemble; - boolean isEnsembleAdheringToPlacementPolicy; + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; for (int ensembleSize = effectiveMinNumRacksPerWriteQuorum; ensembleSize < 40; ensembleSize++) { ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); ensemble = ensembleResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); ensemble = ensembleResponse.getResult(); - isEnsembleAdheringToPlacementPolicy = ensembleResponse.isStrictlyAdheringToPolicy(); + isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); - assertTrue(isEnsembleAdheringToPlacementPolicy); + assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); Collection bookiesOfDefaultRackInEnsemble = CollectionUtils .intersection(Arrays.asList(bookieSocketAddressesInDefaultRack), ensemble); assertTrue("Ensemble is not supposed to contain bookies from default rack, but ensemble contains - " diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWeightedRandomSelection.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWeightedRandomSelection.java index 2aa3c0e89af..00734f0614f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWeightedRandomSelection.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWeightedRandomSelection.java @@ -21,45 +21,71 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; import org.apache.commons.configuration.CompositeConfiguration; import org.apache.commons.configuration.Configuration; import org.junit.After; +import org.junit.Assume; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Test weighted random selection methods. */ +@RunWith(Parameterized.class) public class TestWeightedRandomSelection { static final Logger LOG = LoggerFactory.getLogger(TestWeightedRandomSelection.class); static class TestObj implements WeightedObject { long val; + TestObj(long value) { this.val = value; } + @Override public long getWeight() { return val; } } + Class weightedRandomSelectionClass; WeightedRandomSelection wRS; Configuration conf = new CompositeConfiguration(); int multiplier = 3; + @Parameters + public static Collection weightedRandomSelectionClass() { + return Arrays.asList( + new Object[][] { { WeightedRandomSelectionImpl.class }, { DynamicWeightedRandomSelectionImpl.class } }); + } + + public TestWeightedRandomSelection(Class weightedRandomSelectionClass) { + this.weightedRandomSelectionClass = weightedRandomSelectionClass; + } + @Before public void setUp() throws Exception { - wRS = new WeightedRandomSelectionImpl(); + if (weightedRandomSelectionClass.equals(WeightedRandomSelectionImpl.class)) { + wRS = new WeightedRandomSelectionImpl(); + } else { + wRS = new DynamicWeightedRandomSelectionImpl(); + } } @After @@ -90,7 +116,7 @@ public void testSelectionWithEqualWeights() throws Exception { double actualPct = ((double) e.getValue() / (double) totalTries) * 100; double delta = (Math.abs(expectedPct - actualPct) / expectedPct) * 100; System.out.println("Key:" + e.getKey() + " Value:" + e.getValue() + " Expected: " + expectedPct - + " Actual: " + actualPct); + + " Actual: " + actualPct + " delta: " + delta); // should be within 5% of expected assertTrue("Not doing uniform selection when weights are equal", delta < 5); } @@ -145,7 +171,8 @@ void verifyResult(Map map, Map randomSe double expected; if (map.get(e.getKey()).getWeight() == 0) { - // if the value is 0 for any key, we make it equal to the first non zero value + // if the value is 0 for any key, we make it equal to the first + // non zero value expected = (double) minWeight / (double) totalWeight; } else { expected = (double) map.get(e.getKey()).getWeight() / (double) totalWeight; @@ -153,14 +180,16 @@ void verifyResult(Map map, Map randomSe if (multiplier > 0 && expected > multiplier * medianExpectedWeight) { expected = multiplier * medianExpectedWeight; } - // We can't compare these weights because they are derived from different - // values. But if we express them as a multiple of the min in each, then + // We can't compare these weights because they are derived from + // different + // values. But if we express them as a multiple of the min in each, + // then // they should be comparable double expectedMultiple = expected / medianExpectedWeight; double observedMultiple = observed / medianObservedWeight; double delta = (Math.abs(expectedMultiple - observedMultiple) / expectedMultiple) * 100; - System.out.println("Key:" + e.getKey() + " Value:" + e.getValue() - + " Expected " + expectedMultiple + " actual " + observedMultiple + " delta " + delta + "%"); + System.out.println("Key:" + e.getKey() + " Value:" + e.getValue() + " Expected " + expectedMultiple + + " actual " + observedMultiple + " delta " + delta + "%"); // the observed should be within 5% of expected assertTrue("Not doing uniform selection when weights are equal", delta < 5); @@ -178,7 +207,7 @@ public void testSelectionWithSomeZeroWeights() throws Exception { for (Integer i = 0; i < numKeys; i++) { if (i < numKeys / 3) { val = 0L; - } else if (i < 2 * (numKeys / 3)){ + } else if (i < 2 * (numKeys / 3)) { val = minWeight; } else { val = 2 * minWeight; @@ -189,7 +218,7 @@ public void testSelectionWithSomeZeroWeights() throws Exception { } wRS.updateMap(map); - int totalTries = 10000000; + int totalTries = 1000000; for (int i = 0; i < totalTries; i++) { String key = wRS.getNextRandom(); randomSelection.put(key, randomSelection.get(key) + 1); @@ -208,7 +237,7 @@ public void testSelectionWithUnequalWeights() throws Exception { for (Integer i = 0; i < numKeys; i++) { if (i < numKeys / 3) { val = minWeight; - } else if (i < 2 * (numKeys / 3)){ + } else if (i < 2 * (numKeys / 3)) { val = 2 * minWeight; } else { val = 10 * minWeight; @@ -219,7 +248,7 @@ public void testSelectionWithUnequalWeights() throws Exception { } wRS.updateMap(map); - int totalTries = 10000000; + int totalTries = 1000000; for (int i = 0; i < totalTries; i++) { String key = wRS.getNextRandom(); randomSelection.put(key, randomSelection.get(key) + 1); @@ -247,7 +276,7 @@ public void testSelectionWithHotNode() throws Exception { } wRS.updateMap(map); - int totalTries = 10000000; + int totalTries = 1000000; for (int i = 0; i < totalTries; i++) { String key = wRS.getNextRandom(); randomSelection.put(key, randomSelection.get(key) + 1); @@ -275,11 +304,39 @@ public void testSelectionWithHotNodeWithLimit() throws Exception { } wRS.updateMap(map); - int totalTries = 10000000; + int totalTries = 1000000; for (int i = 0; i < totalTries; i++) { String key = wRS.getNextRandom(); randomSelection.put(key, randomSelection.get(key) + 1); } verifyResult(map, randomSelection, multiplier, minWeight, medianWeight, total, totalTries); } + + @Test + public void testSelectionFromSelectedNodesWithEqualWeights() throws Exception { + /* + * this testcase is for only DynamicWeightedRandomSelectionImpl + */ + Assume.assumeTrue(weightedRandomSelectionClass.equals(DynamicWeightedRandomSelectionImpl.class)); + Map map = new HashMap(); + + Long val = 100L; + int numKeys = 50, totalTries = 1000; + Map randomSelection = new HashMap(); + for (Integer i = 0; i < numKeys; i++) { + map.put(i.toString(), new TestObj(val)); + randomSelection.put(i.toString(), 0); + } + + Set selectFrom = new HashSet(); + for (int i = 0; i < numKeys / 2; i++) { + selectFrom.add(Integer.toString(i)); + } + + wRS.updateMap(map); + for (int i = 0; i < totalTries; i++) { + String selectedKey = wRS.getNextRandom(selectFrom); + assertTrue("NextRandom key should be from selected list", selectFrom.contains(selectedKey)); + } + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestZoneawareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestZoneawareEnsemblePlacementPolicy.java new file mode 100644 index 00000000000..fb669db10a8 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestZoneawareEnsemblePlacementPolicy.java @@ -0,0 +1,1339 @@ +/* + * 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.client; + +import static org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.REPP_DNS_RESOLVER_CLASS; +import static org.apache.bookkeeper.client.RoundRobinDistributionSchedule.writeSetFromValues; +import static org.apache.bookkeeper.feature.SettableFeatureProvider.DISABLE_ALL; +import static org.junit.Assert.assertNotEquals; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.netty.util.HashedWheelTimer; + +import java.net.InetAddress; +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.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import junit.framework.TestCase; + +import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; +import org.apache.bookkeeper.client.EnsemblePlacementPolicy.PlacementPolicyAdherence; +import org.apache.bookkeeper.client.EnsemblePlacementPolicy.PlacementResult; +import org.apache.bookkeeper.client.ZoneawareEnsemblePlacementPolicyImpl.ZoneAwareNodeLocation; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.net.NetworkTopology; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.util.StaticDNSResolver; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test the zoneaware ensemble placement policy. + */ +public class TestZoneawareEnsemblePlacementPolicy extends TestCase { + + static final Logger LOG = LoggerFactory.getLogger(TestZoneawareEnsemblePlacementPolicy.class); + + ZoneawareEnsemblePlacementPolicy zepp; + final List ensemble = new ArrayList(); + DistributionSchedule.WriteSet writeSet = DistributionSchedule.NULL_WRITE_SET; + ClientConfiguration conf = new ClientConfiguration(); + BookieSocketAddress addr1, addr2, addr3, addr4; + io.netty.util.HashedWheelTimer timer; + + @Override + protected void setUp() throws Exception { + super.setUp(); + StaticDNSResolver.reset(); + StaticDNSResolver.addNodeToRack(InetAddress.getLocalHost().getHostAddress(), + NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack("127.0.0.1", NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack("localhost", NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + LOG.info("Set up static DNS Resolver."); + conf.setProperty(REPP_DNS_RESOLVER_CLASS, StaticDNSResolver.class.getName()); + addr1 = new BookieSocketAddress("127.0.0.2", 3181); + addr2 = new BookieSocketAddress("127.0.0.3", 3181); + addr3 = new BookieSocketAddress("127.0.0.4", 3181); + addr4 = new BookieSocketAddress("127.0.0.5", 3181); + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_ZONE + "/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), NetworkTopology.DEFAULT_ZONE + "/ud2"); + ensemble.add(addr1); + ensemble.add(addr2); + ensemble.add(addr3); + ensemble.add(addr4); + writeSet = writeSetFromValues(0, 1, 2, 3); + + timer = new HashedWheelTimer(new ThreadFactoryBuilder().setNameFormat("TestTimer-%d").build(), + conf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS, conf.getTimeoutTimerNumTicks()); + + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(conf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + } + + @Override + protected void tearDown() throws Exception { + zepp.uninitalize(); + super.tearDown(); + } + + static BookiesHealthInfo getBookiesHealthInfo() { + return getBookiesHealthInfo(new HashMap<>(), new HashMap<>()); + } + + static BookiesHealthInfo getBookiesHealthInfo(Map bookieFailureHistory, + Map bookiePendingRequests) { + return new BookiesHealthInfo() { + @Override + public long getBookieFailureHistory(BookieSocketAddress bookieSocketAddress) { + return bookieFailureHistory.getOrDefault(bookieSocketAddress, -1L); + } + + @Override + public long getBookiePendingRequests(BookieSocketAddress bookieSocketAddress) { + return bookiePendingRequests.getOrDefault(bookieSocketAddress, 0L); + } + }; + } + + static void updateMyUpgradeDomain(String zoneAndUD) throws Exception { + StaticDNSResolver.addNodeToRack(InetAddress.getLocalHost().getHostAddress(), zoneAndUD); + StaticDNSResolver.addNodeToRack(InetAddress.getLocalHost().getHostName(), zoneAndUD); + StaticDNSResolver.addNodeToRack("127.0.0.1", zoneAndUD); + StaticDNSResolver.addNodeToRack("localhost", zoneAndUD); + } + + @Test + public void testNotEnoughRWBookies() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone4/ud1"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone5/ud1"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone6/ud1"); + + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(1); + newConf.setMinNumZonesPerWriteQuorum(1); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + + zepp.onClusterChanged(rwAddrs, roAddrs); + try { + // only 3 rw bookies are available + zepp.newEnsemble(6, 3, 2, null, new HashSet<>()); + fail("newEnsemble is expected to fail because enough writable nodes are not available"); + } catch (BKException.BKNotEnoughBookiesException bke) { + // expected to get BKNotEnoughBookiesException + } + + roAddrs.add(addr4); + roAddrs.add(addr5); + roAddrs.add(addr6); + zepp.onClusterChanged(rwAddrs, roAddrs); + try { + // only 3 rw bookies are available + zepp.newEnsemble(6, 3, 2, null, new HashSet<>()); + fail("newEnsemble is expected to fail because enough writable nodes are not available"); + } catch (BKException.BKNotEnoughBookiesException bke) { + // expected to get BKNotEnoughBookiesException + } + + rwAddrs.clear(); + roAddrs.add(addr1); + roAddrs.add(addr2); + roAddrs.add(addr3); + zepp.onClusterChanged(rwAddrs, roAddrs); + try { + // no rw bookie is available + zepp.newEnsemble(6, 3, 2, null, new HashSet<>()); + fail("newEnsemble is expected to fail because enough writable nodes are not available"); + } catch (BKException.BKNotEnoughBookiesException bke) { + // expected to get BKNotEnoughBookiesException + } + } + + @Test + public void testEnoughRWBookies() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone4/ud1"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone5/ud1"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone6/ud1"); + + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(4); + newConf.setMinNumZonesPerWriteQuorum(2); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr6); + + zepp.onClusterChanged(rwAddrs, roAddrs); + /* + * there are enough bookies so newEnsemble should succeed. + */ + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(6, 3, 2, null, + new HashSet<>()); + Set newEnsembleSet = new HashSet( + newEnsemblePlacementResult.getResult()); + assertTrue("New ensemble should contain all 6 rw bookies", newEnsembleSet.containsAll(rwAddrs)); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, + newEnsemblePlacementResult.isAdheringToPolicy()); + + /* + * there are enough bookies so newEnsemble should succeed. + */ + newEnsemblePlacementResult = zepp.newEnsemble(3, 3, 2, null, new HashSet<>()); + newEnsembleSet = new HashSet(newEnsemblePlacementResult.getResult()); + assertTrue("New ensemble should contain 3 rw bookies", + (newEnsembleSet.size() == 3) && (rwAddrs.containsAll(newEnsembleSet))); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, + newEnsemblePlacementResult.isAdheringToPolicy()); + } + + @Test + public void testWithDefaultBookies() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone4/ud1"); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + Set bookiesInDefaultFaultDomain = new HashSet(); + bookiesInDefaultFaultDomain.add(addr5); + bookiesInDefaultFaultDomain.add(addr6); + bookiesInDefaultFaultDomain.add(addr7); + + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(4); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr6); + rwAddrs.add(addr7); + + zepp.onClusterChanged(rwAddrs, roAddrs); + for (int i = 0; i < 3; i++) { + /* + * make sure bookies from DEFAULT_ZONE_AND_UPGRADEDOMAIN are not + * part of the new ensemble created. + */ + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(4, 4, 2, null, + new HashSet<>()); + Set newEnsembleSet = new HashSet( + newEnsemblePlacementResult.getResult()); + assertTrue("Bookie from default faultDomain shouldn't be part of ensemble", + Collections.disjoint(newEnsembleSet, bookiesInDefaultFaultDomain)); + + newEnsemblePlacementResult = zepp.newEnsemble(3, 3, 2, null, new HashSet<>()); + newEnsembleSet = new HashSet(newEnsemblePlacementResult.getResult()); + assertTrue("Bookie from default faultDomain shouldn't be part of ensemble", + Collections.disjoint(newEnsembleSet, bookiesInDefaultFaultDomain)); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, + newEnsemblePlacementResult.isAdheringToPolicy()); + } + } + + @Test + public void testMinZonesPerWriteQuorum() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); + BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/zone1/ud3"); + StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/zone2/ud3"); + StaticDNSResolver.addNodeToRack(addr9.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack(addr10.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(4); + newConf.setMinNumZonesPerWriteQuorum(3); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + Set bookiesInDefaultFaultDomain = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr6); + rwAddrs.add(addr9); + rwAddrs.add(addr10); + roAddrs.add(addr7); + roAddrs.add(addr8); + bookiesInDefaultFaultDomain.add(addr9); + bookiesInDefaultFaultDomain.add(addr10); + + zepp.onClusterChanged(rwAddrs, roAddrs); + PlacementResult> newEnsemblePlacementResult; + + newEnsemblePlacementResult = zepp.newEnsemble(4, 4, 2, null, new HashSet<>()); + Set newEnsembleSet = new HashSet( + newEnsemblePlacementResult.getResult()); + assertTrue("New ensemble should contain all 6 rw bookies in non-default fault domains", + rwAddrs.containsAll(newEnsembleSet) && (newEnsembleSet.size() == 4)); + assertTrue("Bookie from default faultDomain shouldn't be part of ensemble", + Collections.disjoint(newEnsembleSet, bookiesInDefaultFaultDomain)); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, + newEnsemblePlacementResult.isAdheringToPolicy()); + + try { + /* + * If ensembleSize is not multiple of writeQuorumSize, then it is + * expected to fail with IllegalArgumentException. + */ + zepp.newEnsemble(4, 3, 2, null, new HashSet<>()); + fail("newEnsemble is expected to fail with IllegalArgumentException"); + } catch (IllegalArgumentException illExc) { + // expected IllegalArgumentException + } + zepp.uninitalize(); + newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(4); + newConf.setMinNumZonesPerWriteQuorum(3); + newConf.setEnforceStrictZoneawarePlacement(false); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + zepp.onClusterChanged(rwAddrs, roAddrs); + + /* + * If enforceStrictZoneawarePlacement is not enabled, then there are no + * limitations on eligible values of ensembleSize and writeQuorumSize. + */ + newEnsemblePlacementResult = zepp.newEnsemble(4, 3, 2, null, new HashSet<>()); + newEnsembleSet = new HashSet(newEnsemblePlacementResult.getResult()); + assertTrue("New ensemble should contain 4 different bookies", newEnsembleSet.size() == 4); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, + newEnsemblePlacementResult.isAdheringToPolicy()); + } + + @Test + public void testMinUDsNotAvailable() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); + BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/zone1/ud3"); + StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/zone2/ud3"); + StaticDNSResolver.addNodeToRack(addr9.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack(addr10.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(4); + newConf.setMinNumZonesPerWriteQuorum(2); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + Set bookiesInDefaultFaultDomain = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr6); + rwAddrs.add(addr9); + rwAddrs.add(addr10); + + roAddrs.add(addr7); + roAddrs.add(addr8); + + bookiesInDefaultFaultDomain.add(addr9); + bookiesInDefaultFaultDomain.add(addr10); + + zepp.onClusterChanged(rwAddrs, roAddrs); + PlacementResult> newEnsemblePlacementResult; + try { + /* + * since rw bookies are not spread across UDs in zones, newEnsemble + * of writeQuorum 6 is expected to fail. + */ + zepp.newEnsemble(6, 6, 2, null, new HashSet<>()); + fail("newEnsemble is expected to fail because writeQuorum cannt be created with insufficient UDs"); + } catch (BKException.BKNotEnoughBookiesException bkne) { + // expected NotEnoughBookiesException + } + + int ensSize = 6; + int writeQuorum = 3; + /* + * though bookies are not spread across UDs in zones, newEnsemble would + * succeed because writeQuorum is just 3. + */ + newEnsemblePlacementResult = zepp.newEnsemble(ensSize, writeQuorum, 2, null, new HashSet<>()); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, + newEnsemblePlacementResult.isAdheringToPolicy()); + List newEnsemble = newEnsemblePlacementResult.getResult(); + Set newEnsembleSet = new HashSet(newEnsemble); + assertTrue("New ensemble should contain all 6 rw bookies in non-default fault domains", + rwAddrs.containsAll(newEnsembleSet) && (newEnsembleSet.size() == 6)); + assertTrue("Bookie from default faultDomain shouldn't be part of ensemble", + Collections.disjoint(newEnsembleSet, bookiesInDefaultFaultDomain)); + + Set zonesOfBookiesInAWriteQuorum = new HashSet(); + for (int i = 0; i < 6; i++) { + zonesOfBookiesInAWriteQuorum.clear(); + for (int j = 0; j < writeQuorum; j++) { + zonesOfBookiesInAWriteQuorum + .add(zepp.getZoneAwareNodeLocation(newEnsemble.get((i + j) % ensSize)).getZone()); + } + assertEquals("Since bookies are not spread across multiple UDs in a zone, write quorum should" + + " contain bookies from all 3 zones", 3, zonesOfBookiesInAWriteQuorum.size()); + } + } + + @Test + public void testUniqueUds() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); + BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + BookieSocketAddress addr11 = new BookieSocketAddress("127.0.0.12", 3181); + BookieSocketAddress addr12 = new BookieSocketAddress("127.0.0.13", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone1/ud3"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone1/ud3"); + StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr10.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr11.getHostName(), "/zone2/ud3"); + StaticDNSResolver.addNodeToRack(addr12.getHostName(), "/zone2/ud3"); + + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(4); + newConf.setMinNumZonesPerWriteQuorum(2); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr6); + rwAddrs.add(addr7); + rwAddrs.add(addr8); + rwAddrs.add(addr9); + rwAddrs.add(addr10); + rwAddrs.add(addr11); + rwAddrs.add(addr12); + + zepp.onClusterChanged(rwAddrs, roAddrs); + /* + * Since there are enough bookies in different UDs in 2 zones + * (MinNumZonesPerWriteQuorum), new ensemble should succeed. + */ + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(6, 6, 2, null, + new HashSet<>()); + List newEnsembleList = newEnsemblePlacementResult.getResult(); + Set newEnsembleSet = new HashSet(newEnsembleList); + assertTrue("New ensemble should contain 6 rw bookies in non-default fault domains", + rwAddrs.containsAll(newEnsembleSet) && (newEnsembleSet.size() == 6)); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, + newEnsemblePlacementResult.isAdheringToPolicy()); + Set bookiesNetworkLocations = new HashSet(); + + for (BookieSocketAddress bookieAddr : newEnsembleSet) { + bookiesNetworkLocations.add(zepp.resolveNetworkLocation(bookieAddr)); + } + /* + * Since there are enough bookies in different UDs, bookies from same + * zone should be from different UDs. + */ + assertTrue("Bookies should be from different UpgradeDomains if they belong to same zone", + (bookiesNetworkLocations.size() == 6)); + List bookiesNodeLocationList = new ArrayList(); + for (BookieSocketAddress bookieAddr : newEnsembleList) { + bookiesNodeLocationList.add(zepp.getZoneAwareNodeLocation(bookieAddr)); + } + for (int i = 0; i < 5; i++) { + /* + * in newEnsemble order, bookies should be from alternating zones. + */ + assertNotEquals("Alternate bookies should be from different zones", + bookiesNodeLocationList.get(i).getZone(), bookiesNodeLocationList.get(i + 1).getZone()); + } + } + + @Test + public void testNewBookieUniformDistributionWithMinZoneAndMinUDs() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); + BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + BookieSocketAddress addr11 = new BookieSocketAddress("127.0.0.12", 3181); + BookieSocketAddress addr12 = new BookieSocketAddress("127.0.0.13", 3181); + BookieSocketAddress addr13 = new BookieSocketAddress("127.0.0.14", 3181); + BookieSocketAddress addr14 = new BookieSocketAddress("127.0.0.15", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr10.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr11.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr12.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr13.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack(addr14.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr6); + rwAddrs.add(addr7); + rwAddrs.add(addr8); + rwAddrs.add(addr9); + rwAddrs.add(addr10); + rwAddrs.add(addr11); + rwAddrs.add(addr12); + rwAddrs.add(addr13); + rwAddrs.add(addr14); + + int minNumZonesPerWriteQuorum = 3; + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(5); + newConf.setMinNumZonesPerWriteQuorum(minNumZonesPerWriteQuorum); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + zepp.onClusterChanged(rwAddrs, roAddrs); + Set excludedBookies = new HashSet(); + + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(6, 6, 4, null, + excludedBookies); + List newEnsembleList = newEnsemblePlacementResult.getResult(); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, + newEnsemblePlacementResult.isAdheringToPolicy()); + Set newEnsembleSet = new HashSet(newEnsembleList); + Set bookiesNetworkLocationsSet = new HashSet(); + List bookiesNodeLocationList = new ArrayList(); + for (BookieSocketAddress bookieAddr : newEnsembleSet) { + bookiesNetworkLocationsSet.add(zepp.resolveNetworkLocation(bookieAddr)); + } + for (BookieSocketAddress bookieAddr : newEnsembleList) { + bookiesNodeLocationList.add(zepp.getZoneAwareNodeLocation(bookieAddr)); + } + /* + * since there are enough bookies from minNumZonesPerWriteQuorum (3), + * bookies should be from 3 different zones and 2 different UDs. + */ + assertTrue("Bookies should be from different UpgradeDomains if they belong to same zone", + (bookiesNetworkLocationsSet.size() == 6)); + Set zonesOfFirstNodes = new HashSet(); + for (int i = 0; i < minNumZonesPerWriteQuorum; i++) { + zonesOfFirstNodes.add(bookiesNodeLocationList.get(i).getZone()); + } + assertEquals("Num of zones", minNumZonesPerWriteQuorum, zonesOfFirstNodes.size()); + for (int i = 0; i < minNumZonesPerWriteQuorum; i++) { + assertEquals("Zone", bookiesNodeLocationList.get(i).getZone(), + bookiesNodeLocationList.get(i + minNumZonesPerWriteQuorum).getZone()); + assertNotEquals("UpgradeDomain", bookiesNodeLocationList.get(i).getUpgradeDomain(), + bookiesNodeLocationList.get(i + minNumZonesPerWriteQuorum).getUpgradeDomain()); + } + } + + @Test + public void testReplaceBookie() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); + BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + BookieSocketAddress addr11 = new BookieSocketAddress("127.0.0.12", 3181); + BookieSocketAddress addr12 = new BookieSocketAddress("127.0.0.13", 3181); + BookieSocketAddress addr13 = new BookieSocketAddress("127.0.0.14", 3181); + BookieSocketAddress addr14 = new BookieSocketAddress("127.0.0.15", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr10.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr11.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr12.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr13.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack(addr14.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(3); + newConf.setMinNumZonesPerWriteQuorum(3); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr6); + rwAddrs.add(addr7); + rwAddrs.add(addr8); + rwAddrs.add(addr9); + rwAddrs.add(addr10); + rwAddrs.add(addr11); + rwAddrs.add(addr12); + rwAddrs.add(addr13); + rwAddrs.add(addr14); + + zepp.onClusterChanged(rwAddrs, roAddrs); + List ensemble = new ArrayList(); + Set excludedBookies = new HashSet(); + ensemble.add(addr1); + ensemble.add(addr5); + ensemble.add(addr9); + ensemble.add(addr3); + ensemble.add(addr7); + ensemble.add(addr11); + /* + * since addr5 (/zone2/ud1) is already part of ensemble of size 6, write + * quorum of size 6, to replace bookie addr7 (/zone2/ud2), new bookie + * should be from /zone2/ud2. + */ + PlacementResult replacePlacementResult = zepp.replaceBookie(6, 6, 2, null, ensemble, addr7, + excludedBookies); + BookieSocketAddress replacedBookie = replacePlacementResult.getResult(); + assertEquals("replaced bookie", addr8, replacedBookie); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, + replacePlacementResult.isAdheringToPolicy()); + + excludedBookies.add(addr8); + /* + * here addr8 is excluded, and writeQuorumSize is 3. So to replace + * bookie addr7, addr6 (belonging to same zone) is the candidate. + */ + replacePlacementResult = zepp.replaceBookie(6, 3, 2, null, ensemble, addr7, excludedBookies); + replacedBookie = replacePlacementResult.getResult(); + assertEquals("replaced bookie", addr6, replacedBookie); + + excludedBookies.add(addr6); + try { + /* + * here addr6 is also excluded, so replaceBookie should fail. + */ + replacedBookie = zepp.replaceBookie(6, 3, 2, null, ensemble, addr7, excludedBookies).getResult(); + fail("Expected BKNotEnoughBookiesException for replaceBookie with added excludedBookies"); + } catch (BKException.BKNotEnoughBookiesException bkne) { + // expected NotEnoughBookiesException + } + } + + @Test + public void testReplaceBookieMinUDs() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); + BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + BookieSocketAddress addr11 = new BookieSocketAddress("127.0.0.12", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr10.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack(addr11.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(4); + newConf.setMinNumZonesPerWriteQuorum(3); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr6); + rwAddrs.add(addr7); + rwAddrs.add(addr8); + rwAddrs.add(addr9); + rwAddrs.add(addr10); + rwAddrs.add(addr11); + + zepp.onClusterChanged(rwAddrs, roAddrs); + List ensemble = new ArrayList(); + Set excludedBookies = new HashSet(); + ensemble.add(addr1); + ensemble.add(addr2); + ensemble.add(addr3); + ensemble.add(addr4); + ensemble.add(addr5); + ensemble.add(addr6); + /* + * though all the remaining non-default bookies are in /zone3/ud2, for + * replacing addr4 replaceBookie should be able to find some other + * bookie in /zone3/ud2. + */ + PlacementResult replaceResponse = zepp.replaceBookie(6, 6, 2, null, ensemble, addr4, + excludedBookies); + BookieSocketAddress replacedBookie = replaceResponse.getResult(); + assertEquals("replaced bookie", "/zone3/ud2", zepp.resolveNetworkLocation(replacedBookie)); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, + replaceResponse.isAdheringToPolicy()); + } + + @Test + public void testAreAckedBookiesAdheringToPlacementPolicy() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/zone1/ud3"); + StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/zone2/ud3"); + StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/zone3/ud3"); + + ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(4); + newConf.setMinNumZonesPerWriteQuorum(2); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr6); + rwAddrs.add(addr7); + rwAddrs.add(addr8); + rwAddrs.add(addr9); + + zepp.onClusterChanged(rwAddrs, roAddrs); + Set ackedBookies = new HashSet(); + ackedBookies.add(addr1); + ackedBookies.add(addr4); + assertFalse("since both the bookies are in the same zone, it should return false", + zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 10, 2)); + ackedBookies.clear(); + ackedBookies.add(addr1); + ackedBookies.add(addr2); + assertFalse("since ackQuorumSize is 3, it should return false", + zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 10, 3)); + assertTrue("since ackQuorumSize is 2 and bookies are from minNumZonesPerWriteQuorum it should return true", + zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 10, 2)); + + zepp.uninitalize(); + newConf = (ClientConfiguration) this.conf.clone(); + newConf.setDesiredNumZonesPerWriteQuorum(4); + newConf.setMinNumZonesPerWriteQuorum(4); + zepp = new ZoneawareEnsemblePlacementPolicy(); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + zepp.onClusterChanged(rwAddrs, roAddrs); + ackedBookies.clear(); + ackedBookies.add(addr1); + ackedBookies.add(addr2); + ackedBookies.add(addr3); + assertFalse("since minNumZonesPerWriteQuorum is set to 4, it should return false", + zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 4, 3)); + assertTrue("since writeQuorumSize is set to 3, it should return true", + zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 3, 3)); + ackedBookies.clear(); + ackedBookies.add(addr1); + ackedBookies.add(addr2); + ackedBookies.add(addr4); + assertFalse("since bookies are in just 2 zones but not in 3 zones, it should return false", + zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 3, 3)); + } + + @Test + public void testWeightedPlacement() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + // Update cluster + Set addrs = new HashSet(); + addrs.add(addr1); + addrs.add(addr2); + addrs.add(addr3); + addrs.add(addr4); + addrs.add(addr5); + + int multiple = 10; + + ClientConfiguration newConf = new ClientConfiguration(conf); + newConf.addConfiguration(conf); + newConf.setDiskWeightBasedPlacementEnabled(true); + /* + * since BookieMaxWeightMultipleForWeightBasedPlacement is set to -1, + * there is no max cap on weight. + */ + newConf.setBookieMaxWeightMultipleForWeightBasedPlacement(-1); + newConf.setMinNumZonesPerWriteQuorum(0); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + zepp.onClusterChanged(addrs, new HashSet()); + Map bookieInfoMap = new HashMap(); + bookieInfoMap.put(addr1, new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr2, new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr3, new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr4, new BookieInfo(multiple * 100L, multiple * 100L)); + bookieInfoMap.put(addr5, new BookieInfo(100L, 100L)); + zepp.updateBookieInfo(bookieInfoMap); + + Map selectionCounts = new HashMap(); + int numTries = 50000; + EnsemblePlacementPolicy.PlacementResult> newEnsembleResponse; + List newEnsemble; + for (BookieSocketAddress addr : addrs) { + selectionCounts.put(addr, (long) 0); + } + for (int i = 0; i < numTries; i++) { + // new ensemble response + newEnsembleResponse = zepp.newEnsemble(1, 1, 1, null, new HashSet()); + newEnsemble = newEnsembleResponse.getResult(); + selectionCounts.put(newEnsemble.get(0), selectionCounts.get(newEnsemble.get(0)) + 1); + } + double observedMultiple = ((double) selectionCounts.get(addr4) / (double) selectionCounts.get(addr3)); + /* + * since there is no cap on maxWeight, observedMultiple should be + * roughly equal to multiple + */ + assertTrue("Weights not being honored " + observedMultiple, Math.abs(observedMultiple - multiple) < 1); + + selectionCounts.clear(); + selectionCounts.put(addr3, (long) 0); + selectionCounts.put(addr4, (long) 0); + newEnsemble = new ArrayList(); + newEnsemble.add(addr2); + Set excludedBookies = new HashSet(); + excludedBookies.add(addr1); + EnsemblePlacementPolicy.PlacementResult replacedBookieResponse; + BookieSocketAddress replacedBookie; + for (int i = 0; i < numTries; i++) { + // replace bookie response + replacedBookieResponse = zepp.replaceBookie(1, 1, 1, null, newEnsemble, addr2, excludedBookies); + replacedBookie = replacedBookieResponse.getResult(); + /* + * only addr3 and addr4 are eligible for replacedBookie. + */ + assertTrue("replaced : " + replacedBookie, addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); + selectionCounts.put(replacedBookie, selectionCounts.get(replacedBookie) + 1); + } + observedMultiple = ((double) selectionCounts.get(addr4) / (double) selectionCounts.get(addr3)); + /* + * since there is no cap on maxWeight, observedMultiple should be + * roughly equal to multiple + */ + assertTrue("Weights not being honored " + observedMultiple, Math.abs(observedMultiple - multiple) < 1); + } + + @Test + public void testPlacementOnStabilizeNetworkTopology() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone4/ud1"); + + zepp = new ZoneawareEnsemblePlacementPolicy(); + ClientConfiguration confLocal = new ClientConfiguration(); + confLocal.addConfiguration(conf); + confLocal.setNetworkTopologyStabilizePeriodSeconds(99999); + zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set addrs = new HashSet(); + addrs.add(addr1); + addrs.add(addr2); + addrs.add(addr3); + addrs.add(addr4); + zepp.onClusterChanged(addrs, new HashSet()); + // addr4 left + addrs.remove(addr4); + Set deadBookies = zepp.onClusterChanged(addrs, new HashSet()); + assertTrue(deadBookies.isEmpty()); + + // we will never use addr4 even it is in the stabilized network topology + for (int i = 0; i < 5; i++) { + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = zepp.newEnsemble(3, 3, + 2, null, new HashSet()); + List ensemble = ensembleResponse.getResult(); + assertFalse(ensemble.contains(addr4)); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, + ensembleResponse.isAdheringToPolicy()); + } + + // we could still use addr4 for urgent allocation if it is just bookie + // flapping + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = zepp.newEnsemble(4, 4, 2, + null, new HashSet()); + List ensemble = ensembleResponse.getResult(); + assertTrue(ensemble.contains(addr4)); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, + ensembleResponse.isAdheringToPolicy()); + } + + @Test + public void testCreateNewEnsembleRandomly() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone1/ud1"); + + zepp = new ZoneawareEnsemblePlacementPolicy(); + ClientConfiguration confLocal = new ClientConfiguration(); + confLocal.addConfiguration(conf); + confLocal.setEnforceStrictZoneawarePlacement(false); + confLocal.setMinNumZonesPerWriteQuorum(3); + confLocal.setDesiredNumZonesPerWriteQuorum(4); + zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + Set excludeBookies = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + excludeBookies.add(addr5); + zepp.onClusterChanged(rwAddrs, roAddrs); + /* + * if enforceStrictZoneawarePlacement is not enabled, then there is no + * restrictions on ensSize and writeQSize and also bookie belonging to + * DEFAULT_ZONE_AND_UPGRADEDOMAIN can be a candidate. + */ + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(4, 3, 2, null, + excludeBookies); + Set newEnsembleSet = new HashSet( + newEnsemblePlacementResult.getResult()); + assertEquals("New ensemble should contain 4 rw bookies", 4, newEnsembleSet.size()); + assertFalse("excludeBookie should not be included in the ensemble", newEnsembleSet.contains(addr5)); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, + newEnsemblePlacementResult.isAdheringToPolicy()); + + rwAddrs.remove(addr4); + roAddrs.add(addr4); + zepp.onClusterChanged(rwAddrs, roAddrs); + try { + /* + * since there is no bookie available, newEnsemble should fail. + */ + zepp.newEnsemble(4, 3, 2, null, excludeBookies); + fail("Creation of new ensemble randomly should fail because of not sufficient bookies"); + } catch (BKException.BKNotEnoughBookiesException bkne) { + // expected NotEnoughBookiesException + } + } + + @Test + public void testReplaceBookieRandomly() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr7.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + zepp = new ZoneawareEnsemblePlacementPolicy(); + ClientConfiguration confLocal = new ClientConfiguration(); + confLocal.addConfiguration(conf); + confLocal.setEnforceStrictZoneawarePlacement(false); + confLocal.setMinNumZonesPerWriteQuorum(3); + confLocal.setDesiredNumZonesPerWriteQuorum(4); + zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + Set excludeBookies = new HashSet(); + rwAddrs.add(addr1); + rwAddrs.add(addr2); + rwAddrs.add(addr3); + rwAddrs.add(addr4); + rwAddrs.add(addr5); + rwAddrs.add(addr7); + + roAddrs.add(addr6); + excludeBookies.add(addr5); + zepp.onClusterChanged(rwAddrs, roAddrs); + List ensembleList = new ArrayList(); + ensembleList.add(addr1); + ensembleList.add(addr2); + ensembleList.add(addr3); + ensembleList.add(addr4); + + PlacementResult replaceResponse = zepp.replaceBookie(4, 3, 2, null, ensembleList, addr3, + excludeBookies); + BookieSocketAddress replaceBookie = replaceResponse.getResult(); + /* + * if enforceStrictZoneawarePlacement is not enabled, then there is no + * restrictions on ensSize and writeQSize and also bookie belonging to + * DEFAULT_ZONE_AND_UPGRADEDOMAIN can be a candidate. + */ + assertEquals("ReplaceBookie candidate", addr7, replaceBookie); + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, + replaceResponse.isAdheringToPolicy()); + + rwAddrs.remove(addr7); + excludeBookies.add(addr7); + zepp.onClusterChanged(rwAddrs, roAddrs); + try { + /* + * since there is no bookie available, replaceBookie should fail. + */ + zepp.replaceBookie(4, 3, 2, null, ensembleList, addr3, excludeBookies); + fail("ReplaceBookie should fail because of unavailable bookies"); + } catch (BKException.BKNotEnoughBookiesException bkne) { + // expected NotEnoughBookiesException + } + } + + @Test + public void testIsEnsembleAdheringToPlacementPolicy() throws Exception { + zepp.uninitalize(); + updateMyUpgradeDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + // Update cluster + BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); + BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); + BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); + BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); + BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + + // update dns mapping + StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/zone1/ud1"); + StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/zone1/ud2"); + StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone2/ud1"); + StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/zone2/ud2"); + StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/zone3/ud1"); + StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/zone3/ud2"); + StaticDNSResolver.addNodeToRack(addr10.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + zepp = new ZoneawareEnsemblePlacementPolicy(); + ClientConfiguration confLocal = new ClientConfiguration(); + confLocal.addConfiguration(conf); + confLocal.setEnforceStrictZoneawarePlacement(true); + confLocal.setMinNumZonesPerWriteQuorum(2); + confLocal.setDesiredNumZonesPerWriteQuorum(3); + zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); + + List ensemble = new ArrayList(); + ensemble.add(addr1); + ensemble.add(addr2); + ensemble.add(addr3); + // all bookies in same rack + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); + + ensemble.clear(); + ensemble.add(addr1); + ensemble.add(addr2); + ensemble.add(addr4); + // bookies spread across minZones + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); + + ensemble.clear(); + ensemble.add(addr1); + ensemble.add(addr4); + ensemble.add(addr7); + // bookies spread across desirednumofzones + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); + + ensemble.clear(); + ensemble.add(addr1); + ensemble.add(addr4); + // writeQuorum should be greater than minZones + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 2, 2)); + + ensemble.clear(); + ensemble.add(addr2); + ensemble.add(addr3); + ensemble.add(addr4); + // bookies from zone1 (addr2 and addr3) are in same UD + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); + + ensemble.clear(); + ensemble.add(addr1); + ensemble.add(addr4); + ensemble.add(addr7); + ensemble.add(addr10); + // bookie from default faultdomain will cause PlacementPolicyAdherence + // to fail + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 4, 2)); + + ensemble.clear(); + ensemble.add(addr1); + ensemble.add(addr4); + ensemble.add(addr7); + ensemble.add(addr8); + ensemble.add(addr9); + // bookies are spread across desired zones and bookie from same zone are + // spread across 2 UDs + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 5, 2)); + + ensemble.clear(); + ensemble.add(addr1); + ensemble.add(addr4); + ensemble.add(addr7); + ensemble.add(addr2); + ensemble.add(addr8); + ensemble.add(addr9); + /* + * writeset of addr2, addr8 and addr9 fails, because addr8 and addr9 + * belong to z3u2 + */ + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); + + ensemble.clear(); + ensemble.add(addr1); + ensemble.add(addr4); + ensemble.add(addr9); + ensemble.add(addr2); + ensemble.add(addr8); + ensemble.add(addr7); + /* + * writeset of addr9, addr2 and addr8 fails, because addr8 and addr9 + * belong to z3u2 + */ + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); + + ensemble.clear(); + ensemble.add(addr1); + ensemble.add(addr4); + ensemble.add(addr9); + ensemble.add(addr2); + ensemble.add(addr7); + ensemble.add(addr8); + /* + * writeset of addr2, addr7 and addr8 just meets soft. + */ + assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, + zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java index 69134a3a626..1d786f9fdc2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -37,6 +37,7 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; +import org.apache.bookkeeper.client.ZoneawareEnsemblePlacementPolicy; import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -178,18 +179,26 @@ public void testPlacementPolicyCheckWithBookiesFromDifferentRacks() throws Excep ServerConfiguration servConf = new ServerConfiguration(bsConfs.get(0)); servConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); - setServerConfigProperties(servConf); + setServerConfigPropertiesForRackPlacement(servConf); MutableObject auditorRef = new MutableObject(); try { TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); /* * since all of the bookies are in different racks, there shouldn't be any ledger not adhering * to placement policy. */ assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", 0, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + /* + * since all of the bookies are in different racks, there shouldn't be any ledger softly adhering + * to placement policy. + */ + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY guage value", 0, + ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); } finally { Auditor auditor = auditorRef.getValue(); if (auditor != null) { @@ -259,7 +268,7 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPlacementPolicy() th ServerConfiguration servConf = new ServerConfiguration(bsConfs.get(0)); servConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); - setServerConfigProperties(servConf); + setServerConfigPropertiesForRackPlacement(servConf); MutableObject auditorRef = new MutableObject(); try { TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); @@ -267,6 +276,10 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPlacementPolicy() th .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY guage value", + 0, ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); } finally { Auditor auditor = auditorRef.getValue(); if (auditor != null) { @@ -357,7 +370,7 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPolicyWithMultipleSe ServerConfiguration servConf = new ServerConfiguration(bsConfs.get(0)); servConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); - setServerConfigProperties(servConf); + setServerConfigPropertiesForRackPlacement(servConf); MutableObject auditorRef = new MutableObject(); try { TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); @@ -365,6 +378,10 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPolicyWithMultipleSe .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY gauge value", numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY gauge value", + 0, ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); } finally { Auditor auditor = auditorRef.getValue(); if (auditor != null) { @@ -373,10 +390,140 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPolicyWithMultipleSe } } - private void setServerConfigProperties(ServerConfiguration servConf) { + @Test + public void testZoneawarePlacementPolicyCheck() throws Exception { + int numOfBookies = 6; + int numOfLedgersNotAdheringToPlacementPolicy = 0; + int numOfLedgersSoftlyAdheringToPlacementPolicy = 0; + List bookieAddresses = new ArrayList(); + RegistrationManager regManager = driver.getRegistrationManager(); + + /* + * 6 bookies - 3 zones and 2 uds + */ + for (int i = 0; i < numOfBookies; i++) { + BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); + bookieAddresses.add(bookieAddress); + regManager.registerBookie(bookieAddress.toString(), false); + String zone = "/zone" + (i % 3); + String upgradeDomain = "/ud" + (i % 2); + String networkLocation = zone + upgradeDomain; + StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), networkLocation); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + + ServerConfiguration servConf = new ServerConfiguration(bsConfs.get(0)); + servConf.setDesiredNumZonesPerWriteQuorum(3); + servConf.setMinNumZonesPerWriteQuorum(2); + setServerConfigPropertiesForZonePlacement(servConf); + + /* + * this closed ledger adheres to ZoneAwarePlacementPolicy, since + * ensemble is spread across 3 zones and 2 UDs + */ + LedgerMetadata initMeta = LedgerMetadataBuilder.create() + .withEnsembleSize(6) + .withWriteQuorumSize(6) + .withAckQuorumSize(2) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(1L, initMeta).get(); + + /* + * this is non-closed ledger, so though ensemble is not adhering to + * placement policy (since ensemble is not multiple of writeQuorum), + * this shouldn't be reported + */ + initMeta = LedgerMetadataBuilder.create() + .withEnsembleSize(6) + .withWriteQuorumSize(5) + .withAckQuorumSize(2) + .newEnsembleEntry(0L, bookieAddresses) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(2L, initMeta).get(); + + /* + * this is closed ledger, since ensemble is not multiple of writeQuorum, + * this ledger is not adhering to placement policy. + */ + initMeta = LedgerMetadataBuilder.create() + .withEnsembleSize(6) + .withWriteQuorumSize(5) + .withAckQuorumSize(2) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(3L, initMeta).get(); + numOfLedgersNotAdheringToPlacementPolicy++; + + /* + * this closed ledger adheres softly to ZoneAwarePlacementPolicy, since + * ensemble/writeQuorum of size 4 has spread across just + * minNumZonesPerWriteQuorum (2). + */ + List newEnsemble = new ArrayList(); + newEnsemble.add(bookieAddresses.get(0)); + newEnsemble.add(bookieAddresses.get(1)); + newEnsemble.add(bookieAddresses.get(3)); + newEnsemble.add(bookieAddresses.get(4)); + initMeta = LedgerMetadataBuilder.create() + .withEnsembleSize(4) + .withWriteQuorumSize(4) + .withAckQuorumSize(2) + .newEnsembleEntry(0L, newEnsemble) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(4L, initMeta).get(); + numOfLedgersSoftlyAdheringToPlacementPolicy++; + + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", + numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY guage value", + numOfLedgersSoftlyAdheringToPlacementPolicy, + ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + } + } + + private void setServerConfigPropertiesForRackPlacement(ServerConfiguration servConf) { + setServerConfigProperties(servConf, RackawareEnsemblePlacementPolicy.class.getName()); + } + + private void setServerConfigPropertiesForZonePlacement(ServerConfiguration servConf) { + setServerConfigProperties(servConf, ZoneawareEnsemblePlacementPolicy.class.getName()); + } + + private void setServerConfigProperties(ServerConfiguration servConf, String ensemblePlacementPolicyClass) { servConf.setProperty(REPP_DNS_RESOLVER_CLASS, StaticDNSResolver.class.getName()); - servConf.setProperty(ClientConfiguration.ENSEMBLE_PLACEMENT_POLICY, - RackawareEnsemblePlacementPolicy.class.getName()); + servConf.setProperty(ClientConfiguration.ENSEMBLE_PLACEMENT_POLICY, ensemblePlacementPolicyClass); servConf.setAuditorPeriodicCheckInterval(0); servConf.setAuditorPeriodicBookieCheckInterval(0); servConf.setAuditorPeriodicPlacementPolicyCheckInterval(1000); diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index 7cccc6d1d09..081428e2caf 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -647,6 +647,18 @@ groups: - param: enforceMinNumFaultDomainsForWrite description: | 'enforceMinNumFaultDomainsForWrite' enforces EnsemblePlacementPolicy to check if a write has made it to bookies in 'minNumRacksPerWriteQuorum' number of fault domains, before acknowledging the write back. + - param: minNumZonesPerWriteQuorum + description: | + minimum number of zones per write quorum in ZoneawareEnsemblePlacementPolicy. ZoneawareEnsemblePlacementPolicy would get bookies from atleast 'minNumZonesPerWriteQuorum' racks for a writeQuorum. + default: 2 + - param: desiredNumZonesPerWriteQuorum + description: | + desired number of zones per write quorum in ZoneawareEnsemblePlacementPolicy. ZoneawareEnsemblePlacementPolicy will try to get bookies from 'desiredNumZonesPerWriteQuorum' zones for a writeQuorum. + default: 3 + - param: enforceStrictZoneawarePlacement + description: | + in ZoneawareEnsemblePlacementPolicy if strict placement is enabled then minZones/desiredZones in writeQuorum would be maintained otherwise it will pick nodes randomly. + default: true - name: AutoRecovery auditor settings params: From 6bfd567999d2ab005e370d1d91a6eeb593362a36 Mon Sep 17 00:00:00 2001 From: Matteo Minardi Date: Fri, 12 Jul 2019 10:01:55 +0200 Subject: [PATCH 0312/1642] Implement a method to get all the Bookies In this PR I'm adding a `BokKeeperAdmin#getAllBookies` to retrive all the bookies (cookie) retriving them from the `RegistrationClient` This is still a work in progress, I'm adding some tests in the next commit. Reviewers: Enrico Olivelli , Sijie Guo This closes #2117 from mino181295/allbookies --- .../apache/bookkeeper/bookie/BookieShell.java | 22 +++++++++-- .../bookkeeper/client/BookKeeperAdmin.java | 10 +++++ .../bookkeeper/client/BookieWatcher.java | 1 + .../bookkeeper/client/BookieWatcherImpl.java | 10 +++++ .../discover/RegistrationClient.java | 7 ++++ .../discover/ZKRegistrationClient.java | 11 +++++- .../commands/bookies/ListBookiesCommand.java | 15 +++++++- .../bookkeeper/bookie/BookieShellTest.java | 17 +++++++++ .../client/BookKeeperAdminTest.java | 38 +++++++++++++++++++ .../discover/MockRegistrationClient.java | 8 ++++ .../discover/TestZkRegistrationClient.java | 38 +++++++++++++++++++ .../metadata/etcd/EtcdRegistrationClient.java | 6 +++ 12 files changed, 176 insertions(+), 7 deletions(-) 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 1caa1e18c74..98490520799 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 @@ -1104,24 +1104,38 @@ class ListBookiesCmd extends MyCommand { super(CMD_LISTBOOKIES); opts.addOption("rw", "readwrite", false, "Print readwrite bookies"); opts.addOption("ro", "readonly", false, "Print readonly bookies"); + opts.addOption("a", "all", false, "Print all bookies"); // @deprecated 'rw'/'ro' option print both hostname and ip, so this option is not needed anymore opts.addOption("h", "hostnames", false, "Also print hostname of the bookie"); } @Override public int runCmd(CommandLine cmdLine) throws Exception { + int passedCommands = 0; + boolean readwrite = cmdLine.hasOption("rw"); + if (readwrite) { + passedCommands++; + } boolean readonly = cmdLine.hasOption("ro"); + if (readonly) { + passedCommands++; + } + boolean all = cmdLine.hasOption("a"); + if (all) { + passedCommands++; + } - if ((!readwrite && !readonly) || (readwrite && readonly)) { - LOG.error("One and only one of -readwrite and -readonly must be specified"); + if (passedCommands != 1) { + LOG.error("One and only one of -readwrite, -readonly and -all must be specified"); printUsage(); return 1; } ListBookiesCommand.Flags flags = new ListBookiesCommand.Flags() .readwrite(readwrite) - .readonly(readonly); + .readonly(readonly) + .all(all); ListBookiesCommand command = new ListBookiesCommand(flags); @@ -1136,7 +1150,7 @@ String getDescription() { @Override String getUsage() { - return "listbookies [-readwrite|-readonly] [-hostnames]"; + return "listbookies [-readwrite|-readonly|-all] [-hostnames]"; } @Override 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 5a5903d40cf..870db93527a 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 @@ -214,6 +214,16 @@ public Collection getAvailableBookies() return bkc.bookieWatcher.getBookies(); } + /** + * Get a list of all bookies including the not available ones. + * + * @return a collection of bookie addresses + */ + public Collection getAllBookies() + throws BKException { + return bkc.bookieWatcher.getAllBookies(); + } + /** * Get a list of readonly bookies synchronously. * 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 0f760a9d93e..b88156943be 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 @@ -26,6 +26,7 @@ interface BookieWatcher { Set getBookies() throws BKException; + Set getAllBookies() throws BKException; Set getReadOnlyBookies() throws BKException; /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index f26ee61eddd..e38dcf6d787 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -142,6 +142,16 @@ public Set getBookies() throws BKException { } } + @Override + public Set getAllBookies() throws BKException { + try { + return FutureUtils.result(registrationClient.getAllBookies(), EXCEPTION_FUNC).getValue(); + } catch (BKInterruptedException ie) { + Thread.currentThread().interrupt(); + throw ie; + } + } + @Override public Set getReadOnlyBookies() throws BKException { 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 b53fd240545..4056329021f 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 @@ -51,6 +51,13 @@ interface RegistrationListener { */ CompletableFuture>> getWritableBookies(); + /** + * Get the list of all bookies identifiers. + * + * @return a future represents the list of writable bookies. + */ + CompletableFuture>> getAllBookies(); + /** * Get the list of readonly bookie identifiers. * 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 index ce19751c9d3..6629f0c2f1e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java @@ -19,6 +19,7 @@ package org.apache.bookkeeper.discover; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; +import static org.apache.bookkeeper.util.BookKeeperConstants.COOKIE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; import com.google.common.collect.Sets; @@ -173,6 +174,7 @@ public void close() { // registration paths private final String bookieRegistrationPath; + private final String bookieAllRegistrationPath; private final String bookieReadonlyRegistrationPath; public ZKRegistrationClient(ZooKeeper zk, @@ -182,6 +184,7 @@ public ZKRegistrationClient(ZooKeeper zk, this.scheduler = scheduler; this.bookieRegistrationPath = ledgersRootPath + "/" + AVAILABLE_NODE; + this.bookieAllRegistrationPath = ledgersRootPath + "/" + COOKIE_NODE; this.bookieReadonlyRegistrationPath = this.bookieRegistrationPath + "/" + READONLY; } @@ -199,6 +202,11 @@ public CompletableFuture>> getWritableBookies return getChildren(bookieRegistrationPath, null); } + @Override + public CompletableFuture>> getAllBookies() { + return getChildren(bookieAllRegistrationPath, null); + } + @Override public CompletableFuture>> getReadOnlyBookies() { return getChildren(bookieReadonlyRegistrationPath, null); @@ -209,8 +217,7 @@ private CompletableFuture>> getChildren(Strin zk.getChildren(regPath, watcher, (rc, path, ctx, children, stat) -> { if (Code.OK != rc) { ZKException zke = new ZKException(); - zke.fillInStackTrace(); - future.completeExceptionally(zke); + future.completeExceptionally(zke.fillInStackTrace()); return; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java index 4563530e3e0..33e1a2d74c0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java @@ -64,15 +64,18 @@ public static class Flags extends CliFlags { private boolean readwrite = false; @Parameter(names = { "-ro", "--readonly" }, description = "Print readonly bookies") private boolean readonly = false; + @Parameter(names = { "-a", "--all" }, description = "Print all bookies") + private boolean all = false; } @Override protected void run(RegistrationClient regClient, Flags flags) throws Exception { - if (!flags.readwrite && !flags.readonly) { + if (!flags.readwrite && !flags.readonly && !flags.all) { // case: no args is provided. list all the bookies by default. flags.readwrite = true; flags.readonly = true; + flags.all = true; } boolean hasBookies = false; @@ -96,6 +99,16 @@ protected void run(RegistrationClient regClient, Flags flags) throws Exception { hasBookies = true; } } + if (flags.all) { + Set bookies = result( + regClient.getAllBookies() + ).getValue(); + if (!bookies.isEmpty()) { + System.out.println("All Bookies :"); + printBookies(bookies); + hasBookies = true; + } + } if (!hasBookies) { System.err.println("No bookie exists!"); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java index 9fcb2fb546a..b0e0a453faf 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java @@ -395,6 +395,7 @@ public void testListBookiesCmdReadOnly() throws Exception { .apply(same(shell.bkConf), same(mockListBookiesFlags)); verify(mockListBookiesFlags, times(1)).readonly(true); verify(mockListBookiesFlags, times(1)).readwrite(false); + verify(mockListBookiesFlags, times(1)).all(false); } @Test @@ -408,5 +409,21 @@ public void testListBookiesCmdReadWrite() throws Exception { .apply(same(shell.bkConf), same(mockListBookiesFlags)); verify(mockListBookiesFlags, times(1)).readonly(false); verify(mockListBookiesFlags, times(1)).readwrite(true); + verify(mockListBookiesFlags, times(1)).all(false); } + + @Test + public void testListBookiesCmdAll() throws Exception { + assertEquals(0, shell.run(new String[] { + "listbookies", "-a" + })); + verifyNew(ListBookiesCommand.class, times(1)) + .withArguments(same(mockListBookiesFlags)); + verify(mockListBookiesCommand, times(1)) + .apply(same(shell.bkConf), same(mockListBookiesFlags)); + verify(mockListBookiesFlags, times(1)).readonly(false); + verify(mockListBookiesFlags, times(1)).readwrite(false); + verify(mockListBookiesFlags, times(1)).all(true); + } + } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index 2f419b5e6a7..03f182d02eb 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -31,6 +31,7 @@ import com.google.common.net.InetAddresses; import java.io.File; import java.util.ArrayList; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Random; @@ -44,6 +45,7 @@ import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -497,4 +499,40 @@ public void testGetListOfEntriesOfLedgerWithJustOneBookieInWriteQuorum() throws } bkc.close(); } + + @Test + public void testGetBookies() throws Exception { + String ledgersRootPath = "/ledgers"; + Assert.assertTrue("Cluster rootpath should have been created successfully " + ledgersRootPath, + (zkc.exists(ledgersRootPath, false) != null)); + String bookieCookiePath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseConf) + + "/" + BookKeeperConstants.COOKIE_NODE; + Assert.assertTrue("AvailableBookiesPath should have been created successfully " + bookieCookiePath, + (zkc.exists(bookieCookiePath, false) != null)); + + try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) { + Collection availableBookies = bkAdmin.getAvailableBookies(); + Assert.assertEquals(availableBookies.size(), bs.size()); + + for (int i = 0; i < bs.size(); i++) { + availableBookies.contains(bs.get(i).getLocalAddress()); + } + + BookieServer killedBookie = bs.get(1); + killBookieAndWaitForZK(1); + + Collection remainingBookies = bkAdmin.getAvailableBookies(); + Assert.assertFalse(remainingBookies.contains(killedBookie.getLocalAddress())); + + Collection allBookies = bkAdmin.getAllBookies(); + for (int i = 0; i < bs.size(); i++) { + remainingBookies.contains(bs.get(i).getLocalAddress()); + allBookies.contains(bs.get(i).getLocalAddress()); + } + + Assert.assertEquals(remainingBookies.size(), allBookies.size() - 1); + Assert.assertTrue(allBookies.contains(killedBookie.getLocalAddress())); + } + } + } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java index 8ce3686c20a..9ba5ae401b3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java @@ -38,6 +38,7 @@ public class MockRegistrationClient implements RegistrationClient { final ExecutorService executor; private long currentVersion = 0; private Set bookies = new HashSet(); + private Set allBookies = new HashSet(); private Set readOnlyBookies = new HashSet(); private Set bookieWatchers = new HashSet(); private Set readOnlyBookieWatchers = new HashSet(); @@ -114,6 +115,13 @@ public CompletableFuture>> getWritableBookies return promise; } + @Override + public CompletableFuture>> getAllBookies() { + CompletableFuture>> promise = new CompletableFuture<>(); + executor.submit(() -> promise.complete(versioned(allBookies, currentVersion))); + return promise; + } + @Override public CompletableFuture>> getReadOnlyBookies() { CompletableFuture>> promise = new CompletableFuture<>(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationClient.java index 1f5e0b68963..234a860364f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationClient.java @@ -23,6 +23,7 @@ import static org.apache.bookkeeper.common.testing.MoreAsserts.assertSetEquals; import static org.apache.bookkeeper.discover.ZKRegistrationClient.ZK_CONNECT_BACKOFF_MS; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; +import static org.apache.bookkeeper.util.BookKeeperConstants.COOKIE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -88,6 +89,7 @@ public class TestZkRegistrationClient extends MockZooKeeperTestCase { private String ledgersPath; private String regPath; + private String regAllPath; private String regReadonlyPath; private ZKRegistrationClient zkRegistrationClient; private ScheduledExecutorService mockExecutor; @@ -100,6 +102,7 @@ public void setup() throws Exception { this.ledgersPath = "/" + runtime.getMethodName(); this.regPath = ledgersPath + "/" + AVAILABLE_NODE; + this.regAllPath = ledgersPath + "/" + COOKIE_NODE; this.regReadonlyPath = regPath + "/" + READONLY; this.mockExecutor = mock(ScheduledExecutorService.class); this.controller = new MockExecutorController() @@ -150,6 +153,27 @@ public void testGetWritableBookies() throws Exception { addresses, result.getValue()); } + @Test + public void testGetAllBookies() throws Exception { + Set addresses = prepareNBookies(10); + List children = Lists.newArrayList(); + for (BookieSocketAddress address : addresses) { + children.add(address.toString()); + } + Stat stat = mock(Stat.class); + when(stat.getCversion()).thenReturn(1234); + mockGetChildren( + regAllPath, false, + Code.OK.intValue(), children, stat); + + Versioned> result = + result(zkRegistrationClient.getAllBookies()); + + assertEquals(new LongVersion(1234), result.getVersion()); + assertSetEquals( + addresses, result.getValue()); + } + @Test public void testGetReadOnlyBookies() throws Exception { Set addresses = prepareNBookies(10); @@ -185,6 +209,20 @@ public void testGetWritableBookiesFailure() throws Exception { } } + @Test + public void testGetAllBookiesFailure() throws Exception { + mockGetChildren( + regAllPath, false, + Code.NONODE.intValue(), null, null); + + try { + result(zkRegistrationClient.getAllBookies()); + fail("Should fail to get all bookies"); + } catch (ZKException zke) { + // expected to throw zookeeper exception + } + } + @Test public void testGetReadOnlyBookiesFailure() throws Exception { mockGetChildren( diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java index f1e77157ffb..76d731c736d 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java @@ -28,6 +28,7 @@ import java.util.function.Consumer; import java.util.function.Function; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.metadata.etcd.helpers.KeySetReader; @@ -91,6 +92,11 @@ public CompletableFuture>> getWritableBookies return writableBookiesReader.read(); } + @Override + public CompletableFuture>> getAllBookies() { + return FutureUtils.exception(new BKException.BKIllegalOpException()); + } + @Override public CompletableFuture>> getReadOnlyBookies() { return readonlyBookiesReader.read(); From 6ec805659adf88997516a1008f3af964b011cec2 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 12 Jul 2019 01:03:10 -0700 Subject: [PATCH 0313/1642] [BK-SERVER] Clean up over-replicated ledgers owned by different bookies ### Motivation As described at: https://github.com/apache/pulsar/issues/4632 - Sometimes due to overreplication, bookie contains ledgers which are not owned by that bookie anymore and that bookie is not part of the ensemble-list of those ledgers. In this case, GC finds out those overreplicated ledgers and - deletes their index from dbStorage (rocksDB) and - tries to delete them from entrylog files. However, bookie doesn't delete them from entry-log files due to change made in [#870](https://github.com/apache/bookkeeper/issues/870) where bookie avoids deleting ledger if znode of that ledger exists. Because of that bookie ends up storing large number entrylog files with ledgers which are owned by different bookies. It also cause OOM when GC tries to deal with large number of entry log files. ### Modification Delete the ledgers if bookie is not part of ensemble list of over-replicated ledgers Reviewers: Enrico Olivelli , Sijie Guo This closes #2119 from rdhabalia/overRepl --- .../ScanAndCompareGarbageCollector.java | 24 ++++++- .../apache/bookkeeper/meta/GcLedgersTest.java | 62 ++++++++++++++++++- 2 files changed, 82 insertions(+), 4 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index 02e47aec526..53685beb935 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -34,8 +34,10 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; @@ -45,6 +47,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.ZkUtils; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; @@ -149,6 +152,8 @@ public void gc(GarbageCleaner garbageCleaner) { long start; long end = -1; boolean done = false; + AtomicBoolean isBookieInEnsembles = new AtomicBoolean(false); + Versioned metadata = null; while (!done) { start = end + 1; if (ledgerRangeIterator.hasNext()) { @@ -169,9 +174,12 @@ public void gc(GarbageCleaner garbageCleaner) { for (Long bkLid : subBkActiveLedgers) { if (!ledgersInMetadata.contains(bkLid)) { if (verifyMetadataOnGc) { + isBookieInEnsembles.set(false); + metadata = null; int rc = BKException.Code.OK; try { - result(ledgerManager.readLedgerMetadata(bkLid), zkOpTimeoutMs, TimeUnit.MILLISECONDS); + metadata = result(ledgerManager.readLedgerMetadata(bkLid), zkOpTimeoutMs, + TimeUnit.MILLISECONDS); } catch (BKException | TimeoutException e) { if (e instanceof BKException) { rc = ((BKException) e).getCode(); @@ -182,7 +190,19 @@ public void gc(GarbageCleaner garbageCleaner) { continue; } } - if (rc != BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { + // check bookie should be part of ensembles in one + // of the segment else ledger should be deleted from + // local storage + if (metadata != null && metadata.getValue() != null) { + metadata.getValue().getAllEnsembles().forEach((entryId, ensembles) -> { + if (ensembles != null && ensembles.contains(selfBookieAddress)) { + isBookieInEnsembles.set(true); + } + }); + if (isBookieInEnsembles.get()) { + continue; + } + } else if (rc != BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { LOG.warn("Ledger {} Missing in metadata list, but ledgerManager returned rc: {}.", bkLid, rc); continue; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 30121b9182b..238877222f2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -52,6 +52,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.CheckpointSource; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; @@ -93,12 +94,18 @@ public GcLedgersTest(Class lmFactoryCls) { super(lmFactoryCls); } + private void createLedgers(int numLedgers, final Set createdLedgers) throws IOException { + BookieSocketAddress selfBookie = Bookie.getBookieAddress(baseConf); + createLedgers(numLedgers, createdLedgers, selfBookie); + } + /** * Create ledgers. */ - private void createLedgers(int numLedgers, final Set createdLedgers) throws IOException { + private void createLedgers(int numLedgers, final Set createdLedgers, BookieSocketAddress selfBookie) + throws IOException { final AtomicInteger expected = new AtomicInteger(numLedgers); - List ensemble = Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234)); + List ensemble = Lists.newArrayList(selfBookie); for (int i = 0; i < numLedgers; i++) { getLedgerIdGenerator().generateLedgerId(new GenericCallback() { @@ -692,4 +699,55 @@ public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { return null; } } + + /** + * Verifies that gc should cleaned up overreplicatd ledgers which is not + * owned by the bookie anymore. + * + * @throws Exception + */ + @Test + public void testGcLedgersForOverreplicated() throws Exception { + baseConf.setVerifyMetadataOnGc(true); + final SortedSet createdLedgers = Collections.synchronizedSortedSet(new TreeSet()); + final SortedSet cleaned = Collections.synchronizedSortedSet(new TreeSet()); + + // Create few ledgers + final int numLedgers = 5; + + BookieSocketAddress bookieAddress = new BookieSocketAddress("192.0.0.1", 1234); + createLedgers(numLedgers, createdLedgers, bookieAddress); + + LedgerManager mockLedgerManager = new CleanupLedgerManager(getLedgerManager()) { + @Override + public LedgerRangeIterator getLedgerRanges(long zkOpTimeout) { + return new LedgerRangeIterator() { + @Override + public LedgerRange next() throws IOException { + return null; + } + + @Override + public boolean hasNext() throws IOException { + return false; + } + }; + } + }; + + final GarbageCollector garbageCollector = new ScanAndCompareGarbageCollector(mockLedgerManager, + new MockLedgerStorage(), baseConf, NullStatsLogger.INSTANCE); + GarbageCollector.GarbageCleaner cleaner = new GarbageCollector.GarbageCleaner() { + @Override + public void clean(long ledgerId) { + LOG.info("Cleaned {}", ledgerId); + cleaned.add(ledgerId); + } + }; + + validateLedgerRangeIterator(createdLedgers); + + garbageCollector.gc(cleaner); + assertEquals("Should have cleaned all ledgers", cleaned.size(), numLedgers); + } } From a3904d89d67ed4205f6bb3a0f5e633f1e7cb5959 Mon Sep 17 00:00:00 2001 From: massakam Date: Fri, 12 Jul 2019 17:04:46 +0900 Subject: [PATCH 0314/1642] Issue #2103: Avoid stop of entry log compaction ### Motivation As mentioned in #2103, if an exception occurs during compaction of a specific entry log, `GarbageCollectorThread` does not perform compaction of other entry logs until the bookie server is restarted. As a result, the number of entry logs continues to increase and eventually it will run out of disk space. ### Changes The cause of the compaction stop is that the `compacting` flag remains true if `compactor.compact(entryLogMeta)` throws some exception. https://github.com/apache/bookkeeper/blob/b2e099bbc7b13f13825fe78ab009ca132cb3a9ba/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java#L504-L519 Therefore, fixed `GarbageCollectorThread` so that it set the compaction flag to false even if compaction of a specific entry log fails. Master Issue: #2103 Reviewers: Enrico Olivelli , Sijie Guo This closes #2121 from massakam/entry-log-compaction, closes #2103 --- .../bookie/GarbageCollectorThread.java | 14 +++- .../bookie/GarbageCollectorThreadTest.java | 81 +++++++++++++++++++ 2 files changed, 91 insertions(+), 4 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java index 006592646c0..811fbe6dd51 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java @@ -512,10 +512,16 @@ protected void compactEntryLog(EntryLogMetadata entryLogMeta) { // indicates that compaction is in progress for this EntryLogId. return; } - // Do the actual compaction - compactor.compact(entryLogMeta); - // Mark compaction done - compacting.set(false); + + try { + // Do the actual compaction + compactor.compact(entryLogMeta); + } catch (Exception e) { + LOG.error("Failed to compact entry log {} due to unexpected error", entryLogMeta.getEntryLogId(), e); + } finally { + // Mark compaction done + compacting.set(false); + } } /** diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java new file mode 100644 index 00000000000..1b1f6572567 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java @@ -0,0 +1,81 @@ +/* + * + * 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 org.junit.Assert.assertFalse; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.stats.StatsLogger; +import org.junit.Before; +import org.junit.Test; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.Spy; +import org.powermock.reflect.Whitebox; + +/** + * Unit test for {@link GarbageCollectorThread}. + */ +public class GarbageCollectorThreadTest { + + @InjectMocks + @Spy + private GarbageCollectorThread mockGCThread; + + @Mock + private LedgerManager ledgerManager; + @Mock + private StatsLogger statsLogger; + @Mock + private ScheduledExecutorService gcExecutor; + + private ServerConfiguration conf = spy(new ServerConfiguration()); + private CompactableLedgerStorage ledgerStorage = mock(CompactableLedgerStorage.class); + + @Before + public void setUp() throws Exception { + when(ledgerStorage.getEntryLogger()).thenReturn(mock(EntryLogger.class)); + initMocks(this); + } + + @Test + public void testCompactEntryLogWithException() throws Exception { + AbstractLogCompactor mockCompactor = mock(AbstractLogCompactor.class); + when(mockCompactor.compact(any(EntryLogMetadata.class))) + .thenThrow(new RuntimeException("Unexpected compaction error")); + Whitebox.setInternalState(mockGCThread, "compactor", mockCompactor); + + // Although compaction of an entry log fails due to an unexpected error, + // the `compacting` flag should return to false + AtomicBoolean compacting = Whitebox.getInternalState(mockGCThread, "compacting"); + assertFalse(compacting.get()); + mockGCThread.compactEntryLog(new EntryLogMetadata(9999)); + assertFalse(compacting.get()); + } +} From 39d127b4f2f76f67a480f220b611917a8328ecb9 Mon Sep 17 00:00:00 2001 From: Hugo Abreu <6619758+hugomiguelabreu@users.noreply.github.com> Date: Wed, 17 Jul 2019 08:41:39 +0100 Subject: [PATCH 0315/1642] Issue #1961: Included ZooKeeper client AclId. Descriptions of the changes in this PR: Included ZooKeeper Acl to the builder. ### Changes It is now possible to unbind DistributedLog namesapaces. Master Issue: #1961 Reviewers: Ivan Kelly , Sijie Guo This closes #2118 from hugomiguelabreu/master, closes #1961 --- .../org/apache/distributedlog/admin/DistributedLogAdmin.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java index 2ecd3385520..1776c3ef7ca 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java @@ -460,7 +460,7 @@ protected int runCmd(CommandLine cmdline) throws Exception { boolean force = cmdline.hasOption("f"); URI uri = URI.create(args[0]); // resolving the uri to see if there is another bindings in this uri. - ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri) + ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri).zkAclId(null) .sessionTimeoutMs(10000).build(); BKDLConfig bkdlConfig; try { From c6a7b5bd0ed3d3af3628367e44b7153c9f6e496e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 17 Jul 2019 00:46:05 -0700 Subject: [PATCH 0316/1642] Website update for 4.9.2 release Added 4.9.2 release to website Reviewers: Jia Zhai , Sijie Guo This closes #2098 from merlimat/master --- site/_config.yml | 3 +- site/docs/4.9.2/admin/autorecovery.md | 128 +++ site/docs/4.9.2/admin/bookies.md | 180 ++++ site/docs/4.9.2/admin/geo-replication.md | 22 + site/docs/4.9.2/admin/http.md | 457 ++++++++++ site/docs/4.9.2/admin/metrics.md | 41 + site/docs/4.9.2/admin/perf.md | 3 + site/docs/4.9.2/admin/placement.md | 3 + site/docs/4.9.2/admin/upgrade.md | 175 ++++ site/docs/4.9.2/api/distributedlog-api.md | 395 ++++++++ site/docs/4.9.2/api/ledger-adv-api.md | 111 +++ site/docs/4.9.2/api/ledger-api.md | 841 ++++++++++++++++++ site/docs/4.9.2/api/overview.md | 17 + site/docs/4.9.2/deployment/dcos.md | 142 +++ site/docs/4.9.2/deployment/kubernetes.md | 181 ++++ site/docs/4.9.2/deployment/manual.md | 56 ++ site/docs/4.9.2/development/codebase.md | 3 + site/docs/4.9.2/development/protocol.md | 148 +++ site/docs/4.9.2/getting-started/concepts.md | 202 +++++ .../4.9.2/getting-started/installation.md | 82 ++ .../docs/4.9.2/getting-started/run-locally.md | 16 + site/docs/4.9.2/overview/overview.md | 60 ++ site/docs/4.9.2/overview/releaseNotes.md | 26 +- .../4.9.2/overview/releaseNotesTemplate.md | 17 + site/docs/4.9.2/reference/cli.md | 10 + site/docs/4.9.2/reference/config.md | 9 + site/docs/4.9.2/reference/metrics.md | 3 + site/docs/4.9.2/security/overview.md | 21 + site/docs/4.9.2/security/sasl.md | 202 +++++ site/docs/4.9.2/security/tls.md | 210 +++++ site/docs/4.9.2/security/zookeeper.md | 41 + site/releases.md | 8 + 32 files changed, 3799 insertions(+), 14 deletions(-) create mode 100644 site/docs/4.9.2/admin/autorecovery.md create mode 100644 site/docs/4.9.2/admin/bookies.md create mode 100644 site/docs/4.9.2/admin/geo-replication.md create mode 100644 site/docs/4.9.2/admin/http.md create mode 100644 site/docs/4.9.2/admin/metrics.md create mode 100644 site/docs/4.9.2/admin/perf.md create mode 100644 site/docs/4.9.2/admin/placement.md create mode 100644 site/docs/4.9.2/admin/upgrade.md create mode 100644 site/docs/4.9.2/api/distributedlog-api.md create mode 100644 site/docs/4.9.2/api/ledger-adv-api.md create mode 100644 site/docs/4.9.2/api/ledger-api.md create mode 100644 site/docs/4.9.2/api/overview.md create mode 100644 site/docs/4.9.2/deployment/dcos.md create mode 100644 site/docs/4.9.2/deployment/kubernetes.md create mode 100644 site/docs/4.9.2/deployment/manual.md create mode 100644 site/docs/4.9.2/development/codebase.md create mode 100644 site/docs/4.9.2/development/protocol.md create mode 100644 site/docs/4.9.2/getting-started/concepts.md create mode 100644 site/docs/4.9.2/getting-started/installation.md create mode 100644 site/docs/4.9.2/getting-started/run-locally.md create mode 100644 site/docs/4.9.2/overview/overview.md create mode 100644 site/docs/4.9.2/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.9.2/reference/cli.md create mode 100644 site/docs/4.9.2/reference/config.md create mode 100644 site/docs/4.9.2/reference/metrics.md create mode 100644 site/docs/4.9.2/security/overview.md create mode 100644 site/docs/4.9.2/security/sasl.md create mode 100644 site/docs/4.9.2/security/tls.md create mode 100644 site/docs/4.9.2/security/zookeeper.md diff --git a/site/_config.yml b/site/_config.yml index 5097719c125..0b022156682 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -8,6 +8,7 @@ destination: local-generated twitter_url: https://twitter.com/asfbookkeeper versions: +- "4.9.2" - "4.9.1" # [next_version_placeholder] - "4.9.0" @@ -37,7 +38,7 @@ archived_versions: - "4.1.0" - "4.0.0" latest_version: "4.10.0-SNAPSHOT" -latest_release: "4.9.1" +latest_release: "4.9.2" stable_release: "4.8.2" distributedlog_version: "4.8.2" diff --git a/site/docs/4.9.2/admin/autorecovery.md b/site/docs/4.9.2/admin/autorecovery.md new file mode 100644 index 00000000000..b1dd078f9b2 --- /dev/null +++ b/site/docs/4.9.2/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: + +* the `shell recover` option +* 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 shell recover \ + 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 shell recover \ + 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.9.2/admin/bookies.md b/site/docs/4.9.2/admin/bookies.md new file mode 100644 index 00000000000..1b0427dae3c --- /dev/null +++ b/site/docs/4.9.2/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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + \ + \ + + ``` + + 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 + $ bookkeeper-server/bin/bookkeeper shell recover \ + 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.9.2/admin/geo-replication.md b/site/docs/4.9.2/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.9.2/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.9.2/admin/http.md b/site/docs/4.9.2/admin/http.md new file mode 100644 index 00000000000..270dcb3a16f --- /dev/null +++ b/site/docs/4.9.2/admin/http.md @@ -0,0 +1,457 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 4 components: +1. Heartbeat: heartbeat for a specific bookie. +1. Config: doing the server configuration for a specific bookie. +1. Ledger: HTTP endpoints related to ledgers. +1. Bookie: HTTP endpoints related to bookies. +1. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Config + +### Endpoint: /metrics +1. Method: GET + * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/bookie/gc +1. Method: PUT + * Description: trigger gc for this bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: GET + * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + { + "is_in_force_gc" : "false" + } + ``` + +### Endpoint: /api/v1/bookie/gc_details +1. Method: GET + * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + [ { + "forceCompacting" : false, + "majorCompacting" : false, + "minorCompacting" : false, + "lastMajorCompactionTime" : 1544578144944, + "lastMinorCompactionTime" : 1544578144944, + "majorCompactionCounter" : 1, + "minorCompactionCounter" : 0 + } ] + ``` + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.9.2/admin/metrics.md b/site/docs/4.9.2/admin/metrics.md new file mode 100644 index 00000000000..142df3dcd2d --- /dev/null +++ b/site/docs/4.9.2/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.prometheus.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.9.2/admin/perf.md b/site/docs/4.9.2/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.9.2/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.9.2/admin/placement.md b/site/docs/4.9.2/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.9.2/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.9.2/admin/upgrade.md b/site/docs/4.9.2/admin/upgrade.md new file mode 100644 index 00000000000..57c65208131 --- /dev/null +++ b/site/docs/4.9.2/admin/upgrade.md @@ -0,0 +1,175 @@ +--- +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.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.7.x to 4.8.X upgrade + +In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.9.2/api/distributedlog-api.md b/site/docs/4.9.2/api/distributedlog-api.md new file mode 100644 index 00000000000..051ad9c84de --- /dev/null +++ b/site/docs/4.9.2/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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.9.2/api/ledger-adv-api.md b/site/docs/4.9.2/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.9.2/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.9.2/api/ledger-api.md b/site/docs/4.9.2/api/ledger-api.md new file mode 100644 index 00000000000..c0817e93d1f --- /dev/null +++ b/site/docs/4.9.2/api/ledger-api.md @@ -0,0 +1,841 @@ +--- +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.9.2 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.9.2 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.9.2' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.9.2' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` + +### Relaxing Durability + +In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. +In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will +be always readable by other clients. + +On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after +writing the entry to SO buffers without waiting for an fsync. +In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. +Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. + +In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble ackknowledge the call after +performing an fsync to the disk which is storing the journal. +This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. + +The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. +In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. + + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete + + +wh.force().get(); // wait for fsync, make data available to readers and to the replicator + +wh.close(); // seal the ledger + +``` diff --git a/site/docs/4.9.2/api/overview.md b/site/docs/4.9.2/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.9.2/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.9.2/deployment/dcos.md b/site/docs/4.9.2/deployment/dcos.md new file mode 100644 index 00000000000..c8df0167c73 --- /dev/null +++ b/site/docs/4.9.2/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.9.2] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.9.2/deployment/kubernetes.md b/site/docs/4.9.2/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.9.2/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.9.2/deployment/manual.md b/site/docs/4.9.2/deployment/manual.md new file mode 100644 index 00000000000..daafd5556f5 --- /dev/null +++ b/site/docs/4.9.2/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.9.2/development/codebase.md b/site/docs/4.9.2/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.9.2/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.9.2/development/protocol.md b/site/docs/4.9.2/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.9.2/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.9.2/getting-started/concepts.md b/site/docs/4.9.2/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.9.2/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.9.2/getting-started/installation.md b/site/docs/4.9.2/getting-started/installation.md new file mode 100644 index 00000000000..7e92b471581 --- /dev/null +++ b/site/docs/4.9.2/getting-started/installation.md @@ -0,0 +1,82 @@ +--- +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 +``` + +Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. + +```shell +$ mvn package -Dstream +``` + +> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin + +> You can enable `table service` by adding the `-Dstream` flag when running above commands. + +## 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.9.2/getting-started/run-locally.md b/site/docs/4.9.2/getting-started/run-locally.md new file mode 100644 index 00000000000..edbfab9fda6 --- /dev/null +++ b/site/docs/4.9.2/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 +$ bookkeeper-server/bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.9.2/overview/overview.md b/site/docs/4.9.2/overview/overview.md new file mode 100644 index 00000000000..b2945664f3f --- /dev/null +++ b/site/docs/4.9.2/overview/overview.md @@ -0,0 +1,60 @@ +--- +title: Apache BookKeeper™ 4.9.2 +--- + + +This documentation is for Apache BookKeeper™ version 4.9.2. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.9.2 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.9.2/overview/releaseNotes.md b/site/docs/4.9.2/overview/releaseNotes.md index ea01f4bf6a3..52b8e5608da 100644 --- a/site/docs/4.9.2/overview/releaseNotes.md +++ b/site/docs/4.9.2/overview/releaseNotes.md @@ -2,23 +2,23 @@ title: Apache BookKeeper 4.9.2 Release Notes --- -The 4.9.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.9.1. +This is the 18th release of Apache BookKeeper! -Apache BookKeeper users who are using 4.9.1 are encouraged to upgrade to 4.9.2. The technical details of this release are summarized -below. +The 4.9.2 release incorporates a few critical bug fixes, since previous major release, 4.9.0. -## Highlights +Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.9.2](../../admin/upgrade). The technical details of +this release are summarized below. - - Added HTTP handler to expose bookie state [apache/bookkeeper#1995](https://github.com/apache/bookkeeper/pull/1995) - - Fix DbLedgerStorage encountering unexpected entry id [apache/bookkeeper#2002](https://github.com/apache/bookkeeper/pull/2002) - - Close db properly to avoid open RocksDB failure at the second time [apache/bookkeeper#2022](https://github.com/apache/bookkeeper/pull/2022) - - Cancel LAC watch when longpoll LAC times out [apache/bookkeeper#2051](https://github.com/apache/bookkeeper/pull/2051) - - Wait for LAC update even if ledger fenced [apache/bookkeeper#2052](https://github.com/apache/bookkeeper/pull/2052) - - Bugfix for Percentile Calculation in FastCodahale Timer Implementation [apache/bookkeeper#2054](https://github.com/apache/bookkeeper/pull/2054) - - Use pure python implementation of MurmurHash [apache/bookkeeper#2069](https://github.com/apache/bookkeeper/pull/2069) - - Bookieshell lastmark command isn't functional, always returning 0-0 [apache/bookkeeper#2076](https://github.com/apache/bookkeeper/pull/2076) +### Dependencies Changes +No dependency change. + +### Bug Fixes + +- [Issue #1973: [DLOG] Avoid double read in readahead](https://github.com/apache/bookkeeper/pull/1973) +- [Issue #1952: Filter empty string for networkTopologyScriptFileName](https://github.com/apache/bookkeeper/pull/1952) +- [Issue #1950: putEntryOffset translate FileInfoDeletedException](https://github.com/apache/bookkeeper/pull/1950) ## Full list of changes -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.2+is%3Aclosed) +- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.1+is%3Aclosed) diff --git a/site/docs/4.9.2/overview/releaseNotesTemplate.md b/site/docs/4.9.2/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..1e002dbf983 --- /dev/null +++ b/site/docs/4.9.2/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.9.2 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.9.2. The technical details of this release are summarized +below. + +## Highlights + +[List the highlights] + +## Details + +[list to issues list] + diff --git a/site/docs/4.9.2/reference/cli.md b/site/docs/4.9.2/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.9.2/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.9.2/reference/config.md b/site/docs/4.9.2/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.9.2/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.9.2/reference/metrics.md b/site/docs/4.9.2/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.9.2/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.9.2/security/overview.md b/site/docs/4.9.2/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.9.2/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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.9.2/security/sasl.md b/site/docs/4.9.2/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.9.2/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.9.2/security/tls.md b/site/docs/4.9.2/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.9.2/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.9.2/security/zookeeper.md b/site/docs/4.9.2/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.9.2/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) diff --git a/site/releases.md b/site/releases.md index 1f25b2a08b4..d8701190dbe 100644 --- a/site/releases.md +++ b/site/releases.md @@ -64,6 +64,14 @@ Client Guide | API docs ## News +### [date] Release {{ site.latest_version }} available + +This is the 19th release of Apache BookKeeper ! + +The 4.9.2 release is a bugfix release which fixes a couple of issues reported from users of 4.9.1. + +See [BookKeeper 4.9.2 Release Notes](../docs/4.9.2/overview/releaseNotes) for details. + ### 7 April, 2019 Release 4.9.1 available This is the 18th release of Apache BookKeeper ! From 01daa8a7702e946fa6b143d92a2ef2bf1901ac99 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Fri, 19 Jul 2019 16:48:40 +0100 Subject: [PATCH 0317/1642] [CI] Prune networks older than 12h Previously we were using a filter which matched on the network name of the networks we create. However, docker seems to have removed this functionality, so now we just prune all unused networks which are more than 12h old (no CI job should be taking more than 12h). Reviewers: Enrico Olivelli This closes #2126 from ivankelly/ci-failure --- .test-infra/scripts/pre-docker-tests.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.test-infra/scripts/pre-docker-tests.sh b/.test-infra/scripts/pre-docker-tests.sh index 65fac8a5561..d2c812ab7a3 100755 --- a/.test-infra/scripts/pre-docker-tests.sh +++ b/.test-infra/scripts/pre-docker-tests.sh @@ -28,6 +28,6 @@ ps -eo euser,pid,ppid,pgid,start,pcpu,pmem,cmd docker info docker system prune -f # clean up any dangling networks from previous runs -docker network prune -f --filter name=testnetwork_* +docker network prune -f --filter "until=12h" docker system events > docker.debug-info & echo $! > docker-log.pid docker pull quay.io/coreos/etcd:v3.3 From 933b25ab8bf0e5dac42642b393456820608d92cb Mon Sep 17 00:00:00 2001 From: vzhikserg Date: Sat, 20 Jul 2019 03:14:26 +0200 Subject: [PATCH 0318/1642] Fix log statements with incorrect placeholders ### Changes The following changes were made: - remove unused placeholders in the log statements - add missing placeholders to the log strings to show the complete information - remove unnecessary initialization of object arrays in the log statements Reviewers: Sijie Guo This closes #2123 from vzhikserg/fix-log-statements --- .../apache/bookkeeper/bookie/EntryLogger.java | 2 +- .../bookkeeper/bookie/IndexPersistenceMgr.java | 3 +-- .../bookie/ScanAndCompareGarbageCollector.java | 2 +- .../bookie/TransactionalEntryLogCompactor.java | 2 +- .../bookkeeper/client/PendingWriteLacOp.java | 2 +- .../meta/AbstractZkLedgerManager.java | 4 ++-- .../bookkeeper/net/BookieSocketAddress.java | 4 +--- .../client/TestLedgerFragmentReplication.java | 6 +++--- .../replication/TestReplicationWorker.java | 18 +++++++++--------- .../config/ConfigurationSubscription.java | 2 +- .../distributedlog/LocalDLMEmulator.java | 2 +- .../bk/SimpleLedgerAllocator.java | 6 +++--- .../impl/metadata/BKDLConfig.java | 6 +++--- .../apache/distributedlog/net/DNSResolver.java | 2 +- .../TestBKDistributedLogNamespace.java | 14 +++++++------- .../distributedlog/TestLogSegmentCreation.java | 2 +- .../org/apache/distributedlog/TestReader.java | 2 +- .../distributedlog/TestRollLogSegments.java | 2 +- .../bk/TestLedgerAllocatorPool.java | 3 +-- .../stream/server/service/BookieService.java | 12 +++++------- 20 files changed, 45 insertions(+), 51 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index 55ac6111b34..6662d594d27 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -637,7 +637,7 @@ void flushCompactionLog() throws IOException { if (compactionLogChannel != null) { compactionLogChannel.appendLedgersMap(); compactionLogChannel.flushAndForceWrite(false); - LOG.info("Flushed compaction log file {} with logId.", + LOG.info("Flushed compaction log file {} with logId {}.", compactionLogChannel.getLogFile(), compactionLogChannel.getLogId()); // since this channel is only used for writing, after flushing the channel, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java index b728df38454..898e1460e05 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java @@ -451,7 +451,6 @@ void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException { try { fi = getFileInfo(ledgerId, null); fi.setExplicitLac(lac); - return; } finally { if (null != fi) { fi.release(); @@ -465,7 +464,7 @@ public ByteBuf getExplicitLac(long ledgerId) { fi = getFileInfo(ledgerId, null); return fi.getExplicitLac(); } catch (IOException e) { - LOG.error("Exception during getLastAddConfirmed: {}", e); + LOG.error("Exception during getLastAddConfirmed", e); return null; } finally { if (null != fi) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index 53685beb935..21f412998a0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -214,7 +214,7 @@ public void gc(GarbageCleaner garbageCleaner) { } } catch (Throwable t) { // ignore exception, collecting garbage next time - LOG.warn("Exception when iterating over the metadata {}", t); + LOG.warn("Exception when iterating over the metadata", t); } finally { if (zk != null) { try { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java index 74a30a63a56..aec934fa159 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java @@ -96,7 +96,7 @@ public void cleanUpAndRecover() { public boolean compact(EntryLogMetadata metadata) { if (metadata != null) { LOG.info("Compacting entry log {} with usage {}.", - new Object[]{metadata.getEntryLogId(), metadata.getUsage()}); + metadata.getEntryLogId(), metadata.getUsage()); CompactionPhase scanEntryLog = new ScanEntryLogPhase(metadata); if (!scanEntryLog.run()) { LOG.info("Compaction for entry log {} end in ScanEntryLogPhase.", metadata.getEntryLogId()); 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 2881d2f3ad5..e9eb3f23b38 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 @@ -112,7 +112,7 @@ public void writeLacComplete(int rc, long ledgerId, BookieSocketAddress addr, Ob return; } } else { - LOG.warn("WriteLac did not succeed: Ledger {} on {}", new Object[] { ledgerId, addr }); + LOG.warn("WriteLac did not succeed: Ledger {} on {}", ledgerId, addr); } if (receivedResponseSet.isEmpty()){ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 79a4f50c3cc..cd162626f30 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -290,7 +290,7 @@ public void processResult(int rc, String path, Object ctx, String name) { // and then it got removed. I'd argue that returning an error here is the right // path since recreating it is likely to cause problems. LOG.warn("Ledger {} appears to have already existed and then been removed, failing" - + " with LedgerExistException"); + + " with LedgerExistException", ledgerId); promise.completeExceptionally(new BKException.BKLedgerExistException()); } else { LOG.error("Could not validate node for ledger {} after LedgerExistsException", ledgerId, @@ -355,7 +355,7 @@ public void processResult(int rc, String path, Object ctx) { if (LOG.isDebugEnabled()) { LOG.debug( "Remove registered ledger metadata listeners on ledger {} after ledger is deleted.", - ledgerId, listenerSet); + ledgerId); } } else { if (LOG.isDebugEnabled()) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java index e9fc0f57e13..22f026a2ec8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java @@ -117,9 +117,7 @@ public LocalAddress getLocalAddress() { // Return the String "serialized" version of this object. @Override public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(hostname).append(COLON).append(port); - return sb.toString(); + return hostname + COLON + port; } // Implement an equals method comparing two BookiSocketAddress objects. 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 68580edd9bb..33b966bc478 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 @@ -94,7 +94,7 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles() .get(0L).get(0); - LOG.info("Killing Bookie", replicaToKill); + LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); @@ -149,7 +149,7 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() .get(0L).get(0); startNewBookie(); - LOG.info("Killing Bookie", replicaToKill); + LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); // Lets reform ensemble @@ -163,7 +163,7 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); LOG.info("New Bookie addr : {}", newBkAddr); - LOG.info("Killing Bookie", replicaToKill2); + LOG.info("Killing Bookie : {}", replicaToKill2); killBookie(replicaToKill2); Set result = getFragmentsToReplicate(lh); 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 9df734646f0..c155bb456b2 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 @@ -140,7 +140,7 @@ public void testRWShouldReplicateFragmentsToTargetBookie() throws Exception { } BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); - LOG.info("Killing Bookie", replicaToKill); + LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); @@ -187,7 +187,7 @@ public void testRWShouldRetryUntilThereAreEnoughBksAvailableForReplication() } lh.close(); BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); - LOG.info("Killing Bookie", replicaToKill); + LOG.info("Killing Bookie : {}", replicaToKill); ServerConfiguration killedBookieConfig = killBookie(replicaToKill); BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); @@ -236,7 +236,7 @@ public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplicatio } lh.close(); BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); - LOG.info("Killing Bookie", replicaToKill); + LOG.info("Killing Bookie : {}", replicaToKill); ServerConfiguration killedBookieConfig = killBookie(replicaToKill); killAllBookies(lh, null); @@ -292,7 +292,7 @@ public void testRWShouldCleanTheLedgerFromUnderReplicationIfLedgerAlreadyDeleted } lh.close(); BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); - LOG.info("Killing Bookie", replicaToKill); + LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); @@ -327,7 +327,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() } BookieSocketAddress replicaToKillFromFirstLedger = lh1.getLedgerMetadata().getAllEnsembles().get(0L).get(0); - LOG.info("Killing Bookie", replicaToKillFromFirstLedger); + LOG.info("Killing Bookie : {}", replicaToKillFromFirstLedger); // Ledger2 LedgerHandle lh2 = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, @@ -338,7 +338,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() } BookieSocketAddress replicaToKillFromSecondLedger = lh2.getLedgerMetadata().getAllEnsembles().get(0L).get(0); - LOG.info("Killing Bookie", replicaToKillFromSecondLedger); + LOG.info("Killing Bookie : {}", replicaToKillFromSecondLedger); // Kill ledger1 killBookie(replicaToKillFromFirstLedger); @@ -397,7 +397,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR() } BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); - LOG.info("Killing Bookie", replicaToKill); + LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); @@ -464,7 +464,7 @@ public void testBookiesNotAvailableScenarioForReplicationWorker() throws Excepti for (int i = 0; i < ensembleSize; i++) { bookiesKilled[i] = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(i); killedBookiesConfig[i] = getBkConf(bookiesKilled[i]); - LOG.info("Killing Bookie", bookiesKilled[i]); + LOG.info("Killing Bookie : {}", bookiesKilled[i]); killBookie(bookiesKilled[i]); } @@ -583,7 +583,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR() } BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); - LOG.info("Killing Bookie", replicaToKill); + LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java index b6f3be55317..6e4bb6dc5fd 100644 --- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java +++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java @@ -99,7 +99,7 @@ private boolean initConfig() { } } catch (ConfigurationException ex) { if (!fileNotFound(ex)) { - LOG.error("Config init failed {}", ex); + LOG.error("Config init failed", ex); } } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java index 61bc23105cf..ec35a168b00 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java @@ -145,7 +145,7 @@ public void run() { LOG.info("Starting {} bookies : allowLoopback = {}", numBookies, serverConf.getAllowLoopback()); LocalBookKeeper.startLocalBookies(zkHost, zkPort, numBookies, shouldStartZK, initialBookiePort, serverConf); - LOG.info("{} bookies are started."); + LOG.info("{} bookies are started.", numBookies); } catch (InterruptedException e) { Thread.currentThread().interrupt(); // go away quietly diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java index eb03923a107..015a9617273 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java @@ -314,7 +314,7 @@ public void onAbort(Throwable t) { private synchronized void setPhase(Phase phase) { this.phase = phase; LOG.info("Ledger allocator {} moved to phase {} : version = {}.", - new Object[] { allocatePath, phase, version }); + allocatePath, phase, version); } private synchronized void allocateLedger() { @@ -371,11 +371,11 @@ private synchronized void setVersion(LongVersion newVersion) { Version.Occurred occurred = newVersion.compare(version); if (occurred == Version.Occurred.AFTER) { LOG.info("Ledger allocator for {} moved version from {} to {}.", - new Object[] { allocatePath, version, newVersion }); + allocatePath, version, newVersion); version = newVersion; } else { LOG.warn("Ledger allocator for {} received an old version {}, current version is {}.", - new Object[] { allocatePath, newVersion , version }); + allocatePath, newVersion, version); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java index 58cacae3e57..ac53fd5e6f6 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java @@ -60,9 +60,9 @@ public static void propagateConfiguration(BKDLConfig bkdlConfig, DistributedLogC } LOG.info("Propagate BKDLConfig to DLConfig : encodeRegionID = {}," + " firstLogSegmentSequenceNumber = {}, createStreamIfNotExists = {}, isFederated = {}.", - new Object[] { dlConf.getEncodeRegionIDInLogSegmentMetadata(), - dlConf.getFirstLogSegmentSequenceNumber(), dlConf.getCreateStreamIfNotExists(), - bkdlConfig.isFederatedNamespace() }); + dlConf.getEncodeRegionIDInLogSegmentMetadata(), + dlConf.getFirstLogSegmentSequenceNumber(), dlConf.getCreateStreamIfNotExists(), + bkdlConfig.isFederatedNamespace()); } public static BKDLConfig resolveDLConfig(ZooKeeperClient zkc, URI uri) throws IOException { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/net/DNSResolver.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/net/DNSResolver.java index 7170cf99699..3ce4fa4697a 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/net/DNSResolver.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/net/DNSResolver.java @@ -66,7 +66,7 @@ public DNSResolver(String hostRegionOverrides) { for (String override : overrides) { String[] parts = override.split(":"); if (parts.length != 2) { - LOG.warn("Incorrect override specified", override); + LOG.warn("Incorrect override specified : {}", override); } else { hostNameToRegion.putIfAbsent(parts[0], parts[1]); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java index 5b9575bbe34..b20d51ea13c 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java @@ -183,7 +183,7 @@ public void testInvalidStreamName() throws Exception { try { namespace.openLog("/ test2"); - fail("should fail to create invalid stream / test2"); + fail("Should fail to create invalid stream / test2"); } catch (InvalidStreamNameException isne) { // expected } @@ -196,7 +196,7 @@ public void testInvalidStreamName() throws Exception { chars[0] = 0; String streamName = new String(chars); namespace.openLog(streamName); - fail("should fail to create invalid stream " + streamName); + fail("Should fail to create invalid stream " + streamName); } catch (InvalidStreamNameException isne) { // expected } @@ -209,7 +209,7 @@ public void testInvalidStreamName() throws Exception { chars[3] = '\u0010'; String streamName = new String(chars); namespace.openLog(streamName); - fail("should fail to create invalid stream " + streamName); + fail("Should fail to create invalid stream " + streamName); } catch (InvalidStreamNameException isne) { // expected } @@ -344,13 +344,13 @@ public void testAclModifyPermsDlmConflict() throws Exception { try { // Reopening and writing again with a different un will fail. initDlogMeta("/" + runtime.getMethodName(), "not-test-un", streamName); - fail("write should have failed due to perms"); + fail("Write should have failed due to perms"); } catch (ZKException ex) { - LOG.info("caught exception trying to write with no perms {}", ex); + LOG.info("Caught exception trying to write with no perms", ex); assertEquals(KeeperException.Code.NOAUTH, ex.getKeeperExceptionCode()); } catch (Exception ex) { - LOG.info("caught wrong exception trying to write with no perms {}", ex); - fail("wrong exception " + ex.getClass().getName() + " expected " + LockingException.class.getName()); + LOG.info("Caught wrong exception trying to write with no perms", ex); + fail("Wrong exception " + ex.getClass().getName() + " expected " + LockingException.class.getName()); } // Should work again. diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java index cd34e9585fc..90565c9b1b2 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java @@ -103,7 +103,7 @@ public void testCreateLogSegmentAfterLoseLock() throws Exception { assertFalse(hasDuplicatedSegment); LOG.info("Segments : duplicated = {}, inprogress = {}, {}", - new Object[] { hasDuplicatedSegment, hasInprogress, segments }); + hasDuplicatedSegment, hasInprogress, segments); dlm1.close(); dlm2.close(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java index be86b2cdbec..d349ddad156 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java @@ -152,7 +152,7 @@ public void onSuccess(LogRecordWithDLSN value) { try { assertTrue(value.getDlsn().compareTo(nextDLSN) >= 0); LOG.info("Received record {} from log {} for reader {}", - new Object[] { value.getDlsn(), dlm.getStreamName(), readerName }); + value.getDlsn(), dlm.getStreamName(), readerName); assertFalse(value.isControl()); assertEquals(0, value.getDlsn().getSlotId()); DLMTestUtil.verifyLargeLogRecord(value); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java index a3a3f3467fb..99b7424b9bd 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java @@ -279,7 +279,7 @@ public void onSuccess(DLSN value) { } @Override public void onFailure(Throwable cause) { - logger.error("Failed to write entries : {}", cause); + logger.error("Failed to write entries", cause); } }); if (i == 1) { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java index 2286d403ebf..3d258b7d296 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java @@ -289,8 +289,7 @@ public void run() { Utils.ioResult(txn.execute()); lh.close(); allocatedLedgers.putIfAbsent(lh.getId(), lh); - logger.info("[thread {}] allocate {}th ledger {}", - new Object[] { tid, i, lh.getId() }); + logger.info("[thread {}] allocate {}th ledger {}", tid, i, lh.getId()); } } catch (Exception ioe) { numFailures.incrementAndGet(); diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java index 393af62e04d..6f83752ce60 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java @@ -55,13 +55,11 @@ protected void doStart() { } log.info("Hello, I'm your bookie, listening on port {} :" + " metadata service uri = {}, journals = {}, ledgers = {}, index = {}", - new Object[]{ - serverConf.getBookiePort(), - serverConf.getMetadataServiceUriUnchecked(), - Arrays.asList(serverConf.getJournalDirNames()), - Arrays.asList(serverConf.getLedgerDirs()), - indexDirs - }); + serverConf.getBookiePort(), + serverConf.getMetadataServiceUriUnchecked(), + Arrays.asList(serverConf.getJournalDirNames()), + Arrays.asList(serverConf.getLedgerDirs()), + indexDirs); try { this.bs = new BookieServer(serverConf, statsLogger); bs.start(); From 9666e085684aa86b9a1565b5a9fe39b17771d9c7 Mon Sep 17 00:00:00 2001 From: vzhikserg Date: Sat, 20 Jul 2019 03:15:30 +0200 Subject: [PATCH 0319/1642] Use bulk operation instead of iteration ### Changes * Use the `addAll` method instead of iteration Reviewers: Sijie Guo This closes #2122 from vzhikserg/bulk-call-instead-of-iteration --- .../bookkeeper/meta/MetadataDrivers.java | 9 +++---- .../bookkeeper/bookie/CompactionTest.java | 4 +--- .../discover/MockRegistrationClient.java | 17 ++++--------- .../bookkeeper/proto/MockBookieClient.java | 5 ++-- .../distributedlog/ReadAheadEntryReader.java | 8 ++----- .../bk/LedgerAllocatorPool.java | 24 +++++-------------- .../namespace/NamespaceDriverManager.java | 5 ++-- 7 files changed, 21 insertions(+), 51 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java index 44c25df5b53..cf5daf18327 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java @@ -25,6 +25,7 @@ import com.google.common.collect.Sets; import com.google.common.util.concurrent.UncheckedExecutionException; import java.net.URI; +import java.util.Collections; import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -123,9 +124,7 @@ private static void loadInitialClientDrivers() { String driversStr = System.getProperty(BK_METADATA_CLIENT_DRIVERS_PROPERTY); if (null != driversStr) { String[] driversArray = StringUtils.split(driversStr, ':'); - for (String driver : driversArray) { - driverList.add(driver); - } + Collections.addAll(driverList, driversArray); } // initialize the drivers @@ -152,9 +151,7 @@ private static void loadInitialBookieDrivers() { String driversStr = System.getProperty(BK_METADATA_BOOKIE_DRIVERS_PROPERTY); if (null != driversStr) { String[] driversArray = StringUtils.split(driversStr, ':'); - for (String driver : driversArray) { - driverList.add(driver); - } + Collections.addAll(driverList, driversArray); } // initialize the drivers diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java index adef116475c..8fd9c3acf2f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java @@ -1380,9 +1380,7 @@ private Set findCompactedEntryLogFiles() { File[] files = Bookie.getCurrentDirectory(ledgerDirectory).listFiles( file -> file.getName().endsWith(COMPACTED_SUFFIX)); if (files != null) { - for (File file : files) { - compactedLogFiles.add(file); - } + Collections.addAll(compactedLogFiles, files); } } return compactedLogFiles; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java index 9ba5ae401b3..2004a0ac209 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java @@ -18,6 +18,7 @@ package org.apache.bookkeeper.discover; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -60,9 +61,7 @@ public CompletableFuture addBookies(BookieSocketAddress... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; - for (BookieSocketAddress b : bookies) { - this.bookies.add(b); - } + Collections.addAll(this.bookies, bookies); bookieWatchers.forEach(w -> w.onBookiesChanged(versioned(this.bookies, currentVersion))); promise.complete(null); }); @@ -73,9 +72,7 @@ public CompletableFuture removeBookies(BookieSocketAddress... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; - for (BookieSocketAddress b : bookies) { - this.bookies.add(b); - } + this.bookies.addAll(Arrays.asList(bookies)); bookieWatchers.forEach(w -> w.onBookiesChanged(versioned(this.bookies, currentVersion))); promise.complete(null); }); @@ -86,9 +83,7 @@ public CompletableFuture addReadOnlyBookies(BookieSocketAddress... bookies CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; - for (BookieSocketAddress b : bookies) { - this.readOnlyBookies.add(b); - } + this.readOnlyBookies.addAll(Arrays.asList(bookies)); readOnlyBookieWatchers.forEach(w -> w.onBookiesChanged(versioned(readOnlyBookies, currentVersion))); promise.complete(null); }); @@ -99,9 +94,7 @@ public CompletableFuture removeReadOnlyBookies(BookieSocketAddress... book CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; - for (BookieSocketAddress b : bookies) { - this.readOnlyBookies.add(b); - } + this.readOnlyBookies.addAll(Arrays.asList(bookies)); readOnlyBookieWatchers.forEach(w -> w.onBookiesChanged(versioned(readOnlyBookies, currentVersion))); promise.complete(null); }); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java index 1cbddd3abb9..1178a99e98c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java @@ -26,6 +26,7 @@ import io.netty.buffer.Unpooled; import io.netty.buffer.UnpooledByteBufAllocator; +import java.util.Arrays; import java.util.Collections; import java.util.EnumSet; import java.util.List; @@ -100,9 +101,7 @@ public void setPostWriteHook(Hook hook) { } public void errorBookies(BookieSocketAddress... bookies) { - for (BookieSocketAddress b : bookies) { - errorBookies.add(b); - } + errorBookies.addAll(Arrays.asList(bookies)); } public void removeErrors(BookieSocketAddress... bookies) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java index ccd42ebaec4..424615aa309 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java @@ -482,9 +482,7 @@ private void unsafeAsyncClose(CompletableFuture closePromise) { if (null != nextSegmentReader) { segmentReadersToClose.add(nextSegmentReader); } - for (SegmentReader reader : segmentReaders) { - segmentReadersToClose.add(reader); - } + segmentReadersToClose.addAll(segmentReaders); segmentReaders.clear(); for (SegmentReader reader : segmentReadersToClose) { closeFutures.add(reader.close()); @@ -586,9 +584,7 @@ public void onSuccess(List entries) { } lastEntryAddedTime.reset().start(); - for (Entry.Reader entry : entries) { - entryQueue.add(entry); - } + entryQueue.addAll(entries); if (!entries.isEmpty()) { Entry.Reader lastEntry = entries.get(entries.size() - 1); nextEntryPosition.advance(lastEntry.getLSSN(), lastEntry.getEntryId() + 1); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java index 2f654bb6b42..048ce9f3d9a 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java @@ -407,15 +407,9 @@ public CompletableFuture asyncClose() { synchronized (this) { allocatorsToClose = Lists.newArrayListWithExpectedSize( pendingList.size() + allocatingList.size() + obtainMap.size()); - for (LedgerAllocator allocator : pendingList) { - allocatorsToClose.add(allocator); - } - for (LedgerAllocator allocator : allocatingList) { - allocatorsToClose.add(allocator); - } - for (LedgerAllocator allocator : obtainMap.values()) { - allocatorsToClose.add(allocator); - } + allocatorsToClose.addAll(pendingList); + allocatorsToClose.addAll(allocatingList); + allocatorsToClose.addAll(obtainMap.values()); } return FutureUtils.processList( allocatorsToClose, @@ -430,15 +424,9 @@ public CompletableFuture delete() { synchronized (this) { allocatorsToDelete = Lists.newArrayListWithExpectedSize( pendingList.size() + allocatingList.size() + obtainMap.size()); - for (LedgerAllocator allocator : pendingList) { - allocatorsToDelete.add(allocator); - } - for (LedgerAllocator allocator : allocatingList) { - allocatorsToDelete.add(allocator); - } - for (LedgerAllocator allocator : obtainMap.values()) { - allocatorsToDelete.add(allocator); - } + allocatorsToDelete.addAll(pendingList); + allocatorsToDelete.addAll(allocatingList); + allocatorsToDelete.addAll(obtainMap.values()); } return FutureUtils.processList( allocatorsToDelete, diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java index 67b54a93fd6..9446cd6b1df 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java @@ -23,6 +23,7 @@ import com.google.common.base.Objects; import com.google.common.collect.Sets; import java.net.URI; +import java.util.Arrays; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -87,9 +88,7 @@ private static void loadInitialDrivers() { String driversStr = System.getProperty("distributedlog.namespace.drivers"); if (null != driversStr) { String[] driversArray = StringUtils.split(driversStr, ':'); - for (String driver : driversArray) { - driverList.add(driver); - } + driverList.addAll(Arrays.asList(driversArray)); } // initialize the drivers for (String driverClsName : driverList) { From 55954d610f899c970af86526c97a27b726c993ba Mon Sep 17 00:00:00 2001 From: Matteo Minardi Date: Sun, 21 Jul 2019 08:14:52 +0200 Subject: [PATCH 0320/1642] Issue #2124: Test 'ListBookiesCommand ' is failing Introduced and fixed the tests involving the ListBookiesCommand Reviewers: Jia Zhai , Sijie Guo This closes #2125 from mino181295/fix/test-cmd-list, closes #2124 --- .../bookies/ListBookiesCommandTest.java | 57 +++++++++++++++---- 1 file changed, 47 insertions(+), 10 deletions(-) diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java index f887b3bb8a7..9bd68a1e727 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java @@ -31,6 +31,7 @@ import com.google.common.primitives.UnsignedBytes; import java.util.Collections; import java.util.Comparator; +import java.util.HashSet; import java.util.Set; import java.util.TreeSet; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -68,6 +69,7 @@ public int compare(BookieSocketAddress o1, BookieSocketAddress o2) { private Set writableBookies; private Set readonlyBookies; + private Set allBookies; @Before public void setup() throws Exception { @@ -76,10 +78,16 @@ public void setup() throws Exception { writableBookies = createBookies(3181, 10); readonlyBookies = createBookies(4181, 10); + allBookies = new HashSet<>(); + allBookies.addAll(writableBookies); + allBookies.addAll(readonlyBookies); + when(regClient.getWritableBookies()) .thenReturn(value(new Versioned<>(writableBookies, new LongVersion(0L)))); when(regClient.getReadOnlyBookies()) .thenReturn(value(new Versioned<>(readonlyBookies, new LongVersion(0L)))); + when(regClient.getAllBookies()) + .thenReturn(value(new Versioned<>(allBookies, new LongVersion(0L)))); PowerMockito.mockStatic(CommandHelpers.class, CALLS_REAL_METHODS); } @@ -104,51 +112,66 @@ private static void verifyPrintBookies(int startPort, int numBookies, int numCal @Test public void testListReadWriteShortArgs() { - testCommand(true, false, + testCommand(false, true, false, "list", "-rw"); } @Test public void testListReadWriteLongArgs() { - testCommand(true, false, + testCommand(false, true, false, "list", "--readwrite"); } @Test public void testListReadOnlyShortArgs() { - testCommand(false, true, + testCommand(false, false, true, "list", "-ro"); } + @Test + public void testListAllLongArgs() { + testCommand(true, false, false, + "list", + "--all"); + } + + @Test + public void testListAllShortArgs() { + testCommand(true, false, false, + "list", + "-a"); + } + @Test public void testListReadOnlyLongArgs() { - testCommand(false, true, + testCommand(false, false, true, "list", "--readonly"); } @Test public void testListNoArgs() { - testCommand(true, true, + testCommand(true, true, true, "list"); } @Test public void testListTwoFlagsCoexistsShortArgs() { - testCommand(true, true, + testCommand(false, true, true, "list", "-rw", "-ro"); } @Test public void testListTwoFlagsCoexistsLongArgs() { - testCommand(true, true, + testCommand(false, true, true, "list", "--readwrite", "--readonly"); } - private void testCommand(boolean readwrite, + private void testCommand(boolean all, + boolean readwrite, boolean readonly, String... args) { @@ -159,7 +182,21 @@ private void testCommand(boolean readwrite, fail("Should not throw any exception here"); } - if (readwrite && !readonly) { + if (all) { + if (readwrite && readonly) { + verifyPrintBookies(3181, 10, 2); + verifyPrintBookies(4181, 10, 2); + } else if (readwrite && !readonly) { + verifyPrintBookies(3181, 10, 2); + verifyPrintBookies(4181, 10, 1); + } else if (readonly && !readwrite) { + verifyPrintBookies(3181, 10, 1); + verifyPrintBookies(4181, 10, 2); + } else { + verifyPrintBookies(3181, 10, 1); + verifyPrintBookies(4181, 10, 1); + } + } else if (readwrite && !readonly) { verifyPrintBookies(3181, 10, 1); verifyPrintBookies(4181, 10, 0); } else if (readonly && !readwrite) { @@ -187,7 +224,7 @@ public void testListEmptyBookies() throws Exception { times(0)); CommandHelpers.getBookieSocketAddrStringRepresentation(any()); - assertTrue(cmd.apply(bkFlags, new String[] { "-ro"})); + assertTrue(cmd.apply(bkFlags, new String[]{"-ro"})); PowerMockito.verifyStatic( CommandHelpers.class, From eb5894353bfc7f15746116b820338558a219b81b Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Sun, 21 Jul 2019 13:28:55 -0700 Subject: [PATCH 0321/1642] Implementation of new scrutiny - replicas check (Part-1) Descriptions of the changes in this PR: - this is implementation of scrutiny mentioned in https://github.com/apache/bookkeeper/blob/master/site/bps/BP-34-cluster-metadata-checker.md - In this commit, urLedgersElapsedRecoveryGracePeriod check is added to placementPolicyCheck. - new scrutiny is added replicasCheck and it is disabled by default. - this is first part of replicas check, in this part currently metrics are not added and also delayed replicas check for potentially faulty ledgers (failure in trying to get info. from Bookie of the ensembles of the ledger) is not added. - have to add more failure test scenarios for replicas check logic. Reviewers: Enrico Olivelli , Sijie Guo This closes #2120 from reddycharan/replicascheck and squashes the following commits: d4dc81e15 [Sijie Guo] Merge remote-tracking branch 'apache/master' into replicascheck 5a3fa848f [Matteo Minardi] Issue #2124: Test 'ListBookiesCommand ' is failing df24f672a [vzhikserg] Use bulk operation instead of iteration 2a1cfe249 [vzhikserg] Fix log statements with incorrect placeholders --- .../src/main/proto/DataFormats.proto | 6 + .../client/DistributionSchedule.java | 18 + .../RoundRobinDistributionSchedule.java | 32 +- .../bookkeeper/conf/ServerConfiguration.java | 50 + .../meta/LedgerUnderreplicationManager.java | 29 + .../meta/ZkLedgerUnderreplicationManager.java | 115 ++- .../bookkeeper/replication/Auditor.java | 869 +++++++++++++++++- .../replication/ReplicationStats.java | 8 + .../util/AvailabilityOfEntriesOfLedger.java | 57 +- .../bookkeeper/util/BookKeeperConstants.java | 1 + .../client/BookKeeperAdminTest.java | 40 + .../RoundRobinDistributionScheduleTest.java | 46 + .../replication/AuditorLedgerCheckerTest.java | 4 +- .../replication/AuditorPeriodicCheckTest.java | 109 +++ .../AuditorPlacementPolicyCheckTest.java | 130 +++ .../TestLedgerUnderreplicationManager.java | 15 + .../AvailabilityOfEntriesOfLedgerTest.java | 110 +++ site/_data/config/bk_server.yaml | 6 + 18 files changed, 1593 insertions(+), 52 deletions(-) diff --git a/bookkeeper-proto/src/main/proto/DataFormats.proto b/bookkeeper-proto/src/main/proto/DataFormats.proto index 2501affe9b9..6591d651ebd 100644 --- a/bookkeeper-proto/src/main/proto/DataFormats.proto +++ b/bookkeeper-proto/src/main/proto/DataFormats.proto @@ -112,3 +112,9 @@ message PlacementPolicyCheckFormat { optional int64 placementPolicyCheckCTime = 1; } +/** + * information of ReplicasCheck execution + */ +message ReplicasCheckFormat { + optional int64 replicasCheckCTime = 1; +} 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 d53129d22f4..dc6534611f1 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 @@ -17,6 +17,7 @@ */ package org.apache.bookkeeper.client; +import java.util.BitSet; import java.util.Map; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -236,4 +237,21 @@ interface QuorumCoverageSet { * @return true if it has entry otherwise false. */ boolean hasEntry(long entryId, int bookieIndex); + + /** + * Get the bitset representing the entries from entry 'startEntryId' to + * 'lastEntryId', that would be striped to the bookie with index - + * bookieIndex. Value of the bit with the 'bitIndex+n', indicate whether + * entry with entryid 'startEntryId+n' is striped to this bookie or not. + * + * @param bookieIndex + * index of the bookie in the ensemble starting with 0 + * @param startEntryId + * starting entryid + * @param lastEntryId + * last entryid + * @return the bitset representing the entries that would be striped to the + * bookie + */ + BitSet getEntriesStripedToTheBookie(int bookieIndex, long startEntryId, long lastEntryId); } 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 a6506afe72d..05711fd96c1 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 @@ -29,6 +29,8 @@ import java.util.Map; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A specific {@link DistributionSchedule} that places entries in round-robin @@ -37,7 +39,8 @@ * on. * */ -class RoundRobinDistributionSchedule implements DistributionSchedule { +public class RoundRobinDistributionSchedule implements DistributionSchedule { + private static final Logger LOG = LoggerFactory.getLogger(RoundRobinDistributionSchedule.class); private final int writeQuorumSize; private final int ackQuorumSize; private final int ensembleSize; @@ -409,4 +412,31 @@ public boolean hasEntry(long entryId, int bookieIndex) { w.recycle(); } } + + @Override + public BitSet getEntriesStripedToTheBookie(int bookieIndex, long startEntryId, long lastEntryId) { + if ((startEntryId < 0) || (lastEntryId < 0) || (bookieIndex < 0) || (bookieIndex >= ensembleSize) + || (lastEntryId < startEntryId)) { + LOG.error( + "Illegal arguments for getEntriesStripedToTheBookie, bookieIndex : {}," + + " ensembleSize : {}, startEntryId : {}, lastEntryId : {}", + bookieIndex, ensembleSize, startEntryId, lastEntryId); + throw new IllegalArgumentException("Illegal arguments for getEntriesStripedToTheBookie"); + } + BitSet entriesStripedToTheBookie = new BitSet((int) (lastEntryId - startEntryId + 1)); + for (long entryId = startEntryId; entryId <= lastEntryId; entryId++) { + int modValOfFirstReplica = (int) (entryId % ensembleSize); + int modValOfLastReplica = (int) ((entryId + writeQuorumSize - 1) % ensembleSize); + if (modValOfLastReplica >= modValOfFirstReplica) { + if ((bookieIndex >= modValOfFirstReplica) && (bookieIndex <= modValOfLastReplica)) { + entriesStripedToTheBookie.set((int) (entryId - startEntryId)); + } + } else { + if ((bookieIndex >= modValOfFirstReplica) || (bookieIndex <= modValOfLastReplica)) { + entriesStripedToTheBookie.set((int) (entryId - startEntryId)); + } + } + } + return entriesStripedToTheBookie; + } } 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 f4972f0a781..75e230ca7b8 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 @@ -187,6 +187,9 @@ public class ServerConfiguration extends AbstractConfiguration replicaList = underreplicatedLedgerFormat.getReplicaList(); + long ctime = (underreplicatedLedgerFormat.hasCtime() ? underreplicatedLedgerFormat.getCtime() + : UnderreplicatedLedger.UNASSIGNED_CTIME); + underreplicatedLedger.setCtime(ctime); + underreplicatedLedger.setReplicaList(replicaList); + return underreplicatedLedger; + } catch (KeeperException ke) { + throw new ReplicationException.UnavailableException("Error contacting zookeeper", ke); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new ReplicationException.UnavailableException("Interrupted while connecting zookeeper", ie); + } catch (TextFormat.ParseException pe) { + throw new ReplicationException.UnavailableException("Error parsing proto message", pe); + } } @Override @@ -434,28 +460,18 @@ public boolean hasNext() { String parent = queue.remove(); try { for (String c : zkc.getChildren(parent, false)) { - try { - String child = parent + "/" + c; - if (c.startsWith("urL")) { - long ledgerId = getLedgerId(child); - UnderreplicatedLedgerFormat underreplicatedLedgerFormat = - getLedgerUnreplicationInfo(ledgerId); - List replicaList = underreplicatedLedgerFormat.getReplicaList(); - long ctime = (underreplicatedLedgerFormat.hasCtime() - ? underreplicatedLedgerFormat.getCtime() - : UnderreplicatedLedger.UNASSIGNED_CTIME); + String child = parent + "/" + c; + if (c.startsWith("urL")) { + long ledgerId = getLedgerId(child); + UnderreplicatedLedger underreplicatedLedger = getLedgerUnreplicationInfo(ledgerId); + if (underreplicatedLedger != null) { + List replicaList = underreplicatedLedger.getReplicaList(); if ((predicate == null) || predicate.test(replicaList)) { - UnderreplicatedLedger underreplicatedLedger = new UnderreplicatedLedger( - ledgerId); - underreplicatedLedger.setCtime(ctime); - underreplicatedLedger.setReplicaList(replicaList); curBatch.add(underreplicatedLedger); } - } else { - queue.add(child); } - } catch (KeeperException.NoNodeException nne) { - // ignore + } else { + queue.add(child); } } } catch (InterruptedException ie) { @@ -984,4 +1000,49 @@ public long getPlacementPolicyCheckCTime() throws UnavailableException { throw new ReplicationException.UnavailableException("Error while parsing ZK protobuf binary data", ipbe); } } + + @Override + public void setReplicasCheckCTime(long replicasCheckCTime) throws UnavailableException { + try { + List zkAcls = ZkUtils.getACLs(conf); + ReplicasCheckFormat.Builder builder = ReplicasCheckFormat.newBuilder(); + builder.setReplicasCheckCTime(replicasCheckCTime); + byte[] replicasCheckFormatByteArray = builder.build().toByteArray(); + if (zkc.exists(replicasCheckCtimeZnode, false) != null) { + zkc.setData(replicasCheckCtimeZnode, replicasCheckFormatByteArray, -1); + } else { + zkc.create(replicasCheckCtimeZnode, replicasCheckFormatByteArray, zkAcls, CreateMode.PERSISTENT); + } + if (LOG.isDebugEnabled()) { + LOG.debug("setReplicasCheckCTime completed successfully"); + } + } catch (KeeperException ke) { + throw new ReplicationException.UnavailableException("Error contacting zookeeper", ke); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new ReplicationException.UnavailableException("Interrupted while contacting zookeeper", ie); + } + } + + @Override + public long getReplicasCheckCTime() throws UnavailableException { + try { + byte[] data = zkc.getData(replicasCheckCtimeZnode, false, null); + ReplicasCheckFormat replicasCheckFormat = ReplicasCheckFormat.parseFrom(data); + if (LOG.isDebugEnabled()) { + LOG.debug("getReplicasCheckCTime completed successfully"); + } + return replicasCheckFormat.hasReplicasCheckCTime() ? replicasCheckFormat.getReplicasCheckCTime() : -1; + } catch (KeeperException.NoNodeException ne) { + LOG.warn("replicasCheckCtimeZnode is not yet available"); + return -1; + } catch (KeeperException ke) { + throw new ReplicationException.UnavailableException("Error contacting zookeeper", ke); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new ReplicationException.UnavailableException("Interrupted while contacting zookeeper", ie); + } catch (InvalidProtocolBufferException ipbe) { + throw new ReplicationException.UnavailableException("Error while parsing ZK protobuf binary data", ipbe); + } + } } 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 036f5d51aac..3b8219f7345 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 @@ -29,32 +29,49 @@ import static org.apache.bookkeeper.replication.ReplicationStats.NUM_DELAYED_BOOKIE_AUDITS_DELAYES_CANCELLED; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_FRAGMENTS_PER_LEDGER; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_CHECKED; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_HAVING_LESS_THAN_AQ_REPLICAS_OF_AN_ENTRY; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_HAVING_LESS_THAN_WQ_REPLICAS_OF_AN_ENTRY; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_HAVING_NO_REPLICA_OF_AN_ENTRY; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_UNDERREPLICATED_LEDGERS_ELAPSED_RECOVERY_GRACE_PERIOD; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_UNDER_REPLICATED_LEDGERS; import static org.apache.bookkeeper.replication.ReplicationStats.PLACEMENT_POLICY_CHECK_TIME; +import static org.apache.bookkeeper.replication.ReplicationStats.REPLICAS_CHECK_TIME; import static org.apache.bookkeeper.replication.ReplicationStats.URL_PUBLISH_TIME_FOR_LOST_BOOKIE; +import static org.apache.bookkeeper.util.SafeRunnable.safeRun; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; +import com.google.common.collect.HashMultiset; import com.google.common.collect.Lists; +import com.google.common.collect.Multiset; import com.google.common.collect.Sets; import com.google.common.util.concurrent.SettableFuture; import java.io.IOException; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Semaphore; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import java.util.stream.Collectors; import org.apache.bookkeeper.client.BKException; @@ -65,16 +82,21 @@ import org.apache.bookkeeper.client.LedgerChecker; import org.apache.bookkeeper.client.LedgerFragment; import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.RoundRobinDistributionSchedule; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; +import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.MultiCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.ReplicationException.BKAuditException; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -85,8 +107,11 @@ import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.collections4.CollectionUtils; import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.AsyncCallback.VoidCallback; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -106,6 +131,8 @@ ) public class Auditor implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(Auditor.class); + private static final int MAX_CONCURRENT_REPLICAS_CHECK_LEDGER_REQUESTS = 100; + private static final int REPLICAS_CHECK_TIMEOUT_IN_SECS = 120; private final ServerConfiguration conf; private final BookKeeper bkc; private final boolean ownBkc; @@ -124,6 +151,16 @@ public class Auditor implements AutoCloseable { private final AtomicInteger ledgersSoftlyAdheringToPlacementPolicyGuageValue; private final AtomicInteger numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck; private final AtomicInteger numOfClosedLedgersAuditedInPlacementPolicyCheck; + private final AtomicInteger numOfURLedgersElapsedRecoveryGracePeriodGuageValue; + private final AtomicInteger numOfURLedgersElapsedRecoveryGracePeriod; + private final AtomicInteger numLedgersHavingNoReplicaOfAnEntryGuageValue; + private final AtomicInteger numLedgersFoundHavingNoReplicaOfAnEntry; + private final AtomicInteger numLedgersHavingLessThanAQReplicasOfAnEntryGuageValue; + private final AtomicInteger numLedgersFoundHavingLessThanAQReplicasOfAnEntry; + private final AtomicInteger numLedgersHavingLessThanWQReplicasOfAnEntryGuageValue; + private final AtomicInteger numLedgersFoundHavingLessThanWQReplicasOfAnEntry; + private final long underreplicatedLedgerRecoveryGracePeriod; + private final int zkOpTimeoutMs; private final StatsLogger statsLogger; @StatsDoc( @@ -151,6 +188,11 @@ public class Auditor implements AutoCloseable { help = "the latency distribution of placementPolicy check" ) private final OpStatsLogger placementPolicyCheckTime; + @StatsDoc( + name = REPLICAS_CHECK_TIME, + help = "the latency distribution of replicas check" + ) + private final OpStatsLogger replicasCheckTime; @StatsDoc( name = AUDIT_BOOKIES_TIME, help = "the latency distribution of auditing all the bookies" @@ -191,6 +233,28 @@ public class Auditor implements AutoCloseable { help = "Gauge for number of ledgers softly adhering to placement policy found in placement policy check" ) private final Gauge numLedgersSoftlyAdheringToPlacementPolicy; + @StatsDoc( + name = NUM_UNDERREPLICATED_LEDGERS_ELAPSED_RECOVERY_GRACE_PERIOD, + help = "Gauge for number of underreplicated ledgers elapsed recovery grace period" + ) + private final Gauge numUnderreplicatedLedgersElapsedRecoveryGracePeriod; + @StatsDoc( + name = NUM_LEDGERS_HAVING_NO_REPLICA_OF_AN_ENTRY, + help = "Gauge for number of ledgers having an entry with all the replicas missing" + ) + private final Gauge numLedgersHavingNoReplicaOfAnEntry; + @StatsDoc( + name = NUM_LEDGERS_HAVING_LESS_THAN_AQ_REPLICAS_OF_AN_ENTRY, + help = "Gauge for number of ledgers having an entry with less than AQ number of replicas" + + ", this doesn't include ledgers counted towards numLedgersHavingNoReplicaOfAnEntry" + ) + private final Gauge numLedgersHavingLessThanAQReplicasOfAnEntry; + @StatsDoc( + name = NUM_LEDGERS_HAVING_LESS_THAN_WQ_REPLICAS_OF_AN_ENTRY, + help = "Gauge for number of ledgers having an entry with less than WQ number of replicas" + + ", this doesn't include ledgers counted towards numLedgersHavingLessThanAQReplicasOfAnEntry" + ) + private final Gauge numLedgersHavingLessThanWQReplicasOfAnEntry; static BookKeeper createBookKeeperClient(ServerConfiguration conf) throws InterruptedException, IOException { return createBookKeeperClient(conf, NullStatsLogger.INSTANCE); @@ -239,6 +303,8 @@ public Auditor(final String bookieIdentifier, StatsLogger statsLogger) throws UnavailableException { this.conf = conf; + this.underreplicatedLedgerRecoveryGracePeriod = conf.getUnderreplicatedLedgerRecoveryGracePeriod(); + this.zkOpTimeoutMs = conf.getZkTimeout() * 2; this.bookieIdentifier = bookieIdentifier; this.statsLogger = statsLogger; this.numOfLedgersFoundNotAdheringInPlacementPolicyCheck = new AtomicInteger(0); @@ -246,6 +312,14 @@ public Auditor(final String bookieIdentifier, this.numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck = new AtomicInteger(0); this.ledgersSoftlyAdheringToPlacementPolicyGuageValue = new AtomicInteger(0); this.numOfClosedLedgersAuditedInPlacementPolicyCheck = new AtomicInteger(0); + this.numOfURLedgersElapsedRecoveryGracePeriod = new AtomicInteger(0); + this.numOfURLedgersElapsedRecoveryGracePeriodGuageValue = new AtomicInteger(0); + this.numLedgersHavingNoReplicaOfAnEntryGuageValue = new AtomicInteger(0); + this.numLedgersFoundHavingNoReplicaOfAnEntry = new AtomicInteger(0); + this.numLedgersHavingLessThanAQReplicasOfAnEntryGuageValue = new AtomicInteger(0); + this.numLedgersFoundHavingLessThanAQReplicasOfAnEntry = new AtomicInteger(0); + this.numLedgersHavingLessThanWQReplicasOfAnEntryGuageValue = new AtomicInteger(0); + this.numLedgersFoundHavingLessThanWQReplicasOfAnEntry = new AtomicInteger(0); numUnderReplicatedLedger = this.statsLogger.getOpStatsLogger(ReplicationStats.NUM_UNDER_REPLICATED_LEDGERS); uRLPublishTimeForLostBookies = this.statsLogger @@ -254,6 +328,7 @@ public Auditor(final String bookieIdentifier, .getOpStatsLogger(ReplicationStats.BOOKIE_TO_LEDGERS_MAP_CREATION_TIME); checkAllLedgersTime = this.statsLogger.getOpStatsLogger(ReplicationStats.CHECK_ALL_LEDGERS_TIME); placementPolicyCheckTime = this.statsLogger.getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME); + replicasCheckTime = this.statsLogger.getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME); auditBookiesTime = this.statsLogger.getOpStatsLogger(ReplicationStats.AUDIT_BOOKIES_TIME); numLedgersChecked = this.statsLogger.getCounter(ReplicationStats.NUM_LEDGERS_CHECKED); numFragmentsPerLedger = statsLogger.getOpStatsLogger(ReplicationStats.NUM_FRAGMENTS_PER_LEDGER); @@ -288,6 +363,60 @@ public Integer getSample() { this.statsLogger.registerGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY, numLedgersSoftlyAdheringToPlacementPolicy); + numUnderreplicatedLedgersElapsedRecoveryGracePeriod = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + return numOfURLedgersElapsedRecoveryGracePeriodGuageValue.get(); + } + }; + this.statsLogger.registerGauge(ReplicationStats.NUM_UNDERREPLICATED_LEDGERS_ELAPSED_RECOVERY_GRACE_PERIOD, + numUnderreplicatedLedgersElapsedRecoveryGracePeriod); + + numLedgersHavingNoReplicaOfAnEntry = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + return numLedgersHavingNoReplicaOfAnEntryGuageValue.get(); + } + }; + this.statsLogger.registerGauge(ReplicationStats.NUM_LEDGERS_HAVING_NO_REPLICA_OF_AN_ENTRY, + numLedgersHavingNoReplicaOfAnEntry); + numLedgersHavingLessThanAQReplicasOfAnEntry = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + return numLedgersHavingLessThanAQReplicasOfAnEntryGuageValue.get(); + } + }; + this.statsLogger.registerGauge(ReplicationStats.NUM_LEDGERS_HAVING_LESS_THAN_AQ_REPLICAS_OF_AN_ENTRY, + numLedgersHavingLessThanAQReplicasOfAnEntry); + numLedgersHavingLessThanWQReplicasOfAnEntry = new Gauge() { + @Override + public Integer getDefaultValue() { + return 0; + } + + @Override + public Integer getSample() { + return numLedgersHavingLessThanWQReplicasOfAnEntryGuageValue.get(); + } + }; + this.statsLogger.registerGauge(ReplicationStats.NUM_LEDGERS_HAVING_LESS_THAN_WQ_REPLICAS_OF_AN_ENTRY, + numLedgersHavingLessThanWQReplicasOfAnEntry); + this.bkc = bkc; this.ownBkc = ownBkc; initialize(conf, bkc); @@ -346,14 +475,14 @@ private void submitShutdownTask() { LOG.info("executor is already shutdown"); return; } - executor.submit(new Runnable() { + executor.submit(safeRun(new Runnable() { public void run() { synchronized (Auditor.this) { LOG.info("Shutting down Auditor's Executor"); executor.shutdown(); } } - }); + })); } } @@ -364,7 +493,7 @@ synchronized Future submitAuditTask() { f.setException(new BKAuditException("Auditor shutting down")); return f; } - return executor.submit(new Runnable() { + return executor.submit(safeRun(new Runnable() { @SuppressWarnings("unchecked") public void run() { try { @@ -420,13 +549,13 @@ public void run() { } if (auditTask == null) { // if there is no scheduled audit, schedule one - auditTask = executor.schedule(new Runnable() { + auditTask = executor.schedule(safeRun(new Runnable() { public void run() { startAudit(false); auditTask = null; bookiesToBeAudited.clear(); } - }, lostBookieRecoveryDelay, TimeUnit.SECONDS); + }), lostBookieRecoveryDelay, TimeUnit.SECONDS); numBookieAuditsDelayed.inc(); LOG.info("Delaying bookie audit by {} secs for {}", lostBookieRecoveryDelay, bookiesToBeAudited); @@ -440,7 +569,7 @@ public void run() { LOG.error("Exception while watching available bookies", ue); } } - }); + })); } synchronized Future submitLostBookieRecoveryDelayChangedEvent() { @@ -449,7 +578,7 @@ synchronized Future submitLostBookieRecoveryDelayChangedEvent() { f.setException(new BKAuditException("Auditor shutting down")); return f; } - return executor.submit(new Runnable() { + return executor.submit(safeRun(new Runnable() { int lostBookieRecoveryDelay = -1; public void run() { try { @@ -479,13 +608,13 @@ public void run() { } else if (auditTask != null) { LOG.info("lostBookieRecoveryDelay has been set to {}, so rescheduling AuditTask accordingly", lostBookieRecoveryDelay); - auditTask = executor.schedule(new Runnable() { + auditTask = executor.schedule(safeRun(new Runnable() { public void run() { startAudit(false); auditTask = null; bookiesToBeAudited.clear(); } - }, lostBookieRecoveryDelay, TimeUnit.SECONDS); + }), lostBookieRecoveryDelay, TimeUnit.SECONDS); numBookieAuditsDelayed.inc(); } } catch (InterruptedException ie) { @@ -499,7 +628,7 @@ public void run() { } } } - }); + })); } public void start() { @@ -531,6 +660,7 @@ public void start() { scheduleBookieCheckTask(); scheduleCheckAllLedgersTask(); schedulePlacementPolicyCheckTask(); + scheduleReplicasCheckTask(); } } @@ -538,11 +668,11 @@ private void scheduleBookieCheckTask() { long bookieCheckInterval = conf.getAuditorPeriodicBookieCheckInterval(); if (bookieCheckInterval == 0) { LOG.info("Auditor periodic bookie checking disabled, running once check now anyhow"); - executor.submit(bookieCheck); + executor.submit(safeRun(bookieCheck)); } else { LOG.info("Auditor periodic bookie checking enabled" + " 'auditorPeriodicBookieCheckInterval' {} seconds", bookieCheckInterval); - executor.scheduleAtFixedRate(bookieCheck, 0, bookieCheckInterval, TimeUnit.SECONDS); + executor.scheduleAtFixedRate(safeRun(bookieCheck), 0, bookieCheckInterval, TimeUnit.SECONDS); } } @@ -580,7 +710,7 @@ private void scheduleCheckAllLedgersTask(){ + "durationSinceLastExecutionInSecs: {} initialDelay: {} interval: {}", checkAllLedgersLastExecutedCTime, durationSinceLastExecutionInSecs, initialDelay, interval); - executor.scheduleAtFixedRate(new Runnable() { + executor.scheduleAtFixedRate(safeRun(new Runnable() { public void run() { try { if (!ledgerUnderreplicationManager.isLedgerReplicationEnabled()) { @@ -607,7 +737,7 @@ public void run() { LOG.error("Underreplication manager unavailable running periodic check", ue); } } - }, initialDelay, interval, TimeUnit.SECONDS); + }), initialDelay, interval, TimeUnit.SECONDS); } else { LOG.info("Periodic checking disabled"); } @@ -647,7 +777,7 @@ private void schedulePlacementPolicyCheckTask(){ + "durationSinceLastExecutionInSecs: {} initialDelay: {} interval: {}", placementPolicyCheckLastExecutedCTime, durationSinceLastExecutionInSecs, initialDelay, interval); - executor.scheduleAtFixedRate(new Runnable() { + executor.scheduleAtFixedRate(safeRun(new Runnable() { public void run() { try { Stopwatch stopwatch = Stopwatch.createStarted(); @@ -660,18 +790,24 @@ public void run() { numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck.get(); int numOfClosedLedgersAuditedInPlacementPolicyCheckValue = numOfClosedLedgersAuditedInPlacementPolicyCheck.get(); + int numOfURLedgersElapsedRecoveryGracePeriodValue = + numOfURLedgersElapsedRecoveryGracePeriod.get(); LOG.info( "Completed placementPolicyCheck in {} milliSeconds." + " numOfClosedLedgersAuditedInPlacementPolicyCheck {}" + " numOfLedgersNotAdheringToPlacementPolicy {}" - + " numOfLedgersSoftlyAdheringToPlacementPolicy {}", + + " numOfLedgersSoftlyAdheringToPlacementPolicy {}" + + " numOfURLedgersElapsedRecoveryGracePeriod {}", placementPolicyCheckDuration, numOfClosedLedgersAuditedInPlacementPolicyCheckValue, numOfLedgersFoundNotAdheringInPlacementPolicyCheckValue, - numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue); + numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue, + numOfURLedgersElapsedRecoveryGracePeriodValue); ledgersNotAdheringToPlacementPolicyGuageValue .set(numOfLedgersFoundNotAdheringInPlacementPolicyCheckValue); ledgersSoftlyAdheringToPlacementPolicyGuageValue .set(numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue); + numOfURLedgersElapsedRecoveryGracePeriodGuageValue + .set(numOfURLedgersElapsedRecoveryGracePeriodValue); placementPolicyCheckTime.registerSuccessfulEvent(placementPolicyCheckDuration, TimeUnit.MILLISECONDS); } catch (BKAuditException e) { @@ -699,20 +835,132 @@ public void run() { .set(numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue); } + int numOfURLedgersElapsedRecoveryGracePeriodValue = + numOfURLedgersElapsedRecoveryGracePeriod.get(); + if (numOfURLedgersElapsedRecoveryGracePeriodValue > 0) { + /* + * Though there is BKAuditException while doing + * placementPolicyCheck, it found few urledgers have + * elapsed recovery graceperiod. So reporting it. + */ + numOfURLedgersElapsedRecoveryGracePeriodGuageValue + .set(numOfURLedgersElapsedRecoveryGracePeriodValue); + } + LOG.error( "BKAuditException running periodic placementPolicy check." + "numOfLedgersNotAdheringToPlacementPolicy {}, " - + "numOfLedgersSoftlyAdheringToPlacementPolicy {}", + + "numOfLedgersSoftlyAdheringToPlacementPolicy {}," + + "numOfURLedgersElapsedRecoveryGracePeriod {}", numOfLedgersFoundInPlacementPolicyCheckValue, - numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue, e); + numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheckValue, + numOfURLedgersElapsedRecoveryGracePeriodValue, e); } } - }, initialDelay, interval, TimeUnit.SECONDS); + }), initialDelay, interval, TimeUnit.SECONDS); } else { LOG.info("Periodic placementPolicy check disabled"); } } + private void scheduleReplicasCheckTask() { + long interval = conf.getAuditorPeriodicReplicasCheckInterval(); + + if (interval <= 0) { + LOG.info("Periodic replicas check disabled"); + return; + } + + LOG.info("Auditor periodic replicas check enabled" + " 'auditorReplicasCheckInterval' {} seconds", interval); + long replicasCheckLastExecutedCTime; + long durationSinceLastExecutionInSecs; + long initialDelay; + try { + replicasCheckLastExecutedCTime = ledgerUnderreplicationManager.getReplicasCheckCTime(); + } catch (UnavailableException ue) { + LOG.error("Got UnavailableException while trying to get replicasCheckCTime", ue); + replicasCheckLastExecutedCTime = -1; + } + if (replicasCheckLastExecutedCTime == -1) { + durationSinceLastExecutionInSecs = -1; + initialDelay = 0; + } else { + durationSinceLastExecutionInSecs = (System.currentTimeMillis() - replicasCheckLastExecutedCTime) / 1000; + if (durationSinceLastExecutionInSecs < 0) { + // this can happen if there is no strict time ordering + durationSinceLastExecutionInSecs = 0; + } + initialDelay = durationSinceLastExecutionInSecs > interval ? 0 + : (interval - durationSinceLastExecutionInSecs); + } + LOG.info( + "replicasCheck scheduling info. replicasCheckLastExecutedCTime: {} " + + "durationSinceLastExecutionInSecs: {} initialDelay: {} interval: {}", + replicasCheckLastExecutedCTime, durationSinceLastExecutionInSecs, initialDelay, interval); + + executor.scheduleAtFixedRate(safeRun(new Runnable() { + public void run() { + try { + Stopwatch stopwatch = Stopwatch.createStarted(); + LOG.info("Starting ReplicasCheck"); + replicasCheck(); + long replicasCheckDuration = stopwatch.stop().elapsed(TimeUnit.MILLISECONDS); + int numLedgersFoundHavingNoReplicaOfAnEntryValue = numLedgersFoundHavingNoReplicaOfAnEntry.get(); + int numLedgersFoundHavingLessThanAQReplicasOfAnEntryValue = + numLedgersFoundHavingLessThanAQReplicasOfAnEntry.get(); + int numLedgersFoundHavingLessThanWQReplicasOfAnEntryValue = + numLedgersFoundHavingLessThanWQReplicasOfAnEntry.get(); + LOG.info( + "Completed ReplicasCheck in {} milliSeconds numLedgersFoundHavingNoReplicaOfAnEntry {}" + + " numLedgersFoundHavingLessThanAQReplicasOfAnEntry {}" + + " numLedgersFoundHavingLessThanWQReplicasOfAnEntry {}.", + replicasCheckDuration, numLedgersFoundHavingNoReplicaOfAnEntryValue, + numLedgersFoundHavingLessThanAQReplicasOfAnEntryValue, + numLedgersFoundHavingLessThanWQReplicasOfAnEntryValue); + numLedgersHavingNoReplicaOfAnEntryGuageValue.set(numLedgersFoundHavingNoReplicaOfAnEntryValue); + numLedgersHavingLessThanAQReplicasOfAnEntryGuageValue + .set(numLedgersFoundHavingLessThanAQReplicasOfAnEntryValue); + numLedgersHavingLessThanWQReplicasOfAnEntryGuageValue + .set(numLedgersFoundHavingLessThanWQReplicasOfAnEntryValue); + replicasCheckTime.registerSuccessfulEvent(replicasCheckDuration, TimeUnit.MILLISECONDS); + } catch (BKAuditException e) { + LOG.error("BKAuditException running periodic replicas check.", e); + int numLedgersFoundHavingNoReplicaOfAnEntryValue = numLedgersFoundHavingNoReplicaOfAnEntry.get(); + if (numLedgersFoundHavingNoReplicaOfAnEntryValue > 0) { + /* + * Though there is BKAuditException while doing + * replicasCheck, it found few ledgers having no replica + * of an entry. So reporting it. + */ + numLedgersHavingNoReplicaOfAnEntryGuageValue.set(numLedgersFoundHavingNoReplicaOfAnEntryValue); + } + int numLedgersFoundHavingLessThanAQReplicasOfAnEntryValue = + numLedgersFoundHavingLessThanAQReplicasOfAnEntry.get(); + if (numLedgersFoundHavingLessThanAQReplicasOfAnEntryValue > 0) { + /* + * Though there is BKAuditException while doing + * replicasCheck, it found few ledgers having an entry + * less than AQ num of Replicas. So reporting it. + */ + numLedgersHavingLessThanAQReplicasOfAnEntryGuageValue + .set(numLedgersFoundHavingLessThanAQReplicasOfAnEntryValue); + } + int numLedgersFoundHavingLessThanWQReplicasOfAnEntryValue = + numLedgersFoundHavingLessThanWQReplicasOfAnEntry.get(); + if (numLedgersFoundHavingLessThanWQReplicasOfAnEntryValue > 0) { + /* + * Though there is BKAuditException while doing + * replicasCheck, it found few ledgers having an entry + * less than WQ num of Replicas. So reporting it. + */ + numLedgersHavingLessThanWQReplicasOfAnEntryGuageValue + .set(numLedgersFoundHavingLessThanWQReplicasOfAnEntryValue); + } + } + } + }), initialDelay, interval, TimeUnit.SECONDS); + } + private class LostBookieRecoveryDelayChangedCb implements GenericCallback { @Override public void operationComplete(int rc, Void result) { @@ -796,7 +1044,7 @@ private void auditBookies() if (!ledgerUnderreplicationManager.isLedgerReplicationEnabled()) { // has been disabled while we were generating the index // discard this run, and schedule a new one - executor.submit(bookieCheck); + executor.submit(safeRun(bookieCheck)); return; } } catch (UnavailableException ue) { @@ -983,6 +1231,31 @@ void placementPolicyCheck() throws BKAuditException { this.numOfLedgersFoundNotAdheringInPlacementPolicyCheck.set(0); this.numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck.set(0); this.numOfClosedLedgersAuditedInPlacementPolicyCheck.set(0); + this.numOfURLedgersElapsedRecoveryGracePeriod.set(0); + if (this.underreplicatedLedgerRecoveryGracePeriod > 0) { + Iterator underreplicatedLedgersInfo = ledgerUnderreplicationManager + .listLedgersToRereplicate(null); + List urLedgersElapsedRecoveryGracePeriod = new ArrayList(); + while (underreplicatedLedgersInfo.hasNext()) { + UnderreplicatedLedger underreplicatedLedger = underreplicatedLedgersInfo.next(); + long underreplicatedLedgerMarkTimeInMilSecs = underreplicatedLedger.getCtime(); + if (underreplicatedLedgerMarkTimeInMilSecs != UnderreplicatedLedger.UNASSIGNED_CTIME) { + long elapsedTimeInSecs = + (System.currentTimeMillis() - underreplicatedLedgerMarkTimeInMilSecs) / 1000; + if (elapsedTimeInSecs > this.underreplicatedLedgerRecoveryGracePeriod) { + urLedgersElapsedRecoveryGracePeriod.add(underreplicatedLedger.getLedgerId()); + numOfURLedgersElapsedRecoveryGracePeriod.incrementAndGet(); + } + } + } + if (urLedgersElapsedRecoveryGracePeriod.isEmpty()) { + LOG.info("No Underreplicated ledger has elapsed recovery graceperiod: {}", + urLedgersElapsedRecoveryGracePeriod); + } else { + LOG.error("Following Underreplicated ledgers have elapsed recovery graceperiod: {}", + urLedgersElapsedRecoveryGracePeriod); + } + } Processor ledgerProcessor = new Processor() { @Override public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { @@ -1072,6 +1345,560 @@ public void processResult(int rc, String s, Object obj) { } } + private static class MissingEntriesInfo { + // ledger id of missing entries + private final long ledgerId; + /* + * segment details, like start entryid of the segment and ensemble List. + */ + private final Entry> segmentEnsemble; + // bookie missing these entries + private final BookieSocketAddress bookieMissingEntries; + /* + * entries of this segment which are supposed to contain in this bookie + * but missing in this bookie. + */ + private final List unavailableEntriesList; + + private MissingEntriesInfo(long ledgerId, Entry> segmentEnsemble, + BookieSocketAddress bookieMissingEntries, List unavailableEntriesList) { + this.ledgerId = ledgerId; + this.segmentEnsemble = segmentEnsemble; + this.bookieMissingEntries = bookieMissingEntries; + this.unavailableEntriesList = unavailableEntriesList; + } + + private long getLedgerId() { + return ledgerId; + } + + private Entry> getSegmentEnsemble() { + return segmentEnsemble; + } + + private BookieSocketAddress getBookieMissingEntries() { + return bookieMissingEntries; + } + + private List getUnavailableEntriesList() { + return unavailableEntriesList; + } + } + + private static class MissingEntriesInfoOfLedger { + private final long ledgerId; + private final int ensembleSize; + private final int writeQuorumSize; + private final int ackQuorumSize; + private final List missingEntriesInfoList; + + private MissingEntriesInfoOfLedger(long ledgerId, int ensembleSize, int writeQuorumSize, int ackQuorumSize, + List missingEntriesInfoList) { + this.ledgerId = ledgerId; + this.ensembleSize = ensembleSize; + this.writeQuorumSize = writeQuorumSize; + this.ackQuorumSize = ackQuorumSize; + this.missingEntriesInfoList = missingEntriesInfoList; + } + + private long getLedgerId() { + return ledgerId; + } + + private int getEnsembleSize() { + return ensembleSize; + } + + private int getWriteQuorumSize() { + return writeQuorumSize; + } + + private int getAckQuorumSize() { + return ackQuorumSize; + } + + private List getMissingEntriesInfoList() { + return missingEntriesInfoList; + } + } + + private class ReadLedgerMetadataCallbackForReplicasCheck + implements BiConsumer, Throwable> { + private final long ledgerInRange; + private final MultiCallback mcbForThisLedgerRange; + private final ConcurrentHashMap ledgersWithMissingEntries; + private final ConcurrentHashMap ledgersWithUnavailableBookies; + + ReadLedgerMetadataCallbackForReplicasCheck(long ledgerInRange, MultiCallback mcbForThisLedgerRange, + ConcurrentHashMap ledgersWithMissingEntries, + ConcurrentHashMap ledgersWithUnavailableBookies) { + this.ledgerInRange = ledgerInRange; + this.mcbForThisLedgerRange = mcbForThisLedgerRange; + this.ledgersWithMissingEntries = ledgersWithMissingEntries; + this.ledgersWithUnavailableBookies = ledgersWithUnavailableBookies; + } + + @Override + public void accept(Versioned metadataVer, Throwable exception) { + if (exception != null) { + if (BKException + .getExceptionCode(exception) == BKException.Code.NoSuchLedgerExistsOnMetadataServerException) { + LOG.debug("Ignoring replicas check of already deleted ledger {}", ledgerInRange); + mcbForThisLedgerRange.processResult(BKException.Code.OK, null, null); + return; + } else { + LOG.warn("Unable to read the ledger: {} information", ledgerInRange, exception); + mcbForThisLedgerRange.processResult(BKException.getExceptionCode(exception), null, null); + return; + } + } + + LedgerMetadata metadata = metadataVer.getValue(); + if (!metadata.isClosed()) { + LOG.debug("Ledger: {} is not yet closed, so skipping the replicas check analysis for now", + ledgerInRange); + mcbForThisLedgerRange.processResult(BKException.Code.OK, null, null); + return; + } + + if (metadata.getLastEntryId() == -1) { + LOG.debug("Ledger: {} is closed but it doesn't has any entries, so skipping the replicas check", + ledgerInRange); + mcbForThisLedgerRange.processResult(BKException.Code.OK, null, null); + return; + } + + int writeQuorumSize = metadata.getWriteQuorumSize(); + int ackQuorumSize = metadata.getAckQuorumSize(); + int ensembleSize = metadata.getEnsembleSize(); + RoundRobinDistributionSchedule distributionSchedule = new RoundRobinDistributionSchedule(writeQuorumSize, + ackQuorumSize, ensembleSize); + List>> segments = new LinkedList<>( + metadata.getAllEnsembles().entrySet()); + /* + * since there are multiple segments, MultiCallback should be + * created for (ensembleSize * segments.size()) calls. + */ + MultiCallback mcbForThisLedger = new MultiCallback(ensembleSize * segments.size(), mcbForThisLedgerRange, + null, BKException.Code.OK, BKException.Code.ReadException); + HashMap> bookiesSegmentInfoMap = + new HashMap>(); + for (int segmentNum = 0; segmentNum < segments.size(); segmentNum++) { + final Entry> segmentEnsemble = segments.get(segmentNum); + final List ensembleOfSegment = segmentEnsemble.getValue(); + final long startEntryIdOfSegment = segmentEnsemble.getKey(); + final long lastEntryIdOfSegment = (segmentNum == (segments.size() - 1)) ? metadata.getLastEntryId() + : segments.get(segmentNum + 1).getKey() - 1; + for (int bookieIndex = 0; bookieIndex < ensembleOfSegment.size(); bookieIndex++) { + final BookieSocketAddress bookieInEnsemble = ensembleOfSegment.get(bookieIndex); + final BitSet entriesStripedToThisBookie = distributionSchedule + .getEntriesStripedToTheBookie(bookieIndex, startEntryIdOfSegment, lastEntryIdOfSegment); + if (entriesStripedToThisBookie.cardinality() == 0) { + /* + * if no entry is expected to contain in this bookie, + * then there is no point in making + * getListOfEntriesOfLedger call for this bookie. So + * instead callback with success result. + */ + if (LOG.isDebugEnabled()) { + LOG.debug( + "For ledger: {}, in Segment: {}, no entry is expected to contain in" + + " this bookie: {}. So skipping getListOfEntriesOfLedger call", + ledgerInRange, segmentEnsemble, bookieInEnsemble); + } + mcbForThisLedger.processResult(BKException.Code.OK, null, null); + continue; + } + List bookieSegmentInfoList = bookiesSegmentInfoMap + .get(bookieInEnsemble); + if (bookieSegmentInfoList == null) { + bookieSegmentInfoList = new ArrayList(); + bookiesSegmentInfoMap.put(bookieInEnsemble, bookieSegmentInfoList); + } + bookieSegmentInfoList.add(new BookieExpectedToContainSegmentInfo(startEntryIdOfSegment, + lastEntryIdOfSegment, segmentEnsemble, entriesStripedToThisBookie)); + } + } + for (Entry> bookiesSegmentInfoTuple : + bookiesSegmentInfoMap.entrySet()) { + final BookieSocketAddress bookieInEnsemble = bookiesSegmentInfoTuple.getKey(); + final List bookieSegmentInfoList = bookiesSegmentInfoTuple + .getValue(); + admin.asyncGetListOfEntriesOfLedger(bookieInEnsemble, ledgerInRange) + .whenComplete(new GetListOfEntriesOfLedgerCallbackForReplicasCheck(ledgerInRange, ensembleSize, + writeQuorumSize, ackQuorumSize, bookieInEnsemble, bookieSegmentInfoList, + ledgersWithMissingEntries, ledgersWithUnavailableBookies, mcbForThisLedger)); + } + } + } + + private static class BookieExpectedToContainSegmentInfo { + private final long startEntryIdOfSegment; + private final long lastEntryIdOfSegment; + private final Entry> segmentEnsemble; + private final BitSet entriesOfSegmentStripedToThisBookie; + + private BookieExpectedToContainSegmentInfo(long startEntryIdOfSegment, long lastEntryIdOfSegment, + Entry> segmentEnsemble, + BitSet entriesOfSegmentStripedToThisBookie) { + this.startEntryIdOfSegment = startEntryIdOfSegment; + this.lastEntryIdOfSegment = lastEntryIdOfSegment; + this.segmentEnsemble = segmentEnsemble; + this.entriesOfSegmentStripedToThisBookie = entriesOfSegmentStripedToThisBookie; + } + + public long getStartEntryIdOfSegment() { + return startEntryIdOfSegment; + } + + public long getLastEntryIdOfSegment() { + return lastEntryIdOfSegment; + } + + public Entry> getSegmentEnsemble() { + return segmentEnsemble; + } + + public BitSet getEntriesOfSegmentStripedToThisBookie() { + return entriesOfSegmentStripedToThisBookie; + } + } + + private static class GetListOfEntriesOfLedgerCallbackForReplicasCheck + implements BiConsumer { + private final long ledgerInRange; + private final int ensembleSize; + private final int writeQuorumSize; + private final int ackQuorumSize; + private final BookieSocketAddress bookieInEnsemble; + private final List bookieExpectedToContainSegmentInfoList; + private final ConcurrentHashMap ledgersWithMissingEntries; + private final ConcurrentHashMap ledgersWithUnavailableBookies; + private final MultiCallback mcbForThisLedger; + + private GetListOfEntriesOfLedgerCallbackForReplicasCheck(long ledgerInRange, int ensembleSize, + int writeQuorumSize, int ackQuorumSize, BookieSocketAddress bookieInEnsemble, + List bookieExpectedToContainSegmentInfoList, + ConcurrentHashMap ledgersWithMissingEntries, + ConcurrentHashMap ledgersWithUnavailableBookies, + MultiCallback mcbForThisLedger) { + this.ledgerInRange = ledgerInRange; + this.ensembleSize = ensembleSize; + this.writeQuorumSize = writeQuorumSize; + this.ackQuorumSize = ackQuorumSize; + this.bookieInEnsemble = bookieInEnsemble; + this.bookieExpectedToContainSegmentInfoList = bookieExpectedToContainSegmentInfoList; + this.ledgersWithMissingEntries = ledgersWithMissingEntries; + this.ledgersWithUnavailableBookies = ledgersWithUnavailableBookies; + this.mcbForThisLedger = mcbForThisLedger; + } + + @Override + public void accept(AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger, + Throwable listOfEntriesException) { + + if (listOfEntriesException != null) { + if (BKException + .getExceptionCode(listOfEntriesException) == BKException.Code.NoSuchLedgerExistsException) { + LOG.debug("Got NoSuchLedgerExistsException for ledger: {} from bookie: {}", ledgerInRange, + bookieInEnsemble); + /* + * in the case of NoSuchLedgerExistsException, it should be + * considered as empty AvailabilityOfEntriesOfLedger. + */ + availabilityOfEntriesOfLedger = AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER; + } else { + LOG.warn("Unable to GetListOfEntriesOfLedger for ledger: {} from: {}", ledgerInRange, + bookieInEnsemble, listOfEntriesException); + MissingEntriesInfoOfLedger unavailableBookiesInfoOfThisLedger = ledgersWithUnavailableBookies + .get(ledgerInRange); + if (unavailableBookiesInfoOfThisLedger == null) { + ledgersWithUnavailableBookies.putIfAbsent(ledgerInRange, + new MissingEntriesInfoOfLedger(ledgerInRange, ensembleSize, writeQuorumSize, + ackQuorumSize, + Collections.synchronizedList(new ArrayList()))); + unavailableBookiesInfoOfThisLedger = ledgersWithUnavailableBookies.get(ledgerInRange); + } + List missingEntriesInfoList = + unavailableBookiesInfoOfThisLedger.getMissingEntriesInfoList(); + for (BookieExpectedToContainSegmentInfo bookieExpectedToContainSegmentInfo + : bookieExpectedToContainSegmentInfoList) { + missingEntriesInfoList.add(new MissingEntriesInfo(ledgerInRange, + bookieExpectedToContainSegmentInfo.getSegmentEnsemble(), bookieInEnsemble, null)); + /* + * though GetListOfEntriesOfLedger has failed with + * exception, mcbForThisLedger should be called back + * with OK response, because we dont consider this as + * fatal error in replicasCheck and dont want + * replicasCheck to exit just because of this issue. So + * instead maintain the state of + * ledgersWithUnavailableBookies, so that replicascheck + * will report these ledgers/bookies appropriately. + */ + mcbForThisLedger.processResult(BKException.Code.OK, null, null); + } + return; + } + } + + for (BookieExpectedToContainSegmentInfo bookieExpectedToContainSegmentInfo + : bookieExpectedToContainSegmentInfoList) { + final long startEntryIdOfSegment = bookieExpectedToContainSegmentInfo.getStartEntryIdOfSegment(); + final long lastEntryIdOfSegment = bookieExpectedToContainSegmentInfo.getLastEntryIdOfSegment(); + final BitSet entriesStripedToThisBookie = bookieExpectedToContainSegmentInfo + .getEntriesOfSegmentStripedToThisBookie(); + final Entry> segmentEnsemble = + bookieExpectedToContainSegmentInfo.getSegmentEnsemble(); + final List unavailableEntriesList = availabilityOfEntriesOfLedger + .getUnavailableEntries(startEntryIdOfSegment, lastEntryIdOfSegment, entriesStripedToThisBookie); + if ((unavailableEntriesList != null) && (!unavailableEntriesList.isEmpty())) { + MissingEntriesInfoOfLedger missingEntriesInfoOfThisLedger = ledgersWithMissingEntries + .get(ledgerInRange); + if (missingEntriesInfoOfThisLedger == null) { + ledgersWithMissingEntries.putIfAbsent(ledgerInRange, + new MissingEntriesInfoOfLedger(ledgerInRange, ensembleSize, writeQuorumSize, + ackQuorumSize, + Collections.synchronizedList(new ArrayList()))); + missingEntriesInfoOfThisLedger = ledgersWithMissingEntries.get(ledgerInRange); + } + missingEntriesInfoOfThisLedger.getMissingEntriesInfoList().add(new MissingEntriesInfo(ledgerInRange, + segmentEnsemble, bookieInEnsemble, unavailableEntriesList)); + } + /* + * here though unavailableEntriesList is not empty, + * mcbForThisLedger should be called back with OK response, + * because we dont consider this as fatal error in replicasCheck + * and dont want replicasCheck to exit just because of this + * issue. So instead maintain the state of + * missingEntriesInfoOfThisLedger, so that replicascheck will + * report these ledgers/bookies/missingentries appropriately. + */ + mcbForThisLedger.processResult(BKException.Code.OK, null, null); + } + } + } + + private static class ReplicasCheckFinalCallback implements AsyncCallback.VoidCallback { + final AtomicInteger resultCode; + final CountDownLatch replicasCheckLatch; + + private ReplicasCheckFinalCallback(AtomicInteger resultCode, CountDownLatch replicasCheckLatch) { + this.resultCode = resultCode; + this.replicasCheckLatch = replicasCheckLatch; + } + + @Override + public void processResult(int rc, String s, Object obj) { + resultCode.set(rc); + replicasCheckLatch.countDown(); + } + } + + void replicasCheck() throws BKAuditException { + ConcurrentHashMap ledgersWithMissingEntries = + new ConcurrentHashMap(); + ConcurrentHashMap ledgersWithUnavailableBookies = + new ConcurrentHashMap(); + LedgerRangeIterator ledgerRangeIterator = ledgerManager.getLedgerRanges(zkOpTimeoutMs); + final Semaphore maxConcurrentSemaphore = new Semaphore(MAX_CONCURRENT_REPLICAS_CHECK_LEDGER_REQUESTS); + while (true) { + LedgerRange ledgerRange = null; + try { + if (ledgerRangeIterator.hasNext()) { + ledgerRange = ledgerRangeIterator.next(); + } else { + break; + } + } catch (IOException ioe) { + LOG.error("Got IOException while iterating LedgerRangeIterator", ioe); + throw new BKAuditException("Got IOException while iterating LedgerRangeIterator", ioe); + } + ledgersWithMissingEntries.clear(); + ledgersWithUnavailableBookies.clear(); + numLedgersFoundHavingNoReplicaOfAnEntry.set(0); + numLedgersFoundHavingLessThanAQReplicasOfAnEntry.set(0); + numLedgersFoundHavingLessThanWQReplicasOfAnEntry.set(0); + Set ledgersInRange = ledgerRange.getLedgers(); + int numOfLedgersInRange = ledgersInRange.size(); + // Final result after processing all the ledgers + final AtomicInteger resultCode = new AtomicInteger(); + final CountDownLatch replicasCheckLatch = new CountDownLatch(1); + + ReplicasCheckFinalCallback finalCB = new ReplicasCheckFinalCallback(resultCode, replicasCheckLatch); + MultiCallback mcbForThisLedgerRange = new MultiCallback(numOfLedgersInRange, finalCB, null, + BKException.Code.OK, BKException.Code.ReadException) { + @Override + public void processResult(int rc, String path, Object ctx) { + try { + super.processResult(rc, path, ctx); + } finally { + maxConcurrentSemaphore.release(); + } + } + }; + LOG.debug("Number of ledgers in the current LedgerRange : {}", numOfLedgersInRange); + for (Long ledgerInRange : ledgersInRange) { + try { + if (!maxConcurrentSemaphore.tryAcquire(REPLICAS_CHECK_TIMEOUT_IN_SECS, TimeUnit.SECONDS)) { + LOG.error("Timedout ({} secs) while waiting for acquiring semaphore", + REPLICAS_CHECK_TIMEOUT_IN_SECS); + throw new BKAuditException("Timedout while waiting for acquiring semaphore"); + } + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + LOG.error("Got InterruptedException while acquiring semaphore for replicascheck", ie); + throw new BKAuditException("Got InterruptedException while acquiring semaphore for replicascheck", + ie); + } + if (checkUnderReplicationForReplicasCheck(ledgerInRange, mcbForThisLedgerRange)) { + /* + * if ledger is marked underreplicated, then ignore this + * ledger for replicascheck. + */ + continue; + } + ledgerManager.readLedgerMetadata(ledgerInRange) + .whenComplete(new ReadLedgerMetadataCallbackForReplicasCheck(ledgerInRange, + mcbForThisLedgerRange, ledgersWithMissingEntries, ledgersWithUnavailableBookies)); + } + try { + /* + * if mcbForThisLedgerRange is not calledback within + * REPLICAS_CHECK_TIMEOUT_IN_SECS secs then better give up + * doing replicascheck, since there could be an issue and + * blocking the single threaded auditor executor thread is not + * expected. + */ + if (!replicasCheckLatch.await(REPLICAS_CHECK_TIMEOUT_IN_SECS, TimeUnit.SECONDS)) { + LOG.error( + "For LedgerRange with num of ledgers : {} it didn't complete replicascheck" + + " in {} secs, so giving up", + numOfLedgersInRange, REPLICAS_CHECK_TIMEOUT_IN_SECS); + throw new BKAuditException("Got InterruptedException while doing replicascheck"); + } + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + LOG.error("Got InterruptedException while doing replicascheck", ie); + throw new BKAuditException("Got InterruptedException while doing replicascheck", ie); + } + reportLedgersWithMissingEntries(ledgersWithMissingEntries); + reportLedgersWithUnavailableBookies(ledgersWithUnavailableBookies); + int resultCodeIntValue = resultCode.get(); + if (resultCodeIntValue != BKException.Code.OK) { + throw new BKAuditException("Exception while doing replicas check", + BKException.create(resultCodeIntValue)); + } + } + try { + ledgerUnderreplicationManager.setReplicasCheckCTime(System.currentTimeMillis()); + } catch (UnavailableException ue) { + LOG.error("Got exception while trying to set ReplicasCheckCTime", ue); + } + } + + private void reportLedgersWithMissingEntries( + ConcurrentHashMap ledgersWithMissingEntries) { + StringBuilder errMessage = new StringBuilder(); + HashMultiset missingEntries = HashMultiset.create(); + int writeQuorumSize; + int ackQuorumSize; + for (Map.Entry missingEntriesInfoOfLedgerEntry : ledgersWithMissingEntries + .entrySet()) { + missingEntries.clear(); + errMessage.setLength(0); + long ledgerWithMissingEntries = missingEntriesInfoOfLedgerEntry.getKey(); + MissingEntriesInfoOfLedger missingEntriesInfoOfLedger = missingEntriesInfoOfLedgerEntry.getValue(); + List missingEntriesInfoList = missingEntriesInfoOfLedger.getMissingEntriesInfoList(); + writeQuorumSize = missingEntriesInfoOfLedger.getWriteQuorumSize(); + ackQuorumSize = missingEntriesInfoOfLedger.getAckQuorumSize(); + errMessage.append("Ledger : " + ledgerWithMissingEntries + " has following missing entries : "); + for (int listInd = 0; listInd < missingEntriesInfoList.size(); listInd++) { + MissingEntriesInfo missingEntriesInfo = missingEntriesInfoList.get(listInd); + List unavailableEntriesList = missingEntriesInfo.getUnavailableEntriesList(); + Entry> segmentEnsemble = + missingEntriesInfo.getSegmentEnsemble(); + missingEntries.addAll(unavailableEntriesList); + errMessage.append("In segment starting at " + segmentEnsemble.getKey() + " with ensemble " + + segmentEnsemble.getValue() + ", following entries " + unavailableEntriesList + + " are missing in bookie: " + missingEntriesInfo.getBookieMissingEntries()); + if (listInd < (missingEntriesInfoList.size() - 1)) { + errMessage.append(", "); + } + } + LOG.error(errMessage.toString()); + Set> missingEntriesSet = missingEntries.entrySet(); + int maxNumOfMissingReplicas = 0; + long entryWithMaxNumOfMissingReplicas = -1L; + for (Multiset.Entry missingEntryWithCount : missingEntriesSet) { + if (missingEntryWithCount.getCount() > maxNumOfMissingReplicas) { + maxNumOfMissingReplicas = missingEntryWithCount.getCount(); + entryWithMaxNumOfMissingReplicas = missingEntryWithCount.getElement(); + } + } + int leastNumOfReplicasOfAnEntry = writeQuorumSize - maxNumOfMissingReplicas; + if (leastNumOfReplicasOfAnEntry == 0) { + numLedgersFoundHavingNoReplicaOfAnEntry.incrementAndGet(); + LOG.error("Ledger : {} entryId : {} is missing all replicas", ledgerWithMissingEntries, + entryWithMaxNumOfMissingReplicas); + } else if (leastNumOfReplicasOfAnEntry < ackQuorumSize) { + numLedgersFoundHavingLessThanAQReplicasOfAnEntry.incrementAndGet(); + LOG.error("Ledger : {} entryId : {} is having: {} replicas, less than ackQuorum num of replicas : {}", + ledgerWithMissingEntries, entryWithMaxNumOfMissingReplicas, leastNumOfReplicasOfAnEntry, + ackQuorumSize); + } else if (leastNumOfReplicasOfAnEntry < writeQuorumSize) { + numLedgersFoundHavingLessThanWQReplicasOfAnEntry.incrementAndGet(); + LOG.error("Ledger : {} entryId : {} is having: {} replicas, less than writeQuorum num of replicas : {}", + ledgerWithMissingEntries, entryWithMaxNumOfMissingReplicas, leastNumOfReplicasOfAnEntry, + writeQuorumSize); + } + } + } + + private void reportLedgersWithUnavailableBookies( + ConcurrentHashMap ledgersWithUnavailableBookies) { + StringBuilder errMessage = new StringBuilder(); + for (Map.Entry ledgerWithUnavailableBookiesInfo : + ledgersWithUnavailableBookies.entrySet()) { + errMessage.setLength(0); + long ledgerWithUnavailableBookies = ledgerWithUnavailableBookiesInfo.getKey(); + List missingBookiesInfoList = ledgerWithUnavailableBookiesInfo.getValue() + .getMissingEntriesInfoList(); + errMessage.append("Ledger : " + ledgerWithUnavailableBookies + " has following unavailable bookies : "); + for (int listInd = 0; listInd < missingBookiesInfoList.size(); listInd++) { + MissingEntriesInfo missingBookieInfo = missingBookiesInfoList.get(listInd); + Entry> segmentEnsemble = + missingBookieInfo.getSegmentEnsemble(); + errMessage.append("In segment starting at " + segmentEnsemble.getKey() + " with ensemble " + + segmentEnsemble.getValue() + ", following bookie has not responded " + + missingBookieInfo.getBookieMissingEntries()); + if (listInd < (missingBookiesInfoList.size() - 1)) { + errMessage.append(", "); + } + } + LOG.error(errMessage.toString()); + } + } + + boolean checkUnderReplicationForReplicasCheck(long ledgerInRange, VoidCallback mcbForThisLedgerRange) { + try { + if (ledgerUnderreplicationManager.getLedgerUnreplicationInfo(ledgerInRange) == null) { + return false; + } + /* + * this ledger is marked underreplicated, so ignore it for + * replicasCheck. + */ + LOG.debug("Ledger: {} is marked underrreplicated, ignore this ledger for replicasCheck", + ledgerInRange); + mcbForThisLedgerRange.processResult(BKException.Code.OK, null, null); + return true; + } catch (UnavailableException une) { + LOG.error("Got exception while trying to check if ledger: {} is underreplicated", ledgerInRange, une); + mcbForThisLedgerRange.processResult(BKException.getExceptionCode(une), null, null); + return true; + } + } + /** * Shutdown the auditor. */ 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 2f83804a8ee..f553f91cefd 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 @@ -34,6 +34,7 @@ public interface ReplicationStats { String BOOKIE_TO_LEDGERS_MAP_CREATION_TIME = "BOOKIE_TO_LEDGERS_MAP_CREATION_TIME"; String CHECK_ALL_LEDGERS_TIME = "CHECK_ALL_LEDGERS_TIME"; String PLACEMENT_POLICY_CHECK_TIME = "PLACEMENT_POLICY_CHECK_TIME"; + String REPLICAS_CHECK_TIME = "REPLICAS_CHECK_TIME"; String AUDIT_BOOKIES_TIME = "AUDIT_BOOKIES_TIME"; String NUM_FRAGMENTS_PER_LEDGER = "NUM_FRAGMENTS_PER_LEDGER"; String NUM_BOOKIES_PER_LEDGER = "NUM_BOOKIES_PER_LEDGER"; @@ -42,6 +43,13 @@ public interface ReplicationStats { String NUM_DELAYED_BOOKIE_AUDITS_DELAYES_CANCELLED = "NUM_DELAYED_BOOKIE_AUDITS_CANCELLED"; String NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY = "NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY"; String NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY = "NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY"; + String NUM_UNDERREPLICATED_LEDGERS_ELAPSED_RECOVERY_GRACE_PERIOD = + "NUM_UNDERREPLICATED_LEDGERS_ELAPSED_RECOVERY_GRACE_PERIOD"; + String NUM_LEDGERS_HAVING_NO_REPLICA_OF_AN_ENTRY = "NUM_LEDGERS_HAVING_NO_REPLICA_OF_AN_ENTRY"; + String NUM_LEDGERS_HAVING_LESS_THAN_AQ_REPLICAS_OF_AN_ENTRY = + "NUM_LEDGERS_HAVING_LESS_THAN_AQ_REPLICAS_OF_AN_ENTRY"; + String NUM_LEDGERS_HAVING_LESS_THAN_WQ_REPLICAS_OF_AN_ENTRY = + "NUM_LEDGERS_HAVING_LESS_THAN_WQ_REPLICAS_OF_AN_ENTRY"; String REPLICATION_WORKER_SCOPE = "replication_worker"; String REREPLICATE_OP = "rereplicate"; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java index 6042bdb4385..8a3047a477e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java @@ -18,9 +18,11 @@ package org.apache.bookkeeper.util; import io.netty.buffer.ByteBuf; - import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; +import java.util.BitSet; +import java.util.List; import java.util.Map.Entry; import java.util.PrimitiveIterator; import java.util.TreeMap; @@ -44,6 +46,11 @@ */ public class AvailabilityOfEntriesOfLedger { public static final long INVALID_ENTRYID = -1; + public static final AvailabilityOfEntriesOfLedger EMPTY_AVAILABILITYOFENTRIESOFLEDGER; + static { + long tmpArray[] = {}; + EMPTY_AVAILABILITYOFENTRIESOFLEDGER = new AvailabilityOfEntriesOfLedger(Arrays.stream(tmpArray).iterator()); + } /* * @@ -109,6 +116,10 @@ private long getLastSequenceStart() { return lastSequenceStart; } + private long getLastEntryInSequenceGroup() { + return lastSequenceStart + sequenceSize; + } + private void setLastSequenceStart(long lastSequenceStart) { this.lastSequenceStart = lastSequenceStart; } @@ -367,6 +378,50 @@ public boolean isEntryAvailable(long entryId) { return seqGroup.getValue().isEntryAvailable(entryId); } + public List getUnavailableEntries(long startEntryId, long lastEntryId, BitSet availabilityOfEntries) { + if (!isAvailabilityOfEntriesOfLedgerClosed()) { + throw new IllegalStateException( + "AvailabilityOfEntriesOfLedger is not yet closed, it is illegal to call getUnavailableEntries"); + } + List unavailableEntries = new ArrayList(); + SequenceGroup curSeqGroup = null; + boolean noSeqGroupRemaining = false; + int bitSetIndex = 0; + for (long entryId = startEntryId; entryId <= lastEntryId; entryId++, bitSetIndex++) { + if (noSeqGroupRemaining) { + if (availabilityOfEntries.get(bitSetIndex)) { + unavailableEntries.add(entryId); + } + continue; + } + if ((curSeqGroup == null) || (entryId > curSeqGroup.getLastEntryInSequenceGroup())) { + Entry curSeqGroupEntry = sortedSequenceGroups.floorEntry(entryId); + if (curSeqGroupEntry == null) { + if (availabilityOfEntries.get(bitSetIndex)) { + unavailableEntries.add(entryId); + } + if (sortedSequenceGroups.ceilingEntry(entryId) == null) { + noSeqGroupRemaining = true; + } + continue; + } else { + curSeqGroup = curSeqGroupEntry.getValue(); + if (entryId > curSeqGroup.getLastEntryInSequenceGroup()) { + if (availabilityOfEntries.get(bitSetIndex)) { + unavailableEntries.add(entryId); + } + noSeqGroupRemaining = true; + continue; + } + } + } + if (availabilityOfEntries.get(bitSetIndex) && (!curSeqGroup.isEntryAvailable(entryId))) { + unavailableEntries.add(entryId); + } + } + return unavailableEntries; + } + public long getTotalNumOfAvailableEntries() { if (!isAvailabilityOfEntriesOfLedgerClosed()) { throw new IllegalStateException("AvailabilityOfEntriesOfLedger is not yet closed," 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 89b0d0c3aa4..5fbf5ad0eae 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 @@ -44,6 +44,7 @@ public class BookKeeperConstants { public static final String LOSTBOOKIERECOVERYDELAY_NODE = "lostBookieRecoveryDelay"; public static final String CHECK_ALL_LEDGERS_CTIME = "checkallledgersctime"; public static final String PLACEMENT_POLICY_CHECK_CTIME = "placementpolicycheckctime"; + public static final String REPLICAS_CHECK_CTIME = "replicascheckctime"; public static final String DEFAULT_ZK_LEDGERS_ROOT_PATH = "/ledgers"; public static final String LAYOUT_ZNODE = "LAYOUT"; public static final String INSTANCEID = "INSTANCEID"; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index 03f182d02eb..3d409d8002d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -36,7 +36,10 @@ import java.util.List; import java.util.Random; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.client.BookKeeper.DigestType; @@ -535,4 +538,41 @@ public void testGetBookies() throws Exception { } } + @Test + public void testGetListOfEntriesOfLedgerWithEntriesNotStripedToABookie() throws Exception { + ClientConfiguration conf = new ClientConfiguration(); + conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + BookKeeper bkc = new BookKeeper(conf); + /* + * in this testsuite there are going to be 2 (numOfBookies) bookies and + * we are having ensemble of size 2. + */ + LedgerHandle lh = bkc.createLedger(2, 1, digestType, "testPasswd".getBytes()); + long lId = lh.getId(); + /* + * ledger is writeclosed without adding any entry. + */ + lh.close(); + CountDownLatch callbackCalled = new CountDownLatch(1); + AtomicBoolean exceptionInCallback = new AtomicBoolean(false); + AtomicInteger exceptionCode = new AtomicInteger(BKException.Code.OK); + BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString()); + /* + * since no entry is added, callback is supposed to fail with + * NoSuchLedgerExistsException. + */ + bkAdmin.asyncGetListOfEntriesOfLedger(bs.get(0).getLocalAddress(), lId) + .whenComplete((availabilityOfEntriesOfLedger, throwable) -> { + exceptionInCallback.set(throwable != null); + if (throwable != null) { + exceptionCode.set(BKException.getExceptionCode(throwable)); + } + callbackCalled.countDown(); + }); + callbackCalled.await(); + assertTrue("Exception occurred", exceptionInCallback.get()); + assertEquals("Exception code", BKException.Code.NoSuchLedgerExistsException, exceptionCode.get()); + bkAdmin.close(); + bkc.close(); + } } 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 b78897a4cb8..4498eeaf931 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 @@ -28,6 +28,7 @@ import com.google.common.collect.Sets; +import java.util.BitSet; import java.util.HashSet; import java.util.Set; @@ -171,4 +172,49 @@ public void testMoveAndShift() { w.moveAndShift(4, 4); assertEquals(w, writeSetFromValues(1, 2, 3, 4, 5)); } + + @Test + public void testGetEntriesStripedToTheBookie() { + + RoundRobinDistributionSchedule schedule; + BitSet entriesStriped; + + int ensSize = 3; + int writeQuorum = 3; + int ackQuorum = 3; + int startEntryId = 3; + int lastEntryId = 5; + schedule = new RoundRobinDistributionSchedule(writeQuorum, ackQuorum, ensSize); + + for (int bookieIndex = 0; bookieIndex < ensSize; bookieIndex++) { + entriesStriped = schedule.getEntriesStripedToTheBookie(bookieIndex, startEntryId, lastEntryId); + assertEquals("Cardinality", 3, entriesStriped.cardinality()); + for (int i = 0; i < entriesStriped.length(); i++) { + assertEquals("EntryAvailability", schedule.hasEntry((startEntryId + i), bookieIndex), + entriesStriped.get(i)); + } + } + + ensSize = 5; + writeQuorum = 3; + ackQuorum = 2; + startEntryId = 100; + lastEntryId = 122; + schedule = new RoundRobinDistributionSchedule(writeQuorum, ackQuorum, ensSize); + for (int bookieIndex = 0; bookieIndex < ensSize; bookieIndex++) { + entriesStriped = schedule.getEntriesStripedToTheBookie(bookieIndex, startEntryId, lastEntryId); + for (int i = 0; i < entriesStriped.length(); i++) { + assertEquals("EntryAvailability", schedule.hasEntry((startEntryId + i), bookieIndex), + entriesStriped.get(i)); + } + } + + schedule = new RoundRobinDistributionSchedule(2, 2, 3); + entriesStriped = schedule.getEntriesStripedToTheBookie(2, 0, 0); + assertEquals("Cardinality", 0, entriesStriped.cardinality()); + entriesStriped = schedule.getEntriesStripedToTheBookie(2, 3, 3); + assertEquals("Cardinality", 0, entriesStriped.cardinality()); + entriesStriped = schedule.getEntriesStripedToTheBookie(2, 4, 4); + assertEquals("Cardinality", 1, entriesStriped.cardinality()); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index f25aa88180b..bf96a5e37cd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -58,11 +58,11 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.MetadataClientDriver; import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.proto.DataFormats.UnderreplicatedLedgerFormat; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -809,7 +809,7 @@ private boolean waitForLedgerMissingReplicas(Long ledgerId, long secondsToWait, throws Exception { for (int i = 0; i < secondsToWait; i++) { try { - UnderreplicatedLedgerFormat data = urLedgerMgr.getLedgerUnreplicationInfo(ledgerId); + UnderreplicatedLedger data = urLedgerMgr.getLedgerUnreplicationInfo(ledgerId); boolean all = true; for (String r : replicas) { all = all && data.getReplicaList().contains(r); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index dc343eb9344..3999c3c7b52 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -550,6 +550,110 @@ void validateInitialDelayOfPlacementPolicyCheck(LedgerUnderreplicationManager ur auditor.close(); } + @Test + public void testInitialDelayOfReplicasCheck() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + LedgerHandle lh = bkc.createLedger(3, 2, DigestType.CRC32, "passwd".getBytes()); + for (int j = 0; j < 5; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + + long ledgerId = 100000L; + lh = bkc.createLedgerAdv(ledgerId, 3, 2, 2, DigestType.CRC32, "passwd".getBytes(), null); + lh.close(); + + ledgerId = 100001234L; + lh = bkc.createLedgerAdv(ledgerId, 3, 3, 2, DigestType.CRC32, "passwd".getBytes(), null); + for (int j = 0; j < 4; j++) { + lh.addEntry(j, "testdata".getBytes()); + } + lh.close(); + + ledgerId = 991234L; + lh = bkc.createLedgerAdv(ledgerId, 3, 2, 2, DigestType.CRC32, "passwd".getBytes(), null); + lh.addEntry(0, "testdata".getBytes()); + lh.close(); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(bsConfs.get(0)); + validateInitialDelayOfReplicasCheck(urm, -1, 1000, servConf, bkc); + validateInitialDelayOfReplicasCheck(urm, 999, 1000, servConf, bkc); + validateInitialDelayOfReplicasCheck(urm, 1001, 1000, servConf, bkc); + } + + void validateInitialDelayOfReplicasCheck(LedgerUnderreplicationManager urm, long timeSinceLastExecutedInSecs, + long auditorPeriodicReplicasCheckInterval, ServerConfiguration servConf, BookKeeper bkc) + throws UnavailableException, UnknownHostException, InterruptedException { + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + TestOpStatsLogger replicasCheckStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME); + servConf.setAuditorPeriodicReplicasCheckInterval(auditorPeriodicReplicasCheckInterval); + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(0); + final TestAuditor auditor = new TestAuditor(Bookie.getBookieAddress(servConf).toString(), servConf, bkc, false, + statsLogger); + CountDownLatch latch = auditor.getLatch(); + assertEquals("REPLICAS_CHECK_TIME SuccessCount", 0, replicasCheckStatsLogger.getSuccessCount()); + long curTimeBeforeStart = System.currentTimeMillis(); + long replicasCheckCTime = -1; + long initialDelayInMsecs = -1; + long nextExpectedReplicasCheckExecutionTime = -1; + long bufferTimeInMsecs = 20000L; + if (timeSinceLastExecutedInSecs == -1) { + /* + * if we are setting replicasCheckCTime to -1, it means that + * replicasCheck hasn't run before. So initialDelay for + * replicasCheck should be 0. + */ + replicasCheckCTime = -1; + initialDelayInMsecs = 0; + } else { + replicasCheckCTime = curTimeBeforeStart - timeSinceLastExecutedInSecs * 1000L; + initialDelayInMsecs = timeSinceLastExecutedInSecs > auditorPeriodicReplicasCheckInterval ? 0 + : (auditorPeriodicReplicasCheckInterval - timeSinceLastExecutedInSecs) * 1000L; + } + /* + * next replicasCheck should happen atleast after + * nextExpectedReplicasCheckExecutionTime. + */ + nextExpectedReplicasCheckExecutionTime = curTimeBeforeStart + initialDelayInMsecs; + + urm.setReplicasCheckCTime(replicasCheckCTime); + auditor.start(); + /* + * since auditorPeriodicReplicasCheckInterval are higher values (in the + * order of 100s of seconds), its ok bufferTimeInMsecs to be ` 20 secs. + */ + assertTrue("replicasCheck should have executed with initialDelay " + initialDelayInMsecs, + latch.await(initialDelayInMsecs + bufferTimeInMsecs, TimeUnit.MILLISECONDS)); + for (int i = 0; i < 20; i++) { + Thread.sleep(100); + if (replicasCheckStatsLogger.getSuccessCount() >= 1) { + break; + } + } + assertEquals("REPLICAS_CHECK_TIME SuccessCount", 1, replicasCheckStatsLogger.getSuccessCount()); + long currentReplicasCheckCTime = urm.getReplicasCheckCTime(); + assertTrue( + "currentReplicasCheckCTime: " + currentReplicasCheckCTime + + " should be greater than nextExpectedReplicasCheckExecutionTime: " + + nextExpectedReplicasCheckExecutionTime, + currentReplicasCheckCTime > nextExpectedReplicasCheckExecutionTime); + assertTrue( + "currentReplicasCheckCTime: " + currentReplicasCheckCTime + + " should be lesser than nextExpectedReplicasCheckExecutionTime+bufferTimeInMsecs: " + + (nextExpectedReplicasCheckExecutionTime + bufferTimeInMsecs), + currentReplicasCheckCTime < (nextExpectedReplicasCheckExecutionTime + bufferTimeInMsecs)); + auditor.close(); + } + static class TestAuditor extends Auditor { final AtomicReference latchRef = new AtomicReference(new CountDownLatch(1)); @@ -574,6 +678,11 @@ void placementPolicyCheck() throws BKAuditException { latchRef.get().countDown(); } + void replicasCheck() throws BKAuditException { + super.replicasCheck(); + latchRef.get().countDown(); + } + CountDownLatch getLatch() { return latchRef.get(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java index 1d786f9fdc2..b8548e048c1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -288,6 +288,136 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPlacementPolicy() th } } + @Test + public void testPlacementPolicyCheckForURLedgersElapsedRecoveryGracePeriod() throws Exception { + testPlacementPolicyCheckWithURLedgers(true); + } + + @Test + public void testPlacementPolicyCheckForURLedgersNotElapsedRecoveryGracePeriod() throws Exception { + testPlacementPolicyCheckWithURLedgers(false); + } + + public void testPlacementPolicyCheckWithURLedgers(boolean timeElapsed) throws Exception { + int numOfBookies = 4; + /* + * in timeElapsed=true scenario, set some low value, otherwise set some + * highValue. + */ + int underreplicatedLedgerRecoveryGracePeriod = timeElapsed ? 1 : 1000; + int numOfURLedgersElapsedRecoveryGracePeriod = 0; + List bookieAddresses = new ArrayList(); + RegistrationManager regManager = driver.getRegistrationManager(); + + for (int i = 0; i < numOfBookies; i++) { + BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); + bookieAddresses.add(bookieAddress); + regManager.registerBookie(bookieAddress.toString(), false); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + LedgerUnderreplicationManager underreplicationManager = mFactory.newLedgerUnderreplicationManager(); + int ensembleSize = 4; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + + long ledgerId1 = 1L; + LedgerMetadata initMeta = LedgerMetadataBuilder.create() + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(ledgerId1, initMeta).get(); + underreplicationManager.markLedgerUnderreplicated(ledgerId1, bookieAddresses.get(0).toString()); + if (timeElapsed) { + numOfURLedgersElapsedRecoveryGracePeriod++; + } + + /* + * this is non-closed ledger, it should also be reported as + * URLedgersElapsedRecoveryGracePeriod + */ + ensembleSize = 3; + long ledgerId2 = 21234561L; + initMeta = LedgerMetadataBuilder.create() + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, + Arrays.asList(bookieAddresses.get(0), bookieAddresses.get(1), bookieAddresses.get(2))) + .newEnsembleEntry(100L, + Arrays.asList(bookieAddresses.get(3), bookieAddresses.get(1), bookieAddresses.get(2))) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(ledgerId2, initMeta).get(); + underreplicationManager.markLedgerUnderreplicated(ledgerId2, bookieAddresses.get(0).toString()); + if (timeElapsed) { + numOfURLedgersElapsedRecoveryGracePeriod++; + } + + /* + * this ledger is not marked underreplicated. + */ + long ledgerId3 = 31234561L; + initMeta = LedgerMetadataBuilder.create() + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, + Arrays.asList(bookieAddresses.get(1), bookieAddresses.get(2), bookieAddresses.get(3))) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(ledgerId3, initMeta).get(); + + if (timeElapsed) { + /* + * in timeelapsed scenario, by waiting for + * underreplicatedLedgerRecoveryGracePeriod, recovery time must be + * elapsed. + */ + Thread.sleep((underreplicatedLedgerRecoveryGracePeriod + 1) * 1000); + } else { + /* + * in timeElapsed=false scenario, since + * underreplicatedLedgerRecoveryGracePeriod is set to some high + * value, there is no value in waiting. So just wait for some time + * and make sure urledgers are not reported as recoverytime elapsed + * urledgers. + */ + Thread.sleep(5000); + } + + ServerConfiguration servConf = new ServerConfiguration(bsConfs.get(0)); + servConf.setUnderreplicatedLedgerRecoveryGracePeriod(underreplicatedLedgerRecoveryGracePeriod); + setServerConfigPropertiesForRackPlacement(servConf); + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge underreplicatedLedgersElapsedRecoveryGracePeriodGuage = statsLogger + .getGauge(ReplicationStats.NUM_UNDERREPLICATED_LEDGERS_ELAPSED_RECOVERY_GRACE_PERIOD); + assertEquals("NUM_UNDERREPLICATED_LEDGERS_ELAPSED_RECOVERY_GRACE_PERIOD guage value", + numOfURLedgersElapsedRecoveryGracePeriod, + underreplicatedLedgersElapsedRecoveryGracePeriodGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + } + } + @Test public void testPlacementPolicyCheckWithLedgersNotAdheringToPolicyWithMultipleSegments() throws Exception { int numOfBookies = 7; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java index 2245d3f0844..3403d7f4d61 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java @@ -795,6 +795,21 @@ public void testPlacementPolicyCheckCTime() throws Exception { assertEquals(curTime, underReplicaMgr1.getPlacementPolicyCheckCTime()); } + @Test + public void testReplicasCheckCTime() throws Exception { + @Cleanup + LedgerUnderreplicationManager underReplicaMgr1 = lmf1.newLedgerUnderreplicationManager(); + @Cleanup + LedgerUnderreplicationManager underReplicaMgr2 = lmf2.newLedgerUnderreplicationManager(); + assertEquals(-1, underReplicaMgr1.getReplicasCheckCTime()); + long curTime = System.currentTimeMillis(); + underReplicaMgr2.setReplicasCheckCTime(curTime); + assertEquals(curTime, underReplicaMgr1.getReplicasCheckCTime()); + curTime = System.currentTimeMillis(); + underReplicaMgr2.setReplicasCheckCTime(curTime); + assertEquals(curTime, underReplicaMgr1.getReplicasCheckCTime()); + } + private void verifyMarkLedgerUnderreplicated(Collection missingReplica) throws KeeperException, InterruptedException, ReplicationException { Long ledgerA = 0xfeadeefdacL; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedgerTest.java index 4368f897b68..1f6e62a58b5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedgerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedgerTest.java @@ -24,8 +24,11 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.util.ArrayList; import java.util.Arrays; +import java.util.BitSet; import java.util.HashSet; +import java.util.List; import java.util.PrimitiveIterator; import java.util.Set; @@ -189,4 +192,111 @@ public void testNonExistingEntries() { } } } + + @Test + public void testGetUnavailableEntries() { + /* + * AvailabilityOfEntriesOfLedger is going to be created with this + * entries. It is equivalent to considering that Bookie has these + * entries. + */ + long[][] availableEntries = { + { 1, 2}, + { 0, 1, 2 }, + { 1, 2, 3, 5, 6, 7, 8 }, + { 1, 5 }, + { 3 }, + { 1, 2, 4, 5, 7, 8 }, + {}, + { 1, 2, 3, 5, 6, 11, 12, 13, 14, 15, 16, 17, 100, 1000, 1001, 10000, 20000, 20001 } + }; + + /* + * getUnavailableEntries method is going to be called with these entries + * as expected to contain. + */ + long[][] expectedToContainEntries = { + { 1, 2}, + { 0, 1, 2, 3, 5 }, + { 1, 2, 5, 7, 8 }, + { 2, 7 }, + { 3 }, + { 1, 5, 7, 8, 9, 10 }, + { 0, 1, 2, 3, 4, 5 }, + { 4, 18, 1002, 19999, 20003 } + }; + + /* + * Considering what AvailabilityOfEntriesOfLedger contains + * (availableEntries), what it is expected to contain + * (expectedToContainEntries), following are the entries which are + * supposed to be reported as unavailable (unavailableEntries). + */ + long[][] unavailableEntries = { + { }, + { 3, 5 }, + { }, + { 2, 7 }, + { }, + { 9, 10 }, + { 0, 1, 2, 3, 4, 5 }, + { 4, 18, 1002, 19999, 20003 } + }; + + for (int i = 0; i < availableEntries.length; i++) { + long[] availableEntriesTempArray = availableEntries[i]; + long[] expectedToContainEntriesTempArray = expectedToContainEntries[i]; + long[] unavailableEntriesTempArray = unavailableEntries[i]; + List unavailableEntriesTempList = new ArrayList(); + for (int j = 0; j < unavailableEntriesTempArray.length; j++) { + unavailableEntriesTempList.add(unavailableEntriesTempArray[j]); + } + + PrimitiveIterator.OfLong primitiveIterator = Arrays.stream(availableEntriesTempArray).iterator(); + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = new AvailabilityOfEntriesOfLedger( + primitiveIterator); + + long startEntryId; + long lastEntryId; + if (expectedToContainEntriesTempArray[0] == 0) { + startEntryId = expectedToContainEntriesTempArray[0]; + lastEntryId = expectedToContainEntriesTempArray[expectedToContainEntriesTempArray.length - 1]; + } else { + startEntryId = expectedToContainEntriesTempArray[0] - 1; + lastEntryId = expectedToContainEntriesTempArray[expectedToContainEntriesTempArray.length - 1] + 1; + } + BitSet expectedToContainEntriesBitSet = new BitSet((int) (lastEntryId - startEntryId + 1)); + for (int ind = 0; ind < expectedToContainEntriesTempArray.length; ind++) { + int entryId = (int) expectedToContainEntriesTempArray[ind]; + expectedToContainEntriesBitSet.set(entryId - (int) startEntryId); + } + + List actualUnavailableEntries = availabilityOfEntriesOfLedger.getUnavailableEntries(startEntryId, + lastEntryId, expectedToContainEntriesBitSet); + assertEquals("Unavailable Entries", unavailableEntriesTempList, actualUnavailableEntries); + } + } + + @Test + public void testEmptyAvailabilityOfEntriesOfLedger() { + AvailabilityOfEntriesOfLedger emptyOne = AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER; + assertEquals("expected totalNumOfAvailableEntries", 0, emptyOne.getTotalNumOfAvailableEntries()); + assertFalse("empty one is not supposed to contain any entry", emptyOne.isEntryAvailable(100L)); + long startEntryId = 100; + long lastEntryId = 105; + BitSet bitSetOfAvailability = new BitSet((int) (lastEntryId - startEntryId + 1)); + for (int i = 0; i < bitSetOfAvailability.length(); i++) { + if ((i % 2) == 0) { + bitSetOfAvailability.set(i); + } + } + List unavailableEntries = emptyOne.getUnavailableEntries(startEntryId, lastEntryId, bitSetOfAvailability); + assertEquals("Num of unavailable entries", bitSetOfAvailability.cardinality(), unavailableEntries.size()); + for (int i = 0; i < bitSetOfAvailability.length(); i++) { + long entryId = startEntryId + i; + if (bitSetOfAvailability.get(i)) { + assertTrue("Unavailable entry", unavailableEntries.contains(Long.valueOf(entryId))); + } + } + } } diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index 081428e2caf..d83308e70b8 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -684,6 +684,12 @@ groups: - param: storeSystemTimeAsLedgerUnderreplicatedMarkTime description: Enable the Auditor to use system time as underreplicated ledger mark time. If this is enabled, Auditor will write a ctime field into the underreplicated ledger znode. default: true + - param: underreplicatedLedgerRecoveryGracePeriod + description: The grace period (in seconds) for underreplicated ledgers recovery. If ledger is marked underreplicated for more than this period then it will be reported by placementPolicyCheck in Auditor. Setting this to 0 will disable this check. + default: 0 + - param: auditorReplicasCheckInterval + description: Sets the regularity/interval at which the auditor will run a replicas check of all ledgers, which are closed. This should not be run very often since it validates availability of replicas of all ledgers by querying bookies. Setting this to 0 will completely disable the periodic replicas check. By default it is disabled. + default: 0 - name: AutoRecovery replication worker settings params: From c6df4b8bd22a5ee21c94580acff9d90ebb272dd8 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sat, 27 Jul 2019 14:47:07 +0200 Subject: [PATCH 0322/1642] Make build pass on JDK11, fix javadoc configuration Descriptions of the changes in this PR: ### Motivation Build fails on recent JDK11 versions and on JDK11, see #2133, this is because javadoc has problems with references to core Java API when source version is set to 8. ### Changes Upgrade Java doc plugin to latest 3.1.1 and disable auto linking to core Java classes with `false ` Closes #2133 Reviewers: Sijie Guo This closes #2134 from eolivelli/fix/javadoc-jdk11 --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index fddb3badc83..4e496b2f660 100644 --- a/pom.xml +++ b/pom.xml @@ -188,7 +188,7 @@ 2.7 2.5.1 2.4 - 3.0.0 + 3.1.1 3.1.0 2.2.1 2.21.0 @@ -904,6 +904,7 @@ BookKeeper Java API (version ${project.version}) site/_site/overview/index.html package + false From d458998d1fef6ec6e66e31a57418dc3e0cb047c7 Mon Sep 17 00:00:00 2001 From: Pavan Kumar Chaitanya L Date: Sat, 27 Jul 2019 05:48:56 -0700 Subject: [PATCH 0323/1642] Issue #2130 : Fix for broken URLs Fixed broken URLs in README for Programmer Guide, Tutorial and Java API. Descriptions of the changes in this PR: Updated the URLs. ### Motivation Wanted to go through the tutorial when I came to the repo and found the links were broken in the README. ### Changes Master Issue: #2130 Reviewers: Enrico Olivelli , Sijie Guo This closes #2131 from pavankumarchaitanya/master, closes #2130 --- README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index aeeac68c187..b8d55c9c8ad 100644 --- a/README.md +++ b/README.md @@ -26,9 +26,9 @@ It is suitable for being used in following scenarios: ### Developers -* [Programmer Guide](http://bookkeeper.apache.org/docs/master/bookkeeperProgrammer.html) -* [Tutorial](http://bookkeeper.apache.org/docs/master/bookkeeperTutorial.html) -* [Java API](http://bookkeeper.apache.org/docs/master/apidocs/) +* [Programmer Guide](https://bookkeeper.apache.org/archives/docs/master/bookkeeperProgrammer.html) +* [Tutorial](https://bookkeeper.apache.org/archives/docs/master/bookkeeperTutorial.html) +* [Java API](https://bookkeeper.apache.org/archives/docs/master/apidocs/) You can also read [Turning Ledgers into Logs](http://bookkeeper.apache.org/docs/master/bookkeeperLedgers2Logs.html) to learn how to turn **ledgers** into continuous **log streams**. If you are looking for a high level **log stream** API, you can checkout [DistributedLog](http://distributedlog.io). From 58ae7908fc89516357352b2e0e915cbaa7bf4805 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 29 Jul 2019 08:08:44 +0200 Subject: [PATCH 0324/1642] Upgrade to latest Maven version 3.6.1 ### Motivation - ASF CI does not support Maven 3.5.0 - Maven 3.5.0 is a very old version ### Changes - update Jenkins DSL files to "Maven (latest)" - update release procedure to Maven 3.6.1 - update docker based dev environment to use Maven 3.6.1 + jdk11 (it was 3.5.0 + legacy jdk9) Reviewers: Sijie Guo This closes #2135 from eolivelli/fix/upgrade-maven --- .test-infra/jenkins/common_job_properties.groovy | 2 +- dev/docker/Dockerfile | 2 +- dev/docker/run.sh | 4 ++-- dev/release/Dockerfile | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.test-infra/jenkins/common_job_properties.groovy b/.test-infra/jenkins/common_job_properties.groovy index acb615f32f3..54642708191 100644 --- a/.test-infra/jenkins/common_job_properties.groovy +++ b/.test-infra/jenkins/common_job_properties.groovy @@ -179,7 +179,7 @@ class common_job_properties { } // Sets common config for Maven jobs. - static void setMavenConfig(context, mavenInstallation='Maven 3.5.0', mavenOpts='-Xmx4096m -Xms2048m') { + static void setMavenConfig(context, mavenInstallation='Maven 3.6.0', mavenOpts='-Xmx4096m -Xms2048m') { context.mavenInstallation(mavenInstallation) context.mavenOpts('-Dorg.slf4j.simpleLogger.showDateTime=true') context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\\\'T\\\'HH:mm:ss.SSS') diff --git a/dev/docker/Dockerfile b/dev/docker/Dockerfile index 228c23449c8..445f569a8f5 100644 --- a/dev/docker/Dockerfile +++ b/dev/docker/Dockerfile @@ -17,7 +17,7 @@ # under the License. # -FROM maven:3.5.0-jdk-9 +FROM maven:3.6.1-jdk-11 RUN apt-get update RUN apt-get install -y g++ cmake diff --git a/dev/docker/run.sh b/dev/docker/run.sh index 235cba786a8..b4b72a7b036 100755 --- a/dev/docker/run.sh +++ b/dev/docker/run.sh @@ -31,7 +31,7 @@ if [ "$(uname -s)" == "Linux" ]; then USER_NAME=${SUDO_USER:=$USER} USER_ID=$(id -u "${USER_NAME}") GROUP_ID=$(id -g "${USER_NAME}") - LOCAL_HOME="/home/${USER_NAME}" + LOCAL_HOME=$(realpath ~) else # boot2docker uid and gid USER_NAME=$USER USER_ID=1000 @@ -63,7 +63,7 @@ docker run -i -t \ --rm=true \ -w ${BOOKKEEPER_ROOT} \ -u "${USER}" \ - -v "${BOOKKEEPER_ROOT}:${BOOKKEEPER_ROOT}" \ + -v "$(realpath $BOOKKEEPER_ROOT):${BOOKKEEPER_ROOT}" \ -v "${LOCAL_HOME}:/home/${USER_NAME}" \ ${IMAGE_NAME}-${USER_NAME} \ bash -c "${CMD}" diff --git a/dev/release/Dockerfile b/dev/release/Dockerfile index b99e262552f..e09ecb038cb 100644 --- a/dev/release/Dockerfile +++ b/dev/release/Dockerfile @@ -17,7 +17,7 @@ # under the License. # -FROM maven:3.5.0-jdk-8 +FROM maven:3.6.1-jdk-8 RUN apt-get update RUN apt-get install -y g++ cmake gnupg2 vim subversion From 39a9d26e722114226d1550311a25bea9a4e7b52f Mon Sep 17 00:00:00 2001 From: BINLEI XUE Date: Mon, 29 Jul 2019 14:11:14 +0800 Subject: [PATCH 0325/1642] Issue #2127: Allow user override default SASL service name bookkeeper Descriptions of the changes in this PR: default SASL service name "bookkeeper" can be override by JVM property "bookkeeper.sasl.servicename" ### Motivation ### Changes Instead of use a constant value, it would read from JVM property first, if it doesn't exists, then use default value from constant variable SaslConstants.SASL_BOOKKEEPER_PROTOCOL Master Issue: #2127 Reviewers: Enrico Olivelli , Jia Zhai , Sijie Guo This closes #2128 from 29x10/master, closes #2127 --- .../java/org/apache/bookkeeper/sasl/SaslClientState.java | 4 +++- .../java/org/apache/bookkeeper/sasl/SaslConstants.java | 2 ++ .../org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java | 7 +++++-- site/docs/4.9.2/security/sasl.md | 7 +++++++ 4 files changed, 17 insertions(+), 3 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslClientState.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslClientState.java index 324480c4b49..63d7de9b7f6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslClientState.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslClientState.java @@ -50,7 +50,9 @@ public class SaslClientState { private String password; public SaslClientState(String serverHostname, Subject subject) throws SaslException { - String serverPrincipal = SaslConstants.SASL_BOOKKEEPER_PROTOCOL + "/" + serverHostname; + String saslServiceName = System.getProperty(SaslConstants.SASL_SERVICE_NAME, + SaslConstants.SASL_SERVICE_NAME_DEFAULT); + String serverPrincipal = saslServiceName + "/" + serverHostname; this.clientSubject = subject; if (clientSubject == null) { throw new SaslException("Cannot create JAAS Sujbect for SASL"); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslConstants.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslConstants.java index 98a83b7b941..719f1ec358b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslConstants.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslConstants.java @@ -52,6 +52,8 @@ public class SaslConstants { static final String SASL_BOOKKEEPER_PROTOCOL = "bookkeeper"; static final String SASL_BOOKKEEPER_REALM = "bookkeeper"; + static final String SASL_SERVICE_NAME = "bookkeeper.sasl.servicename"; + static final String SASL_SERVICE_NAME_DEFAULT = "bookkeeper"; static final String SASL_MD5_DUMMY_HOSTNAME = "bookkeeper"; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java index a538950f777..fc2df2088b8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java @@ -68,6 +68,8 @@ public class GSSAPIBookKeeperTest extends BookKeeperClusterTestCase { private static MiniKdc kdc; private static Properties conf; + private static final String non_default_sasl_service_name = "non_default_servicename"; + @ClassRule public static TemporaryFolder kdcDir = new TemporaryFolder(); @@ -86,8 +88,8 @@ public static void startMiniKdc() throws Exception { bookieConf.setUseHostNameAsBookieID(true); String localhostName = Bookie.getBookieAddress(bookieConf).getHostName(); - String principalServerNoRealm = "bookkeeper/" + localhostName; - String principalServer = "bookkeeper/" + localhostName + "@" + kdc.getRealm(); + String principalServerNoRealm = non_default_sasl_service_name + "/" + localhostName; + String principalServer = non_default_sasl_service_name + "/" + localhostName + "@" + kdc.getRealm(); LOG.info("principalServer: " + principalServer); String principalClientNoRealm = "bookkeeperclient/" + localhostName; String principalClient = principalClientNoRealm + "@" + kdc.getRealm(); @@ -252,6 +254,7 @@ public void testNotAllowedClientId() throws Exception { } BookieServer startAndStoreBookie(ServerConfiguration conf) throws Exception { + System.setProperty(SaslConstants.SASL_SERVICE_NAME, non_default_sasl_service_name); bsConfs.add(conf); BookieServer s = startBookie(conf); bs.add(s); diff --git a/site/docs/4.9.2/security/sasl.md b/site/docs/4.9.2/security/sasl.md index ffb972a8936..e943ec82f51 100644 --- a/site/docs/4.9.2/security/sasl.md +++ b/site/docs/4.9.2/security/sasl.md @@ -195,6 +195,13 @@ To configure SASL authentication on the clients: ```shell clientAuthProviderFactoryClass=org.apache.bookkeeper.sasl.SASLClientProviderFactory ``` +5. By default bookie service name is `bookkeeper`, you could override it by passing a JVM parameter to the client JVM or set System Property manually. + + For example, if your bookie's principle is bk@bk1.hostname.com@EXAMPLE.COM, then pass: + + ```shell + -Dbookkeeper.sasl.servicename=bk + ``` ## Enabling Logging for SASL From 5ba0cbb76c167a45730fe2e425dfca25cf58f6bc Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 29 Jul 2019 13:49:00 +0200 Subject: [PATCH 0326/1642] Make default Bookie scripts work on JDK11+ - detect new Java versions in bin/common.sh - use different defaults on JDK8 vs JDK11+ Descriptions of the changes in this PR: Change distribution scripts in order to detect a JDK newer then JDK8 and set default JVM options accordingly. ### Motivation Because I want that BookKeeper tools run out of the box with JDK11 ### Changes 1) detect a JDK newer then JDK8, but checking if exists $JAVA_HOME/bin/jshell (this is quite robust, better then parsing some Java version string) 2) use new defaults for JDK11, in particular do not enable experimental and deprecated options, use the new [Java Unified Logging](https://openjdk.java.net/jeps/158) log system for GC Please note the output of "Java Unified Logging" is very different from the old pre-Java9 one and options are different, there is no simple port. Master Issue: #1912 Reviewers: Sijie Guo This closes #2132 from eolivelli/fix/run-jdk11 --- bin/common.sh | 50 +++++++++++++++++++++++++++++++++++++++++--------- 1 file changed, 41 insertions(+), 9 deletions(-) diff --git a/bin/common.sh b/bin/common.sh index f92c0b9629b..d3b16d8914f 100755 --- a/bin/common.sh +++ b/bin/common.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# #/** # * Licensed to the Apache Software Foundation (ASF) under one # * or more contributor license agreements. See the NOTICE file @@ -71,13 +70,31 @@ source ${BK_CONFDIR}/nettyenv.sh source ${BK_CONFDIR}/bkenv.sh source ${BK_CONFDIR}/bk_cli_env.sh +detect_jdk8() { + + if [ -f "$JAVA_HOME/bin/jshell" ]; then + echo "0" + else + echo "1" + fi + return +} + # default netty settings NETTY_LEAK_DETECTION_LEVEL=${NETTY_LEAK_DETECTION_LEVEL:-"disabled"} NETTY_RECYCLER_MAXCAPACITY=${NETTY_RECYCLER_MAXCAPACITY:-"1000"} NETTY_RECYCLER_LINKCAPACITY=${NETTY_RECYCLER_LINKCAPACITY:-"1024"} -# default bookie JVM settings -DEFAULT_BOOKIE_GC_OPTS="-XX:+UseG1GC \ +USING_JDK8=$(detect_jdk8) + +if [ "$USING_JDK8" -ne "1" ]; then + DEFAULT_BOOKIE_GC_OPTS="-XX:+UseG1GC \ + -XX:MaxGCPauseMillis=10 \ + -XX:+ParallelRefProcEnabled \ + -XX:+DisableExplicitGC" + DEFAULT_BOOKIE_GC_LOGGING_OPTS="" +else + DEFAULT_BOOKIE_GC_OPTS="-XX:+UseG1GC \ -XX:MaxGCPauseMillis=10 \ -XX:+ParallelRefProcEnabled \ -XX:+UnlockExperimentalVMOptions \ @@ -88,11 +105,13 @@ DEFAULT_BOOKIE_GC_OPTS="-XX:+UseG1GC \ -XX:G1NewSizePercent=50 \ -XX:+DisableExplicitGC \ -XX:-ResizePLAB" -DEFAULT_BOOKIE_GC_LOGGING_OPTS="-XX:+PrintGCDetails \ + DEFAULT_BOOKIE_GC_LOGGING_OPTS="-XX:+PrintGCDetails \ -XX:+PrintGCApplicationStoppedTime \ -XX:+UseGCLogFileRotation \ -XX:NumberOfGCLogFiles=5 \ -XX:GCLogFileSize=64m" +fi + BOOKIE_MAX_HEAP_MEMORY=${BOOKIE_MAX_HEAP_MEMORY:-"1g"} BOOKIE_MIN_HEAP_MEMORY=${BOOKIE_MIN_HEAP_MEMORY:-"1g"} BOOKIE_MAX_DIRECT_MEMORY=${BOOKIE_MAX_DIRECT_MEMORY:-"2g"} @@ -103,11 +122,16 @@ BOOKIE_GC_LOGGING_OPTS=${BOOKIE_GC_LOGGING_OPTS:-"${DEFAULT_BOOKIE_GC_LOGGING_OP # default CLI JVM settings DEFAULT_CLI_GC_OPTS="-XX:+UseG1GC \ -XX:MaxGCPauseMillis=10" -DEFAULT_CLI_GC_LOGGING_OPTS="-XX:+PrintGCDetails \ +if [ "$USING_JDK8" -ne "1" ]; then + DEFAULT_CLI_GC_LOGGING_OPTS="-XX:+PrintGCDetails \ -XX:+PrintGCApplicationStoppedTime \ -XX:+UseGCLogFileRotation \ -XX:NumberOfGCLogFiles=5 \ -XX:GCLogFileSize=64m" +else + DEFAULT_CLI_GC_LOGGING_OPTS="" +fi + CLI_MAX_HEAP_MEMORY=${CLI_MAX_HEAP_MEMORY:-"512M"} CLI_MIN_HEAP_MEMORY=${CLI_MIN_HEAP_MEMORY:-"256M"} CLI_MEM_OPTS=${CLI_MEM_OPTS:-"-Xms${CLI_MIN_HEAP_MEMORY} -Xmx${CLI_MAX_HEAP_MEMORY}"} @@ -241,15 +265,23 @@ set_module_classpath() { build_bookie_jvm_opts() { LOG_DIR=$1 GC_LOG_FILENAME=$2 - - echo "$BOOKIE_MEM_OPTS $BOOKIE_GC_OPTS $BOOKIE_GC_LOGGING_OPTS $BOOKIE_PERF_OPTS -Xloggc:${LOG_DIR}/${GC_LOG_FILENAME}" + if [ "$USING_JDK8" -eq "1" ]; then + echo "$BOOKIE_MEM_OPTS $BOOKIE_GC_OPTS $BOOKIE_GC_LOGGING_OPTS $BOOKIE_PERF_OPTS -Xloggc:${LOG_DIR}/${GC_LOG_FILENAME}" + else + echo "$BOOKIE_MEM_OPTS $BOOKIE_GC_OPTS $BOOKIE_GC_LOGGING_OPTS $BOOKIE_PERF_OPTS -Xlog:gc=info:file=${LOG_DIR}/${GC_LOG_FILENAME}::filecount=5,filesize=64m" + fi + return } build_cli_jvm_opts() { LOG_DIR=$1 GC_LOG_FILENAME=$2 - - echo "$CLI_MEM_OPTS $CLI_GC_OPTS $CLI_GC_LOGGING_OPTS -Xloggc:${LOG_DIR}/${GC_LOG_FILENAME}" + if [ "$USING_JDK8" -eq "1" ]; then + echo "$CLI_MEM_OPTS $CLI_GC_OPTS $CLI_GC_LOGGING_OPTS -Xloggc:${LOG_DIR}/${GC_LOG_FILENAME}" + else + echo "$CLI_MEM_OPTS $CLI_GC_OPTS $CLI_GC_LOGGING_OPTS -Xlog:gc=info:file=${LOG_DIR}/${GC_LOG_FILENAME}::filecount=5,filesize=64m" + fi + return } build_netty_opts() { From d42b589270439e0b12d09d57829bc6258880a615 Mon Sep 17 00:00:00 2001 From: vzhikserg Date: Mon, 29 Jul 2019 13:52:20 +0200 Subject: [PATCH 0327/1642] Remove redundant array creation ### Changes Remove redundant array creation in log statements to simplify the code. Reviewers: Enrico Olivelli , Sijie Guo This closes #2129 from vzhikserg/remove-redundant-array-creation --- .../bookkeeper/client/UpdateLedgerOp.java | 3 +- .../http/service/ListDiskFilesService.java | 12 ++-- .../TestRackawareEnsemblePlacementPolicy.java | 11 ++-- .../AppendOnlyStreamReader.java | 2 +- .../BKDistributedLogManager.java | 2 +- .../apache/distributedlog/BKLogHandler.java | 4 +- .../distributedlog/BKLogReadHandler.java | 3 +- .../distributedlog/BKLogSegmentWriter.java | 17 +++--- .../distributedlog/BKLogWriteHandler.java | 19 +++---- .../distributedlog/BookKeeperClient.java | 16 +++--- .../distributedlog/ReadAheadEntryReader.java | 10 ++-- .../org/apache/distributedlog/ReadUtils.java | 17 +++--- .../admin/DistributedLogAdmin.java | 3 +- .../distributedlog/auditor/DLAuditor.java | 7 +-- .../bk/SimpleLedgerAllocator.java | 4 +- .../impl/BKNamespaceDriver.java | 11 ++-- .../impl/ZKNamespaceWatcher.java | 2 +- .../impl/acl/ZKAccessControlManager.java | 2 +- .../FederatedZKLogMetadataStore.java | 6 +- .../logsegment/BKLogSegmentEntryStore.java | 3 +- .../metadata/ZKLogStreamMetadataStore.java | 2 +- .../distributedlog/lock/ZKSessionLock.java | 55 +++++++++---------- .../logsegment/PerStreamLogSegmentCache.java | 6 +- .../apache/distributedlog/DLMTestUtil.java | 6 +- .../distributedlog/TestAsyncReaderWriter.java | 2 +- .../TestDistributedLogBase.java | 2 +- .../org/apache/distributedlog/TestReader.java | 6 +- 27 files changed, 109 insertions(+), 124 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java index 268eff6ca1f..83143fb2ad1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java @@ -104,8 +104,7 @@ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final (metadata) -> { return metadata.getAllEnsembles().values().stream() .flatMap(Collection::stream) - .filter(b -> b.equals(oldBookieId)) - .count() > 0; + .anyMatch(b -> b.equals(oldBookieId)); }, (metadata) -> { return replaceBookieInEnsembles(metadata, oldBookieId, newBookieId); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListDiskFilesService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListDiskFilesService.java index 330ea7f0162..4e12bf8afcb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListDiskFilesService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListDiskFilesService.java @@ -92,9 +92,9 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { if (all || journal) { File[] journalDirs = conf.getJournalDirs(); List journalFiles = listFilesAndSort(journalDirs, "txn"); - StringBuffer files = new StringBuffer(); + StringBuilder files = new StringBuilder(); for (File journalFile : journalFiles) { - files.append(journalFile.getName() + "\t"); + files.append(journalFile.getName()).append("\t"); } output.put("journal files", files.toString()); } @@ -102,9 +102,9 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { if (all || entrylog) { File[] ledgerDirs = conf.getLedgerDirs(); List ledgerFiles = listFilesAndSort(ledgerDirs, "log"); - StringBuffer files = new StringBuffer(); + StringBuilder files = new StringBuilder(); for (File ledgerFile : ledgerFiles) { - files.append(ledgerFile.getName() + "\t"); + files.append(ledgerFile.getName()).append("\t"); } output.put("entrylog files", files.toString()); } @@ -112,9 +112,9 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { if (all || index) { File[] indexDirs = (conf.getIndexDirs() == null) ? conf.getLedgerDirs() : conf.getIndexDirs(); List indexFiles = listFilesAndSort(indexDirs, "idx"); - StringBuffer files = new StringBuffer(); + StringBuilder files = new StringBuilder(); for (File indexFile : indexFiles) { - files.append(indexFile.getName() + "\t"); + files.append(indexFile.getName()).append("\t"); } output.put("index files", files.toString()); } 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 6017ad92333..51e7ea6e7ce 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 @@ -23,13 +23,12 @@ import static org.apache.bookkeeper.feature.SettableFeatureProvider.DISABLE_ALL; import com.google.common.util.concurrent.ThreadFactoryBuilder; - import io.netty.util.HashedWheelTimer; - import java.net.InetAddress; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -37,9 +36,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; - import junit.framework.TestCase; - import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.EnsemblePlacementPolicy.PlacementPolicyAdherence; @@ -2130,7 +2127,8 @@ public void testNumBookiesInDefaultRackGauge() throws Exception { assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", 2, numBookiesInDefaultRackGauge.getSample()); // newAddr4 rack is changed and it is not in default anymore - StaticDNSResolver.changeRack(Arrays.asList(newAddr4), Arrays.asList("/default-region/r4")); + StaticDNSResolver + .changeRack(Collections.singletonList(newAddr4), Collections.singletonList("/default-region/r4")); assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", 1, numBookiesInDefaultRackGauge.getSample()); writeableBookies.clear(); @@ -2138,7 +2136,8 @@ public void testNumBookiesInDefaultRackGauge() throws Exception { repp.onClusterChanged(writeableBookies, readOnlyBookies); assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", 0, numBookiesInDefaultRackGauge.getSample()); - StaticDNSResolver.changeRack(Arrays.asList(newAddr1), Arrays.asList("/default-region/r2")); + StaticDNSResolver + .changeRack(Collections.singletonList(newAddr1), Collections.singletonList("/default-region/r2")); readOnlyBookies.clear(); writeableBookies.add(newAddr1); writeableBookies.add(newAddr2); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java index 357fc6c6835..47f1ed62415 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java @@ -47,7 +47,7 @@ private static class LogRecordWithInputStream { checkNotNull(logRecord); LOG.debug("Got record dlsn = {}, txid = {}, len = {}", - new Object[] {logRecord.getDlsn(), logRecord.getTransactionId(), logRecord.getPayload().length}); + logRecord.getDlsn(), logRecord.getTransactionId(), logRecord.getPayload().length); this.logRecord = logRecord; this.payloadStream = logRecord.getPayLoadInputStream(); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java index 43c0973c06b..8a79d8b404a 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java @@ -755,7 +755,7 @@ public CompletableFuture apply(Void complete) { return subscriptionsStore.getLastCommitPosition(subscriberId.get()) .thenCompose(lastCommitPosition -> { LOG.info("Reader {} @ {} positioned to last commit position {}.", - new Object[] { subscriberId.get(), name, lastCommitPosition }); + subscriberId.get(), name, lastCommitPosition); try { reader.setStartDLSN(lastCommitPosition); } catch (UnexpectedException e) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java index a319d44a781..75203cc4619 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java @@ -515,7 +515,7 @@ protected void addLogSegmentToCache(String name, LogSegmentMetadata metadata) { if (elapsedMicroSec > 0) { if (elapsedMillis > metadataLatencyWarnThresholdMillis) { LOG.warn("{} received inprogress log segment in {} millis: {}", - new Object[] { getFullyQualifiedName(), elapsedMillis, metadata }); + getFullyQualifiedName(), elapsedMillis, metadata); } getInprogressSegmentStat.registerSuccessfulEvent(elapsedMicroSec, TimeUnit.MICROSECONDS); } else { @@ -527,7 +527,7 @@ protected void addLogSegmentToCache(String name, LogSegmentMetadata metadata) { if (elapsedMicroSec > 0) { if (elapsedMillis > metadataLatencyWarnThresholdMillis) { LOG.warn("{} received completed log segment in {} millis : {}", - new Object[] { getFullyQualifiedName(), elapsedMillis, metadata }); + getFullyQualifiedName(), elapsedMillis, metadata); } getCompletedSegmentStat.registerSuccessfulEvent(elapsedMicroSec, TimeUnit.MICROSECONDS); } else { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java index 1175f39c93f..c865c28b5f7 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java @@ -205,8 +205,7 @@ public void onSuccess(DistributedLock lock) { @Override public void onFailure(Throwable cause) { - LOG.info("failed to acquire readlock {} at {}", - new Object[]{ getLockClientId(), getReadLockPath(), cause }); + LOG.info("failed to acquire readlock {} at {}", getLockClientId(), getReadLockPath(), cause); threadAcquirePromise.completeExceptionally(cause); } }); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java index 4ad977b2141..880393c9311 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java @@ -332,7 +332,7 @@ public Number getSample() { final int configuredTransmissionThreshold = dynConf.getOutputBufferSize(); if (configuredTransmissionThreshold > MAX_LOGRECORDSET_SIZE) { LOG.warn("Setting output buffer size {} greater than max transmission size {} for log segment {}", - new Object[] {configuredTransmissionThreshold, MAX_LOGRECORDSET_SIZE, fullyQualifiedLogSegment}); + configuredTransmissionThreshold, MAX_LOGRECORDSET_SIZE, fullyQualifiedLogSegment); this.transmissionThreshold = MAX_LOGRECORDSET_SIZE; } else { this.transmissionThreshold = configuredTransmissionThreshold; @@ -610,8 +610,8 @@ private void abortTransmitPacketOnClose(final boolean abort, final CompletableFuture closePromise) { LOG.info("Closing BKPerStreamLogWriter (abort={}) for {} :" + " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {}", - new Object[]{abort, fullyQualifiedLogSegment, getLastDLSN(), - outstandingTransmitsUpdater.get(this), getWritesPendingTransmit()}); + abort, fullyQualifiedLogSegment, getLastDLSN(), + outstandingTransmitsUpdater.get(this), getWritesPendingTransmit()); // Save the current packet to reset, leave a new empty packet to avoid a race with // addCompleteDeferredProcessing. @@ -832,7 +832,7 @@ public synchronized CompletableFuture writeInternal(LogRecord record) if (record.getTransactionId() < lastTxId) { LOG.info("Log Segment {} TxId decreased Last: {} Record: {}", - new Object[] {fullyQualifiedLogSegment, lastTxId, record.getTransactionId()}); + fullyQualifiedLogSegment, lastTxId, record.getTransactionId()); } if (!record.isControl()) { // only update last tx id for user records @@ -1187,7 +1187,7 @@ public void addComplete(final int rc, LedgerHandle handle, // Sanity check to make sure we're receiving these callbacks in order. if (entryId > -1 && lastEntryId >= entryId) { LOG.error("Log segment {} saw out of order entry {} lastEntryId {}", - new Object[] {fullyQualifiedLogSegment, entryId, lastEntryId}); + fullyQualifiedLogSegment, entryId, lastEntryId); } lastEntryId = entryId; @@ -1236,9 +1236,8 @@ public void onSuccess(Void done) { @Override public void onFailure(Throwable cause) { LOG.error("addComplete processing failed for {} entry {} lastTxId {} rc {} with error", - new Object[] { - fullyQualifiedLogSegment, entryId, - transmitPacket.getRecordSet().getMaxTxId(), rc, cause}); + fullyQualifiedLogSegment, entryId, + transmitPacket.getRecordSet().getMaxTxId(), rc, cause); } }); // Race condition if we notify before the addComplete is enqueued. @@ -1266,7 +1265,7 @@ private void addCompleteDeferredProcessing(final BKTransmitPacket transmitPacket cancelPendingPromises = (BKException.Code.OK != rc); } else { LOG.warn("Log segment {} entryId {}: Tried to set transmit result to ({}) but is already ({})", - new Object[] {fullyQualifiedLogSegment, entryId, rc, transmitResultUpdater.get(this)}); + fullyQualifiedLogSegment, entryId, rc, transmitResultUpdater.get(this)); } if (transmitResultUpdater.get(this) != BKException.Code.OK) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java index 3095ec75187..df830cba027 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java @@ -501,8 +501,8 @@ protected long assignLogSegmentSequenceNumber() throws IOException { // case 2 can occur when the writer crashes with an empty in progress ledger. This is then deleted // on recovery, so the next new segment will have a matching sequence number LOG.warn("Unexpected max log segment sequence number {} for {} : list of cached segments = {}", - new Object[]{maxLogSegmentSequenceNo.getSequenceNumber(), getFullyQualifiedName(), - getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)}); + maxLogSegmentSequenceNo.getSequenceNumber(), getFullyQualifiedName(), + getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)); // there is max log segment number recorded there and it isn't match. throw exception. throw new DLIllegalStateException("Unexpected max log segment sequence number " + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName() @@ -942,8 +942,7 @@ private void doCompleteAndCloseLogSegmentAfterLogSegmentListFetched( new Object[] { logSegmentSeqNo, inprogressLogSegment.getZkPath() }); } else { LOG.warn("Unexpected max ledger sequence number {} found while completing log segment {} for {}", - new Object[] { - maxLogSegmentSequenceNo.getSequenceNumber(), logSegmentSeqNo, getFullyQualifiedName() }); + maxLogSegmentSequenceNo.getSequenceNumber(), logSegmentSeqNo, getFullyQualifiedName()); if (validateLogSegmentSequenceNumber) { FutureUtils.completeExceptionally(promise, new DLIllegalStateException("Unexpected max log segment sequence number " @@ -990,8 +989,8 @@ private void doCompleteAndCloseLogSegmentAfterLogSegmentListFetched( @Override public void onSuccess(Void value) { LOG.info("Completed {} to {} for {} : {}", - new Object[] { inprogressZnodeName, completedLogSegment.getSegmentName(), - getFullyQualifiedName(), completedLogSegment }); + inprogressZnodeName, completedLogSegment.getSegmentName(), + getFullyQualifiedName(), completedLogSegment); FutureUtils.complete(promise, completedLogSegment); } @@ -1087,7 +1086,7 @@ CompletableFuture> setLogSegmentsOlderThanDLSNTruncated private CompletableFuture> setLogSegmentsOlderThanDLSNTruncated( List logSegments, final DLSN dlsn) { LOG.debug("Setting truncation status on logs older than {} from {} for {}", - new Object[]{dlsn, logSegments, getFullyQualifiedName()}); + dlsn, logSegments, getFullyQualifiedName()); List truncateList = new ArrayList(logSegments.size()); LogSegmentMetadata partialTruncate = null; LOG.info("{}: Truncating log segments older than {}", getFullyQualifiedName(), dlsn); @@ -1106,7 +1105,7 @@ private CompletableFuture> setLogSegmentsOlderThanDLSNT return FutureUtils.exception(new DLIllegalStateException(logMsg)); } LOG.info("{}: Partially truncating log segment {} older than {}.", - new Object[] {getFullyQualifiedName(), l, dlsn}); + getFullyQualifiedName(), l, dlsn); partialTruncate = l; } else { break; @@ -1163,7 +1162,7 @@ public CompletableFuture> apply(List asyncReadRecordFromEntries( final long endEntryId = context.curEndEntryId.get(); if (LOG.isDebugEnabled()) { LOG.debug("{} reading entries [{} - {}] from {}.", - new Object[] { streamName, startEntryId, endEntryId, metadata}); + streamName, startEntryId, endEntryId, metadata); } FutureEventListener> readEntriesListener = new FutureEventListener>() { @@ -259,7 +259,7 @@ private static CompletableFuture asyncReadRecordFromEntries( public void onSuccess(final List entries) { if (LOG.isDebugEnabled()) { LOG.debug("{} finished reading entries [{} - {}] from {}", - new Object[]{ streamName, startEntryId, endEntryId, metadata}); + streamName, startEntryId, endEntryId, metadata); } for (Entry.Reader entry : entries) { try { @@ -275,8 +275,7 @@ public void onSuccess(final List entries) { LogRecordWithDLSN record = selector.result(); if (LOG.isDebugEnabled()) { LOG.debug("{} got record from entries [{} - {}] of {} : {}", - new Object[]{streamName, startEntryId, endEntryId, - metadata, record}); + streamName, startEntryId, endEntryId, metadata, record); } promise.complete(record); } @@ -348,8 +347,8 @@ private static void asyncReadRecordFromEntries( public void onSuccess(LogRecordWithDLSN value) { if (LOG.isDebugEnabled()) { LOG.debug("{} read record from [{} - {}] of {} : {}", - new Object[]{streamName, context.curStartEntryId.get(), context.curEndEntryId.get(), - metadata, value}); + streamName, context.curStartEntryId.get(), context.curEndEntryId.get(), + metadata, value); } if (null != value) { promise.complete(value); @@ -431,13 +430,11 @@ private static CompletableFuture asyncReadRecord( @Override public void onSuccess(final LogSegmentRandomAccessEntryReader reader) { if (LOG.isDebugEnabled()) { - LOG.debug("{} Opened log segment {} for reading record", - streamName, l); + LOG.debug("{} Opened log segment {} for reading record", streamName, l); } promise.whenComplete((value, cause) -> reader.asyncClose()); if (LOG.isDebugEnabled()) { - LOG.debug("{} {} scanning {}.", new Object[]{ - (backward ? "backward" : "forward"), streamName, l}); + LOG.debug("{} {} scanning {}.", (backward ? "backward" : "forward"), streamName, l); } asyncReadRecordFromLogSegment( streamName, reader, l, executorService, diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java index 1776c3ef7ca..02bd41ab74d 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java @@ -138,8 +138,7 @@ public static void fixInprogressSegmentWithLowerSequenceNumber(final Namespace n final LogSegmentMetadata newSegment = FutureUtils.result(metadataUpdater.changeSequenceNumber(inprogressSegment, newLogSegmentSequenceNumber)); - LOG.info("Fixed {} : {} -> {} ", - new Object[] { streamName, inprogressSegment, newSegment }); + LOG.info("Fixed {} : {} -> {} ", streamName, inprogressSegment, newSegment); if (verbose) { System.out.println("Fixed " + streamName + " : " + inprogressSegment.getZNodeName() + " -> " + newSegment.getZNodeName()); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java index 655762c6de8..d97909ae49c 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java @@ -256,7 +256,7 @@ public void run() { collectLedgersFromAllocator(uri, namespace, aps, ledgers); synchronized (ledgers) { logger.info("Collected {} ledgers from allocators for {} : {} ", - new Object[]{ledgers.size(), uri, ledgers}); + ledgers.size(), uri, ledgers); } collectLedgersFromDL(uri, namespace, ledgers); } catch (IOException e) { @@ -360,8 +360,7 @@ private void collectLedgersFromDL(final URI uri, streamQueue.add(streams.next()); } - logger.info("Collected {} streams from uri {} : {}", - new Object[] { streamQueue.size(), uri, streams }); + logger.info("Collected {} streams from uri {} : {}", streamQueue.size(), uri, streams); executeAction(streamQueue, 10, new Action() { @Override @@ -561,7 +560,7 @@ public void processResult(int rc, String path, Object ctx) { try { doneFuture.get(); logger.info("calculated {} ledgers\n\ttotal bytes = {}\n\ttotal entries = {}", - new Object[] { numLedgers.get(), totalBytes.get(), totalEntries.get() }); + numLedgers.get(), totalBytes.get(), totalEntries.get()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new DLInterruptedException("Interrupted on calculating ledger space : ", e); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java index 015a9617273..0115a6ee782 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java @@ -399,7 +399,7 @@ public void onFailure(Throwable cause) { setPhase(Phase.ERROR); deleteLedger(lh.getId()); LOG.error("Fail mark ledger {} as allocated under {} : ", - new Object[] { lh.getId(), allocatePath, cause }); + lh.getId(), allocatePath, cause); // fail the allocation since failed to mark it as allocated failAllocation(cause); } @@ -414,7 +414,7 @@ void deleteLedger(final long ledgerId) { deleteFuture.whenComplete((value, cause) -> { if (null != cause) { LOG.error("Error deleting ledger {} for ledger allocator {}, retrying : ", - new Object[] { ledgerId, allocatePath, cause }); + ledgerId, allocatePath, cause); if (!isClosing()) { deleteLedger(ledgerId); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java index f64bae1173a..f31698640ba 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java @@ -217,7 +217,7 @@ public synchronized NamespaceDriver initialize(DistributedLogConfiguration conf, initialized = true; LOG.info("Initialized BK namespace driver: clientId = {}, regionId = {}, federated = {}.", - new Object[]{clientId, regionId, bkdlConfig.isFederatedNamespace()}); + clientId, regionId, bkdlConfig.isFederatedNamespace()); return this; } @@ -587,9 +587,10 @@ public static ZooKeeperClientBuilder createZKClientBuilder(String zkcName, .statsLogger(statsLogger) .zkAclId(conf.getZkAclId()); LOG.info("Created shared zooKeeper client builder {}: zkServers = {}, numRetries = {}, sessionTimeout = {}," - + " retryBackoff = {}, maxRetryBackoff = {}, zkAclId = {}.", new Object[] { - zkcName, zkServers, conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(), - conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(), conf.getZkAclId() }); + + " retryBackoff = {}, maxRetryBackoff = {}, zkAclId = {}.", zkcName, zkServers, + conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(), + conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(), + conf.getZkAclId()); return builder; } @@ -611,7 +612,7 @@ private BookKeeperClientBuilder createBKCBuilder(String bkcName, .featureProvider(featureProviderOptional) .statsLogger(statsLogger); LOG.info("Created shared client builder {} : zkServers = {}, ledgersPath = {}, numIOThreads = {}", - new Object[] { bkcName, zkServers, ledgersPath, conf.getBKClientNumberIOThreads() }); + bkcName, zkServers, ledgersPath, conf.getBKClientNumberIOThreads()); return builder; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java index 9d9783f4fdc..952f1b32317 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java @@ -68,7 +68,7 @@ private void scheduleTask(Runnable r, long ms) { try { scheduler.schedule(r, ms, TimeUnit.MILLISECONDS); } catch (RejectedExecutionException ree) { - logger.error("Task {} scheduled in {} ms is rejected : ", new Object[]{r, ms, ree}); + logger.error("Task {} scheduled in {} ms is rejected : ", r, ms, ree); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java index 7b775c41d37..bf11137734c 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java @@ -196,7 +196,7 @@ public void onFailure(Throwable cause) { streamEntries.remove(streamName); } else if (cause instanceof ZKAccessControl.CorruptedAccessControlException) { logger.warn("Access control is corrupted for stream {} @ {},skipped it ...", - new Object[] { streamName, zkRootPath, cause }); + streamName, zkRootPath, cause); streamEntries.remove(streamName); } else { if (1 == numFailures.incrementAndGet()) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java index d208aabd24d..8d458661dc8 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java @@ -150,7 +150,7 @@ public void onStreamsChanged(Iterator newLogsIter) { URI oldURI = log2Locations.putIfAbsent(logName, uri); if (null != oldURI && !Objects.equal(uri, oldURI)) { logger.error("Log {} is found duplicated in multiple locations : old location = {}," - + " new location = {}", new Object[] { logName, oldURI, uri }); + + " new location = {}", logName, oldURI, uri); duplicatedLogFound.set(true); } } @@ -232,7 +232,7 @@ private void scheduleTask(Runnable r, long ms) { try { scheduler.schedule(r, ms, TimeUnit.MILLISECONDS); } catch (RejectedExecutionException ree) { - logger.error("Task {} scheduled in {} ms is rejected : ", new Object[]{r, ms, ree}); + logger.error("Task {} scheduled in {} ms is rejected : ", r, ms, ree); } } @@ -668,7 +668,7 @@ public void onSuccess(List> fetchResults) { if (result.isPresent()) { if (fetchResult.isPresent()) { logger.error("Log {} is found in multiple sub namespaces : {} & {}.", - new Object[] { logName, result.get(), fetchResult.get() }); + logName, result.get(), fetchResult.get()); duplicatedLogName.compareAndSet(null, logName); duplicatedLogFound.set(true); fetchPromise.completeExceptionally(new UnexpectedException("Log " + logName diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java index b164a392455..66e99401ce0 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java @@ -140,8 +140,7 @@ public void deleteComplete(int rc, Object ctx) { deleteRequest.segment.getLogSegmentId(), deleteRequest.segment); } else if (BKException.Code.OK != rc) { logger.error("Couldn't delete ledger {} from bookkeeper for {} : {}", - new Object[]{ deleteRequest.segment.getLogSegmentId(), deleteRequest.segment, - BKException.getMessage(rc) }); + deleteRequest.segment.getLogSegmentId(), deleteRequest.segment, BKException.getMessage(rc)); FutureUtils.completeExceptionally(deleteRequest.deletePromise, new BKTransmitException("Couldn't delete log segment " + deleteRequest.segment, rc)); return; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java index 0e923c66132..869a968e232 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java @@ -585,7 +585,7 @@ static CompletableFuture getLog(final URI uri, try { PathUtils.validatePath(logRootPath); } catch (IllegalArgumentException e) { - LOG.error("Illegal path value {} for stream {}", new Object[]{logRootPath, logName, e}); + LOG.error("Illegal path value {} for stream {}", logRootPath, logName, e); return FutureUtils.exception(new InvalidStreamNameException(logName, "Log name is invalid")); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java index 84c516c344a..80e24d08cf0 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java @@ -203,8 +203,7 @@ static class StateManagement { public void transition(State toState) { if (!validTransition(toState)) { - LOG.error("Invalid state transition from {} to {} ", - new Object[] { this.state, toState, getStack() }); + LOG.error("Invalid state transition from {} to {} ", this.state, toState, getStack()); } this.state = toState; } @@ -400,19 +399,18 @@ public void safeRun() { if (getEpoch() == lockEpoch) { if (LOG.isTraceEnabled()) { LOG.trace("{} executing lock action '{}' under epoch {} for lock {}", - new Object[]{lockId, func.getActionName(), lockEpoch, lockPath}); + lockId, func.getActionName(), lockEpoch, lockPath); } func.execute(); if (LOG.isTraceEnabled()) { LOG.trace("{} executed lock action '{}' under epoch {} for lock {}", - new Object[]{lockId, func.getActionName(), lockEpoch, lockPath}); + lockId, func.getActionName(), lockEpoch, lockPath); } } else { if (LOG.isTraceEnabled()) { LOG.trace("{} skipped executing lock action '{}' for lock {}," + " since epoch is changed from {} to {}.", - new Object[]{lockId, func.getActionName(), - lockPath, lockEpoch, getEpoch()}); + lockId, func.getActionName(), lockPath, lockEpoch, getEpoch()); } } } @@ -440,18 +438,18 @@ public void safeRun() { if (currentEpoch == lockEpoch) { if (LOG.isTraceEnabled()) { LOG.trace("{} executed lock action '{}' under epoch {} for lock {}", - new Object[]{lockId, func.getActionName(), lockEpoch, lockPath}); + lockId, func.getActionName(), lockEpoch, lockPath); } func.execute(); if (LOG.isTraceEnabled()) { LOG.trace("{} executed lock action '{}' under epoch {} for lock {}", - new Object[]{lockId, func.getActionName(), lockEpoch, lockPath}); + lockId, func.getActionName(), lockEpoch, lockPath); } } else { if (LOG.isTraceEnabled()) { LOG.trace("{} skipped executing lock action '{}' for lock {}," + " since epoch is changed from {} to {}.", - new Object[]{lockId, func.getActionName(), lockPath, lockEpoch, currentEpoch}); + lockId, func.getActionName(), lockPath, lockEpoch, currentEpoch); } promise.completeExceptionally(new EpochChangedException(lockPath, lockEpoch, currentEpoch)); } @@ -699,13 +697,13 @@ public void onFailure(final Throwable lockCause) { // If tryLock failed due to state changed, we don't need to cleanup if (lockCause instanceof LockStateChangedException) { LOG.info("skipping cleanup for {} at {} after encountering lock " - + "state change exception : ", new Object[] { lockId, lockPath, lockCause }); + + "state change exception : ", lockId, lockPath, lockCause); result.completeExceptionally(lockCause); return; } if (LOG.isDebugEnabled()) { LOG.debug("{} is cleaning up its lock state for {} due to : ", - new Object[] { lockId, lockPath, lockCause }); + lockId, lockPath, lockCause); } // If we encountered any exception we should cleanup @@ -782,7 +780,7 @@ public void execute() { if (null != currentNode) { LOG.error("Current node for {} overwritten current = {} new = {}", - new Object[] { lockPath, lockId, getLockIdFromPath(currentNode) }); + lockPath, lockId, getLockIdFromPath(currentNode)); } currentNode = name; @@ -899,9 +897,9 @@ public void unlock() { FutureUtils.result(unlockResult, lockOpTimeout, TimeUnit.MILLISECONDS); } catch (TimeoutException toe) { // This shouldn't happen unless we lose a watch, and may result in a leaked lock. - LOG.error("Timeout unlocking {} owned by {} : ", new Object[] { lockPath, lockId, toe }); + LOG.error("Timeout unlocking {} owned by {} : ", lockPath, lockId, toe); } catch (Exception e) { - LOG.warn("{} failed to unlock {} : ", new Object[] { lockId, lockPath, e }); + LOG.warn("{} failed to unlock {} : ", lockId, lockPath, e); } } @@ -915,8 +913,7 @@ private void claimOwnership(int lockEpoch) { lockContext.addLockId(lockId); if (LOG.isDebugEnabled()) { LOG.debug("Notify lock waiters on {} at {} : watcher epoch {}, lock epoch {}", - new Object[] { lockPath, System.currentTimeMillis(), - lockEpoch, getEpoch() }); + lockPath, System.currentTimeMillis(), lockEpoch, getEpoch()); } acquireFuture.complete(true); } @@ -937,8 +934,7 @@ private void unlockInternal(final CompletableFuture promise) { return; } - LOG.info("Lock {} for {} is closed from state {}.", - new Object[] { lockId, lockPath, lockState.getState() }); + LOG.info("Lock {} for {} is closed from state {}.", lockId, lockPath, lockState.getState()); final boolean skipCleanup = lockState.inState(State.INIT) || lockState.inState(State.EXPIRED); @@ -990,10 +986,10 @@ public void safeRun() { } else if (KeeperException.Code.NONODE.intValue() == rc || KeeperException.Code.SESSIONEXPIRED.intValue() == rc) { LOG.info("Delete node failed. Node already gone for node {} id {}, rc = {}", - new Object[] { path, lockId, KeeperException.Code.get(rc) }); + path, lockId, KeeperException.Code.get(rc)); } else { LOG.error("Failed on deleting lock node {} for {} : {}", - new Object[] { path, lockId, KeeperException.Code.get(rc) }); + path, lockId, KeeperException.Code.get(rc)); } FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockUnlockCleanup); @@ -1062,7 +1058,7 @@ public void execute() { // The lock is either expired or closed if (!lockState.inState(State.WAITING)) { LOG.info("{} ignore watched node {} deleted event, since lock state has moved to {}.", - new Object[] { lockId, event.getPath(), lockState.getState() }); + lockId, event.getPath(), lockState.getState()); return; } lockState.transition(State.PREPARED); @@ -1185,7 +1181,7 @@ public String getActionName() { }); } else { LOG.error("Member {} doesn't exist in the members list {} for lock {}.", - new Object[]{ cid, children, lockPath}); + cid, children, lockPath); promise.completeExceptionally( new UnexpectedException("Member " + cid + " doesn't exist in member list " + children + " for lock " + lockPath)); @@ -1235,7 +1231,7 @@ public void execute() { shouldWatch = true; shouldClaimOwnership = true; LOG.info("LockWatcher {} for {} found its previous session {} held lock," - + " watch it to claim ownership.", new Object[] { myNode, lockPath, currentOwner }); + + " watch it to claim ownership.", myNode, lockPath, currentOwner); } else if (lockId.compareTo(currentOwner) == 0 && areLockWaitersInSameSession(siblingNode, ownerNode)) { // I found that my sibling is the current owner with same lock id (client id & session id) // It must be left by any race condition from same zookeeper client @@ -1243,14 +1239,14 @@ public void execute() { shouldClaimOwnership = true; LOG.info("LockWatcher {} for {} found itself {} already held lock at sibling node {}," + " watch it to claim ownership.", - new Object[]{myNode, lockPath, lockId, siblingNode}); + myNode, lockPath, lockId, siblingNode); } else { shouldWatch = wait; if (wait) { if (LOG.isDebugEnabled()) { LOG.debug("Current LockWatcher for {} with ephemeral node {}, " + "is waiting for {} to release lock at {}.", - new Object[]{lockPath, myNode, siblingNode, System.currentTimeMillis()}); + lockPath, myNode, siblingNode, System.currentTimeMillis()); } } shouldClaimOwnership = false; @@ -1275,7 +1271,7 @@ public void execute() { if (shouldClaimOwnership) { // watch owner successfully LOG.info("LockWatcher {} claimed ownership for {} after set watcher on {}.", - new Object[]{ myNode, lockPath, ownerNode }); + myNode, lockPath, ownerNode); claimOwnership(lockWatcher.epoch); promise.complete(currentOwner.getLeft()); } else { @@ -1329,15 +1325,14 @@ class LockWatcher implements Watcher { @Override public void process(WatchedEvent event) { LOG.debug("Received event {} from lock {} at {} : watcher epoch {}, lock epoch {}.", - new Object[] {event, lockPath, System.currentTimeMillis(), epoch, getEpoch() }); + event, lockPath, System.currentTimeMillis(), epoch, getEpoch()); if (event.getType() == Watcher.Event.EventType.None) { switch (event.getState()) { case SyncConnected: break; case Expired: LOG.info("Session {} is expired for lock {} at {} : watcher epoch {}, lock epoch {}.", - new Object[] { lockId.getRight(), lockPath, System.currentTimeMillis(), - epoch, getEpoch() }); + lockId.getRight(), lockPath, System.currentTimeMillis(), epoch, getEpoch()); handleSessionExpired(epoch); break; default: @@ -1348,7 +1343,7 @@ public void process(WatchedEvent event) { // watch on the nextLowestNode. This is a workaround since ZK doesn't support unsub. if (!event.getPath().equals(watchedNode)) { LOG.warn("{} (watching {}) ignored watched event from {} ", - new Object[] { lockId, watchedNode, event.getPath() }); + lockId, watchedNode, event.getPath()); return; } handleNodeDelete(epoch, event); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java index a2796c23dc6..c640b10c847 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java @@ -102,7 +102,7 @@ public List getLogSegments(Comparator co >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value && prevSegment.getLogSegmentSequenceNumber() + 1 != segment.getLogSegmentSequenceNumber()) { LOG.error("{} found ledger sequence number gap between log segment {} and {}", - new Object[] { streamName, prevSegment, segment }); + streamName, prevSegment, segment); throw new UnexpectedException(streamName + " found ledger sequence number gap between log segment " + prevSegment.getLogSegmentSequenceNumber() + " and " + segment.getLogSegmentSequenceNumber()); @@ -122,7 +122,7 @@ public List getLogSegments(Comparator co if (null != prevSegment && prevSegment.supportsSequenceId() && prevSegment.getStartSequenceId() > segment.getStartSequenceId()) { LOG.warn("{} found decreasing start sequence id in log segment {}, previous is {}", - new Object[] { streamName, segment, prevSegment }); + streamName, segment, prevSegment); } } else { startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID; @@ -159,7 +159,7 @@ public void add(String name, LogSegmentMetadata metadata) { if (!logSegments.containsKey(name)) { logSegments.put(name, metadata); LOG.info("{} added log segment ({} : {}) to cache.", - new Object[]{ streamName, name, metadata }); + streamName, name, metadata); } LogSegmentMetadata oldMetadata = lid2LogSegments.remove(metadata.getLogSegmentId()); if (null == oldMetadata) { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java index fa3b7430e24..36b1f384904 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java @@ -63,11 +63,11 @@ public class DLMTestUtil { private static final byte[] payloadStatic = repeatString("abc", 512).getBytes(); static String repeatString(String s, int n) { - String ret = s; + StringBuilder ret = new StringBuilder(s); for (int i = 1; i < n; i++) { - ret += s; + ret.append(s); } - return ret; + return ret.toString(); } public static Map readLogSegments(ZooKeeperClient zkc, String ledgerPath) diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java index b04c27b9bbc..c6833466179 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java @@ -681,7 +681,7 @@ public void onSuccess(DLSN value) { @Override public void onFailure(Throwable cause) { LOG.error("Encountered failures on writing record as (lid = {}, eid = {}) :", - new Object[]{currentLogSegmentSeqNo, currentEntryId, cause}); + currentLogSegmentSeqNo, currentEntryId, cause); errorsFound.set(true); syncLatch.countDown(); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java index 16565a8a4a2..d069e7251c3 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java @@ -135,7 +135,7 @@ public void setup() throws Exception { try { zkc = LocalDLMEmulator.connectZooKeeper("127.0.0.1", zkPort); } catch (Exception ex) { - LOG.error("hit exception connecting to zookeeper at {}:{}", new Object[] { "127.0.0.1", zkPort, ex }); + LOG.error("hit exception connecting to zookeeper at {}:{}", "127.0.0.1", zkPort, ex); throw ex; } } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java index d349ddad156..3219691007d 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java @@ -134,7 +134,7 @@ public void run() { } catch (IOException exc) { int nextMs = nextDelayMs(); LOG.info("Encountered exception {} on opening reader {} at {}, retrying in {} ms", - new Object[] { exc, readerName, dlsn, nextMs }); + exc, readerName, dlsn, nextMs); positionReader(dlsn); } } @@ -158,7 +158,7 @@ public void onSuccess(LogRecordWithDLSN value) { DLMTestUtil.verifyLargeLogRecord(value); } catch (Exception exc) { LOG.error("Exception encountered when verifying received log record {} for reader {} :", - new Object[] { value.getDlsn(), exc, readerName }); + value.getDlsn(), exc, readerName); errorsFound.set(true); completionLatch.countDown(); return; @@ -171,7 +171,7 @@ public void onSuccess(LogRecordWithDLSN value) { completionLatch.countDown(); } else { LOG.info("Reader {} : read count becomes {}, latch = {}", - new Object[] { readerName, readCount.get(), countLatch.getCount() }); + readerName, readCount.get(), countLatch.getCount()); nextDLSN = value.getDlsn().getNextDLSN(); readNext(); } From cdc09c38eddd6caf99a171257b234a8fa3e3f589 Mon Sep 17 00:00:00 2001 From: Sergii Zhevzhyk Date: Mon, 5 Aug 2019 16:28:17 +0200 Subject: [PATCH 0328/1642] Replace C style array declaration with Java style ### Motivation The C style array definitions look weird for a Java developer. ### Changes All C style array declarations were updated with the Java array declarations and the stylechecker rule was added to enforce the Java way. Reviewers: Enrico Olivelli , Sijie Guo This closes #2139 from vzhikserg/replace-c-style-array-declaration-with-java-style --- .../benchmark/BenchThroughputLatency.java | 10 ++++----- .../bookkeeper/benchmark/TestBenchmark.java | 2 +- .../common/util/OrderedExecutor.java | 4 ++-- .../apache/bookkeeper/bookie/BookieShell.java | 2 +- .../org/apache/bookkeeper/bookie/Cookie.java | 2 +- .../bookie/GarbageCollectorThread.java | 2 +- .../bookie/IndexPersistenceMgr.java | 4 ++-- .../org/apache/bookkeeper/bookie/Journal.java | 6 ++--- .../bookkeeper/bookie/LedgerDescriptor.java | 2 +- .../bookie/LedgerDescriptorImpl.java | 2 +- .../bookie/LedgerDescriptorReadOnlyImpl.java | 2 +- .../bookie/storage/ldb/ArrayGroupSort.java | 8 +++---- .../apache/bookkeeper/client/BookKeeper.java | 22 +++++++++---------- .../bookkeeper/meta/FlatLedgerManager.java | 2 +- .../apache/bookkeeper/meta/LedgerLayout.java | 2 +- .../bookkeeper/meta/LedgerMetadataSerDe.java | 2 +- .../bookkeeper/meta/ZkLedgerIdGenerator.java | 2 +- .../meta/ZkLedgerUnderreplicationManager.java | 4 ++-- .../java/org/apache/bookkeeper/net/DNS.java | 2 +- .../bookkeeper/proto/BookieClientImpl.java | 2 +- .../bookkeeper/sasl/SaslServerState.java | 2 +- .../http/service/DecommissionService.java | 2 +- .../http/service/ListBookieInfoService.java | 2 +- .../http/service/RecoveryBookieService.java | 2 +- .../cli/commands/bookies/InfoCommand.java | 2 +- .../cli/commands/bookies/RecoverCommand.java | 2 +- .../util/AvailabilityOfEntriesOfLedger.java | 2 +- .../bookkeeper/util/LocalBookKeeper.java | 8 +++---- .../apache/bookkeeper/util/ZeroBuffer.java | 2 +- .../bookie/LedgerStorageCheckpointTest.java | 2 +- .../bookkeeper/client/BookKeeperTest.java | 2 +- .../bookkeeper/client/MockBookKeeper.java | 4 ++-- .../apache/bookkeeper/client/TestFencing.java | 2 +- .../client/TestPendingReadLacOp.java | 4 ++-- .../bookkeeper/client/TestReadLastEntry.java | 4 ++-- .../meta/LedgerManagerIteratorTest.java | 4 ++-- .../bookkeeper/test/BookieClientTest.java | 2 +- .../bookkeeper/test/BookieReadWriteTest.java | 2 +- .../bookkeeper/test/BookieZKExpireTest.java | 2 +- .../bookkeeper/test/ConcurrentLedgerTest.java | 2 +- .../apache/bookkeeper/test/OpStatTest.java | 4 ++-- .../stats/codahale/FastTimerTest.java | 6 ++--- .../main/resources/bookkeeper/checkstyle.xml | 3 +++ .../com/scurrilous/circe/crc/Sse42Crc32C.java | 2 +- .../FederatedZKLogMetadataStore.java | 2 +- .../org/apache/distributedlog/tools/Tool.java | 2 +- 46 files changed, 80 insertions(+), 77 deletions(-) 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 08dc00266c0..e7f7f8ae3cd 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 @@ -64,13 +64,13 @@ public class BenchThroughputLatency implements AddCallback, Runnable { static final Logger LOG = LoggerFactory.getLogger(BenchThroughputLatency.class); BookKeeper bk; - LedgerHandle lh[]; + LedgerHandle[] lh; AtomicLong counter; Semaphore sem; int numberOfLedgers = 1; final int sendLimit; - final long latencies[]; + final long[] latencies; static class Context { long localStartTime; @@ -115,9 +115,9 @@ public void close() throws InterruptedException, BKException { } long previous = 0; - byte bytes[]; + byte[] bytes; - void setEntryData(byte data[]) { + void setEntryData(byte[] data) { bytes = data; } @@ -310,7 +310,7 @@ public void run() { // Do a warmup run Thread thread; - byte data[] = new byte[entrysize]; + byte[] data = new byte[entrysize]; Arrays.fill(data, (byte) 'x'); ClientConfiguration conf = new ClientConfiguration(); diff --git a/bookkeeper-benchmark/src/test/java/org/apache/bookkeeper/benchmark/TestBenchmark.java b/bookkeeper-benchmark/src/test/java/org/apache/bookkeeper/benchmark/TestBenchmark.java index 8678017d882..bee5f90d335 100644 --- a/bookkeeper-benchmark/src/test/java/org/apache/bookkeeper/benchmark/TestBenchmark.java +++ b/bookkeeper-benchmark/src/test/java/org/apache/bookkeeper/benchmark/TestBenchmark.java @@ -97,7 +97,7 @@ public void run() { t.start(); Thread.sleep(10000); - byte data[] = new byte[1024]; + byte[] data = new byte[1024]; Arrays.fill(data, (byte) 'x'); long lastLedgerId = 0; diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java index a645d64646b..4330340cc1d 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java @@ -75,8 +75,8 @@ public class OrderedExecutor implements ExecutorService { protected static final long WARN_TIME_MICRO_SEC_DEFAULT = TimeUnit.SECONDS.toMicros(1); final String name; - final ExecutorService threads[]; - final long threadIds[]; + final ExecutorService[] threads; + final long[] threadIds; final Random rand = new Random(); final OpStatsLogger taskExecutionStats; final OpStatsLogger taskPendingStats; 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 98490520799..5149898f087 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 @@ -2026,7 +2026,7 @@ public int compare(File file1, File file2) { } } - public static void main(String argv[]) throws Exception { + public static void main(String[] argv) throws Exception { BookieShell shell = new BookieShell(); // handle some common options for multiple cmds 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 1ddc5abd7ed..5e8cb6449d6 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 @@ -363,7 +363,7 @@ public static Cookie readFromDirectory(File directory) throws IOException { * if the 'bookieHost' was created using a hostname */ public boolean isBookieHostCreatedFromIp() throws IOException { - String parts[] = bookieHost.split(":"); + String[] parts = bookieHost.split(":"); if (parts.length != 2) { throw new IOException(bookieHost + " does not have the form host:port"); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java index 811fbe6dd51..94a02a265ba 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java @@ -436,7 +436,7 @@ void doCompactEntryLogs(double threshold) { logsToCompact.sort(Comparator.comparing(EntryLogMetadata::getUsage)); final int numBuckets = 10; - int entryLogUsageBuckets[] = new int[numBuckets]; + int[] entryLogUsageBuckets = new int[numBuckets]; for (EntryLogMetadata meta : logsToCompact) { int bucketIndex = Math.min( diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java index 898e1460e05..c98bc6ee8ff 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java @@ -177,7 +177,7 @@ private void handleLedgerEviction(RemovalNotification noti * the FileInfo from cache, that FileInfo is then evicted and closed before we * could even increase the reference counter. */ - CachedFileInfo getFileInfo(final Long ledger, final byte masterKey[]) throws IOException { + CachedFileInfo getFileInfo(final Long ledger, final byte[] masterKey) throws IOException { try { CachedFileInfo fi; persistenceMgrStats.getPendingGetFileInfoCounter().inc(); @@ -596,7 +596,7 @@ private void writeBuffers(Long ledger, if (count == 0) { return; } - ByteBuffer buffs[] = new ByteBuffer[count]; + ByteBuffer[] buffs = new ByteBuffer[count]; for (int j = 0; j < count; j++) { buffs[j] = entries.get(start + j).getPageToWrite(); if (entries.get(start + j).getLedger() != ledger) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 06fc5e2a98c..4ee540c6916 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -103,7 +103,7 @@ public interface BufferedChannelBuilder { * @return list of filtered ids */ static List listJournalIds(File journalDir, JournalIdFilter filter) { - File logFiles[] = journalDir.listFiles(); + File[] logFiles = journalDir.listFiles(); if (logFiles == null || logFiles.length == 0) { return Collections.emptyList(); } @@ -190,7 +190,7 @@ public LogMark getCurMark() { } void rollLog(LastLogMark lastMark) throws NoWritableLedgerDirException { - byte buff[] = new byte[16]; + byte[] buff = new byte[16]; ByteBuffer bb = ByteBuffer.wrap(buff); // we should record marked in markLog // which is safe since records before lastMark have been @@ -229,7 +229,7 @@ void rollLog(LastLogMark lastMark) throws NoWritableLedgerDirException { * and then max log position in max journal log. */ void readLog() { - byte buff[] = new byte[16]; + byte[] buff = new byte[16]; ByteBuffer bb = ByteBuffer.wrap(buff); LogMark mark = new LogMark(); for (File dir: ledgerDirsManager.getAllLedgerDirs()) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java index abb7b34e287..1acf832d0d8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java @@ -61,7 +61,7 @@ static ByteBuf createLedgerFenceEntry(Long ledgerId) { return bb; } - abstract void checkAccess(byte masterKey[]) throws BookieException, IOException; + abstract void checkAccess(byte[] masterKey) throws BookieException, IOException; abstract long getLedgerId(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java index da884b5b5dc..34c11bcacfb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java @@ -54,7 +54,7 @@ public class LedgerDescriptorImpl extends LedgerDescriptor { } @Override - void checkAccess(byte masterKey[]) throws BookieException, IOException { + void checkAccess(byte[] masterKey) throws BookieException, IOException { if (!Arrays.equals(this.masterKey, masterKey)) { LOG.error("[{}] Requested master key {} does not match the cached master key {}", this.ledgerId, Arrays.toString(masterKey), Arrays.toString(this.masterKey)); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorReadOnlyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorReadOnlyImpl.java index 40bf988582b..2f6655bde2f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorReadOnlyImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorReadOnlyImpl.java @@ -46,7 +46,7 @@ long addEntry(ByteBuf entry) throws IOException { } @Override - void checkAccess(byte masterKey[]) throws BookieException, IOException { + void checkAccess(byte[] masterKey) throws BookieException, IOException { assert false; throw new IOException("Invalid action on read only descriptor"); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ArrayGroupSort.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ArrayGroupSort.java index 719b33dbc5f..16f7557bb63 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ArrayGroupSort.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ArrayGroupSort.java @@ -52,7 +52,7 @@ public void sort(long[] array, int offset, int length) { ////// Private - private void quickSort(long array[], int low, int high) { + private void quickSort(long[] array, int low, int high) { if (low < high) { int pivotIdx = partition(array, low, high); quickSort(array, low, pivotIdx - groupSize); @@ -60,7 +60,7 @@ private void quickSort(long array[], int low, int high) { } } - private int partition(long array[], int low, int high) { + private int partition(long[] array, int low, int high) { int pivotIdx = high; int i = low; @@ -75,7 +75,7 @@ private int partition(long array[], int low, int high) { return i; } - private void swap(long array[], int a, int b) { + private void swap(long[] array, int a, int b) { long tmp; for (int k = 0; k < groupSize; k++) { tmp = array[a + k]; @@ -84,7 +84,7 @@ private void swap(long array[], int a, int b) { } } - private boolean isLess(long array[], int idx1, int idx2) { + private boolean isLess(long[] array, int idx1, int idx2) { for (int i = 0; i < keySize; i++) { long k1 = array[idx1 + i]; long k2 = array[idx2 + i]; 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 94591afc803..52dde9cba6e 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 @@ -846,7 +846,7 @@ public void asyncCreateLedger(final int ensSize, final int writeQuorumSize, fina * @throws InterruptedException * @throws BKException */ - public LedgerHandle createLedger(DigestType digestType, byte passwd[]) + public LedgerHandle createLedger(DigestType digestType, byte[] passwd) throws BKException, InterruptedException { return createLedger(3, 2, digestType, passwd); } @@ -865,7 +865,7 @@ public LedgerHandle createLedger(DigestType digestType, byte passwd[]) * @throws BKException */ public LedgerHandle createLedger(int ensSize, int qSize, - DigestType digestType, byte passwd[]) + DigestType digestType, byte[] passwd) throws InterruptedException, BKException { return createLedger(ensSize, qSize, qSize, digestType, passwd, Collections.emptyMap()); } @@ -885,7 +885,7 @@ public LedgerHandle createLedger(int ensSize, int qSize, * @throws BKException */ public LedgerHandle createLedger(int ensSize, int writeQuorumSize, int ackQuorumSize, - DigestType digestType, byte passwd[]) + DigestType digestType, byte[] passwd) throws InterruptedException, BKException { return createLedger(ensSize, writeQuorumSize, ackQuorumSize, digestType, passwd, Collections.emptyMap()); } @@ -904,7 +904,7 @@ public LedgerHandle createLedger(int ensSize, int writeQuorumSize, int ackQuorum * @throws BKException */ public LedgerHandle createLedger(int ensSize, int writeQuorumSize, int ackQuorumSize, - DigestType digestType, byte passwd[], final Map customMetadata) + DigestType digestType, byte[] passwd, final Map customMetadata) throws InterruptedException, BKException { CompletableFuture future = new CompletableFuture<>(); SyncCreateCallback result = new SyncCreateCallback(future); @@ -939,7 +939,7 @@ public LedgerHandle createLedger(int ensSize, int writeQuorumSize, int ackQuorum * @throws BKException */ public LedgerHandle createLedgerAdv(int ensSize, int writeQuorumSize, int ackQuorumSize, - DigestType digestType, byte passwd[]) + DigestType digestType, byte[] passwd) throws InterruptedException, BKException { return createLedgerAdv(ensSize, writeQuorumSize, ackQuorumSize, digestType, passwd, Collections.emptyMap()); @@ -961,7 +961,7 @@ public LedgerHandle createLedgerAdv(int ensSize, int writeQuorumSize, int ackQuo * @throws BKException */ public LedgerHandle createLedgerAdv(int ensSize, int writeQuorumSize, int ackQuorumSize, - DigestType digestType, byte passwd[], final Map customMetadata) + DigestType digestType, byte[] passwd, final Map customMetadata) throws InterruptedException, BKException { CompletableFuture future = new CompletableFuture<>(); SyncCreateAdvCallback result = new SyncCreateAdvCallback(future); @@ -1053,7 +1053,7 @@ public LedgerHandle createLedgerAdv(final long ledgerId, int writeQuorumSize, int ackQuorumSize, DigestType digestType, - byte passwd[], + byte[] passwd, final Map customMetadata) throws InterruptedException, BKException { CompletableFuture future = new CompletableFuture<>(); @@ -1166,7 +1166,7 @@ public void asyncCreateLedgerAdv(final long ledgerId, * @param ctx * optional control object */ - public void asyncOpenLedger(final long lId, final DigestType digestType, final byte passwd[], + public void asyncOpenLedger(final long lId, final DigestType digestType, final byte[] passwd, final OpenCallback cb, final Object ctx) { closeLock.readLock().lock(); try { @@ -1211,7 +1211,7 @@ public void asyncOpenLedger(final long lId, final DigestType digestType, final b * @param ctx * optional control object */ - public void asyncOpenLedgerNoRecovery(final long lId, final DigestType digestType, final byte passwd[], + public void asyncOpenLedgerNoRecovery(final long lId, final DigestType digestType, final byte[] passwd, final OpenCallback cb, final Object ctx) { closeLock.readLock().lock(); try { @@ -1241,7 +1241,7 @@ public void asyncOpenLedgerNoRecovery(final long lId, final DigestType digestTyp * @throws InterruptedException * @throws BKException */ - public LedgerHandle openLedger(long lId, DigestType digestType, byte passwd[]) + public LedgerHandle openLedger(long lId, DigestType digestType, byte[] passwd) throws BKException, InterruptedException { CompletableFuture future = new CompletableFuture<>(); SyncOpenCallback result = new SyncOpenCallback(future); @@ -1268,7 +1268,7 @@ public LedgerHandle openLedger(long lId, DigestType digestType, byte passwd[]) * @throws InterruptedException * @throws BKException */ - public LedgerHandle openLedgerNoRecovery(long lId, DigestType digestType, byte passwd[]) + public LedgerHandle openLedgerNoRecovery(long lId, DigestType digestType, byte[] passwd) throws BKException, InterruptedException { CompletableFuture future = new CompletableFuture<>(); SyncOpenCallback result = new SyncOpenCallback(future); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java index 7c89326ffe7..17612d7452c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/FlatLedgerManager.java @@ -73,7 +73,7 @@ public String getLedgerPath(long ledgerId) { public long getLedgerId(String nodeName) throws IOException { long ledgerId; try { - String parts[] = nodeName.split(ledgerPrefix); + String[] parts = nodeName.split(ledgerPrefix); ledgerId = Long.parseLong(parts[parts.length - 1]); } catch (NumberFormatException e) { throw new IOException(e); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java index eb5b705c16b..09f3a708e6b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java @@ -100,7 +100,7 @@ public static LedgerLayout parseLayout(byte[] bytes) throws IOException { log.debug("Parsing Layout: {}", layout); } - String lines[] = layout.split(lSplitter); + String[] lines = layout.split(lSplitter); try { int layoutFormatVersion = Integer.parseInt(lines[0]); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java index 2ee441abf4e..40048447a36 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -460,7 +460,7 @@ private static LedgerMetadata parseVersion1Config(InputStream is) throws IOExcep String line = reader.readLine(); while (line != null) { - String parts[] = line.split(FIELD_SPLITTER); + String[] parts = line.split(FIELD_SPLITTER); if (parts[1].equals(V1_CLOSED_TAG)) { Long l = Long.parseLong(parts[0]); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerIdGenerator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerIdGenerator.java index c5d53465085..842f1a072a0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerIdGenerator.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerIdGenerator.java @@ -126,7 +126,7 @@ public void processResult(int rc, String path, Object ctx) { private static long getLedgerIdFromGenPath(String nodeName, String ledgerPrefix) throws IOException { long ledgerId; try { - String parts[] = nodeName.split(ledgerPrefix); + String[] parts = nodeName.split(ledgerPrefix); ledgerId = Long.parseLong(parts[parts.length - 1]); } catch (NumberFormatException e) { throw new IOException(e); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java index 1da65781659..ec5b8db6aa9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java @@ -394,9 +394,9 @@ public void markLedgerReplicated(long ledgerId) throws ReplicationException.Unav try { // clean up the hierarchy - String parts[] = getUrLedgerZnode(ledgerId).split("/"); + String[] parts = getUrLedgerZnode(ledgerId).split("/"); for (int i = 1; i <= 4; i++) { - String p[] = Arrays.copyOf(parts, parts.length - i); + String[] p = Arrays.copyOf(parts, parts.length - i); String path = Joiner.on("/").join(p); Stat s = zkc.exists(path, null); if (s != null) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/DNS.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/DNS.java index 308728e7beb..854145c4b80 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/DNS.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/DNS.java @@ -194,7 +194,7 @@ public static String[] getIPs(String strInterface, allAddrs.removeAll(getSubinterfaceInetAddrs(netIf)); } - String ips[] = new String[allAddrs.size()]; + String[] ips = new String[allAddrs.size()]; int i = 0; for (InetAddress addr : allAddrs) { ips[i++] = addr.getHostAddress(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index 9cd52e1ebd7..5b708e50d58 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -633,7 +633,7 @@ public void writeComplete(int rc, long ledger, long entry, BookieSocketAddress a } }; Counter counter = new Counter(); - byte hello[] = "hello".getBytes(UTF_8); + byte[] hello = "hello".getBytes(UTF_8); long ledger = Long.parseLong(args[2]); EventLoopGroup eventLoopGroup = new NioEventLoopGroup(1); OrderedExecutor executor = OrderedExecutor.newBuilder() diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslServerState.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslServerState.java index 2291526c72f..c297d0d8573 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslServerState.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/sasl/SaslServerState.java @@ -150,7 +150,7 @@ public SaslServerCallbackHandler(Configuration configuration, ServerConfiguratio throws IOException { String configurationEntry = serverConfiguration.getString(SaslConstants.JAAS_BOOKIE_SECTION_NAME, SaslConstants.JAAS_DEFAULT_BOOKIE_SECTION_NAME); - AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(configurationEntry); + AppConfigurationEntry[] configurationEntries = configuration.getAppConfigurationEntry(configurationEntry); if (configurationEntries == null) { String errorMessage = "Could not find a '" + configurationEntry diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java index 0091ce9ee24..41f18cb1f04 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java @@ -74,7 +74,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { HashMap configMap = JsonUtil.fromJson(requestBody, HashMap.class); if (configMap != null && configMap.containsKey("bookie_src")) { try { - String bookieSrcString[] = configMap.get("bookie_src").split(":"); + String[] bookieSrcString = configMap.get("bookie_src").split(":"); BookieSocketAddress bookieSrc = new BookieSocketAddress( bookieSrcString[0], Integer.parseInt(bookieSrcString[1])); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookieInfoService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookieInfoService.java index 14439be86c5..b543cec5239 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookieInfoService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookieInfoService.java @@ -64,7 +64,7 @@ public ListBookieInfoService(ServerConfiguration conf) { } String getReadable(long val) { - String unit[] = {"", "KB", "MB", "GB", "TB" }; + String[] unit = {"", "KB", "MB", "GB", "TB" }; int cnt = 0; double d = val; while (d >= 1000 && cnt < unit.length - 1) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/RecoveryBookieService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/RecoveryBookieService.java index 3a08bafc5b1..4a0a0a38a35 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/RecoveryBookieService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/RecoveryBookieService.java @@ -105,7 +105,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { if (HttpServer.Method.PUT == request.getMethod() && !requestJsonBody.bookieSrc.isEmpty()) { runFunctionWithRegistrationManager(conf, rm -> { - String bookieSrcString[] = requestJsonBody.bookieSrc.get(0).split(":"); + String[] bookieSrcString = requestJsonBody.bookieSrc.get(0).split(":"); BookieSocketAddress bookieSrc = new BookieSocketAddress( bookieSrcString[0], Integer.parseInt(bookieSrcString[1])); boolean deleteCookie = requestJsonBody.deleteCookie; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java index 2e49d3926e2..4abb2c4ec34 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java @@ -52,7 +52,7 @@ public InfoCommand() { } String getReadable(long val) { - String unit[] = {"", "KB", "MB", "GB", "TB"}; + String[] unit = {"", "KB", "MB", "GB", "TB"}; int cnt = 0; double d = val; while (d >= 1000 && cnt < unit.length - 1) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java index ba4acf53113..c1a169495a0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java @@ -125,7 +125,7 @@ private boolean recover(ServerConfiguration conf, RecoverFlags flags) final String[] bookieStrs = flags.bookieAddress.split(","); final Set bookieAddrs = new HashSet<>(); for (String bookieStr : bookieStrs) { - final String bookieStrParts[] = bookieStr.split(":"); + final String[] bookieStrParts = bookieStr.split(":"); if (bookieStrParts.length != 2) { System.err.println("BookieSrcs has invalid bookie address format (host:port expected) : " + bookieStr); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java index 8a3047a477e..b4987c8dce2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java @@ -48,7 +48,7 @@ public class AvailabilityOfEntriesOfLedger { public static final long INVALID_ENTRYID = -1; public static final AvailabilityOfEntriesOfLedger EMPTY_AVAILABILITYOFENTRIESOFLEDGER; static { - long tmpArray[] = {}; + long[] tmpArray = {}; EMPTY_AVAILABILITYOFENTRIESOFLEDGER = new AvailabilityOfEntriesOfLedger(Arrays.stream(tmpArray).iterator()); } 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 d6efeac1af8..b078dd45c23 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 @@ -95,9 +95,9 @@ public LocalBookKeeper( private static String defaultLocalBookiesConfigDir = "/tmp/localbookies-config"; //BookKeeper variables - File journalDirs[]; - BookieServer bs[]; - ServerConfiguration bsConfs[]; + File[] journalDirs; + BookieServer[] bs; + ServerConfiguration[] bsConfs; Integer initialPort = 5000; private ServerConfiguration baseConf; @@ -497,7 +497,7 @@ private static void usage() { public static boolean waitForServerUp(String hp, long timeout) { long start = System.currentTimeMillis(); - String split[] = hp.split(":"); + String[] split = hp.split(":"); String host = split[0]; int port = Integer.parseInt(split[1]); while (true) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZeroBuffer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZeroBuffer.java index 8778d6e1900..73c786c8796 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZeroBuffer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZeroBuffer.java @@ -29,7 +29,7 @@ */ public class ZeroBuffer { - private static final byte zeroBytes[] = new byte[64 * 1024]; + private static final byte[] zeroBytes = new byte[64 * 1024]; /** * Fill zeros into given buffer. diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java index 89a5063514c..1ea661b72b2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java @@ -168,7 +168,7 @@ protected File createTempDir(String prefix, String suffix) throws IOException { } private LogMark readLastMarkFile(File lastMarkFile) throws IOException { - byte buff[] = new byte[16]; + byte[] buff = new byte[16]; ByteBuffer bb = ByteBuffer.wrap(buff); LogMark rolledLogMark = new LogMark(); FileInputStream fis = new FileInputStream(lastMarkFile); 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 3c842e6ad0c..a2cd7a0842b 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 @@ -849,7 +849,7 @@ public MockZooKeeper(String connectString, int sessionTimeout, Watcher watcher, } @Override - public void create(final String path, byte data[], List acl, CreateMode createMode, StringCallback cb, + public void create(final String path, byte[] data, List acl, CreateMode createMode, StringCallback cb, Object ctx) { StringCallback injectedCallback = new StringCallback() { @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java index 81212d1b701..d26b22625c3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeper.java @@ -76,12 +76,12 @@ public MockBookKeeper(ZooKeeper zkc) throws Exception { } @Override - public LedgerHandle createLedger(DigestType digestType, byte passwd[]) throws BKException { + public LedgerHandle createLedger(DigestType digestType, byte[] passwd) throws BKException { return createLedger(3, 2, digestType, passwd); } @Override - public LedgerHandle createLedger(int ensSize, int qSize, DigestType digestType, byte passwd[]) throws BKException { + public LedgerHandle createLedger(int ensSize, int qSize, DigestType digestType, byte[] passwd) throws BKException { return createLedger(ensSize, qSize, qSize, digestType, passwd); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java index aa2dd6b9b07..096273e1f1a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java @@ -182,7 +182,7 @@ public void run() { CyclicBarrier barrier = new CyclicBarrier(numRecovery + 1); - LedgerOpenThread threads[] = new LedgerOpenThread[numRecovery]; + LedgerOpenThread[] threads = new LedgerOpenThread[numRecovery]; for (int i = 0; i < numRecovery; i++) { threads[i] = new LedgerOpenThread(i, digestType, writelh.getId(), barrier); threads[i].start(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestPendingReadLacOp.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestPendingReadLacOp.java index c9ca5083889..34d69e54262 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestPendingReadLacOp.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestPendingReadLacOp.java @@ -36,8 +36,8 @@ */ public class TestPendingReadLacOp extends BookKeeperClusterTestCase { private static final Logger LOG = LoggerFactory.getLogger(TestPendingReadLacOp.class); - byte pwd[] = "asdf".getBytes(); - byte data[] = "foo".getBytes(); + byte[] pwd = "asdf".getBytes(); + byte[] data = "foo".getBytes(); public TestPendingReadLacOp() { super(3); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastEntry.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastEntry.java index 35c0f3b69af..c31467634fc 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastEntry.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastEntry.java @@ -87,7 +87,7 @@ public void testTryReadLastEntryOnEmptyLedger() throws Exception { @Test public void testTryReadLastEntryAsync() throws Exception { final LedgerHandle lh = bkc.createLedger(1, 1, 1, digestType, "".getBytes()); - byte data[] = new byte[1024]; + byte[] data = new byte[1024]; Arrays.fill(data, (byte) 'x'); for (int j = 0; j < 100; j++) { data[1023] = Integer.valueOf(j).byteValue(); @@ -122,7 +122,7 @@ public void readComplete(int rc, LedgerHandle lh, Enumeration seq, @Test public void testTryReadLastEntrySync() throws Exception { final LedgerHandle lh = bkc.createLedger(1, 1, 1, digestType, "".getBytes()); - byte data[] = new byte[1024]; + byte[] data = new byte[1024]; Arrays.fill(data, (byte) 'x'); for (int j = 0; j < 100; j++) { data[1023] = Integer.valueOf(j).byteValue(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java index 1593e4bab98..60c7d0baab1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java @@ -268,7 +268,7 @@ public void validateEmptyL4PathSkipped() throws Throwable { createLedger(lm, id); } - String paths[] = { + String[] paths = { "/ledgers/633/4994/3938/4948", // Empty L4 path, must be skipped }; @@ -315,7 +315,7 @@ public void testWithSeveralIncompletePaths() throws Throwable { createLedger(lm, id); } - String paths[] = { + String[] paths = { "/ledgers/000/0000/0000", // top level, W-4292762 "/ledgers/234/5678/9999", // shares two path segments with the first one, comes after "/ledgers/339/0000/0000", // shares one path segment with the second one, comes first diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java index 322ca6da772..2562009a05c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java @@ -110,7 +110,7 @@ public void tearDown() throws Exception { } private static void recursiveDelete(File dir) { - File children[] = dir.listFiles(); + File[] children = dir.listFiles(); if (children != null) { for (File child : children) { recursiveDelete(child); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieReadWriteTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieReadWriteTest.java index afcb4b475c3..7e4f442ea6d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieReadWriteTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieReadWriteTest.java @@ -286,7 +286,7 @@ public void testReadWriteRangeAsyncSingleClient() throws IOException { // bkc.initMessageDigest("SHA1"); ledgerId = lh.getId(); LOG.info("Ledger ID: " + lh.getId()); - byte bytes[] = {'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i'}; + byte[] bytes = {'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i'}; lh.asyncAddEntry(bytes, 0, bytes.length, this, sync); lh.asyncAddEntry(bytes, 0, 4, this, sync); // abcd diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java index 21ac7e5c4b3..37160b59471 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java @@ -53,7 +53,7 @@ public void testBookieServerZKExpireBehaviour() throws Exception { HashSet threadset = new HashSet(); int threadCount = Thread.activeCount(); - Thread threads[] = new Thread[threadCount * 2]; + Thread[] threads = new Thread[threadCount * 2]; threadCount = Thread.enumerate(threads); for (int i = 0; i < threadCount; i++) { if (threads[i].getName().indexOf("SendThread") != -1) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java index d611e9daf95..282c7a55862 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java @@ -109,7 +109,7 @@ public void tearDown() { recursiveDelete(ledgerDir); } - byte zeros[] = new byte[16]; + byte[] zeros = new byte[16]; int iterations = 51; { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/OpStatTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/OpStatTest.java index a950709a009..256616cb960 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/OpStatTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/OpStatTest.java @@ -70,7 +70,7 @@ private void validateOpStat(TestStatsProvider stats, String path, BiConsumer f) { + private void validateOpStat(TestStatsProvider stats, String[] paths, BiConsumer f) { for (String path : paths) { validateOpStat(stats, path, f); } @@ -83,7 +83,7 @@ private void validateNonMonotonicCounterGauge(TestStatsProvider stats, String pa f.accept(counter.get(), counter.getMax()); } - private void validateNonMonotonicCounterGauges(TestStatsProvider stats, String paths[], BiConsumer f) { + private void validateNonMonotonicCounterGauges(TestStatsProvider stats, String[] paths, BiConsumer f) { for (String path : paths) { validateNonMonotonicCounterGauge(stats, path, f); } diff --git a/bookkeeper-stats-providers/codahale-metrics-provider/src/test/java/org/apache/bookkeeper/stats/codahale/FastTimerTest.java b/bookkeeper-stats-providers/codahale-metrics-provider/src/test/java/org/apache/bookkeeper/stats/codahale/FastTimerTest.java index b3a744fbdf5..84c61e55762 100644 --- a/bookkeeper-stats-providers/codahale-metrics-provider/src/test/java/org/apache/bookkeeper/stats/codahale/FastTimerTest.java +++ b/bookkeeper-stats-providers/codahale-metrics-provider/src/test/java/org/apache/bookkeeper/stats/codahale/FastTimerTest.java @@ -100,9 +100,9 @@ public void testFunctional() { public void testTimer() { // load definitions for testing the timer // following 3 array lengths must match: each element defines values for one phase - final int timeRange[] = new int[] { 90, 190, 50, 90, 100, 100 }; - final int timeBase[] = new int[] { 10, 10, 50, 10, 0, 0 }; - final int rate[] = new int[] { 1000, 1000, 1000, 1000, 0, 1 }; + final int[] timeRange = new int[] { 90, 190, 50, 90, 100, 100 }; + final int[] timeBase = new int[] { 10, 10, 50, 10, 0, 0 }; + final int[] rate = new int[] { 1000, 1000, 1000, 1000, 0, 1 }; final int window = 5; // use a 5 second window for testing FastTimer t = getMockedFastTimer(window, FastTimer.Buckets.fine); diff --git a/buildtools/src/main/resources/bookkeeper/checkstyle.xml b/buildtools/src/main/resources/bookkeeper/checkstyle.xml index 452f64041fd..42b499dbab4 100644 --- a/buildtools/src/main/resources/bookkeeper/checkstyle.xml +++ b/buildtools/src/main/resources/bookkeeper/checkstyle.xml @@ -437,5 +437,8 @@ page at http://checkstyle.sourceforge.net/config.html --> + + + diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/crc/Sse42Crc32C.java b/circe-checksum/src/main/java/com/scurrilous/circe/crc/Sse42Crc32C.java index 28a989d2c56..22379a0e32d 100644 --- a/circe-checksum/src/main/java/com/scurrilous/circe/crc/Sse42Crc32C.java +++ b/circe-checksum/src/main/java/com/scurrilous/circe/crc/Sse42Crc32C.java @@ -54,7 +54,7 @@ public static boolean isSupported() { config = 0; } - public Sse42Crc32C(int chunkWords[]) { + public Sse42Crc32C(int[] chunkWords) { if (chunkWords.length == 0) { config = 0; } else { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java index 8d458661dc8..280d8ea0287 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java @@ -504,7 +504,7 @@ public void onFailure(Throwable cause) { } private String getNamespaceFromZkPath(String zkPath) throws UnexpectedException { - String parts[] = zkPath.split(SUB_NAMESPACE_PREFIX); + String[] parts = zkPath.split(SUB_NAMESPACE_PREFIX); if (parts.length <= 0) { throw new UnexpectedException("Invalid namespace @ " + zkPath); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/Tool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/Tool.java index 31395ca4050..cfcc8a19206 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/Tool.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/Tool.java @@ -223,7 +223,7 @@ public int run(String[] args) throws Exception { return cmd.runCmd(newArgs); } - public static void main(String args[]) { + public static void main(String[] args) { int rc = -1; if (args.length <= 0) { System.err.println("No tool to run."); From 7043847a6c17658e28859bc16fe1ed5da773b76a Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Tue, 13 Aug 2019 10:53:39 -0700 Subject: [PATCH 0329/1642] Replicascheck comprehensive testcases. Descriptions of the changes in this PR: - adding comprehensive testcases for newly added scrutiny - replicasCheck. Reviewers: Enrico Olivelli , Sijie Guo This closes #2141 from reddycharan/replicaschecktest --- .../bookkeeper/replication/Auditor.java | 35 +- .../util/AvailabilityOfEntriesOfLedger.java | 7 + .../replication/AuditorPeriodicCheckTest.java | 6 + .../AuditorPlacementPolicyCheckTest.java | 1 + .../replication/AuditorReplicasCheckTest.java | 810 ++++++++++++++++++ 5 files changed, 852 insertions(+), 7 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java 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 3b8219f7345..7e815de350d 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 @@ -136,7 +136,8 @@ public class Auditor implements AutoCloseable { private final ServerConfiguration conf; private final BookKeeper bkc; private final boolean ownBkc; - private BookKeeperAdmin admin; + private final BookKeeperAdmin admin; + private final boolean ownAdmin; private BookieLedgerIndexer bookieLedgerIndexer; private LedgerManager ledgerManager; private LedgerUnderreplicationManager ledgerUnderreplicationManager; @@ -297,10 +298,27 @@ public Auditor(final String bookieIdentifier, } public Auditor(final String bookieIdentifier, - ServerConfiguration conf, - BookKeeper bkc, - boolean ownBkc, - StatsLogger statsLogger) + ServerConfiguration conf, + BookKeeper bkc, + boolean ownBkc, + StatsLogger statsLogger) + throws UnavailableException { + this(bookieIdentifier, + conf, + bkc, + ownBkc, + new BookKeeperAdmin(bkc, statsLogger), + true, + statsLogger); + } + + public Auditor(final String bookieIdentifier, + ServerConfiguration conf, + BookKeeper bkc, + boolean ownBkc, + BookKeeperAdmin admin, + boolean ownAdmin, + StatsLogger statsLogger) throws UnavailableException { this.conf = conf; this.underreplicatedLedgerRecoveryGracePeriod = conf.getUnderreplicatedLedgerRecoveryGracePeriod(); @@ -419,6 +437,8 @@ public Integer getSample() { this.bkc = bkc; this.ownBkc = ownBkc; + this.admin = admin; + this.ownAdmin = ownAdmin; initialize(conf, bkc); executor = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() { @@ -443,7 +463,6 @@ private void initialize(ServerConfiguration conf, BookKeeper bkc) this.ledgerUnderreplicationManager = ledgerManagerFactory .newLedgerUnderreplicationManager(); - this.admin = new BookKeeperAdmin(bkc, statsLogger); LOG.info("AuthProvider used by the Auditor is {}", admin.getConf().getClientAuthProviderFactoryClass()); if (this.ledgerUnderreplicationManager @@ -1910,7 +1929,9 @@ public void shutdown() { LOG.warn("Executor not shutting down, interrupting"); executor.shutdownNow(); } - admin.close(); + if (ownAdmin) { + admin.close(); + } if (ownBkc) { bkc.close(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java index b4987c8dce2..4695d8c5358 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/AvailabilityOfEntriesOfLedger.java @@ -184,6 +184,13 @@ public AvailabilityOfEntriesOfLedger(PrimitiveIterator.OfLong entriesOfLedgerItr this.closeStateOfEntriesOfALedger(); } + public AvailabilityOfEntriesOfLedger(long[] entriesOfLedger) { + for (long entry : entriesOfLedger) { + this.addEntryToAvailabileEntriesOfLedger(entry); + } + this.closeStateOfEntriesOfALedger(); + } + public AvailabilityOfEntriesOfLedger(byte[] serializeStateOfEntriesOfLedger) { byte[] header = new byte[HEADER_SIZE]; byte[] serializedSequenceGroupByteArray = new byte[SequenceGroup.SEQUENCEGROUP_BYTES]; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 3999c3c7b52..448913e0dde 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -54,6 +54,7 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManagerFactory; @@ -663,6 +664,11 @@ public TestAuditor(String bookieIdentifier, ServerConfiguration conf, BookKeeper super(bookieIdentifier, conf, bkc, ownBkc, statsLogger); } + public TestAuditor(String bookieIdentifier, ServerConfiguration conf, BookKeeper bkc, boolean ownBkc, + BookKeeperAdmin bkadmin, boolean ownadmin, StatsLogger statsLogger) throws UnavailableException { + super(bookieIdentifier, conf, bkc, ownBkc, bkadmin, ownadmin, statsLogger); + } + public TestAuditor(final String bookieIdentifier, ServerConfiguration conf, StatsLogger statsLogger) throws UnavailableException { super(bookieIdentifier, conf, statsLogger); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java index b8548e048c1..468a0324c98 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -656,6 +656,7 @@ private void setServerConfigProperties(ServerConfiguration servConf, String ense servConf.setProperty(ClientConfiguration.ENSEMBLE_PLACEMENT_POLICY, ensemblePlacementPolicyClass); servConf.setAuditorPeriodicCheckInterval(0); servConf.setAuditorPeriodicBookieCheckInterval(0); + servConf.setAuditorPeriodicReplicasCheckInterval(0); servConf.setAuditorPeriodicPlacementPolicyCheckInterval(1000); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java new file mode 100644 index 00000000000..a2d5f6b2ea0 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java @@ -0,0 +1,810 @@ +/** + * + * 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.replication; + +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.net.URI; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.replication.AuditorPeriodicCheckTest.TestAuditor; +import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; +import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; +import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.collections4.map.MultiKeyMap; +import org.apache.commons.lang3.mutable.MutableObject; +import org.apache.zookeeper.KeeperException; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * Tests the logic of Auditor's ReplicasCheck. + */ +public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase { + private MetadataBookieDriver driver; + + public AuditorReplicasCheckTest() { + super(1); + baseConf.setPageLimit(1); // to make it easy to push ledger out of cache + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + StaticDNSResolver.reset(); + driver = MetadataDrivers.getBookieDriver(URI.create(bsConfs.get(0).getMetadataServiceUri())); + driver.initialize(bsConfs.get(0), () -> { + }, NullStatsLogger.INSTANCE); + } + + @After + @Override + public void tearDown() throws Exception { + if (null != driver) { + driver.close(); + } + super.tearDown(); + } + + private class TestBookKeeperAdmin extends BookKeeperAdmin { + + private final MultiKeyMap returnAvailabilityOfEntriesOfLedger; + private final MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger; + + public TestBookKeeperAdmin(BookKeeper bkc, StatsLogger statsLogger, + MultiKeyMap returnAvailabilityOfEntriesOfLedger, + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger) { + super(bkc, statsLogger); + this.returnAvailabilityOfEntriesOfLedger = returnAvailabilityOfEntriesOfLedger; + this.errorReturnValueForGetAvailabilityOfEntriesOfLedger = + errorReturnValueForGetAvailabilityOfEntriesOfLedger; + } + + @Override + public CompletableFuture asyncGetListOfEntriesOfLedger( + BookieSocketAddress address, long ledgerId) { + CompletableFuture futureResult = + new CompletableFuture(); + Integer errorReturnValue = errorReturnValueForGetAvailabilityOfEntriesOfLedger.get(address.toString(), + Long.toString(ledgerId)); + if (errorReturnValue != null) { + futureResult.completeExceptionally(BKException.create(errorReturnValue).fillInStackTrace()); + } else { + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = returnAvailabilityOfEntriesOfLedger + .get(address.toString(), Long.toString(ledgerId)); + futureResult.complete(availabilityOfEntriesOfLedger); + } + return futureResult; + } + } + + private TestStatsLogger startAuditorAndWaitForReplicasCheck(ServerConfiguration servConf, + MutableObject auditorRef, + MultiKeyMap expectedReturnAvailabilityOfEntriesOfLedger, + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger) + throws MetadataException, CompatibilityException, KeeperException, InterruptedException, + UnavailableException, UnknownHostException { + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + TestOpStatsLogger replicasCheckStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME); + + final TestAuditor auditor = new TestAuditor(Bookie.getBookieAddress(servConf).toString(), servConf, bkc, true, + new TestBookKeeperAdmin(bkc, statsLogger, expectedReturnAvailabilityOfEntriesOfLedger, + errorReturnValueForGetAvailabilityOfEntriesOfLedger), + true, statsLogger); + auditorRef.setValue(auditor); + CountDownLatch latch = auditor.getLatch(); + assertEquals("REPLICAS_CHECK_TIME SuccessCount", 0, replicasCheckStatsLogger.getSuccessCount()); + urm.setReplicasCheckCTime(-1); + auditor.start(); + /* + * since replicasCheckCTime is set to -1, replicasCheck should be + * scheduled to run with no initialdelay + */ + assertTrue("replicasCheck should have executed", latch.await(20, TimeUnit.SECONDS)); + for (int i = 0; i < 200; i++) { + Thread.sleep(100); + if (replicasCheckStatsLogger.getSuccessCount() >= 1) { + break; + } + } + assertEquals("REPLICAS_CHECK_TIME SuccessCount", 1, replicasCheckStatsLogger.getSuccessCount()); + return statsLogger; + } + + private void setServerConfigProperties(ServerConfiguration servConf) { + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(0); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(0); + servConf.setAuditorPeriodicReplicasCheckInterval(1000); + } + + List addAndRegisterBookies(RegistrationManager regManager, int numOfBookies) + throws BookieException { + BookieSocketAddress bookieAddress; + List bookieAddresses = new ArrayList(); + for (int i = 0; i < numOfBookies; i++) { + bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); + bookieAddresses.add(bookieAddress); + regManager.registerBookie(bookieAddress.toString(), false); + } + return bookieAddresses; + } + + private void createClosedLedgerMetadata(LedgerManager lm, long ledgerId, int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map> segmentEnsembles, long lastEntryId, int length, + DigestType digestType, byte[] password) throws InterruptedException, ExecutionException { + LedgerMetadataBuilder ledgerMetadataBuilder = LedgerMetadataBuilder.create(); + ledgerMetadataBuilder.withEnsembleSize(ensembleSize).withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize).withClosedState().withLastEntryId(lastEntryId).withLength(length) + .withDigestType(digestType).withPassword(password); + for (Map.Entry> mapEntry : segmentEnsembles.entrySet()) { + ledgerMetadataBuilder.newEnsembleEntry(mapEntry.getKey(), mapEntry.getValue()); + } + LedgerMetadata initMeta = ledgerMetadataBuilder.build(); + lm.createLedgerMetadata(ledgerId, initMeta).get(); + } + + private void createNonClosedLedgerMetadata(LedgerManager lm, long ledgerId, int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map> segmentEnsembles, DigestType digestType, + byte[] password) throws InterruptedException, ExecutionException { + LedgerMetadataBuilder ledgerMetadataBuilder = LedgerMetadataBuilder.create(); + ledgerMetadataBuilder.withEnsembleSize(ensembleSize).withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize).withDigestType(digestType).withPassword(password); + for (Map.Entry> mapEntry : segmentEnsembles.entrySet()) { + ledgerMetadataBuilder.newEnsembleEntry(mapEntry.getKey(), mapEntry.getValue()); + } + LedgerMetadata initMeta = ledgerMetadataBuilder.build(); + lm.createLedgerMetadata(ledgerId, initMeta).get(); + } + + private void runTestScenario(MultiKeyMap returnAvailabilityOfEntriesOfLedger, + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger, + int expectedNumLedgersFoundHavingNoReplicaOfAnEntry, + int expectedNumLedgersHavingLessThanAQReplicasOfAnEntry, + int expectedNumLedgersHavingLessThanWQReplicasOfAnEntry) throws CompatibilityException, + UnavailableException, UnknownHostException, MetadataException, KeeperException, InterruptedException { + ServerConfiguration servConf = new ServerConfiguration(bsConfs.get(0)); + setServerConfigProperties(servConf); + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForReplicasCheck(servConf, auditorRef, + returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger); + checkReplicasCheckStats(statsLogger, expectedNumLedgersFoundHavingNoReplicaOfAnEntry, + expectedNumLedgersHavingLessThanAQReplicasOfAnEntry, + expectedNumLedgersHavingLessThanWQReplicasOfAnEntry); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + } + } + + private void checkReplicasCheckStats(TestStatsLogger statsLogger, + int expectedNumLedgersFoundHavingNoReplicaOfAnEntry, + int expectedNumLedgersHavingLessThanAQReplicasOfAnEntry, + int expectedNumLedgersHavingLessThanWQReplicasOfAnEntry) { + Gauge numLedgersFoundHavingNoReplicaOfAnEntryGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_HAVING_NO_REPLICA_OF_AN_ENTRY); + Gauge numLedgersHavingLessThanAQReplicasOfAnEntryGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_HAVING_LESS_THAN_AQ_REPLICAS_OF_AN_ENTRY); + Gauge numLedgersHavingLessThanWQReplicasOfAnEntryGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_HAVING_LESS_THAN_WQ_REPLICAS_OF_AN_ENTRY); + + assertEquals("NUM_LEDGERS_HAVING_NO_REPLICA_OF_AN_ENTRY guage value", + expectedNumLedgersFoundHavingNoReplicaOfAnEntry, + numLedgersFoundHavingNoReplicaOfAnEntryGuage.getSample()); + assertEquals("NUM_LEDGERS_HAVING_LESS_THAN_AQ_REPLICAS_OF_AN_ENTRY guage value", + expectedNumLedgersHavingLessThanAQReplicasOfAnEntry, + numLedgersHavingLessThanAQReplicasOfAnEntryGuage.getSample()); + assertEquals("NUM_LEDGERS_HAVING_LESS_THAN_WQ_REPLICAS_OF_AN_ENTRY guage value", + expectedNumLedgersHavingLessThanWQReplicasOfAnEntry, + numLedgersHavingLessThanWQReplicasOfAnEntryGuage.getSample()); + } + + /* + * For all the ledgers and for all the bookies, + * asyncGetListOfEntriesOfLedger would return + * BookieHandleNotAvailableException, so these ledgers wouldn't be counted + * against expectedNumLedgersFoundHavingNoReplicaOfAnEntry / + * LessThanAQReplicasOfAnEntry / LessThanWQReplicasOfAnEntry. + */ + @Test + public void testReplicasCheckForBookieHandleNotAvailable() throws Exception { + int numOfBookies = 5; + RegistrationManager regManager = driver.getRegistrationManager(); + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + int ensembleSize = 5; + int writeQuorumSize = 4; + int ackQuorumSize = 2; + long lastEntryId = 100; + int length = 10000; + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + /* + * closed ledger + * + * for this ledger, for all the bookies we are setting + * errorReturnValueForGetAvailabilityOfEntriesOfLedger to + * BookieHandleNotAvailableException so asyncGetListOfEntriesOfLedger will + * return BookieHandleNotAvailableException. + */ + Map> segmentEnsembles = new HashMap>(); + segmentEnsembles.put(0L, bookieAddresses); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + } + + ensembleSize = 4; + /* + * closed ledger with multiple segments + * + * for this ledger, for all the bookies we are setting + * errorReturnValueForGetAvailabilityOfEntriesOfLedger to + * BookieHandleNotAvailableException so asyncGetListOfEntriesOfLedger will + * return BookieHandleNotAvailableException. + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(20L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(60L, bookieAddresses.subList(0, 4)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + } + + /* + * non-closed ledger + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + ledgerId = 3L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + } + + /* + * non-closed ledger with multiple segments + * + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(20L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(60L, bookieAddresses.subList(0, 4)); + ledgerId = 4L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + } + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, 0, 0, + 0); + } + + /* + * In this testscenario all the ledgers have a missing entry. So all closed + * ledgers should be counted towards + * numLedgersFoundHavingNoReplicaOfAnEntry. + */ + @Test + public void testReplicasCheckForLedgersFoundHavingNoReplica() throws Exception { + int numOfBookies = 5; + RegistrationManager regManager = driver.getRegistrationManager(); + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + int ensembleSize = 5; + int writeQuorumSize = 4; + int ackQuorumSize = 2; + long lastEntryId = 100; + int length = 10000; + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + int numLedgersFoundHavingNoReplicaOfAnEntry = 0; + + /* + * closed ledger + * + * for this ledger we are setting returnAvailabilityOfEntriesOfLedger to + * Empty one for all of the bookies, so this ledger would be counted in + * ledgersFoundHavingNoReplicaOfAnEntry . + */ + Map> segmentEnsembles = new HashMap>(); + segmentEnsembles.put(0L, bookieAddresses); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + returnAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + } + numLedgersFoundHavingNoReplicaOfAnEntry++; + + ensembleSize = 4; + /* + * closed ledger with multiple segments + * + * for this ledger we are setting + * errorReturnValueForGetAvailabilityOfEntriesOfLedger to + * NoSuchLedgerExistsException. This is equivalent to + * EMPTY_AVAILABILITYOFENTRIESOFLEDGER. So this ledger would be counted + * in ledgersFoundHavingNoReplicaOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(20L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(60L, bookieAddresses.subList(0, 4)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.NoSuchLedgerExistsException); + } + numLedgersFoundHavingNoReplicaOfAnEntry++; + + /* + * non-closed ledger + * + * since this is non-closed ledger, it should not be counted in + * ledgersFoundHavingNoReplicaOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + ledgerId = 3L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + returnAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + } + + ensembleSize = 3; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 1; + length = 1000; + /* + * closed ledger + * + * for this ledger we are setting returnAvailabilityOfEntriesOfLedger to + * just {0l} for all of the bookies and entry 1l is missing for all of + * the bookies, so this ledger would be counted in + * ledgersFoundHavingNoReplicaOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 3)); + ledgerId = 4L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + returnAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0L })); + } + numLedgersFoundHavingNoReplicaOfAnEntry++; + + /* + * For this closed ledger, entry 1 is missing. So it should be counted + * towards numLedgersFoundHavingNoReplicaOfAnEntry. + */ + ensembleSize = 4; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 3; + length = 10000; + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + ledgerId = 5L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 2, 3 })); + numLedgersFoundHavingNoReplicaOfAnEntry++; + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, + numLedgersFoundHavingNoReplicaOfAnEntry, 0, 0); + } + + /* + * In this testscenario all the ledgers have an entry with less than AQ + * number of copies. So all closed ledgers should be counted towards + * numLedgersFoundHavingLessThanAQReplicasOfAnEntry. + */ + @Test + public void testReplicasCheckForLedgersFoundHavingLessThanAQReplicasOfAnEntry() throws Exception { + int numOfBookies = 5; + RegistrationManager regManager = driver.getRegistrationManager(); + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + int numLedgersFoundHavingLessThanAQReplicasOfAnEntry = 0; + + /* + * closed ledger + * + * for this ledger there is only one copy of entry 2, so this ledger + * would be counted towards + * ledgersFoundHavingLessThanAQReplicasOfAnEntry. + */ + Map> segmentEnsembles = new HashMap>(); + int ensembleSize = 4; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + long lastEntryId = 3; + int length = 10000; + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2, 3 })); + numLedgersFoundHavingLessThanAQReplicasOfAnEntry++; + + /* + * closed ledger with multiple segments. + * + * for this ledger there is only one copy of entry 2, so this ledger + * would be counted towards + * ledgersFoundHavingLessThanAQReplicasOfAnEntry. + * + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] {})); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 3 })); + numLedgersFoundHavingLessThanAQReplicasOfAnEntry++; + + /* + * closed ledger with multiple segments + * + * for this ledger entry 2 is overrreplicated, but it has only one copy + * in the set of bookies it is supposed to be. So it should be counted + * towards ledgersFoundHavingLessThanAQReplicasOfAnEntry. + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 3L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 3 })); + numLedgersFoundHavingLessThanAQReplicasOfAnEntry++; + + /* + * non-closed ledger + * + * since this is non-closed ledger, it should not be counted towards + * ledgersFoundHavingLessThanAQReplicasOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 4L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] {})); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 3 })); + + /* + * this is closed ledger. + * + * For third bookie, asyncGetListOfEntriesOfLedger will fail with + * BookieHandleNotAvailableException, so this should not be counted + * against missing copies of an entry. Other than that, for both entries + * 0 and 1, two copies are missing. Hence this should be counted towards + * numLedgersFoundHavingLessThanAQReplicasOfAnEntry. + */ + ensembleSize = 3; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 1; + length = 1000; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 3)); + ledgerId = 5L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + numLedgersFoundHavingLessThanAQReplicasOfAnEntry++; + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, 0, + numLedgersFoundHavingLessThanAQReplicasOfAnEntry, 0); + } + + /* + * In this testscenario all the ledgers have an entry with less than WQ + * number of copies but greater than AQ. So all closed ledgers should be + * counted towards numLedgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + @Test + public void testReplicasCheckForLedgersFoundHavingLessThanWQReplicasOfAnEntry() throws Exception { + int numOfBookies = 5; + RegistrationManager regManager = driver.getRegistrationManager(); + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + int numLedgersFoundHavingLessThanWQReplicasOfAnEntry = 0; + + /* + * closed ledger + * + * for this ledger a copy of entry 3, so this ledger would be counted + * towards ledgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + Map> segmentEnsembles = new HashMap>(); + int ensembleSize = 4; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + long lastEntryId = 3; + int length = 10000; + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2, 3 })); + numLedgersFoundHavingLessThanWQReplicasOfAnEntry++; + + /* + * closed ledger with multiple segments + * + * for this ledger a copy of entry 0 and entry 2 are missing, so this + * ledger would be counted towards + * ledgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] {})); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 2, 3 })); + numLedgersFoundHavingLessThanWQReplicasOfAnEntry++; + + /* + * non-closed ledger with multiple segments + * + * since this is non-closed ledger, it should not be counted towards + * ledgersFoundHavingLessThanWQReplicasOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 3L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), + Long.toString(ledgerId), BKException.Code.NoSuchLedgerExistsException); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 2, 3 })); + + /* + * closed ledger. + * + * for this ledger entry 0 is overrreplicated, but a copy is missing in + * the set of bookies it is supposed to be. So it should be counted + * towards ledgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + ensembleSize = 4; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 1; + length = 1000; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + ledgerId = 4L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0 })); + numLedgersFoundHavingLessThanWQReplicasOfAnEntry++; + + /* + * this is closed ledger. + * + * For third bookie, asyncGetListOfEntriesOfLedger will fail with + * BookieHandleNotAvailableException, so this should not be counted + * against missing copies of an entry. Other than that, for both entries + * 0 and 1, a copy is missing. Hence this should be counted towards + * numLedgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + ensembleSize = 3; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 1; + length = 1000; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 3)); + ledgerId = 5L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + numLedgersFoundHavingLessThanWQReplicasOfAnEntry++; + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, 0, 0, + numLedgersFoundHavingLessThanWQReplicasOfAnEntry); + } +} From 2f4401c182c78346a008679a01eee3fce77ef30d Mon Sep 17 00:00:00 2001 From: Sergii Zhevzhyk Date: Fri, 23 Aug 2019 13:15:59 +0200 Subject: [PATCH 0330/1642] Simplify if statements ### Changes * Simplify `if` statements where it is possible (mainly for `if` statements returning boolean values) * Remove redundant semicolons * Remove the `close` method where the resource is used withing the try-with-resources block Reviewers: Enrico Olivelli , Sijie Guo This closes #2137 from vzhikserg/simplify-if-statements --- .../common/annotation/InterfaceAudience.java | 6 +++--- .../common/annotation/InterfaceStability.java | 6 +++--- .../conf/validators/RangeValidator.java | 4 +--- .../common/util/OrderedScheduler.java | 2 +- .../bookkeeper/auth/BookKeeperPrincipal.java | 5 +---- .../org/apache/bookkeeper/bookie/Bookie.java | 14 +++---------- .../apache/bookkeeper/bookie/BookieShell.java | 5 +---- .../bookkeeper/bookie/FileSystemUpgrade.java | 5 +---- .../org/apache/bookkeeper/bookie/Journal.java | 6 +----- .../bookkeeper/client/BookKeeperAdmin.java | 6 +----- .../bookkeeper/client/LedgerHandle.java | 2 +- .../bookkeeper/client/api/LedgerMetadata.java | 2 +- .../meta/AbstractZkLedgerManager.java | 21 ++++++++----------- .../meta/MSLedgerManagerFactory.java | 17 +++++++-------- .../meta/ZkLedgerUnderreplicationManager.java | 7 ++----- .../bookkeeper/net/NetworkTopologyImpl.java | 6 +----- .../streaming/LedgerOutputStream.java | 4 +--- .../cli/commands/bookie/LedgerCommand.java | 4 +--- .../org/apache/bookkeeper/auth/TestAuth.java | 1 - .../bookie/BookieDeferredSyncTest.java | 10 ++++----- .../bookkeeper/bookie/LedgerCacheTest.java | 2 +- .../bookkeeper/client/BookKeeperTest.java | 4 ++-- .../client/MetadataUpdateLoopTest.java | 4 ++-- .../RoundRobinDistributionScheduleTest.java | 6 +----- .../org/apache/bookkeeper/tls/TestTLS.java | 6 +++--- .../BKDistributedLogManager.java | 2 +- .../java/org/apache/distributedlog/DLSN.java | 6 +----- 27 files changed, 54 insertions(+), 109 deletions(-) diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/annotation/InterfaceAudience.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/annotation/InterfaceAudience.java index 40fcef66865..571dde3b96a 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/annotation/InterfaceAudience.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/annotation/InterfaceAudience.java @@ -34,7 +34,7 @@ public class InterfaceAudience { */ @Documented @Retention(RetentionPolicy.RUNTIME) - public @interface Public {}; + public @interface Public {} /** * Intended for use only within the project(s) specified in the annotation. @@ -42,14 +42,14 @@ public class InterfaceAudience { */ @Documented @Retention(RetentionPolicy.RUNTIME) - public @interface LimitedPrivate {}; + public @interface LimitedPrivate {} /** * Intended for use only within bookkeeper itself. */ @Documented @Retention(RetentionPolicy.RUNTIME) - public @interface Private {}; + public @interface Private {} private InterfaceAudience() {} // Audience can't exist on its own diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/annotation/InterfaceStability.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/annotation/InterfaceStability.java index 174afd530a4..e57f677c0f9 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/annotation/InterfaceStability.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/annotation/InterfaceStability.java @@ -32,20 +32,20 @@ public class InterfaceStability { * can break compatibility only at major release (ie. at m.0). */ @Documented - public @interface Stable {}; + public @interface Stable {} /** * Evolving, but can break compatibility at minor release (i.e. m.x) */ @Documented - public @interface Evolving {}; + public @interface Evolving {} /** * No guarantee is provided as to reliability or stability across any * level of release granularity. */ @Documented - public @interface Unstable {}; + public @interface Unstable {} private InterfaceStability() {} } diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java index 2dbadf4e28c..827f249cd55 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java @@ -68,10 +68,8 @@ public boolean validate(String name, Object value) { Number n = (Number) value; if (min != null && n.doubleValue() < min.doubleValue()) { return false; - } else if (max != null && n.doubleValue() > max.doubleValue()) { - return false; } else { - return true; + return max == null || !(n.doubleValue() > max.doubleValue()); } } else { return false; diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java index 979b55ded9d..1257f2ebc51 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedScheduler.java @@ -377,6 +377,6 @@ public ListenableFuture submit(Runnable task, T result) { public void execute(Runnable command) { super.execute(timedRunnable(command)); } - }; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/BookKeeperPrincipal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/BookKeeperPrincipal.java index 41eade56ed3..c09b9fd4f3b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/BookKeeperPrincipal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/BookKeeperPrincipal.java @@ -63,10 +63,7 @@ public boolean equals(Object obj) { return false; } final BookKeeperPrincipal other = (BookKeeperPrincipal) obj; - if (!Objects.equals(this.name, other.name)) { - return false; - } - return true; + return Objects.equals(this.name, other.name); } } 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 9bb9e5deaef..6125970997d 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 @@ -898,12 +898,8 @@ public void process(int journalVersion, long offset, ByteBuffer recBuff) throws */ private void replay(Journal journal, JournalScanner scanner) throws IOException { final LogMark markedLog = journal.getLastLogMark().getCurMark(); - List logs = Journal.listJournalIds(journal.getJournalDirectory(), journalId -> { - if (journalId < markedLog.getLogFileId()) { - return false; - } - return true; - }); + List logs = Journal.listJournalIds(journal.getJournalDirectory(), journalId -> + journalId >= markedLog.getLogFileId()); // last log mark may be missed due to no sync up before // validate filtered log ids only when we have markedLogId if (markedLog.getLogFileId() > 0) { @@ -1518,11 +1514,7 @@ public static boolean format(ServerConfiguration conf, if (!isInteractive) { // If non interactive and force is set, then delete old // data. - if (force) { - confirm = true; - } else { - confirm = false; - } + confirm = force; } else { confirm = IOUtils .confirmPrompt("Are you sure to format Bookie data..?"); 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 5149898f087..28ef757895a 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 @@ -2157,9 +2157,6 @@ private static boolean getOptionBooleanValue(CommandLine cmdLine, String option, } private static boolean getOptionalValue(String optValue, String optName) { - if (StringUtils.equals(optValue, optName)) { - return true; - } - return false; + return StringUtils.equals(optValue, optName); } } 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 7600accfd4a..9cd0090480c 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 @@ -91,10 +91,7 @@ public boolean accept(File dir, String name) { || name.equals("lastId") || name.startsWith("lastMark")) { return true; } - if (containsIndexFiles(dir, name)) { - return true; - } - return false; + return containsIndexFiles(dir, name); } }; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 4ee540c6916..54da3bbff17 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -273,11 +273,7 @@ private static class JournalRollingFilter implements JournalIdFilter { @Override public boolean accept(long journalId) { - if (journalId < lastMark.getCurMark().getLogFileId()) { - return true; - } else { - return false; - } + return journalId < lastMark.getCurMark().getLogFileId(); } } 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 870db93527a..bdf56b8ae43 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 @@ -1168,11 +1168,7 @@ public static boolean format(ServerConfiguration conf, if (ledgerRootExists) { if (!isInteractive) { // If non interactive and force is set, then delete old data. - if (force) { - doFormat = true; - } else { - doFormat = false; - } + doFormat = force; } else { // Confirm with the admin. doFormat = IOUtils 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 f3abef821b4..fd03fcc3c3f 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 @@ -107,7 +107,7 @@ public class LedgerHandle implements WriteHandle { private enum HandleState { OPEN, CLOSED - }; + } private HandleState handleState = HandleState.OPEN; private final CompletableFuture closePromise = new CompletableFuture<>(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java index de6cdf2c935..33e7da36b8b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java @@ -158,7 +158,7 @@ enum State { /** The ledger is closed. No new entries may be added to it. The length and lastEntryId are fixed. Ensembles may change, but only for rereplication. */ - CLOSED; + CLOSED } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index cd162626f30..fa64ada7a2c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -574,18 +574,15 @@ public void operationComplete(int rc, List ledgerNodes) { * Znode Name * @return true if the znode is a special znode otherwise false */ - public static boolean isSpecialZnode(String znode) { - if (BookKeeperConstants.AVAILABLE_NODE.equals(znode) - || BookKeeperConstants.COOKIE_NODE.equals(znode) - || BookKeeperConstants.LAYOUT_ZNODE.equals(znode) - || BookKeeperConstants.INSTANCEID.equals(znode) - || BookKeeperConstants.UNDER_REPLICATION_NODE.equals(znode) - || LegacyHierarchicalLedgerManager.IDGEN_ZNODE.equals(znode) - || LongHierarchicalLedgerManager.IDGEN_ZNODE.equals(znode) - || znode.startsWith(ZkLedgerIdGenerator.LEDGER_ID_GEN_PREFIX)) { - return true; - } - return false; + public static boolean isSpecialZnode(String znode) { + return BookKeeperConstants.AVAILABLE_NODE.equals(znode) + || BookKeeperConstants.COOKIE_NODE.equals(znode) + || BookKeeperConstants.LAYOUT_ZNODE.equals(znode) + || BookKeeperConstants.INSTANCEID.equals(znode) + || BookKeeperConstants.UNDER_REPLICATION_NODE.equals(znode) + || LegacyHierarchicalLedgerManager.IDGEN_ZNODE.equals(znode) + || LongHierarchicalLedgerManager.IDGEN_ZNODE.equals(znode) + || znode.startsWith(ZkLedgerIdGenerator.LEDGER_ID_GEN_PREFIX); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java index 9b21c876e34..08a33f82860 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java @@ -653,16 +653,13 @@ public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) { * Znode Name * @return true if the znode is a special znode otherwise false */ - public static boolean isSpecialZnode(String znode) { - if (BookKeeperConstants.AVAILABLE_NODE.equals(znode) - || BookKeeperConstants.COOKIE_NODE.equals(znode) - || BookKeeperConstants.LAYOUT_ZNODE.equals(znode) - || BookKeeperConstants.INSTANCEID.equals(znode) - || BookKeeperConstants.UNDER_REPLICATION_NODE.equals(znode) - || MsLedgerManager.IDGEN_ZNODE.equals(znode)) { - return true; - } - return false; + public static boolean isSpecialZnode(String znode) { + return BookKeeperConstants.AVAILABLE_NODE.equals(znode) + || BookKeeperConstants.COOKIE_NODE.equals(znode) + || BookKeeperConstants.LAYOUT_ZNODE.equals(znode) + || BookKeeperConstants.INSTANCEID.equals(znode) + || BookKeeperConstants.UNDER_REPLICATION_NODE.equals(znode) + || MsLedgerManager.IDGEN_ZNODE.equals(znode); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java index ec5b8db6aa9..7f3d1c2297d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java @@ -721,11 +721,8 @@ public boolean isLedgerReplicationEnabled() LOG.debug("isLedgerReplicationEnabled()"); } try { - if (null != zkc.exists(basePath + '/' - + BookKeeperConstants.DISABLE_NODE, false)) { - return false; - } - return true; + return null == zkc.exists(basePath + '/' + + BookKeeperConstants.DISABLE_NODE, false); } catch (KeeperException ke) { LOG.error("Error while checking the state of " + "ledger re-replication", ke); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java index dcf4cade3d0..0b8a02b74ce 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java @@ -109,11 +109,7 @@ boolean isRack() { } Node firstChild = children.get(0); - if (firstChild instanceof InnerNode) { - return false; - } - - return true; + return !(firstChild instanceof InnerNode); } /** 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 0d2720428a4..ed73293a546 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 @@ -107,9 +107,7 @@ private boolean makeSpace(int len) { if (bytebuff.remaining() < len) { flush(); bytebuff.clear(); - if (bytebuff.capacity() < len) { - return false; - } + return bytebuff.capacity() >= len; } return true; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java index faaefcb82a9..d158c5ab26a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/LedgerCommand.java @@ -94,9 +94,7 @@ public boolean apply(ServerConfiguration conf, LedgerFlags cmdFlags) { long ledgerId = cmdFlags.ledgerId; if (conf.getLedgerStorageClass().equals(DbLedgerStorage.class.getName())) { // dump ledger info - if (!dumpLedgerInfo(ledgerId, conf)) { - return false; - } + return dumpLedgerInfo(ledgerId, conf); } else if (conf.getLedgerStorageClass().equals(SortedLedgerStorage.class.getName()) || conf.getLedgerStorageClass().equals(InterleavedLedgerStorage.class.getName())) { ServerConfiguration tConf = new ServerConfiguration(conf); 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 bf7d84b9546..c435d99f094 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 @@ -100,7 +100,6 @@ private void connectAndWriteToBookie(ClientConfiguration conf, AtomicLong ledger PASSWD)) { ledgerWritten.set(l.getId()); l.addEntry(ENTRY); - l.close(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieDeferredSyncTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieDeferredSyncTest.java index cce445efb8b..11a181a2573 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieDeferredSyncTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieDeferredSyncTest.java @@ -68,7 +68,7 @@ public void testWriteAndRecovery() throws Exception { .withLedgerId(ledgerId) .withRecovery(true) .withPassword(new byte[0]) - .execute());) { + .execute())) { try (LedgerEntries entries = readLh.read(0, n - 1)) { for (int i = 0; i < n; i++) { @@ -121,7 +121,7 @@ private void testClose(boolean force) throws Exception { .withLedgerId(ledgerId) .withRecovery(true) .withPassword(new byte[0]) - .execute());) { + .execute())) { try (LedgerEntries entries = readLh.read(0, n - 1)) { for (int i = 0; i < n; i++) { @@ -143,7 +143,7 @@ private void testClose(boolean force) throws Exception { .withLedgerId(ledgerId) .withRecovery(true) .withPassword(new byte[0]) - .execute());) { + .execute())) { assertEquals(-1, readLh.getLastAddConfirmed()); // entry will be readable with readUnconfirmed @@ -176,15 +176,13 @@ private void testForce(EnumSet writeFlags) throws Exception { .withWriteFlags(writeFlags) .withDigestType(DigestType.CRC32C) .withPassword(new byte[0]) - .execute());) { + .execute())) { int n = 10; for (int i = 0; i < n; i++) { lh.append(("entry-" + i).getBytes(UTF_8)); } result(lh.force()); assertEquals(n - 1, lh.getLastAddConfirmed()); - - lh.close(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java index 405b93a8cce..8258f23d4c2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java @@ -793,7 +793,7 @@ public void testSortedLedgerFlushFailure() throws Exception { public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx){ LOG.info("fail write to bk"); assertTrue(rc != OK); - }; + } }, null, "passwd".getBytes()); bookie.shutdown(); 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 a2cd7a0842b..a00ec9af98b 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 @@ -810,7 +810,7 @@ public void readComplete(int rc, LedgerHandle lh, public void testCannotUseWriteFlagsOnV2Protocol() throws Exception { ClientConfiguration conf = new ClientConfiguration(baseClientConf); conf.setUseV2WireProtocol(true); - try (BookKeeperTestClient bkc = new BookKeeperTestClient(conf);) { + try (BookKeeperTestClient bkc = new BookKeeperTestClient(conf)) { try (WriteHandle wh = result(bkc.newCreateLedgerOp() .withEnsembleSize(3) .withWriteQuorumSize(3) @@ -827,7 +827,7 @@ public void testCannotUseWriteFlagsOnV2Protocol() throws Exception { public void testCannotUseForceOnV2Protocol() throws Exception { ClientConfiguration conf = new ClientConfiguration(baseClientConf); conf.setUseV2WireProtocol(true); - try (BookKeeperTestClient bkc = new BookKeeperTestClient(conf);) { + try (BookKeeperTestClient bkc = new BookKeeperTestClient(conf)) { try (WriteHandle wh = result(bkc.newCreateLedgerOp() .withEnsembleSize(3) .withWriteQuorumSize(3) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java index b96859b217a..e76ee545449 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java @@ -464,7 +464,7 @@ public synchronized CompletableFuture> writeLedgerMeta } }); return promise; - }; + } } @Data @@ -509,6 +509,6 @@ public synchronized CompletableFuture> writeLedgerMeta } else { return super.writeLedgerMetadata(ledgerId, metadata, currentVersion); } - }; + } } } 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 4498eeaf931..1b989f5efc7 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 @@ -93,11 +93,7 @@ public void testCoverageSets() { boolean[] buildAvailable(int ensemble, Set responses) { boolean[] available = new boolean[ensemble]; for (int i = 0; i < ensemble; i++) { - if (responses.contains(i)) { - available[i] = false; - } else { - available[i] = true; - } + available[i] = !responses.contains(i); } return available; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java index 306a7f6a6b7..b353e73c201 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java @@ -292,13 +292,13 @@ private LedgerMetadata testClient(BookKeeper client, int clusterSize) throws Exc int numEntries = 100; long lid; byte[] testEntry = "testEntry".getBytes(); - try (LedgerHandle lh = client.createLedger(clusterSize, clusterSize, DigestType.CRC32, passwd);) { + try (LedgerHandle lh = client.createLedger(clusterSize, clusterSize, DigestType.CRC32, passwd)) { for (int i = 0; i <= numEntries; i++) { lh.addEntry(testEntry); } lid = lh.getId(); } - try (LedgerHandle lh = client.openLedger(lid, DigestType.CRC32, passwd);) { + try (LedgerHandle lh = client.openLedger(lid, DigestType.CRC32, passwd)) { Enumeration entries = lh.readEntries(0, numEntries); while (entries.hasMoreElements()) { LedgerEntry e = entries.nextElement(); @@ -310,7 +310,7 @@ private LedgerMetadata testClient(BookKeeper client, int clusterSize) throws Exc } private LedgerMetadata testClient(ClientConfiguration conf, int clusterSize) throws Exception { - try (BookKeeper client = new BookKeeper(conf);) { + try (BookKeeper client = new BookKeeper(conf)) { return testClient(client, clusterSize); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java index 8a79d8b404a..6392021b534 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java @@ -1031,7 +1031,7 @@ public CompletableFuture asyncClose() { return null; }); } - }; + } /** * Close the distributed log manager, freeing any resources it may hold. diff --git a/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/DLSN.java b/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/DLSN.java index 8a496ab06cd..11699da0ea8 100644 --- a/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/DLSN.java +++ b/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/DLSN.java @@ -225,11 +225,7 @@ public boolean equals(Object o) { if (logSegmentSequenceNo != dlsn.logSegmentSequenceNo) { return false; } - if (slotId != dlsn.slotId) { - return false; - } - - return true; + return slotId == dlsn.slotId; } @Override From 836e4181968c9f5932d59a6454521edfc2e61e88 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 26 Aug 2019 11:15:15 -0700 Subject: [PATCH 0331/1642] Journal should respect to `flushWhenQueueEmpty` setting Descriptions of the changes in this PR: *Motivation* Currently journal doesn't respect to `flushWhenQueueEmpty` setting. Even `flushWhenQueueEmpty` is set to false, we can still see flushes triggered due to queue empty. Reviewers: Enrico Olivelli , Jia Zhai This closes #2147 from sijie/fix_flush_when_empty --- .../main/java/org/apache/bookkeeper/bookie/Journal.java | 9 ++++++--- .../core/src/test/resources/bk_server.conf | 1 - 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 54da3bbff17..4af728ae341 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -983,8 +983,9 @@ public void run() { if (maxGroupWaitInNanos > 0 && !groupWhenTimeout && (MathUtils .elapsedNanos(toFlush.get(0).enqueueTime) > maxGroupWaitInNanos)) { groupWhenTimeout = true; - } else if (maxGroupWaitInNanos > 0 && groupWhenTimeout && qe != null - && MathUtils.elapsedNanos(qe.enqueueTime) < maxGroupWaitInNanos) { + } else if (maxGroupWaitInNanos > 0 && groupWhenTimeout + && (qe == null // no entry to group + || MathUtils.elapsedNanos(qe.enqueueTime) < maxGroupWaitInNanos)) { // when group timeout, it would be better to look forward, as there might be lots of // entries already timeout // due to a previous slow write (writing to filesystem which impacted by force write). @@ -998,13 +999,15 @@ public void run() { && ((bufferedEntriesThreshold > 0 && toFlush.size() > bufferedEntriesThreshold) || (bc.position() > lastFlushPosition + bufferedWritesThreshold))) { // 2. If we have buffered more than the buffWriteThreshold or bufferedEntriesThreshold + groupWhenTimeout = false; shouldFlush = true; journalStats.getFlushMaxOutstandingBytesCounter().inc(); - } else if (qe == null) { + } else if (qe == null && flushWhenQueueEmpty) { // We should get here only if we flushWhenQueueEmpty is true else we would wait // for timeout that would put is past the maxWait threshold // 3. If the queue is empty i.e. no benefit of grouping. This happens when we have one // publish at a time - common case in tests. + groupWhenTimeout = false; shouldFlush = true; journalStats.getFlushEmptyQueueCounter().inc(); } diff --git a/stream/distributedlog/core/src/test/resources/bk_server.conf b/stream/distributedlog/core/src/test/resources/bk_server.conf index 0a746d68090..0d3cd56d4cb 100644 --- a/stream/distributedlog/core/src/test/resources/bk_server.conf +++ b/stream/distributedlog/core/src/test/resources/bk_server.conf @@ -119,7 +119,6 @@ readOnlyModeEnabled=true # Bookie Journal Settings writeBufferSizeBytes=524288 -journalFlushWhenQueueEmpty=false journalRemoveFromPageCache=true journalAdaptiveGroupWrites=true journalMaxGroupWaitMSec=2 From d6ada0fa9d3b15290ac3b89b70bcd57c49c1d41f Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 26 Aug 2019 11:16:08 -0700 Subject: [PATCH 0332/1642] Improve journal throughput when journalSyncData is disabled. Descriptions of the changes in this PR: ### Motivation There was a weird behavior observed: when `journalSyncData` is disabled, the journal throughput decreased with multiple journal dirs on same journal disk. The problem is when `journalSyncData` is disabled, we still sync data to disk when rolling the journal file. The sync time increases to multiple seconds, because it has to flush all the files to disk and wait the whole sync operation to be completed. ### Changes when `journalSyncData` is disabled, we trigger callbacks when the data is flushed to filesystem but still enqueue a fsync request to force write queue to make sure we still fsync data frequently. Also added a journal perf tool to do performance testing on journal. Reviewers: Enrico Olivelli , Jia Zhai , Matteo Merli This closes #2150 from sijie/journal_perf --- bin/bkperf | 2 +- .../org/apache/bookkeeper/bookie/Journal.java | 27 +- .../bookkeeper/conf/ServerConfiguration.java | 22 + tools/perf/pom.xml | 5 + .../apache/bookkeeper/tools/perf/BKPerf.java | 3 +- .../tools/perf/JournalPerfCommandGroup.java | 41 ++ .../tools/perf/journal/AppendCommand.java | 51 ++ .../tools/perf/journal/JournalWriter.java | 518 ++++++++++++++++++ .../tools/perf/journal/package-info.java | 18 + 9 files changed, 665 insertions(+), 22 deletions(-) create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/JournalPerfCommandGroup.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/AppendCommand.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/JournalWriter.java create mode 100644 tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/package-info.java diff --git a/bin/bkperf b/bin/bkperf index 6ec2909989d..857eda4d5fd 100755 --- a/bin/bkperf +++ b/bin/bkperf @@ -47,7 +47,7 @@ if [ -z "${CLI_LOG_CONF}" ]; then fi CLI_LOG_DIR=${CLI_LOG_DIR:-"$BK_HOME/logs"} CLI_LOG_FILE=${CLI_LOG_FILE:-"bkperf.log"} -CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,ROLLINGFILE"} +CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,CONSOLE"} # Configure the classpath CLI_CLASSPATH="$CLI_JAR:$CLI_CLASSPATH:$CLI_EXTRA_CLASSPATH" diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 4af728ae341..19d5a7961f7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -852,7 +852,8 @@ public void logAddEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, } @VisibleForTesting - void logAddEntry(long ledgerId, long entryId, ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Object ctx) + public void logAddEntry(long ledgerId, long entryId, ByteBuf entry, + boolean ackBeforeSync, WriteCallback cb, Object ctx) throws InterruptedException { //Retain entry until it gets written to journal entry.retain(); @@ -1049,25 +1050,11 @@ public void run() { .registerSuccessfulValue(batchSize); boolean shouldRolloverJournal = (lastFlushPosition > maxJournalSize); - if (syncData) { - // Trigger data sync to disk in the "Force-Write" thread. - // Callback will be triggered after data is committed to disk - forceWriteRequests.put(createForceWriteRequest(logFile, logId, lastFlushPosition, - toFlush, shouldRolloverJournal, false)); - toFlush = entryListRecycler.newInstance(); - numEntriesToFlush = 0; - } else { - // Data is already written on the file (though it might still be in the OS page-cache) - lastLogMark.setCurLogMark(logId, lastFlushPosition); - toFlush.clear(); - numEntriesToFlush = 0; - if (shouldRolloverJournal) { - forceWriteRequests.put( - createForceWriteRequest( - logFile, logId, lastFlushPosition, - EMPTY_ARRAY_LIST, shouldRolloverJournal, false)); - } - } + // Trigger data sync to disk in the "Force-Write" thread. + forceWriteRequests.put(createForceWriteRequest(logFile, logId, lastFlushPosition, + toFlush, shouldRolloverJournal, false)); + toFlush = entryListRecycler.newInstance(); + numEntriesToFlush = 0; batchSize = 0L; // check whether journal file is over file limit 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 75e230ca7b8..9a2c42d317b 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 @@ -712,6 +712,17 @@ public int getJournalWriteBufferSizeKB() { return this.getInt(JOURNAL_WRITE_BUFFER_SIZE, 64); } + /** + * Set the size of the write buffers used for the journal. + * + * @param bufferSizeKB the size of the write buffer used for the journal, in KB. + * @return server configuration + */ + public ServerConfiguration setJournalWriteBufferSizeKB(int bufferSizeKB) { + setProperty(JOURNAL_WRITE_BUFFER_SIZE, bufferSizeKB); + return this; + } + /** * Max number of older journal files kept. * @@ -1971,6 +1982,17 @@ public long getJournalBufferedWritesThreshold() { return getLong(JOURNAL_BUFFERED_WRITES_THRESHOLD, 512 * 1024); } + /** + * Set maximum bytes to buffer to impose on a journal write to achieve grouping. + * + * @param maxBytes maximum bytes to buffer to impose on a journal write + * @return max bytes to buffer + */ + public ServerConfiguration setJournalBufferedWritesThreshold(long maxBytes) { + setProperty(JOURNAL_BUFFERED_WRITES_THRESHOLD, maxBytes); + return this; + } + /** * Maximum entries to buffer to impose on a journal write to achieve grouping. * Use {@link #getJournalBufferedWritesThreshold()} if this is set to zero or diff --git a/tools/perf/pom.xml b/tools/perf/pom.xml index ddbbf37030c..cb046204729 100644 --- a/tools/perf/pom.xml +++ b/tools/perf/pom.xml @@ -39,6 +39,11 @@ stream-storage-java-client ${project.version} + + org.apache.bookkeeper.stats + prometheus-metrics-provider + ${project.version} + org.hdrhistogram HdrHistogram diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java index 3aac74eef8b..e7ad543aaec 100644 --- a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/BKPerf.java @@ -36,7 +36,8 @@ public static void main(String[] args) { .withFlags(new BKFlags()) .withConsole(System.out) .addCommand(new DlogPerfCommandGroup()) - .addCommand(new TablePerfCommandGroup()); + .addCommand(new TablePerfCommandGroup()) + .addCommand(new JournalPerfCommandGroup()); CliSpec spec = specBuilder.build(); diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/JournalPerfCommandGroup.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/JournalPerfCommandGroup.java new file mode 100644 index 00000000000..982ed5d08fd --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/JournalPerfCommandGroup.java @@ -0,0 +1,41 @@ +/* + * Licensed 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.tools.perf; + +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliCommandGroup; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.tools.perf.journal.AppendCommand; + +/** + * Commands that evaluate performance of the bookkeeper journal library. + */ +public class JournalPerfCommandGroup extends CliCommandGroup implements PerfCommandGroup { + + private static final String NAME = "journal"; + private static final String DESC = "Commands on evaluating performance of journal library"; + + private static final CliSpec spec = CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withParent(BKPerf.NAME) + .addCommand(new AppendCommand()) + .build(); + + public JournalPerfCommandGroup() { + super(spec); + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/AppendCommand.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/AppendCommand.java new file mode 100644 index 00000000000..4d04961bed6 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/AppendCommand.java @@ -0,0 +1,51 @@ +/* + * Licensed 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.tools.perf.journal; + +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.net.ServiceURI; +import org.apache.bookkeeper.tools.common.BKCommand; +import org.apache.bookkeeper.tools.common.BKFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.tools.perf.journal.JournalWriter.Flags; +import org.apache.commons.configuration.CompositeConfiguration; + +/** + * Command to write log records to distributedlog streams. + */ +@Slf4j +public class AppendCommand extends BKCommand { + + private static final String NAME = "append"; + private static final String DESC = "Append entries to a bookie journal"; + + public AppendCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new Flags()) + .build()); + } + + @Override + protected boolean apply(ServiceURI serviceURI, + CompositeConfiguration conf, + BKFlags globalFlags, Flags cmdFlags) { + JournalWriter writer = new JournalWriter(conf, cmdFlags); + writer.run(); + return true; + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/JournalWriter.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/JournalWriter.java new file mode 100644 index 00000000000..1f38c426bf6 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/JournalWriter.java @@ -0,0 +1,518 @@ +/* + * Licensed 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.tools.perf.journal; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_SCOPE; + +import com.beust.jcommander.Parameter; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.common.util.concurrent.RateLimiter; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.Unpooled; +import java.io.File; +import java.io.IOException; +import java.text.DecimalFormat; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.LongAdder; +import lombok.extern.slf4j.Slf4j; +import org.HdrHistogram.Histogram; +import org.HdrHistogram.Recorder; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; +import org.apache.bookkeeper.bookie.Journal; +import org.apache.bookkeeper.bookie.LedgerDirsManager; +import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.Stats; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.perf.utils.PaddingDecimalFormat; +import org.apache.bookkeeper.util.DiskChecker; +import org.apache.commons.configuration.CompositeConfiguration; + +/** + * A perf writer to evaluate write performance. + */ +@Slf4j +public class JournalWriter implements Runnable { + + /** + * Flags for the write command. + */ + public static class Flags extends CliFlags { + + @Parameter( + names = { + "-t", "--num-test-threads" + }, + description = "Num of test threads to append entries to journal" + ) + public int numTestThreads = 1; + + @Parameter( + names = { + "-nl", "--num-ledgers" + }, + description = "Num of ledgers to append entries to journal" + ) + public int numLedgers = 24; + + @Parameter( + names = { + "-r", "--rate" + }, + description = "Write rate bytes/s across journals") + public int writeRate = 0; + + @Parameter( + names = { + "-s", "--entry-size" + }, + description = "Entry size") + public int recordSize = 1024; + + @Parameter( + names = { + "-j", "--journal-dirs" + }, + description = "The list of journal directories, separated by comma", + required = true) + public List journalDirs; + + @Parameter( + names = { + "-mob", "--max-outstanding-megabytes" + }, + description = "Number of threads writing") + public long maxOutstandingMB = 200; + + @Parameter( + names = { + "-n", "--num-entries" + }, + description = "Number of entries to write in total. If 0, it will keep writing") + public long numEntries = 0; + + @Parameter( + names = { + "-b", "--num-bytes" + }, + description = "Number of bytes to write in total. If 0, it will keep writing") + public long numBytes = 0; + + @Parameter( + names = { + "-wb", "--write-buffer-size-kb" + }, + description = "Journal write buffer size" + ) + public int writeBufferSizeKB = 1024; + + @Parameter( + names = { + "--sync" + }, + description = "Journal sync enabled" + ) + public boolean journalSyncEnabled = false; + + @Parameter( + names = { + "-gci", "--group-commit-interval-ms" + }, + description = "Journal group commit interval in milliseconds" + ) + public int groupCommitIntervalMs = 1; + + @Parameter( + names = { + "-gcb", "--group-commit-max-bytes" + }, + description = "Journal group commit max buffered bytes" + ) + public int groupCommitMaxBytes = 512 * 1024; + + @Parameter( + names = { + "-q", "--journal-queue-size" + }, + description = "Journal queue size" + ) + public int journalQueueSize = 10000; + + @Parameter( + names = { + "-jt", "--num-journal-callback-threads" + }, + description = "Number of journal callback threads" + ) + public int numJournalCallbackThreads = 8; + + } + + + // stats + private final LongAdder recordsWritten = new LongAdder(); + private final LongAdder bytesWritten = new LongAdder(); + + private final ServerConfiguration conf; + private final Flags flags; + private final Recorder recorder = new Recorder( + TimeUnit.SECONDS.toMillis(120000), 5 + ); + private final Recorder cumulativeRecorder = new Recorder( + TimeUnit.SECONDS.toMillis(120000), 5 + ); + private final AtomicBoolean isDone = new AtomicBoolean(false); + + JournalWriter(CompositeConfiguration conf, Flags flags) { + this.conf = new ServerConfiguration(); + this.conf.addConfiguration(conf); + this.flags = flags; + } + + @Override + public void run() { + try { + execute(); + } catch (Exception e) { + log.error("Encountered exception at running dlog perf writer", e); + } + } + + void execute() throws Exception { + ObjectMapper m = new ObjectMapper(); + ObjectWriter w = m.writerWithDefaultPrettyPrinter(); + log.info("Starting journal perf writer with config : {}", w.writeValueAsString(flags)); + + checkArgument(flags.journalDirs.size() > 0, "No journal dirs is provided"); + + updateServerConf(conf, flags); + + log.info("Benchmark the journal perf with server config : {}", + conf.asJson()); + + Stats.loadStatsProvider(conf); + Stats.get().start(conf); + + StatsLogger statsLogger = Stats.get().getStatsLogger("") + .scope(BOOKIE_SCOPE); + + + ByteBufAllocator allocator = getAllocator(conf); + DiskChecker checker = new DiskChecker( + conf.getDiskUsageThreshold(), + conf.getDiskUsageWarnThreshold() + ); + LedgerDirsManager manager = new LedgerDirsManager( + conf, + conf.getLedgerDirs(), + checker, + NullStatsLogger.INSTANCE + ); + Journal[] journals = new Journal[flags.journalDirs.size()]; + for (int i = 0; i < journals.length; i++) { + Journal journal = new Journal( + i, + new File(flags.journalDirs.get(i)), + conf, + manager, + statsLogger.scope(JOURNAL_SCOPE), + allocator); + journals[i] = journal; + journal.start(); + } + try { + execute(journals); + } finally { + for (Journal journal : journals) { + journal.shutdown(); + } + + Stats.get().stop(); + } + } + + void execute(Journal[] journals) throws Exception { + // register shutdown hook to aggregate stats + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + isDone.set(true); + printAggregatedStats(cumulativeRecorder); + })); + + ScheduledExecutorService flushExecutor = Executors.newSingleThreadScheduledExecutor(); + flushExecutor.scheduleAtFixedRate(() -> { + for (Journal journal : journals) { + Checkpoint cp = journal.newCheckpoint(); + try { + journal.checkpointComplete(cp, true); + } catch (IOException e) { + log.error("Failed to complete checkpoint {}", cp, e); + } + } + }, 30L, 30L, TimeUnit.SECONDS); + + ExecutorService executor = Executors.newFixedThreadPool(flags.numTestThreads); + try { + for (int i = 0; i < flags.numTestThreads; i++) { + final int idx = i; + final long numRecordsForThisThread = flags.numEntries / flags.numTestThreads; + final long numBytesForThisThread = flags.numBytes / flags.numTestThreads; + final double writeRateForThisThread = flags.writeRate / (double) flags.numTestThreads; + final long maxOutstandingBytesForThisThread = + flags.maxOutstandingMB * 1024 * 1024 / flags.numTestThreads; + final int numLedgersForThisThread = flags.numLedgers / flags.numTestThreads; + executor.submit(() -> { + try { + write( + idx, + journals, + numLedgersForThisThread, + writeRateForThisThread, + (int) maxOutstandingBytesForThisThread, + numRecordsForThisThread, + numBytesForThisThread); + } catch (Throwable t) { + log.error("Encountered error at writing records", t); + } + }); + } + log.info("Started {} write threads", flags.numTestThreads); + reportStats(); + } finally { + flushExecutor.shutdown(); + executor.shutdown(); + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } + } + + void write(int threadIdx, + Journal[] journals, + int numLedgersForThisThread, + double writeRate, + int maxOutstandingBytesForThisThread, + long numRecordsForThisThread, + long numBytesForThisThread) throws Exception { + log.info("Write thread {} started with : rate = {}," + + " num records = {}, num bytes = {}, max outstanding bytes = {}", + threadIdx, + writeRate, + numRecordsForThisThread, + numBytesForThisThread, + maxOutstandingBytesForThisThread); + + RateLimiter limiter; + if (writeRate > 0) { + limiter = RateLimiter.create(writeRate); + } else { + limiter = null; + } + final Semaphore semaphore; + if (maxOutstandingBytesForThisThread > 0) { + semaphore = new Semaphore(maxOutstandingBytesForThisThread); + } else { + semaphore = null; + } + + // Acquire 1 second worth of records to have a slower ramp-up + if (limiter != null) { + limiter.acquire((int) writeRate); + } + + long totalWritten = 0L; + long totalBytesWritten = 0L; + final int numJournals = journals.length; + byte[] payload = new byte[flags.recordSize]; + ThreadLocalRandom.current().nextBytes(payload); + ByteBuf payloadBuf = Unpooled.wrappedBuffer(payload); + long[] entryIds = new long[numLedgersForThisThread]; + Arrays.fill(entryIds, 0L); + while (true) { + for (int i = 0; i < numJournals; i++) { + int ledgerIdx = ThreadLocalRandom.current().nextInt(numLedgersForThisThread); + long lid = threadIdx * numLedgersForThisThread + ledgerIdx; + long eid = entryIds[ledgerIdx]++; + ByteBuf buf = payloadBuf.retainedDuplicate(); + int len = buf.readableBytes(); + + if (numRecordsForThisThread > 0 + && totalWritten >= numRecordsForThisThread) { + markPerfDone(); + } + if (numBytesForThisThread > 0 + && totalBytesWritten >= numBytesForThisThread) { + markPerfDone(); + } + if (null != semaphore) { + semaphore.acquire(len); + } + + totalWritten++; + totalBytesWritten += len; + if (null != limiter) { + limiter.acquire(len); + } + final long sendTime = System.nanoTime(); + journals[i].logAddEntry( + lid, + eid, + buf, + false, + (rc, ledgerId, entryId, addr, ctx) -> { + buf.release(); + if (0 == rc) { + if (null != semaphore) { + semaphore.release(len); + } + + recordsWritten.increment(); + bytesWritten.add(len); + + long latencyMicros = TimeUnit.NANOSECONDS.toMicros( + System.nanoTime() - sendTime + ); + recorder.recordValue(latencyMicros); + cumulativeRecorder.recordValue(latencyMicros); + } else { + log.warn("Error at writing records : {}", BookieException.create(rc)); + Runtime.getRuntime().exit(-1); + } + }, + null + ); + } + } + } + + @SuppressFBWarnings("DM_EXIT") + void markPerfDone() throws Exception { + log.info("------------------- DONE -----------------------"); + printAggregatedStats(cumulativeRecorder); + isDone.set(true); + Thread.sleep(5000); + System.exit(0); + } + + void reportStats() { + // Print report stats + long oldTime = System.nanoTime(); + + Histogram reportHistogram = null; + + while (true) { + try { + Thread.sleep(10000); + } catch (InterruptedException e) { + break; + } + + if (isDone.get()) { + break; + } + + long now = System.nanoTime(); + double elapsed = (now - oldTime) / 1e9; + + double rate = recordsWritten.sumThenReset() / elapsed; + double throughput = bytesWritten.sumThenReset() / elapsed / 1024 / 1024; + + reportHistogram = recorder.getIntervalHistogram(reportHistogram); + + log.info( + "Throughput written : {} records/s --- {} MB/s --- Latency: mean:" + + " {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}", + throughputFormat.format(rate), throughputFormat.format(throughput), + dec.format(reportHistogram.getMean() / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(50) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(95) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.9) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.99) / 1000.0), + dec.format(reportHistogram.getMaxValue() / 1000.0)); + + reportHistogram.reset(); + + oldTime = now; + } + + } + + private static void updateServerConf(ServerConfiguration conf, Flags flags) { + + conf.setJournalWriteBufferSizeKB(flags.writeBufferSizeKB); + conf.setJournalMaxGroupWaitMSec(flags.groupCommitIntervalMs); + conf.setJournalBufferedWritesThreshold(flags.groupCommitMaxBytes); + conf.setNumJournalCallbackThreads(flags.numJournalCallbackThreads); + conf.setJournalQueueSize(flags.journalQueueSize); + conf.setJournalSyncData(flags.journalSyncEnabled); + conf.setLedgerDirNames(flags.journalDirs.toArray(new String[0])); + conf.setStatsProviderClass(PrometheusMetricsProvider.class); + File[] currentDirs = Bookie.getCurrentDirectories(conf.getLedgerDirs()); + for (File dir : currentDirs) { + if (dir.mkdirs()) { + log.info("Successfully created dir {}", dir); + } + } + } + + private static ByteBufAllocator getAllocator(ServerConfiguration conf) { + return ByteBufAllocatorBuilder.create() + .poolingPolicy(conf.getAllocatorPoolingPolicy()) + .poolingConcurrency(conf.getAllocatorPoolingConcurrency()) + .outOfMemoryPolicy(conf.getAllocatorOutOfMemoryPolicy()) + .outOfMemoryListener((ex) -> { + log.error("Unable to allocate memory, exiting bookie", ex); + }) + .leakDetectionPolicy(conf.getAllocatorLeakDetectionPolicy()) + .build(); + } + + private static final DecimalFormat throughputFormat = new PaddingDecimalFormat("0.0", 8); + private static final DecimalFormat dec = new PaddingDecimalFormat("0.000", 7); + + private static void printAggregatedStats(Recorder recorder) { + Histogram reportHistogram = recorder.getIntervalHistogram(); + + log.info("Aggregated latency stats --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {}" + + " - 99.9pct: {} - 99.99pct: {} - 99.999pct: {} - Max: {}", + dec.format(reportHistogram.getMean() / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(50) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(95) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.9) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.99) / 1000.0), + dec.format(reportHistogram.getValueAtPercentile(99.999) / 1000.0), + dec.format(reportHistogram.getMaxValue() / 1000.0)); + } + +} diff --git a/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/package-info.java b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/package-info.java new file mode 100644 index 00000000000..0f432e905f9 --- /dev/null +++ b/tools/perf/src/main/java/org/apache/bookkeeper/tools/perf/journal/package-info.java @@ -0,0 +1,18 @@ +/* + * Licensed 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. + */ + +/** + * BookKeeper Journal related perf command. + */ +package org.apache.bookkeeper.tools.perf.journal; \ No newline at end of file From 797f5675e8228c47ee6a49b53c6f3bd4088686cb Mon Sep 17 00:00:00 2001 From: Fangbin Sun Date: Tue, 27 Aug 2019 13:47:47 +0800 Subject: [PATCH 0333/1642] Issue #2152: Amend disk usage warn log ### Motivation Fixes #2152 ### Changes Replace `diskUsageThreshold` by `diskUsageWarnThreshold` when disk usage threshold doesn't trigger first. Reviewers: Enrico Olivelli , Sijie Guo This closes #2153 from murong00/branch-2153, closes #2152 --- .../src/main/java/org/apache/bookkeeper/util/DiskChecker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DiskChecker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DiskChecker.java index f03d47519d9..a2e7017c26b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DiskChecker.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DiskChecker.java @@ -161,7 +161,7 @@ float checkDiskFull(File dir) throws DiskOutOfSpaceException, DiskWarnThresholdE // Warn should be triggered only if disk usage threshold doesn't trigger first. if (used > diskUsageWarnThreshold) { LOG.warn("Space left on device {} : {}, Used space fraction: {} > WarnThreshold {}.", - dir, usableSpace, used, diskUsageThreshold); + dir, usableSpace, used, diskUsageWarnThreshold); throw new DiskWarnThresholdException("Space left on device:" + usableSpace + " Used space fraction:" + used + " > WarnThreshold:" + diskUsageWarnThreshold, used); From 16e7038f4b1f3d25af95f12c0e9ceb56610a3eba Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 29 Aug 2019 11:05:45 -0700 Subject: [PATCH 0334/1642] [BOOKIE] auto refresh tls cert at bookie-server ### Motivation Right now, if certs are rotated then bookie doesn't pick up refresh cert and we have to restart the bookie. So, we need a way to auto refresh certs into bookie. ### Modification - add configuration `tlsCertFilesRefreshDurationSeconds` to auto refresh cert - bookie checks last modified date of the cert after auto-refresh duration and refresh certs if requires ### Result - Bookie can refresh certs without restart. Reviewers: Enrico Olivelli , Sijie Guo This closes #2154 from rdhabalia/auto_mtls --- .../conf/AbstractConfiguration.java | 25 +++++++ .../tls/FileModifiedTimeUpdater.java | 66 +++++++++++++++++++ .../bookkeeper/tls/TLSContextFactory.java | 59 ++++++++++++++--- .../org/apache/bookkeeper/tls/TestTLS.java | 46 +++++++++++++ conf/bk_server.conf | 2 + 5 files changed, 188 insertions(+), 10 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/FileModifiedTimeUpdater.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 4886d75bba3..d3ce780578c 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 @@ -117,6 +117,7 @@ public abstract class AbstractConfiguration protected static final Class DEFAULT_LEDGERID_FORMATTER = LedgerIdFormatter.LongLedgerIdFormatter.class; + protected static final String TLS_CERT_FILES_REFRESH_DURATION_SECONDS = "tlsCertFilesRefreshDurationSeconds"; /** * This list will be passed to {@link SSLEngine#setEnabledCipherSuites(java.lang.String[]) }. * Please refer to official JDK JavaDocs @@ -782,6 +783,30 @@ public T setTLSClientAuthentication(boolean enabled) { return getThis(); } + /** + * Set tls certificate files refresh duration in seconds. + * + * @param certFilesRefreshSec + * tls certificate files refresh duration in seconds (set 0 to + * disable auto refresh) + * @return current configuration + */ + public T setTLSCertFilesRefreshDurationSeconds(long certFilesRefreshSec) { + setProperty(TLS_CERT_FILES_REFRESH_DURATION_SECONDS, certFilesRefreshSec); + return getThis(); + } + + /** + * Get tls certificate files refresh duration in seconds. + * + * @return tls certificate files refresh duration in seconds. Default 0 + * to disable auto refresh. + * + */ + public long getTLSCertFilesRefreshDurationSeconds() { + return getLong(TLS_CERT_FILES_REFRESH_DURATION_SECONDS, 0); + } + /** * Set the list of enabled TLS cipher suites. Leave null not to override default JDK list. This list will be passed * to {@link SSLEngine#setEnabledCipherSuites(java.lang.String[]) }. Please refer to official JDK JavaDocs diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/FileModifiedTimeUpdater.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/FileModifiedTimeUpdater.java new file mode 100644 index 00000000000..98d59600225 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/FileModifiedTimeUpdater.java @@ -0,0 +1,66 @@ +/** + * 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.tls; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.attribute.FileTime; +import lombok.Getter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Holder class to validate file modification. + */ +public class FileModifiedTimeUpdater { + @Getter + String fileName; + @Getter + FileTime lastModifiedTime; + + public FileModifiedTimeUpdater(String fileName) { + this.fileName = fileName; + this.lastModifiedTime = updateLastModifiedTime(); + } + + private FileTime updateLastModifiedTime() { + if (fileName != null) { + Path p = Paths.get(fileName); + try { + return Files.getLastModifiedTime(p); + } catch (IOException e) { + LOG.error("Unable to fetch lastModified time for file {}: ", fileName, e); + } + } + return null; + } + + public boolean checkAndRefresh() { + FileTime newLastModifiedTime = updateLastModifiedTime(); + if (newLastModifiedTime != null && !newLastModifiedTime.equals(lastModifiedTime)) { + this.lastModifiedTime = newLastModifiedTime; + return true; + } + return false; + } + + private static final Logger LOG = LoggerFactory.getLogger(FileModifiedTimeUpdater.class); +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java index 32e24264046..77e8dc9dbc5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java @@ -40,6 +40,7 @@ import java.security.cert.CertificateException; import java.security.spec.InvalidKeySpecException; import java.util.Arrays; +import java.util.concurrent.TimeUnit; import javax.net.ssl.KeyManagerFactory; import javax.net.ssl.TrustManagerFactory; @@ -79,8 +80,14 @@ public String toString() { private static final String TLSCONTEXT_HANDLER_NAME = "tls"; private String[] protocols; private String[] ciphers; - private SslContext sslContext; + private volatile SslContext sslContext; private ByteBufAllocator allocator; + private AbstractConfiguration config; + private FileModifiedTimeUpdater tTLSCertificatePath, tLSKeyStoreFilePath, tLSKeyStorePasswordFilePath, + tLSTrustStoreFilePath, tLSTrustStorePasswordFilePath; + private long certRefreshTime; + private volatile long certLastRefreshTime; + private boolean isServerCtx; private String getPasswordFromFile(String path) throws IOException { byte[] pwd; @@ -173,7 +180,7 @@ private SslProvider getTLSProvider(String sslProvider) { return SslProvider.JDK; } - private void createClientContext(AbstractConfiguration conf) + private void createClientContext() throws SecurityException, KeyStoreException, NoSuchAlgorithmException, CertificateException, IOException, UnrecoverableKeyException, InvalidKeySpecException, NoSuchProviderException { final SslContextBuilder sslContextBuilder; @@ -182,11 +189,11 @@ private void createClientContext(AbstractConfiguration conf) final boolean clientAuthentication; // get key-file and trust-file locations and passwords - if (!(conf instanceof ClientConfiguration)) { + if (!(config instanceof ClientConfiguration)) { throw new SecurityException("Client configruation not provided"); } - clientConf = (ClientConfiguration) conf; + clientConf = (ClientConfiguration) config; provider = getTLSProvider(clientConf.getTLSProvider()); clientAuthentication = clientConf.getTLSClientAuthentication(); @@ -262,7 +269,36 @@ private void createClientContext(AbstractConfiguration conf) sslContext = sslContextBuilder.build(); } - private void createServerContext(AbstractConfiguration conf) throws SecurityException, KeyStoreException, + private void createServerContext() + throws SecurityException, KeyStoreException, NoSuchAlgorithmException, CertificateException, IOException, + UnrecoverableKeyException, InvalidKeySpecException, IllegalArgumentException { + isServerCtx = true; + ServerConfiguration serverConf = (ServerConfiguration) config; + tTLSCertificatePath = new FileModifiedTimeUpdater(serverConf.getTLSCertificatePath()); + tLSKeyStoreFilePath = new FileModifiedTimeUpdater(serverConf.getTLSKeyStore()); + tLSKeyStorePasswordFilePath = new FileModifiedTimeUpdater(serverConf.getTLSKeyStorePasswordPath()); + tLSTrustStoreFilePath = new FileModifiedTimeUpdater(serverConf.getTLSTrustStore()); + tLSTrustStorePasswordFilePath = new FileModifiedTimeUpdater(serverConf.getTLSTrustStorePasswordPath()); + updateServerContext(); + } + + private synchronized SslContext getSSLContext() { + long now = System.currentTimeMillis(); + if (isServerCtx && (certRefreshTime > 0 && now > (certLastRefreshTime + certRefreshTime))) { + if (tTLSCertificatePath.checkAndRefresh() || tLSKeyStoreFilePath.checkAndRefresh() + || tLSKeyStorePasswordFilePath.checkAndRefresh() || tLSTrustStoreFilePath.checkAndRefresh() + || tLSTrustStorePasswordFilePath.checkAndRefresh()) { + try { + updateServerContext(); + } catch (Exception e) { + LOG.info("Failed to refresh tls certs", e); + } + } + } + return sslContext; + } + + private synchronized void updateServerContext() throws SecurityException, KeyStoreException, NoSuchAlgorithmException, CertificateException, IOException, UnrecoverableKeyException, InvalidKeySpecException, IllegalArgumentException { final SslContextBuilder sslContextBuilder; @@ -271,11 +307,11 @@ private void createServerContext(AbstractConfiguration conf) throws SecurityExce final boolean clientAuthentication; // get key-file and trust-file locations and passwords - if (!(conf instanceof ServerConfiguration)) { + if (!(config instanceof ServerConfiguration)) { throw new SecurityException("Server configruation not provided"); } - serverConf = (ServerConfiguration) conf; + serverConf = (ServerConfiguration) config; provider = getTLSProvider(serverConf.getTLSProvider()); clientAuthentication = serverConf.getTLSClientAuthentication(); @@ -349,14 +385,17 @@ private void createServerContext(AbstractConfiguration conf) throws SecurityExce } sslContext = sslContextBuilder.build(); + certLastRefreshTime = System.currentTimeMillis(); } @Override public synchronized void init(NodeType type, AbstractConfiguration conf, ByteBufAllocator allocator) throws SecurityException { this.allocator = allocator; + this.config = conf; final String enabledProtocols; final String enabledCiphers; + certRefreshTime = TimeUnit.SECONDS.toMillis(conf.getTLSCertFilesRefreshDurationSeconds()); enabledCiphers = conf.getTLSEnabledCipherSuites(); enabledProtocols = conf.getTLSEnabledProtocols(); @@ -364,10 +403,10 @@ public synchronized void init(NodeType type, AbstractConfiguration conf, ByteBuf try { switch (type) { case Client: - createClientContext(conf); + createClientContext(); break; case Server: - createServerContext(conf); + createServerContext(); break; default: throw new SecurityException(new IllegalArgumentException("Invalid NodeType")); @@ -401,7 +440,7 @@ public synchronized void init(NodeType type, AbstractConfiguration conf, ByteBuf @Override public SslHandler newTLSHandler() { - SslHandler sslHandler = sslContext.newHandler(allocator); + SslHandler sslHandler = getSSLContext().newHandler(allocator); if (protocols != null && protocols.length != 0) { sslHandler.engine().setEnabledProtocols(protocols); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java index b353e73c201..68d86628494 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; import java.security.cert.Certificate; @@ -59,7 +60,10 @@ import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.tls.TLSContextFactory.KeyStoreType; +import org.apache.bookkeeper.util.IOUtils; +import org.apache.commons.io.FileUtils; import org.junit.After; +import org.junit.Assert; import org.junit.Assume; import org.junit.Before; import org.junit.Test; @@ -340,6 +344,48 @@ public void testConnectToLocalTLSClusterTLSClient() throws Exception { testClient(clientConf, numBookies); } + /** + * Verify Bookie refreshes certs at configured duration. + */ + @Test + public void testRefreshDurationForBookieCerts() throws Exception { + Assume.assumeTrue(serverKeyStoreFormat == KeyStoreType.PEM); + ServerConfiguration serverConf = new ServerConfiguration(); + String originalTlsKeyFilePath = bsConfs.get(0).getTLSKeyStore(); + String invalidServerKey = getResourcePath("client-key.pem"); + File originalTlsCertFile = new File(originalTlsKeyFilePath); + File newTlsKeyFile = IOUtils.createTempFileAndDeleteOnExit(originalTlsKeyFilePath, "refresh"); + // clean up temp file even if test fails + newTlsKeyFile.deleteOnExit(); + File invalidServerKeyFile = new File(invalidServerKey); + // copy invalid cert to new temp file + FileUtils.copyFile(invalidServerKeyFile, newTlsKeyFile); + long refreshDurationInSec = 1; + for (ServerConfiguration conf : bsConfs) { + conf.setTLSCertFilesRefreshDurationSeconds(1); + conf.setTLSKeyStore(newTlsKeyFile.getAbsolutePath()); + } + restartBookies(serverConf); + + ClientConfiguration clientConf = new ClientConfiguration(baseClientConf); + try { + testClient(clientConf, numBookies); + Assert.fail("Should have fail due to invalid cert"); + } catch (Exception e) { + // Ok. + } + + // Sleep so, cert file can be refreshed + Thread.sleep(refreshDurationInSec * 1000 + 1000); + + // copy valid key-file at given new location + FileUtils.copyFile(originalTlsCertFile, newTlsKeyFile); + newTlsKeyFile.setLastModified(System.currentTimeMillis() + 1000); + // client should be successfully able to add entries over tls + testClient(clientConf, numBookies); + newTlsKeyFile.delete(); + } + /** * Multiple clients, some with TLS, and some without TLS. */ diff --git a/conf/bk_server.conf b/conf/bk_server.conf index fa77b949a86..4da030d712f 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -257,6 +257,8 @@ httpServerClass=org.apache.bookkeeper.http.vertx.VertxHttpServer # Bookie Truststore password path, if the trust store is protected by a password. # tlsTrustStorePasswordPath=null +# Tls certificate files refresh duration in seconds. +# tlsCertFilesRefreshDurationSeconds=0 ############################################## Bookie Storage ############################################## From 74f45d0283ad4a45a95a2058baec8ad85138038f Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 26 Sep 2019 09:06:44 +0200 Subject: [PATCH 0335/1642] Make the project buildable with JDK12 and JDK13 Upgrade Lombok to 1.18.10 Upgrade Groovy and Groovy Compiler and Maven Compiler plugin Fix an inverted if that prevented **bkctl** to work with JDK11+ and it made GC LOGGING opts not applied with JDK8 (in bkctl command), a regression introduced by #2132 Reviewers: Ivan Kelly This closes #2160 from eolivelli/fix/build-jdk12 --- bin/common.sh | 4 ++-- pom.xml | 5 +++-- tests/integration-tests-base-groovy/pom.xml | 6 ++++-- tests/integration-tests-utils/pom.xml | 1 + tests/pom.xml | 2 +- 5 files changed, 11 insertions(+), 7 deletions(-) diff --git a/bin/common.sh b/bin/common.sh index d3b16d8914f..ea8638f868c 100755 --- a/bin/common.sh +++ b/bin/common.sh @@ -123,13 +123,13 @@ BOOKIE_GC_LOGGING_OPTS=${BOOKIE_GC_LOGGING_OPTS:-"${DEFAULT_BOOKIE_GC_LOGGING_OP DEFAULT_CLI_GC_OPTS="-XX:+UseG1GC \ -XX:MaxGCPauseMillis=10" if [ "$USING_JDK8" -ne "1" ]; then + DEFAULT_CLI_GC_LOGGING_OPTS="" +else DEFAULT_CLI_GC_LOGGING_OPTS="-XX:+PrintGCDetails \ -XX:+PrintGCApplicationStoppedTime \ -XX:+UseGCLogFileRotation \ -XX:NumberOfGCLogFiles=5 \ -XX:GCLogFileSize=64m" -else - DEFAULT_CLI_GC_LOGGING_OPTS="" fi CLI_MAX_HEAP_MEMORY=${CLI_MAX_HEAP_MEMORY:-"512M"} diff --git a/pom.xml b/pom.xml index 4e496b2f660..1c86c8bb655 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,7 @@ 4.12 0.5.0-1 0.9.3 - 1.16.22 + 1.18.10 1.3.0 2.22.0 4.1.32.Final @@ -183,7 +183,7 @@ 3.1.0 3.0.0 2.5 - 3.7.0 + 3.8.1 3.0.2 2.7 2.5.1 @@ -703,6 +703,7 @@ org.codehaus.groovy groovy-all ${groovy.version} + pom org.jboss.shrinkwrap.resolver diff --git a/tests/integration-tests-base-groovy/pom.xml b/tests/integration-tests-base-groovy/pom.xml index ac0167b5845..6422b4c9684 100644 --- a/tests/integration-tests-base-groovy/pom.xml +++ b/tests/integration-tests-base-groovy/pom.xml @@ -31,8 +31,8 @@ Apache BookKeeper :: Tests :: Base module for Arquillian based integration tests using groovy - 2.9.2-04 - 2.4.13-02 + 3.4.0-01 + 2.5.8-01 @@ -73,6 +73,7 @@ org.codehaus.groovy groovy-all ${groovy.version} + pom @@ -101,6 +102,7 @@ org.codehaus.groovy groovy-all + pom diff --git a/tests/integration-tests-utils/pom.xml b/tests/integration-tests-utils/pom.xml index 1324c0a10ea..c09d5958f80 100644 --- a/tests/integration-tests-utils/pom.xml +++ b/tests/integration-tests-utils/pom.xml @@ -57,6 +57,7 @@ org.codehaus.groovy groovy-all + pom diff --git a/tests/pom.xml b/tests/pom.xml index 76d1eed5de3..5611d1853a5 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -28,7 +28,7 @@ Apache BookKeeper :: Tests - 2.4.13 + 2.5.8 From 2f40049dc09301b02c6a8b7d2a1cc1d681857e9d Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 27 Sep 2019 15:59:46 +0200 Subject: [PATCH 0336/1642] Fix tests and upgrade libs to make tests pass on JDK13 - Upgrade Mockito and PowerMock with a version that support JDK12+ - Fix bash script tests Reviewers: Ivan Kelly This closes #2169 from eolivelli/fix/make-tests-pass-jdk13 --- pom.xml | 4 ++-- .../scripts/src/test/bash/bk_test_bin_common.sh | 16 ++++++++++++---- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 1c86c8bb655..604ae52b7a3 100644 --- a/pom.xml +++ b/pom.xml @@ -149,11 +149,11 @@ 0.9.3 1.18.10 1.3.0 - 2.22.0 + 3.0.0 4.1.32.Final 2.0.20.Final 9.1.3 - 2.0.0-beta.5 + 2.0.2 0.0.21 0.8.3 3.5.1 diff --git a/tests/scripts/src/test/bash/bk_test_bin_common.sh b/tests/scripts/src/test/bash/bk_test_bin_common.sh index 8e81f92387d..e41455cbf26 100644 --- a/tests/scripts/src/test/bash/bk_test_bin_common.sh +++ b/tests/scripts/src/test/bash/bk_test_bin_common.sh @@ -200,8 +200,12 @@ testBuildBookieJVMOpts() { TEST_LOG_DIR=${BK_TMPDIR}/logdir TEST_GC_LOG_FILENAME="test-gc.log" ACTUAL_JVM_OPTS=$(build_bookie_jvm_opts ${TEST_LOG_DIR} ${TEST_GC_LOG_FILENAME}) - EXPECTED_JVM_OPTS="-Xms1g -Xmx1g -XX:MaxDirectMemorySize=2g ${DEFAULT_BOOKIE_GC_OPTS} ${DEFAULT_BOOKIE_GC_LOGGING_OPTS} -Xloggc:${TEST_LOG_DIR}/${TEST_GC_LOG_FILENAME}" - + USEJDK8=$(detect_jdk8) + if [ "$USING_JDK8" -ne "1" ]; then + EXPECTED_JVM_OPTS="-Xms1g -Xmx1g -XX:MaxDirectMemorySize=2g ${DEFAULT_BOOKIE_GC_OPTS} ${DEFAULT_BOOKIE_GC_LOGGING_OPTS} -Xlog:gc=info:file=${TEST_LOG_DIR}/${TEST_GC_LOG_FILENAME}::filecount=5,filesize=64m" + else + EXPECTED_JVM_OPTS="-Xms1g -Xmx1g -XX:MaxDirectMemorySize=2g ${DEFAULT_BOOKIE_GC_OPTS} ${DEFAULT_BOOKIE_GC_LOGGING_OPTS} -Xloggc:${TEST_LOG_DIR}/${TEST_GC_LOG_FILENAME}" + fi assertEquals "JVM OPTS is not set correctly" "${EXPECTED_JVM_OPTS}" "${ACTUAL_JVM_OPTS}" } @@ -211,8 +215,12 @@ testBuildCLIJVMOpts() { TEST_LOG_DIR=${BK_TMPDIR}/logdir TEST_GC_LOG_FILENAME="test-gc.log" ACTUAL_JVM_OPTS=$(build_cli_jvm_opts ${TEST_LOG_DIR} ${TEST_GC_LOG_FILENAME}) - EXPECTED_JVM_OPTS="-Xms256M -Xmx512M ${DEFAULT_CLI_GC_OPTS} ${DEFAULT_CLI_GC_LOGGING_OPTS} -Xloggc:${TEST_LOG_DIR}/${TEST_GC_LOG_FILENAME}" - + USEJDK8=$(detect_jdk8) + if [ "$USING_JDK8" -ne "1" ]; then + EXPECTED_JVM_OPTS="-Xms256M -Xmx512M ${DEFAULT_CLI_GC_OPTS} ${DEFAULT_CLI_GC_LOGGING_OPTS} -Xlog:gc=info:file=${TEST_LOG_DIR}/${TEST_GC_LOG_FILENAME}::filecount=5,filesize=64m" + else + EXPECTED_JVM_OPTS="-Xms256M -Xmx512M ${DEFAULT_CLI_GC_OPTS} ${DEFAULT_CLI_GC_LOGGING_OPTS} -Xloggc:${TEST_LOG_DIR}/${TEST_GC_LOG_FILENAME}" + fi assertEquals "JVM OPTS is not set correctly" "${EXPECTED_JVM_OPTS}" "${ACTUAL_JVM_OPTS}" } From 44007c18c1ad47614abcc11e4d079dcaaccb0a51 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 27 Sep 2019 16:00:55 +0200 Subject: [PATCH 0337/1642] Change Python client version to 4.10.0 According to the release guide, before cutting a release we have to remove the 'alpha' qualifier from Python client version Reviewers: Ivan Kelly This closes #2163 from eolivelli/fix/python-client --- stream/clients/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index 2b205dc86c9..633a8156a9f 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -19,7 +19,7 @@ name = 'apache-bookkeeper-client' description = 'Apache BookKeeper client library' -version = '4.10.0-alpha-0' +version = '4.10.0' # Should be one of: # 'Development Status :: 3 - Alpha' # 'Development Status :: 4 - Beta' From 2f803b340b5e439e8c85a2e3370c2495bb14e563 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 2 Oct 2019 10:02:54 +0200 Subject: [PATCH 0338/1642] Update bc tests to include versions up to 4.9.x ### Motivation With backward compatibility tests we should cover the compatibility of current version against old versions, current we are only test current version against ancient releases. This change supersedes #1648 from sijie ### Changes Change the list of versions to test clients and bookie upgrade Reviewers: Matteo Merli This closes #2171 from eolivelli/update_bc_tests --- .../backwardcompat/TestCompatOldClients.groovy | 5 +---- .../TestCompatUpgradeWithHostnameBookieId.groovy | 5 +---- .../tests/backwardcompat/TestCompatUpgrade.groovy | 14 ++++++++++++-- .../all-released-versions-image/Dockerfile | 4 ++++ .../scripts/install-tarball.sh | 3 +++ .../scripts/install-all-tarballs.sh | 3 +++ 6 files changed, 24 insertions(+), 10 deletions(-) diff --git a/tests/backward-compat/current-server-old-clients/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatOldClients.groovy b/tests/backward-compat/current-server-old-clients/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatOldClients.groovy index 4fd35d58058..5a501bf7cae 100644 --- a/tests/backward-compat/current-server-old-clients/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatOldClients.groovy +++ b/tests/backward-compat/current-server-old-clients/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatOldClients.groovy @@ -40,10 +40,7 @@ class TestCompatOldClients { private static byte[] PASSWD = "foobar".getBytes() // 4.1.0 doesn't work because metadata format changed - private def oldClientVersions = ["4.2.0", "4.2.1", "4.2.2", "4.2.3", "4.2.4", - "4.3.0", "4.3.1", "4.3.2", "4.4.0", "4.5.0", "4.5.1", - "4.6.0", "4.6.1", "4.6.2", - "4.7.0"] + private def oldClientVersions = ["4.4.0", "4.5.1", "4.6.2", "4.7.2", "4.8.2", "4.9.2" ] @ArquillianResource DockerClient docker diff --git a/tests/backward-compat/hostname-bookieid/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgradeWithHostnameBookieId.groovy b/tests/backward-compat/hostname-bookieid/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgradeWithHostnameBookieId.groovy index e82cbd89e27..c6cf08a85cd 100644 --- a/tests/backward-compat/hostname-bookieid/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgradeWithHostnameBookieId.groovy +++ b/tests/backward-compat/hostname-bookieid/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgradeWithHostnameBookieId.groovy @@ -37,10 +37,7 @@ class TestCompatUpgradeWithHostnameBookieId { private static final Logger LOG = LoggerFactory.getLogger(TestCompatUpgradeWithHostnameBookieId.class) private static byte[] PASSWD = "foobar".getBytes() - private def oldClientVersions = ["4.1.0", "4.2.0", "4.2.1", "4.2.2", "4.2.3", "4.2.4", - "4.3.0", "4.3.1", "4.3.2", "4.4.0", "4.5.0", "4.5.1", - "4.6.0", "4.6.1", "4.6.2", - "4.7.0"] + private def oldClientVersions = ["4.4.0", "4.5.1", "4.6.2", "4.7.2", "4.8.2", "4.9.2"] @ArquillianResource DockerClient docker diff --git a/tests/backward-compat/upgrade/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgrade.groovy b/tests/backward-compat/upgrade/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgrade.groovy index 1a3f296fb86..23d7b9fc68d 100644 --- a/tests/backward-compat/upgrade/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgrade.groovy +++ b/tests/backward-compat/upgrade/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatUpgrade.groovy @@ -192,7 +192,17 @@ class TestCompatUpgrade { } @Test - public void test470toCurrentMaster() throws Exception { - testUpgrade("4.7.0", System.getProperty("currentVersion")) + public void test470to471() throws Exception { + testUpgrade("4.7.0", "4.7.1") + } + + @Test + public void test471to472() throws Exception { + testUpgrade("4.7.1", "4.7.2") + } + + @Test + public void test472toCurrentMaster() throws Exception { + testUpgrade("4.7.2", System.getProperty("currentVersion")) } } diff --git a/tests/docker-images/all-released-versions-image/Dockerfile b/tests/docker-images/all-released-versions-image/Dockerfile index abe3d65385b..f149349ad90 100644 --- a/tests/docker-images/all-released-versions-image/Dockerfile +++ b/tests/docker-images/all-released-versions-image/Dockerfile @@ -48,6 +48,10 @@ RUN wget -nv https://archive.apache.org/dist/bookkeeper/bookkeeper-4.6.0/bookkee RUN wget -nv https://archive.apache.org/dist/bookkeeper/bookkeeper-4.6.1/bookkeeper-server-4.6.1-bin.tar.gz{,.sha1,.md5,.asc} RUN wget -nv https://archive.apache.org/dist/bookkeeper/bookkeeper-4.6.2/bookkeeper-server-4.6.2-bin.tar.gz{,.sha1,.asc} RUN wget -nv https://archive.apache.org/dist/bookkeeper/bookkeeper-4.7.0/bookkeeper-server-4.7.0-bin.tar.gz{,.sha1,.asc} +RUN wget -nv https://archive.apache.org/dist/bookkeeper/bookkeeper-4.7.1/bookkeeper-server-4.7.1-bin.tar.gz{,.sha1,.asc} +RUN wget -nv https://archive.apache.org/dist/bookkeeper/bookkeeper-4.7.2/bookkeeper-server-4.7.2-bin.tar.gz{,.sha512,.asc} +RUN wget -nv https://archive.apache.org/dist/bookkeeper/bookkeeper-4.8.2/bookkeeper-server-4.8.2-bin.tar.gz{,.sha512,.asc} +RUN wget -nv https://archive.apache.org/dist/bookkeeper/bookkeeper-4.9.2/bookkeeper-server-4.9.2-bin.tar.gz{,.sha512,.asc} RUN wget -nv https://archive.apache.org/dist/incubator/pulsar/pulsar-1.21.0-incubating/apache-pulsar-1.21.0-incubating-bin.tar.gz{,.asc} RUN wget -nv https://dist.apache.org/repos/dist/release/bookkeeper/KEYS RUN wget -nv http://svn.apache.org/repos/asf/zookeeper/bookkeeper/dist/KEYS?p=1620552 -O KEYS.old diff --git a/tests/docker-images/all-released-versions-image/scripts/install-tarball.sh b/tests/docker-images/all-released-versions-image/scripts/install-tarball.sh index d4cfb5982f6..666a5419a48 100755 --- a/tests/docker-images/all-released-versions-image/scripts/install-tarball.sh +++ b/tests/docker-images/all-released-versions-image/scripts/install-tarball.sh @@ -25,6 +25,9 @@ TARBALL=$1 if [ -f $TARBALL.sha1 ]; then sha1sum --check $TARBALL.sha1 > /dev/null fi +if [ -f $TARBALL.sha512 ]; then + sha512sum --check $TARBALL.sha512 > /dev/null +fi if [ -f $T.md5 ]; then md5sum --check $TARBALL.md5 > /dev/null fi diff --git a/tests/docker-images/all-versions-image/scripts/install-all-tarballs.sh b/tests/docker-images/all-versions-image/scripts/install-all-tarballs.sh index 261bf7ba410..af9894ea994 100644 --- a/tests/docker-images/all-versions-image/scripts/install-all-tarballs.sh +++ b/tests/docker-images/all-versions-image/scripts/install-all-tarballs.sh @@ -59,6 +59,9 @@ for T in bookkeeper-{,dist-}server-*-bin.tar.gz; do if [ -f $T.sha1 ]; then sha1sum --check $T.sha1 > /dev/null fi + if [ -f $T.sha512 ]; then + sha512sum --check $T.sha512 > /dev/null + fi if [ -f $T.md5 ]; then md5sum --check $T.md5 > /dev/null fi From ea2644ea4d8eeb32e69065d9b78d36c4be03cdfb Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 2 Oct 2019 01:05:42 -0700 Subject: [PATCH 0339/1642] Avoid throwing exception when doing EntryLogger.internalReadEntry ### Motivation In the refactoring part of #1819, the `internalReadEntry()` behavior was changed into throwing an exception when reading an entry from a different ledger. This is causing a big performance issue when doing read-head from the ledger storage, because we keep reading from the current entry log until we find an entry from a different ledger. Reviewers: Ivan Kelly , Enrico Olivelli This closes #2172 from merlimat/read-internal --- .../apache/bookkeeper/bookie/EntryLogger.java | 80 ++++++++++--------- .../ldb/SingleDirectoryDbLedgerStorage.java | 3 +- 2 files changed, 45 insertions(+), 38 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index 6662d594d27..731275c9986 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -756,37 +756,43 @@ static class WrongEntryException extends EntryLookupException { } } - private static class EntryLogEntry { - final int entrySize; - final BufferedReadChannel fc; - - EntryLogEntry(int entrySize, BufferedReadChannel fc) { - this.entrySize = entrySize; - this.fc = fc; - } - } - - private EntryLogEntry getFCForEntryInternal( + private BufferedReadChannel getFCForEntryInternal( long ledgerId, long entryId, long entryLogId, long pos) throws EntryLookupException, IOException { - ByteBuf sizeBuff = sizeBuffer.get(); - sizeBuff.clear(); - pos -= 4; // we want to get the entrySize as well as the ledgerId and entryId - BufferedReadChannel fc; try { - fc = getChannelForLogId(entryLogId); + return getChannelForLogId(entryLogId); } catch (FileNotFoundException e) { throw new EntryLookupException.MissingLogFileException(ledgerId, entryId, entryLogId, pos); } + } + + private ByteBuf readEntrySize(long ledgerId, long entryId, long entryLogId, long pos, BufferedReadChannel fc) + throws EntryLookupException, IOException { + ByteBuf sizeBuff = sizeBuffer.get(); + sizeBuff.clear(); + + long entrySizePos = pos - 4; // we want to get the entrySize as well as the ledgerId and entryId try { - if (readFromLogChannel(entryLogId, fc, sizeBuff, pos) != sizeBuff.capacity()) { - throw new EntryLookupException.MissingEntryException(ledgerId, entryId, entryLogId, pos); + if (readFromLogChannel(entryLogId, fc, sizeBuff, entrySizePos) != sizeBuff.capacity()) { + throw new EntryLookupException.MissingEntryException(ledgerId, entryId, entryLogId, entrySizePos); } } catch (BufferedChannelBase.BufferedChannelClosedException | AsynchronousCloseException e) { - throw new EntryLookupException.MissingLogFileException(ledgerId, entryId, entryLogId, pos); + throw new EntryLookupException.MissingLogFileException(ledgerId, entryId, entryLogId, entrySizePos); } - pos += 4; + return sizeBuff; + } + + void checkEntry(long ledgerId, long entryId, long location) throws EntryLookupException, IOException { + long entryLogId = logIdForOffset(location); + long pos = posForOffset(location); + BufferedReadChannel fc = getFCForEntryInternal(ledgerId, entryId, entryLogId, pos); + ByteBuf sizeBuf = readEntrySize(ledgerId, entryId, entryLogId, pos, fc); + validateEntry(ledgerId, entryId, entryLogId, pos, sizeBuf); + } + + private void validateEntry(long ledgerId, long entryId, long entryLogId, long pos, ByteBuf sizeBuff) + throws IOException, EntryLookupException { int entrySize = sizeBuff.readInt(); // entrySize does not include the ledgerId @@ -805,23 +811,24 @@ private EntryLogEntry getFCForEntryInternal( throw new EntryLookupException.WrongEntryException( thisEntryId, thisLedgerId, ledgerId, entryId, entryLogId, pos); } - return new EntryLogEntry(entrySize, fc); - } - - void checkEntry(long ledgerId, long entryId, long location) throws EntryLookupException, IOException { - long entryLogId = logIdForOffset(location); - long pos = posForOffset(location); - getFCForEntryInternal(ledgerId, entryId, entryLogId, pos); } - public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) + public ByteBuf internalReadEntry(long ledgerId, long entryId, long location, boolean validateEntry) throws IOException, Bookie.NoEntryException { long entryLogId = logIdForOffset(location); long pos = posForOffset(location); - final EntryLogEntry entry; + + BufferedReadChannel fc = null; + int entrySize = -1; try { - entry = getFCForEntryInternal(ledgerId, entryId, entryLogId, pos); + fc = getFCForEntryInternal(ledgerId, entryId, entryLogId, pos); + + ByteBuf sizeBuff = readEntrySize(ledgerId, entryId, entryLogId, pos, fc); + entrySize = sizeBuff.getInt(0); + if (validateEntry) { + validateEntry(ledgerId, entryId, entryLogId, pos, sizeBuff); + } } catch (EntryLookupException.MissingEntryException entryLookupError) { throw new Bookie.NoEntryException("Short read from entrylog " + entryLogId, ledgerId, entryId); @@ -829,9 +836,9 @@ public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) throw new IOException(e.toString()); } - ByteBuf data = allocator.buffer(entry.entrySize, entry.entrySize); - int rc = readFromLogChannel(entryLogId, entry.fc, data, pos); - if (rc != entry.entrySize) { + ByteBuf data = allocator.buffer(entrySize, entrySize); + int rc = readFromLogChannel(entryLogId, fc, data, pos); + if (rc != entrySize) { // Note that throwing NoEntryException here instead of IOException is not // without risk. If all bookies in a quorum throw this same exception // the client will assume that it has reached the end of the ledger. @@ -842,16 +849,15 @@ public ByteBuf internalReadEntry(long ledgerId, long entryId, long location) data.release(); throw new Bookie.NoEntryException("Short read for " + ledgerId + "@" + entryId + " in " + entryLogId + "@" - + pos + "(" + rc + "!=" + entry.entrySize + ")", ledgerId, entryId); + + pos + "(" + rc + "!=" + entrySize + ")", ledgerId, entryId); } - data.writerIndex(entry.entrySize); + data.writerIndex(entrySize); return data; } public ByteBuf readEntry(long ledgerId, long entryId, long location) throws IOException, Bookie.NoEntryException { - ByteBuf data = internalReadEntry(ledgerId, entryId, location); - return data; + return internalReadEntry(ledgerId, entryId, location, true /* validateEntry */); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 58ce2beb319..e975f160bbe 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -466,7 +466,8 @@ private void fillReadAheadCache(long orginalLedgerId, long firstEntryId, long fi long size = 0; while (count < readAheadCacheBatchSize && currentEntryLogId == firstEntryLogId) { - ByteBuf entry = entryLogger.internalReadEntry(orginalLedgerId, -1, currentEntryLocation); + ByteBuf entry = entryLogger.internalReadEntry(orginalLedgerId, -1, currentEntryLocation, + false /* validateEntry */); try { long currentEntryLedgerId = entry.getLong(0); From 1cd7e92c567d08019800ca2a692cfaa8e7472497 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Wed, 2 Oct 2019 11:10:16 -0700 Subject: [PATCH 0340/1642] Enhance deferLedgerLockReleaseOfFailedLedger in ReplicationWorker Descriptions of the changes in this PR: **Issue:** In the past, ReplicationWorker (RW) retrial logic is enhanced to backoff replication after threshold number of replication failures of a ledger. This is to help in a pathological situation where data (ledger/entry) is unavailable. But this is sub-optimal solution, since there is possibility that each RW can try recovering a ledger threshold number of times, before a RW defers ledgerLockRelease. Also each time a RW tries to recover it would read entry/fragment sequentially and writes to new bookies until it finds a missing entry (completely unavailable) before failing on replication of that ledger. This is done for each retrial and it bloats the storage and overreplication need to detect and delete it, which runs once a day by default. So because of this cluster can run out of storage space and may become RO cluster. Also this puts quite a bit of load on cluster in vain. **So the new proposal is to** - On each RW. remember the state in addition to the counter. State must include the entries which RW failed to read. - Counter and state must kept around in each RW node. And exponential backup should be used for deferLedgerLockReleaseOfFailedLedger - During next attempt by RW, try to read the failed entries which is noted in the state. Read must be successful before proceeding replicating. - With this model we avoid duplicate copies on each attempt. At the most each RW will create only one copy Reviewers: Enrico Olivelli , Venkateswararao Jujjuri (JV) This closes #2166 from reddycharan/enhancereplication --- .../bookkeeper/client/BookKeeperAdmin.java | 21 +- .../bookkeeper/client/LedgerFragment.java | 4 +- .../client/LedgerFragmentReplicator.java | 24 +- .../bookkeeper/conf/ServerConfiguration.java | 27 ++- .../replication/ReplicationStats.java | 2 +- .../replication/ReplicationWorker.java | 130 ++++++++-- .../client/BookKeeperCloseTest.java | 4 +- .../client/TestLedgerFragmentReplication.java | 10 +- .../replication/TestReplicationWorker.java | 225 +++++++++++++++--- site/_data/config/bk_server.yaml | 8 +- 10 files changed, 365 insertions(+), 90 deletions(-) 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 bdf56b8ae43..6bd95d85cc4 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 @@ -50,6 +50,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; import java.util.function.Predicate; import org.apache.bookkeeper.bookie.Bookie; @@ -95,6 +96,7 @@ public class BookKeeperAdmin implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(BookKeeperAdmin.class); private static final Logger VERBOSE = LoggerFactory.getLogger("verbose"); + private static final BiConsumer NOOP_BICONSUMER = (l, e) -> { }; // BookKeeper client instance private BookKeeper bkc; @@ -909,7 +911,7 @@ public void processResult(int rc, String path, Object ctx) { LedgerFragment ledgerFragment = new LedgerFragment(lh, startEntryId, endEntryId, targetBookieAddresses.keySet()); asyncRecoverLedgerFragment(lh, ledgerFragment, cb, - Sets.newHashSet(targetBookieAddresses.values())); + Sets.newHashSet(targetBookieAddresses.values()), NOOP_BICONSUMER); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return; @@ -961,8 +963,9 @@ static String formatEnsemble(List ensemble, Set newBookies) throws InterruptedException { - lfr.replicate(lh, ledgerFragment, ledgerFragmentMcb, newBookies); + final Set newBookies, + final BiConsumer onReadEntryFailureCallback) throws InterruptedException { + lfr.replicate(lh, ledgerFragment, ledgerFragmentMcb, newBookies, onReadEntryFailureCallback); } private Map getReplacementBookies( @@ -1050,18 +1053,20 @@ private ArrayList replaceBookiesInEnsemble( * - LedgerFragment to replicate */ public void replicateLedgerFragment(LedgerHandle lh, - final LedgerFragment ledgerFragment) + final LedgerFragment ledgerFragment, + final BiConsumer onReadEntryFailureCallback) throws InterruptedException, BKException { Optional> excludedBookies = Optional.empty(); Map targetBookieAddresses = getReplacementBookiesByIndexes(lh, ledgerFragment.getEnsemble(), ledgerFragment.getBookiesIndexes(), excludedBookies); - replicateLedgerFragment(lh, ledgerFragment, targetBookieAddresses); + replicateLedgerFragment(lh, ledgerFragment, targetBookieAddresses, onReadEntryFailureCallback); } private void replicateLedgerFragment(LedgerHandle lh, final LedgerFragment ledgerFragment, - final Map targetBookieAddresses) + final Map targetBookieAddresses, + final BiConsumer onReadEntryFailureCallback) throws InterruptedException, BKException { CompletableFuture result = new CompletableFuture<>(); ResultCallBack resultCallBack = new ResultCallBack(result); @@ -1074,7 +1079,7 @@ private void replicateLedgerFragment(LedgerHandle lh, Set targetBookieSet = Sets.newHashSet(); targetBookieSet.addAll(targetBookieAddresses.values()); - asyncRecoverLedgerFragment(lh, ledgerFragment, cb, targetBookieSet); + asyncRecoverLedgerFragment(lh, ledgerFragment, cb, targetBookieSet, onReadEntryFailureCallback); try { SyncCallbackUtils.waitForResult(result); @@ -1132,7 +1137,7 @@ private static boolean containBookies(List ensemble, /** * This is the class for getting the replication result. */ - static class ResultCallBack implements AsyncCallback.VoidCallback { + public static class ResultCallBack implements AsyncCallback.VoidCallback { private final CompletableFuture sync; public ResultCallBack(CompletableFuture sync) { 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 382fe4e6b7c..94e7454ca9d 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 @@ -95,11 +95,11 @@ long getLedgerId() { return ledgerId; } - long getFirstEntryId() { + public long getFirstEntryId() { return firstEntryId; } - long getLastKnownEntryId() { + public long getLastKnownEntryId() { return lastKnownEntryId; } 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 765dec026e5..1da881d19c4 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 @@ -37,6 +37,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import java.util.stream.Collectors; import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; @@ -109,7 +110,8 @@ public LedgerFragmentReplicator(BookKeeper bkc) { private void replicateFragmentInternal(final LedgerHandle lh, final LedgerFragment lf, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final Set newBookies) throws InterruptedException { + final Set newBookies, + final BiConsumer onReadEntryFailureCallback) throws InterruptedException { if (!lf.isClosed()) { LOG.error("Trying to replicate an unclosed fragment;" + " This is not safe {}", lf); @@ -156,7 +158,7 @@ private void replicateFragmentInternal(final LedgerHandle lh, BKException.Code.LedgerRecoveryException); for (final Long entryId : entriesToReplicate) { recoverLedgerFragmentEntry(entryId, lh, ledgerFragmentEntryMcb, - newBookies); + newBookies, onReadEntryFailureCallback); } } @@ -182,14 +184,15 @@ private void replicateFragmentInternal(final LedgerHandle lh, */ void replicate(final LedgerHandle lh, final LedgerFragment lf, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final Set targetBookieAddresses) + final Set targetBookieAddresses, + final BiConsumer onReadEntryFailureCallback) throws InterruptedException { Set partionedFragments = splitIntoSubFragments(lh, lf, bkc.getConf().getRereplicationEntryBatchSize()); LOG.info("Replicating fragment {} in {} sub fragments.", lf, partionedFragments.size()); replicateNextBatch(lh, partionedFragments.iterator(), - ledgerFragmentMcb, targetBookieAddresses); + ledgerFragmentMcb, targetBookieAddresses, onReadEntryFailureCallback); } /** @@ -198,7 +201,8 @@ void replicate(final LedgerHandle lh, final LedgerFragment lf, private void replicateNextBatch(final LedgerHandle lh, final Iterator fragments, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final Set targetBookieAddresses) { + final Set targetBookieAddresses, + final BiConsumer onReadEntryFailureCallback) { if (fragments.hasNext()) { try { replicateFragmentInternal(lh, fragments.next(), @@ -211,11 +215,12 @@ public void processResult(int rc, String v, Object ctx) { } else { replicateNextBatch(lh, fragments, ledgerFragmentMcb, - targetBookieAddresses); + targetBookieAddresses, + onReadEntryFailureCallback); } } - }, targetBookieAddresses); + }, targetBookieAddresses, onReadEntryFailureCallback); } catch (InterruptedException e) { ledgerFragmentMcb.processResult( BKException.Code.InterruptedException, null, null); @@ -289,7 +294,9 @@ static Set splitIntoSubFragments(LedgerHandle lh, private void recoverLedgerFragmentEntry(final Long entryId, final LedgerHandle lh, final AsyncCallback.VoidCallback ledgerFragmentEntryMcb, - final Set newBookies) throws InterruptedException { + final Set newBookies, + final BiConsumer onReadEntryFailureCallback) throws InterruptedException { + final long ledgerId = lh.getId(); final AtomicInteger numCompleted = new AtomicInteger(0); final AtomicBoolean completed = new AtomicBoolean(false); final WriteCallback multiWriteCallback = new WriteCallback() { @@ -328,6 +335,7 @@ public void readComplete(int rc, LedgerHandle lh, if (rc != BKException.Code.OK) { LOG.error("BK error reading ledger entry: " + entryId, BKException.create(rc)); + onReadEntryFailureCallback.accept(ledgerId, entryId); ledgerFragmentEntryMcb.processResult(rc, null, null); return; } 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 9a2c42d317b..cba673bdf80 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 @@ -1552,12 +1552,15 @@ public long getOpenLedgerRereplicationGracePeriod() { } /** - * Set the grace period so that if the replication worker fails to replicate - * a underreplicatedledger for more than - * ReplicationWorker.MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING - * number of times, then instead of releasing the lock immediately after - * failed attempt, it will hold under replicated ledger lock for this grace - * period and then it will release the lock. + * Set the grace period, in milliseconds, which the replication worker has + * to wait before releasing the lock after it failed to replicate a ledger. + * For the first ReplicationWorker.NUM_OF_EXPONENTIAL_BACKOFF_RETRIALS + * failures it will do exponential backoff then it will bound at + * LOCK_RELEASE_OF_FAILED_LEDGER_GRACE_PERIOD. + * + *

            On replication failure, instead of releasing the lock immediately + * after failed attempt, it will hold under replicated ledger lock for the + * grace period and then it will release the lock. * * @param waitTime */ @@ -1566,16 +1569,16 @@ public void setLockReleaseOfFailedLedgerGracePeriod(String waitTime) { } /** - * Get the grace period which the replication worker to wait before - * releasing the lock after replication worker failing to replicate for more - * than - * ReplicationWorker.MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING - * number of times. + * Get the grace period, in milliseconds, which the replication worker has + * to wait before releasing the lock after it failed to replicate a ledger. + * For the first ReplicationWorker.NUM_OF_EXPONENTIAL_BACKOFF_RETRIALS + * failures it will do exponential backoff then it will bound at + * LOCK_RELEASE_OF_FAILED_LEDGER_GRACE_PERIOD. * * @return */ public long getLockReleaseOfFailedLedgerGracePeriod() { - return getLong(LOCK_RELEASE_OF_FAILED_LEDGER_GRACE_PERIOD, 60000); + return getLong(LOCK_RELEASE_OF_FAILED_LEDGER_GRACE_PERIOD, 300000); } /** 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 f553f91cefd..6ec9f4918a7 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 @@ -60,7 +60,7 @@ public interface ReplicationStats { String NUM_BYTES_WRITTEN = "NUM_BYTES_WRITTEN"; String REPLICATE_EXCEPTION = "exceptions"; String NUM_DEFER_LEDGER_LOCK_RELEASE_OF_FAILED_LEDGER = "NUM_DEFER_LEDGER_LOCK_RELEASE_OF_FAILED_LEDGER"; - + String NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION = "NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION"; 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 1bba2d5ebee..80cfed30ef2 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 @@ -20,6 +20,7 @@ package org.apache.bookkeeper.replication; import static org.apache.bookkeeper.replication.ReplicationStats.NUM_DEFER_LEDGER_LOCK_RELEASE_OF_FAILED_LEDGER; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION; 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.REPLICATION_WORKER_SCOPE; @@ -35,13 +36,16 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.NavigableSet; import java.util.Set; import java.util.SortedMap; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import org.apache.bookkeeper.bookie.BookieThread; import org.apache.bookkeeper.client.BKException; @@ -81,8 +85,8 @@ public class ReplicationWorker implements Runnable { private static final Logger LOG = LoggerFactory .getLogger(ReplicationWorker.class); - private static final int REPLICATED_FAILED_LEDGERS_MAXSIZE = 100; - static final int MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING = 10; + private static final int REPLICATED_FAILED_LEDGERS_MAXSIZE = 2000; + public static final int NUM_OF_EXPONENTIAL_BACKOFF_RETRIALS = 5; private final LedgerUnderreplicationManager underreplicationManager; private final ServerConfiguration conf; @@ -96,6 +100,8 @@ public class ReplicationWorker implements Runnable { private final long openLedgerRereplicationGracePeriod; private final Timer pendingReplicationTimer; private final long lockReleaseOfFailedLedgerGracePeriod; + private final long baseBackoffForLockReleaseOfFailedLedger; + private final BiConsumer onReadEntryFailureCallback; // Expose Stats private final StatsLogger statsLogger; @@ -119,8 +125,14 @@ public class ReplicationWorker implements Runnable { help = "the number of defer-ledger-lock-releases of failed ledgers" ) private final Counter numDeferLedgerLockReleaseOfFailedLedger; + @StatsDoc( + name = NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION, + help = "the number of entries ReplicationWorker unable to read" + ) + private final Counter numEntriesUnableToReadForReplication; private final Map exceptionCounters; final LoadingCache replicationFailedLedgers; + final LoadingCache> unableToReadEntriesForReplication; /** * Replication worker for replicating the ledger fragments from @@ -175,6 +187,8 @@ public ReplicationWorker(final ServerConfiguration conf, this.openLedgerRereplicationGracePeriod = conf .getOpenLedgerRereplicationGracePeriod(); this.lockReleaseOfFailedLedgerGracePeriod = conf.getLockReleaseOfFailedLedgerGracePeriod(); + this.baseBackoffForLockReleaseOfFailedLedger = this.lockReleaseOfFailedLedgerGracePeriod + / (long) (Math.pow(2, NUM_OF_EXPONENTIAL_BACKOFF_RETRIALS)); this.rwRereplicateBackoffMs = conf.getRwRereplicateBackoffMs(); this.pendingReplicationTimer = new Timer("PendingReplicationTimer"); this.replicationFailedLedgers = CacheBuilder.newBuilder().maximumSize(REPLICATED_FAILED_LEDGERS_MAXSIZE) @@ -184,6 +198,14 @@ public AtomicInteger load(Long key) throws Exception { return new AtomicInteger(); } }); + this.unableToReadEntriesForReplication = CacheBuilder.newBuilder() + .maximumSize(REPLICATED_FAILED_LEDGERS_MAXSIZE) + .build(new CacheLoader>() { + @Override + public ConcurrentSkipListSet load(Long key) throws Exception { + return new ConcurrentSkipListSet(); + } + }); // Expose Stats this.statsLogger = statsLogger; @@ -192,7 +214,13 @@ public AtomicInteger load(Long key) throws Exception { this.numLedgersReplicated = this.statsLogger.getCounter(NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED); this.numDeferLedgerLockReleaseOfFailedLedger = this.statsLogger .getCounter(NUM_DEFER_LEDGER_LOCK_RELEASE_OF_FAILED_LEDGER); + this.numEntriesUnableToReadForReplication = this.statsLogger + .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION); this.exceptionCounters = new HashMap(); + this.onReadEntryFailureCallback = (ledgerid, entryid) -> { + numEntriesUnableToReadForReplication.inc(); + unableToReadEntriesForReplication.getUnchecked(ledgerid).add(entryid); + }; } /** @@ -270,6 +298,61 @@ private void logBKExceptionAndReleaseLedger(BKException e, long ledgerIdToReplic getExceptionCounter(e.getClass().getSimpleName()).inc(); } + private boolean tryReadingFaultyEntries(LedgerHandle lh, LedgerFragment ledgerFragment) { + long ledgerId = lh.getId(); + ConcurrentSkipListSet entriesUnableToReadForThisLedger = unableToReadEntriesForReplication + .getIfPresent(ledgerId); + if (entriesUnableToReadForThisLedger == null) { + return true; + } + long firstEntryIdOfFragment = ledgerFragment.getFirstEntryId(); + long lastEntryIdOfFragment = ledgerFragment.getLastKnownEntryId(); + NavigableSet entriesOfThisFragmentUnableToRead = entriesUnableToReadForThisLedger + .subSet(firstEntryIdOfFragment, true, lastEntryIdOfFragment, true); + if (entriesOfThisFragmentUnableToRead.isEmpty()) { + return true; + } + final CountDownLatch multiReadComplete = new CountDownLatch(1); + final AtomicInteger numOfResponsesToWaitFor = new AtomicInteger(entriesOfThisFragmentUnableToRead.size()); + final AtomicInteger returnRCValue = new AtomicInteger(BKException.Code.OK); + for (long entryIdToRead : entriesOfThisFragmentUnableToRead) { + if (multiReadComplete.getCount() == 0) { + /* + * if an asyncRead request had already failed then break the + * loop. + */ + break; + } + lh.asyncReadEntries(entryIdToRead, entryIdToRead, (rc, ledHan, seq, ctx) -> { + long thisEntryId = (Long) ctx; + if (rc == BKException.Code.OK) { + entriesUnableToReadForThisLedger.remove(thisEntryId); + if (numOfResponsesToWaitFor.decrementAndGet() == 0) { + multiReadComplete.countDown(); + } + } else { + LOG.error("Received error: {} while trying to read entry: {} of ledger: {} in ReplicationWorker", + rc, entryIdToRead, ledgerId); + returnRCValue.compareAndSet(BKException.Code.OK, rc); + /* + * on receiving a failure error response, multiRead can be + * marked completed, since there is not need to wait for + * other responses. + */ + multiReadComplete.countDown(); + } + }, entryIdToRead); + } + try { + multiReadComplete.await(); + } catch (InterruptedException e) { + LOG.error("Got interrupted exception while trying to read entries", e); + Thread.currentThread().interrupt(); // set interrupt flag + return false; + } + return (returnRCValue.get() == BKException.Code.OK); + } + private boolean rereplicate(long ledgerIdToReplicate) throws InterruptedException, BKException, UnavailableException { if (LOG.isDebugEnabled()) { @@ -292,8 +375,13 @@ private boolean rereplicate(long ledgerIdToReplicate) throws InterruptedExceptio foundOpenFragments = true; continue; } + if (!tryReadingFaultyEntries(lh, ledgerFragment)) { + LOG.error("Failed to read faulty entries, so giving up replicating ledgerFragment {}", + ledgerFragment); + continue; + } try { - admin.replicateLedgerFragment(lh, ledgerFragment); + admin.replicateLedgerFragment(lh, ledgerFragment, onReadEntryFailureCallback); } catch (BKException.BKBookieHandleNotAvailableException e) { LOG.warn("BKBookieHandleNotAvailableException while replicating the fragment", e); } catch (BKException.BKLedgerRecoveryException e) { @@ -302,7 +390,6 @@ private boolean rereplicate(long ledgerIdToReplicate) throws InterruptedExceptio LOG.warn("BKNotEnoughBookiesException while replicating the fragment", e); } } - if (foundOpenFragments || isLastSegmentOpenAndMissingBookies(lh)) { deferLedgerLockRelease = true; deferLedgerLockRelease(ledgerIdToReplicate); @@ -315,16 +402,9 @@ private boolean rereplicate(long ledgerIdToReplicate) throws InterruptedExceptio underreplicationManager.markLedgerReplicated(ledgerIdToReplicate); return true; } else { - if (replicationFailedLedgers.getUnchecked(ledgerIdToReplicate) - .incrementAndGet() == MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING) { - deferLedgerLockRelease = true; - LOG.error( - "ReplicationWorker failed to replicate Ledger : {} for {} number of times, " - + "so deferring the ledger lock release", - ledgerIdToReplicate, MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING); - deferLedgerLockReleaseOfFailedLedger(ledgerIdToReplicate); - numDeferLedgerLockReleaseOfFailedLedger.inc(); - } + deferLedgerLockRelease = true; + deferLedgerLockReleaseOfFailedLedger(ledgerIdToReplicate); + numDeferLedgerLockReleaseOfFailedLedger.inc(); // Releasing the underReplication ledger lock and compete // for the replication again for the pending fragments return false; @@ -414,6 +494,10 @@ private Set getUnderreplicatedFragments(LedgerHandle lh, Long le return fragments; } + void scheduleTaskWithDelay(TimerTask timerTask, long delayPeriod) { + pendingReplicationTimer.schedule(timerTask, delayPeriod); + } + /** * Schedules a timer task for releasing the lock which will be scheduled * after open ledger fragment replication time. Ledger will be fenced if it @@ -489,18 +573,30 @@ public void run() { } } }; - pendingReplicationTimer.schedule(timerTask, gracePeriod); + scheduleTaskWithDelay(timerTask, gracePeriod); } /** * Schedules a timer task for releasing the lock. */ private void deferLedgerLockReleaseOfFailedLedger(final long ledgerId) { + int numOfTimesFailedSoFar = replicationFailedLedgers.getUnchecked(ledgerId).getAndIncrement(); + /* + * for the first NUM_OF_EXPONENTIAL_BACKOFF_RETRIALS retrials do + * exponential backoff, starting from + * baseBackoffForLockReleaseOfFailedLedger + */ + long delayOfLedgerLockReleaseInMSecs = (numOfTimesFailedSoFar >= NUM_OF_EXPONENTIAL_BACKOFF_RETRIALS) + ? this.lockReleaseOfFailedLedgerGracePeriod + : this.baseBackoffForLockReleaseOfFailedLedger * (int) Math.pow(2, numOfTimesFailedSoFar); + LOG.error( + "ReplicationWorker failed to replicate Ledger : {} for {} number of times, " + + "so deferring the ledger lock release by {} msecs", + ledgerId, numOfTimesFailedSoFar, delayOfLedgerLockReleaseInMSecs); TimerTask timerTask = new TimerTask() { @Override public void run() { try { - replicationFailedLedgers.invalidate(ledgerId); underreplicationManager.releaseUnderreplicatedLedger(ledgerId); } catch (UnavailableException e) { LOG.error("UnavailableException while replicating fragments of ledger {}", ledgerId, e); @@ -508,7 +604,7 @@ public void run() { } } }; - pendingReplicationTimer.schedule(timerTask, lockReleaseOfFailedLedgerGracePeriod); + scheduleTaskWithDelay(timerTask, delayOfLedgerLockReleaseInMSecs); } /** 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 eda68c866b1..3bebc2145d2 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 @@ -34,6 +34,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; @@ -65,6 +66,7 @@ public class BookKeeperCloseTest extends BookKeeperClusterTestCase { .getLogger(BookKeeperCloseTest.class); private DigestType digestType = DigestType.CRC32; private static final String PASSWORD = "testPasswd"; + private static final BiConsumer NOOP_BICONSUMER = (l, e) -> { }; public BookKeeperCloseTest() { super(3); @@ -519,7 +521,7 @@ public void testBookKeeperAdmin() throws Exception { try { bkadmin.replicateLedgerFragment(lh3, - checkercb.getResult(10, TimeUnit.SECONDS).iterator().next()); + checkercb.getResult(10, TimeUnit.SECONDS).iterator().next(), NOOP_BICONSUMER); 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/TestLedgerFragmentReplication.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java index 33b966bc478..3105d18ec7f 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 @@ -32,6 +32,7 @@ import java.util.Set; import java.util.SortedMap; import java.util.concurrent.CountDownLatch; +import java.util.function.BiConsumer; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; @@ -53,6 +54,7 @@ public class TestLedgerFragmentReplication extends BookKeeperClusterTestCase { private static final byte[] TEST_PSSWD = "testpasswd".getBytes(); private static final DigestType TEST_DIGEST_TYPE = BookKeeper.DigestType.CRC32; + private static final BiConsumer NOOP_BICONSUMER = (l, e) -> { }; private static final Logger LOG = LoggerFactory .getLogger(TestLedgerFragmentReplication.class); @@ -111,7 +113,7 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() // 0-9 entries should be copy to new bookie for (LedgerFragment lf : result) { - admin.replicateLedgerFragment(lh, lf); + admin.replicateLedgerFragment(lh, lf, NOOP_BICONSUMER); } // Killing all bookies except newly replicated bookie @@ -174,11 +176,11 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() int unclosedCount = 0; for (LedgerFragment lf : result) { if (lf.isClosed()) { - admin.replicateLedgerFragment(lh, lf); + admin.replicateLedgerFragment(lh, lf, NOOP_BICONSUMER); } else { unclosedCount++; try { - admin.replicateLedgerFragment(lh, lf); + admin.replicateLedgerFragment(lh, lf, NOOP_BICONSUMER); fail("Shouldn't be able to rereplicate unclosed ledger"); } catch (BKException bke) { // correct behaviour @@ -222,7 +224,7 @@ public void testReplicateLFShouldReturnFalseIfTheReplicationFails() BookKeeperAdmin admin = new BookKeeperAdmin(baseClientConf); for (LedgerFragment lf : fragments) { try { - admin.replicateLedgerFragment(lh, lf); + admin.replicateLedgerFragment(lh, lf, NOOP_BICONSUMER); } catch (BKException.BKLedgerRecoveryException e) { // expected } 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 c155bb456b2..11531b504ef 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 @@ -19,16 +19,20 @@ */ package org.apache.bookkeeper.replication; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION; 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.IOException; import java.net.URI; import java.util.Enumeration; import java.util.List; import java.util.Map.Entry; import java.util.Optional; +import java.util.TimerTask; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; @@ -47,10 +51,16 @@ import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; +import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.zookeeper.KeeperException; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -438,21 +448,13 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR() } - /** - * Tests that ReplicationWorker will not make more than - * ReplicationWorker.MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING - * number of replication failure attempts and if it fails more these many - * number of times then it will defer lock release by - * lockReleaseOfFailedLedgerGracePeriod. - * - * @throws Exception - */ @Test public void testBookiesNotAvailableScenarioForReplicationWorker() throws Exception { int ensembleSize = 3; LedgerHandle lh = bkc.createLedger(ensembleSize, ensembleSize, BookKeeper.DigestType.CRC32, TESTPASSWD); - for (int i = 0; i < 10; i++) { + int numOfEntries = 7; + for (int i = 0; i < numOfEntries; i++) { lh.addEntry(data); } lh.close(); @@ -474,9 +476,10 @@ public void testBookiesNotAvailableScenarioForReplicationWorker() throws Excepti } // create couple of replicationworkers - baseConf.setLockReleaseOfFailedLedgerGracePeriod("500"); - ReplicationWorker rw1 = new ReplicationWorker(baseConf); - ReplicationWorker rw2 = new ReplicationWorker(baseConf); + ServerConfiguration newRWConf = new ServerConfiguration(baseConf); + newRWConf.setLockReleaseOfFailedLedgerGracePeriod("64"); + ReplicationWorker rw1 = new ReplicationWorker(newRWConf); + ReplicationWorker rw2 = new ReplicationWorker(newRWConf); @Cleanup MetadataClientDriver clientDriver = MetadataDrivers @@ -487,6 +490,7 @@ public void testBookiesNotAvailableScenarioForReplicationWorker() throws Excepti LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); try { + //mark ledger underreplicated for (int i = 0; i < bookiesKilled.length; i++) { underReplicationManager.markLedgerUnderreplicated(lh.getId(), bookiesKilled[i].toString()); } @@ -502,10 +506,10 @@ public void testBookiesNotAvailableScenarioForReplicationWorker() throws Excepti @Override public void run() { try { - Thread.sleep(4000); + Thread.sleep(3000); isBookieRestarted.set(true); /* - * after sleeping for 4000 msecs, restart one of the + * after sleeping for 3000 msecs, restart one of the * bookie, so that replication can succeed. */ startBookie(killedBookiesConfig[0]); @@ -515,6 +519,8 @@ public void run() { } })).start(); + int rw1PrevFailedAttemptsCount = 0; + int rw2PrevFailedAttemptsCount = 0; while (!isBookieRestarted.get()) { /* * since all the bookies containing the ledger entries are down @@ -522,26 +528,21 @@ public void run() { */ assertTrue("Ledger: " + lh.getId() + " should be underreplicated", ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)); - /* - * check for both the replicationworkders number of failed - * attempts should be less than ReplicationWorker. - * MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING - */ - int failedAttempts = rw1.replicationFailedLedgers.get(lh.getId()).get(); + + // the number of failed attempts should have increased. + int rw1CurFailedAttemptsCount = rw1.replicationFailedLedgers.get(lh.getId()).get(); assertTrue( - "The number of failed attempts should be less than " - + "ReplicationWorker.MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING, " - + "but it is " - + failedAttempts, - failedAttempts <= ReplicationWorker.MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING); + "The current number of failed attempts: " + rw1CurFailedAttemptsCount + + " should be greater than or equal to previous value: " + rw1PrevFailedAttemptsCount, + rw1CurFailedAttemptsCount >= rw1PrevFailedAttemptsCount); + rw1PrevFailedAttemptsCount = rw1CurFailedAttemptsCount; - failedAttempts = rw2.replicationFailedLedgers.get(lh.getId()).get(); + int rw2CurFailedAttemptsCount = rw2.replicationFailedLedgers.get(lh.getId()).get(); assertTrue( - "The number of failed attempts should be less than " - + "ReplicationWorker.MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING, " - + "but it is " - + failedAttempts, - failedAttempts <= ReplicationWorker.MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING); + "The current number of failed attempts: " + rw2CurFailedAttemptsCount + + " should be greater than or equal to previous value: " + rw2PrevFailedAttemptsCount, + rw2CurFailedAttemptsCount >= rw2PrevFailedAttemptsCount); + rw2PrevFailedAttemptsCount = rw2CurFailedAttemptsCount; Thread.sleep(50); } @@ -551,7 +552,7 @@ public void run() { * should succeed in replicating this under replicated ledger and it * shouldn't be under replicated anymore. */ - int timeToWaitForReplicationToComplete = 2000; + int timeToWaitForReplicationToComplete = 20000; int timeWaited = 0; while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)) { Thread.sleep(100); @@ -560,6 +561,164 @@ public void run() { fail("Ledger should be replicated by now"); } } + + rw1PrevFailedAttemptsCount = rw1.replicationFailedLedgers.get(lh.getId()).get(); + rw2PrevFailedAttemptsCount = rw2.replicationFailedLedgers.get(lh.getId()).get(); + Thread.sleep(2000); + // now since the ledger is replicated, number of failed attempts + // counter shouldn't be increased even after sleeping for sometime. + assertEquals("rw1 failedattempts", rw1PrevFailedAttemptsCount, + rw1.replicationFailedLedgers.get(lh.getId()).get()); + assertEquals("rw2 failed attempts ", rw2PrevFailedAttemptsCount, + rw2.replicationFailedLedgers.get(lh.getId()).get()); + + /* + * Since these entries are eventually available, and replication has + * eventually succeeded, in one of the RW + * unableToReadEntriesForReplication should be 0. + */ + int rw1UnableToReadEntriesForReplication = rw1.unableToReadEntriesForReplication.get(lh.getId()).size(); + int rw2UnableToReadEntriesForReplication = rw2.unableToReadEntriesForReplication.get(lh.getId()).size(); + assertTrue( + "unableToReadEntriesForReplication in RW1: " + rw1UnableToReadEntriesForReplication + " in RW2: " + + rw2UnableToReadEntriesForReplication, + (rw1UnableToReadEntriesForReplication == 0) || (rw2UnableToReadEntriesForReplication == 0)); + } finally { + rw1.shutdown(); + rw2.shutdown(); + underReplicationManager.close(); + } + } + + class InjectedReplicationWorker extends ReplicationWorker { + CopyOnWriteArrayList delayReplicationPeriods; + + public InjectedReplicationWorker(ServerConfiguration conf, StatsLogger statsLogger, + CopyOnWriteArrayList delayReplicationPeriods) + throws CompatibilityException, KeeperException, InterruptedException, IOException { + super(conf, statsLogger); + this.delayReplicationPeriods = delayReplicationPeriods; + } + + @Override + void scheduleTaskWithDelay(TimerTask timerTask, long delayPeriod) { + delayReplicationPeriods.add(delayPeriod); + super.scheduleTaskWithDelay(timerTask, delayPeriod); + } + } + + @Test + public void testDeferLedgerLockReleaseForReplicationWorker() throws Exception { + int ensembleSize = 3; + LedgerHandle lh = bkc.createLedger(ensembleSize, ensembleSize, BookKeeper.DigestType.CRC32, TESTPASSWD); + int numOfEntries = 7; + for (int i = 0; i < numOfEntries; i++) { + lh.addEntry(data); + } + lh.close(); + + BookieSocketAddress[] bookiesKilled = new BookieSocketAddress[ensembleSize]; + ServerConfiguration[] killedBookiesConfig = new ServerConfiguration[ensembleSize]; + + // kill all bookies + for (int i = 0; i < ensembleSize; i++) { + bookiesKilled[i] = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(i); + killedBookiesConfig[i] = getBkConf(bookiesKilled[i]); + LOG.info("Killing Bookie : {}", bookiesKilled[i]); + killBookie(bookiesKilled[i]); + } + + // start new bookiesToKill number of bookies + for (int i = 0; i < ensembleSize; i++) { + startNewBookieAndReturnAddress(); + } + + // create couple of replicationworkers + long lockReleaseOfFailedLedgerGracePeriod = 64L; + long baseBackoffForLockReleaseOfFailedLedger = lockReleaseOfFailedLedgerGracePeriod + / (int) Math.pow(2, ReplicationWorker.NUM_OF_EXPONENTIAL_BACKOFF_RETRIALS); + ServerConfiguration newRWConf = new ServerConfiguration(baseConf); + newRWConf.setLockReleaseOfFailedLedgerGracePeriod(Long.toString(lockReleaseOfFailedLedgerGracePeriod)); + newRWConf.setRereplicationEntryBatchSize(1000); + CopyOnWriteArrayList rw1DelayReplicationPeriods = new CopyOnWriteArrayList(); + CopyOnWriteArrayList rw2DelayReplicationPeriods = new CopyOnWriteArrayList(); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger1 = statsProvider.getStatsLogger("rw1"); + TestStatsLogger statsLogger2 = statsProvider.getStatsLogger("rw2"); + ReplicationWorker rw1 = new InjectedReplicationWorker(newRWConf, statsLogger1, rw1DelayReplicationPeriods); + ReplicationWorker rw2 = new InjectedReplicationWorker(newRWConf, statsLogger2, rw2DelayReplicationPeriods); + + Counter numEntriesUnableToReadForReplication1 = statsLogger1 + .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION); + Counter numEntriesUnableToReadForReplication2 = statsLogger2 + .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION); + @Cleanup + MetadataClientDriver clientDriver = MetadataDrivers + .getClientDriver(URI.create(baseClientConf.getMetadataServiceUri())); + clientDriver.initialize(baseClientConf, scheduler, NullStatsLogger.INSTANCE, Optional.empty()); + + LedgerManagerFactory mFactory = clientDriver.getLedgerManagerFactory(); + + LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + try { + // mark ledger underreplicated + for (int i = 0; i < bookiesKilled.length; i++) { + underReplicationManager.markLedgerUnderreplicated(lh.getId(), bookiesKilled[i].toString()); + } + while (!ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)) { + Thread.sleep(100); + } + rw1.start(); + rw2.start(); + + // wait for RWs to complete 'numOfAttemptsToWaitFor' failed attempts + int numOfAttemptsToWaitFor = 10; + while ((rw1.replicationFailedLedgers.get(lh.getId()).get() < numOfAttemptsToWaitFor) + || rw2.replicationFailedLedgers.get(lh.getId()).get() < numOfAttemptsToWaitFor) { + Thread.sleep(500); + } + + /* + * since all the bookies containing the ledger entries are down + * replication wouldn't have succeeded. + */ + assertTrue("Ledger: " + lh.getId() + " should be underreplicated", + ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)); + + /* + * since RW failed 'numOfAttemptsToWaitFor' number of times, we + * should have atleast (numOfAttemptsToWaitFor - 1) + * delayReplicationPeriods and their value should be + * (lockReleaseOfFailedLedgerGracePeriod/16) , 2 * previous value,.. + * with max : lockReleaseOfFailedLedgerGracePeriod + */ + for (int i = 0; i < ((numOfAttemptsToWaitFor - 1)); i++) { + long expectedDelayValue = Math.min(lockReleaseOfFailedLedgerGracePeriod, + baseBackoffForLockReleaseOfFailedLedger * (1 << i)); + assertEquals("RW1 delayperiod", (Long) expectedDelayValue, rw1DelayReplicationPeriods.get(i)); + assertEquals("RW2 delayperiod", (Long) expectedDelayValue, rw2DelayReplicationPeriods.get(i)); + } + + /* + * RW wont try to replicate until and unless RW succeed in reading + * those failed entries before proceeding with replication of under + * replicated fragment, so the numEntriesUnableToReadForReplication + * should be just 'numOfEntries', though RW failed to replicate + * multiple times. + */ + assertEquals("numEntriesUnableToReadForReplication for RW1", Long.valueOf((long) numOfEntries), + numEntriesUnableToReadForReplication1.get()); + assertEquals("numEntriesUnableToReadForReplication for RW2", Long.valueOf((long) numOfEntries), + numEntriesUnableToReadForReplication2.get()); + + /* + * Since these entries are unavailable, + * unableToReadEntriesForReplication should be of size numOfEntries. + */ + assertEquals("RW1 unabletoreadentries", numOfEntries, + rw1.unableToReadEntriesForReplication.get(lh.getId()).size()); + assertEquals("RW2 unabletoreadentries", numOfEntries, + rw2.unableToReadEntriesForReplication.get(lh.getId()).size()); } finally { rw1.shutdown(); rw2.shutdown(); diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index d83308e70b8..8e3da6eaade 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -697,11 +697,11 @@ groups: description: The number of entries that a replication will rereplicate in parallel. default: 10 - param: openLedgerRereplicationGracePeriod - description: The grace period, in seconds, that the replication worker waits before fencing and replicating a ledger fragment that's still being written to upon bookie failure. - default: 30 + description: The grace period, in milliseconds, that the replication worker waits before fencing and replicating a ledger fragment that's still being written to upon bookie failure. + default: 30000 - param: lockReleaseOfFailedLedgerGracePeriod - description: the grace period so that if the replication worker fails to replicate a underreplicatedledger for more than ReplicationWorker.MAXNUMBER_REPLICATION_FAILURES_ALLOWED_BEFORE_DEFERRING number of times, then instead of releasing the lock immediately after failed attempt, it will hold under replicated ledger lock for this grace period and then it will release the lock. - default: 60 + description: Set the grace period, in milliseconds, which the replication worker has to wait before releasing the lock after it failed to replicate a ledger. For the first ReplicationWorker.NUM_OF_EXPONENTIAL_BACKOFF_RETRIALS failures it will do exponential backoff then it will bound at lockReleaseOfFailedLedgerGracePeriod. + default: 300000 - param: rwRereplicateBackoffMs description: The time to backoff when replication worker encounters exceptions on replicating a ledger, in milliseconds. default: 5000 From 1e11a41a435bf537c7c523615e72f4cf4e9bd9e9 Mon Sep 17 00:00:00 2001 From: karanmehta93 Date: Thu, 10 Oct 2019 08:31:08 -0700 Subject: [PATCH 0341/1642] Add documentation for ByteBuffer lifecycle when adding entries Descriptions of the changes in this PR: Adding documentation for the issue [here](https://github.com/apache/bookkeeper/issues/2138) as discussed [here](https://github.com/apache/bookkeeper/issues/2138#issuecomment-522099668). Master Issue: #2138 Reviewers: Enrico Olivelli This closes #2175 from karanmehta93/master --- .../org/apache/bookkeeper/client/LedgerHandle.java | 10 ++++++++++ .../org/apache/bookkeeper/client/LedgerHandleAdv.java | 6 ++++++ .../apache/bookkeeper/client/api/WriteAdvHandle.java | 8 ++++++++ .../org/apache/bookkeeper/client/api/WriteHandle.java | 8 ++++++++ 4 files changed, 32 insertions(+) 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 fd03fcc3c3f..b71ac76a02c 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 @@ -906,6 +906,7 @@ CompletableFuture readEntriesInternalAsync(long firstEntry, * * @param data * array of bytes to be written to the ledger + * do not reuse the buffer, bk-client will release it appropriately * @return the entryId of the new inserted entry */ public long addEntry(byte[] data) throws InterruptedException, BKException { @@ -932,6 +933,7 @@ public CompletableFuture appendAsync(ByteBuf data) { * entryId to be added * @param data * array of bytes to be written to the ledger + * do not reuse the buffer, bk-client will release it appropriately * @return the entryId of the new inserted entry */ public long addEntry(final long entryId, byte[] data) throws InterruptedException, BKException { @@ -944,6 +946,7 @@ public long addEntry(final long entryId, byte[] data) throws InterruptedExceptio * * @param data * array of bytes to be written to the ledger + * do not reuse the buffer, bk-client will release it appropriately * @param offset * offset from which to take bytes from data * @param length @@ -971,6 +974,7 @@ public long addEntry(byte[] data, int offset, int length) * entryId to be added. * @param data * array of bytes to be written to the ledger + * do not reuse the buffer, bk-client will release it appropriately * @param offset * offset from which to take bytes from data * @param length @@ -988,6 +992,7 @@ public long addEntry(final long entryId, byte[] data, int offset, int length) th * * @param data * array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param cb * object implementing callbackinterface * @param ctx @@ -1007,6 +1012,7 @@ public void asyncAddEntry(final byte[] data, final AddCallback cb, * entryId to be added * @param data * array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param cb * object implementing callbackinterface * @param ctx @@ -1022,6 +1028,7 @@ public void asyncAddEntry(final long entryId, final byte[] data, final AddCallba * * @param data * array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param offset * offset from which to take bytes from data * @param length @@ -1059,6 +1066,7 @@ public void asyncAddEntry(ByteBuf data, final AddCallback cb, final Object ctx) * entryId of the entry to add. * @param data * array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param offset * offset from which to take bytes from data * @param length @@ -1084,6 +1092,7 @@ public void asyncAddEntry(final long entryId, final byte[] data, final int offse * entryId of the entry to add * @param data * array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param offset * offset from which to take bytes from data * @param length @@ -1111,6 +1120,7 @@ public void asyncAddEntry(final long entryId, final byte[] data, final int offse * entryId of the entry to add. * @param data * io.netty.buffer.ByteBuf of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param cb * object implementing callbackinterface * @param 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 14317d0514a..8b098b9fa4b 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 @@ -73,6 +73,7 @@ public int compare(PendingAddOp o1, PendingAddOp o2) { * entryId of the entry to add * @param data * array of bytes to be written to the ledger + * do not reuse the buffer, bk-client will release it appropriately * @return * entryId that is just created. */ @@ -90,6 +91,7 @@ public long addEntry(final long entryId, byte[] data) throws InterruptedExceptio * entryId of the entry to add * @param data * array of bytes to be written to the ledger + * do not reuse the buffer, bk-client will release it appropriately * @param offset * offset from which to take bytes from data * @param length @@ -120,6 +122,7 @@ public long addEntry(final long entryId, byte[] data, int offset, int length) th * entryId of the entry to add * @param data * array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param cb * object implementing callbackinterface * @param ctx @@ -137,6 +140,7 @@ public void asyncAddEntry(long entryId, byte[] data, AddCallback cb, Object ctx) * entryId of the entry to add * @param data * array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param offset * offset from which to take bytes from data * @param length @@ -162,6 +166,7 @@ public void asyncAddEntry(final long entryId, final byte[] data, final int offse * entryId of the entry to add * @param data * array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param offset * offset from which to take bytes from data * @param length @@ -189,6 +194,7 @@ public void asyncAddEntry(final long entryId, final byte[] data, final int offse * entryId of the entry to add. * @param data * io.netty.buffer.ByteBuf of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately * @param cb * object implementing callbackinterface * @param ctx diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteAdvHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteAdvHandle.java index c24c6d0279a..c007a3b1f3e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteAdvHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteAdvHandle.java @@ -46,6 +46,7 @@ public interface WriteAdvHandle extends ReadHandle, ForceableHandle { * * @param entryId entryId to be added * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return an handle to the result, in case of success it will return the same value of param entryId. */ default CompletableFuture writeAsync(final long entryId, final ByteBuffer data) { @@ -57,6 +58,7 @@ default CompletableFuture writeAsync(final long entryId, final ByteBuffer * * @param entryId entryId to be added * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return the same value of param entryId. */ default long write(final long entryId, final ByteBuffer data) @@ -69,6 +71,7 @@ default long write(final long entryId, final ByteBuffer data) * * @param entryId entryId to be added. * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return an handle to the result, in case of success it will return the same value of param {@code entryId}. */ default CompletableFuture writeAsync(final long entryId, final byte[] data) { @@ -80,6 +83,7 @@ default CompletableFuture writeAsync(final long entryId, final byte[] data * * @param entryId entryId to be added. * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return same value of param {@code entryId}. */ default long write(final long entryId, final byte[] data) @@ -92,6 +96,7 @@ default long write(final long entryId, final byte[] data) * * @param entryId entryId to be added. * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @param offset the offset of the bytes array * @param length the length to data to write * @return an handle to the result, in case of success it will return the same value of param {@code entryId}. @@ -105,6 +110,7 @@ default CompletableFuture writeAsync(final long entryId, final byte[] data * * @param entryId entryId to be added. * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @param offset the offset of the bytes array * @param length the length to data to write * @return the same value of param {@code entryId}. @@ -119,6 +125,7 @@ default long write(final long entryId, final byte[] data, int offset, int length * * @param entryId entryId to be added * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return an handle to the result, in case of success it will return the same value of param entryId */ CompletableFuture writeAsync(long entryId, ByteBuf data); @@ -128,6 +135,7 @@ default long write(final long entryId, final byte[] data, int offset, int length * * @param entryId entryId to be added * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return the same value of param entryId */ default long write(long entryId, ByteBuf data) throws BKException, InterruptedException { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteHandle.java index 95a1765d0a4..ca2b6effb7c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/WriteHandle.java @@ -45,6 +45,7 @@ public interface WriteHandle extends ReadHandle, ForceableHandle { * * @param data a bytebuf to be written. The bytebuf's reference count will be decremented by 1 after the * completable future is returned + * do not reuse the buffer, bk-client will release it appropriately. * @return an handle to the result, in case of success it will return the id of the newly appended entry */ CompletableFuture appendAsync(ByteBuf data); @@ -54,6 +55,7 @@ public interface WriteHandle extends ReadHandle, ForceableHandle { * * @param data a bytebuf to be written. The bytebuf's reference count will be decremented by 1 after the * call completes. + * do not reuse the buffer, bk-client will release it appropriately. * @return the id of the newly appended entry */ default long append(ByteBuf data) throws BKException, InterruptedException { @@ -64,6 +66,7 @@ default long append(ByteBuf data) throws BKException, InterruptedException { * Add entry asynchronously to an open ledger. * * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return an handle to the result, in case of success it will return the id of the newly appended entry */ default CompletableFuture appendAsync(ByteBuffer data) { @@ -74,6 +77,7 @@ default CompletableFuture appendAsync(ByteBuffer data) { * Add entry synchronously to an open ledger. * * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return the id of the newly appended entry */ default long append(ByteBuffer data) throws BKException, InterruptedException { @@ -84,6 +88,7 @@ default long append(ByteBuffer data) throws BKException, InterruptedException { * Add an entry asynchronously to an open ledger. * * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return a completable future represents the add result, in case of success the future returns the entry id * of this newly appended entry */ @@ -95,6 +100,7 @@ default CompletableFuture appendAsync(byte[] data) { * Add an entry synchronously to an open ledger. * * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @return the entry id of this newly appended entry */ default long append(byte[] data) throws BKException, InterruptedException { @@ -105,6 +111,7 @@ default long append(byte[] data) throws BKException, InterruptedException { * Add an entry asynchronously to an open ledger. * * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @param offset the offset in the bytes array * @param length the length of the bytes to be appended * @return a completable future represents the add result, in case of success the future returns the entry id @@ -118,6 +125,7 @@ default CompletableFuture appendAsync(byte[] data, int offset, int length) * Add an entry synchronously to an open ledger. * * @param data array of bytes to be written + * do not reuse the buffer, bk-client will release it appropriately. * @param offset the offset in the bytes array * @param length the length of the bytes to be appended * @return the entry id of this newly appended entry From 18a92f714f15dbe74b1cce3d76f3682a087e1f25 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Fri, 11 Oct 2019 08:22:54 -0700 Subject: [PATCH 0342/1642] Call exceptionHandler if Bookie.start fails with exception. Descriptions of the changes in this PR: When main thread of Bookie/BookieServer has exited with exception while starting Bookie, Bookie process shouldn't be alive because of any non-daemon thread that has already started. Reviewers: Enrico Olivelli , Karan Mehta This closes #2173 from reddycharan/fixbookiestartup --- .../component/AbstractLifecycleComponent.java | 16 +++- .../common/component/ComponentStarter.java | 5 +- .../apache/bookkeeper/proto/BookieServer.java | 3 +- .../replication/AutoRecoveryMain.java | 2 +- .../server/service/AutoRecoveryService.java | 8 +- .../server/service/BookieService.java | 7 +- .../bookie/BookieInitializationTest.java | 93 ++++++++++++++++++- .../bookkeeper/bookie/BookieJournalTest.java | 6 +- 8 files changed, 120 insertions(+), 20 deletions(-) diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/AbstractLifecycleComponent.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/AbstractLifecycleComponent.java index 015d54d6d35..5b6f19ee35b 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/AbstractLifecycleComponent.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/AbstractLifecycleComponent.java @@ -25,6 +25,8 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.conf.ComponentConfiguration; import org.apache.bookkeeper.stats.StatsLogger; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A mix of {@link AbstractComponent} and {@link LifecycleComponent}. @@ -33,6 +35,8 @@ public abstract class AbstractLifecycleComponent extends AbstractComponent implements LifecycleComponent { + private static final Logger LOG = LoggerFactory.getLogger(AbstractLifecycleComponent.class); + protected final Lifecycle lifecycle = new Lifecycle(); private final Set listeners = new CopyOnWriteArraySet<>(); protected final StatsLogger statsLogger; @@ -75,7 +79,17 @@ public void start() { return; } listeners.forEach(LifecycleListener::beforeStart); - doStart(); + try { + doStart(); + } catch (Throwable exc) { + LOG.error("Failed to start Component: {}", getName(), exc); + if (uncaughtExceptionHandler != null) { + LOG.error("Calling uncaughtExceptionHandler"); + uncaughtExceptionHandler.uncaughtException(Thread.currentThread(), exc); + } else { + throw exc; + } + } lifecycle.moveToStarted(); listeners.forEach(LifecycleListener::afterStart); } diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java index e95274722e9..acb643b2ed8 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java @@ -19,6 +19,7 @@ package org.apache.bookkeeper.common.component; import java.util.concurrent.CompletableFuture; + import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -46,7 +47,7 @@ public void run() { component.close(); log.info("Closed component {} in shutdown hook successfully. Exiting.", component.getName()); FutureUtils.complete(future, null); - } catch (Exception e) { + } catch (Throwable e) { log.error("Failed to close component {} in shutdown hook gracefully, Exiting anyway", component.getName(), e); future.completeExceptionally(e); @@ -72,6 +73,8 @@ public static CompletableFuture startComponent(LifecycleComponent componen // register a component exception handler component.setExceptionHandler((t, e) -> { + log.error("Triggered exceptionHandler of Component: {} because of Exception in Thread: {}", + component.getName(), t, e); // start the shutdown hook when an uncaught exception happen in the lifecycle component. shutdownHookThread.start(); }); 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 b8c1adcef06..14d63937df5 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 @@ -39,7 +39,6 @@ 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.common.allocator.ByteBufAllocatorBuilder; import org.apache.bookkeeper.common.util.JsonUtil.ParseJsonException; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -137,7 +136,7 @@ protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) : new Bookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator); } - public void start() throws IOException, UnavailableException, InterruptedException, BKException { + public void start() throws InterruptedException { 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/AutoRecoveryMain.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java index 39b41aa6503..303a401e687 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 @@ -116,7 +116,7 @@ public AutoRecoveryMain(ServerConfiguration conf, StatsLogger statsLogger) /* * Start daemons */ - public void start() throws UnavailableException { + public void start() { auditorElector.start(); replicationWorker.start(); if (null != uncaughtExceptionHandler) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java index f8389df69d0..c7305c2ce80 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/AutoRecoveryService.java @@ -22,7 +22,6 @@ import java.lang.Thread.UncaughtExceptionHandler; import org.apache.bookkeeper.replication.AutoRecoveryMain; -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; @@ -45,6 +44,7 @@ public AutoRecoveryService(BookieConfiguration conf, StatsLogger statsLogger) th @Override public void setExceptionHandler(UncaughtExceptionHandler handler) { + super.setExceptionHandler(handler); main.setExceptionHandler(handler); } @@ -54,11 +54,7 @@ public AutoRecoveryMain getAutoRecoveryServer() { @Override protected void doStart() { - try { - this.main.start(); - } catch (UnavailableException e) { - throw new RuntimeException("Can't not start '" + NAME + "' component.", e); - } + this.main.start(); } @Override 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 d6837f6f90f..645da03bf9a 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 @@ -20,9 +20,7 @@ import java.io.IOException; import java.lang.Thread.UncaughtExceptionHandler; -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; @@ -45,6 +43,7 @@ public BookieService(BookieConfiguration conf, @Override public void setExceptionHandler(UncaughtExceptionHandler handler) { + super.setExceptionHandler(handler); server.setExceptionHandler(handler); } @@ -56,8 +55,8 @@ public BookieServer getServer() { protected void doStart() { try { this.server.start(); - } catch (IOException | UnavailableException | InterruptedException | BKException e) { - throw new RuntimeException("Failed to start bookie server", e); + } catch (InterruptedException exc) { + throw new RuntimeException("Failed to start bookie server", exc); } } 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 8ea63e1490e..77bb0bd5f8d 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 @@ -24,9 +24,13 @@ import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.BOOKIE_STATUS_FILENAME; import static org.apache.bookkeeper.util.TestUtils.countNumOfFiles; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasProperty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.anyBoolean; @@ -35,6 +39,8 @@ import static org.mockito.Mockito.mock; import com.fasterxml.jackson.databind.ObjectMapper; + +import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.UnpooledByteBufAllocator; @@ -58,6 +64,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; + import org.apache.bookkeeper.bookie.BookieException.DiskPartitionDuplicationException; import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; import org.apache.bookkeeper.bookie.Journal.LastLogMark; @@ -95,6 +103,7 @@ import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.util.DiskChecker; +import org.apache.bookkeeper.util.LoggerOutput; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; @@ -105,6 +114,7 @@ import org.powermock.reflect.Whitebox; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.event.LoggingEvent; /** * Testing bookie initialization cases. @@ -117,6 +127,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase { @Rule public final TestName runtime = new TestName(); + @Rule + public LoggerOutput loggerOutput = new LoggerOutput(); ZKMetadataBookieDriver driver; public BookieInitializationTest() { @@ -640,6 +652,85 @@ public void testBookieServiceExceptionHandler() throws Exception { startFuture.get(); } + /** + * Mock InterleavedLedgerStorage class where addEntry is mocked to throw + * OutOfMemoryError. + */ + public static class MockInterleavedLedgerStorage extends InterleavedLedgerStorage { + AtomicInteger atmoicInt = new AtomicInteger(0); + + @Override + public long addEntry(ByteBuf entry) throws IOException { + if (atmoicInt.incrementAndGet() == 10) { + throw new OutOfMemoryError("Some Injected Exception"); + } + return super.addEntry(entry); + } + } + + @Test + public void testBookieStartException() throws Exception { + File journalDir = createTempDir("bookie", "journal"); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir)); + + File ledgerDir = createTempDir("bookie", "ledger"); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir)); + + /* + * add few entries to journal file. + */ + int numOfEntries = 100; + BookieJournalTest.writeV5Journal(Bookie.getCurrentDirectory(journalDir), numOfEntries, + "testV5Journal".getBytes()); + + /* + * This Bookie is configured to use MockInterleavedLedgerStorage. + * MockInterleavedLedgerStorage throws an Error for addEntry request. + * This is to simulate Bookie/BookieServer/BookieService 'start' failure + * because of 'Bookie.readJournal' failure. + */ + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + int port = PortManager.nextFreePort(); + conf.setBookiePort(port).setJournalDirName(journalDir.getPath()) + .setLedgerDirNames(new String[] { ledgerDir.getPath() }).setMetadataServiceUri(metadataServiceUri) + .setLedgerStorageClass(MockInterleavedLedgerStorage.class.getName()); + + BookieConfiguration bkConf = new BookieConfiguration(conf); + + /* + * create cookie and write it to JournalDir/LedgerDir. + */ + Cookie.Builder cookieBuilder = Cookie.generateCookie(conf); + Cookie cookie = cookieBuilder.build(); + cookie.writeToDirectory(new File(journalDir, "current")); + cookie.writeToDirectory(new File(ledgerDir, "current")); + + /* + * Create LifecycleComponent for BookieServer and start it. + */ + LifecycleComponent server = Main.buildBookieServer(bkConf); + CompletableFuture startFuture = ComponentStarter.startComponent(server); + + /* + * Since Bookie/BookieServer/BookieService is expected to fail, it would + * cause bookie-server component's exceptionHandler to get triggered. + * This exceptionHandler will make sure all of the components to get + * closed and then finally completes the Future. + */ + startFuture.get(); + + /* + * make sure that Component's exceptionHandler is called by checking if + * the error message of ExceptionHandler is logged. This Log message is + * defined in anonymous exceptionHandler class defined in + * ComponentStarter.startComponent method. + */ + loggerOutput.expect((List logEvents) -> { + assertThat(logEvents, + hasItem(hasProperty("message", containsString("Triggered exceptionHandler of Component:")))); + }); + } + @Test public void testAutoRecoveryServiceExceptionHandler() throws Exception { ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); @@ -889,8 +980,6 @@ public void testWithDiskFullAndAbilityToCreateNewIndexFile() throws Exception { // are injecting no-op shutdown. server.shutdown(); - long usableSpace = tmpDir.getUsableSpace(); - long totalSpace = tmpDir.getTotalSpace(); conf.setDiskUsageThreshold(0.001f) .setDiskUsageWarnThreshold(0.0f).setReadOnlyModeEnabled(true).setIsForceGCAllowWhenNoSpace(true) .setMinUsableSizeForIndexFileCreation(Long.MAX_VALUE); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java index 0f1b3271032..cbac4559c7f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java @@ -111,7 +111,7 @@ private void writePartialIndexFileForLedger(File indexDir, long ledgerId, /** * Generate fence entry. */ - private ByteBuf generateFenceEntry(long ledgerId) { + private static ByteBuf generateFenceEntry(long ledgerId) { ByteBuf bb = Unpooled.buffer(); bb.writeLong(ledgerId); bb.writeLong(Bookie.METAENTRY_ID_FENCE_KEY); @@ -121,7 +121,7 @@ private ByteBuf generateFenceEntry(long ledgerId) { /** * Generate meta entry with given master key. */ - private ByteBuf generateMetaEntry(long ledgerId, byte[] masterKey) { + private static ByteBuf generateMetaEntry(long ledgerId, byte[] masterKey) { ByteBuf bb = Unpooled.buffer(); bb.writeLong(ledgerId); bb.writeLong(Bookie.METAENTRY_ID_LEDGER_KEY); @@ -290,7 +290,7 @@ private JournalChannel writeV4Journal(File journalDir, int numEntries, byte[] ma return jc; } - private JournalChannel writeV5Journal(File journalDir, int numEntries, byte[] masterKey) throws Exception { + static JournalChannel writeV5Journal(File journalDir, int numEntries, byte[] masterKey) throws Exception { long logId = System.currentTimeMillis(); JournalChannel jc = new JournalChannel(journalDir, logId); From 893c3b5bdd4358d8537d990967f47527e928d1f5 Mon Sep 17 00:00:00 2001 From: karanmehta93 Date: Thu, 17 Oct 2019 14:38:37 -0700 Subject: [PATCH 0343/1642] Update maven version to 3.6.2 for Jenkins Descriptions of the changes in this PR: ### Motivation Update maven version to 3.6.2 for Jenkins build environments, since maven 3.6.0 is no longer available in the environment. Reviewers: Enrico Olivelli , Charan Reddy Guttapalem This closes #2181 from karanmehta93/updateMavenVersion --- .test-infra/jenkins/common_job_properties.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.test-infra/jenkins/common_job_properties.groovy b/.test-infra/jenkins/common_job_properties.groovy index 54642708191..e38bd9a515e 100644 --- a/.test-infra/jenkins/common_job_properties.groovy +++ b/.test-infra/jenkins/common_job_properties.groovy @@ -179,7 +179,7 @@ class common_job_properties { } // Sets common config for Maven jobs. - static void setMavenConfig(context, mavenInstallation='Maven 3.6.0', mavenOpts='-Xmx4096m -Xms2048m') { + static void setMavenConfig(context, mavenInstallation='Maven 3.6.2', mavenOpts='-Xmx4096m -Xms2048m') { context.mavenInstallation(mavenInstallation) context.mavenOpts('-Dorg.slf4j.simpleLogger.showDateTime=true') context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\\\'T\\\'HH:mm:ss.SSS') From fb2e4490068383f95205c345137d36d9043d3d9c Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Wed, 23 Oct 2019 10:49:00 +0200 Subject: [PATCH 0344/1642] [DLOG] LogReader shouldn't be added to pending if not locking The pendingReaders set in BKDistributedLogManager exists so that if the manager is closed which the lock is being acquired for a reader, that reader will be closed (even though it hasn't been returned to the client). In the case that the reader is opened without a lock, there is not async action being performed. Previously we were also adding these readers to the pendingReaders, but they were never being removed from the pendingReaders, causing a memory leak. This change avoids adding no-locking readers to pendingReaders. Reviewers: Enrico Olivelli This closes #2185 from ivankelly/orphan-reader --- .../java/org/apache/distributedlog/BKDistributedLogManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java index 6392021b534..645d6d49e7d 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java @@ -694,7 +694,6 @@ public CompletableFuture openAsyncLogReader(DLSN fromDLSN) { subscriberId, false, statsLogger); - pendingReaders.add(reader); return FutureUtils.value(reader); } From 1d3cd72d96fef85f4611cfbea827cd3512a1a86f Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Thu, 24 Oct 2019 14:31:19 -0700 Subject: [PATCH 0345/1642] Fix logic of areEntriesOfLedgerStoredInTheBookie Descriptions of the changes in this PR: - in BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie method, segmentNum is not correctly incremented while iterating over the segments of the ledger. Fixing the logic of it. - this issue caused decombookie bookieshell command to wait forever. Reviewers: Enrico Olivelli This closes #2186 from reddycharan/fixareledgersstoredcom --- .../bookkeeper/client/BookKeeperAdmin.java | 15 ++++++-- .../client/BookKeeperAdminTest.java | 37 +++++++++++++++++++ 2 files changed, 49 insertions(+), 3 deletions(-) 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 6bd95d85cc4..88a7c089ece 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 @@ -1574,10 +1574,11 @@ public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieS while (ensemblesOfSegmentsIterator.hasNext()) { ensemble = ensemblesOfSegmentsIterator.next(); if (ensemble.contains(bookieAddress)) { - if (areEntriesOfSegmentStoredInTheBookie(ledgerMetadata, bookieAddress, segmentNo++)) { + if (areEntriesOfSegmentStoredInTheBookie(ledgerMetadata, bookieAddress, segmentNo)) { return true; } } + segmentNo++; } return false; } @@ -1590,7 +1591,7 @@ private static boolean areEntriesOfSegmentStoredInTheBookie(LedgerMetadata ledge List>> segments = new LinkedList<>(ledgerMetadata.getAllEnsembles().entrySet()); - + List currentSegmentEnsemble = segments.get(segmentNo).getValue(); boolean lastSegment = (segmentNo == (segments.size() - 1)); /* @@ -1606,6 +1607,14 @@ private static boolean areEntriesOfSegmentStoredInTheBookie(LedgerMetadata ledge return false; } + /* + * If current segment ensemble doesn't contain this bookie then return + * false. + */ + if (!currentSegmentEnsemble.contains(bookieAddress)) { + return false; + } + /* * if ensembleSize is equal to writeQuorumSize, then ofcourse all * the entries of this segment are supposed to be stored in this @@ -1639,7 +1648,7 @@ private static boolean areEntriesOfSegmentStoredInTheBookie(LedgerMetadata ledge DistributionSchedule distributionSchedule = new RoundRobinDistributionSchedule( ledgerMetadata.getWriteQuorumSize(), ledgerMetadata.getAckQuorumSize(), ledgerMetadata.getEnsembleSize()); - List currentSegmentEnsemble = segments.get(segmentNo).getValue(); + int thisBookieIndexInCurrentEnsemble = currentSegmentEnsemble.indexOf(bookieAddress); long firstEntryId = segments.get(segmentNo).getKey(); long lastEntryId = lastSegment ? ledgerMetadata.getLastEntryId() : segments.get(segmentNo + 1).getKey() - 1; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index 3d409d8002d..0bbe0f11e42 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -43,6 +43,7 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.UnderreplicatedLedger; @@ -575,4 +576,40 @@ public void testGetListOfEntriesOfLedgerWithEntriesNotStripedToABookie() throws bkAdmin.close(); bkc.close(); } + + @Test + public void testAreEntriesOfLedgerStoredInTheBookieForMultipleSegments() throws Exception { + int lastEntryId = 10; + long ledgerId = 100L; + BookieSocketAddress bookie0 = new BookieSocketAddress("bookie0:3181"); + BookieSocketAddress bookie1 = new BookieSocketAddress("bookie1:3181"); + BookieSocketAddress bookie2 = new BookieSocketAddress("bookie2:3181"); + BookieSocketAddress bookie3 = new BookieSocketAddress("bookie3:3181"); + + List ensembleOfSegment1 = new ArrayList(); + ensembleOfSegment1.add(bookie0); + ensembleOfSegment1.add(bookie1); + ensembleOfSegment1.add(bookie2); + + List ensembleOfSegment2 = new ArrayList(); + ensembleOfSegment2.add(bookie3); + ensembleOfSegment2.add(bookie1); + ensembleOfSegment2.add(bookie2); + + LedgerMetadataBuilder builder = LedgerMetadataBuilder.create(); + builder.withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withDigestType(digestType.toApiDigestType()) + .withPassword(PASSWORD.getBytes()) + .newEnsembleEntry(0, ensembleOfSegment1) + .newEnsembleEntry(lastEntryId + 1, ensembleOfSegment2) + .withLastEntryId(lastEntryId).withLength(65576).withClosedState(); + LedgerMetadata meta = builder.build(); + + assertFalse("expected areEntriesOfLedgerStoredInTheBookie to return False for bookie3", + BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie(ledgerId, bookie3, meta)); + assertTrue("expected areEntriesOfLedgerStoredInTheBookie to return true for bookie2", + BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie(ledgerId, bookie2, meta)); + } } From 3e5b5b316b6aad96ae8ba506c23242275b839ae5 Mon Sep 17 00:00:00 2001 From: karanmehta93 Date: Thu, 24 Oct 2019 14:39:01 -0700 Subject: [PATCH 0346/1642] Journal scan should throw IOException when it reads negative length Descriptions of the changes in this PR: ### Motivation During journal replay, we can encounter negative length value if there is journal corruption. Currently, due to this bug, we pass negative length to limit a buffer, which throws IllegalArgumentException. ### Changes Updated the Journal class to throw `IOException` with a clear message, instead of unclear `IllegalArgumentException`. Master Issue: #2176 Reviewers: Enrico Olivelli , Charan Reddy Guttapalem This closes #2177 from karanmehta93/master --- .../org/apache/bookkeeper/bookie/Journal.java | 7 ++-- .../bookie/BookieInitializationTest.java | 37 ++++++++++++++++++- .../bookkeeper/bookie/BookieJournalTest.java | 14 ++++++- 3 files changed, 52 insertions(+), 6 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 19d5a7961f7..090c35664f5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -797,8 +797,8 @@ public long scanJournal(long journalId, long journalPos, JournalScanner scanner) break; } boolean isPaddingRecord = false; - if (len == PADDING_MASK) { - if (journalVersion >= JournalChannel.V5) { + if (len < 0) { + if (len == PADDING_MASK && journalVersion >= JournalChannel.V5) { // skip padding bytes lenBuff.clear(); fullRead(recLog, lenBuff); @@ -812,7 +812,8 @@ public long scanJournal(long journalId, long journalPos, JournalScanner scanner) } isPaddingRecord = true; } else { - throw new IOException("Invalid record found with negative length : " + len); + LOG.error("Invalid record found with negative length: {}", len); + throw new IOException("Invalid record found with negative length " + len); } } recBuff.clear(); 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 77bb0bd5f8d..628b56176fb 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 @@ -21,6 +21,7 @@ package org.apache.bookkeeper.bookie; import static com.google.common.base.Charsets.UTF_8; +import static org.apache.bookkeeper.bookie.BookieJournalTest.writeV5Journal; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.BOOKIE_STATUS_FILENAME; import static org.apache.bookkeeper.util.TestUtils.countNumOfFiles; @@ -680,7 +681,7 @@ public void testBookieStartException() throws Exception { * add few entries to journal file. */ int numOfEntries = 100; - BookieJournalTest.writeV5Journal(Bookie.getCurrentDirectory(journalDir), numOfEntries, + writeV5Journal(Bookie.getCurrentDirectory(journalDir), numOfEntries, "testV5Journal".getBytes()); /* @@ -731,6 +732,40 @@ public void testBookieStartException() throws Exception { }); } + /** + * Test that if the journal reads an entry with negative length, it shuts down + * the bookie normally. An admin should look to see what has + * happened in this case. + */ + @Test + public void testNegativeLengthEntryBookieShutdown() throws Exception { + File journalDir = createTempDir("bookie", "journal"); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir)); + + File ledgerDir = createTempDir("bookie", "ledger"); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir)); + + writeV5Journal(Bookie.getCurrentDirectory(journalDir), 5, + "testV5Journal".getBytes(), true); + + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + conf.setJournalDirName(journalDir.getPath()) + .setLedgerDirNames(new String[] { ledgerDir.getPath() }) + .setMetadataServiceUri(null); + + Bookie b = null; + try { + b = new Bookie(conf); + b.start(); + assertFalse("Bookie should shutdown normally after catching IOException" + + " due to corrupt entry with negative length", b.isRunning()); + } finally { + if (b != null) { + b.shutdown(); + } + } + } + @Test public void testAutoRecoveryServiceExceptionHandler() throws Exception { ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java index cbac4559c7f..e8d004cc26f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java @@ -290,7 +290,13 @@ private JournalChannel writeV4Journal(File journalDir, int numEntries, byte[] ma return jc; } - static JournalChannel writeV5Journal(File journalDir, int numEntries, byte[] masterKey) throws Exception { + static JournalChannel writeV5Journal(File journalDir, int numEntries, + byte[] masterKey) throws Exception { + return writeV5Journal(journalDir, numEntries, masterKey, false); + } + + static JournalChannel writeV5Journal(File journalDir, int numEntries, + byte[] masterKey, boolean corruptLength) throws Exception { long logId = System.currentTimeMillis(); JournalChannel jc = new JournalChannel(journalDir, logId); @@ -312,7 +318,11 @@ static JournalChannel writeV5Journal(File journalDir, int numEntries, byte[] mas lastConfirmed = i; length += i; ByteBuf lenBuff = Unpooled.buffer(); - lenBuff.writeInt(packet.readableBytes()); + if (corruptLength) { + lenBuff.writeInt(-1); + } else { + lenBuff.writeInt(packet.readableBytes()); + } bc.write(lenBuff); bc.write(packet); packet.release(); From d52d770fdea5946a7a27356c24422df4428a7cec Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 25 Oct 2019 09:20:21 +0200 Subject: [PATCH 0347/1642] Upgrade Apache Parent Pom to 19 Descriptions of the changes in this PR: Update the reference to the common Apache Parent POM, version 19 ### Motivation Be up to date with ASF-wide configurations ### Changes Change the version Reviewers: Sijie Guo This closes #2184 from eolivelli/fix/update-apache-parent --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 604ae52b7a3..0ffb7f639ca 100644 --- a/pom.xml +++ b/pom.xml @@ -19,7 +19,7 @@ org.apache apache - 18 + 19 4.0.0 org.apache.bookkeeper From 614ac348567987aa0797fffcf2b80f619ad71967 Mon Sep 17 00:00:00 2001 From: Dapeng <42792537+SunDapengcmss@users.noreply.github.com> Date: Sat, 26 Oct 2019 11:53:32 +0800 Subject: [PATCH 0348/1642] [DOC] Improve the Manual deployment document. ### Motivation Following the [Manual deployment](https://bookkeeper.apache.org/docs/latest/deployment/manual/), startup bookie may have failure since there is no znode(e.g. /ledger) about bookie in zookeeper. ### Changes Modify the site/docs/latest/deployment/manual.md, set up cluster metadata before startup of bookie. Reviewers: Enrico Olivelli , Sijie Guo This closes #2183 from SunDapengcmss/branch-2183 --- site/docs/latest/deployment/manual.md | 28 ++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/site/docs/latest/deployment/manual.md b/site/docs/latest/deployment/manual.md index 49800da1822..89b3b0ec649 100644 --- a/site/docs/latest/deployment/manual.md +++ b/site/docs/latest/deployment/manual.md @@ -11,9 +11,9 @@ The easiest way to deploy BookKeeper is using schedulers like [DC/OS](../dcos), 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 +## Cluster metadata setup -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. +Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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: @@ -23,31 +23,33 @@ 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: +Once the bookie's configuration is set, you can set up cluster metadata for the cluster by running the following command from any bookie in the cluster: ```shell -$ bin/bookkeeper bookie +$ bookkeeper-server/bin/bookkeeper shell metaformat ``` -> You can also build BookKeeper [by cloning it from source](../../getting-started/installation#clone) or [using Maven](../../getting-started/installation#build-using-maven). +You can run in the formatting -### System requirements +> 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. -{% include system-requirements.md %} +Once cluster metadata formatting has been completed, your BookKeeper cluster is ready to go! -## Cluster metadata setup +## Starting up bookies + + -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: +Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: ```shell -$ bin/bookkeeper shell metaformat +$ bookkeeper-server/bin/bookkeeper bookie ``` -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. +### System requirements + +{% include system-requirements.md %} -Once cluster metadata formatting has been completed, your BookKeeper cluster is ready to go! diff --git a/site/docs/4.10.0/admin/perf.md b/site/docs/4.10.0/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.10.0/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.10.0/admin/placement.md b/site/docs/4.10.0/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.10.0/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.10.0/admin/upgrade.md b/site/docs/4.10.0/admin/upgrade.md new file mode 100644 index 00000000000..57c65208131 --- /dev/null +++ b/site/docs/4.10.0/admin/upgrade.md @@ -0,0 +1,175 @@ +--- +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.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.7.x to 4.8.X upgrade + +In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.10.0/api/distributedlog-api.md b/site/docs/4.10.0/api/distributedlog-api.md new file mode 100644 index 00000000000..5832b2176dd --- /dev/null +++ b/site/docs/4.10.0/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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.10.0/api/ledger-adv-api.md b/site/docs/4.10.0/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.10.0/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.10.0/api/ledger-api.md b/site/docs/4.10.0/api/ledger-api.md new file mode 100644 index 00000000000..adf2961a989 --- /dev/null +++ b/site/docs/4.10.0/api/ledger-api.md @@ -0,0 +1,841 @@ +--- +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.10.0 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.10.0 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.10.0' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.10.0' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 +$ 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` + +### Relaxing Durability + +In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. +In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will +be always readable by other clients. + +On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after +writing the entry to SO buffers without waiting for an fsync. +In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. +Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. + +In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble ackknowledge the call after +performing an fsync to the disk which is storing the journal. +This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. + +The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. +In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. + + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete + + +wh.force().get(); // wait for fsync, make data available to readers and to the replicator + +wh.close(); // seal the ledger + +``` diff --git a/site/docs/4.10.0/api/overview.md b/site/docs/4.10.0/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.10.0/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.10.0/deployment/dcos.md b/site/docs/4.10.0/deployment/dcos.md new file mode 100644 index 00000000000..4c84dfe846a --- /dev/null +++ b/site/docs/4.10.0/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.10.0] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.10.0/deployment/kubernetes.md b/site/docs/4.10.0/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.10.0/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.10.0/deployment/manual.md b/site/docs/4.10.0/deployment/manual.md new file mode 100644 index 00000000000..49800da1822 --- /dev/null +++ b/site/docs/4.10.0/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 +$ 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 +$ 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.10.0/development/codebase.md b/site/docs/4.10.0/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.10.0/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.10.0/development/protocol.md b/site/docs/4.10.0/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.10.0/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.10.0/getting-started/concepts.md b/site/docs/4.10.0/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.10.0/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.10.0/getting-started/installation.md b/site/docs/4.10.0/getting-started/installation.md new file mode 100644 index 00000000000..7e92b471581 --- /dev/null +++ b/site/docs/4.10.0/getting-started/installation.md @@ -0,0 +1,82 @@ +--- +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 +``` + +Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. + +```shell +$ mvn package -Dstream +``` + +> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin + +> You can enable `table service` by adding the `-Dstream` flag when running above commands. + +## 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.10.0/getting-started/run-locally.md b/site/docs/4.10.0/getting-started/run-locally.md new file mode 100644 index 00000000000..520140317d9 --- /dev/null +++ b/site/docs/4.10.0/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 +$ bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.10.0/overview/overview.md b/site/docs/4.10.0/overview/overview.md new file mode 100644 index 00000000000..9896ad4e56c --- /dev/null +++ b/site/docs/4.10.0/overview/overview.md @@ -0,0 +1,60 @@ +--- +title: Apache BookKeeper™ 4.10.0 +--- + + +This documentation is for Apache BookKeeper™ version 4.10.0. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.10.0 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.10.0/overview/releaseNotes.md b/site/docs/4.10.0/overview/releaseNotes.md new file mode 100644 index 00000000000..13018ad2fb3 --- /dev/null +++ b/site/docs/4.10.0/overview/releaseNotes.md @@ -0,0 +1,29 @@ +--- +title: Apache BookKeeper 4.10.0 Release Notes +--- + +This is the 20th release of Apache BookKeeper! + +The 4.10.0 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.9.0. + +Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.10.0](../../admin/upgrade). The technical details of +this release are summarized below. + +## News and noteworthy + +- [https://github.com/apache/bookkeeper/pull/2069] Use pure python implementation of MurmurHash +- [https://github.com/apache/bookkeeper/pull/1934] Bump Netty and GRPC version +- [https://github.com/apache/bookkeeper/pull/1907] Add new *bkctl* shell tool +- [https://github.com/apache/bookkeeper/issues/1602] Cluster Metadata Checker +- [https://github.com/apache/bookkeeper/pull/2154] Auto refresh TLS certificate at bookie-server +- [https://github.com/apache/bookkeeper/pull/2150] Improve journal throughput when journalSyncData is disabled. +- [https://github.com/apache/bookkeeper/pull/2147] Journal should respect to `flushWhenQueueEmpty` setting +- [https://github.com/apache/bookkeeper/pull/2132] Make default Bookie scripts work on JDK11+ +- [https://github.com/apache/bookkeeper/pull/2128] Allow to override default SASL service name 'bookkeeper' +- [https://github.com/apache/bookkeeper/pull/2117] BookKeeper Admin API: Implement a method to get all the Bookies +- [https://github.com/apache/bookkeeper/pull/2111] Ensure getStickyReadBookieIndex returns valid bookie index + + +## Full list of changes + +- [https://github.com/apache/bookkeeper/milestone/6](https://github.com/apache/bookkeeper/milestone/6?closed=1) diff --git a/site/docs/4.10.0/overview/releaseNotesTemplate.md b/site/docs/4.10.0/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..c29417c179f --- /dev/null +++ b/site/docs/4.10.0/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.10.0 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.10.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.10.0/reference/cli.md b/site/docs/4.10.0/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.10.0/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.10.0/reference/config.md b/site/docs/4.10.0/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.10.0/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.10.0/reference/metrics.md b/site/docs/4.10.0/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.10.0/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.10.0/security/overview.md b/site/docs/4.10.0/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.10.0/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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.10.0/security/sasl.md b/site/docs/4.10.0/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.10.0/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.10.0/security/tls.md b/site/docs/4.10.0/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.10.0/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.10.0/security/zookeeper.md b/site/docs/4.10.0/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.10.0/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) diff --git a/site/docs/latest/overview/overview.md b/site/docs/latest/overview/overview.md index 189520d8a76..9497ad85ec3 100644 --- a/site/docs/latest/overview/overview.md +++ b/site/docs/latest/overview/overview.md @@ -1,5 +1,5 @@ --- -title: Apache BookKeeper™ 4.10.0-SNAPSHOT +title: Apache BookKeeper™ 4.11.0-SNAPSHOT --- @@ -67,11 +62,6 @@ http-server ${project.version} - - org.apache.bookkeeper.http - twitter-http-server - ${project.version} - org.apache.bookkeeper.http vertx-http-server diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index d64529d6b99..d6cc2dc0ceb 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -208,9 +208,6 @@ Apache Software License, Version 2. - lib/com.fasterxml.jackson.core-jackson-annotations-2.9.7.jar [1] - lib/com.fasterxml.jackson.core-jackson-core-2.9.7.jar [2] - lib/com.fasterxml.jackson.core-jackson-databind-2.9.7.jar [3] -- lib/com.fasterxml.jackson.module-jackson-module-paranamer-2.9.7.jar [4] -- lib/com.fasterxml.jackson.module-jackson-module-scala_2.11-2.9.7.jar [5] -- lib/com.github.ben-manes.caffeine-caffeine-2.3.4.jar [9] - lib/com.google.guava-guava-21.0.jar [6] - lib/commons-cli-commons-cli-1.2.jar [7] - lib/commons-codec-commons-codec-1.6.jar [8] @@ -218,112 +215,80 @@ Apache Software License, Version 2. - lib/commons-io-commons-io-2.4.jar [11] - lib/commons-lang-commons-lang-2.6.jar [12] - lib/commons-logging-commons-logging-1.1.1.jar [13] -- lib/com.twitter-finagle-base-http_2.11-6.44.0.jar [14] -- lib/com.twitter-finagle-core_2.11-6.44.0.jar [15] -- lib/com.twitter-finagle-http_2.11-6.44.0.jar [14] -- lib/com.twitter-finagle-http2_2.11-6.44.0.jar [14] -- lib/com.twitter-finagle-netty4_2.11-6.44.0.jar [14] -- lib/com.twitter-finagle-netty4-http_2.11-6.44.0.jar [14] -- lib/com.twitter-finagle-thrift_2.11-6.44.0.jar [14] -- lib/com.twitter-finagle-toggle_2.11-6.44.0.jar [14] -- lib/com.twitter-finagle-tunable_2.11-6.44.0.jar [14] -- lib/com.twitter-finagle-zipkin-core_2.11-6.44.0.jar [14] -- lib/com.twitter-libthrift-0.5.0-7.jar [16] -- lib/com.twitter-scrooge-core_2.11-4.16.0.jar [17] -- lib/com.twitter-twitter-server_2.11-1.29.0.jar [18] -- lib/com.twitter-util-app_2.11-6.43.0.jar [19] -- lib/com.twitter-util-cache_2.11-6.43.0.jar [19] -- lib/com.twitter-util-codec_2.11-6.43.0.jar [19] -- lib/com.twitter-util-collection_2.11-6.43.0.jar [19] -- lib/com.twitter-util-core_2.11-6.43.0.jar [19] -- lib/com.twitter-util-events_2.11-6.43.0.jar [20] -- lib/com.twitter-util-function_2.11-6.43.0.jar [19] -- lib/com.twitter-util-hashing_2.11-6.43.0.jar [19] -- lib/com.twitter-util-jvm_2.11-6.43.0.jar [19] -- lib/com.twitter-util-lint_2.11-6.43.0.jar [19] -- lib/com.twitter-util-logging_2.11-6.43.0.jar [19] -- lib/com.twitter-util-registry_2.11-6.43.0.jar [19] -- lib/com.twitter-util-security_2.11-6.43.0.jar [19] -- lib/com.twitter-util-stats_2.11-6.43.0.jar [19] -- lib/com.twitter-util-tunable_2.11-6.43.0.jar [20] -- lib/io.dropwizard.metrics-metrics-core-3.1.0.jar [21] -- lib/io.dropwizard.metrics-metrics-graphite-3.1.0.jar [21] -- lib/io.dropwizard.metrics-metrics-jvm-3.1.0.jar [21] -- lib/io.netty-netty-3.10.1.Final.jar [22] -- lib/io.netty-netty-buffer-4.1.32.Final.jar [23] -- lib/io.netty-netty-codec-4.1.32.Final.jar [23] -- lib/io.netty-netty-codec-dns-4.1.32.Final.jar [23] -- lib/io.netty-netty-codec-http-4.1.32.Final.jar [23] -- lib/io.netty-netty-codec-http2-4.1.32.Final.jar [23] -- lib/io.netty-netty-codec-socks-4.1.32.Final.jar [23] -- lib/io.netty-netty-common-4.1.32.Final.jar [23] -- lib/io.netty-netty-handler-4.1.32.Final.jar [23] -- lib/io.netty-netty-handler-proxy-4.1.32.Final.jar [23] -- lib/io.netty-netty-resolver-4.1.32.Final.jar [23] -- lib/io.netty-netty-resolver-dns-4.1.32.Final.jar [23] -- lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar [23] -- lib/io.netty-netty-transport-4.1.32.Final.jar [23] -- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar [23] -- lib/io.netty-netty-transport-native-epoll-4.1.32.Final-linux-x86_64.jar [23] -- lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar [23] -- lib/io.prometheus-simpleclient-0.0.21.jar [24] -- lib/io.prometheus-simpleclient_common-0.0.21.jar [24] -- lib/io.prometheus-simpleclient_hotspot-0.0.21.jar [24] -- lib/io.prometheus-simpleclient_servlet-0.0.21.jar [24] -- lib/io.vertx-vertx-auth-common-3.4.1.jar [25] -- lib/io.vertx-vertx-core-3.4.1.jar [26] -- lib/io.vertx-vertx-web-3.4.1.jar [27] -- lib/log4j-log4j-1.2.17.jar [29] -- lib/net.java.dev.jna-jna-3.2.7.jar [30] -- lib/org.apache.commons-commons-collections4-4.1.jar [31] -- lib/org.apache.commons-commons-lang3-3.6.jar [32] -- lib/org.apache.zookeeper-zookeeper-3.4.13.jar [33] -- lib/org.eclipse.jetty-jetty-http-9.4.5.v20170502.jar [34] -- lib/org.eclipse.jetty-jetty-io-9.4.5.v20170502.jar [34] -- lib/org.eclipse.jetty-jetty-security-9.4.5.v20170502.jar [34] -- lib/org.eclipse.jetty-jetty-server-9.4.5.v20170502.jar [34] -- lib/org.eclipse.jetty-jetty-servlet-9.4.5.v20170502.jar [34] -- lib/org.eclipse.jetty-jetty-util-9.4.5.v20170502.jar [34] -- lib/org.rocksdb-rocksdbjni-5.13.1.jar [35] -- lib/com.beust-jcommander-1.48.jar [36] -- lib/com.yahoo.datasketches-memory-0.8.3.jar [37] -- lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [37] -- lib/net.jpountz.lz4-lz4-1.3.0.jar [38] -- lib/org.codehaus.jackson-jackson-core-asl-1.9.11.jar [39] -- lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [39] -- lib/com.google.api.grpc-proto-google-common-protos-1.12.0.jar [40] -- lib/com.google.code.gson-gson-2.7.jar [41] -- lib/io.opencensus-opencensus-api-0.18.0.jar [42] -- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.18.0.jar [42] -- lib/com.squareup.okhttp-okhttp-2.5.0.jar [43] -- lib/com.squareup.okio-okio-1.13.0.jar [44] -- lib/io.grpc-grpc-all-1.18.0.jar [45] -- lib/io.grpc-grpc-auth-1.18.0.jar [45] -- lib/io.grpc-grpc-context-1.18.0.jar [45] -- lib/io.grpc-grpc-core-1.18.0.jar [45] -- lib/io.grpc-grpc-netty-1.18.0.jar [45] -- lib/io.grpc-grpc-okhttp-1.18.0.jar [45] -- lib/io.grpc-grpc-protobuf-1.18.0.jar [45] -- lib/io.grpc-grpc-protobuf-lite-1.18.0.jar [45] -- lib/io.grpc-grpc-protobuf-nano-1.18.0.jar [45] -- lib/io.grpc-grpc-stub-1.18.0.jar [45] -- lib/io.grpc-grpc-testing-1.18.0.jar [45] -- lib/org.apache.curator-curator-client-4.0.1.jar [46] -- lib/org.apache.curator-curator-framework-4.0.1.jar [46] -- lib/org.apache.curator-curator-recipes-4.0.1.jar [46] -- lib/org.inferred-freebuilder-1.14.9.jar [47] -- lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [48] -- lib/org.apache.yetus-audience-annotations-0.5.0.jar [49] -- lib/org.jctools-jctools-core-2.1.2.jar [50] -- lib/org.apache.httpcomponents-httpclient-4.4.1.jar [51] -- lib/org.apache.httpcomponents-httpcore-4.4.1.jar [52] -- lib/org.apache.thrift-libthrift-0.9.3.jar [53] +- lib/io.dropwizard.metrics-metrics-core-3.1.0.jar [14] +- lib/io.dropwizard.metrics-metrics-graphite-3.1.0.jar [14] +- lib/io.dropwizard.metrics-metrics-jvm-3.1.0.jar [14] +- lib/io.netty-netty-buffer-4.1.32.Final.jar [16] +- lib/io.netty-netty-codec-4.1.32.Final.jar [16] +- lib/io.netty-netty-codec-dns-4.1.32.Final.jar [16] +- lib/io.netty-netty-codec-http-4.1.32.Final.jar [16] +- lib/io.netty-netty-codec-http2-4.1.32.Final.jar [16] +- lib/io.netty-netty-codec-socks-4.1.32.Final.jar [16] +- lib/io.netty-netty-common-4.1.32.Final.jar [16] +- lib/io.netty-netty-handler-4.1.32.Final.jar [16] +- lib/io.netty-netty-handler-proxy-4.1.32.Final.jar [16] +- lib/io.netty-netty-resolver-4.1.32.Final.jar [16] +- lib/io.netty-netty-resolver-dns-4.1.32.Final.jar [16] +- lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar [16] +- lib/io.netty-netty-transport-4.1.32.Final.jar [16] +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final-linux-x86_64.jar [16] +- lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar [16] +- lib/io.prometheus-simpleclient-0.0.21.jar [17] +- lib/io.prometheus-simpleclient_common-0.0.21.jar [17] +- lib/io.prometheus-simpleclient_hotspot-0.0.21.jar [17] +- lib/io.prometheus-simpleclient_servlet-0.0.21.jar [17] +- lib/io.vertx-vertx-auth-common-3.4.1.jar [18] +- lib/io.vertx-vertx-core-3.4.1.jar [19] +- lib/io.vertx-vertx-web-3.4.1.jar [20] +- lib/log4j-log4j-1.2.17.jar [21] +- lib/net.java.dev.jna-jna-3.2.7.jar [22] +- lib/org.apache.commons-commons-collections4-4.1.jar [23] +- lib/org.apache.commons-commons-lang3-3.6.jar [24] +- lib/org.apache.zookeeper-zookeeper-3.4.13.jar [25] +- lib/org.eclipse.jetty-jetty-http-9.4.5.v20170502.jar [26] +- lib/org.eclipse.jetty-jetty-io-9.4.5.v20170502.jar [26] +- lib/org.eclipse.jetty-jetty-security-9.4.5.v20170502.jar [26] +- lib/org.eclipse.jetty-jetty-server-9.4.5.v20170502.jar [26] +- lib/org.eclipse.jetty-jetty-servlet-9.4.5.v20170502.jar [26] +- lib/org.eclipse.jetty-jetty-util-9.4.5.v20170502.jar [26] +- lib/org.rocksdb-rocksdbjni-5.13.1.jar [27] +- lib/com.beust-jcommander-1.48.jar [28] +- lib/com.yahoo.datasketches-memory-0.8.3.jar [29] +- lib/com.yahoo.datasketches-sketches-core-0.8.3.jar [29] +- lib/net.jpountz.lz4-lz4-1.3.0.jar [30] +- lib/org.codehaus.jackson-jackson-core-asl-1.9.11.jar [31] +- lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar [31] +- lib/com.google.api.grpc-proto-google-common-protos-1.12.0.jar [32] +- lib/com.google.code.gson-gson-2.7.jar [33] +- lib/io.opencensus-opencensus-api-0.18.0.jar [34] +- lib/io.opencensus-opencensus-contrib-grpc-metrics-0.18.0.jar [35] +- lib/com.squareup.okhttp-okhttp-2.5.0.jar [36] +- lib/com.squareup.okio-okio-1.13.0.jar [36] +- lib/io.grpc-grpc-all-1.18.0.jar [37] +- lib/io.grpc-grpc-auth-1.18.0.jar [37] +- lib/io.grpc-grpc-context-1.18.0.jar [37] +- lib/io.grpc-grpc-core-1.18.0.jar [37] +- lib/io.grpc-grpc-netty-1.18.0.jar [37] +- lib/io.grpc-grpc-okhttp-1.18.0.jar [37] +- lib/io.grpc-grpc-protobuf-1.18.0.jar [37] +- lib/io.grpc-grpc-protobuf-lite-1.18.0.jar [37] +- lib/io.grpc-grpc-protobuf-nano-1.18.0.jar [37] +- lib/io.grpc-grpc-stub-1.18.0.jar [37] +- lib/io.grpc-grpc-testing-1.18.0.jar [37] +- lib/org.apache.curator-curator-client-4.0.1.jar [38] +- lib/org.apache.curator-curator-framework-4.0.1.jar [38] +- lib/org.apache.curator-curator-recipes-4.0.1.jar [38] +- lib/org.inferred-freebuilder-1.14.9.jar [39] +- lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [40] +- lib/org.apache.yetus-audience-annotations-0.5.0.jar [41] +- lib/org.jctools-jctools-core-2.1.2.jar [42] +- lib/org.apache.httpcomponents-httpclient-4.4.1.jar [43] +- lib/org.apache.httpcomponents-httpcore-4.4.1.jar [44] +- lib/org.apache.thrift-libthrift-0.9.3.jar [45] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.9.7 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.9.7 [3] Source available at https://github.com/FasterXML/jackson-databind/tree/jackson-databind-2.9.7 -[4] Source available at https://github.com/FasterXML/jackson-modules-base/tree/jackson-modules-base-2.9.7 -[5] Source available at https://github.com/FasterXML/jackson-module-scala/tree/jackson-module-scala-2.9.7 [6] Source available at https://github.com/google/guava/tree/v21.0 [7] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-cli.git;a=tag;h=bc8f0e [8] Source available at http://svn.apache.org/viewvc/commons/proper/codec/tags/1_6/ @@ -332,79 +297,37 @@ Apache Software License, Version 2. [11] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-io.git;a=tag;h=603579 [12] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=shortlog;h=refs/tags/LANG_3_6 [13] Source available at http://svn.apache.org/viewvc/commons/proper/logging/tags/commons-logging-1.1.1/ -[14] Source available at https://github.com/twitter/finagle/tree/finagle-6.44.0 -[15] Source available at https://github.com/twitter/finagle/tree/finagle-6.43.0 -[16] Source unavailable -[17] Source available at https://github.com/twitter/scrooge/tree/scrooge-4.16.0 -[18] Source available at https://github.com/twitter/twitter-server/tree/twitter-server-1.29.0 -[19] Source available at https://github.com/twitter/util/tree/util-6.43.0 -[20] Source available at https://github.com/twitter/util/tree/util-6.43.0 -[21] Source available at https://github.com/dropwizard/metrics/tree/v3.1.0 -[22] Source available at https://bintray.com/netty/downloads/download_file?file_path=netty-3.10.1.Final-dist.tar.bz2 -[23] Source available at https://github.com/netty/netty/tree/netty-4.1.32.Final -[24] Source available at https://github.com/prometheus/client_java/tree/parent-0.0.21 -[25] Source available at https://github.com/vert-x3/vertx-auth/tree/3.4.1 -[26] Source available at https://github.com/eclipse/vert.x/tree/3.4.1 -[27] Source available at https://github.com/vert-x3/vertx-web/tree/3.4.1 -[29] Source available at http://logging.apache.org/log4j/1.2/download.html -[30] Source available at https://github.com/java-native-access/jna/tree/3.2.7 -[31] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad -[32] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=tag;h=3ad2e8 -[33] Source available at https://github.com/apache/zookeeper/tree/release-3.4.13 -[34] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502 -[35] Source available at https://github.com/facebook/rocksdb/tree/v5.13.1 -[36] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48 -[37] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3 -[38] Source available at https://github.com/lz4/lz4-java/tree/1.3.0 -[39] Source available at https://github.com/codehaus/jackson/tree/d1c26ec538a8701cff7548009254fd6183b71873 -[40] Source available at https://github.com/googleapis/googleapis -[41] Source available at https://github.com/google/gson/tree/gson-parent-2.7 -[42] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.18.0 -[43] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 -[44] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 -[45] Source available at https://github.com/grpc/grpc-java/tree/v1.18.0 -[46] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 -[47] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 -[48] Source available at https://github.com/google/error-prone/tree/v2.1.2 -[49] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 -[50] Source available at https://github.com/JCTools/JCTools/tree/v2.1.2 -[51] Source available at https://github.com/apache/httpcomponents-client/tree/rel/v4.4.1 -[52] Source available at https://github.com/apache/httpcomponents-core/tree/rel/v4.4.1 -[53] Source available at https://github.com/apache/thrift/tree/0.9.3 - ------------------------------------------------------------------------------------- -lib/io.netty-netty-3.10.1.Final.jar contains the extensions to Java Collections Framework which has -been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: - - * LICENSE: - * deps/netty-3.10.1.Final/LICENSE.jsr166y.txt (Public Domain) - * HOMEPAGE: - * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ - * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ - -lib/io.netty-netty-3.10.1.Final.jar contains a modified version of Robert Harder's Public Domain -Base64 Encoder and Decoder, which can be obtained at: - - * LICENSE: - * deps/netty-3.10.1.Final/LICENSE.base64.txt (Public Domain) - * HOMEPAGE: - * http://iharder.sourceforge.net/current/java/base64/ - -lib/io.netty-netty-3.10.1.Final.jar contains a modified version of 'JZlib', a re-implementation of -zlib in pure Java, which can be obtained at: - - * LICENSE: - * deps/netty-3.10.1.Final/LICENSE.jzlib.txt (BSD Style License) - * HOMEPAGE: - * http://www.jcraft.com/jzlib/ - -lib/io.netty-netty-3.10.1.Final.jar contains a modified version of 'Webbit', a Java event based -WebSocket and HTTP server: - - * LICENSE: - * deps/netty-3.10.1.Final/LICENSE.webbit.txt (BSD License) - * HOMEPAGE: - * https://github.com/joewalnes/webbit +[14] Source available at https://github.com/dropwizard/metrics/tree/v3.1.0 +[16] Source available at https://github.com/netty/netty/tree/netty-4.1.32.Final +[17] Source available at https://github.com/prometheus/client_java/tree/parent-0.0.21 +[18] Source available at https://github.com/vert-x3/vertx-auth/tree/3.4.1 +[19] Source available at https://github.com/eclipse/vert.x/tree/3.4.1 +[20] Source available at https://github.com/vert-x3/vertx-web/tree/3.4.1 +[21] Source available at http://logging.apache.org/log4j/1.2/download.html +[22] Source available at https://github.com/java-native-access/jna/tree/3.2.7 +[23] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad +[24] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=tag;h=3ad2e8 +[25] Source available at https://github.com/apache/zookeeper/tree/release-3.4.13 +[26] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502 +[27] Source available at https://github.com/facebook/rocksdb/tree/v5.13.1 +[28] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48 +[29] Source available at https://github.com/DataSketches/sketches-core/tree/sketches-0.8.3 +[30] Source available at https://github.com/lz4/lz4-java/tree/1.3.0 +[31] Source available at https://github.com/codehaus/jackson/tree/d1c26ec538a8701cff7548009254fd6183b71873 +[32] Source available at https://github.com/googleapis/googleapis +[33] Source available at https://github.com/google/gson/tree/gson-parent-2.7 +[34] Source available at https://github.com/census-instrumentation/opencensus-java/tree/v0.18.0 +[35] Source available at https://github.com/square/okhttp/tree/parent-2.5.0 +[36] Source available at https://github.com/square/okio/tree/okio-parent-1.13.0 +[37] Source available at https://github.com/grpc/grpc-java/tree/v1.18.0 +[38] Source available at https://github.com/apache/curator/tree/apache-curator-4.0.1 +[39] Source available at https://github.com/inferred/FreeBuilder/tree/v1.14.9 +[40] Source available at https://github.com/google/error-prone/tree/v2.1.2 +[41] Source available at https://github.com/apache/yetus/tree/rel/0.5.0 +[42] Source available at https://github.com/JCTools/JCTools/tree/v2.1.2 +[43] Source available at https://github.com/apache/httpcomponents-client/tree/rel/v4.4.1 +[44] Source available at https://github.com/apache/httpcomponents-core/tree/rel/v4.4.1 +[45] Source available at https://github.com/apache/thrift/tree/0.9.3 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-4.1.32.Final.jar bundles some 3rd party dependencies @@ -517,36 +440,12 @@ Bundled as lib/com.google.protobuf.nano-protobuf-javanano-3.0.0-alpha-5.jar Source available at https://github.com/google/protobuf/tree/3.0.0-pre For details, see deps/protobuf-3.0.0/LICENSE. ------------------------------------------------------------------------------------ -This product bundles Paranamer, which is available under a "3-clause BSD" license. -For details, see deps/paranamer-2.8/LICENSE.txt. - -Bundled as lib/com.thoughtworks.paranamer-paranamer-2.8.jar -Source available at https://github.com/paul-hammant/paranamer/tree/paranamer-parent-2.8 ------------------------------------------------------------------------------------- This product bundles the JCP Standard Java Servlet API, which is available under a CDDL 1.1 license. For details, see deps/javax.servlet-api-3.1.0/CDDL+GPL-1.1. Bundled as lib/javax.servlet-javax.servlet-api-3.1.0.jar Source available at https://github.com/javaee/servlet-spec/tree/3.1.0 ------------------------------------------------------------------------------------ -This product bundles the Scala Standard Parser Combinator Library, which is available -under a "3-clause BSD" license. For details, see deps/scala-parser-combinators_2.11-1.0.4/LICENSE.md. - -Bundled as lib/org.scala-lang.modules-scala-parser-combinators_2.11-1.0.4.jar -Source available at https://github.com/scala/scala-parser-combinators/tree/v1.0.4 ------------------------------------------------------------------------------------- -This product bundles the Scala Standard Library, which is available under a -"3-clause BSD" license. For details, see deps/scala-library-2.11.7/LICENSE.md. - -Bundled as lib/org.scala-lang-scala-library-2.11.8.jar -Source available at org.scala-lang-scala-library-2.11.8.jar ------------------------------------------------------------------------------------- -This product bundles the Scala Reflection API, which is available under a -"3-clause BSD" license. For details, see deps/scala-reflect-2.11.8/LICENSE.md. - -Bundled as lib/org.scala-lang-scala-reflect-2.11.8.jar -Source available at https://github.com/scala/scala/tree/v2.11.8 ------------------------------------------------------------------------------------- This product bundles Simple Logging Facade for Java, which is available under a MIT license. For details, see deps/slf4j-1.7.25/LICENSE.txt. diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt index 68ee3f4f877..1fe42dd65bd 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -226,7 +226,7 @@ Apache Software License, Version 2. - lib/io.netty-netty-resolver-4.1.32.Final.jar [11] - lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar [11] - lib/io.netty-netty-transport-4.1.32.Final.jar [11] -- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar [11] +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final-linux-x86_64.jar [11] - lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar [11] - lib/log4j-log4j-1.2.17.jar [16] - lib/net.java.dev.jna-jna-3.2.7.jar [17] diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index c66df18d269..15f5962f55b 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -228,7 +228,7 @@ Apache Software License, Version 2. - lib/io.netty-netty-resolver-dns-4.1.32.Final.jar [11] - lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar [11] - lib/io.netty-netty-transport-4.1.32.Final.jar [11] -- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar [11] +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final-linux-x86_64.jar [11] - lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar [11] - lib/io.prometheus-simpleclient-0.0.21.jar [12] - lib/io.prometheus-simpleclient_common-0.0.21.jar [12] diff --git a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt index 6916fda9a0c..5a25036aac2 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt @@ -20,29 +20,6 @@ LongAdder), which was released with the following comments: Written by Doug Lea with assistance from members of JCP JSR-166 Expert Group and released to the public domain, as explained at http://creativecommons.org/publicdomain/zero/1.0/ ------------------------------------------------------------------------------------- -- lib/io.netty-netty-3.10.1.Final.jar - - The Netty Project - ================= - -Please visit the Netty web site for more information: - - * http://netty.io/ - -Copyright 2011 The Netty Project - -The Netty Project 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. ------------------------------------------------------------------------------------ - lib/io.netty-netty-buffer-4.1.32.Final.jar @@ -58,7 +35,6 @@ under the License. - lib/io.netty-netty-resolver-dns-4.1.32.Final.jar - lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar - lib/io.netty-netty-transport-4.1.32.Final.jar -- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar - lib/io.netty-netty-transport-native-epoll-4.1.32.Final-linux-x86_64.jar - lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar diff --git a/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt index d7f93303f70..5d834105907 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt @@ -16,7 +16,7 @@ The Apache Software Foundation (http://www.apache.org/). - lib/io.netty-netty-resolver-4.1.32.Final.jar - lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar - lib/io.netty-netty-transport-4.1.32.Final.jar -- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final-linux-x86_64.jar - lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar diff --git a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt index cc2060e871a..8d24923893c 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt @@ -18,7 +18,7 @@ The Apache Software Foundation (http://www.apache.org/). - lib/io.netty-netty-resolver-dns-4.1.32.Final.jar - lib/io.netty-netty-tcnative-boringssl-static-2.0.20.Final.jar - lib/io.netty-netty-transport-4.1.32.Final.jar -- lib/io.netty-netty-transport-native-epoll-4.1.32.Final.jar +- lib/io.netty-netty-transport-native-epoll-4.1.32.Final-linux-x86_64.jar - lib/io.netty-netty-transport-native-unix-common-4.1.32.Final.jar diff --git a/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.base64.txt b/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.base64.txt deleted file mode 100644 index 31ebc840539..00000000000 --- a/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.base64.txt +++ /dev/null @@ -1,26 +0,0 @@ -The person or persons who have associated work with this document (the -"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of -his knowledge, the work of authorship identified is in the public domain of -the country from which the work is published, or (b) hereby dedicates whatever -copyright the dedicators holds in the work of authorship identified below (the -"Work") to the public domain. A certifier, moreover, dedicates any copyright -interest he may have in the associated work, and for these purposes, is -described as a "dedicator" below. - -A certifier has taken reasonable steps to verify the copyright status of this -work. Certifier recognizes that his good faith efforts may not shield him from -liability if in fact the work certified is not in the public domain. - -Dedicator makes this dedication for the benefit of the public at large and to -the detriment of the Dedicator's heirs and successors. Dedicator intends this -dedication to be an overt act of relinquishment in perpetuate of all present -and future rights under copyright law, whether vested or contingent, in the -Work. Dedicator understands that such relinquishment of all rights includes -the relinquishment of all rights to enforce (by lawsuit or otherwise) those -copyrights in the Work. - -Dedicator recognizes that, once placed in the public domain, the Work may be -freely reproduced, distributed, transmitted, used, modified, built upon, or -otherwise exploited by anyone for any purpose, commercial or non-commercial, -and in any way, including by methods that have not yet been invented or -conceived. diff --git a/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.jsr166y.txt b/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.jsr166y.txt deleted file mode 100644 index b1c292b54cb..00000000000 --- a/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.jsr166y.txt +++ /dev/null @@ -1,26 +0,0 @@ -The person or persons who have associated work with this document (the -"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of -his knowledge, the work of authorship identified is in the public domain of -the country from which the work is published, or (b) hereby dedicates whatever -copyright the dedicators holds in the work of authorship identified below (the -"Work") to the public domain. A certifier, moreover, dedicates any copyright -interest he may have in the associated work, and for these purposes, is -described as a "dedicator" below. - -A certifier has taken reasonable steps to verify the copyright status of this -work. Certifier recognizes that his good faith efforts may not shield him from -liability if in fact the work certified is not in the public domain. - -Dedicator makes this dedication for the benefit of the public at large and to -the detriment of the Dedicator's heirs and successors. Dedicator intends this -dedication to be an overt act of relinquishment in perpetuity of all present -and future rights under copyright law, whether vested or contingent, in the -Work. Dedicator understands that such relinquishment of all rights includes -the relinquishment of all rights to enforce (by lawsuit or otherwise) those -copyrights in the Work. - -Dedicator recognizes that, once placed in the public domain, the Work may be -freely reproduced, distributed, transmitted, used, modified, built upon, or -otherwise exploited by anyone for any purpose, commercial or non-commercial, -and in any way, including by methods that have not yet been invented or -conceived. diff --git a/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.jzlib.txt b/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.jzlib.txt deleted file mode 100644 index 29ad562af0a..00000000000 --- a/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.jzlib.txt +++ /dev/null @@ -1,26 +0,0 @@ -Copyright (c) 2000,2001,2002,2003,2004 ymnk, JCraft,Inc. All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - - 1. Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - - 2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in - the documentation and/or other materials provided with the distribution. - - 3. The names of the authors may not be used to endorse or promote products - derived from this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, -INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND -FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT, -INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT, -INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, -OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, -EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - diff --git a/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.webbit.txt b/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.webbit.txt deleted file mode 100644 index ec5f348998b..00000000000 --- a/bookkeeper-dist/src/main/resources/deps/netty-3.10.1.Final/LICENSE.webbit.txt +++ /dev/null @@ -1,38 +0,0 @@ -(BSD License: http://www.opensource.org/licenses/bsd-license) - -Copyright (c) 2011, Joe Walnes, Aslak Hellesøy and contributors -All rights reserved. - -Redistribution and use in source and binary forms, with or -without modification, are permitted provided that the -following conditions are met: - -* Redistributions of source code must retain the above - copyright notice, this list of conditions and the - following disclaimer. - -* Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the - following disclaimer in the documentation and/or other - materials provided with the distribution. - -* Neither the name of the Webbit nor the names of - its contributors may be used to endorse or promote products - derived from this software without specific prior written - permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND -CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, -INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF -MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR -CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, -INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE -GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR -BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT -OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE -POSSIBILITY OF SUCH DAMAGE. - diff --git a/bookkeeper-dist/src/main/resources/deps/paranamer-2.8/LICENSE.txt b/bookkeeper-dist/src/main/resources/deps/paranamer-2.8/LICENSE.txt deleted file mode 100644 index fca18473ba0..00000000000 --- a/bookkeeper-dist/src/main/resources/deps/paranamer-2.8/LICENSE.txt +++ /dev/null @@ -1,28 +0,0 @@ -[ ParaNamer used to be 'Pubic Domain', but since it includes a small piece of ASM it is now the same license as that: BSD ] - - Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions - are met: - 1. Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - 2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - 3. Neither the name of the copyright holders nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF - THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/bookkeeper-dist/src/main/resources/deps/scala-library-2.11.7/LICENSE.md b/bookkeeper-dist/src/main/resources/deps/scala-library-2.11.7/LICENSE.md deleted file mode 100644 index 2a6034cc548..00000000000 --- a/bookkeeper-dist/src/main/resources/deps/scala-library-2.11.7/LICENSE.md +++ /dev/null @@ -1,34 +0,0 @@ -Scala is licensed under the [BSD 3-Clause License](http://opensource.org/licenses/BSD-3-Clause). - -## Scala License - -Copyright (c) 2002-2013 EPFL - -Copyright (c) 2011-2013 Typesafe, Inc. - -All rights reserved. - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - - * Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - * Neither the name of the EPFL nor the names of its contributors - may be used to endorse or promote products derived from this software - without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR -CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, -EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, -PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - diff --git a/bookkeeper-dist/src/main/resources/deps/scala-parser-combinators_2.11-1.0.4/LICENSE.md b/bookkeeper-dist/src/main/resources/deps/scala-parser-combinators_2.11-1.0.4/LICENSE.md deleted file mode 100644 index c9f2ca8a789..00000000000 --- a/bookkeeper-dist/src/main/resources/deps/scala-parser-combinators_2.11-1.0.4/LICENSE.md +++ /dev/null @@ -1,28 +0,0 @@ -Copyright (c) 2002-2013 EPFL -Copyright (c) 2011-2013 Typesafe, Inc. - -All rights reserved. - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - - * Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - * Neither the name of the EPFL nor the names of its contributors - may be used to endorse or promote products derived from this software - without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR -CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, -EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, -PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/bookkeeper-dist/src/main/resources/deps/scala-reflect-2.11.8/LICENSE.md b/bookkeeper-dist/src/main/resources/deps/scala-reflect-2.11.8/LICENSE.md deleted file mode 100644 index 68afe18b5d8..00000000000 --- a/bookkeeper-dist/src/main/resources/deps/scala-reflect-2.11.8/LICENSE.md +++ /dev/null @@ -1,33 +0,0 @@ -Scala is licensed under the [BSD 3-Clause License](http://opensource.org/licenses/BSD-3-Clause). - -## Scala License - -Copyright (c) 2002-2016 EPFL - -Copyright (c) 2011-2016 Lightbend, Inc. - -All rights reserved. - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - - * Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - * Neither the name of the EPFL nor the names of its contributors - may be used to endorse or promote products derived from this software - without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR -CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, -EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, -PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/bookkeeper-http/pom.xml b/bookkeeper-http/pom.xml index 8fee889fee6..9b32d8c7208 100644 --- a/bookkeeper-http/pom.xml +++ b/bookkeeper-http/pom.xml @@ -28,7 +28,6 @@ Apache BookKeeper :: Http http-server - twitter-http-server vertx-http-server diff --git a/bookkeeper-http/twitter-http-server/pom.xml b/bookkeeper-http/twitter-http-server/pom.xml deleted file mode 100644 index 37d73cd6c7f..00000000000 --- a/bookkeeper-http/twitter-http-server/pom.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - - - bookkeeper - org.apache.bookkeeper - 4.11.0-SNAPSHOT - ../.. - - 4.0.0 - org.apache.bookkeeper.http - twitter-http-server - Apache BookKeeper :: Bookkeeper Http :: Twitter Http Server - http://maven.apache.org - - - com.twitter - twitter-server_2.11 - - - org.apache.bookkeeper.http - http-server - ${project.version} - - - diff --git a/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/TwitterAbstractHandler.java b/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/TwitterAbstractHandler.java deleted file mode 100644 index 87e48dfca18..00000000000 --- a/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/TwitterAbstractHandler.java +++ /dev/null @@ -1,90 +0,0 @@ -/** - * - * 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.http.twitter; - -import com.twitter.finagle.Service; -import com.twitter.finagle.http.Request; -import com.twitter.finagle.http.Response; -import com.twitter.util.Future; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.bookkeeper.http.HttpServer; -import org.apache.bookkeeper.http.service.ErrorHttpService; -import org.apache.bookkeeper.http.service.HttpEndpointService; -import org.apache.bookkeeper.http.service.HttpServiceRequest; -import org.apache.bookkeeper.http.service.HttpServiceResponse; - -/** - * Http handler for TwitterServer. - */ -public abstract class TwitterAbstractHandler extends Service { - - /** - * Process the request using the given httpEndpointService. - */ - Future processRequest(HttpEndpointService httpEndpointService, Request request) { - HttpServiceRequest httpServiceRequest = new HttpServiceRequest() - .setMethod(convertMethod(request)) - .setParams(convertParams(request)) - .setBody(request.contentString()); - HttpServiceResponse httpServiceResponse = null; - try { - httpServiceResponse = httpEndpointService.handle(httpServiceRequest); - } catch (Exception e) { - httpServiceResponse = new ErrorHttpService().handle(httpServiceRequest); - } - Response response = Response.apply(); - response.setContentString(httpServiceResponse.getBody()); - response.statusCode(httpServiceResponse.getStatusCode()); - return Future.value(response); - } - - /** - * Convert http request parameters to Map. - */ - @SuppressWarnings("unchecked") - Map convertParams(Request request) { - Map map = new HashMap<>(); - for (Map.Entry entry : request.getParams()) { - map.put(entry.getKey(), entry.getValue()); - } - return map; - } - - /** - * Convert http request method to the method that - * can be recognized by HttpServer. - */ - HttpServer.Method convertMethod(Request request) { - switch (request.method().name()) { - case "POST": - return HttpServer.Method.POST; - case "DELETE": - return HttpServer.Method.DELETE; - case "PUT": - return HttpServer.Method.PUT; - default: - return HttpServer.Method.GET; - } - } -} diff --git a/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/TwitterHttpHandlerFactory.java b/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/TwitterHttpHandlerFactory.java deleted file mode 100644 index 569235b0cf3..00000000000 --- a/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/TwitterHttpHandlerFactory.java +++ /dev/null @@ -1,52 +0,0 @@ -/** - * - * 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.http.twitter; - -import com.twitter.finagle.http.Request; -import com.twitter.finagle.http.Response; -import com.twitter.util.Future; -import org.apache.bookkeeper.http.AbstractHttpHandlerFactory; -import org.apache.bookkeeper.http.HttpServer; -import org.apache.bookkeeper.http.HttpServiceProvider; -import org.apache.bookkeeper.http.service.HttpEndpointService; - - -/** - * Factory which provide http handlers for TwitterServer based Http Server. - */ -public class TwitterHttpHandlerFactory extends AbstractHttpHandlerFactory { - - public TwitterHttpHandlerFactory(HttpServiceProvider httpServiceProvider) { - super(httpServiceProvider); - } - - - @Override - public TwitterAbstractHandler newHandler(HttpServer.ApiType type) { - return new TwitterAbstractHandler() { - @Override - public Future apply(Request request) { - HttpEndpointService service = getHttpServiceProvider().provideHttpEndpointService(type); - return processRequest(service, request); - } - }; - } -} diff --git a/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/TwitterHttpServer.java b/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/TwitterHttpServer.java deleted file mode 100644 index ad42f469ba0..00000000000 --- a/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/TwitterHttpServer.java +++ /dev/null @@ -1,108 +0,0 @@ -/** - * - * 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.http.twitter; - -import com.twitter.finagle.Http; -import com.twitter.finagle.ListeningServer; -import com.twitter.finagle.http.HttpMuxer; -import com.twitter.finagle.http.HttpMuxer$; -import com.twitter.server.AbstractTwitterServer; - -import java.io.IOException; -import java.net.InetSocketAddress; - -import org.apache.bookkeeper.http.HttpRouter; -import org.apache.bookkeeper.http.HttpServer; -import org.apache.bookkeeper.http.HttpServiceProvider; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * TwitterServer implementation of Http Server. - */ -public class TwitterHttpServer extends AbstractTwitterServer implements HttpServer { - - static final Logger LOG = LoggerFactory.getLogger(TwitterHttpServer.class); - - private ListeningServer server; - private boolean isRunning; - private int port; - private HttpServiceProvider httpServiceProvider; - - @Override - public void initialize(HttpServiceProvider httpServiceProvider) { - this.httpServiceProvider = httpServiceProvider; - } - - @Override - public boolean startServer(int port) { - try { - this.port = port; - this.main(); - isRunning = true; - LOG.info("Twitter HTTP server started successfully"); - return true; - } catch (Throwable throwable) { - LOG.error("Failed to start Twitter Http Server", throwable); - } - return false; - } - - @Override - public void stopServer() { - try { - httpServiceProvider.close(); - } catch (IOException ioe) { - LOG.error("Error while close httpServiceProvider", ioe); - } - if (server != null) { - server.close(); - isRunning = false; - } - } - - @Override - public boolean isRunning() { - return isRunning; - } - - @Override - public void main() throws Throwable { - LOG.info("Starting Twitter HTTP server on port {}", port); - TwitterHttpHandlerFactory handlerFactory = new TwitterHttpHandlerFactory(httpServiceProvider); - HttpRouter requestRouter = new HttpRouter(handlerFactory) { - @Override - public void bindHandler(String endpoint, TwitterAbstractHandler handler) { - HttpMuxer.addHandler(endpoint, handler); - } - }; - requestRouter.bindAll(); - InetSocketAddress addr = new InetSocketAddress(port); - server = Http.server().serve(addr, HttpMuxer$.MODULE$); - } - - @Override - public void onExit() { - stopServer(); - } - -} diff --git a/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/package-info.java b/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/package-info.java deleted file mode 100644 index d52fd62118d..00000000000 --- a/bookkeeper-http/twitter-http-server/src/main/java/org/apache/bookkeeper/http/twitter/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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 for TwitterServer based http server. - */ -package org.apache.bookkeeper.http.twitter; diff --git a/bookkeeper-http/twitter-http-server/src/test/java/org/apache/bookkeeper/http/twitter/TestTwitterHttpServer.java b/bookkeeper-http/twitter-http-server/src/test/java/org/apache/bookkeeper/http/twitter/TestTwitterHttpServer.java deleted file mode 100644 index b7e032e7a3b..00000000000 --- a/bookkeeper-http/twitter-http-server/src/test/java/org/apache/bookkeeper/http/twitter/TestTwitterHttpServer.java +++ /dev/null @@ -1,104 +0,0 @@ -/** - * - * 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.http.twitter; - -import static org.junit.Assert.assertEquals; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.net.HttpURLConnection; -import java.net.URL; - -import org.apache.bookkeeper.http.HttpRouter; -import org.apache.bookkeeper.http.HttpServer; -import org.apache.bookkeeper.http.HttpServiceProvider; -import org.apache.bookkeeper.http.NullHttpServiceProvider; -import org.apache.bookkeeper.http.service.HeartbeatService; -import org.junit.Test; - -/** - * Unit teset {@link TwitterHttpServer}. - */ -public class TestTwitterHttpServer { - - private int port = 8080; - - @Test - public void testStartBasicHttpServer() throws Exception { - TwitterHttpServer httpServer = new TwitterHttpServer(); - HttpServiceProvider httpServiceProvider = NullHttpServiceProvider.getInstance(); - httpServer.initialize(httpServiceProvider); - int port = getNextPort(); - while (!httpServer.startServer(port)) { - httpServer.stopServer(); - port = getNextPort(); - } - HttpResponse httpResponse = sendGet(getUrl(port, HttpRouter.HEARTBEAT)); - assertEquals(HttpServer.StatusCode.OK.getValue(), httpResponse.responseCode); - assertEquals(HeartbeatService.HEARTBEAT.trim(), httpResponse.responseBody.trim()); - httpServer.stopServer(); - } - - // HTTP GET request - private HttpResponse sendGet(String url) throws IOException { - URL obj = new URL(url); - HttpURLConnection con = (HttpURLConnection) obj.openConnection(); - // optional, default is GET - con.setRequestMethod("GET"); - int responseCode = con.getResponseCode(); - StringBuilder response = new StringBuilder(); - BufferedReader in = null; - try { - in = new BufferedReader(new InputStreamReader(con.getInputStream())); - String inputLine; - while ((inputLine = in.readLine()) != null) { - response.append(inputLine); - } - } finally { - if (in != null) { - in.close(); - } - } - return new HttpResponse(responseCode, response.toString()); - } - - private String getUrl(int port, String path) { - return "http://localhost:" + port + path; - } - - private class HttpResponse { - private int responseCode; - private String responseBody; - - public HttpResponse(int responseCode, String responseBody) { - this.responseCode = responseCode; - this.responseBody = responseBody; - } - } - - private int getNextPort() throws Exception { - if (port > 65535) { - throw new Exception("No port available"); - } - return port++; - } -} diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index db7ccfaef56..98748333f1b 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -62,9 +62,10 @@ netty-handler - io.netty - netty-transport-native-epoll - + io.netty + netty-transport-native-epoll + linux-x86_64 + io.netty netty-tcnative-boringssl-static @@ -241,16 +242,6 @@ - - twitter-http-server - - - org.apache.bookkeeper.http - twitter-http-server - ${project.parent.version} - - - vertx-http-server diff --git a/bookkeeper-stats-providers/pom.xml b/bookkeeper-stats-providers/pom.xml index 8c7002625db..42e36702876 100644 --- a/bookkeeper-stats-providers/pom.xml +++ b/bookkeeper-stats-providers/pom.xml @@ -30,21 +30,4 @@ prometheus-metrics-provider
            - - - twitter - - - twitter - - - - - twitter-finagle-provider - twitter-science-provider - twitter-ostrich-provider - - - - diff --git a/bookkeeper-stats-providers/twitter-finagle-provider/pom.xml b/bookkeeper-stats-providers/twitter-finagle-provider/pom.xml deleted file mode 100644 index 8d7c9354171..00000000000 --- a/bookkeeper-stats-providers/twitter-finagle-provider/pom.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - - 4.0.0 - - bookkeeper - org.apache.bookkeeper - 4.11.0-SNAPSHOT - ../.. - - org.apache.bookkeeper.stats - twitter-finagle-provider - Apache BookKeeper :: Stats Providers :: Twitter Finagle Stats - http://maven.apache.org - - - org.apache.bookkeeper.stats - bookkeeper-stats-api - ${project.parent.version} - - - com.twitter - finagle-core_2.11 - - - diff --git a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/CounterImpl.java b/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/CounterImpl.java deleted file mode 100644 index 192474abbb9..00000000000 --- a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/CounterImpl.java +++ /dev/null @@ -1,60 +0,0 @@ -/** - * 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.stats.twitter.finagle; - -import com.twitter.finagle.stats.StatsReceiver; -import org.apache.bookkeeper.stats.Counter; - -/** - * Note: this counter doesn't support resetting values or getting the current value. - * It also has a limitation in size: max bound is signed integer, not long. - */ -class CounterImpl implements Counter { - private final com.twitter.finagle.stats.Counter counter; - - public CounterImpl(final String name, - final StatsReceiver stats) { - this.counter = stats.counter0(name); - } - - @Override - public void clear() { /* not supported */ } - - @Override - public void inc() { - this.counter.incr(); - } - - @Override - public void dec() { - this.counter.incr(-1); - } - - @Override - public void add(final long delta) { - if (delta < Integer.MIN_VALUE || delta > Integer.MAX_VALUE) { - throw new IllegalArgumentException("This counter doesn't support long values"); - } - this.counter.incr((int) delta); - } - - @Override - public Long get() { - return null; // not supported - } -} diff --git a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsLoggerImpl.java b/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsLoggerImpl.java deleted file mode 100644 index 0aa7d12e370..00000000000 --- a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsLoggerImpl.java +++ /dev/null @@ -1,88 +0,0 @@ -/** - * 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.stats.twitter.finagle; - -import com.twitter.finagle.stats.StatsReceiver; -import com.twitter.util.Function0; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.Gauge; -import org.apache.bookkeeper.stats.OpStatsLogger; -import org.apache.bookkeeper.stats.StatsLogger; -import scala.collection.Seq; - -/** - * A Finagle Stats library based {@link StatsLogger} implementation. - */ -public class FinagleStatsLoggerImpl implements StatsLogger { - private final StatsReceiver stats; - // keep the references for finagle gauges. they are destroyed when the stats logger is destroyed. - final Map finagleGauges; - - public FinagleStatsLoggerImpl(final StatsReceiver stats) { - this.stats = stats; - this.finagleGauges = new HashMap(); - } - - @Override - public OpStatsLogger getOpStatsLogger(final String name) { - return new OpStatsLoggerImpl(name, this.stats); - } - - @Override - public Counter getCounter(final String name) { - return new CounterImpl(name, this.stats); - } - - @Override - public void registerGauge(final String name, final Gauge gauge) { - // This is done to inter-op with Scala Seq - final Seq gaugeName = scala.collection.JavaConversions.asScalaBuffer(Arrays.asList(name)).toList(); - synchronized (finagleGauges) { - finagleGauges.put(gauge, this.stats.addGauge(gaugeName, gaugeProvider(gauge))); - } - } - - @Override - public void unregisterGauge(String name, Gauge gauge) { - synchronized (finagleGauges) { - finagleGauges.remove(gauge); - } - } - - private Function0 gaugeProvider(final Gauge gauge) { - return new Function0() { - @Override - public Object apply() { - return gauge.getSample().floatValue(); - } - }; - } - - @Override - public StatsLogger scope(String name) { - return new FinagleStatsLoggerImpl(this.stats.scope(name)); - } - - @Override - public void removeScope(String name, StatsLogger statsLogger) { - // no-op - } -} diff --git a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsProvider.java b/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsProvider.java deleted file mode 100644 index a3b569cc5b5..00000000000 --- a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/FinagleStatsProvider.java +++ /dev/null @@ -1,72 +0,0 @@ -/** - * 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.stats.twitter.finagle; - -import com.twitter.finagle.stats.StatsReceiver; -import org.apache.bookkeeper.stats.CachingStatsProvider; -import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.stats.StatsProvider; -import org.apache.commons.configuration.Configuration; - -/** - * Main entry point to use Finagle stats for Bookkeeper. - * - *

            There's no requirement to start or stop it.

            - */ -public class FinagleStatsProvider implements StatsProvider { - private final StatsReceiver stats; - private final CachingStatsProvider cachingStatsProvider; - - public FinagleStatsProvider(final StatsReceiver stats) { - this.stats = stats; - this.cachingStatsProvider = new CachingStatsProvider( - new StatsProvider() { - @Override - public void start(Configuration conf) { - // nop - } - - @Override - public void stop() { - // nop - } - - @Override - public StatsLogger getStatsLogger(String scope) { - return new FinagleStatsLoggerImpl(stats.scope(scope)); - } - } - ); - } - - @Override - public void start(Configuration conf) { /* no-op */ } - - @Override - public void stop() { /* no-op */ } - - @Override - public StatsLogger getStatsLogger(final String scope) { - return this.cachingStatsProvider.getStatsLogger(scope); - } - - @Override - public String getStatsName(String... statsComponents) { - return cachingStatsProvider.getStatsName(statsComponents); - } -} diff --git a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/OpStatsLoggerImpl.java b/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/OpStatsLoggerImpl.java deleted file mode 100644 index 1fd61feb635..00000000000 --- a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/OpStatsLoggerImpl.java +++ /dev/null @@ -1,76 +0,0 @@ -/** - * 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.stats.twitter.finagle; - -import com.twitter.finagle.stats.Stat; -import com.twitter.finagle.stats.StatsReceiver; -import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.stats.OpStatsData; -import org.apache.bookkeeper.stats.OpStatsLogger; - -/** - * A percentile stat that will delegate to Finagle stats' implementation library - * to compute the percentiles. - * - *

            Note: metrics will be exposed in form $name/success.p99 for successful events, - * and $name/failure.p99 for failed ones.

            - */ -public class OpStatsLoggerImpl implements OpStatsLogger { - private static final OpStatsData NULL_OP_STATS = new OpStatsData(0, 0, 0, new long[6]); - private final Stat success; - private final Stat failure; - - public OpStatsLoggerImpl(final String name, final StatsReceiver stats) { - this.success = stats.scope(String.format("%s/success", name)).stat0(name); - this.failure = stats.scope(String.format("%s/failure", name)).stat0(name); - } - - @Override - public void registerFailedEvent(long eventLatency, TimeUnit unit) { - this.success.add(unit.toMillis(eventLatency)); - } - - @Override - public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) { - this.failure.add(unit.toMillis(eventLatency)); - } - - @Override - public void registerSuccessfulValue(final long value) { - this.success.add(value); - } - - @Override - public void registerFailedValue(final long value) { - this.failure.add(value); - } - - /** - * We don't need to support percentiles as a part of this provider, - * since they're part of the Stats implementation library. - * - * @return dummy null-stats object - */ - @Override - public OpStatsData toOpStatsData() { - return NULL_OP_STATS; - } - - @Override - public void clear() { /* not supported */ } -} diff --git a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/package-info.java b/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/package-info.java deleted file mode 100644 index a839cd32062..00000000000 --- a/bookkeeper-stats-providers/twitter-finagle-provider/src/main/java/org/apache/bookkeeper/stats/twitter/finagle/package-info.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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 stats provider implemented based on finagle stats library. - */ -package org.apache.bookkeeper.stats.twitter.finagle; diff --git a/bookkeeper-stats-providers/twitter-ostrich-provider/pom.xml b/bookkeeper-stats-providers/twitter-ostrich-provider/pom.xml deleted file mode 100644 index 6386d83a9a7..00000000000 --- a/bookkeeper-stats-providers/twitter-ostrich-provider/pom.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - - 4.0.0 - - bookkeeper - org.apache.bookkeeper - 4.11.0-SNAPSHOT - ../.. - - org.apache.bookkeeper.stats - twitter-ostrich-provider - Apache BookKeeper :: Stats Providers :: Twitter Ostrich Stats - http://maven.apache.org - - - org.apache.bookkeeper.stats - bookkeeper-stats-api - ${project.parent.version} - - - com.twitter - ostrich_2.9.2 - - - diff --git a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/CounterImpl.java b/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/CounterImpl.java deleted file mode 100644 index bd5ca697a2c..00000000000 --- a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/CounterImpl.java +++ /dev/null @@ -1,53 +0,0 @@ -/** - * 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.stats.twitter.ostrich; - -import org.apache.bookkeeper.stats.Counter; - -class CounterImpl implements Counter { - - private final com.twitter.ostrich.stats.Counter ostrichCounter; - - CounterImpl(com.twitter.ostrich.stats.Counter ostrichCounter) { - this.ostrichCounter = ostrichCounter; - } - - @Override - public void clear() { - this.ostrichCounter.reset(); - } - - @Override - public void inc() { - this.ostrichCounter.incr(); - } - - @Override - public void dec() { - this.ostrichCounter.incr(-1); - } - - @Override - public void add(long delta) { - this.ostrichCounter.incr((int) delta); - } - - @Override - public Long get() { - return this.ostrichCounter.apply(); - } -} diff --git a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OpStatsLoggerImpl.java b/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OpStatsLoggerImpl.java deleted file mode 100644 index b67976f5606..00000000000 --- a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OpStatsLoggerImpl.java +++ /dev/null @@ -1,114 +0,0 @@ -/** - * 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.stats.twitter.ostrich; - -import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.stats.OpStatsData; -import org.apache.bookkeeper.stats.OpStatsLogger; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class OpStatsLoggerImpl implements OpStatsLogger { - - static final Logger LOG = LoggerFactory.getLogger(OpStatsLoggerImpl.class); - - static final double[] PERCENTILES = new double[] { - 0.1, 0.5, 0.9, 0.99, 0.999, 0.9999 - }; - - private final String scope; - private final com.twitter.ostrich.stats.Counter successCounter; - private final com.twitter.ostrich.stats.Counter failureCounter; - private final com.twitter.ostrich.stats.Metric successMetric; - private final com.twitter.ostrich.stats.Metric failureMetric; - - OpStatsLoggerImpl(String scope, com.twitter.ostrich.stats.StatsProvider statsProvider) { - this.scope = scope; - successCounter = statsProvider.getCounter(statName("requests/success")); - failureCounter = statsProvider.getCounter(statName("requests/failure")); - successMetric = statsProvider.getMetric(statName("latency/success")); - failureMetric = statsProvider.getMetric(statName("latency/failure")); - } - - private String statName(String statName) { - return String.format("%s/%s", scope, statName); - } - - @Override - public void registerFailedEvent(long eventLatency, TimeUnit unit) { - if (eventLatency < 0) { - LOG.debug("{} : tried to register negative failure", scope); - } else { - failureMetric.add((int) unit.toMillis(eventLatency)); - failureCounter.incr(); - } - } - - @Override - public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) { - if (eventLatency < 0) { - LOG.debug("{} : tried to register negative success", scope); - } else { - successMetric.add((int) unit.toMillis(eventLatency)); - successCounter.incr(); - } - } - - @Override - public void registerSuccessfulValue(long value) { - if (value < 0) { - LOG.debug("{} : tried to register negative success", scope); - } else { - successMetric.add((int) value); - successCounter.incr(); - } - } - - @Override - public void registerFailedValue(long value) { - if (value < 0) { - LOG.debug("{} : tried to register negative success", scope); - } else { - failureMetric.add((int) value); - failureCounter.incr(); - } - } - - @Override - public OpStatsData toOpStatsData() { - long numSuccess = successCounter.apply(); - long numFailures = failureCounter.apply(); - com.twitter.ostrich.stats.Distribution distribution = successMetric.apply(); - com.twitter.ostrich.stats.Histogram histogram = distribution.histogram(); - double avgLatency = distribution.average(); - long[] percentiles = new long[PERCENTILES.length]; - int i = 0; - for (double percentile : PERCENTILES) { - percentiles[i] = histogram.getPercentile(percentile); - ++i; - } - return new OpStatsData(numSuccess, numFailures, avgLatency, percentiles); - } - - @Override - public void clear() { - successCounter.reset(); - failureCounter.reset(); - successMetric.clear(); - failureMetric.clear(); - } -} diff --git a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichProvider.java b/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichProvider.java deleted file mode 100644 index 173b20022c1..00000000000 --- a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichProvider.java +++ /dev/null @@ -1,126 +0,0 @@ -/** - * 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.stats.twitter.ostrich; - -import com.twitter.ostrich.admin.CustomHttpHandler; -import com.twitter.ostrich.admin.RuntimeEnvironment; -import com.twitter.ostrich.admin.ServiceTracker; -import com.twitter.ostrich.admin.StatsFactory; -import com.twitter.util.Duration; -import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.stats.CachingStatsProvider; -import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.stats.StatsProvider; -import org.apache.commons.configuration.Configuration; -import scala.Some; -import scala.collection.immutable.$colon$colon; -import scala.collection.immutable.List; -import scala.collection.immutable.List$; -import scala.collection.immutable.Map; -import scala.collection.immutable.Map$; -import scala.util.matching.Regex; - -/** - * A Twitter Ostrich library based {@link StatsProvider} implementation. - */ -public class OstrichProvider implements StatsProvider { - - protected static final String STATS_EXPORT = "statsExport"; - protected static final String STATS_HTTP_PORT = "statsHttpPort"; - protected static final String SHOULD_SHUTDOWN_SERVICE_TRACKER = "shouldShutdownServiceTracker"; - - private com.twitter.ostrich.admin.AdminHttpService statsExporter = null; - private final CachingStatsProvider cachingStatsProvider; - private boolean shutdownServiceTracker = false; - - private static List list(Duration ... ts) { - List result = List$.MODULE$.empty(); - for (int i = ts.length; i > 0; i--) { - result = new $colon$colon<>(ts[i - 1], result); - } - return result; - } - - private static List emptyList() { - return List$.MODULE$.empty(); - } - - private static Map emptyMap() { - Map result = Map$.MODULE$.empty(); - return result; - } - - public OstrichProvider() { - this(""); - } - - public OstrichProvider(final String collectionName) { - this.cachingStatsProvider = new CachingStatsProvider(new StatsProvider() { - @Override - public void start(Configuration conf) { - // nop - } - - @Override - public void stop() { - // nop - } - - @Override - public StatsLogger getStatsLogger(String scope) { - return new OstrichStatsLoggerImpl(scope, - com.twitter.ostrich.stats.Stats.get(collectionName)); - } - }); - } - - @Override - public void start(Configuration conf) { - if (conf.getBoolean(STATS_EXPORT, false)) { - statsExporter = new com.twitter.ostrich.admin.AdminServiceFactory( - conf.getInt(STATS_HTTP_PORT, 9002), 20, OstrichProvider.emptyList(), - Some.apply(""), OstrichProvider.emptyList(), - OstrichProvider.emptyMap(), list(Duration.apply(1, TimeUnit.MINUTES)) - ).apply(RuntimeEnvironment.apply(this, new String[0])); - this.shutdownServiceTracker = conf.getBoolean(SHOULD_SHUTDOWN_SERVICE_TRACKER, false); - } - } - - @Override - public void stop() { - if (null != statsExporter) { - statsExporter.shutdown(); - if (shutdownServiceTracker) { - // ostrich admin service registered some threads in service tracker - // shutdown doesn't stopped those threads. we need to stop service tracker - // to shutdown them. but that potentially has side effects. so adding a flag - // to let caller decide. - ServiceTracker.shutdown(); - } - } - } - - @Override - public StatsLogger getStatsLogger(String scope) { - return cachingStatsProvider.getStatsLogger(scope); - } - - @Override - public String getStatsName(String... statsComponents) { - return cachingStatsProvider.getStatsName(statsComponents); - } -} diff --git a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichStatsLoggerImpl.java b/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichStatsLoggerImpl.java deleted file mode 100644 index 52027fce65d..00000000000 --- a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/OstrichStatsLoggerImpl.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * 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.stats.twitter.ostrich; - -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.Gauge; -import org.apache.bookkeeper.stats.OpStatsLogger; -import org.apache.bookkeeper.stats.StatsLogger; -import scala.Function0; -import scala.runtime.AbstractFunction0; - -/** - * Implementation of ostrich logger. - */ -class OstrichStatsLoggerImpl implements StatsLogger { - - protected final String scope; - protected final com.twitter.ostrich.stats.StatsProvider ostrichProvider; - - OstrichStatsLoggerImpl(String scope, com.twitter.ostrich.stats.StatsProvider ostrichProvider) { - this.scope = scope; - this.ostrichProvider = ostrichProvider; - } - - @Override - public OpStatsLogger getOpStatsLogger(String statName) { - return new OpStatsLoggerImpl(getStatName(statName), ostrichProvider); - } - - @Override - public Counter getCounter(String statName) { - return new CounterImpl(ostrichProvider.getCounter(getStatName(statName))); - } - - @Override - public void registerGauge(final String statName, final Gauge gauge) { - Function0 gaugeFunc = new AbstractFunction0() { - @Override - public Object apply() { - return gauge.getSample().doubleValue(); - } - }; - ostrichProvider.addGauge(getStatName(statName), gaugeFunc); - } - - @Override - public void unregisterGauge(String statName, Gauge gauge) { - ostrichProvider.clearGauge(getStatName(statName)); - } - - private String getStatName(String statName) { - return String.format("%s/%s", scope, statName); - } - - @Override - public StatsLogger scope(String scope) { - return new OstrichStatsLoggerImpl(getStatName(scope), ostrichProvider); - } - - @Override - public void removeScope(String name, StatsLogger statsLogger) { - // no-op - } -} diff --git a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/package-info.java b/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/package-info.java deleted file mode 100644 index 2db25846eda..00000000000 --- a/bookkeeper-stats-providers/twitter-ostrich-provider/src/main/java/org/apache/bookkeeper/stats/twitter/ostrich/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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 stats provider implemented based on Twitter Ostrich library. - */ -package org.apache.bookkeeper.stats.twitter.ostrich; diff --git a/bookkeeper-stats-providers/twitter-science-provider/pom.xml b/bookkeeper-stats-providers/twitter-science-provider/pom.xml deleted file mode 100644 index c79ddad26bf..00000000000 --- a/bookkeeper-stats-providers/twitter-science-provider/pom.xml +++ /dev/null @@ -1,84 +0,0 @@ - - - - 4.0.0 - - bookkeeper - org.apache.bookkeeper - 4.11.0-SNAPSHOT - ../.. - - org.apache.bookkeeper.stats - twitter-science-provider - Apache BookKeeper :: Stats Providers :: Twitter Science Stats - - - - org.apache.bookkeeper.stats - bookkeeper-stats-api - ${project.parent.version} - - - com.twitter.common - stats - 0.0.64 - - - com.twitter.common - net-http-handlers - - - javax.servlet - servlet-api - - - 0.0.39 - compile - - - com.twitter.common - stats-time-series - 0.0.36 - compile - - - com.twitter.common - stats-jvm - 0.0.33 - compile - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-servlet - - - - - twitter - Twitter repo - default - https://maven.twttr.com - - - diff --git a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/CounterImpl.java b/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/CounterImpl.java deleted file mode 100644 index 689a7e8a26b..00000000000 --- a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/CounterImpl.java +++ /dev/null @@ -1,69 +0,0 @@ -/** - * 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.stats.twitter.science; - -import com.twitter.common.stats.Rate; -import com.twitter.common.stats.Stats; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.bookkeeper.stats.Counter; - -/** - * This will export the value and the rate (per sec) to {@link org.apache.bookkeeper.stats.Stats}. - */ -public class CounterImpl implements Counter { - // The name used to export this stat - private String name; - private AtomicLong value; - - public CounterImpl(String name) { - this.name = name; - value = new AtomicLong(0); - setUpStatsExport(); - } - - @Override - public synchronized void clear() { - value.getAndSet(0); - } - - @Override - public Long get() { - return value.get(); - } - - private void setUpStatsExport() { - // Export the value. - Stats.export(name, value); - // Export the rate of this value. - Stats.export(Rate.of(name + "_per_sec", value).build()); - } - - @Override - public void inc() { - value.incrementAndGet(); - } - - @Override - public void dec() { - value.decrementAndGet(); - } - - @Override - public void add(long delta) { - value.addAndGet(delta); - } -} diff --git a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/HTTPStatsExporter.java b/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/HTTPStatsExporter.java deleted file mode 100644 index 34e0eea6d5a..00000000000 --- a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/HTTPStatsExporter.java +++ /dev/null @@ -1,80 +0,0 @@ -/** - * 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.stats.twitter.science; - -import com.twitter.common.application.ShutdownRegistry; -import com.twitter.common.base.Supplier; -import com.twitter.common.net.http.handlers.VarsHandler; -import com.twitter.common.net.http.handlers.VarsJsonHandler; -import com.twitter.common.quantity.Amount; -import com.twitter.common.quantity.Time; -import com.twitter.common.stats.JvmStats; -import com.twitter.common.stats.Stat; -import com.twitter.common.stats.Stats; -import com.twitter.common.stats.TimeSeriesRepository; -import com.twitter.common.stats.TimeSeriesRepositoryImpl; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; - -/** - * Starts a jetty server on a configurable port and the samplers to export stats. - */ -public class HTTPStatsExporter { - final Server jettyServer; - final ShutdownRegistry.ShutdownRegistryImpl shutDownRegistry; - final int port; - - public HTTPStatsExporter(int port) { - // Create the ShutdownRegistry needed for our sampler - this.shutDownRegistry = new ShutdownRegistry.ShutdownRegistryImpl(); - this.port = port; - this.jettyServer = new Server(port); - } - - public void start() throws Exception { - // Start the sampler. Sample every 1 second and retain for 1 hour - TimeSeriesRepository sampler = new TimeSeriesRepositoryImpl(Stats.STAT_REGISTRY, - Amount.of(1L, Time.SECONDS), Amount.of(1L, Time.HOURS)); - sampler.start(this.shutDownRegistry); - // Export JVM stats - JvmStats.export(); - // Configure handlers - Supplier>> supplier = new Supplier>>() { - @Override - public Iterable> get() { - return Stats.getVariables(); - } - }; - - // Start jetty. - ServletContextHandler context = new ServletContextHandler(); - context.setContextPath("/"); - this.jettyServer.setHandler(context); - context.addServlet(new ServletHolder(new VarsHandler(supplier)), "/vars"); - context.addServlet(new ServletHolder(new VarsJsonHandler(supplier)), "/vars.json"); - this.jettyServer.start(); - } - - public void stop() throws Exception { - this.jettyServer.stop(); - if (this.shutDownRegistry != null) { - this.shutDownRegistry.execute(); - } - } -} diff --git a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/OpStatsLoggerImpl.java b/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/OpStatsLoggerImpl.java deleted file mode 100644 index dae7f6fc96c..00000000000 --- a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/OpStatsLoggerImpl.java +++ /dev/null @@ -1,79 +0,0 @@ -/** - * 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.stats.twitter.science; - -import com.twitter.common.stats.RequestStats; -import com.twitter.common.stats.Stat; -import java.util.Arrays; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.stats.OpStatsData; -import org.apache.bookkeeper.stats.OpStatsLogger; - -/** - * An implementation of the OpStatsLogger interface that handles per operation type stats. - * Internals use twitter.common.stats for exporting metrics. - */ -public class OpStatsLoggerImpl implements OpStatsLogger { - private final RequestStats events; - - public OpStatsLoggerImpl(String name) { - this.events = new RequestStats(name); - } - - // OpStatsLogger functions - public void registerFailedEvent(long eventLatency, TimeUnit unit) { - this.events.incErrors(unit.toMicros(eventLatency)); - } - - public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) { - this.events.requestComplete(unit.toMicros(eventLatency)); - } - - public void registerSuccessfulValue(long value) { - this.events.requestComplete(TimeUnit.MILLISECONDS.toMicros(value)); - } - - public void registerFailedValue(long value) { - this.events.incErrors(TimeUnit.MILLISECONDS.toMicros(value)); - } - - public synchronized void clear() { - } - - /** - * This function should go away soon (hopefully). - */ - public synchronized OpStatsData toOpStatsData() { - long numFailed = this.events.getErrorCount(); - long numSuccess = this.events.getSlidingStats().getEventCounter().get() - numFailed; - double avgLatencyMillis = this.events.getSlidingStats().getPerEventLatency().read() / 1000.0; - double[] defaultPercentiles = {10, 50, 90, 99, 99.9, 99.99}; - long[] latenciesMillis = new long[defaultPercentiles.length]; - Arrays.fill(latenciesMillis, Long.MAX_VALUE); - Map realPercentileLatencies = - this.events.getPercentile().getPercentiles(); - for (int i = 0; i < defaultPercentiles.length; i++) { - if (realPercentileLatencies.containsKey(defaultPercentiles[i])) { - @SuppressWarnings("unchecked") - Stat latency = realPercentileLatencies.get(defaultPercentiles[i]); - latenciesMillis[i] = TimeUnit.MICROSECONDS.toMillis(latency.read().longValue()); - } - } - return new OpStatsData(numSuccess, numFailed, avgLatencyMillis, latenciesMillis); - } -} diff --git a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsLoggerImpl.java b/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsLoggerImpl.java deleted file mode 100644 index cf63ed9c87d..00000000000 --- a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsLoggerImpl.java +++ /dev/null @@ -1,81 +0,0 @@ -/** - * 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.stats.twitter.science; - -import com.twitter.common.stats.SampledStat; -import com.twitter.common.stats.Stats; -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.Gauge; -import org.apache.bookkeeper.stats.OpStatsLogger; -import org.apache.bookkeeper.stats.StatsLogger; - -/** - * Implementation of twitter-stats logger. - */ -public class TwitterStatsLoggerImpl implements StatsLogger { - - protected final String name; - - public TwitterStatsLoggerImpl(String name) { - this.name = name; - } - - @Override - public OpStatsLogger getOpStatsLogger(String statName) { - return new OpStatsLoggerImpl(getStatName(statName)); - } - - @Override - public Counter getCounter(String statName) { - return new CounterImpl(getStatName(statName)); - } - - @Override - public void registerGauge(final String statName, final Gauge gauge) { - Stats.export(new SampledStat(getStatName(statName), gauge.getDefaultValue()) { - @Override - public T doSample() { - return gauge.getSample(); - } - }); - } - - @Override - public void unregisterGauge(String name, Gauge gauge) { - // no-op - } - - private String getStatName(String statName) { - return (name + "_" + statName).toLowerCase(); - } - - @Override - public StatsLogger scope(String scope) { - String scopeName; - if (0 == name.length()) { - scopeName = scope; - } else { - scopeName = name + "_" + scope; - } - return new TwitterStatsLoggerImpl(scopeName); - } - - @Override - public void removeScope(String name, StatsLogger statsLogger) { - // no-op - } -} diff --git a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsProvider.java b/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsProvider.java deleted file mode 100644 index 2a410c03b03..00000000000 --- a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/TwitterStatsProvider.java +++ /dev/null @@ -1,99 +0,0 @@ -/** - * 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.stats.twitter.science; - -import org.apache.bookkeeper.stats.CachingStatsProvider; -import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.stats.StatsProvider; -import org.apache.commons.configuration.Configuration; -import org.apache.commons.lang.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A stats provider implemented based on Twitter Stats library. - */ -public class TwitterStatsProvider implements StatsProvider { - - static final Logger LOG = LoggerFactory.getLogger(TwitterStatsProvider.class); - - protected static final String STATS_EXPORT = "statsExport"; - protected static final String STATS_HTTP_PORT = "statsHttpPort"; - - private HTTPStatsExporter statsExporter = null; - private final CachingStatsProvider cachingStatsProvider; - - public TwitterStatsProvider() { - this.cachingStatsProvider = new CachingStatsProvider(new StatsProvider() { - - @Override - public void start(Configuration conf) { - // nop - } - - @Override - public void stop() { - // nop - } - - @Override - public StatsLogger getStatsLogger(String scope) { - return new TwitterStatsLoggerImpl(scope); - } - - @Override - public String getStatsName(String... statsComponents) { - return StringUtils.join(statsComponents, '_').toLowerCase(); - } - }); - } - - @Override - public void start(Configuration conf) { - if (conf.getBoolean(STATS_EXPORT, false)) { - statsExporter = new HTTPStatsExporter(conf.getInt(STATS_HTTP_PORT, 9002)); - } - if (null != statsExporter) { - try { - statsExporter.start(); - } catch (Exception e) { - LOG.error("Fail to start stats exporter : ", e); - } - } - } - - @Override - public void stop() { - if (null != statsExporter) { - try { - statsExporter.stop(); - } catch (Exception e) { - LOG.error("Fail to stop stats exporter : ", e); - } - } - } - - @Override - public StatsLogger getStatsLogger(String name) { - return this.cachingStatsProvider.getStatsLogger(name); - } - - @Override - public String getStatsName(String... statsComponents) { - return this.cachingStatsProvider.getStatsName(statsComponents); - } -} diff --git a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/package-info.java b/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/package-info.java deleted file mode 100644 index bc358f4861a..00000000000 --- a/bookkeeper-stats-providers/twitter-science-provider/src/main/java/org/apache/bookkeeper/stats/twitter/science/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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 stats provider implementation based on Twitter Science Stats library. - */ -package org.apache.bookkeeper.stats.twitter.science; diff --git a/conf/bk_server.conf b/conf/bk_server.conf index 4da030d712f..ddd74e5ee40 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -760,9 +760,6 @@ zkEnableSecurity=false # Options: # - Prometheus : org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider # - Codahale : org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider -# - Twitter Finagle : org.apache.bookkeeper.stats.twitter.finagle.FinagleStatsProvider -# - Twitter Ostrich : org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider -# - Twitter Science : org.apache.bookkeeper.stats.twitter.science.TwitterStatsProvider # Default value is: # org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider # @@ -808,40 +805,6 @@ zkEnableSecurity=false # the jmx endpoint for reporting stats. see {@link https://metrics.dropwizard.io/3.1.0/manual/core/#jmx} for more details. # codahaleStatsJmxEndpoint= -############################################################################# -## Twitter Finagle Metrics Provider -############################################################################# - -# These configs are used when using `FinagleStatsProvider`. -# statsProviderClass=org.apache.bookkeeper.stats.twitter.finagle.FinagleStatsProvider - -############################################################################# -## Twitter Ostrich Metrics Provider -############################################################################# - -# These configs are used when using `OstrichProvider`. -# statsProviderClass=org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider - -# Flag to control whether to expose ostrich metrics via a http endpoint configured by `statsHttpPort`. -# statsExport=false - -# The http port of exposing ostrich stats if `statsExport` is set to true -# statsHttpPort=9002 - -############################################################################# -## Twitter Science Metrics Provider -############################################################################# - -# These configs are used when using `TwitterStatsProvider`. -# statsProviderClass=org.apache.bookkeeper.stats.twitter.science.TwitterStatsProvider - -# Flag to control whether to expose metrics throught a http endpoint configured by `statsHttpPort`. -# statsExport=false - -# The http port of exposing stats if `statsExport` is set to true -# statsHttpPort=9002 - - ############################################## Auto Recovery ############################################## # Whether the bookie itself can start auto-recovery service also or not diff --git a/dev/release/001-release-branch.sh b/dev/release/001-release-branch.sh index dec8e486c73..e137f00e843 100755 --- a/dev/release/001-release-branch.sh +++ b/dev/release/001-release-branch.sh @@ -25,6 +25,5 @@ cd $BK_HOME mvn release:branch \ -DbranchName=${BRANCH_NAME} \ -DdevelopmentVersion=${DEVELOPMENT_VERSION} - -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true -Dstream -Dtwitter" \ - -Dstream \ - -Dtwitter + -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true -Dstream" \ + -Dstream diff --git a/dev/release/002-release-prepare.sh b/dev/release/002-release-prepare.sh index b2878e4d8a0..232fb502aa1 100755 --- a/dev/release/002-release-prepare.sh +++ b/dev/release/002-release-prepare.sh @@ -26,7 +26,6 @@ mvn release:prepare \ -DreleaseVersion=${VERSION} \ -Dtag=${RC_TAG} \ -DupdateWorkingCopyVersions=false \ - -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true -Dstream -Dtwitter" \ + -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true -Dstream" \ -Dstream \ - -Dtwitter \ -Dresume=true diff --git a/dev/release/003-release-perform.sh b/dev/release/003-release-perform.sh index 356ca78b0ad..2a5b531de13 100755 --- a/dev/release/003-release-perform.sh +++ b/dev/release/003-release-perform.sh @@ -23,7 +23,6 @@ BK_HOME=`cd $BINDIR/../..;pwd` cd $BK_HOME mvn release:perform \ - -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true -Dstream -Dtwitter" \ + -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true -Dstream" \ -Dstream \ - -Dtwitter \ -Dresume=true diff --git a/pom.xml b/pom.xml index d0c0a033ac0..c4df5dbde70 100644 --- a/pom.xml +++ b/pom.xml @@ -126,7 +126,6 @@ 4.0.1 3.1.0 0.0.2 - 6.44.0 1.14.9 3.0.2 2.1.2 @@ -167,7 +166,6 @@ 3.1.8 1.3.2 1.8.3 - 1.29.0 3.4.1 3.4.13 2.1.2 @@ -189,7 +187,7 @@ 2.5.1 2.4 3.1.1 - 3.1.0 + 3.2.0 2.2.1 2.21.0 3.5.2 @@ -451,13 +449,6 @@ ${netty-boringssl.version} - - - com.twitter - finagle-core_2.11 - ${finagle.version} - - io.grpc @@ -539,17 +530,6 @@ - - com.twitter - twitter-server_2.11 - ${twitter-server.version} - - - javax.servlet - servlet-api - - - io.vertx vertx-core @@ -632,13 +612,6 @@ ${datasketches.version} - - - com.twitter - ostrich_2.9.2 - ${ostrich.version} - - com.beust @@ -899,7 +872,7 @@ Bookkeeper Stats Providers - org.apache.bookkeeper.stats.codahale:org.apache.bookkeeper.stats.twitter.finagle:org.apache.bookkeeper.stats.twitter.ostrich:org.apache.bookkeeper.stats.twitter.science:org.apache.bookkeeper.stats.prometheus + org.apache.bookkeeper.stats.codahale:org.apache.bookkeeper.stats.prometheus BookKeeper Java API (version ${project.version}) diff --git a/site/docs/latest/admin/metrics.md b/site/docs/latest/admin/metrics.md index 142df3dcd2d..dca236489d2 100644 --- a/site/docs/latest/admin/metrics.md +++ b/site/docs/latest/admin/metrics.md @@ -8,15 +8,12 @@ BookKeeper enables metrics collection through a variety of [stats providers](#st ## Stats providers -BookKeeper has stats provider implementations for four five sinks: +BookKeeper has stats provider implementations for these 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.prometheus.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. diff --git a/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/StatsDocGenerator.java b/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/StatsDocGenerator.java index db2e13f7f0f..f3774f7179f 100644 --- a/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/StatsDocGenerator.java +++ b/stats/utils/src/main/java/org/apache/bookkeeper/stats/utils/StatsDocGenerator.java @@ -282,12 +282,8 @@ public static void main(String[] args) throws Exception { private static String getStatsProviderClass(String providerClass) { switch (providerClass.toLowerCase()) { - case "ostrich": - return "org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider"; case "prometheus": return "org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider"; - case "finagle": - return "org.apache.bookkeeper.stats.twitter.finagle.FinagleStatsProvider"; case "codahale": return "org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider"; default: From 84b6df538674bb16016a35c3185b41634a4cb591 Mon Sep 17 00:00:00 2001 From: Lei Zhang Date: Tue, 7 Jan 2020 16:15:28 +0800 Subject: [PATCH 0367/1642] ISSUE #2224: Config ServerConfiguration with setAllowLoopback(true) for Unit Tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### Motivation Unit test fails when build master branch ### Changes Config ServerConfiguration with setAllowLoopback(true) in the class GarbageCollectorThreadTest Master Issue: #2224 Reviewers: Jia Zhai , Enrico Olivelli This closes #2226 from coolbeevip/ISSUE-2224, closes #2224 and squashes the following commits: fdba4fbce [Lei Zhang] ISSUE-2224 set allowLoopback is true 2ff7ca5af [Enrico Olivelli] Account Apache Thrift and Http Client files in LICENSE files - Fix build on Travis 2bbf58e01 [Dapeng] Fixed bin/bookkeeper shell readjournal 9d61fb4fe [Dapeng] [DOCS] Modify “Using Autoecovery”,recover does not support the specified target bookie 2ca4025e4 [Anup Ghatage] [DOCUMENTATION] Add doc for decommissioning bookie process 62e3b9b6f [Enrico Olivelli] Make 4.9.2 as stable 40f349ccf [Enrico Olivelli] Bump Python client version b1991e6b3 [Enrico Olivelli] Release notes for 4.10 9f4b1359e [冉小龙] Fix the log level of not support Sse42Crc32C 557dfe90f [Jennifer Huang] [DOCS] Update coding_guide.md 43f8cd74f [Enrico Olivelli] Add -Dtwitter to fix auto-build f57962bcc [Matteo Merli] Removed mutex contention on BufferedChannel f5d2e5931 [Enrico Olivelli] Add -Dtwitter to release scripts in order to build Twitter stats providers 7efa8e89e [Enrico Olivelli] Fix version in modules that are not active by default d49b46f4e [Enrico Olivelli] [maven-release-plugin] prepare for next development iteration 5a4213bb0 [Enrico Olivelli] [maven-release-plugin] prepare branch branch-4.10 f8a5974a5 [Charan Reddy Guttapalem] Fix firstStoredEntryId logic in LedgerFragment 8559126a3 [Anup Ghatage] Issue #2145: Make ledgerDirs in Cookie consistent across generating a… --- .../apache/bookkeeper/bookie/GarbageCollectorThreadTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java index 1b1f6572567..be6c81bd8ed 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java @@ -55,7 +55,7 @@ public class GarbageCollectorThreadTest { @Mock private ScheduledExecutorService gcExecutor; - private ServerConfiguration conf = spy(new ServerConfiguration()); + private ServerConfiguration conf = spy(new ServerConfiguration().setAllowLoopback(true)); private CompactableLedgerStorage ledgerStorage = mock(CompactableLedgerStorage.class); @Before From fe7ebc094d69a96e4045abbb06c8613b4eca9cf7 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 16 Jan 2020 06:26:46 -0800 Subject: [PATCH 0368/1642] [bookie-server] add support of client auto cert refresh ### Motivation as described at: https://github.com/apache/pulsar/issues/6010 Bookkeeper-client caches the tls certificates when it first tries to create a cnx with a given bookie and after that it never reloads certs even when valid certs changes on the file-system or new bookie-connection is created. Because of that as soon as client certs expires and bk-client disconnects from bookie then bk-client is not able to reconnect to bookie until we restart the bk-client process. and we see below TLS exception at bk-client. ``` 19:43:03.983 [bookkeeper-io-12-45] ERROR org.apache.bookkeeper.proto.PerChannelBookieClient - Unexpected exception caught by bookie client channel handler io.netty.handler.codec.DecoderException: javax.net.ssl.SSLHandshakeException: General OpenSslEngine problem at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:472) ~[netty-codec-4.1.31.Final.jar:4.1.31.Final] at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:278) ~[netty-codec-4.1.31.Final.jar:4.1.31.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1434) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:965) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:799) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:433) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:330) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:909) [netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [netty-all-4.1.32.Final.jar:4.1.32.Final] at java.lang.Thread.run(Thread.java:834) [?:?] Caused by: javax.net.ssl.SSLHandshakeException: General OpenSslEngine problem at io.netty.handler.ssl.ReferenceCountedOpenSslClientContext$OpenSslClientCertificateCallback.handle(ReferenceCountedOpenSslClientContext.java:273) ~[netty-all-4.1.32.Final.jar :4.1.32.Final] at io.netty.internal.tcnative.SSL.readFromSSL(Native Method) ~[netty-tcnative-boringssl-static-2.0.20.Final.jar:2.0.20.Final] at io.netty.handler.ssl.ReferenceCountedOpenSslEngine.readPlaintextData(ReferenceCountedOpenSslEngine.java:575) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.handler.ssl.ReferenceCountedOpenSslEngine.unwrap(ReferenceCountedOpenSslEngine.java:1124) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.handler.ssl.ReferenceCountedOpenSslEngine.unwrap(ReferenceCountedOpenSslEngine.java:1236) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.handler.ssl.ReferenceCountedOpenSslEngine.unwrap(ReferenceCountedOpenSslEngine.java:1279) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.handler.ssl.SslHandler$SslEngineType$1.unwrap(SslHandler.java:217) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.handler.ssl.SslHandler.unwrap(SslHandler.java:1301) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.handler.ssl.SslHandler.decodeNonJdkCompatible(SslHandler.java:1215) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.handler.ssl.SslHandler.decode(SslHandler.java:1249) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:502) ~[netty-codec-4.1.31.Final.jar:4.1.31.Final] at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:441) ~[netty-codec-4.1.31.Final.jar:4.1.31.Final] ... 14 more ``` ### Modification Add support at bk-client to reload certs once they have changed on file-system. Reviewers: Enrico Olivelli , Jia Zhai , Sijie Guo This closes #2235 from rdhabalia/bk_client_cert --- .../bookkeeper/tls/TLSContextFactory.java | 51 +++++++++++---- .../org/apache/bookkeeper/tls/TestTLS.java | 63 +++++++++++++++++++ 2 files changed, 101 insertions(+), 13 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java index 77e8dc9dbc5..b0bc57f3c60 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java @@ -83,8 +83,8 @@ public String toString() { private volatile SslContext sslContext; private ByteBufAllocator allocator; private AbstractConfiguration config; - private FileModifiedTimeUpdater tTLSCertificatePath, tLSKeyStoreFilePath, tLSKeyStorePasswordFilePath, - tLSTrustStoreFilePath, tLSTrustStorePasswordFilePath; + private FileModifiedTimeUpdater tlsCertificateFilePath, tlsKeyStoreFilePath, tlsKeyStorePasswordFilePath, + tlsTrustStoreFilePath, tlsTrustStorePasswordFilePath; private long certRefreshTime; private volatile long certLastRefreshTime; private boolean isServerCtx; @@ -183,6 +183,16 @@ private SslProvider getTLSProvider(String sslProvider) { private void createClientContext() throws SecurityException, KeyStoreException, NoSuchAlgorithmException, CertificateException, IOException, UnrecoverableKeyException, InvalidKeySpecException, NoSuchProviderException { + ClientConfiguration clientConf = (ClientConfiguration) config; + markAutoCertRefresh(clientConf.getTLSCertificatePath(), clientConf.getTLSKeyStore(), + clientConf.getTLSKeyStorePasswordPath(), clientConf.getTLSTrustStore(), + clientConf.getTLSTrustStorePasswordPath()); + updateClientContext(); + } + + private synchronized void updateClientContext() + throws SecurityException, KeyStoreException, NoSuchAlgorithmException, CertificateException, IOException, + UnrecoverableKeyException, InvalidKeySpecException, NoSuchProviderException { final SslContextBuilder sslContextBuilder; final ClientConfiguration clientConf; final SslProvider provider; @@ -267,29 +277,35 @@ private void createClientContext() } sslContext = sslContextBuilder.build(); + certLastRefreshTime = System.currentTimeMillis(); } private void createServerContext() throws SecurityException, KeyStoreException, NoSuchAlgorithmException, CertificateException, IOException, UnrecoverableKeyException, InvalidKeySpecException, IllegalArgumentException { isServerCtx = true; - ServerConfiguration serverConf = (ServerConfiguration) config; - tTLSCertificatePath = new FileModifiedTimeUpdater(serverConf.getTLSCertificatePath()); - tLSKeyStoreFilePath = new FileModifiedTimeUpdater(serverConf.getTLSKeyStore()); - tLSKeyStorePasswordFilePath = new FileModifiedTimeUpdater(serverConf.getTLSKeyStorePasswordPath()); - tLSTrustStoreFilePath = new FileModifiedTimeUpdater(serverConf.getTLSTrustStore()); - tLSTrustStorePasswordFilePath = new FileModifiedTimeUpdater(serverConf.getTLSTrustStorePasswordPath()); + ServerConfiguration clientConf = (ServerConfiguration) config; + markAutoCertRefresh(clientConf.getTLSCertificatePath(), clientConf.getTLSKeyStore(), + clientConf.getTLSKeyStorePasswordPath(), clientConf.getTLSTrustStore(), + clientConf.getTLSTrustStorePasswordPath()); updateServerContext(); } private synchronized SslContext getSSLContext() { long now = System.currentTimeMillis(); - if (isServerCtx && (certRefreshTime > 0 && now > (certLastRefreshTime + certRefreshTime))) { - if (tTLSCertificatePath.checkAndRefresh() || tLSKeyStoreFilePath.checkAndRefresh() - || tLSKeyStorePasswordFilePath.checkAndRefresh() || tLSTrustStoreFilePath.checkAndRefresh() - || tLSTrustStorePasswordFilePath.checkAndRefresh()) { + if ((certRefreshTime > 0 && now > (certLastRefreshTime + certRefreshTime))) { + if (tlsCertificateFilePath.checkAndRefresh() || tlsKeyStoreFilePath.checkAndRefresh() + || tlsKeyStorePasswordFilePath.checkAndRefresh() || tlsTrustStoreFilePath.checkAndRefresh() + || tlsTrustStorePasswordFilePath.checkAndRefresh()) { try { - updateServerContext(); + LOG.info("Updating tls certs certFile={}, keyStoreFile={}, trustStoreFile={}", + tlsCertificateFilePath.getFileName(), tlsKeyStoreFilePath.getFileName(), + tlsTrustStoreFilePath.getFileName()); + if (isServerCtx) { + updateServerContext(); + } else { + updateClientContext(); + } } catch (Exception e) { LOG.info("Failed to refresh tls certs", e); } @@ -458,4 +474,13 @@ public SslHandler newTLSHandler() { return sslHandler; } + + private void markAutoCertRefresh(String tlsCertificatePath, String tlsKeyStore, String tlsKeyStorePasswordPath, + String tlsTrustStore, String tlsTrustStorePasswordPath) { + tlsCertificateFilePath = new FileModifiedTimeUpdater(tlsCertificatePath); + tlsKeyStoreFilePath = new FileModifiedTimeUpdater(tlsKeyStore); + tlsKeyStorePasswordFilePath = new FileModifiedTimeUpdater(tlsKeyStorePasswordPath); + tlsTrustStoreFilePath = new FileModifiedTimeUpdater(tlsTrustStore); + tlsTrustStorePasswordFilePath = new FileModifiedTimeUpdater(tlsTrustStorePasswordPath); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java index 68d86628494..94dec72fec8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java @@ -36,6 +36,8 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.auth.AuthCallbacks; import org.apache.bookkeeper.auth.AuthToken; @@ -386,6 +388,67 @@ public void testRefreshDurationForBookieCerts() throws Exception { newTlsKeyFile.delete(); } + /** + * Verify Bookkeeper-client refreshes certs at configured duration. + */ + @Test + public void testRefreshDurationForBookkeeperClientCerts() throws Exception { + Assume.assumeTrue(serverKeyStoreFormat == KeyStoreType.PEM); + + ClientConfiguration clientConf = new ClientConfiguration(baseClientConf); + String originalTlsCertFilePath = baseClientConf.getTLSCertificatePath(); + String invalidClientCert = getResourcePath("server-cert.pem"); + File originalTlsCertFile = new File(originalTlsCertFilePath); + File newTlsCertFile = IOUtils.createTempFileAndDeleteOnExit(originalTlsCertFilePath, "refresh"); + // clean up temp file even if test fails + newTlsCertFile.deleteOnExit(); + File invalidClientCertFile = new File(invalidClientCert); + // copy invalid cert to new temp file + FileUtils.copyFile(invalidClientCertFile, newTlsCertFile); + long refreshDurationInSec = 2; + clientConf.setTLSCertFilesRefreshDurationSeconds(1); + clientConf.setTLSCertificatePath(newTlsCertFile.getAbsolutePath()); + + // create a bookkeeper-client + try (BookKeeper client = new BookKeeper(clientConf)) { + byte[] testEntry = "testEntry".getBytes(); + byte[] passwd = "testPassword".getBytes(); + int totalAddEntries = 1; + CountDownLatch latch = new CountDownLatch(totalAddEntries); + AtomicInteger result = new AtomicInteger(-1); + LedgerHandle lh = client.createLedger(1, 1, DigestType.CRC32, passwd); + + for (int i = 0; i <= totalAddEntries; i++) { + lh.asyncAddEntry(testEntry, (rc, lgh, entryId, ctx) -> { + result.set(rc); + latch.countDown(); + }, null); + } + latch.await(1, TimeUnit.SECONDS); + Assert.assertNotEquals(result.get(), BKException.Code.OK); + + // Sleep so, cert file can be refreshed + Thread.sleep(refreshDurationInSec * 1000 + 1000); + + // copy valid key-file at given new location + FileUtils.copyFile(originalTlsCertFile, newTlsCertFile); + newTlsCertFile.setLastModified(System.currentTimeMillis() + 1000); + // client should be successfully able to add entries over tls + CountDownLatch latchWithValidCert = new CountDownLatch(totalAddEntries); + AtomicInteger validCertResult = new AtomicInteger(-1); + lh = client.createLedger(1, 1, DigestType.CRC32, passwd); + for (int i = 0; i <= totalAddEntries; i++) { + lh.asyncAddEntry(testEntry, (rc, lgh, entryId, ctx) -> { + validCertResult.set(rc); + latchWithValidCert.countDown(); + }, null); + } + latchWithValidCert.await(1, TimeUnit.SECONDS); + Assert.assertEquals(validCertResult.get(), BKException.Code.OK); + newTlsCertFile.delete(); + } + } + /** * Multiple clients, some with TLS, and some without TLS. */ From 953520dfb4ef3fec415eb39ad6c99ba7dec3c927 Mon Sep 17 00:00:00 2001 From: Dapeng Date: Wed, 22 Jan 2020 18:08:17 +0800 Subject: [PATCH 0369/1642] Increase the 'segment store path' in the 'Initializing stream cluster' log Increase the 'segment store path' in the 'Initializing stream cluster' log Reviewers: Enrico Olivelli , Sijie Guo This closes #2238 from SunDapeng1/branch-2238 --- .../stream/storage/impl/cluster/ZkClusterInitializer.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java index 53e749d6131..081bd92c141 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java @@ -32,6 +32,7 @@ import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.zookeeper.KeeperException; +import static org.apache.bookkeeper.stream.storage.StorageConstants.getSegmentsRootPath; /** * ZooKeeper Based Cluster Initializer. @@ -73,8 +74,6 @@ public boolean initializeCluster(URI metadataServiceUri, int numStorageContainer return false; } catch (StorageRuntimeException sre) { if (sre.getCause() instanceof KeeperException.NoNodeException) { - log.info("Initializing the stream cluster with {} storage containers with segment store path {}.", - numStorageContainers); String ledgersPath = metadataServiceUri.getPath(); Optional segmentStorePath; @@ -84,6 +83,9 @@ public boolean initializeCluster(URI metadataServiceUri, int numStorageContainer segmentStorePath = Optional.of(ledgersPath); } + log.info("Initializing the stream cluster with {} storage containers with segment store path {}.", + numStorageContainers, segmentStorePath.orElse(getSegmentsRootPath(ZK_METADATA_ROOT_PATH))); + boolean initialized = store.initializeCluster(numStorageContainers, segmentStorePath); log.info("Successfully initialized the stream cluster : \n{}", store.getClusterMetadata()); return initialized; From 01d110ba1677e4cde42ea7e2728b6f8311285f15 Mon Sep 17 00:00:00 2001 From: Dapeng Date: Wed, 22 Jan 2020 18:09:41 +0800 Subject: [PATCH 0370/1642] Create ledgersRootPath recursively for 'bin/bookkeeper shell metaformat' Create ledgersRootPath recursively for `bin/bookkeeper shell metaformat`. The existence of any parent znodes is not an error condition Reviewers: Enrico Olivelli , Sijie Guo This closes #2237 from SunDapeng1/branch-2236 --- .../discover/ZKRegistrationManager.java | 3 +- .../discover/TestZkRegistrationManager.java | 43 +++++++++++++++++++ 2 files changed, 45 insertions(+), 1 deletion(-) 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 index a2a3e7c62a8..e9bd7849807 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java @@ -400,7 +400,8 @@ public boolean prepareFormat() throws Exception { boolean availableNodeExists = null != zk.exists(bookieRegistrationPath, false); // Create ledgers root node if not exists if (!ledgerRootExists) { - zk.create(ledgersRootPath, "".getBytes(Charsets.UTF_8), zkAcls, CreateMode.PERSISTENT); + ZkUtils.createFullPathOptimistic(zk, ledgersRootPath, "".getBytes(Charsets.UTF_8), zkAcls, + CreateMode.PERSISTENT); } // create available bookies node if not exists if (!availableNodeExists) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java index ff1b7ea555b..8949e12bb08 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java @@ -18,8 +18,51 @@ */ package org.apache.bookkeeper.discover; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.test.ZooKeeperCluster; +import org.apache.bookkeeper.test.ZooKeeperUtil; +import org.apache.zookeeper.ZooKeeper; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + + /** * Unit test of {@link RegistrationManager}. */ public class TestZkRegistrationManager { + + private ZooKeeperCluster localZkServer; + private ZooKeeper zkc; + + @Before + public void setup() throws Exception { + localZkServer = new ZooKeeperUtil(); + localZkServer.startCluster(); + } + + @After + public void teardown() throws Exception { + localZkServer.stopCluster(); + } + + @Test + public void testPrepareFormat () throws Exception{ + try { + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + conf.setMetadataServiceUri("zk+hierarchical://localhost:2181/test/ledgers"); + zkc = localZkServer.getZooKeeperClient(); + ZKRegistrationManager zkRegistrationManager = new ZKRegistrationManager(conf, zkc,() -> {} ); + zkRegistrationManager.prepareFormat(); + assertTrue(zkc.exists("/test/ledgers",false) != null); + } finally { + if (zkc != null) { + zkc.close(); + } + } + } + } From 4cfa5522cac7a6956854bdd2f177ea028a4f9fcb Mon Sep 17 00:00:00 2001 From: Aloys <18124766348@163.com> Date: Wed, 22 Jan 2020 18:12:45 +0800 Subject: [PATCH 0371/1642] [HOTFIX] fix the http endpoint component count error This PR is a hotfix for "BookKeeper Admin REST API" document, the compoent of HTTP endpoints should be 5, but it's 4 in the doc. Reviewers: Enrico Olivelli , Sijie Guo This closes #2248 from aloyszhang/hotfix --- site/docs/4.10.0/admin/http.md | 10 +++++----- site/docs/4.6.0/admin/http.md | 10 +++++----- site/docs/4.6.1/admin/http.md | 10 +++++----- site/docs/4.6.2/admin/http.md | 10 +++++----- site/docs/4.7.0/admin/http.md | 10 +++++----- site/docs/4.7.1/admin/http.md | 10 +++++----- site/docs/4.7.2/admin/http.md | 10 +++++----- site/docs/4.7.3/admin/http.md | 10 +++++----- site/docs/4.8.0/admin/http.md | 10 +++++----- site/docs/4.8.1/admin/http.md | 10 +++++----- site/docs/4.8.2/admin/http.md | 10 +++++----- site/docs/4.9.0/admin/http.md | 10 +++++----- site/docs/4.9.1/admin/http.md | 10 +++++----- site/docs/4.9.2/admin/http.md | 10 +++++----- site/docs/latest/admin/http.md | 10 +++++----- 15 files changed, 75 insertions(+), 75 deletions(-) diff --git a/site/docs/4.10.0/admin/http.md b/site/docs/4.10.0/admin/http.md index c7fcf0cf5c7..46dd080f106 100644 --- a/site/docs/4.10.0/admin/http.md +++ b/site/docs/4.10.0/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.6.0/admin/http.md b/site/docs/4.6.0/admin/http.md index 0097adc62b8..f30bfe9c6f9 100644 --- a/site/docs/4.6.0/admin/http.md +++ b/site/docs/4.6.0/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.6.1/admin/http.md b/site/docs/4.6.1/admin/http.md index 0097adc62b8..f30bfe9c6f9 100644 --- a/site/docs/4.6.1/admin/http.md +++ b/site/docs/4.6.1/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.6.2/admin/http.md b/site/docs/4.6.2/admin/http.md index 0097adc62b8..f30bfe9c6f9 100644 --- a/site/docs/4.6.2/admin/http.md +++ b/site/docs/4.6.2/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.7.0/admin/http.md b/site/docs/4.7.0/admin/http.md index 0097adc62b8..f30bfe9c6f9 100644 --- a/site/docs/4.7.0/admin/http.md +++ b/site/docs/4.7.0/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.7.1/admin/http.md b/site/docs/4.7.1/admin/http.md index 0097adc62b8..f30bfe9c6f9 100644 --- a/site/docs/4.7.1/admin/http.md +++ b/site/docs/4.7.1/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.7.2/admin/http.md b/site/docs/4.7.2/admin/http.md index 0097adc62b8..f30bfe9c6f9 100644 --- a/site/docs/4.7.2/admin/http.md +++ b/site/docs/4.7.2/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.7.3/admin/http.md b/site/docs/4.7.3/admin/http.md index 0097adc62b8..f30bfe9c6f9 100644 --- a/site/docs/4.7.3/admin/http.md +++ b/site/docs/4.7.3/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.8.0/admin/http.md b/site/docs/4.8.0/admin/http.md index dc647449621..714d0394242 100644 --- a/site/docs/4.8.0/admin/http.md +++ b/site/docs/4.8.0/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.8.1/admin/http.md b/site/docs/4.8.1/admin/http.md index dc647449621..714d0394242 100644 --- a/site/docs/4.8.1/admin/http.md +++ b/site/docs/4.8.1/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.8.2/admin/http.md b/site/docs/4.8.2/admin/http.md index 7c1abea49f2..ab5e711a90e 100644 --- a/site/docs/4.8.2/admin/http.md +++ b/site/docs/4.8.2/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.9.0/admin/http.md b/site/docs/4.9.0/admin/http.md index 270dcb3a16f..ee6411abb53 100644 --- a/site/docs/4.9.0/admin/http.md +++ b/site/docs/4.9.0/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.9.1/admin/http.md b/site/docs/4.9.1/admin/http.md index 270dcb3a16f..ee6411abb53 100644 --- a/site/docs/4.9.1/admin/http.md +++ b/site/docs/4.9.1/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/4.9.2/admin/http.md b/site/docs/4.9.2/admin/http.md index 270dcb3a16f..ee6411abb53 100644 --- a/site/docs/4.9.2/admin/http.md +++ b/site/docs/4.9.2/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat diff --git a/site/docs/latest/admin/http.md b/site/docs/latest/admin/http.md index c7fcf0cf5c7..46dd080f106 100644 --- a/site/docs/latest/admin/http.md +++ b/site/docs/latest/admin/http.md @@ -7,12 +7,12 @@ To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.c ## All the endpoints -Currently all the HTTP endpoints could be divided into these 4 components: +Currently all the HTTP endpoints could be divided into these 5 components: 1. Heartbeat: heartbeat for a specific bookie. -1. Config: doing the server configuration for a specific bookie. -1. Ledger: HTTP endpoints related to ledgers. -1. Bookie: HTTP endpoints related to bookies. -1. AutoRecovery: HTTP endpoints related to auto recovery. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. ## Heartbeat From ad9d21c8f884d51d2d4e19fea77d52800b7e0e91 Mon Sep 17 00:00:00 2001 From: Aloys <18124766348@163.com> Date: Wed, 22 Jan 2020 18:13:47 +0800 Subject: [PATCH 0372/1642] [HOTFIX] [DOCS] remove redundant word in protocol documents This PR removes redundant word "because" in protocol documents. Reviewers: Sijie Guo This closes #2249 from aloyszhang/hotfix-protocol --- site/docs/4.10.0/development/protocol.md | 2 +- site/docs/4.5.0/development/protocol.md | 2 +- site/docs/4.5.1/development/protocol.md | 2 +- site/docs/4.6.0/development/protocol.md | 2 +- site/docs/4.6.1/development/protocol.md | 2 +- site/docs/4.6.2/development/protocol.md | 2 +- site/docs/4.7.0/development/protocol.md | 2 +- site/docs/4.7.1/development/protocol.md | 2 +- site/docs/4.7.2/development/protocol.md | 2 +- site/docs/4.7.3/development/protocol.md | 2 +- site/docs/4.8.0/development/protocol.md | 2 +- site/docs/4.8.1/development/protocol.md | 2 +- site/docs/4.8.2/development/protocol.md | 2 +- site/docs/4.9.0/development/protocol.md | 2 +- site/docs/4.9.1/development/protocol.md | 2 +- site/docs/4.9.2/development/protocol.md | 2 +- site/docs/latest/development/protocol.md | 2 +- 17 files changed, 17 insertions(+), 17 deletions(-) diff --git a/site/docs/4.10.0/development/protocol.md b/site/docs/4.10.0/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.10.0/development/protocol.md +++ b/site/docs/4.10.0/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.5.0/development/protocol.md b/site/docs/4.5.0/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.5.0/development/protocol.md +++ b/site/docs/4.5.0/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.5.1/development/protocol.md b/site/docs/4.5.1/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.5.1/development/protocol.md +++ b/site/docs/4.5.1/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.6.0/development/protocol.md b/site/docs/4.6.0/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.6.0/development/protocol.md +++ b/site/docs/4.6.0/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.6.1/development/protocol.md b/site/docs/4.6.1/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.6.1/development/protocol.md +++ b/site/docs/4.6.1/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.6.2/development/protocol.md b/site/docs/4.6.2/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.6.2/development/protocol.md +++ b/site/docs/4.6.2/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.7.0/development/protocol.md b/site/docs/4.7.0/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.7.0/development/protocol.md +++ b/site/docs/4.7.0/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.7.1/development/protocol.md b/site/docs/4.7.1/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.7.1/development/protocol.md +++ b/site/docs/4.7.1/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.7.2/development/protocol.md b/site/docs/4.7.2/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.7.2/development/protocol.md +++ b/site/docs/4.7.2/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.7.3/development/protocol.md b/site/docs/4.7.3/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.7.3/development/protocol.md +++ b/site/docs/4.7.3/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.8.0/development/protocol.md b/site/docs/4.8.0/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.8.0/development/protocol.md +++ b/site/docs/4.8.0/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.8.1/development/protocol.md b/site/docs/4.8.1/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.8.1/development/protocol.md +++ b/site/docs/4.8.1/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.8.2/development/protocol.md b/site/docs/4.8.2/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.8.2/development/protocol.md +++ b/site/docs/4.8.2/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.9.0/development/protocol.md b/site/docs/4.9.0/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.9.0/development/protocol.md +++ b/site/docs/4.9.0/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.9.1/development/protocol.md b/site/docs/4.9.1/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.9.1/development/protocol.md +++ b/site/docs/4.9.1/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/4.9.2/development/protocol.md b/site/docs/4.9.2/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/4.9.2/development/protocol.md +++ b/site/docs/4.9.2/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 diff --git a/site/docs/latest/development/protocol.md b/site/docs/latest/development/protocol.md index 6d17aa0ed45..f4410444427 100644 --- a/site/docs/latest/development/protocol.md +++ b/site/docs/latest/development/protocol.md @@ -124,7 +124,7 @@ In many cases, leader election is really leader suggestion. Multiple nodes could 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. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced 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 From 78275ed965c380bfc1a2af0adcf0c33eb53f1a77 Mon Sep 17 00:00:00 2001 From: Aloys <18124766348@163.com> Date: Wed, 22 Jan 2020 18:15:19 +0800 Subject: [PATCH 0373/1642] [HOTFIX] [DOCS] fix error in leger api document This PR : 1. add missing comma and dot 2. fix some layout error in ledger-api document Reviewers: Sijie Guo This closes #2250 from aloyszhang/hotfix-ledger-api --- site/docs/4.10.0/api/ledger-api.md | 11 +++++------ site/docs/4.5.0/api/ledger-api.md | 11 +++++------ site/docs/4.5.1/api/ledger-api.md | 11 +++++------ site/docs/4.6.0/api/ledger-api.md | 11 +++++------ site/docs/4.6.1/api/ledger-api.md | 11 +++++------ site/docs/4.6.2/api/ledger-api.md | 11 +++++------ site/docs/4.7.0/api/ledger-api.md | 11 +++++------ site/docs/4.7.1/api/ledger-api.md | 11 +++++------ site/docs/4.7.2/api/ledger-api.md | 11 +++++------ site/docs/4.7.3/api/ledger-api.md | 11 +++++------ site/docs/4.8.0/api/ledger-api.md | 11 +++++------ site/docs/4.8.1/api/ledger-api.md | 11 +++++------ site/docs/4.8.2/api/ledger-api.md | 11 +++++------ site/docs/4.9.0/api/ledger-api.md | 11 +++++------ site/docs/4.9.1/api/ledger-api.md | 11 +++++------ site/docs/4.9.2/api/ledger-api.md | 11 +++++------ site/docs/latest/api/ledger-api.md | 11 +++++------ 17 files changed, 85 insertions(+), 102 deletions(-) diff --git a/site/docs/4.10.0/api/ledger-api.md b/site/docs/4.10.0/api/ledger-api.md index adf2961a989..a3494926ef3 100644 --- a/site/docs/4.10.0/api/ledger-api.md +++ b/site/docs/4.10.0/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.5.0/api/ledger-api.md b/site/docs/4.5.0/api/ledger-api.md index 4e1070d717a..7507cc7f63b 100644 --- a/site/docs/4.5.0/api/ledger-api.md +++ b/site/docs/4.5.0/api/ledger-api.md @@ -155,8 +155,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -172,7 +172,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -186,9 +186,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -196,6 +194,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.5.1/api/ledger-api.md b/site/docs/4.5.1/api/ledger-api.md index 031389c0fb0..17db9a37fd8 100644 --- a/site/docs/4.5.1/api/ledger-api.md +++ b/site/docs/4.5.1/api/ledger-api.md @@ -155,8 +155,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -172,7 +172,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -186,9 +186,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -196,6 +194,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.6.0/api/ledger-api.md b/site/docs/4.6.0/api/ledger-api.md index acde8d6951b..3f8f478dd9a 100644 --- a/site/docs/4.6.0/api/ledger-api.md +++ b/site/docs/4.6.0/api/ledger-api.md @@ -155,8 +155,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -172,7 +172,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -186,9 +186,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -196,6 +194,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.6.1/api/ledger-api.md b/site/docs/4.6.1/api/ledger-api.md index c247bfb8fa9..ac687d5f359 100644 --- a/site/docs/4.6.1/api/ledger-api.md +++ b/site/docs/4.6.1/api/ledger-api.md @@ -218,8 +218,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -235,7 +235,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -249,9 +249,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -259,6 +257,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.6.2/api/ledger-api.md b/site/docs/4.6.2/api/ledger-api.md index 2007e71a02f..8407a2a60bf 100644 --- a/site/docs/4.6.2/api/ledger-api.md +++ b/site/docs/4.6.2/api/ledger-api.md @@ -218,8 +218,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -235,7 +235,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -249,9 +249,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -259,6 +257,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.7.0/api/ledger-api.md b/site/docs/4.7.0/api/ledger-api.md index edb67f5ab26..862954aa213 100644 --- a/site/docs/4.7.0/api/ledger-api.md +++ b/site/docs/4.7.0/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.7.1/api/ledger-api.md b/site/docs/4.7.1/api/ledger-api.md index d44437b61e6..02129df3d41 100644 --- a/site/docs/4.7.1/api/ledger-api.md +++ b/site/docs/4.7.1/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.7.2/api/ledger-api.md b/site/docs/4.7.2/api/ledger-api.md index c22daeb121a..9ce93b89e8c 100644 --- a/site/docs/4.7.2/api/ledger-api.md +++ b/site/docs/4.7.2/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.7.3/api/ledger-api.md b/site/docs/4.7.3/api/ledger-api.md index 5d69cb1b0fa..2c79794578a 100644 --- a/site/docs/4.7.3/api/ledger-api.md +++ b/site/docs/4.7.3/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.8.0/api/ledger-api.md b/site/docs/4.8.0/api/ledger-api.md index 2b44d47e995..03ea7089e10 100644 --- a/site/docs/4.8.0/api/ledger-api.md +++ b/site/docs/4.8.0/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.8.1/api/ledger-api.md b/site/docs/4.8.1/api/ledger-api.md index 8147d39c9a9..51890b74c4e 100644 --- a/site/docs/4.8.1/api/ledger-api.md +++ b/site/docs/4.8.1/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.8.2/api/ledger-api.md b/site/docs/4.8.2/api/ledger-api.md index b6cb0f02a95..cd46e994126 100644 --- a/site/docs/4.8.2/api/ledger-api.md +++ b/site/docs/4.8.2/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.9.0/api/ledger-api.md b/site/docs/4.9.0/api/ledger-api.md index 15d282945b0..65af1321b16 100644 --- a/site/docs/4.9.0/api/ledger-api.md +++ b/site/docs/4.9.0/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.9.1/api/ledger-api.md b/site/docs/4.9.1/api/ledger-api.md index 08984696f91..c1dc8ec6b3f 100644 --- a/site/docs/4.9.1/api/ledger-api.md +++ b/site/docs/4.9.1/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/4.9.2/api/ledger-api.md b/site/docs/4.9.2/api/ledger-api.md index c0817e93d1f..fa112fb6516 100644 --- a/site/docs/4.9.2/api/ledger-api.md +++ b/site/docs/4.9.2/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example diff --git a/site/docs/latest/api/ledger-api.md b/site/docs/latest/api/ledger-api.md index 38f436d1cf8..e35ebababc0 100644 --- a/site/docs/latest/api/ledger-api.md +++ b/site/docs/latest/api/ledger-api.md @@ -180,8 +180,8 @@ while (entries.hasNextElement()) { ### 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. +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. @@ -197,7 +197,7 @@ while (entries.hasNextElement()) { ## Deleting ledgers -{% pop Ledgers %} can also be deleted synchronously or asynchronously. +{% pop Ledgers %} can be deleted synchronously which may throw exception: ```java long ledgerId = 1234; @@ -211,9 +211,7 @@ try { ### Delete entries asynchronously -Exceptions thrown: - -* +{% pop Ledgers %} can also be deleted asynchronously: ```java class DeleteEntryCallback implements AsyncCallback.DeleteCallback { @@ -221,6 +219,7 @@ class DeleteEntryCallback implements AsyncCallback.DeleteCallback { System.out.println("Delete completed"); } } +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); ``` ## Simple example From 55e5e4ed8644ad97b5ff89ef28beaab3b7932c1a Mon Sep 17 00:00:00 2001 From: Alessandro Luccaroni Date: Wed, 22 Jan 2020 11:16:34 +0100 Subject: [PATCH 0374/1642] Website: Update Release Management for new Time Based Release Plan As discussed on the mailing list, I'm updating the BookKeeper website to reflect the new Time Based Release Plan. You can find the staging website https://aluccaroni.github.io/bookkeeper-staging-site/community/releases/ Please note that I've removed the calendar since it was empty Reviewers: Enrico Olivelli , Sijie Guo This closes #2251 from aluccaroni/master --- site/community/releases.md | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/site/community/releases.md b/site/community/releases.md index df556656d46..9e663eba05b 100644 --- a/site/community/releases.md +++ b/site/community/releases.md @@ -4,7 +4,7 @@ title: Release Management > Apache BookKeeper community adopts [Time Based Release Plan](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-13+-+Time+Based+Release+Plan) starting from 4.6.0. -Apache BookKeeper community makes a feture release every 3 month. +Apache BookKeeper community makes a feture release every 6 month (since `4.10.0`). - A month before the release date, the release manager will cut branches and also publish a list of features that will be included in the release. These features will typically be [BookKeeper Proposals](http://bookkeeper.apache.org/community/bookkeeper_proposals/), but not always. @@ -15,19 +15,17 @@ Apache BookKeeper community makes a feture release every 3 month. ### Feature Release Window -The next feature release is `4.10.0`. The release window is the following: +The next feature release is `4.11.0`. The release window is the following: | **Date** | **Event** | -| February 1, 2019 | Merge window opens on master branch | -| March 12, 2019 | Major feature should be in, Cut release branch | -| March 19, 2019 | Minor feature should be in, Stabilize release branch | -| March 26, 2019 | Code freeze, Only accept fixes for blocking issues, Rolling out release candidates | +| December 1, 2019 | Merge window opens on master branch | +| April 6, 2020 | Major feature should be in, Cut release branch | +| April 13, 2020 | Minor feature should be in, Stabilize release branch | +| April 20, 2020 | Code freeze, Only accept fixes for blocking issues, Rolling out release candidates | ## Release Schedule -- **4.10.0**: February 2019 - March 2019 -- **4.11.0**: April 2019 - May 2019 -- **4.12.0**: June 2019 - July 2019 -- **4.13.0**: August 2019 - September 2019 +- **4.11.0**: December 2019 - May 2020 +- **4.12.0**: June 2020 - November 2020 +- **4.13.0**: December 2020 - May 2021 - From 56ea092e3aadd5495cb7f2f28343250a75f8d552 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 22 Jan 2020 18:18:33 +0800 Subject: [PATCH 0375/1642] [CI] Migrate the Jenkins CI to Github Action --- Master Issue: #2242 *Modifications* Migrate theJenkins CI to Github Action Reviewers: Enrico Olivelli , Sijie Guo This closes #2246 from zymap/migrate-ci --- .github/workflows/bookie-tests.yml | 52 +++++++++++++++++++ .github/workflows/client-tests.yml | 42 +++++++++++++++ .../workflows/compatibility-check-java11.yml | 41 +++++++++++++++ .../workflows/compatibility-check-java8.yml | 41 +++++++++++++++ .github/workflows/integration-tests.yml | 48 +++++++++++++++++ .github/workflows/pr-validation.yml | 42 +++++++++++++++ .github/workflows/remaining-tests.yml | 41 +++++++++++++++ .github/workflows/replication-tests.yml | 42 +++++++++++++++ .github/workflows/tls-tests.yml | 42 +++++++++++++++ ...ryMemTable.java => EntryMemTableTest.java} | 6 +-- ...che.java => FileInfoBackingCacheTest.java} | 4 +- ...r.java => GcOverreplicatedLedgerTest.java} | 4 +- ...anager.java => LedgerDirsManagerTest.java} | 2 +- ...pListArena.java => SkipListArenaTest.java} | 2 +- ...estSyncThread.java => SyncThreadTest.java} | 4 +- .../{EntryLogTest.java => TestEntryLog.java} | 4 +- ...java => TestInterleavedLedgerStorage.java} | 6 +-- 17 files changed, 407 insertions(+), 16 deletions(-) create mode 100644 .github/workflows/bookie-tests.yml create mode 100644 .github/workflows/client-tests.yml create mode 100644 .github/workflows/compatibility-check-java11.yml create mode 100644 .github/workflows/compatibility-check-java8.yml create mode 100644 .github/workflows/integration-tests.yml create mode 100644 .github/workflows/pr-validation.yml create mode 100644 .github/workflows/remaining-tests.yml create mode 100644 .github/workflows/replication-tests.yml create mode 100644 .github/workflows/tls-tests.yml rename bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/{TestEntryMemTable.java => EntryMemTableTest.java} (98%) rename bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/{TestFileInfoBackingCache.java => FileInfoBackingCacheTest.java} (99%) rename bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/{TestGcOverreplicatedLedger.java => GcOverreplicatedLedgerTest.java} (98%) rename bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/{TestLedgerDirsManager.java => LedgerDirsManagerTest.java} (99%) rename bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/{TestSkipListArena.java => SkipListArenaTest.java} (99%) rename bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/{TestSyncThread.java => SyncThreadTest.java} (99%) rename bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/{EntryLogTest.java => TestEntryLog.java} (99%) rename bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/{InterleavedLedgerStorageTest.java => TestInterleavedLedgerStorage.java} (98%) diff --git a/.github/workflows/bookie-tests.yml b/.github/workflows/bookie-tests.yml new file mode 100644 index 00000000000..eb41ce02048 --- /dev/null +++ b/.github/workflows/bookie-tests.yml @@ -0,0 +1,52 @@ +# +# 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. +# + +name: Bookie Tests + +on: + pull_request: + branches: + - master + paths-ignore: + - 'site/**' + + +jobs: + test: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v1 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + + - name: Maven build bookkeeper-server + run: mvn -am -pl bookkeeper-server clean install -DskipTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + + - name: Run EntryLogTests + run: mvn -pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.TestEntryLog" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + + - name: Run InterleavedLedgerStorageTest + run: mvn -pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.TestInterleavedLederStorage" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + + - name: Run bookie tests + run: mvn -pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.*Test" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/client-tests.yml b/.github/workflows/client-tests.yml new file mode 100644 index 00000000000..a38b2ee74c8 --- /dev/null +++ b/.github/workflows/client-tests.yml @@ -0,0 +1,42 @@ +# +# 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. +# + +name: Client Tests + +on: + pull_request: + branches: + - master + paths-ignore: + - 'site/**' + + +jobs: + test: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v1 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Run client tests + run: mvn -am -pl bookkeeper-server clean install test -Dtest="org.apache.bookkeeper.client.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/compatibility-check-java11.yml b/.github/workflows/compatibility-check-java11.yml new file mode 100644 index 00000000000..de63ef1544a --- /dev/null +++ b/.github/workflows/compatibility-check-java11.yml @@ -0,0 +1,41 @@ +# +# 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. +# + +name: Compatibility Check Java11 + +on: + pull_request: + branches: + - master + paths-ignore: + - 'site/**' + +jobs: + check: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v1 + - name: Set up JDK 1.11 + uses: actions/setup-java@v1 + with: + java-version: 1.11 + - name: Build with Maven + run: mvn clean package -Dstream -DskipBookKeeperServerTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/compatibility-check-java8.yml b/.github/workflows/compatibility-check-java8.yml new file mode 100644 index 00000000000..de82ecfd252 --- /dev/null +++ b/.github/workflows/compatibility-check-java8.yml @@ -0,0 +1,41 @@ +# +# 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. +# + +name: Compatibility Check Java8 + +on: + pull_request: + branches: + - master + paths-ignore: + - 'site/**' + +jobs: + check: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v1 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Build with Maven + run: mvn clean package spotbugs:check -Dstream -DskipBookKeeperServerTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml new file mode 100644 index 00000000000..35536de76f8 --- /dev/null +++ b/.github/workflows/integration-tests.yml @@ -0,0 +1,48 @@ +# +# 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. +# + +name: Integration Tests + +on: + pull_request: + branches: + - master + paths-ignore: + - 'site/**' + + +jobs: + test: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v1 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Build with Maven + run: mvn -B clean install -Dstream -Pdocker -DskipTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + + - name: Run metadata driver tests + run: mvn -B -f metadata-drivers/pom.xml test -DintegrationTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + + - name: Run all integration tests + run: mvn -B -f tests/pom.xml test -Dstream -DintegrationTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/pr-validation.yml b/.github/workflows/pr-validation.yml new file mode 100644 index 00000000000..04550e90ba4 --- /dev/null +++ b/.github/workflows/pr-validation.yml @@ -0,0 +1,42 @@ +# +# 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. +# + +name: PR Validation + +on: + pull_request: + branches: + - master + paths-ignore: + - 'site/**' + + +jobs: + check: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v1 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Validate pull request style + run: mvn clean apache-rat:check checkstyle:check package -Ddistributedlog -Dstream -DskipTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/remaining-tests.yml b/.github/workflows/remaining-tests.yml new file mode 100644 index 00000000000..e51b9feee73 --- /dev/null +++ b/.github/workflows/remaining-tests.yml @@ -0,0 +1,41 @@ +# +# 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. +# + +name: Remaining Tests + +on: + pull_request: + branches: + - master + paths-ignore: + - 'site/**' + +jobs: + test: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v1 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Run remaining tests + run: mvn -am -pl bookkeeper-server clean install test -Dtest="!org.apache.bookkeeper.client.**,!org.apache.bookkeeper.bookie.**,!org.apache.bookkeeper.replication.**,!org.apache.bookkeeper.tls.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/replication-tests.yml b/.github/workflows/replication-tests.yml new file mode 100644 index 00000000000..912c276b1f9 --- /dev/null +++ b/.github/workflows/replication-tests.yml @@ -0,0 +1,42 @@ +# +# 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. +# + +name: Replication Tests + +on: + pull_request: + branches: + - master + paths-ignore: + - 'site/**' + + +jobs: + test: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v1 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Run replication tests + run: mvn -am -pl bookkeeper-server clean install test -Dtest="org.apache.bookkeeper.replication.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/tls-tests.yml b/.github/workflows/tls-tests.yml new file mode 100644 index 00000000000..47ef36166fa --- /dev/null +++ b/.github/workflows/tls-tests.yml @@ -0,0 +1,42 @@ +# +# 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. +# + +name: TLS Tests + +on: + pull_request: + branches: + - master + paths-ignore: + - 'site/**' + + +jobs: + test: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v1 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Run tls tests + run: mvn -am -pl bookkeeper-server clean install test -Dtest="org.apache.bookkeeper.tls.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryMemTable.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryMemTableTest.java similarity index 98% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryMemTable.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryMemTableTest.java index 9e6c5595fd2..946af02325e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryMemTable.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryMemTableTest.java @@ -59,7 +59,7 @@ */ @Slf4j @RunWith(Parameterized.class) -public class TestEntryMemTable implements CacheCallback, SkipListFlusher, CheckpointSource { +public class EntryMemTableTest implements CacheCallback, SkipListFlusher, CheckpointSource { private Class entryMemTableClass; private EntryMemTable memTable; @@ -71,7 +71,7 @@ public static Collection memTableClass() { return Arrays.asList(new Object[][] { { EntryMemTable.class }, { EntryMemTableWithParallelFlusher.class } }); } - public TestEntryMemTable(Class entryMemTableClass) { + public EntryMemTableTest(Class entryMemTableClass) { this.entryMemTableClass = entryMemTableClass; } @@ -422,7 +422,7 @@ public void run() { for (long entryId = newNumOfEntries + 1; entryId <= concurrentAddOfEntries; entryId++) { random.nextBytes(data); boolean thisEntryAddedSuccessfully = (memTable.addEntry(ledgerId, entryId, - ByteBuffer.wrap(data), TestEntryMemTable.this) != 0); + ByteBuffer.wrap(data), EntryMemTableTest.this) != 0); successfullyAdded.set(successfullyAdded.get() && thisEntryAddedSuccessfully); Thread.sleep(10); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestFileInfoBackingCache.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/FileInfoBackingCacheTest.java similarity index 99% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestFileInfoBackingCache.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/FileInfoBackingCacheTest.java index 77f5ebace77..267fcbbd8f8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestFileInfoBackingCache.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/FileInfoBackingCacheTest.java @@ -55,14 +55,14 @@ * Tests for FileInfoBackingCache. */ @Slf4j -public class TestFileInfoBackingCache { +public class FileInfoBackingCacheTest { final byte[] masterKey = new byte[0]; final File baseDir; final ThreadFactory threadFactory = new ThreadFactoryBuilder() .setNameFormat("backing-cache-test-%d").setDaemon(true).build(); ExecutorService executor; - public TestFileInfoBackingCache() throws Exception { + public FileInfoBackingCacheTest() throws Exception { baseDir = File.createTempFile("foo", "bar"); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java similarity index 98% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java index bb393294b0f..371971b7138 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java @@ -56,7 +56,7 @@ * Test GC-overreplicated ledger. */ @RunWith(Parameterized.class) -public class TestGcOverreplicatedLedger extends LedgerManagerTestCase { +public class GcOverreplicatedLedgerTest extends LedgerManagerTestCase { @Before @Override @@ -66,7 +66,7 @@ public void setUp() throws Exception { activeLedgers = new SnapshotMap(); } - public TestGcOverreplicatedLedger(Class lmFactoryCls) { + public GcOverreplicatedLedgerTest(Class lmFactoryCls) { super(lmFactoryCls, 3); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerDirsManagerTest.java similarity index 99% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerDirsManagerTest.java index e0a4838e43c..6a877e4a23f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestLedgerDirsManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerDirsManagerTest.java @@ -61,7 +61,7 @@ */ @RunWith(PowerMockRunner.class) @PrepareForTest(LedgerDirsMonitor.class) -public class TestLedgerDirsManager { +public class LedgerDirsManagerTest { ServerConfiguration conf; File curDir; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSkipListArena.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SkipListArenaTest.java similarity index 99% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSkipListArena.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SkipListArenaTest.java index 83e4794d0e7..873e237660d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSkipListArena.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SkipListArenaTest.java @@ -41,7 +41,7 @@ /** * Test the SkipListArena class. */ -public class TestSkipListArena { +public class SkipListArenaTest { class CustomConfiguration extends ServerConfiguration { @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java similarity index 99% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java index 22535f3d557..9529b27989a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestSyncThread.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java @@ -54,8 +54,8 @@ /** * Test a synchronization thread. */ -public class TestSyncThread { - private static final Logger LOG = LoggerFactory.getLogger(TestSyncThread.class); +public class SyncThreadTest { + private static final Logger LOG = LoggerFactory.getLogger(SyncThreadTest.class); ExecutorService executor = null; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java similarity index 99% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java index ed52982d6d5..836fdf25cb2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java @@ -74,8 +74,8 @@ /** * Tests for EntryLog. */ -public class EntryLogTest { - private static final Logger LOG = LoggerFactory.getLogger(EntryLogTest.class); +public class TestEntryLog { + private static final Logger LOG = LoggerFactory.getLogger(TestEntryLog.class); final List tempDirs = new ArrayList(); final Random rand = new Random(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestInterleavedLedgerStorage.java similarity index 98% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestInterleavedLedgerStorage.java index 641617305da..59afa0c1fc4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorageTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestInterleavedLedgerStorage.java @@ -68,15 +68,15 @@ * Test for InterleavedLedgerStorage. */ @RunWith(Parameterized.class) -public class InterleavedLedgerStorageTest { - private static final Logger LOG = LoggerFactory.getLogger(InterleavedLedgerStorageTest.class); +public class TestInterleavedLedgerStorage { + private static final Logger LOG = LoggerFactory.getLogger(TestInterleavedLedgerStorage.class); @Parameterized.Parameters public static Iterable elplSetting() { return Arrays.asList(true, false); } - public InterleavedLedgerStorageTest(boolean elplSetting) { + public TestInterleavedLedgerStorage(boolean elplSetting) { conf.setEntryLogSizeLimit(2048); conf.setEntryLogPerLedgerEnabled(elplSetting); } From 06a34e1d20da7a8e8547f2ea33937426450fc657 Mon Sep 17 00:00:00 2001 From: Dapeng Date: Wed, 22 Jan 2020 18:22:56 +0800 Subject: [PATCH 0376/1642] bin/bookkeeper shell supports the -conf ### Motivation `bin/bookkeeper shell ` support `-conf configuration`, but it doesn't work. bk_server.conf is loaded by default, even if the configuration is specified ![image](https://user-images.githubusercontent.com/42792537/70309755-2a534180-1849-11ea-96b0-bd5b07a6b717.png) Modify `bin/bookkeeper` to specify the configuration file as an absolute path Reviewers: Enrico Olivelli , Sijie Guo This closes #2211 from SunDapeng1/branch-2211 --- bin/bookkeeper | 40 +++++++++++++++++++++++++++++----------- 1 file changed, 29 insertions(+), 11 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index ddaae2286e8..4b55d1eb222 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -108,16 +108,30 @@ shift LOCALBOOKIES_CONFIG_DIR="${LOCALBOOKIES_CONFIG_DIR:-/tmp/localbookies-config}" if [ ${COMMAND} == "shell" ]; then DEFAULT_LOG_CONF=${BK_HOME}/conf/log4j.shell.properties - if [[ $1 == "-localbookie" ]]; then - if [[ $2 == *:* ]]; - then - BOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/$2.conf - shift 2 - else - BOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/baseconf.conf - shift - fi - fi + LOCALBOOKIE=0 + while [ $# -gt 0 ] + do + opt="$1" + case $opt in + (-conf) + BOOKIE_CONF="$2" + shift 2 + ;; + (-localbookie) + LOCALBOOKIE=1 + if [[ $2 == *:* ]]; + then + LOCALBOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/$2.conf + shift 2 + else + LOCALBOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/baseconf.conf + shift + fi + ;; + (*) + ;; + esac + done fi if [ -z "$BOOKIE_ZK_CONF" ]; then @@ -172,7 +186,11 @@ elif [ $COMMAND == "zookeeper" ]; then exec $JAVA $OPTS -Dbookkeeper.log.file=$BOOKIE_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $BOOKIE_ZK_CONF $@ elif [ ${COMMAND} == "shell" ]; then ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}" - exec ${JAVA} ${OPTS} ${ENTRY_FORMATTER_ARG} org.apache.bookkeeper.bookie.BookieShell -conf ${BOOKIE_CONF} $@ + if [ ${LOCALBOOKIE} == 1 ]; then + exec ${JAVA} ${OPTS} ${ENTRY_FORMATTER_ARG} org.apache.bookkeeper.bookie.BookieShell -conf ${LOCALBOOKIE_CONF} $@ + else + exec ${JAVA} ${OPTS} ${ENTRY_FORMATTER_ARG} org.apache.bookkeeper.bookie.BookieShell -conf ${BOOKIE_CONF} $@ + fi elif [ ${COMMAND} == "help" ]; then bookkeeper_help; else From 18d81dc6b1193f0cbf811cc38fd173375a64e965 Mon Sep 17 00:00:00 2001 From: Alessandro Luccaroni Date: Sat, 25 Jan 2020 16:47:26 +0100 Subject: [PATCH 0377/1642] Website: add EOL tag to navbar As discussed on the mailing list, I'm updating the BookKeeper website to reflect the EOLed version of BookKeeper You can find the staging website https://aluccaroni.github.io/bookkeeper-staging-site Here a screenshot of the new navbar: ![image](https://user-images.githubusercontent.com/11058798/72979770-587ce880-3dd9-11ea-86bf-de4a9aa694d3.png) Reviewers: Enrico Olivelli , Sijie Guo This closes #2253 from aluccaroni/master --- site/_config.yml | 22 +++++++++++++++++++++- site/_includes/navbar.html | 12 ++++++++++-- 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/site/_config.yml b/site/_config.yml index 2de1b38dbd0..b293812c62e 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -25,7 +25,6 @@ versions: - "4.6.0" - "4.5.1" - "4.5.0" - archived_versions: - "4.4.0" - "4.3.2" @@ -41,6 +40,27 @@ archived_versions: latest_version: "4.11.0-SNAPSHOT" latest_release: "4.10.0" stable_release: "4.9.2" +eol_releases: +- "4.7.3" +- "4.7.2" +- "4.7.1" +- "4.7.0" +- "4.6.2" +- "4.6.1" +- "4.6.0" +- "4.5.1" +- "4.5.0" +- "4.4.0" +- "4.3.2" +- "4.3.1" +- "4.3.0" +- "4.2.4" +- "4.2.3" +- "4.2.2" +- "4.2.1" +- "4.2.0" +- "4.1.0" +- "4.0.0" distributedlog_version: "4.10.0" defaults: diff --git a/site/_includes/navbar.html b/site/_includes/navbar.html index 0539c718ffc..b9d8499f0a6 100644 --- a/site/_includes/navbar.html +++ b/site/_includes/navbar.html @@ -31,13 +31,21 @@ {% for version in site.versions %} Release {{version}} - {% if version == site.stable_release %}Stable{% endif %} + {% if site.stable_release == version %} + Stable + {% elsif site.eol_releases contains version %} + EOL + {% endif %} {% endfor %} {% for version in site.archived_versions %} Release {{version}} - {% if version == site.stable_release %}Stable{% endif %} + {% if site.stable_release == version %} + Stable + {% elsif site.eol_releases contains version %} + EOL + {% endif %} {% endfor %} From 70f60a6f3e38fb26c2cafa9c1e95368e88146e48 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 27 Jan 2020 08:31:08 -0800 Subject: [PATCH 0378/1642] Remove Jenkins precommit jobs and fix checkstyle errors Reviewers: Enrico Olivelli This closes #2255 from sijie/remove_bookkeeper_tests --- ...b_bookkeeper_precommit_bookie_tests.groovy | 73 --------------- ...b_bookkeeper_precommit_client_tests.groovy | 73 --------------- ...okkeeper_precommit_integrationtests.groovy | 90 ------------------- .../job_bookkeeper_precommit_java11.groovy | 62 ------------- .../job_bookkeeper_precommit_java8.groovy | 61 ------------- ...ookkeeper_precommit_remaining_tests.groovy | 77 ---------------- ...kkeeper_precommit_replication_tests.groovy | 73 --------------- .../job_bookkeeper_precommit_tls_tests.groovy | 73 --------------- ...job_bookkeeper_precommit_validation.groovy | 53 ----------- .../discover/TestZkRegistrationManager.java | 11 ++- 10 files changed, 5 insertions(+), 641 deletions(-) delete mode 100644 .test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy delete mode 100644 .test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy delete mode 100644 .test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy delete mode 100644 .test-infra/jenkins/job_bookkeeper_precommit_java11.groovy delete mode 100644 .test-infra/jenkins/job_bookkeeper_precommit_java8.groovy delete mode 100644 .test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy delete mode 100644 .test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy delete mode 100644 .test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy delete mode 100644 .test-infra/jenkins/job_bookkeeper_precommit_validation.groovy diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy deleted file mode 100644 index a9f4ba0ed91..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_precommit_bookie_tests.groovy +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// test `org.apache.bookkeeper.bookie.**` -freeStyleJob('bookkeeper_precommit_bookie_tests') { - description('Run bookkeeper bookie tests in Java 8.') - - // clean up the workspace before build - wrappers { preBuildCleanup() } - - // Execute concurrent builds if necessary. - concurrentBuild() - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - 'master', - 'JDK 1.8 (latest)', - 200, - 'ubuntu', - '${sha1}') - - // Sets that this is a PreCommit job. - common_job_properties.setPreCommit( - delegate, - 'Bookie Tests (trigger via `run bookkeeper-server bookie tests`)', - '.*(re)?run bookkeeper-server (bookie )?tests.*', - '.*\\[x\\] \\[skip bookkeeper-server (bookie )?tests\\].*', - true) - - steps { - // Temporary information gathering to see if full disks are causing the builds to flake - shell("id") - shell("ulimit -a") - shell("pwd") - shell("df -Th") - shell("ps aux") - - // Build everything - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-B -am -pl bookkeeper-server clean install -DskipTests') - } - - // Test the package - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.**"') - } - } - -} diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy deleted file mode 100644 index 9c2c3fba92f..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_precommit_client_tests.groovy +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// test `org.apache.bookkeeper.client.**` -freeStyleJob('bookkeeper_precommit_client_tests') { - description('Run bookkeeper client tests in Java 8.') - - // clean up the workspace before build - wrappers { preBuildCleanup() } - - // Execute concurrent builds if necessary. - concurrentBuild() - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - 'master', - 'JDK 1.8 (latest)', - 200, - 'ubuntu', - '${sha1}') - - // Sets that this is a PreCommit job. - common_job_properties.setPreCommit( - delegate, - 'Client Tests (trigger via `run bookkeeper-server client tests`)', - '.*(re)?run bookkeeper-server (client )?tests.*', - '.*\\[x\\] \\[skip bookkeeper-server (client )?tests\\].*', - true) - - steps { - // Temporary information gathering to see if full disks are causing the builds to flake - shell("id") - shell("ulimit -a") - shell("pwd") - shell("df -Th") - shell("ps aux") - - // Build everything - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-B -am -pl bookkeeper-server clean install -DskipTests') - } - - // Test the package - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-pl bookkeeper-server test -Dtest="org.apache.bookkeeper.client.**"') - } - } - -} diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy deleted file mode 100644 index 0fa3ddfbfdb..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_precommit_integrationtests.groovy +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// This is the Java precommit which runs a maven install, and the current set of precommit tests. -freeStyleJob('bookkeeper_precommit_integrationtests') { - description('precommit integration test verification for pull requests of Apache BookKeeper.') - - // clean up the workspace before build - wrappers { preBuildCleanup() } - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - 'master', - 'JDK 1.8 (latest)', - 200, - 'ubuntu', - '${sha1}') - - throttleConcurrentBuilds { - // limit builds to 1 per node to avoid conflicts on building docker images - maxPerNode(1) - } - - // Sets that this is a PreCommit job. - common_job_properties.setPreCommit( - delegate, - 'Integration Tests (trigger via `run integration tests`)', - '.*(re)?run integration tests.*', - '.*\\[x\\] \\[skip integration tests\\].*') - - steps { - shell('.test-infra/scripts/pre-docker-tests.sh') - - shell('docker pull apachebookkeeper/bookkeeper-all-released-versions:latest') - - // Build everything - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-B clean install -Dstream -Pdocker') - properties(skipTests: true, interactiveMode: false) - } - - // Run metadata driver tests - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - rootPOM('metadata-drivers/pom.xml') - goals('-B test -DintegrationTests') - } - - // Run all integration tests - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - rootPOM('tests/pom.xml') - goals('-B test -Dstream -DintegrationTests') - } - - shell('.test-infra/scripts/post-docker-tests.sh') - } - - publishers { - archiveArtifacts { - allowEmpty(true) - pattern('**/target/container-logs/**') - pattern('docker.log') - } - archiveJunit('**/surefire-reports/TEST-*.xml') - } -} diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_java11.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_java11.groovy deleted file mode 100644 index cf0857c7e1e..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_precommit_java11.groovy +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// This is the Java precommit which runs a maven install, and the current set of precommit tests. -mavenJob('bookkeeper_precommit_pullrequest_java11') { - description('precommit verification for pull requests of Apache BookKeeper in Java 11.') - - // clean up the workspace before build - wrappers { preBuildCleanup() } - - // Temporary information gathering to see if full disks are causing the builds to flake - preBuildSteps { - shell("id") - shell("ulimit -a") - shell("pwd") - shell("df -Th") - shell("ps aux") - } - - // Execute concurrent builds if necessary. - concurrentBuild() - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - 'master', - 'JDK 11 (latest)', - 200, - 'ubuntu', - '${sha1}') - - // Sets that this is a PreCommit job. - common_job_properties.setPreCommit( - delegate, - 'Build (Java 11) (trigger via `rebuild java11`)', - '.*(re)?build java11.*', - '.*\\[x\\] \\[skip build java11\\].*') - - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - // Maven build project - // Spotbugs disabled on JDK11 due to https://github.com/spotbugs/spotbugs/issues/756 - goals('clean package -Dstream -DskipBookKeeperServerTests') -} diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy deleted file mode 100644 index dcb07c59c38..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_precommit_java8.groovy +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// This is the Java precommit which runs a maven install, and the current set of precommit tests. -mavenJob('bookkeeper_precommit_pullrequest_java8') { - description('precommit verification for pull requests of Apache BookKeeper in Java 8.') - - // clean up the workspace before build - wrappers { preBuildCleanup() } - - // Temporary information gathering to see if full disks are causing the builds to flake - preBuildSteps { - shell("id") - shell("ulimit -a") - shell("pwd") - shell("df -Th") - shell("ps aux") - } - - // Execute concurrent builds if necessary. - concurrentBuild() - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - 'master', - 'JDK 1.8 (latest)', - 200, - 'ubuntu', - '${sha1}') - - // Sets that this is a PreCommit job. - common_job_properties.setPreCommit( - delegate, - 'Build (Java 8) (trigger via `rebuild java8`)', - '.*(re)?build java8.*', - '.*\\[x\\] \\[skip build java8\\].*') - - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - // Maven build project - goals('clean package spotbugs:check -Dstream -DskipBookKeeperServerTests') -} diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy deleted file mode 100644 index 6e8bf4c38d0..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_precommit_remaining_tests.groovy +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// run tests except: -// - `org.apache.bookkeeper.client.**` -// - `org.apache.bookkeeper.bookie.**` -// - `org.apache.bookkeeper.replication.**` -// - `org.apache.bookkeeper.tls.**` -freeStyleJob('bookkeeper_precommit_remaining_tests') { - description('Run bookkeeper remaining tests in Java 8.') - - // clean up the workspace before build - wrappers { preBuildCleanup() } - - // Execute concurrent builds if necessary. - concurrentBuild() - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - 'master', - 'JDK 1.8 (latest)', - 200, - 'ubuntu', - '${sha1}') - - // Sets that this is a PreCommit job. - common_job_properties.setPreCommit( - delegate, - 'All Other Tests (trigger via `run bookkeeper-server remaining tests`)', - '.*(re)?run bookkeeper-server (remaining )?tests.*', - '.*\\[x\\] \\[skip bookkeeper-server (remaining )?tests\\].*', - true) - - steps { - // Temporary information gathering to see if full disks are causing the builds to flake - shell("id") - shell("ulimit -a") - shell("pwd") - shell("df -Th") - shell("ps aux") - - // Build everything - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-B -am -pl bookkeeper-server clean install -DskipTests') - } - - // Test the package - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-pl bookkeeper-server test -Dtest="!org.apache.bookkeeper.client.**,!org.apache.bookkeeper.bookie.**,!org.apache.bookkeeper.replication.**,!org.apache.bookkeeper.tls.**"') - } - } - -} diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy deleted file mode 100644 index a9f356d99c4..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_precommit_replication_tests.groovy +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// test `org.apache.bookkeeper.replication.**` -freeStyleJob('bookkeeper_precommit_replication_tests') { - description('Run bookkeeper replication tests in Java 8.') - - // clean up the workspace before build - wrappers { preBuildCleanup() } - - // Execute concurrent builds if necessary. - concurrentBuild() - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - 'master', - 'JDK 1.8 (latest)', - 200, - 'ubuntu', - '${sha1}') - - // Sets that this is a PreCommit job. - common_job_properties.setPreCommit( - delegate, - 'Replication Tests (trigger via `run bookkeeper-server replication tests`)', - '.*(re)?run bookkeeper-server (replication )?tests.*', - '.*\\[x\\] \\[skip bookkeeper-server (replication )?tests\\].*', - true) - - steps { - // Temporary information gathering to see if full disks are causing the builds to flake - shell("id") - shell("ulimit -a") - shell("pwd") - shell("df -Th") - shell("ps aux") - - // Build everything - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-B -am -pl bookkeeper-server clean install -DskipTests') - } - - // Test the package - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-pl bookkeeper-server test -Dtest="org.apache.bookkeeper.replication.**"') - } - } - -} diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy deleted file mode 100644 index 62549d5cce8..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_precommit_tls_tests.groovy +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// test `org.apache.bookkeeper.tls.**` -freeStyleJob('bookkeeper_precommit_tls_tests') { - description('Run bookkeeper tls tests in Java 8.') - - // clean up the workspace before build - wrappers { preBuildCleanup() } - - // Execute concurrent builds if necessary. - concurrentBuild() - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - 'master', - 'JDK 1.8 (latest)', - 200, - 'ubuntu', - '${sha1}') - - // Sets that this is a PreCommit job. - common_job_properties.setPreCommit( - delegate, - 'TLS Tests (trigger via `run bookkeeper-server tls tests`)', - '.*(re)?run bookkeeper-server (tls )?tests.*', - '.*\\[x\\] \\[skip bookkeeper-server (tls )?tests\\].*', - true) - - steps { - // Temporary information gathering to see if full disks are causing the builds to flake - shell("id") - shell("ulimit -a") - shell("pwd") - shell("df -Th") - shell("ps aux") - - // Build everything - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-B -am -pl bookkeeper-server clean install -DskipTests') - } - - // Test the package - maven { - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - goals('-pl bookkeeper-server test -Dtest="org.apache.bookkeeper.tls.**"') - } - } - -} diff --git a/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy b/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy deleted file mode 100644 index e7e75aada62..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_precommit_validation.groovy +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// This is the Java precommit validation job that validates pull requests (e.g. checkstyle) -mavenJob('bookkeeper_precommit_pullrequest_validation') { - description('precommit validation for pull requests of Apache BookKeeper.') - - // clean up the workspace before build - wrappers { preBuildCleanup() } - - // Execute concurrent builds if necessary. - concurrentBuild() - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - 'master', - 'JDK 1.8 (latest)', - 200, - 'ubuntu', - '${sha1}') - - // Sets that this is a PreCommit job. - common_job_properties.setPreCommit( - delegate, - 'PR Validation (trigger via `run pr validation`)', - '.*(re)?run pr validation.*', - '', - true) - - // Set Maven parameters. - common_job_properties.setMavenConfig(delegate) - - // Maven build project - goals('clean apache-rat:check checkstyle:check package -Ddistributedlog -Dstream -DskipTests') -} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java index 8949e12bb08..5e4ea24e9ee 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationManager.java @@ -18,6 +18,8 @@ */ package org.apache.bookkeeper.discover; +import static org.junit.Assert.assertNotNull; + import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.test.ZooKeeperCluster; @@ -27,9 +29,6 @@ import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.assertTrue; - - /** * Unit test of {@link RegistrationManager}. */ @@ -50,14 +49,14 @@ public void teardown() throws Exception { } @Test - public void testPrepareFormat () throws Exception{ + public void testPrepareFormat () throws Exception { try { ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); conf.setMetadataServiceUri("zk+hierarchical://localhost:2181/test/ledgers"); zkc = localZkServer.getZooKeeperClient(); - ZKRegistrationManager zkRegistrationManager = new ZKRegistrationManager(conf, zkc,() -> {} ); + ZKRegistrationManager zkRegistrationManager = new ZKRegistrationManager(conf, zkc, () -> {}); zkRegistrationManager.prepareFormat(); - assertTrue(zkc.exists("/test/ledgers",false) != null); + assertNotNull(zkc.exists("/test/ledgers", false)); } finally { if (zkc != null) { zkc.close(); From 67ebd70eadf9deb867c3c369c58694e888ba4358 Mon Sep 17 00:00:00 2001 From: Alessandro Luccaroni Date: Fri, 31 Jan 2020 05:02:14 +0100 Subject: [PATCH 0379/1642] Website: update Gemfile dependencies Updating Gemfile dependencies to try fix Travis (see https://github.com/apache/bookkeeper/issues/2252) Also fixing https://www.cvedetails.com/cve/CVE-2018-17567/ in the meantime You can find the staging website https://aluccaroni.github.io/bookkeeper-staging-site Reviewers: Enrico Olivelli , Sijie Guo This closes #2256 from aluccaroni/master --- site/Gemfile | 8 +++--- site/Gemfile.lock | 66 +++++++++++++++++++++++------------------------ site/Makefile | 1 + 3 files changed, 37 insertions(+), 38 deletions(-) diff --git a/site/Gemfile b/site/Gemfile index cc4f51e7f0a..16ae2ae996a 100644 --- a/site/Gemfile +++ b/site/Gemfile @@ -1,7 +1,7 @@ source 'https://rubygems.org' -ruby '>=2.3.1' +ruby '>=2.4.0' -gem 'jekyll', '3.7.0' -gem 'nokogiri', '1.8.2' -gem 'jekyll-toc', '0.2.1' +gem 'jekyll', '3.8.6' +gem 'nokogiri', '1.10.7' +gem 'jekyll-toc', '0.12.2' diff --git a/site/Gemfile.lock b/site/Gemfile.lock index 98548973022..1daa761fdac 100644 --- a/site/Gemfile.lock +++ b/site/Gemfile.lock @@ -1,20 +1,20 @@ GEM remote: https://rubygems.org/ specs: - addressable (2.5.2) - public_suffix (>= 2.0.2, < 4.0) + addressable (2.7.0) + public_suffix (>= 2.0.2, < 5.0) colorator (1.1.0) - concurrent-ruby (1.0.5) + concurrent-ruby (1.1.5) em-websocket (0.5.1) eventmachine (>= 0.12.9) http_parser.rb (~> 0.6.0) - eventmachine (1.2.5) - ffi (1.9.18) + eventmachine (1.2.7) + ffi (1.12.1) forwardable-extended (2.6.0) http_parser.rb (0.6.0) - i18n (0.9.3) + i18n (0.9.5) concurrent-ruby (~> 1.0) - jekyll (3.7.0) + jekyll (3.8.6) addressable (~> 2.4) colorator (~> 1.0) em-websocket (~> 0.5) @@ -27,32 +27,30 @@ GEM pathutil (~> 0.9) rouge (>= 1.7, < 4) safe_yaml (~> 1.0) - jekyll-sass-converter (1.5.1) + jekyll-sass-converter (1.5.2) sass (~> 3.4) - jekyll-toc (0.2.1) - nokogiri (~> 1.6) - jekyll-watch (2.0.0) + jekyll-toc (0.12.2) + nokogiri (~> 1.9) + jekyll-watch (2.2.1) listen (~> 3.0) - kramdown (1.16.2) - liquid (4.0.0) - listen (3.1.5) - rb-fsevent (~> 0.9, >= 0.9.4) - rb-inotify (~> 0.9, >= 0.9.7) - ruby_dep (~> 1.2) + kramdown (1.17.0) + liquid (4.0.3) + listen (3.2.1) + rb-fsevent (~> 0.10, >= 0.10.3) + rb-inotify (~> 0.9, >= 0.9.10) mercenary (0.3.6) - mini_portile2 (2.3.0) - nokogiri (1.8.2) - mini_portile2 (~> 2.3.0) - pathutil (0.16.1) + mini_portile2 (2.4.0) + nokogiri (1.10.7) + mini_portile2 (~> 2.4.0) + pathutil (0.16.2) forwardable-extended (~> 2.6) - public_suffix (3.0.1) - rb-fsevent (0.10.2) - rb-inotify (0.9.10) - ffi (>= 0.5.0, < 2) - rouge (3.1.1) - ruby_dep (1.5.0) - safe_yaml (1.0.4) - sass (3.5.5) + public_suffix (4.0.3) + rb-fsevent (0.10.3) + rb-inotify (0.10.1) + ffi (~> 1.0) + rouge (3.15.0) + safe_yaml (1.0.5) + sass (3.7.4) sass-listen (~> 4.0.0) sass-listen (4.0.0) rb-fsevent (~> 0.9, >= 0.9.4) @@ -62,12 +60,12 @@ PLATFORMS ruby DEPENDENCIES - jekyll (= 3.7.0) - jekyll-toc (= 0.2.1) - nokogiri (= 1.8.2) + jekyll (= 3.8.6) + jekyll-toc (= 0.12.2) + nokogiri (= 1.10.7) RUBY VERSION - ruby 2.4.1p111 + ruby 2.4.0p0 BUNDLED WITH - 1.16.2 + 2.1.4 diff --git a/site/Makefile b/site/Makefile index 5c20bb697c9..37af9b95eeb 100644 --- a/site/Makefile +++ b/site/Makefile @@ -21,6 +21,7 @@ build: clean --config _config.yml apache: clean + gem install bundler JEKYLL_ENV=production $(JEKYLL) build \ --config _config.yml,_config.apache.yml From ea0d19bdd421ca762c1a0dd52a4b233232af1078 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 31 Jan 2020 04:26:16 -0800 Subject: [PATCH 0380/1642] [CODE-STYLE] fix build failure for check-style Descriptions of the changes in this PR: ### Motivation PR Validation build is failing with below exception: ``` [ERROR] src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java:[35] (imports) ImportOrder: Import org.apache.bookkeeper.stream.storage.StorageConstants.getSegmentsRootPath appears after other imports that it should precede [ERROR] Failed to execute goal org.apache.maven.plugins:maven-checkstyle-plugin:3.0.0:check (default-cli) on project stream-storage-service-impl: You have 1 Checkstyle violation. -> [Help 1] ``` Reviewers: Enrico Olivelli This closes #2258 from rdhabalia/checkZk --- .../stream/storage/impl/cluster/ZkClusterInitializer.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java index 081bd92c141..3668691519a 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.stream.storage.impl.cluster; import static org.apache.bookkeeper.stream.storage.StorageConstants.ZK_METADATA_ROOT_PATH; - +import static org.apache.bookkeeper.stream.storage.StorageConstants.getSegmentsRootPath; import com.google.common.base.Strings; import java.net.URI; import java.util.Optional; @@ -32,7 +32,6 @@ import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.zookeeper.KeeperException; -import static org.apache.bookkeeper.stream.storage.StorageConstants.getSegmentsRootPath; /** * ZooKeeper Based Cluster Initializer. From 76f51dcfbc6c2e61163167b5ebc0c8f38473c34e Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 10 Feb 2020 08:13:35 -0800 Subject: [PATCH 0381/1642] Remove travis Descriptions of the changes in this PR: ### Motivation The CI is moved to github actions. ### Changes Delete travis configuration. Reviewers: Enrico Olivelli This closes #2259 from sijie/remove_travis_ci --- .travis.yml | 75 ------------------------------- .travis_scripts/before_install.sh | 40 ----------------- .travis_scripts/build.sh | 40 ----------------- 3 files changed, 155 deletions(-) delete mode 100644 .travis.yml delete mode 100755 .travis_scripts/before_install.sh delete mode 100755 .travis_scripts/build.sh diff --git a/.travis.yml b/.travis.yml deleted file mode 100644 index 4049992b5bc..00000000000 --- a/.travis.yml +++ /dev/null @@ -1,75 +0,0 @@ -# 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. -language: java - -# blacklist -branches: - except: - - asf-site - - gh-pages - -services: - - docker - -matrix: - include: - - os: osx - osx_image: xcode8 - - os: osx - osx_image: xcode9.2 - - os: linux - env: CUSTOM_JDK="oraclejdk8" - - os: linux - env: CUSTOM_JDK="oraclejdk11" - -before_install: -- | - echo "MAVEN_OPTS='-Xmx3072m -XX:MaxPermSize=512m'" > ~/.mavenrc - if [ "$TRAVIS_OS_NAME" == "osx" ]; then - export JAVA_HOME=$(/usr/libexec/java_home); - echo "Forcing Maven 3.6.3 on osx as Travis in January 2020 still bundles 3.3.9" - wget https://archive.apache.org/dist/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.zip - unzip -qq apache-maven-3.6.3-bin.zip -d $PWD/.. - export M2_HOME=$PWD/../apache-maven-3.6.3 - export PATH=$M2_HOME/bin:$PATH - fi - if [ "$TRAVIS_OS_NAME" == "linux" ]; then - jdk_switcher use "$CUSTOM_JDK"; - fi - echo "TRAVIS_PULL_REQUEST=${TRAVIS_PULL_REQUEST}" - if [ "$TRAVIS_PULL_REQUEST" == "false" ]; then - export DLOG_MODIFIED="true" - echo "Enable testing distributedlog modules since they are not pull requests." - else - if [ `git diff --name-only $TRAVIS_COMMIT_RANGE | grep "^stream\/distributedlog" | wc -l` -gt 0 ]; then - export DLOG_MODIFIED="true" - echo "Enable testing distributedlog modules if this pull request modifies files under directory 'stream/distributedlog'." - fi - if [ `git diff --name-only $TRAVIS_COMMIT_RANGE | grep "^site\/" | wc -l` -gt 0 ]; then - # building the website to ensure the changes don't break - export WEBSITE_MODIFIED="true" - echo "Enable building website modules if this pull request modifies files under directory 'site'." - fi - fi - -install: true - -script: .travis_scripts/build.sh - -cache: - directories: - - $HOME/.m2 - - $HOME/apache-maven-3.6.3 diff --git a/.travis_scripts/before_install.sh b/.travis_scripts/before_install.sh deleted file mode 100755 index b5b141f1c3f..00000000000 --- a/.travis_scripts/before_install.sh +++ /dev/null @@ -1,40 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -set -ev - -echo "MAVEN_OPTS='-Xmx3072m -XX:MaxPermSize=512m'" > ~/.mavenrc -if [ "$TRAVIS_OS_NAME" == "osx" ]; then - export JAVA_HOME=$(/usr/libexec/java_home); -fi -echo "TRAVIS_PULL_REQUEST=${TRAVIS_PULL_REQUEST}" -if [ "$TRAVIS_PULL_REQUEST" == "false" ]; then - export DLOG_MODIFIED="true" - echo "Enable testing distributedlog modules since they are not pull requests." -else - if [ `git diff --name-only $TRAVIS_COMMIT_RANGE | grep "^stream\/distributedlog" | wc -l` -gt 0 ]; then - export DLOG_MODIFIED="true" - echo "Enable testing distributedlog modules if this pull request modifies files under directory 'stream/distributedlog'." - fi - if [ `git diff --name-only $TRAVIS_COMMIT_RANGE | grep "^site\/" | wc -l` -gt 0 ]; then - # building the website to ensure the changes don't break - export WEBSITE_MODIFIED="true" - echo "Enable building website modules if this pull request modifies files under directory 'site'." - fi -fi diff --git a/.travis_scripts/build.sh b/.travis_scripts/build.sh deleted file mode 100755 index ee0157f608b..00000000000 --- a/.travis_scripts/build.sh +++ /dev/null @@ -1,40 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -set -ev - -BINDIR=`dirname "$0"` -BK_HOME=`cd $BINDIR/..;pwd` - -mvn -v - -mvn --batch-mode clean apache-rat:check compile spotbugs:check install -DskipTests -Dstream -$BK_HOME/dev/check-binary-license ./bookkeeper-dist/all/target/bookkeeper-all-*-bin.tar.gz; -$BK_HOME/dev/check-binary-license ./bookkeeper-dist/server/target/bookkeeper-server-*-bin.tar.gz; -$BK_HOME/dev/check-binary-license ./bookkeeper-dist/bkctl/target/bkctl-*-bin.tar.gz; -if [ "$DLOG_MODIFIED" == "true" ]; then - cd $BK_HOME/stream/distributedlog - mvn --batch-mode clean package -Ddistributedlog -fi -if [ "$TRAVIS_OS_NAME" == "linux" ] && [ "$WEBSITE_MODIFIED" == "true" ]; then - cd $BK_HOME/site - make clean - # run the docker image to build the website - ./docker/ci.sh -fi From 9deec39db2041b50f495f64646599dc2fa06f7cf Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 13 Feb 2020 16:40:39 +0100 Subject: [PATCH 0382/1642] BP-38 Publish Bookie Service Info on Metadata Service This is the BP page for BP-38 Publish Bookie Service Info on Metadata Service Master Issue: #2215 Reviewers: [Ivan Kelly , Sijie Guo , Flavio Junqueira This closes #2214 from eolivelli/fix/bp-disc --- site/bps/BP-38-bookie-endpoint-discovery.md | 154 ++++++++++++++++++++ 1 file changed, 154 insertions(+) create mode 100644 site/bps/BP-38-bookie-endpoint-discovery.md diff --git a/site/bps/BP-38-bookie-endpoint-discovery.md b/site/bps/BP-38-bookie-endpoint-discovery.md new file mode 100644 index 00000000000..80ad072d8da --- /dev/null +++ b/site/bps/BP-38-bookie-endpoint-discovery.md @@ -0,0 +1,154 @@ +--- +title: "BP-38: Publish Bookie Service Info on Metadata Service" +issue: https://github.com/apache/bookkeeper/2215 +state: 'Accepted' +release: "4.11.0" +--- + +### Motivation + +Bookie server exposes several services and some of them are optional: the binary BookKeeper protocol endpoint, the HTTP service, the StreamStorage service, a Metrics endpoint. + +Currently (in BookKeeper 4.10) the client can only discover the main Bookie endpoint: +the main BookKeeper binary RPC service. +Discovery of the TCP address is implicit, because the *id* of the bookie is made of the same host:port that point to the TCP address of the Bookie service. + +With this proposal we are now introducing a way for the Bookie to advertise the services it exposes, basically the Bookie will be able to store on the Metadata Service a structure that describes the list of *available services*. + +We will also allow to publish a set of additional unstructured properties in form of a key-value pair that will be useful for futher implementations. + +This information will be also useful for Monitoring and Management services as it will enable full discovery of the capabilities of all of the Bookies in a cluster just by having read access to the Metadata Service. + +### Public Interfaces + +On the Registration API, we introduce a new data structure that describes the services +exposed by a Bookie: + +``` +interface BookieServiceInfo { + class Endpoint { + String name; + String hostname; + int port; + String protocol; // "bookie-rpc", "http", "https".... + } + List endpoints; + Map properties; // additional properties +} + +``` + +In RegistrationClient interface we expose a new method: + +``` +CompletableFuture> getBookieServiceInfo(String bookieId) +``` + +The client can derive bookieId from a BookieSocketAddress. He can access the list of available bookies using **RegistrationClient#getAllBookies()** and then use this new method to get the details of the services exposed by the Bookie. + +On the Bookie side we change the RegistrationManager interface in order to let the Bookie +advertise the services: + +in RegistrationManager class the **registerBookie** method +``` +void registerBookie(String bookieId, boolean readOnly) +``` + +becomes + +``` +void registerBookie(String bookieId, boolean readOnly, BookieServiceInfo bookieServieInfo) +``` + +It will be up to the implementation of RegistrationManager and RegistrationClient to serialize +the BookieServiceInfo structure. + +For the ZooKeeper based implementation we are going to store such information in Protobuf binary format, as currently this is the format used for every other kind of metadata (the example here is in JSON-like for readability purposes): + +``` +{ + "endpoints": [ + { + "name": "bookie", + "hostname": "localhost", + "port": 3181, + "protocol": "bookie-rpc" + }, + { + "name": "bookie-http-server", + "hostname": "localhost", + "port": 8080, + "protocol": "http" + } + ], + properties { + "property": "value", + "property2": "value2" + } +} +``` +Such information will be stored inside the '/REGISTRATIONPATH/available' znode (or /REGISTRATIONPATH/available/readonly' in case of readonly bookie), these paths are the same used in 4.10, but in 4.10 we are writing empty znodes. + +The rationale around this choice is that the client is already using these znodes in order to discover available bookies services. + +It is possible that such endpoint information change during the lifetime of a Bookie, like after rebooting a machine and the machine gets a new network address. + +It is out of the scope of this proposal to change semantics of ledger metadata, in which we are currently storing directly the network address of the bookies, but with this proposal we are preparing the way to have an indirection and separate the concept of Bookie ID from the Network address. + +**Endpoint structure** +- **name**: this is an id for the service type, like 'bookie' +- **hostname**: the hostname (or a raw IP address) +- **port**: the TCP port (int) +- **protocol**: the type of service (bookie-rpc, http, https) + +**Well known additional properties** +- **autorecovery.enabled**: 'true' case of the presence of the auto recovery daemon + +The GRPC service would use this mechanism as well. + +#### Which kind of properties should be stored in BookieServiceInfo ? + +We should store here only the minimal set of information useful to reach the bookie in an efficient way. +So we are not storing on the Metadata Service information about the internal state of the server: if you know the address of an HTTP endpoint you can use the REST API to query the Bookie for its state. + +These properties may change during the lifetime of a Bookie, think about a configuration (change network address) or a dynamic assigned DNS name. + +It is better not to expose the version of the Bookie, if the client wants to use particular features of the Bookie this should be implemented on the protocol itself, not just by using the version. The version of the Bookie could be exposed on the HTTP endpoint. + +### Proposed Changes + +See the 'Public interfaces' section. + +### Compatibility, Deprecation, and Migration Plan + +The proposed change will be compatible with all existing clients. + +In case a new client is reading an empty znode it will assume a default configuration with a single 'bookie-rpc' endpoint, like in this example: + +{ + "endpoints": [ + { + "name": "bookie", + "hostname": "hostname-from-bookieid", + "port": port-from-bookieid, + "protocol": "bookie-rpc" + } + ] +} + +This information is enough in order to use the RPC service. + +### Test Plan + +New unit tests will be added to cover all of the code changes. +No need for additional integration tests. + +### Rejected Alternatives + +#### Adding a new set of znodes +For the ZooKeeper implementation we are not introducing a new znode to store BookieServiceInfo. Adding such new node will increase complexity and the usage of resources on ZooKeeper. + +#### Storing information inside the Cookie +The *Cookie* stores information about the *identity* of the bookie and it is expected not to change. +It is exceptional to rewrite the Cookie, like when adding a new data directory. +In some environments it is common to have a new network address after a restart or to change the configuration and enable a new service or feature, and you cannot rewrite the Cookie at each restart: by design every change to the Cookie should be manual or controlled by an external entity other than the Bookie itself. From 23dcaaed9b8724a4e4cd34851858ba30c4396bf8 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 13 Feb 2020 17:44:00 +0100 Subject: [PATCH 0383/1642] Add BP-38 to list of approved releases --- site/community/bookkeeper_proposals.md | 1 + 1 file changed, 1 insertion(+) diff --git a/site/community/bookkeeper_proposals.md b/site/community/bookkeeper_proposals.md index c15e1d26d5c..db836e91926 100644 --- a/site/community/bookkeeper_proposals.md +++ b/site/community/bookkeeper_proposals.md @@ -109,6 +109,7 @@ Proposal | State [BP-35: 128 bits support](../../bps/BP-35-128-bits-support) | Accepted [BP-36: Stats documentation annotation](../../bps/BP-36-stats-documentation-annotation) | Accepted [BP-37: Improve configuration management for better documentation](../../bps/BP-37-conf-documentation) | Accepted +[BP-38: Publish Bookie Service Info on Metadata Service](../../bps/BP-38-bookie-endpoint-discovery) | Accepted ### Adopted From a2783ffbabb1c3cbff59c5dccb4930a5418bb8aa Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sun, 16 Feb 2020 10:14:02 +0100 Subject: [PATCH 0384/1642] Revert "bin/bookkeeper shell supports the -conf" This reverts commit 163db4496d50cae198b80c9d88c14579f6d2e9f3. Descriptions of the changes in this PR: Commit 163db4496d50cae198b80c9d88c14579f6d2e9f3 broke "bookkeeper shell" command and integration tests are no more able to work Reviewers: Sijie Guo This closes #2265 from eolivelli/fix/revert-bad-fix --- bin/bookkeeper | 40 +++++++++++----------------------------- 1 file changed, 11 insertions(+), 29 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index 4b55d1eb222..ddaae2286e8 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -108,30 +108,16 @@ shift LOCALBOOKIES_CONFIG_DIR="${LOCALBOOKIES_CONFIG_DIR:-/tmp/localbookies-config}" if [ ${COMMAND} == "shell" ]; then DEFAULT_LOG_CONF=${BK_HOME}/conf/log4j.shell.properties - LOCALBOOKIE=0 - while [ $# -gt 0 ] - do - opt="$1" - case $opt in - (-conf) - BOOKIE_CONF="$2" - shift 2 - ;; - (-localbookie) - LOCALBOOKIE=1 - if [[ $2 == *:* ]]; - then - LOCALBOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/$2.conf - shift 2 - else - LOCALBOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/baseconf.conf - shift - fi - ;; - (*) - ;; - esac - done + if [[ $1 == "-localbookie" ]]; then + if [[ $2 == *:* ]]; + then + BOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/$2.conf + shift 2 + else + BOOKIE_CONF=${LOCALBOOKIES_CONFIG_DIR}/baseconf.conf + shift + fi + fi fi if [ -z "$BOOKIE_ZK_CONF" ]; then @@ -186,11 +172,7 @@ elif [ $COMMAND == "zookeeper" ]; then exec $JAVA $OPTS -Dbookkeeper.log.file=$BOOKIE_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $BOOKIE_ZK_CONF $@ elif [ ${COMMAND} == "shell" ]; then ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}" - if [ ${LOCALBOOKIE} == 1 ]; then - exec ${JAVA} ${OPTS} ${ENTRY_FORMATTER_ARG} org.apache.bookkeeper.bookie.BookieShell -conf ${LOCALBOOKIE_CONF} $@ - else - exec ${JAVA} ${OPTS} ${ENTRY_FORMATTER_ARG} org.apache.bookkeeper.bookie.BookieShell -conf ${BOOKIE_CONF} $@ - fi + exec ${JAVA} ${OPTS} ${ENTRY_FORMATTER_ARG} org.apache.bookkeeper.bookie.BookieShell -conf ${BOOKIE_CONF} $@ elif [ ${COMMAND} == "help" ]; then bookkeeper_help; else From e6d4709825b1a3e7246fbd94913b29f9211b131d Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 17 Feb 2020 09:12:37 +0100 Subject: [PATCH 0385/1642] Fix method order in order to make the execution of TestEntryLog predictable ### Motivation TestEntyLog reuses the entryLogger variable across test methods execution, resulting in an unpredictable execution. We can at least make the test work consistently across environments using JUnit built-in FixMethodOrder annotation ### Changes FixMethodOrder(MethodSorters.NAME_ASCENDING) Fixes #2260 Reviewers: Atri Sharma This closes #2266 from eolivelli/fix/flaky --- .../test/java/org/apache/bookkeeper/bookie/TestEntryLog.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java index 836fdf25cb2..fd35204b001 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java @@ -67,13 +67,16 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.FixMethodOrder; import org.junit.Test; +import org.junit.runners.MethodSorters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Tests for EntryLog. */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) public class TestEntryLog { private static final Logger LOG = LoggerFactory.getLogger(TestEntryLog.class); From 5b14ab6420e9efae8463a7cb0ef7e4e4da1c605a Mon Sep 17 00:00:00 2001 From: dmercuriali Date: Mon, 17 Feb 2020 14:03:15 +0100 Subject: [PATCH 0386/1642] ISSUE #2218: Write customMetadata on ledgers created by DistributedLog ### Changes Enable DistributedLog to attach custom metadata to underlying ledgers when writing logs. The user can pass a LedgerMetadata object to DistributedLogManager#openLogWriter and DistributedLogManager#openAsyncLogWriter. The LedgerMetadata object is then used by the Allocator backing the BKLogWriteHandler every time it must allocate a new ledger. Master Issue: #2218 Reviewers: Enrico Olivelli , Sijie Guo This closes #2254 from dmercuriali/2218-distributedlog-ledger-custom-metadata, closes #2218 --- .../distributedlog/BKAbstractLogWriter.java | 8 ++- .../BKDistributedLogManager.java | 30 ++++++-- .../distributedlog/BookKeeperClient.java | 6 +- .../api/DistributedLogManager.java | 25 +++++++ .../distributedlog/bk/LedgerMetadata.java | 72 +++++++++++++++++++ .../bk/SimpleLedgerAllocator.java | 50 +++++++++++-- .../logsegment/BKLogSegmentEntryStore.java | 17 ++++- .../logsegment/LogSegmentEntryStore.java | 15 ++++ .../TestBKDistributedLogManager.java | 53 ++++++++++++++ .../bk/TestLedgerAllocator.java | 33 ++++++++- 10 files changed, 292 insertions(+), 17 deletions(-) create mode 100644 stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerMetadata.java diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java index 275c8d7549b..cc3cb96b23d 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java @@ -27,6 +27,7 @@ import java.util.function.Function; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.distributedlog.bk.LedgerMetadata; import org.apache.distributedlog.common.util.PermitManager; import org.apache.distributedlog.config.DynamicDistributedLogConfiguration; import org.apache.distributedlog.exceptions.AlreadyClosedException; @@ -94,6 +95,11 @@ protected BKLogWriteHandler getWriteHandler() throws IOException { protected BKLogWriteHandler createAndCacheWriteHandler() throws IOException { + return createAndCacheWriteHandler(null); + } + + protected BKLogWriteHandler createAndCacheWriteHandler(LedgerMetadata ledgerMetadata) + throws IOException { synchronized (this) { if (writeHandler != null) { return writeHandler; @@ -102,7 +108,7 @@ protected BKLogWriteHandler createAndCacheWriteHandler() // This code path will be executed when the handler is not set or has been closed // due to forceRecovery during testing BKLogWriteHandler newHandler = - Utils.ioResult(bkDistributedLogManager.asyncCreateWriteHandler(false)); + Utils.ioResult(bkDistributedLogManager.asyncCreateWriteHandler(false, ledgerMetadata)); boolean success = false; try { synchronized (this) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java index 645d6d49e7d..b21b210bc81 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java @@ -43,6 +43,7 @@ import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; import org.apache.distributedlog.api.subscription.SubscriptionsStore; +import org.apache.distributedlog.bk.LedgerMetadata; import org.apache.distributedlog.callback.LogSegmentListener; import org.apache.distributedlog.common.stats.BroadCastStatsLogger; import org.apache.distributedlog.common.util.PermitLimiter; @@ -335,10 +336,16 @@ synchronized BKLogReadHandler createReadHandler(Optional subscriberId, public BKLogWriteHandler createWriteHandler(boolean lockHandler) throws IOException { - return Utils.ioResult(asyncCreateWriteHandler(lockHandler)); + return createWriteHandler(lockHandler, null); } - CompletableFuture asyncCreateWriteHandler(final boolean lockHandler) { + public BKLogWriteHandler createWriteHandler(boolean lockHandler, LedgerMetadata ledgerMetadata) + throws IOException { + return Utils.ioResult(asyncCreateWriteHandler(lockHandler, ledgerMetadata)); + } + + CompletableFuture asyncCreateWriteHandler(final boolean lockHandler, + LedgerMetadata ledgerMetadata) { // Fetching Log Metadata (create if not exists) return driver.getLogStreamMetadataStore(WRITER).getLog( uri, @@ -347,12 +354,13 @@ CompletableFuture asyncCreateWriteHandler(final boolean lockH conf.getCreateStreamIfNotExists() ).thenCompose(logMetadata -> { CompletableFuture createPromise = new CompletableFuture(); - createWriteHandler(logMetadata, lockHandler, createPromise); + createWriteHandler(logMetadata, ledgerMetadata, lockHandler, createPromise); return createPromise; }); } private void createWriteHandler(LogMetadataForWriter logMetadata, + LedgerMetadata ledgerMetadata, boolean lockHandler, final CompletableFuture createPromise) { // Build the locks @@ -366,7 +374,7 @@ private void createWriteHandler(LogMetadataForWriter logMetadata, Allocator segmentAllocator; try { segmentAllocator = driver.getLogSegmentEntryStore(WRITER) - .newLogSegmentAllocator(logMetadata, dynConf); + .newLogSegmentAllocator(logMetadata, dynConf, ledgerMetadata); } catch (IOException ioe) { FutureUtils.completeExceptionally(createPromise, ioe); return; @@ -479,11 +487,16 @@ public BKSyncLogWriter startLogSegmentNonPartitioned() throws IOException { @Override public BKSyncLogWriter openLogWriter() throws IOException { + return openLogWriter(null); + } + + @Override + public BKSyncLogWriter openLogWriter(LedgerMetadata ledgerMetadata) throws IOException { checkClosedOrInError("startLogSegmentNonPartitioned"); BKSyncLogWriter writer = new BKSyncLogWriter(conf, dynConf, this); boolean success = false; try { - writer.createAndCacheWriteHandler(); + writer.createAndCacheWriteHandler(ledgerMetadata); BKLogWriteHandler writeHandler = writer.getWriteHandler(); Utils.ioResult(writeHandler.lockHandler()); success = true; @@ -507,6 +520,11 @@ public BKAsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException @Override public CompletableFuture openAsyncLogWriter() { + return openAsyncLogWriter(null); + } + + @Override + public CompletableFuture openAsyncLogWriter(LedgerMetadata ledgerMetadata) { try { checkClosedOrInError("startLogSegmentNonPartitioned"); } catch (AlreadyClosedException e) { @@ -516,7 +534,7 @@ public CompletableFuture openAsyncLogWriter() { CompletableFuture createWriteHandleFuture; synchronized (this) { // 1. create the locked write handler - createWriteHandleFuture = asyncCreateWriteHandler(true); + createWriteHandleFuture = asyncCreateWriteHandler(true, ledgerMetadata); } return createWriteHandleFuture.thenCompose(writeHandler -> { final BKAsyncLogWriter writer; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java index 1f69633bd18..a03530da4ec 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java @@ -41,6 +41,7 @@ import org.apache.commons.configuration.ConfigurationException; import org.apache.distributedlog.ZooKeeperClient.Credentials; import org.apache.distributedlog.ZooKeeperClient.DigestCredentials; +import org.apache.distributedlog.bk.LedgerMetadata; import org.apache.distributedlog.exceptions.AlreadyClosedException; import org.apache.distributedlog.exceptions.DLInterruptedException; import org.apache.distributedlog.net.NetUtils; @@ -203,7 +204,8 @@ public synchronized BookKeeper get() throws IOException { // Util functions public CompletableFuture createLedger(int ensembleSize, int writeQuorumSize, - int ackQuorumSize) { + int ackQuorumSize, + LedgerMetadata ledgerMetadata) { BookKeeper bk; try { bk = get(); @@ -221,7 +223,7 @@ public void createComplete(int rc, LedgerHandle lh, Object ctx) { promise.completeExceptionally(BKException.create(rc)); } } - }, null, Collections.emptyMap()); + }, null, ledgerMetadata == null ? Collections.emptyMap() : ledgerMetadata.getMetadata()); return promise; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java index c5b9e03fc27..236247afc38 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java @@ -23,12 +23,14 @@ import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.AppendOnlyStreamReader; import org.apache.distributedlog.AppendOnlyStreamWriter; import org.apache.distributedlog.DLSN; import org.apache.distributedlog.LogRecordWithDLSN; import org.apache.distributedlog.LogSegmentMetadata; import org.apache.distributedlog.api.subscription.SubscriptionsStore; +import org.apache.distributedlog.bk.LedgerMetadata; import org.apache.distributedlog.callback.LogSegmentListener; import org.apache.distributedlog.io.AsyncCloseable; import org.apache.distributedlog.namespace.NamespaceDriver; @@ -104,6 +106,17 @@ public interface DistributedLogManager extends AsyncCloseable, Closeable { */ CompletableFuture openAsyncLogWriter(); + /** + * Open async log writer to write records to the log stream. + * Provided metadata will be attached to the underlying BookKeeper ledgers. + * + * @param ledgerMetadata + * @return result represents the open result + */ + default CompletableFuture openAsyncLogWriter(LedgerMetadata ledgerMetadata) { + return FutureUtils.exception(new UnsupportedOperationException()); + } + /** * Open sync log writer to write records to the log stream. * @@ -112,6 +125,18 @@ public interface DistributedLogManager extends AsyncCloseable, Closeable { */ LogWriter openLogWriter() throws IOException; + /** + * Open sync log writer to write records to the log stream. + * Provided metadata will be attached to the underlying BookKeeper ledgers. + * + * @param ledgerMetadata + * @return sync log writer + * @throws IOException when fails to open a sync log writer. + */ + default LogWriter openLogWriter(LedgerMetadata ledgerMetadata) throws IOException { + throw new UnsupportedOperationException(); + } + /** * Begin writing to the log stream identified by the name. * diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerMetadata.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerMetadata.java new file mode 100644 index 00000000000..201adfdc2b1 --- /dev/null +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerMetadata.java @@ -0,0 +1,72 @@ +/* + * Copyright 2020 The Apache Software Foundation. + * + * Licensed 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.distributedlog.bk; + +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; + +/** + * Ledger metadata. + */ +public final class LedgerMetadata { + + private String application; + private String component; + private Map customMetadata; + + public void setApplication(String application) { + this.application = application; + } + + public void setComponent(String component) { + this.component = component; + } + + public void addCustomMetadata(String key, String value) { + if (key == null || "".equals(key.trim())) { + throw new IllegalArgumentException("Metadata key cant be empty"); + } + if (value == null || "".equals(value.trim())) { + throw new IllegalArgumentException("Metadata value cant be empty"); + } + + if (customMetadata == null) { + customMetadata = new HashMap<>(); + } + + customMetadata.put(key, value); + } + + public Map getMetadata() { + Map meta = new HashMap<>(); + if (application != null) { + meta.put("application", application.getBytes(StandardCharsets.UTF_8)); + } + if (component != null) { + meta.put("component", component.getBytes(StandardCharsets.UTF_8)); + } + + if (customMetadata != null) { + for (Map.Entry e : customMetadata.entrySet()) { + String value = e.getValue(); + meta.put(e.getKey(), value.getBytes(StandardCharsets.UTF_8)); + } + } + + return meta; + } +} diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java index 0115a6ee782..e866482d717 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java @@ -103,6 +103,8 @@ public ConcurrentObtainException(Phase phase, String msg) { // Allocated Ledger LedgerHandle allocatedLh = null; + LedgerMetadata ledgerMetadata; + CompletableFuture closeFuture = null; final LinkedList> ledgerDeletions = new LinkedList>(); @@ -159,14 +161,23 @@ public static CompletableFuture of(final String allocateP final QuorumConfigProvider quorumConfigProvider, final ZooKeeperClient zkc, final BookKeeperClient bkc) { + return SimpleLedgerAllocator.of(allocatePath, allocationData, quorumConfigProvider, zkc, bkc, null); + } + + public static CompletableFuture of(final String allocatePath, + final Versioned allocationData, + final QuorumConfigProvider quorumConfigProvider, + final ZooKeeperClient zkc, + final BookKeeperClient bkc, + final LedgerMetadata ledgerMetadata) { if (null != allocationData && null != allocationData.getValue() && null != allocationData.getVersion()) { return FutureUtils.value(new SimpleLedgerAllocator(allocatePath, allocationData, - quorumConfigProvider, zkc, bkc)); + quorumConfigProvider, zkc, bkc, ledgerMetadata)); } return getAndCreateAllocationData(allocatePath, zkc) .thenApply(allocationData1 -> new SimpleLedgerAllocator(allocatePath, allocationData1, - quorumConfigProvider, zkc, bkc)); + quorumConfigProvider, zkc, bkc, ledgerMetadata)); } /** @@ -188,10 +199,36 @@ public SimpleLedgerAllocator(String allocatePath, QuorumConfigProvider quorumConfigProvider, ZooKeeperClient zkc, BookKeeperClient bkc) { + this(allocatePath, allocationData, quorumConfigProvider, zkc, bkc, null); + } + + /** + * Construct a ledger allocator. + * + * @param allocatePath + * znode path to store the allocated ledger. + * @param allocationData + * allocation data. + * @param quorumConfigProvider + * Quorum configuration provider. + * @param zkc + * zookeeper client. + * @param bkc + * bookkeeper client. + * @param ledgerMetadata + * metadata to attach to allocated ledgers + */ + public SimpleLedgerAllocator(String allocatePath, + Versioned allocationData, + QuorumConfigProvider quorumConfigProvider, + ZooKeeperClient zkc, + BookKeeperClient bkc, + LedgerMetadata ledgerMetadata) { this.zkc = zkc; this.bkc = bkc; this.allocatePath = allocatePath; this.quorumConfigProvider = quorumConfigProvider; + this.ledgerMetadata = ledgerMetadata; initialize(allocationData); } @@ -231,7 +268,7 @@ public synchronized void allocate() throws IOException { } if (Phase.HANDED_OVER == phase) { // issue an allocate request when ledger is already handed over. - allocateLedger(); + allocateLedger(ledgerMetadata); } } @@ -318,6 +355,10 @@ private synchronized void setPhase(Phase phase) { } private synchronized void allocateLedger() { + allocateLedger(null); + } + + private synchronized void allocateLedger(LedgerMetadata ledgerMetadata) { // make sure previous allocation is already handed over. if (Phase.HANDED_OVER != phase) { LOG.error("Trying allocate ledger for {} in phase {}, giving up.", allocatePath, phase); @@ -329,7 +370,8 @@ private synchronized void allocateLedger() { bkc.createLedger( quorumConfig.getEnsembleSize(), quorumConfig.getWriteQuorumSize(), - quorumConfig.getAckQuorumSize() + quorumConfig.getAckQuorumSize(), + ledgerMetadata ).whenComplete(this); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java index 66e99401ce0..fc14df6929f 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java @@ -36,6 +36,7 @@ import org.apache.distributedlog.bk.DynamicQuorumConfigProvider; import org.apache.distributedlog.bk.LedgerAllocator; import org.apache.distributedlog.bk.LedgerAllocatorDelegator; +import org.apache.distributedlog.bk.LedgerMetadata; import org.apache.distributedlog.bk.QuorumConfigProvider; import org.apache.distributedlog.bk.SimpleLedgerAllocator; import org.apache.distributedlog.config.DynamicDistributedLogConfiguration; @@ -153,7 +154,8 @@ public void deleteComplete(int rc, Object ctx) { // LedgerAllocator createLedgerAllocator(LogMetadataForWriter logMetadata, - DynamicDistributedLogConfiguration dynConf) + DynamicDistributedLogConfiguration dynConf, + LedgerMetadata ledgerMetadata) throws IOException { LedgerAllocator ledgerAllocatorDelegator; if (null == allocator || !dynConf.getEnableLedgerAllocatorPool()) { @@ -164,7 +166,8 @@ LedgerAllocator createLedgerAllocator(LogMetadataForWriter logMetadata, logMetadata.getAllocationData(), quorumConfigProvider, zkc, - bkc); + bkc, + ledgerMetadata); ledgerAllocatorDelegator = new LedgerAllocatorDelegator(allocator, true); } else { ledgerAllocatorDelegator = allocator; @@ -176,8 +179,16 @@ LedgerAllocator createLedgerAllocator(LogMetadataForWriter logMetadata, public Allocator newLogSegmentAllocator( LogMetadataForWriter logMetadata, DynamicDistributedLogConfiguration dynConf) throws IOException { + return newLogSegmentAllocator(logMetadata, dynConf, null); + } + + @Override + public Allocator newLogSegmentAllocator( + LogMetadataForWriter logMetadata, + DynamicDistributedLogConfiguration dynConf, + LedgerMetadata ledgerMetadata) throws IOException { // Build the ledger allocator - LedgerAllocator allocator = createLedgerAllocator(logMetadata, dynConf); + LedgerAllocator allocator = createLedgerAllocator(logMetadata, dynConf, ledgerMetadata); return new BKLogSegmentAllocator(allocator); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java index 4822b7efc33..f57a4b5f233 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.concurrent.CompletableFuture; import org.apache.distributedlog.LogSegmentMetadata; +import org.apache.distributedlog.bk.LedgerMetadata; import org.apache.distributedlog.config.DynamicDistributedLogConfiguration; import org.apache.distributedlog.metadata.LogMetadataForWriter; import org.apache.distributedlog.util.Allocator; @@ -50,6 +51,20 @@ Allocator newLogSegmentAllocator( LogMetadataForWriter metadata, DynamicDistributedLogConfiguration dynConf) throws IOException; + /** + * Create a new log segment allocator for allocating log segment entry writers. + * + * @param metadata the metadata for the log stream + * @param ledgerMetadata metadata to be attached to underlying ledgers + * @return future represent the log segment allocator + */ + default Allocator newLogSegmentAllocator( + LogMetadataForWriter metadata, + DynamicDistributedLogConfiguration dynConf, + LedgerMetadata ledgerMetadata) throws IOException { + throw new UnsupportedOperationException(); + } + /** * Open the reader for reading data to the log segment. * diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java index db0689145c5..3aada6865fe 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -38,6 +39,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.distributedlog.api.AsyncLogReader; import org.apache.distributedlog.api.AsyncLogWriter; @@ -47,6 +49,7 @@ import org.apache.distributedlog.api.namespace.Namespace; import org.apache.distributedlog.api.namespace.NamespaceBuilder; import org.apache.distributedlog.api.subscription.SubscriptionsStore; +import org.apache.distributedlog.bk.LedgerMetadata; import org.apache.distributedlog.callback.LogSegmentListener; import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException; import org.apache.distributedlog.exceptions.BKTransmitException; @@ -1247,4 +1250,54 @@ public void testDeleteLog() throws Exception { fail("Delete log twice should not throw any exception"); } } + + @Test(timeout = 60000) + public void testSyncLogWithLedgerMetadata() throws Exception { + + String application = "myapplication"; + String component = "mycomponent"; + String custom = "mycustommetadata"; + LedgerMetadata ledgerMetadata = new LedgerMetadata(); + ledgerMetadata.setApplication(application); + ledgerMetadata.setComponent(component); + ledgerMetadata.addCustomMetadata("custom", custom); + + BKDistributedLogManager dlm = createNewDLM(conf, "distrlog-writemetadata"); + + BKSyncLogWriter sync = dlm.openLogWriter(ledgerMetadata); + sync.write(DLMTestUtil.getLogRecordInstance(1)); + + LedgerHandle lh = getLedgerHandle(sync.getCachedLogWriter()); + Map customMeta = lh.getCustomMetadata(); + assertEquals(application, new String(customMeta.get("application"), UTF_8)); + assertEquals(component, new String(customMeta.get("component"), UTF_8)); + assertEquals(custom, new String(customMeta.get("custom"), UTF_8)); + + sync.closeAndComplete(); + } + + @Test(timeout = 60000) + public void testAsyncLogWithLedgerMetadata() throws Exception { + DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); + confLocal.addConfiguration(conf); + confLocal.setOutputBufferSize(0); + confLocal.setWriteLockEnabled(false); + + BKDistributedLogManager dlm = createNewDLM(confLocal, "distrlog-writemetadata"); + + String application = "myapplication"; + String custom = "mycustommetadata"; + LedgerMetadata ledgerMetadata = new LedgerMetadata(); + ledgerMetadata.setApplication(application); + ledgerMetadata.addCustomMetadata("custom", custom); + + AsyncLogWriter async = Utils.ioResult(dlm.openAsyncLogWriter(ledgerMetadata)); + Utils.ioResult(async.write(DLMTestUtil.getLogRecordInstance(2))); + + LedgerHandle lh = getLedgerHandle(((BKAsyncLogWriter) async).getCachedLogWriter()); + Map customMeta = lh.getCustomMetadata(); + assertEquals(application, new String(customMeta.get("application"), UTF_8)); + assertNull(customMeta.get("component")); + assertEquals(custom, new String(customMeta.get("custom"), UTF_8)); + } } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java index 011562e6ed0..50cde8014cc 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java @@ -25,6 +25,7 @@ import java.net.URI; import java.util.Enumeration; import java.util.HashSet; +import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.BKException; @@ -121,7 +122,14 @@ private SimpleLedgerAllocator createAllocator(String allocationPath) throws Exce private SimpleLedgerAllocator createAllocator(String allocationPath, DistributedLogConfiguration conf) throws Exception { - return Utils.ioResult(SimpleLedgerAllocator.of(allocationPath, null, newQuorumConfigProvider(conf), zkc, bkc)); + return createAllocator(allocationPath, conf, null); + } + + private SimpleLedgerAllocator createAllocator(String allocationPath, + DistributedLogConfiguration conf, + LedgerMetadata ledgerMetadata) throws Exception { + return Utils.ioResult(SimpleLedgerAllocator.of(allocationPath, null, + newQuorumConfigProvider(conf), zkc, bkc, ledgerMetadata)); } @FlakyTest("https://issues.apache.org/jira/browse/DL-43") @@ -378,4 +386,27 @@ public void testObtainMultipleLedgers() throws Exception { } assertEquals(numLedgers, allocatedLedgers.size()); } + + @Test(timeout = 60000) + public void testAllocationWithMetadata() throws Exception { + String allocationPath = "/" + runtime.getMethodName(); + + String application = "testApplicationMetadata"; + String component = "testComponentMetadata"; + String custom = "customMetadata"; + LedgerMetadata ledgerMetadata = new LedgerMetadata(); + ledgerMetadata.setApplication(application); + ledgerMetadata.setComponent(component); + ledgerMetadata.addCustomMetadata("custom", custom); + + SimpleLedgerAllocator allocator = createAllocator(allocationPath, dlConf, ledgerMetadata); + allocator.allocate(); + + ZKTransaction txn = newTxn(); + LedgerHandle lh = Utils.ioResult(allocator.tryObtain(txn, NULL_LISTENER)); + Map customMeta = lh.getCustomMetadata(); + assertEquals(application, new String(customMeta.get("application"), UTF_8)); + assertEquals(component, new String(customMeta.get("component"), UTF_8)); + assertEquals(custom, new String(customMeta.get("custom"), UTF_8)); + } } From 58fc4a7603466374e1b7a617c5a31fb96716b7bd Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Mon, 24 Feb 2020 15:08:23 +0800 Subject: [PATCH 0387/1642] Move checkout action to v2 (#2272) --- .github/workflows/bookie-tests.yml | 6 +++++- .github/workflows/client-tests.yml | 6 +++++- .github/workflows/compatibility-check-java11.yml | 6 +++++- .github/workflows/compatibility-check-java8.yml | 6 +++++- .github/workflows/integration-tests.yml | 6 +++++- .github/workflows/pr-validation.yml | 6 +++++- .github/workflows/remaining-tests.yml | 6 +++++- .github/workflows/replication-tests.yml | 6 +++++- .github/workflows/tls-tests.yml | 6 +++++- 9 files changed, 45 insertions(+), 9 deletions(-) diff --git a/.github/workflows/bookie-tests.yml b/.github/workflows/bookie-tests.yml index eb41ce02048..7a784a26557 100644 --- a/.github/workflows/bookie-tests.yml +++ b/.github/workflows/bookie-tests.yml @@ -33,7 +33,11 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - name: Checkout + uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/client-tests.yml b/.github/workflows/client-tests.yml index a38b2ee74c8..47f912c0c58 100644 --- a/.github/workflows/client-tests.yml +++ b/.github/workflows/client-tests.yml @@ -33,7 +33,11 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - name: Checkout + uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/compatibility-check-java11.yml b/.github/workflows/compatibility-check-java11.yml index de63ef1544a..360ffba763f 100644 --- a/.github/workflows/compatibility-check-java11.yml +++ b/.github/workflows/compatibility-check-java11.yml @@ -32,7 +32,11 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - name: Checkout + uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.11 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/compatibility-check-java8.yml b/.github/workflows/compatibility-check-java8.yml index de82ecfd252..47cf21e6dc9 100644 --- a/.github/workflows/compatibility-check-java8.yml +++ b/.github/workflows/compatibility-check-java8.yml @@ -32,7 +32,11 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - name: Checkout + uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index 35536de76f8..b0b9fdef6f1 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -33,7 +33,11 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - name: Checkout + uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/pr-validation.yml b/.github/workflows/pr-validation.yml index 04550e90ba4..cf4147a9024 100644 --- a/.github/workflows/pr-validation.yml +++ b/.github/workflows/pr-validation.yml @@ -33,7 +33,11 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - name: Checkout + uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/remaining-tests.yml b/.github/workflows/remaining-tests.yml index e51b9feee73..30f8f0eef44 100644 --- a/.github/workflows/remaining-tests.yml +++ b/.github/workflows/remaining-tests.yml @@ -32,7 +32,11 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - name: Checkout + uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/replication-tests.yml b/.github/workflows/replication-tests.yml index 912c276b1f9..a561949b3bb 100644 --- a/.github/workflows/replication-tests.yml +++ b/.github/workflows/replication-tests.yml @@ -33,7 +33,11 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - name: Checkout + uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/tls-tests.yml b/.github/workflows/tls-tests.yml index 47ef36166fa..0803f8a09a6 100644 --- a/.github/workflows/tls-tests.yml +++ b/.github/workflows/tls-tests.yml @@ -33,7 +33,11 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - name: Checkout + uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: From 75db1d6d20b688778769050481ed6f9f7d805d09 Mon Sep 17 00:00:00 2001 From: Atri Sharma Date: Fri, 6 Mar 2020 12:45:15 +0530 Subject: [PATCH 0388/1642] 2178: IOException Should Be Thrown When Journal Is Corrupted Fixes #2178 Reviewers: Enrico Olivelli , Sijie Guo This closes #2257 from atris/2176 --- .../bookkeeper/bookie/JournalChannel.java | 17 +++++- .../bookkeeper/bookie/BookieJournalTest.java | 60 +++++++++++++++++++ 2 files changed, 75 insertions(+), 2 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java index f8a7230aaf9..2dbaee02197 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java @@ -23,6 +23,8 @@ import static com.google.common.base.Charsets.UTF_8; +import com.google.common.annotations.VisibleForTesting; + import java.io.Closeable; import java.io.File; import java.io.IOException; @@ -30,6 +32,7 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.Arrays; + import org.apache.bookkeeper.util.NativeIO; import org.apache.bookkeeper.util.ZeroBuffer; import org.slf4j.Logger; @@ -160,7 +163,7 @@ private JournalChannel(File journalDirectory, long logId, + " suddenly appeared, is another bookie process running?"); } randomAccessFile = new RandomAccessFile(fn, "rw"); - fc = randomAccessFile.getChannel(); + fc = openFileChannel(randomAccessFile); formatVersion = formatVersionToWrite; int headerSize = (V4 == formatVersion) ? VERSION_HEADER_SIZE : HEADER_SIZE; @@ -178,7 +181,7 @@ private JournalChannel(File journalDirectory, long logId, fc.write(zeros, nextPrealloc - journalAlignSize); } else { // open an existing file randomAccessFile = new RandomAccessFile(fn, "r"); - fc = randomAccessFile.getChannel(); + fc = openFileChannel(randomAccessFile); bc = null; // readonly ByteBuffer bb = ByteBuffer.allocate(VERSION_HEADER_SIZE); @@ -224,6 +227,7 @@ private JournalChannel(File journalDirectory, long logId, } } catch (IOException e) { LOG.error("Bookie journal file can seek to position :", e); + throw e; } } if (fRemoveFromPageCache) { @@ -290,4 +294,13 @@ public void forceWrite(boolean forceMetadata) throws IOException { this.lastDropPosition = newDropPos; } } + + @VisibleForTesting + public static FileChannel openFileChannel(RandomAccessFile randomAccessFile) { + if (randomAccessFile == null) { + throw new IllegalArgumentException("Input cannot be null"); + } + + return randomAccessFile.getChannel(); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java index 15589e344f8..88f2db73b82 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java @@ -47,12 +47,19 @@ import org.apache.commons.io.FileUtils; import org.junit.After; import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Test the bookie journal. */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(JournalChannel.class) public class BookieJournalTest { private static final Logger LOG = LoggerFactory.getLogger(BookieJournalTest.class); @@ -794,4 +801,57 @@ private void testPartialFileInfoPostV3Journal(boolean truncateMasterKey) // correct behaviour } } + + /** + * Test for fake IOException during read of Journal. + */ + @Test + public void testJournalScanIOException() throws Exception { + File journalDir = createTempDir("bookie", "journal"); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir)); + + File ledgerDir = createTempDir("bookie", "ledger"); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir)); + + writeV4Journal(Bookie.getCurrentDirectory(journalDir), 100, "testPasswd".getBytes()); + + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + conf.setJournalDirName(journalDir.getPath()) + .setLedgerDirNames(new String[] { ledgerDir.getPath() }) + .setMetadataServiceUri(null); + + Journal.JournalScanner journalScanner = new DummyJournalScan(); + FileChannel fileChannel = PowerMockito.mock(FileChannel.class); + + PowerMockito.when(fileChannel.position(Mockito.anyLong())) + .thenThrow(new IOException()); + + PowerMockito.mockStatic(JournalChannel.class); + PowerMockito.when(JournalChannel.openFileChannel(Mockito.any(RandomAccessFile.class))).thenReturn(fileChannel); + + Bookie b = new Bookie(conf); + + for (Journal journal : b.journals) { + List journalIds = journal.listJournalIds(journal.getJournalDirectory(), null); + + assertEquals(journalIds.size(), 1); + + try { + journal.scanJournal(journalIds.get(0), Long.MAX_VALUE, journalScanner); + fail("Should not have been able to scan the journal"); + } catch (Exception e) { + // Expected + } + } + + b.shutdown(); + } + + private class DummyJournalScan implements Journal.JournalScanner { + + @Override + public void process(int journalVersion, long offset, ByteBuffer entry) throws IOException { + LOG.warn("Journal Version : " + journalVersion); + } + }; } From e3dfc2d886c93b6599ae5731e7116f62e6d5270f Mon Sep 17 00:00:00 2001 From: Dapeng <1807643853@qq.com> Date: Fri, 6 Mar 2020 15:31:24 +0800 Subject: [PATCH 0389/1642] Move the result of the BKCTL to the console by default ### Motivation ### Changes Move the result of the BKCTL to the console Master Issue: #2236 Reviewers: Enrico Olivelli , Sijie Guo This closes #2268 from SunDapeng1/branch-2267 --- bin/bkctl | 2 +- .../org/apache/bookkeeper/bookie/CreateNewLogTest.java | 3 ++- .../java/org/apache/bookkeeper/bookie/TestEntryLog.java | 1 + conf/log4j.cli.properties | 7 ++++++- conf/log4j.shell.properties | 3 ++- 5 files changed, 12 insertions(+), 4 deletions(-) diff --git a/bin/bkctl b/bin/bkctl index 1cb73571652..ab18d1ee905 100755 --- a/bin/bkctl +++ b/bin/bkctl @@ -47,7 +47,7 @@ if [ -z "${CLI_LOG_CONF}" ]; then fi CLI_LOG_DIR=${CLI_LOG_DIR:-"$BK_HOME/logs"} CLI_LOG_FILE=${CLI_LOG_FILE:-"bkctl.log"} -CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,ROLLINGFILE"} +CLI_ROOT_LOGGER=${CLI_ROOT_LOGGER:-"INFO,CONSOLE"} # Configure the classpath CLI_CLASSPATH="$CLI_JAR:$CLI_CLASSPATH:$CLI_EXTRA_CLASSPATH" diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java index 0b24db23047..d2052451ac2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java @@ -562,6 +562,7 @@ public void testConcurrentEntryLogCreations() throws Exception { startLatch.await(); entrylogManager.createNewLog(ledgerId); createdEntryLogs.incrementAndGet(); + Thread.sleep(2000); } catch (InterruptedException | IOException e) { LOG.error("Got exception while trying to createNewLog for Ledger: " + ledgerId, e); } finally { @@ -572,7 +573,7 @@ public void testConcurrentEntryLogCreations() throws Exception { } startLatch.countDown(); - createdLatch.await(5, TimeUnit.SECONDS); + createdLatch.await(20, TimeUnit.SECONDS); Assert.assertEquals("Created EntryLogs", numOfLedgers * numOfThreadsForSameLedger, createdEntryLogs.get()); Assert.assertEquals("Active currentlogs size", numOfLedgers, entrylogManager.getCopyOfCurrentLogs().size()); Assert.assertEquals("Rotated entrylogs size", (numOfThreadsForSameLedger - 1) * numOfLedgers, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java index fd35204b001..3e54b76ac63 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestEntryLog.java @@ -442,6 +442,7 @@ public void testGetEntryLogsSet() throws Exception { entryLogManagerBase.createNewLog(EntryLogger.UNASSIGNED_LEDGERID); entryLogManagerBase.flushRotatedLogs(); + Thread.sleep(2000); assertEquals(Sets.newHashSet(0L, 1L), entryLogger.getEntryLogsSet()); entryLogManagerBase.createNewLog(EntryLogger.UNASSIGNED_LEDGERID); diff --git a/conf/log4j.cli.properties b/conf/log4j.cli.properties index f90d9551aa6..a5750334981 100644 --- a/conf/log4j.cli.properties +++ b/conf/log4j.cli.properties @@ -54,4 +54,9 @@ log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{ log4j.logger.verbose=INFO,VERBOSECONSOLE log4j.logger.org.apache.zookeeper=ERROR -log4j.logger.org.apache.bookkeeper=INFO +log4j.logger.org.apache.bookkeeper=ERROR +log4j.logger.org.apache.bookkeeper.bookie.BookieShell=INFO +log4j.logger.org.apache.bookkeeper.client.BookKeeperAdmin=INFO +log4j.logger.org.apache.bookkeeper.bookie.InterleavedStorageRegenerateIndexOp=INFO +log4j.logger.org.apache.bookkeeper.tools.cli.commands.bookies.InstanceIdCommand=INFO + diff --git a/conf/log4j.shell.properties b/conf/log4j.shell.properties index 7f8c00f3357..e27f7f5edbe 100644 --- a/conf/log4j.shell.properties +++ b/conf/log4j.shell.properties @@ -48,4 +48,5 @@ log4j.logger.org.apache.zookeeper=ERROR log4j.logger.org.apache.bookkeeper=ERROR log4j.logger.org.apache.bookkeeper.bookie.BookieShell=INFO log4j.logger.org.apache.bookkeeper.client.BookKeeperAdmin=INFO -log4j.logger.org.apache.bookkeeper.bookie.InterleavedStorageRegenerateIndexOp=INFO \ No newline at end of file +log4j.logger.org.apache.bookkeeper.bookie.InterleavedStorageRegenerateIndexOp=INFO +log4j.logger.org.apache.bookkeeper.tools.cli.commands.bookies.InstanceIdCommand=INFO From 8833d3b252ff36e7e168f7e5e0710c86ea92151d Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sat, 7 Mar 2020 14:46:45 +0100 Subject: [PATCH 0390/1642] Drop Maven Profile -Dstream (#2233) ### Motivation The "stream" profile enables the build of all of the additional packages regarding the 'table service'. This profile complicates a lot every script and the Maven project. As the stream storage service is now stable and used in production, it is better to always include those modules. Regualar developers on bookkeeper-server module won't be affected by this change as it affects only the full build and the release process. ### Changes Modify every pom.xml in order to enable all of the parts of the pom.xml that would be enabled with '-Dstream'. Modify release scripts by removing references to -Dstream Modify bin/common.sh about the auto build feature Modify Groovy scripts for Jenkins Do not "shade" libthrift inside Distributed Log Jar I did not modify the website because currently the precommit validation of the website is broken * Remove stream Maven Profile * Delete job_bookkeeper_precommit_java11.groovy * Delete job_bookkeeper_precommit_java8.groovy * Delete ticktoc.sh * Restore -DstreamTests profile Co-authored-by: Enrico Olivelli --- .../job_bookkeeper_codecoverage.groovy | 2 +- ...bookkeeper_postcommit_master_java11.groovy | 2 +- ..._bookkeeper_postcommit_master_java8.groovy | 2 +- ...keeper_postcommit_validation_master.groovy | 2 +- ...bookkeeper_release_nightly_snapshot.groovy | 2 +- bin/common.sh | 6 +- bookkeeper-dist/all/pom.xml | 26 +++---- bookkeeper-dist/server/pom.xml | 25 ++----- dev/release/001-release-branch.sh | 3 +- dev/release/002-release-prepare.sh | 3 +- dev/release/003-release-perform.sh | 3 +- dev/ticktoc.sh | 44 ------------ dev/update-snapshot-version.sh | 4 +- pom.xml | 16 +---- stream/distributedlog/core/pom.xml | 69 ------------------ stream/distributedlog/pom.xml | 23 ------ tests/integration/pom.xml | 15 +--- tools/all/pom.xml | 71 +++++++------------ tools/pom.xml | 18 +---- 19 files changed, 59 insertions(+), 277 deletions(-) delete mode 100755 dev/ticktoc.sh diff --git a/.test-infra/jenkins/job_bookkeeper_codecoverage.groovy b/.test-infra/jenkins/job_bookkeeper_codecoverage.groovy index 07635497d04..8beb6272420 100644 --- a/.test-infra/jenkins/job_bookkeeper_codecoverage.groovy +++ b/.test-infra/jenkins/job_bookkeeper_codecoverage.groovy @@ -39,5 +39,5 @@ mavenJob('bookkeeper_codecoverage') { common_job_properties.setMavenConfig(delegate) // Maven build project. - goals('clean verify jacoco:report coveralls:report -Pcode-coverage -DrepoToken=$COVERALLS_REPO_TOKEN -Dmaven.test.failure.ignore=true -Dstream') + goals('clean verify jacoco:report coveralls:report -Pcode-coverage -DrepoToken=$COVERALLS_REPO_TOKEN -Dmaven.test.failure.ignore=true ') } diff --git a/.test-infra/jenkins/job_bookkeeper_postcommit_master_java11.groovy b/.test-infra/jenkins/job_bookkeeper_postcommit_master_java11.groovy index 25423ad7837..89c4e2137e7 100644 --- a/.test-infra/jenkins/job_bookkeeper_postcommit_master_java11.groovy +++ b/.test-infra/jenkins/job_bookkeeper_postcommit_master_java11.groovy @@ -39,5 +39,5 @@ mavenJob('bookkeeper_postcommit_master_java11') { common_job_properties.setMavenConfig(delegate) // Maven build project. - goals('clean package spotbugs:check -Ddistributedlog -Dstream -DstreamTests') + goals('clean package spotbugs:check -Ddistributedlog -DstreamTests') } diff --git a/.test-infra/jenkins/job_bookkeeper_postcommit_master_java8.groovy b/.test-infra/jenkins/job_bookkeeper_postcommit_master_java8.groovy index bb5a5d5cf08..5d425ab2199 100644 --- a/.test-infra/jenkins/job_bookkeeper_postcommit_master_java8.groovy +++ b/.test-infra/jenkins/job_bookkeeper_postcommit_master_java8.groovy @@ -39,5 +39,5 @@ mavenJob('bookkeeper_postcommit_master_java8') { common_job_properties.setMavenConfig(delegate) // Maven build project. - goals('clean package spotbugs:check -Ddistributedlog -Dstream -DstreamTests') + goals('clean package spotbugs:check -Ddistributedlog -DstreamTests') } diff --git a/.test-infra/jenkins/job_bookkeeper_postcommit_validation_master.groovy b/.test-infra/jenkins/job_bookkeeper_postcommit_validation_master.groovy index f7005b3c469..569a9344ee0 100644 --- a/.test-infra/jenkins/job_bookkeeper_postcommit_validation_master.groovy +++ b/.test-infra/jenkins/job_bookkeeper_postcommit_validation_master.groovy @@ -39,5 +39,5 @@ mavenJob('bookkeeper_postcommit_validation_master') { common_job_properties.setMavenConfig(delegate) // Maven build project. - goals('clean apache-rat:check checkstyle:check package -Ddistributedlog -Dstream -DskipTests') + goals('clean apache-rat:check checkstyle:check package -Ddistributedlog -DskipTests') } diff --git a/.test-infra/jenkins/job_bookkeeper_release_nightly_snapshot.groovy b/.test-infra/jenkins/job_bookkeeper_release_nightly_snapshot.groovy index ced65276d63..5166df274d3 100644 --- a/.test-infra/jenkins/job_bookkeeper_release_nightly_snapshot.groovy +++ b/.test-infra/jenkins/job_bookkeeper_release_nightly_snapshot.groovy @@ -58,7 +58,7 @@ export MAVEN_OPTS=-Xmx2048m common_job_properties.setMavenConfig(delegate) // Maven build project. - goals('clean package -Dmaven.test.failure.ignore=true deploy -Ddistributedlog -Dstream -DstreamTests -Pdocker') + goals('clean package -Dmaven.test.failure.ignore=true deploy -Ddistributedlog -DstreamTests -Pdocker') } // publish the docker images diff --git a/bin/common.sh b/bin/common.sh index cf7cf06dbd3..7ca32efa804 100755 --- a/bin/common.sh +++ b/bin/common.sh @@ -199,13 +199,13 @@ find_module_jar() { BUILT_JAR=$(find_module_jar_at ${BK_HOME}/${MODULE_PATH}/target ${MODULE_NAME}) if [ -z "${BUILT_JAR}" ]; then echo "Couldn't find module '${MODULE_NAME}' jar." >&2 - read -p "Do you want me to run \`mvn package -DskipTests -Dstream\` for you ? (y|n) " answer + read -p "Do you want me to run \`mvn package -DskipTests\` for you ? (y|n) " answer case "${answer:0:1}" in y|Y ) mkdir -p ${BK_HOME}/logs output="${BK_HOME}/logs/build.out" echo "see output at ${output} for the progress ..." >&2 - mvn package -DskipTests -Dstream &> ${output} + mvn package -DskipTests &> ${output} ;; * ) exit 1 @@ -242,7 +242,7 @@ add_maven_deps_to_classpath() { if [ ! -f ${f} ]; then echo "the classpath of module '${MODULE_PATH}' is not found, generating it ..." >&2 echo "see output at ${output} for the progress ..." >&2 - ${MVN} -f "${BK_HOME}/${MODULE_PATH}/pom.xml" -Dstream dependency:build-classpath -Dmdep.outputFile="target/cached_classpath.txt" &> ${output} + ${MVN} -f "${BK_HOME}/${MODULE_PATH}/pom.xml" dependency:build-classpath -Dmdep.outputFile="target/cached_classpath.txt" &> ${output} echo "the classpath of module '${MODULE_PATH}' is generated at '${f}'." >&2 fi } diff --git a/bookkeeper-dist/all/pom.xml b/bookkeeper-dist/all/pom.xml index 1504fa6adda..4b9044da286 100644 --- a/bookkeeper-dist/all/pom.xml +++ b/bookkeeper-dist/all/pom.xml @@ -82,6 +82,14 @@ ${project.version} + + + + org.apache.bookkeeper + stream-storage-server + ${project.version} + + org.apache.bookkeeper @@ -135,22 +143,4 @@ - - - stream - - - stream - - - - - - org.apache.bookkeeper - stream-storage-server - ${project.version} - - - - diff --git a/bookkeeper-dist/server/pom.xml b/bookkeeper-dist/server/pom.xml index 6f060a54a05..a23a66d0360 100644 --- a/bookkeeper-dist/server/pom.xml +++ b/bookkeeper-dist/server/pom.xml @@ -62,6 +62,13 @@ ${project.version} + + + org.apache.bookkeeper + stream-storage-server + ${project.version} + + org.apache.bookkeeper @@ -123,22 +130,4 @@ - - - stream - - - stream - - - - - - org.apache.bookkeeper - stream-storage-server - ${project.version} - - - - diff --git a/dev/release/001-release-branch.sh b/dev/release/001-release-branch.sh index e137f00e843..2e48403fc05 100755 --- a/dev/release/001-release-branch.sh +++ b/dev/release/001-release-branch.sh @@ -25,5 +25,4 @@ cd $BK_HOME mvn release:branch \ -DbranchName=${BRANCH_NAME} \ -DdevelopmentVersion=${DEVELOPMENT_VERSION} - -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true -Dstream" \ - -Dstream + -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true" diff --git a/dev/release/002-release-prepare.sh b/dev/release/002-release-prepare.sh index 232fb502aa1..4746dbce052 100755 --- a/dev/release/002-release-prepare.sh +++ b/dev/release/002-release-prepare.sh @@ -26,6 +26,5 @@ mvn release:prepare \ -DreleaseVersion=${VERSION} \ -Dtag=${RC_TAG} \ -DupdateWorkingCopyVersions=false \ - -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true -Dstream" \ - -Dstream \ + -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true" \ -Dresume=true diff --git a/dev/release/003-release-perform.sh b/dev/release/003-release-perform.sh index 2a5b531de13..36f16b56888 100755 --- a/dev/release/003-release-perform.sh +++ b/dev/release/003-release-perform.sh @@ -23,6 +23,5 @@ BK_HOME=`cd $BINDIR/../..;pwd` cd $BK_HOME mvn release:perform \ - -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true -Dstream" \ - -Dstream \ + -Darguments="-Dmaven.javadoc.skip=true -DskipTests=true" \ -Dresume=true diff --git a/dev/ticktoc.sh b/dev/ticktoc.sh deleted file mode 100755 index 7903905c46b..00000000000 --- a/dev/ticktoc.sh +++ /dev/null @@ -1,44 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# Script from https://github.com/travis-ci/travis-ci/issues/4190 - -set -e -set -u - -command=$1 - -# launch command in the background -${command} & - -# ping every second -seconds=0 -limit=40*60 -while kill -0 $! >/dev/null 2>&1; -do - echo -n -e " \b" # never leave evidence - - if [ $seconds == $limit ]; then - break; - fi - - seconds=$((seconds + 1)) - - sleep 1 -done diff --git a/dev/update-snapshot-version.sh b/dev/update-snapshot-version.sh index 59adf078bf9..8d4edbe0daf 100755 --- a/dev/update-snapshot-version.sh +++ b/dev/update-snapshot-version.sh @@ -33,6 +33,6 @@ if [ "x${PUBLISH_GITSHA}" = "xtrue" ]; then NEW_VERSION=$(get_snapshot_version_with_gitsha) echo "Update version from ${OLD_VERSION} to ${NEW_VERSION}" - mvn versions:set -Dstream -DnewVersion=${NEW_VERSION} - mvn versions:commit -Dstream + mvn versions:set -DnewVersion=${NEW_VERSION} + mvn versions:commit fi diff --git a/pom.xml b/pom.xml index c4df5dbde70..1d130485ba8 100644 --- a/pom.xml +++ b/pom.xml @@ -63,7 +63,7 @@ bookkeeper-benchmark bookkeeper-stats-providers bookkeeper-http - stream/distributedlog + stream tools cpu-affinity metadata-drivers @@ -1062,20 +1062,6 @@ - - - stream - - - stream - - - - - stream - - - jdk11-no-spotbugs diff --git a/stream/distributedlog/core/pom.xml b/stream/distributedlog/core/pom.xml index fa10425f54e..65129dbd4f9 100644 --- a/stream/distributedlog/core/pom.xml +++ b/stream/distributedlog/core/pom.xml @@ -108,35 +108,6 @@ com.github.spotbugs spotbugs-maven-plugin - - org.apache.maven.plugins - maven-shade-plugin - ${maven-shade-plugin.version} - - - shade-thrift - package - - shade - - - true - - - org.apache.thrift:libthrift - - - true - - - org.apache.thrift - dlshade.org.apache.thrift - - - - - - org.apache.maven.plugins maven-checkstyle-plugin @@ -145,46 +116,6 @@ **/thrift/**/* - - org.codehaus.mojo - license-maven-plugin - ${license-maven-plugin.version} - - false - - ${project.basedir} - - - - - update-pom-license - - update-file-header - - package - - apache_v2 - - dependency-reduced-pom.xml - - - - - - - maven-clean-plugin - ${maven-clean-plugin.version} - - - - ${project.basedir} - - dependency-reduced-pom.xml - - - - - diff --git a/stream/distributedlog/pom.xml b/stream/distributedlog/pom.xml index 923dc9c84fe..ff537c47188 100644 --- a/stream/distributedlog/pom.xml +++ b/stream/distributedlog/pom.xml @@ -71,8 +71,6 @@ maven-surefire-plugin ${maven-surefire-plugin.version} - - true ${redirectTestOutputToFile} -Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G -Dio.netty.leakDetection.level=PARANOID always @@ -88,26 +86,5 @@ - - - distributedlog - - - distributedlog - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - false - - - - - - diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index 96eacc5e50d..a26758f7b53 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -30,6 +30,7 @@ smoke standalone + cluster @@ -69,19 +70,5 @@ - - - - stream - - - stream - - - - - cluster - - diff --git a/tools/all/pom.xml b/tools/all/pom.xml index e9cdb550dcc..d38523421a1 100644 --- a/tools/all/pom.xml +++ b/tools/all/pom.xml @@ -32,6 +32,11 @@ bookkeeper-tools-ledger ${project.version} + + org.apache.bookkeeper + stream-storage-cli + ${project.version} + @@ -57,50 +62,28 @@ + + org.apache.maven.plugins + maven-antrun-plugin + ${maven-antrun-plugin.version} + + + append-stream-commands + generate-resources + + run + + + + + + + + + + + + - - - stream - - - stream - - - - - - org.apache.bookkeeper - stream-storage-cli - ${project.version} - - - - - - org.apache.maven.plugins - maven-antrun-plugin - ${maven-antrun-plugin.version} - - - append-stream-commands - generate-resources - - run - - - - - - - - - - - - - - - - diff --git a/tools/pom.xml b/tools/pom.xml index e5ec928543f..76f91036199 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -28,22 +28,8 @@ framework ledger + stream + perf all - - - - stream - - - stream - - - - - stream - perf - - - From 834562003069002d579e9abe12e834fd4ecd1995 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 9 Mar 2020 19:30:00 +0800 Subject: [PATCH 0391/1642] Fixes vertx can not receive the request body --- *Motivation* Vertx can not receive the request body so that the REST API can not work well if you have a request with body. *Modifications* - Add body handler for the vertx http service *Verify this change* - Pass the test `testHttpMethodsWithBody` Descriptions of the changes in this PR: ### Motivation (Explain: why you're making that change, what is the problem you're trying to solve) ### Changes (Describe: what changes you have made) Master Issue: # Reviewers: Enrico Olivelli , Sijie Guo This closes #2241 from zymap/fix-can-not-read-body --- .../http/service/NullHttpService.java | 3 ++ .../http/vertx/VertxHttpServer.java | 2 ++ .../http/vertx/TestVertxHttpServer.java | 29 ++++++++++++++++++- .../impl/cluster/ZkClusterInitializer.java | 1 + 4 files changed, 34 insertions(+), 1 deletion(-) diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/service/NullHttpService.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/service/NullHttpService.java index 2f93f34efd2..6d38a17af15 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/service/NullHttpService.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/service/NullHttpService.java @@ -30,6 +30,9 @@ public class NullHttpService implements HttpEndpointService { @Override public HttpServiceResponse handle(HttpServiceRequest request) { + if (request.getBody() != null) { + return new HttpServiceResponse(request.getBody(), HttpServer.StatusCode.OK); + } return new HttpServiceResponse(CONTENT, HttpServer.StatusCode.OK); } } diff --git a/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java b/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java index 6d0bd63d7ac..31e11173e5d 100644 --- a/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java +++ b/bookkeeper-http/vertx-http-server/src/main/java/org/apache/bookkeeper/http/vertx/VertxHttpServer.java @@ -24,6 +24,7 @@ import io.vertx.core.AsyncResult; import io.vertx.core.Vertx; import io.vertx.ext.web.Router; +import io.vertx.ext.web.handler.BodyHandler; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -66,6 +67,7 @@ public boolean startServer(int port) { CompletableFuture> future = new CompletableFuture<>(); VertxHttpHandlerFactory handlerFactory = new VertxHttpHandlerFactory(httpServiceProvider); Router router = Router.router(vertx); + router.route().handler(BodyHandler.create()); HttpRouter requestRouter = new HttpRouter(handlerFactory) { @Override public void bindHandler(String endpoint, VertxAbstractHandler handler) { diff --git a/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java b/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java index c6cc13301db..79667f73915 100644 --- a/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java +++ b/bookkeeper-http/vertx-http-server/src/test/java/org/apache/bookkeeper/http/vertx/TestVertxHttpServer.java @@ -26,8 +26,10 @@ import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; +import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URL; +import java.nio.charset.StandardCharsets; import org.apache.bookkeeper.http.HttpRouter; import org.apache.bookkeeper.http.HttpServer; @@ -81,12 +83,37 @@ public void testHttpMethods() throws Exception { httpServer.stopServer(); } - // HTTP request + @Test + public void testHttpMethodsWithBody() throws IOException { + VertxHttpServer httpServer = new VertxHttpServer(); + HttpServiceProvider httpServiceProvider = NullHttpServiceProvider.getInstance(); + httpServer.initialize(httpServiceProvider); + assertTrue(httpServer.startServer(0)); + int port = httpServer.getListeningPort(); + String body = "{\"bookie_src\": \"localhost:3181\"}"; + HttpResponse httpResponse = send(getUrl(port, HttpRouter.DECOMMISSION), HttpServer.Method.PUT, body); + assertEquals(HttpServer.StatusCode.OK.getValue(), httpResponse.responseCode); + assertEquals(body, httpResponse.responseBody); + httpServer.stopServer(); + } + private HttpResponse send(String url, HttpServer.Method method) throws IOException { + return send(url, method, ""); + } + + // HTTP request + private HttpResponse send(String url, HttpServer.Method method, String body) throws IOException { URL obj = new URL(url); HttpURLConnection con = (HttpURLConnection) obj.openConnection(); // optional, default is GET con.setRequestMethod(method.toString()); + if (body != "") { + con.setDoOutput(true); + con.setFixedLengthStreamingMode(body.length()); + OutputStream outputStream = con.getOutputStream(); + outputStream.write(body.getBytes(StandardCharsets.UTF_8)); + outputStream.flush(); + } int responseCode = con.getResponseCode(); StringBuilder response = new StringBuilder(); BufferedReader in = null; diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java index 3668691519a..06ca7ab48c4 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java @@ -20,6 +20,7 @@ import static org.apache.bookkeeper.stream.storage.StorageConstants.ZK_METADATA_ROOT_PATH; import static org.apache.bookkeeper.stream.storage.StorageConstants.getSegmentsRootPath; + import com.google.common.base.Strings; import java.net.URI; import java.util.Optional; From ae81512fb7a5e05b0cac3f9c4fe15fe2d3554398 Mon Sep 17 00:00:00 2001 From: hangc0276 Date: Tue, 10 Mar 2020 21:17:53 +0800 Subject: [PATCH 0392/1642] fix bookie decommission sleep timeout value is negative bug MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit when decommission a bookie, and the ledger size of the bookie is big enough, the thread timeout will get negative, and the decommission operation will give up by throw exceptions as follow ``` 14:12:56.982 [main] INFO org.apache.bookkeeper.client.BookKeeperAdmin - Count of Ledgers which need to be rereplicated: 272752 14:12:56.983 [main] ERROR org.apache.bookkeeper.bookie.BookieShell - Received exception in DecommissionBookieCmd java.lang.IllegalArgumentException: timeout value is negative at java.lang.Thread.sleep(Native Method) ~[?:?] at org.apache.bookkeeper.client.BookKeeperAdmin.waitForLedgersToBeReplicated(BookKeeperAdmin.java:1528) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.client.BookKeeperAdmin.decommissionBookie(BookKeeperAdmin.java:1500) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.bookie.BookieShell$DecommissionBookieCmd.runCmd(BookieShell.java:2664) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.bookie.BookieShell$MyCommand.runCmd(BookieShell.java:277) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.bookie.BookieShell.run(BookieShell.java:3081) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.bookie.BookieShell.main(BookieShell.java:3172) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] 14:12:57.013 [main] INFO org.apache.zookeeper.ZooKeeper - Session: 0x206189927840052 closed ``` The exception code is ``` private void waitForLedgersToBeReplicated(Collection ledgers, BookieSocketAddress thisBookieAddress, LedgerManager ledgerManager) throws InterruptedException, TimeoutException { int maxSleepTimeInBetweenChecks = 10 * 60 * 1000; // 10 minutes int sleepTimePerLedger = 10 * 1000; // 10 secs Predicate validateBookieIsNotPartOfEnsemble = ledgerId -> !areEntriesOfLedgerStoredInTheBookie(ledgerId, thisBookieAddress, ledgerManager); while (!ledgers.isEmpty()) { LOG.info("Count of Ledgers which need to be rereplicated: {}", ledgers.size()); int sleepTimeForThisCheck = ledgers.size() * sleepTimePerLedger > maxSleepTimeInBetweenChecks ? maxSleepTimeInBetweenChecks : ledgers.size() * sleepTimePerLedger; Thread.sleep(sleepTimeForThisCheck); LOG.debug("Making sure following ledgers replication to be completed: {}", ledgers); ledgers.removeIf(validateBookieIsNotPartOfEnsemble); } } ``` the ledger size is `272752`, when computing sleepTimeForThisCheck, `ledgers.size() * sleepTimePerLedger` is `272752 * 10 * 1000 = 2727520000`, the value exceeds max int value `2147483647`, it will turn to `-1567447296`, then the sleepTimeForThisCheck will be `-1567447296`. Thread.sleep will throw `java.lang.IllegalArgumentException: timeout value is negative` exception Reviewers: Enrico Olivelli , Jia Zhai This closes #2284 from hangc0276/bug_fix --- .../main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 88a7c089ece..cac1d9d73dc 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 @@ -1536,7 +1536,7 @@ private void waitForLedgersToBeReplicated(Collection ledgers, BookieSocket thisBookieAddress, ledgerManager); while (!ledgers.isEmpty()) { LOG.info("Count of Ledgers which need to be rereplicated: {}", ledgers.size()); - int sleepTimeForThisCheck = ledgers.size() * sleepTimePerLedger > maxSleepTimeInBetweenChecks + int sleepTimeForThisCheck = (long) ledgers.size() * sleepTimePerLedger > maxSleepTimeInBetweenChecks ? maxSleepTimeInBetweenChecks : ledgers.size() * sleepTimePerLedger; Thread.sleep(sleepTimeForThisCheck); LOG.debug("Making sure following ledgers replication to be completed: {}", ledgers); From bd433f9bec2b8eeda58d2c15bf4620cd54494556 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 18 Mar 2020 23:15:33 +0100 Subject: [PATCH 0393/1642] Update Prometheus library to 0.8.1 Descriptions of the changes in this PR: Update Prometheus library to 0.8.1 ### Motivation We have an obsolete version ### Changes Update Prometheus library to latest version 0.8.1 and update NOTICE files Reviewers: Sijie Guo This closes #2291 from eolivelli/fix/update-prom-081 --- bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt | 8 ++++---- bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt | 8 ++++---- pom.xml | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt index 5a25036aac2..f4739b542e6 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt @@ -61,10 +61,10 @@ License for the specific language governing permissions and limitations under the License. ------------------------------------------------------------------------------------ -- lib/io.prometheus-simpleclient-0.0.21.jar -- lib/io.prometheus-simpleclient_common-0.0.21.jar -- lib/io.prometheus-simpleclient_hotspot-0.0.21.jar -- lib/io.prometheus-simpleclient_servlet-0.0.21.jar +- lib/io.prometheus-simpleclient-0.8.1.jar +- lib/io.prometheus-simpleclient_common-0.8.1.jar +- lib/io.prometheus-simpleclient_hotspot-0.8.1.jar +- lib/io.prometheus-simpleclient_servlet-0.8.1.jar Prometheus instrumentation library for JVM applications Copyright 2012-2015 The Prometheus Authors diff --git a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt index 8d24923893c..1b9a34cebb9 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt @@ -44,10 +44,10 @@ License for the specific language governing permissions and limitations under the License. ------------------------------------------------------------------------------------ -- lib/io.prometheus-simpleclient-0.0.21.jar -- lib/io.prometheus-simpleclient_common-0.0.21.jar -- lib/io.prometheus-simpleclient_hotspot-0.0.21.jar -- lib/io.prometheus-simpleclient_servlet-0.0.21.jar +- lib/io.prometheus-simpleclient-0.8.1.jar +- lib/io.prometheus-simpleclient_common-0.8.1.jar +- lib/io.prometheus-simpleclient_hotspot-0.8.1.jar +- lib/io.prometheus-simpleclient_servlet-0.8.1.jar Prometheus instrumentation library for JVM applications Copyright 2012-2015 The Prometheus Authors diff --git a/pom.xml b/pom.xml index 1d130485ba8..0189bd5fade 100644 --- a/pom.xml +++ b/pom.xml @@ -153,7 +153,7 @@ 2.0.20.Final 9.1.3 2.0.2 - 0.0.21 + 0.8.1 0.8.3 3.5.1 3.5.1-1 From 7bd0c4f4670e279d1dc7008f82e88af1e64a07c3 Mon Sep 17 00:00:00 2001 From: hangc0276 Date: Thu, 19 Mar 2020 07:05:04 +0800 Subject: [PATCH 0394/1642] fix some stats monitor metrics Descriptions of the changes in this PR: The journalStats.getJournalQueueSize increase once and decrease twice, so this stat metric will be negative. It increases in `logAddEntry` function and decrease in `QueueEntry recall` methord and Journal main thread run methord. so we should delete the decrease in journal main thread run methord. ``` queue.put(QueueEntry.create( entry, ackBeforeSync, ledgerId, entryId, cb, ctx, MathUtils.nowInNano(), journalStats.getJournalAddEntryStats(), journalStats.getJournalQueueSize())) ``` Reviewers: Sijie Guo This closes #2290 from hangc0276/bugfix_metrics --- .../src/main/java/org/apache/bookkeeper/bookie/Journal.java | 1 - .../bookkeeper/bookie/storage/ldb/DbLedgerStorageStats.java | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 090c35664f5..5d09e2f5944 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -1090,7 +1090,6 @@ public void run() { } else if (qe.entryId != Bookie.METAENTRY_ID_FORCE_LEDGER) { int entrySize = qe.entry.readableBytes(); journalStats.getJournalWriteBytes().add(entrySize); - journalStats.getJournalQueueSize().dec(); batchSize += (4 + entrySize); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageStats.java index bc99c609d25..9088ddab624 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageStats.java @@ -87,12 +87,12 @@ class DbLedgerStorageStats { ) private final OpStatsLogger readAheadBatchCountStats; @StatsDoc( - name = READAHEAD_BATCH_COUNT, + name = READAHEAD_BATCH_SIZE, help = "the distribution of num of bytes to read in one readahead batch" ) private final OpStatsLogger readAheadBatchSizeStats; @StatsDoc( - name = FLUSH_SIZE, + name = FLUSH, help = "operation stats of flushing write cache to entry log files" ) private final OpStatsLogger flushStats; From ca70605a012849ec64919d527a9f4b613017d3ac Mon Sep 17 00:00:00 2001 From: hangc0276 Date: Thu, 19 Mar 2020 07:08:25 +0800 Subject: [PATCH 0395/1642] Add Journal PageCache flush interval control when journalSyncData is disabled MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Descriptions of the changes in this PR: ### Motivation There was a weird behavior observed: when journalSyncData is disabled, it will trigger callbacks when the data is flushed to filesystem(PageCache) but still enqueue a fsync request to force write queue to fsync PageCache data to disk frequently. Though the fsync thread dequeue fsync request and execute fsync operation asynchronously, it still cause disk io util increase when fsync frequency high enough. when disk io util increase, especially reach 100%, it will block new journal file creation and lead to journal sync time increases to mutiple seconds. ### Changes when journalSyncData is disabled, we introduce `journalPageCacheFlushIntervalMSec` to control journal PageCache flush frequency to reduce disk io util, default value is 1000ms Reviewers: Enrico Olivelli , Jia Zhai , Sijie Guo This closes #2287 from hangc0276/bugfix --- .../org/apache/bookkeeper/bookie/Journal.java | 22 +- .../bookkeeper/conf/ServerConfiguration.java | 22 ++ .../BookieJournalPageCacheFlushTest.java | 297 ++++++++++++++++++ conf/bk_server.conf | 3 + 4 files changed, 342 insertions(+), 2 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 5d09e2f5944..4d6d2e839cb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -606,6 +606,8 @@ static void writePaddingBytes(JournalChannel jc, ByteBuf paddingBuffer, int jour private final boolean removePagesFromCache; private final int journalFormatVersionToWrite; private final int journalAlignmentSize; + // control PageCache flush interval when syncData disabled to reduce disk io util + private final long journalPageCacheFlushIntervalMSec; // Should data be fsynced on disk before triggering the callback private final boolean syncData; @@ -666,6 +668,7 @@ public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf this.bufferedEntriesThreshold = conf.getJournalBufferedEntriesThreshold(); this.journalFormatVersionToWrite = conf.getJournalFormatVersionToWrite(); this.journalAlignmentSize = conf.getJournalAlignmentSize(); + this.journalPageCacheFlushIntervalMSec = conf.getJournalPageCacheFlushIntervalMSec(); if (conf.getNumJournalCallbackThreads() > 0) { this.cbThreadPool = Executors.newFixedThreadPool(conf.getNumJournalCallbackThreads(), new DefaultThreadFactory("bookie-journal-callback")); @@ -933,6 +936,7 @@ public void run() { boolean groupWhenTimeout = false; long dequeueStartTime = 0L; + long lastFlushTimeMs = System.currentTimeMillis(); QueueEntry qe = null; while (true) { @@ -1052,8 +1056,22 @@ public void run() { boolean shouldRolloverJournal = (lastFlushPosition > maxJournalSize); // Trigger data sync to disk in the "Force-Write" thread. - forceWriteRequests.put(createForceWriteRequest(logFile, logId, lastFlushPosition, - toFlush, shouldRolloverJournal, false)); + // Trigger data sync to disk has three situations: + // 1. journalSyncData enabled, usually for SSD used as journal storage + // 2. shouldRolloverJournal is true, that is the journal file reaches maxJournalSize + // 3. if journalSyncData disabled and shouldRolloverJournal is false, we can use + // journalPageCacheFlushIntervalMSec to control sync frequency, preventing disk + // synchronize frequently, which will increase disk io util. + // when flush interval reaches journalPageCacheFlushIntervalMSec (default: 1s), + // it will trigger data sync to disk + if (syncData + || shouldRolloverJournal + || (System.currentTimeMillis() - lastFlushTimeMs + >= journalPageCacheFlushIntervalMSec)) { + forceWriteRequests.put(createForceWriteRequest(logFile, logId, lastFlushPosition, + toFlush, shouldRolloverJournal, false)); + lastFlushTimeMs = System.currentTimeMillis(); + } toFlush = entryListRecycler.newInstance(); numEntriesToFlush = 0; 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 cba673bdf80..ab49c71cdfa 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 @@ -135,6 +135,7 @@ public class ServerConfiguration extends AbstractConfiguration enableForceWriteThreadSuspension( + CountDownLatch forceWriteThreadSuspendedLatch, + Journal journal) throws InterruptedException { + LinkedBlockingQueue supportQueue = new LinkedBlockingQueue<>(); + BlockingQueue forceWriteRequests = mock(BlockingQueue.class); + doAnswer((Answer) (InvocationOnMock iom) -> { + supportQueue.put(iom.getArgument(0)); + return null; + }).when(forceWriteRequests).put(any(ForceWriteRequest.class)); + when(forceWriteRequests.take()).thenAnswer(i -> { + // suspend the force write thread + forceWriteThreadSuspendedLatch.await(); + return supportQueue.take(); + }); + Whitebox.setInternalState(journal, "forceWriteRequests", forceWriteRequests); + return supportQueue; + } + + @Test + public void testAckAfterSyncPageCacheFlush() throws Exception { + File journalDir = tempDir.newFolder(); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir)); + + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setJournalDirName(journalDir.getPath()) + .setMetadataServiceUri(null) + .setJournalAdaptiveGroupWrites(false) + .setJournalSyncData(true) + .setJournalPageCacheFlushIntervalMSec(5000); + + JournalChannel jc = spy(new JournalChannel(journalDir, 1)); + whenNew(JournalChannel.class).withAnyArguments().thenReturn(jc); + + LedgerDirsManager ledgerDirsManager = mock(LedgerDirsManager.class); + Journal journal = new Journal(0, journalDir, conf, ledgerDirsManager); + + CountDownLatch forceWriteThreadSuspendedLatch = new CountDownLatch(1); + LinkedBlockingQueue supportQueue = + enableForceWriteThreadSuspension(forceWriteThreadSuspendedLatch, journal); + journal.start(); + + LogMark lastLogMarkBeforeWrite = journal.getLastLogMark().markLog().getCurMark(); + CountDownLatch latch = new CountDownLatch(1); + long ledgerId = 1; + long entryId = 0; + long startTime = System.currentTimeMillis(); + journal.logAddEntry(ledgerId, entryId, DATA, false /* ackBeforeSync */, new WriteCallback() { + @Override + public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + latch.countDown(); + } + }, null); + + while (supportQueue.isEmpty()) { + Thread.sleep(100); + } + + // forceWriteRequest insert into forceWriteRequestQueue not effected by journalPageCacheFlushInterval + assertTrue(System.currentTimeMillis() - startTime < 5000); + + assertEquals(1, latch.getCount()); + assertEquals(1, supportQueue.size()); + + // in constructor of JournalChannel we are calling forceWrite(true) but it is not tracked by PowerMock + // because the 'spy' is applied only on return from the constructor + verify(jc, times(0)).forceWrite(true); + + // should not call forceWrite + verify(jc, times(0)).forceWrite(false); + + // let ForceWriteThread work + forceWriteThreadSuspendedLatch.countDown(); + // callback should complete now + assertTrue(latch.await(20, TimeUnit.SECONDS)); + + verify(jc, times(1)).forceWrite(false); + assertEquals(0, supportQueue.size()); + + // verify that log marker advanced + LastLogMark lastLogMarkAfterForceWrite = journal.getLastLogMark(); + assertTrue(lastLogMarkAfterForceWrite.getCurMark().compare(lastLogMarkBeforeWrite) > 0); + + journal.shutdown(); + } + + @Test + public void testAckBeforeSyncPageCacheFlush() throws Exception { + File journalDir = tempDir.newFolder(); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir)); + + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setJournalDirName(journalDir.getPath()) + .setMetadataServiceUri(null) + .setJournalAdaptiveGroupWrites(false) + .setJournalSyncData(true) + .setJournalPageCacheFlushIntervalMSec(5000); + + JournalChannel jc = spy(new JournalChannel(journalDir, 1)); + whenNew(JournalChannel.class).withAnyArguments().thenReturn(jc); + + LedgerDirsManager ledgerDirsManager = mock(LedgerDirsManager.class); + Journal journal = new Journal(0, journalDir, conf, ledgerDirsManager); + + CountDownLatch forceWriteThreadSuspendedLatch = new CountDownLatch(1); + LinkedBlockingQueue supportQueue = + enableForceWriteThreadSuspension(forceWriteThreadSuspendedLatch, journal); + journal.start(); + + LogMark lastLogMarkBeforeWrite = journal.getLastLogMark().markLog().getCurMark(); + CountDownLatch latch = new CountDownLatch(1); + long ledgerId = 1; + long entryId = 0; + long startTime = System.currentTimeMillis(); + journal.logAddEntry(ledgerId, entryId, DATA, true /* ackBeforeSync */, new WriteCallback() { + @Override + public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + latch.countDown(); + } + }, null); + + while (supportQueue.isEmpty()) { + Thread.sleep(100); + } + + // forceWriteRequest insert into forceWriteRequestQueue not effected by journalPageCacheFlushInterval + assertTrue(System.currentTimeMillis() - startTime < 5000); + assertEquals(1, supportQueue.size()); + + // callback should completed now + assertTrue(latch.await(20, TimeUnit.SECONDS)); + + // in constructor of JournalChannel we are calling forceWrite(true) but it is not tracked by PowerMock + // because the 'spy' is applied only on return from the constructor + verify(jc, times(0)).forceWrite(true); + + // we are never calling foreWrite + verify(jc, times(0)).forceWrite(false); + + // verify that log marker did not advance + LastLogMark lastLogMarkAfterForceWrite = journal.getLastLogMark(); + assertEquals(0, lastLogMarkAfterForceWrite.getCurMark().compare(lastLogMarkBeforeWrite)); + + // let the forceWriteThread exit + forceWriteThreadSuspendedLatch.countDown(); + + journal.shutdown(); + } + + @Test + public void testAckBeforeUnSyncPageCacheFlush() throws Exception { + File journalDir = tempDir.newFolder(); + Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir)); + + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setJournalDirName(journalDir.getPath()) + .setMetadataServiceUri(null) + .setJournalAdaptiveGroupWrites(false) + .setJournalSyncData(false) + .setJournalPageCacheFlushIntervalMSec(5000); + + JournalChannel jc = spy(new JournalChannel(journalDir, 1)); + whenNew(JournalChannel.class).withAnyArguments().thenReturn(jc); + + LedgerDirsManager ledgerDirsManager = mock(LedgerDirsManager.class); + Journal journal = new Journal(0, journalDir, conf, ledgerDirsManager); + + CountDownLatch forceWriteThreadSuspendedLatch = new CountDownLatch(1); + LinkedBlockingQueue supportQueue = + enableForceWriteThreadSuspension(forceWriteThreadSuspendedLatch, journal); + journal.start(); + + CountDownLatch latch = new CountDownLatch(2); + long ledgerId = 1; + long entryId = 0; + LogMark lastLogMarkBeforeWrite = journal.getLastLogMark().markLog().getCurMark(); + journal.logAddEntry(ledgerId, entryId, DATA, true, new WriteCallback() { + @Override + public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + latch.countDown(); + } + }, null); + + // the forceWriteRequest should not generated because of journalPageCacheFlushIntervalMSec control + assertEquals(0, supportQueue.size()); + + // wait journalPageCacheFlushIntervalMsec timeout + Thread.sleep(10000); + + // add an entry to journal, wake up journal main thread which is blocked on queue.take() + journal.logAddEntry(ledgerId, entryId + 1, DATA, true, new WriteCallback() { + @Override + public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + latch.countDown(); + } + }, null); + + // wait forceWriteRequest generated + while (supportQueue.isEmpty()) { + Thread.sleep(100); + } + + // only one forceWriteRequest inserted into forceWriteRequestQueue + assertEquals(1, supportQueue.size()); + + // callback should completed now + assertTrue(latch.await(20, TimeUnit.SECONDS)); + + // in constructor of JournalChannel we are calling forceWrite(true) but it is not tracked by PowerMock + // because the 'spy' is applied only on return from the constructor + verify(jc, times(0)).forceWrite(true); + + // we are never calling foreWrite + verify(jc, times(0)).forceWrite(false); + + // verify that log marker did not advance + LastLogMark lastLogMarkAfterForceWrite = journal.getLastLogMark(); + assertEquals(0, lastLogMarkAfterForceWrite.getCurMark().compare(lastLogMarkBeforeWrite)); + + // let the forceWriteThread exit + forceWriteThreadSuspendedLatch.countDown(); + + journal.shutdown(); + } +} diff --git a/conf/bk_server.conf b/conf/bk_server.conf index ddd74e5ee40..a3cf3452b1a 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -343,6 +343,9 @@ journalDirectories=/tmp/bk-txn # Set the size of the journal queue. # journalQueueSize=10000 +# Set PageCache flush interval (millisecond) when journalSyncData disabled +# journalPageCacheFlushIntervalMSec = 1000 + ############################################################################# ## Ledger storage settings ############################################################################# From 7f53e44eb7a32afd38535e885820873e64b45a4f Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 19 Mar 2020 00:10:18 +0100 Subject: [PATCH 0396/1642] Do not log WARNING about PlacementPolicy for tests and simple setups Descriptions of the changes in this PR: Do not log 'New ensemble: {} is not adhering to Placement Policy. quarantinedBookies' in case of ensembleSize = 1 ### Motivation In tests cases and in simple demos it is very annoying to see that log line at "WARNING" level. If we are requiring only 1 bookie it is not a big deal to WARN that the placement policy is not "adhering" Reviewers: Matteo Minardi , Atri Sharma , Sijie Guo This closes #2286 from eolivelli/fix/no-log-single-bookie --- .../org/apache/bookkeeper/client/BookieWatcherImpl.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index e38dcf6d787..f18799c93be 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -238,7 +238,7 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi long startTime = MathUtils.nowInNano(); EnsemblePlacementPolicy.PlacementResult> newEnsembleResponse; List socketAddresses; - PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = PlacementPolicyAdherence.FAIL; + PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; try { Set quarantinedBookiesSet = quarantinedBookies.asMap().keySet(); newEnsembleResponse = placementPolicy.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, @@ -247,8 +247,10 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.isAdheringToPolicy(); if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) { ensembleNotAdheringToPlacementPolicy.inc(); - log.warn("New ensemble: {} is not adhering to Placement Policy. quarantinedBookies: {}", - socketAddresses, quarantinedBookiesSet); + if (ensembleSize > 1) { + log.warn("New ensemble: {} is not adhering to Placement Policy. quarantinedBookies: {}", + socketAddresses, quarantinedBookiesSet); + } } // we try to only get from the healthy bookies first newEnsembleTimer.registerSuccessfulEvent(MathUtils.nowInNano() - startTime, TimeUnit.NANOSECONDS); From 04dd1e950fea1a1bcacf5e6a2b2899ffb1f2e112 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 18 Mar 2020 16:32:42 -0700 Subject: [PATCH 0397/1642] [BOOKIE-MTLS] add BouncyCastleProvider for security-provider to avoid InvalidKeyException ### Motivation As described at: https://github.com/apache/pulsar/issues/5047 ### Issue Sometimes user sees `Invalid TLS configuration` at bookie while loading PKCS8Key file and that can be fixed by using Bouncycastle provider.: https://stackoverflow.com/questions/6559272/algid-parse-error-not-a-sequence/18912362#18912362 ``` 2019-08-26 16:16:51,983 - ERROR - [BookKeeperClientWorker-OrderedExecutor-0-0:BookieClient179] - Security Exception in creating new default PCBC pool: org.apache.bookkeeper.tls.SecurityException: Invalid TLS configuration at org.apache.bookkeeper.tls.TLSContextFactory.init(TLSContextFactory.java:392) at org.apache.bookkeeper.proto.PerChannelBookieClient.(PerChannelBookieClient.java:266) at org.apache.bookkeeper.proto.BookieClient.create(BookieClient.java:155) at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.(DefaultPerChannelBookieClientPool.java:71) at org.apache.bookkeeper.proto.BookieClient.lookupClient(BookieClient.java:168) at org.apache.bookkeeper.proto.BookieClient.addEntry(BookieClient.java:245) at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:131) at org.apache.bookkeeper.client.PendingAddOp.safeRun(PendingAddOp.java:240) at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.IllegalArgumentException: File does not contain valid private key: /my.key.pem at io.netty.handler.ssl.SslContextBuilder.keyManager(SslContextBuilder.java:267) at org.apache.bookkeeper.tls.TLSContextFactory.createClientContext(TLSContextFactory.java:244) at org.apache.bookkeeper.tls.TLSContextFactory.init(TLSContextFactory.java:363) ... 12 more Caused by: java.security.spec.InvalidKeySpecException: Neither RSA, DSA nor EC worked at io.netty.handler.ssl.SslContext.getPrivateKeyFromByteBuffer(SslContext.java:1045) at io.netty.handler.ssl.SslContext.toPrivateKey(SslContext.java:1014) at io.netty.handler.ssl.SslContextBuilder.keyManager(SslContextBuilder.java:265) ... 14 more Caused by: java.security.spec.InvalidKeySpecException: java.security.InvalidKeyException: IOException : algid parse error, not a sequence at sun.security.ec.ECKeyFactory.engineGeneratePrivate(ECKeyFactory.java:169) at java.security.KeyFactory.generatePrivate(KeyFactory.java:372) at io.netty.handler.ssl.SslContext.getPrivateKeyFromByteBuffer(SslContext.java:1043) ... 16 more Caused by: java.security.InvalidKeyException: IOException : algid parse error, not a sequence at sun.security.pkcs.PKCS8Key.decode(PKCS8Key.java:351) at sun.security.pkcs.PKCS8Key.decode(PKCS8Key.java:356) at sun.security.ec.ECPrivateKeyImpl.(ECPrivateKeyImpl.java:73) at sun.security.ec.ECKeyFactory.implGeneratePrivate(ECKeyFactory.java:237) at sun.security.ec.ECKeyFactory.engineGeneratePrivate(ECKeyFactory.java:165) ... 18 more ``` Reviewers: Enrico Olivelli , Sijie Guo This closes #2151 from rdhabalia/bc-mtls --- bookkeeper-dist/src/assemble/bin-all.xml | 1 + bookkeeper-dist/src/assemble/bin-server.xml | 1 + bookkeeper-dist/src/assemble/bkctl.xml | 1 + .../src/main/resources/LICENSE-all.bin.txt | 8 +++++++ .../src/main/resources/LICENSE-bkctl.bin.txt | 9 ++++++++ .../src/main/resources/LICENSE-server.bin.txt | 8 +++++++ .../deps/bouncycastle-1.60/LICENSE.html | 22 +++++++++++++++++++ bookkeeper-server/pom.xml | 8 +++++++ .../bookkeeper/tls/TLSContextFactory.java | 6 +++++ pom.xml | 13 ++++++++++- 10 files changed, 76 insertions(+), 1 deletion(-) create mode 100644 bookkeeper-dist/src/main/resources/deps/bouncycastle-1.60/LICENSE.html diff --git a/bookkeeper-dist/src/assemble/bin-all.xml b/bookkeeper-dist/src/assemble/bin-all.xml index 6e638eda756..3e24704fd9a 100644 --- a/bookkeeper-dist/src/assemble/bin-all.xml +++ b/bookkeeper-dist/src/assemble/bin-all.xml @@ -58,6 +58,7 @@ google-auth-library-credentials-0.9.0/LICENSE javax.servlet-api-3.1.0/CDDL+GPL-1.1 jline-2.11/LICENSE + bouncycastle-1.60/LICENSE.html jsr-305/LICENSE netty-3.10.1.Final/* netty-4.1.32.Final/* diff --git a/bookkeeper-dist/src/assemble/bin-server.xml b/bookkeeper-dist/src/assemble/bin-server.xml index 57cf40bcc71..fe74c1d6c1a 100644 --- a/bookkeeper-dist/src/assemble/bin-server.xml +++ b/bookkeeper-dist/src/assemble/bin-server.xml @@ -55,6 +55,7 @@ netty-4.1.32.Final/* protobuf-3.0.0/LICENSE jline-2.11/LICENSE + bouncycastle-1.60/LICENSE.html protobuf-3.5.1/LICENSE slf4j-1.7.25/LICENSE.txt diff --git a/bookkeeper-dist/src/assemble/bkctl.xml b/bookkeeper-dist/src/assemble/bkctl.xml index 4f2d92fa994..6e6498bfc4c 100644 --- a/bookkeeper-dist/src/assemble/bkctl.xml +++ b/bookkeeper-dist/src/assemble/bkctl.xml @@ -69,6 +69,7 @@ netty-4.1.32.Final/* protobuf-3.0.0/LICENSE jline-2.11/LICENSE + bouncycastle-1.60/LICENSE.html protobuf-3.5.1/LICENSE slf4j-1.7.25/LICENSE.txt diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index d6cc2dc0ceb..a13d26cc0c9 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -467,6 +467,14 @@ license. For details, see deps/jline-2.11/LICENSE Bundled as - lib/jline-jline-2.11.jar ------------------------------------------------------------------------------------ +This product bundles the bouncycastle Library. +For license details, see deps/bouncycastle-1.60/LICENSE.html + +Bundled as + - lib/org.bouncycastle-bcpkix-jdk15on-1.60.jar + - lib/org.bouncycastle-bcprov-ext-jdk15on-1.60.jar + - lib/org.bouncycastle-bcprov-jdk15on-1.60.jar +------------------------------------------------------------------------------------ This product uses the annotations from The Checker Framework, which are licensed under MIT License. For details, see deps/checker-compat-qual-2.5.2/LICENSE diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt index 1fe42dd65bd..8abca03cd9d 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -391,6 +391,15 @@ license. For details, see deps/jline-2.11/LICENSE Bundled as - lib/jline-jline-2.11.jar ------------------------------------------------------------------------------------ +This product bundles the bouncycastle Library. +For license details, see deps/bouncycastle-1.60/LICENSE.html + +Bundled as + - lib/org.bouncycastle-bcpkix-jdk15on-1.60.jar + - lib/org.bouncycastle-bcprov-ext-jdk15on-1.60.jar + - lib/org.bouncycastle-bcprov-jdk15on-1.60.jar +------------------------------------------------------------------------------------ + This product uses the annotations from The Checker Framework, which are licensed under MIT License. For details, see deps/checker-compat-qual-2.5.2/LICENSE diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index 15f5962f55b..912f730a066 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -456,6 +456,14 @@ license. For details, see deps/jline-2.11/LICENSE Bundled as - lib/jline-jline-2.11.jar ------------------------------------------------------------------------------------ +This product bundles the bouncycastle Library. +For license details, see deps/bouncycastle-1.60/LICENSE.html + +Bundled as + - lib/org.bouncycastle-bcpkix-jdk15on-1.60.jar + - lib/org.bouncycastle-bcprov-ext-jdk15on-1.60.jar + - lib/org.bouncycastle-bcprov-jdk15on-1.60.jar +------------------------------------------------------------------------------------ This product uses the annotations from The Checker Framework, which are licensed under MIT License. For details, see deps/checker-compat-qual-2.5.2/LICENSE diff --git a/bookkeeper-dist/src/main/resources/deps/bouncycastle-1.60/LICENSE.html b/bookkeeper-dist/src/main/resources/deps/bouncycastle-1.60/LICENSE.html new file mode 100644 index 00000000000..a3acf7f069d --- /dev/null +++ b/bookkeeper-dist/src/main/resources/deps/bouncycastle-1.60/LICENSE.html @@ -0,0 +1,22 @@ + + + +Copyright (c) 2000-2019 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org) +

            +Permission is hereby granted, free of charge, to any person obtaining a copy of this software +and associated documentation files (the "Software"), to deal in the Software without restriction, +including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, +and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, +subject to the following conditions: +

            +The above copyright notice and this permission notice shall be included in all copies or substantial +portions of the Software. +

            +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, +INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR +PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR +OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + + \ No newline at end of file diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index 98748333f1b..d2f4f030146 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -100,6 +100,14 @@ org.apache.commons commons-collections4 + + org.bouncycastle + bcpkix-jdk15on + + + org.bouncycastle + bcprov-ext-jdk15on + com.beust jcommander diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java index b0bc57f3c60..1b6b30e1c76 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java @@ -56,6 +56,12 @@ * A factory to manage TLS contexts. */ public class TLSContextFactory implements SecurityHandlerFactory { + + static { + // Fixes loading PKCS8Key file: https://stackoverflow.com/a/18912362 + java.security.Security.addProvider(new org.bouncycastle.jce.provider.BouncyCastleProvider()); + } + /** * Supported Key File Types. */ diff --git a/pom.xml b/pom.xml index 0189bd5fade..7a4bea0bce9 100644 --- a/pom.xml +++ b/pom.xml @@ -123,6 +123,7 @@ 2.6 3.6 2.4 + 1.60 4.0.1 3.1.0 0.0.2 @@ -300,7 +301,17 @@ commons-lang3 ${commons-lang3.version} - + + + org.bouncycastle + bcpkix-jdk15on + ${bouncycastle.version} + + + org.bouncycastle + bcprov-ext-jdk15on + ${bouncycastle.version} + org.reflections From af3903c258506fee2c9cff7c94a13a1036bc77d2 Mon Sep 17 00:00:00 2001 From: Aloys <18124766348@163.com> Date: Thu, 19 Mar 2020 07:51:46 +0800 Subject: [PATCH 0398/1642] 2279 fix NPE when exec readjournal command Descriptions of the changes in this PR: This PR fix NullPointerException when exec readjournal command See issue #2279 Reviewers: Enrico Olivelli , Sijie Guo This closes #2280 from aloyszhang/read-jounral --- .../src/main/java/org/apache/bookkeeper/bookie/BookieShell.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 28ef757895a..d7f557520b2 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 @@ -1013,7 +1013,7 @@ class ReadJournalCmd extends MyCommand { ReadJournalCmd() { super(CMD_READJOURNAL); - rjOpts.addOption("dir", false, "Journal directory (needed if more than one journal configured)"); + rjOpts.addOption("dir", true, "Journal directory (needed if more than one journal configured)"); rjOpts.addOption("m", "msg", false, "Print message body"); } From 9de52db12e1d869e06bb5f6de798efdb8f84456a Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 23 Mar 2020 12:28:41 +0100 Subject: [PATCH 0399/1642] Remove old Jenkins stuff (#2293) Remove old Jenkins stuff for GitHub Create Pull Request template Co-authored-by: Enrico Olivelli --- .github/PULL_REQUEST_TEMPLATE.md | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 1e1991bd86e..84aacc432cc 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -14,21 +14,8 @@ Master Issue: # > --- > In order to uphold a high standard for quality for code contributions, Apache BookKeeper runs various precommit -> checks for pull requests. A pull request can only be merged when it passes precommit checks. However running all -> the precommit checks can take a long time, some trivial changes don't need to run all the precommit checks. You -> can check following list to skip the tests that don't need to run for your pull request. Leave them unchecked if -> you are not sure, committers will help you: +> checks for pull requests. A pull request can only be merged when it passes precommit checks. > -> - [ ] [skip bookkeeper-server bookie tests]: skip testing `org.apache.bookkeeper.bookie` in bookkeeper-server module. -> - [ ] [skip bookkeeper-server client tests]: skip testing `org.apache.bookkeeper.client` in bookkeeper-server module. -> - [ ] [skip bookkeeper-server replication tests]: skip testing `org.apache.bookkeeper.replication` in bookkeeper-server module. -> - [ ] [skip bookkeeper-server tls tests]: skip testing `org.apache.bookkeeper.tls` in bookkeeper-server module. -> - [ ] [skip bookkeeper-server remaining tests]: skip testing all other tests in bookkeeper-server module. -> - [ ] [skip integration tests]: skip docker based integration tests. if you make java code changes, you shouldn't skip integration tests. -> - [ ] [skip build java8]: skip build on java8. *ONLY* skip this when *ONLY* changing files under documentation under `site`. -> - [ ] [skip build java11]: skip build on java11. *ONLY* skip this when *ONLY* changing files under documentation under `site`. -> --- - > --- > Be sure to do all of the following to help us incorporate your contribution > quickly and easily: From 7b70ed4a5ef846ae8980dd3e0fb3099a43847532 Mon Sep 17 00:00:00 2001 From: dmercuriali Date: Mon, 23 Mar 2020 12:29:13 +0100 Subject: [PATCH 0400/1642] #2288: add metadata to StreamStorage ledgers (#2296) Descriptions of the changes in this PR: Added 'application' and 'component' metadata to StreamStorage ledgers. application: bk-stream-storage-service component: state-store / checkpoint-store Master Issue: #2288 * #2288 add metadata to StreamStorage ledgers * #2288 add metadata to StreamStorage ledgers Co-authored-by: Dennis Mercuriali --- .../apache/bookkeeper/statelib/impl/Constants.java | 1 + .../impl/journal/AbstractStateStoreWithJournal.java | 12 ++++++++++-- .../rocksdb/checkpoint/dlog/DLCheckpointStore.java | 7 ++++++- 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/Constants.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/Constants.java index eb5a1b3e394..d4ce08b9d9c 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/Constants.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/Constants.java @@ -27,6 +27,7 @@ @NoArgsConstructor(access = AccessLevel.PRIVATE) public final class Constants { + public static final String LEDGER_METADATA_APPLICATION_STREAM_STORAGE = "bk-stream-storage-service"; public static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; public static final byte[] NULL_START_KEY = EMPTY_BYTE_ARRAY; public static final byte[] NULL_END_KEY = new byte[] { 0 }; diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/journal/AbstractStateStoreWithJournal.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/journal/AbstractStateStoreWithJournal.java index 1ab5da6e0a4..9e566be453d 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/journal/AbstractStateStoreWithJournal.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/journal/AbstractStateStoreWithJournal.java @@ -42,6 +42,7 @@ import org.apache.bookkeeper.statelib.api.exceptions.StateStoreClosedException; import org.apache.bookkeeper.statelib.api.exceptions.StateStoreException; import org.apache.bookkeeper.statelib.api.exceptions.StateStoreRuntimeException; +import org.apache.bookkeeper.statelib.impl.Constants; import org.apache.distributedlog.DLSN; import org.apache.distributedlog.LogRecord; import org.apache.distributedlog.LogRecordWithDLSN; @@ -49,6 +50,7 @@ import org.apache.distributedlog.api.AsyncLogWriter; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.namespace.Namespace; +import org.apache.distributedlog.bk.LedgerMetadata; import org.apache.distributedlog.exceptions.LogEmptyException; import org.apache.distributedlog.exceptions.LogNotFoundException; import org.apache.distributedlog.util.Utils; @@ -218,7 +220,10 @@ private CompletableFuture initializeJournalWriter(StateStoreSpec spec) { } catch (IOException e) { return FutureUtils.exception(e); } - return logManager.openAsyncLogWriter().thenComposeAsync(w -> { + LedgerMetadata metadata = new LedgerMetadata(); + metadata.setApplication(Constants.LEDGER_METADATA_APPLICATION_STREAM_STORAGE); + metadata.setComponent("state-store"); + return logManager.openAsyncLogWriter(metadata).thenComposeAsync(w -> { synchronized (this) { writer = w; nextRevision = writer.getLastTxId(); @@ -233,7 +238,10 @@ private CompletableFuture initializeJournalWriter(StateStoreSpec spec) { } private CompletableFuture writeCatchUpMarker() { - return logManager.openAsyncLogWriter().thenComposeAsync(w -> { + LedgerMetadata metadata = new LedgerMetadata(); + metadata.setApplication(Constants.LEDGER_METADATA_APPLICATION_STREAM_STORAGE); + metadata.setComponent("state-store"); + return logManager.openAsyncLogWriter(metadata).thenComposeAsync(w -> { synchronized (this) { writer = w; nextRevision = writer.getLastTxId(); diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStore.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStore.java index 8dc04472126..0e2de490a9b 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStore.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStore.java @@ -32,11 +32,13 @@ import java.util.concurrent.ExecutionException; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.statelib.api.checkpoint.CheckpointStore; +import org.apache.bookkeeper.statelib.impl.Constants; import org.apache.distributedlog.DLSN; import org.apache.distributedlog.api.AsyncLogWriter; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; import org.apache.distributedlog.api.namespace.Namespace; +import org.apache.distributedlog.bk.LedgerMetadata; import org.apache.distributedlog.exceptions.DLInterruptedException; import org.apache.distributedlog.exceptions.LogEmptyException; import org.apache.distributedlog.exceptions.LogExistsException; @@ -98,7 +100,10 @@ public OutputStream openOutputStream(String filePath) throws IOException { try { DistributedLogManager dlm = namespace.openLog( filePath); - AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter()); + LedgerMetadata metadata = new LedgerMetadata(); + metadata.setApplication(Constants.LEDGER_METADATA_APPLICATION_STREAM_STORAGE); + metadata.setComponent("checkpoint-store"); + AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter(metadata)); return new BufferedOutputStream( new DLOutputStream(dlm, writer), 128 * 1024); } catch (LogNotFoundException le) { From c5cb20a05ed9a330e952de1044276f77dd281ec6 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Mon, 23 Mar 2020 09:02:26 -0700 Subject: [PATCH 0401/1642] [BOOKIE-LEDGER-RECOVERY] Fix bookie recovery stuck even with enough ack-quorum response ### Motivation As discussed at https://github.com/apache/pulsar/issues/6505 Bk-client was not able to recover ledger which has 2 write/ack quorum and one of the bookie went down and recovery was kept failing and bookkeeper client was not able to recover the ledger. **BK-Client log** ``` 20:44:43.721 [BookKeeperClientWorker-OrderedExecutor-1-0] ERROR org.apache.bookkeeper.client.ReadLastConfirmedOp - While readLastConfirmed ledger: 1234567 did not hear success responses from all quorums 20:44:43.721 [bookkeeper-io-12-27] ERROR org.apache.bookkeeper.proto.PerChannelBookieClient - Could not connect to bookie: [id: 0xb8b97441, L:/1.1.1.1:1234]/1.1.1.2:3181, current s tate CONNECTING : io.netty.channel.AbstractChannel$AnnotatedConnectException: finishConnect(..) failed: No route to host: /1.1.1.2:3181 at io.netty.channel.unix.Errors.throwConnectException(Errors.java:112) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.unix.Socket.finishConnect(Socket.java:269) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.doFinishConnect(AbstractEpollChannel.java:665) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:642) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:522) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:423) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:330) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897) [netty-common-4.1.31.Final.jar:4.1.31.Final] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [netty-common-4.1.31.Final.jar:4.1.31.Final] at java.lang.Thread.run(Thread.java:834) [?:?] Caused by: java.net.ConnectException: finishConnect(..) failed: No route to host ``` **Ledger metadata** ``` BookieMetadataFormatVersion 2 quorumSize: 2 ensembleSize: 2 length: 0 lastEntryId: -1 state: IN_RECOVERY segment { ensembleMember: "1.1.1.1:3181" ensembleMember: "1.1.1.2:3181" firstEntryId: 0 } digestType: CRC32 ``` **Root cause:** Bookie should be able to recover ledger once it receives the response from total N (`(Qw - Qa)+1`) bookies. But it was waiting for a successful response from both quorums. Reference: https://bookkeeper.apache.org/docs/4.5.0/development/protocol/ ### Modification Bookie should be able to recover ledger once it receives the response from total N (`(Qw - Qa)+1`) bookies. Reviewers: Diego Salvi , Enrico Olivelli , Sijie Guo This closes #2281 from rdhabalia/recover_q --- .../client/ReadLastConfirmedOp.java | 12 ++- .../client/ParallelLedgerRecoveryTest.java | 90 +++++++++++++++++++ .../bookkeeper/test/BookieFailureTest.java | 18 ++-- 3 files changed, 107 insertions(+), 13 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java index dfbc1672f00..2b75403812d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java @@ -38,6 +38,7 @@ class ReadLastConfirmedOp implements ReadEntryCallback { LedgerHandle lh; BookieClient bookieClient; int numResponsesPending; + int numSuccessfulResponse; RecoveryData maxRecoveredData; volatile boolean completed = false; int lastSeenError = BKException.Code.ReadException; @@ -59,6 +60,7 @@ public ReadLastConfirmedOp(LedgerHandle lh, BookieClient bookieClient, this.bookieClient = bookieClient; this.maxRecoveredData = new RecoveryData(LedgerHandle.INVALID_ENTRY_ID, 0); this.lh = lh; + this.numSuccessfulResponse = 0; this.numResponsesPending = lh.getLedgerMetadata().getEnsembleSize(); this.coverageSet = lh.distributionSchedule.getCoverageSet(); this.currentEnsemble = ensemble; @@ -99,6 +101,7 @@ public synchronized void readEntryComplete(final int rc, final long ledgerId, fi maxRecoveredData = recoveryData; } heardValidResponse = true; + numSuccessfulResponse++; } catch (BKDigestMatchException e) { // Too bad, this bookie didn't give us a valid answer, we // still might be able to recover though so continue @@ -137,8 +140,15 @@ public synchronized void readEntryComplete(final int rc, final long ledgerId, fi } if (numResponsesPending == 0 && !completed) { + int totalExepctedResponse = lh.getLedgerMetadata().getWriteQuorumSize() + - lh.getLedgerMetadata().getAckQuorumSize() + 1; + if (numSuccessfulResponse >= totalExepctedResponse) { + cb.readLastConfirmedDataComplete(BKException.Code.OK, maxRecoveredData); + return; + } // Have got all responses back but was still not enough, just fail the operation - LOG.error("While readLastConfirmed ledger: {} did not hear success responses from all quorums", ledgerId); + LOG.error("While readLastConfirmed ledger: {} did not hear success responses from all quorums {}", ledgerId, + lastSeenError); cb.readLastConfirmedDataComplete(lastSeenError, maxRecoveredData); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index 177cbe38df3..ab69fa75254 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -66,6 +66,7 @@ import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.zookeeper.AsyncCallback.VoidCallback; import org.apache.zookeeper.KeeperException; import org.junit.After; @@ -693,4 +694,93 @@ public void readLastConfirmedDataComplete(int rc, DigestManager.RecoveryData dat readBk.close(); } + /** + * Validate ledger can recover with response: (Qw - Qa)+1. + * @throws Exception + */ + @Test + public void testRecoveryWithUnavailableBookie() throws Exception { + + byte[] passwd = "".getBytes(UTF_8); + ClientConfiguration newConf = new ClientConfiguration(); + newConf.addConfiguration(baseClientConf); + final BookKeeper readBk = new BookKeeper(newConf); + final BookKeeper newBk0 = new BookKeeper(newConf); + + /** + * Test Group-1 : Expected Response for recovery: Qr = (Qw - Qa)+1 = (3 + * -2) + 1 = 2 + */ + int ensembleSize = 3; + int writeQuorumSize = 3; + int ackQuormSize = 2; + LedgerHandle lh0 = newBk0.createLedger(ensembleSize, writeQuorumSize, ackQuormSize, DigestType.DUMMY, passwd); + LedgerHandle readLh = readBk.openLedgerNoRecovery(lh0.getId(), DigestType.DUMMY, passwd); + // Test 1: bookie response: OK, NO_SUCH_LEDGER_EXISTS, NOT_AVAILABLE + // Expected: Recovery successful Q(response) = 2 + int responseCode = readLACFromQuorum(readLh, BKException.Code.BookieHandleNotAvailableException, + BKException.Code.OK, BKException.Code.NoSuchLedgerExistsException); + assertEquals(responseCode, BKException.Code.OK); + // Test 2: bookie response: OK, NOT_AVAILABLE, NOT_AVAILABLE + // Expected: Recovery fail Q(response) = 1 + responseCode = readLACFromQuorum(readLh, BKException.Code.BookieHandleNotAvailableException, + BKException.Code.OK, BKException.Code.BookieHandleNotAvailableException); + assertEquals(responseCode, BKException.Code.BookieHandleNotAvailableException); + + /** + * Test Group-2 : Expected Response for recovery: Qr = (Qw - Qa)+1 = (2 + * -2) + 1 = 1 + */ + ensembleSize = 2; + writeQuorumSize = 2; + ackQuormSize = 2; + lh0 = newBk0.createLedger(ensembleSize, writeQuorumSize, ackQuormSize, DigestType.DUMMY, passwd); + readLh = readBk.openLedgerNoRecovery(lh0.getId(), DigestType.DUMMY, passwd); + // Test 1: bookie response: OK, NOT_AVAILABLE + // Expected: Recovery successful Q(response) = 1 + responseCode = readLACFromQuorum(readLh, BKException.Code.BookieHandleNotAvailableException, + BKException.Code.OK); + assertEquals(responseCode, BKException.Code.OK); + + // Test 1: bookie response: OK, NO_SUCH_LEDGER_EXISTS + // Expected: Recovery successful Q(response) = 2 + responseCode = readLACFromQuorum(readLh, BKException.Code.NoSuchLedgerExistsException, BKException.Code.OK); + assertEquals(responseCode, BKException.Code.OK); + + // Test 3: bookie response: NOT_AVAILABLE, NOT_AVAILABLE + // Expected: Recovery fail Q(response) = 0 + responseCode = readLACFromQuorum(readLh, BKException.Code.BookieHandleNotAvailableException, + BKException.Code.BookieHandleNotAvailableException); + assertEquals(responseCode, BKException.Code.BookieHandleNotAvailableException); + + newBk0.close(); + readBk.close(); + } + + private int readLACFromQuorum(LedgerHandle ledger, int... bookieLACResponse) throws Exception { + MutableInt responseCode = new MutableInt(100); + CountDownLatch responseLatch = new CountDownLatch(1); + ReadLastConfirmedOp readLCOp = new ReadLastConfirmedOp(ledger, bkc.getBookieClient(), + ledger.getLedgerMetadata().getAllEnsembles().lastEntry().getValue(), + new ReadLastConfirmedOp.LastConfirmedDataCallback() { + @Override + public void readLastConfirmedDataComplete(int rc, DigestManager.RecoveryData data) { + System.out.println("response = " + rc); + responseCode.setValue(rc); + responseLatch.countDown(); + } + }); + byte[] lac = new byte[Long.SIZE * 3]; + ByteBuf data = Unpooled.wrappedBuffer(lac, 0, lac.length); + int writerIndex = data.writerIndex(); + data.resetWriterIndex(); + data.writeLong(ledger.getId()); + data.writeLong(0L); + data.writerIndex(writerIndex); + for (int i = 0; i < bookieLACResponse.length; i++) { + readLCOp.readEntryComplete(bookieLACResponse[i], 0, 0, data, i); + } + responseLatch.await(); + return responseCode.intValue(); + } } 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 8a18127a297..255e4f36007 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 @@ -313,12 +313,9 @@ public void testLedgerNoRecoveryOpenAfterBKCrashed() throws Exception { killBookie(0); // try to open ledger no recovery - try { - bkc.openLedgerNoRecovery(beforelh2.getId(), digestType, "".getBytes()); - fail("Should have thrown exception"); - } catch (BKException.BKReadException e) { - // correct behaviour - } + // should be able to open ledger with one bookie down: + // Total bookies available 3 >= 1 (Qw(4) - Qa(4) + 1) + bkc.openLedgerNoRecovery(beforelh2.getId(), digestType, "".getBytes()); } @Test @@ -351,12 +348,9 @@ public void testLedgerOpenAfterBKCrashed() throws Exception { killBookie(0); // try to open ledger no recovery - try { - bkc.openLedger(beforelh2.getId(), digestType, "".getBytes()); - fail("Should have thrown exception"); - } catch (BKException.BKLedgerRecoveryException e) { - // correct behaviour - } + // should be able to open ledger with one bookie down: + // Total bookies available 3 >= 1 (Qw(4) - Qa(4) + 1) + bkc.openLedger(beforelh2.getId(), digestType, "".getBytes()); } /** From c3e4266634f81ab2409f81411872889dce24d14c Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 24 Mar 2020 19:22:41 +0100 Subject: [PATCH 0402/1642] Update Arquillian Cube to 1.18.2 ### Motivation Update Arquillian to latest version Reviewers: Rajan Dhabalia This closes #2297 from eolivelli/fix/update-arquillian --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 7a4bea0bce9..b173421528c 100644 --- a/pom.xml +++ b/pom.xml @@ -112,8 +112,8 @@ 2 - 1.15.1 - 1.1.14.Final + 1.18.2 + 1.6.0.Final 3.0.1 1.2 4.1 From 202307c94d1284071ac308235d67e31ef05fed90 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sun, 29 Mar 2020 17:14:05 -0700 Subject: [PATCH 0403/1642] Unable to enable TLS if using v2 protocol Descriptions of the changes in this PR: *Motivation* TLS is enabled using `startTLS` but the `startTLS` is a v3 protobuf request. So if the client is configured to use v2 protocol, the client is not able to decode `startTLSResponse`. *Modifications* This change is to improve the v2 protocol response handling for handling `startTLS` response. Reviewers: Enrico Olivelli This closes #2300 from sijie/fix_tls_v2_protocol --- .../bookkeeper/proto/BookieProtoEncoding.java | 46 +++++++++++++++++-- .../proto/PerChannelBookieClient.java | 2 +- .../proto/BookieProtoEncodingTest.java | 2 +- .../org/apache/bookkeeper/tls/TestTLS.java | 36 ++++++++++----- 4 files changed, 68 insertions(+), 18 deletions(-) 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 151a799f584..0c237c46c05 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 @@ -41,6 +41,8 @@ import java.security.NoSuchAlgorithmException; import org.apache.bookkeeper.proto.BookieProtocol.PacketHeader; +import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; +import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; import org.apache.bookkeeper.proto.checksum.MacDigestManager; import org.apache.bookkeeper.util.ByteBufList; import org.slf4j.Logger; @@ -479,11 +481,20 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) */ @Sharable public static class ResponseDecoder extends ChannelInboundHandlerAdapter { - final EnDecoder rep; + final EnDecoder repPreV3; + final EnDecoder repV3; + final boolean useV2Protocol; + final boolean tlsEnabled; + boolean usingV3Protocol; - ResponseDecoder(ExtensionRegistry extensionRegistry, boolean useV2Protocol) { - rep = useV2Protocol - ? new ResponseEnDeCoderPreV3(extensionRegistry) : new ResponseEnDecoderV3(extensionRegistry); + ResponseDecoder(ExtensionRegistry extensionRegistry, + boolean useV2Protocol, + boolean tlsEnabled) { + this.repPreV3 = new ResponseEnDeCoderPreV3(extensionRegistry); + this.repV3 = new ResponseEnDecoderV3(extensionRegistry); + this.useV2Protocol = useV2Protocol; + this.tlsEnabled = tlsEnabled; + usingV3Protocol = true; } @Override @@ -499,7 +510,32 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } ByteBuf buffer = (ByteBuf) msg; buffer.markReaderIndex(); - ctx.fireChannelRead(rep.decode(buffer)); + + Object result; + if (!useV2Protocol) { // always use v3 protocol + result = repV3.decode(buffer); + } else { // use v2 protocol but + // if TLS enabled, the first message `startTLS` is a protobuf message + if (tlsEnabled && usingV3Protocol) { + try { + result = repV3.decode(buffer); + if (result instanceof Response + && OperationType.START_TLS == ((Response) result).getHeader().getOperation()) { + usingV3Protocol = false; + if (LOG.isDebugEnabled()) { + LOG.debug("Degrade bookkeeper to v2 after starting TLS."); + } + } + } catch (InvalidProtocolBufferException e) { + usingV3Protocol = false; + buffer.resetReaderIndex(); + result = repPreV3.decode(buffer); + } + } else { + result = repPreV3.decode(buffer); + } + } + ctx.fireChannelRead(result); } finally { ReferenceCountUtil.release(msg); } 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 c151426701f..dd3d130a979 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 @@ -565,7 +565,7 @@ protected void initChannel(Channel ch) throws Exception { pipeline.addLast("bookieProtoEncoder", new BookieProtoEncoding.RequestEncoder(extRegistry)); pipeline.addLast( "bookieProtoDecoder", - new BookieProtoEncoding.ResponseDecoder(extRegistry, useV2WireProtocol)); + new BookieProtoEncoding.ResponseDecoder(extRegistry, useV2WireProtocol, shFactory != null)); pipeline.addLast("authHandler", new AuthHandler.ClientSideHandler(authProviderFactory, txnIdGenerator, connectionPeer, useV2WireProtocol)); pipeline.addLast("mainhandler", PerChannelBookieClient.this); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java index a7f0c9251ad..537c9fcfc75 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java @@ -91,7 +91,7 @@ public void testV3ResponseDecoderNoFallback() throws Exception { ResponseEnDeCoderPreV3 v2Encoder = new ResponseEnDeCoderPreV3(registry); ResponseEnDecoderV3 v3Encoder = new ResponseEnDecoderV3(registry); - ResponseDecoder v3Decoder = new ResponseDecoder(registry, false); + ResponseDecoder v3Decoder = new ResponseDecoder(registry, false, false); try { v3Decoder.channelRead(ctx, v2Encoder.encode(v2Resp, UnpooledByteBufAllocator.DEFAULT) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java index 94dec72fec8..9d00c47a31e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.Assume.assumeTrue; import java.io.File; import java.io.IOException; @@ -66,7 +67,6 @@ import org.apache.commons.io.FileUtils; import org.junit.After; import org.junit.Assert; -import org.junit.Assume; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -93,24 +93,29 @@ public class TestTLS extends BookKeeperClusterTestCase { private KeyStoreType clientTrustStoreFormat; private KeyStoreType serverKeyStoreFormat; private KeyStoreType serverTrustStoreFormat; + private final boolean useV2Protocol; @Parameters public static Collection data() { return Arrays.asList(new Object[][] { - { "JKS", "JKS" }, - { "PEM", "PEM" }, - { "PKCS12", "PKCS12" }, - { "JKS", "PEM" }, - { "PEM", "PKCS12" }, - { "PKCS12", "JKS" } + { "JKS", "JKS", false }, + { "PEM", "PEM", false }, + { "PEM", "PEM", true }, + { "PKCS12", "PKCS12", false }, + { "JKS", "PEM", false }, + { "PEM", "PKCS12", false }, + { "PKCS12", "JKS", false } }); } - public TestTLS(String keyStoreFormat, String trustStoreFormat) { + public TestTLS(String keyStoreFormat, + String trustStoreFormat, + boolean useV2Protocol) { super(3); this.clientKeyStoreFormat = KeyStoreType.valueOf(keyStoreFormat); this.clientTrustStoreFormat = KeyStoreType.valueOf(trustStoreFormat); this.serverKeyStoreFormat = KeyStoreType.valueOf(keyStoreFormat); this.serverTrustStoreFormat = KeyStoreType.valueOf(trustStoreFormat); + this.useV2Protocol = useV2Protocol; } private String getResourcePath(String resource) throws Exception { @@ -123,6 +128,7 @@ public void setUp() throws Exception { /* client configuration */ baseClientConf.setTLSProviderFactoryClass(TLSContextFactory.class.getName()); baseClientConf.setTLSClientAuthentication(true); + baseClientConf.setUseV2WireProtocol(useV2Protocol); switch (clientKeyStoreFormat) { case PEM: @@ -249,7 +255,7 @@ public void testStartTLSServerNoKeyStore() throws Exception { @Test public void testKeyMismatchFailure() throws Exception { // Valid test case only for PEM format keys - Assume.assumeTrue(serverKeyStoreFormat == KeyStoreType.PEM); + assumeTrue(serverKeyStoreFormat == KeyStoreType.PEM); ClientConfiguration clientConf = new ClientConfiguration(baseClientConf); @@ -335,6 +341,10 @@ public void testConnectToTLSClusterTLSClient() throws Exception { */ @Test public void testConnectToLocalTLSClusterTLSClient() throws Exception { + // skip test + if (useV2Protocol) { + return; + } ServerConfiguration serverConf = new ServerConfiguration(); for (ServerConfiguration conf : bsConfs) { conf.setDisableServerSocketBind(true); @@ -351,7 +361,7 @@ public void testConnectToLocalTLSClusterTLSClient() throws Exception { */ @Test public void testRefreshDurationForBookieCerts() throws Exception { - Assume.assumeTrue(serverKeyStoreFormat == KeyStoreType.PEM); + assumeTrue(serverKeyStoreFormat == KeyStoreType.PEM); ServerConfiguration serverConf = new ServerConfiguration(); String originalTlsKeyFilePath = bsConfs.get(0).getTLSKeyStore(); String invalidServerKey = getResourcePath("client-key.pem"); @@ -393,7 +403,7 @@ public void testRefreshDurationForBookieCerts() throws Exception { */ @Test public void testRefreshDurationForBookkeeperClientCerts() throws Exception { - Assume.assumeTrue(serverKeyStoreFormat == KeyStoreType.PEM); + assumeTrue(serverKeyStoreFormat == KeyStoreType.PEM); ClientConfiguration clientConf = new ClientConfiguration(baseClientConf); String originalTlsCertFilePath = baseClientConf.getTLSCertificatePath(); @@ -601,6 +611,10 @@ public void testBookieAuthPluginRequireClientTLSAuthentication() throws Exceptio */ @Test public void testBookieAuthPluginRequireClientTLSAuthenticationLocal() throws Exception { + if (useV2Protocol) { + return; + } + ServerConfiguration serverConf = new ServerConfiguration(baseConf); serverConf.setBookieAuthProviderFactoryClass(AllowOnlyClientsWithX509Certificates.class.getName()); serverConf.setDisableServerSocketBind(true); From c4d83a66e0ae5433e58173b36bb375f6fbd3ffed Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 30 Mar 2020 02:16:40 +0200 Subject: [PATCH 0404/1642] Allow build on JDK14 - upgrade Groovy compiler ### Motivation Current version of Groovy compiler for integration tests do not support JDK14 ### Changes Update Groovy Compiler Update dockerfile-maven-plugin as well Reviewers: Sijie Guo This closes #2298 from eolivelli/fix/build-jdk14 --- pom.xml | 2 +- tests/docker-images/all-released-versions-image/pom.xml | 2 +- tests/integration-tests-base-groovy/pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index b173421528c..278370d75a1 100644 --- a/pom.xml +++ b/pom.xml @@ -174,7 +174,7 @@ 0.12 2.7 4.3.0 - 1.3.7 + 1.4.13 1.6.0 1.6 0.8.0 diff --git a/tests/docker-images/all-released-versions-image/pom.xml b/tests/docker-images/all-released-versions-image/pom.xml index 5fb666cd328..d5896624997 100644 --- a/tests/docker-images/all-released-versions-image/pom.xml +++ b/tests/docker-images/all-released-versions-image/pom.xml @@ -39,7 +39,7 @@ com.spotify dockerfile-maven-plugin - 1.3.7 + 1.4.13 default diff --git a/tests/integration-tests-base-groovy/pom.xml b/tests/integration-tests-base-groovy/pom.xml index 317d70d44f6..f48f8105089 100644 --- a/tests/integration-tests-base-groovy/pom.xml +++ b/tests/integration-tests-base-groovy/pom.xml @@ -31,8 +31,8 @@ Apache BookKeeper :: Tests :: Base module for Arquillian based integration tests using groovy - 3.4.0-01 - 2.5.8-01 + 3.6.0-03 + 3.0.2-02 From 9262d799a1a77e66ce0bc3c2d616bc14313d2fd9 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 31 Mar 2020 12:12:16 -0700 Subject: [PATCH 0405/1642] [bookie-server-config] Avoid considering empty journalDirectories As discussed at : https://github.com/apache/pulsar/issues/6042 ### Issue We have config management tool which manages bookie-server configuration and sets empty value of `journalDirectories` if bookie-server is still wants to use single journal dir: `journalDirectory`. In this case, bookie-server doesn't consider `journalDirectory` value and keeps empty list of `journalDirectories` which stats bookie with invalid configuration. ### Expected behavior Bookie-server should parse empty configuration `journalDirectories=` properly and avoid picking up empty value of `journalDirectories` and in that case, bookie-server should fallback to `journalDirectory` config. Reviewers: Enrico Olivelli , Sijie Guo This closes #2245 from rdhabalia/journal_dir --- .../org/apache/bookkeeper/conf/ServerConfiguration.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 ab49c71cdfa..8cd3ef86640 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 @@ -41,6 +41,7 @@ import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.commons.configuration.ConfigurationException; +import org.apache.commons.lang3.StringUtils; /** * Configuration manages server-side settings. @@ -1096,8 +1097,9 @@ public ServerConfiguration setAllowStorageExpansion(boolean val) { */ public String[] getJournalDirNames() { String[] journalDirs = this.getStringArray(JOURNAL_DIRS); - if (journalDirs == null || journalDirs.length == 0) { - return new String[] {getJournalDirName()}; + if (journalDirs == null || journalDirs.length == 0 + || (journalDirs.length == 1 && StringUtils.isEmpty(journalDirs[0]))) { + return new String[] { getJournalDirName() }; } return journalDirs; } From 4bfac109d15a007a59173976318400f6ae7ded14 Mon Sep 17 00:00:00 2001 From: liudezhi <33149602+liudezhi2098@users.noreply.github.com> Date: Mon, 13 Apr 2020 15:26:07 +0800 Subject: [PATCH 0406/1642] Fix batch read data in cache does not take effect Descriptions of the changes in this PR: ### Motivation Fix batch read data in cache does not take effect ### Changes (Describe: what changes you have made) Master Issue: #2230 Fixes: #2230 Reviewers: Jia Zhai , Enrico Olivelli , Sijie Guo This closes #2231 from liudezhi2098/issue-branch-xxx --- .../bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index e975f160bbe..a6ae8c5bd89 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -466,7 +466,7 @@ private void fillReadAheadCache(long orginalLedgerId, long firstEntryId, long fi long size = 0; while (count < readAheadCacheBatchSize && currentEntryLogId == firstEntryLogId) { - ByteBuf entry = entryLogger.internalReadEntry(orginalLedgerId, -1, currentEntryLocation, + ByteBuf entry = entryLogger.internalReadEntry(orginalLedgerId, firstEntryId, currentEntryLocation, false /* validateEntry */); try { @@ -482,6 +482,7 @@ private void fillReadAheadCache(long orginalLedgerId, long firstEntryId, long fi readCache.put(orginalLedgerId, currentEntryId, entry); count++; + firstEntryId++; size += entry.readableBytes(); currentEntryLocation += 4 + entry.readableBytes(); From 5ebffca56aa81ffa677e89bbaa398787e2c89bec Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 21 Apr 2020 17:00:34 +0200 Subject: [PATCH 0407/1642] Force to use Python grpcio less then 1.26.0 - it looks like a newer version of grpcio has been published and we were using open version range - by forcing a closed ended range of version we workaround the error in integration tests Reviewers: Jia Zhai , Matteo Minardi This closes #2317 from eolivelli/fix/integration-tests-python --- stream/clients/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index b1803af8caf..40268d956d2 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -32,7 +32,7 @@ 'six>=1.10.0', 'pytz', 'futures>=3.2.0;python_version<"3.2"', - 'grpcio>=1.8.2', + 'grpcio<1.26.0,>=1.8.2', 'pymmh3>=0.0.3' ] extras = { From dc90a1006713b046676426456d787b75db2d48c4 Mon Sep 17 00:00:00 2001 From: Matteo Minardi Date: Tue, 21 Apr 2020 22:21:28 +0200 Subject: [PATCH 0408/1642] ISSUE #2274: The 'metaformat' command does not delete 'idgen' znode MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Descriptions of the changes in this PR: ### Motivation After `metaformat`, ledgerId does not count from 0 because,the `metaformat` command does not delete `idgen` znode. ### Changes In the `AbstractZkLedgerManagerFactory` avoid skipping the igen and idgen-long nodes Master Issue: #2274 Reviewers: Enrico Olivelli , Sijie Guo This closes #2315 from mino181295/leadger-metaformat, closes #2274 --- .../meta/AbstractZkLedgerManager.java | 6 ++- .../meta/AbstractZkLedgerManagerFactory.java | 7 +++- .../client/BookKeeperAdminTest.java | 38 +++++++++++++++++++ 3 files changed, 49 insertions(+), 2 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index fa64ada7a2c..808cf8951f9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -580,7 +580,11 @@ public static boolean isSpecialZnode(String znode) { || BookKeeperConstants.LAYOUT_ZNODE.equals(znode) || BookKeeperConstants.INSTANCEID.equals(znode) || BookKeeperConstants.UNDER_REPLICATION_NODE.equals(znode) - || LegacyHierarchicalLedgerManager.IDGEN_ZNODE.equals(znode) + || isLeadgerIdGeneratorZnode(znode); + } + + public static boolean isLeadgerIdGeneratorZnode(String znode) { + return LegacyHierarchicalLedgerManager.IDGEN_ZNODE.equals(znode) || LongHierarchicalLedgerManager.IDGEN_ZNODE.equals(znode) || znode.startsWith(ZkLedgerIdGenerator.LEDGER_ID_GEN_PREFIX); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java index 72bc3e841e6..eef608dd5ec 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerFactory.java @@ -17,6 +17,8 @@ */ package org.apache.bookkeeper.meta; +import static org.apache.bookkeeper.meta.AbstractZkLedgerManager.isLeadgerIdGeneratorZnode; +import static org.apache.bookkeeper.meta.AbstractZkLedgerManager.isSpecialZnode; import java.io.IOException; import java.net.URI; import java.util.List; @@ -48,7 +50,10 @@ public void format(AbstractConfiguration conf, LayoutManager layoutManager) String ledgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf); List children = zk.getChildren(ledgersRootPath, false); for (String child : children) { - if (!AbstractZkLedgerManager.isSpecialZnode(child) && ledgerManager.isLedgerParentNode(child)) { + boolean lParentNode = !isSpecialZnode(child) && ledgerManager.isLedgerParentNode(child); + boolean lIdGenerator = isLeadgerIdGeneratorZnode(child); + + if (lParentNode || lIdGenerator) { ZKUtil.deleteRecursive(zk, ledgersRootPath + "/" + child); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index 0bbe0f11e42..daafc747e04 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -32,9 +32,11 @@ import java.io.File; import java.util.ArrayList; import java.util.Collection; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Random; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -612,4 +614,40 @@ public void testAreEntriesOfLedgerStoredInTheBookieForMultipleSegments() throws assertTrue("expected areEntriesOfLedgerStoredInTheBookie to return true for bookie2", BookKeeperAdmin.areEntriesOfLedgerStoredInTheBookie(ledgerId, bookie2, meta)); } + + @Test + public void testBookkeeperAdminFormatResetsLedgerIds() throws Exception { + ClientConfiguration conf = new ClientConfiguration(); + conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + + /* + * in this testsuite there are going to be 2 (numOfBookies) ledgers + * written and when formatting the BookieAdmin i expect that the + * ledger ids restart from 0 + */ + int numOfLedgers = 2; + try (BookKeeper bkc = new BookKeeper(conf)) { + Set ledgerIds = new HashSet<>(); + for (int n = 0; n < numOfLedgers; n++) { + try (LedgerHandle lh = bkc.createLedger(numOfBookies, numOfBookies, digestType, "L".getBytes())) { + ledgerIds.add(lh.getId()); + lh.addEntry("000".getBytes()); + } + } + + try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) { + bkAdmin.format(baseConf, false, true); + } + + /** + * ledgers created after format produce the same ids + */ + for (int n = 0; n < numOfLedgers; n++) { + try (LedgerHandle lh = bkc.createLedger(numOfBookies, numOfBookies, digestType, "L".getBytes())) { + lh.addEntry("000".getBytes()); + assertTrue(ledgerIds.contains(lh.getId())); + } + } + } + } } From 8236fad6fa240e6b5f6a149aa97daea66b178192 Mon Sep 17 00:00:00 2001 From: Anup Ghatage Date: Tue, 21 Apr 2020 13:27:30 -0700 Subject: [PATCH 0409/1642] Add forkCount option to parallelize build and tests ### Motivation Current code forks a new JVM per module. (bookkeeper-server, bookkeeper-proto etc) This means one fork per module for build and testing, no parallelism within the module where majority of the time goes. We need parallelism within a module during the test execution so we can have the builds complete faster and have the artifacts shipped out quicker. ### Changes We use the maven surefire plugin but don't define the `forkCount` and hence set it to default of 1. This means it executes each module with one thread. This change sets `forkCount` to 5, enabling parallelism in testing and drastically reducing total turnaround time. (by about 2/3rds!) *Total build+test time without this change* ``` [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 01:06 h [INFO] Finished at: 2020-03-05T02:01:29-08:00 [INFO] ------------------------------------------------------------------------ ``` *Total build+test time with this change* ``` [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 18:23 min [INFO] Finished at: 2020-03-05T02:38:22-08:00 [INFO] ------------------------------------------------------------------------ ``` ### Things to watch Added parallelism may cause some flappers but with much trial and error I have come to the number `5`. The flappers are usually only from conflict in obtaining the same port number. If needed, we can increase the retryCount, but as of now I consistently don't see any flappers at a `forkCount` of 5 Reviewers: Enrico Olivelli , Sijie Guo This closes #2278 from Ghatage/parallelizeBuildAndTests --- pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pom.xml b/pom.xml index 278370d75a1..45b4c3db38c 100644 --- a/pom.xml +++ b/pom.xml @@ -196,6 +196,7 @@ 0.5.0 6.19 3.1.8 + 1 @@ -849,6 +850,7 @@ -Xmx2G -Djava.net.preferIPv4Stack=true -Dio.netty.leakDetection.level=paranoid ${redirectTestOutputToFile} + ${forkCount.variable} false 1800 ${testRetryCount} @@ -1010,6 +1012,7 @@ @{argLine} -Xmx2G -Djava.net.preferIPv4Stack=true ${redirectTestOutputToFile} + ${forkCount.variable} false 1800 @@ -1044,6 +1047,7 @@ -Xmx2G -Djava.net.preferIPv4Stack=true false + ${forkCount.variable} false 1800 false @@ -1064,6 +1068,7 @@ -Xmx2G -Djava.net.preferIPv4Stack=true -Dbookkeeper.root.logger=DEBUG,CONSOLE false + ${forkCount.variable} false 1800 false From e4a708352650422663fecbb76bb2d74a5d4cb3f6 Mon Sep 17 00:00:00 2001 From: John Pradeep Date: Tue, 21 Apr 2020 21:32:13 +0100 Subject: [PATCH 0410/1642] Fix mistakes in documentation Fix minor mistakes in the sentences of the documentation Reviewers: Jia Zhai This closes #2271 from yehohanan7/patch-1 --- site/docs/4.10.0/development/protocol.md | 2 +- site/docs/4.10.0/getting-started/concepts.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/site/docs/4.10.0/development/protocol.md b/site/docs/4.10.0/development/protocol.md index f4410444427..41a55c856f5 100644 --- a/site/docs/4.10.0/development/protocol.md +++ b/site/docs/4.10.0/development/protocol.md @@ -25,7 +25,7 @@ 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`. +Last entry | | The last entry in the ledger or `NULL` if the current state is not `CLOSED`. In addition, each ledger's metadata consists of one or more *fragments*. Each fragment is either diff --git a/site/docs/4.10.0/getting-started/concepts.md b/site/docs/4.10.0/getting-started/concepts.md index 7a3c92847b2..2eeece1eaaa 100644 --- a/site/docs/4.10.0/getting-started/concepts.md +++ b/site/docs/4.10.0/getting-started/concepts.md @@ -122,7 +122,7 @@ When a client instructs a {% pop bookie %} to write an entry to a ledger, the en 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. +* A background 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: From dfbcd0e72fd5667b25cf251f3cadd124c9dff3c7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 21 Apr 2020 13:37:04 -0700 Subject: [PATCH 0411/1642] Bump libthrift from 0.9.3 to 0.12.0 (#2201) Bumps [libthrift](https://github.com/apache/thrift) from 0.9.3 to 0.12.0. - [Release notes](https://github.com/apache/thrift/releases) - [Changelog](https://github.com/apache/thrift/blob/master/CHANGES.md) - [Commits](https://github.com/apache/thrift/compare/0.9.3...v0.12.0) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 45b4c3db38c..0a9391a9d91 100644 --- a/pom.xml +++ b/pom.xml @@ -146,7 +146,7 @@ 3.2.7 4.12 0.5.0-1 - 0.9.3 + 0.12.0 1.18.10 1.3.0 3.0.0 From bb932b4e194986ef21fbd8a788af0e2126fe019b Mon Sep 17 00:00:00 2001 From: Yang Yang Date: Sat, 2 May 2020 15:50:45 +0800 Subject: [PATCH 0412/1642] ISSUE #2310: Adds the autorecovery status service. Descriptions of the changes in this PR: ### Motivation It would be convenient to support enable, disable, or check the status of autorecovery with REST API, just like what does in the toggle command. ### Changes Adds an API to enable, disable, and get the current status of autorecovery at `/api/v1/autorecovery/status`. Master Issue: #2310 Reviewers: Sijie Guo , Enrico Olivelli , Matteo Minardi This closes #2313 from fantapsody/autorecovery_status, closes #2310 --- .../apache/bookkeeper/http/HttpRouter.java | 3 + .../apache/bookkeeper/http/HttpServer.java | 2 + .../server/http/BKHttpServiceProvider.java | 3 + .../service/AutoRecoveryStatusService.java | 107 ++++++++++++++++++ .../AutoRecoveryStatusServiceTest.java | 106 +++++++++++++++++ 5 files changed, 221 insertions(+) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/AutoRecoveryStatusService.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/AutoRecoveryStatusServiceTest.java diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java index 99d5be685f6..91dcb883340 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java @@ -50,6 +50,7 @@ public abstract class HttpRouter { public static final String BOOKIE_STATE = "/api/v1/bookie/state"; public static final String BOOKIE_IS_READY = "/api/v1/bookie/is_ready"; // autorecovery + public static final String AUTORECOVERY_STATUS = "/api/v1/autorecovery/status"; public static final String RECOVERY_BOOKIE = "/api/v1/autorecovery/bookie"; public static final String LIST_UNDER_REPLICATED_LEDGER = "/api/v1/autorecovery/list_under_replicated_ledger"; public static final String WHO_IS_AUDITOR = "/api/v1/autorecovery/who_is_auditor"; @@ -83,6 +84,8 @@ public HttpRouter(AbstractHttpHandlerFactory handlerFactory) { this.endpointHandlers.put(BOOKIE_IS_READY, handlerFactory.newHandler(HttpServer.ApiType.BOOKIE_IS_READY)); // autorecovery + this.endpointHandlers.put(AUTORECOVERY_STATUS, handlerFactory + .newHandler(HttpServer.ApiType.AUTORECOVERY_STATUS)); this.endpointHandlers.put(RECOVERY_BOOKIE, handlerFactory.newHandler(HttpServer.ApiType.RECOVERY_BOOKIE)); this.endpointHandlers.put(LIST_UNDER_REPLICATED_LEDGER, handlerFactory.newHandler(HttpServer.ApiType.LIST_UNDER_REPLICATED_LEDGER)); diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java index c694a076721..82c9c33a234 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java @@ -33,6 +33,7 @@ public interface HttpServer { enum StatusCode { OK(200), REDIRECT(302), + BAD_REQUEST(400), FORBIDDEN(403), NOT_FOUND(404), INTERNAL_ERROR(500), @@ -84,6 +85,7 @@ enum ApiType { BOOKIE_IS_READY, // autorecovery + AUTORECOVERY_STATUS, RECOVERY_BOOKIE, LIST_UNDER_REPLICATED_LEDGER, WHO_IS_AUDITOR, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java index 24795e54e83..aeb4d83b1bb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java @@ -40,6 +40,7 @@ import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.Auditor; import org.apache.bookkeeper.replication.AutoRecoveryMain; +import org.apache.bookkeeper.server.http.service.AutoRecoveryStatusService; import org.apache.bookkeeper.server.http.service.BookieIsReadyService; import org.apache.bookkeeper.server.http.service.BookieStateService; import org.apache.bookkeeper.server.http.service.ConfigurationService; @@ -225,6 +226,8 @@ public HttpEndpointService provideHttpEndpointService(ApiType type) { return new BookieIsReadyService(bookieServer.getBookie()); // autorecovery + case AUTORECOVERY_STATUS: + return new AutoRecoveryStatusService(configuration); case RECOVERY_BOOKIE: return new RecoveryBookieService(configuration, bka, executor); case LIST_UNDER_REPLICATED_LEDGER: diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/AutoRecoveryStatusService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/AutoRecoveryStatusService.java new file mode 100644 index 00000000000..f9ab650e0fe --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/AutoRecoveryStatusService.java @@ -0,0 +1,107 @@ +/* + * 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.server.http.service; + +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.Collections; +import java.util.Map; +import org.apache.bookkeeper.common.util.JsonUtil; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.http.HttpServer; +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.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.commons.lang3.ObjectUtils; + +/** + * HttpEndpointService that handles Autorecovery status related http requests. + * + *

            The GET method returns the current status of Autorecovery. The output would be like {"enabled" : true}. + * + *

            The PUT method requires a parameter 'enabled', and enables Autorecovery if its value is 'true', + * and disables Autorecovery otherwise. The behaviour is idempotent if Autorecovery status is already + * the same as desired. The output would be the current status after the action. + * + */ +public class AutoRecoveryStatusService implements HttpEndpointService { + protected final ServerConfiguration conf; + + public AutoRecoveryStatusService(ServerConfiguration conf) { + this.conf = conf; + } + + @Override + public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { + return MetadataDrivers.runFunctionWithLedgerManagerFactory(conf, + ledgerManagerFactory -> { + try (LedgerUnderreplicationManager ledgerUnderreplicationManager = ledgerManagerFactory + .newLedgerUnderreplicationManager()) { + switch (request.getMethod()) { + case GET: + return handleGetStatus(ledgerUnderreplicationManager); + case PUT: + return handlePutStatus(request, ledgerUnderreplicationManager); + default: + return new HttpServiceResponse("Not found method. Should be GET or PUT method", + HttpServer.StatusCode.NOT_FOUND); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedExecutionException(e); + } catch (Exception e) { + throw new UncheckedExecutionException(e); + } + }); + } + + private HttpServiceResponse handleGetStatus(LedgerUnderreplicationManager ledgerUnderreplicationManager) + throws Exception { + String body = JsonUtil.toJson(ImmutableMap.of("enabled", + ledgerUnderreplicationManager.isLedgerReplicationEnabled())); + return new HttpServiceResponse(body, HttpServer.StatusCode.OK); + } + + private HttpServiceResponse handlePutStatus(HttpServiceRequest request, + LedgerUnderreplicationManager ledgerUnderreplicationManager) + throws Exception { + Map params = ObjectUtils.defaultIfNull(request.getParams(), Collections.emptyMap()); + String enabled = params.get("enabled"); + if (enabled == null) { + return new HttpServiceResponse("Param 'enabled' not found in " + params, + HttpServer.StatusCode.BAD_REQUEST); + } + if (Boolean.parseBoolean(enabled)) { + if (!ledgerUnderreplicationManager.isLedgerReplicationEnabled()) { + ledgerUnderreplicationManager.enableLedgerReplication(); + } + } else { + if (ledgerUnderreplicationManager.isLedgerReplicationEnabled()) { + ledgerUnderreplicationManager.disableLedgerReplication(); + } + } + + // use the current status as the response + String body = JsonUtil.toJson(ImmutableMap.of("enabled", + ledgerUnderreplicationManager.isLedgerReplicationEnabled())); + return new HttpServiceResponse(body, HttpServer.StatusCode.OK); + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/AutoRecoveryStatusServiceTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/AutoRecoveryStatusServiceTest.java new file mode 100644 index 00000000000..69e851f0911 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/AutoRecoveryStatusServiceTest.java @@ -0,0 +1,106 @@ +/* + * 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.server.http.service; + +import static org.junit.Assert.assertEquals; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import java.util.Map; +import org.apache.bookkeeper.http.HttpServer; +import org.apache.bookkeeper.http.service.HttpServiceRequest; +import org.apache.bookkeeper.http.service.HttpServiceResponse; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.junit.Before; +import org.junit.Test; + +/** + * Unit tests for {@link AutoRecoveryStatusService}. + */ +public class AutoRecoveryStatusServiceTest extends BookKeeperClusterTestCase { + private final ObjectMapper mapper = new ObjectMapper(); + private AutoRecoveryStatusService autoRecoveryStatusService; + public AutoRecoveryStatusServiceTest() { + super(1); + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + autoRecoveryStatusService = new AutoRecoveryStatusService(baseConf); + } + + @Test + public void testGetStatus() throws Exception { + HttpServiceRequest request = new HttpServiceRequest(null, HttpServer.Method.GET, null); + HttpServiceResponse response = autoRecoveryStatusService.handle(request); + assertEquals(HttpServer.StatusCode.OK.getValue(), response.getStatusCode()); + JsonNode json = mapper.readTree(response.getBody()); + assertEquals(Boolean.TRUE, json.get("enabled").asBoolean()); + } + + @Test + public void testEnableStatus() throws Exception { + Map params = ImmutableMap.of("enabled", "true"); + HttpServiceRequest request = new HttpServiceRequest(null, HttpServer.Method.PUT, params); + HttpServiceResponse response = autoRecoveryStatusService.handle(request); + assertEquals(HttpServer.StatusCode.OK.getValue(), response.getStatusCode()); + JsonNode json = mapper.readTree(response.getBody()); + assertEquals(Boolean.TRUE, json.get("enabled").asBoolean()); + + request = new HttpServiceRequest(null, HttpServer.Method.GET, params); + response = autoRecoveryStatusService.handle(request); + assertEquals(HttpServer.StatusCode.OK.getValue(), response.getStatusCode()); + json = mapper.readTree(response.getBody()); + assertEquals(Boolean.TRUE, json.get("enabled").asBoolean()); + } + + @Test + public void testDisableStatus() throws Exception { + Map params = ImmutableMap.of("enabled", "false"); + HttpServiceRequest request = new HttpServiceRequest(null, HttpServer.Method.PUT, params); + HttpServiceResponse response = autoRecoveryStatusService.handle(request); + assertEquals(HttpServer.StatusCode.OK.getValue(), response.getStatusCode()); + JsonNode json = mapper.readTree(response.getBody()); + assertEquals(Boolean.FALSE, json.get("enabled").asBoolean()); + + request = new HttpServiceRequest(null, HttpServer.Method.GET, params); + response = autoRecoveryStatusService.handle(request); + assertEquals(HttpServer.StatusCode.OK.getValue(), response.getStatusCode()); + json = mapper.readTree(response.getBody()); + assertEquals(Boolean.FALSE, json.get("enabled").asBoolean()); + } + + @Test + public void testInvalidParams() throws Exception { + Map params = ImmutableMap.of("enable", "false"); + HttpServiceRequest request = new HttpServiceRequest(null, HttpServer.Method.PUT, params); + HttpServiceResponse response = autoRecoveryStatusService.handle(request); + assertEquals(HttpServer.StatusCode.BAD_REQUEST.getValue(), response.getStatusCode()); + } + + @Test + public void testInvalidMethod() throws Exception { + HttpServiceRequest request = new HttpServiceRequest(null, HttpServer.Method.POST, null); + HttpServiceResponse response = autoRecoveryStatusService.handle(request); + assertEquals(HttpServer.StatusCode.NOT_FOUND.getValue(), response.getStatusCode()); + } +} From 277000b7d0baaa8cf73adf31f13e361413ede4c6 Mon Sep 17 00:00:00 2001 From: Jia Zhai Date: Sat, 2 May 2020 15:51:49 +0800 Subject: [PATCH 0413/1642] [DOC] SSLContextFactory -> TLSContextFactory ### Motivation From 4.6.0 SSLContextFactory has been changed into TLSContextFactory. but some of the doc not updated. ### Changes Updated related doc. Reviewers: Sijie Guo , Enrico Olivelli This closes #2307 from jiazhai/doc_outdate_ssl --- conf/bk_server.conf | 6 +++--- site/_data/config/bk_server.yaml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/conf/bk_server.conf b/conf/bk_server.conf index a3cf3452b1a..e6e3558a988 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -234,7 +234,7 @@ httpServerClass=org.apache.bookkeeper.http.vertx.VertxHttpServer # tlsProvider=OpenSSL # The path to the class that provides security. -# tlsProviderFactoryClass=org.apache.bookkeeper.security.SSLContextFactory +# tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory # Type of security used by server. # tlsClientAuthentication=true @@ -960,7 +960,7 @@ storage.cluster.controller.schedule.interval.ms=30000 # throughput. # Default is: PooledDirect # allocatorPoolingPolicy=PooledDirect - + # Controls the amount of concurrency for the memory pool. # Default is to have a number of allocator arenas equals to 2 * CPUS. # Decreasing this number will reduce the amount of memory overhead, at the @@ -968,7 +968,7 @@ storage.cluster.controller.schedule.interval.ms=30000 # allocatorPoolingConcurrency=8 # Define the memory allocator out of memory policy. -# Available options are: +# Available options are: # - FallbackToHeap: If it's not possible to allocate a buffer from direct memory, # fallback to allocate an unpooled buffer from JVM heap. # This will help absorb memory allocation spikes because the heap diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index 8e3da6eaade..cd7b76f0f75 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -155,7 +155,7 @@ groups: default: OpenSSL - param: tlsProviderFactoryClass description: The path to the class that provides security. - default: org.apache.bookkeeper.security.SSLContextFactory + default: org.apache.bookkeeper.tls.TLSContextFactory - param: tlsClientAuthentication description: Type of security used by server. default: 'true' From 7f6d060aa2264bd9f92b8c1d40c3b5c8b5a4388c Mon Sep 17 00:00:00 2001 From: Ravi Sharda Date: Mon, 4 May 2020 15:48:43 +0530 Subject: [PATCH 0414/1642] Issue #2212: Fix issues in Bookkeeper Docker image that prevented containers from starting up Descriptions of the changes in this PR: ### Motivation This PR modifies the Apache Bookkeeper Docker image to fix issues that were causing errors upon container bootstrap. The containers would exit soon after they were launched. See issue #2212 for a description of such an error. Note that the problems that are fixed in this PR were observed: * when launching containers using both Docker Compose and Kubernetes. * when we were trying to upgrade the image to: `4.9.2`. It is highly likely that the issue is observed in other versions (except for `4.7.3`) too. * when launching both a standalone container as well as a cluster of three containers. ### Changes The major changes made in this PR are as follows: * Updates the `Dockerfile` to install `zk-shell`. * Update the `init_bookie.sh` file to: * Use `zk-shell` instead of `/opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain` command that doesn't work. * Use `opt/bookkeeper/bin/bookkeeper shell initnewcluster` for initializing the cluster instead of the previously used command that did not work. * Increase the time a container waits for an in-flight `initnewcluster` operation. * Make the comments more descriptive. * Modifies `bin/common.sh` to handle the condition when file `/proc/sys/net/ipv6/bindv6only` is missing in the system. This can prevent a container from starting up in some cases. We have seen this issue on some Kubernetes-based environments. * Fixes errors in `docker-compose.yml` file. *Note:* Some of the changes made in this PR are modeled after changes made by sijie for `v4.7.2` in PR #1666 . ### Master Issue #2212 Reviewers: Jia Zhai , Enrico Olivelli This closes #2219 from ravisharda/startup-failure-docker-image, closes #2212 --- bin/common.sh | 2 +- docker/Dockerfile | 7 ++- docker/docker-compose.yml | 6 +-- docker/scripts/init_bookie.sh | 49 ++++++++----------- .../current-version-image/Dockerfile | 5 ++ 5 files changed, 35 insertions(+), 34 deletions(-) diff --git a/bin/common.sh b/bin/common.sh index 7ca32efa804..66122b8d3db 100755 --- a/bin/common.sh +++ b/bin/common.sh @@ -18,7 +18,7 @@ # */ # Check net.ipv6.bindv6only -if [ -f /sbin/sysctl ]; then +if [ -f /sbin/sysctl ] && [ -f /proc/sys/net/ipv6/bindv6only ]; then # check if net.ipv6.bindv6only is set to 1 bindv6only=$(/sbin/sysctl -n net.ipv6.bindv6only 2> /dev/null) if [ -n "$bindv6only" ] && [ "$bindv6only" -eq "1" ] diff --git a/docker/Dockerfile b/docker/Dockerfile index ff1396b5c8f..f7ed4b1bfc1 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -33,7 +33,7 @@ ENV JAVA_HOME=/usr/lib/jvm/jre-1.8.0 # Download Apache Bookkeeper, untar and clean up RUN set -x \ && adduser "${BK_USER}" \ - && yum install -y java-1.8.0-openjdk-headless wget bash python sudo \ + && yum install -y java-1.8.0-openjdk-headless wget bash python sudo\ && mkdir -pv /opt \ && cd /opt \ && wget -q "${DISTRO_URL}" \ @@ -46,6 +46,11 @@ RUN set -x \ && tar -xzf "$DISTRO_NAME.tar.gz" \ && mv bookkeeper-server-${BK_VERSION}/ /opt/bookkeeper/ \ && rm -rf "$DISTRO_NAME.tar.gz" "$DISTRO_NAME.tar.gz.asc" "$DISTRO_NAME.tar.gz.sha512" \ + # install zookeeper shell + && wget -q https://bootstrap.pypa.io/get-pip.py \ + && python get-pip.py \ + && pip install zk-shell \ + && rm -rf get-pip.py \ && yum remove -y wget \ && yum clean all diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index 41774933cf1..85faada6d11 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -30,7 +30,7 @@ services: environment: - JAVA_HOME=/usr/lib/jvm/jre-1.8.0 - BK_zkServers=zookeeper:2181 - - BK_zkLedgersRootPath = /ledgers + - BK_zkLedgersRootPath=/ledgers bookie2: image: apache/bookkeeper @@ -40,7 +40,7 @@ services: environment: - JAVA_HOME=/usr/lib/jvm/jre-1.8.0 - BK_zkServers=zookeeper:2181 - - BK_zkLedgersRootPath = /ledgers + - BK_zkLedgersRootPath=/ledgers bookie3: image: apache/bookkeeper @@ -50,7 +50,7 @@ services: environment: - JAVA_HOME=/usr/lib/jvm/jre-1.8.0 - BK_zkServers=zookeeper:2181 - - BK_zkLedgersRootPath = /ledgers + - BK_zkLedgersRootPath=/ledgers dice: image: caiok/bookkeeper-tutorial diff --git a/docker/scripts/init_bookie.sh b/docker/scripts/init_bookie.sh index 07a7feecd3c..1617442d179 100755 --- a/docker/scripts/init_bookie.sh +++ b/docker/scripts/init_bookie.sh @@ -19,65 +19,57 @@ # * See the License for the specific language governing permissions and # * limitations under the License. # */ - source ${SCRIPTS_DIR}/common.sh function wait_for_zookeeper() { echo "wait for zookeeper" - until /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} ls /; do sleep 5; done + until zk-shell --run-once "ls /" ${BK_zkServers}; do sleep 5; done } function create_zk_root() { if [ "x${BK_CLUSTER_ROOT_PATH}" != "x" ]; then echo "create the zk root dir for bookkeeper at '${BK_CLUSTER_ROOT_PATH}'" - /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} create ${BK_CLUSTER_ROOT_PATH} + zk-shell --run-once "create ${BK_CLUSTER_ROOT_PATH} '' false false true" ${BK_zkServers} fi } -# Init the cluster if required znodes not exist in Zookeeper. -# Use ephemeral zk node as lock to keep initialize atomic. function init_cluster() { - if [ "x${BK_STREAM_STORAGE_ROOT_PATH}" == "x" ]; then - echo "BK_STREAM_STORAGE_ROOT_PATH is not set. fail fast." - exit -1 - fi - - /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_STREAM_STORAGE_ROOT_PATH} + zk-shell --run-once "ls ${BK_zkLedgersRootPath}/available/readonly" ${BK_zkServers} if [ $? -eq 0 ]; then - echo "Metadata of cluster already exists, no need to init" + echo "Cluster metadata already exists" else - # create ephemeral zk node bkInitLock, initiator who this node, then do init; other initiators will wait. - /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} create -e ${BK_CLUSTER_ROOT_PATH}/bkInitLock - if [ $? -eq 0 ]; then - # bkInitLock created success, this is the successor to do znode init - echo "Initializing bookkeeper cluster at service uri ${BK_metadataServiceUri}." - /opt/bookkeeper/bin/bkctl --service-uri ${BK_metadataServiceUri} cluster init + # Create an ephemeral zk node `bkInitLock` for use as a lock. + lock=`zk-shell --run-once "create ${BK_CLUSTER_ROOT_PATH}/bkInitLock '' true false false" ${BK_zkServers}` + if [ -z "$lock" ]; then + echo "znodes do not exist in Zookeeper for Bookkeeper. Initializing a new Bookkeekeper cluster in Zookeeper." + /opt/bookkeeper/bin/bookkeeper shell initnewcluster if [ $? -eq 0 ]; then - echo "Successfully initialized bookkeeper cluster at service uri ${BK_metadataServiceUri}." + echo "initnewcluster operation succeeded" else - echo "Failed to initialize bookkeeper cluster at service uri ${BK_metadataServiceUri}. please check the reason." + echo "initnewcluster operation failed. Please check the reason." + echo "Exit status of initnewcluster" + echo $? exit fi else - echo "Other docker instance is doing initialize at the same time, will wait in this instance." + echo "Others may be initializing the cluster at the same time." tenSeconds=1 - while [ ${tenSeconds} -lt 10 ] + while [ ${tenSeconds} -lt 100 ] do sleep 10 - echo "run '/opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_STREAM_STORAGE_ROOT_PATH}'" - /opt/bookkeeper/bin/bookkeeper org.apache.zookeeper.ZooKeeperMain -server ${BK_zkServers} stat ${BK_STREAM_STORAGE_ROOT_PATH} + zk-shell --run-once "ls ${BK_zkLedgersRootPath}/available/readonly" ${BK_zkServers} if [ $? -eq 0 ]; then - echo "Waited $tenSeconds * 10 seconds, bookkeeper inited" + echo "Waited $tenSeconds * 10 seconds. Successfully listed ''${BK_zkLedgersRootPath}/available/readonly'" break else - echo "Waited $tenSeconds * 10 seconds, still not init" + echo "Waited $tenSeconds * 10 seconds. Continue waiting." (( tenSeconds++ )) continue fi done - if [ ${tenSeconds} -eq 10 ]; then - echo "Waited 100 seconds for bookkeeper cluster init, something wrong, please check" + if [ ${tenSeconds} -eq 100 ]; then + echo "Waited 100 seconds for bookkeeper cluster to initialize, but to no avail. Something is wrong, please check." exit fi fi @@ -97,5 +89,4 @@ function init_bookie() { # init the cluster init_cluster - } \ No newline at end of file diff --git a/tests/docker-images/current-version-image/Dockerfile b/tests/docker-images/current-version-image/Dockerfile index 7d998a46def..7a14cc48957 100644 --- a/tests/docker-images/current-version-image/Dockerfile +++ b/tests/docker-images/current-version-image/Dockerfile @@ -39,6 +39,11 @@ RUN set -x \ && yum install -y java-1.8.0-openjdk-headless wget bash python-pip python-devel sudo netcat gcc gcc-c++ \ && mkdir -pv /opt \ && cd /opt \ + # install zookeeper shell + && wget -q https://bootstrap.pypa.io/get-pip.py \ + && python get-pip.py \ + && pip install zk-shell \ + && rm -rf get-pip.py \ && yum clean all # untar tarballs From e06fc6a241e8b1005c7d15514a9e8a35d793ec28 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 4 May 2020 18:21:07 +0800 Subject: [PATCH 0415/1642] Rerun the failed tests by comments --- *Motivation* Currently, non-committers cannot rerun the failure tests, we need to provide the ability to let non-committer to rerun the failed tests. I wrote a Github Actions that can rerun the failed test by comments `rerun failure tests` and add it to the bookkeeper ci process. Reviewers: Enrico Olivelli This closes #2309 from zymap/rerun-failure-job --- .github/workflows/bot.yml | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 .github/workflows/bot.yml diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml new file mode 100644 index 00000000000..fdda43fa56f --- /dev/null +++ b/.github/workflows/bot.yml @@ -0,0 +1,21 @@ +name: Bot tests +on: + issue_comment: + types: [created] + +jobs: + bot: + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v2 + + - name: Bot actions + uses: zymap/bot@v1.0.0 + env: + GITHUB_TOKEN: ${{ secrets.BKBOT_TOKEN }} + with: + repo_owner: apache + repo_name: bookkeeper + rerun_cmd: rerun failure checks + comment: ${{ github.event.comment.body }} From 4e8ca4358da3c3280436e5df9cf3566dcb4fedd1 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 5 May 2020 20:24:13 +0200 Subject: [PATCH 0416/1642] Do not download SNAPSHOT during GitHub actions checks ### Motivation Sometimes GitHub actions checks are stuck in downloading SNAPSHOTs, there is not need for it ### Changes Add "-nsu" option to every Maven invokations and remove "-Dstream" that is now useless. I am also adding "-B" that suppresses the "Progress" noise in logs This fix relates to #2294 Reviewers: Anup Ghatage , Matteo Minardi , Rajan Dhabalia This closes #2323 from eolivelli/fix/no-snapshot-updates and squashes the following commits: 69b7817a4 [Enrico Olivelli] Add -B 3fa4523b2 [Enrico Olivelli] GitHub Actions: do not look for SNAPSHOT updates --- .github/workflows/bookie-tests.yml | 8 ++++---- .github/workflows/client-tests.yml | 2 +- .github/workflows/compatibility-check-java11.yml | 2 +- .github/workflows/compatibility-check-java8.yml | 2 +- .github/workflows/integration-tests.yml | 6 +++--- .github/workflows/pr-validation.yml | 2 +- .github/workflows/remaining-tests.yml | 2 +- .github/workflows/replication-tests.yml | 2 +- .github/workflows/tls-tests.yml | 2 +- 9 files changed, 14 insertions(+), 14 deletions(-) diff --git a/.github/workflows/bookie-tests.yml b/.github/workflows/bookie-tests.yml index 7a784a26557..c3db2b9ea08 100644 --- a/.github/workflows/bookie-tests.yml +++ b/.github/workflows/bookie-tests.yml @@ -44,13 +44,13 @@ jobs: java-version: 1.8 - name: Maven build bookkeeper-server - run: mvn -am -pl bookkeeper-server clean install -DskipTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -nsu -am -pl bookkeeper-server clean install -DskipTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR - name: Run EntryLogTests - run: mvn -pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.TestEntryLog" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -nsu -pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.TestEntryLog" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR - name: Run InterleavedLedgerStorageTest - run: mvn -pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.TestInterleavedLederStorage" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -nsu -pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.TestInterleavedLederStorage" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR - name: Run bookie tests - run: mvn -pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.*Test" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -nsu -pl bookkeeper-server test -Dtest="org.apache.bookkeeper.bookie.*Test" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/client-tests.yml b/.github/workflows/client-tests.yml index 47f912c0c58..299f4f6b3fc 100644 --- a/.github/workflows/client-tests.yml +++ b/.github/workflows/client-tests.yml @@ -43,4 +43,4 @@ jobs: with: java-version: 1.8 - name: Run client tests - run: mvn -am -pl bookkeeper-server clean install test -Dtest="org.apache.bookkeeper.client.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -am -nsu -pl bookkeeper-server clean install test -Dtest="org.apache.bookkeeper.client.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/compatibility-check-java11.yml b/.github/workflows/compatibility-check-java11.yml index 360ffba763f..27f0b10f7e6 100644 --- a/.github/workflows/compatibility-check-java11.yml +++ b/.github/workflows/compatibility-check-java11.yml @@ -42,4 +42,4 @@ jobs: with: java-version: 1.11 - name: Build with Maven - run: mvn clean package -Dstream -DskipBookKeeperServerTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn clean package -B -nsu -DskipBookKeeperServerTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/compatibility-check-java8.yml b/.github/workflows/compatibility-check-java8.yml index 47cf21e6dc9..c03878b1707 100644 --- a/.github/workflows/compatibility-check-java8.yml +++ b/.github/workflows/compatibility-check-java8.yml @@ -42,4 +42,4 @@ jobs: with: java-version: 1.8 - name: Build with Maven - run: mvn clean package spotbugs:check -Dstream -DskipBookKeeperServerTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn clean package -B -nsu spotbugs:check -DskipBookKeeperServerTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index b0b9fdef6f1..08c75af295f 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -43,10 +43,10 @@ jobs: with: java-version: 1.8 - name: Build with Maven - run: mvn -B clean install -Dstream -Pdocker -DskipTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -nsu clean install -Pdocker -DskipTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR - name: Run metadata driver tests - run: mvn -B -f metadata-drivers/pom.xml test -DintegrationTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -nsu -f metadata-drivers/pom.xml test -DintegrationTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR - name: Run all integration tests - run: mvn -B -f tests/pom.xml test -Dstream -DintegrationTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -nsu -f tests/pom.xml test -DintegrationTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/pr-validation.yml b/.github/workflows/pr-validation.yml index cf4147a9024..6d45f9041ff 100644 --- a/.github/workflows/pr-validation.yml +++ b/.github/workflows/pr-validation.yml @@ -43,4 +43,4 @@ jobs: with: java-version: 1.8 - name: Validate pull request style - run: mvn clean apache-rat:check checkstyle:check package -Ddistributedlog -Dstream -DskipTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn clean -B -nsu apache-rat:check checkstyle:check package -Ddistributedlog -DskipTests -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/remaining-tests.yml b/.github/workflows/remaining-tests.yml index 30f8f0eef44..baeb110faf3 100644 --- a/.github/workflows/remaining-tests.yml +++ b/.github/workflows/remaining-tests.yml @@ -42,4 +42,4 @@ jobs: with: java-version: 1.8 - name: Run remaining tests - run: mvn -am -pl bookkeeper-server clean install test -Dtest="!org.apache.bookkeeper.client.**,!org.apache.bookkeeper.bookie.**,!org.apache.bookkeeper.replication.**,!org.apache.bookkeeper.tls.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -nsu -am -pl bookkeeper-server clean install test -Dtest="!org.apache.bookkeeper.client.**,!org.apache.bookkeeper.bookie.**,!org.apache.bookkeeper.replication.**,!org.apache.bookkeeper.tls.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/replication-tests.yml b/.github/workflows/replication-tests.yml index a561949b3bb..767c6127460 100644 --- a/.github/workflows/replication-tests.yml +++ b/.github/workflows/replication-tests.yml @@ -43,4 +43,4 @@ jobs: with: java-version: 1.8 - name: Run replication tests - run: mvn -am -pl bookkeeper-server clean install test -Dtest="org.apache.bookkeeper.replication.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -nsu -am -pl bookkeeper-server clean install test -Dtest="org.apache.bookkeeper.replication.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR diff --git a/.github/workflows/tls-tests.yml b/.github/workflows/tls-tests.yml index 0803f8a09a6..3475d8967e4 100644 --- a/.github/workflows/tls-tests.yml +++ b/.github/workflows/tls-tests.yml @@ -43,4 +43,4 @@ jobs: with: java-version: 1.8 - name: Run tls tests - run: mvn -am -pl bookkeeper-server clean install test -Dtest="org.apache.bookkeeper.tls.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR + run: mvn -B -am -nsu -pl bookkeeper-server clean install test -Dtest="org.apache.bookkeeper.tls.**" -DfailIfNoTests=false -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR From a0058056deb22309892140f6ed1cd21bf76c2c2e Mon Sep 17 00:00:00 2001 From: dmercuriali Date: Thu, 7 May 2020 09:23:04 +0200 Subject: [PATCH 0417/1642] Issue #2106: Update ZookKeeper dependency to 3.5.7 Descriptions of the changes in this PR: Updated ZooKeeper dependency from 3.4.13 to 3.5.7 Excluded transitive dependency to netty-all. Updated maven-shade-pluging configuration in 'distributedlog-core-shaded' project. Package 'org.apache.jute' was moved to a ZooKeeper dependency (org.apache.zookeeper:zookeeper-jute), so i added the artifact to the artifactSet. Updated license files: - updated Zookeeper to 3.5.7 - removed Jline 2.11 - removed Netty 3.10.1 - added Zookeeper-jute 3.5.7 Master Issue: #2106 Reviewers: Matteo Minardi , Enrico Olivelli , Jia Zhai This closes #2112 from dmercuriali/fix/#2106-update-zookeeper, closes #2106 --- bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt | 5 +++-- bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt | 5 +++-- bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt | 5 +++-- pom.xml | 6 +++--- shaded/distributedlog-core-shaded/pom.xml | 3 +-- .../apache/bookkeeper/tests/containers/ZKContainer.java | 7 +++++-- 6 files changed, 18 insertions(+), 13 deletions(-) diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index a13d26cc0c9..565d731dacf 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -244,7 +244,8 @@ Apache Software License, Version 2. - lib/net.java.dev.jna-jna-3.2.7.jar [22] - lib/org.apache.commons-commons-collections4-4.1.jar [23] - lib/org.apache.commons-commons-lang3-3.6.jar [24] -- lib/org.apache.zookeeper-zookeeper-3.4.13.jar [25] +- lib/org.apache.zookeeper-zookeeper-3.5.7.jar [25] +- lib/org.apache.zookeeper-zookeeper-jute-3.5.7.jar [25] - lib/org.eclipse.jetty-jetty-http-9.4.5.v20170502.jar [26] - lib/org.eclipse.jetty-jetty-io-9.4.5.v20170502.jar [26] - lib/org.eclipse.jetty-jetty-security-9.4.5.v20170502.jar [26] @@ -307,7 +308,7 @@ Apache Software License, Version 2. [22] Source available at https://github.com/java-native-access/jna/tree/3.2.7 [23] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad [24] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=tag;h=3ad2e8 -[25] Source available at https://github.com/apache/zookeeper/tree/release-3.4.13 +[25] Source available at https://github.com/apache/zookeeper/tree/release-3.5.7 [26] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502 [27] Source available at https://github.com/facebook/rocksdb/tree/v5.13.1 [28] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48 diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt index 8abca03cd9d..da4d619b2ef 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -232,7 +232,8 @@ Apache Software License, Version 2. - lib/net.java.dev.jna-jna-3.2.7.jar [17] - lib/org.apache.commons-commons-collections4-4.1.jar [18] - lib/org.apache.commons-commons-lang3-3.6.jar [19] -- lib/org.apache.zookeeper-zookeeper-3.4.13.jar [20] +- lib/org.apache.zookeeper-zookeeper-3.5.7.jar [20] +- lib/org.apache.zookeeper-zookeeper-jute-3.5.7.jar [20] - lib/com.beust-jcommander-1.48.jar [23] - lib/net.jpountz.lz4-lz4-1.3.0.jar [25] - lib/com.google.api.grpc-proto-google-common-protos-1.12.0.jar [27] @@ -278,7 +279,7 @@ Apache Software License, Version 2. [17] Source available at https://github.com/java-native-access/jna/tree/3.2.7 [18] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad [19] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=shortlog;h=refs/tags/LANG_3_6 -[20] Source available at https://github.com/apache/zookeeper/tree/release-3.4.13 +[20] Source available at https://github.com/apache/zookeeper/tree/release-3.5.7 [23] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48 [25] Source available at https://github.com/lz4/lz4-java/tree/1.3.0 [27] Source available at https://github.com/googleapis/googleapis diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index 912f730a066..206dd886792 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -241,7 +241,8 @@ Apache Software License, Version 2. - lib/net.java.dev.jna-jna-3.2.7.jar [17] - lib/org.apache.commons-commons-collections4-4.1.jar [18] - lib/org.apache.commons-commons-lang3-3.6.jar [19] -- lib/org.apache.zookeeper-zookeeper-3.4.13.jar [20] +- lib/org.apache.zookeeper-zookeeper-3.5.7.jar [20] +- lib/org.apache.zookeeper-zookeeper-jute-3.5.7.jar [20] - lib/org.eclipse.jetty-jetty-http-9.4.5.v20170502.jar [21] - lib/org.eclipse.jetty-jetty-io-9.4.5.v20170502.jar [21] - lib/org.eclipse.jetty-jetty-security-9.4.5.v20170502.jar [21] @@ -302,7 +303,7 @@ Apache Software License, Version 2. [17] Source available at https://github.com/java-native-access/jna/tree/3.2.7 [18] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad [19] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-lang.git;a=shortlog;h=refs/tags/LANG_3_6 -[20] Source available at https://github.com/apache/zookeeper/tree/release-3.4.13 +[20] Source available at https://github.com/apache/zookeeper/tree/release-3.5.7 [21] Source available at https://github.com/eclipse/jetty.project/tree/jetty-9.4.5.v20170502 [22] Source available at https://github.com/facebook/rocksdb/tree/v5.13.1 [23] Source available at https://github.com/cbeust/jcommander/tree/jcommander-1.48 diff --git a/pom.xml b/pom.xml index 0a9391a9d91..efd595d19fa 100644 --- a/pom.xml +++ b/pom.xml @@ -168,7 +168,7 @@ 1.3.2 1.8.3 3.4.1 - 3.4.13 + 3.5.7 2.1.2 0.12 @@ -499,7 +499,7 @@ io.netty - netty + * @@ -523,7 +523,7 @@ io.netty - netty + * diff --git a/shaded/distributedlog-core-shaded/pom.xml b/shaded/distributedlog-core-shaded/pom.xml index 3d763a91100..810014f72b3 100644 --- a/shaded/distributedlog-core-shaded/pom.xml +++ b/shaded/distributedlog-core-shaded/pom.xml @@ -82,8 +82,6 @@ com.fasterxml.jackson.core:jackson-annotations com.google.guava:guava com.google.protobuf:protobuf-java - - io.netty:netty net.java.dev.jna:jna net.jpountz.lz4:lz4 org.apache.bookkeeper:bookkeeper-common @@ -104,6 +102,7 @@ org.apache.httpcomponents:httpcore org.apache.thrift:libthrift org.apache.zookeeper:zookeeper + org.apache.zookeeper:zookeeper-jute org.rocksdb:rocksdbjni diff --git a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java index fd9e4a7d8a4..0b7c46a1182 100644 --- a/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java +++ b/tests/integration-tests-topologies/src/main/java/org/apache/bookkeeper/tests/containers/ZKContainer.java @@ -22,7 +22,7 @@ import java.time.Duration; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.tests.containers.wait.ZKWaitStrategy; +import org.apache.bookkeeper.tests.containers.wait.HttpWaitStrategy; /** * Test container that runs zookeeper. @@ -62,7 +62,10 @@ protected void configure() { @Override public void start() { - this.waitStrategy = new ZKWaitStrategy(ZK_PORT) + this.waitStrategy = new HttpWaitStrategy() + .forPath("/commands/ruok") + .forStatusCode(200) + .forPort(ZK_HTTP_PORT) .withStartupTimeout(Duration.of(60, SECONDS)); this.withCreateContainerCmdModifier(createContainerCmd -> createContainerCmd.withHostName(HOST_NAME)); From 537deba55f7f43d12e59a2d46b2b48e5d84065a0 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Sun, 17 May 2020 19:47:36 +0800 Subject: [PATCH 0418/1642] [CI] Using master action code for checking rerun commands The rerun checks CI can not use ref to get all failure test actions. So I change to use sha of the commits to get all failure test actions to rerun. https://github.com/zymap/bot/pull/3/files Reviewers: Enrico Olivelli This closes #2332 from zymap/fix-bot --- .github/workflows/bot.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index fdda43fa56f..798a20ba846 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -11,7 +11,7 @@ jobs: - uses: actions/checkout@v2 - name: Bot actions - uses: zymap/bot@v1.0.0 + uses: zymap/bot@v1.0.1 env: GITHUB_TOKEN: ${{ secrets.BKBOT_TOKEN }} with: From 4608976e9144f081180a1f78ec5a6b2a1a1871a9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 17 May 2020 13:50:23 +0200 Subject: [PATCH 0419/1642] Bump jackson.version from 2.9.7 to 2.11.0 (#2326) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index efd595d19fa..6c1f44b1756 100644 --- a/pom.xml +++ b/pom.xml @@ -136,7 +136,7 @@ 2.7.3 1.3 2.1.10 - 2.9.7 + 2.11.0 1.9.11 1.48 9.4.5.v20170502 From a0727f57c9c9594ad78dfab2f4d606f65d1f86ba Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sun, 17 May 2020 05:14:36 -0700 Subject: [PATCH 0420/1642] [BOOKIE-MTLS] add support of hostname verification ### Motivation Right now, bookkeeper-client is not able to perform [hostname-verification](https://tersesystems.com/blog/2014/03/23/fixing-hostname-verification/) when it connects to broker over tls. Hostname-verification feature is already implemented in almost all [http-client](https://github.com/apache/httpcomponents-client/blob/master/httpclient5/src/main/java/org/apache/hc/client5/http/ssl/DefaultHostnameVerifier.java) but it's not supported by [netty](https://stackoverflow.com/questions/13315623/netty-ssl-hostname-verification-support) yet. therefore, client should be able to perform hostname-verification as per [RFC-2181](https://tools.ietf.org/html/rfc2818#section-3.1) ### Modifications - added client configuration to enable hostname-verification (default disable) - use [http-client](https://github.com/apache/httpcomponents-client/blob/master/httpclient5/src/main/java/org/apache/hc/client5/http/ssl/DefaultHostnameVerifier.java) but it's not supported by [netty](https://stackoverflow.com/questions/13315623/netty-ssl-hostname-verification-support) to perform hostname-validation rather adding custom logic. - add httpclient-apache dependency into LICENSE and NOTICE files. ### Result Bookkeeper client will be able to perform hostname verification while creating ssl session with bookie. Reviewers: Enrico Olivelli This closes #2156 from rdhabalia/hostname --- .../src/main/resources/LICENSE-all.bin.txt | 6 ++-- .../src/main/resources/LICENSE-bkctl.bin.txt | 4 +-- .../src/main/resources/LICENSE-server.bin.txt | 4 +-- bookkeeper-server/pom.xml | 4 +++ .../bookkeeper/conf/ClientConfiguration.java | 22 ++++++++++++ .../apache/bookkeeper/proto/AuthHandler.java | 35 +++++++++++++++++++ .../proto/PerChannelBookieClient.java | 18 +++++++--- .../org/apache/bookkeeper/tls/TestTLS.java | 21 +++++++++++ pom.xml | 8 +++++ 9 files changed, 111 insertions(+), 11 deletions(-) diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index 565d731dacf..551585e6516 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -283,8 +283,8 @@ Apache Software License, Version 2. - lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [40] - lib/org.apache.yetus-audience-annotations-0.5.0.jar [41] - lib/org.jctools-jctools-core-2.1.2.jar [42] -- lib/org.apache.httpcomponents-httpclient-4.4.1.jar [43] -- lib/org.apache.httpcomponents-httpcore-4.4.1.jar [44] +- lib/org.apache.httpcomponents-httpclient-4.5.5.jar [43] +- lib/org.apache.httpcomponents-httpcore-4.4.9.jar [44] - lib/org.apache.thrift-libthrift-0.9.3.jar [45] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.9.7 @@ -480,4 +480,4 @@ This product uses the annotations from The Checker Framework, which are licensed MIT License. For details, see deps/checker-compat-qual-2.5.2/LICENSE Bundles as - - lib/org.checkerframework-checker-compat-qual-2.5.2.jar + - lib/org.checkerframework-checker-compat-qual-2.5.2.jar \ No newline at end of file diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt index da4d619b2ef..90d1c786331 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -260,8 +260,8 @@ Apache Software License, Version 2. - lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [35] - lib/org.apache.yetus-audience-annotations-0.5.0.jar [36] - lib/org.jctools-jctools-core-2.1.2.jar [37] -- lib/org.apache.httpcomponents-httpclient-4.4.1.jar [38] -- lib/org.apache.httpcomponents-httpcore-4.4.1.jar [39] +- lib/org.apache.httpcomponents-httpclient-4.5.5.jar [38] +- lib/org.apache.httpcomponents-httpcore-4.4.9.jar [39] - lib/org.apache.thrift-libthrift-0.9.3.jar [40] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9 diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index 206dd886792..7c46d62deef 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -280,8 +280,8 @@ Apache Software License, Version 2. - lib/com.google.errorprone-error_prone_annotations-2.1.2.jar [35] - lib/org.apache.yetus-audience-annotations-0.5.0.jar [36] - lib/org.jctools-jctools-core-2.1.2.jar [37] -- lib/org.apache.httpcomponents-httpclient-4.4.1.jar [38] -- lib/org.apache.httpcomponents-httpcore-4.4.1.jar [39] +- lib/org.apache.httpcomponents-httpclient-4.5.5.jar [38] +- lib/org.apache.httpcomponents-httpcore-4.4.9.jar [39] - lib/org.apache.thrift-libthrift-0.9.3.jar [40] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.8.9 diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index d2f4f030146..dcb43b8a292 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -116,6 +116,10 @@ net.java.dev.jna jna + + org.apache.httpcomponents + httpclient + org.apache.bookkeeper 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 f25a0656d44..f752fc8e79f 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 @@ -146,6 +146,7 @@ public class ClientConfiguration extends AbstractConfiguration future) throws Exception { state = ConnectionState.CONNECTED; AuthHandler.ClientSideHandler authHandler = future.get().pipeline() .get(AuthHandler.ClientSideHandler.class); - authHandler.authProvider.onProtocolUpgrade(); - activeTlsChannelCounter.inc(); + if (conf.getHostnameVerificationEnabled() && !authHandler.verifyTlsHostName(channel)) { + // add HostnameVerification or private classes not + // for validation + rc = BKException.Code.UnauthorizedAccessException; + } else { + authHandler.authProvider.onProtocolUpgrade(); + activeTlsChannelCounter.inc(); + } } else if (future.isSuccess() && (state == ConnectionState.CLOSED || state == ConnectionState.DISCONNECTED)) { LOG.warn("Closed before TLS handshake completed, clean up: {}, current state {}", @@ -2427,8 +2433,12 @@ public void operationComplete(ChannelFuture future) throws Exception { state = ConnectionState.CONNECTED; AuthHandler.ClientSideHandler authHandler = future.channel().pipeline() .get(AuthHandler.ClientSideHandler.class); - authHandler.authProvider.onProtocolUpgrade(); - activeTlsChannelCounter.inc(); + if (conf.getHostnameVerificationEnabled() && !authHandler.verifyTlsHostName(channel)) { + rc = BKException.Code.UnauthorizedAccessException; + } else { + authHandler.authProvider.onProtocolUpgrade(); + activeTlsChannelCounter.inc(); + } } else if (future.isSuccess() && (state == ConnectionState.CLOSED || state == ConnectionState.DISCONNECTED)) { LOG.warn("Closed before connection completed, clean up: {}, current state {}", diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java index 9d00c47a31e..cd5d97ed1d8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java @@ -45,6 +45,7 @@ import org.apache.bookkeeper.auth.BookieAuthProvider; import org.apache.bookkeeper.auth.ClientAuthProvider; import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BKException.BKUnauthorizedAccessException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.BookKeeperAdmin; @@ -967,4 +968,24 @@ public void testHandshakeFailure() throws Exception { client.getTestStatsProvider().getCounter(nameBuilder.toString() + BookKeeperClientStats.FAILED_TLS_HANDSHAKE_COUNTER).get().longValue()); } + + /** + * Verify that a client fails to connect to bookie if hostname verification + * fails. + */ + @Test + public void testClientAuthPluginWithHostnameVerificationEnabled() throws Exception { + secureClientSideChannel = false; + secureClientSideChannelPrincipals = null; + ClientConfiguration clientConf = new ClientConfiguration(baseClientConf); + + clientConf.setClientAuthProviderFactoryClass(AllowOnlyBookiesWithX509Certificates.class.getName()); + clientConf.setHostnameVerificationEnabled(true); + try { + testClient(clientConf, numBookies); + fail("should have failed with unauthorized exception"); + } catch (BKUnauthorizedAccessException e) { + // Ok. + } + } } diff --git a/pom.xml b/pom.xml index 6c1f44b1756..191d0e80ac8 100644 --- a/pom.xml +++ b/pom.xml @@ -156,6 +156,7 @@ 2.0.2 0.8.1 0.8.3 + 4.5.5 3.5.1 3.5.1-1 1.12.0 @@ -623,6 +624,13 @@ sketches-core ${datasketches.version} + + + + org.apache.httpcomponents + httpclient + ${httpclient.version} + From defbe777530cd4c17b5e80ac2e6ecd4d041275b1 Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Tue, 19 May 2020 13:55:12 +0800 Subject: [PATCH 0421/1642] Fix wrong module artifactId ### Motivation Fix wrong maven artifactId, there are three module use the same artifactId. ### Changes ![image](https://user-images.githubusercontent.com/20113411/81897790-e4fe2180-95e9-11ea-9449-92941fe4c1bc.png) Reviewers: Matteo Minardi , Enrico Olivelli This closes #2329 from lamber-ken/bk-fix-wrong-artifactId --- tests/backward-compat/current-server-old-clients/pom.xml | 5 +++-- tests/backward-compat/hierarchical-ledger-manager/pom.xml | 5 +++-- tests/backward-compat/hostname-bookieid/pom.xml | 5 +++-- tests/backward-compat/old-cookie-new-cluster/pom.xml | 5 +++-- tests/backward-compat/pom.xml | 4 ++-- tests/backward-compat/recovery-no-password/pom.xml | 5 +++-- tests/backward-compat/upgrade-direct/pom.xml | 5 +++-- tests/backward-compat/upgrade/pom.xml | 5 +++-- tests/backward-compat/yahoo-custom-version/pom.xml | 5 +++-- tests/integration/cluster/pom.xml | 5 +++-- tests/integration/pom.xml | 5 ++--- tests/integration/smoke/pom.xml | 5 +++-- tests/integration/standalone/pom.xml | 5 +++-- 13 files changed, 37 insertions(+), 27 deletions(-) diff --git a/tests/backward-compat/current-server-old-clients/pom.xml b/tests/backward-compat/current-server-old-clients/pom.xml index 11d295ae3e2..29fb3c9912e 100644 --- a/tests/backward-compat/current-server-old-clients/pom.xml +++ b/tests/backward-compat/current-server-old-clients/pom.xml @@ -18,9 +18,10 @@ 4.0.0 - org.apache.bookkeeper.tests.backward-compat - tests-parent + org.apache.bookkeeper.tests + backward-compat 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/hierarchical-ledger-manager/pom.xml b/tests/backward-compat/hierarchical-ledger-manager/pom.xml index d0bea4d500d..92f937f0d88 100644 --- a/tests/backward-compat/hierarchical-ledger-manager/pom.xml +++ b/tests/backward-compat/hierarchical-ledger-manager/pom.xml @@ -18,9 +18,10 @@ 4.0.0 - org.apache.bookkeeper.tests.backward-compat - tests-parent + org.apache.bookkeeper.tests + backward-compat 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/hostname-bookieid/pom.xml b/tests/backward-compat/hostname-bookieid/pom.xml index 0ae23f7ab57..fc6cb7db7f1 100644 --- a/tests/backward-compat/hostname-bookieid/pom.xml +++ b/tests/backward-compat/hostname-bookieid/pom.xml @@ -18,9 +18,10 @@ 4.0.0 - org.apache.bookkeeper.tests.backward-compat - tests-parent + org.apache.bookkeeper.tests + backward-compat 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/old-cookie-new-cluster/pom.xml b/tests/backward-compat/old-cookie-new-cluster/pom.xml index b1dafb7613e..73d3449de48 100644 --- a/tests/backward-compat/old-cookie-new-cluster/pom.xml +++ b/tests/backward-compat/old-cookie-new-cluster/pom.xml @@ -18,9 +18,10 @@ 4.0.0 - org.apache.bookkeeper.tests.backward-compat - tests-parent + org.apache.bookkeeper.tests + backward-compat 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/pom.xml b/tests/backward-compat/pom.xml index c7a245b899b..503eb6395de 100644 --- a/tests/backward-compat/pom.xml +++ b/tests/backward-compat/pom.xml @@ -24,8 +24,8 @@ 4.11.0-SNAPSHOT ../integration-tests-base-groovy - org.apache.bookkeeper.tests.backward-compat - tests-parent + org.apache.bookkeeper.tests + backward-compat Apache BookKeeper :: Tests :: Backward Compatibility upgrade diff --git a/tests/backward-compat/recovery-no-password/pom.xml b/tests/backward-compat/recovery-no-password/pom.xml index 4b977100fe0..b6c7c03cbdb 100644 --- a/tests/backward-compat/recovery-no-password/pom.xml +++ b/tests/backward-compat/recovery-no-password/pom.xml @@ -18,9 +18,10 @@ 4.0.0 - org.apache.bookkeeper.tests.backward-compat - tests-parent + org.apache.bookkeeper.tests + backward-compat 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/upgrade-direct/pom.xml b/tests/backward-compat/upgrade-direct/pom.xml index 4b9094512cf..d7d388c04a9 100644 --- a/tests/backward-compat/upgrade-direct/pom.xml +++ b/tests/backward-compat/upgrade-direct/pom.xml @@ -18,9 +18,10 @@ 4.0.0 - org.apache.bookkeeper.tests.backward-compat - tests-parent + org.apache.bookkeeper.tests + backward-compat 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/upgrade/pom.xml b/tests/backward-compat/upgrade/pom.xml index 2a39243dcee..95e66180267 100644 --- a/tests/backward-compat/upgrade/pom.xml +++ b/tests/backward-compat/upgrade/pom.xml @@ -18,9 +18,10 @@ 4.0.0 - org.apache.bookkeeper.tests.backward-compat - tests-parent + org.apache.bookkeeper.tests + backward-compat 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.backward-compat diff --git a/tests/backward-compat/yahoo-custom-version/pom.xml b/tests/backward-compat/yahoo-custom-version/pom.xml index 3852cd1f5c7..eed8d032def 100644 --- a/tests/backward-compat/yahoo-custom-version/pom.xml +++ b/tests/backward-compat/yahoo-custom-version/pom.xml @@ -18,9 +18,10 @@ 4.0.0 - org.apache.bookkeeper.tests.backward-compat - tests-parent + org.apache.bookkeeper.tests + backward-compat 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.backward-compat diff --git a/tests/integration/cluster/pom.xml b/tests/integration/cluster/pom.xml index 8c7ee7dbc25..42fbac36540 100644 --- a/tests/integration/cluster/pom.xml +++ b/tests/integration/cluster/pom.xml @@ -19,9 +19,10 @@ 4.0.0 - org.apache.bookkeeper.tests.integration - tests-parent + org.apache.bookkeeper.tests + integration 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.integration diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index a26758f7b53..ded0de07068 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -22,10 +22,9 @@ org.apache.bookkeeper.tests tests-parent 4.11.0-SNAPSHOT - .. - org.apache.bookkeeper.tests.integration - tests-parent + org.apache.bookkeeper.tests + integration Apache BookKeeper :: Tests :: Integration smoke diff --git a/tests/integration/smoke/pom.xml b/tests/integration/smoke/pom.xml index 92fcb13aecd..4add9c159ad 100644 --- a/tests/integration/smoke/pom.xml +++ b/tests/integration/smoke/pom.xml @@ -18,9 +18,10 @@ 4.0.0 - org.apache.bookkeeper.tests.integration - tests-parent + org.apache.bookkeeper.tests + integration 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.integration diff --git a/tests/integration/standalone/pom.xml b/tests/integration/standalone/pom.xml index 91c0bf6bbbd..893db1b2287 100644 --- a/tests/integration/standalone/pom.xml +++ b/tests/integration/standalone/pom.xml @@ -19,9 +19,10 @@ 4.0.0 - org.apache.bookkeeper.tests.integration - tests-parent + org.apache.bookkeeper.tests + integration 4.11.0-SNAPSHOT + .. org.apache.bookkeeper.tests.integration From da3a1dd69cfa120c77ed2906837e456dc1e1dd61 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 19 May 2020 07:56:26 +0200 Subject: [PATCH 0422/1642] QuorumCoverage should only count unknown nodes The original patch was contributed by ivankelly in PR #2303, I have only fixed checkstyle and removed two tests that were wrong. Quorum coverage checks if we have heard from enough nodes to know that there is no entry that can have been written to enough nodes that we haven't heard from to have formed an ack quorum. The coverage algorithm was correct pre-5e399df. 5e399df(BOOKKEEPER-759: Delay Ensemble Change & Disable Ensemble Change) broke this, but it still seems to have worked because they had a broken else statement at the end. Why a change which is 100% about the write-path changed something in the read-path is a mystery. dcdd1e(Small fix wrong nodesUninitialized count when checkCovered) went on to fix the broken fix, so the whole thing ended up broke. The change also modifies ReadLastConfirmedOp to make it testable. Reviewers: Sijie Guo , Rajan Dhabalia This closes #2333 from eolivelli/pr2303 --- .../bookkeeper/client/LedgerHandle.java | 8 +- .../bookkeeper/client/LedgerRecoveryOp.java | 7 +- .../client/ReadLastConfirmedOp.java | 69 ++++++----- .../RoundRobinDistributionSchedule.java | 42 ++++--- .../client/ParallelLedgerRecoveryTest.java | 13 +- .../client/ReadLastConfirmedOpTest.java | 111 ++++++++++++++++++ .../RoundRobinDistributionScheduleTest.java | 10 -- .../bookkeeper/test/BookieFailureTest.java | 31 +---- .../bookkeeper/client/LedgerReader.java | 8 +- 9 files changed, 209 insertions(+), 90 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java 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 b71ac76a02c..f80d0bd4053 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 @@ -1407,7 +1407,13 @@ public void readLastConfirmedDataComplete(int rc, DigestManager.RecoveryData dat } }; - new ReadLastConfirmedOp(this, clientCtx.getBookieClient(), getCurrentEnsemble(), innercb).initiate(); + new ReadLastConfirmedOp(clientCtx.getBookieClient(), + distributionSchedule, + macManager, + ledgerId, + getCurrentEnsemble(), + ledgerKey, + innercb).initiate(); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java index 2dd6ea27990..bd19a6b5fab 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java @@ -93,7 +93,12 @@ LedgerRecoveryOp setEntryListener(ReadEntryListener entryListener) { } public CompletableFuture initiate() { - ReadLastConfirmedOp rlcop = new ReadLastConfirmedOp(lh, clientCtx.getBookieClient(), lh.getCurrentEnsemble(), + ReadLastConfirmedOp rlcop = new ReadLastConfirmedOp(clientCtx.getBookieClient(), + lh.distributionSchedule, + lh.macManager, + lh.ledgerId, + lh.getCurrentEnsemble(), + lh.ledgerKey, new ReadLastConfirmedOp.LastConfirmedDataCallback() { public void readLastConfirmedDataComplete(int rc, RecoveryData data) { if (rc == BKException.Code.OK) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java index 2b75403812d..e39bf60237f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java @@ -17,6 +17,8 @@ */ package org.apache.bookkeeper.client; +import com.google.common.annotations.VisibleForTesting; + import io.netty.buffer.ByteBuf; import java.util.List; @@ -25,6 +27,7 @@ import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; +import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.proto.checksum.DigestManager.RecoveryData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,17 +38,18 @@ */ class ReadLastConfirmedOp implements ReadEntryCallback { static final Logger LOG = LoggerFactory.getLogger(ReadLastConfirmedOp.class); - LedgerHandle lh; - BookieClient bookieClient; - int numResponsesPending; - int numSuccessfulResponse; - RecoveryData maxRecoveredData; - volatile boolean completed = false; - int lastSeenError = BKException.Code.ReadException; - - LastConfirmedDataCallback cb; - final DistributionSchedule.QuorumCoverageSet coverageSet; - final List currentEnsemble; + private final long ledgerId; + private final byte[] ledgerKey; + private final BookieClient bookieClient; + private final DigestManager digestManager; + private int numResponsesPending; + private RecoveryData maxRecoveredData; + private volatile boolean completed = false; + private int lastSeenError = BKException.Code.ReadException; + + private final LastConfirmedDataCallback cb; + private final DistributionSchedule.QuorumCoverageSet coverageSet; + private final List currentEnsemble; /** * Wrapper to get all recovered data from the request. @@ -54,22 +58,28 @@ interface LastConfirmedDataCallback { void readLastConfirmedDataComplete(int rc, RecoveryData data); } - public ReadLastConfirmedOp(LedgerHandle lh, BookieClient bookieClient, - List ensemble, LastConfirmedDataCallback cb) { + public ReadLastConfirmedOp(BookieClient bookieClient, + DistributionSchedule schedule, + DigestManager digestManager, + long ledgerId, + List ensemble, + byte[] ledgerKey, + LastConfirmedDataCallback cb) { this.cb = cb; this.bookieClient = bookieClient; this.maxRecoveredData = new RecoveryData(LedgerHandle.INVALID_ENTRY_ID, 0); - this.lh = lh; - this.numSuccessfulResponse = 0; - this.numResponsesPending = lh.getLedgerMetadata().getEnsembleSize(); - this.coverageSet = lh.distributionSchedule.getCoverageSet(); + this.numResponsesPending = ensemble.size(); + this.coverageSet = schedule.getCoverageSet(); this.currentEnsemble = ensemble; + this.ledgerId = ledgerId; + this.ledgerKey = ledgerKey; + this.digestManager = digestManager; } public void initiate() { for (int i = 0; i < currentEnsemble.size(); i++) { bookieClient.readEntry(currentEnsemble.get(i), - lh.ledgerId, + ledgerId, BookieProtocol.LAST_ADD_CONFIRMED, this, i, BookieProtocol.FLAG_NONE); } @@ -78,10 +88,10 @@ public void initiate() { public void initiateWithFencing() { for (int i = 0; i < currentEnsemble.size(); i++) { bookieClient.readEntry(currentEnsemble.get(i), - lh.ledgerId, + ledgerId, BookieProtocol.LAST_ADD_CONFIRMED, this, i, BookieProtocol.FLAG_DO_FENCING, - lh.ledgerKey); + ledgerKey); } } @@ -96,12 +106,11 @@ public synchronized void readEntryComplete(final int rc, final long ledgerId, fi boolean heardValidResponse = false; if (rc == BKException.Code.OK) { try { - RecoveryData recoveryData = lh.macManager.verifyDigestAndReturnLastConfirmed(buffer); + RecoveryData recoveryData = digestManager.verifyDigestAndReturnLastConfirmed(buffer); if (recoveryData.getLastAddConfirmed() > maxRecoveredData.getLastAddConfirmed()) { maxRecoveredData = recoveryData; } heardValidResponse = true; - numSuccessfulResponse++; } catch (BKDigestMatchException e) { // Too bad, this bookie didn't give us a valid answer, we // still might be able to recover though so continue @@ -140,17 +149,15 @@ public synchronized void readEntryComplete(final int rc, final long ledgerId, fi } if (numResponsesPending == 0 && !completed) { - int totalExepctedResponse = lh.getLedgerMetadata().getWriteQuorumSize() - - lh.getLedgerMetadata().getAckQuorumSize() + 1; - if (numSuccessfulResponse >= totalExepctedResponse) { - cb.readLastConfirmedDataComplete(BKException.Code.OK, maxRecoveredData); - return; - } - // Have got all responses back but was still not enough, just fail the operation - LOG.error("While readLastConfirmed ledger: {} did not hear success responses from all quorums {}", ledgerId, - lastSeenError); + LOG.error("While readLastConfirmed ledger: {} did not hear success responses from all quorums, {}", + ledgerId, coverageSet); cb.readLastConfirmedDataComplete(lastSeenError, maxRecoveredData); } } + + @VisibleForTesting + synchronized int getNumResponsesPending() { + return numResponsesPending; + } } 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 05711fd96c1..e4ae4f264e7 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 @@ -373,29 +373,43 @@ public synchronized void addBookie(int bookieIndexHeardFrom, int rc) { public synchronized boolean checkCovered() { // now check if there are any write quorums, with |ackQuorum| nodes available for (int i = 0; i < ensembleSize; i++) { - int nodesNotCovered = 0; - int nodesOkay = 0; - int nodesUninitialized = 0; + /* Nodes which have either responded with an error other than NoSuch{Entry,Ledger}, + or have not responded at all. We cannot know if these nodes ever accepted a entry. */ + int nodesUnknown = 0; + for (int j = 0; j < writeQuorumSize; j++) { int nodeIndex = (i + j) % ensembleSize; - if (covered[nodeIndex] == BKException.Code.OK) { - nodesOkay++; - } else if (covered[nodeIndex] != BKException.Code.NoSuchEntryException - && covered[nodeIndex] != BKException.Code.NoSuchLedgerExistsException) { - nodesNotCovered++; - if (covered[nodeIndex] == BKException.Code.UNINITIALIZED) { - nodesUninitialized++; - } + if (covered[nodeIndex] != BKException.Code.OK + && covered[nodeIndex] != BKException.Code.NoSuchEntryException + && covered[nodeIndex] != BKException.Code.NoSuchLedgerExistsException) { + nodesUnknown++; } } - // if we haven't seen any OK responses and there are still nodes not heard from, - // let's wait until - if (nodesNotCovered >= ackQuorumSize || (nodesOkay == 0 && nodesUninitialized > 0)) { + + /* If nodesUnknown is greater than the ack quorum size, then + it is possible those two unknown nodes accepted an entry which + we do not know about */ + if (nodesUnknown >= ackQuorumSize) { return false; } } return true; } + + @Override + public String toString() { + StringBuilder buffer = new StringBuilder(); + buffer.append("QuorumCoverage(e:").append(ensembleSize) + .append(",w:").append(writeQuorumSize) + .append(",a:").append(ackQuorumSize) + .append(") = ["); + int i = 0; + for (; i < covered.length - 1; i++) { + buffer.append(covered[i]).append(", "); + } + buffer.append(covered[i]).append("]"); + return buffer.toString(); + } } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index ab69fa75254..710756ea9b5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -675,8 +675,12 @@ public void operationComplete(int rc, Void result) { final AtomicInteger rcHolder = new AtomicInteger(-1234); final CountDownLatch doneLatch = new CountDownLatch(1); - new ReadLastConfirmedOp(readLh, bkc.getBookieClient(), + new ReadLastConfirmedOp(bkc.getBookieClient(), + readLh.distributionSchedule, + readLh.macManager, + readLh.ledgerId, readLh.getLedgerMetadata().getAllEnsembles().lastEntry().getValue(), + readLh.ledgerKey, new ReadLastConfirmedOp.LastConfirmedDataCallback() { @Override public void readLastConfirmedDataComplete(int rc, DigestManager.RecoveryData data) { @@ -760,8 +764,13 @@ public void testRecoveryWithUnavailableBookie() throws Exception { private int readLACFromQuorum(LedgerHandle ledger, int... bookieLACResponse) throws Exception { MutableInt responseCode = new MutableInt(100); CountDownLatch responseLatch = new CountDownLatch(1); - ReadLastConfirmedOp readLCOp = new ReadLastConfirmedOp(ledger, bkc.getBookieClient(), + ReadLastConfirmedOp readLCOp = new ReadLastConfirmedOp( + bkc.getBookieClient(), + ledger.getDistributionSchedule(), + ledger.getDigestManager(), + ledger.getId(), ledger.getLedgerMetadata().getAllEnsembles().lastEntry().getValue(), + ledger.getLedgerKey(), new ReadLastConfirmedOp.LastConfirmedDataCallback() { @Override public void readLastConfirmedDataComplete(int rc, DigestManager.RecoveryData data) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java new file mode 100644 index 00000000000..e553e52e98d --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java @@ -0,0 +1,111 @@ +/* + * + * 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.client; + +import com.google.common.collect.Lists; +import io.netty.buffer.UnpooledByteBufAllocator; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; +import org.apache.bookkeeper.proto.MockBookieClient; +import org.apache.bookkeeper.proto.checksum.DigestManager; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Tests about ReadLastConfirmedOp. + */ +public class ReadLastConfirmedOpTest { + private static final Logger log = LoggerFactory.getLogger(ReadLastConfirmedOpTest.class); + private final BookieSocketAddress bookie1 = new BookieSocketAddress("bookie1", 3181); + private final BookieSocketAddress bookie2 = new BookieSocketAddress("bookie2", 3181); + + OrderedExecutor executor = null; + + @Before + public void setup() throws Exception { + executor = OrderedExecutor.newBuilder() + .name("BookKeeperClientWorker") + .numThreads(1) + .build(); + } + + @After + public void teardown() throws Exception { + if (executor != null) { + executor.shutdown(); + } + } + + /** + * Test for specific bug that was introduced with dcdd1e88. + */ + @Test + public void testBookieFailsAfterLedgerMissingOnFirst() throws Exception { + long ledgerId = 0xf00b; + List ensemble = Lists.newArrayList(bookie1, bookie2); + byte[] ledgerKey = new byte[0]; + + MockBookieClient bookieClient = new MockBookieClient(executor); + DistributionSchedule schedule = new RoundRobinDistributionSchedule(2, 2, 2); + DigestManager digestManager = DigestManager.instantiate(ledgerId, ledgerKey, + DigestType.CRC32C, + UnpooledByteBufAllocator.DEFAULT, + true /* useV2 */); + + CompletableFuture blocker = new CompletableFuture<>(); + bookieClient.setPreReadHook((bookie, lId, entryId) -> { + if (bookie.equals(bookie1)) { + return CompletableFuture.completedFuture(null); + } else { + return blocker; + } + }); + CompletableFuture promise = new CompletableFuture<>(); + ReadLastConfirmedOp op = new ReadLastConfirmedOp( + bookieClient, schedule, + digestManager, ledgerId, ensemble, + ledgerKey, + (rc, data) -> { + if (rc != BKException.Code.OK) { + promise.completeExceptionally( + BKException.create(rc)); + } else { + promise.complete(data); + } + }); + op.initiateWithFencing(); + + while (op.getNumResponsesPending() > 1) { + Thread.sleep(100); + } + blocker.completeExceptionally( + new BKException.BKBookieHandleNotAvailableException()); + promise.get(); + } +} 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 1b989f5efc7..391df802de0 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 @@ -74,16 +74,6 @@ public void testCoverageSets() { } } assertEquals("Should be no errors", 0, errors); - - RoundRobinDistributionSchedule schedule = new RoundRobinDistributionSchedule( - 5, 3, 5); - DistributionSchedule.QuorumCoverageSet covSet = schedule.getCoverageSet(); - covSet.addBookie(0, BKException.Code.NoSuchLedgerExistsException); - covSet.addBookie(1, BKException.Code.NoSuchEntryException); - covSet.addBookie(2, BKException.Code.NoSuchLedgerExistsException); - covSet.addBookie(3, BKException.Code.UNINITIALIZED); - covSet.addBookie(4, BKException.Code.UNINITIALIZED); - assertFalse(covSet.checkCovered()); } /** 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 255e4f36007..c60a2dbd645 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 @@ -301,21 +301,6 @@ public void testLedgerNoRecoveryOpenAfterBKCrashed() throws Exception { LedgerHandle afterlh = bkc.openLedgerNoRecovery(beforelh.getId(), digestType, "".getBytes()); assertEquals(numEntries - 2, afterlh.getLastAddConfirmed()); - - startNewBookie(); - LedgerHandle beforelh2 = bkc.createLedger(numBookies, 1, digestType, "".getBytes()); - - for (int i = 0; i < numEntries; i++) { - beforelh2.addEntry(tmp.getBytes()); - } - - // shutdown first bookie server - killBookie(0); - - // try to open ledger no recovery - // should be able to open ledger with one bookie down: - // Total bookies available 3 >= 1 (Qw(4) - Qa(4) + 1) - bkc.openLedgerNoRecovery(beforelh2.getId(), digestType, "".getBytes()); } @Test @@ -333,24 +318,10 @@ public void testLedgerOpenAfterBKCrashed() throws Exception { killBookie(0); startNewBookie(); - // try to open ledger no recovery + // try to open ledger with recovery LedgerHandle afterlh = bkc.openLedger(beforelh.getId(), digestType, "".getBytes()); assertEquals(beforelh.getLastAddPushed(), afterlh.getLastAddConfirmed()); - - LedgerHandle beforelh2 = bkc.createLedger(numBookies, 1, digestType, "".getBytes()); - - for (int i = 0; i < numEntries; i++) { - beforelh2.addEntry(tmp.getBytes()); - } - - // shutdown first bookie server - killBookie(0); - - // try to open ledger no recovery - // should be able to open ledger with one bookie down: - // Total bookies available 3 >= 1 (Qw(4) - Qa(4) + 1) - bkc.openLedger(beforelh2.getId(), digestType, "".getBytes()); } /** diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java index b7a6ba9120a..06ed4d2b5e6 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java @@ -197,7 +197,13 @@ public void onFailure(Throwable throwable) { op.submit(); }; // Read Last AddConfirmed - new ReadLastConfirmedOp(lh, clientCtx.getBookieClient(), lh.getCurrentEnsemble(), readLACCallback).initiate(); + new ReadLastConfirmedOp(clientCtx.getBookieClient(), + lh.distributionSchedule, + lh.macManager, + lh.ledgerId, + lh.getCurrentEnsemble(), + lh.ledgerKey, + readLACCallback).initiate(); } public void readLacs(final LedgerHandle lh, long eid, From 6da8b0d2ddecde78a9cdb0fa31818d28502067a7 Mon Sep 17 00:00:00 2001 From: Anup Ghatage Date: Mon, 18 May 2020 22:57:56 -0700 Subject: [PATCH 0423/1642] Issue #2305: Bookies should not be allowed to come up without a cookie in metadata store. ### Motivation Currently we allow bookies to come up even if they don't have a cookie in the metadata store. The cookie is an identity of a registered bookie and a bookie should not be allowed to come up without it. ### Changes Update handling of bookie boot up. Add test. This scenario is rarely expected but can happen in case of corruption or errant deletion of znodes for cookies Reviewers: Matteo Minardi , Enrico Olivelli , Sijie Guo This closes #2306 from Ghatage/disableBootup-Wo-Cookie, closes #2305 --- .../org/apache/bookkeeper/bookie/Bookie.java | 6 ++ .../bookie/BookieInitializationTest.java | 69 +++++++++++++++++++ 2 files changed, 75 insertions(+) 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 6125970997d..7a655d65d30 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 @@ -467,6 +467,12 @@ public static void checkEnvironmentWithStorageExpansion( + "Empty directories are {}", missedCookieDirs); throw new InvalidCookieException(); } + } else { + if (rmCookie == null) { + // No corresponding cookie found in registration manager. The bookie should fail to come up. + LOG.error("Cookie for this bookie is not stored in metadata store. Bookie failing to come up"); + throw new InvalidCookieException(); + } } } catch (IOException ioe) { LOG.error("Error accessing cookie on disks", ioe); 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 628b56176fb..bff9e55b02b 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 @@ -22,6 +22,7 @@ import static com.google.common.base.Charsets.UTF_8; import static org.apache.bookkeeper.bookie.BookieJournalTest.writeV5Journal; +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.BOOKIE_STATUS_FILENAME; import static org.apache.bookkeeper.util.TestUtils.countNumOfFiles; @@ -86,8 +87,10 @@ import org.apache.bookkeeper.http.HttpRouter; import org.apache.bookkeeper.http.HttpServerLoader; import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.exceptions.MetadataException; import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.AutoRecoveryMain; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -105,6 +108,8 @@ import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.util.DiskChecker; import org.apache.bookkeeper.util.LoggerOutput; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; @@ -697,6 +702,7 @@ public void testBookieStartException() throws Exception { .setLedgerStorageClass(MockInterleavedLedgerStorage.class.getName()); BookieConfiguration bkConf = new BookieConfiguration(conf); + driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); /* * create cookie and write it to JournalDir/LedgerDir. @@ -705,6 +711,10 @@ public void testBookieStartException() throws Exception { Cookie cookie = cookieBuilder.build(); cookie.writeToDirectory(new File(journalDir, "current")); cookie.writeToDirectory(new File(ledgerDir, "current")); + Versioned newCookie = new Versioned<>( + cookie.toString().getBytes(UTF_8), Version.NEW + ); + driver.getRegistrationManager().writeCookie(Bookie.getBookieAddress(conf).toString(), newCookie); /* * Create LifecycleComponent for BookieServer and start it. @@ -1456,4 +1466,63 @@ public void testIOVertexHTTPServerEndpointForARWithPrometheusProvider() throws E } stackComponentFuture.cancel(true); } + + /** + * Test that verifies if a bookie can't come up without its cookie in metadata store. + * @throws Exception + */ + @Test + public void testBookieConnectAfterCookieDelete() throws BookieException.UpgradeException { + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + conf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + + try { + runFunctionWithRegistrationManager(conf, rm -> { + try { + bookieConnectAfterCookieDeleteWorker(conf, rm); + } catch (BookieException | IOException | InterruptedException e) { + fail("Test failed to run: " + e.getMessage()); + } + return null; + }); + } catch (MetadataException | ExecutionException e) { + throw new BookieException.UpgradeException(e); + } + } + + private void bookieConnectAfterCookieDeleteWorker(ServerConfiguration conf, RegistrationManager rm) + throws BookieException, InterruptedException, IOException { + + File tmpLedgerDir = createTempDir("BootupTest", "test"); + File tmpJournalDir = createTempDir("BootupTest", "test"); + Integer numOfJournalDirs = 2; + + String[] journalDirs = new String[numOfJournalDirs]; + for (int i = 0; i < numOfJournalDirs; i++) { + journalDirs[i] = tmpJournalDir.getAbsolutePath() + "/journal-" + i; + } + + conf.setJournalDirsName(journalDirs); + conf.setLedgerDirNames(new String[] { tmpLedgerDir.getPath() }); + + Bookie b = new Bookie(conf); + + final BookieSocketAddress bookieAddress = Bookie.getBookieAddress(conf); + + // Read cookie from registation manager + Versioned rmCookie = Cookie.readFromRegistrationManager(rm, bookieAddress); + + // Shutdown bookie + b.shutdown(); + + // Remove cookie from registration manager + rmCookie.getValue().deleteFromRegistrationManager(rm, conf, rmCookie.getVersion()); + + try { + b = new Bookie(conf); + Assert.fail("Bookie should not have come up. Cookie no present in metadata store."); + } catch (Exception e) { + LOG.info("As expected Bookie fails to come up without a cookie in metadata store."); + } + } } From 4ab4705eb292c9f532f126ea27d0b38d7d8ba717 Mon Sep 17 00:00:00 2001 From: Fretiq Labs <60456830+fretiq@users.noreply.github.com> Date: Tue, 19 May 2020 00:58:44 -0500 Subject: [PATCH 0424/1642] Add /api/v1/bookie/info REST API Descriptions of the changes in this PR: This adds an additional HTTP REST endpoint to expose the bookie's info (only storage numbers for now). ### Motivation I was looking for an HTTP endpoint that returned the info of only the current bookie, but couldn't find one. The only similar API I've found was `/api/v1/bookie/list_bookie_info`, however it connects to all bookies and may take a long time to respond (plus its response format is difficult to parse). ### Changes I've added a new API at `/api/v1/bookie/info` that returns a response like: ```json { "freeSpace" : 0, "totalSpace" : 0 } ``` Reviewers: Matteo Minardi , Enrico Olivelli , Jia Zhai This closes #2322 from fretiq/add-bookie-info-api --- .../apache/bookkeeper/http/HttpRouter.java | 2 + .../apache/bookkeeper/http/HttpServer.java | 1 + .../server/http/BKHttpServiceProvider.java | 3 + .../http/service/BookieInfoService.java | 76 +++++++++++++++++++ .../server/http/TestHttpService.java | 21 +++++ site/docs/latest/admin/http.md | 18 +++++ 6 files changed, 121 insertions(+) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieInfoService.java diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java index 91dcb883340..f6e2c58876e 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java @@ -49,6 +49,7 @@ public abstract class HttpRouter { public static final String GC_DETAILS = "/api/v1/bookie/gc_details"; public static final String BOOKIE_STATE = "/api/v1/bookie/state"; public static final String BOOKIE_IS_READY = "/api/v1/bookie/is_ready"; + public static final String BOOKIE_INFO = "/api/v1/bookie/info"; // autorecovery public static final String AUTORECOVERY_STATUS = "/api/v1/autorecovery/status"; public static final String RECOVERY_BOOKIE = "/api/v1/autorecovery/bookie"; @@ -82,6 +83,7 @@ public HttpRouter(AbstractHttpHandlerFactory handlerFactory) { this.endpointHandlers.put(GC_DETAILS, handlerFactory.newHandler(HttpServer.ApiType.GC_DETAILS)); this.endpointHandlers.put(BOOKIE_STATE, handlerFactory.newHandler(HttpServer.ApiType.BOOKIE_STATE)); this.endpointHandlers.put(BOOKIE_IS_READY, handlerFactory.newHandler(HttpServer.ApiType.BOOKIE_IS_READY)); + this.endpointHandlers.put(BOOKIE_INFO, handlerFactory.newHandler(HttpServer.ApiType.BOOKIE_INFO)); // autorecovery this.endpointHandlers.put(AUTORECOVERY_STATUS, handlerFactory diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java index 82c9c33a234..a9426deb9a3 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java @@ -83,6 +83,7 @@ enum ApiType { GC_DETAILS, BOOKIE_STATE, BOOKIE_IS_READY, + BOOKIE_INFO, // autorecovery AUTORECOVERY_STATUS, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java index aeb4d83b1bb..e9ac4142f9b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java @@ -41,6 +41,7 @@ import org.apache.bookkeeper.replication.Auditor; import org.apache.bookkeeper.replication.AutoRecoveryMain; import org.apache.bookkeeper.server.http.service.AutoRecoveryStatusService; +import org.apache.bookkeeper.server.http.service.BookieInfoService; import org.apache.bookkeeper.server.http.service.BookieIsReadyService; import org.apache.bookkeeper.server.http.service.BookieStateService; import org.apache.bookkeeper.server.http.service.ConfigurationService; @@ -224,6 +225,8 @@ public HttpEndpointService provideHttpEndpointService(ApiType type) { return new BookieStateService(bookieServer.getBookie()); case BOOKIE_IS_READY: return new BookieIsReadyService(bookieServer.getBookie()); + case BOOKIE_INFO: + return new BookieInfoService(bookieServer.getBookie()); // autorecovery case AUTORECOVERY_STATUS: diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieInfoService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieInfoService.java new file mode 100644 index 00000000000..62b118662d5 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieInfoService.java @@ -0,0 +1,76 @@ +/* + * 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.server.http.service; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.NonNull; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.common.util.JsonUtil; +import org.apache.bookkeeper.http.HttpServer; +import org.apache.bookkeeper.http.service.HttpEndpointService; +import org.apache.bookkeeper.http.service.HttpServiceRequest; +import org.apache.bookkeeper.http.service.HttpServiceResponse; + +/** + * HttpEndpointService that exposes the current info of the bookie. + * + *

            + * 
            + * {
            + *  "freeSpace" : 0,
            + *  "totalSpace" : 0
            + * }
            + * 
            + * 
            + */ +@AllArgsConstructor +public class BookieInfoService implements HttpEndpointService { + @NonNull private final Bookie bookie; + + /** + * POJO definition for the bookie info response. + */ + @Data + @NoArgsConstructor + @AllArgsConstructor + public static class BookieInfo { + private long freeSpace; + private long totalSpace; + } + + @Override + public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { + HttpServiceResponse response = new HttpServiceResponse(); + + if (HttpServer.Method.GET != request.getMethod()) { + response.setCode(HttpServer.StatusCode.NOT_FOUND); + response.setBody("Only GET is supported."); + return response; + } + + BookieInfo bi = new BookieInfo(bookie.getTotalFreeSpace(), bookie.getTotalDiskSpace()); + + String jsonResponse = JsonUtil.toJson(bi); + response.setBody(jsonResponse); + response.setCode(HttpServer.StatusCode.OK); + return response; + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index 02aa42b864b..cf839627578 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -51,6 +51,7 @@ import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.replication.AuditorElector; +import org.apache.bookkeeper.server.http.service.BookieInfoService; import org.apache.bookkeeper.server.http.service.BookieStateService.BookieState; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Before; @@ -866,4 +867,24 @@ public void testGetBookieIsReady() throws Exception { HttpServiceResponse response3 = bookieStateServer.handle(request3); assertEquals(HttpServer.StatusCode.SERVICE_UNAVAILABLE.getValue(), response3.getStatusCode()); } + + @Test + public void testGetBookieInfo() throws Exception { + HttpEndpointService bookieStateServer = bkHttpServiceProvider + .provideHttpEndpointService(HttpServer.ApiType.BOOKIE_INFO); + + HttpServiceRequest request1 = new HttpServiceRequest(null, HttpServer.Method.GET, null); + HttpServiceResponse response1 = bookieStateServer.handle(request1); + assertEquals(HttpServer.StatusCode.OK.getValue(), response1.getStatusCode()); + LOG.info("Get response: {}", response1.getBody()); + + BookieInfoService.BookieInfo bs = JsonUtil.fromJson(response1.getBody(), BookieInfoService.BookieInfo.class); + assertTrue(bs.getFreeSpace() > 0); + assertTrue(bs.getTotalSpace() > 0); + + // Try using POST instead of GET + HttpServiceRequest request2 = new HttpServiceRequest(null, HttpServer.Method.POST, null); + HttpServiceResponse response2 = bookieStateServer.handle(request2); + assertEquals(HttpServer.StatusCode.NOT_FOUND.getValue(), response2.getStatusCode()); + } } diff --git a/site/docs/latest/admin/http.md b/site/docs/latest/admin/http.md index 46dd080f106..50c3a7db01d 100644 --- a/site/docs/latest/admin/http.md +++ b/site/docs/latest/admin/http.md @@ -168,6 +168,24 @@ Currently all the HTTP endpoints could be divided into these 5 components: ## Bookie +### Endpoint: /api/v1/bookie/info +1. Method: GET + * Description: Get bookie info + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |501 | Not implemented | + * Body: + ```json + { + "freeSpace" : 0, + "totalSpace" : 0 + } + ``` + ### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> 1. Method: GET * Description: Get all the available bookies. From 7e1b2729031f3c6a4ff82cbbdab2274506592414 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Mon, 18 May 2020 22:59:38 -0700 Subject: [PATCH 0425/1642] Handle NPE when inet address is not able to resolve network address Descriptions of the changes in this PR: *Motivation* ``` 06:00:53.731 [BookKeeperClientScheduler-OrderedScheduler-0-0] ERROR org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Unexpected exception while handling joining bookie pulsar-tls-bookie-0.pulsar-tls-bookie.pulsar.svc.cluster.local:3181 java.lang.NullPointerException: null at org.apache.bookkeeper.net.NetUtils.resolveNetworkLocation(NetUtils.java:77) ~[org.apache.bookkeeper-bookkeeper-server-4.10.0.jar:4.10.0] ``` *TEST* The `getAddress` is final method. It is hard to mock. The method has been covered by other tests. Reviewers: Enrico Olivelli This closes #2301 from sijie/fix_network_utils --- .../client/TopologyAwareEnsemblePlacementPolicy.java | 2 +- .../main/java/org/apache/bookkeeper/net/NetUtils.java | 11 +++++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) 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 6ad06e445dc..0b84b78f4fc 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 @@ -776,7 +776,7 @@ protected BookieNode createBookieNode(BookieSocketAddress addr) { } protected String resolveNetworkLocation(BookieSocketAddress addr) { - return NetUtils.resolveNetworkLocation(dnsResolver, addr.getSocketAddress()); + return NetUtils.resolveNetworkLocation(dnsResolver, addr); } protected Set convertBookiesToNodes(Collection excludeBookies) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetUtils.java index 97762930676..937e67b0c9e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetUtils.java @@ -68,13 +68,20 @@ public static List normalizeHostNames(Collection names) { return hostNames; } - public static String resolveNetworkLocation(DNSToSwitchMapping dnsResolver, InetSocketAddress addr) { + public static String resolveNetworkLocation(DNSToSwitchMapping dnsResolver, + BookieSocketAddress addr) { List names = new ArrayList(1); + InetSocketAddress inetSocketAddress = addr.getSocketAddress(); if (dnsResolver.useHostName()) { names.add(addr.getHostName()); } else { - names.add(addr.getAddress().getHostAddress()); + InetAddress inetAddress = inetSocketAddress.getAddress(); + if (null == inetAddress) { + names.add(addr.getHostName()); + } else { + names.add(inetAddress.getHostAddress()); + } } // resolve network addresses From 87f3f9dd09236452fa53dc3fdf1cd7d2091fbe8c Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Mon, 18 May 2020 23:22:35 -0700 Subject: [PATCH 0426/1642] [BOOKKEEPER-TOOL] Fix update rate-limiting for update-ledger command ### Motivation Right now, `UpdateLedgerCmd` provides option `updatespersec` to throttle number of writes on zk. However, it throttles number of reads instead writes. Because of that it takes long time to complete this command as it slows down the read instead applying throttling while writing to zk. ### Modification - Apply throttling while updating zk. - If writes are being throttled then we also want to avoid accumulating reads so, added `maxOutstandingReads` option to manage max concurrent reads. **Note:** We also need this change for #2321 .. so, I will rebase #2321 once this PR is merged. Reviewers: Enrico Olivelli This closes #2336 from rdhabalia/update_rate --- .../apache/bookkeeper/bookie/BookieShell.java | 7 +++-- .../bookkeeper/client/MetadataUpdateLoop.java | 31 ++++++++++++++----- .../bookkeeper/client/UpdateLedgerOp.java | 12 ++++--- .../commands/bookie/FlipBookieIdCommand.java | 13 +++++++- .../bookkeeper/client/UpdateLedgerOpTest.java | 14 ++++----- .../bookie/FlipBookieIdCommandTest.java | 4 +-- 6 files changed, 57 insertions(+), 24 deletions(-) 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 d7f557520b2..bf4b57c6bd9 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 @@ -1492,6 +1492,7 @@ class UpdateLedgerCmd extends MyCommand { super(CMD_UPDATELEDGER); opts.addOption("b", "bookieId", true, "Bookie Id"); opts.addOption("s", "updatespersec", true, "Number of ledgers updating per second (default: 5 per sec)"); + opts.addOption("r", "maxOutstandingReads", true, "Max outstanding reads (default: 5 * updatespersec)"); opts.addOption("l", "limit", true, "Maximum number of ledgers to update (default: no limit)"); opts.addOption("v", "verbose", true, "Print status of the ledger updation (default: false)"); opts.addOption("p", "printprogress", true, @@ -1510,8 +1511,8 @@ String getDescription() { @Override String getUsage() { - return "updateledgers -bookieId [-updatespersec N] [-limit N] [-verbose true/false] " - + "[-printprogress N]"; + return "updateledgers -bookieId [-updatespersec N] [-maxOutstandingReads N] [-limit N] " + + "[-verbose true/false] [-printprogress N]"; } @Override @@ -1532,6 +1533,7 @@ int runCmd(CommandLine cmdLine) throws Exception { } boolean useHostName = getOptionalValue(bookieId, "hostname"); final int rate = getOptionIntValue(cmdLine, "updatespersec", 5); + final int maxOutstandingReads = getOptionIntValue(cmdLine, "maxOutstandingReads", (rate * 5)); final int limit = getOptionIntValue(cmdLine, "limit", Integer.MIN_VALUE); final boolean verbose = getOptionBooleanValue(cmdLine, "verbose", false); final long printprogress; @@ -1548,6 +1550,7 @@ int runCmd(CommandLine cmdLine) throws Exception { flags.printProgress(printprogress); flags.limit(limit); flags.updatePerSec(rate); + flags.maxOutstandingReads(maxOutstandingReads); flags.verbose(verbose); boolean result = cmd.apply(bkConf, flags); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java index 5c8cb14e652..d4f46f3b0eb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java @@ -19,6 +19,8 @@ */ package org.apache.bookkeeper.client; +import com.google.common.util.concurrent.RateLimiter; + import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.function.Supplier; @@ -56,6 +58,7 @@ class MetadataUpdateLoop { private final NeedsUpdatePredicate needsTransformation; private final MetadataTransform transform; private final LocalValueUpdater updateLocalValue; + private final RateLimiter throttler; private final String logContext; private volatile int writeLoopCount = 0; @@ -74,6 +77,14 @@ interface LocalValueUpdater { boolean updateValue(Versioned oldValue, Versioned newValue); } + MetadataUpdateLoop(LedgerManager lm, + long ledgerId, + Supplier> currentLocalValue, + NeedsUpdatePredicate needsTransformation, + MetadataTransform transform, + LocalValueUpdater updateLocalValue) { + this(lm, ledgerId, currentLocalValue, needsTransformation, transform, updateLocalValue, null); + } /** * Construct the loop. This takes a set of functions which may be called multiple times * during the loop. @@ -89,20 +100,21 @@ interface LocalValueUpdater { * second parameter and return true, return false otherwise */ MetadataUpdateLoop(LedgerManager lm, - long ledgerId, - Supplier> currentLocalValue, - NeedsUpdatePredicate needsTransformation, - MetadataTransform transform, - LocalValueUpdater updateLocalValue) { + long ledgerId, + Supplier> currentLocalValue, + NeedsUpdatePredicate needsTransformation, + MetadataTransform transform, + LocalValueUpdater updateLocalValue, + RateLimiter throttler) { this.lm = lm; this.ledgerId = ledgerId; this.currentLocalValue = currentLocalValue; this.needsTransformation = needsTransformation; this.transform = transform; this.updateLocalValue = updateLocalValue; + this.throttler = throttler; - this.logContext = String.format("UpdateLoop(ledgerId=%d,loopId=%08x)", - ledgerId, System.identityHashCode(this)); + this.logContext = String.format("UpdateLoop(ledgerId=%d,loopId=%08x)", ledgerId, System.identityHashCode(this)); } CompletableFuture> run() { @@ -120,7 +132,10 @@ private void writeLoop(Versioned currentLocal, try { if (needsTransformation.needsUpdate(currentLocal.getValue())) { LedgerMetadata transformed = transform.transform(currentLocal.getValue()); - + if (throttler != null) { + // throttler to control updates per second + throttler.acquire(); + } lm.writeLedgerMetadata(ledgerId, transformed, currentLocal.getVersion()) .whenComplete((writtenMetadata, ex) -> { if (ex == null) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java index 83143fb2ad1..14951fdd66d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java @@ -30,6 +30,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -75,7 +76,8 @@ public UpdateLedgerOp(final BookKeeper bkc, final BookKeeperAdmin admin) { * metadata */ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final BookieSocketAddress newBookieId, - final int rate, final int limit, final UpdateLedgerNotifier progressable) + final int rate, int maxOutstandingReads, final int limit, + final UpdateLedgerNotifier progressable) throws IOException, InterruptedException { final AtomicInteger issuedLedgerCnt = new AtomicInteger(); @@ -84,13 +86,14 @@ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final final Set> outstanding = Collections.newSetFromMap(new ConcurrentHashMap, Boolean>()); final RateLimiter throttler = RateLimiter.create(rate); + final Semaphore outstandingReads = new Semaphore(maxOutstandingReads); final Iterator ledgerItr = admin.listLedgers().iterator(); // iterate through all the ledgers while (ledgerItr.hasNext() && !finalPromise.isDone() && (limit == Integer.MIN_VALUE || issuedLedgerCnt.get() < limit)) { - // throttler to control updates per second - throttler.acquire(); + // semaphore to control reads according to update throttling + outstandingReads.acquire(); final long ledgerId = ledgerItr.next(); issuedLedgerCnt.incrementAndGet(); @@ -109,7 +112,7 @@ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final (metadata) -> { return replaceBookieInEnsembles(metadata, oldBookieId, newBookieId); }, - ref::compareAndSet).run(); + ref::compareAndSet, throttler).run(); }); outstanding.add(writePromise); @@ -127,6 +130,7 @@ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final updatedLedgerCnt.incrementAndGet(); progressable.progress(updatedLedgerCnt.get(), issuedLedgerCnt.get()); } + outstandingReads.release(); outstanding.remove(writePromise); }); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java index 852379f63af..701d562838c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java @@ -77,6 +77,10 @@ public static class FlipBookieIdFlags extends CliFlags { description = "Number of ledgers updating per second (default: 5 per sec)") private int updatePerSec = 5; + @Parameter(names = { "-r", + "--maxOutstandingReads" }, description = "Max outstanding reads (default: 5 * updatespersec)") + private int maxOutstandingReads = updatePerSec * 5; + @Parameter(names = {"-l", "--limit"}, description = "Maximum number of ledgers of ledgers to update (default: no limit)") private int limit = Integer.MIN_VALUE; @@ -115,6 +119,12 @@ private boolean updateLedger(ServerConfiguration conf, FlipBookieIdFlags flags) return false; } + final int maxOutstandingReads = flags.maxOutstandingReads; + if (maxOutstandingReads <= 0) { + LOG.error("Invalid maxOutstandingReads {}, should be > 0", maxOutstandingReads); + return false; + } + final int limit = flags.limit; if (limit <= 0 && limit != Integer.MIN_VALUE) { LOG.error("Invalid limit {}, should be > 0", limit); @@ -154,7 +164,8 @@ public void progress(long updated, long issued) { }; try { - updateLedgerOp.updateBookieIdInLedgers(oldBookieId, newBookieId, rate, limit, progressable); + updateLedgerOp.updateBookieIdInLedgers(oldBookieId, newBookieId, rate, maxOutstandingReads, limit, + progressable); } catch (IOException e) { LOG.error("Failed to update ledger metadata", e); return false; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java index c2b0d75704e..5ead2c972a9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java @@ -108,7 +108,7 @@ public void testManyLedgers(boolean useShortHostName) throws Exception { BookieSocketAddress toBookieAddr = new BookieSocketAddress(curBookieId.getHostName() + ":" + curBookieAddr.getPort()); UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin); - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, Integer.MIN_VALUE, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, 25, Integer.MIN_VALUE, progressable); for (LedgerHandle lh : ledgers) { lh.close(); @@ -146,22 +146,22 @@ public void testLimitLessThanTotalLedgers() throws Exception { BookieSocketAddress toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":" + curBookieAddr.getPort()); UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin); - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 7, 4, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 7, 35, 4, progressable); int updatedLedgersCount = getUpdatedLedgersCount(bk, ledgers, toBookieAddr); assertEquals("Failed to update the ledger metadata to use bookie host name", 4, updatedLedgersCount); // next execution - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 2, 10, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 2, 10, 10, progressable); updatedLedgersCount = getUpdatedLedgersCount(bk, ledgers, toBookieAddr); assertEquals("Failed to update the ledger metadata to use bookie host name", 10, updatedLedgersCount); // no ledgers - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 3, 20, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 3, 15, 20, progressable); updatedLedgersCount = getUpdatedLedgersCount(bk, ledgers, toBookieAddr); assertEquals("Failed to update the ledger metadata to use bookie host name", 10, updatedLedgersCount); // no ledgers - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 3, Integer.MIN_VALUE, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 3, 15, Integer.MIN_VALUE, progressable); updatedLedgersCount = getUpdatedLedgersCount(bk, ledgers, toBookieAddr); assertEquals("Failed to update the ledger metadata to use bookie host name", 10, updatedLedgersCount); } @@ -208,7 +208,7 @@ public void testChangeEnsembleAfterRenaming(boolean useShortHostName) throws Exc BookieSocketAddress toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":" + curBookieAddr.getPort()); UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin); - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, 100, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, 25, 100, progressable); bookieServer.shutdown(); @@ -278,7 +278,7 @@ public void addComplete(int rccb, LedgerHandle lh, long entryId, Object ctx) { BookieSocketAddress curBookieAddr = ensemble.get(0); BookieSocketAddress toBookieAddr = new BookieSocketAddress("localhost:" + curBookieAddr.getPort()); UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin); - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, 100, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, 25, 100, progressable); if (!latch.await(120, TimeUnit.SECONDS)) { throw new Exception("Entries took too long to add"); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java index a57a7584b8e..3be6b3db37a 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java @@ -101,8 +101,8 @@ public void testCommand() throws Exception { verifyNew(UpdateLedgerOp.class, times(1)).withArguments(eq(bookKeeper), eq(bookKeeperAdmin)); verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(conf)); verify(serverConfiguration, times(1)).setUseHostNameAsBookieID(anyBoolean()); - verify(updateLedgerOp, times(1)) - .updateBookieIdInLedgers(eq(bookieSocketAddress), eq(bookieSocketAddress), anyInt(), anyInt(), any()); + verify(updateLedgerOp, times(1)).updateBookieIdInLedgers(eq(bookieSocketAddress), eq(bookieSocketAddress), + anyInt(), anyInt(), anyInt(), any()); } } From f0aaa1921bd5ed935e26565cb0c4689237be0b97 Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Sat, 23 May 2020 20:39:57 +0800 Subject: [PATCH 0427/1642] [BUG] Fix wrong jdk version on install doc page ### Motivation We have adopted `lamada expression` and `stream api` in code base, it needs jdk1.8 at least. e.g org.apache.bookkeeper.meta.zk.ZKMetadataClientDriver ``` Override public void setSessionStateListener(SessionStateListener sessionStateListener) { zk.register((event) -> { // Check for expired connection. if (event.getType().equals(EventType.None) && event.getState().equals(KeeperState.Expired)) { sessionStateListener.onSessionExpired(); } }); } ``` ![image](https://user-images.githubusercontent.com/20113411/82619119-a4b12b80-9c07-11ea-92e5-a1e3f5f9ff6b.png) ### Changes Update `Java Development Kit 1.6` to `Java Development Kit 1.8` Reviewers: Matteo Minardi This closes #2343 from lamber-ken/clean-fix-install-doc --- site/bps/BP-34-cluster-metadata-checker.md | 2 +- site/docs/latest/getting-started/installation.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/site/bps/BP-34-cluster-metadata-checker.md b/site/bps/BP-34-cluster-metadata-checker.md index d022fb08b91..a0b13fd4194 100644 --- a/site/bps/BP-34-cluster-metadata-checker.md +++ b/site/bps/BP-34-cluster-metadata-checker.md @@ -1,5 +1,5 @@ --- -title: "BP-34: Cluster Metadata Checker“ +title: "BP-34: Cluster Metadata Checker" issue: https://github.com/apache/bookkeeper/issues/1602 state: Accepted release: N/A diff --git a/site/docs/latest/getting-started/installation.md b/site/docs/latest/getting-started/installation.md index 7e92b471581..9f856ec1d21 100644 --- a/site/docs/latest/getting-started/installation.md +++ b/site/docs/latest/getting-started/installation.md @@ -11,7 +11,7 @@ You can install BookKeeper either by [downloading](#download) a [GZipped](http:/ ## 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 +* [Java Development Kit 1.8](http://www.oracle.com/technetwork/java/javase/downloads/index.html) or later * [Maven 3.0](https://maven.apache.org/install.html) or later ## Download From 893efc496445e91577f2cce2cb0e65a6ab11ec91 Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Sat, 23 May 2020 20:41:59 +0800 Subject: [PATCH 0428/1642] [BUG] Fix flaky testLedgerOpenAfterBKCrashed ### Motivation `BookieFailureTest#testLedgerOpenAfterBKCrashed` is a flaky test case, it cause some pr failed. ``` org.apache.bookkeeper.client.BKException$BKLedgerRecoveryException: Error while recovering ledger at org.apache.bookkeeper.client.SyncCallbackUtils.finish(SyncCallbackUtils.java:83) at org.apache.bookkeeper.client.SyncCallbackUtils$SyncOpenCallback.openComplete(SyncCallbackUtils.java:157) at org.apache.bookkeeper.client.LedgerOpenOp.openComplete(LedgerOpenOp.java:232) at org.apache.bookkeeper.client.LedgerOpenOp$1.safeOperationComplete(LedgerOpenOp.java:201) at org.apache.bookkeeper.client.LedgerOpenOp$1.safeOperationComplete(LedgerOpenOp.java:193) at org.apache.bookkeeper.util.OrderedGenericCallback.operationComplete(OrderedGenericCallback.java:62) at org.apache.bookkeeper.proto.BookkeeperInternalCallbacks$TimedGenericCallback.operationComplete(BookkeeperInternalCallbacks.java:189) at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.lambda$recover$5(ReadOnlyLedgerHandle.java:295) at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760) at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736) at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977) at org.apache.bookkeeper.client.LedgerRecoveryOp.submitCallback(LedgerRecoveryOp.java:136) at org.apache.bookkeeper.client.LedgerRecoveryOp.onEntryComplete(LedgerRecoveryOp.java:202) at org.apache.bookkeeper.client.ListenerBasedPendingReadOp.submitCallback(ListenerBasedPendingReadOp.java:67) at org.apache.bookkeeper.client.PendingReadOp$LedgerEntryRequest.fail(PendingReadOp.java:170) at org.apache.bookkeeper.client.PendingReadOp$SequenceReadRequest.sendNextRead(PendingReadOp.java:392) at org.apache.bookkeeper.client.PendingReadOp$SequenceReadRequest.logErrorAndReattemptRead(PendingReadOp.java:435) at org.apache.bookkeeper.client.PendingReadOp.readEntryComplete(PendingReadOp.java:581) at org.apache.bookkeeper.proto.BookieClientImpl$2.safeRun(BookieClientImpl.java:369) at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:748) ``` ![image](https://user-images.githubusercontent.com/20113411/82434983-3add3880-9ac6-11ea-96a3-24fbd71d16d4.png) ### Changes Test case should cover following situation, which shutdown any one of bookies, and try to open ledger with no bookie recovery. ``` Bookies: 4 ensSize: 3 writeQuorumSize: 2 ackQuorumSize: 2 ``` Reviewers: Enrico Olivelli This closes #2337 from lamber-ken/bk-fix-wrong-test --- .../apache/bookkeeper/test/BookieFailureTest.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) 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 c60a2dbd645..7b1f6e04c94 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 @@ -320,8 +320,20 @@ public void testLedgerOpenAfterBKCrashed() throws Exception { // try to open ledger with recovery LedgerHandle afterlh = bkc.openLedger(beforelh.getId(), digestType, "".getBytes()); - assertEquals(beforelh.getLastAddPushed(), afterlh.getLastAddConfirmed()); + + // try to open ledger no recovery + // bookies: 4, ensSize: 3, ackQuorumSize: 2 + LedgerHandle beforelhWithNoRecovery = bkc.createLedger(numBookies - 1 , 2, digestType, "".getBytes()); + for (int i = 0; i < numEntries; i++) { + beforelhWithNoRecovery.addEntry(tmp.getBytes()); + } + + // shutdown first bookie server + killBookie(0); + + // try to open ledger no recovery, should be able to open ledger + bkc.openLedger(beforelhWithNoRecovery.getId(), digestType, "".getBytes()); } /** From 0c4aa4824b87e16a01e3e4b2b326b33fad0e204b Mon Sep 17 00:00:00 2001 From: Matteo Minardi Date: Wed, 27 May 2020 08:14:01 +0200 Subject: [PATCH 0429/1642] BP-38: Publish Bookie Service Info on Metadata Service Starting from the implementation of the #2213 PR ### Changes - Adding the ability for the Bookie to advertise on the Metadata Service (ZooKeeper) the exposed ports/services, like the HTTP service, Metrics - Implemented the Protobuf serialization of the BookieServiceInfo - Added the command EndpointInfoCommand to retrieve remote bookie endpoint information both for "bkctl" and "bookkeeper shell". Usage: ` bin/bookkeeper shell endpointinfo -bookieid ` Master Issue: #2215 Proposal Doc: https://bookkeeper.apache.org/bps/BP-38-bookie-endpoint-discovery Reviewers: Jia Zhai , Enrico Olivelli This closes #2314 from mino181295/single/endpoint-discovery-impl --- .../component/ComponentInfoPublisher.java | 146 +++++++++++++++ .../common/component/ComponentStarter.java | 2 + .../common/component/LifecycleComponent.java | 3 + .../component/LifecycleComponentStack.java | 31 ++++ .../component/TestComponentStarter.java | 1 + .../src/main/proto/DataFormats.proto | 23 +++ .../org/apache/bookkeeper/bookie/Bookie.java | 35 +--- .../apache/bookkeeper/bookie/BookieShell.java | 46 +++++ .../bookkeeper/bookie/BookieStateManager.java | 18 +- .../bookkeeper/bookie/ReadOnlyBookie.java | 10 +- .../bookkeeper/client/BookKeeperAdmin.java | 10 ++ .../discover/BookieServiceInfo.java | 166 ++++++++++++++++++ .../discover/BookieServiceInfoUtils.java | 51 ++++++ .../discover/RegistrationClient.java | 21 +++ .../discover/RegistrationManager.java | 3 +- .../discover/ZKRegistrationClient.java | 73 ++++++++ .../discover/ZKRegistrationManager.java | 54 +++++- .../apache/bookkeeper/proto/BookieServer.java | 16 +- .../org/apache/bookkeeper/server/Main.java | 41 ++++- .../server/service/BookieService.java | 35 +++- .../server/service/HttpService.java | 14 ++ .../commands/bookies/EndpointInfoCommand.java | 125 +++++++++++++ .../bookkeeper/util/LocalBookKeeper.java | 74 ++++---- .../bookie/BookieInitializationTest.java | 61 ++++++- .../bookkeeper/bookie/StateManagerTest.java | 3 +- .../client/BookKeeperAdminTest.java | 79 +++++++++ .../discover/BookieServiceInfoTest.java | 91 ++++++++++ .../AuditorPlacementPolicyCheckTest.java | 11 +- .../replication/AuditorReplicasCheckTest.java | 3 +- .../apache/bookkeeper/server/TestMain.java | 18 +- .../test/BookKeeperClusterTestCase.java | 11 +- .../etcd/EtcdRegistrationManager.java | 4 +- .../metadata/etcd/EtcdRegistrationTest.java | 19 +- .../etcd/testing/EtcdBKClusterTestBase.java | 4 +- site/_data/cli/shell.yaml | 2 + .../BKRegistrationNameResolverTest.java | 5 +- .../org/apache/distributedlog/TestTxnId.java | 4 +- .../stream/server/StorageServer.java | 35 +++- .../stream/server/service/BookieService.java | 10 +- .../service/RegistrationStateService.java | 4 +- .../cli/commands/BookieCommandGroup.java | 2 + 41 files changed, 1240 insertions(+), 124 deletions(-) create mode 100644 bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentInfoPublisher.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/BookieServiceInfo.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/BookieServiceInfoUtils.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/EndpointInfoCommand.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/BookieServiceInfoTest.java diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentInfoPublisher.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentInfoPublisher.java new file mode 100644 index 00000000000..0085e82b7c1 --- /dev/null +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentInfoPublisher.java @@ -0,0 +1,146 @@ +/* + * 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.common.component; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import lombok.extern.slf4j.Slf4j; + +/** + * Allows a component to publish information about + * the services it implements, the endpoints it exposes + * and other useful information for management tools and client. + */ +@Slf4j +public class ComponentInfoPublisher { + + private final Map properties = new ConcurrentHashMap<>(); + private final Map endpoints = new ConcurrentHashMap<>(); + + /** + * Endpoint information. + */ + public static final class EndpointInfo { + + private final String id; + private final int port; + private final String host; + private final String protocol; + private final List auth; + private final List extensions; + + public EndpointInfo(String id, int port, String host, String protocol, + List auth, List extensions) { + this.id = id; + this.port = port; + this.host = host; + this.protocol = protocol; + this.auth = auth == null ? Collections.emptyList() : Collections.unmodifiableList(auth); + this.extensions = extensions == null ? Collections.emptyList() : Collections.unmodifiableList(extensions); + } + + public String getId() { + return id; + } + + public int getPort() { + return port; + } + + public String getHost() { + return host; + } + + public String getProtocol() { + return protocol; + } + + public List getAuth() { + return auth; + } + + public List getExtensions() { + return extensions; + } + + @Override + public String toString() { + return "EndpointInfo{" + "id=" + id + ", port=" + port + ", host=" + host + ", protocol=" + protocol + ", " + + "auth=" + auth + ", extensions=" + extensions + '}'; + } + + } + + private volatile boolean startupFinished; + + /** + * Publish an information about the system, like an endpoint address. + * + * @param key the key + * @param value the value, null values are not allowed. + */ + public void publishProperty(String key, String value) { + if (log.isDebugEnabled()) { + log.debug("publish {}={}", key, value); + } + if (startupFinished) { + throw new IllegalStateException("Server already started, cannot publish " + key); + } + Objects.requireNonNull(key); + Objects.requireNonNull(value, "Value for " + key + " cannot be null"); + + properties.put(key, value); + } + + public void publishEndpoint(EndpointInfo endpoint) { + if (log.isDebugEnabled()) { + log.debug("publishEndpoint {} on {}", endpoint, this); + } + EndpointInfo exists = endpoints.put(endpoint.id, endpoint); + if (exists != null) { + throw new IllegalStateException("An endpoint with id " + endpoint.id + + " has already been published: " + exists); + } + } + + public Map getProperties() { + if (!startupFinished) { + throw new IllegalStateException("Startup not yet finished"); + } + return Collections.unmodifiableMap(properties); + } + + public Map getEndpoints() { + if (!startupFinished) { + throw new IllegalStateException("Startup not yet finished"); + } + return Collections.unmodifiableMap(endpoints); + } + + /** + * Called by the framework to signal that preparation of startup is done, + * so we have gathered all of the available information. + */ + public void startupFinished() { + startupFinished = true; + } + +} diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java index acb643b2ed8..50fd81a21c1 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java @@ -79,6 +79,8 @@ public static CompletableFuture startComponent(LifecycleComponent componen shutdownHookThread.start(); }); + component.publishInfo(new ComponentInfoPublisher()); + log.info("Starting component {}.", component.getName()); component.start(); log.info("Started component {}.", component.getName()); diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponent.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponent.java index d3820663ee4..836cffb1f91 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponent.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponent.java @@ -33,6 +33,9 @@ public interface LifecycleComponent extends AutoCloseable { void removeLifecycleListener(LifecycleListener listener); + default void publishInfo(ComponentInfoPublisher componentInfoPublisher) { + } + void start(); void stop(); diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponentStack.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponentStack.java index d60691009f1..5b94e049e7d 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponentStack.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponentStack.java @@ -26,10 +26,12 @@ import com.google.common.collect.Lists; import java.lang.Thread.UncaughtExceptionHandler; import java.util.List; +import lombok.extern.slf4j.Slf4j; /** * A stack of {@link LifecycleComponent}s. */ +@Slf4j public class LifecycleComponentStack implements LifecycleComponent { public static Builder newBuilder() { @@ -42,12 +44,19 @@ public static Builder newBuilder() { public static class Builder { private String name; + private ComponentInfoPublisher componentInfoPublisher; private final List components; private Builder() { components = Lists.newArrayList(); } + public Builder withComponentInfoPublisher(ComponentInfoPublisher componentInfoPublisher) { + checkNotNull(componentInfoPublisher, "ComponentInfoPublisher is null"); + this.componentInfoPublisher = componentInfoPublisher; + return this; + } + public Builder addComponent(LifecycleComponent component) { checkNotNull(component, "Lifecycle component is null"); components.add(component); @@ -64,6 +73,7 @@ public LifecycleComponentStack build() { checkArgument(!components.isEmpty(), "Lifecycle component stack is empty : " + components); return new LifecycleComponentStack( name, + componentInfoPublisher != null ? componentInfoPublisher : new ComponentInfoPublisher(), ImmutableList.copyOf(components)); } @@ -71,10 +81,13 @@ public LifecycleComponentStack build() { private final String name; private final ImmutableList components; + private final ComponentInfoPublisher componentInfoPublisher; private LifecycleComponentStack(String name, + ComponentInfoPublisher componentInfoPublisher, ImmutableList components) { this.name = name; + this.componentInfoPublisher = componentInfoPublisher; this.components = components; } @@ -108,8 +121,26 @@ public void removeLifecycleListener(LifecycleListener listener) { components.forEach(component -> component.removeLifecycleListener(listener)); } + @Override + public void publishInfo(ComponentInfoPublisher componentInfoPublisher) { + components.forEach(component -> { + if (log.isDebugEnabled()) { + log.debug("calling publishInfo on {} ", component); + } + component.publishInfo(componentInfoPublisher); + }); + } + @Override public void start() { + components.forEach(component -> { + if (log.isDebugEnabled()) { + log.debug("calling publishInfo on {} ", component); + } + component.publishInfo(componentInfoPublisher); + }); + componentInfoPublisher.startupFinished(); + components.forEach(component -> component.start()); } diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/component/TestComponentStarter.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/component/TestComponentStarter.java index ed97adcacea..ef35a9d7b39 100644 --- a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/component/TestComponentStarter.java +++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/component/TestComponentStarter.java @@ -42,6 +42,7 @@ public void testStartComponent() { LifecycleComponent component = mock(LifecycleComponent.class); when(component.getName()).thenReturn("test-start-component"); ComponentStarter.startComponent(component); + verify(component).publishInfo(any(ComponentInfoPublisher.class)); verify(component).start(); } diff --git a/bookkeeper-proto/src/main/proto/DataFormats.proto b/bookkeeper-proto/src/main/proto/DataFormats.proto index 6591d651ebd..0af1bb12cc3 100644 --- a/bookkeeper-proto/src/main/proto/DataFormats.proto +++ b/bookkeeper-proto/src/main/proto/DataFormats.proto @@ -118,3 +118,26 @@ message PlacementPolicyCheckFormat { message ReplicasCheckFormat { optional int64 replicasCheckCTime = 1; } + +/** + * information about services exposed by a Bookie. + */ +message BookieServiceInfoFormat { + + /** + * Information about an endpoint. + */ + message Endpoint { + required string id = 1; + required int32 port = 2; + required string host = 3; + required string protocol = 4; + + repeated string auth = 5; + repeated string extensions = 6; + } + + repeated Endpoint endpoints = 6; + map properties = 7; +} + 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 7a655d65d30..6a2209aaeee 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 @@ -57,6 +57,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.bookkeeper.bookie.BookieException.BookieIllegalOpException; @@ -73,6 +74,7 @@ import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage; import org.apache.bookkeeper.common.util.Watcher; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; @@ -123,6 +125,7 @@ public class Bookie extends BookieCriticalThread { static final long METAENTRY_ID_LEDGER_EXPLICITLAC = -0x8000; private final LedgerDirsManager ledgerDirsManager; + protected final Supplier bookieServiceInfoProvider; private final LedgerDirsManager indexDirsManager; LedgerDirsMonitor dirsMonitor; @@ -611,7 +614,7 @@ public static File[] getCurrentDirectories(File[] dirs) { public Bookie(ServerConfiguration conf) throws IOException, InterruptedException, BookieException { - this(conf, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT); + this(conf, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT, BookieServiceInfo.NO_INFO); } private static LedgerStorage buildLedgerStorage(ServerConfiguration conf) throws IOException { @@ -674,9 +677,11 @@ public void checkpointComplete(Checkpoint checkpoint, boolean compact) return ledgerStorage; } - public Bookie(ServerConfiguration conf, StatsLogger statsLogger, ByteBufAllocator allocator) + public Bookie(ServerConfiguration conf, StatsLogger statsLogger, + ByteBufAllocator allocator, Supplier bookieServiceInfoProvider) throws IOException, InterruptedException, BookieException { super("Bookie-" + conf.getBookiePort()); + this.bookieServiceInfoProvider = bookieServiceInfoProvider; this.statsLogger = statsLogger; this.conf = conf; this.journalDirectories = Lists.newArrayList(); @@ -797,7 +802,8 @@ public void start() { } StateManager initializeStateManager() throws IOException { - return new BookieStateManager(conf, statsLogger, metadataDriver, ledgerDirsManager); + return new BookieStateManager(conf, statsLogger, metadataDriver, + ledgerDirsManager, bookieServiceInfoProvider); } void readJournal() throws IOException, BookieException { @@ -1583,29 +1589,6 @@ private static boolean cleanDir(File dir) { return true; } - /** - * @param args - * @throws IOException - * @throws InterruptedException - */ - public static void main(String[] args) - throws IOException, InterruptedException, BookieException { - Bookie b = new Bookie(new ServerConfiguration()); - b.start(); - CounterCallback cb = new CounterCallback(); - long start = System.currentTimeMillis(); - for (int i = 0; i < 100000; i++) { - ByteBuf buff = Unpooled.buffer(1024); - buff.writeLong(1); - buff.writeLong(i); - cb.incCount(); - b.addEntry(buff, false /* ackBeforeSync */, cb, null, new byte[0]); - } - cb.waitZero(); - long end = System.currentTimeMillis(); - System.out.println("Took " + (end - start) + "ms"); - } - /** * Returns exit code - cause of failure. * 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 bf4b57c6bd9..00c99a7ee25 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 @@ -64,6 +64,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.RegenerateInterleavedStorageIndexFileCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.DecommissionCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.EndpointInfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InstanceIdCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; @@ -137,6 +138,7 @@ public class BookieShell implements Tool { static final String CMD_DELETELEDGER = "deleteledger"; static final String CMD_BOOKIEINFO = "bookieinfo"; static final String CMD_DECOMMISSIONBOOKIE = "decommissionbookie"; + static final String CMD_ENDPOINTINFO = "endpointinfo"; static final String CMD_LOSTBOOKIERECOVERYDELAY = "lostbookierecoverydelay"; static final String CMD_TRIGGERAUDIT = "triggeraudit"; static final String CMD_CONVERT_TO_DB_STORAGE = "convert-to-db-storage"; @@ -1707,6 +1709,49 @@ public int runCmd(CommandLine cmdLine) throws Exception { } } + /** + * Command to retrieve remote bookie endpoint information. + */ + class EndpointInfoCmd extends MyCommand { + Options lOpts = new Options(); + + EndpointInfoCmd() { + super(CMD_ENDPOINTINFO); + lOpts.addOption("b", "bookieid", true, "Bookie Id"); + } + + @Override + String getDescription() { + return "Get info about a remote bookie with a specific bookie address (bookieid)"; + } + + @Override + String getUsage() { + return CMD_ENDPOINTINFO + " [-bookieid ]"; + } + + @Override + Options getOptions() { + return lOpts; + } + + @Override + public int runCmd(CommandLine cmdLine) throws Exception { + EndpointInfoCommand cmd = new EndpointInfoCommand(); + EndpointInfoCommand.EndpointInfoFlags flags = new EndpointInfoCommand.EndpointInfoFlags(); + final String bookieId = cmdLine.getOptionValue("bookieid"); + flags.bookie(bookieId); + if (StringUtils.isBlank(bookieId)) { + LOG.error("Invalid argument list!"); + this.printUsage(); + return -1; + } + + boolean result = cmd.apply(bkConf, flags); + return (result) ? 0 : -1; + } + } + /** * A facility for reporting update ledger progress. */ @@ -1907,6 +1952,7 @@ int runCmd(CommandLine cmdLine) throws Exception { commands.put(CMD_DELETELEDGER, new DeleteLedgerCmd()); commands.put(CMD_BOOKIEINFO, new BookieInfoCmd()); commands.put(CMD_DECOMMISSIONBOOKIE, new DecommissionBookieCmd()); + commands.put(CMD_ENDPOINTINFO, new EndpointInfoCmd()); commands.put(CMD_CONVERT_TO_DB_STORAGE, new ConvertToDbStorageCmd()); commands.put(CMD_CONVERT_TO_INTERLEAVED_STORAGE, new ConvertToInterleavedStorageCmd()); commands.put(CMD_REBUILD_DB_LEDGER_LOCATIONS_INDEX, new RebuildDbLedgerLocationsIndexCmd()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java index 54d6993e0ac..45b48fb4d83 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java @@ -39,6 +39,7 @@ import java.util.function.Supplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.stats.Gauge; @@ -61,6 +62,7 @@ public class BookieStateManager implements StateManager { private static final Logger LOG = LoggerFactory.getLogger(BookieStateManager.class); private final ServerConfiguration conf; + private final Supplier bookieServiceInfoProvider; private final List statusDirs; // use an executor to execute the state changes task @@ -90,7 +92,8 @@ public class BookieStateManager implements StateManager { public BookieStateManager(ServerConfiguration conf, StatsLogger statsLogger, MetadataBookieDriver metadataDriver, - LedgerDirsManager ledgerDirsManager) throws IOException { + LedgerDirsManager ledgerDirsManager, + Supplier bookieServiceInfoProvider) throws IOException { this( conf, statsLogger, @@ -102,18 +105,21 @@ public BookieStateManager(ServerConfiguration conf, } catch (UnknownHostException e) { throw new UncheckedIOException("Failed to resolve bookie id", e); } - }); + }, + bookieServiceInfoProvider); } public BookieStateManager(ServerConfiguration conf, StatsLogger statsLogger, Supplier rm, List statusDirs, - Supplier bookieIdSupplier) throws IOException { + Supplier bookieIdSupplier, + Supplier bookieServiceInfoProvider) throws IOException { this.conf = conf; this.rm = rm; this.statusDirs = statusDirs; // ZK ephemeral node for this Bookie. this.bookieId = bookieIdSupplier.get(); + this.bookieServiceInfoProvider = bookieServiceInfoProvider; // 1 : up, 0 : readonly, -1 : unregistered this.serverStatusGauge = new Gauge() { @Override @@ -143,7 +149,7 @@ private boolean isRegistrationManagerDisabled() { BookieStateManager(ServerConfiguration conf, MetadataBookieDriver metadataDriver) throws IOException { this(conf, NullStatsLogger.INSTANCE, metadataDriver, new LedgerDirsManager(conf, conf.getLedgerDirs(), new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), - NullStatsLogger.INSTANCE)); + NullStatsLogger.INSTANCE), BookieServiceInfo.NO_INFO); } @Override @@ -263,7 +269,7 @@ private void doRegisterBookie(boolean isReadOnly) throws IOException { rmRegistered.set(false); try { - rm.get().registerBookie(bookieId, isReadOnly); + rm.get().registerBookie(bookieId, isReadOnly, bookieServiceInfoProvider.get()); rmRegistered.set(true); } catch (BookieException e) { throw new IOException(e); @@ -333,7 +339,7 @@ public void doTransitionToReadOnlyMode() { return; } try { - rm.get().registerBookie(bookieId, true); + rm.get().registerBookie(bookieId, true, bookieServiceInfoProvider.get()); } catch (BookieException e) { LOG.error("Error in transition to ReadOnly Mode." + " Shutting down", e); 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 cb6e9f09817..3c84a06eee4 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 @@ -24,7 +24,9 @@ import io.netty.buffer.ByteBufAllocator; import java.io.IOException; +import java.util.function.Supplier; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -40,9 +42,10 @@ public class ReadOnlyBookie extends Bookie { private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyBookie.class); - public ReadOnlyBookie(ServerConfiguration conf, StatsLogger statsLogger, ByteBufAllocator allocator) + public ReadOnlyBookie(ServerConfiguration conf, StatsLogger statsLogger, + ByteBufAllocator allocator, Supplier bookieServiceInfoProvider) throws IOException, KeeperException, InterruptedException, BookieException { - super(conf, statsLogger, allocator); + super(conf, statsLogger, allocator, bookieServiceInfoProvider); if (conf.isReadOnlyModeEnabled()) { stateManager.forceToReadOnly(); } else { @@ -55,7 +58,8 @@ public ReadOnlyBookie(ServerConfiguration conf, StatsLogger statsLogger, ByteBuf @Override StateManager initializeStateManager() throws IOException { - return new BookieStateManager(conf, statsLogger, metadataDriver, getLedgerDirsManager()) { + return new BookieStateManager(conf, statsLogger, metadataDriver, getLedgerDirsManager(), + bookieServiceInfoProvider) { @Override public void doTransitionToWritableMode() { 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 cac1d9d73dc..ea6cd9fbf17 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 @@ -52,6 +52,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.Predicate; +import lombok.SneakyThrows; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; @@ -62,8 +63,10 @@ import org.apache.bookkeeper.client.SyncCallbackUtils.SyncOpenCallback; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncReadCallback; import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; @@ -226,6 +229,13 @@ public Collection getAllBookies() return bkc.bookieWatcher.getAllBookies(); } + @SneakyThrows + public BookieServiceInfo getBookieServiceInfo(String bookiedId) + throws BKException { + return FutureUtils.result(bkc.getMetadataClientDriver() + .getRegistrationClient().getBookieServiceInfo(bookiedId)).getValue(); + } + /** * Get a list of readonly bookies synchronously. * diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/BookieServiceInfo.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/BookieServiceInfo.java new file mode 100644 index 00000000000..8b23a70143b --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/BookieServiceInfo.java @@ -0,0 +1,166 @@ +/* + * 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.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +/** + * Information about services exposed by a Bookie. + */ +public final class BookieServiceInfo { + + /** + * Default empty implementation. + */ + public static final BookieServiceInfo EMPTY = new BookieServiceInfo( + Collections.emptyMap(), + Collections.emptyList() + ); + + /** + * Default empty implementation. + */ + public static final Supplier NO_INFO = () -> EMPTY; + + private Map properties; + private List endpoints; + + public BookieServiceInfo(Map properties, List endpoints) { + this.properties = Collections.unmodifiableMap(properties); + this.endpoints = Collections.unmodifiableList(endpoints); + } + + public BookieServiceInfo() { + this(Collections.emptyMap(), Collections.emptyList()); + } + + /** + * Unmodifiable map with bookie wide information. + * + * @return the map + */ + public Map getProperties() { + return properties; + } + + /** + * Unmodifieable structure with the list of exposed endpoints. + * + * @return the list. + */ + public List getEndpoints() { + return endpoints; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + public void setEndpoints(List endpoints) { + this.endpoints = endpoints; + } + + /** + * Information about an endpoint. + */ + public static final class Endpoint { + + private String id; + private int port; + private String host; + private String protocol; + private List auth; + private List extensions; + + public Endpoint(String id, int port, String host, String protocol, List auth, List extensions) { + this.id = id; + this.port = port; + this.host = host; + this.protocol = protocol; + this.auth = auth; + this.extensions = extensions; + } + + public Endpoint() { + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public int getPort() { + return port; + } + + public void setPort(int port) { + this.port = port; + } + + public String getHost() { + return host; + } + + public void setHost(String host) { + this.host = host; + } + + public String getProtocol() { + return protocol; + } + + public void setProtocol(String protocol) { + this.protocol = protocol; + } + + public List getAuth() { + return auth; + } + + public void setAuth(List auth) { + this.auth = auth; + } + + public List getExtensions() { + return extensions; + } + + public void setExtensions(List extensions) { + this.extensions = extensions; + } + + @Override + public String toString() { + return "EndpointInfo{" + "id=" + id + ", port=" + port + ", host=" + host + ", protocol=" + protocol + ", " + + "auth=" + auth + ", extensions=" + extensions + '}'; + } + + } + + @Override + public String toString() { + return "BookieServiceInfo{" + "properties=" + properties + ", endpoints=" + endpoints + '}'; + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/BookieServiceInfoUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/BookieServiceInfoUtils.java new file mode 100644 index 00000000000..252cb3bceae --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/BookieServiceInfoUtils.java @@ -0,0 +1,51 @@ +/* + * 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.net.UnknownHostException; +import java.util.Arrays; +import java.util.Collections; +import org.apache.bookkeeper.net.BookieSocketAddress; + +/** + * Utility class for {@link BookieServiceInfo}. + */ +public final class BookieServiceInfoUtils { + + /** + * Creates a default legacy bookie info implementation. + * In the default implementation there is one endpoint with + * bookie-rpc protocol and the bookie id in the host port. + * + * @param bookieId bookie id + * @return default implementation of a BookieServiceInfo + * @throws UnknownHostException if the given bookieId is invalid + */ + public static BookieServiceInfo buildLegacyBookieServiceInfo(String bookieId) throws UnknownHostException { + BookieSocketAddress address = new BookieSocketAddress(bookieId); + BookieServiceInfo.Endpoint endpoint = new BookieServiceInfo.Endpoint(); + endpoint.setId(bookieId); + endpoint.setHost(address.getHostName()); + endpoint.setPort(address.getPort()); + endpoint.setProtocol("bookie-rpc"); + endpoint.setAuth(Collections.emptyList()); + endpoint.setExtensions(Collections.emptyList()); + return new BookieServiceInfo(Collections.emptyMap(), Arrays.asList(endpoint)); + } + +} 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 4056329021f..f7823c05c3c 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,11 +18,14 @@ package org.apache.bookkeeper.discover; +import java.net.UnknownHostException; import java.util.Set; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Versioned; /** @@ -65,6 +68,24 @@ interface RegistrationListener { */ CompletableFuture>> getReadOnlyBookies(); + /** + * Get detailed information about the services exposed by a Bookie. + * For old bookies it is expected to return an empty BookieServiceInfo structure. + * + * @param bookieId this is the id of the bookie, it can be computed from a {@link BookieSocketAddress} + * @return a future represents the available information. + * + * @since 4.11 + */ + default CompletableFuture> getBookieServiceInfo(String bookieId) { + try { + BookieServiceInfo bookieServiceInfo = BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId); + return FutureUtils.value(new Versioned<>(bookieServiceInfo, new LongVersion(-1))); + } catch (UnknownHostException e) { + return FutureUtils.exception(e); + } + } + /** * Watch the changes of bookies. * 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 index 3d357d4dbe8..e5b0f6a4171 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java @@ -59,9 +59,10 @@ interface RegistrationListener { * * @param bookieId bookie id * @param readOnly whether to register it as writable or readonly + * @param serviceInfo information about services exposed by the Bookie * @throws BookieException when fail to register a bookie. */ - void registerBookie(String bookieId, boolean readOnly) throws BookieException; + void registerBookie(String bookieId, boolean readOnly, BookieServiceInfo serviceInfo) throws BookieException; /** * Unregistering the bookie server as bookieId. 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 index 6629f0c2f1e..6995a14011f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java @@ -22,6 +22,7 @@ import static org.apache.bookkeeper.util.BookKeeperConstants.COOKIE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import java.io.IOException; import java.util.HashSet; @@ -33,23 +34,28 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; +import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.Getter; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BKException; 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.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat; 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.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.Stat; /** * ZooKeeper based {@link RegistrationClient}. @@ -212,6 +218,73 @@ public CompletableFuture>> getReadOnlyBookies return getChildren(bookieReadonlyRegistrationPath, null); } + @Override + public CompletableFuture> getBookieServiceInfo(String bookieId) { + String pathAsWritable = bookieRegistrationPath + "/" + bookieId; + String pathAsReadonly = bookieReadonlyRegistrationPath + "/" + bookieId; + + CompletableFuture> promise = new CompletableFuture<>(); + zk.getData(pathAsWritable, false, (int rc, String path, Object o, byte[] bytes, Stat stat) -> { + if (KeeperException.Code.OK.intValue() == rc) { + try { + BookieServiceInfo bookieServiceInfo = deserializeBookieServiceInfo(bookieId, bytes); + promise.complete(new Versioned<>(bookieServiceInfo, new LongVersion(stat.getCversion()))); + } catch (IOException ex) { + promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path)); + return; + } + } else if (KeeperException.Code.NONODE.intValue() == rc) { + // not found, looking for a readonly bookie + zk.getData(pathAsReadonly, false, (int rc2, String path2, Object o2, byte[] bytes2, Stat stat2) -> { + if (KeeperException.Code.OK.intValue() == rc2) { + try { + BookieServiceInfo bookieServiceInfo = deserializeBookieServiceInfo(bookieId, bytes2); + promise.complete(new Versioned<>(bookieServiceInfo, new LongVersion(stat2.getCversion()))); + } catch (IOException ex) { + promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc2), path2)); + return; + } + } else { + // not found as writable and readonly, the bookie is offline + promise.completeExceptionally(BKException.create(BKException.Code.NoBookieAvailableException)); + } + }, null); + } else { + promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path)); + } + }, null); + return promise; + } + + @SuppressWarnings("unchecked") + @VisibleForTesting + static BookieServiceInfo deserializeBookieServiceInfo(String bookieId, byte[] bookieServiceInfo) + throws IOException { + if (bookieServiceInfo == null || bookieServiceInfo.length == 0) { + return BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId); + } + + BookieServiceInfoFormat builder = BookieServiceInfoFormat.parseFrom(bookieServiceInfo); + BookieServiceInfo bsi = new BookieServiceInfo(); + List endpoints = builder.getEndpointsList().stream() + .map(e -> { + BookieServiceInfo.Endpoint endpoint = new BookieServiceInfo.Endpoint(); + endpoint.setId(e.getId()); + endpoint.setPort(e.getPort()); + endpoint.setHost(e.getHost()); + endpoint.setProtocol(e.getProtocol()); + endpoint.setAuth(e.getAuthList()); + endpoint.setExtensions(e.getExtensionsList()); + return endpoint; + }) + .collect(Collectors.toList()); + + bsi.setEndpoints(endpoints); + bsi.setProperties(builder.getPropertiesMap()); + + return bsi; + } + private CompletableFuture>> getChildren(String regPath, Watcher watcher) { CompletableFuture>> future = FutureUtils.createFuture(); zk.getChildren(regPath, watcher, (rc, path, ctx, children, stat) -> { 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 index e9bd7849807..9272eb8401d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java @@ -18,15 +18,17 @@ package org.apache.bookkeeper.discover; -import static java.nio.charset.StandardCharsets.UTF_8; +import static com.google.common.base.Charsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.COOKIE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.EMPTY_BYTE_ARRAY; import static org.apache.bookkeeper.util.BookKeeperConstants.INSTANCEID; import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Charsets; import com.google.common.collect.Lists; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Collection; import java.util.List; @@ -34,6 +36,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.BookieException.BookieIllegalOpException; @@ -52,6 +55,7 @@ import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.LongVersion; @@ -213,21 +217,53 @@ public void process(WatchedEvent event) { } @Override - public void registerBookie(String bookieId, boolean readOnly) throws BookieException { + public void registerBookie(String bookieId, boolean readOnly, + BookieServiceInfo bookieServiceInfo) throws BookieException { if (!readOnly) { String regPath = bookieRegistrationPath + "/" + bookieId; - doRegisterBookie(regPath); + doRegisterBookie(regPath, bookieServiceInfo); } else { - doRegisterReadOnlyBookie(bookieId); + doRegisterReadOnlyBookie(bookieId, bookieServiceInfo); } } - private void doRegisterBookie(String regPath) throws BookieException { + @VisibleForTesting + static byte[] serializeBookieServiceInfo(BookieServiceInfo bookieServiceInfo) { + if (log.isDebugEnabled()) { + log.debug("serialize BookieServiceInfo {}", bookieServiceInfo); + } + try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { + BookieServiceInfoFormat.Builder builder = BookieServiceInfoFormat.newBuilder(); + List bsiEndpoints = bookieServiceInfo.getEndpoints().stream() + .map(e -> { + return BookieServiceInfoFormat.Endpoint.newBuilder() + .setId(e.getId()) + .setPort(e.getPort()) + .setHost(e.getHost()) + .setProtocol(e.getProtocol()) + .addAllAuth(e.getAuth()) + .addAllExtensions(e.getExtensions()) + .build(); + }) + .collect(Collectors.toList()); + + builder.addAllEndpoints(bsiEndpoints); + builder.putAllProperties(bookieServiceInfo.getProperties()); + + builder.build().writeTo(os); + return os.toByteArray(); + } catch (IOException err) { + log.error("Cannot serialize bookieServiceInfo from " + bookieServiceInfo); + throw new RuntimeException(err); + } + } + + private void doRegisterBookie(String regPath, BookieServiceInfo bookieServiceInfo) 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); + zk.create(regPath, serializeBookieServiceInfo(bookieServiceInfo), zkAcls, CreateMode.EPHEMERAL); zkRegManagerInitialized = true; } } catch (KeeperException ke) { @@ -248,11 +284,11 @@ private void doRegisterBookie(String regPath) throws BookieException { } } - private void doRegisterReadOnlyBookie(String bookieId) throws BookieException { + private void doRegisterReadOnlyBookie(String bookieId, BookieServiceInfo bookieServiceInfo) throws BookieException { try { if (null == zk.exists(this.bookieReadonlyRegistrationPath, false)) { try { - zk.create(this.bookieReadonlyRegistrationPath, new byte[0], + zk.create(this.bookieReadonlyRegistrationPath, serializeBookieServiceInfo(bookieServiceInfo), zkAcls, CreateMode.PERSISTENT); } catch (NodeExistsException e) { // this node is just now created by someone. @@ -260,7 +296,7 @@ private void doRegisterReadOnlyBookie(String bookieId) throws BookieException { } String regPath = bookieReadonlyRegistrationPath + "/" + bookieId; - doRegisterBookie(regPath); + doRegisterBookie(regPath, bookieServiceInfo); // clear the write state regPath = bookieRegistrationPath + "/" + bookieId; try { 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 14d63937df5..fda7ed78246 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 @@ -33,6 +33,7 @@ import java.security.AccessControlException; import java.util.Arrays; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieCriticalThread; @@ -42,6 +43,7 @@ import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder; import org.apache.bookkeeper.common.util.JsonUtil.ParseJsonException; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.processor.RequestProcessor; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -82,10 +84,11 @@ public class BookieServer { public BookieServer(ServerConfiguration conf) throws IOException, KeeperException, InterruptedException, BookieException, UnavailableException, CompatibilityException, SecurityException { - this(conf, NullStatsLogger.INSTANCE); + this(conf, NullStatsLogger.INSTANCE, BookieServiceInfo.NO_INFO); } - public BookieServer(ServerConfiguration conf, StatsLogger statsLogger) + public BookieServer(ServerConfiguration conf, StatsLogger statsLogger, + Supplier bookieServiceInfoProvider) throws IOException, KeeperException, InterruptedException, BookieException, UnavailableException, CompatibilityException, SecurityException { this.conf = conf; @@ -102,7 +105,7 @@ public BookieServer(ServerConfiguration conf, StatsLogger statsLogger) this.statsLogger = statsLogger; this.nettyServer = new BookieNettyServer(this.conf, null, allocator); try { - this.bookie = newBookie(conf, allocator); + this.bookie = newBookie(conf, allocator, bookieServiceInfoProvider); } catch (IOException | KeeperException | InterruptedException | BookieException e) { // interrupted on constructing a bookie this.nettyServer.shutdown(); @@ -129,11 +132,12 @@ public void setExceptionHandler(UncaughtExceptionHandler exceptionHandler) { this.uncaughtExceptionHandler = exceptionHandler; } - protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) + protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator, + Supplier bookieServiceInfoProvider) throws IOException, KeeperException, InterruptedException, BookieException { return conf.isForceReadOnlyBookie() - ? new ReadOnlyBookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator) - : new Bookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator); + ? new ReadOnlyBookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator, bookieServiceInfoProvider) + : new Bookie(conf, statsLogger.scope(BOOKIE_SCOPE), allocator, bookieServiceInfoProvider); } public void start() throws InterruptedException { 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 ec163f81fb4..b569d1e997b 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 @@ -26,14 +26,19 @@ import java.util.Arrays; import java.util.List; import java.util.concurrent.ExecutionException; +import java.util.function.Supplier; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.ExitCode; import org.apache.bookkeeper.bookie.ScrubberStats; +import org.apache.bookkeeper.common.component.ComponentInfoPublisher; import org.apache.bookkeeper.common.component.ComponentStarter; import org.apache.bookkeeper.common.component.LifecycleComponent; import org.apache.bookkeeper.common.component.LifecycleComponentStack; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.UncheckedConfigurationException; +import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.discover.BookieServiceInfo.Endpoint; import org.apache.bookkeeper.server.component.ServerLifecycleComponent; import org.apache.bookkeeper.server.conf.BookieConfiguration; import org.apache.bookkeeper.server.http.BKHttpServiceProvider; @@ -287,19 +292,26 @@ private static ServerConfiguration parseCommandLine(String[] args) * @return lifecycle stack */ public static LifecycleComponentStack buildBookieServer(BookieConfiguration conf) throws Exception { - LifecycleComponentStack.Builder serverBuilder = LifecycleComponentStack.newBuilder().withName("bookie-server"); + + final ComponentInfoPublisher componentInfoPublisher = new ComponentInfoPublisher(); + + final Supplier bookieServiceInfoProvider = + () -> buildBookieServiceInfo(componentInfoPublisher); + LifecycleComponentStack.Builder serverBuilder = LifecycleComponentStack + .newBuilder() + .withComponentInfoPublisher(componentInfoPublisher) + .withName("bookie-server"); // 1. build stats provider StatsProviderService statsProviderService = new StatsProviderService(conf); StatsLogger rootStatsLogger = statsProviderService.getStatsProvider().getStatsLogger(""); - serverBuilder.addComponent(statsProviderService); log.info("Load lifecycle component : {}", StatsProviderService.class.getName()); // 2. build bookie server BookieService bookieService = - new BookieService(conf, rootStatsLogger); + new BookieService(conf, rootStatsLogger, bookieServiceInfoProvider); serverBuilder.addComponent(bookieService); log.info("Load lifecycle component : {}", BookieService.class.getName()); @@ -329,7 +341,6 @@ public static LifecycleComponentStack buildBookieServer(BookieConfiguration conf .build(); HttpService httpService = new HttpService(provider, conf, rootStatsLogger); - serverBuilder.addComponent(httpService); log.info("Load lifecycle component : {}", HttpService.class.getName()); } @@ -359,4 +370,26 @@ public static LifecycleComponentStack buildBookieServer(BookieConfiguration conf return serverBuilder.build(); } + /** + * Create the {@link BookieServiceInfo} starting from the published endpoints. + * + * @see ComponentInfoPublisher + * @param componentInfoPublisher the endpoint publisher + * @return the created bookie service info + */ + private static BookieServiceInfo buildBookieServiceInfo(ComponentInfoPublisher componentInfoPublisher) { + List endpoints = componentInfoPublisher.getEndpoints().values() + .stream().map(e -> { + return new Endpoint( + e.getId(), + e.getPort(), + e.getHost(), + e.getProtocol(), + e.getAuth(), + e.getExtensions() + ); + }).collect(Collectors.toList()); + return new BookieServiceInfo(componentInfoPublisher.getProperties(), endpoints); + } + } 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 645da03bf9a..530dd0c2774 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 @@ -20,6 +20,15 @@ import java.io.IOException; import java.lang.Thread.UncaughtExceptionHandler; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Supplier; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.component.ComponentInfoPublisher; +import org.apache.bookkeeper.common.component.ComponentInfoPublisher.EndpointInfo; +import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.server.component.ServerLifecycleComponent; import org.apache.bookkeeper.server.conf.BookieConfiguration; @@ -28,6 +37,7 @@ /** * A {@link ServerLifecycleComponent} that starts the core bookie server. */ +@Slf4j public class BookieService extends ServerLifecycleComponent { public static final String NAME = "bookie-server"; @@ -35,10 +45,11 @@ public class BookieService extends ServerLifecycleComponent { private final BookieServer server; public BookieService(BookieConfiguration conf, - StatsLogger statsLogger) + StatsLogger statsLogger, + Supplier bookieServiceInfoProvider) throws Exception { super(NAME, conf, statsLogger); - this.server = new BookieServer(conf.getServerConf(), statsLogger); + this.server = new BookieServer(conf.getServerConf(), statsLogger, bookieServiceInfoProvider); } @Override @@ -69,4 +80,24 @@ protected void doStop() { protected void doClose() throws IOException { this.server.shutdown(); } + + @Override + public void publishInfo(ComponentInfoPublisher componentInfoPublisher) { + try { + BookieSocketAddress localAddress = getServer().getLocalAddress(); + List extensions = new ArrayList<>(); + if (conf.getServerConf().getTLSProviderFactoryClass() != null) { + extensions.add("tls"); + } + EndpointInfo endpoint = new EndpointInfo("bookie", + localAddress.getPort(), + localAddress.getHostName(), + "bookie-rpc", null, extensions); + componentInfoPublisher.publishEndpoint(endpoint); + + } catch (UnknownHostException err) { + log.error("Cannot compute local address", err); + } + } + } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/HttpService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/HttpService.java index 8a86ffa0260..e5dffc9fcc0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/HttpService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/HttpService.java @@ -21,7 +21,9 @@ import static com.google.common.base.Preconditions.checkNotNull; import java.io.IOException; +import org.apache.bookkeeper.common.component.ComponentInfoPublisher; +import org.apache.bookkeeper.common.component.ComponentInfoPublisher.EndpointInfo; import org.apache.bookkeeper.http.HttpServer; import org.apache.bookkeeper.http.HttpServerLoader; import org.apache.bookkeeper.server.component.ServerLifecycleComponent; @@ -63,4 +65,16 @@ protected void doStop() { protected void doClose() throws IOException { server.stopServer(); } + + @Override + public void publishInfo(ComponentInfoPublisher componentInfoPublisher) { + if (conf.getServerConf().isHttpServerEnabled()) { + EndpointInfo endpoint = new EndpointInfo("httpserver", + conf.getServerConf().getHttpServerPort(), + "0.0.0.0", + "http", null, null); + componentInfoPublisher.publishEndpoint(endpoint); + } + } + } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/EndpointInfoCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/EndpointInfoCommand.java new file mode 100644 index 00000000000..3ef7e7bf59e --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/EndpointInfoCommand.java @@ -0,0 +1,125 @@ +/* + * 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.tools.cli.commands.bookies; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.util.Collection; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Get endpoint information about a Bookie. + */ +public class EndpointInfoCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(EndpointInfoCommand.class); + + private static final String NAME = "endpointinfo"; + private static final String DESC = "Get all end point information about a given bookie."; + + public EndpointInfoCommand() { + this(new EndpointInfoFlags()); + } + + private EndpointInfoCommand(EndpointInfoFlags flags) { + super(CliSpec.newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build()); + } + + /** + * Flags for this command. + */ + @Accessors(fluent = true) + @Setter + public static class EndpointInfoFlags extends CliFlags { + + @Parameter(required = true, names = {"-b", "--bookieid"}, description = "Get information about a remote bookie") + private String bookie; + + } + + @Override + public boolean apply(ServerConfiguration conf, EndpointInfoFlags cmdFlags) { + try { + return getEndpointInfo(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean getEndpointInfo(ServerConfiguration conf, EndpointInfoFlags flags) + throws BKException, InterruptedException, IOException { + ClientConfiguration adminConf = new ClientConfiguration(conf); + BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); + try { + final String bookieId = flags.bookie; + if (bookieId == null || bookieId.isEmpty()) { + throw new IllegalArgumentException("BookieId is required"); + } + BookieSocketAddress address = new BookieSocketAddress(bookieId); + Collection allBookies = admin.getAllBookies(); + if (!allBookies.contains(address)) { + System.out.println("Bookie " + bookieId + " does not exist, only " + allBookies); + return false; + } + BookieServiceInfo bookieServiceInfo = admin.getBookieServiceInfo(bookieId); + + System.out.println("BookiedId: " + bookieId); + if (!bookieServiceInfo.getProperties().isEmpty()) { + System.out.println("Properties"); + bookieServiceInfo.getProperties().forEach((k, v) -> { + System.out.println(k + ":" + v); + }); + } + if (!bookieServiceInfo.getEndpoints().isEmpty()) { + bookieServiceInfo.getEndpoints().forEach(e -> { + System.out.println("Endpoint: " + e.getId()); + System.out.println("Protocol: " + e.getProtocol()); + System.out.println("Address: " + e.getHost() + ":" + e.getPort()); + System.out.println("Auth: " + e.getAuth()); + System.out.println("Extensions: " + e.getExtensions()); + }); + } else { + System.out.println("Bookie did not publish any endpoint info. Maybe it is down"); + return false; + } + + return true; + } catch (Exception e) { + LOG.error("Received exception in EndpointInfoCommand ", e); + return false; + } finally { + if (admin != null) { + admin.close(); + } + } + } +} 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 b078dd45c23..0bfc923a043 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 @@ -34,19 +34,21 @@ import java.util.Iterator; import java.util.List; +import java.util.function.Supplier; +import java.util.stream.Collectors; import org.apache.bookkeeper.bookie.Bookie; -import org.apache.bookkeeper.bookie.BookieException; -import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.common.component.ComponentInfoPublisher; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.discover.BookieServiceInfo.Endpoint; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; -import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.server.conf.BookieConfiguration; +import org.apache.bookkeeper.server.service.BookieService; import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim; import org.apache.bookkeeper.shims.zk.ZooKeeperServerShimFactory; -import org.apache.bookkeeper.tls.SecurityException; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; -import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.io.FileUtils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -164,39 +166,20 @@ private static void cleanupDirectories(List dirs) throws IOException { } private List runBookies(String dirSuffix) - throws IOException, KeeperException, InterruptedException, BookieException, - UnavailableException, CompatibilityException, SecurityException, BKException, - ConfigurationException { + throws Exception { List tempDirs = new ArrayList(); try { runBookies(tempDirs, dirSuffix); return tempDirs; - } catch (IOException ioe) { + } catch (Exception ioe) { cleanupDirectories(tempDirs); throw ioe; - } catch (KeeperException ke) { - cleanupDirectories(tempDirs); - throw ke; - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - cleanupDirectories(tempDirs); - throw ie; - } catch (BookieException be) { - cleanupDirectories(tempDirs); - throw be; - } catch (UnavailableException ue) { - cleanupDirectories(tempDirs); - throw ue; - } catch (CompatibilityException ce) { - cleanupDirectories(tempDirs); - throw ce; } } @SuppressWarnings("deprecation") private void runBookies(List tempDirs, String dirSuffix) - throws IOException, KeeperException, InterruptedException, BookieException, UnavailableException, - CompatibilityException, SecurityException, BKException, ConfigurationException { + throws Exception { LOG.info("Starting Bookie(s)"); // Create Bookie Servers (B1, B2, B3) @@ -272,8 +255,18 @@ private void runBookies(List tempDirs, String dirSuffix) String fileName = Bookie.getBookieAddress(bsConfs[i]).toString() + ".conf"; serializeLocalBookieConfig(bsConfs[i], fileName); - bs[i] = new BookieServer(bsConfs[i]); - bs[i].start(); + // Mimic BookKeeper Main + final ComponentInfoPublisher componentInfoPublisher = new ComponentInfoPublisher(); + final Supplier bookieServiceInfoProvider = + () -> buildBookieServiceInfo(componentInfoPublisher); + BookieService bookieService = new BookieService(new BookieConfiguration(bsConfs[i]), + NullStatsLogger.INSTANCE, + bookieServiceInfoProvider + ); + bs[i] = bookieService.getServer(); + bookieService.publishInfo(componentInfoPublisher); + componentInfoPublisher.startupFinished(); + bookieService.start(); } /* @@ -547,4 +540,25 @@ public void shutdownBookies() { } } + /** + * Create the {@link BookieServiceInfo} starting from the published endpoints. + * + * @see ComponentInfoPublisher + * @param componentInfoPublisher the endpoint publisher + * @return the created bookie service info + */ + private static BookieServiceInfo buildBookieServiceInfo(ComponentInfoPublisher componentInfoPublisher) { + List endpoints = componentInfoPublisher.getEndpoints().values() + .stream().map(e -> { + return new Endpoint( + e.getId(), + e.getPort(), + e.getHost(), + e.getProtocol(), + e.getAuth(), + e.getExtensions() + ); + }).collect(Collectors.toList()); + return new BookieServiceInfo(componentInfoPublisher.getProperties(), endpoints); + } } 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 bff9e55b02b..2fcc8a7a2de 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 @@ -35,6 +35,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doThrow; @@ -59,6 +60,7 @@ import java.net.URLConnection; import java.security.AccessControlException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -67,6 +69,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; import org.apache.bookkeeper.bookie.BookieException.DiskPartitionDuplicationException; import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; @@ -83,6 +86,8 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.discover.BookieServiceInfo.Endpoint; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.http.HttpRouter; import org.apache.bookkeeper.http.HttpServerLoader; @@ -92,6 +97,7 @@ import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat; import org.apache.bookkeeper.replication.AutoRecoveryMain; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; @@ -293,12 +299,13 @@ public void testExitCodeZK_REG_FAIL() throws Exception { RegistrationManager rm = mock(RegistrationManager.class); doThrow(new MetadataStoreException("mocked exception")) .when(rm) - .registerBookie(anyString(), anyBoolean()); + .registerBookie(anyString(), anyBoolean(), any(BookieServiceInfo.class)); // simulating ZooKeeper exception by assigning a closed zk client to bk BookieServer bkServer = new BookieServer(conf) { @Override - protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) + protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator, + Supplier bookieServiceInfoProvider) throws IOException, KeeperException, InterruptedException, BookieException { Bookie bookie = new Bookie(conf); @@ -495,6 +502,49 @@ public void testRegNodeExistsAfterSessionTimeOut() throws Exception { } } + @Test(timeout = 20000) + public void testBookieRegistrationBookieServiceInfo() throws Exception { + final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setMetadataServiceUri(metadataServiceUri) + .setUseHostNameAsBookieID(true) + .setUseShortHostName(true) + .setListeningInterface(null); + + final String bookieId = InetAddress.getLocalHost().getCanonicalHostName().split("\\.", 2)[0] + + ":" + conf.getBookiePort(); + String bkRegPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf) + "/" + AVAILABLE_NODE + "/" + bookieId; + + driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + + Endpoint endpoint = new Endpoint("test", 1281, "localhost", "bookie-rpc", + Collections.emptyList(), Collections.emptyList()); + BookieServiceInfo bsi = new BookieServiceInfo(Collections.emptyMap(), Arrays.asList(endpoint)); + Supplier supplier = () -> bsi; + + DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()); + LedgerDirsManager ledgerDirsManager = new LedgerDirsManager( + conf, conf.getLedgerDirs(), diskChecker); + try (StateManager manager = new BookieStateManager(conf, + NullStatsLogger.INSTANCE, driver, ledgerDirsManager, supplier)) { + manager.registerBookie(true).get(); + assertTrue("Bookie registration node doesn't exists!", + driver.getRegistrationManager().isBookieRegistered(bookieId)); + } + Stat bkRegNode = zkc.exists(bkRegPath, false); + assertNotNull("Bookie registration has been failed", bkRegNode); + + byte[] bkRegNodeData = zkc.getData(bkRegPath, null, null); + assertFalse("Bookie service info not written", bkRegNodeData == null || bkRegNodeData.length == 0); + + BookieServiceInfoFormat serializedBookieServiceInfo = BookieServiceInfoFormat.parseFrom(bkRegNodeData); + BookieServiceInfoFormat.Endpoint serializedEndpoint = serializedBookieServiceInfo.getEndpoints(0); + assertNotNull("Serialized Bookie endpoint not found", serializedEndpoint); + + assertEquals(endpoint.getId(), serializedEndpoint.getId()); + assertEquals(endpoint.getHost(), serializedEndpoint.getHost()); + assertEquals(endpoint.getPort(), serializedEndpoint.getPort()); + } + /** * Verify user cannot start if user is in permittedStartupUsers conf list BKException BKUnauthorizedAccessException * if cannot start. @@ -648,7 +698,7 @@ public void testBookieServiceExceptionHandler() throws Exception { .setMetadataServiceUri(metadataServiceUri); BookieConfiguration bkConf = new BookieConfiguration(conf); - BookieService service = new BookieService(bkConf, NullStatsLogger.INSTANCE); + BookieService service = new BookieService(bkConf, NullStatsLogger.INSTANCE, BookieServiceInfo.NO_INFO); CompletableFuture startFuture = ComponentStarter.startComponent(service); // shutdown the bookie service @@ -971,7 +1021,8 @@ public MockBookieServer(ServerConfiguration conf) throws IOException, KeeperExce } @Override - protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) + protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator, + Supplier bookieServiceInfoProvider) throws IOException, KeeperException, InterruptedException, BookieException { return new MockBookieWithNoopShutdown(conf, NullStatsLogger.INSTANCE); } @@ -980,7 +1031,7 @@ protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) class MockBookieWithNoopShutdown extends Bookie { public MockBookieWithNoopShutdown(ServerConfiguration conf, StatsLogger statsLogger) throws IOException, KeeperException, InterruptedException, BookieException { - super(conf, statsLogger, UnpooledByteBufAllocator.DEFAULT); + super(conf, statsLogger, UnpooledByteBufAllocator.DEFAULT, BookieServiceInfo.NO_INFO); } // making Bookie Shutdown no-op. Ideally for this testcase we need to diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java index 71658e7ab7d..ec5b9ef75c6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/StateManagerTest.java @@ -28,6 +28,7 @@ import java.io.File; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -156,7 +157,7 @@ public void testReadOnlyBookieTransitions() throws Exception{ .setMetadataServiceUri(zkUtil.getMetadataServiceUri()) .setForceReadOnlyBookie(true); ReadOnlyBookie readOnlyBookie = new ReadOnlyBookie(readOnlyConf, NullStatsLogger.INSTANCE, - UnpooledByteBufAllocator.DEFAULT); + UnpooledByteBufAllocator.DEFAULT, BookieServiceInfo.NO_INFO); readOnlyBookie.start(); assertTrue(readOnlyBookie.isRunning()); assertTrue(readOnlyBookie.isReadOnly()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index daafc747e04..ecbc4f0bc60 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -23,8 +23,11 @@ import static com.google.common.base.Charsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -35,6 +38,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -46,15 +50,23 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.common.component.ComponentStarter; +import org.apache.bookkeeper.common.component.Lifecycle; +import org.apache.bookkeeper.common.component.LifecycleComponent; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.server.Main; +import org.apache.bookkeeper.server.conf.BookieConfiguration; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.commons.io.FileUtils; @@ -650,4 +662,71 @@ public void testBookkeeperAdminFormatResetsLedgerIds() throws Exception { } } } + + private void testBookieServiceInfo(boolean readonly, boolean legacy) throws Exception { + File tmpDir = createTempDir("bookie", "test"); + final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setJournalDirName(tmpDir.getPath()) + .setLedgerDirNames(new String[]{tmpDir.getPath()}) + .setBookiePort(PortManager.nextFreePort()) + .setMetadataServiceUri(metadataServiceUri); + + LifecycleComponent server = Main.buildBookieServer(new BookieConfiguration(conf)); + // 2. start the server + CompletableFuture stackComponentFuture = ComponentStarter.startComponent(server); + while (server.lifecycleState() != Lifecycle.State.STARTED) { + Thread.sleep(100); + } + + ServerConfiguration bkConf = newServerConfiguration().setForceReadOnlyBookie(readonly); + BookieServer bkServer = startBookie(bkConf); + + String bookieId = bkServer.getLocalAddress().toString(); + String host = bkServer.getLocalAddress().getHostName(); + int port = bkServer.getLocalAddress().getPort(); + + if (legacy) { + String regPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(bkConf) + "/" + AVAILABLE_NODE; + regPath = readonly + ? regPath + READONLY + "/" + bookieId + : regPath + "/" + bookieId; + // deleting the metadata, so that the bookie registration should + // continue successfully with legacy BookieServiceInfo + zkc.setData(regPath, new byte[]{}, -1); + } + + try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) { + BookieServiceInfo bookieServiceInfo = bkAdmin.getBookieServiceInfo(bookieId); + + assertThat(bookieServiceInfo.getEndpoints().size(), is(1)); + BookieServiceInfo.Endpoint endpoint = bookieServiceInfo.getEndpoints().stream() + .filter(e -> Objects.equals(e.getId(), bookieId)) + .findFirst() + .get(); + assertNotNull("Endpoint " + bookieId + " not found.", endpoint); + + assertThat(endpoint.getHost(), is(host)); + assertThat(endpoint.getPort(), is(port)); + assertThat(endpoint.getProtocol(), is("bookie-rpc")); + } + + bkServer.shutdown(); + stackComponentFuture.cancel(true); + } + + @Test + public void testBookieServiceInfoWritable() throws Exception { + testBookieServiceInfo(false, false); + } + + @Test + public void testBookieServiceInfoReadonly() throws Exception { + testBookieServiceInfo(true, false); + } + + @Test + public void testLegacyBookieServiceInfo() throws Exception { + testBookieServiceInfo(false, true); + } + } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/BookieServiceInfoTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/BookieServiceInfoTest.java new file mode 100644 index 00000000000..120a2055f51 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/BookieServiceInfoTest.java @@ -0,0 +1,91 @@ +/* + * Copyright 2020 The Apache Software Foundation. + * + * Licensed 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.discover.ZKRegistrationClient.deserializeBookieServiceInfo; +import static org.apache.bookkeeper.discover.ZKRegistrationManager.serializeBookieServiceInfo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import org.apache.bookkeeper.discover.BookieServiceInfo.Endpoint; +import org.junit.Test; + +/** + * Unit test of the {@link BookieServiceInfo} serialization/deserialization methods. + */ +public class BookieServiceInfoTest { + + @Test + public void testSerializeDeserializeBookieServiceInfo() throws Exception { + String bookieId = "127.0.0.1:3181"; + { + BookieServiceInfo expected = new BookieServiceInfo(); + Endpoint endpointRPC = new Endpoint("1", 1281, "localhost", "bookie-rpc", + Collections.emptyList(), Collections.emptyList()); + Endpoint endpointHTTP = new Endpoint("2", 1281, "localhost", "bookie-http", + Collections.emptyList(), Collections.emptyList()); + expected.setEndpoints(Arrays.asList(endpointRPC, endpointHTTP)); + + Map properties = new HashMap<>(); + properties.put("test", "value"); + expected.setProperties(properties); + + byte[] serialized = serializeBookieServiceInfo(expected); + BookieServiceInfo deserialized = deserializeBookieServiceInfo(bookieId, serialized); + + assertBookieServiceInfoEquals(expected, deserialized); + } + } + + @Test + public void testDeserializeBookieServiceInfo() throws Exception { + String bookieId = "127.0.0.1:3181"; + { + BookieServiceInfo expected = BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId); + BookieServiceInfo deserialized = deserializeBookieServiceInfo(bookieId, null); + + assertBookieServiceInfoEquals(expected, deserialized); + } + { + BookieServiceInfo expected = BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId); + BookieServiceInfo deserialized = deserializeBookieServiceInfo(bookieId, new byte[]{}); + + assertBookieServiceInfoEquals(expected, deserialized); + } + } + + private void assertBookieServiceInfoEquals(BookieServiceInfo expected, BookieServiceInfo provided) { + for (Endpoint ep : expected.getEndpoints()) { + Endpoint e = provided.getEndpoints().stream() + .filter(ee -> Objects.equals(ee.getId(), ep.getId())) + .findFirst() + .get(); + assertThat(e.getHost(), is(ep.getHost())); + assertThat(e.getPort(), is(ep.getPort())); + assertThat(e.getProtocol(), is(ep.getProtocol())); + assertArrayEquals(e.getAuth().toArray(), ep.getAuth().toArray()); + assertArrayEquals(e.getExtensions().toArray(), ep.getExtensions().toArray()); + } + assertEquals(expected.getProperties(), provided.getProperties()); + } + +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java index 468a0324c98..75a7b1696b3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -42,6 +42,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; @@ -108,7 +109,7 @@ public void testPlacementPolicyCheckWithBookiesFromDifferentRacks() throws Excep bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), "/rack" + (i)); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false); + regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); } LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); @@ -217,7 +218,7 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPlacementPolicy() th for (int i = 0; i < numOfBookies; i++) { BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false); + regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); } // only three racks @@ -312,7 +313,7 @@ public void testPlacementPolicyCheckWithURLedgers(boolean timeElapsed) throws Ex for (int i = 0; i < numOfBookies; i++) { BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false); + regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); } LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); @@ -428,7 +429,7 @@ public void testPlacementPolicyCheckWithLedgersNotAdheringToPolicyWithMultipleSe for (int i = 0; i < numOfBookies; i++) { BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false); + regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); } // only three racks @@ -534,7 +535,7 @@ public void testZoneawarePlacementPolicyCheck() throws Exception { for (int i = 0; i < numOfBookies; i++) { BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false); + regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); String zone = "/zone" + (i % 3); String upgradeDomain = "/ud" + (i % 2); String networkLocation = zone + upgradeDomain; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java index a2d5f6b2ea0..85b2015cade 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java @@ -45,6 +45,7 @@ import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; @@ -185,7 +186,7 @@ List addAndRegisterBookies(RegistrationManager regManager, for (int i = 0; i < numOfBookies; i++) { bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false); + regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); } return bookieAddresses; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java index d5253fdc422..24bce2e1c55 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java @@ -26,12 +26,15 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.whenNew; import java.io.IOException; +import java.util.function.Supplier; import org.apache.bookkeeper.common.component.LifecycleComponentStack; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.server.component.ServerLifecycleComponent; import org.apache.bookkeeper.server.conf.BookieConfiguration; @@ -80,9 +83,12 @@ public void testBuildBookieServer() throws Exception { BookieServer mockServer = PowerMockito.mock(BookieServer.class); whenNew(BookieServer.class) - .withArguments(any(ServerConfiguration.class), any(StatsLogger.class)) + .withArguments(any(ServerConfiguration.class), any(StatsLogger.class), any(Supplier.class)) .thenReturn(mockServer); + BookieSocketAddress bookieAddress = new BookieSocketAddress("127.0.0.1", 1281); + when(mockServer.getLocalAddress()).thenReturn(bookieAddress); + LifecycleComponentStack stack = buildBookieServer(conf); assertEquals(3, stack.getNumComponents()); assertTrue(stack.getComponent(2) instanceof TestComponent); @@ -107,9 +113,12 @@ public void testIgnoreExtraServerComponentsStartupFailures() throws Exception { BookieServer mockServer = PowerMockito.mock(BookieServer.class); whenNew(BookieServer.class) - .withArguments(any(ServerConfiguration.class), any(StatsLogger.class)) + .withArguments(any(ServerConfiguration.class), any(StatsLogger.class), any(Supplier.class)) .thenReturn(mockServer); + BookieSocketAddress bookieAddress = new BookieSocketAddress("127.0.0.1", 1281); + when(mockServer.getLocalAddress()).thenReturn(bookieAddress); + LifecycleComponentStack stack = buildBookieServer(conf); assertEquals(2, stack.getNumComponents()); @@ -133,9 +142,12 @@ public void testExtraServerComponentsStartupFailures() throws Exception { BookieServer mockServer = PowerMockito.mock(BookieServer.class); whenNew(BookieServer.class) - .withArguments(any(ServerConfiguration.class), any(StatsLogger.class)) + .withArguments(any(ServerConfiguration.class), any(StatsLogger.class), any(Supplier.class)) .thenReturn(mockServer); + BookieSocketAddress bookieAddress = new BookieSocketAddress("127.0.0.1", 1281); + when(mockServer.getLocalAddress()).thenReturn(bookieAddress); + try { buildBookieServer(conf); fail("Should fail to start bookie server if `ignoreExtraServerComponentsStartupFailures` is set to false"); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index b8e2426bf96..3d9e88ffb0a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -41,6 +41,7 @@ import java.util.concurrent.Future; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; @@ -50,6 +51,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.metastore.InMemoryMetaStore; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -648,7 +650,8 @@ public BookieSocketAddress startNewBookieAndReturnAddress() protected BookieServer startBookie(ServerConfiguration conf) throws Exception { TestStatsProvider provider = new TestStatsProvider(); - BookieServer server = new BookieServer(conf, provider.getStatsLogger("")); + BookieServer server = new BookieServer(conf, provider.getStatsLogger(""), + BookieServiceInfo.NO_INFO); BookieSocketAddress address = Bookie.getBookieAddress(conf); bsLoggers.put(address, provider); @@ -682,9 +685,11 @@ protected BookieServer startBookie(ServerConfiguration conf) protected BookieServer startBookie(ServerConfiguration conf, final Bookie b) throws Exception { TestStatsProvider provider = new TestStatsProvider(); - BookieServer server = new BookieServer(conf, provider.getStatsLogger("")) { + BookieServer server = new BookieServer(conf, provider.getStatsLogger(""), + BookieServiceInfo.NO_INFO) { @Override - protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator) { + protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator, + Supplier s) { return b; } }; diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java index 87c831c17e4..10015ad3da4 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java @@ -68,6 +68,7 @@ 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.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.LedgerLayout; import org.apache.bookkeeper.versioning.LongVersion; @@ -145,7 +146,8 @@ public void close() { } @Override - public void registerBookie(String bookieId, boolean readOnly) throws BookieException { + public void registerBookie(String bookieId, boolean readOnly, + BookieServiceInfo bookieServiceInfo) throws BookieException { if (readOnly) { doRegisterReadonlyBookie(bookieId, bkRegister.get()); } else { diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java index f36a496c215..0581a2c71ee 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java @@ -42,6 +42,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; @@ -218,7 +219,7 @@ private static List createNumBookies(boolean readonly, Client client = newEtcdClient(); EtcdRegistrationManager regMgr = new EtcdRegistrationManager(client, scope, ttlSeconds); bookies.add(regMgr); - regMgr.registerBookie(newBookie(i), readonly); + regMgr.registerBookie(newBookie(i), readonly, BookieServiceInfo.EMPTY); } return bookies; } @@ -237,7 +238,7 @@ public void testRegisterBookieWaitUntilPreviousExpiredSuccess() throws Exception try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( newEtcdClient(), scope, ttlSeconds) ) { - regManager.registerBookie(bookieId, false); + regManager.registerBookie(bookieId, false, BookieServiceInfo.EMPTY); leaseId = regManager.getBkRegister().getLeaseId(); log.info("Registered bookie under scope '{}' with lease = {}", scope, leaseId); } @@ -246,7 +247,7 @@ public void testRegisterBookieWaitUntilPreviousExpiredSuccess() throws Exception try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( newEtcdClient(), scope, 100000 * ttlSeconds) ) { - regManager.registerBookie(bookieId, false); + regManager.registerBookie(bookieId, false, BookieServiceInfo.EMPTY); leaseId = regManager.getBkRegister().getLeaseId(); log.info("Registered bookie under scope '{}' with new lease = {}", scope, leaseId); } @@ -261,7 +262,7 @@ public void testRegisterBookieWaitUntilPreviousExpiredFailure() throws Exception try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( newEtcdClient(), scope, 10000000 * ttlSeconds) ) { - regManager.registerBookie(bookieId, false); + regManager.registerBookie(bookieId, false, BookieServiceInfo.EMPTY); leaseId = regManager.getBkRegister().getLeaseId(); log.info("Registered bookie under scope '{}' with lease = {}", scope, leaseId); } @@ -269,7 +270,7 @@ public void testRegisterBookieWaitUntilPreviousExpiredFailure() throws Exception try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( newEtcdClient(), scope, ttlSeconds) ) { - regManager.registerBookie(bookieId, false); + regManager.registerBookie(bookieId, false, BookieServiceInfo.EMPTY); fail("Should fail to register bookie under scope '{}'" + " since previous registration has not been expired yet"); } catch (MetadataStoreException mse) { @@ -295,14 +296,14 @@ private void testRegisterBookieWithSameLeaseId(boolean readonly) throws Exceptio try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( newEtcdClient(), scope, 10000000 * ttlSeconds) ) { - regManager.registerBookie(bookieId, readonly); + regManager.registerBookie(bookieId, readonly, BookieServiceInfo.EMPTY); leaseId = regManager.getBkRegister().getLeaseId(); log.info("Registered bookie under scope '{}' with lease = {}", scope, leaseId); log.info("Trying to register using same lease '{}'", leaseId); try (EtcdRegistrationManager regManager2 = new EtcdRegistrationManager( regManager.getClient(), scope, regManager.getBkRegister() )) { - regManager.registerBookie(bookieId, readonly); + regManager.registerBookie(bookieId, readonly, BookieServiceInfo.EMPTY); } } } @@ -341,7 +342,7 @@ private void testRegisterUnregister(boolean readonly) throws Exception { log.info("before registration : bookies = {}", bookies); assertEquals(0, bookies.size()); // registered - regMgr.registerBookie(bookieId, readonly); + regMgr.registerBookie(bookieId, readonly, BookieServiceInfo.EMPTY); bookies = getBookies(readonly); log.info("after registered: bookies = {}", bookies); assertEquals(1, bookies.size()); @@ -388,7 +389,7 @@ private void testConcurrentRegistration(boolean readonly) throws Exception { )) { try { startBarrier.await(); - regMgr.registerBookie(bookieId, readonly); + regMgr.registerBookie(bookieId, readonly, BookieServiceInfo.EMPTY); numSuccesses.incrementAndGet(); } catch (InterruptedException e) { log.warn("Interrupted at waiting for the other threads to start", e); diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java index ccc9a1e8174..7ccb0f09220 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java @@ -27,6 +27,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.metadata.etcd.EtcdMetadataBookieDriver; import org.apache.bookkeeper.metadata.etcd.EtcdMetadataClientDriver; @@ -120,7 +121,8 @@ private static void startNumBookies(int numBookies) throws Exception { private static BookieServer startBookie(ServerConfiguration conf) throws Exception { conf.setAutoRecoveryDaemonEnabled(true); TestStatsProvider provider = new TestStatsProvider(); - BookieServer server = new BookieServer(conf, provider.getStatsLogger("")); + BookieServer server = new BookieServer(conf, provider.getStatsLogger(""), + BookieServiceInfo.NO_INFO); server.start(); return server; } diff --git a/site/_data/cli/shell.yaml b/site/_data/cli/shell.yaml index 092c8b1261e..071fb3400dd 100644 --- a/site/_data/cli/shell.yaml +++ b/site/_data/cli/shell.yaml @@ -40,6 +40,8 @@ commands: description: Ledger ID - flag: -force description: Whether to force delete the Ledger without prompt..? +- name: endpointinfo + description: Get endpoints of a Bookie. - name: expandstorage description: Add new empty ledger/index directories. Update the directories info in the conf file before running the command. - name: help diff --git a/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java index 726c8b00b97..f877aa7ba94 100644 --- a/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java +++ b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java @@ -37,6 +37,7 @@ import java.util.stream.Collectors; import lombok.Cleanup; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -102,7 +103,7 @@ public void testNameResolver() throws Exception { InetSocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i); addressSet.add(address); bookieDriver.getRegistrationManager().registerBookie( - "127.0.0.1:" + (3181 + i), false + "127.0.0.1:" + (3181 + i), false, BookieServiceInfo.EMPTY ); } @@ -137,7 +138,7 @@ public void onError(Status error) { InetSocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i); addressSet.add(address); bookieDriver.getRegistrationManager().registerBookie( - "127.0.0.1:" + (3181 + i), false + "127.0.0.1:" + (3181 + i), false, BookieServiceInfo.EMPTY ); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java index d5f4115abee..5e0a51463df 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.stats.NullStatsProvider; import org.junit.Assert; @@ -107,7 +108,8 @@ private BookieServer startExtraBookie() throws Exception { conf.setJournalDirName(journalDir.getPath()); conf.setLedgerDirNames(new String[] { ledgerDir.getPath() }); - BookieServer server = new BookieServer(conf, new NullStatsProvider().getStatsLogger("")); + BookieServer server = new BookieServer(conf, new NullStatsProvider().getStatsLogger(""), + BookieServiceInfo.NO_INFO); server.start(); while (!server.isRunning()) { diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java index 4385d56e1f1..fe520e6cd7a 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java @@ -22,16 +22,21 @@ import java.net.InetAddress; import java.net.MalformedURLException; import java.net.UnknownHostException; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.function.Supplier; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.clients.impl.channel.StorageServerChannel; import org.apache.bookkeeper.clients.impl.internal.StorageServerClientManagerImpl; +import org.apache.bookkeeper.common.component.ComponentInfoPublisher; import org.apache.bookkeeper.common.component.ComponentStarter; import org.apache.bookkeeper.common.component.LifecycleComponent; import org.apache.bookkeeper.common.component.LifecycleComponentStack; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.statelib.impl.rocksdb.checkpoint.dlog.DLCheckpointStore; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -182,9 +187,14 @@ public static LifecycleComponent buildStorageServer(CompositeConfiguration conf, boolean startBookieAndStartProvider, StatsLogger externalStatsLogger) throws ConfigurationException, UnknownHostException { + final ComponentInfoPublisher componentInfoPublisher = new ComponentInfoPublisher(); + + final Supplier bookieServiceInfoProvider = + () -> buildBookieServiceInfo(componentInfoPublisher); LifecycleComponentStack.Builder serverBuilder = LifecycleComponentStack.newBuilder() - .withName("storage-server"); + .withName("storage-server") + .withComponentInfoPublisher(componentInfoPublisher); BookieConfiguration bkConf = BookieConfiguration.of(conf); bkConf.validate(); @@ -224,7 +234,7 @@ public static LifecycleComponent buildStorageServer(CompositeConfiguration conf, // Create the bookie service ServerConfiguration bkServerConf; if (startBookieAndStartProvider) { - BookieService bookieService = new BookieService(bkConf, rootStatsLogger); + BookieService bookieService = new BookieService(bkConf, rootStatsLogger, bookieServiceInfoProvider); serverBuilder.addComponent(bookieService); bkServerConf = bookieService.serverConf(); } else { @@ -358,4 +368,25 @@ public static LifecycleComponent buildStorageServer(CompositeConfiguration conf, .build(); } + /** + * Create the {@link BookieServiceInfo} starting from the published endpoints. + * + * @see ComponentInfoPublisher + * @param componentInfoPublisher the endpoint publisher + * @return the created bookie service info + */ + private static BookieServiceInfo buildBookieServiceInfo(ComponentInfoPublisher componentInfoPublisher) { + List endpoints = componentInfoPublisher.getEndpoints().values() + .stream().map(e -> { + return new BookieServiceInfo.Endpoint( + e.getId(), + e.getPort(), + e.getHost(), + e.getProtocol(), + e.getAuth(), + e.getExtensions() + ); + }).collect(Collectors.toList()); + return new BookieServiceInfo(componentInfoPublisher.getProperties(), endpoints); + } } diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java index 6f83752ce60..2a414211769 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java @@ -18,11 +18,13 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.function.Supplier; import lombok.Getter; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stream.server.conf.BookieConfiguration; @@ -38,11 +40,15 @@ public class BookieService extends AbstractLifecycleComponent bookieServiceInfoProvider; - public BookieService(BookieConfiguration conf, StatsLogger statsLogger) { + + public BookieService(BookieConfiguration conf, StatsLogger statsLogger, + Supplier bookieServiceInfoProvider) { super("bookie-server", conf, statsLogger); this.serverConf = new ServerConfiguration(); this.serverConf.loadConf(conf.getUnderlyingConf()); + this.bookieServiceInfoProvider = bookieServiceInfoProvider; } @Override @@ -61,7 +67,7 @@ protected void doStart() { Arrays.asList(serverConf.getLedgerDirs()), indexDirs); try { - this.bs = new BookieServer(serverConf, statsLogger); + this.bs = new BookieServer(serverConf, statsLogger, bookieServiceInfoProvider); bs.start(); log.info("Started bookie server successfully."); } catch (Exception e) { diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java index 40261c0cfc3..2485c1791f9 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.clients.utils.NetUtils; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.discover.ZKRegistrationManager; import org.apache.bookkeeper.stats.StatsLogger; @@ -79,7 +80,8 @@ protected void doStart() { statsLogger.scope("state"), () -> regManager, Collections.emptyList(), - () -> NetUtils.endpointToString(myEndpoint)); + () -> NetUtils.endpointToString(myEndpoint), + BookieServiceInfo.NO_INFO); stateManager.initState(); stateManager.registerBookie(true).get(); log.info("Successfully register myself under registration path {}/{}", diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java index 02ccf47a7a0..54e9f7c6cc8 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java @@ -38,6 +38,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.RebuildDBLedgerLocationsIndexCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.RegenerateInterleavedStorageIndexFileCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; +import org.apache.bookkeeper.tools.cli.commands.bookies.EndpointInfoCommand; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.tools.framework.CliCommandGroup; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -58,6 +59,7 @@ public class BookieCommandGroup extends CliCommandGroup { .addCommand(new LastMarkCommand()) .addCommand(new InitCommand()) .addCommand(new FormatCommand()) + .addCommand(new EndpointInfoCommand()) .addCommand(new SanityTestCommand()) .addCommand(new LedgerCommand()) .addCommand(new ListFilesOnDiscCommand()) From ebdc98e579d083f45161d090453b756727cb6635 Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Wed, 27 May 2020 14:15:52 +0800 Subject: [PATCH 0430/1642] [CLEAN] remove duplicate property key in log4j.properties ### Motivation 1. Fix duplicate key in log4j.properties 2. `http` to `https` in `README` ### Changes ![image](https://user-images.githubusercontent.com/20113411/82162451-369ef880-98d7-11ea-9217-777721bba545.png) ![image](https://user-images.githubusercontent.com/20113411/82162816-d2c9ff00-98d9-11ea-967e-53646d351753.png) Reviewers: Enrico Olivelli This closes #2334 from lamber-ken/clean-log --- README.md | 12 ++++++------ bookkeeper-benchmark/conf/log4j.properties | 4 ---- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index b8d55c9c8ad..82f12aba430 100644 --- a/README.md +++ b/README.md @@ -18,9 +18,9 @@ It is suitable for being used in following scenarios: ## Get Started -* *Concepts*: Start with the [basic concepts](http://bookkeeper.apache.org/docs/master/bookkeeperOverview.html) of Apache BookKeeper. +* *Concepts*: Start with the [basic concepts](https://bookkeeper.apache.org/docs/master/bookkeeperOverview.html) of Apache BookKeeper. This will help you to fully understand the other parts of the documentation. -* [Getting Started](http://bookkeeper.apache.org/docs/master/bookkeeperStarted.html) to setup BookKeeper to write logs. +* [Getting Started](https://bookkeeper.apache.org/docs/master/bookkeeperStarted.html) to setup BookKeeper to write logs. ## Documentation @@ -30,17 +30,17 @@ It is suitable for being used in following scenarios: * [Tutorial](https://bookkeeper.apache.org/archives/docs/master/bookkeeperTutorial.html) * [Java API](https://bookkeeper.apache.org/archives/docs/master/apidocs/) -You can also read [Turning Ledgers into Logs](http://bookkeeper.apache.org/docs/master/bookkeeperLedgers2Logs.html) to learn how to turn **ledgers** into continuous **log streams**. +You can also read [Turning Ledgers into Logs](https://bookkeeper.apache.org/docs/master/bookkeeperLedgers2Logs.html) to learn how to turn **ledgers** into continuous **log streams**. If you are looking for a high level **log stream** API, you can checkout [DistributedLog](http://distributedlog.io). ### Administrators -* [Admin Guide](http://bookkeeper.apache.org/docs/master/bookkeeperConfig.html) -* [Configuration Parameters](http://bookkeeper.apache.org/docs/master/bookieConfigParams.html) +* [Admin Guide](https://bookkeeper.apache.org/docs/master/bookkeeperConfig.html) +* [Configuration Parameters](https://bookkeeper.apache.org/docs/master/bookieConfigParams.html) ### Contributors -* [BookKeeper Internals](http://bookkeeper.apache.org/docs/master/bookkeeperInternals.html) +* [BookKeeper Internals](https://bookkeeper.apache.org/docs/master/bookkeeperInternals.html) ## Get In Touch diff --git a/bookkeeper-benchmark/conf/log4j.properties b/bookkeeper-benchmark/conf/log4j.properties index 79e0db26a66..4a0f5e1076c 100644 --- a/bookkeeper-benchmark/conf/log4j.properties +++ b/bookkeeper-benchmark/conf/log4j.properties @@ -57,10 +57,6 @@ 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 From 80e7f561d48074b8eb8aa770fec6ed5e23258dea Mon Sep 17 00:00:00 2001 From: Yang Yang Date: Wed, 27 May 2020 14:17:16 +0800 Subject: [PATCH 0431/1642] ISSUE #2311: Adds support to decode metadata in the API that list ledgers Descriptions of the changes in this PR: ### Motivation The current list ledgers API output the metadata in a serialized binary format, which is not friendly to human operators and external tools, and is not consistent with the output of the [API that gets the metadata](https://bookkeeper.apache.org/docs/4.9.2/admin/http/#endpoint-apiv1ledgermetadataledger_idledger_id). ### Changes The PR adds a parameter called `decode_meta`, and output the ledger metadata in decoded format when the parameter presents and the value of it is 'true'. Master Issue: https://github.com/apache/bookkeeper/issues/2311 Reviewers: Sijie Guo , Enrico Olivelli This closes #2312 from fantapsody/decode_metadata, closes #2311 --- .../http/service/ListLedgerService.java | 18 +- .../http/service/ListLedgerServiceTest.java | 168 ++++++++++++++++++ 2 files changed, 181 insertions(+), 5 deletions(-) create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java index 1683fd4a887..07ea5af4791 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java @@ -66,10 +66,14 @@ public ListLedgerService(ServerConfiguration conf, BookieServer bookieServer) { static final int LIST_LEDGER_BATCH_SIZE = 100; private void keepLedgerMetadata(long ledgerId, CompletableFuture> future, - LinkedHashMap output) + LinkedHashMap output, boolean decodeMeta) throws Exception { LedgerMetadata md = future.get().getValue(); - output.put(Long.valueOf(ledgerId).toString(), new String(serDe.serialize(md), UTF_8)); + if (decodeMeta) { + output.put(Long.valueOf(ledgerId).toString(), md); + } else { + output.put(Long.valueOf(ledgerId).toString(), new String(serDe.serialize(md), UTF_8)); + } } @Override @@ -84,6 +88,10 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { && params.containsKey("print_metadata") && params.get("print_metadata").equals("true"); + // do not decode meta by default for backward compatibility + boolean decodeMeta = (params != null) + && params.getOrDefault("decode_meta", "false").equals("true"); + // Page index should start from 1; int pageIndex = (printMeta && params.containsKey("page")) ? Integer.parseInt(params.get("page")) : -1; @@ -93,7 +101,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { LedgerManager.LedgerRangeIterator iter = manager.getLedgerRanges(0); // output - LinkedHashMap output = Maps.newLinkedHashMap(); + LinkedHashMap output = Maps.newLinkedHashMap(); // futures for readLedgerMetadata for each page. Map>> futures = new LinkedHashMap<>(LIST_LEDGER_BATCH_SIZE); @@ -121,13 +129,13 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { } if (futures.size() >= LIST_LEDGER_BATCH_SIZE) { for (Map.Entry> > e : futures.entrySet()) { - keepLedgerMetadata(e.getKey(), e.getValue(), output); + keepLedgerMetadata(e.getKey(), e.getValue(), output, decodeMeta); } futures.clear(); } } for (Map.Entry> > e : futures.entrySet()) { - keepLedgerMetadata(e.getKey(), e.getValue(), output); + keepLedgerMetadata(e.getKey(), e.getValue(), output, decodeMeta); } futures.clear(); } else { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java new file mode 100644 index 00000000000..88fb3ff2a5b --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java @@ -0,0 +1,168 @@ +/** + * + * 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.server.http.service; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import java.util.Base64; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.http.HttpServer; +import org.apache.bookkeeper.http.service.HttpServiceRequest; +import org.apache.bookkeeper.http.service.HttpServiceResponse; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.commons.lang3.RandomUtils; +import org.junit.Before; +import org.junit.Test; + +/** + * Unit tests for {@link ListLedgerService}. + */ +public class ListLedgerServiceTest extends BookKeeperClusterTestCase { + private final ObjectMapper mapper = new ObjectMapper(); + private ListLedgerService listLedgerService; + + public ListLedgerServiceTest() { + super(1); + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + listLedgerService = new ListLedgerService(bsConfs.get(0), bs.get(0)); + } + + @Test + public void testEmptyList() throws Exception { + HttpServiceResponse response = listLedgerService.handle(new HttpServiceRequest()); + assertEquals(response.getStatusCode(), HttpServer.StatusCode.OK.getValue()); + JsonNode json = mapper.readTree(response.getBody()); + assertEquals(0, json.size()); + } + + @Test + public void testListLedgers() throws Exception { + int ledgerNum = RandomUtils.nextInt(1, 10); + Map ledgers = new HashMap<>(); + for (int i = 0; i < ledgerNum; i++) { + LedgerHandle ledger = bkc.createLedger(1, 1, 1, BookKeeper.DigestType.CRC32, new byte[0]); + ledgers.put(ledger.getId(), ledger.getLedgerMetadata()); + ledger.close(); + } + + HttpServiceResponse response = listLedgerService.handle(new HttpServiceRequest()); + assertEquals(response.getStatusCode(), HttpServer.StatusCode.OK.getValue()); + JsonNode json = mapper.readTree(response.getBody()); + assertEquals(ledgerNum, json.size()); + + json.fieldNames().forEachRemaining(field -> { + assertTrue(ledgers.containsKey(Long.parseLong(field))); + assertTrue(json.get(field).isNull()); + }); + } + + @Test + public void testListLedgersWithMetadata() throws Exception { + int ledgerNum = RandomUtils.nextInt(1, 10); + Map ledgers = new HashMap<>(); + for (int i = 0; i < ledgerNum; i++) { + LedgerHandle ledger = bkc.createLedger(1, 1, 1, BookKeeper.DigestType.CRC32, new byte[0]); + ledger.close(); + ledgers.put(ledger.getId(), ledger.getLedgerMetadata()); + } + + HttpServiceResponse response = listLedgerService.handle(new HttpServiceRequest(null, HttpServer.Method.GET, + ImmutableMap.of("print_metadata", "true"))); + assertEquals(response.getStatusCode(), HttpServer.StatusCode.OK.getValue()); + JsonNode json = mapper.readTree(response.getBody()); + assertEquals(ledgerNum, json.size()); + + json.fieldNames().forEachRemaining(field -> { + LedgerMetadata meta = ledgers.get(Long.parseLong(field)); + assertNotNull(meta); + assertFalse(json.get(field).isNull()); + }); + } + + @Test + public void testListLedgersWithMetadataDecoded() throws Exception { + int ledgerNum = RandomUtils.nextInt(1, 10); + Map ledgers = new HashMap<>(); + for (int i = 0; i < ledgerNum; i++) { + LedgerHandle ledger = bkc.createLedger(1, 1, 1, BookKeeper.DigestType.CRC32, new byte[0], + ImmutableMap.of("test_key", "test_value".getBytes())); + ledger.close(); + ledgers.put(ledger.getId(), ledger.getLedgerMetadata()); + } + + HttpServiceResponse response = listLedgerService.handle(new HttpServiceRequest(null, HttpServer.Method.GET, + ImmutableMap.of("print_metadata", "true", "decode_meta", "true"))); + assertEquals(response.getStatusCode(), HttpServer.StatusCode.OK.getValue()); + JsonNode json = mapper.readTree(response.getBody()); + assertEquals(ledgerNum, json.size()); + + json.fieldNames().forEachRemaining(field -> { + LedgerMetadata meta = ledgers.get(Long.parseLong(field)); + assertNotNull(meta); + JsonNode node = json.get(field); + assertEquals(meta.getMetadataFormatVersion(), node.get("metadataFormatVersion").asInt()); + assertEquals(meta.getEnsembleSize(), node.get("ensembleSize").asInt()); + assertEquals(meta.getWriteQuorumSize(), node.get("writeQuorumSize").asInt()); + assertEquals(meta.getAckQuorumSize(), node.get("ackQuorumSize").asInt()); + assertEquals(meta.getCToken(), node.get("ctoken").asLong()); +// assertEquals(meta.getCtime(), node.get("ctime").asLong()); + assertEquals(meta.getState().name(), node.get("state").asText()); + assertEquals(meta.isClosed(), node.get("closed").asBoolean()); + assertEquals(meta.getLength(), node.get("length").asLong()); + assertEquals(meta.getLastEntryId(), node.get("lastEntryId").asLong()); + assertEquals(meta.getDigestType().name(), node.get("digestType").asText()); + assertEquals(new String(meta.getPassword()), node.get("password").asText()); + + for (Map.Entry entry : meta.getCustomMetadata().entrySet()) { + JsonNode data = node.get("customMetadata").get(entry.getKey()); + assertArrayEquals(entry.getValue(), Base64.getDecoder().decode(data.asText())); + } + + for (Map.Entry> entry : meta.getAllEnsembles().entrySet()) { + JsonNode members = node.get("allEnsembles") + .get(String.valueOf(entry.getKey())); + assertEquals(1, entry.getValue().size()); + assertEquals(entry.getValue().size(), members.size()); + JsonNode member = members.get(0); + assertEquals(entry.getValue().get(0).getHostName(), member.get("hostName").asText()); + assertEquals(entry.getValue().get(0).getPort(), member.get("port").asInt()); + } + }); + } +} From 9a42d3d849e8bbb6e8b7e576772eab78f09ae683 Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Sat, 30 May 2020 08:39:58 +0800 Subject: [PATCH 0432/1642] [LOG] Change ERROR to INFO log level ### Motivation When use shell command, it output nothing caused by `ERROR` level. **OLD** (output nothing) ``` bin/bookkeeper shell readlogmetadata 0 ``` **FIX** ``` bin/bookkeeper shell readlogmetadata 0 06:58:01,364 INFO Print entryLogMetadata of entrylog 0 (0.log) 06:58:01,467 INFO Failed to get ledgers map index from: 0.log : No ledgers map index found on entryLogId0 06:58:01,471 INFO --------- Lid=0, TotalSizeOfEntriesOfLedger=59 --------- ``` ### Changes - Remove duplicate definition `VERBOSECONSOLE ` - Change `ERROR` to `INFO` Reviewers: Sijie Guo This closes #2350 from lamber-ken/bk-improve-shell-log-level --- conf/log4j.shell.properties | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/conf/log4j.shell.properties b/conf/log4j.shell.properties index e27f7f5edbe..32b963e35bb 100644 --- a/conf/log4j.shell.properties +++ b/conf/log4j.shell.properties @@ -25,7 +25,7 @@ # DEFAULT: console appender only # Define some default values that can be overridden by system properties -bookkeeper.root.logger=ERROR,CONSOLE +bookkeeper.root.logger=INFO,CONSOLE log4j.rootLogger=${bookkeeper.root.logger} @@ -37,15 +37,8 @@ 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.zookeeper=INFO +log4j.logger.org.apache.bookkeeper=INFO log4j.logger.org.apache.bookkeeper.bookie.BookieShell=INFO log4j.logger.org.apache.bookkeeper.client.BookKeeperAdmin=INFO log4j.logger.org.apache.bookkeeper.bookie.InterleavedStorageRegenerateIndexOp=INFO From 08671984a68340153bc4d077ab113774949f6bbd Mon Sep 17 00:00:00 2001 From: Matteo Minardi Date: Sat, 30 May 2020 19:56:31 +0200 Subject: [PATCH 0433/1642] [CLEAN] Replace com.google.common.base.Optional with java.util.Optional ### Motivation `com.google.common.base.Optional` and `java.util.Optional` exist in the code base at the same time. ### Changes Replace `com.google.common.base.Optional` with `java.util.Optional`. Reviewers: Enrico Olivelli , Sijie Guo , lamber-ken This closes #2346 from mino181295/merge-optional --- .../bookkeeper/client/ClientInternalConf.java | 7 +++---- .../proto/LongPollReadEntryProcessorV3.java | 4 ++-- .../proto/ReadLastConfirmedAndEntryContext.java | 4 ++-- site/docs/latest/api/distributedlog-api.md | 2 +- .../apache/distributedlog/BKAsyncLogReader.java | 2 +- .../distributedlog/BKDistributedLogManager.java | 14 +++++++------- .../BKDistributedLogNamespace.java | 9 ++++----- .../apache/distributedlog/BKLogReadHandler.java | 2 +- .../apache/distributedlog/BKSyncLogReader.java | 4 ++-- .../apache/distributedlog/BookKeeperClient.java | 4 ++-- .../distributedlog/BookKeeperClientBuilder.java | 4 ++-- .../DistributedLogConfiguration.java | 16 +--------------- .../apache/distributedlog/LocalDLMEmulator.java | 4 ++-- .../org/apache/distributedlog/ReadUtils.java | 8 ++++---- .../config/DynamicConfigurationFactory.java | 2 +- .../distributedlog/impl/BKNamespaceDriver.java | 4 ++-- .../distributedlog/impl/ZKLogMetadataStore.java | 2 +- .../federated/FederatedZKLogMetadataStore.java | 8 ++++---- .../impl/metadata/ZKLogStreamMetadataStore.java | 2 +- .../metadata/LogMetadataForReader.java | 2 +- .../metadata/LogMetadataStore.java | 2 +- .../metadata/LogStreamMetadataStore.java | 2 +- .../distributedlog/util/CommandLineUtils.java | 8 ++++---- .../org/apache/distributedlog/util/Utils.java | 6 +++--- .../distributedlog/TestBKLogReadHandler.java | 2 +- .../distributedlog/TestDistributedLogBase.java | 2 +- .../TestDistributedLogConfiguration.java | 4 ++-- .../distributedlog/TestReadAheadEntryReader.java | 4 ++-- .../org/apache/distributedlog/TestReadUtils.java | 2 +- .../config/TestDynamicConfigurationFactory.java | 2 +- .../impl/TestZKLogMetadataStore.java | 2 +- .../TestFederatedZKLogMetadataStore.java | 2 +- .../apache/distributedlog/util/TestUtils.java | 4 ++-- 33 files changed, 65 insertions(+), 81 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java index 85d42c60268..a1334d87dce 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientInternalConf.java @@ -20,8 +20,7 @@ */ package org.apache.bookkeeper.client; -import com.google.common.base.Optional; - +import java.util.Optional; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -92,7 +91,7 @@ private ClientInternalConf(ClientConfiguration conf, conf.getMaxSpeculativeReadTimeout(), conf.getSpeculativeReadTimeoutBackoffMultiplier())); } else { - this.readSpeculativeRequestPolicy = Optional.absent(); + this.readSpeculativeRequestPolicy = Optional.empty(); } if (conf.getFirstSpeculativeReadLACTimeout() > 0) { this.readLACSpeculativeRequestPolicy = @@ -101,7 +100,7 @@ private ClientInternalConf(ClientConfiguration conf, conf.getMaxSpeculativeReadLACTimeout(), conf.getSpeculativeReadLACTimeoutBackoffMultiplier())); } else { - this.readLACSpeculativeRequestPolicy = Optional.absent(); + this.readLACSpeculativeRequestPolicy = Optional.empty(); } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java index c04ecc92005..d9ab2e6e75b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java @@ -17,12 +17,12 @@ */ package org.apache.bookkeeper.proto; -import com.google.common.base.Optional; import com.google.common.base.Stopwatch; import io.netty.channel.Channel; import io.netty.util.HashedWheelTimer; import io.netty.util.Timeout; import java.io.IOException; +import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; @@ -44,7 +44,7 @@ class LongPollReadEntryProcessorV3 extends ReadEntryProcessorV3 implements Watch private static final Logger logger = LoggerFactory.getLogger(LongPollReadEntryProcessorV3.class); private final Long previousLAC; - private Optional lastAddConfirmedUpdateTime = Optional.absent(); + private Optional lastAddConfirmedUpdateTime = Optional.empty(); // long poll execution state private final ExecutorService longPollThreadPool; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLastConfirmedAndEntryContext.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLastConfirmedAndEntryContext.java index 1bf3685b6c1..6c3a57f9465 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLastConfirmedAndEntryContext.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLastConfirmedAndEntryContext.java @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.proto; -import com.google.common.base.Optional; +import java.util.Optional; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallbackCtx; @@ -30,7 +30,7 @@ public class ReadLastConfirmedAndEntryContext implements ReadEntryCallbackCtx { final int bookieIndex; final BookieSocketAddress bookie; long lac = LedgerHandle.INVALID_ENTRY_ID; - Optional lacUpdateTimestamp = Optional.absent(); + Optional lacUpdateTimestamp = Optional.empty(); public ReadLastConfirmedAndEntryContext(int bookieIndex, BookieSocketAddress bookie) { this.bookieIndex = bookieIndex; diff --git a/site/docs/latest/api/distributedlog-api.md b/site/docs/latest/api/distributedlog-api.md index f073b291048..9002d058dff 100644 --- a/site/docs/latest/api/distributedlog-api.md +++ b/site/docs/latest/api/distributedlog-api.md @@ -222,7 +222,7 @@ logConf.setRetentionPeriodHours(12); DistributedLogManager logManager = namespace.openLog( "test-log", Optional.of(logConf), - Optional.absent() + Optional.empty() ); ``` diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java index 3c8dfb7aa1e..4f4f9574625 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java @@ -18,12 +18,12 @@ package org.apache.distributedlog; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.google.common.base.Stopwatch; import com.google.common.base.Ticker; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ScheduledFuture; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java index b21b210bc81..a99e84479b2 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java @@ -22,11 +22,11 @@ import static org.apache.distributedlog.namespace.NamespaceDriver.Role.WRITER; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import java.io.IOException; import java.net.URI; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; @@ -291,7 +291,7 @@ public void checkClosedOrInError(String operation) throws AlreadyClosedException // Create Read Handler synchronized BKLogReadHandler createReadHandler() { - Optional subscriberId = Optional.absent(); + Optional subscriberId = Optional.empty(); return createReadHandler(subscriberId, false); } @@ -634,7 +634,7 @@ public LogReader openLogReader(long fromTxnId) @Override public LogReader openLogReader(DLSN fromDLSN) throws IOException { - return getInputStreamInternal(fromDLSN, Optional.absent()); + return getInputStreamInternal(fromDLSN, Optional.empty()); } @Override @@ -704,7 +704,7 @@ public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException { @Override public CompletableFuture openAsyncLogReader(DLSN fromDLSN) { - Optional subscriberId = Optional.absent(); + Optional subscriberId = Optional.empty(); AsyncLogReader reader = new BKAsyncLogReader( this, scheduler, @@ -722,7 +722,7 @@ public CompletableFuture openAsyncLogReader(DLSN fromDLSN) { */ @Override public CompletableFuture getAsyncLogReaderWithLock(final DLSN fromDLSN) { - Optional subscriberId = Optional.absent(); + Optional subscriberId = Optional.empty(); return getAsyncLogReaderWithLock(Optional.of(fromDLSN), subscriberId); } @@ -733,7 +733,7 @@ public CompletableFuture getAsyncLogReaderWithLock(final DLSN fr @Override public CompletableFuture getAsyncLogReaderWithLock(String subscriberId) { - Optional fromDLSN = Optional.absent(); + Optional fromDLSN = Optional.empty(); return getAsyncLogReaderWithLock(fromDLSN, Optional.of(subscriberId)); } @@ -1068,7 +1068,7 @@ public CompletableFuture asyncClose() { CompletableFuture closeResult = Utils.closeSequence(null, true, readHandlerToClose, pendingReaders, - resourcesCloseable.or(AsyncCloseable.NULL)); + resourcesCloseable.orElse(AsyncCloseable.NULL)); FutureUtils.proxyTo(closeResult, closeFuture); return closeFuture; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java index 02a0cb8d859..a55d0fc1c83 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java @@ -154,7 +154,7 @@ public void deleteLog(String logName) throws InvalidStreamNameException, LogNotFoundException, IOException { checkState(); logName = validateAndNormalizeName(logName); - com.google.common.base.Optional uri = Utils.ioResult(driver.getLogMetadataStore().getLogLocation(logName)); + Optional uri = Utils.ioResult(driver.getLogMetadataStore().getLogLocation(logName)); if (!uri.isPresent()) { throw new LogNotFoundException("Log " + logName + " isn't found."); } @@ -183,7 +183,7 @@ public DistributedLogManager openLog(String logName, throws InvalidStreamNameException, IOException { checkState(); logName = validateAndNormalizeName(logName); - com.google.common.base.Optional uri = Utils.ioResult(driver.getLogMetadataStore().getLogLocation(logName)); + Optional uri = Utils.ioResult(driver.getLogMetadataStore().getLogLocation(logName)); if (!uri.isPresent()) { throw new LogNotFoundException("Log " + logName + " isn't found."); } @@ -220,7 +220,7 @@ public CompletableFuture renameLog(String oldName, String newName) { public boolean logExists(String logName) throws IOException, IllegalArgumentException { checkState(); - com.google.common.base.Optional uri = Utils.ioResult(driver.getLogMetadataStore().getLogLocation(logName)); + Optional uri = Utils.ioResult(driver.getLogMetadataStore().getLogLocation(logName)); if (uri.isPresent()) { try { Utils.ioResult(driver.getLogStreamMetadataStore(WRITER) @@ -309,8 +309,7 @@ protected DistributedLogManager openLogInternal( failureInjector, /* Failure Injector */ statsLogger, /* Stats Logger */ perLogStatsLogger, /* Per Log Stats Logger */ - com.google.common.base.Optional.absent() - /* shared resources, we don't need to close any resources in dlm */ + Optional.empty() /* shared resources, we don't need to close any resources in dlm */ ); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java index c865c28b5f7..d7606e2fef6 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java @@ -18,11 +18,11 @@ package org.apache.distributedlog; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArraySet; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java index 892ab063e82..37bb8c1d787 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java @@ -18,11 +18,11 @@ package org.apache.distributedlog; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.google.common.base.Ticker; import java.io.IOException; import java.util.LinkedList; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -70,7 +70,7 @@ class BKSyncLogReader implements LogReader, AsyncNotification { StatsLogger statsLogger) throws IOException { this.bkdlm = bkdlm; this.readHandler = bkdlm.createReadHandler( - Optional.absent(), + Optional.empty(), this, true); this.maxReadAheadWaitTime = conf.getReadAheadWaitTime(); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java index a03530da4ec..bffbde3dfa9 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java @@ -19,11 +19,11 @@ import static com.google.common.base.Charsets.UTF_8; -import com.google.common.base.Optional; import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; import java.io.IOException; import java.util.Collections; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BKException; @@ -117,7 +117,7 @@ private synchronized void commonInitialization( .setStatsLogger(statsLogger) .dnsResolver(dnsResolver) .requestTimer(requestTimer) - .featureProvider(featureProvider.orNull()) + .featureProvider(featureProvider.orElse(null)) .build(); } catch (BKException bke) { throw new IOException(bke); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java index b16034cf2f5..c2152a9e978 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java @@ -20,9 +20,9 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import com.google.common.base.Optional; import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; +import java.util.Optional; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -59,7 +59,7 @@ public static BookKeeperClientBuilder newBuilder() { // request timer private HashedWheelTimer requestTimer = null; // feature provider - private Optional featureProvider = Optional.absent(); + private Optional featureProvider = Optional.empty(); // Cached BookKeeper Client private BookKeeperClient cachedClient = null; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java index 7a71ba85dd3..3a0138b6a1d 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java @@ -19,11 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; -import com.google.common.base.Optional; import com.google.common.collect.Sets; import java.net.URL; import java.util.ArrayList; import java.util.Iterator; +import java.util.Optional; import java.util.Set; import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -539,22 +539,8 @@ public void loadConf(Configuration baseConf) { * Load whitelisted stream configuration from another configuration object. * * @param streamConfiguration stream configuration overrides - * @Deprecated since 0.5.0, in favor of using {@link #loadStreamConf(java.util.Optional)} */ public void loadStreamConf(Optional streamConfiguration) { - if (streamConfiguration.isPresent()) { - loadStreamConf(java.util.Optional.of(streamConfiguration.get())); - } else { - loadStreamConf(java.util.Optional.empty()); - } - } - - /** - * Load whitelisted stream configuration from another configuration object. - * - * @param streamConfiguration stream configuration overrides - */ - public void loadStreamConf(java.util.Optional streamConfiguration) { if (!streamConfiguration.isPresent()) { return; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java index ec35a168b00..196caeb4d91 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java @@ -17,13 +17,13 @@ */ package org.apache.distributedlog; -import com.google.common.base.Optional; import java.io.File; import java.io.IOException; import java.net.BindException; import java.net.URI; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -78,7 +78,7 @@ public static class Builder { private int zkPort = DEFAULT_ZK_PORT; private int initialBookiePort = DEFAULT_BOOKIE_INITIAL_PORT; private boolean shouldStartZK = true; - private Optional serverConf = Optional.absent(); + private Optional serverConf = Optional.empty(); public Builder numBookies(int numBookies) { this.numBookies = numBookies; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java index 32334735267..db1174ff477 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java @@ -17,10 +17,10 @@ */ package org.apache.distributedlog; -import com.google.common.base.Optional; import com.google.common.collect.Lists; import java.io.IOException; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; @@ -497,7 +497,7 @@ public static CompletableFuture> getLogRecordNotLess if (segment.getLastTxId() < transactionId) { // all log records whose transaction id is less than provided transactionId // then return none - Optional noneRecord = Optional.absent(); + Optional noneRecord = Optional.empty(); return FutureUtils.value(noneRecord); } } @@ -513,7 +513,7 @@ public void onSuccess(final LogSegmentRandomAccessEntryReader reader) { if (lastEntryId < 0) { // it means that the log segment is created but not written yet or an empty log segment. //it is equivalent to 'all log records whose transaction id is less than provided transactionId' - Optional nonRecord = Optional.absent(); + Optional nonRecord = Optional.empty(); promise.complete(nonRecord); return; } @@ -550,7 +550,7 @@ public void onFailure(Throwable cause) { reader, Lists.newArrayList(0L, lastEntryId), nWays, - Optional.absent(), + Optional.empty(), promise); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java index 1bfcd9cb340..2512c3b341a 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java @@ -19,7 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import com.google.common.base.Optional; import com.google.common.collect.Lists; import java.io.File; import java.net.MalformedURLException; @@ -27,6 +26,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import org.apache.commons.configuration.ConfigurationException; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java index f31698640ba..afb3c33f96e 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java @@ -21,7 +21,6 @@ import static org.apache.distributedlog.util.DLUtils.validateAndNormalizeName; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.channel.EventLoopGroup; import io.netty.channel.epoll.EpollEventLoopGroup; @@ -32,6 +31,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -293,7 +293,7 @@ private void initializeBookKeeperClients() throws IOException { bkdlConfig.getBkLedgersPath(), eventLoopGroup, requestTimer, - Optional.absent(), + Optional.empty(), statsLogger); } this.readerBKC = this.sharedReaderBKCBuilder.build(); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java index a37484e2d56..15277e251df 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java @@ -19,11 +19,11 @@ import static org.apache.distributedlog.util.DLUtils.isReservedStreamName; -import com.google.common.base.Optional; import com.google.common.collect.Lists; import java.net.URI; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java index 280d8ea0287..323b7953b11 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java @@ -21,7 +21,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; -import com.google.common.base.Optional; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -31,6 +30,7 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -641,7 +641,7 @@ public CompletableFuture> getLogLocation(final String logName) { return postStateCheck(FutureUtils.value(Optional.of(location))); } if (!forceCheckLogExistence) { - Optional result = Optional.absent(); + Optional result = Optional.empty(); return FutureUtils.value(result); } return postStateCheck(fetchLogLocation(logName).thenApply((uriOptional) -> { @@ -663,7 +663,7 @@ private CompletableFuture> fetchLogLocation(final String logName) FutureUtils.collect(fetchFutures).whenComplete(new FutureEventListener>>() { @Override public void onSuccess(List> fetchResults) { - Optional result = Optional.absent(); + Optional result = Optional.empty(); for (Optional fetchResult : fetchResults) { if (result.isPresent()) { if (fetchResult.isPresent()) { @@ -701,7 +701,7 @@ public void processResult(int rc, String path, Object ctx, Stat stat) { if (Code.OK.intValue() == rc) { fetchPromise.complete(Optional.of(uri)); } else if (Code.NONODE.intValue() == rc) { - fetchPromise.complete(Optional.absent()); + fetchPromise.complete(Optional.empty()); } else { fetchPromise.completeExceptionally(KeeperException.create(Code.get(rc))); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java index 869a968e232..45368ce3e86 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java @@ -31,13 +31,13 @@ import static org.apache.distributedlog.metadata.LogMetadata.VERSION_PATH; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.google.common.collect.Lists; import java.io.IOException; import java.net.URI; import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.Optional; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java index c07d5b4f2c4..0053ad3a928 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java @@ -17,8 +17,8 @@ */ package org.apache.distributedlog.metadata; -import com.google.common.base.Optional; import java.net.URI; +import java.util.Optional; /** * Log Metadata for Reader. diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataStore.java index 62b75a424ac..c09ceebe3eb 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataStore.java @@ -18,9 +18,9 @@ package org.apache.distributedlog.metadata; import com.google.common.annotations.Beta; -import com.google.common.base.Optional; import java.net.URI; import java.util.Iterator; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.distributedlog.callback.NamespaceListener; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java index 712619a5b5f..3b184d705f2 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java @@ -18,9 +18,9 @@ package org.apache.distributedlog.metadata; import com.google.common.annotations.Beta; -import com.google.common.base.Optional; import java.io.Closeable; import java.net.URI; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.distributedlog.common.util.PermitManager; import org.apache.distributedlog.lock.DistributedLock; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java index a999f71ce90..d0af2770c4c 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.util; -import com.google.common.base.Optional; +import java.util.Optional; import org.apache.commons.cli.CommandLine; /** @@ -29,7 +29,7 @@ public static Optional getOptionalStringArg(CommandLine cmdline, String if (cmdline.hasOption(arg)) { return Optional.of(cmdline.getOptionValue(arg)); } else { - return Optional.absent(); + return Optional.empty(); } } @@ -37,7 +37,7 @@ public static Optional getOptionalBooleanArg(CommandLine cmdline, Strin if (cmdline.hasOption(arg)) { return Optional.of(true); } else { - return Optional.absent(); + return Optional.empty(); } } @@ -47,7 +47,7 @@ public static Optional getOptionalIntegerArg(CommandLine cmdline, Strin if (cmdline.hasOption(arg)) { return Optional.of(Integer.parseInt(cmdline.getOptionValue(arg))); } else { - return Optional.absent(); + return Optional.empty(); } } catch (NumberFormatException ex) { throw new IllegalArgumentException(arg + " is not a number"); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java index 8c4e18b7621..e7cd70b4256 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java @@ -18,12 +18,12 @@ package org.apache.distributedlog.util; import com.google.common.base.Objects; -import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.io.Closeables; import java.io.Closeable; import java.io.IOException; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -216,7 +216,7 @@ public static CompletableFuture zkAsyncCreateFullPathOptimistic( final byte[] data, final List acl, final CreateMode createMode) { - Optional parentPathShouldNotCreate = Optional.absent(); + Optional parentPathShouldNotCreate = Optional.empty(); return zkAsyncCreateFullPathOptimistic( zkc, pathToCreate, @@ -282,7 +282,7 @@ public void processResult(int rc, String path, Object ctx, Stat stat) { return; } - Optional parentPathShouldNotCreate = Optional.absent(); + Optional parentPathShouldNotCreate = Optional.empty(); zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate, data, acl, createMode, new AsyncCallback.StringCallback() { @Override diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java index a8b4f93f781..da4dabd71c4 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java @@ -22,9 +22,9 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import com.google.common.base.Optional; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java index d069e7251c3..cbba9f1c0c8 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java @@ -19,13 +19,13 @@ import static org.junit.Assert.assertTrue; -import com.google.common.base.Optional; import com.google.common.base.Ticker; import java.io.File; import java.io.IOException; import java.net.URI; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.LedgerHandle; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java index 1b864cbe7b1..03a7a209d0c 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java @@ -21,8 +21,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import com.google.common.base.Optional; import java.util.List; +import java.util.Optional; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.commons.configuration.StrictConfigurationComparator; import org.apache.distributedlog.net.DNSResolverForRacks; @@ -88,7 +88,7 @@ public void loadStreamConfBadOverrideIgnored() throws Exception { public void loadStreamConfNullOverrides() throws Exception { DistributedLogConfiguration conf = new DistributedLogConfiguration(); DistributedLogConfiguration confClone = new DistributedLogConfiguration(); - Optional streamConfiguration = Optional.absent(); + Optional streamConfiguration = Optional.empty(); conf.loadStreamConf(streamConfiguration); StrictConfigurationComparator comp = new StrictConfigurationComparator(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java index 437bb6fab50..7cc34534c22 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java @@ -24,10 +24,10 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import com.google.common.base.Optional; import com.google.common.base.Ticker; import com.google.common.collect.Lists; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -116,7 +116,7 @@ private ReadAheadEntryReader createEntryReader(String streamName, DistributedLogConfiguration conf) throws Exception { BKLogReadHandler readHandler = dlm.createReadHandler( - Optional.absent(), + Optional.empty(), true); LogSegmentEntryStore entryStore = new BKLogSegmentEntryStore( conf, diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java index 033065bc34a..9adf6f6eedb 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java @@ -21,9 +21,9 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import com.google.common.base.Optional; import com.google.common.collect.Lists; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java index cba20c2ab46..02eea8083c3 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java @@ -20,8 +20,8 @@ import static org.junit.Assert.assertEquals; import com.google.common.base.Objects; -import com.google.common.base.Optional; import java.io.File; +import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java index 2ba24db47b5..12ad923d1d2 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java @@ -20,9 +20,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import com.google.common.base.Optional; import com.google.common.collect.Sets; import java.net.URI; +import java.util.Optional; import java.util.Set; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.distributedlog.DistributedLogConfiguration; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java index f32f4ce3b27..3ce838dff8d 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java @@ -22,13 +22,13 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import com.google.common.base.Optional; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import java.net.URI; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.TreeSet; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestUtils.java index de5da8c5c83..8abb61ffa8c 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestUtils.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestUtils.java @@ -23,7 +23,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; -import com.google.common.base.Optional; +import java.util.Optional; import java.util.concurrent.CountDownLatch; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Versioned; @@ -64,7 +64,7 @@ public void teardown() throws Exception { @Test(timeout = 60000) public void testZkAsyncCreateFulPathOptimisticRecursive() throws Exception { String path1 = "/a/b/c/d"; - Optional parentPathShouldNotCreate = Optional.absent(); + Optional parentPathShouldNotCreate = Optional.empty(); final CountDownLatch doneLatch1 = new CountDownLatch(1); Utils.zkAsyncCreateFullPathOptimisticRecursive(zkc, path1, parentPathShouldNotCreate, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, From 9726af0bf982fbd8b5e7279d1eeb7c53937134fa Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Sun, 31 May 2020 02:00:15 +0800 Subject: [PATCH 0434/1642] [CLEAN] Replace google.charsets with java.charsets ### Motivation `com.google.common.base.Charsets` and `java.nio.charset.StandardCharsets` exist in the code base at the same time, we can improve them. ### Changes Replace `com.google.common.base.Charsets` with `java.nio.charset.StandardCharsets`. Reviewers: Matteo Minardi , Sijie Guo This closes #2342 from lamber-ken/clean-charsets --- .../benchmark/BenchReadThroughputLatency.java | 2 +- .../bookkeeper/benchmark/BenchThroughputLatency.java | 2 +- .../org/apache/bookkeeper/benchmark/TestClient.java | 2 +- .../org/apache/bookkeeper/bookie/BookieStatus.java | 2 +- .../main/java/org/apache/bookkeeper/bookie/Cookie.java | 2 +- .../java/org/apache/bookkeeper/bookie/EntryLogger.java | 2 +- .../apache/bookkeeper/bookie/EntryLoggerAllocator.java | 2 +- .../java/org/apache/bookkeeper/bookie/FileInfo.java | 2 +- .../apache/bookkeeper/bookie/FileSystemUpgrade.java | 2 +- .../org/apache/bookkeeper/bookie/JournalChannel.java | 2 +- .../apache/bookkeeper/conf/ClientConfiguration.java | 2 +- .../bookkeeper/discover/ZKRegistrationManager.java | 10 +++++----- .../apache/bookkeeper/meta/LedgerMetadataSerDe.java | 2 +- .../meta/ZkLedgerUnderreplicationManager.java | 2 +- .../java/org/apache/bookkeeper/metastore/Value.java | 2 +- .../org/apache/bookkeeper/proto/BookieClientImpl.java | 2 +- .../bookkeeper/proto/checksum/MacDigestManager.java | 2 +- .../apache/bookkeeper/replication/AuditorElector.java | 2 +- .../server/http/service/ListLedgerService.java | 2 +- .../server/http/service/ReadLedgerEntryService.java | 2 +- .../main/java/org/apache/bookkeeper/util/HardLink.java | 2 +- .../org/apache/bookkeeper/util/LocalBookKeeper.java | 2 +- .../main/java/org/apache/bookkeeper/util/Shell.java | 7 ++++--- .../bookkeeper/bookie/BookieDeferredSyncTest.java | 2 +- .../bookkeeper/bookie/BookieInitializationTest.java | 2 +- .../org/apache/bookkeeper/bookie/BookieShellTest.java | 2 +- .../bookkeeper/bookie/IndexPersistenceMgrTest.java | 2 +- .../apache/bookkeeper/client/BookKeeperAdminTest.java | 2 +- .../org/apache/bookkeeper/client/LedgerCloseTest.java | 2 +- .../apache/bookkeeper/client/LedgerMetadataTest.java | 2 +- .../bookkeeper/client/ParallelLedgerRecoveryTest.java | 2 +- .../bookkeeper/client/TestDisableEnsembleChange.java | 2 +- .../client/TestReadLastConfirmedAndEntry.java | 2 +- .../bookkeeper/client/api/BookKeeperApiTest.java | 2 +- .../bookkeeper/client/api/WriteAdvHandleTest.java | 2 +- .../apache/bookkeeper/client/api/WriteHandleTest.java | 2 +- .../bookkeeper/client/impl/LedgerEntriesImplTest.java | 2 +- .../bookkeeper/client/impl/LedgerEntryImplTest.java | 2 +- .../apache/bookkeeper/test/MultipleThreadReadTest.java | 2 +- .../src/main/resources/bookkeeper/checkstyle.xml | 2 +- .../common/util/affinity/impl/IsolatedProcessors.java | 4 ++-- .../common/util/affinity/impl/ProcessorsInfo.java | 5 ++--- .../org/apache/bookkeeper/proto/ProtocolBenchmark.java | 2 +- .../apache/bookkeeper/clients/impl/kv/TestKvUtils.java | 2 +- .../bookkeeper/common/coder/StringUtf8Coder.java | 4 ++-- .../bookkeeper/common/router/StringUtf8HashRouter.java | 2 +- .../apache/distributedlog/io/TestCompressionCodec.java | 2 +- .../org/apache/distributedlog/BKLogSegmentWriter.java | 2 +- .../org/apache/distributedlog/BookKeeperClient.java | 2 +- .../apache/distributedlog/DistributedLogConstants.java | 2 +- .../org/apache/distributedlog/LogSegmentMetadata.java | 2 +- .../org/apache/distributedlog/ZooKeeperClient.java | 2 +- .../org/apache/distributedlog/auditor/DLAuditor.java | 2 +- .../distributedlog/impl/ZKLogSegmentMetadataStore.java | 2 +- .../distributedlog/impl/acl/ZKAccessControl.java | 2 +- .../impl/federated/FederatedZKLogMetadataStore.java | 2 +- .../impl/logsegment/BKLogSegmentEntryReader.java | 2 +- .../impl/logsegment/BKLogSegmentEntryStore.java | 2 +- .../distributedlog/impl/metadata/BKDLConfig.java | 2 +- .../impl/metadata/ZKLogStreamMetadataStore.java | 2 +- .../impl/subscription/ZKSubscriptionStateStore.java | 6 +++--- .../org/apache/distributedlog/lock/ZKSessionLock.java | 2 +- .../org/apache/distributedlog/metadata/DLMetadata.java | 2 +- .../distributedlog/tools/DistributedLogTool.java | 2 +- .../java/org/apache/distributedlog/util/DLUtils.java | 2 +- .../java/org/apache/distributedlog/DLMTestUtil.java | 2 +- .../apache/distributedlog/TestAsyncReaderWriter.java | 2 +- .../distributedlog/TestBKDistributedLogManager.java | 2 +- .../apache/distributedlog/TestBKLogSegmentWriter.java | 2 +- .../test/java/org/apache/distributedlog/TestEntry.java | 2 +- .../apache/distributedlog/TestLogSegmentMetadata.java | 2 +- .../org/apache/distributedlog/TestLogSegmentsZK.java | 2 +- .../org/apache/distributedlog/TestRollLogSegments.java | 2 +- .../apache/distributedlog/acl/TestZKAccessControl.java | 2 +- .../apache/distributedlog/bk/TestLedgerAllocator.java | 2 +- .../impl/metadata/TestZKLogStreamMetadataStore.java | 2 +- .../org/apache/distributedlog/util/TestDLUtils.java | 2 +- .../java/org/apache/distributedlog/util/TestUtils.java | 2 +- .../org/apache/distributedlog/fs/DLOutputStream.java | 2 +- .../org/apache/distributedlog/fs/TestDLFileSystem.java | 2 +- .../org/apache/distributedlog/TestLogRecordSet.java | 2 +- .../bookkeeper/statelib/impl/kv/RocksdbKVStore.java | 2 +- .../impl/rocksdb/checkpoint/dlog/DLOutputStream.java | 2 +- .../statelib/impl/kv/TestRocksdbKVAsyncStore.java | 2 +- .../kv/TestRocksdbKVAsyncStoreWithCheckpoints.java | 2 +- .../impl/mvcc/TestMVCCAsyncBytesStoreImpl.java | 2 +- .../impl/rocksdb/checkpoint/RocksCheckpointerTest.java | 2 +- .../rocksdb/checkpoint/dlog/DLCheckpointStoreTest.java | 2 +- .../rocksdb/checkpoint/fs/FSCheckpointManagerTest.java | 2 +- .../storage/impl/metadata/RootRangeStoreImpl.java | 2 +- 90 files changed, 102 insertions(+), 102 deletions(-) diff --git a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchReadThroughputLatency.java b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchReadThroughputLatency.java index 18c73e88e1b..50a14f19ddd 100644 --- a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchReadThroughputLatency.java +++ b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchReadThroughputLatency.java @@ -19,7 +19,7 @@ */ package org.apache.bookkeeper.benchmark; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import java.util.ArrayList; import java.util.Comparator; 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 e7f7f8ae3cd..3d34a773e00 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 @@ -19,7 +19,7 @@ */ package org.apache.bookkeeper.benchmark; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import java.io.BufferedOutputStream; 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 3d96c5c95d2..050a9b97c1b 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 @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.benchmark; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import java.io.FileOutputStream; import java.io.IOException; 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 index ecdf82e9d64..e24118f1214 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStatus.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStatus.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.BOOKIE_STATUS_FILENAME; import java.io.BufferedReader; 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 79a1d4a6f97..d6711aef821 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 @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.base.Joiner; import com.google.common.collect.Sets; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index 731275c9986..0d7c8614ad3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -21,7 +21,7 @@ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java index 1c32b55de03..1874dfa4ff6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLoggerAllocator.java @@ -21,7 +21,7 @@ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.bookie.TransactionalEntryLogCompactor.COMPACTING_SUFFIX; import io.netty.buffer.ByteBuf; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java index e8571be6fdc..09b751e545b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java @@ -21,7 +21,7 @@ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification.WATCHER_RECYCLER; import com.google.common.annotations.VisibleForTesting; 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 9cd0090480c..3686bef3d3b 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 @@ -21,7 +21,7 @@ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; import com.google.common.collect.Lists; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java index 2dbaee02197..5434d168d4f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java @@ -21,7 +21,7 @@ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; 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 f752fc8e79f..4b255f430c0 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 @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.conf; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.FEATURE_DISABLE_ENSEMBLE_CHANGE; import io.netty.buffer.ByteBuf; 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 index 9272eb8401d..951bb9581ca 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.discover; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.COOKIE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.EMPTY_BYTE_ARRAY; @@ -26,10 +26,10 @@ import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; import com.google.common.collect.Lists; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.List; import java.util.UUID; @@ -436,12 +436,12 @@ public boolean prepareFormat() throws Exception { boolean availableNodeExists = null != zk.exists(bookieRegistrationPath, false); // Create ledgers root node if not exists if (!ledgerRootExists) { - ZkUtils.createFullPathOptimistic(zk, ledgersRootPath, "".getBytes(Charsets.UTF_8), zkAcls, + ZkUtils.createFullPathOptimistic(zk, ledgersRootPath, "".getBytes(StandardCharsets.UTF_8), zkAcls, CreateMode.PERSISTENT); } // create available bookies node if not exists if (!availableNodeExists) { - zk.create(bookieRegistrationPath, "".getBytes(Charsets.UTF_8), zkAcls, CreateMode.PERSISTENT); + zk.create(bookieRegistrationPath, "".getBytes(StandardCharsets.UTF_8), zkAcls, CreateMode.PERSISTENT); } // create readonly bookies node if not exists @@ -586,7 +586,7 @@ public boolean format() throws Exception { // create INSTANCEID String instanceId = UUID.randomUUID().toString(); zk.create(ledgersRootPath + "/" + BookKeeperConstants.INSTANCEID, - instanceId.getBytes(Charsets.UTF_8), zkAcls, CreateMode.PERSISTENT); + instanceId.getBytes(StandardCharsets.UTF_8), zkAcls, CreateMode.PERSISTENT); log.info("Successfully formatted BookKeeper metadata"); return true; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java index 40048447a36..aded61dd8a8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -17,9 +17,9 @@ */ package org.apache.bookkeeper.meta; -import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.protobuf.ByteString; import com.google.protobuf.TextFormat; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java index 7f3d1c2297d..554973273b3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.meta; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.base.Joiner; import com.google.protobuf.InvalidProtocolBufferException; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/Value.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/Value.java index fef83dbd19c..43354c344e5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/Value.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/Value.java @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.metastore; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.metastore.MetastoreTable.ALL_FIELDS; import com.google.common.hash.HashFunction; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index 5b708e50d58..f81650b4087 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.proto; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.util.SafeRunnable.safeRun; import com.google.common.collect.Lists; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java index 92b93f1450c..edcb54ecf8a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/MacDigestManager.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.proto.checksum; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java index 543aaac57de..b23b0b0211d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.replication; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; import static org.apache.bookkeeper.replication.ReplicationStats.ELECTION_ATTEMPTS; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java index 07ea5af4791..45e692d26e3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java @@ -18,8 +18,8 @@ */ package org.apache.bookkeeper.server.http.service; -import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkNotNull; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.collect.Maps; import java.util.LinkedHashMap; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ReadLedgerEntryService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ReadLedgerEntryService.java index 4c2141dc1bd..3c53e26b0fd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ReadLedgerEntryService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ReadLedgerEntryService.java @@ -18,8 +18,8 @@ */ package org.apache.bookkeeper.server.http.service; -import static com.google.common.base.Charsets.US_ASCII; import static com.google.common.base.Preconditions.checkNotNull; +import static java.nio.charset.StandardCharsets.US_ASCII; import com.google.common.collect.Maps; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/HardLink.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/HardLink.java index c8c7a71a3de..ea92867b3eb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/HardLink.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/HardLink.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.util; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import java.io.BufferedReader; import java.io.File; 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 0bfc923a043..36cf72b8b3e 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 @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.util; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java index 8067a608968..16d597eeb19 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/Shell.java @@ -17,11 +17,11 @@ */ package org.apache.bookkeeper.util; -import com.google.common.base.Charsets; import java.io.BufferedReader; import java.io.File; import java.io.IOException; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.Timer; import java.util.TimerTask; @@ -130,8 +130,9 @@ private void runCommand() throws IOException { timeOutTimer.schedule(timeoutTimerTask, timeOutInterval); } final BufferedReader errReader = new BufferedReader(new InputStreamReader(process.getErrorStream(), - Charsets.UTF_8)); - BufferedReader inReader = new BufferedReader(new InputStreamReader(process.getInputStream(), Charsets.UTF_8)); + StandardCharsets.UTF_8)); + BufferedReader inReader = new BufferedReader(new InputStreamReader(process.getInputStream(), + StandardCharsets.UTF_8)); final StringBuffer errMsg = new StringBuffer(); // read error and input streams as this would free up the buffers diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieDeferredSyncTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieDeferredSyncTest.java index 11a181a2573..dcac8f03d20 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieDeferredSyncTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieDeferredSyncTest.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import static org.junit.Assert.assertEquals; 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 2fcc8a7a2de..25df88f4bd1 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,7 +20,7 @@ */ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.bookie.BookieJournalTest.writeV5Journal; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java index b0e0a453faf..b7e73b988b5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java index 5b31b7183a0..54f6c862118 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index ecbc4f0bc60..d10d9abb10c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.client; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; import static org.hamcrest.Matchers.is; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java index e761f1ce194..25f7f1aacba 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.client; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java index 6fee301e70d..b993b4f66d5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.client; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index 710756ea9b5..e3c295e48af 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.client; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java index 000765c2adc..4417f543ae4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.client; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.util.BookKeeperConstants.FEATURE_DISABLE_ENSEMBLE_CHANGE; import static org.apache.bookkeeper.util.TestUtils.assertEventuallyTrue; import static org.junit.Assert.assertArrayEquals; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java index 7c2cf5b785f..e6069d6cc47 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.client; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperApiTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperApiTest.java index a71002dd98c..5cd71247a9d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperApiTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperApiTest.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.client.api; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.hasItem; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/WriteAdvHandleTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/WriteAdvHandleTest.java index 585a7b21ee1..a68f6cd2d16 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/WriteAdvHandleTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/WriteAdvHandleTest.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.client.api; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/WriteHandleTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/WriteHandleTest.java index 113f5857693..576d5537715 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/WriteHandleTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/WriteHandleTest.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.client.api; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/impl/LedgerEntriesImplTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/impl/LedgerEntriesImplTest.java index 901e41ef049..ff7117c8e2e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/impl/LedgerEntriesImplTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/impl/LedgerEntriesImplTest.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.client.impl; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/impl/LedgerEntryImplTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/impl/LedgerEntryImplTest.java index ee716803df0..2b7cc0f6737 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/impl/LedgerEntryImplTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/impl/LedgerEntryImplTest.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.client.impl; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/MultipleThreadReadTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/MultipleThreadReadTest.java index 1d3929ca02e..f7fe99a74e4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/MultipleThreadReadTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/MultipleThreadReadTest.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.test; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; diff --git a/buildtools/src/main/resources/bookkeeper/checkstyle.xml b/buildtools/src/main/resources/bookkeeper/checkstyle.xml index 42b499dbab4..1c938d31194 100644 --- a/buildtools/src/main/resources/bookkeeper/checkstyle.xml +++ b/buildtools/src/main/resources/bookkeeper/checkstyle.xml @@ -102,7 +102,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java index fd9cbbca55d..5d732948965 100644 --- a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/IsolatedProcessors.java @@ -21,11 +21,11 @@ package org.apache.bookkeeper.common.util.affinity.impl; -import com.google.common.base.Charsets; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.IOException; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -42,7 +42,7 @@ @Slf4j class IsolatedProcessors { - private static final Charset ENCODING = Charsets.US_ASCII; + private static final Charset ENCODING = StandardCharsets.US_ASCII; private static final String ISOLATED_CPU_PATH = "/sys/devices/system/cpu/isolated"; diff --git a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java index 5f4449bafcd..8a812a84d1a 100644 --- a/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java +++ b/cpu-affinity/src/main/java/org/apache/bookkeeper/common/util/affinity/impl/ProcessorsInfo.java @@ -22,10 +22,9 @@ import static com.google.common.base.Preconditions.checkArgument; -import com.google.common.base.Charsets; - import java.io.IOException; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.Map; @@ -40,7 +39,7 @@ */ class ProcessorsInfo { - private static final Charset ENCODING = Charsets.US_ASCII; + private static final Charset ENCODING = StandardCharsets.US_ASCII; /** * Given one cpu id, return all CPUs that are sharing the same core. diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java index 6ababbe324d..5baad8071d5 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.proto; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.protobuf.ByteString; import io.netty.buffer.ByteBuf; diff --git a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestKvUtils.java b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestKvUtils.java index d2df33411a5..50887ac7ae4 100644 --- a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestKvUtils.java +++ b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestKvUtils.java @@ -14,7 +14,7 @@ package org.apache.bookkeeper.clients.impl.kv; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.clients.impl.kv.KvUtils.newDeleteRequest; import static org.apache.bookkeeper.clients.impl.kv.KvUtils.newIncrementRequest; import static org.apache.bookkeeper.clients.impl.kv.KvUtils.newPutRequest; diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/coder/StringUtf8Coder.java b/stream/common/src/main/java/org/apache/bookkeeper/common/coder/StringUtf8Coder.java index 5d661d5e8a3..111c472fd09 100644 --- a/stream/common/src/main/java/org/apache/bookkeeper/common/coder/StringUtf8Coder.java +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/coder/StringUtf8Coder.java @@ -20,10 +20,10 @@ import static java.nio.charset.StandardCharsets.UTF_8; -import com.google.common.base.Charsets; import com.google.common.base.Utf8; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; +import java.nio.charset.StandardCharsets; import lombok.AccessLevel; import lombok.NoArgsConstructor; @@ -56,7 +56,7 @@ public int getSerializedSize(String value) { @Override public String decode(byte[] data) { - return new String(data, Charsets.UTF_8); + return new String(data, StandardCharsets.UTF_8); } @Override diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/router/StringUtf8HashRouter.java b/stream/common/src/main/java/org/apache/bookkeeper/common/router/StringUtf8HashRouter.java index c3cac620095..e665f5d1c19 100644 --- a/stream/common/src/main/java/org/apache/bookkeeper/common/router/StringUtf8HashRouter.java +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/router/StringUtf8HashRouter.java @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.common.router; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import io.netty.buffer.ByteBuf; import io.netty.buffer.PooledByteBufAllocator; diff --git a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/io/TestCompressionCodec.java b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/io/TestCompressionCodec.java index 5138c3d7da6..c8edd99e600 100644 --- a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/io/TestCompressionCodec.java +++ b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/io/TestCompressionCodec.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.io; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java index 880393c9311..6a6e41b5b38 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.distributedlog.DistributedLogConstants.INVALID_TXID; import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE; import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java index bffbde3dfa9..8c6c25a0f0e 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java index 0ebb3f27c74..cf2149d4b67 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.collect.ImmutableList; import java.util.List; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java index f13273fdd8e..f61b50a820f 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java index 9bb8813e0b0..1e8a7fd6128 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.base.Stopwatch; import java.io.IOException; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java index d97909ae49c..dd652ee0086 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java @@ -17,8 +17,8 @@ */ package org.apache.distributedlog.auditor; -import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.base.Objects; import com.google.common.collect.Lists; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java index f688f445dbe..4a171585052 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.impl; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.collect.ImmutableList; import java.io.IOException; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java index af358ee11f7..6a3c2200810 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.impl.acl; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java index 323b7953b11..4ed3c5ed46d 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.impl.federated; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java index f414bf45555..91c68d4e29a 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.impl.logsegment; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java index fc14df6929f..bb3df1884c8 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.impl.logsegment; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import java.io.IOException; import java.util.concurrent.CompletableFuture; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java index ac53fd5e6f6..f7ff7ad4b82 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.impl.metadata; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java index 45368ce3e86..9a8886e785f 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.impl.metadata; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.distributedlog.DistributedLogConstants.EMPTY_BYTES; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java index 5a5651779d8..a1c2a9f969d 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.impl.subscription; -import com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; import java.io.IOException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; @@ -82,7 +82,7 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta result.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path)); } else { try { - DLSN dlsn = DLSN.deserialize(new String(data, Charsets.UTF_8)); + DLSN dlsn = DLSN.deserialize(new String(data, StandardCharsets.UTF_8)); result.complete(dlsn); } catch (Exception t) { logger.warn("Invalid last commit position found from path {}", zkPath, t); @@ -113,7 +113,7 @@ public CompletableFuture advanceCommitPosition(DLSN newPosition) { || (newPosition.compareTo(dlsn) > 0)) { lastCommittedPositionUpdater.set(this, newPosition); return Utils.zkAsyncCreateFullPathOptimisticAndSetData(zooKeeperClient, - zkPath, newPosition.serialize().getBytes(Charsets.UTF_8), + zkPath, newPosition.serialize().getBytes(StandardCharsets.UTF_8), zooKeeperClient.getDefaultACL(), CreateMode.PERSISTENT); } else { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java index 80e24d08cf0..e776ee90e19 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.lock; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java index 42c42b14574..db0d277adfd 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.metadata; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import java.io.BufferedReader; import java.io.IOException; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java index 74489965f71..f8445ba6f89 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java @@ -17,8 +17,8 @@ */ package org.apache.distributedlog.tools; -import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.collect.Lists; import com.google.common.collect.Sets; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/DLUtils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/DLUtils.java index 635f86ff592..6c579c29575 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/DLUtils.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/DLUtils.java @@ -17,9 +17,9 @@ */ package org.apache.distributedlog.util; -import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.base.Objects; import java.net.InetAddress; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java index 36b1f384904..1e5adf3f5b1 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java index c6833466179..86714707f89 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.distributedlog.DLMTestUtil.validateFutureFailed; import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE; import static org.junit.Assert.assertArrayEquals; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java index 3aada6865fe..28efe89ac3d 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java index 7b18bcf9bc9..7eea32d0a4a 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntry.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntry.java index 59f9c8a7670..a7f413a7882 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntry.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntry.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.distributedlog.EnvelopedEntry.HEADER_LENGTH; import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE; import static org.junit.Assert.assertEquals; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java index 4912e88e910..6fbdeaac58e 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java @@ -18,7 +18,7 @@ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java index cfaa4066e48..61cf16c1664 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java index 99b7424b9bd..7b992da8d46 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java index b2190e95e5d..9c0b9409b5b 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.acl; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java index 50cde8014cc..5819a0d4c4a 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.bk; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java index 1a5d18b5600..dde450dd50e 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.impl.metadata; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.distributedlog.DistributedLogConstants.EMPTY_BYTES; import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.checkLogMetadataPaths; import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.getLog; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java index 3c663c5df44..29526832737 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.util; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.distributedlog.util.DLUtils.validateAndNormalizeName; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestUtils.java index 8abb61ffa8c..039c71c2810 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestUtils.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestUtils.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog.util; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; diff --git a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java index b0b5bb14fea..68c3d087385 100644 --- a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java +++ b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java @@ -18,7 +18,7 @@ package org.apache.distributedlog.fs; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; diff --git a/stream/distributedlog/io/dlfs/src/test/java/org/apache/distributedlog/fs/TestDLFileSystem.java b/stream/distributedlog/io/dlfs/src/test/java/org/apache/distributedlog/fs/TestDLFileSystem.java index bab38043a2c..4a2fc9e321d 100644 --- a/stream/distributedlog/io/dlfs/src/test/java/org/apache/distributedlog/fs/TestDLFileSystem.java +++ b/stream/distributedlog/io/dlfs/src/test/java/org/apache/distributedlog/fs/TestDLFileSystem.java @@ -18,7 +18,7 @@ package org.apache.distributedlog.fs; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java b/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java index c80d3f4d30e..6aa98a90e0a 100644 --- a/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java +++ b/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java @@ -17,7 +17,7 @@ */ package org.apache.distributedlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE; import static org.apache.distributedlog.LogRecordSet.HEADER_LEN; import static org.junit.Assert.assertEquals; diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java index 46ba7f1c13e..64ff0b1d096 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java @@ -18,8 +18,8 @@ package org.apache.bookkeeper.statelib.impl.kv; -import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkNotNull; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.statelib.impl.rocksdb.RocksConstants.BLOCK_CACHE_SIZE; import static org.apache.bookkeeper.statelib.impl.rocksdb.RocksConstants.BLOCK_SIZE; import static org.apache.bookkeeper.statelib.impl.rocksdb.RocksConstants.DEFAULT_CHECKSUM_TYPE; diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLOutputStream.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLOutputStream.java index 8fe62007ac8..834dd6258c5 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLOutputStream.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLOutputStream.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.statelib.impl.rocksdb.checkpoint.dlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStore.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStore.java index df476e7fc78..0ad73e82b10 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStore.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStore.java @@ -18,7 +18,7 @@ */ package org.apache.bookkeeper.statelib.impl.kv; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertNull; diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStoreWithCheckpoints.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStoreWithCheckpoints.java index 033893ff45a..ea005b44c03 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStoreWithCheckpoints.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStoreWithCheckpoints.java @@ -18,7 +18,7 @@ */ package org.apache.bookkeeper.statelib.impl.kv; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCAsyncBytesStoreImpl.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCAsyncBytesStoreImpl.java index 5314ed8a0a1..520c5e63ba1 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCAsyncBytesStoreImpl.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCAsyncBytesStoreImpl.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.statelib.impl.mvcc; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java index 26cef9a3911..e4935798b28 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java @@ -18,7 +18,7 @@ */ package org.apache.bookkeeper.statelib.impl.rocksdb.checkpoint; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStoreTest.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStoreTest.java index 33f342dbf23..0ef2a4ea6f6 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStoreTest.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStoreTest.java @@ -18,7 +18,7 @@ */ package org.apache.bookkeeper.statelib.impl.rocksdb.checkpoint.dlog; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/fs/FSCheckpointManagerTest.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/fs/FSCheckpointManagerTest.java index 0beb41de335..0db780beae4 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/fs/FSCheckpointManagerTest.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/fs/FSCheckpointManagerTest.java @@ -18,7 +18,7 @@ */ package org.apache.bookkeeper.statelib.impl.rocksdb.checkpoint.fs; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java index c066bd14e67..3a4d78ad66d 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.stream.storage.impl.metadata; -import static com.google.common.base.Charsets.UTF_8; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.MIN_DATA_STREAM_ID; import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.validateNamespaceName; import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.validateStreamName; From 5bd5f5fbf9935499c3eafb1e14bfbd544e7e50be Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Sun, 31 May 2020 02:01:27 +0800 Subject: [PATCH 0435/1642] [BUG] Fix bookie port conflict when using LocalBookKeeper ### Motivation When using `bin/bookkeeper localbookie 10`, always output following exception, and run failed. ``` java.net.BindException: Address already in use at sun.nio.ch.Net.bind0(Native Method) at sun.nio.ch.Net.bind(Net.java:433) at sun.nio.ch.Net.bind(Net.java:425) at sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:223) at io.netty.channel.socket.nio.NioServerSocketChannel.doBind(NioServerSocketChannel.java:130) at io.netty.channel.AbstractChannel$AbstractUnsafe.bind(AbstractChannel.java:562) at io.netty.channel.DefaultChannelPipeline$HeadContext.bind(DefaultChannelPipeline.java:1358) at io.netty.channel.AbstractChannelHandlerContext.invokeBind(AbstractChannelHandlerContext.java:501) at io.netty.channel.AbstractChannelHandlerContext.bind(AbstractChannelHandlerContext.java:486) at io.netty.channel.DefaultChannelPipeline.bind(DefaultChannelPipeline.java:1019) at io.netty.channel.AbstractChannel.bind(AbstractChannel.java:258) at io.netty.bootstrap.AbstractBootstrap$2.run(AbstractBootstrap.java:366) at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:474) at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:909) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:748) ``` ### Changes Use `PortManager` to obtain available ports based on initialPort. Reviewers: Enrico Olivelli , Sijie Guo This closes #2338 from lamber-ken/fix-localbookeeper-error --- .../bookkeeper/util/LocalBookKeeper.java | 11 ++--- .../apache/bookkeeper/util}/PortManager.java | 45 ++++++++++++++----- .../src/main/resources/log4j.properties | 38 ++++++++++++++++ .../bookie/AdvertisedAddressTest.java | 2 +- .../bookie/BookieInitializationTest.java | 2 +- .../apache/bookkeeper/bookie/CookieTest.java | 2 +- .../bookie/LedgerStorageCheckpointTest.java | 2 +- .../apache/bookkeeper/bookie/UpgradeTest.java | 2 +- .../test/BookKeeperClusterTestCase.java | 1 + .../bookkeeper/test/BookieZKExpireTest.java | 1 + .../bookkeeper/test/ReadOnlyBookieTest.java | 1 + .../etcd/testing/EtcdBKClusterTestBase.java | 2 +- 12 files changed, 86 insertions(+), 23 deletions(-) rename bookkeeper-server/src/{test/java/org/apache/bookkeeper/test => main/java/org/apache/bookkeeper/util}/PortManager.java (56%) create mode 100644 bookkeeper-server/src/main/resources/log4j.properties 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 36cf72b8b3e..1c8728490da 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 @@ -87,7 +87,8 @@ public LocalBookKeeper( this.initialPort = initialPort; this.localBookiesConfigDir = new File(localBookiesConfigDirName); this.baseConf = baseConf; - LOG.info("Running {} bookie(s) on zkServer {}.", this.numberOfBookies); + LOG.info("Running {} bookie(s) on zk ensemble = '{}:{}'.", this.numberOfBookies, + zooKeeperDefaultHost, zooKeeperDefaultPort); } private static String zooKeeperDefaultHost = "127.0.0.1"; @@ -238,10 +239,11 @@ private void runBookies(List tempDirs, String dirSuffix) // If the caller specified ephemeral ports then use ephemeral ports for all // the bookies else use numBookie ports starting at initialPort + PortManager.initPort(initialPort); if (0 == initialPort) { bsConfs[i].setBookiePort(0); } else { - bsConfs[i].setBookiePort(initialPort + i); + bsConfs[i].setBookiePort(PortManager.nextFreePort()); } if (null == baseConf.getMetadataServiceUriUnchecked()) { @@ -451,10 +453,10 @@ public static void main(String[] args) { String confFile = args[1]; try { conf.loadConf(new File(confFile).toURI().toURL()); - LOG.info("Using configuration file " + confFile); + LOG.info("Using configuration file {}", confFile); } catch (Exception e) { // load conf failed - LOG.warn("Error loading configuration file " + confFile, e); + LOG.warn("Error loading configuration file {}", confFile, e); } } @@ -472,7 +474,6 @@ public static void main(String[] args) { bookieDefaultInitialPort, false, "test", zkDataDir, localBookiesConfigDirName); } catch (Exception e) { LOG.error("Exiting LocalBookKeeper because of exception in main method", e); - e.printStackTrace(); /* * This is needed because, some non-daemon thread (probably in ZK or * some other dependent service) is preventing the JVM from exiting, though diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/PortManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/PortManager.java similarity index 56% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/test/PortManager.java rename to bookkeeper-server/src/main/java/org/apache/bookkeeper/util/PortManager.java index 0a4b8ab0ef8..864aaf2f56f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/PortManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/PortManager.java @@ -18,36 +18,57 @@ * under the License. * */ -package org.apache.bookkeeper.test; +package org.apache.bookkeeper.util; +import java.io.IOException; import java.net.ServerSocket; +import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Port manager allows a base port to be specified on the commandline. * Tests will then use ports, counting up from this base port. * This allows multiple instances of the bookkeeper tests to run at once. */ public class PortManager { - private static int nextPort = getBasePort(); + private static final Logger LOG = LoggerFactory.getLogger(PortManager.class); + + private static int nextPort = 15000; + + /** + * Init the base port. + * + * @param initPort initial port + */ + public static void initPort(int initPort) { + nextPort = initPort; + } + + /** + * Return the available port. + * + * @return available port. + */ public static synchronized int nextFreePort() { int exceptionCount = 0; while (true) { int port = nextPort++; - try (ServerSocket ss = new ServerSocket(port)) { - ss.close(); - //Give it some time to truly close the connection - Thread.sleep(100); + try (ServerSocket ignored = new ServerSocket(port)) { + // Give it some time to truly close the connection + TimeUnit.MILLISECONDS.sleep(100); return port; - } catch (Exception e) { + } catch (IOException ioe) { exceptionCount++; - if (exceptionCount > 5) { - throw new RuntimeException(e); + if (exceptionCount > 100) { + throw new RuntimeException("Unable to allocate socket port", ioe); } + } catch (InterruptedException ie) { + LOG.error("Failed to allocate socket port", ie); + Thread.currentThread().interrupt(); } } } - private static int getBasePort() { - return Integer.valueOf(System.getProperty("test.basePort", "15000")); - } } diff --git a/bookkeeper-server/src/main/resources/log4j.properties b/bookkeeper-server/src/main/resources/log4j.properties new file mode 100644 index 00000000000..7703c575bff --- /dev/null +++ b/bookkeeper-server/src/main/resources/log4j.properties @@ -0,0 +1,38 @@ +# +# +# 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. +# +# + +# +# Bookkeeper Logging Configuration +# + +# Format is " (, )+ + +# 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.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n + diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java index 0f893ac7f80..72e59b1a932 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java @@ -32,8 +32,8 @@ import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.util.PortManager; import org.junit.Test; /** 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 25df88f4bd1..f57d2685216 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 @@ -110,10 +110,10 @@ import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.util.DiskChecker; import org.apache.bookkeeper.util.LoggerOutput; +import org.apache.bookkeeper.util.PortManager; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; 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 e91500f5f00..e244852f113 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 @@ -45,9 +45,9 @@ import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.util.PortManager; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java index 1ea661b72b2..9d69f3ec79f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java @@ -53,9 +53,9 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.test.ZooKeeperUtil; import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.util.PortManager; import org.apache.commons.io.FileUtils; import org.junit.After; import org.junit.Assert; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java index 0ddd9a6a61d..b49950b4730 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java @@ -42,8 +42,8 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.util.PortManager; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index 3d9e88ffb0a..afc8302b74c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -61,6 +61,7 @@ import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.util.PortManager; import org.apache.commons.io.FileUtils; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java index 37160b59471..3751401645b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java @@ -30,6 +30,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.util.PortManager; import org.junit.Test; /** diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java index 329779ce386..a154121dc1a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java @@ -35,6 +35,7 @@ import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.util.PortManager; import org.junit.Test; /** diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java index 7ccb0f09220..0c3cebfec4d 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java @@ -32,9 +32,9 @@ import org.apache.bookkeeper.metadata.etcd.EtcdMetadataBookieDriver; import org.apache.bookkeeper.metadata.etcd.EtcdMetadataClientDriver; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.util.PortManager; import org.apache.commons.io.FileUtils; import org.junit.After; import org.junit.AfterClass; From 1ba029b318dfae692f5daf70b3ad59cc9b1f95d4 Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Sun, 31 May 2020 02:02:22 +0800 Subject: [PATCH 0436/1642] [CLEAN] Fix missing Override annotation ### Motivation **Code clean** - An overridden method should be marked with Override annotation, once the method signature in the abstract class is changed, the implementation class will report a compile-time error immediately. - Exception class names must be ended with Exception - The negation operator is not easy to be quickly understood ### Changes - Fix missing `Override` annotation - `NoIndexForLedger` to `NoIndexForLedgerException` - `!(n.doubleValue() > max.doubleValue())` to `(n.doubleValue() <= max.doubleValue())` Reviewers: Matteo Minardi , Enrico Olivelli , Sijie Guo This closes #2331 from lamber-ken/bb-clean --- .../bookkeeper/benchmark/BenchBookie.java | 1 + .../benchmark/BenchThroughputLatency.java | 3 +++ .../apache/bookkeeper/benchmark/TestClient.java | 3 +++ .../common/component/LifecycleComponent.java | 1 + .../common/conf/validators/RangeValidator.java | 2 +- .../auth/AuthProviderFactoryFactory.java | 3 +++ .../apache/bookkeeper/bookie/BookieShell.java | 1 + .../bookkeeper/bookie/BookieStateManager.java | 1 + .../org/apache/bookkeeper/bookie/Cookie.java | 1 + .../apache/bookkeeper/bookie/EntryKeyValue.java | 1 + .../bookkeeper/bookie/FileSystemUpgrade.java | 2 ++ .../bookkeeper/bookie/IndexInMemPageMgr.java | 3 +++ .../bookkeeper/bookie/IndexPersistenceMgr.java | 2 +- .../org/apache/bookkeeper/bookie/Journal.java | 2 ++ .../apache/bookkeeper/bookie/LedgerCache.java | 4 ++-- .../bookkeeper/bookie/LedgerCacheImpl.java | 2 ++ .../bookkeeper/bookie/LedgerDescriptorImpl.java | 1 + .../apache/bookkeeper/client/AsyncCallback.java | 1 + .../bookkeeper/client/BookKeeperAdmin.java | 1 + .../bookkeeper/client/BookieInfoReader.java | 1 + ...efaultSpeculativeRequestExecutionPolicy.java | 2 ++ .../apache/bookkeeper/client/LedgerChecker.java | 4 ++++ .../bookkeeper/client/LedgerHandleAdv.java | 1 + .../bookkeeper/client/LedgerRecoveryOp.java | 1 + .../apache/bookkeeper/client/PendingAddOp.java | 2 ++ .../apache/bookkeeper/client/PendingReadOp.java | 1 + .../client/ReadLastConfirmedAndEntryOp.java | 1 + .../bookkeeper/client/ReadLastConfirmedOp.java | 1 + .../client/RoundRobinDistributionSchedule.java | 2 ++ .../TopologyAwareEnsemblePlacementPolicy.java | 1 + .../client/WeightedRandomSelectionImpl.java | 1 + .../client/api/LastConfirmedAndEntry.java | 1 + .../bookkeeper/client/api/LedgerEntry.java | 1 + .../meta/LedgerUnderreplicationManager.java | 1 + .../meta/ZkLedgerUnderreplicationManager.java | 3 +++ .../net/AbstractDNSToSwitchMapping.java | 2 ++ .../bookkeeper/net/NetworkTopologyImpl.java | 1 + .../bookkeeper/processor/RequestProcessor.java | 1 + .../apache/bookkeeper/proto/AuthHandler.java | 3 +++ .../bookkeeper/proto/BookieClientImpl.java | 9 +++++++++ .../apache/bookkeeper/proto/BookieProtocol.java | 3 +++ .../bookkeeper/proto/WriteEntryProcessor.java | 1 + .../apache/bookkeeper/replication/Auditor.java | 10 ++++++++++ .../bookkeeper/replication/AuditorElector.java | 3 +++ .../bookkeeper/tls/TLSContextFactory.java | 1 + .../bookkeeper/util/DaemonThreadFactory.java | 1 + .../bookkeeper/stats/CodahaleOpStatsLogger.java | 6 ++++++ .../stats/codahale/CodahaleOpStatsLogger.java | 6 ++++++ .../circe/impl/IntStatefulLongHash.java | 17 +++++++++++++++++ .../impl/internal/api/LocationClient.java | 1 + .../common/util/AutoAsyncCloseable.java | 1 + .../bookkeeper/statelib/api/StateStore.java | 1 + .../stream/storage/api/sc/StorageContainer.java | 1 + .../api/sc/StorageContainerRegistry.java | 1 + .../api/sc/StorageContainerServiceFactory.java | 1 + .../impl/service/RangeStoreServiceImpl.java | 2 ++ .../integration/utils/MavenClassLoader.java | 1 + 57 files changed, 128 insertions(+), 4 deletions(-) diff --git a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java index 94776f0443c..d91b7256097 100644 --- a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java +++ b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java @@ -83,6 +83,7 @@ public synchronized void waitForComplete() throws InterruptedException { static class ThroughputCallback implements WriteCallback { int count; int waitingCount = Integer.MAX_VALUE; + @Override public synchronized void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { if (rc != 0) { 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 3d34a773e00..c5ec3cb33ac 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 @@ -134,6 +134,7 @@ public synchronized long getDuration() { return duration; } + @Override public void run() { LOG.info("Running..."); long start = previous = System.currentTimeMillis(); @@ -141,6 +142,7 @@ public void run() { int sent = 0; Thread reporter = new Thread() { + @Override public void run() { try { while (true) { @@ -290,6 +292,7 @@ public static void main(String[] args) final long timeout = Long.parseLong(cmd.getOptionValue("timeout", "360")) * 1000; timeouter.schedule(new TimerTask() { + @Override public void run() { System.err.println("Timing out benchmark after " + timeout + "ms"); System.exit(-1); 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 050a9b97c1b..9f1b225fa20 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 @@ -103,6 +103,7 @@ public static void main(String[] args) throws ParseException { final long timeout = Long.parseLong(cmd.getOptionValue("timeout", "360")) * 1000; timeouter.schedule(new TimerTask() { + @Override public void run() { System.err.println("Timing out benchmark after " + timeout + "ms"); System.exit(-1); @@ -209,6 +210,7 @@ static class FileClient implements Callable { this.r = new Random(System.identityHashCode(this)); } + @Override public Long call() { try { long count = 0; @@ -253,6 +255,7 @@ static class BKClient implements Callable, AddCallback { this.sync = sync; } + @Override public Long call() { try { long start = System.currentTimeMillis(); diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponent.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponent.java index 836cffb1f91..660dfb0c78b 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponent.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/LifecycleComponent.java @@ -40,6 +40,7 @@ default void publishInfo(ComponentInfoPublisher componentInfoPublisher) { void stop(); + @Override void close(); /** diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java index 827f249cd55..06147d0c0e5 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/conf/validators/RangeValidator.java @@ -69,7 +69,7 @@ public boolean validate(String name, Object value) { if (min != null && n.doubleValue() < min.doubleValue()) { return false; } else { - return max == null || !(n.doubleValue() > max.doubleValue()); + return max == null || n.doubleValue() <= max.doubleValue(); } } else { return false; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/AuthProviderFactoryFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/AuthProviderFactoryFactory.java index b121a056596..25e5509e59e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/AuthProviderFactoryFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/auth/AuthProviderFactoryFactory.java @@ -75,6 +75,7 @@ public BookieAuthProvider newProvider(BookieConnectionPeer addr, AuthCallbacks.GenericCallback completeCb) { completeCb.operationComplete(BKException.Code.OK, null); return new BookieAuthProvider() { + @Override public void process(AuthToken m, AuthCallbacks.GenericCallback cb) { // any request of authentication for clients is going to be answered with a standard response // the client will d @@ -100,7 +101,9 @@ public ClientAuthProvider newProvider(ClientConnectionPeer addr, addr.setAuthorizedId(BookKeeperPrincipal.ANONYMOUS); completeCb.operationComplete(BKException.Code.OK, null); return new ClientAuthProvider() { + @Override public void init(AuthCallbacks.GenericCallback cb) {} + @Override public void process(AuthToken m, AuthCallbacks.GenericCallback cb) {} }; } 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 00c99a7ee25..0ba984c7b0b 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 @@ -212,6 +212,7 @@ abstract class MyCommand implements Command { this.cmdName = cmdName; } + @Override public String description() { // we used the string returned by `getUsage` as description in showing the list of commands return getUsage(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java index 45b48fb4d83..b848c63171c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java @@ -347,6 +347,7 @@ public void doTransitionToReadOnlyMode() { return; } } + @Override public void setShutdownHandler(ShutdownHandler handler){ shutdownHandler = handler; } 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 d6711aef821..9a6eadfb77e 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 @@ -151,6 +151,7 @@ public void verifyIsSuperSet(Cookie c) throws BookieException.InvalidCookieExcep verifyInternal(c, true); } + @Override public String toString() { if (layoutVersion <= 3) { return toStringVersion3(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryKeyValue.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryKeyValue.java index 43e35fbbaa4..c949a3d4dfb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryKeyValue.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryKeyValue.java @@ -109,6 +109,7 @@ int writeToByteBuffer(ByteBuffer dst) { /** * String representation. */ + @Override public String toString() { return ledgerId + ":" + entryId; } 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 3686bef3d3b..3ba3675e7b5 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 @@ -86,6 +86,7 @@ private boolean containsIndexFiles(File dir, String name) { return false; } + @Override public boolean accept(File dir, String name) { if (name.endsWith(".txn") || name.endsWith(".log") || name.equals("lastId") || name.startsWith("lastMark")) { @@ -195,6 +196,7 @@ private static void upgrade(ServerConfiguration conf, c.writeToDirectory(tmpDir); String[] files = d.list(new FilenameFilter() { + @Override public boolean accept(File dir, String name) { return bookieFilesFilter.accept(dir, name) && !(new File(dir, name).isDirectory()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java index 0b77dd771cf..9abc7b6fc01 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexInMemPageMgr.java @@ -608,14 +608,17 @@ private class PageEntriesImpl implements LedgerCache.PageEntries { this.initEntry = initEntry; } + @Override public LedgerEntryPage getLEP() throws IOException { return getLedgerEntryPage(ledgerId, initEntry); } + @Override public long getFirstEntry() { return initEntry; } + @Override public long getLastEntry() { return initEntry + entriesPerPage; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java index c98bc6ee8ff..dfcb4fba198 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java @@ -213,7 +213,7 @@ CachedFileInfo getFileInfo(final Long ledger, final byte[] masterKey) throws IOE if (ee.getCause() instanceof IOException) { throw (IOException) ee.getCause(); } else { - throw new LedgerCache.NoIndexForLedger("Failed to load file info for ledger " + ledger, ee); + throw new LedgerCache.NoIndexForLedgerException("Failed to load file info for ledger " + ledger, ee); } } finally { persistenceMgrStats.getPendingGetFileInfoCounter().dec(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index 4d6d2e839cb..1eec6a30763 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -342,6 +342,7 @@ private QueueEntry(Handle recyclerHandle) { } private static final Recycler RECYCLER = new Recycler() { + @Override protected QueueEntry newObject(Recycler.Handle handle) { return new QueueEntry(handle); } @@ -448,6 +449,7 @@ private ForceWriteRequest createForceWriteRequest(JournalChannel logFile, } private final Recycler forceWriteRequestsRecycler = new Recycler() { + @Override protected ForceWriteRequest newObject( Recycler.Handle handle) { return new ForceWriteRequest(handle); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java index 606afb46e73..e9b9f3bd7eb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java @@ -66,8 +66,8 @@ void cancelWaitForLastAddConfirmedUpdate(long ledgerId, /** * Specific exception to encode the case where the index is not present. */ - class NoIndexForLedger extends IOException { - NoIndexForLedger(String reason, Exception cause) { + class NoIndexForLedgerException extends IOException { + NoIndexForLedgerException(String reason, Exception cause) { super(reason, cause); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java index 7341ace65c7..d744ee46a14 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCacheImpl.java @@ -159,10 +159,12 @@ public boolean isFenced(long ledgerId) throws IOException { return indexPersistenceManager.isFenced(ledgerId); } + @Override public void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException { indexPersistenceManager.setExplicitLac(ledgerId, lac); } + @Override public ByteBuf getExplicitLac(long ledgerId) { return indexPersistenceManager.getExplicitLac(ledgerId); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java index 34c11bcacfb..77fafa1595b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java @@ -87,6 +87,7 @@ ByteBuf getExplicitLac() { return ledgerStorage.getExplicitLac(ledgerId); } + @Override synchronized SettableFuture fenceAndLogInJournal(Journal journal) throws IOException { boolean success = this.setFenced(); if (success) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/AsyncCallback.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/AsyncCallback.java index cde3f06aefb..a742c2db513 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/AsyncCallback.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/AsyncCallback.java @@ -87,6 +87,7 @@ interface AddCallback extends AddCallbackWithLatency { * @param ctx * context object */ + @Override default void addCompleteWithLatency(int rc, LedgerHandle lh, long entryId, long qwcLatency, Object ctx) { addComplete(rc, lh, entryId, ctx); } 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 ea6cd9fbf17..3cc50a42c43 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 @@ -1345,6 +1345,7 @@ public Iterable listLedgers() throws IOException { final LedgerRangeIterator iterator = bkc.getLedgerManager().getLedgerRanges(0); return new Iterable() { + @Override public Iterator iterator() { return new Iterator() { Iterator currentRange = null; 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 112cd4821e1..c0d412c3fab 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 @@ -79,6 +79,7 @@ public long getTotalDiskSpace() { public long getWeight() { return freeDiskSpace; } + @Override public String toString() { return "FreeDiskSpace: " + this.freeDiskSpace + " TotalDiskCapacity: " + this.totalDiskSpace; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java index b2874e5d67f..d2601d39a73 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java @@ -83,6 +83,7 @@ public void run() { ListenableFuture issueNextRequest = requestExecutor.issueSpeculativeRequest(); Futures.addCallback(issueNextRequest, new FutureCallback() { // we want this handler to run immediately after we push the big red button! + @Override public void onSuccess(Boolean issueNextRequest) { if (issueNextRequest) { scheduleSpeculativeRead(scheduler, requestExecutor, @@ -96,6 +97,7 @@ public void onSuccess(Boolean issueNextRequest) { } } + @Override public void onFailure(Throwable thrown) { LOG.warn("Failed to issue speculative request for {}, speculativeReadTimeout = {} : ", requestExecutor, speculativeRequestTimeout, thrown); 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 fe6370b9926..6cc23cc2699 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 @@ -72,6 +72,7 @@ private static class ReadManyEntriesCallback implements ReadEntryCallback { this.cb = cb; } + @Override public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf buffer, Object ctx) { if (rc == BKException.Code.OK) { @@ -262,6 +263,7 @@ private static class EntryExistsCallback implements ReadEntryCallback { this.cb = cb; } + @Override public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf buffer, Object ctx) { if (BKException.Code.NoSuchEntryException != rc && BKException.Code.NoSuchLedgerExistsException != rc @@ -293,6 +295,7 @@ private static class FullLedgerCallback implements this.cb = cb; } + @Override public void operationComplete(int rc, LedgerFragment result) { if (rc == BKException.Code.ClientClosedException) { cb.operationComplete(BKException.Code.ClientClosedException, badFragments); @@ -371,6 +374,7 @@ public void checkLedger(final LedgerHandle lh, final EntryExistsCallback eecb = new EntryExistsCallback(lh.getLedgerMetadata().getWriteQuorumSize(), new GenericCallback() { + @Override public void operationComplete(int rc, Boolean result) { if (result) { fragments.add(lastLedgerFragment); 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 8b098b9fa4b..2ea0e0a408f 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 @@ -52,6 +52,7 @@ public class LedgerHandleAdv extends LedgerHandle implements WriteAdvHandle { static final Logger LOG = LoggerFactory.getLogger(LedgerHandleAdv.class); static class PendingOpsComparator implements Comparator, Serializable { + @Override public int compare(PendingAddOp o1, PendingAddOp o2) { return Long.compare(o1.entryId, o2.entryId); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java index bd19a6b5fab..534ea83b134 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java @@ -100,6 +100,7 @@ public CompletableFuture initiate() { lh.getCurrentEnsemble(), lh.ledgerKey, new ReadLastConfirmedOp.LastConfirmedDataCallback() { + @Override public void readLastConfirmedDataComplete(int rc, RecoveryData data) { if (rc == BKException.Code.OK) { synchronized (lh) { 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 ff806155e65..cd1a03fd555 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 @@ -245,6 +245,7 @@ void unsetSuccessAndSendWriteRequest(List ensemble, int boo /** * Initiate the add operation. */ + @Override public void safeRun() { hasRun = true; if (callbackTriggered) { @@ -457,6 +458,7 @@ public boolean equals(Object o) { private final Handle recyclerHandle; private static final Recycler RECYCLER = new Recycler() { + @Override protected PendingAddOp newObject(Recycler.Handle handle) { return new PendingAddOp(handle); } 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 22c0fc15b38..bfd9a5d771b 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 @@ -108,6 +108,7 @@ abstract class LedgerEntryRequest implements SpeculativeRequestExecutor, AutoClo } } + @Override public void close() { entryImpl.close(); } 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 ed1bece65f2..3dc1a75fc93 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 @@ -93,6 +93,7 @@ abstract class ReadLACAndEntryRequest implements AutoCloseable { } } + @Override public void close() { entryImpl.close(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java index e39bf60237f..4e3b1277bfa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java @@ -95,6 +95,7 @@ public void initiateWithFencing() { } } + @Override public synchronized void readEntryComplete(final int rc, final long ledgerId, final long entryId, final ByteBuf buffer, final Object ctx) { int bookieIndex = (Integer) ctx; 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 e4ae4f264e7..85615b84a0a 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 @@ -79,6 +79,7 @@ private static class WriteSetImpl implements WriteSet { private final Handle recyclerHandle; private static final Recycler RECYCLER = new Recycler() { + @Override protected WriteSetImpl newObject( Recycler.Handle handle) { return new WriteSetImpl(handle); @@ -269,6 +270,7 @@ private static class AckSetImpl implements AckSet { private final Handle recyclerHandle; private static final Recycler RECYCLER = new Recycler() { + @Override protected AckSetImpl newObject(Recycler.Handle handle) { return new AckSetImpl(handle); } 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 0b84b78f4fc..2e238487814 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 @@ -542,6 +542,7 @@ static class DNSResolverDecorator implements DNSToSwitchMapping { this.failedToResolveNetworkLocationCounter = failedToResolveNetworkLocationCounter; } + @Override public List resolve(List names) { if (names == null) { return Collections.emptyList(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java index 53130a22675..40ad35e5c20 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java @@ -56,6 +56,7 @@ public void updateMap(Map map) { Long totalWeight = 0L, min = Long.MAX_VALUE; List values = new ArrayList(map.values()); Collections.sort(values, new Comparator() { + @Override public int compare(WeightedObject o1, WeightedObject o2) { long diff = o1.getWeight() - o2.getWeight(); if (diff < 0L) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LastConfirmedAndEntry.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LastConfirmedAndEntry.java index 8bbe58e21a7..159efad5852 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LastConfirmedAndEntry.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LastConfirmedAndEntry.java @@ -50,6 +50,7 @@ public interface LastConfirmedAndEntry extends AutoCloseable { /** * {@inheritDoc} */ + @Override void close(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerEntry.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerEntry.java index 0f54961e45c..411ac9b4cf6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerEntry.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerEntry.java @@ -98,6 +98,7 @@ public interface LedgerEntry extends AutoCloseable { /** * {@inheritDoc} */ + @Override void close(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerUnderreplicationManager.java index 9f2d0cb5111..0718fb5366a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerUnderreplicationManager.java @@ -115,6 +115,7 @@ void releaseUnderreplicatedLedger(long ledgerId) /** * Release all resources held by the ledger underreplication manager. */ + @Override void close() throws ReplicationException.UnavailableException; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java index 554973273b3..6376d720e89 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java @@ -587,6 +587,7 @@ public long getLedgerToRereplicate() throws ReplicationException.UnavailableExce while (true) { final CountDownLatch changedLatch = new CountDownLatch(1); Watcher w = new Watcher() { + @Override public void process(WatchedEvent e) { if (e.getType() == Watcher.Event.EventType.NodeChildrenChanged || e.getType() == Watcher.Event.EventType.NodeDeleted @@ -742,6 +743,7 @@ public void notifyLedgerReplicationEnabled(final GenericCallback cb) LOG.debug("notifyLedgerReplicationEnabled()"); } Watcher w = new Watcher() { + @Override public void process(WatchedEvent e) { if (e.getType() == Watcher.Event.EventType.NodeDeleted) { LOG.info("LedgerReplication is enabled externally through Zookeeper, " @@ -858,6 +860,7 @@ public int getLostBookieRecoveryDelay() throws UnavailableException { public void notifyLostBookieRecoveryDelayChanged(GenericCallback cb) throws UnavailableException { LOG.debug("notifyLostBookieRecoveryDelayChanged()"); Watcher w = new Watcher() { + @Override public void process(WatchedEvent e) { if (e.getType() == Watcher.Event.EventType.NodeDataChanged) { cb.operationComplete(0, null); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java index 77e64c0b83b..a0ee42295be 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/AbstractDNSToSwitchMapping.java @@ -59,10 +59,12 @@ protected AbstractDNSToSwitchMapping(Configuration conf) { this.conf = conf; } + @Override public Configuration getConf() { return conf; } + @Override public void setConf(Configuration conf) { this.conf = conf; validateConf(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java index 0b8a02b74ce..52b33259800 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java @@ -399,6 +399,7 @@ public NetworkTopologyImpl() { * @exception IllegalArgumentException if add a node to a leave or node to be added is not a leaf */ + @Override public void add(Node node) { if (node == null) { return; 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 8b328ef7db0..a7b57990da6 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 @@ -30,6 +30,7 @@ public interface RequestProcessor extends AutoCloseable { /** * Close the request processor. */ + @Override void close(); /** 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 52f855f0286..ad40a886cf3 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 @@ -166,6 +166,7 @@ static class AuthResponseCallbackLegacy implements AuthCallbacks.GenericCallback this.channel = channel; } + @Override public void operationComplete(int rc, AuthToken newam) { if (rc != BKException.Code.OK) { LOG.error("Error processing auth message, closing connection"); @@ -189,6 +190,7 @@ static class AuthResponseCallback implements AuthCallbacks.GenericCallback { this.pluginName = pluginName; } + @Override public void operationComplete(int rc, AuthToken newam) { if (rc != BKException.Code.OK) { authenticationError(ctx, rc); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index f81650b4087..63c70fc76cc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -432,6 +432,7 @@ private ChannelReadyForAddEntryCallback( private static final Recycler RECYCLER = new Recycler() { + @Override protected ChannelReadyForAddEntryCallback newObject( Recycler.Handle recyclerHandle) { return new ChannelReadyForAddEntryCallback(recyclerHandle); @@ -479,16 +480,19 @@ public void readLac(final BookieSocketAddress addr, final long ledgerId, final R }, ledgerId, useV3Enforced); } + @Override public void readEntry(BookieSocketAddress addr, long ledgerId, long entryId, ReadEntryCallback cb, Object ctx, int flags) { readEntry(addr, ledgerId, entryId, cb, ctx, flags, null); } + @Override public void readEntry(final BookieSocketAddress addr, final long ledgerId, final long entryId, final ReadEntryCallback cb, final Object ctx, int flags, byte[] masterKey) { readEntry(addr, ledgerId, entryId, cb, ctx, flags, masterKey, false); } + @Override public void readEntry(final BookieSocketAddress addr, final long ledgerId, final long entryId, final ReadEntryCallback cb, final Object ctx, int flags, byte[] masterKey, final boolean allowFastFail) { @@ -509,6 +513,7 @@ public void readEntry(final BookieSocketAddress addr, final long ledgerId, final } + @Override public void readEntryWaitForLACUpdate(final BookieSocketAddress addr, final long ledgerId, final long entryId, @@ -534,6 +539,7 @@ public void readEntryWaitForLACUpdate(final BookieSocketAddress addr, }, ledgerId); } + @Override public void getBookieInfo(final BookieSocketAddress addr, final long requested, final GetBookieInfoCallback cb, final Object ctx) { final PerChannelBookieClientPool client = lookupClient(addr); @@ -564,10 +570,12 @@ private void monitorPendingOperations() { } } + @Override public boolean isClosed() { return closed; } + @Override public void close() { closeLock.writeLock().lock(); try { @@ -624,6 +632,7 @@ public static void main(String[] args) throws NumberFormatException, IOException } WriteCallback cb = new WriteCallback() { + @Override public void writeComplete(int rc, long ledger, long entry, BookieSocketAddress addr, Object ctx) { Counter counter = (Counter) ctx; counter.dec(); 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 9982cca712a..93e8219c49e 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 @@ -282,6 +282,7 @@ private AddRequest(Handle recyclerHandle) { } private static final Recycler RECYCLER = new Recycler() { + @Override protected AddRequest newObject(Handle handle) { return new AddRequest(handle); } @@ -336,6 +337,7 @@ private ParsedAddRequest(Handle recyclerHandle) { } private static final Recycler RECYCLER = new Recycler() { + @Override protected ParsedAddRequest newObject(Handle handle) { return new ParsedAddRequest(handle); } @@ -486,6 +488,7 @@ private AddResponse(Handle recyclerHandle) { } private static final Recycler RECYCLER = new Recycler() { + @Override protected AddResponse newObject(Handle handle) { return new AddResponse(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 70db7ce4919..c8af9f87d93 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 @@ -43,6 +43,7 @@ class WriteEntryProcessor extends PacketProcessorBase implemen long startTimeNanos; + @Override protected void reset() { super.reset(); startTimeNanos = -1L; 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 7e815de350d..eac25b2c678 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 @@ -495,6 +495,7 @@ private void submitShutdownTask() { return; } executor.submit(safeRun(new Runnable() { + @Override public void run() { synchronized (Auditor.this) { LOG.info("Shutting down Auditor's Executor"); @@ -513,6 +514,7 @@ synchronized Future submitAuditTask() { return f; } return executor.submit(safeRun(new Runnable() { + @Override @SuppressWarnings("unchecked") public void run() { try { @@ -569,6 +571,7 @@ public void run() { if (auditTask == null) { // if there is no scheduled audit, schedule one auditTask = executor.schedule(safeRun(new Runnable() { + @Override public void run() { startAudit(false); auditTask = null; @@ -599,6 +602,7 @@ synchronized Future submitLostBookieRecoveryDelayChangedEvent() { } return executor.submit(safeRun(new Runnable() { int lostBookieRecoveryDelay = -1; + @Override public void run() { try { waitIfLedgerReplicationDisabled(); @@ -628,6 +632,7 @@ public void run() { LOG.info("lostBookieRecoveryDelay has been set to {}, so rescheduling AuditTask accordingly", lostBookieRecoveryDelay); auditTask = executor.schedule(safeRun(new Runnable() { + @Override public void run() { startAudit(false); auditTask = null; @@ -730,6 +735,7 @@ private void scheduleCheckAllLedgersTask(){ checkAllLedgersLastExecutedCTime, durationSinceLastExecutionInSecs, initialDelay, interval); executor.scheduleAtFixedRate(safeRun(new Runnable() { + @Override public void run() { try { if (!ledgerUnderreplicationManager.isLedgerReplicationEnabled()) { @@ -797,6 +803,7 @@ private void schedulePlacementPolicyCheckTask(){ placementPolicyCheckLastExecutedCTime, durationSinceLastExecutionInSecs, initialDelay, interval); executor.scheduleAtFixedRate(safeRun(new Runnable() { + @Override public void run() { try { Stopwatch stopwatch = Stopwatch.createStarted(); @@ -918,6 +925,7 @@ private void scheduleReplicasCheckTask() { replicasCheckLastExecutedCTime, durationSinceLastExecutionInSecs, initialDelay, interval); executor.scheduleAtFixedRate(safeRun(new Runnable() { + @Override public void run() { try { Stopwatch stopwatch = Stopwatch.createStarted(); @@ -1141,6 +1149,7 @@ private class ProcessLostFragmentsCb implements GenericCallback fragments) { if (rc == BKException.Code.OK) { Set bookies = Sets.newHashSet(); @@ -1958,6 +1967,7 @@ public boolean isRunning() { } private final Runnable bookieCheck = new Runnable() { + @Override public void run() { if (auditTask == null) { startAudit(true); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java index b23b0b0211d..e644369ed7d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java @@ -252,6 +252,7 @@ public Future start() { */ private void submitShutdownTask() { executor.submit(new Runnable() { + @Override public void run() { if (!running.compareAndSet(true, false)) { return; @@ -281,6 +282,7 @@ public void run() { Future submitElectionTask() { Runnable r = new Runnable() { + @Override public void run() { if (!running.get()) { return; @@ -425,6 +427,7 @@ private static class ElectionComparator * Return -1 if the first vote is less than second. Return 1 if the * first vote is greater than second. Return 0 if the votes are equal. */ + @Override public int compare(String vote1, String vote2) { long voteSeqId1 = getVoteSequenceId(vote1); long voteSeqId2 = getVoteSequenceId(vote2); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java index 1b6b30e1c76..2a9f6df7486 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java @@ -118,6 +118,7 @@ private KeyStore loadKeyStore(String keyStoreType, String keyStoreLocation, Stri return ks; } + @Override public String getHandlerName() { return TLSCONTEXT_HANDLER_NAME; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DaemonThreadFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DaemonThreadFactory.java index cb8caae5751..4c6d445a236 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DaemonThreadFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/DaemonThreadFactory.java @@ -33,6 +33,7 @@ public DaemonThreadFactory(int priority) { assert priority >= Thread.MIN_PRIORITY && priority <= Thread.MAX_PRIORITY; this.priority = priority; } + @Override public Thread newThread(Runnable r) { Thread thread = defaultThreadFactory.newThread(r); thread.setDaemon(true); diff --git a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/CodahaleOpStatsLogger.java b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/CodahaleOpStatsLogger.java index 690f2b51dce..f6ee602ab9c 100644 --- a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/CodahaleOpStatsLogger.java +++ b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/CodahaleOpStatsLogger.java @@ -32,24 +32,29 @@ class CodahaleOpStatsLogger implements OpStatsLogger { } // OpStatsLogger functions + @Override public void registerFailedEvent(long eventLatency, TimeUnit unit) { fail.update(eventLatency, unit); } + @Override public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) { success.update(eventLatency, unit); } + @Override public void registerSuccessfulValue(long value) { // Values are inserted as millis, which is the unit they will be presented, to maintain 1:1 scale success.update(value, TimeUnit.MILLISECONDS); } + @Override public void registerFailedValue(long value) { // Values are inserted as millis, which is the unit they will be presented, to maintain 1:1 scale fail.update(value, TimeUnit.MILLISECONDS); } + @Override public synchronized void clear() { // can't clear a timer } @@ -57,6 +62,7 @@ public synchronized void clear() { /** * This function should go away soon (hopefully). */ + @Override public synchronized OpStatsData toOpStatsData() { long numFailed = fail.getCount(); long numSuccess = success.getCount(); diff --git a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/CodahaleOpStatsLogger.java b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/CodahaleOpStatsLogger.java index ae97c7858aa..1e8f8688604 100644 --- a/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/CodahaleOpStatsLogger.java +++ b/bookkeeper-stats-providers/codahale-metrics-provider/src/main/java/org/apache/bookkeeper/stats/codahale/CodahaleOpStatsLogger.java @@ -33,24 +33,29 @@ class CodahaleOpStatsLogger implements OpStatsLogger { } // OpStatsLogger functions + @Override public void registerFailedEvent(long eventLatency, TimeUnit unit) { fail.update(eventLatency, unit); } + @Override public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) { success.update(eventLatency, unit); } + @Override public void registerSuccessfulValue(long value) { // Values are inserted as millis, which is the unit they will be presented, to maintain 1:1 scale success.update(value, TimeUnit.MILLISECONDS); } + @Override public void registerFailedValue(long value) { // Values are inserted as millis, which is the unit they will be presented, to maintain 1:1 scale fail.update(value, TimeUnit.MILLISECONDS); } + @Override public synchronized void clear() { // can't clear a timer } @@ -58,6 +63,7 @@ public synchronized void clear() { /** * This function should go away soon (hopefully). */ + @Override public synchronized OpStatsData toOpStatsData() { long numFailed = fail.getCount(); long numSuccess = success.getCount(); diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/impl/IntStatefulLongHash.java b/circe-checksum/src/main/java/com/scurrilous/circe/impl/IntStatefulLongHash.java index 3b9a3acd5af..9282691a5ad 100644 --- a/circe-checksum/src/main/java/com/scurrilous/circe/impl/IntStatefulLongHash.java +++ b/circe-checksum/src/main/java/com/scurrilous/circe/impl/IntStatefulLongHash.java @@ -39,70 +39,87 @@ public IntStatefulLongHash(StatefulIntHash intHash) { this.intHash = intHash; } + @Override public StatelessLongHash asStateless() { return new IntStatelessLongHash(intHash.asStateless()); } + @Override public String algorithm() { return intHash.algorithm(); } + @Override public int length() { return intHash.length(); } + @Override public StatefulHash createNew() { return intHash.createNew(); } + @Override public boolean supportsUnsafe() { return intHash.supportsUnsafe(); } + @Override public boolean supportsIncremental() { return intHash.supportsIncremental(); } + @Override public void reset() { intHash.reset(); } + @Override public void update(byte[] input) { intHash.update(input); } + @Override public void update(byte[] input, int index, int length) { intHash.update(input, index, length); } + @Override public void update(ByteBuffer input) { intHash.update(input); } + @Override public void update(long address, long length) { intHash.update(address, length); } + @Override public byte[] getBytes() { return intHash.getBytes(); } + @Override public int getBytes(byte[] output, int index, int maxLength) { return intHash.getBytes(output, index, maxLength); } + @Override public byte getByte() { return intHash.getByte(); } + @Override public short getShort() { return intHash.getShort(); } + @Override public int getInt() { return intHash.getInt(); } + @Override public long getLong() { return intHash.getLong(); } diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/api/LocationClient.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/api/LocationClient.java index 41b7ba38f77..193acc47732 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/api/LocationClient.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/api/LocationClient.java @@ -37,6 +37,7 @@ public interface LocationClient extends AutoCloseable { CompletableFuture> locateStorageContainers( List> storageContainerIds); + @Override void close(); } diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/util/AutoAsyncCloseable.java b/stream/common/src/main/java/org/apache/bookkeeper/common/util/AutoAsyncCloseable.java index febcd09c5c1..db08b488fb2 100644 --- a/stream/common/src/main/java/org/apache/bookkeeper/common/util/AutoAsyncCloseable.java +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/util/AutoAsyncCloseable.java @@ -33,6 +33,7 @@ public interface AutoAsyncCloseable extends AsyncCloseable, AutoCloseable { */ boolean isClosed(); + @Override default void close() { try { FutureUtils.result(closeAsync()); diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/api/StateStore.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/api/StateStore.java index 087e00b04cc..3c0025b7745 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/api/StateStore.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/api/StateStore.java @@ -69,6 +69,7 @@ default void checkpoint() { /** * Close the state store. */ + @Override void close(); } diff --git a/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainer.java b/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainer.java index 83c5dcd3be7..e1c8af25803 100644 --- a/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainer.java +++ b/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainer.java @@ -59,6 +59,7 @@ public interface StorageContainer extends AutoCloseable { /** * Close a storage container. */ + @Override void close(); } diff --git a/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainerRegistry.java b/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainerRegistry.java index a05106058ed..28d9d039ff1 100644 --- a/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainerRegistry.java +++ b/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainerRegistry.java @@ -74,5 +74,6 @@ default CompletableFuture stopStorageContainer(long scId) { /** * Close the registry. */ + @Override void close(); } diff --git a/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainerServiceFactory.java b/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainerServiceFactory.java index e15f7d78ed8..33af0d99ea9 100644 --- a/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainerServiceFactory.java +++ b/stream/storage/api/src/main/java/org/apache/bookkeeper/stream/storage/api/sc/StorageContainerServiceFactory.java @@ -34,6 +34,7 @@ public interface StorageContainerServiceFactory extends AutoCloseable { /** * {@inheritDoc} */ + @Override void close(); } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java index 5fe1612f657..2da9057b1f1 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java @@ -167,6 +167,7 @@ private CompletableFuture startMetaRangeStore(long scId) { }); } + @Override public CompletableFuture start() { List> futures = Lists.newArrayList( startRootRangeStore(), @@ -175,6 +176,7 @@ public CompletableFuture start() { return FutureUtils.collect(futures).thenApply(ignored -> null); } + @Override public CompletableFuture stop() { return storeFactory.closeStores(scId); } diff --git a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/integration/utils/MavenClassLoader.java b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/integration/utils/MavenClassLoader.java index 3c1d13fc31c..3c83f565a77 100644 --- a/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/integration/utils/MavenClassLoader.java +++ b/tests/integration-tests-utils/src/main/java/org/apache/bookkeeper/tests/integration/utils/MavenClassLoader.java @@ -183,6 +183,7 @@ public Object digestType(String type) throws Exception { throw new ClassNotFoundException("No such digest type " + type); } + @Override public void close() throws Exception { classloader.close(); } From 21419e6d52a621487805e87ea5d3f2f48847f6df Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Mon, 1 Jun 2020 21:24:07 +0800 Subject: [PATCH 0437/1642] [BUG] Fix class not found compilation failure ### Motivation Fix compilation failure after merged https://github.com/apache/bookkeeper/pull/2338 ![image](https://user-images.githubusercontent.com/20113411/83361037-3a8f4800-a3b8-11ea-9691-ea2842a18cfd.png) ### Changes Fix wrong package location. Reviewers: Matteo Minardi , Enrico Olivelli This closes #2351 from lamber-ken/misscleass --- .../org/apache/bookkeeper/bookie/BookieStateManager.java | 2 +- .../java/org/apache/bookkeeper/bookie/EntryMemTable.java | 2 +- .../main/java/org/apache/bookkeeper/bookie/FileInfo.java | 2 +- .../apache/bookkeeper/bookie/LedgerDescriptorImpl.java | 2 +- .../storage/ldb/SingleDirectoryDbLedgerStorage.java | 2 +- .../org/apache/bookkeeper/client/BookKeeperAdmin.java | 2 +- .../apache/bookkeeper/client/ExplicitLacFlushPolicy.java | 3 +-- .../java/org/apache/bookkeeper/meta/LedgerLayout.java | 6 ++++-- .../org/apache/bookkeeper/net/ScriptBasedMapping.java | 2 +- .../org/apache/bookkeeper/proto/BookieClientImpl.java | 2 +- .../apache/bookkeeper/proto/BookieRequestProcessor.java | 2 +- .../server/http/service/DecommissionService.java | 2 +- .../org/apache/bookkeeper/tls/TLSContextFactory.java | 3 ++- .../cli/commands/bookie/ReadLogMetadataCommand.java | 2 +- .../util/collections/ConcurrentOpenHashSet.java | 2 +- .../apache/bookkeeper/client/BookKeeperAdminTest.java | 2 +- .../src/main/java/com/scurrilous/circe/Hashes.java | 9 --------- .../org/apache/bookkeeper/proto/ProtocolBenchmark.java | 1 - .../bookkeeper/proto/checksum/DigestTypeBenchmark.java | 4 +++- .../apache/bookkeeper/stats/StatsLoggerBenchmark.java | 4 ---- .../apache/bookkeeper/stats/codahale/TimerBenchmark.java | 2 -- 21 files changed, 23 insertions(+), 35 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java index b848c63171c..54647457745 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java @@ -229,7 +229,7 @@ public Void call() throws IOException { shutdownHandler.shutdown(ExitCode.ZK_REG_FAIL); } } - return (Void) null; + return null; } }); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java index 941906c045d..f54d73a67cb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryMemTable.java @@ -47,7 +47,7 @@ * flusher reports in that the flush succeeded. At that point we let the snapshot go. */ public class EntryMemTable implements AutoCloseable{ - private static Logger logger = LoggerFactory.getLogger(Journal.class); + private static Logger logger = LoggerFactory.getLogger(EntryMemTable.class); /** * Entry skip list. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java index 09b751e545b..acef43f0bb1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java @@ -309,7 +309,7 @@ private synchronized void checkOpen(boolean create, boolean openBeforeClose) try { readHeader(); } catch (BufferUnderflowException buf) { - LOG.warn("Exception when reading header of {} : {}", lf, buf); + LOG.warn("Exception when reading header of {}.", lf, buf); if (null != masterKey) { LOG.warn("Attempting to write header of {} again.", lf); writeHeader(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java index 77fafa1595b..fcad36ecc25 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java @@ -37,7 +37,7 @@ * to write entries to a ledger and read entries from a ledger. */ public class LedgerDescriptorImpl extends LedgerDescriptor { - private static final Logger LOG = LoggerFactory.getLogger(LedgerDescriptor.class); + private static final Logger LOG = LoggerFactory.getLogger(LedgerDescriptorImpl.class); final LedgerStorage ledgerStorage; private long ledgerId; final byte[] masterKey; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index a6ae8c5bd89..60e3d648cdc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -890,7 +890,7 @@ public interface LedgerLoggerProcessor { void process(long entryId, long entryLogId, long position); } - private static final Logger log = LoggerFactory.getLogger(DbLedgerStorage.class); + private static final Logger log = LoggerFactory.getLogger(SingleDirectoryDbLedgerStorage.class); @Override public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { 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 3cc50a42c43..c9cd86fd81c 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 @@ -432,7 +432,7 @@ public boolean hasNext() { close(); return false; } - LOG.error("Error reading entry {} from ledger {}", new Object[] { nextEntryId, ledgerId }, e); + LOG.error("Error reading entry {} from ledger {}", nextEntryId, ledgerId, e); close(); throw new RuntimeException(e); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ExplicitLacFlushPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ExplicitLacFlushPolicy.java index c9e6def287e..e3a333cd53e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ExplicitLacFlushPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ExplicitLacFlushPolicy.java @@ -124,8 +124,7 @@ public String toString() { scheduledFuture = clientCtx.getScheduler().scheduleAtFixedRateOrdered(lh.getId(), updateLacTask, explicitLacIntervalInMs, explicitLacIntervalInMs, TimeUnit.MILLISECONDS); } catch (RejectedExecutionException re) { - LOG.error("Scheduling of ExplictLastAddConfirmedFlush for ledger: {} has failed because of {}", - lh.getId(), re); + LOG.error("Scheduling of ExplictLastAddConfirmedFlush for ledger: {} has failed.", lh.getId(), re); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java index 09f3a708e6b..f0ee03c9ae4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerLayout.java @@ -18,6 +18,8 @@ package org.apache.bookkeeper.meta; import java.io.IOException; +import java.nio.charset.StandardCharsets; + import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.ToString; @@ -82,7 +84,7 @@ public byte[] serialize() throws IOException { if (log.isDebugEnabled()) { log.debug("Serialized layout info: {}", s); } - return s.getBytes("UTF-8"); + return s.getBytes(StandardCharsets.UTF_8); } /** @@ -95,7 +97,7 @@ public byte[] serialize() throws IOException { * if the given byte[] cannot be parsed */ public static LedgerLayout parseLayout(byte[] bytes) throws IOException { - String layout = new String(bytes, "UTF-8"); + String layout = new String(bytes, StandardCharsets.UTF_8); if (log.isDebugEnabled()) { log.debug("Parsing Layout: {}", layout); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java index 890bcfcde64..467bacaf3ee 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/ScriptBasedMapping.java @@ -130,7 +130,7 @@ public void setConf(Configuration conf) { private static final class RawScriptBasedMapping extends AbstractDNSToSwitchMapping { private String scriptName; private int maxArgs; //max hostnames per call of the script - private static final Logger LOG = LoggerFactory.getLogger(ScriptBasedMapping.class); + private static final Logger LOG = LoggerFactory.getLogger(RawScriptBasedMapping.class); /* * extract 'scriptName' and 'maxArgs' parameters from the conf and throw diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index 63c70fc76cc..8fb0f1d1b56 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -78,7 +78,7 @@ * */ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFactory { - static final Logger LOG = LoggerFactory.getLogger(BookieClient.class); + static final Logger LOG = LoggerFactory.getLogger(BookieClientImpl.class); private final OrderedExecutor executor; private final ScheduledExecutorService scheduler; 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 f46d19c8097..82f9f2a660b 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 @@ -567,7 +567,7 @@ public void operationComplete(Future future) throws Exception { if (future.isSuccess()) { LOG.info("Session is protected by: {}", sslHandler.engine().getSession().getCipherSuite()); } else { - LOG.error("TLS Handshake failure: {}", future.cause()); + LOG.error("TLS Handshake failure.", future.cause()); BookkeeperProtocol.Response.Builder errResponse = BookkeeperProtocol.Response.newBuilder() .setHeader(r.getHeader()).setStatus(BookkeeperProtocol.StatusCode.EIO); c.writeAndFlush(errResponse.build()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java index 41f18cb1f04..5295bb78a0b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java @@ -84,7 +84,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { bka.decommissionBookie(bookieSrc); LOG.info("Complete decommissioning bookie."); } catch (Exception e) { - LOG.error("Error handling decommissionBookie: {} with exception {}", bookieSrc, e); + LOG.error("Error handling decommissionBookie: {}.", bookieSrc, e); } }); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java index 2a9f6df7486..a9734862429 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java @@ -32,6 +32,7 @@ import java.io.File; import java.io.FileInputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.security.KeyStore; import java.security.KeyStoreException; import java.security.NoSuchAlgorithmException; @@ -102,7 +103,7 @@ private String getPasswordFromFile(String path) throws IOException { return ""; } pwd = FileUtils.readFileToByteArray(passwdFile); - return new String(pwd, "UTF-8"); + return new String(pwd, StandardCharsets.UTF_8); } @SuppressFBWarnings( diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommand.java index 4b73742e538..12acd4d9c4d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogMetadataCommand.java @@ -41,7 +41,7 @@ */ public class ReadLogMetadataCommand extends BookieCommand { - static final Logger LOG = LoggerFactory.getLogger(ReadLogMetadataFlags.class); + static final Logger LOG = LoggerFactory.getLogger(ReadLogMetadataCommand.class); private static final String NAME = "readlogmetadata"; private static final String DESC = "Prints entrylog's metadata"; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/collections/ConcurrentOpenHashSet.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/collections/ConcurrentOpenHashSet.java index 9cf6d221850..19482e4d9ab 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/collections/ConcurrentOpenHashSet.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/collections/ConcurrentOpenHashSet.java @@ -67,7 +67,7 @@ public ConcurrentOpenHashSet(int expectedItems, int concurrencyLevel) { int numSections = concurrencyLevel; int perSectionExpectedItems = expectedItems / numSections; int perSectionCapacity = (int) (perSectionExpectedItems / MapFillFactor); - this.sections = (Section[]) new Section[numSections]; + this.sections = new Section[numSections]; for (int i = 0; i < numSections; i++) { sections[i] = new Section<>(perSectionCapacity); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index d10d9abb10c..19168808ac3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -66,9 +66,9 @@ import org.apache.bookkeeper.server.Main; import org.apache.bookkeeper.server.conf.BookieConfiguration; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.BookKeeperConstants; +import org.apache.bookkeeper.util.PortManager; import org.apache.commons.io.FileUtils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs.Ids; diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/Hashes.java b/circe-checksum/src/main/java/com/scurrilous/circe/Hashes.java index 786ead5f3d3..16e3b4a9a35 100644 --- a/circe-checksum/src/main/java/com/scurrilous/circe/Hashes.java +++ b/circe-checksum/src/main/java/com/scurrilous/circe/Hashes.java @@ -17,15 +17,6 @@ import java.util.EnumSet; -import com.scurrilous.circe.HashParameters; -import com.scurrilous.circe.HashProviders; -import com.scurrilous.circe.HashSupport; -import com.scurrilous.circe.IncrementalIntHash; -import com.scurrilous.circe.IncrementalLongHash; -import com.scurrilous.circe.StatefulHash; -import com.scurrilous.circe.StatelessIntHash; -import com.scurrilous.circe.StatelessLongHash; - /** * Static methods to obtain various forms of abstract hash functions. Each * method uses {@link HashProviders#best} to find the best provider for the diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java index 5baad8071d5..8ecf3a9287e 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java @@ -25,7 +25,6 @@ import io.netty.buffer.Unpooled; import io.netty.util.ReferenceCountUtil; -import java.util.Map; import java.util.Random; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.proto.BookieProtoEncoding.EnDecoder; diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java index 2319b2ea3eb..b65b7599e80 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java @@ -26,6 +26,8 @@ import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; + +import java.nio.charset.StandardCharsets; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; @@ -105,7 +107,7 @@ public static class MyState { @Setup(Level.Trial) public void doSetup() throws Exception { - final byte[] password = "password".getBytes("UTF-8"); + final byte[] password = "password".getBytes(StandardCharsets.UTF_8); crc32 = DigestManager.instantiate(ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE), password, DigestType.CRC32, PooledByteBufAllocator.DEFAULT, true); diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/StatsLoggerBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/StatsLoggerBenchmark.java index 3986ffadd83..1a88bd0ee65 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/StatsLoggerBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/StatsLoggerBenchmark.java @@ -23,10 +23,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; -import org.apache.bookkeeper.stats.Counter; -import org.apache.bookkeeper.stats.OpStatsLogger; -import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.stats.StatsProvider; import org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider; import org.apache.bookkeeper.stats.codahale.FastCodahaleMetricsProvider; import org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider; diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/codahale/TimerBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/codahale/TimerBenchmark.java index 5d1cf068f23..6d983131828 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/codahale/TimerBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/stats/codahale/TimerBenchmark.java @@ -21,8 +21,6 @@ import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider; -import org.apache.bookkeeper.stats.codahale.FastCodahaleMetricsProvider; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; From 8410f2dd18b97ab1e41f4d253921f743537b62c4 Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Mon, 8 Jun 2020 20:18:01 +0800 Subject: [PATCH 0438/1642] [BUG] Fix avoid invoke init drivers repeatedly ### Motivation static code blocks are always invoked when the class is initialized, when client connect to bookkeeper server, output repeat init log ``` 2020-06-04 10:10:24,751 INFO org.apache.bookkeeper.meta.MetadataDrivers - BookKeeper metadata driver manager initialized 2020-06-04 10:10:24,753 INFO org.apache.bookkeeper.meta.MetadataDrivers - BookKeeper metadata driver manager initialized 2020-06-04 10:10:24,753 INFO org.apache.bookkeeper.meta.MetadataDrivers - BookKeeper metadata driver manager initialized ``` ### Changes Remove redundant initialization code. Reviewers: Jia Zhai , Matteo Minardi , Enrico Olivelli This closes #2356 from lamber-ken/bk-init-repeat --- .../bookkeeper/meta/MetadataDrivers.java | 25 +------------------ 1 file changed, 1 insertion(+), 24 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java index cf5daf18327..90dad34141e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataDrivers.java @@ -91,27 +91,18 @@ static class MetadataBookieDriverInfo { private static final ConcurrentMap clientDrivers; @Getter(AccessLevel.PACKAGE) private static final ConcurrentMap bookieDrivers; - private static boolean initialized = false; static { clientDrivers = new ConcurrentHashMap<>(); bookieDrivers = new ConcurrentHashMap<>(); - initialize(); - } - - static void initialize() { - if (initialized) { - return; - } loadInitialDrivers(); - initialized = true; - log.info("BookKeeper metadata driver manager initialized"); } @VisibleForTesting static void loadInitialDrivers() { loadInitialClientDrivers(); loadInitialBookieDrivers(); + log.info("BookKeeper metadata driver manager initialized"); } private static void loadInitialClientDrivers() { @@ -183,10 +174,6 @@ public static void registerClientDriver(String metadataBackendScheme, public static void registerClientDriver(String metadataBackendScheme, Class driver, boolean allowOverride) { - if (!initialized) { - initialize(); - } - String scheme = metadataBackendScheme.toLowerCase(); MetadataClientDriverInfo oldDriverInfo = clientDrivers.get(scheme); if (null != oldDriverInfo && !allowOverride) { @@ -218,10 +205,6 @@ public static void registerBookieDriver(String metadataBackendScheme, public static void registerBookieDriver(String metadataBackendScheme, Class driver, boolean allowOverride) { - if (!initialized) { - initialize(); - } - String scheme = metadataBackendScheme.toLowerCase(); MetadataBookieDriverInfo oldDriverInfo = bookieDrivers.get(scheme); if (null != oldDriverInfo && !allowOverride) { @@ -247,9 +230,6 @@ public static void registerBookieDriver(String metadataBackendScheme, */ public static MetadataClientDriver getClientDriver(String scheme) { checkNotNull(scheme, "Client Driver Scheme is null"); - if (!initialized) { - initialize(); - } MetadataClientDriverInfo driverInfo = clientDrivers.get(scheme.toLowerCase()); if (null == driverInfo) { throw new IllegalArgumentException("Unknown backend " + scheme); @@ -287,9 +267,6 @@ public static MetadataClientDriver getClientDriver(URI uri) { */ public static MetadataBookieDriver getBookieDriver(String scheme) { checkNotNull(scheme, "Bookie Driver Scheme is null"); - if (!initialized) { - initialize(); - } MetadataBookieDriverInfo driverInfo = bookieDrivers.get(scheme.toLowerCase()); if (null == driverInfo) { throw new IllegalArgumentException("Unknown backend " + scheme); From 4339486d7d79458a732a6deeb2bffcbe203f2ce1 Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Mon, 8 Jun 2020 20:21:57 +0800 Subject: [PATCH 0439/1642] [BOT] Implement ci bot using github-script ### Motivation `GitHub Action` is deprecated! Implement ci bot using `actions/github-script`, instead. https://github.com/actions/github https://github.com/actions/github-script ### Changes Implement ci using `actions/github-script`, based on https://github.com/zymap/bot/tree/v1.0.1 Reviewers: Jia Zhai , Enrico Olivelli , Yong Zhang , Sijie Guo This closes #2352 from lamber-ken/bot-v2 --- .github/actions/bot/package.json | 31 ++++++++++++++ .github/actions/bot/src/run.js | 69 ++++++++++++++++++++++++++++++++ .github/workflows/bot.yml | 39 ++++++++++++++---- .gitignore | 3 ++ 4 files changed, 134 insertions(+), 8 deletions(-) create mode 100644 .github/actions/bot/package.json create mode 100644 .github/actions/bot/src/run.js diff --git a/.github/actions/bot/package.json b/.github/actions/bot/package.json new file mode 100644 index 00000000000..0671a99823c --- /dev/null +++ b/.github/actions/bot/package.json @@ -0,0 +1,31 @@ +{ + "name": "github-action-bot", + "description": "Bot for github actions", + "version": "1.1.1", + "author": "GitHub", + "license": "Apache LICENSE 2.0", + "main": "dist/index.js", + "private": true, + "scripts": { + "build": "ncc build src/run.js" + }, + "dependencies": { + "@actions/core": "^1.2.4", + "@actions/github": "^2.2.0", + "@actions/io": "^1.0.2" + }, + "devDependencies": { + "@types/jest": "^25.1.4", + "@typescript-eslint/eslint-plugin": "^2.33.0", + "@typescript-eslint/parser": "^2.33.0", + "@zeit/ncc": "^0.22.0", + "eslint": "^7.0.0", + "eslint-config-prettier": "^6.11.0", + "husky": "^4.2.5", + "jest": "^25.1.0", + "npm-run-all": "^4.1.5", + "prettier": "^2.0.5", + "ts-jest": "^25.2.1", + "typescript": "^3.8.3" + } +} diff --git a/.github/actions/bot/src/run.js b/.github/actions/bot/src/run.js new file mode 100644 index 00000000000..01524919a66 --- /dev/null +++ b/.github/actions/bot/src/run.js @@ -0,0 +1,69 @@ +/* + * 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. + */ + +async function run(core, context, github) { + + try { + const owner = process.env.PROVIDER; + const repo = process.env.REPOSITORY; + const reRunCmd = process.env.RERUN_CMD; + const comment = context.payload.comment.body; + + if (comment !== reRunCmd) { + console.log("this is not a bot command"); + return; + } + + const { + data: { + head: { + sha: prRef, + } + } + } = await github.pulls.get({ + owner, + repo, + pull_number: context.issue.number, + }); + + const jobs = await github.checks.listForRef({ + owner, + repo, + ref: prRef, + status: "completed" + }); + + jobs.data.check_runs.forEach(job => { + if (job.conclusion === 'failure' || job.conclusion === 'cancelled') { + console.log("rerun job " + job.name); + github.checks.rerequestSuite({ + owner, + repo, + check_suite_id: job.check_suite.id + }) + } + }); + } catch (e) { + core.setFailed(e); + } + +} + +module.exports = ({core}, {context}, {github}) => { + return run(core, context, github); +} diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 798a20ba846..d1eb093c56c 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -1,3 +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. +# + name: Bot tests on: issue_comment: @@ -8,14 +27,18 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v2 + - name: clone repository + uses: actions/checkout@v2 - - name: Bot actions - uses: zymap/bot@v1.0.1 + - name: bot actions + uses: actions/github-script@v1 env: - GITHUB_TOKEN: ${{ secrets.BKBOT_TOKEN }} + PROVIDER: 'apache' + REPOSITORY: 'bookkeeper' + RERUN_CMD: 'rerun failure checks' with: - repo_owner: apache - repo_name: bookkeeper - rerun_cmd: rerun failure checks - comment: ${{ github.event.comment.body }} + github-token: ${{secrets.BKBOT_TOKEN}} + script: | + const path = require('path') + const scriptPath = path.resolve('.github/actions/bot/src/run.js') + require(scriptPath)({core}, {context}, {github}) \ No newline at end of file diff --git a/.gitignore b/.gitignore index 86f2415acf0..f59b5710359 100644 --- a/.gitignore +++ b/.gitignore @@ -38,3 +38,6 @@ tools/all/src/main/resources # Exclude versionBackup file (generated by `mvn versions:set`) **/*.versionsBackup + +node_modules +package-lock.json From b5208053d5c7d0fb55b9955babb378cadd324063 Mon Sep 17 00:00:00 2001 From: lamber-ken Date: Mon, 8 Jun 2020 20:24:32 +0800 Subject: [PATCH 0440/1642] [BUG] Fix redundant statistics ### Motivation Fix redundant statistics, run 5 bookies `bin/bookkeeper localbookie 10`, then show bookieinfo ``` ./bin/bookkeeper shell bookieinfo ``` **Wrong result** ``` Free disk space info: 10.101.52.18(10.101.52.18):5004: Free: 34930151424(34.93GB) Total: 255850758144(255.85GB) 10.101.52.18(10.101.52.18):5003: Free: 34930151424(34.93GB) Total: 255850758144(255.85GB) 10.101.52.18(10.101.52.18):5002: Free: 34930151424(34.93GB) Total: 255850758144(255.85GB) 10.101.52.18(10.101.52.18):5000: Free: 34930151424(34.93GB) Total: 255850758144(255.85GB) 10.101.52.18(10.101.52.18):5005: Free: 34930151424(34.93GB) Total: 255850758144(255.85GB) Total free disk space in the cluster: 174650757120(174.65GB) Total disk capacity in the cluster: 1279253790720(1.279TB) ``` **Right result** ``` Free disk space info: 10.101.52.18(10.101.52.18):5004: Free: 33172185088(33.172GB) Total: 255850758144(255.85GB) 10.101.52.18(10.101.52.18):5003: Free: 33172185088(33.172GB) Total: 255850758144(255.85GB) 10.101.52.18(10.101.52.18):5002: Free: 33172185088(33.172GB) Total: 255850758144(255.85GB) 10.101.52.18(10.101.52.18):5006: Free: 33172185088(33.172GB) Total: 255850758144(255.85GB) 10.101.52.18(10.101.52.18):5005: Free: 33172185088(33.172GB) Total: 255850758144(255.85GB) Total free disk space in the cluster: 33172185088(33.172GB) Total disk capacity in the cluster: 255850758144(255.85GB) ``` Reviewers: Jia Zhai , Sijie Guo , Enrico Olivelli This closes #2348 from lamber-ken/bk-info-command --- .../tools/cli/commands/bookies/InfoCommand.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java index 4abb2c4ec34..047c15e63b0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java @@ -22,6 +22,7 @@ import java.math.RoundingMode; import java.text.DecimalFormat; import java.util.Map; +import java.util.stream.Collectors; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; @@ -86,8 +87,19 @@ public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { System.out.println(CommandHelpers.getBookieSocketAddrStringRepresentation(bookieId) + ":\tFree: " + bInfo.getFreeDiskSpace() + getReadable(bInfo.getFreeDiskSpace()) + "\tTotal: " + bInfo.getTotalDiskSpace() + getReadable(bInfo.getTotalDiskSpace())); - totalFree += bInfo.getFreeDiskSpace(); - total += bInfo.getTotalDiskSpace(); + } + + // group by hostname + Map dedupedMap = map.entrySet() + .stream() + .collect(Collectors.toMap( + entry -> entry.getKey().getHostName(), + entry -> entry.getValue(), + (key1, key2) -> key2 + )); + for (BookieInfo bookieInfo : dedupedMap.values()) { + totalFree += bookieInfo.getFreeDiskSpace(); + total += bookieInfo.getTotalDiskSpace(); } System.out.println("Total free disk space in the cluster:\t" + totalFree + getReadable(totalFree)); From fd445afaf5dd2ecbc462a0c545576741bc4485cb Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Fri, 12 Jun 2020 08:05:39 -0700 Subject: [PATCH 0441/1642] =?UTF-8?q?Remove=20lock=20from=20URManager?= =?UTF-8?q?=E2=80=99s=20state=20only=20if=20lock=20znode=20deletion=20has?= =?UTF-8?q?=20succeeded?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Descriptions of the changes in this PR: - in ZkLedgerUnderreplicationManager.releaseUnderreplicatedLedger remove ‘lock’ from ‘heldLocks’ only if lock znode deletion has succeeded. - This is needed because, if RW.logBKExceptionAndReleaseLedger fails to delete the lock znode, then it needs to be tried once more in the finally block of 'rereplicate(long ledgerIdToReplicate)’ before giving up and shutting down RW. Reviewers: Enrico Olivelli This closes #2206 from reddycharan/fixzklossissue --- .../meta/AbstractZkLedgerManager.java | 2 +- .../meta/HierarchicalLedgerManager.java | 2 +- .../meta/ZkLedgerUnderreplicationManager.java | 3 +- .../replication/TestReplicationWorker.java | 225 +++++++++++++++++- 4 files changed, 228 insertions(+), 4 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 808cf8951f9..0b42c3df80f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -181,7 +181,7 @@ protected AbstractZkLedgerManager(AbstractConfiguration conf, ZooKeeper zk) { * Ledger ID * @return ledger node path */ - protected abstract String getLedgerPath(long ledgerId); + public abstract String getLedgerPath(long ledgerId); /** * Get ledger id from its znode ledger path. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java index 9cd6aed92e3..b5fd0f21ae8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/HierarchicalLedgerManager.java @@ -73,7 +73,7 @@ public void processResult(int rc, String path, Object ctx) { } @Override - protected String getLedgerPath(long ledgerId) { + public String getLedgerPath(long ledgerId) { return ledgerRootPath + StringUtils.getHybridHierarchicalLedgerPath(ledgerId); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java index 6376d720e89..6ed8df3d55d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java @@ -630,7 +630,7 @@ public void releaseUnderreplicatedLedger(long ledgerId) throws ReplicationExcept LOG.debug("releaseLedger(ledgerId={})", ledgerId); } try { - Lock l = heldLocks.remove(ledgerId); + Lock l = heldLocks.get(ledgerId); if (l != null) { zkc.delete(l.getLockZNode(), -1); } @@ -643,6 +643,7 @@ public void releaseUnderreplicatedLedger(long ledgerId) throws ReplicationExcept Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while connecting zookeeper", ie); } + heldLocks.remove(ledgerId); } @Override 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 11531b504ef..09751e99e88 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,6 +22,7 @@ import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -34,6 +35,7 @@ import java.util.TimerTask; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; @@ -44,11 +46,14 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.AbstractZkLedgerManager; +import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.MetadataClientDriver; import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -59,8 +64,17 @@ import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; +import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase; +import org.apache.zookeeper.AsyncCallback.StatCallback; 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.ZooKeeper.States; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -79,8 +93,10 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase { private MetadataBookieDriver driver; private LedgerManagerFactory mFactory; private LedgerUnderreplicationManager underReplicationManager; + private LedgerManager ledgerManager; private static byte[] data = "TestReplicationWorker".getBytes(); private OrderedScheduler scheduler; + private String zkLedgersRootPath; public TestReplicationWorker() { this("org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory"); @@ -100,7 +116,7 @@ public TestReplicationWorker() { public void setUp() throws Exception { super.setUp(); - String zkLedgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseClientConf); + zkLedgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseClientConf); basePath = zkLedgersRootPath + '/' + BookKeeperConstants.UNDER_REPLICATION_NODE + BookKeeperConstants.DEFAULT_ZK_LEDGERS_ROOT_PATH; @@ -121,6 +137,7 @@ public void setUp() throws Exception { NullStatsLogger.INSTANCE); // initialize urReplicationManager mFactory = driver.getLedgerManagerFactory(); + ledgerManager = mFactory.newLedgerManager(); underReplicationManager = mFactory.newLedgerUnderreplicationManager(); } @@ -857,4 +874,210 @@ private void verifyRecoveredLedgers(LedgerHandle lh, long startEntryId, } } + class MockZooKeeperClient extends ZooKeeperClient { + private final String connectString; + private final int sessionTimeoutMs; + private final ZooKeeperWatcherBase watcherManager; + private volatile String pathOfSetDataToFail; + private volatile String pathOfDeleteToFail; + private AtomicInteger numOfTimesSetDataFailed = new AtomicInteger(); + private AtomicInteger numOfTimesDeleteFailed = new AtomicInteger(); + + MockZooKeeperClient(String connectString, int sessionTimeoutMs, ZooKeeperWatcherBase watcher) + throws IOException { + /* + * in OperationalRetryPolicy maxRetries is set to 0. So it wont + * retry incase of any error/exception. + */ + super(connectString, sessionTimeoutMs, watcher, + new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, Integer.MAX_VALUE), + new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0), + NullStatsLogger.INSTANCE, 1, 0, false); + this.connectString = connectString; + this.sessionTimeoutMs = sessionTimeoutMs; + this.watcherManager = watcher; + } + + @Override + protected ZooKeeper createZooKeeper() throws IOException { + return new MockZooKeeper(this.connectString, this.sessionTimeoutMs, this.watcherManager, false); + } + + private void setPathOfSetDataToFail(String pathOfSetDataToFail) { + this.pathOfSetDataToFail = pathOfSetDataToFail; + } + + private void setPathOfDeleteToFail(String pathOfDeleteToFail) { + this.pathOfDeleteToFail = pathOfDeleteToFail; + } + + private int getNumOfTimesSetDataFailed() { + return numOfTimesSetDataFailed.get(); + } + + private int getNumOfTimesDeleteFailed() { + return numOfTimesDeleteFailed.get(); + } + + class MockZooKeeper extends ZooKeeper { + public MockZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) + throws IOException { + super(connectString, sessionTimeout, watcher, canBeReadOnly); + } + + @Override + public void setData(final String path, final byte[] data, final int version, final StatCallback cb, + final Object context) { + if ((pathOfSetDataToFail != null) && (pathOfSetDataToFail.equals(path))) { + /* + * if pathOfSetDataToFail matches with the path of the node, + * then callback with CONNECTIONLOSS error. + */ + LOG.error("setData of MockZooKeeper, is failing with CONNECTIONLOSS for path: {}", path); + numOfTimesSetDataFailed.incrementAndGet(); + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, context, null); + } else { + super.setData(path, data, version, cb, context); + } + } + + @Override + public void delete(final String path, final int version) throws KeeperException, InterruptedException { + if ((pathOfDeleteToFail != null) && (pathOfDeleteToFail.equals(path))) { + /* + * if pathOfDeleteToFail matches with the path of the node, + * then throw CONNECTIONLOSS exception. + */ + LOG.error("delete of MockZooKeeper, is failing with CONNECTIONLOSS for path: {}", path); + numOfTimesDeleteFailed.incrementAndGet(); + throw new KeeperException.ConnectionLossException(); + } else { + super.delete(path, version); + } + } + } + } + + @Test + public void testRWShutDownInTheCaseOfZKOperationFailures() throws Exception { + /* + * create MockZooKeeperClient instance and wait for it to be connected. + */ + int zkSessionTimeOut = 10000; + ZooKeeperWatcherBase zooKeeperWatcherBase = new ZooKeeperWatcherBase(zkSessionTimeOut, + NullStatsLogger.INSTANCE); + MockZooKeeperClient zkFaultInjectionWrapper = new MockZooKeeperClient(zkUtil.getZooKeeperConnectString(), + zkSessionTimeOut, zooKeeperWatcherBase); + zkFaultInjectionWrapper.waitForConnection(); + assertEquals("zkFaultInjectionWrapper should be in connected state", States.CONNECTED, + zkFaultInjectionWrapper.getState()); + long oldZkInstanceSessionId = zkFaultInjectionWrapper.getSessionId(); + + /* + * create ledger and add entries. + */ + BookKeeper bkWithMockZK = new BookKeeper(baseClientConf, zkFaultInjectionWrapper); + long ledgerId = 567L; + LedgerHandle lh = bkWithMockZK.createLedgerAdv(ledgerId, 2, 2, 2, BookKeeper.DigestType.CRC32, TESTPASSWD, + null); + for (int i = 0; i < 10; i++) { + lh.addEntry(i, data); + } + lh.close(); + + /* + * trigger Expired event so that MockZooKeeperClient would run + * 'clientCreator' and create new zk handle. In this case it would + * create MockZooKeeper instance. + */ + zooKeeperWatcherBase.process(new WatchedEvent(EventType.None, KeeperState.Expired, "")); + zkFaultInjectionWrapper.waitForConnection(); + for (int i = 0; i < 10; i++) { + if (zkFaultInjectionWrapper.getState() == States.CONNECTED) { + break; + } + Thread.sleep(200); + } + assertEquals("zkFaultInjectionWrapper should be in connected state", States.CONNECTED, + zkFaultInjectionWrapper.getState()); + assertNotEquals("Session Id of old and new ZK instance should be different", oldZkInstanceSessionId, + zkFaultInjectionWrapper.getSessionId()); + + /* + * Kill a Bookie, so that ledger becomes underreplicated. Since totally + * 3 bookies are available and the ensemblesize of the current ledger is + * 2, we should be able to replicate to the other bookie. + */ + BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + LOG.info("Killing Bookie", replicaToKill); + killBookie(replicaToKill); + + /* + * Start RW. + */ + ReplicationWorker rw = new ReplicationWorker(baseConf, bkWithMockZK, false, NullStatsLogger.INSTANCE); + rw.start(); + try { + for (int i = 0; i < 40; i++) { + if (rw.isRunning()) { + break; + } + LOG.info("Waiting for the RW to start..."); + Thread.sleep(500); + } + assertTrue("RW should be running", rw.isRunning()); + + /* + * Since Auditor is not running, ledger needs to be marked + * underreplicated explicitly. But before marking ledger + * underreplicated, set paths for which MockZooKeeper's setData and + * Delete operation to fail. + * + * ZK.setData will be called by 'updateEnsembleInfo' operation after + * completion of copying to a new bookie. ZK.delete will be called by + * RW.logBKExceptionAndReleaseLedger and finally block in + * 'rereplicate(long ledgerIdToReplicate)' + */ + AbstractZkLedgerManager absZKLedgerManager = (AbstractZkLedgerManager) ledgerManager; + String ledgerPath = absZKLedgerManager.getLedgerPath(ledgerId); + String urLockPath = ZkLedgerUnderreplicationManager + .getUrLedgerLockZnode(ZkLedgerUnderreplicationManager.getUrLockPath(zkLedgersRootPath), ledgerId); + zkFaultInjectionWrapper.setPathOfSetDataToFail(ledgerPath); + zkFaultInjectionWrapper.setPathOfDeleteToFail(urLockPath); + underReplicationManager.markLedgerUnderreplicated(lh.getId(), replicaToKill.toString()); + + /* + * Since there is only one RW, it will try to replicate underreplicated + * ledger. After completion of copying it to a new bookie, it will try + * to update ensembleinfo. Which would fail with our MockZK. After that + * it would try to delete lock znode as part of + * RW.logBKExceptionAndReleaseLedger, which will also fail because of + * our MockZK. In the finally block in 'rereplicate(long + * ledgerIdToReplicate)' it would try one more time to delete the ledger + * and once again it will fail because of our MockZK. So RW gives up and + * shutdowns itself. + */ + for (int i = 0; i < 40; i++) { + if (!rw.isRunning()) { + break; + } + LOG.info("Waiting for the RW to shutdown..."); + Thread.sleep(500); + } + + /* + * as described earlier, numOfTimes setDataFailed should be 1 and + * numOfTimes deleteFailed should be 2 + */ + assertEquals("NumOfTimesSetDataFailed", 1, + zkFaultInjectionWrapper.getNumOfTimesSetDataFailed()); + assertEquals("NumOfTimesDeleteFailed", 2, + zkFaultInjectionWrapper.getNumOfTimesDeleteFailed()); + assertFalse("RW should be shutdown", rw.isRunning()); + } finally { + rw.shutdown(); + zkFaultInjectionWrapper.close(); + bkWithMockZK.close(); + } + } } From 0ae6fdc31bf671664cb88397d11eaac029378717 Mon Sep 17 00:00:00 2001 From: Charan Reddy Guttapalem Date: Fri, 12 Jun 2020 08:06:31 -0700 Subject: [PATCH 0442/1642] Ledger having a Empty segment is causing ReplicasCheck scrutiny to fail Descriptions of the changes in this PR: - in Auditor.ReadLedgerMetadataCallbackForReplicasCheck, fixing logic for empty segments - added test for validating areEntriesOfLedgerStoredInTheBookie for empty middle segment - added testcase for ledger with more empty segments. Reviewers: Enrico Olivelli This closes #2205 from reddycharan/fixreplicascheck --- .../bookkeeper/replication/Auditor.java | 21 ++- .../client/BookKeeperAdminTest.java | 3 +- .../replication/AuditorReplicasCheckTest.java | 122 ++++++++++++++++++ 3 files changed, 140 insertions(+), 6 deletions(-) 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 eac25b2c678..db6e3319dc7 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 @@ -133,6 +133,7 @@ public class Auditor implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(Auditor.class); private static final int MAX_CONCURRENT_REPLICAS_CHECK_LEDGER_REQUESTS = 100; private static final int REPLICAS_CHECK_TIMEOUT_IN_SECS = 120; + private static final BitSet EMPTY_BITSET = new BitSet(); private final ServerConfiguration conf; private final BookKeeper bkc; private final boolean ownBkc; @@ -1489,7 +1490,8 @@ public void accept(Versioned metadataVer, Throwable exception) { return; } - if (metadata.getLastEntryId() == -1) { + final long lastEntryId = metadata.getLastEntryId(); + if (lastEntryId == -1) { LOG.debug("Ledger: {} is closed but it doesn't has any entries, so skipping the replicas check", ledgerInRange); mcbForThisLedgerRange.processResult(BKException.Code.OK, null, null); @@ -1515,12 +1517,23 @@ public void accept(Versioned metadataVer, Throwable exception) { final Entry> segmentEnsemble = segments.get(segmentNum); final List ensembleOfSegment = segmentEnsemble.getValue(); final long startEntryIdOfSegment = segmentEnsemble.getKey(); - final long lastEntryIdOfSegment = (segmentNum == (segments.size() - 1)) ? metadata.getLastEntryId() + final boolean lastSegment = (segmentNum == (segments.size() - 1)); + final long lastEntryIdOfSegment = lastSegment ? lastEntryId : segments.get(segmentNum + 1).getKey() - 1; + /* + * Segment can be empty. If last segment is empty, then + * startEntryIdOfSegment of it will be greater than lastEntryId + * of the ledger. If the segment in middle is empty, then its + * startEntry will be same as startEntry of the following + * segment. + */ + final boolean emptySegment = lastSegment ? (startEntryIdOfSegment > lastEntryId) + : (startEntryIdOfSegment == segments.get(segmentNum + 1).getKey()); for (int bookieIndex = 0; bookieIndex < ensembleOfSegment.size(); bookieIndex++) { final BookieSocketAddress bookieInEnsemble = ensembleOfSegment.get(bookieIndex); - final BitSet entriesStripedToThisBookie = distributionSchedule - .getEntriesStripedToTheBookie(bookieIndex, startEntryIdOfSegment, lastEntryIdOfSegment); + final BitSet entriesStripedToThisBookie = emptySegment ? EMPTY_BITSET + : distributionSchedule.getEntriesStripedToTheBookie(bookieIndex, startEntryIdOfSegment, + lastEntryIdOfSegment); if (entriesStripedToThisBookie.cardinality() == 0) { /* * if no entry is expected to contain in this bookie, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index 19168808ac3..15253fbe04c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -592,7 +592,7 @@ public void testGetListOfEntriesOfLedgerWithEntriesNotStripedToABookie() throws } @Test - public void testAreEntriesOfLedgerStoredInTheBookieForMultipleSegments() throws Exception { + public void testAreEntriesOfLedgerStoredInTheBookieForLastEmptySegment() throws Exception { int lastEntryId = 10; long ledgerId = 100L; BookieSocketAddress bookie0 = new BookieSocketAddress("bookie0:3181"); @@ -728,5 +728,4 @@ public void testBookieServiceInfoReadonly() throws Exception { public void testLegacyBookieServiceInfo() throws Exception { testBookieServiceInfo(false, true); } - } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java index 85b2015cade..fcbde5ea4a1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java @@ -808,4 +808,126 @@ public void testReplicasCheckForLedgersFoundHavingLessThanWQReplicasOfAnEntry() runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, 0, 0, numLedgersFoundHavingLessThanWQReplicasOfAnEntry); } + + /* + * In this testscenario all the ledgers have empty segments. + */ + @Test + public void testReplicasCheckForLedgersWithEmptySegments() throws Exception { + int numOfBookies = 5; + RegistrationManager regManager = driver.getRegistrationManager(); + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + int numLedgersFoundHavingNoReplicaOfAnEntry = 0; + int numLedgersFoundHavingLessThanAQReplicasOfAnEntry = 0; + int numLedgersFoundHavingLessThanWQReplicasOfAnEntry = 0; + + /* + * closed ledger. + * + * This closed Ledger has no entry. So it should not be counted towards + * numLedgersFoundHavingNoReplicaOfAnEntry/LessThanAQReplicasOfAnEntry + * /WQReplicasOfAnEntry. + */ + Map> segmentEnsembles = new HashMap>(); + int ensembleSize = 4; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + long lastEntryId = -1L; + int length = 0; + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + + /* + * closed ledger with multiple segments. + * + * This ledger has empty last segment, but all the entries have + * writeQuorumSize number of copies, So it should not be counted towards + * numLedgersFoundHavingNoReplicaOfAnEntry/LessThanAQReplicasOfAnEntry/ + * WQReplicasOfAnEntry. + */ + lastEntryId = 2; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put((lastEntryId + 1), bookieAddresses.subList(1, 5)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2 })); + + /* + * Closed ledger with multiple segments. + * + * Segment0, Segment1, Segment3, Segment5 and Segment6 are empty. + * Entries from entryid 3 are missing. So it should be counted towards + * numLedgersFoundHavingNoReplicaOfAnEntry. + */ + lastEntryId = 5; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(4L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(4L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put((lastEntryId + 1), bookieAddresses.subList(1, 5)); + segmentEnsembles.put((lastEntryId + 1), bookieAddresses.subList(0, 4)); + ledgerId = 3L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2 })); + numLedgersFoundHavingNoReplicaOfAnEntry++; + + /* + * non-closed ledger with multiple segments + * + * since this is non-closed ledger, it should not be counted towards + * ledgersFoundHavingLessThanWQReplicasOfAnEntry + */ + lastEntryId = 2; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(0L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put((lastEntryId + 1), bookieAddresses.subList(1, 5)); + ledgerId = 4L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2 })); + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, + numLedgersFoundHavingNoReplicaOfAnEntry, numLedgersFoundHavingLessThanAQReplicasOfAnEntry, + numLedgersFoundHavingLessThanWQReplicasOfAnEntry); + } } From 36cf8ee51ebe1a626bf405ecb48e9c135741b1e8 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 18 Jun 2020 00:04:18 -0700 Subject: [PATCH 0443/1642] [BOOKIE-SHELL] Add cmd to change bookie-address into ledger metadata ### Motivation In a large bookie cluster environment, we frequently require a utility to change bookie-ip with a different bookie-ip in ledger's metadata in different scenarios such as: 1. Host Re-IP: requires updating ledger metadata and reuse the host without losing/copying data 2. Backup-restore usecase while doing cluster level data migration. Therefore, we frequently need a tool to update bookie-ip with a new bookie-ip in existing ledgers' ensemble metadata. ### Modification Add CLI command to update bookie-id in ledger metadata. Reviewers: Enrico Olivelli , Sijie Guo , Jia Zhai This closes #2321 from rdhabalia/reip --- .../apache/bookkeeper/bookie/BookieShell.java | 82 ++++++++ .../bookie/UpdateBookieInLedgerCommand.java | 186 ++++++++++++++++++ .../client/UpdateLedgerCmdTest.java | 26 +++ site/_data/cli/shell.yaml | 19 ++ 4 files changed, 313 insertions(+) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/UpdateBookieInLedgerCommand.java 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 0ba984c7b0b..73edab939c5 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 @@ -63,6 +63,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.RebuildDBLedgerLocationsIndexCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.RegenerateInterleavedStorageIndexFileCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.UpdateBookieInLedgerCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.DecommissionCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.EndpointInfoCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand; @@ -135,6 +136,7 @@ public class BookieShell implements Tool { static final String CMD_LISTFILESONDISC = "listfilesondisc"; static final String CMD_UPDATECOOKIE = "updatecookie"; static final String CMD_UPDATELEDGER = "updateledgers"; + static final String CMD_UPDATE_BOOKIE_IN_LEDGER = "updateBookieInLedger"; static final String CMD_DELETELEDGER = "deleteledger"; static final String CMD_BOOKIEINFO = "bookieinfo"; static final String CMD_DECOMMISSIONBOOKIE = "decommissionbookie"; @@ -1561,6 +1563,85 @@ int runCmd(CommandLine cmdLine) throws Exception { } } + /** + * Update bookie into ledger command. + */ + class UpdateBookieInLedgerCmd extends MyCommand { + private final Options opts = new Options(); + + UpdateBookieInLedgerCmd() { + super(CMD_UPDATE_BOOKIE_IN_LEDGER); + opts.addOption("sb", "srcBookie", true, "Source bookie which needs to be replaced by destination bookie."); + opts.addOption("db", "destBookie", true, "Destination bookie which replaces source bookie."); + opts.addOption("s", "updatespersec", true, "Number of ledgers updating per second (default: 5 per sec)"); + opts.addOption("r", "maxOutstandingReads", true, "Max outstanding reads (default: 5 * updatespersec)"); + opts.addOption("l", "limit", true, "Maximum number of ledgers to update (default: no limit)"); + opts.addOption("v", "verbose", true, "Print status of the ledger updation (default: false)"); + opts.addOption("p", "printprogress", true, + "Print messages on every configured seconds if verbose turned on (default: 10 secs)"); + } + + @Override + Options getOptions() { + return opts; + } + + @Override + String getDescription() { + return "Replace bookie in ledger metadata. (useful when re-ip of host) " + + "replace srcBookie with destBookie. (this may take a long time)."; + } + + @Override + String getUsage() { + return "updateBookieInLedger -srcBookie -destBookie " + + "[-updatespersec N] [-maxOutstandingReads N] [-limit N] [-verbose true/false] [-printprogress N]"; + } + + @Override + int runCmd(CommandLine cmdLine) throws Exception { + UpdateBookieInLedgerCommand cmd = new UpdateBookieInLedgerCommand(); + UpdateBookieInLedgerCommand.UpdateBookieInLedgerFlags flags = + new UpdateBookieInLedgerCommand.UpdateBookieInLedgerFlags(); + + final String srcBookie = cmdLine.getOptionValue("srcBookie"); + final String destBookie = cmdLine.getOptionValue("destBookie"); + if (StringUtils.isBlank(srcBookie) || StringUtils.isBlank(destBookie)) { + LOG.error("Invalid argument list (srcBookie and destBookie must be provided)!"); + this.printUsage(); + return -1; + } + if (StringUtils.equals(srcBookie, destBookie)) { + LOG.error("srcBookie and destBookie can't be the same."); + return -1; + } + final int rate = getOptionIntValue(cmdLine, "updatespersec", 5); + final int maxOutstandingReads = getOptionIntValue(cmdLine, "maxOutstandingReads", (rate * 5)); + final int limit = getOptionIntValue(cmdLine, "limit", Integer.MIN_VALUE); + final boolean verbose = getOptionBooleanValue(cmdLine, "verbose", false); + final long printprogress; + if (!verbose) { + if (cmdLine.hasOption("printprogress")) { + LOG.warn("Ignoring option 'printprogress', this is applicable when 'verbose' is true"); + } + printprogress = Integer.MIN_VALUE; + } else { + // defaulting to 10 seconds + printprogress = getOptionLongValue(cmdLine, "printprogress", 10); + } + flags.srcBookie(srcBookie); + flags.destBookie(destBookie); + flags.printProgress(printprogress); + flags.limit(limit); + flags.updatePerSec(rate); + flags.maxOutstandingReads(maxOutstandingReads); + flags.verbose(verbose); + + boolean result = cmd.apply(bkConf, flags); + return (result) ? 0 : -1; + } + } + /** * Command to delete a given ledger. */ @@ -1950,6 +2031,7 @@ int runCmd(CommandLine cmdLine) throws Exception { commands.put(CMD_LISTFILESONDISC, new ListDiskFilesCmd()); commands.put(CMD_UPDATECOOKIE, new UpdateCookieCmd()); commands.put(CMD_UPDATELEDGER, new UpdateLedgerCmd()); + commands.put(CMD_UPDATE_BOOKIE_IN_LEDGER, new UpdateBookieInLedgerCmd()); commands.put(CMD_DELETELEDGER, new DeleteLedgerCmd()); commands.put(CMD_BOOKIEINFO, new BookieInfoCmd()); commands.put(CMD_DECOMMISSIONBOOKIE, new DecommissionBookieCmd()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/UpdateBookieInLedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/UpdateBookieInLedgerCommand.java new file mode 100644 index 00000000000..88d1b4d3e0d --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/UpdateBookieInLedgerCommand.java @@ -0,0 +1,186 @@ +/* + * 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.tools.cli.commands.bookie; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.BookieShell; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.UpdateLedgerOp; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.MathUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command to update ledger command. + */ +public class UpdateBookieInLedgerCommand extends BookieCommand { + + static final Logger LOG = LoggerFactory.getLogger(UpdateBookieInLedgerCommand.class); + + private static final String NAME = "update-bookie-ledger-cmd"; + private static final String DESC = "Update bookie in ledgers metadata (this may take a long time)."; + + public UpdateBookieInLedgerCommand() { + this(new UpdateBookieInLedgerFlags()); + } + + private UpdateBookieInLedgerCommand(UpdateBookieInLedgerFlags flags) { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(flags) + .build()); + } + + /** + * Flags for update bookie in ledger command. + */ + @Accessors(fluent = true) + @Setter + public static class UpdateBookieInLedgerFlags extends CliFlags { + + @Parameter(names = { "-sb", "--srcBookie" }, + description = "Source bookie which needs to be replaced by destination bookie. ") + private String srcBookie; + + @Parameter(names = { "-db", "--destBookie" }, + description = "Destination bookie which replaces source bookie. ") + private String destBookie; + + @Parameter(names = { "-s", "--updatepersec" }, + description = "Number of ledgers updating per second (default: 5 per sec)") + private int updatePerSec = 5; + + @Parameter(names = { "-r", + "--maxOutstandingReads" }, description = "Max outstanding reads (default: 5 * updatespersec)") + private int maxOutstandingReads = updatePerSec * 5; + + @Parameter(names = {"-l", "--limit"}, + description = "Maximum number of ledgers of ledgers to update (default: no limit)") + private int limit = Integer.MIN_VALUE; + + @Parameter(names = { "-v", "--verbose" }, description = "Print status of the ledger updation (default: false)") + private boolean verbose; + + @Parameter(names = { "-p", "--printprogress" }, + description = "Print messages on every configured seconds if verbose turned on (default: 10 secs)") + private long printProgress = 10; + } + + @Override + public boolean apply(ServerConfiguration conf, UpdateBookieInLedgerFlags cmdFlags) { + try { + return updateLedger(conf, cmdFlags); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + } + + private boolean updateLedger(ServerConfiguration conf, UpdateBookieInLedgerFlags flags) + throws InterruptedException, BKException, IOException { + + BookieSocketAddress srcBookieAddress; + BookieSocketAddress destBookieAddress; + try { + String[] bookieAddress = flags.srcBookie.split(":"); + srcBookieAddress = new BookieSocketAddress(bookieAddress[0], Integer.parseInt(bookieAddress[1])); + bookieAddress = flags.destBookie.split(":"); + destBookieAddress = new BookieSocketAddress(bookieAddress[0], Integer.parseInt(bookieAddress[1])); + } catch (Exception e) { + LOG.error("Bookie address must in
            : format"); + return false; + } + + final int rate = flags.updatePerSec; + if (rate <= 0) { + LOG.error("Invalid updatespersec {}, should be > 0", rate); + return false; + } + + final int maxOutstandingReads = flags.maxOutstandingReads; + if (maxOutstandingReads <= 0) { + LOG.error("Invalid maxOutstandingReads {}, should be > 0", maxOutstandingReads); + return false; + } + + final int limit = flags.limit; + if (limit <= 0 && limit != Integer.MIN_VALUE) { + LOG.error("Invalid limit {}, should be > 0", limit); + return false; + } + + final long printProgress; + if (flags.verbose) { + printProgress = 10; + } else { + printProgress = flags.printProgress; + } + + final ClientConfiguration clientConfiguration = new ClientConfiguration(); + clientConfiguration.addConfiguration(conf); + final BookKeeper bk = new BookKeeper(clientConfiguration); + final BookKeeperAdmin admin = new BookKeeperAdmin(bk); + if (admin.getAvailableBookies().contains(srcBookieAddress) + || admin.getReadOnlyBookies().contains(srcBookieAddress)) { + bk.close(); + admin.close(); + LOG.error("Source bookie {} can't be active", srcBookieAddress); + return false; + } + final UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, admin); + + BookieShell.UpdateLedgerNotifier progressable = new BookieShell.UpdateLedgerNotifier() { + long lastReport = System.nanoTime(); + + @Override + public void progress(long updated, long issued) { + if (printProgress <= 0) { + return; // disabled + } + if (TimeUnit.MILLISECONDS.toSeconds(MathUtils.elapsedMSec(lastReport)) >= printProgress) { + LOG.info("Number of ledgers issued={}, updated={}", issued, updated); + lastReport = MathUtils.nowInNano(); + } + } + }; + + try { + updateLedgerOp.updateBookieIdInLedgers(srcBookieAddress, destBookieAddress, rate, maxOutstandingReads, + limit, progressable); + } catch (IOException e) { + LOG.error("Failed to update ledger metadata", e); + return false; + } + + return true; + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java index 4e8d892721e..f74adc6857a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java @@ -83,6 +83,32 @@ public void testUpdateLedgersToHostname() throws Exception { assertEquals("Failed to update the ledger metadata to use bookie host name", 40, updatedLedgersCount); } + /** + * replace bookie address in ledger. + */ + @Test + public void testUpdateBookieInLedger() throws Exception { + BookKeeper bk = new BookKeeper(baseClientConf, zkc); + LOG.info("Create ledger and add entries to it"); + List ledgers = new ArrayList(); + LedgerHandle lh1 = createLedgerWithEntries(bk, 0); + ledgers.add(lh1); + for (int i = 1; i < 40; i++) { + ledgers.add(createLedgerWithEntries(bk, 0)); + } + BookieSocketAddress srcBookie = bs.get(0).getLocalAddress(); + BookieSocketAddress destBookie = new BookieSocketAddress("1.1.1.1", 2181); + String[] argv = new String[] { "updateBookieInLedger", "-sb", srcBookie.toString(), "-db", + destBookie.toString(), "-v", "true", "-p", "2" }; + final ServerConfiguration conf = bsConfs.get(0); + bs.get(0).shutdown(); + updateLedgerCmd(argv, 0, conf); + int updatedLedgersCount = getUpdatedLedgersCount(bk, ledgers, srcBookie); + assertEquals("Failed to update the ledger metadata with new bookie-address", 0, updatedLedgersCount); + updatedLedgersCount = getUpdatedLedgersCount(bk, ledgers, destBookie); + assertEquals("Failed to update the ledger metadata with new bookie-address", 40, updatedLedgersCount); + } + private void updateLedgerCmd(String[] argv, int exitCode, ServerConfiguration conf) throws KeeperException, InterruptedException, IOException, UnknownHostException, Exception { LOG.info("Perform updateledgers command"); diff --git a/site/_data/cli/shell.yaml b/site/_data/cli/shell.yaml index 071fb3400dd..585947e5862 100644 --- a/site/_data/cli/shell.yaml +++ b/site/_data/cli/shell.yaml @@ -180,6 +180,25 @@ commands: description: Print status of the ledger updation (default false) - flag: -printprogress N description: Print messages on every configured seconds if verbose turned on (default 10 secs) +- name: updateBookieInLedger + description: | + Replace srcBookie with destBookie in ledger metadata. (this may take a long time). + Useful when Host-reip or data-migration. In that case, shutdown bookie process in src-bookie, + use this command to update ledger metadata by replacing src-bookie to dest-bookie where data has been copied/moved. + Start the bookie process on dest-bookie and dest-bookie will serve copied ledger data from src-bookie. + options: + - flag: -srcBookie + description: Source Bookie Id + - flag: -destBookie + description: Destination Bookie Id + - flag: -updatespersec N + description: Number of ledgers updating per second (default 5 per sec) + - flag: -limit N + description: Maximum number of ledgers to update (default no limit) + - flag: -verbose + description: Print status of the ledger updation (default false) + - flag: -printprogress N + description: Print messages on every configured seconds if verbose turned on (default 10 secs) - name: whoisauditor description: Print the node which holds the auditor lock - name: whatisinstanceid From 71bc3bfe4f628d9e06ff9011758b643f214f2fb8 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sat, 20 Jun 2020 10:24:54 -0700 Subject: [PATCH 0444/1642] Change Python client version to 4.11.0 According to the release guide, before cutting a release we have to remove the 'alpha' qualifier from Python client version Reviewers: Enrico Olivelli This closes #2362 from rdhabalia/python_client --- stream/clients/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index 40268d956d2..747388ebb13 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -19,7 +19,7 @@ name = 'apache-bookkeeper-client' description = 'Apache BookKeeper client library' -version = '4.11.0-SNAPSHOT' +version = '4.11.0' # Should be one of: # 'Development Status :: 3 - Alpha' # 'Development Status :: 4 - Beta' From 6216f19912ec4466f761eda54355449d0064308a Mon Sep 17 00:00:00 2001 From: rdhabalia Date: Sat, 20 Jun 2020 12:03:23 -0700 Subject: [PATCH 0445/1642] [maven-release-plugin] prepare branch branch-4.11 --- pom.xml | 2 +- stream/api/pom.xml | 3 +-- stream/bk-grpc-name-resolver/pom.xml | 3 +-- stream/clients/java/all/pom.xml | 3 +-- stream/clients/java/base/pom.xml | 3 +-- stream/clients/java/kv/pom.xml | 3 +-- stream/clients/java/pom.xml | 3 +-- stream/clients/pom.xml | 3 +-- stream/common/pom.xml | 3 +-- stream/pom.xml | 3 +-- stream/proto/pom.xml | 3 +-- stream/server/pom.xml | 3 +-- stream/statelib/pom.xml | 3 +-- stream/storage/api/pom.xml | 3 +-- stream/storage/impl/pom.xml | 3 +-- stream/storage/pom.xml | 3 +-- stream/tests-common/pom.xml | 3 +-- tests/integration/cluster/pom.xml | 2 +- tools/perf/pom.xml | 3 +-- tools/stream/pom.xml | 3 +-- 20 files changed, 20 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 191d0e80ac8..6bcbd1c4bb4 100644 --- a/pom.xml +++ b/pom.xml @@ -40,7 +40,7 @@ scm:git:https://github.com/apache/bookkeeper.git scm:git:https://github.com/apache/bookkeeper.git https://github.com/apache/bookkeeper - HEAD + branch-4.11 JIRA diff --git a/stream/api/pom.xml b/stream/api/pom.xml index bd670538d04..abe9264260d 100644 --- a/stream/api/pom.xml +++ b/stream/api/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 stream-storage-parent diff --git a/stream/bk-grpc-name-resolver/pom.xml b/stream/bk-grpc-name-resolver/pom.xml index b3c879d6a7d..aff1915cd1e 100644 --- a/stream/bk-grpc-name-resolver/pom.xml +++ b/stream/bk-grpc-name-resolver/pom.xml @@ -17,8 +17,7 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + 4.0.0 stream-storage-parent diff --git a/stream/clients/java/all/pom.xml b/stream/clients/java/all/pom.xml index e11462cfa69..08251d8c335 100644 --- a/stream/clients/java/all/pom.xml +++ b/stream/clients/java/all/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/stream/clients/java/base/pom.xml b/stream/clients/java/base/pom.xml index 8f8807b7dc9..86edaa05f01 100644 --- a/stream/clients/java/base/pom.xml +++ b/stream/clients/java/base/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/stream/clients/java/kv/pom.xml b/stream/clients/java/kv/pom.xml index 483ba4d44b5..b5ae95b6c6b 100644 --- a/stream/clients/java/kv/pom.xml +++ b/stream/clients/java/kv/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/stream/clients/java/pom.xml b/stream/clients/java/pom.xml index 5f259858e73..4fc2a6b4875 100644 --- a/stream/clients/java/pom.xml +++ b/stream/clients/java/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + pom 4.0.0 diff --git a/stream/clients/pom.xml b/stream/clients/pom.xml index 3364a4ce5c1..fe8674170fa 100644 --- a/stream/clients/pom.xml +++ b/stream/clients/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + pom 4.0.0 diff --git a/stream/common/pom.xml b/stream/common/pom.xml index 81efe6dad28..6a8c137ce58 100644 --- a/stream/common/pom.xml +++ b/stream/common/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 stream-storage-parent diff --git a/stream/pom.xml b/stream/pom.xml index 291c2731673..39aabd33575 100644 --- a/stream/pom.xml +++ b/stream/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/stream/proto/pom.xml b/stream/proto/pom.xml index e5451cd57fd..8f2129d98c9 100644 --- a/stream/proto/pom.xml +++ b/stream/proto/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/stream/server/pom.xml b/stream/server/pom.xml index eb0cbf7ddb9..9603af7ef75 100644 --- a/stream/server/pom.xml +++ b/stream/server/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/stream/statelib/pom.xml b/stream/statelib/pom.xml index 033ea3d6686..e781a8e74c0 100644 --- a/stream/statelib/pom.xml +++ b/stream/statelib/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/stream/storage/api/pom.xml b/stream/storage/api/pom.xml index 68ed1d481cc..9d0d74a6f01 100644 --- a/stream/storage/api/pom.xml +++ b/stream/storage/api/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/stream/storage/impl/pom.xml b/stream/storage/impl/pom.xml index 0ea22671a9a..122536de829 100644 --- a/stream/storage/impl/pom.xml +++ b/stream/storage/impl/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/stream/storage/pom.xml b/stream/storage/pom.xml index 347dd0a3bae..f6cfa849ccf 100644 --- a/stream/storage/pom.xml +++ b/stream/storage/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + pom 4.0.0 diff --git a/stream/tests-common/pom.xml b/stream/tests-common/pom.xml index 9bdf40af700..189cafb3e9c 100644 --- a/stream/tests-common/pom.xml +++ b/stream/tests-common/pom.xml @@ -15,8 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/tests/integration/cluster/pom.xml b/tests/integration/cluster/pom.xml index 42fbac36540..28c24d3c6f3 100644 --- a/tests/integration/cluster/pom.xml +++ b/tests/integration/cluster/pom.xml @@ -16,7 +16,7 @@ ~ See the License for the specific language governing permissions and ~ limitations under the License. --> - + 4.0.0 org.apache.bookkeeper.tests diff --git a/tools/perf/pom.xml b/tools/perf/pom.xml index a2f8ead608c..889f45c2489 100644 --- a/tools/perf/pom.xml +++ b/tools/perf/pom.xml @@ -12,8 +12,7 @@ ~ See the License for the specific language governing permissions and ~ limitations under the License. --> - + 4.0.0 org.apache.bookkeeper diff --git a/tools/stream/pom.xml b/tools/stream/pom.xml index cc4f12349ed..95407a6ce83 100644 --- a/tools/stream/pom.xml +++ b/tools/stream/pom.xml @@ -16,8 +16,7 @@ ~ See the License for the specific language governing permissions and ~ limitations under the License. --> - + 4.0.0 org.apache.bookkeeper From 03f764515d997fbc53de997931965b1f5b5aa289 Mon Sep 17 00:00:00 2001 From: rdhabalia Date: Sat, 20 Jun 2020 12:03:31 -0700 Subject: [PATCH 0446/1642] [maven-release-plugin] prepare for next development iteration --- bookkeeper-benchmark/pom.xml | 2 +- bookkeeper-common-allocator/pom.xml | 2 +- bookkeeper-common/pom.xml | 2 +- bookkeeper-dist/all/pom.xml | 2 +- bookkeeper-dist/bkctl/pom.xml | 2 +- bookkeeper-dist/pom.xml | 2 +- bookkeeper-dist/server/pom.xml | 2 +- bookkeeper-http/http-server/pom.xml | 2 +- bookkeeper-http/pom.xml | 2 +- bookkeeper-http/vertx-http-server/pom.xml | 2 +- bookkeeper-proto/pom.xml | 2 +- bookkeeper-server/pom.xml | 2 +- bookkeeper-stats-providers/codahale-metrics-provider/pom.xml | 2 +- bookkeeper-stats-providers/pom.xml | 2 +- .../prometheus-metrics-provider/pom.xml | 2 +- bookkeeper-stats/pom.xml | 2 +- buildtools/pom.xml | 4 ++-- circe-checksum/pom.xml | 2 +- cpu-affinity/pom.xml | 2 +- metadata-drivers/etcd/pom.xml | 2 +- metadata-drivers/pom.xml | 2 +- microbenchmarks/pom.xml | 2 +- pom.xml | 4 ++-- shaded/bookkeeper-server-shaded/pom.xml | 2 +- shaded/bookkeeper-server-tests-shaded/pom.xml | 2 +- shaded/distributedlog-core-shaded/pom.xml | 2 +- shaded/pom.xml | 2 +- stats/pom.xml | 2 +- stats/utils/pom.xml | 2 +- stream/api/pom.xml | 2 +- stream/bk-grpc-name-resolver/pom.xml | 2 +- stream/clients/java/all/pom.xml | 2 +- stream/clients/java/base/pom.xml | 2 +- stream/clients/java/kv/pom.xml | 2 +- stream/clients/java/pom.xml | 2 +- stream/clients/pom.xml | 2 +- stream/common/pom.xml | 2 +- stream/pom.xml | 2 +- stream/proto/pom.xml | 2 +- stream/server/pom.xml | 2 +- stream/statelib/pom.xml | 2 +- stream/storage/api/pom.xml | 2 +- stream/storage/impl/pom.xml | 2 +- stream/storage/pom.xml | 2 +- stream/tests-common/pom.xml | 2 +- tests/backward-compat/current-server-old-clients/pom.xml | 2 +- tests/backward-compat/hierarchical-ledger-manager/pom.xml | 2 +- tests/backward-compat/hostname-bookieid/pom.xml | 2 +- tests/backward-compat/old-cookie-new-cluster/pom.xml | 2 +- tests/backward-compat/pom.xml | 2 +- tests/backward-compat/recovery-no-password/pom.xml | 2 +- tests/backward-compat/upgrade-direct/pom.xml | 2 +- tests/backward-compat/upgrade/pom.xml | 2 +- tests/backward-compat/yahoo-custom-version/pom.xml | 2 +- tests/docker-images/all-released-versions-image/pom.xml | 2 +- tests/docker-images/all-versions-image/pom.xml | 2 +- tests/docker-images/current-version-image/pom.xml | 2 +- tests/docker-images/pom.xml | 2 +- tests/integration-tests-base-groovy/pom.xml | 2 +- tests/integration-tests-base/pom.xml | 2 +- tests/integration-tests-topologies/pom.xml | 2 +- tests/integration-tests-utils/pom.xml | 2 +- tests/integration/cluster/pom.xml | 2 +- tests/integration/pom.xml | 2 +- tests/integration/smoke/pom.xml | 2 +- tests/integration/standalone/pom.xml | 2 +- tests/pom.xml | 2 +- tests/scripts/pom.xml | 2 +- tests/shaded/bookkeeper-server-shaded-test/pom.xml | 2 +- tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml | 2 +- tests/shaded/distributedlog-core-shaded-test/pom.xml | 2 +- tests/shaded/pom.xml | 2 +- tools/all/pom.xml | 2 +- tools/framework/pom.xml | 2 +- tools/ledger/pom.xml | 2 +- tools/perf/pom.xml | 2 +- tools/pom.xml | 2 +- tools/stream/pom.xml | 2 +- 78 files changed, 80 insertions(+), 80 deletions(-) diff --git a/bookkeeper-benchmark/pom.xml b/bookkeeper-benchmark/pom.xml index 9259b93e462..81e86e068e8 100644 --- a/bookkeeper-benchmark/pom.xml +++ b/bookkeeper-benchmark/pom.xml @@ -21,7 +21,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper bookkeeper-benchmark diff --git a/bookkeeper-common-allocator/pom.xml b/bookkeeper-common-allocator/pom.xml index 409d239cbff..164a7d4f1ac 100644 --- a/bookkeeper-common-allocator/pom.xml +++ b/bookkeeper-common-allocator/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper-common-allocator Apache BookKeeper :: Common :: Allocator diff --git a/bookkeeper-common/pom.xml b/bookkeeper-common/pom.xml index 49a6a62c12a..06b46fc9148 100644 --- a/bookkeeper-common/pom.xml +++ b/bookkeeper-common/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper-common Apache BookKeeper :: Common diff --git a/bookkeeper-dist/all/pom.xml b/bookkeeper-dist/all/pom.xml index 4b9044da286..8334cc9863a 100644 --- a/bookkeeper-dist/all/pom.xml +++ b/bookkeeper-dist/all/pom.xml @@ -23,7 +23,7 @@ bookkeeper-dist org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/bookkeeper-dist/bkctl/pom.xml b/bookkeeper-dist/bkctl/pom.xml index db12125a703..a28e4f6e98e 100644 --- a/bookkeeper-dist/bkctl/pom.xml +++ b/bookkeeper-dist/bkctl/pom.xml @@ -23,7 +23,7 @@ bookkeeper-dist org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/bookkeeper-dist/pom.xml b/bookkeeper-dist/pom.xml index 23d5a08eb6e..c8390d85f88 100644 --- a/bookkeeper-dist/pom.xml +++ b/bookkeeper-dist/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT 4.0.0 bookkeeper-dist diff --git a/bookkeeper-dist/server/pom.xml b/bookkeeper-dist/server/pom.xml index a23a66d0360..498bd9c79fe 100644 --- a/bookkeeper-dist/server/pom.xml +++ b/bookkeeper-dist/server/pom.xml @@ -23,7 +23,7 @@ bookkeeper-dist org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/bookkeeper-http/http-server/pom.xml b/bookkeeper-http/http-server/pom.xml index 4e90ad790ec..91bf1b469a0 100644 --- a/bookkeeper-http/http-server/pom.xml +++ b/bookkeeper-http/http-server/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT ../.. 4.0.0 diff --git a/bookkeeper-http/pom.xml b/bookkeeper-http/pom.xml index 9b32d8c7208..8f44a0e3b7b 100644 --- a/bookkeeper-http/pom.xml +++ b/bookkeeper-http/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.http diff --git a/bookkeeper-http/vertx-http-server/pom.xml b/bookkeeper-http/vertx-http-server/pom.xml index 63f248f386f..70eec158fd9 100644 --- a/bookkeeper-http/vertx-http-server/pom.xml +++ b/bookkeeper-http/vertx-http-server/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT ../.. 4.0.0 diff --git a/bookkeeper-proto/pom.xml b/bookkeeper-proto/pom.xml index d9b051bbab2..20675eb27f3 100644 --- a/bookkeeper-proto/pom.xml +++ b/bookkeeper-proto/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper-proto Apache BookKeeper :: Protocols diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index dcb43b8a292..e990b4d6347 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper-server Apache BookKeeper :: Server diff --git a/bookkeeper-stats-providers/codahale-metrics-provider/pom.xml b/bookkeeper-stats-providers/codahale-metrics-provider/pom.xml index 2cf7da075f4..ab9b5de2c3c 100644 --- a/bookkeeper-stats-providers/codahale-metrics-provider/pom.xml +++ b/bookkeeper-stats-providers/codahale-metrics-provider/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT ../.. org.apache.bookkeeper.stats diff --git a/bookkeeper-stats-providers/pom.xml b/bookkeeper-stats-providers/pom.xml index 42e36702876..cb4d420525d 100644 --- a/bookkeeper-stats-providers/pom.xml +++ b/bookkeeper-stats-providers/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT 4.0.0 bookkeeper-stats-providers diff --git a/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml b/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml index 006e1662162..939aaa2c370 100644 --- a/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml +++ b/bookkeeper-stats-providers/prometheus-metrics-provider/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT ../.. org.apache.bookkeeper.stats diff --git a/bookkeeper-stats/pom.xml b/bookkeeper-stats/pom.xml index 866ea40a461..98e5bde819c 100644 --- a/bookkeeper-stats/pom.xml +++ b/bookkeeper-stats/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper.stats bookkeeper-stats-api diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 2f5c50a09f4..cb91ce0277b 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -20,9 +20,9 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT buildtools Apache BookKeeper :: Build Tools - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT diff --git a/circe-checksum/pom.xml b/circe-checksum/pom.xml index 30f01bf54a4..966092b07bd 100644 --- a/circe-checksum/pom.xml +++ b/circe-checksum/pom.xml @@ -24,7 +24,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/cpu-affinity/pom.xml b/cpu-affinity/pom.xml index 6400eecddff..3eef2b2fd93 100644 --- a/cpu-affinity/pom.xml +++ b/cpu-affinity/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/metadata-drivers/etcd/pom.xml b/metadata-drivers/etcd/pom.xml index 76910f376c4..899a02b6390 100644 --- a/metadata-drivers/etcd/pom.xml +++ b/metadata-drivers/etcd/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper.metadata.drivers metadata-drivers-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. 4.0.0 diff --git a/metadata-drivers/pom.xml b/metadata-drivers/pom.xml index c832395e0bc..a3fce7bd704 100644 --- a/metadata-drivers/pom.xml +++ b/metadata-drivers/pom.xml @@ -19,7 +19,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.metadata.drivers diff --git a/microbenchmarks/pom.xml b/microbenchmarks/pom.xml index 41c650a7672..42be6163896 100644 --- a/microbenchmarks/pom.xml +++ b/microbenchmarks/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT microbenchmarks Apache BookKeeper :: microbenchmarks diff --git a/pom.xml b/pom.xml index 6bcbd1c4bb4..86802c3dd84 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ 4.0.0 org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper pom Apache BookKeeper :: Parent @@ -40,7 +40,7 @@ scm:git:https://github.com/apache/bookkeeper.git scm:git:https://github.com/apache/bookkeeper.git https://github.com/apache/bookkeeper - branch-4.11 + HEAD JIRA diff --git a/shaded/bookkeeper-server-shaded/pom.xml b/shaded/bookkeeper-server-shaded/pom.xml index 4ab27e6337c..e9a5136f273 100644 --- a/shaded/bookkeeper-server-shaded/pom.xml +++ b/shaded/bookkeeper-server-shaded/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper shaded-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. bookkeeper-server-shaded diff --git a/shaded/bookkeeper-server-tests-shaded/pom.xml b/shaded/bookkeeper-server-tests-shaded/pom.xml index 91e34676768..118df06193b 100644 --- a/shaded/bookkeeper-server-tests-shaded/pom.xml +++ b/shaded/bookkeeper-server-tests-shaded/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper shaded-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. bookkeeper-server-tests-shaded diff --git a/shaded/distributedlog-core-shaded/pom.xml b/shaded/distributedlog-core-shaded/pom.xml index 810014f72b3..63907e2c089 100644 --- a/shaded/distributedlog-core-shaded/pom.xml +++ b/shaded/distributedlog-core-shaded/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper shaded-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. org.apache.distributedlog diff --git a/shaded/pom.xml b/shaded/pom.xml index 989b35f33a7..eacd38a3278 100644 --- a/shaded/pom.xml +++ b/shaded/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper shaded-parent diff --git a/stats/pom.xml b/stats/pom.xml index 9479bd63bb7..7d2b15d3372 100644 --- a/stats/pom.xml +++ b/stats/pom.xml @@ -22,7 +22,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. pom diff --git a/stats/utils/pom.xml b/stats/utils/pom.xml index 4255ad6c97f..e5010b093d0 100644 --- a/stats/utils/pom.xml +++ b/stats/utils/pom.xml @@ -22,7 +22,7 @@ bookkeeper-stats-parent org.apache.bookkeeper.stats - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. org.apache.bookkeeper.stats diff --git a/stream/api/pom.xml b/stream/api/pom.xml index abe9264260d..57396092a5f 100644 --- a/stream/api/pom.xml +++ b/stream/api/pom.xml @@ -20,7 +20,7 @@ stream-storage-parent org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/bk-grpc-name-resolver/pom.xml b/stream/bk-grpc-name-resolver/pom.xml index aff1915cd1e..513869ecd24 100644 --- a/stream/bk-grpc-name-resolver/pom.xml +++ b/stream/bk-grpc-name-resolver/pom.xml @@ -22,7 +22,7 @@ stream-storage-parent org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/clients/java/all/pom.xml b/stream/clients/java/all/pom.xml index 08251d8c335..cfd5193b1aa 100644 --- a/stream/clients/java/all/pom.xml +++ b/stream/clients/java/all/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper stream-storage-java-client-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT stream-storage-java-client Apache BookKeeper :: Stream Storage :: Clients :: Java Client diff --git a/stream/clients/java/base/pom.xml b/stream/clients/java/base/pom.xml index 86edaa05f01..6b71b299acf 100644 --- a/stream/clients/java/base/pom.xml +++ b/stream/clients/java/base/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper stream-storage-java-client-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT stream-storage-java-client-base Apache BookKeeper :: Stream Storage :: Clients :: Java Client :: Base diff --git a/stream/clients/java/kv/pom.xml b/stream/clients/java/kv/pom.xml index b5ae95b6c6b..c4ab3cce8e8 100644 --- a/stream/clients/java/kv/pom.xml +++ b/stream/clients/java/kv/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper stream-storage-java-client-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT stream-storage-java-kv-client Apache BookKeeper :: Stream Storage :: Clients :: Java Client :: KV diff --git a/stream/clients/java/pom.xml b/stream/clients/java/pom.xml index 4fc2a6b4875..dbffaf093e0 100644 --- a/stream/clients/java/pom.xml +++ b/stream/clients/java/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-clients-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. stream-storage-java-client-parent diff --git a/stream/clients/pom.xml b/stream/clients/pom.xml index fe8674170fa..90e6f0a7314 100644 --- a/stream/clients/pom.xml +++ b/stream/clients/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. stream-storage-clients-parent diff --git a/stream/common/pom.xml b/stream/common/pom.xml index 6a8c137ce58..d6d4c23046d 100644 --- a/stream/common/pom.xml +++ b/stream/common/pom.xml @@ -20,7 +20,7 @@ stream-storage-parent org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/pom.xml b/stream/pom.xml index 39aabd33575..d8a1e8f7ac0 100644 --- a/stream/pom.xml +++ b/stream/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. pom diff --git a/stream/proto/pom.xml b/stream/proto/pom.xml index 8f2129d98c9..f98879b282d 100644 --- a/stream/proto/pom.xml +++ b/stream/proto/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper stream-storage-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/server/pom.xml b/stream/server/pom.xml index 9603af7ef75..ececbe85469 100644 --- a/stream/server/pom.xml +++ b/stream/server/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper stream-storage-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT stream-storage-server Apache BookKeeper :: Stream Storage :: Server diff --git a/stream/statelib/pom.xml b/stream/statelib/pom.xml index e781a8e74c0..efcfec3f542 100644 --- a/stream/statelib/pom.xml +++ b/stream/statelib/pom.xml @@ -21,7 +21,7 @@ stream-storage-parent org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. org.apache.bookkeeper diff --git a/stream/storage/api/pom.xml b/stream/storage/api/pom.xml index 9d0d74a6f01..577ba168039 100644 --- a/stream/storage/api/pom.xml +++ b/stream/storage/api/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper stream-storage-service-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. stream-storage-service-api diff --git a/stream/storage/impl/pom.xml b/stream/storage/impl/pom.xml index 122536de829..d7df60998e3 100644 --- a/stream/storage/impl/pom.xml +++ b/stream/storage/impl/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper stream-storage-service-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. stream-storage-service-impl diff --git a/stream/storage/pom.xml b/stream/storage/pom.xml index f6cfa849ccf..6eaddeed597 100644 --- a/stream/storage/pom.xml +++ b/stream/storage/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper stream-storage-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. stream-storage-service-parent diff --git a/stream/tests-common/pom.xml b/stream/tests-common/pom.xml index 189cafb3e9c..ec3e439d11b 100644 --- a/stream/tests-common/pom.xml +++ b/stream/tests-common/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper stream-storage-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. org.apache.bookkeeper.tests diff --git a/tests/backward-compat/current-server-old-clients/pom.xml b/tests/backward-compat/current-server-old-clients/pom.xml index 29fb3c9912e..9cf522be283 100644 --- a/tests/backward-compat/current-server-old-clients/pom.xml +++ b/tests/backward-compat/current-server-old-clients/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests backward-compat - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/backward-compat/hierarchical-ledger-manager/pom.xml b/tests/backward-compat/hierarchical-ledger-manager/pom.xml index 92f937f0d88..b7355cda8cd 100644 --- a/tests/backward-compat/hierarchical-ledger-manager/pom.xml +++ b/tests/backward-compat/hierarchical-ledger-manager/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests backward-compat - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/backward-compat/hostname-bookieid/pom.xml b/tests/backward-compat/hostname-bookieid/pom.xml index fc6cb7db7f1..f41368427a8 100644 --- a/tests/backward-compat/hostname-bookieid/pom.xml +++ b/tests/backward-compat/hostname-bookieid/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests backward-compat - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/backward-compat/old-cookie-new-cluster/pom.xml b/tests/backward-compat/old-cookie-new-cluster/pom.xml index 73d3449de48..b1195036eda 100644 --- a/tests/backward-compat/old-cookie-new-cluster/pom.xml +++ b/tests/backward-compat/old-cookie-new-cluster/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests backward-compat - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/backward-compat/pom.xml b/tests/backward-compat/pom.xml index 503eb6395de..4b3addfb850 100644 --- a/tests/backward-compat/pom.xml +++ b/tests/backward-compat/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests integration-tests-base-groovy - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT ../integration-tests-base-groovy org.apache.bookkeeper.tests diff --git a/tests/backward-compat/recovery-no-password/pom.xml b/tests/backward-compat/recovery-no-password/pom.xml index b6c7c03cbdb..f1f7f69095f 100644 --- a/tests/backward-compat/recovery-no-password/pom.xml +++ b/tests/backward-compat/recovery-no-password/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests backward-compat - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/backward-compat/upgrade-direct/pom.xml b/tests/backward-compat/upgrade-direct/pom.xml index d7d388c04a9..4aa55304836 100644 --- a/tests/backward-compat/upgrade-direct/pom.xml +++ b/tests/backward-compat/upgrade-direct/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests backward-compat - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/backward-compat/upgrade/pom.xml b/tests/backward-compat/upgrade/pom.xml index 95e66180267..98a887e593d 100644 --- a/tests/backward-compat/upgrade/pom.xml +++ b/tests/backward-compat/upgrade/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests backward-compat - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/backward-compat/yahoo-custom-version/pom.xml b/tests/backward-compat/yahoo-custom-version/pom.xml index eed8d032def..164c232fb06 100644 --- a/tests/backward-compat/yahoo-custom-version/pom.xml +++ b/tests/backward-compat/yahoo-custom-version/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests backward-compat - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/docker-images/all-released-versions-image/pom.xml b/tests/docker-images/all-released-versions-image/pom.xml index d5896624997..a7589824d08 100644 --- a/tests/docker-images/all-released-versions-image/pom.xml +++ b/tests/docker-images/all-released-versions-image/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper.tests docker-images - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.tests diff --git a/tests/docker-images/all-versions-image/pom.xml b/tests/docker-images/all-versions-image/pom.xml index 7bba2a29ba4..ea382c19221 100644 --- a/tests/docker-images/all-versions-image/pom.xml +++ b/tests/docker-images/all-versions-image/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper.tests docker-images - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.tests diff --git a/tests/docker-images/current-version-image/pom.xml b/tests/docker-images/current-version-image/pom.xml index b3305e28e59..cbcc013ecfd 100644 --- a/tests/docker-images/current-version-image/pom.xml +++ b/tests/docker-images/current-version-image/pom.xml @@ -19,7 +19,7 @@ org.apache.bookkeeper.tests docker-images - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT 4.0.0 org.apache.bookkeeper.tests diff --git a/tests/docker-images/pom.xml b/tests/docker-images/pom.xml index 3107d5531f1..de16b7584e0 100644 --- a/tests/docker-images/pom.xml +++ b/tests/docker-images/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper.tests docker-images diff --git a/tests/integration-tests-base-groovy/pom.xml b/tests/integration-tests-base-groovy/pom.xml index f48f8105089..d123a8f400d 100644 --- a/tests/integration-tests-base-groovy/pom.xml +++ b/tests/integration-tests-base-groovy/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests integration-tests-base - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT ../integration-tests-base diff --git a/tests/integration-tests-base/pom.xml b/tests/integration-tests-base/pom.xml index 822710af9a7..40eb5cb1622 100644 --- a/tests/integration-tests-base/pom.xml +++ b/tests/integration-tests-base/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper.tests diff --git a/tests/integration-tests-topologies/pom.xml b/tests/integration-tests-topologies/pom.xml index 56397790059..96605a70450 100644 --- a/tests/integration-tests-topologies/pom.xml +++ b/tests/integration-tests-topologies/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper.tests diff --git a/tests/integration-tests-utils/pom.xml b/tests/integration-tests-utils/pom.xml index a7ef17bb219..9f2ba029d5b 100644 --- a/tests/integration-tests-utils/pom.xml +++ b/tests/integration-tests-utils/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper.tests diff --git a/tests/integration/cluster/pom.xml b/tests/integration/cluster/pom.xml index 28c24d3c6f3..9f0e1d294d4 100644 --- a/tests/integration/cluster/pom.xml +++ b/tests/integration/cluster/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests integration - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index ded0de07068..88eca62bbaf 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper.tests integration diff --git a/tests/integration/smoke/pom.xml b/tests/integration/smoke/pom.xml index 4add9c159ad..1ad9933db9d 100644 --- a/tests/integration/smoke/pom.xml +++ b/tests/integration/smoke/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests integration - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/integration/standalone/pom.xml b/tests/integration/standalone/pom.xml index 893db1b2287..f6c4de94f0d 100644 --- a/tests/integration/standalone/pom.xml +++ b/tests/integration/standalone/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests integration - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. diff --git a/tests/pom.xml b/tests/pom.xml index 79d7df4a5a3..8ce2a5d1ac3 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper.tests tests-parent diff --git a/tests/scripts/pom.xml b/tests/scripts/pom.xml index 8fe739431f5..601ad740e8d 100644 --- a/tests/scripts/pom.xml +++ b/tests/scripts/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT org.apache.bookkeeper.tests diff --git a/tests/shaded/bookkeeper-server-shaded-test/pom.xml b/tests/shaded/bookkeeper-server-shaded-test/pom.xml index b0f145b0512..18d7ebd2f56 100644 --- a/tests/shaded/bookkeeper-server-shaded-test/pom.xml +++ b/tests/shaded/bookkeeper-server-shaded-test/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.shaded shaded-tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. bookkeeper-server-shaded-test diff --git a/tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml b/tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml index 4ccb0314625..3286adfc0c8 100644 --- a/tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml +++ b/tests/shaded/bookkeeper-server-tests-shaded-test/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.shaded shaded-tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. bookkeeper-server-tests-shaded-test diff --git a/tests/shaded/distributedlog-core-shaded-test/pom.xml b/tests/shaded/distributedlog-core-shaded-test/pom.xml index e7a7518f387..558f84a0799 100644 --- a/tests/shaded/distributedlog-core-shaded-test/pom.xml +++ b/tests/shaded/distributedlog-core-shaded-test/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper.tests.shaded shaded-tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. distributedlog-core-shaded-test diff --git a/tests/shaded/pom.xml b/tests/shaded/pom.xml index 71ce9dd93c2..b7c6e62f683 100644 --- a/tests/shaded/pom.xml +++ b/tests/shaded/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper.tests tests-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT .. org.apache.bookkeeper.tests.shaded diff --git a/tools/all/pom.xml b/tools/all/pom.xml index d38523421a1..476f0ee1aad 100644 --- a/tools/all/pom.xml +++ b/tools/all/pom.xml @@ -22,7 +22,7 @@ bookkeeper-tools-parent org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper-tools Apache BookKeeper :: Tools diff --git a/tools/framework/pom.xml b/tools/framework/pom.xml index 4d3bdaa64ba..ff0bfce4bba 100644 --- a/tools/framework/pom.xml +++ b/tools/framework/pom.xml @@ -20,7 +20,7 @@ bookkeeper-tools-parent org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper-tools-framework Apache BookKeeper :: Tools :: Framework diff --git a/tools/ledger/pom.xml b/tools/ledger/pom.xml index 2b3c55a2a91..fefad89e0db 100644 --- a/tools/ledger/pom.xml +++ b/tools/ledger/pom.xml @@ -20,7 +20,7 @@ bookkeeper-tools-parent org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper-tools-ledger Apache BookKeeper :: Tools :: Ledger diff --git a/tools/perf/pom.xml b/tools/perf/pom.xml index 889f45c2489..d4c1d648b1b 100644 --- a/tools/perf/pom.xml +++ b/tools/perf/pom.xml @@ -17,7 +17,7 @@ org.apache.bookkeeper bookkeeper-tools-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper-perf Apache BookKeeper :: Tools :: Perf diff --git a/tools/pom.xml b/tools/pom.xml index 76f91036199..d95545aec9e 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ bookkeeper org.apache.bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT bookkeeper-tools-parent Apache BookKeeper :: Tools :: Parent diff --git a/tools/stream/pom.xml b/tools/stream/pom.xml index 95407a6ce83..9629d1d71a7 100644 --- a/tools/stream/pom.xml +++ b/tools/stream/pom.xml @@ -21,7 +21,7 @@ org.apache.bookkeeper bookkeeper-tools-parent - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT stream-storage-cli Apache BookKeeper :: Tools :: Stream From f9d86034b785052cb52cabb75686d4c67445b96e Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 24 Jun 2020 08:37:39 +0200 Subject: [PATCH 0447/1642] Fix distributedlog Maven reactor module - fix version - add explicitly distributedlog to the modules of Stream project - fix checkstyle on distributedlog, it wasn't executed Reviewers: Rajan Dhabalia This closes #2364 from eolivelli/fix/fix-dlog-dep --- stream/distributedlog/common/pom.xml | 2 +- stream/distributedlog/core/pom.xml | 5 +++-- .../impl/metadata/ZKLogStreamMetadataStore.java | 2 +- .../impl/subscription/ZKSubscriptionStateStore.java | 2 +- stream/distributedlog/io/dlfs/pom.xml | 2 +- stream/distributedlog/io/pom.xml | 2 +- stream/distributedlog/pom.xml | 2 +- stream/distributedlog/protocol/pom.xml | 2 +- stream/pom.xml | 1 + 9 files changed, 11 insertions(+), 9 deletions(-) diff --git a/stream/distributedlog/common/pom.xml b/stream/distributedlog/common/pom.xml index fc318d9a745..2f50d31b2c7 100644 --- a/stream/distributedlog/common/pom.xml +++ b/stream/distributedlog/common/pom.xml @@ -20,7 +20,7 @@ org.apache.distributedlog distributedlog - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT distributedlog-common Apache BookKeeper :: DistributedLog :: Common diff --git a/stream/distributedlog/core/pom.xml b/stream/distributedlog/core/pom.xml index 65129dbd4f9..c08f9390e32 100644 --- a/stream/distributedlog/core/pom.xml +++ b/stream/distributedlog/core/pom.xml @@ -20,7 +20,7 @@ org.apache.distributedlog distributedlog - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT distributedlog-core Apache BookKeeper :: DistributedLog :: Core Library @@ -112,7 +112,8 @@ org.apache.maven.plugins maven-checkstyle-plugin - buildtools/src/main/resources/distributedlog/suppressions.xml + ../../buildtools/src/main/resources/distributedlog/suppressions.xml + ../../buildtools/src/main/resources/bookkeeper/checkstyle.xml **/thrift/**/* diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java index 9a8886e785f..e16cec88705 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java @@ -17,9 +17,9 @@ */ package org.apache.distributedlog.impl.metadata; -import static java.nio.charset.StandardCharsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.distributedlog.DistributedLogConstants.EMPTY_BYTES; import static org.apache.distributedlog.DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO; import static org.apache.distributedlog.metadata.LogMetadata.ALLOCATION_PATH; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java index a1c2a9f969d..bd60b4179c8 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java @@ -17,8 +17,8 @@ */ package org.apache.distributedlog.impl.subscription; -import java.nio.charset.StandardCharsets; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import org.apache.bookkeeper.common.concurrent.FutureUtils; diff --git a/stream/distributedlog/io/dlfs/pom.xml b/stream/distributedlog/io/dlfs/pom.xml index 9c5195fa075..ab2990f98d7 100644 --- a/stream/distributedlog/io/dlfs/pom.xml +++ b/stream/distributedlog/io/dlfs/pom.xml @@ -20,7 +20,7 @@ distributedlog org.apache.distributedlog - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT ../.. org.apache.distributedlog diff --git a/stream/distributedlog/io/pom.xml b/stream/distributedlog/io/pom.xml index 92827d4d9b6..81c4ebd0428 100644 --- a/stream/distributedlog/io/pom.xml +++ b/stream/distributedlog/io/pom.xml @@ -19,7 +19,7 @@ org.apache.distributedlog distributedlog - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT 4.0.0 distributedlog-io diff --git a/stream/distributedlog/pom.xml b/stream/distributedlog/pom.xml index ff537c47188..afb35811ace 100644 --- a/stream/distributedlog/pom.xml +++ b/stream/distributedlog/pom.xml @@ -20,7 +20,7 @@ org.apache.bookkeeper bookkeeper - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT ../.. org.apache.distributedlog diff --git a/stream/distributedlog/protocol/pom.xml b/stream/distributedlog/protocol/pom.xml index 09da6bd847b..6301ff2e7f2 100644 --- a/stream/distributedlog/protocol/pom.xml +++ b/stream/distributedlog/protocol/pom.xml @@ -20,7 +20,7 @@ org.apache.distributedlog distributedlog - 4.11.0-SNAPSHOT + 4.12.0-SNAPSHOT distributedlog-protocol Apache BookKeeper :: DistributedLog :: Protocol diff --git a/stream/pom.xml b/stream/pom.xml index d8a1e8f7ac0..73caf4c8864 100644 --- a/stream/pom.xml +++ b/stream/pom.xml @@ -29,6 +29,7 @@ Apache BookKeeper :: Stream Storage :: Parent + distributedlog common tests-common statelib From 358fe3e6a068531cf862d3ea97e71528bd756817 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 7 Jul 2020 23:21:31 -0700 Subject: [PATCH 0448/1642] Bump Python client version: 4.12.0-alpha-0 ### Motivation Bump up python client version 4.12.0-alpha-0 Reviewers: Enrico Olivelli This closes #2370 from rdhabalia/py_412 --- stream/clients/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py index 747388ebb13..172545ba9e8 100644 --- a/stream/clients/python/setup.py +++ b/stream/clients/python/setup.py @@ -19,7 +19,7 @@ name = 'apache-bookkeeper-client' description = 'Apache BookKeeper client library' -version = '4.11.0' +version = '4.12.0-alpha-0' # Should be one of: # 'Development Status :: 3 - Alpha' # 'Development Status :: 4 - Beta' From 4ce88eb6ec753eccef5dd399b5d30a06b4bec3bf Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 7 Jul 2020 23:22:20 -0700 Subject: [PATCH 0449/1642] Release note 4.11 note ### Motivation Adding Release-note for 4.11.0 Please review commit https://github.com/apache/bookkeeper/pull/2361/commits/98dbb911353b4fa2262ae8ee79210f7b81bbf7c8 *Note:* This PR is based on #2360 . I will rebase this PR once #2360 is merged. Reviewers: Enrico Olivelli This closes #2361 from rdhabalia/release_note_4.11_note --- site/_config.yml | 6 +- site/docs/4.11.0/admin/autorecovery.md | 125 +++ site/docs/4.11.0/admin/bookies.md | 174 ++++ site/docs/4.11.0/admin/decomission.md | 41 + site/docs/4.11.0/admin/geo-replication.md | 22 + site/docs/4.11.0/admin/http.md | 509 +++++++++++ site/docs/4.11.0/admin/metrics.md | 38 + site/docs/4.11.0/admin/perf.md | 3 + site/docs/4.11.0/admin/placement.md | 3 + site/docs/4.11.0/admin/upgrade.md | 175 ++++ site/docs/4.11.0/api/distributedlog-api.md | 395 ++++++++ site/docs/4.11.0/api/ledger-adv-api.md | 111 +++ site/docs/4.11.0/api/ledger-api.md | 840 ++++++++++++++++++ site/docs/4.11.0/api/overview.md | 17 + site/docs/4.11.0/deployment/dcos.md | 142 +++ site/docs/4.11.0/deployment/kubernetes.md | 181 ++++ site/docs/4.11.0/deployment/manual.md | 58 ++ site/docs/4.11.0/development/codebase.md | 3 + site/docs/4.11.0/development/protocol.md | 148 +++ site/docs/4.11.0/getting-started/concepts.md | 202 +++++ .../4.11.0/getting-started/installation.md | 82 ++ .../4.11.0/getting-started/run-locally.md | 16 + site/docs/4.11.0/overview/overview.md | 60 ++ site/docs/4.11.0/overview/releaseNotes.md | 37 + .../4.11.0/overview/releaseNotesTemplate.md | 17 + site/docs/4.11.0/reference/cli.md | 10 + site/docs/4.11.0/reference/config.md | 9 + site/docs/4.11.0/reference/metrics.md | 3 + site/docs/4.11.0/security/overview.md | 21 + site/docs/4.11.0/security/sasl.md | 202 +++++ site/docs/4.11.0/security/tls.md | 210 +++++ site/docs/4.11.0/security/zookeeper.md | 41 + site/docs/4.11.1/admin/autorecovery.md | 125 +++ site/docs/4.11.1/admin/bookies.md | 174 ++++ site/docs/4.11.1/admin/decomission.md | 41 + site/docs/4.11.1/admin/geo-replication.md | 22 + site/docs/4.11.1/admin/http.md | 509 +++++++++++ site/docs/4.11.1/admin/metrics.md | 38 + site/docs/4.11.1/admin/perf.md | 3 + site/docs/4.11.1/admin/placement.md | 3 + site/docs/4.11.1/admin/upgrade.md | 175 ++++ site/docs/4.11.1/api/distributedlog-api.md | 395 ++++++++ site/docs/4.11.1/api/ledger-adv-api.md | 111 +++ site/docs/4.11.1/api/ledger-api.md | 840 ++++++++++++++++++ site/docs/4.11.1/api/overview.md | 17 + site/docs/4.11.1/deployment/dcos.md | 142 +++ site/docs/4.11.1/deployment/kubernetes.md | 181 ++++ site/docs/4.11.1/deployment/manual.md | 58 ++ site/docs/4.11.1/development/codebase.md | 3 + site/docs/4.11.1/development/protocol.md | 148 +++ site/docs/4.11.1/getting-started/concepts.md | 202 +++++ .../4.11.1/getting-started/installation.md | 82 ++ .../4.11.1/getting-started/run-locally.md | 16 + site/docs/4.11.1/overview/overview.md | 60 ++ site/docs/4.11.1/overview/releaseNotes.md | 17 + .../4.11.1/overview/releaseNotesTemplate.md | 17 + site/docs/4.11.1/reference/cli.md | 10 + site/docs/4.11.1/reference/config.md | 9 + site/docs/4.11.1/reference/metrics.md | 3 + site/docs/4.11.1/security/overview.md | 21 + site/docs/4.11.1/security/sasl.md | 202 +++++ site/docs/4.11.1/security/tls.md | 210 +++++ site/docs/4.11.1/security/zookeeper.md | 41 + site/docs/latest/overview/overview.md | 2 +- site/docs/latest/overview/releaseNotes.md | 4 +- .../latest/overview/releaseNotesTemplate.md | 2 +- site/releases.md | 8 + 67 files changed, 7786 insertions(+), 6 deletions(-) create mode 100644 site/docs/4.11.0/admin/autorecovery.md create mode 100644 site/docs/4.11.0/admin/bookies.md create mode 100644 site/docs/4.11.0/admin/decomission.md create mode 100644 site/docs/4.11.0/admin/geo-replication.md create mode 100644 site/docs/4.11.0/admin/http.md create mode 100644 site/docs/4.11.0/admin/metrics.md create mode 100644 site/docs/4.11.0/admin/perf.md create mode 100644 site/docs/4.11.0/admin/placement.md create mode 100644 site/docs/4.11.0/admin/upgrade.md create mode 100644 site/docs/4.11.0/api/distributedlog-api.md create mode 100644 site/docs/4.11.0/api/ledger-adv-api.md create mode 100644 site/docs/4.11.0/api/ledger-api.md create mode 100644 site/docs/4.11.0/api/overview.md create mode 100644 site/docs/4.11.0/deployment/dcos.md create mode 100644 site/docs/4.11.0/deployment/kubernetes.md create mode 100644 site/docs/4.11.0/deployment/manual.md create mode 100644 site/docs/4.11.0/development/codebase.md create mode 100644 site/docs/4.11.0/development/protocol.md create mode 100644 site/docs/4.11.0/getting-started/concepts.md create mode 100644 site/docs/4.11.0/getting-started/installation.md create mode 100644 site/docs/4.11.0/getting-started/run-locally.md create mode 100644 site/docs/4.11.0/overview/overview.md create mode 100644 site/docs/4.11.0/overview/releaseNotes.md create mode 100644 site/docs/4.11.0/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.11.0/reference/cli.md create mode 100644 site/docs/4.11.0/reference/config.md create mode 100644 site/docs/4.11.0/reference/metrics.md create mode 100644 site/docs/4.11.0/security/overview.md create mode 100644 site/docs/4.11.0/security/sasl.md create mode 100644 site/docs/4.11.0/security/tls.md create mode 100644 site/docs/4.11.0/security/zookeeper.md create mode 100644 site/docs/4.11.1/admin/autorecovery.md create mode 100644 site/docs/4.11.1/admin/bookies.md create mode 100644 site/docs/4.11.1/admin/decomission.md create mode 100644 site/docs/4.11.1/admin/geo-replication.md create mode 100644 site/docs/4.11.1/admin/http.md create mode 100644 site/docs/4.11.1/admin/metrics.md create mode 100644 site/docs/4.11.1/admin/perf.md create mode 100644 site/docs/4.11.1/admin/placement.md create mode 100644 site/docs/4.11.1/admin/upgrade.md create mode 100644 site/docs/4.11.1/api/distributedlog-api.md create mode 100644 site/docs/4.11.1/api/ledger-adv-api.md create mode 100644 site/docs/4.11.1/api/ledger-api.md create mode 100644 site/docs/4.11.1/api/overview.md create mode 100644 site/docs/4.11.1/deployment/dcos.md create mode 100644 site/docs/4.11.1/deployment/kubernetes.md create mode 100644 site/docs/4.11.1/deployment/manual.md create mode 100644 site/docs/4.11.1/development/codebase.md create mode 100644 site/docs/4.11.1/development/protocol.md create mode 100644 site/docs/4.11.1/getting-started/concepts.md create mode 100644 site/docs/4.11.1/getting-started/installation.md create mode 100644 site/docs/4.11.1/getting-started/run-locally.md create mode 100644 site/docs/4.11.1/overview/overview.md create mode 100644 site/docs/4.11.1/overview/releaseNotes.md create mode 100644 site/docs/4.11.1/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.11.1/reference/cli.md create mode 100644 site/docs/4.11.1/reference/config.md create mode 100644 site/docs/4.11.1/reference/metrics.md create mode 100644 site/docs/4.11.1/security/overview.md create mode 100644 site/docs/4.11.1/security/sasl.md create mode 100644 site/docs/4.11.1/security/tls.md create mode 100644 site/docs/4.11.1/security/zookeeper.md diff --git a/site/_config.yml b/site/_config.yml index b293812c62e..2611a8333c4 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -8,6 +8,8 @@ destination: local-generated twitter_url: https://twitter.com/asfbookkeeper versions: +- "4.11.1" +- "4.11.0" - "4.10.0" - "4.9.2" - "4.9.1" @@ -37,8 +39,8 @@ archived_versions: - "4.2.0" - "4.1.0" - "4.0.0" -latest_version: "4.11.0-SNAPSHOT" -latest_release: "4.10.0" +latest_version: "4.12.0-SNAPSHOT" +latest_release: "4.11.1" stable_release: "4.9.2" eol_releases: - "4.7.3" diff --git a/site/docs/4.11.0/admin/autorecovery.md b/site/docs/4.11.0/admin/autorecovery.md new file mode 100644 index 00000000000..4ff881c929a --- /dev/null +++ b/site/docs/4.11.0/admin/autorecovery.md @@ -0,0 +1,125 @@ +--- +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: + +* the `shell recover` option +* the IP and port for the failed bookie + +Here's an example: + +```bash +$ bin/bookkeeper shell recover \ + 192.168.1.10:3181 # IP and port for the failed bookie +``` + +If you wish, you can also specify which ledgers you'd like to recover. Here's an example: + +```bash +$ bin/bookkeeper shell recover \ + 192.168.1.10:3181 \ # IP and port for the failed bookie + --ledger ledgerID # ledgerID which you want to recover +``` + +### 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 +$ 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 +$ bin/bookkeeper shell autorecovery -disable +``` + +Once disabled, you can reenable AutoRecovery using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: + +```bash +$ 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.11.0/admin/bookies.md b/site/docs/4.11.0/admin/bookies.md new file mode 100644 index 00000000000..418c4ca1b79 --- /dev/null +++ b/site/docs/4.11.0/admin/bookies.md @@ -0,0 +1,174 @@ +--- +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 +$ 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 +$ 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 +$ 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 +$ 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 +$ bin/bookkeeper upgrade --upgrade +``` + +...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... + +```shell +$ bin/bookkeeper upgrade --finalize +``` + +...and then restart the server: + +```shell +$ bin/bookkeeper bookie +``` + +If something has gone wrong, you can always perform a rollback: + +```shell +$ 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 +$ 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 +$ 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 shell recover + ``` + + 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 shell recover 192.168.1.10:3181 + ``` + + See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.11.0/admin/decomission.md b/site/docs/4.11.0/admin/decomission.md new file mode 100644 index 00000000000..842b2508b35 --- /dev/null +++ b/site/docs/4.11.0/admin/decomission.md @@ -0,0 +1,41 @@ +--- +title: Decommission Bookies +--- + +In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the +decommissioning was safely done. + +### Before we decommission +1. Ensure state of your cluster can support the decommissioning of the target bookie. +Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie + +2. Ensure target bookie shows up in the listbookies command. + +3. Ensure that there is no other process ongoing (upgrade etc). + +### Process of Decommissioning +1. Log on to the bookie node, check if there are underreplicated ledgers. + +If there are, the decommission command will force them to be replicated. +`$ bin/bookkeeper shell listunderreplicated` + +2. Stop the bookie +`$ bin/bookkeeper-daemon.sh stop bookie` + +3. Run the decommission command. +If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` +If you are running the decommission command for target bookie node from another bookie node you should mention +the target bookie id in the arguments for `-bookieid` +`$ bin/bookkeeper shell decommissionbookie` +or +`$ bin/bookkeeper shell decommissionbookie -bookieid ` + +4. Validate that there are no ledgers on decommissioned bookie +`$ bin/bookkeeper shell listledgers -bookieid ` + +Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: + +```bash +./bookkeeper shell listbookies -rw -h +./bookkeeper shell listbookies -ro -h +``` diff --git a/site/docs/4.11.0/admin/geo-replication.md b/site/docs/4.11.0/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.11.0/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.11.0/admin/http.md b/site/docs/4.11.0/admin/http.md new file mode 100644 index 00000000000..50c3a7db01d --- /dev/null +++ b/site/docs/4.11.0/admin/http.md @@ -0,0 +1,509 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 5 components: +1. Heartbeat: heartbeat for a specific bookie. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Config + +### Endpoint: /metrics +1. Method: GET + * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/info +1. Method: GET + * Description: Get bookie info + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |501 | Not implemented | + * Body: + ```json + { + "freeSpace" : 0, + "totalSpace" : 0 + } + ``` + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/bookie/gc +1. Method: PUT + * Description: trigger gc for this bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: GET + * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + { + "is_in_force_gc" : "false" + } + ``` + +### Endpoint: /api/v1/bookie/gc_details +1. Method: GET + * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + [ { + "forceCompacting" : false, + "majorCompacting" : false, + "minorCompacting" : false, + "lastMajorCompactionTime" : 1544578144944, + "lastMinorCompactionTime" : 1544578144944, + "majorCompactionCounter" : 1, + "minorCompactionCounter" : 0 + } ] + ``` + +### Endpoint: /api/v1/bookie/state +1. Method: GET + * Description: Exposes the current state of bookie + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + { + "running" : true, + "readOnly" : false, + "shuttingDown" : false, + "availableForHighPriorityWrites" : true + } + ``` + +### Endpoint: /api/v1/bookie/is_ready +1. Method: GET + * Description: Return true if the bookie is ready + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + |503 | Bookie is not ready | + * Body: + + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.11.0/admin/metrics.md b/site/docs/4.11.0/admin/metrics.md new file mode 100644 index 00000000000..dca236489d2 --- /dev/null +++ b/site/docs/4.11.0/admin/metrics.md @@ -0,0 +1,38 @@ +--- +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 these 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.prometheus.PrometheusMetricsProvider` + +> 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.11.0/admin/perf.md b/site/docs/4.11.0/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.11.0/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.11.0/admin/placement.md b/site/docs/4.11.0/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.11.0/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.11.0/admin/upgrade.md b/site/docs/4.11.0/admin/upgrade.md new file mode 100644 index 00000000000..57c65208131 --- /dev/null +++ b/site/docs/4.11.0/admin/upgrade.md @@ -0,0 +1,175 @@ +--- +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.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.7.x to 4.8.X upgrade + +In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.11.0/api/distributedlog-api.md b/site/docs/4.11.0/api/distributedlog-api.md new file mode 100644 index 00000000000..a1ca9ef8116 --- /dev/null +++ b/site/docs/4.11.0/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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.11.0/api/ledger-adv-api.md b/site/docs/4.11.0/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.11.0/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.11.0/api/ledger-api.md b/site/docs/4.11.0/api/ledger-api.md new file mode 100644 index 00000000000..60cec16b3bc --- /dev/null +++ b/site/docs/4.11.0/api/ledger-api.md @@ -0,0 +1,840 @@ +--- +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.11.0 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.11.0 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.11.0' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.11.0' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 be deleted synchronously which may throw exception: + +```java +long ledgerId = 1234; + +try { + bkClient.deleteLedger(ledgerId); +} catch (Exception e) { + e.printStackTrace(); +} +``` + +### Delete entries asynchronously + +{% pop Ledgers %} can also be deleted asynchronously: + +```java +class DeleteEntryCallback implements AsyncCallback.DeleteCallback { + public void deleteComplete() { + System.out.println("Delete completed"); + } +} +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); +``` + +## 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 +$ 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` + +### Relaxing Durability + +In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. +In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will +be always readable by other clients. + +On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after +writing the entry to SO buffers without waiting for an fsync. +In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. +Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. + +In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble ackknowledge the call after +performing an fsync to the disk which is storing the journal. +This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. + +The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. +In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. + + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete + + +wh.force().get(); // wait for fsync, make data available to readers and to the replicator + +wh.close(); // seal the ledger + +``` diff --git a/site/docs/4.11.0/api/overview.md b/site/docs/4.11.0/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.11.0/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.11.0/deployment/dcos.md b/site/docs/4.11.0/deployment/dcos.md new file mode 100644 index 00000000000..de67539bd5a --- /dev/null +++ b/site/docs/4.11.0/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.11.0] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.11.0/deployment/kubernetes.md b/site/docs/4.11.0/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.11.0/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.11.0/deployment/manual.md b/site/docs/4.11.0/deployment/manual.md new file mode 100644 index 00000000000..89b3b0ec649 --- /dev/null +++ b/site/docs/4.11.0/deployment/manual.md @@ -0,0 +1,58 @@ +--- +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. + +## Cluster metadata setup + +Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! + +## Starting up bookies + + + +Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: + +```shell +$ bookkeeper-server/bin/bookkeeper bookie +``` + + +### System requirements + +{% include system-requirements.md %} + + + diff --git a/site/docs/4.11.0/development/codebase.md b/site/docs/4.11.0/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.11.0/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.11.0/development/protocol.md b/site/docs/4.11.0/development/protocol.md new file mode 100644 index 00000000000..f4410444427 --- /dev/null +++ b/site/docs/4.11.0/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 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.11.0/getting-started/concepts.md b/site/docs/4.11.0/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.11.0/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.11.0/getting-started/installation.md b/site/docs/4.11.0/getting-started/installation.md new file mode 100644 index 00000000000..9f856ec1d21 --- /dev/null +++ b/site/docs/4.11.0/getting-started/installation.md @@ -0,0 +1,82 @@ +--- +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.8](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 +``` + +Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. + +```shell +$ mvn package -Dstream +``` + +> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin + +> You can enable `table service` by adding the `-Dstream` flag when running above commands. + +## 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.11.0/getting-started/run-locally.md b/site/docs/4.11.0/getting-started/run-locally.md new file mode 100644 index 00000000000..520140317d9 --- /dev/null +++ b/site/docs/4.11.0/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 +$ bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.11.0/overview/overview.md b/site/docs/4.11.0/overview/overview.md new file mode 100644 index 00000000000..71ca50af93d --- /dev/null +++ b/site/docs/4.11.0/overview/overview.md @@ -0,0 +1,60 @@ +--- +title: Apache BookKeeper™ 4.11.0 +--- + + +This documentation is for Apache BookKeeper™ version 4.11.0. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.11.0 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.11.0/overview/releaseNotes.md b/site/docs/4.11.0/overview/releaseNotes.md new file mode 100644 index 00000000000..a5cb8946941 --- /dev/null +++ b/site/docs/4.11.0/overview/releaseNotes.md @@ -0,0 +1,37 @@ +--- +title: Apache BookKeeper 4.11.0 Release Notes +--- + +This is the 21th release of Apache BookKeeper! + +The 4.11.0 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.10.0. + +Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.11.0](../../admin/upgrade). The technical details of +this release are summarized below. + +## News and noteworthy + +- Upgraded ZooKeeper version from `3.4.13` to `3.5.7` with #2112 +- BookKeeper-server depends on `org.apache.httpcomponents-httpcore-4.4.9` with #2156 + + +### Changes + +- [https://github.com/apache/bookkeeper/pull/2338] Fix bookie port conflict when using LocalBookKeeper +- [https://github.com/apache/bookkeeper/pull/2333] Handle QuorumCoverage should only count unknown nodes +- [https://github.com/apache/bookkeeper/pull/2326] Update jackson version 2.11.0 +- [https://github.com/apache/bookkeeper/pull/2314] BP-38: Publish Bookie Service Info including all advertised addresses on Metadata Service and it is backward compatible +- [https://github.com/apache/bookkeeper/pull/2313] add REST API to manage auto-recovery +- [https://github.com/apache/bookkeeper/pull/2312] Support metadata decoding for list-ledger api +- [https://github.com/apache/bookkeeper/pull/2300] files: Fix TLS with with v2 protocol +- [https://github.com/apache/bookkeeper/pull/2297] Update Arquillian Cube to 1.18.2 +- [https://github.com/apache/bookkeeper/pull/2291] Update Prometheus library to 0.8.1 +- [https://github.com/apache/bookkeeper/pull/2205] Handle empty ledger segmant while replica-check +- [https://github.com/apache/bookkeeper/pull/2156] Add Hostname verification for bookie-mTLS +- [https://github.com/apache/bookkeeper/pull/2112] Update ZooKeeper dependency to 3.5.7 + + + +## Full list of changes + +- [https://github.com/apache/bookkeeper/milestone/7](https://github.com/apache/bookkeeper/milestone/6?closed=1) \ No newline at end of file diff --git a/site/docs/4.11.0/overview/releaseNotesTemplate.md b/site/docs/4.11.0/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..9a51cda9fab --- /dev/null +++ b/site/docs/4.11.0/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.11.0 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.11.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.11.0/reference/cli.md b/site/docs/4.11.0/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.11.0/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.11.0/reference/config.md b/site/docs/4.11.0/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.11.0/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.11.0/reference/metrics.md b/site/docs/4.11.0/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.11.0/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.11.0/security/overview.md b/site/docs/4.11.0/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.11.0/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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.11.0/security/sasl.md b/site/docs/4.11.0/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.11.0/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.11.0/security/tls.md b/site/docs/4.11.0/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.11.0/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.11.0/security/zookeeper.md b/site/docs/4.11.0/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.11.0/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) diff --git a/site/docs/4.11.1/admin/autorecovery.md b/site/docs/4.11.1/admin/autorecovery.md new file mode 100644 index 00000000000..4ff881c929a --- /dev/null +++ b/site/docs/4.11.1/admin/autorecovery.md @@ -0,0 +1,125 @@ +--- +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: + +* the `shell recover` option +* the IP and port for the failed bookie + +Here's an example: + +```bash +$ bin/bookkeeper shell recover \ + 192.168.1.10:3181 # IP and port for the failed bookie +``` + +If you wish, you can also specify which ledgers you'd like to recover. Here's an example: + +```bash +$ bin/bookkeeper shell recover \ + 192.168.1.10:3181 \ # IP and port for the failed bookie + --ledger ledgerID # ledgerID which you want to recover +``` + +### 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 +$ 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 +$ bin/bookkeeper shell autorecovery -disable +``` + +Once disabled, you can reenable AutoRecovery using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: + +```bash +$ 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.11.1/admin/bookies.md b/site/docs/4.11.1/admin/bookies.md new file mode 100644 index 00000000000..418c4ca1b79 --- /dev/null +++ b/site/docs/4.11.1/admin/bookies.md @@ -0,0 +1,174 @@ +--- +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 +$ 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 +$ 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 +$ 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 +$ 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 +$ bin/bookkeeper upgrade --upgrade +``` + +...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... + +```shell +$ bin/bookkeeper upgrade --finalize +``` + +...and then restart the server: + +```shell +$ bin/bookkeeper bookie +``` + +If something has gone wrong, you can always perform a rollback: + +```shell +$ 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 +$ 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 +$ 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 shell recover + ``` + + 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 shell recover 192.168.1.10:3181 + ``` + + See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.11.1/admin/decomission.md b/site/docs/4.11.1/admin/decomission.md new file mode 100644 index 00000000000..842b2508b35 --- /dev/null +++ b/site/docs/4.11.1/admin/decomission.md @@ -0,0 +1,41 @@ +--- +title: Decommission Bookies +--- + +In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the +decommissioning was safely done. + +### Before we decommission +1. Ensure state of your cluster can support the decommissioning of the target bookie. +Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie + +2. Ensure target bookie shows up in the listbookies command. + +3. Ensure that there is no other process ongoing (upgrade etc). + +### Process of Decommissioning +1. Log on to the bookie node, check if there are underreplicated ledgers. + +If there are, the decommission command will force them to be replicated. +`$ bin/bookkeeper shell listunderreplicated` + +2. Stop the bookie +`$ bin/bookkeeper-daemon.sh stop bookie` + +3. Run the decommission command. +If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` +If you are running the decommission command for target bookie node from another bookie node you should mention +the target bookie id in the arguments for `-bookieid` +`$ bin/bookkeeper shell decommissionbookie` +or +`$ bin/bookkeeper shell decommissionbookie -bookieid ` + +4. Validate that there are no ledgers on decommissioned bookie +`$ bin/bookkeeper shell listledgers -bookieid ` + +Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: + +```bash +./bookkeeper shell listbookies -rw -h +./bookkeeper shell listbookies -ro -h +``` diff --git a/site/docs/4.11.1/admin/geo-replication.md b/site/docs/4.11.1/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.11.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.11.1/admin/http.md b/site/docs/4.11.1/admin/http.md new file mode 100644 index 00000000000..50c3a7db01d --- /dev/null +++ b/site/docs/4.11.1/admin/http.md @@ -0,0 +1,509 @@ +--- +title: BookKeeper Admin REST API +--- + +This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. +To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. + +## All the endpoints + +Currently all the HTTP endpoints could be divided into these 5 components: +1. Heartbeat: heartbeat for a specific bookie. +2. Config: doing the server configuration for a specific bookie. +3. Ledger: HTTP endpoints related to ledgers. +4. Bookie: HTTP endpoints related to bookies. +5. AutoRecovery: HTTP endpoints related to auto recovery. + +## Heartbeat + +### Endpoint: /heartbeat +* Method: GET +* Description: Get heartbeat status for a specific bookie +* Response: + +| Code | Description | +|:-------|:------------| +|200 | Successful operation | + +## Config + +### Endpoint: /api/v1/config/server_config +1. Method: GET + * Description: Get value of all configured values overridden on local server config + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | +1. Method: PUT + * Description: Update a local server config + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |configName | String | Yes | Configuration name(key) | + |configValue | String | Yes | Configuration value(value) | + * Body: + ```json + { + "configName1": "configValue1", + "configName2": "configValue2" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Config + +### Endpoint: /metrics +1. Method: GET + * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +## Ledger + +### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> +1. Method: DELETE + * Description: Delete a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> +1. Method: GET + * Description: List all the ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |print_metadata | Boolean | No | whether print out metadata | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1", + "ledgerId2": "ledgerMetadata2", + ... + } + ``` + +### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> +1. Method: GET + * Description: Get the metadata of a ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes | ledger id of the ledger. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "ledgerId1": "ledgerMetadata1" + } + ``` + +### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> +1. Method: GET + * Description: Read a range of entries from ledger. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |ledger_id | Long | Yes| ledger id of the ledger. | + |start_entry_id | Long | No | start entry id of read range. | + |end_entry_id | Long | No | end entry id of read range. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "entryId1": "entry content 1", + "entryId2": "entry content 2", + ... + } + ``` + +## Bookie + +### Endpoint: /api/v1/bookie/info +1. Method: GET + * Description: Get bookie info + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |501 | Not implemented | + * Body: + ```json + { + "freeSpace" : 0, + "totalSpace" : 0 + } + ``` + +### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> +1. Method: GET + * Description: Get all the available bookies. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | + |print_hostnames | Boolean | No | whether print hostname of bookies. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieSocketAddress1": "hostname1", + "bookieSocketAddress2": "hostname2", + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_bookie_info +1. Method: GET + * Description: Get bookies disk usage info of this cluster. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "bookieAddress" : {free: xxx, total: xxx}, + "bookieAddress" : {free: xxx, total: xxx}, + ... + "clusterInfo" : {total_free: xxx, total: xxx} + } + ``` + +### Endpoint: /api/v1/bookie/last_log_mark +1. Method: GET + * Description: Get the last log marker. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + JournalId1 : position1, + JournalId2 : position2, + ... + } + ``` + +### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> +1. Method: GET + * Description: Get all the files on disk of current bookie. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |type | String | No | file type: journal/entrylog/index. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "journal files" : "filename1 filename2 ...", + "entrylog files" : "filename1 filename2...", + "index files" : "filename1 filename2 ..." + } + ``` + +### Endpoint: /api/v1/bookie/expand_storage +1. Method: PUT + * Description: Expand storage for a bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/bookie/gc +1. Method: PUT + * Description: trigger gc for this bookie. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: GET + * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + { + "is_in_force_gc" : "false" + } + ``` + +### Endpoint: /api/v1/bookie/gc_details +1. Method: GET + * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + [ { + "forceCompacting" : false, + "majorCompacting" : false, + "minorCompacting" : false, + "lastMajorCompactionTime" : 1544578144944, + "lastMinorCompactionTime" : 1544578144944, + "majorCompactionCounter" : 1, + "minorCompactionCounter" : 0 + } ] + ``` + +### Endpoint: /api/v1/bookie/state +1. Method: GET + * Description: Exposes the current state of bookie + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Body: + ```json + { + "running" : true, + "readOnly" : false, + "shuttingDown" : false, + "availableForHighPriorityWrites" : true + } + ``` + +### Endpoint: /api/v1/bookie/is_ready +1. Method: GET + * Description: Return true if the bookie is ready + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + |503 | Bookie is not ready | + * Body: + + +## Auto recovery + +### Endpoint: /api/v1/autorecovery/bookie/ +1. Method: PUT + * Description: Ledger data recovery for failed bookie + * Body: + ```json + { + "bookie_src": [ "bookie_src1", "bookie_src2"... ], + "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], + "delete_cookie": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |bookie_src | Strings | Yes | bookie source to recovery | + |bookie_dest | Strings | No | bookie data recovery destination | + |delete_cookie | Boolean | No | Whether delete cookie | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> +1. Method: GET + * Description: Get all under replicated ledgers. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |missingreplica | String | No | missing replica bookieId | + |excludingmissingreplica | String | No | exclude missing replica bookieId | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + [ledgerId1, ledgerId2...] + } + ``` + +### Endpoint: /api/v1/autorecovery/who_is_auditor +1. Method: GET + * Description: Get auditor bookie id. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + * Response Body format: + + ```json + { + "Auditor": "hostname/hostAddress:Port" + } + ``` + +### Endpoint: /api/v1/autorecovery/trigger_audit +1. Method: PUT + * Description: Force trigger audit by resting the lostBookieRecoveryDelay. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay +1. Method: GET + * Description: Get lostBookieRecoveryDelay value in seconds. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +1. Method: PUT + * Description: Set lostBookieRecoveryDelay value in seconds. + * Body: + ```json + { + "delay_seconds": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | delay_seconds | Long | Yes | set delay value in seconds. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | + +### Endpoint: /api/v1/autorecovery/decommission +1. Method: PUT + * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. + * Body: + ```json + { + "bookie_src": + } + ``` + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + | bookie_src | String | Yes | Bookie src to decommission.. | + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |404 | Not found | diff --git a/site/docs/4.11.1/admin/metrics.md b/site/docs/4.11.1/admin/metrics.md new file mode 100644 index 00000000000..dca236489d2 --- /dev/null +++ b/site/docs/4.11.1/admin/metrics.md @@ -0,0 +1,38 @@ +--- +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 these 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.prometheus.PrometheusMetricsProvider` + +> 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.11.1/admin/perf.md b/site/docs/4.11.1/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.11.1/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.11.1/admin/placement.md b/site/docs/4.11.1/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.11.1/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.11.1/admin/upgrade.md b/site/docs/4.11.1/admin/upgrade.md new file mode 100644 index 00000000000..57c65208131 --- /dev/null +++ b/site/docs/4.11.1/admin/upgrade.md @@ -0,0 +1,175 @@ +--- +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.6.x to 4.7.0 upgrade + +There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. + +However, we list a list of changes that you might want to know. + +#### Common Configuration Changes + +This section documents the common configuration changes that applied for both clients and servers. + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | +| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | +| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | +| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | + +##### Deprecated Settings + +There are no common settings deprecated at 4.7.0. + +##### Changed Settings + +There are no common settings whose default value are changed at 4.7.0. + +#### Server Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | +| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | +| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | +| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | +| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | +| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | + + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | + +#### Client Configuration Changes + +##### New Settings + +Following settings are newly added in 4.7.0. + +| Name | Default Value | Description | +|------|---------------|-------------| +| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | +| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | +| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | + +##### Deprecated Settings + +Following settings are deprecated since 4.7.0. + +| Name | Description | +|------|-------------| +| clientKeyStoreType | Replaced by `tlsKeyStoreType` | +| clientKeyStore | Replaced by `tlsKeyStore` | +| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | +| clientTrustStoreType | Replaced by `tlsTrustStoreType` | +| clientTrustStore | Replaced by `tlsTrustStore` | +| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | +| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | + +##### Changed Settings + +The default values of following settings are changed since 4.7.0. + +| Name | Old Default Value | New Default Value | Notes | +|------|-------------------|-------------------|-------| +| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | + +### 4.7.x to 4.8.X upgrade + +In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. + +### 4.5.x to 4.6.x upgrade + +There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. + +### 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.11.1/api/distributedlog-api.md b/site/docs/4.11.1/api/distributedlog-api.md new file mode 100644 index 00000000000..f9b9a72406b --- /dev/null +++ b/site/docs/4.11.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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.11.1/api/ledger-adv-api.md b/site/docs/4.11.1/api/ledger-adv-api.md new file mode 100644 index 00000000000..df6224dd7ec --- /dev/null +++ b/site/docs/4.11.1/api/ledger-adv-api.md @@ -0,0 +1,111 @@ +--- +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. + +Creating advanced ledgers can be done throught a fluent API since 4.6. + +```java +BookKeeper bk = ...; + +byte[] passwd = "some-passwd".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(passwd) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .makeAdv() // convert the create ledger builder to create ledger adv builder + .withLedgerId(1234L) + .execute() // execute the creation op + .get(); // wait for the execution to complete + +``` + +### 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()); +``` + +If you are using the new API, you can do as following: + +```java +WriteHandle wh = ...; +long entryId = ...; // entry id generated externally + +wh.write(entryId, "Some entry data".getBytes()).get(); +``` + +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.11.1/api/ledger-api.md b/site/docs/4.11.1/api/ledger-api.md new file mode 100644 index 00000000000..a8d9cb1b24b --- /dev/null +++ b/site/docs/4.11.1/api/ledger-api.md @@ -0,0 +1,840 @@ +--- +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.11.1 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries +a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the +shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. + +```xml + +4.11.1 + + + + org.apache.bookkeeper + bookkeeper-server-shaded + ${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.11.1' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.11.1' +} +``` + +Similarly as using maven, you can also configure to use the shaded jars. + +```groovy +// use the `bookkeeper-server-shaded` jar +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' +} +``` + +## 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 be deleted synchronously which may throw exception: + +```java +long ledgerId = 1234; + +try { + bkClient.deleteLedger(ledgerId); +} catch (Exception e) { + e.printStackTrace(); +} +``` + +### Delete entries asynchronously + +{% pop Ledgers %} can also be deleted asynchronously: + +```java +class DeleteEntryCallback implements AsyncCallback.DeleteCallback { + public void deleteComplete() { + System.out.println("Delete completed"); + } +} +bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); +``` + +## 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 +$ 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 +``` + +## New API + +Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. +[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). + +> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. + +*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. + +### Create a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. + +Here is an example building the bookkeeper client. + +```java +// construct a client configuration instance +ClientConfiguration conf = new ClientConfiguration(); +conf.setZkServers(zkConnectionString); +conf.setZkLedgersRootPath("/path/to/ledgers/root"); + +// build the bookkeeper client +BookKeeper bk = BookKeeper.newBuilder(conf) + .statsLogger(...) + ... + .build(); + +``` + +### Create ledgers + +the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least +a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. + +here's an example: + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + +A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. + +### Write flags + +You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. +These flags are applied only during write operations and are not recorded on metadata. + + +Available write flags: + +| Flag | Explanation | Notes | +:---------|:------------|:------- +DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. + +```java +BookKeeper bk = ...; + +byte[] password = "some-password".getBytes(); + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete +``` + + +### Append entries to ledgers + +The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. + +```java +WriteHandle wh = ...; + +CompletableFuture addFuture = wh.append("Some entry data".getBytes()); + +// option 1: you can wait for add to complete synchronously +try { + long entryId = FutureUtils.result(addFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +addFuture + .thenApply(entryId -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +addFuture.whenComplete(new FutureEventListener() { + @Override + public void onSuccess(long entryId) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. +It is recommended to use `ByteBuf` as it is more gc friendly. + +### Open ledgers + +You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password +in order to open the ledgers. + +here's an example: + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .execute() // execute the open op + .get(); // wait for the execution to complete +``` + +A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. + +#### Recovery vs NoRecovery + +By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying +`withRecovery(true)` in the open builder. + +```java +BookKeeper bk = ...; + +long ledgerId = ...; +byte[] password = "some-password".getBytes(); + +ReadHandle rh = bk.newOpenLedgerOp() + .withLedgerId(ledgerId) + .withPassword(password) + .withRecovery(true) + .execute() + .get(); + +``` + +**What is the difference between "Recovery" and "NoRecovery"?** + +If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed +to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). + +In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. + +### Read entries from ledgers + +The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. + +```java +ReadHandle rh = ...; + +long startEntryId = ...; +long endEntryId = ...; +CompletableFuture readFuture = rh.read(startEntryId, endEntryId); + +// option 1: you can wait for read to complete synchronously +try { + LedgerEntries entries = FutureUtils.result(readFuture.get()); +} catch (BKException bke) { + // error handling +} + +// option 2: you can process the result and exception asynchronously +readFuture + .thenApply(entries -> { + // process the result + }) + .exceptionally(cause -> { + // handle the exception + }) + +// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously +readFuture.whenComplete(new FutureEventListener<>() { + @Override + public void onSuccess(LedgerEntries entries) { + // process the result + } + @Override + public void onFailure(Throwable cause) { + // handle the exception + } +}); +``` + +Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to +release the buffers held by it. + +Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications +attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). + +### Read unconfirmed entries from ledgers + +`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any +repeatable read consistency. + +```java +CompletableFuture readFuture = rh.readUnconfirmed(startEntryId, endEntryId); +``` + +### Tailing Reads + +There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. + +#### Polling + +You can do this in synchronous way: + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + Thread.sleep(1000); + + lac = rh.readLastAddConfirmed().get(); + continue; + } + + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + + nextEntryId = endEntryId + 1; +} +``` + +#### Long Polling + +```java +ReadHandle rh = ...; + +long startEntryId = 0L; +long nextEntryId = startEntryId; +int numEntriesPerBatch = 4; +while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { + long lac = rh.getLastAddConfirmed(); + if (nextEntryId > lac) { + // no more entries are added + try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { + if (lacAndEntry.hasEntry()) { + // process the entry + + ++nextEntryId; + } + } + } else { + long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); + LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); + + // process the entries + nextEntryId = endEntryId + 1; + } +} +``` + +### Delete ledgers + +{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +bk.newDeleteLedgerOp() + .withLedgerId(ledgerId) + .execute() + .get(); +``` + +### Relaxing Durability + +In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. +In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will +be always readable by other clients. + +On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after +writing the entry to SO buffers without waiting for an fsync. +In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. +Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. + +In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble ackknowledge the call after +performing an fsync to the disk which is storing the journal. +This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. + +The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. +In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. + + +```java +BookKeeper bk = ...; +long ledgerId = ...; + +WriteHandle wh = bk.newCreateLedgerOp() + .withDigestType(DigestType.CRC32) + .withPassword(password) + .withEnsembleSize(3) + .withWriteQuorumSize(3) + .withAckQuorumSize(2) + .withWriteFlags(DEFERRED_SYNC) + .execute() // execute the creation op + .get(); // wait for the execution to complete + + +wh.force().get(); // wait for fsync, make data available to readers and to the replicator + +wh.close(); // seal the ledger + +``` diff --git a/site/docs/4.11.1/api/overview.md b/site/docs/4.11.1/api/overview.md new file mode 100644 index 00000000000..3e0adcd61af --- /dev/null +++ b/site/docs/4.11.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.11.1/deployment/dcos.md b/site/docs/4.11.1/deployment/dcos.md new file mode 100644 index 00000000000..7cf8281ccb3 --- /dev/null +++ b/site/docs/4.11.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.11.1] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.11.1/deployment/kubernetes.md b/site/docs/4.11.1/deployment/kubernetes.md new file mode 100644 index 00000000000..0f113169edc --- /dev/null +++ b/site/docs/4.11.1/deployment/kubernetes.md @@ -0,0 +1,181 @@ +--- +title: Deploying Apache BookKeeper on Kubernetes +tags: [Kubernetes, Google Container Engine] +logo: img/kubernetes-logo.png +--- + +Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. + +The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: + +* A three-node ZooKeeper cluster +* A BookKeeper cluster with a bookie runs on each node. + +## Setup on Google Container Engine + +To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. + +If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. + +[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). + +### Prerequisites + +To get started, you'll need: + +* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) +* An existing Cloud Platform project +* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). + +### Create a new Kubernetes cluster + +You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. + +As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. + +```bash +$ gcloud config set compute/zone us-central1-a +$ gcloud config set project your-project-name +$ gcloud container clusters create bookkeeper-gke-cluster \ + --machine-type=n1-standard-8 \ + --num-nodes=3 \ + --local-ssd-count=2 \ + --enable-kubernetes-alpha +``` + +By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). + +### Dashboard + +You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: + +```bash +$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ + --zone=us-central1-a \ + --project=your-project-name +$ kubectl proxy +``` + +By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. + +When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: + +```bash +$ kubectl get nodes +``` + +If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. + +### ZooKeeper + +You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. + +```bash +$ kubectl apply -f zookeeper.yaml +``` + +Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: + +```bash +$ kubectl get pods -l component=zookeeper +NAME READY STATUS RESTARTS AGE +zk-0 1/1 Running 0 18m +zk-1 1/1 Running 0 17m +zk-2 0/1 Running 6 15m +``` + +This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. + + +If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the +remote zookeeper server: + +```bash +$ kubectl port-forward zk-0 2181:2181 +$ zk-shell localhost 2181 +``` + +### Deploy Bookies + +Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). + +> NOTE: _DaemonSet_ vs _StatefulSet_ +> +> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the +> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent +> Volumes. +> +> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods +> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. +> +> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages +> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and +> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency +> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. + +```bash +# deploy bookies in a daemon set +$ kubectl apply -f bookkeeper.yaml + +# deploy bookies in a stateful set +$ kubectl apply -f bookkeeper.stateful.yaml +``` + +You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: + +```bash +$ kubectl get pods +``` + +While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ + +You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. + +```bash +$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +``` + +An example output of Dice instance is like this: +```aidl +➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" +If you don't see a command prompt, try pressing enter. +Value = 1, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 5, epoch = 5, leading +Value = 3, epoch = 5, leading +Value = 2, epoch = 5, leading +Value = 1, epoch = 5, leading +Value = 4, epoch = 5, leading +Value = 2, epoch = 5, leading +``` + +### Un-Deploy + +Delete Demo dice instance + +```bash +$kubectl delete deployment dice +``` + +Delete BookKeeper +```bash +$ kubectl delete -f bookkeeper.yaml +``` + +Delete ZooKeeper +```bash +$ kubectl delete -f zookeeper.yaml +``` + +Delete cluster +```bash +$ gcloud container clusters delete bookkeeper-gke-cluster +``` + + + diff --git a/site/docs/4.11.1/deployment/manual.md b/site/docs/4.11.1/deployment/manual.md new file mode 100644 index 00000000000..89b3b0ec649 --- /dev/null +++ b/site/docs/4.11.1/deployment/manual.md @@ -0,0 +1,58 @@ +--- +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. + +## Cluster metadata setup + +Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! + +## Starting up bookies + + + +Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: + +```shell +$ bookkeeper-server/bin/bookkeeper bookie +``` + + +### System requirements + +{% include system-requirements.md %} + + + diff --git a/site/docs/4.11.1/development/codebase.md b/site/docs/4.11.1/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.11.1/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.11.1/development/protocol.md b/site/docs/4.11.1/development/protocol.md new file mode 100644 index 00000000000..f4410444427 --- /dev/null +++ b/site/docs/4.11.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 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.11.1/getting-started/concepts.md b/site/docs/4.11.1/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.11.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.11.1/getting-started/installation.md b/site/docs/4.11.1/getting-started/installation.md new file mode 100644 index 00000000000..9f856ec1d21 --- /dev/null +++ b/site/docs/4.11.1/getting-started/installation.md @@ -0,0 +1,82 @@ +--- +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.8](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 +``` + +Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. + +```shell +$ mvn package -Dstream +``` + +> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin + +> You can enable `table service` by adding the `-Dstream` flag when running above commands. + +## 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.11.1/getting-started/run-locally.md b/site/docs/4.11.1/getting-started/run-locally.md new file mode 100644 index 00000000000..520140317d9 --- /dev/null +++ b/site/docs/4.11.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 +$ bin/bookkeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.11.1/overview/overview.md b/site/docs/4.11.1/overview/overview.md new file mode 100644 index 00000000000..9027f29b4fd --- /dev/null +++ b/site/docs/4.11.1/overview/overview.md @@ -0,0 +1,60 @@ +--- +title: Apache BookKeeper™ 4.11.1 +--- + + +This documentation is for Apache BookKeeper™ version 4.11.1. + +Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. + +BookKeeper is suitable for a wide variety of use cases, including: + +Use case | Example +:--------|:------- +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) +[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) +Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) +Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines + +Learn more about Apache BookKeeper™ and what it can do for your organization: + +- [Apache BookKeeper 4.11.1 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start [using](../../getting-started/installation) 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.11.1/overview/releaseNotes.md b/site/docs/4.11.1/overview/releaseNotes.md new file mode 100644 index 00000000000..54b22776805 --- /dev/null +++ b/site/docs/4.11.1/overview/releaseNotes.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.11.1 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.11.1. The technical details of this release are summarized +below. + +## Highlights + +[List the highlights] + +## Details + +[list to issues list] + diff --git a/site/docs/4.11.1/overview/releaseNotesTemplate.md b/site/docs/4.11.1/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..54b22776805 --- /dev/null +++ b/site/docs/4.11.1/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.11.1 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.11.1. The technical details of this release are summarized +below. + +## Highlights + +[List the highlights] + +## Details + +[list to issues list] + diff --git a/site/docs/4.11.1/reference/cli.md b/site/docs/4.11.1/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.11.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.11.1/reference/config.md b/site/docs/4.11.1/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.11.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.11.1/reference/metrics.md b/site/docs/4.11.1/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.11.1/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.11.1/security/overview.md b/site/docs/4.11.1/security/overview.md new file mode 100644 index 00000000000..b825776eb67 --- /dev/null +++ b/site/docs/4.11.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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.11.1/security/sasl.md b/site/docs/4.11.1/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.11.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.11.1/security/tls.md b/site/docs/4.11.1/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.11.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.11.1/security/zookeeper.md b/site/docs/4.11.1/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.11.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) diff --git a/site/docs/latest/overview/overview.md b/site/docs/latest/overview/overview.md index 9497ad85ec3..61f76504e68 100644 --- a/site/docs/latest/overview/overview.md +++ b/site/docs/latest/overview/overview.md @@ -1,5 +1,5 @@ --- -title: Apache BookKeeper™ 4.11.0-SNAPSHOT +title: Apache BookKeeper™ 4.12.0-SNAPSHOT --- %s' % (name, toAddress)) + + messages = slack.channels.history(channel=id, + oldest=last_day_timestamp, + count=1000) + digest = '' + for m in reversed(messages.body['messages']): + if not m['type'] == 'message': + continue + + user = m.get('user') + if not user: + user = m['comment']['user'] + sender = users.get(user, '') + + date = datetime.datetime.utcfromtimestamp(float(m['ts'])).strftime('%Y-%m-%d %H:%M:%S UTC') + # Replace users id mentions with real names + text = re.sub(r'<@(\w+)>', lambda m: '@' + users[m.group(1)], m['text']) + + digest += '%s - %s: %s\n' % (date, sender, text) + for reaction in m.get('reactions', []): + digest += '%s : %s\n' % (reaction['name'], ', '.join(map(users.get, reaction['users']))) + digest += '----\n' + + if digest: + send_digest(name, toAddress, digest) From fb6542aa41735dbae0eefbbb911a0b9ff834e725 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 9 Jul 2020 02:37:30 -0700 Subject: [PATCH 0451/1642] Create CI jobs for branch-4.11 Create CI jobs for releases on branch-4.11. Reviewers: Enrico Olivelli This closes #2363 from rdhabalia/4.11-infra --- ...elease_branch_410_integrationtests.groovy} | 6 +- ...ookkeeper_release_branch_410_java8.groovy} | 8 +- ...ookkeeper_release_branch_410_java9.groovy} | 8 +- ...release_branch_411_integrationtests.groovy | 77 +++++++++++++++++++ ...bookkeeper_release_branch_411_java8.groovy | 40 ++++++++++ ...bookkeeper_release_branch_411_java9.groovy | 40 ++++++++++ .../job_bookkeeper_release_branch_46.groovy | 41 ---------- ...release_branch_49_integrationtests.groovy} | 6 +- ...bookkeeper_release_branch_49_java8.groovy} | 8 +- ...bookkeeper_release_branch_49_java9.groovy} | 6 +- 10 files changed, 178 insertions(+), 62 deletions(-) rename .test-infra/jenkins/{job_bookkeeper_release_branch_48_integrationtests.groovy => job_bookkeeper_release_branch_410_integrationtests.groovy} (94%) rename .test-infra/jenkins/{job_bookkeeper_release_branch_47_java8.groovy => job_bookkeeper_release_branch_410_java8.groovy} (88%) rename .test-infra/jenkins/{job_bookkeeper_release_branch_48_java9.groovy => job_bookkeeper_release_branch_410_java9.groovy} (88%) create mode 100644 .test-infra/jenkins/job_bookkeeper_release_branch_411_integrationtests.groovy create mode 100644 .test-infra/jenkins/job_bookkeeper_release_branch_411_java8.groovy create mode 100644 .test-infra/jenkins/job_bookkeeper_release_branch_411_java9.groovy delete mode 100644 .test-infra/jenkins/job_bookkeeper_release_branch_46.groovy rename .test-infra/jenkins/{job_bookkeeper_release_branch_47_integrationtests.groovy => job_bookkeeper_release_branch_49_integrationtests.groovy} (94%) rename .test-infra/jenkins/{job_bookkeeper_release_branch_48_java8.groovy => job_bookkeeper_release_branch_49_java8.groovy} (88%) rename .test-infra/jenkins/{job_bookkeeper_release_branch_47_java9.groovy => job_bookkeeper_release_branch_49_java9.groovy} (93%) diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_48_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_410_integrationtests.groovy similarity index 94% rename from .test-infra/jenkins/job_bookkeeper_release_branch_48_integrationtests.groovy rename to .test-infra/jenkins/job_bookkeeper_release_branch_410_integrationtests.groovy index daf4fbd5f64..5a06c986bc9 100644 --- a/.test-infra/jenkins/job_bookkeeper_release_branch_48_integrationtests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_410_integrationtests.groovy @@ -19,13 +19,13 @@ import common_job_properties // This is the Java postcommit which runs a maven install, and the current set of precommit tests. -freeStyleJob('bookkeeper_release_branch_48_integrationtests') { - description('Run integration tests on bookkeeper branch-4.8.') +freeStyleJob('bookkeeper_release_branch_410_integrationtests') { + description('Run integration tests on bookkeeper branch-4.10.') // Set common parameters. common_job_properties.setTopLevelMainJobProperties( delegate, - 'branch-4.8', + 'branch-4.10', 'JDK 1.8 (latest)', 120) diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_47_java8.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_410_java8.groovy similarity index 88% rename from .test-infra/jenkins/job_bookkeeper_release_branch_47_java8.groovy rename to .test-infra/jenkins/job_bookkeeper_release_branch_410_java8.groovy index 7dc7e78638e..88ec435c81a 100644 --- a/.test-infra/jenkins/job_bookkeeper_release_branch_47_java8.groovy +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_410_java8.groovy @@ -18,13 +18,13 @@ import common_job_properties -// This job runs the Java postcommit tests on Java 8 for branch-4.7 -mavenJob('bookkeeper_release_branch_47_java8') { - description('Runs nightly build for bookkeeper branch-4.7 in Java 8.') +// This job runs the Java postcommit tests on Java 8 for branch-4.10 +mavenJob('bookkeeper_release_branch_410_java8') { + description('Runs nightly build for bookkeeper branch-4.10 in Java 8.') // Set common parameters. common_job_properties.setTopLevelMainJobProperties( - delegate, 'branch-4.7', 'JDK 1.8 (latest)') + delegate, 'branch-4.10', 'JDK 1.8 (latest)') // Sets that this is a PostCommit job. common_job_properties.setPostCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_48_java9.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_410_java9.groovy similarity index 88% rename from .test-infra/jenkins/job_bookkeeper_release_branch_48_java9.groovy rename to .test-infra/jenkins/job_bookkeeper_release_branch_410_java9.groovy index 8a20415e10f..642d0b0155a 100644 --- a/.test-infra/jenkins/job_bookkeeper_release_branch_48_java9.groovy +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_410_java9.groovy @@ -18,13 +18,13 @@ import common_job_properties -// This job runs the Java postcommit tests on Java 9 for branch-4.8 -mavenJob('bookkeeper_release_branch_48_java9') { - description('Runs nightly build for bookkeeper branch-4.8 in Java 9.') +// This job runs the Java postcommit tests on Java 9 for branch-4.10 +mavenJob('bookkeeper_release_branch_410_java9') { + description('Runs nightly build for bookkeeper branch-4.10 in Java 9.') // Set common parameters. common_job_properties.setTopLevelMainJobProperties( - delegate, 'branch-4.8', 'JDK 1.9 (latest)') + delegate, 'branch-4.10', 'JDK 1.9 (latest)') // Sets that this is a PostCommit job. common_job_properties.setPostCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_411_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_411_integrationtests.groovy new file mode 100644 index 00000000000..55353a65c1f --- /dev/null +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_411_integrationtests.groovy @@ -0,0 +1,77 @@ +/* + * 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. + */ + +import common_job_properties + +// This is the Java postcommit which runs a maven install, and the current set of precommit tests. +freeStyleJob('bookkeeper_release_branch_411_integrationtests') { + description('Run integration tests on bookkeeper branch-4.11.') + + // Set common parameters. + common_job_properties.setTopLevelMainJobProperties( + delegate, + 'branch-4.11', + 'JDK 1.8 (latest)', + 120) + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit( + delegate, + 'H 12 * * *', + false) + + steps { + // Temporary information gathering to see if full disks are causing the builds to flake + shell('id') + shell('ulimit -a') + shell('pwd') + shell('df -Th') + shell('ps -eo euser,pid,ppid,pgid,start,pcpu,pmem,cmd') + shell('docker network prune -f --filter name=testnetwork_*') // clean up any dangling networks from previous runs + shell('docker system events > docker.log & echo $! > docker-log.pid') + + shell('docker pull apachebookkeeper/bookkeeper-all-released-versions:latest') + + // Build everything + maven { + // Set Maven parameters. + common_job_properties.setMavenConfig(delegate) + + goals('-B clean install -Pdocker') + properties(skipTests: true, interactiveMode: false) + } + + maven { + // Set Maven parameters. + common_job_properties.setMavenConfig(delegate) + rootPOM('tests/pom.xml') + goals('-B test -DintegrationTests') + } + + shell('kill $(cat docker-log.pid) || true') + } + + publishers { + archiveArtifacts { + allowEmpty(true) + pattern('**/target/container-logs/**') + pattern('docker.log') + } + archiveJunit('**/surefire-reports/TEST-*.xml') + } +} diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_411_java8.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_411_java8.groovy new file mode 100644 index 00000000000..4b9a8937807 --- /dev/null +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_411_java8.groovy @@ -0,0 +1,40 @@ +/* + * 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. + */ + +import common_job_properties + +// This job runs the Java postcommit tests on Java 8 for branch-4.11 +mavenJob('bookkeeper_release_branch_411_java8') { + description('Runs nightly build for bookkeeper branch-4.11 in Java 8.') + + // Set common parameters. + common_job_properties.setTopLevelMainJobProperties( + delegate, 'branch-4.11', 'JDK 1.8 (latest)') + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit( + delegate, + 'H 12 * * *', + false) + + // Set maven parameters. + common_job_properties.setMavenConfig(delegate) + + // Maven build project. + goals('clean apache-rat:check package spotbugs:check -Ddistributedlog -Dstream -DstreamTests') +} diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_411_java9.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_411_java9.groovy new file mode 100644 index 00000000000..42ffc1848b7 --- /dev/null +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_411_java9.groovy @@ -0,0 +1,40 @@ +/* + * 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. + */ + +import common_job_properties + +// This job runs the Java postcommit tests on Java 9 for branch-4.11 +mavenJob('bookkeeper_release_branch_411_java9') { + description('Runs nightly build for bookkeeper branch-4.11 in Java 9.') + + // Set common parameters. + common_job_properties.setTopLevelMainJobProperties( + delegate, 'branch-4.11', 'JDK 1.9 (latest)') + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit( + delegate, + 'H 12 * * *', + false) + + // Set maven parameters. + common_job_properties.setMavenConfig(delegate) + + // Maven build project. + goals('clean apache-rat:check package spotbugs:check -Ddistributedlog -Dstream -DstreamTests') +} diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_46.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_46.groovy deleted file mode 100644 index 0fd822898ca..00000000000 --- a/.test-infra/jenkins/job_bookkeeper_release_branch_46.groovy +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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. - */ - -import common_job_properties - -// This job runs nightly build for bookkeeper release branch-4.6 -mavenJob('bookkeeper_release_branch_46') { - description('Run nightly build for bookkeeper release branch-4.6') - - // Set common parameters. - common_job_properties.setTopLevelMainJobProperties( - delegate, - "branch-4.6") - - // Sets that this is a PostCommit job. - common_job_properties.setPostCommit( - delegate, - 'H 12 * * *', - false) - - // Set maven parameters. - common_job_properties.setMavenConfig(delegate) - - // Maven build project. - goals('clean apache-rat:check package findbugs:check') -} diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_47_integrationtests.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_49_integrationtests.groovy similarity index 94% rename from .test-infra/jenkins/job_bookkeeper_release_branch_47_integrationtests.groovy rename to .test-infra/jenkins/job_bookkeeper_release_branch_49_integrationtests.groovy index b0433aa0e94..85ade3815b1 100644 --- a/.test-infra/jenkins/job_bookkeeper_release_branch_47_integrationtests.groovy +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_49_integrationtests.groovy @@ -19,13 +19,13 @@ import common_job_properties // This is the Java postcommit which runs a maven install, and the current set of precommit tests. -freeStyleJob('bookkeeper_release_branch_47_integrationtests') { - description('Run integration tests on bookkeeper branch-4.7.') +freeStyleJob('bookkeeper_release_branch_49_integrationtests') { + description('Run integration tests on bookkeeper branch-4.9.') // Set common parameters. common_job_properties.setTopLevelMainJobProperties( delegate, - 'branch-4.7', + 'branch-4.9', 'JDK 1.8 (latest)', 120) diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_48_java8.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_49_java8.groovy similarity index 88% rename from .test-infra/jenkins/job_bookkeeper_release_branch_48_java8.groovy rename to .test-infra/jenkins/job_bookkeeper_release_branch_49_java8.groovy index f9f911c7c0c..21baff0ac59 100644 --- a/.test-infra/jenkins/job_bookkeeper_release_branch_48_java8.groovy +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_49_java8.groovy @@ -18,13 +18,13 @@ import common_job_properties -// This job runs the Java postcommit tests on Java 8 for branch-4.8 -mavenJob('bookkeeper_release_branch_48_java8') { - description('Runs nightly build for bookkeeper branch-4.8 in Java 8.') +// This job runs the Java postcommit tests on Java 8 for branch-4.9 +mavenJob('bookkeeper_release_branch_49_java8') { + description('Runs nightly build for bookkeeper branch-4.9 in Java 8.') // Set common parameters. common_job_properties.setTopLevelMainJobProperties( - delegate, 'branch-4.8', 'JDK 1.8 (latest)') + delegate, 'branch-4.9', 'JDK 1.8 (latest)') // Sets that this is a PostCommit job. common_job_properties.setPostCommit( diff --git a/.test-infra/jenkins/job_bookkeeper_release_branch_47_java9.groovy b/.test-infra/jenkins/job_bookkeeper_release_branch_49_java9.groovy similarity index 93% rename from .test-infra/jenkins/job_bookkeeper_release_branch_47_java9.groovy rename to .test-infra/jenkins/job_bookkeeper_release_branch_49_java9.groovy index 4feaa2fb544..65da914ca06 100644 --- a/.test-infra/jenkins/job_bookkeeper_release_branch_47_java9.groovy +++ b/.test-infra/jenkins/job_bookkeeper_release_branch_49_java9.groovy @@ -18,13 +18,13 @@ import common_job_properties -// This job runs the Java postcommit tests on Java 9 for branch-4.7 -mavenJob('bookkeeper_release_branch_47_java9') { +// This job runs the Java postcommit tests on Java 9 for branch-4.9 +mavenJob('bookkeeper_release_branch_49_java9') { description('Runs nightly build for bookkeeper branch-4.7 in Java 9.') // Set common parameters. common_job_properties.setTopLevelMainJobProperties( - delegate, 'branch-4.7', 'JDK 1.9 (latest)') + delegate, 'branch-4.9', 'JDK 1.9 (latest)') // Sets that this is a PostCommit job. common_job_properties.setPostCommit( From 763c602caccba09b64be717ff5ce74798b73328a Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sun, 12 Jul 2020 15:26:31 -0700 Subject: [PATCH 0452/1642] [bookkeeper-site] fix latest version of bookkeeper website ### Motivation After generating website using [script](http://bookkeeper.apache.org/community/release_guide/#update-website), it seems somhow it has updated site with incorrect version. I still have to find out where was the issue while generating the website. But fixing it for now to correct the version on website. Reviewers: Enrico Olivelli This closes #2372 from rdhabalia/fix_site --- site/_config.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/site/_config.yml b/site/_config.yml index 2611a8333c4..0e46b0f3e2d 100644 --- a/site/_config.yml +++ b/site/_config.yml @@ -8,7 +8,6 @@ destination: local-generated twitter_url: https://twitter.com/asfbookkeeper versions: -- "4.11.1" - "4.11.0" - "4.10.0" - "4.9.2" @@ -40,7 +39,7 @@ archived_versions: - "4.1.0" - "4.0.0" latest_version: "4.12.0-SNAPSHOT" -latest_release: "4.11.1" +latest_release: "4.11.0" stable_release: "4.9.2" eol_releases: - "4.7.3" From a65840d4b031eddcb7dbeb691d86dd3a67968491 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 24 Jul 2020 04:53:56 +0200 Subject: [PATCH 0453/1642] Issue 2264: Bookie cannot perform Garbage Collection in case of corrupted EntryLogger file - in case of corrupted entry log file the bookie cannot perform GC - handle the case of an unexpected negative entrysize ### Changes Stop scanning the file Master Issue: #2264 Reviewers: Jia Zhai , Rajan Dhabalia This closes #2373 from eolivelli/fix/issue2264-corruptedfile --- .../main/java/org/apache/bookkeeper/bookie/EntryLogger.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index 0d7c8614ad3..bc16a192ab9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -1011,6 +1011,11 @@ public void scanEntryLog(long entryLogId, EntryLogScanner scanner) throws IOExce // read the entry data.clear(); + if (entrySize < 0) { + LOG.warn("bad read for ledger entry from entryLog {}@{} (entry size {})", + entryLogId, pos, entrySize); + return; + } data.capacity(entrySize); int rc = readFromLogChannel(entryLogId, bc, data, pos); if (rc != entrySize) { From ceaee6bebc6ce3615514664fa6091295ea974cc4 Mon Sep 17 00:00:00 2001 From: lamber-ken <2217232293@qq.com> Date: Mon, 27 Jul 2020 14:52:46 +0800 Subject: [PATCH 0454/1642] [SITE] Improve the user experience of the site ### Motivation Improve the user experience of the site. ![image](https://user-images.githubusercontent.com/20113411/82639390-325a3e80-9c3b-11ea-8a27-6347c6f7178c.png) ### Changes **Compare:** https://lamber-ken.github.io/docs/4.10.0/getting-started/installation/ https://bookkeeper.apache.org/docs/4.10.0/getting-started/installation/ Reviewers: Enrico Olivelli , Matteo Minardi This closes #2344 from lamber-ken/improve-site --- site/_sass/vendor/bulma/sass/layout/footer.sass | 2 +- site/css/style.sass | 11 ++++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/site/_sass/vendor/bulma/sass/layout/footer.sass b/site/_sass/vendor/bulma/sass/layout/footer.sass index f8285ca4306..45259c87880 100755 --- a/site/_sass/vendor/bulma/sass/layout/footer.sass +++ b/site/_sass/vendor/bulma/sass/layout/footer.sass @@ -1,3 +1,3 @@ .footer background-color: $background - padding: 3rem 1.5rem 6rem + padding: 6rem 1.5rem diff --git a/site/css/style.sass b/site/css/style.sass index a7d089849cc..ad223fddce7 100644 --- a/site/css/style.sass +++ b/site/css/style.sass @@ -81,9 +81,9 @@ footer.footer .toc overflow-y: scroll bottom: 0 - top: $navbar-height + 5rem + top: $navbar-height $toc-base-font-size: 1.2rem - position: fixed + position: sticky max-width: 20rem padding: 0 0 5% 0 @@ -156,16 +156,17 @@ h2.bk-subtitle .bk-main-content min-height: 70vh + .container + top: $navbar-height + position: sticky + .bk-community-container width: 60% aside.sidebar width: 15rem - position: fixed overflow-y: scroll bottom: 0 - top: $navbar-height + 5rem - padding: 0 0 5% 0 p + ul.sidebar-items margin-top: .2rem From a81cf69aef630fb1526bf7911d72c8d5ce6b01b0 Mon Sep 17 00:00:00 2001 From: kennyjiang <630862708@qq.com> Date: Mon, 27 Jul 2020 14:59:51 +0800 Subject: [PATCH 0455/1642] fix fillReadAheadCache stat bug These code blocks will not be executed: dbLedgerStorageStats.getReadAheadBatchCountStats().registerSuccessfulValue(count); dbLedgerStorageStats.getReadAheadBatchSizeStats().registerSuccessfulValue(size); ### Motivation fix fillReadAheadCache stat bug ### Changes Replace return with break to exit the while loop Reviewers: Enrico Olivelli , Rajan Dhabalia This closes #2383 from holmes07/master --- .../bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 60e3d648cdc..931c3e00720 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -475,7 +475,7 @@ private void fillReadAheadCache(long orginalLedgerId, long firstEntryId, long fi if (currentEntryLedgerId != orginalLedgerId) { // Found an entry belonging to a different ledger, stopping read-ahead - return; + break; } // Insert entry in read cache From a3fecbacdbf747756983f67b2a9dc10ec58eee7e Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 27 Jul 2020 15:06:29 +0200 Subject: [PATCH 0456/1642] Track ZooKeeper errors as causes of ZKException Descriptions of the changes in this PR: ### Motivation Every time a problem with ZK arises you don't see it in the exception chain of the BKException (in this case ZKException) and you end up with errors like: ``` org.apache.bookkeeper.client.BKException$ZKException: Error while using ZooKeeper at org.apache.bookkeeper.client.SyncCallbackUtils.finish(SyncCallbackUtils.java:83) at org.apache.bookkeeper.client.SyncCallbackUtils$SyncAddCallback.addComplete(SyncCallbackUtils.java:251) at org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:91) at org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:430) at org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1784) at org.apache.bookkeeper.client.LedgerHandle$5.safeRun(LedgerHandle.java:574) ``` ### Changes Add a "cause" to every ZKException. ### Notes There are very few places that cannot be fixed because they are still using the old callback based mechanism without CompletableFuture. Those points are not changed in order to make the patch simple but still useful. Reviewers: Jia Zhai This closes #2384 from eolivelli/fix/zkexception-chain --- .../apache/bookkeeper/client/BKException.java | 4 ++++ .../discover/ZKRegistrationClient.java | 2 +- .../meta/AbstractZkLedgerManager.java | 22 +++++++++++++------ .../meta/AbstractZkLedgerManagerTest.java | 1 + 4 files changed, 21 insertions(+), 8 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java index a3f03c24157..438dd950455 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java @@ -306,6 +306,10 @@ public static class ZKException extends BKException { public ZKException() { super(BKException.Code.ZKException); } + + public ZKException(Throwable cause) { + super(BKException.Code.ZKException, cause); + } } /** 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 index 6995a14011f..367e9ac0ace 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java @@ -289,7 +289,7 @@ private CompletableFuture>> getChildren(Strin CompletableFuture>> future = FutureUtils.createFuture(); zk.getChildren(regPath, watcher, (rc, path, ctx, children, stat) -> { if (Code.OK != rc) { - ZKException zke = new ZKException(); + ZKException zke = new ZKException(KeeperException.create(KeeperException.Code.get(rc), path)); future.completeExceptionally(zke.fillInStackTrace()); return; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java index 0b42c3df80f..8ecb9ddde91 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java @@ -295,7 +295,7 @@ public void processResult(int rc, String path, Object ctx, String name) { } else { LOG.error("Could not validate node for ledger {} after LedgerExistsException", ledgerId, exception); - promise.completeExceptionally(new BKException.ZKException()); + promise.completeExceptionally(new BKException.ZKException(exception)); } return null; }); @@ -306,7 +306,8 @@ public void processResult(int rc, String path, Object ctx, String name) { } else { LOG.error("Could not create node for ledger {}", ledgerId, KeeperException.create(Code.get(rc), path)); - promise.completeExceptionally(new BKException.ZKException()); + promise.completeExceptionally( + new BKException.ZKException(KeeperException.create(Code.get(rc), path))); } } }; @@ -365,7 +366,8 @@ public void processResult(int rc, String path, Object ctx) { } FutureUtils.complete(promise, null); } else { - promise.completeExceptionally(new BKException.ZKException()); + promise.completeExceptionally( + new BKException.ZKException(KeeperException.create(Code.get(rc), path))); } } }; @@ -444,12 +446,15 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta if (rc != KeeperException.Code.OK.intValue()) { LOG.error("Could not read metadata for ledger: " + ledgerId, KeeperException.create(KeeperException.Code.get(rc), path)); - promise.completeExceptionally(new BKException.ZKException()); + promise.completeExceptionally( + new BKException.ZKException(KeeperException.create(Code.get(rc), path))); return; } if (stat == null) { LOG.error("Could not parse ledger metadata for ledger: {}. Stat object is null", ledgerId); - promise.completeExceptionally(new BKException.ZKException()); + promise.completeExceptionally(new BKException.ZKException( + new Exception("Could not parse ledger metadata for ledger: " + + ledgerId + " . Stat object is null").fillInStackTrace())); return; } @@ -459,7 +464,9 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta promise.complete(new Versioned<>(metadata, version)); } catch (Throwable t) { LOG.error("Could not parse ledger metadata for ledger: {}", ledgerId, t); - promise.completeExceptionally(new BKException.ZKException()); + promise.completeExceptionally(new BKException.ZKException( + new Exception("Could not parse ledger metadata for ledger: " + + ledgerId, t).fillInStackTrace())); } } }, null); @@ -498,7 +505,8 @@ public void processResult(int rc, String path, Object ctx, Stat stat) { promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException()); } else { LOG.warn("Conditional update ledger metadata failed: {}", KeeperException.Code.get(rc)); - promise.completeExceptionally(new BKException.ZKException()); + promise.completeExceptionally( + new BKException.ZKException(KeeperException.create(Code.get(rc), path))); } } }, null); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index b54b7d97413..e2c6a12d0e5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -216,6 +216,7 @@ public void testCreateLedgerMetadataException() throws Exception { assertTrue(e instanceof BKException); BKException bke = (BKException) e; assertEquals(Code.ZKException, bke.getCode()); + assertTrue(bke.getCause() instanceof KeeperException); } } From d2ba9b3352a5cd5a9f0ce186efd3c989a0c5b983 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 13 Aug 2020 05:32:54 +0200 Subject: [PATCH 0457/1642] BP-41 Separate BookieId from Bookie Network Address (#2397) This is the design document of "BP-41 Separate BookieId from Bookie Network Address" Master Issue: #2396 --- site/bps/BP-41-bookieid.md | 185 +++++++++++++++++++++++++ site/community/bookkeeper_proposals.md | 3 +- 2 files changed, 187 insertions(+), 1 deletion(-) create mode 100644 site/bps/BP-41-bookieid.md diff --git a/site/bps/BP-41-bookieid.md b/site/bps/BP-41-bookieid.md new file mode 100644 index 00000000000..0514c664099 --- /dev/null +++ b/site/bps/BP-41-bookieid.md @@ -0,0 +1,185 @@ +--- +title: "BP-41: Separate BookieId from Bookie Network Address" +issue: https://github.com/apache/bookkeeper/issues/2396 +state: 'Draft' +release: "4.12.0" +--- + +### Motivation + +We want to separate the concepts of **BookieId** from **BookieSocketAddress**. + +Currently (up to 4.11.x) there is a too strict coupling from the ID of a Bookie (**BookieId**) and its network location (**BookieSocketAddress**). + +The **LedgerMetadata** structure contains the location of the entries of a ledger, and it stores BookieSocketAddresses (simply a hostname:port or ip:port pair). +The client uses this information to connect to the bookies and retrieve ledger data. + +So *the identity of a bookie is bound to the network address* of the primary endpoint of the Bookie: the '**bookie-rpc**' endpoint in terms of [BP-38](../BP-38-bookie-endpoint-discovery/) + +Limits of current version, because: + +- You cannot easily change the network address of a Bookie: a manual intervention is needed. +- The Bookie cannot have a **dynamic network address** or DNS name. +- You cannot use a custom Id for the bookie, something that is **meaningful** in the context of the deployment of the bookie. +- In case of future implementations that will open **multiple endpoints** on the bookie it is not possible to decide which endpoint should be used as BookieId. + +This proposal addresses these problems by proposing to separate the concept of **BookieId** from **BookieSocketAddress**. + +We will have to introduce a little break in the Client API, in order to switch from using BookieSocketAddress to a more opaque BookieId. + +Fortunately we will be able to keep compatibility with old clients and old bookies are far as the Bookie continues to use a BookieId that looks like a BookieSocketAddress. +See the paragraphs below for the details. + +### Public Interfaces + +We are introducing a new class BookieId that is simply a wrapper for a String. + +``` +final class BookieId { + private final String bookieId; + public String toString() { + return bookieId; + } + public static BookieId parse(String bookieId) { + // validation omitted... + return new BookieId(bookieId); + } +} +``` + +Having a class instead of a simple String is better because it provides a strongly typed API. + +The LedgerMetadata class will be changed to use BookieId instead of BookieSocketAddress. +This will break source and binary compatibility for Java clients, applications that use LedgerMetadata (usually for debug or for tools) will have to be recompiled. + +The serialized representation of a BookieSocketAddress, both for LedgerMetadata and Bookie Registration, is a simple String on ZooKeeper: this change is not about the format of data stored on Metadata Service. + +It is simply a pure refactor of Java interfaces. + +We have to introduce an internal API, **BookieAddressResolver**, that maps a *BookieId* to a *BookieSocketAddress*: the the client connectes to a Bookie it looks up the **current network address** using BookieAddressResolver. + +``` +interface BookieAddressResolver { + BookieSocketAddress resolve(BookieId id); +} +``` + +Initially it is not expected that the user provides a custom implementation of BookieAddressResolver. + + +It is expected that the implementation of this interface coordinates with the BookieWatcher and the RegistrationDriver in order to: +- map BookieId to BookieSocketAddress using `getBookieServiceInfo(BookieId id)` API +- cache efficiently this mapping in order to do not have a significant impact on the hot paths (reads and writes), and to save resources on the Metadata Service + +We provide an utility method BookieSocketAddress#toBookieId that helps particularly in test cases, this method simply returns a BookieId +built by the serialized representation of the BookieSocketAddress (hostname:port or ip:port) + +``` +final class BookieSocketAddress { + .... + BookieId toBookieId() { + return BookieId.parse(this.toString()); + } +} +``` + +The RegistrationClient and RegistrationManager interfaces will be refactored to use BookiId instead of String and BookieSocketAddress. + +The Bookie itself will expose an API to return the current BookieSocketAddress and current BookieId, this is useful for tests and for the RegistrationManager. + +The EnsemblePlacementPolicy and the BookieWatcher will deal with BookieIds and not with BookieSocketAddresses. + +The implementations of EnsemblePlacementPolicy that are aware of the network location of the Bookies will need to have access to the +BookieAddressResolver, in order to map a BookieId to the BookieSocketAddress and the BookieSocketAddress to the network location. + + +### Details on the proposed Changes + +#### BookieId validation + +The BookieId is a non empty string that can contain: +- ASCII digits and letters ([a-zA-Z9-0]) +- the colon character (':') +- the dash character ('-') +- the dot character ('.') + +#### BookKeeper Client Side Changes + +See the 'Public interfaces' section. + +On the client side code it will be clearer when we are dealing with BookieId, and basically the client API won't deal with network addresses anymore. +This change will be visible both on the legacy LedgerHandle API and on the new WriteHandle/ReadHandle API, basically because the new API is only a wrapper over the LedgerHandle API. + +When the BookKeeper client connects to a bookie (see **PerChannelBookieClient**) it uses the BookieAddressResolver interface to get the current BookieSocketAddress of the Bookie. +The key of the Connection Pool inside the BookieClient will be BookieId and no more BookieSocketAddress. + +#### Disabling BookieAddressResolver + +Using the BookieServiceInfo abstraction needs additional accesses to the Metadata Service (usually ZooKeeper) and this comes with a cost especially during the bootstrap of the client, because you have to resolve the address of every Bookie you are going to write to or to read from. + +We add a flag to disable the BookieAddressResolver, without this feature the client will be able only to connect to Bookies with the legacy BookieId. +In this case the BookieAddressResolver implementation will be a simple conversion from the BookieId, assuming the 4.11 format hostname:port. + +``` +enableBookieAddressResolver=true +``` +The *enableBookieAddressResolver* flag is used by the Client, by the Auditor and by all of the tools and it is enabled by default. + +#### Handling the Local Bookie Node in EnsemblePlacementPolicy +Unfortunately thru the codebase we used sometimes dummy BookieId that are not mapped to real Bookies, this happens in the EnsamblePlacementPolicies in which we create a BookieId for the 'local node' and using TCP port 0. In this case we have to implement a fallback dummy resolution that created a BookieSocketAddress without using the MetadataService + +#### Bookie Side Changes + +On the Bookie we introduce **a configuration option** (bookieid) to set a custom bookie id. +If you set this option then the Bookie will advertise itself on MetadataService using the configured id, and publish the 'bookie-rpc' endpoint as configured by +the **advertisedAddress** configuration option and the other network related options. +This BookieId will be present only in the configuration file and it is the key to lookup the *Cookie* on the MetadataService. +Inadvertently changing the BookieId will prevent the Bookie to boot as it won't find a matching Cookie. +There is no need to store the BookieId on the cookie or persist it on the local storage (ledgers, indexes or journal directories). + +#### Auditor and Replication Changes + +The Auditor deals with LedgerMetadata and so it will simply work with BookieIds and not with BookieSocketAddress. +When the Auditor needs to connect to a Bookie it will use the BookieAddressResolver to get the current address of the Bookie. + +#### Bookie Garbage Collection Changes + +The Bookie decides to reclaim space by looking into LedgerMetadata and checking that a given ledger does not exist anymore. +It will use its own local BookieId instead of the BookieSocketAddress as expected. + +#### Tools changes +All of the tools that deal with LedgerMetadata will use BookieId instead of BookieSocketAddress, in general this fact will allow to use free forn BookieIDs, +instead of hostname:port pairs (we had validations on tools that helped the user to use always BookieIds in hostname:port form). + +#### REST API Changes +In the REST API we will deal with BookieIds and not with BookieSocketAddresses anymore, the change will be straighforward and compatible with current API. +When new custom BookieIDs will be used then they will appear on the REST API as well, but this will be expected by users. + + +### Compatibility, Deprecation, and Migration Plan + +The proposed change will be compatible with all existing clients and bookies as far as you still use BookieIds in the hostname:port form and to not use a custom BookieId. +The Bookie by default will continue to use as BookieID a compatible value computed exactly as in version 4.11. +Incompatibility will start as soon as you enable custom BookieIDs on the bookies, from that point clients and old Auditors won't be able to deal with new bookies. +New clients will always be able to connect and use legacy bookies. + +Custom EnsemblePlacementPolicies must be adapted to the new interfaces but the change will usually as simple as just replacing BookieSocketAdress with BookieId. +No need to change address to rack mapping scripts, as they will still deal with raw DNS hostnames and not with BookieIds. + +### Test Plan + +New unit tests will be added to cover all of the code changes. +No need for additional integration tests. + +### Rejected Alternatives + +#### Make BookieSocketAddress an abstract class + +In order to preserve most of the binary compatibility in the Java client we could still keep BookieSocketAddress class in LedgerMetadata and have some "GenericBookieSocketAddress" and "PhysicalBookieSocketAddress" implementations. +But this way it won't be easy to understand where we are using a "bookie id" and when we are referring to a network address. +The BookieAddressResolver interface would be needed anyway and it should deal with pure BookieIds and BookieSocketAddress instance that are already resolved to +a network address. + +#### Force a specific format (like UUID) to custom BookieId +The is no need to force the BookieId to use a fixed format, like a UUID or other form of standard ID scheme. +Probably new IDs will include the region/availability zone information in order to simplify EnsemblePlacement policies (no more need to pass from DNS to switch mappers) and we cannot know now all of the usages of this parameter. diff --git a/site/community/bookkeeper_proposals.md b/site/community/bookkeeper_proposals.md index db836e91926..4b06a3c1dd2 100644 --- a/site/community/bookkeeper_proposals.md +++ b/site/community/bookkeeper_proposals.md @@ -85,7 +85,7 @@ using Google Doc. This section lists all the _bookkeeper proposals_ made to BookKeeper. -*Next Proposal Number: 38* +*Next Proposal Number: 41* ### Inprogress @@ -110,6 +110,7 @@ Proposal | State [BP-36: Stats documentation annotation](../../bps/BP-36-stats-documentation-annotation) | Accepted [BP-37: Improve configuration management for better documentation](../../bps/BP-37-conf-documentation) | Accepted [BP-38: Publish Bookie Service Info on Metadata Service](../../bps/BP-38-bookie-endpoint-discovery) | Accepted +[BP-41: Separate BookieId from Separate BookieId from Bookie Network Address](../../bps/BP-41-bookieid) | Draft ### Adopted From 74d21e143f4d60b2c9a19b301a93200db9a7a4f2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Thu, 13 Aug 2020 10:32:26 +0200 Subject: [PATCH 0458/1642] Issue #2385: NullPointerException in Zookeeper multiple operations execution with 3.6.1 Fix for #2385 Now we avoid to send out multiple operations request to ZooKeeper when there are not operations to execute Reviewers: Enrico Olivelli , Sijie Guo This closes #2390 from nicoloboschi/fix/2385/zk-npe, closes #2385 --- .../distributedlog/BKLogWriteHandler.java | 6 ++++-- .../distributedlog/zk/ZKTransaction.java | 21 ++++++++++++------- .../distributedlog/TestZooKeeperClient.java | 8 +++++++ 3 files changed, 25 insertions(+), 10 deletions(-) diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java index df830cba027..60ff5435b0c 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java @@ -1085,8 +1085,10 @@ CompletableFuture> setLogSegmentsOlderThanDLSNTruncated private CompletableFuture> setLogSegmentsOlderThanDLSNTruncated( List logSegments, final DLSN dlsn) { - LOG.debug("Setting truncation status on logs older than {} from {} for {}", - dlsn, logSegments, getFullyQualifiedName()); + if (LOG.isDebugEnabled()) { + LOG.debug("Setting truncation status on logs older than {} from {} for {}", + dlsn, logSegments, getFullyQualifiedName()); + } List truncateList = new ArrayList(logSegments.size()); LogSegmentMetadata partialTruncate = null; LOG.info("{}: Truncating log segments older than {}", getFullyQualifiedName(), dlsn); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java index 37d1477ce8c..a6024e20cef 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java @@ -18,6 +18,7 @@ package org.apache.distributedlog.zk; import com.google.common.collect.Lists; +import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; @@ -44,7 +45,7 @@ public ZKTransaction(ZooKeeperClient zkc) { this.zkc = zkc; this.ops = Lists.newArrayList(); this.zkOps = Lists.newArrayList(); - this.result = new CompletableFuture(); + this.result = new CompletableFuture<>(); } @Override @@ -63,13 +64,17 @@ public CompletableFuture execute() { if (!done.compareAndSet(false, true)) { return result; } - try { - zkc.get().multi(zkOps, this, result); - } catch (ZooKeeperClient.ZooKeeperConnectionException e) { - result.completeExceptionally(Utils.zkException(e, "")); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - result.completeExceptionally(Utils.zkException(e, "")); + if (zkOps.isEmpty()) { + this.processResult(KeeperException.Code.OK.intValue(), null, null, Collections.emptyList()); + } else { + try { + zkc.get().multi(zkOps, this, result); + } catch (ZooKeeperClient.ZooKeeperConnectionException e) { + result.completeExceptionally(Utils.zkException(e, "")); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + result.completeExceptionally(Utils.zkException(e, "")); + } } return result; } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java index 022b02819f2..a5888619f04 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java @@ -26,6 +26,7 @@ import java.lang.reflect.Field; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -33,6 +34,7 @@ import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; import org.apache.distributedlog.ZooKeeperClient.Credentials; import org.apache.distributedlog.ZooKeeperClient.DigestCredentials; +import org.apache.distributedlog.zk.ZKTransaction; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; @@ -459,4 +461,10 @@ public void run() { } assertEquals(ZooKeeper.States.CONNECTED, newZk.getState()); } + + @Test(timeout = 60000) + public void testZKTransactionEmptyOps() throws Exception { + CompletableFuture future = new ZKTransaction(zkc).execute(); + assertTrue(future.isDone()); + } } From c630dc3a35cf6e8a9ad99626c3964aa6523287b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Thu, 13 Aug 2020 10:33:57 +0200 Subject: [PATCH 0459/1642] Issue #2197: bkctl binary distribution needs a 'logs' directory Fix #2197 Create "logs" directory before run bkctl cli Reviewers: Enrico Olivelli , Sijie Guo This closes #2389 from nicoloboschi/fix/2197/bkclt-logs, closes #2197 --- bin/bkctl | 1 + 1 file changed, 1 insertion(+) diff --git a/bin/bkctl b/bin/bkctl index ab18d1ee905..fee0526ed2c 100755 --- a/bin/bkctl +++ b/bin/bkctl @@ -22,6 +22,7 @@ BINDIR=`dirname "$0"` BK_HOME=`cd ${BINDIR}/..;pwd` +mkdir -p $BK_HOME/logs source ${BK_HOME}/bin/common.sh source ${BK_HOME}/conf/bk_cli_env.sh From 4d21a26c63aa9a24445053fce820b5b801039c0a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Thu, 13 Aug 2020 10:36:04 +0200 Subject: [PATCH 0460/1642] Fix site 'responsbility' typo ### Motivation Found 'responsbility' typo Reviewers: Sijie Guo This closes #2392 from nicoloboschi/fix-doc-typo --- site/docs/4.10.0/getting-started/concepts.md | 2 +- site/docs/4.11.0/getting-started/concepts.md | 2 +- site/docs/4.11.1/getting-started/concepts.md | 2 +- site/docs/4.5.0/getting-started/concepts.md | 2 +- site/docs/4.5.1/getting-started/concepts.md | 2 +- site/docs/4.6.0/getting-started/concepts.md | 2 +- site/docs/4.6.1/getting-started/concepts.md | 2 +- site/docs/4.6.2/getting-started/concepts.md | 2 +- site/docs/4.7.0/getting-started/concepts.md | 2 +- site/docs/4.7.1/getting-started/concepts.md | 2 +- site/docs/4.7.2/getting-started/concepts.md | 2 +- site/docs/4.7.3/getting-started/concepts.md | 2 +- site/docs/4.8.0/getting-started/concepts.md | 2 +- site/docs/4.8.1/getting-started/concepts.md | 2 +- site/docs/4.8.2/getting-started/concepts.md | 2 +- site/docs/4.9.0/getting-started/concepts.md | 2 +- site/docs/4.9.1/getting-started/concepts.md | 2 +- site/docs/4.9.2/getting-started/concepts.md | 2 +- site/docs/latest/getting-started/concepts.md | 2 +- 19 files changed, 19 insertions(+), 19 deletions(-) diff --git a/site/docs/4.10.0/getting-started/concepts.md b/site/docs/4.10.0/getting-started/concepts.md index 2eeece1eaaa..8a829b0518a 100644 --- a/site/docs/4.10.0/getting-started/concepts.md +++ b/site/docs/4.10.0/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.11.0/getting-started/concepts.md b/site/docs/4.11.0/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.11.0/getting-started/concepts.md +++ b/site/docs/4.11.0/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.11.1/getting-started/concepts.md b/site/docs/4.11.1/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.11.1/getting-started/concepts.md +++ b/site/docs/4.11.1/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.5.0/getting-started/concepts.md b/site/docs/4.5.0/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.5.0/getting-started/concepts.md +++ b/site/docs/4.5.0/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.5.1/getting-started/concepts.md b/site/docs/4.5.1/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.5.1/getting-started/concepts.md +++ b/site/docs/4.5.1/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.6.0/getting-started/concepts.md b/site/docs/4.6.0/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.6.0/getting-started/concepts.md +++ b/site/docs/4.6.0/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.6.1/getting-started/concepts.md b/site/docs/4.6.1/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.6.1/getting-started/concepts.md +++ b/site/docs/4.6.1/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.6.2/getting-started/concepts.md b/site/docs/4.6.2/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.6.2/getting-started/concepts.md +++ b/site/docs/4.6.2/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.7.0/getting-started/concepts.md b/site/docs/4.7.0/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.7.0/getting-started/concepts.md +++ b/site/docs/4.7.0/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.7.1/getting-started/concepts.md b/site/docs/4.7.1/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.7.1/getting-started/concepts.md +++ b/site/docs/4.7.1/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.7.2/getting-started/concepts.md b/site/docs/4.7.2/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.7.2/getting-started/concepts.md +++ b/site/docs/4.7.2/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.7.3/getting-started/concepts.md b/site/docs/4.7.3/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.7.3/getting-started/concepts.md +++ b/site/docs/4.7.3/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.8.0/getting-started/concepts.md b/site/docs/4.8.0/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.8.0/getting-started/concepts.md +++ b/site/docs/4.8.0/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.8.1/getting-started/concepts.md b/site/docs/4.8.1/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.8.1/getting-started/concepts.md +++ b/site/docs/4.8.1/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.8.2/getting-started/concepts.md b/site/docs/4.8.2/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.8.2/getting-started/concepts.md +++ b/site/docs/4.8.2/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.9.0/getting-started/concepts.md b/site/docs/4.9.0/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.9.0/getting-started/concepts.md +++ b/site/docs/4.9.0/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.9.1/getting-started/concepts.md b/site/docs/4.9.1/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.9.1/getting-started/concepts.md +++ b/site/docs/4.9.1/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/4.9.2/getting-started/concepts.md b/site/docs/4.9.2/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/4.9.2/getting-started/concepts.md +++ b/site/docs/4.9.2/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs diff --git a/site/docs/latest/getting-started/concepts.md b/site/docs/latest/getting-started/concepts.md index 7a3c92847b2..6f34f18494d 100644 --- a/site/docs/latest/getting-started/concepts.md +++ b/site/docs/latest/getting-started/concepts.md @@ -39,7 +39,7 @@ Ledgers are sequences of entries, while each entry is a sequence of bytes. Entri * 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). +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 responsibility of [client applications](#clients). ## Clients and APIs From 01fbef236de53150e93a3efc6d47387f3985a16d Mon Sep 17 00:00:00 2001 From: maoling Date: Thu, 13 Aug 2020 16:38:23 +0800 Subject: [PATCH 0461/1642] The latency of BenchThroughputLatency may be wrong due to Integer overflow when we do a large scale benchmark test - When` latency.size` is is more than `20,000,000`(e.g, `25000000`), `(size * percentile) / 100` will be a negative number due to Integer overflow and the latency will be wrong. - For example: `size `= 25000000, `percentile `= 99, `sampleSize `= -18199672 Reviewers: Enrico Olivelli , Sijie Guo This closes #2381 from maoling/BenchThroughputLatency-latency-wrong --- .../apache/bookkeeper/benchmark/BenchThroughputLatency.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 c5ec3cb33ac..08c08e57cc3 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 @@ -419,7 +419,8 @@ public void process(WatchedEvent event) { private static double percentile(long[] latency, int percentile) { int size = latency.length; - int sampleSize = (size * percentile) / 100; + double percent = (double) percentile / 100; + int sampleSize = (int) (size * percent); long total = 0; int count = 0; for (int i = 0; i < sampleSize; i++) { From dd3738eaa529cb3c2b1e930af52a04ddbf86fe17 Mon Sep 17 00:00:00 2001 From: Rudy Steiner Date: Thu, 13 Aug 2020 16:39:44 +0800 Subject: [PATCH 0462/1642] NP check for print BookieSocketAddress and a better format Descriptions of the changes in this PR: ### Motivation NP check for print BookieSocketAddress and a better format to fix issues #2376 and #2371 ### Changes * NP check for print BookieSocketAddress on listbookies command * A better format mentioned on #2376 Master Issue: #2376 #2371 Reviewers: Enrico Olivelli , Sijie Guo , Rajan Dhabalia This closes #2380 from rudy2steiner/list_bookie --- .../tools/cli/helpers/CommandHelpers.java | 39 +++++++++++++------ .../bookies/ListBookiesCommandTest.java | 15 +++++-- 2 files changed, 39 insertions(+), 15 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java index 88ac52e4abf..a1dc1abd08c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java @@ -19,35 +19,50 @@ package org.apache.bookkeeper.tools.cli.helpers; import com.google.common.net.InetAddresses; +import java.net.InetAddress; import lombok.AccessLevel; import lombok.NoArgsConstructor; import org.apache.bookkeeper.net.BookieSocketAddress; + /** * Helper classes used by the cli commands. */ @NoArgsConstructor(access = AccessLevel.PRIVATE) public final class CommandHelpers { + private static final String UNKNOWN = "UNKNOWN"; /* * The string returned is of the form: - * 'hostname'('otherformofhostname'):'port number' - * - * where hostname and otherformofhostname are ipaddress and - * canonicalhostname or viceversa + * BookieID:bookieId, IP:ip, Port: port, Hostname: hostname + * When using hostname as bookie id, it's possible that the host is no longer valid and + * can't get a ip from the hostname, so using UNKNOWN to indicate ip is unknown for the hostname */ public static String getBookieSocketAddrStringRepresentation(BookieSocketAddress bookieId) { String hostname = bookieId.getHostName(); - boolean isHostNameIpAddress = InetAddresses.isInetAddress(hostname); - String otherFormOfHostname = null; - if (isHostNameIpAddress) { - otherFormOfHostname = bookieId.getSocketAddress().getAddress().getCanonicalHostName(); + String bookieID = bookieId.toString(); + String realHostname; + String ip = null; + if (InetAddresses.isInetAddress(hostname)){ + ip = hostname; + realHostname = bookieId.getSocketAddress().getAddress().getCanonicalHostName(); } else { - otherFormOfHostname = bookieId.getSocketAddress().getAddress().getHostAddress(); + InetAddress ia = bookieId.getSocketAddress().getAddress(); + if (null != ia){ + ip = ia.getHostAddress(); + } else { + ip = UNKNOWN; + } + realHostname = hostname; } - String bookieSocketAddrStringRepresentation = hostname + "(" + otherFormOfHostname + ")" + ":" - + bookieId.getSocketAddress().getPort(); - return bookieSocketAddrStringRepresentation; + return formatBookieSocketAddress(bookieID, ip, bookieId.getPort(), realHostname); + } + + /** + * Format {@link BookieSocketAddress}. + **/ + public static String formatBookieSocketAddress(String bookieId, String ip, int port, String hostName){ + return String.format("BookieID:%s, IP:%s, Port:%d, Hostname:%s", bookieId, ip, port, hostName); } } diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java index 9bd68a1e727..5f394e1b639 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java @@ -94,9 +94,12 @@ public void setup() throws Exception { private static Set createBookies(int startPort, int numBookies) { Set bookies = new TreeSet<>(new BookieAddressComparator()); - for (int i = 0; i < numBookies; i++) { + int i = 0; + for (; i < numBookies - 1; i++) { bookies.add(new BookieSocketAddress("127.0.0.1", startPort + i)); } + // mix an unknown hostname bookieId + bookies.add(new BookieSocketAddress("unknown", startPort + i)); return bookies; } @@ -105,8 +108,14 @@ private static void verifyPrintBookies(int startPort, int numBookies, int numCal PowerMockito.verifyStatic( CommandHelpers.class, times(numCalls)); - CommandHelpers.getBookieSocketAddrStringRepresentation( - eq(new BookieSocketAddress("127.0.0.1", startPort + 1))); + if (i == numBookies - 1){ + CommandHelpers.getBookieSocketAddrStringRepresentation( + eq(new BookieSocketAddress("unknown", startPort + i))); + } else { + CommandHelpers.getBookieSocketAddrStringRepresentation( + eq(new BookieSocketAddress("127.0.0.1", startPort + i))); + } + } } From c240026b27b964d1e192ed5161ece93d29b8f99d Mon Sep 17 00:00:00 2001 From: Rudy Steiner Date: Thu, 13 Aug 2020 16:42:17 +0800 Subject: [PATCH 0463/1642] Active ledgers on entry log file Descriptions of the changes in this PR: ### Motivation For troubleshooting bookkeeper disk usage issues, having a tool to list the ledgers that are still alive in the entry log file is super useful. ### Changes * Add ListActiveLedgerCommand on BookieShell Master Issue: #2358 Reviewers: Enrico Olivelli , Sijie Guo This closes #2369 from rudy2steiner/active_ledger_command --- .../apache/bookkeeper/bookie/BookieShell.java | 50 +++++ .../apache/bookkeeper/bookie/EntryLogger.java | 2 +- .../bookie/ListActiveLedgersCommand.java | 173 ++++++++++++++++++ .../metadata/ZKLogStreamMetadataStore.java | 1 + .../ZKSubscriptionStateStore.java | 1 + 5 files changed, 226 insertions(+), 1 deletion(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListActiveLedgersCommand.java 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 73edab939c5..ad2f23407b4 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 @@ -53,6 +53,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LedgerCommand; +import org.apache.bookkeeper.tools.cli.commands.bookie.ListActiveLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListFilesOnDiscCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand; import org.apache.bookkeeper.tools.cli.commands.bookie.LocalConsistencyCheckCommand; @@ -139,6 +140,7 @@ public class BookieShell implements Tool { static final String CMD_UPDATE_BOOKIE_IN_LEDGER = "updateBookieInLedger"; static final String CMD_DELETELEDGER = "deleteledger"; static final String CMD_BOOKIEINFO = "bookieinfo"; + static final String CMD_ACTIVE_LEDGERS_ON_ENTRY_LOG_FILE = "activeledgers"; static final String CMD_DECOMMISSIONBOOKIE = "decommissionbookie"; static final String CMD_ENDPOINTINFO = "endpointinfo"; static final String CMD_LOSTBOOKIERECOVERYDELAY = "lostbookierecoverydelay"; @@ -714,6 +716,53 @@ Options getOptions() { } } + /** + * List active ledgers on entry log file. + **/ + class ListActiveLedgersCmd extends MyCommand { + Options lOpts = new Options(); + + ListActiveLedgersCmd() { + super(CMD_ACTIVE_LEDGERS_ON_ENTRY_LOG_FILE); + lOpts.addOption("l", "logId", true, "Entry log file id"); + lOpts.addOption("t", "timeout", true, "Read timeout(ms)"); + } + + @Override + public int runCmd(CommandLine cmdLine) throws Exception { + final boolean hasTimeout = cmdLine.hasOption("t"); + final boolean hasLogId = cmdLine.hasOption("l"); + if (!hasLogId){ + printUsage(); + return -1; + } + final long logId = Long.parseLong(cmdLine.getOptionValue("l")); + ListActiveLedgersCommand.ActiveLedgerFlags flags = new ListActiveLedgersCommand.ActiveLedgerFlags(); + flags.logId(logId); + if (hasTimeout){ + flags.timeout(Long.parseLong(cmdLine.getOptionValue("t"))); + } + ListActiveLedgersCommand cmd = new ListActiveLedgersCommand(ledgerIdFormatter); + cmd.apply(bkConf, flags); + return 0; + } + + @Override + String getDescription() { + return "List all active ledgers on the entry log file."; + } + + @Override + String getUsage() { + return "activeledgers [-logId ] [-timeout ] [-formatter ]"; + } + + @Override + Options getOptions() { + return lOpts; + } + } + void printLedgerMetadata(long ledgerId, LedgerMetadata md, boolean printMeta) { System.out.println("ledgerID: " + ledgerIdFormatter.formatLedgerId(ledgerId)); if (printMeta) { @@ -2015,6 +2064,7 @@ int runCmd(CommandLine cmdLine) throws Exception { commands.put(CMD_LEDGER, new LedgerCmd()); commands.put(CMD_READ_LEDGER_ENTRIES, new ReadLedgerEntriesCmd()); commands.put(CMD_LISTLEDGERS, new ListLedgersCmd()); + commands.put(CMD_ACTIVE_LEDGERS_ON_ENTRY_LOG_FILE, new ListActiveLedgersCmd()); commands.put(CMD_LISTUNDERREPLICATED, new ListUnderreplicatedCmd()); commands.put(CMD_WHOISAUDITOR, new WhoIsAuditorCmd()); commands.put(CMD_WHATISINSTANCEID, new WhatIsInstanceId()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java index bc16a192ab9..062432a2c97 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java @@ -1056,7 +1056,7 @@ EntryLogMetadata extractEntryLogMetadataFromIndex(long entryLogId) throws IOExce if (header.ledgersMapOffset == 0L) { // The index was not stored in the log file (possibly because the bookie crashed before flushing it) - throw new IOException("No ledgers map index found on entryLogId" + entryLogId); + throw new IOException("No ledgers map index found on entryLogId " + entryLogId); } if (LOG.isDebugEnabled()) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListActiveLedgersCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListActiveLedgersCommand.java new file mode 100644 index 00000000000..91efe39b15e --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListActiveLedgersCommand.java @@ -0,0 +1,173 @@ +/** + * 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.tools.cli.commands.bookie; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; + +import com.beust.jcommander.Parameter; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.EntryLogger; +import org.apache.bookkeeper.bookie.ReadOnlyEntryLogger; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; +import org.apache.bookkeeper.tools.cli.commands.bookie.ListActiveLedgersCommand.ActiveLedgerFlags; +import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.bookkeeper.util.LedgerIdFormatter; +import org.apache.zookeeper.AsyncCallback.VoidCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * List active(exist in metadata storage) ledgers in a entry log file. + * + **/ +public class ListActiveLedgersCommand extends BookieCommand{ + static final Logger LOG = LoggerFactory.getLogger(ListActiveLedgersCommand.class); + private static final String NAME = "active ledger"; + private static final String DESC = "Retrieve bookie active ledger info."; + private static final long DEFAULT_TIME_OUT = 1000; + private static final long DEFAULT_LOG_ID = 0; + private static final String DEFAULT_LEDGER_ID_FORMATTER = ""; + private LedgerIdFormatter ledgerIdFormatter; + + public ListActiveLedgersCommand(){ + this(new ActiveLedgerFlags()); + } + public ListActiveLedgersCommand(LedgerIdFormatter ledgerIdFormatter){ + this(new ActiveLedgerFlags()); + this.ledgerIdFormatter = ledgerIdFormatter; + } + + public ListActiveLedgersCommand(ActiveLedgerFlags ledgerFlags){ + super(CliSpec.newBuilder(). + withName(NAME). + withDescription(DESC). + withFlags(ledgerFlags). + build()); + } + + /** + * Flags for active ledger command. + */ + @Accessors(fluent = true) + @Setter + public static class ActiveLedgerFlags extends CliFlags { + + @Parameter(names = { "-l", "--logid" }, description = "Entry log file id") + private long logId = DEFAULT_LOG_ID; + @Parameter(names = { "-t", "--timeout" }, description = "Read timeout(ms)") + private long timeout = DEFAULT_TIME_OUT; + @Parameter(names = { "-f", "--ledgerIdFormatter" }, description = "Ledger id formatter") + private String ledgerIdFormatter = DEFAULT_LEDGER_ID_FORMATTER; + } + @Override + public boolean apply(ServerConfiguration bkConf, ActiveLedgerFlags cmdFlags){ + initLedgerFormatter(bkConf, cmdFlags); + try { + handler(bkConf, cmdFlags); + } catch (MetadataException | ExecutionException e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + return true; + } + + private void initLedgerFormatter(ServerConfiguration conf, ActiveLedgerFlags cmdFlags) { + if (!cmdFlags.ledgerIdFormatter.equals(DEFAULT_LEDGER_ID_FORMATTER)) { + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(cmdFlags.ledgerIdFormatter, conf); + } else if (ledgerIdFormatter == null){ + this.ledgerIdFormatter = LedgerIdFormatter.newLedgerIdFormatter(conf); + } + } + + public void handler(ServerConfiguration bkConf, ActiveLedgerFlags cmdFlags) + throws ExecutionException, MetadataException { + runFunctionWithLedgerManagerFactory(bkConf, mFactory -> { + try (LedgerManager ledgerManager = mFactory.newLedgerManager()) { + Set activeLedgersOnMetadata = new HashSet(); + BookkeeperInternalCallbacks.Processor ledgerProcessor = (ledger, cb)->{ + activeLedgersOnMetadata.add(ledger); + cb.processResult(BKException.Code.OK, null, null); + }; + CountDownLatch done = new CountDownLatch(1); + AtomicInteger resultCode = new AtomicInteger(BKException.Code.OK); + VoidCallback endCallback = (rs, s, obj)->{ + resultCode.set(rs); + done.countDown(); + }; + ledgerManager.asyncProcessLedgers(ledgerProcessor, endCallback, null, + BKException.Code.OK, BKException.Code.ReadException); + if (done.await(cmdFlags.timeout, TimeUnit.MILLISECONDS)){ + if (resultCode.get() == BKException.Code.OK) { + EntryLogger entryLogger = new ReadOnlyEntryLogger(bkConf); + EntryLogMetadata entryLogMetadata = entryLogger.getEntryLogMetadata(cmdFlags.logId); + List ledgersOnEntryLog = entryLogMetadata.getLedgersMap().keys(); + if (ledgersOnEntryLog.size() == 0) { + LOG.info("Ledgers on log file {} is empty", cmdFlags.logId); + } + List activeLedgersOnEntryLog = new ArrayList(ledgersOnEntryLog.size()); + for (long ledger : ledgersOnEntryLog) { + if (activeLedgersOnMetadata.contains(ledger)) { + activeLedgersOnEntryLog.add(ledger); + } + } + printActiveLedgerOnEntryLog(cmdFlags.logId, activeLedgersOnEntryLog); + } else { + LOG.info("Read active ledgers id from metadata store,fail code {}", resultCode.get()); + throw BKException.create(resultCode.get()); + } + } else { + LOG.info("Read active ledgers id from metadata store timeout"); + } + } catch (BKException | InterruptedException | IOException e){ + LOG.error("Received Exception while processing ledgers", e); + throw new UncheckedExecutionException(e); + } + return null; + }); + } + + public void printActiveLedgerOnEntryLog(long logId, List activeLedgers){ + if (activeLedgers.size() == 0){ + System.out.println("No active ledgers on log file " + logId); + } else { + System.out.println("Active ledgers on entry log " + logId + " as follow:"); + } + Collections.sort(activeLedgers); + for (long a:activeLedgers){ + System.out.println(ledgerIdFormatter.formatLedgerId(a) + " "); + } + } +} diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java index e16cec88705..17f8113b982 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.net.URI; import java.util.Collections; + import java.util.LinkedList; import java.util.List; import java.util.Optional; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java index bd60b4179c8..2b486d34e77 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java @@ -17,6 +17,7 @@ */ package org.apache.distributedlog.impl.subscription; + import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; From f93d476408edb0964f238a47b75fc083c5f428ba Mon Sep 17 00:00:00 2001 From: hangc0276 Date: Thu, 13 Aug 2020 16:43:48 +0800 Subject: [PATCH 0464/1642] Bookie Client add quarantine ratio when error count exceed threshold ### Motivation When bookie client read/write data from/to bookie servers, it will check the health of each connected server in sepecific interval. Once the amount of errors reached the threshold, the bookie server will be quarantined for server miniutes (configurated by `bookieQuarantineTimeSeconds`) by the bookie client. In most circumstance, there are large amount of bookie clients connected to one bookie server, like pulsar broker. Once the bookie server runs in heavy load, most of bookie clients will receive errors and trigger quarantine in the same time, and then quarantine the server for several miniutes. After a few miniutes passed by, the quarantined server will be put back in the same time for most bookie clients, which will lead to periodic oscillation of in/out throughput of the server. It is the obstacle of tunning the throughput of the bookkeeper cluster. ### Changes I introduce a quarantine probability to determine whether to quarantine the server for the client, avoiding quaraninte the heavy load server in the same time for most of bookie client. I also expose the quarantine stats to prometheus. Reviewers: Jia Zhai , Sijie Guo This closes #2327 from hangc0276/bookieClient_Quarantine_ratio --- .../bookie/BookKeeperServerStats.java | 4 ++++ .../apache/bookkeeper/client/BookKeeper.java | 11 +++++++++- .../bookkeeper/conf/ClientConfiguration.java | 21 +++++++++++++++++++ 3 files changed, 35 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java index bdca0481a07..bc6f7264f88 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookKeeperServerStats.java @@ -74,6 +74,10 @@ public interface BookKeeperServerStats { String FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER = "FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER"; String ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER = "ENSEMBLE_NOT_ADHERING_TO_PLACEMENT_POLICY_COUNTER"; + // Bookie Quarantine Stats + String BOOKIE_QUARANTINE = "BOOKIE_QUARANTINE"; + String BOOKIE_QUARANTINE_SKIP = "BOOKIE_QUARANTINE_SKIP"; + // Bookie Operations String BOOKIE_ADD_ENTRY = "BOOKIE_ADD_ENTRY"; String BOOKIE_RECOVERY_ADD_ENTRY = "BOOKIE_RECOVERY_ADD_ENTRY"; 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 52dde9cba6e..0638a877902 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 @@ -44,6 +44,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.bookkeeper.bookie.BookKeeperServerStats; import org.apache.bookkeeper.client.AsyncCallback.CreateCallback; import org.apache.bookkeeper.client.AsyncCallback.DeleteCallback; import org.apache.bookkeeper.client.AsyncCallback.IsClosedCallback; @@ -111,6 +112,7 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper { // The stats logger for this client. private final StatsLogger statsLogger; private final BookKeeperClientStats clientStats; + private final double bookieQuarantineRatio; // whether the event loop group is one we created, or is owned by whoever // instantiated us @@ -522,6 +524,7 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo this.ledgerManager = new CleanupLedgerManager(ledgerManagerFactory.newLedgerManager()); this.ledgerIdGenerator = ledgerManagerFactory.newLedgerIdGenerator(); + this.bookieQuarantineRatio = conf.getBookieQuarantineRatio(); scheduleBookieHealthCheckIfEnabled(conf); } @@ -549,6 +552,7 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo bookieInfoScheduler = null; bookieClient = null; allocator = UnpooledByteBufAllocator.DEFAULT; + bookieQuarantineRatio = 1.0; } private EnsemblePlacementPolicy initializeEnsemblePlacementPolicy(ClientConfiguration conf, @@ -598,7 +602,12 @@ public void safeRun() { void checkForFaultyBookies() { List faultyBookies = bookieClient.getFaultyBookies(); for (BookieSocketAddress faultyBookie : faultyBookies) { - bookieWatcher.quarantineBookie(faultyBookie); + if (Math.random() <= bookieQuarantineRatio) { + bookieWatcher.quarantineBookie(faultyBookie); + statsLogger.getCounter(BookKeeperServerStats.BOOKIE_QUARANTINE).inc(); + } else { + statsLogger.getCounter(BookKeeperServerStats.BOOKIE_QUARANTINE_SKIP).inc(); + } } } 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 4b255f430c0..eddb913d06c 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 @@ -137,6 +137,7 @@ public class ClientConfiguration extends AbstractConfiguration Date: Sat, 15 Aug 2020 08:34:18 +0200 Subject: [PATCH 0465/1642] Issue 2399 Move CI builds to ci-hadoop.apache.org - rename self job seed configuration in order to disable it temporary --- .test-infra/jenkins/{job_seed.groovy => job_seed.grrovy.bak} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename .test-infra/jenkins/{job_seed.groovy => job_seed.grrovy.bak} (100%) diff --git a/.test-infra/jenkins/job_seed.groovy b/.test-infra/jenkins/job_seed.grrovy.bak similarity index 100% rename from .test-infra/jenkins/job_seed.groovy rename to .test-infra/jenkins/job_seed.grrovy.bak From 1e9a2dcc7fb19b136759500ea3629d8f44f73ba7 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 26 Aug 2020 16:46:28 -0700 Subject: [PATCH 0466/1642] Trigger flush if write cache is full while waiting (#2402) Under heavy load, the write cache can fill up faster than it can be flushed. When this occurs, something needs to trigger the flush. Previously we've triggered the flush and then gone into a loop waiting to be able to add to the write cache. However, if the currently active write cache fills while we are waiting for the flushing write cache to be flushed, we can end up in a situation where nothing is able to flush because all I/O threads are waiting for the write cache to empty. The solution is to move the flush into the loop so that if we fail to add to the cache, we flush the write cache if it is needed. Co-authored-by: Ivan Kelly --- .../ldb/SingleDirectoryDbLedgerStorage.java | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 931c3e00720..9ff35bf5a69 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -338,25 +338,25 @@ public long addEntry(ByteBuf entry) throws IOException, BookieException { private void triggerFlushAndAddEntry(long ledgerId, long entryId, ByteBuf entry) throws IOException, BookieException { - // Write cache is full, we need to trigger a flush so that it gets rotated - // If the flush has already been triggered or flush has already switched the - // cache, we don't need to trigger another flush - if (!isFlushOngoing.get() && hasFlushBeenTriggered.compareAndSet(false, true)) { - // Trigger an early flush in background - log.info("Write cache is full, triggering flush"); - executor.execute(() -> { - try { - flush(); - } catch (IOException e) { - log.error("Error during flush", e); - } - }); - } - dbLedgerStorageStats.getThrottledWriteRequests().inc(); long absoluteTimeoutNanos = System.nanoTime() + maxThrottleTimeNanos; while (System.nanoTime() < absoluteTimeoutNanos) { + // Write cache is full, we need to trigger a flush so that it gets rotated + // If the flush has already been triggered or flush has already switched the + // cache, we don't need to trigger another flush + if (!isFlushOngoing.get() && hasFlushBeenTriggered.compareAndSet(false, true)) { + // Trigger an early flush in background + log.info("Write cache is full, triggering flush"); + executor.execute(() -> { + try { + flush(); + } catch (IOException e) { + log.error("Error during flush", e); + } + }); + } + long stamp = writeCacheRotationLock.readLock(); try { if (writeCache.put(ledgerId, entryId, entry)) { From 2b51eed93116eb3c17e9fc6088b495c3c34f1527 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 7 Sep 2020 09:07:56 +0200 Subject: [PATCH 0467/1642] BP-41 BookieId - client side implementation (#2404) ### Motivation This is the client side implementation of BP-41, see the design doc here https://github.com/apache/bookkeeper/blob/master/site/bps/BP-41-bookieid.md ### Changes - Introduce BookeId concept - Separated BookieId from BookieSocketAddress - in LedgerMetadata we use now BookieId and not BookieSocketAddress - a BookieId is basically a wrapper around a string - BookieId is compatible with BookieSocketAddress - In order to *resolve" a BookieId to a BookieSocketAddress we use a BookieAddressResolver - the DefaultBookieAddressResolver implementation uses BookieServiceInfo metadata, read from ZooKeeper in order to discover the current network address of the Bookie (see BP-38) Unfortunately the patch is huge due to the changes in the client API and LedgerMetadata. This patch tries to be the smallest set of changes, see #2396 for the list of the follow up patches that will be contributed. Notes: - The new client and the bookie is 100% compatible with previous Bookies and Clients (and integration tests were not touched) - We are going to introduce, in a follow up patch, a flag to not use the metadata on ZooKeeper for the resolution of the BookieSocketAddress, in order to save resources - The ability to configure a custom BookieId, different from the network address will come in a separate patch Master Issue: #2396 Co-authored-by: Enrico Olivelli Co-authored-by: Enrico Olivelli --- .../bookkeeper/benchmark/BenchBookie.java | 13 +- .../bookkeeper/benchmark/TestBenchmark.java | 2 +- .../bookkeeper/common/util/JsonUtil.java | 8 +- bookkeeper-server/pom.xml | 1 + .../org/apache/bookkeeper/bookie/Bookie.java | 25 +- .../apache/bookkeeper/bookie/BookieShell.java | 4 +- .../bookkeeper/bookie/BookieStateManager.java | 7 +- .../org/apache/bookkeeper/bookie/Cookie.java | 24 +- .../LocalBookieEnsemblePlacementPolicy.java | 34 +- .../ScanAndCompareGarbageCollector.java | 10 +- .../apache/bookkeeper/client/BKException.java | 3 + .../apache/bookkeeper/client/BookKeeper.java | 34 +- .../bookkeeper/client/BookKeeperAdmin.java | 178 ++- .../bookkeeper/client/BookieInfoReader.java | 44 +- .../bookkeeper/client/BookieWatcher.java | 25 +- .../bookkeeper/client/BookieWatcherImpl.java | 63 +- .../bookkeeper/client/BookiesHealthInfo.java | 6 +- .../client/DefaultBookieAddressResolver.java | 68 + .../DefaultEnsemblePlacementPolicy.java | 76 +- .../client/DistributionSchedule.java | 6 +- .../client/EnsemblePlacementPolicy.java | 38 +- .../bookkeeper/client/EnsembleUtils.java | 22 +- .../bookkeeper/client/ForceLedgerOp.java | 8 +- ...ITopologyAwareEnsemblePlacementPolicy.java | 14 +- .../bookkeeper/client/LedgerChecker.java | 10 +- .../bookkeeper/client/LedgerCreateOp.java | 8 +- .../bookkeeper/client/LedgerFragment.java | 14 +- .../client/LedgerFragmentReplicator.java | 24 +- .../bookkeeper/client/LedgerHandle.java | 50 +- .../client/LedgerMetadataBuilder.java | 8 +- .../bookkeeper/client/LedgerMetadataImpl.java | 12 +- .../client/LedgerMetadataUtils.java | 14 +- .../bookkeeper/client/PendingAddOp.java | 16 +- .../bookkeeper/client/PendingReadLacOp.java | 6 +- .../bookkeeper/client/PendingReadOp.java | 46 +- .../bookkeeper/client/PendingWriteLacOp.java | 8 +- .../RackawareEnsemblePlacementPolicy.java | 41 +- .../RackawareEnsemblePlacementPolicyImpl.java | 86 +- .../client/ReadLastConfirmedAndEntryOp.java | 42 +- .../client/ReadLastConfirmedOp.java | 6 +- .../client/ReadOnlyLedgerHandle.java | 12 +- .../RegionAwareEnsemblePlacementPolicy.java | 68 +- .../RoundRobinDistributionSchedule.java | 12 +- .../TopologyAwareEnsemblePlacementPolicy.java | 66 +- .../client/TryReadLastConfirmedOp.java | 6 +- .../bookkeeper/client/UpdateLedgerOp.java | 12 +- .../ZoneawareEnsemblePlacementPolicy.java | 35 +- .../ZoneawareEnsemblePlacementPolicyImpl.java | 114 +- .../bookkeeper/client/api/LedgerMetadata.java | 6 +- .../discover/RegistrationClient.java | 18 +- .../discover/RegistrationManager.java | 13 +- .../discover/ZKRegistrationClient.java | 113 +- .../discover/ZKRegistrationManager.java | 31 +- .../bookkeeper/meta/LedgerMetadataSerDe.java | 22 +- .../org/apache/bookkeeper/net/BookieId.java | 90 ++ .../org/apache/bookkeeper/net/BookieNode.java | 6 +- .../bookkeeper/net/BookieSocketAddress.java | 51 + .../proto/BookieAddressResolver.java | 53 + .../apache/bookkeeper/proto/BookieClient.java | 30 +- .../bookkeeper/proto/BookieClientImpl.java | 57 +- .../bookkeeper/proto/BookieNettyServer.java | 5 +- .../apache/bookkeeper/proto/BookieServer.java | 15 +- .../proto/BookkeeperInternalCallbacks.java | 8 +- .../DefaultPerChannelBookieClientPool.java | 6 +- .../proto/ForceLedgerProcessorV3.java | 4 +- .../proto/LocalBookiesRegistry.java | 10 +- .../proto/PerChannelBookieClient.java | 246 +++- .../proto/PerChannelBookieClientFactory.java | 4 +- .../ReadLastConfirmedAndEntryContext.java | 8 +- .../bookkeeper/proto/WriteEntryProcessor.java | 4 +- .../proto/WriteEntryProcessorV3.java | 4 +- .../bookkeeper/proto/WriteLacProcessorV3.java | 4 +- .../bookkeeper/replication/Auditor.java | 61 +- .../replication/AuditorElector.java | 8 +- .../replication/BookieLedgerIndexer.java | 9 +- .../replication/ReplicationWorker.java | 10 +- .../http/service/DecommissionService.java | 6 +- .../http/service/ListBookieInfoService.java | 6 +- .../http/service/ListBookiesService.java | 14 +- .../http/service/RecoveryBookieService.java | 9 +- .../http/service/WhoIsAuditorService.java | 9 +- .../autorecovery/WhoIsAuditorCommand.java | 7 +- .../commands/bookie/FlipBookieIdCommand.java | 6 +- .../commands/bookie/ListLedgersCommand.java | 6 +- .../commands/bookie/ReadLedgerCommand.java | 9 +- .../bookie/UpdateBookieInLedgerCommand.java | 14 +- .../commands/bookies/DecommissionCommand.java | 9 +- .../commands/bookies/EndpointInfoCommand.java | 6 +- .../cli/commands/bookies/InfoCommand.java | 13 +- .../commands/bookies/ListBookiesCommand.java | 22 +- .../cli/commands/bookies/RecoverCommand.java | 33 +- .../cli/commands/cookie/AdminCommand.java | 6 +- .../cli/commands/cookie/CookieCommand.java | 5 +- .../commands/cookie/CreateCookieCommand.java | 3 +- .../commands/cookie/DeleteCookieCommand.java | 3 +- .../cookie/GenerateCookieCommand.java | 5 +- .../cli/commands/cookie/GetCookieCommand.java | 8 +- .../commands/cookie/UpdateCookieCommand.java | 3 +- .../tools/cli/helpers/CommandHelpers.java | 18 +- .../bookie/AdvertisedAddressTest.java | 9 +- .../bookie/BookieInitializationTest.java | 26 +- .../bookie/BookieJournalForceTest.java | 14 +- .../BookieJournalPageCacheFlushTest.java | 10 +- .../bookkeeper/bookie/BookieShellTest.java | 22 +- .../bookie/BookieWriteToJournalTest.java | 12 +- .../bookie/GcOverreplicatedLedgerTest.java | 22 +- .../bookkeeper/bookie/LedgerCacheTest.java | 4 +- .../client/BookKeeperAdminTest.java | 43 +- .../client/BookKeeperCloseTest.java | 4 +- ...rDiskSpaceWeightedLedgerPlacementTest.java | 138 +- .../bookkeeper/client/BookKeeperTest.java | 10 +- .../client/BookKeeperTestClient.java | 8 +- .../client/BookieDecommissionTest.java | 8 +- .../bookkeeper/client/BookieRecoveryTest.java | 48 +- .../client/BookieWriteLedgerTest.java | 24 +- .../bookkeeper/client/DeferredSyncTest.java | 6 +- .../GenericEnsemblePlacementPolicyTest.java | 14 +- .../bookkeeper/client/HandleFailuresTest.java | 11 +- .../bookkeeper/client/LedgerClose2Test.java | 11 +- .../bookkeeper/client/LedgerCloseTest.java | 4 +- .../bookkeeper/client/LedgerMetadataTest.java | 15 +- .../client/LedgerRecovery2Test.java | 13 +- .../bookkeeper/client/LedgerRecoveryTest.java | 14 +- .../client/MetadataUpdateLoopTest.java | 66 +- .../client/MockBookKeeperTestCase.java | 67 +- .../bookkeeper/client/MockClientContext.java | 4 +- .../bookkeeper/client/MockLedgerHandle.java | 8 +- .../client/ParallelLedgerRecoveryTest.java | 12 +- .../ReadLastConfirmedAndEntryOpTest.java | 20 +- .../client/ReadLastConfirmedOpTest.java | 7 +- .../bookkeeper/client/SlowBookieTest.java | 6 +- .../client/TestAddEntryQuorumTimeout.java | 8 +- .../client/TestBookieHealthCheck.java | 10 +- .../client/TestDelayEnsembleChange.java | 20 +- .../client/TestDisableEnsembleChange.java | 12 +- .../apache/bookkeeper/client/TestFencing.java | 6 +- .../client/TestGetBookieInfoTimeout.java | 11 +- .../bookkeeper/client/TestLedgerChecker.java | 44 +- .../client/TestLedgerFragmentReplication.java | 44 +- .../bookkeeper/client/TestParallelRead.java | 8 +- .../TestRackawareEnsemblePlacementPolicy.java | 1159 +++++++++-------- ...areEnsemblePlacementPolicyUsingScript.java | 229 ++-- ...estRackawarePolicyNotificationUpdates.java | 19 +- .../client/TestReadEntryListener.java | 6 +- .../bookkeeper/client/TestReadTimeout.java | 8 +- ...estRegionAwareEnsemblePlacementPolicy.java | 804 ++++++------ .../client/TestSpeculativeRead.java | 8 +- .../client/TestWatchEnsembleChange.java | 14 +- .../TestZoneawareEnsemblePlacementPolicy.java | 718 +++++----- .../client/UpdateLedgerCmdTest.java | 13 +- .../bookkeeper/client/UpdateLedgerOpTest.java | 49 +- .../discover/BookieServiceInfoTest.java | 9 +- .../discover/MockRegistrationClient.java | 30 +- .../discover/TestZkRegistrationClient.java | 90 +- .../meta/AbstractZkLedgerManagerTest.java | 12 +- .../apache/bookkeeper/meta/GcLedgersTest.java | 9 +- .../meta/LedgerManagerIteratorTest.java | 8 +- .../meta/TestLedgerMetadataSerDe.java | 20 +- .../apache/bookkeeper/net/BookieIdTest.java | 90 ++ ...a => ResolvedBookieSocketAddressTest.java} | 2 +- .../bookkeeper/proto/MockBookieClient.java | 42 +- .../proto/TestBackwardCompatCMS42.java | 17 +- .../proto/TestPerChannelBookieClient.java | 20 +- .../replication/AuditorBookieTest.java | 13 +- .../replication/AuditorLedgerCheckerTest.java | 23 +- .../AuditorPeriodicBookieCheckTest.java | 7 +- .../replication/AuditorPeriodicCheckTest.java | 22 +- .../AuditorPlacementPolicyCheckTest.java | 57 +- .../replication/AuditorReplicasCheckTest.java | 57 +- .../AuditorRollingRestartTest.java | 4 +- .../replication/AutoRecoveryMainTest.java | 10 +- .../replication/BookieAutoRecoveryTest.java | 40 +- ...estAutoRecoveryAlongWithBookieServers.java | 12 +- .../replication/TestReplicationWorker.java | 52 +- .../apache/bookkeeper/server/TestMain.java | 3 + .../server/http/TestHttpService.java | 5 +- .../http/service/ListLedgerServiceTest.java | 7 +- .../test/BookKeeperClusterTestCase.java | 80 +- .../bookkeeper/test/BookieClientTest.java | 20 +- .../bookkeeper/test/ConcurrentLedgerTest.java | 4 +- .../test/LocalBookiesRegistryTest.java | 6 +- .../bookkeeper/test/ReadOnlyBookieTest.java | 6 +- .../org/apache/bookkeeper/tls/TestTLS.java | 14 +- .../bookkeeper/util/StaticDNSResolver.java | 5 +- .../metadata/etcd/EtcdRegistrationClient.java | 37 +- .../etcd/EtcdRegistrationManager.java | 19 +- .../bookkeeper/metadata/etcd/EtcdUtils.java | 13 +- .../metadata/etcd/EtcdClusterTest.java | 5 +- .../metadata/etcd/EtcdCookieTest.java | 3 +- .../metadata/etcd/EtcdLedgerManagerTest.java | 14 +- .../metadata/etcd/EtcdRegistrationTest.java | 55 +- pom.xml | 1 + .../resolver/BKRegistrationNameResolver.java | 9 +- .../BKRegistrationNameResolverTest.java | 5 +- stream/distributedlog/core/pom.xml | 1 + .../bookkeeper/client/LedgerReader.java | 11 +- .../tools/DistributedLogTool.java | 32 +- .../server/service/BookieWatchService.java | 4 +- .../service/RegistrationStateService.java | 3 +- .../cluster/ClusterControllerLeaderImpl.java | 8 +- .../sc/DefaultStorageContainerController.java | 50 +- .../impl/sc/StorageContainerController.java | 4 +- .../ClusterControllerLeaderImplTest.java | 13 +- ...DefaultStorageContainerControllerTest.java | 61 +- .../cluster/BookKeeperClusterTestBase.java | 14 +- .../cluster/SimpleClusterTest.java | 8 +- .../bookkeeper/tests/integration/TestCLI.java | 6 +- .../SearchReplaceBookieIdCommand.java | 8 +- .../autorecovery/WhoIsAuditorCommandTest.java | 11 +- .../bookie/FlipBookieIdCommandTest.java | 7 +- .../bookie/ListLedgersCommandTest.java | 10 +- .../bookie/ReadLedgerCommandTest.java | 16 +- .../bookies/DecommissionCommandTest.java | 15 +- .../cli/commands/bookies/InfoCommandTest.java | 9 +- .../bookies/ListBookiesCommandTest.java | 46 +- .../commands/bookies/RecoverCommandTest.java | 20 +- .../cli/commands/cookie/AdminCommandTest.java | 10 +- .../cookie/CreateCookieCommandTest.java | 11 +- .../cookie/DeleteCookieCommandTest.java | 11 +- .../commands/cookie/GetCookieCommandTest.java | 15 +- .../cookie/UpdateCookieCommandTest.java | 11 +- 221 files changed, 4318 insertions(+), 3469 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultBookieAddressResolver.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieId.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieAddressResolver.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieIdTest.java rename bookkeeper-server/src/test/java/org/apache/bookkeeper/net/{BookieSocketAddressTest.java => ResolvedBookieSocketAddressTest.java} (97%) diff --git a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java index d91b7256097..a4d2ae99050 100644 --- a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java +++ b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchBookie.java @@ -35,6 +35,7 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; @@ -63,7 +64,7 @@ static class LatencyCallback implements WriteCallback { boolean complete; @Override public synchronized void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (rc != 0) { LOG.error("Got error " + rc); } @@ -85,7 +86,7 @@ static class ThroughputCallback implements WriteCallback { int waitingCount = Integer.MAX_VALUE; @Override public synchronized void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (rc != 0) { LOG.error("Got error " + rc); } @@ -178,7 +179,7 @@ public static void main(String[] args) ClientConfiguration conf = new ClientConfiguration(); BookieClient bc = new BookieClientImpl(conf, eventLoop, PooledByteBufAllocator.DEFAULT, executor, scheduler, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); LatencyCallback lc = new LatencyCallback(); ThroughputCallback tc = new ThroughputCallback(); @@ -191,7 +192,7 @@ public static void main(String[] args) toSend.writeLong(ledger); toSend.writeLong(entry); toSend.writerIndex(toSend.capacity()); - bc.addEntry(new BookieSocketAddress(addr, port), ledger, new byte[20], + bc.addEntry(new BookieSocketAddress(addr, port).toBookieId(), ledger, new byte[20], entry, ByteBufList.get(toSend), tc, null, BookieProtocol.FLAG_NONE, false, WriteFlag.NONE); } @@ -209,7 +210,7 @@ public static void main(String[] args) toSend.writeLong(entry); toSend.writerIndex(toSend.capacity()); lc.resetComplete(); - bc.addEntry(new BookieSocketAddress(addr, port), ledger, new byte[20], + bc.addEntry(new BookieSocketAddress(addr, port).toBookieId(), ledger, new byte[20], entry, ByteBufList.get(toSend), lc, null, BookieProtocol.FLAG_NONE, false, WriteFlag.NONE); lc.waitForComplete(); @@ -228,7 +229,7 @@ public static void main(String[] args) toSend.writeLong(ledger); toSend.writeLong(entry); toSend.writerIndex(toSend.capacity()); - bc.addEntry(new BookieSocketAddress(addr, port), ledger, new byte[20], + bc.addEntry(new BookieSocketAddress(addr, port).toBookieId(), ledger, new byte[20], entry, ByteBufList.get(toSend), tc, null, BookieProtocol.FLAG_NONE, false, WriteFlag.NONE); } diff --git a/bookkeeper-benchmark/src/test/java/org/apache/bookkeeper/benchmark/TestBenchmark.java b/bookkeeper-benchmark/src/test/java/org/apache/bookkeeper/benchmark/TestBenchmark.java index bee5f90d335..bb1e37cfd18 100644 --- a/bookkeeper-benchmark/src/test/java/org/apache/bookkeeper/benchmark/TestBenchmark.java +++ b/bookkeeper-benchmark/src/test/java/org/apache/bookkeeper/benchmark/TestBenchmark.java @@ -68,7 +68,7 @@ public void testThroughputLatency() throws Exception { @Test public void testBookie() throws Exception { - BookieSocketAddress bookie = getBookie(0); + BookieSocketAddress bookie = getBookieAddress(0); BenchBookie.main(new String[] { "--host", bookie.getSocketAddress().getHostName(), "--port", String.valueOf(bookie.getPort()), diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/JsonUtil.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/JsonUtil.java index afc90a4f48b..90576aeab1a 100644 --- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/JsonUtil.java +++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/JsonUtil.java @@ -30,7 +30,7 @@ public static String toJson(Object object) throws ParseJsonException { try { return mapper.writerWithDefaultPrettyPrinter().writeValueAsString(object); } catch (Exception e) { - throw new ParseJsonException("Failed to serialize Object to Json string"); + throw new ParseJsonException("Failed to serialize Object to Json string", e); } } @@ -38,7 +38,7 @@ public static T fromJson(String jsonStr, Class valueType) throws ParseJso try { return mapper.readValue(jsonStr, valueType); } catch (Exception e) { - throw new ParseJsonException("Failed to deserialize Object from Json string"); + throw new ParseJsonException("Failed to deserialize Object from Json string", e); } } @@ -49,5 +49,9 @@ public static class ParseJsonException extends Exception { public ParseJsonException(String message) { super(message); } + + public ParseJsonException(String message, Throwable cause) { + super(message, cause); + } } } diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index e990b4d6347..adae9f68694 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -219,6 +219,7 @@ maven-surefire-plugin ${maven-surefire-plugin.version} + false listener 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 6a2209aaeee..a7960f79b0c 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 @@ -81,6 +81,7 @@ import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNS; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; @@ -188,7 +189,7 @@ public long getEntry() { static class NopWriteCallback implements WriteCallback { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (LOG.isDebugEnabled()) { LOG.debug("Finished writing entry {} @ ledger {} for {} : {}", entryId, ledgerId, addr, rc); @@ -302,11 +303,11 @@ private void checkIfDirsOnSameDiskPartition(List dirs) throws DiskPartitio } } - static List possibleBookieIds(ServerConfiguration conf) + static List possibleBookieIds(ServerConfiguration conf) throws BookieException { // we need to loop through all possible bookie identifiers to ensure it is treated as a new environment // just because of bad configuration - List addresses = Lists.newArrayListWithExpectedSize(3); + List addresses = Lists.newArrayListWithExpectedSize(3); // we are checking all possibilities here, so we don't need to fail if we can only get // loopback address. it will fail anyway when the bookie attempts to listen on loopback address. try { @@ -316,17 +317,17 @@ static List possibleBookieIds(ServerConfiguration conf) .setUseHostNameAsBookieID(false) .setAdvertisedAddress(null) .setAllowLoopback(true) - )); + ).toBookieId()); // host name addresses.add(getBookieAddress( new ServerConfiguration(conf) .setUseHostNameAsBookieID(true) .setAdvertisedAddress(null) .setAllowLoopback(true) - )); + ).toBookieId()); // advertised address if (null != conf.getAdvertisedAddress()) { - addresses.add(getBookieAddress(conf)); + addresses.add(getBookieAddress(conf).toBookieId()); } } catch (UnknownHostException e) { throw new UnknownBookieIdException(e); @@ -336,10 +337,10 @@ static List possibleBookieIds(ServerConfiguration conf) static Versioned readAndVerifyCookieFromRegistrationManager( Cookie masterCookie, RegistrationManager rm, - List addresses, boolean allowExpansion) + List addresses, boolean allowExpansion) throws BookieException { Versioned rmCookie = null; - for (BookieSocketAddress address : addresses) { + for (BookieId address : addresses) { try { rmCookie = Cookie.readFromRegistrationManager(rm, address); // If allowStorageExpansion option is set, we should @@ -419,7 +420,7 @@ public static void checkEnvironmentWithStorageExpansion( // 3. read the cookie from registration manager. it is the `source-of-truth` of a given bookie. // if it doesn't exist in registration manager, this bookie is a new bookie, otherwise it is // an old bookie. - List possibleBookieIds = possibleBookieIds(conf); + List possibleBookieIds = possibleBookieIds(conf); final Versioned rmCookie = readAndVerifyCookieFromRegistrationManager( masterCookie, rm, possibleBookieIds, allowExpansion); @@ -532,6 +533,10 @@ private static void verifyDirsForStorageExpansion( } } + public static BookieId getBookieId(ServerConfiguration conf) throws UnknownHostException { + return getBookieAddress(conf).toBookieId(); + } + /** * Return the configured address of the bookie. */ @@ -1485,7 +1490,7 @@ static class CounterCallback implements WriteCallback { int count; @Override - public synchronized void writeComplete(int rc, long l, long e, BookieSocketAddress addr, Object ctx) { + public synchronized void writeComplete(int rc, long l, long e, BookieId addr, Object ctx) { count--; if (count == 0) { notifyAll(); 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 ad2f23407b4..d17d74d4f79 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 @@ -40,7 +40,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicatedCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand; import org.apache.bookkeeper.tools.cli.commands.autorecovery.ToggleCommand; @@ -600,7 +600,7 @@ int runCmd(CommandLine cmdLine) throws Exception { boolean printMsg = cmdLine.hasOption("m"); boolean forceRecovery = cmdLine.hasOption("r"); - final BookieSocketAddress bookie; + final BookieId bookie; String bookieAddress; if (cmdLine.hasOption("b")) { // A particular bookie was specified diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java index 54647457745..096512478e4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStateManager.java @@ -42,6 +42,7 @@ import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -79,7 +80,7 @@ public class BookieStateManager implements StateManager { private final AtomicBoolean forceReadOnly = new AtomicBoolean(false); private volatile boolean availableForHighPriorityWrites = true; - private final String bookieId; + private final BookieId bookieId; private ShutdownHandler shutdownHandler; private final Supplier rm; // Expose Stats @@ -101,7 +102,7 @@ public BookieStateManager(ServerConfiguration conf, ledgerDirsManager.getAllLedgerDirs(), () -> { try { - return Bookie.getBookieAddress(conf).toString(); + return Bookie.getBookieId(conf); } catch (UnknownHostException e) { throw new UncheckedIOException("Failed to resolve bookie id", e); } @@ -112,7 +113,7 @@ public BookieStateManager(ServerConfiguration conf, StatsLogger statsLogger, Supplier rm, List statusDirs, - Supplier bookieIdSupplier, + Supplier bookieIdSupplier, Supplier bookieServiceInfoProvider) throws IOException { this.conf = conf; this.rm = rm; 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 9a6eadfb77e..b80a7895a90 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 @@ -42,7 +42,7 @@ 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.net.BookieId; import org.apache.bookkeeper.proto.DataFormats.CookieFormat; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.versioning.LongVersion; @@ -234,14 +234,14 @@ public void writeToFile (File versionFile) throws IOException { */ public void writeToRegistrationManager(RegistrationManager rm, ServerConfiguration conf, Version version) throws BookieException { - BookieSocketAddress address = null; + BookieId address = null; try { - address = Bookie.getBookieAddress(conf); + address = Bookie.getBookieId(conf); } catch (UnknownHostException e) { throw new UnknownBookieIdException(e); } byte[] data = toString().getBytes(UTF_8); - rm.writeCookie(address.toString(), new Versioned<>(data, version)); + rm.writeCookie(address, new Versioned<>(data, version)); } /** @@ -255,9 +255,9 @@ public void writeToRegistrationManager(RegistrationManager rm, ServerConfigurati public void deleteFromRegistrationManager(RegistrationManager rm, ServerConfiguration conf, Version version) throws BookieException { - BookieSocketAddress address = null; + BookieId address = null; try { - address = Bookie.getBookieAddress(conf); + address = Bookie.getBookieId(conf); } catch (UnknownHostException e) { throw new UnknownBookieIdException(e); } @@ -273,13 +273,13 @@ public void deleteFromRegistrationManager(RegistrationManager rm, * @throws BookieException when fail to delete cookie. */ public void deleteFromRegistrationManager(RegistrationManager rm, - BookieSocketAddress address, + BookieId address, Version version) throws BookieException { if (!(version instanceof LongVersion)) { throw new IllegalArgumentException("Invalid version type, expected ZkVersion type"); } - rm.removeCookie(address.toString(), version); + rm.removeCookie(address, version); } /** @@ -293,7 +293,7 @@ static Builder generateCookie(ServerConfiguration conf) throws UnknownHostException { Builder builder = Cookie.newBuilder(); builder.setLayoutVersion(CURRENT_COOKIE_LAYOUT_VERSION); - builder.setBookieHost(Bookie.getBookieAddress(conf).toString()); + builder.setBookieHost(Bookie.getBookieId(conf).toString()); builder.setJournalDirs(Joiner.on(',').join(conf.getJournalDirNames())); builder.setLedgerDirs(encodeDirPaths(conf.getLedgerDirNames())); return builder; @@ -310,7 +310,7 @@ static Builder generateCookie(ServerConfiguration conf) public static Versioned readFromRegistrationManager(RegistrationManager rm, ServerConfiguration conf) throws BookieException { try { - return readFromRegistrationManager(rm, Bookie.getBookieAddress(conf)); + return readFromRegistrationManager(rm, Bookie.getBookieId(conf)); } catch (UnknownHostException e) { throw new UnknownBookieIdException(e); } @@ -325,8 +325,8 @@ public static Versioned readFromRegistrationManager(RegistrationManager * @throws BookieException when fail to read cookie */ public static Versioned readFromRegistrationManager(RegistrationManager rm, - BookieSocketAddress address) throws BookieException { - Versioned cookieData = rm.readCookie(address.toString()); + BookieId address) throws BookieException { + Versioned cookieData = rm.readCookie(address); try { try (BufferedReader reader = new BufferedReader( new StringReader(new String(cookieData.getValue(), UTF_8)))) { 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 a0d2edb0af9..d44e576fbcb 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 @@ -33,8 +33,9 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.StatsLogger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,19 +49,20 @@ public class LocalBookieEnsemblePlacementPolicy implements EnsemblePlacementPoli static final Logger LOG = LoggerFactory.getLogger(LocalBookieEnsemblePlacementPolicy.class); - private BookieSocketAddress bookieAddress; + private BookieId bookieAddress; @Override public EnsemblePlacementPolicy initialize(ClientConfiguration conf, Optional optionalDnsResolver, HashedWheelTimer hashedWheelTimer, - FeatureProvider featureProvider, StatsLogger statsLogger) { + FeatureProvider featureProvider, + StatsLogger statsLogger, BookieAddressResolver bookieAddressResolver) { // Configuration will have already the bookie configuration inserted ServerConfiguration serverConf = new ServerConfiguration(); serverConf.addConfiguration(conf); try { - bookieAddress = Bookie.getBookieAddress(serverConf); + bookieAddress = Bookie.getBookieId(serverConf); } catch (UnknownHostException e) { LOG.warn("Unable to get bookie address", e); throw new RuntimeException(e); @@ -74,27 +76,27 @@ public void uninitalize() { } @Override - public Set onClusterChanged(Set writableBookies, - Set readOnlyBookies) { + public Set onClusterChanged(Set writableBookies, + Set readOnlyBookies) { return Collections.emptySet(); } @Override - public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - java.util.Map customMetadata, List currentEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies) + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + java.util.Map customMetadata, List currentEnsemble, + BookieId bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { throw new BKNotEnoughBookiesException(); } @Override - public void registerSlowBookie(BookieSocketAddress bookieSocketAddress, long entryId) { + public void registerSlowBookie(BookieId bookieSocketAddress, long entryId) { return; } @Override public DistributionSchedule.WriteSet reorderReadSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { return null; @@ -102,15 +104,15 @@ public DistributionSchedule.WriteSet reorderReadSequence( @Override public DistributionSchedule.WriteSet reorderReadLACSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { return null; } @Override - public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, java.util.Map customMetadata, Set excludeBookies) + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, java.util.Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { if (ensembleSize > 1) { throw new IllegalArgumentException("Local ensemble policy can only return 1 bookie"); @@ -120,12 +122,12 @@ public PlacementResult> newEnsemble(int ensembleSize, } @Override - public void updateBookieInfo(Map bookieToFreeSpaceMap) { + public void updateBookieInfo(Map bookieToFreeSpaceMap) { return; } @Override - public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, int ackQuorumSize) { return PlacementPolicyAdherence.MEETS_STRICT; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java index 21f412998a0..cff02509071 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java @@ -44,7 +44,7 @@ import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.Versioned; @@ -79,7 +79,7 @@ public class ScanAndCompareGarbageCollector implements GarbageCollector { private final LedgerManager ledgerManager; private final CompactableLedgerStorage ledgerStorage; private final ServerConfiguration conf; - private final BookieSocketAddress selfBookieAddress; + private final BookieId selfBookieAddress; private ZooKeeper zk = null; private boolean enableGcOverReplicatedLedger; private final long gcOverReplicatedLedgerIntervalMillis; @@ -94,7 +94,7 @@ public ScanAndCompareGarbageCollector(LedgerManager ledgerManager, CompactableLe this.ledgerManager = ledgerManager; this.ledgerStorage = ledgerStorage; this.conf = conf; - this.selfBookieAddress = Bookie.getBookieAddress(conf); + this.selfBookieAddress = Bookie.getBookieId(conf); this.gcOverReplicatedLedgerIntervalMillis = conf.getGcOverreplicatedLedgerWaitTimeMillis(); this.lastOverReplicatedLedgerGcTimeMillis = System.currentTimeMillis(); if (gcOverReplicatedLedgerIntervalMillis > 0) { @@ -256,9 +256,9 @@ private Set removeOverReplicatedledgers(Set bkActiveledgers, final G if (!metadata.getValue().isClosed()) { return; } - SortedMap> ensembles = + SortedMap> ensembles = metadata.getValue().getAllEnsembles(); - for (List ensemble : ensembles.values()) { + for (List ensemble : ensembles.values()) { // check if this bookie is supposed to have this ledger if (ensemble.contains(selfBookieAddress)) { return; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java index 438dd950455..393dde9aaf4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java @@ -244,6 +244,9 @@ public static class BKNotEnoughBookiesException extends BKException { public BKNotEnoughBookiesException() { super(BKException.Code.NotEnoughBookiesException); } + public BKNotEnoughBookiesException(Throwable cause) { + super(BKException.Code.NotEnoughBookiesException, cause); + } } /** 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 0638a877902..b0c038e5862 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 @@ -75,8 +75,9 @@ import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.exceptions.MetadataException; import org.apache.bookkeeper.meta.zk.ZKMetadataClientDriver; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.proto.DataFormats; @@ -476,9 +477,6 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo .build(); } - // initialize bookie client - this.bookieClient = new BookieClientImpl(conf, this.eventLoopGroup, this.allocator, this.mainWorkerPool, - scheduler, rootStatsLogger); if (null == requestTimer) { this.requestTimer = new HashedWheelTimer( @@ -491,14 +489,21 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo this.ownTimer = false; } + BookieAddressResolver bookieAddressResolver = + new DefaultBookieAddressResolver(metadataDriver.getRegistrationClient()); + // initialize the ensemble placement this.placementPolicy = initializeEnsemblePlacementPolicy(conf, - dnsResolver, this.requestTimer, this.featureProvider, this.statsLogger); - + dnsResolver, this.requestTimer, this.featureProvider, this.statsLogger, bookieAddressResolver); this.bookieWatcher = new BookieWatcherImpl( - conf, this.placementPolicy, metadataDriver.getRegistrationClient(), + conf, this.placementPolicy, metadataDriver.getRegistrationClient(), bookieAddressResolver, this.statsLogger.scope(WATCHER_SCOPE)); + + // initialize bookie client + this.bookieClient = new BookieClientImpl(conf, this.eventLoopGroup, this.allocator, this.mainWorkerPool, + scheduler, rootStatsLogger, this.bookieWatcher.getBookieAddressResolver()); + if (conf.getDiskWeightBasedPlacementEnabled()) { LOG.info("Weighted ledger placement enabled"); ThreadFactoryBuilder tFBuilder = new ThreadFactoryBuilder() @@ -559,12 +564,13 @@ private EnsemblePlacementPolicy initializeEnsemblePlacementPolicy(ClientConfigur DNSToSwitchMapping dnsResolver, HashedWheelTimer timer, FeatureProvider featureProvider, - StatsLogger statsLogger) + StatsLogger statsLogger, + BookieAddressResolver bookieAddressResolver) throws IOException { try { Class policyCls = conf.getEnsemblePlacementPolicy(); return ReflectionUtils.newInstance(policyCls).initialize(conf, java.util.Optional.ofNullable(dnsResolver), - timer, featureProvider, statsLogger); + timer, featureProvider, statsLogger, bookieAddressResolver); } catch (ConfigurationException e) { throw new IOException("Failed to initialize ensemble placement policy : ", e); } @@ -600,8 +606,8 @@ public void safeRun() { } void checkForFaultyBookies() { - List faultyBookies = bookieClient.getFaultyBookies(); - for (BookieSocketAddress faultyBookie : faultyBookies) { + List faultyBookies = bookieClient.getFaultyBookies(); + for (BookieId faultyBookie : faultyBookies) { if (Math.random() <= bookieQuarantineRatio) { bookieWatcher.quarantineBookie(faultyBookie); statsLogger.getCounter(BookKeeperServerStats.BOOKIE_QUARANTINE).inc(); @@ -644,6 +650,10 @@ BookieWatcher getBookieWatcher() { return bookieWatcher; } + public BookieAddressResolver getBookieAddressResolver() { + return bookieWatcher.getBookieAddressResolver(); + } + public OrderedExecutor getMainWorkerPool() { return mainWorkerPool; } @@ -754,7 +764,7 @@ BookieClient getBookieClient() { * @throws BKException * @throws InterruptedException */ - public Map getBookieInfo() throws BKException, InterruptedException { + public Map getBookieInfo() throws BKException, InterruptedException { return bookieInfoReader.getBookieInfo(); } 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 c9cd86fd81c..7dbf4df8cf9 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 @@ -42,7 +42,6 @@ import java.util.Map.Entry; import java.util.NoSuchElementException; import java.util.Optional; -import java.util.Random; import java.util.Set; import java.util.SortedMap; import java.util.concurrent.CompletableFuture; @@ -74,7 +73,8 @@ import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.MultiCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; @@ -108,12 +108,6 @@ public class BookKeeperAdmin implements AutoCloseable { // LedgerFragmentReplicator instance private LedgerFragmentReplicator lfr; - /* - * Random number generator used to choose an available bookie server to - * replicate data from a dead bookie. - */ - private Random rand = new Random(); - private LedgerManagerFactory mFactory; /* @@ -214,7 +208,7 @@ public void close() throws InterruptedException, BKException { * * @return a collection of bookie addresses */ - public Collection getAvailableBookies() + public Collection getAvailableBookies() throws BKException { return bkc.bookieWatcher.getBookies(); } @@ -224,16 +218,20 @@ public Collection getAvailableBookies() * * @return a collection of bookie addresses */ - public Collection getAllBookies() + public Collection getAllBookies() throws BKException { return bkc.bookieWatcher.getAllBookies(); } + public BookieAddressResolver getBookieAddressResolver() { + return bkc.bookieWatcher.getBookieAddressResolver(); + } + @SneakyThrows public BookieServiceInfo getBookieServiceInfo(String bookiedId) throws BKException { return FutureUtils.result(bkc.getMetadataClientDriver() - .getRegistrationClient().getBookieServiceInfo(bookiedId)).getValue(); + .getRegistrationClient().getBookieServiceInfo(BookieId.parse(bookiedId))).getValue(); } /** @@ -242,7 +240,7 @@ public BookieServiceInfo getBookieServiceInfo(String bookiedId) * @return a collection of bookie addresses * @throws BKException if there are issues trying to read the list. */ - public Collection getReadOnlyBookies() throws BKException { + public Collection getReadOnlyBookies() throws BKException { return bkc.bookieWatcher.getReadOnlyBookies(); } @@ -479,7 +477,7 @@ public SyncObject() { } } - public SortedMap getLedgersContainBookies(Set bookies) + public SortedMap getLedgersContainBookies(Set bookies) throws InterruptedException, BKException { final SyncObject sync = new SyncObject(); final AtomicReference> resultHolder = @@ -507,7 +505,7 @@ public void operationComplete(int rc, SortedMap result) { return resultHolder.get(); } - public void asyncGetLedgersContainBookies(final Set bookies, + public void asyncGetLedgersContainBookies(final Set bookies, final GenericCallback> callback) { final SortedMap ledgers = new ConcurrentSkipListMap(); bkc.getLedgerManager().asyncProcessLedgers(new Processor() { @@ -524,9 +522,9 @@ public void process(final Long lid, final AsyncCallback.VoidCallback cb) { cb.processResult(BKException.getExceptionCode(exception), null, null); return; } - Set bookiesInLedger = + Set bookiesInLedger = LedgerMetadataUtils.getBookiesInThisLedger(metadata.getValue()); - Sets.SetView intersection = + Sets.SetView intersection = Sets.intersection(bookiesInLedger, bookies); if (!intersection.isEmpty()) { ledgers.put(lid, metadata.getValue()); @@ -557,18 +555,18 @@ public void processResult(int rc, String path, Object ctx) { * Source bookie that had a failure. We want to replicate the * ledger fragments that were stored there. */ - public void recoverBookieData(final BookieSocketAddress bookieSrc) + public void recoverBookieData(final BookieId bookieSrc) throws InterruptedException, BKException { - Set bookiesSrc = Sets.newHashSet(bookieSrc); + Set bookiesSrc = Sets.newHashSet(bookieSrc); recoverBookieData(bookiesSrc); } - public void recoverBookieData(final Set bookiesSrc) + public void recoverBookieData(final Set bookiesSrc) throws InterruptedException, BKException { recoverBookieData(bookiesSrc, false, false); } - public void recoverBookieData(final Set bookiesSrc, boolean dryrun, boolean skipOpenLedgers) + 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. @@ -597,7 +595,7 @@ public void recoverComplete(int rc, Object ctx) { } public void recoverBookieData(final long lid, - final Set bookiesSrc, + final Set bookiesSrc, boolean dryrun, boolean skipOpenLedgers) throws InterruptedException, BKException { @@ -644,18 +642,18 @@ public void recoverBookieData(final long lid, * @param context * Context for the RecoverCallback to call. */ - public void asyncRecoverBookieData(final BookieSocketAddress bookieSrc, + public void asyncRecoverBookieData(final BookieId bookieSrc, final RecoverCallback cb, final Object context) { - Set bookiesSrc = Sets.newHashSet(bookieSrc); + Set bookiesSrc = Sets.newHashSet(bookieSrc); asyncRecoverBookieData(bookiesSrc, cb, context); } - public void asyncRecoverBookieData(final Set bookieSrc, + public void asyncRecoverBookieData(final Set bookieSrc, final RecoverCallback cb, final Object context) { asyncRecoverBookieData(bookieSrc, false, false, cb, context); } - public void asyncRecoverBookieData(final Set bookieSrc, boolean dryrun, + public void asyncRecoverBookieData(final Set bookieSrc, boolean dryrun, final boolean skipOpenLedgers, final RecoverCallback cb, final Object context) { getActiveLedgers(bookieSrc, dryrun, skipOpenLedgers, cb, context); } @@ -677,7 +675,7 @@ public void asyncRecoverBookieData(final Set bookieSrc, boo * @param context * Context for the RecoverCallback to call. */ - public void asyncRecoverBookieData(long lid, final Set bookieSrc, boolean dryrun, + public void asyncRecoverBookieData(long lid, final Set bookieSrc, boolean dryrun, boolean skipOpenLedgers, final RecoverCallback callback, final Object context) { AsyncCallback.VoidCallback callbackWrapper = (rc, path, ctx) -> callback.recoverComplete(bkc.getReturnRc(rc), context); @@ -703,7 +701,7 @@ public void asyncRecoverBookieData(long lid, final Set book * @param context * Context for the RecoverCallback to call. */ - private void getActiveLedgers(final Set bookiesSrc, final boolean dryrun, + 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 @@ -748,7 +746,7 @@ ledgerProcessor, new RecoverCallbackWrapper(cb), * IterationCallback to invoke once we've recovered the current * ledger. */ - private void recoverLedger(final Set bookiesSrc, final long lId, final boolean dryrun, + 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); @@ -841,7 +839,7 @@ public void processResult(int rc, String path, Object ctx) { */ Map ledgerFragmentsRange = new HashMap(); Long curEntryId = null; - for (Map.Entry> entry : + for (Map.Entry> entry : lh.getLedgerMetadata().getAllEnsembles().entrySet()) { if (curEntryId != null) { ledgerFragmentsRange.put(curEntryId, entry.getKey() - 1); @@ -887,9 +885,9 @@ public void processResult(int rc, String path, Object ctx) { */ for (final Long startEntryId : ledgerFragmentsToRecover) { Long endEntryId = ledgerFragmentsRange.get(startEntryId); - List ensemble = lh.getLedgerMetadata().getAllEnsembles().get(startEntryId); + List ensemble = lh.getLedgerMetadata().getAllEnsembles().get(startEntryId); // Get bookies to replace - Map targetBookieAddresses; + Map targetBookieAddresses; try { targetBookieAddresses = getReplacementBookies(lh, ensemble, bookiesSrc); } catch (BKException.BKNotEnoughBookiesException e) { @@ -903,7 +901,7 @@ public void processResult(int rc, String path, Object ctx) { } if (dryrun) { - ArrayList newEnsemble = + ArrayList newEnsemble = replaceBookiesInEnsemble(ensemble, targetBookieAddresses); VERBOSE.info(" Fragment [{} - {}] : ", startEntryId, endEntryId); VERBOSE.info(" old ensemble : {}", formatEnsemble(ensemble, bookiesSrc, '*')); @@ -935,7 +933,7 @@ public void processResult(int rc, String path, Object ctx) { }, null); } - static String formatEnsemble(List ensemble, Set bookiesSrc, + static String formatEnsemble(List ensemble, Set bookiesSrc, char marker) { StringBuilder sb = new StringBuilder(); sb.append("["); @@ -973,19 +971,19 @@ static String formatEnsemble(List ensemble, Set newBookies, + final Set newBookies, final BiConsumer onReadEntryFailureCallback) throws InterruptedException { lfr.replicate(lh, ledgerFragment, ledgerFragmentMcb, newBookies, onReadEntryFailureCallback); } - private Map getReplacementBookies( + private Map getReplacementBookies( LedgerHandle lh, - List ensemble, - Set bookiesToRereplicate) + List ensemble, + Set bookiesToRereplicate) throws BKException.BKNotEnoughBookiesException { Set bookieIndexesToRereplicate = Sets.newHashSet(); for (int bookieIndex = 0; bookieIndex < ensemble.size(); bookieIndex++) { - BookieSocketAddress bookieInEnsemble = ensemble.get(bookieIndex); + BookieId bookieInEnsemble = ensemble.get(bookieIndex); if (bookiesToRereplicate.contains(bookieInEnsemble)) { bookieIndexesToRereplicate.add(bookieIndex); } @@ -994,31 +992,31 @@ private Map getReplacementBookies( lh, ensemble, bookieIndexesToRereplicate, Optional.of(bookiesToRereplicate)); } - private Map getReplacementBookiesByIndexes( + private Map getReplacementBookiesByIndexes( LedgerHandle lh, - List ensemble, + List ensemble, Set bookieIndexesToRereplicate, - Optional> excludedBookies) + Optional> excludedBookies) throws BKException.BKNotEnoughBookiesException { // target bookies to replicate - Map targetBookieAddresses = + Map targetBookieAddresses = Maps.newHashMapWithExpectedSize(bookieIndexesToRereplicate.size()); // bookies to exclude for ensemble allocation - Set bookiesToExclude = Sets.newHashSet(); + 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); + BookieId bookie = ensemble.get(bookieIndex); bookiesToExclude.add(bookie); } // allocate bookies for (Integer bookieIndex : bookieIndexesToRereplicate) { - BookieSocketAddress oldBookie = ensemble.get(bookieIndex); - EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = + BookieId oldBookie = ensemble.get(bookieIndex); + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = bkc.getPlacementPolicy().replaceBookie( lh.getLedgerMetadata().getEnsembleSize(), lh.getLedgerMetadata().getWriteQuorumSize(), @@ -1027,7 +1025,7 @@ private Map getReplacementBookiesByIndexes( ensemble, oldBookie, bookiesToExclude); - BookieSocketAddress newBookie = replaceBookieResponse.getResult(); + BookieId newBookie = replaceBookieResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) { if (LOG.isDebugEnabled()) { @@ -1044,11 +1042,11 @@ private Map getReplacementBookiesByIndexes( return targetBookieAddresses; } - private ArrayList replaceBookiesInEnsemble( - List ensemble, - Map replacedBookies) { - ArrayList newEnsemble = Lists.newArrayList(ensemble); - for (Map.Entry entry : replacedBookies.entrySet()) { + 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; @@ -1066,8 +1064,8 @@ public void replicateLedgerFragment(LedgerHandle lh, final LedgerFragment ledgerFragment, final BiConsumer onReadEntryFailureCallback) throws InterruptedException, BKException { - Optional> excludedBookies = Optional.empty(); - Map targetBookieAddresses = + Optional> excludedBookies = Optional.empty(); + Map targetBookieAddresses = getReplacementBookiesByIndexes(lh, ledgerFragment.getEnsemble(), ledgerFragment.getBookiesIndexes(), excludedBookies); replicateLedgerFragment(lh, ledgerFragment, targetBookieAddresses, onReadEntryFailureCallback); @@ -1075,7 +1073,7 @@ public void replicateLedgerFragment(LedgerHandle lh, private void replicateLedgerFragment(LedgerHandle lh, final LedgerFragment ledgerFragment, - final Map targetBookieAddresses, + final Map targetBookieAddresses, final BiConsumer onReadEntryFailureCallback) throws InterruptedException, BKException { CompletableFuture result = new CompletableFuture<>(); @@ -1087,7 +1085,7 @@ private void replicateLedgerFragment(LedgerHandle lh, ledgerFragment.getFirstEntryId(), getReplacementBookiesMap(ledgerFragment, targetBookieAddresses)); - Set targetBookieSet = Sets.newHashSet(); + Set targetBookieSet = Sets.newHashSet(); targetBookieSet.addAll(targetBookieAddresses.values()); asyncRecoverLedgerFragment(lh, ledgerFragment, cb, targetBookieSet, onReadEntryFailureCallback); @@ -1098,45 +1096,45 @@ private void replicateLedgerFragment(LedgerHandle lh, } } - private static Map getReplacementBookiesMap( - List ensemble, - Map targetBookieAddresses) { - Map bookiesMap = - new HashMap(); - for (Map.Entry entry : targetBookieAddresses.entrySet()) { - BookieSocketAddress oldBookie = ensemble.get(entry.getKey()); - BookieSocketAddress newBookie = entry.getValue(); + private static Map getReplacementBookiesMap( + List ensemble, + Map targetBookieAddresses) { + Map bookiesMap = + new HashMap(); + for (Map.Entry entry : targetBookieAddresses.entrySet()) { + BookieId oldBookie = ensemble.get(entry.getKey()); + BookieId newBookie = entry.getValue(); bookiesMap.put(oldBookie, newBookie); } return bookiesMap; } - private static Map getReplacementBookiesMap( + private static Map getReplacementBookiesMap( LedgerFragment ledgerFragment, - Map targetBookieAddresses) { - Map bookiesMap = - new HashMap(); + Map targetBookieAddresses) { + Map bookiesMap = + new HashMap(); for (Integer bookieIndex : ledgerFragment.getBookiesIndexes()) { - BookieSocketAddress oldBookie = ledgerFragment.getAddress(bookieIndex); - BookieSocketAddress newBookie = targetBookieAddresses.get(bookieIndex); + BookieId oldBookie = ledgerFragment.getAddress(bookieIndex); + BookieId newBookie = targetBookieAddresses.get(bookieIndex); bookiesMap.put(oldBookie, newBookie); } return bookiesMap; } private static boolean containBookiesInLastEnsemble(LedgerMetadata lm, - Set bookies) { + Set bookies) { if (lm.getAllEnsembles().size() <= 0) { return false; } Long lastKey = lm.getAllEnsembles().lastKey(); - List lastEnsemble = lm.getAllEnsembles().get(lastKey); + List lastEnsemble = lm.getAllEnsembles().get(lastKey); return containBookies(lastEnsemble, bookies); } - private static boolean containBookies(List ensemble, - Set bookies) { - for (BookieSocketAddress bookie : ensemble) { + private static boolean containBookies(List ensemble, + Set bookies) { + for (BookieId bookie : ensemble) { if (bookies.contains(bookie)) { return true; } @@ -1300,7 +1298,7 @@ public static boolean initBookie(ServerConfiguration conf) throws Exception { * make sure that there is no bookie registered with the same * bookieid and the cookie for the same bookieid is not existing. */ - String bookieId = Bookie.getBookieAddress(conf).toString(); + BookieId bookieId = Bookie.getBookieId(conf); if (rm.isBookieRegistered(bookieId)) { LOG.error("Bookie with bookieId: {} is still registered, " + "If this node is running bookie process, try stopping it first.", bookieId); @@ -1457,7 +1455,7 @@ public void triggerAudit() throw new UnavailableException("Autorecovery is disabled. So giving up!"); } - BookieSocketAddress auditorId = + BookieId auditorId = AuditorElector.getCurrentAuditor(new ServerConfiguration(bkc.getConf()), bkc.getZkHandle()); if (auditorId == null) { LOG.error("No auditor elected, though Autorecovery is enabled. So giving up."); @@ -1489,7 +1487,7 @@ public void triggerAudit() * @throws TimeoutException * @throws BKException */ - public void decommissionBookie(BookieSocketAddress bookieAddress) + public void decommissionBookie(BookieId bookieAddress) throws CompatibilityException, UnavailableException, KeeperException, InterruptedException, IOException, BKAuditException, TimeoutException, BKException { if (getAvailableBookies().contains(bookieAddress) || getReadOnlyBookies().contains(bookieAddress)) { @@ -1539,7 +1537,7 @@ public void decommissionBookie(BookieSocketAddress bookieAddress) } } - private void waitForLedgersToBeReplicated(Collection ledgers, BookieSocketAddress thisBookieAddress, + private void waitForLedgersToBeReplicated(Collection ledgers, BookieId thisBookieAddress, LedgerManager ledgerManager) throws InterruptedException, TimeoutException { int maxSleepTimeInBetweenChecks = 10 * 60 * 1000; // 10 minutes int sleepTimePerLedger = 10 * 1000; // 10 secs @@ -1555,7 +1553,7 @@ private void waitForLedgersToBeReplicated(Collection ledgers, BookieSocket } } - public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieSocketAddress bookieAddress, + public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieId bookieAddress, LedgerManager ledgerManager) { try { LedgerMetadata ledgerMetadata = ledgerManager.readLedgerMetadata(ledgerId).get().getValue(); @@ -1576,11 +1574,11 @@ public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieS } } - public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieSocketAddress bookieAddress, + public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieId bookieAddress, LedgerMetadata ledgerMetadata) { - Collection> ensemblesOfSegments = ledgerMetadata.getAllEnsembles().values(); - Iterator> ensemblesOfSegmentsIterator = ensemblesOfSegments.iterator(); - List ensemble; + Collection> ensemblesOfSegments = ledgerMetadata.getAllEnsembles().values(); + Iterator> ensemblesOfSegmentsIterator = ensemblesOfSegments.iterator(); + List ensemble; int segmentNo = 0; while (ensemblesOfSegmentsIterator.hasNext()) { ensemble = ensemblesOfSegmentsIterator.next(); @@ -1595,14 +1593,14 @@ public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieS } private static boolean areEntriesOfSegmentStoredInTheBookie(LedgerMetadata ledgerMetadata, - BookieSocketAddress bookieAddress, int segmentNo) { + BookieId bookieAddress, int segmentNo) { boolean isLedgerClosed = ledgerMetadata.isClosed(); int ensembleSize = ledgerMetadata.getEnsembleSize(); int writeQuorumSize = ledgerMetadata.getWriteQuorumSize(); - List>> segments = + List>> segments = new LinkedList<>(ledgerMetadata.getAllEnsembles().entrySet()); - List currentSegmentEnsemble = segments.get(segmentNo).getValue(); + List currentSegmentEnsemble = segments.get(segmentNo).getValue(); boolean lastSegment = (segmentNo == (segments.size() - 1)); /* @@ -1691,7 +1689,7 @@ private static boolean areEntriesOfSegmentStoredInTheBookie(LedgerMetadata ledge * @return true if the ledger is adhering to * EnsemblePlacementPolicy */ - public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleBookiesList, + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleBookiesList, int writeQuorumSize, int ackQuorumSize) { return bkc.getPlacementPolicy().isEnsembleAdheringToPlacementPolicy(ensembleBookiesList, writeQuorumSize, ackQuorumSize); @@ -1702,12 +1700,12 @@ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List asyncGetListOfEntriesOfLedger(BookieSocketAddress address, + public CompletableFuture asyncGetListOfEntriesOfLedger(BookieId address, long ledgerId) { return bkc.getBookieClient().getListOfEntriesOfLedger(address, ledgerId); } 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 c0d412c3fab..254260ae701 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 @@ -31,7 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; import org.apache.bookkeeper.proto.BookkeeperProtocol; @@ -95,16 +95,16 @@ private static class BookieInfoMap { * Contains the most recently obtained information on the contained bookies. * When an error happens querying a bookie, the entry is removed. */ - private final Map infoMap = new HashMap<>(); + private final Map infoMap = new HashMap<>(); /** * Contains the most recently reported set of bookies from BookieWatcher * A partial query consists of every member of mostRecentlyReportedBookies * minus the entries in bookieInfoMap. */ - private Collection mostRecentlyReportedBookies = new ArrayList<>(); + private Collection mostRecentlyReportedBookies = new ArrayList<>(); - public void updateBookies(Collection updatedBookieSet) { + public void updateBookies(Collection updatedBookieSet) { if (LOG.isDebugEnabled()) { LOG.debug( "updateBookies: current: {}, new: {}", @@ -115,11 +115,11 @@ public void updateBookies(Collection updatedBookieSet) { } @SuppressWarnings("unchecked") - public Collection getPartialScanTargets() { + public Collection getPartialScanTargets() { return CollectionUtils.subtract(mostRecentlyReportedBookies, infoMap.keySet()); } - public Collection getFullScanTargets() { + public Collection getFullScanTargets() { return mostRecentlyReportedBookies; } @@ -129,7 +129,7 @@ public Collection getFullScanTargets() { * @param bookie bookie for which to get info * @return Info for bookie, null otherwise */ - public BookieInfo getInfo(BookieSocketAddress bookie) { + public BookieInfo getInfo(BookieId bookie) { return infoMap.get(bookie); } @@ -138,7 +138,7 @@ public BookieInfo getInfo(BookieSocketAddress bookie) { * * @param bookie bookie on which we observed an error */ - public void clearInfo(BookieSocketAddress bookie) { + public void clearInfo(BookieId bookie) { infoMap.remove(bookie); } @@ -148,14 +148,14 @@ public void clearInfo(BookieSocketAddress bookie) { * @param bookie bookie for which we obtained new info * @param info the new info */ - public void gotInfo(BookieSocketAddress bookie, BookieInfo info) { + public void gotInfo(BookieId bookie, BookieInfo info) { infoMap.put(bookie, info); } /** * Get bookie info map. */ - public Map getBookieMap() { + public Map getBookieMap() { return infoMap; } } @@ -246,7 +246,7 @@ public void run() { LOG.debug("Running periodic BookieInfo scan"); } try { - Collection updatedBookies = bk.bookieWatcher.getBookies(); + Collection updatedBookies = bk.bookieWatcher.getBookies(); bookieInfoMap.updateBookies(updatedBookies); } catch (BKException e) { LOG.info("Got exception while querying bookies from watcher, rerunning after {}s", @@ -270,7 +270,7 @@ private void submitTaskWithDelay(int delaySeconds) { scheduler.schedule(() -> getReadWriteBookieInfo(), delaySeconds, TimeUnit.SECONDS); } - synchronized void availableBookiesChanged(Set updatedBookiesList) { + synchronized void availableBookiesChanged(Set updatedBookiesList) { if (LOG.isInfoEnabled()) { LOG.info("Scheduling bookie info read due to changes in available bookies."); } @@ -286,7 +286,7 @@ synchronized void availableBookiesChanged(Set updatedBookie * @param bookie to lookup * @return None if absent, free disk space if present */ - synchronized Optional getFreeDiskSpace(BookieSocketAddress bookie) { + synchronized Optional getFreeDiskSpace(BookieId bookie) { BookieInfo bookieInfo = bookieInfoMap.getInfo(bookie); if (bookieInfo != null) { return Optional.of(bookieInfo.getFreeDiskSpace()); @@ -306,7 +306,7 @@ synchronized Optional getFreeDiskSpace(BookieSocketAddress bookie) { */ synchronized void getReadWriteBookieInfo() { State queuedType = instanceState.getAndClearQueuedType(); - Collection toScan; + Collection toScan; if (queuedType == State.FULL) { if (LOG.isDebugEnabled()) { LOG.debug("Doing full scan"); @@ -335,12 +335,12 @@ synchronized void getReadWriteBookieInfo() { if (LOG.isDebugEnabled()) { LOG.debug("Getting bookie info for: {}", toScan); } - for (BookieSocketAddress b : toScan) { + for (BookieId b : toScan) { bkc.getBookieInfo(b, requested, new GetBookieInfoCallback() { void processReadInfoComplete(int rc, BookieInfo bInfo, Object ctx) { synchronized (BookieInfoReader.this) { - BookieSocketAddress b = (BookieSocketAddress) ctx; + BookieId b = (BookieId) ctx; if (rc != BKException.Code.OK) { if (LOG.isErrorEnabled()) { LOG.error("Reading bookie info from bookie {} failed due to {}", @@ -396,26 +396,26 @@ void onExit() { } } - Map getBookieInfo() throws BKException, InterruptedException { + Map getBookieInfo() throws BKException, InterruptedException { BookieClient bkc = bk.getBookieClient(); final AtomicInteger totalSent = new AtomicInteger(); final AtomicInteger totalCompleted = new AtomicInteger(); - final ConcurrentMap map = - new ConcurrentHashMap(); + final ConcurrentMap map = + new ConcurrentHashMap(); final CountDownLatch latch = new CountDownLatch(1); long requested = BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE | BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE; - Collection bookies; + Collection bookies; bookies = bk.bookieWatcher.getBookies(); bookies.addAll(bk.bookieWatcher.getReadOnlyBookies()); totalSent.set(bookies.size()); - for (BookieSocketAddress b : bookies) { + for (BookieId b : bookies) { bkc.getBookieInfo(b, requested, new GetBookieInfoCallback() { @Override public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) { - BookieSocketAddress b = (BookieSocketAddress) ctx; + BookieId b = (BookieId) ctx; if (rc != BKException.Code.OK) { if (LOG.isErrorEnabled()) { LOG.error("Reading bookie info from bookie {} failed due to {}", 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 b88156943be..07b1695642f 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 @@ -22,12 +22,17 @@ import java.util.Set; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookieAddressResolver; -interface BookieWatcher { - Set getBookies() throws BKException; - Set getAllBookies() throws BKException; - Set getReadOnlyBookies() throws BKException; +/** + * Watch for Bookkeeper cluster status. + */ +public interface BookieWatcher { + Set getBookies() throws BKException; + Set getAllBookies() throws BKException; + Set getReadOnlyBookies() throws BKException; + BookieAddressResolver getBookieAddressResolver(); /** * Create an ensemble with given ensembleSize and writeQuorumSize. @@ -39,7 +44,7 @@ interface BookieWatcher { * @return list of bookies for new ensemble. * @throws BKNotEnoughBookiesException */ - List newEnsemble(int ensembleSize, int writeQuorumSize, + List newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata) throws BKNotEnoughBookiesException; @@ -52,10 +57,10 @@ List newEnsemble(int ensembleSize, int writeQuorumSize, * @return the bookie to replace. * @throws BKNotEnoughBookiesException */ - BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, - List existingBookies, int bookieIdx, - Set excludeBookies) + List existingBookies, int bookieIdx, + Set excludeBookies) throws BKNotEnoughBookiesException; @@ -63,5 +68,5 @@ BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ack * Quarantine bookie so it will not be preferred to be chosen for new ensembles. * @param bookie */ - void quarantineBookie(BookieSocketAddress bookie); + void quarantineBookie(BookieId bookie); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java index f18799c93be..d68ded722e0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java @@ -46,7 +46,8 @@ import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.discover.RegistrationClient; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -101,27 +102,31 @@ class BookieWatcherImpl implements BookieWatcher { private final Counter ensembleNotAdheringToPlacementPolicy; // Bookies that will not be preferred to be chosen in a new ensemble - final Cache quarantinedBookies; + final Cache quarantinedBookies; - private volatile Set writableBookies = Collections.emptySet(); - private volatile Set readOnlyBookies = Collections.emptySet(); + private volatile Set writableBookies = Collections.emptySet(); + private volatile Set readOnlyBookies = Collections.emptySet(); private CompletableFuture initialWritableBookiesFuture = null; private CompletableFuture initialReadonlyBookiesFuture = null; + private final BookieAddressResolver bookieAddressResolver; + public BookieWatcherImpl(ClientConfiguration conf, EnsemblePlacementPolicy placementPolicy, RegistrationClient registrationClient, + BookieAddressResolver bookieAddressResolver, StatsLogger statsLogger) { this.conf = conf; + this.bookieAddressResolver = bookieAddressResolver; this.placementPolicy = placementPolicy; this.registrationClient = registrationClient; this.quarantinedBookies = CacheBuilder.newBuilder() .expireAfterWrite(conf.getBookieQuarantineTimeSeconds(), TimeUnit.SECONDS) - .removalListener(new RemovalListener() { + .removalListener(new RemovalListener() { @Override - public void onRemoval(RemovalNotification bookie) { + public void onRemoval(RemovalNotification bookie) { log.info("Bookie {} is no longer quarantined", bookie.getKey()); } @@ -133,7 +138,7 @@ public void onRemoval(RemovalNotification bookie) } @Override - public Set getBookies() throws BKException { + public Set getBookies() throws BKException { try { return FutureUtils.result(registrationClient.getWritableBookies(), EXCEPTION_FUNC).getValue(); } catch (BKInterruptedException ie) { @@ -143,7 +148,7 @@ public Set getBookies() throws BKException { } @Override - public Set getAllBookies() throws BKException { + public Set getAllBookies() throws BKException { try { return FutureUtils.result(registrationClient.getAllBookies(), EXCEPTION_FUNC).getValue(); } catch (BKInterruptedException ie) { @@ -153,7 +158,12 @@ public Set getAllBookies() throws BKException { } @Override - public Set getReadOnlyBookies() + public BookieAddressResolver getBookieAddressResolver() { + return this.bookieAddressResolver; + } + + @Override + public Set getReadOnlyBookies() throws BKException { try { return FutureUtils.result(registrationClient.getReadOnlyBookies(), EXCEPTION_FUNC).getValue(); @@ -164,7 +174,7 @@ public Set getReadOnlyBookies() } // this callback is already not executed in zookeeper thread - private synchronized void processWritableBookiesChanged(Set newBookieAddrs) { + 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 this.writableBookies = newBookieAddrs; @@ -185,7 +195,7 @@ private synchronized void processWritableBookiesChanged(Set // } } - private synchronized void processReadOnlyBookiesChanged(Set readOnlyBookies) { + private synchronized void processReadOnlyBookiesChanged(Set readOnlyBookies) { this.readOnlyBookies = readOnlyBookies; placementPolicy.onClusterChanged(writableBookies, readOnlyBookies); } @@ -196,6 +206,7 @@ private synchronized void processReadOnlyBookiesChanged(Set * @throws BKException when failed to read bookies */ public void initialBlockingBookieRead() throws BKException { + CompletableFuture writable; CompletableFuture readonly; synchronized (this) { @@ -214,7 +225,6 @@ public void initialBlockingBookieRead() throws BKException { readonly = initialReadonlyBookiesFuture; } } - try { FutureUtils.result(writable, EXCEPTION_FUNC); } catch (BKInterruptedException ie) { @@ -232,17 +242,17 @@ public void initialBlockingBookieRead() throws BKException { } @Override - public List newEnsemble(int ensembleSize, int writeQuorumSize, + public List newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata) throws BKNotEnoughBookiesException { long startTime = MathUtils.nowInNano(); - EnsemblePlacementPolicy.PlacementResult> newEnsembleResponse; - List socketAddresses; + EnsemblePlacementPolicy.PlacementResult> newEnsembleResponse; + List socketAddresses; PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; try { - Set quarantinedBookiesSet = quarantinedBookies.asMap().keySet(); + Set quarantinedBookiesSet = quarantinedBookies.asMap().keySet(); newEnsembleResponse = placementPolicy.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, - customMetadata, new HashSet(quarantinedBookiesSet)); + customMetadata, new HashSet(quarantinedBookiesSet)); socketAddresses = newEnsembleResponse.getResult(); isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.isAdheringToPolicy(); if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) { @@ -272,21 +282,21 @@ public List newEnsemble(int ensembleSize, int writeQuorumSi } @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, - List existingBookies, int bookieIdx, - Set excludeBookies) + List existingBookies, int bookieIdx, + Set excludeBookies) throws BKNotEnoughBookiesException { long startTime = MathUtils.nowInNano(); - BookieSocketAddress addr = existingBookies.get(bookieIdx); - EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; - BookieSocketAddress socketAddress; + BookieId addr = existingBookies.get(bookieIdx); + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; + BookieId socketAddress; PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = PlacementPolicyAdherence.FAIL; try { // we exclude the quarantined bookies also first - Set excludedBookiesAndQuarantinedBookies = new HashSet( + Set excludedBookiesAndQuarantinedBookies = new HashSet( excludeBookies); - Set quarantinedBookiesSet = quarantinedBookies.asMap().keySet(); + Set quarantinedBookiesSet = quarantinedBookies.asMap().keySet(); excludedBookiesAndQuarantinedBookies.addAll(quarantinedBookiesSet); replaceBookieResponse = placementPolicy.replaceBookie( ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, @@ -326,11 +336,12 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, * @param bookie */ @Override - public void quarantineBookie(BookieSocketAddress bookie) { + public void quarantineBookie(BookieId bookie) { if (quarantinedBookies.getIfPresent(bookie) == null) { quarantinedBookies.put(bookie, Boolean.TRUE); log.warn("Bookie {} has been quarantined because of read/write errors.", bookie); } } + } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookiesHealthInfo.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookiesHealthInfo.java index b0404407587..32a6a915858 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookiesHealthInfo.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookiesHealthInfo.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.client; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; /** * This interface returns heuristics used to determine the health of a Bookkeeper server for read @@ -34,7 +34,7 @@ public interface BookiesHealthInfo { * @param bookieSocketAddress * @return failed entries on a bookie, -1 if there have been no failures */ - long getBookieFailureHistory(BookieSocketAddress bookieSocketAddress); + long getBookieFailureHistory(BookieId bookieSocketAddress); /** * Returns pending requests to a bookie. @@ -42,6 +42,6 @@ public interface BookiesHealthInfo { * @param bookieSocketAddress * @return number of pending requests */ - long getBookiePendingRequests(BookieSocketAddress bookieSocketAddress); + long getBookiePendingRequests(BookieId bookieSocketAddress); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultBookieAddressResolver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultBookieAddressResolver.java new file mode 100644 index 00000000000..dab93be0352 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultBookieAddressResolver.java @@ -0,0 +1,68 @@ +/** + * 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.client; + +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.discover.RegistrationClient; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookieAddressResolver; + +/** + * Resolve BookieIDs to Network addresses. + */ +@Slf4j +public class DefaultBookieAddressResolver implements BookieAddressResolver { + + private final RegistrationClient registrationClient; + + public DefaultBookieAddressResolver(RegistrationClient registrationClient) { + this.registrationClient = registrationClient; + } + + @Override + public BookieSocketAddress resolve(BookieId bookieId) { + try { + BookieServiceInfo info = FutureUtils.result(registrationClient.getBookieServiceInfo(bookieId)).getValue(); + BookieServiceInfo.Endpoint endpoint = info.getEndpoints() + .stream().filter(e -> e.getProtocol().equals("bookie-rpc")).findAny().orElse(null); + if (endpoint == null) { + throw new Exception("bookie " + bookieId + " does not publish a bookie-rpc endpoint"); + } + BookieSocketAddress res = new BookieSocketAddress(endpoint.getHost(), endpoint.getPort()); + log.info("Resolved {} as {}", bookieId, res); + return res; + } catch (BKException.BKBookieHandleNotAvailableException ex) { + if (BookieSocketAddress.isDummyBookieIdForHostname(bookieId)) { + log.info("Resolving dummy bookie Id {} using legacy bookie resolver", bookieId, ex); + return BookieSocketAddress.resolveDummyBookieId(bookieId); + } + log.info("Cannot resolve {}, bookie is unknown", bookieId, ex); + throw new BookieIdNotResolvedException(bookieId, ex); + } catch (Exception ex) { + if (ex instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + log.info("Cannot resolve {} ", bookieId, ex); + throw new BookieIdNotResolvedException(bookieId, ex); + } + } + +} 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 8569c71c5b9..811766ba88d 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 @@ -35,8 +35,9 @@ import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.collections4.CollectionUtils; import org.slf4j.Logger; @@ -49,32 +50,32 @@ */ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy { static final Logger LOG = LoggerFactory.getLogger(DefaultEnsemblePlacementPolicy.class); - static final Set EMPTY_SET = new HashSet(); + static final Set EMPTY_SET = new HashSet(); private boolean isWeighted; private int maxWeightMultiple; - private Set knownBookies = new HashSet(); - private Map bookieInfoMap; - private WeightedRandomSelection weightedSelection; + private Set knownBookies = new HashSet(); + private Map bookieInfoMap; + private WeightedRandomSelection weightedSelection; private final ReentrantReadWriteLock rwLock; DefaultEnsemblePlacementPolicy() { - bookieInfoMap = new HashMap(); + bookieInfoMap = new HashMap(); rwLock = new ReentrantReadWriteLock(); } @Override - public PlacementResult> newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, - Map customMetadata, Set excludeBookies) + public PlacementResult> newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, + Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { - ArrayList newBookies = new ArrayList(ensembleSize); + ArrayList newBookies = new ArrayList(ensembleSize); if (ensembleSize <= 0) { return PlacementResult.of(newBookies, PlacementPolicyAdherence.FAIL); } - List allBookies; + List allBookies; rwLock.readLock().lock(); try { - allBookies = new ArrayList(knownBookies); + allBookies = new ArrayList(knownBookies); } finally { rwLock.readLock().unlock(); } @@ -88,7 +89,7 @@ public PlacementResult> newEnsemble(int ensembleSize, throw new BKNotEnoughBookiesException(); } while (ensembleSize > 0) { - BookieSocketAddress b = weightedSelection.getNextRandom(); + BookieId b = weightedSelection.getNextRandom(); if (newBookies.contains(b) || excludeBookies.contains(b)) { continue; } @@ -104,7 +105,7 @@ public PlacementResult> newEnsemble(int ensembleSize, } } else { Collections.shuffle(allBookies); - for (BookieSocketAddress bookie : allBookies) { + for (BookieId bookie : allBookies) { if (excludeBookies.contains(bookie)) { continue; } @@ -120,37 +121,37 @@ public PlacementResult> newEnsemble(int ensembleSize, } @Override - public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, List currentEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies) + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, + BookieId bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { excludeBookies.addAll(currentEnsemble); - List addresses = newEnsemble(1, 1, 1, customMetadata, excludeBookies).getResult(); + List addresses = newEnsemble(1, 1, 1, customMetadata, excludeBookies).getResult(); - BookieSocketAddress candidateAddr = addresses.get(0); - List newEnsemble = new ArrayList(currentEnsemble); + BookieId candidateAddr = addresses.get(0); + List newEnsemble = new ArrayList(currentEnsemble); newEnsemble.set(currentEnsemble.indexOf(bookieToReplace), candidateAddr); return PlacementResult.of(candidateAddr, isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); } @Override - public Set onClusterChanged(Set writableBookies, - Set readOnlyBookies) { + public Set onClusterChanged(Set writableBookies, + Set readOnlyBookies) { rwLock.writeLock().lock(); try { - HashSet deadBookies; - deadBookies = new HashSet(knownBookies); + HashSet deadBookies; + deadBookies = new HashSet(knownBookies); deadBookies.removeAll(writableBookies); // readonly bookies should not be treated as dead bookies deadBookies.removeAll(readOnlyBookies); if (this.isWeighted) { - for (BookieSocketAddress b : deadBookies) { + for (BookieId b : deadBookies) { this.bookieInfoMap.remove(b); } @SuppressWarnings("unchecked") - Collection newBookies = CollectionUtils.subtract(writableBookies, knownBookies); - for (BookieSocketAddress b : newBookies) { + Collection newBookies = CollectionUtils.subtract(writableBookies, knownBookies); + for (BookieId b : newBookies) { this.bookieInfoMap.put(b, new BookieInfo()); } if (deadBookies.size() > 0 || newBookies.size() > 0) { @@ -165,13 +166,13 @@ public Set onClusterChanged(Set writab } @Override - public void registerSlowBookie(BookieSocketAddress bookieSocketAddress, long entryId) { + public void registerSlowBookie(BookieId bookieSocketAddress, long entryId) { return; } @Override public DistributionSchedule.WriteSet reorderReadSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { return writeSet; @@ -179,7 +180,7 @@ public DistributionSchedule.WriteSet reorderReadSequence( @Override public DistributionSchedule.WriteSet reorderReadLACSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { writeSet.addMissingIndices(ensemble.size()); @@ -188,23 +189,24 @@ public DistributionSchedule.WriteSet reorderReadLACSequence( @Override public EnsemblePlacementPolicy initialize(ClientConfiguration conf, - Optional optionalDnsResolver, - HashedWheelTimer timer, - FeatureProvider featureProvider, - StatsLogger statsLogger) { + Optional optionalDnsResolver, + HashedWheelTimer hashedWheelTimer, + FeatureProvider featureProvider, + StatsLogger statsLogger, + BookieAddressResolver bookieAddressResolver) { this.isWeighted = conf.getDiskWeightBasedPlacementEnabled(); if (this.isWeighted) { this.maxWeightMultiple = conf.getBookieMaxWeightMultipleForWeightBasedPlacement(); - this.weightedSelection = new WeightedRandomSelectionImpl(this.maxWeightMultiple); + this.weightedSelection = new WeightedRandomSelectionImpl(this.maxWeightMultiple); } return this; } @Override - public void updateBookieInfo(Map bookieInfoMap) { + public void updateBookieInfo(Map bookieInfoMap) { rwLock.writeLock().lock(); try { - for (Map.Entry e : bookieInfoMap.entrySet()) { + for (Map.Entry e : bookieInfoMap.entrySet()) { this.bookieInfoMap.put(e.getKey(), e.getValue()); } this.weightedSelection.updateMap(this.bookieInfoMap); @@ -219,7 +221,7 @@ public void uninitalize() { } @Override - public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, int ackQuorumSize) { return PlacementPolicyAdherence.MEETS_STRICT; } 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 dc6534611f1..4628dff48b3 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 @@ -20,7 +20,7 @@ import java.util.BitSet; import java.util.Map; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; /** * This interface determins how entries are distributed among bookies. @@ -172,14 +172,14 @@ interface AckSet { * bookie address * @return true if ack quorum is broken, false otherwise. */ - boolean failBookieAndCheck(int bookieIndexHeardFrom, BookieSocketAddress address); + boolean failBookieAndCheck(int bookieIndexHeardFrom, BookieId address); /** * Return the list of bookies that already failed. * * @return the list of bookies that already failed. */ - Map getFailedBookies(); + Map getFailedBookies(); /** * Invalidate a previous bookie response. 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 f60f507447c..fcd38f2a92f 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 @@ -33,8 +33,9 @@ import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.StatsLogger; /** @@ -216,7 +217,8 @@ EnsemblePlacementPolicy initialize(ClientConfiguration conf, Optional optionalDnsResolver, HashedWheelTimer hashedWheelTimer, FeatureProvider featureProvider, - StatsLogger statsLogger); + StatsLogger statsLogger, + BookieAddressResolver bookieAddressResolver); /** * Uninitialize the policy. @@ -239,8 +241,8 @@ EnsemblePlacementPolicy initialize(ClientConfiguration conf, * All the bookies in the cluster available for readonly. * @return the dead bookies during this cluster change. */ - Set onClusterChanged(Set writableBookies, - Set readOnlyBookies); + Set onClusterChanged(Set writableBookies, + Set readOnlyBookies); /** * Choose numBookies bookies for ensemble. If the count is more than the number of available @@ -267,11 +269,11 @@ Set onClusterChanged(Set writableBooki * @throws BKNotEnoughBookiesException if not enough bookies available. * @return a placement result containing list of bookie addresses for the ensemble. */ - PlacementResult> newEnsemble(int ensembleSize, + PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, - Set excludeBookies) + Set excludeBookies) throws BKNotEnoughBookiesException; /** @@ -294,13 +296,13 @@ PlacementResult> newEnsemble(int ensembleSize, * @throws BKNotEnoughBookiesException * @return a placement result containing the new bookie address. */ - PlacementResult replaceBookie(int ensembleSize, + PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Map customMetadata, - List currentEnsemble, - BookieSocketAddress bookieToReplace, - Set excludeBookies) + List currentEnsemble, + BookieId bookieToReplace, + Set excludeBookies) throws BKNotEnoughBookiesException; /** @@ -311,7 +313,7 @@ PlacementResult replaceBookie(int ensembleSize, * @param entryId * Entry ID that caused a speculative timeout on the bookie. */ - void registerSlowBookie(BookieSocketAddress bookieSocketAddress, long entryId); + void registerSlowBookie(BookieId bookieSocketAddress, long entryId); /** * Reorder the read sequence of a given write quorum writeSet. @@ -328,7 +330,7 @@ PlacementResult replaceBookie(int ensembleSize, * @since 4.5 */ DistributionSchedule.WriteSet reorderReadSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet); @@ -348,7 +350,7 @@ DistributionSchedule.WriteSet reorderReadSequence( * @since 4.5 */ DistributionSchedule.WriteSet reorderReadLACSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet); @@ -359,7 +361,7 @@ DistributionSchedule.WriteSet reorderReadLACSequence( * A map that has the bookie to BookieInfo * @since 4.5 */ - default void updateBookieInfo(Map bookieInfoMap) { + default void updateBookieInfo(Map bookieInfoMap) { } /** @@ -408,14 +410,14 @@ default int getStickyReadBookieIndex(LedgerMetadata metadata, Optional * considered as FAIL. * * @param ensembleList - * list of BookieSocketAddress of bookies in the ensemble + * list of BookieId of bookies in the ensemble * @param writeQuorumSize * writeQuorumSize of the ensemble * @param ackQuorumSize * ackQuorumSize of the ensemble * @return */ - default PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + default PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, int ackQuorumSize) { return PlacementPolicyAdherence.FAIL; } @@ -426,14 +428,14 @@ default PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ackedBookies, + default boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, int writeQuorumSize, int ackQuorumSize) { return true; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java index 6995a069e17..2126d24b46b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java @@ -29,33 +29,33 @@ import java.util.Set; import org.apache.bookkeeper.client.api.LedgerMetadata; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; class EnsembleUtils { private static final Logger LOG = LoggerFactory.getLogger(EnsembleUtils.class); - static List replaceBookiesInEnsemble(BookieWatcher bookieWatcher, + static List replaceBookiesInEnsemble(BookieWatcher bookieWatcher, LedgerMetadata metadata, - List oldEnsemble, - Map failedBookies, + List oldEnsemble, + Map failedBookies, String logContext) throws BKException.BKNotEnoughBookiesException { - List newEnsemble = new ArrayList<>(oldEnsemble); + List newEnsemble = new ArrayList<>(oldEnsemble); int ensembleSize = metadata.getEnsembleSize(); int writeQ = metadata.getWriteQuorumSize(); int ackQ = metadata.getAckQuorumSize(); Map customMetadata = metadata.getCustomMetadata(); - Set exclude = new HashSet<>(failedBookies.values()); + Set exclude = new HashSet<>(failedBookies.values()); int replaced = 0; - for (Map.Entry entry : failedBookies.entrySet()) { + for (Map.Entry entry : failedBookies.entrySet()) { int idx = entry.getKey(); - BookieSocketAddress addr = entry.getValue(); + BookieId addr = entry.getValue(); if (LOG.isDebugEnabled()) { LOG.debug("{} replacing bookie: {} index: {}", logContext, addr, idx); } @@ -68,7 +68,7 @@ static List replaceBookiesInEnsemble(BookieWatcher bookieWa continue; } try { - BookieSocketAddress newBookie = bookieWatcher.replaceBookie( + BookieId newBookie = bookieWatcher.replaceBookie( ensembleSize, writeQ, ackQ, customMetadata, newEnsemble, idx, exclude); newEnsemble.set(idx, newBookie); @@ -85,8 +85,8 @@ static List replaceBookiesInEnsemble(BookieWatcher bookieWa return newEnsemble; } - static Set diffEnsemble(List e1, - List e2) { + static Set diffEnsemble(List e1, + List e2) { checkArgument(e1.size() == e2.size(), "Ensembles must be of same size"); Set diff = new HashSet<>(); for (int i = 0; i < e1.size(); i++) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ForceLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ForceLedgerOp.java index 2d785ab6a17..23d547c407a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ForceLedgerOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ForceLedgerOp.java @@ -21,7 +21,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback; import org.apache.bookkeeper.util.SafeRunnable; @@ -40,7 +40,7 @@ class ForceLedgerOp extends SafeRunnable implements ForceLedgerCallback { boolean completed = false; boolean errored = false; int lastSeenError = BKException.Code.WriteException; - final List currentEnsemble; + final List currentEnsemble; long currentNonDurableLastAddConfirmed = LedgerHandle.INVALID_ENTRY_ID; @@ -48,7 +48,7 @@ class ForceLedgerOp extends SafeRunnable implements ForceLedgerCallback { final BookieClient bookieClient; ForceLedgerOp(LedgerHandle lh, BookieClient bookieClient, - List ensemble, + List ensemble, CompletableFuture cb) { this.lh = lh; this.bookieClient = bookieClient; @@ -89,7 +89,7 @@ void initiate() { } @Override - public void forceLedgerComplete(int rc, long ledgerId, BookieSocketAddress addr, Object ctx) { + public void forceLedgerComplete(int rc, long ledgerId, BookieId addr, Object ctx) { int bookieIndex = (Integer) ctx; checkState(!completed, "We are waiting for all the bookies, it is not expected an early exit"); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java index d6197c09e8b..1abc31cb994 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java @@ -25,8 +25,8 @@ import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; -import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.Node; /** @@ -66,7 +66,7 @@ interface Ensemble { /** * @return list of addresses representing the ensemble */ - List toList(); + List toList(); /** * Validates if an ensemble is valid. @@ -93,11 +93,11 @@ interface Ensemble { * @return list of bookies forming the ensemble * @throws BKException.BKNotEnoughBookiesException */ - PlacementResult> newEnsemble( + PlacementResult> newEnsemble( int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies, + Set excludeBookies, Ensemble parentEnsemble, Predicate parentPredicate) throws BKException.BKNotEnoughBookiesException; @@ -176,7 +176,7 @@ T selectFromNetworkLocation(String networkLoc, * @param leftBookies * bookies that left */ - void handleBookiesThatLeft(Set leftBookies); + void handleBookiesThatLeft(Set leftBookies); /** * Handle bookies that joined. @@ -184,12 +184,12 @@ T selectFromNetworkLocation(String networkLoc, * @param joinedBookies * bookies that joined. */ - void handleBookiesThatJoined(Set joinedBookies); + void handleBookiesThatJoined(Set joinedBookies); /** * Handle rack change for the bookies. * * @param bookieAddressList */ - void onBookieRackChange(List bookieAddressList); + void onBookieRackChange(List bookieAddressList); } 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 6cc23cc2699..a0a9bb2db8d 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 @@ -31,7 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.client.BKException.Code; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; @@ -186,7 +186,7 @@ private void verifyLedgerFragment(LedgerFragment fragment, long firstStored = fragment.getFirstStoredEntryId(bookieIndex); long lastStored = fragment.getLastStoredEntryId(bookieIndex); - BookieSocketAddress bookie = fragment.getAddress(bookieIndex); + BookieId bookie = fragment.getAddress(bookieIndex); if (null == bookie) { throw new InvalidFragmentException(); } @@ -325,8 +325,8 @@ public void checkLedger(final LedgerHandle lh, final Set fragments = new HashSet(); Long curEntryId = null; - List curEnsemble = null; - for (Map.Entry> e : lh + List curEnsemble = null; + for (Map.Entry> e : lh .getLedgerMetadata().getAllEnsembles().entrySet()) { if (curEntryId != null) { Set bookieIndexes = new HashSet(); @@ -386,7 +386,7 @@ public void operationComplete(int rc, Boolean result) { DistributionSchedule.WriteSet writeSet = lh.getDistributionSchedule().getWriteSet(entryToRead); for (int i = 0; i < writeSet.size(); i++) { - BookieSocketAddress addr = curEnsemble.get(writeSet.get(i)); + BookieId addr = curEnsemble.get(writeSet.get(i)); bookieClient.readEntry(addr, lh.getId(), entryToRead, eecb, null, BookieProtocol.FLAG_NONE); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java index 9e9a2c4add9..557d0c7256e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java @@ -43,7 +43,7 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.api.WriteHandle; import org.apache.bookkeeper.meta.LedgerIdGenerator; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.MathUtils; @@ -137,7 +137,7 @@ public void initiate() { // select bookies for first ensemble try { - List ensemble = bk.getBookieWatcher() + List ensemble = bk.getBookieWatcher() .newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata); metadataBuilder.newEnsembleEntry(0L, ensemble); } catch (BKNotEnoughBookiesException e) { @@ -217,10 +217,10 @@ private void metadataCallback(Versioned writtenMetadata, Throwab return; } - List curEns = lh.getLedgerMetadata().getEnsembleAt(0L); + List curEns = lh.getLedgerMetadata().getEnsembleAt(0L); LOG.info("Ensemble: {} for ledger: {}", curEns, lh.getId()); - for (BookieSocketAddress bsa : curEns) { + for (BookieId bsa : curEns) { clientStats.getEnsembleBookieDistributionCounter(bsa.toString()).inc(); } 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 dcf986f3fe9..1fb1e50cb02 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 @@ -23,7 +23,7 @@ import java.util.List; import java.util.Set; import java.util.SortedMap; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; /** * Represents the entries of a segment of a ledger which are stored on subset of @@ -33,7 +33,7 @@ */ public class LedgerFragment { private final Set bookieIndexes; - private final List ensemble; + private final List ensemble; private final long firstEntryId; private final long lastKnownEntryId; private final long ledgerId; @@ -50,7 +50,7 @@ public class LedgerFragment { this.bookieIndexes = bookieIndexes; this.ensemble = lh.getLedgerMetadata().getEnsembleAt(firstEntryId); this.schedule = lh.getDistributionSchedule(); - SortedMap> ensembles = lh + SortedMap> ensembles = lh .getLedgerMetadata().getAllEnsembles(); this.isLedgerClosed = lh.getLedgerMetadata().isClosed() || !ensemble.equals(ensembles.get(ensembles.lastKey())); @@ -106,12 +106,12 @@ public long getLastKnownEntryId() { /** * Gets the failedBookie address. */ - public BookieSocketAddress getAddress(int bookieIndex) { + public BookieId getAddress(int bookieIndex) { return ensemble.get(bookieIndex); } - public Set getAddresses() { - Set addresses = new HashSet(); + public Set getAddresses() { + Set addresses = new HashSet(); for (int bookieIndex : bookieIndexes) { addresses.add(ensemble.get(bookieIndex)); } @@ -213,7 +213,7 @@ public boolean isStoredEntryId(long entryId, int bookieIndex) { * * @return the ensemble for the segment which this fragment is a part of */ - public List getEnsemble() { + public List getEnsemble() { return this.ensemble; } 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 9510a9656d6..b2c1933dea2 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 @@ -43,7 +43,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.MultiCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; @@ -110,7 +110,7 @@ public LedgerFragmentReplicator(BookKeeper bkc) { private void replicateFragmentInternal(final LedgerHandle lh, final LedgerFragment lf, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final Set newBookies, + final Set newBookies, final BiConsumer onReadEntryFailureCallback) throws InterruptedException { if (!lf.isClosed()) { LOG.error("Trying to replicate an unclosed fragment;" @@ -184,7 +184,7 @@ private void replicateFragmentInternal(final LedgerHandle lh, */ void replicate(final LedgerHandle lh, final LedgerFragment lf, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final Set targetBookieAddresses, + final Set targetBookieAddresses, final BiConsumer onReadEntryFailureCallback) throws InterruptedException { Set partionedFragments = splitIntoSubFragments(lh, lf, @@ -201,7 +201,7 @@ void replicate(final LedgerHandle lh, final LedgerFragment lf, private void replicateNextBatch(final LedgerHandle lh, final Iterator fragments, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final Set targetBookieAddresses, + final Set targetBookieAddresses, final BiConsumer onReadEntryFailureCallback) { if (fragments.hasNext()) { try { @@ -305,14 +305,14 @@ static Set splitIntoSubFragments(LedgerHandle lh, private void recoverLedgerFragmentEntry(final Long entryId, final LedgerHandle lh, final AsyncCallback.VoidCallback ledgerFragmentEntryMcb, - final Set newBookies, + final Set newBookies, final BiConsumer onReadEntryFailureCallback) throws InterruptedException { final long ledgerId = lh.getId(); 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) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { if (rc != BKException.Code.OK) { LOG.error("BK error writing entry for ledgerId: {}, entryId: {}, bookie: {}", ledgerId, entryId, addr, BKException.create(rc)); @@ -363,7 +363,7 @@ public void readComplete(int rc, LedgerHandle lh, .computeDigestAndPackageForSending(entryId, lh.getLastAddConfirmed(), entry.getLength(), Unpooled.wrappedBuffer(data, 0, data.length)); - for (BookieSocketAddress newBookie : newBookies) { + for (BookieId newBookie : newBookies) { bkc.getBookieClient().addEntry(newBookie, lh.getId(), lh.getLedgerKey(), entryId, ByteBufList.clone(toSend), multiWriteCallback, dataLength, BookieProtocol.FLAG_RECOVERY_ADD, @@ -385,11 +385,11 @@ static class SingleFragmentCallback implements AsyncCallback.VoidCallback { final LedgerHandle lh; final LedgerManager ledgerManager; final long fragmentStartId; - final Map oldBookie2NewBookie; + final Map oldBookie2NewBookie; SingleFragmentCallback(AsyncCallback.VoidCallback ledgerFragmentsMcb, LedgerHandle lh, LedgerManager ledgerManager, long fragmentStartId, - Map oldBookie2NewBookie) { + Map oldBookie2NewBookie) { this.ledgerFragmentsMcb = ledgerFragmentsMcb; this.lh = lh; this.ledgerManager = ledgerManager; @@ -414,7 +414,7 @@ public void processResult(int rc, String path, Object ctx) { */ private static void updateEnsembleInfo( LedgerManager ledgerManager, AsyncCallback.VoidCallback ensembleUpdatedCb, long fragmentStartId, - LedgerHandle lh, Map oldBookie2NewBookie) { + LedgerHandle lh, Map oldBookie2NewBookie) { MetadataUpdateLoop updateLoop = new MetadataUpdateLoop( ledgerManager, @@ -422,12 +422,12 @@ private static void updateEnsembleInfo( lh::getVersionedLedgerMetadata, (metadata) -> { // returns true if any of old bookies exist in ensemble - List ensemble = metadata.getAllEnsembles().get(fragmentStartId); + List ensemble = metadata.getAllEnsembles().get(fragmentStartId); return oldBookie2NewBookie.keySet().stream().anyMatch(ensemble::contains); }, (currentMetadata) -> { // replace all old bookies with new bookies in ensemble - List newEnsemble = currentMetadata.getAllEnsembles().get(fragmentStartId) + List newEnsemble = currentMetadata.getAllEnsembles().get(fragmentStartId) .stream().map((bookie) -> oldBookie2NewBookie.getOrDefault(bookie, bookie)) .collect(Collectors.toList()); return LedgerMetadataBuilder.from(currentMetadata) 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 f80d0bd4053..b773d969d78 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 @@ -76,7 +76,7 @@ import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.stats.Counter; @@ -140,15 +140,15 @@ private enum HandleState { final DigestManager macManager; final DistributionSchedule distributionSchedule; final RateLimiter throttler; - final LoadingCache bookieFailureHistory; + final LoadingCache bookieFailureHistory; final BookiesHealthInfo bookiesHealthInfo; final EnumSet writeFlags; ScheduledFuture timeoutFuture = null; @VisibleForTesting - final Map delayedWriteFailedBookies = - new HashMap(); + final Map delayedWriteFailedBookies = + new HashMap(); /** * Invalid entry id. This value is returned from methods which @@ -222,21 +222,21 @@ && getLedgerMetadata().getEnsembleSize() == getLedgerMetadata().getWriteQuorumSi metadata.getEnsembleSize()); this.bookieFailureHistory = CacheBuilder.newBuilder() .expireAfterWrite(clientCtx.getConf().bookieFailureHistoryExpirationMSec, TimeUnit.MILLISECONDS) - .build(new CacheLoader() { + .build(new CacheLoader() { @Override - public Long load(BookieSocketAddress key) { + public Long load(BookieId key) { return -1L; } }); this.bookiesHealthInfo = new BookiesHealthInfo() { @Override - public long getBookieFailureHistory(BookieSocketAddress bookieSocketAddress) { + public long getBookieFailureHistory(BookieId bookieSocketAddress) { Long lastFailure = bookieFailureHistory.getIfPresent(bookieSocketAddress); return lastFailure == null ? -1L : lastFailure; } @Override - public long getBookiePendingRequests(BookieSocketAddress bookieSocketAddress) { + public long getBookiePendingRequests(BookieId bookieSocketAddress) { return clientCtx.getBookieClient().getNumPendingRequests(bookieSocketAddress, ledgerId); } }; @@ -403,9 +403,9 @@ public synchronized long getNumFragments() { * @return count of unique bookies */ public synchronized long getNumBookies() { - Map> m = getLedgerMetadata().getAllEnsembles(); - Set s = Sets.newHashSet(); - for (List aList : m.values()) { + Map> m = getLedgerMetadata().getAllEnsembles(); + Set s = Sets.newHashSet(); + for (List aList : m.values()) { s.addAll(aList); } return s.size(); @@ -1222,7 +1222,7 @@ private boolean isWritesetWritable(DistributionSchedule.WriteSet writeSet, final int requiredWritable = sz - allowedNonWritableCount; int nonWritableCount = 0; - List currentEnsemble = getCurrentEnsemble(); + List currentEnsemble = getCurrentEnsemble(); for (int i = 0; i < sz; i++) { if (!clientCtx.getBookieClient().isWritable(currentEnsemble.get(i), key)) { nonWritableCount++; @@ -1830,7 +1830,7 @@ boolean hasDelayedWriteFailedBookies() { return !delayedWriteFailedBookies.isEmpty(); } - void notifyWriteFailed(int index, BookieSocketAddress addr) { + void notifyWriteFailed(int index, BookieId addr) { synchronized (metadataLock) { delayedWriteFailedBookies.put(index, addr); } @@ -1841,7 +1841,7 @@ void maybeHandleDelayedWriteBookieFailure() { if (delayedWriteFailedBookies.isEmpty()) { return; } - Map toReplace = new HashMap<>(delayedWriteFailedBookies); + Map toReplace = new HashMap<>(delayedWriteFailedBookies); delayedWriteFailedBookies.clear(); // Original intent of this change is to do a best-effort ensemble change. @@ -1852,7 +1852,7 @@ void maybeHandleDelayedWriteBookieFailure() { } } - void handleBookieFailure(final Map failedBookies) { + void handleBookieFailure(final Map failedBookies) { if (clientCtx.getConf().disableEnsembleChangeFeature.isAvailable()) { if (LOG.isDebugEnabled()) { LOG.debug("Ensemble change is disabled. Retry sending to failed bookies {} for ledger {}.", @@ -1874,8 +1874,8 @@ void handleBookieFailure(final Map failedBookies) boolean triggerLoop = false; - Map toReplace = null; - List origEnsemble = null; + Map toReplace = null; + List origEnsemble = null; synchronized (metadataLock) { if (changingEnsemble) { delayedWriteFailedBookies.putAll(failedBookies); @@ -1895,7 +1895,7 @@ void handleBookieFailure(final Map failedBookies) } } - void ensembleChangeLoop(List origEnsemble, Map failedBookies) { + void ensembleChangeLoop(List origEnsemble, Map failedBookies) { int ensembleChangeId = numEnsembleChanges.incrementAndGet(); String logContext = String.format("[EnsembleChange(ledger:%d, change-id:%010d)]", ledgerId, ensembleChangeId); @@ -1922,8 +1922,8 @@ void ensembleChangeLoop(List origEnsemble, Map { attempts.incrementAndGet(); - List currentEnsemble = getCurrentEnsemble(); - List newEnsemble = EnsembleUtils.replaceBookiesInEnsemble( + List currentEnsemble = getCurrentEnsemble(); + List newEnsemble = EnsembleUtils.replaceBookiesInEnsemble( clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext); Long lastEnsembleKey = LedgerMetadataUtils.getLastEnsembleKey(metadata); LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata); @@ -1964,11 +1964,11 @@ void ensembleChangeLoop(List origEnsemble, Map newEnsemble = null; + List newEnsemble = null; Set replaced = null; synchronized (metadataLock) { if (!delayedWriteFailedBookies.isEmpty()) { - Map toReplace = new HashMap<>(delayedWriteFailedBookies); + Map toReplace = new HashMap<>(delayedWriteFailedBookies); delayedWriteFailedBookies.clear(); ensembleChangeLoop(origEnsemble, toReplace); @@ -1987,7 +1987,7 @@ void ensembleChangeLoop(List origEnsemble, Map ensemble, final Set bookies) { + void unsetSuccessAndSendWriteRequest(List ensemble, final Set bookies) { for (PendingAddOp pendingAddOp : pendingAddOps) { for (Integer bookieIndex: bookies) { pendingAddOp.unsetSuccessAndSendWriteRequest(ensemble, bookieIndex); @@ -1995,7 +1995,7 @@ void unsetSuccessAndSendWriteRequest(List ensemble, final S } } - void registerOperationFailureOnBookie(BookieSocketAddress bookie, long entryId) { + void registerOperationFailureOnBookie(BookieId bookie, long entryId) { if (clientCtx.getConf().enableBookieFailureTracking) { bookieFailureHistory.put(bookie, entryId); } @@ -2026,7 +2026,7 @@ public void closeComplete(int rc, LedgerHandle lh, Object ctx) { * operations themselves, to avoid adding more dependencies between the classes. * There are too many already. */ - List getCurrentEnsemble() { + List getCurrentEnsemble() { // Getting current ensemble from the metadata is only a temporary // thing until metadata is immutable. At that point, current ensemble // becomes a property of the LedgerHandle itself. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java index e4f75cedfac..a962fe7f7c5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java @@ -36,7 +36,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata.State; import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; /** * Builder for building LedgerMetadata objects. @@ -54,7 +54,7 @@ public class LedgerMetadataBuilder { private Optional lastEntryId = Optional.empty(); private Optional length = Optional.empty(); - private TreeMap> ensembles = new TreeMap<>(); + private TreeMap> ensembles = new TreeMap<>(); private Optional digestType = Optional.empty(); private Optional password = Optional.empty(); @@ -131,7 +131,7 @@ public LedgerMetadataBuilder withAckQuorumSize(int ackQuorumSize) { return this; } - public LedgerMetadataBuilder newEnsembleEntry(long firstEntry, List ensemble) { + public LedgerMetadataBuilder newEnsembleEntry(long firstEntry, List ensemble) { checkArgument(ensemble.size() == ensembleSize, "Size of passed in ensemble must match the ensembleSize of the builder"); checkArgument(ensembles.isEmpty() || firstEntry > ensembles.lastKey(), @@ -140,7 +140,7 @@ public LedgerMetadataBuilder newEnsembleEntry(long firstEntry, List ensemble) { + public LedgerMetadataBuilder replaceEnsembleEntry(long firstEntry, List ensemble) { checkArgument(ensemble.size() == ensembleSize, "Size of passed in ensemble must match the ensembleSize of the builder"); checkArgument(ensembles.containsKey(firstEntry), diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java index 842244ac7e2..81fb4b5d0ac 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java @@ -35,7 +35,7 @@ import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.LedgerMetadata.State; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,8 +60,8 @@ class LedgerMetadataImpl implements LedgerMetadata { private final long ctime; final boolean storeCtime; // non-private so builder can access for copy - private final NavigableMap> ensembles; - private final ImmutableList currentEnsemble; + private final NavigableMap> ensembles; + private final ImmutableList currentEnsemble; private final boolean hasPassword; private final DigestType digestType; @@ -78,7 +78,7 @@ class LedgerMetadataImpl implements LedgerMetadata { State state, Optional lastEntryId, Optional length, - Map> ensembles, + Map> ensembles, Optional digestType, Optional password, long ctime, @@ -136,7 +136,7 @@ class LedgerMetadataImpl implements LedgerMetadata { } @Override - public NavigableMap> getAllEnsembles() { + public NavigableMap> getAllEnsembles() { return ensembles; } @@ -210,7 +210,7 @@ public State getState() { } @Override - public List getEnsembleAt(long entryId) { + public List getEnsembleAt(long entryId) { // the head map cannot be empty, since we insert an ensemble for // entry-id 0, right when we start return ensembles.get(ensembles.headMap(entryId + 1).lastKey()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java index 07d73409613..0c9b6aa71ad 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java @@ -24,7 +24,7 @@ import java.util.Set; import java.util.SortedMap; import org.apache.bookkeeper.client.api.LedgerMetadata; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,7 +34,7 @@ public class LedgerMetadataUtils { static final Logger LOG = LoggerFactory.getLogger(LedgerMetadataUtils.class); - static List getCurrentEnsemble(LedgerMetadata metadata) { + static List getCurrentEnsemble(LedgerMetadata metadata) { return getLastEnsembleValue(metadata); } @@ -46,7 +46,7 @@ static List getCurrentEnsemble(LedgerMetadata metadata) { * @return the entry id of the next ensemble change (-1 if no further ensemble changes) */ static long getNextEnsembleChange(LedgerMetadata metadata, long entryId) { - SortedMap> tailMap = metadata.getAllEnsembles().tailMap(entryId + 1); + SortedMap> tailMap = metadata.getAllEnsembles().tailMap(entryId + 1); if (tailMap.isEmpty()) { return -1; @@ -55,15 +55,15 @@ static long getNextEnsembleChange(LedgerMetadata metadata, long entryId) { } } - static Set getBookiesInThisLedger(LedgerMetadata metadata) { - Set bookies = new HashSet(); - for (List ensemble : metadata.getAllEnsembles().values()) { + static Set getBookiesInThisLedger(LedgerMetadata metadata) { + Set bookies = new HashSet(); + for (List ensemble : metadata.getAllEnsembles().values()) { bookies.addAll(ensemble); } return bookies; } - static List getLastEnsembleValue(LedgerMetadata metadata) { + static List getLastEnsembleValue(LedgerMetadata metadata) { checkArgument(!metadata.getAllEnsembles().isEmpty(), "Metadata should never be created with no ensembles"); return metadata.getAllEnsembles().lastEntry().getValue(); } 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 cd1a03fd555..b343d1a72f8 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 @@ -38,7 +38,7 @@ import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.AsyncCallback.AddCallbackWithLatency; import org.apache.bookkeeper.client.api.WriteFlag; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.util.MathUtils; @@ -73,7 +73,7 @@ class PendingAddOp extends SafeRunnable implements WriteCallback { boolean isRecoveryAdd = false; long requestTimeNanos; long qwcLatency; // Quorum Write Completion Latency after response from quorum bookies. - Set addEntrySuccessBookies; + Set addEntrySuccessBookies; long writeDelayedStartTime; // min fault domains completion latency after response from ack quorum bookies long currentLedgerLength; @@ -82,10 +82,10 @@ class PendingAddOp extends SafeRunnable implements WriteCallback { boolean hasRun; EnumSet writeFlags; boolean allowFailFast = false; - List ensemble; + List ensemble; static PendingAddOp create(LedgerHandle lh, ClientContext clientCtx, - List ensemble, + List ensemble, ByteBuf payload, EnumSet writeFlags, AddCallbackWithLatency cb, Object ctx) { PendingAddOp op = RECYCLER.get(); @@ -146,7 +146,7 @@ long getEntryId() { return this.entryId; } - void sendWriteRequest(List ensemble, int bookieIndex) { + void sendWriteRequest(List ensemble, int bookieIndex) { int flags = isRecoveryAdd ? FLAG_RECOVERY_ADD | FLAG_HIGH_PRIORITY : FLAG_NONE; clientCtx.getBookieClient().addEntry(ensemble.get(bookieIndex), @@ -188,7 +188,7 @@ public String toString() { } } - void unsetSuccessAndSendWriteRequest(List ensemble, int bookieIndex) { + void unsetSuccessAndSendWriteRequest(List ensemble, int bookieIndex) { // update the ensemble this.ensemble = ensemble; @@ -283,7 +283,7 @@ public void safeRun() { } @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { int bookieIndex = (Integer) ctx; --pendingWriteRequests; @@ -360,7 +360,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre if (clientCtx.getConf().delayEnsembleChange) { if (ackSet.failBookieAndCheck(bookieIndex, addr) || rc == BKException.Code.WriteOnReadOnlyBookieException) { - Map failedBookies = ackSet.getFailedBookies(); + Map failedBookies = ackSet.getFailedBookies(); LOG.warn("Failed to write entry ({}, {}) to bookies {}, handling failures.", ledgerId, entryId, failedBookies); // we can't meet ack quorum requirement, trigger ensemble change. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadLacOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadLacOp.java index 0dad8040ee1..2a15adbe4d5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadLacOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadLacOp.java @@ -22,7 +22,7 @@ import java.util.List; import org.apache.bookkeeper.client.BKException.BKDigestMatchException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadLacCallback; import org.apache.bookkeeper.proto.checksum.DigestManager.RecoveryData; @@ -54,7 +54,7 @@ class PendingReadLacOp implements ReadLacCallback { int lastSeenError = BKException.Code.ReadException; final DistributionSchedule.QuorumCoverageSet coverageSet; long maxLac = LedgerHandle.INVALID_ENTRY_ID; - final List currentEnsemble; + final List currentEnsemble; /* * Wrapper to get Lac from the request @@ -63,7 +63,7 @@ interface LacCallback { void getLacComplete(int rc, long lac); } - PendingReadLacOp(LedgerHandle lh, BookieClient bookieClient, List ensemble, LacCallback cb) { + PendingReadLacOp(LedgerHandle lh, BookieClient bookieClient, List ensemble, LacCallback cb) { this.lh = lh; this.bookieClient = bookieClient; this.cb = cb; 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 bfd9a5d771b..e6c011ee32a 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 @@ -39,7 +39,7 @@ import org.apache.bookkeeper.client.impl.LedgerEntriesImpl; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; import org.apache.bookkeeper.common.util.SafeRunnable; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallbackCtx; @@ -61,9 +61,9 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable { private ScheduledFuture speculativeTask = null; protected final List seq; private final CompletableFuture future; - private final Set heardFromHosts; + private final Set heardFromHosts; private final BitSet heardFromHostsBitSet; - private final Set sentToHosts = new HashSet(); + private final Set sentToHosts = new HashSet(); LedgerHandle lh; final ClientContext clientCtx; @@ -87,12 +87,12 @@ abstract class LedgerEntryRequest implements SpeculativeRequestExecutor, AutoClo int firstError = BKException.Code.OK; int numBookiesMissingEntry = 0; - final List ensemble; + final List ensemble; final DistributionSchedule.WriteSet writeSet; final LedgerEntryImpl entryImpl; final long eId; - LedgerEntryRequest(List ensemble, long lId, long eId) { + LedgerEntryRequest(List ensemble, long lId, long eId) { this.entryImpl = LedgerEntryImpl.create(lId, eId); this.ensemble = ensemble; this.eId = eId; @@ -130,7 +130,7 @@ public void close() { * @return return true if we managed to complete the entry; * otherwise return false if the read entry is not complete or it is already completed before */ - boolean complete(int bookieIndex, BookieSocketAddress host, final ByteBuf buffer) { + boolean complete(int bookieIndex, BookieId host, final ByteBuf buffer) { ByteBuf content; if (isComplete()) { return false; @@ -188,7 +188,7 @@ boolean fail(int rc) { * @param rc * read result code */ - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { if (BKException.Code.OK == firstError || BKException.Code.NoSuchEntryException == firstError || BKException.Code.NoSuchLedgerExistsException == firstError) { @@ -226,7 +226,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress * the set of hosts that we already received responses. * @return host we sent to if we sent. null otherwise. */ - abstract BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet); + abstract BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet); /** * Whether the read request completed. @@ -280,7 +280,7 @@ class ParallelReadRequest extends LedgerEntryRequest { int numPendings; - ParallelReadRequest(List ensemble, long lId, long eId) { + ParallelReadRequest(List ensemble, long lId, long eId) { super(ensemble, lId, eId); numPendings = writeSet.size(); } @@ -288,7 +288,7 @@ class ParallelReadRequest extends LedgerEntryRequest { @Override void read() { for (int i = 0; i < writeSet.size(); i++) { - BookieSocketAddress to = ensemble.get(writeSet.get(i)); + BookieId to = ensemble.get(writeSet.get(i)); try { sendReadTo(writeSet.get(i), to, this); } catch (InterruptedException ie) { @@ -301,7 +301,7 @@ void read() { } @Override - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); // if received all responses or this entry doesn't meet quorum write, complete the request. @@ -317,7 +317,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress } @Override - BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) { + BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) { // no speculative read return null; } @@ -330,7 +330,7 @@ class SequenceReadRequest extends LedgerEntryRequest { final BitSet sentReplicas; final BitSet erroredReplicas; - SequenceReadRequest(List ensemble, long lId, long eId) { + SequenceReadRequest(List ensemble, long lId, long eId) { super(ensemble, lId, eId); this.sentReplicas = new BitSet(lh.getLedgerMetadata().getWriteQuorumSize()); @@ -362,7 +362,7 @@ private boolean readsOutstanding() { * @return host we sent to if we sent. null otherwise. */ @Override - synchronized BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFrom) { + synchronized BookieId maybeSendSpeculativeRead(BitSet heardFrom) { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getWriteQuorumSize()) { return null; } @@ -386,7 +386,7 @@ void read() { sendNextRead(); } - synchronized BookieSocketAddress sendNextRead() { + synchronized BookieId sendNextRead() { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getWriteQuorumSize()) { // we are done, the read has failed from all replicas, just fail the // read @@ -401,7 +401,7 @@ synchronized BookieSocketAddress sendNextRead() { nextReplicaIndexToReadFrom++; try { - BookieSocketAddress to = ensemble.get(bookieIndex); + BookieId to = ensemble.get(bookieIndex); sendReadTo(bookieIndex, to, this); sentToHosts.add(to); sentReplicas.set(replica); @@ -415,7 +415,7 @@ synchronized BookieSocketAddress sendNextRead() { } @Override - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); int replica = writeSet.indexOf(bookieIndex); @@ -438,7 +438,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress } @Override - boolean complete(int bookieIndex, BookieSocketAddress host, ByteBuf buffer) { + boolean complete(int bookieIndex, BookieId host, ByteBuf buffer) { boolean completed = super.complete(bookieIndex, host, buffer); if (completed) { int numReplicasTried = getNextReplicaIndexToReadFrom(); @@ -446,7 +446,7 @@ boolean complete(int bookieIndex, BookieSocketAddress host, ByteBuf buffer) { // the first successful speculative read as "slow" for (int i = 0; i < numReplicasTried - 1; i++) { int slowBookieIndex = writeSet.get(i); - BookieSocketAddress slowBookieSocketAddress = ensemble.get(slowBookieIndex); + BookieId slowBookieSocketAddress = ensemble.get(slowBookieIndex); clientCtx.getPlacementPolicy().registerSlowBookie(slowBookieSocketAddress, eId); } } @@ -510,7 +510,7 @@ public void submit() { void initiate() { long nextEnsembleChange = startEntryId, i = startEntryId; this.requestTimeNanos = MathUtils.nowInNano(); - List ensemble = null; + List ensemble = null; do { if (i == nextEnsembleChange) { ensemble = getLedgerMetadata().getEnsembleAt(i); @@ -542,11 +542,11 @@ public void safeRun() { private static class ReadContext implements ReadEntryCallbackCtx { final int bookieIndex; - final BookieSocketAddress to; + final BookieId to; final LedgerEntryRequest entry; long lac = LedgerHandle.INVALID_ENTRY_ID; - ReadContext(int bookieIndex, BookieSocketAddress to, LedgerEntryRequest entry) { + ReadContext(int bookieIndex, BookieId to, LedgerEntryRequest entry) { this.bookieIndex = bookieIndex; this.to = to; this.entry = entry; @@ -563,7 +563,7 @@ public long getLastAddConfirmed() { } } - void sendReadTo(int bookieIndex, BookieSocketAddress to, LedgerEntryRequest entry) throws InterruptedException { + void sendReadTo(int bookieIndex, BookieId to, LedgerEntryRequest entry) throws InterruptedException { if (lh.throttler != null) { lh.throttler.acquire(); } 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 e9eb3f23b38..0b9a7d01843 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 @@ -21,7 +21,7 @@ import java.util.List; import org.apache.bookkeeper.client.AsyncCallback.AddLacCallback; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteLacCallback; import org.apache.bookkeeper.util.ByteBufList; import org.slf4j.Logger; @@ -51,9 +51,9 @@ class PendingWriteLacOp implements WriteLacCallback { LedgerHandle lh; ClientContext clientCtx; - final List currentEnsemble; + final List currentEnsemble; - PendingWriteLacOp(LedgerHandle lh, ClientContext clientCtx, List ensemble, + PendingWriteLacOp(LedgerHandle lh, ClientContext clientCtx, List ensemble, AddLacCallback cb, Object ctx) { this.lh = lh; this.clientCtx = clientCtx; @@ -91,7 +91,7 @@ void initiate(ByteBufList toSend) { } @Override - public void writeLacComplete(int rc, long ledgerId, BookieSocketAddress addr, Object ctx) { + public void writeLacComplete(int rc, long ledgerId, BookieId addr, Object ctx) { int bookieIndex = (Integer) ctx; if (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 380f556b13e..626b7cc2e60 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 @@ -23,10 +23,11 @@ import java.util.Set; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; -import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.Node; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.StatsLogger; /** @@ -57,19 +58,21 @@ protected RackawareEnsemblePlacementPolicy initialize(DNSToSwitchMapping dnsReso int minNumRacksPerWriteQuorum, boolean enforceMinNumRacksPerWriteQuorum, boolean ignoreLocalNodeInPlacementPolicy, - StatsLogger statsLogger) { + StatsLogger statsLogger, BookieAddressResolver bookieAddressResolver) { if (stabilizePeriodSeconds > 0) { super.initialize(dnsResolver, timer, reorderReadsRandom, 0, reorderThresholdPendingRequests, isWeighted, maxWeightMultiple, minNumRacksPerWriteQuorum, enforceMinNumRacksPerWriteQuorum, - ignoreLocalNodeInPlacementPolicy, statsLogger); + ignoreLocalNodeInPlacementPolicy, statsLogger, bookieAddressResolver); slave = new RackawareEnsemblePlacementPolicyImpl(enforceDurability); slave.initialize(dnsResolver, timer, reorderReadsRandom, stabilizePeriodSeconds, reorderThresholdPendingRequests, isWeighted, maxWeightMultiple, minNumRacksPerWriteQuorum, - enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy, statsLogger); + enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy, statsLogger, + bookieAddressResolver); } else { super.initialize(dnsResolver, timer, reorderReadsRandom, stabilizePeriodSeconds, reorderThresholdPendingRequests, isWeighted, maxWeightMultiple, minNumRacksPerWriteQuorum, - enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy, statsLogger); + enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy, statsLogger, + bookieAddressResolver); slave = null; } return this; @@ -84,9 +87,9 @@ public void uninitalize() { } @Override - public Set onClusterChanged(Set writableBookies, - Set readOnlyBookies) { - Set deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies); + public Set onClusterChanged(Set writableBookies, + Set readOnlyBookies) { + Set deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies); if (null != slave) { deadBookies = slave.onClusterChanged(writableBookies, readOnlyBookies); } @@ -94,8 +97,8 @@ public Set onClusterChanged(Set writab } @Override - public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map customMetadata, Set excludeBookies) + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { try { return super.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, excludeBookies); @@ -109,9 +112,9 @@ public PlacementResult> newEnsemble(int ensembleSize, } @Override - public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, List currentEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies) + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, + BookieId bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { try { return super.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, @@ -128,7 +131,7 @@ public PlacementResult replaceBookie(int ensembleSize, int @Override public DistributionSchedule.WriteSet reorderReadSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { return super.reorderReadSequence(ensemble, bookiesHealthInfo, @@ -137,7 +140,7 @@ public DistributionSchedule.WriteSet reorderReadSequence( @Override public DistributionSchedule.WriteSet reorderReadLACSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { return super.reorderReadLACSequence(ensemble, bookiesHealthInfo, @@ -145,10 +148,10 @@ public DistributionSchedule.WriteSet reorderReadLACSequence( } @Override - public PlacementResult> newEnsemble(int ensembleSize, + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies, + Set excludeBookies, Ensemble parentEnsemble, Predicate parentPredicate) throws BKException.BKNotEnoughBookiesException { @@ -234,7 +237,7 @@ public BookieNode selectFromNetworkLocation( } @Override - public void handleBookiesThatLeft(Set leftBookies) { + public void handleBookiesThatLeft(Set leftBookies) { super.handleBookiesThatLeft(leftBookies); if (null != slave) { slave.handleBookiesThatLeft(leftBookies); @@ -242,7 +245,7 @@ public void handleBookiesThatLeft(Set leftBookies) { } @Override - public void handleBookiesThatJoined(Set joinedBookies) { + public void handleBookiesThatJoined(Set joinedBookies) { super.handleBookiesThatJoined(joinedBookies); if (null != slave) { slave.handleBookiesThatJoined(joinedBookies); 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 13eaddde125..5568f0a8bdd 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 @@ -32,8 +32,8 @@ import io.netty.util.HashedWheelTimer; +import java.io.IOException; import java.net.InetAddress; -import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -53,8 +53,8 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.Configurable; import org.apache.bookkeeper.feature.FeatureProvider; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; -import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.net.NetworkTopologyImpl; @@ -62,6 +62,7 @@ import org.apache.bookkeeper.net.NodeBase; import org.apache.bookkeeper.net.ScriptBasedMapping; import org.apache.bookkeeper.net.StabilizeNetworkTopology; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.OpStatsLogger; @@ -106,7 +107,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP protected HashedWheelTimer timer; // Use a loading cache so slow bookies are expired. Use entryId as values. - protected Cache slowBookies; + protected Cache slowBookies; protected BookieNode localNode; protected boolean reorderReadsRandom = false; protected boolean enforceDurability = false; @@ -159,9 +160,11 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns int minNumRacksPerWriteQuorum, boolean enforceMinNumRacksPerWriteQuorum, boolean ignoreLocalNodeInPlacementPolicy, - StatsLogger statsLogger) { + StatsLogger statsLogger, + BookieAddressResolver bookieAddressResolver) { checkNotNull(statsLogger, "statsLogger should not be null, use NullStatsLogger instead."); this.statsLogger = statsLogger; + this.bookieAddressResolver = bookieAddressResolver; this.bookiesJoinedCounter = statsLogger.getOpStatsLogger(BOOKIES_JOINED); this.bookiesLeftCounter = statsLogger.getOpStatsLogger(BOOKIES_LEFT); this.readReorderedCounter = statsLogger.getOpStatsLogger(READ_REQUESTS_REORDERED); @@ -203,8 +206,8 @@ public Integer getSample() { BookieNode bn = null; if (!ignoreLocalNodeInPlacementPolicy) { try { - bn = createBookieNode(new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), 0)); - } catch (UnknownHostException e) { + bn = createDummyLocalBookieNode(InetAddress.getLocalHost().getHostAddress()); + } catch (IOException e) { LOG.error("Failed to get local host address : ", e); } } else { @@ -247,7 +250,9 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, Optional optionalDnsResolver, HashedWheelTimer timer, FeatureProvider featureProvider, - StatsLogger statsLogger) { + StatsLogger statsLogger, + BookieAddressResolver bookieAddressResolver) { + this.bookieAddressResolver = bookieAddressResolver; DNSToSwitchMapping dnsResolver; if (optionalDnsResolver.isPresent()) { dnsResolver = optionalDnsResolver.get(); @@ -279,9 +284,9 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, } slowBookies = CacheBuilder.newBuilder() .expireAfterWrite(conf.getBookieFailureHistoryExpirationMSec(), TimeUnit.MILLISECONDS) - .build(new CacheLoader() { + .build(new CacheLoader() { @Override - public Long load(BookieSocketAddress key) throws Exception { + public Long load(BookieId key) throws Exception { return -1L; } }); @@ -296,7 +301,8 @@ public Long load(BookieSocketAddress key) throws Exception { conf.getMinNumRacksPerWriteQuorum(), conf.getEnforceMinNumRacksPerWriteQuorum(), conf.getIgnoreLocalNodeInPlacementPolicy(), - statsLogger); + statsLogger, + bookieAddressResolver); } @Override @@ -307,16 +313,16 @@ public void uninitalize() { /* * this method should be called in readlock scope of 'rwLock' */ - protected Set addDefaultRackBookiesIfMinNumRacksIsEnforced( - Set excludeBookies) { - Set comprehensiveExclusionBookiesSet; + protected Set addDefaultRackBookiesIfMinNumRacksIsEnforced( + Set excludeBookies) { + Set comprehensiveExclusionBookiesSet; if (enforceMinNumRacksPerWriteQuorum) { - Set bookiesInDefaultRack = null; + Set bookiesInDefaultRack = null; Set defaultRackLeaves = topology.getLeaves(getDefaultRack()); for (Node node : defaultRackLeaves) { if (node instanceof BookieNode) { if (bookiesInDefaultRack == null) { - bookiesInDefaultRack = new HashSet(excludeBookies); + bookiesInDefaultRack = new HashSet(excludeBookies); } bookiesInDefaultRack.add(((BookieNode) node).getAddr()); } else { @@ -326,7 +332,7 @@ protected Set addDefaultRackBookiesIfMinNumRacksIsEnforced( if ((bookiesInDefaultRack == null) || bookiesInDefaultRack.isEmpty()) { comprehensiveExclusionBookiesSet = excludeBookies; } else { - comprehensiveExclusionBookiesSet = new HashSet(excludeBookies); + comprehensiveExclusionBookiesSet = new HashSet(excludeBookies); comprehensiveExclusionBookiesSet.addAll(bookiesInDefaultRack); LOG.info("enforceMinNumRacksPerWriteQuorum is enabled, so Excluding bookies of defaultRack: {}", bookiesInDefaultRack); @@ -338,14 +344,14 @@ protected Set addDefaultRackBookiesIfMinNumRacksIsEnforced( } @Override - public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map customMetadata, Set excludeBookies) + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { rwLock.readLock().lock(); try { - Set comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced( + Set comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced( excludeBookies); - PlacementResult> newEnsembleResult = newEnsembleInternal(ensembleSize, + PlacementResult> newEnsembleResult = newEnsembleInternal(ensembleSize, writeQuorumSize, ackQuorumSize, comprehensiveExclusionBookiesSet, null, null); return newEnsembleResult; } finally { @@ -354,10 +360,10 @@ public PlacementResult> newEnsemble(int ensembleSize, } @Override - public PlacementResult> newEnsemble(int ensembleSize, + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies, + Set excludeBookies, Ensemble parentEnsemble, Predicate parentPredicate) throws BKNotEnoughBookiesException { @@ -370,11 +376,11 @@ public PlacementResult> newEnsemble(int ensembleSize, parentPredicate); } - protected PlacementResult> newEnsembleInternal( + protected PlacementResult> newEnsembleInternal( int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies, + Set excludeBookies, Ensemble parentEnsemble, Predicate parentPredicate) throws BKNotEnoughBookiesException { rwLock.readLock().lock(); @@ -400,7 +406,7 @@ protected PlacementResult> newEnsembleInternal( } List bns = selectRandom(ensembleSize, excludeNodes, TruePredicate.INSTANCE, ensemble); - ArrayList addrs = new ArrayList(ensembleSize); + ArrayList addrs = new ArrayList(ensembleSize); for (BookieNode bn : bns) { addrs.add(bn.getAddr()); } @@ -422,7 +428,7 @@ protected PlacementResult> newEnsembleInternal( prevNode = selectFromNetworkLocation(curRack, excludeNodes, ensemble, ensemble, !enforceMinNumRacksPerWriteQuorum || firstBookieInTheEnsemble); } - List bookieList = ensemble.toList(); + List bookieList = ensemble.toList(); if (ensembleSize != bookieList.size()) { LOG.error("Not enough {} bookies are available to form an ensemble : {}.", ensembleSize, bookieList); @@ -437,9 +443,9 @@ protected PlacementResult> newEnsembleInternal( } @Override - public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, List currentEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies) + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, + BookieId bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { rwLock.readLock().lock(); try { @@ -474,8 +480,8 @@ public PlacementResult replaceBookie(int ensembleSize, int if (LOG.isDebugEnabled()) { LOG.debug("Bookie {} is chosen to replace bookie {}.", candidate, bn); } - BookieSocketAddress candidateAddr = candidate.getAddr(); - List newEnsemble = new ArrayList(currentEnsemble); + BookieId candidateAddr = candidate.getAddr(); + List newEnsemble = new ArrayList(currentEnsemble); if (currentEnsemble.isEmpty()) { /* * in testing code there are test cases which would pass empty @@ -773,7 +779,7 @@ protected List selectRandomInternal(List bookiesToSelect } @Override - public void registerSlowBookie(BookieSocketAddress bookieSocketAddress, long entryId) { + public void registerSlowBookie(BookieId bookieSocketAddress, long entryId) { if (reorderThresholdPendingRequests <= 0) { // only put bookies on slowBookies list if reorderThresholdPendingRequests is *not* set (0); // otherwise, rely on reordering of reads based on reorderThresholdPendingRequests @@ -783,7 +789,7 @@ public void registerSlowBookie(BookieSocketAddress bookieSocketAddress, long ent @Override public DistributionSchedule.WriteSet reorderReadSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { Map writeSetWithRegion = new HashMap<>(); @@ -824,7 +830,7 @@ public DistributionSchedule.WriteSet reorderReadSequence( * @return ordering of bookies to send read to */ DistributionSchedule.WriteSet reorderReadSequenceWithRegion( - List ensemble, + List ensemble, DistributionSchedule.WriteSet writeSet, Map writeSetWithRegion, BookiesHealthInfo bookiesHealthInfo, @@ -842,7 +848,7 @@ DistributionSchedule.WriteSet reorderReadSequenceWithRegion( isAnyBookieUnavailable = true; } else { for (int i = 0; i < ensemble.size(); i++) { - BookieSocketAddress bookieAddr = ensemble.get(i); + BookieId bookieAddr = ensemble.get(i); if ((!knownBookies.containsKey(bookieAddr) && !readOnlyBookies.contains(bookieAddr)) || slowBookies.getIfPresent(bookieAddr) != null) { // Found at least one bookie not available in the ensemble, or in slowBookies @@ -896,7 +902,7 @@ DistributionSchedule.WriteSet reorderReadSequenceWithRegion( for (int i = 0; i < writeSet.size(); i++) { int idx = writeSet.get(i); - BookieSocketAddress address = ensemble.get(idx); + BookieId address = ensemble.get(idx); String region = writeSetWithRegion.get(idx); Long lastFailedEntryOnBookie = bookiesHealthInfo.getBookieFailureHistory(address); if (null == knownBookies.get(address)) { @@ -1007,12 +1013,12 @@ DistributionSchedule.WriteSet reorderReadSequenceWithRegion( // this method should be called in readlock scope of 'rwlock' @Override - public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, int ackQuorumSize) { int ensembleSize = ensembleList.size(); int minNumRacksPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, minNumRacksPerWriteQuorum); HashSet racksInQuorum = new HashSet(); - BookieSocketAddress bookie; + BookieId bookie; for (int i = 0; i < ensembleList.size(); i++) { racksInQuorum.clear(); for (int j = 0; j < writeQuorumSize; j++) { @@ -1037,7 +1043,7 @@ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ackedBookies, + public boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, int writeQuorumSize, int ackQuorumSize) { HashSet rackCounter = new HashSet<>(); @@ -1046,7 +1052,7 @@ public boolean areAckedBookiesAdheringToPlacementPolicy(Set ReentrantReadWriteLock.ReadLock readLock = rwLock.readLock(); readLock.lock(); try { - for (BookieSocketAddress bookie : ackedBookies) { + for (BookieId bookie : ackedBookies) { rackCounter.add(knownBookies.get(bookie).getNetworkLocation()); } 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 3dc1a75fc93..871626a9d2a 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 @@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.proto.ReadLastConfirmedAndEntryContext; @@ -65,7 +65,7 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt private final long prevEntryId; private long lastAddConfirmed; private long timeOutInMillis; - private final List currentEnsemble; + private final List currentEnsemble; private ScheduledFuture speculativeTask = null; abstract class ReadLACAndEntryRequest implements AutoCloseable { @@ -76,12 +76,12 @@ abstract class ReadLACAndEntryRequest implements AutoCloseable { int firstError = BKException.Code.OK; int numMissedEntryReads = 0; - final List ensemble; + final List ensemble; final DistributionSchedule.WriteSet writeSet; final DistributionSchedule.WriteSet orderedEnsemble; final LedgerEntryImpl entryImpl; - ReadLACAndEntryRequest(List ensemble, long lId, long eId) { + ReadLACAndEntryRequest(List ensemble, long lId, long eId) { this.entryImpl = LedgerEntryImpl.create(lId, eId); this.ensemble = ensemble; this.writeSet = lh.getDistributionSchedule().getEnsembleSet(eId); @@ -119,7 +119,7 @@ synchronized int getFirstError() { * @return return true if we managed to complete the entry; * otherwise return false if the read entry is not complete or it is already completed before */ - boolean complete(int bookieIndex, BookieSocketAddress host, final ByteBuf buffer, long entryId) { + boolean complete(int bookieIndex, BookieId host, final ByteBuf buffer, long entryId) { ByteBuf content; try { content = lh.getDigestManager().verifyDigestAndReturnData(entryId, buffer); @@ -190,7 +190,7 @@ private synchronized void translateAndSetFirstError(int rc) { * @param rc * read result code */ - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { translateAndSetFirstError(rc); if (BKException.Code.NoSuchEntryException == rc || BKException.Code.NoSuchLedgerExistsException == rc) { @@ -217,7 +217,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress * the set of hosts that we already received responses. * @return host we sent to if we sent. null otherwise. */ - abstract BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet); + abstract BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet); /** * Whether the read request completed. @@ -247,7 +247,7 @@ class ParallelReadRequest extends ReadLACAndEntryRequest { int numPendings; - ParallelReadRequest(List ensemble, long lId, long eId) { + ParallelReadRequest(List ensemble, long lId, long eId) { super(ensemble, lId, eId); numPendings = orderedEnsemble.size(); } @@ -255,7 +255,7 @@ class ParallelReadRequest extends ReadLACAndEntryRequest { @Override void read() { for (int i = 0; i < orderedEnsemble.size(); i++) { - BookieSocketAddress to = ensemble.get(orderedEnsemble.get(i)); + BookieId to = ensemble.get(orderedEnsemble.get(i)); try { sendReadTo(orderedEnsemble.get(i), to, this); } catch (InterruptedException ie) { @@ -268,7 +268,7 @@ void read() { } @Override - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); --numPendings; // if received all responses or this entry doesn't meet quorum write, complete the request. @@ -283,7 +283,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress } @Override - BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) { + BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) { // no speculative read return null; } @@ -297,7 +297,7 @@ class SequenceReadRequest extends ReadLACAndEntryRequest { final BitSet erroredReplicas; final BitSet emptyResponseReplicas; - SequenceReadRequest(List ensemble, long lId, long eId) { + SequenceReadRequest(List ensemble, long lId, long eId) { super(ensemble, lId, eId); this.sentReplicas = new BitSet(orderedEnsemble.size()); @@ -335,7 +335,7 @@ private boolean readsOutstanding() { * @return host we sent to if we sent. null otherwise. */ @Override - synchronized BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFrom) { + synchronized BookieId maybeSendSpeculativeRead(BitSet heardFrom) { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getEnsembleSize()) { return null; } @@ -357,7 +357,7 @@ void read() { sendNextRead(); } - synchronized BookieSocketAddress sendNextRead() { + synchronized BookieId sendNextRead() { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getEnsembleSize()) { // we are done, the read has failed from all replicas, just fail the // read @@ -378,7 +378,7 @@ synchronized BookieSocketAddress sendNextRead() { nextReplicaIndexToReadFrom++; try { - BookieSocketAddress to = ensemble.get(bookieIndex); + BookieId to = ensemble.get(bookieIndex); sendReadTo(bookieIndex, to, this); sentReplicas.set(replica); return to; @@ -391,7 +391,7 @@ synchronized BookieSocketAddress sendNextRead() { } @Override - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); int replica = getReplicaIndex(bookieIndex); @@ -412,7 +412,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress } @Override - boolean complete(int bookieIndex, BookieSocketAddress host, ByteBuf buffer, long entryId) { + boolean complete(int bookieIndex, BookieId host, ByteBuf buffer, long entryId) { boolean completed = super.complete(bookieIndex, host, buffer, entryId); if (completed) { int numReplicasTried = getNextReplicaIndexToReadFrom(); @@ -420,7 +420,7 @@ boolean complete(int bookieIndex, BookieSocketAddress host, ByteBuf buffer, long // first speculative read as slow for (int i = 0; i < numReplicasTried; i++) { int slowBookieIndex = orderedEnsemble.get(i); - BookieSocketAddress slowBookieSocketAddress = ensemble.get(slowBookieIndex); + BookieId slowBookieSocketAddress = ensemble.get(slowBookieIndex); clientCtx.getPlacementPolicy().registerSlowBookie(slowBookieSocketAddress, entryId); } } @@ -430,7 +430,7 @@ boolean complete(int bookieIndex, BookieSocketAddress host, ByteBuf buffer, long ReadLastConfirmedAndEntryOp(LedgerHandle lh, ClientContext clientCtx, - List ensemble, + List ensemble, LastConfirmedAndEntryCallback cb, long prevEntryId, long timeOutInMillis) { @@ -506,7 +506,7 @@ public void initiate() { } } - void sendReadTo(int bookieIndex, BookieSocketAddress to, ReadLACAndEntryRequest entry) throws InterruptedException { + void sendReadTo(int bookieIndex, BookieId to, ReadLACAndEntryRequest entry) throws InterruptedException { if (LOG.isDebugEnabled()) { LOG.debug("Calling Read LAC and Entry with {} and long polling interval {} on Bookie {} - Parallel {}", prevEntryId, timeOutInMillis, to, parallelRead); @@ -557,7 +557,7 @@ public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf buffe getClass().getName(), ledgerId, entryId, rc); } ReadLastConfirmedAndEntryContext rCtx = (ReadLastConfirmedAndEntryContext) ctx; - BookieSocketAddress bookie = rCtx.getBookieAddress(); + BookieId bookie = rCtx.getBookieAddress(); numResponsesPending--; if (BKException.Code.OK == rc) { if (LOG.isTraceEnabled()) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java index 4e3b1277bfa..e5d441a37f6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java @@ -23,7 +23,7 @@ import java.util.List; import org.apache.bookkeeper.client.BKException.BKDigestMatchException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; @@ -49,7 +49,7 @@ class ReadLastConfirmedOp implements ReadEntryCallback { private final LastConfirmedDataCallback cb; private final DistributionSchedule.QuorumCoverageSet coverageSet; - private final List currentEnsemble; + private final List currentEnsemble; /** * Wrapper to get all recovered data from the request. @@ -62,7 +62,7 @@ public ReadLastConfirmedOp(BookieClient bookieClient, DistributionSchedule schedule, DigestManager digestManager, long ledgerId, - List ensemble, + List ensemble, byte[] ledgerKey, LastConfirmedDataCallback cb) { this.cb = cb; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java index 72930f7acee..1d21fc9377e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java @@ -39,7 +39,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener; @@ -61,7 +61,7 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListene private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyLedgerHandle.class); private Object metadataLock = new Object(); - private final NavigableMap> newEnsemblesFromRecovery = new TreeMap<>(); + private final NavigableMap> newEnsemblesFromRecovery = new TreeMap<>(); class MetadataUpdater extends SafeRunnable { @@ -212,7 +212,7 @@ public void readLastConfirmedComplete(int rc, long lastConfirmed, Object ctx) { * on the close. */ @Override - void handleBookieFailure(final Map failedBookies) { + void handleBookieFailure(final Map failedBookies) { // handleBookieFailure should always run in the ordered executor thread for this // ledger, so this synchronized should be unnecessary, but putting it here now // just in case (can be removed when we validate threads) @@ -221,9 +221,9 @@ void handleBookieFailure(final Map failedBookies) long lac = getLastAddConfirmed(); LedgerMetadata metadata = getLedgerMetadata(); - List currentEnsemble = getCurrentEnsemble(); + List currentEnsemble = getCurrentEnsemble(); try { - List newEnsemble = EnsembleUtils.replaceBookiesInEnsemble( + List newEnsemble = EnsembleUtils.replaceBookiesInEnsemble( clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext); Set replaced = EnsembleUtils.diffEnsemble(currentEnsemble, newEnsemble); if (!replaced.isEmpty()) { @@ -339,7 +339,7 @@ CompletableFuture> closeRecovered() { } @Override - List getCurrentEnsemble() { + List getCurrentEnsemble() { synchronized (metadataLock) { if (!newEnsemblesFromRecovery.isEmpty()) { return newEnsemblesFromRecovery.lastEntry().getValue(); 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 16bdcfa3862..dee4187c46e 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 @@ -32,12 +32,13 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.FeatureProvider; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; -import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.net.NodeBase; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.commons.lang3.tuple.Pair; @@ -68,7 +69,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme static final int REMOTE_NODE_IN_REORDER_SEQUENCE = 2; protected final Map perRegionPlacement; - protected final ConcurrentMap address2Region; + protected final ConcurrentMap address2Region; protected FeatureProvider featureProvider; protected String disallowBookiePlacementInRegionFeatureName; protected String myRegion = null; @@ -80,10 +81,10 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme RegionAwareEnsemblePlacementPolicy() { super(); perRegionPlacement = new HashMap(); - address2Region = new ConcurrentHashMap(); + address2Region = new ConcurrentHashMap(); } - protected String getRegion(BookieSocketAddress addr) { + protected String getRegion(BookieId addr) { String region = address2Region.get(addr); if (null == region) { String networkLocation = resolveNetworkLocation(addr); @@ -110,7 +111,7 @@ protected String getLocalRegion(BookieNode node) { } @Override - public void handleBookiesThatLeft(Set leftBookies) { + public void handleBookiesThatLeft(Set leftBookies) { super.handleBookiesThatLeft(leftBookies); for (TopologyAwareEnsemblePlacementPolicy policy: perRegionPlacement.values()) { @@ -119,11 +120,11 @@ public void handleBookiesThatLeft(Set leftBookies) { } @Override - public void handleBookiesThatJoined(Set joinedBookies) { - Map> perRegionClusterChange = new HashMap>(); + public void handleBookiesThatJoined(Set joinedBookies) { + Map> perRegionClusterChange = new HashMap>(); // node joined - for (BookieSocketAddress addr : joinedBookies) { + for (BookieId addr : joinedBookies) { BookieNode node = createBookieNode(addr); topology.add(node); knownBookies.put(addr, node); @@ -133,13 +134,13 @@ public void handleBookiesThatJoined(Set joinedBookies) { .initialize(dnsResolver, timer, this.reorderReadsRandom, this.stabilizePeriodSeconds, this.reorderThresholdPendingRequests, this.isWeighted, this.maxWeightMultiple, this.minNumRacksPerWriteQuorum, this.enforceMinNumRacksPerWriteQuorum, - this.ignoreLocalNodeInPlacementPolicy, statsLogger) + this.ignoreLocalNodeInPlacementPolicy, statsLogger, bookieAddressResolver) .withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK)); } - Set regionSet = perRegionClusterChange.get(region); + Set regionSet = perRegionClusterChange.get(region); if (null == regionSet) { - regionSet = new HashSet(); + regionSet = new HashSet(); regionSet.add(addr); perRegionClusterChange.put(region, regionSet); } else { @@ -152,9 +153,9 @@ public void handleBookiesThatJoined(Set joinedBookies) { } for (Map.Entry regionEntry : perRegionPlacement.entrySet()) { - Set regionSet = perRegionClusterChange.get(regionEntry.getKey()); + Set regionSet = perRegionClusterChange.get(regionEntry.getKey()); if (null == regionSet) { - regionSet = new HashSet(); + regionSet = new HashSet(); } regionEntry.getValue().handleBookiesThatJoined(regionSet); } @@ -165,8 +166,9 @@ public RegionAwareEnsemblePlacementPolicy initialize(ClientConfiguration conf, Optional optionalDnsResolver, HashedWheelTimer timer, FeatureProvider featureProvider, - StatsLogger statsLogger) { - super.initialize(conf, optionalDnsResolver, timer, featureProvider, statsLogger) + StatsLogger statsLogger, + BookieAddressResolver bookieAddressResolver) { + super.initialize(conf, optionalDnsResolver, timer, featureProvider, statsLogger, bookieAddressResolver) .withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); myRegion = getLocalRegion(localNode); enableValidation = conf.getBoolean(REPP_ENABLE_VALIDATION, true); @@ -183,7 +185,7 @@ public RegionAwareEnsemblePlacementPolicy initialize(ClientConfiguration conf, .initialize(dnsResolver, timer, this.reorderReadsRandom, this.stabilizePeriodSeconds, this.reorderThresholdPendingRequests, this.isWeighted, this.maxWeightMultiple, this.minNumRacksPerWriteQuorum, this.enforceMinNumRacksPerWriteQuorum, - this.ignoreLocalNodeInPlacementPolicy, statsLogger) + this.ignoreLocalNodeInPlacementPolicy, statsLogger, bookieAddressResolver) .withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK)); } minRegionsForDurability = conf.getInt(REPP_MINIMUM_REGIONS_FOR_DURABILITY, @@ -228,8 +230,8 @@ protected List selectRandomFromRegions(Set availableRegions, @Override - public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map customMetadata, Set excludedBookies) + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludedBookies) throws BKException.BKNotEnoughBookiesException { int effectiveMinRegionsForDurability = disableDurabilityFeature.isAvailable() ? 1 : minRegionsForDurability; @@ -258,7 +260,7 @@ public PlacementResult> newEnsemble(int ensembleSize, rwLock.readLock().lock(); try { - Set comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced( + Set comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced( excludedBookies); Set excludeNodes = convertBookiesToNodes(comprehensiveExclusionBookiesSet); Set availableRegions = new HashSet(); @@ -280,7 +282,7 @@ public PlacementResult> newEnsemble(int ensembleSize, } List bns = selectRandom(ensembleSize, excludeNodes, TruePredicate.INSTANCE, EnsembleForReplacementWithNoConstraints.INSTANCE); - ArrayList addrs = new ArrayList(ensembleSize); + ArrayList addrs = new ArrayList(ensembleSize); for (BookieNode bn : bns) { addrs.add(bn.getAddr()); } @@ -352,7 +354,7 @@ public PlacementResult> newEnsemble(int ensembleSize, int newEnsembleSize = currentAllocation.getLeft() + addToEnsembleSize; int newWriteQuorumSize = currentAllocation.getRight() + addToWriteQuorum; try { - List allocated = policyWithinRegion + List allocated = policyWithinRegion .newEnsemble(newEnsembleSize, newWriteQuorumSize, newWriteQuorumSize, comprehensiveExclusionBookiesSet, tempEnsemble, tempEnsemble) .getResult(); @@ -402,7 +404,7 @@ public PlacementResult> newEnsemble(int ensembleSize, } } while ((remainingEnsemble > 0) && (remainingEnsemble < remainingEnsembleBeforeIteration)); - List bookieList = ensemble.toList(); + List bookieList = ensemble.toList(); if (ensembleSize != bookieList.size()) { LOG.error("Not enough {} bookies are available to form an ensemble : {}.", ensembleSize, bookieList); @@ -415,7 +417,7 @@ public PlacementResult> newEnsemble(int ensembleSize, throw new BKException.BKNotEnoughBookiesException(); } LOG.info("Bookies allocated successfully {}", ensemble); - List ensembleList = ensemble.toList(); + List ensembleList = ensemble.toList(); return PlacementResult.of(ensembleList, isEnsembleAdheringToPlacementPolicy(ensembleList, writeQuorumSize, ackQuorumSize)); } finally { @@ -424,15 +426,15 @@ public PlacementResult> newEnsemble(int ensembleSize, } @Override - public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, List currentEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies) + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, + BookieId bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { rwLock.readLock().lock(); try { boolean enforceDurability = enforceDurabilityInReplace && !disableDurabilityFeature.isAvailable(); int effectiveMinRegionsForDurability = enforceDurability ? minRegionsForDurability : 1; - Set comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced( + Set comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced( excludeBookies); Set excludeNodes = convertBookiesToNodes(comprehensiveExclusionBookiesSet); RRTopologyAwareCoverageEnsemble ensemble = new RRTopologyAwareCoverageEnsemble(ensembleSize, @@ -448,7 +450,7 @@ public PlacementResult replaceBookie(int ensembleSize, int } excludeNodes.add(bookieNodeToReplace); - for (BookieSocketAddress bookieAddress: currentEnsemble) { + for (BookieId bookieAddress: currentEnsemble) { if (bookieAddress.equals(bookieToReplace)) { continue; } @@ -481,8 +483,8 @@ public PlacementResult replaceBookie(int ensembleSize, int if (LOG.isDebugEnabled()) { LOG.debug("Bookie {} is chosen to replace bookie {}.", candidate, bookieNodeToReplace); } - BookieSocketAddress candidateAddr = candidate.getAddr(); - List newEnsemble = new ArrayList(currentEnsemble); + BookieId candidateAddr = candidate.getAddr(); + List newEnsemble = new ArrayList(currentEnsemble); if (currentEnsemble.isEmpty()) { /* * in testing code there are test cases which would pass empty @@ -546,7 +548,7 @@ protected BookieNode replaceFromRack(BookieNode bookieNodeToReplace, @Override public final DistributionSchedule.WriteSet reorderReadSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { if (UNKNOWN_REGION.equals(myRegion)) { @@ -564,7 +566,7 @@ public final DistributionSchedule.WriteSet reorderReadSequence( @Override public final DistributionSchedule.WriteSet reorderReadLACSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { if (UNKNOWN_REGION.equals(myRegion)) { @@ -576,7 +578,7 @@ public final DistributionSchedule.WriteSet reorderReadLACSequence( } @Override - public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, int ackQuorumSize) { /** * TODO: have to implement actual logic for this method for 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 85615b84a0a..f092839f318 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 @@ -28,7 +28,7 @@ import java.util.BitSet; import java.util.Map; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -266,7 +266,7 @@ private static class AckSetImpl implements AckSet { private int ackQuorumSize; private final BitSet ackSet = new BitSet(); // grows on reset() - private BookieSocketAddress[] failureMap = new BookieSocketAddress[0]; + private BookieId[] failureMap = new BookieId[0]; private final Handle recyclerHandle; private static final Recycler RECYCLER = new Recycler() { @@ -295,7 +295,7 @@ private void reset(int ensembleSize, this.writeQuorumSize = writeQuorumSize; ackSet.clear(); if (failureMap.length < ensembleSize) { - failureMap = new BookieSocketAddress[ensembleSize]; + failureMap = new BookieId[ensembleSize]; } Arrays.fill(failureMap, null); } @@ -309,15 +309,15 @@ public boolean completeBookieAndCheck(int bookieIndexHeardFrom) { @Override public boolean failBookieAndCheck(int bookieIndexHeardFrom, - BookieSocketAddress address) { + BookieId address) { ackSet.clear(bookieIndexHeardFrom); failureMap[bookieIndexHeardFrom] = address; return failed() > (writeQuorumSize - ackQuorumSize); } @Override - public Map getFailedBookies() { - ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); + 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]); 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 2e238487814..7167c96b6c7 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 @@ -41,6 +41,7 @@ import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; @@ -48,6 +49,7 @@ import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.net.NodeBase; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; @@ -58,16 +60,17 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements ITopologyAwareEnsemblePlacementPolicy { static final Logger LOG = LoggerFactory.getLogger(TopologyAwareEnsemblePlacementPolicy.class); public static final String REPP_DNS_RESOLVER_CLASS = "reppDnsResolverClass"; - protected final Map knownBookies = new HashMap(); + protected final Map knownBookies = new HashMap(); protected final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); protected Map bookieInfoMap = new HashMap(); // Initialize to empty set - protected ImmutableSet readOnlyBookies = ImmutableSet.of(); + protected ImmutableSet readOnlyBookies = ImmutableSet.of(); boolean isWeighted; protected WeightedRandomSelection weightedSelection; // for now, we just maintain the writable bookies' topology protected NetworkTopology topology; protected DNSToSwitchMapping dnsResolver; + protected BookieAddressResolver bookieAddressResolver; @StatsDoc( name = BOOKIES_JOINED, help = "The distribution of number of bookies joined the cluster on each network topology change" @@ -92,7 +95,7 @@ protected static class EnsembleForReplacementWithNoConstraints implements Ensemb public static final EnsembleForReplacementWithNoConstraints INSTANCE = new EnsembleForReplacementWithNoConstraints(); - static final List EMPTY_LIST = new ArrayList(0); + static final List EMPTY_LIST = new ArrayList(0); @Override public boolean addNode(BookieNode node) { @@ -101,7 +104,7 @@ public boolean addNode(BookieNode node) { } @Override - public List toList() { + public List toList() { return EMPTY_LIST; } @@ -458,8 +461,8 @@ public boolean addNode(BookieNode node) { } @Override - public List toList() { - ArrayList addresses = new ArrayList(ensembleSize); + public List toList() { + ArrayList addresses = new ArrayList(ensembleSize); for (BookieNode bn : chosenNodes) { addresses.add(bn.getAddr()); } @@ -473,7 +476,7 @@ public List toList() { */ @Override public boolean validate() { - HashSet addresses = new HashSet(ensembleSize); + HashSet addresses = new HashSet(ensembleSize); HashSet racksOrRegions = new HashSet(); for (BookieNode bn : chosenNodes) { if (addresses.contains(bn.getAddr())) { @@ -619,7 +622,7 @@ static void shuffleWithMask(DistributionSchedule.WriteSet writeSet, @Override public DistributionSchedule.WriteSet reorderReadSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { return writeSet; @@ -627,7 +630,7 @@ public DistributionSchedule.WriteSet reorderReadSequence( @Override public DistributionSchedule.WriteSet reorderReadLACSequence( - List ensemble, + List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { DistributionSchedule.WriteSet retList = reorderReadSequence( @@ -637,12 +640,12 @@ public DistributionSchedule.WriteSet reorderReadLACSequence( } @Override - public Set onClusterChanged(Set writableBookies, - Set readOnlyBookies) { + public Set onClusterChanged(Set writableBookies, + Set readOnlyBookies) { rwLock.writeLock().lock(); try { - ImmutableSet joinedBookies, leftBookies, deadBookies; - Set oldBookieSet = knownBookies.keySet(); + ImmutableSet joinedBookies, leftBookies, deadBookies; + Set oldBookieSet = knownBookies.keySet(); // left bookies : bookies in known bookies, but not in new writable bookie cluster. leftBookies = Sets.difference(oldBookieSet, writableBookies).immutableCopy(); // joined bookies : bookies in new writable bookie cluster, but not in known bookies @@ -670,8 +673,8 @@ public Set onClusterChanged(Set writab * this method should be called in writelock scope of 'rwLock' */ @Override - public void handleBookiesThatLeft(Set leftBookies) { - for (BookieSocketAddress addr : leftBookies) { + public void handleBookiesThatLeft(Set leftBookies) { + for (BookieId addr : leftBookies) { try { BookieNode node = knownBookies.remove(addr); if (null != node) { @@ -701,9 +704,9 @@ public void handleBookiesThatLeft(Set leftBookies) { * this method should be called in writelock scope of 'rwLock' */ @Override - public void handleBookiesThatJoined(Set joinedBookies) { + public void handleBookiesThatJoined(Set joinedBookies) { // node joined - for (BookieSocketAddress addr : joinedBookies) { + for (BookieId addr : joinedBookies) { try { BookieNode node = createBookieNode(addr); topology.add(node); @@ -729,15 +732,15 @@ public void handleBookiesThatJoined(Set joinedBookies) { } @Override - public void onBookieRackChange(List bookieAddressList) { + public void onBookieRackChange(List bookieAddressList) { rwLock.writeLock().lock(); try { - for (BookieSocketAddress bookieAddress : bookieAddressList) { + for (BookieId bookieAddress : bookieAddressList) { BookieNode node = knownBookies.get(bookieAddress); if (node != null) { // refresh the rack info if its a known bookie - topology.remove(node); BookieNode newNode = createBookieNode(bookieAddress); + topology.remove(node); topology.add(newNode); knownBookies.put(bookieAddress, newNode); } @@ -748,7 +751,7 @@ public void onBookieRackChange(List bookieAddressList) { } @Override - public void updateBookieInfo(Map bookieInfoMap) { + public void updateBookieInfo(Map bookieInfoMap) { if (!isWeighted) { LOG.info("bookieFreeDiskInfo callback called even without weighted placement policy being used."); return; @@ -772,17 +775,28 @@ public void updateBookieInfo(Map bookieInfoMap) } } - protected BookieNode createBookieNode(BookieSocketAddress addr) { + protected BookieNode createBookieNode(BookieId addr) { return new BookieNode(addr, resolveNetworkLocation(addr)); } - protected String resolveNetworkLocation(BookieSocketAddress addr) { - return NetUtils.resolveNetworkLocation(dnsResolver, addr); + protected BookieNode createDummyLocalBookieNode(String hostname) { + return new BookieNode(BookieSocketAddress.createDummyBookieIdForHostname(hostname), + NetUtils.resolveNetworkLocation(dnsResolver, new BookieSocketAddress(hostname, 0))); + } + + protected String resolveNetworkLocation(BookieId addr) { + try { + return NetUtils.resolveNetworkLocation(dnsResolver, bookieAddressResolver.resolve(addr)); + } catch (BookieAddressResolver.BookieIdNotResolvedException err) { + LOG.error("Cannot resolve bookieId {} to a network address, resolving as {}", addr, + NetworkTopology.DEFAULT_REGION_AND_RACK, err); + return NetworkTopology.DEFAULT_REGION_AND_RACK; + } } - protected Set convertBookiesToNodes(Collection excludeBookies) { + protected Set convertBookiesToNodes(Collection excludeBookies) { Set nodes = new HashSet(); - for (BookieSocketAddress addr : excludeBookies) { + for (BookieId addr : excludeBookies) { BookieNode bn = knownBookies.get(addr); if (null == bn) { bn = createBookieNode(addr); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java index 1311d316c86..ad095212b10 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java @@ -21,7 +21,7 @@ import java.util.List; import org.apache.bookkeeper.client.ReadLastConfirmedOp.LastConfirmedDataCallback; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; @@ -45,10 +45,10 @@ class TryReadLastConfirmedOp implements ReadEntryCallback { volatile boolean hasValidResponse = false; volatile boolean completed = false; RecoveryData maxRecoveredData; - final List currentEnsemble; + final List currentEnsemble; TryReadLastConfirmedOp(LedgerHandle lh, BookieClient bookieClient, - List ensemble, LastConfirmedDataCallback cb, long lac) { + List ensemble, LastConfirmedDataCallback cb, long lac) { this.lh = lh; this.bookieClient = bookieClient; this.cb = cb; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java index 14951fdd66d..1444277548d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java @@ -38,7 +38,7 @@ import org.apache.bookkeeper.bookie.BookieShell.UpdateLedgerNotifier; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,7 +75,7 @@ public UpdateLedgerOp(final BookKeeper bkc, final BookKeeperAdmin admin) { * if there is an error when updating bookie id in ledger * metadata */ - public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final BookieSocketAddress newBookieId, + public void updateBookieIdInLedgers(final BookieId oldBookieId, final BookieId newBookieId, final int rate, int maxOutstandingReads, final int limit, final UpdateLedgerNotifier progressable) throws IOException, InterruptedException { @@ -161,11 +161,11 @@ public void updateBookieIdInLedgers(final BookieSocketAddress oldBookieId, final } private static LedgerMetadata replaceBookieInEnsembles(LedgerMetadata metadata, - BookieSocketAddress oldBookieId, - BookieSocketAddress newBookieId) { + BookieId oldBookieId, + BookieId newBookieId) { LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata); - for (Map.Entry> e : metadata.getAllEnsembles().entrySet()) { - List newEnsemble = e.getValue().stream() + for (Map.Entry> e : metadata.getAllEnsembles().entrySet()) { + List newEnsemble = e.getValue().stream() .map(b -> b.equals(oldBookieId) ? newBookieId : b) .collect(Collectors.toList()); builder.replaceEnsembleEntry(e.getKey(), newEnsemble); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicy.java index 265b0ca6441..9554b4104b5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicy.java @@ -26,9 +26,10 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; -import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.StatsLogger; /** @@ -46,16 +47,18 @@ public ZoneawareEnsemblePlacementPolicy() { @Override public EnsemblePlacementPolicy initialize(ClientConfiguration conf, - Optional optionalDnsResolver, HashedWheelTimer timer, FeatureProvider featureProvider, - StatsLogger statsLogger) { + Optional optionalDnsResolver, HashedWheelTimer timer, + FeatureProvider featureProvider, + StatsLogger statsLogger, BookieAddressResolver bookieAddressResolver) { if (conf.getNetworkTopologyStabilizePeriodSeconds() > 0) { ClientConfiguration confClone = new ClientConfiguration(conf); confClone.setNetworkTopologyStabilizePeriodSeconds(0); - super.initialize(confClone, optionalDnsResolver, timer, featureProvider, statsLogger); + super.initialize(confClone, optionalDnsResolver, timer, featureProvider, + statsLogger, bookieAddressResolver); slave = new ZoneawareEnsemblePlacementPolicyImpl(); - slave.initialize(conf, optionalDnsResolver, timer, featureProvider, statsLogger); + slave.initialize(conf, optionalDnsResolver, timer, featureProvider, statsLogger, bookieAddressResolver); } else { - super.initialize(conf, optionalDnsResolver, timer, featureProvider, statsLogger); + super.initialize(conf, optionalDnsResolver, timer, featureProvider, statsLogger, bookieAddressResolver); slave = null; } return this; @@ -70,9 +73,9 @@ public void uninitalize() { } @Override - public Set onClusterChanged(Set writableBookies, - Set readOnlyBookies) { - Set deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies); + public Set onClusterChanged(Set writableBookies, + Set readOnlyBookies) { + Set deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies); if (null != slave) { deadBookies = slave.onClusterChanged(writableBookies, readOnlyBookies); } @@ -80,8 +83,8 @@ public Set onClusterChanged(Set writab } @Override - public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map customMetadata, Set excludeBookies) + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { try { return super.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, excludeBookies); @@ -95,9 +98,9 @@ public PlacementResult> newEnsemble(int ensembleSize, } @Override - public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, List currentEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies) + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, + BookieId bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { try { return super.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, @@ -113,7 +116,7 @@ public PlacementResult replaceBookie(int ensembleSize, int } @Override - public void handleBookiesThatLeft(Set leftBookies) { + public void handleBookiesThatLeft(Set leftBookies) { super.handleBookiesThatLeft(leftBookies); if (null != slave) { slave.handleBookiesThatLeft(leftBookies); @@ -121,7 +124,7 @@ public void handleBookiesThatLeft(Set leftBookies) { } @Override - public void handleBookiesThatJoined(Set joinedBookies) { + public void handleBookiesThatJoined(Set joinedBookies) { super.handleBookiesThatJoined(joinedBookies); if (null != slave) { slave.handleBookiesThatJoined(joinedBookies); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java index dc15c9b2471..fcb3c7396dc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java @@ -29,8 +29,8 @@ import io.netty.util.HashedWheelTimer; +import java.io.IOException; import java.net.InetAddress; -import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -52,8 +52,8 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.Configurable; import org.apache.bookkeeper.feature.FeatureProvider; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; -import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.net.NetworkTopologyImpl; @@ -61,6 +61,7 @@ import org.apache.bookkeeper.net.NodeBase; import org.apache.bookkeeper.net.ScriptBasedMapping; import org.apache.bookkeeper.net.StabilizeNetworkTopology; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.StatsLogger; @@ -90,7 +91,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP private final Random rand; protected StatsLogger statsLogger = null; // Use a loading cache so slow bookies are expired. Use entryId as values. - protected Cache slowBookies; + protected Cache slowBookies; protected BookieNode myNode = null; protected String myZone = null; protected boolean reorderReadsRandom = false; @@ -101,7 +102,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP protected int desiredNumZonesPerWriteQuorum; protected boolean enforceStrictZoneawarePlacement; protected HashedWheelTimer timer; - protected final ConcurrentMap address2NodePlacement; + protected final ConcurrentMap address2NodePlacement; @StatsDoc(name = FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER, help = "Counter for number of times" + " DNSResolverDecorator failed to resolve Network Location") @@ -147,11 +148,11 @@ public boolean equals(Object obj) { ZoneawareEnsemblePlacementPolicyImpl() { super(); - address2NodePlacement = new ConcurrentHashMap(); + address2NodePlacement = new ConcurrentHashMap(); rand = new Random(System.currentTimeMillis()); } - protected ZoneAwareNodeLocation getZoneAwareNodeLocation(BookieSocketAddress addr) { + protected ZoneAwareNodeLocation getZoneAwareNodeLocation(BookieId addr) { ZoneAwareNodeLocation nodeLocation = address2NodePlacement.get(addr); if (null == nodeLocation) { String networkLocation = resolveNetworkLocation(addr); @@ -181,8 +182,9 @@ protected ZoneAwareNodeLocation getZoneAwareNodeLocation(BookieNode node) { @Override public EnsemblePlacementPolicy initialize(ClientConfiguration conf, Optional optionalDnsResolver, HashedWheelTimer timer, FeatureProvider featureProvider, - StatsLogger statsLogger) { + StatsLogger statsLogger, BookieAddressResolver bookieAddressResolver) { this.statsLogger = statsLogger; + this.bookieAddressResolver = bookieAddressResolver; this.timer = timer; this.bookiesJoinedCounter = statsLogger.getOpStatsLogger(BOOKIES_JOINED); this.bookiesLeftCounter = statsLogger.getOpStatsLogger(BOOKIES_LEFT); @@ -246,9 +248,9 @@ public Integer getSample() { this.topology = new NetworkTopologyImpl(); } try { - myNode = createBookieNode(new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), 0)); + myNode = createDummyLocalBookieNode(InetAddress.getLocalHost().getHostAddress()); myZone = getZoneAwareNodeLocation(myNode).getZone(); - } catch (UnknownHostException e) { + } catch (IOException e) { LOG.error("Failed to get local host address : ", e); throw new RuntimeException(e); } @@ -257,9 +259,9 @@ public Integer getSample() { slowBookies = CacheBuilder.newBuilder() .expireAfterWrite(conf.getBookieFailureHistoryExpirationMSec(), TimeUnit.MILLISECONDS) - .build(new CacheLoader() { + .build(new CacheLoader() { @Override - public Long load(BookieSocketAddress key) throws Exception { + public Long load(BookieId key) throws Exception { return -1L; } }); @@ -287,8 +289,8 @@ public String getDefaultFaultDomain() { } @Override - public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Set excludeBookies, + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Set excludeBookies, org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble parentEnsemble, org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate parentPredicate) throws BKNotEnoughBookiesException { @@ -329,8 +331,8 @@ public void uninitalize() { } @Override - public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map customMetadata, Set excludeBookies) + public PlacementResult> newEnsemble(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKNotEnoughBookiesException { if (enforceStrictZoneawarePlacement) { if (ensembleSize % writeQuorumSize != 0) { @@ -387,7 +389,7 @@ public PlacementResult> newEnsemble(int ensembleSize, } } int desiredNumZonesPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, desiredNumZonesPerWriteQuorum); - List newEnsemble = new ArrayList( + List newEnsemble = new ArrayList( Collections.nCopies(ensembleSize, null)); rwLock.readLock().lock(); try { @@ -395,9 +397,9 @@ public PlacementResult> newEnsemble(int ensembleSize, return createNewEnsembleRandomly(newEnsemble, writeQuorumSize, ackQuorumSize, customMetadata, excludeBookies); } - Set comprehensiveExclusionBookiesSet = addDefaultFaultDomainBookies(excludeBookies); + Set comprehensiveExclusionBookiesSet = addDefaultFaultDomainBookies(excludeBookies); for (int index = 0; index < ensembleSize; index++) { - BookieSocketAddress selectedBookie = setBookieInTheEnsemble(ensembleSize, writeQuorumSize, newEnsemble, + BookieId selectedBookie = setBookieInTheEnsemble(ensembleSize, writeQuorumSize, newEnsemble, newEnsemble, index, desiredNumZonesPerWriteQuorumForThisEnsemble, comprehensiveExclusionBookiesSet); comprehensiveExclusionBookiesSet.add(selectedBookie); @@ -410,23 +412,23 @@ public PlacementResult> newEnsemble(int ensembleSize, } @Override - public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Map customMetadata, List currentEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies) + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Map customMetadata, List currentEnsemble, + BookieId bookieToReplace, Set excludeBookies) throws BKNotEnoughBookiesException { int bookieToReplaceIndex = currentEnsemble.indexOf(bookieToReplace); int desiredNumZonesPerWriteQuorumForThisEnsemble = (writeQuorumSize < desiredNumZonesPerWriteQuorum) ? writeQuorumSize : desiredNumZonesPerWriteQuorum; - List newEnsemble = new ArrayList(currentEnsemble); + List newEnsemble = new ArrayList(currentEnsemble); rwLock.readLock().lock(); try { if (!enforceStrictZoneawarePlacement) { return selectBookieRandomly(newEnsemble, bookieToReplace, excludeBookies, writeQuorumSize, ackQuorumSize); } - Set comprehensiveExclusionBookiesSet = addDefaultFaultDomainBookies(excludeBookies); + Set comprehensiveExclusionBookiesSet = addDefaultFaultDomainBookies(excludeBookies); comprehensiveExclusionBookiesSet.addAll(currentEnsemble); - BookieSocketAddress candidateAddr = setBookieInTheEnsemble(ensembleSize, writeQuorumSize, currentEnsemble, + BookieId candidateAddr = setBookieInTheEnsemble(ensembleSize, writeQuorumSize, currentEnsemble, newEnsemble, bookieToReplaceIndex, desiredNumZonesPerWriteQuorumForThisEnsemble, comprehensiveExclusionBookiesSet); return PlacementResult.of(candidateAddr, @@ -436,9 +438,9 @@ public PlacementResult replaceBookie(int ensembleSize, int } } - private PlacementResult> createNewEnsembleRandomly(List newEnsemble, + private PlacementResult> createNewEnsembleRandomly(List newEnsemble, int writeQuorumSize, int ackQuorumSize, Map customMetadata, - Set excludeBookies) throws BKNotEnoughBookiesException { + Set excludeBookies) throws BKNotEnoughBookiesException { int ensembleSize = newEnsemble.size(); Set bookiesToConsider = getBookiesToConsider(excludeBookies); if (bookiesToConsider.size() < newEnsemble.size()) { @@ -455,10 +457,10 @@ private PlacementResult> createNewEnsembleRandomly(Lis isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize)); } - private PlacementResult selectBookieRandomly(List newEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies, int writeQuorumSize, + private PlacementResult selectBookieRandomly(List newEnsemble, + BookieId bookieToReplace, Set excludeBookies, int writeQuorumSize, int ackQuorumSize) throws BKNotEnoughBookiesException { - Set bookiesToExcludeIncludingEnsemble = new HashSet(excludeBookies); + Set bookiesToExcludeIncludingEnsemble = new HashSet(excludeBookies); bookiesToExcludeIncludingEnsemble.addAll(newEnsemble); Set bookiesToConsider = getBookiesToConsider(bookiesToExcludeIncludingEnsemble); int bookieToReplaceIndex = newEnsemble.indexOf(bookieToReplace); @@ -467,13 +469,13 @@ private PlacementResult selectBookieRandomly(List getBookiesToConsider(Set excludeBookies) { + private Set getBookiesToConsider(Set excludeBookies) { Set leaves = topology.getLeaves(NodeBase.ROOT); Set bookiesToConsider = new HashSet(); BookieNode bookieNode; @@ -535,11 +537,11 @@ private Set getBookiesToConsider(Set excludeBoo * a node randomly. 12) If even after Step10 there are no eligible * candidates then throw BKNotEnoughBookiesException. */ - private BookieSocketAddress setBookieInTheEnsemble(int ensembleSize, int writeQuorumSize, - List currentEnsemble, List newEnsemble, int bookieToReplaceIndex, - int desiredNumZonesPerWriteQuorumForThisEnsemble, Set excludeBookies) + private BookieId setBookieInTheEnsemble(int ensembleSize, int writeQuorumSize, + List currentEnsemble, List newEnsemble, int bookieToReplaceIndex, + int desiredNumZonesPerWriteQuorumForThisEnsemble, Set excludeBookies) throws BKNotEnoughBookiesException { - BookieSocketAddress bookieToReplace = currentEnsemble.get(bookieToReplaceIndex); + BookieId bookieToReplace = currentEnsemble.get(bookieToReplaceIndex); Set zonesToExclude = null; Set bookiesToConsiderAfterExcludingZonesAndUDs = null; for (int numberOfNeighborsToConsider = (desiredNumZonesPerWriteQuorumForThisEnsemble @@ -564,7 +566,7 @@ private BookieSocketAddress setBookieInTheEnsemble(int ensembleSize, int writeQu throw new BKNotEnoughBookiesException(); } - BookieSocketAddress candidateAddr = selectCandidateNode(bookiesToConsiderAfterExcludingZonesAndUDs).getAddr(); + BookieId candidateAddr = selectCandidateNode(bookiesToConsiderAfterExcludingZonesAndUDs).getAddr(); newEnsemble.set(bookieToReplaceIndex, candidateAddr); return candidateAddr; } @@ -574,8 +576,8 @@ private BookieSocketAddress setBookieInTheEnsemble(int ensembleSize, int writeQu * returns a new set, by adding excludedBookies and bookies in * defaultfaultdomain. */ - protected Set addDefaultFaultDomainBookies(Set excludeBookies) { - Set comprehensiveExclusionBookiesSet = new HashSet(excludeBookies); + protected Set addDefaultFaultDomainBookies(Set excludeBookies) { + Set comprehensiveExclusionBookiesSet = new HashSet(excludeBookies); Set defaultFaultDomainLeaves = topology.getLeaves(getDefaultFaultDomain()); for (Node node : defaultFaultDomainLeaves) { if (node instanceof BookieNode) { @@ -626,7 +628,7 @@ private String getExcludedZonesString(Set excludeZones) { return excludedZonesString.toString(); } - private Set getBookiesToConsider(String excludedZonesString, Set excludeBookies) { + private Set getBookiesToConsider(String excludedZonesString, Set excludeBookies) { Set bookiesToConsider = new HashSet(); Set leaves = topology.getLeaves(excludedZonesString); for (Node leaf : leaves) { @@ -652,8 +654,8 @@ private Set getBookiesToConsider(String excludedZonesString, Set getBookiesToConsiderAfterExcludingZonesAndUDs(int ensembleSize, int writeQuorumSize, - List currentEnsemble, int bookieToReplaceIndex, - Set excludeBookies, Set excludeZones) { + List currentEnsemble, int bookieToReplaceIndex, + Set excludeBookies, Set excludeZones) { Set bookiesToConsiderAfterExcludingZonesAndUDs = new HashSet(); HashMap> excludingUDsOfZonesToConsider = new HashMap>(); Set bookiesToConsiderAfterExcludingZones = getBookiesToConsider( @@ -711,7 +713,7 @@ private void updateBookiesToConsiderAfterExcludingZonesAndUDs(Set bo /* * Gets the set of zones of neighboring nodes. */ - private Set getZonesOfNeighboringNodesInEnsemble(List currentEnsemble, int indexOfNode, + private Set getZonesOfNeighboringNodesInEnsemble(List currentEnsemble, int indexOfNode, int numOfNeighboringNodes) { Set zonesOfNeighboringNodes = new HashSet(); int ensembleSize = currentEnsemble.size(); @@ -720,7 +722,7 @@ private Set getZonesOfNeighboringNodesInEnsemble(List getZonesOfNeighboringNodesInEnsemble(List getZonesToExcludeToMaintainMinZones(List currentEnsemble, int indexOfNode, + private Set getZonesToExcludeToMaintainMinZones(List currentEnsemble, int indexOfNode, int writeQuorumSize) { int ensSize = currentEnsemble.size(); Set zonesToExclude = new HashSet(); @@ -747,7 +749,7 @@ private Set getZonesToExcludeToMaintainMinZones(List getZonesOfBookies(Collection bookieNodes) { * Gets the set of upgradedomains of neighboring nodes (writeQuorumSize) * which belong to this 'zone'. */ - private Set getUpgradeDomainsOfAZoneInNeighboringNodes(List currentEnsemble, + private Set getUpgradeDomainsOfAZoneInNeighboringNodes(List currentEnsemble, int indexOfNode, int writeQuorumSize, String zone) { int ensSize = currentEnsemble.size(); Set upgradeDomainsOfAZoneInNeighboringNodes = new HashSet(); @@ -783,7 +785,7 @@ private Set getUpgradeDomainsOfAZoneInNeighboringNodes(List getUpgradeDomainsOfAZoneInNeighboringNodes(List getUDsToExcludeToMaintainMinUDsInWriteQuorums(List currentEnsemble, + private Set getUDsToExcludeToMaintainMinUDsInWriteQuorums(List currentEnsemble, int indexOfNode, int writeQuorumSize, String zone) { int ensSize = currentEnsemble.size(); Set upgradeDomainsToExclude = new HashSet(); @@ -813,7 +815,7 @@ private Set getUDsToExcludeToMaintainMinUDsInWriteQuorums(List getUDsToExcludeToMaintainMinUDsInWriteQuorums(List ensemble, + public DistributionSchedule.WriteSet reorderReadSequence(List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { return writeSet; } @Override - public DistributionSchedule.WriteSet reorderReadLACSequence(List ensemble, + public DistributionSchedule.WriteSet reorderReadLACSequence(List ensemble, BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) { DistributionSchedule.WriteSet retList = reorderReadSequence(ensemble, bookiesHealthInfo, writeSet); retList.addMissingIndices(ensemble.size()); @@ -857,14 +859,14 @@ public DistributionSchedule.WriteSet reorderReadLACSequence(List ensembleList, + public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ensembleList, int writeQuorumSize, int ackQuorumSize) { PlacementPolicyAdherence placementPolicyAdherence = PlacementPolicyAdherence.MEETS_STRICT; rwLock.readLock().lock(); try { HashMap> bookiesLocationInWriteSet = new HashMap>(); HashMap numOfBookiesInZones = new HashMap(); - BookieSocketAddress bookieNode; + BookieId bookieNode; if (ensembleList.size() % writeQuorumSize != 0) { placementPolicyAdherence = PlacementPolicyAdherence.FAIL; if (LOG.isDebugEnabled()) { @@ -966,7 +968,7 @@ private boolean validateMinUDsAreMaintained(HashMap numOfNodesI } @Override - public boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, int writeQuorumSize, + public boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, int writeQuorumSize, int ackQuorumSize) { HashSet zonesOfAckedBookies = new HashSet<>(); int minNumZonesPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, minNumZonesPerWriteQuorum); @@ -974,7 +976,7 @@ public boolean areAckedBookiesAdheringToPlacementPolicy(Set ReentrantReadWriteLock.ReadLock readLock = rwLock.readLock(); readLock.lock(); try { - for (BookieSocketAddress ackedBookie : ackedBookies) { + for (BookieId ackedBookie : ackedBookies) { zonesOfAckedBookies.add(getZoneAwareNodeLocation(ackedBookie).getZone()); } areAckedBookiesAdheringToPlacementPolicy = ((zonesOfAckedBookies diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java index 33e7da36b8b..5c703a7c430 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java @@ -24,7 +24,7 @@ import java.util.NavigableMap; import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; /** * Represents the client-side metadata of a ledger. It is immutable. @@ -125,14 +125,14 @@ public interface LedgerMetadata { * @param entryId the entry id to retrieve its ensemble information * @return the ensemble which contains the given {@code entryId}. */ - List getEnsembleAt(long entryId); + List getEnsembleAt(long entryId); /** * Returns all the ensembles of this entry. * * @return all the ensembles of this entry. */ - NavigableMap> getAllEnsembles(); + NavigableMap> getAllEnsembles(); /** * Returns the state of the metadata. 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 f7823c05c3c..96cc03572b9 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 @@ -24,13 +24,14 @@ import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Versioned; /** * A registration client, which the bookkeeper client will use to interact with registration service. */ + @LimitedPrivate @Evolving public interface RegistrationClient extends AutoCloseable { @@ -40,7 +41,7 @@ public interface RegistrationClient extends AutoCloseable { */ interface RegistrationListener { - void onBookiesChanged(Versioned> bookies); + void onBookiesChanged(Versioned> bookies); } @@ -52,34 +53,35 @@ interface RegistrationListener { * * @return a future represents the list of writable bookies. */ - CompletableFuture>> getWritableBookies(); + CompletableFuture>> getWritableBookies(); /** * Get the list of all bookies identifiers. * * @return a future represents the list of writable bookies. */ - CompletableFuture>> getAllBookies(); + CompletableFuture>> getAllBookies(); /** * Get the list of readonly bookie identifiers. * * @return a future represents the list of readonly bookies. */ - CompletableFuture>> getReadOnlyBookies(); + CompletableFuture>> getReadOnlyBookies(); /** * Get detailed information about the services exposed by a Bookie. * For old bookies it is expected to return an empty BookieServiceInfo structure. * - * @param bookieId this is the id of the bookie, it can be computed from a {@link BookieSocketAddress} + * @param bookieId this is the id of the bookie, it can be computed from a {@link BookieId} * @return a future represents the available information. * * @since 4.11 */ - default CompletableFuture> getBookieServiceInfo(String bookieId) { + default CompletableFuture> getBookieServiceInfo(BookieId bookieId) { try { - BookieServiceInfo bookieServiceInfo = BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId); + BookieServiceInfo bookieServiceInfo = BookieServiceInfoUtils + .buildLegacyBookieServiceInfo(bookieId.toString()); return FutureUtils.value(new Versioned<>(bookieServiceInfo, new LongVersion(-1))); } catch (UnknownHostException e) { return FutureUtils.exception(e); 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 index e5b0f6a4171..50688516b08 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java @@ -21,6 +21,7 @@ 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.net.BookieId; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; @@ -62,7 +63,7 @@ interface RegistrationListener { * @param serviceInfo information about services exposed by the Bookie * @throws BookieException when fail to register a bookie. */ - void registerBookie(String bookieId, boolean readOnly, BookieServiceInfo serviceInfo) throws BookieException; + void registerBookie(BookieId bookieId, boolean readOnly, BookieServiceInfo serviceInfo) throws BookieException; /** * Unregistering the bookie server as bookieId. @@ -71,7 +72,7 @@ interface RegistrationListener { * @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; + void unregisterBookie(BookieId bookieId, boolean readOnly) throws BookieException; /** * Checks if Bookie with the given BookieId is registered as readwrite or @@ -82,7 +83,7 @@ interface RegistrationListener { * readwrite or readonly bookie. * @throws BookieException */ - boolean isBookieRegistered(String bookieId) throws BookieException; + boolean isBookieRegistered(BookieId bookieId) throws BookieException; /** * Write the cookie data, which will be used for verifying the integrity of the bookie environment. @@ -91,7 +92,7 @@ interface RegistrationListener { * @param cookieData cookie data * @throws BookieException when fail to write cookie */ - void writeCookie(String bookieId, Versioned cookieData) throws BookieException; + void writeCookie(BookieId bookieId, Versioned cookieData) throws BookieException; /** * Read the cookie data, which will be used for verifying the integrity of the bookie environment. @@ -100,7 +101,7 @@ interface RegistrationListener { * @return versioned cookie data * @throws BookieException when fail to read cookie */ - Versioned readCookie(String bookieId) throws BookieException; + Versioned readCookie(BookieId bookieId) throws BookieException; /** * Remove the cookie data. @@ -109,7 +110,7 @@ interface RegistrationListener { * @param version version of the cookie data * @throws BookieException when fail to remove cookie */ - void removeCookie(String bookieId, Version version) throws BookieException; + void removeCookie(BookieId bookieId, Version version) throws BookieException; /** * Prepare ledgers root node, availableNode, readonly node.. 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 index 367e9ac0ace..dc17507637b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java @@ -25,10 +25,12 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import java.io.IOException; +import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; @@ -39,11 +41,10 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException; -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.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; @@ -60,6 +61,7 @@ /** * ZooKeeper based {@link RegistrationClient}. */ + @Slf4j public class ZKRegistrationClient implements RegistrationClient { @@ -68,13 +70,13 @@ public class ZKRegistrationClient implements RegistrationClient { class WatchTask implements SafeRunnable, Watcher, - BiConsumer>, Throwable>, + BiConsumer>, Throwable>, AutoCloseable { private final String regPath; private final Set listeners; private volatile boolean closed = false; - private Set bookies = null; + private Set bookies = null; private Version version = Version.NEW; private final CompletableFuture firstRunFuture; @@ -127,7 +129,7 @@ public void safeRun() { } @Override - public void accept(Versioned> bookieSet, Throwable throwable) { + public void accept(Versioned> bookieSet, Throwable throwable) { if (throwable != null) { if (firstRunFuture.isDone()) { scheduleWatchTask(ZK_CONNECT_BACKOFF_MS); @@ -140,9 +142,10 @@ public void accept(Versioned> bookieSet, Throwable thro if (this.version.compare(bookieSet.getVersion()) == Occurred.BEFORE) { this.version = bookieSet.getVersion(); this.bookies = bookieSet.getValue(); - - for (RegistrationListener listener : listeners) { - listener.onBookiesChanged(bookieSet); + if (!listeners.isEmpty()) { + for (RegistrationListener listener : listeners) { + listener.onBookiesChanged(bookieSet); + } } } FutureUtils.complete(firstRunFuture, null); @@ -177,6 +180,8 @@ public void close() { private WatchTask watchWritableBookiesTask = null; @Getter(AccessLevel.PACKAGE) private WatchTask watchReadOnlyBookiesTask = null; + private final ConcurrentHashMap> bookieServiceInfoCache = + new ConcurrentHashMap<>(); // registration paths private final String bookieRegistrationPath; @@ -204,42 +209,69 @@ public ZooKeeper getZk() { } @Override - public CompletableFuture>> getWritableBookies() { + public CompletableFuture>> getWritableBookies() { return getChildren(bookieRegistrationPath, null); } @Override - public CompletableFuture>> getAllBookies() { + public CompletableFuture>> getAllBookies() { return getChildren(bookieAllRegistrationPath, null); } @Override - public CompletableFuture>> getReadOnlyBookies() { + public CompletableFuture>> getReadOnlyBookies() { return getChildren(bookieReadonlyRegistrationPath, null); } @Override - public CompletableFuture> getBookieServiceInfo(String bookieId) { + public CompletableFuture> getBookieServiceInfo(BookieId bookieId) { + // we can only serve data from cache here, + // because it can happen than this method is called inside the main + // zookeeper client event loop thread + Versioned resultFromCache = bookieServiceInfoCache.get(bookieId); + log.info("getBookieServiceInfo {} -> {}", bookieId, resultFromCache); + if (resultFromCache != null) { + return CompletableFuture.completedFuture(resultFromCache); + } else { + return FutureUtils.exception(new BKException.BKBookieHandleNotAvailableException()); + } + } + + /** + * Read BookieServiceInfo from ZooKeeper and updates the local cache. + * + * @param bookieId + * @return an handle to the result of the operation. + */ + private CompletableFuture> readBookieServiceInfo(BookieId bookieId) { String pathAsWritable = bookieRegistrationPath + "/" + bookieId; String pathAsReadonly = bookieReadonlyRegistrationPath + "/" + bookieId; CompletableFuture> promise = new CompletableFuture<>(); - zk.getData(pathAsWritable, false, (int rc, String path, Object o, byte[] bytes, Stat stat) -> { + zk.getData(pathAsWritable, null, (int rc, String path, Object o, byte[] bytes, Stat stat) -> { if (KeeperException.Code.OK.intValue() == rc) { try { BookieServiceInfo bookieServiceInfo = deserializeBookieServiceInfo(bookieId, bytes); - promise.complete(new Versioned<>(bookieServiceInfo, new LongVersion(stat.getCversion()))); + Versioned result = new Versioned<>(bookieServiceInfo, + new LongVersion(stat.getCversion())); + log.info("Update BookieInfoCache (writable bookie) {} -> {}", bookieId, result.getValue()); + bookieServiceInfoCache.put(bookieId, result); + promise.complete(result); } catch (IOException ex) { promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path)); return; } } else if (KeeperException.Code.NONODE.intValue() == rc) { // not found, looking for a readonly bookie - zk.getData(pathAsReadonly, false, (int rc2, String path2, Object o2, byte[] bytes2, Stat stat2) -> { + zk.getData(pathAsReadonly, null, (int rc2, String path2, Object o2, byte[] bytes2, Stat stat2) -> { if (KeeperException.Code.OK.intValue() == rc2) { try { BookieServiceInfo bookieServiceInfo = deserializeBookieServiceInfo(bookieId, bytes2); - promise.complete(new Versioned<>(bookieServiceInfo, new LongVersion(stat2.getCversion()))); + Versioned result = + new Versioned<>(bookieServiceInfo, new LongVersion(stat2.getCversion())); + log.info("Update BookieInfoCache (readonly bookie) {} -> {}", bookieId, result.getValue()); + bookieServiceInfoCache.put(bookieId, result); + promise.complete(result); } catch (IOException ex) { promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc2), path2)); return; @@ -258,10 +290,10 @@ public CompletableFuture> getBookieServiceInfo(Stri @SuppressWarnings("unchecked") @VisibleForTesting - static BookieServiceInfo deserializeBookieServiceInfo(String bookieId, byte[] bookieServiceInfo) + static BookieServiceInfo deserializeBookieServiceInfo(BookieId bookieId, byte[] bookieServiceInfo) throws IOException { if (bookieServiceInfo == null || bookieServiceInfo.length == 0) { - return BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId); + return BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId.toString()); } BookieServiceInfoFormat builder = BookieServiceInfoFormat.parseFrom(bookieServiceInfo); @@ -285,18 +317,40 @@ static BookieServiceInfo deserializeBookieServiceInfo(String bookieId, byte[] bo return bsi; } - private CompletableFuture>> getChildren(String regPath, Watcher watcher) { - CompletableFuture>> future = FutureUtils.createFuture(); + /** + * Reads the list of bookies at the given path and eagerly caches the BookieServiceInfo + * structure. + * + * @param regPath the path on ZooKeeper + * @param watcher an optional watcher + * @return an handle to the operation + */ + private CompletableFuture>> getChildren(String regPath, Watcher watcher) { + CompletableFuture>> future = FutureUtils.createFuture(); zk.getChildren(regPath, watcher, (rc, path, ctx, children, stat) -> { - if (Code.OK != rc) { + if (KeeperException.Code.OK.intValue() != rc) { ZKException zke = new ZKException(KeeperException.create(KeeperException.Code.get(rc), path)); future.completeExceptionally(zke.fillInStackTrace()); return; } Version version = new LongVersion(stat.getCversion()); - Set bookies = convertToBookieAddresses(children); - future.complete(new Versioned<>(bookies, version)); + Set bookies = convertToBookieAddresses(children); + List>> bookieInfoUpdated = new ArrayList<>(bookies.size()); + for (BookieId id : bookies) { + // update the cache for new bookies + if (!bookieServiceInfoCache.containsKey(id)) { + bookieInfoUpdated.add(readBookieServiceInfo(id)); + } + } + FutureUtils + .collect(bookieInfoUpdated) + .whenComplete((List> info, Throwable error) -> { + // we are ignoring errors intentionally + // there could be bookies that published unparseable information + // or other temporary/permanent or temporary errors + future.complete(new Versioned<>(bookies, version)); + }); }, null); return future; } @@ -372,21 +426,14 @@ public synchronized void unwatchReadOnlyBookies(RegistrationListener listener) { } } - private static HashSet convertToBookieAddresses(List children) { + private static HashSet convertToBookieAddresses(List children) { // Read the bookie addresses into a set for efficient lookup - HashSet newBookieAddrs = Sets.newHashSet(); + 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; - } + BookieId bookieAddr = BookieId.parse(bookieAddrString); newBookieAddrs.add(bookieAddr); } return newBookieAddrs; 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 index 951bb9581ca..e8eea4939dc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java @@ -54,7 +54,7 @@ import org.apache.bookkeeper.meta.ZkLayoutManager; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.ZkUtils; @@ -158,7 +158,7 @@ public void close() { * @param bookieId bookie id * @return */ - public String getCookiePath(String bookieId) { + public String getCookiePath(BookieId bookieId) { return this.cookiePath + "/" + bookieId; } @@ -217,7 +217,7 @@ public void process(WatchedEvent event) { } @Override - public void registerBookie(String bookieId, boolean readOnly, + public void registerBookie(BookieId bookieId, boolean readOnly, BookieServiceInfo bookieServiceInfo) throws BookieException { if (!readOnly) { String regPath = bookieRegistrationPath + "/" + bookieId; @@ -284,7 +284,8 @@ private void doRegisterBookie(String regPath, BookieServiceInfo bookieServiceInf } } - private void doRegisterReadOnlyBookie(String bookieId, BookieServiceInfo bookieServiceInfo) throws BookieException { + private void doRegisterReadOnlyBookie(BookieId bookieId, BookieServiceInfo bookieServiceInfo) + throws BookieException { try { if (null == zk.exists(this.bookieReadonlyRegistrationPath, false)) { try { @@ -312,7 +313,7 @@ private void doRegisterReadOnlyBookie(String bookieId, BookieServiceInfo bookieS } @Override - public void unregisterBookie(String bookieId, boolean readOnly) throws BookieException { + public void unregisterBookie(BookieId bookieId, boolean readOnly) throws BookieException { String regPath; if (!readOnly) { regPath = bookieRegistrationPath + "/" + bookieId; @@ -338,7 +339,7 @@ private void doUnregisterBookie(String regPath) throws BookieException { // @Override - public void writeCookie(String bookieId, + public void writeCookie(BookieId bookieId, Versioned cookieData) throws BookieException { String zkPath = getCookiePath(bookieId); try { @@ -365,16 +366,16 @@ public void writeCookie(String bookieId, Thread.currentThread().interrupt(); throw new MetadataStoreException("Interrupted writing cookie for bookie " + bookieId, ie); } catch (NoNodeException nne) { - throw new CookieNotFoundException(bookieId); + throw new CookieNotFoundException(bookieId.toString()); } catch (NodeExistsException nee) { - throw new CookieExistException(bookieId); + throw new CookieExistException(bookieId.toString()); } catch (KeeperException e) { throw new MetadataStoreException("Failed to write cookie for bookie " + bookieId); } } @Override - public Versioned readCookie(String bookieId) throws BookieException { + public Versioned readCookie(BookieId bookieId) throws BookieException { String zkPath = getCookiePath(bookieId); try { Stat stat = zk.exists(zkPath, false); @@ -383,19 +384,19 @@ public Versioned readCookie(String bookieId) throws BookieException { LongVersion version = new LongVersion(stat.getVersion()); return new Versioned<>(data, version); } catch (NoNodeException nne) { - throw new CookieNotFoundException(bookieId); + throw new CookieNotFoundException(bookieId.toString()); } catch (KeeperException | InterruptedException e) { throw new MetadataStoreException("Failed to read cookie for bookie " + bookieId); } } @Override - public void removeCookie(String bookieId, Version version) throws BookieException { + public void removeCookie(BookieId bookieId, Version version) throws BookieException { String zkPath = getCookiePath(bookieId); try { zk.delete(zkPath, (int) ((LongVersion) version).getLongVersion()); } catch (NoNodeException e) { - throw new CookieNotFoundException(bookieId); + throw new CookieNotFoundException(bookieId.toString()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new MetadataStoreException("Interrupted deleting cookie for bookie " + bookieId, e); @@ -517,7 +518,7 @@ public boolean nukeExistingCluster() throws Exception { null )) { if (availableNodeExists) { - Collection rwBookies = FutureUtils + Collection rwBookies = FutureUtils .result(regClient.getWritableBookies(), EXCEPTION_FUNC).getValue(); if (rwBookies != null && !rwBookies.isEmpty()) { log.error("Bookies are still up and connected to this cluster, " @@ -527,7 +528,7 @@ public boolean nukeExistingCluster() throws Exception { boolean readonlyNodeExists = null != zk.exists(bookieReadonlyRegistrationPath, false); if (readonlyNodeExists) { - Collection roBookies = FutureUtils + Collection roBookies = FutureUtils .result(regClient.getReadOnlyBookies(), EXCEPTION_FUNC).getValue(); if (roBookies != null && !roBookies.isEmpty()) { log.error("Readonly Bookies are still up and connected to this cluster, " @@ -593,7 +594,7 @@ public boolean format() throws Exception { } @Override - public boolean isBookieRegistered(String bookieId) throws BookieException { + public boolean isBookieRegistered(BookieId bookieId) throws BookieException { String regPath = bookieRegistrationPath + "/" + bookieId; String readonlyRegPath = bookieReadonlyRegistrationPath + "/" + bookieId; try { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java index aded61dd8a8..1e46b7e52a1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -46,7 +46,7 @@ import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.LedgerMetadata.State; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; import org.slf4j.Logger; @@ -201,10 +201,10 @@ private static byte[] serializeVersion3(LedgerMetadata metadata) throws IOExcept } } - for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { + for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder(); segmentBuilder.setFirstEntryId(entry.getKey()); - for (BookieSocketAddress addr : entry.getValue()) { + for (BookieId addr : entry.getValue()) { segmentBuilder.addEnsembleMember(addr.toString()); } builder.addSegment(segmentBuilder.build()); @@ -268,11 +268,11 @@ private static byte[] serializeVersion2(LedgerMetadata metadata) throws IOExcept } } - for (Map.Entry> entry : + for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder(); segmentBuilder.setFirstEntryId(entry.getKey()); - for (BookieSocketAddress addr : entry.getValue()) { + for (BookieId addr : entry.getValue()) { segmentBuilder.addEnsembleMember(addr.toString()); } builder.addSegment(segmentBuilder.build()); @@ -294,10 +294,10 @@ private static byte[] serializeVersion1(LedgerMetadata metadata) throws IOExcept writer.append(String.valueOf(metadata.getEnsembleSize())).append(LINE_SPLITTER); writer.append(String.valueOf(metadata.getLength())).append(LINE_SPLITTER); - for (Map.Entry> entry : + for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { writer.append(String.valueOf(entry.getKey())); - for (BookieSocketAddress addr : entry.getValue()) { + for (BookieId addr : entry.getValue()) { writer.append(FIELD_SPLITTER).append(addr.toString()); } writer.append(LINE_SPLITTER); @@ -431,9 +431,9 @@ private static void decodeFormat(LedgerMetadataFormat data, LedgerMetadataBuilde } for (LedgerMetadataFormat.Segment s : data.getSegmentList()) { - List addrs = new ArrayList<>(); + List addrs = new ArrayList<>(); for (String addr : s.getEnsembleMemberList()) { - addrs.add(new BookieSocketAddress(addr)); + addrs.add(BookieId.parse(addr)); } builder.newEnsembleEntry(s.getFirstEntryId(), addrs); } @@ -472,9 +472,9 @@ private static LedgerMetadata parseVersion1Config(InputStream is) throws IOExcep break; } - ArrayList addrs = new ArrayList(); + ArrayList addrs = new ArrayList(); for (int j = 1; j < parts.length; j++) { - addrs.add(new BookieSocketAddress(parts[j])); + addrs.add(BookieId.parse(parts[j])); } builder.newEnsembleEntry(Long.parseLong(parts[0]), addrs); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieId.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieId.java new file mode 100644 index 00000000000..d4e63556daf --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieId.java @@ -0,0 +1,90 @@ +/** + * + * 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.net; + +import java.util.Objects; + +/** + * This is an identifier for a BookieID. + */ +public final class BookieId { + + private final String id; + + private BookieId(String id) { + validateBookieId(id); + this.id = id; + } + + /** + * Returns the serialized version of this object. + * @return the bookieId + */ + @Override + public String toString() { + return id; + } + + /** + * Parses the given serialized representation of a BookieId. + * @param serialized + * @return the parsed BookieId + */ + public static BookieId parse(String serialized) { + return new BookieId(serialized); + } + + public String getId() { + return id; + } + + @Override + public int hashCode() { + return this.id.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final BookieId other = (BookieId) obj; + if (!Objects.equals(this.id, other.id)) { + return false; + } + return true; + } + + private static void validateBookieId(String id) { + Objects.requireNonNull(id, "BookieId cannot be null"); + if (!(id.matches("[a-zA-Z0-9:-_.\\-]+")) + || "readonly".equalsIgnoreCase(id)) { + throw new IllegalArgumentException("BookieId " + id + " is not valid"); + } + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java index e7d0ef50482..b22ae5f61a1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java @@ -21,14 +21,14 @@ * Bookie Node implementation. */ public class BookieNode extends NodeBase { - private final BookieSocketAddress addr; // identifier of a bookie node. + private final BookieId addr; // identifier of a bookie node. - public BookieNode(BookieSocketAddress addr, String networkLoc) { + public BookieNode(BookieId addr, String networkLoc) { super(addr.toString(), networkLoc); this.addr = addr; } - public BookieSocketAddress getAddr() { + public BookieId getAddr() { return addr; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java index 22f026a2ec8..06da0dc5202 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java @@ -29,6 +29,7 @@ import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.Optional; +import org.apache.bookkeeper.proto.BookieAddressResolver; /** * This is a data wrapper class that is an InetSocketAddress, it would use the hostname @@ -135,4 +136,54 @@ public int hashCode() { return this.hostname.hashCode() + 13 * this.port; } + /** + * Create a BookieID in legacy format hostname:port. + * @return the BookieID + */ + public BookieId toBookieId() { + return BookieId.parse(this.toString()); + } + + /** + * Simple converter from legacy BookieId to a real network address. + */ + public static final BookieAddressResolver LEGACY_BOOKIEID_RESOLVER = (BookieId b) -> { + try { + return new BookieSocketAddress(b.toString()); + } catch (UnknownHostException err) { + throw new BookieAddressResolver.BookieIdNotResolvedException(b, err); + } + }; + + /** + * Utility for Placement Policies that need to create a dummy BookieId that represents + * a given host. + * @param hostname the hostname + * @return a dummy bookie id, compatible with the BookieSocketAddress#toBookieId, with a 0 tcp port. + */ + public static BookieId createDummyBookieIdForHostname(String hostname) { + return BookieId.parse(hostname + ":0"); + } + + /** + * Tells whether a BookieId may be a dummy id. + * @param bookieId + * @return true if the BookieId looks like it has been generated by + * {@link #createDummyBookieIdForHostname(java.lang.String)} + */ + public static boolean isDummyBookieIdForHostname(BookieId bookieId) { + return bookieId.getId().endsWith(":0"); + } + + /** + * Use legacy resolver to resolve a bookieId. + * @param bookieId id supposed to be generated by + * {@link #createDummyBookieIdForHostname(java.lang.String)} + * @return the BookieSocketAddress + */ + public static BookieSocketAddress resolveDummyBookieId(BookieId bookieId) + throws BookieAddressResolver.BookieIdNotResolvedException { + return LEGACY_BOOKIEID_RESOLVER.resolve(bookieId); + } + } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieAddressResolver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieAddressResolver.java new file mode 100644 index 00000000000..f70740b1e68 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieAddressResolver.java @@ -0,0 +1,53 @@ +/* + * Copyright 2020 The Apache Software Foundation. + * + * Licensed 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.proto; + +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieSocketAddress; + +/** + * Maps a logical BookieId to a ResolvedBookieSocketAddress + that it to a network address. + */ +public interface BookieAddressResolver { + + /** + * Maps a logical address to a network address. + * @param bookieId + * @return a mapped address. + * @throws BookieIdNotResolvedException if it is not possible to resolve the address of the BookieId + */ + BookieSocketAddress resolve(BookieId bookieId) throws BookieIdNotResolvedException; + + /** + * This error happens when there is not enough information to resolve a BookieId + * to a BookieSocketAddress, this can happen when the Bookie is down + * and it is not publishing its EndpointInfo. + */ + class BookieIdNotResolvedException extends RuntimeException { + private final BookieId bookieId; + + public BookieIdNotResolvedException(BookieId bookieId, Throwable cause) { + super("Cannot resolve bookieId " + bookieId + ", bookie does not exist or it is not running", cause); + this.bookieId = bookieId; + } + + public BookieId getBookieId() { + return bookieId; + } + + } +} 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 2092a67af41..ddd7f18863d 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 @@ -25,7 +25,7 @@ import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.api.WriteFlag; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; @@ -47,7 +47,7 @@ public interface BookieClient { * * @return the list of faulty bookies */ - List getFaultyBookies(); + List getFaultyBookies(); /** * Check whether the channel used to write to a bookie channel is writable. @@ -67,7 +67,7 @@ public interface BookieClient { * @param ledgerId the ledger we wish to send a request to * */ - boolean isWritable(BookieSocketAddress address, long ledgerId); + boolean isWritable(BookieId address, long ledgerId); /** * Get the number of outstanding requests on the channel used to connect @@ -82,7 +82,7 @@ public interface BookieClient { * @param ledgerId the ledger whose channel we wish to query * @return the number of requests currently outstanding */ - long getNumPendingRequests(BookieSocketAddress address, long ledgerId); + long getNumPendingRequests(BookieId address, long ledgerId); /** * Send a force request to the server. When complete all entries which have @@ -94,7 +94,7 @@ public interface BookieClient { * @param cb the callback notified when the request completes * @param ctx a context object passed to the callback on completion */ - void forceLedger(BookieSocketAddress address, long ledgerId, + void forceLedger(BookieId address, long ledgerId, ForceLedgerCallback cb, Object ctx); /** @@ -106,7 +106,7 @@ void forceLedger(BookieSocketAddress address, long ledgerId, * @param cb the callback notified when the request completes * @param ctx a context object passed to the callback on completion */ - void readLac(BookieSocketAddress address, long ledgerId, ReadLacCallback cb, Object ctx); + void readLac(BookieId address, long ledgerId, ReadLacCallback cb, Object ctx); /** * Explicitly write the last add confirmed for ledger {@code ledgerId} to the bookie at @@ -120,7 +120,7 @@ void forceLedger(BookieSocketAddress address, long ledgerId, * @param cb the callback notified when the request completes * @param ctx a context object passed to the callback on completion */ - void writeLac(BookieSocketAddress address, long ledgerId, byte[] masterKey, + void writeLac(BookieId address, long ledgerId, byte[] masterKey, long lac, ByteBufList toSend, WriteLacCallback cb, Object ctx); /** @@ -139,7 +139,7 @@ void writeLac(BookieSocketAddress address, long ledgerId, byte[] masterKey, * @param writeFlags a set of write flags * {@link org.apache.bookkeeper.client.api.WriteFlags} */ - void addEntry(BookieSocketAddress address, long ledgerId, byte[] masterKey, + void addEntry(BookieId address, long ledgerId, byte[] masterKey, long entryId, ByteBufList toSend, WriteCallback cb, Object ctx, int options, boolean allowFastFail, EnumSet writeFlags); @@ -147,7 +147,7 @@ void addEntry(BookieSocketAddress address, long ledgerId, byte[] masterKey, * Read entry with a null masterkey, disallowing failfast. * @see #readEntry(BookieSocketAddress,long,long,ReadEntryCallback,Object,int,byte[],boolean) */ - default void readEntry(BookieSocketAddress address, long ledgerId, long entryId, + default void readEntry(BookieId address, long ledgerId, long entryId, ReadEntryCallback cb, Object ctx, int flags) { readEntry(address, ledgerId, entryId, cb, ctx, flags, null); } @@ -156,7 +156,7 @@ default void readEntry(BookieSocketAddress address, long ledgerId, long entryId, * Read entry, disallowing failfast. * @see #readEntry(BookieSocketAddress,long,long,ReadEntryCallback,Object,int,byte[],boolean) */ - default void readEntry(BookieSocketAddress address, long ledgerId, long entryId, + default void readEntry(BookieId address, long ledgerId, long entryId, ReadEntryCallback cb, Object ctx, int flags, byte[] masterKey) { readEntry(address, ledgerId, entryId, cb, ctx, flags, masterKey, false); } @@ -176,7 +176,7 @@ default void readEntry(BookieSocketAddress address, long ledgerId, long entryId, * @param allowFastFail fail the read immediately if the channel is non-writable * {@link #isWritable(BookieSocketAddress,long)} */ - void readEntry(BookieSocketAddress address, long ledgerId, long entryId, + void readEntry(BookieId address, long ledgerId, long entryId, ReadEntryCallback cb, Object ctx, int flags, byte[] masterKey, boolean allowFastFail); @@ -194,7 +194,7 @@ void readEntry(BookieSocketAddress address, long ledgerId, long entryId, * @param cb the callback notified when the request completes * @param ctx a context object passed to the callback on completion */ - void readEntryWaitForLACUpdate(BookieSocketAddress address, + void readEntryWaitForLACUpdate(BookieId address, long ledgerId, long entryId, long previousLAC, @@ -214,7 +214,7 @@ void readEntryWaitForLACUpdate(BookieSocketAddress address, * * @see org.apache.bookkeeper.client.BookieInfoReader.BookieInfo */ - void getBookieInfo(BookieSocketAddress address, long requested, + void getBookieInfo(BookieId address, long requested, GetBookieInfoCallback cb, Object ctx); /** @@ -222,12 +222,12 @@ void getBookieInfo(BookieSocketAddress address, long requested, * and returns Future for the result. * * @param address - * BookieSocketAddress of the bookie + * BookieId of the bookie * @param ledgerId * ledgerId * @return returns Future */ - CompletableFuture getListOfEntriesOfLedger(BookieSocketAddress address, + CompletableFuture getListOfEntriesOfLedger(BookieId address, long ledgerId); /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index 8fb0f1d1b56..b904c16c77a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -55,6 +55,7 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.common.util.SafeRunnable; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.FutureGetListOfEntriesOfLedger; @@ -86,8 +87,8 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFac private final EventLoopGroup eventLoopGroup; private final ByteBufAllocator allocator; - final ConcurrentHashMap channels = - new ConcurrentHashMap(); + final ConcurrentHashMap channels = + new ConcurrentHashMap(); private final ClientAuthProvider.Factory authProviderFactory; private final ExtensionRegistry registry; @@ -99,13 +100,14 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFac private final ReentrantReadWriteLock closeLock; private final StatsLogger statsLogger; private final int numConnectionsPerBookie; + private final BookieAddressResolver bookieAddressResolver; private final long bookieErrorThresholdPerInterval; public BookieClientImpl(ClientConfiguration conf, EventLoopGroup eventLoopGroup, ByteBufAllocator allocator, OrderedExecutor executor, ScheduledExecutorService scheduler, - StatsLogger statsLogger) throws IOException { + StatsLogger statsLogger, BookieAddressResolver bookieAddressResolver) throws IOException { this.conf = conf; this.v3Conf = new ClientConfiguration(conf); this.v3Conf.setUseV2WireProtocol(false); @@ -115,7 +117,7 @@ public BookieClientImpl(ClientConfiguration conf, EventLoopGroup eventLoopGroup, this.executor = executor; this.closed = false; this.closeLock = new ReentrantReadWriteLock(); - + this.bookieAddressResolver = bookieAddressResolver; this.registry = ExtensionRegistry.newInstance(); this.authProviderFactory = AuthProviderFactoryFactory.newClientAuthProviderFactory(conf); @@ -150,8 +152,8 @@ private int getRc(int rc) { } @Override - public List getFaultyBookies() { - List faultyBookies = Lists.newArrayList(); + public List getFaultyBookies() { + List faultyBookies = Lists.newArrayList(); for (PerChannelBookieClientPool channelPool : channels.values()) { if (channelPool instanceof DefaultPerChannelBookieClientPool) { DefaultPerChannelBookieClientPool pool = (DefaultPerChannelBookieClientPool) channelPool; @@ -164,14 +166,14 @@ public List getFaultyBookies() { } @Override - public boolean isWritable(BookieSocketAddress address, long key) { + public boolean isWritable(BookieId address, long key) { final PerChannelBookieClientPool pcbcPool = lookupClient(address); // if null, let the write initiate connect of fail with whatever error it produces return pcbcPool == null || pcbcPool.isWritable(key); } @Override - public long getNumPendingRequests(BookieSocketAddress address, long ledgerId) { + public long getNumPendingRequests(BookieId address, long ledgerId) { PerChannelBookieClientPool pcbcPool = lookupClient(address); if (pcbcPool == null) { return 0; @@ -183,7 +185,7 @@ public long getNumPendingRequests(BookieSocketAddress address, long ledgerId) { } @Override - public PerChannelBookieClient create(BookieSocketAddress address, PerChannelBookieClientPool pcbcPool, + public PerChannelBookieClient create(BookieId address, PerChannelBookieClientPool pcbcPool, SecurityHandlerFactory shFactory, boolean forceUseV3) throws SecurityException { StatsLogger statsLoggerForPCBC = statsLogger; if (conf.getLimitStatsLogging()) { @@ -194,10 +196,11 @@ public PerChannelBookieClient create(BookieSocketAddress address, PerChannelBook clientConfiguration = v3Conf; } return new PerChannelBookieClient(clientConfiguration, executor, eventLoopGroup, allocator, address, - statsLoggerForPCBC, authProviderFactory, registry, pcbcPool, shFactory); + statsLoggerForPCBC, authProviderFactory, registry, pcbcPool, + shFactory, bookieAddressResolver); } - public PerChannelBookieClientPool lookupClient(BookieSocketAddress addr) { + public PerChannelBookieClientPool lookupClient(BookieId addr) { PerChannelBookieClientPool clientPool = channels.get(addr); if (null == clientPool) { closeLock.readLock().lock(); @@ -227,7 +230,7 @@ public PerChannelBookieClientPool lookupClient(BookieSocketAddress addr) { } @Override - public void forceLedger(final BookieSocketAddress addr, final long ledgerId, + public void forceLedger(final BookieId addr, final long ledgerId, final ForceLedgerCallback cb, final Object ctx) { final PerChannelBookieClientPool client = lookupClient(addr); if (client == null) { @@ -252,7 +255,7 @@ public void forceLedger(final BookieSocketAddress addr, final long ledgerId, } @Override - public void writeLac(final BookieSocketAddress addr, final long ledgerId, final byte[] masterKey, + public void writeLac(final BookieId addr, final long ledgerId, final byte[] masterKey, final long lac, final ByteBufList toSend, final WriteLacCallback cb, final Object ctx) { final PerChannelBookieClientPool client = lookupClient(addr); if (client == null) { @@ -282,7 +285,7 @@ public void writeLac(final BookieSocketAddress addr, final long ledgerId, final private void completeAdd(final int rc, final long ledgerId, final long entryId, - final BookieSocketAddress addr, + final BookieId addr, final WriteCallback cb, final Object ctx) { try { @@ -302,7 +305,7 @@ public String toString() { } @Override - public void addEntry(final BookieSocketAddress addr, + public void addEntry(final BookieId addr, final long ledgerId, final byte[] masterKey, final long entryId, @@ -330,7 +333,7 @@ public void addEntry(final BookieSocketAddress addr, } @Override - public CompletableFuture getListOfEntriesOfLedger(BookieSocketAddress address, + public CompletableFuture getListOfEntriesOfLedger(BookieId address, long ledgerId) { FutureGetListOfEntriesOfLedger futureResult = new FutureGetListOfEntriesOfLedger(ledgerId); final PerChannelBookieClientPool client = lookupClient(address); @@ -383,7 +386,7 @@ private static class ChannelReadyForAddEntryCallback private ByteBufList toSend; private long ledgerId; private long entryId; - private BookieSocketAddress addr; + private BookieId addr; private Object ctx; private WriteCallback cb; private int options; @@ -393,7 +396,7 @@ private static class ChannelReadyForAddEntryCallback static ChannelReadyForAddEntryCallback create( BookieClientImpl bookieClient, ByteBufList toSend, long ledgerId, - long entryId, BookieSocketAddress addr, Object ctx, + long entryId, BookieId addr, Object ctx, WriteCallback cb, int options, byte[] masterKey, boolean allowFastFail, EnumSet writeFlags) { ChannelReadyForAddEntryCallback callback = RECYCLER.get(); @@ -456,7 +459,7 @@ public void recycle() { } @Override - public void readLac(final BookieSocketAddress addr, final long ledgerId, final ReadLacCallback cb, + public void readLac(final BookieId addr, final long ledgerId, final ReadLacCallback cb, final Object ctx) { final PerChannelBookieClientPool client = lookupClient(addr); if (client == null) { @@ -481,19 +484,19 @@ public void readLac(final BookieSocketAddress addr, final long ledgerId, final R } @Override - public void readEntry(BookieSocketAddress addr, long ledgerId, long entryId, + public void readEntry(BookieId addr, long ledgerId, long entryId, ReadEntryCallback cb, Object ctx, int flags) { readEntry(addr, ledgerId, entryId, cb, ctx, flags, null); } @Override - public void readEntry(final BookieSocketAddress addr, final long ledgerId, final long entryId, + public void readEntry(final BookieId addr, final long ledgerId, final long entryId, final ReadEntryCallback cb, final Object ctx, int flags, byte[] masterKey) { readEntry(addr, ledgerId, entryId, cb, ctx, flags, masterKey, false); } @Override - public void readEntry(final BookieSocketAddress addr, final long ledgerId, final long entryId, + public void readEntry(final BookieId addr, final long ledgerId, final long entryId, final ReadEntryCallback cb, final Object ctx, int flags, byte[] masterKey, final boolean allowFastFail) { final PerChannelBookieClientPool client = lookupClient(addr); @@ -514,7 +517,7 @@ public void readEntry(final BookieSocketAddress addr, final long ledgerId, final @Override - public void readEntryWaitForLACUpdate(final BookieSocketAddress addr, + public void readEntryWaitForLACUpdate(final BookieId addr, final long ledgerId, final long entryId, final long previousLAC, @@ -540,7 +543,7 @@ public void readEntryWaitForLACUpdate(final BookieSocketAddress addr, } @Override - public void getBookieInfo(final BookieSocketAddress addr, final long requested, final GetBookieInfoCallback cb, + public void getBookieInfo(final BookieId addr, final long requested, final GetBookieInfoCallback cb, final Object ctx) { final PerChannelBookieClientPool client = lookupClient(addr); if (client == null) { @@ -633,7 +636,7 @@ public static void main(String[] args) throws NumberFormatException, IOException WriteCallback cb = new WriteCallback() { @Override - public void writeComplete(int rc, long ledger, long entry, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledger, long entry, BookieId addr, Object ctx) { Counter counter = (Counter) ctx; counter.dec(); if (rc != 0) { @@ -652,8 +655,8 @@ public void writeComplete(int rc, long ledger, long entry, BookieSocketAddress a ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor( new DefaultThreadFactory("BookKeeperClientScheduler")); BookieClientImpl bc = new BookieClientImpl(new ClientConfiguration(), eventLoopGroup, - null, executor, scheduler, NullStatsLogger.INSTANCE); - BookieSocketAddress addr = new BookieSocketAddress(args[0], Integer.parseInt(args[1])); + null, executor, scheduler, NullStatsLogger.INSTANCE, b -> BookieSocketAddress.class.cast(b)); + BookieId addr = new BookieSocketAddress(args[0], Integer.parseInt(args[1])).toBookieId(); for (int i = 0; i < 100000; i++) { counter.inc(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java index 5b5c288123a..a48f7495886 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java @@ -351,6 +351,7 @@ protected void initChannel(SocketChannel ch) throws Exception { }); // Bind and start to accept incoming connections + LOG.info("Binding bookie-rpc endpoint to {}", address); Channel listen = bootstrap.bind(address.getAddress(), address.getPort()).sync().channel(); if (listen.localAddress() instanceof InetSocketAddress) { if (conf.getBookiePort() == 0) { @@ -412,7 +413,7 @@ protected void initChannel(LocalChannel ch) throws Exception { // use the same address 'name', so clients can find local Bookie still discovering them using ZK jvmBootstrap.bind(bookieAddress.getLocalAddress()).sync(); - LocalBookiesRegistry.registerLocalBookieAddress(bookieAddress); + LocalBookiesRegistry.registerLocalBookieAddress(bookieAddress.toBookieId()); } } @@ -440,7 +441,7 @@ void shutdown() { } } if (jvmEventLoopGroup != null) { - LocalBookiesRegistry.unregisterLocalBookieAddress(bookieAddress); + LocalBookiesRegistry.unregisterLocalBookieAddress(bookieAddress.toBookieId()); jvmEventLoopGroup.shutdownGracefully(); } 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 fda7ed78246..2864b299a70 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 @@ -44,6 +44,7 @@ import org.apache.bookkeeper.common.util.JsonUtil.ParseJsonException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.processor.RequestProcessor; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -167,6 +168,11 @@ public BookieSocketAddress getLocalAddress() throws UnknownHostException { return Bookie.getBookieAddress(conf); } + @VisibleForTesting + public BookieId getBookieId() throws UnknownHostException { + return Bookie.getBookieId(conf); + } + @VisibleForTesting public Bookie getBookie() { return bookie; @@ -319,13 +325,14 @@ public static void main(String[] args) { @Override public String toString() { - String id = "UNKNOWN"; - + String addr = "UNKNOWN"; + String id = "?"; try { - id = Bookie.getBookieAddress(conf).toString(); + addr = Bookie.getBookieAddress(conf).toString(); + id = getBookieId().toString(); } catch (UnknownHostException e) { //Ignored... } - return "Bookie Server listening on " + id; + return "Bookie Server listening on " + addr + " with id " + id; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java index fdfd379aa70..9904c90aef0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java @@ -35,7 +35,7 @@ import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.LedgerMetadata; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; import org.apache.bookkeeper.util.MathUtils; @@ -77,7 +77,7 @@ public interface LedgerMetadataListener { * A writer callback interface. */ public interface WriteCallback { - void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx); + void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx); } /** @@ -91,14 +91,14 @@ public interface ReadLacCallback { * A last-add-confirmed (LAC) writer callback interface. */ public interface WriteLacCallback { - void writeLacComplete(int rc, long ledgerId, BookieSocketAddress addr, Object ctx); + void writeLacComplete(int rc, long ledgerId, BookieId addr, Object ctx); } /** * Force callback interface. */ public interface ForceLedgerCallback { - void forceLedgerComplete(int rc, long ledgerId, BookieSocketAddress addr, Object ctx); + void forceLedgerComplete(int rc, long ledgerId, BookieId addr, Object ctx); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DefaultPerChannelBookieClientPool.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DefaultPerChannelBookieClientPool.java index 71193de3f71..7305bef249d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DefaultPerChannelBookieClientPool.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DefaultPerChannelBookieClientPool.java @@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.tls.SecurityHandlerFactory; @@ -45,7 +45,7 @@ class DefaultPerChannelBookieClientPool implements PerChannelBookieClientPool, static final Logger LOG = LoggerFactory.getLogger(DefaultPerChannelBookieClientPool.class); final PerChannelBookieClientFactory factory; - final BookieSocketAddress address; + final BookieId address; final PerChannelBookieClient[] clients; final PerChannelBookieClient[] clientsV3Enforced; @@ -57,7 +57,7 @@ class DefaultPerChannelBookieClientPool implements PerChannelBookieClientPool, final AtomicLong errorCounter = new AtomicLong(0); DefaultPerChannelBookieClientPool(ClientConfiguration conf, PerChannelBookieClientFactory factory, - BookieSocketAddress address, + BookieId address, int coreSize) throws SecurityException { checkArgument(coreSize > 0); this.factory = factory; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java index f8891722a9e..55aa97bcac4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java @@ -25,7 +25,7 @@ import io.netty.channel.Channel; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.bookie.Bookie; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerRequest; import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerResponse; import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; @@ -58,7 +58,7 @@ private ForceLedgerResponse getForceLedgerResponse() { } BookkeeperInternalCallbacks.WriteCallback wcb = - (int rc, long ledgerId1, long entryId, BookieSocketAddress addr, Object ctx) -> { + (int rc, long ledgerId1, long entryId, BookieId addr, Object ctx) -> { checkArgument(entryId == Bookie.METAENTRY_ID_FORCE_LEDGER, "entryId must be METAENTRY_ID_FORCE_LEDGER but was {}", entryId); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LocalBookiesRegistry.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LocalBookiesRegistry.java index 59b6ed0ae6f..b023cf61fee 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LocalBookiesRegistry.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LocalBookiesRegistry.java @@ -21,25 +21,25 @@ package org.apache.bookkeeper.proto; import java.util.concurrent.ConcurrentHashMap; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; /** * Local registry for embedded Bookies. */ public class LocalBookiesRegistry { - private static final ConcurrentHashMap localBookiesRegistry = + private static final ConcurrentHashMap localBookiesRegistry = new ConcurrentHashMap<>(); - static void registerLocalBookieAddress(BookieSocketAddress address) { + static void registerLocalBookieAddress(BookieId address) { localBookiesRegistry.put(address, Boolean.TRUE); } - static void unregisterLocalBookieAddress(BookieSocketAddress address) { + static void unregisterLocalBookieAddress(BookieId address) { if (address != null) { localBookiesRegistry.remove(address); } } - public static boolean isLocalBookie(BookieSocketAddress address) { + public static boolean isLocalBookie(BookieId address) { return localBookiesRegistry.containsKey(address); } 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 94ed865e75c..45650b2c153 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 @@ -75,13 +75,16 @@ import java.util.Optional; import java.util.Queue; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.BiPredicate; import javax.net.ssl.SSLHandshakeException; import javax.net.ssl.SSLPeerUnverifiedException; +import lombok.SneakyThrows; import org.apache.bookkeeper.auth.BookKeeperPrincipal; import org.apache.bookkeeper.auth.ClientAuthProvider; @@ -92,6 +95,7 @@ import org.apache.bookkeeper.common.util.MdcUtils; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; @@ -167,7 +171,8 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter { private static final int DEFAULT_HIGH_PRIORITY_VALUE = 100; // We may add finer grained priority later. private static final AtomicLong txnIdGenerator = new AtomicLong(0); - final BookieSocketAddress addr; + final BookieId bookieId; + final BookieAddressResolver bookieAddressResolver; final EventLoopGroup eventLoopGroup; final ByteBufAllocator allocator; final OrderedExecutor executor; @@ -318,6 +323,14 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter { private final ClientConnectionPeer connectionPeer; private volatile BookKeeperPrincipal authorizedId = BookKeeperPrincipal.ANONYMOUS; + @SneakyThrows + private FailedChannelFutureImpl processBookieNotResolvedError(long startTime, + BookieAddressResolver.BookieIdNotResolvedException err) { + FailedChannelFutureImpl failedFuture = new FailedChannelFutureImpl(err); + contextPreservingListener(new ConnectionFutureListener(startTime)).operationComplete(failedFuture); + return failedFuture; + } + enum ConnectionState { DISCONNECTED, CONNECTING, CONNECTED, CLOSED, START_TLS } @@ -333,41 +346,46 @@ enum ConnectionState { private volatile boolean isWritable = true; public PerChannelBookieClient(OrderedExecutor executor, EventLoopGroup eventLoopGroup, - BookieSocketAddress addr) throws SecurityException { + BookieId addr, BookieAddressResolver bookieAddressResolver) throws SecurityException { this(new ClientConfiguration(), executor, eventLoopGroup, addr, NullStatsLogger.INSTANCE, null, null, - null); + null, bookieAddressResolver); } public PerChannelBookieClient(OrderedExecutor executor, EventLoopGroup eventLoopGroup, - BookieSocketAddress addr, + BookieId bookieId, ClientAuthProvider.Factory authProviderFactory, - ExtensionRegistry extRegistry) throws SecurityException { - this(new ClientConfiguration(), executor, eventLoopGroup, addr, NullStatsLogger.INSTANCE, - authProviderFactory, extRegistry, null); + ExtensionRegistry extRegistry, BookieAddressResolver bookieAddressResolver) + throws SecurityException { + this(new ClientConfiguration(), executor, eventLoopGroup, bookieId, + NullStatsLogger.INSTANCE, + authProviderFactory, extRegistry, null, bookieAddressResolver); } public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor, - EventLoopGroup eventLoopGroup, BookieSocketAddress addr, + EventLoopGroup eventLoopGroup, BookieId bookieId, StatsLogger parentStatsLogger, ClientAuthProvider.Factory authProviderFactory, ExtensionRegistry extRegistry, - PerChannelBookieClientPool pcbcPool) throws SecurityException { - this(conf, executor, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, addr, NullStatsLogger.INSTANCE, - authProviderFactory, extRegistry, pcbcPool, null); + PerChannelBookieClientPool pcbcPool, BookieAddressResolver bookieAddressResolver) + throws SecurityException { + this(conf, executor, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, bookieId, NullStatsLogger.INSTANCE, + authProviderFactory, extRegistry, pcbcPool, null, bookieAddressResolver); } public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor, EventLoopGroup eventLoopGroup, ByteBufAllocator allocator, - BookieSocketAddress addr, + BookieId bookieId, StatsLogger parentStatsLogger, ClientAuthProvider.Factory authProviderFactory, ExtensionRegistry extRegistry, PerChannelBookieClientPool pcbcPool, - SecurityHandlerFactory shFactory) throws SecurityException { + SecurityHandlerFactory shFactory, + BookieAddressResolver bookieAddressResolver) throws SecurityException { this.maxFrameSize = conf.getNettyMaxFrameSizeBytes(); this.conf = conf; - this.addr = addr; + this.bookieId = bookieId; + this.bookieAddressResolver = bookieAddressResolver; this.executor = executor; - if (LocalBookiesRegistry.isLocalBookie(addr)) { + if (LocalBookiesRegistry.isLocalBookie(bookieId)) { this.eventLoopGroup = new DefaultEventLoopGroup(); } else { this.eventLoopGroup = eventLoopGroup; @@ -389,7 +407,7 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor } this.statsLogger = parentStatsLogger.scope(BookKeeperClientStats.CHANNEL_SCOPE) - .scope(buildStatsLoggerScopeName(addr)); + .scope(buildStatsLoggerScopeName(bookieId)); readEntryOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_READ_OP); addEntryOpLogger = statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_ADD_OP); @@ -489,9 +507,9 @@ public boolean isSecure() { }; } - public static String buildStatsLoggerScopeName(BookieSocketAddress addr) { + public static String buildStatsLoggerScopeName(BookieId addr) { StringBuilder nameBuilder = new StringBuilder(); - nameBuilder.append(addr.getHostName().replace('.', '_').replace('-', '_')).append("_").append(addr.getPort()); + nameBuilder.append(addr.toString().replace('.', '_').replace('-', '_').replace(":", "_")); return nameBuilder.toString(); } @@ -516,7 +534,14 @@ protected long getNumPendingCompletionRequests() { protected ChannelFuture connect() { final long startTime = MathUtils.nowInNano(); if (LOG.isDebugEnabled()) { - LOG.debug("Connecting to bookie: {}", addr); + LOG.debug("Connecting to bookie: {}", bookieId); + } + BookieSocketAddress addr; + try { + addr = bookieAddressResolver.resolve(bookieId); + } catch (BookieAddressResolver.BookieIdNotResolvedException err) { + LOG.error("Cannot connect to {} as endpopint resolution failed", bookieId, err); + return processBookieNotResolvedError(startTime, err); } // Set up the ClientBootStrap so we can create a new Channel connection to the bookie. @@ -688,7 +713,7 @@ void forceLedger(final long ledgerId, ForceLedgerCallback cb, Object ctx) { if (useV2WireProtocol) { LOG.error("force is not allowed with v2 protocol"); executor.executeOrdered(ledgerId, () -> { - cb.forceLedgerComplete(BKException.Code.IllegalOpException, ledgerId, addr, ctx); + cb.forceLedgerComplete(BKException.Code.IllegalOpException, ledgerId, bookieId, ctx); }); return; } @@ -744,7 +769,7 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBuf if (writeFlags.contains(WriteFlag.DEFERRED_SYNC)) { LOG.error("invalid writeflags {} for v2 protocol", writeFlags); executor.executeOrdered(ledgerId, () -> { - cb.writeComplete(BKException.Code.IllegalOpException, ledgerId, entryId, addr, ctx); + cb.writeComplete(BKException.Code.IllegalOpException, ledgerId, entryId, bookieId, ctx); }); return; } @@ -1002,7 +1027,7 @@ public void checkTimeoutOnPendingOperations() { if (timedOutOperations > 0) { LOG.info("Timed-out {} operations to channel {} for {}", - timedOutOperations, channel, addr); + timedOutOperations, channel, bookieId); } } @@ -1014,7 +1039,7 @@ public void disconnect() { } public void disconnect(boolean wait) { - LOG.info("Disconnecting the per channel bookie client for {}", addr); + LOG.info("Disconnecting the per channel bookie client for {}", bookieId); closeInternal(false, wait); } @@ -1026,7 +1051,7 @@ public void close() { } public void close(boolean wait) { - LOG.info("Closing the per channel bookie client for {}", addr); + LOG.info("Closing the per channel bookie client for {}", bookieId); closeLock.writeLock().lock(); try { if (ConnectionState.CLOSED == state) { @@ -1318,7 +1343,7 @@ private void readV2Response(final BookieProtocol.Response response) { if (null == completionValue) { // Unexpected response, so log it. The txnId should have been present. if (LOG.isDebugEnabled()) { - LOG.debug("Unexpected response received from bookie : " + addr + " for type : " + operationType + LOG.debug("Unexpected response received from bookie : " + bookieId + " for type : " + operationType + " and ledger:entry : " + response.ledgerId + ":" + response.entryId); } response.release(); @@ -1434,7 +1459,7 @@ private void readV3Response(final Response response) { if (null == completionValue) { // Unexpected response, so log it. The txnId should have been present. if (LOG.isDebugEnabled()) { - LOG.debug("Unexpected response received from bookie : " + addr + " for type : " + LOG.debug("Unexpected response received from bookie : " + bookieId + " for type : " + header.getOperation() + " and txnId : " + header.getTxnId()); } } else { @@ -1471,7 +1496,7 @@ public void operationComplete(Future future) throws Exception { synchronized (PerChannelBookieClient.this) { if (future.isSuccess() && state == ConnectionState.CONNECTING) { - LOG.error("Connection state changed before TLS handshake completed {}/{}", addr, state); + LOG.error("Connection state changed before TLS handshake completed {}/{}", bookieId, state); rc = BKException.Code.BookieHandleNotAvailableException; closeChannel(channel); channel = null; @@ -1480,7 +1505,7 @@ public void operationComplete(Future future) throws Exception { } } else if (future.isSuccess() && state == ConnectionState.START_TLS) { rc = BKException.Code.OK; - LOG.info("Successfully connected to bookie using TLS: " + addr); + LOG.info("Successfully connected to bookie using TLS: " + bookieId); state = ConnectionState.CONNECTED; AuthHandler.ClientSideHandler authHandler = future.get().pipeline() @@ -1507,7 +1532,7 @@ public void operationComplete(Future future) throws Exception { return; // pendingOps should have been completed when other channel connected } else { LOG.error("TLS handshake failed with bookie: {}/{}, current state {} : ", - channel, addr, state, future.cause()); + channel, bookieId, state, future.cause()); rc = BKException.Code.SecurityException; closeChannel(channel); channel = null; @@ -1599,7 +1624,7 @@ void timeout() { protected void logResponse(StatusCode status, Object... extraInfo) { if (LOG.isDebugEnabled()) { - LOG.debug("Got {} response from bookie:{} rc:{}, {}", operationName, addr, status, + LOG.debug("Got {} response from bookie:{} rc:{}, {}", operationName, bookieId, status, Joiner.on(":").join(extraInfo)); } } @@ -1609,7 +1634,7 @@ protected int convertStatus(StatusCode status, int defaultStatus) { int rcToRet = statusCodeToExceptionCode(status); if (rcToRet == BKException.Code.UNINITIALIZED) { LOG.error("{} for failed on bookie {} code {}", - operationName, addr, status); + operationName, bookieId, status); return defaultStatus; } else { return rcToRet; @@ -1670,7 +1695,7 @@ public WriteLacCompletion(final CompletionKey key, this.cb = new WriteLacCallback() { @Override public void writeLacComplete(int rc, long ledgerId, - BookieSocketAddress addr, + BookieId addr, Object ctx) { logOpResult(rc); originalCallback.writeLacComplete(rc, ledgerId, @@ -1688,7 +1713,7 @@ public void errorOut() { @Override public void errorOut(final int rc) { errorOutAndRunCallback( - () -> cb.writeLacComplete(rc, ledgerId, addr, ctx)); + () -> cb.writeLacComplete(rc, ledgerId, bookieId, ctx)); } @Override @@ -1702,7 +1727,7 @@ public void handleV3Response(BookkeeperProtocol.Response response) { logResponse(status, "ledger", ledgerId); } int rc = convertStatus(status, BKException.Code.WriteException); - cb.writeLacComplete(rc, ledgerId, addr, ctx); + cb.writeLacComplete(rc, ledgerId, bookieId, ctx); } } @@ -1719,7 +1744,7 @@ public ForceLedgerCompletion(final CompletionKey key, this.cb = new ForceLedgerCallback() { @Override public void forceLedgerComplete(int rc, long ledgerId, - BookieSocketAddress addr, + BookieId addr, Object ctx) { logOpResult(rc); originalCallback.forceLedgerComplete(rc, ledgerId, @@ -1737,7 +1762,7 @@ public void errorOut() { @Override public void errorOut(final int rc) { errorOutAndRunCallback( - () -> cb.forceLedgerComplete(rc, ledgerId, addr, ctx)); + () -> cb.forceLedgerComplete(rc, ledgerId, bookieId, ctx)); } @Override @@ -1751,7 +1776,7 @@ public void handleV3Response(BookkeeperProtocol.Response response) { logResponse(status, "ledger", ledgerId); } int rc = convertStatus(status, BKException.Code.WriteException); - cb.forceLedgerComplete(rc, ledgerId, addr, ctx); + cb.forceLedgerComplete(rc, ledgerId, bookieId, ctx); } } @@ -2116,7 +2141,7 @@ void reset(final CompletionKey key, @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, + BookieId addr, Object ctx) { logOpResult(rc); originalCallback.writeComplete(rc, ledgerId, entryId, addr, ctx); @@ -2142,7 +2167,7 @@ public void errorOut() { @Override public void errorOut(final int rc) { errorOutAndRunCallback( - () -> writeComplete(rc, ledgerId, entryId, addr, ctx)); + () -> writeComplete(rc, ledgerId, entryId, bookieId, ctx)); } @Override @@ -2176,7 +2201,7 @@ private void handleResponse(long ledgerId, long entryId, } int rc = convertStatus(status, BKException.Code.WriteException); - writeComplete(rc, ledgerId, entryId, addr, ctx); + writeComplete(rc, ledgerId, entryId, bookieId, ctx); } } @@ -2395,7 +2420,7 @@ class ConnectionFutureListener implements ChannelFutureListener { } @Override - public void operationComplete(ChannelFuture future) throws Exception { + public void operationComplete(ChannelFuture future) { if (LOG.isDebugEnabled()) { LOG.debug("Channel connected ({}) {}", future.isSuccess(), future.channel()); } @@ -2422,13 +2447,13 @@ public void operationComplete(ChannelFuture future) throws Exception { initiateTLS(); return; } else { - LOG.info("Successfully connected to bookie: " + addr); + LOG.info("Successfully connected to bookie: " + bookieId); state = ConnectionState.CONNECTED; activeNonTlsChannelCounter.inc(); } } else if (future.isSuccess() && state == ConnectionState.START_TLS) { rc = BKException.Code.OK; - LOG.info("Successfully connected to bookie using TLS: " + addr); + LOG.info("Successfully connected to bookie using TLS: " + bookieId); state = ConnectionState.CONNECTED; AuthHandler.ClientSideHandler authHandler = future.channel().pipeline() @@ -2458,15 +2483,18 @@ public void operationComplete(ChannelFuture future) throws Exception { if (cause instanceof UnknownHostException || cause instanceof NativeIoException) { // Don't log stack trace for common errors LOG.warn("Could not connect to bookie: {}/{}, current state {} : {}", - future.channel(), addr, state, future.cause().getMessage()); + future.channel(), bookieId, state, future.cause().getMessage()); } else { // Regular exceptions, include stack trace LOG.error("Could not connect to bookie: {}/{}, current state {} : ", - future.channel(), addr, state, future.cause()); + future.channel(), bookieId, state, future.cause()); } rc = BKException.Code.BookieHandleNotAvailableException; - closeChannel(future.channel()); + Channel failedChannel = future.channel(); + if (failedChannel != null) { // can be null in case of dummy failed ChannelFuture + closeChannel(failedChannel); + } channel = null; if (state != ConnectionState.CLOSED) { state = ConnectionState.DISCONNECTED; @@ -2521,4 +2549,130 @@ private void failTLS(int rc) { } failedTlsHandshakeCounter.inc(); } + + private static class FailedChannelFutureImpl implements ChannelFuture { + + private final Throwable failureCause; + public FailedChannelFutureImpl(Throwable failureCause) { + this.failureCause = failureCause; + } + + @Override + public Channel channel() { + // used only for log + return null; + } + + @Override + public ChannelFuture addListener(GenericFutureListener> listener) { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + @SuppressWarnings({"unchecked", "varargs"}) + public ChannelFuture addListeners(GenericFutureListener>... listeners) { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public ChannelFuture removeListener(GenericFutureListener> listener) { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + @SuppressWarnings({"unchecked", "varargs"}) + public ChannelFuture removeListeners(GenericFutureListener>... listeners) { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public ChannelFuture sync() throws InterruptedException { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public ChannelFuture syncUninterruptibly() { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public ChannelFuture await() throws InterruptedException { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public ChannelFuture awaitUninterruptibly() { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public boolean isVoid() { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public boolean isSuccess() { + return false; + } + + @Override + public boolean isCancellable() { + return false; + } + + @Override + public Throwable cause() { + return failureCause; + } + + @Override + public boolean await(long timeout, TimeUnit unit) throws InterruptedException { + return true; + } + + @Override + public boolean await(long timeoutMillis) throws InterruptedException { + return true; + } + + @Override + public boolean awaitUninterruptibly(long timeout, TimeUnit unit) { + return true; + } + + @Override + public boolean awaitUninterruptibly(long timeoutMillis) { + return true; + } + + @Override + public Void getNow() { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return false; + } + + @Override + public boolean isCancelled() { + return false; + } + + @Override + public boolean isDone() { + return true; + } + + @Override + public Void get() throws InterruptedException, ExecutionException { + throw new ExecutionException(failureCause); + } + + @Override + public Void get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + throw new ExecutionException(failureCause); + } + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientFactory.java index 48797cd968d..d4fd5750a60 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClientFactory.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.proto; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tls.SecurityException; import org.apache.bookkeeper.tls.SecurityHandlerFactory; @@ -36,7 +36,7 @@ interface PerChannelBookieClientFactory { * @return the client connected to address. * @throws SecurityException */ - PerChannelBookieClient create(BookieSocketAddress address, PerChannelBookieClientPool pcbcPool, + PerChannelBookieClient create(BookieId address, PerChannelBookieClientPool pcbcPool, SecurityHandlerFactory shFactory, boolean forceUseV3) throws SecurityException; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLastConfirmedAndEntryContext.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLastConfirmedAndEntryContext.java index 6c3a57f9465..dea6f6dd6a7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLastConfirmedAndEntryContext.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLastConfirmedAndEntryContext.java @@ -19,7 +19,7 @@ import java.util.Optional; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallbackCtx; /** @@ -28,11 +28,11 @@ public class ReadLastConfirmedAndEntryContext implements ReadEntryCallbackCtx { final int bookieIndex; - final BookieSocketAddress bookie; + final BookieId bookie; long lac = LedgerHandle.INVALID_ENTRY_ID; Optional lacUpdateTimestamp = Optional.empty(); - public ReadLastConfirmedAndEntryContext(int bookieIndex, BookieSocketAddress bookie) { + public ReadLastConfirmedAndEntryContext(int bookieIndex, BookieId bookie) { this.bookieIndex = bookieIndex; this.bookie = bookie; } @@ -41,7 +41,7 @@ public int getBookieIndex() { return bookieIndex; } - public BookieSocketAddress getBookieAddress() { + public BookieId getBookieAddress() { return bookie; } 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 c8af9f87d93..a61e0d5024f 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 @@ -27,7 +27,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.util.MathUtils; @@ -116,7 +116,7 @@ protected void processPacket() { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (BookieProtocol.EOK == rc) { requestProcessor.getRequestStats().getAddEntryStats() .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java index c8ea0672104..d8ed70ce215 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java @@ -31,7 +31,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; import org.apache.bookkeeper.client.api.WriteFlag; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperProtocol.AddRequest; import org.apache.bookkeeper.proto.BookkeeperProtocol.AddResponse; import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; @@ -78,7 +78,7 @@ private AddResponse getAddResponse() { BookkeeperInternalCallbacks.WriteCallback wcb = new BookkeeperInternalCallbacks.WriteCallback() { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (BookieProtocol.EOK == rc) { requestProcessor.getRequestStats().getAddEntryStats() .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java index 691102bae12..11e030df69d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java @@ -28,7 +28,7 @@ import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.bookie.BookieException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; @@ -69,7 +69,7 @@ private WriteLacResponse getWriteLacResponse() { BookkeeperInternalCallbacks.WriteCallback writeCallback = new BookkeeperInternalCallbacks.WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { if (BookieProtocol.EOK == rc) { requestProcessor.getRequestStats().getWriteLacStats() .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); 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 db6e3319dc7..48f510158cc 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 @@ -94,7 +94,7 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.meta.UnderreplicatedLedger; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.MultiCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; @@ -1015,12 +1015,12 @@ private void waitIfLedgerReplicationDisabled() throws UnavailableException, private List getAvailableBookies() throws BKException { // Get the available bookies - Collection availableBkAddresses = admin.getAvailableBookies(); - Collection readOnlyBkAddresses = admin.getReadOnlyBookies(); + Collection availableBkAddresses = admin.getAvailableBookies(); + Collection readOnlyBkAddresses = admin.getReadOnlyBookies(); availableBkAddresses.addAll(readOnlyBkAddresses); List availableBookies = new ArrayList(); - for (BookieSocketAddress addr : availableBkAddresses) { + for (BookieId addr : availableBkAddresses) { availableBookies.add(addr.toString()); } return availableBookies; @@ -1153,7 +1153,7 @@ private class ProcessLostFragmentsCb implements GenericCallback fragments) { if (rc == BKException.Code.OK) { - Set bookies = Sets.newHashSet(); + Set bookies = Sets.newHashSet(); for (LedgerFragment f : fragments) { bookies.addAll(f.getAddresses()); } @@ -1162,8 +1162,7 @@ public void operationComplete(int rc, Set fragments) { callback.processResult(Code.OK, null, null); return; } - publishSuspectedLedgersAsync( - bookies.stream().map(BookieSocketAddress::toString).collect(Collectors.toList()), + publishSuspectedLedgersAsync(bookies.stream().map(BookieId::toString).collect(Collectors.toList()), Sets.newHashSet(lh.getId()) ).whenComplete((result, cause) -> { if (null != cause) { @@ -1296,10 +1295,10 @@ public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { if (metadata.isClosed()) { boolean foundSegmentNotAdheringToPlacementPolicy = false; boolean foundSegmentSoftlyAdheringToPlacementPolicy = false; - for (Map.Entry> ensemble : metadata + for (Map.Entry> ensemble : metadata .getAllEnsembles().entrySet()) { long startEntryIdOfSegment = ensemble.getKey(); - List ensembleOfSegment = ensemble.getValue(); + List ensembleOfSegment = ensemble.getValue(); PlacementPolicyAdherence segmentAdheringToPlacementPolicy = admin .isEnsembleAdheringToPlacementPolicy(ensembleOfSegment, writeQuorumSize, ackQuorumSize); @@ -1380,17 +1379,17 @@ private static class MissingEntriesInfo { /* * segment details, like start entryid of the segment and ensemble List. */ - private final Entry> segmentEnsemble; + private final Entry> segmentEnsemble; // bookie missing these entries - private final BookieSocketAddress bookieMissingEntries; + private final BookieId bookieMissingEntries; /* * entries of this segment which are supposed to contain in this bookie * but missing in this bookie. */ private final List unavailableEntriesList; - private MissingEntriesInfo(long ledgerId, Entry> segmentEnsemble, - BookieSocketAddress bookieMissingEntries, List unavailableEntriesList) { + private MissingEntriesInfo(long ledgerId, Entry> segmentEnsemble, + BookieId bookieMissingEntries, List unavailableEntriesList) { this.ledgerId = ledgerId; this.segmentEnsemble = segmentEnsemble; this.bookieMissingEntries = bookieMissingEntries; @@ -1401,11 +1400,11 @@ private long getLedgerId() { return ledgerId; } - private Entry> getSegmentEnsemble() { + private Entry> getSegmentEnsemble() { return segmentEnsemble; } - private BookieSocketAddress getBookieMissingEntries() { + private BookieId getBookieMissingEntries() { return bookieMissingEntries; } @@ -1503,7 +1502,7 @@ public void accept(Versioned metadataVer, Throwable exception) { int ensembleSize = metadata.getEnsembleSize(); RoundRobinDistributionSchedule distributionSchedule = new RoundRobinDistributionSchedule(writeQuorumSize, ackQuorumSize, ensembleSize); - List>> segments = new LinkedList<>( + List>> segments = new LinkedList<>( metadata.getAllEnsembles().entrySet()); /* * since there are multiple segments, MultiCallback should be @@ -1511,11 +1510,11 @@ public void accept(Versioned metadataVer, Throwable exception) { */ MultiCallback mcbForThisLedger = new MultiCallback(ensembleSize * segments.size(), mcbForThisLedgerRange, null, BKException.Code.OK, BKException.Code.ReadException); - HashMap> bookiesSegmentInfoMap = - new HashMap>(); + HashMap> bookiesSegmentInfoMap = + new HashMap>(); for (int segmentNum = 0; segmentNum < segments.size(); segmentNum++) { - final Entry> segmentEnsemble = segments.get(segmentNum); - final List ensembleOfSegment = segmentEnsemble.getValue(); + final Entry> segmentEnsemble = segments.get(segmentNum); + final List ensembleOfSegment = segmentEnsemble.getValue(); final long startEntryIdOfSegment = segmentEnsemble.getKey(); final boolean lastSegment = (segmentNum == (segments.size() - 1)); final long lastEntryIdOfSegment = lastSegment ? lastEntryId @@ -1530,7 +1529,7 @@ public void accept(Versioned metadataVer, Throwable exception) { final boolean emptySegment = lastSegment ? (startEntryIdOfSegment > lastEntryId) : (startEntryIdOfSegment == segments.get(segmentNum + 1).getKey()); for (int bookieIndex = 0; bookieIndex < ensembleOfSegment.size(); bookieIndex++) { - final BookieSocketAddress bookieInEnsemble = ensembleOfSegment.get(bookieIndex); + final BookieId bookieInEnsemble = ensembleOfSegment.get(bookieIndex); final BitSet entriesStripedToThisBookie = emptySegment ? EMPTY_BITSET : distributionSchedule.getEntriesStripedToTheBookie(bookieIndex, startEntryIdOfSegment, lastEntryIdOfSegment); @@ -1560,9 +1559,9 @@ public void accept(Versioned metadataVer, Throwable exception) { lastEntryIdOfSegment, segmentEnsemble, entriesStripedToThisBookie)); } } - for (Entry> bookiesSegmentInfoTuple : + for (Entry> bookiesSegmentInfoTuple : bookiesSegmentInfoMap.entrySet()) { - final BookieSocketAddress bookieInEnsemble = bookiesSegmentInfoTuple.getKey(); + final BookieId bookieInEnsemble = bookiesSegmentInfoTuple.getKey(); final List bookieSegmentInfoList = bookiesSegmentInfoTuple .getValue(); admin.asyncGetListOfEntriesOfLedger(bookieInEnsemble, ledgerInRange) @@ -1576,11 +1575,11 @@ public void accept(Versioned metadataVer, Throwable exception) { private static class BookieExpectedToContainSegmentInfo { private final long startEntryIdOfSegment; private final long lastEntryIdOfSegment; - private final Entry> segmentEnsemble; + private final Entry> segmentEnsemble; private final BitSet entriesOfSegmentStripedToThisBookie; private BookieExpectedToContainSegmentInfo(long startEntryIdOfSegment, long lastEntryIdOfSegment, - Entry> segmentEnsemble, + Entry> segmentEnsemble, BitSet entriesOfSegmentStripedToThisBookie) { this.startEntryIdOfSegment = startEntryIdOfSegment; this.lastEntryIdOfSegment = lastEntryIdOfSegment; @@ -1596,7 +1595,7 @@ public long getLastEntryIdOfSegment() { return lastEntryIdOfSegment; } - public Entry> getSegmentEnsemble() { + public Entry> getSegmentEnsemble() { return segmentEnsemble; } @@ -1611,14 +1610,14 @@ private static class GetListOfEntriesOfLedgerCallbackForReplicasCheck private final int ensembleSize; private final int writeQuorumSize; private final int ackQuorumSize; - private final BookieSocketAddress bookieInEnsemble; + private final BookieId bookieInEnsemble; private final List bookieExpectedToContainSegmentInfoList; private final ConcurrentHashMap ledgersWithMissingEntries; private final ConcurrentHashMap ledgersWithUnavailableBookies; private final MultiCallback mcbForThisLedger; private GetListOfEntriesOfLedgerCallbackForReplicasCheck(long ledgerInRange, int ensembleSize, - int writeQuorumSize, int ackQuorumSize, BookieSocketAddress bookieInEnsemble, + int writeQuorumSize, int ackQuorumSize, BookieId bookieInEnsemble, List bookieExpectedToContainSegmentInfoList, ConcurrentHashMap ledgersWithMissingEntries, ConcurrentHashMap ledgersWithUnavailableBookies, @@ -1688,7 +1687,7 @@ public void accept(AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger, final long lastEntryIdOfSegment = bookieExpectedToContainSegmentInfo.getLastEntryIdOfSegment(); final BitSet entriesStripedToThisBookie = bookieExpectedToContainSegmentInfo .getEntriesOfSegmentStripedToThisBookie(); - final Entry> segmentEnsemble = + final Entry> segmentEnsemble = bookieExpectedToContainSegmentInfo.getSegmentEnsemble(); final List unavailableEntriesList = availabilityOfEntriesOfLedger .getUnavailableEntries(startEntryIdOfSegment, lastEntryIdOfSegment, entriesStripedToThisBookie); @@ -1856,7 +1855,7 @@ private void reportLedgersWithMissingEntries( for (int listInd = 0; listInd < missingEntriesInfoList.size(); listInd++) { MissingEntriesInfo missingEntriesInfo = missingEntriesInfoList.get(listInd); List unavailableEntriesList = missingEntriesInfo.getUnavailableEntriesList(); - Entry> segmentEnsemble = + Entry> segmentEnsemble = missingEntriesInfo.getSegmentEnsemble(); missingEntries.addAll(unavailableEntriesList); errMessage.append("In segment starting at " + segmentEnsemble.getKey() + " with ensemble " @@ -1907,7 +1906,7 @@ private void reportLedgersWithUnavailableBookies( errMessage.append("Ledger : " + ledgerWithUnavailableBookies + " has following unavailable bookies : "); for (int listInd = 0; listInd < missingBookiesInfoList.size(); listInd++) { MissingEntriesInfo missingBookieInfo = missingBookiesInfoList.get(listInd); - Entry> segmentEnsemble = + Entry> segmentEnsemble = missingBookieInfo.getSegmentEnsemble(); errMessage.append("In segment starting at " + segmentEnsemble.getKey() + " with ensemble " + segmentEnsemble.getValue() + ", following bookie has not responded " diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java index e644369ed7d..f396e5f8a17 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorElector.java @@ -43,7 +43,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.ZkLayoutManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.DataFormats.AuditorVoteFormat; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.Counter; @@ -352,7 +352,7 @@ Auditor getAuditor() { * Query zookeeper for the currently elected auditor. * @return the bookie id of the current auditor */ - public static BookieSocketAddress getCurrentAuditor(ServerConfiguration conf, ZooKeeper zk) + public static BookieId getCurrentAuditor(ServerConfiguration conf, ZooKeeper zk) throws KeeperException, InterruptedException, IOException { String electionRoot = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf) + '/' + BookKeeperConstants.UNDER_REPLICATION_NODE + '/' + ELECTION_ZNODE; @@ -368,9 +368,7 @@ public static BookieSocketAddress getCurrentAuditor(ServerConfiguration conf, Zo AuditorVoteFormat.Builder builder = AuditorVoteFormat.newBuilder(); TextFormat.merge(new String(data, UTF_8), builder); AuditorVoteFormat v = builder.build(); - String[] parts = v.getBookieId().split(":"); - return new BookieSocketAddress(parts[0], - Integer.parseInt(parts[1])); + return BookieId.parse(v.getBookieId()); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java index 026fd7262fc..f136f282cee 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/BookieLedgerIndexer.java @@ -28,7 +28,7 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.ReplicationException.BKAuditException; import org.apache.zookeeper.AsyncCallback; @@ -65,12 +65,11 @@ public Map> getBookieToLedgerIndex() Processor ledgerProcessor = new Processor() { @Override public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { - ledgerManager.readLedgerMetadata(ledgerId).whenComplete( - (metadata, exception) -> { + ledgerManager.readLedgerMetadata(ledgerId).whenComplete((metadata, exception) -> { if (exception == null) { - for (Map.Entry> ensemble + for (Map.Entry> ensemble : metadata.getValue().getAllEnsembles().entrySet()) { - for (BookieSocketAddress bookie : ensemble.getValue()) { + for (BookieId bookie : ensemble.getValue()) { putLedger(bookie2ledgersMap, bookie.toString(), ledgerId); } } 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 80cfed30ef2..524d07dba08 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 @@ -61,7 +61,7 @@ import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; @@ -468,10 +468,10 @@ private boolean isLastSegmentOpenAndMissingBookies(LedgerHandle lh) throws BKExc return false; } - SortedMap> ensembles = admin.getLedgerMetadata(lh).getAllEnsembles(); - List finalEnsemble = ensembles.get(ensembles.lastKey()); - Collection available = admin.getAvailableBookies(); - for (BookieSocketAddress b : finalEnsemble) { + SortedMap> ensembles = admin.getLedgerMetadata(lh).getAllEnsembles(); + List finalEnsemble = ensembles.get(ensembles.lastKey()); + Collection available = admin.getAvailableBookies(); + for (BookieId b : finalEnsemble) { if (!available.contains(b)) { if (LOG.isDebugEnabled()) { LOG.debug("Bookie {} is missing from the list of Available Bookies. ledger {}:ensemble {}.", diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java index 5295bb78a0b..19cd9925461 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/DecommissionService.java @@ -30,7 +30,7 @@ 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.net.BookieId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,9 +74,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { HashMap configMap = JsonUtil.fromJson(requestBody, HashMap.class); if (configMap != null && configMap.containsKey("bookie_src")) { try { - String[] bookieSrcString = configMap.get("bookie_src").split(":"); - BookieSocketAddress bookieSrc = new BookieSocketAddress( - bookieSrcString[0], Integer.parseInt(bookieSrcString[1])); + BookieId bookieSrc = BookieId.parse(configMap.get("bookie_src")); executor.execute(() -> { try { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookieInfoService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookieInfoService.java index b543cec5239..33a1007627a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookieInfoService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookieInfoService.java @@ -36,7 +36,7 @@ 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.net.BookieId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,7 +85,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { clientConf.setDiskWeightBasedPlacementEnabled(true); BookKeeper bk = new BookKeeper(clientConf); - Map map = bk.getBookieInfo(); + Map map = bk.getBookieInfo(); if (map.size() == 0) { bk.close(); response.setCode(HttpServer.StatusCode.NOT_FOUND); @@ -104,7 +104,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { */ LinkedHashMap output = Maps.newLinkedHashMapWithExpectedSize(map.size()); Long totalFree = 0L, total = 0L; - for (Map.Entry infoEntry : map.entrySet()) { + for (Map.Entry infoEntry : map.entrySet()) { BookieInfoReader.BookieInfo bInfo = infoEntry.getValue(); output.put(infoEntry.getKey().toString(), ": {Free: " + bInfo.getFreeDiskSpace() + getReadable(bInfo.getFreeDiskSpace()) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookiesService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookiesService.java index f969ff77978..7269b2efcd3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookiesService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListBookiesService.java @@ -33,6 +33,7 @@ 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.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,7 +60,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { HttpServiceResponse response = new HttpServiceResponse(); // GET if (HttpServer.Method.GET == request.getMethod()) { - Collection bookies = new ArrayList(); + Collection bookies = new ArrayList(); Map params = request.getParams(); // default print rw @@ -79,9 +80,14 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { // output Map output = Maps.newHashMap(); - for (BookieSocketAddress b : bookies) { - output.putIfAbsent(b.toString(), printHostname ? b.getHostName() : null); - LOG.debug("bookie: " + b.toString() + " hostname:" + b.getHostName()); + for (BookieId b : bookies) { + String hostname = null; + if (printHostname) { + BookieSocketAddress resolved = bka.getBookieAddressResolver().resolve(b); + hostname = resolved.getHostName(); + } + output.putIfAbsent(b.toString(), hostname); + LOG.debug("bookie: " + b.toString() + " hostname:" + hostname); } String jsonResponse = JsonUtil.toJson(output); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/RecoveryBookieService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/RecoveryBookieService.java index 4a0a0a38a35..a1ca6155587 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/RecoveryBookieService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/RecoveryBookieService.java @@ -34,7 +34,7 @@ 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.net.BookieId; import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -105,12 +105,11 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { if (HttpServer.Method.PUT == request.getMethod() && !requestJsonBody.bookieSrc.isEmpty()) { runFunctionWithRegistrationManager(conf, rm -> { - String[] bookieSrcString = requestJsonBody.bookieSrc.get(0).split(":"); - BookieSocketAddress bookieSrc = new BookieSocketAddress( - bookieSrcString[0], Integer.parseInt(bookieSrcString[1])); - boolean deleteCookie = requestJsonBody.deleteCookie; + final String bookieSrcSerialized = requestJsonBody.bookieSrc.get(0); executor.execute(() -> { try { + BookieId bookieSrc = BookieId.parse(bookieSrcSerialized); + boolean deleteCookie = requestJsonBody.deleteCookie; LOG.info("Start recovering bookie."); bka.recoverBookieData(bookieSrc); if (deleteCookie) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/WhoIsAuditorService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/WhoIsAuditorService.java index 9154ccd6433..8bb78247afa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/WhoIsAuditorService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/WhoIsAuditorService.java @@ -25,7 +25,7 @@ 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.net.BookieId; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; @@ -57,7 +57,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { HttpServiceResponse response = new HttpServiceResponse(); if (HttpServer.Method.GET == request.getMethod()) { - BookieSocketAddress bookieId = null; + BookieId bookieId = null; try { bookieId = AuditorElector.getCurrentAuditor(conf, zk); @@ -74,10 +74,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { } response.setCode(HttpServer.StatusCode.OK); - response.setBody("Auditor: " - + bookieId.getSocketAddress().getAddress().getCanonicalHostName() + "/" - + bookieId.getSocketAddress().getAddress().getHostAddress() + ":" - + bookieId.getSocketAddress().getPort()); + response.setBody("Auditor: " + bookieId); LOG.debug("response body:" + response.getBody()); return response; } else { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommand.java index 9653db8c469..3133ede61d5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommand.java @@ -18,14 +18,13 @@ */ package org.apache.bookkeeper.tools.cli.commands.autorecovery; -import static org.apache.bookkeeper.tools.cli.helpers.CommandHelpers.getBookieSocketAddrStringRepresentation; import com.google.common.util.concurrent.UncheckedExecutionException; import java.io.IOException; import java.net.URI; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; @@ -74,12 +73,12 @@ public boolean getAuditor(ServerConfiguration conf) .connectString(zkServers) .sessionTimeoutMs(conf.getZkTimeout()) .build(); - BookieSocketAddress bookieId = AuditorElector.getCurrentAuditor(conf, zk); + BookieId bookieId = AuditorElector.getCurrentAuditor(conf, zk); if (bookieId == null) { LOG.info("No auditor elected"); return false; } - LOG.info("Auditor: " + getBookieSocketAddrStringRepresentation(bookieId)); + LOG.info("Auditor: " + bookieId); } finally { if (zk != null) { zk.close(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java index 701d562838c..adcf2fd503b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java @@ -32,7 +32,7 @@ import org.apache.bookkeeper.client.UpdateLedgerOp; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -144,9 +144,9 @@ private boolean updateLedger(ServerConfiguration conf, FlipBookieIdFlags flags) final BookKeeperAdmin admin = new BookKeeperAdmin(bk); final UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, admin); final ServerConfiguration serverConfiguration = new ServerConfiguration(conf); - final BookieSocketAddress newBookieId = Bookie.getBookieAddress(serverConfiguration); + final BookieId newBookieId = Bookie.getBookieId(serverConfiguration); serverConfiguration.setUseHostNameAsBookieID(!flags.hostname); - final BookieSocketAddress oldBookieId = Bookie.getBookieAddress(serverConfiguration); + final BookieId oldBookieId = Bookie.getBookieId(serverConfiguration); BookieShell.UpdateLedgerNotifier progressable = new BookieShell.UpdateLedgerNotifier() { long lastReport = System.nanoTime(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommand.java index 542216f2727..664c7c8da93 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommand.java @@ -34,7 +34,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.exceptions.MetadataException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.tools.cli.commands.bookie.ListLedgersCommand.ListLedgersFlags; import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; @@ -121,8 +121,8 @@ private void initLedgerFrommat(ServerConfiguration conf, ListLedgersFlags cmdFla public boolean handler(ServerConfiguration conf, ListLedgersFlags flags) throws UnknownHostException, MetadataException, ExecutionException { - final BookieSocketAddress bookieAddress = StringUtils.isBlank(flags.bookieId) ? null : - new BookieSocketAddress(flags.bookieId); + final BookieId bookieAddress = StringUtils.isBlank(flags.bookieId) ? null : + BookieId.parse(flags.bookieId); runFunctionWithLedgerManagerFactory(conf, mFactory -> { try (LedgerManager ledgerManager = mFactory.newLedgerManager()) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommand.java index 98c1b11cf11..f41ce5ec2fe 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommand.java @@ -40,7 +40,7 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.proto.BookieProtocol; @@ -143,10 +143,10 @@ private boolean readledger(ServerConfiguration serverConf, ReadLedgerFlags flags long lastEntry = flags.lastEntryId; - final BookieSocketAddress bookie; + final BookieId bookie; if (flags.bookieAddresss != null) { // A particular bookie was specified - bookie = new BookieSocketAddress(flags.bookieAddresss); + bookie = BookieId.parse(flags.bookieAddresss); } else { bookie = null; } @@ -184,7 +184,8 @@ private boolean readledger(ServerConfiguration serverConf, ReadLedgerFlags flags new DefaultThreadFactory("BookKeeperClientSchedulerPool")); BookieClient bookieClient = new BookieClientImpl(conf, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, - executor, scheduler, NullStatsLogger.INSTANCE); + executor, scheduler, NullStatsLogger.INSTANCE, + bk.getBookieAddressResolver()); LongStream.range(flags.firstEntryId, lastEntry).forEach(entryId -> { CompletableFuture future = new CompletableFuture<>(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/UpdateBookieInLedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/UpdateBookieInLedgerCommand.java index 88d1b4d3e0d..6155027a442 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/UpdateBookieInLedgerCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/UpdateBookieInLedgerCommand.java @@ -31,7 +31,7 @@ import org.apache.bookkeeper.client.UpdateLedgerOp; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -108,13 +108,13 @@ public boolean apply(ServerConfiguration conf, UpdateBookieInLedgerFlags cmdFlag private boolean updateLedger(ServerConfiguration conf, UpdateBookieInLedgerFlags flags) throws InterruptedException, BKException, IOException { - BookieSocketAddress srcBookieAddress; - BookieSocketAddress destBookieAddress; + BookieId srcBookieAddress; + BookieId destBookieAddress; try { - String[] bookieAddress = flags.srcBookie.split(":"); - srcBookieAddress = new BookieSocketAddress(bookieAddress[0], Integer.parseInt(bookieAddress[1])); - bookieAddress = flags.destBookie.split(":"); - destBookieAddress = new BookieSocketAddress(bookieAddress[0], Integer.parseInt(bookieAddress[1])); + String bookieAddress = flags.srcBookie; + srcBookieAddress = BookieId.parse(bookieAddress); + bookieAddress = flags.destBookie; + destBookieAddress = BookieId.parse(bookieAddress); } catch (Exception e) { LOG.error("Bookie address must in
            : format"); return false; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java index c7aa537fedf..3466efe0253 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java @@ -32,7 +32,7 @@ import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -91,10 +91,9 @@ private boolean decommission(ServerConfiguration conf, DecommissionFlags flags) BookKeeperAdmin admin = new BookKeeperAdmin(adminConf); try { final String remoteBookieidToDecommission = flags.remoteBookieIdToDecommission; - final BookieSocketAddress bookieAddressToDecommission = (StringUtils.isBlank(remoteBookieidToDecommission) - ? Bookie.getBookieAddress(conf) - : new BookieSocketAddress( - remoteBookieidToDecommission)); + final BookieId bookieAddressToDecommission = (StringUtils.isBlank(remoteBookieidToDecommission) + ? Bookie.getBookieId(conf) + : BookieId.parse(remoteBookieidToDecommission)); admin.decommissionBookie(bookieAddressToDecommission); LOG.info("The ledgers stored in the given decommissioning bookie: {} are properly replicated", bookieAddressToDecommission); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/EndpointInfoCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/EndpointInfoCommand.java index 3ef7e7bf59e..1c098de5449 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/EndpointInfoCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/EndpointInfoCommand.java @@ -29,7 +29,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.BookieServiceInfo; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -84,8 +84,8 @@ private boolean getEndpointInfo(ServerConfiguration conf, EndpointInfoFlags flag if (bookieId == null || bookieId.isEmpty()) { throw new IllegalArgumentException("BookieId is required"); } - BookieSocketAddress address = new BookieSocketAddress(bookieId); - Collection allBookies = admin.getAllBookies(); + BookieId address = BookieId.parse(bookieId); + Collection allBookies = admin.getAllBookies(); if (!allBookies.contains(address)) { System.out.println("Bookie " + bookieId + " does not exist, only " + allBookies); return false; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java index 047c15e63b0..f433b7c2210 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommand.java @@ -29,7 +29,7 @@ import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; import org.apache.bookkeeper.tools.cli.helpers.CommandHelpers; import org.apache.bookkeeper.tools.framework.CliFlags; @@ -72,7 +72,7 @@ public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { ClientConfiguration clientConf = new ClientConfiguration(conf); clientConf.setDiskWeightBasedPlacementEnabled(true); try (BookKeeper bk = new BookKeeper(clientConf)) { - Map map = bk.getBookieInfo(); + Map map = bk.getBookieInfo(); if (map.size() == 0) { System.out.println("Failed to retrieve bookie information from any of the bookies"); bk.close(); @@ -81,10 +81,11 @@ public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { System.out.println("Free disk space info:"); long totalFree = 0, total = 0; - for (Map.Entry e : map.entrySet()) { + for (Map.Entry e : map.entrySet()) { BookieInfo bInfo = e.getValue(); - BookieSocketAddress bookieId = e.getKey(); - System.out.println(CommandHelpers.getBookieSocketAddrStringRepresentation(bookieId) + BookieId bookieId = e.getKey(); + System.out.println(CommandHelpers.getBookieSocketAddrStringRepresentation(bookieId, + bk.getBookieAddressResolver()) + ":\tFree: " + bInfo.getFreeDiskSpace() + getReadable(bInfo.getFreeDiskSpace()) + "\tTotal: " + bInfo.getTotalDiskSpace() + getReadable(bInfo.getTotalDiskSpace())); } @@ -93,7 +94,7 @@ public boolean apply(ServerConfiguration conf, CliFlags cmdFlags) { Map dedupedMap = map.entrySet() .stream() .collect(Collectors.toMap( - entry -> entry.getKey().getHostName(), + entry -> entry.getKey().toString(), entry -> entry.getValue(), (key1, key2) -> key2 )); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java index 33e1a2d74c0..ee988739d9b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommand.java @@ -26,8 +26,10 @@ import java.util.Set; import lombok.Setter; import lombok.experimental.Accessors; +import org.apache.bookkeeper.client.DefaultBookieAddressResolver; import org.apache.bookkeeper.discover.RegistrationClient; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand.Flags; import org.apache.bookkeeper.tools.cli.helpers.DiscoveryCommand; import org.apache.bookkeeper.tools.framework.CliFlags; @@ -80,32 +82,32 @@ protected void run(RegistrationClient regClient, Flags flags) throws Exception { boolean hasBookies = false; if (flags.readwrite) { - Set bookies = result( + Set bookies = result( regClient.getWritableBookies() ).getValue(); if (!bookies.isEmpty()) { System.out.println("ReadWrite Bookies :"); - printBookies(bookies); + printBookies(bookies, new DefaultBookieAddressResolver(regClient)); hasBookies = true; } } if (flags.readonly) { - Set bookies = result( + Set bookies = result( regClient.getReadOnlyBookies() ).getValue(); if (!bookies.isEmpty()) { System.out.println("Readonly Bookies :"); - printBookies(bookies); + printBookies(bookies, new DefaultBookieAddressResolver(regClient)); hasBookies = true; } } if (flags.all) { - Set bookies = result( + Set bookies = result( regClient.getAllBookies() ).getValue(); if (!bookies.isEmpty()) { System.out.println("All Bookies :"); - printBookies(bookies); + printBookies(bookies, new DefaultBookieAddressResolver(regClient)); hasBookies = true; } } @@ -114,9 +116,9 @@ protected void run(RegistrationClient regClient, Flags flags) throws Exception { } } - private static void printBookies(Collection bookies) { - for (BookieSocketAddress b : bookies) { - System.out.println(getBookieSocketAddrStringRepresentation(b)); + private static void printBookies(Collection bookies, BookieAddressResolver bookieAddressResolver) { + for (BookieId b : bookies) { + System.out.println(getBookieSocketAddrStringRepresentation(b, bookieAddressResolver)); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java index c1a169495a0..0cce375d5d5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommand.java @@ -39,7 +39,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -123,16 +123,15 @@ private boolean recover(ServerConfiguration conf, RecoverFlags flags) // Get bookies list final String[] bookieStrs = flags.bookieAddress.split(","); - final Set bookieAddrs = new HashSet<>(); + final Set bookieAddrs = new HashSet<>(); for (String bookieStr : bookieStrs) { - final String[] bookieStrParts = bookieStr.split(":"); - if (bookieStrParts.length != 2) { - System.err.println("BookieSrcs has invalid bookie address format (host:port expected) : " + try { + bookieAddrs.add(BookieId.parse(bookieStr)); + } catch (IllegalArgumentException err) { + System.err.println("BookieSrcs has invalid bookie id format: " + bookieStr); return false; } - bookieAddrs.add(new BookieSocketAddress(bookieStrParts[0], - Integer.parseInt(bookieStrParts[1]))); } if (!force) { @@ -160,7 +159,7 @@ private boolean recover(ServerConfiguration conf, RecoverFlags flags) } } - private boolean bkQuery(BookKeeperAdmin bkAdmin, Set bookieAddrs) + private boolean bkQuery(BookKeeperAdmin bkAdmin, Set bookieAddrs) throws InterruptedException, BKException { SortedMap ledgersContainBookies = bkAdmin.getLedgersContainBookies(bookieAddrs); @@ -180,14 +179,14 @@ private boolean bkQuery(BookKeeperAdmin bkAdmin, Set bookie return true; } - private Map inspectLedger(LedgerMetadata metadata, Set bookiesToInspect) { + private Map inspectLedger(LedgerMetadata metadata, Set bookiesToInspect) { Map numBookiesToReplacePerEnsemble = new TreeMap(); - for (Map.Entry> ensemble : + for (Map.Entry> ensemble : metadata.getAllEnsembles().entrySet()) { - List bookieList = ensemble.getValue(); + List bookieList = ensemble.getValue(); System.out.print(ensemble.getKey() + ":\t"); int numBookiesToReplace = 0; - for (BookieSocketAddress bookie : bookieList) { + for (BookieId bookie : bookieList) { System.out.print(bookie); if (bookiesToInspect.contains(bookie)) { System.out.print("*"); @@ -205,7 +204,7 @@ private Map inspectLedger(LedgerMetadata metadata, Set bookieAddrs, + Set bookieAddrs, boolean dryrun, boolean skipOpenLedgers, boolean removeCookies) @@ -218,12 +217,12 @@ private boolean bkRecoveryLedger(BookKeeperAdmin bkAdmin, } private void deleteCookies(ClientConfiguration conf, - Set bookieAddrs) throws BKException { + Set bookieAddrs) throws BKException { ServerConfiguration serverConf = new ServerConfiguration(conf); try { runFunctionWithRegistrationManager(serverConf, rm -> { try { - for (BookieSocketAddress addr : bookieAddrs) { + for (BookieId addr : bookieAddrs) { deleteCookie(rm, addr); } } catch (Exception e) { @@ -247,7 +246,7 @@ private void deleteCookies(ClientConfiguration conf, } - private void deleteCookie(RegistrationManager rm, BookieSocketAddress bookieSrc) throws BookieException { + private void deleteCookie(RegistrationManager rm, BookieId bookieSrc) throws BookieException { try { Versioned cookie = Cookie.readFromRegistrationManager(rm, bookieSrc); cookie.getValue().deleteFromRegistrationManager(rm, bookieSrc, cookie.getVersion()); @@ -257,7 +256,7 @@ private void deleteCookie(RegistrationManager rm, BookieSocketAddress bookieSrc) } private boolean bkRecovery(BookKeeperAdmin bkAdmin, - Set bookieAddrs, + Set bookieAddrs, boolean dryrun, boolean skipOpenLedgers, boolean removeCookies) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java index dce11ce1dfe..5cdefd04e98 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java @@ -35,7 +35,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.Cookie; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -149,7 +149,7 @@ private boolean updateBookieIdInCookie(ServerConfiguration bkConf, final boolean return runFunctionWithRegistrationManager(bkConf, rm -> { try { ServerConfiguration conf = new ServerConfiguration(bkConf); - String newBookieId = Bookie.getBookieAddress(conf).toString(); + String newBookieId = Bookie.getBookieId(conf).toString(); // read oldcookie Versioned oldCookie = null; try { @@ -247,7 +247,7 @@ private boolean expandStorage(ServerConfiguration bkConf) throws Exception { } private boolean listOrDeleteCookies(ServerConfiguration bkConf, boolean delete, boolean force) throws Exception { - BookieSocketAddress bookieAddress = Bookie.getBookieAddress(bkConf); + BookieId bookieAddress = Bookie.getBookieId(bkConf); File[] journalDirs = bkConf.getJournalDirs(); File[] ledgerDirs = bkConf.getLedgerDirs(); File[] indexDirs = bkConf.getIndexDirs(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CookieCommand.java index fe2c5318df0..d08f1f1eeff 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CookieCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CookieCommand.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.tools.cli.helpers.BookieShellCommand; import org.apache.bookkeeper.tools.common.BKCommand; @@ -85,7 +86,7 @@ protected boolean apply(ServiceURI serviceURI, } } - protected String getBookieId(CookieFlagsT cmdFlags) throws UnknownHostException { + protected BookieId getBookieId(CookieFlagsT cmdFlags) throws UnknownHostException { checkArgument( cmdFlags.arguments.size() == 1, "No bookie id or more bookie ids is specified"); @@ -100,7 +101,7 @@ protected String getBookieId(CookieFlagsT cmdFlags) throws UnknownHostException + " Bookie id should be in the format of ':'"); throw nhe; } - return bookieId; + return BookieId.parse(bookieId); } protected byte[] readCookieDataFromFile(String cookieFile) throws IOException { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommand.java index 430fed9dce8..46c597d49b2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommand.java @@ -27,6 +27,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.BookieException.CookieExistException; import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.commands.cookie.CreateCookieCommand.Flags; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -81,7 +82,7 @@ private CreateCookieCommand(Flags flags, PrintStream console) { @Override protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { - String bookieId = getBookieId(cmdFlags); + BookieId bookieId = getBookieId(cmdFlags); byte[] data = readCookieDataFromFile(cmdFlags.cookieFile); Versioned cookie = new Versioned<>(data, Version.NEW); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommand.java index 4c42615e8b6..73cb4e7f51e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommand.java @@ -26,6 +26,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.commands.cookie.DeleteCookieCommand.Flags; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -72,7 +73,7 @@ private DeleteCookieCommand(Flags flags, PrintStream console) { @Override protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { - String bookieId = getBookieId(cmdFlags); + BookieId bookieId = getBookieId(cmdFlags); try { rm.removeCookie(bookieId, new LongVersion(-1)); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java index 6e3d7bd22d7..3ef811602ea 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.bookie.Cookie; import org.apache.bookkeeper.bookie.Cookie.Builder; import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.commands.cookie.GenerateCookieCommand.Flags; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -99,7 +100,7 @@ private GenerateCookieCommand(Flags flags, PrintStream console) { @Override protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { - String bookieId = getBookieId(cmdFlags); + BookieId bookieId = getBookieId(cmdFlags); String instanceId; if (null == cmdFlags.instanceId) { @@ -109,7 +110,7 @@ protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { } Builder builder = Cookie.newBuilder(); - builder.setBookieHost(bookieId); + builder.setBookieHost(bookieId.toString()); if (StringUtils.isEmpty(instanceId)) { builder.setInstanceId(null); } else { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommand.java index 76f5f7c0099..598127a75ad 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommand.java @@ -27,7 +27,7 @@ import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.Cookie; import org.apache.bookkeeper.discover.RegistrationManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.commands.cookie.GetCookieCommand.Flags; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -74,12 +74,10 @@ private GetCookieCommand(Flags flags, PrintStream console) { @Override protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { - String bookieId = getBookieId(cmdFlags); + BookieId bookieId = getBookieId(cmdFlags); try { - Versioned cookie = Cookie.readFromRegistrationManager( - rm, new BookieSocketAddress(bookieId) - ); + Versioned cookie = Cookie.readFromRegistrationManager(rm, bookieId); spec.console().println("Cookie for bookie '" + bookieId + "' is:"); spec.console().println("---"); spec.console().println( diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommand.java index 77e5f05857b..4979b6ec0ed 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommand.java @@ -27,6 +27,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.commands.cookie.UpdateCookieCommand.Flags; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -81,7 +82,7 @@ private UpdateCookieCommand(Flags flags, PrintStream console) { @Override protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { - String bookieId = getBookieId(cmdFlags); + BookieId bookieId = getBookieId(cmdFlags); byte[] data = readCookieDataFromFile(cmdFlags.cookieFile); Versioned cookie = new Versioned<>(data, new LongVersion(-1L)); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java index a1dc1abd08c..de527ad62da 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/helpers/CommandHelpers.java @@ -22,7 +22,9 @@ import java.net.InetAddress; import lombok.AccessLevel; import lombok.NoArgsConstructor; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookieAddressResolver; /** @@ -38,16 +40,18 @@ public final class CommandHelpers { * When using hostname as bookie id, it's possible that the host is no longer valid and * can't get a ip from the hostname, so using UNKNOWN to indicate ip is unknown for the hostname */ - public static String getBookieSocketAddrStringRepresentation(BookieSocketAddress bookieId) { - String hostname = bookieId.getHostName(); - String bookieID = bookieId.toString(); + public static String getBookieSocketAddrStringRepresentation(BookieId bookidId, + BookieAddressResolver bookieAddressResolver) { + BookieSocketAddress networkAddress = bookieAddressResolver.resolve(bookidId); + String hostname = networkAddress.getHostName(); + String bookieID = networkAddress.toString(); String realHostname; String ip = null; if (InetAddresses.isInetAddress(hostname)){ ip = hostname; - realHostname = bookieId.getSocketAddress().getAddress().getCanonicalHostName(); + realHostname = networkAddress.getSocketAddress().getAddress().getCanonicalHostName(); } else { - InetAddress ia = bookieId.getSocketAddress().getAddress(); + InetAddress ia = networkAddress.getSocketAddress().getAddress(); if (null != ia){ ip = ia.getHostAddress(); } else { @@ -55,13 +59,13 @@ public static String getBookieSocketAddrStringRepresentation(BookieSocketAddress } realHostname = hostname; } - return formatBookieSocketAddress(bookieID, ip, bookieId.getPort(), realHostname); + return formatBookieSocketAddress(bookieID, ip, networkAddress.getPort(), realHostname); } /** * Format {@link BookieSocketAddress}. **/ - public static String formatBookieSocketAddress(String bookieId, String ip, int port, String hostName){ + public static String formatBookieSocketAddress(String bookieId, String ip, int port, String hostName) { return String.format("BookieID:%s, IP:%s, Port:%d, Hostname:%s", bookieId, ip, port, hostName); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java index 72e59b1a932..ad522c7a246 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java @@ -30,6 +30,7 @@ import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.util.IOUtils; @@ -71,16 +72,17 @@ public void testSetAdvertisedAddress() throws Exception { BookieSocketAddress bkAddress = new BookieSocketAddress("10.0.0.1", bookiePort); assertEquals(bkAddress, Bookie.getBookieAddress(conf)); + assertEquals(bkAddress.toBookieId(), Bookie.getBookieId(conf)); Bookie b = new Bookie(conf); b.start(); BookKeeperAdmin bka = new BookKeeperAdmin(baseClientConf); - Collection bookies = bka.getAvailableBookies(); + Collection bookies = bka.getAvailableBookies(); assertEquals(1, bookies.size()); - BookieSocketAddress address = bookies.iterator().next(); - assertEquals(bkAddress, address); + BookieId address = bookies.iterator().next(); + assertEquals(bkAddress.toBookieId(), address); b.shutdown(); bka.close(); @@ -100,6 +102,7 @@ public void testBothUseHostnameAndAdvertisedAddress() throws Exception { BookieSocketAddress bkAddress = new BookieSocketAddress("10.0.0.1", bookiePort); assertEquals(bkAddress, Bookie.getBookieAddress(conf)); + assertEquals(bkAddress.toBookieId(), Bookie.getBookieId(conf)); } } 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 f57d2685216..d627bf764ca 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 @@ -37,7 +37,6 @@ import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -95,7 +94,7 @@ import org.apache.bookkeeper.meta.exceptions.MetadataException; import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat; import org.apache.bookkeeper.replication.AutoRecoveryMain; @@ -299,7 +298,7 @@ public void testExitCodeZK_REG_FAIL() throws Exception { RegistrationManager rm = mock(RegistrationManager.class); doThrow(new MetadataStoreException("mocked exception")) .when(rm) - .registerBookie(anyString(), anyBoolean(), any(BookieServiceInfo.class)); + .registerBookie(any(BookieId.class), anyBoolean(), any(BookieServiceInfo.class)); // simulating ZooKeeper exception by assigning a closed zk client to bk BookieServer bkServer = new BookieServer(conf) { @@ -327,7 +326,7 @@ public void testBookieRegistrationWithSameZooKeeperClient() throws Exception { conf.setMetadataServiceUri(metadataServiceUri) .setListeningInterface(null); - String bookieId = Bookie.getBookieAddress(conf).toString(); + BookieId bookieId = Bookie.getBookieId(conf); driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); try (StateManager manager = new BookieStateManager(conf, driver)) { @@ -413,7 +412,8 @@ public void testBookieRegistrationWithFQDNHostNameAsBookieID() throws Exception .setUseHostNameAsBookieID(true) .setListeningInterface(null); - final String bookieId = InetAddress.getLocalHost().getCanonicalHostName() + ":" + conf.getBookiePort(); + final BookieId bookieId = + BookieId.parse(InetAddress.getLocalHost().getCanonicalHostName() + ":" + conf.getBookiePort()); driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); try (StateManager manager = new BookieStateManager(conf, driver)) { @@ -431,8 +431,8 @@ public void testBookieRegistrationWithShortHostNameAsBookieID() throws Exception .setUseShortHostName(true) .setListeningInterface(null); - final String bookieId = InetAddress.getLocalHost().getCanonicalHostName().split("\\.", 2)[0] - + ":" + conf.getBookiePort(); + final BookieId bookieId = BookieId.parse(InetAddress.getLocalHost().getCanonicalHostName().split("\\.", 2)[0] + + ":" + conf.getBookiePort()); driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); try (StateManager manager = new BookieStateManager(conf, driver)) { @@ -453,8 +453,8 @@ public void testRegNodeExistsAfterSessionTimeOut() throws Exception { .setMetadataServiceUri(metadataServiceUri) .setListeningInterface(null); - String bookieId = InetAddress.getLocalHost().getHostAddress() + ":" - + conf.getBookiePort(); + BookieId bookieId = BookieId.parse(InetAddress.getLocalHost().getHostAddress() + ":" + + conf.getBookiePort()); String bkRegPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf) + "/" + AVAILABLE_NODE + "/" + bookieId; driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); @@ -510,8 +510,8 @@ public void testBookieRegistrationBookieServiceInfo() throws Exception { .setUseShortHostName(true) .setListeningInterface(null); - final String bookieId = InetAddress.getLocalHost().getCanonicalHostName().split("\\.", 2)[0] - + ":" + conf.getBookiePort(); + final BookieId bookieId = BookieId.parse(InetAddress.getLocalHost().getCanonicalHostName().split("\\.", 2)[0] + + ":" + conf.getBookiePort()); String bkRegPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf) + "/" + AVAILABLE_NODE + "/" + bookieId; driver.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); @@ -764,7 +764,7 @@ public void testBookieStartException() throws Exception { Versioned newCookie = new Versioned<>( cookie.toString().getBytes(UTF_8), Version.NEW ); - driver.getRegistrationManager().writeCookie(Bookie.getBookieAddress(conf).toString(), newCookie); + driver.getRegistrationManager().writeCookie(Bookie.getBookieId(conf), newCookie); /* * Create LifecycleComponent for BookieServer and start it. @@ -1558,7 +1558,7 @@ private void bookieConnectAfterCookieDeleteWorker(ServerConfiguration conf, Regi Bookie b = new Bookie(conf); - final BookieSocketAddress bookieAddress = Bookie.getBookieAddress(conf); + final BookieId bookieAddress = Bookie.getBookieId(conf); // Read cookie from registation manager Versioned rmCookie = Cookie.readFromRegistrationManager(rm, bookieAddress); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java index 3ac507c8e98..ee55fb2ab33 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalForceTest.java @@ -45,7 +45,7 @@ import org.apache.bookkeeper.bookie.stats.JournalStats; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.test.TestStatsProvider; @@ -101,7 +101,7 @@ public void testAckAfterSync() throws Exception { long entryId = 0; journal.logAddEntry(ledgerId, entryId, DATA, false /* ackBeforeSync */, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latch.countDown(); } }, null); @@ -162,7 +162,7 @@ public void testAckBeforeSync() throws Exception { long entryId = 0; journal.logAddEntry(ledgerId, entryId, DATA, true /* ackBeforeSync */, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latch.countDown(); } }, null); @@ -225,7 +225,7 @@ public void testAckBeforeSyncWithJournalBufferedEntriesThreshold() throws Except for (long entryId = 0; entryId < numEntries; entryId++) { journal.logAddEntry(ledgerId, entryId, DATA, true /* ackBeforeSync */, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latch.countDown(); } }, null); @@ -277,13 +277,13 @@ public void testInterleavedRequests() throws Exception { for (long entryId = 0; entryId < numEntries; entryId++) { journal.logAddEntry(ledgerIdAckBeforeSync, entryId, DATA, true, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latchAckBeforeSynch.countDown(); } }, null); journal.logAddEntry(ledgerIdAckAfterSync, entryId, DATA, false, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latchAckAfterSynch.countDown(); } }, null); @@ -345,7 +345,7 @@ public void testForceLedger() throws Exception { long ledgerId = 1; journal.forceLedger(ledgerId, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latch.countDown(); } }, null); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java index 34d80f5e58f..538139d0e8f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java @@ -45,7 +45,7 @@ import org.apache.bookkeeper.bookie.Journal.LastLogMark; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.junit.Rule; import org.junit.Test; @@ -119,7 +119,7 @@ public void testAckAfterSyncPageCacheFlush() throws Exception { long startTime = System.currentTimeMillis(); journal.logAddEntry(ledgerId, entryId, DATA, false /* ackBeforeSync */, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latch.countDown(); } }, null); @@ -186,7 +186,7 @@ public void testAckBeforeSyncPageCacheFlush() throws Exception { long startTime = System.currentTimeMillis(); journal.logAddEntry(ledgerId, entryId, DATA, true /* ackBeforeSync */, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latch.countDown(); } }, null); @@ -248,7 +248,7 @@ public void testAckBeforeUnSyncPageCacheFlush() throws Exception { LogMark lastLogMarkBeforeWrite = journal.getLastLogMark().markLog().getCurMark(); journal.logAddEntry(ledgerId, entryId, DATA, true, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latch.countDown(); } }, null); @@ -262,7 +262,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre // add an entry to journal, wake up journal main thread which is blocked on queue.take() journal.logAddEntry(ledgerId, entryId + 1, DATA, true, new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { latch.countDown(); } }, null); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java index b7e73b988b5..8ab38c1466e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.same; import static org.mockito.Mockito.doReturn; @@ -49,6 +48,7 @@ import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand; import org.apache.bookkeeper.tools.cli.commands.bookies.RecoverCommand; @@ -147,7 +147,7 @@ public void setup() throws Exception { .setLayoutVersion(Cookie.CURRENT_COOKIE_LAYOUT_VERSION) .build(); this.version = new LongVersion(1L); - when(rm.readCookie(anyString())) + when(rm.readCookie(any(BookieId.class))) .thenReturn(new Versioned<>(cookie.toString().getBytes(UTF_8), version)); this.driver = mock(MetadataBookieDriver.class); @@ -188,7 +188,7 @@ public void testRecoverCmdMissingArgument() throws Exception { @Test public void testRecoverCmdInvalidBookieAddress() throws Exception { RecoverCmd cmd = (RecoverCmd) shell.commands.get("recover"); - CommandLine cmdLine = parseCommandLine(cmd, "127.0.0.1"); + CommandLine cmdLine = parseCommandLine(cmd, "non.valid$$bookie.id"); assertEquals(-1, cmd.runCmd(cmdLine)); PowerMockito.verifyNew(BookKeeperAdmin.class, never()).withArguments(any(ClientConfiguration.class)); } @@ -268,11 +268,11 @@ void testRecoverCmdRecoverLedger(long ledgerId, if (removeCookies) { PowerMockito.verifyStatic(MetadataDrivers.class); MetadataDrivers.runFunctionWithRegistrationManager(any(ServerConfiguration.class), any(Function.class)); - verify(rm, times(1)).readCookie(anyString()); - verify(rm, times(1)).removeCookie(anyString(), eq(version)); + verify(rm, times(1)).readCookie(any(BookieId.class)); + verify(rm, times(1)).removeCookie(any(BookieId.class), eq(version)); } else { - verify(rm, times(0)).readCookie(anyString()); - verify(rm, times(0)).removeCookie(anyString(), eq(version)); + verify(rm, times(0)).readCookie(any(BookieId.class)); + verify(rm, times(0)).removeCookie(any(BookieId.class), eq(version)); } } @@ -333,11 +333,11 @@ void testRecoverCmdRecover(boolean dryrun, if (removeCookies) { PowerMockito.verifyStatic(MetadataDrivers.class); MetadataDrivers.runFunctionWithRegistrationManager(any(ServerConfiguration.class), any(Function.class)); - verify(rm, times(1)).readCookie(anyString()); - verify(rm, times(1)).removeCookie(anyString(), eq(version)); + verify(rm, times(1)).readCookie(any(BookieId.class)); + verify(rm, times(1)).removeCookie(any(BookieId.class), eq(version)); } else { - verify(rm, times(0)).readCookie(anyString()); - verify(rm, times(0)).removeCookie(anyString(), eq(version)); + verify(rm, times(0)).readCookie(any(BookieId.class)); + verify(rm, times(0)).removeCookie(any(BookieId.class), eq(version)); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java index 27a5d28618b..9420661d17e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieWriteToJournalTest.java @@ -41,7 +41,7 @@ import org.apache.bookkeeper.client.api.BKException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.commons.lang3.mutable.MutableBoolean; import org.junit.Rule; @@ -91,7 +91,7 @@ public void testJournalLogAddEntryCalledCorrectly() throws Exception { conf.setJournalDirName(journalDir.getPath()) .setLedgerDirNames(new String[]{ledgerDir.getPath()}) .setMetadataServiceUri(null); - BookieSocketAddress bookieAddress = Bookie.getBookieAddress(conf); + BookieId bookieAddress = Bookie.getBookieId(conf); CountDownLatch journalJoinLatch = new CountDownLatch(1); Journal journal = mock(Journal.class); MutableBoolean effectiveAckBeforeSync = new MutableBoolean(false); @@ -128,7 +128,7 @@ public void testJournalLogAddEntryCalledCorrectly() throws Exception { final ByteBuf data = buildEntry(ledgerId, entryId, -1); final long expectedEntryId = entryId; b.addEntry(data, ackBeforeSync, (int rc, long ledgerId1, long entryId1, - BookieSocketAddress addr, Object ctx) -> { + BookieId addr, Object ctx) -> { assertSame(expectedCtx, ctx); assertEquals(ledgerId, ledgerId1); assertEquals(expectedEntryId, entryId1); @@ -171,7 +171,7 @@ public void testForceLedger() throws Exception { final ByteBuf data = buildEntry(ledgerId, entryId, -1); final long expectedEntryId = entryId; b.forceLedger(ledgerId, (int rc, long ledgerId1, long entryId1, - BookieSocketAddress addr, Object ctx) -> { + BookieId addr, Object ctx) -> { if (rc != BKException.Code.OK) { latchForceLedger1.completeExceptionally(org.apache.bookkeeper.client.BKException.create(rc)); return; @@ -181,7 +181,7 @@ public void testForceLedger() throws Exception { result(latchForceLedger1); b.addEntry(data, true /* ackBeforesync */, (int rc, long ledgerId1, long entryId1, - BookieSocketAddress addr, Object ctx) -> { + BookieId addr, Object ctx) -> { if (rc != BKException.Code.OK) { latchAddEntry.completeExceptionally(org.apache.bookkeeper.client.BKException.create(rc)); return; @@ -192,7 +192,7 @@ public void testForceLedger() throws Exception { // issue a new "forceLedger" b.forceLedger(ledgerId, (int rc, long ledgerId1, long entryId1, - BookieSocketAddress addr, Object ctx) -> { + BookieId addr, Object ctx) -> { if (rc != BKException.Code.OK) { latchForceLedger2.completeExceptionally(org.apache.bookkeeper.client.BKException.create(rc)); return; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java index 371971b7138..d5b454bf39e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GcOverreplicatedLedgerTest.java @@ -40,7 +40,7 @@ import org.apache.bookkeeper.meta.LedgerManagerTestCase; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.SnapshotMap; @@ -82,7 +82,7 @@ public void testGcOverreplicatedLedger() throws Exception { LedgerMetadata newLedgerMetadata = ledgerManager.readLedgerMetadata(lh.getId()).get().getValue(); - BookieSocketAddress bookieNotInEnsemble = getBookieNotInEnsemble(newLedgerMetadata); + BookieId bookieNotInEnsemble = getBookieNotInEnsemble(newLedgerMetadata); ServerConfiguration bkConf = getBkConf(bookieNotInEnsemble); bkConf.setGcOverreplicatedLedgerWaitTime(10, TimeUnit.MILLISECONDS); @@ -114,9 +114,9 @@ public void testNoGcOfLedger() throws Exception { activeLedgers.put(lh.getId(), true); LedgerMetadata newLedgerMetadata = ledgerManager.readLedgerMetadata(lh.getId()).get().getValue(); - BookieSocketAddress address = null; - SortedMap> ensembleMap = newLedgerMetadata.getAllEnsembles(); - for (List ensemble : ensembleMap.values()) { + BookieId address = null; + SortedMap> ensembleMap = newLedgerMetadata.getAllEnsembles(); + for (List ensemble : ensembleMap.values()) { address = ensemble.get(0); } ServerConfiguration bkConf = getBkConf(address); @@ -150,7 +150,7 @@ public void testNoGcIfLedgerBeingReplicated() throws Exception { activeLedgers.put(lh.getId(), true); LedgerMetadata newLedgerMetadata = ledgerManager.readLedgerMetadata(lh.getId()).get().getValue(); - BookieSocketAddress bookieNotInEnsemble = getBookieNotInEnsemble(newLedgerMetadata); + BookieId bookieNotInEnsemble = getBookieNotInEnsemble(newLedgerMetadata); ServerConfiguration bkConf = getBkConf(bookieNotInEnsemble); bkConf.setGcOverreplicatedLedgerWaitTime(10, TimeUnit.MILLISECONDS); @@ -182,13 +182,13 @@ public void clean(long ledgerId) { Assert.assertTrue(activeLedgers.containsKey(lh.getId())); } - private BookieSocketAddress getBookieNotInEnsemble(LedgerMetadata ledgerMetadata) throws UnknownHostException { - List allAddresses = Lists.newArrayList(); + private BookieId getBookieNotInEnsemble(LedgerMetadata ledgerMetadata) throws UnknownHostException { + List allAddresses = Lists.newArrayList(); for (BookieServer bk : bs) { - allAddresses.add(bk.getLocalAddress()); + allAddresses.add(bk.getBookieId()); } - SortedMap> ensembles = ledgerMetadata.getAllEnsembles(); - for (List fragmentEnsembles : ensembles.values()) { + SortedMap> ensembles = ledgerMetadata.getAllEnsembles(); + for (List fragmentEnsembles : ensembles.values()) { allAddresses.removeAll(fragmentEnsembles); } Assert.assertEquals(allAddresses.size(), 1); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java index 8258f23d4c2..63bd7f79e5e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerCacheTest.java @@ -46,7 +46,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; @@ -790,7 +790,7 @@ public void testSortedLedgerFlushFailure() throws Exception { assertTrue("Bookie is expected to be in Read mode", bookie.isReadOnly()); // write fail bookie.addEntry(generateEntry(1, 3), false, new BookkeeperInternalCallbacks.WriteCallback(){ - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx){ + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx){ LOG.info("fail write to bk"); assertTrue(rc != OK); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index 15253fbe04c..6d2b972a423 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -60,6 +60,7 @@ import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; @@ -164,8 +165,8 @@ public void testTriggerAudit(boolean storeSystemTimeAsLedgerUnderreplicatedMarkT assertTrue("There are supposed to be underreplicatedledgers", underreplicatedLedgerItr.hasNext()); UnderreplicatedLedger underreplicatedLedger = underreplicatedLedgerItr.next(); assertEquals("Underreplicated ledgerId", ledgerId, underreplicatedLedger.getLedgerId()); - assertTrue("Missingreplica of Underreplicated ledgerId should contain " + bookieToKill.getLocalAddress(), - underreplicatedLedger.getReplicaList().contains(bookieToKill.getLocalAddress().toString())); + assertTrue("Missingreplica of Underreplicated ledgerId should contain " + bookieToKill.getBookieId(), + underreplicatedLedger.getReplicaList().contains(bookieToKill.getBookieId().toString())); if (storeSystemTimeAsLedgerUnderreplicatedMarkTime) { long ctimeOfURL = underreplicatedLedger.getCtime(); assertTrue("ctime of underreplicated ledger should be greater than test starttime", @@ -441,7 +442,7 @@ public void testGetListOfEntriesOfNonExistingLedger() throws Exception { try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) { for (int i = 0; i < bs.size(); i++) { CompletableFuture futureResult = bkAdmin - .asyncGetListOfEntriesOfLedger(bs.get(i).getLocalAddress(), nonExistingLedgerId); + .asyncGetListOfEntriesOfLedger(bs.get(i).getBookieId(), nonExistingLedgerId); try { futureResult.get(); fail("asyncGetListOfEntriesOfLedger is supposed to be failed with NoSuchLedgerExistsException"); @@ -470,7 +471,7 @@ public void testGetListOfEntriesOfLedger(boolean isLedgerClosed) throws Exceptio try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) { for (int i = 0; i < bs.size(); i++) { CompletableFuture futureResult = bkAdmin - .asyncGetListOfEntriesOfLedger(bs.get(i).getLocalAddress(), lId); + .asyncGetListOfEntriesOfLedger(bs.get(i).getBookieId(), lId); AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = futureResult.get(); assertEquals("Number of entries", numOfEntries, availabilityOfEntriesOfLedger.getTotalNumOfAvailableEntries()); @@ -503,7 +504,7 @@ public void testGetListOfEntriesOfLedgerWithJustOneBookieInWriteQuorum() throws try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) { for (int i = 0; i < bs.size(); i++) { CompletableFuture futureResult = bkAdmin - .asyncGetListOfEntriesOfLedger(bs.get(i).getLocalAddress(), lId); + .asyncGetListOfEntriesOfLedger(bs.get(i).getBookieId(), lId); AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = futureResult.get(); /* * since num of bookies in the ensemble is 2 and @@ -529,27 +530,27 @@ public void testGetBookies() throws Exception { (zkc.exists(bookieCookiePath, false) != null)); try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) { - Collection availableBookies = bkAdmin.getAvailableBookies(); + Collection availableBookies = bkAdmin.getAvailableBookies(); Assert.assertEquals(availableBookies.size(), bs.size()); for (int i = 0; i < bs.size(); i++) { - availableBookies.contains(bs.get(i).getLocalAddress()); + availableBookies.contains(bs.get(i).getBookieId()); } BookieServer killedBookie = bs.get(1); killBookieAndWaitForZK(1); - Collection remainingBookies = bkAdmin.getAvailableBookies(); - Assert.assertFalse(remainingBookies.contains(killedBookie.getLocalAddress())); + Collection remainingBookies = bkAdmin.getAvailableBookies(); + Assert.assertFalse(remainingBookies.contains(killedBookie.getBookieId())); - Collection allBookies = bkAdmin.getAllBookies(); + Collection allBookies = bkAdmin.getAllBookies(); for (int i = 0; i < bs.size(); i++) { - remainingBookies.contains(bs.get(i).getLocalAddress()); - allBookies.contains(bs.get(i).getLocalAddress()); + remainingBookies.contains(bs.get(i).getBookieId()); + allBookies.contains(bs.get(i).getBookieId()); } Assert.assertEquals(remainingBookies.size(), allBookies.size() - 1); - Assert.assertTrue(allBookies.contains(killedBookie.getLocalAddress())); + Assert.assertTrue(allBookies.contains(killedBookie.getBookieId())); } } @@ -576,7 +577,7 @@ public void testGetListOfEntriesOfLedgerWithEntriesNotStripedToABookie() throws * since no entry is added, callback is supposed to fail with * NoSuchLedgerExistsException. */ - bkAdmin.asyncGetListOfEntriesOfLedger(bs.get(0).getLocalAddress(), lId) + bkAdmin.asyncGetListOfEntriesOfLedger(bs.get(0).getBookieId(), lId) .whenComplete((availabilityOfEntriesOfLedger, throwable) -> { exceptionInCallback.set(throwable != null); if (throwable != null) { @@ -595,17 +596,17 @@ public void testGetListOfEntriesOfLedgerWithEntriesNotStripedToABookie() throws public void testAreEntriesOfLedgerStoredInTheBookieForLastEmptySegment() throws Exception { int lastEntryId = 10; long ledgerId = 100L; - BookieSocketAddress bookie0 = new BookieSocketAddress("bookie0:3181"); - BookieSocketAddress bookie1 = new BookieSocketAddress("bookie1:3181"); - BookieSocketAddress bookie2 = new BookieSocketAddress("bookie2:3181"); - BookieSocketAddress bookie3 = new BookieSocketAddress("bookie3:3181"); + BookieId bookie0 = new BookieSocketAddress("bookie0:3181").toBookieId(); + BookieId bookie1 = new BookieSocketAddress("bookie1:3181").toBookieId(); + BookieId bookie2 = new BookieSocketAddress("bookie2:3181").toBookieId(); + BookieId bookie3 = new BookieSocketAddress("bookie3:3181").toBookieId(); - List ensembleOfSegment1 = new ArrayList(); + List ensembleOfSegment1 = new ArrayList(); ensembleOfSegment1.add(bookie0); ensembleOfSegment1.add(bookie1); ensembleOfSegment1.add(bookie2); - List ensembleOfSegment2 = new ArrayList(); + List ensembleOfSegment2 = new ArrayList(); ensembleOfSegment2.add(bookie3); ensembleOfSegment2.add(bookie1); ensembleOfSegment2.add(bookie2); @@ -681,7 +682,7 @@ private void testBookieServiceInfo(boolean readonly, boolean legacy) throws Exce ServerConfiguration bkConf = newServerConfiguration().setForceReadOnlyBookie(readonly); BookieServer bkServer = startBookie(bkConf); - String bookieId = bkServer.getLocalAddress().toString(); + String bookieId = bkServer.getBookieId().toString(); String host = bkServer.getLocalAddress().getHostName(); int port = bkServer.getLocalAddress().getPort(); 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 3bebc2145d2..168d9af6f62 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 @@ -45,7 +45,7 @@ import org.apache.bookkeeper.client.BKException.BKClientClosedException; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -480,7 +480,7 @@ public void testBookKeeperAdmin() throws Exception { LedgerHandle lh3 = createLedgerWithEntries(bk, 100); lh3.close(); - BookieSocketAddress bookieToKill = getBookie(0); + BookieId bookieToKill = getBookie(0); killBookie(bookieToKill); startNewBookie(); 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 33512ae883a..a7fe962bb4b 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,7 @@ import org.apache.bookkeeper.common.testing.annotations.FlakyTest; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.slf4j.Logger; @@ -57,7 +57,7 @@ class BookKeeperCheckInfoReader extends BookKeeper { super(conf); } - void blockUntilBookieWeightIs(BookieSocketAddress bookie, Optional target) throws InterruptedException { + void blockUntilBookieWeightIs(BookieId bookie, Optional target) throws InterruptedException { long startMsecs = System.currentTimeMillis(); Optional freeDiskSpace = Optional.empty(); while (System.currentTimeMillis() < (startMsecs + MS_WEIGHT_UPDATE_TIMEOUT)) { @@ -95,7 +95,7 @@ public long getTotalFreeSpace() { bsConfs.add(conf); BookieServer server = startBookie(conf, bookieWithCustomFreeDiskSpace); bs.add(server); - client.blockUntilBookieWeightIs(server.getLocalAddress(), Optional.of(initialFreeDiskSpace)); + client.blockUntilBookieWeightIs(server.getBookieId(), Optional.of(initialFreeDiskSpace)); if (useFinal == null) { ready.set(true); } @@ -114,7 +114,7 @@ private BookieServer replaceBookieWithCustomFreeDiskSpaceBookie( BookieServer bookie, final long freeDiskSpace) throws Exception { for (int i = 0; i < bs.size(); i++) { - if (bs.get(i).getLocalAddress().equals(bookie.getLocalAddress())) { + if (bs.get(i).getBookieId().equals(bookie.getBookieId())) { return replaceBookieWithCustomFreeDiskSpaceBookie(client, i, freeDiskSpace); } } @@ -125,7 +125,7 @@ private BookieServer replaceBookieWithCustomFreeDiskSpaceBookie( BookKeeperCheckInfoReader client, int bookieIdx, long initialFreeDiskSpace, long finalFreeDiskSpace, AtomicBoolean useFinal) throws Exception { - BookieSocketAddress addr = bs.get(bookieIdx).getLocalAddress(); + BookieId addr = bs.get(bookieIdx).getBookieId(); LOG.info("Killing bookie {}", addr); ServerConfiguration conf = killBookieAndWaitForZK(bookieIdx); client.blockUntilBookieWeightIs(addr, Optional.empty()); @@ -155,14 +155,14 @@ public void testDiskSpaceWeightedBookieSelection() throws Exception { replaceBookieWithCustomFreeDiskSpaceBookie(client, 0, multiple * freeDiskSpace); } } - Map m = new HashMap(); + Map m = new HashMap(); for (BookieServer b : bs) { - m.put(b.getLocalAddress(), 0); + m.put(b.getBookieId(), 0); } for (int i = 0; i < 2000; i++) { LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes()); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembleAt(0)) { m.put(b, m.get(b) + 1); } } @@ -170,12 +170,12 @@ public void testDiskSpaceWeightedBookieSelection() throws Exception { // make sure that bookies with higher weight(the last 2 bookies) are chosen 3X as often as the median; // since the number of ledgers created is small (2000), we allow a range of 2X to 4X instead of the exact 3X for (int i = 0; i < numBookies - 2; i++) { - double ratio1 = (double) m.get(bs.get(numBookies - 2).getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio1 = (double) m.get(bs.get(numBookies - 2).getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio1 - multiple), Math.abs(ratio1 - multiple) < 1); - double ratio2 = (double) m.get(bs.get(numBookies - 1).getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio2 = (double) m.get(bs.get(numBookies - 1).getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio2 - multiple), Math.abs(ratio2 - multiple) < 1); } @@ -205,14 +205,14 @@ public void testDiskSpaceWeightedBookieSelectionWithChangingWeights() throws Exc replaceBookieWithCustomFreeDiskSpaceBookie(client, 0, multiple * freeDiskSpace); } } - Map m = new HashMap(); + Map m = new HashMap(); for (BookieServer b : bs) { - m.put(b.getLocalAddress(), 0); + m.put(b.getBookieId(), 0); } for (int i = 0; i < 2000; i++) { LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes()); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembleAt(0)) { m.put(b, m.get(b) + 1); } } @@ -220,12 +220,12 @@ public void testDiskSpaceWeightedBookieSelectionWithChangingWeights() throws Exc // make sure that bookies with higher weight(the last 2 bookies) are chosen 3X as often as the median; // since the number of ledgers created is small (2000), we allow a range of 2X to 4X instead of the exact 3X for (int i = 0; i < numBookies - 2; i++) { - double ratio1 = (double) m.get(bs.get(numBookies - 2).getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio1 = (double) m.get(bs.get(numBookies - 2).getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio1 - multiple), Math.abs(ratio1 - multiple) < 1); - double ratio2 = (double) m.get(bs.get(numBookies - 1).getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio2 = (double) m.get(bs.get(numBookies - 1).getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio2 - multiple), Math.abs(ratio2 - multiple) < 1); } @@ -243,11 +243,11 @@ public void testDiskSpaceWeightedBookieSelectionWithChangingWeights() throws Exc server4 = replaceBookieWithCustomFreeDiskSpaceBookie(client, server4, freeDiskSpace); for (BookieServer b : bs) { - m.put(b.getLocalAddress(), 0); + m.put(b.getBookieId(), 0); } for (int i = 0; i < 2000; i++) { LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes()); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembleAt(0)) { m.put(b, m.get(b) + 1); } } @@ -255,16 +255,16 @@ public void testDiskSpaceWeightedBookieSelectionWithChangingWeights() throws Exc // make sure that bookies with higher weight(the last 2 bookies) are chosen 3X as often as the median; // since the number of ledgers created is small (2000), we allow a range of 2X to 4X instead of the exact 3X for (int i = 0; i < numBookies; i++) { - if (server1.getLocalAddress().equals(bs.get(i).getLocalAddress()) - || server2.getLocalAddress().equals(bs.get(i).getLocalAddress())) { + if (server1.getBookieId().equals(bs.get(i).getBookieId()) + || server2.getBookieId().equals(bs.get(i).getBookieId())) { continue; } - double ratio1 = (double) m.get(server1.getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio1 = (double) m.get(server1.getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio1 - multiple), Math.abs(ratio1 - multiple) < 1); - double ratio2 = (double) m.get(server2.getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio2 = (double) m.get(server2.getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio2 - multiple), Math.abs(ratio2 - multiple) < 1); } @@ -295,33 +295,33 @@ public void testDiskSpaceWeightedBookieSelectionWithBookiesDying() throws Except replaceBookieWithCustomFreeDiskSpaceBookie(client, 0, multiple * freeDiskSpace); } } - Map m = new HashMap(); + Map m = new HashMap(); for (BookieServer b : bs) { - m.put(b.getLocalAddress(), 0); + m.put(b.getBookieId(), 0); } for (int i = 0; i < 2000; i++) { LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes()); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembleAt(0)) { m.put(b, m.get(b) + 1); } } // make sure that bookies with higher weight are chosen 3X as often as the median; // since the number of ledgers is small (2000), there may be variation - double ratio1 = (double) m.get(bs.get(numBookies - 2).getLocalAddress()) - / (double) m.get(bs.get(0).getLocalAddress()); + double ratio1 = (double) m.get(bs.get(numBookies - 2).getBookieId()) + / (double) m.get(bs.get(0).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio1 - multiple), Math.abs(ratio1 - multiple) < 1); - double ratio2 = (double) m.get(bs.get(numBookies - 1).getLocalAddress()) - / (double) m.get(bs.get(1).getLocalAddress()); + double ratio2 = (double) m.get(bs.get(numBookies - 1).getBookieId()) + / (double) m.get(bs.get(1).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio2 - multiple), Math.abs(ratio2 - multiple) < 1); // Bring down the 2 bookies that had higher weight; after this the allocation to all // the remaining bookies should be uniform for (BookieServer b : bs) { - m.put(b.getLocalAddress(), 0); + m.put(b.getBookieId(), 0); } BookieServer server1 = bs.get(numBookies - 2); BookieServer server2 = bs.get(numBookies - 1); @@ -330,24 +330,24 @@ public void testDiskSpaceWeightedBookieSelectionWithBookiesDying() throws Except for (int i = 0; i < 2000; i++) { LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes()); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembleAt(0)) { m.put(b, m.get(b) + 1); } } // make sure that bookies with higher weight are chosen 3X as often as the median; for (int i = 0; i < numBookies - 3; i++) { - double delta = Math.abs((double) m.get(bs.get(i).getLocalAddress()) - - (double) m.get(bs.get(i + 1).getLocalAddress())); - delta = (delta * 100) / (double) m.get(bs.get(i + 1).getLocalAddress()); + double delta = Math.abs((double) m.get(bs.get(i).getBookieId()) + - (double) m.get(bs.get(i + 1).getBookieId())); + delta = (delta * 100) / (double) m.get(bs.get(i + 1).getBookieId()); // the deviation should be less than 30% assertTrue("Weigheted placement is not honored: " + delta, delta <= 30); } // since the following 2 bookies were down, they shouldn't ever be selected - assertTrue("Weigheted placement is not honored" + m.get(server1.getLocalAddress()), - m.get(server1.getLocalAddress()) == 0); - assertTrue("Weigheted placement is not honored" + m.get(server2.getLocalAddress()), - m.get(server2.getLocalAddress()) == 0); + assertTrue("Weigheted placement is not honored" + m.get(server1.getBookieId()), + m.get(server1.getBookieId()) == 0); + assertTrue("Weigheted placement is not honored" + m.get(server2.getBookieId()), + m.get(server2.getBookieId()) == 0); client.close(); } @@ -375,14 +375,14 @@ public void testDiskSpaceWeightedBookieSelectionWithBookiesBeingAdded() throws E // let the last two bookies be down initially ServerConfiguration conf1 = killBookieAndWaitForZK(numBookies - 1); ServerConfiguration conf2 = killBookieAndWaitForZK(numBookies - 2); - Map m = new HashMap(); + Map m = new HashMap(); for (BookieServer b : bs) { - m.put(b.getLocalAddress(), 0); + m.put(b.getBookieId(), 0); } for (int i = 0; i < 2000; i++) { LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes()); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembleAt(0)) { m.put(b, m.get(b) + 1); } } @@ -390,9 +390,9 @@ public void testDiskSpaceWeightedBookieSelectionWithBookiesBeingAdded() throws E // make sure that bookies with higher weight are chosen 3X as often as the median; // since the number of ledgers is small (2000), there may be variation for (int i = 0; i < numBookies - 3; i++) { - double delta = Math.abs((double) m.get(bs.get(i).getLocalAddress()) - - (double) m.get(bs.get(i + 1).getLocalAddress())); - delta = (delta * 100) / (double) m.get(bs.get(i + 1).getLocalAddress()); + double delta = Math.abs((double) m.get(bs.get(i).getBookieId()) + - (double) m.get(bs.get(i + 1).getBookieId())); + delta = (delta * 100) / (double) m.get(bs.get(i + 1).getBookieId()); // the deviation should be less than 30% assertTrue("Weigheted placement is not honored: " + delta, delta <= 30); } @@ -402,11 +402,11 @@ public void testDiskSpaceWeightedBookieSelectionWithBookiesBeingAdded() throws E restartBookie(client, conf2, multiple * freeDiskSpace, multiple * freeDiskSpace, null); for (BookieServer b : bs) { - m.put(b.getLocalAddress(), 0); + m.put(b.getBookieId(), 0); } for (int i = 0; i < 2000; i++) { LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes()); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembleAt(0)) { m.put(b, m.get(b) + 1); } } @@ -414,12 +414,12 @@ public void testDiskSpaceWeightedBookieSelectionWithBookiesBeingAdded() throws E // make sure that bookies with higher weight(the last 2 bookies) are chosen 3X as often as the median; // since the number of ledgers created is small (2000), we allow a range of 2X to 4X instead of the exact 3X for (int i = 0; i < numBookies - 2; i++) { - double ratio1 = (double) m.get(bs.get(numBookies - 2).getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio1 = (double) m.get(bs.get(numBookies - 2).getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio1 - multiple), Math.abs(ratio1 - multiple) < 1); - double ratio2 = (double) m.get(bs.get(numBookies - 1).getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio2 = (double) m.get(bs.get(numBookies - 1).getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio2 - multiple), Math.abs(ratio2 - multiple) < 1); } @@ -455,22 +455,22 @@ public void testDiskSpaceWeightedBookieSelectionWithPeriodicBookieInfoUpdate() t client, 0, freeDiskSpace, multiple * freeDiskSpace, useHigherValue); } } - Map m = new HashMap(); + Map m = new HashMap(); for (BookieServer b : bs) { - m.put(b.getLocalAddress(), 0); + m.put(b.getBookieId(), 0); } for (int i = 0; i < 2000; i++) { LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes()); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembleAt(0)) { m.put(b, m.get(b) + 1); } } for (int i = 0; i < numBookies - 1; i++) { - double delta = Math.abs((double) m.get(bs.get(i).getLocalAddress()) - - (double) m.get(bs.get(i + 1).getLocalAddress())); - delta = (delta * 100) / (double) m.get(bs.get(i + 1).getLocalAddress()); + double delta = Math.abs((double) m.get(bs.get(i).getBookieId()) + - (double) m.get(bs.get(i + 1).getBookieId())); + delta = (delta * 100) / (double) m.get(bs.get(i + 1).getBookieId()); assertTrue("Weigheted placement is not honored: " + delta, delta <= 30); // the deviation should be <30% } @@ -480,18 +480,18 @@ public void testDiskSpaceWeightedBookieSelectionWithPeriodicBookieInfoUpdate() t Thread.sleep(updateIntervalSecs * 1000); for (int i = 0; i < numBookies; i++) { if (i < numBookies - 2) { - client.blockUntilBookieWeightIs(bs.get(i).getLocalAddress(), Optional.of(freeDiskSpace)); + client.blockUntilBookieWeightIs(bs.get(i).getBookieId(), Optional.of(freeDiskSpace)); } else { - client.blockUntilBookieWeightIs(bs.get(i).getLocalAddress(), Optional.of(freeDiskSpace * multiple)); + client.blockUntilBookieWeightIs(bs.get(i).getBookieId(), Optional.of(freeDiskSpace * multiple)); } } for (BookieServer b : bs) { - m.put(b.getLocalAddress(), 0); + m.put(b.getBookieId(), 0); } for (int i = 0; i < 2000; i++) { LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes()); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembleAt(0)) { m.put(b, m.get(b) + 1); } } @@ -499,12 +499,12 @@ public void testDiskSpaceWeightedBookieSelectionWithPeriodicBookieInfoUpdate() t // make sure that bookies with higher weight(the last 2 bookies) are chosen 3X as often as the median; // since the number of ledgers created is small (2000), we allow a range of 2X to 4X instead of the exact 3X for (int i = 0; i < numBookies - 2; i++) { - double ratio1 = (double) m.get(bs.get(numBookies - 2).getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio1 = (double) m.get(bs.get(numBookies - 2).getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio1 - multiple), Math.abs(ratio1 - multiple) < 1); - double ratio2 = (double) m.get(bs.get(numBookies - 1).getLocalAddress()) - / (double) m.get(bs.get(i).getLocalAddress()); + double ratio2 = (double) m.get(bs.get(numBookies - 1).getBookieId()) + / (double) m.get(bs.get(i).getBookieId()); assertTrue("Weigheted placement is not honored: " + Math.abs(ratio2 - multiple), Math.abs(ratio2 - multiple) < 1); } 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 a00ec9af98b..c19e3918361 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 @@ -52,7 +52,7 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.api.WriteHandle; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -982,7 +982,7 @@ void setConditionFirstInvocationLatch(CountDownLatch conditionFirstInvocationLat } @Override - public boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, + public boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBookies, int writeQuorumSize, int ackQuorumSize) { conditionFirstInvocationLatch.countDown(); @@ -1027,7 +1027,7 @@ public void testEnforceMinNumFaultDomainsForWrite() throws Exception { currPlacementPolicy.setConditionFirstInvocationLatch(countDownLatch); currPlacementPolicy.setWriteQuorumSizeToUseForTesting(writeQuorumSize); - BookieSocketAddress bookieToSleep; + BookieId bookieToSleep; try (LedgerHandle lh = bk.createLedger(ensembleSize, writeQuorumSize, ackQuorumSize, digestType, password)) { CountDownLatch sleepLatchCase1 = new CountDownLatch(1); @@ -1037,8 +1037,8 @@ public void testEnforceMinNumFaultDomainsForWrite() throws Exception { LOG.info("Putting all non ensemble bookies to sleep."); for (BookieServer bookieServer : bs) { try { - if (!lh.getCurrentEnsemble().contains(bookieServer.getLocalAddress())) { - sleepBookie(bookieServer.getLocalAddress(), sleepLatchCase2); + if (!lh.getCurrentEnsemble().contains(bookieServer.getBookieId())) { + sleepBookie(bookieServer.getBookieId(), sleepLatchCase2); } } catch (UnknownHostException ignored) {} } 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 e1d32af4638..e361294cb99 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 @@ -31,7 +31,7 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.TestStatsProvider; @@ -70,12 +70,12 @@ public BookieClient getBookieClient() { return bookieClient; } - public Future waitForReadOnlyBookie(BookieSocketAddress b) + public Future waitForReadOnlyBookie(BookieId b) throws Exception { return waitForBookieInSet(b, false); } - public Future waitForWritableBookie(BookieSocketAddress b) + public Future waitForWritableBookie(BookieId b) throws Exception { return waitForBookieInSet(b, true); } @@ -85,7 +85,7 @@ public Future waitForWritableBookie(BookieSocketAddress b) * or the read only set of bookies. Also ensure that it doesn't exist * in the other set before completing. */ - private Future waitForBookieInSet(BookieSocketAddress b, + private Future waitForBookieInSet(BookieId b, boolean writable) throws Exception { log.info("Wait for {} to become {}", b, writable ? "writable" : "readonly"); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieDecommissionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieDecommissionTest.java index 906db8f5cc8..1fddabbb897 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieDecommissionTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieDecommissionTest.java @@ -77,7 +77,7 @@ public void testDecommissionBookie() throws Exception { * if we try to call decommissionBookie for a bookie which is not * shutdown, then it should throw BKIllegalOpException */ - bkAdmin.decommissionBookie(bs.get(0).getLocalAddress()); + bkAdmin.decommissionBookie(bs.get(0).getBookieId()); fail("Expected BKIllegalOpException because that bookie is not shutdown yet"); } catch (BKIllegalOpException bkioexc) { // expected IllegalException @@ -88,7 +88,7 @@ public void testDecommissionBookie() throws Exception { * this decommisionBookie should make sure that there are no * underreplicated ledgers because of this bookie */ - bkAdmin.decommissionBookie(Bookie.getBookieAddress(killedBookieConf)); + bkAdmin.decommissionBookie(Bookie.getBookieId(killedBookieConf)); bkAdmin.triggerAudit(); Thread.sleep(500); Iterator ledgersToRereplicate = urLedgerMgr.listLedgersToRereplicate(null); @@ -101,7 +101,7 @@ public void testDecommissionBookie() throws Exception { } killedBookieConf = killBookie(0); - bkAdmin.decommissionBookie(Bookie.getBookieAddress(killedBookieConf)); + bkAdmin.decommissionBookie(Bookie.getBookieId(killedBookieConf)); bkAdmin.triggerAudit(); Thread.sleep(500); ledgersToRereplicate = urLedgerMgr.listLedgersToRereplicate(null); @@ -161,7 +161,7 @@ public void testDecommissionForLedgersWithMultipleSegmentsAndNotWriteClosed() th * info. Check BOOKKEEPER-237 and BOOKKEEPER-325. But later * ReplicationWorker will fence the ledger. */ - bkAdmin.decommissionBookie(Bookie.getBookieAddress(killedBookieConf)); + bkAdmin.decommissionBookie(Bookie.getBookieId(killedBookieConf)); bkAdmin.triggerAudit(); Thread.sleep(500); Iterator ledgersToRereplicate = urLedgerMgr.listLedgersToRereplicate(null); 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 2fc1d5fa3b7..ce66c9f3937 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 @@ -44,7 +44,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -263,7 +263,7 @@ void metadataConflictWithRecovery(BookKeeper bkc) throws Exception { for (int i = 0; i < numEntries; i++) { lh.addEntry(data); } - BookieSocketAddress bookieToKill = lh.getLedgerMetadata().getEnsembleAt(numEntries - 1).get(1); + BookieId bookieToKill = lh.getLedgerMetadata().getEnsembleAt(numEntries - 1).get(1); killBookie(bookieToKill); startNewBookie(); for (int i = 0; i < numEntries; i++) { @@ -306,7 +306,7 @@ public void testAsyncBookieRecoveryToSpecificBookie() throws Exception { // Shutdown the first bookie server LOG.info("Finished writing all ledger entries so shutdown one of the bookies."); - BookieSocketAddress bookieSrc = bs.get(0).getLocalAddress(); + BookieId bookieSrc = bs.get(0).getBookieId(); bs.get(0).shutdown(); bs.remove(0); @@ -356,7 +356,7 @@ public void testAsyncBookieRecoveryToRandomBookies() throws Exception { // Shutdown the first bookie server LOG.info("Finished writing all ledger entries so shutdown one of the bookies."); - BookieSocketAddress bookieSrc = bs.get(0).getLocalAddress(); + BookieId bookieSrc = bs.get(0).getBookieId(); bs.get(0).shutdown(); bs.remove(0); @@ -409,7 +409,7 @@ public void testSyncBookieRecoveryToSpecificBookie() throws Exception { // Shutdown the first bookie server LOG.info("Finished writing all ledger entries so shutdown one of the bookies."); - BookieSocketAddress bookieSrc = bs.get(0).getLocalAddress(); + BookieId bookieSrc = bs.get(0).getBookieId(); bs.get(0).shutdown(); bs.remove(0); @@ -450,7 +450,7 @@ public void testSyncBookieRecoveryToRandomBookies() throws Exception { // Shutdown the first bookie server LOG.info("Finished writing all ledger entries so shutdown one of the bookies."); - BookieSocketAddress bookieSrc = bs.get(0).getLocalAddress(); + BookieId bookieSrc = bs.get(0).getBookieId(); bs.get(0).shutdown(); bs.remove(0); @@ -505,7 +505,7 @@ long await() throws InterruptedException { private boolean verifyFullyReplicated(LedgerHandle lh, long untilEntry) throws Exception { LedgerMetadata md = getLedgerMetadata(lh); - Map> ensembles = md.getAllEnsembles(); + Map> ensembles = md.getAllEnsembles(); HashMap ranges = new HashMap(); ArrayList keyList = Collections.list( @@ -516,7 +516,7 @@ private boolean verifyFullyReplicated(LedgerHandle lh, long untilEntry) throws E } ranges.put(keyList.get(keyList.size() - 1), untilEntry); - for (Map.Entry> e : ensembles.entrySet()) { + for (Map.Entry> e : ensembles.entrySet()) { int quorum = md.getAckQuorumSize(); long startEntryId = e.getKey(); long endEntryId = ranges.get(startEntryId); @@ -525,7 +525,7 @@ private boolean verifyFullyReplicated(LedgerHandle lh, long untilEntry) throws E ReplicationVerificationCallback cb = new ReplicationVerificationCallback(numRequests); for (long i = startEntryId; i < endEntryId; i++) { - for (BookieSocketAddress addr : e.getValue()) { + for (BookieId addr : e.getValue()) { bkc.getBookieClient().readEntry(addr, lh.getId(), i, cb, addr, BookieProtocol.FLAG_NONE); } @@ -564,11 +564,11 @@ private boolean findDupesInEnsembles(List lhs) throws Exception { long numDupes = 0; for (LedgerHandle lh : lhs) { LedgerMetadata md = getLedgerMetadata(lh); - for (Map.Entry> e : md.getAllEnsembles().entrySet()) { - HashSet set = new HashSet(); + for (Map.Entry> e : md.getAllEnsembles().entrySet()) { + HashSet set = new HashSet(); long fragment = e.getKey(); - for (BookieSocketAddress addr : e.getValue()) { + for (BookieId addr : e.getValue()) { if (set.contains(addr)) { LOG.error("Dupe " + addr + " found in ensemble for fragment " + fragment + " of ledger " + lh.getId()); @@ -597,9 +597,9 @@ public void testBookieRecoveryOnClosedLedgers() throws Exception { closeLedgers(lhs); // Shutdown last bookie server in last ensemble - List lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles() + List lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles() .entrySet().iterator().next().getValue(); - BookieSocketAddress bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); + BookieId bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); killBookie(bookieToKill); // start a new bookie @@ -626,9 +626,9 @@ public void testBookieRecoveryOnOpenedLedgers() throws Exception { writeEntriestoLedgers(numMsgs, 0, lhs); // Shutdown the first bookie server - List lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles() + List lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles() .entrySet().iterator().next().getValue(); - BookieSocketAddress bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); + BookieId bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); killBookie(bookieToKill); // start a new bookie @@ -662,13 +662,13 @@ public void testBookieRecoveryOnInRecoveryLedger() throws Exception { writeEntriestoLedgers(numMsgs, 0, lhs); // Shutdown the first bookie server - List lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles() + List lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles() .entrySet().iterator().next().getValue(); // removed bookie - BookieSocketAddress bookieToKill = lastEnsemble.get(0); + BookieId bookieToKill = lastEnsemble.get(0); killBookie(bookieToKill); // temp failure - BookieSocketAddress bookieToKill2 = lastEnsemble.get(1); + BookieId bookieToKill2 = lastEnsemble.get(1); ServerConfiguration conf2 = killBookie(bookieToKill2); // start a new bookie @@ -705,7 +705,7 @@ public void testBookieRecoveryOnInRecoveryLedger() throws Exception { List newLhs = openLedgers(lhs); for (LedgerHandle newLh : newLhs) { // first ensemble should contains bookieToKill2 and not contain bookieToKill - Map.Entry> entry = + Map.Entry> entry = newLh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next(); assertFalse(entry.getValue().contains(bookieToKill)); assertTrue(entry.getValue().contains(bookieToKill2)); @@ -725,7 +725,7 @@ public void testAsyncBookieRecoveryToRandomBookiesNotEnoughBookies() throws Exce // Shutdown the first bookie server LOG.info("Finished writing all ledger entries so shutdown one of the bookies."); - BookieSocketAddress bookieSrc = bs.get(0).getLocalAddress(); + BookieId bookieSrc = bs.get(0).getBookieId(); bs.get(0).shutdown(); bs.remove(0); @@ -757,7 +757,7 @@ public void testSyncBookieRecoveryToRandomBookiesCheckForDupes() throws Exceptio // Shutdown the first bookie server LOG.info("Finished writing all ledger entries so shutdown one of the bookies."); int removeIndex = r.nextInt(bs.size()); - BookieSocketAddress bookieSrc = bs.get(removeIndex).getLocalAddress(); + BookieId bookieSrc = bs.get(removeIndex).getBookieId(); bs.get(removeIndex).shutdown(); bs.remove(removeIndex); @@ -799,7 +799,7 @@ public void recoverWithoutPasswordInConf() throws Exception { } lh.close(); - BookieSocketAddress bookieSrc = bs.get(0).getLocalAddress(); + BookieId bookieSrc = bs.get(0).getBookieId(); bs.get(0).shutdown(); bs.remove(0); startNewBookie(); @@ -826,7 +826,7 @@ public void recoverWithoutPasswordInConf() throws Exception { assertTrue("Should be back to fully replication", verifyFullyReplicated(lh, 100)); lh.close(); - bookieSrc = bs.get(0).getLocalAddress(); + bookieSrc = bs.get(0).getBookieId(); bs.get(0).shutdown(); bs.remove(0); startNewBookie(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java index a3fe0706f8e..7c1e3dadf90 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java @@ -62,7 +62,7 @@ import org.apache.bookkeeper.client.api.WriteAdvHandle; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LongHierarchicalLedgerManagerFactory; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.commons.lang3.tuple.Pair; import org.apache.zookeeper.KeeperException; @@ -151,7 +151,7 @@ public void testWithMultipleBookieFailuresInLastEnsemble() throws Exception { startNewBookie(); // Shutdown three bookies in the last ensemble and continue writing - List ensemble = lh.getLedgerMetadata() + List ensemble = lh.getLedgerMetadata() .getAllEnsembles().entrySet().iterator().next().getValue(); killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); @@ -196,7 +196,7 @@ public void testWriteAndReadStats() throws Exception { CountDownLatch sleepLatch1 = new CountDownLatch(1); CountDownLatch sleepLatch2 = new CountDownLatch(1); - List ensemble = lh.getLedgerMetadata() + List ensemble = lh.getLedgerMetadata() .getAllEnsembles().entrySet().iterator().next().getValue(); sleepBookie(ensemble.get(0), sleepLatch1); @@ -293,7 +293,7 @@ public void testDelayedWriteEnsembleChange() throws Exception { CountDownLatch sleepLatch1 = new CountDownLatch(1); // get bookie at index-0 - BookieSocketAddress bookie1 = lh.getCurrentEnsemble().get(0); + BookieId bookie1 = lh.getCurrentEnsemble().get(0); sleepBookie(bookie1, sleepLatch1); int i = numEntriesToWrite; @@ -332,7 +332,7 @@ public void testDelayedWriteEnsembleChange() throws Exception { sleepLatch1.countDown(); // get the bookie at index-0 again, this must be different. - BookieSocketAddress bookie2 = lh.getCurrentEnsemble().get(0); + BookieId bookie2 = lh.getCurrentEnsemble().get(0); assertFalse( "Delayed write error must have forced ensemble change", @@ -390,7 +390,7 @@ public void testLedgerCreateAdv() throws Exception { startNewBookie(); // Shutdown one bookie in the last ensemble and continue writing - List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() .getValue(); killBookie(ensemble.get(0)); @@ -529,7 +529,7 @@ public void testLedgerCreateAdvWithLedgerId() throws Exception { startNewBookie(); // Shutdown one bookie in the last ensemble and continue writing - List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() .getValue(); killBookie(ensemble.get(0)); @@ -879,7 +879,7 @@ public void testAsyncWritesWithMultipleFailuresInLastEnsemble() startNewBookie(); // Shutdown three bookies in the last ensemble and continue writing - List ensemble = lh.getLedgerMetadata() + List ensemble = lh.getLedgerMetadata() .getAllEnsembles().entrySet().iterator().next().getValue(); killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); @@ -952,7 +952,7 @@ public void testLedgerCreateAdvWithAsyncWritesWithBookieFailures() throws Except } // Start One more bookie and shutdown one from last ensemble before reading startNewBookie(); - List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() .getValue(); killBookie(ensemble.get(0)); @@ -1020,7 +1020,7 @@ public void testLedgerHandleAdvOutOfOrderWriteAndFrocedEnsembleChange() throws E } CountDownLatch sleepLatch1 = new CountDownLatch(1); - List ensemble; + List ensemble; ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next().getValue(); @@ -1106,7 +1106,7 @@ public void testLedgerCreateAdvWithRandomAsyncWritesWithBookieFailuresBetweenWri if (j == numEntriesToWrite / 2) { // Start One more bookie and shutdown one from last ensemble at half-way startNewBookie(); - List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet() .iterator().next().getValue(); killBookie(ensemble.get(0)); } @@ -1175,7 +1175,7 @@ public void testLedgerCreateAdvWithRandomAsyncWritesWithBookieFailures() throws } // Start One more bookie and shutdown one from last ensemble before reading startNewBookie(); - List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next() .getValue(); killBookie(ensemble.get(0)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/DeferredSyncTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/DeferredSyncTest.java index 996c902ffe7..dabf2b6088c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/DeferredSyncTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/DeferredSyncTest.java @@ -29,7 +29,7 @@ import org.apache.bookkeeper.client.api.WriteAdvHandle; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.api.WriteHandle; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.junit.Test; /** @@ -128,7 +128,7 @@ public void testForceRequiresFullEnsemble() throws Exception { assertEquals(NUM_ENTRIES - 1, wh.getLastAddPushed()); assertEquals(-1, wh.getLastAddConfirmed()); - BookieSocketAddress bookieAddress = wh.getLedgerMetadata().getEnsembleAt(wh.getLastAddPushed()).get(0); + BookieId bookieAddress = wh.getLedgerMetadata().getEnsembleAt(wh.getLastAddPushed()).get(0); killBookie(bookieAddress); // write should succeed (we still have 2 bookies out of 3) @@ -162,7 +162,7 @@ public void testForceWillAdvanceLacOnlyUpToLastAcknoledgedWrite() throws Excepti assertEquals(-1, wh.getLastAddConfirmed()); // one bookie will stop sending acks for forceLedger - BookieSocketAddress bookieAddress = wh.getLedgerMetadata().getEnsembleAt(wh.getLastAddPushed()).get(0); + BookieId bookieAddress = wh.getLedgerMetadata().getEnsembleAt(wh.getLastAddPushed()).get(0); suspendBookieForceLedgerAcks(bookieAddress); // start and complete a force, lastAddConfirmed cannot be "lastAddPushedAfterSuspendedWrite" diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java index 828a431920a..646fb32751e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java @@ -31,7 +31,7 @@ import java.util.Map; import java.util.Set; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Before; import org.junit.Test; @@ -69,9 +69,9 @@ public GenericEnsemblePlacementPolicyTest(boolean diskWeightBasedPlacementEnable public static final class CustomEnsemblePlacementPolicy extends DefaultEnsemblePlacementPolicy { @Override - public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map customMetadata, List currentEnsemble, - BookieSocketAddress bookieToReplace, Set excludeBookies) + public PlacementResult replaceBookie(int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map customMetadata, List currentEnsemble, + BookieId bookieToReplace, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { new Exception("replaceBookie " + ensembleSize + "," + customMetadata).printStackTrace(); assertNotNull(customMetadata); @@ -81,8 +81,8 @@ public PlacementResult replaceBookie(int ensembleSize, int } @Override - public PlacementResult> newEnsemble(int ensembleSize, int quorumSize, - int ackQuorumSize, Map customMetadata, Set excludeBookies) + public PlacementResult> newEnsemble(int ensembleSize, int quorumSize, + int ackQuorumSize, Map customMetadata, Set excludeBookies) throws BKException.BKNotEnoughBookiesException { assertNotNull(customMetadata); customMetadataOnNewEnsembleStack.add(customMetadata); @@ -144,7 +144,7 @@ public void testReplaceBookie() throws Exception { try (LedgerHandle lh = bk.createLedger(2, 2, 2, digestType, PASSWORD.getBytes(), customMetadata)) { lh.addEntry(value); long lId = lh.getId(); - List ensembleAtFirstEntry = lh.getLedgerMetadata().getEnsembleAt(lId); + List ensembleAtFirstEntry = lh.getLedgerMetadata().getEnsembleAt(lId); assertEquals(2, ensembleAtFirstEntry.size()); killBookie(ensembleAtFirstEntry.get(0)); lh.addEntry(value); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java index 1b8220c65d4..406fbd571aa 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java @@ -36,6 +36,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.versioning.Versioned; @@ -51,11 +52,11 @@ public class HandleFailuresTest { private static final Logger log = LoggerFactory.getLogger(LedgerRecovery2Test.class); - private static final BookieSocketAddress b1 = new BookieSocketAddress("b1", 3181); - private static final BookieSocketAddress b2 = new BookieSocketAddress("b2", 3181); - private static final BookieSocketAddress b3 = new BookieSocketAddress("b3", 3181); - private static final BookieSocketAddress b4 = new BookieSocketAddress("b4", 3181); - private static final BookieSocketAddress b5 = new BookieSocketAddress("b5", 3181); + private static final BookieId b1 = new BookieSocketAddress("b1", 3181).toBookieId(); + private static final BookieId b2 = new BookieSocketAddress("b2", 3181).toBookieId(); + private static final BookieId b3 = new BookieSocketAddress("b3", 3181).toBookieId(); + private static final BookieId b4 = new BookieSocketAddress("b4", 3181).toBookieId(); + private static final BookieId b5 = new BookieSocketAddress("b5", 3181).toBookieId(); @Test public void testChangeTriggeredOneTimeForOneFailure() throws Exception { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java index 612b36e27e4..95a5949bdfa 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java @@ -23,6 +23,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.versioning.Versioned; @@ -38,11 +39,11 @@ public class LedgerClose2Test { private static final Logger log = LoggerFactory.getLogger(LedgerRecovery2Test.class); - private static final BookieSocketAddress b1 = new BookieSocketAddress("b1", 3181); - private static final BookieSocketAddress b2 = new BookieSocketAddress("b2", 3181); - private static final BookieSocketAddress b3 = new BookieSocketAddress("b3", 3181); - private static final BookieSocketAddress b4 = new BookieSocketAddress("b4", 3181); - private static final BookieSocketAddress b5 = new BookieSocketAddress("b5", 3181); + private static final BookieId b1 = new BookieSocketAddress("b1", 3181).toBookieId(); + private static final BookieId b2 = new BookieSocketAddress("b2", 3181).toBookieId(); + private static final BookieId b3 = new BookieSocketAddress("b3", 3181).toBookieId(); + private static final BookieId b4 = new BookieSocketAddress("b4", 3181).toBookieId(); + private static final BookieId b5 = new BookieSocketAddress("b5", 3181).toBookieId(); @Test public void testTryAddAfterCloseHasBeenCalled() throws Exception { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java index 25f7f1aacba..5ece1a1f31b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java @@ -39,7 +39,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -145,7 +145,7 @@ private void verifyMetadataConsistency(int numEntries, LedgerHandle lh) final CountDownLatch recoverDoneLatch = new CountDownLatch(1); final CountDownLatch failedLatch = new CountDownLatch(1); // kill first bookie to replace with a unauthorize bookie - BookieSocketAddress bookie = lh.getCurrentEnsemble().get(0); + BookieId bookie = lh.getCurrentEnsemble().get(0); ServerConfiguration conf = killBookie(bookie); // replace a unauthorize bookie startUnauthorizedBookie(conf, addDoneLatch); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java index b993b4f66d5..25a4fd8acdf 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java @@ -30,6 +30,7 @@ import java.util.List; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.junit.Test; @@ -42,10 +43,9 @@ public class LedgerMetadataTest { @Test public void testGetters() { - List ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234)); + List ensemble = Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.2", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.3", 1234).toBookieId()); org.apache.bookkeeper.client.api.LedgerMetadata metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) .withDigestType(DigestType.CRC32.toApiDigestType()).withPassword(passwd) @@ -68,10 +68,9 @@ public void testGetters() { @Test public void testToString() { - List ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234)); + List ensemble = Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.2", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.3", 1234).toBookieId()); LedgerMetadata lm1 = LedgerMetadataBuilder.create() .withDigestType(DigestType.CRC32.toApiDigestType()) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java index 2e17836bd54..4e2ca422528 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture; import org.apache.bookkeeper.versioning.Versioned; @@ -44,14 +45,14 @@ public class LedgerRecovery2Test { private static final Logger log = LoggerFactory.getLogger(LedgerRecovery2Test.class); private static final byte[] PASSWD = "foobar".getBytes(); - private static final BookieSocketAddress b1 = new BookieSocketAddress("b1", 3181); - private static final BookieSocketAddress b2 = new BookieSocketAddress("b2", 3181); - private static final BookieSocketAddress b3 = new BookieSocketAddress("b3", 3181); - private static final BookieSocketAddress b4 = new BookieSocketAddress("b4", 3181); - private static final BookieSocketAddress b5 = new BookieSocketAddress("b5", 3181); + private static final BookieId b1 = new BookieSocketAddress("b1", 3181).toBookieId(); + private static final BookieId b2 = new BookieSocketAddress("b2", 3181).toBookieId(); + private static final BookieId b3 = new BookieSocketAddress("b3", 3181).toBookieId(); + private static final BookieId b4 = new BookieSocketAddress("b4", 3181).toBookieId(); + private static final BookieId b5 = new BookieSocketAddress("b5", 3181).toBookieId(); private static Versioned setupLedger(ClientContext clientCtx, long ledgerId, - List bookies) throws Exception { + List bookies) throws Exception { LedgerMetadata md = LedgerMetadataBuilder.create() .withPassword(PASSWD).withDigestType(DigestType.CRC32C) .newEnsembleEntry(0, bookies).build(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java index 9a927b68852..b89c370e5f0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java @@ -40,7 +40,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -185,7 +185,7 @@ private void ledgerRecoveryWithSlowBookie(int ensembleSize, int writeQuorumSize, // kill first bookie server to start a fake one to simulate a slow bookie // and failed to add entry on crash // until write succeed - BookieSocketAddress host = beforelh.getCurrentEnsemble().get(slowBookieIdx); + BookieId host = beforelh.getCurrentEnsemble().get(slowBookieIdx); ServerConfiguration conf = killBookie(host); Bookie fakeBookie = new Bookie(conf) { @@ -259,7 +259,7 @@ public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] bs.add(startBookie(conf, deadBookie1)); // kill first bookie server - BookieSocketAddress bookie1 = lhbefore.getCurrentEnsemble().get(0); + BookieId bookie1 = lhbefore.getCurrentEnsemble().get(0); ServerConfiguration conf1 = killBookie(bookie1); // Try to recover and fence the ledger after killing one bookie in the @@ -274,7 +274,7 @@ public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] // restart the first server, kill the second bsConfs.add(conf1); bs.add(startBookie(conf1)); - BookieSocketAddress bookie2 = lhbefore.getCurrentEnsemble().get(1); + BookieId bookie2 = lhbefore.getCurrentEnsemble().get(1); ServerConfiguration conf2 = killBookie(bookie2); // using async, because this could trigger an assertion @@ -340,7 +340,7 @@ public void recoveryAddEntry(ByteBuf entry, WriteCallback cb, Object ctx, byte[] bs.add(startBookie(conf, deadBookie1)); // kill first bookie server - BookieSocketAddress bookie1 = lhbefore.getCurrentEnsemble().get(0); + BookieId bookie1 = lhbefore.getCurrentEnsemble().get(0); killBookie(bookie1); // Try to recover and fence the ledger after killing one bookie in the @@ -391,9 +391,9 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { fail("Failed to add " + numEntries + " to ledger handle " + lh.getId()); } // kill first 2 bookies to replace bookies - BookieSocketAddress bookie1 = lh.getCurrentEnsemble().get(0); + BookieId bookie1 = lh.getCurrentEnsemble().get(0); ServerConfiguration conf1 = killBookie(bookie1); - BookieSocketAddress bookie2 = lh.getCurrentEnsemble().get(1); + BookieId bookie2 = lh.getCurrentEnsemble().get(1); ServerConfiguration conf2 = killBookie(bookie2); // replace these two bookies diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java index e76ee545449..1f43cb45729 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java @@ -46,6 +46,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.MockLedgerManager; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; @@ -71,25 +72,24 @@ public void testBasicUpdate() throws Exception { try (LedgerManager lm = new MockLedgerManager()) { LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(5) .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) - .newEnsembleEntry(0L, Lists.newArrayList( - new BookieSocketAddress("0.0.0.0:3181"), - new BookieSocketAddress("0.0.0.1:3181"), - new BookieSocketAddress("0.0.0.2:3181"), - new BookieSocketAddress("0.0.0.3:3181"), - new BookieSocketAddress("0.0.0.4:3181"))).build(); + .newEnsembleEntry(0L, Lists.newArrayList(BookieId.parse("0.0.0.0:3181"), + BookieId.parse("0.0.0.1:3181"), + BookieId.parse("0.0.0.2:3181"), + BookieId.parse("0.0.0.3:3181"), + BookieId.parse("0.0.0.4:3181"))).build(); long ledgerId = 1234L; Versioned writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get(); AtomicReference> reference = new AtomicReference<>(writtenMetadata); - BookieSocketAddress newAddress = new BookieSocketAddress("0.0.0.5:3181"); + BookieId newAddress = BookieId.parse("0.0.0.5:3181"); MetadataUpdateLoop loop = new MetadataUpdateLoop( lm, ledgerId, reference::get, (currentMetadata) -> true, (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, newAddress); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -111,10 +111,10 @@ public void testConflictOnWrite() throws Exception { lm.blockWrites(); long ledgerId = 1234L; - BookieSocketAddress b0 = new BookieSocketAddress("0.0.0.0:3181"); - BookieSocketAddress b1 = new BookieSocketAddress("0.0.0.1:3181"); - BookieSocketAddress b2 = new BookieSocketAddress("0.0.0.2:3181"); - BookieSocketAddress b3 = new BookieSocketAddress("0.0.0.3:3181"); + BookieId b0 = BookieId.parse("0.0.0.0:3181"); + BookieId b1 = BookieId.parse("0.0.0.1:3181"); + BookieId b2 = BookieId.parse("0.0.0.2:3181"); + BookieId b3 = BookieId.parse("0.0.0.3:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) @@ -129,7 +129,7 @@ public void testConflictOnWrite() throws Exception { reference1::get, (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b2); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -142,7 +142,7 @@ public void testConflictOnWrite() throws Exception { reference2::get, (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b1), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(1, b3); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -179,9 +179,9 @@ public void testConflictOnWriteBothWritingSame() throws Exception { lm.blockWrites(); long ledgerId = 1234L; - BookieSocketAddress b0 = new BookieSocketAddress("0.0.0.0:3181"); - BookieSocketAddress b1 = new BookieSocketAddress("0.0.0.1:3181"); - BookieSocketAddress b2 = new BookieSocketAddress("0.0.0.2:3181"); + BookieId b0 = BookieId.parse("0.0.0.0:3181"); + BookieId b1 = BookieId.parse("0.0.0.1:3181"); + BookieId b2 = BookieId.parse("0.0.0.2:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) @@ -195,7 +195,7 @@ public void testConflictOnWriteBothWritingSame() throws Exception { reference::get, (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b2); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -206,7 +206,7 @@ public void testConflictOnWriteBothWritingSame() throws Exception { reference::get, (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b2); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -232,10 +232,10 @@ public void testConflictOnWriteBothWritingSame() throws Exception { public void testConflictOnLocalUpdate() throws Exception { try (DeferCallbacksMockLedgerManager lm = spy(new DeferCallbacksMockLedgerManager(1))) { long ledgerId = 1234L; - BookieSocketAddress b0 = new BookieSocketAddress("0.0.0.0:3181"); - BookieSocketAddress b1 = new BookieSocketAddress("0.0.0.1:3181"); - BookieSocketAddress b2 = new BookieSocketAddress("0.0.0.2:3181"); - BookieSocketAddress b3 = new BookieSocketAddress("0.0.0.3:3181"); + BookieId b0 = BookieId.parse("0.0.0.0:3181"); + BookieId b1 = BookieId.parse("0.0.0.1:3181"); + BookieId b2 = BookieId.parse("0.0.0.2:3181"); + BookieId b3 = BookieId.parse("0.0.0.3:3181"); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(2) .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) @@ -249,7 +249,7 @@ public void testConflictOnLocalUpdate() throws Exception { reference::get, (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b2); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -262,7 +262,7 @@ public void testConflictOnLocalUpdate() throws Exception { reference::get, (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b1), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(1, b3); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -280,9 +280,9 @@ public void testConflictOnLocalUpdate() throws Exception { } } - private static BookieSocketAddress address(String s) { + private static BookieId address(String s) { try { - return new BookieSocketAddress(s); + return BookieId.parse(s); } catch (Exception e) { throw new RuntimeException(e); } @@ -299,7 +299,7 @@ public void testHammer() throws Exception { long ledgerId = 1234L; int ensembleSize = 100; - List initialEnsemble = IntStream.range(0, ensembleSize) + List initialEnsemble = IntStream.range(0, ensembleSize) .mapToObj((i) -> address(String.format("0.0.0.%d:3181", i))) .collect(Collectors.toList()); @@ -310,7 +310,7 @@ public void testHammer() throws Exception { AtomicReference> reference = new AtomicReference<>(writtenMetadata); - List replacementBookies = IntStream.range(0, ensembleSize) + List replacementBookies = IntStream.range(0, ensembleSize) .mapToObj((i) -> address(String.format("0.0.%d.1:3181", i))) .collect(Collectors.toList()); @@ -321,7 +321,7 @@ public void testHammer() throws Exception { reference::get, (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(initialEnsemble.get(i)), (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(i, replacementBookies.get(i)); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, @@ -344,8 +344,8 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { lm.blockWrites(); long ledgerId = 1234L; - BookieSocketAddress b0 = new BookieSocketAddress("0.0.0.0:3181"); - BookieSocketAddress b1 = new BookieSocketAddress("0.0.0.1:3181"); + BookieId b0 = new BookieSocketAddress("0.0.0.0:3181").toBookieId(); + BookieId b1 = new BookieSocketAddress("0.0.0.1:3181").toBookieId(); LedgerMetadata initMeta = LedgerMetadataBuilder.create().withEnsembleSize(1) .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) @@ -376,7 +376,7 @@ public void testNewestValueCannotBeUsedAfterReadBack() throws Exception { } }, (currentMetadata) -> { - List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); + List ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L)); ensemble.set(0, b1); return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build(); }, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java index c5878f1b860..fd0a2ba0a92 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java @@ -62,7 +62,9 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.LedgerIdGenerator; import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; @@ -99,25 +101,25 @@ public abstract class MockBookKeeperTestCase { protected ConcurrentMap mockLedgerMetadataRegistry; protected AtomicLong mockNextLedgerId; protected ConcurrentSkipListSet fencedLedgers; - protected ConcurrentMap>> mockLedgerData; + protected ConcurrentMap>> mockLedgerData; - private Map> deferredBookieForceLedgerResponses; - private Set suspendedBookiesForForceLedgerAcks; + private Map> deferredBookieForceLedgerResponses; + private Set suspendedBookiesForForceLedgerAcks; - List failedBookies; - Set availableBookies; + List failedBookies; + Set availableBookies; private int lastIndexForBK; - private Map> getMockLedgerContents(long ledgerId) { + private Map> getMockLedgerContents(long ledgerId) { return mockLedgerData.computeIfAbsent(ledgerId, (id) -> new ConcurrentHashMap<>()); } - private Map getMockLedgerContentsInBookie(long ledgerId, BookieSocketAddress bookieSocketAddress) { + private Map getMockLedgerContentsInBookie(long ledgerId, BookieId bookieSocketAddress) { return getMockLedgerContents(ledgerId).computeIfAbsent(bookieSocketAddress, addr -> new ConcurrentHashMap<>()); } private MockEntry getMockLedgerEntry(long ledgerId, - BookieSocketAddress bookieSocketAddress, long entryId) throws BKException{ + BookieId bookieSocketAddress, long entryId) throws BKException{ if (failedBookies.contains(bookieSocketAddress)) { throw BKException.create(NoBookieAvailableException); } @@ -151,6 +153,8 @@ public void setup() throws Exception { bookieClient = mock(BookieClient.class); ledgerManager = mock(LedgerManager.class); ledgerIdGenerator = mock(LedgerIdGenerator.class); + BookieAddressResolver bookieAddressResolver = BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER; + when(bookieWatcher.getBookieAddressResolver()).thenReturn(bookieAddressResolver); bk = mock(BookKeeper.class); @@ -160,6 +164,7 @@ public void setup() throws Exception { when(bk.getCloseLock()).thenReturn(new ReentrantReadWriteLock()); when(bk.isClosed()).thenReturn(false); when(bk.getBookieWatcher()).thenReturn(bookieWatcher); + when(bk.getBookieAddressResolver()).thenReturn(bookieAddressResolver); when(bk.getMainWorkerPool()).thenReturn(executor); when(bk.getBookieClient()).thenReturn(bookieClient); when(bk.getScheduler()).thenReturn(scheduler); @@ -274,23 +279,23 @@ protected void closeBookkeeper() { when(bk.isClosed()).thenReturn(true); } - protected void killBookie(BookieSocketAddress killedBookieSocketAddress) { + protected void killBookie(BookieId killedBookieSocketAddress) { failedBookies.add(killedBookieSocketAddress); availableBookies.remove(killedBookieSocketAddress); } - protected void startKilledBookie(BookieSocketAddress killedBookieSocketAddress) { + protected void startKilledBookie(BookieId killedBookieSocketAddress) { checkState(failedBookies.contains(killedBookieSocketAddress)); checkState(!availableBookies.contains(killedBookieSocketAddress)); failedBookies.remove(killedBookieSocketAddress); availableBookies.add(killedBookieSocketAddress); } - protected void suspendBookieForceLedgerAcks(BookieSocketAddress address) { + protected void suspendBookieForceLedgerAcks(BookieId address) { suspendedBookiesForForceLedgerAcks.add(address); } - protected void resumeBookieWriteAcks(BookieSocketAddress address) { + protected void resumeBookieWriteAcks(BookieId address) { suspendedBookiesForForceLedgerAcks.remove(address); List pendingResponses = deferredBookieForceLedgerResponses.remove(address); if (pendingResponses != null) { @@ -298,18 +303,18 @@ protected void resumeBookieWriteAcks(BookieSocketAddress address) { } } - protected BookieSocketAddress startNewBookie() { - BookieSocketAddress address = generateBookieSocketAddress(lastIndexForBK++); + protected BookieId startNewBookie() { + BookieId address = generateBookieSocketAddress(lastIndexForBK++); availableBookies.add(address); return address; } - protected BookieSocketAddress generateBookieSocketAddress(int index) { - return new BookieSocketAddress("localhost", 1111 + index); + protected BookieId generateBookieSocketAddress(int index) { + return new BookieSocketAddress("localhost", 1111 + index).toBookieId(); } - protected ArrayList generateNewEnsemble(int ensembleSize) { - ArrayList ensemble = new ArrayList<>(ensembleSize); + protected ArrayList generateNewEnsemble(int ensembleSize) { + ArrayList ensemble = new ArrayList<>(ensembleSize); for (int i = 0; i < ensembleSize; i++) { ensemble.add(generateBookieSocketAddress(i)); } @@ -320,10 +325,10 @@ protected ArrayList generateNewEnsemble(int ensembleSize) { private void setupBookieWatcherForNewEnsemble() throws BKException.BKNotEnoughBookiesException { when(bookieWatcher.newEnsemble(anyInt(), anyInt(), anyInt(), any())) - .thenAnswer((Answer>) new Answer>() { + .thenAnswer((Answer>) new Answer>() { @Override @SuppressWarnings("unchecked") - public ArrayList answer(InvocationOnMock invocation) throws Throwable { + public ArrayList answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); int ensembleSize = (Integer) args[0]; return generateNewEnsemble(ensembleSize); @@ -333,15 +338,15 @@ public ArrayList answer(InvocationOnMock invocation) throws private void setupBookieWatcherForEnsembleChange() throws BKException.BKNotEnoughBookiesException { when(bookieWatcher.replaceBookie(anyInt(), anyInt(), anyInt(), anyMap(), anyList(), anyInt(), anySet())) - .thenAnswer((Answer) new Answer() { + .thenAnswer((Answer) new Answer() { @Override @SuppressWarnings("unchecked") - public BookieSocketAddress answer(InvocationOnMock invocation) throws Throwable { + public BookieId answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); - List existingBookies = (List) args[4]; - Set excludeBookies = (Set) args[6]; + List existingBookies = (List) args[4]; + Set excludeBookies = (Set) args[6]; excludeBookies.addAll(existingBookies); - Set remainBookies = new HashSet(availableBookies); + Set remainBookies = new HashSet(availableBookies); remainBookies.removeAll(excludeBookies); if (remainBookies.iterator().hasNext()) { return remainBookies.iterator().next(); @@ -351,7 +356,7 @@ public BookieSocketAddress answer(InvocationOnMock invocation) throws Throwable }); } - protected void registerMockEntryForRead(long ledgerId, long entryId, BookieSocketAddress bookieSocketAddress, + protected void registerMockEntryForRead(long ledgerId, long entryId, BookieId bookieSocketAddress, byte[] entryData, long lastAddConfirmed) { getMockLedgerContentsInBookie(ledgerId, bookieSocketAddress).put(entryId, new MockEntry(entryData, lastAddConfirmed)); @@ -463,7 +468,7 @@ private void setupWriteLedgerMetadata() { protected void setupBookieClientReadEntry() { final Stubber stub = doAnswer(invokation -> { Object[] args = invokation.getArguments(); - BookieSocketAddress bookieSocketAddress = (BookieSocketAddress) args[0]; + BookieId bookieSocketAddress = (BookieId) args[0]; long ledgerId = (Long) args[1]; long entryId = (Long) args[2]; BookkeeperInternalCallbacks.ReadEntryCallback callback = @@ -542,7 +547,7 @@ protected void setupBookieClientAddEntry() { final Stubber stub = doAnswer(invokation -> { Object[] args = invokation.getArguments(); BookkeeperInternalCallbacks.WriteCallback callback = (BookkeeperInternalCallbacks.WriteCallback) args[5]; - BookieSocketAddress bookieSocketAddress = (BookieSocketAddress) args[0]; + BookieId bookieSocketAddress = (BookieId) args[0]; long ledgerId = (Long) args[1]; long entryId = (Long) args[3]; ByteBufList toSend = (ByteBufList) args[4]; @@ -587,7 +592,7 @@ protected void setupBookieClientAddEntry() { return null; }); - stub.when(bookieClient).addEntry(any(BookieSocketAddress.class), + stub.when(bookieClient).addEntry(any(BookieId.class), anyLong(), any(byte[].class), anyLong(), any(ByteBufList.class), any(BookkeeperInternalCallbacks.WriteCallback.class), @@ -598,7 +603,7 @@ protected void setupBookieClientAddEntry() { protected void setupBookieClientForceLedger() { final Stubber stub = doAnswer(invokation -> { Object[] args = invokation.getArguments(); - BookieSocketAddress bookieSocketAddress = (BookieSocketAddress) args[0]; + BookieId bookieSocketAddress = (BookieId) args[0]; long ledgerId = (Long) args[1]; BookkeeperInternalCallbacks.ForceLedgerCallback callback = (BookkeeperInternalCallbacks.ForceLedgerCallback) args[2]; @@ -623,7 +628,7 @@ protected void setupBookieClientForceLedger() { return null; }); - stub.when(bookieClient).forceLedger(any(BookieSocketAddress.class), + stub.when(bookieClient).forceLedger(any(BookieId.class), anyLong(), any(BookkeeperInternalCallbacks.ForceLedgerCallback.class), any()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java index f0be8d02a3b..68d0502a7be 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockClientContext.java @@ -61,7 +61,9 @@ static MockClientContext create() throws Exception { MockRegistrationClient regClient = new MockRegistrationClient(); EnsemblePlacementPolicy placementPolicy = new DefaultEnsemblePlacementPolicy(); BookieWatcherImpl bookieWatcherImpl = new BookieWatcherImpl(conf, placementPolicy, - regClient, NullStatsLogger.INSTANCE); + regClient, + new DefaultBookieAddressResolver(regClient), + NullStatsLogger.INSTANCE); bookieWatcherImpl.initialBlockingBookieRead(); return new MockClientContext() diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java index 9d68ec8a145..5792fcf6726 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java @@ -43,6 +43,7 @@ import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Versioned; @@ -269,10 +270,9 @@ public CompletableFuture readLastAddConfirmedAndEntryAsyn } private static LedgerMetadata createMetadata(DigestType digest, byte[] passwd) { - List ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234)); + List ensemble = Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.2", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.3", 1234).toBookieId()); return LedgerMetadataBuilder.create() .withDigestType(digest.toApiDigestType()) .withPassword(passwd) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index e3c295e48af..0b525017038 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -55,7 +55,7 @@ import org.apache.bookkeeper.meta.exceptions.MetadataException; import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver; import org.apache.bookkeeper.meta.zk.ZKMetadataClientDriver; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; @@ -440,7 +440,7 @@ public void testRecoveryOnEntryGap() throws Exception { new WriteCallback() { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { addSuccess.set(BKException.Code.OK == rc); addLatch.countDown(); } @@ -494,12 +494,12 @@ static final class WriteCallbackEntry { private final int rc; private final long ledgerId; private final long entryId; - private final BookieSocketAddress addr; + private final BookieId addr; private final Object ctx; WriteCallbackEntry(WriteCallback cb, int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { this.cb = cb; this.rc = rc; this.ledgerId = ledgerId; @@ -531,7 +531,7 @@ public void addEntry(ByteBuf entry, boolean ackBeforeSync, final WriteCallback c super.addEntry(entry, ackBeforeSync, new WriteCallback() { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (delayAddResponse.get()) { delayQueue.add(new WriteCallbackEntry(cb, rc, ledgerId, entryId, addr, ctx)); } else { @@ -594,7 +594,7 @@ public void testRecoveryWhenClosingLedgerHandle() throws Exception { LOG.info("Create ledger {}", lh0.getId()); // 0) place the bookie with a fake bookie - BookieSocketAddress address = lh0.getCurrentEnsemble().get(0); + BookieId address = lh0.getCurrentEnsemble().get(0); ServerConfiguration conf = killBookie(address); conf.setLedgerStorageClass(InterleavedLedgerStorage.class.getName()); DelayResponseBookie fakeBookie = new DelayResponseBookie(conf); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java index e1c203d2b96..b1f385ab333 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java @@ -55,6 +55,7 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieProtocol; @@ -102,9 +103,9 @@ public void setup() throws Exception { internalConf = ClientInternalConf.fromConfig(conf); // metadata - ArrayList ensemble = new ArrayList<>(3); + ArrayList ensemble = new ArrayList<>(3); for (int i = 0; i < 3; i++) { - ensemble.add(new BookieSocketAddress("127.0.0.1", 3181 + i)); + ensemble.add(new BookieSocketAddress("127.0.0.1", 3181 + i).toBookieId()); } this.ledgerMetadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(2) @@ -147,7 +148,7 @@ public void teardown() { @Data static class ReadLastConfirmedAndEntryHolder { - private final BookieSocketAddress address; + private final BookieId address; private final ReadEntryCallback callback; private final ReadLastConfirmedAndEntryContext context; @@ -171,10 +172,10 @@ public void testSpeculativeResponses() throws Exception { byte[] bytesWithDigest = new byte[dataWithDigest.readableBytes()]; assertEquals(bytesWithDigest.length, dataWithDigest.getBytes(bytesWithDigest)); - final Map callbacks = + final Map callbacks = Collections.synchronizedMap(new HashMap<>()); doAnswer(invocationOnMock -> { - BookieSocketAddress address = invocationOnMock.getArgument(0); + BookieId address = invocationOnMock.getArgument(0); ReadEntryCallback callback = invocationOnMock.getArgument(6); ReadLastConfirmedAndEntryContext context = invocationOnMock.getArgument(7); @@ -184,8 +185,7 @@ public void testSpeculativeResponses() throws Exception { callbacks.put(address, holder); return null; - }).when(mockBookieClient).readEntryWaitForLACUpdate( - any(BookieSocketAddress.class), + }).when(mockBookieClient).readEntryWaitForLACUpdate(any(BookieId.class), anyLong(), anyLong(), anyLong(), @@ -222,9 +222,9 @@ public void testSpeculativeResponses() throws Exception { // 2) complete second bookie with valid entry response. this will trigger double-release bug described in // {@link https://github.com/apache/bookkeeper/issues/1476} - Iterator> iter = callbacks.entrySet().iterator(); + Iterator> iter = callbacks.entrySet().iterator(); assertTrue(iter.hasNext()); - Entry firstBookieEntry = iter.next(); + Entry firstBookieEntry = iter.next(); ReadLastConfirmedAndEntryHolder firstBookieHolder = firstBookieEntry.getValue(); ReadLastConfirmedAndEntryContext firstContext = firstBookieHolder.context; firstContext.setLastAddConfirmed(entryId); @@ -236,7 +236,7 @@ public void testSpeculativeResponses() throws Exception { LedgerEntryImpl entry = LedgerEntryImpl.create(LEDGERID, Long.MAX_VALUE); assertTrue(iter.hasNext()); - Entry secondBookieEntry = iter.next(); + Entry secondBookieEntry = iter.next(); ReadLastConfirmedAndEntryHolder secondBookieHolder = secondBookieEntry.getValue(); ReadLastConfirmedAndEntryContext secondContext = secondBookieHolder.context; secondContext.setLastAddConfirmed(entryId); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java index e553e52e98d..e063e502142 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; import org.apache.bookkeeper.proto.MockBookieClient; @@ -42,8 +43,8 @@ */ public class ReadLastConfirmedOpTest { private static final Logger log = LoggerFactory.getLogger(ReadLastConfirmedOpTest.class); - private final BookieSocketAddress bookie1 = new BookieSocketAddress("bookie1", 3181); - private final BookieSocketAddress bookie2 = new BookieSocketAddress("bookie2", 3181); + private final BookieId bookie1 = new BookieSocketAddress("bookie1", 3181).toBookieId(); + private final BookieId bookie2 = new BookieSocketAddress("bookie2", 3181).toBookieId(); OrderedExecutor executor = null; @@ -68,7 +69,7 @@ public void teardown() throws Exception { @Test public void testBookieFailsAfterLedgerMissingOnFirst() throws Exception { long ledgerId = 0xf00b; - List ensemble = Lists.newArrayList(bookie1, bookie2); + List ensemble = Lists.newArrayList(bookie1, bookie2); byte[] ledgerKey = new byte[0]; MockBookieClient bookieClient = new MockBookieClient(executor); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java index a70ec32de7c..47df744525c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java @@ -33,7 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -73,7 +73,7 @@ public void testSlowBookie() throws Exception { final CountDownLatch b0latch = new CountDownLatch(1); final CountDownLatch b1latch = new CountDownLatch(1); final CountDownLatch addEntrylatch = new CountDownLatch(1); - List curEns = lh.getCurrentEnsemble(); + List curEns = lh.getCurrentEnsemble(); try { sleepBookie(curEns.get(0), b0latch); for (int i = 0; i < 10; i++) { @@ -295,7 +295,7 @@ private LedgerHandle doBackpressureTest(byte[] entry, ClientConfiguration conf, return lh2; } - private void setTargetChannelState(BookKeeper bkc, BookieSocketAddress address, + private void setTargetChannelState(BookKeeper bkc, BookieId address, long key, boolean state) throws Exception { ((BookieClientImpl) bkc.getBookieClient()).lookupClient(address).obtain((rc, pcbc) -> { pcbc.setWritable(state); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestAddEntryQuorumTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestAddEntryQuorumTimeout.java index efbc43d5070..27b56af7a91 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestAddEntryQuorumTimeout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestAddEntryQuorumTimeout.java @@ -28,7 +28,7 @@ import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Before; import org.junit.Test; @@ -81,7 +81,7 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { public void testBasicTimeout() throws Exception { BookKeeperTestClient bkc = new BookKeeperTestClient(baseClientConf); LedgerHandle lh = bkc.createLedger(3, 3, 3, digestType, testPasswd); - List curEns = lh.getCurrentEnsemble(); + List curEns = lh.getCurrentEnsemble(); byte[] data = "foobar".getBytes(); lh.addEntry(data); sleepBookie(curEns.get(0), 5).await(); @@ -105,7 +105,7 @@ private void waitForSyncObj(SyncObj syncObj) throws Exception { public void testTimeoutWithPendingOps() throws Exception { BookKeeperTestClient bkc = new BookKeeperTestClient(baseClientConf); LedgerHandle lh = bkc.createLedger(3, 3, 3, digestType, testPasswd); - List curEns = lh.getCurrentEnsemble(); + List curEns = lh.getCurrentEnsemble(); byte[] data = "foobar".getBytes(); SyncObj syncObj1 = new SyncObj(); @@ -130,7 +130,7 @@ public void testTimeoutWithPendingOps() throws Exception { public void testLedgerClosedAfterTimeout() throws Exception { BookKeeperTestClient bkc = new BookKeeperTestClient(baseClientConf); LedgerHandle lh = bkc.createLedger(3, 3, 3, digestType, testPasswd); - List curEns = lh.getCurrentEnsemble(); + List curEns = lh.getCurrentEnsemble(); byte[] data = "foobar".getBytes(); CountDownLatch b0latch = sleepBookie(curEns.get(0), 5); try { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java index cbee711624a..00fabeee485 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Assert; import org.junit.Test; @@ -55,7 +55,7 @@ public void testBkQuarantine() throws Exception { lh.addEntry(msg); } - BookieSocketAddress bookieToQuarantine = lh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); + BookieId bookieToQuarantine = lh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); sleepBookie(bookieToQuarantine, baseClientConf.getAddEntryTimeout() * 2).await(); byte[] tempMsg = "temp-msg".getBytes(); @@ -97,7 +97,7 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { public void testNoQuarantineOnBkRestart() throws Exception { final LedgerHandle lh = bkc.createLedger(2, 2, 2, BookKeeper.DigestType.CRC32, new byte[] {}); final int numEntries = 20; - BookieSocketAddress bookieToRestart = lh.getLedgerMetadata().getEnsembleAt(0).get(0); + BookieId bookieToRestart = lh.getLedgerMetadata().getEnsembleAt(0).get(0); // we add entries on a separate thread so that we can restart a bookie on the current thread Thread addEntryThread = new Thread() { @@ -132,8 +132,8 @@ public void testNoQuarantineOnExpectedBkErrors() throws Exception { byte[] msg = ("msg-" + i).getBytes(); lh.addEntry(msg); } - BookieSocketAddress bookie1 = lh.getLedgerMetadata().getEnsembleAt(0).get(0); - BookieSocketAddress bookie2 = lh.getLedgerMetadata().getEnsembleAt(0).get(1); + BookieId bookie1 = lh.getLedgerMetadata().getEnsembleAt(0).get(0); + BookieId bookie2 = lh.getLedgerMetadata().getEnsembleAt(0).get(1); try { // we read an entry that is not added lh.readEntries(10, 10); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java index f8c5be5876a..7b73f8b1acc 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java @@ -39,7 +39,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Before; @@ -104,9 +104,9 @@ private void verifyEntries(LedgerHandle lh, long startEntry, long untilEntry, LedgerMetadata md = lh.getLedgerMetadata(); for (long eid = startEntry; eid < untilEntry; eid++) { - List addresses = md.getEnsembleAt(eid); + List addresses = md.getEnsembleAt(eid); VerificationCallback callback = new VerificationCallback(addresses.size()); - for (BookieSocketAddress addr : addresses) { + for (BookieId addr : addresses) { bkc.getBookieClient().readEntry(addr, lh.getId(), eid, callback, addr, 0, null); } @@ -122,9 +122,9 @@ private void verifyEntriesRange(LedgerHandle lh, long startEntry, long untilEntr LedgerMetadata md = lh.getLedgerMetadata(); for (long eid = startEntry; eid < untilEntry; eid++) { - List addresses = md.getEnsembleAt(eid); + List addresses = md.getEnsembleAt(eid); VerificationCallback callback = new VerificationCallback(addresses.size()); - for (BookieSocketAddress addr : addresses) { + for (BookieId addr : addresses) { bkc.getBookieClient().readEntry(addr, lh.getId(), eid, callback, addr, 0, null); } @@ -193,7 +193,7 @@ public void testChangeEnsembleIfBrokenAckQuorum() throws Exception { lh.addEntry(data); } - for (BookieSocketAddress addr : lh.getLedgerMetadata().getAllEnsembles().get(0L)) { + for (BookieId addr : lh.getLedgerMetadata().getAllEnsembles().get(0L)) { assertTrue( LEDGER_ENSEMBLE_BOOKIE_DISTRIBUTION + " should be > 0 for " + addr, bkc.getTestStatsProvider().getCounter( @@ -258,8 +258,8 @@ public void testChangeEnsembleIfBrokenAckQuorum() throws Exception { CLIENT_SCOPE + "." + WATCHER_SCOPE + "." + REPLACE_BOOKIE_TIME) .getSuccessCount() > 0); - List firstFragment = lh.getLedgerMetadata().getEnsembleAt(0); - List secondFragment = lh.getLedgerMetadata().getEnsembleAt(3 * numEntries); + List firstFragment = lh.getLedgerMetadata().getEnsembleAt(0); + List secondFragment = lh.getLedgerMetadata().getEnsembleAt(3 * numEntries); assertFalse(firstFragment.get(0).equals(secondFragment.get(0))); assertFalse(firstFragment.get(1).equals(secondFragment.get(1))); assertFalse(firstFragment.get(2).equals(secondFragment.get(2))); @@ -430,8 +430,8 @@ public void testChangeEnsembleSecondBookieReadOnly() throws Exception { lh.addEntry(data); } - BookieSocketAddress failedBookie = lh.getCurrentEnsemble().get(0); - BookieSocketAddress readOnlyBookie = lh.getCurrentEnsemble().get(1); + BookieId failedBookie = lh.getCurrentEnsemble().get(0); + BookieId readOnlyBookie = lh.getCurrentEnsemble().get(1); ServerConfiguration conf0 = killBookie(failedBookie); for (int i = 0; i < numEntries; i++) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java index 4417f543ae4..9a3c41e816a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java @@ -41,7 +41,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.feature.SettableFeature; import org.apache.bookkeeper.feature.SettableFeatureProvider; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; import org.slf4j.Logger; @@ -89,7 +89,7 @@ void disableEnsembleChangeTest(boolean startNewBookie) throws Exception { final byte[] entry = "test-disable-ensemble-change".getBytes(UTF_8); assertEquals(1, lh.getLedgerMetadata().getAllEnsembles().size()); - ArrayList ensembleBeforeFailure = + ArrayList ensembleBeforeFailure = new ArrayList<>(lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next().getValue()); final RateLimiter rateLimiter = RateLimiter.create(10); @@ -121,10 +121,10 @@ public void run() { // check the ensemble after failure assertEquals("No new ensemble should be added when disable ensemble change.", 1, lh.getLedgerMetadata().getAllEnsembles().size()); - ArrayList ensembleAfterFailure = + ArrayList ensembleAfterFailure = new ArrayList<>(lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next().getValue()); - assertArrayEquals(ensembleBeforeFailure.toArray(new BookieSocketAddress[ensembleBeforeFailure.size()]), - ensembleAfterFailure.toArray(new BookieSocketAddress[ensembleAfterFailure.size()])); + assertArrayEquals(ensembleBeforeFailure.toArray(new BookieId[ensembleBeforeFailure.size()]), + ensembleAfterFailure.toArray(new BookieId[ensembleAfterFailure.size()])); // enable ensemble change disableEnsembleChangeFeature.set(false); @@ -241,7 +241,7 @@ public void testRetrySlowBookie() throws Exception { lh.addEntry(entry); } - List curEns = lh.getCurrentEnsemble(); + List curEns = lh.getCurrentEnsemble(); final CountDownLatch wakeupLatch = new CountDownLatch(1); final CountDownLatch suspendLatch = new CountDownLatch(1); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java index 096273e1f1a..208742fcc8e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java @@ -29,7 +29,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; import org.slf4j.Logger; @@ -266,7 +266,7 @@ public void testFencingInteractionWithBookieRecovery() throws Exception { writelh.addEntry(tmp.getBytes()); } - BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); + BookieId bookieToKill = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); killBookie(bookieToKill); // write entries to change ensemble @@ -318,7 +318,7 @@ public void testFencingInteractionWithBookieRecovery2() throws Exception { LedgerHandle readlh = bkc.openLedger(writelh.getId(), digestType, "testPasswd".getBytes()); // should be fenced by now - BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); + BookieId bookieToKill = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); killBookie(bookieToKill); admin.recoverBookieData(bookieToKill); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java index 39c615a63a4..8541420c050 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java @@ -37,7 +37,9 @@ import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; @@ -101,16 +103,15 @@ public void testGetBookieInfoTimeout() throws Exception { ClientConfiguration cConf = new ClientConfiguration(); cConf.setGetBookieInfoTimeout(2); - final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsembleAt(0).get(0); + final BookieId bookieToSleep = writelh.getLedgerMetadata().getEnsembleAt(0).get(0); int sleeptime = cConf.getBookieInfoTimeout() * 3; CountDownLatch latch = sleepBookie(bookieToSleep, sleeptime); latch.await(); + BookieAddressResolver bookieAddressResolver = BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER; // try to get bookie info from the sleeping bookie. It should fail with timeout error - BookieSocketAddress addr = new BookieSocketAddress(bookieToSleep.getSocketAddress().getHostString(), - bookieToSleep.getPort()); BookieClient bc = new BookieClientImpl(cConf, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, executor, - scheduler, NullStatsLogger.INSTANCE); + scheduler, NullStatsLogger.INSTANCE, bookieAddressResolver); long flags = BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE | BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; @@ -128,7 +129,7 @@ class CallbackObj { } } CallbackObj obj = new CallbackObj(flags); - bc.getBookieInfo(addr, flags, new GetBookieInfoCallback() { + bc.getBookieInfo(bookieToSleep, flags, new GetBookieInfoCallback() { @Override public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) { CallbackObj obj = (CallbackObj) ctx; 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 9e3162645f9..de86776c03b 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 @@ -28,7 +28,7 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; @@ -78,7 +78,7 @@ public void testChecker() throws Exception { for (int i = 0; i < 10; i++) { lh.addEntry(TEST_LEDGER_ENTRY_DATA); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles() + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles() .get(0L).get(0); LOG.info("Killing {}", replicaToKill); killBookie(replicaToKill); @@ -97,7 +97,7 @@ public void testChecker() throws Exception { assertTrue("Fragment should be missing from first replica", result.iterator().next().getAddresses().contains(replicaToKill)); - BookieSocketAddress replicaToKill2 = lh.getLedgerMetadata() + BookieId replicaToKill2 = lh.getLedgerMetadata() .getAllEnsembles().get(0L).get(1); LOG.info("Killing {}", replicaToKill2); killBookie(replicaToKill2); @@ -137,9 +137,9 @@ public void testShouldNotGetTheFragmentIfThereIsNoMissedEntry() // Entry should have added in first 2 Bookies. // Kill the 3rd BK from ensemble. - List firstEnsemble = lh.getLedgerMetadata() + List firstEnsemble = lh.getLedgerMetadata() .getAllEnsembles().get(0L); - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(2); + BookieId lastBookieFromEnsemble = firstEnsemble.get(2); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -180,13 +180,13 @@ public void testShouldGetTwoFrgamentsIfTwoBookiesFailedInSameEnsemble() startNewBookie(); lh.addEntry(TEST_LEDGER_ENTRY_DATA); - List firstEnsemble = lh.getLedgerMetadata() + List firstEnsemble = lh.getLedgerMetadata() .getAllEnsembles().get(0L); - BookieSocketAddress firstBookieFromEnsemble = firstEnsemble.get(0); + BookieId firstBookieFromEnsemble = firstEnsemble.get(0); killBookie(firstEnsemble, firstBookieFromEnsemble); - BookieSocketAddress secondBookieFromEnsemble = firstEnsemble.get(1); + BookieId secondBookieFromEnsemble = firstEnsemble.get(1); killBookie(firstEnsemble, secondBookieFromEnsemble); lh.addEntry(TEST_LEDGER_ENTRY_DATA); Set result = getUnderReplicatedFragments(lh); @@ -213,9 +213,9 @@ public void testShouldNotGetAnyFragmentIfNoLedgerPresent() LedgerHandle lh = bkc.createLedger(3, 2, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); - List firstEnsemble = lh.getLedgerMetadata() + List firstEnsemble = lh.getLedgerMetadata() .getAllEnsembles().get(0L); - BookieSocketAddress firstBookieFromEnsemble = firstEnsemble.get(0); + BookieId firstBookieFromEnsemble = firstEnsemble.get(0); killBookie(firstBookieFromEnsemble); startNewBookie(); lh.addEntry(TEST_LEDGER_ENTRY_DATA); @@ -258,9 +258,9 @@ public void testShouldGetFailedEnsembleNumberOfFgmntsIfEnsembleBookiesFailedOnNe } // Kill all three bookies - List firstEnsemble = lh.getLedgerMetadata() + List firstEnsemble = lh.getLedgerMetadata() .getAllEnsembles().get(0L); - for (BookieSocketAddress bkAddr : firstEnsemble) { + for (BookieId bkAddr : firstEnsemble) { killBookie(firstEnsemble, bkAddr); } @@ -302,7 +302,7 @@ public void testShouldNotGetAnyFragmentWithEmptyLedger() throws Exception { public void testShouldGet2FragmentsWithEmptyLedgerButBookiesDead() throws Exception { LedgerHandle lh = bkc.createLedger(3, 2, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); - for (BookieSocketAddress b : lh.getLedgerMetadata().getAllEnsembles().get(0L)) { + for (BookieId b : lh.getLedgerMetadata().getAllEnsembles().get(0L)) { killBookie(b); } Set result = getUnderReplicatedFragments(lh); @@ -321,9 +321,9 @@ public void testShouldGetOneFragmentWithSingleEntryOpenedLedger() throws Excepti LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); lh.addEntry(TEST_LEDGER_ENTRY_DATA); - List firstEnsemble = lh.getLedgerMetadata() + List firstEnsemble = lh.getLedgerMetadata() .getAllEnsembles().get(0L); - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(0); + BookieId lastBookieFromEnsemble = firstEnsemble.get(0); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -355,10 +355,10 @@ public void testSingleEntryAfterEnsembleChange() throws Exception { for (int i = 0; i < 10; i++) { lh.addEntry(TEST_LEDGER_ENTRY_DATA); } - List firstEnsemble = lh.getLedgerMetadata() + List firstEnsemble = lh.getLedgerMetadata() .getAllEnsembles().get(0L); DistributionSchedule.WriteSet writeSet = lh.getDistributionSchedule().getWriteSet(lh.getLastAddPushed()); - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(writeSet.get(0)); + BookieId lastBookieFromEnsemble = firstEnsemble.get(writeSet.get(0)); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -400,11 +400,11 @@ public void testSingleEntryAfterEnsembleChange() throws Exception { public void testClosedEmptyLedger() throws Exception { LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); - List firstEnsemble = lh.getLedgerMetadata() + List firstEnsemble = lh.getLedgerMetadata() .getAllEnsembles().get(0L); lh.close(); - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(0); + BookieId lastBookieFromEnsemble = firstEnsemble.get(0); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -427,13 +427,13 @@ public void testClosedEmptyLedger() throws Exception { public void testClosedSingleEntryLedger() throws Exception { LedgerHandle lh = bkc.createLedger(3, 2, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); - List firstEnsemble = lh.getLedgerMetadata() + List firstEnsemble = lh.getLedgerMetadata() .getAllEnsembles().get(0L); lh.addEntry(TEST_LEDGER_ENTRY_DATA); lh.close(); // kill bookie 2 - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(2); + BookieId lastBookieFromEnsemble = firstEnsemble.get(2); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -496,7 +496,7 @@ private Set getUnderReplicatedFragments(LedgerHandle lh) return result; } - private void killBookie(List firstEnsemble, BookieSocketAddress ensemble) + private void killBookie(List firstEnsemble, BookieId ensemble) throws Exception { LOG.info("Killing " + ensemble + " from ensemble=" + firstEnsemble); killBookie(ensemble); 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 ae8dfdd5cd7..25a3de80fac 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 @@ -40,6 +40,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.WriteFlag; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -96,13 +97,13 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles() + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles() .get(0L).get(0); LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr : {}", newBkAddr); for (int i = 0; i < 10; i++) { @@ -120,11 +121,11 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() } // Killing all bookies except newly replicated bookie - SortedMap> allBookiesBeforeReplication = lh + SortedMap> allBookiesBeforeReplication = lh .getLedgerMetadata().getAllEnsembles(); - for (Entry> entry : allBookiesBeforeReplication.entrySet()) { - List bookies = entry.getValue(); - for (BookieSocketAddress bookie : bookies) { + for (Entry> entry : allBookiesBeforeReplication.entrySet()) { + List bookies = entry.getValue(); + for (BookieId bookie : bookies) { if (newBkAddr.equals(bookie)) { continue; } @@ -150,7 +151,7 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles() + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles() .get(0L).get(0); startNewBookie(); @@ -162,10 +163,10 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() lh.addEntry(data); } - BookieSocketAddress replicaToKill2 = lh.getLedgerMetadata() + BookieId replicaToKill2 = lh.getLedgerMetadata() .getAllEnsembles().get(0L).get(1); - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr : {}", newBkAddr); LOG.info("Killing Bookie : {}", replicaToKill2); @@ -209,7 +210,7 @@ public void testReplicateLFShouldReturnFalseIfTheReplicationFails() } // Kill the first Bookie - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles() + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles() .get(0L).get(0); killBookie(replicaToKill); LOG.info("Killed Bookie =" + replicaToKill); @@ -241,10 +242,9 @@ public void testReplicateLFShouldReturnFalseIfTheReplicationFails() @Test public void testSplitIntoSubFragmentsWithDifferentFragmentBoundaries() throws Exception { - List ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234)); + List ensemble = Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.2", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.3", 1234).toBookieId()); LedgerMetadata metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) .withPassword(TEST_PSSWD).withDigestType(TEST_DIGEST_TYPE.toApiDigestType()) @@ -353,14 +353,14 @@ public void testSplitLedgerFragmentState() throws Exception { int lastEntryId = 10; int rereplicationEntryBatchSize = 10; - List ensemble = new ArrayList(); - ensemble.add(new BookieSocketAddress("bookie0:3181")); - ensemble.add(new BookieSocketAddress("bookie1:3181")); - ensemble.add(new BookieSocketAddress("bookie2:3181")); - ensemble.add(new BookieSocketAddress("bookie3:3181")); - ensemble.add(new BookieSocketAddress("bookie4:3181")); - ensemble.add(new BookieSocketAddress("bookie5:3181")); - ensemble.add(new BookieSocketAddress("bookie6:3181")); + List ensemble = new ArrayList(); + ensemble.add(BookieId.parse("bookie0:3181")); + ensemble.add(BookieId.parse("bookie1:3181")); + ensemble.add(BookieId.parse("bookie2:3181")); + ensemble.add(BookieId.parse("bookie3:3181")); + ensemble.add(BookieId.parse("bookie4:3181")); + ensemble.add(BookieId.parse("bookie5:3181")); + ensemble.add(BookieId.parse("bookie6:3181")); LedgerMetadataBuilder builder = LedgerMetadataBuilder.create(); builder.withEnsembleSize(7).withWriteQuorumSize(3).withAckQuorumSize(2) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java index 84edc62eb95..e4801cf2fb4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java @@ -35,7 +35,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; import org.slf4j.Logger; @@ -156,7 +156,7 @@ public void testFailParallelRecoveryReadMissingEntryImmediately() throws Excepti LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - List ensemble = lh.getLedgerMetadata().getEnsembleAt(10); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(10); CountDownLatch latch1 = new CountDownLatch(1); CountDownLatch latch2 = new CountDownLatch(1); // sleep two bookie @@ -187,7 +187,7 @@ public void testParallelReadWithFailedBookies() throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - List ensemble = lh.getLedgerMetadata().getEnsembleAt(5); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(5); // kill two bookies killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); @@ -223,7 +223,7 @@ public void testParallelReadFailureWithFailedBookies() throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - List ensemble = lh.getLedgerMetadata().getEnsembleAt(5); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(5); // kill two bookies killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); 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 51e7ea6e7ce..f0520d45e24 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 @@ -44,11 +44,13 @@ import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.EnsembleForReplacementWithNoConstraints; import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.TruePredicate; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.net.Node; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.TestStatsProvider; @@ -67,10 +69,11 @@ public class TestRackawareEnsemblePlacementPolicy extends TestCase { static final Logger LOG = LoggerFactory.getLogger(TestRackawareEnsemblePlacementPolicy.class); RackawareEnsemblePlacementPolicy repp; - final List ensemble = new ArrayList(); + final List ensemble = new ArrayList(); DistributionSchedule.WriteSet writeSet = DistributionSchedule.NULL_WRITE_SET; ClientConfiguration conf = new ClientConfiguration(); - BookieSocketAddress addr1, addr2, addr3, addr4; + BookieSocketAddress addr1; + BookieSocketAddress addr2, addr3, addr4; io.netty.util.HashedWheelTimer timer; final int minNumRacksPerWriteQuorumConfValue = 2; @@ -94,10 +97,10 @@ protected void setUp() throws Exception { StaticDNSResolver.addNodeToRack(addr2.getHostName(), NetworkTopology.DEFAULT_REGION_AND_RACK); StaticDNSResolver.addNodeToRack(addr3.getHostName(), NetworkTopology.DEFAULT_REGION_AND_RACK); StaticDNSResolver.addNodeToRack(addr4.getHostName(), NetworkTopology.DEFAULT_REGION + "/rack2"); - ensemble.add(addr1); - ensemble.add(addr2); - ensemble.add(addr3); - ensemble.add(addr4); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr3.toBookieId()); + ensemble.add(addr4.toBookieId()); writeSet = writeSetFromValues(0, 1, 2, 3); timer = new HashedWheelTimer( @@ -106,7 +109,8 @@ protected void setUp() throws Exception { conf.getTimeoutTimerNumTicks()); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); } @@ -120,16 +124,16 @@ static BookiesHealthInfo getBookiesHealthInfo() { return getBookiesHealthInfo(new HashMap<>(), new HashMap<>()); } - static BookiesHealthInfo getBookiesHealthInfo(Map bookieFailureHistory, - Map bookiePendingRequests) { + static BookiesHealthInfo getBookiesHealthInfo(Map bookieFailureHistory, + Map bookiePendingRequests) { return new BookiesHealthInfo() { @Override - public long getBookieFailureHistory(BookieSocketAddress bookieSocketAddress) { + public long getBookieFailureHistory(BookieId bookieSocketAddress) { return bookieFailureHistory.getOrDefault(bookieSocketAddress, -1L); } @Override - public long getBookiePendingRequests(BookieSocketAddress bookieSocketAddress) { + public long getBookiePendingRequests(BookieId bookieSocketAddress) { return bookiePendingRequests.getOrDefault(bookieSocketAddress, 0L); } }; @@ -148,17 +152,18 @@ public void testNodeDown() throws Exception { updateMyRack(NetworkTopology.DEFAULT_REGION_AND_RACK); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -176,19 +181,20 @@ public void testNodeReadOnly() throws Exception { updateMyRack("/r1/rack1"); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - Set ro = new HashSet(); - ro.add(addr1); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + Set ro = new HashSet(); + ro.add(addr1.toBookieId()); repp.onClusterChanged(addrs, ro); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -204,19 +210,20 @@ public void testNodeSlow() throws Exception { updateMyRack("/r1/rack1"); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - repp.registerSlowBookie(addr1, 0L); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 1L); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + repp.registerSlowBookie(addr1.toBookieId(), 0L); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 1L); repp.onClusterChanged(addrs, new HashSet<>()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -234,21 +241,22 @@ public void testTwoNodesSlow() throws Exception { updateMyRack("/r1/rack1"); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - repp.registerSlowBookie(addr1, 0L); - repp.registerSlowBookie(addr2, 0L); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 1L); - bookiePendingMap.put(addr2, 2L); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + repp.registerSlowBookie(addr1.toBookieId(), 0L); + repp.registerSlowBookie(addr2.toBookieId(), 0L); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 1L); + bookiePendingMap.put(addr2.toBookieId(), 2L); repp.onClusterChanged(addrs, new HashSet<>()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -266,19 +274,20 @@ public void testTwoNodesDown() throws Exception { updateMyRack("/r1/rack1"); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - addrs.remove(addr2); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + addrs.remove(addr2.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -295,20 +304,21 @@ public void testNodeDownAndReadOnly() throws Exception { updateMyRack("/r1/rack1"); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - addrs.remove(addr2); - Set roAddrs = new HashSet(); - roAddrs.add(addr2); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + addrs.remove(addr2.toBookieId()); + Set roAddrs = new HashSet(); + roAddrs.add(addr2.toBookieId()); repp.onClusterChanged(addrs, roAddrs); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -324,20 +334,21 @@ public void testNodeDownAndNodeSlow() throws Exception { updateMyRack("/r1/rack1"); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - repp.registerSlowBookie(addr1, 0L); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 1L); - addrs.remove(addr2); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + repp.registerSlowBookie(addr1.toBookieId(), 0L); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 1L); + addrs.remove(addr2.toBookieId()); repp.onClusterChanged(addrs, new HashSet<>()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -355,24 +366,25 @@ public void testNodeDownAndReadOnlyAndNodeSlow() throws Exception { updateMyRack("/r1/rack1"); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - addrs.remove(addr2); - Set ro = new HashSet(); - ro.add(addr2); - repp.registerSlowBookie(addr3, 0L); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr3, 1L); - addrs.remove(addr2); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + addrs.remove(addr2.toBookieId()); + Set ro = new HashSet(); + ro.add(addr2.toBookieId()); + repp.registerSlowBookie(addr3.toBookieId(), 0L); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr3.toBookieId(), 1L); + addrs.remove(addr2.toBookieId()); repp.onClusterChanged(addrs, ro); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -397,21 +409,22 @@ public void testPendingRequestsReorder() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); ClientConfiguration conf = (ClientConfiguration) this.conf.clone(); conf.setReorderThresholdPendingRequests(10); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 20L); - bookiePendingMap.put(addr2, 7L); - bookiePendingMap.put(addr3, 1L); // best bookie -> this one first - bookiePendingMap.put(addr4, 5L); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 20L); + bookiePendingMap.put(addr2.toBookieId(), 7L); + bookiePendingMap.put(addr3.toBookieId(), 1L); // best bookie -> this one first + bookiePendingMap.put(addr4.toBookieId(), 5L); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -435,38 +448,39 @@ public void testPendingRequestsReorderLargeEnsemble() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); ClientConfiguration conf = (ClientConfiguration) this.conf.clone(); conf.setReorderThresholdPendingRequests(10); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - repp.onClusterChanged(addrs, new HashSet()); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 1L); // not in write set - bookiePendingMap.put(addr2, 20L); - bookiePendingMap.put(addr3, 0L); // not in write set - bookiePendingMap.put(addr4, 12L); - bookiePendingMap.put(addr5, 9L); // not in write set - bookiePendingMap.put(addr6, 2L); // best bookie -> this one first - bookiePendingMap.put(addr7, 10L); - List ensemble = new ArrayList(); - ensemble.add(addr1); - ensemble.add(addr2); - ensemble.add(addr3); - ensemble.add(addr4); - ensemble.add(addr5); - ensemble.add(addr6); - ensemble.add(addr7); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 1L); // not in write set + bookiePendingMap.put(addr2.toBookieId(), 20L); + bookiePendingMap.put(addr3.toBookieId(), 0L); // not in write set + bookiePendingMap.put(addr4.toBookieId(), 12L); + bookiePendingMap.put(addr5.toBookieId(), 9L); // not in write set + bookiePendingMap.put(addr6.toBookieId(), 2L); // best bookie -> this one first + bookiePendingMap.put(addr7.toBookieId(), 10L); + List ensemble = new ArrayList(); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr3.toBookieId()); + ensemble.add(addr4.toBookieId()); + ensemble.add(addr5.toBookieId()); + ensemble.add(addr6.toBookieId()); + ensemble.add(addr7.toBookieId()); DistributionSchedule.WriteSet writeSet = writeSetFromValues(1, 3, 5, 6); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -491,21 +505,22 @@ public void testPendingRequestsNoReorder1() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); ClientConfiguration conf = (ClientConfiguration) this.conf.clone(); conf.setReorderThresholdPendingRequests(10); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 10L); // -> this one first - bookiePendingMap.put(addr2, 7L); - bookiePendingMap.put(addr3, 1L); // best bookie, but below threshold - bookiePendingMap.put(addr4, 5L); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 10L); // -> this one first + bookiePendingMap.put(addr2.toBookieId(), 7L); + bookiePendingMap.put(addr3.toBookieId(), 1L); // best bookie, but below threshold + bookiePendingMap.put(addr4.toBookieId(), 5L); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -527,21 +542,22 @@ public void testPendingRequestsNoReorder2() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); ClientConfiguration conf = (ClientConfiguration) this.conf.clone(); conf.setReorderThresholdPendingRequests(10); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 1L); // -> this one first - bookiePendingMap.put(addr2, 7L); - bookiePendingMap.put(addr3, 1L); - bookiePendingMap.put(addr4, 5L); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 1L); // -> this one first + bookiePendingMap.put(addr2.toBookieId(), 7L); + bookiePendingMap.put(addr3.toBookieId(), 1L); + bookiePendingMap.put(addr4.toBookieId(), 5L); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -562,18 +578,18 @@ public void testReplaceBookieWithEnoughBookiesInSameRack() throws Exception { StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r3"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = - repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); - BookieSocketAddress replacedBookie = replaceBookieResponse.getResult(); + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = + repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2.toBookieId(), new HashSet<>()); + BookieId replacedBookie = replaceBookieResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); - assertEquals(addr3, replacedBookie); + assertEquals(addr3.toBookieId(), replacedBookie); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @@ -589,21 +605,21 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRack() throws Exception StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r4"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr1); - EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = - repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, excludedAddrs); - BookieSocketAddress replacedBookie = replaceBookieResponse.getResult(); + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr1.toBookieId()); + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = + repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2.toBookieId(), excludedAddrs); + BookieId replacedBookie = replaceBookieResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); - assertFalse(addr1.equals(replacedBookie)); - assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); + assertFalse(addr1.toBookieId().equals(replacedBookie)); + assertTrue(addr3.toBookieId().equals(replacedBookie) || addr4.toBookieId().equals(replacedBookie)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @@ -619,19 +635,19 @@ public void testReplaceBookieWithNotEnoughBookies() throws Exception { StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r4"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr1); - excludedAddrs.add(addr3); - excludedAddrs.add(addr4); + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr1.toBookieId()); + excludedAddrs.add(addr3.toBookieId()); + excludedAddrs.add(addr4.toBookieId()); try { - repp.replaceBookie(1, 1, 1, null, new ArrayList(), addr2, excludedAddrs); + repp.replaceBookie(1, 1, 1, null, new ArrayList(), addr2.toBookieId(), excludedAddrs); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not enou @@ -650,24 +666,24 @@ public void testReplaceBookieWithEnoughBookiesInSameRackAsEnsemble() throws Exce StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/r3"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - List ensembleBookies = new ArrayList(); - ensembleBookies.add(addr2); - ensembleBookies.add(addr4); - EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = repp.replaceBookie( + List ensembleBookies = new ArrayList(); + ensembleBookies.add(addr2.toBookieId()); + ensembleBookies.add(addr4.toBookieId()); + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse = repp.replaceBookie( 1, 1, 1 , null, ensembleBookies, - addr4, + addr4.toBookieId(), new HashSet<>()); - BookieSocketAddress replacedBookie = replaceBookieResponse.getResult(); + BookieId replacedBookie = replaceBookieResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); - assertEquals(addr1, replacedBookie); + assertEquals(addr1.toBookieId(), replacedBookie); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @@ -678,24 +694,26 @@ public void testNewEnsembleWithSingleRack() throws Exception { BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.8", 3181); BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.9", 3181); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - EnsemblePlacementPolicy.PlacementResult> ensembleResponse; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; ensembleResponse = repp.newEnsemble(3, 2, 2, null, new HashSet<>()); - List ensemble = ensembleResponse.getResult(); + List ensemble = ensembleResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); - assertEquals(0, getNumCoveredWriteQuorums(ensemble, 2, conf.getMinNumRacksPerWriteQuorum())); + assertEquals(0, getNumCoveredWriteQuorums(ensemble, 2, conf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); - EnsemblePlacementPolicy.PlacementResult> ensembleResponse2; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse2; ensembleResponse2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()); - List ensemble2 = ensembleResponse2.getResult(); + List ensemble2 = ensembleResponse2.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isAdheringToPolicy(); - assertEquals(0, getNumCoveredWriteQuorums(ensemble2, 2, conf.getMinNumRacksPerWriteQuorum())); + assertEquals(0, getNumCoveredWriteQuorums(ensemble2, 2, conf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -717,16 +735,16 @@ public void testSingleRackWithEnforceMinNumRacks() throws Exception { clientConf.setEnforceMinNumRacksPerWriteQuorum(true); repp = new RackawareEnsemblePlacementPolicy(); repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - List ensemble; + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + List ensemble; try { ensemble = repp.newEnsemble(3, 2, 2, null, new HashSet<>()).getResult(); fail("Should get not enough bookies exception since there is only one rack."); @@ -755,36 +773,37 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { TestStatsProvider statsProvider = new TestStatsProvider(); TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, statsLogger); + repp.initialize(clientConf, Optional. empty(), timer, + DISABLE_ALL, statsLogger, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(defaultRackForThisTest); Gauge numBookiesInDefaultRackGauge = statsLogger .getGauge(BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK); int numOfRacks = 3; int numOfBookiesPerRack = 5; - BookieSocketAddress[] bookieSocketAddresses = new BookieSocketAddress[numOfRacks * numOfBookiesPerRack]; + BookieId[] bookieSocketAddresses = new BookieId[numOfRacks * numOfBookiesPerRack]; for (int i = 0; i < numOfRacks; i++) { for (int j = 0; j < numOfBookiesPerRack; j++) { int index = i * numOfBookiesPerRack + j; - bookieSocketAddresses[index] = new BookieSocketAddress("128.0.0." + index, 3181); - StaticDNSResolver.addNodeToRack(bookieSocketAddresses[index].getHostName(), "/default-region/r" + i); + bookieSocketAddresses[index] = new BookieSocketAddress("128.0.0." + index, 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("128.0.0." + index, "/default-region/r" + i); } } int numOfBookiesInDefaultRack = 5; - BookieSocketAddress[] bookieSocketAddressesInDefaultRack = new BookieSocketAddress[numOfBookiesInDefaultRack]; + BookieId[] bookieSocketAddressesInDefaultRack = new BookieId[numOfBookiesInDefaultRack]; for (int i = 0; i < numOfBookiesInDefaultRack; i++) { - bookieSocketAddressesInDefaultRack[i] = new BookieSocketAddress("128.0.0." + (100 + i), 3181); - StaticDNSResolver.addNodeToRack(bookieSocketAddressesInDefaultRack[i].getHostName(), + bookieSocketAddressesInDefaultRack[i] = new BookieSocketAddress("128.0.0." + (100 + i), 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("128.0.0." + (100 + i), defaultRackForThisTest); } - List nonDefaultRackBookiesList = Arrays.asList(bookieSocketAddresses); - List defaultRackBookiesList = Arrays.asList(bookieSocketAddressesInDefaultRack); - Set writableBookies = new HashSet(nonDefaultRackBookiesList); + List nonDefaultRackBookiesList = Arrays.asList(bookieSocketAddresses); + List defaultRackBookiesList = Arrays.asList(bookieSocketAddressesInDefaultRack); + Set writableBookies = new HashSet(nonDefaultRackBookiesList); writableBookies.addAll(defaultRackBookiesList); - repp.onClusterChanged(writableBookies, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, numBookiesInDefaultRackGauge.getSample()); @@ -808,8 +827,8 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { * and there are enough bookies in 3 racks, this newEnsemble calls * should succeed. */ - EnsemblePlacementPolicy.PlacementResult> ensembleResponse; - List ensemble; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; + List ensemble; PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; int ensembleSize = numOfRacks * numOfBookiesPerRack; int writeQuorumSize = numOfRacks; @@ -819,7 +838,8 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { ensemble = ensembleResponse.getResult(); isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, - getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, @@ -828,7 +848,8 @@ public void testNewEnsembleWithEnforceMinNumRacks() throws Exception { ensemble = ensembleResponse.getResult(); isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, - getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @@ -843,7 +864,7 @@ public void testNewEnsembleWithSufficientRacksAndEnforceMinNumRacks() throws Exc clientConf.setEnforceMinNumRacksPerWriteQuorum(true); repp = new RackawareEnsemblePlacementPolicy(); repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); int writeQuorumSize = 3; @@ -852,19 +873,19 @@ public void testNewEnsembleWithSufficientRacksAndEnforceMinNumRacks() throws Exc int numOfRacks = 2 * effectiveMinNumRacksPerWriteQuorum - 1; int numOfBookiesPerRack = 20; - BookieSocketAddress[] bookieSocketAddresses = new BookieSocketAddress[numOfRacks * numOfBookiesPerRack]; + BookieId[] bookieSocketAddresses = new BookieId[numOfRacks * numOfBookiesPerRack]; for (int i = 0; i < numOfRacks; i++) { for (int j = 0; j < numOfBookiesPerRack; j++) { int index = i * numOfBookiesPerRack + j; - bookieSocketAddresses[index] = new BookieSocketAddress("128.0.0." + index, 3181); - StaticDNSResolver.addNodeToRack(bookieSocketAddresses[index].getHostName(), "/default-region/r" + i); + bookieSocketAddresses[index] = new BookieSocketAddress("128.0.0." + index, 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("128.0.0." + index, "/default-region/r" + i); } } - Set addrs = new HashSet(); - repp.onClusterChanged(new HashSet(Arrays.asList(bookieSocketAddresses)), - new HashSet()); + Set addrs = new HashSet(); + repp.onClusterChanged(new HashSet(Arrays.asList(bookieSocketAddresses)), + new HashSet()); /* * in this scenario we have enough number of racks (2 * @@ -873,15 +894,16 @@ public void testNewEnsembleWithSufficientRacksAndEnforceMinNumRacks() throws Exc * ensembleSizes (as long as there are enough number of bookies in each * rack). */ - EnsemblePlacementPolicy.PlacementResult> ensembleResponse; - List ensemble; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; + List ensemble; PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; for (int ensembleSize = effectiveMinNumRacksPerWriteQuorum; ensembleSize < 40; ensembleSize++) { ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); ensemble = ensembleResponse.getResult(); isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, - getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, new HashSet<>(), @@ -889,7 +911,8 @@ public void testNewEnsembleWithSufficientRacksAndEnforceMinNumRacks() throws Exc ensemble = ensembleResponse.getResult(); isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, - getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } } @@ -909,23 +932,23 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); repp = new RackawareEnsemblePlacementPolicy(); repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, - statsLogger); + statsLogger, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(defaultRackForThisTest); Gauge numBookiesInDefaultRackGauge = statsLogger .getGauge(BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK); int numOfRacks = 3; int numOfBookiesPerRack = 5; - Set bookieSocketAddresses = new HashSet(); - Map bookieRackMap = new HashMap(); - BookieSocketAddress bookieAddress; + Set bookieSocketAddresses = new HashSet(); + Map bookieRackMap = new HashMap(); + BookieId bookieAddress; String rack; for (int i = 0; i < numOfRacks; i++) { for (int j = 0; j < numOfBookiesPerRack; j++) { int index = i * numOfBookiesPerRack + j; - bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181); + bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181).toBookieId(); rack = "/default-region/r" + i; - StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), rack); + StaticDNSResolver.addNodeToRack("128.0.0." + index, rack); bookieSocketAddresses.add(bookieAddress); bookieRackMap.put(bookieAddress, rack); } @@ -935,18 +958,18 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * response. */ int numOfBookiesInDefaultRack = 5; - BookieSocketAddress[] bookieSocketAddressesInDefaultRack = new BookieSocketAddress[numOfBookiesInDefaultRack]; + BookieId[] bookieSocketAddressesInDefaultRack = new BookieId[numOfBookiesInDefaultRack]; for (int i = 0; i < numOfBookiesInDefaultRack; i++) { - bookieSocketAddressesInDefaultRack[i] = new BookieSocketAddress("127.0.0." + (i + 100), 3181); - StaticDNSResolver.addNodeToRack(bookieSocketAddressesInDefaultRack[i].getHostName(), + bookieSocketAddressesInDefaultRack[i] = new BookieSocketAddress("127.0.0." + (i + 100), 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("127.0.0." + (i + 100), defaultRackForThisTest); } - Set nonDefaultRackBookiesList = bookieSocketAddresses; - List defaultRackBookiesList = Arrays.asList(bookieSocketAddressesInDefaultRack); - Set writableBookies = new HashSet(nonDefaultRackBookiesList); + Set nonDefaultRackBookiesList = bookieSocketAddresses; + List defaultRackBookiesList = Arrays.asList(bookieSocketAddressesInDefaultRack); + Set writableBookies = new HashSet(nonDefaultRackBookiesList); writableBookies.addAll(defaultRackBookiesList); - repp.onClusterChanged(writableBookies, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, numBookiesInDefaultRackGauge.getSample()); @@ -955,8 +978,8 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * and there are enough bookies in 3 racks, this newEnsemble call should * succeed. */ - EnsemblePlacementPolicy.PlacementResult> ensembleResponse; - List ensemble; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; + List ensemble; int ensembleSize = numOfRacks * numOfBookiesPerRack; int writeQuorumSize = numOfRacks; int ackQuorumSize = numOfRacks; @@ -964,7 +987,7 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); ensemble = ensembleResponse.getResult(); - BookieSocketAddress bookieInEnsembleToBeReplaced = ensemble.get(7); + BookieId bookieInEnsembleToBeReplaced = ensemble.get(7); // get rack of some other bookie String rackOfOtherBookieInEnsemble = bookieRackMap.get(ensemble.get(8)); BookieSocketAddress newBookieAddress1 = new BookieSocketAddress("128.0.0.100", 3181); @@ -973,11 +996,11 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * ensemble */ StaticDNSResolver.addNodeToRack(newBookieAddress1.getHostName(), rackOfOtherBookieInEnsemble); - bookieSocketAddresses.add(newBookieAddress1); - writableBookies.add(newBookieAddress1); - bookieRackMap.put(newBookieAddress1, rackOfOtherBookieInEnsemble); + bookieSocketAddresses.add(newBookieAddress1.toBookieId()); + writableBookies.add(newBookieAddress1.toBookieId()); + bookieRackMap.put(newBookieAddress1.toBookieId(), rackOfOtherBookieInEnsemble); - repp.onClusterChanged(writableBookies, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, numBookiesInDefaultRackGauge.getSample()); try { @@ -996,30 +1019,30 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * add the newBookie to a new rack. */ StaticDNSResolver.addNodeToRack(newBookieAddress2.getHostName(), newRack); - bookieSocketAddresses.add(newBookieAddress2); - writableBookies.add(newBookieAddress2); - bookieRackMap.put(newBookieAddress2, newRack); + bookieSocketAddresses.add(newBookieAddress2.toBookieId()); + writableBookies.add(newBookieAddress2.toBookieId()); + bookieRackMap.put(newBookieAddress2.toBookieId(), newRack); - repp.onClusterChanged(writableBookies, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, numBookiesInDefaultRackGauge.getSample()); /* * this replaceBookie should succeed, because a new bookie is added to a * new rack. */ - EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; - BookieSocketAddress replacedBookieAddress; + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; + BookieId replacedBookieAddress; PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; replaceBookieResponse = repp.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, null, ensemble, bookieInEnsembleToBeReplaced, new HashSet<>()); replacedBookieAddress = replaceBookieResponse.getResult(); isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); - assertEquals("It should be newBookieAddress2", newBookieAddress2, replacedBookieAddress); + assertEquals("It should be newBookieAddress2", newBookieAddress2.toBookieId(), replacedBookieAddress); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); - Set bookiesToExclude = new HashSet<>(); - bookiesToExclude.add(newBookieAddress2); - repp.onClusterChanged(writableBookies, new HashSet()); + Set bookiesToExclude = new HashSet<>(); + bookiesToExclude.add(newBookieAddress2.toBookieId()); + repp.onClusterChanged(writableBookies, new HashSet()); assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, numBookiesInDefaultRackGauge.getSample()); try { @@ -1038,11 +1061,11 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { * add the newBookie to rack of the bookie to be replaced. */ StaticDNSResolver.addNodeToRack(newBookieAddress3.getHostName(), rackOfBookieToBeReplaced); - bookieSocketAddresses.add(newBookieAddress3); - writableBookies.add(newBookieAddress3); - bookieRackMap.put(newBookieAddress3, rackOfBookieToBeReplaced); + bookieSocketAddresses.add(newBookieAddress3.toBookieId()); + writableBookies.add(newBookieAddress3.toBookieId()); + bookieRackMap.put(newBookieAddress3.toBookieId(), rackOfBookieToBeReplaced); - repp.onClusterChanged(writableBookies, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, numBookiesInDefaultRackGauge.getSample()); /* @@ -1054,7 +1077,7 @@ public void testReplaceBookieWithEnforceMinNumRacks() throws Exception { ensemble, bookieInEnsembleToBeReplaced, bookiesToExclude); replacedBookieAddress = replaceBookieResponse.getResult(); isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); - assertEquals("It should be newBookieAddress3", newBookieAddress3, replacedBookieAddress); + assertEquals("It should be newBookieAddress3", newBookieAddress3.toBookieId(), replacedBookieAddress); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @@ -1069,30 +1092,30 @@ public void testSelectBookieFromNetworkLoc() throws Exception { clientConf.setEnforceMinNumRacksPerWriteQuorum(true); repp = new RackawareEnsemblePlacementPolicy(); repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); int numOfRacks = 3; int numOfBookiesPerRack = 5; String[] rackLocationNames = new String[numOfRacks]; - List bookieSocketAddresses = new ArrayList(); - Map bookieRackMap = new HashMap(); - BookieSocketAddress bookieAddress; + List bookieSocketAddresses = new ArrayList(); + Map bookieRackMap = new HashMap(); + BookieId bookieAddress; for (int i = 0; i < numOfRacks; i++) { rackLocationNames[i] = "/default-region/r" + i; for (int j = 0; j < numOfBookiesPerRack; j++) { int index = i * numOfBookiesPerRack + j; - bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181); - StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), rackLocationNames[i]); + bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("128.0.0." + index, rackLocationNames[i]); bookieSocketAddresses.add(bookieAddress); bookieRackMap.put(bookieAddress, rackLocationNames[i]); } } String nonExistingRackLocation = "/default-region/r25"; - repp.onClusterChanged(new HashSet(bookieSocketAddresses), - new HashSet()); + repp.onClusterChanged(new HashSet(bookieSocketAddresses), + new HashSet()); String rack = bookieRackMap.get(bookieSocketAddresses.get(0)); BookieNode bookieNode = repp.selectFromNetworkLocation(rack, new HashSet(), TruePredicate.INSTANCE, @@ -1113,7 +1136,7 @@ public void testSelectBookieFromNetworkLoc() throws Exception { repp.selectFromNetworkLocation(nonExistingRackLocation, new HashSet(), TruePredicate.INSTANCE, EnsembleForReplacementWithNoConstraints.INSTANCE, true); - Set excludeBookiesOfRackR0 = new HashSet(); + Set excludeBookiesOfRackR0 = new HashSet(); for (int i = 0; i < numOfBookiesPerRack; i++) { excludeBookiesOfRackR0.add(bookieSocketAddresses.get(i)); } @@ -1147,31 +1170,31 @@ public void testSelectBookieFromExcludingRacks() throws Exception { clientConf.setEnforceMinNumRacksPerWriteQuorum(true); repp = new RackawareEnsemblePlacementPolicy(); repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); int numOfRacks = 3; int numOfBookiesPerRack = 5; String[] rackLocationNames = new String[numOfRacks]; - List bookieSocketAddresses = new ArrayList(); - Map bookieRackMap = new HashMap(); - BookieSocketAddress bookieAddress; + List bookieSocketAddresses = new ArrayList(); + Map bookieRackMap = new HashMap(); + BookieId bookieAddress; for (int i = 0; i < numOfRacks; i++) { rackLocationNames[i] = "/default-region/r" + i; for (int j = 0; j < numOfBookiesPerRack; j++) { int index = i * numOfBookiesPerRack + j; - bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181); - StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), rackLocationNames[i]); + bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("128.0.0." + index, rackLocationNames[i]); bookieSocketAddresses.add(bookieAddress); bookieRackMap.put(bookieAddress, rackLocationNames[i]); } } - repp.onClusterChanged(new HashSet(bookieSocketAddresses), - new HashSet()); + repp.onClusterChanged(new HashSet(bookieSocketAddresses), + new HashSet()); - Set excludeBookiesOfRackR0 = new HashSet(); + Set excludeBookiesOfRackR0 = new HashSet(); for (int i = 0; i < numOfBookiesPerRack; i++) { excludeBookiesOfRackR0.add(bookieSocketAddresses.get(i)); } @@ -1215,32 +1238,32 @@ public void testSelectBookieFromNetworkLocAndExcludingRacks() throws Exception { clientConf.setEnforceMinNumRacksPerWriteQuorum(true); repp = new RackawareEnsemblePlacementPolicy(); repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); int numOfRacks = 3; int numOfBookiesPerRack = 5; String[] rackLocationNames = new String[numOfRacks]; - List bookieSocketAddresses = new ArrayList(); - Map bookieRackMap = new HashMap(); - BookieSocketAddress bookieAddress; + List bookieSocketAddresses = new ArrayList(); + Map bookieRackMap = new HashMap(); + BookieId bookieAddress; for (int i = 0; i < numOfRacks; i++) { rackLocationNames[i] = "/default-region/r" + i; for (int j = 0; j < numOfBookiesPerRack; j++) { int index = i * numOfBookiesPerRack + j; - bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181); - StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), rackLocationNames[i]); + bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("128.0.0." + index, rackLocationNames[i]); bookieSocketAddresses.add(bookieAddress); bookieRackMap.put(bookieAddress, rackLocationNames[i]); } } String nonExistingRackLocation = "/default-region/r25"; - repp.onClusterChanged(new HashSet(bookieSocketAddresses), - new HashSet()); + repp.onClusterChanged(new HashSet(bookieSocketAddresses), + new HashSet()); - Set excludeBookiesOfRackR0 = new HashSet(); + Set excludeBookiesOfRackR0 = new HashSet(); for (int i = 0; i < numOfBookiesPerRack; i++) { excludeBookiesOfRackR0.add(bookieSocketAddresses.get(i)); } @@ -1299,31 +1322,31 @@ public void testSelectBookieByExcludingRacksAndBookies() throws Exception { */ repp = new RackawareEnsemblePlacementPolicy(true); repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); int numOfRacks = 3; int numOfBookiesPerRack = 5; String[] rackLocationNames = new String[numOfRacks]; - List bookieSocketAddresses = new ArrayList(); - Map bookieRackMap = new HashMap(); - BookieSocketAddress bookieAddress; + List bookieSocketAddresses = new ArrayList(); + Map bookieRackMap = new HashMap(); + BookieId bookieAddress; for (int i = 0; i < numOfRacks; i++) { rackLocationNames[i] = "/default-region/r" + i; for (int j = 0; j < numOfBookiesPerRack; j++) { int index = i * numOfBookiesPerRack + j; - bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181); - StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), rackLocationNames[i]); + bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("128.0.0." + index, rackLocationNames[i]); bookieSocketAddresses.add(bookieAddress); bookieRackMap.put(bookieAddress, rackLocationNames[i]); } } - repp.onClusterChanged(new HashSet(bookieSocketAddresses), - new HashSet()); + repp.onClusterChanged(new HashSet(bookieSocketAddresses), + new HashSet()); - Set excludeBookiesOfRackR0 = new HashSet(); + Set excludeBookiesOfRackR0 = new HashSet(); for (int i = 0; i < numOfBookiesPerRack; i++) { excludeBookiesOfRackR0.add(bookieSocketAddresses.get(i)); } @@ -1364,7 +1387,7 @@ public boolean addNode(BookieNode node) { } @Override - public List toList() { + public List toList() { return null; } @@ -1392,31 +1415,33 @@ public void testNewEnsembleWithMultipleRacks() throws Exception { StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r2"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { int ensembleSize = 3; int writeQuorumSize = 2; int acqQuorumSize = 2; - EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, null, new HashSet<>()); - List ensemble = ensembleResponse.getResult(); + List ensemble = ensembleResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); - int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); + int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, + conf.getMinNumRacksPerWriteQuorum(), repp.bookieAddressResolver); assertTrue(numCovered >= 1 && numCovered < 3); assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); ensembleSize = 4; - EnsemblePlacementPolicy.PlacementResult> ensembleResponse2 = + EnsemblePlacementPolicy.PlacementResult> ensembleResponse2 = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, null, new HashSet<>()); - List ensemble2 = ensembleResponse2.getResult(); + List ensemble2 = ensembleResponse2.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isAdheringToPolicy(); - numCovered = getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()); + numCovered = getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, + conf.getMinNumRacksPerWriteQuorum(), repp.bookieAddressResolver); assertTrue(numCovered >= 1 && numCovered < 3); assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy2); } catch (BKNotEnoughBookiesException bnebe) { @@ -1430,13 +1455,13 @@ public void testMinNumRacksPerWriteQuorumOfRacks() throws Exception { int numOfNodesInEachRack = 5; // Update cluster - Set addrs = new HashSet(); - BookieSocketAddress addr; + Set addrs = new HashSet(); + BookieId addr; for (int i = 0; i < numOfRacksToCreate; i++) { for (int j = 0; j < numOfNodesInEachRack; j++) { - addr = new BookieSocketAddress("128.0.0." + ((i * numOfNodesInEachRack) + j), 3181); + addr = new BookieSocketAddress("128.0.0." + ((i * numOfNodesInEachRack) + j), 3181).toBookieId(); // update dns mapping - StaticDNSResolver.addNodeToRack(addr.getHostName(), "/default-region/r" + i); + StaticDNSResolver.addNodeToRack("128.0.0." + ((i * numOfNodesInEachRack) + j), "/default-region/r" + i); addrs.add(addr); } } @@ -1479,20 +1504,22 @@ public void testMinNumRacksPerWriteQuorumOfRacks() throws Exception { } } - void validateNumOfWriteQuorumsCoveredInEnsembleCreation(Set addrs, + void validateNumOfWriteQuorumsCoveredInEnsembleCreation(Set addrs, int minNumRacksPerWriteQuorum, int ensembleSize, int writeQuorumSize) throws Exception { ClientConfiguration newConf = new ClientConfiguration(conf); newConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorum); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(newConf, Optional. empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); - repp.onClusterChanged(addrs, new HashSet()); - EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + repp.onClusterChanged(addrs, new HashSet()); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, writeQuorumSize, null, new HashSet<>()); - List ensemble = ensembleResponse.getResult(); + List ensemble = ensembleResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); - int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, minNumRacksPerWriteQuorum); + int numCovered = getNumCoveredWriteQuorums(ensemble, writeQuorumSize, + minNumRacksPerWriteQuorum, repp.bookieAddressResolver); assertEquals("minimum number of racks covered for writequorum ensemble: " + ensemble, ensembleSize, numCovered); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); } @@ -1518,36 +1545,38 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/default-region/r4"); int availableNumOfRacks = 4; // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { int ensembleSize = 3; int writeQuorumSize = 3; int ackQuorumSize = 2; - EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); - List ensemble1 = ensembleResponse.getResult(); + List ensemble1 = ensembleResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy1 = ensembleResponse.isAdheringToPolicy(); assertEquals(ensembleSize, - getNumCoveredWriteQuorums(ensemble1, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); + getNumCoveredWriteQuorums(ensemble1, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy1); ensembleSize = 4; writeQuorumSize = 4; - EnsemblePlacementPolicy.PlacementResult> ensembleResponse2 = + EnsemblePlacementPolicy.PlacementResult> ensembleResponse2 = repp.newEnsemble(ensembleSize, writeQuorumSize, 2, null, new HashSet<>()); - List ensemble2 = ensembleResponse2.getResult(); + List ensemble2 = ensembleResponse2.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy2 = ensembleResponse2.isAdheringToPolicy(); assertEquals(ensembleSize, - getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum())); + getNumCoveredWriteQuorums(ensemble2, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy2); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -1569,14 +1598,14 @@ public void testRemoveBookieFromCluster() { StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r3"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); } @Test @@ -1595,43 +1624,47 @@ public void testWeightedPlacementAndReplaceBookieWithEnoughBookiesInSameRack() t StaticDNSResolver.addNodeToRack(addr4.getSocketAddress().getAddress().getHostAddress(), NetworkTopology.DEFAULT_REGION + "/r2"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); int multiple = 10; conf.setDiskWeightBasedPlacementEnabled(true); conf.setBookieMaxWeightMultipleForWeightBasedPlacement(-1); // no max cap on weight - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); - repp.onClusterChanged(addrs, new HashSet()); - Map bookieInfoMap = new HashMap(); - bookieInfoMap.put(addr1, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr2, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr3, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr4, new BookieInfo(multiple * 100L, multiple * 100L)); + repp.onClusterChanged(addrs, new HashSet()); + Map bookieInfoMap = new HashMap(); + bookieInfoMap.put(addr1.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr2.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr3.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr4.toBookieId(), new BookieInfo(multiple * 100L, multiple * 100L)); repp.updateBookieInfo(bookieInfoMap); - Map selectionCounts = new HashMap(); - selectionCounts.put(addr3, 0L); - selectionCounts.put(addr4, 0L); + Map selectionCounts = new HashMap(); + selectionCounts.put(addr3.toBookieId(), 0L); + selectionCounts.put(addr4.toBookieId(), 0L); int numTries = 50000; - EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; - BookieSocketAddress replacedBookie; + BookieId replacedBookie; for (int i = 0; i < numTries; i++) { // replace node under r2 - replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); + replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2.toBookieId(), new HashSet<>()); replacedBookie = replaceBookieResponse.getResult(); isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); - assertTrue("replaced : " + replacedBookie, addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); + assertTrue("replaced : " + replacedBookie, addr3.toBookieId().equals(replacedBookie) + || addr4.toBookieId().equals(replacedBookie)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); selectionCounts.put(replacedBookie, selectionCounts.get(replacedBookie) + 1); } - double observedMultiple = ((double) selectionCounts.get(addr4) / (double) selectionCounts.get(addr3)); + double observedMultiple = ((double) selectionCounts.get(addr4.toBookieId()) + / (double) selectionCounts.get(addr3.toBookieId())); assertTrue("Weights not being honored " + observedMultiple, Math.abs(observedMultiple - multiple) < 1); } @@ -1655,47 +1688,51 @@ public void testWeightedPlacementAndReplaceBookieWithoutEnoughBookiesInSameRack( StaticDNSResolver.addNodeToRack(addr4.getSocketAddress().getAddress().getHostAddress(), NetworkTopology.DEFAULT_REGION + "/r4"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr0); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); + Set addrs = new HashSet(); + addrs.add(addr0.toBookieId()); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); int multiple = 10, maxMultiple = 4; conf.setDiskWeightBasedPlacementEnabled(true); conf.setBookieMaxWeightMultipleForWeightBasedPlacement(maxMultiple); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); - repp.onClusterChanged(addrs, new HashSet()); - Map bookieInfoMap = new HashMap(); - bookieInfoMap.put(addr0, new BookieInfo(50L, 50L)); - bookieInfoMap.put(addr1, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr2, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr3, new BookieInfo(200L, 200L)); - bookieInfoMap.put(addr4, new BookieInfo(multiple * 50L, multiple * 50L)); + repp.onClusterChanged(addrs, new HashSet()); + Map bookieInfoMap = new HashMap(); + bookieInfoMap.put(addr0.toBookieId(), new BookieInfo(50L, 50L)); + bookieInfoMap.put(addr1.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr2.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr3.toBookieId(), new BookieInfo(200L, 200L)); + bookieInfoMap.put(addr4.toBookieId(), new BookieInfo(multiple * 50L, multiple * 50L)); repp.updateBookieInfo(bookieInfoMap); - Map selectionCounts = new HashMap(); - selectionCounts.put(addr0, 0L); - selectionCounts.put(addr1, 0L); - selectionCounts.put(addr2, 0L); - selectionCounts.put(addr3, 0L); - selectionCounts.put(addr4, 0L); + Map selectionCounts = new HashMap(); + selectionCounts.put(addr0.toBookieId(), 0L); + selectionCounts.put(addr1.toBookieId(), 0L); + selectionCounts.put(addr2.toBookieId(), 0L); + selectionCounts.put(addr3.toBookieId(), 0L); + selectionCounts.put(addr4.toBookieId(), 0L); int numTries = 50000; - EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; - BookieSocketAddress replacedBookie; + EnsemblePlacementPolicy.PlacementResult replaceBookieResponse; + BookieId replacedBookie; PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; for (int i = 0; i < numTries; i++) { // addr2 is on /r2 and this is the only one on this rack. So the replacement // will come from other racks. However, the weight should be honored in such // selections as well - replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), addr2, new HashSet<>()); + replaceBookieResponse = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2.toBookieId(), new HashSet<>()); replacedBookie = replaceBookieResponse.getResult(); isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.isAdheringToPolicy(); - assertTrue(addr0.equals(replacedBookie) || addr1.equals(replacedBookie) || addr3.equals(replacedBookie) - || addr4.equals(replacedBookie)); + assertTrue(addr0.toBookieId().equals(replacedBookie) + || addr1.toBookieId().equals(replacedBookie) + || addr3.toBookieId().equals(replacedBookie) + || addr4.toBookieId().equals(replacedBookie)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); selectionCounts.put(replacedBookie, selectionCounts.get(replacedBookie) + 1); } @@ -1704,15 +1741,17 @@ public void testWeightedPlacementAndReplaceBookieWithoutEnoughBookiesInSameRack( * So the median calculated by WeightedRandomSelection is (100 + 200) / 2 = 150 */ double medianWeight = 150; - double medianSelectionCounts = (double) (medianWeight / bookieInfoMap.get(addr1).getWeight()) - * selectionCounts.get(addr1); - double observedMultiple1 = ((double) selectionCounts.get(addr4) / (double) medianSelectionCounts); - double observedMultiple2 = ((double) selectionCounts.get(addr4) / (double) selectionCounts.get(addr3)); + double medianSelectionCounts = (double) (medianWeight / bookieInfoMap.get(addr1.toBookieId()).getWeight()) + * selectionCounts.get(addr1.toBookieId()); + double observedMultiple1 = ((double) selectionCounts.get(addr4.toBookieId()) + / (double) medianSelectionCounts); + double observedMultiple2 = ((double) selectionCounts.get(addr4.toBookieId()) + / (double) selectionCounts.get(addr3.toBookieId())); LOG.info("oM1 " + observedMultiple1 + " oM2 " + observedMultiple2); assertTrue("Weights not being honored expected " + maxMultiple + " observed " + observedMultiple1, Math.abs(observedMultiple1 - maxMultiple) < 1); // expected multiple for addr3 - double expected = (medianWeight * maxMultiple) / bookieInfoMap.get(addr3).getWeight(); + double expected = (medianWeight * maxMultiple) / bookieInfoMap.get(addr3.toBookieId()).getWeight(); assertTrue("Weights not being honored expected " + expected + " observed " + observedMultiple2, Math.abs(observedMultiple2 - expected) < 1); } @@ -1750,46 +1789,47 @@ public void testWeightedPlacementAndNewEnsembleWithEnoughBookiesInSameRack() thr NetworkTopology.DEFAULT_REGION + "/r3"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - addrs.add(addr9); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + addrs.add(addr9.toBookieId()); int maxMultiple = 4; conf.setDiskWeightBasedPlacementEnabled(true); conf.setBookieMaxWeightMultipleForWeightBasedPlacement(maxMultiple); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); - repp.onClusterChanged(addrs, new HashSet()); - Map bookieInfoMap = new HashMap(); - bookieInfoMap.put(addr1, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr2, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr3, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr4, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr5, new BookieInfo(1000L, 1000L)); - bookieInfoMap.put(addr6, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr7, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr8, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr9, new BookieInfo(1000L, 1000L)); + repp.onClusterChanged(addrs, new HashSet()); + Map bookieInfoMap = new HashMap(); + bookieInfoMap.put(addr1.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr2.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr3.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr4.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr5.toBookieId(), new BookieInfo(1000L, 1000L)); + bookieInfoMap.put(addr6.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr7.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr8.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr9.toBookieId(), new BookieInfo(1000L, 1000L)); repp.updateBookieInfo(bookieInfoMap); - Map selectionCounts = new HashMap(); - for (BookieSocketAddress b : addrs) { + Map selectionCounts = new HashMap(); + for (BookieId b : addrs) { selectionCounts.put(b, 0L); } int numTries = 10000; - Set excludeList = new HashSet(); - EnsemblePlacementPolicy.PlacementResult> ensembleResponse; - List ensemble; + Set excludeList = new HashSet(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; + List ensemble; int ensembleSize = 3; int writeQuorumSize = 2; int acqQuorumSize = 2; @@ -1801,17 +1841,21 @@ public void testWeightedPlacementAndNewEnsembleWithEnoughBookiesInSameRack() thr ensemble = ensembleResponse.getResult(); assertTrue( "Rackaware selection not happening " - + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()), - getNumCoveredWriteQuorums(ensemble, writeQuorumSize, conf.getMinNumRacksPerWriteQuorum()) >= 2); - for (BookieSocketAddress b : ensemble) { + + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, + conf.getMinNumRacksPerWriteQuorum(), repp.bookieAddressResolver), + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, + conf.getMinNumRacksPerWriteQuorum(), repp.bookieAddressResolver) >= 2); + for (BookieId b : ensemble) { selectionCounts.put(b, selectionCounts.get(b) + 1); } } // the median weight used is 100 since addr2 and addr6 have the same weight, we use their // selection counts as the same as median - double observedMultiple1 = ((double) selectionCounts.get(addr5) / (double) selectionCounts.get(addr2)); - double observedMultiple2 = ((double) selectionCounts.get(addr9) / (double) selectionCounts.get(addr6)); + double observedMultiple1 = ((double) selectionCounts.get(addr5.toBookieId()) + / (double) selectionCounts.get(addr2.toBookieId())); + double observedMultiple2 = ((double) selectionCounts.get(addr9.toBookieId()) + / (double) selectionCounts.get(addr6.toBookieId())); assertTrue("Weights not being honored expected 2 observed " + observedMultiple1, Math.abs(observedMultiple1 - maxMultiple) < 0.5); assertTrue("Weights not being honored expected 4 observed " + observedMultiple2, @@ -1838,36 +1882,37 @@ public void testWeightedPlacementAndNewEnsembleWithoutEnoughBookies() throws Exc StaticDNSResolver.addNodeToRack(addr5.getSocketAddress().getAddress().getHostAddress(), NetworkTopology.DEFAULT_REGION + "/r3"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); int maxMultiple = 4; conf.setDiskWeightBasedPlacementEnabled(true); conf.setBookieMaxWeightMultipleForWeightBasedPlacement(maxMultiple); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); - repp.onClusterChanged(addrs, new HashSet()); - Map bookieInfoMap = new HashMap(); - bookieInfoMap.put(addr1, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr2, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr3, new BookieInfo(1000L, 1000L)); - bookieInfoMap.put(addr4, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr5, new BookieInfo(1000L, 1000L)); + repp.onClusterChanged(addrs, new HashSet()); + Map bookieInfoMap = new HashMap(); + bookieInfoMap.put(addr1.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr2.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr3.toBookieId(), new BookieInfo(1000L, 1000L)); + bookieInfoMap.put(addr4.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr5.toBookieId(), new BookieInfo(1000L, 1000L)); repp.updateBookieInfo(bookieInfoMap); - EnsemblePlacementPolicy.PlacementResult> ensembleResponse; - List ensemble; - Set excludeList = new HashSet(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; + List ensemble; + Set excludeList = new HashSet(); try { - excludeList.add(addr1); - excludeList.add(addr2); - excludeList.add(addr3); - excludeList.add(addr4); + excludeList.add(addr1.toBookieId()); + excludeList.add(addr2.toBookieId()); + excludeList.add(addr3.toBookieId()); + excludeList.add(addr4.toBookieId()); ensembleResponse = repp.newEnsemble(3, 2, 2, null, excludeList); ensemble = ensembleResponse.getResult(); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies" + ensemble); @@ -1882,16 +1927,16 @@ public void testWeightedPlacementAndNewEnsembleWithoutEnoughBookies() throws Exc } } - static int getNumCoveredWriteQuorums(List ensemble, int writeQuorumSize, - int minNumRacksPerWriteQuorumConfValue) throws Exception { + static int getNumCoveredWriteQuorums(List ensemble, int writeQuorumSize, + int minNumRacksPerWriteQuorumConfValue, BookieAddressResolver bookieAddressResolver) throws Exception { int ensembleSize = ensemble.size(); int numCoveredWriteQuorums = 0; for (int i = 0; i < ensembleSize; i++) { Set racks = new HashSet(); for (int j = 0; j < writeQuorumSize; j++) { int bookieIdx = (i + j) % ensembleSize; - BookieSocketAddress addr = ensemble.get(bookieIdx); - racks.add(StaticDNSResolver.getRack(addr.getHostName())); + BookieId addr = ensemble.get(bookieIdx); + racks.add(StaticDNSResolver.getRack(bookieAddressResolver.resolve(addr).getHostName())); } int numOfRacksToCoverTo = Math.max(Math.min(writeQuorumSize, minNumRacksPerWriteQuorumConfValue), 2); numCoveredWriteQuorums += (racks.size() >= numOfRacksToCoverTo ? 1 : 0); @@ -1905,34 +1950,35 @@ public void testNodeWithFailures() throws Exception { updateMyRack(NetworkTopology.DEFAULT_REGION_AND_RACK); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); - HashMap bookieFailures = new HashMap(); + HashMap bookieFailures = new HashMap(); - bookieFailures.put(addr1, 20L); - bookieFailures.put(addr2, 22L); + bookieFailures.put(addr1.toBookieId(), 20L); + bookieFailures.put(addr2.toBookieId(), 22L); // remove failure bookies: addr1 and addr2 - addrs = new HashSet(); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + addrs = new HashSet(); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); DistributionSchedule.WriteSet reoderSet = repp.reorderReadSequence( ensemble, getBookiesHealthInfo(bookieFailures, new HashMap<>()), writeSet); LOG.info("reorder set : {}", reoderSet); - assertEquals(ensemble.get(reoderSet.get(2)), addr1); - assertEquals(ensemble.get(reoderSet.get(3)), addr2); - assertEquals(ensemble.get(reoderSet.get(0)), addr3); - assertEquals(ensemble.get(reoderSet.get(1)), addr4); + assertEquals(ensemble.get(reoderSet.get(2)), addr1.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(3)), addr2.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(0)), addr3.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(1)), addr4.toBookieId()); } @Test @@ -1944,37 +1990,38 @@ public void testPlacementOnStabilizeNetworkTopology() throws Exception { ClientConfiguration confLocal = new ClientConfiguration(); confLocal.addConfiguration(conf); confLocal.setNetworkTopologyStabilizePeriodSeconds(99999); - repp.initialize(confLocal, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(confLocal, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // addr4 left - addrs.remove(addr4); - Set deadBookies = repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr4.toBookieId()); + Set deadBookies = repp.onClusterChanged(addrs, new HashSet()); assertTrue(deadBookies.isEmpty()); // we will never use addr4 even it is in the stabilized network topology for (int i = 0; i < 5; i++) { - EnsemblePlacementPolicy.PlacementResult> ensembleResponse = - repp.newEnsemble(3, 2, 2, null, new HashSet()); - List ensemble = ensembleResponse.getResult(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + repp.newEnsemble(3, 2, 2, null, new HashSet()); + List ensemble = ensembleResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); - assertFalse(ensemble.contains(addr4)); + assertFalse(ensemble.contains(addr4.toBookieId())); assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); } // we could still use addr4 for urgent allocation if it is just bookie flapping - EnsemblePlacementPolicy.PlacementResult> ensembleResponse = - repp.newEnsemble(4, 2, 2, null, new HashSet()); - List ensemble = ensembleResponse.getResult(); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = + repp.newEnsemble(4, 2, 2, null, new HashSet()); + List ensemble = ensembleResponse.getResult(); PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals(PlacementPolicyAdherence.FAIL, isEnsembleAdheringToPlacementPolicy); - assertTrue(ensemble.contains(addr4)); + assertTrue(ensemble.contains(addr4.toBookieId())); } @Test @@ -2103,24 +2150,25 @@ public void testNumBookiesInDefaultRackGauge() throws Exception { TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional. empty(), timer, DISABLE_ALL, statsLogger); + repp.initialize(conf, Optional. empty(), timer, + DISABLE_ALL, statsLogger, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(defaultRackForThisTest); Gauge numBookiesInDefaultRackGauge = statsLogger .getGauge(BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK); - Set writeableBookies = new HashSet(); - writeableBookies.add(newAddr1); - writeableBookies.add(newAddr2); - Set readOnlyBookies = new HashSet(); - readOnlyBookies.add(newAddr3); - readOnlyBookies.add(newAddr4); + Set writeableBookies = new HashSet(); + writeableBookies.add(newAddr1.toBookieId()); + writeableBookies.add(newAddr2.toBookieId()); + Set readOnlyBookies = new HashSet(); + readOnlyBookies.add(newAddr3.toBookieId()); + readOnlyBookies.add(newAddr4.toBookieId()); repp.onClusterChanged(writeableBookies, readOnlyBookies); // only writable bookie - newAddr1 in default rack assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", 1, numBookiesInDefaultRackGauge.getSample()); - readOnlyBookies.remove(newAddr4); - writeableBookies.add(newAddr4); + readOnlyBookies.remove(newAddr4.toBookieId()); + writeableBookies.add(newAddr4.toBookieId()); repp.onClusterChanged(writeableBookies, readOnlyBookies); // newAddr4 is also added to writable bookie so 2 writable bookies - // newAddr1 and newAddr4 @@ -2139,10 +2187,10 @@ public void testNumBookiesInDefaultRackGauge() throws Exception { StaticDNSResolver .changeRack(Collections.singletonList(newAddr1), Collections.singletonList("/default-region/r2")); readOnlyBookies.clear(); - writeableBookies.add(newAddr1); - writeableBookies.add(newAddr2); - writeableBookies.add(newAddr3); - writeableBookies.add(newAddr4); + writeableBookies.add(newAddr1.toBookieId()); + writeableBookies.add(newAddr2.toBookieId()); + writeableBookies.add(newAddr3.toBookieId()); + writeableBookies.add(newAddr4.toBookieId()); repp.onClusterChanged(writeableBookies, readOnlyBookies); // newAddr1 rack is changed and it is not in default anymore. So no // bookies in default rack anymore @@ -2164,7 +2212,8 @@ public void testNewEnsembleExcludesDefaultRackBookiesEnforceMinNumRacks() throws TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(clientConf, Optional. empty(), timer, DISABLE_ALL, statsLogger); + repp.initialize(clientConf, Optional. empty(), timer, + DISABLE_ALL, statsLogger, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(defaultRackForThisTest); Gauge numBookiesInDefaultRackGauge = statsLogger .getGauge(BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK); @@ -2175,28 +2224,27 @@ public void testNewEnsembleExcludesDefaultRackBookiesEnforceMinNumRacks() throws int numOfRacks = 2 * effectiveMinNumRacksPerWriteQuorum - 1; int numOfBookiesPerRack = 20; - BookieSocketAddress[] bookieSocketAddresses = new BookieSocketAddress[numOfRacks * numOfBookiesPerRack]; + BookieId[] bookieSocketAddresses = new BookieId[numOfRacks * numOfBookiesPerRack]; for (int i = 0; i < numOfRacks; i++) { for (int j = 0; j < numOfBookiesPerRack; j++) { int index = i * numOfBookiesPerRack + j; - bookieSocketAddresses[index] = new BookieSocketAddress("128.0.0." + index, 3181); - StaticDNSResolver.addNodeToRack(bookieSocketAddresses[index].getHostName(), "/default-region/r" + i); + bookieSocketAddresses[index] = new BookieSocketAddress("128.0.0." + index, 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("128.0.0." + index, "/default-region/r" + i); } } int numOfBookiesInDefaultRack = 10; - BookieSocketAddress[] bookieSocketAddressesInDefaultRack = new BookieSocketAddress[numOfBookiesInDefaultRack]; + BookieId[] bookieSocketAddressesInDefaultRack = new BookieId[numOfBookiesInDefaultRack]; for (int i = 0; i < numOfBookiesInDefaultRack; i++) { - bookieSocketAddressesInDefaultRack[i] = new BookieSocketAddress("127.0.0." + (i + 100), 3181); - StaticDNSResolver.addNodeToRack(bookieSocketAddressesInDefaultRack[i].getHostName(), - defaultRackForThisTest); + bookieSocketAddressesInDefaultRack[i] = new BookieSocketAddress("127.0.0." + (i + 100), 3181).toBookieId(); + StaticDNSResolver.addNodeToRack("127.0.0." + (i + 100), defaultRackForThisTest); } - Set writableBookies = new HashSet( + Set writableBookies = new HashSet( Arrays.asList(bookieSocketAddresses)); writableBookies.addAll(Arrays.asList(bookieSocketAddressesInDefaultRack)); - repp.onClusterChanged(writableBookies, new HashSet()); + repp.onClusterChanged(writableBookies, new HashSet()); assertEquals("NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK guage value", numOfBookiesInDefaultRack, numBookiesInDefaultRackGauge.getSample()); @@ -2210,24 +2258,26 @@ public void testNewEnsembleExcludesDefaultRackBookiesEnforceMinNumRacks() throws * Since minNumRacksPerWriteQuorum is enforced, it shouldn't select node * from default rack. */ - EnsemblePlacementPolicy.PlacementResult> ensembleResponse; - List ensemble; + EnsemblePlacementPolicy.PlacementResult> ensembleResponse; + List ensemble; PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy; for (int ensembleSize = effectiveMinNumRacksPerWriteQuorum; ensembleSize < 40; ensembleSize++) { ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); ensemble = ensembleResponse.getResult(); isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, - getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, null, new HashSet<>()); ensemble = ensembleResponse.getResult(); isEnsembleAdheringToPlacementPolicy = ensembleResponse.isAdheringToPolicy(); assertEquals("Number of writeQuorum sets covered", ensembleSize, - getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum())); + getNumCoveredWriteQuorums(ensemble, writeQuorumSize, clientConf.getMinNumRacksPerWriteQuorum(), + repp.bookieAddressResolver)); assertEquals(PlacementPolicyAdherence.MEETS_STRICT, isEnsembleAdheringToPlacementPolicy); - Collection bookiesOfDefaultRackInEnsemble = CollectionUtils + Collection bookiesOfDefaultRackInEnsemble = CollectionUtils .intersection(Arrays.asList(bookieSocketAddressesInDefaultRack), ensemble); assertTrue("Ensemble is not supposed to contain bookies from default rack, but ensemble contains - " + bookiesOfDefaultRackInEnsemble, bookiesOfDefaultRackInEnsemble.isEmpty()); @@ -2252,27 +2302,26 @@ private void testAreAckedBookiesAdheringToPlacementPolicyHelper(int minNumRacksP TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, statsLogger); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, statsLogger, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(defaultRackForThisTest); - List bookieSocketAddressesDefaultRack = new ArrayList<>(); - List bookieSocketAddressesNonDefaultRack = new ArrayList<>(); - Set writableBookies; - Set bookiesForEntry = new HashSet<>(); + List bookieSocketAddressesDefaultRack = new ArrayList<>(); + List bookieSocketAddressesNonDefaultRack = new ArrayList<>(); + Set writableBookies; + Set bookiesForEntry = new HashSet<>(); for (int i = 0; i < numOfRacks; i++) { for (int j = 0; j < numOfBookiesPerRack; j++) { int index = i * numOfBookiesPerRack + j; - bookieSocketAddressesNonDefaultRack.add(new BookieSocketAddress("128.0.0." + index, 3181)); - StaticDNSResolver.addNodeToRack(bookieSocketAddressesNonDefaultRack.get(index).getHostName(), - "/default-region/r" + i); + bookieSocketAddressesNonDefaultRack.add(new BookieSocketAddress("128.0.0." + index, 3181).toBookieId()); + StaticDNSResolver.addNodeToRack("128.0.0." + index, "/default-region/r" + i); } } for (int i = 0; i < numOfBookiesInDefaultRack; i++) { - bookieSocketAddressesDefaultRack.add(new BookieSocketAddress("127.0.0." + (i + 100), 3181)); - StaticDNSResolver.addNodeToRack(bookieSocketAddressesDefaultRack.get(i).getHostName(), - defaultRackForThisTest); + bookieSocketAddressesDefaultRack.add(new BookieSocketAddress("127.0.0." + (i + 100), 3181).toBookieId()); + StaticDNSResolver.addNodeToRack("127.0.0." + (i + 100), defaultRackForThisTest); } writableBookies = new HashSet<>(bookieSocketAddressesNonDefaultRack); @@ -2282,7 +2331,7 @@ private void testAreAckedBookiesAdheringToPlacementPolicyHelper(int minNumRacksP // Case 1 : Bookies in the ensemble from the same rack. // Manually crafting the ensemble here to create the error case when the check should return false - List ensemble = new ArrayList<>(bookieSocketAddressesDefaultRack); + List ensemble = new ArrayList<>(bookieSocketAddressesDefaultRack); for (int entryId = 0; entryId < 10; entryId++) { DistributionSchedule ds = new RoundRobinDistributionSchedule(writeQuorumSize, ackQuorumSize, ensembleSize); DistributionSchedule.WriteSet ws = ds.getWriteSet(entryId); @@ -2296,7 +2345,7 @@ private void testAreAckedBookiesAdheringToPlacementPolicyHelper(int minNumRacksP // Case 2 : Bookies in the ensemble from the different racks - EnsemblePlacementPolicy.PlacementResult> + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = repp.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java index f90f45d0f91..a5a56b2c45e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java @@ -40,6 +40,7 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.CommonConfigurationKeys; import org.apache.bookkeeper.net.DNSToSwitchMapping; @@ -85,7 +86,8 @@ public void setUp() throws Exception { conf.getTimeoutTimerNumTicks()); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); } @After @@ -106,16 +108,16 @@ public void testReplaceBookieWithEnoughBookiesInSameRack() throws Exception { BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.4", 3181); // /4 rack // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, new HashSet<>()).getResult(); - assertEquals(addr3, replacedBookie); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2.toBookieId(), new HashSet<>()).getResult(); + assertEquals(addr3.toBookieId(), replacedBookie); } @Test @@ -127,20 +129,21 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRack() throws Exception BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.4", 3181); // /4 rack // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, excludedAddrs).getResult(); - - assertFalse(addr1.equals(replacedBookie)); - assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr1.toBookieId()); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2.toBookieId(), excludedAddrs).getResult(); + + assertFalse(addr1.toBookieId().equals(replacedBookie)); + assertTrue(addr3.toBookieId().equals(replacedBookie) + || addr4.toBookieId().equals(replacedBookie)); } @Test @@ -152,19 +155,19 @@ public void testReplaceBookieWithNotEnoughBookies() throws Exception { BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.4", 3181); // /4 rack // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr1); - excludedAddrs.add(addr3); - excludedAddrs.add(addr4); + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr1.toBookieId()); + excludedAddrs.add(addr3.toBookieId()); + excludedAddrs.add(addr4.toBookieId()); try { - repp.replaceBookie(1, 1, 1, null, new ArrayList(), addr2, excludedAddrs); + repp.replaceBookie(1, 1, 1, null, new ArrayList(), addr2.toBookieId(), excludedAddrs); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not BKNotEnoughBookiesException @@ -187,26 +190,26 @@ public void testReplaceBookieWithScriptMappingError() throws Exception { BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.2", 3181); // /2 rack // Update cluster, add node that maps to non-default rack - Set addrs = new HashSet(); - addrs.add(addr1); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - addrs = new HashSet(); - addrs.add(addr0); - addrs.add(addr1); - addrs.add(addr2); - repp.onClusterChanged(addrs, new HashSet()); + addrs = new HashSet(); + addrs.add(addr0.toBookieId()); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, excludedAddrs).getResult(); - - assertFalse(addr1.equals(replacedBookie)); - assertFalse(addr2.equals(replacedBookie)); - assertTrue(addr0.equals(replacedBookie)); + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr1.toBookieId()); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2.toBookieId(), excludedAddrs).getResult(); + + assertFalse(addr1.toBookieId().equals(replacedBookie)); + assertFalse(addr2.toBookieId().equals(replacedBookie)); + assertTrue(addr0.toBookieId().equals(replacedBookie)); } /* @@ -226,26 +229,26 @@ public void testReplaceBookieWithScriptMappingError2() throws Exception { BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.2", 3181); // /2 rack // Update cluster, add node that maps to default rack first - Set addrs = new HashSet(); - addrs.add(addr0); + Set addrs = new HashSet(); + addrs.add(addr0.toBookieId()); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - addrs = new HashSet(); - addrs.add(addr0); - addrs.add(addr1); - addrs.add(addr2); - repp.onClusterChanged(addrs, new HashSet()); + addrs = new HashSet(); + addrs.add(addr0.toBookieId()); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), - addr2, excludedAddrs).getResult(); - - assertFalse(addr1.equals(replacedBookie)); - assertFalse(addr2.equals(replacedBookie)); - assertTrue(addr0.equals(replacedBookie)); + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr1.toBookieId()); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, null, new ArrayList<>(), + addr2.toBookieId(), excludedAddrs).getResult(); + + assertFalse(addr1.toBookieId().equals(replacedBookie)); + assertFalse(addr2.toBookieId().equals(replacedBookie)); + assertTrue(addr0.toBookieId().equals(replacedBookie)); } @Test @@ -256,17 +259,17 @@ public void testNewEnsembleWithSingleRack() throws Exception { BookieSocketAddress addr3 = new BookieSocketAddress("127.0.2.1", 3181); // /1 rack BookieSocketAddress addr4 = new BookieSocketAddress("127.0.3.1", 3181); // /1 rack // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(3, 2, 2, null, + List ensemble = repp.newEnsemble(3, 2, 2, null, new HashSet<>()).getResult(); assertEquals(0, getNumCoveredWriteQuorums(ensemble, 2)); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, + List ensemble2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()).getResult(); assertEquals(0, getNumCoveredWriteQuorums(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { @@ -282,18 +285,18 @@ public void testNewEnsembleWithMultipleRacks() throws Exception { BookieSocketAddress addr3 = new BookieSocketAddress("127.0.1.2", 3181); // /2 rack BookieSocketAddress addr4 = new BookieSocketAddress("127.0.2.2", 3181); // /2 rack // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(3, 2, 2, null, + List ensemble = repp.newEnsemble(3, 2, 2, null, new HashSet<>()).getResult(); int numCovered = getNumCoveredWriteQuorums(ensemble, 2); assertTrue(numCovered == 2); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, + List ensemble2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()).getResult(); numCovered = getNumCoveredWriteQuorums(ensemble2, 2); assertTrue(numCovered == 2); @@ -314,21 +317,21 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { BookieSocketAddress addr7 = new BookieSocketAddress("127.0.1.3", 3181); // /3 rack BookieSocketAddress addr8 = new BookieSocketAddress("127.0.1.4", 3181); // /4 rack // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble1 = repp.newEnsemble(3, 2, 2, null, + List ensemble1 = repp.newEnsemble(3, 2, 2, null, new HashSet<>()).getResult(); assertEquals(3, getNumCoveredWriteQuorums(ensemble1, 2)); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, + List ensemble2 = repp.newEnsemble(4, 2, 2, null, new HashSet<>()).getResult(); assertEquals(4, getNumCoveredWriteQuorums(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { @@ -348,14 +351,14 @@ public void testRemoveBookieFromCluster() { BookieSocketAddress addr3 = new BookieSocketAddress("127.0.1.2", 3181); // /2 rack BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.4", 3181); // /4 rack // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); } @Test @@ -373,7 +376,7 @@ public void testNetworkTopologyScriptFileNameIsEmpty() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); try { repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); } catch (RuntimeException re) { fail("EnforceMinNumRacksPerWriteQuorum is not set, so repp.initialize should succeed even if" + " networkTopologyScriptFileName is empty"); @@ -384,7 +387,7 @@ public void testNetworkTopologyScriptFileNameIsEmpty() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); try { repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); fail("EnforceMinNumRacksPerWriteQuorum is set, so repp.initialize should fail if" + " networkTopologyScriptFileName is empty"); } catch (RuntimeException re) { @@ -395,7 +398,7 @@ public void testNetworkTopologyScriptFileNameIsEmpty() throws Exception { "src/test/resources/networkmappingscript.sh"); try { repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); } catch (RuntimeException re) { fail("EnforceMinNumRacksPerWriteQuorum is set and networkTopologyScriptFileName is not empty," + " so it should succeed"); @@ -420,13 +423,14 @@ public void testIfValidateConfFails() throws Exception { newConf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS, newConf.getTimeoutTimerNumTicks()); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(newConf, Optional. empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.uninitalize(); repp = new RackawareEnsemblePlacementPolicy(); try { repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); } catch (RuntimeException re) { fail("EnforceMinNumRacksPerWriteQuorum is not set, so repp.initialize should succeed" + " even if mapping.validateConf fails"); @@ -437,7 +441,7 @@ public void testIfValidateConfFails() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); try { repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); fail("EnforceMinNumRacksPerWriteQuorum is set, so repp.initialize should fail" + " if mapping.validateConf fails"); } catch (RuntimeException re) { @@ -454,14 +458,14 @@ public void testIfValidateConfFails() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); try { repp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, - NullStatsLogger.INSTANCE); + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); } catch (RuntimeException re) { fail("EnforceMinNumRacksPerWriteQuorum is set, and mapping.validateConf succeeds." + " So repp.initialize should succeed"); } } - private int getNumCoveredWriteQuorums(List ensemble, int writeQuorumSize) + private int getNumCoveredWriteQuorums(List ensemble, int writeQuorumSize) throws Exception { int ensembleSize = ensemble.size(); int numCoveredWriteQuorums = 0; @@ -469,8 +473,9 @@ private int getNumCoveredWriteQuorums(List ensemble, int wr Set racks = new HashSet(); for (int j = 0; j < writeQuorumSize; j++) { int bookieIdx = (i + j) % ensembleSize; - BookieSocketAddress addr = ensemble.get(bookieIdx); - String hostAddress = addr.getSocketAddress().getAddress().getHostAddress(); + BookieId addr = ensemble.get(bookieIdx); + String hostAddress = repp.bookieAddressResolver.resolve(addr) + .getSocketAddress().getAddress().getHostAddress(); String rack = "/" + hostAddress.charAt(hostAddress.length() - 1); racks.add(rack); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java index b206e649665..e7858f04440 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawarePolicyNotificationUpdates.java @@ -37,6 +37,7 @@ import junit.framework.TestCase; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; @@ -74,7 +75,8 @@ protected void setUp() throws Exception { conf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS, conf.getTimeoutTimerNumTicks()); repp = new RackawareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional. empty(), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); } @@ -99,18 +101,19 @@ public void testNotifyRackChange() throws Exception { int numOfAvailableRacks = 2; // Update cluster - Set addrs = Sets.newHashSet(addr1, addr2, addr3, addr4); + Set addrs = Sets.newHashSet(addr1.toBookieId(), + addr2.toBookieId(), addr3.toBookieId(), addr4.toBookieId()); repp.onClusterChanged(addrs, new HashSet<>()); int ensembleSize = 3; int writeQuorumSize = 2; int acqQuorumSize = 2; - List ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, + List ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, Collections.emptyMap(), Collections.emptySet()).getResult(); int numCovered = TestRackawareEnsemblePlacementPolicy.getNumCoveredWriteQuorums(ensemble, writeQuorumSize, - conf.getMinNumRacksPerWriteQuorum()); + conf.getMinNumRacksPerWriteQuorum(), repp.bookieAddressResolver); assertTrue(numCovered >= 1 && numCovered < 3); - assertTrue(ensemble.contains(addr1)); + assertTrue(ensemble.contains(addr1.toBookieId())); List bookieAddressList = new ArrayList<>(); List rackList = new ArrayList<>(); @@ -122,8 +125,8 @@ public void testNotifyRackChange() throws Exception { ensemble = repp.newEnsemble(ensembleSize, writeQuorumSize, acqQuorumSize, Collections.emptyMap(), Collections.emptySet()).getResult(); assertEquals(3, TestRackawareEnsemblePlacementPolicy.getNumCoveredWriteQuorums(ensemble, writeQuorumSize, - conf.getMinNumRacksPerWriteQuorum())); - assertTrue(ensemble.contains(addr1)); - assertTrue(ensemble.contains(addr2)); + conf.getMinNumRacksPerWriteQuorum(), repp.bookieAddressResolver)); + assertTrue(ensemble.contains(addr1.toBookieId())); + assertTrue(ensemble.contains(addr2.toBookieId())); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java index 2ef72ff7dd2..16a2616ca4d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java @@ -29,7 +29,7 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; @@ -230,7 +230,7 @@ private void readWithFailedBookiesTest(boolean parallelRead) throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - List ensemble = + List ensemble = lh.getLedgerMetadata().getEnsembleAt(5); // kill two bookies killBookie(ensemble.get(0)); @@ -269,7 +269,7 @@ private void readFailureWithFailedBookiesTest(boolean parallelRead) throws Excep LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - List ensemble = + List ensemble = lh.getLedgerMetadata().getEnsembleAt(5); // kill bookies killBookie(ensemble.get(0)); 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 1e2f17e9a67..1ae5fddd59b 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 @@ -27,7 +27,7 @@ import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Assert; import org.junit.Test; @@ -61,10 +61,10 @@ public void testReadTimeout() throws Exception { writelh.addEntry(tmp.getBytes()); } - Set beforeSet = new HashSet(); + Set beforeSet = new HashSet(); beforeSet.addAll(writelh.getLedgerMetadata().getEnsembleAt(numEntries)); - final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); + final BookieId bookieToSleep = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0); int sleeptime = baseClientConf.getReadTimeout() * 3; CountDownLatch latch = sleepBookie(bookieToSleep, sleeptime); latch.await(); @@ -79,7 +79,7 @@ public void addComplete(int rc, LedgerHandle lh, Thread.sleep((baseClientConf.getReadTimeout() * 3) * 1000); Assert.assertTrue("Write request did not finish", completed.get()); - Set afterSet = new HashSet(); + Set afterSet = new HashSet(); afterSet.addAll(writelh.getLedgerMetadata().getEnsembleAt(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/client/TestRegionAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java index a7f42aa35d1..bc0805a04b1 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 @@ -45,6 +45,7 @@ import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.feature.SettableFeature; import org.apache.bookkeeper.feature.SettableFeatureProvider; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; @@ -64,9 +65,10 @@ public class TestRegionAwareEnsemblePlacementPolicy extends TestCase { RegionAwareEnsemblePlacementPolicy repp; final ClientConfiguration conf = new ClientConfiguration(); - final ArrayList ensemble = new ArrayList(); + final ArrayList ensemble = new ArrayList(); DistributionSchedule.WriteSet writeSet = DistributionSchedule.NULL_WRITE_SET; - BookieSocketAddress addr1, addr2, addr3, addr4; + BookieSocketAddress addr1; + BookieSocketAddress addr2, addr3, addr4; HashedWheelTimer timer; static void updateMyRack(String rack) throws Exception { @@ -97,10 +99,10 @@ protected void setUp() throws Exception { StaticDNSResolver.addNodeToRack(addr2.getHostName(), NetworkTopology.DEFAULT_REGION_AND_RACK); StaticDNSResolver.addNodeToRack(addr3.getHostName(), NetworkTopology.DEFAULT_REGION_AND_RACK); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/r1/rack2"); - ensemble.add(addr1); - ensemble.add(addr2); - ensemble.add(addr3); - ensemble.add(addr4); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr3.toBookieId()); + ensemble.add(addr4.toBookieId()); writeSet = writeSetFromValues(0, 1, 2, 3); @@ -110,7 +112,8 @@ protected void setUp() throws Exception { conf.getTimeoutTimerNumTicks()); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); } @Override @@ -123,16 +126,16 @@ static BookiesHealthInfo getBookiesHealthInfo() { return getBookiesHealthInfo(new HashMap<>(), new HashMap<>()); } - static BookiesHealthInfo getBookiesHealthInfo(Map bookieFailureHistory, - Map bookiePendingRequests) { + static BookiesHealthInfo getBookiesHealthInfo(Map bookieFailureHistory, + Map bookiePendingRequests) { return new BookiesHealthInfo() { @Override - public long getBookieFailureHistory(BookieSocketAddress bookieSocketAddress) { + public long getBookieFailureHistory(BookieId bookieSocketAddress) { return bookieFailureHistory.getOrDefault(bookieSocketAddress, -1L); } @Override - public long getBookiePendingRequests(BookieSocketAddress bookieSocketAddress) { + public long getBookiePendingRequests(BookieId bookieSocketAddress) { return bookiePendingRequests.getOrDefault(bookieSocketAddress, 0L); } }; @@ -145,7 +148,8 @@ public void testNotReorderReadIfInDefaultRack() throws Exception { updateMyRack(NetworkTopology.DEFAULT_REGION_AND_RACK); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -159,17 +163,18 @@ public void testNodeInSameRegion() throws Exception { updateMyRack("/r1/rack3"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // make sure we've detected the right region assertEquals("r1", repp.myRegion); - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( ensemble, getBookiesHealthInfo(), writeSet.copy()); @@ -188,7 +193,8 @@ public void testNodeNotInSameRegions() throws Exception { updateMyRack("/r2/rack1"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -203,17 +209,18 @@ public void testNodeDown() throws Exception { updateMyRack("/r1/rack1"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -230,18 +237,19 @@ public void testNodeReadOnly() throws Exception { updateMyRack("/r1/rack1"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - Set ro = new HashSet(); - ro.add(addr1); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + Set ro = new HashSet(); + ro.add(addr1.toBookieId()); repp.onClusterChanged(addrs, ro); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -259,18 +267,19 @@ public void testNodeSlow() throws Exception { updateMyRack("/r1/rack1"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - repp.registerSlowBookie(addr1, 0L); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 1L); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + repp.registerSlowBookie(addr1.toBookieId(), 0L); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 1L); repp.onClusterChanged(addrs, new HashSet<>()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -288,20 +297,21 @@ public void testTwoNodesSlow() throws Exception { updateMyRack("/r1/rack1"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - repp.registerSlowBookie(addr1, 0L); - repp.registerSlowBookie(addr2, 0L); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 1L); - bookiePendingMap.put(addr2, 2L); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + repp.registerSlowBookie(addr1.toBookieId(), 0L); + repp.registerSlowBookie(addr2.toBookieId(), 0L); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 1L); + bookiePendingMap.put(addr2.toBookieId(), 2L); repp.onClusterChanged(addrs, new HashSet<>()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -319,18 +329,19 @@ public void testTwoNodesDown() throws Exception { updateMyRack("/r1/rack1"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - addrs.remove(addr2); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + addrs.remove(addr2.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -347,20 +358,21 @@ public void testNodeDownAndNodeSlow() throws Exception { updateMyRack("/r1/rack1"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - repp.registerSlowBookie(addr1, 0L); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr1, 1L); - addrs.remove(addr2); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + repp.registerSlowBookie(addr1.toBookieId(), 0L); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr1.toBookieId(), 1L); + addrs.remove(addr2.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( @@ -377,22 +389,23 @@ public void testNodeDownAndReadOnlyAndNodeSlow() throws Exception { updateMyRack("/r1/rack1"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); - addrs.remove(addr1); - addrs.remove(addr2); - Set ro = new HashSet<>(); - ro.add(addr2); - repp.registerSlowBookie(addr3, 0L); - Map bookiePendingMap = new HashMap<>(); - bookiePendingMap.put(addr3, 1L); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr1.toBookieId()); + addrs.remove(addr2.toBookieId()); + Set ro = new HashSet<>(); + ro.add(addr2.toBookieId()); + repp.registerSlowBookie(addr3.toBookieId(), 0L); + Map bookiePendingMap = new HashMap<>(); + bookiePendingMap.put(addr3.toBookieId(), 1L); repp.onClusterChanged(addrs, ro); DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); @@ -416,16 +429,16 @@ public void testReplaceBookieWithEnoughBookiesInSameRegion() throws Exception { StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r3"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, - new ArrayList(), addr2, new HashSet()).getResult(); - assertEquals(addr3, replacedBookie); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, null, + new ArrayList(), addr2.toBookieId(), new HashSet()).getResult(); + assertEquals(addr3.toBookieId(), replacedBookie); } @Test @@ -440,20 +453,21 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRegion() throws Excepti StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region2/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region3/r4"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, null, - new ArrayList(), addr2, excludedAddrs).getResult(); - - assertFalse(addr1.equals(replacedBookie)); - assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr1.toBookieId()); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, null, + new ArrayList(), addr2.toBookieId(), excludedAddrs).getResult(); + + assertFalse(addr1.toBookieId().equals(replacedBookie)); + assertTrue(addr3.toBookieId().equals(replacedBookie) + || addr4.toBookieId().equals(replacedBookie)); } @Test @@ -468,15 +482,15 @@ public void testNewEnsembleBookieWithNotEnoughBookies() throws Exception { StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region3/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region4/r4"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - List list = repp.newEnsemble(5, 5, 3, null, - new HashSet()).getResult(); + List list = repp.newEnsemble(5, 5, 3, null, + new HashSet()).getResult(); LOG.info("Ensemble : {}", list); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { @@ -496,19 +510,19 @@ public void testReplaceBookieWithNotEnoughBookies() throws Exception { StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region3/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region4/r4"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr1); - excludedAddrs.add(addr3); - excludedAddrs.add(addr4); + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr1.toBookieId()); + excludedAddrs.add(addr3.toBookieId()); + excludedAddrs.add(addr4.toBookieId()); try { - repp.replaceBookie(1, 1, 1, null, new ArrayList(), addr2, excludedAddrs); + repp.replaceBookie(1, 1, 1, null, new ArrayList(), addr2.toBookieId(), excludedAddrs); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not enou @@ -519,7 +533,8 @@ public void testReplaceBookieWithNotEnoughBookies() throws Exception { public void testNewEnsembleWithSingleRegion() throws Exception { repp.uninitalize(); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); @@ -530,18 +545,18 @@ public void testNewEnsembleWithSingleRegion() throws Exception { StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region1/r2"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(3, 2, 2, null, - new HashSet()).getResult(); + List ensemble = repp.newEnsemble(3, 2, 2, null, + new HashSet()).getResult(); assertEquals(0, getNumCoveredRegionsInWriteQuorum(ensemble, 2)); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, - new HashSet()).getResult(); + List ensemble2 = repp.newEnsemble(4, 2, 2, null, + new HashSet()).getResult(); assertEquals(0, getNumCoveredRegionsInWriteQuorum(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -552,7 +567,8 @@ public void testNewEnsembleWithSingleRegion() throws Exception { public void testNewEnsembleWithMultipleRegions() throws Exception { repp.uninitalize(); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); @@ -563,15 +579,15 @@ public void testNewEnsembleWithMultipleRegions() throws Exception { StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region1/r2"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(3, 2, 2, null, - new HashSet()).getResult(); + List ensemble = repp.newEnsemble(3, 2, 2, null, + new HashSet()).getResult(); int numCovered = getNumCoveredRegionsInWriteQuorum(ensemble, 2); assertTrue(numCovered >= 1); assertTrue(numCovered < 3); @@ -579,8 +595,8 @@ public void testNewEnsembleWithMultipleRegions() throws Exception { fail("Should not get not enough bookies exception even there is only one rack."); } try { - List ensemble2 = repp.newEnsemble(4, 2, 2, null, - new HashSet()).getResult(); + List ensemble2 = repp.newEnsemble(4, 2, 2, null, + new HashSet()).getResult(); int numCovered = getNumCoveredRegionsInWriteQuorum(ensemble2, 2); assertTrue(numCovered >= 1 && numCovered < 3); } catch (BKNotEnoughBookiesException bnebe) { @@ -608,22 +624,22 @@ public void testNewEnsembleWithEnoughRegions() throws Exception { StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/region2/r13"); StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/region3/r14"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble1 = repp.newEnsemble(3, 2, 2, null, - new HashSet()).getResult(); + List ensemble1 = repp.newEnsemble(3, 2, 2, null, + new HashSet()).getResult(); assertEquals(3, getNumCoveredRegionsInWriteQuorum(ensemble1, 2)); - List ensemble2 = repp.newEnsemble(4, 2, 2, null, - new HashSet()).getResult(); + List ensemble2 = repp.newEnsemble(4, 2, 2, null, + new HashSet()).getResult(); assertEquals(4, getNumCoveredRegionsInWriteQuorum(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -634,7 +650,8 @@ public void testNewEnsembleWithEnoughRegions() throws Exception { public void testNewEnsembleWithThreeRegions() throws Exception { repp.uninitalize(); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); @@ -657,38 +674,38 @@ public void testNewEnsembleWithThreeRegions() throws Exception { StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region2/r23"); StaticDNSResolver.addNodeToRack(addr10.getHostName(), "/region1/r24"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - addrs.add(addr9); - addrs.add(addr10); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + addrs.add(addr9.toBookieId()); + addrs.add(addr10.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(6, 6, 4, null, - new HashSet()).getResult(); - assert(ensemble.contains(addr4)); - assert(ensemble.contains(addr8)); + List ensemble = repp.newEnsemble(6, 6, 4, null, + new HashSet()).getResult(); + assert(ensemble.contains(addr4.toBookieId())); + assert(ensemble.contains(addr8.toBookieId())); assert(ensemble.size() == 6); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensemble = repp.newEnsemble(7, 7, 4, null, new HashSet()).getResult(); - assert(ensemble.contains(addr4)); - assert(ensemble.contains(addr8)); + ensemble = repp.newEnsemble(7, 7, 4, null, new HashSet()).getResult(); + assert(ensemble.contains(addr4.toBookieId())); + assert(ensemble.contains(addr8.toBookieId())); assert(ensemble.size() == 7); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensemble = repp.newEnsemble(8, 8, 5, null, new HashSet()).getResult(); - assert(ensemble.contains(addr4)); - assert(ensemble.contains(addr8)); + ensemble = repp.newEnsemble(8, 8, 5, null, new HashSet()).getResult(); + assert(ensemble.contains(addr4.toBookieId())); + assert(ensemble.contains(addr8.toBookieId())); assert(ensemble.size() == 8); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensemble = repp.newEnsemble(9, 9, 5, null, new HashSet()).getResult(); - assert(ensemble.contains(addr4)); - assert(ensemble.contains(addr8)); + ensemble = repp.newEnsemble(9, 9, 5, null, new HashSet()).getResult(); + assert(ensemble.contains(addr4.toBookieId())); + assert(ensemble.contains(addr8.toBookieId())); assert(ensemble.size() == 9); assertEquals(3, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -702,7 +719,8 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { repp.uninitalize(); repp = new RegionAwareEnsemblePlacementPolicy(); conf.setProperty(REPP_DISALLOW_BOOKIE_PLACEMENT_IN_REGION_FEATURE_NAME, "disallowBookies"); - repp.initialize(conf, Optional.empty(), timer, featureProvider, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, featureProvider, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); @@ -725,50 +743,50 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region2/r23"); StaticDNSResolver.addNodeToRack(addr10.getHostName(), "/region1/r24"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - addrs.add(addr9); - addrs.add(addr10); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + addrs.add(addr9.toBookieId()); + addrs.add(addr10.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { ((SettableFeature) featureProvider.scope("region1").getFeature("disallowBookies")).set(true); - List ensemble = repp.newEnsemble(6, 6, 4, null, - new HashSet()).getResult(); + List ensemble = repp.newEnsemble(6, 6, 4, null, + new HashSet()).getResult(); assertEquals(2, getNumRegionsInEnsemble(ensemble)); - assert(ensemble.contains(addr1)); - assert(ensemble.contains(addr3)); - assert(ensemble.contains(addr4)); - assert(ensemble.contains(addr7)); - assert(ensemble.contains(addr8)); - assert(ensemble.contains(addr9)); + assert(ensemble.contains(addr1.toBookieId())); + assert(ensemble.contains(addr3.toBookieId())); + assert(ensemble.contains(addr4.toBookieId())); + assert(ensemble.contains(addr7.toBookieId())); + assert(ensemble.contains(addr8.toBookieId())); + assert(ensemble.contains(addr9.toBookieId())); assert(ensemble.size() == 6); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); } try { ((SettableFeature) featureProvider.scope("region2").getFeature("disallowBookies")).set(true); - repp.newEnsemble(6, 6, 4, null, new HashSet()); + repp.newEnsemble(6, 6, 4, null, new HashSet()); fail("Should get not enough bookies exception even there is only one region with insufficient bookies."); } catch (BKNotEnoughBookiesException bnebe) { // Expected } try { ((SettableFeature) featureProvider.scope("region2").getFeature("disallowBookies")).set(false); - List ensemble = repp.newEnsemble(6, 6, 4, null, - new HashSet()).getResult(); - assert(ensemble.contains(addr1)); - assert(ensemble.contains(addr3)); - assert(ensemble.contains(addr4)); - assert(ensemble.contains(addr7)); - assert(ensemble.contains(addr8)); - assert(ensemble.contains(addr9)); + List ensemble = repp.newEnsemble(6, 6, 4, null, + new HashSet()).getResult(); + assert(ensemble.contains(addr1.toBookieId())); + assert(ensemble.contains(addr3.toBookieId())); + assert(ensemble.contains(addr4.toBookieId())); + assert(ensemble.contains(addr7.toBookieId())); + assert(ensemble.contains(addr8.toBookieId())); + assert(ensemble.contains(addr9.toBookieId())); assert(ensemble.size() == 6); assertEquals(2, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -783,7 +801,8 @@ public void testNewEnsembleWithFiveRegions() throws Exception { repp = new RegionAwareEnsemblePlacementPolicy(); conf.setProperty(REPP_REGIONS_TO_WRITE, "region1;region2;region3;region4;region5"); conf.setProperty(REPP_MINIMUM_REGIONS_FOR_DURABILITY, 5); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr1 = new BookieSocketAddress("127.1.0.2", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.1.0.3", 3181); BookieSocketAddress addr3 = new BookieSocketAddress("127.1.0.4", 3181); @@ -816,27 +835,27 @@ public void testNewEnsembleWithFiveRegions() throws Exception { StaticDNSResolver.addNodeToRack(addr14.getHostName(), "/region5/r34"); StaticDNSResolver.addNodeToRack(addr15.getHostName(), "/region5/r35"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - addrs.add(addr9); - addrs.add(addr10); - addrs.add(addr11); - addrs.add(addr12); - addrs.add(addr13); - addrs.add(addr14); - addrs.add(addr15); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + addrs.add(addr9.toBookieId()); + addrs.add(addr10.toBookieId()); + addrs.add(addr11.toBookieId()); + addrs.add(addr12.toBookieId()); + addrs.add(addr13.toBookieId()); + addrs.add(addr14.toBookieId()); + addrs.add(addr15.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); try { - List ensemble = repp.newEnsemble(10, 10, 10, null, - new HashSet()).getResult(); + List ensemble = repp.newEnsemble(10, 10, 10, null, + new HashSet()).getResult(); assert(ensemble.size() == 10); assertEquals(5, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -845,11 +864,11 @@ public void testNewEnsembleWithFiveRegions() throws Exception { } try { - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr10); - List ensemble = repp.newEnsemble(10, 10, 10, null, + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr10.toBookieId()); + List ensemble = repp.newEnsemble(10, 10, 10, null, excludedAddrs).getResult(); - assert(ensemble.contains(addr11) && ensemble.contains(addr12)); + assert(ensemble.contains(addr11.toBookieId()) && ensemble.contains(addr12.toBookieId())); assert(ensemble.size() == 10); assertEquals(5, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -891,7 +910,8 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole } conf.setProperty(REPP_DISALLOW_BOOKIE_PLACEMENT_IN_REGION_FEATURE_NAME, "disallowBookies"); - repp.initialize(conf, Optional.empty(), timer, featureProvider, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, featureProvider, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr1 = new BookieSocketAddress("127.1.0.2", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.1.0.3", 3181); BookieSocketAddress addr3 = new BookieSocketAddress("127.1.0.4", 3181); @@ -913,17 +933,17 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region3/r23"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - addrs.add(addr9); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + addrs.add(addr9.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); SettableFeature disableDurabilityFeature = (SettableFeature) featureProvider.getFeature( @@ -938,9 +958,9 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole ackQuorum = 5; } - List ensemble; + List ensemble; try { - ensemble = repp.newEnsemble(6, 6, ackQuorum, null, new HashSet()).getResult(); + ensemble = repp.newEnsemble(6, 6, ackQuorum, null, new HashSet()).getResult(); assert(ensemble.size() == 6); assertEquals(3, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -951,17 +971,17 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole if (disableOneRegion) { ((SettableFeature) featureProvider.scope("region2").getFeature("disallowBookies")).set(true); - Set region2Bookies = new HashSet(); - region2Bookies.add(addr4); - region2Bookies.add(addr5); - region2Bookies.add(addr6); - Set region1And3Bookies = new HashSet(addrs); + Set region2Bookies = new HashSet(); + region2Bookies.add(addr4.toBookieId()); + region2Bookies.add(addr5.toBookieId()); + region2Bookies.add(addr6.toBookieId()); + Set region1And3Bookies = new HashSet(addrs); region1And3Bookies.removeAll(region2Bookies); - Set excludedAddrs = new HashSet(); - for (BookieSocketAddress addr: region2Bookies) { + Set excludedAddrs = new HashSet(); + for (BookieId addr: region2Bookies) { if (ensemble.contains(addr)) { - BookieSocketAddress replacedBookie = repp.replaceBookie(6, 6, ackQuorum, null, + BookieId replacedBookie = repp.replaceBookie(6, 6, ackQuorum, null, ensemble, addr, excludedAddrs).getResult(); ensemble.remove(addr); ensemble.add(replacedBookie); @@ -970,23 +990,23 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole assertEquals(2, getNumRegionsInEnsemble(ensemble)); assertTrue(ensemble.containsAll(region1And3Bookies)); } else { - BookieSocketAddress bookieToReplace; - BookieSocketAddress replacedBookieExpected; - if (ensemble.contains(addr4)) { - bookieToReplace = addr4; - if (ensemble.contains(addr5)) { - replacedBookieExpected = addr6; + BookieId bookieToReplace; + BookieId replacedBookieExpected; + if (ensemble.contains(addr4.toBookieId())) { + bookieToReplace = addr4.toBookieId(); + if (ensemble.contains(addr5.toBookieId())) { + replacedBookieExpected = addr6.toBookieId(); } else { - replacedBookieExpected = addr5; + replacedBookieExpected = addr5.toBookieId(); } } else { - replacedBookieExpected = addr4; - bookieToReplace = addr5; + replacedBookieExpected = addr4.toBookieId(); + bookieToReplace = addr5.toBookieId(); } - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); try { - BookieSocketAddress replacedBookie = repp.replaceBookie(6, 6, ackQuorum, null, + BookieId replacedBookie = repp.replaceBookie(6, 6, ackQuorum, null, ensemble, bookieToReplace, excludedAddrs).getResult(); assert (replacedBookie.equals(replacedBookieExpected)); assertEquals(3, getNumRegionsInEnsemble(ensemble)); @@ -1030,7 +1050,8 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di conf.setProperty(REPP_ENABLE_DURABILITY_ENFORCEMENT_IN_REPLACE, true); } - repp.initialize(conf, Optional.empty(), timer, featureProvider, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, featureProvider, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr1 = new BookieSocketAddress("127.1.0.2", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.1.0.3", 3181); BookieSocketAddress addr3 = new BookieSocketAddress("127.1.0.4", 3181); @@ -1052,17 +1073,17 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region1/r23"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - addrs.add(addr9); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + addrs.add(addr9.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); if (disableDurability) { ((SettableFeature) featureProvider.getFeature( @@ -1070,9 +1091,9 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di .set(true); } - List ensemble; + List ensemble; try { - ensemble = repp.newEnsemble(6, 6, 4, null, new HashSet()).getResult(); + ensemble = repp.newEnsemble(6, 6, 4, null, new HashSet()).getResult(); assert(ensemble.size() == 6); } catch (BKNotEnoughBookiesException bnebe) { LOG.error("BKNotEnoughBookiesException", bnebe); @@ -1080,7 +1101,7 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di throw bnebe; } - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); try { repp.replaceBookie(6, 6, 4, null, ensemble, ensemble.get(2), excludedAddrs); @@ -1096,7 +1117,8 @@ public void testNewEnsembleFailWithFiveRegions() throws Exception { conf.setProperty(REPP_REGIONS_TO_WRITE, "region1;region2;region3;region4;region5"); conf.setProperty(REPP_MINIMUM_REGIONS_FOR_DURABILITY, 5); conf.setProperty(REPP_ENABLE_VALIDATION, false); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); @@ -1119,22 +1141,22 @@ public void testNewEnsembleFailWithFiveRegions() throws Exception { StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region5/r23"); StaticDNSResolver.addNodeToRack(addr10.getHostName(), "/region5/r24"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - addrs.add(addr9); - addrs.add(addr10); - repp.onClusterChanged(addrs, new HashSet()); - - Set excludedAddrs = new HashSet(); - excludedAddrs.add(addr10); - excludedAddrs.add(addr9); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + addrs.add(addr9.toBookieId()); + addrs.add(addr10.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); + + Set excludedAddrs = new HashSet(); + excludedAddrs.add(addr10.toBookieId()); + excludedAddrs.add(addr9.toBookieId()); try { LOG.info("Ensemble : {}", repp.newEnsemble(5, 5, 5, null, excludedAddrs).getResult()); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); @@ -1148,7 +1170,8 @@ private void prepareNetworkTopologyForReorderTests(String myRegion) throws Excep updateMyRack("/" + myRegion); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); @@ -1170,17 +1193,17 @@ private void prepareNetworkTopologyForReorderTests(String myRegion) throws Excep StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/region3/r2"); StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region3/r3"); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - addrs.add(addr9); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + addrs.add(addr9.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); } @Test @@ -1195,8 +1218,8 @@ public void testBasicReorderReadLACSequenceWithLocalRegion() throws Exception { private void basicReorderReadSequenceWithLocalRegionTest(String myRegion, boolean isReadLAC) throws Exception { prepareNetworkTopologyForReorderTests(myRegion); - List ensemble = repp.newEnsemble(9, 9, 5, null, - new HashSet()).getResult(); + List ensemble = repp.newEnsemble(9, 9, 5, null, + new HashSet()).getResult(); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); @@ -1223,18 +1246,22 @@ private void basicReorderReadSequenceWithLocalRegionTest(String myRegion, boolea // first few nodes less than REMOTE_NODE_IN_REORDER_SEQUENCE should be local region int k = 0; for (; k < RegionAwareEnsemblePlacementPolicy.REMOTE_NODE_IN_REORDER_SEQUENCE; k++) { - BookieSocketAddress address = ensemble.get(readSet.get(k)); - assertEquals(myRegion, StaticDNSResolver.getRegion(address.getHostName())); + BookieId address = ensemble.get(readSet.get(k)); + assertEquals(myRegion, StaticDNSResolver.getRegion(repp.bookieAddressResolver + .resolve(address).getHostName())); } - BookieSocketAddress remoteAddress = ensemble.get(readSet.get(k)); - assertFalse(myRegion.equals(StaticDNSResolver.getRegion(remoteAddress.getHostName()))); + BookieId remoteAddress = ensemble.get(readSet.get(k)); + assertFalse(myRegion.equals(StaticDNSResolver.getRegion(repp.bookieAddressResolver + .resolve(remoteAddress).getHostName()))); k++; - BookieSocketAddress localAddress = ensemble.get(readSet.get(k)); - assertEquals(myRegion, StaticDNSResolver.getRegion(localAddress.getHostName())); + BookieId localAddress = ensemble.get(readSet.get(k)); + assertEquals(myRegion, StaticDNSResolver.getRegion(repp.bookieAddressResolver + .resolve(localAddress).getHostName())); k++; for (; k < ensembleSize; k++) { - BookieSocketAddress address = ensemble.get(readSet.get(k)); - assertFalse(myRegion.equals(StaticDNSResolver.getRegion(address.getHostName()))); + BookieId address = ensemble.get(readSet.get(k)); + assertFalse(myRegion.equals(StaticDNSResolver.getRegion(repp.bookieAddressResolver + .resolve(address).getHostName()))); } } } @@ -1252,8 +1279,8 @@ public void testBasicReorderReadLACSequenceWithRemoteRegion() throws Exception { private void basicReorderReadSequenceWithRemoteRegionTest(String myRegion, boolean isReadLAC) throws Exception { prepareNetworkTopologyForReorderTests(myRegion); - List ensemble = repp.newEnsemble(9, 9, 5, null, - new HashSet()).getResult(); + List ensemble = repp.newEnsemble(9, 9, 5, null, + new HashSet()).getResult(); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); @@ -1291,10 +1318,11 @@ public void testReorderReadLACSequenceWithUnavailableOrReadOnlyBookies() throws reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(true); } - static Set getBookiesForRegion(List ensemble, String region) { - Set regionBookies = new HashSet(); - for (BookieSocketAddress address : ensemble) { - String r = StaticDNSResolver.getRegion(address.getHostName()); + private Set getBookiesForRegion(List ensemble, String region) { + Set regionBookies = new HashSet(); + for (BookieId address : ensemble) { + String r = StaticDNSResolver.getRegion(repp.bookieAddressResolver + .resolve(address).getHostName()); if (r.equals(region)) { regionBookies.add(address); } @@ -1302,13 +1330,14 @@ static Set getBookiesForRegion(List en return regionBookies; } - static void appendBookieIndexByRegion(List ensemble, + void appendBookieIndexByRegion(List ensemble, DistributionSchedule.WriteSet writeSet, String region, List finalSet) { for (int i = 0; i < writeSet.size(); i++) { int bi = writeSet.get(i); - String r = StaticDNSResolver.getRegion(ensemble.get(bi).getHostName()); + String r = StaticDNSResolver.getRegion(repp.bookieAddressResolver + .resolve(ensemble.get(bi)).getHostName()); if (r.equals(region)) { finalSet.add(bi); } @@ -1323,16 +1352,16 @@ private void reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(boolean isR prepareNetworkTopologyForReorderTests(myRegion); - List ensemble = repp.newEnsemble(9, 9, 5, null, - new HashSet()).getResult(); + List ensemble = repp.newEnsemble(9, 9, 5, null, + new HashSet()).getResult(); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); LOG.info("My region is {}, ensemble : {}", repp.myRegion, ensemble); - Set readOnlyBookies = getBookiesForRegion(ensemble, readOnlyRegion); - Set writeBookies = getBookiesForRegion(ensemble, writeRegion); + Set readOnlyBookies = getBookiesForRegion(ensemble, readOnlyRegion); + Set writeBookies = getBookiesForRegion(ensemble, writeRegion); repp.onClusterChanged(writeBookies, readOnlyBookies); @@ -1368,15 +1397,15 @@ ensemble, getBookiesHealthInfo(), } } - private int getNumRegionsInEnsemble(List ensemble) { + private int getNumRegionsInEnsemble(List ensemble) { Set regions = new HashSet(); - for (BookieSocketAddress addr: ensemble) { - regions.add(StaticDNSResolver.getRegion(addr.getHostName())); + for (BookieId addr: ensemble) { + regions.add(StaticDNSResolver.getRegion(repp.bookieAddressResolver.resolve(addr).getHostName())); } return regions.size(); } - private int getNumCoveredRegionsInWriteQuorum(List ensemble, int writeQuorumSize) + private int getNumCoveredRegionsInWriteQuorum(List ensemble, int writeQuorumSize) throws Exception { int ensembleSize = ensemble.size(); int numCoveredWriteQuorums = 0; @@ -1384,8 +1413,8 @@ private int getNumCoveredRegionsInWriteQuorum(List ensemble Set regions = new HashSet(); for (int j = 0; j < writeQuorumSize; j++) { int bookieIdx = (i + j) % ensembleSize; - BookieSocketAddress addr = ensemble.get(bookieIdx); - regions.add(StaticDNSResolver.getRegion(addr.getHostName())); + BookieId addr = ensemble.get(bookieIdx); + regions.add(StaticDNSResolver.getRegion(repp.bookieAddressResolver.resolve(addr).getHostName())); } numCoveredWriteQuorums += (regions.size() > 1 ? 1 : 0); } @@ -1398,7 +1427,8 @@ public void testNodeWithFailures() throws Exception { updateMyRack("/r2/rack1"); repp = new RegionAwareEnsemblePlacementPolicy(); - repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); @@ -1411,43 +1441,43 @@ public void testNodeWithFailures() throws Exception { StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/r2/rack3"); StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/r2/rack4"); StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/r1/rack4"); - ensemble.add(addr5); - ensemble.add(addr6); - ensemble.add(addr7); - ensemble.add(addr8); + ensemble.add(addr5.toBookieId()); + ensemble.add(addr6.toBookieId()); + ensemble.add(addr7.toBookieId()); + ensemble.add(addr8.toBookieId()); DistributionSchedule.WriteSet writeSet2 = writeSetFromValues(0, 1, 2, 3, 4, 5, 6, 7); - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); - addrs.add(addr6); - addrs.add(addr7); - addrs.add(addr8); - repp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); + addrs.add(addr6.toBookieId()); + addrs.add(addr7.toBookieId()); + addrs.add(addr8.toBookieId()); + repp.onClusterChanged(addrs, new HashSet()); - HashMap bookieFailures = new HashMap(); + HashMap bookieFailures = new HashMap(); - bookieFailures.put(addr1, 20L); - bookieFailures.put(addr2, 22L); - bookieFailures.put(addr3, 24L); - bookieFailures.put(addr4, 25L); + bookieFailures.put(addr1.toBookieId(), 20L); + bookieFailures.put(addr2.toBookieId(), 22L); + bookieFailures.put(addr3.toBookieId(), 24L); + bookieFailures.put(addr4.toBookieId(), 25L); LOG.info("write set : {}", writeSet2); DistributionSchedule.WriteSet reoderSet = repp.reorderReadSequence( ensemble, getBookiesHealthInfo(bookieFailures, new HashMap<>()), writeSet2); LOG.info("reorder set : {}", reoderSet); - assertEquals(ensemble.get(reoderSet.get(0)), addr6); - assertEquals(ensemble.get(reoderSet.get(1)), addr7); - assertEquals(ensemble.get(reoderSet.get(2)), addr5); - assertEquals(ensemble.get(reoderSet.get(3)), addr2); - assertEquals(ensemble.get(reoderSet.get(4)), addr3); - assertEquals(ensemble.get(reoderSet.get(5)), addr8); - assertEquals(ensemble.get(reoderSet.get(6)), addr1); - assertEquals(ensemble.get(reoderSet.get(7)), addr4); + assertEquals(ensemble.get(reoderSet.get(0)), addr6.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(1)), addr7.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(2)), addr5.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(3)), addr2.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(4)), addr3.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(5)), addr8.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(6)), addr1.toBookieId()); + assertEquals(ensemble.get(reoderSet.get(7)), addr4.toBookieId()); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java index 510dd3fac67..c0b8e2aa8e4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java @@ -38,7 +38,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; import org.junit.Test; @@ -138,7 +138,7 @@ public void testSpeculativeRead() throws Exception { // sleep second bookie CountDownLatch sleepLatch = new CountDownLatch(1); - BookieSocketAddress second = lnospec.getLedgerMetadata().getAllEnsembles().get(0L).get(1); + BookieId second = lnospec.getLedgerMetadata().getAllEnsembles().get(0L).get(1); sleepBookie(second, sleepLatch); try { @@ -217,7 +217,7 @@ public void testSpeculativeReadMultipleReplicasDown() throws Exception { && latch1.getDuration() < timeout * 3); // bookies 1 & 2 should be registered as slow bookies because of speculative reads - Set expectedSlowBookies = new HashSet<>(); + Set expectedSlowBookies = new HashSet<>(); expectedSlowBookies.add(l.getLedgerMetadata().getAllEnsembles().get(0L).get(1)); expectedSlowBookies.add(l.getLedgerMetadata().getAllEnsembles().get(0L).get(2)); assertEquals(((RackawareEnsemblePlacementPolicy) bkspec.getPlacementPolicy()).slowBookies.asMap().keySet(), @@ -328,7 +328,7 @@ public void testSpeculativeReadScheduling() throws Exception { LedgerHandle l = bkspec.openLedger(id, digestType, passwd); - List ensemble = l.getLedgerMetadata().getAllEnsembles().get(0L); + List ensemble = l.getLedgerMetadata().getAllEnsembles().get(0L); BitSet allHosts = new BitSet(ensemble.size()); for (int i = 0; i < ensemble.size(); i++) { allHosts.set(i, true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java index 26a265b6a8f..2120df1048e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java @@ -40,6 +40,7 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LongHierarchicalLedgerManagerFactory; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; @@ -94,9 +95,9 @@ public void testWatchEnsembleChange() throws Exception { LedgerHandle readLh = bkc.openLedgerNoRecovery(lh.getId(), digestType, "".getBytes()); long lastLAC = readLh.getLastAddConfirmed(); assertEquals(numEntries - 2, lastLAC); - List ensemble = + List ensemble = lh.getCurrentEnsemble(); - for (BookieSocketAddress addr : ensemble) { + for (BookieId addr : ensemble) { killBookie(addr); } // write another batch of entries, which will trigger ensemble change @@ -132,11 +133,10 @@ private void testWatchMetadataRemoval(LedgerManagerFactory factory) throws Excep final CountDownLatch createLatch = new CountDownLatch(1); final CountDownLatch removeLatch = new CountDownLatch(1); - List ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234), - new BookieSocketAddress("192.0.2.4", 1234)); + List ensemble = Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.2", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.3", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.4", 1234).toBookieId()); idGenerator.generateLedgerId(new GenericCallback() { @Override public void operationComplete(int rc, final Long lid) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestZoneawareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestZoneawareEnsemblePlacementPolicy.java index fb669db10a8..12c06742495 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestZoneawareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestZoneawareEnsemblePlacementPolicy.java @@ -43,6 +43,7 @@ import org.apache.bookkeeper.client.EnsemblePlacementPolicy.PlacementResult; import org.apache.bookkeeper.client.ZoneawareEnsemblePlacementPolicyImpl.ZoneAwareNodeLocation; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; @@ -60,10 +61,11 @@ public class TestZoneawareEnsemblePlacementPolicy extends TestCase { static final Logger LOG = LoggerFactory.getLogger(TestZoneawareEnsemblePlacementPolicy.class); ZoneawareEnsemblePlacementPolicy zepp; - final List ensemble = new ArrayList(); + final List ensemble = new ArrayList(); DistributionSchedule.WriteSet writeSet = DistributionSchedule.NULL_WRITE_SET; ClientConfiguration conf = new ClientConfiguration(); - BookieSocketAddress addr1, addr2, addr3, addr4; + BookieSocketAddress addr1; + BookieSocketAddress addr2, addr3, addr4; io.netty.util.HashedWheelTimer timer; @Override @@ -85,17 +87,18 @@ protected void setUp() throws Exception { StaticDNSResolver.addNodeToRack(addr2.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); StaticDNSResolver.addNodeToRack(addr3.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); StaticDNSResolver.addNodeToRack(addr4.getHostName(), NetworkTopology.DEFAULT_ZONE + "/ud2"); - ensemble.add(addr1); - ensemble.add(addr2); - ensemble.add(addr3); - ensemble.add(addr4); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr3.toBookieId()); + ensemble.add(addr4.toBookieId()); writeSet = writeSetFromValues(0, 1, 2, 3); timer = new HashedWheelTimer(new ThreadFactoryBuilder().setNameFormat("TestTimer-%d").build(), conf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS, conf.getTimeoutTimerNumTicks()); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(conf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(conf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); } @@ -109,16 +112,16 @@ static BookiesHealthInfo getBookiesHealthInfo() { return getBookiesHealthInfo(new HashMap<>(), new HashMap<>()); } - static BookiesHealthInfo getBookiesHealthInfo(Map bookieFailureHistory, - Map bookiePendingRequests) { + static BookiesHealthInfo getBookiesHealthInfo(Map bookieFailureHistory, + Map bookiePendingRequests) { return new BookiesHealthInfo() { @Override - public long getBookieFailureHistory(BookieSocketAddress bookieSocketAddress) { + public long getBookieFailureHistory(BookieId bookieSocketAddress) { return bookieFailureHistory.getOrDefault(bookieSocketAddress, -1L); } @Override - public long getBookiePendingRequests(BookieSocketAddress bookieSocketAddress) { + public long getBookiePendingRequests(BookieId bookieSocketAddress) { return bookiePendingRequests.getOrDefault(bookieSocketAddress, 0L); } }; @@ -152,14 +155,15 @@ public void testNotEnoughRWBookies() throws Exception { newConf.setDesiredNumZonesPerWriteQuorum(1); newConf.setMinNumZonesPerWriteQuorum(1); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); try { @@ -170,9 +174,9 @@ public void testNotEnoughRWBookies() throws Exception { // expected to get BKNotEnoughBookiesException } - roAddrs.add(addr4); - roAddrs.add(addr5); - roAddrs.add(addr6); + roAddrs.add(addr4.toBookieId()); + roAddrs.add(addr5.toBookieId()); + roAddrs.add(addr6.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); try { // only 3 rw bookies are available @@ -183,9 +187,9 @@ public void testNotEnoughRWBookies() throws Exception { } rwAddrs.clear(); - roAddrs.add(addr1); - roAddrs.add(addr2); - roAddrs.add(addr3); + roAddrs.add(addr1.toBookieId()); + roAddrs.add(addr2.toBookieId()); + roAddrs.add(addr3.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); try { // no rw bookie is available @@ -217,25 +221,26 @@ public void testEnoughRWBookies() throws Exception { newConf.setDesiredNumZonesPerWriteQuorum(4); newConf.setMinNumZonesPerWriteQuorum(2); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr6); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr6.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); /* * there are enough bookies so newEnsemble should succeed. */ - PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(6, 3, 2, null, + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(6, 3, 2, null, new HashSet<>()); - Set newEnsembleSet = new HashSet( + Set newEnsembleSet = new HashSet( newEnsemblePlacementResult.getResult()); assertTrue("New ensemble should contain all 6 rw bookies", newEnsembleSet.containsAll(rwAddrs)); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, @@ -245,7 +250,7 @@ public void testEnoughRWBookies() throws Exception { * there are enough bookies so newEnsemble should succeed. */ newEnsemblePlacementResult = zepp.newEnsemble(3, 3, 2, null, new HashSet<>()); - newEnsembleSet = new HashSet(newEnsemblePlacementResult.getResult()); + newEnsembleSet = new HashSet(newEnsemblePlacementResult.getResult()); assertTrue("New ensemble should contain 3 rw bookies", (newEnsembleSet.size() == 3) && (rwAddrs.containsAll(newEnsembleSet))); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, @@ -267,26 +272,27 @@ public void testWithDefaultBookies() throws Exception { BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); - Set bookiesInDefaultFaultDomain = new HashSet(); - bookiesInDefaultFaultDomain.add(addr5); - bookiesInDefaultFaultDomain.add(addr6); - bookiesInDefaultFaultDomain.add(addr7); + Set bookiesInDefaultFaultDomain = new HashSet(); + bookiesInDefaultFaultDomain.add(addr5.toBookieId()); + bookiesInDefaultFaultDomain.add(addr6.toBookieId()); + bookiesInDefaultFaultDomain.add(addr7.toBookieId()); ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); newConf.setDesiredNumZonesPerWriteQuorum(4); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr6); - rwAddrs.add(addr7); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr6.toBookieId()); + rwAddrs.add(addr7.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); for (int i = 0; i < 3; i++) { @@ -294,15 +300,15 @@ public void testWithDefaultBookies() throws Exception { * make sure bookies from DEFAULT_ZONE_AND_UPGRADEDOMAIN are not * part of the new ensemble created. */ - PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(4, 4, 2, null, + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(4, 4, 2, null, new HashSet<>()); - Set newEnsembleSet = new HashSet( + Set newEnsembleSet = new HashSet( newEnsemblePlacementResult.getResult()); assertTrue("Bookie from default faultDomain shouldn't be part of ensemble", Collections.disjoint(newEnsembleSet, bookiesInDefaultFaultDomain)); newEnsemblePlacementResult = zepp.newEnsemble(3, 3, 2, null, new HashSet<>()); - newEnsembleSet = new HashSet(newEnsemblePlacementResult.getResult()); + newEnsembleSet = new HashSet(newEnsemblePlacementResult.getResult()); assertTrue("Bookie from default faultDomain shouldn't be part of ensemble", Collections.disjoint(newEnsembleSet, bookiesInDefaultFaultDomain)); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, @@ -339,30 +345,31 @@ public void testMinZonesPerWriteQuorum() throws Exception { newConf.setDesiredNumZonesPerWriteQuorum(4); newConf.setMinNumZonesPerWriteQuorum(3); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - Set bookiesInDefaultFaultDomain = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr6); - rwAddrs.add(addr9); - rwAddrs.add(addr10); - roAddrs.add(addr7); - roAddrs.add(addr8); - bookiesInDefaultFaultDomain.add(addr9); - bookiesInDefaultFaultDomain.add(addr10); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + Set bookiesInDefaultFaultDomain = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr6.toBookieId()); + rwAddrs.add(addr9.toBookieId()); + rwAddrs.add(addr10.toBookieId()); + roAddrs.add(addr7.toBookieId()); + roAddrs.add(addr8.toBookieId()); + bookiesInDefaultFaultDomain.add(addr9.toBookieId()); + bookiesInDefaultFaultDomain.add(addr10.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); - PlacementResult> newEnsemblePlacementResult; + PlacementResult> newEnsemblePlacementResult; newEnsemblePlacementResult = zepp.newEnsemble(4, 4, 2, null, new HashSet<>()); - Set newEnsembleSet = new HashSet( + Set newEnsembleSet = new HashSet( newEnsemblePlacementResult.getResult()); assertTrue("New ensemble should contain all 6 rw bookies in non-default fault domains", rwAddrs.containsAll(newEnsembleSet) && (newEnsembleSet.size() == 4)); @@ -387,7 +394,8 @@ public void testMinZonesPerWriteQuorum() throws Exception { newConf.setMinNumZonesPerWriteQuorum(3); newConf.setEnforceStrictZoneawarePlacement(false); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); zepp.onClusterChanged(rwAddrs, roAddrs); @@ -396,7 +404,7 @@ public void testMinZonesPerWriteQuorum() throws Exception { * limitations on eligible values of ensembleSize and writeQuorumSize. */ newEnsemblePlacementResult = zepp.newEnsemble(4, 3, 2, null, new HashSet<>()); - newEnsembleSet = new HashSet(newEnsemblePlacementResult.getResult()); + newEnsembleSet = new HashSet(newEnsemblePlacementResult.getResult()); assertTrue("New ensemble should contain 4 different bookies", newEnsembleSet.size() == 4); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, newEnsemblePlacementResult.isAdheringToPolicy()); @@ -431,29 +439,30 @@ public void testMinUDsNotAvailable() throws Exception { newConf.setDesiredNumZonesPerWriteQuorum(4); newConf.setMinNumZonesPerWriteQuorum(2); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - Set bookiesInDefaultFaultDomain = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr6); - rwAddrs.add(addr9); - rwAddrs.add(addr10); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + Set bookiesInDefaultFaultDomain = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr6.toBookieId()); + rwAddrs.add(addr9.toBookieId()); + rwAddrs.add(addr10.toBookieId()); - roAddrs.add(addr7); - roAddrs.add(addr8); + roAddrs.add(addr7.toBookieId()); + roAddrs.add(addr8.toBookieId()); - bookiesInDefaultFaultDomain.add(addr9); - bookiesInDefaultFaultDomain.add(addr10); + bookiesInDefaultFaultDomain.add(addr9.toBookieId()); + bookiesInDefaultFaultDomain.add(addr10.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); - PlacementResult> newEnsemblePlacementResult; + PlacementResult> newEnsemblePlacementResult; try { /* * since rw bookies are not spread across UDs in zones, newEnsemble @@ -474,8 +483,8 @@ public void testMinUDsNotAvailable() throws Exception { newEnsemblePlacementResult = zepp.newEnsemble(ensSize, writeQuorum, 2, null, new HashSet<>()); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, newEnsemblePlacementResult.isAdheringToPolicy()); - List newEnsemble = newEnsemblePlacementResult.getResult(); - Set newEnsembleSet = new HashSet(newEnsemble); + List newEnsemble = newEnsemblePlacementResult.getResult(); + Set newEnsembleSet = new HashSet(newEnsemble); assertTrue("New ensemble should contain all 6 rw bookies in non-default fault domains", rwAddrs.containsAll(newEnsembleSet) && (newEnsembleSet.size() == 6)); assertTrue("Bookie from default faultDomain shouldn't be part of ensemble", @@ -526,40 +535,41 @@ public void testUniqueUds() throws Exception { newConf.setDesiredNumZonesPerWriteQuorum(4); newConf.setMinNumZonesPerWriteQuorum(2); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr6); - rwAddrs.add(addr7); - rwAddrs.add(addr8); - rwAddrs.add(addr9); - rwAddrs.add(addr10); - rwAddrs.add(addr11); - rwAddrs.add(addr12); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr6.toBookieId()); + rwAddrs.add(addr7.toBookieId()); + rwAddrs.add(addr8.toBookieId()); + rwAddrs.add(addr9.toBookieId()); + rwAddrs.add(addr10.toBookieId()); + rwAddrs.add(addr11.toBookieId()); + rwAddrs.add(addr12.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); /* * Since there are enough bookies in different UDs in 2 zones * (MinNumZonesPerWriteQuorum), new ensemble should succeed. */ - PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(6, 6, 2, null, + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(6, 6, 2, null, new HashSet<>()); - List newEnsembleList = newEnsemblePlacementResult.getResult(); - Set newEnsembleSet = new HashSet(newEnsembleList); + List newEnsembleList = newEnsemblePlacementResult.getResult(); + Set newEnsembleSet = new HashSet(newEnsembleList); assertTrue("New ensemble should contain 6 rw bookies in non-default fault domains", rwAddrs.containsAll(newEnsembleSet) && (newEnsembleSet.size() == 6)); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, newEnsemblePlacementResult.isAdheringToPolicy()); Set bookiesNetworkLocations = new HashSet(); - for (BookieSocketAddress bookieAddr : newEnsembleSet) { + for (BookieId bookieAddr : newEnsembleSet) { bookiesNetworkLocations.add(zepp.resolveNetworkLocation(bookieAddr)); } /* @@ -569,7 +579,7 @@ public void testUniqueUds() throws Exception { assertTrue("Bookies should be from different UpgradeDomains if they belong to same zone", (bookiesNetworkLocations.size() == 6)); List bookiesNodeLocationList = new ArrayList(); - for (BookieSocketAddress bookieAddr : newEnsembleList) { + for (BookieId bookieAddr : newEnsembleList) { bookiesNodeLocationList.add(zepp.getZoneAwareNodeLocation(bookieAddr)); } for (int i = 0; i < 5; i++) { @@ -614,46 +624,47 @@ public void testNewBookieUniformDistributionWithMinZoneAndMinUDs() throws Except StaticDNSResolver.addNodeToRack(addr13.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); StaticDNSResolver.addNodeToRack(addr14.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr6); - rwAddrs.add(addr7); - rwAddrs.add(addr8); - rwAddrs.add(addr9); - rwAddrs.add(addr10); - rwAddrs.add(addr11); - rwAddrs.add(addr12); - rwAddrs.add(addr13); - rwAddrs.add(addr14); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr6.toBookieId()); + rwAddrs.add(addr7.toBookieId()); + rwAddrs.add(addr8.toBookieId()); + rwAddrs.add(addr9.toBookieId()); + rwAddrs.add(addr10.toBookieId()); + rwAddrs.add(addr11.toBookieId()); + rwAddrs.add(addr12.toBookieId()); + rwAddrs.add(addr13.toBookieId()); + rwAddrs.add(addr14.toBookieId()); int minNumZonesPerWriteQuorum = 3; ClientConfiguration newConf = (ClientConfiguration) this.conf.clone(); newConf.setDesiredNumZonesPerWriteQuorum(5); newConf.setMinNumZonesPerWriteQuorum(minNumZonesPerWriteQuorum); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); zepp.onClusterChanged(rwAddrs, roAddrs); - Set excludedBookies = new HashSet(); + Set excludedBookies = new HashSet(); - PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(6, 6, 4, null, + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(6, 6, 4, null, excludedBookies); - List newEnsembleList = newEnsemblePlacementResult.getResult(); + List newEnsembleList = newEnsemblePlacementResult.getResult(); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, newEnsemblePlacementResult.isAdheringToPolicy()); - Set newEnsembleSet = new HashSet(newEnsembleList); + Set newEnsembleSet = new HashSet(newEnsembleList); Set bookiesNetworkLocationsSet = new HashSet(); List bookiesNodeLocationList = new ArrayList(); - for (BookieSocketAddress bookieAddr : newEnsembleSet) { + for (BookieId bookieAddr : newEnsembleSet) { bookiesNetworkLocationsSet.add(zepp.resolveNetworkLocation(bookieAddr)); } - for (BookieSocketAddress bookieAddr : newEnsembleList) { + for (BookieId bookieAddr : newEnsembleList) { bookiesNodeLocationList.add(zepp.getZoneAwareNodeLocation(bookieAddr)); } /* @@ -712,62 +723,66 @@ public void testReplaceBookie() throws Exception { newConf.setDesiredNumZonesPerWriteQuorum(3); newConf.setMinNumZonesPerWriteQuorum(3); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr6); - rwAddrs.add(addr7); - rwAddrs.add(addr8); - rwAddrs.add(addr9); - rwAddrs.add(addr10); - rwAddrs.add(addr11); - rwAddrs.add(addr12); - rwAddrs.add(addr13); - rwAddrs.add(addr14); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr6.toBookieId()); + rwAddrs.add(addr7.toBookieId()); + rwAddrs.add(addr8.toBookieId()); + rwAddrs.add(addr9.toBookieId()); + rwAddrs.add(addr10.toBookieId()); + rwAddrs.add(addr11.toBookieId()); + rwAddrs.add(addr12.toBookieId()); + rwAddrs.add(addr13.toBookieId()); + rwAddrs.add(addr14.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); - List ensemble = new ArrayList(); - Set excludedBookies = new HashSet(); - ensemble.add(addr1); - ensemble.add(addr5); - ensemble.add(addr9); - ensemble.add(addr3); - ensemble.add(addr7); - ensemble.add(addr11); + List ensemble = new ArrayList(); + Set excludedBookies = new HashSet(); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr5.toBookieId()); + ensemble.add(addr9.toBookieId()); + ensemble.add(addr3.toBookieId()); + ensemble.add(addr7.toBookieId()); + ensemble.add(addr11.toBookieId()); /* * since addr5 (/zone2/ud1) is already part of ensemble of size 6, write * quorum of size 6, to replace bookie addr7 (/zone2/ud2), new bookie * should be from /zone2/ud2. */ - PlacementResult replacePlacementResult = zepp.replaceBookie(6, 6, 2, null, ensemble, addr7, + PlacementResult replacePlacementResult = zepp.replaceBookie(6, 6, 2, null, ensemble, + addr7.toBookieId(), excludedBookies); - BookieSocketAddress replacedBookie = replacePlacementResult.getResult(); - assertEquals("replaced bookie", addr8, replacedBookie); + BookieId replacedBookie = replacePlacementResult.getResult(); + assertEquals("replaced bookie", addr8.toBookieId(), replacedBookie); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, replacePlacementResult.isAdheringToPolicy()); - excludedBookies.add(addr8); + excludedBookies.add(addr8.toBookieId()); /* * here addr8 is excluded, and writeQuorumSize is 3. So to replace * bookie addr7, addr6 (belonging to same zone) is the candidate. */ - replacePlacementResult = zepp.replaceBookie(6, 3, 2, null, ensemble, addr7, excludedBookies); + replacePlacementResult = zepp.replaceBookie(6, 3, 2, null, ensemble, addr7.toBookieId(), + excludedBookies); replacedBookie = replacePlacementResult.getResult(); - assertEquals("replaced bookie", addr6, replacedBookie); + assertEquals("replaced bookie", addr6.toBookieId(), replacedBookie); - excludedBookies.add(addr6); + excludedBookies.add(addr6.toBookieId()); try { /* * here addr6 is also excluded, so replaceBookie should fail. */ - replacedBookie = zepp.replaceBookie(6, 3, 2, null, ensemble, addr7, excludedBookies).getResult(); + replacedBookie = zepp.replaceBookie(6, 3, 2, null, ensemble, addr7.toBookieId(), excludedBookies) + .getResult(); fail("Expected BKNotEnoughBookiesException for replaceBookie with added excludedBookies"); } catch (BKException.BKNotEnoughBookiesException bkne) { // expected NotEnoughBookiesException @@ -805,40 +820,41 @@ public void testReplaceBookieMinUDs() throws Exception { newConf.setDesiredNumZonesPerWriteQuorum(4); newConf.setMinNumZonesPerWriteQuorum(3); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr6); - rwAddrs.add(addr7); - rwAddrs.add(addr8); - rwAddrs.add(addr9); - rwAddrs.add(addr10); - rwAddrs.add(addr11); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr6.toBookieId()); + rwAddrs.add(addr7.toBookieId()); + rwAddrs.add(addr8.toBookieId()); + rwAddrs.add(addr9.toBookieId()); + rwAddrs.add(addr10.toBookieId()); + rwAddrs.add(addr11.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); - List ensemble = new ArrayList(); - Set excludedBookies = new HashSet(); - ensemble.add(addr1); - ensemble.add(addr2); - ensemble.add(addr3); - ensemble.add(addr4); - ensemble.add(addr5); - ensemble.add(addr6); + List ensemble = new ArrayList(); + Set excludedBookies = new HashSet(); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr3.toBookieId()); + ensemble.add(addr4.toBookieId()); + ensemble.add(addr5.toBookieId()); + ensemble.add(addr6.toBookieId()); /* * though all the remaining non-default bookies are in /zone3/ud2, for * replacing addr4 replaceBookie should be able to find some other * bookie in /zone3/ud2. */ - PlacementResult replaceResponse = zepp.replaceBookie(6, 6, 2, null, ensemble, addr4, + PlacementResult replaceResponse = zepp.replaceBookie(6, 6, 2, null, ensemble, addr4.toBookieId(), excludedBookies); - BookieSocketAddress replacedBookie = replaceResponse.getResult(); + BookieId replacedBookie = replaceResponse.getResult(); assertEquals("replaced bookie", "/zone3/ud2", zepp.resolveNetworkLocation(replacedBookie)); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, replaceResponse.isAdheringToPolicy()); @@ -871,30 +887,31 @@ public void testAreAckedBookiesAdheringToPlacementPolicy() throws Exception { newConf.setDesiredNumZonesPerWriteQuorum(4); newConf.setMinNumZonesPerWriteQuorum(2); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr6); - rwAddrs.add(addr7); - rwAddrs.add(addr8); - rwAddrs.add(addr9); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr6.toBookieId()); + rwAddrs.add(addr7.toBookieId()); + rwAddrs.add(addr8.toBookieId()); + rwAddrs.add(addr9.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); - Set ackedBookies = new HashSet(); - ackedBookies.add(addr1); - ackedBookies.add(addr4); + Set ackedBookies = new HashSet(); + ackedBookies.add(addr1.toBookieId()); + ackedBookies.add(addr4.toBookieId()); assertFalse("since both the bookies are in the same zone, it should return false", zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 10, 2)); ackedBookies.clear(); - ackedBookies.add(addr1); - ackedBookies.add(addr2); + ackedBookies.add(addr1.toBookieId()); + ackedBookies.add(addr2.toBookieId()); assertFalse("since ackQuorumSize is 3, it should return false", zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 10, 3)); assertTrue("since ackQuorumSize is 2 and bookies are from minNumZonesPerWriteQuorum it should return true", @@ -905,22 +922,23 @@ public void testAreAckedBookiesAdheringToPlacementPolicy() throws Exception { newConf.setDesiredNumZonesPerWriteQuorum(4); newConf.setMinNumZonesPerWriteQuorum(4); zepp = new ZoneawareEnsemblePlacementPolicy(); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); zepp.onClusterChanged(rwAddrs, roAddrs); ackedBookies.clear(); - ackedBookies.add(addr1); - ackedBookies.add(addr2); - ackedBookies.add(addr3); + ackedBookies.add(addr1.toBookieId()); + ackedBookies.add(addr2.toBookieId()); + ackedBookies.add(addr3.toBookieId()); assertFalse("since minNumZonesPerWriteQuorum is set to 4, it should return false", zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 4, 3)); assertTrue("since writeQuorumSize is set to 3, it should return true", zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 3, 3)); ackedBookies.clear(); - ackedBookies.add(addr1); - ackedBookies.add(addr2); - ackedBookies.add(addr4); + ackedBookies.add(addr1.toBookieId()); + ackedBookies.add(addr2.toBookieId()); + ackedBookies.add(addr4.toBookieId()); assertFalse("since bookies are in just 2 zones but not in 3 zones, it should return false", zepp.areAckedBookiesAdheringToPlacementPolicy(ackedBookies, 3, 3)); } @@ -940,12 +958,12 @@ public void testWeightedPlacement() throws Exception { StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/zone2/ud2"); StaticDNSResolver.addNodeToRack(addr5.getHostName(), NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); // Update cluster - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - addrs.add(addr5); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + addrs.add(addr5.toBookieId()); int multiple = 10; @@ -958,32 +976,34 @@ public void testWeightedPlacement() throws Exception { */ newConf.setBookieMaxWeightMultipleForWeightBasedPlacement(-1); newConf.setMinNumZonesPerWriteQuorum(0); - zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(newConf, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - zepp.onClusterChanged(addrs, new HashSet()); - Map bookieInfoMap = new HashMap(); - bookieInfoMap.put(addr1, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr2, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr3, new BookieInfo(100L, 100L)); - bookieInfoMap.put(addr4, new BookieInfo(multiple * 100L, multiple * 100L)); - bookieInfoMap.put(addr5, new BookieInfo(100L, 100L)); + zepp.onClusterChanged(addrs, new HashSet()); + Map bookieInfoMap = new HashMap(); + bookieInfoMap.put(addr1.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr2.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr3.toBookieId(), new BookieInfo(100L, 100L)); + bookieInfoMap.put(addr4.toBookieId(), new BookieInfo(multiple * 100L, multiple * 100L)); + bookieInfoMap.put(addr5.toBookieId(), new BookieInfo(100L, 100L)); zepp.updateBookieInfo(bookieInfoMap); - Map selectionCounts = new HashMap(); + Map selectionCounts = new HashMap(); int numTries = 50000; - EnsemblePlacementPolicy.PlacementResult> newEnsembleResponse; - List newEnsemble; - for (BookieSocketAddress addr : addrs) { + EnsemblePlacementPolicy.PlacementResult> newEnsembleResponse; + List newEnsemble; + for (BookieId addr : addrs) { selectionCounts.put(addr, (long) 0); } for (int i = 0; i < numTries; i++) { // new ensemble response - newEnsembleResponse = zepp.newEnsemble(1, 1, 1, null, new HashSet()); + newEnsembleResponse = zepp.newEnsemble(1, 1, 1, null, new HashSet()); newEnsemble = newEnsembleResponse.getResult(); selectionCounts.put(newEnsemble.get(0), selectionCounts.get(newEnsemble.get(0)) + 1); } - double observedMultiple = ((double) selectionCounts.get(addr4) / (double) selectionCounts.get(addr3)); + double observedMultiple = ((double) selectionCounts.get(addr4.toBookieId()) + / (double) selectionCounts.get(addr3.toBookieId())); /* * since there is no cap on maxWeight, observedMultiple should be * roughly equal to multiple @@ -991,25 +1011,28 @@ public void testWeightedPlacement() throws Exception { assertTrue("Weights not being honored " + observedMultiple, Math.abs(observedMultiple - multiple) < 1); selectionCounts.clear(); - selectionCounts.put(addr3, (long) 0); - selectionCounts.put(addr4, (long) 0); - newEnsemble = new ArrayList(); - newEnsemble.add(addr2); - Set excludedBookies = new HashSet(); - excludedBookies.add(addr1); - EnsemblePlacementPolicy.PlacementResult replacedBookieResponse; - BookieSocketAddress replacedBookie; + selectionCounts.put(addr3.toBookieId(), (long) 0); + selectionCounts.put(addr4.toBookieId(), (long) 0); + newEnsemble = new ArrayList(); + newEnsemble.add(addr2.toBookieId()); + Set excludedBookies = new HashSet(); + excludedBookies.add(addr1.toBookieId()); + EnsemblePlacementPolicy.PlacementResult replacedBookieResponse; + BookieId replacedBookie; for (int i = 0; i < numTries; i++) { // replace bookie response - replacedBookieResponse = zepp.replaceBookie(1, 1, 1, null, newEnsemble, addr2, excludedBookies); + replacedBookieResponse = zepp.replaceBookie(1, 1, 1, null, newEnsemble, addr2.toBookieId(), + excludedBookies); replacedBookie = replacedBookieResponse.getResult(); /* * only addr3 and addr4 are eligible for replacedBookie. */ - assertTrue("replaced : " + replacedBookie, addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); + assertTrue("replaced : " + replacedBookie, addr3.toBookieId().equals(replacedBookie) + || addr4.toBookieId().equals(replacedBookie)); selectionCounts.put(replacedBookie, selectionCounts.get(replacedBookie) + 1); } - observedMultiple = ((double) selectionCounts.get(addr4) / (double) selectionCounts.get(addr3)); + observedMultiple = ((double) selectionCounts.get(addr4.toBookieId()) + / (double) selectionCounts.get(addr3.toBookieId())); /* * since there is no cap on maxWeight, observedMultiple should be * roughly equal to multiple @@ -1032,36 +1055,37 @@ public void testPlacementOnStabilizeNetworkTopology() throws Exception { ClientConfiguration confLocal = new ClientConfiguration(); confLocal.addConfiguration(conf); confLocal.setNetworkTopologyStabilizePeriodSeconds(99999); - zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set addrs = new HashSet(); - addrs.add(addr1); - addrs.add(addr2); - addrs.add(addr3); - addrs.add(addr4); - zepp.onClusterChanged(addrs, new HashSet()); + Set addrs = new HashSet(); + addrs.add(addr1.toBookieId()); + addrs.add(addr2.toBookieId()); + addrs.add(addr3.toBookieId()); + addrs.add(addr4.toBookieId()); + zepp.onClusterChanged(addrs, new HashSet()); // addr4 left - addrs.remove(addr4); - Set deadBookies = zepp.onClusterChanged(addrs, new HashSet()); + addrs.remove(addr4.toBookieId()); + Set deadBookies = zepp.onClusterChanged(addrs, new HashSet()); assertTrue(deadBookies.isEmpty()); // we will never use addr4 even it is in the stabilized network topology for (int i = 0; i < 5; i++) { - EnsemblePlacementPolicy.PlacementResult> ensembleResponse = zepp.newEnsemble(3, 3, - 2, null, new HashSet()); - List ensemble = ensembleResponse.getResult(); - assertFalse(ensemble.contains(addr4)); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = zepp.newEnsemble(3, 3, + 2, null, new HashSet()); + List ensemble = ensembleResponse.getResult(); + assertFalse(ensemble.contains(addr4.toBookieId())); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, ensembleResponse.isAdheringToPolicy()); } // we could still use addr4 for urgent allocation if it is just bookie // flapping - EnsemblePlacementPolicy.PlacementResult> ensembleResponse = zepp.newEnsemble(4, 4, 2, - null, new HashSet()); - List ensemble = ensembleResponse.getResult(); - assertTrue(ensemble.contains(addr4)); + EnsemblePlacementPolicy.PlacementResult> ensembleResponse = zepp.newEnsemble(4, 4, 2, + null, new HashSet()); + List ensemble = ensembleResponse.getResult(); + assertTrue(ensemble.contains(addr4.toBookieId())); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, ensembleResponse.isAdheringToPolicy()); } @@ -1087,35 +1111,37 @@ public void testCreateNewEnsembleRandomly() throws Exception { confLocal.setEnforceStrictZoneawarePlacement(false); confLocal.setMinNumZonesPerWriteQuorum(3); confLocal.setDesiredNumZonesPerWriteQuorum(4); - zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - Set excludeBookies = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - excludeBookies.add(addr5); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + Set excludeBookies = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + excludeBookies.add(addr5.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); /* * if enforceStrictZoneawarePlacement is not enabled, then there is no * restrictions on ensSize and writeQSize and also bookie belonging to * DEFAULT_ZONE_AND_UPGRADEDOMAIN can be a candidate. */ - PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(4, 3, 2, null, + PlacementResult> newEnsemblePlacementResult = zepp.newEnsemble(4, 3, 2, null, excludeBookies); - Set newEnsembleSet = new HashSet( + Set newEnsembleSet = new HashSet( newEnsemblePlacementResult.getResult()); assertEquals("New ensemble should contain 4 rw bookies", 4, newEnsembleSet.size()); - assertFalse("excludeBookie should not be included in the ensemble", newEnsembleSet.contains(addr5)); + assertFalse("excludeBookie should not be included in the ensemble", + newEnsembleSet.contains(addr5.toBookieId())); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, newEnsemblePlacementResult.isAdheringToPolicy()); - rwAddrs.remove(addr4); - roAddrs.add(addr4); + rwAddrs.remove(addr4.toBookieId()); + roAddrs.add(addr4.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); try { /* @@ -1153,48 +1179,49 @@ public void testReplaceBookieRandomly() throws Exception { confLocal.setEnforceStrictZoneawarePlacement(false); confLocal.setMinNumZonesPerWriteQuorum(3); confLocal.setDesiredNumZonesPerWriteQuorum(4); - zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - Set rwAddrs = new HashSet(); - Set roAddrs = new HashSet(); - Set excludeBookies = new HashSet(); - rwAddrs.add(addr1); - rwAddrs.add(addr2); - rwAddrs.add(addr3); - rwAddrs.add(addr4); - rwAddrs.add(addr5); - rwAddrs.add(addr7); - - roAddrs.add(addr6); - excludeBookies.add(addr5); + Set rwAddrs = new HashSet(); + Set roAddrs = new HashSet(); + Set excludeBookies = new HashSet(); + rwAddrs.add(addr1.toBookieId()); + rwAddrs.add(addr2.toBookieId()); + rwAddrs.add(addr3.toBookieId()); + rwAddrs.add(addr4.toBookieId()); + rwAddrs.add(addr5.toBookieId()); + rwAddrs.add(addr7.toBookieId()); + + roAddrs.add(addr6.toBookieId()); + excludeBookies.add(addr5.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); - List ensembleList = new ArrayList(); - ensembleList.add(addr1); - ensembleList.add(addr2); - ensembleList.add(addr3); - ensembleList.add(addr4); + List ensembleList = new ArrayList(); + ensembleList.add(addr1.toBookieId()); + ensembleList.add(addr2.toBookieId()); + ensembleList.add(addr3.toBookieId()); + ensembleList.add(addr4.toBookieId()); - PlacementResult replaceResponse = zepp.replaceBookie(4, 3, 2, null, ensembleList, addr3, + PlacementResult replaceResponse = zepp.replaceBookie(4, 3, 2, null, ensembleList, addr3.toBookieId(), excludeBookies); - BookieSocketAddress replaceBookie = replaceResponse.getResult(); + BookieId replaceBookie = replaceResponse.getResult(); /* * if enforceStrictZoneawarePlacement is not enabled, then there is no * restrictions on ensSize and writeQSize and also bookie belonging to * DEFAULT_ZONE_AND_UPGRADEDOMAIN can be a candidate. */ - assertEquals("ReplaceBookie candidate", addr7, replaceBookie); + assertEquals("ReplaceBookie candidate", addr7.toBookieId(), replaceBookie); assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, replaceResponse.isAdheringToPolicy()); - rwAddrs.remove(addr7); - excludeBookies.add(addr7); + rwAddrs.remove(addr7.toBookieId()); + excludeBookies.add(addr7.toBookieId()); zepp.onClusterChanged(rwAddrs, roAddrs); try { /* * since there is no bookie available, replaceBookie should fail. */ - zepp.replaceBookie(4, 3, 2, null, ensembleList, addr3, excludeBookies); + zepp.replaceBookie(4, 3, 2, null, ensembleList, addr3.toBookieId(), excludeBookies); fail("ReplaceBookie should fail because of unavailable bookies"); } catch (BKException.BKNotEnoughBookiesException bkne) { // expected NotEnoughBookiesException @@ -1232,76 +1259,77 @@ public void testIsEnsembleAdheringToPlacementPolicy() throws Exception { confLocal.setEnforceStrictZoneawarePlacement(true); confLocal.setMinNumZonesPerWriteQuorum(2); confLocal.setDesiredNumZonesPerWriteQuorum(3); - zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + zepp.initialize(confLocal, Optional. empty(), timer, DISABLE_ALL, + NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); zepp.withDefaultFaultDomain(NetworkTopology.DEFAULT_ZONE_AND_UPGRADEDOMAIN); - List ensemble = new ArrayList(); - ensemble.add(addr1); - ensemble.add(addr2); - ensemble.add(addr3); + List ensemble = new ArrayList(); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr3.toBookieId()); // all bookies in same rack assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); ensemble.clear(); - ensemble.add(addr1); - ensemble.add(addr2); - ensemble.add(addr4); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr4.toBookieId()); // bookies spread across minZones assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_SOFT, zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); ensemble.clear(); - ensemble.add(addr1); - ensemble.add(addr4); - ensemble.add(addr7); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr4.toBookieId()); + ensemble.add(addr7.toBookieId()); // bookies spread across desirednumofzones assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); ensemble.clear(); - ensemble.add(addr1); - ensemble.add(addr4); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr4.toBookieId()); // writeQuorum should be greater than minZones assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 2, 2)); ensemble.clear(); - ensemble.add(addr2); - ensemble.add(addr3); - ensemble.add(addr4); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr3.toBookieId()); + ensemble.add(addr4.toBookieId()); // bookies from zone1 (addr2 and addr3) are in same UD assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); ensemble.clear(); - ensemble.add(addr1); - ensemble.add(addr4); - ensemble.add(addr7); - ensemble.add(addr10); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr4.toBookieId()); + ensemble.add(addr7.toBookieId()); + ensemble.add(addr10.toBookieId()); // bookie from default faultdomain will cause PlacementPolicyAdherence // to fail assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.FAIL, zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 4, 2)); ensemble.clear(); - ensemble.add(addr1); - ensemble.add(addr4); - ensemble.add(addr7); - ensemble.add(addr8); - ensemble.add(addr9); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr4.toBookieId()); + ensemble.add(addr7.toBookieId()); + ensemble.add(addr8.toBookieId()); + ensemble.add(addr9.toBookieId()); // bookies are spread across desired zones and bookie from same zone are // spread across 2 UDs assertEquals("PlacementPolicyAdherence", PlacementPolicyAdherence.MEETS_STRICT, zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 5, 2)); ensemble.clear(); - ensemble.add(addr1); - ensemble.add(addr4); - ensemble.add(addr7); - ensemble.add(addr2); - ensemble.add(addr8); - ensemble.add(addr9); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr4.toBookieId()); + ensemble.add(addr7.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr8.toBookieId()); + ensemble.add(addr9.toBookieId()); /* * writeset of addr2, addr8 and addr9 fails, because addr8 and addr9 * belong to z3u2 @@ -1310,12 +1338,12 @@ public void testIsEnsembleAdheringToPlacementPolicy() throws Exception { zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); ensemble.clear(); - ensemble.add(addr1); - ensemble.add(addr4); - ensemble.add(addr9); - ensemble.add(addr2); - ensemble.add(addr8); - ensemble.add(addr7); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr4.toBookieId()); + ensemble.add(addr9.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr8.toBookieId()); + ensemble.add(addr7.toBookieId()); /* * writeset of addr9, addr2 and addr8 fails, because addr8 and addr9 * belong to z3u2 @@ -1324,12 +1352,12 @@ public void testIsEnsembleAdheringToPlacementPolicy() throws Exception { zepp.isEnsembleAdheringToPlacementPolicy(ensemble, 3, 2)); ensemble.clear(); - ensemble.add(addr1); - ensemble.add(addr4); - ensemble.add(addr9); - ensemble.add(addr2); - ensemble.add(addr7); - ensemble.add(addr8); + ensemble.add(addr1.toBookieId()); + ensemble.add(addr4.toBookieId()); + ensemble.add(addr9.toBookieId()); + ensemble.add(addr2.toBookieId()); + ensemble.add(addr7.toBookieId()); + ensemble.add(addr8.toBookieId()); /* * writeset of addr2, addr7 and addr8 just meets soft. */ diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java index f74adc6857a..1a849ae828b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java @@ -35,6 +35,7 @@ import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.zookeeper.KeeperException; @@ -74,8 +75,8 @@ public void testUpdateLedgersToHostname() throws Exception { final ServerConfiguration conf = bsConfs.get(0); conf.setUseHostNameAsBookieID(true); BookieSocketAddress toBookieId = Bookie.getBookieAddress(conf); - BookieSocketAddress toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":" - + conf.getBookiePort()); + BookieId toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":" + + conf.getBookiePort()).toBookieId(); updateLedgerCmd(argv, 0, conf); @@ -96,8 +97,8 @@ public void testUpdateBookieInLedger() throws Exception { for (int i = 1; i < 40; i++) { ledgers.add(createLedgerWithEntries(bk, 0)); } - BookieSocketAddress srcBookie = bs.get(0).getLocalAddress(); - BookieSocketAddress destBookie = new BookieSocketAddress("1.1.1.1", 2181); + BookieId srcBookie = bs.get(0).getBookieId(); + BookieId destBookie = new BookieSocketAddress("1.1.1.1", 2181).toBookieId(); String[] argv = new String[] { "updateBookieInLedger", "-sb", srcBookie.toString(), "-db", destBookie.toString(), "-v", "true", "-p", "2" }; final ServerConfiguration conf = bsConfs.get(0); @@ -118,9 +119,9 @@ private void updateLedgerCmd(String[] argv, int exitCode, ServerConfiguration co assertEquals("Failed to return exit code!", exitCode, bkShell.run(argv)); } - private int getUpdatedLedgersCount(BookKeeper bk, List ledgers, BookieSocketAddress toBookieAddr) + private int getUpdatedLedgersCount(BookKeeper bk, List ledgers, BookieId toBookieAddr) throws InterruptedException, BKException { - List ensemble; + List ensemble; int updatedLedgersCount = 0; for (LedgerHandle lh : ledgers) { lh.close(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java index 5ead2c972a9..6c08ddd4b59 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java @@ -37,6 +37,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -99,16 +100,17 @@ public void testManyLedgers(boolean useShortHostName) throws Exception { ledgers.add(createLedgerWithEntries(bk, 0)); } - List ensemble = lh1.getLedgerMetadata().getEnsembleAt(0); + List ensemble = lh1.getLedgerMetadata().getEnsembleAt(0); - BookieSocketAddress curBookieAddr = ensemble.get(0); + BookieSocketAddress curBookieAddr = bk.getBookieAddressResolver().resolve(ensemble.get(0)); baseConf.setUseHostNameAsBookieID(true); baseConf.setUseShortHostName(useShortHostName); BookieSocketAddress curBookieId = Bookie.getBookieAddress(baseConf); - BookieSocketAddress toBookieAddr = new BookieSocketAddress(curBookieId.getHostName() + ":" - + curBookieAddr.getPort()); + BookieId toBookieAddr = new BookieSocketAddress(curBookieId.getHostName() + ":" + + curBookieAddr.getPort()).toBookieId(); UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin); - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, 25, Integer.MIN_VALUE, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr.toBookieId(), toBookieAddr, + 5, 25, Integer.MIN_VALUE, progressable); for (LedgerHandle lh : ledgers) { lh.close(); @@ -117,7 +119,7 @@ public void testManyLedgers(boolean useShortHostName) throws Exception { assertTrue("Failed to update the ledger metadata to use bookie host name", ensemble.contains(toBookieAddr)); assertFalse("Failed to update the ledger metadata to use bookie host name", - ensemble.contains(curBookieAddr)); + ensemble.contains(curBookieAddr.toBookieId())); } } } @@ -138,13 +140,13 @@ public void testLimitLessThanTotalLedgers() throws Exception { ledgers.add(createLedgerWithEntries(bk, 0)); } - List ensemble = lh1.getLedgerMetadata().getEnsembleAt(0); + List ensemble = lh1.getLedgerMetadata().getEnsembleAt(0); - BookieSocketAddress curBookieAddr = ensemble.get(0); + BookieId curBookieAddr = ensemble.get(0); baseConf.setUseHostNameAsBookieID(true); BookieSocketAddress toBookieId = Bookie.getBookieAddress(baseConf); - BookieSocketAddress toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":" - + curBookieAddr.getPort()); + BookieId toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":" + + bk.getBookieAddressResolver().resolve(curBookieAddr).getPort()).toBookieId(); UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin); updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 7, 35, 4, progressable); int updatedLedgersCount = getUpdatedLedgersCount(bk, ledgers, toBookieAddr); @@ -194,21 +196,22 @@ public void testChangeEnsembleAfterRenaming(boolean useShortHostName) throws Exc LedgerHandle lh = createLedgerWithEntries(bk, 100); BookieServer bookieServer = bs.get(0); - List ensemble = lh.getLedgerMetadata().getEnsembleAt(0); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(0); BookieSocketAddress curBookieAddr = null; - for (BookieSocketAddress bookieSocketAddress : ensemble) { - if (bookieServer.getLocalAddress().equals(bookieSocketAddress)) { - curBookieAddr = bookieSocketAddress; + for (BookieId bookieSocketAddress : ensemble) { + BookieSocketAddress resolved = bk.getBookieAddressResolver().resolve(bookieSocketAddress); + if (bookieServer.getLocalAddress().equals(resolved)) { + curBookieAddr = resolved; } } assertNotNull("Couldn't find the bookie in ledger metadata!", curBookieAddr); baseConf.setUseHostNameAsBookieID(true); baseConf.setUseShortHostName(useShortHostName); BookieSocketAddress toBookieId = Bookie.getBookieAddress(baseConf); - BookieSocketAddress toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":" - + curBookieAddr.getPort()); + BookieId toBookieAddr = new BookieSocketAddress(toBookieId.getHostName() + ":" + + curBookieAddr.getPort()).toBookieId(); UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin); - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, 25, 100, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr.toBookieId(), toBookieAddr, 5, 25, 100, progressable); bookieServer.shutdown(); @@ -274,11 +277,11 @@ public void addComplete(int rccb, LedgerHandle lh, long entryId, Object ctx) { } }; th.start(); - List ensemble = lh.getLedgerMetadata().getEnsembleAt(0); - BookieSocketAddress curBookieAddr = ensemble.get(0); - BookieSocketAddress toBookieAddr = new BookieSocketAddress("localhost:" + curBookieAddr.getPort()); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(0); + BookieSocketAddress curBookieAddr = bk.getBookieAddressResolver().resolve(ensemble.get(0)); + BookieId toBookieAddr = BookieId.parse("localhost:" + curBookieAddr.getPort()); UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin); - updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, 25, 100, progressable); + updateLedgerOp.updateBookieIdInLedgers(curBookieAddr.toBookieId(), toBookieAddr, 5, 25, 100, progressable); if (!latch.await(120, TimeUnit.SECONDS)) { throw new Exception("Entries took too long to add"); @@ -294,9 +297,9 @@ public void addComplete(int rccb, LedgerHandle lh, long entryId, Object ctx) { } } - private int getUpdatedLedgersCount(BookKeeper bk, List ledgers, BookieSocketAddress toBookieAddr) + private int getUpdatedLedgersCount(BookKeeper bk, List ledgers, BookieId toBookieAddr) throws InterruptedException, BKException { - List ensemble; + List ensemble; int updatedLedgersCount = 0; for (LedgerHandle lh : ledgers) { lh.close(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/BookieServiceInfoTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/BookieServiceInfoTest.java index 120a2055f51..e326b3fa0b0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/BookieServiceInfoTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/BookieServiceInfoTest.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Objects; import org.apache.bookkeeper.discover.BookieServiceInfo.Endpoint; +import org.apache.bookkeeper.net.BookieId; import org.junit.Test; /** @@ -50,7 +51,7 @@ public void testSerializeDeserializeBookieServiceInfo() throws Exception { expected.setProperties(properties); byte[] serialized = serializeBookieServiceInfo(expected); - BookieServiceInfo deserialized = deserializeBookieServiceInfo(bookieId, serialized); + BookieServiceInfo deserialized = deserializeBookieServiceInfo(BookieId.parse(bookieId), serialized); assertBookieServiceInfoEquals(expected, deserialized); } @@ -58,15 +59,15 @@ public void testSerializeDeserializeBookieServiceInfo() throws Exception { @Test public void testDeserializeBookieServiceInfo() throws Exception { - String bookieId = "127.0.0.1:3181"; + BookieId bookieId = BookieId.parse("127.0.0.1:3181"); { - BookieServiceInfo expected = BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId); + BookieServiceInfo expected = BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId.toString()); BookieServiceInfo deserialized = deserializeBookieServiceInfo(bookieId, null); assertBookieServiceInfoEquals(expected, deserialized); } { - BookieServiceInfo expected = BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId); + BookieServiceInfo expected = BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId.toString()); BookieServiceInfo deserialized = deserializeBookieServiceInfo(bookieId, new byte[]{}); assertBookieServiceInfoEquals(expected, deserialized); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java index 2004a0ac209..3a0815085d5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationClient.java @@ -26,7 +26,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Versioned; @@ -38,9 +38,9 @@ public class MockRegistrationClient implements RegistrationClient { final ExecutorService executor; private long currentVersion = 0; - private Set bookies = new HashSet(); - private Set allBookies = new HashSet(); - private Set readOnlyBookies = new HashSet(); + private Set bookies = new HashSet(); + private Set allBookies = new HashSet(); + private Set readOnlyBookies = new HashSet(); private Set bookieWatchers = new HashSet(); private Set readOnlyBookieWatchers = new HashSet(); @@ -53,11 +53,11 @@ public void close() { executor.shutdownNow(); } - private static Versioned> versioned(Set bookies, long version) { + private static Versioned> versioned(Set bookies, long version) { return new Versioned<>(Collections.unmodifiableSet(bookies), new LongVersion(version)); } - public CompletableFuture addBookies(BookieSocketAddress... bookies) { + public CompletableFuture addBookies(BookieId... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; @@ -68,7 +68,7 @@ public CompletableFuture addBookies(BookieSocketAddress... bookies) { return promise; } - public CompletableFuture removeBookies(BookieSocketAddress... bookies) { + public CompletableFuture removeBookies(BookieId... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; @@ -79,7 +79,7 @@ public CompletableFuture removeBookies(BookieSocketAddress... bookies) { return promise; } - public CompletableFuture addReadOnlyBookies(BookieSocketAddress... bookies) { + public CompletableFuture addReadOnlyBookies(BookieId... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; @@ -90,7 +90,7 @@ public CompletableFuture addReadOnlyBookies(BookieSocketAddress... bookies return promise; } - public CompletableFuture removeReadOnlyBookies(BookieSocketAddress... bookies) { + public CompletableFuture removeReadOnlyBookies(BookieId... bookies) { CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { currentVersion++; @@ -102,22 +102,22 @@ public CompletableFuture removeReadOnlyBookies(BookieSocketAddress... book } @Override - public CompletableFuture>> getWritableBookies() { - CompletableFuture>> promise = new CompletableFuture<>(); + public CompletableFuture>> getWritableBookies() { + CompletableFuture>> promise = new CompletableFuture<>(); executor.submit(() -> promise.complete(versioned(bookies, currentVersion))); return promise; } @Override - public CompletableFuture>> getAllBookies() { - CompletableFuture>> promise = new CompletableFuture<>(); + public CompletableFuture>> getAllBookies() { + CompletableFuture>> promise = new CompletableFuture<>(); executor.submit(() -> promise.complete(versioned(allBookies, currentVersion))); return promise; } @Override - public CompletableFuture>> getReadOnlyBookies() { - CompletableFuture>> promise = new CompletableFuture<>(); + public CompletableFuture>> getReadOnlyBookies() { + CompletableFuture>> promise = new CompletableFuture<>(); executor.submit(() -> promise.complete(versioned(readOnlyBookies, currentVersion))); return promise; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationClient.java index 234a860364f..a8d4f1dd479 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/TestZkRegistrationClient.java @@ -56,6 +56,7 @@ import org.apache.bookkeeper.common.testing.executors.MockExecutorController; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; import org.apache.bookkeeper.discover.ZKRegistrationClient.WatchTask; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.LongVersion; @@ -124,28 +125,56 @@ public void teardown() { } } - private static Set prepareNBookies(int num) { - Set bookies = new HashSet<>(); + private static Set prepareNBookies(int num) { + Set bookies = new HashSet<>(); for (int i = 0; i < num; i++) { - bookies.add(new BookieSocketAddress("127.0.0.1", 3181 + i)); + bookies.add(new BookieSocketAddress("127.0.0.1", 3181 + i).toBookieId()); } return bookies; } + private void prepareReadBookieServiceInfo(BookieId address, boolean readonly) throws Exception { + if (readonly) { + mockZkGetData(regPath + "/" + address.toString(), + false, + Code.NONODE.intValue(), + new byte[] {}, + new Stat()); + mockZkGetData(regReadonlyPath + "/" + address.toString(), + false, + Code.OK.intValue(), + new byte[] {}, + new Stat()); + } else { + mockZkGetData(regPath + "/" + address.toString(), + false, + Code.OK.intValue(), + new byte[] {}, + new Stat()); + mockZkGetData(regReadonlyPath + "/" + address.toString(), + false, + Code.NONODE.intValue(), + new byte[] {}, + new Stat()); + } + } + @Test public void testGetWritableBookies() throws Exception { - Set addresses = prepareNBookies(10); + Set addresses = prepareNBookies(10); List children = Lists.newArrayList(); - for (BookieSocketAddress address : addresses) { + for (BookieId address : addresses) { children.add(address.toString()); + prepareReadBookieServiceInfo(address, false); } + Stat stat = mock(Stat.class); when(stat.getCversion()).thenReturn(1234); mockGetChildren( regPath, false, Code.OK.intValue(), children, stat); - Versioned> result = + Versioned> result = result(zkRegistrationClient.getWritableBookies()); assertEquals(new LongVersion(1234), result.getVersion()); @@ -155,10 +184,14 @@ public void testGetWritableBookies() throws Exception { @Test public void testGetAllBookies() throws Exception { - Set addresses = prepareNBookies(10); + Set addresses = prepareNBookies(10); List children = Lists.newArrayList(); - for (BookieSocketAddress address : addresses) { + + int i = 0; + for (BookieId address : addresses) { children.add(address.toString()); + boolean readonly = i++ % 2 == 0; + prepareReadBookieServiceInfo(address, readonly); } Stat stat = mock(Stat.class); when(stat.getCversion()).thenReturn(1234); @@ -166,7 +199,7 @@ public void testGetAllBookies() throws Exception { regAllPath, false, Code.OK.intValue(), children, stat); - Versioned> result = + Versioned> result = result(zkRegistrationClient.getAllBookies()); assertEquals(new LongVersion(1234), result.getVersion()); @@ -176,10 +209,11 @@ public void testGetAllBookies() throws Exception { @Test public void testGetReadOnlyBookies() throws Exception { - Set addresses = prepareNBookies(10); + Set addresses = prepareNBookies(10); List children = Lists.newArrayList(); - for (BookieSocketAddress address : addresses) { + for (BookieId address : addresses) { children.add(address.toString()); + prepareReadBookieServiceInfo(address, false); } Stat stat = mock(Stat.class); when(stat.getCversion()).thenReturn(1234); @@ -187,7 +221,7 @@ public void testGetReadOnlyBookies() throws Exception { regReadonlyPath, false, Code.OK.intValue(), children, stat); - Versioned> result = + Versioned> result = result(zkRegistrationClient.getReadOnlyBookies()); assertEquals(new LongVersion(1234), result.getVersion()); @@ -254,7 +288,7 @@ private void testWatchBookiesSuccess(boolean isWritable) // 1. test watch bookies with a listener // - LinkedBlockingQueue>> updates = + LinkedBlockingQueue>> updates = spy(new LinkedBlockingQueue<>()); RegistrationListener listener = bookies -> { try { @@ -264,10 +298,11 @@ private void testWatchBookiesSuccess(boolean isWritable) } }; - Set addresses = prepareNBookies(10); + Set addresses = prepareNBookies(10); List children = Lists.newArrayList(); - for (BookieSocketAddress address : addresses) { + for (BookieId address : addresses) { children.add(address.toString()); + prepareReadBookieServiceInfo(address, !isWritable); } Stat stat = mock(Stat.class); when(stat.getCversion()).thenReturn(1234); @@ -283,7 +318,7 @@ private void testWatchBookiesSuccess(boolean isWritable) result(zkRegistrationClient.watchReadOnlyBookies(listener)); } - Versioned> update = updates.take(); + Versioned> update = updates.take(); verify(updates, times(1)).put(any(Versioned.class)); assertEquals(new LongVersion(1234), update.getVersion()); assertSetEquals( @@ -298,7 +333,7 @@ private void testWatchBookiesSuccess(boolean isWritable) // // register another listener - LinkedBlockingQueue>> secondUpdates = + LinkedBlockingQueue>> secondUpdates = spy(new LinkedBlockingQueue<>()); RegistrationListener secondListener = bookies -> { try { @@ -312,7 +347,7 @@ private void testWatchBookiesSuccess(boolean isWritable) } else { result(zkRegistrationClient.watchReadOnlyBookies(secondListener)); } - Versioned> secondListenerUpdate = secondUpdates.take(); + Versioned> secondListenerUpdate = secondUpdates.take(); // first listener will not be notified with any update verify(updates, times(1)).put(any(Versioned.class)); // second listener will receive same update as the first listener received before @@ -350,10 +385,11 @@ private void testWatchBookiesSuccess(boolean isWritable) // 4. notify with new bookies. both listeners will be notified with new bookies. // - Set newAddresses = prepareNBookies(20); + Set newAddresses = prepareNBookies(20); List newChildren = Lists.newArrayList(); - for (BookieSocketAddress address : newAddresses) { + for (BookieId address : newAddresses) { newChildren.add(address.toString()); + prepareReadBookieServiceInfo(address, !isWritable); } Stat newStat = mock(Stat.class); when(newStat.getCversion()).thenReturn(1235); @@ -390,8 +426,9 @@ private void testWatchBookiesSuccess(boolean isWritable) newAddresses = prepareNBookies(25); newChildren.clear(); newChildren = Lists.newArrayList(); - for (BookieSocketAddress address : newAddresses) { + for (BookieId address : newAddresses) { newChildren.add(address.toString()); + prepareReadBookieServiceInfo(address, !isWritable); } newStat = mock(Stat.class); when(newStat.getCversion()).thenReturn(1236); @@ -464,10 +501,11 @@ private void testWatchBookiesTwice(boolean isWritable) throws Exception { int zkCallbackDelayMs = 100; - Set addresses = prepareNBookies(10); + Set addresses = prepareNBookies(10); List children = Lists.newArrayList(); - for (BookieSocketAddress address : addresses) { + for (BookieId address : addresses) { children.add(address.toString()); + prepareReadBookieServiceInfo(address, !isWritable); } Stat stat = mock(Stat.class); when(stat.getCversion()).thenReturn(1234); @@ -477,10 +515,10 @@ private void testWatchBookiesTwice(boolean isWritable) true, Code.OK.intValue(), children, stat, zkCallbackDelayMs); - CompletableFuture>> firstResult = new CompletableFuture<>(); + CompletableFuture>> firstResult = new CompletableFuture<>(); RegistrationListener firstListener = bookies -> firstResult.complete(bookies); - CompletableFuture>> secondResult = new CompletableFuture<>(); + CompletableFuture>> secondResult = new CompletableFuture<>(); RegistrationListener secondListener = bookies -> secondResult.complete(bookies); List> watchFutures = Lists.newArrayListWithExpectedSize(2); @@ -519,7 +557,7 @@ private void testWatchBookiesFailure(boolean isWritable) true, Code.NONODE.intValue(), null, null, zkCallbackDelayMs); - CompletableFuture>> listenerResult = new CompletableFuture<>(); + CompletableFuture>> listenerResult = new CompletableFuture<>(); RegistrationListener listener = bookies -> listenerResult.complete(bookies); CompletableFuture watchFuture; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java index e2c6a12d0e5..2086a3feaee 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java @@ -56,6 +56,7 @@ import org.apache.bookkeeper.common.testing.executors.MockExecutorController; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.util.ZkUtils; @@ -115,12 +116,11 @@ public void setup() throws Exception { withSettings() .useConstructor(conf, mockZk) .defaultAnswer(CALLS_REAL_METHODS)); - List ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 3181), - new BookieSocketAddress("192.0.2.2", 3181), - new BookieSocketAddress("192.0.2.3", 3181), - new BookieSocketAddress("192.0.2.4", 3181), - new BookieSocketAddress("192.0.2.5", 3181)); + List ensemble = Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 3181).toBookieId(), + new BookieSocketAddress("192.0.2.2", 3181).toBookieId(), + new BookieSocketAddress("192.0.2.3", 3181).toBookieId(), + new BookieSocketAddress("192.0.2.4", 3181).toBookieId(), + new BookieSocketAddress("192.0.2.5", 3181).toBookieId()); this.metadata = LedgerMetadataBuilder.create() .withDigestType(DigestType.CRC32C).withPassword(new byte[0]) .withEnsembleSize(5) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java index 238877222f2..ce535981517 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java @@ -73,6 +73,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -95,17 +96,17 @@ public GcLedgersTest(Class lmFactoryCls) { } private void createLedgers(int numLedgers, final Set createdLedgers) throws IOException { - BookieSocketAddress selfBookie = Bookie.getBookieAddress(baseConf); + BookieId selfBookie = Bookie.getBookieId(baseConf); createLedgers(numLedgers, createdLedgers, selfBookie); } /** * Create ledgers. */ - private void createLedgers(int numLedgers, final Set createdLedgers, BookieSocketAddress selfBookie) + private void createLedgers(int numLedgers, final Set createdLedgers, BookieId selfBookie) throws IOException { final AtomicInteger expected = new AtomicInteger(numLedgers); - List ensemble = Lists.newArrayList(selfBookie); + List ensemble = Lists.newArrayList(selfBookie); for (int i = 0; i < numLedgers; i++) { getLedgerIdGenerator().generateLedgerId(new GenericCallback() { @@ -715,7 +716,7 @@ public void testGcLedgersForOverreplicated() throws Exception { // Create few ledgers final int numLedgers = 5; - BookieSocketAddress bookieAddress = new BookieSocketAddress("192.0.0.1", 1234); + BookieId bookieAddress = new BookieSocketAddress("192.0.0.1", 1234).toBookieId(); createLedgers(numLedgers, createdLedgers, bookieAddress); LedgerManager mockLedgerManager = new CleanupLedgerManager(getLedgerManager()) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java index 60c7d0baab1..f1e9161f384 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java @@ -50,6 +50,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.ZkUtils; @@ -87,10 +88,9 @@ void removeLedger(LedgerManager lm, Long ledgerId) throws Exception { * @throws InterruptedException */ void createLedger(LedgerManager lm, Long ledgerId) throws Exception { - List ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234)); + List ensemble = Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.2", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.3", 1234).toBookieId()); LedgerMetadata meta = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) .withPassword("passwd".getBytes()) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java index a159e6cefcd..07e4bb3fc7b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestLedgerMetadataSerDe.java @@ -127,9 +127,9 @@ public void testPeggedToV3SerDe() throws Exception { LedgerMetadata metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) - .newEnsembleEntry(0L, Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 3181), - new BookieSocketAddress("192.0.2.2", 3181), - new BookieSocketAddress("192.0.2.3", 3181))) + .newEnsembleEntry(0L, Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 3181).toBookieId(), + new BookieSocketAddress("192.0.2.2", 3181).toBookieId(), + new BookieSocketAddress("192.0.2.3", 3181).toBookieId())) .build(); byte[] encoded = serDe.serialize(metadata); @@ -143,10 +143,9 @@ public void testStoreSystemtimeAsLedgerCtimeEnabledWithNewerVersion() LedgerMetadata lm = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) - .newEnsembleEntry(0L, Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234))) + .newEnsembleEntry(0L, Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.2", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.3", 1234).toBookieId())) .withCreationTime(123456L) .storingCreationTime(true) .build(); @@ -166,10 +165,9 @@ public void testStoreSystemtimeAsLedgerCtimeDisabledWithNewerVersion() LedgerMetadata lm = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(1) .withPassword("foobar".getBytes(UTF_8)).withDigestType(DigestType.CRC32C) - .newEnsembleEntry(0L, Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234))) + .newEnsembleEntry(0L, Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.2", 1234).toBookieId(), + new BookieSocketAddress("192.0.2.3", 1234).toBookieId())) .build(); LedgerMetadataSerDe serDe = new LedgerMetadataSerDe(); byte[] serialized = serDe.serialize(lm); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieIdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieIdTest.java new file mode 100644 index 00000000000..852a7557d01 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieIdTest.java @@ -0,0 +1,90 @@ +/** + * + * 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.net; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import java.util.UUID; +import org.junit.Test; + +/** + * Unit tests for BookieId class. + */ +public class BookieIdTest { + + @Test + public void testToString() { + assertEquals("test", BookieId.parse("test").toString()); + } + + @Test + public void testParse() { + assertEquals("test", BookieId.parse("test").getId()); + } + + @Test + public void testEquals() { + assertEquals(BookieId.parse("test"), BookieId.parse("test")); + assertNotEquals(BookieId.parse("test"), BookieId.parse("test2")); + } + + @Test + public void testHashcode() { + assertEquals(BookieId.parse("test").hashCode(), BookieId.parse("test").hashCode()); + } + + @Test(expected = IllegalArgumentException.class) + public void testValidate1() { + BookieId.parse("non valid"); + } + + @Test(expected = IllegalArgumentException.class) + public void testValidate2() { + BookieId.parse("non$valid"); + } + + @Test(expected = IllegalArgumentException.class) + public void testValidateReservedWord() { + // 'readonly' is a reserved word for the ZK based implementation + BookieId.parse("readonly"); + } + + @Test + public void testValidateHostnamePort() { + BookieId.parse("this.is.an.hostname:1234"); + } + + @Test + public void testValidateIPv4Port() { + BookieId.parse("1.2.3.4:1234"); + } + + @Test + public void testValidateUUID() { + BookieId.parse(UUID.randomUUID().toString()); + } + + @Test + public void testWithDashAndUnderscore() { + BookieId.parse("testRegisterUnregister_ReadonlyBookie-readonly:3181"); + } + +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieSocketAddressTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/ResolvedBookieSocketAddressTest.java similarity index 97% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieSocketAddressTest.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/net/ResolvedBookieSocketAddressTest.java index 1da5643d998..8f129842ab7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/BookieSocketAddressTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/net/ResolvedBookieSocketAddressTest.java @@ -30,7 +30,7 @@ * Tests for BookieSocketAddress getSocketAddress cache logic. */ -public class BookieSocketAddressTest { +public class ResolvedBookieSocketAddressTest { @Test public void testHostnameBookieId() throws Exception { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java index 1178a99e98c..fa3d1ba5d36 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java @@ -41,7 +41,7 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedExecutor; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.FutureGetListOfEntriesOfLedger; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; @@ -64,15 +64,15 @@ public class MockBookieClient implements BookieClient { static final Logger LOG = LoggerFactory.getLogger(MockBookieClient.class); final OrderedExecutor executor; - final ConcurrentHashMap> data = new ConcurrentHashMap<>(); - final Set errorBookies = - Collections.newSetFromMap(new ConcurrentHashMap()); + final ConcurrentHashMap> data = new ConcurrentHashMap<>(); + final Set errorBookies = + Collections.newSetFromMap(new ConcurrentHashMap()); /** * Runs before or after an operation. Can stall the operation or error it. */ public interface Hook { - CompletableFuture runHook(BookieSocketAddress bookie, long ledgerId, long entryId); + CompletableFuture runHook(BookieId bookie, long ledgerId, long entryId); } private Hook preReadHook = (bookie, ledgerId, entryId) -> FutureUtils.value(null); @@ -100,17 +100,17 @@ public void setPostWriteHook(Hook hook) { this.postWriteHook = hook; } - public void errorBookies(BookieSocketAddress... bookies) { + public void errorBookies(BookieId... bookies) { errorBookies.addAll(Arrays.asList(bookies)); } - public void removeErrors(BookieSocketAddress... bookies) { - for (BookieSocketAddress b : bookies) { + public void removeErrors(BookieId... bookies) { + for (BookieId b : bookies) { errorBookies.remove(b); } } - public void seedEntries(BookieSocketAddress bookie, long ledgerId, long entryId, long lac) throws Exception { + public void seedEntries(BookieId bookie, long ledgerId, long entryId, long lac) throws Exception { DigestManager digestManager = DigestManager.instantiate(ledgerId, new byte[0], DigestType.CRC32C, UnpooledByteBufAllocator.DEFAULT, false); ByteBuf entry = ByteBufList.coalesce(digestManager.computeDigestAndPackageForSending( @@ -121,22 +121,22 @@ public void seedEntries(BookieSocketAddress bookie, long ledgerId, long entryId, } @Override - public List getFaultyBookies() { + public List getFaultyBookies() { return Collections.emptyList(); } @Override - public boolean isWritable(BookieSocketAddress address, long ledgerId) { + public boolean isWritable(BookieId address, long ledgerId) { return true; } @Override - public long getNumPendingRequests(BookieSocketAddress address, long ledgerId) { + public long getNumPendingRequests(BookieId address, long ledgerId) { return 0; } @Override - public void forceLedger(BookieSocketAddress addr, long ledgerId, + public void forceLedger(BookieId addr, long ledgerId, ForceLedgerCallback cb, Object ctx) { executor.executeOrdered(ledgerId, safeRun(() -> { @@ -146,7 +146,7 @@ public void forceLedger(BookieSocketAddress addr, long ledgerId, } @Override - public void writeLac(BookieSocketAddress addr, long ledgerId, byte[] masterKey, + public void writeLac(BookieId addr, long ledgerId, byte[] masterKey, long lac, ByteBufList toSend, WriteLacCallback cb, Object ctx) { executor.executeOrdered(ledgerId, safeRun(() -> { @@ -156,7 +156,7 @@ public void writeLac(BookieSocketAddress addr, long ledgerId, byte[] masterKey, } @Override - public void addEntry(BookieSocketAddress addr, long ledgerId, byte[] masterKey, + public void addEntry(BookieId addr, long ledgerId, byte[] masterKey, long entryId, ByteBufList toSend, WriteCallback cb, Object ctx, int options, boolean allowFastFail, EnumSet writeFlags) { toSend.retain(); @@ -185,7 +185,7 @@ public void addEntry(BookieSocketAddress addr, long ledgerId, byte[] masterKey, } @Override - public void readLac(BookieSocketAddress addr, long ledgerId, ReadLacCallback cb, Object ctx) { + public void readLac(BookieId addr, long ledgerId, ReadLacCallback cb, Object ctx) { executor.executeOrdered(ledgerId, safeRun(() -> { cb.readLacComplete(BKException.Code.IllegalOpException, @@ -194,7 +194,7 @@ public void readLac(BookieSocketAddress addr, long ledgerId, ReadLacCallback cb, } @Override - public void readEntry(BookieSocketAddress addr, long ledgerId, long entryId, + public void readEntry(BookieId addr, long ledgerId, long entryId, ReadEntryCallback cb, Object ctx, int flags, byte[] masterKey, boolean allowFastFail) { preReadHook.runHook(addr, ledgerId, entryId) @@ -232,7 +232,7 @@ public void readEntry(BookieSocketAddress addr, long ledgerId, long entryId, } @Override - public void readEntryWaitForLACUpdate(BookieSocketAddress addr, + public void readEntryWaitForLACUpdate(BookieId addr, long ledgerId, long entryId, long previousLAC, @@ -248,7 +248,7 @@ public void readEntryWaitForLACUpdate(BookieSocketAddress addr, } @Override - public void getBookieInfo(BookieSocketAddress addr, long requested, + public void getBookieInfo(BookieId addr, long requested, GetBookieInfoCallback cb, Object ctx) { executor.executeOrdered(addr, safeRun(() -> { @@ -258,7 +258,7 @@ public void getBookieInfo(BookieSocketAddress addr, long requested, } @Override - public CompletableFuture getListOfEntriesOfLedger(BookieSocketAddress address, + public CompletableFuture getListOfEntriesOfLedger(BookieId address, long ledgerId) { FutureGetListOfEntriesOfLedger futureResult = new FutureGetListOfEntriesOfLedger(ledgerId); executor.executeOrdered(address, safeRun(() -> { @@ -277,7 +277,7 @@ public boolean isClosed() { public void close() { } - private ConcurrentHashMap getBookieData(BookieSocketAddress addr) { + private ConcurrentHashMap getBookieData(BookieId addr) { return data.computeIfAbsent(addr, (key) -> new ConcurrentHashMap<>()); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java index a07acef811a..39e72ea97a5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java @@ -41,6 +41,7 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol.AuthRequest; import org.apache.bookkeeper.proto.BookieProtocol.AuthResponse; @@ -87,7 +88,7 @@ public void testAuthSingleMessage() throws Exception { builder.setPayload(ByteString.copyFrom(PAYLOAD_MESSAGE)); final AuthMessage authMessage = builder.build(); - CompatClient42 client = newCompatClient(bookie1.getLocalAddress()); + CompatClient42 client = newCompatClient(bookie1.getBookieId()); Request request = new AuthRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, authMessage); client.sendRequest(request); @@ -108,7 +109,7 @@ public void testAuthMultiMessage() throws Exception { .setAuthPluginName(TestAuth.TEST_AUTH_PROVIDER_PLUGIN_NAME); builder.setPayload(ByteString.copyFrom(PAYLOAD_MESSAGE)); final AuthMessage authMessage = builder.build(); - CompatClient42 client = newCompatClient(bookie1.getLocalAddress()); + CompatClient42 client = newCompatClient(bookie1.getBookieId()); Request request = new AuthRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, authMessage); for (int i = 0; i < 3; i++) { @@ -141,7 +142,7 @@ public void testAuthFail() throws Exception { .setAuthPluginName(TestAuth.TEST_AUTH_PROVIDER_PLUGIN_NAME); builder.setPayload(ByteString.copyFrom(PAYLOAD_MESSAGE)); final AuthMessage authMessage = builder.build(); - CompatClient42 client = newCompatClient(bookie1.getLocalAddress()); + CompatClient42 client = newCompatClient(bookie1.getBookieId()); Request request = new AuthRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, authMessage); for (int i = 0; i < 3; i++) { @@ -178,7 +179,7 @@ BookieServer startAndStoreBookie(ServerConfiguration conf) throws Exception { return s; } - CompatClient42 newCompatClient(BookieSocketAddress addr) throws Exception { + CompatClient42 newCompatClient(BookieId addr) throws Exception { ClientConfiguration conf = new ClientConfiguration(); conf.setUseV2WireProtocol(true); return new CompatClient42(conf, executor, eventLoopGroup, addr, authProvider, extRegistry); @@ -193,18 +194,18 @@ class CompatClient42 extends PerChannelBookieClient { CompatClient42(ClientConfiguration conf, OrderedExecutor executor, EventLoopGroup eventLoopGroup, - BookieSocketAddress addr, + BookieId addr, ClientAuthProvider.Factory authProviderFactory, ExtensionRegistry extRegistry) throws Exception { - super( - conf, + super(conf, executor, eventLoopGroup, addr, NullStatsLogger.INSTANCE, authProviderFactory, extRegistry, - null); + null, + BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); state = ConnectionState.CONNECTING; ChannelFuture future = connect(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java index 7dc62779e1c..db6f2c56625 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java @@ -42,6 +42,7 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; @@ -81,10 +82,10 @@ public void testConnectCloseRace() throws Exception { EventLoopGroup eventLoopGroup = new NioEventLoopGroup(); OrderedExecutor executor = getOrderedSafeExecutor(); - BookieSocketAddress addr = getBookie(0); + BookieId addr = getBookie(0); for (int i = 0; i < 1000; i++) { PerChannelBookieClient client = new PerChannelBookieClient(executor, eventLoopGroup, addr, - authProvider, extRegistry); + authProvider, extRegistry, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); client.connectIfNeededAndDoOp(new GenericCallback() { @Override public void operationComplete(int rc, PerChannelBookieClient client) { @@ -124,10 +125,11 @@ public void operationComplete(int rc, PerChannelBookieClient pcbc) { EventLoopGroup eventLoopGroup = new NioEventLoopGroup(); OrderedExecutor executor = getOrderedSafeExecutor(); - BookieSocketAddress addr = getBookie(0); + BookieId addr = getBookie(0); for (int i = 0; i < 100; i++) { PerChannelBookieClient client = new PerChannelBookieClient(executor, eventLoopGroup, addr, - authProvider, extRegistry); + authProvider, extRegistry, + BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); for (int j = i; j < 10; j++) { client.connectIfNeededAndDoOp(nullop); } @@ -155,10 +157,11 @@ public void operationComplete(int rc, PerChannelBookieClient client) { final int iterations = 100000; EventLoopGroup eventLoopGroup = new NioEventLoopGroup(); OrderedExecutor executor = getOrderedSafeExecutor(); - BookieSocketAddress addr = getBookie(0); + BookieId addr = getBookie(0); final PerChannelBookieClient client = new PerChannelBookieClient(executor, eventLoopGroup, - addr, authProvider, extRegistry); + addr, authProvider, extRegistry, + BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); final AtomicBoolean shouldFail = new AtomicBoolean(false); final AtomicBoolean running = new AtomicBoolean(true); final CountDownLatch disconnectRunning = new CountDownLatch(1); @@ -251,10 +254,11 @@ public ByteBuf readEntry(long ledgerId, long entryId) EventLoopGroup eventLoopGroup = new NioEventLoopGroup(); final OrderedExecutor executor = getOrderedSafeExecutor(); - BookieSocketAddress addr = getBookie(0); + BookieId addr = getBookie(0); final PerChannelBookieClient client = new PerChannelBookieClient(executor, eventLoopGroup, - addr, authProvider, extRegistry); + addr, authProvider, extRegistry, + BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); final CountDownLatch completion = new CountDownLatch(1); final ReadEntryCallback cb = new ReadEntryCallback() { @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java index f32b7876938..71475565a27 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java @@ -167,7 +167,7 @@ public void testShutdown() throws Exception { for (String child : children) { byte[] data = zkc.getData(electionPath + '/' + child, false, null); String bookieIP = new String(data); - String addr = auditor.getLocalAddress().toString(); + String addr = auditor.getBookieId().toString(); assertFalse("AuditorElection cleanup fails", bookieIP .contains(addr)); } @@ -182,7 +182,7 @@ public void testRestartAuditorBookieAfterCrashing() throws Exception { BookieServer auditor = verifyAuditor(); shutdownBookie(auditor); - String addr = auditor.getLocalAddress().toString(); + String addr = auditor.getBookieId().toString(); // restarting Bookie with same configurations. int indexOfDownBookie = bs.indexOf(auditor); @@ -203,8 +203,7 @@ public void testRestartAuditorBookieAfterCrashing() throws Exception { "Auditor re-election is not happened for auditor failure!", auditor, newAuditor); assertFalse("No relection after old auditor rejoins", auditor - .getLocalAddress().getPort() == newAuditor.getLocalAddress() - .getPort()); + .getBookieId().equals(newAuditor.getBookieId())); } private void startAuditorElector(String addr) throws Exception { @@ -217,7 +216,7 @@ private void startAuditorElector(String addr) throws Exception { private void startAuditorElectors() throws Exception { for (BookieServer bserver : bs) { - String addr = bserver.getLocalAddress().toString(); + String addr = bserver.getBookieId().toString(); startAuditorElector(addr); } } @@ -242,7 +241,7 @@ private List getAuditorBookie() throws Exception { byte[] data = zkc.getData(electionPath, false, null); assertNotNull("Auditor election failed", data); for (BookieServer bks : bs) { - if (new String(data).contains(bks.getLocalAddress().getPort() + "")) { + if (new String(data).contains(bks.getBookieId() + "")) { auditors.add(bks); } } @@ -250,7 +249,7 @@ private List getAuditorBookie() throws Exception { } private void shutdownBookie(BookieServer bkServer) throws Exception { - String addr = bkServer.getLocalAddress().toString(); + String addr = bkServer.getBookieId().toString(); LOG.debug("Shutting down bookie:" + addr); // shutdown bookie which is an auditor diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index bf96a5e37cd..fba22460359 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -61,6 +61,7 @@ import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -146,7 +147,7 @@ public void tearDown() throws Exception { private void startAuditorElectors() throws Exception { for (BookieServer bserver : bs) { - String addr = bserver.getLocalAddress().toString(); + String addr = bserver.getBookieId().toString(); AuditorElector auditorElector = new AuditorElector(addr, baseConf); auditorElectors.put(addr, auditorElector); auditorElector.start(); @@ -307,7 +308,7 @@ public void testReadOnlyBookieExclusionFromURLedgersCheck() throws Exception { BookieServer bk = bs.get(bkIndex); bookieConf.setReadOnlyModeEnabled(true); bk.getBookie().getStateManager().doTransitionToReadOnlyMode(); - bkc.waitForReadOnlyBookie(Bookie.getBookieAddress(bsConfs.get(bkIndex))).get(30, TimeUnit.SECONDS); + bkc.waitForReadOnlyBookie(Bookie.getBookieId(bsConfs.get(bkIndex))).get(30, TimeUnit.SECONDS); // grace period for publishing the bk-ledger LOG.debug("Waiting for Auditor to finish ledger check."); @@ -334,7 +335,7 @@ public void testReadOnlyBookieShutdown() throws Exception { BookieServer bk = bs.get(bkIndex); bookieConf.setReadOnlyModeEnabled(true); bk.getBookie().getStateManager().doTransitionToReadOnlyMode(); - bkc.waitForReadOnlyBookie(Bookie.getBookieAddress(bsConfs.get(bkIndex))).get(30, TimeUnit.SECONDS); + bkc.waitForReadOnlyBookie(Bookie.getBookieId(bsConfs.get(bkIndex))).get(30, TimeUnit.SECONDS); // grace period for publishing the bk-ledger LOG.debug("Waiting for Auditor to finish ledger check."); @@ -546,10 +547,10 @@ public void testTriggerAuditorWithNoPendingAuditTask() throws Exception { int numofledgers = 5; Random rand = new Random(); for (int i = 0; i < numofledgers; i++) { - ArrayList ensemble = new ArrayList(); - ensemble.add(new BookieSocketAddress("99.99.99.99:9999")); - ensemble.add(new BookieSocketAddress("11.11.11.11:1111")); - ensemble.add(new BookieSocketAddress("88.88.88.88:8888")); + ArrayList ensemble = new ArrayList(); + ensemble.add(new BookieSocketAddress("99.99.99.99:9999").toBookieId()); + ensemble.add(new BookieSocketAddress("11.11.11.11:1111").toBookieId()); + ensemble.add(new BookieSocketAddress("88.88.88.88:8888").toBookieId()); LedgerMetadata metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(2) @@ -849,7 +850,7 @@ private void doLedgerRereplication(Long... ledgerIds) private String shutdownBookie(int bkShutdownIndex) throws Exception { BookieServer bkServer = bs.get(bkShutdownIndex); - String bookieAddr = bkServer.getLocalAddress().toString(); + String bookieAddr = bkServer.getBookieId().toString(); LOG.debug("Shutting down bookie:" + bookieAddr); killBookie(bkShutdownIndex); auditorElectors.get(bookieAddr).shutdown(); @@ -929,7 +930,7 @@ private BookieServer getAuditorBookie() throws Exception { byte[] data = zkc.getData(electionPath, false, null); assertNotNull("Auditor election failed", data); for (BookieServer bks : bs) { - if (new String(data).contains(bks.getLocalAddress().getPort() + "")) { + if (new String(data).contains(bks.getBookieId() + "")) { auditors.add(bks); } } @@ -940,7 +941,7 @@ private BookieServer getAuditorBookie() throws Exception { private Auditor getAuditorBookiesAuditor() throws Exception { BookieServer auditorBookieServer = getAuditorBookie(); - String bookieAddr = auditorBookieServer.getLocalAddress().toString(); + String bookieAddr = auditorBookieServer.getBookieId().toString(); return auditorElectors.get(bookieAddr).auditor; } @@ -961,7 +962,7 @@ private int getShutDownNonAuditorBookieIdx(String exclude) throws Exception { int indexOf = bs.indexOf(getAuditorBookie()); int bkIndexDownBookie = 0; for (int i = 0; i < bs.size(); i++) { - if (i == indexOf || bs.get(i).getLocalAddress().toString().equals(exclude)) { + if (i == indexOf || bs.get(i).getBookieId().toString().equals(exclude)) { continue; } bkIndexDownBookie = i; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java index c503f12a1f8..d4f3c260245 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java @@ -67,7 +67,7 @@ public void setUp() throws Exception { conf.setAuditorPeriodicBookieCheckInterval(CHECK_INTERVAL); conf.setMetadataServiceUri(metadataServiceUri); conf.setProperty("clientConnectTimeoutMillis", 500); - String addr = bs.get(0).getLocalAddress().toString(); + String addr = bs.get(0).getBookieId().toString(); auditorElector = new AuditorElector(addr, conf); auditorElector.start(); @@ -96,8 +96,9 @@ public void testPeriodicBookieCheckInterval() throws Exception { LedgerMetadataBuilder.create().withEnsembleSize(3) .withWriteQuorumSize(3).withAckQuorumSize(3) .newEnsembleEntry(0L, Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1000), - getBookie(0), getBookie(1)))); + new BookieSocketAddress("192.0.2.1", 1000).toBookieId(), + getBookie(0), + getBookie(1)))); long underReplicatedLedger = -1; for (int i = 0; i < 10; i++) { underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 448913e0dde..01ee5d85ce2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -61,7 +61,7 @@ import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.MetadataDrivers; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.replication.ReplicationException.BKAuditException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; @@ -105,7 +105,7 @@ public void setUp() throws Exception { ServerConfiguration conf = new ServerConfiguration(bsConfs.get(i)); conf.setAuditorPeriodicCheckInterval(CHECK_INTERVAL); - String addr = bs.get(i).getLocalAddress().toString(); + String addr = bs.get(i).getBookieId().toString(); AuditorElector auditorElector = new AuditorElector(addr, conf); auditorElectors.put(addr, auditorElector); @@ -698,23 +698,23 @@ void setLatch(CountDownLatch latch) { } } - private BookieSocketAddress replaceBookieWithWriteFailingBookie(LedgerHandle lh) throws Exception { + private BookieId replaceBookieWithWriteFailingBookie(LedgerHandle lh) throws Exception { int bookieIdx = -1; Long entryId = lh.getLedgerMetadata().getAllEnsembles().firstKey(); - List curEnsemble = lh.getLedgerMetadata().getAllEnsembles().get(entryId); + List curEnsemble = lh.getLedgerMetadata().getAllEnsembles().get(entryId); // Identify a bookie in the current ledger ensemble to be replaced - BookieSocketAddress replacedBookie = null; + BookieId replacedBookie = null; for (int i = 0; i < numBookies; i++) { - if (curEnsemble.contains(bs.get(i).getLocalAddress())) { + if (curEnsemble.contains(bs.get(i).getBookieId())) { bookieIdx = i; - replacedBookie = bs.get(i).getLocalAddress(); + replacedBookie = bs.get(i).getBookieId(); break; } } assertNotEquals("Couldn't find ensemble bookie in bookie list", -1, bookieIdx); - LOG.info("Killing bookie " + bs.get(bookieIdx).getLocalAddress()); + LOG.info("Killing bookie " + bs.get(bookieIdx).getBookieId()); ServerConfiguration conf = killBookie(bookieIdx); Bookie writeFailingBookie = new Bookie(conf) { @Override @@ -754,7 +754,7 @@ public void testFailedWriteRecovery() throws Exception { // kill one of the bookies and replace it with one that rejects write; // This way we get into the under replication state - BookieSocketAddress replacedBookie = replaceBookieWithWriteFailingBookie(lh); + BookieId replacedBookie = replaceBookieWithWriteFailingBookie(lh); // Write a few entries; this should cause under replication byte[] data = "foobar".getBytes(); @@ -796,9 +796,9 @@ public void testFailedWriteRecovery() throws Exception { // check that ensemble has changed and the bookie that rejected writes has // been replaced in the ensemble LedgerHandle newLh = bkc.openLedger(lh.getId(), DigestType.CRC32, "passwd".getBytes()); - for (Map.Entry> e : + for (Map.Entry> e : newLh.getLedgerMetadata().getAllEnsembles().entrySet()) { - List ensemble = e.getValue(); + List ensemble = e.getValue(); assertFalse("Ensemble hasn't been updated", ensemble.contains(replacedBookie)); } newLh.close(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java index 75a7b1696b3..2f8e7e063ad 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -50,6 +50,7 @@ import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.replication.AuditorPeriodicCheckTest.TestAuditor; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -100,7 +101,7 @@ public void tearDown() throws Exception { @Test public void testPlacementPolicyCheckWithBookiesFromDifferentRacks() throws Exception { int numOfBookies = 5; - List bookieAddresses = new ArrayList(); + List bookieAddresses = new ArrayList<>(); BookieSocketAddress bookieAddress; RegistrationManager regManager = driver.getRegistrationManager(); @@ -108,8 +109,8 @@ public void testPlacementPolicyCheckWithBookiesFromDifferentRacks() throws Excep for (int i = 0; i < numOfBookies; i++) { bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), "/rack" + (i)); - bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); + bookieAddresses.add(bookieAddress.toBookieId()); + regManager.registerBookie(bookieAddress.toBookieId(), false, BookieServiceInfo.EMPTY); } LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); @@ -212,21 +213,21 @@ public void testPlacementPolicyCheckWithBookiesFromDifferentRacks() throws Excep public void testPlacementPolicyCheckWithLedgersNotAdheringToPlacementPolicy() throws Exception { int numOfBookies = 5; int numOfLedgersNotAdheringToPlacementPolicy = 0; - List bookieAddresses = new ArrayList(); + List bookieAddresses = new ArrayList<>(); RegistrationManager regManager = driver.getRegistrationManager(); for (int i = 0; i < numOfBookies; i++) { - BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); + BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); } // only three racks - StaticDNSResolver.addNodeToRack(bookieAddresses.get(0).getHostName(), "/rack1"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(1).getHostName(), "/rack2"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(2).getHostName(), "/rack3"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(3).getHostName(), "/rack1"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(4).getHostName(), "/rack2"); + StaticDNSResolver.addNodeToRack("98.98.98.0", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.1", "/rack2"); + StaticDNSResolver.addNodeToRack("98.98.98.2", "/rack3"); + StaticDNSResolver.addNodeToRack("98.98.98.3", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.4", "/rack2"); LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); LedgerManager lm = mFactory.newLedgerManager(); @@ -307,13 +308,13 @@ public void testPlacementPolicyCheckWithURLedgers(boolean timeElapsed) throws Ex */ int underreplicatedLedgerRecoveryGracePeriod = timeElapsed ? 1 : 1000; int numOfURLedgersElapsedRecoveryGracePeriod = 0; - List bookieAddresses = new ArrayList(); + List bookieAddresses = new ArrayList(); RegistrationManager regManager = driver.getRegistrationManager(); for (int i = 0; i < numOfBookies; i++) { - BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); + BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); } LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); @@ -423,23 +424,23 @@ public void testPlacementPolicyCheckWithURLedgers(boolean timeElapsed) throws Ex public void testPlacementPolicyCheckWithLedgersNotAdheringToPolicyWithMultipleSegments() throws Exception { int numOfBookies = 7; int numOfLedgersNotAdheringToPlacementPolicy = 0; - List bookieAddresses = new ArrayList(); + List bookieAddresses = new ArrayList<>(); RegistrationManager regManager = driver.getRegistrationManager(); for (int i = 0; i < numOfBookies; i++) { - BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); + BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); } // only three racks - StaticDNSResolver.addNodeToRack(bookieAddresses.get(0).getHostName(), "/rack1"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(1).getHostName(), "/rack2"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(2).getHostName(), "/rack3"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(3).getHostName(), "/rack4"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(4).getHostName(), "/rack1"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(5).getHostName(), "/rack2"); - StaticDNSResolver.addNodeToRack(bookieAddresses.get(6).getHostName(), "/rack3"); + StaticDNSResolver.addNodeToRack("98.98.98.0", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.1", "/rack2"); + StaticDNSResolver.addNodeToRack("98.98.98.2", "/rack3"); + StaticDNSResolver.addNodeToRack("98.98.98.3", "/rack4"); + StaticDNSResolver.addNodeToRack("98.98.98.4", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.5", "/rack2"); + StaticDNSResolver.addNodeToRack("98.98.98.6", "/rack3"); LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); LedgerManager lm = mFactory.newLedgerManager(); @@ -526,7 +527,7 @@ public void testZoneawarePlacementPolicyCheck() throws Exception { int numOfBookies = 6; int numOfLedgersNotAdheringToPlacementPolicy = 0; int numOfLedgersSoftlyAdheringToPlacementPolicy = 0; - List bookieAddresses = new ArrayList(); + List bookieAddresses = new ArrayList(); RegistrationManager regManager = driver.getRegistrationManager(); /* @@ -534,8 +535,8 @@ public void testZoneawarePlacementPolicyCheck() throws Exception { */ for (int i = 0; i < numOfBookies; i++) { BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); - bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); + bookieAddresses.add(bookieAddress.toBookieId()); + regManager.registerBookie(bookieAddress.toBookieId(), false, BookieServiceInfo.EMPTY); String zone = "/zone" + (i % 3); String upgradeDomain = "/ud" + (i % 2); String networkLocation = zone + upgradeDomain; @@ -605,7 +606,7 @@ public void testZoneawarePlacementPolicyCheck() throws Exception { * ensemble/writeQuorum of size 4 has spread across just * minNumZonesPerWriteQuorum (2). */ - List newEnsemble = new ArrayList(); + List newEnsemble = new ArrayList(); newEnsemble.add(bookieAddresses.get(0)); newEnsemble.add(bookieAddresses.get(1)); newEnsemble.add(bookieAddresses.get(3)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java index fcbde5ea4a1..abf439f1ec8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java @@ -53,6 +53,7 @@ import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.replication.AuditorPeriodicCheckTest.TestAuditor; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -119,7 +120,7 @@ public TestBookKeeperAdmin(BookKeeper bkc, StatsLogger statsLogger, @Override public CompletableFuture asyncGetListOfEntriesOfLedger( - BookieSocketAddress address, long ledgerId) { + BookieId address, long ledgerId) { CompletableFuture futureResult = new CompletableFuture(); Integer errorReturnValue = errorReturnValueForGetAvailabilityOfEntriesOfLedger.get(address.toString(), @@ -179,26 +180,26 @@ private void setServerConfigProperties(ServerConfiguration servConf) { servConf.setAuditorPeriodicReplicasCheckInterval(1000); } - List addAndRegisterBookies(RegistrationManager regManager, int numOfBookies) + List addAndRegisterBookies(RegistrationManager regManager, int numOfBookies) throws BookieException { - BookieSocketAddress bookieAddress; - List bookieAddresses = new ArrayList(); + BookieId bookieAddress; + List bookieAddresses = new ArrayList(); for (int i = 0; i < numOfBookies; i++) { - bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); + bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); bookieAddresses.add(bookieAddress); - regManager.registerBookie(bookieAddress.toString(), false, BookieServiceInfo.EMPTY); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); } return bookieAddresses; } private void createClosedLedgerMetadata(LedgerManager lm, long ledgerId, int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map> segmentEnsembles, long lastEntryId, int length, + int ackQuorumSize, Map> segmentEnsembles, long lastEntryId, int length, DigestType digestType, byte[] password) throws InterruptedException, ExecutionException { LedgerMetadataBuilder ledgerMetadataBuilder = LedgerMetadataBuilder.create(); ledgerMetadataBuilder.withEnsembleSize(ensembleSize).withWriteQuorumSize(writeQuorumSize) .withAckQuorumSize(ackQuorumSize).withClosedState().withLastEntryId(lastEntryId).withLength(length) .withDigestType(digestType).withPassword(password); - for (Map.Entry> mapEntry : segmentEnsembles.entrySet()) { + for (Map.Entry> mapEntry : segmentEnsembles.entrySet()) { ledgerMetadataBuilder.newEnsembleEntry(mapEntry.getKey(), mapEntry.getValue()); } LedgerMetadata initMeta = ledgerMetadataBuilder.build(); @@ -206,12 +207,12 @@ private void createClosedLedgerMetadata(LedgerManager lm, long ledgerId, int ens } private void createNonClosedLedgerMetadata(LedgerManager lm, long ledgerId, int ensembleSize, int writeQuorumSize, - int ackQuorumSize, Map> segmentEnsembles, DigestType digestType, + int ackQuorumSize, Map> segmentEnsembles, DigestType digestType, byte[] password) throws InterruptedException, ExecutionException { LedgerMetadataBuilder ledgerMetadataBuilder = LedgerMetadataBuilder.create(); ledgerMetadataBuilder.withEnsembleSize(ensembleSize).withWriteQuorumSize(writeQuorumSize) .withAckQuorumSize(ackQuorumSize).withDigestType(digestType).withPassword(password); - for (Map.Entry> mapEntry : segmentEnsembles.entrySet()) { + for (Map.Entry> mapEntry : segmentEnsembles.entrySet()) { ledgerMetadataBuilder.newEnsembleEntry(mapEntry.getKey(), mapEntry.getValue()); } LedgerMetadata initMeta = ledgerMetadataBuilder.build(); @@ -278,7 +279,7 @@ public void testReplicasCheckForBookieHandleNotAvailable() throws Exception { new MultiKeyMap(); MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = new MultiKeyMap(); - List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); LedgerManager lm = mFactory.newLedgerManager(); @@ -299,12 +300,12 @@ public void testReplicasCheckForBookieHandleNotAvailable() throws Exception { * BookieHandleNotAvailableException so asyncGetListOfEntriesOfLedger will * return BookieHandleNotAvailableException. */ - Map> segmentEnsembles = new HashMap>(); + Map> segmentEnsembles = new HashMap>(); segmentEnsembles.put(0L, bookieAddresses); long ledgerId = 1L; createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, lastEntryId, length, digestType, password); - for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + for (BookieId bookieSocketAddress : bookieAddresses) { errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); } @@ -325,7 +326,7 @@ public void testReplicasCheckForBookieHandleNotAvailable() throws Exception { ledgerId = 2L; createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, lastEntryId, length, digestType, password); - for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + for (BookieId bookieSocketAddress : bookieAddresses) { errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); } @@ -338,7 +339,7 @@ public void testReplicasCheckForBookieHandleNotAvailable() throws Exception { ledgerId = 3L; createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, digestType, password); - for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + for (BookieId bookieSocketAddress : bookieAddresses) { errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); } @@ -354,7 +355,7 @@ public void testReplicasCheckForBookieHandleNotAvailable() throws Exception { ledgerId = 4L; createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, digestType, password); - for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + for (BookieId bookieSocketAddress : bookieAddresses) { errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); } @@ -376,7 +377,7 @@ public void testReplicasCheckForLedgersFoundHavingNoReplica() throws Exception { new MultiKeyMap(); MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = new MultiKeyMap(); - List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); LedgerManager lm = mFactory.newLedgerManager(); @@ -398,12 +399,12 @@ public void testReplicasCheckForLedgersFoundHavingNoReplica() throws Exception { * Empty one for all of the bookies, so this ledger would be counted in * ledgersFoundHavingNoReplicaOfAnEntry . */ - Map> segmentEnsembles = new HashMap>(); + Map> segmentEnsembles = new HashMap>(); segmentEnsembles.put(0L, bookieAddresses); long ledgerId = 1L; createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, lastEntryId, length, digestType, password); - for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + for (BookieId bookieSocketAddress : bookieAddresses) { returnAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); } @@ -426,7 +427,7 @@ public void testReplicasCheckForLedgersFoundHavingNoReplica() throws Exception { ledgerId = 2L; createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, lastEntryId, length, digestType, password); - for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + for (BookieId bookieSocketAddress : bookieAddresses) { errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), BKException.Code.NoSuchLedgerExistsException); } @@ -443,7 +444,7 @@ public void testReplicasCheckForLedgersFoundHavingNoReplica() throws Exception { ledgerId = 3L; createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, digestType, password); - for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + for (BookieId bookieSocketAddress : bookieAddresses) { returnAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); } @@ -466,7 +467,7 @@ public void testReplicasCheckForLedgersFoundHavingNoReplica() throws Exception { ledgerId = 4L; createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, lastEntryId, length, digestType, password); - for (BookieSocketAddress bookieSocketAddress : bookieAddresses) { + for (BookieId bookieSocketAddress : bookieAddresses) { returnAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), new AvailabilityOfEntriesOfLedger(new long[] { 0L })); } @@ -512,7 +513,7 @@ public void testReplicasCheckForLedgersFoundHavingLessThanAQReplicasOfAnEntry() new MultiKeyMap(); MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = new MultiKeyMap(); - List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); LedgerManager lm = mFactory.newLedgerManager(); @@ -529,7 +530,7 @@ public void testReplicasCheckForLedgersFoundHavingLessThanAQReplicasOfAnEntry() * would be counted towards * ledgersFoundHavingLessThanAQReplicasOfAnEntry. */ - Map> segmentEnsembles = new HashMap>(); + Map> segmentEnsembles = new HashMap>(); int ensembleSize = 4; int writeQuorumSize = 3; int ackQuorumSize = 2; @@ -667,7 +668,7 @@ public void testReplicasCheckForLedgersFoundHavingLessThanWQReplicasOfAnEntry() new MultiKeyMap(); MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = new MultiKeyMap(); - List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); LedgerManager lm = mFactory.newLedgerManager(); @@ -683,7 +684,7 @@ public void testReplicasCheckForLedgersFoundHavingLessThanWQReplicasOfAnEntry() * for this ledger a copy of entry 3, so this ledger would be counted * towards ledgersFoundHavingLessThanWQReplicasOfAnEntry. */ - Map> segmentEnsembles = new HashMap>(); + Map> segmentEnsembles = new HashMap>(); int ensembleSize = 4; int writeQuorumSize = 3; int ackQuorumSize = 2; @@ -820,7 +821,7 @@ public void testReplicasCheckForLedgersWithEmptySegments() throws Exception { new MultiKeyMap(); MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = new MultiKeyMap(); - List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); + List bookieAddresses = addAndRegisterBookies(regManager, numOfBookies); LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); LedgerManager lm = mFactory.newLedgerManager(); @@ -839,7 +840,7 @@ public void testReplicasCheckForLedgersWithEmptySegments() throws Exception { * numLedgersFoundHavingNoReplicaOfAnEntry/LessThanAQReplicasOfAnEntry * /WQReplicasOfAnEntry. */ - Map> segmentEnsembles = new HashMap>(); + Map> segmentEnsembles = new HashMap>(); int ensembleSize = 4; int writeQuorumSize = 3; int ackQuorumSize = 2; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java index 632a31b43df..66c46501cf4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java @@ -29,7 +29,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestCallbacks; import org.junit.Test; @@ -77,7 +77,7 @@ private void testAuditingDuringRollingRestart(LedgerManagerFactory mFactory) thr underReplicationManager.pollLedgerToRereplicate(), -1); underReplicationManager.disableLedgerReplication(); - BookieSocketAddress auditor = AuditorElector.getCurrentAuditor(baseConf, zkc); + BookieId auditor = AuditorElector.getCurrentAuditor(baseConf, zkc); ServerConfiguration conf = killBookie(auditor); Thread.sleep(2000); startBookie(conf); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java index 8aa969f5291..23c7a7b862d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java @@ -25,7 +25,7 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.meta.zk.ZKMetadataClientDriver; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.zookeeper.KeeperException; @@ -99,8 +99,8 @@ public void testAutoRecoverySessionLoss() throws Exception { ZKMetadataClientDriver zkMetadataClientDriver1 = startAutoRecoveryMain(main1); ZooKeeper zk1 = zkMetadataClientDriver1.getZk(); Auditor auditor1 = main1.auditorElector.getAuditor(); - BookieSocketAddress currentAuditor = AuditorElector.getCurrentAuditor(bsConfs.get(0), zk1); - assertTrue("Current Auditor should be AR1", currentAuditor.equals(Bookie.getBookieAddress(bsConfs.get(0)))); + BookieId currentAuditor = AuditorElector.getCurrentAuditor(bsConfs.get(0), zk1); + assertTrue("Current Auditor should be AR1", currentAuditor.equals(Bookie.getBookieId(bsConfs.get(0)))); assertTrue("Auditor of AR1 should be running", auditor1.isRunning()); /* @@ -116,7 +116,7 @@ public void testAutoRecoverySessionLoss() throws Exception { * auditors are not running. */ assertTrue("Current Auditor should still be AR1", - currentAuditor.equals(Bookie.getBookieAddress(bsConfs.get(0)))); + currentAuditor.equals(Bookie.getBookieId(bsConfs.get(0)))); Auditor auditor2 = main2.auditorElector.getAuditor(); Auditor auditor3 = main3.auditorElector.getAuditor(); assertTrue("AR2's Auditor should not be running", (auditor2 == null || !auditor2.isRunning())); @@ -154,7 +154,7 @@ public void testAutoRecoverySessionLoss() throws Exception { * the AR3 should be current auditor. */ currentAuditor = AuditorElector.getCurrentAuditor(bsConfs.get(2), zk3); - assertTrue("Current Auditor should be AR3", currentAuditor.equals(Bookie.getBookieAddress(bsConfs.get(2)))); + assertTrue("Current Auditor should be AR3", currentAuditor.equals(Bookie.getBookieId(bsConfs.get(2)))); auditor3 = main3.auditorElector.getAuditor(); assertTrue("Auditor of AR3 should be running", auditor3.isRunning()); 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 0a6d264f643..6362421eeb1 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 @@ -44,7 +44,7 @@ import org.apache.bookkeeper.meta.MetadataClientDriver; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; @@ -149,7 +149,7 @@ public void testOpenLedgers() throws Exception { List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); LedgerHandle lh = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - BookieSocketAddress replicaToKillAddr = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKillAddr = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); final String urLedgerZNode = getUrLedgerZNode(lh); ledgerReplicaIndex = getReplicaIndexInLedger(lh, replicaToKillAddr); @@ -197,7 +197,7 @@ public void testClosedLedgers() throws Exception { closeLedgers(listOfLedgerHandle); LedgerHandle lhandle = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - BookieSocketAddress replicaToKillAddr = lhandle.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKillAddr = lhandle.getLedgerMetadata().getAllEnsembles().get(0L).get(0); CountDownLatch latch = new CountDownLatch(listOfLedgerHandle.size()); for (LedgerHandle lh : listOfLedgerHandle) { @@ -258,7 +258,7 @@ public void testStopWhileReplicationInProgress() throws Exception { numberOfLedgers, 5); closeLedgers(listOfLedgerHandle); LedgerHandle handle = listOfLedgerHandle.get(0); - BookieSocketAddress replicaToKillAddr = handle.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKillAddr = handle.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie:" + replicaToKillAddr); // Each ledger, there will be two events : create urLedger and after @@ -336,7 +336,7 @@ public void testNoSuchLedgerExists() throws Exception { assertNull("UrLedger already exists!", watchUrLedgerNode(getUrLedgerZNode(lh), latch)); } - BookieSocketAddress replicaToKillAddr = listOfLedgerHandle.get(0) + BookieId replicaToKillAddr = listOfLedgerHandle.get(0) .getLedgerMetadata().getAllEnsembles() .get(0L).get(0); killBookie(replicaToKillAddr); @@ -379,7 +379,7 @@ public void testEmptyLedgerLosesQuorumEventually() throws Exception { String urZNode = getUrLedgerZNode(lh); watchUrLedgerNode(urZNode, latch); - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(2); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(2); LOG.info("Killing last bookie, {}, in ensemble {}", replicaToKill, lh.getLedgerMetadata().getAllEnsembles().get(0L)); killBookie(replicaToKill); @@ -437,11 +437,11 @@ public void testLedgerMetadataContainsIpAddressAsBookieID() List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); LedgerHandle lh = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - final SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); - final List bkAddresses = ensembles.get(0L); - BookieSocketAddress replicaToKillAddr = bkAddresses.get(0); - for (BookieSocketAddress bookieSocketAddress : bkAddresses) { - if (!isCreatedFromIp(bookieSocketAddress)){ + final SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); + final List bkAddresses = ensembles.get(0L); + BookieId replicaToKillAddr = bkAddresses.get(0); + for (BookieId bookieSocketAddress : bkAddresses) { + if (!isCreatedFromIp(bookieSocketAddress)) { replicaToKillAddr = bookieSocketAddress; LOG.info("Kill bookie which has registered using hostname"); break; @@ -515,10 +515,10 @@ public void testLedgerMetadataContainsHostNameAsBookieID() List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); LedgerHandle lh = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - final SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); - final List bkAddresses = ensembles.get(0L); - BookieSocketAddress replicaToKillAddr = bkAddresses.get(0); - for (BookieSocketAddress bookieSocketAddress : bkAddresses) { + final SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); + final List bkAddresses = ensembles.get(0L); + BookieId replicaToKillAddr = bkAddresses.get(0); + for (BookieId bookieSocketAddress : bkAddresses) { if (isCreatedFromIp(bookieSocketAddress)) { replicaToKillAddr = bookieSocketAddress; LOG.info("Kill bookie which has registered using ipaddress"); @@ -568,10 +568,10 @@ public void testLedgerMetadataContainsHostNameAsBookieID() } - private int getReplicaIndexInLedger(LedgerHandle lh, BookieSocketAddress replicaToKill) { - SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); + private int getReplicaIndexInLedger(LedgerHandle lh, BookieId replicaToKill) { + SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); int ledgerReplicaIndex = -1; - for (BookieSocketAddress addr : ensembles.get(0L)) { + for (BookieId addr : ensembles.get(0L)) { ++ledgerReplicaIndex; if (addr.equals(replicaToKill)) { break; @@ -586,10 +586,10 @@ private void verifyLedgerEnsembleMetadataAfterReplication( LedgerHandle openLedger = bkc .openLedger(lh.getId(), digestType, PASSWD); - BookieSocketAddress inetSocketAddress = openLedger.getLedgerMetadata().getAllEnsembles().get(0L) + BookieId inetSocketAddress = openLedger.getLedgerMetadata().getAllEnsembles().get(0L) .get(ledgerReplicaIndex); assertEquals("Rereplication has been failed and ledgerReplicaIndex :" - + ledgerReplicaIndex, newBookieServer.getLocalAddress(), + + ledgerReplicaIndex, newBookieServer.getBookieId(), inetSocketAddress); openLedger.close(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java index 0e6f9276c14..ea11f858c57 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java @@ -30,7 +30,7 @@ import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.util.BookKeeperConstants; import org.junit.Before; @@ -73,11 +73,11 @@ public void testAutoRecoveryAlongWithBookieServers() throws Exception { lh.addEntry(testData); } lh.close(); - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); killBookie(replicaToKill); - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)) { @@ -85,10 +85,10 @@ public void testAutoRecoveryAlongWithBookieServers() throws Exception { } // Killing all bookies except newly replicated bookie - for (Entry> entry : + for (Entry> entry : lh.getLedgerMetadata().getAllEnsembles().entrySet()) { - List bookies = entry.getValue(); - for (BookieSocketAddress bookie : bookies) { + List bookies = entry.getValue(); + for (BookieId bookie : bookies) { if (bookie.equals(newBkAddr)) { continue; } 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 09751e99e88..f68dfd6d7b0 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 @@ -55,7 +55,7 @@ import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -165,12 +165,12 @@ public void testRWShouldReplicateFragmentsToTargetBookie() throws Exception { for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr : {}", newBkAddr); for (int i = 0; i < 10; i++) { @@ -213,11 +213,11 @@ public void testRWShouldRetryUntilThereAreEnoughBksAvailableForReplication() lh.addEntry(data); } lh.close(); - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie : {}", replicaToKill); ServerConfiguration killedBookieConfig = killBookie(replicaToKill); - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr :" + newBkAddr); killAllBookies(lh, newBkAddr); @@ -262,18 +262,18 @@ public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplicatio lh.addEntry(data); } lh.close(); - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie : {}", replicaToKill); ServerConfiguration killedBookieConfig = killBookie(replicaToKill); killAllBookies(lh, null); // Starte RW1 - BookieSocketAddress newBkAddr1 = startNewBookieAndReturnAddress(); + BookieId newBkAddr1 = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr : {}", newBkAddr1); ReplicationWorker rw1 = new ReplicationWorker(baseConf); // Starte RW2 - BookieSocketAddress newBkAddr2 = startNewBookieAndReturnAddress(); + BookieId newBkAddr2 = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr : {}", newBkAddr2); ReplicationWorker rw2 = new ReplicationWorker(baseConf); rw1.start(); @@ -318,11 +318,11 @@ public void testRWShouldCleanTheLedgerFromUnderReplicationIfLedgerAlreadyDeleted lh.addEntry(data); } lh.close(); - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr : {}", newBkAddr); ReplicationWorker rw = new ReplicationWorker(baseConf); rw.start(); @@ -352,7 +352,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() for (int i = 0; i < 10; i++) { lh1.addEntry(data); } - BookieSocketAddress replicaToKillFromFirstLedger = lh1.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKillFromFirstLedger = lh1.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie : {}", replicaToKillFromFirstLedger); @@ -363,7 +363,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() for (int i = 0; i < 10; i++) { lh2.addEntry(data); } - BookieSocketAddress replicaToKillFromSecondLedger = lh2.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKillFromSecondLedger = lh2.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie : {}", replicaToKillFromSecondLedger); @@ -374,7 +374,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() killBookie(replicaToKillFromFirstLedger); lh2.close(); - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr : {}", newBkAddr); ReplicationWorker rw = new ReplicationWorker(baseConf); @@ -422,12 +422,12 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr : {}", newBkAddr); // set to 3s instead of default 30s @@ -476,7 +476,7 @@ public void testBookiesNotAvailableScenarioForReplicationWorker() throws Excepti } lh.close(); - BookieSocketAddress[] bookiesKilled = new BookieSocketAddress[ensembleSize]; + BookieId[] bookiesKilled = new BookieId[ensembleSize]; ServerConfiguration[] killedBookiesConfig = new ServerConfiguration[ensembleSize]; // kill all bookies @@ -489,7 +489,7 @@ public void testBookiesNotAvailableScenarioForReplicationWorker() throws Excepti // start new bookiesToKill number of bookies for (int i = 0; i < ensembleSize; i++) { - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); } // create couple of replicationworkers @@ -634,7 +634,7 @@ public void testDeferLedgerLockReleaseForReplicationWorker() throws Exception { } lh.close(); - BookieSocketAddress[] bookiesKilled = new BookieSocketAddress[ensembleSize]; + BookieId[] bookiesKilled = new BookieId[ensembleSize]; ServerConfiguration[] killedBookiesConfig = new ServerConfiguration[ensembleSize]; // kill all bookies @@ -647,7 +647,7 @@ public void testDeferLedgerLockReleaseForReplicationWorker() throws Exception { // start new bookiesToKill number of bookies for (int i = 0; i < ensembleSize; i++) { - startNewBookieAndReturnAddress(); + startNewBookieAndReturnBookieId(); } // create couple of replicationworkers @@ -757,12 +757,12 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie : {}", replicaToKill); killBookie(replicaToKill); - BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress(); + BookieId newBkAddr = startNewBookieAndReturnBookieId(); LOG.info("New Bookie addr : {}", newBkAddr); // Reform ensemble...Making sure that last fragment is not in @@ -846,13 +846,13 @@ public void testRWZKConnectionLost() throws Exception { } } - private void killAllBookies(LedgerHandle lh, BookieSocketAddress excludeBK) + private void killAllBookies(LedgerHandle lh, BookieId excludeBK) throws Exception { // Killing all bookies except newly replicated bookie - for (Entry> entry : + for (Entry> entry : lh.getLedgerMetadata().getAllEnsembles().entrySet()) { - List bookies = entry.getValue(); - for (BookieSocketAddress bookie : bookies) { + List bookies = entry.getValue(); + for (BookieId bookie : bookies) { if (bookie.equals(excludeBK)) { continue; } @@ -1008,7 +1008,7 @@ public void testRWShutDownInTheCaseOfZKOperationFailures() throws Exception { * 3 bookies are available and the ensemblesize of the current ledger is * 2, we should be able to replicate to the other bookie. */ - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); killBookie(replicaToKill); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java index 24bce2e1c55..fd4be6d6c25 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestMain.java @@ -88,6 +88,7 @@ public void testBuildBookieServer() throws Exception { BookieSocketAddress bookieAddress = new BookieSocketAddress("127.0.0.1", 1281); when(mockServer.getLocalAddress()).thenReturn(bookieAddress); + when(mockServer.getBookieId()).thenReturn(bookieAddress.toBookieId()); LifecycleComponentStack stack = buildBookieServer(conf); assertEquals(3, stack.getNumComponents()); @@ -118,6 +119,7 @@ public void testIgnoreExtraServerComponentsStartupFailures() throws Exception { BookieSocketAddress bookieAddress = new BookieSocketAddress("127.0.0.1", 1281); when(mockServer.getLocalAddress()).thenReturn(bookieAddress); + when(mockServer.getBookieId()).thenReturn(bookieAddress.toBookieId()); LifecycleComponentStack stack = buildBookieServer(conf); assertEquals(2, stack.getNumComponents()); @@ -147,6 +149,7 @@ public void testExtraServerComponentsStartupFailures() throws Exception { BookieSocketAddress bookieAddress = new BookieSocketAddress("127.0.0.1", 1281); when(mockServer.getLocalAddress()).thenReturn(bookieAddress); + when(mockServer.getBookieId()).thenReturn(bookieAddress.toBookieId()); try { buildBookieServer(conf); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index cf839627578..92a575dd825 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -593,7 +593,7 @@ public void testRecoveryBookieService() throws Exception { AuditorElector auditorElector; private Future startAuditorElector() throws Exception { - String addr = bs.get(0).getLocalAddress().toString(); + String addr = bs.get(0).getBookieId().toString(); ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); conf.setAuditorPeriodicBookieCheckInterval(1); conf.setMetadataServiceUri("zk://" + zkUtil.getZooKeeperConnectString() + "/ledgers"); @@ -675,8 +675,7 @@ private void testListUnderReplicatedLedgerService(LedgerManagerFactory mFactory) // 192.0.2.0/24 is reserved TEST-NET range LedgerMetadataBuilder metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3) - .newEnsembleEntry(0L, Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1000), + .newEnsembleEntry(0L, Lists.newArrayList(new BookieSocketAddress("192.0.2.1", 1000).toBookieId(), getBookie(0), getBookie(1))); ClientUtil.setupLedger(ledgerManager, 1L, metadata); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java index 88fb3ff2a5b..a8436ffda29 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/service/ListLedgerServiceTest.java @@ -39,7 +39,7 @@ import org.apache.bookkeeper.http.HttpServer; import org.apache.bookkeeper.http.service.HttpServiceRequest; import org.apache.bookkeeper.http.service.HttpServiceResponse; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.commons.lang3.RandomUtils; import org.junit.Before; @@ -154,14 +154,13 @@ public void testListLedgersWithMetadataDecoded() throws Exception { assertArrayEquals(entry.getValue(), Base64.getDecoder().decode(data.asText())); } - for (Map.Entry> entry : meta.getAllEnsembles().entrySet()) { + for (Map.Entry> entry : meta.getAllEnsembles().entrySet()) { JsonNode members = node.get("allEnsembles") .get(String.valueOf(entry.getKey())); assertEquals(1, entry.getValue().size()); assertEquals(entry.getValue().size(), members.size()); JsonNode member = members.get(0); - assertEquals(entry.getValue().get(0).getHostName(), member.get("hostName").asText()); - assertEquals(entry.getValue().get(0).getPort(), member.get("port").asInt()); + assertEquals(entry.getValue().get(0).getId(), member.get("id").asText()); } }); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index afc8302b74c..78a3b116856 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -54,6 +54,7 @@ import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.metastore.InMemoryMetaStore; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.Auditor; @@ -95,7 +96,7 @@ public abstract class BookKeeperClusterTestCase { protected final List tmpDirs = new LinkedList(); protected final List bs = new LinkedList(); protected final List bsConfs = new LinkedList(); - private final Map bsLoggers = new HashMap<>(); + private final Map bsLoggers = new HashMap<>(); protected int numBookies; protected BookKeeperTestClient bkc; @@ -273,7 +274,7 @@ protected void stopBKCluster() throws Exception { if (autoRecovery != null && isAutoRecoveryEnabled()) { autoRecovery.shutdown(); LOG.debug("Shutdown auto recovery for bookieserver:" - + server.getLocalAddress()); + + server.getBookieId()); } } bs.clear(); @@ -345,21 +346,25 @@ protected String newMetadataServiceUri(String ledgersRootPath, String type) { /** * Get bookie address for bookie at index. */ - public BookieSocketAddress getBookie(int index) throws Exception { + public BookieId getBookie(int index) throws Exception { if (bs.size() <= index || index < 0) { throw new IllegalArgumentException("Invalid index, there are only " + bs.size() + " bookies. Asked for " + index); } - return bs.get(index).getLocalAddress(); + return bs.get(index).getBookieId(); + } + + public BookieSocketAddress getBookieAddress(int index) throws Exception { + return bkc.getBookieAddressResolver().resolve(getBookie(index)); } /** * Get bookie configuration for bookie. */ - public ServerConfiguration getBkConf(BookieSocketAddress addr) throws Exception { + public ServerConfiguration getBkConf(BookieId addr) throws Exception { int bkIndex = 0; for (BookieServer server : bs) { - if (server.getLocalAddress().equals(addr)) { + if (server.getBookieId().equals(addr)) { break; } ++bkIndex; @@ -379,11 +384,11 @@ public ServerConfiguration getBkConf(BookieSocketAddress addr) throws Exception * @return the configuration of killed bookie * @throws InterruptedException */ - public ServerConfiguration killBookie(BookieSocketAddress addr) throws Exception { + public ServerConfiguration killBookie(BookieId addr) throws Exception { BookieServer toRemove = null; int toRemoveIndex = 0; for (BookieServer server : bs) { - if (server.getLocalAddress().equals(addr)) { + if (server.getBookieId().equals(addr)) { server.shutdown(); toRemove = server; break; @@ -406,9 +411,9 @@ public ServerConfiguration killBookie(BookieSocketAddress addr) throws Exception * Socket Address * @throws InterruptedException */ - public void setBookieToReadOnly(BookieSocketAddress addr) throws InterruptedException, UnknownHostException { + public void setBookieToReadOnly(BookieId addr) throws InterruptedException, UnknownHostException { for (BookieServer server : bs) { - if (server.getLocalAddress().equals(addr)) { + if (server.getBookieId().equals(addr)) { server.getBookie().getStateManager().doTransitionToReadOnlyMode(); break; } @@ -433,7 +438,7 @@ public ServerConfiguration killBookie(int index) throws Exception { server.shutdown(); stopAutoRecoveryService(server); bs.remove(server); - bsLoggers.remove(server.getLocalAddress()); + bsLoggers.remove(server.getBookieId()); return bsConfs.remove(index); } @@ -451,7 +456,7 @@ public ServerConfiguration killBookieAndWaitForZK(int index) throws Exception { BookieServer server = bs.get(index); ServerConfiguration ret = killBookie(index); while (zkc.exists(ZKMetadataDriverBase.resolveZkLedgersRootPath(baseConf) + "/" + AVAILABLE_NODE + "/" - + server.getLocalAddress().toString(), false) != null) { + + server.getBookieId().toString(), false) != null) { Thread.sleep(500); } return ret; @@ -468,21 +473,21 @@ public ServerConfiguration killBookieAndWaitForZK(int index) throws Exception { * @throws InterruptedException * @throws IOException */ - public CountDownLatch sleepBookie(BookieSocketAddress addr, final int seconds) + public CountDownLatch sleepBookie(BookieId addr, final int seconds) throws Exception { for (final BookieServer bookie : bs) { - if (bookie.getLocalAddress().equals(addr)) { + if (bookie.getBookieId().equals(addr)) { final CountDownLatch l = new CountDownLatch(1); Thread sleeper = new Thread() { @Override public void run() { try { bookie.suspendProcessing(); - LOG.info("bookie {} is asleep", bookie.getLocalAddress()); + LOG.info("bookie {} is asleep", bookie.getBookieId()); l.countDown(); Thread.sleep(seconds * 1000); bookie.resumeProcessing(); - LOG.info("bookie {} is awake", bookie.getLocalAddress()); + LOG.info("bookie {} is awake", bookie.getBookieId()); } catch (Exception e) { LOG.error("Error suspending bookie", e); } @@ -505,17 +510,17 @@ public void run() { * @throws InterruptedException * @throws IOException */ - public void sleepBookie(BookieSocketAddress addr, final CountDownLatch l) + public void sleepBookie(BookieId addr, final CountDownLatch l) throws InterruptedException, IOException { final CountDownLatch suspendLatch = new CountDownLatch(1); sleepBookie(addr, l, suspendLatch); suspendLatch.await(); } - public void sleepBookie(BookieSocketAddress addr, final CountDownLatch l, final CountDownLatch suspendLatch) + public void sleepBookie(BookieId addr, final CountDownLatch l, final CountDownLatch suspendLatch) throws InterruptedException, IOException { for (final BookieServer bookie : bs) { - if (bookie.getLocalAddress().equals(addr)) { + if (bookie.getBookieId().equals(addr)) { LOG.info("Sleep bookie {}.", addr); Thread sleeper = new Thread() { @Override @@ -563,11 +568,11 @@ public void restartBookies() * @throws KeeperException * @throws BookieException */ - public void restartBookie(BookieSocketAddress addr) throws Exception { + public void restartBookie(BookieId addr) throws Exception { BookieServer toRemove = null; int toRemoveIndex = 0; for (BookieServer server : bs) { - if (server.getLocalAddress().equals(addr)) { + if (server.getBookieId().equals(addr)) { toRemove = server; break; } @@ -626,10 +631,15 @@ public void restartBookies(ServerConfiguration newConf) */ public int startNewBookie() throws Exception { - return startNewBookieAndReturnAddress().getPort(); + ServerConfiguration conf = newServerConfiguration(); + bsConfs.add(conf); + LOG.info("Starting new bookie on port: {}", conf.getBookiePort()); + BookieServer server = startBookie(conf); + bs.add(server); + return server.getLocalAddress().getPort(); } - public BookieSocketAddress startNewBookieAndReturnAddress() + public BookieId startNewBookieAndReturnBookieId() throws Exception { ServerConfiguration conf = newServerConfiguration(); bsConfs.add(conf); @@ -637,7 +647,7 @@ public BookieSocketAddress startNewBookieAndReturnAddress() BookieServer server = startBookie(conf); bs.add(server); - return server.getLocalAddress(); + return server.getBookieId(); } /** @@ -653,7 +663,7 @@ protected BookieServer startBookie(ServerConfiguration conf) TestStatsProvider provider = new TestStatsProvider(); BookieServer server = new BookieServer(conf, provider.getStatsLogger(""), BookieServiceInfo.NO_INFO); - BookieSocketAddress address = Bookie.getBookieAddress(conf); + BookieId address = Bookie.getBookieId(conf); bsLoggers.put(address, provider); if (bkc == null) { @@ -695,7 +705,7 @@ protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator, } }; - BookieSocketAddress address = Bookie.getBookieAddress(conf); + BookieId address = Bookie.getBookieId(conf); if (bkc == null) { bkc = new BookKeeperTestClient(baseClientConf, new TestStatsProvider()); } @@ -704,7 +714,7 @@ protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator, : bkc.waitForWritableBookie(address); server.start(); - bsLoggers.put(server.getLocalAddress(), provider); + bsLoggers.put(server.getBookieId(), provider); waitForBookie.get(30, TimeUnit.SECONDS); LOG.info("New bookie '{}' has been created.", address); @@ -754,7 +764,7 @@ private void startAutoRecovery(BookieServer bserver, autoRecoveryProcess.start(); autoRecoveryProcesses.put(bserver, autoRecoveryProcess); LOG.debug("Starting Auditor Recovery for the bookie:" - + bserver.getLocalAddress()); + + bserver.getBookieId()); } } @@ -764,7 +774,7 @@ private void stopAutoRecoveryService(BookieServer toRemove) throws Exception { if (null != autoRecoveryMain && isAutoRecoveryEnabled()) { autoRecoveryMain.shutdown(); LOG.debug("Shutdown auto recovery for bookieserver:" - + toRemove.getLocalAddress()); + + toRemove.getBookieId()); } } @@ -792,7 +802,7 @@ public void stopReplicationService() throws Exception{ .entrySet()) { autoRecoveryProcess.getValue().shutdown(); LOG.debug("Shutdown Auditor Recovery for the bookie:" - + autoRecoveryProcess.getKey().getLocalAddress()); + + autoRecoveryProcess.getKey().getBookieId()); } } @@ -816,12 +826,12 @@ public Auditor getAuditor(int timeout, TimeUnit unit) throws Exception { * created using hostname. Represent as '/IPaddress' if the * InetSocketAddress was created using an IPaddress * - * @param addr - * inetaddress + * @param bookieId id * @return true if the address was created using an IP address, false if the * address was created using a hostname */ - public static boolean isCreatedFromIp(BookieSocketAddress addr) { + public boolean isCreatedFromIp(BookieId bookieId) { + BookieSocketAddress addr = bkc.getBookieAddressResolver().resolve(bookieId); return addr.getSocketAddress().toString().startsWith("/"); } @@ -831,12 +841,12 @@ public void resetBookieOpLoggers() { } } - public TestStatsProvider getStatsProvider(BookieSocketAddress addr) { + public TestStatsProvider getStatsProvider(BookieId addr) { return bsLoggers.get(addr); } public TestStatsProvider getStatsProvider(int index) throws Exception { - return getStatsProvider(bs.get(index).getLocalAddress()); + return getStatsProvider(bs.get(index).getBookieId()); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java index 2562009a05c..ff838f3cc2c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java @@ -47,6 +47,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; @@ -141,7 +142,7 @@ public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf bb, O }; WriteCallback wrcb = new WriteCallback() { - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { if (ctx != null) { synchronized (ctx) { if (ctx instanceof ResultStruct) { @@ -159,11 +160,12 @@ public void testWriteGaps() throws Exception { final Object notifyObject = new Object(); byte[] passwd = new byte[20]; Arrays.fill(passwd, (byte) 'a'); - BookieSocketAddress addr = bs.getLocalAddress(); + BookieId addr = bs.getBookieId(); ResultStruct arc = new ResultStruct(); BookieClient bc = new BookieClientImpl(new ClientConfiguration(), eventLoopGroup, - UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE); + UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE, + BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); ByteBufList bb = createByteBuffer(1, 1, 1); bc.addEntry(addr, 1, passwd, 1, bb, wrcb, arc, BookieProtocol.FLAG_NONE, false, WriteFlag.NONE); synchronized (arc) { @@ -262,9 +264,10 @@ private ByteBufList createByteBuffer(int i, long lid, long eid) { @Test public void testNoLedger() throws Exception { ResultStruct arc = new ResultStruct(); - BookieSocketAddress addr = bs.getLocalAddress(); + BookieId addr = bs.getBookieId(); BookieClient bc = new BookieClientImpl(new ClientConfiguration(), eventLoopGroup, - UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE); + UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE, + BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); synchronized (arc) { bc.readEntry(addr, 2, 13, recb, arc, BookieProtocol.FLAG_NONE); arc.wait(1000); @@ -283,13 +286,14 @@ public void testGetBookieInfoWithoutLimitStatsLogging() throws IOException, Inte } public void testGetBookieInfo(boolean limitStatsLogging) throws IOException, InterruptedException { + BookieId bookieId = bs.getBookieId(); BookieSocketAddress addr = bs.getLocalAddress(); ClientConfiguration clientConf = new ClientConfiguration(); clientConf.setLimitStatsLogging(limitStatsLogging); TestStatsProvider statsProvider = new TestStatsProvider(); TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); BookieClient bc = new BookieClientImpl(clientConf, new NioEventLoopGroup(), UnpooledByteBufAllocator.DEFAULT, - executor, scheduler, statsLogger); + executor, scheduler, statsLogger, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); long flags = BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE | BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; @@ -307,7 +311,7 @@ class CallbackObj { } } CallbackObj obj = new CallbackObj(flags); - bc.getBookieInfo(addr, flags, new GetBookieInfoCallback() { + bc.getBookieInfo(bookieId, flags, new GetBookieInfoCallback() { @Override public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) { CallbackObj obj = (CallbackObj) ctx; @@ -334,7 +338,7 @@ public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) { TestOpStatsLogger perChannelBookieClientScopeOfThisAddr = (TestOpStatsLogger) statsLogger .scope(BookKeeperClientStats.CHANNEL_SCOPE) - .scope(PerChannelBookieClient.buildStatsLoggerScopeName(addr)) + .scope(PerChannelBookieClient.buildStatsLoggerScopeName(addr.toBookieId())) .getOpStatsLogger(BookKeeperClientStats.GET_BOOKIE_INFO_OP); int expectedBookieInfoSuccessCount = (limitStatsLogging) ? 0 : 1; assertEquals("BookieInfoSuccessCount", expectedBookieInfoSuccessCount, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java index 282c7a55862..e8ecba5b4a4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java @@ -38,7 +38,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.junit.After; import org.junit.Before; @@ -168,7 +168,7 @@ private long doWrites(int ledgers, int size, int totalwrites) WriteCallback cb = new WriteCallback() { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { AtomicInteger counter = (AtomicInteger) ctx; counter.getAndIncrement(); throttle.release(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LocalBookiesRegistryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LocalBookiesRegistryTest.java index f1c1c0a3859..661ef21f31b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LocalBookiesRegistryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LocalBookiesRegistryTest.java @@ -34,16 +34,16 @@ public class LocalBookiesRegistryTest extends BookKeeperClusterTestCase { public LocalBookiesRegistryTest() { - super(3); + super(1); baseConf.setDisableServerSocketBind(true); baseConf.setEnableLocalTransport(true); } @Test public void testAccessibleLocalBookiesRegistry() throws Exception { - assertEquals(3, bs.size()); + assertEquals(1, bs.size()); for (BookieServer bk : bs) { - assertTrue(LocalBookiesRegistry.isLocalBookie(bk.getLocalAddress())); + assertTrue(LocalBookiesRegistry.isLocalBookie(bk.getBookieId())); } } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java index a154121dc1a..8bfbd8e6d92 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ReadOnlyBookieTest.java @@ -127,7 +127,7 @@ public void testBookieShouldTurnWritableFromReadOnly() throws Exception { // Expected } - bkc.waitForReadOnlyBookie(Bookie.getBookieAddress(bsConfs.get(1))) + bkc.waitForReadOnlyBookie(Bookie.getBookieId(bsConfs.get(1))) .get(30, TimeUnit.SECONDS); LOG.info("bookie is running {}, readonly {}.", bookie.isRunning(), bookie.isReadOnly()); @@ -145,7 +145,7 @@ public void testBookieShouldTurnWritableFromReadOnly() throws Exception { // Now add the current ledger dir back to writable dirs list ledgerDirsManager.addToWritableDirs(testDir, true); - bkc.waitForWritableBookie(Bookie.getBookieAddress(bsConfs.get(1))) + bkc.waitForWritableBookie(Bookie.getBookieId(bsConfs.get(1))) .get(30, TimeUnit.SECONDS); LOG.info("bookie is running {}, readonly {}.", bookie.isRunning(), bookie.isReadOnly()); @@ -262,7 +262,7 @@ public void testLedgerCreationShouldFailWithReadonlyBookie() throws Exception { startNewBookie(); bs.get(1).getBookie().getStateManager().doTransitionToReadOnlyMode(); try { - bkc.waitForReadOnlyBookie(Bookie.getBookieAddress(bsConfs.get(1))) + bkc.waitForReadOnlyBookie(Bookie.getBookieId(bsConfs.get(1))) .get(30, TimeUnit.SECONDS); bkc.createLedger(2, 2, DigestType.CRC32, "".getBytes()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java index cd5d97ed1d8..dc9ce40dfe6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java @@ -56,7 +56,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieConnectionPeer; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.proto.ClientConnectionPeer; @@ -556,11 +556,13 @@ public void testTLSClientButOnlyFewTLSServers() throws Exception { ClientConfiguration clientConf = new ClientConfiguration(baseClientConf); LedgerMetadata metadata = testClient(clientConf, 2); assertTrue(metadata.getAllEnsembles().size() > 0); - Collection> ensembles = metadata.getAllEnsembles().values(); - for (List bookies : ensembles) { - for (BookieSocketAddress bookieAddress : bookies) { - int port = bookieAddress.getPort(); - assertTrue(tlsBookiePorts.contains(port)); + Collection> ensembles = metadata.getAllEnsembles().values(); + try (BookKeeper client = new BookKeeper(clientConf)) { + for (List bookies : ensembles) { + for (BookieId bookieAddress : bookies) { + int port = client.getBookieAddressResolver().resolve(bookieAddress).getPort(); + assertTrue(tlsBookiePorts.contains(port)); + } } } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java index b21d8aa683e..f82310adfa5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy; import org.apache.bookkeeper.client.RackChangeNotifier; import org.apache.bookkeeper.net.AbstractDNSToSwitchMapping; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; @@ -96,10 +97,12 @@ public void registerRackChangeListener(ITopologyAwareEnsemblePlacementPolicy bookieAddressList, List rack) { + List bookieIds = new ArrayList<>(); for (int i = 0; i < bookieAddressList.size(); i++) { BookieSocketAddress bkAddress = bookieAddressList.get(i); name2Racks.put(bkAddress.getHostName(), rack.get(i)); + bookieIds.add(bkAddress.toBookieId()); } - rackawarePolicy.onBookieRackChange(bookieAddressList); + rackawarePolicy.onBookieRackChange(bookieIds); } } diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java index 76d731c736d..77ecb6ea00d 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationClient.java @@ -21,7 +21,6 @@ import com.coreos.jetcd.Client; import com.coreos.jetcd.data.ByteSequence; import com.google.common.collect.Maps; -import java.net.UnknownHostException; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -32,7 +31,7 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.metadata.etcd.helpers.KeySetReader; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.versioning.Versioned; /** @@ -41,23 +40,19 @@ @Slf4j class EtcdRegistrationClient implements RegistrationClient { - private static Function newBookieSocketAddressFunc(String prefix) { + private static Function newBookieSocketAddressFunc(String prefix) { return bs -> { String addrStr = bs.toStringUtf8(); - try { - return new BookieSocketAddress(addrStr.replace(prefix, "")); - } catch (UnknownHostException e) { - throw new RuntimeException("Unknown bookie address '" + addrStr + "' : ", e); - } + return BookieId.parse(addrStr.replace(prefix, "")); }; } private final EtcdWatchClient watchClient; - private final KeySetReader writableBookiesReader; - private final KeySetReader readonlyBookiesReader; - private Map>>> writableListeners = + private final KeySetReader writableBookiesReader; + private final KeySetReader readonlyBookiesReader; + private Map>>> writableListeners = Maps.newHashMap(); - private Map>>> readonlyListeners = + private Map>>> readonlyListeners = Maps.newHashMap(); EtcdRegistrationClient(String scope, @@ -88,26 +83,26 @@ public void close() { } @Override - public CompletableFuture>> getWritableBookies() { + public CompletableFuture>> getWritableBookies() { return writableBookiesReader.read(); } @Override - public CompletableFuture>> getAllBookies() { + public CompletableFuture>> getAllBookies() { return FutureUtils.exception(new BKException.BKIllegalOpException()); } @Override - public CompletableFuture>> getReadOnlyBookies() { + public CompletableFuture>> getReadOnlyBookies() { return readonlyBookiesReader.read(); } private static CompletableFuture registerListener( - KeySetReader keySetReader, - Map>>> listeners, + KeySetReader keySetReader, + Map>>> listeners, RegistrationListener listener ) { - Consumer>> consumer; + Consumer>> consumer; synchronized (listeners) { consumer = listeners.get(listener); if (null != consumer) { @@ -124,11 +119,11 @@ private static CompletableFuture registerListener( } private static CompletableFuture unregisterListener( - KeySetReader keySetReader, - Map>>> listeners, + KeySetReader keySetReader, + Map>>> listeners, RegistrationListener listener ) { - Consumer>> consumer = listeners.get(listener); + Consumer>> consumer = listeners.get(listener); if (null == consumer) { return FutureUtils.Void(); } else { diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java index 10015ad3da4..27b5a4d09f7 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationManager.java @@ -71,6 +71,7 @@ import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.LedgerLayout; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; @@ -146,7 +147,7 @@ public void close() { } @Override - public void registerBookie(String bookieId, boolean readOnly, + public void registerBookie(BookieId bookieId, boolean readOnly, BookieServiceInfo bookieServiceInfo) throws BookieException { if (readOnly) { doRegisterReadonlyBookie(bookieId, bkRegister.get()); @@ -267,7 +268,7 @@ private void doRegisterBookie(String regPath, long leaseId) throws MetadataStore } } - private void doRegisterReadonlyBookie(String bookieId, long leaseId) throws MetadataStoreException { + private void doRegisterReadonlyBookie(BookieId bookieId, long leaseId) throws MetadataStoreException { String readonlyRegPath = getReadonlyBookiePath(scope, bookieId); doRegisterBookie(readonlyRegPath, leaseId); String writableRegPath = getWritableBookiePath(scope, bookieId); @@ -275,7 +276,7 @@ private void doRegisterReadonlyBookie(String bookieId, long leaseId) throws Meta } @Override - public void unregisterBookie(String bookieId, boolean readOnly) throws BookieException { + public void unregisterBookie(BookieId bookieId, boolean readOnly) throws BookieException { String regPath; if (readOnly) { regPath = getReadonlyBookiePath(scope, bookieId); @@ -291,7 +292,7 @@ public void unregisterBookie(String bookieId, boolean readOnly) throws BookieExc } @Override - public boolean isBookieRegistered(String bookieId) throws BookieException { + public boolean isBookieRegistered(BookieId bookieId) throws BookieException { CompletableFuture getWritableFuture = kvClient.get( ByteSequence.fromString(getWritableBookiePath(scope, bookieId)), GetOption.newBuilder() @@ -308,7 +309,7 @@ public boolean isBookieRegistered(String bookieId) throws BookieException { } @Override - public void writeCookie(String bookieId, Versioned cookieData) throws BookieException { + public void writeCookie(BookieId bookieId, Versioned cookieData) throws BookieException { ByteSequence cookiePath = ByteSequence.fromString(getCookiePath(scope, bookieId)); Txn txn = kvClient.txn(); if (Version.NEW == cookieData.getVersion()) { @@ -346,11 +347,11 @@ public void writeCookie(String bookieId, Versioned cookieData) throws Bo } @Override - public Versioned readCookie(String bookieId) throws BookieException { + public Versioned readCookie(BookieId bookieId) throws BookieException { ByteSequence cookiePath = ByteSequence.fromString(getCookiePath(scope, bookieId)); GetResponse resp = msResult(kvClient.get(cookiePath)); if (resp.getCount() <= 0) { - throw new CookieNotFoundException(bookieId); + throw new CookieNotFoundException(bookieId.toString()); } else { KeyValue kv = resp.getKvs().get(0); return new Versioned<>( @@ -360,7 +361,7 @@ public Versioned readCookie(String bookieId) throws BookieException { } @Override - public void removeCookie(String bookieId, Version version) throws BookieException { + public void removeCookie(BookieId bookieId, Version version) throws BookieException { ByteSequence cookiePath = ByteSequence.fromString(getCookiePath(scope, bookieId)); Txn delTxn = kvClient.txn() .If(new Cmp( @@ -384,7 +385,7 @@ public void removeCookie(String bookieId, Version version) throws BookieExceptio "Failed to remove cookie from " + cookiePath.toStringUtf8() + " for bookie " + bookieId + " : bad version '" + version + "'"); } else { - throw new CookieNotFoundException(bookieId); + throw new CookieNotFoundException(bookieId.toString()); } } else { log.info("Removed cookie from {} for bookie {}", diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdUtils.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdUtils.java index 296ff6c5092..e8cf87a68d8 100644 --- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdUtils.java +++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdUtils.java @@ -38,6 +38,7 @@ import lombok.NoArgsConstructor; import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.net.BookieId; import org.apache.commons.lang3.StringUtils; /** @@ -103,9 +104,9 @@ static String getWritableBookiesEndPath(String scope) { return String.format("%s/%s/%s%s", scope, MEMBERS_NODE, WRITEABLE_NODE, END_SEP); } - static String getWritableBookiePath(String scope, String bookieId) { + static String getWritableBookiePath(String scope, BookieId bookieId) { return String.format("%s/%s/%s/%s", - scope, MEMBERS_NODE, WRITEABLE_NODE, bookieId); + scope, MEMBERS_NODE, WRITEABLE_NODE, bookieId.toString()); } static String getReadonlyBookiesPath(String scope) { @@ -120,17 +121,17 @@ static String getReadonlyBookiesEndPath(String scope) { return String.format("%s/%s/%s%s", scope, MEMBERS_NODE, READONLY_NODE, END_SEP); } - static String getReadonlyBookiePath(String scope, String bookieId) { + static String getReadonlyBookiePath(String scope, BookieId bookieId) { return String.format("%s/%s/%s/%s", - scope, MEMBERS_NODE, READONLY_NODE, bookieId); + scope, MEMBERS_NODE, READONLY_NODE, bookieId.toString()); } static String getCookiesPath(String scope) { return String.format("%s/%s", scope, COOKIES_NODE); } - static String getCookiePath(String scope, String bookieId) { - return String.format("%s/%s/%s", scope, COOKIES_NODE, bookieId); + static String getCookiePath(String scope, BookieId bookieId) { + return String.format("%s/%s/%s", scope, COOKIES_NODE, bookieId.toString()); } static String getClusterInstanceIdPath(String scope) { diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdClusterTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdClusterTest.java index c13e56f84ba..1a7488d584a 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdClusterTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdClusterTest.java @@ -45,6 +45,7 @@ import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.LedgerLayout; import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +import org.apache.bookkeeper.net.BookieId; import org.apache.commons.lang.RandomStringUtils; import org.junit.After; import org.junit.Before; @@ -202,7 +203,7 @@ private static void createNumBookies(Client client, int numBookies, boolean readonly) throws Exception { for (int i = 0; i < numBookies; i++) { - String bookieId = "bookie-" + i + ":3181"; + BookieId bookieId = BookieId.parse("bookie-" + i + ":3181"); String bookiePath; if (readonly) { bookiePath = EtcdUtils.getReadonlyBookiePath(scope, bookieId); @@ -221,7 +222,7 @@ private static void removeNumBookies(Client client, int numBookies, boolean readonly) throws Exception { for (int i = 0; i < numBookies; i++) { - String bookieId = "bookie-" + i + ":3181"; + BookieId bookieId = BookieId.parse("bookie-" + i + ":3181"); String bookiePath; if (readonly) { bookiePath = EtcdUtils.getReadonlyBookiePath(scope, bookieId); diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdCookieTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdCookieTest.java index 692ff87ca7d..1b0397e4776 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdCookieTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdCookieTest.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Version.Occurred; @@ -74,7 +75,7 @@ private static void assertCookieEquals(Versioned expected, Versioned ensemble = Lists.newArrayList( - new BookieSocketAddress("192.0.2.1", 1234), - new BookieSocketAddress("192.0.2.2", 1234), - new BookieSocketAddress("192.0.2.3", 1234)); + List ensemble = Lists.newArrayList( + BookieId.parse("192.0.2.1:1234"), + BookieId.parse("192.0.2.2:1234"), + BookieId.parse("192.0.2.3:1234")); LedgerMetadata metadata = LedgerMetadataBuilder.create() .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(2) .withPassword("test-password".getBytes(UTF_8)) @@ -352,9 +352,9 @@ public void testRegisterLedgerMetadataListener() throws Exception { assertNull(metadataQueue2.poll()); } - static List createNumBookies(int numBookies) { + static List createNumBookies(int numBookies) { return IntStream.range(0, numBookies) - .mapToObj(idx -> new BookieSocketAddress("127.0.0.1", 3181 + idx)) + .mapToObj(idx -> BookieId.parse("127.0.0.1:" + (3181 + idx))) .collect(Collectors.toList()); } } diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java index 0581a2c71ee..7713d53ed17 100644 --- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java +++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdRegistrationTest.java @@ -46,7 +46,7 @@ import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; import org.apache.bookkeeper.metadata.etcd.testing.EtcdTestBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Version.Occurred; @@ -64,8 +64,8 @@ @Slf4j public class EtcdRegistrationTest extends EtcdTestBase { - static String newBookie(int i) { - return "127.0.0.1:" + (3181 + i); + static BookieId newBookie(int i) { + return BookieId.parse("127.0.0.1:" + (3181 + i)); } @Rule @@ -75,7 +75,7 @@ static String newBookie(int i) { private RegistrationClient regClient; protected static RegistrationListener newRegistrationListener( - LinkedBlockingQueue>> notifications) { + LinkedBlockingQueue>> notifications) { return bookies -> { log.info("Received new bookies: {}", bookies); try { @@ -133,12 +133,12 @@ public void testRegisterReadonlyBookies() throws Exception { private void testRegisterBookie(boolean readonly) throws Exception { runNumBookiesTest(scope, 3, readonly, (scope, numBookies, ro) -> { - Set expectedBookies = Sets.newHashSet(); + Set expectedBookies = Sets.newHashSet(); for (int i = 0; i < numBookies; i++) { - expectedBookies.add(new BookieSocketAddress(newBookie(i))); + expectedBookies.add(newBookie(i)); } - Set writableBookies = result(regClient.getWritableBookies()).getValue(); - Set readonlyBookies = result(regClient.getReadOnlyBookies()).getValue(); + Set writableBookies = result(regClient.getWritableBookies()).getValue(); + Set readonlyBookies = result(regClient.getReadOnlyBookies()).getValue(); if (ro) { assertEquals(0, writableBookies.size()); assertEquals(numBookies, readonlyBookies.size()); @@ -163,11 +163,11 @@ public void testWatchReadonlyBookies() throws Exception { } private void testWatchBookies(boolean readonly) throws Exception { - LinkedBlockingQueue>> writableChanges = new LinkedBlockingQueue<>(); - LinkedBlockingQueue>> readonlyChanges = new LinkedBlockingQueue<>(); + LinkedBlockingQueue>> writableChanges = new LinkedBlockingQueue<>(); + LinkedBlockingQueue>> readonlyChanges = new LinkedBlockingQueue<>(); result(regClient.watchReadOnlyBookies(newRegistrationListener(readonlyChanges))); result(regClient.watchWritableBookies(newRegistrationListener(writableChanges))); - Versioned> versionedBookies = writableChanges.take(); + Versioned> versionedBookies = writableChanges.take(); assertTrue(versionedBookies.getValue().isEmpty()); versionedBookies = readonlyChanges.take(); assertTrue(versionedBookies.getValue().isEmpty()); @@ -175,7 +175,7 @@ private void testWatchBookies(boolean readonly) throws Exception { final int numBookies = 3; final List bookies = createNumBookies(readonly, numBookies, scope, 1); - LinkedBlockingQueue>> changes; + LinkedBlockingQueue>> changes; if (readonly) { changes = readonlyChanges; } else { @@ -183,9 +183,9 @@ private void testWatchBookies(boolean readonly) throws Exception { } Version preVersion = new LongVersion(-1); - Set expectedBookies = new HashSet<>(); + Set expectedBookies = new HashSet<>(); for (int i = 0; i < numBookies; i++) { - BookieSocketAddress address = new BookieSocketAddress(newBookie(i)); + BookieId address = newBookie(i); expectedBookies.add(address); versionedBookies = changes.take(); @@ -234,7 +234,7 @@ private static List createNumBookies(boolean readonly, public void testRegisterBookieWaitUntilPreviousExpiredSuccess() throws Exception { long ttlSeconds = 1; long leaseId = -0xabcd; - String bookieId = runtime.getMethodName() + ":3181"; + BookieId bookieId = BookieId.parse(runtime.getMethodName() + ":3181"); try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( newEtcdClient(), scope, ttlSeconds) ) { @@ -258,7 +258,7 @@ public void testRegisterBookieWaitUntilPreviousExpiredSuccess() throws Exception public void testRegisterBookieWaitUntilPreviousExpiredFailure() throws Exception { long ttlSeconds = 1; long leaseId = -0xabcd; - String bookieId = runtime.getMethodName() + ":3181"; + BookieId bookieId = BookieId.parse(runtime.getMethodName() + ":3181"); try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( newEtcdClient(), scope, 10000000 * ttlSeconds) ) { @@ -292,7 +292,7 @@ public void testRegisterReadonlyBookieWithSameLeaseId() throws Exception { private void testRegisterBookieWithSameLeaseId(boolean readonly) throws Exception { long ttlSeconds = 1; long leaseId = -0xabcd; - String bookieId = runtime.getMethodName() + ":3181"; + BookieId bookieId = BookieId.parse(runtime.getMethodName() + ":3181"); try (EtcdRegistrationManager regManager = new EtcdRegistrationManager( newEtcdClient(), scope, 10000000 * ttlSeconds) ) { @@ -308,8 +308,8 @@ private void testRegisterBookieWithSameLeaseId(boolean readonly) throws Exceptio } } - private Set getBookies(boolean readonly) throws Exception { - Set bookies; + private Set getBookies(boolean readonly) throws Exception { + Set bookies; if (readonly) { bookies = result(regClient.getReadOnlyBookies()).getValue(); } else { @@ -329,16 +329,17 @@ public void testRegisterUnregisterReadonlyBookie() throws Exception { } private void testRegisterUnregister(boolean readonly) throws Exception { - String bookieId = runtime.getMethodName(); + String bookieIdStr = runtime.getMethodName(); if (readonly) { - bookieId += "-readonly"; + bookieIdStr += "-readonly"; } - bookieId += ":3181"; + bookieIdStr += ":3181"; + BookieId bookieId = BookieId.parse(bookieIdStr); try (EtcdRegistrationManager regMgr = new EtcdRegistrationManager( newEtcdClient(), scope, 1000000000 )) { // before registration - Set bookies = getBookies(readonly); + Set bookies = getBookies(readonly); log.info("before registration : bookies = {}", bookies); assertEquals(0, bookies.size()); // registered @@ -347,7 +348,7 @@ private void testRegisterUnregister(boolean readonly) throws Exception { log.info("after registered: bookies = {}", bookies); assertEquals(1, bookies.size()); assertEquals( - Sets.newHashSet(new BookieSocketAddress(bookieId)), + Sets.newHashSet(bookieId), bookies); // unregistered regMgr.unregisterBookie(bookieId, readonly); @@ -368,11 +369,11 @@ public void testConcurrentReadonlyRegistration() throws Exception { } private void testConcurrentRegistration(boolean readonly) throws Exception { - final String bookieId; + final BookieId bookieId; if (readonly) { - bookieId = runtime.getMethodName() + "-readonly:3181"; + bookieId = BookieId.parse(runtime.getMethodName() + "-readonly:3181"); } else { - bookieId = runtime.getMethodName() + ":3181"; + bookieId = BookieId.parse(runtime.getMethodName() + ":3181"); } final int numBookies = 10; @Cleanup("shutdown") diff --git a/pom.xml b/pom.xml index 86802c3dd84..9f5e01651ac 100644 --- a/pom.xml +++ b/pom.xml @@ -860,6 +860,7 @@ ${redirectTestOutputToFile} ${forkCount.variable} false + false 1800 ${testRetryCount} diff --git a/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolver.java b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolver.java index cc25978079b..7e39471d657 100644 --- a/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolver.java +++ b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolver.java @@ -34,10 +34,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; +import org.apache.bookkeeper.client.DefaultBookieAddressResolver; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.MetadataClientDriver; import org.apache.bookkeeper.meta.exceptions.MetadataException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.NullStatsLogger; /** @@ -52,11 +53,13 @@ class BKRegistrationNameResolver extends NameResolver { private Listener listener; private boolean shutdown; private boolean resolving; + private DefaultBookieAddressResolver bookieAddressResolver; BKRegistrationNameResolver(MetadataClientDriver clientDriver, URI serviceURI) { this.clientDriver = clientDriver; this.serviceURI = serviceURI; + this.bookieAddressResolver = new DefaultBookieAddressResolver(clientDriver.getRegistrationClient()); this.executor = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("registration-name-resolver").build()); } @@ -110,10 +113,10 @@ private synchronized void resolve() { }); } - private static List hostsToEquivalentAddressGroups(Set bookies) { + private List hostsToEquivalentAddressGroups(Set bookies) { return bookies.stream() .map(addr -> new EquivalentAddressGroup( - Collections.singletonList(addr.getSocketAddress()), + Collections.singletonList(bookieAddressResolver.resolve(addr).getSocketAddress()), Attributes.EMPTY )) .collect(Collectors.toList()); diff --git a/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java index f877aa7ba94..93544407369 100644 --- a/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java +++ b/stream/bk-grpc-name-resolver/src/test/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverTest.java @@ -40,6 +40,7 @@ import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.zookeeper.CreateMode; @@ -103,7 +104,7 @@ public void testNameResolver() throws Exception { InetSocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i); addressSet.add(address); bookieDriver.getRegistrationManager().registerBookie( - "127.0.0.1:" + (3181 + i), false, BookieServiceInfo.EMPTY + BookieId.parse("127.0.0.1:" + (3181 + i)), false, BookieServiceInfo.EMPTY ); } @@ -138,7 +139,7 @@ public void onError(Status error) { InetSocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i); addressSet.add(address); bookieDriver.getRegistrationManager().registerBookie( - "127.0.0.1:" + (3181 + i), false, BookieServiceInfo.EMPTY + BookieId.parse("127.0.0.1:" + (3181 + i)), false, BookieServiceInfo.EMPTY ); } diff --git a/stream/distributedlog/core/pom.xml b/stream/distributedlog/core/pom.xml index c08f9390e32..84c4fd2e319 100644 --- a/stream/distributedlog/core/pom.xml +++ b/stream/distributedlog/core/pom.xml @@ -92,6 +92,7 @@ maven-surefire-plugin ${maven-surefire-plugin.version} + false ${redirectTestOutputToFile} -Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G always diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java index 06ed4d2b5e6..f5850bf0ba0 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.client.api.LedgerEntries; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; import org.apache.bookkeeper.common.concurrent.FutureEventListener; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; @@ -82,7 +83,7 @@ public LedgerReader(BookKeeper bkc) { clientCtx = bkc.getClientCtx(); } - public static SortedMap> bookiesForLedger(final LedgerHandle lh) { + public static SortedMap> bookiesForLedger(final LedgerHandle lh) { return lh.getLedgerMetadata().getAllEnsembles(); } @@ -119,11 +120,13 @@ public void readEntryComplete(int rc, long lid, long eid, ByteBuf buffer, Object } }; - List ensemble = lh.getLedgerMetadata().getEnsembleAt(eid); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(eid); for (int i = 0; i < writeSet.size(); i++) { int idx = writeSet.get(i); + BookieSocketAddress resolvedNetworkAddress = + clientCtx.getBookieWatcher().getBookieAddressResolver().resolve(ensemble.get(idx)); clientCtx.getBookieClient().readEntry(ensemble.get(idx), lh.getId(), eid, readEntryCallback, - ensemble.get(idx), BookieProtocol.FLAG_NONE); + resolvedNetworkAddress, BookieProtocol.FLAG_NONE); } } @@ -230,7 +233,7 @@ public void readLacs(final LedgerHandle lh, long eid, } }; - List ensemble = lh.getLedgerMetadata().getEnsembleAt(eid); + List ensemble = lh.getLedgerMetadata().getEnsembleAt(eid); for (int i = 0; i < writeSet.size(); i++) { int idx = writeSet.get(i); clientCtx.getBookieClient().readEntry(ensemble.get(idx), lh.getId(), eid, readEntryCallback, diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java index f8445ba6f89..bec9abff72d 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java @@ -64,7 +64,7 @@ import org.apache.bookkeeper.client.LedgerReader; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.util.IOUtils; import org.apache.commons.cli.CommandLine; @@ -1000,29 +1000,29 @@ boolean include(LogSegmentMetadata segment) { private void printEppStatsHeader(DistributedLogManager dlm) throws Exception { String label = "Ledger Placement :"; System.out.println(label); - Map totals = new HashMap(); + Map totals = new HashMap(); List segments = dlm.getLogSegments(); for (LogSegmentMetadata segment : segments) { if (include(segment)) { merge(totals, getBookieStats(segment)); } } - List> entries = - new ArrayList>(totals.entrySet()); - Collections.sort(entries, new Comparator>() { + List> entries = + new ArrayList>(totals.entrySet()); + Collections.sort(entries, new Comparator>() { @Override - public int compare(Map.Entry o1, - Map.Entry o2) { + public int compare(Map.Entry o1, + Map.Entry o2) { return o2.getValue() - o1.getValue(); } }); int width = 0; int totalEntries = 0; - for (Map.Entry entry : entries) { + for (Map.Entry entry : entries) { width = Math.max(width, label.length() + 1 + entry.getKey().toString().length()); totalEntries += entry.getValue(); } - for (Map.Entry entry : entries) { + for (Map.Entry entry : entries) { System.out.println(String.format("%" + width + "s\t%6.2f%%\t\t%d", entry.getKey(), entry.getValue() * 1.0 / totalEntries, entry.getValue())); } @@ -1032,16 +1032,16 @@ private void printLedgerRow(LogSegmentMetadata segment) throws Exception { System.out.println(segment.getLogSegmentSequenceNumber() + "\t: " + segment); } - private Map getBookieStats(LogSegmentMetadata segment) throws Exception { - Map stats = new HashMap(); + private Map getBookieStats(LogSegmentMetadata segment) throws Exception { + Map stats = new HashMap(); LedgerHandle lh = bkc.client().get().openLedgerNoRecovery(segment.getLogSegmentId(), BookKeeper.DigestType.CRC32, getConf().getBKDigestPW().getBytes(UTF_8)); long eidFirst = 0; - for (SortedMap.Entry> + for (SortedMap.Entry> entry : LedgerReader.bookiesForLedger(lh).entrySet()) { long eidLast = entry.getKey().longValue(); long count = eidLast - eidFirst + 1; - for (BookieSocketAddress bookie : entry.getValue()) { + for (BookieId bookie : entry.getValue()) { merge(stats, bookie, (int) count); } eidFirst = eidLast; @@ -1049,7 +1049,7 @@ private Map getBookieStats(LogSegmentMetadata segm return stats; } - void merge(Map m, BookieSocketAddress bookie, Integer count) { + void merge(Map m, BookieId bookie, Integer count) { if (m.containsKey(bookie)) { m.put(bookie, count + m.get(bookie).intValue()); } else { @@ -1057,8 +1057,8 @@ void merge(Map m, BookieSocketAddress bookie, Inte } } - void merge(Map m1, Map m2) { - for (Map.Entry entry : m2.entrySet()) { + void merge(Map m1, Map m2) { + for (Map.Entry entry : m2.entrySet()) { merge(m1, entry.getKey(), entry.getValue()); } } diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieWatchService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieWatchService.java index 4a3741f9f0f..fb9f3833df3 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieWatchService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieWatchService.java @@ -32,7 +32,7 @@ import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.meta.exceptions.MetadataException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stream.server.conf.BookieConfiguration; @@ -77,7 +77,7 @@ protected void doStart() { private static void waitingForNumBookies(RegistrationClient client, int minNumBookies) throws Exception { Stopwatch stopwatch = Stopwatch.createStarted(); - Set bookies = FutureUtils.result(client.getWritableBookies()).getValue(); + Set bookies = FutureUtils.result(client.getWritableBookies()).getValue(); while (bookies.size() < minNumBookies) { TimeUnit.SECONDS.sleep(1); bookies = FutureUtils.result(client.getWritableBookies()).getValue(); diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java index 2485c1791f9..df34e87f4a1 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.discover.BookieServiceInfo; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.discover.ZKRegistrationManager; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stream.proto.common.Endpoint; import org.apache.bookkeeper.stream.server.conf.BookieConfiguration; @@ -80,7 +81,7 @@ protected void doStart() { statsLogger.scope("state"), () -> regManager, Collections.emptyList(), - () -> NetUtils.endpointToString(myEndpoint), + () -> BookieId.parse(NetUtils.endpointToString(myEndpoint)), BookieServiceInfo.NO_INFO); stateManager.initState(); stateManager.registerBookie(true).get(); diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImpl.java index f117b8b96b3..97b41f3eae3 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImpl.java @@ -27,7 +27,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stream.proto.cluster.ClusterAssignmentData; import org.apache.bookkeeper.stream.proto.cluster.ClusterMetadata; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterControllerLeader; @@ -55,7 +55,7 @@ public class ClusterControllerLeaderImpl implements ClusterControllerLeader, Reg private final RegistrationClient regClient; // keep a reference to a set of available servers - private volatile Set availableServers; + private volatile Set availableServers; // variables for suspending controller @Getter(AccessLevel.PACKAGE) @@ -172,7 +172,7 @@ private void processServerChange() throws InterruptedException { // now, the controller has permits and meet the time requirement for assigning containers. performServerChangesPermits.drainPermits(); - Set availableServersSnapshot = availableServers; + Set availableServersSnapshot = availableServers; if (null == availableServersSnapshot || availableServersSnapshot.isEmpty()) { // haven't received any servers from registration service, wait for 200ms and retry. if (lastSuccessfulAssigmentAt < 0) { @@ -210,7 +210,7 @@ private void processServerChange() throws InterruptedException { } @Override - public void onBookiesChanged(Versioned> bookies) { + public void onBookiesChanged(Versioned> bookies) { log.info("Cluster topology is changed - new cluster : {}", bookies); // when bookies are changed, notify the leader to take actions this.availableServers = bookies.getValue(); diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerController.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerController.java index c68f4bb1cf8..d65cf92d2a0 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerController.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerController.java @@ -22,7 +22,6 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.util.concurrent.UncheckedExecutionException; -import java.net.UnknownHostException; import java.util.Collections; import java.util.Comparator; import java.util.LinkedList; @@ -33,7 +32,7 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stream.proto.cluster.ClusterAssignmentData; import org.apache.bookkeeper.stream.proto.cluster.ClusterMetadata; import org.apache.bookkeeper.stream.proto.cluster.ServerAssignmentData; @@ -51,11 +50,11 @@ public class DefaultStorageContainerController implements StorageContainerController { static final class ServerAssignmentDataComparator - implements Comparator>> { + implements Comparator>> { @Override - public int compare(Pair> o1, - Pair> o2) { + public int compare(Pair> o1, + Pair> o2) { int res = Integer.compare(o1.getValue().size(), o2.getValue().size()); if (0 == res) { // two servers have same number of container @@ -73,7 +72,7 @@ public int compare(Pair> o1, @Override public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, ClusterAssignmentData currentState, - Set currentCluster) { + Set currentCluster) { if (currentCluster.isEmpty()) { log.info("Current cluster is empty. No alive server is found."); @@ -81,20 +80,13 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, } // 1. get current server assignments - Map> currentServerAssignments; + Map> currentServerAssignments; try { currentServerAssignments = currentState.getServersMap() .entrySet() .stream() - .collect(Collectors.toMap( - e1 -> { - try { - return new BookieSocketAddress(e1.getKey()); - } catch (UnknownHostException uhe) { - log.error("Invalid cluster "); - throw new UncheckedExecutionException("Invalid server found in current assignment map" - + e1.getKey(), uhe); - } + .collect(Collectors.toMap(e1 -> { + return BookieId.parse(e1.getKey()); }, e2 -> e2.getValue().getContainersList().stream().collect(Collectors.toSet()) )); @@ -103,7 +95,7 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, currentState, uee.getCause().getMessage()); currentServerAssignments = Maps.newHashMap(); } - Set currentServersAssigned = currentServerAssignments.keySet(); + Set currentServersAssigned = currentServerAssignments.keySet(); // 2. if no servers is assigned, initialize the ideal state if (currentServersAssigned.isEmpty()) { @@ -111,9 +103,9 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, } // 3. get the cluster diffs - Set serversAdded = + Set serversAdded = Sets.difference(currentCluster, currentServersAssigned).immutableCopy(); - Set serversRemoved = + Set serversRemoved = Sets.difference(currentServersAssigned, currentCluster).immutableCopy(); if (serversAdded.isEmpty() && serversRemoved.isEmpty()) { @@ -132,10 +124,10 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, .collect(Collectors.toSet()); // 5. use an ordered set as priority deque to sort the servers by the number of assigned containers - TreeSet>> assignmentQueue = + TreeSet>> assignmentQueue = new TreeSet<>(new ServerAssignmentDataComparator()); - for (Map.Entry> entry : currentServerAssignments.entrySet()) { - BookieSocketAddress host = entry.getKey(); + for (Map.Entry> entry : currentServerAssignments.entrySet()) { + BookieId host = entry.getKey(); if (!currentCluster.contains(host)) { if (log.isTraceEnabled()) { @@ -151,13 +143,13 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, } // 6. add new servers - for (BookieSocketAddress server : serversAdded) { + for (BookieId server : serversAdded) { assignmentQueue.add(Pair.of(server, Lists.newLinkedList())); } // 7. assign the containers that are needed to be reassigned. for (Long containerId : containersToReassign) { - Pair> leastLoadedServer = assignmentQueue.pollFirst(); + Pair> leastLoadedServer = assignmentQueue.pollFirst(); leastLoadedServer.getValue().add(containerId); assignmentQueue.add(leastLoadedServer); } @@ -170,8 +162,8 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, diffAllowed = clusterMetadata.getNumStorageContainers() % assignmentQueue.size() == 0 ? 0 : 1; } - Pair> leastLoaded = assignmentQueue.first(); - Pair> mostLoaded = assignmentQueue.last(); + Pair> leastLoaded = assignmentQueue.first(); + Pair> mostLoaded = assignmentQueue.last(); while (mostLoaded.getValue().size() - leastLoaded.getValue().size() > diffAllowed) { leastLoaded = assignmentQueue.pollFirst(); mostLoaded = assignmentQueue.pollLast(); @@ -201,8 +193,8 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, } static ClusterAssignmentData initializeIdealState(ClusterMetadata clusterMetadata, - Set currentCluster) { - List serverList = Lists.newArrayListWithExpectedSize(currentCluster.size()); + Set currentCluster) { + List serverList = Lists.newArrayListWithExpectedSize(currentCluster.size()); serverList.addAll(currentCluster); Collections.shuffle(serverList); @@ -212,7 +204,7 @@ static ClusterAssignmentData initializeIdealState(ClusterMetadata clusterMetadat Map assignmentMap = Maps.newHashMap(); for (int serverIdx = 0; serverIdx < serverList.size(); serverIdx++) { - BookieSocketAddress server = serverList.get(serverIdx); + BookieId server = serverList.get(serverIdx); int finalServerIdx = serverIdx; ServerAssignmentData assignmentData = ServerAssignmentData.newBuilder() diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerController.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerController.java index 246f52c58e5..6a7542e90c2 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerController.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerController.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.stream.storage.impl.sc; import java.util.Set; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stream.proto.cluster.ClusterAssignmentData; import org.apache.bookkeeper.stream.proto.cluster.ClusterMetadata; @@ -38,6 +38,6 @@ public interface StorageContainerController { */ ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, ClusterAssignmentData currentState, - Set currentCluster); + Set currentCluster); } diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImplTest.java index 721668ef5a0..6ed0c432dbf 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImplTest.java @@ -44,7 +44,7 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stream.proto.cluster.ClusterAssignmentData; import org.apache.bookkeeper.stream.proto.cluster.ClusterMetadata; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterMetadataStore; @@ -187,8 +187,7 @@ public void testProcessAsLeader() throws Exception { assertTrue(clusterController.getLastSuccessfulAssigmentAt() < 0); // notify the registration client that a new host is added - Set cluster = Sets.newSet( - new BookieSocketAddress("127.0.0.1", 4181)); + Set cluster = Sets.newSet(BookieId.parse("127.0.0.1:4181")); Version version = new LongVersion(0L); regListenerRef.get().onBookiesChanged(new Versioned<>(cluster, version)); @@ -205,10 +204,10 @@ public void testProcessAsLeader() throws Exception { assertEquals(lastSuccessfulAssignmentAt, clusterController.getLastSuccessfulAssigmentAt()); // multiple hosts added and removed - cluster.add(new BookieSocketAddress("127.0.0.1", 4182)); - cluster.add(new BookieSocketAddress("127.0.0.1", 4183)); - cluster.add(new BookieSocketAddress("127.0.0.1", 4184)); - cluster.add(new BookieSocketAddress("127.0.0.1", 4185)); + cluster.add(BookieId.parse("127.0.0.1:4182")); + cluster.add(BookieId.parse("127.0.0.1:4183")); + cluster.add(BookieId.parse("127.0.0.1:4184")); + cluster.add(BookieId.parse("127.0.0.1:4185")); version = new LongVersion(1L); regListenerRef.get().onBookiesChanged(new Versioned<>(cluster, version)); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerControllerTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerControllerTest.java index e152998c395..027964e4aed 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerControllerTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerControllerTest.java @@ -34,6 +34,7 @@ import java.util.stream.IntStream; import java.util.stream.LongStream; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.stream.proto.cluster.ClusterAssignmentData; import org.apache.bookkeeper.stream.proto.cluster.ClusterMetadata; @@ -78,12 +79,12 @@ public void testServerAssignmentDataComparator() { serverList2.add(2L); serverList2.add(3L); - BookieSocketAddress address1 = new BookieSocketAddress("127.0.0.1", 4181); - BookieSocketAddress address2 = new BookieSocketAddress("127.0.0.1", 4182); + BookieId address1 = new BookieSocketAddress("127.0.0.1", 4181).toBookieId(); + BookieId address2 = new BookieSocketAddress("127.0.0.1", 4182).toBookieId(); - Pair> pair1 = Pair.of(address1, serverList1); - Pair> pair2 = Pair.of(address1, serverList2); - Pair> pair3 = Pair.of(address2, serverList2); + Pair> pair1 = Pair.of(address1, serverList1); + Pair> pair2 = Pair.of(address1, serverList2); + Pair> pair3 = Pair.of(address2, serverList2); assertEquals(-1, comparator.compare(pair1, pair2)); assertEquals(-1, comparator.compare(pair1, pair2)); @@ -102,36 +103,36 @@ public void testComputeIdealStateEmptyCluster() { Collections.emptySet())); } - private static Set newCluster(int numServers) { - Set cluster = IntStream.range(0, numServers) - .mapToObj(idx -> new BookieSocketAddress("127.0.0.1", 4181 + idx)) + private static Set newCluster(int numServers) { + Set cluster = IntStream.range(0, numServers) + .mapToObj(idx -> new BookieSocketAddress("127.0.0.1", 4181 + idx).toBookieId()) .collect(Collectors.toSet()); return ImmutableSet.copyOf(cluster); } - private static Set newCluster(int numServers, int startServerIdx) { - Set cluster = IntStream.range(0, numServers) - .mapToObj(idx -> new BookieSocketAddress("127.0.0.1", 4181 + startServerIdx + idx)) + private static Set newCluster(int numServers, int startServerIdx) { + Set cluster = IntStream.range(0, numServers) + .mapToObj(idx -> new BookieSocketAddress("127.0.0.1", 4181 + startServerIdx + idx).toBookieId()) .collect(Collectors.toSet()); return ImmutableSet.copyOf(cluster); } private static void verifyAssignmentData(ClusterAssignmentData newAssignment, - Set currentCluster, + Set currentCluster, boolean isInitialIdealState) throws Exception { int numServers = currentCluster.size(); assertEquals(numServers, newAssignment.getServersCount()); Set assignedContainers = Sets.newHashSet(); - Set assignedServers = Sets.newHashSet(); + Set assignedServers = Sets.newHashSet(); int numContainersPerServer = NUM_STORAGE_CONTAINERS / numServers; int serverIdx = 0; for (Map.Entry entry : newAssignment.getServersMap().entrySet()) { log.info("Check assignment for server {} = {}", entry.getKey(), entry.getValue()); - BookieSocketAddress address = new BookieSocketAddress(entry.getKey()); + BookieId address = BookieId.parse(entry.getKey()); assignedServers.add(address); assertEquals(serverIdx + 1, assignedServers.size()); @@ -160,13 +161,13 @@ private static void verifyAssignmentData(ClusterAssignmentData newAssignment, } private static void verifyAssignmentDataWhenHasMoreServers(ClusterAssignmentData newAssignment, - Set currentCluster) + Set currentCluster) throws Exception { int numServers = currentCluster.size(); assertEquals(numServers, newAssignment.getServersCount()); Set assignedContainers = Sets.newHashSet(); - Set assignedServers = Sets.newHashSet(); + Set assignedServers = Sets.newHashSet(); int numEmptyServers = 0; int numAssignedServers = 0; @@ -174,7 +175,7 @@ private static void verifyAssignmentDataWhenHasMoreServers(ClusterAssignmentData for (Map.Entry entry : newAssignment.getServersMap().entrySet()) { log.info("Check assignment for server {} = {}", entry.getKey(), entry.getValue()); - BookieSocketAddress address = new BookieSocketAddress(entry.getKey()); + BookieId address = BookieId.parse(entry.getKey()); assignedServers.add(address); assertEquals(serverIdx + 1, assignedServers.size()); @@ -209,7 +210,7 @@ public void testComputeIdealStateFromEmptyAssignment() throws Exception { ClusterAssignmentData emptyAssignment = ClusterAssignmentData.newBuilder().build(); int numServers = 8; - Set currentCluster = newCluster(numServers); + Set currentCluster = newCluster(numServers); ClusterAssignmentData newAssignment = controller.computeIdealState( clusterMetadata, @@ -224,7 +225,7 @@ public void testComputeIdealStateIfClusterUnchanged() throws Exception { ClusterAssignmentData emptyAssignment = ClusterAssignmentData.newBuilder().build(); int numServers = 8; - Set currentCluster = newCluster(numServers); + Set currentCluster = newCluster(numServers); ClusterAssignmentData newAssignment = controller.computeIdealState( clusterMetadata, emptyAssignment, @@ -245,7 +246,7 @@ public void testComputeIdealStateWhenHostsRemoved() throws Exception { ClusterAssignmentData emptyAssignment = ClusterAssignmentData.newBuilder().build(); int numServers = 8; - Set currentCluster = newCluster(numServers); + Set currentCluster = newCluster(numServers); ClusterAssignmentData assignmentData = controller.computeIdealState( clusterMetadata, @@ -254,7 +255,7 @@ public void testComputeIdealStateWhenHostsRemoved() throws Exception { verifyAssignmentData(assignmentData, currentCluster, true); int newNumServers = 4; - Set newCluster = newCluster(newNumServers); + Set newCluster = newCluster(newNumServers); ClusterAssignmentData newAssignmentData = controller.computeIdealState( clusterMetadata, @@ -268,7 +269,7 @@ public void testComputeIdealStateWhenHostsAdded() throws Exception { ClusterAssignmentData emptyAssignment = ClusterAssignmentData.newBuilder().build(); int numServers = 4; - Set currentCluster = newCluster(numServers); + Set currentCluster = newCluster(numServers); ClusterAssignmentData assignmentData = controller.computeIdealState( clusterMetadata, @@ -277,7 +278,7 @@ public void testComputeIdealStateWhenHostsAdded() throws Exception { verifyAssignmentData(assignmentData, currentCluster, true); int newNumServers = 8; - Set newCluster = newCluster(newNumServers); + Set newCluster = newCluster(newNumServers); ClusterAssignmentData newAssignmentData = controller.computeIdealState( clusterMetadata, @@ -291,7 +292,7 @@ public void testComputeIdealStateWhenHostsRemovedAdded() throws Exception { ClusterAssignmentData emptyAssignment = ClusterAssignmentData.newBuilder().build(); int numServers = 4; - Set currentCluster = newCluster(numServers); + Set currentCluster = newCluster(numServers); ClusterAssignmentData assignmentData = controller.computeIdealState( clusterMetadata, @@ -299,10 +300,10 @@ public void testComputeIdealStateWhenHostsRemovedAdded() throws Exception { currentCluster); verifyAssignmentData(assignmentData, currentCluster, true); - Set serversToAdd = newCluster(6, numServers); - Set serversToRemove = newCluster(2); + Set serversToAdd = newCluster(6, numServers); + Set serversToRemove = newCluster(2); - Set newCluster = Sets.newHashSet(currentCluster); + Set newCluster = Sets.newHashSet(currentCluster); newCluster.addAll(serversToAdd); serversToRemove.forEach(newCluster::remove); @@ -318,7 +319,7 @@ public void testComputeIdealStateWhenHasMoreServers() throws Exception { ClusterAssignmentData emptyAssignment = ClusterAssignmentData.newBuilder().build(); int numServers = 2 * NUM_STORAGE_CONTAINERS; - Set currentCluster = newCluster(numServers); + Set currentCluster = newCluster(numServers); ClusterAssignmentData assignmentData = controller.computeIdealState( clusterMetadata, @@ -332,7 +333,7 @@ public void testComputeIdealStateWhenScaleToMoreServers() throws Exception { ClusterAssignmentData emptyAssignment = ClusterAssignmentData.newBuilder().build(); int numServers = 4; - Set currentCluster = newCluster(numServers); + Set currentCluster = newCluster(numServers); ClusterAssignmentData assignmentData = controller.computeIdealState( clusterMetadata, @@ -341,7 +342,7 @@ public void testComputeIdealStateWhenScaleToMoreServers() throws Exception { verifyAssignmentData(assignmentData, currentCluster, true); numServers = 2 * NUM_STORAGE_CONTAINERS; - Set newCluster = newCluster(numServers); + Set newCluster = newCluster(numServers); ClusterAssignmentData newAssignment = controller.computeIdealState( clusterMetadata, assignmentData, diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/BookKeeperClusterTestBase.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/BookKeeperClusterTestBase.java index 73a458d1b6d..b15e917e250 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/BookKeeperClusterTestBase.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/BookKeeperClusterTestBase.java @@ -27,11 +27,12 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.DefaultBookieAddressResolver; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.MetadataClientDriver; import org.apache.bookkeeper.meta.MetadataDrivers; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.tests.integration.topologies.BKCluster; import org.apache.bookkeeper.tests.integration.topologies.BKClusterSpec; @@ -97,10 +98,13 @@ public static void teardownCluster() { } private static boolean findIfBookieRegistered(String bookieName) throws Exception { - Set bookies = - FutureUtils.result(metadataClientDriver.getRegistrationClient().getWritableBookies()).getValue(); - Optional registered = - bookies.stream().filter(addr -> addr.getHostName().equals(bookieName)).findFirst(); + DefaultBookieAddressResolver resolver = + new DefaultBookieAddressResolver(metadataClientDriver.getRegistrationClient()); + Set bookies = + FutureUtils.result(metadataClientDriver + .getRegistrationClient().getWritableBookies()).getValue(); + Optional registered = + bookies.stream().filter(addr -> resolver.resolve(addr).getHostName().equals(bookieName)).findFirst(); return registered.isPresent(); } diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/SimpleClusterTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/SimpleClusterTest.java index 9d09c67c467..0f79cbc8dd0 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/SimpleClusterTest.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/cluster/SimpleClusterTest.java @@ -27,7 +27,7 @@ import java.util.Set; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tests.containers.BookieContainer; import org.junit.FixMethodOrder; import org.junit.Test; @@ -42,7 +42,7 @@ public class SimpleClusterTest extends BookKeeperClusterTestBase { @Test public void test000_ClusterIsEmpty() throws Exception { - Set bookies = + Set bookies = FutureUtils.result(metadataClientDriver.getRegistrationClient().getWritableBookies()).getValue(); assertTrue(bookies.isEmpty()); } @@ -55,7 +55,7 @@ public void test001_StartBookie() throws Exception { assertEquals(1, bkCluster.getBookieContainers().size()); assertSame(container, bkCluster.getBookie(bookieName)); - Set bookies = + Set bookies = FutureUtils.result(metadataClientDriver.getRegistrationClient().getWritableBookies()).getValue(); assertEquals(1, bookies.size()); } @@ -70,7 +70,7 @@ public void test002_StopBookie() throws Exception { waitUntilBookieUnregistered(bookieName); - Set bookies = + Set bookies = FutureUtils.result(metadataClientDriver.getRegistrationClient().getWritableBookies()).getValue(); assertEquals(0, bookies.size()); } diff --git a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java index 781d28e8aa8..66a78e9ca65 100644 --- a/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java +++ b/tests/integration/smoke/src/test/java/org/apache/bookkeeper/tests/integration/TestCLI.java @@ -27,7 +27,7 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.WriteHandle; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tests.integration.utils.BookKeeperClusterUtils; import org.apache.bookkeeper.tests.integration.utils.DockerUtils; import org.jboss.arquillian.junit.Arquillian; @@ -116,8 +116,8 @@ public void test004_SearchReplaceBookieId() throws Exception { int numEntries = 100; try (BookKeeper bk = new BookKeeper(zookeeper)) { long ledgerId; - BookieSocketAddress toReplace; - BookieSocketAddress replaceWith = new BookieSocketAddress("192.0.2.1:3181"); + BookieId toReplace; + BookieId replaceWith = BookieId.parse("192.0.2.1:3181"); try (WriteHandle writelh = bk.newCreateLedgerOp() .withDigestType(DigestType.CRC32C).withPassword(TestSmoke.PASSWD) .withEnsembleSize(1).withWriteQuorumSize(1).withAckQuorumSize(1).execute().get()) { diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/SearchReplaceBookieIdCommand.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/SearchReplaceBookieIdCommand.java index d2486b44584..bff389579dd 100644 --- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/SearchReplaceBookieIdCommand.java +++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookieid/SearchReplaceBookieIdCommand.java @@ -30,7 +30,7 @@ import org.apache.bookkeeper.client.api.BookKeeper; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.ClientCommand; import org.apache.bookkeeper.tools.framework.CliFlags; import org.apache.bookkeeper.tools.framework.CliSpec; @@ -83,8 +83,8 @@ protected void run(BookKeeper bk, Flags flags) throws Exception { LedgerManager ledgerManager = ((org.apache.bookkeeper.client.BookKeeper) bk).getLedgerManager(); long i = 0; - BookieSocketAddress fromAddr = new BookieSocketAddress(flags.from); - BookieSocketAddress toAddr = new BookieSocketAddress(flags.to); + BookieId fromAddr = BookieId.parse(flags.from); + BookieId toAddr = BookieId.parse(flags.to); System.out.println(String.format("Replacing bookie id %s with %s in metadata", fromAddr, toAddr)); RateLimiter limiter = RateLimiter.create(flags.rate); for (Long lid : admin.listLedgers()) { @@ -97,7 +97,7 @@ protected void run(BookKeeper bk, Flags flags) throws Exception { md.getValue().getAllEnsembles().entrySet().stream() .filter(e -> e.getValue().contains(fromAddr)) .forEach(e -> { - List ensemble = new ArrayList<>(e.getValue()); + List ensemble = new ArrayList<>(e.getValue()); ensemble.replaceAll((a) -> { if (a.equals(fromAddr)) { return toAddr; diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommandTest.java index b4f248a233f..d21856a4d03 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/WhoIsAuditorCommandTest.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.tools.cli.commands.autorecovery; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; @@ -25,9 +26,11 @@ import static org.powermock.api.mockito.PowerMockito.when; import java.net.URI; +import java.util.UUID; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; import org.apache.bookkeeper.tools.cli.helpers.CommandHelpers; @@ -70,14 +73,16 @@ public void setup() throws Exception { when(builder.sessionTimeoutMs(anyInt())).thenReturn(builder); when(builder.build()).thenReturn(zk); - BookieSocketAddress bookieId = mock(BookieSocketAddress.class); + BookieId bookieId = BookieId.parse(UUID.randomUUID().toString()); PowerMockito.mockStatic(AuditorElector.class); PowerMockito.when(AuditorElector.getCurrentAuditor(eq(conf), eq(zk))) .thenReturn(bookieId); PowerMockito.mockStatic(CommandHelpers.class); - PowerMockito.when(CommandHelpers.getBookieSocketAddrStringRepresentation(eq(bookieId))).thenReturn(""); + PowerMockito.when(CommandHelpers + .getBookieSocketAddrStringRepresentation( + eq(bookieId), any(BookieAddressResolver.class))).thenReturn(""); } @Test diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java index 3be6b3db37a..5a99ab99e2e 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java @@ -33,7 +33,7 @@ import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; import org.junit.Assert; import org.junit.Test; @@ -65,8 +65,7 @@ public class FlipBookieIdCommandTest extends BookieCommandTestBase { @Mock private ServerConfiguration serverConfiguration; - @Mock - private BookieSocketAddress bookieSocketAddress; + private BookieId bookieSocketAddress = BookieId.parse("localhost:9000"); public FlipBookieIdCommandTest() { super(3, 0); @@ -87,7 +86,7 @@ public void setup() throws Exception { PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class) .withArguments(eq(conf)).thenReturn(serverConfiguration); PowerMockito.mockStatic(Bookie.class); - PowerMockito.when(Bookie.getBookieAddress(eq(serverConfiguration))).thenReturn(bookieSocketAddress); + PowerMockito.when(Bookie.getBookieId(eq(serverConfiguration))).thenReturn(bookieSocketAddress); } @Test diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommandTest.java index 9470023f9f1..2209bc6a63f 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ListLedgersCommandTest.java @@ -25,13 +25,14 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.function.Function; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.MetadataDrivers; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; import org.apache.zookeeper.AsyncCallback; @@ -50,6 +51,8 @@ CountDownLatch.class }) public class ListLedgersCommandTest extends BookieCommandTestBase { + private final BookieId bookieAddress = BookieId.parse(UUID.randomUUID().toString()); + public ListLedgersCommandTest() { super(3, 3); } @@ -61,8 +64,7 @@ public void setup() throws Exception { PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); - BookieSocketAddress bookieAddress = mock(BookieSocketAddress.class); - PowerMockito.whenNew(BookieSocketAddress.class).withParameterTypes(String.class).withArguments(anyString()) + PowerMockito.whenNew(BookieId.class).withParameterTypes(String.class).withArguments(anyString()) .thenReturn(bookieAddress); PowerMockito.mockStatic(MetadataDrivers.class); @@ -95,7 +97,7 @@ public void testWithoutBookieId() { @Test public void testWithBookieId() { - testCommand("-id", "1"); + testCommand("-id", bookieAddress.getId()); } private void testCommand(String... args) { diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommandTest.java index 7eb651d0472..a7e2062400b 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLedgerCommandTest.java @@ -41,6 +41,7 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -57,12 +58,11 @@ * Unit test for {@link ReadLedgerCommand}. */ @RunWith(PowerMockRunner.class) -@PrepareForTest({ ReadLedgerCommand.class, BookKeeperAdmin.class, BookieSocketAddress.class, ClientConfiguration.class, +@PrepareForTest({ ReadLedgerCommand.class, BookKeeperAdmin.class, ClientConfiguration.class, LedgerHandle.class, LedgerEntry.class, OrderedExecutor.class }) public class ReadLedgerCommandTest extends BookieCommandTestBase { - @Mock - private BookieSocketAddress bookieSocketAddress; + private BookieId bookieSocketAddress = BookieId.parse("localhost:9000"); @Mock private ClientConfiguration clientConfiguration; @@ -100,10 +100,10 @@ public void setup() throws Exception { super.setup(); PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); - PowerMockito.whenNew(BookieSocketAddress.class).withArguments(anyString()).thenReturn(bookieSocketAddress); PowerMockito.whenNew(ClientConfiguration.class).withNoArguments().thenReturn(clientConfiguration); PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class) .withArguments(eq(clientConfiguration)).thenReturn(bookKeeperAdmin); + when(bookKeeperAdmin.getBookieAddressResolver()).thenReturn(BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); when(bookKeeperAdmin.openLedger(anyLong())).thenReturn(ledgerHandle); when(ledgerHandle.getLastAddConfirmed()).thenReturn(1L); @@ -130,8 +130,7 @@ public void setup() throws Exception { when(Executors.newSingleThreadScheduledExecutor(eq(defaultThreadFactory))).thenReturn(scheduledExecutorService); PowerMockito.whenNew(BookieClientImpl.class) - .withArguments(eq(clientConfiguration), eq(nioEventLoopGroup), eq(UnpooledByteBufAllocator.DEFAULT), - eq(orderedExecutor), eq(scheduledExecutorService), eq(NullStatsLogger.INSTANCE)) + .withAnyArguments() .thenReturn(bookieClient); @@ -155,12 +154,13 @@ public void testWithoutBookieAddress() throws Exception { @Test public void testWithBookieAddress() throws Exception { ReadLedgerCommand cmd = new ReadLedgerCommand(); - Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-b", "localhost:9000" })); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-b", bookieSocketAddress.getId() })); verifyNew(NioEventLoopGroup.class, times(1)).withNoArguments(); verifyNew(DefaultThreadFactory.class, times(1)).withArguments(anyString()); verifyNew(BookieClientImpl.class, times(1)) .withArguments(eq(clientConfiguration), eq(nioEventLoopGroup), eq(UnpooledByteBufAllocator.DEFAULT), - eq(orderedExecutor), eq(scheduledExecutorService), eq(NullStatsLogger.INSTANCE)); + eq(orderedExecutor), eq(scheduledExecutorService), + eq(NullStatsLogger.INSTANCE), eq(BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER)); verify(nioEventLoopGroup, times(1)).shutdownGracefully(); verify(orderedExecutor, times(1)).shutdown(); verify(bookieClient, times(1)).close(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java index 7ce5a026e45..f1455c57b04 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java @@ -19,15 +19,14 @@ package org.apache.bookkeeper.tools.cli.commands.bookies; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.powermock.api.mockito.PowerMockito.mock; import static org.powermock.api.mockito.PowerMockito.verifyNew; import static org.powermock.api.mockito.PowerMockito.when; +import java.util.UUID; import java.util.function.Function; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.Cookie; @@ -36,7 +35,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.MetadataDrivers; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; @@ -61,8 +60,7 @@ public class DecommissionCommandTest extends BookieCommandTestBase { @Mock private BookKeeperAdmin bookKeeperAdmin; - @Mock - private BookieSocketAddress bookieSocketAddress; + private BookieId bookieSocketAddress = BookieId.parse(UUID.randomUUID().toString()); @Mock private Versioned cookieVersioned; @@ -84,9 +82,8 @@ public void setup() throws Exception { PowerMockito.whenNew(ClientConfiguration.class).withArguments(eq(conf)).thenReturn(clientConfiguration); PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class) .withArguments(eq(clientConfiguration)).thenReturn(bookKeeperAdmin); - PowerMockito.whenNew(BookieSocketAddress.class).withArguments(anyString()).thenReturn(bookieSocketAddress); PowerMockito.mockStatic(Bookie.class); - PowerMockito.when(Bookie.getBookieAddress(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress); + PowerMockito.when(Bookie.getBookieId(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress); PowerMockito.doNothing().when(bookKeeperAdmin).decommissionBookie(eq(bookieSocketAddress)); RegistrationManager registrationManager = mock(RegistrationManager.class); @@ -114,7 +111,6 @@ public void testWithoutBookieId() throws Exception { verifyNew(ClientConfiguration.class, times(1)).withArguments(eq(conf)); verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration)); - verifyNew(BookieSocketAddress.class, never()).withArguments(anyString()); verify(bookKeeperAdmin, times(1)).decommissionBookie(eq(bookieSocketAddress)); verify(cookieVersioned, times(1)).getValue(); verify(cookieVersioned, times(1)).getVersion(); @@ -123,11 +119,10 @@ public void testWithoutBookieId() throws Exception { @Test public void testWithBookieId() throws Exception { DecommissionCommand cmd = new DecommissionCommand(); - Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-b", "1" })); + Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-b", bookieSocketAddress.getId() })); verifyNew(ClientConfiguration.class, times(1)).withArguments(eq(conf)); verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration)); - verifyNew(BookieSocketAddress.class, times(1)).withArguments(anyString()); verify(bookKeeperAdmin, times(1)).decommissionBookie(eq(bookieSocketAddress)); verify(cookieVersioned, times(1)).getValue(); verify(cookieVersioned, times(1)).getVersion(); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommandTest.java index 05575cbecdc..5bb346289ae 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/InfoCommandTest.java @@ -30,6 +30,7 @@ import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; import org.junit.Before; @@ -46,10 +47,10 @@ @PrepareForTest({InfoCommand.class}) public class InfoCommandTest extends BookieCommandTestBase { - private BookieSocketAddress bookieId; + private BookieId bookieId; private BookieInfoReader.BookieInfo bInfo; private BookKeeper bk; - private Map map = new HashMap<>(); + private Map map = new HashMap<>(); public InfoCommandTest() { super(1, 0); @@ -73,8 +74,8 @@ public void setup() throws Exception { .withParameterTypes(ClientConfiguration.class) .withArguments(any(ClientConfiguration.class)) .thenReturn(bk); - - this.bookieId = new BookieSocketAddress("localhost", 9999); + when(bk.getBookieAddressResolver()).thenReturn(BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); + this.bookieId = BookieId.parse("localhost:9999"); this.bInfo = mock(BookieInfoReader.BookieInfo.class); map.put(bookieId, bInfo); when(bk.getBookieInfo()).thenReturn(map); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java index 5f394e1b639..4c7fd3a171a 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/ListBookiesCommandTest.java @@ -18,7 +18,6 @@ */ package org.apache.bookkeeper.tools.cli.commands.bookies; -import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.common.concurrent.FutureUtils.value; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -28,13 +27,14 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.when; -import com.google.common.primitives.UnsignedBytes; import java.util.Collections; import java.util.Comparator; import java.util.HashSet; import java.util.Set; import java.util.TreeSet; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.discover.BookieServiceInfoUtils; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.tools.cli.helpers.CommandHelpers; import org.apache.bookkeeper.tools.cli.helpers.DiscoveryCommandTestBase; import org.apache.bookkeeper.versioning.LongVersion; @@ -53,23 +53,17 @@ @PrepareForTest({ ListBookiesCommand.class, CommandHelpers.class }) public class ListBookiesCommandTest extends DiscoveryCommandTestBase { - private static class BookieAddressComparator implements Comparator { + private static class BookieAddressComparator implements Comparator { @Override - public int compare(BookieSocketAddress o1, BookieSocketAddress o2) { - int ret = UnsignedBytes.lexicographicalComparator() - .compare(o1.getHostName().getBytes(UTF_8), o2.getHostName().getBytes(UTF_8)); - if (ret == 0) { - return Integer.compare(o1.getPort(), o2.getPort()); - } else { - return ret; - } + public int compare(BookieId o1, BookieId o2) { + return o1.toString().compareToIgnoreCase(o2.toString()); } } - private Set writableBookies; - private Set readonlyBookies; - private Set allBookies; + private Set writableBookies; + private Set readonlyBookies; + private Set allBookies; @Before public void setup() throws Exception { @@ -84,6 +78,9 @@ public void setup() throws Exception { when(regClient.getWritableBookies()) .thenReturn(value(new Versioned<>(writableBookies, new LongVersion(0L)))); + when(regClient.getBookieServiceInfo(any(BookieId.class))) + .thenReturn(value(new Versioned<>( + BookieServiceInfoUtils.buildLegacyBookieServiceInfo("localhost:1234"), new LongVersion(0)))); when(regClient.getReadOnlyBookies()) .thenReturn(value(new Versioned<>(readonlyBookies, new LongVersion(0L)))); when(regClient.getAllBookies()) @@ -92,14 +89,14 @@ public void setup() throws Exception { PowerMockito.mockStatic(CommandHelpers.class, CALLS_REAL_METHODS); } - private static Set createBookies(int startPort, int numBookies) { - Set bookies = new TreeSet<>(new BookieAddressComparator()); + private static Set createBookies(int startPort, int numBookies) { + Set bookies = new TreeSet<>(new BookieAddressComparator()); int i = 0; for (; i < numBookies - 1; i++) { - bookies.add(new BookieSocketAddress("127.0.0.1", startPort + i)); + bookies.add(BookieId.parse("127.0.0.1" + (startPort + i))); } // mix an unknown hostname bookieId - bookies.add(new BookieSocketAddress("unknown", startPort + i)); + bookies.add(BookieId.parse("unknown" + (startPort + i))); return bookies; } @@ -110,12 +107,13 @@ private static void verifyPrintBookies(int startPort, int numBookies, int numCal times(numCalls)); if (i == numBookies - 1){ CommandHelpers.getBookieSocketAddrStringRepresentation( - eq(new BookieSocketAddress("unknown", startPort + i))); + eq(BookieId.parse("unknown" + (startPort + i))), + any(BookieAddressResolver.class)); } else { CommandHelpers.getBookieSocketAddrStringRepresentation( - eq(new BookieSocketAddress("127.0.0.1", startPort + i))); + eq(BookieId.parse("127.0.0.1" + (startPort + i))), + any(BookieAddressResolver.class)); } - } } @@ -231,14 +229,14 @@ public void testListEmptyBookies() throws Exception { PowerMockito.verifyStatic( CommandHelpers.class, times(0)); - CommandHelpers.getBookieSocketAddrStringRepresentation(any()); + CommandHelpers.getBookieSocketAddrStringRepresentation(any(), any()); assertTrue(cmd.apply(bkFlags, new String[]{"-ro"})); PowerMockito.verifyStatic( CommandHelpers.class, times(0)); - CommandHelpers.getBookieSocketAddrStringRepresentation(any()); + CommandHelpers.getBookieSocketAddrStringRepresentation(any(), any()); } } diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommandTest.java index 12f336b7950..a7a014f8e1c 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/RecoverCommandTest.java @@ -20,9 +20,7 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.verify; @@ -52,7 +50,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.MetadataDrivers; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; @@ -68,11 +66,10 @@ * Unit test for {@link RecoverCommand}. */ @RunWith(PowerMockRunner.class) -@PrepareForTest({ RecoverCommand.class, MetadataDrivers.class, Cookie.class }) +@PrepareForTest({ RecoverCommand.class, MetadataDrivers.class, Cookie.class}) public class RecoverCommandTest extends BookieCommandTestBase { - @Mock - private BookieSocketAddress bookieSocketAddress; + private BookieId bookieSocketAddress = BookieId.parse("127.0.0.1:8000"); @Mock private ClientConfiguration clientConfiguration; @@ -99,12 +96,9 @@ public RecoverCommandTest() { @Override public void setup() throws Exception { super.setup(); - PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf); PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class) .withArguments(eq(clientConfiguration)).thenReturn(conf); - PowerMockito.whenNew(BookieSocketAddress.class).withArguments(anyString(), anyInt()) - .thenReturn(bookieSocketAddress); PowerMockito.whenNew(ClientConfiguration.class).withParameterTypes(AbstractConfiguration.class) .withArguments(eq(conf)).thenReturn(clientConfiguration); PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class) @@ -121,11 +115,11 @@ private void mockBkQuery() throws BKException, InterruptedException { SortedMap ledgerMetadataSortedMap = new TreeMap<>(); ledgerMetadataSortedMap.put(1L, ledgerMetadata); when(bookKeeperAdmin.getLedgersContainBookies(any())).thenReturn(ledgerMetadataSortedMap); - ArrayList arrayList = new ArrayList<>(); + ArrayList arrayList = new ArrayList<>(); arrayList.add(bookieSocketAddress); - Map> map = new HashMap<>(); + Map> map = new HashMap<>(); map.put(1L, arrayList); - NavigableMap> navigableMap = Collections.unmodifiableNavigableMap( + NavigableMap> navigableMap = Collections.unmodifiableNavigableMap( map.entrySet().stream() .collect(TreeMap::new, (m, e) -> m.put(e.getKey(), ImmutableList.copyOf(e.getValue())), TreeMap::putAll)); @@ -164,7 +158,7 @@ private void mockBkRecovery() throws BKException, InterruptedException { @Test public void testBookieListCheck() { RecoverCommand cmd = new RecoverCommand(); - Assert.assertFalse(cmd.apply(bkFlags, new String[] { "-bs", "127.0.0.1:8000,8001" })); + Assert.assertFalse(cmd.apply(bkFlags, new String[] { "-bs", "127.0.0.1:8000,$nonvalidbookieid:8001" })); } @Test diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java index 9f39867981c..b9c0a092472 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java @@ -41,7 +41,7 @@ import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.meta.MetadataDrivers; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.versioning.Version; @@ -70,8 +70,7 @@ public class AdminCommandTest extends BookieCommandTestBase { @Mock private Cookie cookie; - @Mock - private BookieSocketAddress bookieSocketAddress; + private BookieId bookieSocketAddress = BookieId.parse("localhost:9000"); public AdminCommandTest() throws IOException { super(3, 3); @@ -124,8 +123,7 @@ private void mockUpdateBookieIdInCookie() throws Exception { }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", any(ServerConfiguration.class), any(Function.class)); - PowerMockito.when(Bookie.getBookieAddress(eq(serverConfiguration))).thenReturn(bookieSocketAddress); - when(bookieSocketAddress.toString()).thenReturn("1"); + PowerMockito.when(Bookie.getBookieId(eq(serverConfiguration))).thenReturn(bookieSocketAddress); PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(serverConfiguration))) .thenReturn(cookieVersioned); PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress))) @@ -174,7 +172,7 @@ private void mockExpandStorage() throws Exception { private void mockListOrDeleteCookies() throws UnknownHostException { - when(Bookie.getBookieAddress(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress); + when(Bookie.getBookieId(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress); } @Test diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java index fcfae0be770..3ff48a113f5 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/CreateCookieCommandTest.java @@ -36,6 +36,7 @@ import org.apache.bookkeeper.bookie.BookieException.CookieExistException; import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.CookieCommandTestBase; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.versioning.Versioned; @@ -129,7 +130,7 @@ public void testCreateCookieFromExistentCookieFile() throws Exception { assertTrue(runCommand(new String[] { "-cf", fileName, BOOKIE_ID })); String consoleOutput = getConsoleOutput(); assertTrue(consoleOutput, consoleOutput.isEmpty()); - verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + verify(rm, times(1)).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); } /** @@ -139,7 +140,7 @@ public void testCreateCookieFromExistentCookieFile() throws Exception { @Test public void testCreateAlreadyExistedCookie() throws Exception { doThrow(new CookieExistException()) - .when(rm).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + .when(rm).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); File file = testFolder.newFile("test-cookie-file"); byte[] content = "test-create-cookie".getBytes(UTF_8); @@ -150,7 +151,7 @@ public void testCreateAlreadyExistedCookie() throws Exception { assertTrue( consoleOutput, consoleOutput.contains("Cookie already exist for bookie '" + BOOKIE_ID + "'")); - verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + verify(rm, times(1)).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); } /** @@ -160,7 +161,7 @@ public void testCreateAlreadyExistedCookie() throws Exception { @Test public void testCreateCookieException() throws Exception { doThrow(new OperationRejectedException()) - .when(rm).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + .when(rm).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); File file = testFolder.newFile("test-cookie-file"); byte[] content = "test-create-cookie".getBytes(UTF_8); @@ -174,7 +175,7 @@ public void testCreateCookieException() throws Exception { assertTrue( consoleOutput, consoleOutput.contains(OperationRejectedException.class.getName())); - verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + verify(rm, times(1)).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); } } diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java index 01ad58d42bb..a955fd4a108 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/DeleteCookieCommandTest.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.CookieCommandTestBase; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.versioning.LongVersion; @@ -94,7 +95,7 @@ public void testDeleteCookieFromExistentCookieFile() throws Exception { assertTrue(runCommand(new String[] { BOOKIE_ID })); String consoleOutput = getConsoleOutput(); assertTrue(consoleOutput, consoleOutput.isEmpty()); - verify(rm, times(1)).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + verify(rm, times(1)).removeCookie(eq(BookieId.parse(BOOKIE_ID)), eq(new LongVersion(-1L))); } /** @@ -103,14 +104,14 @@ public void testDeleteCookieFromExistentCookieFile() throws Exception { @Test public void testDeleteNonExistedCookie() throws Exception { doThrow(new CookieNotFoundException()) - .when(rm).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + .when(rm).removeCookie(eq(BookieId.parse(BOOKIE_ID)), eq(new LongVersion(-1L))); assertFalse(runCommand(new String[] { BOOKIE_ID })); String consoleOutput = getConsoleOutput(); assertTrue( consoleOutput, consoleOutput.contains("Cookie not found for bookie '" + BOOKIE_ID + "'")); - verify(rm, times(1)).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + verify(rm, times(1)).removeCookie(eq(BookieId.parse(BOOKIE_ID)), eq(new LongVersion(-1L))); } /** @@ -119,7 +120,7 @@ public void testDeleteNonExistedCookie() throws Exception { @Test public void testDeleteCookieException() throws Exception { doThrow(new OperationRejectedException()) - .when(rm).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + .when(rm).removeCookie(eq(BookieId.parse(BOOKIE_ID)), eq(new LongVersion(-1L))); assertFalse(runCommand(new String[] { BOOKIE_ID })); String consoleOutput = getConsoleOutput(); @@ -129,7 +130,7 @@ public void testDeleteCookieException() throws Exception { assertTrue( consoleOutput, consoleOutput.contains(OperationRejectedException.class.getName())); - verify(rm, times(1)).removeCookie(eq(BOOKIE_ID), eq(new LongVersion(-1L))); + verify(rm, times(1)).removeCookie(eq(BookieId.parse(BOOKIE_ID)), eq(new LongVersion(-1L))); } } diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java index ef45a05d359..4c555074be0 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; import org.apache.bookkeeper.bookie.Cookie; import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.CookieCommandTestBase; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.versioning.LongVersion; @@ -100,12 +101,12 @@ public void testGetCookieFromExistentCookieFile() throws Exception { .setJournalDirs("/path/to/journal/dir") .setLedgerDirs("/path/to/ledger/dirs") .build(); - when(rm.readCookie(eq(BOOKIE_ID))) + when(rm.readCookie(eq(BookieId.parse(BOOKIE_ID)))) .thenReturn(new Versioned<>(cookie.toString().getBytes(UTF_8), new LongVersion(-1L))); - assertTrue(runCommand(new String[] { BOOKIE_ID })); + assertTrue(runCommand(new String[] { BOOKIE_ID.toString() })); String consoleOutput = getConsoleOutput(); assertTrue(consoleOutput, consoleOutput.contains(cookie.toString())); - verify(rm, times(1)).readCookie(eq(BOOKIE_ID)); + verify(rm, times(1)).readCookie(eq(BookieId.parse(BOOKIE_ID))); } /** @@ -114,14 +115,14 @@ public void testGetCookieFromExistentCookieFile() throws Exception { @Test public void testGetNonExistedCookie() throws Exception { doThrow(new CookieNotFoundException()) - .when(rm).readCookie(eq(BOOKIE_ID)); + .when(rm).readCookie(eq(BookieId.parse(BOOKIE_ID))); assertFalse(runCommand(new String[] { BOOKIE_ID })); String consoleOutput = getConsoleOutput(); assertTrue( consoleOutput, consoleOutput.contains("Cookie not found for bookie '" + BOOKIE_ID + "'")); - verify(rm, times(1)).readCookie(eq(BOOKIE_ID)); + verify(rm, times(1)).readCookie(eq(BookieId.parse(BOOKIE_ID))); } /** @@ -130,7 +131,7 @@ public void testGetNonExistedCookie() throws Exception { @Test public void testGetCookieException() throws Exception { doThrow(new OperationRejectedException()) - .when(rm).readCookie(eq(BOOKIE_ID)); + .when(rm).readCookie(eq(BookieId.parse(BOOKIE_ID))); assertFalse(runCommand(new String[] { BOOKIE_ID })); String consoleOutput = getConsoleOutput(); @@ -140,7 +141,7 @@ public void testGetCookieException() throws Exception { assertTrue( consoleOutput, consoleOutput.contains(OperationRejectedException.class.getName())); - verify(rm, times(1)).readCookie(eq(BOOKIE_ID)); + verify(rm, times(1)).readCookie(eq(BookieId.parse(BOOKIE_ID))); } } diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java index 308cd2aee1b..472c8325341 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/UpdateCookieCommandTest.java @@ -36,6 +36,7 @@ import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.OperationRejectedException; import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.tools.cli.helpers.CookieCommandTestBase; import org.apache.bookkeeper.tools.common.BKFlags; import org.apache.bookkeeper.versioning.Versioned; @@ -129,7 +130,7 @@ public void testUpdateCookieFromExistentCookieFile() throws Exception { assertTrue(runCommand(new String[] { "-cf", fileName, BOOKIE_ID })); String consoleOutput = getConsoleOutput(); assertTrue(consoleOutput, consoleOutput.isEmpty()); - verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + verify(rm, times(1)).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); } /** @@ -139,7 +140,7 @@ public void testUpdateCookieFromExistentCookieFile() throws Exception { @Test public void testUpdateNonExistedCookie() throws Exception { doThrow(new CookieNotFoundException()) - .when(rm).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + .when(rm).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); File file = testFolder.newFile("test-cookie-file"); byte[] content = "test-update-cookie".getBytes(UTF_8); @@ -150,7 +151,7 @@ public void testUpdateNonExistedCookie() throws Exception { assertTrue( consoleOutput, consoleOutput.contains("Cookie not found for bookie '" + BOOKIE_ID + "'")); - verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + verify(rm, times(1)).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); } /** @@ -160,7 +161,7 @@ public void testUpdateNonExistedCookie() throws Exception { @Test public void testUpdateCookieException() throws Exception { doThrow(new OperationRejectedException()) - .when(rm).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + .when(rm).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); File file = testFolder.newFile("test-cookie-file"); byte[] content = "test-update-cookie".getBytes(UTF_8); @@ -174,7 +175,7 @@ public void testUpdateCookieException() throws Exception { assertTrue( consoleOutput, consoleOutput.contains(OperationRejectedException.class.getName())); - verify(rm, times(1)).writeCookie(eq(BOOKIE_ID), any(Versioned.class)); + verify(rm, times(1)).writeCookie(eq(BookieId.parse(BOOKIE_ID)), any(Versioned.class)); } } From 6887ee0a7fe236e40089ad1565b644253860cbc2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Fri, 11 Sep 2020 01:57:20 +0800 Subject: [PATCH 0468/1642] Upgrade the `vertx` version to 3.5.3 Signed-off-by: xiaolong.ran ### Motivation As the https://github.com/apache/pulsar/issues/7931 said, CVE-2018-12540 has been raised on vertx 3.4.1 which is found in the pulsar dependencies. ### Changes - Upgrade the vertx version from 3.4.1 to 3.5.3 - Update license files Reviewers: Enrico Olivelli , Sijie Guo This closes #2410 from wolfstudy/xiaolong/upgrade-vertx-version --- .../src/main/resources/LICENSE-all.bin.txt | 12 ++++++------ .../src/main/resources/LICENSE-server.bin.txt | 12 ++++++------ pom.xml | 2 +- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index 551585e6516..b21f10b2807 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -237,9 +237,9 @@ Apache Software License, Version 2. - lib/io.prometheus-simpleclient_common-0.0.21.jar [17] - lib/io.prometheus-simpleclient_hotspot-0.0.21.jar [17] - lib/io.prometheus-simpleclient_servlet-0.0.21.jar [17] -- lib/io.vertx-vertx-auth-common-3.4.1.jar [18] -- lib/io.vertx-vertx-core-3.4.1.jar [19] -- lib/io.vertx-vertx-web-3.4.1.jar [20] +- lib/io.vertx-vertx-auth-common-3.5.3.jar [18] +- lib/io.vertx-vertx-core-3.5.3.jar [19] +- lib/io.vertx-vertx-web-3.5.3.jar [20] - lib/log4j-log4j-1.2.17.jar [21] - lib/net.java.dev.jna-jna-3.2.7.jar [22] - lib/org.apache.commons-commons-collections4-4.1.jar [23] @@ -301,9 +301,9 @@ Apache Software License, Version 2. [14] Source available at https://github.com/dropwizard/metrics/tree/v3.1.0 [16] Source available at https://github.com/netty/netty/tree/netty-4.1.32.Final [17] Source available at https://github.com/prometheus/client_java/tree/parent-0.0.21 -[18] Source available at https://github.com/vert-x3/vertx-auth/tree/3.4.1 -[19] Source available at https://github.com/eclipse/vert.x/tree/3.4.1 -[20] Source available at https://github.com/vert-x3/vertx-web/tree/3.4.1 +[18] Source available at https://github.com/vert-x3/vertx-auth/tree/3.5.3 +[19] Source available at https://github.com/eclipse/vert.x/tree/3.5.3 +[20] Source available at https://github.com/vert-x3/vertx-web/tree/3.5.3 [21] Source available at http://logging.apache.org/log4j/1.2/download.html [22] Source available at https://github.com/java-native-access/jna/tree/3.2.7 [23] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index 7c46d62deef..533f5348fd2 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -234,9 +234,9 @@ Apache Software License, Version 2. - lib/io.prometheus-simpleclient_common-0.0.21.jar [12] - lib/io.prometheus-simpleclient_hotspot-0.0.21.jar [12] - lib/io.prometheus-simpleclient_servlet-0.0.21.jar [12] -- lib/io.vertx-vertx-auth-common-3.4.1.jar [13] -- lib/io.vertx-vertx-core-3.4.1.jar [14] -- lib/io.vertx-vertx-web-3.4.1.jar [15] +- lib/io.vertx-vertx-auth-common-3.5.3.jar [13] +- lib/io.vertx-vertx-core-3.5.3.jar [14] +- lib/io.vertx-vertx-web-3.5.3.jar [15] - lib/log4j-log4j-1.2.17.jar [16] - lib/net.java.dev.jna-jna-3.2.7.jar [17] - lib/org.apache.commons-commons-collections4-4.1.jar [18] @@ -296,9 +296,9 @@ Apache Software License, Version 2. [10] Source available at http://svn.apache.org/viewvc/commons/proper/logging/tags/commons-logging-1.1.1/ [11] Source available at https://github.com/netty/netty/tree/netty-4.1.32.Final [12] Source available at https://github.com/prometheus/client_java/tree/parent-0.0.21 -[13] Source available at https://github.com/vert-x3/vertx-auth/tree/3.4.1 -[14] Source available at https://github.com/eclipse/vert.x/tree/3.4.1 -[15] Source available at https://github.com/vert-x3/vertx-web/tree/3.4.1 +[13] Source available at https://github.com/vert-x3/vertx-auth/tree/3.5.3 +[14] Source available at https://github.com/eclipse/vert.x/tree/3.5.3 +[15] Source available at https://github.com/vert-x3/vertx-web/tree/3.5.3 [16] Source available at http://logging.apache.org/log4j/1.2/download.html [17] Source available at https://github.com/java-native-access/jna/tree/3.2.7 [18] Source available at https://git-wip-us.apache.org/repos/asf?p=commons-collections.git;a=tag;h=a3a5ad diff --git a/pom.xml b/pom.xml index 9f5e01651ac..2b8835a1e79 100644 --- a/pom.xml +++ b/pom.xml @@ -168,7 +168,7 @@ 3.1.8 1.3.2 1.8.3 - 3.4.1 + 3.5.3 3.5.7 2.1.2 From cde3bcd276cc58aa7d7be28637bb4b3ccbbfadfc Mon Sep 17 00:00:00 2001 From: WJL3333 Date: Fri, 11 Sep 2020 12:59:22 +0800 Subject: [PATCH 0469/1642] fix maven-javadoc-plugin configuration. Descriptions of the changes in this PR: change the doclint configuration in pom.xml ### Motivation when you clone the project there is always warning in IntelliJ IDEA ### Changes change the doclint configuration in pom.xml according to https://stackoverflow.com/questions/52547306/maven-javadoc-plugin-not-accepting-additionalparam-xdoclintnone-additionalpa Reviewers: Enrico Olivelli , Sijie Guo This closes #2409 from WJL3333/fix_maven_pom_doclint_config --- bookkeeper-server/pom.xml | 2 +- bookkeeper-stats/pom.xml | 2 +- pom.xml | 2 +- stream/distributedlog/pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index adae9f68694..5bef417a026 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -234,7 +234,7 @@ ${maven-javadoc-plugin.version} - -Xdoclint:none + -Xdoclint:none org.apache.bookkeeper.client:org.apache.bookkeeper.conf:org.apache.bookkeeper.feature diff --git a/bookkeeper-stats/pom.xml b/bookkeeper-stats/pom.xml index 98e5bde819c..40867c8802f 100644 --- a/bookkeeper-stats/pom.xml +++ b/bookkeeper-stats/pom.xml @@ -34,7 +34,7 @@ ${maven-javadoc-plugin.version} - -Xdoclint:none + -Xdoclint:none org.apache.bookkeeper.stats diff --git a/pom.xml b/pom.xml index 2b8835a1e79..e76e8831795 100644 --- a/pom.xml +++ b/pom.xml @@ -871,7 +871,7 @@ ${maven-javadoc-plugin.version} - -Xdoclint:none -notimestamp + -Xdoclint:none -notimestamp -Xdoclint:none -notimestamp diff --git a/stream/distributedlog/pom.xml b/stream/distributedlog/pom.xml index afb35811ace..06e6b65ca6b 100644 --- a/stream/distributedlog/pom.xml +++ b/stream/distributedlog/pom.xml @@ -45,7 +45,7 @@ ${maven-javadoc-plugin.version} - -Xdoclint:none -notimestamp + -Xdoclint:none -notimestamp Core Library From 20b54f77105833ba17014b9e74cf8ea2eee9a5e3 Mon Sep 17 00:00:00 2001 From: odidev Date: Fri, 11 Sep 2020 10:31:46 +0530 Subject: [PATCH 0470/1642] Updated netty,netty-boringssl and rocksdb ### Descriptions of the changes in this PR: -Updated netty,netty-boringssl and rocksdb to latest version for aarch64 support -Updated deprecated methods of rocksdb to remove compilation warning and resolve build failure of bookkeeper-server on amd64 and aarch64 platforms as Werror flag is enabled. ### Motivation Build of the bookkeeper-server package was failing on amd64 and aarch64 platforms. ### Changes There are many methods that are marked deprecated in rocksdb, but in use by bookkeeper package. So updated those methods in bookkeeper package according to current rocksdb implementation. Also updated netty,netty-boringssl and rocksdb to latest version as they are having aarch64 support. Master Issue: #2378 Reviewers: Enrico Olivelli , Sijie Guo This closes #2379 from odidev/bookkeeper_aarch --- .../bookie/storage/ldb/KeyValueStorageRocksDB.java | 10 ++++++---- pom.xml | 8 ++++---- .../bookkeeper/statelib/impl/kv/RocksdbKVStore.java | 5 ++++- 3 files changed, 14 insertions(+), 9 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java index 88411ced3e7..c2747918dfc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java @@ -37,9 +37,11 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.BloomFilter; +import org.rocksdb.Cache; import org.rocksdb.ChecksumType; import org.rocksdb.CompressionType; import org.rocksdb.InfoLogLevel; +import org.rocksdb.LRUCache; import org.rocksdb.Options; import org.rocksdb.ReadOptions; import org.rocksdb.RocksDB; @@ -125,21 +127,21 @@ public KeyValueStorageRocksDB(String path, DbConfigType dbConfigType, ServerConf } options.setLevelZeroFileNumCompactionTrigger(numFilesInLevel0); options.setMaxBytesForLevelBase(maxSizeInLevel1MB * 1024 * 1024); - options.setMaxBackgroundCompactions(16); - options.setMaxBackgroundFlushes(16); + options.setMaxBackgroundJobs(32); options.setIncreaseParallelism(32); options.setMaxTotalWalSize(512 * 1024 * 1024); options.setMaxOpenFiles(-1); options.setTargetFileSizeBase(sstSizeMB * 1024 * 1024); options.setDeleteObsoleteFilesPeriodMicros(TimeUnit.HOURS.toMicros(1)); + final Cache cache = new LRUCache(blockCacheSize); BlockBasedTableConfig tableOptions = new BlockBasedTableConfig(); tableOptions.setBlockSize(blockSize); - tableOptions.setBlockCacheSize(blockCacheSize); + tableOptions.setBlockCache(cache); tableOptions.setFormatVersion(2); tableOptions.setChecksumType(ChecksumType.kxxHash); if (bloomFilterBitsPerKey > 0) { - tableOptions.setFilter(new BloomFilter(bloomFilterBitsPerKey, false)); + tableOptions.setFilterPolicy(new BloomFilter(bloomFilterBitsPerKey, false)); } // Options best suited for HDDs diff --git a/pom.xml b/pom.xml index e76e8831795..5302235567a 100644 --- a/pom.xml +++ b/pom.xml @@ -150,8 +150,8 @@ 1.18.10 1.3.0 3.0.0 - 4.1.32.Final - 2.0.20.Final + 4.1.50.Final + 2.0.31.Final 9.1.3 2.0.2 0.8.1 @@ -161,7 +161,7 @@ 3.5.1-1 1.12.0 0.9.11 - 5.13.1 + 6.10.2 3.0.1 1.7.25 1.19 @@ -191,7 +191,7 @@ 3.1.1 3.2.0 2.2.1 - 2.21.0 + 3.0.0-M5 3.5.2 1.4.1.Final 0.5.0 diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java index 64ff0b1d096..0c4b95f9a21 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/RocksdbKVStore.java @@ -66,11 +66,13 @@ import org.apache.bookkeeper.statelib.impl.rocksdb.checkpoint.RocksCheckpointer; import org.apache.commons.lang3.tuple.Pair; import org.rocksdb.BlockBasedTableConfig; +import org.rocksdb.Cache; import org.rocksdb.ColumnFamilyDescriptor; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.ColumnFamilyOptions; import org.rocksdb.DBOptions; import org.rocksdb.FlushOptions; +import org.rocksdb.LRUCache; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.rocksdb.RocksIterator; @@ -254,7 +256,8 @@ protected void openRocksdb(StateStoreSpec spec) throws StateStoreException { // initialize the db options final BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); - tableConfig.setBlockCacheSize(BLOCK_CACHE_SIZE); + final Cache cache = new LRUCache(BLOCK_CACHE_SIZE); + tableConfig.setBlockCache(cache); tableConfig.setBlockSize(BLOCK_SIZE); tableConfig.setChecksumType(DEFAULT_CHECKSUM_TYPE); From 03c79fb789b6f8161169765cd086b2a3ee7a31d4 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sat, 19 Sep 2020 13:32:03 +0200 Subject: [PATCH 0471/1642] BP-41 make bookieId configurable (#2412) ### Motivation This is the bookie side implementation of BP-41 ### Changes - add new `bookieId` configuration option - implement Bookie#getBookieId - add test cases - make all BookKeeperClusterTestCase to use a random UUID as bookieId in order to increase code coverage - add useUUIDasBookieId flag to BookKeeperClusterTestCase for test cases that require using hostname:port - add a default BookieServiceInfo provider in tests - enable Auditor to use BookieId and not BookieSocketAddress.toBookieId() - use BookieId for LocalTransport addresses - in Cookie.java rename BookieHost to BookieId (only in Java code, no data format changes) Master Issue: #2396 Co-authored-by: Enrico Olivelli --- .../org/apache/bookkeeper/bookie/Bookie.java | 12 ++++- .../org/apache/bookkeeper/bookie/Cookie.java | 40 +++++++------- .../bookkeeper/conf/ServerConfiguration.java | 34 ++++++++++++ .../discover/ZKRegistrationClient.java | 28 ++++++---- .../bookkeeper/net/BookieSocketAddress.java | 10 ---- .../bookkeeper/proto/BookieNettyServer.java | 11 ++-- .../apache/bookkeeper/proto/BookieServer.java | 13 ++++- .../proto/PerChannelBookieClient.java | 3 +- .../SimpleBookieServiceInfoProvider.java | 54 +++++++++++++++++++ .../replication/AutoRecoveryMain.java | 2 +- .../cli/commands/cookie/AdminCommand.java | 2 +- .../cookie/GenerateCookieCommand.java | 2 +- .../bookkeeper/util/LocalBookKeeper.java | 2 +- .../bookie/AdvertisedAddressTest.java | 35 ++++++++++++ .../bookkeeper/bookie/BookieShellTest.java | 2 +- .../bookie/UpdateCookieCmdTest.java | 3 +- .../client/BookKeeperAdminTest.java | 2 +- .../client/TestGetBookieInfoTimeout.java | 5 +- .../client/UpdateLedgerCmdTest.java | 1 + .../bookkeeper/client/UpdateLedgerOpTest.java | 1 + .../replication/AuditorPeriodicCheckTest.java | 8 +-- .../AuditorPlacementPolicyCheckTest.java | 2 +- .../replication/AuditorReplicasCheckTest.java | 2 +- .../test/BookKeeperClusterTestCase.java | 14 +++-- .../org/apache/bookkeeper/tls/TestTLS.java | 13 ++--- conf/bk_server.conf | 4 ++ .../cli/commands/cookie/AdminCommandTest.java | 2 +- .../cookie/GenerateCookieCommandTest.java | 4 +- .../commands/cookie/GetCookieCommandTest.java | 2 +- 29 files changed, 231 insertions(+), 82 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/SimpleBookieServiceInfoProvider.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 a7960f79b0c..7e8452a2ca1 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 @@ -85,6 +85,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNS; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; +import org.apache.bookkeeper.proto.SimpleBookieServiceInfoProvider; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; @@ -329,6 +330,9 @@ static List possibleBookieIds(ServerConfiguration conf) if (null != conf.getAdvertisedAddress()) { addresses.add(getBookieAddress(conf).toBookieId()); } + if (null != conf.getBookieId()) { + addresses.add(BookieId.parse(conf.getBookieId())); + } } catch (UnknownHostException e) { throw new UnknownBookieIdException(e); } @@ -534,7 +538,11 @@ private static void verifyDirsForStorageExpansion( } public static BookieId getBookieId(ServerConfiguration conf) throws UnknownHostException { - return getBookieAddress(conf).toBookieId(); + String customBookieId = conf.getBookieId(); + if (customBookieId != null) { + return BookieId.parse(customBookieId); + } + return getBookieAddress(conf).toBookieId(); } /** @@ -619,7 +627,7 @@ public static File[] getCurrentDirectories(File[] dirs) { public Bookie(ServerConfiguration conf) throws IOException, InterruptedException, BookieException { - this(conf, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT, BookieServiceInfo.NO_INFO); + this(conf, NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT, new SimpleBookieServiceInfoProvider(conf)); } private static LedgerStorage buildLedgerStorage(ServerConfiguration conf) throws IOException { 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 b80a7895a90..5b542b57ead 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 @@ -68,15 +68,15 @@ public class Cookie { static final int CURRENT_COOKIE_LAYOUT_VERSION = 4; private final int layoutVersion; - private final String bookieHost; + private final String bookieId; private final String journalDirs; private final String ledgerDirs; private final String instanceId; private static final String SEPARATOR = "\t"; - private Cookie(int layoutVersion, String bookieHost, String journalDirs, String ledgerDirs, String instanceId) { + private Cookie(int layoutVersion, String bookieId, String journalDirs, String ledgerDirs, String instanceId) { this.layoutVersion = layoutVersion; - this.bookieHost = bookieHost; + this.bookieId = bookieId; this.journalDirs = journalDirs; this.ledgerDirs = ledgerDirs; this.instanceId = instanceId; @@ -130,7 +130,7 @@ private void verifyInternal(Cookie c, boolean checkIfSuperSet) throws BookieExce errMsg = "Cookie is of too old version " + c.layoutVersion; LOG.error(errMsg); throw new BookieException.InvalidCookieException(errMsg); - } else if (!(c.layoutVersion >= 3 && c.bookieHost.equals(bookieHost) + } else if (!(c.layoutVersion >= 3 && c.bookieId.equals(bookieId) && c.journalDirs.equals(journalDirs) && verifyLedgerDirs(c, checkIfSuperSet))) { errMsg = "Cookie [" + this + "] is not matching with [" + c + "]"; throw new BookieException.InvalidCookieException(errMsg); @@ -157,7 +157,7 @@ public String toString() { return toStringVersion3(); } CookieFormat.Builder builder = CookieFormat.newBuilder(); - builder.setBookieHost(bookieHost); + builder.setBookieHost(bookieId); builder.setJournalDir(journalDirs); builder.setLedgerDirs(ledgerDirs); if (null != instanceId) { @@ -172,7 +172,7 @@ public String toString() { private String toStringVersion3() { StringBuilder b = new StringBuilder(); b.append(CURRENT_COOKIE_LAYOUT_VERSION).append("\n") - .append(bookieHost).append("\n") + .append(bookieId).append("\n") .append(journalDirs).append("\n") .append(ledgerDirs).append("\n"); return b.toString(); @@ -193,14 +193,14 @@ private static Builder parse(BufferedReader reader) throws IOException { + "', cannot parse cookie."); } if (layoutVersion == 3) { - cBuilder.setBookieHost(reader.readLine()); + cBuilder.setBookieId(reader.readLine()); cBuilder.setJournalDirs(reader.readLine()); cBuilder.setLedgerDirs(reader.readLine()); } else if (layoutVersion >= 4) { CookieFormat.Builder cfBuilder = CookieFormat.newBuilder(); TextFormat.merge(reader, cfBuilder); CookieFormat data = cfBuilder.build(); - cBuilder.setBookieHost(data.getBookieHost()); + cBuilder.setBookieId(data.getBookieHost()); cBuilder.setJournalDirs(data.getJournalDir()); cBuilder.setLedgerDirs(data.getLedgerDirs()); // Since InstanceId is optional @@ -293,7 +293,7 @@ static Builder generateCookie(ServerConfiguration conf) throws UnknownHostException { Builder builder = Cookie.newBuilder(); builder.setLayoutVersion(CURRENT_COOKIE_LAYOUT_VERSION); - builder.setBookieHost(Bookie.getBookieId(conf).toString()); + builder.setBookieId(Bookie.getBookieId(conf).toString()); builder.setJournalDirs(Joiner.on(',').join(conf.getJournalDirNames())); builder.setLedgerDirs(encodeDirPaths(conf.getLedgerDirNames())); return builder; @@ -364,15 +364,17 @@ public static Cookie readFromDirectory(File directory) throws IOException { * if the 'bookieHost' was created using a hostname */ public boolean isBookieHostCreatedFromIp() throws IOException { - String[] parts = bookieHost.split(":"); + String[] parts = bookieId.split(":"); if (parts.length != 2) { - throw new IOException(bookieHost + " does not have the form host:port"); + // custom BookieId ? + return false; } int port; try { port = Integer.parseInt(parts[1]); } catch (NumberFormatException e) { - throw new IOException(bookieHost + " does not have the form host:port"); + // custom BookieId ? + return false; } InetSocketAddress addr = new InetSocketAddress(parts[0], port); @@ -384,7 +386,7 @@ public boolean isBookieHostCreatedFromIp() throws IOException { */ public static class Builder { private int layoutVersion = CURRENT_COOKIE_LAYOUT_VERSION; - private String bookieHost = null; + private String bookieId = null; private String journalDirs = null; private String ledgerDirs = null; private String instanceId = null; @@ -392,10 +394,10 @@ public static class Builder { private Builder() { } - private Builder(int layoutVersion, String bookieHost, String journalDirs, String ledgerDirs, + private Builder(int layoutVersion, String bookieId, String journalDirs, String ledgerDirs, String instanceId) { this.layoutVersion = layoutVersion; - this.bookieHost = bookieHost; + this.bookieId = bookieId; this.journalDirs = journalDirs; this.ledgerDirs = ledgerDirs; this.instanceId = instanceId; @@ -406,8 +408,8 @@ public Builder setLayoutVersion(int layoutVersion) { return this; } - public Builder setBookieHost(String bookieHost) { - this.bookieHost = bookieHost; + public Builder setBookieId(String bookieId) { + this.bookieId = bookieId; return this; } @@ -427,7 +429,7 @@ public Builder setInstanceId(String instanceId) { } public Cookie build() { - return new Cookie(layoutVersion, bookieHost, journalDirs, ledgerDirs, instanceId); + return new Cookie(layoutVersion, bookieId, journalDirs, ledgerDirs, instanceId); } } @@ -447,7 +449,7 @@ public static Builder newBuilder() { * @return cookie builder */ public static Builder newBuilder(Cookie oldCookie) { - return new Builder(oldCookie.layoutVersion, oldCookie.bookieHost, oldCookie.journalDirs, oldCookie.ledgerDirs, + return new Builder(oldCookie.layoutVersion, oldCookie.bookieId, oldCookie.journalDirs, oldCookie.ledgerDirs, oldCookie.instanceId); } } 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 8cd3ef86640..6dd2c9fc0f5 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 @@ -38,6 +38,7 @@ import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.discover.ZKRegistrationManager; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.commons.configuration.ConfigurationException; @@ -148,6 +149,7 @@ public class ServerConfiguration extends AbstractConfigurationIf present, this setting will take precedence over the + * automatic BookieId generation, based on Network Addresses. + * + * @see #setBookieId(java.lang.String) + * @see #getAdvertisedAddress() + * @return the configure address to be advertised + */ + public String getBookieId() { + return this.getString(BOOKIE_ID, null); + } + + /** + * Configure the bookie to advertise a specific BookieId. + * + *

            By default, a bookie will advertise a BookieId computed + * from the primary network endpoint addresss. + * + * @see #getBookieId() + * @see #setAdvertisedAddress(java.lang.String) + * @param bookieId the bookie id + * + * @return server configuration + */ + public ServerConfiguration setBookieId(String bookieId) { + BookieId.parse(bookieId); + this.setProperty(BOOKIE_ID, bookieId); + return this; + } + /** * Get the configured advertised address for the bookie. * 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 index dc17507637b..dd9bac18248 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java @@ -258,7 +258,9 @@ private CompletableFuture> readBookieServiceInfo(Bo bookieServiceInfoCache.put(bookieId, result); promise.complete(result); } catch (IOException ex) { - promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path)); + log.error("Cannot update BookieInfo for {}", ex); + promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path) + .initCause(ex)); return; } } else if (KeeperException.Code.NONODE.intValue() == rc) { @@ -273,7 +275,9 @@ private CompletableFuture> readBookieServiceInfo(Bo bookieServiceInfoCache.put(bookieId, result); promise.complete(result); } catch (IOException ex) { - promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc2), path2)); + log.error("Cannot update BookieInfo for {}", ex); + promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc2), path2) + .initCause(ex)); return; } } else { @@ -343,14 +347,18 @@ private CompletableFuture>> getChildren(String regPath, bookieInfoUpdated.add(readBookieServiceInfo(id)); } } - FutureUtils - .collect(bookieInfoUpdated) - .whenComplete((List> info, Throwable error) -> { - // we are ignoring errors intentionally - // there could be bookies that published unparseable information - // or other temporary/permanent or temporary errors - future.complete(new Versioned<>(bookies, version)); - }); + if (bookieInfoUpdated.isEmpty()) { + future.complete(new Versioned<>(bookies, version)); + } else { + FutureUtils + .collect(bookieInfoUpdated) + .whenComplete((List> info, Throwable error) -> { + // we are ignoring errors intentionally + // there could be bookies that publish unparseable information + // or other temporary/permanent errors + future.complete(new Versioned<>(bookies, version)); + }); + } }, null); return future; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java index 06da0dc5202..3e3fb85b741 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieSocketAddress.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.net.InetAddresses; -import io.netty.channel.local.LocalAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; @@ -106,15 +105,6 @@ public InetSocketAddress getSocketAddress() { }); } - /** - * Maps the socketAddress to a "local" address. - */ - @JsonIgnore - public LocalAddress getLocalAddress() { - // for local address, we just need "port" to differentiate different addresses. - return new LocalAddress("" + port); - } - // Return the String "serialized" version of this object. @Override public String toString() { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java index a48f7495886..d6e6f21645a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java @@ -44,6 +44,7 @@ import io.netty.channel.group.ChannelGroup; import io.netty.channel.group.ChannelGroupFuture; import io.netty.channel.group.DefaultChannelGroup; +import io.netty.channel.local.LocalAddress; import io.netty.channel.local.LocalChannel; import io.netty.channel.local.LocalServerChannel; import io.netty.channel.socket.SocketChannel; @@ -77,6 +78,7 @@ import org.apache.bookkeeper.common.collections.BlockingMpscQueue; import org.apache.bookkeeper.common.util.affinity.CpuAffinity; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.processor.RequestProcessor; import org.apache.bookkeeper.util.ByteBufList; @@ -103,6 +105,7 @@ class BookieNettyServer { volatile boolean suspended = false; ChannelGroup allChannels; final BookieSocketAddress bookieAddress; + final BookieId bookieId; final InetSocketAddress bindAddress; final BookieAuthProvider.Factory authProviderFactory; @@ -160,7 +163,7 @@ protected Queue newTaskQueue(int maxPendingTasks) { } else { jvmEventLoopGroup = null; } - + bookieId = Bookie.getBookieId(conf); bookieAddress = Bookie.getBookieAddress(conf); if (conf.getListeningInterface() == null) { bindAddress = new InetSocketAddress(conf.getBookiePort()); @@ -410,10 +413,10 @@ protected void initChannel(LocalChannel ch) throws Exception { pipeline.addLast("contextHandler", contextHandler); } }); - + LOG.info("Binding jvm bookie-rpc endpoint to {}", bookieId.toString()); // use the same address 'name', so clients can find local Bookie still discovering them using ZK - jvmBootstrap.bind(bookieAddress.getLocalAddress()).sync(); - LocalBookiesRegistry.registerLocalBookieAddress(bookieAddress.toBookieId()); + jvmBootstrap.bind(new LocalAddress(bookieId.toString())).sync(); + LocalBookiesRegistry.registerLocalBookieAddress(bookieId); } } 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 2864b299a70..ee83327b2f2 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 @@ -44,6 +44,7 @@ import org.apache.bookkeeper.common.util.JsonUtil.ParseJsonException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.discover.BookieServiceInfoUtils; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.processor.RequestProcessor; @@ -85,13 +86,23 @@ public class BookieServer { public BookieServer(ServerConfiguration conf) throws IOException, KeeperException, InterruptedException, BookieException, UnavailableException, CompatibilityException, SecurityException { - this(conf, NullStatsLogger.INSTANCE, BookieServiceInfo.NO_INFO); + this(conf, NullStatsLogger.INSTANCE, null); } public BookieServer(ServerConfiguration conf, StatsLogger statsLogger, Supplier bookieServiceInfoProvider) throws IOException, KeeperException, InterruptedException, BookieException, UnavailableException, CompatibilityException, SecurityException { + if (bookieServiceInfoProvider == null) { + bookieServiceInfoProvider = () -> { + try { + return BookieServiceInfoUtils + .buildLegacyBookieServiceInfo(this.getLocalAddress().toBookieId().toString()); + } catch (IOException err) { + throw new RuntimeException(err); + } + }; + } this.conf = conf; validateUser(conf); String configAsString; 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 45650b2c153..45d42c59485 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 @@ -46,6 +46,7 @@ import io.netty.channel.WriteBufferWaterMark; import io.netty.channel.epoll.EpollEventLoopGroup; import io.netty.channel.epoll.EpollSocketChannel; +import io.netty.channel.local.LocalAddress; import io.netty.channel.local.LocalChannel; import io.netty.channel.socket.nio.NioSocketChannel; import io.netty.channel.unix.Errors.NativeIoException; @@ -599,7 +600,7 @@ protected void initChannel(Channel ch) throws Exception { SocketAddress bookieAddr = addr.getSocketAddress(); if (eventLoopGroup instanceof DefaultEventLoopGroup) { - bookieAddr = addr.getLocalAddress(); + bookieAddr = new LocalAddress(bookieId.toString()); } ChannelFuture future = bootstrap.connect(bookieAddr); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/SimpleBookieServiceInfoProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/SimpleBookieServiceInfoProvider.java new file mode 100644 index 00000000000..2231b9521df --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/SimpleBookieServiceInfoProvider.java @@ -0,0 +1,54 @@ +/** + * + * 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.proto; + +import java.net.UnknownHostException; +import java.util.function.Supplier; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.discover.BookieServiceInfoUtils; +import org.apache.bookkeeper.net.BookieSocketAddress; + +/** + * Simple Implementation of BookieServiceInfo supplier. + */ +public class SimpleBookieServiceInfoProvider implements Supplier { + private final BookieSocketAddress bookieSocketAddress; + + public SimpleBookieServiceInfoProvider(ServerConfiguration serverConfiguration) { + try { + this.bookieSocketAddress = Bookie.getBookieAddress(serverConfiguration); + } catch (UnknownHostException err) { + throw new RuntimeException(err); + } + } + + @Override + public BookieServiceInfo get() { + try { + return BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieSocketAddress.toBookieId().toString()); + } catch (UnknownHostException err) { + throw new RuntimeException(err); + } + } + +} 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 303a401e687..1ccb3228261 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 @@ -100,7 +100,7 @@ public AutoRecoveryMain(ServerConfiguration conf, StatsLogger statsLogger) }); auditorElector = new AuditorElector( - Bookie.getBookieAddress(conf).toString(), + Bookie.getBookieId(conf).toString(), conf, bkc, statsLogger.scope(AUDITOR_SCOPE), diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java index 5cdefd04e98..7429afcb93e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommand.java @@ -160,7 +160,7 @@ private boolean updateBookieIdInCookie(ServerConfiguration bkConf, final boolean useHostname, nne); return false; } - Cookie newCookie = Cookie.newBuilder(oldCookie.getValue()).setBookieHost(newBookieId).build(); + Cookie newCookie = Cookie.newBuilder(oldCookie.getValue()).setBookieId(newBookieId).build(); boolean hasCookieUpdatedInDirs = verifyCookie(newCookie, journalDirectories[0]); for (File dir : ledgerDirectories) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java index 3ef811602ea..a5997b0761b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommand.java @@ -110,7 +110,7 @@ protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception { } Builder builder = Cookie.newBuilder(); - builder.setBookieHost(bookieId.toString()); + builder.setBookieId(bookieId.toString()); if (StringUtils.isEmpty(instanceId)) { builder.setInstanceId(null); } else { 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 1c8728490da..d9570ffc3bf 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 @@ -254,7 +254,7 @@ private void runBookies(List tempDirs, String dirSuffix) bsConfs[i].setLedgerDirNames(ledgerDirs); // write config into file before start so we can know what's wrong if start failed - String fileName = Bookie.getBookieAddress(bsConfs[i]).toString() + ".conf"; + String fileName = Bookie.getBookieId(bsConfs[i]).toString() + ".conf"; serializeLocalBookieConfig(bsConfs[i], fileName); // Mimic BookKeeper Main diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java index ad522c7a246..2b0288303bb 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/AdvertisedAddressTest.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.UUID; import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; @@ -105,4 +106,38 @@ public void testBothUseHostnameAndAdvertisedAddress() throws Exception { assertEquals(bkAddress.toBookieId(), Bookie.getBookieId(conf)); } + /** + * Test starting bookie with a bookieId. + */ + @Test + public void testSetBookieId() throws Exception { + String uuid = UUID.randomUUID().toString(); + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + conf.setJournalDirName(newDirectory(false)) + .setLedgerDirNames(new String[] { newDirectory(false) }) + .setBookiePort(bookiePort) + .setBookieId(uuid) + .setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + + conf.setAdvertisedAddress("10.0.0.1"); + assertEquals("10.0.0.1", conf.getAdvertisedAddress()); + assertEquals(uuid, conf.getBookieId()); + + BookieSocketAddress bkAddress = new BookieSocketAddress("10.0.0.1", bookiePort); + assertEquals(bkAddress, Bookie.getBookieAddress(conf)); + assertEquals(uuid, Bookie.getBookieId(conf).getId()); + + Bookie b = new Bookie(conf); + b.start(); + + BookKeeperAdmin bka = new BookKeeperAdmin(baseClientConf); + Collection bookies = bka.getAvailableBookies(); + + assertEquals(1, bookies.size()); + BookieId address = bookies.iterator().next(); + assertEquals(BookieId.parse(uuid), address); + + b.shutdown(); + bka.close(); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java index 8ab38c1466e..6fb759b24dd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java @@ -140,7 +140,7 @@ public void setup() throws Exception { when(admin.getConf()).thenReturn(this.clientConf); this.rm = PowerMockito.mock(RegistrationManager.class); this.cookie = Cookie.newBuilder() - .setBookieHost("127.0.0.1:3181") + .setBookieId("127.0.0.1:3181") .setInstanceId("xyz") .setJournalDirs("/path/to/journal/dir") .setLedgerDirs("/path/to/journal/dir") 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 e0d37bbc57f..840f2bd9b48 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 @@ -54,6 +54,7 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase { public UpdateCookieCmdTest() { super(1); + useUUIDasBookieId = false; } @Override @@ -148,7 +149,7 @@ public void testWhenBothIPaddressAndHostNameCookiesExists() throws Exception { Cookie.Builder cookieBuilder = Cookie.newBuilder(cookie); conf.setUseHostNameAsBookieID(false); // sets to hostname final String newBookieHost = Bookie.getBookieAddress(conf).toString(); - cookieBuilder.setBookieHost(newBookieHost); + cookieBuilder.setBookieId(newBookieHost); cookieBuilder.build().writeToRegistrationManager(rm, conf, Version.NEW); verifyCookieInZooKeeper(conf, 2); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java index 6d2b972a423..f6477d4a897 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperAdminTest.java @@ -210,7 +210,7 @@ public void testBookieInit() throws Exception { } Assert.assertFalse("initBookie shouldn't have succeeded, since cookie in ZK is not deleted yet", BookKeeperAdmin.initBookie(confOfExistingBookie)); - String bookieId = Bookie.getBookieAddress(confOfExistingBookie).toString(); + String bookieId = Bookie.getBookieId(confOfExistingBookie).toString(); String bookieCookiePath = ZKMetadataDriverBase.resolveZkLedgersRootPath(confOfExistingBookie) + "/" + BookKeeperConstants.COOKIE_NODE diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java index 8541420c050..3a97461bada 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java @@ -38,8 +38,6 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; @@ -108,10 +106,9 @@ public void testGetBookieInfoTimeout() throws Exception { CountDownLatch latch = sleepBookie(bookieToSleep, sleeptime); latch.await(); - BookieAddressResolver bookieAddressResolver = BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER; // try to get bookie info from the sleeping bookie. It should fail with timeout error BookieClient bc = new BookieClientImpl(cConf, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, executor, - scheduler, NullStatsLogger.INSTANCE, bookieAddressResolver); + scheduler, NullStatsLogger.INSTANCE, bkc.getBookieAddressResolver()); long flags = BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE | BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java index 1a849ae828b..cad0718b986 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java @@ -54,6 +54,7 @@ public class UpdateLedgerCmdTest extends BookKeeperClusterTestCase { public UpdateLedgerCmdTest() { super(3); + useUUIDasBookieId = false; baseConf.setGcWaitTime(100000); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java index 6c08ddd4b59..788922d0794 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java @@ -57,6 +57,7 @@ public class UpdateLedgerOpTest extends BookKeeperClusterTestCase { public UpdateLedgerOpTest() { super(3); + useUUIDasBookieId = false; baseConf.setGcWaitTime(100000); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 01ee5d85ce2..8ea66360aa0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -338,7 +338,7 @@ public void testPeriodicCheckWhenLedgerDeleted() throws Exception { } try (final Auditor auditor = new Auditor( - Bookie.getBookieAddress(bsConfs.get(0)).toString(), + Bookie.getBookieId(bsConfs.get(0)).toString(), bsConfs.get(0), NullStatsLogger.INSTANCE)) { final AtomicBoolean exceptionCaught = new AtomicBoolean(false); final CountDownLatch latch = new CountDownLatch(1); @@ -401,7 +401,7 @@ void validateInitialDelayOfCheckAllLedgers(LedgerUnderreplicationManager urm, lo servConf.setAuditorPeriodicCheckInterval(auditorPeriodicCheckInterval); servConf.setAuditorPeriodicPlacementPolicyCheckInterval(0); servConf.setAuditorPeriodicBookieCheckInterval(0); - final TestAuditor auditor = new TestAuditor(Bookie.getBookieAddress(servConf).toString(), servConf, bkc, false, + final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf, bkc, false, statsLogger); CountDownLatch latch = auditor.getLatch(); assertEquals("CHECK_ALL_LEDGERS_TIME SuccessCount", 0, checkAllLedgersStatsLogger.getSuccessCount()); @@ -494,7 +494,7 @@ void validateInitialDelayOfPlacementPolicyCheck(LedgerUnderreplicationManager ur servConf.setAuditorPeriodicPlacementPolicyCheckInterval(auditorPeriodicPlacementPolicyCheckInterval); servConf.setAuditorPeriodicCheckInterval(0); servConf.setAuditorPeriodicBookieCheckInterval(0); - final TestAuditor auditor = new TestAuditor(Bookie.getBookieAddress(servConf).toString(), servConf, bkc, false, + final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf, bkc, false, statsLogger); CountDownLatch latch = auditor.getLatch(); assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 0, placementPolicyCheckStatsLogger.getSuccessCount()); @@ -598,7 +598,7 @@ void validateInitialDelayOfReplicasCheck(LedgerUnderreplicationManager urm, long servConf.setAuditorPeriodicReplicasCheckInterval(auditorPeriodicReplicasCheckInterval); servConf.setAuditorPeriodicCheckInterval(0); servConf.setAuditorPeriodicBookieCheckInterval(0); - final TestAuditor auditor = new TestAuditor(Bookie.getBookieAddress(servConf).toString(), servConf, bkc, false, + final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf, bkc, false, statsLogger); CountDownLatch latch = auditor.getLatch(); assertEquals("REPLICAS_CHECK_TIME SuccessCount", 0, replicasCheckStatsLogger.getSuccessCount()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java index 2f8e7e063ad..3c8d0c432f1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -672,7 +672,7 @@ private TestStatsLogger startAuditorAndWaitForPlacementPolicyCheck(ServerConfigu TestOpStatsLogger placementPolicyCheckStatsLogger = (TestOpStatsLogger) statsLogger .getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME); - final TestAuditor auditor = new TestAuditor(Bookie.getBookieAddress(servConf).toString(), servConf, + final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf, statsLogger); auditorRef.setValue(auditor); CountDownLatch latch = auditor.getLatch(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java index abf439f1ec8..8d54113210b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java @@ -149,7 +149,7 @@ private TestStatsLogger startAuditorAndWaitForReplicasCheck(ServerConfiguration TestOpStatsLogger replicasCheckStatsLogger = (TestOpStatsLogger) statsLogger .getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME); - final TestAuditor auditor = new TestAuditor(Bookie.getBookieAddress(servConf).toString(), servConf, bkc, true, + final TestAuditor auditor = new TestAuditor(Bookie.getBookieId(servConf).toString(), servConf, bkc, true, new TestBookKeeperAdmin(bkc, statsLogger, expectedReturnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger), true, statsLogger); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index 78a3b116856..21568591e7a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -37,6 +37,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.SynchronousQueue; @@ -99,6 +100,7 @@ public abstract class BookKeeperClusterTestCase { private final Map bsLoggers = new HashMap<>(); protected int numBookies; protected BookKeeperTestClient bkc; + protected boolean useUUIDasBookieId = true; /* * Loopback interface is set as the listening interface and allowloopback is @@ -632,6 +634,12 @@ public void restartBookies(ServerConfiguration newConf) public int startNewBookie() throws Exception { ServerConfiguration conf = newServerConfiguration(); + + // use a random BookieId + if (useUUIDasBookieId) { + conf.setBookieId(UUID.randomUUID().toString()); + } + bsConfs.add(conf); LOG.info("Starting new bookie on port: {}", conf.getBookiePort()); BookieServer server = startBookie(conf); @@ -661,8 +669,7 @@ public BookieId startNewBookieAndReturnBookieId() protected BookieServer startBookie(ServerConfiguration conf) throws Exception { TestStatsProvider provider = new TestStatsProvider(); - BookieServer server = new BookieServer(conf, provider.getStatsLogger(""), - BookieServiceInfo.NO_INFO); + BookieServer server = new BookieServer(conf, provider.getStatsLogger(""), null); BookieId address = Bookie.getBookieId(conf); bsLoggers.put(address, provider); @@ -696,8 +703,7 @@ protected BookieServer startBookie(ServerConfiguration conf) protected BookieServer startBookie(ServerConfiguration conf, final Bookie b) throws Exception { TestStatsProvider provider = new TestStatsProvider(); - BookieServer server = new BookieServer(conf, provider.getStatsLogger(""), - BookieServiceInfo.NO_INFO) { + BookieServer server = new BookieServer(conf, provider.getStatsLogger(""), null) { @Override protected Bookie newBookie(ServerConfiguration conf, ByteBufAllocator allocator, Supplier s) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java index dc9ce40dfe6..37f6e25c5d3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java @@ -60,6 +60,7 @@ import org.apache.bookkeeper.proto.BookieConnectionPeer; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.proto.ClientConnectionPeer; +import org.apache.bookkeeper.proto.PerChannelBookieClient; import org.apache.bookkeeper.proto.TestPerChannelBookieClient; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; @@ -856,11 +857,7 @@ public void testTLSChannelCounters() throws Exception { InetSocketAddress addr = bookie.getLocalAddress().getSocketAddress(); StringBuilder nameBuilder = new StringBuilder(BookKeeperClientStats.CHANNEL_SCOPE) .append(".") - .append(addr.getAddress().getHostAddress() - .replace('.', '_') - .replace('-', '_')) - .append("_") - .append(addr.getPort()) + .append(PerChannelBookieClient.buildStatsLoggerScopeName(bookie.getBookieId())) .append("."); // check stats on TLS enabled client @@ -959,11 +956,7 @@ public void testHandshakeFailure() throws Exception { InetSocketAddress addr = bookie.getLocalAddress().getSocketAddress(); StringBuilder nameBuilder = new StringBuilder(BookKeeperClientStats.CHANNEL_SCOPE) .append(".") - .append(addr.getAddress().getHostAddress() - .replace('.', '_') - .replace('-', '_')) - .append("_") - .append(addr.getPort()) + .append(PerChannelBookieClient.buildStatsLoggerScopeName(bookie.getBookieId())) .append("."); assertEquals("TLS handshake failure expected", 1, diff --git a/conf/bk_server.conf b/conf/bk_server.conf index e6e3558a988..e58577727b2 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -33,6 +33,10 @@ ## Server parameters ############################################################################# +# Unique ID for the bookie, if it is not set if will be computed from the network address +# of the Bookie. +#bookieId= + # The port that the bookie server listens on. bookiePort=3181 diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java index b9c0a092472..2d710ad6cb6 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/AdminCommandTest.java @@ -131,7 +131,7 @@ private void mockUpdateBookieIdInCookie() throws Exception { when(cookieVersioned.getValue()).thenReturn(cookie); Cookie.Builder builder = mock(Cookie.Builder.class); PowerMockito.when(Cookie.newBuilder(eq(cookie))).thenReturn(builder); - PowerMockito.when(builder.setBookieHost(anyString())).thenReturn(builder); + PowerMockito.when(builder.setBookieId(anyString())).thenReturn(builder); when(builder.build()).thenReturn(cookie); PowerMockito.when(serverConfiguration.setUseHostNameAsBookieID(anyBoolean())).thenReturn(serverConfiguration); diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java index 5b571dfd0d5..7a39d2b6068 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GenerateCookieCommandTest.java @@ -136,7 +136,7 @@ public void testGenerateCookieWithoutInstanceId() throws Exception { String instanceId = "test-instance-id"; Cookie cookie = Cookie.newBuilder() - .setBookieHost(BOOKIE_ID) + .setBookieId(BOOKIE_ID) .setInstanceId(instanceId) .setJournalDirs(journalDir) .setLedgerDirs(Cookie.encodeDirPaths(ledgersDir.split(","))) @@ -172,7 +172,7 @@ public void testGenerateCookieWithInstanceId() throws Exception { String instanceId = "test-instance-id"; Cookie cookie = Cookie.newBuilder() - .setBookieHost(BOOKIE_ID) + .setBookieId(BOOKIE_ID) .setInstanceId(instanceId) .setJournalDirs(journalDir) .setLedgerDirs(Cookie.encodeDirPaths(ledgersDir.split(","))) diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java index 4c555074be0..36f9d7fb0fe 100644 --- a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java +++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/cookie/GetCookieCommandTest.java @@ -96,7 +96,7 @@ public void testInvalidBookieId() { @Test public void testGetCookieFromExistentCookieFile() throws Exception { Cookie cookie = Cookie.newBuilder() - .setBookieHost(BOOKIE_ID) + .setBookieId(BOOKIE_ID) .setInstanceId("test-instance-id") .setJournalDirs("/path/to/journal/dir") .setLedgerDirs("/path/to/ledger/dirs") From bf4f68049605b113bc52026538485f8900dfc190 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Mon, 21 Sep 2020 11:44:58 +0200 Subject: [PATCH 0472/1642] Spammy log when one bookie of ensemble is down #2285 Descriptions of the changes in this PR: ### Motivation Resolves #2285 ### Changes We write "bookie server unavailable log" every 30 seconds per bookie client Reviewers: Enrico Olivelli This closes #2393 from nicoloboschi/fix/2285/spammy-logs, closes #2285 --- bookkeeper-server/pom.xml | 2 +- .../bookkeeper/conf/ClientConfiguration.java | 26 +++++++++++++++++++ .../proto/PerChannelBookieClient.java | 17 +++++++++--- bookkeeper-stats/pom.xml | 2 +- pom.xml | 6 ++--- stream/distributedlog/pom.xml | 3 ++- 6 files changed, 45 insertions(+), 11 deletions(-) diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index 5bef417a026..67e7524b831 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -234,7 +234,7 @@ ${maven-javadoc-plugin.version} - -Xdoclint:none + none org.apache.bookkeeper.client:org.apache.bookkeeper.conf:org.apache.bookkeeper.feature 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 eddb913d06c..663ad63e50b 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 @@ -190,6 +190,10 @@ public class ClientConfiguration extends AbstractConfiguration LOG.warn("Could not connect to bookie: {}/{}, current state {} : {}", + future.channel(), bookieId, state, future.cause().getMessage())); } else { // Regular exceptions, include stack trace - LOG.error("Could not connect to bookie: {}/{}, current state {} : ", - future.channel(), bookieId, state, future.cause()); + logBookieUnavailable(() -> LOG.error("Could not connect to bookie: {}/{}, current state {} : ", + future.channel(), bookieId, state, future.cause())); } rc = BKException.Code.BookieHandleNotAvailableException; @@ -2517,6 +2518,14 @@ public void operationComplete(ChannelFuture future) { makeWritable(); } + + private void logBookieUnavailable(Runnable logger) { + final long now = System.currentTimeMillis(); + if ((now - lastBookieUnavailableLogTimestamp) > conf.getClientConnectBookieUnavailableLogThrottlingMs()) { + logger.run(); + lastBookieUnavailableLogTimestamp = now; + } + } } private void initiateTLS() { diff --git a/bookkeeper-stats/pom.xml b/bookkeeper-stats/pom.xml index 40867c8802f..d8eca58d7c3 100644 --- a/bookkeeper-stats/pom.xml +++ b/bookkeeper-stats/pom.xml @@ -34,7 +34,7 @@ ${maven-javadoc-plugin.version} - -Xdoclint:none + none org.apache.bookkeeper.stats diff --git a/pom.xml b/pom.xml index 5302235567a..6b9b0391e05 100644 --- a/pom.xml +++ b/pom.xml @@ -870,11 +870,9 @@ maven-javadoc-plugin ${maven-javadoc-plugin.version} + -notimestamp - -Xdoclint:none -notimestamp - - -Xdoclint:none -notimestamp - + none org.apache.bookkeeper.client:org.apache.bookkeeper.client.api:org.apache.bookkeeper.common.annotation:org.apache.bookkeeper.conf:org.apache.bookkeeper.feature:org.apache.bookkeeper.stats diff --git a/stream/distributedlog/pom.xml b/stream/distributedlog/pom.xml index 06e6b65ca6b..1dceec9a75b 100644 --- a/stream/distributedlog/pom.xml +++ b/stream/distributedlog/pom.xml @@ -45,7 +45,8 @@ ${maven-javadoc-plugin.version} - -Xdoclint:none -notimestamp + -notimestamp + none Core Library From 9c657d4fe056b139818f36963bc94edeb305173b Mon Sep 17 00:00:00 2001 From: gaoxinge Date: Mon, 21 Sep 2020 22:34:28 +0800 Subject: [PATCH 0473/1642] Fix potential NullPointerException when local book keeper stop on exit Descriptions of the changes in this PR: - Fix potential NullPointerException when local book keeper stop on exit. Reviewers: Enrico Olivelli , Sijie Guo This closes #2414 from gaoxinge/fix --- .../java/org/apache/bookkeeper/util/LocalBookKeeper.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 d9570ffc3bf..c264e6b6777 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 @@ -168,7 +168,7 @@ private static void cleanupDirectories(List dirs) throws IOException { private List runBookies(String dirSuffix) throws Exception { - List tempDirs = new ArrayList(); + List tempDirs = new ArrayList<>(); try { runBookies(tempDirs, dirSuffix); return tempDirs; @@ -391,7 +391,9 @@ static void startLocalBookiesInternal(ServerConfiguration conf, throw e; } finally { if (stopOnExit) { - cleanupDirectories(bkTmpDirs); + if (null != bkTmpDirs) { + cleanupDirectories(bkTmpDirs); + } if (null != zkTmpDir) { FileUtils.deleteDirectory(zkTmpDir); } From 6669a64a9d7ad0aba162233301a0563d569d3f27 Mon Sep 17 00:00:00 2001 From: WJL3333 Date: Sat, 26 Sep 2020 14:46:15 +0800 Subject: [PATCH 0474/1642] fix doc (#2419) the web site rendering error in section reference/cli/ updateBookieInLedger Options --- site/_data/cli/shell.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/site/_data/cli/shell.yaml b/site/_data/cli/shell.yaml index 585947e5862..edd56ecb31a 100644 --- a/site/_data/cli/shell.yaml +++ b/site/_data/cli/shell.yaml @@ -187,9 +187,9 @@ commands: use this command to update ledger metadata by replacing src-bookie to dest-bookie where data has been copied/moved. Start the bookie process on dest-bookie and dest-bookie will serve copied ledger data from src-bookie. options: - - flag: -srcBookie + - flag: -srcBookie BOOKIE_ID description: Source Bookie Id - - flag: -destBookie + - flag: -destBookie BOOKIE_ID description: Destination Bookie Id - flag: -updatespersec N description: Number of ledgers updating per second (default 5 per sec) From b9af3f08102e2b01f18805c8683a0784f8d039ba Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 6 Oct 2020 10:49:05 +0200 Subject: [PATCH 0475/1642] Update Copyright year to 2020 in NOTICE files MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Changes: - update Copyright reference year to 2020 in main NOTICE file and in templates for the binary distributions Reviewers: Nicolò Boschi , Flavio Junqueira , Alessandro Luccaroni , Rajan Dhabalia This closes #2428 from eolivelli/fix/update-year-notice --- NOTICE | 2 +- bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt | 2 +- bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt | 2 +- bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/NOTICE b/NOTICE index 64cd460010b..c2788b62efd 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache BookKeeper -Copyright 2011-2018 The Apache Software Foundation +Copyright 2011-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt index f4739b542e6..695f3786917 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt @@ -1,5 +1,5 @@ Apache BookKeeper -Copyright 2011-2018 The Apache Software Foundation +Copyright 2011-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt index 5d834105907..c0c1ba22472 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-bkctl.bin.txt @@ -1,5 +1,5 @@ Apache BookKeeper -Copyright 2011-2018 The Apache Software Foundation +Copyright 2011-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt index 1b9a34cebb9..b650cb1cd77 100644 --- a/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt @@ -1,5 +1,5 @@ Apache BookKeeper -Copyright 2011-2018 The Apache Software Foundation +Copyright 2011-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From 7a8bb21e755c3b0219e2ac0a99a599bcc18cc20b Mon Sep 17 00:00:00 2001 From: Anup Ghatage Date: Tue, 6 Oct 2020 01:53:15 -0700 Subject: [PATCH 0476/1642] New Bookkeeper docusaurus based website The Apache Bookkeeper website is a custom built website. While that is cool, its also additional maintenance for the PMC/Committers. sijie suggested that we can follow the same path as the Pulsar project and have a docusaurus based website which uses markdown heavily and is easier to manage and update. This is a a WIP and this draft PR is to keep the PMC/Committers updated about the progress on the website. The goal of this PR is to be able to create a basic skeleton of the website with the following pages (and their sub pages) fully fleshed out: * Project * Community * Blog * Landing page So with this, the only thing that remains is the rest of the versioned documentation. That work can be done in smaller modules as follow ups. This PR will be opened for review and then can be merged once the above mentioned pages are declared to be complete. cc eolivelli Reviewers: Enrico Olivelli , Sijie Guo This closes #2426 from Ghatage/docusaurus-website-poc --- pom.xml | 1 + site2/.dockerignore | 2 + site2/.gitignore | 12 + site2/Dockerfile | 10 + site2/docker-compose.yml | 18 ++ site2/docs/bylaws.md | 16 + site2/docs/doc2.md | 7 + site2/docs/doc3.md | 14 + site2/docs/exampledoc4.md | 6 + site2/docs/exampledoc5.md | 6 + site2/docs/getting-started.md | 29 ++ site2/website/README.md | 198 ++++++++++++ .../blog/2017-10-24-new-version-1.0.0.md | 7 + site2/website/core/Footer.js | 203 ++++++++++++ site2/website/package.json | 14 + site2/website/pages/en/community.js | 300 ++++++++++++++++++ site2/website/pages/en/index.js | 206 ++++++++++++ site2/website/pages/en/powered-by.js | 60 ++++ site2/website/pages/en/project.js | 56 ++++ site2/website/pages/en/users.js | 48 +++ site2/website/sidebars.json | 10 + site2/website/siteConfig.js | 124 ++++++++ .../static/css/code-blocks-buttons.css | 46 +++ site2/website/static/css/custom.css | 149 +++++++++ site2/website/static/img/bk-header.png | Bin 0 -> 25961 bytes site2/website/static/img/favicon.ico | Bin 0 -> 37987 bytes site2/website/static/img/oss_logo.png | Bin 0 -> 4370 bytes .../website/static/img/undraw_code_review.svg | 1 + site2/website/static/img/undraw_monitor.svg | 1 + site2/website/static/img/undraw_note_list.svg | 1 + site2/website/static/img/undraw_online.svg | 1 + .../website/static/img/undraw_open_source.svg | 1 + .../static/img/undraw_operating_system.svg | 1 + site2/website/static/img/undraw_react.svg | 1 + .../website/static/img/undraw_tweetstorm.svg | 1 + .../static/img/undraw_youtube_tutorial.svg | 1 + 36 files changed, 1551 insertions(+) create mode 100644 site2/.dockerignore create mode 100644 site2/.gitignore create mode 100644 site2/Dockerfile create mode 100644 site2/docker-compose.yml create mode 100644 site2/docs/bylaws.md create mode 100644 site2/docs/doc2.md create mode 100644 site2/docs/doc3.md create mode 100644 site2/docs/exampledoc4.md create mode 100644 site2/docs/exampledoc5.md create mode 100644 site2/docs/getting-started.md create mode 100644 site2/website/README.md create mode 100644 site2/website/blog/2017-10-24-new-version-1.0.0.md create mode 100644 site2/website/core/Footer.js create mode 100644 site2/website/package.json create mode 100644 site2/website/pages/en/community.js create mode 100644 site2/website/pages/en/index.js create mode 100644 site2/website/pages/en/powered-by.js create mode 100644 site2/website/pages/en/project.js create mode 100644 site2/website/pages/en/users.js create mode 100644 site2/website/sidebars.json create mode 100644 site2/website/siteConfig.js create mode 100644 site2/website/static/css/code-blocks-buttons.css create mode 100644 site2/website/static/css/custom.css create mode 100644 site2/website/static/img/bk-header.png create mode 100644 site2/website/static/img/favicon.ico create mode 100644 site2/website/static/img/oss_logo.png create mode 100644 site2/website/static/img/undraw_code_review.svg create mode 100644 site2/website/static/img/undraw_monitor.svg create mode 100644 site2/website/static/img/undraw_note_list.svg create mode 100644 site2/website/static/img/undraw_online.svg create mode 100644 site2/website/static/img/undraw_open_source.svg create mode 100644 site2/website/static/img/undraw_operating_system.svg create mode 100644 site2/website/static/img/undraw_react.svg create mode 100644 site2/website/static/img/undraw_tweetstorm.svg create mode 100644 site2/website/static/img/undraw_youtube_tutorial.svg diff --git a/pom.xml b/pom.xml index 6b9b0391e05..ee65bf2e9da 100644 --- a/pom.xml +++ b/pom.xml @@ -964,6 +964,7 @@ site/** + site2/** **/org/apache/distributedlog/thrift/* diff --git a/site2/.dockerignore b/site2/.dockerignore new file mode 100644 index 00000000000..27d2dae2b49 --- /dev/null +++ b/site2/.dockerignore @@ -0,0 +1,2 @@ +*/node_modules +*.log diff --git a/site2/.gitignore b/site2/.gitignore new file mode 100644 index 00000000000..5395ea795d6 --- /dev/null +++ b/site2/.gitignore @@ -0,0 +1,12 @@ +.DS_Store + +node_modules + +lib/core/metadata.js +lib/core/MetadataBlog.js + +website/translated_docs +website/build/ +website/yarn.lock +website/node_modules +website/i18n/* diff --git a/site2/Dockerfile b/site2/Dockerfile new file mode 100644 index 00000000000..c7f1856c141 --- /dev/null +++ b/site2/Dockerfile @@ -0,0 +1,10 @@ +FROM node:lts + +WORKDIR /app/website + +EXPOSE 3000 35729 +COPY ./docs /app/docs +COPY ./website /app/website +RUN yarn install + +CMD ["yarn", "start"] diff --git a/site2/docker-compose.yml b/site2/docker-compose.yml new file mode 100644 index 00000000000..6711192ae1e --- /dev/null +++ b/site2/docker-compose.yml @@ -0,0 +1,18 @@ +version: "3" + +services: + docusaurus: + build: . + ports: + - 3000:3000 + - 35729:35729 + volumes: + - ./docs:/app/docs + - ./website/blog:/app/website/blog + - ./website/core:/app/website/core + - ./website/i18n:/app/website/i18n + - ./website/pages:/app/website/pages + - ./website/static:/app/website/static + - ./website/sidebars.json:/app/website/sidebars.json + - ./website/siteConfig.js:/app/website/siteConfig.js + working_dir: /app/website diff --git a/site2/docs/bylaws.md b/site2/docs/bylaws.md new file mode 100644 index 00000000000..78dd503863d --- /dev/null +++ b/site2/docs/bylaws.md @@ -0,0 +1,16 @@ +--- +id: bylaws +title: Bylaws +--- + +# Introduction +Lorem ipsum dolor sit amet, consectetur adipiscing elit. In ac euismod odio, eu consequat dui. Nullam molestie consectetur risus id imperdiet. Proin sodales ornare turpis, non mollis massa ultricies id. Nam at nibh scelerisque, feugiat ante non, dapibus tortor. Vivamus volutpat diam quis tellus elementum bibendum. Praesent semper gravida velit quis aliquam. Etiam in cursus neque. Nam lectus ligula, malesuada et mauris a, bibendum faucibus mi. Phasellus ut interdum felis. Phasellus in odio pulvinar, porttitor urna eget, fringilla lectus. Aliquam sollicitudin est eros. Mauris consectetur quam vitae mauris interdum hendrerit. Lorem ipsum dolor sit amet, consectetur adipiscing elit. + +# Roles and responsibilities +Duis et egestas libero, imperdiet faucibus ipsum. Sed posuere eget urna vel feugiat. Vivamus a arcu sagittis, fermentum urna dapibus, congue lectus. Fusce vulputate porttitor nisl, ac cursus elit volutpat vitae. Nullam vitae ipsum egestas, convallis quam non, porta nibh. Morbi gravida erat nec neque bibendum, eu pellentesque velit posuere. Fusce aliquam erat eu massa eleifend tristique. + +Sed consequat sollicitudin ipsum eget tempus. Integer a aliquet velit. In justo nibh, pellentesque non suscipit eget, gravida vel lacus. Donec odio ante, malesuada in massa quis, pharetra tristique ligula. Donec eros est, tristique eget finibus quis, semper non nisl. Vivamus et elit nec enim ornare placerat. Sed posuere odio a elit cursus sagittis. + +Phasellus feugiat purus eu tortor ultrices finibus. Ut libero nibh, lobortis et libero nec, dapibus posuere eros. Sed sagittis euismod justo at consectetur. Nulla finibus libero placerat, cursus sapien at, eleifend ligula. Vivamus elit nisl, hendrerit ac nibh eu, ultrices tempus dui. Nam tellus neque, commodo non rhoncus eu, gravida in risus. Nullam id iaculis tortor. + +Nullam at odio in sem varius tempor sit amet vel lorem. Etiam eu hendrerit nisl. Fusce nibh mauris, vulputate sit amet ex vitae, congue rhoncus nisl. Sed eget tellus purus. Nullam tempus commodo erat ut tristique. Cras accumsan massa sit amet justo consequat eleifend. Integer scelerisque vitae tellus id consectetur. diff --git a/site2/docs/doc2.md b/site2/docs/doc2.md new file mode 100644 index 00000000000..20c0c95a6bc --- /dev/null +++ b/site2/docs/doc2.md @@ -0,0 +1,7 @@ +--- +id: doc2 +title: document number 2 +--- + +This is a link to [another document.](doc3.md) +This is a link to an [external page.](http://www.example.com) diff --git a/site2/docs/doc3.md b/site2/docs/doc3.md new file mode 100644 index 00000000000..2cc6183cfff --- /dev/null +++ b/site2/docs/doc3.md @@ -0,0 +1,14 @@ +--- +id: doc3 +title: This is document number 3 +--- + +Lorem ipsum dolor sit amet, consectetur adipiscing elit. In ac euismod odio, eu consequat dui. Nullam molestie consectetur risus id imperdiet. Proin sodales ornare turpis, non mollis massa ultricies id. Nam at nibh scelerisque, feugiat ante non, dapibus tortor. Vivamus volutpat diam quis tellus elementum bibendum. Praesent semper gravida velit quis aliquam. Etiam in cursus neque. Nam lectus ligula, malesuada et mauris a, bibendum faucibus mi. Phasellus ut interdum felis. Phasellus in odio pulvinar, porttitor urna eget, fringilla lectus. Aliquam sollicitudin est eros. Mauris consectetur quam vitae mauris interdum hendrerit. Lorem ipsum dolor sit amet, consectetur adipiscing elit. + +Duis et egestas libero, imperdiet faucibus ipsum. Sed posuere eget urna vel feugiat. Vivamus a arcu sagittis, fermentum urna dapibus, congue lectus. Fusce vulputate porttitor nisl, ac cursus elit volutpat vitae. Nullam vitae ipsum egestas, convallis quam non, porta nibh. Morbi gravida erat nec neque bibendum, eu pellentesque velit posuere. Fusce aliquam erat eu massa eleifend tristique. + +Sed consequat sollicitudin ipsum eget tempus. Integer a aliquet velit. In justo nibh, pellentesque non suscipit eget, gravida vel lacus. Donec odio ante, malesuada in massa quis, pharetra tristique ligula. Donec eros est, tristique eget finibus quis, semper non nisl. Vivamus et elit nec enim ornare placerat. Sed posuere odio a elit cursus sagittis. + +Phasellus feugiat purus eu tortor ultrices finibus. Ut libero nibh, lobortis et libero nec, dapibus posuere eros. Sed sagittis euismod justo at consectetur. Nulla finibus libero placerat, cursus sapien at, eleifend ligula. Vivamus elit nisl, hendrerit ac nibh eu, ultrices tempus dui. Nam tellus neque, commodo non rhoncus eu, gravida in risus. Nullam id iaculis tortor. + +Nullam at odio in sem varius tempor sit amet vel lorem. Etiam eu hendrerit nisl. Fusce nibh mauris, vulputate sit amet ex vitae, congue rhoncus nisl. Sed eget tellus purus. Nullam tempus commodo erat ut tristique. Cras accumsan massa sit amet justo consequat eleifend. Integer scelerisque vitae tellus id consectetur. diff --git a/site2/docs/exampledoc4.md b/site2/docs/exampledoc4.md new file mode 100644 index 00000000000..6f94ffe91b5 --- /dev/null +++ b/site2/docs/exampledoc4.md @@ -0,0 +1,6 @@ +--- +id: doc4 +title: Other Document +--- + +this is another document diff --git a/site2/docs/exampledoc5.md b/site2/docs/exampledoc5.md new file mode 100644 index 00000000000..92e2d0d5b91 --- /dev/null +++ b/site2/docs/exampledoc5.md @@ -0,0 +1,6 @@ +--- +id: doc5 +title: Fifth Document +--- + +Another one diff --git a/site2/docs/getting-started.md b/site2/docs/getting-started.md new file mode 100644 index 00000000000..7381eb50dcf --- /dev/null +++ b/site2/docs/getting-started.md @@ -0,0 +1,29 @@ +--- +id: getting-started +title: Latin-ish Lol +sidebar_label: Example Page +--- + +Check the [documentation](https://docusaurus.io) for how to use Docusaurus. + +## Lorem + +Lorem ipsum dolor sit amet, consectetur adipiscing elit. Pellentesque elementum dignissim ultricies. Fusce rhoncus ipsum tempor eros aliquam consequat. Lorem ipsum dolor sit amet, consectetur adipiscing elit. Vivamus elementum massa eget nulla aliquet sagittis. Proin odio tortor, vulputate ut odio in, ultrices ultricies augue. Cras ornare ultrices lorem malesuada iaculis. Etiam sit amet libero tempor, pulvinar mauris sed, sollicitudin sapien. + +## Mauris In Code + +``` +Mauris vestibulum ullamcorper nibh, ut semper purus pulvinar ut. Donec volutpat orci sit amet mauris malesuada, non pulvinar augue aliquam. Vestibulum ultricies at urna ut suscipit. Morbi iaculis, erat at imperdiet semper, ipsum nulla sodales erat, eget tincidunt justo dui quis justo. Pellentesque dictum bibendum diam at aliquet. Sed pulvinar, dolor quis finibus ornare, eros odio facilisis erat, eu rhoncus nunc dui sed ex. Nunc gravida dui massa, sed ornare arcu tincidunt sit amet. Maecenas efficitur sapien neque, a laoreet libero feugiat ut. +``` + +## Nulla + +Nulla facilisi. Maecenas sodales nec purus eget posuere. Sed sapien quam, pretium a risus in, porttitor dapibus erat. Sed sit amet fringilla ipsum, eget iaculis augue. Integer sollicitudin tortor quis ultricies aliquam. Suspendisse fringilla nunc in tellus cursus, at placerat tellus scelerisque. Sed tempus elit a sollicitudin rhoncus. Nulla facilisi. Morbi nec dolor dolor. Orci varius natoque penatibus et magnis dis parturient montes, nascetur ridiculus mus. Cras et aliquet lectus. Pellentesque sit amet eros nisi. Quisque ac sapien in sapien congue accumsan. Nullam in posuere ante. Vestibulum ante ipsum primis in faucibus orci luctus et ultrices posuere cubilia Curae; Proin lacinia leo a nibh fringilla pharetra. + +## Orci + +Orci varius natoque penatibus et magnis dis parturient montes, nascetur ridiculus mus. Proin venenatis lectus dui, vel ultrices ante bibendum hendrerit. Aenean egestas feugiat dui id hendrerit. Orci varius natoque penatibus et magnis dis parturient montes, nascetur ridiculus mus. Curabitur in tellus laoreet, eleifend nunc id, viverra leo. Proin vulputate non dolor vel vulputate. Curabitur pretium lobortis felis, sit amet finibus lorem suscipit ut. Sed non mollis risus. Duis sagittis, mi in euismod tincidunt, nunc mauris vestibulum urna, at euismod est elit quis erat. Phasellus accumsan vitae neque eu placerat. In elementum arcu nec tellus imperdiet, eget maximus nulla sodales. Curabitur eu sapien eget nisl sodales fermentum. + +## Phasellus + +Phasellus pulvinar ex id commodo imperdiet. Praesent odio nibh, sollicitudin sit amet faucibus id, placerat at metus. Donec vitae eros vitae tortor hendrerit finibus. Interdum et malesuada fames ac ante ipsum primis in faucibus. Quisque vitae purus dolor. Duis suscipit ac nulla et finibus. Phasellus ac sem sed dui dictum gravida. Phasellus eleifend vestibulum facilisis. Integer pharetra nec enim vitae mattis. Duis auctor, lectus quis condimentum bibendum, nunc dolor aliquam massa, id bibendum orci velit quis magna. Ut volutpat nulla nunc, sed interdum magna condimentum non. Sed urna metus, scelerisque vitae consectetur a, feugiat quis magna. Donec dignissim ornare nisl, eget tempor risus malesuada quis. diff --git a/site2/website/README.md b/site2/website/README.md new file mode 100644 index 00000000000..6477a55a509 --- /dev/null +++ b/site2/website/README.md @@ -0,0 +1,198 @@ +This website was created with [Docusaurus](https://docusaurus.io/). + +# What's In This Document + +- [Get Started in 5 Minutes](#get-started-in-5-minutes) +- [Directory Structure](#directory-structure) +- [Editing Content](#editing-content) +- [Adding Content](#adding-content) +- [Full Documentation](#full-documentation) + +# Get Started in 5 Minutes + +1. Make sure all the dependencies for the website are installed: + +```sh +# Install dependencies +$ yarn +``` + +2. Run your dev server: + +```sh +# Start the site +$ yarn start +``` + +## Directory Structure + +Your project file structure should look something like this + +``` +my-docusaurus/ + docs/ + doc-1.md + doc-2.md + doc-3.md + website/ + blog/ + 2016-3-11-oldest-post.md + 2017-10-24-newest-post.md + core/ + node_modules/ + pages/ + static/ + css/ + img/ + package.json + sidebars.json + siteConfig.js +``` + +# Editing Content + +## Editing an existing docs page + +Edit docs by navigating to `docs/` and editing the corresponding document: + +`docs/doc-to-be-edited.md` + +```markdown +--- +id: page-needs-edit +title: This Doc Needs To Be Edited +--- + +Edit me... +``` + +For more information about docs, click [here](https://docusaurus.io/docs/en/navigation) + +## Editing an existing blog post + +Edit blog posts by navigating to `website/blog` and editing the corresponding post: + +`website/blog/post-to-be-edited.md` + +```markdown +--- +id: post-needs-edit +title: This Blog Post Needs To Be Edited +--- + +Edit me... +``` + +For more information about blog posts, click [here](https://docusaurus.io/docs/en/adding-blog) + +# Adding Content + +## Adding a new docs page to an existing sidebar + +1. Create the doc as a new markdown file in `/docs`, example `docs/newly-created-doc.md`: + +```md +--- +id: newly-created-doc +title: This Doc Needs To Be Edited +--- + +My new content here.. +``` + +1. Refer to that doc's ID in an existing sidebar in `website/sidebars.json`: + +```javascript +// Add newly-created-doc to the Getting Started category of docs +{ + "docs": { + "Getting Started": [ + "quick-start", + "newly-created-doc" // new doc here + ], + ... + }, + ... +} +``` + +For more information about adding new docs, click [here](https://docusaurus.io/docs/en/navigation) + +## Adding a new blog post + +1. Make sure there is a header link to your blog in `website/siteConfig.js`: + +`website/siteConfig.js` + +```javascript +headerLinks: [ + ... + { blog: true, label: 'Blog' }, + ... +] +``` + +2. Create the blog post with the format `YYYY-MM-DD-My-Blog-Post-Title.md` in `website/blog`: + +`website/blog/2018-05-21-New-Blog-Post.md` + +```markdown +--- +author: Frank Li +authorURL: https://twitter.com/foobarbaz +authorFBID: 503283835 +title: New Blog Post +--- + +Lorem Ipsum... +``` + +For more information about blog posts, click [here](https://docusaurus.io/docs/en/adding-blog) + +## Adding items to your site's top navigation bar + +1. Add links to docs, custom pages or external links by editing the headerLinks field of `website/siteConfig.js`: + +`website/siteConfig.js` + +```javascript +{ + headerLinks: [ + ... + /* you can add docs */ + { doc: 'my-examples', label: 'Examples' }, + /* you can add custom pages */ + { page: 'help', label: 'Help' }, + /* you can add external links */ + { href: 'https://github.com/facebook/docusaurus', label: 'GitHub' }, + ... + ], + ... +} +``` + +For more information about the navigation bar, click [here](https://docusaurus.io/docs/en/navigation) + +## Adding custom pages + +1. Docusaurus uses React components to build pages. The components are saved as .js files in `website/pages/en`: +1. If you want your page to show up in your navigation header, you will need to update `website/siteConfig.js` to add to the `headerLinks` element: + +`website/siteConfig.js` + +```javascript +{ + headerLinks: [ + ... + { page: 'my-new-custom-page', label: 'My New Custom Page' }, + ... + ], + ... +} +``` + +For more information about custom pages, click [here](https://docusaurus.io/docs/en/custom-pages). + +# Full Documentation + +Full documentation can be found on the [website](https://docusaurus.io/). diff --git a/site2/website/blog/2017-10-24-new-version-1.0.0.md b/site2/website/blog/2017-10-24-new-version-1.0.0.md new file mode 100644 index 00000000000..4a88cdcf823 --- /dev/null +++ b/site2/website/blog/2017-10-24-new-version-1.0.0.md @@ -0,0 +1,7 @@ +--- +title: New Website Version 1.0.0 +author: Anup Ghatage +authorURL: http://twitter.com/ghatageanup +--- + +Welcome to the new Apache Bookkeeper website! diff --git a/site2/website/core/Footer.js b/site2/website/core/Footer.js new file mode 100644 index 00000000000..0b273456015 --- /dev/null +++ b/site2/website/core/Footer.js @@ -0,0 +1,203 @@ +/** + * Copyright (c) 2017-present, Facebook, Inc. + * + * This source code is licensed under the MIT license found in the + * LICENSE file in the root directory of this source tree. + */ + +const React = require('react'); + +/* +class Footer extends React.Component { + docUrl(doc, language) { + const baseUrl = this.props.config.baseUrl; + const docsUrl = this.props.config.docsUrl; + const docsPart = `${docsUrl ? `${docsUrl}/` : ''}`; + const langPart = `${language ? `${language}/` : ''}`; + return `${baseUrl}${docsPart}${langPart}${doc}`; + } + + pageUrl(doc, language) { + const baseUrl = this.props.config.baseUrl; + return baseUrl + (language ? `${language}/` : '') + doc; + } + + render() { + return ( +

            + ); + } +} +*/ + +class Footer extends React.Component { + docUrl(doc, language) { + const baseUrl = this.props.config.baseUrl; + return baseUrl + 'docs/' + (language ? language + '/' : '') + doc; + } + + pageUrl(doc, language) { + const baseUrl = this.props.config.baseUrl; + return baseUrl + (language ? language + '/' : '') + doc; + } + + render() { + const currentYear = new Date().getFullYear(); + + const contactUrl = this.pageUrl('contact', this.props.language) + const eventsUrl = this.pageUrl('events', this.props.language) + const twitterUrl = 'https://twitter.com/asfbookkeeper' + const wikiUrl = 'https://github.com/apache/bookkeeper/wiki' + const issuesUrl = 'https://github.com/apache/bookkeeper/issues' + const resourcesUrl = this.pageUrl('resources', this.props.language) + const teamUrl = this.pageUrl('team', this.props.language) + const poweredByUrl = this.pageUrl('powered-by', this.props.language) + const contributingUrl = this.pageUrl('contributing', this.props.language) + const codingUrl = this.pageUrl('coding-guide', this.props.language) + + const communityMenuJs = ` + const community = document.querySelector("a[href='#community']").parentNode; + const communityMenu = + '
          • ' + + 'Community  ▼' + + '
            ' + + '' + + '
            ' + + '
          • '; + + community.innerHTML = communityMenu; + + const communityMenuItem = document.getElementById("community-menu"); + const communityDropDown = document.getElementById("community-dropdown"); + communityMenuItem.addEventListener("click", function(event) { + event.preventDefault(); + + if (communityDropDown.className == 'hide') { + communityDropDown.className = 'visible'; + } else { + communityDropDown.className = 'hide'; + } + }); + ` + + return ( +
            +
            {this.props.config.copyright}
            + + diff --git a/site/_includes/head.html b/site/_includes/head.html deleted file mode 100644 index bd929244b42..00000000000 --- a/site/_includes/head.html +++ /dev/null @@ -1,20 +0,0 @@ -{{ site.title }} - {{ page.title }} - - - - - - - - - - - - - - \ No newline at end of file diff --git a/site/_includes/javadoc-button.html b/site/_includes/javadoc-button.html deleted file mode 100644 index 31d37386aa3..00000000000 --- a/site/_includes/javadoc-button.html +++ /dev/null @@ -1,8 +0,0 @@ - - - - - - Javadoc - - diff --git a/site/_includes/javascript.html b/site/_includes/javascript.html deleted file mode 100644 index e6b54ece641..00000000000 --- a/site/_includes/javascript.html +++ /dev/null @@ -1 +0,0 @@ - diff --git a/site/_includes/navbar.html b/site/_includes/navbar.html deleted file mode 100644 index b9d8499f0a6..00000000000 --- a/site/_includes/navbar.html +++ /dev/null @@ -1,119 +0,0 @@ -{% capture docs_url %}{{ site.baseurl }}docs/{% endcapture %} - diff --git a/site/_includes/popovers.html b/site/_includes/popovers.html deleted file mode 100644 index 0aa83a1d689..00000000000 --- a/site/_includes/popovers.html +++ /dev/null @@ -1,16 +0,0 @@ -{% assign popovers = site.data.popovers %} -{% for popover in popovers %} -
            - {{ popover.d | markdownify }} -
            -{% endfor %} - - \ No newline at end of file diff --git a/site/_includes/shell.html b/site/_includes/shell.html deleted file mode 100644 index 5e0927f66e3..00000000000 --- a/site/_includes/shell.html +++ /dev/null @@ -1,37 +0,0 @@ -{% assign commands = site.data.cli.shell.commands %} - -{% for command in commands %} -

            {{ command.name }}

            - -{{ command.description | markdownify }} - -
            Usage
            - -```shell -$ bin/bookkeeper shell {{ command.name }}{% if command.argument %} \ - {{ command.argument }}{% endif %}{% if command.options %} \ - {% endif %} -``` - -{% if command.options %} -
            Options
            - -

    - - - - - - - - {% for option in command.options %} - - - - - {% endfor %} - -
    FlagDescription
    {{ option.flag }}{{ option.description }}
    -{% endif %} -{% unless forloop.last %}


    {% endunless %} -{% endfor %} diff --git a/site/_includes/sidebar.html b/site/_includes/sidebar.html deleted file mode 100644 index d3d9de47b6e..00000000000 --- a/site/_includes/sidebar.html +++ /dev/null @@ -1,24 +0,0 @@ -{% unless page.no_sidebar %} - -{% endunless %} diff --git a/site/_includes/system-requirements.md b/site/_includes/system-requirements.md deleted file mode 100644 index 69e372f7cf1..00000000000 --- a/site/_includes/system-requirements.md +++ /dev/null @@ -1,8 +0,0 @@ -The number of bookies you should run in a BookKeeper cluster depends on the quorum mode that you've chosen, the desired throughput, and the number of clients using the cluster simultaneously. - -Quorum type | Number of bookies -:-----------|:----------------- -Self-verifying quorum | 3 -Generic | 4 - -Increasing the number of bookies will enable higher throughput, and there is **no upper limit** on the number of bookies. \ No newline at end of file diff --git a/site/_includes/table-of-contents.html b/site/_includes/table-of-contents.html deleted file mode 100644 index 053d812ef1c..00000000000 --- a/site/_includes/table-of-contents.html +++ /dev/null @@ -1,6 +0,0 @@ -{% if page.toc %} -
    -

    {{ page.title }}

    - {{ content | toc_only }} -
    -{% endif %} diff --git a/site/_layouts/community.html b/site/_layouts/community.html deleted file mode 100644 index b42b870637e..00000000000 --- a/site/_layouts/community.html +++ /dev/null @@ -1,48 +0,0 @@ ---- -layout: default ---- - -
    -
    -
    -
    - - - {% if page.subtitle %}

    {{ page.subtitle }}

    {% endif %} -
    - -
    - -
    -
    - {{ content }} -
    - - {% if page.next or page.prev %} - - {% endif %} -
    -
    -
    -
    \ No newline at end of file diff --git a/site/_layouts/default.html b/site/_layouts/default.html deleted file mode 100644 index 5e544667301..00000000000 --- a/site/_layouts/default.html +++ /dev/null @@ -1,20 +0,0 @@ - - - - {% include head.html %} - - -
    - {% include navbar.html %} - - {{ content }} -
    - - {% include footer.html %} - - - {% include javascript.html %} - {% if jekyll.environment == "production" %} - {% include google-analytics.html %} - {% endif %} - diff --git a/site/_layouts/docs.html b/site/_layouts/docs.html deleted file mode 100644 index d2313beb093..00000000000 --- a/site/_layouts/docs.html +++ /dev/null @@ -1,62 +0,0 @@ ---- -layout: default ---- - -
    -
    -
    -
    - {% include sidebar.html %} -
    -
    - -
    -
    - - - {% if page.subtitle %}

    {{ page.subtitle }}

    {% endif %} -
    - -
    - -
    -
    - {{ content }} -
    - - {% if page.next or page.prev %} - - {% endif %} -
    -
    - -
    - {% unless page.toc_disable %} - {% include table-of-contents.html %} - {% endunless %} -
    -
    -
    - -{% include popovers.html %} diff --git a/site/_layouts/javadoc-overview.html b/site/_layouts/javadoc-overview.html deleted file mode 100644 index 71220a5de63..00000000000 --- a/site/_layouts/javadoc-overview.html +++ /dev/null @@ -1,3 +0,0 @@ - - {{ content }} - diff --git a/site/_plugins/popovers.rb b/site/_plugins/popovers.rb deleted file mode 100644 index 6bac5bc2e6c..00000000000 --- a/site/_plugins/popovers.rb +++ /dev/null @@ -1,22 +0,0 @@ -require 'yaml' - -TERMS = YAML.load(File.open("_data/popovers.yaml")) - -module Jekyll - class RenderPopover < Liquid::Tag - def initialize(tag_name, text, tokens) - @original_term = text.strip.split(' ').join(' ') - @term = @original_term.gsub(' ', '-').downcase - @term = 'entry' if @term == 'entries' - @term = 'fencing' if @term == 'fence' - @term = 'striping' if @term == 'stripe' or @term == 'stripedma' - @term = @term[0...-1] if @term.end_with? 's' - end - - def render(ctx) - return "#{@original_term}" - end - end -end - -Liquid::Template.register_tag('pop', Jekyll::RenderPopover) diff --git a/site/_sass/_mixins.sass b/site/_sass/_mixins.sass deleted file mode 100644 index bb011e35458..00000000000 --- a/site/_sass/_mixins.sass +++ /dev/null @@ -1,8 +0,0 @@ -=sticky-footer - .body - display: flex - min-height: 100vh - flex-direction: column - - .main - flex: 1 diff --git a/site/_sass/_navbar.sass b/site/_sass/_navbar.sass deleted file mode 100644 index 785e926e31d..00000000000 --- a/site/_sass/_navbar.sass +++ /dev/null @@ -1,15 +0,0 @@ -.bk-topnav - width: 90% - margin: 0 auto - position: sticky - top: 0 - left: 0 - right: 0 - display: block - - a.bk-brand - font-size: 2rem - margin-right: 2rem - - img - margin-right: 1rem \ No newline at end of file diff --git a/site/_sass/_syntax.sass b/site/_sass/_syntax.sass deleted file mode 100644 index 1a6b2cfeda0..00000000000 --- a/site/_sass/_syntax.sass +++ /dev/null @@ -1,120 +0,0 @@ -blockquote - code - font-size: 1.1rem - background-color: lighten($sx-light-gray, 30%) - -.highlighter-rouge - & + blockquote, & + h1, & + h2, & + h3, & + h4, & + h5, & + p - margin-top: 1.5rem - - // Language-specific overrides - &.language-shell, &.language-bash - .highlight - .nb, .o, .k - // This prevents keywords like cd and help from being highlighted - color: $sx-light-gray - - pre.highlight - padding: $sx-padding - color: $sx-light-gray - background-color: $sx-bg-color - border-radius: $sx-border-radius - border: $sx-border - overflow: $sx-overflow - - code - font-size: .9rem - - .w - color: $sx-light-gray - background-color: $sx-bg-color - - .n - color: $sx-red - - .err - color: #002b36 - background-color: #dc322f - - .c, .cd, .cm, .c1, .cs - color: $sx-comment - - .gp - color: $sx-red - - .cp - color: $sx-tan - - .nd - color: $sx-purple - - .nt - color: $sx-tan - - .o - color: $sx-tan - - .ow - color: $sx-dark-gray - - .p, .pi - color: $sx-dark-gray - - .bp - color: $sx-dark-gray - - .gi - color: $sx-olive - - .gd - color: #dc322f - - .gh - color: $sx-7 - background-color: $sx-bg-color - font-weight: bold - - .k, .kn, .kp, .kr, .kv - color: $sx-purple - - .kc - color: $sx-red - - .kt - color: $sx-purple - - .kd - color: $sx-magenta - - .s, .sb, .sc, .sd, .s2, .sh, .sx, .s1 - color: $sx-olive - - .sr - color: #2aa198 - - .si, .se - color: $sx-magenta - - .nn - color: $sx-tan - - .nf - color: $sx-purple - - .nc, .nb - color: $sx-tan - - .no - color: $sx-tan - - .na - color: $sx-7 - - .m, .il, .mo, .mb, .mx - color: $sx-olive - - .mi, .mf, .mh - color: $sx-tan - - .ss - color: $sx-olive diff --git a/site/_sass/_typography.sass b/site/_sass/_typography.sass deleted file mode 100644 index 690761cf6f1..00000000000 --- a/site/_sass/_typography.sass +++ /dev/null @@ -1,11 +0,0 @@ -$base-docs-font-size: 1.1rem - -.content - h2 - font-size: $base-docs-font-size * 1.5 !important - - h3 - font-size: $base-docs-font-size * 1.25 !important - - p, td, li - font-size: $base-docs-font-size \ No newline at end of file diff --git a/site/_sass/_variables.sass b/site/_sass/_variables.sass deleted file mode 100644 index 67c9a5eb254..00000000000 --- a/site/_sass/_variables.sass +++ /dev/null @@ -1,32 +0,0 @@ -@import url(https://fonts.googleapis.com/css?family=Lato|Source+Code+Pro) - -$base-margin: 1rem -$navbar-height: 5rem -$docs-container-vertical-margin: 3 * $base-margin -$docs-title-margin: 0 * $base-margin -$em-right-margin: 0.2rem - -// Bulma variables -$tan: #c88e56 -$blue: #1976d2 -$primary: $tan -$twitter-blue: #4099ff -$almost-black: rgb(35, 41, 55) - -$family-sans-serif: 'Lato', -apple-system, BlinkMacSystemFont, "Segoe UI", "Roboto", "Oxygen", "Ubuntu", "Cantarell", "Fira Sans", "Droid Sans", "Helvetica Neue", "Helvetica", "Arial", sans-serif -$family-monospace: 'Source Code Pro', monospace - -$sx-border-radius: .25rem -$sx-border: none -$sx-bg-color: $almost-black -$sx-light-gray: lighten(#93a1a1, 15%) -$sx-dark-gray: lighten(#657b83, 30%) -$sx-olive: #859900 -$sx-tan: $tan -$sx-purple: lighten(#6c71c4, 8%) -$sx-magenta: #d33682 -$sx-red: lighten(#cb4b16, 10%) -$sx-7: lighten(#268bd2, 5%) -$sx-comment: darken($sx-dark-gray, 20%) -$sx-overflow: scroll -$sx-padding: 0 diff --git a/site/_sass/vendor/bulma/bulma.sass b/site/_sass/vendor/bulma/bulma.sass deleted file mode 100755 index e8e9abf48a5..00000000000 --- a/site/_sass/vendor/bulma/bulma.sass +++ /dev/null @@ -1,8 +0,0 @@ -@charset "utf-8" -/*! bulma.io v0.4.4 | MIT License | github.com/jgthms/bulma */ -@import "sass/utilities/_all" -@import "sass/base/_all" -@import "sass/elements/_all" -@import "sass/components/_all" -@import "sass/grid/_all" -@import "sass/layout/_all" diff --git a/site/_sass/vendor/bulma/sass/base/_all.sass b/site/_sass/vendor/bulma/sass/base/_all.sass deleted file mode 100755 index e913d6ba394..00000000000 --- a/site/_sass/vendor/bulma/sass/base/_all.sass +++ /dev/null @@ -1,5 +0,0 @@ -@charset "utf-8" - -@import "minireset.sass" -@import "generic.sass" -@import "helpers.sass" diff --git a/site/_sass/vendor/bulma/sass/base/generic.sass b/site/_sass/vendor/bulma/sass/base/generic.sass deleted file mode 100755 index f1b85968b69..00000000000 --- a/site/_sass/vendor/bulma/sass/base/generic.sass +++ /dev/null @@ -1,106 +0,0 @@ -html - background-color: $body-background - font-size: $body-size - -moz-osx-font-smoothing: grayscale - -webkit-font-smoothing: antialiased - min-width: 300px - overflow-x: hidden - overflow-y: scroll - text-rendering: $render-mode - -article, -aside, -figure, -footer, -header, -hgroup, -section - display: block - -body, -button, -input, -select, -textarea - font-family: $family-primary - -code, -pre - -moz-osx-font-smoothing: auto - -webkit-font-smoothing: auto - font-family: $family-code - -body - color: $text - font-size: 1rem - font-weight: $weight-normal - line-height: 1.5 - -// Inline - -a - color: $link - cursor: pointer - text-decoration: none - transition: none $speed $easing - &:hover - color: $link-hover - -code - background-color: $code-background - color: $code - font-size: 0.8em - font-weight: normal - padding: 0.25em 0.5em 0.25em - -hr - background-color: $border - border: none - display: block - height: 1px - margin: 1.5rem 0 - -img - height: auto - max-width: 100% - -input[type="checkbox"], -input[type="radio"] - vertical-align: baseline - -small - font-size: 0.875em - -span - font-style: inherit - font-weight: inherit - -strong - color: $text-strong - font-weight: $weight-bold - -// Block - -pre - background-color: $pre-background - color: $pre - font-size: 0.8em - white-space: pre - word-wrap: normal - code - +overflow-touch - background: none - color: inherit - display: block - font-size: 1em - overflow-x: auto - padding: 1.25rem 1.5rem - -table - width: 100% - td, - th - text-align: left - vertical-align: top - th - color: $text-strong diff --git a/site/_sass/vendor/bulma/sass/base/helpers.sass b/site/_sass/vendor/bulma/sass/base/helpers.sass deleted file mode 100755 index 84d85e1d621..00000000000 --- a/site/_sass/vendor/bulma/sass/base/helpers.sass +++ /dev/null @@ -1,136 +0,0 @@ -// Display - -$displays: 'block' 'flex' 'inline' 'inline-block' 'inline-flex' - -@each $display in $displays - .is-#{$display} - display: #{$display} - +mobile - .is-#{$display}-mobile - display: #{$display} !important - +tablet - .is-#{$display}-tablet - display: #{$display} !important - +tablet-only - .is-#{$display}-tablet-only - display: #{$display} !important - +touch - .is-#{$display}-touch - display: #{$display} !important - +desktop - .is-#{$display}-desktop - display: #{$display} !important - +desktop-only - .is-#{$display}-desktop-only - display: #{$display} !important - +widescreen - .is-#{$display}-widescreen - display: #{$display} !important - -// Float - -.is-clearfix - +clearfix - -.is-pulled-left - float: left !important - -.is-pulled-right - float: right !important - -// Overflow - -.is-clipped - overflow: hidden !important - -// Overlay - -.is-overlay - +overlay - -// Text - -@each $size in $sizes - $i: index($sizes, $size) - .is-size-#{$i} - font-size: $size - +mobile - .is-size-#{$i}-mobile - font-size: $size - +tablet - .is-size-#{$i}-tablet - font-size: $size - +touch - .is-size-#{$i}-touch - font-size: $size - +desktop - .is-size-#{$i}-desktop - font-size: $size - +widescreen - .is-size-#{$i}-widescreen - font-size: $size - +fullhd - .is-size-#{$i}-fullhd - font-size: $size - -.has-text-centered - text-align: center !important - -.has-text-left - text-align: left !important - -.has-text-right - text-align: right !important - -@each $name, $pair in $colors - $color: nth($pair, 1) - .has-text-#{$name} - color: $color - a.has-text-#{$name} - &:hover, - &:focus - color: darken($color, 10%) - -// Visibility - -.is-hidden - display: none !important - -+mobile - .is-hidden-mobile - display: none !important - -+tablet - .is-hidden-tablet - display: none !important - -+tablet-only - .is-hidden-tablet-only - display: none !important - -+touch - .is-hidden-touch - display: none !important - -+desktop - .is-hidden-desktop - display: none !important - -+desktop-only - .is-hidden-desktop-only - display: none !important - -+widescreen - .is-hidden-widescreen - display: none !important - -// Other - -.is-marginless - margin: 0 !important - -.is-paddingless - padding: 0 !important - -.is-unselectable - +unselectable diff --git a/site/_sass/vendor/bulma/sass/base/minireset.sass b/site/_sass/vendor/bulma/sass/base/minireset.sass deleted file mode 100755 index 98ad8147d65..00000000000 --- a/site/_sass/vendor/bulma/sass/base/minireset.sass +++ /dev/null @@ -1,80 +0,0 @@ -/*! minireset.css v0.0.2 | MIT License | github.com/jgthms/minireset.css */ -// Blocks -html, -body, -p, -ol, -ul, -li, -dl, -dt, -dd, -blockquote, -figure, -fieldset, -legend, -textarea, -pre, -iframe, -hr, -h1, -h2, -h3, -h4, -h5, -h6 - margin: 0 - padding: 0 - -// Headings -h1, -h2, -h3, -h4, -h5, -h6 - font-size: 100% - font-weight: normal - -// List -ul - list-style: none - -// Form -button, -input, -select, -textarea - margin: 0 - -// Box sizing -html - box-sizing: border-box - -* - box-sizing: inherit - &:before, - &:after - box-sizing: inherit - -// Media -img, -embed, -object, -audio, -video - max-width: 100% - -// Iframe -iframe - border: 0 - -// Table -table - border-collapse: collapse - border-spacing: 0 - -td, -th - padding: 0 - text-align: left diff --git a/site/_sass/vendor/bulma/sass/components/_all.sass b/site/_sass/vendor/bulma/sass/components/_all.sass deleted file mode 100755 index ed5276e290b..00000000000 --- a/site/_sass/vendor/bulma/sass/components/_all.sass +++ /dev/null @@ -1,15 +0,0 @@ -@charset "utf-8" - -@import "breadcrumb.sass" -@import "card.sass" -@import "dropdown.sass" -@import "level.sass" -@import "media.sass" -@import "menu.sass" -@import "message.sass" -@import "modal.sass" -@import "nav.sass" -@import "navbar.sass" -@import "pagination.sass" -@import "panel.sass" -@import "tabs.sass" diff --git a/site/_sass/vendor/bulma/sass/components/breadcrumb.sass b/site/_sass/vendor/bulma/sass/components/breadcrumb.sass deleted file mode 100755 index e898d258133..00000000000 --- a/site/_sass/vendor/bulma/sass/components/breadcrumb.sass +++ /dev/null @@ -1,66 +0,0 @@ -.breadcrumb - +block - +unselectable - align-items: stretch - display: flex - font-size: $size-normal - overflow: hidden - overflow-x: auto - white-space: nowrap - a - align-items: center - color: $text-light - display: flex - justify-content: center - padding: 0.5em 0.75em - &:hover - color: $link-hover - li - align-items: center - display: flex - &.is-active - a - color: $text-strong - cursor: default - pointer-events: none - & + li:before - color: $text - content: '\0002f' - ul, ol - align-items: center - display: flex - flex-grow: 1 - flex-shrink: 0 - justify-content: flex-start - .icon - &:first-child - margin-right: 0.5em - &:last-child - margin-left: 0.5em - // Alignment - &.is-centered - ol, ul - justify-content: center - &.is-right - ol, ul - justify-content: flex-end - // Sizes - &.is-small - font-size: $size-small - &.is-medium - font-size: $size-medium - &.is-large - font-size: $size-large - // Styles - &.has-arrow-separator - li + li:before - content: '\02192' - &.has-bullet-separator - li + li:before - content: '\02022' - &.has-dot-separator - li + li:before - content: '\000b7' - &.has-succeeds-separator - li + li:before - content: '\0227B' diff --git a/site/_sass/vendor/bulma/sass/components/card.sass b/site/_sass/vendor/bulma/sass/components/card.sass deleted file mode 100755 index 999f56cb948..00000000000 --- a/site/_sass/vendor/bulma/sass/components/card.sass +++ /dev/null @@ -1,65 +0,0 @@ -$card: $text !default -$card-background: $white !default -$card-shadow: 0 2px 3px rgba($black, 0.1), 0 0 0 1px rgba($black, 0.1) !default - -$card-header: $text-strong !default -$card-header-shadow: 0 1px 2px rgba($black, 0.1) !default -$card-header-weight: $weight-bold !default - -$card-footer-border: $border !default - -.card - background-color: $card-background - box-shadow: $card-shadow - color: $card - max-width: 100% - position: relative - -.card-header - align-items: stretch - box-shadow: $card-header-shadow - display: flex - -.card-header-title - align-items: center - color: $card-header - display: flex - flex-grow: 1 - font-weight: $card-header-weight - padding: 0.75rem - -.card-header-icon - align-items: center - cursor: pointer - display: flex - justify-content: center - padding: 0.75rem - -.card-image - display: block - position: relative - -.card-content - padding: 1.5rem - -.card-footer - border-top: 1px solid $card-footer-border - align-items: stretch - display: flex - -.card-footer-item - align-items: center - display: flex - flex-basis: 0 - flex-grow: 1 - flex-shrink: 0 - justify-content: center - padding: 0.75rem - &:not(:last-child) - border-right: 1px solid $card-footer-border - -// Combinations - -.card - .media:not(:last-child) - margin-bottom: 0.75rem diff --git a/site/_sass/vendor/bulma/sass/components/dropdown.sass b/site/_sass/vendor/bulma/sass/components/dropdown.sass deleted file mode 100755 index b4701f55920..00000000000 --- a/site/_sass/vendor/bulma/sass/components/dropdown.sass +++ /dev/null @@ -1,70 +0,0 @@ -$dropdown-content-background: $white !default -$dropdown-content-arrow: $link !default -$dropdown-content-offset: 4px !default -$dropdown-content-radius: $radius !default -$dropdown-content-shadow: 0 2px 3px rgba($black, 0.1), 0 0 0 1px rgba($black, 0.1) !default -$dropdown-content-z: 20 !default - -$dropdown-item: $grey-dark !default -$dropdown-item-hover: $black !default -$dropdown-item-hover-background: $background !default -$dropdown-item-active: $primary-invert !default -$dropdown-item-active-background: $primary !default - -$dropdown-divider-background: $border !default - -.dropdown - display: inline-flex - position: relative - vertical-align: top - &.is-active, - &.is-hoverable:hover - .dropdown-menu - display: block - &.is-right - .dropdown-menu - left: auto - right: 0 - -.dropdown-menu - display: none - left: 0 - max-width: 20rem - min-width: 12rem - padding-top: $dropdown-content-offset - position: absolute - top: 100% - width: 100% - z-index: $dropdown-content-z - -.dropdown-content - background-color: $dropdown-content-background - border-radius: $dropdown-content-radius - box-shadow: $dropdown-content-shadow - padding-bottom: 0.5rem - padding-top: 0.5rem - -.dropdown-item - color: $dropdown-item - display: block - font-size: 0.875rem - line-height: 1.5 - padding: 0.375rem 1rem - position: relative - -a.dropdown-item - padding-right: 3rem - white-space: nowrap - &:hover - background-color: $dropdown-item-hover-background - color: $dropdown-item-hover - &.is-active - background-color: $dropdown-item-active-background - color: $dropdown-item-active - -.dropdown-divider - background-color: $dropdown-divider-background - border: none - display: block - height: 1px - margin: 0.5rem 0 diff --git a/site/_sass/vendor/bulma/sass/components/level.sass b/site/_sass/vendor/bulma/sass/components/level.sass deleted file mode 100755 index 34b982c185f..00000000000 --- a/site/_sass/vendor/bulma/sass/components/level.sass +++ /dev/null @@ -1,72 +0,0 @@ -.level-item - align-items: center - display: flex - flex-basis: auto - flex-grow: 0 - flex-shrink: 0 - justify-content: center - .title, - .subtitle - margin-bottom: 0 - // Responsiveness - +mobile - &:not(:last-child) - margin-bottom: 0.75rem - -.level-left, -.level-right - flex-basis: auto - flex-grow: 0 - flex-shrink: 0 - .level-item - &:not(:last-child) - margin-right: 0.75rem - // Modifiers - &.is-flexible - flex-grow: 1 - -.level-left - align-items: center - justify-content: flex-start - // Responsiveness - +mobile - & + .level-right - margin-top: 1.5rem - +tablet - display: flex - -.level-right - align-items: center - justify-content: flex-end - // Responsiveness - +tablet - display: flex - -.level - +block - align-items: center - justify-content: space-between - code - border-radius: $radius - img - display: inline-block - vertical-align: top - // Modifiers - &.is-mobile - display: flex - .level-left, - .level-right - display: flex - .level-left + .level-right - margin-top: 0 - .level-item - &:not(:last-child) - margin-bottom: 0 - &:not(.is-narrow) - flex-grow: 1 - // Responsiveness - +tablet - display: flex - & > .level-item - &:not(.is-narrow) - flex-grow: 1 diff --git a/site/_sass/vendor/bulma/sass/components/media.sass b/site/_sass/vendor/bulma/sass/components/media.sass deleted file mode 100755 index 575e4fcc95f..00000000000 --- a/site/_sass/vendor/bulma/sass/components/media.sass +++ /dev/null @@ -1,44 +0,0 @@ -.media-left, -.media-right - flex-basis: auto - flex-grow: 0 - flex-shrink: 0 - -.media-left - margin-right: 1rem - -.media-right - margin-left: 1rem - -.media-content - flex-basis: auto - flex-grow: 1 - flex-shrink: 1 - text-align: left - -.media - align-items: flex-start - display: flex - text-align: left - .content:not(:last-child) - margin-bottom: 0.75rem - .media - border-top: 1px solid rgba($border, 0.5) - display: flex - padding-top: 0.75rem - .content:not(:last-child), - .control:not(:last-child) - margin-bottom: 0.5rem - .media - padding-top: 0.5rem - & + .media - margin-top: 0.5rem - & + .media - border-top: 1px solid rgba($border, 0.5) - margin-top: 1rem - padding-top: 1rem - // Sizes - &.is-large - & + .media - margin-top: 1.5rem - padding-top: 1.5rem diff --git a/site/_sass/vendor/bulma/sass/components/menu.sass b/site/_sass/vendor/bulma/sass/components/menu.sass deleted file mode 100755 index 47289704e8d..00000000000 --- a/site/_sass/vendor/bulma/sass/components/menu.sass +++ /dev/null @@ -1,32 +0,0 @@ -.menu - font-size: $size-normal - -.menu-list - line-height: 1.25 - a - border-radius: $radius-small - color: $text - display: block - padding: 0.5em 0.75em - &:hover - background-color: $background - color: $link - // Modifiers - &.is-active - background-color: $link - color: $link-invert - li - ul - border-left: 1px solid $border - margin: 0.75em - padding-left: 0.75em - -.menu-label - color: $text-light - font-size: 0.8em - letter-spacing: 0.1em - text-transform: uppercase - &:not(:first-child) - margin-top: 1em - &:not(:last-child) - margin-bottom: 1em diff --git a/site/_sass/vendor/bulma/sass/components/message.sass b/site/_sass/vendor/bulma/sass/components/message.sass deleted file mode 100755 index cff51e58fd7..00000000000 --- a/site/_sass/vendor/bulma/sass/components/message.sass +++ /dev/null @@ -1,61 +0,0 @@ -.message - +block - background-color: $background - border-radius: $radius - font-size: $size-normal - // Colors - @each $name, $pair in $colors - $color: nth($pair, 1) - $color-invert: nth($pair, 2) - $color-lightning: max((100% - lightness($color)) - 2%, 0%) - $color-luminance: colorLuminance($color) - $darken-percentage: $color-luminance * 70% - $desaturate-percentage: $color-luminance * 30% - &.is-#{$name} - background-color: lighten($color, $color-lightning) - .message-header - background-color: $color - color: $color-invert - .message-body - border-color: $color - color: desaturate(darken($color, $darken-percentage), $desaturate-percentage) - -.message-header - align-items: center - background-color: $text - border-radius: $radius $radius 0 0 - color: $text-invert - display: flex - justify-content: space-between - line-height: 1.25 - padding: 0.5em 0.75em - position: relative - a:not(.button), - strong - color: currentColor - a:not(.button) - text-decoration: underline - .delete - flex-grow: 0 - flex-shrink: 0 - margin-left: 0.75em - & + .message-body - border-top-left-radius: 0 - border-top-right-radius: 0 - border-top: none - -.message-body - border: 1px solid $border - border-radius: $radius - color: $text - padding: 1em 1.25em - a:not(.button), - strong - color: currentColor - a:not(.button) - text-decoration: underline - code, - pre - background: $white - pre code - background: transparent diff --git a/site/_sass/vendor/bulma/sass/components/modal.sass b/site/_sass/vendor/bulma/sass/components/modal.sass deleted file mode 100755 index d3fc015118f..00000000000 --- a/site/_sass/vendor/bulma/sass/components/modal.sass +++ /dev/null @@ -1,81 +0,0 @@ -.modal-background - +overlay - background-color: rgba($black, 0.86) - -.modal-content, -.modal-card - margin: 0 20px - max-height: calc(100vh - 160px) - overflow: auto - position: relative - width: 100% - // Responsiveness - +tablet - margin: 0 auto - max-height: calc(100vh - 40px) - width: 640px - -.modal-close - +delete - background: none - height: 40px - position: fixed - right: 20px - top: 20px - width: 40px - -.modal-card - display: flex - flex-direction: column - max-height: calc(100vh - 40px) - overflow: hidden - -.modal-card-head, -.modal-card-foot - align-items: center - background-color: $background - display: flex - flex-shrink: 0 - justify-content: flex-start - padding: 20px - position: relative - -.modal-card-head - border-bottom: 1px solid $border - border-top-left-radius: $radius-large - border-top-right-radius: $radius-large - -.modal-card-title - color: $text-strong - flex-grow: 1 - flex-shrink: 0 - font-size: $size-4 - line-height: 1 - -.modal-card-foot - border-bottom-left-radius: $radius-large - border-bottom-right-radius: $radius-large - border-top: 1px solid $border - .button - &:not(:last-child) - margin-right: 10px - -.modal-card-body - +overflow-touch - background-color: $white - flex-grow: 1 - flex-shrink: 1 - overflow: auto - padding: 20px - -.modal - +overlay - align-items: center - display: none - justify-content: center - overflow: hidden - position: fixed - z-index: 20 - // Modifiers - &.is-active - display: flex diff --git a/site/_sass/vendor/bulma/sass/components/nav.sass b/site/_sass/vendor/bulma/sass/components/nav.sass deleted file mode 100755 index e6aaa29472b..00000000000 --- a/site/_sass/vendor/bulma/sass/components/nav.sass +++ /dev/null @@ -1,125 +0,0 @@ -$nav-height: 3.25rem !default - -// Components - -.nav-toggle - +hamburger($nav-height) - // Responsiveness - +tablet - display: none - -.nav-item - align-items: center - display: flex - flex-grow: 0 - flex-shrink: 0 - font-size: $size-normal - justify-content: center - line-height: 1.5 - padding: 0.5rem 0.75rem - a - flex-grow: 1 - flex-shrink: 0 - img - max-height: 1.75rem - .tag - &:first-child:not(:last-child) - margin-right: 0.5rem - &:last-child:not(:first-child) - margin-left: 0.5rem - // Responsiveness - +mobile - justify-content: flex-start - -.nav-item a:not(.button), -a.nav-item:not(.button) - color: $text-light - &:hover - color: $link-hover - // Modifiers - &.is-active - color: $link-active - &.is-tab - border-bottom: 1px solid transparent - border-top: 1px solid transparent - padding-bottom: calc(0.75rem - 1px) - padding-left: 1rem - padding-right: 1rem - padding-top: calc(0.75rem - 1px) - &:hover - border-bottom-color: $primary - border-top-color: transparent - &.is-active - border-bottom: 3px solid $primary - color: $primary - padding-bottom: calc(0.75rem - 3px) - // Responsiveness - +desktop - &.is-brand - padding-left: 0 - -// Containers - -.nav-left, -.nav-right - +overflow-touch - align-items: stretch - display: flex - flex-grow: 1 - flex-shrink: 0 - max-width: 100% - overflow: auto - +widescreen - flex-basis: 0 - -.nav-left - justify-content: flex-start - white-space: nowrap - -.nav-right - justify-content: flex-end - -.nav-center - align-items: stretch - display: flex - flex-grow: 0 - flex-shrink: 0 - justify-content: center - margin-left: auto - margin-right: auto - -.nav-menu - // Responsiveness - +mobile - &.nav-right - background-color: $white - box-shadow: 0 4px 7px rgba($black, 0.1) - left: 0 - display: none - right: 0 - top: 100% - position: absolute - .nav-item - border-top: 1px solid rgba($border, 0.5) - padding: 0.75rem - &.is-active - display: block - -// Main container - -.nav - align-items: stretch - background-color: $white - display: flex - height: $nav-height - position: relative - text-align: center - z-index: 10 - & > .container - align-items: stretch - display: flex - min-height: $nav-height - width: 100% - // Modifiers - &.has-shadow - box-shadow: 0 2px 3px rgba($black, 0.1) diff --git a/site/_sass/vendor/bulma/sass/components/navbar.sass b/site/_sass/vendor/bulma/sass/components/navbar.sass deleted file mode 100755 index a5848d01b71..00000000000 --- a/site/_sass/vendor/bulma/sass/components/navbar.sass +++ /dev/null @@ -1,236 +0,0 @@ -$navbar-background: $white !default -$navbar-height: 3.25rem !default - -$navbar-item: $grey-dark !default -$navbar-item-hover: $black !default -$navbar-item-hover-background: $background !default -$navbar-item-active: $black !default -$navbar-item-active-background: transparent !default - -$navbar-tab-hover-background: transparent !default -$navbar-tab-hover-border: $primary !default -$navbar-tab-active: $primary !default -$navbar-tab-active-background: transparent !default -$navbar-tab-active-border: $primary !default - -$navbar-dropdown-background: $white !default -$navbar-dropdown-border: $border !default -$navbar-dropdown-offset: -4px !default -$navbar-dropdown-arrow: $link !default -$navbar-dropdown-radius: $radius-large !default -$navbar-dropdown-z: 20 !default - -$navbar-dropdown-item-hover: $black !default -$navbar-dropdown-item-hover-background: $background !default -$navbar-dropdown-item-active: $primary !default -$navbar-dropdown-item-active-background: $background !default - -$navbar-divider-background: $border !default - -.navbar - background-color: $navbar-background - min-height: $navbar-height - position: relative - & > .container - align-items: stretch - display: flex - min-height: $navbar-height - width: 100% - &.has-shadow - box-shadow: 0 2px 3px rgba($black, 0.1) - -.navbar-brand - +overflow-touch - align-items: stretch - display: flex - min-height: $navbar-height - overflow-x: auto - overflow-y: hidden - -.navbar-burger - +hamburger($navbar-height) - margin-left: auto - -.navbar-menu - display: none - -.navbar-item, -.navbar-link - color: $navbar-item - display: block - line-height: 1.5 - padding: 0.5rem 1rem - position: relative - -a.navbar-item, -.navbar-link - &:hover, - &.is-active - background-color: $navbar-item-hover-background - color: $navbar-item-hover - -.navbar-item - flex-grow: 0 - flex-shrink: 0 - img - max-height: 1.75rem - &.has-dropdown - padding: 0 - &.is-tab - border-bottom: 1px solid transparent - min-height: $navbar-height - padding-bottom: calc(0.5rem - 1px) - &:hover - background-color: $navbar-tab-hover-background - border-bottom-color: $navbar-tab-hover-border - &.is-active - background-color: $navbar-tab-active-background - border-bottom: 3px solid $navbar-tab-active-border - color: $navbar-tab-active - padding-bottom: calc(0.5rem - 3px) - -.navbar-content - flex-grow: 1 - flex-shrink: 1 - -.navbar-link - padding-right: 2.5em - -.navbar-dropdown - font-size: 0.875rem - padding-bottom: 0.5rem - padding-top: 0.5rem - .navbar-item - padding-left: 1.5rem - padding-right: 1.5rem - -.navbar-divider - background-color: $navbar-divider-background - border: none - display: none - height: 1px - margin: 0.5rem 0 - -+touch - .navbar-brand - .navbar-item - align-items: center - display: flex - .navbar-menu - box-shadow: 0 8px 16px rgba($black, 0.1) - padding: 0.5rem 0 - &.is-active - display: block - -+desktop - .navbar, - .navbar-menu, - .navbar-start, - .navbar-end - align-items: stretch - display: flex - .navbar - min-height: $navbar-height - &.is-transparent - a.navbar-item, - .navbar-link - &:hover, - &.is-active - background-color: transparent - .navbar-item.has-dropdown - &.is-active, - &.is-hoverable:hover - .navbar-link - background-color: transparent - .navbar-dropdown - a.navbar-item - &:hover - background-color: $navbar-dropdown-item-hover-background - color: $navbar-dropdown-item-hover - &.is-active - background-color: $navbar-dropdown-item-active-background - color: $navbar-dropdown-item-active - .navbar-burger - display: none - .navbar-item, - .navbar-link - align-items: center - display: flex - .navbar-item - &.has-dropdown - align-items: stretch - &.is-active, - &.is-hoverable:hover - .navbar-dropdown - display: block - &.is-boxed - opacity: 1 - pointer-events: auto - transform: translateY(0) - .navbar-link - &::after - +arrow($navbar-dropdown-arrow) - margin-top: -0.375em - right: 1.125em - top: 50% - .navbar-menu - flex-grow: 1 - flex-shrink: 0 - .navbar-start - justify-content: flex-start - margin-right: auto - .navbar-end - justify-content: flex-end - margin-left: auto - .navbar-dropdown - background-color: $navbar-dropdown-background - border-bottom-left-radius: $navbar-dropdown-radius - border-bottom-right-radius: $navbar-dropdown-radius - border-top: 1px solid $navbar-dropdown-border - box-shadow: 0 8px 8px rgba($black, 0.1) - display: none - font-size: 0.875rem - left: 0 - min-width: 100% - position: absolute - top: 100% - z-index: $navbar-dropdown-z - .navbar-item - padding: 0.375rem 1rem - white-space: nowrap - a.navbar-item - padding-right: 3rem - &:hover - background-color: $navbar-dropdown-item-hover-background - color: $navbar-dropdown-item-hover - &.is-active - background-color: $navbar-dropdown-item-active-background - color: $navbar-dropdown-item-active - &.is-boxed - border-radius: $navbar-dropdown-radius - border-top: none - box-shadow: 0 8px 8px rgba($black, 0.1), 0 0 0 1px rgba($black, 0.1) - display: block - opacity: 0 - pointer-events: none - top: calc(100% + (#{$navbar-dropdown-offset})) - transform: translateY(-5px) - transition-duration: $speed - transition-property: opacity, transform - .navbar-divider - display: block - .container > .navbar - margin-left: -1rem - margin-right: -1rem - // Hover/Active states - a.navbar-item, - .navbar-link - &.is-active - color: $navbar-item-active - &.is-active:not(:hover) - background-color: $navbar-item-active-background - .navbar-item.has-dropdown - &:hover, - &.is-active - .navbar-link - background-color: $navbar-item-hover-background diff --git a/site/_sass/vendor/bulma/sass/components/pagination.sass b/site/_sass/vendor/bulma/sass/components/pagination.sass deleted file mode 100755 index 71af9f0ed88..00000000000 --- a/site/_sass/vendor/bulma/sass/components/pagination.sass +++ /dev/null @@ -1,134 +0,0 @@ -$pagination: $grey-darker !default -$pagination-background: $white !default -$pagination-border: $grey-lighter !default - -$pagination-hover: $link-hover !default -$pagination-hover-border: $link-hover-border !default - -$pagination-focus: $link-focus !default -$pagination-focus-border: $link-focus-border !default - -$pagination-active: $link-active !default -$pagination-active-border: $link-active-border !default - -$pagination-disabled: $grey !default -$pagination-disabled-background: $grey-lighter !default -$pagination-disabled-border: $grey-lighter !default - -$pagination-current: $link-invert !default -$pagination-current-background: $link !default -$pagination-current-border: $link !default - -$pagination-ellipsis: $grey-light !default - -$pagination-shadow-inset: inset 0 1px 2px rgba($black, 0.2) - -.pagination - font-size: $size-normal - margin: -0.25rem - // Sizes - &.is-small - font-size: $size-small - &.is-medium - font-size: $size-medium - &.is-large - font-size: $size-large - -.pagination, -.pagination-list - align-items: center - display: flex - justify-content: center - text-align: center - -.pagination-previous, -.pagination-next, -.pagination-link, -.pagination-ellipsis - +control - +unselectable - font-size: 1em - padding-left: 0.5em - padding-right: 0.5em - justify-content: center - margin: 0.25rem - text-align: center - -.pagination-previous, -.pagination-next, -.pagination-link - border-color: $pagination-border - min-width: 2.25em - &:hover - border-color: $pagination-hover-border - color: $pagination-hover - &:focus - border-color: $pagination-focus-border - &:active - box-shadow: $pagination-shadow-inset - &[disabled] - background-color: $pagination-disabled-background - border-color: $pagination-disabled-border - box-shadow: none - color: $pagination-disabled - opacity: 0.5 - -.pagination-previous, -.pagination-next - padding-left: 0.75em - padding-right: 0.75em - white-space: nowrap - -.pagination-link - &.is-current - background-color: $pagination-current-background - border-color: $pagination-current-border - color: $pagination-current - -.pagination-ellipsis - color: $pagination-ellipsis - pointer-events: none - -.pagination-list - flex-wrap: wrap - -+mobile - .pagination - flex-wrap: wrap - .pagination-previous, - .pagination-next - flex-grow: 1 - flex-shrink: 1 - .pagination-list - li - flex-grow: 1 - flex-shrink: 1 - -+tablet - .pagination-list - flex-grow: 1 - flex-shrink: 1 - justify-content: flex-start - order: 1 - .pagination-previous - order: 2 - .pagination-next - order: 3 - .pagination - justify-content: space-between - &.is-centered - .pagination-previous - order: 1 - .pagination-list - justify-content: center - order: 2 - .pagination-next - order: 3 - &.is-right - .pagination-previous - order: 1 - .pagination-next - order: 2 - .pagination-list - justify-content: flex-end - order: 3 diff --git a/site/_sass/vendor/bulma/sass/components/panel.sass b/site/_sass/vendor/bulma/sass/components/panel.sass deleted file mode 100755 index a37a7731162..00000000000 --- a/site/_sass/vendor/bulma/sass/components/panel.sass +++ /dev/null @@ -1,76 +0,0 @@ -.panel - font-size: $size-normal - &:not(:last-child) - margin-bottom: 1.5rem - -.panel-heading, -.panel-tabs, -.panel-block - border-bottom: 1px solid $border - border-left: 1px solid $border - border-right: 1px solid $border - &:first-child - border-top: 1px solid $border - -.panel-heading - background-color: $background - border-radius: $radius $radius 0 0 - color: $text-strong - font-size: 1.25em - font-weight: $weight-light - line-height: 1.25 - padding: 0.5em 0.75em - -.panel-tabs - align-items: flex-end - display: flex - font-size: 0.875em - justify-content: center - a - border-bottom: 1px solid $border - margin-bottom: -1px - padding: 0.5em - // Modifiers - &.is-active - border-bottom-color: $link-active-border - color: $link-active - -.panel-list - a - color: $text - &:hover - color: $link - -.panel-block - align-items: center - color: $text-strong - display: flex - justify-content: flex-start - padding: 0.5em 0.75em - input[type="checkbox"] - margin-right: 0.75em - & > .control - flex-grow: 1 - flex-shrink: 1 - width: 100% - &.is-wrapped - flex-wrap: wrap - &.is-active - border-left-color: $link - color: $link-active - .panel-icon - color: $link - -a.panel-block, -label.panel-block - cursor: pointer - &:hover - background-color: $background - -.panel-icon - +fa(14px, 1em) - color: $text-light - margin-right: 0.75em - .fa - font-size: inherit - line-height: inherit diff --git a/site/_sass/vendor/bulma/sass/components/tabs.sass b/site/_sass/vendor/bulma/sass/components/tabs.sass deleted file mode 100755 index 605176c55ae..00000000000 --- a/site/_sass/vendor/bulma/sass/components/tabs.sass +++ /dev/null @@ -1,107 +0,0 @@ -.tabs - +block - +overflow-touch - +unselectable - align-items: stretch - display: flex - font-size: $size-normal - justify-content: space-between - overflow: hidden - overflow-x: auto - white-space: nowrap - a - align-items: center - border-bottom: 1px solid $border - color: $text - display: flex - justify-content: center - margin-bottom: -1px - padding: 0.5em 1em - vertical-align: top - &:hover - border-bottom-color: $text-strong - color: $text-strong - li - display: block - &.is-active - a - border-bottom-color: $primary - color: $primary - ul - align-items: center - border-bottom: 1px solid $border - display: flex - flex-grow: 1 - flex-shrink: 0 - justify-content: flex-start - &.is-left - padding-right: 0.75em - &.is-center - flex: none - justify-content: center - padding-left: 0.75em - padding-right: 0.75em - &.is-right - justify-content: flex-end - padding-left: 0.75em - .icon - &:first-child - margin-right: 0.5em - &:last-child - margin-left: 0.5em - // Alignment - &.is-centered - ul - justify-content: center - &.is-right - ul - justify-content: flex-end - // Styles - &.is-boxed - a - border: 1px solid transparent - border-radius: $radius $radius 0 0 - &:hover - background-color: $background - border-bottom-color: $border - li - &.is-active - a - background-color: $white - border-color: $border - border-bottom-color: transparent !important - &.is-fullwidth - li - flex-grow: 1 - flex-shrink: 0 - &.is-toggle - a - border: 1px solid $border - margin-bottom: 0 - position: relative - &:hover - background-color: $background - border-color: $border-hover - z-index: 2 - li - & + li - margin-left: -1px - &:first-child a - border-radius: $radius 0 0 $radius - &:last-child a - border-radius: 0 $radius $radius 0 - &.is-active - a - background-color: $primary - border-color: $primary - color: $primary-invert - z-index: 1 - ul - border-bottom: none - // Sizes - &.is-small - font-size: $size-small - &.is-medium - font-size: $size-medium - &.is-large - font-size: $size-large diff --git a/site/_sass/vendor/bulma/sass/elements/_all.sass b/site/_sass/vendor/bulma/sass/elements/_all.sass deleted file mode 100755 index 54f5b2fcde8..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/_all.sass +++ /dev/null @@ -1,15 +0,0 @@ -@charset "utf-8" - -@import "box.sass" -@import "button.sass" -@import "content.sass" -@import "form.sass" -@import "icon.sass" -@import "image.sass" -@import "notification.sass" -@import "progress.sass" -@import "table.sass" -@import "tag.sass" -@import "title.sass" - -@import "other.sass" diff --git a/site/_sass/vendor/bulma/sass/elements/box.sass b/site/_sass/vendor/bulma/sass/elements/box.sass deleted file mode 100755 index dec3be79c85..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/box.sass +++ /dev/null @@ -1,23 +0,0 @@ -$box: $text !default -$box-background: $white !default -$box-radius: $radius-large !default -$box-shadow: 0 2px 3px rgba($black, 0.1), 0 0 0 1px rgba($black, 0.1) !default - -$box-link-hover-shadow: 0 2px 3px rgba($black, 0.1), 0 0 0 1px $link -$box-link-active-shadow: inset 0 1px 2px rgba($black, 0.2), 0 0 0 1px $link - -.box - +block - background-color: $box-background - border-radius: $box-radius - box-shadow: $box-shadow - color: $box - display: block - padding: 1.25rem - -a.box - &:hover, - &:focus - box-shadow: $box-link-hover-shadow - &:active - box-shadow: $box-link-active-shadow diff --git a/site/_sass/vendor/bulma/sass/elements/button.sass b/site/_sass/vendor/bulma/sass/elements/button.sass deleted file mode 100755 index 23032f9277c..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/button.sass +++ /dev/null @@ -1,197 +0,0 @@ -$button: $grey-darker !default -$button-background: $white !default -$button-border: $grey-lighter !default - -$button-hover: $link-hover !default -$button-hover-border: $link-hover-border !default - -$button-focus: $link-focus !default -$button-focus-border: $link-focus-border !default - -$button-active: $link-active !default -$button-active-border: $link-active-border !default - -$button-static: $grey !default -$button-static-background: $white-ter !default -$button-static-border: $grey-lighter !default - -$button-shadow-inset: inset 0 1px 2px rgba($black, 0.2) !default - -// The button sizes use mixins so they can be used at different breakpoints -=button-small - border-radius: $radius-small - font-size: $size-small -=button-medium - font-size: $size-medium -=button-large - font-size: $size-large - -.button - +control - +unselectable - background-color: $button-background - border-color: $button-border - color: $button - cursor: pointer - justify-content: center - padding-left: 0.75em - padding-right: 0.75em - text-align: center - white-space: nowrap - strong - color: inherit - .icon - &, - &.is-small, - &.is-medium, - &.is-large - height: 1.5em - width: 1.5em - &:first-child:not(:last-child) - margin-left: calc(-0.375em - 1px) - margin-right: 0.1875em - &:last-child:not(:first-child) - margin-left: 0.1875em - margin-right: calc(-0.375em - 1px) - &:first-child:last-child - margin-left: calc(-0.375em - 1px) - margin-right: calc(-0.375em - 1px) - // States - &:hover, - &.is-hovered - border-color: $button-hover-border - color: $button-hover - &:focus, - &.is-focused - border-color: $button-focus-border - box-shadow: 0 0 0.5em rgba($button-focus-border, 0.25) - color: $button-focus - &:active, - &.is-active - border-color: $button-active-border - box-shadow: $button-shadow-inset - color: $button-active - // Colors - &.is-link - background-color: transparent - border-color: transparent - color: $text - text-decoration: underline - &:hover, - &.is-hovered, - &:focus, - &.is-focused, - &:active, - &.is-active - background-color: $background - color: $text-strong - &[disabled] - background-color: transparent - border-color: transparent - box-shadow: none - @each $name, $pair in $colors - $color: nth($pair, 1) - $color-invert: nth($pair, 2) - &.is-#{$name} - background-color: $color - border-color: transparent - color: $color-invert - &:hover, - &.is-hovered - background-color: darken($color, 2.5%) - border-color: transparent - color: $color-invert - &:focus, - &.is-focused - border-color: transparent - box-shadow: 0 0 0.5em rgba($color, 0.25) - color: $color-invert - &:active, - &.is-active - background-color: darken($color, 5%) - border-color: transparent - box-shadow: $button-shadow-inset - color: $color-invert - &[disabled] - background-color: $color - border-color: transparent - box-shadow: none - &.is-inverted - background-color: $color-invert - color: $color - &:hover - background-color: darken($color-invert, 5%) - &[disabled] - background-color: $color-invert - border-color: transparent - box-shadow: none - color: $color - &.is-loading - &:after - border-color: transparent transparent $color-invert $color-invert !important - &.is-outlined - background-color: transparent - border-color: $color - color: $color - &:hover, - &:focus - background-color: $color - border-color: $color - color: $color-invert - &.is-loading - &:after - border-color: transparent transparent $color $color !important - &[disabled] - background-color: transparent - border-color: $color - box-shadow: none - color: $color - &.is-inverted.is-outlined - background-color: transparent - border-color: $color-invert - color: $color-invert - &:hover, - &:focus - background-color: $color-invert - color: $color - &[disabled] - background-color: transparent - border-color: $color-invert - box-shadow: none - color: $color-invert - // Sizes - &.is-small - +button-small - &.is-medium - +button-medium - &.is-large - +button-large - // Modifiers - &[disabled] - background-color: $button-background - border-color: $button-border - box-shadow: none - opacity: 0.5 - &.is-fullwidth - display: flex - width: 100% - &.is-loading - color: transparent !important - pointer-events: none - &:after - +loader - +center(1em) - position: absolute !important - &.is-static - background-color: $button-static-background - border-color: $button-static-border - color: $button-static - box-shadow: none - pointer-events: none - -// Adjustment for vertical spacing -button.button, -input[type="submit"].button - line-height: 1 - padding-bottom: 0.4em - padding-top: 0.35em diff --git a/site/_sass/vendor/bulma/sass/elements/content.sass b/site/_sass/vendor/bulma/sass/elements/content.sass deleted file mode 100755 index 978fd30a09a..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/content.sass +++ /dev/null @@ -1,119 +0,0 @@ -.content - +block - // Inline - li + li - margin-top: 0.25em - // Block - p, - dl, - ol, - ul, - blockquote, - pre, - table - &:not(:last-child) - margin-bottom: 1em - h1, - h2, - h3, - h4, - h5, - h6 - color: $text-strong - font-weight: $weight-normal - line-height: 1.125 - h1 - font-size: 2em - margin-bottom: 0.5em - &:not(:first-child) - margin-top: 1em - h2 - font-size: 1.75em - margin-bottom: 0.5714em - &:not(:first-child) - margin-top: 1.1428em - h3 - font-size: 1.5em - margin-bottom: 0.6666em - &:not(:first-child) - margin-top: 1.3333em - h4 - font-size: 1.25em - margin-bottom: 0.8em - h5 - font-size: 1.125em - margin-bottom: 0.8888em - h6 - font-size: 1em - margin-bottom: 1em - blockquote - background-color: $background - border-left: 5px solid $border - padding: 1.25em 1.5em - ol - list-style: decimal outside - margin-left: 2em - margin-top: 1em - ul - list-style: disc outside - margin-left: 2em - margin-top: 1em - ul - list-style-type: circle - margin-top: 0.5em - ul - list-style-type: square - dd - margin-left: 2em - figure - text-align: center - img - display: inline-block - figcaption - font-style: italic - pre - +overflow-touch - overflow-x: auto - padding: 1.25em 1.5em - white-space: pre - word-wrap: normal - sup, - sub - font-size: 70% - table - width: 100% - td, - th - border: 1px solid $border - border-width: 0 0 1px - padding: 0.5em 0.75em - vertical-align: top - th - color: $text-strong - text-align: left - tr - &:hover - background-color: $background - thead - td, - th - border-width: 0 0 2px - color: $text-strong - tfoot - td, - th - border-width: 2px 0 0 - color: $text-strong - tbody - tr - &:last-child - td, - th - border-bottom-width: 0 - // Sizes - &.is-small - font-size: $size-small - &.is-medium - font-size: $size-medium - &.is-large - font-size: $size-large diff --git a/site/_sass/vendor/bulma/sass/elements/form.sass b/site/_sass/vendor/bulma/sass/elements/form.sass deleted file mode 100755 index af1a120b046..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/form.sass +++ /dev/null @@ -1,391 +0,0 @@ -$input: $grey-darker !default -$input-background: $white !default -$input-border: $grey-lighter !default - -$input-hover: $grey-darker !default -$input-hover-border: $grey-light !default - -$input-focus: $grey-darker !default -$input-focus-border: $link !default - -$input-disabled: $text-light !default -$input-disabled-background: $background !default -$input-disabled-border: $background !default - -$input-arrow: $link !default - -$input-icon: $grey-lighter !default -$input-icon-active: $grey !default - -$input-radius: $radius !default - -=input - +control - background-color: $input-background - border-color: $input-border - color: $input - &:hover, - &.is-hovered - border-color: $input-hover-border - &:focus, - &.is-focused, - &:active, - &.is-active - border-color: $input-focus-border - &[disabled] - background-color: $input-disabled-background - border-color: $input-disabled-border - box-shadow: none - color: $input-disabled - +placeholder - color: rgba($input, 0.3) - -.input, -.textarea - +input - box-shadow: inset 0 1px 2px rgba($black, 0.1) - max-width: 100% - width: 100% - &[type="search"] - border-radius: 290486px - // Colors - @each $name, $pair in $colors - $color: nth($pair, 1) - &.is-#{$name} - border-color: $color - // Sizes - &.is-small - +control-small - &.is-medium - +control-medium - &.is-large - +control-large - // Modifiers - &.is-fullwidth - display: block - width: 100% - &.is-inline - display: inline - width: auto - -.textarea - display: block - max-width: 100% - min-width: 100% - padding: 0.625em - resize: vertical - &:not([rows]) - max-height: 600px - min-height: 120px - &[rows] - height: unset - -.checkbox, -.radio - cursor: pointer - display: inline-block - line-height: 1.25 - position: relative - input - cursor: pointer - &:hover - color: $input-hover - &[disabled] - color: $input-disabled - cursor: not-allowed - -.radio - & + .radio - margin-left: 0.5em - -.select - display: inline-block - max-width: 100% - position: relative - vertical-align: top - &:not(.is-multiple) - height: 2.25em - &::after - +arrow($input-arrow) - margin-top: -0.375em - right: 1.125em - top: 50% - z-index: 4 - select - +input - cursor: pointer - display: block - font-size: 1em - max-width: 100% - outline: none - &:hover - border-color: $input-hover-border - &:focus, - &.is-focused, - &:active, - &.is-active - border-color: $input-focus-border - &::-ms-expand - display: none - &[disabled]:hover - border-color: $input-disabled-border - &:not([multiple]) - padding-right: 2.5em - &[multiple] - height: unset - padding: 0 - option - padding: 0.5em 1em - // States - &:hover - &::after - border-color: $input-hover - // Colors - @each $name, $pair in $colors - $color: nth($pair, 1) - &.is-#{$name} select - border-color: $color - // Sizes - &.is-small - +control-small - &.is-medium - +control-medium - &.is-large - +control-large - // Modifiers - &.is-disabled - &::after - border-color: $input-disabled - &.is-fullwidth - width: 100% - select - width: 100% - &.is-loading - &::after - +loader - margin-top: 0 - position: absolute - right: 0.625em - top: 0.625em - transform: none - &.is-small:after - font-size: $size-small - &.is-medium:after - font-size: $size-medium - &.is-large:after - font-size: $size-large - -.label - color: $input - display: block - font-size: $size-normal - font-weight: $weight-bold - &:not(:last-child) - margin-bottom: 0.5em - // Sizes - &.is-small - font-size: $size-small - &.is-medium - font-size: $size-medium - &.is-large - font-size: $size-large - -.help - display: block - font-size: $size-small - margin-top: 0.25rem - @each $name, $pair in $colors - $color: nth($pair, 1) - &.is-#{$name} - color: $color - -// Containers - -.field - &:not(:last-child) - margin-bottom: 0.75rem - // Modifiers - &.has-addons - display: flex - justify-content: flex-start - .control - &:not(:last-child) - margin-right: -1px - &:first-child - .button, - .input, - .select select - border-bottom-left-radius: $input-radius - border-top-left-radius: $input-radius - &:last-child - .button, - .input, - .select select - border-bottom-right-radius: $input-radius - border-top-right-radius: $input-radius - .button, - .input, - .select select - border-radius: 0 - &:hover, - &.is-hovered - z-index: 2 - &:focus, - &.is-focused, - &:active, - &.is-active - z-index: 3 - &:hover - z-index: 4 - &.is-expanded - flex-grow: 1 - &.has-addons-centered - justify-content: center - &.has-addons-right - justify-content: flex-end - &.has-addons-fullwidth - .control - flex-grow: 1 - flex-shrink: 0 - &.is-grouped - display: flex - justify-content: flex-start - & > .control - flex-shrink: 0 - &:not(:last-child) - margin-bottom: 0 - margin-right: 0.75rem - &.is-expanded - flex-grow: 1 - flex-shrink: 1 - &.is-grouped-centered - justify-content: center - &.is-grouped-right - justify-content: flex-end - &.is-horizontal - +tablet - display: flex - -.field-label - .label - font-size: inherit - +mobile - margin-bottom: 0.5rem - +tablet - flex-basis: 0 - flex-grow: 1 - flex-shrink: 0 - margin-right: 1.5rem - text-align: right - &.is-small - font-size: $size-small - padding-top: 0.375em - &.is-normal - padding-top: 0.375em - &.is-medium - font-size: $size-medium - padding-top: 0.375em - &.is-large - font-size: $size-large - padding-top: 0.375em - -.field-body - .field .field - margin-bottom: 0 - +tablet - display: flex - flex-basis: 0 - flex-grow: 5 - flex-shrink: 1 - .field - margin-bottom: 0 - & > .field - flex-shrink: 1 - &:not(.is-narrow) - flex-grow: 1 - &:not(:last-child) - margin-right: 0.75rem - -.control - font-size: $size-normal - position: relative - text-align: left - // Modifiers - // DEPRECATED - &.has-icon - .icon - color: $input-icon - height: 2.25em - pointer-events: none - position: absolute - top: 0 - width: 2.25em - z-index: 4 - .input - &:focus - & + .icon - color: $input-icon-active - &.is-small - & + .icon - font-size: $size-small - &.is-medium - & + .icon - font-size: $size-medium - &.is-large - & + .icon - font-size: $size-large - &:not(.has-icon-right) - .icon - left: 0 - .input - padding-left: 2.25em - &.has-icon-right - .icon - right: 0 - .input - padding-right: 2.25em - &.has-icons-left, - &.has-icons-right - .input, - .select - &:focus - & ~ .icon - color: $input-icon-active - &.is-small ~ .icon - font-size: $size-small - &.is-medium ~ .icon - font-size: $size-medium - &.is-large ~ .icon - font-size: $size-large - .icon - color: $input-icon - height: 2.25em - pointer-events: none - position: absolute - top: 0 - width: 2.25em - z-index: 4 - &.has-icons-left - .input, - .select select - padding-left: 2.25em - .icon.is-left - left: 0 - &.has-icons-right - .input, - .select select - padding-right: 2.25em - .icon.is-right - right: 0 - &.is-loading - &::after - +loader - position: absolute !important - right: 0.625em - top: 0.625em - &.is-small:after - font-size: $size-small - &.is-medium:after - font-size: $size-medium - &.is-large:after - font-size: $size-large diff --git a/site/_sass/vendor/bulma/sass/elements/icon.sass b/site/_sass/vendor/bulma/sass/elements/icon.sass deleted file mode 100755 index e1c3735aea2..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/icon.sass +++ /dev/null @@ -1,24 +0,0 @@ -.icon - align-items: center - display: inline-flex - justify-content: center - height: 1.5rem - width: 1.5rem - .fa - font-size: 21px - // Sizes - &.is-small - height: 1rem - width: 1rem - .fa - font-size: 14px - &.is-medium - height: 2rem - width: 2rem - .fa - font-size: 28px - &.is-large - height: 3rem - width: 3rem - .fa - font-size: 42px diff --git a/site/_sass/vendor/bulma/sass/elements/image.sass b/site/_sass/vendor/bulma/sass/elements/image.sass deleted file mode 100755 index cc27b444fee..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/image.sass +++ /dev/null @@ -1,36 +0,0 @@ -$dimensions: 16 24 32 48 64 96 128 - -.image - display: block - position: relative - img - display: block - height: auto - width: 100% - // Ratio - &.is-square, - &.is-1by1, - &.is-4by3, - &.is-3by2, - &.is-16by9, - &.is-2by1 - img - +overlay - height: 100% - width: 100% - &.is-square, - &.is-1by1 - padding-top: 100% - &.is-4by3 - padding-top: 75% - &.is-3by2 - padding-top: 66.6666% - &.is-16by9 - padding-top: 56.25% - &.is-2by1 - padding-top: 50% - // Sizes - @each $dimension in $dimensions - &.is-#{$dimension}x#{$dimension} - height: $dimension * 1px - width: $dimension * 1px diff --git a/site/_sass/vendor/bulma/sass/elements/notification.sass b/site/_sass/vendor/bulma/sass/elements/notification.sass deleted file mode 100755 index a3e3d60aba4..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/notification.sass +++ /dev/null @@ -1,31 +0,0 @@ -.notification - +block - background-color: $background - border-radius: $radius - padding: 1.25rem 2.5rem 1.25rem 1.5rem - position: relative - a:not(.button) - color: currentColor - text-decoration: underline - strong - color: currentColor - code, - pre - background: $white - pre code - background: transparent - & > .delete - position: absolute - right: 0.5em - top: 0.5em - .title, - .subtitle, - .content - color: currentColor - // Colors - @each $name, $pair in $colors - $color: nth($pair, 1) - $color-invert: nth($pair, 2) - &.is-#{$name} - background-color: $color - color: $color-invert diff --git a/site/_sass/vendor/bulma/sass/elements/other.sass b/site/_sass/vendor/bulma/sass/elements/other.sass deleted file mode 100755 index 93d87858db7..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/other.sass +++ /dev/null @@ -1,70 +0,0 @@ -.block - +block - -.container - margin: 0 auto - position: relative - +desktop - max-width: $desktop - (2 * $gap) - width: $desktop - (2 * $gap) - &.is-fluid - margin-left: $gap - margin-right: $gap - max-width: none - width: auto - +until($widescreen) - &.is-widescreen - max-width: $widescreen - (2 * $gap) - width: auto - +until($fullhd) - &.is-fullhd - max-width: $fullhd - (2 * $gap) - width: auto - +widescreen - max-width: $widescreen - (2 * $gap) - width: $widescreen - (2 * $gap) - +fullhd - max-width: $fullhd - (2 * $gap) - width: $fullhd - (2 * $gap) - -.delete - +delete - -.fa - font-size: 21px - text-align: center - vertical-align: top - -.heading - display: block - font-size: 11px - letter-spacing: 1px - margin-bottom: 5px - text-transform: uppercase - -.highlight - +block - font-weight: $weight-normal - max-width: 100% - overflow: hidden - padding: 0 - pre - overflow: auto - max-width: 100% - -.loader - +loader - -.number - align-items: center - background-color: $background - border-radius: 290486px - display: inline-flex - font-size: $size-medium - height: 2em - justify-content: center - margin-right: 1.5rem - min-width: 2.5em - padding: 0.25rem 0.5rem - text-align: center - vertical-align: top diff --git a/site/_sass/vendor/bulma/sass/elements/progress.sass b/site/_sass/vendor/bulma/sass/elements/progress.sass deleted file mode 100755 index b17a68787db..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/progress.sass +++ /dev/null @@ -1,32 +0,0 @@ -.progress - +block - -moz-appearance: none - -webkit-appearance: none - border: none - border-radius: 290486px - display: block - height: $size-normal - overflow: hidden - padding: 0 - width: 100% - &::-webkit-progress-bar - background-color: $border - &::-webkit-progress-value - background-color: $text - &::-moz-progress-bar - background-color: $text - // Colors - @each $name, $pair in $colors - $color: nth($pair, 1) - &.is-#{$name} - &::-webkit-progress-value - background-color: $color - &::-moz-progress-bar - background-color: $color - // Sizes - &.is-small - height: $size-small - &.is-medium - height: $size-medium - &.is-large - height: $size-large diff --git a/site/_sass/vendor/bulma/sass/elements/table.sass b/site/_sass/vendor/bulma/sass/elements/table.sass deleted file mode 100755 index 8bbaf468686..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/table.sass +++ /dev/null @@ -1,82 +0,0 @@ -$table: $grey-darker !default -$table-background: $white !default -$table-border: $grey-lighter !default - -$table-head: $grey !default - -$table-row-hover-background: $white-bis !default - -$table-row-active-background: $primary !default -$table-row-active: $primary-invert !default - -$table-row-even-background: $white-bis !default -$table-row-even-hover-background: $white-ter !default - -.table - background-color: $table-background - color: $table - margin-bottom: 1.5rem - width: 100% - td, - th - border: 1px solid $table-border - border-width: 0 0 1px - padding: 0.5em 0.75em - vertical-align: top - // Modifiers - &.is-narrow - white-space: nowrap - width: 1% - th - color: $text-strong - text-align: left - tr - &:hover - background-color: $table-row-hover-background - &.is-selected - background-color: $table-row-active-background - color: $table-row-active - a, - strong - color: currentColor - td, - th - border-color: $table-row-active - color: currentColor - thead - td, - th - border-width: 0 0 2px - color: $table-head - tfoot - td, - th - border-width: 2px 0 0 - color: $table-head - tbody - tr - &:last-child - td, - th - border-bottom-width: 0 - // Modifiers - &.is-bordered - td, - th - border-width: 1px - tr - &:last-child - td, - th - border-bottom-width: 1px - &.is-narrow - td, - th - padding: 0.25em 0.5em - &.is-striped - tbody - tr:not(.is-selected) - &:nth-child(even) - background-color: $table-row-even-background - &:hover - background-color: $table-row-even-hover-background diff --git a/site/_sass/vendor/bulma/sass/elements/tag.sass b/site/_sass/vendor/bulma/sass/elements/tag.sass deleted file mode 100755 index 477c6622f97..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/tag.sass +++ /dev/null @@ -1,28 +0,0 @@ -.tag - align-items: center - background-color: $background - border-radius: 290486px - color: $text - display: inline-flex - font-size: $size-small - height: 2em - justify-content: center - line-height: 1.5 - padding-left: 0.875em - padding-right: 0.875em - white-space: nowrap - .delete - margin-left: 0.25em - margin-right: -0.375em - // Colors - @each $name, $pair in $colors - $color: nth($pair, 1) - $color-invert: nth($pair, 2) - &.is-#{$name} - background-color: $color - color: $color-invert - // Sizes - &.is-medium - font-size: $size-normal - &.is-large - font-size: $size-medium diff --git a/site/_sass/vendor/bulma/sass/elements/title.sass b/site/_sass/vendor/bulma/sass/elements/title.sass deleted file mode 100755 index 98893ec30bb..00000000000 --- a/site/_sass/vendor/bulma/sass/elements/title.sass +++ /dev/null @@ -1,53 +0,0 @@ -$title: $grey-darker !default -$title-size: $size-3 !default -$title-weight: $weight-light !default -$title-weight-bold: $weight-semibold !default - -$subtitle: $grey-dark !default -$subtitle-size: $size-5 !default -$subtitle-strong: $grey-darker !default -$subtitle-weight: $weight-light !default - -.title, -.subtitle - +block - word-break: break-word - em, - span - font-weight: $title-weight - strong - font-weight: $title-weight-bold - .tag - vertical-align: middle - -.title - color: $title - font-size: $title-size - font-weight: $title-weight - line-height: 1.125 - strong - color: inherit - & + .highlight - margin-top: -0.75rem - &:not(.is-spaced) + .subtitle - margin-top: -1.5rem - // Sizes - @each $size in $sizes - $i: index($sizes, $size) - &.is-#{$i} - font-size: $size - -.subtitle - color: $subtitle - font-size: $subtitle-size - font-weight: $subtitle-weight - line-height: 1.25 - strong - color: $subtitle-strong - &:not(.is-spaced) + .title - margin-top: -1.5rem - // Sizes - @each $size in $sizes - $i: index($sizes, $size) - &.is-#{$i} - font-size: $size diff --git a/site/_sass/vendor/bulma/sass/grid/_all.sass b/site/_sass/vendor/bulma/sass/grid/_all.sass deleted file mode 100755 index e53070f6c32..00000000000 --- a/site/_sass/vendor/bulma/sass/grid/_all.sass +++ /dev/null @@ -1,4 +0,0 @@ -@charset "utf-8" - -@import "columns.sass" -@import "tiles.sass" diff --git a/site/_sass/vendor/bulma/sass/grid/columns.sass b/site/_sass/vendor/bulma/sass/grid/columns.sass deleted file mode 100755 index 0dcfeb503d4..00000000000 --- a/site/_sass/vendor/bulma/sass/grid/columns.sass +++ /dev/null @@ -1,325 +0,0 @@ -.column - display: block - flex-basis: 0 - flex-grow: 1 - flex-shrink: 1 - padding: 0.75rem - .columns.is-mobile > &.is-narrow - flex: none - .columns.is-mobile > &.is-full - flex: none - width: 100% - .columns.is-mobile > &.is-three-quarters - flex: none - width: 75% - .columns.is-mobile > &.is-two-thirds - flex: none - width: 66.6666% - .columns.is-mobile > &.is-half - flex: none - width: 50% - .columns.is-mobile > &.is-one-third - flex: none - width: 33.3333% - .columns.is-mobile > &.is-one-quarter - flex: none - width: 25% - .columns.is-mobile > &.is-offset-three-quarters - margin-left: 75% - .columns.is-mobile > &.is-offset-two-thirds - margin-left: 66.6666% - .columns.is-mobile > &.is-offset-half - margin-left: 50% - .columns.is-mobile > &.is-offset-one-third - margin-left: 33.3333% - .columns.is-mobile > &.is-offset-one-quarter - margin-left: 25% - @for $i from 1 through 12 - .columns.is-mobile > &.is-#{$i} - flex: none - width: ($i / 12) * 100% - .columns.is-mobile > &.is-offset-#{$i} - margin-left: ($i / 12) * 100% - +mobile - &.is-narrow-mobile - flex: none - &.is-full-mobile - flex: none - width: 100% - &.is-three-quarters-mobile - flex: none - width: 75% - &.is-two-thirds-mobile - flex: none - width: 66.6666% - &.is-half-mobile - flex: none - width: 50% - &.is-one-third-mobile - flex: none - width: 33.3333% - &.is-one-quarter-mobile - flex: none - width: 25% - &.is-offset-three-quarters-mobile - margin-left: 75% - &.is-offset-two-thirds-mobile - margin-left: 66.6666% - &.is-offset-half-mobile - margin-left: 50% - &.is-offset-one-third-mobile - margin-left: 33.3333% - &.is-offset-one-quarter-mobile - margin-left: 25% - @for $i from 1 through 12 - &.is-#{$i}-mobile - flex: none - width: ($i / 12) * 100% - &.is-offset-#{$i}-mobile - margin-left: ($i / 12) * 100% - +tablet - &.is-narrow, - &.is-narrow-tablet - flex: none - &.is-full, - &.is-full-tablet - flex: none - width: 100% - &.is-three-quarters, - &.is-three-quarters-tablet - flex: none - width: 75% - &.is-two-thirds, - &.is-two-thirds-tablet - flex: none - width: 66.6666% - &.is-half, - &.is-half-tablet - flex: none - width: 50% - &.is-one-third, - &.is-one-third-tablet - flex: none - width: 33.3333% - &.is-one-quarter, - &.is-one-quarter-tablet - flex: none - width: 25% - &.is-offset-three-quarters, - &.is-offset-three-quarters-tablet - margin-left: 75% - &.is-offset-two-thirds, - &.is-offset-two-thirds-tablet - margin-left: 66.6666% - &.is-offset-half, - &.is-offset-half-tablet - margin-left: 50% - &.is-offset-one-third, - &.is-offset-one-third-tablet - margin-left: 33.3333% - &.is-offset-one-quarter, - &.is-offset-one-quarter-tablet - margin-left: 25% - @for $i from 1 through 12 - &.is-#{$i}, - &.is-#{$i}-tablet - flex: none - width: ($i / 12) * 100% - &.is-offset-#{$i}, - &.is-offset-#{$i}-tablet - margin-left: ($i / 12) * 100% - +touch - &.is-narrow-touch - flex: none - &.is-full-touch - flex: none - width: 100% - &.is-three-quarters-touch - flex: none - width: 75% - &.is-two-thirds-touch - flex: none - width: 66.6666% - &.is-half-touch - flex: none - width: 50% - &.is-one-third-touch - flex: none - width: 33.3333% - &.is-one-quarter-touch - flex: none - width: 25% - &.is-offset-three-quarters-touch - margin-left: 75% - &.is-offset-two-thirds-touch - margin-left: 66.6666% - &.is-offset-half-touch - margin-left: 50% - &.is-offset-one-third-touch - margin-left: 33.3333% - &.is-offset-one-quarter-touch - margin-left: 25% - @for $i from 1 through 12 - &.is-#{$i}-touch - flex: none - width: ($i / 12) * 100% - &.is-offset-#{$i}-touch - margin-left: ($i / 12) * 100% - +desktop - &.is-narrow-desktop - flex: none - &.is-full-desktop - flex: none - width: 100% - &.is-three-quarters-desktop - flex: none - width: 75% - &.is-two-thirds-desktop - flex: none - width: 66.6666% - &.is-half-desktop - flex: none - width: 50% - &.is-one-third-desktop - flex: none - width: 33.3333% - &.is-one-quarter-desktop - flex: none - width: 25% - &.is-offset-three-quarters-desktop - margin-left: 75% - &.is-offset-two-thirds-desktop - margin-left: 66.6666% - &.is-offset-half-desktop - margin-left: 50% - &.is-offset-one-third-desktop - margin-left: 33.3333% - &.is-offset-one-quarter-desktop - margin-left: 25% - @for $i from 1 through 12 - &.is-#{$i}-desktop - flex: none - width: ($i / 12) * 100% - &.is-offset-#{$i}-desktop - margin-left: ($i / 12) * 100% - +widescreen - &.is-narrow-widescreen - flex: none - &.is-full-widescreen - flex: none - width: 100% - &.is-three-quarters-widescreen - flex: none - width: 75% - &.is-two-thirds-widescreen - flex: none - width: 66.6666% - &.is-half-widescreen - flex: none - width: 50% - &.is-one-third-widescreen - flex: none - width: 33.3333% - &.is-one-quarter-widescreen - flex: none - width: 25% - &.is-offset-three-quarters-widescreen - margin-left: 75% - &.is-offset-two-thirds-widescreen - margin-left: 66.6666% - &.is-offset-half-widescreen - margin-left: 50% - &.is-offset-one-third-widescreen - margin-left: 33.3333% - &.is-offset-one-quarter-widescreen - margin-left: 25% - @for $i from 1 through 12 - &.is-#{$i}-widescreen - flex: none - width: ($i / 12) * 100% - &.is-offset-#{$i}-widescreen - margin-left: ($i / 12) * 100% - +fullhd - &.is-narrow-fullhd - flex: none - &.is-full-fullhd - flex: none - width: 100% - &.is-three-quarters-fullhd - flex: none - width: 75% - &.is-two-thirds-fullhd - flex: none - width: 66.6666% - &.is-half-fullhd - flex: none - width: 50% - &.is-one-third-fullhd - flex: none - width: 33.3333% - &.is-one-quarter-fullhd - flex: none - width: 25% - &.is-offset-three-quarters-fullhd - margin-left: 75% - &.is-offset-two-thirds-fullhd - margin-left: 66.6666% - &.is-offset-half-fullhd - margin-left: 50% - &.is-offset-one-third-fullhd - margin-left: 33.3333% - &.is-offset-one-quarter-fullhd - margin-left: 25% - @for $i from 1 through 12 - &.is-#{$i}-fullhd - flex: none - width: ($i / 12) * 100% - &.is-offset-#{$i}-fullhd - margin-left: ($i / 12) * 100% - -.columns - margin-left: -0.75rem - margin-right: -0.75rem - margin-top: -0.75rem - &:last-child - margin-bottom: -0.75rem - &:not(:last-child) - margin-bottom: 0.75rem - // Modifiers - &.is-centered - justify-content: center - &.is-gapless - margin-left: 0 - margin-right: 0 - margin-top: 0 - &:last-child - margin-bottom: 0 - &:not(:last-child) - margin-bottom: 1.5rem - & > .column - margin: 0 - padding: 0 - &.is-grid - // Responsiveness - +tablet - flex-wrap: wrap - & > .column - max-width: 33.3333% - padding: 0.75rem - width: 33.3333% - & + .column - margin-left: 0 - &.is-mobile - display: flex - &.is-multiline - flex-wrap: wrap - &.is-vcentered - align-items: center - // Responsiveness - +tablet - &:not(.is-desktop) - display: flex - +desktop - // Modifiers - &.is-desktop - display: flex diff --git a/site/_sass/vendor/bulma/sass/grid/tiles.sass b/site/_sass/vendor/bulma/sass/grid/tiles.sass deleted file mode 100755 index 1f6a626fc6b..00000000000 --- a/site/_sass/vendor/bulma/sass/grid/tiles.sass +++ /dev/null @@ -1,32 +0,0 @@ -.tile - align-items: stretch - display: block - flex-basis: 0 - flex-grow: 1 - flex-shrink: 1 - min-height: min-content - // Modifiers - &.is-ancestor - margin-left: -0.75rem - margin-right: -0.75rem - margin-top: -0.75rem - &:last-child - margin-bottom: -0.75rem - &:not(:last-child) - margin-bottom: 0.75rem - &.is-child - margin: 0 !important - &.is-parent - padding: 0.75rem - &.is-vertical - flex-direction: column - & > .tile.is-child:not(:last-child) - margin-bottom: 1.5rem !important - // Responsiveness - +tablet - &:not(.is-child) - display: flex - @for $i from 1 through 12 - &.is-#{$i} - flex: none - width: ($i / 12) * 100% diff --git a/site/_sass/vendor/bulma/sass/layout/_all.sass b/site/_sass/vendor/bulma/sass/layout/_all.sass deleted file mode 100755 index 143ada35be8..00000000000 --- a/site/_sass/vendor/bulma/sass/layout/_all.sass +++ /dev/null @@ -1,5 +0,0 @@ -@charset "utf-8" - -@import "hero.sass" -@import "section.sass" -@import "footer.sass" diff --git a/site/_sass/vendor/bulma/sass/layout/footer.sass b/site/_sass/vendor/bulma/sass/layout/footer.sass deleted file mode 100755 index 45259c87880..00000000000 --- a/site/_sass/vendor/bulma/sass/layout/footer.sass +++ /dev/null @@ -1,3 +0,0 @@ -.footer - background-color: $background - padding: 6rem 1.5rem diff --git a/site/_sass/vendor/bulma/sass/layout/hero.sass b/site/_sass/vendor/bulma/sass/layout/hero.sass deleted file mode 100755 index 9c694c6eca7..00000000000 --- a/site/_sass/vendor/bulma/sass/layout/hero.sass +++ /dev/null @@ -1,152 +0,0 @@ -// Components - -.hero-video - +overlay - overflow: hidden - video - left: 50% - min-height: 100% - min-width: 100% - position: absolute - top: 50% - transform: translate3d(-50%, -50%, 0) - // Modifiers - &.is-transparent - opacity: 0.3 - // Responsiveness - +mobile - display: none - -.hero-buttons - margin-top: 1.5rem - // Responsiveness - +mobile - .button - display: flex - &:not(:last-child) - margin-bottom: 0.75rem - +tablet - display: flex - justify-content: center - .button:not(:last-child) - margin-right: 1.5rem - -// Containers - -.hero-head, -.hero-foot - flex-grow: 0 - flex-shrink: 0 - -.hero-body - flex-grow: 1 - flex-shrink: 0 - padding: 3rem 1.5rem - -// Main container - -.hero - align-items: stretch - background-color: $white - display: flex - flex-direction: column - justify-content: space-between - .nav - background: none - box-shadow: 0 1px 0 rgba($border, 0.3) - .tabs - ul - border-bottom: none - // Colors - @each $name, $pair in $colors - $color: nth($pair, 1) - $color-invert: nth($pair, 2) - &.is-#{$name} - background-color: $color - color: $color-invert - a:not(.button), - strong - color: inherit - .title - color: $color-invert - .subtitle - color: rgba($color-invert, 0.9) - a:not(.button), - strong - color: $color-invert - .nav - box-shadow: 0 1px 0 rgba($color-invert, 0.2) - .nav-menu - +mobile - background-color: $color - a.nav-item, - .nav-item a:not(.button) - color: rgba($color-invert, 0.7) - &:hover, - &.is-active - color: $color-invert - .tabs - a - color: $color-invert - opacity: 0.9 - &:hover - opacity: 1 - li - &.is-active a - opacity: 1 - &.is-boxed, - &.is-toggle - a - color: $color-invert - &:hover - background-color: rgba($black, 0.1) - li.is-active a - &, - &:hover - background-color: $color-invert - border-color: $color-invert - color: $color - // Modifiers - &.is-bold - $gradient-top-left: darken(saturate(adjust-hue($color, -10deg), 10%), 10%) - $gradient-bottom-right: lighten(saturate(adjust-hue($color, 10deg), 5%), 5%) - background-image: linear-gradient(141deg, $gradient-top-left 0%, $color 71%, $gradient-bottom-right 100%) - +mobile - .nav-menu - background-image: linear-gradient(141deg, $gradient-top-left 0%, $color 71%, $gradient-bottom-right 100%) - // Responsiveness - +mobile - .nav-toggle - span - background-color: $color-invert - &:hover - background-color: rgba($black, 0.1) - &.is-active - span - background-color: $color-invert - .nav-menu - .nav-item - border-top-color: rgba($color-invert, 0.2) - // Sizes - &.is-medium - +tablet - .hero-body - padding-bottom: 9rem - padding-top: 9rem - &.is-large - +tablet - .hero-body - padding-bottom: 18rem - padding-top: 18rem - &.is-halfheight, - &.is-fullheight - .hero-body - align-items: center - display: flex - & > .container - flex-grow: 1 - flex-shrink: 1 - &.is-halfheight - min-height: 50vh - &.is-fullheight - min-height: 100vh diff --git a/site/_sass/vendor/bulma/sass/layout/section.sass b/site/_sass/vendor/bulma/sass/layout/section.sass deleted file mode 100755 index 7139cfb5fac..00000000000 --- a/site/_sass/vendor/bulma/sass/layout/section.sass +++ /dev/null @@ -1,10 +0,0 @@ -.section - background-color: $white - padding: 3rem 1.5rem - // Responsiveness - +desktop - // Sizes - &.is-medium - padding: 9rem 1.5rem - &.is-large - padding: 18rem 1.5rem diff --git a/site/_sass/vendor/bulma/sass/utilities/_all.sass b/site/_sass/vendor/bulma/sass/utilities/_all.sass deleted file mode 100755 index bf4ecfe3585..00000000000 --- a/site/_sass/vendor/bulma/sass/utilities/_all.sass +++ /dev/null @@ -1,8 +0,0 @@ -@charset "utf-8" - -@import "initial-variables.sass" -@import "functions.sass" -@import "derived-variables.sass" -@import "animations.sass" -@import "mixins.sass" -@import "controls.sass" diff --git a/site/_sass/vendor/bulma/sass/utilities/animations.sass b/site/_sass/vendor/bulma/sass/utilities/animations.sass deleted file mode 100755 index a14525d75ec..00000000000 --- a/site/_sass/vendor/bulma/sass/utilities/animations.sass +++ /dev/null @@ -1,5 +0,0 @@ -@keyframes spinAround - from - transform: rotate(0deg) - to - transform: rotate(359deg) diff --git a/site/_sass/vendor/bulma/sass/utilities/controls.sass b/site/_sass/vendor/bulma/sass/utilities/controls.sass deleted file mode 100755 index cb258df115b..00000000000 --- a/site/_sass/vendor/bulma/sass/utilities/controls.sass +++ /dev/null @@ -1,41 +0,0 @@ -$control-radius: $radius !default -$control-radius-small: $radius-small !default - -$control-padding-vertical: calc(0.375em - 1px) !default -$control-padding-horizontal: calc(0.625em - 1px) !default - -=control - -moz-appearance: none - -webkit-appearance: none - align-items: center - border: 1px solid transparent - border-radius: $control-radius - box-shadow: none - display: inline-flex - font-size: $size-normal - height: 2.25em - justify-content: flex-start - line-height: 1.5 - padding-bottom: $control-padding-vertical - padding-left: $control-padding-horizontal - padding-right: $control-padding-horizontal - padding-top: $control-padding-vertical - position: relative - vertical-align: top - // States - &:focus, - &.is-focused, - &:active, - &.is-active - outline: none - &[disabled] - cursor: not-allowed - -// The controls sizes use mixins so they can be used at different breakpoints -=control-small - border-radius: $control-radius-small - font-size: $size-small -=control-medium - font-size: $size-medium -=control-large - font-size: $size-large diff --git a/site/_sass/vendor/bulma/sass/utilities/derived-variables.sass b/site/_sass/vendor/bulma/sass/utilities/derived-variables.sass deleted file mode 100755 index 73e8a66bd9f..00000000000 --- a/site/_sass/vendor/bulma/sass/utilities/derived-variables.sass +++ /dev/null @@ -1,80 +0,0 @@ -$primary: $turquoise !default - -$info: $blue !default -$success: $green !default -$warning: $yellow !default -$danger: $red !default - -$light: $white-ter !default -$dark: $grey-darker !default - -// Invert colors - -$orange-invert: findColorInvert($orange) !default -$yellow-invert: findColorInvert($yellow) !default -$green-invert: findColorInvert($green) !default -$turquoise-invert: findColorInvert($turquoise) !default -$blue-invert: findColorInvert($blue) !default -$purple-invert: findColorInvert($purple) !default -$red-invert: findColorInvert($red) !default - -$primary-invert: $turquoise-invert !default -$info-invert: $blue-invert !default -$success-invert: $green-invert !default -$warning-invert: $yellow-invert !default -$danger-invert: $red-invert !default -$light-invert: $dark !default -$dark-invert: $light !default - -// General colors - -$background: $white-ter !default - -$border: $grey-lighter !default -$border-hover: $grey-light !default - -// Text colors - -$text: $grey-dark !default -$text-invert: findColorInvert($text) !default -$text-light: $grey !default -$text-strong: $grey-darker !default - -// Code colors - -$code: $red !default -$code-background: $background !default - -$pre: $text !default -$pre-background: $background !default - -// Link colors - -$link: $primary !default -$link-invert: $primary-invert !default -$link-visited: $purple !default - -$link-hover: $grey-darker !default -$link-hover-border: $grey-light !default - -$link-focus: $grey-darker !default -$link-focus-border: $primary !default - -$link-active: $grey-darker !default -$link-active-border: $grey-dark !default - -// Typography - -$family-primary: $family-sans-serif !default -$family-code: $family-monospace !default - -$size-small: $size-7 !default -$size-normal: $size-6 !default -$size-medium: $size-5 !default -$size-large: $size-4 !default - -// Lists and maps - -$colors: ("white": ($white, $black), "black": ($black, $white), "light": ($light, $light-invert), "dark": ($dark, $dark-invert), "primary": ($primary, $primary-invert), "info": ($info, $info-invert), "success": ($success, $success-invert), "warning": ($warning, $warning-invert), "danger": ($danger, $danger-invert)) !default - -$sizes: $size-1 $size-2 $size-3 $size-4 $size-5 $size-6 !default diff --git a/site/_sass/vendor/bulma/sass/utilities/functions.sass b/site/_sass/vendor/bulma/sass/utilities/functions.sass deleted file mode 100755 index e38d1727cd8..00000000000 --- a/site/_sass/vendor/bulma/sass/utilities/functions.sass +++ /dev/null @@ -1,28 +0,0 @@ -@function powerNumber($number, $exp) - $value: 1 - @if $exp > 0 - @for $i from 1 through $exp - $value: $value * $number - @else if $exp < 0 - @for $i from 1 through -$exp - $value: $value / $number - @return $value - -@function colorLuminance($color) - $color-rgb: ('red': red($color),'green': green($color),'blue': blue($color)) - @each $name, $value in $color-rgb - $adjusted: 0 - $value: $value / 255 - @if $value < 0.03928 - $value: $value / 12.92 - @else - $value: ($value + .055) / 1.055 - $value: powerNumber($value, 2) - $color-rgb: map-merge($color-rgb, ($name: $value)) - @return (map-get($color-rgb, 'red') * .2126) + (map-get($color-rgb, 'green') * .7152) + (map-get($color-rgb, 'blue') * .0722) - -@function findColorInvert($color) - @if (colorLuminance($color) > 0.55) - @return rgba(#000, 0.7) - @else - @return #fff diff --git a/site/_sass/vendor/bulma/sass/utilities/initial-variables.sass b/site/_sass/vendor/bulma/sass/utilities/initial-variables.sass deleted file mode 100755 index 9777d58b0d5..00000000000 --- a/site/_sass/vendor/bulma/sass/utilities/initial-variables.sass +++ /dev/null @@ -1,68 +0,0 @@ -// Colors - -$black: hsl(0, 0%, 4%) !default -$black-bis: hsl(0, 0%, 7%) !default -$black-ter: hsl(0, 0%, 14%) !default - -$grey-darker: hsl(0, 0%, 21%) !default -$grey-dark: hsl(0, 0%, 29%) !default -$grey: hsl(0, 0%, 48%) !default -$grey-light: hsl(0, 0%, 71%) !default -$grey-lighter: hsl(0, 0%, 86%) !default - -$white-ter: hsl(0, 0%, 96%) !default -$white-bis: hsl(0, 0%, 98%) !default -$white: hsl(0, 0%, 100%) !default - -$orange: hsl(14, 100%, 53%) !default -$yellow: hsl(48, 100%, 67%) !default -$green: hsl(141, 71%, 48%) !default -$turquoise: hsl(171, 100%, 41%) !default -$blue: hsl(217, 71%, 53%) !default -$purple: hsl(271, 100%, 71%) !default -$red: hsl(348, 100%, 61%) !default - -// Typography - -$family-sans-serif: BlinkMacSystemFont, -apple-system, "Segoe UI", "Roboto", "Oxygen", "Ubuntu", "Cantarell", "Fira Sans", "Droid Sans", "Helvetica Neue", "Helvetica", "Arial", sans-serif !default -$family-monospace: monospace !default -$render-mode: optimizeLegibility !default - -$size-1: 3rem !default -$size-2: 2.5rem !default -$size-3: 2rem !default -$size-4: 1.5rem !default -$size-5: 1.25rem !default -$size-6: 1rem !default -$size-7: 0.75rem !default - -$weight-light: 300 !default -$weight-normal: 400 !default -$weight-semibold: 500 !default -$weight-bold: 700 !default - -// Body - -$body-background: #fff !default -$body-size: 16px !default - -// Responsiveness - -// The container gap, which acts as the offset for breakpoints -$gap: 24px !default -// 960, 1152, and 1344 have been chosen because they are divisible by both 12 and 16 -$tablet: 769px !default -// 960px container + 3rem -$desktop: 960px + (2 * $gap) !default -// 1152px container + 3rem -$widescreen: 1152px + (2 * $gap) !default -// 1344px container + 3rem -$fullhd: 1344px + (2 * $gap) !default - -// Miscellaneous - -$easing: ease-out !default -$radius-small: 2px !default -$radius: 3px !default -$radius-large: 5px !default -$speed: 86ms !default diff --git a/site/_sass/vendor/bulma/sass/utilities/mixins.sass b/site/_sass/vendor/bulma/sass/utilities/mixins.sass deleted file mode 100755 index 5997c1a660b..00000000000 --- a/site/_sass/vendor/bulma/sass/utilities/mixins.sass +++ /dev/null @@ -1,225 +0,0 @@ -=arrow($color) - border: 1px solid $color - border-right: 0 - border-top: 0 - content: " " - display: block - height: 0.5em - pointer-events: none - position: absolute - transform: rotate(-45deg) - width: 0.5em - -=block - &:not(:last-child) - margin-bottom: 1.5rem - -=clearfix - &:after - clear: both - content: " " - display: table - -=center($width, $height: 0) - position: absolute - @if $height != 0 - left: calc(50% - (#{$width} / 2)) - top: calc(50% - (#{$height} / 2)) - @else - left: calc(50% - (#{$width} / 2)) - top: calc(50% - (#{$width} / 2)) - -=delete - +unselectable - -moz-appearance: none - -webkit-appearance: none - background-color: rgba($black, 0.2) - border: none - border-radius: 290486px - cursor: pointer - display: inline-block - flex-grow: 0 - flex-shrink: 0 - font-size: $size-normal - height: 20px - max-height: 20px - max-width: 20px - min-height: 20px - min-width: 20px - outline: none - position: relative - vertical-align: top - width: 20px - &:before, - &:after - background-color: $white - content: "" - display: block - left: 50% - position: absolute - top: 50% - transform: translateX(-50%) translateY(-50%) rotate(45deg) - transform-origin: center center - &:before - height: 2px - width: 50% - &:after - height: 50% - width: 2px - &:hover, - &:focus - background-color: rgba($black, 0.3) - &:active - background-color: rgba($black, 0.4) - // Sizes - &.is-small - height: 16px - max-height: 16px - max-width: 16px - min-height: 16px - min-width: 16px - width: 16px - &.is-medium - height: 24px - max-height: 24px - max-width: 24px - min-height: 24px - min-width: 24px - width: 24px - &.is-large - height: 32px - max-height: 32px - max-width: 32px - min-height: 32px - min-width: 32px - width: 32px - -=fa($size, $dimensions) - display: inline-block - font-size: $size - height: $dimensions - line-height: $dimensions - text-align: center - vertical-align: top - width: $dimensions - -=hamburger($dimensions) - cursor: pointer - display: block - height: $dimensions - position: relative - width: $dimensions - span - background-color: $text - display: block - height: 1px - left: 50% - margin-left: -7px - position: absolute - top: 50% - transition: none $speed $easing - transition-property: background, left, opacity, transform - width: 15px - &:nth-child(1) - margin-top: -6px - &:nth-child(2) - margin-top: -1px - &:nth-child(3) - margin-top: 4px - &:hover - background-color: $background - // Modifers - &.is-active - span - background-color: $link - &:nth-child(1) - margin-left: -5px - transform: rotate(45deg) - transform-origin: left top - &:nth-child(2) - opacity: 0 - &:nth-child(3) - margin-left: -5px - transform: rotate(-45deg) - transform-origin: left bottom - -=loader - animation: spinAround 500ms infinite linear - border: 2px solid $border - border-radius: 290486px - border-right-color: transparent - border-top-color: transparent - content: "" - display: block - height: 1em - position: relative - width: 1em - -=overflow-touch - -webkit-overflow-scrolling: touch - -=overlay($offset: 0) - bottom: $offset - left: $offset - position: absolute - right: $offset - top: $offset - -=placeholder - $placeholders: ':-moz' ':-webkit-input' '-moz' '-ms-input' - @each $placeholder in $placeholders - &:#{$placeholder}-placeholder - @content - -=unselectable - -webkit-touch-callout: none - -webkit-user-select: none - -moz-user-select: none - -ms-user-select: none - user-select: none - -// Responsiveness - -=from($device) - @media screen and (min-width: $device) - @content - -=until($device) - @media screen and (max-width: $device - 1px) - @content - -=mobile - @media screen and (max-width: $tablet - 1px) - @content - -=tablet - @media screen and (min-width: $tablet), print - @content - -=tablet-only - @media screen and (min-width: $tablet) and (max-width: $desktop - 1px) - @content - -=touch - @media screen and (max-width: $desktop - 1px) - @content - -=desktop - @media screen and (min-width: $desktop) - @content - -=desktop-only - @media screen and (min-width: $desktop) and (max-width: $widescreen - 1px) - @content - -=widescreen - @media screen and (min-width: $widescreen) - @content - -=widescreen-only - @media screen and (min-width: $widescreen) and (max-width: $fullhd - 1px) - @content - -=fullhd - @media screen and (min-width: $fullhd) - @content diff --git a/site/_sass/vendor/bulma/sass/utilities/variables.sass b/site/_sass/vendor/bulma/sass/utilities/variables.sass deleted file mode 100755 index 0a9b9cb86df..00000000000 --- a/site/_sass/vendor/bulma/sass/utilities/variables.sass +++ /dev/null @@ -1,150 +0,0 @@ -//////////////////////////////////////////////// -//////////////////////////////////////////////// -// 1. Initial variables - -// Colors -$black: hsl(0, 0%, 4%) !default -$black-bis: hsl(0, 0%, 7%) !default -$black-ter: hsl(0, 0%, 14%) !default - -$grey-darker: hsl(0, 0%, 21%) !default -$grey-dark: hsl(0, 0%, 29%) !default -$grey: hsl(0, 0%, 48%) !default -$grey-light: hsl(0, 0%, 71%) !default -$grey-lighter: hsl(0, 0%, 86%) !default - -$white-ter: hsl(0, 0%, 96%) !default -$white-bis: hsl(0, 0%, 98%) !default -$white: hsl(0, 0%, 100%) !default - -$orange: hsl(14, 100%, 53%) !default -$yellow: hsl(48, 100%, 67%) !default -$green: hsl(141, 71%, 48%) !default -$turquoise: hsl(171, 100%, 41%) !default -$blue: hsl(217, 71%, 53%) !default -$purple: hsl(271, 100%, 71%) !default -$red: hsl(348, 100%, 61%) !default - -// Typography -$family-sans-serif: BlinkMacSystemFont, -apple-system, "Segoe UI", "Roboto", "Oxygen", "Ubuntu", "Cantarell", "Fira Sans", "Droid Sans", "Helvetica Neue", "Helvetica", "Arial", sans-serif !default -$family-monospace: monospace !default -$render-mode: optimizeLegibility !default - -$size-1: 3rem !default -$size-2: 2.5rem !default -$size-3: 2rem !default -$size-4: 1.5rem !default -$size-5: 1.25rem !default -$size-6: 1rem !default -$size-7: 0.75rem !default - -$weight-light: 300 !default -$weight-normal: 400 !default -$weight-semibold: 500 !default -$weight-bold: 700 !default - -// Body -$body-background: #fff !default -$body-size: 16px !default - -// Responsiveness -// 960, 1152, and 1344 have been chosen because they are divisible by both 12 and 16 -$tablet: 769px !default -// 960px container + 40px -$desktop: 1000px !default -// 1152px container + 40 -$widescreen: 1192px !default -// 1344px container + 40 -$fullhd: 1384px !default - -// Miscellaneous -$easing: ease-out !default -$radius-small: 2px !default -$radius: 3px !default -$radius-large: 5px !default -$speed: 86ms !default - -//////////////////////////////////////////////// -//////////////////////////////////////////////// -// 2. Primary colors - -$primary: $turquoise !default - -$info: $blue !default -$success: $green !default -$warning: $yellow !default -$danger: $red !default - -$light: $white-ter !default -$dark: $grey-darker !default - -//////////////////////////////////////////////// -//////////////////////////////////////////////// -// 3. Applied variables - -// Invert colors -$orange-invert: findColorInvert($orange) !default -$yellow-invert: findColorInvert($yellow) !default -$green-invert: findColorInvert($green) !default -$turquoise-invert: findColorInvert($turquoise) !default -$blue-invert: findColorInvert($blue) !default -$purple-invert: findColorInvert($purple) !default -$red-invert: findColorInvert($red) !default - -$primary-invert: $turquoise-invert !default -$info-invert: $blue-invert !default -$success-invert: $green-invert !default -$warning-invert: $yellow-invert !default -$danger-invert: $red-invert !default -$light-invert: $dark !default -$dark-invert: $light !default - -// General colors -$background: $white-ter !default - -$border: $grey-lighter !default -$border-hover: $grey-light !default - -// Text colors -$text: $grey-dark !default -$text-invert: findColorInvert($text) !default -$text-light: $grey !default -$text-strong: $grey-darker !default - -// Code colors -$code: $red !default -$code-background: $background !default - -$pre: $text !default -$pre-background: $background !default - -// Link colors -$link: $primary !default -$link-invert: $primary-invert !default -$link-visited: $purple !default - -$link-hover: $grey-darker !default -$link-hover-border: $grey-light !default - -$link-focus: $grey-darker !default -$link-focus-border: $primary !default - -$link-active: $grey-darker !default -$link-active-border: $grey-dark !default - -// Typography -$family-primary: $family-sans-serif !default -$family-code: $family-monospace !default - -$size-small: $size-7 !default -$size-normal: $size-6 !default -$size-medium: $size-5 !default -$size-large: $size-4 !default - -//////////////////////////////////////////////// -//////////////////////////////////////////////// -// 4. Lists and maps - -$colors: ("white": ($white, $black), "black": ($black, $white), "light": ($light, $light-invert), "dark": ($dark, $dark-invert), "primary": ($primary, $primary-invert), "info": ($info, $info-invert), "success": ($success, $success-invert), "warning": ($warning, $warning-invert), "danger": ($danger, $danger-invert)) !default - -$sizes: $size-1 $size-2 $size-3 $size-4 $size-5 $size-6 !default diff --git a/site/_sass/vendor/font-awesome/_animated.scss b/site/_sass/vendor/font-awesome/_animated.scss deleted file mode 100644 index 8a020dbfff7..00000000000 --- a/site/_sass/vendor/font-awesome/_animated.scss +++ /dev/null @@ -1,34 +0,0 @@ -// Spinning Icons -// -------------------------- - -.#{$fa-css-prefix}-spin { - -webkit-animation: fa-spin 2s infinite linear; - animation: fa-spin 2s infinite linear; -} - -.#{$fa-css-prefix}-pulse { - -webkit-animation: fa-spin 1s infinite steps(8); - animation: fa-spin 1s infinite steps(8); -} - -@-webkit-keyframes fa-spin { - 0% { - -webkit-transform: rotate(0deg); - transform: rotate(0deg); - } - 100% { - -webkit-transform: rotate(359deg); - transform: rotate(359deg); - } -} - -@keyframes fa-spin { - 0% { - -webkit-transform: rotate(0deg); - transform: rotate(0deg); - } - 100% { - -webkit-transform: rotate(359deg); - transform: rotate(359deg); - } -} diff --git a/site/_sass/vendor/font-awesome/_bordered-pulled.scss b/site/_sass/vendor/font-awesome/_bordered-pulled.scss deleted file mode 100644 index d4b85a02f24..00000000000 --- a/site/_sass/vendor/font-awesome/_bordered-pulled.scss +++ /dev/null @@ -1,25 +0,0 @@ -// Bordered & Pulled -// ------------------------- - -.#{$fa-css-prefix}-border { - padding: .2em .25em .15em; - border: solid .08em $fa-border-color; - border-radius: .1em; -} - -.#{$fa-css-prefix}-pull-left { float: left; } -.#{$fa-css-prefix}-pull-right { float: right; } - -.#{$fa-css-prefix} { - &.#{$fa-css-prefix}-pull-left { margin-right: .3em; } - &.#{$fa-css-prefix}-pull-right { margin-left: .3em; } -} - -/* Deprecated as of 4.4.0 */ -.pull-right { float: right; } -.pull-left { float: left; } - -.#{$fa-css-prefix} { - &.pull-left { margin-right: .3em; } - &.pull-right { margin-left: .3em; } -} diff --git a/site/_sass/vendor/font-awesome/_core.scss b/site/_sass/vendor/font-awesome/_core.scss deleted file mode 100644 index 7425ef85fc8..00000000000 --- a/site/_sass/vendor/font-awesome/_core.scss +++ /dev/null @@ -1,12 +0,0 @@ -// Base Class Definition -// ------------------------- - -.#{$fa-css-prefix} { - display: inline-block; - font: normal normal normal #{$fa-font-size-base}/#{$fa-line-height-base} FontAwesome; // shortening font declaration - font-size: inherit; // can't have font-size inherit on line above, so need to override - text-rendering: auto; // optimizelegibility throws things off #1094 - -webkit-font-smoothing: antialiased; - -moz-osx-font-smoothing: grayscale; - -} diff --git a/site/_sass/vendor/font-awesome/_fixed-width.scss b/site/_sass/vendor/font-awesome/_fixed-width.scss deleted file mode 100644 index b221c98133a..00000000000 --- a/site/_sass/vendor/font-awesome/_fixed-width.scss +++ /dev/null @@ -1,6 +0,0 @@ -// Fixed Width Icons -// ------------------------- -.#{$fa-css-prefix}-fw { - width: (18em / 14); - text-align: center; -} diff --git a/site/_sass/vendor/font-awesome/_icons.scss b/site/_sass/vendor/font-awesome/_icons.scss deleted file mode 100644 index e63e702c4d9..00000000000 --- a/site/_sass/vendor/font-awesome/_icons.scss +++ /dev/null @@ -1,789 +0,0 @@ -/* Font Awesome uses the Unicode Private Use Area (PUA) to ensure screen - readers do not read off random characters that represent icons */ - -.#{$fa-css-prefix}-glass:before { content: $fa-var-glass; } -.#{$fa-css-prefix}-music:before { content: $fa-var-music; } -.#{$fa-css-prefix}-search:before { content: $fa-var-search; } -.#{$fa-css-prefix}-envelope-o:before { content: $fa-var-envelope-o; } -.#{$fa-css-prefix}-heart:before { content: $fa-var-heart; } -.#{$fa-css-prefix}-star:before { content: $fa-var-star; } -.#{$fa-css-prefix}-star-o:before { content: $fa-var-star-o; } -.#{$fa-css-prefix}-user:before { content: $fa-var-user; } -.#{$fa-css-prefix}-film:before { content: $fa-var-film; } -.#{$fa-css-prefix}-th-large:before { content: $fa-var-th-large; } -.#{$fa-css-prefix}-th:before { content: $fa-var-th; } -.#{$fa-css-prefix}-th-list:before { content: $fa-var-th-list; } -.#{$fa-css-prefix}-check:before { content: $fa-var-check; } -.#{$fa-css-prefix}-remove:before, -.#{$fa-css-prefix}-close:before, -.#{$fa-css-prefix}-times:before { content: $fa-var-times; } -.#{$fa-css-prefix}-search-plus:before { content: $fa-var-search-plus; } -.#{$fa-css-prefix}-search-minus:before { content: $fa-var-search-minus; } -.#{$fa-css-prefix}-power-off:before { content: $fa-var-power-off; } -.#{$fa-css-prefix}-signal:before { content: $fa-var-signal; } -.#{$fa-css-prefix}-gear:before, -.#{$fa-css-prefix}-cog:before { content: $fa-var-cog; } -.#{$fa-css-prefix}-trash-o:before { content: $fa-var-trash-o; } -.#{$fa-css-prefix}-home:before { content: $fa-var-home; } -.#{$fa-css-prefix}-file-o:before { content: $fa-var-file-o; } -.#{$fa-css-prefix}-clock-o:before { content: $fa-var-clock-o; } -.#{$fa-css-prefix}-road:before { content: $fa-var-road; } -.#{$fa-css-prefix}-download:before { content: $fa-var-download; } -.#{$fa-css-prefix}-arrow-circle-o-down:before { content: $fa-var-arrow-circle-o-down; } -.#{$fa-css-prefix}-arrow-circle-o-up:before { content: $fa-var-arrow-circle-o-up; } -.#{$fa-css-prefix}-inbox:before { content: $fa-var-inbox; } -.#{$fa-css-prefix}-play-circle-o:before { content: $fa-var-play-circle-o; } -.#{$fa-css-prefix}-rotate-right:before, -.#{$fa-css-prefix}-repeat:before { content: $fa-var-repeat; } -.#{$fa-css-prefix}-refresh:before { content: $fa-var-refresh; } -.#{$fa-css-prefix}-list-alt:before { content: $fa-var-list-alt; } -.#{$fa-css-prefix}-lock:before { content: $fa-var-lock; } -.#{$fa-css-prefix}-flag:before { content: $fa-var-flag; } -.#{$fa-css-prefix}-headphones:before { content: $fa-var-headphones; } -.#{$fa-css-prefix}-volume-off:before { content: $fa-var-volume-off; } -.#{$fa-css-prefix}-volume-down:before { content: $fa-var-volume-down; } -.#{$fa-css-prefix}-volume-up:before { content: $fa-var-volume-up; } -.#{$fa-css-prefix}-qrcode:before { content: $fa-var-qrcode; } -.#{$fa-css-prefix}-barcode:before { content: $fa-var-barcode; } -.#{$fa-css-prefix}-tag:before { content: $fa-var-tag; } -.#{$fa-css-prefix}-tags:before { content: $fa-var-tags; } -.#{$fa-css-prefix}-book:before { content: $fa-var-book; } -.#{$fa-css-prefix}-bookmark:before { content: $fa-var-bookmark; } -.#{$fa-css-prefix}-print:before { content: $fa-var-print; } -.#{$fa-css-prefix}-camera:before { content: $fa-var-camera; } -.#{$fa-css-prefix}-font:before { content: $fa-var-font; } -.#{$fa-css-prefix}-bold:before { content: $fa-var-bold; } -.#{$fa-css-prefix}-italic:before { content: $fa-var-italic; } -.#{$fa-css-prefix}-text-height:before { content: $fa-var-text-height; } -.#{$fa-css-prefix}-text-width:before { content: $fa-var-text-width; } -.#{$fa-css-prefix}-align-left:before { content: $fa-var-align-left; } -.#{$fa-css-prefix}-align-center:before { content: $fa-var-align-center; } -.#{$fa-css-prefix}-align-right:before { content: $fa-var-align-right; } -.#{$fa-css-prefix}-align-justify:before { content: $fa-var-align-justify; } -.#{$fa-css-prefix}-list:before { content: $fa-var-list; } -.#{$fa-css-prefix}-dedent:before, -.#{$fa-css-prefix}-outdent:before { content: $fa-var-outdent; } -.#{$fa-css-prefix}-indent:before { content: $fa-var-indent; } -.#{$fa-css-prefix}-video-camera:before { content: $fa-var-video-camera; } -.#{$fa-css-prefix}-photo:before, -.#{$fa-css-prefix}-image:before, -.#{$fa-css-prefix}-picture-o:before { content: $fa-var-picture-o; } -.#{$fa-css-prefix}-pencil:before { content: $fa-var-pencil; } -.#{$fa-css-prefix}-map-marker:before { content: $fa-var-map-marker; } -.#{$fa-css-prefix}-adjust:before { content: $fa-var-adjust; } -.#{$fa-css-prefix}-tint:before { content: $fa-var-tint; } -.#{$fa-css-prefix}-edit:before, -.#{$fa-css-prefix}-pencil-square-o:before { content: $fa-var-pencil-square-o; } -.#{$fa-css-prefix}-share-square-o:before { content: $fa-var-share-square-o; } -.#{$fa-css-prefix}-check-square-o:before { content: $fa-var-check-square-o; } -.#{$fa-css-prefix}-arrows:before { content: $fa-var-arrows; } -.#{$fa-css-prefix}-step-backward:before { content: $fa-var-step-backward; } -.#{$fa-css-prefix}-fast-backward:before { content: $fa-var-fast-backward; } -.#{$fa-css-prefix}-backward:before { content: $fa-var-backward; } -.#{$fa-css-prefix}-play:before { content: $fa-var-play; } -.#{$fa-css-prefix}-pause:before { content: $fa-var-pause; } -.#{$fa-css-prefix}-stop:before { content: $fa-var-stop; } -.#{$fa-css-prefix}-forward:before { content: $fa-var-forward; } -.#{$fa-css-prefix}-fast-forward:before { content: $fa-var-fast-forward; } -.#{$fa-css-prefix}-step-forward:before { content: $fa-var-step-forward; } -.#{$fa-css-prefix}-eject:before { content: $fa-var-eject; } -.#{$fa-css-prefix}-chevron-left:before { content: $fa-var-chevron-left; } -.#{$fa-css-prefix}-chevron-right:before { content: $fa-var-chevron-right; } -.#{$fa-css-prefix}-plus-circle:before { content: $fa-var-plus-circle; } -.#{$fa-css-prefix}-minus-circle:before { content: $fa-var-minus-circle; } -.#{$fa-css-prefix}-times-circle:before { content: $fa-var-times-circle; } -.#{$fa-css-prefix}-check-circle:before { content: $fa-var-check-circle; } -.#{$fa-css-prefix}-question-circle:before { content: $fa-var-question-circle; } -.#{$fa-css-prefix}-info-circle:before { content: $fa-var-info-circle; } -.#{$fa-css-prefix}-crosshairs:before { content: $fa-var-crosshairs; } -.#{$fa-css-prefix}-times-circle-o:before { content: $fa-var-times-circle-o; } -.#{$fa-css-prefix}-check-circle-o:before { content: $fa-var-check-circle-o; } -.#{$fa-css-prefix}-ban:before { content: $fa-var-ban; } -.#{$fa-css-prefix}-arrow-left:before { content: $fa-var-arrow-left; } -.#{$fa-css-prefix}-arrow-right:before { content: $fa-var-arrow-right; } -.#{$fa-css-prefix}-arrow-up:before { content: $fa-var-arrow-up; } -.#{$fa-css-prefix}-arrow-down:before { content: $fa-var-arrow-down; } -.#{$fa-css-prefix}-mail-forward:before, -.#{$fa-css-prefix}-share:before { content: $fa-var-share; } -.#{$fa-css-prefix}-expand:before { content: $fa-var-expand; } -.#{$fa-css-prefix}-compress:before { content: $fa-var-compress; } -.#{$fa-css-prefix}-plus:before { content: $fa-var-plus; } -.#{$fa-css-prefix}-minus:before { content: $fa-var-minus; } -.#{$fa-css-prefix}-asterisk:before { content: $fa-var-asterisk; } -.#{$fa-css-prefix}-exclamation-circle:before { content: $fa-var-exclamation-circle; } -.#{$fa-css-prefix}-gift:before { content: $fa-var-gift; } -.#{$fa-css-prefix}-leaf:before { content: $fa-var-leaf; } -.#{$fa-css-prefix}-fire:before { content: $fa-var-fire; } -.#{$fa-css-prefix}-eye:before { content: $fa-var-eye; } -.#{$fa-css-prefix}-eye-slash:before { content: $fa-var-eye-slash; } -.#{$fa-css-prefix}-warning:before, -.#{$fa-css-prefix}-exclamation-triangle:before { content: $fa-var-exclamation-triangle; } -.#{$fa-css-prefix}-plane:before { content: $fa-var-plane; } -.#{$fa-css-prefix}-calendar:before { content: $fa-var-calendar; } -.#{$fa-css-prefix}-random:before { content: $fa-var-random; } -.#{$fa-css-prefix}-comment:before { content: $fa-var-comment; } -.#{$fa-css-prefix}-magnet:before { content: $fa-var-magnet; } -.#{$fa-css-prefix}-chevron-up:before { content: $fa-var-chevron-up; } -.#{$fa-css-prefix}-chevron-down:before { content: $fa-var-chevron-down; } -.#{$fa-css-prefix}-retweet:before { content: $fa-var-retweet; } -.#{$fa-css-prefix}-shopping-cart:before { content: $fa-var-shopping-cart; } -.#{$fa-css-prefix}-folder:before { content: $fa-var-folder; } -.#{$fa-css-prefix}-folder-open:before { content: $fa-var-folder-open; } -.#{$fa-css-prefix}-arrows-v:before { content: $fa-var-arrows-v; } -.#{$fa-css-prefix}-arrows-h:before { content: $fa-var-arrows-h; } -.#{$fa-css-prefix}-bar-chart-o:before, -.#{$fa-css-prefix}-bar-chart:before { content: $fa-var-bar-chart; } -.#{$fa-css-prefix}-twitter-square:before { content: $fa-var-twitter-square; } -.#{$fa-css-prefix}-facebook-square:before { content: $fa-var-facebook-square; } -.#{$fa-css-prefix}-camera-retro:before { content: $fa-var-camera-retro; } -.#{$fa-css-prefix}-key:before { content: $fa-var-key; } -.#{$fa-css-prefix}-gears:before, -.#{$fa-css-prefix}-cogs:before { content: $fa-var-cogs; } -.#{$fa-css-prefix}-comments:before { content: $fa-var-comments; } -.#{$fa-css-prefix}-thumbs-o-up:before { content: $fa-var-thumbs-o-up; } -.#{$fa-css-prefix}-thumbs-o-down:before { content: $fa-var-thumbs-o-down; } -.#{$fa-css-prefix}-star-half:before { content: $fa-var-star-half; } -.#{$fa-css-prefix}-heart-o:before { content: $fa-var-heart-o; } -.#{$fa-css-prefix}-sign-out:before { content: $fa-var-sign-out; } -.#{$fa-css-prefix}-linkedin-square:before { content: $fa-var-linkedin-square; } -.#{$fa-css-prefix}-thumb-tack:before { content: $fa-var-thumb-tack; } -.#{$fa-css-prefix}-external-link:before { content: $fa-var-external-link; } -.#{$fa-css-prefix}-sign-in:before { content: $fa-var-sign-in; } -.#{$fa-css-prefix}-trophy:before { content: $fa-var-trophy; } -.#{$fa-css-prefix}-github-square:before { content: $fa-var-github-square; } -.#{$fa-css-prefix}-upload:before { content: $fa-var-upload; } -.#{$fa-css-prefix}-lemon-o:before { content: $fa-var-lemon-o; } -.#{$fa-css-prefix}-phone:before { content: $fa-var-phone; } -.#{$fa-css-prefix}-square-o:before { content: $fa-var-square-o; } -.#{$fa-css-prefix}-bookmark-o:before { content: $fa-var-bookmark-o; } -.#{$fa-css-prefix}-phone-square:before { content: $fa-var-phone-square; } -.#{$fa-css-prefix}-twitter:before { content: $fa-var-twitter; } -.#{$fa-css-prefix}-facebook-f:before, -.#{$fa-css-prefix}-facebook:before { content: $fa-var-facebook; } -.#{$fa-css-prefix}-github:before { content: $fa-var-github; } -.#{$fa-css-prefix}-unlock:before { content: $fa-var-unlock; } -.#{$fa-css-prefix}-credit-card:before { content: $fa-var-credit-card; } -.#{$fa-css-prefix}-feed:before, -.#{$fa-css-prefix}-rss:before { content: $fa-var-rss; } -.#{$fa-css-prefix}-hdd-o:before { content: $fa-var-hdd-o; } -.#{$fa-css-prefix}-bullhorn:before { content: $fa-var-bullhorn; } -.#{$fa-css-prefix}-bell:before { content: $fa-var-bell; } -.#{$fa-css-prefix}-certificate:before { content: $fa-var-certificate; } -.#{$fa-css-prefix}-hand-o-right:before { content: $fa-var-hand-o-right; } -.#{$fa-css-prefix}-hand-o-left:before { content: $fa-var-hand-o-left; } -.#{$fa-css-prefix}-hand-o-up:before { content: $fa-var-hand-o-up; } -.#{$fa-css-prefix}-hand-o-down:before { content: $fa-var-hand-o-down; } -.#{$fa-css-prefix}-arrow-circle-left:before { content: $fa-var-arrow-circle-left; } -.#{$fa-css-prefix}-arrow-circle-right:before { content: $fa-var-arrow-circle-right; } -.#{$fa-css-prefix}-arrow-circle-up:before { content: $fa-var-arrow-circle-up; } -.#{$fa-css-prefix}-arrow-circle-down:before { content: $fa-var-arrow-circle-down; } -.#{$fa-css-prefix}-globe:before { content: $fa-var-globe; } -.#{$fa-css-prefix}-wrench:before { content: $fa-var-wrench; } -.#{$fa-css-prefix}-tasks:before { content: $fa-var-tasks; } -.#{$fa-css-prefix}-filter:before { content: $fa-var-filter; } -.#{$fa-css-prefix}-briefcase:before { content: $fa-var-briefcase; } -.#{$fa-css-prefix}-arrows-alt:before { content: $fa-var-arrows-alt; } -.#{$fa-css-prefix}-group:before, -.#{$fa-css-prefix}-users:before { content: $fa-var-users; } -.#{$fa-css-prefix}-chain:before, -.#{$fa-css-prefix}-link:before { content: $fa-var-link; } -.#{$fa-css-prefix}-cloud:before { content: $fa-var-cloud; } -.#{$fa-css-prefix}-flask:before { content: $fa-var-flask; } -.#{$fa-css-prefix}-cut:before, -.#{$fa-css-prefix}-scissors:before { content: $fa-var-scissors; } -.#{$fa-css-prefix}-copy:before, -.#{$fa-css-prefix}-files-o:before { content: $fa-var-files-o; } -.#{$fa-css-prefix}-paperclip:before { content: $fa-var-paperclip; } -.#{$fa-css-prefix}-save:before, -.#{$fa-css-prefix}-floppy-o:before { content: $fa-var-floppy-o; } -.#{$fa-css-prefix}-square:before { content: $fa-var-square; } -.#{$fa-css-prefix}-navicon:before, -.#{$fa-css-prefix}-reorder:before, -.#{$fa-css-prefix}-bars:before { content: $fa-var-bars; } -.#{$fa-css-prefix}-list-ul:before { content: $fa-var-list-ul; } -.#{$fa-css-prefix}-list-ol:before { content: $fa-var-list-ol; } -.#{$fa-css-prefix}-strikethrough:before { content: $fa-var-strikethrough; } -.#{$fa-css-prefix}-underline:before { content: $fa-var-underline; } -.#{$fa-css-prefix}-table:before { content: $fa-var-table; } -.#{$fa-css-prefix}-magic:before { content: $fa-var-magic; } -.#{$fa-css-prefix}-truck:before { content: $fa-var-truck; } -.#{$fa-css-prefix}-pinterest:before { content: $fa-var-pinterest; } -.#{$fa-css-prefix}-pinterest-square:before { content: $fa-var-pinterest-square; } -.#{$fa-css-prefix}-google-plus-square:before { content: $fa-var-google-plus-square; } -.#{$fa-css-prefix}-google-plus:before { content: $fa-var-google-plus; } -.#{$fa-css-prefix}-money:before { content: $fa-var-money; } -.#{$fa-css-prefix}-caret-down:before { content: $fa-var-caret-down; } -.#{$fa-css-prefix}-caret-up:before { content: $fa-var-caret-up; } -.#{$fa-css-prefix}-caret-left:before { content: $fa-var-caret-left; } -.#{$fa-css-prefix}-caret-right:before { content: $fa-var-caret-right; } -.#{$fa-css-prefix}-columns:before { content: $fa-var-columns; } -.#{$fa-css-prefix}-unsorted:before, -.#{$fa-css-prefix}-sort:before { content: $fa-var-sort; } -.#{$fa-css-prefix}-sort-down:before, -.#{$fa-css-prefix}-sort-desc:before { content: $fa-var-sort-desc; } -.#{$fa-css-prefix}-sort-up:before, -.#{$fa-css-prefix}-sort-asc:before { content: $fa-var-sort-asc; } -.#{$fa-css-prefix}-envelope:before { content: $fa-var-envelope; } -.#{$fa-css-prefix}-linkedin:before { content: $fa-var-linkedin; } -.#{$fa-css-prefix}-rotate-left:before, -.#{$fa-css-prefix}-undo:before { content: $fa-var-undo; } -.#{$fa-css-prefix}-legal:before, -.#{$fa-css-prefix}-gavel:before { content: $fa-var-gavel; } -.#{$fa-css-prefix}-dashboard:before, -.#{$fa-css-prefix}-tachometer:before { content: $fa-var-tachometer; } -.#{$fa-css-prefix}-comment-o:before { content: $fa-var-comment-o; } -.#{$fa-css-prefix}-comments-o:before { content: $fa-var-comments-o; } -.#{$fa-css-prefix}-flash:before, -.#{$fa-css-prefix}-bolt:before { content: $fa-var-bolt; } -.#{$fa-css-prefix}-sitemap:before { content: $fa-var-sitemap; } -.#{$fa-css-prefix}-umbrella:before { content: $fa-var-umbrella; } -.#{$fa-css-prefix}-paste:before, -.#{$fa-css-prefix}-clipboard:before { content: $fa-var-clipboard; } -.#{$fa-css-prefix}-lightbulb-o:before { content: $fa-var-lightbulb-o; } -.#{$fa-css-prefix}-exchange:before { content: $fa-var-exchange; } -.#{$fa-css-prefix}-cloud-download:before { content: $fa-var-cloud-download; } -.#{$fa-css-prefix}-cloud-upload:before { content: $fa-var-cloud-upload; } -.#{$fa-css-prefix}-user-md:before { content: $fa-var-user-md; } -.#{$fa-css-prefix}-stethoscope:before { content: $fa-var-stethoscope; } -.#{$fa-css-prefix}-suitcase:before { content: $fa-var-suitcase; } -.#{$fa-css-prefix}-bell-o:before { content: $fa-var-bell-o; } -.#{$fa-css-prefix}-coffee:before { content: $fa-var-coffee; } -.#{$fa-css-prefix}-cutlery:before { content: $fa-var-cutlery; } -.#{$fa-css-prefix}-file-text-o:before { content: $fa-var-file-text-o; } -.#{$fa-css-prefix}-building-o:before { content: $fa-var-building-o; } -.#{$fa-css-prefix}-hospital-o:before { content: $fa-var-hospital-o; } -.#{$fa-css-prefix}-ambulance:before { content: $fa-var-ambulance; } -.#{$fa-css-prefix}-medkit:before { content: $fa-var-medkit; } -.#{$fa-css-prefix}-fighter-jet:before { content: $fa-var-fighter-jet; } -.#{$fa-css-prefix}-beer:before { content: $fa-var-beer; } -.#{$fa-css-prefix}-h-square:before { content: $fa-var-h-square; } -.#{$fa-css-prefix}-plus-square:before { content: $fa-var-plus-square; } -.#{$fa-css-prefix}-angle-double-left:before { content: $fa-var-angle-double-left; } -.#{$fa-css-prefix}-angle-double-right:before { content: $fa-var-angle-double-right; } -.#{$fa-css-prefix}-angle-double-up:before { content: $fa-var-angle-double-up; } -.#{$fa-css-prefix}-angle-double-down:before { content: $fa-var-angle-double-down; } -.#{$fa-css-prefix}-angle-left:before { content: $fa-var-angle-left; } -.#{$fa-css-prefix}-angle-right:before { content: $fa-var-angle-right; } -.#{$fa-css-prefix}-angle-up:before { content: $fa-var-angle-up; } -.#{$fa-css-prefix}-angle-down:before { content: $fa-var-angle-down; } -.#{$fa-css-prefix}-desktop:before { content: $fa-var-desktop; } -.#{$fa-css-prefix}-laptop:before { content: $fa-var-laptop; } -.#{$fa-css-prefix}-tablet:before { content: $fa-var-tablet; } -.#{$fa-css-prefix}-mobile-phone:before, -.#{$fa-css-prefix}-mobile:before { content: $fa-var-mobile; } -.#{$fa-css-prefix}-circle-o:before { content: $fa-var-circle-o; } -.#{$fa-css-prefix}-quote-left:before { content: $fa-var-quote-left; } -.#{$fa-css-prefix}-quote-right:before { content: $fa-var-quote-right; } -.#{$fa-css-prefix}-spinner:before { content: $fa-var-spinner; } -.#{$fa-css-prefix}-circle:before { content: $fa-var-circle; } -.#{$fa-css-prefix}-mail-reply:before, -.#{$fa-css-prefix}-reply:before { content: $fa-var-reply; } -.#{$fa-css-prefix}-github-alt:before { content: $fa-var-github-alt; } -.#{$fa-css-prefix}-folder-o:before { content: $fa-var-folder-o; } -.#{$fa-css-prefix}-folder-open-o:before { content: $fa-var-folder-open-o; } -.#{$fa-css-prefix}-smile-o:before { content: $fa-var-smile-o; } -.#{$fa-css-prefix}-frown-o:before { content: $fa-var-frown-o; } -.#{$fa-css-prefix}-meh-o:before { content: $fa-var-meh-o; } -.#{$fa-css-prefix}-gamepad:before { content: $fa-var-gamepad; } -.#{$fa-css-prefix}-keyboard-o:before { content: $fa-var-keyboard-o; } -.#{$fa-css-prefix}-flag-o:before { content: $fa-var-flag-o; } -.#{$fa-css-prefix}-flag-checkered:before { content: $fa-var-flag-checkered; } -.#{$fa-css-prefix}-terminal:before { content: $fa-var-terminal; } -.#{$fa-css-prefix}-code:before { content: $fa-var-code; } -.#{$fa-css-prefix}-mail-reply-all:before, -.#{$fa-css-prefix}-reply-all:before { content: $fa-var-reply-all; } -.#{$fa-css-prefix}-star-half-empty:before, -.#{$fa-css-prefix}-star-half-full:before, -.#{$fa-css-prefix}-star-half-o:before { content: $fa-var-star-half-o; } -.#{$fa-css-prefix}-location-arrow:before { content: $fa-var-location-arrow; } -.#{$fa-css-prefix}-crop:before { content: $fa-var-crop; } -.#{$fa-css-prefix}-code-fork:before { content: $fa-var-code-fork; } -.#{$fa-css-prefix}-unlink:before, -.#{$fa-css-prefix}-chain-broken:before { content: $fa-var-chain-broken; } -.#{$fa-css-prefix}-question:before { content: $fa-var-question; } -.#{$fa-css-prefix}-info:before { content: $fa-var-info; } -.#{$fa-css-prefix}-exclamation:before { content: $fa-var-exclamation; } -.#{$fa-css-prefix}-superscript:before { content: $fa-var-superscript; } -.#{$fa-css-prefix}-subscript:before { content: $fa-var-subscript; } -.#{$fa-css-prefix}-eraser:before { content: $fa-var-eraser; } -.#{$fa-css-prefix}-puzzle-piece:before { content: $fa-var-puzzle-piece; } -.#{$fa-css-prefix}-microphone:before { content: $fa-var-microphone; } -.#{$fa-css-prefix}-microphone-slash:before { content: $fa-var-microphone-slash; } -.#{$fa-css-prefix}-shield:before { content: $fa-var-shield; } -.#{$fa-css-prefix}-calendar-o:before { content: $fa-var-calendar-o; } -.#{$fa-css-prefix}-fire-extinguisher:before { content: $fa-var-fire-extinguisher; } -.#{$fa-css-prefix}-rocket:before { content: $fa-var-rocket; } -.#{$fa-css-prefix}-maxcdn:before { content: $fa-var-maxcdn; } -.#{$fa-css-prefix}-chevron-circle-left:before { content: $fa-var-chevron-circle-left; } -.#{$fa-css-prefix}-chevron-circle-right:before { content: $fa-var-chevron-circle-right; } -.#{$fa-css-prefix}-chevron-circle-up:before { content: $fa-var-chevron-circle-up; } -.#{$fa-css-prefix}-chevron-circle-down:before { content: $fa-var-chevron-circle-down; } -.#{$fa-css-prefix}-html5:before { content: $fa-var-html5; } -.#{$fa-css-prefix}-css3:before { content: $fa-var-css3; } -.#{$fa-css-prefix}-anchor:before { content: $fa-var-anchor; } -.#{$fa-css-prefix}-unlock-alt:before { content: $fa-var-unlock-alt; } -.#{$fa-css-prefix}-bullseye:before { content: $fa-var-bullseye; } -.#{$fa-css-prefix}-ellipsis-h:before { content: $fa-var-ellipsis-h; } -.#{$fa-css-prefix}-ellipsis-v:before { content: $fa-var-ellipsis-v; } -.#{$fa-css-prefix}-rss-square:before { content: $fa-var-rss-square; } -.#{$fa-css-prefix}-play-circle:before { content: $fa-var-play-circle; } -.#{$fa-css-prefix}-ticket:before { content: $fa-var-ticket; } -.#{$fa-css-prefix}-minus-square:before { content: $fa-var-minus-square; } -.#{$fa-css-prefix}-minus-square-o:before { content: $fa-var-minus-square-o; } -.#{$fa-css-prefix}-level-up:before { content: $fa-var-level-up; } -.#{$fa-css-prefix}-level-down:before { content: $fa-var-level-down; } -.#{$fa-css-prefix}-check-square:before { content: $fa-var-check-square; } -.#{$fa-css-prefix}-pencil-square:before { content: $fa-var-pencil-square; } -.#{$fa-css-prefix}-external-link-square:before { content: $fa-var-external-link-square; } -.#{$fa-css-prefix}-share-square:before { content: $fa-var-share-square; } -.#{$fa-css-prefix}-compass:before { content: $fa-var-compass; } -.#{$fa-css-prefix}-toggle-down:before, -.#{$fa-css-prefix}-caret-square-o-down:before { content: $fa-var-caret-square-o-down; } -.#{$fa-css-prefix}-toggle-up:before, -.#{$fa-css-prefix}-caret-square-o-up:before { content: $fa-var-caret-square-o-up; } -.#{$fa-css-prefix}-toggle-right:before, -.#{$fa-css-prefix}-caret-square-o-right:before { content: $fa-var-caret-square-o-right; } -.#{$fa-css-prefix}-euro:before, -.#{$fa-css-prefix}-eur:before { content: $fa-var-eur; } -.#{$fa-css-prefix}-gbp:before { content: $fa-var-gbp; } -.#{$fa-css-prefix}-dollar:before, -.#{$fa-css-prefix}-usd:before { content: $fa-var-usd; } -.#{$fa-css-prefix}-rupee:before, -.#{$fa-css-prefix}-inr:before { content: $fa-var-inr; } -.#{$fa-css-prefix}-cny:before, -.#{$fa-css-prefix}-rmb:before, -.#{$fa-css-prefix}-yen:before, -.#{$fa-css-prefix}-jpy:before { content: $fa-var-jpy; } -.#{$fa-css-prefix}-ruble:before, -.#{$fa-css-prefix}-rouble:before, -.#{$fa-css-prefix}-rub:before { content: $fa-var-rub; } -.#{$fa-css-prefix}-won:before, -.#{$fa-css-prefix}-krw:before { content: $fa-var-krw; } -.#{$fa-css-prefix}-bitcoin:before, -.#{$fa-css-prefix}-btc:before { content: $fa-var-btc; } -.#{$fa-css-prefix}-file:before { content: $fa-var-file; } -.#{$fa-css-prefix}-file-text:before { content: $fa-var-file-text; } -.#{$fa-css-prefix}-sort-alpha-asc:before { content: $fa-var-sort-alpha-asc; } -.#{$fa-css-prefix}-sort-alpha-desc:before { content: $fa-var-sort-alpha-desc; } -.#{$fa-css-prefix}-sort-amount-asc:before { content: $fa-var-sort-amount-asc; } -.#{$fa-css-prefix}-sort-amount-desc:before { content: $fa-var-sort-amount-desc; } -.#{$fa-css-prefix}-sort-numeric-asc:before { content: $fa-var-sort-numeric-asc; } -.#{$fa-css-prefix}-sort-numeric-desc:before { content: $fa-var-sort-numeric-desc; } -.#{$fa-css-prefix}-thumbs-up:before { content: $fa-var-thumbs-up; } -.#{$fa-css-prefix}-thumbs-down:before { content: $fa-var-thumbs-down; } -.#{$fa-css-prefix}-youtube-square:before { content: $fa-var-youtube-square; } -.#{$fa-css-prefix}-youtube:before { content: $fa-var-youtube; } -.#{$fa-css-prefix}-xing:before { content: $fa-var-xing; } -.#{$fa-css-prefix}-xing-square:before { content: $fa-var-xing-square; } -.#{$fa-css-prefix}-youtube-play:before { content: $fa-var-youtube-play; } -.#{$fa-css-prefix}-dropbox:before { content: $fa-var-dropbox; } -.#{$fa-css-prefix}-stack-overflow:before { content: $fa-var-stack-overflow; } -.#{$fa-css-prefix}-instagram:before { content: $fa-var-instagram; } -.#{$fa-css-prefix}-flickr:before { content: $fa-var-flickr; } -.#{$fa-css-prefix}-adn:before { content: $fa-var-adn; } -.#{$fa-css-prefix}-bitbucket:before { content: $fa-var-bitbucket; } -.#{$fa-css-prefix}-bitbucket-square:before { content: $fa-var-bitbucket-square; } -.#{$fa-css-prefix}-tumblr:before { content: $fa-var-tumblr; } -.#{$fa-css-prefix}-tumblr-square:before { content: $fa-var-tumblr-square; } -.#{$fa-css-prefix}-long-arrow-down:before { content: $fa-var-long-arrow-down; } -.#{$fa-css-prefix}-long-arrow-up:before { content: $fa-var-long-arrow-up; } -.#{$fa-css-prefix}-long-arrow-left:before { content: $fa-var-long-arrow-left; } -.#{$fa-css-prefix}-long-arrow-right:before { content: $fa-var-long-arrow-right; } -.#{$fa-css-prefix}-apple:before { content: $fa-var-apple; } -.#{$fa-css-prefix}-windows:before { content: $fa-var-windows; } -.#{$fa-css-prefix}-android:before { content: $fa-var-android; } -.#{$fa-css-prefix}-linux:before { content: $fa-var-linux; } -.#{$fa-css-prefix}-dribbble:before { content: $fa-var-dribbble; } -.#{$fa-css-prefix}-skype:before { content: $fa-var-skype; } -.#{$fa-css-prefix}-foursquare:before { content: $fa-var-foursquare; } -.#{$fa-css-prefix}-trello:before { content: $fa-var-trello; } -.#{$fa-css-prefix}-female:before { content: $fa-var-female; } -.#{$fa-css-prefix}-male:before { content: $fa-var-male; } -.#{$fa-css-prefix}-gittip:before, -.#{$fa-css-prefix}-gratipay:before { content: $fa-var-gratipay; } -.#{$fa-css-prefix}-sun-o:before { content: $fa-var-sun-o; } -.#{$fa-css-prefix}-moon-o:before { content: $fa-var-moon-o; } -.#{$fa-css-prefix}-archive:before { content: $fa-var-archive; } -.#{$fa-css-prefix}-bug:before { content: $fa-var-bug; } -.#{$fa-css-prefix}-vk:before { content: $fa-var-vk; } -.#{$fa-css-prefix}-weibo:before { content: $fa-var-weibo; } -.#{$fa-css-prefix}-renren:before { content: $fa-var-renren; } -.#{$fa-css-prefix}-pagelines:before { content: $fa-var-pagelines; } -.#{$fa-css-prefix}-stack-exchange:before { content: $fa-var-stack-exchange; } -.#{$fa-css-prefix}-arrow-circle-o-right:before { content: $fa-var-arrow-circle-o-right; } -.#{$fa-css-prefix}-arrow-circle-o-left:before { content: $fa-var-arrow-circle-o-left; } -.#{$fa-css-prefix}-toggle-left:before, -.#{$fa-css-prefix}-caret-square-o-left:before { content: $fa-var-caret-square-o-left; } -.#{$fa-css-prefix}-dot-circle-o:before { content: $fa-var-dot-circle-o; } -.#{$fa-css-prefix}-wheelchair:before { content: $fa-var-wheelchair; } -.#{$fa-css-prefix}-vimeo-square:before { content: $fa-var-vimeo-square; } -.#{$fa-css-prefix}-turkish-lira:before, -.#{$fa-css-prefix}-try:before { content: $fa-var-try; } -.#{$fa-css-prefix}-plus-square-o:before { content: $fa-var-plus-square-o; } -.#{$fa-css-prefix}-space-shuttle:before { content: $fa-var-space-shuttle; } -.#{$fa-css-prefix}-slack:before { content: $fa-var-slack; } -.#{$fa-css-prefix}-envelope-square:before { content: $fa-var-envelope-square; } -.#{$fa-css-prefix}-wordpress:before { content: $fa-var-wordpress; } -.#{$fa-css-prefix}-openid:before { content: $fa-var-openid; } -.#{$fa-css-prefix}-institution:before, -.#{$fa-css-prefix}-bank:before, -.#{$fa-css-prefix}-university:before { content: $fa-var-university; } -.#{$fa-css-prefix}-mortar-board:before, -.#{$fa-css-prefix}-graduation-cap:before { content: $fa-var-graduation-cap; } -.#{$fa-css-prefix}-yahoo:before { content: $fa-var-yahoo; } -.#{$fa-css-prefix}-google:before { content: $fa-var-google; } -.#{$fa-css-prefix}-reddit:before { content: $fa-var-reddit; } -.#{$fa-css-prefix}-reddit-square:before { content: $fa-var-reddit-square; } -.#{$fa-css-prefix}-stumbleupon-circle:before { content: $fa-var-stumbleupon-circle; } -.#{$fa-css-prefix}-stumbleupon:before { content: $fa-var-stumbleupon; } -.#{$fa-css-prefix}-delicious:before { content: $fa-var-delicious; } -.#{$fa-css-prefix}-digg:before { content: $fa-var-digg; } -.#{$fa-css-prefix}-pied-piper-pp:before { content: $fa-var-pied-piper-pp; } -.#{$fa-css-prefix}-pied-piper-alt:before { content: $fa-var-pied-piper-alt; } -.#{$fa-css-prefix}-drupal:before { content: $fa-var-drupal; } -.#{$fa-css-prefix}-joomla:before { content: $fa-var-joomla; } -.#{$fa-css-prefix}-language:before { content: $fa-var-language; } -.#{$fa-css-prefix}-fax:before { content: $fa-var-fax; } -.#{$fa-css-prefix}-building:before { content: $fa-var-building; } -.#{$fa-css-prefix}-child:before { content: $fa-var-child; } -.#{$fa-css-prefix}-paw:before { content: $fa-var-paw; } -.#{$fa-css-prefix}-spoon:before { content: $fa-var-spoon; } -.#{$fa-css-prefix}-cube:before { content: $fa-var-cube; } -.#{$fa-css-prefix}-cubes:before { content: $fa-var-cubes; } -.#{$fa-css-prefix}-behance:before { content: $fa-var-behance; } -.#{$fa-css-prefix}-behance-square:before { content: $fa-var-behance-square; } -.#{$fa-css-prefix}-steam:before { content: $fa-var-steam; } -.#{$fa-css-prefix}-steam-square:before { content: $fa-var-steam-square; } -.#{$fa-css-prefix}-recycle:before { content: $fa-var-recycle; } -.#{$fa-css-prefix}-automobile:before, -.#{$fa-css-prefix}-car:before { content: $fa-var-car; } -.#{$fa-css-prefix}-cab:before, -.#{$fa-css-prefix}-taxi:before { content: $fa-var-taxi; } -.#{$fa-css-prefix}-tree:before { content: $fa-var-tree; } -.#{$fa-css-prefix}-spotify:before { content: $fa-var-spotify; } -.#{$fa-css-prefix}-deviantart:before { content: $fa-var-deviantart; } -.#{$fa-css-prefix}-soundcloud:before { content: $fa-var-soundcloud; } -.#{$fa-css-prefix}-database:before { content: $fa-var-database; } -.#{$fa-css-prefix}-file-pdf-o:before { content: $fa-var-file-pdf-o; } -.#{$fa-css-prefix}-file-word-o:before { content: $fa-var-file-word-o; } -.#{$fa-css-prefix}-file-excel-o:before { content: $fa-var-file-excel-o; } -.#{$fa-css-prefix}-file-powerpoint-o:before { content: $fa-var-file-powerpoint-o; } -.#{$fa-css-prefix}-file-photo-o:before, -.#{$fa-css-prefix}-file-picture-o:before, -.#{$fa-css-prefix}-file-image-o:before { content: $fa-var-file-image-o; } -.#{$fa-css-prefix}-file-zip-o:before, -.#{$fa-css-prefix}-file-archive-o:before { content: $fa-var-file-archive-o; } -.#{$fa-css-prefix}-file-sound-o:before, -.#{$fa-css-prefix}-file-audio-o:before { content: $fa-var-file-audio-o; } -.#{$fa-css-prefix}-file-movie-o:before, -.#{$fa-css-prefix}-file-video-o:before { content: $fa-var-file-video-o; } -.#{$fa-css-prefix}-file-code-o:before { content: $fa-var-file-code-o; } -.#{$fa-css-prefix}-vine:before { content: $fa-var-vine; } -.#{$fa-css-prefix}-codepen:before { content: $fa-var-codepen; } -.#{$fa-css-prefix}-jsfiddle:before { content: $fa-var-jsfiddle; } -.#{$fa-css-prefix}-life-bouy:before, -.#{$fa-css-prefix}-life-buoy:before, -.#{$fa-css-prefix}-life-saver:before, -.#{$fa-css-prefix}-support:before, -.#{$fa-css-prefix}-life-ring:before { content: $fa-var-life-ring; } -.#{$fa-css-prefix}-circle-o-notch:before { content: $fa-var-circle-o-notch; } -.#{$fa-css-prefix}-ra:before, -.#{$fa-css-prefix}-resistance:before, -.#{$fa-css-prefix}-rebel:before { content: $fa-var-rebel; } -.#{$fa-css-prefix}-ge:before, -.#{$fa-css-prefix}-empire:before { content: $fa-var-empire; } -.#{$fa-css-prefix}-git-square:before { content: $fa-var-git-square; } -.#{$fa-css-prefix}-git:before { content: $fa-var-git; } -.#{$fa-css-prefix}-y-combinator-square:before, -.#{$fa-css-prefix}-yc-square:before, -.#{$fa-css-prefix}-hacker-news:before { content: $fa-var-hacker-news; } -.#{$fa-css-prefix}-tencent-weibo:before { content: $fa-var-tencent-weibo; } -.#{$fa-css-prefix}-qq:before { content: $fa-var-qq; } -.#{$fa-css-prefix}-wechat:before, -.#{$fa-css-prefix}-weixin:before { content: $fa-var-weixin; } -.#{$fa-css-prefix}-send:before, -.#{$fa-css-prefix}-paper-plane:before { content: $fa-var-paper-plane; } -.#{$fa-css-prefix}-send-o:before, -.#{$fa-css-prefix}-paper-plane-o:before { content: $fa-var-paper-plane-o; } -.#{$fa-css-prefix}-history:before { content: $fa-var-history; } -.#{$fa-css-prefix}-circle-thin:before { content: $fa-var-circle-thin; } -.#{$fa-css-prefix}-header:before { content: $fa-var-header; } -.#{$fa-css-prefix}-paragraph:before { content: $fa-var-paragraph; } -.#{$fa-css-prefix}-sliders:before { content: $fa-var-sliders; } -.#{$fa-css-prefix}-share-alt:before { content: $fa-var-share-alt; } -.#{$fa-css-prefix}-share-alt-square:before { content: $fa-var-share-alt-square; } -.#{$fa-css-prefix}-bomb:before { content: $fa-var-bomb; } -.#{$fa-css-prefix}-soccer-ball-o:before, -.#{$fa-css-prefix}-futbol-o:before { content: $fa-var-futbol-o; } -.#{$fa-css-prefix}-tty:before { content: $fa-var-tty; } -.#{$fa-css-prefix}-binoculars:before { content: $fa-var-binoculars; } -.#{$fa-css-prefix}-plug:before { content: $fa-var-plug; } -.#{$fa-css-prefix}-slideshare:before { content: $fa-var-slideshare; } -.#{$fa-css-prefix}-twitch:before { content: $fa-var-twitch; } -.#{$fa-css-prefix}-yelp:before { content: $fa-var-yelp; } -.#{$fa-css-prefix}-newspaper-o:before { content: $fa-var-newspaper-o; } -.#{$fa-css-prefix}-wifi:before { content: $fa-var-wifi; } -.#{$fa-css-prefix}-calculator:before { content: $fa-var-calculator; } -.#{$fa-css-prefix}-paypal:before { content: $fa-var-paypal; } -.#{$fa-css-prefix}-google-wallet:before { content: $fa-var-google-wallet; } -.#{$fa-css-prefix}-cc-visa:before { content: $fa-var-cc-visa; } -.#{$fa-css-prefix}-cc-mastercard:before { content: $fa-var-cc-mastercard; } -.#{$fa-css-prefix}-cc-discover:before { content: $fa-var-cc-discover; } -.#{$fa-css-prefix}-cc-amex:before { content: $fa-var-cc-amex; } -.#{$fa-css-prefix}-cc-paypal:before { content: $fa-var-cc-paypal; } -.#{$fa-css-prefix}-cc-stripe:before { content: $fa-var-cc-stripe; } -.#{$fa-css-prefix}-bell-slash:before { content: $fa-var-bell-slash; } -.#{$fa-css-prefix}-bell-slash-o:before { content: $fa-var-bell-slash-o; } -.#{$fa-css-prefix}-trash:before { content: $fa-var-trash; } -.#{$fa-css-prefix}-copyright:before { content: $fa-var-copyright; } -.#{$fa-css-prefix}-at:before { content: $fa-var-at; } -.#{$fa-css-prefix}-eyedropper:before { content: $fa-var-eyedropper; } -.#{$fa-css-prefix}-paint-brush:before { content: $fa-var-paint-brush; } -.#{$fa-css-prefix}-birthday-cake:before { content: $fa-var-birthday-cake; } -.#{$fa-css-prefix}-area-chart:before { content: $fa-var-area-chart; } -.#{$fa-css-prefix}-pie-chart:before { content: $fa-var-pie-chart; } -.#{$fa-css-prefix}-line-chart:before { content: $fa-var-line-chart; } -.#{$fa-css-prefix}-lastfm:before { content: $fa-var-lastfm; } -.#{$fa-css-prefix}-lastfm-square:before { content: $fa-var-lastfm-square; } -.#{$fa-css-prefix}-toggle-off:before { content: $fa-var-toggle-off; } -.#{$fa-css-prefix}-toggle-on:before { content: $fa-var-toggle-on; } -.#{$fa-css-prefix}-bicycle:before { content: $fa-var-bicycle; } -.#{$fa-css-prefix}-bus:before { content: $fa-var-bus; } -.#{$fa-css-prefix}-ioxhost:before { content: $fa-var-ioxhost; } -.#{$fa-css-prefix}-angellist:before { content: $fa-var-angellist; } -.#{$fa-css-prefix}-cc:before { content: $fa-var-cc; } -.#{$fa-css-prefix}-shekel:before, -.#{$fa-css-prefix}-sheqel:before, -.#{$fa-css-prefix}-ils:before { content: $fa-var-ils; } -.#{$fa-css-prefix}-meanpath:before { content: $fa-var-meanpath; } -.#{$fa-css-prefix}-buysellads:before { content: $fa-var-buysellads; } -.#{$fa-css-prefix}-connectdevelop:before { content: $fa-var-connectdevelop; } -.#{$fa-css-prefix}-dashcube:before { content: $fa-var-dashcube; } -.#{$fa-css-prefix}-forumbee:before { content: $fa-var-forumbee; } -.#{$fa-css-prefix}-leanpub:before { content: $fa-var-leanpub; } -.#{$fa-css-prefix}-sellsy:before { content: $fa-var-sellsy; } -.#{$fa-css-prefix}-shirtsinbulk:before { content: $fa-var-shirtsinbulk; } -.#{$fa-css-prefix}-simplybuilt:before { content: $fa-var-simplybuilt; } -.#{$fa-css-prefix}-skyatlas:before { content: $fa-var-skyatlas; } -.#{$fa-css-prefix}-cart-plus:before { content: $fa-var-cart-plus; } -.#{$fa-css-prefix}-cart-arrow-down:before { content: $fa-var-cart-arrow-down; } -.#{$fa-css-prefix}-diamond:before { content: $fa-var-diamond; } -.#{$fa-css-prefix}-ship:before { content: $fa-var-ship; } -.#{$fa-css-prefix}-user-secret:before { content: $fa-var-user-secret; } -.#{$fa-css-prefix}-motorcycle:before { content: $fa-var-motorcycle; } -.#{$fa-css-prefix}-street-view:before { content: $fa-var-street-view; } -.#{$fa-css-prefix}-heartbeat:before { content: $fa-var-heartbeat; } -.#{$fa-css-prefix}-venus:before { content: $fa-var-venus; } -.#{$fa-css-prefix}-mars:before { content: $fa-var-mars; } -.#{$fa-css-prefix}-mercury:before { content: $fa-var-mercury; } -.#{$fa-css-prefix}-intersex:before, -.#{$fa-css-prefix}-transgender:before { content: $fa-var-transgender; } -.#{$fa-css-prefix}-transgender-alt:before { content: $fa-var-transgender-alt; } -.#{$fa-css-prefix}-venus-double:before { content: $fa-var-venus-double; } -.#{$fa-css-prefix}-mars-double:before { content: $fa-var-mars-double; } -.#{$fa-css-prefix}-venus-mars:before { content: $fa-var-venus-mars; } -.#{$fa-css-prefix}-mars-stroke:before { content: $fa-var-mars-stroke; } -.#{$fa-css-prefix}-mars-stroke-v:before { content: $fa-var-mars-stroke-v; } -.#{$fa-css-prefix}-mars-stroke-h:before { content: $fa-var-mars-stroke-h; } -.#{$fa-css-prefix}-neuter:before { content: $fa-var-neuter; } -.#{$fa-css-prefix}-genderless:before { content: $fa-var-genderless; } -.#{$fa-css-prefix}-facebook-official:before { content: $fa-var-facebook-official; } -.#{$fa-css-prefix}-pinterest-p:before { content: $fa-var-pinterest-p; } -.#{$fa-css-prefix}-whatsapp:before { content: $fa-var-whatsapp; } -.#{$fa-css-prefix}-server:before { content: $fa-var-server; } -.#{$fa-css-prefix}-user-plus:before { content: $fa-var-user-plus; } -.#{$fa-css-prefix}-user-times:before { content: $fa-var-user-times; } -.#{$fa-css-prefix}-hotel:before, -.#{$fa-css-prefix}-bed:before { content: $fa-var-bed; } -.#{$fa-css-prefix}-viacoin:before { content: $fa-var-viacoin; } -.#{$fa-css-prefix}-train:before { content: $fa-var-train; } -.#{$fa-css-prefix}-subway:before { content: $fa-var-subway; } -.#{$fa-css-prefix}-medium:before { content: $fa-var-medium; } -.#{$fa-css-prefix}-yc:before, -.#{$fa-css-prefix}-y-combinator:before { content: $fa-var-y-combinator; } -.#{$fa-css-prefix}-optin-monster:before { content: $fa-var-optin-monster; } -.#{$fa-css-prefix}-opencart:before { content: $fa-var-opencart; } -.#{$fa-css-prefix}-expeditedssl:before { content: $fa-var-expeditedssl; } -.#{$fa-css-prefix}-battery-4:before, -.#{$fa-css-prefix}-battery:before, -.#{$fa-css-prefix}-battery-full:before { content: $fa-var-battery-full; } -.#{$fa-css-prefix}-battery-3:before, -.#{$fa-css-prefix}-battery-three-quarters:before { content: $fa-var-battery-three-quarters; } -.#{$fa-css-prefix}-battery-2:before, -.#{$fa-css-prefix}-battery-half:before { content: $fa-var-battery-half; } -.#{$fa-css-prefix}-battery-1:before, -.#{$fa-css-prefix}-battery-quarter:before { content: $fa-var-battery-quarter; } -.#{$fa-css-prefix}-battery-0:before, -.#{$fa-css-prefix}-battery-empty:before { content: $fa-var-battery-empty; } -.#{$fa-css-prefix}-mouse-pointer:before { content: $fa-var-mouse-pointer; } -.#{$fa-css-prefix}-i-cursor:before { content: $fa-var-i-cursor; } -.#{$fa-css-prefix}-object-group:before { content: $fa-var-object-group; } -.#{$fa-css-prefix}-object-ungroup:before { content: $fa-var-object-ungroup; } -.#{$fa-css-prefix}-sticky-note:before { content: $fa-var-sticky-note; } -.#{$fa-css-prefix}-sticky-note-o:before { content: $fa-var-sticky-note-o; } -.#{$fa-css-prefix}-cc-jcb:before { content: $fa-var-cc-jcb; } -.#{$fa-css-prefix}-cc-diners-club:before { content: $fa-var-cc-diners-club; } -.#{$fa-css-prefix}-clone:before { content: $fa-var-clone; } -.#{$fa-css-prefix}-balance-scale:before { content: $fa-var-balance-scale; } -.#{$fa-css-prefix}-hourglass-o:before { content: $fa-var-hourglass-o; } -.#{$fa-css-prefix}-hourglass-1:before, -.#{$fa-css-prefix}-hourglass-start:before { content: $fa-var-hourglass-start; } -.#{$fa-css-prefix}-hourglass-2:before, -.#{$fa-css-prefix}-hourglass-half:before { content: $fa-var-hourglass-half; } -.#{$fa-css-prefix}-hourglass-3:before, -.#{$fa-css-prefix}-hourglass-end:before { content: $fa-var-hourglass-end; } -.#{$fa-css-prefix}-hourglass:before { content: $fa-var-hourglass; } -.#{$fa-css-prefix}-hand-grab-o:before, -.#{$fa-css-prefix}-hand-rock-o:before { content: $fa-var-hand-rock-o; } -.#{$fa-css-prefix}-hand-stop-o:before, -.#{$fa-css-prefix}-hand-paper-o:before { content: $fa-var-hand-paper-o; } -.#{$fa-css-prefix}-hand-scissors-o:before { content: $fa-var-hand-scissors-o; } -.#{$fa-css-prefix}-hand-lizard-o:before { content: $fa-var-hand-lizard-o; } -.#{$fa-css-prefix}-hand-spock-o:before { content: $fa-var-hand-spock-o; } -.#{$fa-css-prefix}-hand-pointer-o:before { content: $fa-var-hand-pointer-o; } -.#{$fa-css-prefix}-hand-peace-o:before { content: $fa-var-hand-peace-o; } -.#{$fa-css-prefix}-trademark:before { content: $fa-var-trademark; } -.#{$fa-css-prefix}-registered:before { content: $fa-var-registered; } -.#{$fa-css-prefix}-creative-commons:before { content: $fa-var-creative-commons; } -.#{$fa-css-prefix}-gg:before { content: $fa-var-gg; } -.#{$fa-css-prefix}-gg-circle:before { content: $fa-var-gg-circle; } -.#{$fa-css-prefix}-tripadvisor:before { content: $fa-var-tripadvisor; } -.#{$fa-css-prefix}-odnoklassniki:before { content: $fa-var-odnoklassniki; } -.#{$fa-css-prefix}-odnoklassniki-square:before { content: $fa-var-odnoklassniki-square; } -.#{$fa-css-prefix}-get-pocket:before { content: $fa-var-get-pocket; } -.#{$fa-css-prefix}-wikipedia-w:before { content: $fa-var-wikipedia-w; } -.#{$fa-css-prefix}-safari:before { content: $fa-var-safari; } -.#{$fa-css-prefix}-chrome:before { content: $fa-var-chrome; } -.#{$fa-css-prefix}-firefox:before { content: $fa-var-firefox; } -.#{$fa-css-prefix}-opera:before { content: $fa-var-opera; } -.#{$fa-css-prefix}-internet-explorer:before { content: $fa-var-internet-explorer; } -.#{$fa-css-prefix}-tv:before, -.#{$fa-css-prefix}-television:before { content: $fa-var-television; } -.#{$fa-css-prefix}-contao:before { content: $fa-var-contao; } -.#{$fa-css-prefix}-500px:before { content: $fa-var-500px; } -.#{$fa-css-prefix}-amazon:before { content: $fa-var-amazon; } -.#{$fa-css-prefix}-calendar-plus-o:before { content: $fa-var-calendar-plus-o; } -.#{$fa-css-prefix}-calendar-minus-o:before { content: $fa-var-calendar-minus-o; } -.#{$fa-css-prefix}-calendar-times-o:before { content: $fa-var-calendar-times-o; } -.#{$fa-css-prefix}-calendar-check-o:before { content: $fa-var-calendar-check-o; } -.#{$fa-css-prefix}-industry:before { content: $fa-var-industry; } -.#{$fa-css-prefix}-map-pin:before { content: $fa-var-map-pin; } -.#{$fa-css-prefix}-map-signs:before { content: $fa-var-map-signs; } -.#{$fa-css-prefix}-map-o:before { content: $fa-var-map-o; } -.#{$fa-css-prefix}-map:before { content: $fa-var-map; } -.#{$fa-css-prefix}-commenting:before { content: $fa-var-commenting; } -.#{$fa-css-prefix}-commenting-o:before { content: $fa-var-commenting-o; } -.#{$fa-css-prefix}-houzz:before { content: $fa-var-houzz; } -.#{$fa-css-prefix}-vimeo:before { content: $fa-var-vimeo; } -.#{$fa-css-prefix}-black-tie:before { content: $fa-var-black-tie; } -.#{$fa-css-prefix}-fonticons:before { content: $fa-var-fonticons; } -.#{$fa-css-prefix}-reddit-alien:before { content: $fa-var-reddit-alien; } -.#{$fa-css-prefix}-edge:before { content: $fa-var-edge; } -.#{$fa-css-prefix}-credit-card-alt:before { content: $fa-var-credit-card-alt; } -.#{$fa-css-prefix}-codiepie:before { content: $fa-var-codiepie; } -.#{$fa-css-prefix}-modx:before { content: $fa-var-modx; } -.#{$fa-css-prefix}-fort-awesome:before { content: $fa-var-fort-awesome; } -.#{$fa-css-prefix}-usb:before { content: $fa-var-usb; } -.#{$fa-css-prefix}-product-hunt:before { content: $fa-var-product-hunt; } -.#{$fa-css-prefix}-mixcloud:before { content: $fa-var-mixcloud; } -.#{$fa-css-prefix}-scribd:before { content: $fa-var-scribd; } -.#{$fa-css-prefix}-pause-circle:before { content: $fa-var-pause-circle; } -.#{$fa-css-prefix}-pause-circle-o:before { content: $fa-var-pause-circle-o; } -.#{$fa-css-prefix}-stop-circle:before { content: $fa-var-stop-circle; } -.#{$fa-css-prefix}-stop-circle-o:before { content: $fa-var-stop-circle-o; } -.#{$fa-css-prefix}-shopping-bag:before { content: $fa-var-shopping-bag; } -.#{$fa-css-prefix}-shopping-basket:before { content: $fa-var-shopping-basket; } -.#{$fa-css-prefix}-hashtag:before { content: $fa-var-hashtag; } -.#{$fa-css-prefix}-bluetooth:before { content: $fa-var-bluetooth; } -.#{$fa-css-prefix}-bluetooth-b:before { content: $fa-var-bluetooth-b; } -.#{$fa-css-prefix}-percent:before { content: $fa-var-percent; } -.#{$fa-css-prefix}-gitlab:before { content: $fa-var-gitlab; } -.#{$fa-css-prefix}-wpbeginner:before { content: $fa-var-wpbeginner; } -.#{$fa-css-prefix}-wpforms:before { content: $fa-var-wpforms; } -.#{$fa-css-prefix}-envira:before { content: $fa-var-envira; } -.#{$fa-css-prefix}-universal-access:before { content: $fa-var-universal-access; } -.#{$fa-css-prefix}-wheelchair-alt:before { content: $fa-var-wheelchair-alt; } -.#{$fa-css-prefix}-question-circle-o:before { content: $fa-var-question-circle-o; } -.#{$fa-css-prefix}-blind:before { content: $fa-var-blind; } -.#{$fa-css-prefix}-audio-description:before { content: $fa-var-audio-description; } -.#{$fa-css-prefix}-volume-control-phone:before { content: $fa-var-volume-control-phone; } -.#{$fa-css-prefix}-braille:before { content: $fa-var-braille; } -.#{$fa-css-prefix}-assistive-listening-systems:before { content: $fa-var-assistive-listening-systems; } -.#{$fa-css-prefix}-asl-interpreting:before, -.#{$fa-css-prefix}-american-sign-language-interpreting:before { content: $fa-var-american-sign-language-interpreting; } -.#{$fa-css-prefix}-deafness:before, -.#{$fa-css-prefix}-hard-of-hearing:before, -.#{$fa-css-prefix}-deaf:before { content: $fa-var-deaf; } -.#{$fa-css-prefix}-glide:before { content: $fa-var-glide; } -.#{$fa-css-prefix}-glide-g:before { content: $fa-var-glide-g; } -.#{$fa-css-prefix}-signing:before, -.#{$fa-css-prefix}-sign-language:before { content: $fa-var-sign-language; } -.#{$fa-css-prefix}-low-vision:before { content: $fa-var-low-vision; } -.#{$fa-css-prefix}-viadeo:before { content: $fa-var-viadeo; } -.#{$fa-css-prefix}-viadeo-square:before { content: $fa-var-viadeo-square; } -.#{$fa-css-prefix}-snapchat:before { content: $fa-var-snapchat; } -.#{$fa-css-prefix}-snapchat-ghost:before { content: $fa-var-snapchat-ghost; } -.#{$fa-css-prefix}-snapchat-square:before { content: $fa-var-snapchat-square; } -.#{$fa-css-prefix}-pied-piper:before { content: $fa-var-pied-piper; } -.#{$fa-css-prefix}-first-order:before { content: $fa-var-first-order; } -.#{$fa-css-prefix}-yoast:before { content: $fa-var-yoast; } -.#{$fa-css-prefix}-themeisle:before { content: $fa-var-themeisle; } -.#{$fa-css-prefix}-google-plus-circle:before, -.#{$fa-css-prefix}-google-plus-official:before { content: $fa-var-google-plus-official; } -.#{$fa-css-prefix}-fa:before, -.#{$fa-css-prefix}-font-awesome:before { content: $fa-var-font-awesome; } -.#{$fa-css-prefix}-handshake-o:before { content: $fa-var-handshake-o; } -.#{$fa-css-prefix}-envelope-open:before { content: $fa-var-envelope-open; } -.#{$fa-css-prefix}-envelope-open-o:before { content: $fa-var-envelope-open-o; } -.#{$fa-css-prefix}-linode:before { content: $fa-var-linode; } -.#{$fa-css-prefix}-address-book:before { content: $fa-var-address-book; } -.#{$fa-css-prefix}-address-book-o:before { content: $fa-var-address-book-o; } -.#{$fa-css-prefix}-vcard:before, -.#{$fa-css-prefix}-address-card:before { content: $fa-var-address-card; } -.#{$fa-css-prefix}-vcard-o:before, -.#{$fa-css-prefix}-address-card-o:before { content: $fa-var-address-card-o; } -.#{$fa-css-prefix}-user-circle:before { content: $fa-var-user-circle; } -.#{$fa-css-prefix}-user-circle-o:before { content: $fa-var-user-circle-o; } -.#{$fa-css-prefix}-user-o:before { content: $fa-var-user-o; } -.#{$fa-css-prefix}-id-badge:before { content: $fa-var-id-badge; } -.#{$fa-css-prefix}-drivers-license:before, -.#{$fa-css-prefix}-id-card:before { content: $fa-var-id-card; } -.#{$fa-css-prefix}-drivers-license-o:before, -.#{$fa-css-prefix}-id-card-o:before { content: $fa-var-id-card-o; } -.#{$fa-css-prefix}-quora:before { content: $fa-var-quora; } -.#{$fa-css-prefix}-free-code-camp:before { content: $fa-var-free-code-camp; } -.#{$fa-css-prefix}-telegram:before { content: $fa-var-telegram; } -.#{$fa-css-prefix}-thermometer-4:before, -.#{$fa-css-prefix}-thermometer:before, -.#{$fa-css-prefix}-thermometer-full:before { content: $fa-var-thermometer-full; } -.#{$fa-css-prefix}-thermometer-3:before, -.#{$fa-css-prefix}-thermometer-three-quarters:before { content: $fa-var-thermometer-three-quarters; } -.#{$fa-css-prefix}-thermometer-2:before, -.#{$fa-css-prefix}-thermometer-half:before { content: $fa-var-thermometer-half; } -.#{$fa-css-prefix}-thermometer-1:before, -.#{$fa-css-prefix}-thermometer-quarter:before { content: $fa-var-thermometer-quarter; } -.#{$fa-css-prefix}-thermometer-0:before, -.#{$fa-css-prefix}-thermometer-empty:before { content: $fa-var-thermometer-empty; } -.#{$fa-css-prefix}-shower:before { content: $fa-var-shower; } -.#{$fa-css-prefix}-bathtub:before, -.#{$fa-css-prefix}-s15:before, -.#{$fa-css-prefix}-bath:before { content: $fa-var-bath; } -.#{$fa-css-prefix}-podcast:before { content: $fa-var-podcast; } -.#{$fa-css-prefix}-window-maximize:before { content: $fa-var-window-maximize; } -.#{$fa-css-prefix}-window-minimize:before { content: $fa-var-window-minimize; } -.#{$fa-css-prefix}-window-restore:before { content: $fa-var-window-restore; } -.#{$fa-css-prefix}-times-rectangle:before, -.#{$fa-css-prefix}-window-close:before { content: $fa-var-window-close; } -.#{$fa-css-prefix}-times-rectangle-o:before, -.#{$fa-css-prefix}-window-close-o:before { content: $fa-var-window-close-o; } -.#{$fa-css-prefix}-bandcamp:before { content: $fa-var-bandcamp; } -.#{$fa-css-prefix}-grav:before { content: $fa-var-grav; } -.#{$fa-css-prefix}-etsy:before { content: $fa-var-etsy; } -.#{$fa-css-prefix}-imdb:before { content: $fa-var-imdb; } -.#{$fa-css-prefix}-ravelry:before { content: $fa-var-ravelry; } -.#{$fa-css-prefix}-eercast:before { content: $fa-var-eercast; } -.#{$fa-css-prefix}-microchip:before { content: $fa-var-microchip; } -.#{$fa-css-prefix}-snowflake-o:before { content: $fa-var-snowflake-o; } -.#{$fa-css-prefix}-superpowers:before { content: $fa-var-superpowers; } -.#{$fa-css-prefix}-wpexplorer:before { content: $fa-var-wpexplorer; } -.#{$fa-css-prefix}-meetup:before { content: $fa-var-meetup; } diff --git a/site/_sass/vendor/font-awesome/_larger.scss b/site/_sass/vendor/font-awesome/_larger.scss deleted file mode 100644 index 41e9a8184aa..00000000000 --- a/site/_sass/vendor/font-awesome/_larger.scss +++ /dev/null @@ -1,13 +0,0 @@ -// Icon Sizes -// ------------------------- - -/* makes the font 33% larger relative to the icon container */ -.#{$fa-css-prefix}-lg { - font-size: (4em / 3); - line-height: (3em / 4); - vertical-align: -15%; -} -.#{$fa-css-prefix}-2x { font-size: 2em; } -.#{$fa-css-prefix}-3x { font-size: 3em; } -.#{$fa-css-prefix}-4x { font-size: 4em; } -.#{$fa-css-prefix}-5x { font-size: 5em; } diff --git a/site/_sass/vendor/font-awesome/_list.scss b/site/_sass/vendor/font-awesome/_list.scss deleted file mode 100644 index 7d1e4d54d6c..00000000000 --- a/site/_sass/vendor/font-awesome/_list.scss +++ /dev/null @@ -1,19 +0,0 @@ -// List Icons -// ------------------------- - -.#{$fa-css-prefix}-ul { - padding-left: 0; - margin-left: $fa-li-width; - list-style-type: none; - > li { position: relative; } -} -.#{$fa-css-prefix}-li { - position: absolute; - left: -$fa-li-width; - width: $fa-li-width; - top: (2em / 14); - text-align: center; - &.#{$fa-css-prefix}-lg { - left: -$fa-li-width + (4em / 14); - } -} diff --git a/site/_sass/vendor/font-awesome/_mixins.scss b/site/_sass/vendor/font-awesome/_mixins.scss deleted file mode 100644 index c3bbd5745d3..00000000000 --- a/site/_sass/vendor/font-awesome/_mixins.scss +++ /dev/null @@ -1,60 +0,0 @@ -// Mixins -// -------------------------- - -@mixin fa-icon() { - display: inline-block; - font: normal normal normal #{$fa-font-size-base}/#{$fa-line-height-base} FontAwesome; // shortening font declaration - font-size: inherit; // can't have font-size inherit on line above, so need to override - text-rendering: auto; // optimizelegibility throws things off #1094 - -webkit-font-smoothing: antialiased; - -moz-osx-font-smoothing: grayscale; - -} - -@mixin fa-icon-rotate($degrees, $rotation) { - -ms-filter: "progid:DXImageTransform.Microsoft.BasicImage(rotation=#{$rotation})"; - -webkit-transform: rotate($degrees); - -ms-transform: rotate($degrees); - transform: rotate($degrees); -} - -@mixin fa-icon-flip($horiz, $vert, $rotation) { - -ms-filter: "progid:DXImageTransform.Microsoft.BasicImage(rotation=#{$rotation}, mirror=1)"; - -webkit-transform: scale($horiz, $vert); - -ms-transform: scale($horiz, $vert); - transform: scale($horiz, $vert); -} - - -// Only display content to screen readers. A la Bootstrap 4. -// -// See: http://a11yproject.com/posts/how-to-hide-content/ - -@mixin sr-only { - position: absolute; - width: 1px; - height: 1px; - padding: 0; - margin: -1px; - overflow: hidden; - clip: rect(0,0,0,0); - border: 0; -} - -// Use in conjunction with .sr-only to only display content when it's focused. -// -// Useful for "Skip to main content" links; see http://www.w3.org/TR/2013/NOTE-WCAG20-TECHS-20130905/G1 -// -// Credit: HTML5 Boilerplate - -@mixin sr-only-focusable { - &:active, - &:focus { - position: static; - width: auto; - height: auto; - margin: 0; - overflow: visible; - clip: auto; - } -} diff --git a/site/_sass/vendor/font-awesome/_path.scss b/site/_sass/vendor/font-awesome/_path.scss deleted file mode 100644 index bb457c23a8e..00000000000 --- a/site/_sass/vendor/font-awesome/_path.scss +++ /dev/null @@ -1,15 +0,0 @@ -/* FONT PATH - * -------------------------- */ - -@font-face { - font-family: 'FontAwesome'; - src: url('#{$fa-font-path}/fontawesome-webfont.eot?v=#{$fa-version}'); - src: url('#{$fa-font-path}/fontawesome-webfont.eot?#iefix&v=#{$fa-version}') format('embedded-opentype'), - url('#{$fa-font-path}/fontawesome-webfont.woff2?v=#{$fa-version}') format('woff2'), - url('#{$fa-font-path}/fontawesome-webfont.woff?v=#{$fa-version}') format('woff'), - url('#{$fa-font-path}/fontawesome-webfont.ttf?v=#{$fa-version}') format('truetype'), - url('#{$fa-font-path}/fontawesome-webfont.svg?v=#{$fa-version}#fontawesomeregular') format('svg'); -// src: url('#{$fa-font-path}/FontAwesome.otf') format('opentype'); // used when developing fonts - font-weight: normal; - font-style: normal; -} diff --git a/site/_sass/vendor/font-awesome/_rotated-flipped.scss b/site/_sass/vendor/font-awesome/_rotated-flipped.scss deleted file mode 100644 index a3558fd09ca..00000000000 --- a/site/_sass/vendor/font-awesome/_rotated-flipped.scss +++ /dev/null @@ -1,20 +0,0 @@ -// Rotated & Flipped Icons -// ------------------------- - -.#{$fa-css-prefix}-rotate-90 { @include fa-icon-rotate(90deg, 1); } -.#{$fa-css-prefix}-rotate-180 { @include fa-icon-rotate(180deg, 2); } -.#{$fa-css-prefix}-rotate-270 { @include fa-icon-rotate(270deg, 3); } - -.#{$fa-css-prefix}-flip-horizontal { @include fa-icon-flip(-1, 1, 0); } -.#{$fa-css-prefix}-flip-vertical { @include fa-icon-flip(1, -1, 2); } - -// Hook for IE8-9 -// ------------------------- - -:root .#{$fa-css-prefix}-rotate-90, -:root .#{$fa-css-prefix}-rotate-180, -:root .#{$fa-css-prefix}-rotate-270, -:root .#{$fa-css-prefix}-flip-horizontal, -:root .#{$fa-css-prefix}-flip-vertical { - filter: none; -} diff --git a/site/_sass/vendor/font-awesome/_screen-reader.scss b/site/_sass/vendor/font-awesome/_screen-reader.scss deleted file mode 100644 index 637426f0da6..00000000000 --- a/site/_sass/vendor/font-awesome/_screen-reader.scss +++ /dev/null @@ -1,5 +0,0 @@ -// Screen Readers -// ------------------------- - -.sr-only { @include sr-only(); } -.sr-only-focusable { @include sr-only-focusable(); } diff --git a/site/_sass/vendor/font-awesome/_stacked.scss b/site/_sass/vendor/font-awesome/_stacked.scss deleted file mode 100644 index aef7403660c..00000000000 --- a/site/_sass/vendor/font-awesome/_stacked.scss +++ /dev/null @@ -1,20 +0,0 @@ -// Stacked Icons -// ------------------------- - -.#{$fa-css-prefix}-stack { - position: relative; - display: inline-block; - width: 2em; - height: 2em; - line-height: 2em; - vertical-align: middle; -} -.#{$fa-css-prefix}-stack-1x, .#{$fa-css-prefix}-stack-2x { - position: absolute; - left: 0; - width: 100%; - text-align: center; -} -.#{$fa-css-prefix}-stack-1x { line-height: inherit; } -.#{$fa-css-prefix}-stack-2x { font-size: 2em; } -.#{$fa-css-prefix}-inverse { color: $fa-inverse; } diff --git a/site/_sass/vendor/font-awesome/_variables.scss b/site/_sass/vendor/font-awesome/_variables.scss deleted file mode 100644 index 498fc4a087c..00000000000 --- a/site/_sass/vendor/font-awesome/_variables.scss +++ /dev/null @@ -1,800 +0,0 @@ -// Variables -// -------------------------- - -$fa-font-path: "../fonts" !default; -$fa-font-size-base: 14px !default; -$fa-line-height-base: 1 !default; -//$fa-font-path: "//netdna.bootstrapcdn.com/font-awesome/4.7.0/fonts" !default; // for referencing Bootstrap CDN font files directly -$fa-css-prefix: fa !default; -$fa-version: "4.7.0" !default; -$fa-border-color: #eee !default; -$fa-inverse: #fff !default; -$fa-li-width: (30em / 14) !default; - -$fa-var-500px: "\f26e"; -$fa-var-address-book: "\f2b9"; -$fa-var-address-book-o: "\f2ba"; -$fa-var-address-card: "\f2bb"; -$fa-var-address-card-o: "\f2bc"; -$fa-var-adjust: "\f042"; -$fa-var-adn: "\f170"; -$fa-var-align-center: "\f037"; -$fa-var-align-justify: "\f039"; -$fa-var-align-left: "\f036"; -$fa-var-align-right: "\f038"; -$fa-var-amazon: "\f270"; -$fa-var-ambulance: "\f0f9"; -$fa-var-american-sign-language-interpreting: "\f2a3"; -$fa-var-anchor: "\f13d"; -$fa-var-android: "\f17b"; -$fa-var-angellist: "\f209"; -$fa-var-angle-double-down: "\f103"; -$fa-var-angle-double-left: "\f100"; -$fa-var-angle-double-right: "\f101"; -$fa-var-angle-double-up: "\f102"; -$fa-var-angle-down: "\f107"; -$fa-var-angle-left: "\f104"; -$fa-var-angle-right: "\f105"; -$fa-var-angle-up: "\f106"; -$fa-var-apple: "\f179"; -$fa-var-archive: "\f187"; -$fa-var-area-chart: "\f1fe"; -$fa-var-arrow-circle-down: "\f0ab"; -$fa-var-arrow-circle-left: "\f0a8"; -$fa-var-arrow-circle-o-down: "\f01a"; -$fa-var-arrow-circle-o-left: "\f190"; -$fa-var-arrow-circle-o-right: "\f18e"; -$fa-var-arrow-circle-o-up: "\f01b"; -$fa-var-arrow-circle-right: "\f0a9"; -$fa-var-arrow-circle-up: "\f0aa"; -$fa-var-arrow-down: "\f063"; -$fa-var-arrow-left: "\f060"; -$fa-var-arrow-right: "\f061"; -$fa-var-arrow-up: "\f062"; -$fa-var-arrows: "\f047"; -$fa-var-arrows-alt: "\f0b2"; -$fa-var-arrows-h: "\f07e"; -$fa-var-arrows-v: "\f07d"; -$fa-var-asl-interpreting: "\f2a3"; -$fa-var-assistive-listening-systems: "\f2a2"; -$fa-var-asterisk: "\f069"; -$fa-var-at: "\f1fa"; -$fa-var-audio-description: "\f29e"; -$fa-var-automobile: "\f1b9"; -$fa-var-backward: "\f04a"; -$fa-var-balance-scale: "\f24e"; -$fa-var-ban: "\f05e"; -$fa-var-bandcamp: "\f2d5"; -$fa-var-bank: "\f19c"; -$fa-var-bar-chart: "\f080"; -$fa-var-bar-chart-o: "\f080"; -$fa-var-barcode: "\f02a"; -$fa-var-bars: "\f0c9"; -$fa-var-bath: "\f2cd"; -$fa-var-bathtub: "\f2cd"; -$fa-var-battery: "\f240"; -$fa-var-battery-0: "\f244"; -$fa-var-battery-1: "\f243"; -$fa-var-battery-2: "\f242"; -$fa-var-battery-3: "\f241"; -$fa-var-battery-4: "\f240"; -$fa-var-battery-empty: "\f244"; -$fa-var-battery-full: "\f240"; -$fa-var-battery-half: "\f242"; -$fa-var-battery-quarter: "\f243"; -$fa-var-battery-three-quarters: "\f241"; -$fa-var-bed: "\f236"; -$fa-var-beer: "\f0fc"; -$fa-var-behance: "\f1b4"; -$fa-var-behance-square: "\f1b5"; -$fa-var-bell: "\f0f3"; -$fa-var-bell-o: "\f0a2"; -$fa-var-bell-slash: "\f1f6"; -$fa-var-bell-slash-o: "\f1f7"; -$fa-var-bicycle: "\f206"; -$fa-var-binoculars: "\f1e5"; -$fa-var-birthday-cake: "\f1fd"; -$fa-var-bitbucket: "\f171"; -$fa-var-bitbucket-square: "\f172"; -$fa-var-bitcoin: "\f15a"; -$fa-var-black-tie: "\f27e"; -$fa-var-blind: "\f29d"; -$fa-var-bluetooth: "\f293"; -$fa-var-bluetooth-b: "\f294"; -$fa-var-bold: "\f032"; -$fa-var-bolt: "\f0e7"; -$fa-var-bomb: "\f1e2"; -$fa-var-book: "\f02d"; -$fa-var-bookmark: "\f02e"; -$fa-var-bookmark-o: "\f097"; -$fa-var-braille: "\f2a1"; -$fa-var-briefcase: "\f0b1"; -$fa-var-btc: "\f15a"; -$fa-var-bug: "\f188"; -$fa-var-building: "\f1ad"; -$fa-var-building-o: "\f0f7"; -$fa-var-bullhorn: "\f0a1"; -$fa-var-bullseye: "\f140"; -$fa-var-bus: "\f207"; -$fa-var-buysellads: "\f20d"; -$fa-var-cab: "\f1ba"; -$fa-var-calculator: "\f1ec"; -$fa-var-calendar: "\f073"; -$fa-var-calendar-check-o: "\f274"; -$fa-var-calendar-minus-o: "\f272"; -$fa-var-calendar-o: "\f133"; -$fa-var-calendar-plus-o: "\f271"; -$fa-var-calendar-times-o: "\f273"; -$fa-var-camera: "\f030"; -$fa-var-camera-retro: "\f083"; -$fa-var-car: "\f1b9"; -$fa-var-caret-down: "\f0d7"; -$fa-var-caret-left: "\f0d9"; -$fa-var-caret-right: "\f0da"; -$fa-var-caret-square-o-down: "\f150"; -$fa-var-caret-square-o-left: "\f191"; -$fa-var-caret-square-o-right: "\f152"; -$fa-var-caret-square-o-up: "\f151"; -$fa-var-caret-up: "\f0d8"; -$fa-var-cart-arrow-down: "\f218"; -$fa-var-cart-plus: "\f217"; -$fa-var-cc: "\f20a"; -$fa-var-cc-amex: "\f1f3"; -$fa-var-cc-diners-club: "\f24c"; -$fa-var-cc-discover: "\f1f2"; -$fa-var-cc-jcb: "\f24b"; -$fa-var-cc-mastercard: "\f1f1"; -$fa-var-cc-paypal: "\f1f4"; -$fa-var-cc-stripe: "\f1f5"; -$fa-var-cc-visa: "\f1f0"; -$fa-var-certificate: "\f0a3"; -$fa-var-chain: "\f0c1"; -$fa-var-chain-broken: "\f127"; -$fa-var-check: "\f00c"; -$fa-var-check-circle: "\f058"; -$fa-var-check-circle-o: "\f05d"; -$fa-var-check-square: "\f14a"; -$fa-var-check-square-o: "\f046"; -$fa-var-chevron-circle-down: "\f13a"; -$fa-var-chevron-circle-left: "\f137"; -$fa-var-chevron-circle-right: "\f138"; -$fa-var-chevron-circle-up: "\f139"; -$fa-var-chevron-down: "\f078"; -$fa-var-chevron-left: "\f053"; -$fa-var-chevron-right: "\f054"; -$fa-var-chevron-up: "\f077"; -$fa-var-child: "\f1ae"; -$fa-var-chrome: "\f268"; -$fa-var-circle: "\f111"; -$fa-var-circle-o: "\f10c"; -$fa-var-circle-o-notch: "\f1ce"; -$fa-var-circle-thin: "\f1db"; -$fa-var-clipboard: "\f0ea"; -$fa-var-clock-o: "\f017"; -$fa-var-clone: "\f24d"; -$fa-var-close: "\f00d"; -$fa-var-cloud: "\f0c2"; -$fa-var-cloud-download: "\f0ed"; -$fa-var-cloud-upload: "\f0ee"; -$fa-var-cny: "\f157"; -$fa-var-code: "\f121"; -$fa-var-code-fork: "\f126"; -$fa-var-codepen: "\f1cb"; -$fa-var-codiepie: "\f284"; -$fa-var-coffee: "\f0f4"; -$fa-var-cog: "\f013"; -$fa-var-cogs: "\f085"; -$fa-var-columns: "\f0db"; -$fa-var-comment: "\f075"; -$fa-var-comment-o: "\f0e5"; -$fa-var-commenting: "\f27a"; -$fa-var-commenting-o: "\f27b"; -$fa-var-comments: "\f086"; -$fa-var-comments-o: "\f0e6"; -$fa-var-compass: "\f14e"; -$fa-var-compress: "\f066"; -$fa-var-connectdevelop: "\f20e"; -$fa-var-contao: "\f26d"; -$fa-var-copy: "\f0c5"; -$fa-var-copyright: "\f1f9"; -$fa-var-creative-commons: "\f25e"; -$fa-var-credit-card: "\f09d"; -$fa-var-credit-card-alt: "\f283"; -$fa-var-crop: "\f125"; -$fa-var-crosshairs: "\f05b"; -$fa-var-css3: "\f13c"; -$fa-var-cube: "\f1b2"; -$fa-var-cubes: "\f1b3"; -$fa-var-cut: "\f0c4"; -$fa-var-cutlery: "\f0f5"; -$fa-var-dashboard: "\f0e4"; -$fa-var-dashcube: "\f210"; -$fa-var-database: "\f1c0"; -$fa-var-deaf: "\f2a4"; -$fa-var-deafness: "\f2a4"; -$fa-var-dedent: "\f03b"; -$fa-var-delicious: "\f1a5"; -$fa-var-desktop: "\f108"; -$fa-var-deviantart: "\f1bd"; -$fa-var-diamond: "\f219"; -$fa-var-digg: "\f1a6"; -$fa-var-dollar: "\f155"; -$fa-var-dot-circle-o: "\f192"; -$fa-var-download: "\f019"; -$fa-var-dribbble: "\f17d"; -$fa-var-drivers-license: "\f2c2"; -$fa-var-drivers-license-o: "\f2c3"; -$fa-var-dropbox: "\f16b"; -$fa-var-drupal: "\f1a9"; -$fa-var-edge: "\f282"; -$fa-var-edit: "\f044"; -$fa-var-eercast: "\f2da"; -$fa-var-eject: "\f052"; -$fa-var-ellipsis-h: "\f141"; -$fa-var-ellipsis-v: "\f142"; -$fa-var-empire: "\f1d1"; -$fa-var-envelope: "\f0e0"; -$fa-var-envelope-o: "\f003"; -$fa-var-envelope-open: "\f2b6"; -$fa-var-envelope-open-o: "\f2b7"; -$fa-var-envelope-square: "\f199"; -$fa-var-envira: "\f299"; -$fa-var-eraser: "\f12d"; -$fa-var-etsy: "\f2d7"; -$fa-var-eur: "\f153"; -$fa-var-euro: "\f153"; -$fa-var-exchange: "\f0ec"; -$fa-var-exclamation: "\f12a"; -$fa-var-exclamation-circle: "\f06a"; -$fa-var-exclamation-triangle: "\f071"; -$fa-var-expand: "\f065"; -$fa-var-expeditedssl: "\f23e"; -$fa-var-external-link: "\f08e"; -$fa-var-external-link-square: "\f14c"; -$fa-var-eye: "\f06e"; -$fa-var-eye-slash: "\f070"; -$fa-var-eyedropper: "\f1fb"; -$fa-var-fa: "\f2b4"; -$fa-var-facebook: "\f09a"; -$fa-var-facebook-f: "\f09a"; -$fa-var-facebook-official: "\f230"; -$fa-var-facebook-square: "\f082"; -$fa-var-fast-backward: "\f049"; -$fa-var-fast-forward: "\f050"; -$fa-var-fax: "\f1ac"; -$fa-var-feed: "\f09e"; -$fa-var-female: "\f182"; -$fa-var-fighter-jet: "\f0fb"; -$fa-var-file: "\f15b"; -$fa-var-file-archive-o: "\f1c6"; -$fa-var-file-audio-o: "\f1c7"; -$fa-var-file-code-o: "\f1c9"; -$fa-var-file-excel-o: "\f1c3"; -$fa-var-file-image-o: "\f1c5"; -$fa-var-file-movie-o: "\f1c8"; -$fa-var-file-o: "\f016"; -$fa-var-file-pdf-o: "\f1c1"; -$fa-var-file-photo-o: "\f1c5"; -$fa-var-file-picture-o: "\f1c5"; -$fa-var-file-powerpoint-o: "\f1c4"; -$fa-var-file-sound-o: "\f1c7"; -$fa-var-file-text: "\f15c"; -$fa-var-file-text-o: "\f0f6"; -$fa-var-file-video-o: "\f1c8"; -$fa-var-file-word-o: "\f1c2"; -$fa-var-file-zip-o: "\f1c6"; -$fa-var-files-o: "\f0c5"; -$fa-var-film: "\f008"; -$fa-var-filter: "\f0b0"; -$fa-var-fire: "\f06d"; -$fa-var-fire-extinguisher: "\f134"; -$fa-var-firefox: "\f269"; -$fa-var-first-order: "\f2b0"; -$fa-var-flag: "\f024"; -$fa-var-flag-checkered: "\f11e"; -$fa-var-flag-o: "\f11d"; -$fa-var-flash: "\f0e7"; -$fa-var-flask: "\f0c3"; -$fa-var-flickr: "\f16e"; -$fa-var-floppy-o: "\f0c7"; -$fa-var-folder: "\f07b"; -$fa-var-folder-o: "\f114"; -$fa-var-folder-open: "\f07c"; -$fa-var-folder-open-o: "\f115"; -$fa-var-font: "\f031"; -$fa-var-font-awesome: "\f2b4"; -$fa-var-fonticons: "\f280"; -$fa-var-fort-awesome: "\f286"; -$fa-var-forumbee: "\f211"; -$fa-var-forward: "\f04e"; -$fa-var-foursquare: "\f180"; -$fa-var-free-code-camp: "\f2c5"; -$fa-var-frown-o: "\f119"; -$fa-var-futbol-o: "\f1e3"; -$fa-var-gamepad: "\f11b"; -$fa-var-gavel: "\f0e3"; -$fa-var-gbp: "\f154"; -$fa-var-ge: "\f1d1"; -$fa-var-gear: "\f013"; -$fa-var-gears: "\f085"; -$fa-var-genderless: "\f22d"; -$fa-var-get-pocket: "\f265"; -$fa-var-gg: "\f260"; -$fa-var-gg-circle: "\f261"; -$fa-var-gift: "\f06b"; -$fa-var-git: "\f1d3"; -$fa-var-git-square: "\f1d2"; -$fa-var-github: "\f09b"; -$fa-var-github-alt: "\f113"; -$fa-var-github-square: "\f092"; -$fa-var-gitlab: "\f296"; -$fa-var-gittip: "\f184"; -$fa-var-glass: "\f000"; -$fa-var-glide: "\f2a5"; -$fa-var-glide-g: "\f2a6"; -$fa-var-globe: "\f0ac"; -$fa-var-google: "\f1a0"; -$fa-var-google-plus: "\f0d5"; -$fa-var-google-plus-circle: "\f2b3"; -$fa-var-google-plus-official: "\f2b3"; -$fa-var-google-plus-square: "\f0d4"; -$fa-var-google-wallet: "\f1ee"; -$fa-var-graduation-cap: "\f19d"; -$fa-var-gratipay: "\f184"; -$fa-var-grav: "\f2d6"; -$fa-var-group: "\f0c0"; -$fa-var-h-square: "\f0fd"; -$fa-var-hacker-news: "\f1d4"; -$fa-var-hand-grab-o: "\f255"; -$fa-var-hand-lizard-o: "\f258"; -$fa-var-hand-o-down: "\f0a7"; -$fa-var-hand-o-left: "\f0a5"; -$fa-var-hand-o-right: "\f0a4"; -$fa-var-hand-o-up: "\f0a6"; -$fa-var-hand-paper-o: "\f256"; -$fa-var-hand-peace-o: "\f25b"; -$fa-var-hand-pointer-o: "\f25a"; -$fa-var-hand-rock-o: "\f255"; -$fa-var-hand-scissors-o: "\f257"; -$fa-var-hand-spock-o: "\f259"; -$fa-var-hand-stop-o: "\f256"; -$fa-var-handshake-o: "\f2b5"; -$fa-var-hard-of-hearing: "\f2a4"; -$fa-var-hashtag: "\f292"; -$fa-var-hdd-o: "\f0a0"; -$fa-var-header: "\f1dc"; -$fa-var-headphones: "\f025"; -$fa-var-heart: "\f004"; -$fa-var-heart-o: "\f08a"; -$fa-var-heartbeat: "\f21e"; -$fa-var-history: "\f1da"; -$fa-var-home: "\f015"; -$fa-var-hospital-o: "\f0f8"; -$fa-var-hotel: "\f236"; -$fa-var-hourglass: "\f254"; -$fa-var-hourglass-1: "\f251"; -$fa-var-hourglass-2: "\f252"; -$fa-var-hourglass-3: "\f253"; -$fa-var-hourglass-end: "\f253"; -$fa-var-hourglass-half: "\f252"; -$fa-var-hourglass-o: "\f250"; -$fa-var-hourglass-start: "\f251"; -$fa-var-houzz: "\f27c"; -$fa-var-html5: "\f13b"; -$fa-var-i-cursor: "\f246"; -$fa-var-id-badge: "\f2c1"; -$fa-var-id-card: "\f2c2"; -$fa-var-id-card-o: "\f2c3"; -$fa-var-ils: "\f20b"; -$fa-var-image: "\f03e"; -$fa-var-imdb: "\f2d8"; -$fa-var-inbox: "\f01c"; -$fa-var-indent: "\f03c"; -$fa-var-industry: "\f275"; -$fa-var-info: "\f129"; -$fa-var-info-circle: "\f05a"; -$fa-var-inr: "\f156"; -$fa-var-instagram: "\f16d"; -$fa-var-institution: "\f19c"; -$fa-var-internet-explorer: "\f26b"; -$fa-var-intersex: "\f224"; -$fa-var-ioxhost: "\f208"; -$fa-var-italic: "\f033"; -$fa-var-joomla: "\f1aa"; -$fa-var-jpy: "\f157"; -$fa-var-jsfiddle: "\f1cc"; -$fa-var-key: "\f084"; -$fa-var-keyboard-o: "\f11c"; -$fa-var-krw: "\f159"; -$fa-var-language: "\f1ab"; -$fa-var-laptop: "\f109"; -$fa-var-lastfm: "\f202"; -$fa-var-lastfm-square: "\f203"; -$fa-var-leaf: "\f06c"; -$fa-var-leanpub: "\f212"; -$fa-var-legal: "\f0e3"; -$fa-var-lemon-o: "\f094"; -$fa-var-level-down: "\f149"; -$fa-var-level-up: "\f148"; -$fa-var-life-bouy: "\f1cd"; -$fa-var-life-buoy: "\f1cd"; -$fa-var-life-ring: "\f1cd"; -$fa-var-life-saver: "\f1cd"; -$fa-var-lightbulb-o: "\f0eb"; -$fa-var-line-chart: "\f201"; -$fa-var-link: "\f0c1"; -$fa-var-linkedin: "\f0e1"; -$fa-var-linkedin-square: "\f08c"; -$fa-var-linode: "\f2b8"; -$fa-var-linux: "\f17c"; -$fa-var-list: "\f03a"; -$fa-var-list-alt: "\f022"; -$fa-var-list-ol: "\f0cb"; -$fa-var-list-ul: "\f0ca"; -$fa-var-location-arrow: "\f124"; -$fa-var-lock: "\f023"; -$fa-var-long-arrow-down: "\f175"; -$fa-var-long-arrow-left: "\f177"; -$fa-var-long-arrow-right: "\f178"; -$fa-var-long-arrow-up: "\f176"; -$fa-var-low-vision: "\f2a8"; -$fa-var-magic: "\f0d0"; -$fa-var-magnet: "\f076"; -$fa-var-mail-forward: "\f064"; -$fa-var-mail-reply: "\f112"; -$fa-var-mail-reply-all: "\f122"; -$fa-var-male: "\f183"; -$fa-var-map: "\f279"; -$fa-var-map-marker: "\f041"; -$fa-var-map-o: "\f278"; -$fa-var-map-pin: "\f276"; -$fa-var-map-signs: "\f277"; -$fa-var-mars: "\f222"; -$fa-var-mars-double: "\f227"; -$fa-var-mars-stroke: "\f229"; -$fa-var-mars-stroke-h: "\f22b"; -$fa-var-mars-stroke-v: "\f22a"; -$fa-var-maxcdn: "\f136"; -$fa-var-meanpath: "\f20c"; -$fa-var-medium: "\f23a"; -$fa-var-medkit: "\f0fa"; -$fa-var-meetup: "\f2e0"; -$fa-var-meh-o: "\f11a"; -$fa-var-mercury: "\f223"; -$fa-var-microchip: "\f2db"; -$fa-var-microphone: "\f130"; -$fa-var-microphone-slash: "\f131"; -$fa-var-minus: "\f068"; -$fa-var-minus-circle: "\f056"; -$fa-var-minus-square: "\f146"; -$fa-var-minus-square-o: "\f147"; -$fa-var-mixcloud: "\f289"; -$fa-var-mobile: "\f10b"; -$fa-var-mobile-phone: "\f10b"; -$fa-var-modx: "\f285"; -$fa-var-money: "\f0d6"; -$fa-var-moon-o: "\f186"; -$fa-var-mortar-board: "\f19d"; -$fa-var-motorcycle: "\f21c"; -$fa-var-mouse-pointer: "\f245"; -$fa-var-music: "\f001"; -$fa-var-navicon: "\f0c9"; -$fa-var-neuter: "\f22c"; -$fa-var-newspaper-o: "\f1ea"; -$fa-var-object-group: "\f247"; -$fa-var-object-ungroup: "\f248"; -$fa-var-odnoklassniki: "\f263"; -$fa-var-odnoklassniki-square: "\f264"; -$fa-var-opencart: "\f23d"; -$fa-var-openid: "\f19b"; -$fa-var-opera: "\f26a"; -$fa-var-optin-monster: "\f23c"; -$fa-var-outdent: "\f03b"; -$fa-var-pagelines: "\f18c"; -$fa-var-paint-brush: "\f1fc"; -$fa-var-paper-plane: "\f1d8"; -$fa-var-paper-plane-o: "\f1d9"; -$fa-var-paperclip: "\f0c6"; -$fa-var-paragraph: "\f1dd"; -$fa-var-paste: "\f0ea"; -$fa-var-pause: "\f04c"; -$fa-var-pause-circle: "\f28b"; -$fa-var-pause-circle-o: "\f28c"; -$fa-var-paw: "\f1b0"; -$fa-var-paypal: "\f1ed"; -$fa-var-pencil: "\f040"; -$fa-var-pencil-square: "\f14b"; -$fa-var-pencil-square-o: "\f044"; -$fa-var-percent: "\f295"; -$fa-var-phone: "\f095"; -$fa-var-phone-square: "\f098"; -$fa-var-photo: "\f03e"; -$fa-var-picture-o: "\f03e"; -$fa-var-pie-chart: "\f200"; -$fa-var-pied-piper: "\f2ae"; -$fa-var-pied-piper-alt: "\f1a8"; -$fa-var-pied-piper-pp: "\f1a7"; -$fa-var-pinterest: "\f0d2"; -$fa-var-pinterest-p: "\f231"; -$fa-var-pinterest-square: "\f0d3"; -$fa-var-plane: "\f072"; -$fa-var-play: "\f04b"; -$fa-var-play-circle: "\f144"; -$fa-var-play-circle-o: "\f01d"; -$fa-var-plug: "\f1e6"; -$fa-var-plus: "\f067"; -$fa-var-plus-circle: "\f055"; -$fa-var-plus-square: "\f0fe"; -$fa-var-plus-square-o: "\f196"; -$fa-var-podcast: "\f2ce"; -$fa-var-power-off: "\f011"; -$fa-var-print: "\f02f"; -$fa-var-product-hunt: "\f288"; -$fa-var-puzzle-piece: "\f12e"; -$fa-var-qq: "\f1d6"; -$fa-var-qrcode: "\f029"; -$fa-var-question: "\f128"; -$fa-var-question-circle: "\f059"; -$fa-var-question-circle-o: "\f29c"; -$fa-var-quora: "\f2c4"; -$fa-var-quote-left: "\f10d"; -$fa-var-quote-right: "\f10e"; -$fa-var-ra: "\f1d0"; -$fa-var-random: "\f074"; -$fa-var-ravelry: "\f2d9"; -$fa-var-rebel: "\f1d0"; -$fa-var-recycle: "\f1b8"; -$fa-var-reddit: "\f1a1"; -$fa-var-reddit-alien: "\f281"; -$fa-var-reddit-square: "\f1a2"; -$fa-var-refresh: "\f021"; -$fa-var-registered: "\f25d"; -$fa-var-remove: "\f00d"; -$fa-var-renren: "\f18b"; -$fa-var-reorder: "\f0c9"; -$fa-var-repeat: "\f01e"; -$fa-var-reply: "\f112"; -$fa-var-reply-all: "\f122"; -$fa-var-resistance: "\f1d0"; -$fa-var-retweet: "\f079"; -$fa-var-rmb: "\f157"; -$fa-var-road: "\f018"; -$fa-var-rocket: "\f135"; -$fa-var-rotate-left: "\f0e2"; -$fa-var-rotate-right: "\f01e"; -$fa-var-rouble: "\f158"; -$fa-var-rss: "\f09e"; -$fa-var-rss-square: "\f143"; -$fa-var-rub: "\f158"; -$fa-var-ruble: "\f158"; -$fa-var-rupee: "\f156"; -$fa-var-s15: "\f2cd"; -$fa-var-safari: "\f267"; -$fa-var-save: "\f0c7"; -$fa-var-scissors: "\f0c4"; -$fa-var-scribd: "\f28a"; -$fa-var-search: "\f002"; -$fa-var-search-minus: "\f010"; -$fa-var-search-plus: "\f00e"; -$fa-var-sellsy: "\f213"; -$fa-var-send: "\f1d8"; -$fa-var-send-o: "\f1d9"; -$fa-var-server: "\f233"; -$fa-var-share: "\f064"; -$fa-var-share-alt: "\f1e0"; -$fa-var-share-alt-square: "\f1e1"; -$fa-var-share-square: "\f14d"; -$fa-var-share-square-o: "\f045"; -$fa-var-shekel: "\f20b"; -$fa-var-sheqel: "\f20b"; -$fa-var-shield: "\f132"; -$fa-var-ship: "\f21a"; -$fa-var-shirtsinbulk: "\f214"; -$fa-var-shopping-bag: "\f290"; -$fa-var-shopping-basket: "\f291"; -$fa-var-shopping-cart: "\f07a"; -$fa-var-shower: "\f2cc"; -$fa-var-sign-in: "\f090"; -$fa-var-sign-language: "\f2a7"; -$fa-var-sign-out: "\f08b"; -$fa-var-signal: "\f012"; -$fa-var-signing: "\f2a7"; -$fa-var-simplybuilt: "\f215"; -$fa-var-sitemap: "\f0e8"; -$fa-var-skyatlas: "\f216"; -$fa-var-skype: "\f17e"; -$fa-var-slack: "\f198"; -$fa-var-sliders: "\f1de"; -$fa-var-slideshare: "\f1e7"; -$fa-var-smile-o: "\f118"; -$fa-var-snapchat: "\f2ab"; -$fa-var-snapchat-ghost: "\f2ac"; -$fa-var-snapchat-square: "\f2ad"; -$fa-var-snowflake-o: "\f2dc"; -$fa-var-soccer-ball-o: "\f1e3"; -$fa-var-sort: "\f0dc"; -$fa-var-sort-alpha-asc: "\f15d"; -$fa-var-sort-alpha-desc: "\f15e"; -$fa-var-sort-amount-asc: "\f160"; -$fa-var-sort-amount-desc: "\f161"; -$fa-var-sort-asc: "\f0de"; -$fa-var-sort-desc: "\f0dd"; -$fa-var-sort-down: "\f0dd"; -$fa-var-sort-numeric-asc: "\f162"; -$fa-var-sort-numeric-desc: "\f163"; -$fa-var-sort-up: "\f0de"; -$fa-var-soundcloud: "\f1be"; -$fa-var-space-shuttle: "\f197"; -$fa-var-spinner: "\f110"; -$fa-var-spoon: "\f1b1"; -$fa-var-spotify: "\f1bc"; -$fa-var-square: "\f0c8"; -$fa-var-square-o: "\f096"; -$fa-var-stack-exchange: "\f18d"; -$fa-var-stack-overflow: "\f16c"; -$fa-var-star: "\f005"; -$fa-var-star-half: "\f089"; -$fa-var-star-half-empty: "\f123"; -$fa-var-star-half-full: "\f123"; -$fa-var-star-half-o: "\f123"; -$fa-var-star-o: "\f006"; -$fa-var-steam: "\f1b6"; -$fa-var-steam-square: "\f1b7"; -$fa-var-step-backward: "\f048"; -$fa-var-step-forward: "\f051"; -$fa-var-stethoscope: "\f0f1"; -$fa-var-sticky-note: "\f249"; -$fa-var-sticky-note-o: "\f24a"; -$fa-var-stop: "\f04d"; -$fa-var-stop-circle: "\f28d"; -$fa-var-stop-circle-o: "\f28e"; -$fa-var-street-view: "\f21d"; -$fa-var-strikethrough: "\f0cc"; -$fa-var-stumbleupon: "\f1a4"; -$fa-var-stumbleupon-circle: "\f1a3"; -$fa-var-subscript: "\f12c"; -$fa-var-subway: "\f239"; -$fa-var-suitcase: "\f0f2"; -$fa-var-sun-o: "\f185"; -$fa-var-superpowers: "\f2dd"; -$fa-var-superscript: "\f12b"; -$fa-var-support: "\f1cd"; -$fa-var-table: "\f0ce"; -$fa-var-tablet: "\f10a"; -$fa-var-tachometer: "\f0e4"; -$fa-var-tag: "\f02b"; -$fa-var-tags: "\f02c"; -$fa-var-tasks: "\f0ae"; -$fa-var-taxi: "\f1ba"; -$fa-var-telegram: "\f2c6"; -$fa-var-television: "\f26c"; -$fa-var-tencent-weibo: "\f1d5"; -$fa-var-terminal: "\f120"; -$fa-var-text-height: "\f034"; -$fa-var-text-width: "\f035"; -$fa-var-th: "\f00a"; -$fa-var-th-large: "\f009"; -$fa-var-th-list: "\f00b"; -$fa-var-themeisle: "\f2b2"; -$fa-var-thermometer: "\f2c7"; -$fa-var-thermometer-0: "\f2cb"; -$fa-var-thermometer-1: "\f2ca"; -$fa-var-thermometer-2: "\f2c9"; -$fa-var-thermometer-3: "\f2c8"; -$fa-var-thermometer-4: "\f2c7"; -$fa-var-thermometer-empty: "\f2cb"; -$fa-var-thermometer-full: "\f2c7"; -$fa-var-thermometer-half: "\f2c9"; -$fa-var-thermometer-quarter: "\f2ca"; -$fa-var-thermometer-three-quarters: "\f2c8"; -$fa-var-thumb-tack: "\f08d"; -$fa-var-thumbs-down: "\f165"; -$fa-var-thumbs-o-down: "\f088"; -$fa-var-thumbs-o-up: "\f087"; -$fa-var-thumbs-up: "\f164"; -$fa-var-ticket: "\f145"; -$fa-var-times: "\f00d"; -$fa-var-times-circle: "\f057"; -$fa-var-times-circle-o: "\f05c"; -$fa-var-times-rectangle: "\f2d3"; -$fa-var-times-rectangle-o: "\f2d4"; -$fa-var-tint: "\f043"; -$fa-var-toggle-down: "\f150"; -$fa-var-toggle-left: "\f191"; -$fa-var-toggle-off: "\f204"; -$fa-var-toggle-on: "\f205"; -$fa-var-toggle-right: "\f152"; -$fa-var-toggle-up: "\f151"; -$fa-var-trademark: "\f25c"; -$fa-var-train: "\f238"; -$fa-var-transgender: "\f224"; -$fa-var-transgender-alt: "\f225"; -$fa-var-trash: "\f1f8"; -$fa-var-trash-o: "\f014"; -$fa-var-tree: "\f1bb"; -$fa-var-trello: "\f181"; -$fa-var-tripadvisor: "\f262"; -$fa-var-trophy: "\f091"; -$fa-var-truck: "\f0d1"; -$fa-var-try: "\f195"; -$fa-var-tty: "\f1e4"; -$fa-var-tumblr: "\f173"; -$fa-var-tumblr-square: "\f174"; -$fa-var-turkish-lira: "\f195"; -$fa-var-tv: "\f26c"; -$fa-var-twitch: "\f1e8"; -$fa-var-twitter: "\f099"; -$fa-var-twitter-square: "\f081"; -$fa-var-umbrella: "\f0e9"; -$fa-var-underline: "\f0cd"; -$fa-var-undo: "\f0e2"; -$fa-var-universal-access: "\f29a"; -$fa-var-university: "\f19c"; -$fa-var-unlink: "\f127"; -$fa-var-unlock: "\f09c"; -$fa-var-unlock-alt: "\f13e"; -$fa-var-unsorted: "\f0dc"; -$fa-var-upload: "\f093"; -$fa-var-usb: "\f287"; -$fa-var-usd: "\f155"; -$fa-var-user: "\f007"; -$fa-var-user-circle: "\f2bd"; -$fa-var-user-circle-o: "\f2be"; -$fa-var-user-md: "\f0f0"; -$fa-var-user-o: "\f2c0"; -$fa-var-user-plus: "\f234"; -$fa-var-user-secret: "\f21b"; -$fa-var-user-times: "\f235"; -$fa-var-users: "\f0c0"; -$fa-var-vcard: "\f2bb"; -$fa-var-vcard-o: "\f2bc"; -$fa-var-venus: "\f221"; -$fa-var-venus-double: "\f226"; -$fa-var-venus-mars: "\f228"; -$fa-var-viacoin: "\f237"; -$fa-var-viadeo: "\f2a9"; -$fa-var-viadeo-square: "\f2aa"; -$fa-var-video-camera: "\f03d"; -$fa-var-vimeo: "\f27d"; -$fa-var-vimeo-square: "\f194"; -$fa-var-vine: "\f1ca"; -$fa-var-vk: "\f189"; -$fa-var-volume-control-phone: "\f2a0"; -$fa-var-volume-down: "\f027"; -$fa-var-volume-off: "\f026"; -$fa-var-volume-up: "\f028"; -$fa-var-warning: "\f071"; -$fa-var-wechat: "\f1d7"; -$fa-var-weibo: "\f18a"; -$fa-var-weixin: "\f1d7"; -$fa-var-whatsapp: "\f232"; -$fa-var-wheelchair: "\f193"; -$fa-var-wheelchair-alt: "\f29b"; -$fa-var-wifi: "\f1eb"; -$fa-var-wikipedia-w: "\f266"; -$fa-var-window-close: "\f2d3"; -$fa-var-window-close-o: "\f2d4"; -$fa-var-window-maximize: "\f2d0"; -$fa-var-window-minimize: "\f2d1"; -$fa-var-window-restore: "\f2d2"; -$fa-var-windows: "\f17a"; -$fa-var-won: "\f159"; -$fa-var-wordpress: "\f19a"; -$fa-var-wpbeginner: "\f297"; -$fa-var-wpexplorer: "\f2de"; -$fa-var-wpforms: "\f298"; -$fa-var-wrench: "\f0ad"; -$fa-var-xing: "\f168"; -$fa-var-xing-square: "\f169"; -$fa-var-y-combinator: "\f23b"; -$fa-var-y-combinator-square: "\f1d4"; -$fa-var-yahoo: "\f19e"; -$fa-var-yc: "\f23b"; -$fa-var-yc-square: "\f1d4"; -$fa-var-yelp: "\f1e9"; -$fa-var-yen: "\f157"; -$fa-var-yoast: "\f2b1"; -$fa-var-youtube: "\f167"; -$fa-var-youtube-play: "\f16a"; -$fa-var-youtube-square: "\f166"; - diff --git a/site/_sass/vendor/font-awesome/font-awesome.scss b/site/_sass/vendor/font-awesome/font-awesome.scss deleted file mode 100644 index f1c83aaa5d2..00000000000 --- a/site/_sass/vendor/font-awesome/font-awesome.scss +++ /dev/null @@ -1,18 +0,0 @@ -/*! - * Font Awesome 4.7.0 by @davegandy - http://fontawesome.io - @fontawesome - * License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License) - */ - -@import "variables"; -@import "mixins"; -@import "path"; -@import "core"; -@import "larger"; -@import "fixed-width"; -@import "list"; -@import "bordered-pulled"; -@import "animated"; -@import "rotated-flipped"; -@import "stacked"; -@import "icons"; -@import "screen-reader"; diff --git a/site/bps/BP-20-github-workflow-for-bookkeeper-proposals.md b/site/bps/BP-20-github-workflow-for-bookkeeper-proposals.md deleted file mode 100644 index 2b5941c031e..00000000000 --- a/site/bps/BP-20-github-workflow-for-bookkeeper-proposals.md +++ /dev/null @@ -1,46 +0,0 @@ ---- -title: "BP-20: github workflow for bookkeeper proposals" -issue: https://github.com/apache/bookkeeper/761 -state: "Adopted" -release: "4.7.0" ---- - -### Motivation - -We have a good BP process for introducing enhancements, features. However, this process is not well integrated with our github review process, and the content of a BP -is not used for documentation. This proposal is to propose moving the BP workflow from ASF wiki to Github. There are a couple of reasons for making this change: - -- the ASF cwiki is disconnected from Github, and usually becomes out of date quickly. It isn't really caught up with the code changes. - Most of the content (documentation, contribution/release guides) are already in website, the ASF wiki is only used for tracking BPs and community meeting notes at this point. -- Moving BP workflow from wiki to github will leverage the same github review process as code changes. So developers are easier to review BPs and make comments. -- The BPs can eventually be used as a basis for documentation. - -### Proposed Changes - -All the BPs are maintained in `site/bps` directory. To make a bookkeeper proposal, a developer does following steps: - -1. Create an issue `BP-: [capation of bookkeeper proposal]`. E.g. `BP-1: 64 bits ledger id support`. - - Take the next available BP number from this page. - - Write a brief description about what BP is for in this issue. This issue will be the master issue for tracking the status of this BP and its implementations. - All the implementations of this BP should be listed and linked to this master issues. -1. Write the proposal for this BP. - - Make a copy of the [BP-Template](https://github.com/apache/bookkeeper/tree/master/site/bps/BP-template.md). Name the BP file as `BP--[caption-of-proposal].md`. - ```shell - $ cp site/bps/BP-template.md site/bps/BP-xyz-capation-of-proposal.md - ``` - - Fill the sections listed in the BP template. - - issue: replace `` with the issue number. - - state: "Under Discussion" - - release: leave the release to `N/A`. you can only mark a release after a BP is implemented. -1. Send a PR for this BP. Following the instructions in the pull request template. - - add `BP` label to this BP - - don't associate this PR with any release or milestone -1. You can tag committers on this RP for reviewers, or start a `[DISCUSS]` thread on Apache mailing list. If you are sending an email, please make sure that the subject - of the thread is of the format `[DISCUSS] BP-: capation of bookkeeper proposal`. -1. Once the BP is finalized, reviewed and approved by committers, the BP is accepted. The criteria for acceptance is [lazy majority](http://bookkeeper.apache.org/bylaws.html). -1. Committers merge the PR after a BP is accepted. The development for this BP moves forward with implementations. The BP should be updated if there is anything changed during - implementing it. -1. After all the implementations for a given BP are completed, a new PR should be sent for changing the state of a BP: - - state: "Adopted" - - release: set to the release that includes this BP. -1. The final PR for changing BP state will be used as the criteria for marking a BP as completed. diff --git a/site/bps/BP-21-new-api-close-inconsistencies.md b/site/bps/BP-21-new-api-close-inconsistencies.md deleted file mode 100644 index 4bc1ff8f0eb..00000000000 --- a/site/bps/BP-21-new-api-close-inconsistencies.md +++ /dev/null @@ -1,65 +0,0 @@ ---- -title: "BP-21: New API close inconsistencies" -issue: https://github.com/apache/bookkeeper/issues/789 -state: "Rejected" -release: "4.6.0" ---- - -Rejected due to lack of agreement that the issues raised in the motivation are valid. - -### Motivation - -The [Handle](http://bookkeeper.apache.org/docs/latest/api/javadoc/org/apache/bookkeeper/client/api/Handle.html) interface provides two methods, #asyncClose and #close (overriding AutoCloseable). - -#close is implemented in both [ReadHandle](http://bookkeeper.apache.org/docs/latest/api/javadoc/org/apache/bookkeeper/client/api/ReadHandle.html) and [WriteHandle](http://bookkeeper.apache.org/docs/latest/api/javadoc/org/apache/bookkeeper/client/api/WriteHandle.html). - -1. The implementations in ReadHandle and WriteHandle do vastly different things. In ReadHandle, #close unregisters listeners from the ledger manager. This is local resource cleanup, which is in line with what AutoCloseable is designed for. In WriteHandle, #close calls #asyncClose which writes the lastAddConfirmed to the LedgerMetadata. This violates the principle of separation of concerns, and overloads the meaning of the term "close". - -2. #asyncClose is defined in Handle, but it only has any meaning in the WriteHandle. In ReadHandle, closing only cleans up local resources, there's no network nor disk I/O involved. The implementation directly calls the callback. It's only in WriteHandle that asyncClose has any meaning, and here it is completely different to in ReadHandle. - -3. The name #asyncClose is inconsistent with every other method on the new api Handles (append, read, readLastAddConfirmed, etc). - -4. #close is part of AutoClosable, so its not unreasonable for it to be used in a try-with-resource block. This means that a ledger closure (i.e. distributed state mutation) could be triggered by an exception within the block. This is nasty. - -Overloading the meaning of the term "close" is very problematic on its own. Closing a WriteHandle is a very important part of the BookKeeper protocol, so it should at least have it's own _verb_. I propose that we stop using "closing a ledger" to describe setting the last entry of a ledger, and instead call it "sealing a ledger". - -### Public Interfaces - -1. Remove Handle#asyncClose -2. Add new method WriteHandle#seal. -``` -class WriteHandle { - CompletableFuture seal(); -} -``` - -### Proposed Changes - -The proposed change remove asyncClose from all handles and replaces it with a async #seal method on WriteHandle. WriteHandle will still have a #close method for cleaning up local resources. - -The proposed usage would look like: - -``` -try (WriteHandle writer = bk.newCreateLedgerOp().withPassword("bleh".getBytes()).execute().get()) { - for (int i = 0; i < 100; i++) { - writer.append("foobar".getBytes()); - } - writer.seal().get(); // no more entries can be added -} -``` - -_What if the user forgets to call #seal before closing the ledger?_ - -The ledger is left unsealed. Readers will not read past the end of the unsealed ledger, and they will either try to recover the ledger or wait forever. In both cases, the consistency of the data is guaranteed as the writer would only acknowledge writes which have hit the full ack quorum, which will always be picked up by recovery. If the writer had made any writes that were not acknowledged, it would have halted and not moved onto writing a new ledger. - -### Compatibility, Deprecation, and Migration Plan - -None, this only affects the new api. - -### Test Plan - -The current tests for #asyncClose will be migrated to use #seal(). - -### Rejected Alternatives - -The alternative is how it is now. The movitation section describes the problem with this. diff --git a/site/bps/BP-22-separate-closing-ledgers-from-opening-ledgers.md b/site/bps/BP-22-separate-closing-ledgers-from-opening-ledgers.md deleted file mode 100644 index f6b8c294abc..00000000000 --- a/site/bps/BP-22-separate-closing-ledgers-from-opening-ledgers.md +++ /dev/null @@ -1,99 +0,0 @@ ---- -title: "BP-22: Separate closing ledgers from opening ledgers" -issue: https://github.com/apache/bookkeeper/794 -state: "Rejected" -release: "4.7.0" ---- - -Rejected due to lack of agreement that the issues raised in the motivation are valid. - -### Motivation - -In the beginning there was openLedger. Opening a ledger meant also figuring out what the last entry of a ledger should be, and writing it to ZooKeeper. For a long time this was the only way to read a ledger. If a writer was writing to the ledger, then anything it wrote after this point would be lost (fencing was added later). The open operation was the natural place to put recovery, as at this point it was only possible to read a recovered ledger. - -openLedgerNoRecovery was added in 2011. This allowed users to read from a ledger as it was being written to, which opened up a bunch of tailing uses cases. Recovery was still the default, because that was what it had always been, and tailing was still considered a secondary usecase. If the user wanted to skip recovery, they'd have to explicitly call the no recovery method. - -Then the new API arrived. In the new API, tailing is the primary read use case, and recovery has been demoted to a boolean flag on a builder for the open operation. The user is for the most part unaware of recovery. - -However, recovery is still one of the most important aspects of BookKeeper. It is the mechanism on which our Total Order Atomic Broadcast guarantees are built. It deserves to be a bit more prominent than a boolean flag in a builder. It also doesn't help that the terminology is inconsistent. The flag is called withRecovery, while the to check if recovery is needed, we call isClosed. Closed itself is ambiguous because it may refer to the local handle, or it may refer to the state of the ledger. - -As tailing is now the primary usecase, we expect that if a writer fails, then whichever node takes over as writer already has a non-recovered ReadHandle open. It would be nice to be able to continue using this Handle to read to the end of the ledger. - -### Public Interfaces - -I proposes the removal of OpenOpBuilder#withRecovery() - -A new method on BookKeeper: -``` -interface BookKeeper { - - ... - - CompletableFuture seal(ReadHandle handle); -} -``` - -### Proposed Changes - -With the proposed interface changes, every ReadHandle will be opened without recovery. For the tailing usecase, usage will look like. - -```java -ReadHandle reader = bk.newOpenLedgerOp().withLedgerId(X).execute().get(); -long lastReadEntry = -1; -while (!leader) { - long lac = reader.getLastAddConfirmed(); - if (lac > lastReadEntry) { - LedgerEntries entries = reader.read(lastReadEntry+1, lac).get(); - doSomethingWithEntries(entries); - lastReadEntry = lac; - } -} -assert (leader); -bk.seal(reader).get(); -long lac = reader.readLastAddConfirmed().get(); -if (lac > lastReadEntry) { - LedgerEntries entries = reader.read(lastReadEntry+1, lac).get(); - doSomethingWithEntries(entries); -} -WriteHandle writer = bk.newCreateLedgerOp().execute().get(); -``` - -Constrast this with how it is with the current recovery on open mechanism. - -``` -ReadHandle reader = bk.newOpenLedgerOp().withLedgerId(X).execute().get(); -long lastReadEntry = -1; -while (!leader) { - long lac = reader.getLastAddConfirmed(); - if (lac > lastReadEntry) { - LedgerEntries entries = reader.read(lastReadEntry+1, lac).get(); - doSomethingWithEntries(entries); - lastReadEntry = lac; - } -} -assert (leader); -reader.close(); -reader = bk.newOpenLedgerOp().withLedgerId(reader.getId()).withRecovery(true).execute.get(); -long lac = reader.readLastAddConfirmed().get(); -if (lac > lastReadEntry) { - LedgerEntries entries = reader.read(lastReadEntry+1, lac).get(); - doSomethingWithEntries(entries); -} -WriteHandle writer = bk.newCreateLedgerOp().execute().get(); -``` - -The second one is more code, you need to remember to close the previous handle, and the intent of the operation is less clear. - -### Compatibility, Deprecation, and Migration Plan - -This change is only on the new API, so there's no promise of compatibility. - -### Test Plan - -This change replaces #withRecovery() with #seal(), so anyplace withRecovery was tested, should be replaced with #seal(). - -### Rejected Alternatives - -- ReadHandle#seal: Rejected as ReadHandle should be side effect free -- OpenOpBuilder#withRecovery(true): Rejected as we want tailing to be default usecase. -- ReadHandle#forceClosed() or BookKeeper#forceClosed(ReadHandle): Rejected as unclear what the state of handle would be after (has ReadHandle#close been called). diff --git a/site/bps/BP-25-MovingChecksumToProto.md b/site/bps/BP-25-MovingChecksumToProto.md deleted file mode 100644 index 4471bc83432..00000000000 --- a/site/bps/BP-25-MovingChecksumToProto.md +++ /dev/null @@ -1,27 +0,0 @@ ---- -title: "BP-25: MovingChecksumToProto--Refactor the checksum part of bookkeeper" -issue: https://github.com/apache/bookkeeper/issues/1007 -state: "Adopted" -release: "4.7.0" ---- - -### Motivation - -Current the checksum implementation is in client module while the checksum semantic is more close to protocol. Moreover, moving the checksum implementation to protocol will avoid server module's dependency to client module when doing checksum in server side. - -### Public Interfaces - -An internal refactor not affecting public interfaces. - -### Proposed Changes - -Move the DigestManager and related classes to proto module - -### Compatibility, Deprecation, and Migration Plan -N/A - -### Test Plan -The original all tests should work as before. - -### Rejected Alternatives -N/A diff --git a/site/bps/BP-26-move-distributedlog-core-library.md b/site/bps/BP-26-move-distributedlog-core-library.md deleted file mode 100644 index 6cf646ab1d8..00000000000 --- a/site/bps/BP-26-move-distributedlog-core-library.md +++ /dev/null @@ -1,62 +0,0 @@ ---- -title: "BP-26: Move distributedlog library as part of bookkeeper" -issue: https://github.com/apache/bookkeeper/1024 -state: 'Accepted' -release: "N/A" ---- - -### Motivation - -DistributedLog is an extension of Apache BookKeeper, which offers *reopenable* log streams as its storage primitives. -It is tightly built over bookkeeper ledgers, and provides an easier-to-use abstraction and api to use. Applications -can use *named* log streams rather than *numbered* ledgers to store their data. For example, users can use log streams -as files to storge objects, checkpoints and other more general filesystem related use cases. - -Moving the distributedlog core library as part of bookkeeper would have following benefits: - -- It provides more generic "reopenable" log abstraction. It lowers the barrier for people to use bookkeeper to store - data, and bring in more use cases into bookkeeper ecosystem. -- Using ledgers to build continous log stream has been a pattern that been reimplemented multiple times at multiple places, - from older projects like HDFS namenode log manager, Hedwig to the newer projects like DistributedLog and Pulsar. -- Most of the distributedlog usage is using the distributedlog library which only depends Apache BookKeeper and there is no - additional components introduced. To simplify those usages, it is better to release distributedlog library along with - bookkeeper. It provides a better integration and release procedure. - -This proposal proposes "moving the distributedlog library code base as part of bookkeeper and continuing the library -development in bookkeeper". - -### Public Interfaces - -This is a new library moved in bookkeeper. It will *NOT* touch any existing bookkeeper modules and ledger api. - -### Proposed Changes - -This proposal will *ONLY* move following library-only modules from distributedlog repo: - -- distributedlog-core: the log stream library that build over bookkeeper ledgr api. It doesn't introduce any service components. Library only. -- distributedlog-io/dlfs: A hdfs filesystem api wrapper over the log stream api, to provide filesystem-like usage over bookkeeper. - -This proposal will *NOT* move other service components like "distributedlog-proxy". - -The steps to make this change are described as following: - -- the proposed modules (`distributedlog-core` and `distributedlog-io/dlfs`) will be moved under `stream/distributedlog` directory at apache bookkeeper repo. -- a new "stream" profile will be added to the root `pom.xml` file. The distributedlog module will only be build when "-Pstream" is specified - in the maven build command. This allows users who only use ledger api skip building distributedlog module. -- the distributedlog api, javadoc api and some tutorials will be integrated with current bookkeeper website to provide integrated experiences - when users browse bookkeeper website. - -### Compatibility, Deprecation, and Migration Plan - -This doesn't change existing modules or api. so no compatibility, deprecation and migration plan for bookkeeper users. - -For distributedlog users, the distributedlog library will begin release under groupId `org.apache.bookkeeper` instead of `org.apache.distributedlog`. -API documentation will be updated to reflect this. - -### Test Plan - -N/A - -### Rejected Alternatives - -N/A diff --git a/site/bps/BP-27-new-bookkeeper-cli.md b/site/bps/BP-27-new-bookkeeper-cli.md deleted file mode 100644 index d8786f4c54e..00000000000 --- a/site/bps/BP-27-new-bookkeeper-cli.md +++ /dev/null @@ -1,109 +0,0 @@ ---- -title: "BP-27: New BookKeeper CLI" -issue: https://github.com/apache/bookkeeper/1000 -state: 'Under Discussion' -release: "N/A" ---- - -### Motivation - -`BookieShell` is the current bookkeeper cli for interacting and operating a bookkeeper cluster. However, this class is getting bigger with more commands added to it. It is facing a few problems for maintenance and extensibility. - -- All commands sit in one gaint shell class. It is hard to tell if a command is used for managing a bookie only or if a command is used for managing a cluster. -- Lack of unit tests. This class has very few test coverage. Most of the commands (introduced in early days) don't have a unit test. -- Lack of extensibility. If a new function component (for example, dlog) is introduced, it is a bit hard to extend this CLI to have commands for new function component. - -All these problems lead to the proposal here. This proposal is to propose refactoring/redesigning the bookkeeper CLI to allow better managebility for maintenance, better test coverage and better extensibility for new function components. - -### Public Interfaces - -This proposal will not change existing `BookieShell`. All functionalities will remain as same when using `bin/bookkeeper shell`. -Instead a new module `bookkeeper-tools` will be introduced for developing the new BookKeeper CLI and a new script `bin/bookkeeper-cli` for executing CLI commands. - -The new bookkeeper CLI follows the pattern that pulsar-admin is using. The CLI commandline format would be: - -``` -$ bookkeeper-cli [general options] [options of command] -``` - -#### CommandGroup and Command - -`` and `` are introduced for categorizing the commands into groups. So the commands within same group have same operation scope (e.g. whether a command is applied to a cluster or a command is applied to a bookie). - -When a new function component is introduced, all its related commands can be managed in its own command group and register the group to CLI. This would allow flexible extensibility and make maintenance easier and clearer. - -The proposed command groups are: - -- "cluster": commands that operate on a cluster -- "bookie": commands that operate on a single bookie -- "metadata": commands that operate with metadata store -- "client": commands that use a bookkeeper client for interacting with a cluster. - -Example Outputs for the new BookKeeper CLI: - -- Show all command groups: `bookkeeper-cli --help` - -``` -Usage: bookkeeper-cli [options] [command] [command options] - Options: - -c, --conf - Bookie Configuration File - -h, --help - Show this help message - Default: false - Commands: - bookie Commands on operating a single bookie - Usage: bookie [options] - - client Commands that interact with a cluster - Usage: client [options] - - cluster Commands that operate a cluster - Usage: cluster [options] - - metadata Commands that interact with metadata storage - Usage: metadata [options] -``` -- Show commands under `cluster`: `bookkeeper-cli cluster --help` -``` -Usage: bookie-shell cluster [options] [command] [command options] - Commands: - listbookies List the bookies, which are running as either readwrite or readonly mode. - Usage: listbookies [options] - Options: - -ro, --readonly - Print readonly bookies - Default: false - -rw, --readwrite - Print readwrite bookies - Default: false -``` - -### Proposed Changes - -- Introduced a new module called `bookkeeper-tools` for developing the new CLI. -- The new CLI will use [JCommander](http://jcommander.org) for parse command line paramters: better on supporting this proposal commandline syntax. -- All the actual logic of the commands will be organized under `org.apache.bookkeeper.tools.cli.commands`. Each command group has its own subpackage and each command will be a class file under that command-group subpackage. - Doing this provides better testability, since the command logic is limited in one file rather than in a gaint shell class. Proposed layout can be found [here](https://github.com/sijie/bookkeeper/tree/bookie_shell_refactor/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands). -- For each command: the logic of a command will be moved out of `BookieShell` to its own class `org.apache.bookkeeper.tools.cli.commands...java`. The old BookieShell will use the new Command class and delegate the actual logic. - -An initial prototype is available: https://github.com/sijie/bookkeeper/tree/bookie_shell_refactor - -### Compatibility, Deprecation, and Migration Plan - -`bin/bookkeeper shell` and `bin/bookkeeper-cli` will co-exist for a few releases. After `bin/bookkeeper-cli` takes over all the functionalities of `BookieShell`, we will consider deprecating the old `BookieShell`. - -So no compatibility concern at this moment. - -### Test Plan - -When a command is moved from BookieShell to `org.apache.bookkeeper.tools.cli`, several unit tests should be added: - -- Unit tests for the command logic itself. -- Unit tests in new CLI for this command. -- Unit tests in old BookieShell for this command. - -### Rejected Alternatives - -Another proposal is to redesign the bookkeeper CLI using admin REST api. This is not considered at this moment because some of the commands are not well supported in admin REST api (for example, metaformat, bookieformat, and most of the commands -used for troubleshooting individual bookies). If we want to support a CLI using admin REST api, we can have a separate CLI called `bookkeeper-rest-ci` to use admin REST api for operating the cluster. diff --git a/site/bps/BP-28-etcd-as-metadata-store.md b/site/bps/BP-28-etcd-as-metadata-store.md deleted file mode 100644 index f60ef19c65f..00000000000 --- a/site/bps/BP-28-etcd-as-metadata-store.md +++ /dev/null @@ -1,102 +0,0 @@ ---- -title: "BP-28: use etcd as metadata store" -issue: https://github.com/apache/bookkeeper/ -state: 'Under Discussion' -release: "N/A" ---- - -### Motivation - -Currently bookkeeper uses zookeeper as the metadata store. However there is a couple of issues with current approach, especially using zookeeper. - -These issues includes: - -1. You need to allocate special nodes for zookeeper. These nodes need to be treated specially, and have their own monitoring. - Ops need to understand both bookies and zookeeper. -2. ZooKeeper is the scalability bottleneck. ZooKeeper doesn’t scale writes as you add nodes. This means that if your bookkeeper - cluster reaches the maximum write throughput that ZK can sustain, you’ve reached the maximum capacity of your cluster, and there’s nothing you - can do (except buy bigger hardware for your special nodes). -3. ZooKeeper enforces you into its programming model. In general, its programming model is not too bad. However it becomes problematic when - the scale goes up (e.g. the number of clients and watcher increase). The issues usually comes from _session expires_ and _watcher_. - - *Session Expires*: For simplicity, ZooKeeper ties session state directly with connection state. So when a connection is broken, a session is usually expired (unless it reconnects before session expires), and when a session is expired, the underlying connection can not be used anymore, the application has to close the connection and re-establish a new client (a new connection). It is understandable that it makes zookeeper development easy. However in reality, it means if you can not establish a session, you can’t use this connection and you have to create new connections. Once your zookeeper cluster is in a bad state (e.g. network issue or jvm gc), the whole cluster is usually unable to recover because of the connection storm introduced by session expires. - - *Watchers*: The zookeeper watcher is one time watcher, applications can’t reliably use it to get updates. In order to set a watcher, you have to read a znode or get children. Imagine such a use case, clients are watching a list of znodes (e.g. list of bookies), when those clients expire, they have to get the list of znodes in order to rewatch the list, even the list is never changed. - - The combination of session expires and watchers is often the root cause of critical zookeeper outages. - -This proposal is to explore other existing systems such as etcd as the metadata store. Using Etcd doesn't address concerns #1, however it might potentially -address concern #2 and #3 to some extend. And if you are running bookkeeper in k8s, there is already an Etcd instance available. It can become easier to run -bookkeeper on k8s if we can use Etcd as the metadata store. - -NOTE: this proposal has some overlaps on goals/vision with the distributed k/v store work (a separate BP will be sent out soon). However they are not conflicting with each other. -Both proposals are exploring a better metadata storage solution for bookkeeper. - -### Public Interfaces - -A new metadata store module `metadata-store-etcd` will be added to bookkeeper. This module will be implementing all the required metadata interfaces: - -These interfaces include: - -- RegistrationClient -- RegistrationManager -- LayoutManager -- LedgerIdGenerator -- LedgerManager -- LedgerUnderreplicatedManager -- LedgerManagerFactory - -### Proposed Changes - -Since Etcd provides a key/value model rather than a tree-like structure. So the metadata will be organized in a key/value way as below: - -- `scope`: the prefix used for prefixing all the keys used for storing the metadata for a given cluster. The `scope` is effectively same as `zkLedgerRootPath`. -- `scope`/LAYOUT: key for storing layout data -- `scope`/INSTANCEID: key for storing instance id -- `scope`/IDGEN/``: key for id generation. `` is to allow concurrent id generation. -- `scope`/cookies/``: key for storing a bookie's cookie -- `scope`/available/readwrite/``: key for registering readwrite bookie. (lease will be applied to this key) -- `scope`/available/readonly/``: key for registering readonly bookie. (lease will be applied to this key) -- `scope`/ledgers/``: key for storing a ledger metadata. `` is `String.format("%19d", ledgerId)`, pre-padding with 0s to make sure ledgers are stored in order. - -#### Registration Manager - -- Bookie Register: write to key "`scope`/available/readwrite/``" with a keepalive lease. -- Bookie Unregister: delete key "`scope`/available/readwrite/``". - -#### Registration Client - -- Get readwrite bookies: range operation to fetch keys between "`scope`/available/readwrite/" and "`scope`/available/readwrite_end/". -- Get readonly bookies: range operation to fetch keys between "`scope`/available/readonly/" and "`scope`/available/readonly_end/". -- Watch readwrite bookies: watch operation to watch keys between "`scope`/available/readwrite/" and "`scope`/available/readwrite_end/". -- Watch readonly bookies: watch operation to watch keys between "`scope`/available/readonly/" and "`scope`/available/readonly_end/". - -#### Layout Manager - -- Create layout: a txn operation: put layout data to key "`scope`/LAYOUT" when this key doesn't exist. -- Delete layout: a delete operation on "`scope`/LAYOUT" -- Read layout: a get operation on "`scope`/LAYOUT" - -#### Ledger Id Generator - -- Id generation: get key, increment by 1 and then update with a txn operation. - -#### Ledger Manager - -- Create Ledger: a txn operation to put ledger metadata to key "`scope`/ledgers/``" when this key doesn't exist -- Write Ledger: a txn operation to put ledger metadata to key "`scope`/ledgers/``" when key version matches -- Delete Ledger: a delete operation on key "`scope`/ledgers/``" -- Read Ledger: a get operation on key "`scope`/ledgers/``" -- Iteration: a range operation fetch keys between "`scope`/ledgers/" and "`scope`/ledgers_end/" - -### Compatibility, Deprecation, and Migration Plan - -- This is a new metadata store. No impacts to existing users. -- We are not planning to implement any migration tools any time soon. This only for new users or setting up new clusters. - -### Test Plan - -- Unit tests for all the metadata manager implementation. -- End-to-end integration tests -- Jepsen tests - -### Rejected Alternatives - -N/A diff --git a/site/bps/BP-29-metadata-store-api-module.md b/site/bps/BP-29-metadata-store-api-module.md deleted file mode 100644 index 9a9b887af19..00000000000 --- a/site/bps/BP-29-metadata-store-api-module.md +++ /dev/null @@ -1,87 +0,0 @@ ---- -title: "BP-29: Metadata API module" -issue: https://github.com/apache/bookkeeper/ -state: 'Accepted' -release: "N/A" ---- - -### Motivation - -We have already abstracted all the metadata operations into interfaces. And all the bookkeeper implementations only reply on metadata interfaces, -rather than depending on zookeeper. This proposal is to organize the metadata interfaces and its implementations in a separate module and make -bookkeeper implementation only depends on metadata interfaces, not depends on zookeeper. This would a few benefits: - -- It allows supporting different metadata storages, without bringing in dependencies of metadata store implementation directly into - bookkeeper-server module. The development of different metadata storage can be done without interleaving with each other. -- It would define a clean module dependency between bookkeeper implementation and metadata api, and how bookkeeper load a different metadata - implementation. - -### Public Interfaces - -A more generic setting `metadataServiceUri` is introduced for replacing implementation specific settings `zkServers` and `zkLedgersRootPath`. - -A metadata service uri defines the location of a metadata storage. In zookeeper based implementation, the metadata service url will be -`zk:///`. - -This new setting in bookie configuration will be like as below: - -``` -metadataServiceUri=zk://127.0.0.1/ledgers -``` - -If we eventually support Etcd as one of the metadata storages. Then the setting in bookie configuration to use Etcd will be: - -``` -metadataServiceUri=etcd:/// -``` - -### Proposed Changes - -#### Configuration - -This BP proposes introducing a more generic metadata setting `metadataServiceUri` to replace implementation specific settings -`zkServers` and `zkLedgersRootPath`. All implementation specific settings should be considered moving to implementation itself. - -The `metadataServiceUri` can also be used for replacing the need of configuring `ledgerManagerFactoryClass`, `registrationClientClass` and -`registrationManagerClass`. It is unnecessarily complicated to configure multiple settings to load a specific metadata implementation. -We can just use the `scheme` field in `metadataServiceUri` to resolve which metadata implementation to use. Using uri to resolve -different driver or implementation is commonly seen at java world, for example, jdbc to support different database drivers. Also, distributedlog -uses this pattern to load different metadata driver. - -So in zookeeper based metadata implementation, the metadata service uri can be: - -- `zk+flat://127.0.0.1/ledgers`: the scheme is "zk+flat". it means a zookeeper base metadata implementation and it uses flat ledger manager. -- `zk+hierarchical://127.0.0.1/ledgers`: the scheme is "zk+hierarchical". it means a zookeeper base metadata implementation and it - uses hierarchical ledger manager. -- `zk+longhierarchical://127.0.0.1/ledgers`: the scheme is "zk+longhierarchical". it means a zookeeper base metadata implementation and it - uses long hierarchical ledger manager. - -#### Metadata Stores - -Introduce a new directory called `metadata-stores` for storing all the metadata related modules. Under this directory, it will have following modules: - -- `api`: it is the metadata api module `metadata-store-api`. It contains all the files defining the metadata interfaces. -- `zookeeper`: it is the zookeeper implementation module `metadata-store-zookeeper`. It contains all the files that implementing the metadata interfaces - using zookeeper. - -If a new metadata implementation is added, a new directory will be created under `metadata-stores` to contain the implementation. For example, if we -are adding `Etcd` as the metadata store. A `Etcd` directory will be created under `metadata-stores/etcd` to store the files that implement metadata -interfaces using etcd client. And its module is named as `metadata-store-etcd`. - -We then change bookkeeper-server to depend on `metadata-store-api` only. - -This approach is same as other pluggable modules `stats-api` and `http-server`. - -### Compatibility, Deprecation, and Migration Plan - -No compatibility concern at this moment. New setting is introduced and old settings will still continue to work. -No immediate deprecation. -No migration is needed. - -### Test Plan - -This proposal is mostly around refactor. So existing test cases would cover this. - -### Rejected Alternatives - -N/A diff --git a/site/bps/BP-31-durability b/site/bps/BP-31-durability deleted file mode 100644 index 7c16eb22bf0..00000000000 --- a/site/bps/BP-31-durability +++ /dev/null @@ -1,134 +0,0 @@ - ---- -title: "BP-31: BookKeeper Durability (Anchor)" -issue: https://github.com/apache/bookkeeper/1202 -state: 'Accepted' -release: "N/A" ---- -## Motivation -Apache BookKeeper is transitioning into a full fledged distributed storage that can keep the data for long term. Durability is paramount to achieve the status of trusted store. This Anchor BP discusses many gaps and areas of improvement. Each issue listed here will have another issue and this BP is expected to be updated when that issue is created. - -## Durability Contract -1. **Maintain WQ copies all the time**. If any ledger falls into under replicated state, there needs to be an SLA on how quickly the replication levels can be brought back to normal levels. -2. **Enforce Placement Policy** strictly during write and replication. -3. **Protect the data** against corruption on the wire or at rest. - -## Work Grouping (In the order of priority) -### Detect Durability Validation -First step is to understand the areas of durability breaches. Design metrics that record durability contract violations. -* At the Creation: Validate durability contract when the ledger is being created -* At the Deletion: Validate durability contract when ledger is deleted -* During lifetime: Validate durability contract during the lifetime of the ledger.(periodic validator) -* During Read: IO or Checksum errors in the read path -### Delete Discipline -* Build a single delete choke point with stringent validations -* Archival bit in the metadata to assist Two phase Deletes -* Stateful/Explicit Deletes -### Metadata Recovery -* Metadata recovery tool to reconstruct the metadata if the metadata server gets wiped out. This tool need to make sure that the data is readable even if we can't get all the metadata (ex: ctime) back. - -### Plug Durability Violations -Our first step is to identify durability viloations. That gives us the magnitude of the problem and areas that we need to focus. In this phase, fix high impact areas. -* Identify source of problems detected by the work we did in step-1 above (Detect Durability Validation) -* Rereplicate under replicated ledgers detected during write -* Rereplicate under replicated / corrupted ledgers detected during read -* Replicated under replicated ledgers identified by periodic validator. -### Durability Test -* Test plan, new tests and integrating it into CI pipeline. -### Introduce bookie incarnation -* Design/Implement bookie incarnation mechanism -### End 2 End Checksum -* Efficient checksum implementation (crc32c?) -* Implement checksum validation on bookies in the write path. -### Soft Deletes -* Design and implement soft delete feature -### BitRot detection -* Design and implement bitrot detection/correction. - -## Durability Contract Violations -### Write errors beyond AQ are ignored. -BK client library transparently corrects any write errors while writing to bookie by changing the ensemble. Take a case where `WQ:3 and AQ:2`. This works fine only if the write fails to the bookie before it gets 2 successful responses. But if the 3rd bookie write fails **after** 2 successful responses and the response sent to client, this error is logged and no immediate action is taken to bring up the replication of the entry. -This case **may not be** detected by the auditor’s periodic ledger check. Given that we allow out of order write, that in the combination of 2 out of 3 to satisfy client, it is possible to have under replication in the middle of the ensemble entry. Hence ledgercheck is not going to find all under replication cases, on top of that, periodic ledger check is a complete sweep of the store, an very expensive and slow crawl hence defaulted to once a week run. - -### Strict enforcement of placement policy -The best effort placement policy increases the write availability but at the cost of durability. Due to this non-strict placement, BK can’t guarantee data availability when a fault domain (rack) is lost. This also makes rolling upgrade across fault domains more difficult/non possible. Need to enforce strict ensemble placement and fail the write if all WQ copies are not able to be placed across different fault domains. Minor fix/enhancement if we agree to give placement higher priority than a successful write(availability) - -The auditor re-replication uses client library to find a replacement bookie for each ledger in the lost bookie. But bookies are unaware of the ledger ensemble placement policy as this information is not part of metadata. - -### Detect and act on Ledger disk problems -While Auditor mechanism detects complete bookie crash, there is no mechanism to detect individual ledger disk errors. So if a ledger disk goes bad, bookie continues to run, and auditor can’t recognize under replication condition, until it runs the complete sweep, periodic ledger check. On the other hand bookie refuses to come up if it finds a bad disk, which is right thing to do. This is easy to fix, in the interleaved ledger manger bad disk handle. - -### Checksum at bookies in the write path -Lack of checksum calculations on the write path makes the store not to detect any corruption at the source issues. Imagine NIC issues on the client. If data gets corrupted at the client NIC’s level it safely gets stored on bookies (for the lack of crc calculations in the write path). This is a silent corruption of all 3 copies. For additional durability guarantees we can add checksum verification on bookies in the write path. Checksum calculations are cpu intensive and will add to the latency. But Java9 is adding native support for CRC32-C - A hardware assisted CRC calculation. We can consider adding this additional during JAVA-9 port after evaluating performance tradeoffs. - -### No repair in the read path -When a checksum error is detected, in addition to finding good replica, sfstore need to repair(replace with good one) bad replica too. - - -## Operations -### No bookie incarnation mechanism -A bookie `B1 at time t1` ; and same bookie `B1 at time t2` after bookie format are treated in the same way. -For this to cause any durability issues: -* Replication/Auditor mechanism is stopped or not running for some reason. (A stuck auditor will start a new one due to ZK) -* One of bookies(B1) went down (crash or something) -* B1’s Journal dir and all ledger dir got wiped. -* B1 came back to life as a fresh bookie -* Auditor is enabled monitoring again - -At this point auditor doesn’t have capability to know that the B1 in the cluster is not the same B1 that it used to be. Hence doesn’t consider it for under replication. This is a pathological scenario but we at least need to have a mechanism to identify and alert this scenario if not taking care of bookie incarnation issue. - -## Enhancements -### Delete Choke Points -Every delete must go through single routine/path in the code and that needs to implement additional checks to perform physical delete. - -### Archival bit in the metadata to assist Two phase Deletes -Main aim of this feature is to be as conservative as possible on the delete path. As explained in the stateful explicit deletes section, lack of ledgerId in the metadata means that ledger is deleted. A bug in the client code may erroneously delete the ledger. To protect from that, we want to introduce a archive/backedup bit. A separate backup/archival application can mark the bit after successfully backing up the ledger, and later on main client application will send the delete. If this feature is enabled, BK client will reject and throw an exception if it receives a delete request without archival/backed-up bit is not set. This protects the data from bugs and erroneous deletes. - -### Stateful explicit deltes -Current bookkeeper deletes synchronously deletes the metadata in the zookeeper. Bookies implicitly assume that a particular ledger is deleted if it is not present in the metadata. This process has no crosscheck if the ledger is actually deleted. Any ZK corruption or loss of the ledger path znodes will make bookies to delete data on the disk. No cross check. Even bugs in bookie code which ‘determines’ if a ledger is present on the zk or not, may lead to data deletion. - -Right way to deal with this is to asynchronously delete metadata after each bookie explicitly checks that a particular ledger is deleted. This way each bookie explicitly checks the ‘delete state’ of the ledger before deleting on the disk data. One of the proposal is to move the deleted ledgers under /deleted/ other idea is to add a delete state, Open->Closed->Deleted. - -As soon as we make the metadata deletions asynchronous, the immediate question is who will delete it? -Option-1: A centralized process like auditor will be responsible for deleting metadata after each bookie deletes on disk data. -Option-2: A decentralized, more complicated approach: Last bookie that deletes its on disk data, deletes the metadata too. -I am sure there can be more ideas. Any path will need a detailed design and need to consider many corner cases. - -#### Obvious points to consider: -ZK as-is heavily loaded with BK metadata. Keeping these znodes around for more time ineeded puts more pressure on ZK. -If a bookie is down for long time, what would be the delete policy for the metadata? -There will be lots of corner case scenarios we need to deal with. For example: -A bookie-1 hosting data for ledger-1 is down for long time -Ledger-1 data has been replicated to other bookies -Ledger-1 is deleted, and its data and metadata is clared. -Now bookie-1 came back to life. Since our policy is ‘explicit state check delete’ bookie-1 can’t delete ledger-1 data as it can’t explicitly validate that the ledger-1 has been deleted. -One possible solution: keep tomestones of deleted ledgers around for some duration. If a bookie is down for more than that duration, it needs to be decommissioned and add as a new bookie. -Enhance: Archival bit in the metadata to assist Two phase Deletes -Main aim of this feature is to be as conservative as possible on the delete path. As explained in the stateful explicit deletes section, lack of ledgerId in the metadata means that ledger is deleted. A bug in the client code may erroneously delete the ledger. To protect from that, we want to introduce a archive/backedup bit. A separate backup/archival application can mark the bit after successfully backing up the ledger, and later on main client application will send the delete. If this feature is enabled, BK client will reject and throw an exception if it receives a delete request without archival/backed-up bit is not set. This protects the data from bugs and erroneous deletes. - -### Metadata recovery tool -In case zookkeper completely wiped we need a way to reconstruct enough metadata to read ledgers back. Currently metadata contains ensemble information which is critical for reading ledgers back, and also it has additional metadata like ctime and custom metadata. Every bookie has one index file per ledger and that has enough information to reconstruct the ensemble information so that the ledgers can be made readable. This tool can be built in two ways. -If ZK is completely wiped, reconstruct entire data from bookie index files. -If ZK is completely wiped, but snapshots are available, restore ZK from snapshots and built the delta from bookie index files. - -### Bit Rot Detection (BP-24) -If the data stays on the disk for long time(years), it is possible to experience silent data degradation on the disk. In the current scenario we will not identify this until the data is read by the application. - -### End to end checksum -Bookies never validate the payload checksum. If the the client’s socket has issues, it might corrupt the data (at the source) and it won’t be detected until client reads it back. That will be too late as the original write was successful for the application. Use efficient checksum mechanisms and enforce checksum validations on the bookie’s write path. If checksum validation fails, the the write itself will fail and application will be notified. - - -## Test strategy to validate durability -BK need to develop a comprehensive testing strategy to test and validate the store’s durability. Various methods and levels are tests are needed to gain confidence for deploying the store in production. Specific points are mentioned here and these are in addition to regular functional testing/validation. -### White box error injection -Introduce all possible errors in the write path, kick replication mechanism and make sure cluster reached desired replica levels. -Corrupt first readable copy and make sure that the corruption is detected on the read path, and ultimately read must succeed after trying second replica. -Corrupt packet after checksum calculation on the client and make sure that it is detected in the read path, and ultimately read fails as this is corruption at the source. -After a write make sure that the replica is distributed across fault zones. -Kill a bookie, make sure that the auditor detected and replicated all ledgers in that bookie according to allocation policy (across fault zones) -### Black box error injection (Chaos Monkey) -While keeping longevity testing which is doing continues IO to the store introduce following errors. -Kill random bookie and reads should continue. -Kill random bookies keeping minimum fault zones to satisfy AQ Quorum during write workload. -Simulate disk errors in random bookies and allow the bookie to go down and replication gets started. -Make sure that the cluster is running in full durable state through the tools and monitoring built. diff --git a/site/bps/BP-32-advisory-write-close.md b/site/bps/BP-32-advisory-write-close.md deleted file mode 100644 index afe8e971f50..00000000000 --- a/site/bps/BP-32-advisory-write-close.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -title: "BP-32: Advisory (optimistic) write close" -issue: https://github.com/apache/bookkeeper/issues/1343 -state: Under Discussion -release: N/A ---- - -### Motivation - -With [entrylog per ledger feature](https://github.com/apache/bookkeeper/issues/570) there will be dedicated entrylog for each ledger and it provides EntryLogManagerForEntryLogPerLedger (EntryLogManager). Since there is going to be entrylog per ledger, with the current Bookie implementation there is no way for EntryLogManagerForEntryLogPerLedger (EntryLogManager) to know when the entrylog is writeclosed, so that entrylog for the active ledger can be rotated. So it would be ideal to have explicit call to EntryLogger when the write for this ledger is done and it is writeclosed, so that it can rotate the entrylog as soon as it is write closed. This will minimize the number of entrylogs/file descriptors that are open/active and also it will make progress in leastUnflushedLogId, so that GarbageCollectorThread can consider these entrylogs for garbage collection. - -### Proposed Changes - -So in entrylog per ledger feature implementation, as a first step of handling this, expireAfterAccess (each entrylog should be automatically rotated from the current active entrylog list once a fixed duration has elapsed after the last access of entrylog for addEntry) will be implemented and also the maximum number of entrylogs that can be active at a given time (this is for limiting number of entrylogs/filedescriptors open at a given time). - -The above mentioned approaches are preliminary ways of handling but it would be ideal to have explicit call to EntryLogger when the write for this ledger is done so that it can rotate the entrylog as soon as it is write closed. So to implement this following can be done - -- have explicit write close request. This write close request should be sent to the current ensemble when the write handle is closed. This should be just optimistic write close operation and callback of this operation should be just logger, saying if it is succeeded / partially succeeded / failed. This should be done asynchronously and the write handle close operation should not be blocked for this response. - -- In the case of ledger recover open, readrequest (ReadEntryProcessorV3) with fence flag, can take care of calling appropriate methods in ledgerdescriptor / entry logger to rotate the entrylog for the ledger - -- in the case of auto-replication case, LedgerFragmentReplicator already uses bookieclient for addEntry (bkc.getBookieClient().addEntry), the same bookieClient instance can be used to call explicit writeClose method in bookieclient. - -- in the case of any write failure in bookie for an entry, then before sending error response to the client do entrylog rotation - -EntryLogManagerForEntryLogPerLedger is the class which is last layer in the stack to receive this writeClose call for a ledger, it would store this info in in-memory data structure and when next next checkpoint is called it would rotate entrylogs of these writeclosed ledgers. - -### Public Interfaces - -As explained above, this advisory write close should be transparent to Bookkeeper API user, this should be piggybagged in writehandle close call and other internal Bookie / BookieClient internal class methods (ReadEntryProcessorV3.readrequest and LedgerFragmentReplicator). But this feature introduces new protobuf message between Client and Bookie. - -``` -message WriteCloseRequest { - required int64 ledgerId = 1; - required bytes masterKey = 2; -} - -message WriteCloseResponse { - required StatusCode status = 1; - required int64 ledgerId = 2; -} -``` - -### Compatibility, Deprecation, and Migration Plan - -- With this feature we are introducing new protocol message. Will do the required Compatibility testing. But since it is going to be advisory (optimistic) in nature, failure in this request in anyway should not affect functioning in any way. -- Also, possibly this explicit write close can be used in future for other purposes. - -### Test Plan - -- unit tests for all the write/read handle close scenarios, recoveropen and replicator scenarios -- end-to-end integrations tests. - -### Rejected Alternatives - -N/A diff --git a/site/bps/BP-33-building-official-docker-imags.md b/site/bps/BP-33-building-official-docker-imags.md deleted file mode 100644 index d50cac1fcbc..00000000000 --- a/site/bps/BP-33-building-official-docker-imags.md +++ /dev/null @@ -1,40 +0,0 @@ ---- -title: "BP-33: Move releasing official docker images out of main repo" -issue: https://github.com/apache/bookkeeper/1449 -state: "Under Discussion" -release: "4.7.1" ---- - -### Motivation - -Current bookkeeper docker images are auto-built by apache docker account. However it becomes problematic in the release process: - -Docker autobuild uses release tag for labeling the versions for docker images. But the `Dockerfile` can only be updated after -a release is successfully made. So we have to retag a release after a release, in order to update `Dockerfile` to build the docker -image. - -### Proposed Changes - -Follow what `flink` is doing, and maintain the docker files outside of the bookkeeper main repo. - -- Create an organization `asfbookkeeper-ecosystem` for hosting repos that related bookkeeper but not necessarily needed to be put in main repo. -- Create a repo `docker-bookkeeper` under `asfbookkeeper-ecosystem` for hosting the docker files following the suggested practices from making a docker official image. -- Add a library definition file under `docker-library/official-images` for bookkeeper. -- Add an image doc under `docker-library/docs` for bookkeeper. -- Update the release guide on how to update docker images at the end of each release. -- Remove `docker` dir from main repo or make it used for building *unreleased* docker images only. -- Disable docker autobuild from apache account. - -Proposed docker file repo: https://github.com/asfbookkeeper-ecosystem/docker-bookkeeper - -### Compatibility, Deprecation, and Migration Plan - -N/A - -### Test Plan - -N/A - -### Rejected Alternatives - -N/A diff --git a/site/bps/BP-34-cluster-metadata-checker.md b/site/bps/BP-34-cluster-metadata-checker.md deleted file mode 100644 index a0b13fd4194..00000000000 --- a/site/bps/BP-34-cluster-metadata-checker.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -title: "BP-34: Cluster Metadata Checker" -issue: https://github.com/apache/bookkeeper/issues/1602 -state: Accepted -release: N/A ---- - -### Motivation - -Currently in the Auditor we have two checkers - Periodic Bookie Check and Periodic Ledger check. Bookie Check validates the availability of bookies and if it finds any lost bookies it will mark ledgers residing in the lost bookies to be under replicated. Ledger Check reads the first entry and last entry of the segment from all the corresponding bookies of the ensemble and if it fails to read any entry then it will mark that ledger under replicated. By setting appropriate value to the conf - ‘auditorLedgerVerificationPercentage’ we can read upto 100% of entries of the ledger from all the corresponding bookies of the ensemble and any failure in reading will lead to mark ledger under replicated. - -Ideally for having complete confidence on the date in the cluster, it is needed to have a new checker - validating ledger placement policy, durability contract, progress in handling under replication and availability of bookies of the ensemble of ledgers. Though by configuring 'auditorLedgerVerificationPercentage' to 100% in periodic ledger check, we would get most of what we are intending to achieve. But this comes at heavy price since it involves reading all the entries from all the corresponding bookies in the ensemble, so it is not a performant solution. - -### Proposed Changes - -Intention of this new checker is to validate following things - - ledger placement policy : Ensemble of each segment in Ledger should adhere to LedgerPlacementPolicy - - durability contract : Every entry has WQ number of replicas and entries are replicated according to RoundRobinDistributionSchedule - - progress in handling under replication : No ledger is marked underreplicated for more than acceptable time - - availability of bookies of the ensemble of ledgers : If Auditor fails to get response from a Bookie, then that Bookie shouldn’t be registered to metadata server and Auditor should be aware of it unavailability or if it is a transient error in getting response from Bookie then subsequent calls to that Bookie should succeed. - -Roles and Responsibilities of the cluster metadata checker - - Police the durability contract and report violations. Its job is to make sure that the metadata server(zk) and the storage servers (bookies) are in sync. Simply put, check if bookies agree with the metadata server metadata and if not, raise an alert. - - Scrutiny’s job is not to fix if it finds any inconsistency. Instead make a noise about it. If the scrutiny fails, it means that we have a potential hole(bug) in our service to meet the durability contract. Scrutiny exposes that hole with enough information the help identify the issue and fix it. - - The Metadata Scrutiny needs to be light weighted esp., on Bookie and must run regularly giving the confidence that the cluster is in good state. - -High Level Logic - - Things would get complicated analyzing ledgers which are not closed because of several reasons, viz., unable to know lastEntryId by reading ZK metadata, possibility of change in ensemble because of write failure to a bookie, and other subtleties in dealing with last unclosed segment of the ledger. So for the sake of simplicity this checker should be limited to ledgers which are write closed/fenced. - - This durability check for each ledger will be run as a processor in ledgerManager.asyncProcessLedgers and it would ignore ledgers which are still open for write. - - first step is to check if this ledger is marked underreplicated already. If it is marked underreplicated for more than acceptable time then report it as violation otherwise skip this underreplicated ledger for this iteration of durability check. Since there is no point in further analyzing this ledger if it is already marked under replicated. - - get the ledger metadata of the ledger from the metadata server - - make sure that the ensemble of the ledger segments is in agreement with ledgerplacement policy. Any violation should be reported. - - get the info about available entries of the ledger from the bookies of the ensemble. Bookie is expected to return list of entries it contains for a given ledger - - Have to make sure that Bookies contain all the entries it is supposed to contain according to the RoundRobinDistributionSchedule and each entry has writequorum number of copies. Any violation should be reported. - - If there is any failure in trying to get info. from Bookie of the ensembles of the ledger, then add this ledger to potentially faulty ledgers list (but don't report it yet.) - - (in previous steps, in case of any violation or bookie read error, before reporting violation, check if the ledger is marked underreplicated. If it is marked underreplicated then ignore this ledger for this iteration. If it is not marked underreplicated, then get the ledgermetadata of this ledger onemore time. Check if it is any different from the ledgermetadata we got initially then instead of reporting the violation, redo the analysis for this ledger because apparently something had changed in the metadata (esp. with ensemble) and hence it is better to reevaluate instead of false alarm.) - - if there are potentially faulty ledgers because of unavailable/unreachable bookies, then schedule a new durability check task with time delay just for the potentially faulty ledgers. Even after subsequent delayed checks, if Auditor failed to get response from bookies then make sure that Bookie isn’t registered to metadata server and Auditor is aware of it unavailability, if not then report the violation. - - Auditor is going to use existing mechanisms/frameworks to report the violations - bookkeeper-stats statslogger/counters and complementing information in logs. - - It makes sense to group all the durability violations found in a scrutiny run according to the categories and report the aggregated count for each category after the end of the scrutiny run. - - before reporting these violations, each violation should be logged with complete information, so that it can be used to understand what went wrong. - -### Public Interfaces - -To know the entries of a ledger data persisted in a Bookie, currently there is no other way than reading the entry from bookie using BookieClient instance. So for auditor to know what entries of a ledger, Bookie contains we need to have a new on wire Bookkeeper protocol API as mentioned below. - -``` -message GetListOfEntriesOfALedgerRequest { - required int64 ledgerId = 1; -} - -message GetListOfEntriesOfALedgerResponse { - required StatusCode status = 1; - required int64 ledgerId = 2; - optional bytes availabilityOfEntriesOfLedger = 3; // explained below -} -``` - -For ‘availabilityOfEntriesOfLedger’ we can use following condensed encoding format, which helps in reducing the number of bytes needed to represent the list of entries. - -(note: following representation is used just for understanding purpose, but this is not protobuf (or any other) representation) - -``` -AvailabilityOfEntriesOfLedger { - OrderedCollection sequenceGroup; -} - -SequenceGroup { - long firstSequenceStart; - long lastSequenceStart; - int sequenceSize; - int sequencePeriod; -} -``` - -Nomenclature: - - - Continuous entries are grouped as a ’Sequence’. - - Number of continuous entries in a ‘Sequence’ is called ‘sequenceSize’. - - Gap between Consecutive sequences is called ‘sequencePeriod’. - - Consecutive sequences with same sequenceSize and same sequencePeriod in between consecutive sequences are grouped as a SequenceGroup. - - ‘firstSequenceStart’ is the first entry in the first sequence of the SequenceGroup. - - ‘lastSequenceStart’ is the first entry in the last sequence of the SequenceGroup. - - Ordered collection of such SequenceGroups will represent entries of a ledger residing in a bookie. - -(in the best case scenario there will be only one SequenceGroup, ‘sequencePeriod’ will be ensembleSize and ‘sequenceSize’ will be writeQuorumSize of the ledger). - -for example, - -example 1 (best case scenario): - -1, 2, 4, 5, 7, 8, 10, 11 - -in this case (1, 2), (4, 5), (7, 8), (10, 11) are sequences and in this scenario there happens to be just one SequenceGroup, which can be represented like - -{ firstSequenceStart - 1, lastSequenceStart - 10, sequenceSize - 2, sequencePeriod - 3 } - -example 2 (an entry is missing): - -1, 2, 3, 6, 7, 8, 11, 13, 16, 17, 18, 21, 22 - -(entry 12 is missing and in the last sequence there are only 2 entries 21, 22) -in this case (1, 2, 3), (6, 7, 8), (11), (13), (16, 17, 18), (21, 22) are the sequences -so the sequence groups are - -{ firstSequenceStart - 1, lastSequenceStart - 6, sequenceSize - 3, sequencePeriod - 5 }, { firstSequenceStart - 11, lastSequenceStart - 13, sequenceSize - 1, sequencePeriod - 2 }, { firstSequenceStart - 16, lastSequenceStart - 16, sequenceSize - 3, sequencePeriod - 0 }, { firstSequenceStart - 21, lastSequenceStart - 21, sequenceSize - 2, sequencePeriod - 0 } - -As you can notice to represent a SequenceGroup, two long values and two int values are needed, so each SequenceGroup can be represented with (2 * 8 + 2 * 4 = 24 bytes). - -In the ‘availabilityOfEntriesOfLedger’ byte array, for the sake of future extensibility it would be helpful to have reserved space for metadata at the beginning. So the first 64 bytes will be used for metadata, with the first four bytes specifying the int version number, next four bytes specifying the number of entries for now and the rest of the bytes in the reserved space will be 0's. The encoded format will be represented after the first 64 bytes. The ordered collection of SequenceGroups will be appended sequentially to this byte array, with each SequenceGroup taking 24 bytes. - -So for a ledger having thousands of entries, this condensed encoded format would need one or two SequenceGroups (in the best case, with no holes and no overreplication) 24/48 bytes, which would be much less than what is needed to represent using bit vector (array of bits indicating availability of entry at that particular index location) - -Any encoded format needs encoder and decoder at the sending/receiving ends of the channel, so the encoding/decoding logic should be handled optimally from computation and memory perspective. - -Here Bookie is expected to just attain index information (say from LedgerCache (Index Files) - IndexPersistenceMgr and IndexInMemPageMgr, unflushed entries in EntryMemTable in SortedLedgerStorage case and from rocksdb database in DBLedgerStorage case) but it doesn’t actually check the availability of this entry in Entrylogger. Since the intention of this checker is limited to do just metadata validation at cluster level. - -### Compatibility, Deprecation, and Migration Plan - -- With this feature we are introducing new protocol message. Will do the required Compatibility testing. - -### Test Plan - -- unit tests for newly introduced API/code at LedgerCache Level -- end-to-end tests for the new Protocol request/response -- validating the checker in all cases of violations - -### Rejected Alternatives - -N/A diff --git a/site/bps/BP-35-128-bits-support.md b/site/bps/BP-35-128-bits-support.md deleted file mode 100644 index cae6b8c4ac3..00000000000 --- a/site/bps/BP-35-128-bits-support.md +++ /dev/null @@ -1,399 +0,0 @@ ---- -title: "BP-34: 128 bits support" -issue: https://github.com/apache/bookkeeper/603 -state: "Under Discussion" -design doc: https://docs.google.com/document/d/1cu54dNSV2ZrdWCi40LcyX8NxXGRCW0609T_ewmK9BWM -release: "4.9.0" ---- - -### Motivation - -BookKeeper coordinates with a metadata store to generate a cluster wide `ledgerId`. -Currently this is a signed `64 bit` number (effectively 63 bits). This method works -great because we have a centralized metadata store for coordinating the id generation. -However this method may not scale as the cluster size and number of ledgers grow. - -[Universally unique identifier - Wikipedia](https://en.wikipedia.org/wiki/Globally_unique_identifier) -is a preferred way to generate decentralized globally unique IDs and it takes `128 bits`. -This method can scale well as it doesn't need a centralized coordination. - -This BP proposes the changes for increasing ledger id from `63 bits` to `128 bits`. - -### 128 bits - -Since there is no native support for `128 bits` in both Java and -[Protobuf](https://github.com/google/protobuf/issues/2180), we have to break `128 bits` -into 2 `64 bits` numbers for representing the `128 bits` id: - -- ledger-id-msb: the most significant 64 bits, bit 64 - 127 -- ledger-id-lsb: the least significant 64 bits, bit 0 - 63 - -For backward compatibility, the `ledger-id-lsb` is the current `64 bits` ledger-id. -The `ledger-id-msb` will be added as a new field in both API and protocol. - -I am proposing calling `ledger-id-msb` as `ledger-scope-id`. So the current 64bits `ledgerId` and -the newly introduced 64bits `ledgerScopeId` together will be forming the new `128 bits` ledger id. - -The default `ledgerScopeId` is `0`. That means any ledgers created prior to this change are allocated -under scope `0`. Hence it maintains backward compatibility during upgrade. - -The combination of `ledgerScopeId` and `ledgerId` forms the `128 bits` ledger id. We can introduce a -hex representation of this `128 bits` ledger id - `ledgerQualifiedName` . This `ledgerQualifiedName` can -be useful for CLI tooling, REST api and troubleshooting purpose. The API internally can convert -`ledgerQualifiedName` to `ledgerScopeId` and `ledgerId`. - -### Public Interfaces - -#### API Change - -The API will be introducing `ledgerScopeId` across the interfaces. This field will be optional and default to `0`. - -##### Handle - -Introduce a new method `getScopeId` for representing the scope id (the most significant `128 bits` ledger id). - -```java -public interface Handle extends AutoCloseable { - - ... - - /** - * Return the ledger scope id. The most significant 64 bits of 128 bits. - */ - long getScopeId(); - - /** - * Return the ledger id. The least significant 64 bits of 128 bits. - */ - long getId(); - - ... - -} -``` - -##### Create LedgerAdv - -Introduce a new method `withLedgerScopeId` in `CreateAdvBuilder` for providing `scopeId` -(the most significant 64 bits for 128 bits ledger id) on creating a ledger. - -```java -public interface CreateAdvBuilder extends OpBuilder { - ... - - /** - * Set the scope id for the newly created ledger. - * If no explicit scopeId is passed, the new ledger - * will be created under scope `0`. - */ - CreateAdvBuilder withLedgerScopeId(long scopeId); - - ... -} -``` - -##### Open Ledger - -Introduce a new method `withLedgerScopeId` in `OpenBuilder` for providing `scopeId` -(the most significant 64 bits for 128 bits ledger id) on opening a ledger. - -```java -public interface OpenBuilder extends OpBuilder { - ... - /** - * Set the scope id of the ledger to open. - */ - OpenBuilder withLedgerScopeId(long scopeId); - ... -} -``` - -##### Delete Ledger - -Introduce a new method `withLedgerScopeId` in `DeleteBuilder` for providing `scopeId` -(the most significant 64 bits for 128 bits ledger id) on deleting a ledger. - -```java -public interface DeleteBuilder extends OpBuilder { - ... - /** - * Set the scope id of the ledger to delete. - */ - DeleteBuilder withLedgerScopeId(long scopeId); - ... -} -``` - -#### CLI - -All BookKeeper CLI tools will be updated with additional option `—ledger-scope-id`. -Optionally we can add option `—ledger-qualified-name` (the hex representation of 128 bits). -Internally all the CLI tools will convert ledger qualified name to `ledgerId` and `ledgerScopeId`. - -#### REST - -1. All ledger related endpoints will be adding a new parameter `ledger_scope_id`. -2. `ListLedgerService` only supports listing ledgers under a given ledger scope id. - If `ledger_scope_id` is missing, it will be listing ledgers under scope `0`. - -#### Wire Protocol - -> There will be no plan for supporting 128 bits in v2 protocol, due to the limitation in v2 protocol. -> So any operations in v2 protocol with scope id not equal to 0 will be failed immediately with -> `ILLEGAL_OP` exceptions. - -All the request and response messages will be adding an optional field `optional int64 ledgerScopeId`. - -#### Entry Format - -Currently all the entries written to bookies are encoded in a certain format, including `metadata`, -`digest code` and `payload`. The entry format is not *versioned*. - -In order to support adding another field `ledgerScopeId` in the `metadata` section, we are introducing -`version` in the entry format. - -##### Entry Format V1 - -```json -Entry Format V1 -=============== ---- header --- -Bytes (0 - 7) : Ledger ID -Bytes (8 - 15) : Entry ID -Bytes (16 - 23) : LastAddConfirmed -Bytes (24 - 31) : Length ---- digest --- -Bytes (32 - (32 + x - 1)) : Digest Code (e.g. CRC32) ---- payload --- -Bytes ((32 + x) - ) : Payload -``` - -> `x` is the length of digest code. - -> Prior to introducing `ledgerScopeId`, ledgerId is assumed to be a positive value. - -##### Entry Format V2 - -```json -Entry Format V2 -=============== ---- header --- -Bytes (0 - 7) : Metadata Flags -Bytes (8 - 15) : Ledger Scope ID -Bytes (16 - 23) : Ledger ID -Bytes (24 - 31) : Entry ID -Bytes (32 - 39) : LastAddConfirmed -Bytes (40 - 47) : Length ---- digest --- -Bytes (37 - (37 + x - 1)) : Digest Code (e.g. CRC32) ---- payload --- -Bytes ((37 + x) - ) : Payload -``` - -> `x` is the length of digest code. - -###### Metadata Flags - -```json -Metadata: 1 Bytes (Long) ------------------------- -0x 0 0 - |__| - | - version - ----- -Bit 0 - 3: digest type (e.g. CRC32, CRC32C and such) -Bit 4 - 7: version, the most significant bit of this byte will be always set to 1. -it will be used for differentiating entry format v1 and v2. - -``` - -We are setting the most significant bit to be `1`. So the first byte in entry v2 will -be a negative value, which can be used for differentiating entry format v1 and v2. -The version will be encoded into the first byte. The version will be used for describing -the entry format. - -##### Decoding Entry - -The pseudo code for decoding an entry will be described as followings: - -```java - -ByteBuf entry = ...; - -int metadataFlags = entry.getByte(); - -if (metadataFlags <= 128) { // the entry is encoded in v1 format - // decoding the entry in v1 format - ... -} else { - // decoding the entry in v2 format -} - -``` - -#### Bookie Storage - -##### Journal - -A new method should be added in journal `WriteCallback` to handle `ledgerScopeId`. - -```java -public interface WriteCallback { - - void writeComplete(int rc, - long ledgerScopeId, - long ledgerId, - long entryId, - BookieSocketAddress addr, - Object ctx); - - default void writeComplete(int rc, - long ledgerId, - long entryId, - BookieSocketAddress addr, - Object ctx) { - writeComplete(rc, 0L, ledgerId, entryId, addr, ctx); - } - -} -``` - -The journal should be changed to be able to retrieve `ledgerScopeId` from the entry -payload based on [Entry Format](#entry-format). - -##### Ledger Storage - -###### EntryLogger - -1. Methods in `EntryLogger` should be able to accept `ledgerScopeId` as a parameter. -2. EntryLogger should be updated to retrieve `ledgerScopeId` from the entry payload - based on [Entry Format](#entry-format). - -###### EntryMemTable - -`ledgerScopeId` should be added as part of `EntryKey`. - -###### IndexPersistenceMgr - -Currently the ledger index files (64 bits) are stored into 2-level-hirechicy -directories - `//.idx`. - -If `ledgerScopeId` is 0, it will be using existing scheme for storing and retrieving -ledger index files. - -If `ledgerScopeId` is not 0, that means the ledgers are produced by new clients that -support 128-bits, those ledgers will be stored in a 4-level-hirechicy -directories - -`///`. - -All the file info caches should be updated to use `` -as index keys. - -###### IndexInMemPageMgr - -The LRU pages map will be updated to use `` as index -keys. - -###### DBLedgerStorage - -Currently DBLedgerStorage use `` as the index key for indexing entry -locations for each entry. - -Similar as `SortedLedgerStorage` and `InterleavedLedgerStorage`, for ledgers whose -`ledgerScopeId` is 0, they will be using existing scheme for storing their entry locations. - -For ledgers whose `ledgerScopeId` is not 0, they will be stored in a new rocksdb, -whose index key will be ``. - -#### Metadata Store - -##### LedgerManager - -All the interfaces should be updated with accepting `ledgerScopeId`. - -The actual implementation should decide how to store metadata -for ``. - -###### ZooKeeper Ledger Manager - -We need to introduce a LongLongHierchicalLedgerManager for storing metadata -indexing by ``. - -If `ledgerScopeId` is 0, then it will be falling back to `LongHierachicalLedgerManager`. -So no behavior is changed. - -If `ledgerScopeId` is not 0, those ledgers will be indexed in new hierarchy -(possible under a different znode). - -###### Ledger ID generation - -When upgrading from 64bit to 128bits, we probably don't need any centralized mechanism -for generating ledger id. It can be implemented using UUID generation. - -Especially since we are supporting 128bits by introducing `ledgerScopeId`. That means -application of bookkeeper can decide its own way for generating their `scopeId`. -An application or even bookkeeper client can generate its ledgerId using UUID generation, -then breaks the 128 bits UUID into two parts, one serves as `ledgerScopeId` and the other -one serves as `ledgerId`. - -###### Etcd - -Since Etcd has a better key/value presentation, we can basically just combine -`` as the index key for storing ledger metadata in Etcd. -Nothing is needed for special consideration. - -### Performance Concerns - -There shouldn't be any performance difference when not using 128 bit ledger id -(`ledgerScopeId` is omitted). - -Performance concerns can be arised in following areas: - -- **Wire Protocol**: additional 9 bytes will be added per entry, one byte for version - and 8 bytes for the msb of 128 bit ledger id -- **Journal**: additional 9 bytes will be added per entry (same as wire protocol). -- **EntryLogger**: additional 9 bytes will be added per entry (same as wire protocol) -- **Memtable**: additional 8 bytes will be added per indexed entry. -- **FileInfo**: there is no change to the index file format itself. -- **IndexPersistenceManager**: Files will be organized in more directory hierarchy. - It shouldn't be a big deal. -- **IndexInMemoryManager (LedgerCache)**: additional 8 bytes per index page. -- **DbLedgerStorage**: additional 8 bytes per entry for entry location. -- **Metadata**: on zookeeper, we need a 128 bit ledger manager, that means more znode - hierarchy than 64 bit ledger manager. Etcd like key/value metadata store is probably - more preferrable for 128 bit ledger manager. - -However increasing ledger id from 64 bits to 128 bits can get rid of the only remaining -central point, since we don't need to use zookeeper for ledger id generation. The id -generation can become decentralized. - -### Proposed Changes - -All the required changes are described above. In summary, the changes can -happen in following 2 phases: - -1. Ensure all components have `ledgerScopeId` added (both wire protocol, storage and such). - Assuming `ledgerScopeId` will be 0. The changes can happen independently and ensure - they are backward compatible with old clients. -2. Add `ledgerScopeId` into public API, so application can start using `ledgerScopeId`. - After that, applications can use UUID to generate ledger id and break UUID into two parts, - one is `ledgerScopeId`, while the other one is `ledgerId`. - -### Compatibility, Deprecation, and Migration Plan - -All the changes are backward compatible, since we are doing the changes by adding an optional -field `ledgerScopeId`. Old clients can still operating in the mode of `ledgerScopeId == 0`. -The new application can activate the feature by starting using `ledgerScopeId` in the new API. - -### Test Plan - -1. Add unit tests for individual components on introducing `ledgerScopeId`. -2. Add backward compatibility tests for individual components. -3. Add end-to-end integration tests for introducing `ledgerScopeId`. -4. Add end-to-end backward compatibility tests. - -### Rejected Alternatives - -N/A diff --git a/site/bps/BP-36-stats-documentation-annotation.md b/site/bps/BP-36-stats-documentation-annotation.md deleted file mode 100644 index c8905ddc8fc..00000000000 --- a/site/bps/BP-36-stats-documentation-annotation.md +++ /dev/null @@ -1,157 +0,0 @@ ---- -title: "BP-36: Stats documentation annotation" -issue: https://github.com/apache/bookkeeper/ -state: "Accepted" -release: "4.9.0" ---- - -### Motivation - -A common ask from people using bookkeeper is how they can monitor bookies and bookkeeper clients, what kind of metrics that bookkeeper exposes -and what are the important metrics. Currently bookkeeper doesn't provide any metrics page for guiding people on monitoring bookkeeper services. - -In order to help people on this, we need to provide a few documentation pages about metrics. However if we just write such pages, those pages -can quickly get out-of-dated when code is changed. The proposal here is to seek a programming way for generating metrics related pages. - -### Public Interfaces - -Introduced a `StatsDoc` annotation. - -```bash -/** - * Documenting the stats. - */ -@Retention(RetentionPolicy.RUNTIME) -@Documented -public @interface StatsDoc { - - /** - * The name of the category to group stats together. - * - * @return name of the stats category. - */ - String category() default ""; - - /** - * The scope of this stats. - * - * @return scope of this stats - */ - String scope() default ""; - - /** - * The name of this stats - * - * @return name of this stats - */ - String name(); - - /** - * The help message of this stats - * - * @return help message of this stats - */ - String help(); - - /** - * The parent metric name. - * - *

    It can used for analyzing the relationships - * between the metrics, especially for the latency metrics. - * - * @return the parent metric name - */ - default String parent() { return ""; } - - /** - * The metric name of an operation that happens - * after the operation of this metric. - * - *

    similar as {@link #parent()}, it can be used for analyzing - * the dependencies between metrics. - * - * @return the metric name of an operation that happens after the operation of this metric. - */ - default String happensAfter() { return ""; } - - -} -``` - -The `StatsDoc` annotation provides a way to annotate metrics we added to bookkeeper. - -- category: which category that the metric belongs to. e.g. server, or client. -- scope: the scope of the metric. e.g. `bookie` scope. -- name: the name of the metric. -- help: the description of the metric. - -### Proposed Changes - -In addition to the `StatsDoc` annotation, bookkeeper should provide a tool for generating the metrics yaml file -for documenting all annotated metrics. - -```bash -bin/stats-doc-gen -``` - -Example output: - -```yaml -"server": - "bookie_BOOKIE_READ_ENTRY_BYTES": - "description": |- - bytes stats of ReadEntry on a bookie - "type": |- - OPSTATS - "bookie_WRITE_BYTES": - "description": |- - total bytes written to a bookie - "type": |- - COUNTER - "bookie_BOOKIE_ADD_ENTRY": - "description": |- - operations stats of AddEntry on a bookie - "type": |- - OPSTATS - "bookie_BOOKIE_RECOVERY_ADD_ENTRY": - "description": |- - operation stats of RecoveryAddEntry on a bookie - "type": |- - OPSTATS - "bookie_BOOKIE_ADD_ENTRY_BYTES": - "description": |- - bytes stats of AddEntry on a bookie - "type": |- - OPSTATS - "bookie_BOOKIE_FORCE_LEDGER": - "description": |- - total force operations occurred on a bookie - "type": |- - COUNTER - "bookie_READ_BYTES": - "description": |- - total bytes read from a bookie - "type": |- - COUNTER - "bookie_BOOKIE_READ_ENTRY": - "description": |- - operation stats of ReadEntry on a bookie - "type": |- - OPSTATS -``` - -### Compatibility, Deprecation, and Migration Plan - -It is a new feature, which doesn't have any compatibility impacts. - -There is nothing deprecated. - -There is nothing to migrate. - -### Test Plan - -Existing testing is good enough to cover code changes. No new tests are needed. - -### Rejected Alternatives - -Alternatively, we have to manually maintain the metrics page and update each time when a new metric is added. diff --git a/site/bps/BP-37-conf-documentation.md b/site/bps/BP-37-conf-documentation.md deleted file mode 100644 index 4c9d0d9388f..00000000000 --- a/site/bps/BP-37-conf-documentation.md +++ /dev/null @@ -1,222 +0,0 @@ ---- -title: "BP-37: Improve configuration management for better documentation" -issue: https://github.com/apache/bookkeeper/1867 -state: "Accepted" -release: "4.9.0" ---- - -### Motivation - -One common task in developing bookkeeper is to make sure all the configuration -settings are well documented, and the configuration file we ship in each release -is in-sync with the code itself. - -However maintaining things in-sync is non-trivial. This proposal is exploring -a new way to manage configuration settings for better documentation. - -### Public Interfaces - -1. Introduced `ConfigKey` for defining a configuration key. A configuration key - will include informations, such as required/optional, deprecated, documentation - and etc. - -```java -public class ConfigKey { - /** - * Flag indicates whether the setting is required. - */ - @Default - private boolean required = false; - - /** - * Name of the configuration setting. - */ - private String name; - - /** - * Type of the configuration setting. - */ - @Default - private Type type = Type.STRING; - - /** - * Description of the configuration setting. - */ - @Default - private String description = ""; - - /** - * Documentation of the configuration setting. - */ - @Default - private String documentation = ""; - - /** - * Default value as a string representation. - */ - @Default - private Object defaultValue = null; - - /** - * The list of options for this setting. - */ - @Default - private List optionValues = Collections.emptyList(); - - /** - * The validator used for validating configuration value. - */ - @Default - private Validator validator = NullValidator.of(); - - /** - * The key-group to group settings together. - */ - @Default - private ConfigKeyGroup group = ConfigKeyGroup.DEFAULT; - - /** - * The order of the setting in the key-group. - */ - @Default - private int orderInGroup = Integer.MIN_VALUE; - - /** - * The list of settings dependents on this setting. - */ - @Default - private List dependents = Collections.emptyList(); - - /** - * Whether this setting is deprecated or not. - */ - @Default - private boolean deprecated = false; - - /** - * The config key that deprecates this key. - */ - @Default - private String deprecatedByConfigKey = ""; - - /** - * The version when this settings was deprecated. - */ - @Default - private String deprecatedSince = ""; - - /** - * The version when this setting was introduced. - */ - @Default - private String since = ""; -} -``` - -2. Introduced `ConfigKeyGroup` for grouping configuration keys together. - -```java -public class ConfigKeyGroup { - /** - * Name of the key group. - */ - private String name; - - /** - * Description of the key group. - */ - @Default - private String description = ""; - - /** - * The list of sub key-groups of this key group. - */ - @Default - private List children = Collections.emptyList(); - - /** - * The order of the key-group in a configuration. - */ - @Default - private int order = Integer.MIN_VALUE; -} -``` - -### Proposed Changes - -Besides introducing `ConfigKey` and `ConfigKeyGroup`, this BP will also introduce a class -`ConfigDef` - it defines the keys for a configuration. - -The `ConfigDef` will be generated via `ConfigDef.of(Configuration.class)`. It will retrieve -all the static fields of `ConfigKey` defined in the configuration class and build the configuration -definition. - -The `ConfigDef` will also provide a `save` method for saving the configuration definition -as a configuration file. - -### Example - -Following is an example how to use `ConfigKey` and `ConfigKeyGroup` to organize -configuration settings. - -```java -// Ledger Storage Settings - -private static final ConfigKeyGroup GROUP_LEDGER_STORAGE = ConfigKeyGroup.builder("ledgerstorage") - .description("Ledger Storage related settings") - .order(10) // place a place holder here - .build(); - -protected static final String LEDGER_STORAGE_CLASS = "ledgerStorageClass"; -protected static final ConfigKey LEDGER_STORAGE_CLASS_KEY = ConfigKey.builder(LEDGER_STORAGE_CLASS) - .type(Type.CLASS) - .description("Ledger storage implementation class") - .defaultValue(SortedLedgerStorage.class.getName()) - .optionValues(Lists.newArrayList( - InterleavedLedgerStorage.class.getName(), - SortedLedgerStorage.class.getName(), - DbLedgerStorage.class.getName() - )) - .validator(ClassValidator.of(LedgerStorage.class)) - .group(GROUP_LEDGER_STORAGE) - .build(); -``` - -Example on how to generate the `ConfigDef` and use the configuration definition to -validate if a configuration instance is valid. - -```java -// generate config def -ConfigDef configDef = ConfigDef.of(ServerConfiguration.class); -try { - configDef.validate(this); -} catch (ConfigException e) { - throw new ConfigurationException(e.getMessage(), e.getCause()); -} -``` - -Example on how to save the configuration definition to a configuration file. - -```java -ConfigDef configDef = ConfigDef.of(TestConfig2.class); -String savedConf; -try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - configDef.save(baos); - savedConf = baos.toString(); -} -``` - -### Compatibility, Deprecation, and Migration Plan - -It only changes the way how we organize configuration settings and how we document them. -It doesn't change the public interfaces for existing configuration. So there is nothing -to deprecate and migrate. - -### Test Plan - -Existing testing is good enough to cover code changes. No new tests are needed. - -### Rejected Alternatives - -Alternatively, we have to manually maintain the configuration files and update each time -when a new configuration setting is added. diff --git a/site/bps/BP-38-bookie-endpoint-discovery.md b/site/bps/BP-38-bookie-endpoint-discovery.md deleted file mode 100644 index 80ad072d8da..00000000000 --- a/site/bps/BP-38-bookie-endpoint-discovery.md +++ /dev/null @@ -1,154 +0,0 @@ ---- -title: "BP-38: Publish Bookie Service Info on Metadata Service" -issue: https://github.com/apache/bookkeeper/2215 -state: 'Accepted' -release: "4.11.0" ---- - -### Motivation - -Bookie server exposes several services and some of them are optional: the binary BookKeeper protocol endpoint, the HTTP service, the StreamStorage service, a Metrics endpoint. - -Currently (in BookKeeper 4.10) the client can only discover the main Bookie endpoint: -the main BookKeeper binary RPC service. -Discovery of the TCP address is implicit, because the *id* of the bookie is made of the same host:port that point to the TCP address of the Bookie service. - -With this proposal we are now introducing a way for the Bookie to advertise the services it exposes, basically the Bookie will be able to store on the Metadata Service a structure that describes the list of *available services*. - -We will also allow to publish a set of additional unstructured properties in form of a key-value pair that will be useful for futher implementations. - -This information will be also useful for Monitoring and Management services as it will enable full discovery of the capabilities of all of the Bookies in a cluster just by having read access to the Metadata Service. - -### Public Interfaces - -On the Registration API, we introduce a new data structure that describes the services -exposed by a Bookie: - -``` -interface BookieServiceInfo { - class Endpoint { - String name; - String hostname; - int port; - String protocol; // "bookie-rpc", "http", "https".... - } - List endpoints; - Map properties; // additional properties -} - -``` - -In RegistrationClient interface we expose a new method: - -``` -CompletableFuture> getBookieServiceInfo(String bookieId) -``` - -The client can derive bookieId from a BookieSocketAddress. He can access the list of available bookies using **RegistrationClient#getAllBookies()** and then use this new method to get the details of the services exposed by the Bookie. - -On the Bookie side we change the RegistrationManager interface in order to let the Bookie -advertise the services: - -in RegistrationManager class the **registerBookie** method -``` -void registerBookie(String bookieId, boolean readOnly) -``` - -becomes - -``` -void registerBookie(String bookieId, boolean readOnly, BookieServiceInfo bookieServieInfo) -``` - -It will be up to the implementation of RegistrationManager and RegistrationClient to serialize -the BookieServiceInfo structure. - -For the ZooKeeper based implementation we are going to store such information in Protobuf binary format, as currently this is the format used for every other kind of metadata (the example here is in JSON-like for readability purposes): - -``` -{ - "endpoints": [ - { - "name": "bookie", - "hostname": "localhost", - "port": 3181, - "protocol": "bookie-rpc" - }, - { - "name": "bookie-http-server", - "hostname": "localhost", - "port": 8080, - "protocol": "http" - } - ], - properties { - "property": "value", - "property2": "value2" - } -} -``` -Such information will be stored inside the '/REGISTRATIONPATH/available' znode (or /REGISTRATIONPATH/available/readonly' in case of readonly bookie), these paths are the same used in 4.10, but in 4.10 we are writing empty znodes. - -The rationale around this choice is that the client is already using these znodes in order to discover available bookies services. - -It is possible that such endpoint information change during the lifetime of a Bookie, like after rebooting a machine and the machine gets a new network address. - -It is out of the scope of this proposal to change semantics of ledger metadata, in which we are currently storing directly the network address of the bookies, but with this proposal we are preparing the way to have an indirection and separate the concept of Bookie ID from the Network address. - -**Endpoint structure** -- **name**: this is an id for the service type, like 'bookie' -- **hostname**: the hostname (or a raw IP address) -- **port**: the TCP port (int) -- **protocol**: the type of service (bookie-rpc, http, https) - -**Well known additional properties** -- **autorecovery.enabled**: 'true' case of the presence of the auto recovery daemon - -The GRPC service would use this mechanism as well. - -#### Which kind of properties should be stored in BookieServiceInfo ? - -We should store here only the minimal set of information useful to reach the bookie in an efficient way. -So we are not storing on the Metadata Service information about the internal state of the server: if you know the address of an HTTP endpoint you can use the REST API to query the Bookie for its state. - -These properties may change during the lifetime of a Bookie, think about a configuration (change network address) or a dynamic assigned DNS name. - -It is better not to expose the version of the Bookie, if the client wants to use particular features of the Bookie this should be implemented on the protocol itself, not just by using the version. The version of the Bookie could be exposed on the HTTP endpoint. - -### Proposed Changes - -See the 'Public interfaces' section. - -### Compatibility, Deprecation, and Migration Plan - -The proposed change will be compatible with all existing clients. - -In case a new client is reading an empty znode it will assume a default configuration with a single 'bookie-rpc' endpoint, like in this example: - -{ - "endpoints": [ - { - "name": "bookie", - "hostname": "hostname-from-bookieid", - "port": port-from-bookieid, - "protocol": "bookie-rpc" - } - ] -} - -This information is enough in order to use the RPC service. - -### Test Plan - -New unit tests will be added to cover all of the code changes. -No need for additional integration tests. - -### Rejected Alternatives - -#### Adding a new set of znodes -For the ZooKeeper implementation we are not introducing a new znode to store BookieServiceInfo. Adding such new node will increase complexity and the usage of resources on ZooKeeper. - -#### Storing information inside the Cookie -The *Cookie* stores information about the *identity* of the bookie and it is expected not to change. -It is exceptional to rewrite the Cookie, like when adding a new data directory. -In some environments it is common to have a new network address after a restart or to change the configuration and enable a new service or feature, and you cannot rewrite the Cookie at each restart: by design every change to the Cookie should be manual or controlled by an external entity other than the Bookie itself. diff --git a/site/bps/BP-41-bookieid.md b/site/bps/BP-41-bookieid.md deleted file mode 100644 index 0514c664099..00000000000 --- a/site/bps/BP-41-bookieid.md +++ /dev/null @@ -1,185 +0,0 @@ ---- -title: "BP-41: Separate BookieId from Bookie Network Address" -issue: https://github.com/apache/bookkeeper/issues/2396 -state: 'Draft' -release: "4.12.0" ---- - -### Motivation - -We want to separate the concepts of **BookieId** from **BookieSocketAddress**. - -Currently (up to 4.11.x) there is a too strict coupling from the ID of a Bookie (**BookieId**) and its network location (**BookieSocketAddress**). - -The **LedgerMetadata** structure contains the location of the entries of a ledger, and it stores BookieSocketAddresses (simply a hostname:port or ip:port pair). -The client uses this information to connect to the bookies and retrieve ledger data. - -So *the identity of a bookie is bound to the network address* of the primary endpoint of the Bookie: the '**bookie-rpc**' endpoint in terms of [BP-38](../BP-38-bookie-endpoint-discovery/) - -Limits of current version, because: - -- You cannot easily change the network address of a Bookie: a manual intervention is needed. -- The Bookie cannot have a **dynamic network address** or DNS name. -- You cannot use a custom Id for the bookie, something that is **meaningful** in the context of the deployment of the bookie. -- In case of future implementations that will open **multiple endpoints** on the bookie it is not possible to decide which endpoint should be used as BookieId. - -This proposal addresses these problems by proposing to separate the concept of **BookieId** from **BookieSocketAddress**. - -We will have to introduce a little break in the Client API, in order to switch from using BookieSocketAddress to a more opaque BookieId. - -Fortunately we will be able to keep compatibility with old clients and old bookies are far as the Bookie continues to use a BookieId that looks like a BookieSocketAddress. -See the paragraphs below for the details. - -### Public Interfaces - -We are introducing a new class BookieId that is simply a wrapper for a String. - -``` -final class BookieId { - private final String bookieId; - public String toString() { - return bookieId; - } - public static BookieId parse(String bookieId) { - // validation omitted... - return new BookieId(bookieId); - } -} -``` - -Having a class instead of a simple String is better because it provides a strongly typed API. - -The LedgerMetadata class will be changed to use BookieId instead of BookieSocketAddress. -This will break source and binary compatibility for Java clients, applications that use LedgerMetadata (usually for debug or for tools) will have to be recompiled. - -The serialized representation of a BookieSocketAddress, both for LedgerMetadata and Bookie Registration, is a simple String on ZooKeeper: this change is not about the format of data stored on Metadata Service. - -It is simply a pure refactor of Java interfaces. - -We have to introduce an internal API, **BookieAddressResolver**, that maps a *BookieId* to a *BookieSocketAddress*: the the client connectes to a Bookie it looks up the **current network address** using BookieAddressResolver. - -``` -interface BookieAddressResolver { - BookieSocketAddress resolve(BookieId id); -} -``` - -Initially it is not expected that the user provides a custom implementation of BookieAddressResolver. - - -It is expected that the implementation of this interface coordinates with the BookieWatcher and the RegistrationDriver in order to: -- map BookieId to BookieSocketAddress using `getBookieServiceInfo(BookieId id)` API -- cache efficiently this mapping in order to do not have a significant impact on the hot paths (reads and writes), and to save resources on the Metadata Service - -We provide an utility method BookieSocketAddress#toBookieId that helps particularly in test cases, this method simply returns a BookieId -built by the serialized representation of the BookieSocketAddress (hostname:port or ip:port) - -``` -final class BookieSocketAddress { - .... - BookieId toBookieId() { - return BookieId.parse(this.toString()); - } -} -``` - -The RegistrationClient and RegistrationManager interfaces will be refactored to use BookiId instead of String and BookieSocketAddress. - -The Bookie itself will expose an API to return the current BookieSocketAddress and current BookieId, this is useful for tests and for the RegistrationManager. - -The EnsemblePlacementPolicy and the BookieWatcher will deal with BookieIds and not with BookieSocketAddresses. - -The implementations of EnsemblePlacementPolicy that are aware of the network location of the Bookies will need to have access to the -BookieAddressResolver, in order to map a BookieId to the BookieSocketAddress and the BookieSocketAddress to the network location. - - -### Details on the proposed Changes - -#### BookieId validation - -The BookieId is a non empty string that can contain: -- ASCII digits and letters ([a-zA-Z9-0]) -- the colon character (':') -- the dash character ('-') -- the dot character ('.') - -#### BookKeeper Client Side Changes - -See the 'Public interfaces' section. - -On the client side code it will be clearer when we are dealing with BookieId, and basically the client API won't deal with network addresses anymore. -This change will be visible both on the legacy LedgerHandle API and on the new WriteHandle/ReadHandle API, basically because the new API is only a wrapper over the LedgerHandle API. - -When the BookKeeper client connects to a bookie (see **PerChannelBookieClient**) it uses the BookieAddressResolver interface to get the current BookieSocketAddress of the Bookie. -The key of the Connection Pool inside the BookieClient will be BookieId and no more BookieSocketAddress. - -#### Disabling BookieAddressResolver - -Using the BookieServiceInfo abstraction needs additional accesses to the Metadata Service (usually ZooKeeper) and this comes with a cost especially during the bootstrap of the client, because you have to resolve the address of every Bookie you are going to write to or to read from. - -We add a flag to disable the BookieAddressResolver, without this feature the client will be able only to connect to Bookies with the legacy BookieId. -In this case the BookieAddressResolver implementation will be a simple conversion from the BookieId, assuming the 4.11 format hostname:port. - -``` -enableBookieAddressResolver=true -``` -The *enableBookieAddressResolver* flag is used by the Client, by the Auditor and by all of the tools and it is enabled by default. - -#### Handling the Local Bookie Node in EnsemblePlacementPolicy -Unfortunately thru the codebase we used sometimes dummy BookieId that are not mapped to real Bookies, this happens in the EnsamblePlacementPolicies in which we create a BookieId for the 'local node' and using TCP port 0. In this case we have to implement a fallback dummy resolution that created a BookieSocketAddress without using the MetadataService - -#### Bookie Side Changes - -On the Bookie we introduce **a configuration option** (bookieid) to set a custom bookie id. -If you set this option then the Bookie will advertise itself on MetadataService using the configured id, and publish the 'bookie-rpc' endpoint as configured by -the **advertisedAddress** configuration option and the other network related options. -This BookieId will be present only in the configuration file and it is the key to lookup the *Cookie* on the MetadataService. -Inadvertently changing the BookieId will prevent the Bookie to boot as it won't find a matching Cookie. -There is no need to store the BookieId on the cookie or persist it on the local storage (ledgers, indexes or journal directories). - -#### Auditor and Replication Changes - -The Auditor deals with LedgerMetadata and so it will simply work with BookieIds and not with BookieSocketAddress. -When the Auditor needs to connect to a Bookie it will use the BookieAddressResolver to get the current address of the Bookie. - -#### Bookie Garbage Collection Changes - -The Bookie decides to reclaim space by looking into LedgerMetadata and checking that a given ledger does not exist anymore. -It will use its own local BookieId instead of the BookieSocketAddress as expected. - -#### Tools changes -All of the tools that deal with LedgerMetadata will use BookieId instead of BookieSocketAddress, in general this fact will allow to use free forn BookieIDs, -instead of hostname:port pairs (we had validations on tools that helped the user to use always BookieIds in hostname:port form). - -#### REST API Changes -In the REST API we will deal with BookieIds and not with BookieSocketAddresses anymore, the change will be straighforward and compatible with current API. -When new custom BookieIDs will be used then they will appear on the REST API as well, but this will be expected by users. - - -### Compatibility, Deprecation, and Migration Plan - -The proposed change will be compatible with all existing clients and bookies as far as you still use BookieIds in the hostname:port form and to not use a custom BookieId. -The Bookie by default will continue to use as BookieID a compatible value computed exactly as in version 4.11. -Incompatibility will start as soon as you enable custom BookieIDs on the bookies, from that point clients and old Auditors won't be able to deal with new bookies. -New clients will always be able to connect and use legacy bookies. - -Custom EnsemblePlacementPolicies must be adapted to the new interfaces but the change will usually as simple as just replacing BookieSocketAdress with BookieId. -No need to change address to rack mapping scripts, as they will still deal with raw DNS hostnames and not with BookieIds. - -### Test Plan - -New unit tests will be added to cover all of the code changes. -No need for additional integration tests. - -### Rejected Alternatives - -#### Make BookieSocketAddress an abstract class - -In order to preserve most of the binary compatibility in the Java client we could still keep BookieSocketAddress class in LedgerMetadata and have some "GenericBookieSocketAddress" and "PhysicalBookieSocketAddress" implementations. -But this way it won't be easy to understand where we are using a "bookie id" and when we are referring to a network address. -The BookieAddressResolver interface would be needed anyway and it should deal with pure BookieIds and BookieSocketAddress instance that are already resolved to -a network address. - -#### Force a specific format (like UUID) to custom BookieId -The is no need to force the BookieId to use a fixed format, like a UUID or other form of standard ID scheme. -Probably new IDs will include the region/availability zone information in order to simplify EnsemblePlacement policies (no more need to pass from DNS to switch mappers) and we cannot know now all of the usages of this parameter. diff --git a/site/bps/BP-42-new-api-list-ledgers.md b/site/bps/BP-42-new-api-list-ledgers.md deleted file mode 100644 index 2262ae287cc..00000000000 --- a/site/bps/BP-42-new-api-list-ledgers.md +++ /dev/null @@ -1,94 +0,0 @@ ---- -title: "BP-42: New Client API - list ledgers" -issue: https://github.com/apache/bookkeeper/issues/2422 -state: "Under Discussion" -release: "N/A" ---- - -### Motivation - -The new Client API (`org.apache.bookkeeper.client.api.BookKeeper`) aims to replace obsolete BookKeeper API but some features are not implemented yet, like the functionalities provided by `BookKeeperAdmin`. -For example, it does not expose a method to list available ledgers, comparable to `BookKeeperAdmin#listLedgers()`. - -#### Ledgers listing -The goal here is to extend the Client API for listing ledgers. Moreover current method `BookKeeperAdmin#listLedgers()` does not report errors from the metadata driver; for instance, if an IOException occurs during iterator flow, the error is not visible to the caller and the iteration is stopped (e.g. hasNext will return false). However there is no intention to change this behaviour in this proposal. - -#### Simpler access to LedgerMetadata -The goal here is to streamline the access to `LedgerMetadata`, directly from BookKeeper interface. - -#### Ledger id inside LedgerMetadata -Currently there is no `ledgerId` property inside `LedgerMetadata` interface, this can be helpful in some contexts. - - -### Public Interfaces - -This proposal adds new interfaces to `org.apache.bookkeeper.client.api` package, similar to `org.apache.bookkeeper.client.api.BookKeeper` methods. - - // new interface - interface LedgersIterator { - - boolean hasNext() throws IOException; - - long next() throws IOException; - } - - // new interface - interface ListLedgersResult extends AutoCloseable { - - LedgersIterator iterator(); - - Iterable toIterable(); - } - - // new interface - interface ListLedgersResultBuilder extends OpBuilder{ - - // empty now, maybe some filters in future - } - - interface BookKeeper { - - .... - - ListLedgersResultBuilder newListLedgersOp(); - - CompleatableFuture getLedgerMetadata(long ledgerId); - - } - - interface LedgerMetadata { - - .... - - long getLedgerId(); - - } - -### Proposed Changes - -#### Ledgers listing - -The implementation is pretty similar to `BookKeeperAdmin#listLedgers()` but there are few enhancements: -- Handle metadata driver errors, since the IOException is directly thrown up to caller, allowing user to handle network errors in a more suitable way. -- Leave the possibility to restrict/filter returned ledgers in future, without API breaking changes -- Dispose some resources needed to retrieve ledgers (`ListLedgersResult` extends `AutoCloseable`)) (will be empty for current implementations) - -The implementation will be the same used in BookKeeperAdmin, iterating over `LedgerRangeIterator`, which already handles ledgers search properly. - -#### Simpler access to LedgerMetadata -The implementation will use LedgerManager to retrieve metadata for a specified ledgerId. - -#### Ledger id inside LedgerMetadata -Each time a LedgerMetadata instance is created, the ledgerId is known, so it is trivial to set it in the instance. - -### Compatibility, Deprecation, and Migration Plan - -No impact for current API, the proposal does not aim to explicit deprecate `BookKeeperAdmin#listLedgers()` method. - -### Test Plan - -This proposal needs only new unit tests, other tests must continue pass without any changes. - -### Rejected Alternatives - -An alternative could be creates a new API similar to BookKeeperAdmin but it is better to invest enhancing `org.apache.bookkeeper.client.api.BookKeeper` API. diff --git a/site/bps/BP-43-gradle-migration.md b/site/bps/BP-43-gradle-migration.md deleted file mode 100644 index dbe6f07c67a..00000000000 --- a/site/bps/BP-43-gradle-migration.md +++ /dev/null @@ -1,63 +0,0 @@ ---- -title: "BP-43: Gradle migration" -issue: https://github.com/apache/bookkeeper/issues/2640 -state: "Under Discussion" -release: "N/A" ---- - -### Motivation -- Current maven based build system is very slow. It takes about 4m30 to execute - mvn clean package -DskipTests while a gradle counterpart - `./gradlew clean jar` completes in 40 seconds on the same hardware. Using the gradle build cache. -- Incremental build- Gradle provides better incremental build support. Subsequent builds run in very little time. -- Caching of tests execution - Gradle build cache also keeps track of test successes with respect to the dependencies and can do test retries of just what has failed. -- Better handling of multi module projects - Bookkeeper is a multi module project where bookkeeper-server module for an example depends on following modules - - bookkeeper-stats - - cpu-affinity - - bookkeeper-common - - bookkeeper-common-allocator - - bookkeeper-http:http-server - - bookkeeper-proto - - circe-checksum - - bookkeeper-tools-framework - - If change is made on any of the above mentioned dependent modules, building module `bookkeeper-server` will pick up those changes. - While in case of maven changed modules had to be built explicitly in order for the changes to be picked up. - For instance If a change is made in `bookkeeper-common` with gradle - `./gradlew bookkeeper-server:test` will pick up those changes and run tests based on those changes. - While on the other hands for maven `mvn install bookkeeper-common` had to run before running mvn build `bookkeeper-server` - -### Public Interfaces -N/A -### Proposed Changes -- Every module and sub module which are currently being built using maven would be built using gradle. -- Run all the unit tests, functional tests etc using gradle. -- Integrating gradle build to CI/CD pipeline -- Build ASF release capability using gradle -- Remove Maven build - -### Compatibility, Deprecation, and Migration Plan -Since this is migration to the new build system using gradle, it should be done at phases. - -- Phase 1: After phase one Bookkeeper should be able to build and run unit tests and integration tests using gradle. -After the end of this phase CI/CD pipeline for BookKeeper still remains to be the existing one. -A sample PR may look like this. - - Timeline: 1.5 Week -- Phase 2: Spin off new CI job for gradle which would run in parallel with existing maven based one. -In this phase any dependency upgrade should be done at both maven as well as gradle based build system. - - Timeline: 1 Week - -- Phase 3: Gradle based build should be enhanced to do full CI/CD including release and a minor release should be done completely using gradle based CI/CD pipeline. -Update release docuementation as how to do release using gradle based pipeline, update information on how to setup major IDE such as IntelliJ. - - Timeline: 1 Week -- Phase 4: Shut down maven based CI/CD pipeline. Remove all pom.xml files - - Timeline: Based on confidence on the new system by the community - -### Test Plan - -Compare test results with maven based build vs gradle based build to verify that gradle based -build is running exact same set of tests as maven based build. - -### Rejected Alternatives - -N/A \ No newline at end of file diff --git a/site/bps/BP-44-use-metrics.md b/site/bps/BP-44-use-metrics.md deleted file mode 100644 index e09be5af144..00000000000 --- a/site/bps/BP-44-use-metrics.md +++ /dev/null @@ -1,164 +0,0 @@ ---- -title: "BP-44: USE metrics" -issue: https://github.com/apache/bookkeeper/issues/2834 -state: "Under Discussion" -release: "N/A" ---- - -### Motivation -Based on our experience (at Splunk) running many BookKeeper clusters in production, from very small to very large deployments (in terms of number of bookies, size of VMs and load) we have identified a number of short-comings with the current BookKeeper metrics that make it harder than it should be to identify bottlenecks in performance. The [USE method](https://www.brendangregg.com/usemethod.html) (Utilization, Saturation, Errors) is an effective strategy for diagnosing where bottlenecks in a system lie but the current metrics do not always expose metrics related to utilization and saturation. Also, even if you have a good mental model for how BookKeeper works internally, there are blindspots in the metrics that make it difficult to know what is happening at times. - -Finally, many of the metrics are aggregated, such as journal and DbLedgerStorage. When these components are configured with multiple directories, it is currently not possible to inspect the metrics of only a single journal or DbLedgerStorage instance. One bad volume can be hard to identify. - -This BP proposes a number of improvements to help identify performance issues in production. The objective is for a human operator to rapidly see where a bottleneck exists and where it exists by using utilization and saturation metrics. - -#### Utilization -Ultimately all work in a bookie is performed by threads. Some operations are performed by only a single thread while others are spread out over multiple threads. Some threads play a core role in the critical path of reads and writes and when fully utilized indicate a bottleneck. We propose that each thread reports the time that it is busy which will allow time utilization to be calculated. Because all file i/o is synchronous, thread utilization is a very close approximation of resource utilization for a bookie. For a thread whose sole role is file i/o, if it is 100% utilized then it can take little to no more work. - -For example, if the Force Write thread is reporting 100% time utilization we’ll know that it has become a bottleneck and to look further into the force write metrics. - -Likewise, some operations are in the critical path that are executed across multiple threads. For example, with DbLedgerStorage, flushes are in the critical path for writes and the work is performed across at least two threads (Sync thread, DbStorage thread). Knowing the time utilization of the Sync Thread and the DbStorage thread is useful, but knowing the time utilization of flushes as a whole (regardless of thread) is even more useful in this particular case. Once flushes are being performed 100% of the time we know that we have a write bottleneck in DbLedgerStorage. - -So utilization metrics of both threads and operations can be extremely useful and often combining the two give even more insight. - -#### Saturation -Metrics that indicate saturation are things like rejected requests, full queues/caches. We need to ensure that there are no blindspots for saturation metrics. - -Once we start getting saturation indicators we have strong signals of a bottleneck existing. It may be a high level metric (rejected requests) which only tells us we have a problem, or a component level metric (journal queue length) that tells us the locality of a bottleneck. We also need to know things like the queue size bounds in order to detect when a component is fully saturated, so emitting metrics for certain configurations is useful. - -#### Component and Thread Labels -Work is parallelized by thread pools and by using multiple directories that allow for multiple journal instances and multiple ledger storage instances. We have seen in production cases where a single bad volume has a large negative impact on BookKeeper performance. Being able to pick apart metrics by the thread and component instance can be extremely helpful in diagnosing issues that only affect one journal/ledger storage instance or a subset of threads. - -#### Sub-Operation Time -In some cases a single operation can consist of multiple smaller operations. For example, a DbLedgerStorage flush involves flushing two RocksDB instances and entry log files. The only way to currently confirm whether RocksDB has gotten into a bad state is by profiling on the machine/pod. - -Adding extra metrics to cover sub-operations can help to diagnose or rule-out an issue quickly. - -#### Read Cache Thrashing -Read cache thrashing can have a huge negative impact on bookie performance. We have most of the metrics already for detecting when it happens except for one problem: we don’t report read and write cache hits/misses separately. - -When we separate out cache hits/misses, we can start calculating the actual read cache hit-to-miss rate and compare that to the readahead batch sizes and calculate that the miss rate is too high (indicating thrashing). - -### The Current State of Affairs - -#### Utilization -The OpsStatsLogger gives us visibility into some time utilization metrics by using the “_sum” suffix metric for latency based OpStatsLogger metrics. We can’t see the utilization of each thread, just the thread pool as a whole. Knowing the number of threads in the pool allows us to calculate the % time utilization (given that each cluster may have a different size of thread pool this is inconvenient). - -Operations such as journal flushes, journal force writes and DbLedgerStorage already record latencies with OpStatsLoggers which allows us to calculate utilization of IO operations as long as we know the number of journal/ledger storage directories (again, inconvenient). - -Blindspots: -- Write/read/high priority thread pool -- Utilization of each thread -- Journal - - Drill-down by journal instance - - Utilization of the threads (they do other work such as taking work from queues and placing work on queues) -- Ledger Storage - - Drill-down by ledger storage instance - - Where is time spent on reads? Locations index, read from entry log, readahead? - - Where is time spent on flushes? Ledger index, locations index, entry log files? - -#### Saturation -There are some existing indicators of saturation, such as the thread pool task queue lengths, journal queue length, force write queue length, DbLedgerStorage rejected writes. But there are some blindspots: - -- Requests rejected due to full thread pool task queues are logged as request failures. More helpful is a specific metric for rejected read and write requests -- Queue lengths are in aggregate, need to be able to drill-down by thread/journal/ledger storage instance. -- We don’t have metrics for things like maximum queue/cache sizes so we can’t calculate when a queue or a cache is full. -- Netty can become saturated, tracking the number of unwritable channels, bytes for unwritable can help diagnose Netty saturation - - -### Public Interfaces -N/A -### Proposed Changes - -#### Thread scoped OpStatsLogger and Counter -Add a new “thread scoped” variant of the OpStatsLogger and the Counter where each adds labels: -- threadPool (or just the thread name for lone threads such as a journal thread) -- thread (the thread ordinal, for example a read thread pool with four threads, they are numbers 0-3) - -The following methods are added to the StatsLogger interface: -``` -OpStatsLogger getThreadScopedOpStatsLogger(String name) -Counter getThreadScopedCounter(String name) -``` - -The OpStatsLogger and Counter interfaces remain unchanged. - -For each thread that mutates a thread scoped OpStatsLogger, internally it will end up registering one OpStatsLogger per thread. For example, with four read threads, then the following metrics for successful task executions would be created: - -``` -bookkeeper_server_BookieReadThreadPool_task_execution_sum{success="true", thread="0", threadPool="BookieReadThreadPool"} -bookkeeper_server_BookieReadThreadPool_task_execution_sum{success="true", thread="1", threadPool="BookieReadThreadPool"} -bookkeeper_server_BookieReadThreadPool_task_execution_sum{success="true", thread="2", threadPool="BookieReadThreadPool"} -bookkeeper_server_BookieReadThreadPool_task_execution_sum{success="true", thread="3", threadPool="BookieReadThreadPool"} -``` - -In order for thread scoped metrics to know which thread pool and thread ordinal the metrics of each thread should be labelled with, a thread registry is maintained. After each thread is started, the first work it performs is registering itself with a thread registry object that maps thread id -> {threadPool, thread}. In some cases this work can be performed outside of the thread itself, such as in a thread factory. - -Each thread scoped metric registers per-thread metrics lazily. For example, when a thread scoped counter is incremented by a given thread, the first time that occurs on that given thread the thread labels are retrieved from the thread registry and a counter for that thread is created with the right labels and registered with the provider. In the case that the calling thread is unregistered, a default metric is used. Per thread OpStatsLoggers/Counters are not visible externally and are stored inside the thread scoped metric using a thread-local variable. - -With these new variants, we can now replace calls to `getCounter` with `getThreadScopedCounter` and `getOpsStatsLogger` with `getThreadScopedOpsStatsLogger` for the metrics where we want to be able to drill down by thread pool and thread. - -#### How to Report Time Spent (For utilization calculation) -We want to be able to report time spent by each thread as a whole, but also on operations and sub-operations. The simplest way to do this is simply measure elapsed time by taking a System.nanoTime() before the work and then logging the elapsed time via an OpStatsLogger or a Counter. - -One challenge regarding this simple method is when operations take a long time, like a journal fsync or a ledger flush on a heavily overloaded bookie. Long execution times can cause per second calculations to go to 0 then to spike to very high levels (above 100% utilization). This can be mitigated by using larger windows (like 1 minute windows rather than 1 second windows) and heavily loaded systems don’t tend to fluctuate too much from second to second. - -##### Discarded time measurement alternatives: -Use a sampling technique: too costly and complex. -Use proc filesystem: not portable, only thread level utilization metrics. - -#### Per Component Instance labels -All journal metrics get the additional label of `journalIndex`. -All DbLedgerStorage metrics get the additional label of `ledgerDir`. - -This does not prevent aggregating metrics of the journal or DbLedgerStorage as a whole but does allow for drill-down. - -### New Metrics List - -The following new metrics to be added to cover saturation/utilization or blindspots. - -Saturation: -- bookkeeper_server_ADD_ENTRY_REJECTED (thread scoped counter) -- bookkeeper_server_READ_ENTRY_REJECTED (thread scoped counter) - -Thread time utilization: -- bookie_sync_thread_time (thread scoped counter) -- bookie_journal_journal_thread_time (thread scoped counter, journalIndex label) -- bookie_journal_force_write_thread_time (thread scoped counter, journalIndex label) -- bookie_journal_callback_thread_time (thread scoped counter, journalIndex label) -- bookie_db_storage_thread_time (thread scoped counter, ledgerDir label) - -Operation time spent. Counters for high frequency ops and OpStatsLoggers for less frequent ops: -- bookie_read_locations_index_time (thread scoped counter, ledgerDir label) -- bookie_read_entrylog_time (thread scoped counter, ledgerDir label) -- bookie_readahead_time (thread scoped counter, ledgerDir label) -- bookie_flush_entrylog (thread scoped OpStatsLogger, ledgerDir label) -- bookie_flush_locations_index(thread scoped OpStatsLogger, ledgerDir label) -- bookie_flush_ledger_index(thread scoped OpStatsLogger, ledgerDir label) - -Queue/cache max sizes, number of dirs, thread counts (useful for calculations such as current queue length vs max size and useful for operators to see on dashboards). All gauges based on config values or calculated values such as write cache being 25% of direct memory: -- bookkeeper_server__threads -- bookkeeper_server__max_queue_size -- bookie_JOURNAL_DIRS -- bookie_ledger_num_dirs -- bookie_JOURNAL_QUEUE_MAX_SIZE -- bookie_write_cache_max_size - -Other: -- bookie_write_cache_hits (thread scoped counter, ledgerDir label) -- bookie_write_cache_misses (thread scoped counter, ledgerDir label) - -#### Breaking Changes -Some proposed changes are breaking: -1. Remove thread ordinal from OrderedExecutor based metric names and use the label “thread” instead. This makes it easier to work with when using modern observability tooling and inline with the rest of the proposed changes. -2. Replace cache hit and cache miss OpStatsLoggers with Counters. OpStatsLoggers are relatively expensive and cost far more CPU cycle than the actual accesses to the caches themselves. Latency for cache hits/misses is extremely low and recording these latencies is not worth the cost. Counters are far cheaper. - -### Compatibility, Deprecation, and Migration Plan -Two changes are breaking and will cause existing dashboards or alerts based on the affected metrics to stop working. The affected release should document this changes in order to warn operators of this impact. - -### Test Plan -These changes have been tested extensively using various sized deployments (including multi journal and ledger storage configurations) and loads to ensure that the utilization and saturation metrics are accurate and useful. - -### Rejected Alternatives - -Time reporting alternatives discussed above. \ No newline at end of file diff --git a/site/bps/BP-46-run-without-journal.md b/site/bps/BP-46-run-without-journal.md deleted file mode 100644 index 83030356db2..00000000000 --- a/site/bps/BP-46-run-without-journal.md +++ /dev/null @@ -1,205 +0,0 @@ ---- -title: "BP-46: Running without the journal" -issue: https://github.com/apache/bookkeeper/2705 -state: "Under Discussion" -release: "N/A" ---- - -### Motivation - -The journal allows for fast add operations that provide strong data safety guarantees. An add operation is only acked to a client once written to the journal and an fsync performed. This however means that every entry must be written twice: once to the journal and once to an entry log file. - -This double write increases the cost of ownership as more disks must be provisioned to service requests and makes disk provisioning more complex (separating journal from entry log writes onto separate disks). Running without the journal would halve the disk IO required (ignoring indexes) thereby reducing costs and simplifying provisioning. - -However, running without the journal would introduce data consistency problems as the BookKeeper Replication Protocol requires that all writes are persistent for correctness. Running without the journal introduces the possibility of lost writes. In order to continue to offer strong data safety and support running without the journal, changes to the protocol are required. - -### A note on Response Codes - -The following categories are relevant: - -- Positive: OK -- Explicit Negative: NoSuchEntry/NoSuchLedger -- Unknown: Any other non-success response that is not an explicit negative. - -For correctness explicit negatives must be treated differently than other errors. - -### A note on Quorums - -In order to explain the protocol changes, it is useful to first consider how quorums are used for safety. We have the following relevant quorums: - -- Single bookie (S) -- Ack quorum (AQ) -- Write quorum (WQ) -- Quorum Coverage (QC) where QC = (WQ - AQ) + 1 -- Ensemble Coverage (EC) where EC = (E - AQ) + 1 -- Whole Ensemble - -Quorum Coverage (QC) and Ensemble Coverage (EC) are both defined by the following, only the cohorts differ: - -- A given property is satisfied by at least one bookie from every possible ack quorum within the cohort. -- There exists no ack quorum of bookies that do not satisfy the property within the cohort. - -For QC, the cohort is the writeset of a given entry, and therefore QC is only used when we need guarantees regarding a single entry. For EC, the cohort is the ensemble of bookies of the current fragment. EC is required when we need a guarantee across an entire fragment. - -For example: - -- For fencing, we need to ensure that no AQ of bookies is unfenced before starting the read/write phase of recovery. This is true once EC successful fencing responses have been received. -- For a recovery read, a read is only negative once we know that no AQ of bookies could exist that might have the entry. Doing otherwise could truncate committed entries from a ledger. A read is negative once NoSuchEntry responses reach QC. - -Different protocol actions require different quorums: - -- Add entry: AQ success responses -- Read entry: - - Positive when positive response from a single bookie - - Negative when explicit negative from all bookies - - Unknown: when at least one unknown and no positive from all bookies -- Fencing phase, LAC read (sent to ensemble of current fragment): - - Complete when EC positive responses - - Unknown (cannot make progress) when AQ unknown responses (fencing LAC reads cannot cause an explicit negative as fencing creates the ledger on the bookie if it doesn’t exist) -- Recovery read (sent to writeset of entry): - - Entry recoverable: AQ positive read responses - - Entry Unrecoverable: QC negative read responses - - Unknown (cannot make progress): - - QC unknown responses or - - All responses received, but not enough for either a positive or negative - - -### Impact of Undetected Data Loss on Consistency - -The ledger recovery process assumes that ledger entries are never arbitrarily lost. In the event of the loss of an entry, the recovery process can: -- allow the original client to keep writing entries to a ledger that has just been fenced and closed, thus losing those entries -- allow the recovery client to truncate the ledger too soon, closing it with a last entry id lower than that of previously acknowledged entries - thus losing data. - -The following scenarios assume existing behaviour but simply skipping the writing of entries and fencing ops to the journal. - -### Scenario 1 - Lost Fenced Status Allows Writes After Ledger Close - -1. 3 bookies, B1, B2 & B3 -2. 2 clients, C1 & C2 -3. 1 ledger, L1, with e3:w3:a2 configuration. -4. C1 writes entry E1 to L1. The write hits all three bookies. -5. C1 hangs for an indeterminate length of time. -6. C2 sees that C1 is unresponsive, and assumes it has failed. C2 tries to recover the ledger L1. -7. L1 sends a fencing message to all bookies in the ensemble. -8. The fencing message succeeds in arriving at B1 & B2 and is acknowledged by both. The message to B3 is lost. -9. C2 sees that at least one bookie in each possible ack quorum has acknowledged the fencing message (EC threshold reached), so continues with the read/write phase of recovery, finding that E1 is the last entry of the ledger, and committing the endpoint of the ledger in the ZK. -10. B2 crashes and boots again with all unflushed operations lost. -11. C1 wakes up and writes entry E2 to all bookies. B2 & B3 acknowledge positively, so C1 considers E2 as persisted. B1 rejects the message as the ledger is fenced, but since ack quorum is 2, B2 & B3 are enough to consider the entry written. - -### Scenario 2 - Recovery Truncates Previously Acknowledged Entries - -1. C1 adds E0 to B1, B2, B3 -2. B1 and B3 confirms. C1 confirms the write to its client. -3. C2 starts recovery -4. B2 fails to respond. C1 tries to change ensemble but gets a metadata version conflict. -5. B1 crashes and restarts, has lost E0 (undetected) -6. C2 fences the ledger on B1, B2, B3 -7. C2 sends Read E0 to B1, B2, B3 -8. B1 responds with NoSuchEntry -9. B2 responds with NoSuchEntry -10. QC negative response threshold reached. C2 closes the ledger as empty. Losing E0. - -The problem is that without the journal (and syncing to entry log files before acknowledgement) a bookie can: -- lose the fenced status of a previously existing ledger -- respond with an explicit negative even though it had previously seen an entry. - -Undetected data loss could occur when running without the journal. Bookie crashes and loses most recent entries and fence statuses that had not yet been written and synced to disk. - -### A note on cookies - -Cookies play an essential part in the bookkeeper replication protocol, but their purpose is often unclear. - -When a bookie boots for the first time, it generates a cookie. The cookie encapsulates the identity of the bookie and should be considered immutable. This identity contains the advertised address of the bookie, the disks used for the journal, index, and ledger storage, and a unique ID. The bookie writes the cookie to ZK and each of the disks in use. On all subsequent boots, if the cookie is missing from any of these places, the bookie fails to boot. - -The absence of a disk's cookie implies that the rest of the disk's data is also missing. Cookie validation is performed on boot-up and prevents the boot from succeeding if the validation fails, thus preventing the bookie starting with undetected data loss. - -This proposal improves the cookie mechanism by automating the resolution of a cookie validation error which currently requires human intervention to resolve. This automated feature will be configurable (enabled or disabled) and additionally a CLI command will be made available so an admin can manually run the operation (for when this feature is disabled - likely to be the default). - -### Proposed Changes - -The proposed changes involve: -- A new config that controls whether add operations go into the journal -- Detecting possible data loss on boot -- Prevent explicit negative responses when data loss may have occurred, instead reply with unknown code, until data is repaired. -- Repair data loss -- Auto fix cookies (with new config to enable or disable the feature) -- CLI command for admin to run fix cookie logic in the case that auto fix is disabled - -In these proposed changes, when running "without" the journal, the journal still exists, but add entry operations skip the addition to the journal. The boot-up sequence still replays the journal. - -Add operations can be configured to be written to the journal or not based on the config `journalWriteData`. When set to `false`, add operations are not added to the journal. - -### Detecting Data Loss On Boot - -The new mechanism for data loss detection is checking for an unclean shutdown (aka a crash or abrupt termination of the bookie). When an unclean shutdown is detected further measures are taken to prevent data inconsistency. - -The unclean shutdown detection will consist of setting a bit in the index on start-up and clearing it on shutdown. On subsequent start-up, the value will be checked and if it remains set, it knows that the prior shutdown was not clean. - -Cookie validation will continue to be used to detect booting with one or more missing or empty disks (that once existed and contained a cookie). - -### Protection Mechanism - -Once possible data loss has been detected the following protection mechanism is carried out during the boot: - -- Fencing: Ledger metadata for all ledgers of the cluster are obtained and all those ledgers are fenced on this bookie. This prevents data loss scenario 1. -- Limbo: All open ledgers are placed in the limbo status. Limbo ledgers can serve read requests, but never respond with an explicit negative, all explicit negatives are converted to unknowns (with the use of a new code EUNKNOWN). -- Recovery: All open ledgers are opened and recovered. -- Repair: Each ledger is scanned and any missing entries are sourced from peers. -- Limbo ledgers that have been repaired have their limbo status cleared. - -### The Full Boot-Up Sequence - -This mechanism of limbo ledgers and self-repair needs to work hand-in hand with the cookie validation check. Combining everything together: - -On boot: -1. Check for unclean shutdown and validate cookies -2. Fetch the metadata for all ledgers in the cluster from ZK where the bookie is a member of its ensemble. -3. Phase one: - - If the cookie check fails or unclean shutdown is detected: - - For each non-closed ledger, mark the ledger as fenced and in-limbo in the index. - - Update the cookie if it was a cookie failure -4. Phase two - - For each ledger - 1. If the ledger is in-limbo, open and recover the ledger. - 2. Check that all entries assigned to this bookie exist in the index. - 3. For any entries that are missing, copy from another bookie. - 4. Clear limbo status if set - -When booting a bookie with empty disks, only phase one needs to be complete before the bookie makes itself available for client requests. - -In phase one, if the cookie check fails, we mark all non-closed ledgers as “fenced”. This prevents any future writes to these ledgers on this bookie. This solves the problem of an empty bookie disk allowing writes to closed ledgers (Scenario 1). - -Given that the algorithm solves both the issues that cookies are designed to solve, we can now allow the bookie to update its cookie without operator intervention. - -### Formal Verification of Proposed Changes - -The use of the limbo status and fencing of all ledgers on boot-up when detecting an unclean shutdown has been modelled in TLA+. It does not model the whole boot-up sequence but a simplified version with only fencing and limbo status. - -The specification models the lifetime of a single ledger and includes a single bookie crashing, losing all data. The specification allows the testing of: - -- enabling/disabling the fencing -- enabling/disabling the limbo status. - -When running without limbo status, the model checker finds the counterexample of scenario 2. When running without fencing of all ledgers, the model checker finds the counterexample of scenario 1. When running with both enabled, the model checker finds no invariant violation. - -The specification can be found here: https://github.com/Vanlightly/bookkeeper-tlaplus - -### Public Interfaces - -- Return codes. Addition of a new return code: `EUNKNOWN` which is returned when a read hits an in-limbo ledger and that ledger not contain the requested entry id. -- Bookie ledger metadata format (LedgerData). Addition of the limbo status. - -### Compatibility, Deprecation, and Migration Plan - -- Because we only skip the journal for add operations, there is no impact on existing deployments. When a bookie is booted with the new version, and `journalWriteData` is set to false, the journal is still replayed on boot-up causing no risk of data loss in the transition. - -### Test Plan - -- There is confidence in the design due to the modelling in TLA+ but this model does not include the full boot sequence. -- The implementation will require aggressive chaos testing to ensure correctness. - -### Rejected Alternatives - -Entry Log Per Ledger (ELPL) without the journal. From our performance testing of ELPL, performance degrades significantly with a large number of active ledgers and syncing to disk multiple times a second (which is required to offer low latency writes). - -In the future this design could be extended to offer ledger level configuration of journal use. The scope of this BP is limited to cluster level. \ No newline at end of file diff --git a/site/bps/BP-template.md b/site/bps/BP-template.md deleted file mode 100644 index 3207074aa86..00000000000 --- a/site/bps/BP-template.md +++ /dev/null @@ -1,42 +0,0 @@ ---- -title: "BP-XYZ: capation of bookkeeper proposal" -issue: https://github.com/apache/bookkeeper/ -state: "one of ['Under Discussion', 'Accepted', 'Adopted', 'Rejected']" -release: "x.y.z" ---- - -### Motivation - -_Describe the problems you are trying to solve_ - -### Public Interfaces - -_Briefly list any new interfaces that will be introduced as part of this proposal or any existing interfaces that will be removed or changed. The purpose of this section is to concisely call out the public contract that will come along with this feature._ - -A public interface is any change to the following: - -- Data format, Metadata format -- The wire protocol and api behavior -- Any class in the public packages -- Monitoring -- Command line tools and arguments -- Anything else that will likely break existing users in some way when they upgrade - -### Proposed Changes - -_Describe the new thing you want to do in appropriate detail. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences. Use judgement based on the scope of the change._ - -### Compatibility, Deprecation, and Migration Plan - -- What impact (if any) will there be on existing users? -- If we are changing behavior how will we phase out the older behavior? -- If we need special migration tools, describe them here. -- When will we remove the existing behavior? - -### Test Plan - -_Describe in few sentences how the BP will be tested. We are mostly interested in system tests (since unit-tests are specific to implementation details). How will we know that the implementation works as expected? How will we know nothing broke?_ - -### Rejected Alternatives - -_If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way._ diff --git a/site/community/bookkeeper_proposals.md b/site/community/bookkeeper_proposals.md deleted file mode 100644 index 0894638e597..00000000000 --- a/site/community/bookkeeper_proposals.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -title: BookKeeper Proposals ---- - -This page describes a proposed *BookKeeper Proposal (BP)* process for proposing a major change to BookKeeper. - -## Process - -### What is considered a "major change" that needs a BP? - -Any of the following should be considered a major change: - -- Any major new feature, subsystem, or piece of functionality -- Any change that impacts the public interfaces of the project -- Any change that impacts developer workflow of the project - -All the following are public interfaces that people build around: - -- Binary log format -- The network protocol and api behavior -- Configuration, especially client configuration -- Monitoring/Stats provider -- Command line tools and arguments - -### What should be included in a BP? - -A BP should contain the following sections: - -- *Motivation*: describe the problem to be solved -- *Proposed Change*: describe the new thing you want to do. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences, depending on the scope of the change. -- *New or Changed Public Interfaces*: impact to any of the "compatibility commitments" described above. We want to call these out in particular so everyone thinks about them. -- *Migration Plan and Compatibility*: if this feature requires additional support for a no-downtime upgrade describe how that will work -- *Rejected Alternatives*: What are the other alternatives you considered and why are they worse? The goal of this section is to help people understand why this is the best solution now, and also to prevent churn in the future when old alternatives are reconsidered. - -### Who should initiate the BP? - -Anyone can initiate a BP but you shouldn't do it unless you have an intention of getting the work done to implement it (otherwise it is silly). - -### How to make a BP? - -Here is the process for making a BP: - -1. Create an issue `BP-: [capation of bookkeeper proposal]`. E.g. `BP-1: 64 bits ledger id support`. - - Take the next available BP number from this page. - - Write a brief description about what BP is for in this issue. This issue will be the master issue for tracking the status of this BP and its implementations. - All the implementations of this BP should be listed and linked to this master issues. -1. Write the proposal for this BP. There are two ways to write a bookkeeper proposal. You can choose to write a BP using markdown, or write a BP -using Google Doc. - - Markdown - - Make a copy of the [BP-Template](https://github.com/apache/bookkeeper/tree/master/site/bps/BP-template.md). Name the BP file as `BP--[caption-of-proposal].md`. - ```shell - $ cp site/bps/BP-template.md site/bps/BP-xyz-capation-of-proposal.md - ``` - - Fill the sections listed in the BP template. - - issue: replace `` with the issue number. - - state: "Under Discussion" - - release: leave the release to `N/A`. you can only mark a release after a BP is implemented. - - Google Doc - - Make a copy of the [BP-Template](https://docs.google.com/document/d/1DsmH54LoohgwqnEjESPQNtIYxxcOy2rwonZ_TJCwws0). Name the BP file as `BP--[caption-of-proposal]`. - - Fill the sections listed in the BP template. -1. Send a PR for this BP. Following the instructions in the pull request template. - - add `BP` label to this PR - - attach the google doc link in the PR description if the BP is written in google doc - - don't associate this PR with any release or milestone - - edit `site/community/bookkeeper_proposals.md`: - - bump the next bp number - - add this BP to `Inprogress` section -1. You can tag committers on this RP for reviewers, or start a `[DISCUSS]` thread on Apache mailing list. If you are sending an email, please make sure that the subject - of the thread is of the format `[DISCUSS] BP-: capation of bookkeeper proposal`. -1. Once the BP is finalized, reviewed and approved by committers, the BP is accepted. The criteria for acceptance is [lazy majority](http://bookkeeper.apache.org/bylaws.html). - 1. Committers merge the PR after a BP is accepted. The development for this BP moves forward with implementations. The BP should be updated if there is anything changed during implementing it. - 1. After all the implementations for a given BP are completed, a new PR should be sent for changing the state of a BP: - - state: "Adopted" - - release: set to the release that includes this BP. - - moving the BP from `Inprogress` to `Adopted`. - 1. The final PR for changing BP state will be used as the criteria for marking a BP as completed. -1. If a BP is failed or rejected: - 1. Update the PR to change the state of a BP - - state: "Discarded" - - add a paragraph at the first paragraph of this BP for describing the reasons. - - moving the BP from `Inprogress` to `Discarded`. - 2. Once the PR is updated, committers can merge this proposal PR and close the master issue of this BP. - -## All Proposals - -This section lists all the _bookkeeper proposals_ made to BookKeeper. - -*Next Proposal Number: 43* - -### Inprogress - -Proposal | State -:--------|:----- -[BP-4 - BookKeeper Lifecycle Management](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-4+-+BookKeeper+Lifecycle+Management) | Draft -[BP-8 - Queue based auto rereplicator](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-8+-+Queue+based+auto+rereplicator) | Draft -[BP-12 - Improve documentation](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-12+-+Improve+documentation) | Accepted -[BP-14 Relax durability](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-14+Relax+durability) | Accepted -[BP-16: Thin Client - Remove direct metadata storage access from clients](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-16%3A+Thin+Client+-+Remove+direct+metadata+storage+access+from+clients) | Draft -[BP-18: LedgerType, Flags and StorageHints](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-18%3A+LedgerType%2C+Flags+and+StorageHints) | Accepted -[BP-26: Move distributedlog library as part of bookkeeper](../../bps/BP-26-move-distributedlog-core-library) | Accepted -[BP-27: New BookKeeper CLI](../../bps/BP-27-new-bookkeeper-cli) | Accepted -[BP-28: use etcd as metadata store](../../bps/BP-28-etcd-as-metadata-store) | Accepted -[BP-29: Metadata API module](../../bps/BP-29-metadata-store-api-module) | Accepted -[BP-30: BookKeeper Table Service](https://docs.google.com/document/d/155xAwWv5IdOitHh1NVMEwCMGgB28M3FyMiQSxEpjE-Y/edit#heading=h.56rbh52koe3f) | Accepted -[BP-31: BookKeeper Durability Anchor](../../bps/BP-31-durability) | Accepted -[BP-32: Advisory (optimistic) write close](../../bps/BP-32-advisory-write-close) | Accepted -[BP-33: Move releasing docker images out of main repo](../../bps/BP-33-building-official-docker-imags) | Draft -[BP-34: Cluster Metadata Checker](../../bps/BP-34-cluster-metadata-checker) | Accepted -[BP-35: 128 bits support](../../bps/BP-35-128-bits-support) | Accepted -[BP-36: Stats documentation annotation](../../bps/BP-36-stats-documentation-annotation) | Accepted -[BP-37: Improve configuration management for better documentation](../../bps/BP-37-conf-documentation) | Accepted -[BP-41: Separate BookieId from Separate BookieId from Bookie Network Address](../../bps/BP-41-bookieid) | Accepted -[BP-42: New Client API - list ledgers](../../bps/BP-42-new-api-list-ledgers) | Accepted -[BP-43: Migration to gradle](../../bps/BP-43-gradle-migration) | Draft - - -### Adopted - -Proposal | Release -:--------|:------- -[BP-1 - 64 bits ledger id support](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-1+-+64+bits+ledger+id+support) | 4.5.0 -[BP-2 - Resource aware data placement](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-2+-+Resource+aware+data+placement) | 4.5.0 -[BP-3 - Security support](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-3+-+Security+support) | 4.5.0 -[BP-5 - Allow reads outside the LAC Protocol](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-5+Allow+reads+outside+the+LAC+Protocol) | 4.5.0 -[BP-6 - Use separate log for compaction](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-6+-+Use+separate+log+for+compaction) | 4.6.0 -[BP-9 - Github issues for Issue Tracking](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-9+-+Github+issues+for+Issue+Tracking) | 4.5.0 -[BP-10 - Official Bookkeeper Docker Image](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-10+-+Official+Bookkeeper+Docker+Image) | 4.5.0 -[BP-11 - Move website/documentation to Jekyll based site](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=71012301) | 4.5.0 -[BP-13 - Time Based Release Plan](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-13+-+Time+Based+Release+Plan) | 4.6.0 -[BP-15 - New CreateLedger API](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-15+New+CreateLedger+API) | 4.6.0 -[BP-17 - Define BookKeeper public http endpoints](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-17%3A+Define+BookKeeper+public+http+endpoints) | 4.6.0 -[BP-20: Github workflow for bookkeeper proposals](../../bps/BP-20-github-workflow-for-bookkeeper-proposals) | 4.7.0 -[BP-25: Move checksum to proto](../../bps/BP-25-MovingChecksumToProto) | 4.7.0 -[BP-38: Publish Bookie Service Info on Metadata Service](../../bps/BP-38-bookie-endpoint-discovery) | 4.11.0 - -### Discarded - -Proposal | Reason -:--------|:------ -[BP-7 - Explicit LAC on addEntry](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-7+-+Explicit+LAC+on+addEntry) | Not A Problem -[BP-21: New API close inconsistencies](../../bps/BP-21-new-api-close-inconsistencies) | Not A Problem -[BP-22: Separate closing ledgers from opening ledgers](../../bps/BP-22-separate-closing-ledgers-from-opening-ledgers) | Not A Problem diff --git a/site/community/coding_guide.md b/site/community/coding_guide.md deleted file mode 100644 index 614b626836d..00000000000 --- a/site/community/coding_guide.md +++ /dev/null @@ -1,87 +0,0 @@ ---- -title: Coding Guide ---- - -These guidelines are meant to encourage consistency and best practices among people working on _Apache BookKeeper_ code base. -They should be observed unless there is compelling reason to ignore them. We are also using checkstyle to enforce coding style. -Please refer to our [checkstyle rules](https://github.com/apache/bookkeeper/blob/master/buildtools/src/main/resources/bookkeeper/checkstyle.xml) for all enforced checkstyle rules. - -### Java - -Apache BookKeeper code should follow the [Sun Java Coding Convention](http://www.oracle.com/technetwork/java/javase/documentation/codeconvtoc-136057.html), with the following additions. - -* Lines can not be longer than 120 characters. -* Indentation should be **4 spaces**. Tabs should never be used. -* Use curly braces even for single-line ifs and elses. -* No @author tags in any javadoc. -* Use try-with-resources blocks whenever is possible. -* **TODO**s should be associated to at least one issue. E.g. `// TODO: make this parameter configurable (https://github.com/apache/bookkeeper/issues/280)` - -### Dependencies - -Apache BookKeeper uses following libraries a lot: - -* [Guava](https://github.com/google/guava): as a fundamental core library -* [Netty](http://netty.io/): for network communications and memory buffer management. - -Please use these libraries whenever possible rather than introducing more dependencies. - -Dependencies are bundled with our binary distributions, so we need to attach the relevant licenses. See [Third party dependencies and licensing](/community/licensing) for a guide on how to do this correctly. - -#### Future - -We prefer Java-8 Future over Guava's Listenable Future. Please use Java-8 Future whenever possible. - -#### Memory - -We prefer using netty _ByteBuf_ over java nio _ByteBuffer_ for internal usage. As we are using Netty Buffer for memory management. - -### Logging - -* Logging should be taken seriously. Please take the time to access the logs when making a change to ensure that the important things are getting logged and there is no junk there. -* Logging statements should be complete sentences with proper capitalization that are written to be read by a person not necessarily familiar with the source code. -* All loggings should be done with **SLF4j**, never _System.out_ or _System.err_. - -#### Logging levels - -- _INFO_ is the level you should assume the software will be run in. INFO messages are things which are not bad but which the user will definitely want to know about every time they occur. -- _TRACE_ and _DEBUG_ are both things you turn on when something is wrong and you want to figure out what is going on. _DEBUG_ should not be so fine grained that it will seriously affect performance of the program. _TRACE_ can be anything. Both _DEBUG_ and _TRACE_ statements should be considered to be wrapped in an _if (logger.isDebugEnabled)_ or _if (logger.isTraceEnabled)_ check to avoid performance degradation. -- _WARN_ and _ERROR_ indicate something that is **BAD**. Use _WARN_ if you aren't totally sure it is bad, and _ERROR_ if you are. - -Please log the _stack traces_ at **ERROR** level, but never at **INFO** level or below. They can be logged at **WARN** level when they are interesting for debugging. - -### Monitoring - -* Apache BookKeeper uses a pluggable [StatsProvider](https://github.com/apache/bookkeeper/tree/master/bookkeeper-stats) on exporting metrics -* Any new features should come with appropriate metrics for monitoring the feature is working correctly. -* Those metrics should be taken seriously and only export useful metrics that would be used on production on monitoring/alerting healthy of the system, or troubleshooting problems. - -### Unit Tests - -* New changes should come with unit tests that verify the functionality being added -* Unit tests should test the least amount of code possible. Don't start the whole server unless there is no other way to test a single class or small group of classes in isolation. -* Tests should not depend on any external resources. They need to setup and teardown their own stuff. -* It is okay to use the filesystem and network in tests since that's our business but you need to clean up them after yourself. -* _Do not_ use sleep or other timing assumptions in tests. It is always, always, wrong and will fail intermittently on any test server with other things going on that causes delays. -* We are strongly recommending adding a _timeout_ value to all our test cases, to prevent a build from completing indefinitely. -`@Test(timeout=60000)` - -### Configuration - -* Names should be thought through from the point of view of the person using the config. -* The default values should be thought as best value for people who runs the program without tuning parameters. -* All configuration settings should be added to [default configuration file](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/conf/bk_server.conf) and [documented](https://github.com/apache/bookkeeper/blob/master/site/_data/config/bk_server.yaml). - -### Concurrency - -Apache BookKeeper is a low latency system. So it is implemented as a purely asynchronous service. This is accomplished as follows: - -* All public classes should be **thread-safe**. -* We prefer using [OrderedExecutor](https://github.com/apache/bookkeeper/blob/master/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/OrderedExecutor.java) for executing any asynchronous actions. The mutations to same instance should be submitted to same thread to execute. -* If synchronization and locking is required, they should be in a fine granularity way. -* All threads should have proper meaningful name. -* If a class is not threadsafe, it should be annotated [@NotThreadSafe](https://github.com/misberner/jsr-305/blob/master/ri/src/main/java/javax/annotation/concurrent/NotThreadSafe.java). The instances that use this class is responsible for its synchronization. - -### Backwards Compatibility -* Wire protocol should support backwards compatibility to enable no-downtime upgrades. This means the servers **MUST** be able to support requests from both old and new clients simultaneously. -* Metadata formats and data formats should support backwards compatibility. diff --git a/site/community/contributing.md b/site/community/contributing.md deleted file mode 100644 index b56dcb742cd..00000000000 --- a/site/community/contributing.md +++ /dev/null @@ -1,262 +0,0 @@ ---- -title: Contributing to Apache BookKeeper ---- - -* TOC -{:toc} - -The Apache BookKeeper community welcomes contributions from anyone with a passion for distributed systems! BookKeeper has many different opportunities for contributions -- -write new examples/tutorials, add new user-facing libraries, work on the core storage components, integrate with different metadata stores (ZooKeeper, Etcd etc), or -participate on the documentation effort. - -We use a review-then-commit workflow in BookKeeper for all contributions. - -**For larger contributions or those that affect multiple components:** - -1. **Engage**: We encourage you to work with the BookKeeper community on the [Github Issues](https://github.com/apache/bookkeeper/issues) and [developer’s mailing list](../mailing-lists) to identify good areas for contribution. -1. **Design:** More complicated contributions will likely benefit from some early discussion in order to scope and design them well. - -**For all contributions:** - -1. **Code:** The best part ;-) -1. **Review:** Submit a pull request with your contribution to our [GitHub Repo](https://github.com/apache/bookkeeper). Work with a committer to review and iterate on the code, if needed. -1. **Commit:** A BookKeeper committer merges the pull request into our [GitHub Repo](https://github.com/apache/bookkeeper). - -We look forward to working with you! - -## Engage - -### Mailing list(s) - -We discuss design and implementation issues on the [dev@bookkeeper.apache.org](mailto:dev@bookkeeper.apache.org) mailing list, which is archived [here](https://lists.apache.org/list.html?dev@bookkeeper.apache.org). Join by emailing [`dev-subscribe@bookkeeper.apache.org`](mailto:dev-subscribe@bookkeeper.apache.org). - -If interested, you can also join the other [mailing lists](../mailing-lists). - -### Github Issues - -We are moving to use [Github Issues](https://github.com/apache/bookkeeper/issues) as an issue tracking and project management tool, as well as a way to communicate among a very diverse and distributed set of contributors. To be able to gather feedback, avoid frustration, and avoid duplicated efforts all BookKeeper-related work should be tracked there. - -If you do not already have an Github account, sign up [here](https://github.com/join). - -If a quick [search](https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93) doesn’t turn up an existing Github issue for the work you want to contribute, create it. Please discuss your idea with a committer in Github or, alternatively, on the developer mailing list. - -If there’s an existing Github issue for your intended contribution, please comment about your intended work. Once the work is understood, a committer will assign the issue to you. If an issue is currently assigned, please check with the current assignee before reassigning. - -For moderate or large contributions, you should not start coding or writing a design document unless there is a corresponding Github issue assigned to you for that work. Simple changes, like fixing typos, do not require an associated issue. - -### Online discussions - -We are using [Apache BookKeeper Slack channel](https://apachebookkeeper.slack.com/) for online discussions. You can self-invite yourself by accessing [this link](http://apachebookkeeper.slack.com/). - -Slack channels are great for quick questions or discussions on specialized topics. Remember that we strongly encourage communication via the mailing lists, and we prefer to discuss more complex subjects by email. Developers should be careful to move or duplicate all the official or useful discussions to the issue tracking system and/or the dev mailing list. - -## Design - -To avoid potential frustration during the code review cycle, we encourage you to clearly scope and design non-trivial contributions with the BookKeeper community before you start coding. - -We are using [BookKeeper Proposals](http://bookkeeper.apache.org/community/bookkeeper_proposals/) for managing major changes to BookKeeper. - -## Code - -To contribute code to Apache BookKeeper, you’ll have to do a few administrative steps once, and then follow the [Coding Guide](../coding_guide). - -### One-time Setup - -#### [Optionally] Submit Contributor License Agreement - -Apache Software Foundation (ASF) desires that all contributors of ideas, code, or documentation to the Apache projects complete, sign, and submit an [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf) (ICLA). The purpose of this agreement is to clearly define the terms under which intellectual property has been contributed to the ASF and thereby allow us to defend the project should there be a legal dispute regarding the software at some future time. - -We require you to have an ICLA on file with the Apache Secretary for larger contributions only. For smaller ones, however, we rely on [clause five](http://www.apache.org/licenses/LICENSE-2.0#contributions) of the Apache License, Version 2.0, describing licensing of intentionally submitted contributions and do not require an ICLA in that case. - -#### Obtain a GitHub account - -We use GitHub’s pull request functionality to review proposed code changes. - -If you do not already have a personal GitHub account, sign up [here](https://github.com/join). - -#### Fork the repository on GitHub - -Go to the [BookKeeper GitHub Repo](https://github.com/apache/bookkeeper/) and fork the repository to your own private account. This will be your private workspace for staging changes. - -#### Clone the repository locally - -You are now ready to create the development environment on your local machine. Feel free to repeat these steps on all machines that you want to use for development. - -We assume you are using SSH-based authentication with GitHub. If necessary, exchange SSH keys with GitHub by following [their instructions](https://help.github.com/articles/generating-an-ssh-key/). - -Clone your personal BookKeeper’s GitHub fork. - - $ git clone https://github.com//bookkeeper.git - $ cd bookkeeper - -Add Apache Repo as additional Git remotes, where you can sync the changes (for committers, you need these two remotes for pushing changes). - - $ git remote add apache https://github.com/apache/bookkeeper - $ git remote add apache-github https://github.com/apache/bookkeeper - -You are now ready to start developing! - -#### [Optional] IDE Setup - -Depending on your preferred development environment, you may need to prepare it to develop BookKeeper code. - -##### IntelliJ - -###### Checkstyle -IntelliJ supports checkstyle within the IDE using the Checkstyle-IDEA plugin. - -1. Install the "Checkstyle-IDEA" plugin from the IntelliJ plugin repository. -1. Configure the plugin by going to Settings -> Other Settings -> Checkstyle. -1. Set the "Scan Scope" to "Only Java sources (including tests)". -1. In the "Configuration File" pane, add a new configuration using the plus icon: - 1. Set the "Description" to "BookKeeper". - 1. Select "Use a local Checkstyle file", and point it to - "buildtools/src/main/resources/bookkeeper/checkstyle.xml" within - your repository. - 1. Check the box for "Store relative to project location", and click - "Next". - 1. Configure the "checkstyle.suppressions.file" property value to - "suppressions.xml", and click "Next", then "Finish". -1. Select "BookKeeper" as the only active configuration file, and click "Apply" and - "OK". -1. Checkstyle will now give warnings in the editor for any Checkstyle - violations. - -You can also scan an entire module by opening the Checkstyle tools window and -clicking the "Check Module" button. The scan should report no errors. - -Note: Selecting "Check Project" may report some errors from the archetype -modules as they are not configured for Checkstyle validation. - -##### Eclipse - -Use a recent Eclipse version that includes m2e. Currently we recommend Eclipse Neon. -Start Eclipse with a fresh workspace in a separate directory from your checkout. - -###### Initial setup - -1. Import the bookkeeper projects - - File - -> Import... - -> Browse to the directory you cloned into and select "bookkeeper" - -> make sure all bookkeeper projects are selected - -> Finalize - -You now should have all the bookkeeper projects imported into eclipse and should see no compile errors. - -###### Checkstyle -Eclipse supports checkstyle within the IDE using the Checkstyle plugin. - -1. Install the [Checkstyle plugin](https://marketplace.eclipse.org/content/checkstyle-plug). -1. Configure Checkstyle plugin by going to Preferences - Checkstyle. - 1. Click "New...". - 1. Select "External Configuration File" for type. - 1. Click "Browse..." and select "buildtools/src/main/resources/bookkeeper/checkstyle.xml". - 1. Enter "BookKeeper Checks" under "Name:". - 1. Click "OK", then "OK". - -### Create a branch in your fork - -You’ll work on your contribution in a branch in your own (forked) repository. Create a local branch, initialized with the state of the branch you expect your changes to be merged into. Keep in mind that we use several branches, including `master`, feature-specific, and release-specific branches. If you are unsure, initialize with the state of the `master` branch. - - $ git fetch apache - $ git checkout -b apache/master - -At this point, you can start making and committing changes to this branch in a standard way. - -### Syncing and pushing your branch - -Periodically while you work, and certainly before submitting a pull request, you should update your branch with the most recent changes to the target branch. - - $ git pull --rebase - -Remember to always use `--rebase` parameter to avoid extraneous merge commits. - -Then you can push your local, committed changes to your (forked) repository on GitHub. Since rebase may change that branch's history, you may need to force push. You'll run: - - $ git push --force - -### Testing - -All code should have appropriate unit testing coverage. New code should have new tests in the same contribution. Bug fixes should include a regression test to prevent the issue from reoccurring. - -## Review - -Once the initial code is complete and the tests pass, it’s time to start the code review process. We review and discuss all code, no matter who authors it. It’s a great way to build community, since you can learn from other developers, and they become familiar with your contribution. It also builds a strong project by encouraging a high quality bar and keeping code consistent throughout the project. - -### Create a pull request - -Organize your commits to make a committer’s job easier when reviewing. Committers normally prefer multiple small pull requests, instead of a single large pull request. Within a pull request, a relatively small number of commits that break the problem into logical steps is preferred. For most pull requests, you'll squash your changes down to 1 commit. You can use the following command to re-order, squash, edit, or change description of individual commits. - - $ git rebase -i apache/master - -You'll then push to your branch on GitHub. Note: when updating your commit after pull request feedback and use squash to get back to one commit, you will need to do a force submit to the branch on your repo. - -Navigate to the [BookKeeper GitHub Repo](https://github.com/apache/bookkeeper) to create a pull request. The title of the pull request should be strictly in the following format: - - Issue - -Please include a descriptive pull request message to help make the comitter’s job easier when reviewing. It’s fine to refer to existing design docs or the contents of the associated JIRA as appropriate. - -If you know a good committer to review your pull request, please make a comment like the following. If not, don’t worry -- a committer will pick it up. - - Hi @<GitHub-committer-username>, can you please take a look? - -When choosing a committer to review, think about who is the expert on the relevant code, who the stakeholders are for this change, and who else would benefit from becoming familiar with the code. If you’d appreciate comments from additional folks but already have a main committer, you can explicitly cc them using `@<GitHub-committer-username>`. - -### Code Review and Revision - -During the code review process, don’t rebase your branch or otherwise modify published commits, since this can remove existing comment history and be confusing to the committer when reviewing. When you make a revision, always push it in a new commit. - -Our GitHub repo automatically provides pre-commit testing coverage using Jenkins. Please make sure those tests pass; the contribution cannot be merged otherwise. - -### LGTM - -Once the committer is happy with the change, they’ll approve the pull request with an LGTM (“*looks good to me!*”) or a `+1`. At this point, the committer will take over, possibly make some additional touch ups, and merge your changes into the codebase. - -In the case the author is also a committer, either can merge the pull request. Just be sure to communicate clearly whose responsibility it is in this particular case. - -Thank you for your contribution to BookKeeper! - -### Deleting your branch -Once the pull request is merged into the BookKeeper repository, you can safely delete the branch locally and purge it from your forked repository. - -From another local branch, run: - - $ git fetch origin - $ git branch -d <my-branch> - $ git push origin --delete <my-branch> - -## Commit (committers only) - -Once the code has been peer reviewed by a committer, the next step is for the committer to merge it into the Github repo. - -Pull requests should not be merged before the review has approved from another committer. Exceptions to this rule may be made rarely, on a case-by-case basis only, in the committer’s discretion for situations such as build breakages. - -Committers should never commit anything without going through a pull request, since that would bypass test coverage and potentially cause the build to fail due to checkstyle, etc. In addition, pull requests ensure that changes are communicated properly and potential flaws or improvements can be spotted. **Always go through the pull request, even if you won’t wait for the code review.** Even then, comments can be provided in the pull requests after it has been merged to work on follow-ups. - -Committing is managed by a python script [bk-merge-pr.py](https://github.com/apache/bookkeeper/blob/master/dev/bk-merge-pr.py). Just follow the instructions promoted by the -script and types the information needed by the script. - -### Contributor License Agreement - -If you are merging a larger contribution, please make sure that the contributor has an ICLA on file with the Apache Secretary. You can view the list of committers [here](http://home.apache.org/phonebook.html?unix=committers), as well as [ICLA-signers who aren’t yet committers](http://home.apache.org/unlistedclas.html). - -For smaller contributions, however, this is not required. In this case, we rely on [clause five](http://www.apache.org/licenses/LICENSE-2.0#contributions) of the Apache License, Version 2.0, describing licensing of intentionally submitted contributions. - -### Tests -Before merging, please make sure that Jenkins tests pass, as visible in the GitHub pull request. Do not merge the pull request otherwise. - -### Finishing touches - -At some point in the review process, you should take the pull request over and complete any outstanding work that is either minor, stylistic, or otherwise outside the expertise of the contributor. The [merge script](https://github.com/apache/bookkeeper/blob/master/dev/bk-merge-pr.py) provides instructions for committers to address such minor conflicts. - -## Documentation - -### Website - -The BookKeeper website is in the same [BookKeeper Github Repo](https://github.com/apache/bookkeeper). The source files are hosted under `site` directory in `master` branch, -the static content is generated by CI job and merged into the `asf-site` branch. - -Follow the [README](https://github.com/apache/bookkeeper/tree/master/site) for making contributions to the website. diff --git a/site/community/issue-report.md b/site/community/issue-report.md deleted file mode 100644 index f7f5912e79f..00000000000 --- a/site/community/issue-report.md +++ /dev/null @@ -1,93 +0,0 @@ ---- -title: Issue Report ---- - -To report an issue, you will need to create a **[New Issue](https://github.com/apache/bookkeeper/issues/new)**. -Be aware that resolving your issue may require **your participation**. Please be willing and prepared to aid the developers in finding the actual cause of the issue so that they can develop a comprehensive solution. - -## Before creating a new Issue: - -- Search for the issue you want to report, it may already have been reported. -- If you find a similar issue, add any new information you might have as a comment on the existing issue. If it's different enough, you might decide it needs to be reported in a new issue. -- If an issue you recently reported was closed, and you don't agree with the reasoning for it being closed, you will need to reopen it to let us re-investigate the issue. -- Do not reopen the tickets that are in a previously completed milestone. Instead, open a new issue. - -## Creating a Issue: - -Here is an very useful artical [How to report bugs effectively]( http://www.chiark.greenend.org.uk/%7Esgtatham/bugs.html) - -### Provide useful and required information - -#### If it is a question - -- Please check our [documentation](http://bookkeeper.apache.org/docs/latest/) first. -- If you could not find an answer there, please consider asking your question in our community mailing list at [dev@bookkeeper.apache.org](mailto:dev@bookkeeper.apache.org), or reach out us on our [Slack channel](../slack). It would benefit other members of our community. - -#### If it is a **FEATURE REQUEST** - -- Please describe the feature you are requesting. -- Indicate the importance of this issue to you (_blocker_, _must-have_, _should-have_, _nice-to-have_). Are you currently using any workarounds to address this issue? -- Provide any additional detail on your proposed use case for this feature. -- If it is a [BookKeeper Proposal](http://bookkeeper.apache.org/community/bookkeeper_proposals/), please label this issue as `BP`. - -#### If it is a **BUG REPORT** - -Please describe the issue you observed: - -- What did you do? -- What did you expect to see? -- What did you see instead? - -### Use Labels - -Issue labels help to find issue reports and recognize the status of where an issue is in the lifecycle. An issue typically has the following 2 types of labels: - -1. **type** identifying its type. -1. **area** identifying the areas it belongs to. - -#### Type - -- **type/bug**: The issue describes a product defect. -- **type/feature**: The issue describes a new feature, which requires extensive design and testing. -- **type/question**: The issue contains a user or contributor question requiring a response. -- **type/task**: The issue describes a new task, which requires extensive design and testing. - -#### Area - -- **area/bookie**: Code changes related to bookie storage. -- **area/build**: Code changes related to project build. -- **area/client**: Code changes related to clients. -- **area/docker**: Code changes related to docker builds. -- **area/documentation**: Code changes related to documentation (including website changes). -- **area/metadata**: Code changes related to metadata management. -- **area/protocol**: Protocol changes. -- **area/release**: Release related tasks. -- **area/security**: Security related changes. -- **area/tests**: Tests related changes. - -#### Priority - -At most of the time, it is hard to find a right `priority` for issues. Currently we only have one label `priority/blocker` for marking an issue as a blocker -for a given release. Please only mark this issue as *blocker* only when it is a real blocker for a given release. If you have no idea about this, just leave -it as empty. - -#### Status - -If an issue is assigned to a contributor, that means there is already a contributor working on it. If an issue is unassigned, you can pick this up by assigning -it to yourself (for committers), or comment on the issue saying you would like to give it a try. - -If an issue is not an issue anymore, close it and mark it as `status/wontfix`. - -All the issues marked as `status/help-needed` are good candidates for new contributors to start with. - -#### BookKeeper Proposal - -If an issue is a `BookKeeper Proposal`, label it as `BP`. - -#### Milestone and Release - -If you want some features merge into a given milestone or release, please associate the issue with a given milestone or release. - -If you have no idea, just leave them empty. The committers will manage them for you. - -Thank you for contributing to Apache BookKeeper! diff --git a/site/community/licensing.md b/site/community/licensing.md deleted file mode 100644 index 90256dc2dd9..00000000000 --- a/site/community/licensing.md +++ /dev/null @@ -1,100 +0,0 @@ ---- -title: Third party dependencies and licensing ---- - -The bookkeeper project ships one source distribution and two binary distributions. - -- ```bookkeeper-<version>-src.tar.gz```, which contains the source code to build bookkeeper. -- ```bookkeeper-all-<version>-bin.tar.gz```, which contains the bookkeeper server and all optional dependencies. -- ```bookkeeper-server-<version>-bin.tar.gz```, which contains the bare minimum to run a bookkeeper server. - -The source distribution can contain source code copied from third parties. The binaries ship with third party dependencies in jar file form. - -As the ASF may not own the copyright on the contents of this copied source code or third party jars, we may need to account for them in the LICENSE and/or NOTICE file of the distribution. - -The LICENSE and NOTICE files for the source distribution are found at: -- [bookkeeper/LICENSE](https://github.com/apache/bookkeeper/blob/master/LICENSE) -- [bookkeeper/NOTICE](https://github.com/apache/bookkeeper/blob/master/NOTICE) - -The LICENSE and NOTICE files for the binary distribution are found at: -- [bookkeeper/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt (for -all package)](https://github.com/apache/bookkeeper/blob/master/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt) -- [bookkeeper/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt (for -all package)](https://github.com/apache/bookkeeper/blob/master/bookkeeper-dist/src/main/resources/NOTICE-all.bin.txt) -- [bookkeeper/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt (for -server package)](https://github.com/apache/bookkeeper/blob/master/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt) -- [bookkeeper/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt (for -server package)](https://github.com/apache/bookkeeper/blob/master/bookkeeper-dist/src/main/resources/NOTICE-server.bin.txt) - -When updating these files, use the following rules of thumb: -- BSD/MIT-style dependencies should be covered in LICENSE. -- Apache Software License dependences should be covered in NOTICE, but only if they themselves contain a NOTICE file. -- NOTICE should be kept to a minimum, and only contain what is legally required. -- The LICENSE and NOTICE files for the binary packages should contains everything in source LICENSE and NOTICE packages, unless the source code being referred to does not ship in the binary packages (for example, if it was copied in only for tests). -- All shipped dependencies should be mentioned in LICENSE for completeness, along with a link to their source code if available. -- Any license other than BSD/MIT-style or ASL should be discussed on [the dev list](/community/mailing-lists). -- If you have any doubts, raise them on [the dev list](/community/mailing-lists). - -# Handling new/updated source dependencies - -For bookkeeper, a source dependency is any code which has been copied in code form into our source tree. An example of this is [circe-checksum](https://github.com/apache/bookkeeper/tree/master/circe-checksum) which was copied into our codebase and modified. Depending on the license of source code, you may need to update the source distribution LICENSE and NOTICE files. - -In the case of circe-checksum, the original code is under the Apache Software License, Version 2 (ASLv2), and there is no NOTICE file, so neither LICENSE nor NOTICE need to be updated. - -If, for example, we were to copy code from [Hadoop](https://github.com/apache/hadoop), and the code in question was originally written for Hadoop, then we would not need to update LICENSE or NOTICE, as Hadoop is also licensed under the ASLv2, and while it has a NOTICE file, the part covering code originally written for Hadoop is covered by the line, "This product includes software developed by The Apache Software Foundation (http://www.apache.org/).", which already exists in our NOTICE. However, if we were to copy code from Hadoop that originally originated elsewhere, such as their [pure java CRC library](https://github.com/apache/hadoop/blob/f67237cbe7bc48a1b9088e990800b37529f1db2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java), this code is originally from Intel, under a BSD style license, so you would have to track down the original license, add it to [deps/](https://github.com/apache/bookkeeper/blob/master/bookkeeper-dist/src/main/resources/) and link it from our LICENSE file. - -If we were to copy code from [Netty](https://github.com/netty/netty/), and the code in question was originally written for Netty, then we would need to update NOTICE with the relevant portions (i.e. the first section) from the [Netty NOTICE file](https://github.com/netty/netty/blob/4.1/NOTICE.txt), as Netty is licensed under the ASLv2 and it _does_ contain a NOTICE file. If we were to copy code from Netty which originally originated elsewhere, but had also been modified by Netty, for example [their SLF4J modifications](https://github.com/netty/netty/blob/b60e0b6a51d59fb9a98918c8783265b30531de57/common/src/main/java/io/netty/logging/CommonsLogger.java), we would need to update our NOTICE with the relevant portions (i.e. the first section) from Netty's NOTICE, and also add the SLF4J license to [deps/](https://github.com/apache/bookkeeper/blob/master/bookkeeper-dist/src/main/resources/) and link it from our LICENSE file (as it has an MIT-style license). - -If we were to copy code from [Protobuf](https://github.com/google/protobuf) or [SLF4J](https://www.slf4j.org/) into our code base, then we would have to copy their license to [deps/](https://github.com/apache/bookkeeper/blob/master/bookkeeper-dist/src/main/resources/) and link it from our LICENSE file, as these projects are under [BSD-style](https://github.com/google/protobuf/blob/master/LICENSE) and [MIT-style](https://www.slf4j.org/license.html) licenses respectively. - -# Handling new/updated binary dependencies - -When a new binary dependency is added, or a dependency version is updated, we need to update the LICENSE and NOTICE files for our binary packages. There is a separate version of each of these files for both the -all tarball and the -server tarball. The files can be found at ```bookkeeper-dist/src/main/resources```. - -How you update the files depends on the licensing of the dependency. Most dependencies come under either the Apache Software License, Version 2, or an MIT/BSD style license. If the software comes under anything else, it's best to ask for advice on the [dev@ list](/community/mailing-lists). - -## dev/check-binary-license script - -We provide a script which will check if the LICENSE file attached to a binary tarball matches the jar files distributed in that tarball. The goal of the script is to ensure that all shipped binaries are accounted for, and that nothing else is mentioned in the LICENSE or NOTICE files. - -To check that licensing is correct, generate the tarball and run the script against it as follows (in this example I've removed references to protobuf from the LICENSE file). - -```shell -~/src/bookkeeper $ mvn clean package -DskipTests -... - -~/src/bookkeeper $ dev/check-binary-license bookkeeper-dist/server/target/bookkeeper-server-4.7.0-SNAPSHOT-bin.tar.gz -com.google.protobuf-protobuf-java-3.4.0.jar unaccounted for in LICENSE -deps/protobuf-3.4.0/LICENSE bundled, but not linked from LICENSE - -~/src/bookkeeper $ -``` - -The script checks the following: -1. If a jar file is included in the tarball, this file must be mentioned in the LICENSE file. -2. If a jar file is mentioned in LICENSE or NOTICE, then this jar file must exist in the tarball. -3. If a license exists under deps/ in the tarball, this is license must be linked in the LICENSE file. -3. If a license is linked from the LICENSE file, it must exist under deps/ in the tarball. - -This script will fail the check even if only the version of the dependency has changed. This is intentional. The licensing requirements of a dependency can change between versions, so if a dependency version changes, we should check that the entries for that dependency are correct in our LICENSE and NOTICE files. - -## Apache Software License, Version 2 binary dependencies - -1. Add the jar under "The following bundled 3rd party jars are distributed under the Apache Software License, Version 2." -2. Add a link to the source code of this dependency if available. This will help anyone updating the license in the future. -3. Check the LICENSE file of the dependency. - - If it only contains the Apache Software License, Version 2, nothing needs to be copied into our LICENSE file. - - If there is something other than the ASLv2, then you must figure out if it refers to code that is actually shipped in the jar. If it is shipped in the jar, then check the license of that code and apply the rules to it as you would if it was a first order dependency. -4. Check the NOTICE file of the dependency, if there is one. - - Copy any copyright notices to our NOTICE, unless they are for Apache Software Foundation (already covered by our own copyright notice), or they are refer to code covered by a BSD/MIT style license (some projects mistakenly put these in the NOTICE file, but these should be noted in the _LICENSE_ file). - - Ensure that anything copies from the NOTICE file refers to code which is actually shipped with our tarball. Some projects put optional dependencies in their NOTICE, which are not actually pulled into our distribution, so we should not include these. - -## BSD/MIT style license binary dependencies - -1. Add a section to the LICENSE file, stating that "This product bundles X, which is available under X". -2. Add the license to ```bookkeeper-dist/src/main/resources/deps/``` and add a link to this file from the LICENSE file. -3. Ensure that the deps/ license is in the inclusion lists for the correct package assembly specs (```bookkeeper-dist/src/assemble/```). -4. The section must state the path of the jar that is covered by the license, so that the tool can pick it up. -5. Add a link to the source code of the dependency if available, to make it easier to update the dependency in future. -6. Sometimes the LICENSE of a dependency refers to a dependency which they themselves has bundled. These references should be copied to our LICENSE file, as if they were first order dependencies. - -## Further resources - -- [Assembling LICENSE and NOTICE](http://www.apache.org/dev/licensing-howto.html) -- [ASF Source Header and Copyright Notice Policy](http://apache.org/legal/src-headers.html) diff --git a/site/community/mailing-lists.md b/site/community/mailing-lists.md deleted file mode 100644 index 0d64380a086..00000000000 --- a/site/community/mailing-lists.md +++ /dev/null @@ -1,50 +0,0 @@ ---- -title: BookKeeper mailing lists ---- - -Apache BookKeeper has several mailing lists that you can participate in. - -> In order to post to a mailing list, you must first subscribe to it. - -## Users - -If you use Apache BookKeeper, please subscribe to the BookKeeper user mailing list at [user@bookkeeper.apache.org](mailto:user@bookkeeper.apache.org). - -* [Subscribe](mailto:user-subscribe@bookkeeper.apache.org) -* [Unsubscribe](mailto:user-unsubscribe@bookkeeper.apache.org) -* [Archives](http://mail-archives.apache.org/mod_mbox/bookkeeper-user/) - -## Developers - -If you'd like to contribute to the Apache BookKeeper project, please subscribe to the BookKeeper developer mailing list at [dev@bookkeeper.apache.org](mailto:dev@bookkeeper.apache.org). - -* [Subscribe](mailto:dev-subscribe@bookkeeper.apache.org) -* [Unsubscribe](mailto:dev-unsubscribe@bookkeeper.apache.org) -* [Archives](http://mail-archives.apache.org/mod_mbox/bookkeeper-dev/) - -## Issues - -If you'd like to see updates from BookKeeper's JIRA, Github and Jenkins and follow what's happening in the community, please subscribe to the -BookKeeper issues mailing list at [issues@bookkeeper.apache.org](mailto:issues@bookkeeper.apache.org). - -* [Subscribe](mailto:issues-subscribe@bookkeeper.apache.org) -* [Unsubscribe](mailto:issues-unsubscribe@bookkeeper.apache.org) -* [Archives](http://mail-archives.apache.org/mod_mbox/bookkeeper-issues/) - -### Mail Filters - -This mailing list contains notifications from different systems (JIRA, Github and Jenkins). Following filter rules are useful for organizing -the emails in your inbox. - -- from:(git@git.apache.org) to:(issues@bookkeeper.apache.org): all the notifications from github repo. -- from:(jenkins@builds.apache.org) to:(issues@bookkeeper.apache.org): all the notifications from jenkins CI. -- from:(jira@apache.org) to:(dev@bookkeeper.apache.org): all the notifications from JIRA. -- to:(bookkeeper@noreply.github.com, mention@noreply.github.com, ${user email}): all the notifications when you were tagged on github. - -## Commits - -If you'd like to see changes made in BookKeeper's version control system then subscribe to the BookKeeper commit mailing list. - -* [Subscribe](mailto:commits-subscribe@bookkeeper.apache.org) -* [Unsubscribe](mailto:commits-unsubscribe@bookkeeper.apache.org) -* [Archives](http://mail-archives.apache.org/mod_mbox/bookkeeper-commits/) diff --git a/site/community/meeting.md b/site/community/meeting.md deleted file mode 100644 index 6bcd049c6b2..00000000000 --- a/site/community/meeting.md +++ /dev/null @@ -1,19 +0,0 @@ ---- -title: Community Meetings ---- - -The community meeting runs bi-weekly on Thursday 8am - 9am PST. The meeting link is [https://goo.gl/iyRA6G](https://goo.gl/iyRA6G). - -The meeting is typically comprised of 3 parts: - -- Discuss [BookKeeper Proposals](http://bookkeeper.apache.org/community/bookkeeper_proposals/). -- Review and address concerns for any open pull requests. -- Open discussion. - -As a member of the BookKeeper community, you are welcome to join any of the meetings if you are interested in. No registration required. - -<iframe src="https://calendar.google.com/calendar/embed?src=cdph8n8a4bmmvkkghc81mvkd4c%40group.calendar.google.com&ctz=America/Los_Angeles" style="border: 0" width="800" height="600" frameborder="0" scrolling="no"></iframe> - -### Past Community Meetings - -You can read the meeting notes from [past community meetings](https://cwiki.apache.org/confluence/display/BOOKKEEPER/Community+Meetings). diff --git a/site/community/presentations.md b/site/community/presentations.md deleted file mode 100644 index c5ac39be2fa..00000000000 --- a/site/community/presentations.md +++ /dev/null @@ -1,31 +0,0 @@ ---- -title: Papers and Presentations ---- - -## Papers - -- [DistributedLog: A High Performance Replicated Log Service](http://ieeexplore.ieee.org/abstract/document/7930058/), Sijie Guo, Robin Dhamankar, and Leigh Stewart. Data Engineering (ICDE), 2017 IEEE 33rd International Conference on. IEEE, 2017. -- [Durability with BookKeeper](http://dl.acm.org/citation.cfm?id=2433144), Flavio P. Junqueira, Ivan Kelly, Benjamin Reed - -## Presentations - -- [Apache BookKeeper: A High Performance and Low Latency Storage Service](http://www.slideshare.net/hustlmsp/apache-bookkeeper-a-high-performance-and-low-latency-storage-service), Sijie Guo, Linux Vault 2017 -- [Apache BookKeeper as distributed Store](http://www.slideshare.net/jujjuri/apache-con2016final), ([audio](http://feathercast.apache.org/apachecon-na-2016-low-latency-distributed-storage-services-using-apache-bookkeeper-venkateswararao-jujjuri-sijie-guo/)) JV Jujjuri, ApacheCon 2016 -- [Building a Durable Real-Time Data Pipeline: Apache BookKeeper at Twitter](https://apachebigdata2016.sched.org/event/6M74/building-a-durable-real-time-data-pipeline-apache-bookkeeper-at-twitter-sijie-guo-leigh-stewart-twitter), Leigh Stewart, Apache Big Data 2016 ( [Slides](http://schd.ws/hosted_files/apachebigdata2016/65/Building%20a%20Durable%20Real-Time%20Data%20Pipeline-Apache%20BookKeeper%20at%20Twitter.pdf) ) -- [Scale DistributedLog at Twitter](http://daxue.qq.com/content/content/id/2492), Sijie Guo, QCon Beijing, April 2016 -- [Building DistributedLog, a high-performance replicated log service](https://www.oreilly.com/ideas/twitters-real-time-data-stack?twitter=@bigdata) ([Slides](http://conferences.oreilly.com/strata/strata-ca-2016/public/schedule/detail/46897)), Sijie Guo, Strata+Hadoop World, March 2016 -- [Building DistributedLog, a high-performance replicated log service](https://youtu.be/QW1OEQxcjZc), Sijie Guo, Sep 2015 -- [Cloud Messaging Service - Technical Overview](http://www.slideshare.net/MessagingMeetup/cloud-messaging-service-technical-overview), Matteo Merli, Sep 2015 -- [Building reliable systems with Apache BookKeeper](http://www.slideshare.net/MatthieuMorel/building-reliable-systems-with-apache-bookkeeper), Matthieu Morel, Jun 2014 -- [Serving millions of journals with Apache BookKeeper](https://cwiki.apache.org/confluence/download/attachments/27832576/bk-hadoop-summit-2013.pdf?version=1&modificationDate=1363841022000&api=v2), Flavio P. Junqueira, Ivan Kelly, March 2013 -- [Namenode High Availability with BookKeeper](http://hbtc2012.hadooper.cn/subject/track1maheswara2.pdf), Uma Maheswara Rao G, At Hadoop And BigData Technology Conference 2012 -- [Durability with BookKeeper](https://cwiki.apache.org/confluence/download/attachments/27832576/bookkeeper-ladis-2012.pdf?version=1&modificationDate=1343016091000&api=v2), Flavio P. Junqueira, LADIS Workshop, July 2012 -- [BookKeeper](https://cwiki.apache.org/confluence/download/attachments/27832576/bookkeeper-hic-2011.pdf?version=1&modificationDate=1322791384000&api=v2), Flavio P. Junqueira, Hadoop in China, 2011 - -## Blog Posts - -- [Open-sourcing Pulsar, Pub-sub Messaging at Scale](https://yahooeng.tumblr.com/post/150078336821/open-sourcing-pulsar-pub-sub-messaging-at-scale), Sep 2016 -- [Majordodo - a Distributed Resource Manager built on top of Apache BookKeeper](http://eolivelli.blogspot.it/2016/03/majordodo-distributed-resource-manager.html), March 2016 -- [Building DistributedLog - Twitter's high performance replicated log service](https://blog.twitter.com/2015/building-distributedlog-twitter-s-high-performance-replicated-log-service), Sep 2015 -- [BookKeeper - Yahoo's Distributed Log Storage - is an Apache Top Level Project](http://yahooeng.tumblr.com/post/109908973316/bookkeeper-yahoos-distributed-log-storage-is), Feb 2015 -- [BookKeeper - Durability at Scale](https://developer.yahoo.com/blogs/ydn/bookkeeper-durability-scale-54048.html), Nov 2012 diff --git a/site/community/release_guide.md b/site/community/release_guide.md deleted file mode 100644 index ef2ef09ec1d..00000000000 --- a/site/community/release_guide.md +++ /dev/null @@ -1,738 +0,0 @@ ---- -title: Apache BookKeeper Release Guide -layout: community ---- - -* TOC -{:toc} - -This page documents the procedure to make an Apache BookKeeper release. - -## Introduction - -The Apache BookKeeper project periodically declares and publishes releases. A release is one or more packages of the project artifact(s) that are approved for general public distribution and use. They may come with various degrees of caveat regarding their perceived quality and potential for change, such as “alpha”, “beta”, “incubating”, “stable”, etc. - -The BookKeeper community treats releases with great importance. They are a public face of the project and most users interact with the project only through the releases. Releases are signed off by the entire BookKeeper community in a public vote. - -Each release is executed by a *Release Manager*, who is selected among the [BookKeeper committers](http://bookkeeper.apache.org/credits.html). This document describes the process that the Release Manager follows to perform a release. Any changes to this process should be discussed and adopted on the [dev@ mailing list](http://bookkeeper.apache.org/lists.html). - -Please remember that publishing software has legal consequences. This guide complements the foundation-wide [Product Release Policy](http://www.apache.org/dev/release.html) and [Release Distribution Policy](http://www.apache.org/dev/release-distribution). - -## Overview - -The release process consists of several steps: - -1. Decide to release -2. Prepare for the release -3. Build a release candidate -4. Vote on the release candidate -5. If necessary, fix any issues and go back to step 3. -6. Finalize the release -7. Promote the release - -********** - -## Decide to release - -Deciding to release and selecting a Release Manager is the first step of the release process. This is a consensus-based decision of the entire community. - -Anybody can propose a release on the dev@ mailing list, giving a solid argument and nominating a committer as the Release Manager (including themselves). There’s no formal process, no vote requirements, and no timing requirements. Any objections should be resolved by consensus before starting the release. - -In general, the community prefers to have a rotating set of 3-5 Release Managers. Keeping a small core set of managers allows enough people to build expertise in this area and improve processes over time, without Release Managers needing to re-learn the processes for each release. That said, if you are a committer interested in serving the community in this way, please reach out to the community on the dev@ mailing list. - -### Checklist to proceed to the next step - -1. Community agrees to release -2. Community selects a Release Manager - -********** - -## Prepare for the release - -Before your first release, you should perform one-time configuration steps. This will set up your security keys for signing the release and access to various release repositories. - -To prepare for each release, you should audit the project status in Github issue tracker, and do necessary bookkeeping. Finally, you should create a release branch from which individual release candidates will be built. - -### One-time setup instructions - -#### GPG Key - -You need to have a GPG key to sign the release artifacts. Please be aware of the ASF-wide [release signing guidelines](https://www.apache.org/dev/release-signing.html). -If you don’t have a GPG key associated with your Apache account, please create one according to the [guidelines](http://apache.org/dev/openpgp.html#generate-key) and [upload](https://www.apache.org/dev/release-signing.html#keyserver-upload) your key to a public key server. - -> It is important to [link](https://www.apache.org/dev/release-signing.html#apache-wot) your GPG key into the Apache web of trust. -> You can reach out other committers in Apache BookKeeper community for signing your key. - -Once you have a GPG key associated with your Apache count, then: - -**First**, Determine your Apache GPG Key and Key ID, as follows: - - gpg --list-keys - -This will list your GPG keys. One of these should reflect your Apache account, for example: - - -------------------------------------------------- - pub 2048R/845E6689 2016-02-23 - uid Nomen Nescio <anonymous@apache.org> - sub 2048R/BA4D50BE 2016-02-23 - -Here, the key ID is the 8-digit hex string in the `pub` line: `845E6689`. - -**Second**, add your Apache GPG key to the BookKeeper’s `KEYS` file in [`dist`](https://dist.apache.org/repos/dist/release/bookkeeper/KEYS). - -```shell - -# checkout the svn folder that contains the KEYS file -svn co https://dist.apache.org/repos/dist/release/bookkeeper bookkeeper_dist -cd bookkeeper_dist - -# Export the key in ascii format and append it to the file -( gpg --list-sigs $USER@apache.org - gpg --export --armor $USER@apache.org ) >> KEYS - -# Commit to svn -svn ci -m "Added gpg key for $USER" - -``` - -Once you committed, please verify if your GPG key shows up in the BookkKeeper's `KEYS` file in [`dist`](https://dist.apache.org/repos/dist/release/bookkeeper/KEYS). - -**Third**, configure `git` to use this key when signing code by giving it your key ID, as follows: - - git config --global user.signingkey 845E6689 - -You may drop the `--global` option if you’d prefer to use this key for the current repository only. - -You may wish to start `gpg-agent` to unlock your GPG key only once using your passphrase. Otherwise, you may need to enter this passphrase hundreds of times. The setup for `gpg-agent` varies based on operating system, but may be something like this: - - eval $(gpg-agent --daemon --no-grab --write-env-file $HOME/.gpg-agent-info) - export GPG_TTY=$(tty) - export GPG_AGENT_INFO - -#### Access to Apache Nexus repository - -Configure access to the [Apache Nexus repository](http://repository.apache.org/), which enables final deployment of releases to the Maven Central Repository. - -1. You log in with your Apache account. -2. Confirm you have appropriate access by finding `org.apache.bookkeeper` under `Staging Profiles`. -3. Navigate to your `Profile` (top right dropdown menu of the page). -4. Choose `User Token` from the dropdown, then click `Access User Token`. Copy a snippet of the Maven XML configuration block. -5. Insert this snippet twice into your global Maven `settings.xml` file (use command `mvn -X | grep settings`, and read out the global Maven setting file), typically `${HOME}/.m2/settings.xml`. The end result should look like this, where `TOKEN_NAME` and `TOKEN_PASSWORD` are your secret tokens: - - <settings> - <servers> - <server> - <id>apache.releases.https</id> - <username>TOKEN_NAME</username> - <password>TOKEN_PASSWORD</password> - </server> - <server> - <id>apache.snapshots.https</id> - <username>TOKEN_NAME</username> - <password>TOKEN_PASSWORD</password> - </server> - </servers> - </settings> - -#### Create an account on PyPi - -Since 4.9.0 we are releasing a python client for table service during release process. In order to publishing -a python package to PyPi, you need to [create an account](https://pypi.org/account/register/) there. After -you create the account successfully, you also need to add the account as a maintainer -for [bookkeeper-client](https://pypi.org/project/apache-bookkeeper-client/) project. You can checkout who -are the maintainers at the project page and ask them for adding your account as the maintainer. - -You can also read the instructions on [how to upload packages to PyPi](https://twine.readthedocs.io/en/latest/) -if you are interested in learning more details. - -### Create a new version in Github - -When contributors resolve an issue in GitHub, they are tagging it with a release that will contain their changes. With the release currently underway, new issues should be resolved against a subsequent future release. Therefore, you should create a release item for this subsequent release, as follows: - -1. In Github, navigate to the [`Issues > Milestones`](https://github.com/apache/bookkeeper/milestones). -2. Add a new milestone: choose the next minor version number compared to the one currently underway, select a day that is 3-months from now as the `Due Date`, write a description `Release x.y.z` and choose `Create milestone`. - -Skip this step in case of a minor release, as milestones are only for major releases. - -### Triage release-blocking issues in Github - -There could be outstanding release-blocking issues, which should be triaged before proceeding to build a release candidate. We track them by assigning a specific `Milestone` field even before the issue resolved. - -The list of release-blocking issues is available at the [milestones page](https://github.com/apache/bookkeeper/milestones). Triage each unresolved issue with one of the following resolutions: - -* If the issue has been resolved and was not updated, close it accordingly. -* If the issue has not been resolved and it is acceptable to defer this until the next release, update the `Milestone` field to the new milestone you just created. Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate. -* If the issue has not been resolved and it is not acceptable to release until it is fixed, the release cannot proceed. Instead, work with the BookKeeper community to resolve the issue. - -### Change Python Client Version - -Before cutting a release, you need to update the python client version in -[setup.py](https://github.com/apache/bookkeeper/blob/master/stream/clients/python/setup.py#L22) -from `SNAPSHOT` version to a release version and get the change merge to master. For example, -in release 4.10.0, you need to change the version from `4.10.0-alpha-0` to `4.10.0`. - -### Review Release Notes in Github - -> Github does not automatically generates Release Notes based on the `Milestone` field applied to issues. -> We can use [github-changelog-generator](https://github.com/skywinder/github-changelog-generator) to generate a ChangeLog for a milestone in future. - -For Github, we can use the milestone link in the Release Notes. E.g. [Release 4.5.0 milestone](https://github.com/apache/bookkeeper/milestone/1?closed=1). - -#### Prepare Release Notes - -After review the release notes on both Github, you should write a `releaseNotes` under `site/docs/${release_version}/overview/releaseNotes.md` and then send out a pull request for review. - -[4.5.0 Release Notes](https://github.com/apache/bookkeeper/pull/402) is a good example to follow. - -### Prepare release branch - -Release candidates are built from a release branch. As a final step in preparation for the release, you should create the release branch, push it to the code repository, and update version information on the original branch. - -Check out the version of the codebase from which you start the release. For a new minor or major release, this may be `HEAD` of the `master` branch. To build a hotfix/incremental release, instead of the `master` branch, use the release tag of the release being patched. (Please make sure your cloned repository is up-to-date before starting.) - - git checkout <master branch OR release tag> - - -Set up a few environment variables to simplify Maven commands that follow. (We use `bash` Unix syntax in this guide.) - -For a major release (for instance 4.5.0): - - MAJOR_VERSION="4.5" - VERSION="4.5.0" - NEXT_VERSION="4.6.0" - BRANCH_NAME="branch-${MAJOR_VERSION}" - DEVELOPMENT_VERSION="${NEXT_VERSION}-SNAPSHOT" - -For a minor release (for instance 4.5.1): - - MAJOR_VERSION="4.5" - VERSION="4.5.1" - NEXT_VERSION="4.5.2" - BRANCH_NAME="branch-${MAJOR_VERSION}" - DEVELOPMENT_VERSION="${NEXT_VERSION}-SNAPSHOT" - -Version represents the release currently underway, while next version specifies the anticipated next version to be released from that branch. Normally, 4.5.0 is followed by 4.6.0, while 4.5.0 is followed by 4.5.1. - -#### Create branch for major release - -If you are cutting a minor release, you can skip this step and go to section [Checkout release branch](#checkout-release-branch). - -If you are cutting a major release use Maven release plugin to create the release branch and update the current branch to use the new development version. This command applies for the new major or minor version. - -> This command automatically check in and tag your code in the code repository configured in the SCM. -> It is recommended to do a "dry run" before executing the command. To "dry run", you can provide "-DdryRun" -> at the end of this command. "dry run" will generate some temporary files in the project folder, you can remove -> them by running "mvn release:clean". - - mvn release:branch \ - -DbranchName=${BRANCH_NAME} \ - -DdevelopmentVersion=${DEVELOPMENT_VERSION} \ - [-DdryRun] - -> If you failed at the middle of running this command, please check if you have `push` permissions on `github.com`. -> You need use [personal access token](https://help.github.com/articles/creating-a-personal-access-token-for-the-command-line/) rather than your own password, if you enabled `2 factor authentication`. -> -> On failures, you need to reset on failures: -> -> $ git reset --hard apache/<master branch OR release tag> -> $ git branch -D ${BRANCH_NAME} - -##### Create CI jobs for release branch - -Once the release branch is created, please create corresponding CI jobs for the release branch. These CI jobs includes postcommit jobs for different java versions and -integration tests. - -Example PR: [release-4.7.0](https://github.com/apache/bookkeeper/pull/1328) [integration tests for release-4.7.0](https://github.com/apache/bookkeeper/pull/1353) - -#### Checkout release branch -<a name="checkout-release-branch"></a> - -Check out the release branch. - - git checkout ${BRANCH_NAME} - -The rest of this guide assumes that commands are run in the root of a repository on `${BRANCH_NAME}` with the above environment variables set. - -Verify that pom.xml contains the correct VERSION, it should still end with the '-SNAPSHOT' suffix. - -### Checklist to proceed to the next step - -1. Release Manager’s GPG key is published to `dist.apache.org` -2. Release Manager’s GPG key is configured in `git` configuration -3. Release Manager has `org.apache.bookkeeper` listed under `Staging Profiles` in Nexus -4. Release Manager’s Nexus User Token is configured in `settings.xml` -5. Github milestone item for the subsequet release has been created -6. There are no release blocking Github issues -7. Release Notes for Github Milestone is generated, audited and adjusted -8. Release branch has been created -9. Originating branch has the version information updated to the new version - -********** - -## Build a release candidate - -The core of the release process is the build-vote-fix cycle. Each cycle produces one release candidate. The Release Manager repeats this cycle until the community approves one release candidate, which is then finalized. - -> Since 4.7.0, bookkeeper is releasing a CRC32C module `circe-checksum`. so all the steps on building a release candidate should happen in linux environment. -> It ensures the release candidate built with right jni library for `circe-checksum`. - -Set up a few environment variables to simplify Maven commands that follow. This identifies the release candidate being built. Start with `release candidate number` equal to `0` and increment it for each candidate. - - RC_NUM="0" - TAG="release-${VERSION}" - RC_DIR="bookkeeper-${VERSION}-rc${RC_NUM}" - RC_TAG="v${VERSION}-rc${RC_NUM}" - -> Please make sure `gpg` command is in your $PATH. The maven release plugin use `gpg` to sign generated jars and packages. - -### Run linux docker container to build release candidate - -```shell -./dev/release/000-run-docker.sh ${RC_NUM} -``` - -After the docker process is lauched, use `cache` credential helper to cache github credentials during releasing process. - -```shell -$ git config --global credential.helper "cache --timeout=3600" -``` - -Then run a dry-run github push to apache github repo. You will be asked for typing your github password, so the password will be cached for the whole releasing process. -If your account is configured with 2FA, use your personal token as the github password. - -The remote `apache` should point to `https://github.com/apache/bookkeeper`. - -```shell -$ git push apache --dry-run -``` - -### Build and stage Java artifacts with Maven - - -Use Maven release plugin to build the release artifacts, as follows: - -```shell -./dev/release/002-release-prepare.sh -``` - -Use Maven release plugin to stage these artifacts on the Apache Nexus repository, as follows: - -```shell -./dev/release/003-release-perform.sh -``` - -> If `release:perform` failed, -> delete the release tag: git tag -d release-${VERSION} && git push apache :refs/tags/release-${VERSION} -> -> Also, you need to check the git commits on the github and if needed you may have to -> force push backed out local git branch to github again. -> -> After reset, run `./dev/release/002-release-prepare.sh` again. - -Review all staged artifacts. They should contain all relevant parts for each module, including `pom.xml`, jar, test jar, source, test source, javadoc, etc. Artifact names should follow [the existing format](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.bookkeeper%22) in which artifact name mirrors directory structure, e.g., `bookkeeper-server`. Carefully review any new artifacts. - -Close the staging repository on Apache Nexus. When prompted for a description, enter “Apache BookKeeper, version X, release candidate Y”. - -### Stage source release on dist.apache.org - -1. Copy the source release to the dev repository of `dist.apache.org`. - -```shell -./dev/release/004-stage-packages.sh -``` - -2. Verify that files are [present](https://dist.apache.org/repos/dist/dev/bookkeeper). - -### Checklist to proceed to the next step - -1. Maven artifacts deployed to the staging repository of [repository.apache.org](https://repository.apache.org/content/repositories/) -1. Source and Binary distribution deployed to the dev repository of [dist.apache.org](https://dist.apache.org/repos/dist/dev/bookkeeper/) - -********** - -## Vote on the release candidate - -Once you have built and individually reviewed the release candidate, please share it for the community-wide review. Please review foundation-wide [voting guidelines](http://www.apache.org/foundation/voting.html) for more information. - -Start the review-and-vote thread on the dev@ mailing list. Here’s an email template; please adjust as you see fit. - - From: Release Manager - To: dev@bookkeeper.apache.org - Subject: [VOTE] Release 4.5.0, release candidate #0 - - Hi everyone, - Please review and vote on the release candidate #0 for the version 0.4.0, as follows: - [ ] +1, Approve the release - [ ] -1, Do not approve the release (please provide specific comments) - - The complete staging area is available for your review, which includes: - * Release notes [1] - * The official Apache source and binary distributions to be deployed to dist.apache.org [2] - * All artifacts to be deployed to the Maven Central Repository [3] - * Source code tag "release-4.5.0" [4] with git sha XXXXXXXXXXXXXXXXXXXX - - BookKeeper's KEYS file contains PGP keys we used to sign this release: - https://dist.apache.org/repos/dist/release/bookkeeper/KEYS - - Please download these packages and review this release candidate: - - - Review release notes - - Download the source package (verify shasum, and asc) and follow the - instructions to build and run the bookkeeper service. - - Download the binary package (verify shasum, and asc) and follow the - instructions to run the bookkeeper service. - - Review maven repo, release tag, licenses, and any other things you think - it is important to a release. - - The vote will be open for at least 72 hours. It is adopted by majority approval, with at least 3 PMC affirmative votes. - - Thanks, - Release Manager - - [1] link - [2] link - [3] link - [4] link - [5] link - -If there are any issues found in the release candidate, reply on the vote thread to cancel the vote. There’s no need to wait 72 hours. Proceed to the `Fix Issues` step below and address the problem. However, some issues don’t require cancellation. For example, if an issue is found in the website pull request, just correct it on the spot and the vote can continue as-is. - -If there are no issues, reply on the vote thread to close the voting. Then, tally the votes in a separate email. Here’s an email template; please adjust as you see fit. (NOTE: the approver list are binding approvers.) - - From: Release Manager - To: dev@bookkeeper.apache.org - Subject: [RESULT] [VOTE] Release 0.4.0, release candidate #0 - - I'm happy to announce that we have unanimously approved this release. - - There are XXX approving votes, XXX of which are binding: - * approver 1 - * approver 2 - * approver 3 - * approver 4 - - There are no disapproving votes. - - Thanks everyone! - - -### Checklist to proceed to the finalization step - -1. Community votes to release the proposed candidate - -********** - -## Fix any issues - -Any issues identified during the community review and vote should be fixed in this step. - -Code changes should be proposed as standard pull requests to the `master` branch and reviewed using the normal contributing process. Then, relevant changes should be cherry-picked into the release branch. The cherry-pick commits should then be proposed as the pull requests against the release branch, again reviewed and merged using the normal contributing process. - -Once all issues have been resolved, you should go back and build a new release candidate with these changes. - -### Checklist to proceed to the next step - -1. Issues identified during vote have been resolved, with fixes committed to the release branch. - -********** - -## Finalize the release - -Once the release candidate has been reviewed and approved by the community, the release should be finalized. This involves the final deployment of the release candidate to the release repositories, merging of the website changes, etc. - -### Deploy artifacts to Maven Central Repository - -Use the Apache Nexus repository to release the staged binary artifacts to the Maven Central repository. In the `Staging Repositories` section, find the relevant release candidate `orgapachebookkeeper-XXX` entry and click `Release`. Drop all other release candidates that are not being released. - -### Deploy source release to dist.apache.org - -Copy the source release from the `dev` repository to the `release` repository at `dist.apache.org` using Subversion. - - svn move https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-${VERSION}-rc${RC_NUM} https://dist.apache.org/repos/dist/release/bookkeeper/bookkeeper-${VERSION} - -### Update Website - -1. Create the documentation for `${VERSION}`. Run the `release.sh` to generate the branch for `${VERSION}` and bump - the versions for website documentation; or run the `release_minor.sh` to release documentation when doing a - mintor release. - - ```shell - $ cd site - - // use `release.sh` for major releases - $ ./scripts/release.sh - - // or `release_minor.sh` for minor releases - $ ./scripts/release_minor.sh - ``` - - Once run the `release.sh`, please send a pull request for it and get approval from any committers, then merge it. - The CI job will automatically update the website in a few minutes. Please review the website to make sure the - documentation for `${VERSION}` is live. - -2. Merge the Release Notes pull request and make sure the Release Notes is updated. - -### Git tag - -> NOTE: Only create the release tag after the release package is showed up under https://archive.apache.org/dist/bookkeeper/ as creating the tag triggers a docker autobuild which needs the package to exist. If you forget to do so, the build will fail. In this case you can delete the tag from github and push it again. - -Create and push a new signed for the released version by copying the tag for the final release tag, as follows - -```shell -git tag -s "${TAG}" "${RC_TAG}" -git push apache "${TAG}" -``` - -Remove rc tags: - -```shell -for num in $(seq 0 ${RC_NUM}); do - git tag -d "v${VERSION}-rc${num}" - git push apache :"v${VERSION}-rc${num}" -done -``` - -### Update DC/OS BookKeeper package - -> NOTE: Please update DC/OS bookkeeper package only after the release package is showed up under https://archive.apache.org/dist/bookkeeper/ - -Once we have new version of BookKeeper docker image available at [docker hub](https://hub.docker.com/r/apache/bookkeeper/), We could update DC/OS BookKeeper package in [mesosphere universe](https://github.com/mesosphere/universe). A new pull request is needed in it. - -It is easy if only version need be bump. - -1. Clone repo [mesosphere universe](https://github.com/mesosphere/universe). - - ```shell - $ git clone https://github.com/mesosphere/universe - ``` - -2. cd into the repo, Checkout a branch for the changes. - - ```shell - $ cd universe - $ git checkout -b bookkeeper_new_version - ``` - -3. Make a copy of latest code of BookKeeper package. - - ```shell - $ cp -rf repo/packages/B/bookkeeper/1 repo/packages/B/bookkeeper/2 - $ git add repo/packages/B/bookkeeper/2 - $ git commit -m "copy old version" - ``` - -4. Bump the version of BookKeeper docker image in file [resource.json](https://github.com/mesosphere/universe/blob/version-3.x/repo/packages/B/bookkeeper/1/resource.json#L5) and [package.json](https://github.com/mesosphere/universe/blob/version-3.x/repo/packages/B/bookkeeper/1/package.json#L4). - - ``` - diff --git repo/packages/B/bookkeeper/2/package.json repo/packages/B/bookkeeper/2/package.json - index 07199d56..75f4aa81 100644 - --- repo/packages/B/bookkeeper/2/package.json - +++ repo/packages/B/bookkeeper/2/package.json - @@ -1,7 +1,7 @@ - { - "packagingVersion": "3.0", - "name": "bookkeeper", - - "version": "4.5.1", - + "version": "4.7.0", - "scm": "https://github.com/apache/bookkeeper", - "maintainer": "zhaijia@apache.org", - "description": "BookKeeper is A scalable, fault-tolerant, and low-latency storage service optimized for real-time workloads.Further information can be found here: http://bookkeeper.apache.org/", - diff --git repo/packages/B/bookkeeper/2/resource.json repo/packages/B/bookkeeper/2/resource.json - index 3801750e..72690ea0 100644 - --- repo/packages/B/bookkeeper/2/resource.json - +++ repo/packages/B/bookkeeper/2/resource.json - @@ -2,7 +2,7 @@ - "assets": { - "container": { - "docker": { - - "bookkeeper": "apache/bookkeeper:4.5.1" - + "bookkeeper": "apache/bookkeeper:4.7.0" - } - } - }, - ``` - -5. Commit the change, create a pull request and wait for it to be approved and merged. - - ```shell - $ git add repo/packages/B/bookkeeper/2 - $ git commit -m "new bookkeeper version" - ``` - -### Verify Docker Image - -> After release tag is created, it will automatically trigger docker auto build. - -1. Verify the [docker hub](https://hub.docker.com/r/apache/bookkeeper/) to see if a new build for the given tag is build. - -2. Once the new docker image is built, update BC tests to include new docker image. Example: [release-4.7.0](https://github.com/apache/bookkeeper/pull/1352) - -### Release Python Client - -Make sure you have installed [`pip`](https://pypi.org/project/pip/) and -[`twine`](https://twine.readthedocs.io/en/latest/). - -- Install Pip - ```bash - brew install pip - ``` - -- Install Twine - ```bash - pip install twine - ``` - -After install `twine`, make sure `twine` exist in your PATH before releasing python client. - -```bash -twine --version -``` - -Now, you are ready to publish the python client. - -```bash -cd stream/clients/python -./publish.sh -``` - -Check the PyPi project package to make sure the python client is uploaded to https://pypi.org/project/apache-bookkeeper-client/ . - -### Advance version on release branch - -> only do this for minor release - -Use the Maven Release plugin in order to advance the version in all poms. - -> This command will upgrade the <version> tag on every pom.xml locally to your workspace. - - mvn release:update-versions - -DdevelopmentVersion=${DEVELOPMENT_VERSION} - -Dstream - -For instance if you have released 4.5.1, you have to change version to 4.5.2-SNAPSHOT. -Then you have to create a PR and submit it for review. - -Example PR: [release-4.7.0](https://github.com/apache/bookkeeper/pull/1350) - -### Advance python client version - -If you are doing a major release, you need to update the python client version to next major development version in master -and next minor development version in the branch. For example, if you are doing 4.9.0 release, you need to bump the version -in master to `4.10.0-alpha-0` (NOTE: we are using `alpha-0` as `SNAPSHOT`, otherwise pypi doesn't work), and the version in -`branch-4.9` to `4.9.1-alpha-0`. - -If you are only doing a minor release, you just need to update the version in release branch. For example, if you are doing -4.9.1 release, you need to bump the version in `branch-4.9` to `4.9.2-alpha-0`. - -### Mark the version as released in Github - -> only do this for feature release - -In Github, inside [milestones](https://github.com/apache/bookkeeper/milestones), hover over the current milestone and click `close` button to close a milestone and set today's date as due-date. - -### Update Release Schedule - -> only do this for feature release - -Update the [release schedule](../releases) page: - -- Bump the next feature release version and update its release window. -- Update the release schedule to remove released version and add a new release. - -### Checklist to proceed to the next step - -* Maven artifacts released and indexed in the [Maven Central Repository](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.bookkeeper%22) -* Source and Binary distribution available in the release repository of [dist.apache.org](https://dist.apache.org/repos/dist/release/bookkeeper/) -* Website is updated with new release -* Docker image is built with new release -* Release tagged in the source code repository -* Release version finalized in Github -* Release section with release summary is added in [releases.md](https://github.com/apache/bookkeeper/blob/master/site/releases.md) -* Release schedule page is updated - -********** - -## Promote the release - -Once the release has been finalized, the last step of the process is to promote the release within the project and beyond. - -### Apache mailing lists - -- Announce on the dev@ mailing list that the release has been finished. -- Announce on the release on the user@ mailing list, listing major improvements and contributions. -- Announce the release on the announce@apache.org mailing list - -Use the template below for all the messages. - -> NOTE: Make sure sending the announce email using apache email, otherwise announce@apache.org will reject your email. - - - From: xxx@apache.org - To: dev@bookkeeper.apache.org, user@bookkeeper.apache.org, announce@apache.org - Subject: [ANNOUNCE] Apache BookKeeper x.y.z released - - The Apache BookKeeper team is proud to announce Apache BookKeeper version - x.y.z. - - Apache BookKeeper is a scalable, fault-tolerant, and low-latency storage service optimized for - real-time workloads. It has been used for a fundamental service to build reliable services. - It is also the log segment store for Apache DistributedLog and the message store for Apache Pulsar. - - This is the N release of the Apache BookKeeper. - - [highlights the release and why users need to try the release] - - For BookKeeper release details and downloads, visit: - - [download link] - - BookKeeper x.y.z Release Notes are at: - - [release notes link] - - We would like to thank the contributors that made the release possible. - - Regards, - - The BookKeeper Team - - -### Recordkeeping - -Use reporter.apache.org to seed the information about the release into future project reports. - -This step can be done only by PMC. - -### Social media - -Tweet, post on Facebook, LinkedIn, and other platforms. Ask other contributors to do the same. - -This step can be done only by PMC. - -### Cleanup old releases - -According to [ASF policy](http://www.apache.org/legal/release-policy.html#when-to-archive), `/www.apache.org/dist` should contain the latest release in each branch that -is currently under development. We need to remove the old releases from `release` repository. - -For example, if 4.6.1 is a newer release, we need to remove releases older than 4.6.1. - - - ```shell - $ svn del https://dist.apache.org/repos/dist/release/bookkeeper/bookkeeper-${old-release} -m "remove bookkeeper release <old-release>" - ``` - -### Checklist to declare the process completed - -1. Release announced on the user@ mailing list. -1. Blog post published, if applicable. -1. Apache Software Foundation press release published. -1. Release announced on social media. -1. Completion declared on the dev@ mailing list. - -********** - -## Improve the process - -It is important that we improve the release processes over time. Once you’ve finished the release, please take a step back and look what areas of this process and be improved. Perhaps some part of the process can be simplified. Perhaps parts of this guide can be clarified. - -If we have specific ideas, please start a discussion on the dev@ mailing list and/or propose a pull request to update this guide. Thanks! diff --git a/site/community/releases.md b/site/community/releases.md deleted file mode 100644 index 2420ce7e5d7..00000000000 --- a/site/community/releases.md +++ /dev/null @@ -1,14 +0,0 @@ ---- -title: Release Management ---- - -> Apache BookKeeper community adopts [Time Based Release Plan](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-13+-+Time+Based+Release+Plan) starting from 4.6.0. - -Apache BookKeeper community makes a feture release every 6 month (since `4.10.0`). - -- A month before the release date, the release manager will cut branches and also publish a list of features that will be included in the release. These features will typically - be [BookKeeper Proposals](http://bookkeeper.apache.org/community/bookkeeper_proposals/), but not always. -- Another week will be left for *minor* features to get in, but at this point the community will start efforts to stabilize the release branch and contribute mostly tests and fixes. -- Two weeks before the release date, the bookkeeper community will announce code-freeze and start rolling out release candidates, after which only fixes for blocking bugs will be merged. - - diff --git a/site/community/slack.md b/site/community/slack.md deleted file mode 100644 index 291082572da..00000000000 --- a/site/community/slack.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -title: BookKeeper on Slack ---- - -There is an [Apache BookKeeper](http://apachebookkeeper.slack.com/) channel that is used for informal discussions for BookKeeper developers and users. - -The Slack channel is at [http://apachebookkeeper.slack.com/](http://apachebookkeeper.slack.com/). - -You can self-register at [https://apachebookkeeper.herokuapp.com/](https://apachebookkeeper.herokuapp.com/). diff --git a/site/community/testing.md b/site/community/testing.md deleted file mode 100644 index 7d9613c7df1..00000000000 --- a/site/community/testing.md +++ /dev/null @@ -1,34 +0,0 @@ ---- -title: BookKeeper Testing Guide ---- - -* TOC -{:toc} - -## Overview - -Apache BookKeeper is a well adopted software project with a strong commitment to testing. -Consequently, it has many testing-related needs. It requires precommit tests to ensure -code going to the repository meets a certain quality bar and it requires ongoing postcommit -tests to make sure that more subtle changes which escape precommit are nonetheless caught. -This document outlines how to write tests, which tests are appropriate where, and when tests -are run, with some additional information about the testing systems at the bottom. - -## Testing Scenarios - -With the tools at our disposal, we have a good set of utilities which we can use to verify -BookKeeper correctness. To ensure an ongoing high quality of code, we use precommit and postcommit -testing. - -### Precommit - -For precommit testing, BookKeeper uses GitHub Actions to ensure that pull requests meet a certain quality bar. -These precommits verify correctness via unit/integration tests. - -In case of failures, you can re-run failing checks commenting `rerun failure checks` in the pull. -More details on GitHub actions [here](https://github.com/apache/bookkeeper/tree/master/.github/workflows/bot.yml) - -### Postcommit - -Currently in postcommit, we re-run precommit tests against both master and the most recent release branch. -In this way we can ensure also the main branches are not broken. \ No newline at end of file diff --git a/site/css/normalize.css b/site/css/normalize.css deleted file mode 100644 index fa4e73dd418..00000000000 --- a/site/css/normalize.css +++ /dev/null @@ -1,447 +0,0 @@ -/*! normalize.css v7.0.0 | MIT License | github.com/necolas/normalize.css */ - -/* Document - ========================================================================== */ - -/** - * 1. Correct the line height in all browsers. - * 2. Prevent adjustments of font size after orientation changes in - * IE on Windows Phone and in iOS. - */ - -html { - line-height: 1.15; /* 1 */ - -ms-text-size-adjust: 100%; /* 2 */ - -webkit-text-size-adjust: 100%; /* 2 */ -} - -/* Sections - ========================================================================== */ - -/** - * Remove the margin in all browsers (opinionated). - */ - -body { - margin: 0; -} - -/** - * Add the correct display in IE 9-. - */ - -article, -aside, -footer, -header, -nav, -section { - display: block; -} - -/** - * Correct the font size and margin on `h1` elements within `section` and - * `article` contexts in Chrome, Firefox, and Safari. - */ - -h1 { - font-size: 2em; - margin: 0.67em 0; -} - -/* Grouping content - ========================================================================== */ - -/** - * Add the correct display in IE 9-. - * 1. Add the correct display in IE. - */ - -figcaption, -figure, -main { /* 1 */ - display: block; -} - -/** - * Add the correct margin in IE 8. - */ - -figure { - margin: 1em 40px; -} - -/** - * 1. Add the correct box sizing in Firefox. - * 2. Show the overflow in Edge and IE. - */ - -hr { - box-sizing: content-box; /* 1 */ - height: 0; /* 1 */ - overflow: visible; /* 2 */ -} - -/** - * 1. Correct the inheritance and scaling of font size in all browsers. - * 2. Correct the odd `em` font sizing in all browsers. - */ - -pre { - font-family: monospace, monospace; /* 1 */ - font-size: 1em; /* 2 */ -} - -/* Text-level semantics - ========================================================================== */ - -/** - * 1. Remove the gray background on active links in IE 10. - * 2. Remove gaps in links underline in iOS 8+ and Safari 8+. - */ - -a { - background-color: transparent; /* 1 */ - -webkit-text-decoration-skip: objects; /* 2 */ -} - -/** - * 1. Remove the bottom border in Chrome 57- and Firefox 39-. - * 2. Add the correct text decoration in Chrome, Edge, IE, Opera, and Safari. - */ - -abbr[title] { - border-bottom: none; /* 1 */ - text-decoration: underline; /* 2 */ - text-decoration: underline dotted; /* 2 */ -} - -/** - * Prevent the duplicate application of `bolder` by the next rule in Safari 6. - */ - -b, -strong { - font-weight: inherit; -} - -/** - * Add the correct font weight in Chrome, Edge, and Safari. - */ - -b, -strong { - font-weight: bolder; -} - -/** - * 1. Correct the inheritance and scaling of font size in all browsers. - * 2. Correct the odd `em` font sizing in all browsers. - */ - -code, -kbd, -samp { - font-family: monospace, monospace; /* 1 */ - font-size: 1em; /* 2 */ -} - -/** - * Add the correct font style in Android 4.3-. - */ - -dfn { - font-style: italic; -} - -/** - * Add the correct background and color in IE 9-. - */ - -mark { - background-color: #ff0; - color: #000; -} - -/** - * Add the correct font size in all browsers. - */ - -small { - font-size: 80%; -} - -/** - * Prevent `sub` and `sup` elements from affecting the line height in - * all browsers. - */ - -sub, -sup { - font-size: 75%; - line-height: 0; - position: relative; - vertical-align: baseline; -} - -sub { - bottom: -0.25em; -} - -sup { - top: -0.5em; -} - -/* Embedded content - ========================================================================== */ - -/** - * Add the correct display in IE 9-. - */ - -audio, -video { - display: inline-block; -} - -/** - * Add the correct display in iOS 4-7. - */ - -audio:not([controls]) { - display: none; - height: 0; -} - -/** - * Remove the border on images inside links in IE 10-. - */ - -img { - border-style: none; -} - -/** - * Hide the overflow in IE. - */ - -svg:not(:root) { - overflow: hidden; -} - -/* Forms - ========================================================================== */ - -/** - * 1. Change the font styles in all browsers (opinionated). - * 2. Remove the margin in Firefox and Safari. - */ - -button, -input, -optgroup, -select, -textarea { - font-family: sans-serif; /* 1 */ - font-size: 100%; /* 1 */ - line-height: 1.15; /* 1 */ - margin: 0; /* 2 */ -} - -/** - * Show the overflow in IE. - * 1. Show the overflow in Edge. - */ - -button, -input { /* 1 */ - overflow: visible; -} - -/** - * Remove the inheritance of text transform in Edge, Firefox, and IE. - * 1. Remove the inheritance of text transform in Firefox. - */ - -button, -select { /* 1 */ - text-transform: none; -} - -/** - * 1. Prevent a WebKit bug where (2) destroys native `audio` and `video` - * controls in Android 4. - * 2. Correct the inability to style clickable types in iOS and Safari. - */ - -button, -html [type="button"], /* 1 */ -[type="reset"], -[type="submit"] { - -webkit-appearance: button; /* 2 */ -} - -/** - * Remove the inner border and padding in Firefox. - */ - -button::-moz-focus-inner, -[type="button"]::-moz-focus-inner, -[type="reset"]::-moz-focus-inner, -[type="submit"]::-moz-focus-inner { - border-style: none; - padding: 0; -} - -/** - * Restore the focus styles unset by the previous rule. - */ - -button:-moz-focusring, -[type="button"]:-moz-focusring, -[type="reset"]:-moz-focusring, -[type="submit"]:-moz-focusring { - outline: 1px dotted ButtonText; -} - -/** - * Correct the padding in Firefox. - */ - -fieldset { - padding: 0.35em 0.75em 0.625em; -} - -/** - * 1. Correct the text wrapping in Edge and IE. - * 2. Correct the color inheritance from `fieldset` elements in IE. - * 3. Remove the padding so developers are not caught out when they zero out - * `fieldset` elements in all browsers. - */ - -legend { - box-sizing: border-box; /* 1 */ - color: inherit; /* 2 */ - display: table; /* 1 */ - max-width: 100%; /* 1 */ - padding: 0; /* 3 */ - white-space: normal; /* 1 */ -} - -/** - * 1. Add the correct display in IE 9-. - * 2. Add the correct vertical alignment in Chrome, Firefox, and Opera. - */ - -progress { - display: inline-block; /* 1 */ - vertical-align: baseline; /* 2 */ -} - -/** - * Remove the default vertical scrollbar in IE. - */ - -textarea { - overflow: auto; -} - -/** - * 1. Add the correct box sizing in IE 10-. - * 2. Remove the padding in IE 10-. - */ - -[type="checkbox"], -[type="radio"] { - box-sizing: border-box; /* 1 */ - padding: 0; /* 2 */ -} - -/** - * Correct the cursor style of increment and decrement buttons in Chrome. - */ - -[type="number"]::-webkit-inner-spin-button, -[type="number"]::-webkit-outer-spin-button { - height: auto; -} - -/** - * 1. Correct the odd appearance in Chrome and Safari. - * 2. Correct the outline style in Safari. - */ - -[type="search"] { - -webkit-appearance: textfield; /* 1 */ - outline-offset: -2px; /* 2 */ -} - -/** - * Remove the inner padding and cancel buttons in Chrome and Safari on macOS. - */ - -[type="search"]::-webkit-search-cancel-button, -[type="search"]::-webkit-search-decoration { - -webkit-appearance: none; -} - -/** - * 1. Correct the inability to style clickable types in iOS and Safari. - * 2. Change font properties to `inherit` in Safari. - */ - -::-webkit-file-upload-button { - -webkit-appearance: button; /* 1 */ - font: inherit; /* 2 */ -} - -/* Interactive - ========================================================================== */ - -/* - * Add the correct display in IE 9-. - * 1. Add the correct display in Edge, IE, and Firefox. - */ - -details, /* 1 */ -menu { - display: block; -} - -/* - * Add the correct display in all browsers. - */ - -summary { - display: list-item; -} - -/* Scripting - ========================================================================== */ - -/** - * Add the correct display in IE 9-. - */ - -canvas { - display: inline-block; -} - -/** - * Add the correct display in IE. - */ - -template { - display: none; -} - -/* Hidden - ========================================================================== */ - -/** - * Add the correct display in IE 10-. - */ - -[hidden] { - display: none; -} diff --git a/site/css/style.sass b/site/css/style.sass deleted file mode 100644 index 6301fb92049..00000000000 --- a/site/css/style.sass +++ /dev/null @@ -1,189 +0,0 @@ ---- ---- - -$fa-font-path: "../fonts" -@import vendor/font-awesome/font-awesome -@import vendor/bulma/sass/utilities/initial-variables -@import variables -@import mixins -@import navbar -@import syntax -@import typography -@import vendor/bulma/bulma - -img.bk-javadoc-icon - margin-right: 1.5rem - -.bk-level - height: 4rem - - img - height: 4rem - -span.bk-javadoc-icon - margin-right: .5rem - -span.tag - margin-left: 1em - -nav.pagination - margin-top: 5rem - -@media (max-width: 767px) - .bk-docs-container - margin-top: 1rem !important - -@media (min-width: 768px) - .bk-docs-container .bk-docs-block - padding: 0 5% !important - -.bk-twitter span - color: $twitter-blue - -em - margin-right: $em-right-margin - -table - tbody - td - max-width: 300px - white-space: normal - overflow-x: scroll - -.fa-stack-overflow - &:hover - color: #f48024 - -.javadoc-button - span + span - margin-left: .5rem - -ul, li - margin: 0 - padding: 0 - -span.pop - color: $blue - -+sticky-footer - -$base-font-size: .8rem -$footer-height: 20rem - -footer.footer - z-index: 10 - height: $footer-height - -// Popovers -.popover-template - display: none - -.toc - overflow-y: scroll - bottom: 0 - top: $navbar-height - $toc-base-font-size: 1.2rem - position: sticky - max-width: 20rem - padding: 0 0 5% 0 - - .section-nav - .toc-entry - &.toc-h2 - font-size: $toc-base-font-size - - &.toc-h3 - padding-left: 10px - font-size: $toc-base-font-size * .8 - - &.toc-h4 - padding-left: 20px - font-size: $toc-base-font-size * .6 - - &.toc-h4, &.toc-h5 - display: none - - a - &:hover - color: darken($tan, 20%) - - & + .toc-entry - margin-top: $toc-base-font-size - -body - font-size: 1.25em - -h1.bk-title - font-size: $base-font-size * 4.25 - margin-bottom: 1.5rem - -h2.bk-subtitle - font-size: $base-font-size * 2 - margin-bottom: 1.5rem - -.bk-docs-container, .bk-community-container - margin: $docs-container-vertical-margin auto $docs-container-vertical-margin * 2 auto - - hr - height: 2px - - & + .pagination - margin-top: 2rem - - .bk-docs-block - .docs-title - h1.title - font-size: 2.4rem - width: 100% - - .title + .subtitle - padding-top: .75rem - font-size: $base-font-size * 1.2 - -.bk-main-content - ol ol, ol ul - margin: .5rem 0 .5rem .75rem - - ul, li, ol - margin-left: .75rem - - ul li - list-style-type: square - -.bk-docs-container - width: 80% - - .bk-main-content - min-height: 70vh - - .container - top: $navbar-height - position: sticky - flex: 1 1 auto - max-width: 240px - -.bk-community-container - width: 60% - -aside.sidebar - width: 15rem - overflow-y: scroll - bottom: 0 - - p + ul.sidebar-items - margin-top: .2rem - - ul.sidebar-items - margin-bottom: 1rem - - li - padding: .2rem 0 .2rem 1rem - font-size: 1rem - width: 100% - - &.active - background-color: $tan - border-radius: 3px - - a - color: white \ No newline at end of file diff --git a/site/css/tippy.css b/site/css/tippy.css deleted file mode 100644 index 91202c5f1d9..00000000000 --- a/site/css/tippy.css +++ /dev/null @@ -1 +0,0 @@ -.tippy-touch{cursor:pointer!important}.tippy-notransition{-webkit-transition:none!important;transition:none!important}.tippy-popper{max-width:400px;-webkit-perspective:800px;perspective:800px;z-index:9999;outline:0}.tippy-popper.html-template{max-width:96%;max-width:calc(100% - 20px)}.tippy-popper[x-placement^=top] [x-arrow]{position:absolute;width:0;height:0;border-top:7px solid #333;border-right:7px solid transparent;border-left:7px solid transparent;bottom:-7px}.tippy-popper[x-placement^=top] [x-arrow].arrow-small{border-top:5px solid #333;border-right:5px solid transparent;border-left:5px solid transparent;bottom:-5px}.tippy-popper[x-placement^=top] [x-arrow].arrow-big{border-top:10px solid #333;border-right:10px solid transparent;border-left:10px solid transparent;bottom:-10px}.tippy-popper[x-placement^=top] [x-circle]{-webkit-transform-origin:0 100%;transform-origin:0 100%}.tippy-popper[x-placement^=top] [x-circle].enter{-webkit-transform:scale(1) translate(-50%,-50%);transform:scale(1) translate(-50%,-50%);opacity:1}.tippy-popper[x-placement^=top] [x-circle].leave{-webkit-transform:scale(.45) translate(-50%,-100%);transform:scale(.45) translate(-50%,-100%);opacity:0}.tippy-popper[x-placement^=top] .tippy-tooltip.light-theme [x-circle]{background-color:#fff}.tippy-popper[x-placement^=top] .tippy-tooltip.light-theme [x-arrow]{border-top:7px solid #fff;border-right:7px solid transparent;border-left:7px solid transparent}.tippy-popper[x-placement^=top] .tippy-tooltip.light-theme [x-arrow].arrow-small{border-top:5px solid #fff;border-right:5px solid transparent;border-left:5px solid transparent}.tippy-popper[x-placement^=top] .tippy-tooltip.light-theme [x-arrow].arrow-big{border-top:10px solid #fff;border-right:10px solid transparent;border-left:10px solid transparent}.tippy-popper[x-placement^=top] .tippy-tooltip.transparent-theme [x-circle]{background-color:rgba(0,0,0,.7)}.tippy-popper[x-placement^=top] .tippy-tooltip.transparent-theme [x-arrow]{border-top:7px solid rgba(0,0,0,.7);border-right:7px solid transparent;border-left:7px solid transparent}.tippy-popper[x-placement^=top] .tippy-tooltip.transparent-theme [x-arrow].arrow-small{border-top:5px solid rgba(0,0,0,.7);border-right:5px solid transparent;border-left:5px solid transparent}.tippy-popper[x-placement^=top] .tippy-tooltip.transparent-theme [x-arrow].arrow-big{border-top:10px solid rgba(0,0,0,.7);border-right:10px solid transparent;border-left:10px solid transparent}.tippy-popper[x-placement^=top] [data-animation=perspective]{-webkit-transform-origin:bottom;transform-origin:bottom}.tippy-popper[x-placement^=top] [data-animation=perspective].enter{opacity:1;-webkit-transform:translateY(-10px) rotateX(0);transform:translateY(-10px) rotateX(0)}.tippy-popper[x-placement^=top] [data-animation=perspective].leave{opacity:0;-webkit-transform:translateY(0) rotateX(90deg);transform:translateY(0) rotateX(90deg)}.tippy-popper[x-placement^=top] [data-animation=fade].enter{opacity:1;-webkit-transform:translateY(-10px);transform:translateY(-10px)}.tippy-popper[x-placement^=top] [data-animation=fade].leave{opacity:0;-webkit-transform:translateY(-10px);transform:translateY(-10px)}.tippy-popper[x-placement^=top] [data-animation=shift].enter{opacity:1;-webkit-transform:translateY(-10px);transform:translateY(-10px)}.tippy-popper[x-placement^=top] [data-animation=shift].leave{opacity:0;-webkit-transform:translateY(0);transform:translateY(0)}.tippy-popper[x-placement^=top] [data-animation=scale].enter{opacity:1;-webkit-transform:translateY(-10px) scale(1);transform:translateY(-10px) scale(1)}.tippy-popper[x-placement^=top] [data-animation=scale].leave{opacity:0;-webkit-transform:translateY(0) scale(0);transform:translateY(0) scale(0)}.tippy-popper[x-placement^=bottom] [x-arrow]{position:absolute;width:0;height:0;border-bottom:7px solid #333;border-right:7px solid transparent;border-left:7px solid transparent;top:-7px}.tippy-popper[x-placement^=bottom] [x-arrow].arrow-small{border-bottom:5px solid #333;border-right:5px solid transparent;border-left:5px solid transparent;top:-5px}.tippy-popper[x-placement^=bottom] [x-arrow].arrow-big{border-bottom:10px solid #333;border-right:10px solid transparent;border-left:10px solid transparent;top:-10px}.tippy-popper[x-placement^=bottom] [x-circle]{-webkit-transform-origin:0 -100%;transform-origin:0 -100%}.tippy-popper[x-placement^=bottom] [x-circle].enter{-webkit-transform:scale(1) translate(-50%,-50%);transform:scale(1) translate(-50%,-50%);opacity:1}.tippy-popper[x-placement^=bottom] [x-circle].leave{-webkit-transform:scale(.45) translate(-50%,8%);transform:scale(.45) translate(-50%,8%);opacity:0}.tippy-popper[x-placement^=bottom] .tippy-tooltip.light-theme [x-circle]{background-color:#fff}.tippy-popper[x-placement^=bottom] .tippy-tooltip.light-theme [x-arrow]{border-bottom:7px solid #fff;border-right:7px solid transparent;border-left:7px solid transparent}.tippy-popper[x-placement^=bottom] .tippy-tooltip.light-theme [x-arrow].arrow-small{border-bottom:5px solid #fff;border-right:5px solid transparent;border-left:5px solid transparent}.tippy-popper[x-placement^=bottom] .tippy-tooltip.light-theme [x-arrow].arrow-big{border-bottom:10px solid #fff;border-right:10px solid transparent;border-left:10px solid transparent}.tippy-popper[x-placement^=bottom] .tippy-tooltip.transparent-theme [x-circle]{background-color:rgba(0,0,0,.7)}.tippy-popper[x-placement^=bottom] .tippy-tooltip.transparent-theme [x-arrow]{border-bottom:7px solid rgba(0,0,0,.7);border-right:7px solid transparent;border-left:7px solid transparent}.tippy-popper[x-placement^=bottom] .tippy-tooltip.transparent-theme [x-arrow].arrow-small{border-bottom:5px solid rgba(0,0,0,.7);border-right:5px solid transparent;border-left:5px solid transparent}.tippy-popper[x-placement^=bottom] .tippy-tooltip.transparent-theme [x-arrow].arrow-big{border-bottom:10px solid rgba(0,0,0,.7);border-right:10px solid transparent;border-left:10px solid transparent}.tippy-popper[x-placement^=bottom] [data-animation=perspective]{-webkit-transform-origin:top;transform-origin:top}.tippy-popper[x-placement^=bottom] [data-animation=perspective].enter{opacity:1;-webkit-transform:translateY(10px) rotateX(0);transform:translateY(10px) rotateX(0)}.tippy-popper[x-placement^=bottom] [data-animation=perspective].leave{opacity:0;-webkit-transform:translateY(0) rotateX(-90deg);transform:translateY(0) rotateX(-90deg)}.tippy-popper[x-placement^=bottom] [data-animation=fade].enter{opacity:1;-webkit-transform:translateY(10px);transform:translateY(10px)}.tippy-popper[x-placement^=bottom] [data-animation=fade].leave{opacity:0;-webkit-transform:translateY(10px);transform:translateY(10px)}.tippy-popper[x-placement^=bottom] [data-animation=shift].enter{opacity:1;-webkit-transform:translateY(10px);transform:translateY(10px)}.tippy-popper[x-placement^=bottom] [data-animation=shift].leave{opacity:0;-webkit-transform:translateY(0);transform:translateY(0)}.tippy-popper[x-placement^=bottom] [data-animation=scale].enter{opacity:1;-webkit-transform:translateY(10px) scale(1);transform:translateY(10px) scale(1)}.tippy-popper[x-placement^=bottom] [data-animation=scale].leave{opacity:0;-webkit-transform:translateY(0) scale(0);transform:translateY(0) scale(0)}.tippy-popper[x-placement^=left] [x-arrow]{position:absolute;width:0;height:0;border-left:7px solid #333;border-top:7px solid transparent;border-bottom:7px solid transparent;right:-7px}.tippy-popper[x-placement^=left] [x-arrow].arrow-small{border-left:5px solid #333;border-top:5px solid transparent;border-bottom:5px solid transparent;right:-5px}.tippy-popper[x-placement^=left] [x-arrow].arrow-big{border-left:10px solid #333;border-top:10px solid transparent;border-bottom:10px solid transparent;right:-10px}.tippy-popper[x-placement^=left] [x-circle]{-webkit-transform-origin:50% 0;transform-origin:50% 0}.tippy-popper[x-placement^=left] [x-circle].enter{-webkit-transform:scale(1) translate(-50%,-50%);transform:scale(1) translate(-50%,-50%);opacity:1}.tippy-popper[x-placement^=left] [x-circle].leave{-webkit-transform:scale(.45) translate(-50%,-50%);transform:scale(.45) translate(-50%,-50%);opacity:0}.tippy-popper[x-placement^=left] .tippy-tooltip.light-theme [x-circle]{background-color:#fff}.tippy-popper[x-placement^=left] .tippy-tooltip.light-theme [x-arrow]{border-left:7px solid #fff;border-top:7px solid transparent;border-bottom:7px solid transparent}.tippy-popper[x-placement^=left] .tippy-tooltip.light-theme [x-arrow].arrow-small{border-left:5px solid #fff;border-top:5px solid transparent;border-bottom:5px solid transparent}.tippy-popper[x-placement^=left] .tippy-tooltip.light-theme [x-arrow].arrow-big{border-left:10px solid #fff;border-top:10px solid transparent;border-bottom:10px solid transparent}.tippy-popper[x-placement^=left] .tippy-tooltip.transparent-theme [x-circle]{background-color:rgba(0,0,0,.7)}.tippy-popper[x-placement^=left] .tippy-tooltip.transparent-theme [x-arrow]{border-left:7px solid rgba(0,0,0,.7);border-top:7px solid transparent;border-bottom:7px solid transparent}.tippy-popper[x-placement^=left] .tippy-tooltip.transparent-theme [x-arrow].arrow-small{border-left:5px solid rgba(0,0,0,.7);border-top:5px solid transparent;border-bottom:5px solid transparent}.tippy-popper[x-placement^=left] .tippy-tooltip.transparent-theme [x-arrow].arrow-big{border-left:10px solid rgba(0,0,0,.7);border-top:10px solid transparent;border-bottom:10px solid transparent}.tippy-popper[x-placement^=left] [data-animation=perspective]{-webkit-transform-origin:right;transform-origin:right}.tippy-popper[x-placement^=left] [data-animation=perspective].enter{opacity:1;-webkit-transform:translateX(-10px) rotateY(0);transform:translateX(-10px) rotateY(0)}.tippy-popper[x-placement^=left] [data-animation=perspective].leave{opacity:0;-webkit-transform:translateX(0) rotateY(-90deg);transform:translateX(0) rotateY(-90deg)}.tippy-popper[x-placement^=left] [data-animation=fade].enter{opacity:1;-webkit-transform:translateX(-10px);transform:translateX(-10px)}.tippy-popper[x-placement^=left] [data-animation=fade].leave{opacity:0;-webkit-transform:translateX(-10px);transform:translateX(-10px)}.tippy-popper[x-placement^=left] [data-animation=shift].enter{opacity:1;-webkit-transform:translateX(-10px);transform:translateX(-10px)}.tippy-popper[x-placement^=left] [data-animation=shift].leave{opacity:0;-webkit-transform:translateX(0);transform:translateX(0)}.tippy-popper[x-placement^=left] [data-animation=scale].enter{opacity:1;-webkit-transform:translateX(-10px) scale(1);transform:translateX(-10px) scale(1)}.tippy-popper[x-placement^=left] [data-animation=scale].leave{opacity:0;-webkit-transform:translateX(0) scale(0);transform:translateX(0) scale(0)}.tippy-popper[x-placement^=right] [x-arrow]{position:absolute;width:0;height:0;border-right:7px solid #333;border-top:7px solid transparent;border-bottom:7px solid transparent;left:-7px}.tippy-popper[x-placement^=right] [x-arrow].arrow-small{border-right:5px solid #333;border-top:5px solid transparent;border-bottom:5px solid transparent;left:-5px}.tippy-popper[x-placement^=right] [x-arrow].arrow-big{border-right:10px solid #333;border-top:10px solid transparent;border-bottom:10px solid transparent;left:-10px}.tippy-popper[x-placement^=right] [x-circle]{-webkit-transform-origin:-50% 0;transform-origin:-50% 0}.tippy-popper[x-placement^=right] [x-circle].enter{-webkit-transform:scale(1) translate(-50%,-50%);transform:scale(1) translate(-50%,-50%);opacity:1}.tippy-popper[x-placement^=right] [x-circle].leave{-webkit-transform:scale(.45) translate(-50%,-50%);transform:scale(.45) translate(-50%,-50%);opacity:0}.tippy-popper[x-placement^=right] .tippy-tooltip.light-theme [x-circle]{background-color:#fff}.tippy-popper[x-placement^=right] .tippy-tooltip.light-theme [x-arrow]{border-right:7px solid #fff;border-top:7px solid transparent;border-bottom:7px solid transparent}.tippy-popper[x-placement^=right] .tippy-tooltip.light-theme [x-arrow].arrow-small{border-right:5px solid #fff;border-top:5px solid transparent;border-bottom:5px solid transparent}.tippy-popper[x-placement^=right] .tippy-tooltip.light-theme [x-arrow].arrow-big{border-right:10px solid #fff;border-top:10px solid transparent;border-bottom:10px solid transparent}.tippy-popper[x-placement^=right] .tippy-tooltip.transparent-theme [x-circle]{background-color:rgba(0,0,0,.7)}.tippy-popper[x-placement^=right] .tippy-tooltip.transparent-theme [x-arrow]{border-right:7px solid rgba(0,0,0,.7);border-top:7px solid transparent;border-bottom:7px solid transparent}.tippy-popper[x-placement^=right] .tippy-tooltip.transparent-theme [x-arrow].arrow-small{border-right:5px solid rgba(0,0,0,.7);border-top:5px solid transparent;border-bottom:5px solid transparent}.tippy-popper[x-placement^=right] .tippy-tooltip.transparent-theme [x-arrow].arrow-big{border-right:10px solid rgba(0,0,0,.7);border-top:10px solid transparent;border-bottom:10px solid transparent}.tippy-popper[x-placement^=right] [data-animation=perspective]{-webkit-transform-origin:left;transform-origin:left}.tippy-popper[x-placement^=right] [data-animation=perspective].enter{opacity:1;-webkit-transform:translateX(10px) rotateY(0);transform:translateX(10px) rotateY(0)}.tippy-popper[x-placement^=right] [data-animation=perspective].leave{opacity:0;-webkit-transform:translateX(0) rotateY(90deg);transform:translateX(0) rotateY(90deg)}.tippy-popper[x-placement^=right] [data-animation=fade].enter{opacity:1;-webkit-transform:translateX(10px);transform:translateX(10px)}.tippy-popper[x-placement^=right] [data-animation=fade].leave{opacity:0;-webkit-transform:translateX(10px);transform:translateX(10px)}.tippy-popper[x-placement^=right] [data-animation=shift].enter{opacity:1;-webkit-transform:translateX(10px);transform:translateX(10px)}.tippy-popper[x-placement^=right] [data-animation=shift].leave{opacity:0;-webkit-transform:translateX(0);transform:translateX(0)}.tippy-popper[x-placement^=right] [data-animation=scale].enter{opacity:1;-webkit-transform:translateX(10px) scale(1);transform:translateX(10px) scale(1)}.tippy-popper[x-placement^=right] [data-animation=scale].leave{opacity:0;-webkit-transform:translateX(0) scale(0);transform:translateX(0) scale(0)}.tippy-popper .tippy-tooltip.transparent-theme{background-color:rgba(0,0,0,.7)}.tippy-popper .tippy-tooltip.transparent-theme[data-animatefill]{background-color:transparent}.tippy-popper .tippy-tooltip.light-theme{color:#26323d;box-shadow:0 4px 20px 4px rgba(0,20,60,.1),0 4px 80px -8px rgba(0,20,60,.2);background-color:#fff}.tippy-popper .tippy-tooltip.light-theme[data-animatefill]{background-color:transparent}.tippy-tooltip{position:relative;color:#fff;border-radius:4px;font-size:.95rem;padding:.4rem .8rem;text-align:center;will-change:transform;-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale;background-color:#333;pointer-events:none}.tippy-tooltip--small{padding:.25rem .5rem;font-size:.8rem}.tippy-tooltip--big{padding:.6rem 1.2rem;font-size:1.2rem}.tippy-tooltip[data-animatefill]{overflow:hidden;background-color:transparent}.tippy-tooltip[data-interactive]{pointer-events:auto}.tippy-tooltip[data-inertia]{-webkit-transition-timing-function:cubic-bezier(.53,1,.36,.85);transition-timing-function:cubic-bezier(.53,2,.36,.85)}.tippy-tooltip [x-circle]{position:absolute;will-change:transform;background-color:#333;border-radius:50%;width:120%;width:calc(100% + 2rem);-webkit-transition:all ease-out;transition:all ease-out;left:50%;top:50%;z-index:-1;overflow:hidden}.tippy-tooltip [x-circle]:before{content:"";padding-top:75%;float:left}@media (max-width:450px){.tippy-popper{max-width:96%;max-width:calc(100% - 20px)}} \ No newline at end of file diff --git a/site/docker/Dockerfile b/site/docker/Dockerfile deleted file mode 100644 index 3643bd14e62..00000000000 --- a/site/docker/Dockerfile +++ /dev/null @@ -1,20 +0,0 @@ -# -# 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. -# - -FROM ruby:2.4.1 diff --git a/site/docker/ci.sh b/site/docker/ci.sh deleted file mode 100755 index 44c4e237eba..00000000000 --- a/site/docker/ci.sh +++ /dev/null @@ -1,64 +0,0 @@ -#!/bin/bash - -# 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. - -set -e -x -u - -SCRIPT_DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) - -export IMAGE_NAME="bookkeeper/docs" - -pushd ${SCRIPT_DIR} - -docker build --rm=true -t ${IMAGE_NAME} . - -popd - -if [ "$(uname -s)" == "Linux" ]; then - USER_NAME=${SUDO_USER:=$USER} - USER_ID=$(id -u "${USER_NAME}") - GROUP_ID=$(id -g "${USER_NAME}") - LOCAL_HOME="/home/${USER_NAME}" -else # boot2docker uid and gid - USER_NAME=$USER - USER_ID=1000 - GROUP_ID=50 - LOCAL_HOME="/Users/${USER_NAME}" -fi - -docker build -t "${IMAGE_NAME}-${USER_NAME}" - <<UserSpecificDocker -FROM ${IMAGE_NAME} -RUN groupadd --non-unique -g ${GROUP_ID} ${USER_NAME} && \ - useradd -g ${GROUP_ID} -u ${USER_ID} -k /root -m ${USER_NAME} -ENV HOME /home/${USER_NAME} -UserSpecificDocker - -BOOKKEEPER_DOC_ROOT=${SCRIPT_DIR}/.. - -pushd ${BOOKKEEPER_DOC_ROOT} - -docker run \ - --rm=true \ - -w ${BOOKKEEPER_DOC_ROOT} \ - -u "${USER}" \ - -v "${BOOKKEEPER_DOC_ROOT}:${BOOKKEEPER_DOC_ROOT}" \ - -v "${LOCAL_HOME}:/home/${USER_NAME}" \ - -e JEKYLL_ENV='production' \ - ${IMAGE_NAME}-${USER_NAME} \ - bash -c "make setup && make apache" - -popd - diff --git a/site/docker/run.sh b/site/docker/run.sh deleted file mode 100755 index f4b2bb988e0..00000000000 --- a/site/docker/run.sh +++ /dev/null @@ -1,76 +0,0 @@ -#!/bin/bash - -# 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. - -set -e -x -u - -SCRIPT_DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) - -export IMAGE_NAME="bookkeeper/docs" - -pushd ${SCRIPT_DIR} - -docker build --rm=true -t ${IMAGE_NAME} . - -popd - -if [ "$(uname -s)" == "Linux" ]; then - USER_NAME=${SUDO_USER:=$USER} - USER_ID=$(id -u "${USER_NAME}") - GROUP_ID=$(id -g "${USER_NAME}") - LOCAL_HOME="/home/${USER_NAME}" -else # boot2docker uid and gid - USER_NAME=$USER - USER_ID=1000 - GROUP_ID=50 - LOCAL_HOME="/Users/${USER_NAME}" -fi - -docker build -t "${IMAGE_NAME}-${USER_NAME}" - <<UserSpecificDocker -FROM ${IMAGE_NAME} -RUN groupadd --non-unique -g ${GROUP_ID} ${USER_NAME} && \ - useradd -l -g ${GROUP_ID} -u ${USER_ID} -k /root -m ${USER_NAME} -ENV HOME /home/${USER_NAME} -UserSpecificDocker - -BOOKKEEPER_DOC_ROOT=${SCRIPT_DIR}/.. - -CMD=" -echo -echo 'Welcome to Apache BookKeeper docs' -echo 'To build, execute' -echo ' make build' -echo 'To watch and regenerate automatically' -echo ' make serve' -echo 'and access http://localhost:4000' -echo -bash -" - -pushd ${BOOKKEEPER_DOC_ROOT} - -docker run -i -t \ - --rm=true \ - -w ${BOOKKEEPER_DOC_ROOT} \ - -u "${USER}" \ - -v "${BOOKKEEPER_DOC_ROOT}:${BOOKKEEPER_DOC_ROOT}" \ - -v "${LOCAL_HOME}:/home/${USER_NAME}" \ - -p 4000:4000 \ - ${IMAGE_NAME}-${USER_NAME} \ - bash -c "${CMD}" - -popd - diff --git a/site/docs/4.10.0/admin/autorecovery.md b/site/docs/4.10.0/admin/autorecovery.md deleted file mode 100644 index e64dbac6abe..00000000000 --- a/site/docs/4.10.0/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* an IP and port for your BookKeeper cluster's ZooKeeper ensemble -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 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 -$ bin/bookkeeper shell recover \ - 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 -$ 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 -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.10.0/admin/bookies.md b/site/docs/4.10.0/admin/bookies.md deleted file mode 100644 index 68e8a7df7e2..00000000000 --- a/site/docs/4.10.0/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 shell recover \ - 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.10.0/admin/geo-replication.md b/site/docs/4.10.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.10.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.10.0/admin/http.md b/site/docs/4.10.0/admin/http.md deleted file mode 100644 index 46dd080f106..00000000000 --- a/site/docs/4.10.0/admin/http.md +++ /dev/null @@ -1,491 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.10.0/admin/metrics.md b/site/docs/4.10.0/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.10.0/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.10.0/admin/perf.md b/site/docs/4.10.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.10.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.10.0/admin/placement.md b/site/docs/4.10.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.10.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.10.0/admin/upgrade.md b/site/docs/4.10.0/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.10.0/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.10.0/api/distributedlog-api.md b/site/docs/4.10.0/api/distributedlog-api.md deleted file mode 100644 index 5832b2176dd..00000000000 --- a/site/docs/4.10.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.10.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.10.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.10.0/api/ledger-adv-api.md b/site/docs/4.10.0/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.10.0/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.10.0/api/ledger-api.md b/site/docs/4.10.0/api/ledger-api.md deleted file mode 100644 index e498d7af288..00000000000 --- a/site/docs/4.10.0/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.10.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.10.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.10.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.10.0' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.10.0/api/overview.md b/site/docs/4.10.0/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.10.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.10.0/deployment/dcos.md b/site/docs/4.10.0/deployment/dcos.md deleted file mode 100644 index 4c84dfe846a..00000000000 --- a/site/docs/4.10.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.10.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.10.0/deployment/kubernetes.md b/site/docs/4.10.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.10.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.10.0/deployment/manual.md b/site/docs/4.10.0/deployment/manual.md deleted file mode 100644 index 49800da1822..00000000000 --- a/site/docs/4.10.0/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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 -$ 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 -$ 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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.10.0/development/codebase.md b/site/docs/4.10.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.10.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.10.0/development/protocol.md b/site/docs/4.10.0/development/protocol.md deleted file mode 100644 index f87323e40ee..00000000000 --- a/site/docs/4.10.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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` if 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub> = 3**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.10.0/getting-started/concepts.md b/site/docs/4.10.0/getting-started/concepts.md deleted file mode 100644 index e73557dc6eb..00000000000 --- a/site/docs/4.10.0/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.10.0/getting-started/installation.md b/site/docs/4.10.0/getting-started/installation.md deleted file mode 100644 index 7e92b471581..00000000000 --- a/site/docs/4.10.0/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.10.0/getting-started/run-locally.md b/site/docs/4.10.0/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.10.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.10.0/overview/overview.md b/site/docs/4.10.0/overview/overview.md deleted file mode 100644 index 9896ad4e56c..00000000000 --- a/site/docs/4.10.0/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.10.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.10.0. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.10.0 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.10.0/overview/releaseNotes.md b/site/docs/4.10.0/overview/releaseNotes.md deleted file mode 100644 index 13018ad2fb3..00000000000 --- a/site/docs/4.10.0/overview/releaseNotes.md +++ /dev/null @@ -1,29 +0,0 @@ ---- -title: Apache BookKeeper 4.10.0 Release Notes ---- - -This is the 20th release of Apache BookKeeper! - -The 4.10.0 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.9.0. - -Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.10.0](../../admin/upgrade). The technical details of -this release are summarized below. - -## News and noteworthy - -- [https://github.com/apache/bookkeeper/pull/2069] Use pure python implementation of MurmurHash -- [https://github.com/apache/bookkeeper/pull/1934] Bump Netty and GRPC version -- [https://github.com/apache/bookkeeper/pull/1907] Add new *bkctl* shell tool -- [https://github.com/apache/bookkeeper/issues/1602] Cluster Metadata Checker -- [https://github.com/apache/bookkeeper/pull/2154] Auto refresh TLS certificate at bookie-server -- [https://github.com/apache/bookkeeper/pull/2150] Improve journal throughput when journalSyncData is disabled. -- [https://github.com/apache/bookkeeper/pull/2147] Journal should respect to `flushWhenQueueEmpty` setting -- [https://github.com/apache/bookkeeper/pull/2132] Make default Bookie scripts work on JDK11+ -- [https://github.com/apache/bookkeeper/pull/2128] Allow to override default SASL service name 'bookkeeper' -- [https://github.com/apache/bookkeeper/pull/2117] BookKeeper Admin API: Implement a method to get all the Bookies -- [https://github.com/apache/bookkeeper/pull/2111] Ensure getStickyReadBookieIndex returns valid bookie index - - -## Full list of changes - -- [https://github.com/apache/bookkeeper/milestone/6](https://github.com/apache/bookkeeper/milestone/6?closed=1) diff --git a/site/docs/4.10.0/overview/releaseNotesTemplate.md b/site/docs/4.10.0/overview/releaseNotesTemplate.md deleted file mode 100644 index c29417c179f..00000000000 --- a/site/docs/4.10.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.10.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.10.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.10.0/reference/cli.md b/site/docs/4.10.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.10.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.10.0/reference/config.md b/site/docs/4.10.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.10.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.10.0/reference/metrics.md b/site/docs/4.10.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.10.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.10.0/security/overview.md b/site/docs/4.10.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.10.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.10.0/security/sasl.md b/site/docs/4.10.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.10.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.10.0/security/tls.md b/site/docs/4.10.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.10.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.10.0/security/zookeeper.md b/site/docs/4.10.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.10.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.11.0/admin/autorecovery.md b/site/docs/4.11.0/admin/autorecovery.md deleted file mode 100644 index 4ff881c929a..00000000000 --- a/site/docs/4.11.0/admin/autorecovery.md +++ /dev/null @@ -1,125 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 -$ 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 -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.11.0/admin/bookies.md b/site/docs/4.11.0/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.11.0/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.11.0/admin/decomission.md b/site/docs/4.11.0/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.11.0/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.11.0/admin/geo-replication.md b/site/docs/4.11.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.11.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.11.0/admin/http.md b/site/docs/4.11.0/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.11.0/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.11.0/admin/metrics.md b/site/docs/4.11.0/admin/metrics.md deleted file mode 100644 index fb27c01bcbc..00000000000 --- a/site/docs/4.11.0/admin/metrics.md +++ /dev/null @@ -1,37 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.11.0/admin/perf.md b/site/docs/4.11.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.11.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.11.0/admin/placement.md b/site/docs/4.11.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.11.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.11.0/admin/upgrade.md b/site/docs/4.11.0/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.11.0/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.11.0/api/distributedlog-api.md b/site/docs/4.11.0/api/distributedlog-api.md deleted file mode 100644 index a1ca9ef8116..00000000000 --- a/site/docs/4.11.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.11.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.11.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.11.0/api/ledger-adv-api.md b/site/docs/4.11.0/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.11.0/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.11.0/api/ledger-api.md b/site/docs/4.11.0/api/ledger-api.md deleted file mode 100644 index 3128ff48c6f..00000000000 --- a/site/docs/4.11.0/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.11.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.11.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.11.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.11.0' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.11.0/api/overview.md b/site/docs/4.11.0/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.11.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.11.0/deployment/dcos.md b/site/docs/4.11.0/deployment/dcos.md deleted file mode 100644 index de67539bd5a..00000000000 --- a/site/docs/4.11.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.11.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.11.0/deployment/kubernetes.md b/site/docs/4.11.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.11.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.11.0/deployment/manual.md b/site/docs/4.11.0/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.11.0/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.11.0/development/codebase.md b/site/docs/4.11.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.11.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.11.0/development/protocol.md b/site/docs/4.11.0/development/protocol.md deleted file mode 100644 index 55cc13ac937..00000000000 --- a/site/docs/4.11.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub> = 3**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.11.0/getting-started/concepts.md b/site/docs/4.11.0/getting-started/concepts.md deleted file mode 100644 index d53f9a66cc2..00000000000 --- a/site/docs/4.11.0/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.11.0/getting-started/installation.md b/site/docs/4.11.0/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.11.0/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.11.0/getting-started/run-locally.md b/site/docs/4.11.0/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.11.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.11.0/overview/overview.md b/site/docs/4.11.0/overview/overview.md deleted file mode 100644 index 71ca50af93d..00000000000 --- a/site/docs/4.11.0/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.11.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.11.0. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.11.0 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.11.0/overview/releaseNotes.md b/site/docs/4.11.0/overview/releaseNotes.md deleted file mode 100644 index a5cb8946941..00000000000 --- a/site/docs/4.11.0/overview/releaseNotes.md +++ /dev/null @@ -1,37 +0,0 @@ ---- -title: Apache BookKeeper 4.11.0 Release Notes ---- - -This is the 21th release of Apache BookKeeper! - -The 4.11.0 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.10.0. - -Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.11.0](../../admin/upgrade). The technical details of -this release are summarized below. - -## News and noteworthy - -- Upgraded ZooKeeper version from `3.4.13` to `3.5.7` with #2112 -- BookKeeper-server depends on `org.apache.httpcomponents-httpcore-4.4.9` with #2156 - - -### Changes - -- [https://github.com/apache/bookkeeper/pull/2338] Fix bookie port conflict when using LocalBookKeeper -- [https://github.com/apache/bookkeeper/pull/2333] Handle QuorumCoverage should only count unknown nodes -- [https://github.com/apache/bookkeeper/pull/2326] Update jackson version 2.11.0 -- [https://github.com/apache/bookkeeper/pull/2314] BP-38: Publish Bookie Service Info including all advertised addresses on Metadata Service and it is backward compatible -- [https://github.com/apache/bookkeeper/pull/2313] add REST API to manage auto-recovery -- [https://github.com/apache/bookkeeper/pull/2312] Support metadata decoding for list-ledger api -- [https://github.com/apache/bookkeeper/pull/2300] files: Fix TLS with with v2 protocol -- [https://github.com/apache/bookkeeper/pull/2297] Update Arquillian Cube to 1.18.2 -- [https://github.com/apache/bookkeeper/pull/2291] Update Prometheus library to 0.8.1 -- [https://github.com/apache/bookkeeper/pull/2205] Handle empty ledger segmant while replica-check -- [https://github.com/apache/bookkeeper/pull/2156] Add Hostname verification for bookie-mTLS -- [https://github.com/apache/bookkeeper/pull/2112] Update ZooKeeper dependency to 3.5.7 - - - -## Full list of changes - -- [https://github.com/apache/bookkeeper/milestone/7](https://github.com/apache/bookkeeper/milestone/6?closed=1) \ No newline at end of file diff --git a/site/docs/4.11.0/overview/releaseNotesTemplate.md b/site/docs/4.11.0/overview/releaseNotesTemplate.md deleted file mode 100644 index 9a51cda9fab..00000000000 --- a/site/docs/4.11.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.11.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.11.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.11.0/reference/cli.md b/site/docs/4.11.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.11.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.11.0/reference/config.md b/site/docs/4.11.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.11.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.11.0/reference/metrics.md b/site/docs/4.11.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.11.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.11.0/security/overview.md b/site/docs/4.11.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.11.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.11.0/security/sasl.md b/site/docs/4.11.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.11.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.11.0/security/tls.md b/site/docs/4.11.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.11.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.11.0/security/zookeeper.md b/site/docs/4.11.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.11.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.11.1/admin/autorecovery.md b/site/docs/4.11.1/admin/autorecovery.md deleted file mode 100644 index 4ff881c929a..00000000000 --- a/site/docs/4.11.1/admin/autorecovery.md +++ /dev/null @@ -1,125 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 -$ 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 -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.11.1/admin/bookies.md b/site/docs/4.11.1/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.11.1/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.11.1/admin/decomission.md b/site/docs/4.11.1/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.11.1/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.11.1/admin/geo-replication.md b/site/docs/4.11.1/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.11.1/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.11.1/admin/http.md b/site/docs/4.11.1/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.11.1/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.11.1/admin/metrics.md b/site/docs/4.11.1/admin/metrics.md deleted file mode 100644 index 0048f30289c..00000000000 --- a/site/docs/4.11.1/admin/metrics.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.11.1/admin/perf.md b/site/docs/4.11.1/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.11.1/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.11.1/admin/placement.md b/site/docs/4.11.1/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.11.1/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.11.1/admin/upgrade.md b/site/docs/4.11.1/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.11.1/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.11.1/api/distributedlog-api.md b/site/docs/4.11.1/api/distributedlog-api.md deleted file mode 100644 index f9b9a72406b..00000000000 --- a/site/docs/4.11.1/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.11.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.11.1' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.11.1/api/ledger-adv-api.md b/site/docs/4.11.1/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.11.1/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.11.1/api/ledger-api.md b/site/docs/4.11.1/api/ledger-api.md deleted file mode 100644 index 4fbe4d2cd91..00000000000 --- a/site/docs/4.11.1/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.11.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.11.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.11.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.11.1' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.11.1/api/overview.md b/site/docs/4.11.1/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.11.1/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.11.1/deployment/dcos.md b/site/docs/4.11.1/deployment/dcos.md deleted file mode 100644 index 7cf8281ccb3..00000000000 --- a/site/docs/4.11.1/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.11.1] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.11.1/deployment/kubernetes.md b/site/docs/4.11.1/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.11.1/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.11.1/deployment/manual.md b/site/docs/4.11.1/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.11.1/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.11.1/development/codebase.md b/site/docs/4.11.1/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.11.1/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.11.1/development/protocol.md b/site/docs/4.11.1/development/protocol.md deleted file mode 100644 index 55cc13ac937..00000000000 --- a/site/docs/4.11.1/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub> = 3**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.11.1/getting-started/concepts.md b/site/docs/4.11.1/getting-started/concepts.md deleted file mode 100644 index d53f9a66cc2..00000000000 --- a/site/docs/4.11.1/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.11.1/getting-started/installation.md b/site/docs/4.11.1/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.11.1/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.11.1/getting-started/run-locally.md b/site/docs/4.11.1/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.11.1/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.11.1/overview/overview.md b/site/docs/4.11.1/overview/overview.md deleted file mode 100644 index 9027f29b4fd..00000000000 --- a/site/docs/4.11.1/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.11.1 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.11.1. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.11.1 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.11.1/overview/releaseNotes.md b/site/docs/4.11.1/overview/releaseNotes.md deleted file mode 100644 index 9cbc4690521..00000000000 --- a/site/docs/4.11.1/overview/releaseNotes.md +++ /dev/null @@ -1,45 +0,0 @@ ---- -title: Apache BookKeeper 4.11.1 Release Notes ---- - -Apache BookKeeper users are encouraged to upgrade to 4.11.1. The technical details of this release are summarized -below. - -## Highlights - -- Upgrade Netty,Vertx and RocksDB -- Better error reporting in case of ZooKeeper related errors -- Fix error that prevents Garbage Collections in case of corrupted EntryLogger file -- Support for Apache ZooKeeper 3.6.x - -### Changes - -- [https://github.com/apache/bookkeeper/pull/2410] Upgrade the `vertx` version to 3.5.3 - -- [https://github.com/apache/bookkeeper/pull/2390] Issue #2385: NullPointerException in Zookeeper multiple operations execution with 3.6.1 - -- [https://github.com/apache/bookkeeper/pull/2389] Issue #2197: bkctl binary distribution needs a 'logs' directory - -- [https://github.com/apache/bookkeeper/pull/2384] Track ZooKeeper errors as causes of ZKException - -- [https://github.com/apache/bookkeeper/pull/2383] fix fillReadAheadCache stat bug - -- [https://github.com/apache/bookkeeper/pull/2381] The latency of BenchThroughputLatency may be wrong due to Integer overflow when we do a large scale benchmark test - -- [https://github.com/apache/bookkeeper/pull/2380] NP check for print BookieSocketAddress and a better format - -- [https://github.com/apache/bookkeeper/pull/2379] Updated netty,netty-boringssl and rocksdb - -- [https://github.com/apache/bookkeeper/pull/2373] Issue 2264: Bookie cannot perform Garbage Collection in case of corrupted EntryLogger file - -- [https://github.com/apache/bookkeeper/pull/2327] Bookie Client add quarantine ratio when error count exceed threshold - -- [https://github.com/apache/bookkeeper/pull/2415] Spammy log when one bookie of ensemble is down - -## Compatiblity - -This is a point release and it does not bring API changes. - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.11.1+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.11.1+is%3Aclosed) diff --git a/site/docs/4.11.1/overview/releaseNotesTemplate.md b/site/docs/4.11.1/overview/releaseNotesTemplate.md deleted file mode 100644 index 54b22776805..00000000000 --- a/site/docs/4.11.1/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.11.1 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.11.1. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.11.1/reference/cli.md b/site/docs/4.11.1/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.11.1/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.11.1/reference/config.md b/site/docs/4.11.1/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.11.1/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.11.1/reference/metrics.md b/site/docs/4.11.1/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.11.1/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.11.1/security/overview.md b/site/docs/4.11.1/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.11.1/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.11.1/security/sasl.md b/site/docs/4.11.1/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.11.1/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.11.1/security/tls.md b/site/docs/4.11.1/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.11.1/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.11.1/security/zookeeper.md b/site/docs/4.11.1/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.11.1/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.12.0/admin/autorecovery.md b/site/docs/4.12.0/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/4.12.0/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.12.0/admin/bookies.md b/site/docs/4.12.0/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.12.0/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.12.0/admin/decomission.md b/site/docs/4.12.0/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.12.0/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.12.0/admin/geo-replication.md b/site/docs/4.12.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.12.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.12.0/admin/http.md b/site/docs/4.12.0/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.12.0/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.12.0/admin/metrics.md b/site/docs/4.12.0/admin/metrics.md deleted file mode 100644 index dca236489d2..00000000000 --- a/site/docs/4.12.0/admin/metrics.md +++ /dev/null @@ -1,38 +0,0 @@ ---- -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 these 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.prometheus.PrometheusMetricsProvider` - -> 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.12.0/admin/perf.md b/site/docs/4.12.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.12.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.12.0/admin/placement.md b/site/docs/4.12.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.12.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.12.0/admin/upgrade.md b/site/docs/4.12.0/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.12.0/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.12.0/api/distributedlog-api.md b/site/docs/4.12.0/api/distributedlog-api.md deleted file mode 100644 index 33c18f731a8..00000000000 --- a/site/docs/4.12.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.12.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.12.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.12.0/api/ledger-adv-api.md b/site/docs/4.12.0/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.12.0/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.12.0/api/ledger-api.md b/site/docs/4.12.0/api/ledger-api.md deleted file mode 100644 index 447d77da0de..00000000000 --- a/site/docs/4.12.0/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.12.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.12.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.12.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.12.0' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.12.0/api/overview.md b/site/docs/4.12.0/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.12.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.12.0/deployment/dcos.md b/site/docs/4.12.0/deployment/dcos.md deleted file mode 100644 index f800bbcaf93..00000000000 --- a/site/docs/4.12.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.12.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.12.0/deployment/kubernetes.md b/site/docs/4.12.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.12.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.12.0/deployment/manual.md b/site/docs/4.12.0/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.12.0/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.12.0/development/codebase.md b/site/docs/4.12.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.12.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.12.0/development/protocol.md b/site/docs/4.12.0/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.12.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.12.0/getting-started/concepts.md b/site/docs/4.12.0/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.12.0/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.12.0/getting-started/installation.md b/site/docs/4.12.0/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.12.0/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.12.0/getting-started/run-locally.md b/site/docs/4.12.0/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.12.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.12.0/overview/overview.md b/site/docs/4.12.0/overview/overview.md deleted file mode 100644 index ff11666e4ed..00000000000 --- a/site/docs/4.12.0/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.12.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.12.0. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.12.0 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.12.0/overview/releaseNotes.md b/site/docs/4.12.0/overview/releaseNotes.md deleted file mode 100644 index 147e8a80f68..00000000000 --- a/site/docs/4.12.0/overview/releaseNotes.md +++ /dev/null @@ -1,24 +0,0 @@ ---- -title: Apache BookKeeper 4.12.0 Release Notes ---- - -This is the 23rd release of Apache BookKeeper, it is a great milestone for the project, and we are introducing a few breaking changes on the API. -There are not changes on the wire protocol, on metadata and on persisted data on disks by default, so the new version is totally compatible with the previous ones. -With BookKeeper 4.12.0 we are making a step toward better deployment on environments with dynamic network addresses with BP-41. -We are also enhancing the new Client API by adding features that were still missing, like the ability of queryng for ledger metadata. - -Apache BookKeeper users are encouraged to [upgrade to 4.12.0](../../admin/upgrade). The technical details of this release are summarized -below. - -## News and noteworthy - -- [https://github.com/apache/bookkeeper/pull/1901] Enable ExplicitLAC but default on the reader side and in the New org.apache.bookkeeper.client.api.ReadHandle API -- [https://github.com/apache/bookkeeper/issues/2396] BP-41 Bookie Network Address Change Tracking + BookieId -- [https://github.com/apache/bookkeeper/issues/2422] BP-42 List and Access LedgerMetadata on the new API -- [https://github.com/apache/bookkeeper/pull/2433] Support Java 11 and switch to Java 11 default Docker images -- [https://github.com/apache/bookkeeper/pull/2455] BP-40 clean up output for tools -- [https://github.com/apache/bookkeeper/pull/2429] Certificate role based authorization - -## Details - -https://github.com/apache/bookkeeper/issues?q=+label%3Arelease%2F4.12.0+ diff --git a/site/docs/4.12.0/overview/releaseNotesTemplate.md b/site/docs/4.12.0/overview/releaseNotesTemplate.md deleted file mode 100644 index 2314f508dee..00000000000 --- a/site/docs/4.12.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.12.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.12.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.12.0/reference/cli.md b/site/docs/4.12.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.12.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.12.0/reference/config.md b/site/docs/4.12.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.12.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.12.0/reference/metrics.md b/site/docs/4.12.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.12.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.12.0/security/overview.md b/site/docs/4.12.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.12.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.12.0/security/sasl.md b/site/docs/4.12.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.12.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.12.0/security/tls.md b/site/docs/4.12.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.12.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.12.0/security/zookeeper.md b/site/docs/4.12.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.12.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.12.1/admin/autorecovery.md b/site/docs/4.12.1/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/4.12.1/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.12.1/admin/bookies.md b/site/docs/4.12.1/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.12.1/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.12.1/admin/decomission.md b/site/docs/4.12.1/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.12.1/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.12.1/admin/geo-replication.md b/site/docs/4.12.1/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.12.1/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.12.1/admin/http.md b/site/docs/4.12.1/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.12.1/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.12.1/admin/metrics.md b/site/docs/4.12.1/admin/metrics.md deleted file mode 100644 index dca236489d2..00000000000 --- a/site/docs/4.12.1/admin/metrics.md +++ /dev/null @@ -1,38 +0,0 @@ ---- -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 these 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.prometheus.PrometheusMetricsProvider` - -> 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.12.1/admin/perf.md b/site/docs/4.12.1/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.12.1/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.12.1/admin/placement.md b/site/docs/4.12.1/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.12.1/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.12.1/admin/upgrade.md b/site/docs/4.12.1/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.12.1/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.12.1/api/distributedlog-api.md b/site/docs/4.12.1/api/distributedlog-api.md deleted file mode 100644 index 745d2e62bca..00000000000 --- a/site/docs/4.12.1/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.12.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.12.1' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.12.1/api/ledger-adv-api.md b/site/docs/4.12.1/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.12.1/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.12.1/api/ledger-api.md b/site/docs/4.12.1/api/ledger-api.md deleted file mode 100644 index 913428dba5b..00000000000 --- a/site/docs/4.12.1/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.12.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.12.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.12.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.12.1' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.12.1/api/overview.md b/site/docs/4.12.1/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.12.1/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.12.1/deployment/dcos.md b/site/docs/4.12.1/deployment/dcos.md deleted file mode 100644 index c7a00139a78..00000000000 --- a/site/docs/4.12.1/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.12.1] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.12.1/deployment/kubernetes.md b/site/docs/4.12.1/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.12.1/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.12.1/deployment/manual.md b/site/docs/4.12.1/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.12.1/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.12.1/development/codebase.md b/site/docs/4.12.1/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.12.1/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.12.1/development/protocol.md b/site/docs/4.12.1/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.12.1/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.12.1/getting-started/concepts.md b/site/docs/4.12.1/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.12.1/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.12.1/getting-started/installation.md b/site/docs/4.12.1/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.12.1/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.12.1/getting-started/run-locally.md b/site/docs/4.12.1/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.12.1/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.12.1/overview/overview.md b/site/docs/4.12.1/overview/overview.md deleted file mode 100644 index f90c4cec3ac..00000000000 --- a/site/docs/4.12.1/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.12.1 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.12.1. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.12.1 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.12.1/overview/releaseNotes.md b/site/docs/4.12.1/overview/releaseNotes.md deleted file mode 100644 index ad83c4eb02a..00000000000 --- a/site/docs/4.12.1/overview/releaseNotes.md +++ /dev/null @@ -1,19 +0,0 @@ ---- -title: Apache BookKeeper 4.12.1 Release Notes ---- - -This is the 24th release of Apache BookKeeper, it contains a few bugfixes, new features and dependency upgrades - -Apache BookKeeper users are encouraged to [upgrade to 4.12.1](../../admin/upgrade). The technical details of this release are summarized -below. - -## News and noteworthy -- [https://github.com/apache/bookkeeper/pull/2519] Allow DNSToSwitchMapping to access BookieAddressResolver -- [https://github.com/apache/bookkeeper/pull/2493] Opportunistic Striping -- [https://github.com/apache/bookkeeper/pull/2398] getBookieInfo is stuck if no bookie is up -- [https://github.com/apache/bookkeeper/pull/2491] Upgrade to Curator 5.1 -- [https://github.com/apache/bookkeeper/pull/2523] Update jcommander from 1.48 to 1.78 - -## Details - -https://github.com/apache/bookkeeper/issues?q=+label%3Arelease%2F4.12.1+ diff --git a/site/docs/4.12.1/overview/releaseNotesTemplate.md b/site/docs/4.12.1/overview/releaseNotesTemplate.md deleted file mode 100644 index 917b6c7e692..00000000000 --- a/site/docs/4.12.1/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.12.1 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.12.1. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.12.1/reference/cli.md b/site/docs/4.12.1/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.12.1/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.12.1/reference/config.md b/site/docs/4.12.1/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.12.1/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.12.1/reference/metrics.md b/site/docs/4.12.1/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.12.1/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.12.1/security/overview.md b/site/docs/4.12.1/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.12.1/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.12.1/security/sasl.md b/site/docs/4.12.1/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.12.1/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.12.1/security/tls.md b/site/docs/4.12.1/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.12.1/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.12.1/security/zookeeper.md b/site/docs/4.12.1/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.12.1/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.13.0/admin/autorecovery.md b/site/docs/4.13.0/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/4.13.0/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.13.0/admin/bookies.md b/site/docs/4.13.0/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.13.0/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.13.0/admin/decomission.md b/site/docs/4.13.0/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.13.0/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.13.0/admin/geo-replication.md b/site/docs/4.13.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.13.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.13.0/admin/http.md b/site/docs/4.13.0/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.13.0/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.13.0/admin/metrics.md b/site/docs/4.13.0/admin/metrics.md deleted file mode 100644 index 0048f30289c..00000000000 --- a/site/docs/4.13.0/admin/metrics.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.13.0/admin/perf.md b/site/docs/4.13.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.13.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.13.0/admin/placement.md b/site/docs/4.13.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.13.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.13.0/admin/upgrade.md b/site/docs/4.13.0/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.13.0/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.13.0/api/distributedlog-api.md b/site/docs/4.13.0/api/distributedlog-api.md deleted file mode 100644 index 5e5bedf783a..00000000000 --- a/site/docs/4.13.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.13.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.13.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.13.0/api/ledger-adv-api.md b/site/docs/4.13.0/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.13.0/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.13.0/api/ledger-api.md b/site/docs/4.13.0/api/ledger-api.md deleted file mode 100644 index 95a771e58cc..00000000000 --- a/site/docs/4.13.0/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.13.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.13.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.13.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.13.0' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.13.0/api/overview.md b/site/docs/4.13.0/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.13.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.13.0/deployment/dcos.md b/site/docs/4.13.0/deployment/dcos.md deleted file mode 100644 index fe45bfd5788..00000000000 --- a/site/docs/4.13.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.13.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.13.0/deployment/kubernetes.md b/site/docs/4.13.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.13.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.13.0/deployment/manual.md b/site/docs/4.13.0/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.13.0/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.13.0/development/codebase.md b/site/docs/4.13.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.13.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.13.0/development/protocol.md b/site/docs/4.13.0/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/4.13.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.13.0/getting-started/concepts.md b/site/docs/4.13.0/getting-started/concepts.md deleted file mode 100644 index 94376bbd435..00000000000 --- a/site/docs/4.13.0/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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 - -The BookKeeper metadata store maintains all the metadata of the BookKeeper cluster, such as [ledger](#ledgers) metadata, available bookies, and so on. Currently, BookKeeper uses [ZooKeeper](https://zookeeper.apache.org) for metadata storage. - -## 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.13.0/getting-started/installation.md b/site/docs/4.13.0/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.13.0/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.13.0/getting-started/run-locally.md b/site/docs/4.13.0/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.13.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.13.0/overview/overview.md b/site/docs/4.13.0/overview/overview.md deleted file mode 100644 index 598f99b20fc..00000000000 --- a/site/docs/4.13.0/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.13.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.13.0. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.13.0 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.13.0/overview/releaseNotes.md b/site/docs/4.13.0/overview/releaseNotes.md deleted file mode 100644 index c65096b163c..00000000000 --- a/site/docs/4.13.0/overview/releaseNotes.md +++ /dev/null @@ -1,38 +0,0 @@ ---- -title: Apache BookKeeper 4.13.0 Release Notes ---- - -Release 4.13 improves reliability of the Stream Storage, -brings additional configuration options for the Stream Storage and Prometheus HTTP Server, -fixes multiple bugs and brings critical dependencies up-to-date. - -Apache BookKeeper users are encouraged to upgrade to 4.13.0. The technical details of this release are summarized -below. - -## Highlights - -### Table Service (stream storage) reliability improvements - -- [https://github.com/apache/pulsar/pull/9481] Rocksdb DLCheckpoint SST file corruption in statestore -- [https://github.com/apache/bookkeeper/pull/2564] Fix SST file corruption -- [https://github.com/apache/bookkeeper/pull/2566] Handling checkpoint corruption in case of bookie crash -- [https://github.com/apache/bookkeeper/issues/2567] Save latest revision information in statestore -- [https://github.com/apache/bookkeeper/pull/2568] Save last revision in rocksdb - -### Other improvements - -- [https://github.com/apache/bookkeeper/pull/2560] Allow stream storage to use hostname instead of IP address -- [https://github.com/apache/bookkeeper/pull/2597] Skip unavailable bookies during verifyLedgerFragment -- [https://github.com/apache/bookkeeper/pull/2543] Allow to configure Prometheus HTTP Server bind address -- various fixes of the tests, documentation, etc. - -### Dependency updates - -- [https://github.com/apache/bookkeeper/pull/2580] Upgrade protobuf to 3.14.0 -- [https://github.com/apache/bookkeeper/pull/2582] Upgrading GRPC version to 1.33, Netty to 4.1.50Final and ETCD client driver -- [https://github.com/apache/bookkeeper/pull/2602] Upgrading dropwizard to 3.2.5 - -## Details - -https://github.com/apache/bookkeeper/issues?q=+label%3Arelease%2F4.13.0 - diff --git a/site/docs/4.13.0/overview/releaseNotesTemplate.md b/site/docs/4.13.0/overview/releaseNotesTemplate.md deleted file mode 100644 index 3fbebda2c72..00000000000 --- a/site/docs/4.13.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.13.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.13.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.13.0/reference/cli.md b/site/docs/4.13.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.13.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.13.0/reference/config.md b/site/docs/4.13.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.13.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.13.0/reference/metrics.md b/site/docs/4.13.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.13.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.13.0/security/overview.md b/site/docs/4.13.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.13.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.13.0/security/sasl.md b/site/docs/4.13.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.13.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.13.0/security/tls.md b/site/docs/4.13.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.13.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.13.0/security/zookeeper.md b/site/docs/4.13.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.13.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.14.0/admin/autorecovery.md b/site/docs/4.14.0/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/4.14.0/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.14.0/admin/bookies.md b/site/docs/4.14.0/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.14.0/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.14.0/admin/decomission.md b/site/docs/4.14.0/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.14.0/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.14.0/admin/geo-replication.md b/site/docs/4.14.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.14.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.14.0/admin/http.md b/site/docs/4.14.0/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.14.0/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.14.0/admin/metrics.md b/site/docs/4.14.0/admin/metrics.md deleted file mode 100644 index 0048f30289c..00000000000 --- a/site/docs/4.14.0/admin/metrics.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.14.0/admin/perf.md b/site/docs/4.14.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.14.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.14.0/admin/placement.md b/site/docs/4.14.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.14.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.14.0/admin/upgrade.md b/site/docs/4.14.0/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.14.0/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.14.0/api/distributedlog-api.md b/site/docs/4.14.0/api/distributedlog-api.md deleted file mode 100644 index ad478f4138f..00000000000 --- a/site/docs/4.14.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.14.0/api/ledger-adv-api.md b/site/docs/4.14.0/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.14.0/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.14.0/api/ledger-api.md b/site/docs/4.14.0/api/ledger-api.md deleted file mode 100644 index 910b420cdab..00000000000 --- a/site/docs/4.14.0/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.14.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.14.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.0' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.14.0/api/overview.md b/site/docs/4.14.0/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.14.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.14.0/deployment/dcos.md b/site/docs/4.14.0/deployment/dcos.md deleted file mode 100644 index 8fd26cb1611..00000000000 --- a/site/docs/4.14.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.14.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.14.0/deployment/kubernetes.md b/site/docs/4.14.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.14.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.14.0/deployment/manual.md b/site/docs/4.14.0/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.14.0/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.14.0/development/codebase.md b/site/docs/4.14.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.14.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.14.0/development/protocol.md b/site/docs/4.14.0/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/4.14.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.14.0/getting-started/concepts.md b/site/docs/4.14.0/getting-started/concepts.md deleted file mode 100644 index 94376bbd435..00000000000 --- a/site/docs/4.14.0/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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 - -The BookKeeper metadata store maintains all the metadata of the BookKeeper cluster, such as [ledger](#ledgers) metadata, available bookies, and so on. Currently, BookKeeper uses [ZooKeeper](https://zookeeper.apache.org) for metadata storage. - -## 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.14.0/getting-started/installation.md b/site/docs/4.14.0/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.14.0/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.14.0/getting-started/run-locally.md b/site/docs/4.14.0/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.14.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.14.0/overview/overview.md b/site/docs/4.14.0/overview/overview.md deleted file mode 100644 index a0a7dd97665..00000000000 --- a/site/docs/4.14.0/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.14.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.14.0. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.14.0 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.14.0/overview/releaseNotes.md b/site/docs/4.14.0/overview/releaseNotes.md deleted file mode 100644 index 23621abb6a6..00000000000 --- a/site/docs/4.14.0/overview/releaseNotes.md +++ /dev/null @@ -1,68 +0,0 @@ ---- -title: Apache BookKeeper 4.14.0 Release Notes ---- - -Release 4.14 adds FIPS compliance, improves compaction logic and the Stream Storage, -improves data reliability in the recovery scenarios, -fixes multiple bugs and brings critical dependencies up-to-date. - -Apache BookKeeper users are encouraged to upgrade to 4.14.0. -The technical details of this release are summarized below. - -## Highlights - -### Bookkeeper is FIPS compliant by default now - -- [https://github.com/apache/bookkeeper/pull/2631] Make Bookkeeper FIPS compliant by default - - FIPS is 'Federal Information Processing Standard'. - It's a set of guidelines for security functions such as encryption/decryption/RNG etc. - Applications running in FIPS mode are said to be more secure as they adhere to more stringent standards. - -### Data reliability - -- [https://github.com/apache/bookkeeper/pull/2616] Add fencing to recovery reads to avoid data loss issue - -### Table Service (stream storage) reliability improvements - -- [https://github.com/apache/bookkeeper/pull/2686] Improved handling of RocksDB tombstones -- [https://github.com/apache/bookkeeper/pull/2641] Checksum validation for SST files -- [https://github.com/apache/bookkeeper/pull/2635] Better handling of corrupted checkpoints -- [https://github.com/apache/bookkeeper/pull/2643] Adjusted default rocksDbBlockCache size to 10%/numberOfLedgers of direct memory -- [https://github.com/apache/bookkeeper/pull/2698] RocksDB log path is configurable now - -### Compaction logic improvements - -- [https://github.com/apache/bookkeeper/pull/2675] forceAllowCompaction to run only when force is set or configured interval -- [https://github.com/apache/bookkeeper/pull/2670] Allow a customer to set a limit on the duration of the major and minor compaction runs -- [https://github.com/apache/bookkeeper/pull/2645] Fix: The compaction status report is off by 1 -- [https://github.com/apache/bookkeeper/pull/2626] Allow force compact entry log when entry log compaction is disabled -- [https://github.com/apache/bookkeeper/pull/2627] Allow DBLedgerStorage to force GC by disk listener - -### Dependency updates - -- [https://github.com/apache/bookkeeper/pull/2696] SECURITY: Upgraded Netty to 4.1.63.Final -- [https://github.com/apache/bookkeeper/pull/2701] SECURITY: Removed jackson-mapper-asl dependency to resolve multiple CVEs -- [https://github.com/apache/bookkeeper/pull/2697] Upgraded Lombok to 1.18.20 (required for Java 16 support) -- [https://github.com/apache/bookkeeper/pull/2686] Upgraded rocksdb to 6.16.4 - -### Other improvements and fixes - -- [https://github.com/apache/bookkeeper/pull/2658] Fix: always select the same region set bug for RegionAwareEnsemblePlacementPolicy -- [https://github.com/apache/bookkeeper/pull/2650] Allow to attach labels to metrics -- [https://github.com/apache/bookkeeper/pull/2401] Allow to bypass journal for writes -- [https://github.com/apache/bookkeeper/pull/2710] Imposed a memory limit on the bookie journal -- [https://github.com/apache/bookkeeper/pull/2664] Bookkeeper client throttling logic is based upon entryId instead of ledgerId -- [https://github.com/apache/bookkeeper/pull/2694] Performance: unnecessary copy to heap from CompositeByteBuf -- [https://github.com/apache/bookkeeper/pull/2654] Ensure that only entries of the current ensemble are included in the ledger recovery process -- [https://github.com/apache/bookkeeper/pull/2646] Auto-throttle read operations -- [https://github.com/apache/bookkeeper/pull/2647] Limit read-ahead bytes to the size of the read cache -- [https://github.com/apache/bookkeeper/pull/2632] Fixed NetworkTopologyImpl#getLeaves returning set with null value in case of non existing scope - -### Other - -Documentation, build, CI, tests improvements - -## Details - -https://github.com/apache/bookkeeper/issues?q=+label%3Arelease%2F4.14.0 diff --git a/site/docs/4.14.0/overview/releaseNotesTemplate.md b/site/docs/4.14.0/overview/releaseNotesTemplate.md deleted file mode 100644 index db36abbdd9d..00000000000 --- a/site/docs/4.14.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.14.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to {{ site.latest_version }}. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.14.0/reference/cli.md b/site/docs/4.14.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.14.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.14.0/reference/config.md b/site/docs/4.14.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.14.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.14.0/reference/metrics.md b/site/docs/4.14.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.14.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.14.0/security/overview.md b/site/docs/4.14.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.14.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.14.0/security/sasl.md b/site/docs/4.14.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.14.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.14.0/security/tls.md b/site/docs/4.14.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.14.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.14.0/security/zookeeper.md b/site/docs/4.14.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.14.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.14.1/admin/autorecovery.md b/site/docs/4.14.1/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/4.14.1/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.14.1/admin/bookies.md b/site/docs/4.14.1/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.14.1/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.14.1/admin/decomission.md b/site/docs/4.14.1/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.14.1/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.14.1/admin/geo-replication.md b/site/docs/4.14.1/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.14.1/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.14.1/admin/http.md b/site/docs/4.14.1/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.14.1/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.14.1/admin/metrics.md b/site/docs/4.14.1/admin/metrics.md deleted file mode 100644 index 0048f30289c..00000000000 --- a/site/docs/4.14.1/admin/metrics.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.14.1/admin/perf.md b/site/docs/4.14.1/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.14.1/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.14.1/admin/placement.md b/site/docs/4.14.1/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.14.1/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.14.1/admin/upgrade.md b/site/docs/4.14.1/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.14.1/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.14.1/api/distributedlog-api.md b/site/docs/4.14.1/api/distributedlog-api.md deleted file mode 100644 index e3512c0f7cc..00000000000 --- a/site/docs/4.14.1/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.1' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.14.1/api/ledger-adv-api.md b/site/docs/4.14.1/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.14.1/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.14.1/api/ledger-api.md b/site/docs/4.14.1/api/ledger-api.md deleted file mode 100644 index ec280e47a37..00000000000 --- a/site/docs/4.14.1/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.14.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.14.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.1' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.14.1/api/overview.md b/site/docs/4.14.1/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.14.1/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.14.1/deployment/dcos.md b/site/docs/4.14.1/deployment/dcos.md deleted file mode 100644 index 8b9369d6671..00000000000 --- a/site/docs/4.14.1/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.14.1] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.14.1/deployment/kubernetes.md b/site/docs/4.14.1/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.14.1/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.14.1/deployment/manual.md b/site/docs/4.14.1/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.14.1/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.14.1/development/codebase.md b/site/docs/4.14.1/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.14.1/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.14.1/development/protocol.md b/site/docs/4.14.1/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/4.14.1/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.14.1/getting-started/concepts.md b/site/docs/4.14.1/getting-started/concepts.md deleted file mode 100644 index 94376bbd435..00000000000 --- a/site/docs/4.14.1/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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 - -The BookKeeper metadata store maintains all the metadata of the BookKeeper cluster, such as [ledger](#ledgers) metadata, available bookies, and so on. Currently, BookKeeper uses [ZooKeeper](https://zookeeper.apache.org) for metadata storage. - -## 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.14.1/getting-started/installation.md b/site/docs/4.14.1/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.14.1/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.14.1/getting-started/run-locally.md b/site/docs/4.14.1/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.14.1/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.14.1/overview/overview.md b/site/docs/4.14.1/overview/overview.md deleted file mode 100644 index 54901589b77..00000000000 --- a/site/docs/4.14.1/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.14.1 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.14.1. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.14.1 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.14.1/overview/releaseNotes.md b/site/docs/4.14.1/overview/releaseNotes.md deleted file mode 100644 index cd2ab50b414..00000000000 --- a/site/docs/4.14.1/overview/releaseNotes.md +++ /dev/null @@ -1,20 +0,0 @@ ---- -title: Apache BookKeeper 4.14.1 Release Notes ---- - -Release 4.14.1 fixes an issue with Prometheus metrics that was -found in 4.14.0. - -Apache BookKeeper users are encouraged to upgrade to 4.14.1. -The technical details of this release are summarized below. - -### Bugs - -- [https://github.com/apache/bookkeeper/pull/2718] Fix prometheus metric provider bug and add test to cover label scope - - After add label for prometheus metric by #2650, it will cause prometheus metric format check failed when no label specified for a statsLogger. The metric list as follow. - - -## Details - -https://github.com/apache/bookkeeper/issues?q=+label%3Arelease%2F4.14.1 diff --git a/site/docs/4.14.1/overview/releaseNotesTemplate.md b/site/docs/4.14.1/overview/releaseNotesTemplate.md deleted file mode 100644 index 6158d36116e..00000000000 --- a/site/docs/4.14.1/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.14.1 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to {{ site.latest_version }}. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.14.1/reference/cli.md b/site/docs/4.14.1/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.14.1/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.14.1/reference/config.md b/site/docs/4.14.1/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.14.1/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.14.1/reference/metrics.md b/site/docs/4.14.1/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.14.1/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.14.1/security/overview.md b/site/docs/4.14.1/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.14.1/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.14.1/security/sasl.md b/site/docs/4.14.1/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.14.1/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.14.1/security/tls.md b/site/docs/4.14.1/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.14.1/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.14.1/security/zookeeper.md b/site/docs/4.14.1/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.14.1/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.14.2/admin/autorecovery.md b/site/docs/4.14.2/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/4.14.2/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.14.2/admin/bookies.md b/site/docs/4.14.2/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.14.2/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.14.2/admin/decomission.md b/site/docs/4.14.2/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.14.2/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.14.2/admin/geo-replication.md b/site/docs/4.14.2/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.14.2/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.14.2/admin/http.md b/site/docs/4.14.2/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.14.2/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.14.2/admin/metrics.md b/site/docs/4.14.2/admin/metrics.md deleted file mode 100644 index 0048f30289c..00000000000 --- a/site/docs/4.14.2/admin/metrics.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.14.2/admin/perf.md b/site/docs/4.14.2/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.14.2/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.14.2/admin/placement.md b/site/docs/4.14.2/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.14.2/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.14.2/admin/upgrade.md b/site/docs/4.14.2/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.14.2/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.14.2/api/distributedlog-api.md b/site/docs/4.14.2/api/distributedlog-api.md deleted file mode 100644 index 21e8a520511..00000000000 --- a/site/docs/4.14.2/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.2' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.2' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.14.2/api/ledger-adv-api.md b/site/docs/4.14.2/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.14.2/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.14.2/api/ledger-api.md b/site/docs/4.14.2/api/ledger-api.md deleted file mode 100644 index e437afe5365..00000000000 --- a/site/docs/4.14.2/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.14.2</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.14.2</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.2' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.2' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.14.2/api/overview.md b/site/docs/4.14.2/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.14.2/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.14.2/deployment/dcos.md b/site/docs/4.14.2/deployment/dcos.md deleted file mode 100644 index 64a3d66c198..00000000000 --- a/site/docs/4.14.2/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.14.2] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.14.2/deployment/kubernetes.md b/site/docs/4.14.2/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.14.2/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.14.2/deployment/manual.md b/site/docs/4.14.2/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.14.2/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.14.2/development/codebase.md b/site/docs/4.14.2/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.14.2/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.14.2/development/protocol.md b/site/docs/4.14.2/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/4.14.2/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.14.2/getting-started/concepts.md b/site/docs/4.14.2/getting-started/concepts.md deleted file mode 100644 index 94376bbd435..00000000000 --- a/site/docs/4.14.2/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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 - -The BookKeeper metadata store maintains all the metadata of the BookKeeper cluster, such as [ledger](#ledgers) metadata, available bookies, and so on. Currently, BookKeeper uses [ZooKeeper](https://zookeeper.apache.org) for metadata storage. - -## 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.14.2/getting-started/installation.md b/site/docs/4.14.2/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.14.2/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.14.2/getting-started/run-locally.md b/site/docs/4.14.2/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.14.2/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.14.2/overview/overview.md b/site/docs/4.14.2/overview/overview.md deleted file mode 100644 index 8c0cc45df7f..00000000000 --- a/site/docs/4.14.2/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.14.2 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.14.2. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.14.2 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.14.2/overview/releaseNotes.md b/site/docs/4.14.2/overview/releaseNotes.md deleted file mode 100644 index de69f2b3a1b..00000000000 --- a/site/docs/4.14.2/overview/releaseNotes.md +++ /dev/null @@ -1,42 +0,0 @@ ---- -title: Apache BookKeeper 4.14.2 Release Notes ---- - -Release 4.14.2 fixes an issue with Prometheus metrics that was -found in 4.14.0. - -Apache BookKeeper users are encouraged to upgrade to 4.14.2. -The technical details of this release are summarized below. - -### Bugs - -- [https://github.com/apache/bookkeeper/pull/2740] Fix Bouncy Castle fips incompatible issue - - In #2631, the default BouncyCastle was changed from non-fips into fips version. But the default version of BouncyCastle in Pulsar is the non-fips one(aimed to make it compatible with the old version of Pulsar). - - Bouncy Castle provides both FIPS and non-FIPS versions, but in a JVM, it can not include both of the 2 versions(non-Fips and Fips), and we have to exclude the current version before including the other. This makes the backward compatible a little hard, and that's why Pulsar has to involve an individual module for Bouncy Castle. - -- [https://github.com/apache/bookkeeper/pull/2762] Upgrade libthrift to 0.14.2 to address multiple CVEs - - The current libthrift version 0.12.0 has multiple vulnerabilities: CVE-2019-0205 , CVE-2019-0210 , CVE-2020-13949 - -- [https://github.com/apache/bookkeeper/pull/2735] Exclude grpc-okhttp dependency - - The okhttp dependency version 2.7.4 is old and vulnerable. This dependency isn't needed and it causes Bookkeeper to be flagged for security vulnerabilities. - -- [https://github.com/apache/bookkeeper/pull/2734] Upgrade Freebuilder version and fix the dependency - - - Freebuilder 1.14.9 contains an outdate jquery js file which causes the library to be flagged as vulnerable with the highest threat level in Sonatype IQ vulnerability scanner. This also flags Bookkeeper as vulnerable with the highest threat level although it is a false positive and not an actual threat. - - - Freebuilder shouldn't be exposed as a transitive dependency - - it's an annotation processor which should be defined - - [optional in maven](https://github.com/inferred/FreeBuilder#maven) - - [compileOnly in gradle](https://github.com/inferred/FreeBuilder#gradle) - -- [https://github.com/apache/bookkeeper/pull/2693] Upgrade vertx to 3.9.8, addresses CVE-2018-12541 - - The current vertx version is 3.5.3 which has a vulnerability, CVE-2018-12541 . - -## Details - -https://github.com/apache/bookkeeper/issues?q=+label%3Arelease%2F4.14.2 diff --git a/site/docs/4.14.2/overview/releaseNotesTemplate.md b/site/docs/4.14.2/overview/releaseNotesTemplate.md deleted file mode 100644 index 845fce1acd7..00000000000 --- a/site/docs/4.14.2/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.14.2 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to {{ site.latest_version }}. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.14.2/reference/cli.md b/site/docs/4.14.2/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.14.2/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.14.2/reference/config.md b/site/docs/4.14.2/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.14.2/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.14.2/reference/metrics.md b/site/docs/4.14.2/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.14.2/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.14.2/security/overview.md b/site/docs/4.14.2/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.14.2/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.14.2/security/sasl.md b/site/docs/4.14.2/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.14.2/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.14.2/security/tls.md b/site/docs/4.14.2/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.14.2/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.14.2/security/zookeeper.md b/site/docs/4.14.2/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.14.2/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.14.3/admin/autorecovery.md b/site/docs/4.14.3/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/4.14.3/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.14.3/admin/bookies.md b/site/docs/4.14.3/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.14.3/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.14.3/admin/decomission.md b/site/docs/4.14.3/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.14.3/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.14.3/admin/geo-replication.md b/site/docs/4.14.3/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.14.3/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.14.3/admin/http.md b/site/docs/4.14.3/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.14.3/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.14.3/admin/metrics.md b/site/docs/4.14.3/admin/metrics.md deleted file mode 100644 index 0048f30289c..00000000000 --- a/site/docs/4.14.3/admin/metrics.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.14.3/admin/perf.md b/site/docs/4.14.3/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.14.3/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.14.3/admin/placement.md b/site/docs/4.14.3/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.14.3/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.14.3/admin/upgrade.md b/site/docs/4.14.3/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.14.3/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.14.3/api/distributedlog-api.md b/site/docs/4.14.3/api/distributedlog-api.md deleted file mode 100644 index a3829a31a2f..00000000000 --- a/site/docs/4.14.3/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.3' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.3' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.14.3/api/ledger-adv-api.md b/site/docs/4.14.3/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.14.3/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.14.3/api/ledger-api.md b/site/docs/4.14.3/api/ledger-api.md deleted file mode 100644 index 2f3eeb18d4f..00000000000 --- a/site/docs/4.14.3/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.14.3</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.14.3</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.3' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.3' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.14.3/api/overview.md b/site/docs/4.14.3/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.14.3/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.14.3/deployment/dcos.md b/site/docs/4.14.3/deployment/dcos.md deleted file mode 100644 index d023482d537..00000000000 --- a/site/docs/4.14.3/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.14.3] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.14.3/deployment/kubernetes.md b/site/docs/4.14.3/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.14.3/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.14.3/deployment/manual.md b/site/docs/4.14.3/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.14.3/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.14.3/development/codebase.md b/site/docs/4.14.3/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.14.3/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.14.3/development/protocol.md b/site/docs/4.14.3/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/4.14.3/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.14.3/getting-started/concepts.md b/site/docs/4.14.3/getting-started/concepts.md deleted file mode 100644 index 94376bbd435..00000000000 --- a/site/docs/4.14.3/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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 - -The BookKeeper metadata store maintains all the metadata of the BookKeeper cluster, such as [ledger](#ledgers) metadata, available bookies, and so on. Currently, BookKeeper uses [ZooKeeper](https://zookeeper.apache.org) for metadata storage. - -## 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.14.3/getting-started/installation.md b/site/docs/4.14.3/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.14.3/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.14.3/getting-started/run-locally.md b/site/docs/4.14.3/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.14.3/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.14.3/overview/overview.md b/site/docs/4.14.3/overview/overview.md deleted file mode 100644 index 0b0bdde0a8d..00000000000 --- a/site/docs/4.14.3/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.14.3 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.14.3. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.14.3 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.14.3/overview/releaseNotes.md b/site/docs/4.14.3/overview/releaseNotes.md deleted file mode 100644 index 358fe1f157e..00000000000 --- a/site/docs/4.14.3/overview/releaseNotes.md +++ /dev/null @@ -1,33 +0,0 @@ ---- -title: Apache BookKeeper 4.14.3 Release Notes ---- - -Release 4.14.3 includes multiple stability, performance, and security fixes along with the fix for Prometheus metrics. - -Apache BookKeeper users are encouraged to upgrade to 4.14.3. -The technical details of this release are summarized below. - -## Highlights - -### Improvements - -- [https://github.com/apache/bookkeeper/pull/2768] Add metrics and internal command for AutoRecovery -- [https://github.com/apache/bookkeeper/pull/2788] Fix npe when pulsar ZkBookieRackAffinityMapping getBookieAddressResolver -- [https://github.com/apache/bookkeeper/pull/2794] Heap memory leak problem when ledger replication failed -- [https://github.com/apache/bookkeeper/pull/2802] Add auditor get ledger throttle to avoid auto recovery zk session expire -- [https://github.com/apache/bookkeeper/pull/2813] Add ensemble check to over-replicated ledger GC -- [https://github.com/apache/bookkeeper/pull/2832] Fix semaphore leak when EntryMemTable#addEntry accepts the same entries -- [https://github.com/apache/bookkeeper/pull/2833] Eliminate direct ZK access in ScanAndCompareGarbageCollector -- [https://github.com/apache/bookkeeper/pull/2842] Remove direct ZK access for Auditor -- [https://github.com/apache/bookkeeper/pull/2844] Add error handling to readLedgerMetadata in over-replicated ledger GC -- [https://github.com/apache/bookkeeper/pull/2845] A empty implmentation of newLedgerAuditorManager in EtcdLedgerManagerFactory to fix build - -### Dependency updates - -- [https://github.com/apache/bookkeeper/pull/2792] Upgraded dependencies with CVEs -- [https://github.com/apache/bookkeeper/pull/2793] Upgrade httpclient from 4.5.5 to 4.5.13 to address CVE-2020-13956 -- [https://github.com/apache/bookkeeper/pull/2811] Upgrade Netty to 4.1.68.Final - -## Details - -https://github.com/apache/bookkeeper/issues?q=+label%3Arelease%2F4.14.3 diff --git a/site/docs/4.14.3/overview/releaseNotesTemplate.md b/site/docs/4.14.3/overview/releaseNotesTemplate.md deleted file mode 100644 index c35d3f9dcd1..00000000000 --- a/site/docs/4.14.3/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.14.3 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to {{ site.latest_version }}. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.14.3/reference/cli.md b/site/docs/4.14.3/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.14.3/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.14.3/reference/config.md b/site/docs/4.14.3/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.14.3/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.14.3/reference/metrics.md b/site/docs/4.14.3/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.14.3/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.14.3/security/overview.md b/site/docs/4.14.3/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.14.3/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.14.3/security/sasl.md b/site/docs/4.14.3/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.14.3/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.14.3/security/tls.md b/site/docs/4.14.3/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.14.3/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.14.3/security/zookeeper.md b/site/docs/4.14.3/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.14.3/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.14.4/admin/autorecovery.md b/site/docs/4.14.4/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/4.14.4/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.14.4/admin/bookies.md b/site/docs/4.14.4/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.14.4/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.14.4/admin/decomission.md b/site/docs/4.14.4/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.14.4/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.14.4/admin/geo-replication.md b/site/docs/4.14.4/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.14.4/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.14.4/admin/http.md b/site/docs/4.14.4/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.14.4/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.14.4/admin/metrics.md b/site/docs/4.14.4/admin/metrics.md deleted file mode 100644 index 0048f30289c..00000000000 --- a/site/docs/4.14.4/admin/metrics.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.14.4/admin/perf.md b/site/docs/4.14.4/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.14.4/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.14.4/admin/placement.md b/site/docs/4.14.4/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.14.4/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.14.4/admin/upgrade.md b/site/docs/4.14.4/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.14.4/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.14.4/api/distributedlog-api.md b/site/docs/4.14.4/api/distributedlog-api.md deleted file mode 100644 index 954d6b7e64e..00000000000 --- a/site/docs/4.14.4/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.4' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.4' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.14.4/api/ledger-adv-api.md b/site/docs/4.14.4/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.14.4/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.14.4/api/ledger-api.md b/site/docs/4.14.4/api/ledger-api.md deleted file mode 100644 index 29891732dc4..00000000000 --- a/site/docs/4.14.4/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.14.4</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.14.4</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.14.4' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.14.4' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.14.4/api/overview.md b/site/docs/4.14.4/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.14.4/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.14.4/deployment/dcos.md b/site/docs/4.14.4/deployment/dcos.md deleted file mode 100644 index 54d6df96edf..00000000000 --- a/site/docs/4.14.4/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.14.4] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.14.4/deployment/kubernetes.md b/site/docs/4.14.4/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.14.4/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.14.4/deployment/manual.md b/site/docs/4.14.4/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.14.4/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.14.4/development/codebase.md b/site/docs/4.14.4/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.14.4/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.14.4/development/protocol.md b/site/docs/4.14.4/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/4.14.4/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.14.4/getting-started/concepts.md b/site/docs/4.14.4/getting-started/concepts.md deleted file mode 100644 index 94376bbd435..00000000000 --- a/site/docs/4.14.4/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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 - -The BookKeeper metadata store maintains all the metadata of the BookKeeper cluster, such as [ledger](#ledgers) metadata, available bookies, and so on. Currently, BookKeeper uses [ZooKeeper](https://zookeeper.apache.org) for metadata storage. - -## 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.14.4/getting-started/installation.md b/site/docs/4.14.4/getting-started/installation.md deleted file mode 100644 index 9f856ec1d21..00000000000 --- a/site/docs/4.14.4/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.8](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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.14.4/getting-started/run-locally.md b/site/docs/4.14.4/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.14.4/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.14.4/overview/overview.md b/site/docs/4.14.4/overview/overview.md deleted file mode 100644 index d05c6eaf2ed..00000000000 --- a/site/docs/4.14.4/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.14.4 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.14.4. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.14.4 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.14.4/overview/releaseNotes.md b/site/docs/4.14.4/overview/releaseNotes.md deleted file mode 100644 index 10b5d130e89..00000000000 --- a/site/docs/4.14.4/overview/releaseNotes.md +++ /dev/null @@ -1,25 +0,0 @@ ---- -title: Apache BookKeeper 4.14.4 Release Notes ---- - -Release 4.14.4 includes multiple stability, performance, and security fixes. - -Apache BookKeeper users are encouraged to upgrade to 4.14.4. -The technical details of this release are summarized below. - -## Highlights - -### Improvements - -- [https://github.com/apache/bookkeeper/pull/2952] Allow to easily override zkServers with metadataServiceUri -- [https://github.com/apache/bookkeeper/pull/2935] ReplicationWorker should not try to create a ZK based LedgerManagerFactory -- [https://github.com/apache/bookkeeper/pull/2870] Add skip unrecoverable ledger option for bookkeeper shell recover command -- [https://github.com/apache/bookkeeper/pull/2847] ISSUE #2846 Allow to run on java 17 - -### Dependency updates - -- [https://github.com/apache/bookkeeper/pull/2934] Upgrade to Grpc 1.42.1 - -## Details - -https://github.com/apache/bookkeeper/issues?q=+label%3Arelease%2F4.14.4 diff --git a/site/docs/4.14.4/overview/releaseNotesTemplate.md b/site/docs/4.14.4/overview/releaseNotesTemplate.md deleted file mode 100644 index 4c89b7c7070..00000000000 --- a/site/docs/4.14.4/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.14.4 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to {{ site.latest_version }}. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.14.4/reference/cli.md b/site/docs/4.14.4/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.14.4/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.14.4/reference/config.md b/site/docs/4.14.4/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.14.4/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.14.4/reference/metrics.md b/site/docs/4.14.4/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.14.4/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.14.4/security/overview.md b/site/docs/4.14.4/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.14.4/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.14.4/security/sasl.md b/site/docs/4.14.4/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.14.4/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.14.4/security/tls.md b/site/docs/4.14.4/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.14.4/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.14.4/security/zookeeper.md b/site/docs/4.14.4/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.14.4/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.15.0/admin/autorecovery.md b/site/docs/4.15.0/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/4.15.0/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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.15.0/admin/bookies.md b/site/docs/4.15.0/admin/bookies.md deleted file mode 100644 index 418c4ca1b79..00000000000 --- a/site/docs/4.15.0/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.15.0/admin/decomission.md b/site/docs/4.15.0/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/4.15.0/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/4.15.0/admin/geo-replication.md b/site/docs/4.15.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.15.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.15.0/admin/http.md b/site/docs/4.15.0/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/4.15.0/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.15.0/admin/metrics.md b/site/docs/4.15.0/admin/metrics.md deleted file mode 100644 index 0048f30289c..00000000000 --- a/site/docs/4.15.0/admin/metrics.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.15.0/admin/perf.md b/site/docs/4.15.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.15.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.15.0/admin/placement.md b/site/docs/4.15.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.15.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.15.0/admin/upgrade.md b/site/docs/4.15.0/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.15.0/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.15.0/api/distributedlog-api.md b/site/docs/4.15.0/api/distributedlog-api.md deleted file mode 100644 index 5a197758caa..00000000000 --- a/site/docs/4.15.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.15.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.15.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.15.0/api/ledger-adv-api.md b/site/docs/4.15.0/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.15.0/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.15.0/api/ledger-api.md b/site/docs/4.15.0/api/ledger-api.md deleted file mode 100644 index ced244516ca..00000000000 --- a/site/docs/4.15.0/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.15.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.15.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.15.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.15.0' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.15.0/api/overview.md b/site/docs/4.15.0/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.15.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.15.0/deployment/dcos.md b/site/docs/4.15.0/deployment/dcos.md deleted file mode 100644 index bf59afc6cbd..00000000000 --- a/site/docs/4.15.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.15.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.15.0/deployment/kubernetes.md b/site/docs/4.15.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.15.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.15.0/deployment/manual.md b/site/docs/4.15.0/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/4.15.0/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.15.0/development/codebase.md b/site/docs/4.15.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.15.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.15.0/development/protocol.md b/site/docs/4.15.0/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/4.15.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.15.0/getting-started/concepts.md b/site/docs/4.15.0/getting-started/concepts.md deleted file mode 100644 index 94376bbd435..00000000000 --- a/site/docs/4.15.0/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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 - -The BookKeeper metadata store maintains all the metadata of the BookKeeper cluster, such as [ledger](#ledgers) metadata, available bookies, and so on. Currently, BookKeeper uses [ZooKeeper](https://zookeeper.apache.org) for metadata storage. - -## 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.15.0/getting-started/installation.md b/site/docs/4.15.0/getting-started/installation.md deleted file mode 100644 index dcc4a82a24e..00000000000 --- a/site/docs/4.15.0/getting-started/installation.md +++ /dev/null @@ -1,51 +0,0 @@ ---- -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.8](http://www.oracle.com/technetwork/java/javase/downloads/index.html) or later - -## Download - -You can download Apache BookKeeper releases from one of many [Apache mirrors](https://dlcdn.apache.org/bookkeeper/). - -## Clone - -To build BookKeeper from source, clone the repository, either from the [GitHub mirror]({{ site.github_repo }}): - -```shell -$ git clone {{ site.github_repo}} -``` - -## Build using Gradle - -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 Gradle: - -```shell -$ ./gradlew build -x signDistTar -x test -``` - -To run all the tests: - -```shell -$ ./gradlew test -x signDistTar -``` - -## 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.15.0/getting-started/run-locally.md b/site/docs/4.15.0/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/4.15.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.15.0/overview/overview.md b/site/docs/4.15.0/overview/overview.md deleted file mode 100644 index 0644c537120..00000000000 --- a/site/docs/4.15.0/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.15.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.15.0. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.15.0 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.15.0/overview/releaseNotes.md b/site/docs/4.15.0/overview/releaseNotes.md deleted file mode 100644 index 27eb0ebf71e..00000000000 --- a/site/docs/4.15.0/overview/releaseNotes.md +++ /dev/null @@ -1,107 +0,0 @@ ---- -title: Apache BookKeeper 4.15.0 Release Notes ---- - -Release 4.15 includes many upgrades to third party libraries marked with CVEs, -adds more configuration options, extends REST API, -adds an option to run without journal, improves memory utilization and stability, and more! - -Apache BookKeeper users are encouraged to upgrade to 4.15.0. The technical details of this release are summarized -below. - -## Breaking Changes - -* `BookieServer` API changed and the code that creates its instances will require addition -of the `UncleanShutdownDetection` parameter. -See [PR 2936](https://github.com/apache/bookkeeper/pull/2936) for details and examples. - -* `Bookie` class now is an interface with implementation in `BookieImpl`. -Code that uses it may need changes. -For details please refer to [PR 2717](https://github.com/apache/bookkeeper/pull/2717). - -* `LedgerUnderreplicationManager` interface added a new method. -Code that implements the interface will need changes. -See [PR 2805](https://github.com/apache/bookkeeper/pull/2805) for details. - -* `MetadataBookieDriver` interface added a new method and removed an old one. -`RegistrationManager` interface added a new method. -`ByteBufAllocatorWithOomHandler` interface is added and used instead of -the `ByteBufAllocator` in multiple places. -Code that implements the interfaces will need changes. -See [PR 2901](https://github.com/apache/bookkeeper/pull/2901) for details. - -## Highlights - -### Configuration - -* [ledgerMetadataVersion](https://github.com/apache/bookkeeper/pull/2708): - BookKeeper-Client config to write ledger metadata with configured version. -* [clientTcpUserTimeoutMillis](https://github.com/apache/bookkeeper/pull/2761): - Added TCP_USER_TIMEOUT to Epoll channel config. -* [auditorMaxNumberOfConcurrentOpenLedgerOperations and auditorAcquireConcurrentOpenLedgerOperationsTimeOutMSec](https://github.com/apache/bookkeeper/pull/2802) - Add auditor get ledger throttle to avoid auto recovery zk session. -* [dbStorage_rocksDB_format_version](https://github.com/apache/bookkeeper/pull/2824) - make rocksdb format version configurable. - -### Features - -* Running without journal. See [BP-46](https://github.com/apache/bookkeeper/pull/2706) for details. -* A REST API to get or update bookie readOnly state. [Details](https://github.com/apache/bookkeeper/pull/2799) -* Separate config files for Rocks DB. [Details](https://github.com/apache/bookkeeper/pull/3056/) - -### Improvements - -* Build and tests work on JDK 17 -* CLI: listunderreplicated command has an option to return count without printing all ledgers https://github.com/apache/bookkeeper/pull/3228 -* Stream Storage: support an optional time to live (TTL) on a per table basis https://github.com/apache/bookkeeper/pull/2775 -* Added dDb ledger index rebuild operation and CLI commands https://github.com/apache/bookkeeper/pull/2774 -* Support multi ledger directories for rocksdb backend entryMetadataMap https://github.com/apache/bookkeeper/pull/2965 -* Improved memory utilization - * support shrink for ConcurrentLong map or set https://github.com/apache/bookkeeper/pull/3074 - * reduce unnecessary expansions for ConcurrentLong map and set https://github.com/apache/bookkeeper/pull/3072 -* read speed rate limiter for scanning entry log file in entryMetadataMap rebuild https://github.com/apache/bookkeeper/pull/2963 -* Other improvements in areas such as test, documentation, CI, metrics, logging, and CLI tools. - -### Notable fixes - -* Bookkeeper client might not close the channel for several minutes after a Bookie crashes https://github.com/apache/bookkeeper/issues/2482 -* Stream storage: Ensure progress while restoring from checkpoint. https://github.com/apache/bookkeeper/pull/2764 -* Entry Log GC may get blocked when using entryLogPerLedgerEnabled option https://github.com/apache/bookkeeper/pull/2779 -* Fix region aware placement policy use disk weight https://github.com/apache/bookkeeper/pull/2981 -* Some cases that could cause RocksDB segfault -* DistributedLogManager can skip over a segment on read. https://github.com/apache/bookkeeper/pull/3064 -* Backpressure: check all bookies of writeset are writable https://github.com/apache/bookkeeper/pull/3055 -* Fix Journal.ForceWriteThread.forceWriteRequests.put deadlock https://github.com/apache/bookkeeper/pull/2962 -* PendingReadOp: Fix ledgerEntryImpl reuse problem https://github.com/apache/bookkeeper/pull/3110 -* Region/rack aware placement policy: replace bookie bug https://github.com/apache/bookkeeper/pull/2642 -* ReplicationWorker: numLedgersReplicated metric does not update https://github.com/apache/bookkeeper/pull/3218 -* Force GC doesn't work under forceAllowCompaction when disk is full https://github.com/apache/bookkeeper/pull/3205 - -### Dependencies changes - -Upgraded dependencies to address CVEs include: -* vertx -* freebuilder -* libthrift -* netty -* bouncycastle -* commonsIO -* jetty -* log4j -* grpc -* protobuf -* snakeyaml -* RocksDB -* jackson -* jackson-databind -* Zookeeper -* http-core -* dropwizard metrics - -Dependency on log4j v.1 is removed. - -## Details - -https://github.com/apache/bookkeeper/issues?q=+label%3Arelease%2F4.15.0 - - diff --git a/site/docs/4.15.0/overview/releaseNotesTemplate.md b/site/docs/4.15.0/overview/releaseNotesTemplate.md deleted file mode 100644 index 16a9b3cf014..00000000000 --- a/site/docs/4.15.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.15.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.15.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.15.0/reference/cli.md b/site/docs/4.15.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.15.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.15.0/reference/config.md b/site/docs/4.15.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.15.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.15.0/reference/metrics.md b/site/docs/4.15.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.15.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.15.0/security/overview.md b/site/docs/4.15.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.15.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.15.0/security/sasl.md b/site/docs/4.15.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.15.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.15.0/security/tls.md b/site/docs/4.15.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.15.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.15.0/security/zookeeper.md b/site/docs/4.15.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.15.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.5.0/admin/autorecovery.md b/site/docs/4.5.0/admin/autorecovery.md deleted file mode 100644 index bd11a8886d3..00000000000 --- a/site/docs/4.5.0/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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.0/admin/bookies.md b/site/docs/4.5.0/admin/bookies.md deleted file mode 100644 index f9b1dcf94dc..00000000000 --- a/site/docs/4.5.0/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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.0/admin/geo-replication.md b/site/docs/4.5.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.5.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.0/admin/metrics.md b/site/docs/4.5.0/admin/metrics.md deleted file mode 100644 index fbcec41a773..00000000000 --- a/site/docs/4.5.0/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.5.0/admin/perf.md b/site/docs/4.5.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.5.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.5.0/admin/placement.md b/site/docs/4.5.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.5.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.5.0/admin/upgrade.md b/site/docs/4.5.0/admin/upgrade.md deleted file mode 100644 index 456df99a276..00000000000 --- a/site/docs/4.5.0/admin/upgrade.md +++ /dev/null @@ -1,73 +0,0 @@ ---- -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.0/api/distributedlog-api.md b/site/docs/4.5.0/api/distributedlog-api.md deleted file mode 100644 index a13aa8b5868..00000000000 --- a/site/docs/4.5.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.5.0/api/ledger-adv-api.md b/site/docs/4.5.0/api/ledger-adv-api.md deleted file mode 100644 index f46950dd984..00000000000 --- a/site/docs/4.5.0/api/ledger-adv-api.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.0/api/ledger-api.md b/site/docs/4.5.0/api/ledger-api.md deleted file mode 100644 index 84357272953..00000000000 --- a/site/docs/4.5.0/api/ledger-api.md +++ /dev/null @@ -1,472 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.5.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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.0/api/overview.md b/site/docs/4.5.0/api/overview.md deleted file mode 100644 index 3eb649273c1..00000000000 --- a/site/docs/4.5.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.0/deployment/dcos.md b/site/docs/4.5.0/deployment/dcos.md deleted file mode 100644 index 3e174384ec1..00000000000 --- a/site/docs/4.5.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.0/deployment/kubernetes.md b/site/docs/4.5.0/deployment/kubernetes.md deleted file mode 100644 index f65172112a7..00000000000 --- a/site/docs/4.5.0/deployment/kubernetes.md +++ /dev/null @@ -1,4 +0,0 @@ ---- -title: Deploying BookKeeper on Kubernetes -logo: img/kubernetes-logo.png ---- diff --git a/site/docs/4.5.0/deployment/manual.md b/site/docs/4.5.0/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.5.0/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.5.0/development/codebase.md b/site/docs/4.5.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.5.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.5.0/development/protocol.md b/site/docs/4.5.0/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.5.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.0/getting-started/concepts.md b/site/docs/4.5.0/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.5.0/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.0/getting-started/installation.md b/site/docs/4.5.0/getting-started/installation.md deleted file mode 100644 index fac16ddd390..00000000000 --- a/site/docs/4.5.0/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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.0/getting-started/run-locally.md b/site/docs/4.5.0/getting-started/run-locally.md deleted file mode 100644 index ab33642c852..00000000000 --- a/site/docs/4.5.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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.0/overview/overview.md b/site/docs/4.5.0/overview/overview.md deleted file mode 100644 index 11d74e93f45..00000000000 --- a/site/docs/4.5.0/overview/overview.md +++ /dev/null @@ -1,59 +0,0 @@ ---- -title: Apache BookKeeper™ 4.5.0 ---- -<!-- -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. ---> - -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. - -BookKeeper is well suited for scenarios like this: - -Scenario | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (Write-Ahead-Logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -Message storage | [Apache Pulsar](https://pulsar.incubator.apache.org/) -Offset/cursor storage | Apache Pulsar -Object/BLOG storage | 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.0/overview/releaseNotes.md b/site/docs/4.5.0/overview/releaseNotes.md deleted file mode 100644 index f1d3e3364aa..00000000000 --- a/site/docs/4.5.0/overview/releaseNotes.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: Apache BookKeeper 4.5.0 Release Notes ---- - -This is the fifth release of BookKeeper as an Apache Top Level Project! - -The 4.5.0 release incorporates hundreds of new fixes, improvements, and features since previous major release, 4.4.0, -which was released over a year ago. It is a big milestone in Apache BookKeeper community, converging from three -main branches (Salesforce, Twitter and Yahoo). - -Apache BookKeeper users are encouraged to upgrade to 4.5.0. The technical details of this release are summarized -below. - -## Highlights - -The main features in 4.5.0 cover are around following areas: - -- Dependencies Upgrade -- Security -- Public API -- Performance -- Operations - -### 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://bookkeeper.apache.org/distributedlog/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<String, byte[]>` 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 -<ul> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-552'>BOOKKEEPER-552</a>] - 64 Bits Ledger ID Generation -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-553'>BOOKKEEPER-553</a>] - New LedgerManager for 64 Bits Ledger ID Management in ZooKeeper -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-588'>BOOKKEEPER-588</a>] - SSL support -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-873'>BOOKKEEPER-873</a>] - Enhance CreatedLedger API to accept ledgerId as input -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-949'>BOOKKEEPER-949</a>] - Allow entryLog creation even when bookie is in RO mode for compaction -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-965'>BOOKKEEPER-965</a>] - Long Poll: Changes to the Write Path -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-997'>BOOKKEEPER-997</a>] - Wire protocol change for supporting long poll -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1017'>BOOKKEEPER-1017</a>] - Create documentation for ZooKeeper ACLs -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1086'>BOOKKEEPER-1086</a>] - Ledger Recovery - Refactor PendingReadOp -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1087'>BOOKKEEPER-1087</a>] - Ledger Recovery - Add a parallel reading request in PendingReadOp -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1088'>BOOKKEEPER-1088</a>] - Ledger Recovery - Add a ReadEntryListener to callback on individual request -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1089'>BOOKKEEPER-1089</a>] - Ledger Recovery - allow batch reads in ledger recovery -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1092'>BOOKKEEPER-1092</a>] - Ledger Recovery - Add Test Case for Parallel Ledger Recovery -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1093'>BOOKKEEPER-1093</a>] - Piggyback LAC on ReadResponse -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1094'>BOOKKEEPER-1094</a>] - Long Poll - Server and Client Side Changes -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1095'>BOOKKEEPER-1095</a>] - Long Poll - Client side changes -</li> -</ul> - -#### Bug -<ul> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-852'>BOOKKEEPER-852</a>] - Release LedgerDescriptor and master-key objects when not used anymore -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-903'>BOOKKEEPER-903</a>] - MetaFormat BookieShell Command is not deleting UnderReplicatedLedgers list from the ZooKeeper -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-907'>BOOKKEEPER-907</a>] - for ReadLedgerEntriesCmd, EntryFormatter should be configurable and HexDumpEntryFormatter should be one of them -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-908'>BOOKKEEPER-908</a>] - Case to handle BKLedgerExistException -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-924'>BOOKKEEPER-924</a>] - addEntry() is susceptible to spurious wakeups -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-927'>BOOKKEEPER-927</a>] - Extend BOOKKEEPER-886 to LedgerHandleAdv too (BOOKKEEPER-886: Allow to disable ledgers operation throttling) -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-933'>BOOKKEEPER-933</a>] - ClientConfiguration always inherits System properties -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-938'>BOOKKEEPER-938</a>] - LedgerOpenOp should use digestType from metadata -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-939'>BOOKKEEPER-939</a>] - Fix typo in bk-merge-pr.py -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-940'>BOOKKEEPER-940</a>] - Fix findbugs warnings after bumping to java 8 -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-952'>BOOKKEEPER-952</a>] - Fix RegionAwarePlacementPolicy -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-955'>BOOKKEEPER-955</a>] - in BookKeeperAdmin listLedgers method currentRange variable is not getting updated to next iterator when it has run out of elements -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-956'>BOOKKEEPER-956</a>] - HierarchicalLedgerManager doesn't work for ledgerid of length 9 and 10 because of order issue in HierarchicalLedgerRangeIterator -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-958'>BOOKKEEPER-958</a>] - ZeroBuffer readOnlyBuffer returns ByteBuffer with 0 remaining bytes for length > 64k -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-959'>BOOKKEEPER-959</a>] - ClientAuthProvider and BookieAuthProvider Public API used Protobuf Shaded classes -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-976'>BOOKKEEPER-976</a>] - Fix license headers with "Copyright 2016 The Apache Software Foundation" -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-980'>BOOKKEEPER-980</a>] - BookKeeper Tools doesn't process the argument correctly -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-981'>BOOKKEEPER-981</a>] - NullPointerException in RackawareEnsemblePlacementPolicy while running in Docker Container -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-984'>BOOKKEEPER-984</a>] - BookieClientTest.testWriteGaps tested -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-986'>BOOKKEEPER-986</a>] - Handle Memtable flush failure -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-987'>BOOKKEEPER-987</a>] - BookKeeper build is broken due to the shade plugin for commit ecbb053e6e -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-988'>BOOKKEEPER-988</a>] - Missing license headers -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-989'>BOOKKEEPER-989</a>] - Enable travis CI for bookkeeper git -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-999'>BOOKKEEPER-999</a>] - BookKeeper client can leak threads -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1013'>BOOKKEEPER-1013</a>] - Fix findbugs errors on latest master -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1018'>BOOKKEEPER-1018</a>] - Allow client to select older V2 protocol (no protobuf) -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1020'>BOOKKEEPER-1020</a>] - Fix Explicit LAC tests on master -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1021'>BOOKKEEPER-1021</a>] - Improve the merge script to handle github reviews api -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1031'>BOOKKEEPER-1031</a>] - ReplicationWorker.rereplicate fails to call close() on ReadOnlyLedgerHandle -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1044'>BOOKKEEPER-1044</a>] - Entrylogger is not readding rolled logs back to the logChannelsToFlush list when exception happens while trying to flush rolled logs -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1047'>BOOKKEEPER-1047</a>] - Add missing error code in ZK setData return path -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1058'>BOOKKEEPER-1058</a>] - Ignore already deleted ledger on replication audit -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1061'>BOOKKEEPER-1061</a>] - BookieWatcher should not do ZK blocking operations from ZK async callback thread -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1065'>BOOKKEEPER-1065</a>] - OrderedSafeExecutor should only have 1 thread per bucket -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1071'>BOOKKEEPER-1071</a>] - BookieRecoveryTest is failing due to a Netty4 IllegalReferenceCountException -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1072'>BOOKKEEPER-1072</a>] - CompactionTest is flaky when disks are almost full -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1073'>BOOKKEEPER-1073</a>] - Several stats provider related changes. -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1074'>BOOKKEEPER-1074</a>] - Remove JMX Bean -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1075'>BOOKKEEPER-1075</a>] - BK LedgerMetadata: more memory-efficient parsing of configs -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1076'>BOOKKEEPER-1076</a>] - BookieShell should be able to read the 'FENCE' entry in the log -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1077'>BOOKKEEPER-1077</a>] - BookKeeper: Local Bookie Journal and ledger paths -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1079'>BOOKKEEPER-1079</a>] - shell lastMark throws NPE -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1098'>BOOKKEEPER-1098</a>] - ZkUnderreplicationManager can build up an unbounded number of watchers -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1101'>BOOKKEEPER-1101</a>] - BookKeeper website menus not working under https -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1102'>BOOKKEEPER-1102</a>] - org.apache.bookkeeper.client.BookKeeperDiskSpaceWeightedLedgerPlacementTest.testDiskSpaceWeightedBookieSelectionWithBookiesBeingAdded is unreliable -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1103'>BOOKKEEPER-1103</a>] - LedgerMetadataCreateTest bug in ledger id generation causes intermittent hang -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1104'>BOOKKEEPER-1104</a>] - BookieInitializationTest.testWithDiskFullAndAbilityToCreateNewIndexFile testcase is unreliable -</li> -</ul> - -#### Improvement -<ul> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-612'>BOOKKEEPER-612</a>] - RegionAwarePlacement Policy -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-748'>BOOKKEEPER-748</a>] - Move fence requests out of read threads -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-757'>BOOKKEEPER-757</a>] - Ledger Recovery Improvement -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-759'>BOOKKEEPER-759</a>] - bookkeeper: delay ensemble change if it doesn't break ack quorum requirement -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-772'>BOOKKEEPER-772</a>] - Reorder read sequnce -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-874'>BOOKKEEPER-874</a>] - Explict LAC from Writer to Bookies -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-881'>BOOKKEEPER-881</a>] - upgrade surefire plugin to 2.19 -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-887'>BOOKKEEPER-887</a>] - Allow to use multiple bookie journals -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-922'>BOOKKEEPER-922</a>] - Create a generic (K,V) map to store ledger metadata -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-935'>BOOKKEEPER-935</a>] - Publish sources and javadocs to Maven Central -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-937'>BOOKKEEPER-937</a>] - Upgrade protobuf to 2.6 -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-944'>BOOKKEEPER-944</a>] - Multiple issues and improvements to BK Compaction. -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-945'>BOOKKEEPER-945</a>] - Add counters to track the activity of auditor and replication workers -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-946'>BOOKKEEPER-946</a>] - Provide an option to delay auto recovery of lost bookies -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-961'>BOOKKEEPER-961</a>] - Assing read/write request for same ledger to a single thread -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-962'>BOOKKEEPER-962</a>] - Add more journal timing stats -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-963'>BOOKKEEPER-963</a>] - Allow to use multiple journals in bookie -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-964'>BOOKKEEPER-964</a>] - Add concurrent maps and sets for primitive types -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-966'>BOOKKEEPER-966</a>] - change the bookieServer cmdline to make conf-file and option co-exist -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-968'>BOOKKEEPER-968</a>] - Entry log flushes happen on log rotation and cause long spikes in IO utilization -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-970'>BOOKKEEPER-970</a>] - Bump zookeeper version to 3.5 -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-971'>BOOKKEEPER-971</a>] - update bk codahale stats provider version -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-998'>BOOKKEEPER-998</a>] - Increased the max entry size to 5MB -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1001'>BOOKKEEPER-1001</a>] - Make LocalBookiesRegistry.isLocalBookie() public -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1002'>BOOKKEEPER-1002</a>] - BookieRecoveryTest can run out of file descriptors -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1003'>BOOKKEEPER-1003</a>] - Fix TestDiskChecker so it can be used on /dev/shm -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1004'>BOOKKEEPER-1004</a>] - Allow bookie garbage collection to be triggered manually from tests -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1007'>BOOKKEEPER-1007</a>] - Explicit LAC: make the interval configurable in milliseconds instead of seconds -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1008'>BOOKKEEPER-1008</a>] - Move to netty4 -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1010'>BOOKKEEPER-1010</a>] - Bump up Guava version to 20.0 -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1022'>BOOKKEEPER-1022</a>] - Make BookKeeperAdmin implement AutoCloseable -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1039'>BOOKKEEPER-1039</a>] - bk-merge-pr.py ask to run findbugs and rat before merge -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1046'>BOOKKEEPER-1046</a>] - Avoid long to Long conversion in OrderedSafeExecutor task submit -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1048'>BOOKKEEPER-1048</a>] - Use ByteBuf in LedgerStorageInterface -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1050'>BOOKKEEPER-1050</a>] - Cache journalFormatVersionToWrite when starting Journal -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1051'>BOOKKEEPER-1051</a>] - Fast shutdown for GarbageCollectorThread -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1052'>BOOKKEEPER-1052</a>] - Print autorecovery enabled or not in bookie shell -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1053'>BOOKKEEPER-1053</a>] - Upgrade RAT maven version to 0.12 and ignore Eclipse project files -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1055'>BOOKKEEPER-1055</a>] - Optimize handling of masterKey in case it is empty -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1056'>BOOKKEEPER-1056</a>] - Removed PacketHeader serialization/deserialization allocation -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1063'>BOOKKEEPER-1063</a>] - Use executure.execute() instead of submit() to avoid creation of unused FutureTask -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1066'>BOOKKEEPER-1066</a>] - Introduce GrowableArrayBlockingQueue -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1068'>BOOKKEEPER-1068</a>] - Expose ByteBuf in LedgerEntry to avoid data copy -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1069'>BOOKKEEPER-1069</a>] - If client uses V2 proto, set the connection to always decode V2 messages -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1083'>BOOKKEEPER-1083</a>] - Improvements on OrderedSafeExecutor -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1084'>BOOKKEEPER-1084</a>] - Make variables finale if necessary -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1085'>BOOKKEEPER-1085</a>] - Introduce the AlertStatsLogger -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1090'>BOOKKEEPER-1090</a>] - Use LOG.isDebugEnabled() to avoid unexpected allocations -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1096'>BOOKKEEPER-1096</a>] - When ledger is deleted, along with leaf node all the eligible branch nodes also should be deleted in ZooKeeper. -</li> -</ul> - -#### New Feature -<ul> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-390'>BOOKKEEPER-390</a>] - Provide support for ZooKeeper authentication -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-391'>BOOKKEEPER-391</a>] - Support Kerberos authentication of bookkeeper -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-575'>BOOKKEEPER-575</a>] - Bookie SSL support -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-670'>BOOKKEEPER-670</a>] - Longpoll Read & Piggyback Support -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-912'>BOOKKEEPER-912</a>] - Allow EnsemblePlacementPolicy to choose bookies using ledger custom data (multitenancy support) -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-928'>BOOKKEEPER-928</a>] - Add custom client supplied metadata field to LedgerMetadata -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-930'>BOOKKEEPER-930</a>] - Option to disable Bookie networking -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-941'>BOOKKEEPER-941</a>] - Introduce Feature Switches For controlling client and server behavior -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-948'>BOOKKEEPER-948</a>] - Provide an option to add more ledger/index directories to a bookie -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-950'>BOOKKEEPER-950</a>] - Ledger placement policy to accomodate different storage capacity of bookies -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-969'>BOOKKEEPER-969</a>] - Security Support -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-983'>BOOKKEEPER-983</a>] - BookieShell Command for LedgerDelete -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-991'>BOOKKEEPER-991</a>] - bk shell - Get a list of all on disk files -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-992'>BOOKKEEPER-992</a>] - ReadLog Command Enhancement -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1019'>BOOKKEEPER-1019</a>] - Support for reading entries after LAC (causal consistency driven by out-of-band communications) -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1034'>BOOKKEEPER-1034</a>] - When all disks are full, start Bookie in RO mode if RO mode is enabled -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1067'>BOOKKEEPER-1067</a>] - Add Prometheus stats provider -</li> -</ul> - -#### Story -<ul> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-932'>BOOKKEEPER-932</a>] - Move to JDK 8 -</li> -</ul> - -#### Task -<ul> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-931'>BOOKKEEPER-931</a>] - Update the committers list on website -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-996'>BOOKKEEPER-996</a>] - Apache Rat Check Failures -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1012'>BOOKKEEPER-1012</a>] - Shade and relocate Guava -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1027'>BOOKKEEPER-1027</a>] - Cleanup main README and main website page -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1038'>BOOKKEEPER-1038</a>] - Fix findbugs warnings and upgrade to 3.0.4 -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1043'>BOOKKEEPER-1043</a>] - Upgrade Apache Parent Pom Reference to latest version -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1054'>BOOKKEEPER-1054</a>] - Add gitignore file -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1059'>BOOKKEEPER-1059</a>] - Upgrade to SLF4J-1.7.25 -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1060'>BOOKKEEPER-1060</a>] - Add utility to use SafeRunnable from Java8 Lambda -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1070'>BOOKKEEPER-1070</a>] - bk-merge-pr.py use apache-rat:check goal instead of rat:rat -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1091'>BOOKKEEPER-1091</a>] - Remove Hedwig from BookKeeper website page -</li> -</ul> - -#### Test -<ul> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-967'>BOOKKEEPER-967</a>] - Create new testsuite for testing RackAwareEnsemblePlacementPolicy using ScriptBasedMapping. -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1045'>BOOKKEEPER-1045</a>] - Execute tests in different JVM processes -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1064'>BOOKKEEPER-1064</a>] - ConcurrentModificationException in AuditorLedgerCheckerTest -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1078'>BOOKKEEPER-1078</a>] - Local BookKeeper enhancements for testability -</li> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-1097'>BOOKKEEPER-1097</a>] - GC test when no WritableDirs -</li> -</ul> - -#### Wish -<ul> -<li>[<a href='https://issues.apache.org/jira/browse/BOOKKEEPER-943'>BOOKKEEPER-943</a>] - Reduce log level of AbstractZkLedgerManager for register/unregister ReadOnlyLedgerHandle -</li> -</ul> - -### Github - -- [https://github.com/apache/bookkeeper/milestone/1](https://github.com/apache/bookkeeper/milestone/1) diff --git a/site/docs/4.5.0/overview/releaseNotesTemplate.md b/site/docs/4.5.0/overview/releaseNotesTemplate.md deleted file mode 100644 index c7845ae51bf..00000000000 --- a/site/docs/4.5.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.0/reference/cli.md b/site/docs/4.5.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.5.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.0/reference/config.md b/site/docs/4.5.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.5.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.0/reference/metrics.md b/site/docs/4.5.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.5.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.5.0/security/overview.md b/site/docs/4.5.0/security/overview.md deleted file mode 100644 index 62da8edae76..00000000000 --- a/site/docs/4.5.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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.0/security/sasl.md b/site/docs/4.5.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.5.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.0/security/tls.md b/site/docs/4.5.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.5.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.0/security/zookeeper.md b/site/docs/4.5.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.5.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.5.1/admin/autorecovery.md b/site/docs/4.5.1/admin/autorecovery.md deleted file mode 100644 index bd11a8886d3..00000000000 --- a/site/docs/4.5.1/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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 deleted file mode 100644 index f9b1dcf94dc..00000000000 --- a/site/docs/4.5.1/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.5.1/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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 deleted file mode 100644 index fbcec41a773..00000000000 --- a/site/docs/4.5.1/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.5.1/admin/perf.md b/site/docs/4.5.1/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.5.1/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.5.1/admin/placement.md b/site/docs/4.5.1/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.5.1/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.5.1/admin/upgrade.md b/site/docs/4.5.1/admin/upgrade.md deleted file mode 100644 index 456df99a276..00000000000 --- a/site/docs/4.5.1/admin/upgrade.md +++ /dev/null @@ -1,73 +0,0 @@ ---- -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 deleted file mode 100644 index 9064a5b94e3..00000000000 --- a/site/docs/4.5.1/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.5.1' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.5.1/api/ledger-adv-api.md b/site/docs/4.5.1/api/ledger-adv-api.md deleted file mode 100644 index f46950dd984..00000000000 --- a/site/docs/4.5.1/api/ledger-adv-api.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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 deleted file mode 100644 index b1bf7a375f2..00000000000 --- a/site/docs/4.5.1/api/ledger-api.md +++ /dev/null @@ -1,472 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.5.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 deleted file mode 100644 index 3eb649273c1..00000000000 --- a/site/docs/4.5.1/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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 deleted file mode 100644 index 3f77695b412..00000000000 --- a/site/docs/4.5.1/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.1] -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 deleted file mode 100644 index f65172112a7..00000000000 --- a/site/docs/4.5.1/deployment/kubernetes.md +++ /dev/null @@ -1,4 +0,0 @@ ---- -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 deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.5.1/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.5.1/development/codebase.md b/site/docs/4.5.1/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.5.1/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.5.1/development/protocol.md b/site/docs/4.5.1/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.5.1/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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/getting-started/concepts.md b/site/docs/4.5.1/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.5.1/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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 deleted file mode 100644 index fac16ddd390..00000000000 --- a/site/docs/4.5.1/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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 deleted file mode 100644 index ab33642c852..00000000000 --- a/site/docs/4.5.1/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 deleted file mode 100644 index 9bef23255bd..00000000000 --- a/site/docs/4.5.1/overview/overview.md +++ /dev/null @@ -1,59 +0,0 @@ ---- -title: Apache BookKeeper™ 4.5.1 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version `4.5.1`. - -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. - -BookKeeper is well suited for scenarios like this: - -Scenario | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (Write-Ahead-Logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -Message storage | [Apache Pulsar](https://pulsar.incubator.apache.org/) -Offset/cursor storage | Apache Pulsar -Object/BLOG storage | Storing snapshots to replicated state machines. - -Learn more about Apache BookKeeper and what it can do for your organization: - -- [Apache BookKeeper 4.5.1 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 deleted file mode 100644 index 3b55869b3c0..00000000000 --- a/site/docs/4.5.1/overview/releaseNotes.md +++ /dev/null @@ -1,28 +0,0 @@ ---- -title: Apache BookKeeper 4.5.1 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 reported 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 - -- Fix critical bug on Parallel Recovery, see [https://github.com/apache/bookkeeper/issues/343](https://github.com/apache/bookkeeper/issues/343) - -- Fix critical bug on Prometheus stats provider, see [https://github.com/apache/bookkeeper/pull/535](https://github.com/apache/bookkeeper/pull/535) - -- Fix critical bug ledger length for LedgerHandleAdv, see [https://github.com/apache/bookkeeper/issues/683](https://github.com/apache/bookkeeper/issues/683) - -- Fix critical bug on RackAwarePolicy, see [https://github.com/apache/bookkeeper/issues/551](https://github.com/apache/bookkeeper/issues/551) - -### Dependencies Upgrade - -There is no dependency upgrade since 4.5.0. - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93&q=label%3Arelease%2F4.5.1%20](https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93&q=label%3Arelease%2F4.5.1%20) diff --git a/site/docs/4.5.1/overview/releaseNotesTemplate.md b/site/docs/4.5.1/overview/releaseNotesTemplate.md deleted file mode 100644 index ea00419aa05..00000000000 --- a/site/docs/4.5.1/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper x.x.x 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 deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.5.1/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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 deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.5.1/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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 deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.5.1/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.5.1/security/overview.md b/site/docs/4.5.1/security/overview.md deleted file mode 100644 index 62da8edae76..00000000000 --- a/site/docs/4.5.1/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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 deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.5.1/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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 deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.5.1/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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 deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.5.1/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.6.0/admin/autorecovery.md b/site/docs/4.6.0/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.6.0/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.6.0/admin/bookies.md b/site/docs/4.6.0/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.6.0/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.6.0/admin/geo-replication.md b/site/docs/4.6.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.6.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.6.0/admin/http.md b/site/docs/4.6.0/admin/http.md deleted file mode 100644 index f30bfe9c6f9..00000000000 --- a/site/docs/4.6.0/admin/http.md +++ /dev/null @@ -1,394 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.6.0/admin/metrics.md b/site/docs/4.6.0/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.6.0/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.6.0/admin/perf.md b/site/docs/4.6.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.6.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.6.0/admin/placement.md b/site/docs/4.6.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.6.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.6.0/admin/upgrade.md b/site/docs/4.6.0/admin/upgrade.md deleted file mode 100644 index 1aa84a4ab40..00000000000 --- a/site/docs/4.6.0/admin/upgrade.md +++ /dev/null @@ -1,76 +0,0 @@ ---- -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.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.6.0/api/distributedlog-api.md b/site/docs/4.6.0/api/distributedlog-api.md deleted file mode 100644 index cbd8b4b9888..00000000000 --- a/site/docs/4.6.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.6.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.6.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.6.0/api/ledger-adv-api.md b/site/docs/4.6.0/api/ledger-adv-api.md deleted file mode 100644 index f46950dd984..00000000000 --- a/site/docs/4.6.0/api/ledger-adv-api.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.6.0/api/ledger-api.md b/site/docs/4.6.0/api/ledger-api.md deleted file mode 100644 index 3a33e4bfb1d..00000000000 --- a/site/docs/4.6.0/api/ledger-api.md +++ /dev/null @@ -1,746 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.6.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.6.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.6.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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` diff --git a/site/docs/4.6.0/api/overview.md b/site/docs/4.6.0/api/overview.md deleted file mode 100644 index 3eb649273c1..00000000000 --- a/site/docs/4.6.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.6.0/deployment/dcos.md b/site/docs/4.6.0/deployment/dcos.md deleted file mode 100644 index ee3c956fd8c..00000000000 --- a/site/docs/4.6.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.6.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.6.0/deployment/kubernetes.md b/site/docs/4.6.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.6.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.6.0/deployment/manual.md b/site/docs/4.6.0/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.6.0/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.6.0/development/codebase.md b/site/docs/4.6.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.6.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.6.0/development/protocol.md b/site/docs/4.6.0/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.6.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.6.0/getting-started/concepts.md b/site/docs/4.6.0/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.6.0/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.6.0/getting-started/installation.md b/site/docs/4.6.0/getting-started/installation.md deleted file mode 100644 index fac16ddd390..00000000000 --- a/site/docs/4.6.0/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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.6.0/getting-started/run-locally.md b/site/docs/4.6.0/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.6.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.6.0/overview/overview.md b/site/docs/4.6.0/overview/overview.md deleted file mode 100644 index 3e01abbccad..00000000000 --- a/site/docs/4.6.0/overview/overview.md +++ /dev/null @@ -1,57 +0,0 @@ ---- -title: Apache BookKeeper™ 4.6.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version `4.6.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.6.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.6.0/overview/releaseNotes.md b/site/docs/4.6.0/overview/releaseNotes.md deleted file mode 100644 index 3bd0116fcdc..00000000000 --- a/site/docs/4.6.0/overview/releaseNotes.md +++ /dev/null @@ -1,66 +0,0 @@ ---- -title: Apache BookKeeper 4.6.0 Release Notes ---- - -This is the seventh release of BookKeeper as an Apache Top Level Project! - -The 4.6.0 release incorporates new fixes, improvements, and features since previous major release 4.5.0. - -Apache BookKeeper users are encouraged to upgrade to 4.6.0. The technical details of this release are summarized -below. - -## Highlights - -The main features in 4.6.0 cover are around following areas: -- Dependencies Upgrade -- Bookie enhancement -- BookKeeper Admin REST API -- New BookKeeper API -- Performance improvement -- Deployment or Ease of use - -### Dependencies Upgrade - -- Upgrade Protobuf to `3.4`. - -### Bookie enhancement - -- Persistable bookie status. - - Prior to this release, bookie status was transient. It is a bit hard for management tooling. This feature adds persistable bookies status. See [Issue-265](https://github.com/apache/bookkeeper/issues/265) for more details. - -- Introduce Bookie Discovery Interface. Prior to this release, bookkeeper client only provides interfaces for ledger metadata management. It doesn't provide any interface for service discovery part. This feature introduces bookie discovery interface, so it allows plugging in different service discovery backends for bookkeeper. - - Introduce Bookie Registration Manager for bookie server, see [Issue-662](https://github.com/apache/bookkeeper/issues/662) for more details. - - Introduce registration client for bookkeeper client, see [Issue-666](https://github.com/apache/bookkeeper/issues/666) for more details. - -- Lifecycle components for managing components in bookie server. - - Introduce lifecycle component for each service component, which includes "stats provider", "auto recovery", "http endpoint", and "bookie server(both storage and netty server)", to run these components in a clear way. See [Issue-508](https://github.com/apache/bookkeeper/issues/508) and [Issue-547](https://github.com/apache/bookkeeper/issues/547) for more details. - -- Make bookie recovery work with recovering multiple bookies. - - Make recovery tool work with multiple bookies, so that one call could recover multiple bookies. See [Issue-612](https://github.com/apache/bookkeeper/issues/612) for more details. - -### BookKeeper Admin REST API - -- Introduce a bookkeeper admin endpoint for operations to interact and administer the bookkeeper cluster using REST API. see [PR-278](https://github.com/apache/bookkeeper/pull/278), [Issue-520](https://github.com/apache/bookkeeper/issues/520), and [Issue-674](https://github.com/apache/bookkeeper/issues/674) for more details. - -### New BookKeeper API - -- New Fluent Style API. - - A brand new API to manage ledgers using the Builder pattern, and new interfaces to make it clear operations on ledgers, like WriteHandle and ReadHandle, are provided in this release. See [Issue-506](https://github.com/apache/bookkeeper/issues/506), [Issue-673](https://github.com/apache/bookkeeper/issues/673) and [Issue-550](https://github.com/apache/bookkeeper/issues/550) for more details - -### Performance improvement -- Use ByteBuf in multiple places to avoid unnecessary memory allocation and reduce the garbage produced in JVM. See [PR-640](https://github.com/apache/bookkeeper/pull/640) for more details. - -- Separate the FileInfo cache into write and read cache. It avoids catchup reads impact tailing reads and writes. See [PR-513](https://github.com/apache/bookkeeper/pull/513) for more details. - -### Deployment or Ease of use -- Deployment BookKeeper on K8s. - - Provide yaml files to run BookKeeper on Kubernetes using both StatefulSets and DaemonSet. See [Issue-337](https://github.com/apache/bookkeeper/issues/337) and [Issue-681](https://github.com/apache/bookkeeper/issues/681)for more details. - -## Existing API changes - -- BookKeeper constructor now throws BKException instead of KeeperException. -- The signatures of `reorderReadSequence` and `reorderReadLACSequence` are changed in EnsemblePlacementPolicy. - -## Full list of changes - -- [https://github.com/apache/bookkeeper/milestone/2?closed=1](https://github.com/apache/bookkeeper/milestone/2?closed=1) \ No newline at end of file diff --git a/site/docs/4.6.0/overview/releaseNotesTemplate.md b/site/docs/4.6.0/overview/releaseNotesTemplate.md deleted file mode 100644 index 2df2589eb15..00000000000 --- a/site/docs/4.6.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.6.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.6.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.6.0/reference/cli.md b/site/docs/4.6.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.6.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.6.0/reference/config.md b/site/docs/4.6.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.6.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.6.0/reference/metrics.md b/site/docs/4.6.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.6.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.6.0/security/overview.md b/site/docs/4.6.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.6.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.6.0/security/sasl.md b/site/docs/4.6.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.6.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.6.0/security/tls.md b/site/docs/4.6.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.6.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.6.0/security/zookeeper.md b/site/docs/4.6.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.6.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.6.1/admin/autorecovery.md b/site/docs/4.6.1/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.6.1/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.6.1/admin/bookies.md b/site/docs/4.6.1/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.6.1/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.6.1/admin/geo-replication.md b/site/docs/4.6.1/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.6.1/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.6.1/admin/http.md b/site/docs/4.6.1/admin/http.md deleted file mode 100644 index f30bfe9c6f9..00000000000 --- a/site/docs/4.6.1/admin/http.md +++ /dev/null @@ -1,394 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.6.1/admin/metrics.md b/site/docs/4.6.1/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.6.1/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.6.1/admin/perf.md b/site/docs/4.6.1/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.6.1/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.6.1/admin/placement.md b/site/docs/4.6.1/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.6.1/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.6.1/admin/upgrade.md b/site/docs/4.6.1/admin/upgrade.md deleted file mode 100644 index 1aa84a4ab40..00000000000 --- a/site/docs/4.6.1/admin/upgrade.md +++ /dev/null @@ -1,76 +0,0 @@ ---- -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.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.6.1/api/distributedlog-api.md b/site/docs/4.6.1/api/distributedlog-api.md deleted file mode 100644 index 9676fa9f04e..00000000000 --- a/site/docs/4.6.1/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.6.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.6.1' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.6.1/api/ledger-adv-api.md b/site/docs/4.6.1/api/ledger-adv-api.md deleted file mode 100644 index f46950dd984..00000000000 --- a/site/docs/4.6.1/api/ledger-adv-api.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.6.1/api/ledger-api.md b/site/docs/4.6.1/api/ledger-api.md deleted file mode 100644 index 737230cf464..00000000000 --- a/site/docs/4.6.1/api/ledger-api.md +++ /dev/null @@ -1,809 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.6.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -You can use the shaded artifact of `bookkeeper-server`. Please note that [maven-shade-plugin](https://maven.apache.org/plugins/maven-shade-plugin) doesn't generate -a dependency-reduced pom file for shaded artifact using [shadedArtifactAttached](https://maven.apache.org/plugins/maven-shade-plugin/examples/attached-artifact.html). You need to manually to exclude relocated packages when using the shaded artifact. Full example of how to use this is -showed as below. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.6.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> - <classifier>shaded</classifier> <!-- specify "shaded" classifier to use shaded artifact --> - <exclusions> - <exclusion> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-common</artifactId> - </exclusion> - <exclusion> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-proto</artifactId> - </exclusion> - </exclusions> -</dependency> -``` - -Or you can use a separate shaded artifact `bookkeeper-server-shaded`. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.6.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.6.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.6.1' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the shaded artifact of `bookkeeper-server` jar -dependencies { - compile ('org.apache.bookkeeper:bookkeeper-server:{{ site.latest-version }}:shaded') { - exclude group: 'org.apache.bookkeeper', module: "bookkeeper-common' - exclude group: 'org.apache.bookkeeper', module: 'bookkeeper-proto' - } -} - - -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` diff --git a/site/docs/4.6.1/api/overview.md b/site/docs/4.6.1/api/overview.md deleted file mode 100644 index 3eb649273c1..00000000000 --- a/site/docs/4.6.1/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.6.1/deployment/dcos.md b/site/docs/4.6.1/deployment/dcos.md deleted file mode 100644 index 4060519772d..00000000000 --- a/site/docs/4.6.1/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.6.1] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.6.1/deployment/kubernetes.md b/site/docs/4.6.1/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.6.1/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.6.1/deployment/manual.md b/site/docs/4.6.1/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.6.1/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.6.1/development/codebase.md b/site/docs/4.6.1/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.6.1/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.6.1/development/protocol.md b/site/docs/4.6.1/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.6.1/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.6.1/getting-started/concepts.md b/site/docs/4.6.1/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.6.1/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.6.1/getting-started/installation.md b/site/docs/4.6.1/getting-started/installation.md deleted file mode 100644 index fac16ddd390..00000000000 --- a/site/docs/4.6.1/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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.6.1/getting-started/run-locally.md b/site/docs/4.6.1/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.6.1/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.6.1/overview/overview.md b/site/docs/4.6.1/overview/overview.md deleted file mode 100644 index 5adc9be3016..00000000000 --- a/site/docs/4.6.1/overview/overview.md +++ /dev/null @@ -1,57 +0,0 @@ ---- -title: Apache BookKeeper™ 4.6.1 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version `4.6.1`. - -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.6.1 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.6.1/overview/releaseNotes.md b/site/docs/4.6.1/overview/releaseNotes.md deleted file mode 100644 index 7d7303606bf..00000000000 --- a/site/docs/4.6.1/overview/releaseNotes.md +++ /dev/null @@ -1,32 +0,0 @@ ---- -title: Apache BookKeeper 4.6.1 Release Notes ---- - -This is the eighth release of BookKeeper as an Apache Top Level Project! - -The 4.6.1 release is a bugfix release which fixes a bunch of issues reported from users of 4.6.0. - -Apache BookKeeper users are encouraged to upgrade to 4.6.1. The technical details of this release are summarized -below. - -## Highlights - -- Fix critical bug on index persistence manager, see [https://github.com/apache/bookkeeper/pull/913](https://github.com/apache/bookkeeper/pull/913) - -- Fix critical bug to allow using versions of Netty newer than 4.1.2 on classpath, see [https://github.com/apache/bookkeeper/pull/996](https://github.com/apache/bookkeeper/pull/996) - -- Enhance Java 9 compatibility, see [https://github.com/apache/bookkeeper/issues/326](https://github.com/apache/bookkeeper/issues/326) - -- New option to track task execution time, see [https://github.com/apache/bookkeeper/issues/931](https://github.com/apache/bookkeeper/issues/931) - -- Distribute a version of BookKeeper which embeds and relocates Guava and Protobuf, see [https://github.com/apache/bookkeeper/issues/922](https://github.com/apache/bookkeeper/issues/922) - -- Add description for the new error code "Too many requests", see [https://github.com/apache/bookkeeper/pull/921](https://github.com/apache/bookkeeper/pull/921) - -### Dependencies Upgrade - -There is no dependency upgrade since 4.6.0, but now we distribute a 'shaded' version of main artifacts, see [Ledger API](../ledger-api) - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93&q=label%3Arelease%2F4.6.1+is%3Aclosed](https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93&q=label%3Arelease%2F4.6.1+is%3Aclosed) diff --git a/site/docs/4.6.1/overview/releaseNotesTemplate.md b/site/docs/4.6.1/overview/releaseNotesTemplate.md deleted file mode 100644 index 53848e37a02..00000000000 --- a/site/docs/4.6.1/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.6.1 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.6.1. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.6.1/reference/cli.md b/site/docs/4.6.1/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.6.1/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.6.1/reference/config.md b/site/docs/4.6.1/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.6.1/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.6.1/reference/metrics.md b/site/docs/4.6.1/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.6.1/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.6.1/security/overview.md b/site/docs/4.6.1/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.6.1/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.6.1/security/sasl.md b/site/docs/4.6.1/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.6.1/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.6.1/security/tls.md b/site/docs/4.6.1/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.6.1/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.6.1/security/zookeeper.md b/site/docs/4.6.1/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.6.1/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.6.2/admin/autorecovery.md b/site/docs/4.6.2/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.6.2/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.6.2/admin/bookies.md b/site/docs/4.6.2/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.6.2/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.6.2/admin/geo-replication.md b/site/docs/4.6.2/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.6.2/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.6.2/admin/http.md b/site/docs/4.6.2/admin/http.md deleted file mode 100644 index f30bfe9c6f9..00000000000 --- a/site/docs/4.6.2/admin/http.md +++ /dev/null @@ -1,394 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.6.2/admin/metrics.md b/site/docs/4.6.2/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.6.2/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.6.2/admin/perf.md b/site/docs/4.6.2/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.6.2/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.6.2/admin/placement.md b/site/docs/4.6.2/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.6.2/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.6.2/admin/upgrade.md b/site/docs/4.6.2/admin/upgrade.md deleted file mode 100644 index 1aa84a4ab40..00000000000 --- a/site/docs/4.6.2/admin/upgrade.md +++ /dev/null @@ -1,76 +0,0 @@ ---- -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.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.6.2/api/distributedlog-api.md b/site/docs/4.6.2/api/distributedlog-api.md deleted file mode 100644 index 85462452691..00000000000 --- a/site/docs/4.6.2/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.6.2' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.6.2' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.6.2/api/ledger-adv-api.md b/site/docs/4.6.2/api/ledger-adv-api.md deleted file mode 100644 index f46950dd984..00000000000 --- a/site/docs/4.6.2/api/ledger-adv-api.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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.6.2/api/ledger-api.md b/site/docs/4.6.2/api/ledger-api.md deleted file mode 100644 index a80a4e46f71..00000000000 --- a/site/docs/4.6.2/api/ledger-api.md +++ /dev/null @@ -1,809 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.6.2</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -You can use the shaded artifact of `bookkeeper-server`. Please note that [maven-shade-plugin](https://maven.apache.org/plugins/maven-shade-plugin) doesn't generate -a dependency-reduced pom file for shaded artifact using [shadedArtifactAttached](https://maven.apache.org/plugins/maven-shade-plugin/examples/attached-artifact.html). You need to manually to exclude relocated packages when using the shaded artifact. Full example of how to use this is -showed as below. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.6.2</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> - <classifier>shaded</classifier> <!-- specify "shaded" classifier to use shaded artifact --> - <exclusions> - <exclusion> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-common</artifactId> - </exclusion> - <exclusion> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-proto</artifactId> - </exclusion> - </exclusions> -</dependency> -``` - -Or you can use a separate shaded artifact `bookkeeper-server-shaded`. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.6.2</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.6.2' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.6.2' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the shaded artifact of `bookkeeper-server` jar -dependencies { - compile ('org.apache.bookkeeper:bookkeeper-server:{{ site.latest-version }}:shaded') { - exclude group: 'org.apache.bookkeeper', module: "bookkeeper-common' - exclude group: 'org.apache.bookkeeper', module: 'bookkeeper-proto' - } -} - - -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` diff --git a/site/docs/4.6.2/api/overview.md b/site/docs/4.6.2/api/overview.md deleted file mode 100644 index 3eb649273c1..00000000000 --- a/site/docs/4.6.2/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.6.2/deployment/dcos.md b/site/docs/4.6.2/deployment/dcos.md deleted file mode 100644 index 90bf75cd660..00000000000 --- a/site/docs/4.6.2/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.6.2] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.6.2/deployment/kubernetes.md b/site/docs/4.6.2/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.6.2/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.6.2/deployment/manual.md b/site/docs/4.6.2/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.6.2/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.6.2/development/codebase.md b/site/docs/4.6.2/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.6.2/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.6.2/development/protocol.md b/site/docs/4.6.2/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.6.2/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.6.2/getting-started/concepts.md b/site/docs/4.6.2/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.6.2/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.6.2/getting-started/installation.md b/site/docs/4.6.2/getting-started/installation.md deleted file mode 100644 index fac16ddd390..00000000000 --- a/site/docs/4.6.2/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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.6.2/getting-started/run-locally.md b/site/docs/4.6.2/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.6.2/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.6.2/overview/overview.md b/site/docs/4.6.2/overview/overview.md deleted file mode 100644 index 95a0214c1da..00000000000 --- a/site/docs/4.6.2/overview/overview.md +++ /dev/null @@ -1,57 +0,0 @@ ---- -title: Apache BookKeeper™ 4.6.2 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version `4.6.2`. - -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.6.2 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.6.2/overview/releaseNotes.md b/site/docs/4.6.2/overview/releaseNotes.md deleted file mode 100644 index 4f7cb7a5233..00000000000 --- a/site/docs/4.6.2/overview/releaseNotes.md +++ /dev/null @@ -1,30 +0,0 @@ ---- -title: Apache BookKeeper 4.6.2 Release Notes ---- - -This is the ninth release of BookKeeper as an Apache Top Level Project! - -The 4.6.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.6.1. - -Apache BookKeeper users are encouraged to upgrade to 4.6.2. The technical details of this release are summarized -below. - -## Highlights - -- Fix performance regression is using Netty > 4.1.12, see [https://github.com/apache/bookkeeper/pull/1108](https://github.com/apache/bookkeeper/pull/1108) - -- Enhance performances on Prometheus stats provider, see [https://github.com/apache/bookkeeper/pull/1081](https://github.com/apache/bookkeeper/pull/1081) - -- Save memory resources on client by retaining for less time references to data to write, see [https://github.com/apache/bookkeeper/issues/1063](https://github.com/apache/bookkeeper/issues/1063) - -- Fix a problem on Java 9/10 with the 'shaded' artifacts, due to a bug in Maven Shade Plugin, see [https://github.com/apache/bookkeeper/pull/1144](https://github.com/apache/bookkeeper/pull/1144) - -- Fix Journal stats names, see [https://github.com/apache/bookkeeper/pull/1250](https://github.com/apache/bookkeeper/pull/1250) - -### Dependencies Upgrade - -There is no dependency upgrade since 4.6.0, and since 4.6.1 we distribute a 'shaded' version of main artifacts, see [Ledger API](../ledger-api) - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93&q=label%3Arelease%2F4.6.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93&q=label%3Arelease%2F4.6.2+is%3Aclosed) diff --git a/site/docs/4.6.2/overview/releaseNotesTemplate.md b/site/docs/4.6.2/overview/releaseNotesTemplate.md deleted file mode 100644 index 53848e37a02..00000000000 --- a/site/docs/4.6.2/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.6.1 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.6.1. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.6.2/reference/cli.md b/site/docs/4.6.2/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.6.2/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.6.2/reference/config.md b/site/docs/4.6.2/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.6.2/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.6.2/reference/metrics.md b/site/docs/4.6.2/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.6.2/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.6.2/security/overview.md b/site/docs/4.6.2/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.6.2/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.6.2/security/sasl.md b/site/docs/4.6.2/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.6.2/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.6.2/security/tls.md b/site/docs/4.6.2/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.6.2/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.6.2/security/zookeeper.md b/site/docs/4.6.2/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.6.2/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.7.0/admin/autorecovery.md b/site/docs/4.7.0/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.7.0/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.7.0/admin/bookies.md b/site/docs/4.7.0/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.7.0/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.7.0/admin/geo-replication.md b/site/docs/4.7.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.7.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.7.0/admin/http.md b/site/docs/4.7.0/admin/http.md deleted file mode 100644 index f30bfe9c6f9..00000000000 --- a/site/docs/4.7.0/admin/http.md +++ /dev/null @@ -1,394 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.7.0/admin/metrics.md b/site/docs/4.7.0/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.7.0/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.7.0/admin/perf.md b/site/docs/4.7.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.7.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.7.0/admin/placement.md b/site/docs/4.7.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.7.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.7.0/admin/upgrade.md b/site/docs/4.7.0/admin/upgrade.md deleted file mode 100644 index 89cd3c7dfc1..00000000000 --- a/site/docs/4.7.0/admin/upgrade.md +++ /dev/null @@ -1,171 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.7.0/api/distributedlog-api.md b/site/docs/4.7.0/api/distributedlog-api.md deleted file mode 100644 index db6acd07a5b..00000000000 --- a/site/docs/4.7.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.7.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.7.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.7.0/api/ledger-adv-api.md b/site/docs/4.7.0/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.7.0/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.7.0/api/ledger-api.md b/site/docs/4.7.0/api/ledger-api.md deleted file mode 100644 index d409876e157..00000000000 --- a/site/docs/4.7.0/api/ledger-api.md +++ /dev/null @@ -1,801 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.7.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.7.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.7.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.7.0' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for -guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` diff --git a/site/docs/4.7.0/api/overview.md b/site/docs/4.7.0/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.7.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.7.0/deployment/dcos.md b/site/docs/4.7.0/deployment/dcos.md deleted file mode 100644 index b7dd65946a2..00000000000 --- a/site/docs/4.7.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.7.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.7.0/deployment/kubernetes.md b/site/docs/4.7.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.7.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.7.0/deployment/manual.md b/site/docs/4.7.0/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.7.0/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.7.0/development/codebase.md b/site/docs/4.7.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.7.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.7.0/development/protocol.md b/site/docs/4.7.0/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.7.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.7.0/getting-started/concepts.md b/site/docs/4.7.0/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.7.0/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.7.0/getting-started/installation.md b/site/docs/4.7.0/getting-started/installation.md deleted file mode 100644 index 9986cd8e043..00000000000 --- a/site/docs/4.7.0/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.7.0/getting-started/run-locally.md b/site/docs/4.7.0/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.7.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.7.0/overview/overview.md b/site/docs/4.7.0/overview/overview.md deleted file mode 100644 index 610057b76c1..00000000000 --- a/site/docs/4.7.0/overview/overview.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -title: Apache BookKeeper™ 4.7.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.7.0. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.7.0 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.7.0/overview/releaseNotes.md b/site/docs/4.7.0/overview/releaseNotes.md deleted file mode 100644 index efefe3e93e6..00000000000 --- a/site/docs/4.7.0/overview/releaseNotes.md +++ /dev/null @@ -1,107 +0,0 @@ ---- -title: Apache BookKeeper 4.7.0 Release Notes ---- - -This is the tenth release of Apache BookKeeper! - -The 4.7.0 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.6.0, -which was released four months ago. It is a big milestone in Apache BookKeeper community - Yahoo branch is fully merged -back to upstream, and Apache Pulsar (incubating) starts using official BookKeeper release for its upcoming 2.0 release. - -It is also the first release of Apache DistributedLog after it is merged as sub modules of Apache BookKeeper. - -Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.7.0](../../admin/upgrade). The technical details of -this release are summarized below. - -## Highlights - -The main features in 4.7.0 cover are around following areas: - -- Dependencies Changes -- Public API -- Security -- DbLedgerStorage -- Metadata API -- Performance -- Operations -- Builds & Testing -- Bug Fixes - -### Dependencies Changes - -Here is a list of dependencies changed in 4.7.0: - -- [JCommander](http://jcommander.org/) 1.48 is added as a dependency of bookkeeper-server module. -- [RocksDB](http://rocksdb.org/) 5.8.6 is introduced as part of `DbLedgerStorage` as a dependency of bookkeeper-server module. -- [DataSketches](https://datasketches.github.io/) 0.8.3 is introduced as a dependency of prometheus-metrics-provider module. -- Upgrade [Guava](https://github.com/google/guava) from `20.0` to `21.0`. - -### Public API - -There are multiple new client features introduced in 4.7.0. Here are two highlighted features: - -#### Fluent API - -The new fluent style APi is evolving in 4.7.0. All the methods in handlers are now having both async and sync methods. -See [#1288](https://github.com/apache/bookkeeper/pull/1288) for more details - -#### CRC32C - -`circe-checksum` module is ported from Apache Pulsar to Apache BookKeeper, and CRC32C digest type is added as one digest type option. -The JNI based CRC32C in `circe-checksum` module provides excellent performance than existing CRC32 digest type. Users are encouraged -to start use CRC32C digest type. - -### Security - -- New PEM format `X.509` certificates are introduced for TLS authentication. See [#965](https://github.com/apache/bookkeeper/pull/965) for more details. -- TLS related settings are converged into same settings as bookie server. See [Upgrade Guide](../../admin/upgrade) for more details. - -### DbLedgerStorage - -`DbLedgerStorage` is a new ledger storage that introduced by Yahoo and now fully merged into Apache BookKeeper. It is fully compatible for both v2 and v3 -protocols and also support long polling. It uses [RocksDB](http://rocksdb.org/) to store ledger index, which eliminates the needed of ledger index files and -reduces the number of open file descriptors and the amount of random IOs can occurs during flushing ledger index. - -### Metadata API - -New serviceUri based metadata API is introduced as [BP-29](http://bookkeeper.apache.org/bps/BP-29-metadata-store-api-module). This metadata API provides the metadata -abstraction over ledger manager, registration service, allowing plugin different type of data stores as the metadata service. - -### Performance - -There are a lot for performance related bug fixes and improvements in 4.7.0. Some of the changes are highlighted as below: - -- Leverage netty object recycler to reduce object allocations -- A bunch of contentions around locking are removed. E.g. [#1321](https://github.com/apache/bookkeeper/pull/1321) [#1292](https://github.com/apache/bookkeeper/pull/1292) [#1258](https://github.com/apache/bookkeeper/pull/1258) -- Introduce priority thread pool for accepting high priority reads/writes. This allows high priority reads/writes such as ledger recovery operations can - succeed even bookies are overwhelmed. [#898](https://github.com/apache/bookkeeper/pull/898) -- Reorder slow bookies in read sequence. [#883](https://github.com/apache/bookkeeper/pull/883) -- Use atomic field updater and long adder to replace AtomicInteger/AtomicLong/AtomicReference in Dlog. [#1299](https://github.com/apache/bookkeeper/pull/1299) -- DataSketches library is used for implementing prometheus provider. [#1245](https://github.com/apache/bookkeeper/pull/1245) - -### Operations - -### BookieShell - -There are are multiple new commands are added in BookieShell. Here are a few highlighted: - -- `metaformat` is deprecated with two new commands `initnewcluster` and `nukeexistingcluster`. This separation provides better operability and reduces mistakes. -- `initbookie` command is introduced for initializing a new bookie. `bookieformat` keeps serving as the purpose of reformatting a bookie. - -A new BookKeeper CLI is proposed in [BP-27](http://bookkeeper.apache.org/bps/BP-27-new-bookkeeper-cli). Some commands are already ported to new bookkeeper CLI. -The full list of shell commands will be fully ported to new bookkeeper CLI in next release. - -### ReadOnly Mode Support - -Operations are improved around readonly mode for handling bookkeeper outage situation. New settings are introduce allow entry log creation, high priority writes -even when bookies are readonly. See [Upgrade Guide](../../admin/upgrade) to learn all newly added settings. - - -### Builds & Testing - -- [Arquillian](http://arquillian.org/) framework is introduced in 4.7.0 for backward compatibility and integration tests. -- Both Java8 and Java9 are now supported for running bookkeeper. - -## Full list of changes - -- [https://github.com/apache/bookkeeper/milestone/3](https://github.com/apache/bookkeeper/milestone/3?closed=1) diff --git a/site/docs/4.7.0/overview/releaseNotesTemplate.md b/site/docs/4.7.0/overview/releaseNotesTemplate.md deleted file mode 100644 index 1330c0355c4..00000000000 --- a/site/docs/4.7.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.7.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.7.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.7.0/reference/cli.md b/site/docs/4.7.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.7.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.7.0/reference/config.md b/site/docs/4.7.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.7.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.7.0/reference/metrics.md b/site/docs/4.7.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.7.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.7.0/security/overview.md b/site/docs/4.7.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.7.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.7.0/security/sasl.md b/site/docs/4.7.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.7.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.7.0/security/tls.md b/site/docs/4.7.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.7.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.7.0/security/zookeeper.md b/site/docs/4.7.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.7.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.7.1/admin/autorecovery.md b/site/docs/4.7.1/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.7.1/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.7.1/admin/bookies.md b/site/docs/4.7.1/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.7.1/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.7.1/admin/geo-replication.md b/site/docs/4.7.1/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.7.1/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.7.1/admin/http.md b/site/docs/4.7.1/admin/http.md deleted file mode 100644 index f30bfe9c6f9..00000000000 --- a/site/docs/4.7.1/admin/http.md +++ /dev/null @@ -1,394 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.7.1/admin/metrics.md b/site/docs/4.7.1/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.7.1/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.7.1/admin/perf.md b/site/docs/4.7.1/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.7.1/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.7.1/admin/placement.md b/site/docs/4.7.1/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.7.1/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.7.1/admin/upgrade.md b/site/docs/4.7.1/admin/upgrade.md deleted file mode 100644 index 1188282b400..00000000000 --- a/site/docs/4.7.1/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.7.0 to 4.7.1 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.1. So you can follow the general upgrade sequence to upgrade from 4.7.0 to 4.7.1. - -### 4.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.7.1/api/distributedlog-api.md b/site/docs/4.7.1/api/distributedlog-api.md deleted file mode 100644 index 22a12ec5603..00000000000 --- a/site/docs/4.7.1/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.7.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.7.1' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.7.1/api/ledger-adv-api.md b/site/docs/4.7.1/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.7.1/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.7.1/api/ledger-api.md b/site/docs/4.7.1/api/ledger-api.md deleted file mode 100644 index c9c12c68152..00000000000 --- a/site/docs/4.7.1/api/ledger-api.md +++ /dev/null @@ -1,801 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.7.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.7.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.7.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.7.1' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for -guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` diff --git a/site/docs/4.7.1/api/overview.md b/site/docs/4.7.1/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.7.1/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.7.1/deployment/dcos.md b/site/docs/4.7.1/deployment/dcos.md deleted file mode 100644 index a1e4996d6f6..00000000000 --- a/site/docs/4.7.1/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.7.1] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.7.1/deployment/kubernetes.md b/site/docs/4.7.1/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.7.1/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.7.1/deployment/manual.md b/site/docs/4.7.1/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.7.1/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.7.1/development/codebase.md b/site/docs/4.7.1/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.7.1/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.7.1/development/protocol.md b/site/docs/4.7.1/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.7.1/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.7.1/getting-started/concepts.md b/site/docs/4.7.1/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.7.1/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.7.1/getting-started/installation.md b/site/docs/4.7.1/getting-started/installation.md deleted file mode 100644 index 9986cd8e043..00000000000 --- a/site/docs/4.7.1/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.7.1/getting-started/run-locally.md b/site/docs/4.7.1/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.7.1/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.7.1/overview/overview.md b/site/docs/4.7.1/overview/overview.md deleted file mode 100644 index 6c81716eac0..00000000000 --- a/site/docs/4.7.1/overview/overview.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -title: Apache BookKeeper™ 4.7.1 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.7.1. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.7.1 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.7.1/overview/releaseNotes.md b/site/docs/4.7.1/overview/releaseNotes.md deleted file mode 100644 index 5c2b2110a99..00000000000 --- a/site/docs/4.7.1/overview/releaseNotes.md +++ /dev/null @@ -1,44 +0,0 @@ ---- -title: Apache BookKeeper 4.7.1 Release Notes ---- - -This is the eleventh release of Apache BookKeeper! - -The 4.7.1 release is a bugfix release which fixes a bunch of issues reported from users of 4.7.0. - -Apache BookKeeper users who are using 4.7.0 are encouraged to upgrade to 4.7.1. The technical details of this release are summarized -below. - -## Highlights - -- Performance enhancement on eliminating bytes copying in `AddEntry` code path, see [apache/bookkeeper#1361](https://github.com/apache/bookkeeper/pull/1361) - -- Introduce Fast and Garbage-Free Statistics Timers in Codahale Stats Provider, see [apache/bookkeeper#1364](https://github.com/apache/bookkeeper/pull/1364) - -- Fix OrderedScheduler handling null key, see [apache/bookkeeper#1372](https://github.com/apache/bookkeeper/pull/1372) - -- Fix zookeeper ledger manager on handling no ledger exists, see [apache/bookkeeper#1382](https://github.com/apache/bookkeeper/pull/1382) - -- Fix long poll reads when ensemble size is larger than write quorum size, see [apache/bookkeeper#1404](https://github.com/apache/bookkeeper/pull/1404) - -- Fix IllegalReferenceCount on filling readahead cache for DbLedgerStorage, see [apache/bookkeeper#1487](https://github.com/apache/bookkeeper/issues/1487) - -- Fix LedgerEntry recycling issue on long poll speculative reads, see [apache/bookkeeper#1509](https://github.com/apache/bookkeeper/pull/1509) - -- Various bug fixes and improvements around bookkeeper table service, see changes under [apache/bookkeeper#release/4.7.1](https://github.com/apache/bookkeeper/issues?utf8=%E2%9C%93&q=is%3Aclosed+label%3Aarea%2Ftableservice+label%3Arelease%2F4.7.1) - -### Dependencies Upgrade - -Here is a list of dependencies changed in 4.7.1: - -- [Grpc](https://grpc.io/) is upgraded from `1.5.0` to `1.12.0`. See [apache/bookkeeper#1441](https://github.com/apache/bookkeeper/pull/1441) -- [Netty](http://netty.io/) is upgraded from `4.1.12` to `4.1.22`. See [apache/bookkeeper#1441](https://github.com/apache/bookkeeper/pull/1441) -- [Protobuf](https://developers.google.com/protocol-buffers/) is upgraded from `3.4.0` to `3.5.1`. See [apache/bookkeeper#1466](https://github.com/apache/bookkeeper/pull/1466) -- [RocksDB](http://rocksdb.org/) is upgraded from `5.8.6` to `5.13.1`. See [apache/bookkeeper#1466](https://github.com/apache/bookkeeper/pull/1466) - -`Reflective setAccessible(true)` is disabled by default in Netty while using java9+. This might result in performance degradation. Consider reenabling `Reflective setAccessible(true)` by setting -environment value `io.netty.tryReflectionSetAccessible` to `true`. See [netty/netty#7650](https://github.com/netty/netty/pull/7650) for more details. - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.1+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.1+is%3Aclosed) diff --git a/site/docs/4.7.1/overview/releaseNotesTemplate.md b/site/docs/4.7.1/overview/releaseNotesTemplate.md deleted file mode 100644 index 6fecd045c21..00000000000 --- a/site/docs/4.7.1/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.7.1 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.7.1. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.7.1/reference/cli.md b/site/docs/4.7.1/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.7.1/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.7.1/reference/config.md b/site/docs/4.7.1/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.7.1/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.7.1/reference/metrics.md b/site/docs/4.7.1/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.7.1/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.7.1/security/overview.md b/site/docs/4.7.1/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.7.1/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.7.1/security/sasl.md b/site/docs/4.7.1/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.7.1/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.7.1/security/tls.md b/site/docs/4.7.1/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.7.1/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.7.1/security/zookeeper.md b/site/docs/4.7.1/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.7.1/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.7.2/admin/autorecovery.md b/site/docs/4.7.2/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.7.2/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.7.2/admin/bookies.md b/site/docs/4.7.2/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.7.2/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.7.2/admin/geo-replication.md b/site/docs/4.7.2/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.7.2/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.7.2/admin/http.md b/site/docs/4.7.2/admin/http.md deleted file mode 100644 index f30bfe9c6f9..00000000000 --- a/site/docs/4.7.2/admin/http.md +++ /dev/null @@ -1,394 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.7.2/admin/metrics.md b/site/docs/4.7.2/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.7.2/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.7.2/admin/perf.md b/site/docs/4.7.2/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.7.2/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.7.2/admin/placement.md b/site/docs/4.7.2/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.7.2/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.7.2/admin/upgrade.md b/site/docs/4.7.2/admin/upgrade.md deleted file mode 100644 index ddbaf0eda11..00000000000 --- a/site/docs/4.7.2/admin/upgrade.md +++ /dev/null @@ -1,179 +0,0 @@ ---- -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.7.1 to 4.7.2 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.2. So you can follow the general upgrade sequence to upgrade from 4.7.1 to 4.7.2. - -### 4.7.0 to 4.7.1 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.1. So you can follow the general upgrade sequence to upgrade from 4.7.0 to 4.7.1. - -### 4.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.7.2/api/distributedlog-api.md b/site/docs/4.7.2/api/distributedlog-api.md deleted file mode 100644 index 786d7a95a1e..00000000000 --- a/site/docs/4.7.2/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.7.2' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.7.2' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.7.2/api/ledger-adv-api.md b/site/docs/4.7.2/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.7.2/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.7.2/api/ledger-api.md b/site/docs/4.7.2/api/ledger-api.md deleted file mode 100644 index 015e1804ea7..00000000000 --- a/site/docs/4.7.2/api/ledger-api.md +++ /dev/null @@ -1,801 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.7.2</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.7.2</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.7.2' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.7.2' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for -guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` diff --git a/site/docs/4.7.2/api/overview.md b/site/docs/4.7.2/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.7.2/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.7.2/deployment/dcos.md b/site/docs/4.7.2/deployment/dcos.md deleted file mode 100644 index ed7cdf59ae3..00000000000 --- a/site/docs/4.7.2/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.7.2] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.7.2/deployment/kubernetes.md b/site/docs/4.7.2/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.7.2/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.7.2/deployment/manual.md b/site/docs/4.7.2/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.7.2/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.7.2/development/codebase.md b/site/docs/4.7.2/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.7.2/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.7.2/development/protocol.md b/site/docs/4.7.2/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.7.2/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.7.2/getting-started/concepts.md b/site/docs/4.7.2/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.7.2/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.7.2/getting-started/installation.md b/site/docs/4.7.2/getting-started/installation.md deleted file mode 100644 index 9986cd8e043..00000000000 --- a/site/docs/4.7.2/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.7.2/getting-started/run-locally.md b/site/docs/4.7.2/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.7.2/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.7.2/overview/overview.md b/site/docs/4.7.2/overview/overview.md deleted file mode 100644 index 0c30a6e51b3..00000000000 --- a/site/docs/4.7.2/overview/overview.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -title: Apache BookKeeper™ 4.7.2 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.7.2. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.7.2 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.7.2/overview/releaseNotes.md b/site/docs/4.7.2/overview/releaseNotes.md deleted file mode 100644 index 45b0592bf2e..00000000000 --- a/site/docs/4.7.2/overview/releaseNotes.md +++ /dev/null @@ -1,35 +0,0 @@ ---- -title: Apache BookKeeper 4.7.2 Release Notes ---- - -This is the 12th release of Apache BookKeeper! - -The 4.7.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.7.1. - -Apache BookKeeper users who are using 4.7.1 are encouraged to upgrade to 4.7.2. The technical details of this release are summarized -below. - -## Highlights - -- Fix high cpu usage issue in DbLedgerStorage by avoiding using RocksDD#deleteRange, see [apache/bookkeeper#1620](https://github.com/apache/bookkeeper/pull/1620) - -- Fix deadlock in Auditor blocking zookeeper thread, see [apache/bookkeeper#1619](https://github.com/apache/bookkeeper/pull/1619) - -- Fix ArrayIndexOutOfBoundsException on ConcurrentLongHashMap, see [apache/bookkeeper#1606](https://github.com/apache/bookkeeper/pull/1606) - -- Fix deferred failure handling causes data loss, see [apache/bookkeeper#1591](https://github.com/apache/bookkeeper/pull/1591) - -- Fix ConcurrentModificationException using nonblocking logReader#readNext, see [apache/bookkeeper#1544](https://github.com/apache/bookkeeper/pull/1544) - -- Fix Bookie shutdown fails to exit, see [apache/bookkeeper#1543](https://github.com/apache/bookkeeper/issues/1543) - -- Fix race conditions on accessing guava multimap in PCBC when using v2 protocol, see [apache/bookkeeper#1618](https://github.com/apache/bookkeeper/pull/1618) - -### Dependency Changes - -In 4.7.2, [Zookeeper](https://zookeeper.apache.org/) version is downgraded from `3.5.3-beta` to `3.4.13` to avoid having a `beta` dependency and address maturity concerns. -The downgrade is safe and smooth. No extra actions are required from switching bookkeeper 4.7.1 to 4.7.2. - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.2+is%3Aclosed) diff --git a/site/docs/4.7.2/overview/releaseNotesTemplate.md b/site/docs/4.7.2/overview/releaseNotesTemplate.md deleted file mode 100644 index cd7570d4db6..00000000000 --- a/site/docs/4.7.2/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.7.2 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.7.2. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.7.2/reference/cli.md b/site/docs/4.7.2/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.7.2/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.7.2/reference/config.md b/site/docs/4.7.2/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.7.2/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.7.2/reference/metrics.md b/site/docs/4.7.2/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.7.2/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.7.2/security/overview.md b/site/docs/4.7.2/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.7.2/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.7.2/security/sasl.md b/site/docs/4.7.2/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.7.2/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.7.2/security/tls.md b/site/docs/4.7.2/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.7.2/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.7.2/security/zookeeper.md b/site/docs/4.7.2/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.7.2/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.7.3/admin/autorecovery.md b/site/docs/4.7.3/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.7.3/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.7.3/admin/bookies.md b/site/docs/4.7.3/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.7.3/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.7.3/admin/geo-replication.md b/site/docs/4.7.3/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.7.3/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.7.3/admin/http.md b/site/docs/4.7.3/admin/http.md deleted file mode 100644 index f30bfe9c6f9..00000000000 --- a/site/docs/4.7.3/admin/http.md +++ /dev/null @@ -1,394 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.7.3/admin/metrics.md b/site/docs/4.7.3/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.7.3/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.7.3/admin/perf.md b/site/docs/4.7.3/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.7.3/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.7.3/admin/placement.md b/site/docs/4.7.3/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.7.3/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.7.3/admin/upgrade.md b/site/docs/4.7.3/admin/upgrade.md deleted file mode 100644 index f35b8a43904..00000000000 --- a/site/docs/4.7.3/admin/upgrade.md +++ /dev/null @@ -1,183 +0,0 @@ ---- -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.7.2 to 4.7.3 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.3. So you can follow the general upgrade sequence to upgrade from 4.7.2 to 4.7.3. - -### 4.7.1 to 4.7.2 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.2. So you can follow the general upgrade sequence to upgrade from 4.7.1 to 4.7.2. - -### 4.7.0 to 4.7.1 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.1. So you can follow the general upgrade sequence to upgrade from 4.7.0 to 4.7.1. - -### 4.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.7.3/api/distributedlog-api.md b/site/docs/4.7.3/api/distributedlog-api.md deleted file mode 100644 index 27293028609..00000000000 --- a/site/docs/4.7.3/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.7.3' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.7.3' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.7.3/api/ledger-adv-api.md b/site/docs/4.7.3/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.7.3/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.7.3/api/ledger-api.md b/site/docs/4.7.3/api/ledger-api.md deleted file mode 100644 index a68477bfeef..00000000000 --- a/site/docs/4.7.3/api/ledger-api.md +++ /dev/null @@ -1,801 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.7.3</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.7.3</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.7.3' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.7.3' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for -guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` diff --git a/site/docs/4.7.3/api/overview.md b/site/docs/4.7.3/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.7.3/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.7.3/deployment/dcos.md b/site/docs/4.7.3/deployment/dcos.md deleted file mode 100644 index 69d22c888ab..00000000000 --- a/site/docs/4.7.3/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.7.3] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.7.3/deployment/kubernetes.md b/site/docs/4.7.3/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.7.3/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.7.3/deployment/manual.md b/site/docs/4.7.3/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.7.3/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.7.3/development/codebase.md b/site/docs/4.7.3/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.7.3/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.7.3/development/protocol.md b/site/docs/4.7.3/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.7.3/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.7.3/getting-started/concepts.md b/site/docs/4.7.3/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.7.3/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.7.3/getting-started/installation.md b/site/docs/4.7.3/getting-started/installation.md deleted file mode 100644 index 9986cd8e043..00000000000 --- a/site/docs/4.7.3/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.7.3/getting-started/run-locally.md b/site/docs/4.7.3/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.7.3/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.7.3/overview/overview.md b/site/docs/4.7.3/overview/overview.md deleted file mode 100644 index ccca3a48ece..00000000000 --- a/site/docs/4.7.3/overview/overview.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -title: Apache BookKeeper™ 4.7.3 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.7.3. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.7.3 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.7.3/overview/releaseNotes.md b/site/docs/4.7.3/overview/releaseNotes.md deleted file mode 100644 index 0e574540ed5..00000000000 --- a/site/docs/4.7.3/overview/releaseNotes.md +++ /dev/null @@ -1,34 +0,0 @@ ---- -title: Apache BookKeeper 4.7.3 Release Notes ---- - -This is the 16th release of Apache BookKeeper! - -The 4.7.3 release is a bugfix release which fixes a bunch of issues reported from users of 4.7.2. - -Apache BookKeeper users who are using 4.7.2 are encouraged to upgrade to 4.7.3. The technical details of this release are summarized -below. - -## Highlights - -- Cancel Scheduled SpeculativeReads, see [apache/bookkeeper#1665](https://github.com/apache/bookkeeper/pull/1665) - -- IllegalReferenceCountException at closing EntryLogManagerForSingleEntryLog, see [apache/bookkeeper#1703](https://github.com/apache/bookkeeper/issues/1703) - -- EntryMemTable.newEntry retains reference to passed ByteBuffer array can cause corruption on journal replay, see [apache/bookkeeper#1737](https://github.com/apache/bookkeeper/issues/1737) - -- Ledger deletion racing with flush can cause a ledger index to be resurrected, see [apache/bookkeeper#1757](https://github.com/apache/bookkeeper/issues/1757) - -- Don't cache Bookie hostname DNS resolution forever, see [apache/bookkeeper#1762](https://github.com/apache/bookkeeper/pull/1762) - -- Use default metric registry in Prometheus export, see [apache/bookkeeper#1765](https://github.com/apache/bookkeeper/pull/1765) - -- Fix Auth with v2 protocol, see [apache/bookkeeper#1805](https://github.com/apache/bookkeeper/pull/1805) - -- Remove MathUtils.now to address compaction scheduling deplay issues, see [apache/bookkeeper#1837](https://github.com/apache/bookkeeper/pull/1837) - -- DbLedgerStorage should do periodical flush, see [apache/bookkeeper#1843](https://github.com/apache/bookkeeper/pull/1843) - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.3+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.7.3+is%3Aclosed) diff --git a/site/docs/4.7.3/overview/releaseNotesTemplate.md b/site/docs/4.7.3/overview/releaseNotesTemplate.md deleted file mode 100644 index 9cec1b73542..00000000000 --- a/site/docs/4.7.3/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.7.3 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.7.3. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.7.3/reference/cli.md b/site/docs/4.7.3/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.7.3/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.7.3/reference/config.md b/site/docs/4.7.3/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.7.3/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.7.3/reference/metrics.md b/site/docs/4.7.3/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.7.3/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.7.3/security/overview.md b/site/docs/4.7.3/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.7.3/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.7.3/security/sasl.md b/site/docs/4.7.3/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.7.3/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.7.3/security/tls.md b/site/docs/4.7.3/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.7.3/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.7.3/security/zookeeper.md b/site/docs/4.7.3/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.7.3/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.8.0/admin/autorecovery.md b/site/docs/4.8.0/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.8.0/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.8.0/admin/bookies.md b/site/docs/4.8.0/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.8.0/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.8.0/admin/geo-replication.md b/site/docs/4.8.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.8.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.8.0/admin/http.md b/site/docs/4.8.0/admin/http.md deleted file mode 100644 index 714d0394242..00000000000 --- a/site/docs/4.8.0/admin/http.md +++ /dev/null @@ -1,407 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.8.0/admin/metrics.md b/site/docs/4.8.0/admin/metrics.md deleted file mode 100644 index 179a84e93be..00000000000 --- a/site/docs/4.8.0/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.8.0/admin/perf.md b/site/docs/4.8.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.8.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.8.0/admin/placement.md b/site/docs/4.8.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.8.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.8.0/admin/upgrade.md b/site/docs/4.8.0/admin/upgrade.md deleted file mode 100644 index 690de2dcc9e..00000000000 --- a/site/docs/4.8.0/admin/upgrade.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -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.7.x to 4.8.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.7.x to 4.8.0. - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. -(Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it). -Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. -But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - - -### 4.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.8.0/api/distributedlog-api.md b/site/docs/4.8.0/api/distributedlog-api.md deleted file mode 100644 index 40ac4782cc3..00000000000 --- a/site/docs/4.8.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.8.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.8.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.8.0/api/ledger-adv-api.md b/site/docs/4.8.0/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.8.0/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.8.0/api/ledger-api.md b/site/docs/4.8.0/api/ledger-api.md deleted file mode 100644 index bd08ffb4420..00000000000 --- a/site/docs/4.8.0/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.8.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.8.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.8.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.8.0' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.8.0/api/overview.md b/site/docs/4.8.0/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.8.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.8.0/deployment/dcos.md b/site/docs/4.8.0/deployment/dcos.md deleted file mode 100644 index 9aca61c2e14..00000000000 --- a/site/docs/4.8.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.8.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.8.0/deployment/kubernetes.md b/site/docs/4.8.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.8.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.8.0/deployment/manual.md b/site/docs/4.8.0/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.8.0/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.8.0/development/codebase.md b/site/docs/4.8.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.8.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.8.0/development/protocol.md b/site/docs/4.8.0/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.8.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.8.0/getting-started/concepts.md b/site/docs/4.8.0/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.8.0/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.8.0/getting-started/installation.md b/site/docs/4.8.0/getting-started/installation.md deleted file mode 100644 index 9986cd8e043..00000000000 --- a/site/docs/4.8.0/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.8.0/getting-started/run-locally.md b/site/docs/4.8.0/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.8.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.8.0/overview/overview.md b/site/docs/4.8.0/overview/overview.md deleted file mode 100644 index 80de28b55bf..00000000000 --- a/site/docs/4.8.0/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.8.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.8.0. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.8.0 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.8.0/overview/releaseNotes.md b/site/docs/4.8.0/overview/releaseNotes.md deleted file mode 100644 index 02012cfe804..00000000000 --- a/site/docs/4.8.0/overview/releaseNotes.md +++ /dev/null @@ -1,68 +0,0 @@ ---- -title: Apache BookKeeper 4.8.0 Release Notes ---- - -This is the 13th release of Apache BookKeeper! - -The 4.8.0 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.7.0. -It is a new big milestone in Apache BookKeeper community, -this release include great new features, like Relaxed Durability, Stream Storage service and Multiple active Entrylogs. - -Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.8.0](../../admin/upgrade). The technical details of -this release are summarized below. - -## Highlights - -The main features in 4.8.0 are around following areas: - -- Durability -- ExplicitLAC feature -- New Table Storage Service -- Bug Fixes - - -### New WriteFlag DEFERRED_SYNC - -The writer may ask for temporary relaxed durability writes, that is to receive early acknowledge from Bookies, before an fsync() on Journal. -Together with this new flag we introduced the new WriteHandle#force() API, this this API the writer is able to request an explicit guarantee of durability to the Bookies -it is mostly like and explicit fsync() on a file system. - -See [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) and [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) for reference - -### New behaviour for Netty ByteBuf reference count management - -All the client side APIs which take ByteBufs now will have the net effect of decrementing by 1 the refcount. -This is consistent with general contract of Netty. -It is expected that the client passes the ownership of the ByteBuf to BookKeeper client. - -### Multiple Active Entrylogs - -It is now possible on the Bookie to have multiple active entry loggers, -this new feature will help with compaction performance and some specific workloads. - -See [Multiple active entrylogs](https://github.com/apache/bookkeeper/issues/570) - -### Table Storage Service - -From this version we are providing the a table (key/value) service embedded in Bookies. - -See [BP-30: BookKeeper Table Service](https://github.com/apache/bookkeeper/issues/1205) - -### Make ExplicitLAC persistent - -ExplicitLAC was contributed from Salesforce in 4.5.0 release, but in the first release -it was a beft-effort in-memory mechanism. Now you can configure Bookies to store durably ExplicitLAC. - -See [Make ExplicitLAC persistent](https://github.com/apache/bookkeeper/issues/1527) - -### Ensemble change on Delayed Write Failure - -We are handling more gracefully the case of a failure of a Bookie in spite of a succeeded write. -If you are writing with Ack Quorum = 2 and Write Quorum = 3, writes will succeeed even if 1 of 3 Bookies fail, -now BookKeeper will trigger an *ensemble change* and replace the failed bookie earlier. - -See [Ensemble change on Delayed Write Failure](https://github.com/apache/bookkeeper/issues/1390) - -## Full list of changes - -- [https://github.com/apache/bookkeeper/milestone/4](https://github.com/apache/bookkeeper/milestone/4?closed=1) diff --git a/site/docs/4.8.0/overview/releaseNotesTemplate.md b/site/docs/4.8.0/overview/releaseNotesTemplate.md deleted file mode 100644 index 3ad4d98b1b5..00000000000 --- a/site/docs/4.8.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.8.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.8.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.8.0/reference/cli.md b/site/docs/4.8.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.8.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.8.0/reference/config.md b/site/docs/4.8.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.8.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.8.0/reference/metrics.md b/site/docs/4.8.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.8.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.8.0/security/overview.md b/site/docs/4.8.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.8.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.8.0/security/sasl.md b/site/docs/4.8.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.8.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.8.0/security/tls.md b/site/docs/4.8.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.8.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.8.0/security/zookeeper.md b/site/docs/4.8.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.8.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.8.1/admin/autorecovery.md b/site/docs/4.8.1/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.8.1/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.8.1/admin/bookies.md b/site/docs/4.8.1/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.8.1/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.8.1/admin/geo-replication.md b/site/docs/4.8.1/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.8.1/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.8.1/admin/http.md b/site/docs/4.8.1/admin/http.md deleted file mode 100644 index 714d0394242..00000000000 --- a/site/docs/4.8.1/admin/http.md +++ /dev/null @@ -1,407 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.8.1/admin/metrics.md b/site/docs/4.8.1/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.8.1/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.8.1/admin/perf.md b/site/docs/4.8.1/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.8.1/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.8.1/admin/placement.md b/site/docs/4.8.1/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.8.1/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.8.1/admin/upgrade.md b/site/docs/4.8.1/admin/upgrade.md deleted file mode 100644 index 6825949dd77..00000000000 --- a/site/docs/4.8.1/admin/upgrade.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -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.7.x to 4.8.1 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.7.x to 4.8.1. - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. -(Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it). -Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. -But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - - -### 4.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.8.1/api/distributedlog-api.md b/site/docs/4.8.1/api/distributedlog-api.md deleted file mode 100644 index 8a1ea830422..00000000000 --- a/site/docs/4.8.1/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.8.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.8.1' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.8.1/api/ledger-adv-api.md b/site/docs/4.8.1/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.8.1/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.8.1/api/ledger-api.md b/site/docs/4.8.1/api/ledger-api.md deleted file mode 100644 index 5a18ccd97dd..00000000000 --- a/site/docs/4.8.1/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.8.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.8.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.8.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.8.1' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.8.1/api/overview.md b/site/docs/4.8.1/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.8.1/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.8.1/deployment/dcos.md b/site/docs/4.8.1/deployment/dcos.md deleted file mode 100644 index 113bfda18a0..00000000000 --- a/site/docs/4.8.1/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.8.1] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.8.1/deployment/kubernetes.md b/site/docs/4.8.1/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.8.1/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.8.1/deployment/manual.md b/site/docs/4.8.1/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.8.1/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.8.1/development/codebase.md b/site/docs/4.8.1/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.8.1/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.8.1/development/protocol.md b/site/docs/4.8.1/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.8.1/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.8.1/getting-started/concepts.md b/site/docs/4.8.1/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.8.1/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.8.1/getting-started/installation.md b/site/docs/4.8.1/getting-started/installation.md deleted file mode 100644 index 9986cd8e043..00000000000 --- a/site/docs/4.8.1/getting-started/installation.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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.8.1/getting-started/run-locally.md b/site/docs/4.8.1/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.8.1/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.8.1/overview/overview.md b/site/docs/4.8.1/overview/overview.md deleted file mode 100644 index 1d1f71b30a5..00000000000 --- a/site/docs/4.8.1/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.8.1 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.8.1. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.8.1 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.8.1/overview/releaseNotes.md b/site/docs/4.8.1/overview/releaseNotes.md deleted file mode 100644 index 7434f47c84e..00000000000 --- a/site/docs/4.8.1/overview/releaseNotes.md +++ /dev/null @@ -1,31 +0,0 @@ ---- -title: Apache BookKeeper 4.8.1 Release Notes ---- - -This is the 14th release of Apache BookKeeper! - -The 4.8.1 release is a bugfix release which fixes a bunch of issues reported from users of 4.8.0. - -Apache BookKeeper users who are using 4.8.0 are encouraged to upgrade to 4.8.1. The technical details of this release are summarized -below. - -## Highlights - -- Use default metrics registry in Prometheus exporter, see [apache/bookkeeper#1765](https://github.com/apache/bookkeeper/pull/1765) - -- Don't cache Bookie hostname DNS resolution forever, see [apache/bookkeeper#1762](https://github.com/apache/bookkeeper/pull/1762) - -- Reduce stack traces in logs for common cases, see [apache/bookkeeper#1762](https://github.com/apache/bookkeeper/pull/1776) - -- Ledger deletion racing with flush can cause a ledger index to be resurrected, see [apache/bookkeeper#1757](https://github.com/apache/bookkeeper/pull/1757) - -- EntryMemTable.newEntry retains reference to passed ByteBuffer array, can cause corruption on journal replay, see [apache/bookkeeper#1737](https://github.com/apache/bookkeeper/pull/1737) - - -### Dependency Changes - -There is no dependecy upgrade from 4.8.0. - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.1+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.1+is%3Aclosed) diff --git a/site/docs/4.8.1/overview/releaseNotesTemplate.md b/site/docs/4.8.1/overview/releaseNotesTemplate.md deleted file mode 100644 index d1061738ab2..00000000000 --- a/site/docs/4.8.1/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.8.1 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.8.1. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.8.1/reference/cli.md b/site/docs/4.8.1/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.8.1/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.8.1/reference/config.md b/site/docs/4.8.1/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.8.1/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.8.1/reference/metrics.md b/site/docs/4.8.1/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.8.1/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.8.1/security/overview.md b/site/docs/4.8.1/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.8.1/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.8.1/security/sasl.md b/site/docs/4.8.1/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.8.1/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.8.1/security/tls.md b/site/docs/4.8.1/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.8.1/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.8.1/security/zookeeper.md b/site/docs/4.8.1/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.8.1/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.8.2/admin/autorecovery.md b/site/docs/4.8.2/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.8.2/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.8.2/admin/bookies.md b/site/docs/4.8.2/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.8.2/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.8.2/admin/geo-replication.md b/site/docs/4.8.2/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.8.2/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.8.2/admin/http.md b/site/docs/4.8.2/admin/http.md deleted file mode 100644 index ab5e711a90e..00000000000 --- a/site/docs/4.8.2/admin/http.md +++ /dev/null @@ -1,418 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.8.2/admin/metrics.md b/site/docs/4.8.2/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.8.2/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.8.2/admin/perf.md b/site/docs/4.8.2/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.8.2/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.8.2/admin/placement.md b/site/docs/4.8.2/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.8.2/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.8.2/admin/upgrade.md b/site/docs/4.8.2/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.8.2/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.8.2/api/distributedlog-api.md b/site/docs/4.8.2/api/distributedlog-api.md deleted file mode 100644 index f073b291048..00000000000 --- a/site/docs/4.8.2/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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: '{{ site.latest_version }}' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:{{ site.latest_version }}' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.8.2/api/ledger-adv-api.md b/site/docs/4.8.2/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.8.2/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.8.2/api/ledger-api.md b/site/docs/4.8.2/api/ledger-api.md deleted file mode 100644 index 5365033a0d1..00000000000 --- a/site/docs/4.8.2/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>{{ site.latest_version }}</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>{{ site.latest_version }}</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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: '{{ site.latest_version }}' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:{{ site.latest_version }}' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.8.2/api/overview.md b/site/docs/4.8.2/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.8.2/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.8.2/deployment/dcos.md b/site/docs/4.8.2/deployment/dcos.md deleted file mode 100644 index dab023deb7a..00000000000 --- a/site/docs/4.8.2/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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 [{{ site.latest_version }}] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.8.2/deployment/kubernetes.md b/site/docs/4.8.2/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.8.2/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.8.2/deployment/manual.md b/site/docs/4.8.2/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.8.2/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.8.2/development/codebase.md b/site/docs/4.8.2/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.8.2/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.8.2/development/protocol.md b/site/docs/4.8.2/development/protocol.md deleted file mode 100644 index 2ca604e85b3..00000000000 --- a/site/docs/4.8.2/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.8.2/getting-started/concepts.md b/site/docs/4.8.2/getting-started/concepts.md deleted file mode 100644 index 6f34f18494d..00000000000 --- a/site/docs/4.8.2/getting-started/concepts.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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 responsibility 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 `<path>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.8.2/getting-started/installation.md b/site/docs/4.8.2/getting-started/installation.md deleted file mode 100644 index 7e92b471581..00000000000 --- a/site/docs/4.8.2/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.8.2/getting-started/run-locally.md b/site/docs/4.8.2/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.8.2/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.8.2/overview/overview.md b/site/docs/4.8.2/overview/overview.md deleted file mode 100644 index c3a3fd7a592..00000000000 --- a/site/docs/4.8.2/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.8.2 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version {{ site.latest_version }}. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.8.2 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.8.2/overview/releaseNotes.md b/site/docs/4.8.2/overview/releaseNotes.md deleted file mode 100644 index 6a76e9ac1f8..00000000000 --- a/site/docs/4.8.2/overview/releaseNotes.md +++ /dev/null @@ -1,49 +0,0 @@ ---- -title: Apache BookKeeper 4.8.2 Release Notes ---- - -This is the 17th release of Apache BookKeeper! - -The 4.8.2 release is a bugfix release which fixes a bunch of issues reported from users of 4.8.1. - -Apache BookKeeper users who are using 4.8.1 are encouraged to upgrade to 4.8.2. The technical details of this release are summarized -below. - -## Highlights - -- [DLOG] Avoid double read in readahead, see [apache/bookkeeper#1973](https://github.com/apache/bookkeeper/pull/1973) - -- Small fix wrong nodesUninitialized count when checkCovered, see [apache/bookkeeper#1900](https://github.com/apache/bookkeeper/pull/1900) - -- Handle double bookie failures, see [apache/bookkeeper#1886](https://github.com/apache/bookkeeper/pull/1886) - -- dir_\*_usage stats are reported as 0, see [apache/bookkeeper#1884](https://github.com/apache/bookkeeper/pull/1884) - -- Fix selectFromNetworkLocation in RackawareEnsemblePlacementPolicyImpl, see [apache/bookkeeper#1862](https://github.com/apache/bookkeeper/pull/1862) - -- DbLedgerStorage should do periodical flush, see [apache/bookkeeper#1842](https://github.com/apache/bookkeeper/pull/1842) - -- Add rest endpoint trigger_gc to trigger GC on Bookie, see [apache/bookkeeper#1838](https://github.com/apache/bookkeeper/pull/1838) - -- Fix sorted ledger storage rotating entry log files too frequent, see [apache/bookkeeper#1807](https://github.com/apache/bookkeeper/pull/1807) - -- Fixed Auth with v2 protocol, see [apache/bookkeeper#1805](https://github.com/apache/bookkeeper/pull/1805) - -- [tools] add cookie related commands, see [apache/bookkeeper#1974](https://github.com/apache/bookkeeper/pull/1794) - -- [tools] improve bkctl help message, see [apache/bookkeeper#1793](https://github.com/apache/bookkeeper/pull/1793) - -- Read Submission should bypass OSE Threads, see [apache/bookkeeper#1791](https://github.com/apache/bookkeeper/pull/1791) - -- Cache InetSocketAddress if hostname is IPAddress, see [apache/bookkeeper#1789](https://github.com/apache/bookkeeper/pull/1789) - -- Fix bugs in DefaultEnsemblePlacementPolicy, see [apache/bookkeeper#1788](https://github.com/apache/bookkeeper/pull/1788) - -### Dependency Changes - -There is no dependency upgrade from 4.8.1. - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.8.2+is%3Aclosed) - diff --git a/site/docs/4.8.2/overview/releaseNotesTemplate.md b/site/docs/4.8.2/overview/releaseNotesTemplate.md deleted file mode 100644 index 4dedbb4b486..00000000000 --- a/site/docs/4.8.2/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.8.2 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to {{ site.latest_version }}. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.8.2/reference/cli.md b/site/docs/4.8.2/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.8.2/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.8.2/reference/config.md b/site/docs/4.8.2/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.8.2/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.8.2/reference/metrics.md b/site/docs/4.8.2/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.8.2/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.8.2/security/overview.md b/site/docs/4.8.2/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.8.2/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.8.2/security/sasl.md b/site/docs/4.8.2/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.8.2/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.8.2/security/tls.md b/site/docs/4.8.2/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.8.2/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.8.2/security/zookeeper.md b/site/docs/4.8.2/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.8.2/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.9.0/admin/autorecovery.md b/site/docs/4.9.0/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.9.0/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.9.0/admin/bookies.md b/site/docs/4.9.0/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.9.0/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.9.0/admin/geo-replication.md b/site/docs/4.9.0/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.9.0/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.9.0/admin/http.md b/site/docs/4.9.0/admin/http.md deleted file mode 100644 index ee6411abb53..00000000000 --- a/site/docs/4.9.0/admin/http.md +++ /dev/null @@ -1,457 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.9.0/admin/metrics.md b/site/docs/4.9.0/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.9.0/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.9.0/admin/perf.md b/site/docs/4.9.0/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.9.0/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.9.0/admin/placement.md b/site/docs/4.9.0/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.9.0/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.9.0/admin/upgrade.md b/site/docs/4.9.0/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.9.0/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.9.0/api/distributedlog-api.md b/site/docs/4.9.0/api/distributedlog-api.md deleted file mode 100644 index 5b1ebfb9d62..00000000000 --- a/site/docs/4.9.0/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.9.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.9.0' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.9.0/api/ledger-adv-api.md b/site/docs/4.9.0/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.9.0/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.9.0/api/ledger-api.md b/site/docs/4.9.0/api/ledger-api.md deleted file mode 100644 index 8b15a99e9ac..00000000000 --- a/site/docs/4.9.0/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.9.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.9.0</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.9.0' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.9.0' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.9.0/api/overview.md b/site/docs/4.9.0/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.9.0/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.9.0/deployment/dcos.md b/site/docs/4.9.0/deployment/dcos.md deleted file mode 100644 index 1373b01628f..00000000000 --- a/site/docs/4.9.0/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.9.0] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.9.0/deployment/kubernetes.md b/site/docs/4.9.0/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.9.0/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.9.0/deployment/manual.md b/site/docs/4.9.0/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.9.0/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.9.0/development/codebase.md b/site/docs/4.9.0/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.9.0/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.9.0/development/protocol.md b/site/docs/4.9.0/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/4.9.0/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.9.0/getting-started/concepts.md b/site/docs/4.9.0/getting-started/concepts.md deleted file mode 100644 index d53f9a66cc2..00000000000 --- a/site/docs/4.9.0/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.9.0/getting-started/installation.md b/site/docs/4.9.0/getting-started/installation.md deleted file mode 100644 index 7e92b471581..00000000000 --- a/site/docs/4.9.0/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.9.0/getting-started/run-locally.md b/site/docs/4.9.0/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.9.0/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.9.0/overview/overview.md b/site/docs/4.9.0/overview/overview.md deleted file mode 100644 index dc219e73489..00000000000 --- a/site/docs/4.9.0/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.9.0 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.9.0. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.9.0 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.9.0/overview/releaseNotes.md b/site/docs/4.9.0/overview/releaseNotes.md deleted file mode 100644 index 6dcaef36a51..00000000000 --- a/site/docs/4.9.0/overview/releaseNotes.md +++ /dev/null @@ -1,144 +0,0 @@ ---- -title: Apache BookKeeper 4.9.0 Release Notes ---- - -This is the 16th release of Apache BookKeeper! - -The 4.9.0 release incorporates hundreds of bug fixes, improvements, and features since previous major release, 4.8.0, -which was released four months ago. It is a new milestone in Apache BookKeeper community. - -Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.9.0](../../admin/upgrade). The technical details of -this release are summarized below. - -## Highlights - -The main features in 4.9.0 cover are around following areas: - -- Dependencies Changes -- Public API -- Configuration -- Metadata -- Table Service -- Operations -- Builds & Testing -- Enhancements -- Bug Fixes - -### Dependencies Changes - -Here is a list of dependencies changed in 4.9.0: - -- Upgrade [Jackson](http://fasterxml.com/) from `2.8.9` to `2.9.7`. -- Upgrade [Jline](https://jline.github.io/) to `2.11`. -- Upgrade [Netty](https://netty.io/) from `4.1.22` to `4.1.31`. -- Upgrade [TestContainers](https://www.testcontainers.org/) from `1.7.0` to `1.8.3`. - -### Public API - -There are multiple new client features introduced in 4.9.0. Here are two highlighted features: - -- LedgerHandleAdv exposes `asyncAddEntry` variant that takes ByteBuf - -### Configuration - -There are bunch of new settings introduced in both bookie and client in 4.9.0. Here are those settings: - -#### Bookie - -- `serverNumIOThreads`: configures the number of IO threads for bookies - (see [#1612](https://github.com/apache/bookkeeper/pull/1612)) -- The default value of `fileInfoFormatVersionToWrite` is bumped from `0` to `1`. - (see [#1689](https://github.com/apache/bookkeeper/pull/1689)) -- The default value of `journalFormatVersionToWrite` is bumped from `5` to `6`. - (see [#1689](https://github.com/apache/bookkeeper/pull/1689)) - -#### Client - -- `numIOThreads`: configures the number of IO threads for client - (see [#1612](https://github.com/apache/bookkeeper/pull/1612)) - -### Metadata - -There are a few big changes around metadata in 4.9.0. They are: - -- Refactor ledger metadata in LedgerHandle to make ledger metadata instance immutable (see [#281](https://github.com/apache/bookkeeper/issues/281)) -- Store ledger metadata in binary protobuf format (see details at [#723](https://github.com/apache/bookkeeper/issues/723)) -- Etcd based metadata driver implementation is in BETA release (see details at [#1639](https://github.com/apache/bookkeeper/issues/1639)) - -Additionally, there are bunch of new interfaces introduced in the metadata driver API. - -- [Issue #1619: Provide async version of markLedgerUnderreplicated for LedgerUnderreplicationManager](https://github.com/apache/bookkeeper/pull/1619) - -### Table Service - -There are a lot of improvements and features introduced into the table service. The maturity of table service is moving from alpha to beta, -and has started to be used as the state storage for Pulsar Functions. More table service usage will come in Pulsar's future releases. - -Starting from 4.9.0, bookkeeper will release a python client for table service. See details at [#1691](https://github.com/apache/bookkeeper/pull/1691) - -### Operations - -#### HTTP Admin REST Endpoint - -- `/api/v1/bookie/gc_details` is introduced to retrieve the GC details. -- `/api/v1/bookie/gc` is introduced to trigger GC through HTTP REST endpoint. - -#### BookieShell - -There are are multiple new commands are added in BookieShell. Here are a few highlighted: - -- `regenerate-interleaved-storage-index-file` command is introduced for rebuilding the index files for interleaved based ledger storage. ([#1642](https://github.com/apache/bookkeeper/pull/1642)) -- `ledgermetadata` command now supports dumping/restoring ledger metadata to/from file. -- `localconsistencycheck` command is introduce for running consistency check on bookies locally. ([#1819](https://github.com/apache/bookkeeper/pull/1819)) -- a new `bk-perf` script is introduced for running performance benchmark on bookkeeper. ([1697](https://github.com/apache/bookkeeper/pull/1697)) - -A new BookKeeper CLI package is released as `bkctl`. This `bkctl` package includes both the existing bookie shell and the new `bkctl` tool. - -#### MDC - -Mapped Diagnostic Context (MDC) is now supported at both bookie and client sides. Application request context can be passed as context -and being logged through slf4j/log4j. This simplifies throubleshooting of request-level failures/errors. See details at [#1672](https://github.com/apache/bookkeeper/pull/1672). - -#### Stats Annotation - -`StatsDoc` annotation is introduced in [BP-36](https://github.com/apache/bookkeeper/pull/1786). The `StatsDoc` annotation is -used for documenting stats added across the project. - -### Builds & Testing - -- Java 11 is supported for building bookkeeper. - -### Enhancements - -- [Issue 1791: Read Submission should bypass OSE Threads](https://github.com/apache/bookkeeper/pull/1792) -- A new module is introduced for enabling CPU affinity [#1641](https://github.com/apache/bookkeeper/pull/1641) -- [Issue 1682: Added BlockingQueue implementation based on JCtools](https://github.com/apache/bookkeeper/pull/1682) -- [Issue 1813: Set default sizes of DbLedgerStorage read and write cache to be proportional to JVM direct memory](https://github.com/apache/bookkeeper/pull/1813) -- [Issue 1808: Allow to configure sticky reads](https://github.com/apache/bookkeeper/pull/1808) -- [Issue 1754: Netty allocator wrapper](https://github.com/apache/bookkeeper/pull/1754) - -### Bug Fixes - -#### Bookie - -- [Issue #1414: Ensure BufferedChannel instance is properly closed](https://github.com/apache/bookkeeper/pull/1414) -- [Issue #1805: Fixed Auth with V2 protocol](https://github.com/apache/bookkeeper/pull/1805) -- [Issue #1769: prevent race between flush and delete from recreating index](https://github.com/apache/bookkeeper/pull/1769) -- [Issue #1807: Fix sorted ledger storage rotating entry log files too frequent](https://github.com/apache/bookkeeper/pull/1807) -- [Issue #1843: DbLedgerStorage should do periodical flush](https://github.com/apache/bookkeeper/pull/1843) - -#### AutoRecovery - -- [Issue #1578: Fixed deadlock in auditor blocking ZK thread](https://github.com/apache/bookkeeper/pull/1608) -- [Issue #1834: Only publish suspect ledgers if they have missing fragments](https://github.com/apache/bookkeeper/pull/1834) - -#### Client - -- [Issue #1762: Don't cache Bookie hostname DNS resolution forever](https://github.com/apache/bookkeeper/pull/1762) -- [Issue #1788: Fix bugs in DefaultEnsemblePlacementPolicy](https://github.com/apache/bookkeeper/pull/1788) -- [Issue #1862: Fix selectFromNetworkLocation in RackawareEnsemblePlacementPolicyImpl](https://github.com/apache/bookkeeper/pull/1862) -- [Issue #1857: changingEnsemble should be negated before calling unset success](https://github.com/apache/bookkeeper/pull/1857) - -## Full list of changes - -- [https://github.com/apache/bookkeeper/milestone/5](https://github.com/apache/bookkeeper/milestone/5?closed=1) diff --git a/site/docs/4.9.0/overview/releaseNotesTemplate.md b/site/docs/4.9.0/overview/releaseNotesTemplate.md deleted file mode 100644 index 28a667b2e26..00000000000 --- a/site/docs/4.9.0/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.9.0 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.9.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.9.0/reference/cli.md b/site/docs/4.9.0/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.9.0/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.9.0/reference/config.md b/site/docs/4.9.0/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.9.0/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.9.0/reference/metrics.md b/site/docs/4.9.0/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.9.0/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.9.0/security/overview.md b/site/docs/4.9.0/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.9.0/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.9.0/security/sasl.md b/site/docs/4.9.0/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.9.0/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.9.0/security/tls.md b/site/docs/4.9.0/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.9.0/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.9.0/security/zookeeper.md b/site/docs/4.9.0/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.9.0/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.9.1/admin/autorecovery.md b/site/docs/4.9.1/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.9.1/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.9.1/admin/bookies.md b/site/docs/4.9.1/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.9.1/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.9.1/admin/geo-replication.md b/site/docs/4.9.1/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.9.1/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.9.1/admin/http.md b/site/docs/4.9.1/admin/http.md deleted file mode 100644 index ee6411abb53..00000000000 --- a/site/docs/4.9.1/admin/http.md +++ /dev/null @@ -1,457 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.9.1/admin/metrics.md b/site/docs/4.9.1/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.9.1/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.9.1/admin/perf.md b/site/docs/4.9.1/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.9.1/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.9.1/admin/placement.md b/site/docs/4.9.1/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.9.1/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.9.1/admin/upgrade.md b/site/docs/4.9.1/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.9.1/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.9.1/api/distributedlog-api.md b/site/docs/4.9.1/api/distributedlog-api.md deleted file mode 100644 index cce2cdb9f45..00000000000 --- a/site/docs/4.9.1/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.9.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.9.1' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.9.1/api/ledger-adv-api.md b/site/docs/4.9.1/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.9.1/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.9.1/api/ledger-api.md b/site/docs/4.9.1/api/ledger-api.md deleted file mode 100644 index d3ae403d25d..00000000000 --- a/site/docs/4.9.1/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.9.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.9.1</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.9.1' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.9.1' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.9.1/api/overview.md b/site/docs/4.9.1/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.9.1/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.9.1/deployment/dcos.md b/site/docs/4.9.1/deployment/dcos.md deleted file mode 100644 index 9b91dd5f209..00000000000 --- a/site/docs/4.9.1/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.9.1] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.9.1/deployment/kubernetes.md b/site/docs/4.9.1/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.9.1/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.9.1/deployment/manual.md b/site/docs/4.9.1/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.9.1/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.9.1/development/codebase.md b/site/docs/4.9.1/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.9.1/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.9.1/development/protocol.md b/site/docs/4.9.1/development/protocol.md deleted file mode 100644 index 55cc13ac937..00000000000 --- a/site/docs/4.9.1/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub> = 3**, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.9.1/getting-started/concepts.md b/site/docs/4.9.1/getting-started/concepts.md deleted file mode 100644 index d53f9a66cc2..00000000000 --- a/site/docs/4.9.1/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.9.1/getting-started/installation.md b/site/docs/4.9.1/getting-started/installation.md deleted file mode 100644 index 7e92b471581..00000000000 --- a/site/docs/4.9.1/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.9.1/getting-started/run-locally.md b/site/docs/4.9.1/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.9.1/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.9.1/overview/overview.md b/site/docs/4.9.1/overview/overview.md deleted file mode 100644 index 291e26f688b..00000000000 --- a/site/docs/4.9.1/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.9.1 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.9.1. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.9.1 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.9.1/overview/releaseNotes.md b/site/docs/4.9.1/overview/releaseNotes.md deleted file mode 100644 index b97ecb9aac7..00000000000 --- a/site/docs/4.9.1/overview/releaseNotes.md +++ /dev/null @@ -1,24 +0,0 @@ ---- -title: Apache BookKeeper 4.9.1 Release Notes ---- - -This is the 18th release of Apache BookKeeper! - -The 4.9.1 release incorporates a few critical bug fixes, since previous major release, 4.9.0. - -Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.9.1](../../admin/upgrade). The technical details of -this release are summarized below. - -### Dependencies Changes - -No dependency change. - -### Bug Fixes - -- [Issue #1973: [DLOG] Avoid double read in readahead](https://github.com/apache/bookkeeper/pull/1973) -- [Issue #1952: Filter empty string for networkTopologyScriptFileName](https://github.com/apache/bookkeeper/pull/1952) -- [Issue #1950: putEntryOffset translate FileInfoDeletedException](https://github.com/apache/bookkeeper/pull/1950) - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.1+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.1+is%3Aclosed) diff --git a/site/docs/4.9.1/overview/releaseNotesTemplate.md b/site/docs/4.9.1/overview/releaseNotesTemplate.md deleted file mode 100644 index 3f7a0398638..00000000000 --- a/site/docs/4.9.1/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.9.1 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.9.1. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.9.1/reference/cli.md b/site/docs/4.9.1/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.9.1/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.9.1/reference/config.md b/site/docs/4.9.1/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.9.1/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.9.1/reference/metrics.md b/site/docs/4.9.1/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.9.1/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.9.1/security/overview.md b/site/docs/4.9.1/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.9.1/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.9.1/security/sasl.md b/site/docs/4.9.1/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/4.9.1/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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.9.1/security/tls.md b/site/docs/4.9.1/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.9.1/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.9.1/security/zookeeper.md b/site/docs/4.9.1/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.9.1/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/4.9.2/admin/autorecovery.md b/site/docs/4.9.2/admin/autorecovery.md deleted file mode 100644 index b1dd078f9b2..00000000000 --- a/site/docs/4.9.2/admin/autorecovery.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -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: - -* the `shell recover` option -* 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 shell recover \ - 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 shell recover \ - 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.9.2/admin/bookies.md b/site/docs/4.9.2/admin/bookies.md deleted file mode 100644 index 1b0427dae3c..00000000000 --- a/site/docs/4.9.2/admin/bookies.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -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.<init>(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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - <zkserver> \ - <oldbookie> \ - <newbookie> - ``` - - 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 - $ bookkeeper-server/bin/bookkeeper shell recover \ - 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.9.2/admin/geo-replication.md b/site/docs/4.9.2/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/4.9.2/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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.9.2/admin/http.md b/site/docs/4.9.2/admin/http.md deleted file mode 100644 index ee6411abb53..00000000000 --- a/site/docs/4.9.2/admin/http.md +++ /dev/null @@ -1,457 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/4.9.2/admin/metrics.md b/site/docs/4.9.2/admin/metrics.md deleted file mode 100644 index ff46f01e75c..00000000000 --- a/site/docs/4.9.2/admin/metrics.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## 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.prometheus.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 - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/4.9.2/admin/perf.md b/site/docs/4.9.2/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/4.9.2/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/4.9.2/admin/placement.md b/site/docs/4.9.2/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/4.9.2/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/4.9.2/admin/upgrade.md b/site/docs/4.9.2/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/4.9.2/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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.9.2/api/distributedlog-api.md b/site/docs/4.9.2/api/distributedlog-api.md deleted file mode 100644 index 051ad9c84de..00000000000 --- a/site/docs/4.9.2/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.9.2' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.9.2' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.absent() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/4.9.2/api/ledger-adv-api.md b/site/docs/4.9.2/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/4.9.2/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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.9.2/api/ledger-api.md b/site/docs/4.9.2/api/ledger-api.md deleted file mode 100644 index e3ba719d122..00000000000 --- a/site/docs/4.9.2/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>4.9.2</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>4.9.2</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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.9.2' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:4.9.2' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/4.9.2/api/overview.md b/site/docs/4.9.2/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/4.9.2/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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.9.2/deployment/dcos.md b/site/docs/4.9.2/deployment/dcos.md deleted file mode 100644 index c8df0167c73..00000000000 --- a/site/docs/4.9.2/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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.9.2] -Thank you for using bookkeeper. -``` diff --git a/site/docs/4.9.2/deployment/kubernetes.md b/site/docs/4.9.2/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/4.9.2/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/4.9.2/deployment/manual.md b/site/docs/4.9.2/deployment/manual.md deleted file mode 100644 index daafd5556f5..00000000000 --- a/site/docs/4.9.2/deployment/manual.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -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! - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/4.9.2/development/codebase.md b/site/docs/4.9.2/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/4.9.2/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/4.9.2/development/protocol.md b/site/docs/4.9.2/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/4.9.2/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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.9.2/getting-started/concepts.md b/site/docs/4.9.2/getting-started/concepts.md deleted file mode 100644 index d53f9a66cc2..00000000000 --- a/site/docs/4.9.2/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/4.9.2/getting-started/installation.md b/site/docs/4.9.2/getting-started/installation.md deleted file mode 100644 index 7e92b471581..00000000000 --- a/site/docs/4.9.2/getting-started/installation.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -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 -``` - -Since 4.8.0, bookkeeper introduces `table service`. If you would like to build and tryout table service, you can build it with `stream` profile. - -```shell -$ mvn package -Dstream -``` - -> 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 spotbugs:spotbugs` | Compile using the Maven [SpotBugs](https://github.com/spotbugs/spotbugs-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 -am -pl bookkeeper-dist/server package` | Build a server distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin - -> You can enable `table service` by adding the `-Dstream` flag when running above commands. - -## 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.9.2/getting-started/run-locally.md b/site/docs/4.9.2/getting-started/run-locally.md deleted file mode 100644 index edbfab9fda6..00000000000 --- a/site/docs/4.9.2/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bookkeeper-server/bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.9.2/overview/overview.md b/site/docs/4.9.2/overview/overview.md deleted file mode 100644 index b2945664f3f..00000000000 --- a/site/docs/4.9.2/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.9.2 ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version 4.9.2. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper 4.9.2 Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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.9.2/overview/releaseNotes.md b/site/docs/4.9.2/overview/releaseNotes.md deleted file mode 100644 index 52b8e5608da..00000000000 --- a/site/docs/4.9.2/overview/releaseNotes.md +++ /dev/null @@ -1,24 +0,0 @@ ---- -title: Apache BookKeeper 4.9.2 Release Notes ---- - -This is the 18th release of Apache BookKeeper! - -The 4.9.2 release incorporates a few critical bug fixes, since previous major release, 4.9.0. - -Apache BookKeeper/DistributedLog users are encouraged to [upgrade to 4.9.2](../../admin/upgrade). The technical details of -this release are summarized below. - -### Dependencies Changes - -No dependency change. - -### Bug Fixes - -- [Issue #1973: [DLOG] Avoid double read in readahead](https://github.com/apache/bookkeeper/pull/1973) -- [Issue #1952: Filter empty string for networkTopologyScriptFileName](https://github.com/apache/bookkeeper/pull/1952) -- [Issue #1950: putEntryOffset translate FileInfoDeletedException](https://github.com/apache/bookkeeper/pull/1950) - -## Full list of changes - -- [https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.2+is%3Aclosed](https://github.com/apache/bookkeeper/issues?q=label%3Arelease%2F4.9.1+is%3Aclosed) diff --git a/site/docs/4.9.2/overview/releaseNotesTemplate.md b/site/docs/4.9.2/overview/releaseNotesTemplate.md deleted file mode 100644 index 1e002dbf983..00000000000 --- a/site/docs/4.9.2/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.9.2 Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to 4.9.2. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/4.9.2/reference/cli.md b/site/docs/4.9.2/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/4.9.2/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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.9.2/reference/config.md b/site/docs/4.9.2/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/4.9.2/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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.9.2/reference/metrics.md b/site/docs/4.9.2/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/4.9.2/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/4.9.2/security/overview.md b/site/docs/4.9.2/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/4.9.2/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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.9.2/security/sasl.md b/site/docs/4.9.2/security/sasl.md deleted file mode 100644 index e943ec82f51..00000000000 --- a/site/docs/4.9.2/security/sasl.md +++ /dev/null @@ -1,209 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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 - ``` -5. By default bookie service name is `bookkeeper`, you could override it by passing a JVM parameter to the client JVM or set System Property manually. - - For example, if your bookie's principle is bk@bk1.hostname.com@EXAMPLE.COM, then pass: - - ```shell - -Dbookkeeper.sasl.servicename=bk - ``` - -## 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.9.2/security/tls.md b/site/docs/4.9.2/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/4.9.2/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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.9.2/security/zookeeper.md b/site/docs/4.9.2/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/4.9.2/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/docs/latest/admin/autorecovery.md b/site/docs/latest/admin/autorecovery.md deleted file mode 100644 index 5e3a8bc3cdf..00000000000 --- a/site/docs/latest/admin/autorecovery.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -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: - -* the `shell recover` option -* the IP and port for the failed bookie - -Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 # IP and port for the failed bookie -``` - -If you wish, you can also specify which ledgers you'd like to recover. Here's an example: - -```bash -$ bin/bookkeeper shell recover \ - 192.168.1.10:3181 \ # IP and port for the failed bookie - --ledger ledgerID # ledgerID which you want to recover -``` - -### 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 three ways: - -1. On dedicated nodes in your BookKeeper cluster -1. On the same machines on which your bookies are running -1. On a combination of autorecovery nodes and bookie nodes - -## Running AutoRecovery - -You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. - -```bash -$ 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. - -Note that if you _only_ want the AutoRecovery process to run on your dedicated AutoRecovery nodes, you must set `autoRecoveryDaemonEnabled` to `false` in the `bookkeeper` configuration. Otherwise, -bookkeeper nodes will also handle rereplication work. - -## 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 for the whole cluster 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 for the whole cluster using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: - -```bash -$ bin/bookkeeper shell autorecovery -disable -``` - -Once disabled, you can reenable AutoRecovery for the whole cluster using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: - -```bash -$ 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/latest/admin/bookies.md b/site/docs/latest/admin/bookies.md deleted file mode 100644 index 35f629c847f..00000000000 --- a/site/docs/latest/admin/bookies.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -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 -$ 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 -$ 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 -$ 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 `log4j2.xml` file and point the `BOOKIE_LOG_CONF` environment variable to the configuration file. Here's an example: - -```shell -$ export BOOKIE_LOG_CONF=/some/path/log4j2.xml -$ 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 -$ bin/bookkeeper upgrade --upgrade -``` - -...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... - -```shell -$ bin/bookkeeper upgrade --finalize -``` - -...and then restart the server: - -```shell -$ bin/bookkeeper bookie -``` - -If something has gone wrong, you can always perform a rollback: - -```shell -$ 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 -$ 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 -$ 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.<init>(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 shell recover <oldbookie> - ``` - - 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 shell recover 192.168.1.10:3181 - ``` - - See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/latest/admin/decomission.md b/site/docs/latest/admin/decomission.md deleted file mode 100644 index 842b2508b35..00000000000 --- a/site/docs/latest/admin/decomission.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -title: Decommission Bookies ---- - -In case the user wants to decommission a bookie, the following process is useful to follow in order to verify if the -decommissioning was safely done. - -### Before we decommission -1. Ensure state of your cluster can support the decommissioning of the target bookie. -Check if `EnsembleSize >= Write Quorum >= Ack Quorum` stays true with one less bookie - -2. Ensure target bookie shows up in the listbookies command. - -3. Ensure that there is no other process ongoing (upgrade etc). - -### Process of Decommissioning -1. Log on to the bookie node, check if there are underreplicated ledgers. - -If there are, the decommission command will force them to be replicated. -`$ bin/bookkeeper shell listunderreplicated` - -2. Stop the bookie -`$ bin/bookkeeper-daemon.sh stop bookie` - -3. Run the decommission command. -If you have logged onto the node you wish to decommission, you don't need to provide `-bookieid` -If you are running the decommission command for target bookie node from another bookie node you should mention -the target bookie id in the arguments for `-bookieid` -`$ bin/bookkeeper shell decommissionbookie` -or -`$ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>` - -4. Validate that there are no ledgers on decommissioned bookie -`$ bin/bookkeeper shell listledgers -bookieid <target bookieid>` - -Last step to verify is you could run this command to check if the bookie you decommissioned doesn’t show up in list bookies: - -```bash -./bookkeeper shell listbookies -rw -h -./bookkeeper shell listbookies -ro -h -``` diff --git a/site/docs/latest/admin/geo-replication.md b/site/docs/latest/admin/geo-replication.md deleted file mode 100644 index 38b972345ef..00000000000 --- a/site/docs/latest/admin/geo-replication.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -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/latest/admin/http.md b/site/docs/latest/admin/http.md deleted file mode 100644 index 50c3a7db01d..00000000000 --- a/site/docs/latest/admin/http.md +++ /dev/null @@ -1,509 +0,0 @@ ---- -title: BookKeeper Admin REST API ---- - -This document introduces BookKeeper HTTP endpoints, which can be used for BookKeeper administration. -To use this feature, set `httpServerEnabled` to `true` in file `conf/bk_server.conf`. - -## All the endpoints - -Currently all the HTTP endpoints could be divided into these 5 components: -1. Heartbeat: heartbeat for a specific bookie. -2. Config: doing the server configuration for a specific bookie. -3. Ledger: HTTP endpoints related to ledgers. -4. Bookie: HTTP endpoints related to bookies. -5. AutoRecovery: HTTP endpoints related to auto recovery. - -## Heartbeat - -### Endpoint: /heartbeat -* Method: GET -* Description: Get heartbeat status for a specific bookie -* Response: - -| Code | Description | -|:-------|:------------| -|200 | Successful operation | - -## Config - -### Endpoint: /api/v1/config/server_config -1. Method: GET - * Description: Get value of all configured values overridden on local server config - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | -1. Method: PUT - * Description: Update a local server config - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |configName | String | Yes | Configuration name(key) | - |configValue | String | Yes | Configuration value(value) | - * Body: - ```json - { - "configName1": "configValue1", - "configName2": "configValue2" - } - ``` - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Config - -### Endpoint: /metrics -1. Method: GET - * Description: Get all metrics by calling `writeAllMetrics()` of `statsProvider` internally - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -## Ledger - -### Endpoint: /api/v1/ledger/delete/?ledger_id=<ledger_id> -1. Method: DELETE - * Description: Delete a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/ledger/list/?print_metadata=<metadata> -1. Method: GET - * Description: List all the ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |print_metadata | Boolean | No | whether print out metadata | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1", - "ledgerId2": "ledgerMetadata2", - ... - } - ``` - -### Endpoint: /api/v1/ledger/metadata/?ledger_id=<ledger_id> -1. Method: GET - * Description: Get the metadata of a ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes | ledger id of the ledger. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "ledgerId1": "ledgerMetadata1" - } - ``` - -### Endpoint: /api/v1/ledger/read/?ledger_id=<ledger_id>&start_entry_id=<start_entry_id>&end_entry_id=<end_entry_id> -1. Method: GET - * Description: Read a range of entries from ledger. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |ledger_id | Long | Yes| ledger id of the ledger. | - |start_entry_id | Long | No | start entry id of read range. | - |end_entry_id | Long | No | end entry id of read range. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "entryId1": "entry content 1", - "entryId2": "entry content 2", - ... - } - ``` - -## Bookie - -### Endpoint: /api/v1/bookie/info -1. Method: GET - * Description: Get bookie info - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |501 | Not implemented | - * Body: - ```json - { - "freeSpace" : 0, - "totalSpace" : 0 - } - ``` - -### Endpoint: /api/v1/bookie/list_bookies/?type=<type>&print_hostnames=<hostnames> -1. Method: GET - * Description: Get all the available bookies. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | Yes | value: "rw" or "ro" , list read-write/read-only bookies. | - |print_hostnames | Boolean | No | whether print hostname of bookies. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieSocketAddress1": "hostname1", - "bookieSocketAddress2": "hostname2", - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_bookie_info -1. Method: GET - * Description: Get bookies disk usage info of this cluster. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "bookieAddress" : {free: xxx, total: xxx}, - "bookieAddress" : {free: xxx, total: xxx}, - ... - "clusterInfo" : {total_free: xxx, total: xxx} - } - ``` - -### Endpoint: /api/v1/bookie/last_log_mark -1. Method: GET - * Description: Get the last log marker. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - JournalId1 : position1, - JournalId2 : position2, - ... - } - ``` - -### Endpoint: /api/v1/bookie/list_disk_file/?file_type=<type> -1. Method: GET - * Description: Get all the files on disk of current bookie. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |type | String | No | file type: journal/entrylog/index. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "journal files" : "filename1 filename2 ...", - "entrylog files" : "filename1 filename2...", - "index files" : "filename1 filename2 ..." - } - ``` - -### Endpoint: /api/v1/bookie/expand_storage -1. Method: PUT - * Description: Expand storage for a bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/bookie/gc -1. Method: PUT - * Description: trigger gc for this bookie. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: GET - * Description: whether force triggered Garbage Collection is running or not for this bookie. true for is running. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "is_in_force_gc" : "false" - } - ``` - -### Endpoint: /api/v1/bookie/gc_details -1. Method: GET - * Description: get details of Garbage Collection Thread, like whether it is in compacting, last compaction time, compaction counter, etc. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - [ { - "forceCompacting" : false, - "majorCompacting" : false, - "minorCompacting" : false, - "lastMajorCompactionTime" : 1544578144944, - "lastMinorCompactionTime" : 1544578144944, - "majorCompactionCounter" : 1, - "minorCompactionCounter" : 0 - } ] - ``` - -### Endpoint: /api/v1/bookie/state -1. Method: GET - * Description: Exposes the current state of bookie - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Body: - ```json - { - "running" : true, - "readOnly" : false, - "shuttingDown" : false, - "availableForHighPriorityWrites" : true - } - ``` - -### Endpoint: /api/v1/bookie/is_ready -1. Method: GET - * Description: Return true if the bookie is ready - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - |503 | Bookie is not ready | - * Body: <empty> - - -## Auto recovery - -### Endpoint: /api/v1/autorecovery/bookie/ -1. Method: PUT - * Description: Ledger data recovery for failed bookie - * Body: - ```json - { - "bookie_src": [ "bookie_src1", "bookie_src2"... ], - "bookie_dest": [ "bookie_dest1", "bookie_dest2"... ], - "delete_cookie": <bool_value> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |bookie_src | Strings | Yes | bookie source to recovery | - |bookie_dest | Strings | No | bookie data recovery destination | - |delete_cookie | Boolean | No | Whether delete cookie | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/list_under_replicated_ledger/?missingreplica=<bookie_address>&excludingmissingreplica=<bookie_address> -1. Method: GET - * Description: Get all under replicated ledgers. - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - |missingreplica | String | No | missing replica bookieId | - |excludingmissingreplica | String | No | exclude missing replica bookieId | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - [ledgerId1, ledgerId2...] - } - ``` - -### Endpoint: /api/v1/autorecovery/who_is_auditor -1. Method: GET - * Description: Get auditor bookie id. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - * Response Body format: - - ```json - { - "Auditor": "hostname/hostAddress:Port" - } - ``` - -### Endpoint: /api/v1/autorecovery/trigger_audit -1. Method: PUT - * Description: Force trigger audit by resting the lostBookieRecoveryDelay. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/lost_bookie_recovery_delay -1. Method: GET - * Description: Get lostBookieRecoveryDelay value in seconds. - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -1. Method: PUT - * Description: Set lostBookieRecoveryDelay value in seconds. - * Body: - ```json - { - "delay_seconds": <delay_seconds> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | delay_seconds | Long | Yes | set delay value in seconds. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | - -### Endpoint: /api/v1/autorecovery/decommission -1. Method: PUT - * Description: Decommission Bookie, Force trigger Audit task and make sure all the ledgers stored in the decommissioning bookie are replicated. - * Body: - ```json - { - "bookie_src": <bookie_src> - } - ``` - * Parameters: - - | Name | Type | Required | Description | - |:-----|:-----|:---------|:------------| - | bookie_src | String | Yes | Bookie src to decommission.. | - * Response: - - | Code | Description | - |:-------|:------------| - |200 | Successful operation | - |403 | Permission denied | - |404 | Not found | diff --git a/site/docs/latest/admin/metrics.md b/site/docs/latest/admin/metrics.md deleted file mode 100644 index 0048f30289c..00000000000 --- a/site/docs/latest/admin/metrics.md +++ /dev/null @@ -1,36 +0,0 @@ ---- -title: Metric collection ---- - -BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). - -## Stats providers - -BookKeeper has stats provider implementations for these 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.prometheus.PrometheusMetricsProvider` - -> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. - -## Enabling stats providers in bookies - -Two stats-related [configuration parameters](../../reference/config/) are 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) - -<!-- ## Enabling stats in the bookkeeper library - -TODO ---> diff --git a/site/docs/latest/admin/perf.md b/site/docs/latest/admin/perf.md deleted file mode 100644 index 82956326e5d..00000000000 --- a/site/docs/latest/admin/perf.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Performance tuning ---- diff --git a/site/docs/latest/admin/placement.md b/site/docs/latest/admin/placement.md deleted file mode 100644 index ded456e1aea..00000000000 --- a/site/docs/latest/admin/placement.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: Customized placement policies ---- diff --git a/site/docs/latest/admin/upgrade.md b/site/docs/latest/admin/upgrade.md deleted file mode 100644 index 57c65208131..00000000000 --- a/site/docs/latest/admin/upgrade.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -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.6.x to 4.7.0 upgrade - -There isn't any protocol related backward compabilities changes in 4.7.0. So you can follow the general upgrade sequence to upgrade from 4.6.x to 4.7.0. - -However, we list a list of changes that you might want to know. - -#### Common Configuration Changes - -This section documents the common configuration changes that applied for both clients and servers. - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| allowShadedLedgerManagerFactoryClass | false | The allows bookkeeper client to connect to a bookkeeper cluster using a shaded ledger manager factory | -| shadedLedgerManagerFactoryClassPrefix | `dlshade.` | The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true | -| metadataServiceUri | null | metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location | -| permittedStartupUsers | null | The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set | - -##### Deprecated Settings - -There are no common settings deprecated at 4.7.0. - -##### Changed Settings - -There are no common settings whose default value are changed at 4.7.0. - -#### Server Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| verifyMetadataOnGC | false | Whether the bookie is configured to double check the ledgers' metadata prior to garbage collecting them | -| auditorLedgerVerificationPercentage | 0 | The percentage of a ledger (fragment)'s entries will be verified by Auditor before claiming a ledger (fragment) is missing | -| numHighPriorityWorkerThreads | 8 | The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by Netty threads directly. | -| useShortHostName | false | Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when useHostNameAsBookieID is enabled. | -| minUsableSizeForEntryLogCreation | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to create entry log files (in bytes). | -| minUsableSizeForHighPriorityWrites | 1.2 * `logSizeLimit` | Minimum safe usable size to be available in ledger directory for bookie to accept high priority writes even it is in readonly mode. | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| registrationManagerClass | The registration manager class used by server to discover registration manager. It is replaced by `metadataServiceUri`. | - - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| numLongPollWorkerThreads | 10 | 0 | If the number of threads is zero or negative, bookie can fallback to use read threads for long poll. This allows not creating threads if application doesn't use long poll feature. | - -#### Client Configuration Changes - -##### New Settings - -Following settings are newly added in 4.7.0. - -| Name | Default Value | Description | -|------|---------------|-------------| -| maxNumEnsembleChanges | Integer.MAX\_VALUE | The max allowed ensemble change number before sealing a ledger on failures | -| timeoutMonitorIntervalSec | min(`addEntryTimeoutSec`, `addEntryQuorumTimeoutSec`, `readEntryTimeoutSec`) | The interval between successive executions of the operation timeout monitor, in seconds | -| ensemblePlacementPolicyOrderSlowBookies | false | Flag to enable/disable reordering slow bookies in placement policy | - -##### Deprecated Settings - -Following settings are deprecated since 4.7.0. - -| Name | Description | -|------|-------------| -| clientKeyStoreType | Replaced by `tlsKeyStoreType` | -| clientKeyStore | Replaced by `tlsKeyStore` | -| clientKeyStorePasswordPath | Replaced by `tlsKeyStorePasswordPath` | -| clientTrustStoreType | Replaced by `tlsTrustStoreType` | -| clientTrustStore | Replaced by `tlsTrustStore` | -| clientTrustStorePasswordPath | Replaced by `tlsTrustStorePasswordPath` | -| registrationClientClass | The registration client class used by client to discover registration service. It is replaced by `metadataServiceUri`. | - -##### Changed Settings - -The default values of following settings are changed since 4.7.0. - -| Name | Old Default Value | New Default Value | Notes | -|------|-------------------|-------------------|-------| -| enableDigestTypeAutodetection | false | true | Autodetect the digest type and passwd when opening a ledger. It will ignore the provided digest type, but still verify the provided passwd. | - -### 4.7.x to 4.8.X upgrade - -In 4.8.x a new feature is added to persist explicitLac in FileInfo and explicitLac entry in Journal. (Note: Currently this feature is not available if your ledgerStorageClass is DbLedgerStorage, ISSUE #1533 is going to address it) Hence current journal format version is bumped to 6 and current FileInfo header version is bumped to 1. But since default config values of 'journalFormatVersionToWrite' and 'fileInfoFormatVersionToWrite' are set to older versions, this feature is off by default. To enable this feature those config values should be set to current versions. Once this is enabled then we cannot rollback to previous Bookie versions (4.7.x and older), since older version code would not be able to deal with explicitLac entry in Journal file while replaying journal and also reading Header of Index files / FileInfo would fail reading Index files with newer FileInfo version. So in summary, it is a non-rollbackable feature and it applies even if explicitLac is not being used. - -### 4.5.x to 4.6.x upgrade - -There isn't any protocol related backward compabilities changes in 4.6.x. So you can follow the general upgrade sequence to upgrade from 4.5.x to 4.6.x. - -### 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/latest/api/distributedlog-api.md b/site/docs/latest/api/distributedlog-api.md deleted file mode 100644 index 9002d058dff..00000000000 --- a/site/docs/latest/api/distributedlog-api.md +++ /dev/null @@ -1,395 +0,0 @@ ---- -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 (<strong>D</strong>istributed<strong>L</strong>og <strong>S</strong>equence <strong>N</strong>umber). - -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://bookkeeper.apache.org/distributedlog/docs/latest/user_guide/api/core). - -> At a later date, the DistributedLog API docs will be added here. - -<!-- - -The DistributedLog core library is written in Java and interacts with namespaces and logs directly. - -### 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 -<-- in your <properties> block -> -<bookkeeper.version>{{ site.distributedlog_version }}</bookkeeper.version> - -<-- in your <dependencies> block -> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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: '{{ site.latest_version }}' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:{{ site.latest_version }}' -} -``` - -### Namespace API - -A DL [namespace](#namespace) is a collection of [log streams](#log-streams). When using the DistributedLog API with BookKeeper, you need to provide your Java client with a namespace URI. That URI consists of three elements: - -1. The `distributedlog-bk` scheme -1. A connection string for your BookKeeper cluster. You have three options for the connection string: - * An entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181` - * A host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full ZooKeeper connection string. - * If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. -1. A path that points to the location where logs are stored. This could be a ZooKeeper [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html). - -This is the general structure of a namespace URI: - -```shell -distributedlog-bk://{connection-string}/{path} -``` - -Here are some example URIs: - -```shell -distributedlog-bk://zk1:2181,zk2:2181,zk3:2181/my-namespace # Full ZooKeeper connection string -distributedlog-bk://localhost:2181/my-namespace # Single ZooKeeper node -distributedlog-bk://my-zookeeper-cluster.com/my-namespace # DNS name for ZooKeeper -``` - -#### Creating namespaces - -In order to create namespaces, you need to use the command-line tool. - -```shell -$ -``` - -#### Using namespaces - -Once you have a namespace URI, you can build a namespace instance, which will be used for operating streams. Use the `DistributedLogNamespaceBuilder` to build a `DistributedLogNamespace` object, passing in a `DistributedLogConfiguration`, a URI, and optionally a stats logger and a feature provider. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace "); -DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder(); -DistributedLogNamespace = builder - .conf(conf) // Configuration for the namespace - .uri(uri) // URI for the namespace - .statsLogger(...) // Stats logger for statistics - .featureProvider(...) // Feature provider for controlling features - .build(); -``` - -### Log API - -#### Creating logs - -You can create a log by calling the `createLog` method on a `DistributedLogNamespace` object, passing in a name for the log. This creates the log under the namespace but does *not* return a handle for operating the log. - -```java -DistributedLogNamespace namespace = /* Create namespace */; -try { - namespace.createLog("test-log"); -} catch (IOException e) } - // Handle the log creation exception -} -``` - -#### Opening logs - -A `DistributedLogManager` handle will be returned when opening a log using the `openLog` function, which takes the name of the log. This handle can be used for writing records to or reading records from the log. - -> If the log doesn't exist and `createStreamIfNotExists` is set to `true` in the configuration, the log will be created automatically when writing the first record. - -```java -DistributedLogConfiguration conf = new DistributedLogConfiguration(); -conf.setCreateStreamIfNotExists(true); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(conf) - // Other builder attributes - .build(); -DistributedLogManager logManager = namespace.openLog("test-log"); -``` - -Sometimes, applications may open a log with a different configuration from the enclosing namespace. This can be done using the same `openLog` method: - -```java -// Namespace configuration -DistributedLogConfiguration namespaceConf = new DistributedLogConfiguration(); -conf.setRetentionPeriodHours(24); -URI uri = URI.create("distributedlog-bk://localhost:2181/my-namespace"); -DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder() - .conf(namespaceConf) - .uri(uri) - // Other builder attributes - .build(); -// Log-specific configuration -DistributedLogConfiguration logConf = new DistributedLogConfiguration(); -logConf.setRetentionPeriodHours(12); -DistributedLogManager logManager = namespace.openLog( - "test-log", - Optional.of(logConf), - Optional.empty() -); -``` - -#### Deleting logs - -The `DistributedLogNamespace` class provides `deleteLog` function that can be used to delete logs. When you delete a lot, the client library will attempt to acquire a lock on the log before deletion. If the log is being written to by an active writer, deletion will fail (as the other writer currently holds the lock). - -```java -try { - namespace.deleteLog("test-log"); -} catch (IOException e) { - // Handle exception -} -``` - -#### Checking for the existence of a log - -Applications can check whether a log exists by calling the `logExists` function. - -```java -if (namespace.logExists("test-log")) { - // Perform some action when the log exists -} else { - // Perform some action when the log doesn't exist -} -``` - -#### Listing logs - -Applications can retrieve a list of all logs under a namespace using the `getLogs` function. - -```java -Iterator<String> logs = namespace.getLogs(); -while (logs.hasNext()) { - String logName = logs.next(); - // Do something with the log name, such as print -} -``` - -### Writer API - -You can write to DistributedLog logs either [synchronously](#writing-to-logs-synchronously) using the `LogWriter` class or [asynchronously](#writing-to-logs-asynchronously) using the `AsyncLogWriter` class. - -#### Immediate flush - -By default, records are buffered rather than being written immediately. You can disable this behavior and make DL writers write ("flush") entries immediately by adding the following to your configuration object: - -```java -conf.setImmediateFlushEnabled(true); -conf.setOutputBufferSize(0); -conf.setPeriodicFlushFrequencyMilliSeconds(0); -``` - -#### Immediate locking - -By default, DL writers can write to a log stream when other writers are also writing to that stream. You can override this behavior and disable other writers from writing to the stream by adding this to your configuration: - -```java -conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE); -``` - -#### Writing to logs synchronously - -To write records to a log synchronously, you need to instantiate a `LogWriter` object using a `DistributedLogManager`. Here's an example: - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-log"); -LogWriter writer = logManager.startLogSegmentNonPartitioned(); -``` - -> The DistributedLog library enforces single-writer semantics by deploying a ZooKeeper locking mechanism. If there is only one active writer, subsequent calls to `startLogSegmentNonPartitioned` will fail with an `OwnershipAcquireFailedException`. - -Log records represent the data written to a log stream. Each log record is associated with an application-defined [TransactionID](#log-records). This ID must be non decreasing or else writing a record will be rejected with `TransactionIdOutOfOrderException`. The application is allowed to bypass the TransactionID sanity checking by setting `maxIdSanityCheck` to `false` in the configuration. System time and atomic numbers are good candidates for TransactionID. - -```java -long txid = 1L; -byte[] data = "some byte array".getBytes(); -LogRecord record = new LogRecord(txid, data); -``` - -Your application can write either a single record, using the `write` method, or many records, using the `writeBulk` method. - -```java -// Single record -writer.write(record); - -// Bulk write -List<LogRecord> records = Lists.newArrayList(); -records.add(record); -writer.writeBulk(records); -``` - -The write calls return immediately after the records are added into the output buffer of writer. This means that the data isn't guaranteed to be durable until the writer explicitly calls `setReadyToFlush` and `flushAndSync`. Those two calls will first transmit buffered data to the backend, wait for transmit acknowledgements (acks), and commit the written data to make them visible to readers. - -```java -// Flush the records -writer.setReadyToFlush(); - -// Commit the records to make them visible to readers -writer.flushAndSync(); -``` - -Log streams in DistributedLog are endless streams *unless they are sealed*. Endless in this case means that writers can keep writing records to those streams, readers can keep reading from the end of those streams, and the process never stops. Your application can seal a log stream using the `markEndOfStream` method: - -```java -writer.markEndOfStream(); -``` - -#### Writing to logs asynchronously - -In order to write to DistributedLog logs asynchronously, you need to create an `AsyncLogWriter` instread of a `LogWriter`. - -```java -DistributedLogNamespace namespace = /* Some namespace object */; -DistributedLogManager logManager = namespace.openLog("test-async-log"); -AsyncLogWriter asyncWriter = logManager.startAsyncLogSegmentNonPartitioned(); -``` - -All writes to `AsyncLogWriter` are non partitioned. The futures representing write results are only satisfied when the data is durably persisted in the stream. A [DLSN](#log-records) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream. All the records added in order are guaranteed to be persisted in order. Here's an example of an async writer that gathers a list of futures representing multiple async write results: - -```java -List<Future<DLSN>> addFutures = Lists.newArrayList(); -for (long txid = 1L; txid <= 100L; txid++) { - byte[] data = /* some byte array */; - LogRecord record = new LogRecord(txid, data); - addFutures.add(asyncWriter.write(record)); -} -List<DLSN> addResults = Await.result(Future.collect(addFutures)); -``` - -The `AsyncLogWriter` also provides a method for truncating a stream to a given DLSN. This is useful for building replicated state machines that need explicit controls on when the data can be deleted. - -```java -DLSN truncateDLSN = /* some DLSN */; -Future<DLSN> truncateFuture = asyncWriter.truncate(truncateDLSN); - -// Wait for truncation result -Await.result(truncateFuture); -``` - -##### Register a listener - -Instead of returning a future from write operations, you can also set up a listener that performs assigned actions upon success or failure of the write. Here's an example: - -```java -asyncWriter.addEventListener(new FutureEventListener<DLSN>() { - @Override - public void onFailure(Throwable cause) { - // Execute if the attempt fails - } - - @Override - public void onSuccess(DLSN value) { - // Execute if the attempt succeeds - } -}); -``` - -##### Close the writer - -You can close an async writer when you're finished with it like this: - -```java -FutureUtils.result(asyncWriter.asyncClose()); -``` - -<!-- -TODO: Reader API ---> diff --git a/site/docs/latest/api/ledger-adv-api.md b/site/docs/latest/api/ledger-adv-api.md deleted file mode 100644 index df6224dd7ec..00000000000 --- a/site/docs/latest/api/ledger-adv-api.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -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. - -Creating advanced ledgers can be done throught a fluent API since 4.6. - -```java -BookKeeper bk = ...; - -byte[] passwd = "some-passwd".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(passwd) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .makeAdv() // convert the create ledger builder to create ledger adv builder - .withLedgerId(1234L) - .execute() // execute the creation op - .get(); // wait for the execution to complete - -``` - -### 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()); -``` - -If you are using the new API, you can do as following: - -```java -WriteHandle wh = ...; -long entryId = ...; // entry id generated externally - -wh.write(entryId, "Some entry data".getBytes()).get(); -``` - -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/latest/api/ledger-api.md b/site/docs/latest/api/ledger-api.md deleted file mode 100644 index 7ce2d991741..00000000000 --- a/site/docs/latest/api/ledger-api.md +++ /dev/null @@ -1,840 +0,0 @@ ---- -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 -<!-- in your <properties> block --> -<bookkeeper.version>{{ site.latest_version }}</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -BookKeeper uses google [protobuf](https://github.com/google/protobuf/tree/master/java) and [guava](https://github.com/google/guava) libraries -a lot. If your application might include different versions of protobuf or guava introduced by other dependencies, you can choose to use the -shaded library, which relocate classes of protobuf and guava into a different namespace to avoid conflicts. - -```xml -<!-- in your <properties> block --> -<bookkeeper.version>{{ site.latest_version }}</bookkeeper.version> - -<!-- in your <dependencies> block --> -<dependency> - <groupId>org.apache.bookkeeper</groupId> - <artifactId>bookkeeper-server-shaded</artifactId> - <version>${bookkeeper.version}</version> -</dependency> -``` - -### 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: '{{ site.latest_version }}' -} - -// Alternatively: -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server:{{ site.latest_version }}' -} -``` - -Similarly as using maven, you can also configure to use the shaded jars. - -```groovy -// use the `bookkeeper-server-shaded` jar -dependencies { - compile 'org.apache.bookkeeper:bookkeeper-server-shaded:{{ site.latest-version }}' -} -``` - -## 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<LedgerEntry> entries = handle.readEntries(1, 99); -``` - -To read all possible entries from the ledger: - -```java -Enumerator<LedgerEntry> 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<LedgerEntry> 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 be deleted synchronously which may throw exception: - -```java -long ledgerId = 1234; - -try { - bkClient.deleteLedger(ledgerId); -} catch (Exception e) { - e.printStackTrace(); -} -``` - -### Delete entries asynchronously - -{% pop Ledgers %} can also be deleted asynchronously: - -```java -class DeleteEntryCallback implements AsyncCallback.DeleteCallback { - public void deleteComplete() { - System.out.println("Delete completed"); - } -} -bkClient.asyncDeleteLedger(ledgerID, new DeleteEntryCallback(), null); -``` - -## 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<LedgerEntry> entries = lh.readEntries(0, numberOfEntries - 1); - -while(entries.hasMoreElements()) { - ByteBuffer result = ByteBuffer.wrap(entries.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 -$ 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 -``` - -## New API - -Since 4.6 BookKeeper provides a new client API which leverages Java8 [CompletableFuture](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html) facility. -[WriteHandle](../javadoc/org/apache/bookkeeper/client/api/WriteHandle), [WriteAdvHandle](../javadoc/org/apache/bookkeeper/client/api/WriteAdvHandle), [ReadHandle](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) are introduced for replacing the generic [LedgerHandle](../javadoc/org/apache/bookkeeper/client/LedgerHandle). - -> All the new API now is available in `org.apache.bookkeeper.client.api`. You should only use interfaces defined in this package. - -*Beware* that this API in 4.6 is still experimental API and can be subject to changes in next minor releases. - -### Create a new client - -In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/api/BookKeeper) client object, you need to construct a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object and set a [connection string](#connection-string) first, and then use [`BookKeeperBuilder`](../javadoc/org/apache/bookkeeper/client/api/BookKeeperBuilder) to build the client. - -Here is an example building the bookkeeper client. - -```java -// construct a client configuration instance -ClientConfiguration conf = new ClientConfiguration(); -conf.setZkServers(zkConnectionString); -conf.setZkLedgersRootPath("/path/to/ledgers/root"); - -// build the bookkeeper client -BookKeeper bk = BookKeeper.newBuilder(conf) - .statsLogger(...) - ... - .build(); - -``` - -### Create ledgers - -the easiest way to create a {% pop ledger %} using the java client is via the [`createbuilder`](../javadoc/org/apache/bookkeeper/client/api/createbuilder). you must specify at least -a [`digesttype`](../javadoc/org/apache/bookkeeper/client/api/digesttype) and a password. - -here's an example: - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - -A [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) is returned for applications to write and read entries to and from the ledger. - -### Write flags - -You can specify behaviour of the writer by setting [`WriteFlags`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) at ledger creation type. -These flags are applied only during write operations and are not recorded on metadata. - - -Available write flags: - -| Flag | Explanation | Notes | -:---------|:------------|:------- -DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees of durability | Data will be only written to the OS page cache, without forcing an fsync. - -```java -BookKeeper bk = ...; - -byte[] password = "some-password".getBytes(); - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete -``` - - -### Append entries to ledgers - -The [`WriteHandle`](../javadoc/org/apache/bookkeeper/client/api/WriteHandle) can be used for applications to append entries to the ledgers. - -```java -WriteHandle wh = ...; - -CompletableFuture<Long> addFuture = wh.append("Some entry data".getBytes()); - -// option 1: you can wait for add to complete synchronously -try { - long entryId = FutureUtils.result(addFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -addFuture - .thenApply(entryId -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -addFuture.whenComplete(new FutureEventListener() { - @Override - public void onSuccess(long entryId) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -The append method supports three representations of a bytes array: the native java `byte[]`, java nio `ByteBuffer` and netty `ByteBuf`. -It is recommended to use `ByteBuf` as it is more gc friendly. - -### Open ledgers - -You can open ledgers to read entries. Opening ledgers is done by [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder). You must specify the ledgerId and the password -in order to open the ledgers. - -here's an example: - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .execute() // execute the open op - .get(); // wait for the execution to complete -``` - -A [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) is returned for applications to read entries to and from the ledger. - -#### Recovery vs NoRecovery - -By default, the [`openBuilder`](../javadoc/org/apache/bookkeeper/client/api/openBuilder) opens the ledger in a `NoRecovery` mode. You can open the ledger in `Recovery` mode by specifying -`withRecovery(true)` in the open builder. - -```java -BookKeeper bk = ...; - -long ledgerId = ...; -byte[] password = "some-password".getBytes(); - -ReadHandle rh = bk.newOpenLedgerOp() - .withLedgerId(ledgerId) - .withPassword(password) - .withRecovery(true) - .execute() - .get(); - -``` - -**What is the difference between "Recovery" and "NoRecovery"?** - -If you are opening a ledger in "Recovery" mode, it will basically fence and seal the ledger -- no more entries are allowed -to be appended to it. The writer which is currently appending entries to the ledger will fail with [`LedgerFencedException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#LedgerFencedException). - -In constrat, opening a ledger in "NoRecovery" mode, it will not fence and seal the ledger. "NoRecovery" mode is usually used by applications to tailing-read from a ledger. - -### Read entries from ledgers - -The [`ReadHandle`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle) returned from the open builder can be used for applications to read entries from the ledgers. - -```java -ReadHandle rh = ...; - -long startEntryId = ...; -long endEntryId = ...; -CompletableFuture<LedgerEntries> readFuture = rh.read(startEntryId, endEntryId); - -// option 1: you can wait for read to complete synchronously -try { - LedgerEntries entries = FutureUtils.result(readFuture.get()); -} catch (BKException bke) { - // error handling -} - -// option 2: you can process the result and exception asynchronously -readFuture - .thenApply(entries -> { - // process the result - }) - .exceptionally(cause -> { - // handle the exception - }) - -// option 3: bookkeeper provides a twitter-future-like event listener for processing result and exception asynchronously -readFuture.whenComplete(new FutureEventListener<>() { - @Override - public void onSuccess(LedgerEntries entries) { - // process the result - } - @Override - public void onFailure(Throwable cause) { - // handle the exception - } -}); -``` - -Once you are done with processing the [`LedgerEntries`](../javadoc/org/apache/bookkeeper/client/api/LedgerEntries), you can call `#close()` on the `LedgerEntries` instance to -release the buffers held by it. - -Applications are allowed to read any entries between `0` and [`LastAddConfirmed`](../javadoc/org/apache/bookkeeper/client/api/ReadHandle.html#getLastAddConfirmed). If the applications -attempts to read entries beyond `LastAddConfirmed`, they will receive [`IncorrectParameterException`](../javadoc/org/apache/bookkeeper/client/api/BKException.Code#IncorrectParameterException). - -### Read unconfirmed entries from ledgers - -`readUnconfirmed` is provided the mechanism for applications to read entries beyond `LastAddConfirmed`. Applications should be aware of `readUnconfirmed` doesn't provide any -repeatable read consistency. - -```java -CompletableFuture<LedgerEntries> readFuture = rh.readUnconfirmed(startEntryId, endEntryId); -``` - -### Tailing Reads - -There are two methods for applications to achieve tailing reads: `Polling` and `Long-Polling`. - -#### Polling - -You can do this in synchronous way: - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - Thread.sleep(1000); - - lac = rh.readLastAddConfirmed().get(); - continue; - } - - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - - nextEntryId = endEntryId + 1; -} -``` - -#### Long Polling - -```java -ReadHandle rh = ...; - -long startEntryId = 0L; -long nextEntryId = startEntryId; -int numEntriesPerBatch = 4; -while (!rh.isClosed() || nextEntryId <= rh.getLastAddConfirmed()) { - long lac = rh.getLastAddConfirmed(); - if (nextEntryId > lac) { - // no more entries are added - try (LastConfirmedAndEntry lacAndEntry = rh.readLastAddConfirmedAndEntry(nextEntryId, 1000, false).get()) { - if (lacAndEntry.hasEntry()) { - // process the entry - - ++nextEntryId; - } - } - } else { - long endEntryId = Math.min(lac, nextEntryId + numEntriesPerBatch - 1); - LedgerEntries entries = rh.read(nextEntryId, endEntryId).get(); - - // process the entries - nextEntryId = endEntryId + 1; - } -} -``` - -### Delete ledgers - -{% pop Ledgers %} can be deleted by using [`DeleteBuilder`](../javadoc/org/apache/bookkeeper/client/api/DeleteBuilder). - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -bk.newDeleteLedgerOp() - .withLedgerId(ledgerId) - .execute() - .get(); -``` - -### Relaxing Durability - -In BookKeeper by default each write will be acklowledged to the client if and only if it has been persisted durably (fsync called on the file system) by a quorum of bookies. -In this case the LastAddConfirmed pointer is updated on the writer side, this is the guarantee for the writer that data will not be lost and it will -be always readable by other clients. - -On the client side you can temporary relax this constraint by using the [`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write flag. Using this flag bookies will acknowledge each entry after -writing the entry to SO buffers without waiting for an fsync. -In this case the LastAddConfirmed pointer is not advanced to the writer side neither is updated on the reader's side, this is because **there is some chance to lose the entry**. -Such entries will be still readable using readUnconfirmed() API, but they won't be readable using Long Poll reads or regular read() API. - -In order to get guarantees of durability the writer must use explicitly the [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API which will return only after all the bookies in the ensemble acknowledge the call after -performing an fsync to the disk which is storing the journal. -This way the LastAddConfirmed pointer is advanced on the writer side and it will be eventually available to the readers. - -The *close()* operation on the writer writes on ledger's metadata the current LastAddConfirmed pointer, **it is up to the application to call force() before issuing the close command**. -In case that you never call explicitly [force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers won't be able to access data. - - -```java -BookKeeper bk = ...; -long ledgerId = ...; - -WriteHandle wh = bk.newCreateLedgerOp() - .withDigestType(DigestType.CRC32) - .withPassword(password) - .withEnsembleSize(3) - .withWriteQuorumSize(3) - .withAckQuorumSize(2) - .withWriteFlags(DEFERRED_SYNC) - .execute() // execute the creation op - .get(); // wait for the execution to complete - - -wh.force().get(); // wait for fsync, make data available to readers and to the replicator - -wh.close(); // seal the ledger - -``` diff --git a/site/docs/latest/api/overview.md b/site/docs/latest/api/overview.md deleted file mode 100644 index 3e0adcd61af..00000000000 --- a/site/docs/latest/api/overview.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -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/latest/deployment/dcos.md b/site/docs/latest/deployment/dcos.md deleted file mode 100644 index dab023deb7a..00000000000 --- a/site/docs/latest/deployment/dcos.md +++ /dev/null @@ -1,142 +0,0 @@ ---- -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 <strong>D</strong>ata<strong>C</strong>enter <strong>O</strong>perating <strong>S</strong>ystem) 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 [{{ site.latest_version }}] -Thank you for using bookkeeper. -``` diff --git a/site/docs/latest/deployment/kubernetes.md b/site/docs/latest/deployment/kubernetes.md deleted file mode 100644 index 0f113169edc..00000000000 --- a/site/docs/latest/deployment/kubernetes.md +++ /dev/null @@ -1,181 +0,0 @@ ---- -title: Deploying Apache BookKeeper on Kubernetes -tags: [Kubernetes, Google Container Engine] -logo: img/kubernetes-logo.png ---- - -Apache BookKeeper can be easily deployed in [Kubernetes](https://kubernetes.io/) clusters. The managed clusters on [Google Container Engine](https://cloud.google.com/compute/) is the most convenient way. - -The deployment method shown in this guide relies on [YAML](http://yaml.org/) definitions for Kubernetes [resources](https://kubernetes.io/docs/resources-reference/v1.6/). The [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) subdirectory holds resource definitions for: - -* A three-node ZooKeeper cluster -* A BookKeeper cluster with a bookie runs on each node. - -## Setup on Google Container Engine - -To get started, get source code of [`kubernetes`](https://github.com/apache/bookkeeper/tree/master/deploy/kubernetes) from github by git clone. - -If you'd like to change the number of bookies, or ZooKeeper nodes in your BookKeeper cluster, modify the `replicas` parameter in the `spec` section of the appropriate [`Deployment`](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/) or [`StatefulSet`](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/) resource. - -[Google Container Engine](https://cloud.google.com/container-engine) (GKE) automates the creation and management of Kubernetes clusters in [Google Compute Engine](https://cloud.google.com/compute/) (GCE). - -### Prerequisites - -To get started, you'll need: - -* A Google Cloud Platform account, which you can sign up for at [cloud.google.com](https://cloud.google.com) -* An existing Cloud Platform project -* The [Google Cloud SDK](https://cloud.google.com/sdk/downloads) (in particular the [`gcloud`](https://cloud.google.com/sdk/gcloud/) and [`kubectl`]() tools). - -### Create a new Kubernetes cluster - -You can create a new GKE cluster using the [`container clusters create`](https://cloud.google.com/sdk/gcloud/reference/container/clusters/create) command for `gcloud`. This command enables you to specify the number of nodes in the cluster, the machine types of those nodes, and more. - -As an example, we'll create a new GKE cluster for Kubernetes version [1.6.4](https://github.com/kubernetes/kubernetes/blob/master/CHANGELOG.md#v164) in the [us-central1-a](https://cloud.google.com/compute/docs/regions-zones/regions-zones#available) zone. The cluster will be named `bookkeeper-gke-cluster` and will consist of three VMs, each using two locally attached SSDs and running on [n1-standard-8](https://cloud.google.com/compute/docs/machine-types) machines. These SSDs will be used by Bookie instances, one for the BookKeeper journal and the other for storing the actual data. - -```bash -$ gcloud config set compute/zone us-central1-a -$ gcloud config set project your-project-name -$ gcloud container clusters create bookkeeper-gke-cluster \ - --machine-type=n1-standard-8 \ - --num-nodes=3 \ - --local-ssd-count=2 \ - --enable-kubernetes-alpha -``` - -By default, bookies will run on all the machines that have locally attached SSD disks. In this example, all of those machines will have two SSDs, but you can add different types of machines to the cluster later. You can control which machines host bookie servers using [labels](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels). - -### Dashboard - -You can observe your cluster in the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) by downloading the credentials for your Kubernetes cluster and opening up a proxy to the cluster: - -```bash -$ gcloud container clusters get-credentials bookkeeper-gke-cluster \ - --zone=us-central1-a \ - --project=your-project-name -$ kubectl proxy -``` - -By default, the proxy will be opened on port 8001. Now you can navigate to [localhost:8001/ui](http://localhost:8001/ui) in your browser to access the dashboard. At first your GKE cluster will be empty, but that will change as you begin deploying. - -When you create a cluster, your `kubectl` config in `~/.kube/config` (on MacOS and Linux) will be updated for you, so you probably won't need to change your configuration. Nonetheless, you can ensure that `kubectl` can interact with your cluster by listing the nodes in the cluster: - -```bash -$ kubectl get nodes -``` - -If `kubectl` is working with your cluster, you can proceed to deploy ZooKeeper and Bookies. - -### ZooKeeper - -You *must* deploy ZooKeeper as the first component, as it is a dependency for the others. - -```bash -$ kubectl apply -f zookeeper.yaml -``` - -Wait until all three ZooKeeper server pods are up and have the status `Running`. You can check on the status of the ZooKeeper pods at any time: - -```bash -$ kubectl get pods -l component=zookeeper -NAME READY STATUS RESTARTS AGE -zk-0 1/1 Running 0 18m -zk-1 1/1 Running 0 17m -zk-2 0/1 Running 6 15m -``` - -This step may take several minutes, as Kubernetes needs to download the Docker image on the VMs. - - -If you want to connect to one of the remote zookeeper server, you can use[zk-shell](https://github.com/rgs1/zk_shell), you need to forward a local port to the -remote zookeeper server: - -```bash -$ kubectl port-forward zk-0 2181:2181 -$ zk-shell localhost 2181 -``` - -### Deploy Bookies - -Once ZooKeeper cluster is Running, you can then deploy the bookies. You can deploy the bookies either using a [DaemonSet](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) or a [StatefulSet](https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/). - -> NOTE: _DaemonSet_ vs _StatefulSet_ -> -> A _DaemonSet_ ensures that all (or some) nodes run a pod of bookie instance. As nodes are added to the cluster, bookie pods are added automatically to them. As nodes are removed from the -> cluster, those bookie pods are garbage collected. The bookies deployed in a DaemonSet stores data on the local disks on those nodes. So it doesn't require any external storage for Persistent -> Volumes. -> -> A _StatefulSet_ maintains a sticky identity for the pods that it runs and manages. It provides stable and unique network identifiers, and stable and persistent storage for each pod. The pods -> are not interchangeable, the idenifiers for each pod are maintained across any rescheduling. -> -> Which one to use? A _DaemonSet_ is the easiest way to deploy a bookkeeper cluster, because it doesn't require additional persistent volume provisioner and use local disks. BookKeeper manages -> the data replication. It maintains the best latency property. However, it uses `hostIP` and `hostPort` for communications between pods. In some k8s platform (such as DC/OS), `hostIP` and -> `hostPort` are not well supported. A _StatefulSet_ is only practical when deploying in a cloud environment or any K8S installation that has persistent volumes available. Also be aware, latency -> can be potentially higher when using persistent volumes, because there is usually built-in replication in the persistent volumes. - -```bash -# deploy bookies in a daemon set -$ kubectl apply -f bookkeeper.yaml - -# deploy bookies in a stateful set -$ kubectl apply -f bookkeeper.stateful.yaml -``` - -You can check on the status of the Bookie pods for these components either in the Kubernetes Dashboard or using `kubectl`: - -```bash -$ kubectl get pods -``` - -While all BookKeeper pods is Running, by zk-shell you could find all available bookies under /ledgers/ - -You could also run a [bookkeeper tutorial](https://github.com/ivankelly/bookkeeper-tutorial/) instance, which named as 'dice' here, in this bookkeeper cluster. - -```bash -$kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -``` - -An example output of Dice instance is like this: -```aidl -➜ $ kubectl run -i --tty --attach dice --image=caiok/bookkeeper-tutorial --env ZOOKEEPER_SERVERS="zk-0.zookeeper" -If you don't see a command prompt, try pressing enter. -Value = 1, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 5, epoch = 5, leading -Value = 3, epoch = 5, leading -Value = 2, epoch = 5, leading -Value = 1, epoch = 5, leading -Value = 4, epoch = 5, leading -Value = 2, epoch = 5, leading -``` - -### Un-Deploy - -Delete Demo dice instance - -```bash -$kubectl delete deployment dice -``` - -Delete BookKeeper -```bash -$ kubectl delete -f bookkeeper.yaml -``` - -Delete ZooKeeper -```bash -$ kubectl delete -f zookeeper.yaml -``` - -Delete cluster -```bash -$ gcloud container clusters delete bookkeeper-gke-cluster -``` - - - diff --git a/site/docs/latest/deployment/manual.md b/site/docs/latest/deployment/manual.md deleted file mode 100644 index 89b3b0ec649..00000000000 --- a/site/docs/latest/deployment/manual.md +++ /dev/null @@ -1,58 +0,0 @@ ---- -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. - -## Cluster metadata setup - -Once your ZooKeeper cluster is up and running, there is some metadata that needs to be written to ZooKeeper, so 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 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! - -## Starting up bookies - - - -Before you start up your bookies, you should make sure that all bookie hosts have the correct configuration, then you can start up as many {% pop bookies %} as you'd like to form a cluster by using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: - -```shell -$ bookkeeper-server/bin/bookkeeper bookie -``` - - -### System requirements - -{% include system-requirements.md %} - - -<!-- -## AutoRecovery - -[this guide](../../admin/autorecovery) ---> diff --git a/site/docs/latest/development/codebase.md b/site/docs/latest/development/codebase.md deleted file mode 100644 index 9a83073ea4c..00000000000 --- a/site/docs/latest/development/codebase.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: The BookKeeper codebase ---- diff --git a/site/docs/latest/development/protocol.md b/site/docs/latest/development/protocol.md deleted file mode 100644 index c662bcf186f..00000000000 --- a/site/docs/latest/development/protocol.md +++ /dev/null @@ -1,148 +0,0 @@ ---- -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 | **Q<sub>w</sub>** | The number of nodes each entry is written to. In effect, the max replication for the entry. -Ack quorum size | **Q<sub>a</sub>** | 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 >= Q<sub>w</sub> >= Qa** - -Thus, the ensemble size (**E**) must be larger than the write quorum size (**Q<sub>w</sub>**), which must in turn be larger than the ack quorum size (**Q<sub>a</sub>**). 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 **Q<sub>w</sub>** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Q<sub>w</sub>** in length, and starting at the bookie at index (entryid % **E**). - -For example, in a ledger of **E** = 4, **Q<sub>w</sub>** = 3, and **Q<sub>a</sub>** = 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 **Q<sub>w</sub>** = **E**, 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 **Q<sub>a</sub>**. If **Q<sub>a</sub>** bookies acknowledge an entry, it means it has been fully replicated. - -### Guarantees - -The system can tolerate **Q<sub>a</sub>** – 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 **Q<sub>a</sub>** 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. - -By ensuring we have received a response from at last (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 (**Q<sub>w</sub>** - **Q<sub>a</sub>**) + 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 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/latest/getting-started/concepts.md b/site/docs/latest/getting-started/concepts.md deleted file mode 100644 index 94376bbd435..00000000000 --- a/site/docs/latest/getting-started/concepts.md +++ /dev/null @@ -1,206 +0,0 @@ ---- -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 responsibility 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 - -The BookKeeper metadata store maintains all the metadata of the BookKeeper cluster, such as [ledger](#ledgers) metadata, available bookies, and so on. Currently, BookKeeper uses [ZooKeeper](https://zookeeper.apache.org) for metadata storage. - -## 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. - - -### Hierarchical ledger manager - -> default ledger manager. -> -> The hierarchical ledger manager is able to manage very large numbers of BookKeeper ledgers (> 50,000). - -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 `<path>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). - -### Flat ledger manager - -> deprecated since 4.7.0, not recommand now. - -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). diff --git a/site/docs/latest/getting-started/installation.md b/site/docs/latest/getting-started/installation.md deleted file mode 100644 index dcc4a82a24e..00000000000 --- a/site/docs/latest/getting-started/installation.md +++ /dev/null @@ -1,51 +0,0 @@ ---- -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.8](http://www.oracle.com/technetwork/java/javase/downloads/index.html) or later - -## Download - -You can download Apache BookKeeper releases from one of many [Apache mirrors](https://dlcdn.apache.org/bookkeeper/). - -## Clone - -To build BookKeeper from source, clone the repository, either from the [GitHub mirror]({{ site.github_repo }}): - -```shell -$ git clone {{ site.github_repo}} -``` - -## Build using Gradle - -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 Gradle: - -```shell -$ ./gradlew build -x signDistTar -x test -``` - -To run all the tests: - -```shell -$ ./gradlew test -x signDistTar -``` - -## 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/latest/getting-started/run-locally.md b/site/docs/latest/getting-started/run-locally.md deleted file mode 100644 index 520140317d9..00000000000 --- a/site/docs/latest/getting-started/run-locally.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -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 -$ bin/bookkeeper localbookie 10 -``` - -> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/latest/overview/overview.md b/site/docs/latest/overview/overview.md deleted file mode 100644 index 8aa6971220d..00000000000 --- a/site/docs/latest/overview/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: Apache BookKeeper™ 4.16.0-SNAPSHOT ---- -<!-- -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. ---> - -This documentation is for Apache BookKeeper™ version {{ site.latest_version }}. - -Apache BookKeeper™ is a scalable, fault-tolerant, low-latency storage service optimized for real-time workloads. It offers durability, replication, and strong consistency as essentials for building reliable real-time applications. - -BookKeeper is suitable for a wide variety of use cases, including: - -Use case | Example -:--------|:------- -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | The HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | Twitter [Manhattan](https://blog.twitter.com/engineering/en_us/a/2016/strong-consistency-in-manhattan.html) -[WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (write-ahead logging) | [HerdDB](https://github.com/diennea/herddb) -Message storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Offset/cursor storage | [Apache Pulsar](http://pulsar.incubator.apache.org/docs/latest/getting-started/ConceptsAndArchitecture/#persistent-storage) -Object/[BLOB](https://en.wikipedia.org/wiki/Binary_large_object) storage | Storing snapshots to replicated state machines - -Learn more about Apache BookKeeper™ and what it can do for your organization: - -- [Apache BookKeeper {{ site.latest_version }} Release Notes](../releaseNotes) -- [Java API docs](../../api/javadoc) - -Or start [using](../../getting-started/installation) 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/latest/overview/releaseNotes.md b/site/docs/latest/overview/releaseNotes.md deleted file mode 100644 index e6a1f3a2922..00000000000 --- a/site/docs/latest/overview/releaseNotes.md +++ /dev/null @@ -1,18 +0,0 @@ ---- -title: Apache BookKeeper 4.16.0-SNAPSHOT Release Notes ---- - -Apache BookKeeper {{ site.latest_version }} is still under developement. - -If you want to learn the progress of `{{ site.latest_version }}`, you can do: - -- Track the progress by following the [issues](https://github.com/apache/bookkeeper/issues) on Github. -- Or [subscribe](mailto:dev-subscribe@bookkeeper.apache.org) the [dev@bookkeeper.apache.org](mailto:dev@bookkeeper.apache.org) - to join development discussions, propose new ideas and connect with contributors. -- Or [join us on Slack](https://apachebookkeeper.herokuapp.com/) to connect with Apache BookKeeper committers and contributors. - -### Dependencies Changes - -Here is a list of dependencies changed in 4.16.0-SNAPSHOT: - -### Existing API changes diff --git a/site/docs/latest/overview/releaseNotesTemplate.md b/site/docs/latest/overview/releaseNotesTemplate.md deleted file mode 100644 index cd60fcde8c8..00000000000 --- a/site/docs/latest/overview/releaseNotesTemplate.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -title: Apache BookKeeper 4.16.0-SNAPSHOT Release Notes ---- - -[provide a summary of this release] - -Apache BookKeeper users are encouraged to upgrade to {{ site.latest_version }}. The technical details of this release are summarized -below. - -## Highlights - -[List the highlights] - -## Details - -[list to issues list] - diff --git a/site/docs/latest/reference/cli.md b/site/docs/latest/reference/cli.md deleted file mode 100644 index 8beb36ff071..00000000000 --- a/site/docs/latest/reference/cli.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -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/latest/reference/config.md b/site/docs/latest/reference/config.md deleted file mode 100644 index 8997b6b62f0..00000000000 --- a/site/docs/latest/reference/config.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -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/latest/reference/metrics.md b/site/docs/latest/reference/metrics.md deleted file mode 100644 index 8bd6fe0a165..00000000000 --- a/site/docs/latest/reference/metrics.md +++ /dev/null @@ -1,3 +0,0 @@ ---- -title: BookKeeper metrics reference ---- diff --git a/site/docs/latest/security/overview.md b/site/docs/latest/security/overview.md deleted file mode 100644 index b825776eb67..00000000000 --- a/site/docs/latest/security/overview.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -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: authorization is not yet available in 4.5.0. The Apache BookKeeper community is looking to add 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/latest/security/sasl.md b/site/docs/latest/security/sasl.md deleted file mode 100644 index ffb972a8936..00000000000 --- a/site/docs/latest/security/sasl.md +++ /dev/null @@ -1,202 +0,0 @@ ---- -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. - -#### <a name="notes"></a> 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. - -## <a name="kerberos"></a> 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/latest/security/tls.md b/site/docs/latest/security/tls.md deleted file mode 100644 index b9ecc10dcf4..00000000000 --- a/site/docs/latest/security/tls.md +++ /dev/null @@ -1,212 +0,0 @@ ---- -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) - -## <a name="bookie-keystore"></a> 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. - -<div class="alert alert-success"> -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. -</div> - -## 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -# 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 -tlsProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory -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/latest/security/zookeeper.md b/site/docs/latest/security/zookeeper.md deleted file mode 100644 index e16be69a1d3..00000000000 --- a/site/docs/latest/security/zookeeper.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -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) diff --git a/site/fonts/FontAwesome.otf b/site/fonts/FontAwesome.otf deleted file mode 100644 index 401ec0f36e4f73b8efa40bd6f604fe80d286db70..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 134808 zcmbTed0Z368#p`*x!BDCB%zS7iCT}g-at@1S{090>rJgUas+}vf=M{#z9E1d;RZp( zTk)*csx3XW+FN?rySCrfT6=x96PQ4M&nDV$`+NU*-_Pr^*_qjA=9!u2oM&cT84<A( z(I_8zqEX?C7mo_Fd^ZCjbQDoTmlmy>zXq}B5k!$BD4Vu&?bM+1pscNs?|}TanB=Gw z>T*v6<edn41{(BHvBI`rq3&P)@IKT41u{?i4CGfqeykzcnz3}^LsuYgMTq_>IVvN? z<7If|L2rZi0%KIN{&DZI4@2I75Kod~vRI*C@Lrk$zoRI`^F$Oyi5HuU*7@mriz!*p z<-;A`Xy{#P=sl02_dFc|Je%0lCgxR=#y~GBP(blD-RPP8(7$Z9zY}6%V9+^PV9-}S zeJrBBmiT&{^*|I7AO`uM0Hi@<&?Gbsg`hd;akL06LCaAD+KeKR9vM(F+JQ1r4k|#^ zs1dcJZgd2lM9-ss^cuQ?K0u$NAJA{;Pc%#+ibshkZ%Rq2DJ}Id^(YlWJx)DIMNpAc z5|u*jq{^s9s)OpGj#8(nv(yXJOVn%B73xFkTk0q37wW$hrbawy4?hpJ#{`cMkGUR8 zJl1$@@QCv;d1QK&dhGIO_1Npt2c7Ttc++FR<7<!KJnnipJe;09Pl;!Mr^a)<=OoWy z&oIw$&y}9*JYzgfo>`t1o^76cJ&$`{^t|GE>K)k3GNh{I92zC*(<aYTg*$RGyAE zo41&^iuVj}BTvt>@N#&?yeeKjuZ6dlx1V>2carxUub+37cb#{GcawLQFW@Wr<M|W$ z)A^zNIs8TZmHcP<k$eL`nZJXd#?R*G@r(G?{5pO+zlYz;-_JkIzsP@?{|5gWe}Mlj z{}2AKz(XJwC<Ge8qk?II5W!r*LcuD*dckHvlwi9cRZt)(5mX86f>yy^!4biE!Rvyz z1Ro2&68s>zBluk~A`}Rv!iR*c@Dbr8VURFXxJ0-?Xb@%!i-a}8CSkYmfbf{`wD2Y2 zHQ|TCuZ2Gd?+E`8Iz?iUS~N~HT@)&sEqYwENVHt^j3`EwC^CsML}j8zQLCs&bWn6u zbWZe&=$hzV(PyIXMgJ8IdI`P!y)<59y>wnnyw-WednI|Lc%^yedzE{&dmZ&U;dS2Y zC9k)=KJoh6>nE?fUc)p+Gqf+QqQ}#Z(Ua+EbTA!ChtYHBC+G$AVtOSVNypHsw2f|| z57Ecylk_F}HTnwuKK%v#9sN5!#306#5i&|f&5UPs%mQXL6UD?a$&8iBWb&C3W*5`Q zv@>1IKIR~ElsV0uWu9j)F|RV0nGcyynO~Sc#7N8&dy5s~(c*F9N5zxH)5SV*n0T&u zzW7P;)8bX)2=RLHX7M(0tk@t<5~ql*;tX-NIA2^QwuyI%8^q1xc5#<@ulRuYi1@hp zwD_F(g7_uz8{)Uc?~6Yae=7b${Ehf~@h$Nk@$ce$;z9ASgp!CPGKrr=CDBO6NhV2x zB{L+mB~M7gB}*jBBr7HBBpW4LCDD>N$##iRVwR*yvLv~ZLP@ElQc@#nl(b4ZC3__M zB!?u&Bqt@$NzO|yNnVz`E_qY(w&Z=uhmubvUr4@@d@s2rxg+^qa!)cS8J1E~zSK)9 zk@`rL(f}zd9W5OveN;MGI$f%hhDqm2=Svq!mr7Si*GSh%H%hlkqor}u?NX!EEKQSU zNpq!z(o$)qv_@JlZIZT0cT0Pu`=y7aebQ6Xv(gu&FG^pLz9GFTeMkC%^dspF>6g-P zrT>xsB>hGDhxAYBkaR@mArr`GnN;R0^OLD$8rc}xc-dpJDY770sBD((aoGadV%bvJ z3fUUjI@w0qR#~(xPPScUl$m8|vMgDytWZ`etCZEq>Sax`HrZ}jk8Ho}u&ht^oa~~k zU-p{pitJt4N3t8TFJ<4#{v-QI_KWNf*`Kl@*@(A?x4@hBmU{bo`+2LpHQr;q$9q5K zJ;gi7JIs5Y_Y&_F-p_b%_Kxx1?!Ci1!#mHr)Vtc-?%nR)<9*2cg!eh`7rkHie#`s1 z_YLoFynpom)%#EHVIQ6k<KycS;4{i+g3lBmozEPfaG#|<t9{n{MEb<}B>Px>cKQ_h zRQS~TH2duK+2?cA=d{lYJ}>)R@p;$hBcCsPzVo^5^M}u%FY*=oN_~BO1AIsMPVk-L ztMi@Xo9LSspA==WB&S*uVl4V7bBsZ6Ow%WsQuJUl%vOsv%FNx7`s5UAW~xPRj!Q^N zwi+UnqRjDntAR@;SgfW*vp(6Brq42&k|Pt0u7@erYKn`qB*Yt|l44BpR&$iaU;sM- z4d^4IlC0K*WWCuG6&q_xHzvW8D|?VmP2oxsjM1iyl%%N4$e09kOp@NLPtiwN&H6aA z-eTa;a#fN{F^O?WQSqF~OEH*?dP|xqDK%Li3CQoKxK{5cQ&V=BV@$F7Xc#FxtWojs zXNfkM61h7$%AA;DPB2<y!2X$|;=s5?o04J^tWik`F;;zsHPWC@h&NbW$@GL+s{vlb z8zYnSaU2MwV)RBU)O4lI9H=|FJ=J1Oh|A>qoM4Ov7+011Nf%sPRE(aRk;t@!SiLC) z(4}(2HO9bnN2Nq^J%e^*xrU$#s~$RKF+`d5K(ClYZt5*oeM)3>R7_%elsPso3MS`4 z=E0Mj$&@IdA<!!&&Sd@@@P69W!u_;<yFSJWZKs({?%_R1yE>balxm6OD4U#Myq|K@ z-&JTzbUk*Y0-^+{&H*ME<4mrECC04R8!ZMC(2?u*ebPc5H;tpCU=m%_jxw7~>F%j@ zrQFl$N~Wf`Uvh+X%>u^=z!V8t`pCG{q@?>vOLA0Fl0G9QDJnVY@1Ddb#95Q{QE_nz z(2-1F6PRS~8IxqP=wV8rtMRU$!gLw+F;Pi+V=Q2cGRB&cV@%1(K)mFrc%%OB*-1@# zFgILx%zA6OUJtY}rKE5z#efjS0T1cTZVdO+9M=22Ow*gK34rH*)?hLxWC7zvB>|5{ z#sH12*7O8mIkT%*9G`Hk>dLs;G!k%{O^NzUkTT2tE?TUH)Z}POWNL~_)Z7`ae_Ylj z(7?KJE)jQ&Hb*3o*rWtwBJh@*Xep@{0}KNAUT+2=21z$2x`_$+QVf~#34kTq)f2bC zy5teaYIF&ri#6S?KM*c=&h^$+?f%Ff49eYLDyV~)MBo$Pac=%%%@&IxHZ~dv3zK7v z)+Z&!aB~(1vu4#BfHILT-f*QjQFJ9zQ(O;j%x->){2xR8tH4$FUnM|M7YE+2!8H+| zWQx|On?W8yq%DaSP+~AC(dGnwTuhWj&oP~wvyCRJen%=uy)iDqm|)FJ(pxO9f_SqD zCJAN`7%eq6S|0`S9FuB|F{OY|rnuN6A;l5}g3RfWXkb3jsU|ZpPHK`V$znApB!a$$ zM&b>rphC>h6sWK0Bt38=XbW>{Od`+<It5bqcfvKv@g`F|sNBDW_8)m56|fDc5o@Gt zqFgB$zU#?Nr90)GB2KLuEfy141-M-mT!~cRWw*(2(f>XNK_^W~`uM1%SkU{?CLrT| z*5rU5a4DAt4QsU|SYaF~z_MnbZd3}WFFoi`11Pc7q-YRfpk=(?HFGY!oON*L+>FN= zrpV-2sAV;nKn7Cumed63yhYD(iyLEHoL(PiGR3;=k4uAd$Ws$QzZ>JBRtl%)qmlt( zlrcu1tdC7hu*PwHfTp+Wt<ieD+2ESbxM^|vSW_xYeI)b>ez}SISAlE3{#BBi@~<p6 z+5fBSPPr<%lc3*R4Rc_zdP^ecYnT-HbrIZ}Y>MV=s9VU~oa*A29jU;4uHLv)t`=cj zMkBD=0}Gn;Kx|?3|5QxeB>h7<aanx^^uuMBV@w=XEH1lkNe1L`pqq`!dP6)I%9N;B zP(CgabB5T}5chP+Kk#EL%pe#F&I<s()u$w70tj$4s5edtxpe}Fyjw@Rq%;Qf3kMwu z<|q;L0q|60(2THu=yD5iof$+fA;oG*jdmv@gJuS&q-JGtbCaM48YXb<a6ZBRQk+|0 zF(l}dVjrk>H-63>M1rORUPw)_81!IgVnE<vVH^_+Lhm+&F41&}#Xk$&((%ud|Ct0= z4UUWkYjTn<#$pKu<^nC|)^Xfaaca#11_<~R<Uhd@`9A~jchLhA5g9x!*yRs#&4SV3 zI{3E*<{ERWTvu6=9%K^qzx(NuZq8M3aj;9Xy1EuTJ;-f9xg1C(NFIofo7fMYk-&J7 z!P95xQ_b<wU=u)I%-d5k&8g9e=JaT53@1>33zbVFL~|4d{TmH>B{(ST?=mZBvFKDQ zs6<YIa8Q4jaJ_#}>e71u%5ZNZgM&lh)@6d3N{!aL268{00aWAef0lv1i^_}z`hyP% zyasc1UyCFdAscUwN{$1kE)jexW8Cx^)1woB65NEk+OUEqN;12DT?I)dX#Iaq$3L>1 z0{Z(M#~c61x<xt#Xpj!Ro+;e|;$$`@!~#F3X2il=fDW-FW~P8%N;O-+Y;cwwsBVe_ zZOCbMP@rJ?fiF^xk-?#%oO_;-rjJgIPfJYKCq$dTAi-Z`@QfhW5P^Z+ni5Jv+`WBZ z$zcAt&cTb&SQD7l`v#7i))_O?!0`n4%+-O&;F;6hrox>yK|v7Q!EnR;&(y&k3ik}S zXTlwpYD`!>eg3q#=~2@ogTnwcEEv)N8U~<Y02&6MVE`Hy78x>)gNue|5Zu9Vhq$UQ zm=4KMxM#pU6K(*VJ`HXtpAMkY0d#r@+&Z`cZaTnC2e|2O?BUZ~t%L(~5I_e3bPzxX z0dx>R2LW^tKnFpq!O&_jzy$+bFu(=7JFw8*!oumUh8A)!p+c~``Gq=nX{h@Ft%X3% z5Wo-u7(xI;2v-IbLfjP=0TLY`(Lp;p0M!Ag4nTDPssm6Rfa;(#p#T>OaG?Mf3UHzB z&MfAN0W@?*-1IoE7(i!0*$e=k0iZLWYz8zr1Dc!>3NSJ7geGSI+)RL*32;EO5TIEI z&@2RK76LR20h)yX%|d1ZTo}NG0UQu4Bn;rfLgIqB84nAECszh=Krr33X>d=6I|%Mz zxI^I9!5s?s47g{)9hRo&)&V*omkuiHfLuBtmk!9K19ItrTsk0^ZaOp=1PulO91uze zgwg?_bU-K_5K0Gx(gC4#Kqws$N(Y3}0ikq2C>;pDE*Ri~0WKKefIhllfC~Y*5P%B- zI3SA-$f5(X=zuIbAd3#jq6+~y9l!xibU+gw&_o9`(E&|#KocF%L<cm{0Znv36CKb* z2Q<+EO?06E7YcAd5M3z1g#uh?SX@x(G`In7Mi4v!4hW(Hg6L)d+zfyNg6M!CIv|J+ z2%-am=zt(PAczhKq632HfFL>`hz<y%1A^#)AUYt34hW(P12~W%9Y~N4BuE!FGcE{J zFbG@FOc%2TL93xamQWx|s9TPLX1X{v2-*$>;)DWmLP3;5fv}-Kn^2%lD9|PpXcG#w z2?g4O0&PNpHlaY9P@qjH&?XdU6AH8m1=@rHZ9;)Ip+K8ZpiO9yi^YTHyZbQTB``tr zgIpb(AMAd(*f?muyEF4$ViPofhWp)2_v3ym^WC`x?nk)$vC#ck<a#V{BU<1_w7`vM zfg8~RH=+e@L<`)A7Pt{Da3fmaMzp|<Xn`Bi0ym-sZbS>*h}=pfDBO)G+>I#QjVRoW zDBO)G+>I#QjVRoWDBO)G+>I#QjVRoWDBO)G+>OYsYl7UmCTO7>(Ly((g>FP{jT5xc zjcB18(Ly((g>FO(-G~;t5iN8hTIfc!(2Z!3d+HXsN3_U|XptMyA~&K%?h!3=BU%JB z4s&B!<i@hdjb)J=%OW?HMQ$vM7KOv&#bx70Erf;3s3l<DGDihXn;tr4)cmBRQEvZp z6s$<}<}`imlz(g%P<OZ}zZh+TNPZ2fN6n}Q?MElj7t|PPA~lr?re;vHAv(Vv;`6bT zfihCnR2@X<Pg3WoH>kI%_aQR>IrR<o1NA#~j~epOcsvYo`Bfffk1US@h{PX)Nc@`~ z??N2@JC7eB7C-Fi;py!;#WTn=)N_{SJkN!mPkJu*?DjkXvG-Sa3@?DE;f>=x#+$+m z;mzdD<1ON?aK+rWLd3m{XXDlKF7tlj5kBJc_#(bPKaf9_AIz`iH}m)K`}oiCFYx>M zm-%n=-{;@vV?KeH`Llwpf*3)(AW4u1G4l#RpWvL}qTr5jrf`mMv2dxdS=b@mD?BVb zC463ZN%*qxvhY3O_rhO=4pE>e9OB<mqU{j>P801E<w5+r7oy*1L@$Zn5?%N5@|x$h z)N7+xrPnU6F0VsgFMGZ2^^w;tum5@tdi_O<Xo)MnJ%ye{&!;2kWZF#U(1mmf-9aCv zpQA6(H|ZbgpCOJdXJ$hTdolAg6Tz%!BAFN_fiW@`CX4BY2=-Ct6my>GXWnOSFyAwG zTv6*$;wj=_@l5eN@nZ2Zh*qaSY`R=r4N>V1@qY0M@g?y!@q6OWAO?L){EI{=882BR ziIpTnM7d02lhi{L`JCic$vcvdC7(mg_&<_gB)>zHn1$%@bchNskS>9k@H5g)QoS@! z+A2K_vEG-ZuS?&8IPWLY-yx#=u<TLUWZ4YaTv<58Z&%8;$PyuDn=dPqHOM+-`yfJl zT6SLcqU<unXFryG22ojuY}i}w{iydW?^O_ajrLB0xNDksws(<trFX4&qj#tGDetr1 z*Sx>>zUPB{q&{POCP9RCmd^r+u&(rp@QL@y@~QS|_v!Z8?{m!OIiHIVSH0@<j?Z<U z8$Mt9-1NBzG1UOyvA&_cPx@~3-QsKTwfOGzE$}Vzt@Lg8?e;zC`-<;%-_Lx1_Wj-W zp6`&Hmix;`$;Znl$wTGy<V)od^7Zm<a)aC`x5~5SMe+)Hjl5CbF7J}>lOL9!ke`vC zm%k`~TmGs1M>&>{C?twN#iNRuig}8ainWUMip`2>g+Y;`$W@dm8Wf$1Ud1uRDa8fF z%Zkg2w-oOyK2dzBxT(0M_(gG7NhzgDwQ`Jdsxm}5Tls`?vGQr<TII9K%}RqZS!q?~ zE31?(%3fuk@_FSe%FD`kl^-j=QvRU)MR`~0P>%R{`icA`e!hMW`33q-@SEfp<Tuyv zNx!vzTmAHYNq*^md49!y6@E2-4Sp@k$7npC9*P*xN(1>919`<t@)mxJe<E)F;dSZ} z#Wgx`us?#uFR=YdAI6=~I9K6vWk*RzVSO=vo4oB`Uu@5+%Mv-o0*3;BNe2?NiXOB( zL%v`c<!G8w4%w~jh+Q~h59IYLtKpSjHSJJb`<^9=+kzdmQ*m3kgC+`LG;3u)Vuj;r zCNn1|lM&!xtb{9A%Zy{S3<)MmxFC0v4&-mkVgrY|NrW&ruPConTbfssYcIe9WWXN_ z*oKChx_T|Xj<53>B@V$_Hqg<(g&v8BX9I=vHqtmmC?CQiTI)~<@i|)VblQ3H8$=<H zJMkJCU?;Id5t}$zTtH)mQ0kaNe#SpL=P-g0nwdmvn5}FdBMhP$XOVLpTt&_z2Rt1` zaE*hjhzVvj%r%-s%%Pb+<3R&64@NMXh5j-txVedVignmY#0nbQuPMXnWrD@5c3C2O zO~433X-4n_%p5LYNk1uvtA8o2g*s=hvo>5wV+lKpUN(tkX3=CokeSoksl^f7X+{TA zIF)6dh2AY2%Q6!H89e$99_(Y*(NEJ_CXL1~&@gHZ!{tKhI3Nu-(Ha=IyBUSBv$eHT zgB60#)|^Z&R`8NoCM!ETi&2iFnc+M<ogP#ir)^mA;DY2b93dZmZ%~0Fl%H)Fn0r9O zC_A#-vNX#CTutpVVOCpqhvu4~qpht&OYC@H1<f3$M}MR1q!~OK3wP7Z$Fwuf8HXRC z9V~Y_S$;{Ky{tiPZ@^3FOP3mHW;GoQ6PV572;Ti4rkx#$$Oon!fEyam&a$v2^qSW` zU>aF`j>W($I9M|{Fdn9I0?i2Fo&$U{Z$8c3Z@s||tuw%~3Wi@-Qn;%~T~t_BQle$H z(%4@xz~aD7*k|q?4X(!xeC$IzBLc~&skAbfW@1<yBb>}K{oBs2(=e?$os8k2kr~4h zJ2O0>T)++~{L*NRd_Vq^9U6!SiC8JPP*C~V5;d_4fTOkv@S@>s{2b%v$CGe8J!BW$ zWJe|m8oOG%dsIDzy=8keLkF>xe{|R014mR+Y`{OWCs<;@^T<4GVD_^hV!}nQuYO;{ z5XCB*x<BKAgY(>T4s7O{^guzsd)gfXJQqzy2L25&H1IC#;IT7k4stQAl`4B!EN5{B z%pdSc|Jk$sj4=3m_)QJ7aLt;9j9?+l;Lq7qmdS+Ivq3g^vuWr9Ori3g?wip|f$O8$ zKoR<gy}^DDvu>c7K@j_H<&QM^hJ3>(Z90(msVr_2V938oGun{|A+`@ijA8@%`OHKb zX4RUNno+1Fsm@K#$_0FLSyEoIDzhc4IalLA<WHTmoL^a7SyHUp(PBB++EQEJqH#Qf z^BJYVXf$xlY%m_|1LhTSyw5Rn-$A263;M;O;2m-G);ctRt&|O089^_wl`w*40)@z> zb%1SMvT*GQkdEyv6C56npQmv*NZ^3*=Jo3^6G|OS!ffJ!A0cyp)U<7ESpTewESXBe z$ZR6j5FVLIBA1gywK2K6+Nce~K6us!{FM628+DDZYQJ1{Yuj%-_7@*4Jyh0S(blr7 zQ-nqAuHCuK`7N>MB2OiJDPqjMF*dWAQ9BcC&ID(IiorKn=&gOoj_sZd&SY^p4GIN6 z$ujr8`Q{!onZ=4VG(+JDv?mkDM~vf;4L=7e7Nj%+!^8^nu>vGj-o{J^t(iXu^z1a6 z0mZ>6lSYiTBz1Onc}b2oGRqXbRTVgdg<B6qpUcvew6>MEsSh7)?(We#mOJJ+mOJP0 z(|Qi(A6B=uRoAs@&vhI)^SmmM?<IWRbSAPxQ<sxnldYDTX~)2aGy_yt02*w=TNqmo zOuUV3VQ$b)`)|MkZuWK9Gg+pUPJ5GO1do8AenArf7Cg(6ugTZ=Jbi_AMoeWL5%?Dt za`<cfHK+o|cAF!?VNfx|z7q&x^&GV0$7m;LXaXAAIS0cNpQ7>4jyV%qZQ#(?JiOp< zO{!&p^j-9@LQu~-JXr0BLP+N0wPX}7F42$#vX!5n)@nGY9y%j9*xJ{XrX>k@D<2ov z;k9@ap064LgRzKg!4DG~FhVD&S$f$cv~yq~%`67qSK?$420t)W6Gjt0(Gb6%U_j&E zc%%E!0Zp~w;f&=Ih*)jhQCFX?&9BMdRk$mb@co-hTT9zZMTPrL6hE)Vh1dg|@K!K* zTZoNO{z3a$X(ofl(}7b#UtVCzXvSV&Z`U&KzyA9B4F4p{ELy#Kk(SYcNpULjSf-&I zC$NOGes#q~y9(8uDPS^NbFd%F(Htv)nK+TfCuw38tlM_BUwZ`qLE~4!4&lS}a0Gsy z)i@LaJOb1^3B(c{rnOE5SBkCp2Rcz0O>36T0c(Z(aF&Ay)<n6U4jh^%0BH%<27;2R zWAVf6Dwr50_653fCU7-q3}Pr|nY`xq;tn;ZHSA2FAfLfg*}+*t4g`ev8yYv!AijgM zUIzgr_8PE|cs9#qfFcE20~6tkEVePJEKUMzu<QcO@$_JR4XChu;I=Qn{Bq_cmPwcy z85<j!pPQSvD}xiTteV!mwyG*ymHN=GgSGqXf7^HK?3Ztd3SNDq<ciu6fZxTL_&DPj zgTt^tUJ3IPhR5L|2M>hz3moP-^ynaT#zZENX=Dem$rBj#FkIX-f$24$w)OS~yvH)( z;A7l3<wFr>ngKsZp>)h9ckmtOY_fr@okIf1XkZJh%-n6NwH5?e3U*p|sN8HWU{vQg zCL+RkEEHe`i*@)@mf6%Uu+exiEpRDX8aihIL)OnReaLhgw+fiIp;iYz59ArZ1N^$W z8he9^5ti4N)s@r@Zyem{Z|+Sm1c_1NM_Js=uBDk{aG(Y}0$W-k%aA^j1y>(PYAw(T z+zKnO1%98!@D$>A;fbvRM)^KWHGP|@VZn;bpoa!(Sl4WS1|n(q!%|jb6E0=7PP@Zy zghoFgO>licKEUwAAHdZF*9VMpB6Jp?IRcHA<BKXL8MFkG$x83d#`Go)nMo8!h_?zn zl@DHf`5r)m&T}*DPhf38`vKRunAc#$k3(;pTWno<ExFC<w%j~oCwBk5Tw8i`uIPVW zAT+>dma(6LTQ!$uG!tPgz^r867LH@VA>{RgLukD%WQ6OsZCj^x4qz~8LrOebNhkr? zhA-l$aTnNsJcl$2$S9Iwjw&rKE3POGC>Jna&>Jp23*GpIQ^=f)f@R}>BQhZ34VuY? zuC(OB3vdOMU^W>c_GFn)xdG!Q_8Z-3M%jIh-&wc2wL|T=E9h*@$t=;PE#qgFWaMP2 zop%M91+ATRTE++?hk@I073jMNb_UCs&9<0cGt&Zt&uwAA!5GR1s|QvN61bM;yqFCe zz`4P-q;?feYH=;olG|l#X$fGIj>qtq<V)vc&d12#PyD6hF|M3@f-uj8nj!cH);Rz! zYzV{8Yc6T;P<7O`H#Y-C9tA<3!p?)fl;eI3YP+N!g968!fTOShUh>Nu8Y&vpO-(hm zc5O#vb9>EhY+ptD@9Hhso7N_RG2mP_3t9*N6mMs3^hANHvM2Ut83!nEPIqgioI}Ap z1!jzd;1ZSz<b4e+59<_=vIjm3h*&rbz90AzTsz1`)1V!3=<)k9#Nbc3*#?(30$f_= zC|<-eqcEWd2d;q%^tGMlJSl8<P^{d*;NZ`vmepwO`BgiM^487uFD)u8EvPK0EZkLG zF9J<GfQ|KCRc$rR0X1FxU4oLTlByz2^XBWs$R8FnDtu>)l6Zhy;JQJHyHgbL5aKZA zb(hGdvC@4#?Ry)wjXk9YGCG;OyqzUk>a3l0&3WL4tcPibPC<tc;qZ-k#-muq`JVnn z1?GiBI0lrRIj!KXN6ZAxCxDTY2pBl#6b%O3iosrN;?yB7;#O14uP}8Cu3!%`K&EmA zd>GDuVP>#WUrwqV58>0~87#&v_za1|68Z4FK;8kSI~i6PbuJ&@4!#2{Vqkt@6*CBW zq^@pPT}^!eGrVzlV@XL_NqKPqQ_g}FCW-|#)7xu1ZSDo{#df;4m&vN%*__AV_vnc< ztWQ9f&-r{KOo>#5r5CZsjn6eVW?h8olB$@4yBkiYA0i8Ii+|h6)AqA!ybzBiW646s z&sK&@$s>5K20Z3KVyGY+Z7N$isbziwvcf!l0qZni2*D?ux8bmZ{_kk7Z*FE>ejwv4 zbdHCs&{^n!r=t+A@o*I~+Qz*6`kiWWejWLhq>&kaPQ)SF!4UxyB<#v;-jSl>Gy!K9 z_c!nB>ePHEWR}vf9AoeXS}I(AX~Ua%53qTT!;@|Wis8qh2iyWg3#%=of#GLn7MRT{ zbECO46BI#;)taIiFG#WW?AHQuh+RiB*5<UXJWXk1Q+boRi4nM*2*%;d*5TW9NnSI^ z5vTUxVjQzE7!Sv6SPjd5`wh#qplsu5rXKV-&5UJn2!rnd9Z!>cfVZ=^pjXXMwjsOc zkew0cLXVfj0@@R=uF#&k)P3!ms<pDMqE*9m-%w)<i%;Ww%<A_R8m%Syz*dKZ1)V*m z2h|_E{vK1`&|t5#2RQhS{rvBo*GMMt2^^&arCq|UI$^a+tWE_ppvTPg>3YH}Sa6as z-+zA+GXolCB%%>8a~>xQfqOv4<#Gf8qw+ZQUkE=Sl(6)xtKZdNR{`&U2{nTY%Z=Gy zQU@?kaW+rLjjCYpK2>ky-cG170gvZ*bTZ5S3j(38Pj8ECkL-!*sp+ZT(;%wrtK`(y z01g4q*A56nU{!-dJel_Py5?r>pr_+!zTJ*f@D^OGV%D(a3?88IT_J;)u-qa<F%$uA z3{z2GUSCpQ+)~_D*h;iG!=Hcvfe#A;c88XZni(hv#;~-pIYgk)`_k=Zvj)~VU4^~9 z&}ktNf|En$F|QIK3A{pPt4SPin2&y$mf4&UlaQ!sPOGz6R9mcTqO2N5<>oyN@E#8N z^ERHLWduYvems$BhX*iN))}m0fC1Zjm{SewU=_fC!sS8&%w(Ed<}e?+tO*DVTnibc zjb?5OCxLy>IcnXjVQj0odcr<t+g$Ryb0k(}6Ow3rG7QgP@B-4p&uH!{=vFtk)HE~! z&vB&s=O2Sl!AfE942Bgi*5G`saoDlu39yF>tYOZ@ACHWTkB^Kz9)IrK@#E)UG?-_@ zyb8?I6c$t!s-r5ImuYEjb4^RDid!giOzq+bATcBw*$R$JIHO+5-eYcF4-aNs#yc<I z{FdfCuvX0h@E#Tp&H(Z*$t1@~=zqyL96Fi7;9P)P0nUZn$zVRfUO4h2JXVcmpT(^I z4aSm74@Rtk_SfMcxYlt`5=7R)@~B_HoMr{k8Ca~Zn)kA{Im@2elqJfqZzy519EjEs zh$8VzO3DG&q6FuZnGdWYWV*Z4%2}qOpt_*0D4@8GDdrS+J*T)^ptH@*0q4)VhGXLf z35EzdaRxLi)TBNc%JLk#%#Z=$WKe`y2+9K_f*U)-JAs990+3lN>&Z9}$OTab3Op!K zsi#?r5kN3(ctA*k8KJ|2W*Y1@b#+WBhy@XXJaSCQxr>XI5JASqMq`;Kld-bAz#$00 ztpcFt_QsBe-J-5)tZZ$AWh9Fys_?{Bn4R>8<~U#wLVSWzwKg=i)@Xj{dgtn?uS85y zNkc=G_ASRGep6Lr12>{F&gJADOr+tAHu+dj#*69~_v}8z2!d$r2jgt0YpT~ab=W(b zJ47G74Bb=05~M-RRIo}0>@4_3J@h$l%(1K^1eme4Lj_D}-_=l8r>SE?z=CZ86S8e& zIUj#3z}tqF^W95v5&=;zj_qMSouCH^rw1L}n$iK99dvpj=Sq}-Dj0CFsFSua$FYND zPO;olnE~&00?SOH$8oJ(gUJSmPspUu-~}@~tUIj*+5$_hX?G^01!GoJsIuU3WGsOG zeQ|v1iw{E-Ah;}8oko^b*A#PdasuQbgi|n#U^C0)=GoF(@|bS?1w>+UwkN0(S{Y$D zjA$O7#}Jli^7AV*8gm0cg@;4M8|<=lUq&}-bjUY<-uw33dw(+NiCU5+%q}j@)-ak$ zV^=|)i7GM?C@UchsS@NB+89kuQDJqV8u;ga?>H6f4(GwZl=v*SS`x%#fq>y#dXDBC zQ-e)v&&jOPGW^b}cJMHP-VQ#;_zG|&m|oztI3heD0H^c?u<jhsl84D6PDmpdE|SDJ z72rQS`s}&BXzif{>uv@gfh7oFhvfqi-60R*koEXQCOtVrdnj{zmqE>_i9bPb`GX62 z%G49LQ6IZ8mJvQn#{n`8INIQ-m<ueKU~49j@eFtuoJN3kJmBUxH@8C%3D^n<b1~g% zpz7c=5QN}&6oQXNP!U3v2)-OatqAR-&|yTC{pC^imuJ~&kC)+F7XZ~E6o$~N2;D<e z1ELxc)r_cCM71NT6H#4=I*q93;QJip5rjORLLT2DPZ9E*i9BnN=T*cDK)jiVSBQA; zBK~N^Z%6#^k-!HDYLVa_BoZJ|F7gr~FBS573VGc{^m0VkBKix&JOQC2#C(awEl7L| zNghLzV@SFHNmn6hJ(6W3@0rL4{$?YeBIKKdeDjd+Z%DocDRfAwLCQYlSA!l(Mh~^3 zhc2RrenbxqA^(ZUe>3v0MgE_nfH^4OB@{rAN`_R8NF9v=C!@fh5W57ik%-Mi>^{T} zAofqh{)IFXkmhluc?M}pk>(20Qb_wa(#9a|5E``xjrtsoo`yz$h{jApW459(SJ1=L z(8JwmtQd{mfyRE0#@<HbJkhvZG+u|s-$WDUqX|EwM+NB7di2;*H1Q!cu^&y+ph?MS z(jheI8#K8dP4PujrlTp%Xv*hk$_Sde3QfI(rU}us8Z<o&1zkr$1cj)PZU@p;BHf=T zbPfvbM>D3Q85wBC1vJxu!iLbSwP<z*n!}<w5ok^UnkztakE6MR=!p{a#CbGtG@ADj zn!g>*{{<~*LE-IaVGUYz04?rEOYWd2m!c<6qo?@jsR*<}jaD?G6<?#3W6;VBv`UUv zDbcEj(5e8m>O-_<GFp>{*1Urv_MvB%pml+0-2t@jI9m56dX`1&r=tz)(Z<)&rip0N z%V={r+TxA2^rJ0KwAGFxC!)wO6uAUNnowi|iu?dYeupA|N0EP_ZFMNhA4M%e(V-~% zB^3P~idltXE~D59DE0=@uRw82P+SL!yMy8%NAaH_Lpd_MixMWIgnX3n9ojw$ZNGsM z(^1kml+=onXQ1RRl>7!t{uLR=BI9giT#1Y^$XJYwmyq!-Wc&=7#voHYGQEaUSd=mz zr96&O)}tL1+CifoImrAJGS?%^Ok|mbEOU^h8d<(XmLX)VM5&c1Z4OF*3Z)xR`T)vU zf->GgnWIo<5y~2mc7~#zsc7f(C|irN3sLq*DCb3#%SX9wDEBv%>qL3aq5N=^-+}T! zK?OdjU^yx%K?S!^VHhg%Mn&PMC>s^EqoT8@I0zNjppu!WWF0Emg-U)!rK?bBIV$r) zWihDiYgDd4<!e!SJu3eeRlJNUS0bAn*-oIUr%=_~sQMPFxruf?jCN(CT~|@<L#Xy? zR9lB?&!O5cQC%mh--hhZq2>V8{4#1uMy)hzZ9r`lYF~xgO{l#ab@ZdokJ0YwXm=&r zeFJqphPpCP*Bhw27InXa_PmAmhoA#-=-?D|$P*oU5*_*o9af{m&!8il(UITK(dp>u zPw3bW==d&l!UvtWicU^IC&SUnbae7CI{7?0wF#XXM5mucr@PUa{ph)JbXJ7UJ%Y}) zq32oj{2g>Y8l8U^z3?`=a2#EnjV^wUE-BEZqv*w@sDCGV`8;}c3VPiez21r5SdHE| zhAzjU%YEp|W9Z5!=*=tWYCF2tjNYn1Z&#tWucCJX&^y`a-EHXIBj|&T=z~r)@CX`s z1%0>_efSdkh(aIzf<C^0KGC30K0%+Rpc`T6Gk^5iH|X<)=nEP8Dhz#HiN0~5Z-&vg zZ=su)(09G)yT8!)@#y>K(Dxss|NMo1u%aJ6M?c1+A06nYN$97~(e0z?XMgl_8M?Cr z-T4;%`ULv*F8b{&^t%cDu?78CgYHg8gHebqrBFBpTm7Eh6pu&oj!^t*6#son@FgXT zr-U~tQ3WOHr9@v*USlbUQ`6s4%nFKWqQotfWHBY3LU{*JJ_5=olk(j``F=<#Kc)Oa zD8KKhhlVKsbCjxyQct7;HB{hoDzJ@W=TMpwO1q01b(R|aI5qkkYRqhEjDZ^SCH1hJ zdbo-j8%>Rir^YX&#@A631k{9TYQkx1!e`WkFQ^G$QI7;tk6fZ2y+l1WhI(u-HL;PJ z_$4*z32IUbHR&uhc`-Hl87ky)D&!!g%cXR`QK3RAl%+z0snEx%&{}GS7d3MX71lz9 zy-m%UOwC?Q&Hj;^6GqJ;)Z7Ww+|AV7R%-4`)Z>2C6C0>`YpD6}Q420m3l-F&`PAYo z)RIc-$w#Osd#I=Q)KkgSvL)2hfz;EVP|LScD>hOqFHx&9sMYhRHBxHrIBIPYwe~M+ z-4W{9)71J|)cQ5l`hC>;@2CwTYQq+4!w1yHd}`y%)TW8lCL^`!3bi?w+FVC%iKn)1 zptk-%MFvrkH>qtpYTGp`Y7Z6l3l+0~iuI&oXH&7yQn6`NY&)eNO~v_BaX(P;CMy1I z%CLemyh0@;QrqWI+drieuTx21P|1aqv5PWwQz=erhk-KJQr7cSY9f`kfl7~~GJdAA z)=@jnRCXbiGnL8}P`S@jc|}ydlPWkt6+c52S5w6!RB0+zrlraiRK=TA<tC~UQ?{#A zRV!63p{lJ^jUQFBo7%OM+I5?%JwesyQuXgs^`BAo-&6I!QuTvW!x73}OEq4hnjfN? zgQ@1FRP#Zq<zcF&j%w+n+FqsFcTpWNROesR?jNYGcc?vMsXd3Np4n9IRjT(bs`nkL z_dTlj1FCm`>ivl7{e<ehLG^x4^?ph9eogg$OZ9$7_5Nr0FIc|>^0k;pVIJl=A~4Sr zmb^S=Ab*r20=5#I5klDC;VB10R?)*D;Aab@fkPikN5!xh;yZTFK>k%nmXhqoQ!w0D z`nqozt^_Q@9)>G(x>pzi$Zj&3k1q>vKz!ymnp_qF<m457bJ7ee0+taE!;0=XR}2Ao zH<<#EQ&t$#=Jc%yxJEqsR@_Lt(g!c_WInl!`)OzcpeA22-01GR5^xQB^j%52v7#?P zO8)9j!_}-a#KFbyNW?XW5j#WNkxgzh+5x#QAbK<(Qs9d(r=`PrHasvm4L?u8s(HY3 zFajS{?#b*-)sSLAYG!7tmJ|z9J2UraaIs)dXXhR*esAz|CC((p_zAL|Y$KCG$P$7y z`I);?J96<%;^SZ1&`@qyH#OHaHfZ1YaPQu|9ip~9XIh_Ee?G8b?Xtw}j~leXF+}_@ zk*Kx`Q(F^UZTEES+oQD!51x4Mjboah`mu5VuTe?6|8bGO9CqkqHN@=Z1#xlP`82%6 zAv6U$Krr`$y@CA2%8SeGaT+cj8n?V~qp`=>m9B;FD#iR^J1oBn=phB{wUU8ByI>H$ zx8!$q^&C71XwoQrfyNoM=PID%C?&UCEhwxkFVqYV5Ia96*Ay3}8rg(L(}Np?fUSV< zJO&x*C>!j`DNaJG(1B7|a?Yb+Ls8lddmB)K6#yE|o@S4<!?xSJe0tC>?6&lz_NK%B zkq5-McvwqBqNhLl@$vtvtKdW3|Ni*N)sM7Ti$$=S=i!I3M{ifpp6<P(VY_hXI4?Ae zcrIP9A*wM%HJSvB7HwNMaq1#<rJz!<XyOlN*J(+>J)(lYyQ1kItoa2CREud1?qW}t zM4Dkg^u(WZ_eR(ZM4m(7XDhLZ?W2K;DP&7Sv38K>`~~8??IrDMDYinNha}2FiOrT> z8fWDINp)=E?=H;RV^ycIj%P?dzqq-zv{ikudG9{VMbCj6I~)g<*PUTb3Et$Cl1&4S zF!Bbz<e2p)UhVM3tDU}%S+Ks_arysCt3CUnRcGEth!A*Iu$Wci_NDkW(x$=fG-+E( zUL);f{B6QuFYVw@0_=4gGk83lEf`{yQ7nY!r1()-RAjr?t#;ThaVVCh!<ul|juFir zL&?VoA{&8(dpbEPApmhI(z&Q&2)CCt2**Vr?g`rif_YczQ6N;`1N9tsWdmDE^BQ1- z->G<zmg4;od&Flpu!9l2VA$~?0~^?31u&?Kk071^du*_?0K4+*&Ti|6O&S5Wv99F+ zPtz{I)4)1V_5W%KhW;@`rXJLWeF=zGL$e!TUjFYlP=E8kVWr)zfxKP{O9BN_5RS5; ztx5<FW_D^wAP&SH^zhi>apVPj0g@yT%AR8J2pNGeYam|7_VzY*!nqQF95f6X_??}N zy}c^XE;S%19?&dkI$yl~L4z+~*L5H4Us%Ws+y(Fdhs9L_Wq|Ns$Xsne`9HBgz|0BS zI@STA#{FWu!U-$<>onnZrtTk~;dZTr?qf9E#+Bd{t+{3f-o#en<FJ8pWFB|+Me7n- zcT1emD!z8L<8*~I;Q)?f0IK{A%WYA@GN02rKhla6*KQsz1F^M|XYqK=+OxCrqt<L& z+dS*lC=vF1^ds^p2~}NXxeYbg6~&<+;YYFG9nGuHHGjVC)u`8IH?7$wBF~cXU_-2= zOogwrconM|1_o5Vzxwq>+%_)cTwCLKgmtMA7k=EzdSd(S4Zx%j-keF30X!bM3MnU- z8j66_NCc!Hx&=wlHNVnQJ)A2URP3aIH7R9BUVB!JhAcZ!a5U#=){%f?FPu1c?7XP9 zzNX%;g3X%JI!)9Yi{4y!QB+r42wTR5h2^k^M8=FVwk0<wH3pdQ)BKK#oh_MaVuk|} zc{|IrJ6D!0EnbmSkd$l8CYnF}>x#IF2}DiCZ?|Z<EwVxt9IB}+)|}XSeCx4IJCk#h z3lhl+lYe%3US_t6n1xx*J3GoXcv(tgLhJU9#DIj&aghmIM5V>$P`9YMsJ2-1-0Jt2 z_iqvv*W1hNYCD9#;9S?}KM!Uf$~#;TaDY6`&#G?E?Nnnk?C&(U@6xtku6wKg%HhVt zEeG4Mh9EFTT+L%xjVB!0tF3bl7)na&HF3|!pG&ydez5sa(-FM<fPZ^Ub7vdJ<jGw4 zh|}OFT)PaAxCa70ct+U)0SgG8{O_S4k0>{#m`cG+2uf29T+j|ZIiwhQQaBtkbmc4h zV*1L{>(re1uZ-E4u3bcC^U0g<up>_kh{yHmH{o!S;O6yP*aK?eR8GlIrLf!WX=NQ} zl-0KC%4&`Cy2I$a?lkf%Dk~~fPAeR#xB?(fU;`Fg9OsoyEfw9lO~izk`a33NvE*4H zDaYHQ`j*(D3<1M2&fB^96=_Ym0dLN)Eomrgs0^@IHq_MD4nFDl(0}kr=ZE~#y84O+ z*T#55Rl}~@x;H=cmzD$PU^(bJoKBC1kexsZf?x%YLg6^$J~snT1>~(@NrtTWEt=dV zRujbWz^k~ed>8_3pfCq;1O%)v1quT_hi*GgD0fz6=Vhx&xga~cxxGreOSl(62#Z(X zA$BiBT+4)mHfOx@bpGk=;~J-K=pethAZ1UAn*0C&Z6t!9S(Tdu{5MOGncLb~rEP=Q zA4JN25TvA}nhUf}-N-?Hc6@$JjLO&$c~UbNA;^NWaaGzbFvNhS7h358Tb@~!1DmVx z_GH7kgD!P2M1wlDgH!Yx?Ti(0x{x0qw<&$Sdi|!Z<8fM|#({jN9*5Fk5_<})?K|KU zmm@-em$A+WVi)4C;e?7a!XImBM}#9{cW3Q^g1rIK4463J7MLW(%%QuEyEkF00SI&# ztib=vkwqK_V2*(>_Fql>G5CnGwz<5euo0wxz#mR_)WCtYqVkerExAsv^Gk}k5axK; zxQifne+6VXLfF#W&|Iq}e>l3s*zU9;pvZUhPy=xAB$!U%%Sjj>?+L1FtLmz2vB6R7 zKe%3i4bI}~(yEf`(g3_6S$RCaKj)Z+6gn>QkLJYeGpK>p4KX{m=V(cx^CCYdA%9)G z%9#ec&S$|3=!WwSJ$c>fO&aGJJdn|Bwx#C>r03<OSyiCe@Y8HlO<kiF*k^#>)dc5? zAQ0>a{PHX8IojnXR?+w>n0uP|5v4zdlM-a@4YEOv+h{nRk@Oqv3y#+|w%B&(H3302 zFb9P-psFeh%SwwyME)q55Ke;Ccr1+{!rmJ~ZfWK3!4VwLFF=?C4hb%2<tGC9KRB$) zj+{1bCsmMLP?%Gy&5Fs>TVh3I(i9Rll`K}nIa8lYHz#W$V$QxpPX|K7v9$=H{JrZm zcO;b$JTV5ZejGomcJT4@usihU*V?LTTTQj97t{otb%O!$v5Jf#Yd<CH)lNRSKq?8= zX)A2y6{@V(Y}m@$Bxq~2x7KPqPB)+2drE{S{fNVUz>C#@z-MFdPg<_)c3024Z7yxZ zX{0cYR~4RM2kwqx@c?f$?fNN&-YH+?3Lg9@h7}K-&Vd2f-t!U`HWFZyYv51X39AI~ zBX9(T6FB=2;R#CsyAn7C`_jOmcwiy~)DvNo8CR06cq{ZBo^VydlqG%zmI)R-aLjT5 z$dyKK>5V>R)dUhLoL@E5fxJJ2r+RwNoQHE^{mbI%NHP~hYPvefSlepSzD2Y|_7Y@a zY9_B;Mtrq9a*a8bouZ7Kyex}qI7>K%ZEmcoYtnoOJ5IB&!x3QPO*ozPv>IsY^U4*> z*B)%^X+5Emg1U4M0T>=S!tD|Oe|w&02Q^B^RHqOA)%h%3KIB*DR6=!)KK+QMYa?F1 zolmHPzs$mnI&mQlCiH1I%`|c5y19|sCC&VdHw&)4qr$J?mv9HZ1=mZYgS_%&!Lp3y znk9MsPa|jcPgEZfcCbf;nEB;%OdZtX<wv_N@=yQLbq`Z2=}6uY=oDoew0oLasq-&a zjx#2Zb%6F%{<`AM<P(}*MTJ#GYJ<@fuYCfGx*^OjA)@XlRPhH*eGT@iT@9LBC$Iec zD-1G_za*W41|sSIUkzY5m=j5)cr{sd1K(v$?HN6J+Wk4bd3!TOzl6UIdz4ZYk*+4f zN5VChOmkjJj>wv~GsC3X${ug9SJyOXFjR#4I8w#6b(t)~he;onKx4+XoqKb%twrsn zZAAyN4`l6wgH|(%<Y%&$Y*UeKkHHhNnjIOYEK_!N(azGGB9SF0d#6z~g|)le<{E+q z@nz2H+JN!p7*;0OZaWT6*#GmAf8_x4r^L5t!H|N-Jz&M=%8uAa<VuGee0aGK{P_kY zz`+?qhr-XD7*-Iwau{S{5KgsSaM$71XHbU~$Vd6ZA<;nv&y}g2mdrW<5{hSnk%Q+~ z4hIFEmtfeAuYh0gYGlw3itc_|3Oj>)(tK@K4<EUD82cXn>CK-GAA#%E)mvA&e}}LB zbPKXq<#~VgU-fe&x{oiW!Qm^{3D50t!n3=}wnu%nO4-cj7ufO(*=D<~Nqwt<ZfdkO zs`2y&y4j9jq0`@F_jK91)Y$j_o2&nO%DWM|4{c}V8~XEmcH6q4s-5j|SFv+-p~Uxo z;ugGdfvj$&6O(L7g{(Xu^&dHGJFLFiO1rxuU;G4eccoJ4NU%A!I}%iX_dCJ4UFB+9 zngt7&N3&px{BRjN967j^|Kj?`PA^o06aK-^G3#H{4n_`d<@Z=xxxKNR?N+PCx!t*) zZ#cHk@{;=4(bk^5T1TEEA3qBVYVf@jcnGW*#9=Oe1?!ghp|i^DuC!LGy(-g|UXnrj zM@;_l`P<Us)FV>`5sRB&PuCXhsj@dTi<<52H7)AFK>?QUJBFvcpvC)#G_5a`ys+bV zK%Y6Pd$W4DT9B1hT9&1)sv+{@MTCu79<bADNiWMN&uq-9$SPSEGs&_&J3Y0a*jlo) z(OQvKy4etvvZWw1Bd0V&RGQV8TAGpnY{IPM4LeiPvdSzKdG_4Oot2id^osNxy)CkR ztLWhqn40ywdT6TPo11Tc-K)h1pcChdUoamJ7`a&SGdq*me+J!<HztsOi?YEAH;`8a zN1v<NQC!quP=WtLbH^aK4Oh^y+@wN?0sM>+c&8kM9}+SLzF>e;nb^MU4(oR}<l8Zt zW!Nt0Pae>p)R0Md691%r!J&2P;SdP_oLMFu6B05;>kLWc4)lfKS#W5?wI%|hoq`hu zfx>*xp@_k|@M(qn0}BG5U2uozAAEj+p&UwrwSy6k5G4?GJvc;fo9Di~NbR%>7R`O; zDYJGxI8E>dA7Mun!eUxuWd+Mv?U2Gj!*NnrXHTVJbU#n}+OZll+_5Y9iNS;+y;7d? z0U39NOnr$=5>;koRA#6jd8DT55v}v3;fIx1->hl6s;zGAs%wRSh*vrmsjKW&cDt&} zw!3n-W=#W`Q1glE<Zrlb-|HI3x(a@KG<5S)yee*8aSEK~eaOTw-%+_^hiVttj)$7y z2foSKiREg06&yL)T@IIKbLZ5$?fY_?at;-@Z}{Et=hpv<<R_lM3m4-h^WdOwl*8%| z$0*?vtrN@>kfXx}Qs8t(5j3uAvN51y4j&X3@w_#tyW_a<hcC9^_Z>0#W72@XmpdFU zwJ9yH+wscx?pEEqr)oTK)^<n@M><tS<gp*MN7pqhU9N(o9FI&R8Jjk>?2gpr4CX53 zcPo2r+|^&z-!C2~cl=iL+i$A+vuEqhsqt()|4CRs?j#ddlj!)ks=9cs^W=y`S&tXv zr`qw7n>R~ts_}XJHWt7kx;Qcy=3~uSSTJ3~f$!iYD%?V<F(PRi`Q5Ess<x_00V&KX zBZb;sGx>7I(K0-txXmcqySZXyRjTUA+J_CRG|P7^tz5RVVzNI33P*p{0cvi@F5gCc zd9^pcZTn6w?|%2a%F6e&m9M>#@!Fp5nmy`<qjg8Ck1lUqUcaKXveUopxr=yH<Ar|Q zcnvpSz|D=X)xNRk<#+cFG`+XC;b8rtU57*^qemBvR)fp*P-7iYYBR=OE_kQmmE-{V zHnXs1HZhJQ<Sr>T<i}Z!E0Y!%ElU^4yR$<1usW9EU0@n_k;z~+ys;0QoMRmRxM{E% zj=J{+3f=?#K6}Xinc!UOfxZ;2{2k(pLsR-9Z7C3m^(CQvLrSaZocgm7`)omEj>)iJ zi=lMC;hb$h#99HCFYoKypK~Bm9XMDJ$omVwLyP3QFYmJ9%@>Y}x)1)@aYEgJAF9c2 z)i&ppg=eaWmym3&;~XW`(=}vo>PGl*;8;06R*8>k<b9RT?|Py3oT%yhpUVEDKHO8g zqgz{Du@@(^r}}U$%h#t^?J;m9O%T4YDja{uN|V8z{uV1C{=hgoah>Pqf&4t^!sXg3 zyyb<%qV~NwZ_jfNI?$F?O!A_$YqN7y!S&8$^IAY1T7g3=@eIwg!b&{JjXj_hEbf?M zEK@gLs48#JHgOB#!m5g1=*G$8(2d;8w4Btc06Xa<-6f<G`Vq)$4kQ7}E$yQ+wIg;x zXZxR93~-2<NlVdW=jCUmtJ0gY+VV7M`@vsl`qbxl{@JO84_Y#Jjym6m12EIQU(?py z)X}Bt%J0Z-)^w*3C^V?0$zY)I8t(k%90+Gk<BoYlZU}O#;J^C2{|-J?D5VqNqexeN zL@@`f0|yF$I2%}Ls4)y>g9;ABVdud~@CVJga}S!k|L*VRApay+;r@@byUz821q4~J zRS758;d>ePZy(nsI9jUgbCvnt|COeLwHvZ3H`A^ILubet?!ZuCk*cVsu&zYI9sA)v zGJ-=ekJDBN!^g7eup%3bP`Z!i!?_^tiz8UTLA=U2kV(7FZo5idXSW0S-A-#P3w{Nj z#x1Ip`*!wN8(l|0ir~;uNp7CjIl(!ekHdtIfqrddhhbmhzSf3??|2r^5;`V0C-8G2 zp!+swo#B{R1cZqcz)f(j2>j7O#ZZKi9kN3h(-{K00(PezY(t3a>=TKwvclWo?6?j! zLbP4j$>Kxc+4nnyU_25bKx%^ssc<cKjQW0vG?|+~g(v>YZxnb-e+vHdADl<>_>P5x zpDIf#N=i#L&Qs1){L)g$sB;VLEp^p(wY6HuDaR>(Z7pQfE%w4(?KAKd+3>*d0H5oW zaByI7fRDQ{d__>kl02Nt-)q_4nxIbDo@23U$t)7a?PuUwaDneIoL36}2_&4tfiFUa zAn?UGti?3u(<|zq-WQ>9P{VEf$gcA#7t|Nd??2bAb)dmE{=Qf0uU=8XY8@)wR>FsN zBLfiN2Ty$z&FzfXNgk*?ya#4VzDi!pZ9pg?WGC<LZP?q~rD^Hvf4BP`QBzH0ZG*a@ zu{6I%+x=w!l9qH)b9#fttdg(dyv^T|Z>|4Kv;H%(9q*lmdqijRqPr8-i7{#0a<#Ka z5A34sT|ZkS-?m|P(&X__ha89P75E+j!zU9`_u}vNP>7p&4*P8`_~JPv#&?x#Z%=$x z0Jaepk7N=bf8zK}X)mnIE-WN}kU#tj3$rT=?S=NLHaPY82mZs~Zf~oy7m7Y}{zutT z)Rb4N$*aw+C@5IA%paJys7M9+aXkw`skXL?vNq5S%{6xW#f$#%HDzN(Q$=I3y>OSP zBQB;P24VoK*@;6T%HfdV5IzCM6%K|BhVbz;JWYAxgze3^6Pz33A9rH8EiP{ARDVt& ze)xgU1z#1V^kEjq555e8fJoOlWlN#ED>-F_g*&q|bJGh&`6b2qc`BH$^(^KI>T0X2 zYqckPp6|K@8%Z@yE$yn#?AHIo*qgvNRqXBKAkAX*;*td0q&cU`A_^i%0XJ5GB4sD+ zTiIy~rL^h3rEQvKY11T4_kE*4Tb5E4WZwiS2x8q)@hYHl-79m_N%8kgTD;!(zVGM% zH_{|0=ggTi=giD^d7ftyIjhwQxcS3R(fs)ulJ3q{k{2{UIQbT(B{>tp<ujWyd7YPD zLA5kc(j734UZB-<CG43PXF8whC@>bN^YU_X^7vwhtHfNgl_b`YXRm)J{q|E5@CJ!g zqd#cHJIZvm>6|Iw1xR~&nWMOfhfi_;Qix(^97Aj)aHo)eB0q#H`mMKdbF;H^vRQ=2 zVBmv;+4#Vk*eU5@l*vE&JE!cgMz`2(7MnVsF%yp-?P++w|7v-X<UXQoYO)>+Z(?wB z-|(ho*6{Fdb+_7=mXWfauYL@R9v*I8))ek1Oz})<3O{CTYVvcRcApmYC*Nz_E(~^$ zU|>Zo0g)MC>L1gzAaWu@9)-GGxE>E)aEz{EsPn)r19p)FYIyX81`QdH4=8}eMqssG zKt5B9(1>>n`XOm!@tl5Ln;C+#%^Q^l^1Zruv%mNQQm=6@C$X9~_U5k%z%Qh~zgP@= zf8qV#7|8q=jh`EDqWY*R*It!(U)Wpz{^Cbrw~Eq`h1eqeq1;n$ZQNS!-*wd;>$|l) zDtU{Fe5u(|pS-7>Llm54^d@bVd0by(#215ydrtv#`~HSdS??add23-sB}j>^dpU_i z)o{WWG=7XhBkEz$V7tGJT?ZmnuKWA7vEBVKTwptE)qaPlMA^oo@F=7|O%asHB0bQr zL^!34igLy6RU;+0*Hu*?#j}#raf#{v^dHJka0F;f@C*j~i)ZyEBf6^L8sz)?e83)T zib2jdUDKV|o#^|E#?9V(Xh&@H^TiIHMxoJHz#q~55^kb^uG{XX+2P%Z?nE4pA@gM% zE;M=?eLeVt_9fWVAamn)*s==J0r#r|L%H`I=RZmGGWI}-BQ?155^{-Q_FUpE>~WER zfyj83q@x|f<#GgI*ulLAbz`R<9ws@3$D?<rjhIDJgcp>FhQzcqZqz7IT3RC6rJ=8r z*C}53n#6Fmi40de>LwDBhH?;3oQ!xvy!#OBQ)FOl6lXa$-n<FmV7U{pc>`ectPr*v zko3-Sb$L14c5{@dD9xFes7f>>;gswwY&W(sDNzLyL@esgShSB@J2moZf02*-O+qxD zgPwz|a;Qy`w>C(P-NUJSh%oHbw{DWzG7?K;h2g?5e7wa@<iLqQx*2;`Bfkowb}L$| z-K`}q#9ditv4?K0z0cYvC%n-%mm6-?@PYFuhHxt*BbMuc(gyo#pX6Wv`tn*6ccF1- z!wJ<}^V)(phoM4Td`eux%DK|atn};*wZV{^XXL5xፈ|NFLGVp8C|9vt}4Pu%W z(eYqV+^H5<h??jSAk)FG)R90`$A#*=jIbejM|KDCzTPSYhM}X}A$K$wFsY(T@jq%D z$ZCvvS>XvpnGEm>>I`mp3k<RlP&UMP4GNKxzBL-!S3(E9$>^LRWDvH1T?jtan@DV9 z6B+cTl=jWjkiHT!D1_j!H|Zd3c@<m4<TlZ6meGOY<s=*~%y=4pR7wV-#DNkTz8vAi z@yltrln%ts1I0&aIQAG%!k1$U3LGdQ;YSIMI!eNER4Ag-U0(>Rl)q{aGS>LAfbOpv zKRSdAA!3;yTFATI`*{c*atr;zyNPPpM{M~62e22_;1iA#k#G`>6bB1-=eswvzBTw) z*0UOEqc44$JdOT5crfc%NOLyGgqMYvMdZmBaRfS-uIp2wzYL>Rfcpt0Jq_p242pl> z!OdsJaBibJOLTf{(-7KMbuWpYP%ivB>{rrHMNWZcWd?(%-)~{_zvhH3o)t=AJSeU| zGO{a3uRnUmdnSPN`XeK~{wPe~py3c4*S8(vSD+aXGq|$){A*k{V!4OOVNqRONpp(| z^nmC(ZqkRar^0*fsc62N@8(205-SU<)p2gVJAho4ee|)YuJ-;BwH!T6-WDNu^1-3= zSNNXuU>rV)D>{j+LQ86MbS>A-yZQTeT6juyG(TyQC|XB;(1g|LIC7Z2Eka#hTRk_3 z4IM#;=6=9ZHS{n&EQ)65u8ZbAnk3TIHG!*zz>wQpT3s<J&7z@`;`*NS7Z1QZ5iGpR ztit<eD`r+<Dy%|W1@A2>yr-n-TJnUZu9im%`Y_HcdF}k_D~uF=<@})!5YYhonVs3Y zQyu@&N21!gk|uVpN&cetzs?2A9p{>aU+>$WI@q7M!)T0NG!HYuk--+#>Uu3yT{J%# zSMI&0p7s>!*lBt$Du7w6z=;4~fYCOrUlNOZ?b9&!<OOhqkjkdlJ;UeIA1TkTeNvA| zt&fTZyibGAQUzD2D-17<m8LtLh7xsEQCVeKc7>&kH?^7D+El_0vhPdbHBfaiYJY$^ zPrx*ddC;9L=n6IN8h2-ztUs0bi*EHT#vj<dLPr^X?dmMl^k)Os9d{rburQVV-?VC? z@TX5=a7a!hf69_e0ToyvsVFT)S_4`PCCLfAnT$}TSd#Tf*v%HlCl{9RiSE84WM7?m zoZaEFxzr?5>~fim4&Iq<GZ<9;VMXDTZy@-ULlk!aJ?J3*cbKIcNM?ZAp9pra7E?<3 zUjB?k*dk$3f21lrTO<o5o|Wz!O2*gPq-m*3NG)WJ{1}mteg_*v7ISp581&6CqT<53 ziWc?BH-W_v9;U7Z8qX{-dGzO>$)n`ar+=o8&X~P@`35|dVDcl=B09QZcH;~+ee~(4 z5nb2_2K20<$h;5I++h%^t_}vFLfRHi8t&XzCWgrnWXO{|Ka-B5uX8I_uUWBtjWjJa z#gKqd|E|3i&XS^Hp5&7x5>JMbyJ|Lj3NEr-d1Dj0g=k#l%B5Nk`4L~wjL+!WASvDd z9Cgq*dQG*(w#5<3<;68D&X`Y^zdTSC>&$W`a;tV$ZoT-=^CaY$`rw^eNk{mtw|+{x zqb9@2u!C2Knnz@vBP+@3cG4~_Zg*a4XJK||cz9_&G!VKYj5^r^nLyWy!bIQIsU)`m zi+PRiB62RrV#*QinX`AqG@9?xhI-^GdW-1kYh)LdbC#SuizxiUmhavt`GU4ZkOM}A zd)Vbe2K5!RWDrs@7!!~{nMilhS@c6S{SbxDBG|zH03z1_gjhy?E?plKJN{Mhp2<#G z?5FF|HAlVz0{!DZ(5I!{8{lp2h>6)j#m_y5nPipB{Vn{}`b=aPIdU3>-Xv=&QBy*1 z(zO^*XYpyVnL1GK@<FiU=xLk{KM<lOgxn+{<P4d`7ggui0?NW}o8f*<g%bqa&8eSZ z{8kSnWB#jlbl_EF{G{VJ(k|S)U!xJA;H;97KgmL6VOBx91I(`H&LybR(zDXD_(ji( z3g^vn%w?&+cj+1+mX`CWafYa9Rg^QjIF18RYz7%epCyY&OVblW7bU6E9O(sFTw3)H z^DDrdbrd-Gqu-Lv!avAH;Su)khy0>FSGC`>P}yi|G&XXy*<%rr$(M-)Cg2>Eprs0B zgP}ULhGSvB$H-&!(JyCFA73IG|HF_EF@TJuMo2JBqi;n`roO(IS86e_#gL_Z>!H@8 zdyY$sYn;^$Xc;yJ5QPaYFB!wScmle3N^ci0DTRmtx;I@QF$*$fswFwSw}%%L^NGSL zk;7Ktw6h-W=<VTItcM$InOUlEVY{*|CZu7ddS%FtbuoOm3kGk@hMX0(8~irZz7c&$ zJx?Y+^hRB4EnnN3b0nr!ipP~1&!R+3Fd_I~8Qe|-gNF^Hfi%#!`#CZ?-IkV}Y4Ahh z#$Yh%b29uu&dP8Z1v_o>rA2rxJ}JsEo2(`^;xzoQXOSe&z+O2(s^lACr_J|8YRvA) z%+D^c_~lq34}eGvf9DQ(R-k73G1^!WUQHf5JHTc3v)BO4P&=Kud3GS`?iA$Pi%ms- zG|)W@f!#58?zEG@;C8?M0VWw~YlmG73RocNJRxgpZ-V6&h@XKj@_t5Wzb_I|&6@TB zWWTH%dnqyEwE?7v4INC$2q+Rf|JXy&cI%XEC#~E2-t)a#bN`^8eKD?Ug7r9WhpZip zMi9^3y6(RU?I~-&423siei3y4bLanCkf|CqXB26Z#yz6zpprZ_gg)^lOOorrLq^Ph zSUXE#p5qUG-}c>^uccjG-3OI0>0J^!EEwU&f6V9CKeuj#c8ru3gN_=!mmE`L;D$iW zIm~%JJ$rtN@NYH9eEs<71yS=O7D{QKg|kLdzrRlMDaMOx2nh7!>(17n+jT}t`kc9V zi}<M#qJceKnsN>frZ-*&i-+9x3?{8imB}-hQDf;E;tR8X9et2nNnd$w?yRZF35m(} zC@De+7L`4^I;keN)!ypdS3oAeMMi#sRDo1#eEX>BsG12nkydh-_j;1d4j2rpnucbC zgwRkI35F>l!6wgeME#En^O4{9m>d;`bN5_s@N~h%_Nv`<pfd@Izkf$%NvDjgRf09$ z7F7@bETPA}OYjA`e_eeP{IZvNT|DO1!L{==HptM`Uqa|R9LKu1_8tq@@Fg6}{5d#v zLL)3t78>g*#t*Jyg4e%GfZP8J@j4<dpemb`PE&rigQI}}fjG0$f!*?SAQqs7i(G0f zO)umrzKTlH67u8K@d=qJY1rUVh|=}O<Yb5z5v?as3|}D1xM4rX#4g+lArNM}RtN-_ zRZ?$n-NbptFbuX#xF{@?B0|UoSS$g>Q0){MqSXa@p0GkwiYhWH)s^sI;KZ@h78Ke` zfyH86edNLZBI?T{-HHMCp>j+B2{1WmE&Y89C*K7KF2gz8*IhDyj#>Qgx=Tr0S5NwH z-KDzBT4QaG?vi{<Q(iq&UJAeybyEf~Y#>QPAALhcANgend4zG<$b1djlMPRjCH?SE zxUM|3v~V+buR}bV$`%F9=jpee08vsxGU&dmkL&kwU4VNL*{Lh%c=D|fAS$aUt*cYf zJIK_e$vkau$TD*fK(;%`P5gN0I(hyYc}(r@5Cc>|cyDY4;B0o{eVYFY)!cJI9_Igu z&R`fve7qW#2C#(wl0FFfV0VS&Dttg#;D3c}$nKsPE^(zGf~r6_qAm{(f~Z@U3!ib2 zOUw>Y`U`plwG}KfF6|@k?)e$nakeX>#?-}twJtAejD-@~@U(Tkpxhp^dDFTGX-<YU zBSS^^fF<4G832Cc1AE^Y!I{g-E#>N;Znm8HfPX%B!iC5$rRL&dbFsRz#AdJHhgD9v z@v92*Emp26xjB8WMY`ZXXnTk1K;iz1J>2gw*Pefoyp|!&F13`GsfhIZ?}_yM>8N!F zxFfDZ6>W7%%fr^L+3}|1VBvvsDQ36D0UGyQ2p?=C$$kArkC9CButwN*Mn>k5*EH21 zYTgyz{GKQ-lP@&wEUb;7E1m#miedm5tYJnax$ad{m<52fjt<zUx##j+Qxb#`(4>f| zT~nr^mE8ld2@W_mx!{Gv!1a~16NShPT#}f|fW{#%B?RculHx7UDuNcpL4=kN(gjep znsr8`gSDuE_r0IH12<xiEMmK}Xa`rc!?`^|MLv-+38-C-hO`DQt-;W^OLa%Xxo>xC zmAhyYDT7*HkF=TY`R8>zzJIwomdEr7b4c`Q=SiI2S4AS|F!C(jMz8n2w&B|_5&<0? z#mP@QIrr%9(SYQhX>UK{1@`hZl0@FQBZ{rQ{#=8)_V(>s9{pgOCOh_UEL!#!dr}pT zGa#dULKmK*BsdZtmvY*I`BSIOKYNX=$7AR7*SC8bx%2&VP%lET@g-$RdT|O+s>5qD z8q;>B?(}PH-Mw#Ds}!OW4yURSLqVS%b(}p5BMJf^W+MQqvKOL@q6&B9`{_W9C@~|E ztEO|rDQW2`*?j79qt>`AG9xNIDwRrZ`sR5Li~#udACYl95)tq^3^qev7T2_K_ol}6 zsZsi<%pLUkXkSFdlT%f6wj`w>wZzPk;nA+`MUf?uei0kCZHm|^h4KaD$0CR<pdp$A zf)aTiXMQfy8ASGNU0S`qA=9rGOplb@5`Br9%n_Fq6_g;zeAIs0etc(seL;OOk&N=q z_8zzkK>z+bt9ZLT*XdN{n;aOE!w+oRzx`lwePMlm19`sAw>Y<;v{;4A|1U~%Oco*| z-^k<>D%Sp-QN@uH2t?%gV6%Kmh)kY=pL%|f&%sX&P!0w^9<qHD5=S0CQmWyO!0k`~ z52M$RvEToWfXp}*l8sg}U1R;e2X}KG?$QQ49j_C`2%;E`(Do9$d`Y#?PosPevUWu$ znLm@I;f*p__?xzmT4~{`qf1tXtK{45BuQgS$c>K&uISa(RK(GL;7O<Ay$9$6eC49& z_bpV1yA@4hl=vA?`xDO{;_LU+?P=WguV2H+bM2R;L{mG7K8QOzY%G0HMLNarT|lUS zC~}mna1IZ_gOuawRPpY2E6%^KdN2Fj)(hOfmOs6JpgJ@>1y1+V&ot2&<_2$Ew<RQW zjTDvLwjpTLh8<G6ll8G~1-ck%!}+TluWw7=#jnDH?z@&erACdu6Ymp~Wt#f8ijN&r z)tc(EYPhB&<!8^S0Eu1P(?X22tibCXG|@)9|AQ!k{%WPUC*>cT0N3d7Hq*F&H<PDB z88m0^432h+>4SI1QW<i^x=B>S1z&0=&prF=_Fd6?qV`D7tp=xI;;ZU#v3%}Hw36h^ z?R}M}_yf>Q5$`23HNqD1xz(iKhs)4H^11eSGjJ>18@k#Bt5i61bXIg)EY}iVxqhW8 zJY{8UG>3iOwlt2~1em2oi9^pNo((_3IcjWmwJMzASn9E;x47JroYE3idu;oLW1L+g zf9oWfn*(+?XnktxBc>yuUa^c0;?pBu-nLy$(R6c9{?(8>#jQK8jM}}SWzF7@1MA<J zxs|@~^D|OJZ&F&+zyG}dUq|^p=kv~AR=uBae%reonJQ)t#1Y&?!twpP|72<!fXjh* z)Zs6>p|nb3H6p8|Kf<tmyv&)dNlI8-v*`qriSvBoIpg_MI4d&e(%EDJ309F7vfv$< z)XzYhY0T^sTwTO5YuzsCyC=x2Fvp)k{)Q}=M_cgp4EW!vF5tkx#)v`kPjN@Ne}gfK z!}WO+Guj@pKX5e;rCw=ViL6M+f2LV`f8o0Vg|=n{R20PKp7}j7kV1^t>2UJp_-Dkw z^nUo-U+JDnlDcO~O1lD-uP<BlGH10Z{`O1kCpM7?8Nl*?id~q*8DM%#Whh2Oz zcw@*q=<g+4V&haEUz%i<*xV1AE?HG1X^2^TWcTT-<Wc75i-*!s-t&}%q<19-Xgq(i zVV8l20{aO%1eGyKae@|lAeg2AUv&r4P9-pUkCKtp7g`>YdJVIj&?m%<dcf={-UGAU zmp%$-@x>7sCx(hY_9TdsY{mLAHD+IHS#fb$E_Ymr6A6=HRA6qzDZfUJTj*pk@D7$h z)P`!hwex{oLgt#KS*G;lji%D6-2vSJK{6KZU8HdbxC02bk@En1!Gu71Q^yk1ILNJN zX87e!$kGC&yt+7O`=(YqfK<3OMd-m=NhA~L@cz&WaUn>2_78y5+M`n;bTEuQQ7B#% zR=b~6(q(M`9QgmJx<hbfqnc>{H=gIZE|Ny&Ge9x;(`D<pH*Plp;y4NbN9a(ozCBt- z+wMRl%+s+-HVl4L4|_);p$kYNl#(tba|f61-&T<#J-MP}S+FWN94_gw;9=YZjk{x< zrd@JpSo^SH;qC41CC5%4;VMe@?>=~3N-mX>M6!vI+DOgC@5<aOkgI;a0a1y3O!TgP zLK*p1X)<Ny81u68ERI~8&E~M!v-7fZvdku_RkFQJX){{0OlEvGnT@&Gc2l0!Vav_S zwVQ3((sc)w7E`trDaTy1E!UP~%d^@ndFI?~3p=1%hoX(SCYveGl4r9apK7sX=b5ZV zGyC=HNlJ6J(VUf=g;I0!;M?Xp%noy&(UOf{-!Lh&jZhRb=CS`APJ4DXl-^pM%Is`o zmW9i*7;z#C`zZ%6a}GPkJ%GHggznAL$h@uz;&vD_pg@sx`ikmE;+}*uj);rN!*7yh zsxxti!uE0Y;))a7uSr9SpLyM?Dq0s}U!z_VVoY1dTc5cclFDtLTCv2QDh=Blw|b$9 zE+r4IA>vdnIW<*h42wveq+9)&bo<d|c~;f3!{JuGCAoH<Rkizh^3)!yDksFcK24R9 zk+j3k(KS~P-_ep-74qsxt|P@x_i=bwbOB!IT`vuw9V~!};>nRy7rn^5h8L%v`Y@9B zOl0u?mC7F3E{|5w`WB}pI+BnZ@`5q69xYJjAZ8$)0(TvcT93>Z8x|Orj-!3a6aGH? z;qnu16y^}bXB1B&i0X5gC;&5+I|Jk|AiSOCUamy6Y&m1Njo>0)q&|ihkW%Tlhl-c2 zj9IRh&kxv^RNKhERrAJSmE2x^J?gXTDw6d+X(p@5bKE;`ebjVir?lnkn|r@g%Z&k; zU_~p)L#?f@R&}1;YRTi}&PlGMoVfVa>8n?%78OQTuHeenyXYe;F+=1k+x5gxcaB4C z(wZ_#_8lrXd`R{Cy6aTTZP=K;kv>R8N9aRpxn&aVH)zwk!6+@@)vaSU1uc<MJq-$^ z@6r7#oc5*qS6^Ngvv#El)gAuP3qNtkj_f)@M8qe8h}FN7M_hY<0*e%OPZIf^e68v) zJc0p)99254kY3^p)yr$*3KlfY-?Ok;TDBv6_trTt(AO8KBe&F-MDuj!FMugLK7`Da zrqo64-Fb?<{_O?To^4G@4P4lnt>?nerdP!rjde;9Q??q^o2Mluhw;l}!xu)amWI!Z zpF2Y};=s5)W4W3+JLk1%JLv>O5Z96kPn`~ZC-Op!bnA_;Hh!mm?|fy`JN%*gGfmY; zrKQbf@9$%g)BA&6S0`gBu#w0++;xZ%wF$&nW$o^e4E-P4!^p)FWYxXn8wjE}(4P*G zcwP~nec{FnV?D2Uo)!7~eAeZX0JD~>$z(y~JIWntOVgvd*SFEfS4>yWn6tBXHcz*I zPBTcxD`dM=_ip5c_f%JpkjF3Y<_hYL7d5Eu4y)PDS7d!ihm>uX7RJ};bZh7nGdHN> zDxwM!xDToCt&zlcvNXM-KB21h5_#e+b!}~ozLIZDB10xS5~R5pS&SF}-4*By;32)` zFCK~<i?++7oF6qSVW#1|gU2t{eKdR1w8$wb(u?!PwT@7O^*e|TrqiQrQFh#y>Jpj> z9NuWMRJwgdl6J0&`kWp5&-vWq+-0R9byADfY*Eosq#v{|hi>BxkrCMu>e#qkTO8kp zPV&$Q@{~y$Nc&MhNr$N;qjGFJ_~*fZov@e$tA$(SQ$a6GEU}hYO8AS1PoI6OT?(9m z`yr?^eoc1u1-#{*eq9UwMV-pL$PxLpj~au|^I%Xocp5?T=~0s3Z6)uxt;8v5B}YZb zW6c-esC@^nJQ*eKKgwV9nSa;QWHO)}dx*Z>{VLfbKZI<=zY`$5JRU@(NZLlu4dz-6 zC3RJmmh<f@D-Bz<nn3z;%eTvFIqS3|A6Yo-$60So;ogI#>eKR8mGfv-OHGxOPOPLs zm&x0zuXbNKdWy@e+VSZde@NS_$kRius`3k$U6<6CE@vcO;H~88pW5TNH=f)vJ~K{w zbkXjhaVoG!X3V4$<GIG@x>c_Yvb-3jiYtk3b#mm~uh27VBezxZL(tXq?6~(0hH^F} zXW2}4%ndeBd&~}#&1lY+?g_<^4Qh|w=&(5RY<YQ3v)yRV$}wb{q`OxuvrU;cqtocL z=G$%dynIuh!EUf-8ndNuti=S7kz;Tg9A=k2FVF6>;A2*9Ms~LJY?RWRm4PEOa<iPd zfN{BO`DT~NVK8S&FB+9u24e=7ZZX)MDoh+M4#-=hL3OS`nUj^3m0`{>XJV?eI2{gG zE`GvPC;d0C1I@2R&_atmLYG!a25FH0=??q~Nd?JD%`nDI0awNKyrv!0o@ej~;RQ)H zyt%v-8GkX8iv&z<E0kfO`OYFI(1CZKWC0BEBo-mnK{poJ@Fqs9RRZ89bRN`rIsta` z?+O5RlsU4P0hs@;Tnx0Km<?3SDUZ+y7&4WKBy<E6xR7muzdW3-`i_Q{OU-!D)k#TN zdOe?#W=I4mR4oask8B3b@mvJN42`?!Icb<}XH>JAsKpiKPDH$liXG*a3aQ{SD-+0X zn54b{OgD$-kX-r&d7A!KA+=bn7FKFn8lReGNJ6OtC1DNQTg;sBX{fN?v%cB$sWddV zaYu_9Iq`}zCs0botkiNT%d26i4a7eH%kjl+Ac1$h-x1KLXV^NV%>k9eUmqF>(hvnx zoiNf6S`4k!A@Qd#2s$MhCB%x#?Ult9YIm);qB1oR{_ZGGtcXm<@V7IwHnX0i%Y@%V z@9Sn9oviMz6;GbAd>YcE%RIk{GNUqekt*8Z)myzNtL{>hfAl3Uu+SPv7z&m{4TP=G zL3JL5+M`>AIO1kNg2dBk%-3}KIXeCJSW=k#F6sZ|m!qz~PbA|%Zv##Kp@Zb-2&f;f zK^2Bd5%xn#h@D(paCR!vc%EOBw1ljr4y^FuY?P8(32`<O2l6G@Dn;_u^*I=23eb6d znBDw17(|NCKZk|Th6(v{*C1RKWJ!ONaor<Go00^P)&C?riJ$^cr9b4LOY!Otl8cfG z|D-!f5M9lYnFF9!`51)1bV8g80F2Q9VOO+5K^~C77G_4K*i!8Xq*iPI*nSh$s^phF zadZ;Z(V_G)&a3DSW}*Bm3SE4pm!QEeXg|^bG$RUOVQr2eM(W&|Rqto0kE@Du(0NCE zPm(##dc)563crNPNN1FR&ROicg3QbE?%=n>xx<hU4^A%ciwqXuvx{F;R$N?G)CxM! zm(DpE<&4BGe(?zg9Db6{@ul-tupJChep&dfG6B{LfMxi@c5y`cIisrL>a)na6~2q< z9D{ckzl!*shI%KNbJF(+o#%+EjB7CX)o1N=R#YPS#`z*g$B9ykD>EzA4rfk|gRgg1 zRXOU9ka@mj&SF#_JNmIpGt@68b9~9XBlV7|Drdc)!+UAc{$#k<?REze1=0~C;9MQU zP1?Sng(T$rt}c?v<MzV|=}=CeH*Y#ezY_PvBM^+hPQCGnhbOG0K1c>by;(tD>j^{r zaqVVDJKuKrz~SbT#nnYMMK#je!sA5Rs78S|J_;X(=V;i>St_C9-*Je)f)E~=xU|jr z=36QtP?Z0qqdC-sszT_*5%c+ND?`_9UMCHU2pY43InD5xQIqc8=)=XIHpN`vH~#*| zR^p>Z#G!hB@j=@gQZil)m2q$#NC1Lrxa4C*jsQ#$QLab7#kI4SJmN(>4j7;0dzaGJ z=mg}eafW_VjuII!k2qABQ)#Q<*4FCI9#+*k>WZp4`Suq>o8k|?t!gTHySk1w&h&Zj zT)lGP{ChkuOCI~;#bK9-LUre(rW-qtQIW2QE7BF|N@AK9A6V74N;;+e+NeL&O>h!{ zW%`k|F<UZY`-`}!6+vG&NAR25Lff{zE&XnO2$@J#YDOaoq~q5(LrT~2Xd*m0GMJ7W zSbS2-t_)XcTU)%odXp5@H3KA&q6)4KKDZ1R`K1=4h%U4c)j-Dhd!Vyv|F5{F7i%v3 zwEv=-`F|`hirgBS;XfXQDKh!(5VHn!7`I(1zda8(S%JLsYpU>WL{a`2b!|#Jhif^o zxH+~srYNRJsw<?@<Bh5tO&{Ii<hO1R#id~vk(iLYcmBc^PX2Sq)pzDi=jm4xISHw* zd%L7b{r185&?Y30(WGje;P6@9+FIULQJGtbk?in93egyaT}fuselYmwo4M9yxF&T< zvc;Uj%gMY=)$5Db+Hiv<Bmj*dg^o0bMv)<u<&jBB*M&IOsOk7X3?`$gicAdoZa)9^ zHni(zDJs((@paG&6&z7k)3;@u8@B=t0o3iq{Xf-Q;12x0=MUn;f>i(81B157>**V` z-|{Jx#qV~-$LH7*__ewPx>f4vXh%^j9~!<sex*8WO=;B*?{)9n&w5hO+zan<XqH(s zhf#%kSolXbb`sIC+9JNTsJ5u4<YY5CPMfr&{pPhlR8123mHlo7Vpn;QfCwn?WTMgv z7JSUm+}7Z%4Df)u5LY)^>VfdiO}}z67dHKLQH3jE&s5PaJY?u7xY8A4g2Ey=^q|m{ z+oU7r(}^KerJ|$1fiLyy8*e+xT3NG!+KVQ{s2G4ABP9VG&Wsjr%{yGuQYl4k%q69k z5_Nlf^}%Dj-6E3j+fNo+ekUq23--LCQv-7^ud4)+>KQN@^fHe{jCAmPk^B&Vd;kZ^ zXFyhQ<VXS$Y=Hkp-XafZ0BgrbGF;Y+AM6DcvZDlxSbm5&Wef|+3Kt%gq3k^QQh$*M zVSJ()Ig&zP1EmAx2-Ro#(jMLCk+=RrG<so<^T~V>tH~t|N~HMKbJ{sxd5&8n8ORWI zBY6YlhZwAnox=-Vv@__U(t92TqhzSco}wg?C`m$5M^Yz4VeATU9m8cz@8f=Pb_*bj z-vP1+OUm0O-ZJO0GUX_f)f_ER=WU6e3IY7sbJ;sI9*YFkoZr(d-rCu7{#_hLOsAoy zFE_i0rj$HhT2WbE3j3P|lD;EKtPOX|b81@15ZsF+WLooQUu4w0-PqtdQk8!qwu(qy z@-Lol(f@}j{y&#^kbi|e$WBj%ve1bPVs@d)m7SU)mH&v%S=mtUHoMHl+1VKl$)O2} zxzc<~RC10g!<c3@WMt|K<CF8UQgzPsl)|)_M15&`wq2i|l?QQeLw0UXQM%0{h3%Ye zRNGVaDH)mRnYv7qIUzAEs>vYDv4&Z4_}n!6me}HSdsd^V&{SlxW)`I;n+x?$ski2O zN0K?qk*wF-Oy${``DqrDF+C$U(~(-RJu%rS&B@C)+jvu&!I_oaQ)7b>_z`1qR7!MC zq%^L0OQoK38F!mqc_j{Wp}ojn>~NIkyqO!e#h73M{KA|jHQVhuc6FZ3Zc{nZt4xj} zXIe={Zi+M|w>UXool>^ln9CQ&Rb*BbNHa|_dNY@9j<3!uv}Bu1CUbgGq9dcoY>RAj z<fde8FGzL97MgbL$WAaB4YB#s6nk1hiZjWz-IZ09RhC<sQ7SFSN~lUz7wLA}OuW8$ z{oeelTw4rm_q_BRy*<MY0EQ#IFrhTY9Fv-zSDBq}D2}%pOQg9FnZUpDsFJ**ltOED zdPzocvLn@Hv~50IY>P9dzilg$TFurRRbG+d-Lf3L#kA7~7p62h$Bg_>K4h8m_3%4P zx$7G&mOQ7$nPr#8Cl~BWw;||-Xx6#g*FU*)Qkvt)x8|!W%mvBC8M*fCe3RXlUzF>F ze^H#9pPl70<QwuW783+D^sd;n^lWKnw!xTbG6E%%ox|?g7y8|0drF!Mk%%$43|V%o zE;n0Rm|Ry-YB`jZ@$evLx-}WB3Rh|Yyq3HQI~39FX-*T6CmU>)wa)zd?0h528FpM> zm{p`tPIp?GGmNQH2gLC6)h<hJY@V?ss_s~xIl<Wi0G!iQ7@f*aoV{0Hn4S|8my?o{ zRGFHLEkP~%jPz{KIZ{&Ma}sl%mKYtJJLo)^Ga*B)H>Q`{U0V&7YFoLr%Ft6niLn|_ zTb`rR<S-Xy<)mVbk>uj2@_buvO+lsu`#iB%pXtn~$S=q*thCunr1`bsrgBw5vCUG% z6(m;`Ik^JIk#tv<E2}XxH>1a$@piC$gEKiL+m+jpo{)uWF+1{{@E~2rTuWh%!-DHd z&CANmC^Y3|NS%qMq}nW}xw6obEX{)xnxo1|aU_-J0&fv-HgQ=Q$+;OulO;OVW=buM zwI<?e1NqjFnVpHtGi4hSjFy62dtpASFgh_EcdxI~Vz(6Js4QjWg_cruro!B!-15?* zqQas=Tdk?YSZA%zuFJ_P&js25H{0$+)({S<y}(gjTa;3e7L{78&&$s)%Fi#WD2yvJ zmKz;e#ro{LDratvv@$PWUz}c&USlxF6~JAFdU1APt|RODXKDW+s)>eIO4Izs;eD(9 z#i0;iXpfM&eT5g5^obKsbuJ-KbdT>I?|UEV`3JJNmu2n=?g=7ye<4U&l~x)TN0aH0 z_%Mzxx+?<r7D(LjBA3HiSg0z>a-}=DwmHLVrl?oQ0E3%PCPMaq`bEC5si>{F2UFK$ z`2F?Q1GkA~qg~8NMT!;q<$Er;${7Hg0Epe2awdxI4&`Aa|9pD?AcRE~2(+~VQI+KH z^J%Y`37lUs(=bW*r2BdjB|s5yK>GJm$J~h$AzetnFKWUNHb<CF8k0-=RHN|6#D-WH z&Wbv?72V*(2hzYPEIfY`QUC5hkM5ML+Xe5Dx>_}2KutSA9;2P4uZDJlKju*+X(T|_ z_>1~=#lgp?gD@AC87|8NZM@6_?u{-f8Y;~?rqaxQ^##-qFZ>6+b8n?;{p!4uEIkSx zBvQtHA>O^P-(lJRw#*9Au;qk&Sux%{QLtAdWF$^2Ve%tAXF`&^SA7l%CLWYG5T%8i z@WYmT6mj#GswTI_R>LKStjSzO)dO$Ds;S&Y>t6;Nc*V~=QHkIC{QE<{+oWA*x*t=L z*u~^$dYB7EW`(CK@p_c-p?@tvF!t`VJqr*(1pZ%SEO?gwKHVFUNdel?D`+M_f=zkd zM(TmPj2$?Zs@1F3<d=r<wwT<Q7}dy#QA;Lqzni{a{oX55ocZVPj{qkHx#vGNA~OR2 zb`f3VFAljEh!10Wf+~-f!8_IVY+OiRAPePF`p7f-_>1-<ARs)V5hW%>WkjjLSE&Hl zZyj0BWcVQgw!5gdx{3>HZrpHOJzFM!tk3ZcjbY7PbyaQQE_Ho<dO`(HHqZ^CiiCpb z9jfky?nTONX}Z`rt~qRb>rypyftR*!Zw}*Q<8<i}z2)QFbORF3;TojI@0YTUu4koS zA%!k^_afyd5g%wzb8I)!sYR1_PgK7;Ieej(AN`H!%;~0&K2v=h(Xr?h=dG!Hxc>B_ zDZ3}A<^KAKQz8~E;+fpEXwl-WlP9Vs?0W6Amh;we(Wwu&eXRcM!=^K*`EN#x7HY#M zy{eMe^qIJ8%Be*h&|>RF+EX3dK2f8<k2`jqI>mdJA2@Y#&xao)iPMAq(F6OVXE42) zRE{9fgo9ke!P2*nlSWzaeBFjM9GN?T29qafm>NXHl$_)o=;jQc`XqvrK_@jp1pQMM zz`|91?=V^b`9|rnx?4oTz;?+uz=C6~xOUG#vB%ooBBBpXI{7SlQf&l0<jIr&A`eaC zyQ|!x%Bnr0{Ssjy=vo6Mn<Q0RNxf&XsC%WDfy+stJ{F0zKtz2YRt{8Q;Nv-DH1ZRp zbK2XP|2zrAe}!_khMWvm3VFg)6~(R^bG4tTCbcrYfGXO3?HSO!SL^jAV+P+n-SY%} z;Bs~@UO9fQJ$cQh=8WH_|3n|SDm^v!ncdH+M@@`evXEb}Bx0g-yle2)p``CL>7pAy zZTnt*=6GS%Tf74+M!K>{|0%xm%s#aLl#DEcAuGeLYR%HZh3e;qZd){#r+ueQADS`P zFn-s>vx}um&wLztQ!Ss{=ldUbpSr=52j0K>qw6<Yb9Qsao;;SR$|Vn6_I0CTBBP5f zrcr=-!v$BYy|SRV(pKY_Tb)spP%QPQE8Rns(L2_qty0VD`plVD)jEf#eXeZ$<-!?h zT|-;ZVfFcIaT_l1q)*U+A#)Z=$9AVEH_VwDH&-ox^yI2_MRhy)IcFd>(C3P@^}_pA z7u1K_(xMyq3kx?6p?!j+WV+y1LewNTH^*l4%Xd2R^Ya@Td_P;6k|~NyONIK89$+8( zvXTZ4+tHAjpOv4P?`O(2=a_97`M!w9VHH|NJB8a6+^zF;h=fjbea~m)b34SDY<YfU z8WvwD_Y4PyFOTI*5t|G^hLx<<EY`?>+V3x}2Jp%gDBiFvQMZ97*WtL%Tgf&op1gI_ zCf+j~hi=-mb@F0WH`F6=gwTdi_RGMIoJ2I$(?&y;@}I8K6ZC|He(#>B^nMaD0XXS7 zib25`zz>R{LLm5nSU~e9ID7Xxl}wfbkUu#Y+4GZxO*4-Yc^B5WA~y19-#paTf@!LV z$nl6LlVQqlHr<%@E{9b9r=o)!7S%3P(+9?kp$}+lwFfuw!U)d@aHk<wrmjAFwmP45 z%RD255u)U>^y(T_>#oKFH8mN@We9wFK84Oj{SvKe?5tU17cH(ou#xL7cUOp39NB*9 zii$i5)P#gQb>-5wl}9+?H_z|hQeEomGiQ2A{S~pw52ifRHdqZT+AH7{Z5i^$GuK|@ z-4)&CqS^1>*a$6!kw~FEL`L!~k*7d=vxdj}2^pqah{7ob2yk$rGy{YI8fT@ZyMrmN zQU&YN9<;RJr3px?T9Z;rc+x^!M8&D)>*7`S7$mF<(N>BzELpG>VMlMQ6%MqrSIDE8 zH1`U5+{1mu$cfdRunemgh}zW|ps`{_tRXVR4R8^)pu<DJruPqU_&><n<4kxqSg6{k zcV$K$o(Ew5$-6SFmgM7I?-2m(uFeQRg#POnxAw#T*pr1zLE?IvxSsiuD1O>ST$T8$ z`04ScKPtiJ2W0<2A|KQ#pQ#rf8>hUw=ERIL?gt_feS>8mhyNjwp9(lBk=Fz?HRm>| zEs~H8VM{l!YFOyoW@|SsRIT5XxMkzIs`^N7!Dtb7U45uM_M-atuiu3>UaniBd`c{T zAYd+)OKhK#ZOvq;>ZeyukC+&=VR{&MW1gt7eAn*1>gMW%P<|YZ-A-q#5^Q*J<eu1H zlnyuZA;zri`dBG{Ko46-{^+B^&?m7v3zc7+SXo|GT8LpHP5iRu<(E|fV!2|{Q1Xog z_{KII$65jVBP31^A|x&GDtEEAlS}mzy!@R%ggtn+q1I{w$@2yVQ-AlQQEWNQPz2KZ zwJY68;>e2d^3CNzyBE}~D4|cajd*j-A?cb!F^7+;&ea?})XKFUx={78`txhs=DfqV zY~CBxGNi=p`&CwvO=K&}1v2MN<AsdRIKv6UK|{mjx_M=5@})(Swa$%dI+;GPcI*rM z3u9LghLF?TvRO?txzdR8$jWG`kOzd<)$NC)YvQG+mhW7$P_-&4Wb1lv+xm48OB1CD z%T{KrRj*v_EM3p9E7?*RQ5R7cxhJ|+Uy*ox(~ZGq-!+oB>@B&=xV&NJC7G&Ji9XMe zm(3Mq)@HQoNx*vF*bgt8PpiLt&slPkKUsXN_So*Dd-m<Dt8H$sZ7JO^ZM<51?E_V_ zc@KyR$UvAVNa|h3z9047m~S4(0uP7WvBc-4yYgK5q9gvICzW5AkEWFAq=iY2sq}Fb z{aP}}{%WC?D}AML=D}qLmz`ht*;eV+&wfZI52*ijqp+rf@2EMy`{@272M)9~yj8xt zq^;mMUcVa2Fsa;@BJ|I!u&ONe=Bi><rK8y8;_UXF_P3q8avHJppA?=$flK8wmuJ^= z3AwWjGqZwhespbtXq?zQro{Af?yj7Mv>KgXNwRaBEhKNAue_m@#ugiCkZPb|V#;zZ zeM{no9qZHLVq&-Iwnm2~ZP82P=LKg3sprotZJNuks|nwuYu$P(>AmdhDWuugLJ~x! zmdZNSr+II=3b^v(hWvx-H`{EEgS<;(ZqF$ZS&}0xYt<o^<>p0Zsl33fU1(XLPFk32 ze~!0p*qF0Losw#`r1Ca&jzvYLQfq}p>My$L-<1XiCuqiEd2XOAhKal_@JbRZNQgJn zgYoKDHc$noVWjeDgh7E|Tn`1c<30tocg5e1o)v%bh_f{$cLKHJcI`y6%V!J*GMI#r z#O-1$D6<5Ph$-R@@fUCGyAyu^*xA`NR~c}Z(F^Yeh{%Wm<ymEEMI2pDmLs~9<Y4RM zu*{WzR~P<AY$^-!D}X18(m1l5E@u{h76`l!tkL8zLYEbT((3@j*T3s9SwIHMz|-(m z&LvCd(xr4Mdp}Q?o+nGm((@vBlxHmV=#F*2fHyGAGcgR3Z#2S#GDNK=$lig60gq_6 zB+pT5vvFhzSt7F8N*#G>@`70YGdKzm@^!s~><@#B-^0>eNJ0flHm`__ibB{HK#b)g zt+wFRsVcHpGx^hkV|=^#Z@C%8-@Y9CH2p*GG|}!JMP31efZ@P$;W<1*><G)wBlaFP z<}i(?cd1e!GUwA;Qb!-+%%^fLm7J4WOW*y92wc=rYX_iS;>$O_c)w-wtZA#C(ml() z6o3Bp&(&nek7O>{frJCnpL88fK?Z&bT|A>|<(^G^Nn&o6F)lkLGc-HZ7zZM?QyTEr zGJx$E$`@RyQlSr6kc+T>WgN&-uhJN5eR2Gu<2$(3bXrEJRh2X^Y+l4FY3%zS=s!kO zn}q^DaX*8lFb4ptG!(BK96kp#;KLdcEY3Qeaku6+tMiwnlZ!rT{Q!0Lx%AcbtIbPh zPhT@oH;j83b;e3#gZ>5H$9624>q8!eV0a?@tBF)QqiWS|)Hx~FV2o#VHl-Tly>)&P zb%va-ifkn_LB8oGZ(@PgO{nd0&>Ett>7@y89gpPJ(AQX<V_#bIa@b0#I~2g}qnh9T zxpDS;+`u8C#)xHmCSyIC04ztNqN=;_$$iQ2v!w5te4cK@^3GOrPhyL=#YGXdMreX9 zwR;5kP*?G+Z6cdU7pX-&vdLdOootdrc&EbMPkF?+->{$So?#VJJLdX;MB0~bq;IOJ z4U0ssN2|Di<ysGuXW$QWmcUn7K^BV0!^lIt^Z^BO=cwly3W&Am8P78W@4_=!*{y{f z6UOmw0(c<ZNH*dkH{!JkpEjZr*dm6T0c#KGTA_`-VbQ2&ER5+jLyj<=6L=ojO4`hl zq^M+lbnJxBeQ}>OA|m!^iNcF#LqK3AWFk^g`X*>Xq|%vmCe|oS#ThoiL`o$y0R_Zl z0qri}_QkbW`qd?Yco!TE2zdbyi203iDcpU=AW^P=9_#&uGO>dWp@S>|;w^(IuXr(c zOP~OtOqJdHli^+ZwhKUYD!Mu#hw0IJwCMK+7Pm%tfyt!;_Sd_g75fPt=(b?LY6a~D z4QwOOR`C(ERp`O7+^jcmtpGw9V5z_Xb+WEbHwdVDn9Pt?_jE#eU2(4y;5|&uJwp|e z{%n})PQzOqswrqQ*l3oDEy3P;vkj<s@6T>lZ#Ybdj*Qf}-&1Z23ys(u1*1@eZXyPs zQzo4~Zs0`P*DJP8`wsm0-Elk}M;@ZDBDwrB5pAju-L<V*+P(LbtgFN4n<*azLW6#i zIAHO(XGZ=)Mb}pF<as=Uqwo)El+x9g$OF$`_iDahClA~{bO}GhoEk;?z~by9{pP}# zzp#Xkdn^>YULk`XuOwf(e<EMVBgwB_1C{j4d(q@eF+j49DdqeBWtsLN6#=R{dGfQv zuT6yKpKBEp$p;_*VB!6JY~9<GkWL><o}$}0c_f`aerQk#FW;YwaLN~AYu<?LdjmKt z`9e4**=^gEU&Fmqck03|HF@X=*jKz~C;39sAIhn)XQ9t3zj^}7Z3v%u<yWuBpmFdc zHUPvqglru`64a2hX6pBdm7VE(5PyIny2C^PK|B4kmpi?2Yf@w+4+ab9pO0fbK6_EI zs@PbR;o`RKuGxNA-B4RlQOWOXIkNARsz*uOGaQSNRU}|NMfIZ1D>jGn3GwMzGj~;E z%<gTRD+3}n>eMu2238FJh5jPSKx98vg)F-(gWJ6=rg4>ehYs?6{N~UVn-}#i$|%4c z0;l2Bz9aiu_=?Jc+6L9(?KRtWa~ZB8W3jrp$nJs@iTbfXSY%|<){R)x%S&JX)6?fK z7WZA;Ek@$@KBDWGGIJ1AmIQ5(MwsM@QC?cz@>1-}k%OO_J!t3PowGZ4{#JAS>gmrM zzX*@}x?1*Dw`2e)*^*JUB{NhioT0x$pH<;j;9xC95uinBmE=Rs{WUD<q@(ne+jJ0@ zP*kO_QG*jHuBj@lDB^FE!AHr<WKu8tmMu8Yg?^xKl-*~+0j=pJ!nM-7RgA98To@!& zzJ_%Rr;%I&BX=rA&<0f9sr1s4rFPdcesRH?t#K)_sj+E(d%t@2jEE{eocsrQK!qqx zIY1MaKzs86mw-Q_xz8-%-534ocI97`0=)+CZtB7!cGfO=zpe)%KaYFmNL6QNTP)eU zC*T<}QCKgs0W;)K)y7puaaMX=5}S}34?T-j-Iq-%dC5iTeun&_>_VvYSfSD*Jo^h> z)_v3%TO3<s@Uz(MmOOPuSwU$&FYNdC$#RyO%JoHlAkK8KffC~%Y@b`uZBGK=AD~`_ z8Dh(i6YZVJ@VVfvQFv>#<5k%ms%5K^Q|&Oxjh<hyF9t>JF!6tXXJZl+9IyZ!>?<Mz z$7QL~85^4yqmGHqHpKFtPb4$u%$YG`&V~1|R4!JKd=Rb?Th!q}ym$^{_A#Z+tL`oI zALS8|FppI2k$L*kFi-#Id5hU<HmO==E+^vG+^KV?4t%Np;R{l$DOQ(|^P21L{Gk5f zI=+^ui}CdfeEss&#zp7&ET<FHx(TvwmpdFQerlePROi&>R9DwnsvjN%!w9VJBNzeM zy+`9foyTh&x?R9FfyJTl`l^9QzhXH8QFR#r+<IAi2YW(-G{F*NYy*|#gamrHCqyI^ z$Mm?UtY_*AA%U6ihmZr-ALt;@X%JapB9j<gyV@Av6@-a_z-z=4`OEC9LVio$<<>Ds zS3mm1(Gk-%t+JDMBd52@*kTod1A=$VSi78ykBLEqaO&8(Pp4Cnl*WtGiD>T6Q*Xr8 z##G1GNY@_S@m{+M-1aqCm-KaH@Ih5sLm#Fq5&9W`C}|Op<qo`x@I==SYI))6L{8py zb>gjn`~Yc0VnTSBD%zzhOXQLgGj!3au<~t<30!81F)>Lczcust)^ptahI1P)sxO{9 zaIS$rcYMz!Bn&c3_{NIz-OZ}HjM}7fuB_ZuTc>JHXo@K3^6%cdd-Y@K)sI`g{SEyP zP5hk<6A2LPUZE=gu4+7b_(<wB(CCJ6H`9pwfUPp=%*S1;C@C$+8o5AqV5I`L4wP6N zAz@1zhft`PGoq~54$$?8b(%GZqGx1hW*Ip8J4IN70r1ornKlOqQGm@O&SAm=;N>Mu zjzI?o4Qp6$c%c(t@4!N)x*TBU@DSWD&>g5u1ksxV5UEpK(G!&Dq&i6g6x7)|jS$`c zo&1iK#R2bAyYfw04xV(<PKemqz*$6rZ8<WQU0@FvS%(Q5Z!W6CS=kN$9nYF}pGN*i zmun6nRqM%nG~)!fE66Z@it4k~N5_s-k;z^rf_DJ~d=A!d?Gi*6V|H?K`uezm8w1n< z0}ost$HzpP!opN^jpXCV6H{JR$$tm`?fUVKPj+%-jik-dRtEjkj3w&<L|#FX(Y?DK z>s=6piTX1^)ef&(7jgXnHV<3tRDP_F{GQ$nGX_ekBuz8!<g||>IS)^gU^Pp~ww*BL z5jI!BBpR*BGFmJ~t~F-u&K2q`+1UlxYHOT@mAq#N_7;Xn^p!P+TF3-=@n<X6MsA4V zQGAXOTIyK0(aWq@1S2?*Y>VWmuY_&^cyLm?hAkz}3A_aL_-N<J_rGEf5Fo>CxL3E> z@)d2cqS!dC@FrQhI|l@l6ivIhi=mLw;>e`H6zbFEl7Oe#1}bSVzO^%UYW3eBZ0@sw zu>D`yw7-C9+`oZo{|hYbZ;lT@X-qtp-B<RnFY-}ZhSOAlp2!6wK_>nK%bWASS9ZIU zup-S~IoNi%pK$*FrJ-9O7p@;8>(*h7TZ}RDHBIf3f8q&ZX%=W*!?+WjWTP13jO4N= zV%L@}SlpcZ&u`rd$;&6Ed>qMjS7AjYca`MhohLf3tC%t~Xvi)xStR4T+nDGrQ>g{F z1#{L%8bq;PVlM69mp8cQ0@M%W4KHzJD0(2(DZ90!P_t0%?{ohn3vBit%^vfYyf7qu zU~xdAyD!J?YM&!RNKmURPcBX5g2jo+SQt8((cR0rb}SQ(u8vYVUf2Bp*y;bHjIo;O zOsx&;Qjyi5jT#w`6xKS>t&IB2%yl=+bu-L$Z_U}@Z)SayQP_TBji8W|MgLj%u^PE_ z>I5`jcN@xNrgu1knA*uQxk1!K7_k@ZR#0@j>H&9vjRRVii4Guw$wUW+!Aa?m$z@uv z0zrpFo;^))HQ{zZ*+49h+=EcF7E^8;ylKXE?Wr6*WUt%K>h}$*)#}xsU}FeID7m{D zeteLo*N@L}*s-cS^W%NxcTd{$3c)&&VrgG6lNBBp%qE39@DfC%WK`!J>k!buRM)0N zF-#m3&m8T5gTH0D*TKJg((B<zMa3=OqQcXaj$6uHdNHt<PvIr51OQV3H>meB!7>7n z$AIyK%ArF(DuZVRkIc#twWulv5&@@|-_`%S2H1*9U=yr69m~yP%9UW_J;i`<bmS&6 zT9fr;pkeTb_2do->GbyGaC~d(;h9^TFqXQ)@jnocO^>r&q`Vn_fX1_0n`m1*M?0IS zu3Z!iDJ4t+SA~DbhJl_h4i0Ze7C?R-AE}n;M8m}4;UcPS3MYz83Dri!vV)XPv?!A* z!oyL~rf`wG`HmQ8(}^H59f;#W=NI2WdDEGKRHq2vb?v0HNd$!pYm?PWlE*{z9dg3B zgFVdg<jvLz>ZuFPUgM$Bh?WAi0QhOBjcSz`va}+1o1`68(2DM9#o<&T^61!GdoUKI zVB_K>#9Oy;g?~T<9sV=csL+zPHT}Kp2(1!AbR8ZSc8tV$vjc-Xth|mL%xgpxCorIg zL;=yd4%)#)>+t4Pt?K|`Zwq@6@zp64+5$A)X;_!J@1d^c{oKfUE5DF=G=le4Aj7O2 z4y$Oue{F+R!wxFOLBee`zMbu5hiKoQ<jJn(qy?jmau&t|pIJ=)pG@{{!|aMBC=@=y zvi|q}^qy1r4Cn09DP7t5;j+J<q<4p6?BFf`W7)E0nBcQ9%Vy>=X<0#oTFPa;+t~U# zS=_N@ySz215k6xz=tK?J$xnH|y4!Gam=9z_4{9JuBeaz<IahZF)Lgv^uia=8`1s6o zX6s(DAfM1C4h#<31f2)LHBz=W_=6iNq3t3w8t%X}4rbXypa33c{Q!f6LcUrE1eLw3 zV>uhnc^HDLWZgh;hr2tKus*svFgAd<SAie&Ub9ArIQdCDpToq9KM47T!;pogaIhxU z9f|z-J)dDp5+x%Nb`@5fsU1rWWAg5Yfgl3yNRT$^Mmb$t7^od&2fp-F+TC$cb}!9( zgajUem}?WWbMFWpclrt7GAr3q<>V_^LL1oe9v4<)!|`}_yfvd*_qPn~&EdoVR+inw z9>2)$xx8yJAt3UR=1p{abk&y_KZfbdGT}Se@*Pch3I#<I673kR`a%+~=LbY0n(>QU z+l+}A&#!A4+RBKr=vLh0?Qkm(!p38vG`0!9%5{B&TJn^VLD#3vUoe%;SJ%#-d!G}G zbe(bv8qcl8o4-%1$EdtE|Ln9anrUa}UxWO`y`^38%5Pr#V05Hx^arnf!y%cz9_bw? z_QPSQfRfw*=5u!+a!)4gL}BESA-~W^AZvwH<{@i^pn#q{@(V<;dL>R2z%TX+llhCE z^-7Zofl7ik(<E#E;IIlevWe&Dbttny;{mZ9m&f(@*bctF%O)9}1li1eFM4*YGp{qP z<1I<q2}!E>qNJ)4r?bGxl~xxv71l}-%6cD5Km=eEp^6{im*_B{!gvnE+Cpvx!bxNe z>{Tpc0d<Q3^c~u~lA1XBsu<FF&2NK*{oHrJmyf&Lo|bWm&pdI$bW(ltgf0IB@49@k z=rVr!4$yZj<Dw%|m1M|z!Ve=NuHuyI>{-=Ei64bt;poUAGe*#d_?nT!3!YOC9H@^T z!hcU69&(kwpbia6oHR+bz%{=@%MGJG>w(xEqN4o@=|jhda0uLL1f`CYt05!tX9Glv zefeX*79!Z%57&Z0uM5mSB;UOK1d(5i3(U;okbPr9Wqg;GtY&@XHu?$cecJy+U<4(3 z3vu<7HeCZPK#*j`e+a)SlQU8?^c-a9{uHeZoffuO4egPbt<J_$SFsB^DsLm^($?~R z^GP*u1~q#O+cD)@$vEZ6fTdHRPU50hMDFiV3%JX-WtbENGaaX_GuEb8bM4!L_5?wt z2lpXG#<WR#bx%;xP15(o`)7H8HUAXtY^i+XE!eAgVOW0yhQcR!L?eW?%EFfTy<vr+ zg`4B`q0)e`p|4RNM5u;-a5s$Cw`{Hq(}(Kgw-km->6l|+xbz|8)zEBw8Ud9t$9PYM z5cHyKn+E+NROT&^oL7=D%Rr3jL&pOq4LC<1I%XNK53StNqHoskt1N7h-fjNr0|ut| z`RTQQX1*|VUwlhpb7AFPeTx(Ye*K~hHN2+z1U8MJ-7JHrn+`J*LgVOuFM6FJZ7^xW zD5gc=7p~Yz^vOdQBDF}dASa*|%j4lb;DaPk2AHp61uR}TbqH<Yjp=17Btl0p3=kK_ zL3#W<4g3HOJN^U1i3<+1JRVxq*l<rF1`y;gh35f8cJ#BGJhbp$y-8%t$pAITsZ3A< z_njzumO(+qUDh6Zjct_QYXaaz@KL%9&J6B#W`%Z=EDhBm=%*PbVGs+reYe>4cHZ9y zGjAaFkw4j|Pj~0v_H%dMLR0*EzkeS?9?{67CiQv!Z^f`pBkj<S(Y(%@?XSY27|6<w z9E`ntKaYF}flMqeNqr>$St(@22Vv;fqjyxpSR25^PuzM2`o8C-Mqr~?`-IdH1t^iw zGF0S4P6XHZ1;Z+^nFg|QY09wK^x=85pL#=RK2{alULraf@bqyyLM{IitnOEr%)uJ; z!X0R>z&5-{lwiIP>C(k_`ItA4rk^Cg$UGhi@>%ZPO8M$o+?CXo4eJiXuqBM9%H&_N z6^w{VM$XFQt4X3p{$)JYuZmG&Z6bLpR<J5!D_D_o4St-6y{ijLYU-HJ=B@E#r;CM= zMByaLzt?#m9;9X^eKeg8T1BOcq+~Xl<t%ynDCu7)#Umw*^`Va%=~GLnY>t%7myic8 zkfHC8#~o6N<wiR^>;Jmm&~1*wNS@4-q~@jCQytQ?&~<zC6djh81+puNlH`_`<&>$( zu<sYmClbCmtJqkaErlabRO`s-_ii1M(peH=qR+^sbo2zOIv^!WC0X?i=ACF0`Qvmx zYZY=&87uo+gRth1&qRaB)6pbokiVMaeq-QQG{+lgj^4&#;~&H&#nJr)1eOoWy4pRf z7HIClrNXfHEL=zWfWK&E-$VYAA^n7Itnc|NvuZTH%>05n>#}1^kJYouvk4-s0^a`6 z96KfwzUexlw3nw>B-&?}`zF~F(v69p2mQPL@Wrw$3FXFj6Mf5!6$SQk;X!}VL%#08 z-TYy1iXO%Vn^^osGclO~tg>9`c~W?ij7Hf{3QviyUV`V;1n^-3*#sir^BnlakPYad zyDFum^pcF^K~gr6a7%9t|AqRr&>0c5!IJDsDK$!=)@`+^iwYfucHUWx@clbv1CU{C zIn-L=W99OdMX#R+Uhx`vb>1FP*AfYo$3NOV_i{QBmWarbBIR3ero1uNg#}i9y(_Hl zOi3(BP+KJl2`Q1OJdN?J@K~nI%}81MW{98Ahu$6IF^Sd~%69Bg7nbDZm-50QqW7-G znpq0eyLwMq!&?S^j9?;vlDpo8N$#UP6a0PZl*RSN-Eo!DVsAz^J>3jM7yOHE#g5dJ zZO#b42xooVZl=xEA>LLMwadV<_^Mr9S5sV5h^0!+8c3c)J&aj5!YPb#Fi&rbJhvs? zibLMd65&*L-~tRo?%QHwC6=OMYgJmYUusdDH8l;gm{#BJ+fa+s$`E7HNhZQj?(QTo zsyZ=n?Z&tNN7#FSH*sxU!#1|0xeg%-@(^3HM)ZUddJQEeK!DJ}1TdJ6ZQOA0MY83h z<|?^Y+%edI4Vd10CqPJmgc2YLNeBt#jC5q)e~q1c-}`+3^L(F+Mw*#(&dg}$oU`{{ zdo4^D#t9J_>ihx^`irI)J@qfp6YF7Ey@1D7`U2(#TZ*sBu@oIQdeqM0R7!-=^!Pr$ zrxWloh&A*;rrnF}PBZq*KkcW~(#?I=(glk=<Cn#*2xX*%PnDeBvFE}~8LCTCbne7; zoCjEHiQlIAGh}@pjV*J!x#_}^9bB#C*JIUW^bw9s#o6A=Bppmw$xc9}$D&An%q1RX z?vEHSfTbiq{`-?OPQ}k9W%t05j=9!-WUbzkMBuXUZDcGYmQ-$zf8gNn27bK^aK<0W zkFllEg;52rG02{-qxJs)CxO89@tJGzX>p~sSe+765LFmm8taP6$z%HDA6(+yum1x| zJb9w=>$@^rhsBqbcDGBaNGy*nrH{!Imo6ma)an0$L3%6;oIX`HwQ>3hz#xC5KbFRp zCsrg0HJ1?$@)+v?!>l&f%4@4T!JM^Nl~N|MygMF;Z)<}o{hxE<hma}v6tGzI!i>#B zpbfV;3$r$iuL!b<L5nvGjy#+5T+p@#`2vtiCS#m*6Cv4lF2>E_7%aCS3W$93-}pri znC75zY!Fl~dpRi^VHGzUwl??*3YxxKgM1Cj`VN!G*U%UQ3iV%|8XKCi#$plyUowdg zBt3n=`tkyaByOUmc+e0Zm!6i^JXA<elTe6!-9@NQAvxk?`M~m+r9CIncySRN(+?|% zMn+r*KBNQ2E%5gv{!Tl~cFT_+^j)r$p5`S^(Z&QWGg!pugU-x^fKV?u1iwJID}_u( zL(BeY=$4d6L$}b7(25wXNLT)+k^Vv>DgS9CU<(@AQMRY65i}8Fi087pn&=$&yPUEx zc-Rh;7*uiK3xitqM9UoZK%`g0N;%eg`^Iez!;tyb<hMEHs|qhyugU{KN95RVS93{y zHH4R}rX9QUjz=8;7iDQ=Z6pBEb9?d;WUe$A<L8&`>&3rP2}h+KgTIjb22@ptD}%PD z?%ykWkpH0YK4&!Np3Tf+j1uXtRD?gpAygutF|Gaq0GPx9WGOOYKlbc^K7%0~hdO@s z_(J9z5fB#61qG~4T`!+FF~9IrrP{a%#J-F)7)F#%h<9*>+Omvt{JSRJf1r9G-@8Aj zVY{+=Th;dF>w`}csf<W|J@fj{M~kREM|)5Qf7(qp@%Gl6fBjWq{*yE0=ffM2Eiv5n zzw@o^cH@zI9O>4CY`Y$EVt@A0pGw$@0)O2u#Cs49hT-5K%*j?ck)^=1JO3<vATyl4 zcLMjNiA4zOfD^)&%Q^$xcq2f-XAq5ogOd-QVi({R$x|WPT+KgWlUq`E?*TzZXC<E% zPp_N9{qcv$`aAzS2Cj(@TrgU>(P8*=d8T+U(WNl4LSI-&a!Ibsjdk~e9wsy2W0KZc zc$L$%ndMCjIPj+>?cAl=Ek~0GSx86+=@8l8CoV`WUPGOJq?}xEUn2N!u?KB3SR{nW zkB7bW7W}N%TW~x8_u))G>^+{FG;iYS6~T-k!0pk2nmh#F$xcsKhe=|a$UmaxH7X7c z4Xp_P)x7TgYx4O=q@14!Ger=3)uBsw>W2ueV8_FK*ORopfL9CMuyhx1LVP^P$?Dw1 zg19jyN8nyFYUEn2UYDV?c?=OHWT+CM<a<(Vs>p_zXO|i3Zw@LB<)lARuP;BMU!|$z z{0ld4k7LqIW~~{#6T*06G=KwsEAf@%8x+%C8$ZDp-cQ!ih7JO*A%w`gVF(`B$h`uS zN_>7|Q3fyrLqz`}U(L=z1UoM$%VZYp#&E#c?Sa);2Y6{E@CK!wUURlAt|$f(;iZ$P zk!EsB7B8B!aE9%@C>OO(jfe>iw>i6Ll8kX?)up*EU0OXD%?+7K((q6KYL24~8LG^r zyku9nrHELO0~{{&YMe>9DJRElFuPXp@7+9i<?NZ$uHbh{ehyI1LreofD?FdvgM&_d zkA6qKlRk74->_t{^~5EJxK8?w`E4?N?-cO+ZlKm8pU`{cIubI(!s`@qOJh=Gsj@6G z+dsvZe$jEug*+A`#6H22)hW%8i7-+o_&fWMJ}mKevU&2JE||seol76Zs{t-#rV~9! z&$&RS@f_Z}@>P7F&TK^TPg%?QuCk!4M@e#yoO8jR=Y+Y?t5?JaGa^r$XJ<+Kb`*r9 zLuWx?yo{&`jS73C2o~N>t^;0mPNLBMe-|ZHXyd=iLg_{Q-^cq3ZTq0@&f`SeX!X?q zp-ob?LO9s};Z;urJu@;L7A*1`-&#LoJI0BNq1j+@5wEnhQTnk+moA}iUq+DaA~IcE zh}7a0Uy+r^t4OrS#*0_;m~Am)H=0Hc!sF^@-N4_Z<M=yd96pH3k=0>w03>TEIbvVn zCjQBR)PpHv5j_GbmUi)Gx>V#wXNed8^LZA1Zi}U3ZJ&~{4df#cJtCe#dCLM?VQGia zU+yLvi~2Atg0(7`jvw<X_{X5khscPEd@Mb0^}v9Ke4$%!h1t<-@qdf|zWm5#B)D^@ zji#Q#j71Lp)iDZk2lgfzFaBo#u?N2(jbMY*&|?$7S<BJ!66qc40S7kR$LB|3!0nKW zsb?LX_GBb^_vK3$lQ|PO3Z@_}DKAluhNQPXLVJ?LyrhCO4*X0AmJ8&@Km8;^BIroc zOy4CJ<m3VoL^!cR`4eE*xp}0)N%~7cb{<d}=ux4uDuR&2e`I2<Z3y8YYI93`Uopz* z5~yOq%K>UMXu|SBK)r|H$w!RDiG1gT{3MI>X2HlyLeKJ#6w`kUUq~Ba<$5QwOz55w zC;uPbgojIrDZyj8R&dOD{O_WNo7D`eRo+=pz7;k@?*5+_P}W<+$X+3&Ei4`2frAzP z*C(tYIXyX*TyrWc)hXk_@-vZ4r0a{BSVJPYs>m^AnRMi0Ec9)4rSu}hgCEa;FscRx zii86EXi%L$vyB!CB%nZUZl+nsm&WoFZ4*mvAQ9bbUD_MW3^?2WC5ibzGgEozj!P_V zSOj|2stgtKC^ECv%BX@Q^pzH8$+m*ZiUO`8zXpoNh??JWsZbRlRUkYmGD-#EC%V>6 zY^Hn3-kv7}{iJ_BNVBab>vh(4-FBT^r`LJ>ifq*#aG7$*(nW5sVAs6m-&R-e)mMkP z3OT-=4_9?Ld-$;af#(sJHy^mTyVD+e_dD))^rXj~J5baU2*Xz%nW*<%=_>Vot9;9? zT&bUU#M2dQ7CrCWAwBeW++FXu>uC>ncK{E2x*Ya=pg(fhs<ifY>49#-WQE@YJg>;2 z7Cao6;rbN+<7P)xFT4|uDhx2r4>350L$>V}!fUt4O(&Z(o2am0ve?O|)a8eUrWy35 zU<>@?QFX9pS|_skRq1tc<#6{qyM#5Y)Q1JpTj;{$qBDZc5y<DSN~pmD)ZvWDB9bOG z2!)TK9w}-akU*}g>;g>zG{48g+`vOtQ&qGrAMArk!a)lzTg+)LDw2{?RB6gIl_4Q7 zSzs%6>C&7hw@{~tI5Z+YLWNAU%;1t}fwI`8i)&CID|RU<&#F^xW2#gU#i4MTS^g52 z3F^|qbqPXjF37<$t*Z;9R$>)8-haA4AL`@6`|v*h)di|a70AJy5#%|AJFC=Q|L=DW z{KvdIyL`Dw(EO4d0}P{>-@|J160}hJ+E4dG?Ms`09L<y>qsc_}ll@TpG8U!eg7&iG z3zoJa{>Hb#2EmOax^$^?#q;O8c3sf#@^%%}!*+S==X>LAJ82gVfHYfUJ7IU7OMJ0# z_k_fSheHSp!dij|T~1+=5|b#~cH8#<8Vj}q4u8NYx-6~UT<yBeyse~G&W9H}*}{b| zoT;<sF~U@4Oy~3Z8bpOwHF6}8udWd!yNsC{Wtce*cq~vz(}jHkzgVFBd}1^m+(=W6 z2`LGQX$+X^bm{Ws<g7f6GBpqRY|Dw4jOfAD%ijckL{eL_t=i0OGuNk;MGF`OVNhg1 z-twq7^wb4v7kKE<hjjXv;oZXoL-5au;A`5*BCKQ*+4e|qOCY*M`a!iN1-5*Q<rRxB zw=hiJ;Vce|NW_hxiZq542{(d=OKd^8x>8<QZZ3t-6%NCqa$~Kk(v`?N7B56F3dQ;H zY<{J-HbJ;~JzPqYqRo-=$Y^zJC!@AULbNs{KhiZKzbdg>ZgCcOS=?YuDG-WVZy~3k zQe7Tf00u`WsuzVABUP>us>BGWWjjm43L~miT&1ekSYCt?=$1=qfw{aA)HAklI4<9M z3{_Y?R^h)B-W`UJmmWZzTr%@DMpzArwEvxCIaoK57*?B?mY0&9f+X&g3`RF2Y>XWI z4gG&3BcLGkp}4p(zc^D_O&pCTtvNN%H8&NB-g4Vov38GcXJ!+_$BRq;*+pzLWtdZQ zUGq|tv#^V=m<+l~`aC0(Z(fTv$V<~o%~_@U$Y>X1p3amGx+zUgijgs-kFDw_N79jr zE}%O`DF;DmL)>3+Rjl>ZZ#MWdbA<E@QCIrKFJgM|2dI87BA4Z)`%g^(h81U<O43UN zMleXJKsGK2k_pRsmc+r6L$df$BvKl0&F1@I03Iin9>%yh$2LkLjmK_h;B_D$E>+Mo z#9#dCn`=b$$D>&~1DBHq^+w3e3NWlciPXhhsDtc0lbs3%3gC?7G#By{6KS-Ph7FaV z!Vmi^ez8dh3&%OQzrwl*ZZ4o=l}^`4?(byPYv^}cy~$rJNu`_a(|I>J+V>>waqx}o z*^`R^M-3+L_C}+5sknAVvmq}h+jO4{bjdByf`~mm3l8#bbnP~V%)o)l0Vzm8Qs!(4 z-MkS{>Y;R=jAoJWk!1D^5CknFPOFE=sHo5KLC|{WO=Jcw2aV6nWF3Cf(=`1-=98Rc zh&3l=ry?b-H%atk=yVAf^h;5Cyn;-Z5Z`84xMRsWS&xnmOlT(nU)Y~~3LsxE2Wv0u zQC!B)#Hy2#hy2?Zk}zKJYAO12d}FR%Ul17p7MrJ=-FGW(BR_T;&|krSCZ_g5wA&&I zO=w5q5=kZhfS?vrFY+;+NygG;OiGR^-7F`|#fAB~aH!?vYl~7$@W{;vjgki)1UcfU zI>ZP**iJkcnEJTD@c=WvC6gYK$@a*AM0W1WUZuqb1^J%r!`J#JF4n$>WZ!tjUy@Rx zL#F;>a)tjU+pI^{wW~Q*ouiV|rD6b+lYlu~YMT(fHe!A3I@h?}ajjtosXsr(B|lY_ zn<R&Tj7lZQdZelKN!e<0+jhqVw^^%Gzy9b_#t}!S{=yrIb^+%R!(79Gg~KJ{M7H3m z+#*o2#_`8N1#7Lhu4Sog=t$ZJjv?LyIP_Z*(&v|vRHh{AUc^inklD0?)7x5bS9Sc0 z+*8m&s5=}~G2ytx&YkZ)gmxz3yYk?T!u{*KRADosv7f}4lA;1)1`4+_60R78Pb78@ zd_MV^)q?aurh|Ni&gf2#V;BOZuro^|#QtC;C+%W@lbzX}cz;9OEOdw$ZJ-6!Y5ysc z`6)T(LdJ=A2xVf$TZA%4#QeKdnAOe_3+2cFz-1i7&&aQtCT)IaFQG;+K^5ZoM0G-j zau~IVZ1&<o#3m|APcp~LX-{ziwxlZuV+*C?k;0aYL}80UtX8LC5KjfuL8kgg`l}1I z2ZXV&cn1!RPRH9#fF;;$qJ&H&?#pp9#ho16@B{%;xmip;6}Dylv#qF5;NK<_gl$`J z<JgkY$U>mt=Ry@`7)%gw>yhz7FuNQKg~Pz^HB36!%`waB%*JBd$n(?_6TWOZOd?%M zwUUh+bh-^nq8C2TrP&glpPxPeZd>YW5J~6L2@)bQ!bFx`tnl#%|6nVUPxQJR5RU89 z<hFs5sEnk5b#lE?OGQR5fX)?3=aSXpT`js6In9$u^*DfRv2Jghevh1Hp#^--6O;TR zGW+s2l3qrZ4}VMtHZ#(_Pufr2OC+ug^|$S1K}7alRX>hAll(=#1B0k?1|Q5KL9C`? z3`fpM9+R3nItTeFCfpB#`kNIV+yHTMQF4LWEWkKj)aE2pf{6ibnt|opI{sn3MU>t{ zVQsSs9}%_e(K&c_-d18e=ZBDJx3;rF@vhRYwg5gr(p4#A3#Jp`q(!O!Uv<C`b%e9| zuwfkkd4U;kB4U0Q+ybUI=xn-?-C#i&qOjVCO!2mKkOJs9;;pdUrSFix6xw8Ef>vad z<mVGF1&NoC_nm&Uz9q3vVrhpvS&g&-`d_vDyW$}VQ2!`W{#}=KB-Nn^b|2;D_%mwN zq1FshrX=5tS|vhF@sNiCc47#otS<&B(7v^eIt!E637E2yGKAOh#u;yH9uouu5Xear zG*Ms@C{PIV9Dh6<oomJ`Y_OU->#&UBQAbw^;SsiYpvKOM{`2WpXZ?dwmS==mx|rV* zMM9h)FYbrFv#XZm>*b0-%lbQ@p2iN=zQUd%X!8f`<3`n8J8h!LcbppCM78AtK4Ck8 z=nev7norPHU!Se@EzR`}Eg)sWv<XVev<-kfXZT**MxdHXOgsJ>{iGj98^w7|W^;ZO zQ+KT4%mdk7J*e)&p%cojTc0#vwJ2$^YT>3$0Rdaq`FO2eJcPdEox%8JY~AW7>tH3m zjazr>xMtnC$cqt-H^RH})uf-iRQwI*Bl;})6T_9-eMfh<m!40(Q&U~O!@MiMRo|)t zUvlw5?GdIZy;0S~?c7s&`Z)YNR%fzH0flBDhg*W`ar|3-eCvZ+p5vr!CV_CBwmn5; zfV2TSS&TE0SUbodHoqyWu}Ee-5`QqTHGdZ>Z&mM#-Vs`zb0_xv=Js_*=hTiiFzE^U z82M-7STXHK<*U7^opN5p!bo2ovqcxU)mJzXzxu79aNL#gg1)nVaf{c^b=w2>Y|39) zusDBF!Tf#ence83abfO02s{&VOsT3;n^T$?(kTAx@sqy{%Hxq|w(N#$(U~}q-scH( z^5MCoH;D69KJ^#441&m*+fT2oc~)>W=~DL9w37u_RA;lUT)Fyy1W8+N?XnIb39O$w zE?T9^&Q~F{i`zawJ6~RIj`dU0k-*sX%|>!p4|b<yXMSl_RetOiZi}Sj15(b?_?h3@ zCf(*I-Gg}6Qg$}oA&wOKVtH~}wppjdU|=eik!^G6K1ql%HcQ1t=M*Pbf`Os9u~xb9 zssx#9Cbn5yomR}%r^FNm%frGGLL!x9>};F*YKtVeYFolKd0kmieV#JA*jTdztW>4! zEOCe~K3x`@u1=1VhpS3=DlZe)ZzOv(^$F!%O-yj1pL|PjVraB7Av$&ICK+WVn{tDS zVz|)qy2NJr&icZ-GG!ikj*P{OA=gk;C9^HJ+-7&G$|57wF<XGxkZdu>R#oPg?&SDJ z+X+P0Z?7At9}zX4OI*Ba-4YEGPZbo&1PY8ISQb--a!Ky0eTiq7s2}vt9ztC6k>OeS z_gvxGL;KF;FvU=sLjsHfG=*5k6F24Q)I;lv7BS@$^drV%?~ZhflBHhLh?hju5`Qf0 zM*M-;1Mvr#Z^g&y@}o#7ydx&7Z11w0G=T{?i|CL<oM1>{O^h<3T+;x*aW9Z%Hx%LA z%W4aE%6HTzhL$UfqH}|A?!6??BJIw$N&<sJ<3r_<@uo73GQB`s3IpiCRazsFc4O?~ z^eLFbsSKEs$_tCkmBmWZa@8i5@UzKjSX3sYLVkAk4iVqE^D~+D{R8sqeri<DIuh`b zbf59^+;_20OTJBcmh(g0v!WYm&oZApBM#DzTMgE=q@CTLxGf|;UK1bB(G!yR;$ls) zytc5aT}hr1Z?-*tTVP;(_!`dZspPFEDP?U}K9WfJ+bj4heSapGKf$rvuCOO08+^7z z`OAYM3u;o7X~h}anq|EFN1L4_Qm-;(a0$j-*xqI3MfDX*zQ>QYWC{6+e9U@j{WOuB zk190USMDEBwkuG%YLsQjj}obPupJGQv@~ol+aYhRiT2J{=0+L)ykv-klV@f&NFSw5 z=Cn~MF{(JmH_ST*YGS^nJ42Mw)#^RR0VJ0kH|;L3;da(GmmZL}H^*+NRhEUCHh(4S z4~A-qS8@3Es=|WmY|fBvsA!QrOBCB)TL-XSiD7|33DpNU;w?E)w5_4BFx-oy-V)2k zjue(K@REcOM=s{OFV9RhF%_8lFVNHZkT%3J3L>jhlIJdtp3H<&M;$!b4DK2#(bM;8 z!8chp`SRksDNH0D(FJ-kUyfAB1^P+|(cR6vbf)|}riM5gFw{w8Z)4pYZR{*sGJ}+e z`iLv%SIw)M-!!aZrU}xf)h|i4guKi56Ol^#h&`UXCmQD%>Rak1U*j9QB~%$5n!M>N z87A^ynKqS&a9e7cW838inoD=qD9dY1t++Bz$WwNN?E`U8RCEGl>NI&pTA>FhsFd*z zBW#?+Co?QNo(nZqCN;=+?5x<^q6BPJWLNnNkuN~|-NccCckXA4h1Kf}$bH+*RVKw$ z`^aeu^j6X^Io7BR3Au@w$~U>_AQhmK(;S<JCee_Xqjpv2<U^{Gq==hKVc%l%#LqRC z>SdOLkjOEos<IR0G*#>q9}%9YwB^6;9~-Ebp$782!=8)GFAr-GiWcQ(n{$;pW_^*S zkp9S17oFZ<kn=UxPjd^g&g|NQ496I0y+gv_aj_JzdQBprJI}e;<b>#8L5EV6lAQ+^ zPoB=4W5!eSy9*9e&%yN-kY?89XTz?|Hf0sa$vkm=QA`|A9zAJ@UWdbU<Wt$~6uml4 zZTqCd#|4L`=SZO~I7f|jDXuAL$>}g9=81z6%1e-kR?LS(EJ3C(+{X8{e8rWS3rg$c zWT7}eFFggMxl#1v-ik`Io8zyLR9nRlWqG}XkH*!CrkNr#-|{DPFl_JA%ox4WH+`yp z)^tYiu`G_h&qdP#20B15qizztjt(fN1Gp0U-boL=?AnZ{##RmP(|!rOx4_R2;lRvt zy|Ov$uKwChMt|~T3AnDy$p9Ted4lo=G9a1^;Nr;p9w+p&Szk}p`(`nEnptLhSMWXJ z`*yOw)QVvLKntk+pV4YQk$z2nA-hGqie|F(qapMK*@a1%PNy@7v=aIY-9<loA)<B? zJ^_-dI*Gwx(1{EZK2Z=}z2oNiBWn|?ct*GJ$>g+%Po}3?TQUsq7j!qDK)x2)5-gzX z6+U4Tx}a^M9+$~zd(7-cBee6cAuJDcAQF_U8!*g|5qwHB_)6ANO(*OiBRZ;~jCO+r zvX(9M*;O*2V+(mM0@b58%Uf;cSL8jLl{bq3Tgw9kc?ciUfylrMc>0%h++;0C59?^_ z6s*b=NFg&7(wFXn`<f+E1+d_qOuW+XkADj=7uO4lNi&dN6og;sR|v~a9g83g<XBIF zIAX^SV~=iVTd`uph7~K?HXJ?L)^?P$=}0hOltg3N_=mDNp*FTIxGuOfq{x?P9hfXY zrNEeOqMD5Zxdh+XkYN0T#QG*M8wXO2sS`a%HOZ|5mA*wGr6^bzTbt0#Z2ZFnH;yCn zu9JzT^$)J7xwN(pKeeUJMNI1-M3W3ioW<1P#J(}SPJP3rlJ~X}`;kvaKBe~c^t~LW zPflN5pFVvJeY*Fj@1;`vtt0WXmD;D$_n6c^hQ9hf`aZh8eYy|Gq3_jGdlVc=?6;El zQkgfYuMKbByw<(x-k-deL+tCH;-{Y2=aBc9oHx4HZ)Af1vyPukdgH&&IXKM8cCZ7X z*YQsL`CfPxBxz07ff!opSrI=_3}KRRwjUWIvXzVb(J`V=#Qa*zEPidrED@p<rq0k1 zctxK=cxN=`Q4#ph!9(n5&p#!heD7)p%PGr5_S?ZHW^dr8_~@zhp6|_dudb+mV2DE{ zTTFWlh1ggRmGH;BOd~zge>(N#`(5P2vt;ZiWwb9tQs7XXKYw`21U3CQnhrJ4kIN^T zN0{cG+jHth{sl8xxPy4;$il!Ysypiai<#4JD_FzM=F_W-;I~?78>^>B$;y~ym(;kD zK_!D~hPa*{M0)uB6-`$9lE8d2>-WD-#}SwM-xxB-x{S?k&f62V{j00vo2G1|TQAYL zJQ^9%N8LO2BX9Su12-j&tf3oQ>H22yQY_NXJidV;qA{eeHxWV^5hSRDEd2Rc-G!F? zOS?(X9ul+@!T`ejat=v*M#T5X_<P8m?{lBbfuFUIyu3%o-XnjILI=x_mSJq(!p~PP z-@dq7|NL|vbEx6QrX{j_C2Zm=%3S+o{lef4b2H~;jrxB2le{N+*B_~#FvOjgvjJ_( z@>b<ic@^ahR35fQvodAt#sljvYtE(+yF_BI`@86D?$w<A?abbeL^g6p!BF@0l$}9G zrqI5%?uDbPH{6W5uDY3YBfG$*Acq9dzT~)`_>;b_JJq2Z!Z1w&z#){54yL&OMy7bJ z4cQz;<+JEW75%v6qx}ALpI+G9s6UdjH<SztA|J(&!KO>M>Q7WMU)SC(yqinLm5@oP zWR%zG*mL2#SCvMj1*L~Er1YhL^SAs#vhA-~7dcpGkd16W{G!CQI)=(JLVmp=<BQ2- z*!GY3$r7qb*Xxz(AXwBX6KJmmG@53QfO$<O!~TV1MnOx-|KxC_hmp7)ZEOcC^>8q~ z*daO^e1{F+(s$D*T81{I^#u<=KN&v`N(U1q<f|J8$&CEpnOsA5W~N-9i8%m=$gl|% z#bzk?cvMb%mC5aFHy2bZFTfeV_qR6oR=iCT$k~n)Utr905IZPHQ{^J47w+cx5C|$a zum^`r5>=h?iX>xVo|+IuBoM?#G9mGGGUa9E;4uH>o%75_!~|U-Aqd0&-}PDR+3W&s zVTzd&1TO@6xMZPJGRPNGIr^u~IYq4%q9#e%`I<sGfOE7r5L##&rbl6Z9$a%h$j^#P zs1K^uLNEtA0NGWTlCR3<kd3WXC)=Fq-!4MhLNdho6kS;gyfgjNoKmPQ$aXEr&do2B zp=QU20_Y31AzpqQsOIC)MMnXoMWp6%HfQpuIxo?fl;)b0rcptVO*j-l$50HQvBW%8 zej0~-ZLql@D)_fc4gdkBPKo1Y<QRlt$Os{~*+pev?e)6CcwGs<^Fa1Z)%ATBxa#KJ zdyehjve$Q~Z#A=T{*0<ga?iy{5p$H`b5^N><l$cCiZ7HEGZ&jZ<y7*?sv+)5_raUN z-1mu$PB=m*{Yu?A>i+xhWB!!y*q^`cq_XP7q5M{P+fjAIS!Lw81FD_!hmRn#@kn{* zaqAB?-!ZoCZjNR)R|gS0U5++aYobi>c+Zv7S56NZtNr+3*3O)5xh(}P)h#W1_ijH> zafB&9Y(CHilQ&gRpR`Qn>sWoqRND!OW$Gs)H&Li#2bQ)AmZ=h}-+1<|vSX0gs-z!? zS{06Og=NP`t5TrhvO1ATc>dR;uUrr7W&>Q3>m7KtbvGLsTUJ?FT2@(A8WR~A8xx`A zKkXIKwXUkNYh9$W<2aqiF7fhOs<eGBS#rSZ^y&5kr#b#}$tkY`OF3(MH2ZdN$1w4N z!=C54+hPNa5@DxPrw?E!5(Sd<sIEd?pYKS{9X@<sN#m`j*qh`-Xiu0lS+1JQWNo@; zI^>A!7R)N1E}uRtK6rt0I&n$QO*U#WTs7%h@b})NAG**!(}x0pKU!uTDJG+bqWa!n zb9{&`o;~f=zGSJ_nk8J<QFa{woFa$UwjHdDdO;qQ@<*=juON~NV?{!-rtwf^d|A(2 zKp~M+)&J*Te>5HP-)?T(vitI*x??*_n$NUUp%)#WTueTwl$L*a;aAHLtA+J9YQxP2 zCSOx#tWfGDj}usPmbxM+5h?s-*@kFyCPV+Sea7a2Coe5FH31W112!cX%gnijrXp>b zDTA@Rpp@OP1EX%nBqkzG8<(h*er#tqV&$R()G2K)Bkg5(-Y$J<oJY#iKtpDwX*(nB zX7|3GsjhH`p~=vgQI-Nl4i%(~ws=en`yypl^4#1lyH$lzv06ww6(#3KF;cq`V!M*9 z#cMZ58UFCk@w<RSkp7G5d;L<b=~_TTSPkV~`L@QEg`1KqBM!x1uRDzCZhCqQ(|H^b zeXorz-k;y7K9RE@Rk?p|Xa43o`DlCI>L;(R>F(-|v{Q%nup=QSzxj4|RepVe)+{vW z=$_m@Y~c8e&AJ3re9_u{hkdRTG-R8zw-+`QG?zDHpA5!+M@^2lT%8RSXuU=iA2K68 zLKBo6kh0!5*I3->RhyWbRZ&`IHr3=5Rx-xSlF~v`R;K>jO<=|CX4m`uEe3UnA%qDr z7DXUe+7KJ1&WKNox|rE$Y$`d`s%z2JuF*|l63>)ZL~=z5^C64I<+o^>lZwWtr4%iW z&;%#PnoDZUwdyM#=}R;6J}%Z4Yj+3Nr7@3V=dR3Oz)0V>%eE_=)n3*{zsytZRPUg@ z8|VichTq65F;r)pTWX(gBn}(zgzt}NNHQM?K0BspE>kwHz$bVlQ=-`eiH{D(a*fRZ zD2kK1J7(A=>p(cHG#S%!(%}_O)oRNM1UBB7^iYN$Pgk;;(4$H+MrEx&RJo0jGWK?M z_?nn*c6PbBSyAOlCF-KwtZ0UQLAJ0N>U5(_Tbxpa7#XTErsovGZmmqxg)t}K6-rZu zL)j%-lNytptIjJnW#wb9OtZSO0yNionv^`HNmB?l7>2*#hUac;*{t$Z(kmo9lfL_P z*uCH*Yv`aAIDH(!pe?cLDPK;WL!D|XartiLoQ=7d+?d{)Q<s0Dp`>9&nP1N4OBsxG zk)xg6%k+vrnzAc1tIo&$7V~;OnK=0eMyj&2bDVQy!}*ZM5x0|WW?j#D;z{0{a>lb| zYQ+~iW|Mbn{8lAp=EaRP_BRg6q}}rSC9aw^V%^fkOM<Aw=D|@b{K7Kk=%Nj)N{g-? zdq`b?|5}5SsowM$>?=bfS7;`-Os<$w`g#7w{Loyr5QVI3*==YtHYJv-YE`uv6{dV9 z$5fQLP1}&soKs$~y}Wo&!XajLT<qou&L_&FG$cGQr0t806&ag1t}WZtbz7EgdYJYk zhV%@%<#k5;Fr)Axamn0c*lpZxVD|0P>-H<3WCVJh4muqA*j!mrU-!+W(+#-iRd(*T zc9AI;>3iRF&bb`<C$Hk*qb(zjupJ!TzzyLjy>B(Ouzr)rMvo8#5eA(8iHenaQ)*5c z2M}o;4@o+xlYtLg{+w!d)79q144u#a#inFH6$f%}^l#uUXVI@YjE4OPBLo4!P5Lnu zvJAOgKDnFn2YIF}_b&4;@n(7xfPU{!px0<!W~_W4I1${COf&b9DT%yGCm@-14P$Yq zU$cR9Za+GTe)O?sk1ts^7N|Af9c1p}qI9O?fF*{79j8}<q#GjW5HuOUYBPq>zEnRP z5xWf_bR4fPWD1TP%RMfaA{I!7&L4mT0}^J7VN(n=>@bZCVx%k5^3w~_@)Mfko8q^V zf;X?pP^0lVbv#M?8R>9_IBGD9pG!2>DMDx#jCodfa@n$*90N?w(aZ<3bS+)+30(xP zr$sNxdndOaxxxKyro-Sid2)Ks(MulYQB_JhutkIb2z5M%OM;X2x<jS`z_iqzN)~Tg z@g+An1KKGg<)fE|eYN;>;x{qMzrsYMuRocxkbW*B|3d@WCxQ1@Ugpe)a*iIA@vflZ zx@L1-u_9HyiaYY<r;VfK8pY_wPmg@5RIf^1u|fvX)NAw_?|SrTj`<L`^78$_CvxfC z$4<5+eh{2F2Rm_UbM5z&xz>1-gEijzn2k&ijtG<ML-q9brmex7xb>1v^;`Fl@_Kk1 z>goc65Z4OYN(W}dF>x8uTm9tvU_JF+o0RGs$mxT;X)(RVft%fsDYHHTSf!!KGObQ1 zSsm)HQIaL~fcn(?-lo0e9k9wUW2HTOhA&2@?P51;yKGK#SVam~k#a(_V>kL6J~lT` zFUvO@borHJoF0^x;<5(^3zX(I;=o_oMP@U4M{hctI@qqLH+0_4ZPr`lnF3G|XZ(+G zo?rp64OjwOIIsk!RSG_Qi4!2bLKNelwH72p32WhUCu1z8KM`I7cEx0`*D3_yNH|-b zTCOhU5X^8Eo!vP9&@{QtSv+n2szn=-geEA8$EQLrcDYkiV@X|^Fm?D@)J|Q*RBsy& z+*F1tsZ(v7)`;gHU3ng{3NfjI9bN+<kGrBEBR|!wq<Vh1h`6*%;9B`v6kOIA*93NA z+Zan*ypU4Rbu<KM{qW8|Kxj>f-|WT_i?;)1JBEK3S+kek0s^eyH(j!A!qVFR5`B&J zw9WDwmB3alB8e=0#RmrO@+a^7an<$lsR!%!tz=?K>LQNGkJVR|l_>Wed9d%%(pR(n z={v#R3_o%<Zg=ozx{3b>evhwvlIZ7YPS2&g+(gIWTA(+fcb|_}EFo-v6Tkmi3hO!2 zKpR=0&Jaqavx&h4aa}`>$zaYfyJna{;+{#{U$~I75_1};-8r!C8`bHw{Sy~q=cJOY z`lL8le6a@F{<AtutxZmKO-W5tt7Y`8gtiY6ux=S@*9^4*g#8FP8&HfIQd4xQOtovK zI!|NDAnik3urp$~d8wQ!73ke8(*83JgBk=S_cSxucC+)q1cLTR`zPjfv(c2}3haW} zEW=@u_7@^)FbG3{oMRv`jM*m=a)G^h-Kp~VJH{`7n7h+Cqv<j^eM4Iba_aVh@h^jv zf2W+?#~U+lsmvxJC+G=YvTIjnP77E*9<uq>X${fk(dApSLsiU{&p(TuET_k528tag z!!8P$`hO`QCDfp*QCEkTY}GNgQStO!`qVaBM!r^%qsVZWj%2M5;N`-N;nC^j0?Njt zGlXP9szO6EP?)A-Auke{44@7j3n0yKkfe@qy5uHO39IZfofbK5aY8CEZ~7KF<^ufK z9rnvQ{uam%!oftQe|ZJYX#9>+xT+Nh#7=YRcqpb=qgJ^7p&-JFIr@*NGprh<BEo*6 z%(hX@&@Ng-qr!tBauDXKDlASbmJ1qT@R-g82e+QMU}T)bAvZuscb^KD!R~q`NEBC! z2<L>Rz>mGzrS)dr&*TG`SIBM*2UMKQ<ei;>1(`|v@!cQ}4k0r#s4CK`Z%E1Q=_c7) zEWPd~Nw6ANeM0LP<de2(MG={C$?#vsCZ|MjtLeK7`@!evOoNsuMHVrh+Y5gIy{>Q5 zlcC$VfZXuxPYwMIV|1P%!VL8()|O}NOWqd1=xa7)jpXvFaYcY$wkdK}^G9R@qhI`L z4czD{m2vr~J*FrmivxRDomR9yK3cDjk1O(1f(}Wb3(dxM5=Ik9P6>iD5=k?pcCf0X zOt*v6l3`zO)5~sDJ*A($n8WCAtvs0z9nUNgksIa`N4+e~<bNH5Mn6V+okASAGK_qM zN6U$fJoqBM4Z>ezU)@50c^1g}26QsAO(P9N(Ub4}D_N0$n=IkIiPIaxNy$UYc#_Qq zdCiaVs$5f<aJ-?te*d|O_UMCA#b0gpVtyv?vg(-dl$hL@ytHz2nYpwouPn7plM|(m zVaVsb*>glT4Tj1`yJ?>mI(p`O`u=<>JqLb?eqNaO0Uf-Ge17{Jaf3E2_y@}Aa->Gh zp+^E4X|_8(5`@T(ESfCGA0C}K<VYhq0@__F!;TC&pUx6DD!$vli3`n(%`V(g|8?et zfc=ZVr=!#9ms1(Tl8~YUT>aDZZ`SVn_;*?|0D_2-$bfo?^w}wcFtr#iqeuAn>1>|i z<H%j;p+<1iERO7wOXI4i#NzPW{WVbmx&Ar+O;JtIG9eH3lHGZ2N%aBs0eS0_He+t4 z4k-28uHLTRQEJ{^^G)32B^Tk^LVhtMc|N4m4%4w)u#<O6m`|Q}p7*Y>U3o-YP2ThU zVb~ADtEkk6I$*QPr($zUQcKeAih>qU#43)E5djc$b0WQjvB*vI=Z}a*2X0{j5ptyc z$dpyYb2T_S`r#~QQb%SXNb^3}LR{r=^nS4O9I;p0Qrtu)mcCs88P#jH_hoePHIPY& zsEi|(NZwhD@%k5;wHK{saq<lq+25Y$Y%%a&gNySZVk^Bk2$jmNfs|*l;7XvTAKe_Q zT7zqPTDCnD2=G9>#?NHwx1^Y!qEGa)rYAMOl)Pm0ynbLYpTN;an0!p6-|A(?X8nC_ z4m|R4{A}AQGLl0Y!eicrR_SFKsr19t1-SJAr{!1KX3^<fi|tE3_nuxB+c#ve�L| z-OeTJZGAC)8wq-$mi=V#8Sq%@=z0*xC+V2bsOZp8sIVzXvjH8S2RI*(;TR(6N#ez) za{wR}l5ICh-Ywy88)PdS@4!p-g0!%1bnqN86+HpXAjj9zDly!HRie{kVu!$>NXfhL z-JSS*!i&<8IF5cs?YNG|Vrn;f1a(x-Mm?Yd9E&hJ3wfc};HUz`@*j#SB<S~Z!Zz}? z6Mc=;=+gLN@LdVS6`2BY1^BMM{)g`hh$Rp58eCVPP3eSK0*>Orj#eZlrl+U?a|B*G zHc1^7C5tpimnI?g11nPU3)2hbLdQ(UECd-t7q}dAiZ(DZfZdE26677MdE^yK&1E37 z3#P!5Eme>&05T=xzgEVQ4@ER;0^o81G)+ctkOHuT-2h!@C>c+Z?{fT-zgX(|F^%R| zi7M6MMPYK=DsdcOO-OTdwoMXylf9zn>U-Zl>&$YQF?Y=u(HzXP2!r}XM}>=jR()ub z9Eci{Vha&PnztoXV|47~q6gfxGkv4Y>OtBt0M51kOfuk{>Td1Drc=AmApJLxE@D7# zJA^t9>L>ql**Wsg8f75q7D(*z%8+;be9mo_rv$}pS*cup_2i-Bhff@I{rb|Wrk1S7 zdB+!3(4JLPQ9M2m>GY!7+NF*1ZOtvW4=NAbsyUUpo4J%5+O$+29IQ#&sysnv{q>j( zOC#d+6Q67700uWts307!ClPdAqyT{m2aY9N8Z6xfpf->xbc}d_0$@i^T++-~CHjhg zIsJrxG6(3oF+ikclI~8#|B7fBmf)wvI~yS$3Nh~jHr4CA3ou8W0C0f7oo!v<Z`>ZQ z$$Z>D^z~NZ26`<{>D2q~gtGl#0O6Q#-?~=<Mwg`*$N(Y7hEYiirFSiM{212yoA`F& z)%*i_4Do2Xbg4<!H1`r{ZaA;J+4<87nYX`*Tfg|Ex-a--<`z2ntW`?eV=`m261b__ z#ffpUxHE}&wVd>BdO`;5`L#tpW!$B?-~xL6b9L)=rS&fi1NR$6Z9#QwJ!PK3Yc~XO zpEin`sw#KvlI@Dz;a|l`3*Y`uE7=Xx28R!j2Z?{OZ4&Lch^hI-%S}y9%BCjVgJWL2 zVDw0>a^^_NUJ|%l4}xPJNB-*9@C~<>R=rqH19#Juy&S?*FZ9YGFEDnE@o!?9{6Xt2 z*MF%G;D({v9=%C3m|SoJy|ftE__&O;cqN^%v@fpq$P=Pd<%f=4klmYoW=ed5HXZ%Z zIFGN$Skc+2rLFVilfRrZIW99UJ6?GL;P{Jumm%14F3MxiJo%)#|K4&O*6PTwM2n&} zE}bu%bYa20l9J5q5{`^G@tR(tBmTYR)AI}OmzHJ;TRu5{l8zTGtT?&pqWs>atKXJn zl%y3aJ;(%d@y$s(5nE1S%XgQqd{?3swk$;krTbaYxyl{wmt+s-otwyYG}B_XFS$Z4 z{{0%H6g~LxOL$I90y^Iz%&F;ZTUV}c$1Skn3vja8l5MeN5!>Q_n)}<5pXM@t2haGN zm6LCs&Yo%6aZvfwrC-nde4)Cyvb?;KAqvNpixzGQ;YKYQwPe&{CUo;WFE6>*yaP3x zm7~v$I63+(v%Y@m*%LBvOpI=cPqnUDCJ>mK+K4YwUtZ#QZR0ck<b%j4eruZpkqYzn z(_|(s0aEh=&Ia_lXx)&oKGZ?xCSvxmzxZ7VJj!jl<bC|2?B<EfcQ`?YU5?&dLBG5! zpoXg;l+A(TSD`E`g*2W1zJa_?wmFd<9k<A2-axyP`y%~@tW8<m`poQ$dovg!O}>K& zwEms}aWCw+z2oXP#3X9^yY8DSGFv7D?qfSfi6XDxQr(e1eOOX|PpQq+BG-rECtI(v zS)s;|t+FXmV>b!Pmq{I;ibxD`g)>1<cFBm$mwd30bmKb4@zpFrO7?BAa#;pB+WdKu zg7@eBL1`*1Ko99`^T%)J64wW*I7p0aB{dG#o20^t)=*cGn{}52u!jqKz$`ok&|$KS zcq8Fz&k>HeOKfw#qTkbGx(AaE@;BA;>oy=p4I2)*ts|`qSlW9s?e!h~^c0<6<x1Tp zZ=zpp`PPemG{J?hqGE_%G-5IY`^_0MUw~;7kX+<}hTnR?d<;}be76Vf=G;Voi+68t zXhuFIo?Ng>P^2oE7D+Y-AoqA~tKyQRIiO)Px5xsJe}_pBCj38_;2xj!)&ukuPU6l& zn1D!BM5_>r_23&l6>k4Rut)s6Wf5z;iFCBIICya(%WKSzQ`&BlIWhFQi1tY#hY&J; zBPVajp>n4bB`?I0fwN4^=H8;?6Qvt6^sw&r>D~LkMc*e%OiNBmkR_Os3gH`i)NlS6 z=zgctf4Ods2;Q(twr1O==5TJYZKe(o?i`J)rYp$fAvT$^a&we9xtS)NX)!<3rFq-7 zJ?*lCp{<*%xI7|nCEZT9TYA$CE?LOF%|vQrR`>o^q5Z;aQ$Z0}3ic{2Bgjez%S$j7 zfSGh1{@0Rs$lB}VUsp)?dl-21_(GGtH>GWs`}ky=kiabi*Y!x6iV-UfWGoqwK2AmG z$H1icY}R<oy?t`96EdwXkm*QlTtIIpb}V<vX>QJLmbWygrS8N~0G4O+11aU-AuV{s z+rgk@NoHv&9%(9yfy*n1o|eP^;YR{7U8^L*vX~5dIoIQ~l58ekB0Nem`uR6>que$H zNP!o&DYhxV54_-~@Cz}uyUc%iG;OzLkFsM61aL^heyD)V0{7Ksd;SgH1dv${)_c5& zP035pr=&36-cyr2<Q`KBcKIl~RwXCv6HF=0lGCJHT9dqqbjv<{T4~DD7bOE4dydRu zk8Eu9T)A=MO3&7fM-I2P9tLw`FRWYX*|qNQ5n&z4BHh{5!5cS4aoI7(sAw5oLjCA4 z5bT!V)P|8IkPXc$<2F`oZ9OEra^m=vo!b+_tGN58h+PZmo_(EMcFD{~q%DJlc)Bzc zH&!;wkF;*^^xU`s?X(_-f>irFWYWExPV9Z|FLkY|YAo6*zjETMIZ9#;WV4(`Adi{c z--X0JsK?^GfpNywK8I-QFu;(8VR_EM`WZh2`9n}aOkn~7W~+dsnw`HrK-slQqtPej zY8cPMKd0Br>wnHVd{~*At1r+XpQwb4fUt`bdDcsK_5YLI81CyA%VotGLGKM`?L6ut z*czC?x{&cD#?s7UZcAxcbDQiGB0&wcNm1q8^+P{x|1;|xsdPcIQm#3JEMD(YTUcA# zDBs)cyMDbd{<rMht5ed?_AI06e`fp~aATEgd3kL?z5Mh6#G}YL-m|v?@Pb=p$WLU@ zPh>Fu$WsT)-<J98!g|pONo{0hXh>va2uF8FdXF00o7#_lOzb&0H_5v)2zGZDhg3w? z)>c;5a->D_=IIY_-aH-GhXXH5It^v9_ZUzN*^PSqH%H!+oZI@eRz%;Egj7b>bQS4I z221F>ohYEEgoBrd3>xMpI*5yW9}m)Z|NP%~upYErX32*O$nrBHfNn?}U5<2y1gOES zz;%k@I_xA%yw)sT>eY^zSuyyJX^B1qh$OYZGz1525-iunB$4BJ39jC$Q#g4JBwjzU zv|fUkmr(E&2VrZvd@=p-yogpxXc7qimk<>Sd*D}%Q_dtMFlC%Cg)1<u3a^+uAOj10 zG#`BgP?0U%;?d#3!)fvUOR#`(e7Hb121dS1Ru3>mHrA5y4*;DPkqP<-@NcgNSZy6X z3Cr~laHd#DUmlmPu_O209G|gt553<ppKq}GOQZpIbX~_ea>I%2Arn}#zGFUJFShzS zlJ#Qga%`jPC8TvC+c94veR7=KpGfc1@qDB8b1_|SYZQvLqF4v=sVCBV*wSGAT=LHr zoX?Mz_se;n%*I7OKzwks`H)q}DX(_0Zs!ZxM`X3)p%NW~JNpoCA1V2>w&^VFUOAjj zpRU`KQ|Jq|FbVb9AhNtKxtDdP<<$9Iduk69A7zY%g$BgEKSc`G06I&k1A0hZ1t+cF zlw0t>1@Dsul5P7A7ao>lPSdqFZzZ#F)hco$_mzOty%$N?pLr1(SG{`j2VrRZ(V`(A zN^jV?Ii7{LUssuakT@;QBk#Db3>A^lU+igwRKSY$sp=KV%xIzGSevvVz@NJoElO3T ztCD2W_f?;hK^J?==E5B_VBS__#(dsv;0z_?%T`fERzYbwsI*HW5~;#JErKi4L~oBk z(kW6;mD0f~|K!hfI~Lkv`?y4>C&fg|BFked>-lNF7oOrws$5lm3bXPC+!e+%@*jxP zx7Q9R^O5#dt~IWrjx*BynDjt{Z-6XbkLR4zY^%<wkzc)BrM;>wzEyQA<Bo2JOBxK7 z;t4m>v(mEDvvaas%tjG8PaQj?g6JFwn2r%eJF&Yu@W+WaW`a5234W{oNY^S<Gv`PW zn+s+60vQzLN;fp5?d0}U95(DT6ZL6i`bgi9(8~Q42aJa`$u1w$&*Qwn%I{1afUxkO zy;dTErHU|!zL6y9JTv0YZK-zsa{8K?Lw(l9hQ&l|V4AMo&c7r7dV8j-TDeVK8nI^$ z16U&sdHV@lO-G6}ZF|%A%71uJ((tXaIrd93(oa5zj`Z{KP|_5Ooc)1rxphc{7N|X$ zDdJb!YS{$we8U%U^EJ8aT()Nu_p(|U)G&3~oXyL^u(`<bO_Ya6s<h$CoLCXrZ?m#P zC2a-j??sm1-9$H8gG7g6Rqv=x*JT)uB11N7Fu*=c{{95%`6bfN$UAt*9=SOt9+%cg zPa_LuC2Q1Vs*{o!X-!JXlx4${N2JjAQ>R@^D#$9$%Vly+phT6MwfgjIWysE>;lxf( z?7rDvvr{R(RZ;+_u!h-0By4W1MxCHZO4Vg1RWVgb>Z(QZMbVMrLCURRsuYBFq&4cI z%);{0^3uk-24s;p6l?3`bq(6Y3Z?XLMM6PfZY%?}#GUL{v7c;Q$Zc2@8nG&CK^Bt8 zmrluKG6z9aWD}h%9~e-yZHrP`v!Xfdq~W#^Pvv`<;Epg5Pb1(np1&j2?;&P|pWc&8 zcRbuSdbv{Qh`?d=kgQ#{gBx{fT-CT!%bP!cxZoC!NJanUyK24PxLM00-8VAx{OC_~ zjcvBfHivhhxA~zk%>O2bc@M5f74fq)6MuWSLHsN`!SZB1iEK`!jt!+_Vd)H^Ljwan zJtyfs54(CE(cL?8I6vP-*qW3ydUPOtzk!NeM?}t^I9Nu-&xaGyZx60LujGg$aBhuH z9yd0+5bP^ha3W}<O=ZVCqVH~}4k661apUjwms8R3;Z!{6+QjEkN?-b~divq>5siT^ znBJmYpkc=dr3G6KpN0lCcplc@KYZBr@Zo#<!=436P$eNlMV6Mgt?XY@AAVHC)f2}6 z^4^%oL5(g#fv$phHwN#Jr`s5K*i%2r)6=!<e3A-qB6Mhe5{}z-D`8q(^^#A@zn}3E za)ZyG6R}cjMINMh_;V`iH+JerFbJ8Qv{%rG5kY~d9=|v~5e^k^g%cDGrm?Z>*j&3B zO2Q$cg@S@-&l(8pM<lK<i@tsI7e*A*ZtD<qUzhFmdn{k>=WpzBu=M5Eu*N*qfmCCv zk-l>zHZLJ}OHo{I`;GeJS$Vm|hki!%I>%52E!XT=byx}$ma--=CL=a|X=IQ(NWCmB zA~hm4N|%(*7-F+h^|H*gg2cj%qV#PBb7sD=405~1tc-%JtgOtFg%vrKx!={9bs0(X zXwS&aOw?w;`#uc~iVF8y5|@;vZGax~j>;3)$|{eYKXAF_BxbX@8K+kltBciV{RCpP z!{J8EX4dnuY+(lSUgc_CU`l*iLV7@QVn$*{P*ysAO}+(*RS{(wCLL2z1L0+5aZXL4 zx!jnQotsh0fCYkOKcn-Bay@{gfwmj0wM1h1k|c=UmP+{j4_R*v3O<+D&~5{^lK_6l z%K$Q`V}Qu^${NA)H^>SwzDQ`X8#S`~J`acuiuQ|l^`zo)ar6WEK-#mdeWWrcadkto zT%D4l(jfMqrd;p?SvK#D{0DKvj+~qZB|ML<_m8#CaXEo|lkBtJ1uXZVh#w~@OwLm! zcXXrvS`BAA2^}Vzvt(S*f~X8#Dzt-BHCnAMO_#yEy(rNcbUJwGa?|qUX0U^#<(4P` zUA7caoqz&{J4i6Qgg?AH)G7N49xh=;8=^RPIj^A3UF@sG+0zN3LnXu!)`3WpjF%h_ zxb3}*6YgTsF7IjEzmj*1xg-Qnd=!?~Vkpd5Op>3MfB)Hjt|R<y$V|BQI-LLg`T~H@ zx6Xp;M8`ZphL2bfw}(Ff281&ybprD;UZuzW5a=l?0Eqivvg;9Wjzl|k+@fvlG=+3p z<=ms-({TLCNjmcQbVb((5X^V3#P+ohg}2Ty6+3mY=61<qUFuTBy40C94Q77FiSRFi z8jqcXDJMQir;Z?|x3~UZW#ERT8aMGGK`0^|kw>^-YplWSuHE``-n%#NTBzUb4Txd1 zi_K9?qe*nv8dvYl`h~kTlXlwf(s5acNIHW;3rovogw#m8h~6a=5RvTd2@Y8YOQrQN zOL`9`xa5>w4Dv%q+WR*M5{)D58Cd$T`hT%Sv19-=C|05?v|m18FdYC%iWPX+yB+=G zSB~fESgNHzz#9jtg-3qBDiIYC{|JY=GqD><Wb#*kRUkuIMSmb~s8$K$D<6n_{Dz4i zU=E9kD`n~U!I;WHXZ8$rC9IO40Y~Z>`Y*bY4j6oNAR;YeU|Oyq1AblpirOoIMMPTk zC4ni-!>U34J>2>=UC}A{5lnRTWBMWKv5H&MaY5v(trNJuJjBg)4b58R8p{O{>2c^W z!d|OEwbLaoLg0Cc71WTOhp`q7M2PYDb-XXZjJA;NSU_?uo&Pi!UVSZlV#}eGWn6~` zJSf=-@tN`R`1p*p1Z9T@^8Q!GY+1ET2GXR}wd>j<I6-_@q$;mWLvDX%Hfa7W9*AGJ zRn_i0D1$Q~H_{$1XE|W@uiy<ILR_3I?i~^=I(~xfM=HR^PbP?Hxdd7JeynXiS<H9- z)5Ri-ykj|gH1NQxO3wBLSpdkV<Y>Tw)%b)NyC^p<7ATI`*bEJv3a|o1t0M!vfI{dm zv3)@o{QJ`w$*Q_F`y&P4c({lZI%NV&Vl=uMwMJd0PFU%Jm7@KXb?t{>>Njf1B7_qB zfC(OzOO|NK;=hSMrWuX=R|M!|()fU6Nt^B5Boo{mcfu~P<&pO#q`)?nB|R@rqwnT} z@>fi{=iR$Qy30#!575m_eMAN-Ed#}dVnay@a>$?|9D%9-cDfketvb33NrKDKJp_?H zzmd)0*$oj-2^+NGGr61f!Vy;bm5RJ1CnYcfNRPWKa0^L?Z=@n6J<RW9Va69qr0%Mh zPq$L0_sn%4D{UwF90V?$c>wWaV7zuiPcX_IH}UZON+LRO_5sMlq&wZg39#@y4S=i0 zg#^;+H-9HR3}jx`U7V;h0<q1s87{_YNBOH@5e+yV=G39KIk2D){?}$Ya<cb|nOl@E zwjTWaV;OE<*(}<Rv-Pypi^(f`lfAIN?&_^2rSp}7c`Jqm_k_K9y87O|RXaXY;)Dam zmT&yogXe1xAvKEwpVj)&3#dXa&9y^J-4Ry!7tr*62Y2l>pulM#IvH6bI<?X^Ts-2? zvge#sZKvkDvF<BZyQe8huK12-)4?fnX>WI^HkGqe$=7!!LPEw<KI7>!GMN9H4DRVB z_9KI(?QY^>aGqh1=|=3~7m-7e%pR{`M8j-Vh>2l6k;AXuk>3%^LV4N&zseyKPJFi> zRJ3hzZLw`}uhtXhNZYHnS1XBRKwH1PE?H$|#xj91wR2~s<tDH}2)8L9m^oQkcA|^G za);k3s2cNg$U)J`N;0??_-?b%_pGB?a1^aWB-^Ov=q}q3kwKMYNRrQ66dSNY2|Ue6 zk!_lU#10pMmG6@sC@&%7<Vc+uh=X=7QSx(;3<4j@Iyj^NHb$R6=H^L0{s`aeATA?x zcW#^|pC9Hw6h46q+3`E;yKL_<<PXK;F?~4O-?oP=={EWa?UoR)%}C@P_}%y4Gu(fY ztKX-?H%)5tEAlV&D`e*FxU%jWIqCM~2{cBnZkBWGU?;LpVe^%cBlI0nb?d>xBXYAz zuY(X&1i2$3D~(`87(-Udp*k}b(<UJdIhKxt<P80dW{ap>B9-)}y#>O0yJzIx5G8eo zH}De)Of(jp5u-V)$3O+u3+g;F@Hq&wbgqJrL0ICG9Xe|n5@fN&z^jei4fpeksGcQm z;)l{;%U#}qwaqA*TA-H&j#^H;wGJy^yU+7jIzJ)E#aLC$JBn-{^53(znWd!nSkYwq zf$u!{jD6?rS<I0o)FMK{*;1T_nIj(!7g<{<$Li_S5%hT*rvVbdhaz{{=8%F8i+r4Y zGZ#g_&Z&jQ(;`Ae9ktdG?A+_qfBnehnLRIV>so-bc$e}da)T}ufobDk2QMH&svkYa zMyn7Z0I_MD&3@+$z3gcX>0WW-huXa*7lXk&OZZ2uH2d@akFocFi{fhAhgZYQZZ^gk zmm#pj&Zw~)V=S>p(b!F5Lu1E=Ac7#hvvgP%SlFfa-ocK&ml!ogi6$l*O;6OACzdnI zS$zK2pn2Z+`G4Q{`+ctLPC4hynRd#3U-xwpZp$Yq-~GbuM8P%;0rP%o;85%dPK|2< z9r3O-A%yrzFUuBRytGiSmEBQc>NZ$<T+~FO9axdQhNRDshp=6c8g-Y2CB4ywv!~Uh zO^q*m5~L;#2&)D4K|h#Ykp@ptAUi~j_O%1cN~B+i<d-@v3zhgGE7TG;J}=R^v!5;S zf{v_|0_7vFlvjF#p|S}!6WC4eT#{w#21`R^L;9aDmR4a27ITQUya6<kz0KaDCaYN6 zVA~KZguPghHnAxS<L52Vde!>12w>1^sjY3k9RFF$B~jY6O%1Xz@G=o4tQoPLH-Xdc zq~s>&8x-On9iN#UBYY;mxova^KXH;i;yp1XCL$@0_X(}4ZYnLTG>PSZ{GR`Smsv5~ zr=br9Rf*nLdyj1AymtC+i_m9h>4mT8>vYC3x|AP2Au4pXm>e0O9L0P2)iyU5RWw<| zs=Ggy$V|!W$ck0(kdb0_WKO7`{6reLjoW<OBVsenQNnGsA>N1R7Jk5hSij+7iashS zlHcUrv~Pb+6@q}9(A@Mcl-=>cBzEm!GDED2Dhl1Ig-v)EjASyot23*I9G|n@mmE2R znA6l$KVJk24xlw|K8!8XHkLH8RX+5L?OTSPA*Yn->9uu69-y9@_67zD<zs0R8_)<B zQG+D?@lDC+r9GrQhfkj%{IT7?JpS?}Yc&i;(T3RAXg($ulr3TuNi2}cnEc$_y!;$} zv{5Lsl`*85dU6J6UZf->CJ9MN2>5_}Qf79dn2ecxmbN=8P)}my7``0ohB1rDF<Hze z!IWWwSeYr+)N7zQ3Cif&Jlk#N**?N?CPPYQ5}%X;ZXz+9+85KlOp-Ap*~FVNGEGLY z3`%!b5P4!Uz&OGuB_}5#?l6#U7Axs`CNVSFWXeP`p@z6IB%2x#{s&LWV$fV?Cz%ox zQ9*+cOjFBgER$$7C1vqhCNLmYlUN3Nm7Lj-XiDI5Ch=l0HI>s8fU}aav$ITQqfkjw zn5)38nGIlu;^Pw%;>8deT}BNIXu{3r>}-osC?^I6EMbYykGkL5gUg9G$HgXqI}66c zv@lyAp#&LXjoI-z(0(%K0RJxM>5#T^xpC%LJ!U7}DI;v22uDm|^hR?$ED{!TE>f1F z1~(-WmuHB}iQ)CJu`yzVEu)AgF)>C<TrQtyvE&(%u7o5plA7vK3wZK?1fi>~(OiK( zH!4c6j}oG6*#$J7i8AKs3;2TE+yZ1NB=OAmxJX3?eI7<~F)<MaF~+9Pqk1ONm={yX z7v|*^B3%K|Gi0?xlvM*LT~3_IDI+;CJqcF>w@XYwkcuHrm7<L4>XSuZ&Vsio+*lA* z%oi6F6eF{oJ%Z`HU&;Y0q#+vm&X%q5QQHJ!4umOxEiK>|ei#$vDh9<T#)oU4y`c;% zh%QbQ<T+Fgv>Y{ftKUK7zlE4}-D2Hvcv!eBv|4sqXm#)fLSvgO2&<(1!H|n@f@QKt z4e1$~7_>jVPn5Q)f;|7RKjjrns!!H^Dh2+omWnTA9r0;Hb7xPy_sTz-HcNk<rca7A zj<w~&*tz(~$9Fs*=Y3(kVEj~-lLl0tMx~OOSTh7!r{0*6)s#{ZQ=me=ocI`1B(yDq z*^tOc{{%5?eh#&3qCEm8c#d5^dh=wD1>P%FMngI{ijvH+8SzQ9&w}OCV%MdFWa>>x z-8%M$su;&43xL`Dg`0QDtiQ#lyU5^1A{MILzQ4cY5`VI=tRw><Rb!ffz)&EcbE|-r z%2Dm}A8($=du&Nf+9-OL_?GyWsWAAnb=XF6g)N7#TtByP@P-XOQ7UOb=UXt98Pl~I zyM9>-S$bob5n6dhLu!fv)HW)Ool9y=N>pliYIJHOkhLfz{!H4DoH}5cRJ2dmFs`t+ zu&xlReN=5%>n@jm(lWDs(a{aqZD)zkNyv$p6AlX-<~!C?Wz`mO#_p-H0q-gr+Vwdl zt3}eICNv2H5}7s?0#efCZ1O7!QTNy3iaWyqhQ8)xztQZUwgqs8fM?JtJ($U4Gs`pb zjm4QoPGq38A55Yw8ED%tC&-9)GA5+QCu%d<^m1c<x}4@uX#q3(DT<lR(urNe7&?~4 zxHl}R$WWA0;FMCuB(w1be1y~p1VKcCttHG+PBYOujF~M@k~=MUG;Tf`{pfRIze1-W z-IaUh(bdR(&cd0{2JXjdX6{pRpx<=%2_0*WGbLmuIHjkgrx=q}>8!z0m{%(NO~x`a zo|2}1^H_k=TH%bSVLtEAYA9`ga)a$h-c86!%t|&p!PT4rS92<xk{B5pBMzkRGS~al zmhQ=VG)&?MRwqfQPbtZZ?ygRk{&{@@QJ-I>6QiC=cI=@;$&tIo+n%Q;&>mXaW7*rI zy@hBz4;y6uhAF@Gry#F*A~|qifN88T<&=y2%gYX&(Vh(1=TR=?1^Z=zAi5VV?>;D$ zuBHcf+W)SGI1SGJMEB8fkvcex96IE#*+<7{zDHEJD@27lEy}JA$-+Ikd-n-MQsf)k z{W^uJ<r#wox&vusq`Zux?(c#qnVp_2MiRWui!qMq{*l)>P4TX;bgXqT$>->0a`}a| z<C#6I9Ws9oIjnnb3b8FFDO_=4F&bZTg<f%lR6}@HIO4-tP5*wFFy59e8H;5sQ@qXw zy9b`DE+sv~)y4al?%<~ak*xuf^-Pzc^oQUNfpZL((JJ6TR!oNFb0|@KM_wvP0mYht z;?jaHjwgE|2jfEf+uXTXOY2)(&(>ePdUl7W=h7Xs<dek_Ez<LgwGb!VfhOcw_Z8F3 z?5d|*ImGg%#Ku8%d5R-_W1^0{0a_(G<poRVbB<7HpE5r_FUBND?{>}RqM}SWF`{op z^4`ii)#YznA3V}N@_ex1TOqJ6b8lT`ZN<Li3SZUjL5DEobJSADb(EWk5cKG<_)r+i zzzPu=k(C=Q`d6jytFBQsx0N0`sy!Nd$g558uSwrmT~$-H&rr2fb!JiHv`N}Y-qV&X z5~8ES!5sMF!N*-+D4zySq_g`(K;3B1+3X?sVE;nN$36@O73EE}=BcDl<&4S6#$;Yn zKt`>EmNKK2ME*e_C1_AzoM6X`6O<p0LBXAl|E(N14&}qwc`9C?7Gc&q5iI6Lm8BHq zIvw9#kegAaFK`0SR5B*s$+-n7Wl_0KD|bibhG5|2WIaGnAul<lF9^wq(mSnmkJ6`v z<VHDxataVmnyin?2r1Az9d|F#mu2J@pm1`^oSRk@pX=16ux;c%jV(?o&dn%sB7*f= zMzOv)EW_!899R$ZDPc}j=(?5?o*NFj?cP!(*)07?GJ06UND-z1<rv6*V_n6?Zrqr> zm4_<yc+9-a<n<$$zqov034LW=*o^26;$_N^S*>Z>-M7n#;twq`Bc63AFdV5sUoHli z(Ey~Q2U#*gm`cYEqW$~#r^`qrok>2OCH<wMZ7gwk7AI+G$KM~G_dJ$<vf=x~n9`YG zH9D+1_hoj`mxv-=q$Y1$M?U|v_k)QP^YIG6$$y~GC8Nr&k<tN<<SOCO<QOa41*0Om z#ved~R6*x4T~py~tj3k?{-X<+uIccLBD?fZ8Z2kGvh)iMFcglVxk{L?;MYZ724zSO zZG#!&?NA+E9b6!EU62ki=IBWDfRT|&@zG+}1=|58I4auTUu!#{^w0M%j}ol%w)k#9 zc448(oG)7CT;&D%)zz|r#YiGCNKFEDy9y8yrOJvCk1;P#OTI@HE9lR-qg!AiTR%)_ zTeGNQAy41IXdDrxJ2L&um@2+HYch49!?e^NH-9<_j2296I%|YB+#xnQ+hB&Y2cWfc z;NF}x5K7L2E{M*Zmnp_1lZfsQNLG-`XJI4U#$P{k^y@F@AD+K-z5kZYqIa9muI;U= zFTU)k!`xsDaT<<yc_MWjM&A)>$65sB`tfr<S~GLTCu^^sJ6OGUpV;8h<XP{bnmcQj z_b@qb$>|UBp4j_|y3-z3)^~K7cu%1F>p))fT1pfmLYP-DB`aKW7V}G%#fGiG2C{-V zi#fw<%>>aYlb>~QNaqC~kOShoo5^d~ClEPT*os)!#o8q~%Su)VQmE|#htq$p`7D^1 z&`DwU$uqI%`17Z8N={+}(l5nC`86+uykN`(fw=oR;#q<AOGjOH=|m|Mv=1dGbIzR8 zo{K#bb`pn>>p>L=wxkYV+3}*Up#a&S9Y_LuG?BnmL?Zyna|hEyX%4yuY8!V^prJ6Z zE+&3ZjlHOq0}}9g@=svGMdAl7`h({M5~{R~`;c}}YMZ0A?UdfY%zGz3Z{V{Nhj3=* zhg5|0EhWLALXE^<m?l(nu+%!7vn54JyltYC#!GpUo%D)S2?kSXE<o-x=zQX+O7qXD z2ochxk+vY4lHL*0U}KmLyY0Ke5Te^h{LcQZO1lhwqb5NUI*=%tw0?_MO%&2hr1w{k z1bBz$!yd7aNxMarDgA`Bx&IowTX>Tq8R1;pMgv9PA9gvB&PTa}!0kDY%!Pa``Iq#% zw7k4bWy(lQ#YC)x&IB5@IF{}KPM%uY+W`fFC1Pzz^Og4YzG>|T$VfT9ZRCM=4LNCj zHi+9~++^C4U3}M(4z8#6H%2~Pu+-77(Z4yk6%Lmr+X!S#z?AnEX^nTX{UQCv1zw51 z_LcUlyla(Lgh_Szdy03LwmL0sW<I2>2Y@4@R-WZLUZkvWwmGydVpr52r`vTP=KhJ! z=7K%_z5KivoOK)tv9RfMFe1)gRusRxC1F$2CW8}P$Mcn>)eLOgTd-aQsi?bjhYR|2 z+u03ALDVze5s>?>2Ua#N&O1U99J9T>GPd#CyiyXp#UnIfam-5Zts9)+%Nf66^|qx! zA2^YyDNLMSlCO`}$K-2)Vr%4-@()^;9sngW67AY>+~<6Z(;Aw{BsMlDOE0N2vl_)U zB=LOS@rGRokcN&waJ1!Y`KL}a@>|AIYpQF|HYC->L8&(CTgH}#KzGdXTH~n!{yUKd zpY?LAXsv3lZMeM5@%N|1{stLb7k<}qk9l9_KBLNd4fZ=C0_E@_VTGk$rJlv^`CFVO z`7)LB^WLAKoe}+h;C$h>Z`78Et)U)HXT6w<gl&WO71y(uGzF$V^h^f$@mkHQ`O1BU zeaU;`!{2ozgMEqp^xykkKI2&HQ{`3RdA1pt9orPn=4I=PkC*%_^_9Hv>Hd|8Ww0pk z65Aaz)mVQAitn(mEPRT&P6wI!_z$$-sj`2jFJ?!J;QO3>kvLu;pFvNn>kbqNL%CCn zvNyUdk8@piDdB)DSJ!?t@093)+2rBC{VSJ-xPSa{#rD$}!YEFawH_16<m+5SQh0Ji zvMMV)6XTr$3#JYV5xkSaqP(=8M?&h7gxit7_zcm~{v*~>`~LLRHlq3J;DOI8gbd}5 z;+WcIZBy2srUI;eSib4*MGzAF{5@g!?2Zj>77iWCFFJsbdF6TA1TLdG4UM_vtgK9{ zPN@{2<B=Me78t0ty`l6=^G){?Bvla?=H;G<KGQMc5gI_^lb)`Ok2fYjfIig#KIE4# z99EVZN>UKU){jlvmcDJ9_Az~#4GT{X<39$~=2r9igH=`81!V$#RS6pT72G<H4QtkH z-GBH<+kvA;Tiw<I>T?9-Kp0!jKrqyLDFHaT>12N2&tX+v4zxs1peo-)K;{s#9__3b z{Bk~;-|k4iR&e9q3!6D-VD8U9{ZM%I^ZPMlfpkpfCU0LhZmh?N+ut{R^6Txkxh?|w z*RMIhIWt0B_{QZQ7Ikx24Z=Ws(cmjo{A-(-to%4o|G`S_@^ZIBz5-bGdw9&8LwjlI zCi3x8n6bBzQP)YBpt0AJR@=}w$w=*~`toBiEKY8GL^$%Ewmz{gwpOUks>!agsL0i> zDO~cwwDyBq$%^N0ziFR9{aMpS!-fr7+Y{ybG`HmS&|GAt2k4%Iw!7=M@H3*XofkE6 z3aQ5(WnF!8Jr4`!bfqRme>(NF8JamEtZ9eQ$49Ffpr1ZM3FA3ks>~=Y%P7kOsRfU8 z$*J^_QnP#momoxaBVHFi$*Dgn*gBl;Lb&V8u1%e?WcIY_=jYrMG#mPTeeTQaV(-K1 zpMZgnk(7UTE`8MZ?4y;BI(3gUUu%A|-tJt<Jso$qf4_I%?dNCOnHwN%bX&iC{XFz5 z3-Q)|y2rLa<(E+zQj=~>OXuq{%BxfBeaJUoko~~=r0zMl_h{Q5RZ!FJ=zRzoee%N( zPekc;Jx8w70#ZP))2{$^#P6tzQTrzg`8yk9Yx3b<Zj7!`ZTmc|<7+J`hPPuozVS=9 z13p-cZ~S^}g=bpu&GHLN@JsY9_N_}$#T{r*YUjyyf|AKsV(-W55@+_imq?{`2ZzCi zdIUh5YqeV~J_TMvYF0|TnK!cMvQC_?IIc3KXB1}hrj)|>@6(xIL|`(=q!`i+2EmY& zY)IlgQUk-i6IEM0Vj`BIFC~YQZrmlqN<RBUs7YzdtV&RABg#LwLx<&8GBWq>S<##e zijUmzKSm`jJ$?CN>o-leO_`2}D>fL#odpNp+QXkICB0k8nD>bAF42I3EYX}^RZ?54 zJ+<@1j&{gSts*fi$Okm$Pp6hiBg)4DU_lk(s|Sj7$`lMeqv(g)kZ}D9Fam@JhpqS3 zh8e@N!-02fFb7-vlLOC(VA9u}7r5mf9<Iw>+fJQ6jlVVzSHT)#%jC9VtA|J1t~UI` zRu6&drA#^Pa@XZZcd8Bl<+QKKX}5Y{$MdwOcFAc=WgU!zAJQvuF`+kqlis9NZ~&}< z%Vi>ZV2$`b=%BKQh6(%STG%gqWrZ=lQj9zje;f>KUtp-3L+)2q8qmB*KiST4<N!SY z*Xm7_Pz7YTyoO)$;95s%Nrynz&>pU2K7-MD54`My$OH^E7lCr--x$06<jg$bvSXnG z2ZUEjVyrB-{meBuCMM7r;FRuP7+e`AiRkq@Pj!y&di<=Nq5o3)6$ezu2`SlTLpEPp zVlK=Rp+9;=T7_=!WoS42SJ5E_e#YSF?m``dW1<7o16~yBO_w{`Rw{QE1yq9#9%Y%k zLljZgaq_KHMYp3Y`iC<}^=x`XBl%l*cRd0I>?Z<XK%o`ze&{=6jwW?LBqIbdvE9>9 z&37l@P|~S1_u*g?n9tSZfll)sc(w);@4+ODCyRArmrUD!Sxp~<6j^hB8uk-ckjH@Y z4eDfY1X(R$@rRzoMm3NHUG~>>P$5&3SJ9Z-BOt90>4QIw^eq`H)so(QaVIjY<s@CR z-kHtDEUq<wU;PhMv1w(KxV<d(hi0v2nxXT3rp@?wONDT(@o%>uv<*>vJ%o4PO?Y?g z*zB>qN7QDY@elVN^ATHv(*|wT8W5$VhhtAKq(n!j#qeE=SWPLGGNMI8Zdy*RR<qtf zI+(V!zgx?Nqm6%Vy!*Vs3F8wcjCwnJ6z>_mX~*cNM~-=m2mKQ0+iSF4r<BH}jHlt< z?LoroZKK-8X?+Xbn}s`fvhV!*Tf%RAYr)=zpQ(EN$4%nainK?&wMI$F;84aK!GuW3 zV%|D|Eh;f46^TZ89Q9hGjb}X5CU^x4A8#A7dyIBR(bg8>#~-tQ{OPBJA9<h^ob6&r zP7kSqgrkoMhiC1XGxI*z;#a#(um!L{td5ECcM+%f&0ev1s%ptppAm?Ac&lRE{%Jz{ zluL`g-*VaecF6sBl`VlBMcCeH@}@JLhDF#v%2TUMr9x&=UT#sQDzmsOrI>H2Jr6`U z1e@UU2<+@2f%bRg&|nTg1bgzB#j<5TkROsg*M%)Wj6lp5djqjI5J>%g&#(h4)<E`| zEU^MOyS+`$(IHRCpQp0g+r7w(0B+Fn!qzikn%9b@BDIluj4{`kmj+cSLtbvLK}6oN z;_TO6-}7WP&i+k&|5x0s-WX%lBO_*>CznoZp1{9|r$uDqn}9IP{{HLclK`p9`we<S zj9&ZCj{mDT0nfiQolKLa@0QJo7W+GRcz@^U%OgerGoc=FkGyOK@2Od!iKG|f>Ao^( z8IPTRAbwSS?+^0wnd3p8yG0`JG~hipYst$9DpKS7d47B^TUpWOj{LM2W5nPjEj}&Y zkPwe^l()3)K3;JKPH!ZarAe)27;SW7UJ03HL@B}IHOblT2pMI%WP%J6Jg=G#>GRIH zT!B}_R<9^(w|?~K^$5K5*9S)K<m2PhGvY<B{K#GT&F8?{j0jFL@SIAspgDh6WWJZG zb0(2EOnp=4e(qnrDb3&CY2iZV%wYO*GaybjFfSlP)9D|61#n`3i2>iQdy$uy{Uu(y zR9&66&%fG9<39Iu#Hl4S?*HQQ^U}(r^G5&T7~QQa7!#cqk{A8UXmDRa;fgn#$y_K@ z(s1s%`rtc1JI3S(r^Q5*-*i8}<EWVl9VfFnBP(0nM;0#CjX-GI2(@^RL?a-vTi0#` zr25E+eJ`><KgQ^H9Fbl1zmD+B&+t!<Qxh&2kFGr#qn4gnx4`F2e_{v-tb0IKZ!g)x zNRREhr_9p7ojVV6K8W4Exfz3o<3u%w@pqXVHG!WnxuDtw$k;D1K0i)x?)9OL%t&JI z3dg}TBgiea{~7~&zrYX44a&#-0RVyXUu4WNrxj?(_QJ2cmyBHjbmf>;#Ch-^^bIGf z&HI4ffQnz>zkXum9$ZVOxzcw=QhUrx5m1G?%6}`!NOA}x^o6oY(f`YTO=mrvu7Rt7 zo02+Ksih9;x(d|mI!%INyc%&Xk2y)hw$<0SiG;J|g1^_Je#b5Wh*jIZRcg&e#s8h{ z2bb|^<E8uDXF!X7(^b?}414W2^Mq(AH%Gav=@R+d2(q&Drg>Ynu~M$mCfd2;&`Qlo zQ-e-AU?(4f#Ua`R$)45t4edTMT;#xu$-t_POT==CblCe@UGau<wyEHhj{~PXP49xF zngxntQU|;^h#c*J=LI1@u8^%p?2ju))2&@#qK~#j@wTaOBacyob`xP(x@Y^G*>d8i zvyKDk%}>|+0J_|75lyw~*yOZTt89a81050M6fF&u1|2(^c5Br!r&UL>XSHphZIB}! zPKE<PGLHIEUvC(kCQ_plg$NIWIX>p6vO<p$Lxmq2Va)M$0-|7?oI@HU4Lo`*?Yq^- zoMaP1)o(qX-g9Px?!IL0vhD(F<nW(CeEx%)ymBA(y@F(LAa)~uMzk0<^|~i-B0+Q> zhgbd$x}}0LrimHep2@Bug&{@3Wyu*S_=J`ESk@ZoOUcwN2=N7dRMvOl2yfhtyq)*i zC%e{DrPwt}NhX-MrX!xmS8Pp4l0Pcz0_DB;zZnB@+&9=U@4q)f>{_5qFvXh^Oe=PI zu54O!X)5VGoP0E$uId_Vo!n1P?yC}w@FKsdElDm+E=*C;0Y<tkORJ~%t&^Y9Ro1cO zqZ_36{Yy7tVfcw;b+(W}Jsx7nG>FW<hA(*mf53BrNQDjpLTV5z4Up#ivrKQ=*#@xG zhx#xWg5B(FakgvhYEzOyoi3bn=etmvp!<Qw43`E_AL64#01*=2WTiRM9K5xn8l?3r zUa}UzVq|OxtMkhD%JYa1@mDRLZAb&j!8#fjr%#GXiBF9)#-+mmo})hu<ix~ldDW(p z0&}@3AKfcyRgu@z(h;XIjAU}@ZJ~`md8+Zv7~^V*njVPBisXZgA^NDG9qXd$klbMz z+q3-45qUvn(G><&fhGMesSru8J#emS8!Tlt>8&d3XY?4CSrcC#R-m_l*rVb{6;<y6 zi>`J@&i1$<zd+v<j}qU*(;V@%<mZLu#T7?YrdC=$uSIOrlALR*u5s30Tq1eZLuyN| zPOJ<o^3Qs&jGma_x-={-G0+sOO3MkC>}=l%XU4YY7i1Qi+VhhhsjS1Pg6nQ);;#dA z_wj<pz}&#JP$-ne>tQDhRLvL+P9SYqfWfQOr_`q<s==%>q{`JUG}UGw%_Zl)%FE0% zm*!i_Q>(#-2+)N+KB;h-OosafLpu%qt6OS7_PijN5b{o4=(X+9YumG(_I7DqShv~( zv?rVCE%0<%SQz;Jzm`}HqeluLNV_^XvIVj>@Q~sV&s>#zbq-*Fm+yaeS!P9rwzFfg z`dJ5#C$|aCRt2j`G|3(tr6zR4vkr1l2RZ;9d4}O*gJciiY>)lU%4YjJotAvA1}5r$ zwMVIat-Cw5_gn2p0PCp{NhPV`s_<|Qtg?_U^^<;d=6O1l$FyqZ;{N@}U0sz>`1B#X zFhfX>Aq70CA=O+Z`ow`%W+Vq3ZZ56-lV(EGfmRO1%3Klri1G2-00QmFN+B0xE>Cir zM~s>{9sTYkF&UA5F#J~Gu$BKgEbvuXwjQvmJ>}_BTMu+6*nopqn$4Lea6Y<`2$BxJ z8>DeAlXT3Sut7{h=V<18lT6$c^jMK<sHiy*AN3$LP%CqFR*TJkt_)y#@%6vfF^;xy zvzIKH4MWfDCFj0`{vnz$U7GUCb4z9e>H;ALs|DH649oN>@Lv5a!*utlQ+0)ETy5H6 zHweRXtNqX5deZ+TgMXj<VFGp6b@|x#H!0|Aac2HRG>BS*hVNl#Z!YGF_i5LC38s|v z)R_47F>aA=UL#jem^pXy^kHsP5imJyV)FY&m2u@}!)87pB03;N45M~o^rh}^yKs5g zPUV|i5?IHROtz)2x+PmoFFZ~D%q(SEvargxvjl{x=&EmD77MOtd=Y&C#!Apcv~uLF z_dql;;IvRPZ)oWT-u4H(W!nySh>1lycg|pTBv<Tg;K>ozoRN`j6pJ37CQl1)s4nI0 zYr4!|xL`0|5bqlA<b799-a7MuC{B<OU`(sMQToj-Ehgk`@`POa(E6COz0B^6@Chyw z$$My|)HM5B+NhG2vI|HPg~SF}ZsS!&F(El2ymW?Ixd~)5V1(X)285*0QIT}mPC$6o zbjcmM@+5UXMj*ZYMl<16f5N9@o6C|~+bm+RWi;DR=*R3D=_vJ=O!S2Gj><Y8u26D# z3)wTBR0_2TJ4=0e2(JW3h6J0$lk}rDy0VzAhri_k>20%Xx3Q{ENz!h>jvHmnD+2B~ zXXU?T%$>3wu9>uiCT}uQh&de}5b16-I(<wBL~TveHs4IE0nT&nWo|A=HAXz79~@54 zcO>O(TVwPlvv`gkVGxt}FNm<Rw4e*#nfF=BOWF6f+<CY4ZKvWfWIUYUMC@|l9$6h2 zGMRm|%YmVj(P>**E|7|mW}kx1xyubs3w(V2d|HFg?GXQ1chGgFHWi3EW*nVqRJqJ5 zD%m39^{db`{wLewKjROdC_PXYT)v=D{Gf5-apSLO!Hop6C=>ZhC!(U8Md`gF0Q2Mn zz0F2`l?0ZK0Qz29D4&)P?mJbWGg)Gg?lAj{8}jz@2roudYR49})POgYPcF!B_P#yw zu6I){fX-`ktVg;%$G3>`)A~;vY8t+)Yx!kQXl3Z(hHH&qHZ(L`PTliGedBj^d+IMY zd|TfhotsfuMs8^m?u}U9`N-L>iKC@-N2+ZU*hqG$Tqh3m8NzFNo>C}ii;NP-liQ4M z{EFRK9zO7Ky)8Bez)?osj5Yz@i}hf(SZ|aBklwhdnya|ew;wbhAf$x=Y)+eDTT?wR z3~Mbzhc=v^C|d=6lBIWO3E82thIMV_!c&S9AU*)Lzl`D(Wkonws7#6m_#iQ#iA*Uo zDYK%p@)=VI8)N%`>&A4T_cZV+DH&`xft>uMjk8NOF@~g+{47=z*V9Fj4nzfS#JKeN z$IxpKmQwl5Bt|o!r(WSqU;CU3C=9I;G<FIXMO9HrbyY-Z0IG|;J0nA})Cb!opLR~u zS5dAAiHQzP5s|i^;!CzXB|0~hM}mF|BDu-MR98PQz~3KPh3>4R+999_y!qWFRu!ZC zaJl?`ilGYs2)X=z;M*i)-sfP=Ga4aMi+?gB9)475SOazi2pA*kot`G6LvSvsMpgF@ z`pMK@17!+5gF%HK17wrr^8_g*&Jj7})B-Z&5*Xy-@q(Pl<gP0#WAj4<uqXCYghs^# zh%P&xZwiTy2@qU-2(Rp}UPA<1m|I{f;L8i+A}wNRUWuugZ#I+%Z`UHPonpL_I6D+8 zA?=Y6%DeAq*e7DMO0loLVRvakd`yl|q{M}?Q`FC6yVK8Al41L}oq=Inf<m%#BZT}) zCLyOVp^S&!l~I&I&^|jMGdx4os1!XE(dmqLW<-94d?-dyTT$FnQJNH2AjAYPrnrc# zFg_wKF)l7AB|lX*+cJuRRHi<;BqjhZzipf*PXAJDVq9c$4DT6J+9-bFTU)oUVbc-6 zH6JOFTHz9%8CQrqqEPfRTV%czvp>_l{Vv3ich~ILC?=;RCu;|@0jA=(QoIOAm|vJ> z$rTHNn5c-*q!78zihi4S)EyAzy?yrA)$b9=SO<BtD$8t0$rj^$8Pr|TGb1t4n8=63 zMF)w~acvuv@>W$u_fOBf>|Ap(-!O~YSJ%)ECeI!{dzKX>=?lcD0LHA>!_KDB<9!GS z58t`7IJ`>ChhjjkS%wcO6a@h|0DfblqLNXe1Vtacn=kGHNuA5#8Y=X-H*wwf#;0N5 zzJ}*_#UkRapaS}adF)(ecc#CI$jO`fWLXR;S#rIfS2;8mRhA3tGkpi)>z~)S&+{5% zcp`Go%ManVJ}-Y)8Sc78yo&PsC=~UyHx6*Lj7x|17v4ZT#0D^S4pjisWdwpsB?GCt zAJtU(QN_cHhgj1CjGo<#1{Gw$(z^e84McK$y7%_Pa=NiwQcQj`($dp=4FWzZ-6(YD zmEWFpqYCQ)aN3;hetzCwUXp&iavXE?ATY@X4!%F*tG;PZE|USDHC*0Lww05dQtRM) z^1*@2mblww#3jvF|8^l)tZBH4ClyW6je%uCS@6#6jeI!uD`xlCnoAI$h%}Yu`Hf9l zXZEklNcobYDX4gp5Hh%w-Ct3HcG7O5i?emv0&aECTKDaOrk|t2Z~IpLDqi047P<Qe z`=n#eFtLG7=E&X4^sbVa|Kb*`T|alVM_Id%uzxS3>B}m16jnzzB8x&_UtU&QkeC;3 z786X-CVz|Sql)0FL)udZ_nmKRiSe%!wz)C5S^CoO2y+PU8xj#5mK(b#O8m;NB4CA< zG>+z?b_68(@<XTHcyRyb*AMEd5~J$GY06r^z+7+sqctlR&O5Z`N_}N+O%1j>+kIjC zt9x{1{T@0`WV&<#_S10>RkkW+*RR%8Zph@xL*zD7KV<TE4<CLf|4kkuBBXBy=gER* z{SQWCr&3&Hf1-gulR)MwTMq>ha+iFtl)f^9D3?*?X!6Q3CE4sSnm93W)M){^%gW{5 zXRjad_+X`<*Xmdi%(jZhv>(D#t?zMPExs^QaF$f;%*Bglh|aW^a>n^Z9fGq`Vmr=X zfcHUaAXRN1=bBHiJ-zPq$ET0LlD+!OsUOFZVF_oJ5fxP-U}P)VN?p#lo!<TV(zAWj zuFm>~yjOAR@}bg8mm<p!UpaD_e0X`qr9<SUfn|HXSaNOd<+97`uQZi~TyY~a>FZbL zUVa1750{CqvhuS<@QuyC{8@F#=jJO*KR^7`^|WU8EYWM_FXgE1A6z?89Ha_Hs<%~g zbnGcI;4~UReNQ`;st+A-6jIAyPGvNT1V=^B0p;HtxIdpV5THTW{b&v>$O<%33jZ*D zprBEt^hA@QnE1u_Y(+_2fJpXda(=;xv!2W%A>K2E;*(p-vWjGXkv77exwCuUg<IHb z+TwzDw|kf5HV5V6hTtE<q1#cjXN_bOw)L5XM2G}nr_m3_lMk5wGzVGxb=LuYzZ_-+ zR?)~P?4V#ktDw_YeF2<zY&CHBj>MDwoqB@E>v!VGP|qt$=_K9FeZHm~JY$MJE^<D) z8Tql;Q=ynNX39?t&8Jn*w_qoXBDa73b~G-8d<68OeScxUBCmvjQjNCD_Q|*8q<Z<_ zr^wLLbm&p?mhzwTIE(3c6(-xx%v4(%9`0=^+j%;*iVX|&NZi5OYJOmIQ;o4gJmyOO zVq<oA8+`a+R<p5B5416gG1No(@p(L>xI$QUUCf}%>t00UeQ)wF_SlkBU{8qtPlnn9 zsUhWJ1#wr_wI-n<t{`71HCf2oU!e$6Y8GQjp;wfYwq#cG|7<6-gV^HA22&jmm_T>o zq?dIv+p+kQe;(wIW{Ngm`3-^E#CvQ7Uf}-yT}Gp%cARBT7nL5DXf=Ca_<{S3RmIlS zCWn=Y71*UxbnkKr!sY3yP`M}+CCz&>ckv{htwbT%FW<j*0*ZqdX{+6sb~4Y_TdA3? zz`rF5x`5qwj#&b}bpKD7Wvu^4%-k{hmId9tnL_cpB9JyIH8b=``?}(;GTV?HC&U_< zp|;(J5hjJLC^uv+LmBp;0z(fm>*x--H0Tz8#L$<zCbKiofAr4_`bme_Wk7uNC0ENz z0jwldm*t%Dtx)`lEBQL^0<PqN!uWOK`pc)3tqD~g`9ecpQA2~4Hb@@GXZ~2ABaIBL zy7b^n<+aGoON=7Qc=6<%RJJ8nZp{}o>h4!!aeZEKL!(xzu{}XVwvqYg=^1ebL~K>W zTWOnS4d&+4sw*sJC$DqFflht*ytbk=qgWuXoTU!zs*O7ljL(rN-!9Pxhb2b{wC@tq zmp#{BaS7pwh$h1Wjei?9oubU@Bif3R47lIbXJIv5wc$n1n@iy{OhV4rmyp-lrd`=} zr6QeVU5eu_W+_V+GefBbrX$1!4rfQvZOjh#V|~-1-!4XeZV=CZpd7Vn?K|W4uKP*6 z-u=#L*_!Tm&JCd_6nEK0FF#X@e`V#kgneXaA$b{wbbHC2yw&LqGzumJnn-JuRW0?> z)duf6x@Xr>0r2o)2#7i0p1w^8V-u2+6A(JkugS=qXv@1Gl1FqH64wRqIwB`_?yQIJ z{g{sSWb}sEcs<1G$Qd07?#2<LdJY?x3>JWNOL~^*>%Tt2gMV-J@o)aPe)qxdmc(t9 zA~~m)hNp8WX{o6Q$1>aOm_%q?B=FPNgv6}uysN+E7K#bw?~!1WHajajTe!~VSQ6qg z#CAIT33-Rf%FNEp=D%jMvl0?Ssn1cl8Y(6sH8C-spTuhBp(42u;6z0<DKDy$VKJgA z0qY~Sc2`BzES77t2eqjXP#btehH?vRBfHNfmY>hYCuV1h#`Me5I3~-OWy<2e!qF1r z;nGx5o;zjPmbIP_WnnMrzDCVProAQWxLI^ohD!PJs6vXli%_{S4}Lp@dfdaM*OEWJ zB+*An?k+O?Jg8wHLfi<`Oi$1O*=tTbc4ptRzRG<Yy5BuMFvV}DpjoP*fy%_Zf|NoC zIp<g8i3Qc!t%tPIf;R3-YFHD1LSf$Xl10)Jyi&iTN-M*+Z1WQWyxpRg^K%!JmwAdM zgkb{qeHyx(Z!FF&D-X;6xKx}}zpCw)_F&QOrW&ETVSoN{{@RI<m}=Qv!4z#@2kowa zsJQTus_6CzL>k=oIqo?@i)Up!H;t}hx8+CF7nGaQEdo_5lfwfOw(zSwa?1S09aWKg z&T5J8hsxr=51C7FZd^G-`FnEUnlqOk3vUna;TInWY2x#AI7qzSQ06RS_U5-#?B^{O zLn`Q!MddDpFk;tm+jgboP13p1A#*pm3F|hx#%|?<12VG%MLI%Bhx;>DCnYWzab(SF zncZ!>OAhddcZGY_iVg0CA5GEPJjq|2o2Q2x#>@6@o^9>zt*!X;bQ3|bY31~WZH5Ga z8rckQOHfg?3MEAslqJ^lM-Jqc?GlRyGX7f^M=s=NFE81(Rn(NLHtr3+^u3n6b@O*( zfAMJ0#%7^uW6@$4#3Eb8Er{x(mT$?*;ELeBR?D~F5?4?uvkq1lPV+@qW7iCDZyCXM z&XWGTW*5TCC0Ag5U)HH?ja`3n57b1d>x>3XFE`<yz985bt#`7Gr~hU&2eZ|TW><cm z5nx@}h$$aq7GT|=zl37Fr7yzXiGBCcZg*DGaU5Br9zONveG7tT1kG_vc0%0L%L{JG zzoO>0twr+XekJc81T@E@1t6w30`CezYOESE;Fuu!J)6s+O7x}Sju0ET4qV(z^mSEN zDocj};`%@Je^L9p&Ws=Tys~m#9kbQXtLX$z#XYdw!PFM7>q{oV6{0zz`ChVsOk=Xn z>beHd_e&t;h7;v`VsV&^RjccCdA)n>#j<T*XW;j0&G$z`7VTZZQ?TnT+PpeMoDGOK zy*L!n^$U@Gu=?J!4_Mms+UOqzl+wZec<pyNVsaj$?7-;^O(N?gyj9~!KOL^x1gRL_ z4;|jPT)b${^l%DEn>b5+cDz7eVG(~6C(c%WK%M>GN7$@0Or?l61Dq7vXt&6#J3bI* zD*=tiW$n@v^)G7DLy6eHyw;%rM{K~S3WTkjs5=Op`;(v(1hJldJI4ays}pgkjcVb4 zy#AtG!mBz|a1j`7dJ)b#2#~<xawP|okx(QXOdg<uxAmG6`pym3bdGKUIyW5Ve}@@& z$S^_ky`Gc-Z#xMrKz*;jO9$fI_0lKwRZemPz1c;Z8!42Vu^AVkLgg%EXJwj=Pzz5n zCuMIs>Igu0dQ^<+ZSa{5T#1mqe=wv^;IUhS%HGz)%b7_t;Q_6ue!g>4#Z3{prwWXP znWgXxNS#KL!JLxel$ny0oy1c$n~)F-MI!yO)KKQms*%U&%RH^5J7MU#MkC2<2p`>! zE2y~f%|$W8E7!L)NafjhH0)x5NoFxxng!_a%jA+AFK-XFYqCuZ@JOXIgR$`IU{iB5 z0*2g|2GAhKHy;sJ?F2aZ)?ai^j|bQu+8#0i0nyvHX{no1HlBkL6aGVnxUnrw`BhaS zfYuKm4|oD$T(b3FIw#~00yeuZ>0=;na^X(SbiH#YWJnR$&Pp9Xe7GX+;yKRb8EUZz zpyJi*g0_2#U43mgn8nMz-kYMOQ*p-zlK1XhYdH(HcZ5U|5bJ(JhN`L#mjgxf$Ar({ z5uWvbhGK(asnh21)L#`C7aZl!LvHHt>a8MZ+J?|dMCR-vt3f-kJ5exPr9JE4y7BQ} z@U6jAZRtTas_p$EfEnQ=R=0|Ls>aVseq~Uo&o<F><4U(-{Lq!{t((LK&!Ezk*ln|q z&?&91cBHpXSSY!IwH|-}{ku?Rl84vwcx7ori`csFc>ACHgA?SO4lDbQw?E+jJdTyt zfA$=A^V}!;v{r;3=V3JO+{fL}Nfw6}U%iPF4hd=vn?3EY;kwyeZ5@oQW3LW@;9&oh zwUS^A)pFJh8R4>xtoQ+MgeX!f?c${UwgZg3`U76AZCV6&T+?+~K(!&4iug-r1H^~t zvc8eqg3Cn+M7(O-V%q`?a+G}YZMST<<eMaolg5!K8udhHebxG^oQf$AZYiw8Q<O#~ zBFb_C{0I{}KgJ94Gns&Jf?n1R?z2-zFE<=822}|LWp!vlV4*MJ{&Wnk4{7nMRvFpZ z7guDAg-*^86TB80I#jDS>eKbYMH`QJ@9{KFOM8x*_a<dsh~if_Fd@?M0n!+oKTr38 z!@h*>20e2yEhDGl@)BCf%YTUmV{v&=Rc^J@1oBqU1|N5CPmtfZEF2p077vizC_p1O zgF1UA8sF6<;5$s2R(~zhgx?<81ah6n#hDC8&l<9lj`@jBIV`%Ae^BgqOO=`(UzgP_ zT{pm)Q9r_|<x8O)aA&;+quP=0lU>ARo<QDQw&XS)$D`4(M*ufEOW?oy0b<|li$hP& zh!<!N=#z$nfIm$+{{12Thzbt|Vsi=cN)U}ovJs$8bPJn#a=(P0pz%Z{&Tl&T-Yx$6 zXGgv@h)@*&YJBaYV%ocP&i)|KgTKIe^!W4c<BRPH8xO-7QyNwXh}YY*9QEaKMWF?_ zyhUN&h9G{eyXTr@Q>ZaXEL(Ii`gEj<^x8()g|xr+k+lz6zXlQn>SQuU_Y$ah?K$A3 z2C7M`44I&<Xg?i3)d@BS7Wjjg|AjibYl-_H08p_iNZ9-$As99tKnAiGAcQ9Z9OrWT z_8cpZEqlomj9w8;)$I<%vFv-uESi940lAMBP);Bq!9N9v|EmBoP-nAn|Eo%*QH>$B z>{hfO5=$Oa!|gvur@5iGW&ju@v1&lX4yn=eB<PMF(ueF95t@?8Y~0?k=p;`*V<FmR z&!LGP%4mMo@7h$eGH;oR9Co0S*r*5Vf1G#aPvYnZ<~u9iF!Uq;6eghjAKPx8JtvSf zo#vjNOhBW31LxoXDEm6Hj;7675aGK-pk>lPrZ^@fH<-ul0VMwZ>>bF{+vb8W+WtAI zKMo6U?Lww?;mk5{I^58&QMc<zFs|(s;?pQ1OuVerQVi_e6KUJ!Vaz*|5PpNuC}cyj zG<1)7YN2-;afNC(#v%8*0Xp_+fbF2~$JNkSW!KwW+){@@>UB~-ZgaMe$7Wvh^x0u{ zvrpUJZ1EaMOB%9jDjNCD;cR0~kWZF)4a6oiSdw782=)`8fuXVP3@Wd!tthV%;g_u~ z5B3wKfnD3UTS=dUeJc!*Rx@NA90&L4?>zmTHjkj=LdAi$)lArwgpVd^Z4YsKPRXN@ zQ)p4q%rv0Gbs?9?^zVtw_n5X^A}&2}Cexi6Co&x`RJ+xcJM6w^jnK7}UE{uG?b_X2 zj)>N!?2+Aj4uk*S0T`=8^dO})2B70UWD!*go&B(P_mRWyyVr=%yx7Ro@n_C!0oghP z*OZM!%K|mPnk$88{ZOL&nzg&#kBFUKY@w@p*<!5(dmeWLSp`h?$ZkZM>;?7Q9p1La z#@JZf>LpoAb1}hml(Vi~BWEQ`Sh^eIlD%{_xywtdB}QVU)#nn=>Q9S^f<sjfi!!;7 z1$ZZs^r=LrJ&wG(nAk~Gq}OD^_RS?M*f+gN&r9T$Jwzo<BFb^3m*z@f=ZDUAN!)>g z3uM6=zQOG6KacV@#%Gd9U&bK*Lnwr`=vz}-6Ly9M1_t@ZHpJBH>s9n%r#)Ah*HnAr z99`g^FQ7es#H0uKWdy(+sR|EEjgJ!D{{pz?>c6y8yVAJY_QSQe{-B%Z)d-fL%B6wY zu<#%_8Tz`+1no~n2mB~{=m7o5ooKoJDHs;1$NF%;n5gBeF7MePgw_OChg7RVLZZWc z&>{odrXh+iFQ4py^iXQHkY8lT$P+W)szY!X8?Va9t}uSG_2fnEpEvG(eMYD&Z_01Z zYsqgbtf@&YOD>HrQsJBnV&Y7p{BU|B3IO4>(ma!xlUrqki<}|5eP?_xwr@6!0kU|k z8+_>s+Do8zgQ)!yidK9JM6g)$@l-LoIi|Hut7#ZVS<uCLfPnd9vh;lch5KE*9(jY8 zMk9XhQ)wU>5dc+$sr!KMVu6Xf{Y0x#yZq+*4I-YXVB1K0x(N@r(Xk*}?#FA!rO+NL zrwqoKyh?xEPhSzuK>^tT{G`EyCV3aT<?evZ!E@ES2q2OlX?r}uDzaAE2>OqyWGTA8 z6_C{14w_B3v-r`2tYkECeaTuQRdZA0w=bFlGL{g4c9mqz!EdjBzJK-jY!Tl10RW`p zb@3<_rF4g>@m}5OLjRNQvjeNgLr`UdoUYgNbO39;g0Qw|`tk>pgqV<^`0!}e+7IZV zu;*{%h0;SGieUx8=BQHDN4KL;#|kYe&nGWmgu;1oMNUb+>d-}Up_u&6li$gq@O7Vx z#WCgj{BYI92?gjA%eBN6<6mb<0pC1=*I2YRft`SV;S2*YtpCs7OPzt8136NQ5H){V zE7-OSg*X4?LmlQw)k+MldqenoxM)jw2sA)vH*x$>^)oxnA+a5M1X^vifP+KkjDO}j z5IQ^XQ)6iAPikQ<t|mKhJckEl^?kDT!4D6}xj#u?G!>$C0oN2-wjHV{?Dmk5?ILBB z+si_l1hSrODlKagZP8T4MJ6Of39f8pLUy4@!j;__h9f=smu@*5nfPLB2#OiWdWB-E zD;w3FHbZ&!$l)&q;=mqk4)rP#n@gHY5Awu`y?S`<ERkj?rMvgI*=yF!66i8^y-!gM z=G60)k%@_sI9yab_~4|s<sz;Mq?o;WM9%ZJlJu2q&=!945ny-E&LVvK%{QsN`0Ol( z(2w2(`+FzE7VIBABKGJaZ|Z(hcMg$OfKW*~LWkInFi@qq&P^nr*);SsCXHTFlCL{0 zj0D!9ub){M+?qg+i~$PYfsPTLxw3eV9q<_=0s{jICM=UNV47hu6hRs?M(&2G{u}bN z!P)Mp%!DMHidh8YsVMtkQA4MOsE7RtlnM*m2&=if3r~QR<7z`I{Y?T|)UTOLbA}~H z+u&dAV-)CS(#fP}fdfO^w6oefEHXYCat#C6{Mbly7%#8FPjJ}9){o8C%Uis6`fpE6 zPe@M^!3p(mCIm9vfvLvSR4rXY?=yZ$A%VVHztX@Oh(m0m5|aw0x`gfxAi`cwf5e8= zgaqyO8vn+ULUUe@Aic@P<P^n|duptqi-H;7q=(zGh;OQ{X#@plD#W#ylb@LyQ)y+b zHVa~#fW&Mh5`#0VF_l%;X`lIY?d2@7QQ3&~j7o?@eNXGDjU|7Orwn-$zw{}X%+Fd+ za(KUg0+Kodl1Au)RxtFTQe$<&@M29;dQwIbKz&<?jY&79nM_(!8W`6F>oaRL2iB29 zFi+%X<>ZK@nYA595Z_X=mg&6VOlNV^+2Wg*=BB2A{4?39zk_Wv`@to06wJ&fgdNkK zHXkm@kerGDmb>JhqcojeKtE-kO>*NBvl24nGLo|#$&b>@vefod#v9`wvQvpxXEM1+ zzgjq-vHj{`$V|lt4b*H$x<m<9fGZrCc%GpZN(2$jW=!UcoZN1woo0&y%CsB_xdoLS zJv~N0qa#7flqpe8vvF6Z%no5B%jh~sx^V_Wij|$3azHC>%jq@}WbFYjlI<-U0$Dx< zFYi%$fnEY(lY0gSiYN%w?@~(PHgFocG2>aOx8%%8J*C$ec+As;j3nyVWyd_RikwYh z>rFpJ#K3%Mvs`PF!HIa=0BQ!1KnoEnQ#{~AuA~p><rX8DrWKezUpv~Sgz|hF;sNhC z+yc=SG)Kuaa*Rt##V2LjEyxCf5_md23fw_zQld7=m~2c!Y={W-_ZMW=OV;~#U5!cv zDGPKcnI%}uSqvQoSO#Z`&dhNopy!=TS2zn*Z(#c<Tw8WubR}_Jx$H-)PkbUudgUia zkAH%IRrZskD?bu#?<srd2ICE$?96y^d9F*dm-4nLxI6c3Qx<10&R-%zHef0|ZNErU z$h&Bn=_{NqhrR`z=ukS$mV4mClgrPqBSWPkX_zzuLc}B1UT{6};Q>|GPUp@~xr;k5 zhkq7_a0Q-x3TAUH85j3i*cHEvHXl0Lrn0H&+csZS=kX=ncJjJA>9d}^dg5;DgMx>k z(Hla8Fyk0ZYyK|$bJvfjNw4+fH6+>IZQrsd6C#PO(;b>ea=5a_&spj2Y!}LXhgr_d zLv#`d#Hi@|9{AY40f0=bqdX5uo0;n-(>F!PHH~tH`Pan$bgR7WJ5l3z7E^SG79z+b zJ#VZX{FnIGUj)ot19)6lhiyyA>&WB&{kNgN@fyD_f$Zim9)8txCRK?Y=zd;pr8*w$ z=ngAqQ5U2neLAz4<4{R=swJ=Sn4rDkHvDh#{@>({cG8bWyXE8u$#0Cgo@FstsS9;D z4niZ1-`*B(vynPxpvR`nY^N_#Z?1_t@`!hK+VUYCArcnwtpkrpuS#OaqqllxO~1$D zUw;$!C>fX`UzK;rCTF|fLVA#$ux70L<;DNy#Ef3(J2Hv$3k>uV-e&y*D{DpTPGwzX zWv%cVTU!|jS<78rJIMl_R7XBi(}T7;d3nb3<r!E8;P)fw5vNdZ4pB`SOm*kA3psl; z_wCb?(+J8m(9_fextbX?Wsz`T)xxUzJVHWW!)HW5hJSW|Nc+uy2MkFNY(fw?;&yie zYyBOIiq1!<r9X%qg26AXoI-|r66MZc!O$A?-A<z1Mx5th^kLU_>>*LN9e&t1?P2>a z55gWM${NJ+Yl!kNVJDDv7-0b?g&{lE<xL*D+8=5fPYheRSZ(LrX;NB^^cQO}`31tm ze##-vonLZY6X@ZtVdOB=ImVfpsC<ZIpOAG}1PO|h4Z2kS)_K{1Z+eiTd+EQvG+)h7 zC~ADU$Lk842OP%8Yoxr`GT;zHdi~Ght8a6U))uzBiLF!DafRC+AC7Eh8Vcm4Ly;{^ z<A0VK^0`8<?rqP21CAw<j*M@^G-N#aMhAb}O>hlk)tSzrXSr|Mz_Fv;#R5^Ul#{e^ zlw~!`H?IByR|QB>OkQ;4^{L!05~}m~hNU57w+>|Y|Bo-*uTwY#X96UOZx_t^`{UMu zWCI@;=)3jD78f{|q}RD0{;K%m-2RZ@6N1kYCWUPY`XF~J?>#GVy*LAas~&Wc7A*52 z^FCai)3j1({FKRHH<n+B+s`-L$~-9k1Qggy<gM<(XMbl_i}BHQXC{(df64ktGID61 zV3on>3cnaq4#PA3pI>>qV10x{!@Cm=lYg;$IFkM67kh@m5Mn*XonLcgkzjkDUA%hD zVv)Yvl|<fEo?m@rvAk{38f<g!JgQuLWX%OQ_=mC=jvP5Jk1F<pvaJ_+TUtgGKjYPZ z=Me@MF3ya@V{(Q}*2Fk7xt$*(#94+s0T!K4H*oG-l49M(j0z+0xuCgPhHf<*L&Lsn zMokuCqN5|jv>`<iWkTIujNGU3#Dh$Lb8?%wIeg1jtzT7Ot+4ydE<~>MeJ}#%Bi&%I zG>SGr7_4=+pLxv*S_6OLdRj;8U?y4u>n#jFw=k}GLo6xU-&U<HjVKBY_1F>}CQPM0 z>8PdDnWvlSIGE_YL`@7#MMJQ-UXV&3bnTUZ9NmImbQCJF8esiFbOlb?5wv9|VduK3 z1KS+n$5IcqvQn*C`753rKmrqWQ0^f^bWj_yb!^Zfd8!Vn!xJK6VjzAAhEXt7k$Ro< zx{is-ODHPVy6B3F5@PZM%}Q7-K}c~(DVK3biK+~i`s%Wac`{E9dqZIjm|p93GPwlt zL>L3P!IG0*BN?)!A2cbg`Hb}=w(Eu*JoP6__F>9T3R!8pGX+)aNh^}wz^fS}n?g3o z`)XOT0X6_K$bojR7b1^r6Og%(i(^79A+Sm6*^tn<@EDoS&Jr4s?pYq_)ai;5Xmnn2 zLWvykm!Btgx^`O1E7My;tDNLvrUj354>H6ZC)0!AamD}cC1|$5R3ZCO@be9#^6WK+ zvzqL)&H!U`ngM4gPMmlfqKN-LevnB{HF`8IeYO8ygljt;2A|J@v$w%qD5$af_U+pf zfBxA=hw?OOvz)CrcXNkz&-ebXT@xowyoD5@Ve&Ocd;eKwYs8VwplX>7puq{HCT$+> zu*PtZ*rx!+{2Vu)HW2Jwn#5UHJHgV~OEyPEtf};L0*K`^2KQ{?!tNq*W^&=(HDpkO z=e1NxL!e^EY0?JbInfyE;Ti@KT|NrFXW?X6n0sL}g7FAKnLS9y1L^ATFG(E^c%Y`K z7v95mG7cuH5t8dY`B}TfG)XLH0C5>)J>!!yl4De}cE-4lrd%6&Wg{QMZft`YiQ`Ad zoW8nKgd}fDqB#{hF$POFO>8<y<F6SD`Lm#{IU{0PpOy`AkL+FpbI+VIpXCd$84@g_ zOzLG!RY~KXfs|#AHwG_re-mzzAclIeB`2~^C7n|3F_TeEXSC<(w5H8wA^7?#!md28 zstoZj@Lsg$!$phMTwKI^`$q+ah$|Pb0?OD$wK<+n+rC)4jEvg8CtffhSW>TbGjAx^ zB%suvsUJf>8oeDf74u1??z!Pl=3Kj{-h)>T&YS1PzdF5UyWUyVC8cmdm?sQFOvJL* zA*CZDCT{^fjEf_{#b?xm+3@g$m>5hL!RV%`)6ahVkEJe)_4Wz!P7*gKG<xm<u}Zhi zjw)H3h1KlUx2*#QJPibmCk%L(Skm{eO>@2$1J*OeYgXp0;Q!lv_XR9*Y+GGJ8=3Vj z2I74mi&y(G8V~)TQH!Xqh`yylMJqrPHwU9{uP7C&L7Kuq9I4+<T2#)`8JmjxANyrf zLsERbh@o49KExQ18|oCAR~}O#TQOng4AeQiNZD<e6k(WzRT3skv`eGq2uk~xkOa05 zw@Y+dz2KSU6`1Im=vnF2nn*TzoLiO=tMZ7Yu5a7aJO+-gzqQ1Q8F}Z<kWFo^S(VL+ zt%<E(m7a+zI&BAaTS%R?6i?+8_)vb8dcYeh*z93}BJJ=>u%0@!38Qo}C-r$u^)Df^ zYJ}ASLh5qpBPkWK;;)4Z2r4MoL+Q(o4z`6ce)0aHzC7_%@9;0Jg(q;Sb<}Ly!uTfa z3;{ZbVRK{53F!u_o$XJ@n7pFIBEG07D=$y9z9ijGPd8`h<vW4D$c1ZFQ{}M%C#Q(g z6c#qMElgc@by+>%P#x-L7RkykaEnSavui4fYcrgx(`%w~1L0lW=_oPm$#0K6CQ2<# zcDPV@i0ozV<`7Wtb-HroH#iom=wDj|TIqu>Bp`@Z`$HZu5>!HGyi@>51^Pms6)LR| zsS6~5%2_%ZNb=bZ-7|~BZ1oy7L<WW4XLK@Q26J;5ef=<cmaAaNU5mlQ0};F#8ip3m zQcL4P)z9n^1u+WB(LS=-1Zudu)X-PH>TGwGd;H0*d;5q=Rc?-`2;x6tgZ1$-m^X_{ zsBSn#4E$KCyHCU=VqTKo9L>*RgCc^0&Eh_)x;5hQM=H8>B*;@%{vW#D10ag4Z5sw< zcGpcF+p-3B*%?jj-H2Ud?_IHCK|rNT?;REvmbS3;4uT4(s9?i_(ZqsX)WpQZ5>2AU z_!#4v<qSOEHJIdi-~ao5S9W&J%$(`>Ip@Bw`?_eLip-I3kt1B+3NJIXV%O7Ezp^y5 zWBn*ZYq3v3jx#qvJ_|_~kDh3#<T)LBnvM^)rWGFI$p?xje-ojfa`L5{Y`vzeb1TrF z!|LGAh#YQg#=xQ0t4VA@ZD=q1X+yYG=bydK%}Xwe%(1oypdA$ck=Elz^uw6i+Eqgv z#>r{J963=*aYHOVrP8R#l)$`b>!z)F(WNQ4y>Cd@vul}YL+oiUJbO3=>=<{-#^Peo zH)uI<$lElEw>FZFwm7`<d#Dex-Kwi--+cPXj(&6c-KL+^)4mb3+6Ph5fB?GZ|Ns8+ zM(U3v`5*U@oUa3SQ@Jm>CF|&oyx{Q~#S7YfBkeMEGD};5^-#RU9p)6TNVWWK;LfY$ zt>!DLdD)-cxoBqKR5gNgV(Jneh+ngx?7w&V-i9ZxzsAT~FmRnZv+N*HTyI~#{fabe zuHGfcpBO^3h(f&gI6d*xI|V7}mbfDyX3;eM*t|mC_U?&h^c~8apgj%N0hc{4IGsip zKg){rlD`I6;cPRNcHXyf!L-T)*t_5mS{+EgMZ(W+ax?4+O(h0coWn<mZa-pQs~bT; z24=_~o4=O}M!{AoR^X2z-BRifGKi9-l2V8TBb2w4xYx0%!KuMPDy-aL*(XDT_yZWW z#vrL~%q^CdZ$uP6iXKn}r39r0v(#%J2sW0JKvkO^GP39*ItIzFW9D+C{1kkcQRINC zG`TdXge6{{$aGnb$I~9+46+Y#c_pYck&@qL3ECJC*lX5`{Q^XJo_Age#Sf2;{C^)c zIqkA@*(ezF`}DJVI>Mi(YzGDNCRdue3FKaJw1HfAk!_Jn6lWe0D=F?q-M!N?R751x z$!9yr@Cu?mh<G5bM*Kc9dn1`J1jEU{uBzhnqjqD*Z@Bq?-}-hVf$yYRelKt*5>z!` zQ_Tz9^2IZ7%R3*3A0D-dL8GZN$__5(UcCJpcev#q?(lgHh#*}>f~wEt7#+-*Htqjm z6ux}`&~`tvPm`OgFOABx#*m>e!nkh#x1rF%Nd0ZDOqOjum2ltLiYCaGOcJ$9{#(Ts zvKd_(^nf>$Jk8HPGq}IDFkH5xlKOc!C{C5{rnk!RfZ#1B6`nHk#u-fOmE;!{IYs>; z=GIWlF7C(xn}Qf`!!!9<ZX@*s8!XJ@$q!l7)?2n96I$dpTFRi*U)1pQr&dxoQ2Ca_ zVBYejy8{3v@bIJ+GFViY)sy5GH5^<ztlUt`AqDy`GkQ{rr*ElP+wm>Ak!5<(#$!LC zTDDEw9U(?ElF-`z%SL*OmYV1h=aUOOOersI)qo+?PFzb*E<pFy$hRu(<+L+ANAz<= z?$kb{Q&^qez76|ahL34Ev~E2~xVa6}i^O4EUrUC<hg~Dtgt!j|IW<zmjZU&Azw>fl zEjcL$d5|kAMbK%JsHh7+&Lq=+IwRjpO@EN^u5HsT=qG0}j`_?1tR<OA8S1){pR2f< zsn>`SK6tzVt3ccmM5co6Fow>ZLm$!5iE}PKW=Zd-zy<YDJ=oGRflB65%Ymyim+oHA zZl)VY&?f4jT4G#kSi%}(@-r|93Dq9IPtI9SYxr{;xC!0^;r1IsYCqD$xRcl6Y<f<E z1%~TBg7l6CQ<seN?!AZfhA)HkRs^?s@?R@`HVpLy+ihBUJhyt!#=V=5Z9ll}T);`0 z4*t@Nikn9dsj}A-oA@HODkdSzJ2EOQN*x#$7ZJ|cZ}Hodv``i&O`f-Z-l5fxsp2OL zNk3a83p)CGTa)_2F9*+8aK?(PJ-b!udP7nMn;TCjJPK7Sj?=9IR3*C8f7M?fz|n*+ z9zc{~+WT#I)2&qGHR8wz@Qo=wKwsn2Ifa1Z4hz8+=kpxjntH0fO}lZ_Mi{~<g86Yo z1*q0ic!qkr6NMV_w&!rG0j5AP6>K3&sed`_ZzFmT5Q)Ao6;XJ8@QIao7}12p%J~Mo zu|?qIe1xazpIP2$Q6zr}`-L=7^lt$43DbzlshzX``=>a{0SU=VVto11+#jebXjmYM zUM}CJ!C;7@i}a3Y(Y=z)({S)5zLQS)Aa8pZ&!e612aQ{@NZ!#({gnh@tPTzFleDaw zQ9E88799_2<QYSTZ=$``wD<6n_lA%env0k751!&kEK(^u(=TxVVlRvQ&;4GetxDv4 zrZNUYT9#3jrOVP9Sb(c+7(gh~L}qK9772{0;zwKyg|T36=q$9MHjGxQLmFyqTAV1- z+_@G_?JSIiE=*@(=UN7GSz`ou0>V?MMqCj*nOQoKbfL4bbB8#BEEQl-ID+;lzzW5j zcgC+WvTnbssjRB5mQ4>v^YYipP9HX8Gwr3Oy@s5)KMW^ZP>_<E&rSTHbAvk7{=B-% zO0l&l3na!vpdn~MV`1zH%2K1!WLFNa&J45{M$o19((rb{piYe@tzja~2Q?}&!&^#z z1{Mwp#TueRKUsty0sZuo92Bn5ECp=ge!Gzox?^fGGMFcJg2)OBr*pOBOANW_LjYk$ zNr<Vvm6%Me3=c>NeJJ@-gg{k`C>e>+iu71e_ZvYbDd}Dw$lt*(9*W&@JD6>|t_2#} zD$2(68~6Cnml^<sL=aTxu^AKOlI|(+y>AJGj;cR4g8RglZ-C`(MJFJ#K-1n})As11 z29J1yQfS~YI61>NNce`12C&n27Pj(6z7;Z;6yC*GIt~A8+waO05b~z5LKY4wGa@1@ zOzj=z?~4qL6sc$V&OH$<znw-f3E4TiT=o8fy)|4*+XbcYO3K$D{=olg`bjC&cbM}_ zF%pZ*FHAs=SxH45r?auhQKltfM92z<NClG~7n>TZ4us4-2vNQfDtT3Vcjib7pKtmu zT?IBR{$I$%7vqU5aFP&kP1}9?%=*jz#BEb^%^61oI|m(gKIYb#e&q1En@4uuBlbsr zJWrN<|HG5sPn+*I+=qAaUv;rHX<r_!CYpNkSpx|=Nj#g#!?QVO)Az;h|M8;Zq4J~t zPZhi8zil6kTtI-Yd0!+!<y3_?xoh3DU#w2`03Wnip6Ykh|3L2U?7e%l%Vdf!)@EJ$ z+A!1lL{}9w)oAOq->%kqB>Qdkcg^+5_Szd;CTk+*%D|%szx^^^_LY|O8oN;Cu+nQ; z5xXUKPIJgXnN8caKIKPuerp#mTdAd;i@)-^RKy<7z13WNP-gOi+SZ?srwkrEZc4v? z<G*`dkeZj4&-oVa3)!ze(|`s=VWu{kMc4b(y5z9g6;&D4YPie1lo6zq(mf*)QkwLP zOmjy&IY4HLwE=5>f+0#Dkq})RUKC!KQIuSONRS~sDJ(8DH!wFaTUM;ikIP`A4FQQE zA%SUu`e1MuM8!wN%2F!zmAh3LnJFn5+|``hCyMT6>`tkQ-xqy)+g_(aUAb?Kx53*G z?57QqB_P929h&5o5D^B1xGq^2l!~fSvoo^|Iq9YQ_h*5C5HiMTDgf<~JaH%WN$HW} zC(mR)iMtlt;(gEVut)jE;Kc1oA-Yvzv9e?_b!fDi*{<+)poZN3bnQ0_F3=p}L;n*% z4=$HM6s513S!?Kn@S9#kV~4oeZe8uQZ2RV|n>Jg0nRPbj%Y>al?!KO2c5KG&lX)e3 zrH2^9jJmIqiV_cREcOVrbM~GQw+JNO;^NqaS+*zE%RW2;N47i*ZcUOQ*#;RG$%)X| zRUJvHjVp1>NzB$7<Aag{bwNqVso<<m#Y!^(*pDni?cz*dq)>q8J5jAI3#r@{?;G#! zsSDU1=HL|taY6H*$R^Qx>AelUg)?q%xf%tGSccx9_SO6OsiKULnUQJ18G-shT}W|Y zdX!ccmyi$<qWW+c0Wm>Qp-}EKn`1W7EG#Q5HD0UL>ci7R!^0xNqJkqbBK3*dgm^<p zc4%T`Qe<*O68Ju+(2w6IVw?VER%R_pEp;{L`3WKX#58_l!M@ckt}SWAEFJKnk9DNm zUp43J@)D77VQvxaVwPQGC{BQ|*`Br~%JSV=)!uA_Lw?^x=kU{R7Z&f9rRN|Hs+u-P z35xyVv~rBI@A&VZnm#o#CLWakfdM<ioj`<+Gj=d5d=UQ9{wr|$Xg<`uBT*?{Hw&{f zTPp?zjFBaUhbJQ`67N|Tv-s~r9`3p?OU^IU=c{c@KTT359?pB;cvMzZT@@Fg^3MQK zJiFM}ZB_UN*}(xyF~i5gEIvz{osgaJpzgt)3Yo$v|6IEuH>zA)4ApBHI0o=#zcPGS z;Z&!ro%w+kGBS6KGCVvbHIxgznSHPNtSni2yrej@II|?(+Ig1ml-NnKwsp?RQ^}|F zO}gZTzErxxGax!XBe5dpTEex+YhsT70Ytaq)>Q!VItrMO57SX_GJ&RFEXQ;dM}pfG z%CwLi`bm)1A@Wn5V`+F!62yc`u*X{|xAnJ@ft#TAO8dxuN%m!a+1X@J=KkBMxAk|B z4J=Lf$f9FIV`YFDu2ddRJCS-E*~8M<RY{-4yC?ca>4S`u4+j2P+A0(Gu7q4udQ#fn z^u1|&(+vJuc&TN$IOfr2^-D<adFg*v*b&R@vl(Ay?9=Bood~;;aw@$s?LcyVO?q&e zUv}`3slVMlK)TCnb#;k#I@yO*Z1Kt^i>&yG<h?v8JU^;99udu1h=L5GX>(}gH)xhW z1L^au(#*n~q+;2Gc9}9_;exFT(~!+7W-QG~8+dWkofw3VW)O=Xe8sm7IW}L0H4P~n zhbobRk`&9Pk?G3V@~Ena-FRLs@H!=()}Kx}4Jab)24o^C4V8IW1(^j=xuMx9kf2UU z!=~BkIq6v$I7M?iv$9Uv8}otWv+2}k8?{<ad(PH1MjeZ)10m96r5FfiQ>3C82S@sR zM>JQ-kfTR~8^ex8Wa;$!thDBWvn6LL$Vdmm&LlQdg<g?D?<Apjfy~CD?1Yk9-=UgI zwEt1k?=I=}1{i4btOR3Xj!p)q`;;U$G$txCGdwGzGNLlU7?T~Fqm`{sUZ1oc>I4yf z(Y|p3)=_SeTXfrGyp6wd)9iuE=jayd795MXCW9vxY;I+bPyKeT@W<U4<RKqVDucVS zZ4ns{`aw}7oO$P~d_L>$=+QH0jvjq?*7N7Bt<Tx^m_E;EqHGq*T2XdaEGx8O?m^Cm zF1tbBf`6^Z#-hHs-J5<-UBSM<zcShKz9gZeRgF&Lz7wQ{Hg)RLx3mBQXj#kq!x=29 z`eGJ;=sNIr;xDl0K(7X1b-tU>P1uUhKU2ONN>MIOxt0$MRYHGsf88a>kP!SoAn<FS z%XxpPUe5PltaQ<As&YIOfRQ&mClcwQ@~VcDDmVj8#t+IBuK)~~3e}2NP^}=Z_^HFh z&aZ$TZi$8+c1DiGlul`#zvPBdVtUtZDJHabSUwZoQ4OG%F8CYg2_+oB)X{72i+(Qp z?-+|J+VHCleL7P#M&)H1bmEj`mE4AJ9<EUcZOC@8k<M5pSG2(eU@2H4lqW(Xj$!O8 ziV^b7kS9bUgfRBb?5Bue&De+qctg-5OhykLI)ez)7y-hTC<N?h@lefQ#ch@*V+Y0q z|GN~DghtU*{7)%<$)elrhJrv8Fav!8;Pn=se2lWg2;5FyRG1-D9T%IVi|71e!u)+y zHf?>0w;bdwSIKH&eZG5rSRI(%=iaN$FRYKKv!<JQZ-(zJ5xEe%w@2x4k%4zLUkjzb zR*Y6#h~;7lM8&MujhM~-I-6RZr3=*10*0g7g_zBtLpZ^f{3;h3q=*Y*-i7_JZ~?J8 z`|BNU#2sR_jx11rsK$kw-6KP|&Q7N}{FC4WASiJW``kJ)R`cOMv3w!=W;*}({pT0D z;<!m7DpzqJw@hI^V+Dd!fUNW49YCt=3_#7PnN(r!VkX`cv9KS=6)Zb&?aBt55kaGv zA)6t3W2K5RvlSRtfgT8)zW`LXVHGSptO1HjqUb9#6T>9f7%q7{0*GQM%&{vh!d@VV zfPI*uB6wDn;`W|UNT_mMf#q<B;}8&CX%i0>d-8<K7L|~44fGq@MVGdM#WRMx+-~G8 zsZqS+Qj8qn(G!&?1tUcKFqUY?#yC2-`tgqcePIrxHi%*4&-lNtjCfi8x^A7n#5F7x znN_QBi5Qp$lTrECC5pC$Fok%nX7m(kjLcRH79U)31Zwsvu0n@cwh`y~;o^0fgP$T+ zS1#|oc>TLXi>r&5rp$as=jAj*)>4}|Z^ry}IR|v<(n+<1OR4D61r<mXsHgLqnawh5 zx5P+F+?g1F!eL;iSU#gCE}9rB6lcWSAyxD@`&7JaxOZa1Z6U5UL)@8o*nQ1rBx(9! zl`=ecQia@XFx=v(5ac1zV(>~_$K1@K4claW<mRctPI1d{PNIfJk{H%mAy*hSj_}lg zRZi1^Iz90Wbsm4`7j7rG8*v94lshymZ`)=|eg<z)GZ~?2v&AvtRTp|mn~dkG_U(hV z{SvEv`>M_vn`DTi;Z|G_zd%>R1miu|hQ@}*$BTX^tN3{Q*2+i<Ns-bzob6f%hqWAS z=)hBwgrk5B!H&j?2uE>8MoIJCn)-T9+yPTxUvsxvq{HDiA^NnC^nE~-7`%bt?wo1x zU9tnA<V4)^2pefBe?WP_uiDeVrX<FuAeCnX9DO4s8OD^9bZ(!avg)8Jgt%grO0Nu7 z7cZ_M=DBi`j7)tB$TxI*Gb$?%sBFMBJW)wMo;j3evfxtJ(~s+*PAuJd_Z$Cj)jyv( ze~%-V!0mELYT81VDEoR3OaznJyRbpPCJ#`N0l#DVE{xT`Nqi3)a|`L(>P5RJ8DzA7 z&bYa>r;7G`JeTy(VILZ<Rh_MtDIzE%HTgm^T2VfE_PU`gh><gSdx<e8*`R)ucxCH! z)e498)4*~nRYW}2UbCMmX6L1W;E9I`b=;*v?mwJ}uPy-<8%s*?=guq0tFETkqzOFa z&4ZDv^tp<B{u)#69NWH7%BPUmO8WU=iql`if^Uv+YhGgpYq|F!-_Qb5K)<2)X(8B> zF(rjSW!xv<AgvBlmRH0h1dddwL-Z|6&q?TedYl{ruYGS|!;8p!j+~Q_x8x2j5_)sw zl}Y6+{}U3i!wNMWt<mzf)>izH`Ir&!d8=|gyfYv4Y};Bl%7xBm^uJ|jQY@+M|JV$E zSU}!Ivmkmn5$P@@7QOW?CQuUMQAXp8Uy9$Ok+FlidCPV?2I&qRmL|J@W^61PVTkxB zS2Q4!d){-KC#WaPT|2{@6Qah*`6x-rnqynf1!Ls-r|=H`+y!!scE-yU6=pl+!aE!0 zBgwgvW5-I)$>_o`CHYalb>~hbU$%Bwh(cOka+0iJv3~&Q4m~7}a0Hn3!S+}n7NVj1 zP|kMmFGrT-dZlk{sGqmWyOSoEY?%&Tg;K#>1)I&A!<|`5w%li5$@?RXsLxiNgVvGl zh?Qs?bVrY=5Kn3|Lz^cd6cLAFV*edWLM6n03h)!fl&Y`;Y(xjTQRO;n&bGghtRv=b z@COc5wb{dyqwM$;bOUQ3f~X<Z=Vaox)|jK3yL650|8;ba-Ji_AJxtcVLUQ}%lN*m~ zWv}4UA$E`80-ZtZeu7Kn(c*vO{jF$G+E@sZVDb#WkcD&!9IxUZewUDEG8<|kDWT60 zQd$UXR(CMmb_arFVKBua2E@Q-|CYm$v{?|#TC}_bI<I10XkVyD)ar@l%UWWDm_+T% z58aff{)&tKXs<76($n?H8Qk^wrWFfRw0o^+16%w_xjZW?IXRPUjy||?t%~-lMOyz; zvACsa-4d22OMP@c@xH9$2~W+01x&oJ&c{dPD@8{McHhNo?|~yRbzX=NUm>TMfbz(_ zHHg|su{o=_<1bbL#Yt(cC&NQp^RGHbcJBJ3KYBZGh+8aL>bGSRhqd!P+%jF^W$ZVE zD&n}5gao~o|44%r=!JV1pWGrI0l5SWCGGO<Xvhs9CnP~rsjo0pa^mrt^9z`)0~z%P zRU}zTtowoYR23AIln~6~B(H9nb!e*E!67OD`MekX>m1e<tpW2K$WntvVmL9@W2xtR ziWl(j|FD&RJW{-TgsVJMe);r<rCfXeT7I`wK_1~e!~#~>T`Pjj|DH>b1|<dQg4yYC z4R48ca@(wKpCfVFUQ+9aJR13}3t!znawyonk|iVQHin=T{~z1#t~K?SVsGMg!zCFr zas6aR4!cT=h)9XRMG)pmD_z!SZxb~MC88)BNtK)CNQl`l-~3s^-RB208P#cf_Ne#- zDPlO?-cv4)Et0}vV}{X;RYXLxE`mk%-#bn`Ge+GV;Zxpn57>19wd{O`U?nUwVHi@y z)32?C$v{5(skX1+JHB!ys{o1rKR-fd#h&l}P2?)mXkIQC21wdvP`b<U@-TR)9p(M5 z4ST(VI~FPuJ`e>+7B!?FNAe{JF?#Q4#O=aIHBWfx#3o2xvRn$>*<nW?Z21F>WhQ&2 zopiy;6;~rzc-TiW@eyIVF!j<6r!OC?I&!3#BNOg2{4N@=-0I`x6vD!LZObIYgn_nc z!RDrG_b*jmtmYs{V8vwS7p4`eJMR+>H^nP&N@&*sjF)$)vy+N$l+uWPj8H3?v+BZa z4yncBlV?KrRHy(3dSi)OQ?u&!R~K#-7U&Yd`t)Ns56FT{Ia&gQYd_{pMcvu+IE7QU z)?b>NgOuA-2dc{(kE@8YJ9U;W+hDhJ+4>WgS#nBRlee#;jD-?yZ-!iwkblX!_R-Q6 zPU~0U?0z24L~dBCU5Cd`#3Z4I@S^i^vpkD&2I7n8pGUy~+_75B*mRdJtXR|t8Vsu( z(scl_R-0x?wuw1h6SFn$B26TJR6-5|)lBDh&Y>IBAtx9Z_i-e>zW9R`Zko!OYxdI) zPga|<Z2%j`+bUpKLpDec2G+S3u|N`h4w|f()EKq!)iZgm^y!^DPg#Dsq@lk0a2^LK zbtcW2nv<>Cq!}&2d%k?l(XXSq#FCWK5*6Int+nl~l5IP7IYx3WN0aNDQP#Fv(r_rq z9qG5X+RK@Xlj;Tz>;wsl0|gU$W%lCGi9w$dKu4rFBVif-@<oy_&lT4l;~=5gleu=f z;oBcoczk69_}))fu3SEaB|Uf}lMWSy5*6w3`5&}LK1*c5LmtzQ)U@lc>D0^zDPJ=t zk~fUvH8JxUcAs`tQ`yidl)=ETN92eB=t;n}pAn4B1Ro|NKp)_*+L^H<%Y}U-3}6&L z4BGwE+_!3z^%0Ho>WQ^WVnrVUM~4CpUL~SA0-4jf#}A%Wx13zNG$u)07UMvbLUo)9 zyeI(3hcZRw)y6&Qn_t<@bqH{D_2Hlv+JgxV@Q(FXw=a@x-M;T=G&hJJ5dKy6R}o)X zQyK5eBxNNVjjGFMPG3HI+<9Xz`&t-|y-_Rv7$d@=Ac*+-a?_cXGskys$Ysd@;Wa}P z62%Y5aQ&k5aL)W~x?o4`iRBbr(|4lrGS<3xS}$tXX~pbtou3sco_UxoVZvI!TsoT* zuGeDRE9;zL$JDm`W0JvocCDyZvP1J_gZ)|-L_>?>7KJTlM}d{&10JT`@h?-RxLX8k zruez&=J~I0H696c+s#72WedYwN_nGLw`jjetwuN|t#ICwyID*|l>k!RSF~7;lBeHX zd{oB$3~68-Sjk=E{d>qNED{-Udk%R=dk2Sz7W>OB3u<SpU0vfNJh)?<mhD~yh0f&U zXpVSGAu|O`E@ZGj46WRG%UTIBVrb+p!I5E3JdqCth_s;U+D-+&2GOd)xTk)wb1-+c zAcGZq;k)*%g&v{5g8Cy$3B_gwdH2w1Lh~qfSXh!S3?@ukp>dS6=zWGBV_xqVcC8<* z9c&&Fu}ECIj1dM%<6%r-E9C$F4knU&M1E!pE@oZ1q9Sua1MC0CmIuR*vW0FtGIyvI z2#$JWDn&B|I~N~;#2osZxf-$J<N~#w<Te>~mrP)e6d$QNriN=;t-RK>c|lZSSV9a( zZRtD4Da6TVYo~RDvCGUy;F=s|E>>4wx({fiAE8RIk!fyn+X!sKCZU3XoIM_5<aXih zua)*gjD>E5T;eMy=TI+iZUF7d+?3K36U!tN=n4u|ZS^*^ud;pg2Qx`7A!i8Tx{9)W zc{PZZOD>;Szig@9hGiUe#>GZV(OGi5vHU<aH#g_f`djMrXY0;@)ukV~!i<)#)2@x& zz>cRsGuYj#i1kh@@XT&03p70<3(Uzwvaze_H{=Wzhv$c~?fVDIX*X%;X0YF$Zf_<> zHDHe_%1_aln#mbyQ2_)`+mOo$LDh)7P&Mr*iHwem1_;SVD2fl$hQx<jYXSa@FQndl zkyTD75Q%;&35$q?@anXVHH;WTjAuU(rK9G;{Jg>x?l}L1tPrL%QHGrOTs8Svl9!W- z6hN|)pLRlc#Dt~fM;1b=Tw)Zt+YOm%cx5}Krx4?M3xxZAVBG!5b2OvqS2jaW0+iWZ z+p0}>m18!n8_U9rxu5iq+}sl%UCJE^D0N(^It$(_ok5qO%aFZly7UL>p&~YO0X$+F z*#hUy#!uDs<cAbAqMD$=$N-$zYXx&AIwMDm;~Or?$uVSSa0a73Hz8Xl0JwSRy{65w z-Cy~S+>xlxV+;Qp4om#D?aKd~oLBN6$pPFQKsFF-jotZ)#6zB)l&wvVJwC}QGdd|e zE=HD^`1v3@QEig<5!W4zb=PCvHRmT_-JB$&HbY$3@b|i72Z^Z|Kev7L9`U{pemb;h z?&#l|x4===)#PvTR}LFS8j*UvhOQC(p_Pr#o!Kv6f<s|yyDdne@dwWZL+k)_6D&dT zD_CM_BX}s*kTtx&(D|8t2enM4Q%zW(0ol1}8(E|jHX`j6AGpdbZEc;7&{|pxcG_BO zZbw!>eac{Xfm!AWEmXpNu6XkFh!g2tgVdrrJGvTcj2(+FaXXR4nBRz$VN#fg>o^*S z41V8E(sgAZDS7moEPwsz0txvH!Tl~TdS_rV=kX)piX@MKps>(me(|G65F=+Elf}eB zvHwA{iQ^9{&unX4zi!*M_3Ik9ojudocou09u_?;4+Zxub+vd1VEIlihcI-}uI{Y|j z_&k39=i?{u{}ff?kt~p+>^lyc@sBar(VVO#BY;Qh1v4=cAhcc>s*l86FESDzl#`Jk zYDbr{7o4>tv0T*e!`fJ@CrEG=UE!0$3|1b=DYVgM9qV;Ungxit6U_oUj#)Io?oRLx zWZ@%Dfjk1OFBWp>=G{`#%dtSO7-)-%+(JN`-b!I_lZnLPFxe*ZNzOnT+cM|bWD>{w z30OM|geBNk+<{mp2sCvw{;F8qLFYmgT9`qw=86*XC+lhHL;AHElt70jfh2xCCzwkv z&OJ6FXOV2)a7Q#7y;bO{WaG)ci8pTCL(=D6XQf9s+#Z<kC3{GEoIDS&g7O`)odgde z*2;!azeRdGLeAg?r<mv$opbfruNQC0M5s&^@qTk8frKJEAhb7<-NX1*Bup{YWNGTA zBxx8*ZpC*x#gwMu<3P89#~S6Jt4`xglPpM&-v{CH_82rsOoSQqJCIsEf5$hc_?|Cq z_g4CxnR9f+Oqrf42GVW#DBkGV5qIY}_^jVL=ClKm%zU+!sQe$t&Yo>GVBpXp^XEG{ z>K8UR<wjZ^ak{2&ID(;Unbp4s{5ItJ{D#hg+&rrzNA%a68fYmgwVHM3iIG@UzPw~j zBzI3A-TF*)C-JqfOg6+4>0V>oRw$p&xjlC5oH=91-k$UH>FwK3S!i?pM_Idgr^n>A z^R|u%U8+61&I%cHtM+>7H+gwk$HsbjZPI(~wcgk?_txxIx|*)G`cM*UwDQ`kKe>1B zsis@E?%X+Z)@qqySkb&=lbd(e)V35KJX3RhtxW%XHaKerKEI=9uQ#9ZDBdaCNdBV) zjrah3L~ii`uqN~I`DZGYv-}D&v9D%5wOk?M3x1|Q+enT>iRULpnc}961Ux+$AxBBZ z&zUox6AGn*AFqJkn=kLpD}Y<|WBEeq<N3F2*><~*Q%X<I+_FGU7{XgI{^gNi+q~M` ze0AU6s6cRf-N&HxKH8(Q^1Zn=>Z{Fb7r94x_y=&pV8MzB4DgKdRO5xWVQf#?pGMMI zH#3EU$o74&zfylnuV=|}emXf|>i>*5AAWl2+?%wNV^#`><W#{Soe9+kMXC-b{~l5^ zQCQlMY87}Ic?z5Szap;uGOJ@Zemwepn(UO+A(_VOscmv{REl46B<ZC);pIM4rYN(g z%%^l`kn<8^we8J((%<9S^{QK`vXfFFJB3j=1nMKhRJW5~Xb&ttGlGm0dY&Gz1c@;t z>EShfr-Enlq-oYvGT-$c`PZ?V>8S3s@SQX~#TVl&hhI~OhK_C+My3gU$y~t(Q%;uL zjC>asgcCs+=*A)D6hfNX7h8!^iZ4w;q`T?Upm#6L^)F4k@H^^d*S3Yw0X*PQ;qKz+ z;pST7S9hSIrj9LGsf-R577If*JHU_ija6<Lzla}n@|cXA28HFxgp;u+*j15CLUK%c zrYbANU`%Dt=Q@&E{b0ie|85->@4YTU9iL#x%&I+^na$lsxA2ogRHfESw`@s>+sYLz zgpND{z7UO1%}V0JuhThBbX4B~bcl6sT(ftC3S#o{arSkF7<LY2wgXwLQf<oEY~0M= zC;&lQc4E8=^lx!V?7Z+sjEl^{wK8mwb*^#W#e6k*+v4N4+d+?NKJ2z~2l+&O>QqK{ z6Bl-a$w*Gm&Qxa^l4HT0zJSbvm?SZKO@>-WWp1j>1Nj_|xY08qo4rB09>fLwMD?hT zu#C3RHes1KC2jmNei`{^DweY^Awwv(Cr9ONy+mA3Q8LY;a-?Fpk-frHtDERHY$9^9 zBgz!&Y&9M1R3E__j(JW$eMm<irX!MN?NP!I<tW;34V^|8ssuaoy&&ahE`<MzUq_vt zfv1!VC(EtrGt)4}M)EnGrW8_raPMtXX&4*?bFGCT?On;#STeJojCpR&TcN))hIgZ5 z_`$OFZ-SI`3VfMrd`}^mB5;o>KA2(-<(=_78_8v%k^HN7Ten(1;5S9R!n+NeB1(8( zmHaAxh89AhGr)ULMqj^yqiV=oni)j>x4)Tv;<MvVXMIC+$I@A2>1_H2lB_wP9{VEv z-IotYFWE1#`RDX1MSae3*QRk9wi#O|)1HCUBAA-JIgZ>YZh=)eS&2bU#mTFB)xpzg zmqM~vq*IHOSrySgq0c+}LK7XTqsu3*q+LTR`U2OGL-t#Nhdh(^7VaPq9qq<_bVM(L zPNWaK9cVq^c>4~ZZMhCzqq{bY4IH~jiF1BTgAp4C7q(i6gMi8ad<XE|1si@>0GFI! z0MGzll^u_fNcK55_fy)#iGHF6kah*|#1O3IhLMjKkS`Jl457YJ&t{Od*U1+z$;UD@ zkyhv#fYwS4d7K_jbKh~~Z2M>>$pv>s1X3m@vW@emS4>uq8t1uoIv5yc0D_%Ozg8h> zc_@Btoyo4b|HSiW^@Drm4L3MYeoe$<8%gp-zO48wCR^fd>JjwpcQM1lMl$(W*DwwL zQb}xFh<YLUh&}YU{*RX`0xiJHbUeWDGncQTJ=v7-aDBL1_z1dgxk8r-91u_4k)=JE zjJSqW*J)T2g0$4nYwcnx4_hho{XA&oNENlc*&k@nUS!PI7l;MB1_}8+yu=*XU&vQ* zrQanIPM9L0%mUhLG3}!UPHi^w_U)0P93{!vpL*bcYA&FNbA6`xE@VdmLIqhNI{u36 z;Jp|70UPb|-8^amKE<zX=rE<IOkjE9C3~69C%uvPJz<#4VcKA)&(%~i?+VC>_!QG- zC0Ub6rXg~$0_1Gu3j`+CWOD65xphJyE#X#?i2@(^Z)pQ2t%gG6sL9*xFp4NBV!^UU zd^B)}h@sb=8k0YgrrwQ_n_7_!@D9Ex|10t`Cr$Y?8;R9#U6Cg|RK9rKy2XIt{vus` zc3lfgc1s|sHO7&6Z6qPf$$=&C^^YQP_2(N;pFApSOYGA+>(a0jR4%v-vReOo+7EPu z`-G6y_P*;p7l)&5eR+qzIJ*2CfUdWK9u<?}wTIptNLSwVja3dsU~&}<+9xtHPttPs zYzrX1R??PrXBV^Gsqr1|z1gBb#Bj{`a9Hu)#O(dshonpI{6EN$Kc0{-d<05+5c7mQ z#o1sRCL~&!Wy~yeuLHm%GB-LaHZ>+K4x9yAt<|DM)7MYfDcdo2WbknHu#qM8w%quG z)6XorI{(J{`)&{2AH-ZtER}Wg$g_zRfvFw|kx9<l^j)aQxTti=?qv-_zqY+M^X`nN zD_%Luw*CG-fONra{Ik6A4fm?y$=CPp$@J;zsi~~sft>yPg2<w%+~AUsjOge^i>wx1 zW6}~6Qxnv&F<Rt61qCHU<V0UvXti(=lc3chFFq<KI#bK0CNa@DISGJpW~8O2av2%t z&u3&@US<_<Fz7PX*<gXs;i{`63*s}ET(P=#jp5x_DZdN5@7c<F-r9C~&ABz_mNd@Y zC##wLMZr|{lqq4JbGTK0Lzk{zAxq|iOlKwg^bPrW`bJN#!(8GQ5fs9;Pn8B#hws+1 zr=xQUbBj__txBpdd<}N}XV1vLP7|dXM*rgtW1ex(N1uj9l%<L)Hqd{K83R|~yU#zT zJ<*X-+fJ-FHk;$eNWZMSc;q;H_QaJvH`UiK`))qLo!C;fY>|qx$W}0;9P6_&H%YxK zD{6aUWcbF4n2aP@<gvXvq<pPI7-uZTUTf)C*s%4(#_U+QfLpZ0Wqi^ES&}qHPg5qV zk|jGEc3k#riQOHL=oK54=oDq#1!zl>(bo{k?w#AX6lcHY%C=jcGLJjogg;O}_@<!d z4tHzcmBw@U*LZo~E!n<1h2K0-dGOyV=U<)CH0rRdaZulKYOem2S_JN##Kr0o;*(;M zW78rHVHuJ6u2tEx+8=)@dZd2&OUU*o+%2!G+wLrtC1hkI8PxfC`ACRT{1{wPo>v@P z^kINJoWx!aBALi}UJ72X<lf{DFuXh?Ps#PiiXUwxJuWKSgXmjf3pqhgka;1w9ud_Z zkQMq33VDb5@oIiNoHW5lYPgys{uQ2J3UfltFQ{r@`jYr1al@RA{-Enjk2R(rO+S`S zx))ZtRk<#rwoyJ&K85k+#WKZ@WkrXMRb9baRV@Wi!%G2p;+HG=$I>@L5RCi-9^~c7 zYTv+;liti#w8F!o8$^c3&>r5Pf0NR6@j{TDFdXh)VG(~i1VjCUY-V&;RCbI^e|_#x z6Ik@2{K0^td_%gZ+HC`spikR!h^W&s=7+8febz*_!tZG-2jayNf41b^*?+QV;Hdjk z1Dx*_1ejk+d=STbDfK}FO6sWb*MuO%D}5lADM^)PfQHSJ=NE&93?b(KF`ocHv8X5o z<cYrqo>@T0(XcO(Q~&=vA?&}0k&Ju|9%PvE4x`}z83yhMT_?-iUXo$T54j#_(pHEq z){0Jrx?JncC!#u)?5x2of)AD;Z)7EY;tz=&m|saSgG3Le!=2XtQ>6{_34im0PF?Qi z6ILH85mpE*tf)7n%27!JZODr%)#v3}11D?*eTHlMiqAAh#p_inCvkwmM~~9jNTNpr zG968d<$Mo(we<*=19t+JKsYyWzQ(TD*iO0CAtT$7YyT`=WBN=Q<LHpERo&KVK^Qqb zNJ3}`IY?H@NjwM;{VkBGro?9~`CbkRmCx~6yF@-n1N|F>#*AQnyk%o?Ux~O%Kc+au zH``Y&7+WM`G-Qm1TP(C9+Qm`hC=KGAyLV?7BQAjz!7bUby<-^CtkRKOC<TZb-VBjU z(n#%q#qsrx2?^oE1bpyTqO;MDtVQQz2AKj$(|v`V8hw023-IH7L#K32r*djYZa`{K zoFLF-D?uR1VVPn2uz2fsOQdQ0#fJHbK%Ho5U`|j`6dRwDo0tcfdrEp1$75X{8x_VE z>I*Zid2<a@X)LU)Qu%A7WznUfxu9wm;7^2TUMMcPTojZGWrTNahm_d6=ZB&OXqTbQ zbO5d!zicetOFNgL3(FgEZ<=vK9`$LKcf9L^JOc8ACMq*JN6R90&B!FBXC!B+zd$dm zS(+Q46K!Atq%bn6i3vatbT2ClG=~qG$T83cg2Pk(*Z=pz0CE^zE_A)<VFbw6fvss4 z2B+OvAG&kKrX?$)qN8Hc!$jC+dU!!}Vd}oLy&2URWJWz1@-6y_5?R#s+uBw&AIGC2 z#MDI^NK3fnjGcl~m=XuPdzOzn_d5&Dx;}q>33&AOfa?zja72g$abf2%fH$yI-X2Bu zHj>xo`Zn<)BflwypWxU=Y?FT~6^sxG!kIN8ijDJb!hB~rZ)^jFiZ~-Y{qM?8EwIji zw-W{QW(1i(w2^GW<F>yoO_@zxrec^fC4&ZL!gHgTLJMR?jYo`!)ejGD9vRCetll|k zJ~fk3vw7>+x~jK2|3D`1;G&xRNiPqw$&)Po0=X|yYZ4}J>NjHQys5LN%=u=B)tT1D z-MQ-X&9-!Q6S%U+b^f=N(b-qO8~Z{HU(ho2&yIkg1O4&6=r(v}lFwzLRC+g&i)Q&x za&kr^tn2t)NpH~$@V#6hKBkY5+IX5VAt%9yo@T_A{Y{pyhQbEq5`T=~8}RwpVbRu+ z2E|!a&@Q8`$`_L6mrSjsc^LCTlIu2OBBS`RhT^s8d!g?t-`zDtGUEpZo}xa=B}uN! zxhc}PsCWo=he@`JNe-)pPb5L{y5c0342fXI33g9G_}rSw6sKkwN>qGrX%@6&+3ARO z-;t0np5FqmLbrFj=m=;c1u`uuVFiwA{*QLJq~1N2+%jUbtaNN9k><x3Bh%T5<rQxj zvPUSA1jQzm=-6|y3_8}M;n$l8iPg~>(>&;Af`GHj>h=EHA+K!nD_wMvZZ`bEdsvYt zGnq-(7d-so`t=_kF1S8%<$70pKUQGA4@nP>N(@1WM<}M7;^~5AR6WA_@Q(GBtJJg$ z`Uzd8o|u2#jf?k8baz)Fo7Due*2Vl1V#0HJvo5hVu7<r@ckKGNc*iRM<9&GjI`*+( z(!>P|CQe##{Rh@`h7#rQ;dF8Q8uc2wIP=ADF1$crQIMaXU!l*BkS)6i>Cc~`cdabD zbdmc|SP-rc2oIO($TsCf)PXwj*IDNzye+(z+=hL9(HmZuK$|vu(yDl*xOvkQ0=FY5 z&?<-*FVB<d`M&xcLJzjR2Pj$8$G*LA2b9OXF<|gXr;pPxu$u>grmP|49F_8Yej?M~ z%J_dt6_3D`=+HhXEP;2HwVB8Y2^qVK44h8j<Sz!Jo&n@87u7;q3Oy+FUMJuY6jiGZ zb%tcR)>{09ifrB}=ik{7Gf43v#KT*P(6mlc0wv_gU=$@bQU|oAHvEjuXaV8CLEFG- z#1Y?H(|*uX{`S^f{}u#~FY(5WCdo?pGW!9rGo03|g+-JQ0uRO_OfUuYNh-#}fn*Q| zn$}(n=|7N8d_-rf=^5x(YVmy<LbY0Q25sN0--zvR9PvT`MK}ZHZJ=Q_4EY-3gh$bP zd-vbLBn*=<0Om>3Iaqo`hJ&b0lo;zCgJuGeN*nqP<e5m8v0?0j%xgskT^lCLbOSml z{C=kWB}o5=fzzs;Q?`YXFdMqf6vni_hW5&l5~ZLNI{3ZdBQw300ze{Dc7$Q%&kHvg z(@WH7GXB-?n05jKCNSn&^1GEpF7JascjYBH1z^r(K|TU;sa#sND<P64*3!(qH5vO3 z9SE;i7k-~!$&nqv+J?A<d$>B|ecH7vQR~eWNlT1*rDdJmYo5Noo`HEmC9y0tDk67f z1Y)ELF;GoA>c*I5p}ajFcE45n68s^prcOi>vZkIv?XMG!EPG?xrKD&vV-1lhF<R>w ztu`h~1&rZqY3=FiuPe{Xh*{Gq()E`5y<|r9t+g01=4i$}?)L$R)K@}B%%fu{yOis@ z35n73)gVgi;x*_YV#9wU5XeWrW1O@X`p1$Rr)ZbHCppSqzKML`5o)C6A<$$eC#|cI z4mDUlY?yTJM%Y6$d(Q8?_t);HWv17F6h;|hvbC%(12k@G10?AYBEkVP*%=sxsB*M9 zF&W6>#7UOJvtSWvDp1~AesKoia0aBF8uZe87oj^t=Jx>?59Au@tPe}*f;LNjE<zQv z5bZ`O$2(EgKLLoBpaRdtz%ql3!1#WI4bw_Sydopsk`Z`{J^4k-&&H4cl<laI;8~WU z8AMkEk$+5Yw0PS$ar-<^=^^K~Pk+jD-`cZ_4$du>5!*Xt{Cm+qo(^ZW15Mi)XCJ<q zi*zb;Xo!#{0soVwuH9E!o6TkK%c{w!k=^n>Gk=PTjOYWh8yTERBY^C?=t=YN2Ha57 zd^~4Uscs@iH+bP)nnt&&XaKwoi%B4hyj3&{BVj*4GnUqeNZd%5#lNzC2kf(5{9OEE zH&wdGPR^^GJW(~lZ_1{5te=a~{(!$MHV>k#@C5Fz%qcJ6T3*zN#D6N#!jrL^$%wI} z59@bulMyxe$JnEWTb~|+A07iS%k8x1+<vGfd(#2G&RX(2Ysp0|Pst}Ia2#V*7($cF zK;M(I<_B$<<9NlskHm;>*eeX?J{~$0-yfkd`xuh7ui!kP5oEuTEDa@_1t-K;=$F5H z|9C@ny#+@!fYp=!`nnw~tszT`PM;x~BV-&I2VYW@FhQ7ri;@M-taQ?4AURH17GEHB zSOYb3Q2R(`(qXv!!}N<UvK1%w`yoi^gH6%7_M}tRZ}>s@nBNQUTlalU&)C3*sHRf@ zBf>%0hYT-eyE`FcP~<AMgUaz_Aj*bo_*Q!a{8!1*H|6d6{NH7k7&i2k1BK&{)<x_I z`R8k15`S0#1lwQ(G$MSpwX-^*^+R=h>tEG%ZYnnNSfP_}v#m8>LmRL)-%27it2F}N z7ooL33@x%vJ6S74{EFlu5UVz(c@h^2bqYgBZiIDYZgE_(8sPZi;w&)p<mMZSk=TK% zFKKEMy?498nTU661=<7{c8r5Me<7>X&D+;KksH@u2-haq3f&MV1d{xfrXGd_AOk0y zI)c-<5aMsq_k;68XVr+~!{Oja#Z!hHWHfNiHjr7>$}g<C&4Exz+CG$7meaiDvN}m4 zx!ipGQVvI?rlAZL$>g_JU6=!J&-V5PWfC;<)NZ?~>U5ktZ>u{{U2`DK`aoKZcbZGB zU~84;;_cz0lkuZk$a*=@(YBb7cfus4n{JnnTj$0uY2Gzy2Wok&e4wTp<hi<P51k93 zeVa^;%CGa!G+czU72?jMRfFlpGmb0rxnG3QAi|Uk<d-s}n@|6T^kX<;#ZgF{98>yn z|4Fo)4>wT2Vk?+<oF>khG<;|{+WdHAeP&9KbHR{I37(Y{WvUqK&5~tmV>4pZphHwc z)KmQWP7)4LJ{`B3`s-rSVhnNC@djf8gj-rb%8jg3ERTwTS~ZrFJ(|CkOruvZlMTlV z36SLHW#^}J-;?jfef_-z75M+pCErO3uv!{-p7^I_>u@C2e;>(*qr~!Du^KE#uhNM8 za0wEr&EMNFL%W(D@<GaEbT?rphmau*9Rj&Ey)*^Q@L&*U)7p-UN}fo?VB^SNV3Y5P zF<nf4`Guf}!6IH{kpG%~jfVMa2GPf_m8Okg;9n-F1yQMBS^LKJjS&1=#;Y*^0GlBe znKp75W+GvNUrB~_<$ocJh*y0jC7@Dc=u7~oF@uuM*?#Db5uWhELB5`8x0m9`PQk-t z#><3mI2dptcI!+fLb14*7grPe&gF0cbQnc|KE9yjq3<kd@0@{R#w;^X!5F`mw;N6P zZZjg6cbi6pQ|^l%*4C<2*qDuhmmfS>F=0_03OkUI8_fU_5g9>tB8ddl-Pwg;!D{f= zFj+YndHHZtpf|n^h+7-8C-O47)JEc~)BIt&jdRmW2hvNiyRtnhL#$1FyPTmvwCR=P zhYmf?04It$bT~lD9bL0kAMHUm3cQt`ca*lh?;|d6uj|m8c$2)cIJ+ixkM%%uNl7>I z{D+mT#kCpU5l<@r1*yS%`4S4hz!>AXwFRovG>JY^dd!;?0>XOdWIE+rYW_O;r4^Bl zA=9UjH7So%Zf8E;CmSUdz9o;ak;xJp@y1#uKNaJ)SAPv0k>*1c2kFOGK4n)gcAGj* z1tpG+^b3*%$9Dg3iS#~Ol3b!MDZ$^z{i*am=|7E3R%7u-P;_p8?Dk-F3wPz+L70Dq zN<`;tVLCp16nuY?=mB$Tl7USBUoo}p%IBIGC9J$9$&m003;a^xmnj+jQ~IkOyt?F9 z<Y!pmnp>J|#WnCtfnP-3?xT!`j5qj02TP)3Ar)z3@r^XcXv|@2K}d?ne+QWk-md9T z7c(;YS}cl<1~huGwEbn<3nhkNLm7Ukge1|SN^n$sn0XYWe7Nx1q|Q1gEnGOMbNxxz z7Cr<n;2X)qYu6s4n;%Uef!2i#?M(*3y%O}!#v+Y$S3E*qiL>%KxB+c}TxZ4;W&-K4 z6m7f(&Bxy=@Kp3B+M#6WM3AH`MASwP+Urk<JwOHDKnCG+(v3y}h7m7ASO0h6h>{54 zes}>UztKfxKRsmi2Qt{ncMMiupTw`QvG~)5PXd2k`>r7Rg0$1aptrO|=8&z)SPL5Y z7UBr+$d<OQ$}E>aSJ$|HzJmjXM5oi|^&=XonK95R&n<L^Tck`EAD0}b_VFvIis9l) zcir*19!fs8-X;m9ANwrWH9IcI$|uGz+y_hoDQWSXTV7~=UY)Vbs*#x0|C&yo56Kx^ z-fy>SR^a}u16lj`mmP?cxnjiEXBV-=%_V*I>?fabSQ41!Dx+`70EkGp;?DBc^ai;h zSVJ1+2JM^@OnGa-eo)R^BNUC626U>w(cgqA!W8<?l>CO$72sj8#C!Y?R0lVE?Y%(0 zp17LdAnQyk$XawtN=!SI0TrG(9!Y{U$O_1c@V)ypkHs9ej;{`{@+pu(vsDO#JJP9g zLxQUZjiats4$g@S4sSiY^?Ks5BXCuYvm!%mX%TIv<{?8id@&2Kb;>dqt~@;OTn%W= z81$Ccj&Yf|dMSqm8s_I$=W#>(s~!hEbh!iZh%6UjX5z}D>%LC3PEJE=r25MfjpsAC zV|-KEzUX~{<#?g_&C1u`J$U`wl<x))nKEU_;P1dC)sEz(fl7dV#(->WO>6m$L+8N| zML1^GNC!mX6e`*b9v2-shrmU*qpd%)oeQ_Gp6@?fExvL6(RR0h$NaCi4XoQD3Y+Z4 z%LefEPpdSDpi2kA=KT)4Xad>yEDU%0(220x=zT)BM+vWWL|SlO3^AKzl?cicL<x-P zSq85DL|J)xfvm8iKIee?TC2m#Ejxmj@8agTk9DR!RP%q=)Jkh>OU~|NTN_@VC!eYW z3%Kwg+_O#2{a3UHf<5#<hCseJkM|?%&2d~&WaTmbMNL<}i;cx)<>Q;T9zU9QYuvcG zbH|UnHTN;cH$fvB4R3-GNt?Q~#LPs4Hr-m7$``|?RtCEku2C=B8RI94Ye9sUibLxY z^emH<KlMfqtG}HS`6KH>d>@gC34$#{*9ota!t^SgXYTsO;M(wg2@PfY3qjt0lBi_* zd&KE6Nn?}AdkQvTCOR)OORv)B<`(*}d{<!+Q?->y{fL=L7zCp+8iVeh^p8~F;nL!) zQ}mKT*RM9-X>4uW@Tb>ZnSLBuGYpU&(^cUorT$Ygn_lAeY+Q7#p4CUkYExNqMTi72 zce-9x=4x;$$<4_OsSK<W$L&yk_QL+XklEtZ+XU#_T@gk@3@<N|9+$|+m%w<!r|^fB zFu29gOu_i{wequ>qiHX89dCs+80(fvv@0jv20=qfcmW8U9!a8O5@NNS(A=KH1cVlP zfcUahM8Fvh+?VKa99t?0E(kAXL2pr9P*B2|uJb*VNWif}fH9AyWs>0V@L;YTsX%pR zSh0i^IaewqP=B%m+h`$2Mkg!vi6jAR%hOoJ!Dt60Hd2=)x)B#o2a9e)$FpZ7P{=dM zk(M!0^LN1rv0$NCp#JX~5WS*C8_8R9laXwd^X+tm(sj%RuV_{q9-b7gc5^ctK@dOj zl=JV4NI%(JGAtBN`Xm*ZR7CpUBE#6Lq~GD+$;4AKV{M(WPF+xtq%Gj~MnBu&s`6V) zzle5XwZ2J?!6CA!$iSq~O`CEysUrfD!O9XA8Mg&I34RkJ$J?rG^Tt<BWDP-CTOeo_ zlUtUYfAVC`Np(?KQdAyS5|Zj@@Rj8_HPbJke>}ErfU>X<1a@3gQ}xvwsvF){?VH#b zjjwOAQ<bl*(l-y{O0JRaetxWvZ;In?U)doIl2jHd3yTcs#lL#h3{$C55zSg+IrJ4a ze4~-wnRR4Xe^q~{VY6qkn2iKs9(Y3kyOD?h`VKZ|n~H2h<1PY&K5Pgu@4v3oadf(h zPG5j2X8&73cYcVzmb3!rOpGFbi4mP`e09Zrd{+jiyM;g8_kyuZBRX{dDnNYKhB_q4 z#UkaOa2G7nut=UWl0$O6bP<S4jcrCRF`e$85^>EWFa^RYKZJ=9zZ&3JB$oGs&^ddk zfm+Ki#L`_XN6%mwv3w0=^?y8(bYpiAE(C(_<Gi_Y%oud!o;_oJAbSpKOzY4m1vQj~ zX0Dvd*DhN+hh6Z+7h~x-6~M#_X9EHXiuk!B(Xk`U=?hoxus0rj`xBY05^|{}D#iXm z^^QjNb3i9B#b>R!8R{cF-+Ta`0g8sv56_ZD<r>0`g7f_2XS>Rrv;n&UcNv`a1iqR6 z?SSL7o6N_!JAAhoC`ilX>hg-}BkN>j$M?#4@Y~7BXg~#}GKFd=woC~03fz_9v^S8b z2EL^>7wKr3Pj+Q^l{zakB`piv7S<yP9xpuF>%};4S2@0scx2Z*#YXlYg>zdGXk=WH z-GahgWm^Ka?%JUC@X9F-;9{~Ezw#)M?O=>``q-{5<!qsAUC@DK8GjY0>7v=Nb<gnt zHf+CJsY}X%l)BQw{j%fb$M&37*X)fBD(AA}n96`$H&>PL1@Tc<HU(X<uawozpIJ0r zt=R4Ari*vyPVPBZ`c-DLp}r};S=(%1zczkw)R>*q*4Capa`gD2hW&<%t_^Mt%M6Za z)yGro0d%E5kcxw8sTCvuKJp5U-cjHI1TSr60&*%ME6{wTW@K{;XMm+XW)yYgsCPkf zesVz)gp*RCD2?3zk3U7gow-B0HggqCffwv6WQM57v<QSm(+miSIyY66UGQ&`k*;eg zbBFd&D?GIE$y#+zO`YL@TH$>1cuZg;chdi>(u$Lyhk!s{d9;6?zd9y1Nd$Yx;Wao` zjnto%h*axjNs=goE$$Qe3}!a%x|Z{|FI&~*FVp7c>GIVPkveS@XYU`ls={7IyEYSM zHtAu=OfjgVJ>0Y|>P=g+%eHZwDpm&hZ}PJ*UDf0#bGvaj^uBt3U0P->w`td!pq24! zwL9!H*UA)j_J)R?O={$dAsbZT{5tp9!Ec-0H#s?M+3x77UB2H@=3i1BwMSi6o>_o6 z*mz?7Z?dw2IAT;*YNfCv+sQ|Ji*oA2YoKb@*6`At|Kt~w-RrJx4PwW?=fK}Z<m5o{ z(T&w06CZrV)CW%uxlM8akh)DLPy2lSRBp-a4XY@lir0wwWivF3HDug3Pp^N&UcTLY z2^mPFzed~~+u(DXrMh+}rEp)k9~P$fDGaj~QNx8{Rr<n0YyLiepD`V@v8Ymp75eBy z!$jn()#%ZW=Uy9obddG!+f};>M8*n>^i^Sn&@V*ZFO+Z~q+-J?AWOQM-nSW)`xEy$ zhJ<S4!q}m9j+FaMHMer)V+a<cWU~dJfk?|Q`@Qt{vfpKfnoPhCAu=#ksf~-*LYzD$ zna$qnoV8Uwd4}K4Ib3XlE-@)KdTZR4=&dq3)EQ3nMRcZ`cANDbkzH;e-S0P^@!wSg zj@t}q%O@oaT4$^B&I{Um{p^b`3)t-X+{W}nvJ`+bQdJp&Nr`kFQX$L$7+8wIO90t$ zfK4l}NUKnjF^^^<+}U;K@UXt9i#c-ZZv8{Gu_!^8#TCWhrQJeQWC>r|R|ACwBiYDL zBf-(ck1r+Lde?)Ua|<Ao0^&Fi6!%QwHWoLO8)G-d*t70*B-G3BEpv(6p5>{gRy)v+ znUV3A0RtNL1D9V}ZLC(eWNco`nG)LjEBC-RxzHz@&4}6sW>7fmB`cRvGfwe9m&R0* z2^ZiagojZNGEjylu!^HQU36L(j()Y4E~EdZhgI}EnFGN1IYVuF92+a8-NR<lqjao# zOHgeC24bVWOU>dG_ZpMwxMoLO!X<R&P^f$WI%I)oYY~vN>j1%zxX2dW$h}p3L#B9; zo}XsO&y<~qk5^hxdZ}+-42ikH8IqaoJcwd+@9Pd3LL25NS<}^Y$MlEN%PZ11gmc@P zv-E@qw8nZ_g;a+-dM1H<SWP|a1-q9_O1w5DUcK4T%|4EcTYqTt=}7X&2rD7rBBKM% zkJ^5#ySTva@WyjmzId^fuq>Hbx7m4}j<eGp+x(@Px0BF3cV>fjo6`o>nq%9}vYmZy z@~)PzJbyG}e{EKy^&Ngp=Ar1rzI(0dK=Orq{f;`vYHR8X|3_{}kReb#mu^vdl?K&l z_iGPi9VpwImX?;9mIiV4K~^sHtFoOu9NglU*EoVAOP87izP19ZgWEHbh}RCrw35HC zJgeJwY@OOJ*<o{<T`u*vN_E{78CElpthb_T__6+{mzQnyvEJ+#xw6C_331P?sFt7T zb9`mVW<TqVK4HuDY$64My%jAX@7x<#<U4p;Z}x~;TDO%%(W_P!6*c+$)qASq!pk{G z^9(-bb=YOUvo#q=zo_=e+n(%X{pYy@#g#Q#hpd~(uI#4t)4EeI8D)WObBMD!=tfBY zt^kcJP-ftt0SOgN5mg*Q*`<*smGxBzFi`Us##sGHH!!go#M-e)^v-7)gx;pr0f)J& z%Ib=}W&Ec0hgPxa$5V6E<$IDc%DIe`^fagm3qEaTQ59@$ab3aw{P=h)entDG7634N z(hw%bAm&>XJ!{S><#G&$oLp7$a56c(nk5cT;I1D<J|fFyd8L_PvDzJ98Cl9gEWwW6 zBim?)dhsG{B+^}bk`N|a8*PYIhm2Vp9mK_G<D;WgAus6U0wlUG`~nm|eK0QiT;}Dx zbK5!<v=s7#*t^(&H<+Jp(`3o1smZBo({03wm>;hp_qZQ&-!_nLpFd*Bs_Ezve2TP@ z=|B@r10uLDT|QkVbTO?_R+X1m0jUR8JUZ1UAi&2bpuFnKfM(~z>|y7%<#uXup5wb* zRf6>+lK~w5Q_{<H8RW#hg3z=2<p1>c9$-;j>$~^>)0nNaVF=7Pdr-0Wc5K9;u_f3= zBVtzs6r_vvp*QJ6laAOGjbe$45@U+dSV_^um~Nsb0o1I4HR^rWz!=Z@<(~h2p8tKW z<7TbB_Ue6o>-*lXW5{{HaFAa2Ejk<Y(-gPjS^}@~8^hK{txlBIE-hNTNJ$wut2^C> z-y}#pgn^%9GI%K>&Yn%&c8bqCS$3lOsI+F`+@iTE`aV3TL4Ql%CTjPnkA_;b5``xj zr~)a^{v0s}v)Gd+90&U<!EC<}s_?cEbDt-Cj$f89)Oyw|HMrB`)SOSRuF0rTm2C%( z9qH8u`beG<e6R{F888#F1a{xurJ7rG;CVVsiUa0K#b00i_VW=<#fgHur<6a1|NPG9 zysz8ab0>#;#LSCWw?XRT8|v<*TvzH{>&FxR02$c!A#uovjt@?bUC@^*#`aq*U3=of zrb{ZTqf9RL8~y4ZGKzPf1scO$`E^uEk^)yJBj|X#j+g(6?ZXHxerxf=L`K%1IG!AP zOcNWF5Re`qE%o1&4?*UU;KOyIL$JdVgOoB#BfkzbCt!Dz;YU-BMjr;&!rqcy<<hDZ zI!{j}h$}=<|7PhMQ<QY-bm~F}4&xKV5dY1N)ex^Q599beS6P>}Gh-*8CG>gX*|zw> zU5^WNaNb}k`SFRuKXq|@06#b6owui{)_B+L-J+4Ve0YEidX)dQRQ~JwQT=BO4VT8$ zCGOs>{O!h(JGK0U9j8w0JSRQ8Y{%SrN^%#vL<Y;Adx+)N2dbY#Y10@2b{R4TN}Kwt z*|K=Hb1B&m`E}t7y`y4f?<oINJ}fbJt#5EhVNmbTp#Y7t3VhHfC?qjN<&%4SXQIYG zuXn|J<bpb<Ft<>5irOY!QtsJbUeDK5#?-0u^0KmXH5u=wzx%GTA^XgZ{m`j?;lX>D zm5KP*d411lcKBy|`6|8By)(S|%v`83s;w-qQ|&w$6{K;ewz^fy#9SO=`FF=(pYuzE zv@E?aAyx^|k38IYIImal=p|lf(eV=)IH^|#9W-+cT_g=#o;GEP(miiZ?i@ZfL7So7 z;J?dX<-0OugJw8cRX$!BlM<T}%dhA&yr1*xU~O<pV2U&{Eg>#aIg3mUd@q^bToX0* zgTp6woKn@)WTw?<B8c=9+b@7|?-;SKCY&a>x@LRL$;P-wRdY<s{%Xy81!U@;Ys$)m z(#TvsFh4jyxR?q~jcoOLB26hvDcf<6tc-l9JhptNAMX<o<fBT|=Hw=6axdq7dRh5? z+LvKxc{iwK8PrN=XBas&&SImukDY=#K`Um6=8lRLy`wgxC6aR1`UMB;f}Kq*(K|RK zAz0;WY>CZiiPLBa=*(g*VZ&NtUj<sCD!LXe!JzokyILaW%ns-<lU;alnze-FzOf}X zXM6c6y)!nXceg}cWqGZsC2Uwr<RoE*MoYN=vdz>Ix{e@chPVNxuncwz_wv=UzH6xS zA}sFF;3WmxNwhOf-{vRHitw8VY0g=|oGb<>9(bR%bcP|DR%&Rh2j$_EmXVPLrK*{k z$~yo1Lr8p%G#8Rv(LazQD(rpCV-nA3s?w@-x(duizdII|rB=iiO1Gz{XQ!z~mr&nY zIw6Sq`Ofg775$}Io*}(`dE!It?l*(&ZxQs41-?&$6VLwkF)=&7=foZ|?CSCFj^C>! zQ+J-MKd~S9$0rGp9`x6U#w_dOb1nK3qSlwTockE`y1`&(+LgI0t)8a|u_WwvT+_BQ z!6%%kUtg$T9^>EWb9nuJCmh^nwv$b3cCD!PEOmOFhL@29QAln`c5p~=MraS0QmUOo z!aU<vM63yMY6s`RXfkuXzmJdqI>0Ys7q{tg$eM^1ah^^j+?6JliPA$dg0t|;4hiYe zk0g}QFxOJg>J{~?oyexgfKnU1f8F7YjR8&|#m#h~n@<Fvj%bQ=x=psm(nL{x+OCZH zUG>@ZJzQc*@*TRZsqA#siCs=E*ussXGaL6GKD@6H>LzgWxXGpdMD^*?b2#zPu-il% zE6T0kUcXDZ&jDa3JHSKn1)xvL0Cn;exlNe)CHVq?DCP7v-=dc*p7qnqpY=1yMb8Q( z9WXoaE`q}x#j|Dlk)n>vl8$Bi5gp46BSg<GOFr)SQvs|ak7qhSodV_tXL`_6^-#*} zOr2VKiIlg@vTpSfwcfHI_j`#zYn%_l-!M8~^Q=Syb#Sng`TVI{;!T?}3P5YI*)<BA zJUeAa3Da5910?AIe8({Hu3J!70aQza&GSY&Uw>Cbw?XgbvtUuFUxAO0(kIzB&X4zY znLdwNL`vy95^}Z>9Q-*ylVm;MJFFZ@gyDjM^c@9Mg<kLRA+tq)dEe?s#meV(&n8LY zgQLQtL!}*uaoa(42T9qX$2dHC9ZWh1s^QRM5qpUZ)|9;@@gV8FUq@^r4yGO;eei$Z zkNe5Uz4Dv)n>&8(CA_R?2y5K1K75_8Pwo0+N9&Fq=IMl9oi&Q}{(kG%2Q(bz0d*!% zcwc*T-=SkX3w3P2-v(fy0Ta(*Lx3*{l<o6_&H#8C3CX`{4nWS+A}rkD&*&}{G{b#^ zzwi3>{$24M-GAs9i-vtBHBeliKt0Fcbb(o2dN9hj&RgZXDIy?Jvu_(t=&<W$YM;&c zv%xRS4;+r;iFzzLkPie}50PFsh>VY2l)P|(61$=>dKQ4lNzhs|6nwk_o(|rt2ucY~ z4(8X)n;PV%!h+fZoArf{_C0F;MiVtVZq`gC9dd018QpYNSJcGk>|m%4O|>DO8pFJf z0SfokZ_S*!`m@WQp8V|k^^<SS;X5qpedu>vKsEhG!uR&_9m;FI$7V)GrKd;o2`g44 zdO`kt=~u+*$GS)L-)g?R`A73pmD~nZvl{9(-=+&RsGw$uj0PxvjUqj#UEy~I`P6Sz zg>H?HjM0RWzH^|H&HRxxzo4kFNLjhQDkhKD6&*fQs)TB|^c?=M&(fM@DvzaM>!3m? zV(a#;D$H<G2V(gaX?$131q7~6O^8cKPK=3)jf~qSZHU=Yu~9{%ryCsTG|${Q)hjf| z2OuMguck-?mM@B4s0t0u&I{9=oOfkqON3v%PqI&@blFE=Z23yna;^TvMa}#p9-lQF ztln422UH!1IIJ?-H33y1r>Nv28v%Q-(gakp_YY4tU4(`)N$z%Hc@WBd<zN5c^uxnr zP%B`kv^uMP^poFPH~GH(MaR}Hu3D_3PQAgU;7GBhHYZN4e}43|Ttfq}BYZB--I~2M zoT$_nu)5-I6uRlgT0k3sPBBL9H*9m#I$pbEak7VM%-DUU<1|HYy_NTt3OgTZWH4kq zu1W44ym0qtCFwcncc%4e5EMg=x-jwHCUB(g#c=#ZvWH#0RfC!`iO3g_!72>h9@Pi_ z((Em)uG`N5tsqfiKL(Vyaz=f_PiLgTfjox+rNC}Vp?8PyMl7S)8DHfm^<e!1{p?Th zqrU&4Vpz*uNX2rh`0(wzMChUqSR1z0chl+^uiUUHW7oRvl5XDY6|+h;*7f{HkZo@w zr$mQ**QcyjfsbH>M1Dq(*>JSz`0<N7KT=)&qImNq4V^*Py*?60*Dpsn{e>-nXF7O8 zY^5w+TjKolu&?^uad9GJ7AjKChn?|1w)|7CE1s7&o?Lgr`((|P@n=>p!(GW1#|3Zo z*}mwS&&jMyM^1ujlID2)@cZ>pBsE!l`O`qJ;~LD!vqka<{jUZcFrXb!8kDNVM@F%Q zbfgkj99N)Y?xY@^0dLQV@L8%kymU_W+c*k~>9onXhn7N@onhiQ*|V_{!~#ZxPBAnG zHxO$m-I_OvO#Id9r<9+LU%2sk`DbTNe0sn1&WDG8km_fOQR1=SshBS#>wAgTk@b)* z>J%$<X}aWmzR&&4VndbQR(fhK6}C=LO`sM7R?g8FstmT7+CC$s$th`xiOT02-<KIa z(W@0prh>#Fp^hqu_JUgW!Rs3ESc<6Goyi}^7Nu7gm%V%5vAC={r%ZciArZKO7%7sj zx<rHnzEyoognyp81WG-Vefj#%hh@dp>BX_{zT;RNn;sFHFnK;TbHxT*WV}UWT>{9~ z>;~~dhlN607LgOHowa0;8`Rc_q~4wbhtE*q_6*3KprOqe`0Kl#8XTg`hI~G&IkseL zx;AFxJC0i1AeCu<e*<ULH;NOb3vWJjeXPm?eTUyJew_L6>zf}I6_O}2<Imfr#W_{l zGASAIY-Bkw-`s$^HCFQHuQ5q_C?b^}ZTjQ48^(YL+@sHpKZb2+V*Fj5F@=8)mp&hB z#Bfs_jyV*(lcy^{S;)mY??}v~AI=#3bzhY+lPsHk2Txu&1KFgO*bz;HF!3U3aA=I5 zkMj9glCLh?YS&S-Kcy^h*AKP#s)=Q({3`i&yTS{1Q~uQN5Y>uy#zV?+JFp2h7t;)p z;jVsy;w@0jGU%E!^lMR_R<Rs2^Aob7jD98BPs0`Rnv$Gr35EQ>ZrnaED$GwSD^$vx z+g-D1lIU4uM~h-4SR@b7sn-nNqK<0AdIiMbrepxiC5lWCJu3lWcBbARSDoXlz?}jS z{tpzhPZtnwdrn4fdbSgFd64}Cw52{G^2RU)4z9{-TpG;+WI5epa8l%^Lse-GSxkmG zW^V@pLzz=|kc4LxWHNN`Y??t-j`AvO=(3=K6z4w2bZiOJmFd)c{0HgTsafe6PPFIL zRAMb+sX-yE-FHOxi3nmyxw*;+{d!SOIx@j9Z-$AmF$8CiVFp#DW~8TXPjPx^*q9Sf zq~puuo#ZvcR;8wAKs%??E!>kOd^5d7>m+ZUw=tc0O>@c%IZLzhQXxi?>IlH*tei|~ zcJ}t|*%~PPjuYi%Z%59P$++Jq<fiANJWdeUo8RqRNG!3L>6*O2y6S!gvl-+3_))$W zNDkzjV&L1;C-a6D@#ME}{y}D(09?aN&E^YVc-&Rp{o=v_==Yv^f_hSPh^hKt6wrui ziSgZ+nNY3V7lgPjvoB}}K+xkmYz#*hsc}>B5Lgl(i`7HKxQ4eUOEHB=Dr3tczg1V3 zLAb=q831uzO!AD+fvF&}=q&AoIu92XaaRH?LWsQ~Vk88UCCGcxAjO8aW_!7+TxXv- z`j#dYI_(2!EbTqMdE9;A$&2qde}9h*2p|!3v8Drv_)M`tMa+(<Z`IH=z%nN8HbcP0 z^<0O6L@|&cKhVOU@_~E@uC6Wy8T<0%H;fgh)7vn?J_t9+??JQ1AjQ~4xK+XIIDOHj zAIOMzfeJiY6L|JBuNGCyXJ__6LBZ&EA#e-bETi-l3b_ll>(?I(fo;E5EE=|LZNwH( zPq6f(wwlgShJ0|=8Cv$q7#p0sgp>*+qN5{t!xeEvba}Pr14(sxc{Q)UBCalvj?gTY zkUXJ$5(@#e*L&fnP&&e}`g(P^`GX(qp?E4&LiO+s6!?i`y^JxcVFAMx)(@y@R^v;7 z@d}Mk#?p`x-T>_#%?B=j%WIly+FNJ#EZ5M{-mC;;FV4NG0oMM_i<f@rDh;vi*JdmW zGEeQ=`eWg-Z9}<u6l5Xfe8_cSb#!@m_Ko>9Dls%>AEm+P0mwR#{94FO*>n4HHDg4c zs~+-9_YlHFL+BI9PSy@+3^8jAG!Eu1IG73t=TE_FBm++mN}yw6wU3FX0(cG@8VNa@ z5*00h0FDBho-~?WWd4^}-KW$^hx|z7^N2Ikpeq05;g1?JCG1N&X&0R@rD+}W74b4X zq)EUg!Nf6)(zuCWpzaR_>SVo(et<p!ytYjJ$~twYJx-J_eI@8!3<KaNMc-FHP_NB} zMuQi<I^R=}Sw3TNCG4&u<ud9%78gFVd~n*gUyRey6eZX)OdK0@%3?ZdI5cC^_zo4a z!MJe%Tg6zY?=WPpcY!F@bZ76ce=4CIB#K1QhELT4_6~1y<!SGAxS!J)yp(hu<lMpQ z!FN0mgGYQAs6y-XUUVse3{a!SFaw+m(D^{!j^L`9DdO=Z7Bc-C44)9NgVb^e&KkYQ zwWk>Q%ZoIwKNCx@F3Cg7Gk1R0kmU&=b<%4}+G_|Xf0j)13&!pSbR9Nkb!5MSjNAae zv{C%Z<!^0a{b%oc=s%-2fb)3dxQp)H{pa%cfE?17D~PR28)!Ci7i0fv?+*w(c{}Sr zq~W<OFep9PSX{&46Khz*1GIUWj6NA6CO6a%(K!|apux>Y-RXf&!1^>;qJgM%;4)LB z$oe(<M4Jq4^p#?o*8*tloFZP}^-&9PJb-h6qh|fF3!$H=KDn^-*fBsD9l$xmQA6$Z z3l!ul8Cb<Tb%roSF~*<)dp*YP+9HmO#cXwX6Wk!f6B!!|Bsh<5M{=HxR-KRqU^u)_ z6c4>1Ki0fRHUv3;`0pK-<#i&v;?=QShA~?a>q}oj1I%WeBOUqm>peo}spfg?Jhom# z9XGSQO*^yTBaMEF_@gr)wHWic1<<R8v5VBtJRm&mF<y;)wkX)N3NH6im|}5BeVI&< zxnsxHf^pX64dd(?o8~&$e&I>9`uUT87*XsBIwuhOAi-8JB)WB6AtUYf_7Z<2ckLy- z-;n^J{cx&UHGr3|0HJvBeY#jBccoTC*DqV3IXhS+uPCYCoeSL!eOhqKW_1Y+Ch_an zq~ZwF3<d1I`Y{^$3h=QxK=P*<0tpUulVKb$cZHse9*QL?J~KzDCN8-6{UVLL0t5^- zgv0&F4yAsR;Wcu@FvQT;(2L$O_!>6oRrHqL<;D$N<rT()qrCGNS3S#B<ZI9<ZzXH6 zWA`PFy@WqAva(IbV33N-iRC@$lm3m^S?E3cD-JC`wY834`u?i-y{~}TZ}=vh17?6B z;t=Wc9qGe2UO)HYg^SXvs*{cTl)HkfLlzFAqBWa%EJyA$s27b<((LOPg(D%wd1%e8 z@gKy2qLi7NmyxU5Q@?8uM5o^%N*~$t&%_%-{5K8<@6~7GqxJibK(0fN8A;>w=iqj} zBKn=?5LHSV5U@jzEnlS!h}i1y760U53Li?Gx3p5tXVUUb>q>o8@mtcP5{i=x(=?UZ z-M+<<(klP_;Ee!ENdj~|M!hRmMkN`(7*&yxSC^Ql(&_Swixame=4gD&!Ya4!m-;m& zHGK>+zWYw%bZ+yGGNmpjOLy=+kDxMMw{3gM)-CA)Ta;_6Hl5ymwEO^HA5*tenUj^B zQ&zt@p@84Hv3U7v3b@XhTa<}A5({-jd3l9=^X{vk9y}{ObF&JFc^y7m6g8Q(nKgV2 z30VX+SV}TmdfIm=v3g4t5*!rb)3mBCRC9Cc>A9yyNL%QjY7nI-D5=*1pzqtzk^Gj8 z*iD%EDYw=K*Zcyp_hmPZ^S_WGr*Y1ku7va-E>B6MLc4rR{JJ^{g=_$o>??|oPe=$; zm6L5Ea$BY!qvtBi!*!w2PKF}Tg@Uhp?Z`a%QJquA6Y~AB9Sxyz^PKc6XhXM%!)$dY z#?f<4AK7em2W-!bHa%<xqYX2rJpce9iU%c<9rlN1#+4F(z-}Oa#-uGg6p#P)qqFnB z#`d`Y0Y3*E$olHd)pUm5UPfj-X}W*=BOaL9m`k4NB)1ndjp(D4C%VC-9E*q5a$@fK z4d3A<VQwr4f_Fv0gH6|U5@boJRkIp?^VCo@ry<YOm~G^V0!$trjJFdPK7f`(U=L{V z>3-Yhj5jNGz43=}e!*U)L-&VTexRtAsH~SrqL>J+zcQ!QtEu@9w0{+~Tjum|ICc1# zx~Ry0$n-*655#}n)z>Zst$vT6N}WpR<VVzM^=ijo)ORO3mc~J3wzNasA$Dw2OGg9U zYA%uiJg1wwVas0M1f$#(+~mT+&+N})4sus8RuKy|X}Qdtr5;A-e+pqU&;UuQ{}GY% z0t1Uk=YY@*gZr#Zu61JuxOsJ8{UDbaH_VS}X10JIM(gH$ObzxqENG}LDsJk3dpb|& zNy5t_iMP74RwLIsI}>wB?6DI`r&Jv}@u?GqWyds-MU^*S7eI;SQpxR`O|6jnVA$%< zJ@ijv)p8qq!R5y?xfJvof0T_OwL5G=X#g6|-i1cPTq@{nG3XZIEauz=c*o0yW`aZe z<HGAS8)xT_KQk{p+z|>+67o}yuXW5%Day*vCs)Z;$Nc=PqLlo##~oAh6S7iLpozy^ z5FYMvVybR#h|`%BZ|{3k1th~~3@cnH7&3}&hQ_O(+k>x&&Gu{^iY$w*WLs(8{qjpU zz;gnkTzg7AL^c$>K4!o{XSoK0o(yUgG5tDpFsxNOws3DHj}$;#F*}H3vV@v#qN=wF z-YR;V-_du6bA3PQw90EypQ%2(R?$+asc+ly*N(^1qALZTeWuhO)w?S6a|{ylmtj#L zZ+I<~UZFR(8D5K`zX8ANENP<B;?&ZjjFPn4^xCwY8RZ!}GRiW6K9`zJy&r@9YI6q3 zmS*4#SPN#OUn-?~Q8gIlPXkvO8o6AcbaO**9!I3K%#;i$_Q{A&3ri1A-<lC$l2Dv0 zB}YIxUjgy{uat(P2Vx-Ay&0+vXc~#5r-5}^8ISNGDYPn`PPs#e5St-INo57uB{_Qw z4_5Ba&Ci4;u7BP3B+^o1R{?Mn7l263sFsrGdg4jnf<8tn^cwKS3_c|#GbuNv(!X$1 zPGELiS#nXKl=S@d4p>blG9VO)3o=%D=-vVwQ3u8kMmsJ?o*Yu+8#?JoNWZZ4zmrJ^ zdf?Pd_5s6;t^RD!%1#q^F|~l-OD6vd9i8b=kjOg?ED|&^4#yfCq2Txo1Q=b%6GZjg z12H`@Jdw!%T8tOA16q!azTUXIN228Wj!yDD69p?Fn-y_!5m|AikSB_D#L+0W>y_Q) z_m3;hsxB>cVyq|Zv*{IIN=q@&<M4&KlUUPhf7-t^4v<rEPAn$&wfj}SxsyYA$sM-u zLKQHv7vXRE*A;DQ`Z{^t5wd@;Lr|S1%H|&&z43!1vm;U_nr?KWuQMxLo~l_8<_H@? ziJr&OCPgQrer~E=U>aQ@or-6D#N;FWC!&r%V*S{clY1SuFsnh08%;-)KWNT*e;ols z+-vV2yb<vc;qCN4>?Yz*F20}Byqb&}{B9jteD6c~o(?x4hIgJ)d^~$}XwbpHgXcdv z;3G9S(@aHCQC3AlkyI`gXtl*rSqWNgLRM69LXoy2tGHN7CQbz-W7h8Ia_^&#QRP8d z(b2xXj?q!z0*ZoK;|{lXy(^-2XO&ktH8g<iL>v^w#aR_v#Fy&UoPhWc9pWp}7AI6> z6%|1r_V0?5_vV~k(>U|W%ssD<avjyw_trbNmJwOQsqdg}2Gh_=!&b#sEKgNr&SYHj zxVsIZwr+E91UP4~9MbH1@+sjyy&=sGZdj6~TnVN}YcliT$I`6x6*-^rxvld*Uv*M? zz`Jn4Qe{>a<+qgaYqp0Z3<#AT&8~^eQig6^wqjB6gbkrzooFg5DJm)|OesjyWul-` zb?9RZlzweTrCB)Zx!-Q!%gT0E=<?nsqF6Gf@<c`k6zin#zH}p{dY!a5G}}K+ndOzU z#)Cgv)r3p5Qf>LxEM@pwzp*=q*G#(QeLnS#cSjS8d!*mHS8gBqI*|zDzUdc7g-Ns4 zEn4g^%_{YYU4_jRP|L!kS!)W`Zs8x*om+W!Y~`kJGZGg{<!+dPlgMy|J_*M&z$Wo> zsZfCPSbyWGElCd(r#6^+m>Mf^e_M87ym!1!EX^R;SY@H#(M$A}qCUH<k=LIRKRiEy zjyS40Ky5_7in9PPTL$bx8XrV8$osQJVndG-Q9lWhhyD~*ACWV`shFmv8~TG=H^a~` z1?)qa(}IFWM^Sl*V?T)=emkV%EYcgioAG?Xk7x9~3r<K!i-%KxurE)gf_NrZ)d6{` zMQ0yt^vx1d^?g#7=4xhRSk2JARPNX=PA0d?$QL3P@g~|Qd_-*Muym&^Y86jovU#iJ zK!G4ujen5|PmdF;cZp>q`ws|wi_YO45sJh4b*p)LNpdPP`QTwCx&FPPI(K(ac<!cH zaBR60ZsSvO^3wBFds7ZpYo4E=5{2HMSQ-2!<cb5iC0RpniQW*eB3DH8iZi~9mXu5@ znp!k<4+(Z8V+{Uca*KSZpkFuE(wn^A{};)XKl3M)Yw`m#HwOTsk$Nf?CN7?~DXh4D zE1w(ysC2|(Nr#Jf*47psRNcv0;jW=h5;D6Zm`#inWC|SWKb7eR(SpA^!fW(XL0k@k zx**c|ptJpFz2j}nCJ~60Hl3!a#D6R~jJ@{9aknFnKuv`O1j6o8babeS%FG31nL^*L zNIa7IZaR9Yr1Hk$pRQh!W);wPF48yng&nqvtW?kJ06sP$C~T8&&H4JK{qMnFFHxmo zCV(cf2gC7)I$$HcVB44&Dy@=c8N|PVNBa>^Mx=k3`*;T#TSvy7ApNhMsZGC_ay;q$ z#`LuTkW2ZVCK}$Z1{#3FCeng?U02Ylra+VDmhHQW?+wjGJT|95uY8Lyx>|O=rcsI! zq#<vANmi)z)5#P1mtnT%^COA|@&g@9zMyBwaAb>q0)EhDA7CK#S-CYTJkoFN>!DL) z=8o$-m)ZnU^_ppGhbB@hX;!*Fxcq3}N;>J6Eai~}#P`ilFk}i0eISOW;#b~CDnU1; zP9&|4%m#;7W{!%IM@XeqZ>y@`xjlQQ=3>f)+;f$CbbBgxRYFC?802o+&!oEcO7We7 zYYbCoI{`n`Cl`Jyg|x;9vm?hIp6DeE23<V-3T~la2Wi50Nd{P|x89@bb8dbO&4=G7 zBl1blNyKVWBUSq@k&+jNn+jcBYIbCZA>!GTUergQMSMD*Y@+6yr=(L!&~sHUAq6bi z;f^^{nxtQ%AcyHTkU0+Fw~a>8!vIu)368o$pxZ`42!$MjlxX@zFCtuf*-+9^->Wm% zkWGGh{yiPvd9Rn~9OUHn&(2Ec(g%ttdY{$;-fH(79e2wDdkJqoE8QhcTUU#-61hGW zTZZT;`U~jz_PE!9JkUS?wYzL2@!QMy9|5faf{sFHdvUIj$!nZ%%H%f8Hjvqb%qC+t zGiEcdflaUmHn$^ZqQ!{?$vWsL5qGv=(=$f)tmQJ>9k|LmTBfocbTUa%%e6Ka)ba&3 zJJsc9Bs;;0EzFY1otc~czq?79o9N%&%$b|nf`1Du$b*}}<OU)zZqT+daamqJUh!>3 z2(g_IO+TIMNOyuN#hy>+ig23E%2j<?OoTh!G<5PoGViZ_6~)IQ_c=xG+fW=3N$2%; zyg<h8`I=fa2DyTtxfzfQkRAO@f~(fV4?Z3Iork*<lkgXjO~0w>CJDH-?L96J{?`X{ zoX7@n0?^MSNN;36(j0V$TCLkN+35lhrsq8ksN9ec>F*R7P`rL$6q)DjNGER+#kdty z;g>4p2`s_n(@RjGJPPTJqMu%xP#!{Uzm0MtlQ+?M&H+){^_2lml>tY!`zp!2r;Z*_ z_6(Wkb-V9?OSl=O8)-}#IaoaB(Z4QSc0w=49l$1|NH6{(#~0imeYf~iC+M6^G?oYD zYNO4&T`}bbe(l5nmFD%{7kRX}a-UP>KJBr93OesEN5J@iEWNUqFqy2xn0R0R7`^T$ zz=4zKwJLhE3Reh~m87K-$gl^{%Gb7$8{2RdQW;5Gq~uoTI0gNFHT_{V{u+dyP}$NH zX0VK-A>UDdG6pPPf6_l4$@eF_{_8E805;Q9tCyCMka4(f83V4sHqvT@(<H)PVrL@6 zUaZSv(K%b6r}!|WrimuEZC!qQ6P;y{iOz~4>DLYsn|9GTvEfuFu0$N@MRE~T8V7Pw zbj(B1k0z6(e(g}O(6~Y|3Bq`bCfy~AMCAR|3d3~z1bfiw%*57nI-9~wCUZysb|9at z$s0hQ1gfB}HHJ*kKPG{1>c~{$c$LWRkr80@9acheT!3)j=MP4dn?}X~H$+|?(+h%t z7Zhc~=&XkI)$Rv2w3Oc}eIKh^P~JglLvCb_Ru!{dn;a7!7lFIA^Kl{TTzi+6e4VrN zH?k@BP)>DPZA5WIQD}5>d_oj1lOM+hOG8$L#BRtKnL6vMeZQ6-|B+lj_4U5@ziqr2 zvM=uV){>Mxar+udiuUiWDm#%Z-J4bsQM<J=vb4A;zg$;RcslcJ<_D1_<4cDDTO$>{ zu+Wt_eo*|T^tn6rSEN-(lx$1emKGn8yDc}OD!vL>s5aW_+>$C_*y*q0kQ`IzpC1+- z9-ZR9Bdk1Ze@b0>ZF&Cw=sM}M3MfU`c{uTmZ@uqMuf$Lv;1Dct2yF;CquY5{YODv@ zvxy2s7ktFCXk)NXaN@H1jqF4H#-_w0^+$H;&V?M2LbDeU>RVaG5$PZ6$Rg@;vI+>o zDUf{8zD}2cqzFF7F;H_pH@H9b{ew<`jzJ-qH^+WYPm)OQ>_rue4tYL+K-@e(qJEH@ zo0o%oFk6h)m7g3Z6R&4nulnQ!3MFJaKjH;IQ|WVk$3R8o?v44ukwM#1HdY2z1|3P+ zRk^z=|41a%Bq1YXfM1YS7hV>g8lD;(o*SMQRvTNJSDRN>n_3GcgmuqnD^hm_R|Ka9 zr$hzk2jvCtirSUGE3aZ#%5Leip`Er0`Mee3M^=>hg!_cYd)02N@i`rTxb{eG@tLjA zB^w9c?zHM{sQ3t0@u>Q$xa!=hywa-FYAIbzQWO#U))j8q8n88aU3EZpKx6X0>b*4u zjS>5>l>L`q&~CsZ?S|?s5Og@U7WC+0{M!@iZh&$5P|+Yadt@#!6Z90Q1V;qTW=>{( z%?6kaF&kkv+RW9=&1{C*+h+64)|>g5Z8i%ui!zHhOEOC{%Qf3&_MzD&vm0ign>{f5 z!>rwWn)yugx6S97FEaNuUuEuZ9%-ItUTEH6e$4!&`8o3s%s)22W`4{3OY`r|e>MNz zyxm-H!C6>a*jqSRs4a$DOtfgW_|oD#i(f4Muy|_GVew2T6iS3v!v4bH!imDyg;Rwy zg>!`qh0BHOgd2qc!cbv^Fk09wyej-f_)ugaau6v+ylA3mn&@rOJkcVNr)ZTZT$Ccp z5`84PCi+5jPb?M>6Gw@Y#M$B^agBJFc)z$o+$g>+ejxrs{8-{DnJZZ$@sg~S_(%dJ zp_2C`7bG7`u1H!WMDjw~M><+MQR*h0A)O~(B@L2plg3F;OYd3QTPiJ`Etgs@w_I(R zZCPYlVR_B+Tgx`f=Q0bKrOZlZD|3{MkWG=zlm*JtW#zI%vPRi^vL@MYvUXVqXU0i5 zp6kyI<=i-LE|iPr;<*$qlgr@>xE)+Aw~sr_o#ejeTDeZ{c@Og*c0FF}q3Yq>V_1(# zJ=}XN>9M|tPY?ed;XPt{B=$(_vA4&^J?{2+-qWI|rss&B^LsAsxxD9^o|}3G_6+YC z-E&9J6Foog`K0GFE1A`6Rw}FhR@1H4S%q4~S>;;ktV*q_t?I4zTD@m=-s+mwEvwsB z_pE-ldT8~h)njXswcL7`^(gBJ)>Eu!Si4)#xAw3Ouuiouw%%=h$oiD^dFzj?FI!)? zZn3^&{j2pK)}1y|n;tf{HcA_3n?W|iZN}TU+Dx}uXya+K#U|7y!=~Eipv`+W=WQ<9 zT($Ya=AO+jHox1n+5BZgZEbA(*-o-`vt45AXB%ysZCho#)AoSvVcSOA)3)brKe7GV z_K|J7?O(WRd|@ZH<Lqqg6n4B_AG-l|L+wV{y=k|^Zn>SmU7TH>U8!A_-5$Gl?M~WV zu>08Viro#nAM7655jlpuTqAdp50np+kCso9&z3I$G_{X>vpifLEsvL{$TQ{n@?v?F ze7F3d{FwZ-{G9xv{IdLp{7d;a^6%xp$e-E^?R(hU+V`?|u^(zb+J3720{eIDm)ozl z-(VkNA7LMBpJrcVztjGJeWU$*_UG*{+F!B1VSn5HJNw`4+w40PW(u)_Q#dL#iXn;# ziW!ReiX{p!#X5zbVv8b75vhn%BrEb16^gxzgNmbyCdDPi=Zd?EpA`=kkFl7UIaoSa zJIEcJ95fCt4uc$qJB)Fd;P9ryJO@vQ)eajR0v)0pQXKLeN*yX4>Kyhs9CUd1hD;A_ zolH?DZ}q0ko$0D~->kkIBI6{l2YODMto%Qx^x~c!lwP-gqx1p{`@c|n-TphJm(h0r zru619N-uU?kZFcw^E7~$gbl)|Ss)`va4`g`9`2O}%O3hM-jJ(mu|W(5j~ZNrI`Ft2 zWwh!VgIGBP*H^KT8h27JyDS+l<A;d#VlZCTV9js<2_E|Zyy(SBZh^X6g9kT6#&ht= zi3g}T2VC(2(GX19V|XrdrJ7?zNX~c57GI8$COL+5wGuMbS@?CCp=$A1H7EX9#tm2V z+zMbGjRxU%s*H<_-LB@Q$u!(hJg9Z*v02S20F<W#S{@`C6>DV>i3UQ;Aer&z&At2L zO=6^bUKUrDp&Z0RI8V(1w3181{4GgSqt(>L{P3WaGbt_&u@469rG%S_WF%9OgqO^e z$r&=h2tI339Ev>{R>#wa<e}&rh>GKuxR3IGCwdP|X6F;|#gm7?6X-zE=E^wnFd4T3 zRU}E0ae3+zS+$yD$iJK@1&m2a%B0-H{1l!Wg<CkSYZu5QM@HhJ(0DF*_=}8V%}PXS zl1`er@6`siUM>T)SAGiE%~gp>kJb8(hK+k=sO{KDZlhYmtwtU8QFFs&!_^!XDr1R3 zc<01#s<|K(wCh&TW1x(Kz*-8bXPEl3m|J>cO*8l7o43$*-S>v<HWSrIJ^n-?*2}m8 zHOo5qB{=b7<jQeKpzPrgagyxeL&<dY5Y~JiXg&`#pRW4&pOwn&%p7pI6vR7!nyO04 z@be4U5V*l9VLN^a37H8!Ty`Y5{Sl{1b#{lKU^)mz8Oo^R{dEz1d~`x&QlvB@gDPTi zujVX?A&S`Q48Cw%QB+B^G%;dr|0pG-hjKFb<dVPK^Hil@{J8@#QJMG^6>Tr-;Sy8y z#eh;3N1sC92LKeANdQgs6bD2vHOC;T@axSn{ZbmPOC4jNdO0dzV8LBpjBYSW&E3aU z!VVcXQf7saV87r}@_Emuchm;d_AD8z^Cjx0rXm@)lF=-D)LewDmqdVDpxH7`u>>;& zdi9t$-yFj&lew>y4dKL7P~SEn&Js^pO4Q^Yn(8vL!w`Oa)m%-!IvqU}DNByZIL2?{ zfgQVth2EpHWtO`0yrD%w($vpZcdQbfTQ>OEbd_OjtIRM~GX2=#bDn(1>St?2VRhs+ zbse-_#p|`?9b^NLW4H#D0E^3xy}hDan0U*KY9efSj_B%<Aqmv+c<4P2$;fLmjlsj< zp~GFdpDfwL7*LA~WOP9`)OX6b`RGQt&9t8eehalm5AVA`qGdL{$5jkp+(7V`$Fr(7 zDqZ7z)zeWVEr*M9`=M~gjKrB_pKq{HO{~<Qu|N`<!(pl$ofbSPPt}|*xgZf;5`w|j zWCWeo$nS0}0|46|*}Mhe>sRu`!xh}tc65UZ5UWf$H3kd@)B1zOeOj}+vqk)aY!c4P z5}?&`<G|H3Ld}6Z*pHD1xi4j&p8qxo^<9ID3`FzlH;VG}b9IGE4st^m-Us8+L=Bz7 zl-xZ+l^&H<L8la6;Y=?qq3^4>Swu$VkEmO{loY6$j?~zkxV(7WJ8S^Q{6^}bG(>=H zCJg)@wtQ$ocu52<qUg9dZDvGqtkb<A#j!P+rNvIYBnx_pmvUvY1Z`GAqME~<W^!~C zFmyOmb|&HOejl#*(Id&q%W95IeVjuHhBujw>hqBqJi1y1{8BFTJNn%$XriX#C2Hsh z{EoR@l5s41OV^xeZa$&6ldW0Gb5B#%=mMlS2dyHG09IK?Ej26X<TvNfnI01v6_XbN zF!`)ZWq4j?bTx!MvsFj)jusp)B2!P2k#|7>l1fugpG`me3hF5oWJi0U@2NL;O=KMF zK5oPpvk~T9E-Ge61=`x46so!UkYic(^-i2(4@RCI%}?X#e*9n>#;#eNleb2*D1VLj z#5YGQ>c7@$*L(FBs&4Ln=s30s=tsW~z??fsN%rHs8K)o1ciJ0t3T_GJMEypL&7taW z8P|K<xZd3B=)c0(QD=wMhDLqkCu-Vg@L&+S2Ky1XBBYJtTQWU9p!bSO;-d?qR2;yd zXW=}29?AkHQ2upS{<bTf0M_d_RdAaH*Y}KP0(K%W{+v;V;GS9vgRyexxTqa56`ZHK z!=AQy!g09z*`!>6D%ZmNNX;D}u`;lcK=Qahwbnqs2~vD)3bEkG0QKGmj-RuUsx!Uk zNfRYe*^%3$_}13SRu!m-&f&SFkLJ*JQ8p$!ow6dmBBPvty<t@T8%L$#pO<{|m6F3v zivcy@h$|IOPu$kBp6!Tr1NSLSo==UnBjRv1^x!91N0WZ9g5K;;zLL;#eF7Aji80Qu zh_dQdR;}c#PwqSe;^in1e?g+sujOLZWH=^&dtBghxYn-+1JFqk?j9zSVuT|(MdiBk z{1T^}g4}}a0%?voJ2@vgKY8a`P>N}uh-?>gl1XZAKPFc$H8nFmRbvPPxK~0d6Gz0} zBvJ<9pPW2i9|pXkqPzmgI)c%Mq{uiQuyX-=lk5HcxJt}I`ukv1jlq528)Bd)SwZM` z#=Vx5^ctS7hg@!^XmI4J*&5JkBP9VeMnt^~_c^F|)j2G|RsdpxV=zJIB<H5&r*r^u z%dtMBTGxPTo3U`=j2R0*yg{(s4Ny-}b3jWNDdTpju}xVnvl1J1A9E6-29kq8=h%v= z3q~A(B|ejJ6{<pAHZTe5ipuMDDbbBe3d$>#+z-DJn<nV?APpd18o9JEAq==h<byP+ z1&O(d+og$#=_$z??pMrDs0-tWS7Kh}vC+B2PM$;5z>|W~c$4yYy({+$-H>epg<|ZW zFacvWe;t)0d=t|>o!9}{d@&dU=H4B5>BG{}!lFEYot22Pqs0lCadAozYbH~%-cQ2a zm9gIPj+z^bySi-{By8Ho0(oQMhckF?m+aebzn$=(e>u_!od!Y~SC~fpFr_;J_$~pQ z5#k@!nBE=5Ef~yaiDeEjZ}PW0ksIQ?OkGM&+8Ju;s1Mt`NKG$^XOPJv<6NYnEw128 z!p>nFXrI8^=D>$$#XxpEIMQEc!HMgz1=*?Q&d7}S*W4I2mMIk09%}>}b~-X2f0+tx zR9C&OV&`t<?x>w1I-aij64IR2dNZiq6&uVT+fhwdy}?@zcD?gRS5TnS6(lFRUU~Zt zGr1{hC|3h`TLCB8hxv3jN`Nj2MR4}m5racd&4tPII_`2TR%=j9ImQ`vjzNH&Ll)WH z1-sOJ-hxYArrYwF?q~QWU^~}I*jAW0sIi;kx}m(gkhr;8ETps%TQQKcfeua&b8)4( zppD}ylFQ>uxSJO*-sB{DHR&lT%hQ#VL4UNQD77dlpHIryW+$dYafZ~9BVO36iev>k z4Yb^{Qt=PPtU$mR2R0eDb4;ThHYq5Hha{>jrc!T(T?UPvE{aV}jE@Ckr6eIQp)iF{ z%g+Z+5k$VBQX6S6n$F>DU^SH5`D^+Z#)|^Q)COv%Y%piKs2_4*!Ux;SVKwfrF`e3T zB}LmI|DK<_Jy(@3(I%#*CM6`rI~hcVU7}I?ZzLR5PM3Wn<Gx|NRLdp+v=(;e#=Ssq zceG!T4AWi|=ydT|?;n7x1#H~(s>I+yb|?%3$yB}Zp;JX1*%x5s>9go16*%<LYr~K} zm^|<%=H9K{)p(lC8t#hP_?m4c{I=q-?GXvdY1)*yWGN@UO|Eo75CZrwAumxy!;HX# zX_`s&8xi*rJ0Xz>wbicZy09WXv?wq&avK*{Qjt=w>Vlf#O4VlEB6Sz1D)u;%-Sgin zfpm!(^;yP{)rrqCuuYl~pL5VQi&c4J6i8<_bcG6{JucWTRN$WWHApM_lc|U|A}c=L zY30iJ_^gPMI46!WR?g35dW<VY!0!Qr2I0@XLHmCHeV>RkBiJBjMXR}4vL??ZY77FL zEW*?ZV?Wdp9Ep6@sIwL<AoJ9}jAYe|y3I2E=iGl)bAYxwB;$qvbW+WMS98!-4o5_L z^xhTd)9Gwt2k_Nc@EVNOTgmLdOO<h#u}HZxCg`a$dYhw`0DHE?-v?m2wtUd_7cSVL zd0^ha@H-D<9h<dWWu5FSKw5Ab<0La>96F0Vw<lH<_Fi-RM-26Q$lXJiH3hl(Il0+V zPVp1z5EbWCmzOX`oRaEjZI0$A+JUxSqsHR+QuvpHR#B9R{gyHd=cdNy6{F}ZB)ihV z^uc38k+nV~gkyINM9Agl=W967@v!oIBI7Vee4*yHMWzN8g*k;5)kW`8af{XN7=d~_ zbpd%OOO7wWS#Wn<-EO{Byt^(eNW(pmWpRqn)naTOpn3-o{ur{7NivQdtTFo0kz3ea zmkCI_(CHjW#Xd|_S62pf8Fb{&)O+_fH0<5t3vv~<|B}HtNoD0{7HG!l)5#0+m#DO~ zw4zdJm6(hvh#Y(BCO%DEL?lgon!1SGdXq*mXA+$U`>q<l$-8R9gK?nTh$W(h+?Tn# zc2#{>t=I=~*i~WsL39t`4h`JK%HrzPH$Gg5=^T`Ru3S@_KL-#SE+k}qR!BXk94+Ip z$;)Dm=)ox#du(`n=*mxSeSY%djjykcoyZ&h;@0vZ5fNJ>L!OLqEG{i6D=n7R)N=!; zPwVH>GPRYz|LN83s)E9z+@egbpA0;)+)>)5f4=56U#$%Xj7%8l^I8qJ9)jxkA^z8J zl*xe^#r!x)aCz9y1U|h$mr<x<=Ln8<<(Witov1GmXWX7l;{Y$2m|UR4?blf2<c54= z)+Gno9E%oMDzgeD4={<KS6Fc1rLjlV-vh54W*fx}F^vMCD*YEsfUVBxjoGg^+Dd{p zL#tsg&yg}VBZ+@fbK`N2MQ_AGp@mK3+#(rmL|lhuoWXt^Dijkn6geE`Bv@oGE3g>? zudY3Zy}d81x>tT#aF+a!l^d8~SX(~75;$H%F3~FrZAM~}R>gT#dK_G>0c@*IH0R7$ z8@^U?CwvdBUF++&W^IG-@#75*$9Xo+**e6Hz$OyRZYU{Bj$`|NOyR7>?a7xiY%Cc# z75mGPN3y+~-WGot-Gxi2#4UuXx+=G*5=S)>##x-gWj{8ioCzL~+){I{lc@P}YNdjL zck{D%CK<QsOgw;-j8VtNqyjC{aL|BBn2}j9eJF6uz4aq^sj2A_r{f!Q^5(5em99-) znVzNO$Yt5cUpN=Gb;Ae=6!+1O7ZE#uYW09Rs6Y(?E8`tbS%<l(5a>SJah1mbDoZQl zK1Cm3jQ(z17W7baObWydUGun__0LYQ3}Uz32<<G55;P6Yj-1|=k?J@chGL>He($3v zuqxuBQljJIdE+6Q=f?2QTErZ6Auil>fbVj~t|Rf=9dw8%0`Z~UyANr&9Z(Szk<BMn zCRE0kOEHwXRwxNE!<KQzhYN3j>J*9C8)Y3j&<C=BFFUG?(Fw>GGH&Bs>flCYs!aj; zrNJ5wcs#W`R9}h<^OKS?LCiwm#ex5l%u0`q3x^e1%&C@zZ42<z_^x|r^&k}|BD@-F z3!9EaVLRDrzD~m_CS{5Ya!F}aeRQRkOOd9eB&A{5vKL3i!Y_^x{jgD?)_M`NyQ24^ z&>dk4bWSYyVH{Qxw(&%*v3;EmJp|@{S?_V*Kjj!&D*JJ8Gxj72wQlWCta%X47<W6X zA04u(RdB0)4xc=Em_0tLITEkdo3km0O@5GZ!}>wF!J{zWT09y_I4<w6%LOGG$R1_q zDA9g8NXKA2b7on`Ff@_S8Eod7Ppnyro~6TrrNgQdpysx-b2>KB73FXiH*hq|3)A}L ztd~D-Jd(S2FN@lbS8=K=1}`o=bK+|acLWmw*i`w;824fmm8Y}X3`(=+;7+>`0~cCd zqG}U&?@@9fV+*7L0m}z!15*VX<s2E7MoL+SW#d<8#vL^-Eyg!{GcFChff|lY>qZ`b zE(sg<6!^ua2gi}8+##S=abQ7cz{;AK%+dY<5H~TWBS3=cN87{bE@fOc2a(cYkRz=i zJvefcwGxy#^Bi4)?$`&wKpvd17adFsdkMb~bK-`**qd%C@I@7cp_aosTQFMb3n0}W zRdbNhVq+b3#E$Ts0f##d(olUl0sff@>;x9f^75ZlAYt|wF9foeHp`bb3$d?Ro<JlJ z2)7Hzz*}mUX&#EGGGE5=50;?FkWhh}!1<{_EO>$MVkC`!#y>{y&H`tn$#R3otWWp1 zUU-8qybH|4Mju^&SjfLazx?nIPA|XxzqH7DSc=3)CDLR6w-Xhbbt1}bs7sMxg1}j@ zPtYJ}6nrH3s&}70e4jO~R;_&Nl-7Bzt6Dd<`n7Ipjcd(mt!iy(J=%J;_1o4zTA#OB zwef8O+6J}_Z=2FKuWeP^mbSRIoVKdAhPHEUSKGdA`=jl7yHz{iKBawL`>OUW?Q!in z?N#j!?dRIBwtw6H$5Ylf1W0-Bf21sEwQ23$>ejlTbxo^J>!#MAR&8ruYfbBs*5=mh zt>3k_wh7v7+MJQ{ptg~1Zfy(N*0cq+Y1{JJYTAypHMd=F`>w6EUC?gR-n-qceL?%0 z_MmocdtQ4@`;qqM_UrB6v6NqYkG{F$#lja;UyS_r{<lTnF0$!~OE8&@B`(9zb8Un+ z7B=QKW;O!r7uHX$pIAS(Znti;Zngf?>Kj~{{ciop`l0m$>)&vJcHjCJ>z}QEvi{Nf z2kY;xzq7t)eb@RM>#uRScH8<Z>o2Xpu>KrZZMUp%a*f8Gw)MX><*NVk?f>5=v7iS= z04HD<#~5~Im%r>6^Vw=^*QWvt<3JT$p6@!6CDAg<_q`V{p1-g(6EmL{2+{QqZ(U=~ zlGPu+|L3?dZ?w<~g3OxXPb=6e(jpmwU^R>VpC0zT+kGV)kO*UXH`>`dCJ2E9=BwWj zCK6${FgN4F{NQ16usGqSG{(o=wSv(mKPI<VO%_h5HHLR%{@A1;gvsot{4xJsd3f=H z<%fMU<%#)08DFN~U7nZa_#f}^&*EcYOr|mK%pH$0Kls+|7G*3?x8GO4i_MLB*@65l zf3-aSA9gq6cdcL?;u>d6qbu&7rf|&7RBmQBy_?cDg@L);_-MQGZTt>9>d%e&!BS@| zAB&g08y{_Vxw^kunBHMBe?pkdUw0n=&188pK7W57%KDbcFKZ7|U3I7DhQ9iu+ujwI zDeQlmT7iQ3GnM<_@(lOxwzlauH=5#vf1xq`?)bXht(j@c7wScYcjV>o`mpSdll1}i zm}>=Yc#Q3Da%1Mpc)IKZyW=;yTfo2Zd$(!w&+=%h3sZUE&&}k<^1<AV-;k~`PP`+Z zFY}1FhT-Wc)C%UZ2YDNY=jFR84SZwmEnso7_BZwcL03Gcycyl_&6tkKZ?|cR$Cy7B z4r#z`m?<pFBMZ;U#!TdaJ?oI3h&?R4@jFcT?P(gD-Qk>#@d)7OmB(0afuINbCe(I) zV{T^McIFq~#xaw*v$T!r!+bTK|FoO@!5n6hh%l%amLHZ5%n2|3YXutQSp#?D19y$_ z(RP)k+n>r<hVJn2>jrnO`<pQzrte5^I&(*I#)tVerr8}|w?DBdEb=mx*;(F9<%0jg z%*~jNR^Tkrc8wdOKg0vS3RC!(Cfu1_2$Q8D#lzZyxv})w!{z|i*Wngma%ZN|uI(~S z^MvFt!64JOSN$SwHx@6-XDk<s?`0TMTv)7`TerJ23(v~dowhN(SL6QQ*aw*M+?{`C zn0CzGogZU8BR><FJ4>s}--{Qf`0zdj-yKcw-Ql|Znfx0~w!zqd?@PM#J($IXcPY%i zEZ_h1z^@g1Ol|+4@tg8wGTC=#XOF2am>qfKn907Io>$+Q-Sqy_u7zJb-R}@W`8!UQ zcf@Io%VaV)<Ac6rii>??c4o52<cG!eYB*Dxrf{Y_GXKmj)e9D}^04=RswWl~YkwBr zRBls#Ukzu9_diWjzD)Trr7Jo9cb@Uh&6GaU^ESqDsw+KH_*YF97v3G<uP030xBVTL z$<Fc&d%teK#`3Xm-R_G_`Rw*T!&DZ$J2N-adv}>#O#V%#1nXgU+|F>@jCcpKZ_J&A z@3MF03-+%5t`!Vm@tMZ>tLZTRq8EaGtY0v9QyVgOxLGr^J1@q*V@d<={Y-i7cC%-3 zywbm3mfe^J;$ivj&b!(ametFDK5R`erNd12{AYbi%)83U;>Nr+5`MbsN-G#{3WIoD znEk*1TOcrh-{|8tGo`?++wTaNU3N3<wnuc^Ed;iLN!@ngEAAqko87zJ!8G&IzeKQ5 zu(I1-ik*8yx7`w_>C@eIPM{E6?6zA8c)@K<lV(CzJ*xzy+ioUsFk8}XHy2o$ZSJ;P z2>O^scH4!o_z?+Q%*wmn#jm(a1a)TTyWOP%NAtDac1wZ1xhWn_FxWi1+ucgwYJT#~ zK%Cb7e0;;4r?1`W?L2GkmJN~4qeqVV*Kp^l{{GI!Pod5s-l5(hTfH|7pBcC%Y-)se zXkdW%%=z;?=1iS7X}-tI8Os*TU*xgWJ0#REaEtTU;p2yoG{&*O-+OJSH$rdp4si|( zbPn_NcK$oTQ1A6&%>Twfe8iWHh}$_VWbFp;fVCl;o!5qih4`%tH+tC;80NR$I~2)> zggJMo|95_U!@`0ljTphgukFg)aKFHRbQ}R(I`1u^-XjEW3IYW|f=EG#z)#>K@D+p! zoCVVbYXw^c-muMrZHr(7zB>y>3q}e?3H~J*4*OJrKYq@ygbFpjc?&`jF2opm1ANXz z>{}4$R6zvXL-7^>a}gdNK{#Sq3%@f3^9Az+9)daWH4PnaKI}6EGX%>73t(S_x2487 zLy<!ko3mgnVjhoYB<!YKEkJDEh>xYu^5reqXbk0y)C1uXhO)6Q|5RQUW<7kE;@^l6 zA+LmC@2nIomJp<|0saGwdEX4TwQyzbeu8x<)8DadK`8dN9==1n>mmd$toB~5jen|b s)(&B4mq{38BT$mA^w<7dxZ%e9{-66Cfg0+{%@$)VvB8fK@L&J^FN3;7EdT%j diff --git a/site/fonts/fontawesome-webfont.eot b/site/fonts/fontawesome-webfont.eot deleted file mode 100644 index e9f60ca953f93e35eab4108bd414bc02ddcf3928..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 165742 zcmd443w)Ht)jvM-T=tf|Uz5#kH`z;W1W0z103j^*Tev7F2#5hiQ9w~aka}5_DkxP1 zRJ3Y?7YePlysh?CD|XvjdsAv#YOS?>W2@EHO9NV8h3u2x_sp}KECIB>@9+Qn{FBV{ zJTr4<=FH5QnRCvZnOu5{#2&j@Vw_3r#2?PKa|-F4dtx{Ptp0P(#$Rn88poKQO<|X@ zOW8U$o^4<&*p=|D!J9EVI}`7V*m|~_En`<8B*M-{$Q6LOSfmND1Z!lia3ffVHQ_mu zwE*t)c_Na~v9UCh+1x2p=FeL7+|;L;bTeUAHg(eEDN-*};9m=WXwJOhO<S5arI2VL z{9Wg-S+>^lgVEPBX5Gh_bo8QSSFY{vM^4hsD-mzHX!X?>-tpg$&tfe27?V1mUAbb} z1<tK+4MW`Pa4jnl;E*j(B0n$uk(F!KZJc~{@t5I$iZM&?>dVewCjIN7C5$=lXROG% zX4%HIa)VTc_%^_YE?u@}#b58a4S8RL@|2s`UUucWZ{P9NJxp5Fi!#@Xx+(mZ+kdt3 zobw#*|6)Z(BxCGw^Gi+ncRvs|a|3xz=tRA9@HDV~1eqD)`^`KTPEg`UdXhq<A4?@H zt_R?$L#2eDG1+i2-_BIvwM}V8*v<M|x)Ol5=3q|wT6^^sSF#if{VRuqK`&g#OivMF zTuw9U6DFO7{4$JD&HfE{F4feK9)zWDazK<BbKeE|T&<?rrFj@iOL6o@IHe1U{3txl zngm=fJ|CWcQ#vIr(x(VcX1brgDX)CKIUFp655?tiA{Yf>18})-@}JTHp30^)`L{?* z;c)alkYAc@67|W!7RDPu6Tsy@xJCK8{2T9-fJw6?@=A(w^}KCVjwlOd=JTO=3Zr+< zIdd?1zo-M^76}Jf!cpLfH`+2q=}d5id5XLcPw#xVocH5RVG7;@@%R>Sxpy8{(H9JH zY1V)?J1-AIeIxKhoG1%;AWq7C50ok3DSe?!Gatbry_zpS*VoS6`$~lK9E?(!mcrm1 z^cLZ1fm<p3RQ^CYW`aW;^)ops%~EhwkAbiO8bCayMIJSjZ{Epi9S70Y2+zY)zU1<4 zl!H&@!zizTIFu{m0grrA6h{tu4TG~OX*P?TlxD-=2K@Ol5J!D1(hwdwTtq&U??4#h za(L*lxitBF5Jz~V0B=Lsuv}URcuJH9)h+k#0v-%*xdGK62Y4>x5Ds`-ethCvMtDTz zMd=G1)gR$jic|1SaTLaL-{ePJOFkUs%j634IMp}dnR5yGMtsXmA$+JDyxRuSq*)bk zt3tSN2(J<@ooh3|!(R%VsE#5%U{m-mB7fcy&h(8kC(#>yA(JCmQ6|O1<=_U=0+$AY zC)@~M`UboR6Xm2?$e8Z$r#u8)TEP0~`viw@@+){#874R?kHRP|IU4&!?+9Cy52v^I zPV4Xd{9yc;)#l?0VS#6g@<rW5y(qtdH2J(3b5d9?JVfcKZn?5Wd4+kO<~yFpIfs7; zZx4Ov@Dc3{d`2P6O=aca2J(l$1M$U$`R8GVyi4G}0C4Bz@UEkH_|1673GY)mz}m^> z`#y))03Laq@^6Z#Z*uvzpl{$<L~n}EyFx#G=luiUX~4qe{2oe|&y#RRo)U%OO?l?$ zTZzUf(in1a628T_AUH~wt9L$~@aM~<H>JzFJgn&xHlNBS|Eb!E@}~Z$^m!a9k34KX zT|VETZ;B_E$Ai8J#t5#kATCAUlqbr&P~-s)k^FfWyz}iK@`B$FI6L0u1uz5fgfqgU zRBmB>F8s_qp1HWm1!aXOEbpf`U?X|>{F`8Md500U3i;Mh9Kvbd(CeuC>077ww4g^h zKgM(A48W`XEDE~N*Th^NqP#S7&^w2Vpq+df2#@A*&4u~I+>t)9&GYcop9OtUo=;2d zGSq?IMBAYZffMC1v^|Z|AWdQ38UdJS4(H(nFI<|%=>0iAn3lvcSjIR(^7r7QuQI0a zm+@Z9QXmf!efG1**%Ryq_G-AQs-mi^*WO#v+tE9_cWLjXz1Q{L-uqzh<GtVMJ=FW> z-Vb`UBlaT|M;ecG9GQJ&>5)s1TzBO5BM%;V{K#`h4juXPkq?e&N9{)|j&>ZKeRS#3 zOOIZ6^!B3<9)0}ib4L#y{qxZe{ss8}C5PC)Atkb2XK%PS)jPMht9Na0x_5hTckhAT zOz+FRJ-xk0*b(QE(2)^GQb*<<={mCZNczb3Bi%<19LXGc`AE-^-lOcO^Jw^J>ge2~ zT}Rg*O&{HUwEO6RqnV>GAMK$M`~TX%q<>-my#5LOBmex)pWgq|V@{jX>a;k`PLtE< zG&ohK;*_0|<6n-C93MK4I*vGc9shKE;CSEhp5tA|KOBE|yyJM=@i)g?jyD~Db^OKg zhNH*vXUCr$uRH$ec+K$#$E%LtJ6>`8&T-iBTicKH)<!M=AO4U)COR2qa!?Q>SNMZS zB8UG!{1{Y=QL&oLMgLzR(}0Y>sN0TqgG|kLqv_VcVSLD)aJ?AC^D!bLa6K5Ut1)YA zghRXq;YBrYhrzOK23vXorq6v~v*CBb?*bYw$l-3J@c<vstBiTmaD9yVwqv?YFcxrP zNk+Qj7a1#A3mJSG+%B+K5e!p=mm^&TrihAlj8&qn&^EYU#=_lT%|<X*9l}%!m`D#} zu}N5--^8?uGHd5ER)=(T&!Q<%M#D#7ol$;c3J&k#U2upS;e~sIv5~tNYue6O^9770 z5^(S*k#{l;_Y_!8e77utLz-6L*t!pFqZJPMv^@c~qZ1DA(JSHJV{A-@vG%o$jfLxI zhXY>Y5H}8Gr;t8{e8!J}L*5e>!hOQnM3g=8eoXDiYZBlmBW?=(Qvo;ib;hP4-|5>J zo6*MD%*UW90?aI=ncV;fJZB$fY|a73<^rd=!0(I%TsLE9TH#hRHV<&~b~82~@n<2= z1-*<dM81nmj4e*VZDVXnI~;I17jR2!;84bS$fIj5W6P0#`A1-l6u9>oTQL{zWh}4H zGj<X3ya;7p+zFRq?2<`X!mMz>X>}SbW{R;(k^VBouiebp<&Q9S1<Jm1BV+5Fa7eoz zX}*Mfu3>P`GIlM(uLaz7TNt~37h`FJ-B1j-jj@}iF}B$Yhy1^cv|oM`3X|20-GXwq z0QapK#%@FUZ9ik|D}cWpad#li_7EK6?wrrq4l5kOc5H@2*p5ENc6Pxb%`OEl1=q{i zU1`Sdjxcu562^8fWbEEDi1(A=o?`5)DC_=i#vVX^4<hctPZ;~!{fu=Ze$QOS9%_Jl zgt3R);Px^0$OVi&ihTBMXY89M#vVgnkG;p(x8VQQJB&S!vc6pmcYv`2)8Kj-`;U!q zFEaL>5ZpSrpE35`g>WA+_QYDo!1%Byk?;4A*Y^%H_McC{^)mJp(mf6Mr$1rr8Klp< z@9$&m+0Bd{OfmMH!q^XxU*>tneq@E)#@LU6-}5Nz`DYpXi4*QA#$MRP*w045^)U8x zl=XAu_Y36n%QPIqUi^r$mjH7JWgdEmv0oiv>}BNj>jt<jjQz$0hxc!-a0vhHd(f)2 zG4?ye|L%UqUg?5Ey5Fy6?A2*-PcZfeg#7_{y*3{XIQ+2~4rRQag8La`e*)~EP|rVa zX6(NZ_g{}O)&rd1K=@zQGWORH+?$NOnP%)Q!2d17*xP`6`#r|qLHOTaW$Yht@4~%z zn6dX?X6%F48T%*Vjw+0O_z`0tA??2sjD75c1D^kl^q(w*LtdXo7(0ftj-$-J8n~wz z%Ua>O;GSSiGr=LO--M;f3$4%-kcdA5=kp1;?w1)iU%<G<3AdZ^qGHCa3fv=%+Zq_R zV=8uRVcdmpS2yGC7a8}o!M%xTdOPF38aTlE7sB;19>_3WyqWQmjf@AcVZ3xc<7I~# zFHgbYU4b-}3LN4>NEZft6=17@TlH$jBZ!NjjQC2%Yu;hJu9NWw<k5)t$aQcZG2Ya} zc=HIj1B@s3F`h(t3t(C!P_J4UA2p5f(N4Ht#>Z@DynQp=tBj8Wjw$e9<5A{>pD{iW zZqogXPX_!HxT$LypN98z;4>ox_a@^r4>R7`&G@Wh#%HG(p9^;e{AczsK5r7^^FxfE z1>DZ=f&=UVl(8@Y2be_)+!n?cUjPUAC8+bcuQI+Aab3F@Uxu=lJ<IrV;CVjMUGNy= zE7mc7;Y!9YdY$o=PcVM*62>pt$oQq38DE=X{7U3=m6P!eKVy6&>UK5q-?WYKFCon} zcwbuv_Xy+HBi;48;XYwJy_)eGknfFvzbOHS_{~WF<pYf0jPEUo+lsWewlRJiIpBAD zC)_^9@0ic{c9eIgm+_rQzYFiX6u3Ud??&GD0QTN%7{3qs-4D12OmMF<{vg7>Rt)zJ z<K53Q{*aaNhZBrHvXSveQSRO}<6lP^`^w;u@4oj2-TsA)f1`%+Z-(GrWc;zm82{Ej z#vh0O@hyyh8|8l+{sSpE_`mlo<NvuDs`nnoG0*U45dX|B#=jqddxY_4O>ijpU?=0x zkwe%IkXL3J<39wBKYX6?A1iQgGX8uw<3E|t_zN{~?=k)}E8{<h`vuDRCE{NEgz-bb z`B%vQ<<*S;`U1v(^E%_doePJ!-)(066_oY+wQzlmzxom5uRX^2ACc$laDPHxm@D|7 zk>7uHGX6%I@xLJ5o5hU3g}A@9GyXR4dV3$^??m7ZGyeD0jQ;~={sZ6d0>}3fa8JQ~ z#Q6Kj>z^jLM;Px_;9g|>2<mne_<V?TAHKu*zmV_84R9I8{~NgdcMs#A0KZQWb`1XG z>lp6?Oy32JW8UD|ZH#LugXW9=mzl&9Ov2uUBsVZgS;-{zFeKKwOfn<ff^TaNlWe_A zauhSki7?ljO!Bld$qRTN9QZ3KFoH?NZA>bOFe$i&Nu~HMe}YLB^Wk1(Qs^2cg^_pF zV@!&4GARo9*fb`^0bBDClWMmysSaUvuQREB7n2(BZbV*<?=fiv+{n#LYD&TNF{$|} zCMDJ}DT(+Nq#YGv(rBa~eV9pOtZ>M)y$0@8CXG!nX&m5FyO}f|^_bYrq)EtQ3jEW$ z;E;a$iwt`}|2xOlf`@fNIFLzjYz@1@vMcQB;TbKpR_b1>hK{W@uw#sVI6JqW86H;C ztQ;P%k-Nf8ey^cATop^SG>2V0mP~Z;=5SL5H#}UQ-NIABSS;9=rYBEjx70^!0%|%? z6H%vBBRb1si5UK{xwWyrI#6mdl~NhlB{DFSQ4f#HYnQ4Tr9_9++!S!BCwdbtt-PhV z2|9^MD=%7f(aK494ZC<j_C#;2LQYwoHEn9>cz4t6dY`X;_62ywrIPovV+sT0pH?<o zI8)k4G6D{1;9z87)Z58I<T$EY73X+FVv^I{5(z1Ow-3$WSJK+2+U?q>+{mwxjh%^> zh_?T`uiv2^KX}<?2f4~CxO(uiYxshw1ORT-fwgp^Gse%pW)R}CgWB;sUjMGPKT5s9 z`qM_z%>>z4HVY!Y%V1QDcBvi>!sD@MEbj99(bg@lcBxTD9~gYzfIm>7jFFl;<kptf z4&Lh0J16O)?m<JPLs9Lc2b)uCMsPl2O{)1p?W1VOV^cFW4_6IV)3KDzv-5{98%9^P zHhMYtHnvucHoW}9mUGifCmc&nSh{qAnwhXPB(q4xq0RLpMvfd&zxhx_B+G0bk5VJK z%`THg_1N~U+x@6IG8lEcqrs8tqr2<x>^hEgOD8Clhu+6jw>0z&OhJ=2DoJ42R3QaA zWOOLCseE6;o!xG!?ra~f^>o~D+1yBE?qxT0^k{Eo?@YU;MW)Dk7u-Ja^-t=jry`Nm z^!iU;|I=I9eR|&CLf`eUDtM5Q2iZ}-MO8dOpsgMv)7Ge`r77T1(I!FduCuw%>+xyh zv~lQApLDjitE7#8{D!C9^9KL8O}^S6)E?BVMw_qP`rdoia-YG@KjO<veNppt{?po8 zZPawq=?J>f%Qh4Bnt8Mcoi9h#JRYY<L%EzcX^RdX)E4ok>3kEvn*UVbReO50BrmV+ z;MZw4c4)uX7XS38vL%mZ(`R5ww4GL|?R_+gqd5vmpyB<Y)}{)XISrwhvn8amYZvx? zW8t-~#;NN&Z{n+O>Rdmy(bdo1(0=sB8@yxdn)~lxbJjigu9=)pPhNBHJ@OCr@Hfy7 zMKpelG=3bck_~6$*c^5qw$ra?cd)OqZ$smlOvLJWm7$z_{bM*t_;dW+m52!n&yhSI z0)LYKbKpO(yrBb!r(;1ei=F17uvjq5XquDp?1L{4s1~Hu@I46id3j>UeJTcx0fQ!$ z&o9RBJJn}4D52n3P@|_Z2y%SzQ!WJ22E$LC;WNiX*{T?@;Pj!}DC|#~nZ>-HpIS<2 za>P22_kUiz%sLYqOLTT7B=H>lmeZ$;kr+*xoe54)>BRz1U!muO7@@$$G=552gn*!9 zJ(lYeq-%(OX#D?e|IqRz)>flsYTDXrc#58b-%`5Jmp#FEV%&+o&w?z>k%vUF^x&@! zd}aqf<-yN_(1OoX0~BNi5+XV}sW1Mo_rky5sw&#MPqeg*Iv+ow^-qi|g!>=1)d@|( zIJ=tJ4Yw%YfhiFbenxIIR1N1mmKeveFq!eF<c3?yvJG3x=9X>I?k+2%4<3`YlV3hM zS45R<;g^uVtW5iZbSGet@1^}8sBUEktA@_c>)?i}IE-<ScP!CrJ~4m4*J$RdTjZ?0 zroE=;#AEo%mBwSa++RbeDCcysdJ4)#=Uw#qq8jVqB&@?2SM~a%DmIp59i|aC2Gcek zP5yF@wV&s>EQTR@N-j%b9$Syc1{S3U?8e~d3B1?L<mq@Xspq7Wl*!);7_uiXnRyz9 ztgucF7Ql|kP^$g+@XuxlbqSb!5v)a}|7}clJ|u0R;a1C%#05zPB?O)|nQ*tZ_yc}} z(T>ij0H27USiF&gR}A>wG-vBGIPuh*4ry;{Khxekv}w<d`im<=j=JTm&bam9%Li{g zW7YCHN2v0Wx*dD6>CTm%_><E26Xd2tyzKt(@d;}}_WHVuLo;5vdL_V-YUS0h%m`gv zS8ork(VmlLWW_u#(UYbGFu8))E#wziRM1SGQkwixU<aN-tT040L;Zw(Vf;yc=C)_- zZt>vhFZSJ)Pw2iv6Q4YVoQ`J2w?yCkiavVTWeVa)j|q=T9@J0pTtcQX!VHnIM6Al- z^*7Og!1y$xN4)5fYK&2X5x-Om4A;1k20|=O+$wl^1T}IRHkcq<^P$a{C0fAii(ypB z{ef1n(U1a&g|>5}zY?N{!tOqN_uYr3yPejjJ>KeR7IW!#ztw(g!*Hj~SpH|bkC%<C z{ffn5FS;&nG&!cZOYQ$@b2^Kjv6s|Lw3vf;`fauofp0fEY}Okh0yyXZu;ix_?&fR$ zZ1qcJEzzch)}SSNyLolMv8AcZWqZo$TV*b~qQzpf7Wo#JG>t5kd^Q2w*f{D8tJPwQ z++kT&2yEHVY_jXXBg!P7SUbSC;y1@rj$sqoMWF2=y$%ua1S%Nn_dvGwR*;O^!Fd?1 z8#WkKL1{>+GcdW?sX2^RC#k8D;~{~1M4#fpPxGDbOWPf?oRS^(Y!}arFj}-9Ta5B$ zZhP0#34P$Fx`;w}a*AU%t?#oPQ+U$umO}+(WIxS!wnBcQuM;%yiYhbKn<dG77ZB;| zoOkv5dGb>NwXa7LiRjmf+(2(ZG}wiz%sgWJi>jgGIsPnZ=KfX?8mJ2^L!4-hBx#UR zZa((80+3k2t!n9h@La(dm&Qrs_<wbnY_iRi?aX#stu|AKNwP@43M~wMBnHzTEs_+{ zLI6$^qsx1^!ue}jOd3DKEE%Qo*&mp=WS(IuHAxGC!54o_ZK19F9?AS9>teRTeB}Y= zShqm6zJdPGS+juA6^_Mu3_1sz1Hvx#*|M6pnqz`jk<&F@Wt;g%i&gunm7lM5)wE@q zvbn6Q=6IU;C_@UMWs|fm<yJn*dd#h0rRUv3sUhTSF*dy{#2R#pB0|ttC`isU*)V4# z581i1f`?j|Q^ITt1pwrK854;<BlL();N3fpX|EpJvEvx8ImWLzq<u|Ws(tOyA-;@1 z#FxoE+5im^Eo3!70am1t&MO2xbV&4L6Zoasm>ylAcBqr(MowarQT7@9BsXzyH534G z1e0`Rlnqb_RAIW{M7dQoxd<?tIRNMnKvS`xn0sx{*u1=$yR9Et-O|U6*_6%Ay$IHN z5$@$~TQ=1g<a<mJFQ11H2SWA&!~qaK&l@rEJwd5V;aJ5yHZ7xAolZ!=9vf#A(2wWz zmJY@1Df8Ge)XNzgK99fG!E$m0`q_w49EDzkMRIdt{_Q!T+XMclaWwk4-$;B6k70TX z=ad;fD4$eagTzUT6ie&{IiRD_wZFb)^!=l^@Rw`awC}qrwUoQXt)(hmt~$;@nUIk< ztJ^DdPr3$*hXy(AEp2UE+9X$Pxt8*Hc&5BI&ND)^O6fWoK`X^9B6tAsZ8^NgKt>g$ z;&VZRA?1jrgF9nN0lg?)7VU>c#YI}iVKVtMV&I^SUL2sA9Xn2<8mY@_)qZF;^OV!$ z<wQ4jt=la%6;Rs2lSt<jxME&J<LXPKt_dMdlb;^fr3DyT_&A`=Q;y$!CaSx<S~nrc zyWDoGm5(!<wTzR0`WdKVZKTN^G;R|kKMduT9G0<og#%$)<8xAFt4E)W10KKJ&)d1o z4`<sC<Hg(0<0+{t7CeTDs^kMpGAO5lNy`T%Qc5n8c(3NfEAx<6L7m9|(lJ)5DXFR| z$-X+app6`?l*NGv^(QU55Mx_g&^D(T_DXjs+v9LKouN>;QVM<MpqCz<;np%{lP4;r zqaL?M%lz`QMm;B9NYEH8+^NG*|9EU1k5yOMCGc#`O|;%)uC9Q@pbV%Yo0}A@{b21t z;99EA<*K}O*U$6J*M`H>jZTMUtC^eDXuo)DkX75sJ*#d6g{w?U1!Fbwid(nlSiF_z zStRqVrV`8MJBg{|ZM^Kzrps2`fI(Eq&qUZ%VCjWLQn)GthGkFz0LcT(tUy)_i~PWb ze1obC@Hu0-n}r4LO@8%lp3+uoAMDWnx#|WFhG&pQo@eXSCzjp(&Xl4$kfY60LiIx^ zs+SA=sm(K<-^V>WxOdf!NXC0qN&86q?xh#r;L)>)B|KXvOuO+4*98HO?4jfcxpk`^ zU^8+npM|P<VnmAAkvxTrO1i6@6q8>Wn*7Nj9O_U%@pt)^gcu2m|17^}h}J6KWCJ>t zv@Qsc2z0711@V0%PDVqW?i)a)=GC>nC+Kx~*FeS}p5iNes=&dpY_lv9^<|K`GOJMG zE5^7&yqgjFK*qz6I-su3QFo4`PbRSbk|gNIa3+>jPUVH}5I6C)+!U&5lUe4HyYIe4 z>&a$lqL(n;XP)9F?USc6ZA6!;oE+i8ksYGTfe8;xbPFg9e&VVdrRpkO9Zch#cxJH7 z%@Bt~=_%2;shO9|R5K-|zrSznwM%ZBp3!<;&S0$4H~PJ&S3PrGtf}StbLZKDF_le= z9k)|^Do10}k~3$n&#EP*_H_-3h8^ZuQ2JXaU@zY|dW@$oQAY%Z@s0V8+F~YQ=#aqp z=je#~nV5}oI1J`wLIQ^&`Mj01oDZ;O`V>BvWCRJd%56g!((T@-{aY6fa;a0Vs+v@O z0IK2dXum&DKB?-ese^F~xB8#<w%^>t6TFirdTy3(-MedKc;2cI&D}ztv4^I%ThCj* ziyQ90UpuyI`FYm%sUlWqP(!Qcg-7n%dk-&uY15{cw0HD+g<v$^10GJyTUf4h{AP0) z3yojKTJ6!ks~r>buz}CQP*u8*(+KCYFiz80m1pT=kmx0(q(xrCPMsUH1k{mefDSp) zD5G^q?m1N%Jbl&_iz65-uBs{~7YjNpQ%+H^=H7i%nHnwimHSGDPZ(Z;cWG1wcZw|v z%*juq&!(bo!`O7T>Wkon<v)k1*pq?0Erd6LZ$TU+JP@MWDIuEncfM$i_*viBX0w*Y z;%(j;^XAR)w#8#Xn{6BahW1+<$TV)OG&V-QbnC5OiZtR=ME>^QZ-rLvkd_^z#)5Hg zxufObryg!`lzZc#{xRRv6592P5fce0Hl-xEm^*nBcP<W=z#E7FWHE!q2ZVg0dD>$v z0`KR64y6=xK{a*<n#EZ4RVO`gVK@#+g?{PH@xgQ9klm8iu4-4<*PNtq`sHc-$Z3dP zBzBO40V|&UESpx+2uQ;t-q~~(7-U6|%n+9D<NSIq(r|=l(^6U!;iP9qDhVJmqhNAW z1t&d({_^B{#G&cZ>oNxW9jv+9)$I9SxN-Oig_c%UK7hZDj_WEb$BDlO#*M?@b>eU7 zxN!%UE+w#W<J9!A8`SInM~?bMU(@eW-^I6)_Jsy?6giFv5=4F(0$ap0m>g$bqFfc# zeDOpwnoY)%(93rx(=q9nQKg6?XKJZrRP#oo(u>h_l6NOMld)_IF<GDj)_<3ARjw>( zs6M+iRmTC+ALc}C7V>JEuRjk9o)*YO8Y}oKQNl2t?D;qFLv4U`StSyoFzFYuq>i@C zEa1!N?B0BK0gjTwsL04McVmu=$6B!!-4bi<kVsKOAEZ$`@dHtYfDz(LQ$C&1fHeo= z8$|E8UnSA4iF7+LWSJA?3jBtPYW&*!o-4=qt(bL3S+U>1u_j7ZpCQm-l2u7AlYMmx zH!4a*@eEhENs{b-gUMy{c*AjMjcwAWGv@lW4<b{~?fYtJQIVy1qx{wK787qUO0nbb zD^|PIWh+s9_{etSW93O9O4X=oeOILxteQQsU1_wNN)1KcQcG-A%u-`2sxm~@R+$@B zTeTWoA2U^(P2Ql%8V$!w{M;a$*Usp>YQtoQvvf*jQ2wL8+EGF4rQjAc;uiEzG%4uf z9<euxePP}NB?M^!gc=Nr3h?DR5n{bowgr6Qcs%SW8BwiG)uu*<>wX{X3(U5*s$>6M z)n+q=_&#l6nEa|4ez8YOb9q{<zcS_}FKdUARG&<Q-Y1+VMcM)sDr=o%qoqbH&pGLn z_-`TUam+q_aNSTjq?NQTh(fWf^DaF$&*gK#$4K2!Q0y+NNzJ*OV}9c4wwms&E_<f5 zE}I9b5dsmqd(P!jh?Et(p}gp4my5BIHl~Hl9ELs@Mz}D!fWZZrghCz37$P~Mu1d~? zwE}a>(?8h1|AYN<53x+g()8?U_N+)sEV;tdoV{pJ^DTD)ZvO|;^t&(V6L2z~TSiWu zI&#bLG#NGMHVY^mJXXH_jBGA?Np1q;)EYzS3U=1VKn3aXyU}xGihu`L8($R|e#HpJ zzo`QozgXO&25>bM*l>oHk|GV&2I+U-2>)u7C$^yP7gAuth~}8}eO^2>X_8+G@2GX0 zUG8;wZgm*=I4#ww{Ufg2!~-Uu*`{`!$+eE)in1}WPMJ%i|32CjmFLR8);bg^+jrF* zW0A!Zuas6whwVl!G+Vp(ysAHq9%glv8)6>Sr8w=pzPe1s`fRb9oO^yGOQW^-OZ=5? zNNaJk+iSAxa}{PtjC&tu_+{8J_cw=JiFhMqFC!}FHB@j}@Q$b&*h-^U)Y&U$fDWad zC!K&D&RZgww6M(~`@DA92;#vDM1_`->Ss*g8*57^PdIP-=;>u#;wD4g#4|T7ZytTY zx(Q8lO+5Ri<k%K578z71a3aaZW>s0v-@GZXC@|&A*DPrZ51ZeSyziwc>%X>dNyCAL zOSDTJAwK7d2@UOGmtsjCPM9{#I9Gbb7#z25{*;Tyl-Zho(Oh~-u(5CLQl;2ot%#Nl z_cf{VEA=LuSylKv$-{%A=U+QBv0&8bP;vDOcU|zc3n!Nu{9=5j6^6DL&6tm-J4|~) z9#1w(@m3N|G3n9Xf)O<|NO+P)+F(TgqN3E#F8`eIrDZn0=@MQ%cDBb8e*D_eBUXH+ zOtn|s5j9y2W~uaQm*j{3fV=j|wxar?@^xjmPHKMYy0eTPkG*<=QA$Wf)g`tfRlZ0v ztEyRwH(8<%&+zbQ+pg>z^Ucf8Jj>x$N*h{buawh;61^S+&ZX>H^j?#nw!}!~35^Z# zqU|=INy-tBD+E^RCJdtvC_M2+Bx*2%C6nTfGS!1b*MJvhKZZPkBfkjIFf@kLBCdo) zszai4sxmBgklbZ>Iqddc=N%2_4$qxi==t>5E!Ll+-y(NJc+^l)<g=UV6t5zw@$!;# z*=gejtH)R_xz2Bz9Wrx8RV_99BC1@~sl7QKJt@1+SzYWga7mGi?g?-7dMd)fI(s@A zd?<*&sKWAJeN5>uMgMZH+KM<|+cUS^t~AUy&z{UpW?AA~QO;;xntfuA^Rj7SU%j)& zVs~)K>u%=e(ooP|$In{9cdb}2l?KYZinZ8o+i;N-baM#CG$-JMDcX1$y9-L(TsuaT zfPY9MCb3xN8WGxNDB@4sjvZ10JTUS1Snvy5l9QPbZJ1#AG@_xCVXxndg&0Cz99x`Z zKvV%^1YbB2L)tU+ww(e6EZYzc6gI5g;!?*}TsL=hotb0Mow8kxW*HVdXfdVep4yL` zdfTcM*7nwv5)3M-)^@ASp~`(sR<R{ABvheL^?J0q1YLNjpvh-V;WmRLDe4)myA!Q1 zJT&dpP3~Ox?OPs`jpZiejA>`IsMgXV>xPx0&5!lR8(L&vn@?_Oi2EXy)sj?Q8S$Mm zP{=PsbQ)rJtxy*+R9EqNek1fupF(7d1z|uHBZdEQMm`l!QnDTsJ_DX2E=_R?o*D5) z4}Rh2eEvVeTQ^UXfsDXgAf@6dtaXG>!t?(&-a~B^KF@z*dl$BLVOt|yVElz!`rm5n z&%<$O{7{?+>7|f%3ctTlD}Sc0Zs_hY;YO-&eOIT+Kh%FJdM|_@8b7qIL;aj#^MhF1 z(>x4_KPKYTl+AOj0Q$t3La4&;o`HP%m8bgb`*0vs83ZT@J#{j%7e8dKm;){k%rMw* zG9eKbw_mh1PHLUB$7VNcJ=oL;nV~#W;r|rv;ISD5+Q-FH5g~=&gD`RrnNm>lGJ1GE zw`K+PW!P*uxsEyAzhLvBOEUkj>)1sV6q-RhP*nGS(JD%Z$|wijTm)a5S+oj03MzBz zPjp$XjyM!3`cFtv`8wrA`EpL(8Soof9J(X7wr2l^Y-+>){TrmrhW&h}yVPonlai>; zrF!_zz4@5^8y@95z(7+GLY@+~o<>}!RDp|<u!}<SNlO#Nsw784LY{@BBIH{X919RJ z4X6gOSC(Gey>@N4vi4Y-r@AF@6Q7ET8d9j~&O$3l#Yuo`voKB12v8pK*p3sJO+k{- zak5sNppfOFju-S9tC#^&UI}&^S-3TB^fmi<0$e%==MK3AqBrn!K@ZCzuah-}pRZc{ z?&7p`mEU5_{>6x=RAFr4-F+FYOMN%GSL@mvX-UT3jRI;_TJH7}l*La_ztFn+GQ3;r zNk;eb?nh&>e?Z$I<R!r~8=><$LDON!e1tJ26yLILq`~hFYrCA|rj2uGJHxzz@8b<} z&bETBnbLPG9E*iz!<03Ld4q;C140%fzRO5j*Ql#XY*C-ELCtp24zs*#$X0ZhlF~Qj zq$4Nq9U@=qSTzHghxD(IcI0@hO0e}l7_PKLX|J5jQe+67(8W~90a!?QdAYyLs6f^$ zgAUsZ6%aIOhqZ;;;WG@EpL1!Mxhc_XD!cTY%MEAnbR^8{!>s|QGte5Y=ivx6=T9Ei zP_M&x-e`XKwm+O(fpg~P{^7QV&DZPW)$j@GX#kClVjXN6u+n=I$K0{Y-O4?f;0vgV zY+%5cgK;dNK1}{#_x-Zyaw9sN`r9jST(^5&m&8IY?IBml#h0G3e?uSWfByzKHLe8) z9oCU{cfd~u97`w2ATe{wQPagk*)FX|S+YdySpplm-DSKB*|c>@nSp$=zj{v3WyAgw zqtk_K3<lj0Wa)(KY{<nay@Z_%WF?g?;h`mE$OR&`pc&dDfXrXd7KzbYP_>c5J|0pC zSpww86>3JZSitYm_b*{%7cv?=elhCFy1v6m)^<eVP@H+ybiRtL-{uL<YO@&D=4<CJ z3pzc%_qDVb{oUHZD>n?211803vG_;TRU3WPV`g7=>ywvsW6B76c-kXXYuS7~J+@Lc zSf%7^`HIJ4D|VX9{B<pj`Jyg!(^l<3ph&XWt1jx=+%|H4*W7vIN5}nQf&oY(U-#?L z*TaS7;E9t(u3Kq~_T-MGnGE$8>lBG~IV;M->JId%#U?}jR@kQ&o5A3HyYDx}6N<Gn zi;FMCn;WHD@Vc~ETB`jhUq!tuEz&?&YUoOfY~qKJwwOOI5)^BnA}M7;{mC&-F+|ay z6X6p!17JkLWGEm!w6Kym8U_vLS6SKy3X_0JORnfN5>c^pMjj0Jeun)M=&7-NLZ9@2 z)j60}@#z8oft^qhO`qgPG;Gf4Q@Zbq!Fx_DP1GkX<}_%EF`!5fg*xCsir}$yMH#85 zT3Y4bdV)bucC=X;w24>D>XjaA@K`En^++$6E!jmvauA$rc9F%b=P&f^I7M+{{--HM z0JXFl21+}*Oz8zr@T8JQp9Td0TZ7rr0+&rWePPKdaG}l-^)$@O*ON;2pkAjf4ZSg# zy{PLo>h<boIyC_;r{F)t7+rbn<v=(bP)7u$W!VawH>hTUUK_q5L{o!vKb^7AIkbXB zm3BG{rbFE>fKfZsL4iKVYubQMO_AvYWH<3F_@;7*b}<uSJUUJ;g{%z*;ZMkCFs%V; z(RP7>ss*4!r5a-5Mr{qoVbpXW1cja+YCd!nQ3xt*CEBq_FNhDc93rhj=>>F59=AN5 zoRmKmL))oDox0VF;gltwNSdcF9cb*OX3{Gx?X{Q-krC~b9}_3yG8Bn{`W6m}6YD#q zAk<adqiDb>Ezk)zB|ZA2Ao`dW^gC77j#kXk7>zOYg~2Y0NyG9@9L)X=yRL!=`tj7; z^S=K3l)dWTz%eni<kzjob*^75mCDZS`;y1w*-JeU1eN~{AA3E@Z?o@b-xr^H5W)#Y z&vOy-yfBxiu6iplL<bot^N<x~<B>ebMP!Z)q@7d(l_cR;2OvPv7I~Va{X>R@4XXh- zOMOMef=}m)U?`>^E`qUO(+Ng<N+uTU2jzm?spixQ!b6S0Q$rXfDFv{6Go6@**AHQx z3dbb+NXILW&B-_voCP{%sXBR_R7&C4uNQl#iQ$~g1)$Wdu5t)v8CV~(8!(EezZq6^ zq?5y<SjZbH*Lt3O(&Jg=DGky`uoRyshwz(*v<QFjv%qs1@I$9<n}D$MB5c8igRK;} zVm=^t9QJa9#+9n#UW|U%R{jOg#W1VBgm9E9-F-talB>$xKwZ1|FQ|>X41&zvAf`(9 zj3GGCzGHqa8_lMGV+Q3A(d5seacFHJ92meB0vj+?SfQ~dL#3UE!1{}wjz|HPWCEHI zW{zYTeA(UwAEq6F%|@%!oD5ebM$D`kG45gkQ6COfjjk-==^@y6=Tp0-#~0px=I@H# z7Z|LQii;EBSfjse{lo}m?iuTG`$i6*F?L9m*kGMV_JUqsuT##HNJkrNL~cklwZK&3 zgesq4oycISoHuCg>Jo;0K(3&I(n-j7+uaf)NPK7+@p8+z!=r!xa45cmV`Mna1hT=i zAkgv-=xDHofR+dHn7FZvghtoxVqmi^U=Tk5i*(?UbiEGt9|mBN4tXfwT0<jzO=!<* zF`QQK2@MV4SH^;OMBm*b;`h88y@L`A36S&|;uNU|vf~YyC{B|+mmkfCgZ)Kw*BG>b zIQSzTbod84Y<){2C!IJja=k65vqPM|!xFS?-HOK!3%&6=!T(Z$<>g6+rTpioPBf57 z$!8fVo=}&Z?KB-UB4$>vfxffiJ*^StPHhnl@7Fw@3-N|6BAyp|HhmV#(r=Ll2Y3af zNJ44J*!nZfs0Z5o%Qy|_7UzOtMt~9CA*sTy5=4c0Q9mP-JJ+p-7G&*PyD$6sj<fqp zqz`X-6k@a*f~nJAaVF-kWb}k7p^cZnB;B;<=B(qK1v`HEtaNnoj=o(FVUzL&?Jwdm zOa&j!%jpZ<4BNCM_^6pI1P`dUI*hdsbDz+wk<B(H8xlv2WbkZY;}h8wkH%s%$qXU8 z9%dA`(SFFxSWNq)lu3+-bxLxk_F80hq!V%52uiv&7Ml^f1OQ}OhPf`OW^&~buSMre zNSXk*K%)g#S)!waM5tz>+4b>6a~%2eXf~A?KRzL4v_GQ!SRxsdZi`B(7Jx*fGf@DK z&P<|o9z*F!kX>I*<s~t<&Qwz@3{Spd1FV<6@aL+D$=)y%Ini0I*F;vLBFIX*t-7@Z zRgFr?&%lLnrLjb1K8Z5Z`Lo{n#svLQwFVf#T-%~hswM1I)}y89gU31(IEBPx)P6Ig zQU-DBs69!?Gcy|1UPD{s$SL|FhPtCvYS%P)&J3Lu$jNd^Ot5yGA_<7SU$Rq>;y78= z>JB#p1zld#NFeK3{?&UgU*1uzsxF7qYP34!>yr;jKktE5CNZ3N_W+965o=}3S?jx3 zv`#Wqn;l-4If#|AeD6<h<~xZK0aPIm5Fj3iFu_DI!1IHY#PB1{E&4srchU5Mo`=%y zLyPvrb`+MghTYCPF>_oY2Y||U?Fss}Sa>HvkP$9_KPcb_jB*Jc;M0XIE+qhbP$U2d z&;h?{>;H=Sp?W2>Uc{rF29ML>EiCy?fyim_mQtrgMA~^uv<W}-W6maxh*ylq0b(5( zAp-=27E@ufp>?&@WN@gUOPn(379I}U4Vg~Qo)jwJb7e_Pg^`Gmp+s5vF{tNzJVhBQ z$VB8M@`XJsXC!-){6wetDsTY94 G*yFsbY~cLNXLP73aA74Mq6M9f^&YV`isWW zU@CY~qxP|&bnWBDi{LM9<Q9DDQQV>r0!uDR`<VX?DK16yz1mCS#C7w%NOCFS@4GJ_ z!<#8adubr1aBU+Q1)4)xal)ceKP^Mta3QZHdlGR90FUa?8Fgtm9g;aTz#d88abx?$ z>&3$@xh)p^>voF;SAaZi_ozepkmLV+&hGKrp0jy9{6cAs)nGCitl6Cw2c%Z0GVz1C zH-$3>en`tRh)Z(8))4y=esC5oyjkopd;K_uLM(K16Uoowyo4@9gTv5u=A_uBd0McB zG~8g=+O1_GWtp;w*7oD;g7xT0>D9KH`rx%cs^JH~P_@+@N5^&vZtAIXZ@TH+Rb$iX zv8(8dKV^46(Z&yFGFn4hNolFPVozn;+&27G?m@2LsJe7YgGEHj?!M`nn`S-w=q$Y4 zB>(63Fnnw_J_&IJT0ztZtSecc!QccI&<3XK0KsV4VV(j@25^A-xlh_$hgq6}Ke~GZ zhiQV3X|Mlv6UKb8uXL$*D>r^GD8;;u+Pi;zrDxZzjvWE#@cNGO`q~o7B+DH$I?5#T zf_t7@)B41BzjIgI68Bcci{s-$P8pU>=kLG8SB$x;c&X=_mE3UN@*eF+YgP|eXQVn) z)pd&9U^7<FY8mZOCRv)3SN`T}8zOFpY>r1QaaX{+Wb-<z9`TA8H_)3oU?s}voE~+D zzIwq<+$$#Cct9vPRm|Xlc!ffHNrO9U{cPl%uC6&FCn|i=wyld=b>9S8_jQZC19~W) z*_+RuH*MPD=B_m7we#2A@YwQv$kH2gA%qk7H)?k!jWbzcHWK497Ke<$ggzW+IYI2A zFQ_A$Ae4bxFvl4XPu2-7cn1vW-EWQ6?|>Qm*6uI!JNaRLXZFc5@3r48t0~)bwpU*5 z-KNE}N45AiuXh{&18l_quuV$6w|?c-PtzqcPhY)q{d+Hc_@OkartG`dddteZXK&Je zGpYJ-+PmEUR`sOnx42*X$6KT~@9ze#J>YvvaN24jI}4QG3M;w<>~!2i@r)9lI!6N1 z0GN((xJjHUB^|#9vJgy=07qv}Kw>zE+6qQns-L}JIqLFtY3pDu_$~YrZOO$WEpF>3 zXTu#w7J9w+@)x-6oW(5`w;GI8gk@*+!5ew8iD$g=DR*n@|2*R`zxe7azdr7~Z;$%< zSH@*lQ9U(Hx^%Fb|1?Smv({(NaZW+DGsnNWwX(DFUG8)(b6Rn>MzUxlZhNbVe>`mS zl&aJjk3F~9{lT-}y>e~pI}kOf@0^%Vdj&m(iK4LTf6kmF!_0HQ$`f-eBnmdTsf$_3 zR`hz2EjKIKWL6z@jj1}us>ZmY)iQInPifzSiOFN92j9$pX*CuV8SPrD#b%Qa97~TI zS6)?BPUgFnkqG8{{HUwd)%ZsvurI~=Jr8YSkhUA!RANJ;o|D->9S9QB5DxT<?|_OS zuXZ)0I`?#|uC&pr$PR<{cP*P#ZCz%&<S-xX!(k{3z75ikxU4C5St?4~ozlJCof)+& zk?Gu%o#wH38MtiYeOmTME_;_*D(X9EblR-T%sh#cRsj=E1*dK5*Sb4Xw35*sNodQH zZK27co2;Pc;NPgQKv=irjt+T4yDcodMfia7CO01=E)U;2B#zd<yk@iD27&N^;0LKM zjRSo>ybH&PGFtc0Z>dLwr|Ah}aX`XwTtE&UssYSEILtNijh)8)WWjMm$uT;+p1|=L z><4lEg%APBLn+FRr&2tGd)7icqrVX<g?NZ|E43ax`@SVWfM^I11HrS7Gs;@Uvy1j* zy7Xr!)7MvwXSD}XOe<_daap3VQNbFT6Mt!@A##BiJAsXccuQ-A(o8Kw+^~BfGK{4F zL)eZLZXhX{uE9ik09v}QeU`tXxp`(o53O81$5aE(^3i^F=#sWYiO!}8iP5EFav*df zI)B`cK>FE;+3j`3p~mvsiDMU>yK$19$B@8$Dy4GClfzo4)s_o2NuM3t-WhCrXE>LQ z_CQtR*!a0mh<GfBI6f(F$OBZgk8A8$Ghxxjxo0<rMdA}eba1pA#ue?qU^*n$6buPQ zltIND7K<liEiqaI)mFM5p@U=5RE(@wyWpcKGhfh-jU4YPRb-W0B&%c`;j1mKD0<+@ z+k1KE_deqH%Z=J2+8@4Y`u0S-2}X-taViekE}4>nw#I2S=WxT_H@^Saif`)uhLNJC zq4{bSCwYBd!4>6KGH5y~WZc@7_X~RqtaSN(`jfT!KhgGR)3iN50ecR$!|?Vq8|xa+ zY#*+B=>j4;wypc<a-u<>lu7?wd+y06`GlVf2vBXzuPA;JgpfkIa1gXG88sZ*aS`(w z_9`LL4@aT0p!4H7sWP`mwUZRKCu@UWdNi-yebkfmNN+*QU+N*lf6BAJ$FNs^SLmDz z^algGcLq`f>-uKOd_Ws4y^1_2ucQaL>xyaQjy!eVD6OQi>km;_zvHS=ZpZZrw4)}Z zPz(rC?a`hZiQV9o^s>b?f-~ljm1*4IE<3plqCV}_shIiuQl=uKB4vUx2T$RCFr0{u z1v660Y3?>kX@{19i6;*CA}pJsFpo{n<WpdbD;9&x;N{)C8$1>culW61+66XAOBZD< z{H|h`mJS5C2;ymL##}U*MC%fL0R97OSQ@lUXQ-j?i{z{=l-!$64H{LlTLo{Ln<|OV zBWq*5LP`KJl74fC{GzzP<Hd0C@%L=S@@^uUJ{F8Qou$%$yeLSBIZNzT#mJTa2*wiP zwh#Tz4>_Z;;;6i--QpZUrtHC@+RBlt+=_3TyV4gk=4b{TBJAx!GehYbTby(&-R337 zQ%g2)Uc&K|x|eL0yR*VCXDBqZ89C(obOFYYht(k`^q0OaQ*Y{)@7xE~KQ7XN)hGlZ zl5$1<#s!tyf%>mbIG(9WR`R*{Qc_h(ZGT^8>7lXOw^g1iIE2EdRaR^3nx_UUDy#W6 zy!q(v^QLL*42nxBK!$WVOv)I9Z4InlKtv#qJOzoZTxx86<5tQ*v528nxJ^sm+_tRp zT7oVNE7-NgcoqA#NPr*AT|8xEa)x&K#QaWEb{M34!cH-0Ro63!ec@APIJoOuP&|13 z9CFAVMAe@*(L6<Kx(ytzkNOPuG30#Mu-3oelhSAbIF)_EK#aPF4p2$>g{3h&p2m!K zEG?(A$c(3tr<FAIA!wKKAHh<{*<i$$tr3C?nB@T2O?t(k1%}Z$W@zx0^ny6k0g)x% zGL(35LDTq2CF3WLU9#Xy>J5LHQ@(h3@`CB*ep}GDYSOwpgT=cZU;F&F6(b=V*TLLD z*fq(p>yRHTG1ttB*(Q8xLAl4cZdp^?6=QjcG;_V(q>MY0FOru|-SE}@^WElQTpCQZ zAMJy_$l;GISf1ZmbTzkD(^S!#q?(lDIA?SIrj2H$hs*|^{b|Kp!zXPTcjcCcfA+KN zdlV!rFo2RY@10$^a_d*-?j7HJC;KhfoB%@<h4x2TVsDe~8!E-h5r=-i12H7&6-XB? zxpB|K$uM)qLI$TlMBj24WMV8lLNA7d-0ikcSn4iwx7FB>;*{;(hx_iP`#qI(?qa{b zH|YEvx~cE^RQ4J}dS>z%gK-XYm&uvZcgoyLClEhS(`FJ^zV!Vl&2c{U4N9z_|1($J znob`V2~>KDKA&dTi9YwyS#e-5dYkH?3rN(#;$}@K&5Yu}2s&MGF*w{xhbAzS@z(qi z&k99O!34}xTQ`?X!RRgjc)80Qud0{3UN4(nS5uZ<cHrQ{FFnVbx*vY&CVqaG+*lP_ z?y^}7bLXEkN<IJ+vblA&9wZiTjk%VGsv0SAP2KD=>1#K=^l&$CdhVr%4<67S=#uNP z$hnqV471K$Gy&){4ElZt?A?0NLoW2o_3R)!o~sw#>7&;Vq954STsM(+32Z#w^MksO zsrqpE@Js9$)|uQzKbXiMwttapenf8iB|j(wIa2-@GqE@(2P#M09Rvvhd<D2WTKL2f z=K$sk95sWv4B&wk)FA)!#By3tk$g8+1(<y4Pc3aXf3?V0)n+sXHh79GS4L}!tj54I zMQ$Fbs=dl+E3#}iS~{Eo>u!sE0Mx&cK&$EtK}}WywYEC~MF5r3cUj%d$|lLwY4>`) z_D++uNojUl@4Cz8YF3nvwp>JWtwGtSG`nnfeNp(_RYv`S2?qhgb_(1$KD6ymTRgnD zx^~3GBD2+4vB9{=V_iMG*kQTX;ycG^`f{n+VxR4Ah!t~JQ6Z?Q;ws}Jw|#YE0jR0S z+36oq6_8xno^4J?Y02d!iad3xPm+8~r^*Vvr4A<|$^#UEbKvJ9YHF=Ch2jF`4!QS# zl8We8%)x>ejzT^IH%ymE#EBe<Zg6A*lB6P31a!YG2Bxt>2~-$}ZXtz&vZ_NgVk4kc zOv-dk(6ie2e{lAqYwn9Q$weL<BatOlhM_{lS?;r3bNjP=GQW;bmfCN>#^Nh?MpPUK z#Cb)4d96*6`>t7Zwsz#_qbv6CnswLS9Jt|b`8Mqz?`?H1tT99K#4#d+VwAy}#eC74 z;%UFxaNB!Zw`R9){Pncrny4>k;D}TV2BU0ua-+Fsp>wmcX#SGkn`h0O`pN*`jUj8q zIlnc7x6NRbR)=wP1g`-}2unC>O6ow=s{=NV6pfEo3=tY8<P5lc7mbh?KbynYA{NJ> z=*$TKFk8Wv0K8B_**m*Q>+VW*1&gD#{#GSc(h#YQ<Gv7Ew0wAAg!<c?uFj|x)$XL% z5vos*G{?Z5KK7N!xn1A?(mKD^LlZ7{M^=m;SsaMf$D8NhJY~dzmAm&+)-fsFJZtnX z$1GnOyz0yKfk|ppC|MnGXVV6w!{m~te?3^?oIWGeG`VD~i!X_sGb0?CHPPo^G3&NH z!|H2hCZ%+8X7NoklO<Pgu8WSpb^W>L?*<(ZUx~>L^RyAG3}j0&Q|mJtT7ec|Y7cr~ z+A`Wz!Sqz9bk0u-kftk^q{FPl4N+T(>4(fl@jEEVfNE$b*XSE)(t-A>4>`O^cXfrj zd_nrA-@@u?czM(o3OVDok%p3(((12`76;LwysK$;diTl$BdV)!p5Gj=swpb=j2N>b zqJ1D5E#zO9e(vJ6+rGuy<(PS-B6=gHvFat<HMV4OQ)MhV-B}Sl>&)qr%j7T`vT1ju zIvHwGCk5)id{uDi@-e?0J*(-W-RGZs)uhSeqv7TA&h|CUx(R0ysoiQC8XnxL&RXI3 zO`H`8Pe&^ePw*`{rIJhzUg@MuhUL`IONG^*V?R0h5@BRDFgEF4<Wa~C7aIsk9UDsc zz{x?9i9nA+QX!=$<of)b$%b_O#I7YPH?Ejg?Cx+EPg}8Z<&v(6_1~8!OOu{W<I-gJ zY3@^&A3Nik>5b0jSrg0r{<4X)nw^c)uQ_Ai_p>ic!=K$pmnyqYb=`6fUo40ru#Gh= zMRJxOD(1n?Mjz_|IWyJK5^fh3*n>eI0MmEKq%=-oIdGd4F-LT>RL)Bp5FWxb<T6+k zx!aTNA+yJn5KJ<(Z5HZMLcX9)H(7@rIC+*%$U-e6DaInH34&t}ZDNIVVVH-&dS`ob zcI$%PWyn!c+F@yJc2#5M6RWnqD?nBhRmSFIwO__g9*DQO>4aNLNXB^o?YBSXQ`SwN zI*N~(CQW~P$HpzwrMG4IZKI>TVI4nQ$a-#)zV}LE(xgQ5<jH+s=eOVY;QAF4vfKFP zn7ex91Jad++$PP_Oi5!r7LiRo_A-f`fJqbG`UM3TZb-s7jLX!HpaWM<xFd<rg;}Tv z93K88$eG%_X31scrD~_PIv9K4d|J=M)r*PqACS5^v?MV3Na3kH4nefs>MG@L#e!e@ ziNtg{Ph&qpX9FLaMlqMh>3)Nu%sAO<pz8+=lJhp81P69!KuwU}e;l+xmIMOSVY+p- zeveujeS09gPTz-=wjDqhlHsD%rBjFu)>#1NEsbe=#4Vqx0Y;<~+mV!xwj%}Z=xZn= zSqjxSH4T~v>Xd*=2wmHPN?@+9!}aQz-9(UIITZ==EB9}pgY1H4xu^-WdOFSK!ocZc zd-qhN$eZcN#Q^0>8J%)XI$4W(IW6R810*ucIM7Q#`twI|?$L<V3FnA~E)UBOmStfx zLqkw#lEkcx`L{8rSRz8nOU3*bGy2&<<;VSO##aue=Y8d+%j=S{^05<Vu6Nt}&?W0< zP8?evOV(Zf(pTo;Mzzq=2@)#CLp5#ES04Ix*X(^Ccm4XI@_mQWQ@5|1qqfvm&1!C$ zv3N3dgFQ8Q@r;({Syi<y>YR1kr>3#{B{Z4X(xm&Cb21d^F9MKi<Jd`WVmGimQMZ<O zjIKq{pRFzNoIlWlWdz{?-CsdRIJ~VC(!vug#C%>D=wk_r+a=nyK!s^$<T!yYE})|R zYsKAh*b@ck+kxYww4B>zdXglCdshbfKBqa5aMwN#LmSNj6+DPhH4K-GxRl;#@=IJc zm{h}JsmQFrHCioWCBGzjr5p9L4$t4`c5#Cz(NJ#+R7q-)Tx2)6>#WZDhLGJD964iJ zJXu`snOYJYy=`<+b*HDiI9XPo8XK$TF86)Ub5=NC@VN#f$~GDsjk01g$;wDY!KqOh zC$x={(PT7CH7c?ZPH{RNz}Tel$>M0p;je4|O2|%Yq8@sCb7gRhgR4a*qf+WGD>E8~ z`wb<@^QX)i-7&*Z>U6qXMt_B2M#tzmqZTA1PNgzcvs|(|<peL1ZYi9Tu#YNqPZ>-E z4t*ZT-`kgepLl0g1>H!{(h8b`Ko=fR+|!L_Iji>5-Qf34-}z%X8+*Qwe^XrIS4Re$ zWUblH=yEfj!IgeIQ>m}+`V(4u?6c;s&Ym_6+pt|V`IQ1!oAC@R1XC3tL4BQ7`!TnU zWaoqG=nh<He~ExdE(Ifu%~IH9;p{f312jTI3lxPX)&O@a?$Z@wmJTZtViN8U$KeGO z!3xsC>I@e7dV7)8VzO8ivuC!q{hcxO7fo#2I=<`rktP0OfAO-CQE!ZT@}e7lw;{c) z@2l7RV$@&S5H@{<za%m}efI48ezd9m?a`i2?-?_&0$OqJr1V0=ZKV}u!-9du{-~Rq zM+A!fEhDG@{hv3~bk_zJG?e=*Bg1~ry|+E8egAlCLwWgS(~B1dYWCD#_Payh8$V|3 z>=Bj~^Kp5At=Jq=Y92rXP@{-D4j>U=-a^gM2s-nIZA;u=fbm2BP=Zca5W81_cA>Tr z)x+r@{pu_la2Q(wm`Zqyd@GhNDNT&4oNHb_>w4{jIU}m&iXykMxvi;WL8;y7t}cp& z9CEpR)WlI1qmOq!zg4QTmzv#eP3>NLd7V-+YKmuyLFP533rd>WnvL$F3b}g39PYk; z)^hXQ%5jO(B}-TMio7@t<(V?7M5!ycd)u4Z+~!hym9+KwPVO^Wkhi^Dc7$R@)o$oh z^mRbgQ@5EvalJa}V4Bi3cs^w5pYtbXXz5W|e%+z-K;8M%Lf~BlZRvNI7=)cG6lbjg z?)l8iOw!mU`uaKN@UL4>d#edM9^-ePb(VICy6Cg-H^Ew$n_s801w`A83W!_Z{D+1G z(<9A>WB@>)D%cxw7c?Xv7N}6gg?&TkLX|0@k&VL)YMI~SsE^dzj2^3BKL7SM$!0Lt zj;ytKWw|(58n6_NNH$JVRh!W*wewMr7)H2jOCruuJAIIfPMFpf6j=hL!D3nVT9Dpo zut}|VoG<%v&w;HrQtz<%%T&X##*z5{D!!egoRN}R_Xxuy+E3dhx6!7mlNyuqsKR-P zlP#8EKGt{Ij~8kXY?&*%q)PkPG;rziWPd>HefyPwV49!>f&Q_@Fn{8Cyz{HCXuo+( zJMu<#{Tl}^-dh<itFK$TbJFYyvm37Fi(fDuym?#Kny%=@=PE0gPqvrcsAWI+&oy@{ zw@TM8Q;P!sv0jPE4Y$u*IP<RWT4Ir#4_@dSb>%n<XpXb<`qT0cz&8o~H>M0IrDa@V zMHgAog4`tk;DNK-c{HwRhx%Fn%ir3mex!XeZQ4QY)vQ_iZ(j4-GcO?@6Z-Y*f?u7_ zmf!}WRoGkI#BO9;5C<z8SRi0KOP|9!FkETngwhEn){S5f0$AlLWL(7|jvO}f;b3wr zptloF9tUfA%vSLjj`X=b152*H2<uC3e&rQ!EW51<W0lX_+SH2W+TSAShCA9H-zg39 z#Hhu~#@Eh?4lBDP9y|AibDha6%7#Vf)TWj#9u>FvMobtV@Qm?#eNKbbX!O@xEVhnm z6LFnWu=E}6kB82ZEf!g}n5&IuivccTHk-_5cazDAe+O!_j+dQ~aUBy~PM34E<a8<_ zVF~o><Jx7j)MB^1bL_HVhT@hvH=VU%!Q#u!HO0ljU{Pq8`SQgJHk@_i+?HZPWTM4V zUmL1XWV79lJ0IEtf!J^x?!JhvP~64pEPm(lU-M;s*Bez?#nKy8i_8YK$PzPJZAKU^ zn2M}sIA@Wn_+f5ilZtGTuSk-M(Ld;G{0Lqx3J;b$Qt%NINu5qh!oUhhjF>q0X-LOl zjunFnO<4Nq|BL`!xwvyj&g9Q0(A_<mwwi)l51*zhc~BgP9(v+5)-nfdJV;AikFtl9 z<r7o~CKj>*xLT~l{^nM&k<D<19^%dPe>GzB7+^hP^L&bD7iVdXe3wobJXVX~o*tX$ zI5xthE?gAl!4+v~+ASbN2nYIqNn_#3>!fi2k=g*Hg_%caA#p<x<{Gy;5F@e3V4M|J z4sNa=2$t4L9(VaIB@xN5egCZzSZf8<ZXUV(id!Tbt|ckEBN*k@@4z7U+TWK{d)+~q z8~A@c^Zi%2Q=Agkep2T3g-hPzll@Yp<d$-`%XZ5Zmus&-RO<Csm;9dJ#LK^9<I$2k zASl^h`TjFnnA#HkM=#=UDskHqB%OR+^80HzRs^A7P}0F@Zq%<|E|@n9ZmU(CxJ_{Q z1P+J4uN59xKHj9<|A10#?>lNQR+RtHTiW>(*OFG*-nzu~6DMCrX>xzP`3sj}D!||8 zf3dk-w(NCUMu^C%k|t?sa>9gU_Ms-R2Hhm~4jNfPPy<PYa4c%a9BAiQrtQ!>H!3Zy zV0QFf=MWK%>|(eV$pB5qOkC)uou{oIJwb_i4epV{W95%N)`+uOrLx7fNtD^czsq4B znAWb+Zsk|YX}a?b+sS-!*t2w1JUqU6Ol`&Jrqa5=4eeLWzr1DX1fWW`6MYf+8SOW< z+EMJ|fp${RJ7q9G7J+`pLof$#kBJP^i@%wNnG3fnK?&k>3IUVo3dbs9Nt)x_q|wIB zlBAi#1Xv-<+nr<13SBfkdzI?dJ|3~?-e>MzG(yRsA}I_oEd{HEGZ&7H|Km9mEbL6r z{Ubhh;h6_QXN_?>r(eWJ@CM1-yn6Y#am!aXXW!EfCpu}=btdYT?EJ>j+jeuc%;P2g z5*J%*$9La$^cy>u0DqjO#J%*IdaaPnAX#A6rRQ+sAHhY@o32==Ct3IF&sM14!2`FD zA))>ZKsccTyp$U0)vjABEY_N5lh(@e+Gj>sYOTgf?=82K)zw-?JX2d<X-z&`W3^P= zIs~e={FYnFYfprc(nFF-eo9tUJD7Q0*@;=#TCA0~-BwX#IRQqOMFm|Mp)XlERG%uW z3hDTa{;MkpG4EkD8qlYbI6(8FzE|N#0~M-MaX}dInJ`DBP9?EH*QTn_re-r0k&1Js z;&`pCX4=$>$x}n2Y0v%SjDtBXDxV2TyyxQmN?2%8zkKkKF*!AA$P$1#qrF%fUu~<Y z8a^&|?ZbY4RiQG~J1RM^e&X2M8hoiU&L|m`O{Y?YO4QEe#k)I3=^E6@V{}EJ+2bm6 zl*uz%eP?#0s&C#jsU1pGu{ST;Cy7}QI@BmMO5$E3>URt`tp3C_(>^tkcbHhO0Hh0A zpTVQR{DjsD=y-Bsl#nuTVKRxYbjpSJg|K+SEP+^Y*z3S9p(_-s9^YP5Zc?Vz*o(Qx z?f03co`dGfW}0T>UdEZaW>s0XVEzlw@s&bc+B-9;^^AGsx$AE~!1-7?tn9z|p4}_? zRsM&sjg1>#Rb#6jFBRKMeZ>I_4<%=&rF3yqUD&Lik@7<@2*(0rC)UqPj`Gfe8L&{S zhGtB67KhF{GnLZCF}gN0IrIPU_9lQ)mFNEOyl0tx-!qeCCX<;7*??>lNC*Q7`xe43 z2$7wD3MhiII4W*v6;Y775v{FSYqhp+|6)6BZR@Rdz4}#KZR4%=+E%T%_gX8-9KPT4 zo|$Aa1ohtUet#uro3p&@^FHhEX`OcGjq==$UeAQ~<6AZzZ|l75nn<#}+mo0rqWv5$ z1N<|1yMgX<Jv|>+Qmz?53v|%P=^&74bwqfH?xIC`L()W{|G`j^>kbs7q<$hb6fL@S za#nHyi$$TJ7*i!6estChR}QriMs#yy!@Po#AYde<sLQD;qjdSE;el}uSIvuU`Q!0y zgLdgx7@b2yc|`j(!j7>WL<h_}cLLzLADVAgNgY8#1Fh|}$YYy;vM@nsz|vAPj<`XW zXwGRVeu@T^gKq(%OY=@kT00t1XhxFalF7^$^m^TdY4y2Ihq?}y2^PzCcJmbK?B>~* zUR%)FT#4Q~O-N!O&it}b8zFOmbe=egH*Ka<9jT?dFCMAcagAo<>tKrW%w?P_A_gd& zXwHTn>a>WEWRzimu7EJ*$3~Jfv|@bLg}6iH4mgJB!<DQThd6ka{evPxhuqWwEe5xB zf#O7;g?|$6p!;#Fatu%5;Mv$0#&GhoX;1`k>o60eP#_N!xYrQoMf4&rGLau~D9ila zYGD*3*M<KhTU6G&l35CiwgI=I^x48t+-tU=xLYX*&>NN?v*n6op+dQM!Kkr@qH1|^ zh7skG&aC;+$C$OSR2!ke>7|B6JDpjV%$Jo5hI14PGyx1I=Diw7>h@vzL?PLTzC;`; z?}nkmP%J6$BG!9mxz?+Np<W(k3}{Xl6f8SG)p2HGZwh`~$&Y)0CW(L%$yC_vL#CB> zIHbVy&<#H&Ekz1(ksSJ_NDQ+XH<JJceP;vzNe4O2<E7<+egftoEGifzw4A`fitfZ8 z%yW3H5b@&OiDoF>yg-!YcW8YvE5v*jFQ->F;|Q-IB@Mw6YP~v=jY$~9n@~8MVO{1g z@g=-I$aXs1BH&>hK(~|d>Y9n*;xRm&07=pLuqVYV-bwyCUIKgMdLSrovEs2f3{<EC z{Td;l7^sLu+7e}I#;X$QZ$1d(Uc@kK<6!qdqET2X3|a~Bq2L~XG()AKLenoq?)g>b z<++d|UX&}*7)y8){Ntc{RL*udOS8r%JV4EZ64fUF85n7%NAWejYbLV}NB|lS>SnYN z?PFpysSR*OodDcNK;OVKsSbKS^g;|bSdogA=};1?3rYq|Nc_tR!b2ln>=bNTL59uS zZjF^Y1RoS7qF^>LEqt<#Mu0ZjpiUNLtsc5%t*8}5lW4OWwFXfqGn-q~H)5}2mSRZ^ zKpfQxOe+KC(M5V`tz1zQ)@pTTQ2?NgStmwpvPCi&U9wd)m<^I-w&{(`Vb?Q*4ApV5 z(G}DMfgox!S_C+OTa5UkEbB#G$SC<8vLrDPPT_Uq5N~7`%Js5Ut3!o!f@HJm?b;(N zbbv90V6J7=E&)E`b|}N4n`VOOuvo$<xzmhK>IEMx`%EkX8mpug0yY80enF3?M57gI zQ((b(;dv_v7PDKFgL<RcA=`>|6)q^sb%Gp_aU)<?%bd`03z9(q%&}XLy|QRQI|<C7 z6--bPF*5m=>wp^uX96>jGEsOmBhyuDZ8}+y{bG?UqGqyDfYMtJ{6@xXI>fVC9g+uG zbQzl4fY>P6VAkv8GEpapl2>quqSIoui)Mr95Nuw@voGBux<n^@3=DQrFiIw^j=D-< z9-C}70E|_@WRgXj!75n<^c@^ip^SG01t{a;7BMsk)b&C6Idd?hnd$Uet==Vg8HR<I z*)14S0-E077iH8$r<ILH_)wr-VK~xC@F~_x2E8B~B(1?EYOS))VbR+qy+_OK0B9Rq zkqpfUkXsp&PicMBu{N`0ll^E*BL!DD@c|46=EJxvVm7^%nQdq@odIak%qRidQUKhj zTqFVO5+%^17-()=07Wn`t4_8XBvGq1Yeg$svg;-tV|G-*#EKmf`pk-MWTCjk%(bFe z2~29zmqfJMVkCXC(ldg)sz{POuu%Y?+GF<$+MvgvuYm4Ei;PhYQ9@nIC6}IAoF-B0 z)Jn46FNh_!FoWg?Q7V$&FB$}lVvGQAxClo;30$HL5lDihud*A%4!dB3LkV;^7>%Mq zYqG!&A9RXvoI%gZRwI->g2SYPB1tbg0U9UkC70cRFPTKU0L{E!2e?|as;p-wNwA;> zm}yKfYURNzE545Jz^T+srPZUGX{3qx0H&3ol`)Eow3xXj!2lx+DkB=}EoF`(n^)2W z_26hljpwvSdw}akJQN9;WAQnnHTN=3Ko19hR`Qqt#60*^1acxN84Oi8W-4nXd^@w0 zVpMzKqWw_(cHwQ`*uQ>F4F;Ncc?}XU{q867ZF>zihsu1j_i%f38%41S53RkO-5Bq< z<^ffy6fQNDn;z=lDz2OXjU+MMr0ziZ)HseHI3+}-N8v$8UWEK_n5pL6VPUS<!gEPO zyuXr?;0P)}hXg2na8qB7{+PnO^luOv0oC&uTlW6hPDu8aA$#wIzJ~l%fvfJ|?r(@d zZ$d6|gy<nd)L!I&a&dG!c*ZI&N&+Qos-(A&-w6;V9N13d3b6OtN63SNrhtu$+JpBT z`CX#NMz#SNU&*Qkh%FEB?VwCkm3h2rAc~M{HE_=~5;SL&Gn-8C3@W}JnDaHC!*IY5 z5lD$kSkKhevIhRKN=Q|w^iiez&;<RI$pCf*(l+$1rg{$R^nuGlYJ#kwL)rw241%Dj zN<E}ZFkp4&zJDq=PdWty*EVEXd;Y5|DtEJWhf!%TIXq#|VdZY+?!(kTWHp1u>@YH^ z-F?^<Fq<n;1z>bJ%5Vt}@l0B2B$XfpF!7J0KUW$rc!~hPD3+Ms%)ia=pl{0nuS0_) zMk9rt16uqE&;%{gtVGqhUs{u$%()O~zzC_11`vYVVXfdfEU}YwTDn~JYTSiTDRNih z4#ap?$m%48h4*c`rhEH7?VLTW9aCi~b>z~)W0xM$c|y(8H%u~4?Yic=Yr3WyCvBMC z9P;P}Ra`!CY1TVd3~%qgX48EO<*6O5d**2Osm_lAM&ZKw?7XUKU$o?gjCIcqH|%NJ zuxtIAj>_t$YW%D0ShIfD2DzU5%qnHsRN0vm^B3-wcim7D^;K7~Uj8EuKZ;X3tlbVD z(=eh%wxAVAWPvDL3Mmg=TPKpMGzTdG=aT&qTw(TFBIg<;`kFOrB)&>#;&>KE1kb>+ z2B2dhdAN+pj}^ZH_t#P}WOC_RDs4ppbD0<}eknMnviR2G%#`AniYwzKw-y(_5*$-_ zmw5S-TNmxQbkR$TmM>p=*`CF(EG{@lszbazB$k;2MYhTooy&w{`02hJ3>+yIKEOe7 z@JMkSHwDW^-jsRwlSM}sEqQs-p1n(#FUOllp3=O)Tup&?1<^)a@`nk7JGz35N>n$} zBOy~(>fI9qX^_jCE*5|=cn@Q((|dZ4jk)4MmOAk+0xA#wuDRF-%lTtBwIA!9Gr9Ct z$c`7mj%LBTedq<CzVjKD-~lA6kW3>C%Rm_T=dk5?<j-2B^$w(Zr?s%pFn?5k6)5+W z50y`Z=ao;~spTs!zkG#Q#BNq;w6dd9S;QVGi?W-Q9Z_Bt4dXuFM1bbQ%G0aaqq#<N zBYw(a{saG=^oumRF~`B2g9oJ(o^b-TC9hs^t#pal*JhAFGCYDlK)aajbezcGb7Q2a z!%74D6L@Jlv*JFuVgqdjFQGT%AD)WCl1dCd%tb6gUlqT_<#D!+4HS!VY06@ESTpl4 z&`aQn*DgAE%XLf7G3nNJA3U<C(_p#mGK-;e(UF7QYjyIN>Lu6Ta&XaF9q!a$AUtk$ z*e$72Su7q{Rad`o)%w|Sbyv5rzAip{{VH|GtUY1tf`Dk1!6*HuN9YH|>@$Gpvq}N6 zCzbi<_XLxmE|LLdr@JCzPlDyUYO2J>kDK?krp5CY@11*7)8aCVVb&~zrEGE2O><kf z#TP7YylLLJ@l2Y^uF~t0_DF^MP}RYP)(YBWS{n{lRq&0_H_=q#){hN{cCbb4&mke1 zR2J^&(F+p7y#h))$&Mcqv1nis0{tQ~Rt7YRDB`@3S%Ps@N|rXO1RCi+U`OXVmgWXv zeE^HnMT-cHUcQzLooG4zCyP*~pE5!)N(n(|1D$Ot<qcMiB^KM~Seud}qya%wDZZ&5 zO1Pjx3b0T%_XHlsJCvs@-%#EV9#-B^?ne&2J_;{KMjTCL&)`3$lzN8I#()%mP~Io+ zBl`zMPhq&JT1Pc#il3`0ID%0jBFmGyL7B;esz%%|@4IK$wd)5z+wt~^doB{rHBPcy zjG6luZ0tQUSJY4LPA;CD`C*{EBv!=kfJ3<1sBE0P6Iyy<+WOmf-LqcQUv$@r-|xuW zXEfU<8HF|Tj`ePtJNVh;#mVl;!Zd%ZBotIO;<t7qy93(%op{+=TEnw8a1nENyk?r3 z<AcG64}hzX0EZYFWa!DMyoOZ?Y7A;&%$<+8w3MhpJWDF&HY)#a9A{H4aw#;oh;9Oa z<q+I6Wns_eChbG&59TkfBt}R~S;re$k<pzC>>tojkD`+_dDb1*Ao``HQpP(gi<J+G zdw8GZ>SRL)4OKuTMcNVOb@(m7M?noGc?geUJ;8t6u0>WYa5RLDJ>(^Zu~>-DTzEbb z=Pw6=C#Q(ao#I<MPV*z$u=hth<ZFByyk=6P+SlTUVns|yE`rUOvl<qZF1NNrS^kBk z&e{zrE@XS3{7Js0?UZU#9C<PShF7@FkR9f`Odp*2)V>t|Sa^jEBWtV8YNL5Ce+KO1 zHqBg6?QNQUAP0QbaOG=Lqb?5ZLlZP3JdqXFBbSG?_!QPegco`UzEDBCfy7n?l|<N8 zQ$r(hn_68Q9;$`E7j_lffBwSWsT0OeD5>5O(2uWh*{9fh*}OFkZGv)4J9g^Su_Z-y zktO~$6KAdO?4HIhm;a)+gVRbF%BNDw_qH-YUp3>pUiriPU-DaPao4J;%WF%Dllm58 z#~3FQnvO5O$UIv}o~Up(EN-l>@f8Ipwl+*yG^2h|U81N>`H9+~R;Nq6WZk+k_l_|; zqH`}-wki9Eekf?yVOxp~wx$i7mS&wyRfA;|YZ$pD0iFQM7=^Of;Mb5{*g%Q+MV}ZZ z4uCY|_@8q>JQ{}h=B5NG!svf6mRKr5#bVli@?ZR%doi+~75m0rb2XFdcTK&}XtK)Y z#n$?!<(KX3?3g<BjcJ=FMn7bC>c;rSMQ3)+>e{<=;f)h)dXgJA+DdJ5q_(=fbyjlD zyxOq~%LPEFsh*KmXEIW|_M9hDm%Gdrv97&s&LCvUqb)02CoZ4W(b4X%EB2q(#G5YM z&@wJkH_qwtRocyZt7Y4`(pa=cD4!kEPl#4{yum=*q|U{&O2DV&=)yXRws%3})r>`7 zty6tM=kuW2FpR*(!{^GYty*Jp1woSmG%(Qs4H^#!;!Q>OdkH@{*K(vzM1v#qO$_R{ z7+Jto9d&*4xTs#V1lt-9mM`tTxU{8|32n(X<T7GbZEa-I@)etxwN#IDR5(038)R-* zdG&g$u=vHfdm!uA#%F15qTZ_YAVc5mIa|*^^1!a$ZIirqN0GeLX&p|ZE{6qSIYTT6 zWPEGVU?{R)Vu@@2q<pY%ertGau`^sz+%j?2T?^LUy0o>!6M-UNsS#R?m__F|Gn3X9 z&{djT%C$c`e{S8Bi4#KMy0LTS?(Vvq%{y6Caq7xk-@t{Re0DV4heM^6gkrEpL-{{% z)|>$4EU3Gq;JmPH{<qHyzc<p~60#>E@zsRX+#@>gc;qk2i2FwVHuCI??#%xdiMweM zWaT78*EG!|+OV634wd0UaR@TenRhksaP%AUUdHC<XBegBEqMCQEFOgm@<0T73?O_! z@MRU;-<t#<ayl(zQ6EtX7zxg@m%hcKTY<7x<veu?c7tCyowAnpr(RgQwzL#~Y~ORw zwYA|-{C)OH=AP_<_(T5e8_o1A9&M&)oxiw+KUca|8RXBkJ(pRXy&{xm4>0VcZ2nT> z|Lq#TX5O&2h!GYviFiX{IRHYEViDCLf^Wf)se&K4oOU>MQK$_!7!L(|E5Bx`dn|^Z z8D!P9pUu^~tYLFpB<~24WRqgt9Jadj5ce6JRV}}8O%6hRA!!0JH5LHs91WhgWWLJ- z!KL(|#^$p^amdJ5g8rZ$Ggy6?%`B;J_Kppf<0XMKcmmW9@>-TJn~gIShXI5aI(xEx zlSd-_6cOeEGR2J$MBqWpK*2%7D7_wEFG0(EP;?Sr1EpZsk|pld3%9nq47KjwNtga; z^X`AUY0HzBudMExSE>hYgVxdT>O;3bbp6&zv#t6lVjtU=7OitgFDbdK><hU?Cv@(N z^6$kbT>r_jozEYb*t7qdj?MRk%pu)4==CR^bNgHOU-j*emraW7T2WR%b?1^<<CWFZ zx2|OS{tRz1O4dGHHz0Kk@%_2lIo0G~*~-yX*bu&;a@;hFx~uWSnd&7zLyc!k+Y^79 zzW2%A%}?w<r+UFWi$^lbT1WlM4;+2q+TJ#*oVRKGq$U2fe#cY!F~*lIFMl-N!s7q7 z`0h(*P3YNs_0)}bg=C|(#^H0dFF*FieOEmA;j*^!UCYYGPua0(=6Gf8`a6aKk@-5z zZK#wXo3iRQ#^E@|lb2EF_!2P;JUiB#<|qPXn!w{A$3jCPhbkV`D#zXugX5Moue<Ta zb*m<A-gdOFzrXLBFR&HopML@VW6pdRQDJu=cF^D09J%TFn_4%lC+*8?yXm$IgsVqz zx3vC!!=RDR#f&Fi9e_24lVXQ_I+V@|bPXC|Vgd?HZxjVYF`F8i8a(pd-&KC}%2DN$ z=PthZIc7h~N?-itu1EL2df8>K?p<`lIUQwM$W=cui|bx}?bTOb6E1v3`QcM^BdcQe z=PpkFc*njs2H)6MH*NX+$l&D3bkD1=@_CF6^b#6m7%YZwDoKJobt%*>6l7EZ=V>@G zzzY{zEr!q?#B%Vk9VD%4E~MxbJ)hcn+q^0Z=@qNy9XNJiUX{8Ns(OzNq-fqrsbhbE ziWT!T7SLhKQavnveOJ`2^uK@O;eGSx?>nsSlq%#_#sdo9iphZ#Jwo|{FhMbfSrS>R zQiwFss8KQy?9j`|&<*8j64q^OVgV#e63^ksE_l^9($wb9f`EyHv4&?kqn<@TAOMm< ze1YGL4dcENbcWZd&n7h~Atmwe(#RoslRpeyDguG<QT0PoGXFh1Suo0sXOrr)Ly<ik z4M9lqGQ%+ePgD3!uJAxmOlt6-I<`}C3af{OvYrAn8j`p-iY+v}g+0*+pGTxII3MX? zd*J$*zurlyfj%JPNIe57m-OPW!{lntpQV8dPVE9M)yP(b#6x1(P&@JR4i~ea*?ybX z5~H-C<}|*8>F}j}$MRo9?SM8!=4Q2wU($EzceOopeaHDv$UhoQfY3;W=e^g5xM87H z;I{8*GeL)G;HH8ITBt8$<nx(boOIBMlcKz40_(kY!i-BA7;Cs>#)NOPnG>ql&Qh*h zWt>ty34rm;*F33uigBg#?eg{u7R{5>Q`U$R2j3@_Lkx_M{bOC#*zx1XR_*<LoGo2& zxIr06>c*B-IGq(GV|B@o{8hJ3p1*lD@AJn%&$i*n1|9(=hKoMs|KsjeFu0HwhG-gj z6NR02xQ2KllvU2l&Q+ddYuKj6LihSj-&!x-tUR@F>EtCIlkybUel`o1t{IyqKm3Y# z^I%x~1FN64cI~X$=bbnBPUd;Rxn=jXhSG-2Z`jT3lX2q?hsL#({W072*)<WKnphJl z9J>OlJJQjT){R0dcw$MIV@Im_3E)riYBiU=q`Y_6ca&e9uVeb_jW)Y(*6X`BKYM85 z!b8t)Ui*XT*XL>UuiVO9x8B8yUlNM}WBcAqm)&yESfoE>5R7X!<MvnZ)QqWz@-+8X zY`75MwNMbjhV@tZP4a0A=dfK#`hq?XKpj|-VF}G@eGom4WdVV~*<oF$#eJl!=ei;2 z2(gyKH;b@uxuk-&|7&zBfw3w_W7VfM2dtHF_k?l{Hfu3x3OEB6C_y$C8w_5AfRx6J zc83udI)c^8R*iNmG+k%jdVp!FNlUdg+7#UujWV)~RWp3c3bO#-XE(<1Zj;wwAT?X1 z1z$A<@qO4EC69p5;9F<jic*bo>w(jnYSbl8TpaivJ~v3;LD^f$vOykiS%0kDp1GRq zVCg_iC;5ATIf&(~gt<F%9&yZ$_M`YIM=tX_Bfr}@z09QUKHH!T*4ODaw5_(gyY4vT zw$})2`7fCl_)f^)cf;3QkyF=1TwfzJE9i3PesUAzH_Q=CYWn6A>_DK_8Vo2`%JbUh z9jfe_*S6Eje-d8cyItyiX=UK|B_;1L?UVG9n?6x~K;xR|0vZ5x!At8OJYq-&B}jT5 z#x}{P70vb-p^szS5EvI&o&q#3;_jrm%4X&6S8u*@S<qj1d~fxHX_J1<*1h^FTbPH@ zCENUGfOLJr?qVOXyQCBl1Gc{~yS``T!pYlWTKx?lGUtcC90u<4I7Y$k`Yww<{BU-C z;B|0=wyn>v#ZVm@V<@Hf3s4l;7vm>@w-r|)yZS%w?(I1*QeIrsG=I+5nepzsGxrc~ z!pSc|SCA)uB~*o*q}1leH+COyX<6)cl^Ly@AOH2^A6)<8mq0BH{PW9E7WVFW74(6f z)`<JCj)p-`VTZ*O2YpYtC>kEd2^SPxr15s^#3*QkxXWqEyk{wqj1GtNbEQ|(J1tK6 zUnIYs&2$CihuMv=&x^lu`v>+G339PrtlYp%HorK*>MU~Tjmr477+hGhviLYl@>d-K zU!uT<e8u}&uMua{lw-BIN}g;w9vty-4V?xW^E7E<dlU`V$c}o3XlnIW7`o_Q0nj11 zJ3kyTozA_9?kPBHoQq%8XX*WLS5LPL^CoHFT`kb8Q0tgoqA-!j!?U?w)iRRU#2u5= zW=G2xobttt>PY~kv}%w^h<Okkge4dNwyEQ<9EVM6{`&dhb}{U-n9SpsPwd=Nt@B7G zhtni^bk)1B+s#j%4t~0)bK>&xW}uU?TFq&;<VQC$OaC{+ETgHasluCt=Q1ZZA3wgC zc+2|v^Xr99Q_SKr*452!GgfFF4sC_8ZFc@~U7gWok%gz2<GQX}|9D6cURx^&Yl$f( zT|Vdt>?(Rl#6glkWN>Gw4B#URl`pWSWHsaPj-^{T?+Rl%;){@`StD{A2dwJ|V96v& z$16bph~Zles|b2KXKVo$Gy2J6qqP8xDY~bRh4}rn$()b-mt@e#Fwd)MdNQq8Y*-I^ zKqOSY68uyOQhX&e!epDI){mhNNM=IwXQLY2+&brLfPWf!2x1u(hS5ey?BxMlyyvL* z=no!g*pcWU2>q^rYg;4Lqki3-zG)X;d+6E=r*#^~7*m$_EGg_eQ=4jA+oZ8YMYWd6 zb?&a!UGBQcmfE7Cu~J)W?WPsCJoTfeZdoCs5nPtKdb}+(w{hma1+}#c_RZX|z*J-U z`YpG79lHe^?%Xkc?nU**&Cy^m+F0WA*VWfFHrCYF`F$mgbgj9#{-U|#cig$|;T=<^ z?0A^d|2~dA8{jc0T&>LodGPkA2Ce<%xn1wIlX?a%!@Eq4Md6Y$Pjh8C)#tL9&B{-Z zDl*AaMfM==qY6ZMs*j2-_o&#DtOvEgKO^o#a!G8V!FLJa99SgR=R+3-1WD>6kPt4T zQEnn&KOhDe*4&&kDJBfJWl@4anq%Se(e27Iv}pbO#r>3wvWJpUt}zNZYx9klkhS?P zCbrI418eh@4+uTT5z<4YR!}Wu!0bb{)|g-CHs~wgPLx_;gZ}Pe*r4aOmyr#+pp0lb zHFY6iYKHu9A$fn1?OWE+XV41w8uJSK1!e3*<MkMDzu61`>OLwh>v1U`ou!Z{BA27G z@n6d|J;N3qwe4uQiV3KTDcpf57p!m?0p3so1Ax@X#2IiaA}2>9&SUXL^1&>Xh8#Oo zQ?C?L-8M|oiJLpU6Q{%GGh;&0K{owhQSY%3!h1qcSn>U|R_L;f`cCNUO-efJ#sSbh zkg5Hb9y)Ys=YeAvt+X|EzTjRz37BGClh(UmXfNBmxvV{Ttan9870vRhk`;uSF?`m! zyWBXXtg*^vTY1s31F*aP^xb!Xf`+yrz9*G!3+V51{2PK^bPhMbp(nxq$mtS*2*~V% z(N&JbY2FYBI?V#24?IeNyZFFOpZ~&zB|@M?sbh`bnlV9zkG}tH<X<*A+9DLM6)ro; z^e4;0bR21O7`1`e+xOF@2{X_-gvpJ6$@aA#W!H&!-#u~A{pMi~sZm-1zR#&#z_oWr zoH(f{72p)4IsB&kAisEnXKg#0hSTB{_@NYfoTS+(<_%OTG6&V<v=*{cLsqOSVOb1y z)JOG*Qu;WmqtrL30ZAnnC_!^@RT38^L=g_)Sj6!Fc*Z_XTKfmns}9PVgJ{#_P>dLK zx+5aQXm)byO7#8<qr$}GR}2LEN`YT|%<GciZ;7`r=RM6Mo*0TlLOlj7cnS45yAWwp z+q;s<uFpTwLHjbG={=@PzI!5ha226lk`m^GIIeHIlFFpf>XHFtDn$5~LO*5aqH%?m z$2wT6nTmGDI)?$JimeWHNO7Kra|S#r4ugug1UgoGf)+&L0<x>3keV@p1OHE$p^lBA zt*GJGLDNniq=XZ4I+Mb*82pqbfoQ@+p_JGdB0aQaeTB!Lr#Z$97FjWL@MMe@Z^D+s z&IK)jih;Wbb%1MocDc@#$)|IKVWN*g2&aNVGFMmdoaL`cE`T^;1?Tcf@^i>q-czu= zA7p!sX62V=__ATa&S(g9I0rd{)J6Sdr^qB}JA4(U(1Y-`7)a4D)MA`g7I!Mwm6+KC z^C_nUK7sX}(ukntS*u>(uyyY=UeDi#4Mlus`)o8@(xaLmYhKp;LGw3oP&Rni)G|cQ z7Ur#P!U!VO1g(pNoJAP;`R9fA(}??`-wW?AJpaG_{Fi;Nu)eT^;QuU%IRlFc*+_>_ zx`&U5+e^|ih7FuRhmOU(<NWJqJO2OUe&H--7+Ge*Po=tXU@T};2QVp-uMyk+T*X}f zIdn8s!Mtmk;h(?w=zr@*Wwg%>m+aK71UlNUGH`jW!KA(Xf;sb)=69M;|L@O||H&xL zl74Wt!{fDxvzf&5M8E`Lo>IUfK@P&dqXA1j9Ysfw#32a=jPn2f=>Dps?=)zh0y=nF zlN*J67GXr@2Az6He%|WXWJyrTG^F6<|JoS+k`Xm{tCR{6!43_i__z|&s!LT*4`;a3 zwB^UO!_$ZGtWdT77?_S^7Dqv~y|xiDP)-YnK8%pxr7p+Lxp?<w`w+dUIzzN#oaA~F zOgUekRo;YeRqhTws@i6UmSJ#Qb9$8*V4Jn-e}*U+4ZrXLxOHwq@27QU=nxUnfppGZ zn<6q0aQ(uS4D_j19RBDWGSdpWpP7xBN4Sm-S%s|5k?)~b3Uy@uAr_DRu>4~wPvULd zUmZLLn47GQg>WUt!yAzB$G%F{zY<r-rKeG<2>S~B=am%aex&q3x^I|U4B;Xp?}AZk z^YIrlk>Jo6{xrIjl;V~Ot%d0#DhpmMHo+{Xi^Rz)*c5L{kRh`PE-|>;1QQ0h^lDfo zd@>|=U5Y91Dt-M)<#*Gl`Fr}3$-Z<?D3nU4Q=w2#n$COZmuW1T>}Nfx!+IeZ!v7G% ztcDQl>kp+vdVk8V$G)HSg>V(Daj1A4`JRB+&HA5cq3-~n7Y2oBATKb2YG`uA6X8S{ zY?6>Vt(nsVyAxRF6YnNNtUn~CLrIFaIITfuxMVt=e)j}2Or%oj&|p93A5+|pOZ*pd z#pmb`Sv&G65piAWD5e2SoNSIcgY-cWl#06J$28$_X(YT)8umd{pHg7Zo=kQW0->a_ z7yr))>upwE8ZMWr(itk!ke5-mNGO~-u?owjq}8&~H}EaBRQUYJk_kzaMJ-j~1H#0S z1rxw$&lCSsY5*5Eh9p`{{~@y^&(mjM(r6cji;VSvEmZ0dZ}u7v>WxNaH@lu48ujuc z{04p_HtH?AmEG!dXI$pv!-8`CYpz_XJ(2siAQuczyy!!@pi$wT{)yp>!Xhe@`nl`z z1^zAe8p<`=WnrFL1*!@PPZ=huBJ={PS>a{s$9bBsNe$AX5$!cHKZH|luaOs}hA*pi zw$Rj=>@_5!LqS+x4X9Y`l2I@7_L`@81m(I&E!VL96$Z9khIpPCg?Db=MU?BT)g7f3 z1oR}eOn#rEov2`=TqatC@g-cu`;n}|1~nUG-Vnn;qJfhg6hp5T(E`dSLj-kY;GX6Q zi-z9$l?TDudYiv<9<xXdBK0fE$!Eb=>p*t?+4_WO=CNA5llp|}o}F1=q4CAqvoxnl z-+26xjr)Osgn&kH{tC8-tSujYAX&ByDk<0rhH0A)eE8>_MbIX>Z9mf=3Xu{d5DSGe z{bXd;!bUBGMEs02AatuZk6h5A3ny8K=vdpjVylr_0=J@48tARLevxvQQ6xQRF2uMT zDdlo6=qryT!$n?JVgWh91v4nu1G=%?-N5?j)BLSd2l{{#%0EAV&&xf1Dr{4qxZQ5= zL(D1c=mH9)qTh-=!wPQK;G!Plb9%5!QL&)AKmk+G<LD6G0z+kv&!JV}%vKvNb~es} zeJk4n7HSPRLJZ^NZRA>}epRD9NQD(&9O0<P+eIYSg1*hY(a-jEfj2=b8_5xa4}OJX zB56)TjjEyE5U@>C6ZElh(DA_jLN=MkxobFd(kG<j&+7Zdd7)5fkl!HDHh=vwp*$K* z$ja=cEQCbV+&CGk0N^4leBy1&o0MagXPP}Dldja4m$dfTGXMghyUDKU6wie<eP~^6 z7f7fl?FKEu<WS3n{4cvf)NJokQeE50#E(S|QAWMTS}Oaywu{g(jrxO-q|4j8gfzJZ z@a6GXFPLBT_ED}=k<To*RX{m&$_NN8oOKu))lf5i@eQKpj1q6}VknrHI?E4II+H9c zF+H%o>nzu)+<NfxNo^YXCIg}g2^EdzUu!njfY>M~#d1*vxjpI7N&Q;y&0Q(nt9Ov@ z0UAx~93%#q(<@Bk9CzjhzLPRMRY32Y!M4>0SFb)OeWL#Q0u->@`-CeGuA;1us}BAQ zc@mIQK>2shoeQcVJ#!PiaLyd@Kj_ibnQy2+9_9fE%1-skgH%88v00xH6V6~l&y7;< z3z<LsMAf%3>*+Y;rwAP`&tJ>jA`DJcZ`7&@iupQ%b%(G56`bmS<#9BG;0CU_T(luy zt=;C3<w*P59bdn5Z9h3?X^?<PLy2vLf4Q%T^?ghEqpmz)aJX%Th_*Aap>Nlc<}xz{ z@bcSeLnyAw`PUGAL>*F~12pf(YnG!XZdkkO7$`Hc?ByN%$Z$rECfLDLP%2`Mw2Lkn z%iuczcuO)T(Vwa}C$&16nxS+qnzVRQ5p9I84;?;p=#nva%=pfX<JFfOO0GR}P4|P{ z*Bn`!?4K07_ST;qTQxuRz@h7U!|m4v8@AoQ_161tJGkZktqsBJ*ygUqot=wEZuaeY z&|@-r9^AA1syVe*Ywet?nEvH`^R~A|3|iNi_VrWtzVgwD?&TM6T(CG2>Yl&x;$;i_ zP|dt~<s;<EEY+oH^@Y@zTg$W%#WP7RwWeuHNn=xm36%N;{u{rILkbEX6lPT7Yk+PS z)&K`qfN2<1jYRpe7JJ;ZV8Zf+7;s*oyn?~0SjKvdQ1!gElb4HI%za+%r%eOt%9R1- zx6UTmP*)YPm~-F;kqXU=L>6wqbsm-{)G2ROAL$rK4<&wrWS4F}$7>VLjZ~K@NB#Cl zO&Qzj{Xrj9Q?1IwthH&{H`*sEN1LX>TEL$T9bDBnzAi-V%H>rqOSs{8i9DPnOQEm? zKnSNAa;HMY+M##OP3;`0pT=G%gsg(SQ~>24N?A+(Cl^G2rTi+Y_Xmo`>Wi*@@Y*8% zxO%^0<nvm9+>U>2&c=s7QU*VIcq8^q`sm^J3$P#9i<jPZ+ogB`UemR1U01*lud<i< z^yV*^lTqj~dOZ6Ykn7dTiI?)`?dWy$au`1>9SGJWj|-YQ|Bbro{q^IrwHjL#@aw6r zO5(p)w}zsz_FT2}`msf*s$lq^*3AS90U;2;%8zQ$AmjS~uU@58ERcbWhv?f>K#BeL zYN8qi*%SY*!e{wB?9^3;*7vWVA<6l3<rGR4b`9)+5s+enfC$%%Nhn5~)|Ax=T}R%b zkjxIb>`r<8_4JXqkECB$U^#wWOuf$1XFNlXZ{n58dU(CAELUC!&Oi-&kb(YyL&bkw zFG94K{HSTIT!grnt(x7Mt9azgH#FZz%{*?b|DaQ#z(AfKI!4Z}p<~>Ge#1Se1*{80 z*9-3X((C!(%0GrhVCY#e9J%8rDwB&WM#Ib#hh$(WdygIeQucm3{$<Z#PaXurDE8Ph zL%;LehHoEnn!BuM)VBd(O8s3!uiZ}X|2(CGM&@%M#UWW1wro4U<CDB?CjUe|R`M@p zPqI6v{$%Gbuq%C3i%;wne>#|=Kl+eJTk1Z-(L@12&%MZxw-kLv=48+WES(PWIT1Ks z0C<=YX2Yy?Fc%$1$a>sE6N@S(ydbyNTznjed+MR<yC<)IaP=eae|j=;-nvAhxMsqh zf%7BC+?ljGOClJQaynwv{r2|d(~FbSw@=)te7eTwwA({vk!4@If7bS&ZjUuwcG7Do zD=TAv6*jL7jZa*jd1!|%S`@T;#T}7}jsYu;cj@nriDcB?m6794S-Lo44VM-#pJ>p# zqQd(Tx2JkitUck{ZkFv%h>+T$y361us*p`!x@ITML#@u!?BZJ-!@DqEXFzk1cNoI{ zJl=+S{D?*ZKK1{XW)YK5yzt`pzw`QU#6SP_sM{sCSn6GMftpB-*B5YYd}6E1T{V8s zBM)6)8@_GeJO87$68vfVhG%-%V?Wnl^6Z65%hMOv_5&oUSnJohv?fUse?PIwpgrjj zbkDBTKUc**{+~4@<S3F;xc;DmVyDhfB6;1Iidh*|waf~$xR#_)CnMD&jReMo(<*|1 z31dhHQm8aC29#QU3vJXfTzmRf-6H8D=mjb$0}jtInq@_^IybKyphO7Y!eR$zgdnU( zEvB%AzIKM@MVO}CAo<Mg7K_AosRzQ>My<g0R3la8l-0m+or@Gc^(h{9NIr{fQV%kl zN3UE~w#1zLA@FHhIrSu7JIh-WZY4nVe8^&sG*`GBOE@uIK_*As<iV%F%5nr7mE8tp zi*k|6rL~k-Hx+C3@E<J_s*fh$xOe0|?9P)9v0qFlbQNOJu^~tBY06HD;+Qw__Wk=? z%nqgxu*2V+vA{A=aG!Fj_$QB0p9sLWeFXSL{fLaLL>+3;_M*cli^%=z;`psm^74d} zCj*Zab%E6QT+owC_c5m2HMR6aD{F5vvrm<yOJ%Rm;Zr6nld<#4OO@L~&RX`5cikn$ zo_CS5+bMLMYBaX7se_Zs9%SpLvL@vo#T<^BKm5=f4YU7ZYeNa$EcVMWW0VQAbi^U3 z*t;Y{c=>4M^bRUw2oc1;q9jPZaA_vxsFaP~U?%O27@cleW3dOF$d>Vq0Zl}ZBVHjH ztf_?4md<5`q8EHId=*llqXPIzIAX%~1B?b5_S~HV>kar}&i$g+Smv7ZlTat1QzXxJ z$_Fac3X5RMSd@80O63eVgMA|`7viFSV3ZmRpY_8pOoLm0i@%=q@I7J=7Vq5YX9ffA z{>R`WG+DU(#C;6O|HMaL<vBfyGwWTR<1xth2vQzc#1c#wRD_Fr3Gx~t@{bExA>g9l zl)V7Zh_060KjCS9biA=f=<m2LwWy{MvTS2bcGtW<F=cL5+`&)%{3+>azMILnJ&h}h zly@(WRadr83lyzrB*7h*#Kz%c#TEcwR<x>ZLH44Gb)Vv~oEAv$QE>6AfHr(F(C#@+ zLJlGHE;Y1|WL2(ysP_V;dWc_?Nl(dVTAaYOpjag5{{*~1y#T?AsgabJdOGqoA-oeB zE0oxN_!V3X&c0eE1?A93*;A)ACcg=udm8GzJ~h))e_kxCET|AT%Htl--<TuV9Rzfa z^071GVO@(exBpG~TNX}-IdM&SEQ^@K=6i&j@?MkqyZUN&`-l%R!)Xm42?)i+fN01I z_wDCNpZfL?Y7SfAi8z&ygG>e2VXnV<@TsN3YA17M0e6&-Kk=YQOE2LMDBtsJQIke# z@?QDP5g#LZ(1S@bh&gBDacz8F<R$`ib41O^&yN&Ze0o48Au$pHJkd2yxShzVEC^{o zxEVj+u(q=JjGFkE1kP-pE9bY?pF3$vb!4W+X|~*Nk@e8)b=`MmH}mHn@C!4WSGTqW zy}rdoj<Tq$cG0bo;KZ?2$&#XT?E3x2QVTP-Z@!(=FWS(2E~PwL<nKbmM4h<g4n;>` zRpD-jIg8-ap`Ym@6rNlM3=JFCvr)2b9N_9ODp{J#8`v;h=Es?IOxl<xDTFrEW^1u2 zMHxsJOkDJckGK%`L1l`IF<uffstv!sw3x-|G|q}ylq|DR7V4*W(?-%}G;}P|mQySZ zu*kP!&LF}Tk&YDIA0ZWw4UT#ZE-}&+=8y*s&65R$HHj`57P&hy;MCPVibGIFLm<D4 z1T+fqihrK)18EP~kj!YiQ&Cbsf985*mJvL-DC&!DWNiP_zZU+IQADY6{yFm-TgNoi zS-o2V(M4M>xNiKM<#Q9_2M;_jSY<GaI)mW$pL&2rnEk=`q!=(N%~kI`sC=rtDLn8} zu-KXG?La#BaBO_s6kHW8nX-Fb%j(vN)otM}^dQxZe8|KrQ8!qjze?TYv)4`h>UH}t zqe$Y&x^->4;JRt+*3Xu{ylQW~6s%=u)@<sQj)|Rdsu+M@AbVYo`6PmhYode*<Rvu> z9}!qmL7OlT#T4rTQru(OPi>~6!BlKwMiZNC$FYcG5yvTlmyw#v=M)cWYQ~gfFJVt> zq~`S7oR)6J2?icV&xW6Z&I8CNu=}8Y!-3V5*oU(pJV!{pyvacr8HA5P0nDoEQ%(JY zi_HlS4K2djpeQwr8f|LDf-$pdJEIqbnAcQ(`R2Mwiz8zq+ZHaqq%>Mu7wuYe%n&tL zfGjDLMa5%lx}tTse#w%qZMbXkq~r%<8NgEgk(yfXgz;U~-7DFX3+bnQ@#AqBY=^OF zLbS7X)|dq=R(4l+ji2DHt%>*r30Rp-(iA+JEy;u?keU%+qc(@`QA$BS9Orf!N}fVd zAL_Iua?ljh5MAJ^c}*yLOiMzDF9{(p(30MIi+m$<`Ua+XOL>c2D0t=$9GupiRQ`FA z{BOl%>K)}7|3O^Dzk_}@<DC>em{Rc@>6mR)GzU+fJP3!_lP56}Ebt+|2<0=uUVxPy z3)N6@44izF$8~7*yh5H)fjBg#!<YwNcNFgJ&FCX$+dv^2(iY|RlLr&cJ?CvhDiK^b zqdxmD0JgNT8)>VE4emB7mt}4}d2r)5g#{ZnU8q)|NhnorPaQnz>S+LontCn2s+La0 zh$jQ|3fkihRKrX7xJMtz8qh?orW`ed<POS%kXt~UVzouoVAelIN-D|StN)trSa<!! z2b%2WVza&Jz{S_E>rfqDgrtxfxOwvIr^UxInxzk2wXb_tKnHl(z^v|lS3R^;C5-qU z@k^Q^e256y0(|hy8uo+8d0&n6hRC-)<uiJi3S_C_T2Cx4r!f779Y00Q@G&L4IF%xc z6zpU$4l;FzD=A5nW+4)h6tI6zYu@qv0dAs_vF#y)UXQwkxe8-fM7yQrevJ-jLqq=M zC?}t{QSfkK&q&8wc(VXjAv>)pyDz3Z=lgVFfaOs{79<SSvt0a^8ThZFk~CRQYc9M+ zUHta13$<>aG081CD(x1Z!z{a6rfg{`f{nt;>Z~S~76JTgmet|iqonNy9qSRCrj5SG zE*k8okuHXMA1b|YZ0qc>KB6<%`;DPFQ>HnqYN&4EGLuv20mv@Zt>Scu^WHjG$A{{M zn0_!1B4y#@2tE)shK{KGiRKDSUb&Ams?2};;|q5pJXA^P3}#c(A}>+?UHMSdS`A5u zx!-7KdwaT0vc*icx+RrkWvS1Vqu=l9QLeTd`z1pXyttbcEn$YF%gs^<``o$khc~%U z9?(+A$FHjL21BG2Kpc=@FYF5APed6YZ)jh=UwQm-OL4H}p<gSH|62Js+J0Rc-0;2D zwklyDBa6wFP<U{L_zeBA84G94`Z2wh)1<8KI)rrqF!hkIa#-~wCwlAiO+Dh*HX;Q3 zMb_YQZ6t|qB0}VU;c67FDbPLRHiW-7;J!xONcZuZh2In?m<zv;PlGom$PvmqOQ`fC zzc0Lq&#AA&YgvQxA}@$Iz3}@~Dk_><%olMV739mlk7y|VeJq6h({N-N`F)AkKU*9A zZncuEumPCb0)>TTg$*!DALN=JPBdym6qG@%J)>S~Clne0KH`mlb{f%P!tPP}AjxA# z93;`Q1V$D?)kIu!LsQfhjw9EQ9F=<QgEx*9)_Z^coaIq3>y_B1`piC?(juo)nIC0- zDn9&Z<}dFxHQlKEWj$Lbgq~n;oLYO|eW)MPm|++FFVI|Qe8Ff4uCPwVdtGoTV=nn! z<sXA@XQt0!V@lY!*!!IWGbc*V#0F-9yuF}&%1mtit@qx`7O^ojU}?PAbs*lLLx{l4 z>9Mg!5}_H(v@l9y2_n5lmXZ?=E&S(lJU6Imo&ZWZIn@mAKqMS=Au89C=0ru@=+;YS z)498q9ZI9JWB0j$+}686F?+mvy={HRr$^I7WzrL;!!dIDMD^t8ryc8UdcBwRSe?@Q zeCZwRQ~JDm!Eo-)4?J-5xd4^sKe}D^^(*(gg=;zY{*Cfo)5#lh`mXYC@C%ts-TPOr zx4Ya<b^dd{A%gZJkwN7csDcQ{QZtcq!3W$l`JPB2vR!ctTxBTB83d9%-Pf>5jAH>O zc|Naas2cQjC5<M9{sC(wrwOWz&$*SHEkt9Fg@C&fg1sR7cQzB23SrB<+1Mv`o|>qX ztN*_<Y;zjbbS3;ng{(af(M|Ncu$XKIs^uVa;7hGmT%Q8gJteBL4<WSV$%eFq7JB#> zp0iX-C5(oALou489mBshd<ac}LWi(CgsaDL(eO*GXYH2uLp{vr@SV&-2TX_wJ$c zu;DVWH;0OocbL`LWcxFSsKaT)I-4jmq{X-c2t|aJQkL}QXiTVMz=F`J*S(Tc{UO0! zi%CAn@koN|GR(ehQJ(p;)$Op{@wSOMEh&o|_<kF|_h;qZJFEty)ADP!>Qx>8!DwP- z`FJ}oaQjgCpV<rxbK^lLGrfI>#o@Nx!OH&py^S(Mo<6#&dsVsr*A}PIAih}WFPR&w zCRp$^BQjucQVv0ZvdTb~5Y%*mLkorYIJsDrg^}#t?y#MKoS(VfIorvSE~hJ+Nkv_H z1NyT0bd&Z4`Byk{k++vY9$qbIp;T4E&6tF`tlp*!>j)C5KxYI&p)K>A@*LYD^nxH$ z?vczftYFCQBHl2#E4np$pk;es%l>Foya6Zs>Eu9EYEz!e5Y{R^h4l>CRPYp*(qm5H z=D~}jc&KkX?%Ns_4@L11PWDH)q8*0URaN#UIU9C%a`k~+cScW=kFDx3OHQ<-c(1A| zhLPT?d~EY|Lya>!Q^W8jeqE%Xq@>T#)`R;Q;n0=BC`ofPQDBM+{rFksZ55a(iGAa) zU*eU+_dJAYMzc*kC0`CJJP^FOO9?7Xpo<{uSO7rZNrA__;wfikngXyqdcC>NU}wp6 zrPBc|2Xff6WKjHOlr*OB8%+b_HySNtDX$lf;WU+r55_k%G}>I?y}14c>;mc66GV=~ zB>p6tL*)LIuB-?uX}lCp$PRoG3NBNh#Q-2Qmv!*o*&zk*WvQ}QR7jc9RyUZv;eI1q z1myA@D>js9##>)#Y7`z3u*P$CtoC0yo8w|Q6F271w2yF)%8KD0_2xTV;x+lRX_)S7 zLESy7mmECL$tj(~EAaM1nhN5QP)RT+`Em;B3)pSP8(VtVYgUKyj>BSg0P|KE5JF0S zre930DlR@=+*Q0v=*uq{`_A#ko)-3hEcA%gLXTvULWp5*D*ZywDm-z#xOi1heo6D& zsfhffDTW$dt<wT6I?K6nu#Yoo7L9Zz#6jgVe^DDk@E&#s&6aq<u<v30z?vUZav1mV zmcxAjlNYM;OETPFQHtj#))@nVNN2efONy$*hzrn0!poA*3Oi5yi1mOT^a1^5_fM}n zuO)1Jz+?qPwpi0!_u!3{W^+J@jcBg#<9<+r9x|)+#V4=nxokt*6Mrxxl?Pk9yK!tE zEGj}ezlf^Np3{E;jFS@@NwTLfn5^M{0`ZS~9*#&uMS&a?bfaj*P@aq784ifB`lr}} z%)gz6no)U&J1L&x4vWH?0v;1HGs{yD;uWMUKA_Y|gMs5JpcI9zBwgzCDQVYKmy-55 z9hE|=@;H1!ljLAYKA$6dPCPeKrX+(wB<hYtGNBPpq4^DSW|>I)HAE!7yiAVDOsdl1 z^kJ2l>S9UXuCtekeIpWyAb)r;s3gmj-+uKnaX)3%EDkWLFD+A&-j7eww|<vz3mw@+ z!o7J-i5e*B%!gz#%|0Q%l;2JabWbDxCwQ11P$bynZOG38kCFJ0B`TK7=h@1d0fA4t zUfe_GQkg(N2rHK?BVnyiJtW0*DPc@WAk99n_)0`HBeku?<qx6*Tz^sSo4khlcB&wD zP@&d#u=Dyiu0n<sv0C(6B+2G33YJ+9JYUaVw;PP2Uk)lr>&#xTfkW^^2cYa9_rm4Q zin3x4(yLf3=0BY<dnio{QV${T43pn+S$0@l6UI5MJFU*wn8t_iTTYO97Aj|CUP)~S z3c=1mw9{<1Z8zC=U5^wiGG2gWNVEn}dkmsICKKgRnemaDeXg_NhCtL?r$q`U9;*ck zNzw*Eu4gql;vky569C1Q^VU#J*L?QlXO2Nc*o53VM!Bk|cFAToo|-~=ncd^G-UlVY z&B`rw=P?ZX%UzbMtj<3?{QLd-BID&ICK%-6iq&&}<FsC7aVb|nfd^4krfCFD&TEjF z*5U+aw;`63ycp2~$m}tW`ix_$*NO2uJxZ4t(lr4|h`Nue0nbNrg#^T!R($Q3C!E%6 zEv~71yE=<xhi#M2ZZilwu8c+(?k$aWH8#g<=Z~E_rmo2G+}$qAwN~eZtsQMnt;4)f zZ?jrNU!r|k)tWtys_2}$u}v*KiD@u1toiyofk&w|9yQd|Hu&+aYmxuPB&-YQmo6x- zFCXLc*qxDJ&6o+Tv&S5IrSu0Dm(%)~wme*A*E-!tY{>T{IwK{%rJaGAcrfB}x_x6~ z?NgR#`|L{eSv%T*Hvmwtyp-4g+;<#Yu-bvpE@#a&$atCK%V}j(r9`g}0;71P)B2$A z^>07GDy&Am=Vx|<@=_YGAKMS!>s6Le->|zU{Oc`LG~#QV)<2JRJPc{DYNOS8_y_LC zl{@TCr<q;&YlG!-i)(4O(~2K<xGY!Gy*Qk-h-5x!4#m?bFSzwEsmm!eoyrI;arOuZ z6P)FM<=g9n`066vAkf+<xJ(^pDy9%0aBi)z@;ew5{WO>W62$lakMd)^-st?P%lI2t z)Hp`>W4-6c4x>S@{PH(^%>AB~t9w+1&30NhSzJq;*3A}|Fx76iJC$XzW&Y(3cE8JR zb!47(SvFgpOI(&s!0&j{;v!y#gh|u^kVZJ9B^rTLKq!cWhf6jz7>B3{VIyUy6St8` zt}7v#!kob_%sj7rhkZ`%r086h2XZFre!9|+So+}e;-=^KDM@y(a^Sx%DRgARg`+6@ zF2u-VGLQ-ZWzz#K(++!YiRJ=~3|GVj`!3)x5$zUkh)3uGfML}Os*EV|5hF(UJ{A{; zN;^ys#azEYS4VvUT}QTW$g@cuN;(_~!om}CfZ=y>M0q>J?!6&0ot>C}-$GouFs%Hh zTmXOk#{D|~3BT@JuRegi$szQ;LUnyKd=u@?UxB<`_Ui-kIc(E;I{yK`ZY?|iTsd&P z-Ds3oUP!mxQvQ9=j3s~$dYyr~$?Q9b+{-|eMivJd_6zn%Diy*g%^dgph0WMnjlyQm zYvbd%&X(IOX1{WrZT72MGXRGk%-(<@szG$F^a0wjK{JzM4tXi@39NXYNK<*-69LR< zHA_JJax@?fIF6fq^$B30HaB2{+{uk~5)kSg_1^k+EuCO#z)8DSy4iVj*ToiH!~Bac z@4lm}>JH~j*Yjl;)*~sL(K7eK*OTEpx-0KkaM|Wbua?%#Xj@*tK(C(|>l{C&ZhWb0 zMo~pu{jBOKI=QucYE5gb!YQVnoLhYCh8f$YkM&BY2iPFc51wjZM;I&Xyq~eb&xB70 zb!DyRW$vzMsVFjQ1?9U8snP5KICcCp+z|F5YaW9djR7^>S60XQbPOU4qinn+8ToxO zNmqH=nTD{Wfv@awt2Of=f=NR|5D_7WgKt``%4VxKRM|4nPih20e86-edqM8Km6$g( zF)F>V8F&FIKjPI0<Dq!qe6g+&rn~T5K8bHudaa%LMQM3{G$@LESPD9pDy&{Aa^|_= z(o%wu3=JMtHe*8&6wbUu(m#}=PW#bTM#wZ%HQlID1LTLMhz^SpQjlexCb;;d8aTLA z1w(2RL)0>*Fu<I{fC_sUm=6!#*o0&TO_-p%2HLwCT+##In`x$0cv9vQfGrs<X5j9S zCJurg%8!3yXXa_P13C1NG|lY%V2_YcZr0jNR)goCuPOa=#{TEnIm)}!-@osDY4dN# z+9fwDHxHENK%k(BlY6M9r}~^%|M?uj;WPk$TD<5_D>5JJohBIjc8gc^_8vam+bbN) z^b&a)S?@-wcXYVk<j_uf%B<L!S8s~C&99mGh3)Va>V5Z!+PTi!3PaWYx6x{?3=UUM zy8MhLFoOTujq!`V*3tMSxoiS#=D?7Pp0%n(Q89qC3)`8F5QUBrh37*5=v^&^@-+(> z0htu_oq#P)lq8+7G(S15;V0Pkj8^Mm@ObujJiy12bM!;%^Wpm2hU;Hg%d@u!H?ron zhpV7{3eP3fX1D@MX!O<)`U>hiqBVv!FrlFe?i{Tt*v_Hf&)NWd%*!uj=XwWu1V=%m zC=E2Y%d?O9C>(f5K@*3!6y2GKU?CtUfo5X3XhJ~Qjcg?3QbPGi<f+SB8{21f)=$se zbCybf9BAp<HLbnQ?~L2*(b(b*cEPi-ruUk|=kLA0RH=S|A?c*Ey*s`450iUm@0{Db zbhJ7r+IOGdJ>IU@?a)b<UCVsMH*eT-<XK_t&YkQronN+_EzXHcpULbTn(sq(PjGMO z;kth@ohzuG>x-J>E7bj!{QCXu3mQVoR({~yqt$+}u$pqisO>>~0Lk}B@ByTU1@@rY z>u~r$XBHw_V;CUK2l9wfE-|f+u$d`;80<3WWT;92N!SjR2{H~6qAwgjz)%Q~BE5t{ z5sXHIfmk23I8e_Z=spyPNqq^MSm$uq;)aRIt1IR@rrxz|-rh(cR#D{NJiasR3>XYL zQ?c6>sGBu5Y=Z}>%ZU`B67$U8nWmTEokDOZfCCqnPOb^fozyaELUjAIxk6bm033#B zK)9kPDhNB1%fimKXjQzX&F%7()mOHa`eSoz%C&yCm5&2z3k}+W{3v)^aQ~O=ST2;{ zqh1e}hLNfmPB0wKxK4n)$lD{=B-9?QB4!5iAyd1#&(;uI5^TqO<*$<7Dnfn947Tvt zS#<%IyV#^N7y{04=lIS3qKa4`vUlFHyQVtkR$QH&Xo%Y!jyh4ywM6DmD$Evdk4Gmh zpTE=U_G_b+^J4zew#xc4kIUUw6R(Q4Im646I|U(HBwPXSFjgH1mI-sGZI4bs!_5s5 z3VlxJW8l7`)tX5d8S9bL<VMXsn#VOi#VBcNB+qyZk^}(U%02L}LQ=a(6R_QyvPQgy ztoq)8fxr!WxP{n9(lXE$)z}Alfjhwh6a|4~^@+yjM#^bK0U`BB9YEtN9HP{6<TnIU z2T%ZMoPoX9#VK<%-qk>fPC=@;-9uH}`2fVh;~5}+A$u3Um=pMOMiBA#5(f+jB~MSC zn)!Lx?D_0_9r0+`pq+|DG;S}OtTT^^ggZJy6=Tf00YNken;J_z?vjl`&(-CAEmN*Y zCIyenIJNpZr0o0Xx|%6Qw;Ryo*9)=h0Xy!_Sk9T#&@^8c(nn0QS=duDz9H!G1RKVe zc%JC!;BeL*S`*&RKFe1V{`u~DM2I|G-q7&DbY%s5VEO^&mde^<N0gN-fA<TvGUmVV zNTa7}Oa=Ss-pVS^lZFzj*I5}0wqDU1jKxYO%$?E@U}ld=Y`LqU;jYGv_`O9nt+LG~ zx7I9s<gvxoZFJmLy%<rEl|TELZ<TLT;$QY$Ik5!KPUO2_8{4V;smw0;8Pdc?op4Cg z=WCVt>;UG{pRiU8kB^nWzuB+3UUR4BQ7)%rO`tFm8O&c}Ju*E2W7p9T9;I7yo!5lX z(M02^IocHA0|sI3XLKxj9>WcSSUt~xtJ8+~5J5C2jfxN-A*?|}r&Io+23KzE5u-v> z$p^6hGe@ZSLfq%|`r@qnoO1>zZdIP&vYv%jtSCiNV75YUt{d0P9x(tvw|d2j+HuYB z@9tg+vR3!~V7#L<UR)BNHWPx1km*~xRCqaHT|V*YOyKE>D=YyVw>~Aj&yNQK8!ugN z9UCp~oxz?gj&*j#ii=|%ov~uJU}aN%okhQriOygttN7OrFRS%-*41?$TfI8-OZKsH zO_fIsv2Dtw<NeV|tA9Q#(9--?X3MxlBDl!XV62$7Y}0~hLql}IBiJ<sg9h=ZuT7iw z+Vsq_zg*nfrS*8UU9H<cremGkt;4Zog<<?Zm~Hl<O^cP!r$4$7Psh3*?V^`-E0#oi z)Zf4kDYt|LZ#CObI(s#5?mu8Xi=%guznC;E30FN_P>H7}(~ORJa!MK2%;=)9#Q0e- z_BW5)m|^T*v&rE5TV+7}mC2O(gmsyWM(^LM{K_LvffdF7!z*rZDzod#Dcu7mwar$` z*4sUU=djGz-40u=a6w4CiClcL>lMlWR2F#kgGfL)<rV}F(!6-_;RNGX;JnSxn&S-O z2>E^!$C{h|!XpPfWluYi?|c7qNc3!frpzTKbdDdEx|9tNx80$qoyY*K46?85f0sW& z!7aa2ZZbRGWXiX!R!fDr&>YFc1tlDTfX&`!!oS+D8#!ILKE()Z+kfC_7D`;pT=h~J zBhY)eOM-}%pyjLp^|L}=3dbtO3hGJ%;x`FW2IZS?*ETc@zhv(z#m_v*Cd`@z?SI%G zDz$1|ag-7Xu5}ewtF<)b4}(GsDA&ELygY7vMMZRq|I9nAAvVB{pUSXJ24sg9wMM(o zrY%~PNZvB0^154YNvyzv?6VoQqUfS5)sk!s6`k=rvd$y_Iq}U&@DFME5PHT1kJKP} zEE^;b^Tc&c&>7%g!ecN)VE<V8w&Im*6ffj)->qyZlqJhD3)xb|seD(iW8I2Rd5A4z ze^$P$IK@fI%gP_wWaYhW%I|O^7V&L8tQdZqg7Tj9rt(MS6=qfbuKb7c6ILP~P=2EP zosEO=Vgg<I6m&_472K9B58*#Y;4T=6CS)C@B&9zje>afln`{`kuTQ?GZ?HQo+QOOT z9l{$Ong7}-Y~1)3dncttGLMU)9@dYzj8x6t-@Ho*98n&*MR;;==JZ~1Z|3qI;fhoD zo;ZPVIc$SdeJ>VhHsNXxx8JS}#q7<YT;{@x@)-Cc;*LZ(k*Z^$RD_QmaZiZK1^6nV z*rNaIklUB9ekz@_50g15>!uNUUwQid_t{L=-8{Fsd9E_Udc(|1mz31cb(?I^6JaRZ zOzye$B}*=ydBfR%5-yO9@4d2IX<q)ydq$ZbvtQA%;N=b-?0)qt@AHo|r@rEAM}ASc z;LP{)Z;hDO6rF-G#76qdH00Vc7qz0MI#OpEyJO#D`}RF1JjQo2*`S9qS`I!)KV|>r z(+>fwmj~Z*h2;hVYe<AgxXWq&MY%v|Uad4KO{+HxMm8|ye!$LS_p$TDBg#wt!uK=t z`WR9>of&)GC0`+b19}sRuI!+(055HHC{*^C?{$<c7ZV@7=|VqO$QTXeI1O&mWu<rs z@YG0p5}(Y#?!ZUTrRBH_NMQ)YKYpQ{8HozSM**aKtbSKhyrei*F)MMQ^%rX=&k?0t zSFOD4J@?F-@ya{ON7Z#nXX&a$+k0<MZd$ceZ?n`?EPeUso9bqCC*4IMt@95;qTi#n zKM`0W*HniGl{@}<#OATcI>8X}1Po$Hc}qp<{*!Dk8*^uyoeAHZJU8U%?shoMt&Xib zYl<(OwlbyH9~UkQMhyC~<8{XJKyk#ND=F6NBZJPshK^b8abrb?-d)}l>3Pm>xa~G= zd5ie;1B$=2vDk4S7Tj(w853+Y)IY!XJ2L~drKL7goin<uZmzqXlo46mqn=ayiO@-G zK}$BmP78jzIl;r(d8ob%Er|9ND05~rsH&?GIg7y@*r0;L|J1N>zKq9^I6`gfQW4iB zl2x2%Fos>-71gXdzIe8N`N3XMNYqZh`AK(2yynh_YGNH8OI>;CFJ22*)VG*q+r7%> z`^<8{Humn%zh7QzyVl^S-u|WnM2=W>gQWLXXqjH?v~2l46QA&xl}Y1RW&YR{?x?Qw zy0<uysb=a0=2aczm%Gcl*+fr3E4!g#D{{O1IvC<)T~S8V#fzlgUak3#mW9E(b*?GB z!Y`Y=iSXp0x!mR+SK4{wuPWq5ce$y{UA)rbj(OmJS3f)p9>NsUFij`?*r{2|!NL28 zsjd^jAOi;(BavJnJkV5@q6Njrx_pnV*!;-$`QZm=?(7`rmYGiaFE&qk+!E>-H~;02 zBJE6QS+!@+L?QH>z_N2MTvjXVl;wk&Q>BefNa&bv=T|ex#<8>^A^`R?a_9izLs%{U zRyz#ZBUff=dwWf5MPreXAx*?dJ(G)?HgsNDz3k3))2?Or<+tCQr@YKpImX9s`YD@k ztXaBwY0)>8)e|o6og%Pt(%Ag!lmACj$e`|sn$To(P86!}giq}j<vyduYE<sF=nd|E zHJT1Wn_WpEzB6TaxukwlCJlsmdx|iiXXG@!GwZiPj1h-TRs#x|SI7*#$78p!)%13@ z&LX*710i~o)E~1myj)4y@k;bjl8HKvnBni_S|n_sYy^u-CRYxgq0>+a3JN9kL(9`Y z{Ef9%UIYG44HLEL>^n)PM^>{TZ54Di;NP@qDndc2gsadLfSJs%0vZVKL>I%ad<WVy z#-kpy;v7j~k#Z5CA1b#^{5+TV@H=!0VA^qj|3L>q*nDoUyd%E&iq!a(OQ%d)xUk{) z(OY-yczEWP&E>UgH_q6-y0LLVWXd7s-ICJD&CSscan9_=7?KCFDf{<77Yc>TaU%cy zy(5Q9OUuirR3tkZR`1yN3+b{+bLLELcAB(Dw{0CG+Tm`l`qF8*ueg}y4qyR}!j*y$ z0Mxzk?aWg8)20S@k!zRW%<Ni?sLX%QoG&zGb`O5MUpOjWF!;W3H|kHc30$ar0B^Iw zZz%yhyOp?JPhMnNx0Kxjf112LRkjoG1Edb8Q)tAfZ*iTBPfZRAv?T9(I59Mrk*^b| z7zQcL4E0%P)7U#9pD*NPCqrIu$Tv7p)z-SStxZ~xtew-gwCzw^byXXiU6&NTyEQeq zF12lj&TP@mT=V*xnL3ME$BxnOZB^B6Qi+e=qyBrjtxD;xZfmP%->qtMWj59&|43(l zRJX}G;SP2*@$+4~exA6>qSKlWR#hD|Yju{)(cDwjt*ux`iSPOxO`=Czlrud<fhgRo zXR-glfQFVtOK3YjZA^zsu^7=E9XNF;N5nQ@cQ?j~;Q{RizXMuLKN9D|SGpL;IGxbK ziC?^MU~0kwh#ZR%WI&2jVtU+;l@!10IBts%VMTZaF;`e(LFMrkS`uIfS3ew&sVf4O zPYNEflEx-RqthyIl8DdH6}%7c!E@?AEf#1`GgcT-O1u-FQ%lMXxXrW%HMB$EpaOZk zzIvS&vM*H$=m@A&a~#Pkb>(#EbK_y0L1SShwjawriLP+%D;20XRBpcdlLLkoHhta{ z^Z{xF;tp98FCrCAgdqm6q(YM3jowOiLFwCZj(R6>PGxJRo2b$0UM!pZ&2S<>8&R`n zUrgV^M@nVkc9Q|AcjZ-*&4_qD<xWTeqNp<(5Xp{cEI4E=Ng%ndDCiJQZ=4<RhnIy- z@iH4T89Yvlg;~psyprTL#ci!xz1CY4EjF5*;3^zNUi(%jGmI$`S$T1=L|_Jo&Zw14 zItOEJzr)R#*PxFx3(`$`jY03&gisX>$p(`w8qDrlhMGW8GnNH=QI#WB9u9gff}qu! zbQZCAL9^FW<rP_JiN)z#fV|3Dy(c6gJ(?^C6?Jl@<bKd#7t2cZRo1$=WYIFwZLHgK zb&bzt7Vu52NAwATL-5)vSjYU#y`o8L5a5#_nnd;u!Qjx!23aeJ)+(pL^aG<=)B;2S zPUS+gY!Pkn3ls%r6eOtc^vrG*5^k@c^ZTNKm_D|q*s(F@^cl-bYnHm^x@%@sG!&QI zpL8WFs{OLDoWWC;8LdklrG8ISsG;0oaSCR5tdxslx%-NUfBh6+b+zbp8!sK#S!<F& z{G6paWbTZ)FR+>=p|LAIrKz`K!ZhG)m9I;zuz}q$8H2&*a%a$KunOLo)9!W|Th6I$ zoiwXyoGBg(hea#1+5+~Vw1K&p){Ik|XtHRPZl(uZm)?Z-H6oK4I$TihaQbaUL3@d@ zTvsiRyTI+9eBZ^Df>e81UA(Ofz7Xx*r4?S!lybd@%#`(wOq^QeLacmJF0J$!MEwC9 z1W4TksMIEu*=ouJ(PUsHE^jHTs*r3}vyWK=vfgKd1B`>24GzQqOWS*Z$5EYa!+WM| z@4c_KuX<grx>m)KB}*=Hmz!{J;EH=$7dkdzzy@rv=rM+bVv4~K1p*-uz`UjeUW!S8 z03o3UjIAAi_nDP!;gG<4{nzg@J9DO=Iprz$b3a-so`jY9I1>j66mTJ=@l)$fIt8a- zfa8&};F79ws#SG91uJvZ7d3mNzp6COmD?@8dbisIw|K)Gbrxs4M4>B)vAXKw0(-Mu zFK2j#tW2*P9+68698FNSO)Il33nn{_;Vc!KV{kIS-w>VoX*u#mvr4!&8GV8y#^Wl3 zoNyfBTrAIg#z^Iij%YMePQ$|jqGkzq@_DtxX0-zLY~)PsF1^gC@L183@s-?J4nk@) zXxVCm$~IA@FA9egYEEek1ls&&p4I4bq;|DcrEAt26jFy=nx$o>d1Vbz!&7DL0fk*} z_0V+QbIY5}SCuV&u6up1g?L;!`<rL1Ubdk6=)~gI7{dg@sCBtX?cu}Pafz0V4XwPt zIOxD@IaI3lR<*Cx>r&}3Di6xhT1ghHCIw(Tse_keCZxa!8>CMEC@gPmB+B{eEN#oA z1IAc_fg+2Kz<3QQEg&oBsg)HQoGB8eXNjW;IHZ6pDjz~C$4PQ#GK{|bx=oh`b&q|v zz1ET?{889VCXFt+_VV?SFlU^%X2a!uS)_n{=YRe%F?-2%{a;~HXGR@9(J^Y<iec+o zYjhv7yNJTvG-Bg02em?kI$N4+;<;WWvKQ_js&9qpnNmyV#GRe0-ORPcb<<?p4IfON zymxxDRzvtVUca;9>pfr8_`djf#7FG;gj{on>7Lh|!^&$cLg14JiQ18@Y;(tRcsrUG z3+;eso*#O7N`aS=bwnIyon$&@w6X#g2swm6!^;6&2#s}x&kI=yAv+`PiDpH|v|Rwd z7_Chj>zYZtg~AX`Lo5c=K`Me|#9587g<HAnqg9hSi|2dYIY!;pxy`Xqp0luW>AgM8 zsU=O3_6aq+x~*BG8%oC%=ahI#O20kOcJY!%vgm{TTjzJST_v1)a*2NQzy{&z26?Mw zY<ZqtWtYkHdZ(e&?hn|L`H9H|zWOC?wNq-s+K^Rijzvqe^7FH5@~3VKw_JGj{trg$ zmpn=MtJ*r}Z_R87r|1UifR*GycX$zVd=TilX4C?*4pImM#2d=20<#<DnM~QBSGYVO zRbr;$aF|RAIp9oak5$H&I|=(qn0H&!6*ixg1Z`gb7gjH8*Kj_I@HVwZlWt?+V_jz* zCzV=VC4S*64e(9Vt!?T=M;E(hM$&o<SJ%mneA*TWm`^=~8}vAJ9^KYj3(n2)gkAq^ z6E+HeFjy^lR<lwexEwNtwS(W9n6YqS|7QkLNw(p&hS(Dau<++{nhqe8&tZlg5X#`% zV&KUYL`j}NGY~ztXSU$%I+>z=Djv%|PD17Ve!3((nH1d+{kg36>_HLwOjNdpL5V*u z=6|HfKUmY*pv6QRmWYl&qh+8mnc_e+Q7Mrs2td3+mLH7y0U=4O)brQ;?-hu4YAon2 zXoRmw@qPYZJ*BY<5Wu$0BdK|9;HDCKwmrUW+v5bdkX$l;yD&#*1abG51&xgbAU1Ux zb!6{$;b3k>%ws31MT>-#o$a9~Y|A_=ctwsQ&Yq%!2ZUWXT|}Yx++VnbQD=kChukQm zE0T><5$KBlSO>8v$U24N;?uB6nt}y+0ebqEicfM>D5AgY)k3dW-V1sV^3vJoNQr&a zBJpEfLz9H)gYk>jT>&+=S#6;qV-(Ai>2UrO#wOI-Lp9YQd+mhm0yu=YN#_hOpOLq$ z?L9sx<ds;=9Ux9$PrG&JPmB?H@EyJe(g7W{@k%lY+b|#?pKTb!3XOA%Qhp+YtV2GF z8(_=<gp<md2{bu~lRSh#9|=RQv=|_p0=Ws+o4zS~U;pg&y;CwjWC^Di-qGl^!aIkQ zS;|AgJ6fGacn45f#i2tm=ITiZJ4xt}BA_?~kfTmRLIXP))!ED7YH=OkBfJc>nRNOI zjpoF3Dd1?Nq=(lT)F)18^w>*EGJDnP%wFMT?A2>doKTD3JjFkScnu?3s3c6sH9D+G z#SsvhI>TaCS~25#c}SF$Da8i`4r2pcKmRPRctm*N(ELB1MmX8lt1(|jrVAGx-$zr- zu6ULhZ_G0o{S&6_I(gly3$lG$*{67$@<;matPy_w=2j3Nu7BpmZ`Qp`-1}}Mwm)r@ zGTGU_k*}<{?&PjgqfZ+{pU&8%Gd}HH`ZdI%3S+VV-*Eir`nb8|5H<~F?$92LJtrl! zJ4>--?h<1JiKIVCi$pIhx$7(s2YNCi$vWLD?SXxuk)pxS>T{t0Bc@1f1{fD%mj=B; z;XosWnIF(9N?{074C0VzbMT{43=jkn=!aQWX%Cn@nvTK|UT%DjHzyls7Ntt(v{h?$ zkDA?f&?g&Ss5(v`==gmmFs|OmcH9TPRnvXPokB}G^#oBq!5}5`!PT!K7QtkCme*%z zAwPG2$`y@jw66f98#n)Tc`w2!NhEV(<}$+DjO3yxo<IjbjraoDMy^$I5jLPZK!c&C zOR8BgQ1y6e1al*7n4HZF$OrPY!t6s!pk|50IS^Us97&g6wglnjP&t4p6FUOhK6Dzu z1~AUyqFw?mhN>p;e=xQ%bQsx2+kN)znAayW6$Ci4qlA^o<T>C@uqVxC@94?~JFB#t zbTC$N#^8$9-OHxg9m?S1`8#T)ET_vMMzxja^>TBWPVXttjkz_9)TmJM3<5VCH5#Md z8h^YiZgy#93<P?qK`S=~a_Yi0g&ah3Mru})tZ0dpJ(2u4D_)vY>B@mf%WUiBbrG+F z<IQTdWx}vR7_98UVYxbolsB<lc0-fhu8i&MCDI*wD=#%c*5p`~rDT!YQ!}a9%@cX3 zYJOX+QKJqh7^_L;WQdwK`21C6lOw9SK%q~;aL(K*b@SIMb(C0z1(;o@fYm{A`X1@` z{0P{X>4;Z|sM-ba&`ZK+bYeOii|R4-PiVHNXH+FB6*2!InG{fP0yA<503J#ROk-<} z*re(pQVIiHP7%pk8i5N!42ldDFHjEc5*Nj#@f}fyYvLvaXu%m3ow*%!j)9RDtFd{^ zN;wiMdSnK#*86b&UzRKyQ&{-w!X-1HBlZfXcfBwCuU64Z$gcNcD~PmT{W~Eod@OwX z`qnE_2gv01hI~${)k&pSyit&!&+uBMx^ims%5e^pJlBQ?Gf%3w=Wx8!UPH!DER8Bk z%AIm|sIKnbiS8n`&%OTZ{y>XP>+}bPWx4ihTs+9vd|F;LeQr-EaCpYFsV>jMH9gn0 zXl?)4mHFA(eATx3bxo@uUA%&DsRI|cC$G_}(F&OA+WHk5ElBf>RSTFI)7Mwv?s$g! z9u4kp&*n9wdeSRgPGgCy>rnHsxKZk>D3m%u!f{r%SPlz`iRO!^Gz3wo@Q~UKASs|p znM26XjDgaCXie_?gU|l{;N{N*g3kzh(|>vxFm*2e@SoBTk<i_N>C-2kxccf7e68T> z7tWjYCb2(3hP{!_5k7fy7TMoVKJvaHpnJl8NM(n0kkb%NNVF^!RizS`MlkbYEY>ox zo`BJov6a(xp04vSIK>Ni=>41)8V-i1I?O*>+L5Jnm0y=NY5M$G(?`|l4a<c6M@>i} zb05i_8yY@+(##2C{mY-fWO=<K?vz<un`d-QNajmhHqV?;Tk5y+AbN1MR+MQYrETZe zgm|eL{#K|*+QRZlGqyA{(b|#Y$8}U247L)feZ=V1D|eCyMz5^RVzpj}Qu(3q193(i z<n<$3qg<2MzG-ZsIXJQ-Z(A>68P?#bXkXFdHkh)j>+6ek`gLtm^RV`%%XTz7+D3Oz z8rxE?({WRsGFyGT%E#D7Ztkk}8qs~&YcG}AstY1av4oRYfPwxyTz3>nZWiOKL<tL4 zL29CdNzOq3hXG^D2rk^;SB$9?`qN*GS;4(=DrfNjiZSfSF)N6xar|oG8=_e~zLB(? z22go87FRYkja$8Xh@43o$}nq+@j)2ve{6__RzR%(A-Ic64J0xhOtP6tWP0Gg1$r?% ze<Gjh!T%QUHKg6rT&Lo*((xG3e^Cz2&?|~)ct~=_|M;vlXDNNav#D5VCexwB_x~1^ z1`9SqFOhPU)YHtUnT)K$>Hqq)>>1s5FqT!cnZjT$io>v){#=BbB;qt1GGS*1GmWAB z&%t19AH`Ow2g1hGk^bj?K|B~zMNog{pv-Ih4;cdn{JA;*EpNa;bUhgw+xPG312QtX zbQ)xGi=-T*fK3#~AfXu(m<PaK1eKJ8ZQ^LeqNP(52jU!Is)nb*1Sb%U_A;@j=T-?v zpV%jSzxUO94C{7*v1ilk7c#D@(_lo58NLu=_AuMS$h8y78>i224wJiu1$y#_nBhY* z?N1NAx0fjPJxp@yww1qs5r~VnzUy3`LjI(8{dQJmaFo_hZya`>On5()3JPHE%*d3Y z{4VAjBJkF+(2p_2V93OblQHR1l^OFE#d9IPn|^6L{ve`*S1S+xZA@Ndyo$Rrm>bn( zdAC+Ca4mL~b*L&!bTzu>o}2&j&dH(vBX;YbrE=jLQ%~hP2g?8Wq*^x3-eYendnob0 ziHBgAc9G5fXZ*<UpkZu<BO%47ME%#ya2YjBR57T|3&i`-^$arH1CgFaj>ve+;EJJ~ zrU!<`Y~@l<3P*n1t2Mp}7=}V)`*iTvs6`=Jt#jIt(<SqBgm7Xh)Aw89$dkZC&uS#V zZlrAT)@f(fPExJ%;g!6Zmam;QxFR?ZrvDW7(Qyd#5PgSCUd@5nZ&YRGWyFDx4t63( z1@&H_m!FG$Jh$d)JUH#`NtytdJ^|?wH%QOVMu-2~@#2;KTeNt_my?pGFm6X7_tZ<k zi$=X~RCb0If{Xw#dy5quM{7a6A|x0Qib`Y&Ol~$B>Fbxm8m|M=kARQ|rmvt0%^yj> zxl-OAVHRI-ODd@`$*MX#s}Qb~Ox<z;H#ujEe)78f{B@J{TXLlF_?YJAQ{&n1-}<6* zsml#W>*V~NX`Y*J_Dt(3m;`Vur!6dL3z6sh6)Q<^GFj-iI~arAz&Pyw!emlrWp$-_ zp}bNZYnAnfmWI4V*A)qGL~@D{tON0#93{ueQ3{piG=7I=baJ47K*L2e0<hU=I~I^u z4w#xSQ}bt6X&a>PUk^v(nN_Hq_^KsVXqabL;TRA*y^fdwtP8U||3%%{Y4=vh##I+~ z>Jq{W3Hi91!VX>HMvtX-Od@aJf_+YFO;;lC=6GfYfL`VD@$}&MZ5C_I_?o<%7u;d* z?<eI8gcq0-V`|UySKn=MR!=C;zjNUob{k!}J+2sCi;KodWx0F|y?lJX-BvTcJim8A zFFhoYcWXjAsye?-S1$a%LN1(KK&mM<!jL(XV)UV9(^?oh2!O*H0|s8Y5}|>jGlQl| zhSFC)I0?YGN!x?8q>fL7>&Q?L2@6Vzz_an0jg2!4pDI-6C@W%YGFFku?(d6L)P@Tm zj<Rm8+hn(xEp`*uSfUEn{}^j>>Nq(RG+Q@?h7HSFn<bCPg$@`hm^s;16|wLrN7iyD zV}g~N0&MVOVt#EKMc)e_bs+YFM{91CjFFrtSq7E@%wQ2lWXNPi*H1*3v1w4<$#kYD z5DAutU`e!z=0-1kJUqe*siaBJMi`k1_DRrdV5){UBnt*ey|G-96)MMe1Q`pY*tTwN zB1?Y%PhQUJ<Swp5_^cquq&1md8nIbzkRp@E=rz9*J{c&JWhobsvCmygR11`ec8*g^ zjcCpFf)so(o3<=nz^<Qh^f!O6?!S#JCLLey-1((&NVrZoMAs0_)=g?Zy7bbIglB}i zglFg)le&F(zi9?p2|ISQ#@($!#A_AI?S2nY!ndS0nzTeNJSNE5<X~y<6AOFBB-D13 zOXpRG_(jLkYo&Z%7T5LbXI>Td&t>j9uqcNq`_YX%#E1Fe(MvxfwdXto>Yv)%Qey0j zk+MS&10M;|?h;B^q@2af*$l)Kh9@n~*|<94%MXPs-}ob$_SRd%rzHLvdtW&H&9$p< zC6+(Y6s0Ni9qC<Het(Da-pKK*Zur>Cj|PMBy5(bAJooxH476d1n0HDI&v_AL9<T%l z<}t7}ARjh7(r5nDm_sUiRueIiu_%VJeQaMU+kH~$&Q1{tkzkSeSMARJ0XwRtSUG_e zsYFOYTmb{fX$e^1hy2gbllEZ`_7A+jLV`I#BgzZlX{HS&Vd6o7BFd?OXc+wp%}Ivj zE_9}Vx^6-hQRCGB(Jhhw)5zT`ydm5nym2pQWa_MCR&KQQR4=FI(kpW-r1e!*j8dWN z(<zipRaJv(qVSN_%JpD3*TWnUem`v0F#IDm56fhXzCbQOH+(s;cTT8nSVS15RpF18 zJP;Yy7I|g;e0hPK^Z-ZLi}ybtI0R&erI;3Cioii(0<$UgXF$y|!j+MrtHoN3h3phf zU2b+#6BAC~Kdq`Huc>~=?{dP|bgwBak5^Q=lfjY7T})HDR;6N|8AhHZu`6`CCI7&a z)qZ;IOB1!)=&Y)X4JU9L+Ftk%#5q(#{Ir)LzB<#hLZw+Y8J<EUXMmi=Cj6{~d0qX8 z@SC@9hJ9=8TG*{_CV6kZy8h8E|C>tv@0N+XrnmT|LI?BDrrNiJgMIV>QbpV^ul?g6 zS8sh^IPw10qTy4!!kD(tj1x5OH6R%&dL!^bvZ(b0`Z~3*m53liw3!k(9jMw@VogwD zn@H3IxCMnJpo$<*fgcZRqPqtR4puvWt?OVfJUdEYbg*)*dVQVn&pJKgw53IB*Az>Q z!m+aUc)XqbHr`%_wNov#Lt7uNf1VbG%bo9<q{9wv36_<#gkjgWKl>c9%e)~n_b2)z zS*F+3)#>z7X>qaiHCzmBsXI)sS=LqD66%%`SAMuG-X1S0<}JeWvhHw8aj;6~^6Y%! zg`HUrUF8#JMwUzm#~4G$Q(8|MTd)rG6coo((N;y9Ev+Y7O<~bMO{+(&Ct6{&qEI=J zXabW2{5n5fRj6f34-Jpl(5VMf5_?diiGLo~Xm~xJ^KuTa7leYkg8XDY>B{`R2?&O7 z*-hmKNxqNzU5YGE8n~L9mU#1WYqFgDmj~|oQtI%L(xD<M*@KJXno?;_j&!%Lpg=rx z;I}oW#y67tCYqIpyX5BZM&Vm&XI4#)w?yNxt-J5Obq>3xn0z=?h&`(>c`^FbpfQ6l zKqMbK14|KK5aJ(X0}tWj13;BpA_Lbv8qkkmk~6zk_O5hCTzgh@jalI`n_T3w-Snrs zX60=w$e43%>C9nQ-KeEYMhPF8T`u#QbzRGsjV72(-KO&Q*KIPp+@|$T_xjNYUb^pG z13Mj~Z<nR`Y0%O!YQkDI@X%D&!H_wayk2~086dw@S;Zb(BD0H&)Qqd?f8fDWWH?sR zzq!Bv8FCdFe(J#o`nNrU61HL$g~z1#{pQ4}2Z>TR31CYuv-sfG-`;y^)vdyJ51#tr zexk0e628u<uFe@4Gev!c@}eLo6C&iPGYmgJ5jWMBP6}DmwlRI=__S@wq2ITc9B2Br zO%t-x$!(lx=$8_C%_UB`g!>pRT7j{d<|gw%BhSYB(<#F5K+H9`;|;8(G;YFn9Dfnt zV8AqTc76Dt(w~#z>&cBTz4THSV@dy=3>O}w1vfEf>}eIiD!HEfxIddYjD5?5t8h#! zbC`Jl1UAb4uG_or$P}Jg9n!z3T`P$1kwmYf6)whn3|Z6D{v^d;Ln4l5#faO%%*MIh zhqHFXb6xJ7xbUxm6=u`@8_gzLV&aBlrHvc!eqdvJ)8oeywHsO6&>Cc#Q{9LyHjpu? zDfBm8Ow>=YBdcae)7!IOHZcpZ8R~xwtK`Iw>sKksKCO_wgt=p@<cQ=2$;*;=B!89s zP4cPaACfOn$K_rYs)J3-$Uuiek)&Q3OxM_q&{w6?Umb&yl&Yu@HsL+b$lY{G#JDeN z)Kp=`{2<jaQn6c!tjLKd9YC-L0RRat2sH|1a8c#pGBCF#yi7f@z&uwcuZR5&(=KjE zY4}-sK|B$?H%ZuJRJy~g2E;4KbTJ%3B`!s2+@d4(4ENlK%Vy1M$n{UD8CJA*UnsA} z6)l+62B?ifnkVz|Miyc5L?$<|!R=X5KMbAK3G~F!o?6G8rEaN3@cQ(2gO+?OS6NIh zp0i1HPTB1WgTtVEgpfO}g;y6Q3YE>dd{<pwBFAh|+SMg&EI&wUd4o(VRmv2cOy@Kf zYBx>M$C~Rst#Wl%mQ`*2euFzN+Y!(PRk?B*lRc{ckhUVvz~+7*JzTDEd29}5?fTlJ z@I%r0ZRA!qSXo*DLV{5ZZeduDRGF_f9rG!(*|h`+B*M&K3tLv7H@sqDqSl+J*N6Ar zcjWr>82G~Yu*{?OI>J`Jvp%~6Z9=K{wOcinwHC%1pSI~nGv{1t)$45RLakM!1VV^t zvJ7FXL1$%Sdgr6P#i0Oew(E_iyf$Z+o<)#{FX?u~VvI`n25*t;q!8d4Fr4Rl{muf{ zScM|rO-KisF~bsy+VTyRrVgDVKH<*ia#@8^VJerY`o}qQedPree7=eesUIj3j>1Ku zQ^6LR%V=cGN;A+e=?!Dm(qiE1>6J4&t`XzQKY;@+mrO%eB?*8S8EXjIi3lG@8-ag> zT1PUyOoY^do`PyPu*(Cd0QMT30+cUpM-<FP!3ZKMPDFf}2j+TEdh-df$tDK$Bkn5R z<XzsrB?&x`PDbQ`MKepXC+5o3c_y<hUTF6eJGBafjn#oCSFbgxTtSsw$t!K7TWQPp z`cqp&uC`H=mL^u*!7v@3rk2XxtJY;XM>e#YgN0dcPkh5s;qSsx;p5j+(dw=dU4<rX zd$86U%9C%Y%)2+&5*pt$HW4i`X{~``Xc%T?mYZccze}dxUZ~Y6)p>TaTxMo8oD!HI zMyJ&oq@0=*TJ!VWW5<zLRaK0Ty>ph9nGFq{NkVGd>IfSs$X@gE9m3y!yLiPPh`V?4 z-5ZvTNP3j=usLRTPad;3;u-1E*oO^Ywdo*6GqAV}$Pix4lHHOu7!P!Ca7F1Spvpla z0tMS91Kq8)q@HDMkg0(C^szET?+_Rva0t4-t(@ix!WmI&PEX)iFtD)+AN8mJybq8! zWo3#2)(BQMHd@cr5t}%0a0R`4ybbq_*Dq}wzh?3<Ye9p0k18Av&kuX|-1?OI26cFT zd)RYB@19%tc$@QFE!}IzcdSs4JIKykGroPfcES_Qsy*T{-Wz)H*OM%?bj)Gqb~p=~ z)Y#FE>!A478$3;qO;D{EIera!rS}GJvcS^Py>|TYrTPiKZcyK#3eS&(>4A)q-m!fF zy(9j5n+{LZ;lb9<Y&RF4x_<oH>8<X(X7P~Bb?K$<8?Rcy**aIM#y{FL@VuF4@iyf< zN6cND$=d<t7vt|R<a3;Y{9*GX3lT$jJL;NNmk9l&vj?hOY_D*gpbs#pbULZfhL3R% zItayHqAyUt5N+o&G<5ufX$40G=}`%|v~XMrj~KtOfhRULiecAAcmqCR6h~3x*MMC_ zJQp$^?OcEcQ`>2@3=<o@Pr|^L<e?K;49rfTm(6*(vSzNSX^frovC<C97OkO~k<XP! z;W8t<(`Gj+IU)_KstRW_%0`vK!Ep@haI=oGq<SN0vtD6k>WJ6tv}rlQ`prcllYx1v z{)$s4m`Bp>+*@-Wp8e;!`NxC;rdBw4OL=VTt}6eyQD4=|m2%GQ=i2UTopJSeoiD5; z*Y}^)rVC^mklrKS2kLJD14XwQR2VO?hz~P+_&76f<P06S@3~3#M_k^^6ePf?<P>+O z1UD9EkQx{%tJepaAP{f>-C3BDO1@-_TUy4DVsc!kvFX&TP3J^6<lq?F_1B#Ajh}Gx zPt)Z-<{Dr66W;|5Z1rPg!%2gtw{fgPJ+eTiz|di2!o{AD%m76?b?T4bNxuP^KUWUE zO+KV0spl?OWD%x|@y2jyq8;eOn7&rYM6`s}l1n6eBzH?5mGq%KWGp=y8zp)cG6wy~ z@&oak=614Bxey15e3=xIM2S@X+xGw8$D1<r#qc9#QTEFeUGT?2JiM^Kw{beCWqM;h zph=9Svqa9L|F|HM|F}!s@pJoofS5_AI8b$-&8~cVVAZg+WZ`b4s+PnmVt9<6*r`;C z+HdKCh|6<Vg`X@Pa_r9@Kz0(ql6nP5j8vcHQ{s20Q6lw{Q5XZ3z>9sAWIy7Fe=B)K z@;)T7(+G|90VGg=rX8Fy`$I0GF`k2|g{5HO{XcE9Khr*buKk?5pSCAFoY?+EyW{`I z>;GTd=ef^w?lzyK2BA|Dx+HxW`k%AxKmTbh^-B*tdmMuXJ0va8f4cJ76T~&zjFYqh z{vQ@nIPiWD?OakUh2v*V6~6wt)d$ZUFogH$XID>ATA~b}40HBDfA+Ng|HH9EE(TeI z0iH?E_3=IMBO?A<K{Y3;+YW;3sIPMnECh7XV3a9$hw#n-woZ7*?e>gve@K>o2wGOR z(3=6+y(7HS|GWsTO9?3vT310r^Z@sVAJP*(%3$j<_LLOtT{`HWrHE%7gPw?~mg+r_ z9jRUd_&&s(0kH>Z)Jix2Tg7}aFfs)LG-*tD$kEtG!c;RF5T_uYsUwqWJ2uo{*}1+( zxMy5v$F>%6K`viKjE@EC8*`h#sBcWSKf3hpqhxsPq)5&BPP*JcW_ONj+15c9T&!l% z$QAqA=yGrR*yvSD_O*{*z2xS?XM|5z6x4cD-II4sIQHvR$3`xyY2Uj7%eH+h=C2;z zzHiB@(d{=cfo(5|n65sINi;ST@)?Ywbk<3jGOvm^W%`!S$Y(-G))Zp$XDlDT`<~t7 z*)OkoHr)Rr?N)3&{OmQUZ*IQ%8+<y^XCXNgcLYB$5A$*~pJ?buL?jk;Ib4|1r-%&4 zkXlkqDxFF)kEdJGE&R!lFP(HZy3<Kt2$+RFYaRfnk%DPt8nfZkWr83uX-TlgfXFQb zkr`@&tR&W`B@($D<U)<~{uC!9k<&4DFz$&)X=4`;IS9|eQ+bH~MhCa)bMbgC02BvS z1|ASFMVNbGKNBZkEWkF4c9SI635$ir{A)ut@-pj(g;t@J{~_WRl8&T9NOCO053GWu zF-RW7|2`X{&-#M7<iSu*PamneW9fbO-Iv}mxVGj_qP%12lTSXGt`&Mhb^7C4?Qw8+ zG4xko7b3lpr2cVoHI{u!WuHD75tcr#PiA~7f?UEQ{|Rg-KG@m|Bn|kY@NTn|*)+mJ z!YgI`0l3BC52p&Y%x|`a{Uu;Ff$@ae_=0|HiYbohZtxuWWfcN~68pQCO34kzyXsdN z-kmKRej`ozmoI;54wJrM#i?=<d+86t{}9K>DNhOg!rz&$iI-kjfA8{@#bcMJTGBUj z_iYgVXF>Nf=|__Z(9+4@JW5QLzIU0yyJT(2-G`oP>%96+chjaR4|iqVwRXh%aaGQN zZ-_4__CGJ|KY4hQRx!`dIsPwd0}_psc=!Sa*}EXAng@P(j2M2DLs!h8(kW9DTVg{b zCyPoM>Ipk0>>!&i?7eDHw0&IX{kN|^@9>iw7-jQtvX@-HC3VLw7r#_@xvH&rnM&YV z79vRhcR%)m3D@-hW5u#ta>|xgj><6zPe0Z@U3lQFW%IK-hAGY4AGmkxC3pNb5F;0? zt7s(3PQ0I}Yl)nWGWcJjkOR)3B`9(;K;?O=1Hi~aHCV*|4!%Qq!Ym2W2(tjx1p^O_ z%O(=pN~8r>y>Qi4FQj+un(uPW?`-h-Zs@RdnX^{4&S#H4v}yB04{hG`&~D*hM}!gT zr?;<t-0?BFXw-C#m2wQIo;X5OoR6M=)v2eB?vm#C_H=Y6eR99Nrj31n(?dA_!3Q@# zDSRjVZp(u=tei^jySa!Vd!B%!>R)*DA-ba+@6&|HK#D*WtGz@tjzwsk8`KFrG#+`- z5LQc-7OHrJ={KbBC}Zi{(|$)$)6f=07#CmzZ!hm%wyamsuk5Or?kFp$S>v#m)^=IV zU2K2GG<L7~+8Oh2Qj*ELj~&~6-TyEjC`=>jgf|bYX8Tqj_<rZ}J9f;TvxCj<UABDG z=w-t5cde)@GMjOYnl*Mm&z#>c!X9oMHg(OF^ZJinzx&v$*9lLN@M`iJsNIF$**kVT zzjKEKY~!aVNWTE)Sp%zVKJ?@fltBt^XFv?`wV*&*UC@|W(7P7Utcr;!uwM}7prNrQ zS_7aG2}e!PdA&T%4k|+cTm&TvHk_cqHNG5Dy_Id&F~U^zeU(h72rwh_4qaP+UXhRG zo<ag!d2rN~($o=kvB}tao&4itvU~ngnT{5|Ej(lTz&tu;FAf&ji<mvH8C6zIEL7`n zQ3d`+DmUzWN0_!J&^4^aU?A7~%GIhUSt49*^{|y;S4Dehs!YvP2^UQre$9e$NO?HN zSz|EPZ=8_+Ot{e#L_`yR2U$WE-lw-(*$3-|E1yu4**-VNn5}t^a^YFw`tZo0IbgLa zjqGr;_?dr<6|OK(n0(W;CWOYZE}gD6(?*gQpHdj1sa>~eppC$ejr2eTG{K)#HpqEE z@fK$SNBuA-QrH+ZL!f0;6VxAV9ySVLAjgqrY5Ml9?1{;YU6Gb3><rI}MS@Y=gH)XT z5-fVO#%<Hw3v9Q|WF<3ivn_B>+eS9g^QHrKFh_1O$<vO^CR~VsaL_cIsmv1Cjuei* zefs9j(}ii<#I}wABg`+Q$XPp#w)Hc*+Q5VU)B|UDmhseLMl73YFm9HdeRt}Afkj0I zyZKK(I)qZn1el^P@WaW><IW!P-ke`7V^XlL<0X>xC6bxt*_Sv@CAs7DRfH_Dn#k5n z1@u25ZbBZ&f{t=rd_M^!E6RV3_YxHlOox8-$OQcqXO@^B0ind_8d&nj0plnk%8*0o zbA*&cC~-ziWY<UH@>#k}QCj$vDdK#V?85RRvI_`p!;Xj}7<5E-7=Yp?*PdCVz&Vc- zBEtFNV#ruyk>moGM6oafY*=FK5rueA$6$E^r8Ev_ury07HK8;l+7k!M0VKfTb!14a z1UJw7JK>_6a$HtEYx|PF90WGN-4pzW@W&f>7X=+M@479-_Nra$2riCo5+1z&PrWu@ zwom1`=-2y6{ydAxll#&+ejw74Wm*wX<eEKl*F2T$^O$8vj;7QRl>0Ymg2Yg0Ya3B0 z3wwPz@^EvlI(y1F&LBc<b0UucSh5ua9IJjAD}#<5#ahCY@N*L3RXec>eBMs4aEuh% z;i*4`b&}7$ntt3ToaYt3@RCBN)l2q!iNTA$XTbj}6%uZxM2i`gX0)#XW`7)F<Yobj zv01W1ay7Wm?v?aOUW2zML3=0$bod7_Vq+84Nb6R{n!#<Bam2={bua*<UwH|v^-{>d z(F7vK2uy{5NYnCC0Q}GH$gCqE92{t+NJ(NsY%e{|ge`00+^x(m(Z+~SCYJ7|b0Byx z=twZQh1fi+NmeZGV@z>OIkYt(hcp_nDAmydiH+U?#veV=C>5X)A{vF2<fzmdgiazc zDnenkNMC#Eb2HMUF$M#po#b>fa)r&NkQ3(-heM@gEEYzonr^c(YK_IBQTJe5D^-}y z3aOTC5#G00lrlYIG%|Xba=OW+l4A|qa@9<fnN8RL-GZ82_MA?qgG!*)>dd-XTCLuy zCu%j(TXnB%jZPzxO4Wc6z-|u6`rNxN?Ek06=pNtm4DlM`l^5Q1$5)I>snsge|N2U) zDLclr>*WY%)l1V)lD`wBOr?-%$l}x{g<?}m(elcR3grv0D9|(i{;z+iRR8#wn(zNs zqxrU9<NNG8WTgA<GpRzbGUp51zmutq-;oVQ)hHqFTe-^gEqTqP$`YhsTP$Cblc0M~ z8<@`o<_o1xtNMZnD$p<q=Fe4nz3OxFxmvFm<bTr`jhf{bG8+&oEaT<I^mP~B$rv^4 zHbeyrKlEAeLffM<5JpjqGaASYlz~+OkWK<a4TWOVBBtxVZgCW1IB;lLIb$MklYOgS zMndc!;g*+IlYKw4eVFu;nJ*6>|1v9?Fz%iV9^;;I{r3#nAUQ)exEvgl${dFuG0rse z4kn2ce!=PJJ1fz5F2R_DQ4^DxIBX7xGd7vQPxC1g3bv*$TsYXo=848Dv!H!b{R0k+ zOmGOb^8(^VZLl=vpqfEDhItpSjRhnNEuuhe804@&635@D88L=96vkhecM-U11vsLN zKjMa^>m&eO0C%NedfQIcDAmFr)MOToHA_pt<5gN+b*&dc+(g<?baDN=2XCCL)7S~G z<P_7#YZXjcqB&&OXjDE6v8jy;8Vs*a2(Gg6rEO#j0<NIcQHBaN*>K7AjFs;wbyawo z)%KMgMOu#AE}Gcr-6?5w%-t+p>QR$Q^+_W_;bNrsq=Xsc^va5@P_94{AM@L*g_ANh z;grtUynKa@Va6}LbW_*fl9<j?h<J3~yMEXmg3Eq!Y!bUJr&|_gV~AJAxmMIidO^7h zLUGgi6P8q{>~K+`NeyXdnQt`imwg+Pg;F)6_T!}(@*rxML`pvv&Wj+TU*o7~HYmz= zLDV=~8vogvUeI#K{*;Ub@iXDs)c!kKgx9)f@eBig0U~9tUVb&hBlenM_*vb*pxW5f zqVyv2k=d!2+t~o3J(=qfrr2(FT4)|&K1;#))9)*MAj5N-$s<4$p6zd$dKml5>Vbv= z1mPK|rrux#`v&PYo2d+_D5wp%5eh+E2);uT`?Hk*Dmcf8dAyRxOLIt4!7l0`!REea znuJf==W%L;pAb%}TG%1H*Zkzuzn~gETe$F6nMuw`IXGZ%UAT}Kh;z}R{W25B;yUX6 zsFN>+k7zp(u|(o{lX?FNDuMozUMkiA6ifKGp`^g|NSPghL!c82rS<&zcg`ZM(=O}C zX&TjDU(_XBJ(cjQ*Od7x>U_WK1@G3`Qe9)#xJ--EuM;~Eg8r__KHX2fQx4+Xf6+T( z2#UiS#8LGM;dVd!3S6pR(npOSqkES^oc;yRO^`yWkDijk@k@IlwwxL72kkOJFoh+M zhr0{U4A2dLH=coC%g=w8ASGD`Op#&@Fq&c*G=Zic(>gOCMl-1taDwzdTk~JXz!Z`P zF*_E?uX*npxn)*rlr?Zf%=N}0{lJ+&<mU#mR0>1ctHSLr$Jq1FAM0?{lTKg_1t$Uv zBW3hkVWJzD?=tPL64_~||H7|DLBCXPLZ(Zq2vHpf-fn=p^iVp{3vE`t$hs0m5v7o& zB{%^(_s@P=0wIUyj=T%$S&)q7E<HKqGi}+TFE?9w!|d7MvNb;GGEbanERTR=tjxri z%)j94vjuW%`Z!G`dADdczFU`WweKf4?6*h_hPY+D@br31T(7&%JgtA4i3!EcmDg38 z<3=izp4(5U_4Svv>2qvD{9vt#Y?xrD`Pr#Z%t9=POLj4>7Og_~o+yw^^Ow9b@)&2% zCAb1oXQun;`x9k1QKIet+xJhvb};1^zF8fO9mQB{qrP*5BO-jo4@v<sZxPH42v3Bq z0a7jm1?a}~Sh_R%vu{j#<W5zoij_a|h@8cWQ~nn%URJC9%em#kDvgf0oUv-^2Y!E1 zjo``=T<M!{aODW@K=KiSYZ8u~Tb`EU$}GJ!=odBpzrzaj(CFzs{9EWYG@=LMk(7a4 z1AZYTv>vOI%1#Lya7{&d48vLyz?3}H+{eE)=e&kL-c~re%iXYG_KKc~F5+@dTDxx4 zfmJ(iJ9_BBr>bO*rs@Wxuc{=T{GZ$Em}j4}T`GKit24jI5MO@P2jI=T;FY(9J;E2y z^&I%ea1uM*_pf7p`!^F#9nG3IW@7iODUZK7;L{g<xh%f1655mzmUIW@1~9>!&L@zi zI6P=@hVEwI!;n$XpEH^GVA04J!mWR1rU(xT5C86WY$?{h5gzO$dQ4tlUO`5t@8n+k zo$xTxr0--)1N|>q@+|!?1p;g-R!{&-&IM%N`=Kpc`rjeD4!wWzBab{X?R_#2^pjs~ zAx!8H*(KbVn|?3bmVQs8VFI>n2KkAY03`YMC^;O(gVPt`*Fc7ym}!$#6~k1Q%Rttl z*blLyZ6fX-ehw+k&R9aFO?sHP&&!K2(FnC(X1)n_WwL6?mt6Mw-JFg+)rwHwdp^Hl zs``!#XLODr(TDCL_S?zHKmBUMW%Km)>ZZ;_XJ<TbOhZH4m}L!X@BY>Lt7cAX>?j-E zUYR?pp|P!NN&UKenErx4th?h=qWs&P7d&1b&0TR@)lElk6+XXRY8Sp-w{w=cP212^ z9&gTR?&@mJxoY*=o#!o1HkMWn%M|ROuPTnk1O9i)y-A~L5-2|>Xdsk@S1GY20KzCs zM5V|hi)A1xGiH^Gxn+5fz#z@MnR(&gq5n*uu>IiEUH5c7ed?>H-R`HmnMSf9Q}6=G zq>5!{Ki%E^G*Ih5ffUwahnt>CuW(S<o_<O!AdwF;C(EBcn7JJKNfvT~ALrkc`0?e= z0-TH#%rBt~Em?Kc<R6VA;#PY6vcCy^La)&Gx67U;qhEekco&2OVwPIr-FIImqn~C{ z_tTx1{f#8=f1RxO()VG$(EBmvAfU_zACsQ^4}D(>s6~VgVm|vPs&W=udbu%CQjA{6 ziC_{jfE}X|4TFc?Ps2B;>6ZrM>A+I~7!h5e3>AoY7lYjkIA}ek)?%;RW*oqlo8*6f z7Qy1NWQCt^8(uQM6OinvTjv6uV0M0vRx>|3(rhAt=-%4vkFuO~l-oToughfe1<Zy1 z$zxirmYOy@mz7hTt<`8{%2@uW;^y2Ua3-<u2^X9`EFAj#Ma=HE2y8iwymapB8?L{r zwIr(d8T6&wme2Nj9L0X8RJy{@;&c@)$?`q=aKW;GKiq0Ctk4ekcq;ZDPUic}euF-~ zVfBWk^TyU13>t8UHkOQTpF4kRD`LB6e|+5u(v^{W#I~k}o*RR`YMNxRWGzrXH)680 zL_$$O(C`mR9q5H*5q-i2YcZ@=G>TCM3kHxtwsIED45bvhV?z@}Y=#UVAKEPGUMx#+ z0bB+H<-lRl@(`GGv0KDm;)Db}MLdf(1%R5*1j9h#rol01f@LTSo?UoUxMg9LC$HhU zcMJ{bzl^oIDre5D^qRVYyu50maLdt(2E#koHRP@PRIB~O*L1kDyQpkxSy6Z8;U?cF zTJ5L)#>3T<it-lc6-}x>+$iKURM5jC!ODfChttojbXmuSf?XzWrL{5`p*N{$coiWI znoB+ueveq0-+y??B_EO+#IDqQ_|Q*ukhzW0SMCiImsI{LZ-SaJxNFM%hsaHb{1p}M z*-OtCJ_+3W3W)916Y_plS;9;ioiib4^wiGVnv7p5m0uZ~ZtI*X7ESB8t=agcQu(E^ z`L+%w(#WVL<Q%BTsqo+XO8THM_mID=#y#)+rL)}?75<1HI}YSbP%*f5S|UCfp7HF{ zGW&lDolpIdDbbfDL(Fw0EXE5YD=4}#45JujRAL8yt%<lah~bkOENvDn#f6tCg@cX) z!zlU|i)gv2l-j`)K`P(yI0vZ%Fb9^#gS3pM6`;d!sRb~0O4bDDQN|Hder^g>re)fq znR7$!ot>e`T_Y<?Y9_3z&WmI(i#2!T9`70HDlYEqR)$+IW0=cWa?*=oUJ+C*V6WtA zLWI>rdo%hfB1z%-qT$6QEyc|2p%~>48|#zg`tjqsOT!yIp5+rt=IdBPbKK5`=jJyB z^+%eLTHa^Rlj|-RWk<mFd9DQ6R$#Lg32RCeYb{+LFc)^2?Y_bbLH_VoxV$=IXWj;{ zN;niRADeO2P`A#3%vj!xap(xfAzHuGhPfC>DrEHt255c-whUEDS7^_m$^s+>R19y? z`@uwlI)&{73vrf%Mpr_D<*3|fDWyLOL+SvlRUAD1m<t24IHl*uo>B`<6=uLiGtMn> z{$s}8dCR?fs%xq@Y*<Y+p<Mm04Y6o`>x2od`NH+X)?Lu>NK^gr8Bbl=(>0Sk@*c;% z$1&4d=hbzWc;ukYlUgD@(!WX%>MFJ4C)TFF99da4dQ^3lb@u!@?9|$>Yc3%#y`Wa+ zW^aDTCXYmY$S&y3A6qFLbyO~Dzq5wR9)G@@vmY39#o@yKr}8H==S>gzr=<5ze&F}f zSWVBQYBB?C9#3_Y2eUUk#R=DL<FqR$POPe)gbKvmKhQqKywnN;awQpT8-Upl`st|G zMfAsI6XnvI?dF`lYlv(P1LxY=Z^4+!H)slZQ%Qwj5{|H!pWSr;_oxK#z#s69JR(;R z6)vc)i+aGQ*((AlZbrr^;b4ZxPDrwFl0n9(V=}}-)W}R}qVctW|1RO!_T{5$O6%g^ z=M$HO$1JC{L|U3-wXOPPz2u+!gl}$1Hd$;uAF`BIp7&s?qa*d;)A%V<Y9iUn<g|T% zUUdu6aOU1+!KP8d4#95qGcNO{Z+`pm)|&1S!JN^{3ed5?-KXWlxHG32I2}LdEuU|W znYE_%J6HdC-_(7W2aN;s-~vpFR2BkIHbJRH(44$T^Gh2Y<pQ!~{3rqNd`6HRi1J@k z>?XyKz=DJY_3EOv;R3MzL6eK4un;VCI7+OfxSnX`R^TYKhc{kv_@ax7yJ|`TKC_x6 zj4anVF&a`>3>K9h)-b-h%{(?C2Q)nS&-jWlNu6AqlxN@96>MHLuEFe6Rhu~^t1Mch z;W@dnEgNPhkU_p}@|&yl);jeSB)6t9VJWW~*)nT%6+gB~Tc##FPnQ32aqe=RIm_aM zk>;jh=5Rp{XP2I5w3>Jru}D7n2c6~NSk%K<L755a!zb%lb9NJlRKo>?ruP)(t~$t> zPm4U^e#ppeB8M#PqjcC4N2|fra^|Ot2@d8!yhP&y3fQPD5u&Ujlv$3VS8P-w4S{=J zEMb~UvU3|7bF*1TY0Q<M?~0XvUYsdm?k$Vbl4zOmoiBPo`0#dH(C;j&b9E`w!-e;r zA)U|OPb!ZwOD?-G@rT{b^nlxm`@pTleUMXjm6U=?PMG1&%Xhi+^T-$H7Tm+#B>b>% zWIM|$IRmr#?H7?vp15z{{%N}Y!q+E0e13Sx*Tnnvjve2i{ZPBWY4i<vWxy{4dn(=_ z8m1Iybks-k4a%GaC84xL`k`M6=bTGxP-%qfuZ=#9!enE6P8}0246{r?n$lrL_?JC) zQsRLt;IQzPjRVGmOyh$6vu@L7$4qP=RYk=<k9+dp!)l%Nqq~ZU-soKi?g_$79^lm> z_f3B#ykYcc6(*|?3$tuc3O<7u-#s~(jAmyDfwOmiQ#fo9@BaJWX|tndw$E}>%jfn# zdl|F2|E~kjkeL_D#4&-&ANX<^UAB};h69}+?Ew^0s1(s^4nq%wN%7-Sc41nWF^Gts zVNl^pK$!U9zI%li&IgMBGNn#0YkO_={3kCTGv@Lq=g&OUav4oWEdUi5i+Z;%BBpEi zA@VSNauB?CT!iAWZsB>#&2`Oor9*zXf>F+xkJFFhDy@x|BLOzW64K1vTjnfT_wo&y zENw~f7xci0@}qatLFSW4vb2m|l*2(D@}p?7twMiBvKB?~xd+KL=Qs{|3B>N92MLe< zn{TiV<M-2iit<wd^MdLURkAunO2ny~4pjt49sG1&79$b%{pg4&>J1}O0U1!^&<D}b zFlJHIIT9#g4EEEp2!2yKUDAE$9wA;=x-0L2{Ji_~?9M=5mEKQ?CY+uY(IAh7K3NiT z*b5IA<ld8GbNI5$b=ZqBO0HGtY6?Pjd%@v?oC7%yM<BOC7sQE9<_WMjkf3o)sl$Ow z=H8p@a0K(Kurr{qDGb?DGFi66=jD`2%W86oMagkWi?Cy-6`3zRK4+dxmSgvNk#E$t zrjpHP3tT0Es9mb$+~U4Gr`N?um6j`nKI9AnF%?N(ffZYXo`{`pcs^{G_m5c(4<6(3 zI`i=Th{O*6tAIMjmgo&fm7;n?lmhkTP+%ej&BetmJXE59=O7!tNKOdbOE{;L4LKG& zLRSTyi=Dx1XKrdtPMWlK2`M4(J3@R@n<q)64psbA$`J}VxTjCm(O>eVy0B{Pg*)$B zvno3r67>k$Uns6^Fz*OO5H|rCC80KIiY^@LaUv))!AeSh*>m@uvrV%W(KMB$N9bkx zD5!6M*R8j|_xN$CB%O8qY#|HO>EHoO^7!%oUTP*CEFluGIbfTSq+m2orMMsM5rADi zOBpwCm^cPz#)2^Fx5P@bhoBBA&mKl{%%fpCuV$efV<Hbth%e{*z|JOo_XH!{Fm)4F zFWNMf%x5Uw8(kG{fFoi4R7OG%dEz3q_0R;mYr-L`I&&J<>?r(EUkyv*5(%<xK+=oC zjGCYgJtI7e0nTD#KqzOJr_Lal1|=h2vT@S5+OHVqtJ-msHg3Fs6Z2KwSgcGMTlW=$ z`pCwiqEh<)O;v03TK$?Tg!^rrB-5{{+<5=S$~Ah~q>b$Hp>mUmWfXNs11uDEuozE5 zR<ad%YSHW~T6jgr1gya}%Cm=<J;Y|8Rpd~0B2d<Zii3iUW(f~G(gssE{h6)uOxvYa z<r1jVFsIWI6<&35A`(gL;Oa0l!^}mF#dqZ?%?^z!#~9c$&cQxb@NKinl$^ZeN#T_6 z7fQ=2(y2ajYtj@qHJ?(dHOeUph0>|)R=%UMtGbm+g-bC-kp+AUH8=NYe{FOd@o&!* zdZ-eIIguCrrV_I<@2<s;A|)Riw<Vw=_Z588j0+}jeMF>wrT2i16TGjJlO|I$$s0Hk zS9X1&pi6~V@`QNp-ho>gjl%}-k0;9DRK>dGfXm01hn0@?Gv}Cq2!Qr71d>OhHa?t? z$^c7171<KJNrs^6s42{x2jhnCeBoyP-)%tL`Pa|=vi{=TU&MtEgb(6h?7pP#%emKk zNy`;0FZ}Mpl~<4!=IGI*!u?$8CluUM!(TkbPCibW)Gv2z*uQ^6$IEKr$>WpRQ!j3h z32zLGMu(A{7+M0T{;BGNu_?m`Rgc+}W(}bhhTD+4?g$+nGG90|Q3CmJ&Ndy<=;-yI z_J`>%KMo51+>t-O-ybjIIg#U`j)R@S%OQZ_M>nV2nOU8}_4{Zu!D7fNll;lz^waJL z!$e%n>7U&FAI>7Fv><F1wx$I`>F6B~0i|3=)Q5JAE;XFJO2j3kToIaVB2zXbyQnZE z(dgOLT@lxoEv`uV|8NSqT%(-NkU2_?p{!#>XH_^{)j0wVg^6eHIu4h_h3V%OeI#Pr zr7Ug~y#w@wsI8ru005!^HVDDenc9payEPyOfNEis&uDY}nKb~coxp5i;Qm2oXFh?d zhEbYsVkG~SUDp2=r8+_aE|C2Wu5o>7>`(X6nE;661-5jO>Fb9lO)N+P6fUum#PQ>_ z&cvlS#-p8zIw0g+*uOEpa8ZH@Dq@615NL3*5Wmv@4Tps#yL)dJst*ghA0`Vo6yDyu z8<^*X?O|c*XXKj5LasWp0LW(?Q@BAq<S==b9&=5&eG$(6j9iVG1n=QEt(N``_oSJR zst{vE5$>X-BeEcff)W*J&hkBZdB{HiUf^%J4OnQziArTgI@?1AXGOO^WKk$=5m16h z$|*KrKs&Y=66IEQ!R7}y;~)8MQ}^V}n<sZ~`XD*#)UU=?=T2#zQaaf^o@sVTIcHF| zgs-<XM_NnUYTKV%durUG#tY|<oWK#eUq-lzqvl`Ouwd+|wT_%9%;wXZxKEp=TcSCv zd+h4fW5=ypHE!TnIKk|l*w#EXJ;N5TB5IW*ZYk?<PNA&%tW>49`Rv!v6aIQ=Sum@x zbQx)ZrIQH1US3j|6<U6XxaM6SE^{$TmU(3O+_}Tk?K&4cm7HWY<jH-gGwP?Md=h9` zVmdK{DM4fKEeH}Mei?PYn%LAejPIIj(x)bnmbV}@j}#sUOU&}~xsk$1@*w@6mJ6)w z)ZgBEi@T>^C5*)H#l)X!!;?=F{vJM!j8VCeV@68m(2)vKr%Z~PMQw{(FsuMxco}qr z6XO~q*v4c;U0kpq(+|PoD<tR#!<Rt*I~36*H=y;jS&ck4L2;g-7lcG6LTYdrsE~Pe zwET2X!x5gV*+Z&z#SXh`H(R_$_=h2AQUkJ-6VyD}nm5koU@K(@O2ba=J%w82`Ya-= zUK}ZRr@Q&;stVy^xhtD=R>c%-gxSk_bi#8@K;ac=y<XD3u|(kx7wwk5wHXA)S#rhn z9Gyk2RklCWVp1uTkq;BP=p-iKDCu~(nd$Tx0S*+7l)qutT9m}nX(_Sf0^Qf>l3AHC zbIpcH%!HsTcbZNaG^T&|eAKM$(8)p1YAuYBIR_i1CWGx=il3r+YN#J4C4RfJ8R3GE zTPyG#@%2P0j}8n}+8g?x%CHF5rMwOZ3>Zr3;Ew}dNIm&9DO@_mOW-db@*hGToZM3Q zzg0ZqK~hUc{{ZAHK|>N!ry&5c67f8&4fx~5-~J@q*Po=L1(!V4=l4apw@-;!RW6yr zsW}pj><zpJjbx~#(Kn%iUTZNYU@!@Y2zZEy6z?Yzfq|iiA`}s`Ko3L%KvD#1c`<>v z0P9qg`B6D%j_ummwQ)Yvv3cv}5v*~Ka^&Y9e?C&VM{-)FzVwqD#vj}~yNWUFRst|Z zQe@3`<!(WxdPvwTR0x|NQmG7kKrGBLPKJ9zxkq!YIV&$O%e<ike|8%+%FaoXI+YsZ zZlaejn3Y#om$%!fR&L+5Yr9g7LY?B~{rhiL&;uX;)1Mxv2T~LqwF5v37JJ=UuT(w! z#AD%OStMDH&M6{<ttEpw;Rf;2zx0=KL&DM`e(Aw;&gF~L58UHi&O4GCL2(h}Me#?- z{IDp&zk#ivEM!M?%<sTIVetIHa6BH9g?RRh9c3p)uRD)rTm)Z;<s-<8;)am<8QA*C z!Zg}vS;v3}o;fX^hRmAcbR_B<!kuCnD>*5l$4TiD%~%0*$``2fDD3jo`oj339Rs}& zqnj86MGcdHK2dc}96-?60JOsp1xRZYN+7H>us~3+yNF1KQ2K?@I#CGZIU+olVECxx zl*P^}g2s@7k8HbW-fx!9joVcOF~y^9EExUXvMai~XB(NZL?yfhEdD2azK59**j%(| z8M|)W8ll<mGzoX<oPMvR;`q|{E5eK|x?skhmU=08k*gnCwB^&T7+`Zf^7MB~kk9_s z6LFfd@1SFNbbwz?32(FmUMFZ(Hi-$e7m@N(SeO_<1&t;nEm06XsGr3he)J_HIqWwP zWslIf`AfCY%-&JlG3tRBckF||liZ+Cn=MGV`E#0tcN}9llb+181o;(gj}wlMx#Tb@ z(+7pUe|Q?{P~Go-DBMe~5{{%z!ZPMYHk$4eCJ7suA_gD;;?g^uW?>#$I&9A(4;Rg& zWJgx1I#GI+zzPovY&Z;g1cdlyTv$vCWGV%9p(#j{a^MSKz^9@jG#Qz-6rmLq_(DY+ z*oVSU;n>mytVpHjwqn_%mut(AAd6L>+*+kd3g0rwj;XuN;9NEQlHU+MeAoQDm>Y(T zUcV1S%|(%#=!6!lt$oSXo0%(%^NI_=u}k_=4c6~|9ej<~-2{8`39&iJu|#r`oeGfD zC)NOmpcyq)XrJ7&+9NQ`mh>iOtKPM0`rP5Rkj0zjS6v+-Yi2KOb_6U|KXJ(SmZuN( zSlijBPl*@f#kOfbQ#UkPA{WsHNoe|$FcQoIK6{;HpX4#gA0!`1en8$k2kI25u*f82 zExZEX8WogD&H?2x!Wh9*kBoapaD*8d)D>*%G+HVc0BSD?XGS#>56Yrgi`z;QtOdN1 z)x=U7Ehz<<2=-^hVU)&8L!#+Ntnd(Gs5q)1id*FaYXMsziXoN`vKW4gOX5^-w-(zh zR*TF{VDJt~k*pV<tbh!UC!=?!-Y<kh&>xGflx7H{UzVDI>k00ROHuummRZcA9Ua;~ zeg1M=R4RJC;z3-7z5-k^i2)08g6@mbJC&Zj3$9|N*TqgeBz+a}y64{XM<)#I9DE>I zAc#gM`sHX|Zd{A9yTdXD6I+zl6L7tQvUWzm=4PaBocH9VW5!&1Wd4n*ZPRDmzG>=| z&6}r8owjwx^lhmd=O3Z_o}70hGe>5Su^x_>N_iw&;^ho75rGs%`~z?(OHNs>CZpAA zG?6=N_!e@B74nVAc+wWK*+Q34%p?qIqRkzkN_rNGP9A{|J4>ha*>zs8-|O*v@A7yI zPMT=Mt$VOgYjfDlY7oYF3pIA1!>n=mJ^rn7jmA_|<aN6XEpt4PsMl}R8Z80YoX|F# z+Uo^(h(ckE`aLGC(O|UMj3&L-?DoHm9F{Im6gfGy7unPv4?Rr2+2hewM3F06I}#5A zZvZ4xKwp7x#g_Pl^r`0!?LgeY@u;eKv|7e50-Hn4*jd6c;k8+_NFK>wzX%kH&n%=z z%%6uN`rl$%q#@FnbsCLOiOf|<{fb)9@Ocrt!)UTk%<^Sc93cnY_Fyl43f!LFoq}$$ zjxBCH_Sx-b{Uswpp%L_dbCcd2tBaZK0V%^Nbt=2oZuZkvgVtt1)Q8Mk>&nh{)t2mx z`Ld!WtIn^^isJl^Am`?AqTa3{_K00=*IzMssda<9uV`M^YR<07Hlscmu}0`ah|feh zzVY?218?%t(4j!&i^z<VO2lo@jKht<>C6Oo$TH+0zg%(?`aEVO^jzBK!e()Wr$i7y zsX{nL7IJJ2jE`r!6y`EfL>lZ>qAwYpj`of??RBC<2AoK0hKE2nC@+M?O!TG%29Nl_ ze^M$UujuXK|K>F$l_3wJ&T8Eu>6b~9x&DW-vq#OC(Vk!9ZD=6L?1abSvUu!)?8>~F zP(fI3a$AdRIeD$6Nn#CW7uVMp<VIq#_IY#LbGYvt_QNZ8Krjn})bEV+-?wo0!iBpD zb9HO?<kp=>A6va*#p=h%C8HN~)K#3<pdkLv#0fbtMG4u7L?`ooso(jI%cm)uv%Gzp z@KB_5GU@yzmS<KPwR(#yFPw-4v-KLO+G=*?#getzo#BR(qNYfvdCsmm=|>q|Y|^eR zR~AK>-_x5el#>a^j|=xGD!MD$D}{%y)Q>DI6CS#V37t|`j2v0PeTyX($KekcnBy4a zXx2gxbpvG;fi^k{zOR=hf58aOgZMK99L!80X-dI$MF(SyYhhd5Rz`>4l5pmSWPbQk z#4ZQpvS8E_j0R<(@--Ps0aG$-Iav2mhR`6tErHW4fGLXuWDxnO2S+DNj5cwshxnhs z0PK%@nexFxL(qb|M>8WdoqNSC*%=*I+<|e@Z$ay#|7Btf5-y0AMkfl9!IQ31!a-2} z0FZ#O7{^k?wCJJ}%iwij#X_Vn6!#52CiD=JX}~xQqCVOqrX%XZx0ZVeFim3P#y+Ik zIJ*yF<c7iO2*k^9Mj)aEm8lg-S+REFt0(6(4ec$r@7zghXFVAkH#VF2bGpXl3ZKy> zd2w=HzqN6C<@D{2OB^jLdoEZwzLU8@WpLZ0_H4zb(PNPXgd5%U%K5^(Z@qQHb=UE) zW!lyfN5b*8X_=YvAg!IvmdqZna8x+{8hGT8<FM_AVX=Gd#px?^bDy&w7@28?Yf8Nj zBy7U(eHE|Tnm;Qq5pE|F6J^IN)xKZL<s4(E^!=(Poo}}~j1Bg{u!ft8s)WBgEKWlM zn6lJ1V@vYJV(2A*{_~#et`q*z$b3I{)20A$FoCkz*43e4adGeqdvfDOzayL*cEn>_ zR)wlYT{m^zcIU;85nC>*m*wbuptyB~JX6m*f7Wt#!s7JBqec}c%12)CR*ipH%u`Fg z_S8fc7Ybj!hCekmL<c^4^4MeP*UVlssJqCN?rx%+Jh{Ju=qhnu`qhsTqedl|yJ~9- z3JMEn5$yyUBAA%k+NvtDE)RdK_;cQQd3g_!OX!j5ZdyS9R8<KVS65Hg%$>!_C)(|& zY%zr*;3?1dTV@fR7nUb%`@L~RP-j)jW&$wgNw36RD{xolfbbR3rB_a<a_E&ZV5o8` z9eVgF!`eB)Du)K{YE{@v3(cVWZ4|zK@x_{&i!QIm=&Ce$Op(~@_?nU<FpWykCkOkd z3OoTp^vki53*bxJFZs3Pu;h<)WQ9PH85S|LN`o@05tKmEA6hPw?#Ngv+8czII!5%8 z@uqS#d{N%Xc(;)s%won77)ntN8Z7#QBG@D{I}#-PORq#h1-O(cjxhn!u1utXNx&#F z@PNnzFxcTU;t2C;Y1k7i3A@9r0m`a@Q5mgHJDE`$@`PyV$zOHW^_rLlByjF9Ge*Q2 z=&kI8nIxV~OSG)3dH9@RRqh%irx|7<Ps{mEm26=iZy3SLL8s8A!6t}nHg(qRH;WhP zQgi=)5$7;^5zGhlBJXP2RpOzICgz7WJ_jQSqcX)pvgJd?0%ldXILjR_^{>hCl0_=c zav)S9Zttv)n}qpNrRf4WY*^?0h450PKeo87y2Wl*EA(K&Qz-ZC)+=~s`F3upT%#mQ zD+W%{to-*=h#u*r?j>54(1Y}eCSnR&aXTA%|3_0XwXqD0=St`-CBPd^#5lefabH(R z_Gac`OsG`)<%4uFFz*gXoRA<zMsAukCL#*6aNQ(=aUHO{nJ~&YT3anxNa+NO!53tg zxQa=f!Qsn=YifW^GI3@?&V;|>!W1u)5q~4m((-dPA8D<{<hL0;?n?5=1=5nzA3kqv z4rN8E8jVd|odrb=O;KNwr^{y?-ZQ4CltfKETZ{#Uy1YnRR$-Pphg_!9WECov+jcuE z^o6^3GZ*B?T9a}4_T8bZv8A;_UO{6)K_h#AQOWe4CH0AgbE@@~t+{3!e}eh3mb7+Q zIH)-8oiOv$>IR3#ij*}=vm()!ss_8(ruR9F%d*4&kGb~_jH*ie$LHKKHPc(_WG2bX zg!DF<1V}Oo5K1V45Qx;!JA__D7&;0lMG!$SE24;s;@U-w?%I`AS6p>1aaUd4RoB;D zT}U#Q@8`LbgrK<YyTA93A7O4U=bU@abDsKnda=|++w4o!v;H_b&81tRDtJC_2k&;3 zNU*3TOAoL=e6*&ts3bRXZQL!ol1lbldULMyx5X8q%}c_4gXy!9t=3!vclwGC$14nw ziNAxaPve}Bl`Kful<FaUuwN{IyBNke%`(Ug4JQn^tCCl}52?Z_jRcK<5_mA7K~oL1 zaYxzL&^Ck)yuBY`W+qdgdiN}$P|dU#9@HkNM`PnjYr&|7GXG!Cs*&bRz4mN?-GMXQ zebQ|&6%CzxVE^0)-1U7-ropq6A#H-@;z>29ZNvq?a;IcW*mv@~9S511Xthz~oXu+4 zFp<r}1w^lqRIj@p4)76RGK9;8FVw*8R+j!uM7N@~3j+KjXrbtVU#yrAt;*oYzxV|i zk8GKE>$p6jrK_U*x$o~PTU5sSQT_gXMIY>}9Qzx0p<#K&)cJ){SPDfezTqimnj+mM zoIrj5vx-x_$>tH3^EgE9TtV_2qTGct357-r#1Pucf4|Q>5Y{|Ec>yy-9(-saeD)}0 z8Bs~-6G@Mg%&;Iprx4jMu;>ZX)N?!1%3AVNTIn}h6~74f%t=)pEme~m=`I$iHV#i` zq4eR#Y8Eh9nz<G_9FZUSGu!+(%Mo9x*^g^FhSsfKUdKN!zy5AuM0@X+b6Iry>Sf8E zj^v9#kVD9>L69yyLSoSxFyj&NKv#yS+-1|_e$EF)ST}g->eAPxubJu9l)71?N=z$E zn+EMX{n(BDcWRU?mD-M;?kDg9|A~(ZJGY=<cqYcp?_OmlO+v3h*$Ygj>dgGd_TKV* zUPiS_qv11u$&00@AEE)04PyFH2U23766Kg{;f_L%E%<z$s$UDpqQgmAB7AuG2k~-+ zF<MOu4{n6N4T(PZlnyBC=$>x4as~g|yh#;nrk2f{(%4+j6%Dy|XN}UTnw*;`7TrGS zSEo1sY0KE{J}9a*;tFI4;8uxo?!?{=Re3;q|Dekg{?pTlY3T(#LG8@;Epi?|IX@p% zFekW+^VgKkziUd<nk_Gy7u`3tkYj+?scLxU)DfmVJTPP}QO4**UF1+~tUsP<{ADYK zgZrP*)kM2BoGw2mMz5(GM~`GB@K0JJk*g;3mCzB(HnddI=_!Eeict-#KiTssBZ%k+ zKM(|8LAnW;`h!3o^fSA%|Cvdj%+_k(f_Z&T<hVER>Lo=e?B&MKi5{E%@x+ejxll`_ zMX5L={cGaKvvJ{DTKQVQ9VuQ7$k)opW`8oNEhJyt5-pEX0!=l^7|k+;RCMXup#~(+ ze}@8odR%~fk&*mPIih+_w)F6pDXZ5#GJ#vyr{hWgwmK$A-~Zv-vrBuc`j?a&dl}*? z;Y6=gOsuYGi0rs_{1fZLqq%;??LQ2i?-+Pq`sc(uURxm+_*1-96Z@o5ASBU-XuD*0 zqv^>A)#y4jq`|Erc$GR5B3Y^1$XP1oGqi2BlMiMTI~I}lG&5gyha?&Beq;pe{EJF7 z^3;KzciE=+(;b!Kq9VK2m*~n&jZJqrlG18(vTM^^cBel!HPe;os~s0TnIi9GcV3g7 zQ=69LaHP{UKfOghiw6ScgYqIo|6oLER}3l%)L0W!60N>*+|TZW$*7Z<5S!pIn5=Q} ziAiyBQ0O>tAW=RlZ?RBI^lV~$^z4r=jE_rjw7}fcB89qsO}uGXT}>b<E7qOd+jsNO zpWTuBPgu5BiNuF{kb0~T{*E5})QSrV7(<xiVwkmXA#ApV1w}`gL*I+ED9AdA`9O&d zi#tuQAp#deb|6G{kF-L#vLX+N#|%(5BqK<fD}8nZnKI6uV^iLIiWgVSzcZqElTA6t zZH;_1e-)0hJFpi~9Fabo%iX~%;DmQY*??jeEgMZ4G}36e`Q;IYv4(l0z#*4crSO|p zN48<`d?_`yaD!=sy|LP4wzFRYxJ9l*t2E4Rmapo%D0+<_iUNjup&tul6^+D65^gbM zSdhIOr9X!UJp{cc9uF}eN`!e}?t%zRJ(`lBfY?K=3yUZ^DH%gk;b5%t`2#c;AXAjq zRg~ss1Af21&z#{y3rb!2I(MIjRiheD%{+hP^vIHg(Z(dUNxHP}`|V@*yf|m<?pg2; z&*&ZBc>TzwzKT&}8-|qV_y-mZug_yK4wtYYKG8WOznTvzQ<Hinz{}@<_CD(^o?N?r z#MZ=-a~jeYJ=ECz_sJE3gDGk3;8Ez(=e%;u1Yh~o>06iXEq-ZAZAM>rvNOBSoNAMK z;hpe4&d?=fi_`L<rKmy^5S>G7!Tv|MsD$s5!}%%dUe-;eI-tCjt$oDv($L1l=b*`f z!p#u-YLC+XVAoV3&lE1;ME`^*77zY4H7#8uaQSJ)P&-&B`n8?`g|%xr)0F8+=>-X_ zuFsTeXQ_X{h;ZGEN9Xdw#8V5NoM_Ya%~*2H(t~%-Zd#V3PIdH33ziJcn0Ih?PcJX_ z>HSq&y*H85>$tRBqcLq@u{O!Jv{q$mY)DcY6MMyry{mWU?w`4GP=3?n)7kt-7cWeR zT~Isd)bcqe=B>0(?mfP=zdvCI_gPPmFuC8$HeSMxO@>uKaYg3cG*aw)DD@3&xaG_O zSO>5;Ih+Z-1ki3w2zUCiMpwM-6)UY<xnsvlwh>;kZ&H+3MA0?N@wCOolH=<Y_l+DU z{Zo2P`scWjHy{hzl|HP|@7GyILd#dgLLjEMbDgDOLbqE#L|=NN{2n>NOn$fU&=qfF zQm1=tmnZC=D+(jie{%7_G(gdpv9NX%Di?+a7<lx46O9djA=rHQEjXNl49ZWj2h?X| zzQGL=;-(yB0PMmvO6&rTkjw_!PK=@51C^M(MSyDn_HUD3NKcJDQ#ok!nKPT0K6va& zODS8-+>(3R9J?r<+1$76lu_$2+EXp3CZ1tx)>pbH-6&lgQC%tBZt*<t>^OlOamX;Y zWXAQaWCe$f`PcOy$y*AKjp@eEc!Gti-R;R|qzh;E{Jp;7W)|K&YyWSV`b@0U;Vd%f zpwXVZaq}4_KNnA$a(~5CDKq}g4-mMz1ew1cgH;}GnMJ-tsR?eY@*FASACOl^GAv3p z)OTPGhS|T%o@^zU9|GcnCIeqgcEQIkh>iz7kCYgr%N2~)sfa>?<&(n2oK{DteOQQE zgp&q|sm_kM&Qx)b=yM4^m+vo$wn*5Pm}uj|Hg+EwgChzo!f~@Sr;&MX3`;nznd4-- z9`;`@hJ~F;Nlq#3%E{ptrY9z*Cq~9cj)wy^HGyz+$&GJX#9kP_qHo_7!=>Ic<#}N{ z=9CMV7jg(&fMRse73eEM8ut^!Puqk7C5I7!c+09$2U5b6Bl{G-KMu&==nDGixVjJ7 zqAcWfu5e<?7auD~i&Ui*6r^!y)0o=XkwM%1&IalI`%g>1f56GVLkBvRH8B7Eo4-3X zn=LI!+hpGKf%Ln(e~{))dz#K}#y-nG@jcr=?Mzw$_vh-u!s@~?V@4OGrWM?D;sNRH z(_P!M9{3-&Iklj^{%+}aA8umW_X^VFJ(mCBCh3Rw3Mj5Z2dAy?F&EOeO+f!&E@O)G zP76RCQ{-6b98?WXVFgZDR8y3^oSd4BS2V9+H)_&C+AxYnLDP_;!X*R?a08@WnT5vO zW5;3O%OLcOW+gOA5GDk9;-QDCE(Z#eY8Gk>hqD}E!MK_yCvlF(mEXtlPb^t}+*c~? zbn)Jln2c2E_1n#EW8c*^c~;wqS({S~PPg7yT9srgJQ~;M;*mceJ_tFWM0$CtHzp>t z|Ja66NhVdS$tWcDFLQ^k@$$m;8nuTTSv=|<Pmy|9NlelMNtwk<>L(?xDNE{gY}D{g z&mnd^r&qu75#E8LZZ8|*GfX<!ad1)7{`ZBg(&Kjan|>u7O||NbI8LSFw@j6;fiY?F z2dN$3r`@$P-Vi(7T{|^YEFI}pvFFZ{_b@IqZ>S|dpc7pwMTu4*wpguciSdruob3aW zm%3sA*mRCl83KcE8=2w>#mqLxq<nNuaK_sBA+GW@Ms{va?TpA%W~FKU!aXwvO&vU6 zsy3ESnix7Db(~$8QOe*3LNY$)KbbFzNAx@4#3TO+HJ;pc!M~<rn97Pn%>CY<JyQ&2 zBF&sWj~i?)gdluDu)9u&;Bgb!dpWa$kLAt3Gu*TBjwhbndGiVVt72JUX@xev*gu(j z=bYh0*YVec@&Y;@@aG5Fg5+$aF_9Y+ITE?4UW_v<vt6!irNybtV%wONn>tpEHH$f} zmJ15bbo7xgUV83tr<z06tC!~u>X)|T#|MT!`n#9P)G-#WqCzn0)qP)l^NknF)CPm- zaaRI~K-2dH{?#`0aQX+n0EDa&d_fZM%4Cm6$h#2WAuM{pnsx5bNQZxz*@h;g;ocb< zf?PFVkvezyRynt1bCdL~ya9pzjcuQ9Vc{*GZj<szQg4RXZ)$?)BR2Tml1Ucozin!6 z57U+i8GXyAY;ebJ++{KFEwXU0Xl~-0u!UmE-%#dZjPPn84<0#kfQr~9lZ&!p1o~kd zBmiv)vR$XHw&%s=M4aKiBVQ?NJbtX@6O$8SUx`gj#3Dg*R0g*6PfYB;F;Hm-GG*MP z*hG7=BQbUgc>bWB8&(yNE(EHunOyNqplaRr#`ZTFw{LG0@*1~uk1nC7&_ZepR2CIg z2HG5s&*|9b-Rl*H0+p2kX{O!&a7HC}<nrCyyZCeB;oxN>dl7mPn1}vkIOnbpgHPq) z_et;X`;rBvGtwaG4E!<Hb8}Cw=m@Pid2$8cvf?Crj`Mfi0`6bIdUJPOK7NwAgOg-G z8IN%oGJ;Q0%vLN=EJ5~)^@`2#fZn4x08fc##ht*_KM92O#}!X0&MKZ&yo#>@^At~n zEV=|`@*uL>(@EDb5rVqO%i<uA2eCu>--v*E5Nz$i2JTf^$q9v)s8}k)8Jas(RwQBa zL)qqWdhtwn3HVj1K^~gJpw+{Q#X?9pP6zLS;|aVUR1PSwaFf#<G&>RShtxrSr8iY{ z+BKZlZx&UBfS=0c&}(>~U&94>YpRv0Dv<gOWzsQcv04#td!3K7J<B8Krb%it7n!BS z=6Gj{@T@J4KWB?eu)WAq-V{=tK$P2QDmJkjEp+yGPPghQYg~t)ZIMpiB%NflEg2o- z8TW?q%u{+3cS2fwmP5L2Qx^9C6GCQ}qeFVPf~_=FO7(1*x}!;CW27AdM#<ig^y-mk zkDY+~>b<U8S+9>j7G8fw$*(j;_MMmhfbW?expq7IJfog@zuC+)hx%PnE!D8%j+SHi zCzR!FO#dCn-@9R$$ZfDE3({>GjSZ^@)M{sn#b&d4V%0Hhgph30XxMZy*@kPNXAxMM zk<HVpV?B2K^TaA+q(jW(&>N&PLUPCJY^rqB#3u?!J}DhkzR1Qur{-A8OD~z)M=Qnt zBjzCG)$1W?cOom6?h%Z*`m|DHtEyP#T^~MuTFnPwo;T@FGrdlF`3UR%)kkXS!jPA_ znAT4+fp_{WD>UwsKK(F@ZExq$5O%Z|`~(FlAIYVD_*nY9<9g{cmhk64SF<_Dh+#wv z+%^i5DD_nt|DQ1L6tYpZTMLPA-95e?g^z9G0JiYhrjCDZdQ5oZ!BCErm=mhZ<{LIW z!)CTsZ9aQ;bK1k~9>Oq}Y&rd+^kx(2&2_L)P-gF5=;4BbM<=1+NaQ!C9SE7sqVPs{ zL_&%yR=~g6!6P}Pl(N$HI%|Am6q`PApmc5I`9%}Uo48`>*iz)on3iskK9E8yXYs## z_SCk+3)qm??6sBR+|^Q&^z1cb-(XW-zoBy6;>feowS&g7ja={czHB;YTQOnQDybZa z?`;K@qn)p_nuP~9KhQ}Vkmu`PvhOcZa&prI(?LH_aceO=)r$+=3{xGkEAnxk1YKuw z5aG#mNX`!BEOx499Nx6<rDy+i^UZ&H_NDMz@o+2SzC3j3OU6n6v*Fy4wf8>Xdf-6o z^Y^Zuv--htuiSUvcfsG^eDI?Oo0qJ8bNQRc?|Vg9)vhibfAh`bON9&T=gw`vtF)4j z4BxeDcn6=El{$ZZ3co|R<#1I;U17n@d0?W6k3NpMdA!U;Qv?=djbG9`|Kj;5j|%$I z6KO@JEig2G;Id7$x#WfPsmnHlwy}_K{A%0c_OI@0PrK`@b#t`8T0C=jHp_T=f5$$< zw)>8AAKG0mdnA<}03atUBVW^!-A_xYPTrm?Zy&(&uDiba>aJzaBYbZ0ulhaq*L@xP zt4ch71kLrM4a#L%LI7>2JZ*${lLQ13%GH*QZ0`Yh?Un(xdjS0ThQWWg9x*8sL7iv8 zk983um{!7@bv>-C*8^vCk77TtFpewEV?>bZhg^^~P?_2(dd>OcAD~5@J${susOJx^ z0=V<%e{{ak9{iaroB=wEK>wfo5CbDqf0{5D!p)1Zfhi-k+n)|5qiALTI2{Ial%%{? zDmpGi)Z%SzFLC?1V{I>uL^`ABzY60VV={g&c|F@WVvcdnD*RS=t~)B1FxygQU&?IQ zxV+u|xOXYi3|@Ks+u=*Qp6m5Swr_a+@eLavdrW%I-?x8Xf76tBKDpoIq+m&Euy#bS zSGqlAuo2vNn#N^_cf=$G10JZQc1x$&s7n55$5iQkG5zJ2rFWJty}8H#n^JN;hLoHX z`sqD6DJeOg+(|hpIrN*Di;(s=(|+_%x^KkND-SIlk#@y1@%+@sHbzU!u1o8s0V1|N zzpx@h>&QyZ$yG5O@(u&TtT!|AI$p^k&lb)1Jo?^JjK5uwbxiORzfy(;hx?P@JUQB^ zSY|XP-`;xkXe%!rZN2^WR@PdPec|2gii&LZKvszRE|kR{$gW`9>D*Deuxas8p``6h zRz*dY*q@fa`W2RVBk`f>pkMD{Jr2|hxoTyBC`To8<s=bB70sB|NhwLZ1Q}?$A(GHN zs5=ouhkg>3q)1Oqd_b{yfC)Fh_5RWNLu;1Ip0#Av!Ma1gdE@r!@79a%M76=*cZT%+ z`YoSqV+rS0oj<x~%hu!LW^UOq*)_{-OE&-E_TKZ#3e-k>T%QLgJtGOF{1dM|zxT+S z!3nE2Z&@`V_}HySo~$VolB{+^Y@lKOvUj$=&P-!>+<jul*lTA@4trfT%W&K6Z+Nr3 zcB3_+-{AJM-XvouFB=BwOMpHn6IwZHbvWXX`T!0-q#=OI4aE--76~M7lD7+l&NcC< zDpAfW0B)~~)Ik{!(K9NSy96iB2#7->g+-XuAkmG;=TH&U%;jH|SFgI`+P`8dF_u3_ zmvq3r+u`L-zZO-SnBt5&0YNaQ<9+;<Poc*}dODW#Es`SE%db%-#cEA?xY!qODC{rr zK-BT+Ey)hA%7-fUsCU#KzfP_<95*mi6CVx?WTZo5hy-N+ZJk90kgkDt?b`<?(+W@} zfL|*j`~K@%TlX?AHHY-ay;eN=lEO;QOX;Ch`48((zS+@O5)N~o20n-sYlddAYIu*| z>H)y0*Tc&Uy*Fwymos|=p&j!Syv;3=-ezC2iIM8-Uz6ITRz89wPj@`WoqSFDhFiqO zNv%>FyM~2fsp|+?dRsa|Ca4F(7LO42@QTPR?$(YDUI+tnGTiYO?pAq&g=b0%ORl*? zVY3MebFPI0egUGPVf*<S0^(p~&uYd5O@=R(UxuRo_luxYj)(o(g;z$i?!3qh*ZB%^ zaP(k)@c;r88WR<dm@nDP_8+{G*}Z_c#LX%zK5}?mU7e-aQdfuVpIy*J|L%NUTXyH8 z=&I0LW}EADm?`KleA;C*zeR77AMuN?V<BiD!(j>iMJ}6_?z`$wF4R@e)UBp_M*)Lt zRET+5@AxupZ<dK5@33^fx3(-^dO`Z51t)RAzj=s&B;E9{Yy8RlGR30xQ>;)ZJXV-q ztVTvqFvKiI`9`p?vLQeN6&?@an2e3(YA871UDHi(_#kw^keTR5XFzTV>ws<~y6aFC zs$4u5YHXy22sbhX$7#n@Pf;bRrc{psUJCx{@Sl$n^*Xpe>(g?qTD>ktr`K9@()3OX zKsm%1o-Tny?;U$rcN|!~SCf=8GBEBP2lw1t<^gH$EZ6+L^Ici)v;pR~o>L{fGpgd6 z3=<*>LKGqu3UdVlr?zsO70@jf4UaT+9(BChrb5Q>xYQINB%~stUX03ygB}68Dow|+ z)i>O*x@^hy3#Y_?5DLY>U!*jne0PSoyxg0yyF8<`Bz@$FPdw|JZ=!h<I{64__)u`H zO_;Ti*cDb33ZH|agyt_#D(RRy72l+k0CA?q<eZ*D(uAVbuuE2DtYrM{Oo*RoKX_gs zF#uxhC>=S}?dc2vdH6a#b?oX$O#h8f&HB~XrkD{U1~xAACR|bs=vIRd9U6P>BO#gY z58pa1D~VGqt^de{7#d$}#AB;oVojJqCx5+k)9#yIx$ySV2c6OjsWyvwUv3r@@M0Kh z@hf%i?4Prq**;XI`?Pt{iv#D?e!4Ni-=!H($X*C~n^2JC2xq&TuEaS@kc0qp&V3aL z@$W_2_bf_wCqtqm#XB_jSE}2i{D%U5D6QaeN6<{@fp3DFd{LoMgJ%%T3I;*tf{B9< z%D@_EHCU)f%)8R#gfvmalyIH1q!_;T_3x#&?_a;RYT2rR@mYeH9N)XKG#$}Mc~dt& z^Y$|vr{?j@m|oi0J3d(yvf>A>T2>{6k=i~Asesn22{0(d8|7SA6*J0`lgnmQ<c}y$ zjlrYvKy5;-(<JDUQ(ZQjD=j%jFvL0&w0IJp7n52%f*&EF8R$p$2>L<K&Q3&gVv+0` zmgCG5WbY{tLHNNEif~Im^lHHb82gFuL7EfzqI=M@pfuLO8;Pt59^!Vu0C7l9k6|Y& zSU&K@`N$qiktMQQG_l2wRI&Q3GR0v}QSL24sLCwqrbX&hYqoB~Z<VQ8DdIBenpsk7 z`G!TcX=$~KHk7wYoI<JLg~$m2>W||r33e72nPH0u+Vy8msqDTzhd(siII)*BiaTYC zPq0gQhxdGNA#-pjEiE)S^8)d39CYSku|tlnfi_5?A_rwcm4{z)RF?=7N0+wFoWr0n z#TOPVX=E$HPY6rzz1K>5Kj;#n4vcOd_{WAA-HuPToMaiNps<o{F=YGUTRxPYfa#{k zrBT-+!E}mQ4sZQ@uhF_!`m2SkMA5rsKC{yfN(hw-3_3#ih-x`W(PZFY+LH~yP6kCM zDGv*yWmA~MmUy510%_yA@rTd<_59)SSS^3`HnvV`mz9;X$V?P9ONsc}QX?H@N@>Gw zuP%>XO*gG$>*U9@g)i5INQtb=5W<*u%c8M!fCW{k;P(BqO&IXO!Uk75P#n+?kPY+} znUbiKU4`b$_nbzf$|Y%(UmM+gPkQh4p5qk=bRA$<XSICQlBG^6Gh3AvkvlI}Lip_< z>2G&aD{t;`tGu~6mJR&yZe}0Uc-oX;o4ax2Tw8+abbF_%jM^aDALO~F3YgTeIm?5y ztG$5&f%g7|`cW5wJ_SSo0cgHJSEU36MbCGAjdfS6-~NAWj4?6yt1CWeP+Zz-utc_9 zu9k>?g|CC9#jy3#(U-4YL3ASX;n!HE(@<57%s1_gJ-?Rxt>oC!d4wMF-_(u19n_fJ zki(rLq>G3}hm8}ot`n)a*nMRqh`-zj_{i&uW@zHId0M8K19!R*Rh)1KEQT#}$8??; zS9+A~J^Ej^5_N-@j|LWLnL10Ipk3O8w(jw9=1uB6F|B0Xx}UTn>3%>nloDdrOQ6%Q zfpw8AGY$^v-hbNfJwHQ4sE1(IbRgZj381okfy|I#x&%#Ozz@R1;2~~;*A#U*q)V1! zHvHp&{Q0AF20ZYU<OY%Aub;d3-gB=rf5$$sE-g!E9Xd2?scc~8E?&QK<@H?X2^#88 z$S3%}AC|s3Wx9eC^>{ps5~OngYql?4Y6o0%<dscq{-#X`^8BIn_@7$uE{zfmxDN@& z5hILJ2Utg$&XwNX`lsUvL>Cn7l2S#qp&EFnli(eFl|BddSqWdUG*}>I!WtblG7ZD5 z*mK~)0x1tD_<<0k;w)!g7_u;>D1bnWc0+SP67|ai)Wwun^t7QBj%4Y($KH~T^;`bN zzFM{BhCgjv@yBcA{?p^jOMOxv-76nNfa@La<9|o^qvJd?yc+m$8yb>tK?C9dLJ0yN z3XMHS<m7L{k4hc!N$)?SoOSG@mD~Qh-xwW(p`KR_nXFlg9wjx~d~t1e%Zn2o&Lr1c ztiro$uXM3P1rt7wBqqNlJ&Pk~LuIF3V+h)C0_++;--hTj|CaD}5hmzV3p$%p?t3z; z7X;)idS7>+Goj0cdo~T4&@KJzk&mBTz5^A9munB|didgX&N!xjvh~Tmr(W(Hl?rr0 z#AB<TSU%&>p&84c;7g;OPu{(fnxX9;mO2tr)($uRlxCZsU@3Pz#f(WQYp2Mg@h_d- z5O~*^BunpREq9l8bay=|bT?rj$b5=yck2U*;mSEP3Xw!o9SyA>vuE(K$K=n>qvv;O zG&vwbJBMF6pANq-di=ig|9)P5XQwtE576uyapn9v{J!Y%`_9Yl`qO!qyClf-Y^j{j z(E&_n4uEYi>spF~fo=vRAj`U4j-Oplp_jV_7xi&5apCuv|CIF3$t|Dk&=F;6rf=Fj zAzFx6ATYiXt<Qw$iHnzKB7SYG^u&jEOMl&KHSYa<>tSX&Wr}{b;}fFyyll0;9DUG) z<8p1<J3o&+#p2t*Dyj+#>!2O3B+4nHpc52T1?xdBm7slTo!l0*sbC$W@`k7LD>=Jn zR@DNa$-fV{r);hE3F&?Ljhlb2jLi3hR-28B+e4SD#38E~9uYn9L@PB#E9Rk7ETg-9 zq6eRdzNO>qpUkWBw;}ydl!xr%&uGF#9FU9aDy+;d%0<KrIytgPH+x7VhGzE2ZBzzs zI?Ys_Tv+7QAH5R<HY|i85ySRLjHvwU?#|@prHE2!`MzzJxj*o8u+VpNeO#cED~zsC z0pjuHv6I$4a$s05M1Ts|AwcCqx`Vn8$`6bZdFmJMO)V%$l~%v#`YgBqnu?{{kG=F# z#6!o#a6xL@$<zX_?u(mCO4#o-&3o_tB62@Yj7f#dO7vZ#0&~PrvV@UbAkd&BouJ<U z3<lYe5$vKYIp9pxp`F4pXs(2J3Y<ceP{HT$HJk5yVb)P*I_Vp=;=Wn^w<c-RbS_6m zffopand4OM*|lZ;8^(o#lPdBP^nZNtCF!4WNpUU*XDl3%>EQ33|ICfEi?&G3jgOz) zFf3H!-<LTpbJXAhQF|nL!e?xpt8#YT-jU^1ZRHhp1!HC}pObt4vr^>6tWkNHn#6Iu zan!s8s1C{3m)4-|wnCmLC&Us3j8`Z&SSBhYsuPT+BXfXN0P`zX2s0c0fKuG;5Qpha z6?9m-V90Q*NQPcZG5=cpJtAi|EzB+5GIjURL5v?5o2ZOcS&eFS!2mI(f63$+t+8qS zmnWuAKk=o6)v6KS9R*ou&R15gdPVy3*590zCU2j=>J_e_K_hBCnf^d|_THv>W7XsP zIe5L@wq0c(tW~K8hXQ#jX+-Bkuv-7>@h^wX7H85!q;t}judJH1mF<7%_qXE7<aURD zvRoL&3$n^OVn12|5=#)uGNhXEborSf*0g)u(&>9fJ}Bf<O}{W1`RS*0BWk9kwk@{D z4)*c?;6=PG`U9d2W}1U^v_x=sZE^BtB@s8dHv_1qd)aMw6lv^aQsj)n4SYf13NmmU z#KQLQBI%k3JYC)+l=8r1$0ek?C%2|f2?iUa^#}UtbDt18KNn(+daG40HCy#9daKk- zJFHROiMWRrF-8Abq#9D$-Nn^2%HZ;J%R{$}uvs_Pe+&?ba3{dObo)>5jy^ZiQZ)3N zf*V!`W-OmRxnH`u4FAlHLn+A&^}(>}Uvm8l6@+fsRX^&92osReGUO%dP$3U71PV}E zK2nFt7z-+qT)&cW?d6I(+;kdn#ps=v>-oq<jb^(>Z_r%4s4?iVNgF>p60twx_14*) zS5){A8*<2IO-xFR_jcDe^6}3<<nm)5@fHWXb^PVZuUUg&)*Mf<w^P3b79Ad-@d~0} zG8Y;3)Q`>}_O5Q|AsXT#4L(ySAtzr_v_aV|D}gwKbR9VGwm9aK+asZPABUsxY{yvv z*J0a1XAgvK{{-7%G%)5goRn>$4%y2EfqWhnG{kUY4|x2ZKq2YKk=!s<sc31Dz6r9y z(hTV^{${YjLFt>87HDhxu{Erpq?rG%QXz#}!Yv&wJgpc&)_4V`D|!!o+vs~}u1Q7x z3It-3!PCf}ssgGOkmR&NOJ@Qk8czc8{p}B*H<=vmtqzmv{KM_w%f6M9IN`~l^-pc- z2yc8`e8rfaZhS?2d?O#;@>E-koU@6&K`>AB4~=@oyXCR{bMNm;z(nuw&T{&*W%*My zXK5$`tDL;aLXnoADONPqD|?QL73sM{Wdvt&=?2iD75M%XV^5ejXdVzy<mQ#<D#}yp z)NnY6W2El^Fg1EL<hS~`ShGpWwI|uE1{b@;hd4*%ufn{<Jf}2}i;G;!lGd>P=2Sxr zmm~<|+vg#1=a<@Cr?AYHXuPE0XLTH9TCTeNPjSim5BSgcj%NmPYdB+~Qu+>BCX@^9 zj4?@gT!>QWiLVatyB}eyBa76PNb17LsP|i}V)P}Y`cC8?j>akHD*D5+-ocd20`FNb z=zL!`kd0)MfJ3>G{hB?;-h%-~;^0sy5>gteU7(sk7V~H(X1`Avl($KA@+qU&V6MeA z49F>+;5z>3tP31eh+3+04!T|kcxOlSiGtTaX^#<)0C+XHW<-~Oe<AtS8R(q$$QBAV zj;1=O-SIMi5gr!3uh`*4mI9}P9)ZAW{6S}$hVSGE7MAlA-9-~51#qc=-%oe~cn$KH zmqXh~XXu176-LwOJM9j7GmHUH`_qzYhI{BVy;7-4@jIM|pYVxzA1aR9@f0*nKE;*M z9YJs&0Yp<!JGm%$DCkpwP>^XeP{jLG0a&Ev<36z*n$Lg|I&(VWrEFU=#2jo9Du>`K zPD67Pl>^7bF27lcdgCSPR3-95qs&S`(a;eR_#J#PAq)CY8md-tkP0H-1+ItU*OaPM zl*uUol^Z+qJ*oBrFI7ubjNFg-Lw)2&i2z%tRw0jG6rX*h_F3Wr92=E@N)@Sm);PE} z)g?F_rTVcc*+aJFrRTOS(T|C4=5Q~wUa1Kw#lE6Mv1tS{2)9oA$J&HN*R2@IeW$jn z*!Xa9UV|etGV)vJ*nD8>a-vnOj58#tG`hqjm)@C}8gH@bRDlNMPc;tbQhbS`KF7dw z+Fn|t(b=DsFHUsZ)utiN-hjA4TIq!Ryn^&Kxn(o=TyM)L@|4E_3o9_SZ+#jQRltg2 zd~fGq3uem1MSTax0`@#Z1NB6fUQG0*a3c&FbxcD*t70}wd}^Z8;E7MrY1N5(r}VvM zluJlRw7G|;#_9XH^detUXdL1)Wa#V;lk4JH*C>t0nwXHD)L$Q$>NOSy1<jL&KGwAC zocK8YJ++zV_2%>}7Av)Wao1g6+*LehE>mffHY95VQTk2|n3lIWL8;WGY?Th0dX*Y2 zfO!`OJjZ)CGv{6RG5cW;fM(29#`uy#XzEp3PN`AFAh)blm|H5uxJ*E4{B<l>oSPM+ zHfwq(v60A);qSG&K}_9PTsTJW6n^vk)Z<f>PA*v!lclu+oy%I!*|-_fsiC!Mb!F&{ zHvkdSEW{d+%*JTUFldrFQ_O3>et~Ng8&+lb2AFy6n8M<XJ4{}cCeG@p5`{v0kL`wp zN{t@*a2JSTeM*mQp~|SvPDm}tbXteZnRD};_ZMXM>pNJPzM$;`U9!_$vbdV#askxc zE05z3*EuZ7I<3Z$l%&xbY=$ItOd>v+aWJPH5b$M|d(2*KoJB-t0-&4dl<B}V;ylS| zO(p9BCT9|hH^mxw1DBI1SaNy?2J6*u{Wb9Gq<@XRL2Km=u||U-G1gqPuCyuD?Lk(s zULr6ZkT(-gj8*&0?uztYfoctxA8SM>N{rDYnk;&aHqm8Q^A7;_Xu9{>B&)C@V@q$n z+h7RIFd4OM=~}-3*8J)2xFm~UO}chRvZ42u45iUDz0zE{c9DR#yk;Kn_wBM;RBGF% zz8tsd__F24k1t;)`Opy)R$x%+_(A=i6dD@P?6%RPL?ic7pOtZHrNwk}61UN*-}OQ; z|G8WBcEC3g#*m7Q%fOIS>+?l5fSvFVrm>l=I>4=&ODi<$9KAj%4b2kSY%mR6p^FL3 zD-P6hT;C5WN*0$DZJ&a~2>|Z0I(2$oUB8sq?e=~7sScjEC-x1q+~O*qhYcHw{u67n z2*~4bc2b|6#q$C&x|P)?Lq3X+#Ms0$^wR(+8T_u1Jf@M)`wGtt=0dx|E+Y_0Qk9E2 zSf%Bt#D6w!pE6~8Wa*Ucjg8wQ<4WgkyZ$%OF0#^hcl`dADcO9+!1-&3JuxF`^2Ek! zU(AR@(&-b@2Om7WacTelp4?2j3AfWy%<p?mO<}sz08n*zZpomLGv-$w8CSS$^30L7 zeG8KTJ!f(i6%Oe$I!F?e>~kQ;w?-pW2>WmrWpjbCMTx*ZM`xxYLUg1Ur*5EYYXMjx z*hMhU7YgJ>1BF<Z35T3-0PBp)aW@`f1u=&{L5_@;x&E#_pIiB3EJUG~bV_>d<IS5t zW<_iiE26E)f}bKiFEN?i$R>U5+?v!RS;S9D9Vy2YcEkC<B_D=mPF|mrJ~+oW!h<DD zOeF!Ti!GZ!Mnu(>Z~N_4aG@i^O%lDU)fB1;r1my1A$`FTbMMpuU(@|ICPy?%-!#(6 z#)+FYO^j~sJ$J6-MtDsSCreATEc!@i>=Yn-Wh)bSH3qzip5CZ1@C9UUibU=%*<Y5K z3LL8N#jpxGKa1f)3%yPiqImh5z@(}Rh=re2r3zI|3e*S+;$<T(P!+rBst#NQJx8+y z8)GwF#>*EsQ&7?sWlHESQ&cHTK}bD|V2`6XBwv)BmjjjH<Iljda&ml7&r<xTh$xPN zYpDtRs0fk9E6d2-{}<)qA=*;Z(*Iw|1E-$<yY>N(+u4VlkgFk?L^BcmCtpha?@Ph| zN8bkm(j`&27P_QFyd4Zvst2wI(Nviv^g@+{P&H!qg#~i@kBu*DZLz20@^sHgFInSb zV$#!NViGLuYozv&(r~y2r`d0DPBdqTtr=#~s-Sl$cyRLYaaAz4oq)B>HV>9=ztRJ@ zQ8#cT0)^%xdD~fxGki#DfsP^+3Q6BKA<!fgVj=j#v|w#&YGS647;cps!;#k3@ZP=Q zR?Zz}&GKfnIq^O@uAwX214&&$Q1ME<pKxIyU&;Z~0A~i0mq!Hp7W|R|<^YhxP!QP% z7NvyKksSdC@*k~9F7w#c7_m~gT_J@@ivoju7+Ad*aZrh0dvq<~C@{Uld>8`-Dt!SZ zlERb=IC__W^PT_Na0hZdU`aV2Xe)vi!w3s=G|K1(R7y*2s8OH|NrH{)hzj9NKshYn zNzt=bSJn-ohn+QKJ!=U~q!$u)S5+x{FtSq<MS$8V+CY%T0~!<wZe;l3f3)a)7)8f8 zr&>o8;WiXm#IGH7MHTSl6!L+tTlg^5C3-L2$kF}sK336IXvY@)<M4~%kN&}jsOz27 z0GDM!Gon->pY|Z7h)zmTIz7~DRZw~%IeSUEh@9z^rajEAGZs8vFbeUdjnShe=^c$F zgGS*XWJ<JimyD^zJWsj-B!Pos$BQqGzoKHI-IQ>#C*c%VT}X;~B1Za-x!cjPOV~^4 ziH{>)dxxUy)l6|giz|-s=n%}EUcxuyTq7<*CU+`Y30_Sfvl9<VhdTMuJ4Yi&g#!MN z-GGEhFwwc~D0BcsJ1UkVA67DRdjmKJKO;yt5P@I{FmTZ9BGntD(m*uhHZbi_i=PH> zt8Pzrs<pS?s#QDnE+-$eV+S(KI@{-bI<(*X)q(B3GT0#Qm1#5PuURvH#<aZIiHYkU zo0^q1^|AFc_`zeUN<*Wnz)nh~>~BLRUkOnJuoaQp$%zjXqzG&S6Ixl3^jh!1eVU9& zuH{)=q*70Pa;jQY*c5~O^vd+w#$}DQ=}O_o;sGMB?w1p+;vshr=8LbuA0iz}SjM^~ ztb=&Orj}C=FhH${=v%+Jm=XiYNEry&a0^Th<d!2;Edtzx5SdEN)}zDl<K}MO%uU^l zr03(0NZ;MQ^1UMykKh}ZGXs~n=<$DSl79c7^xgw7Wt?K!8@_*h5o_oPtHu1>BfXyf z>(lt(D>9@PdsBK&`VLQcZ{_XGaO8+IbjSC1HQph;^W?qKA5YG>=PO=$MRnvpr|9O@ zz*~wxnuUKHnMR)Xm*;62(=Td603V?YTlMWwmRj{fNN){Ks%n?H0RgN7#$4CAW|>i- zgN<}q=V4*k<%=h=@@84zN)N+h=vpM%rar1rhp{4G)&M+K>JcRdT?}dI&}1rfuTK4M zO4N(S1AiY16^@#t%Q2&ogR-n57P|CnQHu+7!N7=yGFTvx8bUhhKA>y??NnR@ncx-d z5ko~f*GNoHTZ_#4G^SS=Bs*=gzuBj*ooZ))qn$`aRc>xouCROJjr%t5yK!RmlIgPr z%TS9jd-{^<cE#0>3L(nA5DD>NJhJV3nZuM9q7E;Ww@L>NER{D*cy?}8$CSa#syv>m zWrKA)-+c5*mB*uc^3gYU>aKdUr;allIwu7Kx`4yd<?=v%G8(pMg0+Ub45rlCUaXuC zcrd1+j{qGo@OBTnHncxQ#bWq;bMyIVn}n$DjotgA+(wZbhV5Qj-Jz&nx_g+_>9o?G z(6uLqk#lCz+_};ssr_=5Atmm?h}gr#%f}*plh!}<-R8~TJ+wYa<w^IbuH22%PjnLW zk8B_+z{CH4Rl`i?u^}K3q0^i{74`eYYPEJKH2gQ3{)=}G{e0-o#r^;DnpBmSwe{Qh z_tWc>lh>dA`$nR_MEft7onoo}H(#f-?1*zj(cxMDOJ4*<ujj|qmli)Z>+@NU;S2t! z-{9Os4|N!Jy_}Kp@~$iU)4=~_iBqraPfC@Cut5Hc&UF1e?##UF(XIaTO8lfF74F$n zNImL`?_h*=dobwXk4Q=o4#_!czsI0fA<MD#M`y_|0;EA622=U?l-~KZMb~6yW@cSe zRGZ&BC7u^Lu6A_l<-0-_Lzwok2nA=4X$aM1R|3h|RheBAY8W#+c(r5XE7Nro>d?iX zC@_o9#dnddy+pL-V29`iXdqPPkfAXtkqjNQ(vmKLWf+%`TXy%RpThV+J86L%RRp#X zoy1s_v=%@m47R+Ohj8Q$<>ge#i&R$ZM_w6-#oGB=`DlUPpux$?0#QA>vb3tt?34ue z^qu+z%BI>#c=UYfwV}JF=|ts@$wfJXgfPG%Cg$}+WMrM|K3cctrb_SnD@g2(>y^eH zPV4mp9d=)rUa97)a>8p0hlwm)kW!qlx@r0kg{9Ka*xcHt<)c~p;F+z{cCpDD?E`46 zQTr&Aji3|xKw?*rVpx`wv5tfKmYRtghgt^B0+~aO5+U)l>&ou7K>Qf;Z17Q*%uo0d zB%Y8<Ftx_f303Z@2~~%Dp+F6};)b{fQ;JnXMI58!?uWN8O}09x?V59odZ^*Lk&)WI z3(}-;;eols3wBL&T9d_aRZgUdGf{YG<l9S%*9&U`nJJQ@Fe`=acPFs#<hapqyDs2% z=R7yCOxQKetlT+AGNflp^GD1M=5mVuW9A-6VGm8)C6rP9WgbSh^}@RWUXoKOk1;TJ z{P>upW`Ps9>@to48Lba+qh(Q0B`SI1KdIXk1j!&HcNvu^WAxIYa>je34d`$pGf@<C zBuGX|ILP5FH73xADfm4aWdzH*Y@<+#LjLJv70&X(qSQY3z=64ovxba6FgGWgTZWn7 z!2B`N{S$6Fe9V<rl>^`4QTY`tL|f8FiIz;0siMG!tc|X;FCr^q9f6u`FK39z5-I2W zGH22JQG;1sW-(L*uWe7Gb}ua&kmHkH3Gd1eh_2-Wd|KE7&54_8=N>Ts{lMJF^oAYw zdMEedz#)d9C#On#NLyQQNr8>cdUd?r>nI3mnhinTd_i3kNUt)y6hfHK+!rb`XLcy8 z^|}FB+--rHb)J0b-JJ63oHyR6&QgyIWDGKcVs`dDSsqN2@$t};Fbq3+!ZPOVW>)AU z&<8<Rf;tObOID2##w>;!Bt^NC!dKgaF-b;YxeH>%$|KqdyGQ3{v9P{uVH($WMN_SW zgf7ybA|KT@-LsP2nGqQ^eV@9rsaDxCG4dOKsG|}AS0=NzFqsc^v|w93D4Pq9PcIQe zTHtjKsG5YaoNv;zvREXjU>Ma(MM-|gKW=|XIsywr?dhAEYTYaE32&P=VwStM>0%3; zc4R%TFY?8^Q*&&|J~vV`8nSwqq#KPbN#03S?s%W-s6Hp*d0Bxak4f3rumBjWpjkdY z1wG3Pvd0klNdQw!YdN5n?}Q{le7-W3C-3xBOn=d_YwfX#218sw#xg>hWYVVsUPC;L zT~RuS+c3n7eC*X>tF1Hi;xg6RiRMjX>o(fzX4y8@U9-h7VU_AyZP1aIk{>tcKxu&_ z_OH+Pm1*u=zeiK%%M0<F<ELrLeYsWqGL@j7t}gO$k7lH>_L7<+4As{|gLom7>o3zR zi$B0uTvAM~VS7povmNZi1lPpv+WPskMoM?G`$o=MI#zqb#Mo3xp~^J5bh?}8lsEaL z&4tQvo-Z<n8|TZb$~EdEAF_kq!m1oYjFghN%q1;y_ljrNX7(!@9RK;B1s227^%h-j z|2&`M;+hPGy#Df7SI9oYg)#yO<(;4z;cAqQCIFm~`TH*7ADsm-YW+m|WwJ)3t!pCq zDdDq%lzKXUq`SMNS5?uFDDgXo=zs~a@au?95krbGSqv$jx8I?iHYB%b@xX+nNgER< zm}XZ+3ZjH?|D1t+4*b>4-1J|>d>|>L@GHebsbv*~h!tpRocdm`z9s2pG!KNv1xM5b z8oA!V5#hu0KHvt}$EvnXdT-eRX?JL3lnl<fC6!B^djQUP;O4u2ooz&XqgO#$Y^Nx| zk`WrEJl9357(n1y)P*UTh$7KLY6PHJ8f7R#xf3~&xJaHHK_2C)1O3Y8wA#r(KG-#V zM0+WdF8~+>9*@3`Xn+9jA>v4Ji5SG9x^M0-XT5z#LuC5g1AjLkm|MFk(F{VBU>~sj zNl(x)WMHtM7PP7A0f*NfuhwtYR^{MuvnJGDslG5Xv*HC%rJB%7hN^VvZ4<Kw=<2Pz z%!4ulBh^{eZ@iIHU7jPoo9|gy6W~`nD~_F>G(oz5%=`mjy18Z9Idcz;ACk402(i>I z4i2WdjvcPZXQOQKIaS+Crc6ts^bu{Rxmcsc2CVE^j@ZbG0gH0J<ea1*g~G_-&8}(t zTX80<8$4>f^olQMKv5~pdTHCG*8;MB7-JsBf`?)9kAvn&##OnR=MDl*tWXA0yo6sz zxLzq($%%cS5Cm`)MIjJG5yNCn9)|oi@Y;FDqTdFuoj>TUKy``JTLr@~rqSxR##mU+ z(`x%Fo90Y5v&3xEYc<2MzR{-nK&$2T!iO5$F1>|sU9Puuye;3HWzjD;SghKP3cXHi zj^Tz%V-bvbZ{(pEvsP>1pN%nFBNt*5RH+&SeVM6Bs8A=4r3R7By`ymm1QHHes~AO< z>*D80ff5Y@0gVSzLUbN<z{qihkpv2bZUYG!Oax0`&**FKs2WuCIudkP58Bkv9Y#n2 z@r=wDF0Y?jKUkS9y)AwF{PN|`GZV6InB=YZuVRJZD;qo(=`TaxlXjR5CNo<g-9gVG z7@XMT+h$JKm!{>5mp?Ck`=jScHSi*T_}d$A{FV*vGNbgYcQ$B^oau_eN)K(2--ihb z97gvLas)}S<<Ehtgsk$EoA^Jt>?ck0Bl{6I@z&V}9WabcIzcen5?o&E(5a0>yaP-o zozbKY=#9K7D=;ei=HEWY$KX<Jw%Ys)m+pK1IscC;k7h-@%6vAI>MuRq-4eO8EtXMw zfzu-|k<k#Nnm%mkv_ykJ$GDi`Hmd+sY}>QD_dY{c!Ib_BR|)x7X?AA6;)T(sC!Qj7 zsa4e?x@Dgdg+_3y{2CV2@cy7v1Lsi{<64Q>MH;#06ODr;H*0-X`j~6xnj?+aXRVU^ zS>|b!!dxpUR_TO%868fhi#ji(+dgSzVd~?uyejLB$dAPj(up@Y;fv!8`ZZ$E9|U48 zBKxoGy4>r?L-1uoOQZB9bEc17FZJfL*b7o`WC3vED050*rjO-^UZs+cB1+BK@C+`Y z8^gGzioJka{|AqI29Lvy4S>-5X{RJz^#{<`rJ-%Cuq#BfYz_dD(|83cLe7F+y|T-y z3aoeHTMLSz&_nmc7Uc_&4XzGcBX1!(o<cjI_5*KGz<eIJ3O>SixC(c9@>)F*#KD=7 zHjq3zAes}YPlIBKd_p{O@^fwn9BG1ZTMr5wgTsTt;T`_P&5QA0*s!>E#FE9$9RrRn zU3Tow&yNWkk1bnz3_BekOaJrCb#Jd-`}TFu@b^j*;tZtaZ{Iq8?EZ7yNa;IdK}AXh zwoYK{v&uCK4@nmeZ~3A&ca*N)UHj#h!_tLA3pM3gY{7nZ+n-w54O~L>^+Ar_UOb83 zxp*;?%g`df_!#^A*s;%#N$G4IGp;?~c7Cm(TeNWep|_VWee>WXcs}DWJ_BAW2!-nl zZ+Y@I>B6l|(@L&&toBY@d@EDm_T()%K7DZ$`pir?;2pv|tHHN`zp%m$?`kX%k|mP? za?XKA5aldafi0F1k>M001GOU0F?k*3AmthPA-Mqa2NFUKM0{UqyYvIo0=Y*k9e8}x zrpGt2EWMyl&-O2UX)x2dTrtUGlKZ_ReV;rAo5@T!=+!0u>~vhBP0I^;L|fIMrqc0u zd3~NxUK+O?8K%$RNk5!<iXGlGal)oe6DCIPPRLH{<%t_?A0$6T?{f8r|FO%Zca5Rv z*$2mYdZlG2*jxrnykJO;cOd6^yv5*>=Yp{8H>LsxT)FJ6+G)LqtOZ3HoNIFBE%H1< zE>)G1l4M~<#V(e}-Nh0A%b9#`gygz^qCUQT;^v7HH?u-*TAyUCZ|%kv2?@!4(zK5B zeswn$-k9%jXdGpZXO;}ZQsZzuQ?zSzzx07;rGK71i-bUHdP1GTa}Q6N82P~#E5@l~ z)6*=LI5F0i-6tzxD7rDP^8rhTMjv^$$Pmct1FyB1v-C9fMMr4mJ@>5STd>5JC4N4v zd|V8}kB@x#WC2n}V+4RVq(DeDmpO8cjPEH6-O8lOaoazWo_*j!>DkY>PY7|(=BBcn zy#w+g`#&u`otl$BAdT(!h~e>-k&6#XEuU}O_BjhZ$f-gT+TZmMz+(OYkMs&F_6*1` zOp(@-PKTi^2SEd7QJ)hLSp-uBq8Jf;kqSgGkKF()Jq0qWLG6j&77*=G2QIi}`H(?8 z007oP90I<W#mS%?42blZ6e{r@>Ag7V`$`rVB^@7QAHOV%aRdD$i%jwCy6oil9oBb} ze8)J}x1ZfJ-@ULRw*O=nI=|0azQl80|Cx$CVHnsap1sD{j`GNNo>|;u`H@Ro;BfLR zZ+oR+=@`+cF5nV-r}pXCJ-v(_&hWEO0|U4MmdoYjRR6vIJNtwAoGMMpSUy)?AXR&i z`k24y%QwKElgkozwTEh=e638QwXo?d0av@X2gM`F6Cuv5T=3ddXbL1vfNQWy)_;)S zaEhN2%n^+v+9k_NMpAGD36>WUQ!WNyki6b8bAuJ8)F;pYK-_|KZ*x>&V467c@aW0R zT*1ijk9gwZeJKUt4JK)pZ{0DOmyW4cZQePFyJ0q;7$@la4Eb=A34DW+nFbAc@qQL- z)nkxwi;pG`(CWngh6S7_LD0w9Y{ObN8#z6$GY+hH?E!y`&b#Q=<Pxn;C^gite@A*( zdQy7#o%QSAVd+SooBqznSD9eAP4PY|j;D?rpkA<c<9KD{jB4Svo5uHj`6OpC>a{6N zN8J7J$o|GToYy7jlhXN`Pc|C?BY@Wq>UZvb<}k%5tuZl8hg`T$tkN$i(da`pA8m}` zs0#W)f018~Vq7i|x8W*NmP|8P=iKU0q!2m|Bg>lChtE}<reO{t$onvpn7l{@2#Vol z#hgElFj+85@z;I(`!7~S;+v&^7D``**+K}7BL^|Zq0irz?s`%A+ur>2b2oi1{gdr) z(9Mua+D@NtJFQf3Yqoyl*WA6Aow)seX?|qRO*bb=WuA*{{Rd1JJRm(IeHf|RV&E2S zVihZtxZ`vijVr`aLXY&aY)x=0fC&o08i-!Ri_;i_M<`J^mD8_;F|eF$2Z*Z2Jm`0^ za##n^uh3smc0plva0Vvu+oaE=0rPuXst?Z6>6Yj-zFt<mNflR9Sg$tf;zZ}T65|ew zy|A`d;iSRdvQiDiCr7XNzV}==<aG1?c?p^7e%WKECm8o!&p=WHuKtd9r7tfjSoVcY zY&09dGABxZG4SS*OStUc^;LsMWa7wFl`~KKbid4DCGdVM^m9V*G)`sFtNF<zb!C~! zgJ%ZDzk15&s~dQdjaxKKnkPNK{*vN@`=)i$IMg5WSt>003L;_x`E0@@3UE#g1_BKN z3@gEV19lb(NCgH!a~fL3Ky>B&G;EOG`26wb4ohFnthq)IuBn;HY=@sazFK<yo+QHq z(r-WAN<8+X{0WxN?mV=8_KNFQ9D4DELo2tfoVV>3F>&GE^%L86W$bF3xPI@#`Ky@v z=5JX4(~lBw%2sw<Xp!z+S24PTP5t!GY-(v;<#p11C^yL`CVaPV;pGv4ourzmg33cm zJpe%9GoXMdB>7qdEnX#WQ9wEY`kV~?<KT%ag3^whb8(04;L1EXq<R5SY~S$5o2T7h z!0s;jOgeYsse3-X>+5Xugcq6Z@qbhxwP>8nsJQe{Xm)*G&5Y`~qv!8k{px_ii!V$W zv-FlVkL65d7r1xDcW>JL2X1Uh-rnaYj=ue$Tk4iE)zap^_psSNj6iw|3!BWA#|NiY zEj#%rd$4Y5b?!ZjwzaPvGqG;aM_XU#hTM4eEUFlte^g=2KSn~={;@|`)T(LkG6r^Q z-2&K>XD6IdDXjX7FhGLpz)T4<WW>!HNj&O+cm!dqG2$kVCnb!N%+1RecHlxQ|9S@w z!AmJbmtlch`4-uNN#$~2Ui>S{<s{3owUBIS)L-Pk&bK2$j;s^RWW~G)m4cGu1akmD ziE&koQRVgB@O*Vi8a#4iEmnambd^q)zRm03U-*I*VU_pL+a@WKvt<kmUOUvCQI?Ww z*II1#eY1y^79uU%+ptuH>PuE^nRjIJHCD|x<L{l_KTnLM8{59f>;D#;HY0mTb$(2I zRYL!>$Bw-;+}A6lkI^}E^WD=QpthBB*NCfSeMz<AUd8j6Rf%z-&K~`P8#lbmqm@Dz zAz6=hAR&-f4+U<@OUxz|y%NMYpwJa~Hd9hw0}1;03#5`GI$X}poe@`LbigZu#T1;5 zzDhYe2|UJbMn9y<D!L0%lUl3uGuoh1fS)<N10?`nC#YmsQR!xS9-YL?S1T9qJ#_5& z!J8K?F3*S&iV8($Ni3K-HN1KEO<TfKlv-nqJx#J#RoD~FMy;m2O4J%noJCz#WpY{c zN>yd0#g)Kb%*h^E`_6ao)Q-wDGEGr|*4vly)8^c~?~OP2_AX8|njjPUbhCF48aR92 zz|g|YjSp=dyldx+FYOG(a%$xNwI|!n`~sJ&<2*}Wo3mie>UU~KX6Gbpbh>!GMm2Xv z_~tDe5-cEn`i=M8dGLCja&dVmRMFJ5ch;ChwK|dU;|8pqIkmW?B#06Vyw%H%l1r>D zs}fC|(V)^+R+*A4VpXNtl`v$*!Z{;rCrqdvHQS>~Fq;ym^=Eb5_QqM~_U?Pbq$?;? z^Stt=Su?5!)(&crru7@V^})$6?Ap0AkisGTxmt7@xf4d`LMbU@v^8f!?Z`Pz><zWe zq>opP&nU^)=EmtwLTRWs^_e8tTs}dcNkG3}MjAG6F#<;oAT~La7Py=kUbw~=dogF= zk6>!R?E_ZLz-MrnDde~Z!t4Vql<Z;x2YlH~FT%b+UmgR+D+eF~q#9~Bs2GB<X&HE> z(daPh%QxKm@rsq-JbZk5ids-=^<v9>wuK!!%a9$=mQrZ8XzaOWm@MM6teH${P-|f8 zfd8*@Zb8mkX>)?tXVCvSeYn-CGx%0+-@R#ec}c@{t9DK+u&0bw+WQvuwMg%0jazqm z=JY$JRK`UbtE&c&b{YE2UQpRrsZ6q(f+PFomycgQv6sdOggjw+{)1!E-!je1uj^&d zTC;C;s5Cr)iK5A3InI=)RK>7+lB)_bbh=jWFq=*1=rcB5nOAqy_|ZEj4(^qx;nr8W z1DwM<DwxoB#r#qE1%oG!NQoRBlbhv;o0?s62WQI#szH<ol=Z)<po8S0H4oG<N?y9$ zQ2LNYvokigd&>(YB>C537(sJ|+!H_AXVCJJHXb@sXt6LfNtIPb%1p9ZbU)Irl#?Mx z6N7^g60wY~F2QKoMIj?SwuNvT94%UjcDBk_^w<;?LyIo^uQU?*ZR}h|ku{=TsXeya zEEIakg?{`b`Jq>|j}bB{wGnx+b(%M2>kDQA2FIme#QyBz*VA45C}v@_Y0*|f7>*$= zR5LDw+)xS;RRvgDcQf#c%i9djOjl{OaM4iKjGLnuM&1$>EkCKVL9YMst2Y#hK$!m( zoqfU&&PDDM-pe3s6vurzlAe&!NEAngqW`mY7)ufOXU;@p%%6Tb8g<^af98y)!~Nei z%`FJbzslp}fPZ?t)cXIey=;)9(t#QRtXO#U6KE2eiW*2>{NFW@=#&)5IwQ44Tjm26 zZL0Rh|E^iMzLEl<%kF4<<7x6^BfbBN#voZb%JU|5(h(B=z^!zyFhzH<P^<-km$7DN z*U}e%F-*35C4w<Kwy^sR@LrFu+M~y@jOrBunnC%1<g6o#f?&6voa+QX@XD5wZk16n zJG;4D4*ML<POvPfYa67uIRy=pz2_{6mU~!>F|wFm&D|vAM^8g7eqt!jo!d*7tt6EN z-tEP>_@g{Wc`42!s)FjSkf)nCf*;0M=v3cdrlwF~Q-3HVmtN(YTJ5gH^tKlHy`gAS zsv<qH;|xRnQB^)fXWClQ6L-m1atfvp{(!H(D-q^ZMZ8ZwA4+WhK=$#E=UbuxVYr;G zX=#!9c*1sU$DA7V#6^k8mJE}q-z&<@5^+JWd^$yMv1k^}Ui>kvRi7q0ERk?*Y~*0% zpw?hDW0%7&H=CR7Zja?c?Tt{jw?<mU?`@v8u+6PPz@qMlA4HuEx)<y+c~^g^UHiH4 zgT7t+p729kMEK#ob|_pg>xRvssDZBeh77ebca8FZsFLHv6-T-Z;WVtM*qlOdHA`-l z8Y|YS627=%xBY}#$tf&Wy;=z*9jg+|dRxe*hJw+Gx!tBlWB&9Ae@UUWwt-3K88$@l z?DXA99&$q-qR15^_;PZH?bHExWmM@}L!&KAM<Q`3U>(a<kbel9+bwXE3#Dc*i(;I1 zufCP0|8wDt`=z#`lo%T?#EM>n#~5!gihJ+=mfgm_V7GDdeYo}Vf0lzJb?@D4xxYjU z@EV=bA$knn_`JM+{&A6;PBH(z_folKI^Lt)IW%|u7{OHN)Hags1bP`TPe2O?)G}D+ zG{E~oAnmFU>8S(0Vjm>)auK>PctA4L%f+r*voEFD(vdfB+Bh~LHs|2AnWY2DUSreV ze3Ol&3Rl;>AhqRJipE%h7ZFq&!>RJ@y<%OuBad7*8F7#FsByIREWG2Z>ziI3QqVYl zWW{`+QoZ9VX8B6maSDy0exRR04LT#31S8l&b--DYGbsHUraZ9m>-%QRxbJKEJ8A@l z_%HN8CA`%2M5Td2ZDw&uBY`ys@e3woc}d$qF7-!FOYib4Bd1xqaFn*W5z>2f6fMaV zqb{{5?-xUI9J-Q0;m`YcXv$Q65-5Vj4yT3Mkv4JAB07}!Yo)W&uRptSYF5Lbddq@g zu_tnFtDn5gndJyp7S5WX)~_iItzvcUeA`#j6lo+=HM1(F96Hs0OZp9J&4wM)Cu1)D z>R0tU;@R~&HGSi#9#sK(kte@<as+3*)q@TUyk~___Th(-tsj2K$}lk~*l}(p>m~gm za=r8h-AnyCs(S`w0bj8C&ii4faRy<hL4tB1U&Fyo6(l!h+2jwT8KNLY28Vk{o#e3t z+>jLFq+#4(I0o)6VD><g>%5N2!S9TzNsgO0FD|(zW^%wCkPf)x*s0X2LHS!YHx9LF z^@CZ<ex-=exo3QTQ3?VdlfbPgR*9VSwX@C{W2mO$3oV|yC2^&taZBnvjg80Lx%Oa< z*Rb@uL1DGD(GbH_4Py)lm=uuKWm8zG<ZBZWl5`!v83R#>k5O{!84i_Ay3wHFG=NN? zx=)vNGr92N8wqO<*?OV|8N`ptMi`KD@@4SChU^rf<c%uO0?0(;q&W@hjm~)nT6kat zcM7JE84;ArK`ZPd5fp-l(xy`MDTp~Q?Urh#J?C|W<;&{o`@YY5E#qSiY_VlfW_d%y z+Og6@vsu<3d)L=3YrA9Y+J=VmvIZXPz1F4c)!f|NOXt$+beXqIYM6A3W9`_6vhoJ* zq280@atdIyxfOPl!@0R}Q+khDUU5*%Y4j$3q&FK+8CyO?O4r;{wp{v){E`N#QRmWW zwHfK@8CtD2S(jO$(G*;umyKPEKEV_NE^*4IFvsPQ_Mx1IUPmBL2agMiyP#Pa1J(~} za0?W8Do=5akP8WF05jF^Im_71e_;z3&2B%;=A8KbA6_0jLHb&{W7pH)bNr{j&+BXA zu2s9sM~v$4aBREy!XfUnb$@wv|Cryu@YJO<i;j(Q$M-9gwgrc9#e-SnZ@*)sr}o`6 zY1q19CGmit6h~~lrK`WnzV%Tw`UHgnJq^8)W?<@&aC-`1wvBStg&GB-upm!R5C?^+ zC?N^_3&-4CW270P0ZXCS4FS?dYikwU$D{L-T=!(u9`m~wsz`{RAmEggz=3>pX;9%s z71kh+VDS{59tlUCd@6#4pa+BZfimy?A>Z%XcVTz^o);Hx`f}(W7D~6j@+;~6x7V$E zoB4iqo-LL_+#}0iDF5csE=&2NNOp1jy4(GY+uhkQ+Uy?|t-4|Ng}n=3+*7}L{&n}X ztb1E}AJhYnc!#T&nj;b{_Fd+6>H9CGWz7shBqizS+ivhFt@wt7)zXPa5cDv=8KD?v zAUZQ~U*ymPer($#j|;ck_C>y86Qr1qd)Rb<>TbNH<D+H#xJX>%<c_=82yT7HM;w|$ zp0kx<=mg20dmTE8oRJ*T9i$hetbueF-4^jhqK=SQ<6IJg3_I6>?lmlQg=RALW16?A z>@=F7uPMaEvi%gq(q2&P;&AWfd+;noWBots-UB?2>gpTcduL{QlXkVMu2oz0w%T14 z+p?PFZp*z}bycit6*r0n#x`K8u^pO?3B83-LJh<~0)&JTLJK6s7*a?=38`Rf{Qb_% z$d(Psn|$x{J^$x#YiI7<xoz$}_uO;OJw<uuW6owL{&1M8-N(mZ%VM0OPyHmT!O487 z`Ki7Ff6RPm)Kr7BAt8FUD?7lP2CH-`9hKue>OB27?qt;@uqGejpF5p{d=MAqr#Fzo z?`}uB*XQ%5JEEZL?tI;0b69aK116lB$mtxvY7i#=08co^1YX{Nz5*jdCAX%rRGdvp z$_5ZJ9SV*l=%tNup#*+LI{2$tXbJOxvjwhIS(SbYm>+mlx+V*J3=vB-(VAW(+9w|| z8chc0iQ6*^olz;?6kk*`c#p~sP(EUhZuV8?7ba#!yS$0{1+ntAo=aDf(9X(BJzcQ{ z`H5avbXH<vex<J`rnQPbx~es%Ckjp*Q=*)Onc`}nlfP&58;?k$IeM(JCo`kUm*H$C zl9&pID8D-)?4OsF6RMb<wzL=0N4-naW><uAvgY~23GRxS7?qh7&J15wMrKdt7;FYm zx16T`4DczDiWs7lF>!P-Crlb$6gpEfKs<k8`S^G_<B#r;7d||GA>aKCXEZ|9-~wio z|G~t^U@y+by1(J@gz)|^FfLh;NvOoRL<>d-!fV7;1n-cHT)?{~f>;W$p;hfptB&!) zW!m0_jAsBV>Tp`&1wT^D=FIXdEUFCWsVHJQDO7;IuRdgO8ggQ-)|5oEciZdd>^c_i zZS>?<d*^o3aEP9T1K0idisOH{>+=`)SFx(+{>avNN3Q#-#hVig#l`5EGo!7+>Cr7r zx67O3b;aAFdwZj8@$psB?2#!=F$G1jiGsNzdFHHheztAz*2D$g>U_`K{cr3aSa8LQ zpWSucN1n$%lArrs+>=}Hzbe%hH9fwI@viu)3|ssa^>XYBX}0L9_*<WS?%VOxqoS<| zpeZuozvh>~A0}Nt$Vj3PmAMLZh(kbpaUo<Rs|48JR>X5thz%5kMGrcDrx!qhctbY6 z(sNm%sAzoQoDjym1aGoY`sMi#Z{Pm#`5zD8kh=HdzQ@jKh3R5bV!@IPi}MqV-o)Ol z?BN5^1>yDUW+ysEuIS9kS+nbfZChTvV6{IvFPtC6^{)6}Mq#4cu`)BWzAe}6uRnjq zyz|!0E>3fqxoy?<?W<OA-~I!~{hQ`gFUYN1zI1V^-Bnamnvq^KCCi>xl#t9>$Kv>c ze1D)I&1NWDJ#@+X1y}88sR%CK&|O+MJ1@y>j`oLFgq<$NsupC%`oqOjlHw}D)nyIg z**Gj9_*Lm9RexP~_UQrff-tKUDQ3)aMdwRVN~dkWk!W~!r@6y$WoJH(ou%5%nu!rK znJJ`&*-3f5>giV1Kc7U)sq!{BZ-O@cDQ$S2uZlSf!3knc5BWI3_KCPoM4}P;IpdiZ zovG8#4zcX7_U`>keg{|fDYZwL`zohO2})--{P=hFeswC>0+pZj_0K>XPt&jD(eP_M z2|S>x^P}g)>d7UrBmb_izScjd$4rw)`d7VEruN1u<os`Yqn|oM9isoD3-H;KaG(V; zM-aQuB)LC~1vZT?1_B-^KNb5iQocBK4|u?JrOBwRuv!?V&Gf%Y7hJBSFUOK_zLc(C zxBSbc%Y}?MDClpPt9a~Wy0{E^2MnSEN8?K7$HwFr5<M|7o<vq5%sla>V2DjsWa2fC zo2fUS1e1YS4TPa4!Z&^Jfewg4(^-ze{=Ep4(rnVR13VEPpHOxn3x6cW0XDr*2#QD% zv!<D$&Wk|!jIaxby;xsyeh)%`LFk&~Hj$myNLmUNk?lMp4+vZdKPzo@3>#+^9@iDl zG7dXPu9QXM)47l51nHU?#}4CL@dw=s_1^4*Oh*phrN>Kgna9sxcTvQ3+3Gt~dG$M1 zU*?Kjw9Yc401;##{f>ee0`=hdhQg^+3;6*APaNeCsXiQ^F6O|Lc3fID!ssNqS?Q|N z;TXi{i0Skqho_0}%I)m&l>?M$V5K~h-I!la;c~!#DsaiKK_>{XGY=10=>i>o!Q}={ zoXC`0sz97`f{OH0A%YTxkK{TXqWO%|Goe%wa-|TJApE*ot`_8S1I%SsvoeR-ES5|0 z^5csPu}7U|ldwQW=mQ*9A@pOqAtjqxO<^S^o4LpkcT|0UDn#X&h#iHa^M4+VJ*l(W z?MGwf$FRIPS^2~r4@YB}`i{+_ck+u9cdM1=fT-)iI<p_(MEaUhm<KvYheUbd@j!?Y zNLG+x{wvy+FzBIJ*%W5)+rNL`4)&V-3Jb`^C)crR`N%rmN&af}z}?UM{h7N5swV>M z!+raO%l7X((ZXJ10sMb${GjgSI*2O#02$aI5a<g(KDVUg+#j<FAiN4Stj^7B*Z_c) zHmzN{%(5(a>vIvOfCMLT<4f<uKObu!|Bd{1FB_3B>t#7SVdK5`vi^JT9sjd@<M>DX z1^Jy`Hp)hO!8Lec{3Cqh#JZvKk#eA4q&vkq(l|;wr(Ut<=OXSGota=O$`oWRYHx7J z(KT;g*EoLo6X$)PS|q%{cKoQz2MDx@KIJ~%tiAaurJE-x$>+%_69x>AxTC)si}%O7 zqb1y))S}S=l1?}|Q$H>}j+t(TyrLIAzu*rBQfOta90(K<dI|>^Y%gGpN+|5@5@Ju> z2%{ho_6px8KQjLL^K#&MV?Zj77;unrqY$e+8ilG8Ccep*7sG-lO!_tBH}ZDx_)ht! zF?qJ}OND>n$*aJH%5OW0IYFl`=p}3f(wU+|o&~b2EI?NGa2Sl;1GrNl-_n$wS_b+G z{YBiiXf}5EurQ-*&+adq*~)+JyFkuXY#WTVt&+zd+xAMOYo4p}m2Hp7<Ez>}X9wAD z*}>2Gk)z{ptj*x8X>N043uEUUJ@Vvj9orAS-@THtmEG?j+}?59ljKkyD-Xem>C|{m z?6X|p{^w~r-_VmF&t|kQJ@o_j%Y#dK0}+^5dp$%Pu(DJMf0I^XLV8>{0na#J$oH^i zB$hkgEM!@YK6%&cugkl9Myu5*zGK9e?QwYn-}5V6jxDb`o?W$kd6oE1)pEXZY)p4@ z`*xYEAL!KZiCZbhN!>m7U``s3XQK>p{ec4q+^4gVB}rP3v1tVCr_icIqS^Fck0W(R z>p-lM&P^$XvqFhy`K*WsCqN$qznC!e#D%f0@;$GmWvnu1WmQF1hVo5fe&fjSHFK|n z`;buL{GZB;=WSdvrLu5t7N*fNEcEfEi<2e0&Bp4w<llMrbltoE3I5s-7r!*(ipq8- zr{qO0TE8^Pxb)JyFC6vN>V>q7m`cq2^QT^T@Y-KK&jJ_E8hqf+-`xG-=A}!$aLSm( zW8tO)AENO-@f~DMgX~Up;_C{TLG<CMRyj+ybKqA{`K2UASyl?xCxSRthPWj}Z$RT~ zGg3TjF9OP8vdo(Q?7fQAZY_w1!tC$m<M9R7g!pIsctU)l$?S<U+4G&oa8zzo*yzl+ zo8mlXQ(}Aq?|U|0epF#_@;w7jA?Gpq_)`M|PeI8UZULVHkKjf5qgS?Vdxa&ka+aus zUi_y0&$3{GzN$)}5G>FaS`WRyYGzDav02P<@7c0tk2^;+7stiST=o7TYoY!Yg|)iz zteU9K-fgeQADva9T>K3?DWYNOfxn4YM14F9{fkv+VjtzA$!W+^IbgV#0qpgVQBjQj zQU5zwCS+TQ1>lCLr?RU6PXPf?J<_@LQocAXM=#`82<car?Ym^n%UAE}xMa(%%((x1 zsVgfb-)Hgl$?W$T)~LLf*>KLjuC9IEC*Iw#de7dc_8s3lvS;ec{O=7#<G;u)XF^>* zyU)0B`#U#Y64`b2D{C(uN?`dbZcdhJS0=sbHAKt5i7BcJ{NBy(>Y`%4dV1QPk-cB- z`~JQ?EBmf~8DB+v#tC|#By?9}UYt76RtaeaqX3X(QxCh9BW{=rQ0!We3<>QBNr+bw zGT}Zr!%F79DyU`B`gV%G6$UjI#fQnVQu4Gszc0zFM8zbOrX+>(R|Lzml1fcZi?P=% z8n%6S!F!*|CqB8SqvM`Wn5f*@)n^mMjVMelmK_T;Rwl<p!<eOy;_B0<WTZQ7aTzJO zQxe?Mt@_1=c=M5mSRIO+T&(=)Sz8VCN7hL}T7_{pK~`g-t{Htq>y*OH0f`2Q>_W(x z182D4#S{OPeRTp!_b77?n?ynJQO@YNfow2h>XGCRq&U+3S#TW-$e{;6^N?szh<#^l z?b@+5?6RqKcKK?^ga`)9Hg<DeoWm9fCB+H+IcZrsI23M!!Icta5et9rT|L*w?lgd5 zW03pu*2KnU=q*L*RoP@~?OfrS6v1S1>xbl@2#{Z~h(BIaQ@v(Qb<KEsARSV#k5~;{ zFxvLWhvc#&x$*k=INQ{OD87mvzV-ds3|RT)g4G@C;4F`wmzF*L54eM3w%a!zEiYT= zYe?`|eJ-boHOcpt7$q)Qa?JO_LMN;onuuX<z^wVuq!SZ?X)U0hOJMsf0w0-J;RGla zs5(H*H6bpMO0*zcGSQ(S<O7__6YbXU2$&NL>0~}L2nm_eWFh50i1D(2-ou2Ik>+r4 zP4D=#%w>Pa?vj61W{#Hs7UQz<d{Z==XLk!zj=vy(VRyUj%=#j`mN^oq<(2v?;$azu z+2hOoc{x={QT%blSDlWJn}~-thLXNqO+OLOcrr8HzG8!X?ht!v+)AZaEplW;@xR&K z&&%ik5_9OUcDLMa$>?d>oL8{9drd-uF=@@(9aD<7bgqhz|1aZ}c?%Al^aV7m)?$YO znIZ|y9TJxFV*w_{4J-k|OBgJBV2?q_pQKR1v#0lvy94afhMB~|=)bZ$xPY^WNra4` zd%)P!dq9mN3Jf46296b!2yD1fjuM4!xPf=agR(HfUS@`OeQcUdZ<dTKGm{x5v)+~# zyWX4OFqw>uXT-1Yxv{UPSU5c?MK6^2{UzlI(?<lMIxWuaTX9po-<6q3KL6&<jd?5# zKWl@7^mO3BE%dOurDZie_Kl46U1wRp#UvIa80{8|6yr(~$6(b7E+Hk0#coU}5KUXw zTRhXO#^|V`Wo30%L(jYN7w`6L%UYAcShUePO}GQ<b71Xek;}rlZ_IUlkNnIHv8h88 zaX?;q;6;j9^9QE;!y1a}!;<$?`S;igf}=+l;~4feDy^FWejBY4)3FwlKdmOL#{;nW zw_UeecO^Xf5kaMD)dVxNJJ;@K$DjRtDjjWy(ccp%nrEMMPCWll_kWe2an*A1x0Ty^ z<;v9urryTScv4e*=~JvM^Fie%tG6?sBsFqey=hP1m{QBfn$@zED_^hNWc8+AwYW1L zPS_QD&`L7~*;QYEIIhMC>P>t4ri5w{D*da|pTIgmV@wv|=fNseH+=qH22wy9jj(oy zGjj&*C}o7y)e<a1%t4HpBC4rPJsoNL5Q{_gsxUDY(6FL7nzq_&Fg+AWKqCLacO)aY z5YZ&rK#~ESh*0<azg2esOPFpocC7a)5By_qFe5!B$?y08Q_3H|;efX^BP}~CLv^Wu zG!woX#+CG}0XJy-T!h;zEaMe+sY1`WQVsPt!qRP2Fy>K~X^M%nSo580U-lTB&S1<R zZ&#QO*Q>0Df|I({Ot)Ko&`oJuS(KCRud2;~jd5^gHdM4ME6yqmwv?$}RH#jwV~F>Z zEY%c4CLZYy1CLh{Y3Ff0IEsqUfJ=5Nq~51D;1RWJa=4IZFpgt<NY0{D_g#gaSGe0` z`W^TTRBmPvQYq+G;5v}^BsL3Pje`H^Kn7TkdX$t;feQ1m9e^AdI#}_N>4Hj37@l~L zRbg{0f|<yK#NcM(j2P~G)#1Qv_#a0`jFl2rndUTPK&pg_)fc4-3m&W=XmCk{lttr| zxTvbk(A*4jT5)DoR2(0k2u;!6$jhVVrB}@=Ef#FT+Jo6q2^rBbG0_<bQP~IA>YdO- z{><*kjyi0ydw#YrYX8=hg#klKL(w@`WltBS;_Rh!3q!-58S%mcr&7eH7bL~0X+&d2 z+2mBw|E4NtPh{y-7q8~9i9I(|o@z|VN()`6-MJFWqSND}QleP<l8m`gCR5bhj3w=y zCu}jKvubaNZ8Njnz@pmY3{?x_r_5Lyg%+Zn<^ppddhWUM5lSKd?FRX?c29bWnt~N) zoJx`LFUkgPB^p-U*STO(d;v+QDd*RiQ$C`&iGvUNjl)l|_N2^2Z0%s`h~_CBf>0uw zr(p6IGH_?e#SZD+VHtG5>pV!cfas$M0=uWUUG&&RUF35FK}>%5Bgx3hPRl6u9@s!I zeA5RGe^N?%M$o(FhVf^QjXz~gv)*a7>Z@`2IDTgB1#4clrST&gxbM}#pM6N~?dU<B zCuJ?c!mQ$DEfpISw&qPawN?J~oT8O&Q!snRFQZ~nYMNEA$JO?6+|28`*Sreu(`kON z8#Z3Ls5b!M^hdymzEDU4IvVnW6o5PjsXBqeN9U9Vu;?RYdK#T|V11++r5QcJ5xFY* zK?39Gud}VA39U8|grExDmdap5eWEupiv4zhEylKHHy292h|OQWf6mR<#4*uhpIeX{ zla%Z+#Lh@buZZhhv3T#Q7`;mpESvg@7cxOIJ$8n2-grV?;KhIi7OC5+SKjNlr^*%f zb0p(@?m1TTu|DeR?>Fr|q~~c%f~`fdMZP#pPJ<_@esS8$-VJ*jJ*zxc{nTh?;*Jw% zsOf=9h0L4uF6`0AflkF)83}?I^ymjt^YQ>12ni5h7GxE@QF@Vhzvvt~we*5YRXPn+ z7Jw~R73m@{3YYreyV2mKWI!4G_fVShW@UBvMrF(>5)-X%Gj~=yUHl7&QSWK2PPyYT zhu)lI^se9WVDs*qvQ~usx3bj2LLUxz8$)>>$pCo<_Tg7E&UvaIrVuyHlZ41E%RMQs zZQ`r3NhuC*rTmXe@|P?qf;@rMJfDT;uNl9?U}J*Qw9e?t*pss6fos>_adBv@yDpJ= zvjVgHsoB%lZEDUnae@8qSnsiCFL#;bYg^@SX9yKlHp349Lk#Ea+aX^!4L;&_qjyLY z7Jsx0M#&l=k<AuBMQH);0Ew+F!7>g-1iX@0Irvuhh6ZmD2d7*;GfV*%25AW<8#Yo7 zM%wQRo;CpUl3)?^mz29pdv>7*DN(o#1`ekC65gLyvNzi@OJC#zGxD%0t0L@YqFkL* z0n5`_?1}Mz<OKBkicB;6K)%^utk;Lr^>%jT7mz^kI^0jB+v5^qo_JTv_>>7O*5XT< zlW+ysGheiDn?rOITgx`^oV}sy_tSDqGyfQ8PfML23ys*XVq!AW=eqxVu_Goeb3xQI z5o2;Jlt{~SvdV>~=zZB0cNb2T+kAOqxvxAM@`k>tIaxtgEmh~F7ffAmo}QUez?(B! zq3t~HqE!D&=Vfv~{2oXwWkH<Rl6Vc1ctJ0_L;m~hIqywL3|AH1JuP>iH<Oj8F7kSu zKKU?vA-&NR9h)YvWsA}mM7iMPwB?5m$?ZIvGnFa+>U1ZQArIGz(OQT7z#vXtXu*Lh zNw7+fr4V<T_9@tH`E*3vV+@qND7NpMVFmNVW8<$ZX;}5hMHhy+2+JI)JGtDM-V7IX z|A;rBBQ$}m3SI>U$;|RXmO@;9TSW{6lni!#G=Gd)`=dsz(dKj4wnI7j)oa}DH7CD? zD2vN{Zna!*sLT=m`Kie^r2_o>th`uuuEl<dxZ!Ai8uuB#D+Ehe)B_L8AJ@%VFZU<9 zmv2kVPJHlum_9NY9_AI_U7MU`ay#8tkI^d}A9~*AGFLl)<B~u5eagI)D}G)5yv=Pw z#AYFAi1kWxSe_+d&ub87CuZl3*NNtQ{g2qeP)NwP%8#2n)(k+cDrT_yuC*JU;#0B{ zw=H)kCc4jmFiJEKnNMrWw?tX$2G6hv@|<*zq4tu71iX3!`hb=hAMKpr_#n#-<{LCf z;pQpJlNIBKo}UX^Zy*!U<;X=LAX4gl=$pk~v+uXR-+Rr)UtH6?+-Inrf5(R(u*FAx zwG}x(z3C=<QtFcJvkDwSS=sEJEBnTm%zt;5i|0Pqlb#k|I51^>!kk#&M)sYzZ@T&B zo8G?WAA3`(suTZy=iQ%ta`&qFwv5)fN90%9ndH0t&e!i>Gb8QrxA|Mgrks=?pSxvy zrfdDxap5VMOXKsCoy#h__w`Mi5ABFaeEfJ_4!FJbpn8EBvj7qk#3|-BTu<Pep^_aI z5RJ{GX9|P$K!P`TXh34+FdZLBK&FynXYqi=_{5xi&-K5*;l|-mf#{uDbN#EYvf!%^ za&t;xMMX@E`8~hbSh<#6P@1!3Mq})pok`;5^})*01(1TAJ$Z3ppDmaXX9`r0-g3+6 z@LjIF{Fq<JFTL?58;>oTzUAuS7LTxpIY;^$AI-Wkr(@P~uWLq4c4kz2O>nb6I46|* z`PbHj34Yi@MQ%>{CK_tmI^&x`+|e-8vPinV#M+~1)t47m2#TZC15=G|ifk2bV2@2^ zhlwXWbsb5DtfH(;w>8@$8l|X=UCUmW7X?`qYqmKi9d8WPyF8b0qr+(}wWn9-&&<i2 zl~<i^FSR-CMt%unXLG|Y@<@)A)h@lmV@>k7;+(w6wJ?3birdl`x|+Bn)*X{%^*Hpd zOOqr|p-0MfnUd3!@n>{rOCEOoY(5y%Ilvd(h&}Eaj6aYvfh!HAGWCg808%E#0YNbq zM|8r3J`?o^NtO}nQ9&I&M%qf07bG!7!&X}3t~V<ZOm=&8ZFEw2`rPT(%pD68gNg6K zb%EEt)RP(Ks|r`G4hL6F3s<GYXT~mxigOzc?;**8of+2Y^QH%r9JTIfyD6di9chg1 zIj3@K(N%_oj6`3)D=X2FxcK5!t3{vGkZLK<v`JEWPIf{<c22q^IWj9PsSQbbi`94O zA|%Xm=cOcM#v86E-d1_ZHq3$3=k^P+mnZ1g@aX><2F|u%An8;%C<Gi>vaJdn>|Fl* z{Ah4cKuftncqnjiDL2}kwo+SqjS2@f>9(NF;V`mGneL3q03fihtRbms4G5+O7i0hk z{PX?uxHC=#0*jr1pooCLtO9|_l_z)v%UN@Q5pP(rbxl~$E~(@XfII^t;8hIVZZMZ5 zW&b4TiI#-$Rv}~xf}tRWIa-G)AbHEGL=e>`-HgH7kjEpKOTCVUnnq($mwb=>>$N{G zTHtidd~C_ic~5}mHd*xgXC1z=V|!)Y#fx_}=31Hl(vOd@z8_1jicmv&(B8rQr88TC zwdZcG)$0n^Hq6c~(no(%m^9s=uTOc=esAb}XR^VNFxQu9OY!5x-6G$SWQbkGSz=*Y z6!?4kGS&|-LncRB!R*2Z#QDwVTvfAp^PE)mOhvJu+5nn)J?uY|Y#W&T!0(fOX<20k zSS>mIBd$Jh`=lSxBi!Ge@e6XuR??gyl#mhaQslCsi$I62%0<eR{&dup_0|LV(+_q| zTe#5fpXawPm{Z$%a7xL8Z)LfTej-Vqen1Hdryc67n+@ci<yTW2npXJ0o0&04k$h18 z%c0Hh99+Bh;5(ZSG26k};{)RZ{A&Kg@u}ler5}##$bSJJD@d^~m|fp_Xj;*OZ)bWw z`dqL7^q4FA?FaK}4t33#wa}gm$tL^4*>znvQ3_Q4C%yiY4_w)AJy<mDrmm`m3+yR{ zhnWM0mfzj7`CYQu?BL1cT)u#9=Q7*=<xkj+LK%a1uoFKP*1%+WH9G=Z<ww+giB8eI za${`?qkrLMOg${bh9ZVYN^pNOOPO77AhlQESyt+TvK=(Dak`jQ2G>nX_(SpIo&5*5 zuJg_<xa@@&gkS$dPWGmkh2|~kt-7Vc?Uq0M=t*JuiQi-z`R#3uagp;R|Exp(C&ruh zyQEn!UM2{az4-1|UrbvzFmF*wDt|8i#$YHM<i8q!lznmhuCi#+zU;E3y0nLY22@+` z%6XvyJik;})~x_G?ALA4U8VaS-fJ8mY?C1Of#VTEkF*c?hKI>7z=a^?c*2NfST3Ty zz>Dfnxxv(EbQW#MfJD_4gfzpdeL5n#uusA2qbxPb8wDd{K1!rtFG6~qwzPC?tlX$q zDS#zAi;`p0M_W5(5y!HGy^2DuQyXY0=OFh8(<=?~2ust-)6&W>%$b^<la!ouZ}!xj zoaCgO@8sY<b>haXOXYX&Kj+P>7RPj5xFva7d9tqzzkXkGd18re@WLx*MI|?dk0md8 zaPL5yO>U@et)AXKosZ7_R_pw$%8J)?gjQuh_*I;{jCt#(R?45Q5vSy71(czXqVm<f zo7RPc8)x{kR?R(992;Fw(_1^m<MML1V2+P<uK4oe^n@uZmQ+Yq8^FcyJeVDqUQo7L z{C#CnQDxCrONWy4@{)&=^Yd}}HzEGg;TJmZIl5)#^84TZw){q<BB1<BiCfQH56#uP z{<3`2_I(F_)lkeI3@lt2C|$TvKH7YHZ^M+E`noo_^yR^Lcl~oi1M(khs}IJrp}cA} ze_e5XBXgxR8w!sMm3LKKa^v|+gE-iMqCuhGJ#dj&T_G8yD2I#bV_`(&($AAi8Yl1J z;~+aiQUbv&xJICzxx!V1yf$FKi4nzw38lkj06YoAlujlxg#CK?z%Q)lHMU)LS=+j) zUA_AcA3b{bwx6(7eSQ7-$D+?7A^7Ihl#AxAy7Kuer$pLGzT)-Gl&+s`I-5E`a}<3s zI!n*O1Js!SV?c5SSU)70a^<*8uSh?GP5{^uka_gFlJDF)TmEp_t*;!OK5y)iv3b*< zzOJp!{#<bWO3S70ct7-2A1}5CrxeTIkygeH)~^{GTT?$6SD)|TInkV)z0leBBFMN> zr~>{W*Xs7^bnq95Nhd+b*g%>|I9Ds=XpaNl7$9mbK)DJnAfIGt22BE}FF><Coy^i9 z5vep_F$;qY2$F`V8rVlidFo;f3Rd~cvSW*z#YjR+Iie!bWz1JxlHv0p;JLQZ=WF!o zP3DUDKzgC)yd}pQD_G?Np32O5mbJBqZjgM|1bd8;<qQulD5@BUniF#SI9!<4rx-43 zUU#sz^Nv*&KTAx=+m-)J9QLlxMt4$=1)esQVwn+QSPK&R8&aFfa?h)sdeJ!p#qtO8 z^~5V;x7llxrsPho&KPZ5ytwU-^y-pudT9bk@3wddhrxA+-Zm$#IydOOY`xe#<H7n@ zEHKMfduaS9B)jz%qx{Q(B6oR(J`i2zHRo0@jODtA=hgisb!loT54`IYN>f}bV>9+R zYUiLRxWa%uP0bQ>ah)|(A*NZf>WdiUZ1~}Lzr8*&=uNbgms_JU;zKDlP7IeqOX(CG znyKuaPHzJs{0+hYRI(Qx=wTTc8{!p!ys!&Ej^K0q!5knV1}Rw#R0#&CH+%(^2aB;P zrlDcmZT(VHabsm;V6DFYwrvd!F;zy(_)nQ(u|<GTcf93ygVOpEyf0ZVJ#o+yk2t}y z7%tEn0}JNQnZKxbu`qO^kd&x}*BvH9SyW|c-PT5NR%&KuT7%TMbzP`3s?1;-zw-yv zjB%DA3cYKQKyOoL3A;v6R?re>oc06b)U*PRr^q**)(hghsoz=xf9KeN1C;PJI6N2f z$gI9<$wKo8m@G_z9t|(c0LQ}>g^$fFq*Rm|XxyL)&`jd7VF!W!LMG}lSZ$J?%`yt+ zygSYpvvL>C$z&{Z&VqcuwB?R0G&a+iU|Ii$G(UevEMu`V@?jjBms#SUUp-@u{Fcy| z+d$C`xsAfxKdubf4Wu@xnE9X%&N+uY4;NbV=Tez-=ND$=9Xqx%hYytEi_<P>5q!RY z*BeMp5!YRitn`g&nth8{m6Dd0QYAj0ZxqJ;!r>+5bAHQflhf0aYx(Url?1GY6U}5F z<h~QlN~a$#_YbiaDLK6)Hnt=^Ipf|(78AJO+&Bh2xjzE0OvoL@>ylvy$dA2fK(`58 z4KJ8nnOPF^3Rx@@8g_Vg6GI*_Bng?U4A#>qx-1Jv@{q$QbMPz!SyL+_iFRlz_(NHK z0V0O}tchz`Cb(6e7?+~x9pfb%8)c-+N~ShwBa6&z&P!?UfKd=_feP)X9~S=&MC3F( z*fN(l@lMz-Sg_16J{@jx<&VV<$8Y)g2W-?OuM)0zALCcypa7@C54l}4jp82+hE{_p zzbA6zM`9T_Oj{2RAI9}Nc{4Y$2PA<_)4TPX&X=UEl76Wmy`q=?CUS>c{DGdm^`|%G z(s%#%Hrw?koB7l6V{b8-VY{XAvxUrI5`qnSe&|K^v-^%e^oLtN=Nq48kKc0Q$&at- zZW5)<Yu_LHD79o%5}uT#B{xgLAK=IB3z0<`#LVS-)?|<h+KudWv#^_82j_q#24UJV zu0LmvD3<r@h4Terx2PZ5ElB)2U|j{q6&fxHm>*hobU>eO7s-$XtWXd)6mnm%lcTUi zK&*foQA{K#vaRajK9rcS7^w0jBmjFlBtBqCDQ+x<ThG#>!lKgTGJR=daf)T>G+sSz z>3!F|bshfrxlql3dksJ;yki`JCk>MLXg+mixfSh^nFV61GuCX5b*731Gb8O4vs+sD z4ZYW1+uL*PwerFv_UNOOT|#!KNGU?!W7<_aPf)(m1c|p*IQ7F$KslqsvIdML5`{$z z0qCeH@IM!*f^8%E$}_%2`zkHz<S&moTc%zV)lVxfoL%H|8Wz@$AE{jkt${@$$q<Ms zNXqf&Bo)L2_(LvRGtHuj>lwXZbDe}9@bPMTFJd+e=i*a)@X7LHY13w}nwL}8*;!Y- zX2blTm}2po@Xu>WVIroz;-*=>PVN;djL-t96631*$$`%G82II>ph;?=TR4h2OMLSQ z2;d3;a80}nlz<;SHDQ`N9Q8jut4l5tVPQt5)YGAfWfy`Xy6Bw73Vm@xer|4VenPRn zqA@3W4m762OLl&L=g#koX_H0iV;tizI$~lRyxb8pIi6uPkq;}DBs2pY@?nAnJs^TD z8|!JS5EC74lgaH!6f4?##+LEvRQOK$x77r0bYambGsZy|W;q?ZfFQGZ5=^R43MD)+ z6i<$Qt^anS2UQ>elc`i$>dK&I$F<#sLe2x&ChT#9G~oMJ&o1ngsLNFmOi*H=P&BPU zE%f!18&NkWEbGE^zTUBW{);XJ1bwMMA8S@RNVDicF2Bdt*M5m!(Yp7|v1MQDVfLib zz2nWNI`Y#~z5BOQaVG)<*(#Jz?qZkt@@afP>W-7vV$y2Q#<~IOO|h;-EJ;N!4Tpo^ zU@8)hpk4hC!wy5Z)+7DJvtx7J<F7nkw|R$W1DgXQpK~tJq>cFpS9~Tv{OBpIM#U2D zk8XI`IcLd|InI}FIB@^{{6VN6P;wTAVBz=ve3qTy(=>t;n$`JeDcSLbsnk>E0m)Rm zW;_r~w&+rLE)V!M3z+;R)%Nb?WP5k7{P1TeUF_R`TC8z@?dLmK?~c#!(i*JSku2pS z--8<Fl~??k%~i4=pZW4vH>$Fh@<%s*^)j0|Hg>bt>QjBE@Ipwk1==?343tL<JVjs~ zrc0GBgfbDGp7JccFN;pkJf-3TMa$OJ=D&FPiMua<F~4@*vZ95HE`92?r!HM2-=xW6 z=dC>N;5Apv7hZkM!Shz~&+WynJAc08`uE<ue#g&uwiesZyQT}Nk*Vt%?$SQys}HX1 z_zXOm?b{9VpFZojlG>`A{YtbCi2_ziC%N89v&j=UV=9qCt+GB%BC8;6h8AOLkTMEk zmx-ycsJ!u=#_~lu7w>+0_wJ|J&2VsFBTHw1WwLR$zLvoJ2*eqifiaekEnhy?+g>qu zZUvMf6i_~XSZe<2FrZa>nW!ptu~C5*5DIxY4HuAXNgnh}=7P5nA$+QwLt^``9#_+H z`mfOG+2|DlO&aD@zvygqs~}VbIiMpZi`#jGF-KZ`QT1chMfGWp>G|yL{OMzgD2xcf z&2eS^aeS+cMN(CcBrQxb--Af)ayk_`(~P!%i4=x2Cw_f+-HJeUbzsH1aM}F%>=s2% zM?Q*#8b&>34M=@f(d_9+*56D?Cr|Z%*N>-GX<n)T0{>SyHS;W-Dk(&ZigO8Ro{e)| z{{oOe9gI!SmzU>HpVXWG_x(8bB|uKEg4`tZS&zOeJJplyEu|O751;DAFHVI{_uT2Y z6Ay~b#|bRYM44Q%QFaXTC?4xNd0<LU<5A9NY@EqZIfR@++`#N%sfSb_v6rF!zGmD} zRf#0d5x5cJ?_U4l{nuW5e@=xbEgYJiQ<z%Pea}UA)YY+Fqu8#@zI^S>&1-8@TY3-3 zAO33h?)O>J{;hv};kxBFUs|-Ta#}6_1WHvE^7Ha@@(<-7N99dz$V+mztm%#Hmv<&K z_OGe&&w<si5%~DN>u#3!(#WjKp8E2Vr{y2@G|Zkmfe#|!58R;hVaITt?gwBL01ilO z3ZFxoXLNL_9Mm{*e31+Tuo^8#Vy7NKITuBG1;>E_=_lK;$bl%VrP|4lA`n66UO>>; zpAzE?H7L6DBr}1{9C5%&p}?Iip-(U^m1ib7u@_Ve$B7W}G$G9eeN%KUjA3F2^CMpj zvrcdO;LWT-zsonhwPf=-f#p2T?lwu&)02+B5bsY<5-Z~UZ`Z}G%5qu^PJba{q69~t zw^lIQDm{`Y`26svo|_baJZrQ*Ve_>mGaE|ck`i1wfvGuDvl5*~y<jxMfWJ2}+ni84 zYa0LgrLM{aOMTAd%7vy!bApxK++%TBj6PSx`K`@X9*Id69Opqk|6;Zbn#|n^u7v3w zi(mLk{@v@#RtoXXIM2CB@O6YApS`QizsPO2>P@+UWrg#?xstWW=82!@sC2}|#8tq6 z1uss{tST(5%51I5b4wBzoR++2wv}z|>)jj-<rF(jR(F!Cv~*5&F^mkQ#&|4=-ngai zhIxl~%b)(n*SKz43eF3bSbob2cD&ZJxlf3(dZXMXSL$6m<d5#noy>0_YgN!Z4Eqh( z#6fa_%rF{Q1v5Y;0ydA&QhX3^yT+8|J8?KE#u@u7&SESEi`)VT={;J_d%r;+;Wzwy z`F^YXkR>tBFoVH5i)5BB`N-3CTL!=3n-mH#v0$Eu)+w8El3a>)m8>vm`-(DXhJ*72 zfB;Ys@uq;74|>^vV{n17eegk})k9i06F*LvrJ-`HvSF-#DuPq%pM?4DF;&QKObL%2 zQT~zg`_%RrVb6)tnD(jjcNGXaiW=7y?3%yx$tQO{E`P}kk3X`5zd%pp6+76as&b8@ zU_*`m|Ge#d&-nju+s^jL|4-T;DkW>X|8HSt&z}Dqh|&C2D)4Sn=$j%~7X&3a0q<B# z{`+<V#{UyYU$}AO!v7k^odn~CF@QR8n#fvx10)HT<0P;HCqA2W=j(RquFxIO9m3iO z;TcUl5R7IR=BM+2vJ9de_Iq(wPR=wrX$*r`FDg>O9yeGA>hr{%c;twgFkKCw@86vM zU*w<2r`PgL+@<Tx=J6|FCT+3VoIlG|KBr^Dfs;BSq&GZ0bNwre^YJQG71Uj=+-_H{ z<LV9e+31nM1{eYpk&_e3?@r1kYq^w6el?kTg~VDswFAwFH|MYUecQ|f8;*l#PgMQl zIax_H`DU|xhmvpg*RDaO3RS-O>u=xvT6$`$KR7uhb^|n?gu0S&eo_F*ooTumu!(V= zZl~^Y-G1Fc-EF%2bl=lGMHYOq$2OcI`G_3II`xEo_ry70SQ(#iz^~oa@jCrH5kGmy zJ_W2ETHF<&An7^cLxTBu8f*fdiSj4%Pu%}i`De#ZJnPAUJ!rq_HRHOP=`LF}_A0y@ zcK)Ih7c197<+^uLSd9@EtJFHUXa_d*&MWN7@mMUd&Llst+&mekM4U0rm5xH)b?j@o zU;no;YHjSuk-J8pCE9(H$I~C>^+r80de;&59co*2;iRil))_J5r?v-tY{P*CF1zo{ z#ubhP(#hu%%uP%xM=f*lzl~ArQudG}>!_1ttj*QX_1g%DP)J0dO3L||o7^TqmPPqb z=F2lc$0-yW(U8RE2lYqdqG7P}v7et1?FU;>Igx^jJ4xB%bOYQ6I?|w14k+s==dU<; z5{^Zs#Cqfto>+)aAK}UJU*9nzr65A9=B8&Jkzf4YxyNp9V(f=EL6S{iM$R0@eaE&M z4V!+zgez}lMepqxKepqE9Xp<2xAd$tg0}G*%$2pH&u`p$#AdFmF&knf?ld;_aN(l& zFTCoXSF@GN2i|U7y}I@7{uOsJ-RJVT%LS{cINAqZ@*);^>|s`Lr`gbZ-|x<QOn!Bt zV2X(~6<JMIvGUif<c=$u^vpJsdE;yHdzn<{j+44=Z_d;k&B1K5IlOoAz!NQ2#}9X| zy>qJBoD(z|^>f}mZ^yAq^oCu3R%L4-r#J=<4Ooig-dkn*oo4Vcpo!xc5B0c5-8YXx z9<_P$zK>ykW1Gpy#<}k7{oBM*k(&4D5!!vz1!Jx7UlbpNg3bzDughUkIULxV_62H7 z&e$4jd|Sm4Jm@!a1&{r{fX0m<o?ry<X4S|!N9iZ=hU<typz;$1)Jq%!s#}pOQofyR z?LzY&4o<KQASDGHL&C}$okgYqKr}BXaDsAn--zGZYc_xUvDv&AXchN(1kQNo!;`!1 z?SGX_%O<)e_bK%a`V?bBp6!Y6^B>#A)izODZ;2mMy?5QEHV=2Dxs#qx*uFl*>@IxD zH>5q4SAJR4odE;XpDK=5V2K=Ie~qj!WP$M^`4y@88)$ge!Gkz5eC?a)b>h|P3>@nR zOyQ$H3SmF`hq^b=Cw`dw@Icyv>?c9K4I4K%+<qw90;D7WB=08S3+0{1ovBQ7#cK~{ zJS>6W6p%q!19G?!yjT2)z|)GK&;jrWc$9ufXrw99RU~#s+9!Ivp!ekG66gjP#Z3p< zWrf^OC6;;=IT?@oUh;VTS#}W!29oPYf&h@xSz8^+;>fmI>_Mlz+UPYHjRvpLa46lH zZu48M>TN4U8H^q$+mm)p*k35lnP2Va9)nA77bL;(oZ$7P>9bePaOGO99DY~?A+KC- z-mr9PZ(_0`qco*pxjk{J(-z2b720ezb3uuX;|we_InI+FNlRV*h?Bv*SWI4S4un}v zz9?^bY)Xs`PKC2KNG#E26O$p??%<|$?upBF*=??Z=O0a3zA2%or)zrF-!YI6VZ<I% zJh06NLv#M;Vn$R{c65}K@rKc8^w1}H7N^^6ep{L?-D=IUrf;HnqtWSEU7hDooZq_K zot(jA^fBfbhsPZ&N4t~kZu87Cr_I7xL4i952S<woOLv+Lr6r~9bHWa>y1aKN#^Q>N zho*lbG9`&ZV$+_G-Q(;lDolHHrqg1Lj;r)Uxuzv^y@^Q<39iR-GD983og+!Pdc7f# zGkr>3ZE`q1HaYCi_gUf|WTxie_VRVhmI$0}{U#995sm{M1Psmu+(nVTFiG8&3NFY6 z0#d-lBW`Auh&UWFA}T#q3emX3@)?<l55ajY<NzQCfk)!|aee0luq#y%mD*`HbS62b zhF8e{Sh1Q#(o3cl1#+X;t<slDd;eIFKYp2f@%FkBhJ$WiRKJ~V;ZN`Rquwb>>wGE8 z8^(W`=#XZQZ^VJCzzb$w0n2^QY_AV6<M<m3=8BfMtg761GUdh7;QGbd5oaqd<2%^i zzf3E%#dcT|>c`iuJ$LIU2sGt9MDY(51x|P|XznE%2NWz97{`x-sjWl?W*k(jiGvfG zDiDdSL_&N6#`n?<{w!D}jB=H_Aa-0RrKP7q%Q#T#ff)y|RTQm_5E7I@=;Q19D%Uf{ zC8OPB!tNcuieO*U0@L@RAnGN(5ofW--`}>4J-FefM7Q-&Pr<xAc9$eLqkS<sQ87kw zjXUG6a@fq9COXoR3R6b=4ar3T-@}uS<oJh=h#sTUU=&!08wIE2Y1_)YMhApMZs2Bv zMNbX}^cKnfDSHP##@_kWzS+2y!<5W)d!5%A=d?-!B1&(lx@l*UFWr%Ly_}t#F-?e` zjI^<?NV?*a+&EYICswo=@b8vD>r^L!vqV<QJdi>lSbzYxi?9i!!v#fD(@+Ji>SV#- zhrj^|6jX77FNHXf^jV~GO~?b8NYf39?)r3}PJo~<{Mq1@w@`q%2GVhCca;BtyKn|< zXhe&f^^&dd{GQR2s6(}EvApiiIG-Rc&6Kv~rR66}htK`F{QgbX$ba3C?3jA{w|3`b zr)HZ(;ryT6vaLaMl&78Z<-=EJW_r@$Of2-8JihypoJ%i0FDvWHEzf;A#~$DC>sO1@ zX06G{ByTx$pz^MdO3wuHD4f|7ND{bIkzEVtS4P+LTdKKbNzU%XkR#1^2o^jl4*c@i zkC<RgZtuHig^>29{1%^*IPcMLXz>*_ytsO4p+`P+Gs}46yzb`8j?$VKy(qAx%uKT- zrgr|+jE#S()aTUJ$Hh8LuDF)imQ1(UeDk^*i`DCIW9Kr{?)k6De;iJ=#KUOuYS`xs zoY%c3KHl2kzvRjtxw$;X5g(h7U^S;qHTw2n{?aYOZHZ})IaB=$hUEr~U*<`x{vGMB zIH@WI1-e49IE7__@IR<U68JDcb&-o1L}gY9W%_vdy9KFfaC3<4fgP(H$S|o8roACE z%e{VHB~msfgmqb%I%UbCUBG}#(~@Q`uiPDpw!+e5N@YUE^t{YiH=KL2jlpOB1Q&W= zeg1H##2jY7SlR_krL38)9&z)(EV}snuYLdGMe2<W*8C3UPk$o6C(DmCCYMO_u6XK$ z&;EYwmAR>vQ?2sb|1@$Qf8OgCH^+F}um0fT-Y0Kv<)7!@Q<0VAPVkx~L3EgHnVH!c zsj)UT{*&!bw8WO~IKsTQ=B&usVtY;ACCk@aZ@x7F?j%!Qdzub`o>p)AYhG(JE_&ea z@~to2%nJVc`nMuE-etEA2d<DW_$ymkFQMeM`=0vQtKZ+diYK@4eHBv!@+6>X6dX$S z?24eHO)}jB(9OOQdfE5G_7CJv$wDR0Q^|5=>Hqebte64SYEojbq#NTV`3J?vEy+FL zEa89kd}PpB?8F}|a{k-9_}%jC6GzBqs!*L>4#Mbv&Y~0vmY>t<^x^lPh7Ny)3d*x3 zs_eLta-xLK|A#w`4bv52eOrX}?JA-*0j;27Ag1Gi5TB44g=ctmEu!r-9mU|CVqzsq zf(9D4&=aD5m?c%PVO#);3<ncrghh!-!2*e0w@Zpr9>D-sq!N=zI}Liha5PM|k0Bvc zhE$6D5LJg|Cey|;!$_e|zT*k6&1MgHpD42hX4*RBKfmVWv8g%EL9iPJojIwo-1(aP z=MLM<fUWEaZvcjVyieSb6mQCfvuMRJ<V&I{Ax(@=pn~Fq9<3ba_=a+r<E(NL;>ENC zlPJHW__Pcs<(lHzEvY@WQZE{{;jq8doXPTUlwbHXIyc2-j2?T7WC7nAi#EDaa-%<G z6*tOnWSUF;hc}>A-cnmns=lx&RbO@RAPk%5=Soykq1~<)B)@SZtN7-E<QHya`DYd+ z<>qHFDoCGNR7m4^nhuYq9Tg)YmlhQ)6kbmT-1T^(v4)5SiTP=d47`;gJ!5Fx``YNp zd$)BP5c=8Z4a|Knn<?M2Wvkipi2TMQC*s%|CCmyQxr1NNn>PL8=7_<t;L2A=m5}LT z-AQqpeCw$E((RARFTN0W5nD6D%5Hy*mA??@SxH2bwl^^A?63gtLCZ+u+?8%&>8`9Y zuK~nM0Zg)GW#R`jNPe9CPd0sY>O7ug0)&TeDZT%ml7|+=d>$juV8s{8ud#PO@BEBy z|H0y?`7~P4<Y(pge}3+{KWAwG$F!gS>6`W&C*()jdimRIQ))>^fOn&m3paOu*0Flg z(~H(Cxsd;KNqqA+P=(mDo@9pA&{<unYt{9_%ESfmLiMQbS>4OJcXS`=KE*de6w41m zS8OY=Wq>RtCWKzuVnB~s-D?OjdSwft>=M9@P`DCd5(W=@1Il_&s}49BSbvbCiZKu7 zoMHu5XIJ?an5Gno35N*;4|X6BD2bW@l8)grnwKcj<An$=2n;>bN>ei^<xx<HR*RWP zfq6L*q!d|{QSp)_g~g&^fgsJBBiTC(lEEJ_2nNYOHqb3bi$$NDk-(g$7^@|moU_5} z3}ohc7BwX2L>sP>^eOfPJ#S_D(gwG<q)`u57?E!`SOX<d^>YI!YV=NrJx&muiF}3C zkd|Y$;4&VQF&&F|bTqD#=(3jA_^k<vH5bhY^{(!7UvOxem397gW<Xf7IoDs6A%-^8 z`*z**z|2_}Z(UiW4>rX3jt|*QZdZv-x!x;ArzOHEl`|?)ybUsBt~6te+nqYz>vSY0 zOmjLN;VS->=yW)!8EDM+9dKG2PB!O<SYfQ2#l{ys{y<iQ!<0XV=BgmVkFsQjrzV^# zdYQw3JywD(x-d8K@^i0ZGulNSpK4>HMvL9x@JIi};?MN@jd$K;N@9Me{AFUOJ=SCs zQtnJvD~s35??&as8l&hUgu_->bai}!HQF`K66^fd@>;jc%BwfZU(TB@G_IH6;do|2 z*X%X+jaS}WIrZY9C8lNPS9r@}3^h%=XFC@+ck)4Zi5*|9T+zTJxCh5)i>?z>+-ag1 zlbt4sUSUJRbbNL~VpW=Re5oT&6r${oczpaZPuS@&=ZAf;`mc*+e%c8s|B7_YS{Ob! zba!fDj-A90wXgur@8?=r)LB@(7M66d{iB8Th~KP*4Z1}<OA8xyR(_2ZHtEd#5iJav zRQ`z;w(F9G8t{3T2n%^Q9UzVpg&8o+Ra%$><2<Z|1<0=cR11qb4<jBCrI&OW_%1c9 zM|_hOHt4?1c4%RvE|zcA!X}-YAJ)PaT`qq{3)^*51p&N?0mvI~g`d-I*uv<7k1-!K zH(PLS5kY7sLjBNa+KO-h6lMu9VIjgbz#qN1pUyR`JVkWF$Ty_qZbwR8-N4}1q3+JE z5np}FrigE5OMm-TU!b@oT<ELm?e!_IhJ8bk;mFXYNINn#pgsL)MGf#1l{Ns1HiGDZ z{*ju^k>P<pJS;{`C*)mv(Yhf#Mmk4(TM%dfg|+}%r}L1z7Iv%`BHRGp{YuDGFGMcr zqc$vDuykHcZRNtH4Kr3YE?i7e|3Qt*m9H8`A5rUkx*)!-48LN8C;COVJTf%gJ<#t9 z7M2wjD<Aov72b#siQrSH>!?d3I5?tC^r0IDlxvsr=9`9!^0Xn{M8i6eL(Qq?p=at& zDr*RJv?G0=(rrD6Ye6i<cVj^qMQEawe&nL3Hl!Y=VQlP;v<ycuP92dU-^hS(q$}b( zJ+#A-wh{X9jsd*Hn+^<?Z)B*YJ<``Q)Z=Rz85!zs9aWO_4~%rTMTWJ39zrbyOlJ-C z(1`EUSbq~N>Q2LwP662wfN&*9^dj_}`n@e@lv${JnXYSOWDt5i)VvlImI}KE{+kkt zFj8u-^edxPgv{SmW>GIbvVS;&_X>?ew}17IKZiFAl#qZ^!acf6amI9&?rPWy+N-;g z5xR!ERY;K=m=WGt&CG&bnhoTpgE^rB7|mSF&0?_Vd08y{wZyXoNLwU<Oif`vFg4S# zd(B{(%n$vOY?i}vVXP*fk;#Q3Rt%G+0anU_ECiyn3^o*@bpksK5KqVQJDpXt8hBZ$ zWA$(W+rVbBS!^~$8t1~d!hE)XEo6;s5nIfbV5hf?EoUp(O8A9Y&DOBBY#m$Any~+C zVXeStpXqF@jkU7~>tLO%i*>UNtOv}uKIl^putByFHc*Dy2u#9mVw>TOd@I|=&cVj` zJcv(jXJhOFb|KrrE`r;^U2HcbNiKov>K=9(yPRFYu4GrStJz+54co`|vjgl~Fv@lv zyPn+uA3+CUq5CFwnBC02&2C}0vfJ40><)Okx{KY-?qT<```CBb{p`E!0rnt!h&{}{ z#~xvivd7?V^$GSQ`#yV$J<Wc=j=&n{v+OzcL-r%~JbQt?$bQUTVn2Zg>X+Fo>{S@i z{TX|m{hYnQ-ehmFx7j=F7wld39{VNx6?>oknjK{yuw(2)_7VFHtf~GEo{K(ae_(%P ze`24oPuXYebM|NU1^Wy8EBhP!JNpOwC;O6p#g4NRY@EsLB-e4qITyIdB@S*1H|o;3 ziJQ3v-hpf!h6A~iNAYOx;%*+pJ>1J<c^r@D2|SS}aqwGU)j5@?@pPWSVL5_l@ob*M zb9o-m=LH-(x4f8_@BlC6K_22^UdGFL1+U~&_*6cPgR#S_c@3}Sb-bR>;0=5xpT%eM zIeadk$LI3}d?9b-i}+%`ME5#h%9ruwd<9?0SMk++4PVRG@%6lkH}e+W%G-E5kMIsC zJ#_JIzJd4fUf#$1`2Zi}8~G3)<|BNRZ{nNz7QU5l<LB^m`FZ?&zMYTp3;2b62fv8# z<h%H8elfp<U&{CJ%lPH|3VtQO3Yw~W`89kW-_H;5Z}DsSb)fri;5YJv{1Cs1ALcjn zZ}VIDt^78AJHLb9$?xKK^LzNc{6792en0;%e}F&8AL0-5@9{_Yqx>=cIDdja$-mE^ z;!pD*@FV;g{w#lv|B(NPKhIy_FY+Jrm-tWkPx;II75*xJjsJ|l&VSC|;BWG`_}ly) z{tNyte~<r?|BAoQf6b5b5BM?uA^(W~hX0oTj{ly2%>Tgu$p6GY;h*x)_~-o3{0sgU z{#X7t{&)Tl{!jiT|B4^yCpdIt`AI<snhl3{pxY!t4|5boSb{bS7QqVJPIkc|IAIq$ zT5t(&AqKkGULjV96XJygAyG&Yz>E`oLaLA^qzf5Brr;N{glr*4$QAO0e4#)9FHR^H zN`!z=Dg<GHGAxt{<wAu}DNGTj3e$utVY*N))Cjdgolq~#5E_J;!YpC7Fh`gx%oFAd z3xtJ2qp(O=EG!Y03d@A$!U|!fuu51htP$1<>xA_}lh7=*2(3b!&@M!T4xv-%61s&A zLXXfZ^a=gKfG{X*6o!OhVMG`eHVK=BEy7k|n{bYBu5ccdNVW@O!Ue*G!VcjgVW+T5 z*ezTvTq0a5>=7;#E*Gv4t`x2kt`_zR*9iNB{lWp^Tf()%b;9++4Z@AWLE(^alW<tL zS@^bai*T!On{c~uhj6EGmvFamk8rPWpYR>we&M^q1G;@uXK%~!u+%p?+})-hj<gK5 zbr~c5n<BjfgAu6<&m-dSNXw9!E=`fX!I7=vD5w^(qr11yIMUSw3Zyf_N4g9Y?j9cD z13mhoNZ-Jwh+*5nKwndLzfrjljE)Ec9Uc1N?#}*}UZHKEQ^Ha?+$DB_wlSi#NK;Gi zh&a;S7ZHaBTH3Ac1DpGM5uj+}#Djixkm987{?>slmcibZtxav+Lv6hg)HxVw88Kj~ z236H%q^2kZ_71f5h#kExoo0MY`(W2Ve`MIaX`pwsFVckeShOHjVA8^)gZhm_Z3FEQ zLo2!icVVQZQ^aprY#kWrG17%rcxiB`yMILA*3uUlY7uF9#rxiNefLNU7DCHNWXniX zSA?iQvl8Ci-9FM~#=Fk`rrt=$h*b?@$sCCcS=0xGGPJ4T4Wq*&-5py+`W8!fe>>8t z`LwW-*51+57NK5i+SJ`1888fXw~dSrMf8J_{<iL3b6?9~6SXxmq;F}b0@1^0XQaJ* zL>lgD8Hz}4T@myU4VZ0sBr@34+S1muxn-!`*3p74oOm)$1Vrj|X|M%A0Kga+G=Tb{ z<W?p5#Dh|j_8>(zfKalco=rmo>X+Ll9+Xco4fc)>HxXc%`?~wJphX2DCE761qugy9 zM<eJG6hetj-TfT{YPR9Fp$G_%u7MGomQ95U6EdnHb8AcgM6hLOXkfF_lNL3ow8E%{ zM+ePnK!FIQiwKU;oZ;?mk*1E(-d?K~9`0-D?R7@BwDq?1wVY~}*xB7NB7$)2z#JKh z7$aK&Xc!F>1=@NCh9g$=SATbZr_y!_{n;Newzc#|`rBKE^h4Mx4D=b=2KxFi-uk|l z&i=@Vd7{5Y2T%1QwGZGvvN;kNvEkDP2dT(5Ojv6NpfEC|R%X#2s0j|O;hQ2uAV*tz zqqOI)fuZhgL>=~;0P#(2fQu39$mZ@5z@^&p1Y`vE%9B-v_$E|7G$8auwu+d|!$z&i z!?uyG(Z1H<rqMy_IEVH`&826Pf=IerdOIu%c2RMPfd&*P!q(f}-vi)ON7yhpI^2cN zvSB7ghJZGJVfuR%6w=+VN9BWETP>a4sG(Jb0~I?^HBv8dP`{+icZ&kzYDM;m$*Vq^ zl>|y=gZ9D3iEq`bCF@6lhT3{805MD&>fm-^Xn0uYYHv5T0vgbH{bFmRx7X4}-P(bU z9f_E`FpNzqbSpuc?*=6_I%rbv)FDwSa5kNW$mla-lmZ-QM2!xfnTd)44j*WZ=r<2x z&UZ;8EyF#-dSF!anW=TCJJQjHO^lf!SDhzP=g`3DAka#Gj|6}mZP&L(T7V&hw$Tv` z<=|HHV9THaKiz}kF!rxz8l9$A0BR2)ZeR$&#YcPjKrb-HP<KycqzeUhc9}+jqlQpy z1Z`;TjYt6A?lz$L(Y78Fh7s+-^tGP~Dc$bu92n>X@;`+GER!N6jA3M}8GRlZX`(O1 zJfR>asT!bewWvX*uP|?b+53mZ;ejE58ZJsUgA&5znONBfM6gDvuqLA20|1y#z<)cI zq}Bn9u|)%CN@<+{ZF(RaKLU6i!7gvm2uL5o*tY;90_T~5+q-}?M|)e1zzZ1X&WK&< zVx<|hbXnC$6;chfls5IXTab68YhW0iA2AM(c8}1A840MUMtvI=sz?MY%mA=5t(3}g zLZ8q&+TDxU(rHBIL0WfAEq$oHrN1qr?~AnebdOj%s7a`0Lj+BaU>)dE`d#cO?ubOS z4~$}<n$S*MsWYR`h*9D1j3~h08P7B-=F|BtYEa2&RKtW(mGUX6p*=F(gJnbC+cHQu z1yDw;eFLrZUCL~<YVfYWn|b5tz=+y3iyBnN8DHDqkB?E)N?84Sx0*FnfR1raMuF6a zj!9^cp3RXhgEU1E#OPuG2Zt2`L>lfxL!=I@5dA`5q|4BW)qSv~-3T<Nz=e@$3Cd<y z0t#vYT(=w1$5_2udNG6ABPOLmNY?8(h3_;31Z1Kv7Rsa-+knMPl%)yFWe*`6K&99e zD66ncT3IZ^qnMGHzTJZW>(N#<gk<HV*1^$j+o<olBW)2Zpj04HIrh^*B6Re_UEPu1 zcKgIaqqZiBmZ&D6SHRF{_iz__bO^{ULJQB9wss&WZPivVfC3<A#HavrMie2a$(a-c z)iu)B8xq@whfDQ<O`uG(idCs26)ZZ8)zJlb?;afP9&S3lAVr;un^?5Orqbe4kPIOG zs8M>XWN0tGc7k%CGBuR1L>hY|AZH0@r~w6H(Zn`&H8Uw_or*%qB>}U#<kK@P!kE^y zvZ(-a8d3ndl)6laj1CE%t%JhoaJ$gmKO}4z+$s!>whBE%n}ybqHX@TFrc-m)so<g3 zt!Uz4S4%5qWK(INBKlOU#xaTvd%hk;pJkhfR&a3C8Bg>c#gzu>60&Z^YC75)QI|ID zLEM62Hqk|iK9z<#)6fpM0Z|Q<4gzojd4a~lbLUV?pS}Y$ZO@R<(%vt2l$4d&Tf0YE zf!KkK)nNc8>>aX<C^sd_(K~>OP7_nMNzbE$liw0tIVZhUr}$=&xdWSr4Vb1w1KsTs zCdTL%G_$*v)|TO(t%F$921bX5H;!Ua0673q8PInCE%!!5y3hhX(mf~)kJ8YF!v@;i zbZ?3Xt)rcMQ;)Pc(%m|MjYB{Fkf1DJSH2z7LB-q@7mQIqU}6pKRY`Dq6}GnzfF4k` zA6n;^m0LG~6bDtRv<a!oi*4%4rbILiP*i{}rF5aODi6^mOqVjcl+&fcFxubUP*PKj zzHTW&Oa*1CD5XcrR6&_4C{qPxs-R326=GAcSjo{!A>;@aqncoGP%W(%1qF+dDOik5 z!D3_z7E`8@V!F`V63SFUnMzPiumsfvODIPP<tU*XC0g6+iq%r<C|`i`1t?#D@&za# zG}~~Yd;!WApqfglrc%lQYtFb(j#908^;&`TC2GO-N(#zas#OWUiAuNBhqXHu9;7mY zRBMoG4N|Q^<ppI8YFTGc#UZLVM8$?EQ;0H!C{u_sg(y>qGQmzuQ!q?9!juDcjB%kH zVXdhR$~(#wF2j&?DDNm!8NDc@Ol6d*j9!#cHDy!{B%P7CjY3pS8RaOa9OaaQ;37zH z5hS<>5?llcE`kIXL4u25IpwIJ92Jy<fFcN>z$GYl1e9R}P#~ndpd17gApiv~$Ppr- z2oX?(icv?X7<GgQB0>ZaA%cidafP%g0$hq9fkcSP3K2+z2qZ!T5+MSK5P?L9Kq6E^ zl?14g0OcTH2oW%Z2pB>H3?TxB5CKDofFVS{5F%g*5io=Z7(xULAwpjvn6|=&a+Fez zQp!<EIf9fUNI8O(gN8mtLm#4{hqMA+C`XWT1Sv;|a?rSkXxu|I?jaiY5RH3?#yv#i z9-?s%(YS|b+(R_(AsY7(jXUg}(S>q^DF+4}7s?T?KyM=lE|dd<ju7RbK@ZWOhcM{4 zP!1aO5Dj{W20cWB9-=`H(V&ND&_gunAsX}$4SI+MJw$^ZqCpSQpoeJCLp10i3_329 zgD^peFhK~Ipd(NgQRYXWT%Az?swqrEAEu!XYxp2gu1==_)fXlJ2onH=2_=LHmclgt zVH*E1jenTNKTP8vrtuHc_=joy!!-V38vihjf0)KUOyeJ>@ekAZhiUx7H2z^4|8PK^ zmVp|rg*ED&57Y$Ime-VOcXh%AYP6=-s53uMQ>MKy*X|SL)o9PP+PzM@*K79~>b+L0 zw^pmSR;#yGtG8CGw^pmSR;#yGtG8CGw^pmSR;#yGtG8CGw^pmSR;yP-nt?j4-a4(` zI<4M1t=>AV-a4(`I<4M1t=>AV-a4(`I<4M1t=>AV-a4&b4Yvj~+#0CY>aEx6t=H<+ zFl<1>uz`B5-g>Rxdad4it=@XA-g>Rxdad4it=<`0KhO9-gZkGMYOgEQURS8Su2BEF zLjCIsN-365OI@Lsx<V~=g<9$gwbT`AsjDkwvWfBkf8e|lBap)f#1ddZ5TB9h5Re3( de*jv40|X!y3}<W?od?mU{cQ|D^cBO5od6617EJ&E diff --git a/site/fonts/fontawesome-webfont.svg b/site/fonts/fontawesome-webfont.svg deleted file mode 100644 index 855c845e538..00000000000 --- a/site/fonts/fontawesome-webfont.svg +++ /dev/null @@ -1,2671 +0,0 @@ -<?xml version="1.0" standalone="no"?> -<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" > -<svg> -<metadata> -Created by FontForge 20120731 at Mon Oct 24 17:37:40 2016 - By ,,, -Copyright Dave Gandy 2016. All rights reserved. -</metadata> -<defs> -<font id="FontAwesome" horiz-adv-x="1536" > - <font-face - font-family="FontAwesome" - font-weight="400" - font-stretch="normal" - units-per-em="1792" - panose-1="0 0 0 0 0 0 0 0 0 0" - ascent="1536" - descent="-256" - bbox="-1.02083 -256.962 2304.6 1537.02" - underline-thickness="0" - underline-position="0" - unicode-range="U+0020-F500" - /> -<missing-glyph horiz-adv-x="896" -d="M224 112h448v1312h-448v-1312zM112 0v1536h672v-1536h-672z" /> - <glyph glyph-name=".notdef" horiz-adv-x="896" -d="M224 112h448v1312h-448v-1312zM112 0v1536h672v-1536h-672z" /> - <glyph glyph-name=".null" horiz-adv-x="0" - /> - <glyph glyph-name="nonmarkingreturn" horiz-adv-x="597" - /> - <glyph glyph-name="space" unicode=" " horiz-adv-x="448" - /> - <glyph glyph-name="dieresis" unicode="¨" horiz-adv-x="1792" - /> - <glyph glyph-name="copyright" unicode="©" horiz-adv-x="1792" - /> - <glyph glyph-name="registered" unicode="®" horiz-adv-x="1792" - /> - <glyph glyph-name="acute" unicode="´" horiz-adv-x="1792" - /> - <glyph glyph-name="AE" unicode="Æ" horiz-adv-x="1792" - /> - <glyph glyph-name="Oslash" unicode="Ø" horiz-adv-x="1792" - /> - <glyph glyph-name="trademark" unicode="™" horiz-adv-x="1792" - /> - <glyph glyph-name="infinity" unicode="∞" horiz-adv-x="1792" - /> - <glyph glyph-name="notequal" unicode="≠" horiz-adv-x="1792" - /> - <glyph glyph-name="glass" unicode="" horiz-adv-x="1792" -d="M1699 1350q0 -35 -43 -78l-632 -632v-768h320q26 0 45 -19t19 -45t-19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45t45 19h320v768l-632 632q-43 43 -43 78q0 23 18 36.5t38 17.5t43 4h1408q23 0 43 -4t38 -17.5t18 -36.5z" /> - <glyph glyph-name="music" unicode="" -d="M1536 1312v-1120q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v537l-768 -237v-709q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89 -t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v967q0 31 19 56.5t49 35.5l832 256q12 4 28 4q40 0 68 -28t28 -68z" /> - <glyph glyph-name="search" unicode="" horiz-adv-x="1664" -d="M1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -52 -38 -90t-90 -38q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5 -t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" /> - <glyph glyph-name="envelope" unicode="" horiz-adv-x="1792" -d="M1664 32v768q-32 -36 -69 -66q-268 -206 -426 -338q-51 -43 -83 -67t-86.5 -48.5t-102.5 -24.5h-1h-1q-48 0 -102.5 24.5t-86.5 48.5t-83 67q-158 132 -426 338q-37 30 -69 66v-768q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1664 1083v11v13.5t-0.5 13 -t-3 12.5t-5.5 9t-9 7.5t-14 2.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5q0 -168 147 -284q193 -152 401 -317q6 -5 35 -29.5t46 -37.5t44.5 -31.5t50.5 -27.5t43 -9h1h1q20 0 43 9t50.5 27.5t44.5 31.5t46 37.5t35 29.5q208 165 401 317q54 43 100.5 115.5t46.5 131.5z -M1792 1120v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" /> - <glyph glyph-name="heart" unicode="" horiz-adv-x="1792" -d="M896 -128q-26 0 -44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5q224 0 351 -124t127 -344q0 -221 -229 -450l-623 -600 -q-18 -18 -44 -18z" /> - <glyph glyph-name="star" unicode="" horiz-adv-x="1664" -d="M1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -21 -10.5 -35.5t-30.5 -14.5q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455 -l502 -73q56 -9 56 -46z" /> - <glyph glyph-name="star_empty" unicode="" horiz-adv-x="1664" -d="M1137 532l306 297l-422 62l-189 382l-189 -382l-422 -62l306 -297l-73 -421l378 199l377 -199zM1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -50 -41 -50q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500 -l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455l502 -73q56 -9 56 -46z" /> - <glyph glyph-name="user" unicode="" horiz-adv-x="1280" -d="M1280 137q0 -109 -62.5 -187t-150.5 -78h-854q-88 0 -150.5 78t-62.5 187q0 85 8.5 160.5t31.5 152t58.5 131t94 89t134.5 34.5q131 -128 313 -128t313 128q76 0 134.5 -34.5t94 -89t58.5 -131t31.5 -152t8.5 -160.5zM1024 1024q0 -159 -112.5 -271.5t-271.5 -112.5 -t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5z" /> - <glyph glyph-name="film" unicode="" horiz-adv-x="1920" -d="M384 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 320v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 704v128q0 26 -19 45t-45 19h-128 -q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 -64v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM384 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45 -t45 -19h128q26 0 45 19t19 45zM1792 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 704v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1792 320v128 -q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1792 704v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1792 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19 -t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1920 1248v-1344q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1344q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" /> - <glyph glyph-name="th_large" unicode="" horiz-adv-x="1664" -d="M768 512v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM768 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 512v-384q0 -52 -38 -90t-90 -38 -h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" /> - <glyph glyph-name="th" unicode="" horiz-adv-x="1792" -d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 288v-192q0 -40 -28 -68t-68 -28h-320 -q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 -h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192 -q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68z" /> - <glyph glyph-name="th_list" unicode="" horiz-adv-x="1792" -d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-960 -q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 -h960q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68z" /> - <glyph glyph-name="ok" unicode="" horiz-adv-x="1792" -d="M1671 970q0 -40 -28 -68l-724 -724l-136 -136q-28 -28 -68 -28t-68 28l-136 136l-362 362q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -295l656 657q28 28 68 28t68 -28l136 -136q28 -28 28 -68z" /> - <glyph glyph-name="remove" unicode="" horiz-adv-x="1408" -d="M1298 214q0 -40 -28 -68l-136 -136q-28 -28 -68 -28t-68 28l-294 294l-294 -294q-28 -28 -68 -28t-68 28l-136 136q-28 28 -28 68t28 68l294 294l-294 294q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -294l294 294q28 28 68 28t68 -28l136 -136q28 -28 28 -68 -t-28 -68l-294 -294l294 -294q28 -28 28 -68z" /> - <glyph glyph-name="zoom_in" unicode="" horiz-adv-x="1664" -d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-224q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v224h-224q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h224v224q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5v-224h224 -q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5 -t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" /> - <glyph glyph-name="zoom_out" unicode="" horiz-adv-x="1664" -d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-576q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h576q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5z -M1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z -" /> - <glyph glyph-name="off" unicode="" -d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61t-298 61t-245 164t-164 245t-61 298q0 182 80.5 343t226.5 270q43 32 95.5 25t83.5 -50q32 -42 24.5 -94.5t-49.5 -84.5q-98 -74 -151.5 -181t-53.5 -228q0 -104 40.5 -198.5t109.5 -163.5t163.5 -109.5 -t198.5 -40.5t198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5q0 121 -53.5 228t-151.5 181q-42 32 -49.5 84.5t24.5 94.5q31 43 84 50t95 -25q146 -109 226.5 -270t80.5 -343zM896 1408v-640q0 -52 -38 -90t-90 -38t-90 38t-38 90v640q0 52 38 90t90 38t90 -38t38 -90z" /> - <glyph glyph-name="signal" unicode="" horiz-adv-x="1792" -d="M256 96v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM640 224v-320q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1024 480v-576q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23 -v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1408 864v-960q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 1376v-1472q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v1472q0 14 9 23t23 9h192q14 0 23 -9t9 -23z" /> - <glyph glyph-name="cog" unicode="" -d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1536 749v-222q0 -12 -8 -23t-20 -13l-185 -28q-19 -54 -39 -91q35 -50 107 -138q10 -12 10 -25t-9 -23q-27 -37 -99 -108t-94 -71q-12 0 -26 9l-138 108q-44 -23 -91 -38 -q-16 -136 -29 -186q-7 -28 -36 -28h-222q-14 0 -24.5 8.5t-11.5 21.5l-28 184q-49 16 -90 37l-141 -107q-10 -9 -25 -9q-14 0 -25 11q-126 114 -165 168q-7 10 -7 23q0 12 8 23q15 21 51 66.5t54 70.5q-27 50 -41 99l-183 27q-13 2 -21 12.5t-8 23.5v222q0 12 8 23t19 13 -l186 28q14 46 39 92q-40 57 -107 138q-10 12 -10 24q0 10 9 23q26 36 98.5 107.5t94.5 71.5q13 0 26 -10l138 -107q44 23 91 38q16 136 29 186q7 28 36 28h222q14 0 24.5 -8.5t11.5 -21.5l28 -184q49 -16 90 -37l142 107q9 9 24 9q13 0 25 -10q129 -119 165 -170q7 -8 7 -22 -q0 -12 -8 -23q-15 -21 -51 -66.5t-54 -70.5q26 -50 41 -98l183 -28q13 -2 21 -12.5t8 -23.5z" /> - <glyph glyph-name="trash" unicode="" horiz-adv-x="1408" -d="M512 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM768 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1024 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576 -q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1152 76v948h-896v-948q0 -22 7 -40.5t14.5 -27t10.5 -8.5h832q3 0 10.5 8.5t14.5 27t7 40.5zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q0 -14 -9 -23t-23 -9h-96v-948q0 -83 -47 -143.5t-113 -60.5h-832 -q-66 0 -113 58.5t-47 141.5v952h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h309l70 167q15 37 54 63t79 26h320q40 0 79 -26t54 -63l70 -167h309q14 0 23 -9t9 -23z" /> - <glyph glyph-name="home" unicode="" horiz-adv-x="1664" -d="M1408 544v-480q0 -26 -19 -45t-45 -19h-384v384h-256v-384h-384q-26 0 -45 19t-19 45v480q0 1 0.5 3t0.5 3l575 474l575 -474q1 -2 1 -6zM1631 613l-62 -74q-8 -9 -21 -11h-3q-13 0 -21 7l-692 577l-692 -577q-12 -8 -24 -7q-13 2 -21 11l-62 74q-8 10 -7 23.5t11 21.5 -l719 599q32 26 76 26t76 -26l244 -204v195q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-408l219 -182q10 -8 11 -21.5t-7 -23.5z" /> - <glyph glyph-name="file_alt" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -" /> - <glyph glyph-name="time" unicode="" -d="M896 992v-448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 -q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="road" unicode="" horiz-adv-x="1920" -d="M1111 540v4l-24 320q-1 13 -11 22.5t-23 9.5h-186q-13 0 -23 -9.5t-11 -22.5l-24 -320v-4q-1 -12 8 -20t21 -8h244q12 0 21 8t8 20zM1870 73q0 -73 -46 -73h-704q13 0 22 9.5t8 22.5l-20 256q-1 13 -11 22.5t-23 9.5h-272q-13 0 -23 -9.5t-11 -22.5l-20 -256 -q-1 -13 8 -22.5t22 -9.5h-704q-46 0 -46 73q0 54 26 116l417 1044q8 19 26 33t38 14h339q-13 0 -23 -9.5t-11 -22.5l-15 -192q-1 -14 8 -23t22 -9h166q13 0 22 9t8 23l-15 192q-1 13 -11 22.5t-23 9.5h339q20 0 38 -14t26 -33l417 -1044q26 -62 26 -116z" /> - <glyph glyph-name="download_alt" unicode="" horiz-adv-x="1664" -d="M1280 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 416v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h465l135 -136 -q58 -56 136 -56t136 56l136 136h464q40 0 68 -28t28 -68zM1339 985q17 -41 -14 -70l-448 -448q-18 -19 -45 -19t-45 19l-448 448q-31 29 -14 70q17 39 59 39h256v448q0 26 19 45t45 19h256q26 0 45 -19t19 -45v-448h256q42 0 59 -39z" /> - <glyph glyph-name="download" unicode="" -d="M1120 608q0 -12 -10 -24l-319 -319q-11 -9 -23 -9t-23 9l-320 320q-15 16 -7 35q8 20 30 20h192v352q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-352h192q14 0 23 -9t9 -23zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273 -t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="upload" unicode="" -d="M1118 660q-8 -20 -30 -20h-192v-352q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v352h-192q-14 0 -23 9t-9 23q0 12 10 24l319 319q11 9 23 9t23 -9l320 -320q15 -16 7 -35zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198 -t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="inbox" unicode="" -d="M1023 576h316q-1 3 -2.5 8.5t-2.5 7.5l-212 496h-708l-212 -496q-1 -3 -2.5 -8.5t-2.5 -7.5h316l95 -192h320zM1536 546v-482q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v482q0 62 25 123l238 552q10 25 36.5 42t52.5 17h832q26 0 52.5 -17t36.5 -42l238 -552 -q25 -61 25 -123z" /> - <glyph glyph-name="play_circle" unicode="" -d="M1184 640q0 -37 -32 -55l-544 -320q-15 -9 -32 -9q-16 0 -32 8q-32 19 -32 56v640q0 37 32 56q33 18 64 -1l544 -320q32 -18 32 -55zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 -q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="repeat" unicode="" -d="M1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l138 138q-148 137 -349 137q-104 0 -198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5q119 0 225 52t179 147q7 10 23 12q15 0 25 -9 -l137 -138q9 -8 9.5 -20.5t-7.5 -22.5q-109 -132 -264 -204.5t-327 -72.5q-156 0 -298 61t-245 164t-164 245t-61 298t61 298t164 245t245 164t298 61q147 0 284.5 -55.5t244.5 -156.5l130 129q29 31 70 14q39 -17 39 -59z" /> - <glyph glyph-name="refresh" unicode="" -d="M1511 480q0 -5 -1 -7q-64 -268 -268 -434.5t-478 -166.5q-146 0 -282.5 55t-243.5 157l-129 -129q-19 -19 -45 -19t-45 19t-19 45v448q0 26 19 45t45 19h448q26 0 45 -19t19 -45t-19 -45l-137 -137q71 -66 161 -102t187 -36q134 0 250 65t186 179q11 17 53 117 -q8 23 30 23h192q13 0 22.5 -9.5t9.5 -22.5zM1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-26 0 -45 19t-19 45t19 45l138 138q-148 137 -349 137q-134 0 -250 -65t-186 -179q-11 -17 -53 -117q-8 -23 -30 -23h-199q-13 0 -22.5 9.5t-9.5 22.5v7q65 268 270 434.5t480 166.5 -q146 0 284 -55.5t245 -156.5l130 129q19 19 45 19t45 -19t19 -45z" /> - <glyph glyph-name="list_alt" unicode="" horiz-adv-x="1792" -d="M384 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M384 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1536 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5z -M1536 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5zM1536 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5 -t9.5 -22.5zM1664 160v832q0 13 -9.5 22.5t-22.5 9.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5v-832q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1792 1248v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47 -t47 -113z" /> - <glyph glyph-name="lock" unicode="" horiz-adv-x="1152" -d="M320 768h512v192q0 106 -75 181t-181 75t-181 -75t-75 -181v-192zM1152 672v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v192q0 184 132 316t316 132t316 -132t132 -316v-192h32q40 0 68 -28t28 -68z" /> - <glyph glyph-name="flag" unicode="" horiz-adv-x="1792" -d="M320 1280q0 -72 -64 -110v-1266q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v1266q-64 38 -64 110q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -25 -12.5 -38.5t-39.5 -27.5q-215 -116 -369 -116q-61 0 -123.5 22t-108.5 48 -t-115.5 48t-142.5 22q-192 0 -464 -146q-17 -9 -33 -9q-26 0 -45 19t-19 45v742q0 32 31 55q21 14 79 43q236 120 421 120q107 0 200 -29t219 -88q38 -19 88 -19q54 0 117.5 21t110 47t88 47t54.5 21q26 0 45 -19t19 -45z" /> - <glyph glyph-name="headphones" unicode="" horiz-adv-x="1664" -d="M1664 650q0 -166 -60 -314l-20 -49l-185 -33q-22 -83 -90.5 -136.5t-156.5 -53.5v-32q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-32q71 0 130 -35.5t93 -95.5l68 12q29 95 29 193q0 148 -88 279t-236.5 209t-315.5 78 -t-315.5 -78t-236.5 -209t-88 -279q0 -98 29 -193l68 -12q34 60 93 95.5t130 35.5v32q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v32q-88 0 -156.5 53.5t-90.5 136.5l-185 33l-20 49q-60 148 -60 314q0 151 67 291t179 242.5 -t266 163.5t320 61t320 -61t266 -163.5t179 -242.5t67 -291z" /> - <glyph glyph-name="volume_off" unicode="" horiz-adv-x="768" -d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45z" /> - <glyph glyph-name="volume_down" unicode="" horiz-adv-x="1152" -d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 36 -t12 56.5t-12 56.5t-29 36t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142z" /> - <glyph glyph-name="volume_up" unicode="" horiz-adv-x="1664" -d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 36 -t12 56.5t-12 56.5t-29 36t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142zM1408 640q0 -153 -85 -282.5t-225 -188.5q-13 -5 -25 -5q-27 0 -46 19t-19 45q0 39 39 59q56 29 76 44q74 54 115.5 135.5t41.5 173.5t-41.5 173.5 -t-115.5 135.5q-20 15 -76 44q-39 20 -39 59q0 26 19 45t45 19q13 0 26 -5q140 -59 225 -188.5t85 -282.5zM1664 640q0 -230 -127 -422.5t-338 -283.5q-13 -5 -26 -5q-26 0 -45 19t-19 45q0 36 39 59q7 4 22.5 10.5t22.5 10.5q46 25 82 51q123 91 192 227t69 289t-69 289 -t-192 227q-36 26 -82 51q-7 4 -22.5 10.5t-22.5 10.5q-39 23 -39 59q0 26 19 45t45 19q13 0 26 -5q211 -91 338 -283.5t127 -422.5z" /> - <glyph glyph-name="qrcode" unicode="" horiz-adv-x="1408" -d="M384 384v-128h-128v128h128zM384 1152v-128h-128v128h128zM1152 1152v-128h-128v128h128zM128 129h384v383h-384v-383zM128 896h384v384h-384v-384zM896 896h384v384h-384v-384zM640 640v-640h-640v640h640zM1152 128v-128h-128v128h128zM1408 128v-128h-128v128h128z -M1408 640v-384h-384v128h-128v-384h-128v640h384v-128h128v128h128zM640 1408v-640h-640v640h640zM1408 1408v-640h-640v640h640z" /> - <glyph glyph-name="barcode" unicode="" horiz-adv-x="1792" -d="M63 0h-63v1408h63v-1408zM126 1h-32v1407h32v-1407zM220 1h-31v1407h31v-1407zM377 1h-31v1407h31v-1407zM534 1h-62v1407h62v-1407zM660 1h-31v1407h31v-1407zM723 1h-31v1407h31v-1407zM786 1h-31v1407h31v-1407zM943 1h-63v1407h63v-1407zM1100 1h-63v1407h63v-1407z -M1226 1h-63v1407h63v-1407zM1352 1h-63v1407h63v-1407zM1446 1h-63v1407h63v-1407zM1635 1h-94v1407h94v-1407zM1698 1h-32v1407h32v-1407zM1792 0h-63v1408h63v-1408z" /> - <glyph glyph-name="tag" unicode="" -d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 -l715 -714q37 -39 37 -91z" /> - <glyph glyph-name="tags" unicode="" horiz-adv-x="1920" -d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 -l715 -714q37 -39 37 -91zM1899 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-36 0 -59 14t-53 45l470 470q37 37 37 90q0 52 -37 91l-715 714q-38 38 -102 64.5t-117 26.5h224q53 0 117 -26.5t102 -64.5l715 -714q37 -39 37 -91z" /> - <glyph glyph-name="book" unicode="" horiz-adv-x="1664" -d="M1639 1058q40 -57 18 -129l-275 -906q-19 -64 -76.5 -107.5t-122.5 -43.5h-923q-77 0 -148.5 53.5t-99.5 131.5q-24 67 -2 127q0 4 3 27t4 37q1 8 -3 21.5t-3 19.5q2 11 8 21t16.5 23.5t16.5 23.5q23 38 45 91.5t30 91.5q3 10 0.5 30t-0.5 28q3 11 17 28t17 23 -q21 36 42 92t25 90q1 9 -2.5 32t0.5 28q4 13 22 30.5t22 22.5q19 26 42.5 84.5t27.5 96.5q1 8 -3 25.5t-2 26.5q2 8 9 18t18 23t17 21q8 12 16.5 30.5t15 35t16 36t19.5 32t26.5 23.5t36 11.5t47.5 -5.5l-1 -3q38 9 51 9h761q74 0 114 -56t18 -130l-274 -906 -q-36 -119 -71.5 -153.5t-128.5 -34.5h-869q-27 0 -38 -15q-11 -16 -1 -43q24 -70 144 -70h923q29 0 56 15.5t35 41.5l300 987q7 22 5 57q38 -15 59 -43zM575 1056q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5 -t-16.5 -22.5zM492 800q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5t-16.5 -22.5z" /> - <glyph glyph-name="bookmark" unicode="" horiz-adv-x="1280" -d="M1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289q0 34 19.5 62t52.5 41q21 9 44 9h1048z" /> - <glyph glyph-name="print" unicode="" horiz-adv-x="1664" -d="M384 0h896v256h-896v-256zM384 640h896v384h-160q-40 0 -68 28t-28 68v160h-640v-640zM1536 576q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 576v-416q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-160q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68 -v160h-224q-13 0 -22.5 9.5t-9.5 22.5v416q0 79 56.5 135.5t135.5 56.5h64v544q0 40 28 68t68 28h672q40 0 88 -20t76 -48l152 -152q28 -28 48 -76t20 -88v-256h64q79 0 135.5 -56.5t56.5 -135.5z" /> - <glyph glyph-name="camera" unicode="" horiz-adv-x="1920" -d="M960 864q119 0 203.5 -84.5t84.5 -203.5t-84.5 -203.5t-203.5 -84.5t-203.5 84.5t-84.5 203.5t84.5 203.5t203.5 84.5zM1664 1280q106 0 181 -75t75 -181v-896q0 -106 -75 -181t-181 -75h-1408q-106 0 -181 75t-75 181v896q0 106 75 181t181 75h224l51 136 -q19 49 69.5 84.5t103.5 35.5h512q53 0 103.5 -35.5t69.5 -84.5l51 -136h224zM960 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" /> - <glyph glyph-name="font" unicode="" horiz-adv-x="1664" -d="M725 977l-170 -450q33 0 136.5 -2t160.5 -2q19 0 57 2q-87 253 -184 452zM0 -128l2 79q23 7 56 12.5t57 10.5t49.5 14.5t44.5 29t31 50.5l237 616l280 724h75h53q8 -14 11 -21l205 -480q33 -78 106 -257.5t114 -274.5q15 -34 58 -144.5t72 -168.5q20 -45 35 -57 -q19 -15 88 -29.5t84 -20.5q6 -38 6 -57q0 -5 -0.5 -13.5t-0.5 -12.5q-63 0 -190 8t-191 8q-76 0 -215 -7t-178 -8q0 43 4 78l131 28q1 0 12.5 2.5t15.5 3.5t14.5 4.5t15 6.5t11 8t9 11t2.5 14q0 16 -31 96.5t-72 177.5t-42 100l-450 2q-26 -58 -76.5 -195.5t-50.5 -162.5 -q0 -22 14 -37.5t43.5 -24.5t48.5 -13.5t57 -8.5t41 -4q1 -19 1 -58q0 -9 -2 -27q-58 0 -174.5 10t-174.5 10q-8 0 -26.5 -4t-21.5 -4q-80 -14 -188 -14z" /> - <glyph glyph-name="bold" unicode="" horiz-adv-x="1408" -d="M555 15q74 -32 140 -32q376 0 376 335q0 114 -41 180q-27 44 -61.5 74t-67.5 46.5t-80.5 25t-84 10.5t-94.5 2q-73 0 -101 -10q0 -53 -0.5 -159t-0.5 -158q0 -8 -1 -67.5t-0.5 -96.5t4.5 -83.5t12 -66.5zM541 761q42 -7 109 -7q82 0 143 13t110 44.5t74.5 89.5t25.5 142 -q0 70 -29 122.5t-79 82t-108 43.5t-124 14q-50 0 -130 -13q0 -50 4 -151t4 -152q0 -27 -0.5 -80t-0.5 -79q0 -46 1 -69zM0 -128l2 94q15 4 85 16t106 27q7 12 12.5 27t8.5 33.5t5.5 32.5t3 37.5t0.5 34v35.5v30q0 982 -22 1025q-4 8 -22 14.5t-44.5 11t-49.5 7t-48.5 4.5 -t-30.5 3l-4 83q98 2 340 11.5t373 9.5q23 0 68 -0.5t68 -0.5q70 0 136.5 -13t128.5 -42t108 -71t74 -104.5t28 -137.5q0 -52 -16.5 -95.5t-39 -72t-64.5 -57.5t-73 -45t-84 -40q154 -35 256.5 -134t102.5 -248q0 -100 -35 -179.5t-93.5 -130.5t-138 -85.5t-163.5 -48.5 -t-176 -14q-44 0 -132 3t-132 3q-106 0 -307 -11t-231 -12z" /> - <glyph glyph-name="italic" unicode="" horiz-adv-x="1024" -d="M0 -126l17 85q22 7 61.5 16.5t72 19t59.5 23.5q28 35 41 101q1 7 62 289t114 543.5t52 296.5v25q-24 13 -54.5 18.5t-69.5 8t-58 5.5l19 103q33 -2 120 -6.5t149.5 -7t120.5 -2.5q48 0 98.5 2.5t121 7t98.5 6.5q-5 -39 -19 -89q-30 -10 -101.5 -28.5t-108.5 -33.5 -q-8 -19 -14 -42.5t-9 -40t-7.5 -45.5t-6.5 -42q-27 -148 -87.5 -419.5t-77.5 -355.5q-2 -9 -13 -58t-20 -90t-16 -83.5t-6 -57.5l1 -18q17 -4 185 -31q-3 -44 -16 -99q-11 0 -32.5 -1.5t-32.5 -1.5q-29 0 -87 10t-86 10q-138 2 -206 2q-51 0 -143 -9t-121 -11z" /> - <glyph glyph-name="text_height" unicode="" horiz-adv-x="1792" -d="M1744 128q33 0 42 -18.5t-11 -44.5l-126 -162q-20 -26 -49 -26t-49 26l-126 162q-20 26 -11 44.5t42 18.5h80v1024h-80q-33 0 -42 18.5t11 44.5l126 162q20 26 49 26t49 -26l126 -162q20 -26 11 -44.5t-42 -18.5h-80v-1024h80zM81 1407l54 -27q12 -5 211 -5q44 0 132 2 -t132 2q36 0 107.5 -0.5t107.5 -0.5h293q6 0 21 -0.5t20.5 0t16 3t17.5 9t15 17.5l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 48t-14.5 73.5t-7.5 35.5q-6 8 -12 12.5t-15.5 6t-13 2.5t-18 0.5t-16.5 -0.5 -q-17 0 -66.5 0.5t-74.5 0.5t-64 -2t-71 -6q-9 -81 -8 -136q0 -94 2 -388t2 -455q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 -q19 42 19 383q0 101 -3 303t-3 303v117q0 2 0.5 15.5t0.5 25t-1 25.5t-3 24t-5 14q-11 12 -162 12q-33 0 -93 -12t-80 -26q-19 -13 -34 -72.5t-31.5 -111t-42.5 -53.5q-42 26 -56 44v383z" /> - <glyph glyph-name="text_width" unicode="" -d="M81 1407l54 -27q12 -5 211 -5q44 0 132 2t132 2q70 0 246.5 1t304.5 0.5t247 -4.5q33 -1 56 31l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 47.5t-15 73.5t-7 36q-10 13 -27 19q-5 2 -66 2q-30 0 -93 1t-103 1 -t-94 -2t-96 -7q-9 -81 -8 -136l1 -152v52q0 -55 1 -154t1.5 -180t0.5 -153q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 -q7 16 11.5 74t6 145.5t1.5 155t-0.5 153.5t-0.5 89q0 7 -2.5 21.5t-2.5 22.5q0 7 0.5 44t1 73t0 76.5t-3 67.5t-6.5 32q-11 12 -162 12q-41 0 -163 -13.5t-138 -24.5q-19 -12 -34 -71.5t-31.5 -111.5t-42.5 -54q-42 26 -56 44v383zM1310 125q12 0 42 -19.5t57.5 -41.5 -t59.5 -49t36 -30q26 -21 26 -49t-26 -49q-4 -3 -36 -30t-59.5 -49t-57.5 -41.5t-42 -19.5q-13 0 -20.5 10.5t-10 28.5t-2.5 33.5t1.5 33t1.5 19.5h-1024q0 -2 1.5 -19.5t1.5 -33t-2.5 -33.5t-10 -28.5t-20.5 -10.5q-12 0 -42 19.5t-57.5 41.5t-59.5 49t-36 30q-26 21 -26 49 -t26 49q4 3 36 30t59.5 49t57.5 41.5t42 19.5q13 0 20.5 -10.5t10 -28.5t2.5 -33.5t-1.5 -33t-1.5 -19.5h1024q0 2 -1.5 19.5t-1.5 33t2.5 33.5t10 28.5t20.5 10.5z" /> - <glyph glyph-name="align_left" unicode="" horiz-adv-x="1792" -d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45 -t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" /> - <glyph glyph-name="align_center" unicode="" horiz-adv-x="1792" -d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h896q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45t-45 -19 -h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h640q26 0 45 -19t19 -45z" /> - <glyph glyph-name="align_right" unicode="" horiz-adv-x="1792" -d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 -t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" /> - <glyph glyph-name="align_justify" unicode="" horiz-adv-x="1792" -d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 -t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" /> - <glyph glyph-name="list" unicode="" horiz-adv-x="1792" -d="M256 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM256 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5 -t9.5 -22.5zM256 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344 -q13 0 22.5 -9.5t9.5 -22.5zM256 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5 -t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192 -q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5z" /> - <glyph glyph-name="indent_left" unicode="" horiz-adv-x="1792" -d="M384 992v-576q0 -13 -9.5 -22.5t-22.5 -9.5q-14 0 -23 9l-288 288q-9 9 -9 23t9 23l288 288q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 -t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 -q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" /> - <glyph glyph-name="indent_right" unicode="" horiz-adv-x="1792" -d="M352 704q0 -14 -9 -23l-288 -288q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v576q0 13 9.5 22.5t22.5 9.5q14 0 23 -9l288 -288q9 -9 9 -23zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 -t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 -q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" /> - <glyph glyph-name="facetime_video" unicode="" horiz-adv-x="1792" -d="M1792 1184v-1088q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-403 403v-166q0 -119 -84.5 -203.5t-203.5 -84.5h-704q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h704q119 0 203.5 -84.5t84.5 -203.5v-165l403 402q18 19 45 19q12 0 25 -5 -q39 -17 39 -59z" /> - <glyph glyph-name="picture" unicode="" horiz-adv-x="1920" -d="M640 960q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 576v-448h-1408v192l320 320l160 -160l512 512zM1760 1280h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5v1216 -q0 13 -9.5 22.5t-22.5 9.5zM1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" /> - <glyph glyph-name="pencil" unicode="" -d="M363 0l91 91l-235 235l-91 -91v-107h128v-128h107zM886 928q0 22 -22 22q-10 0 -17 -7l-542 -542q-7 -7 -7 -17q0 -22 22 -22q10 0 17 7l542 542q7 7 7 17zM832 1120l416 -416l-832 -832h-416v416zM1515 1024q0 -53 -37 -90l-166 -166l-416 416l166 165q36 38 90 38 -q53 0 91 -38l235 -234q37 -39 37 -91z" /> - <glyph glyph-name="map_marker" unicode="" horiz-adv-x="1024" -d="M768 896q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1024 896q0 -109 -33 -179l-364 -774q-16 -33 -47.5 -52t-67.5 -19t-67.5 19t-46.5 52l-365 774q-33 70 -33 179q0 212 150 362t362 150t362 -150t150 -362z" /> - <glyph glyph-name="adjust" unicode="" -d="M768 96v1088q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="tint" unicode="" horiz-adv-x="1024" -d="M512 384q0 36 -20 69q-1 1 -15.5 22.5t-25.5 38t-25 44t-21 50.5q-4 16 -21 16t-21 -16q-7 -23 -21 -50.5t-25 -44t-25.5 -38t-15.5 -22.5q-20 -33 -20 -69q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 512q0 -212 -150 -362t-362 -150t-362 150t-150 362 -q0 145 81 275q6 9 62.5 90.5t101 151t99.5 178t83 201.5q9 30 34 47t51 17t51.5 -17t33.5 -47q28 -93 83 -201.5t99.5 -178t101 -151t62.5 -90.5q81 -127 81 -275z" /> - <glyph glyph-name="edit" unicode="" horiz-adv-x="1792" -d="M888 352l116 116l-152 152l-116 -116v-56h96v-96h56zM1328 1072q-16 16 -33 -1l-350 -350q-17 -17 -1 -33t33 1l350 350q17 17 1 33zM1408 478v-190q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 -q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-14 -14 -32 -8q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v126q0 13 9 22l64 64q15 15 35 7t20 -29zM1312 1216l288 -288l-672 -672h-288v288zM1756 1084l-92 -92 -l-288 288l92 92q28 28 68 28t68 -28l152 -152q28 -28 28 -68t-28 -68z" /> - <glyph glyph-name="share" unicode="" horiz-adv-x="1664" -d="M1408 547v-259q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h255v0q13 0 22.5 -9.5t9.5 -22.5q0 -27 -26 -32q-77 -26 -133 -60q-10 -4 -16 -4h-112q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832 -q66 0 113 47t47 113v214q0 19 18 29q28 13 54 37q16 16 35 8q21 -9 21 -29zM1645 1043l-384 -384q-18 -19 -45 -19q-12 0 -25 5q-39 17 -39 59v192h-160q-323 0 -438 -131q-119 -137 -74 -473q3 -23 -20 -34q-8 -2 -12 -2q-16 0 -26 13q-10 14 -21 31t-39.5 68.5t-49.5 99.5 -t-38.5 114t-17.5 122q0 49 3.5 91t14 90t28 88t47 81.5t68.5 74t94.5 61.5t124.5 48.5t159.5 30.5t196.5 11h160v192q0 42 39 59q13 5 25 5q26 0 45 -19l384 -384q19 -19 19 -45t-19 -45z" /> - <glyph glyph-name="check" unicode="" horiz-adv-x="1664" -d="M1408 606v-318q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-10 -10 -23 -10q-3 0 -9 2q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832 -q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v254q0 13 9 22l64 64q10 10 23 10q6 0 12 -3q20 -8 20 -29zM1639 1095l-814 -814q-24 -24 -57 -24t-57 24l-430 430q-24 24 -24 57t24 57l110 110q24 24 57 24t57 -24l263 -263l647 647q24 24 57 24t57 -24l110 -110 -q24 -24 24 -57t-24 -57z" /> - <glyph glyph-name="move" unicode="" horiz-adv-x="1792" -d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-384v-384h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v384h-384v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45 -t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h384v384h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45t-19 -45t-45 -19h-128v-384h384v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" /> - <glyph glyph-name="step_backward" unicode="" horiz-adv-x="1024" -d="M979 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 10 13 19z" /> - <glyph glyph-name="fast_backward" unicode="" horiz-adv-x="1792" -d="M1747 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 10 13 19l710 710 -q19 19 32 13t13 -32v-710q4 10 13 19z" /> - <glyph glyph-name="backward" unicode="" horiz-adv-x="1664" -d="M1619 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-19 19 -19 45t19 45l710 710q19 19 32 13t13 -32v-710q4 10 13 19z" /> - <glyph glyph-name="play" unicode="" horiz-adv-x="1408" -d="M1384 609l-1328 -738q-23 -13 -39.5 -3t-16.5 36v1472q0 26 16.5 36t39.5 -3l1328 -738q23 -13 23 -31t-23 -31z" /> - <glyph glyph-name="pause" unicode="" -d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45zM640 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45z" /> - <glyph glyph-name="stop" unicode="" -d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" /> - <glyph glyph-name="forward" unicode="" horiz-adv-x="1664" -d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q9 -9 13 -19v710q0 26 13 32t32 -13l710 -710q19 -19 19 -45t-19 -45l-710 -710q-19 -19 -32 -13t-13 32v710q-4 -10 -13 -19z" /> - <glyph glyph-name="fast_forward" unicode="" horiz-adv-x="1792" -d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q9 -9 13 -19v710q0 26 13 32t32 -13l710 -710q9 -9 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-4 -10 -13 -19l-710 -710 -q-19 -19 -32 -13t-13 32v710q-4 -10 -13 -19z" /> - <glyph glyph-name="step_forward" unicode="" horiz-adv-x="1024" -d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q9 -9 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-4 -10 -13 -19z" /> - <glyph glyph-name="eject" unicode="" horiz-adv-x="1538" -d="M14 557l710 710q19 19 45 19t45 -19l710 -710q19 -19 13 -32t-32 -13h-1472q-26 0 -32 13t13 32zM1473 0h-1408q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1408q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19z" /> - <glyph glyph-name="chevron_left" unicode="" horiz-adv-x="1280" -d="M1171 1235l-531 -531l531 -531q19 -19 19 -45t-19 -45l-166 -166q-19 -19 -45 -19t-45 19l-742 742q-19 19 -19 45t19 45l742 742q19 19 45 19t45 -19l166 -166q19 -19 19 -45t-19 -45z" /> - <glyph glyph-name="chevron_right" unicode="" horiz-adv-x="1280" -d="M1107 659l-742 -742q-19 -19 -45 -19t-45 19l-166 166q-19 19 -19 45t19 45l531 531l-531 531q-19 19 -19 45t19 45l166 166q19 19 45 19t45 -19l742 -742q19 -19 19 -45t-19 -45z" /> - <glyph glyph-name="plus_sign" unicode="" -d="M1216 576v128q0 26 -19 45t-45 19h-256v256q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-256h-256q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h256v-256q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v256h256q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5 -t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="minus_sign" unicode="" -d="M1216 576v128q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 -t103 -385.5z" /> - <glyph glyph-name="remove_sign" unicode="" -d="M1149 414q0 26 -19 45l-181 181l181 181q19 19 19 45q0 27 -19 46l-90 90q-19 19 -46 19q-26 0 -45 -19l-181 -181l-181 181q-19 19 -45 19q-27 0 -46 -19l-90 -90q-19 -19 -19 -46q0 -26 19 -45l181 -181l-181 -181q-19 -19 -19 -45q0 -27 19 -46l90 -90q19 -19 46 -19 -q26 0 45 19l181 181l181 -181q19 -19 45 -19q27 0 46 19l90 90q19 19 19 46zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="ok_sign" unicode="" -d="M1284 802q0 28 -18 46l-91 90q-19 19 -45 19t-45 -19l-408 -407l-226 226q-19 19 -45 19t-45 -19l-91 -90q-18 -18 -18 -46q0 -27 18 -45l362 -362q19 -19 45 -19q27 0 46 19l543 543q18 18 18 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 -t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="question_sign" unicode="" -d="M896 160v192q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h192q14 0 23 9t9 23zM1152 832q0 88 -55.5 163t-138.5 116t-170 41q-243 0 -371 -213q-15 -24 8 -42l132 -100q7 -6 19 -6q16 0 25 12q53 68 86 92q34 24 86 24q48 0 85.5 -26t37.5 -59 -q0 -38 -20 -61t-68 -45q-63 -28 -115.5 -86.5t-52.5 -125.5v-36q0 -14 9 -23t23 -9h192q14 0 23 9t9 23q0 19 21.5 49.5t54.5 49.5q32 18 49 28.5t46 35t44.5 48t28 60.5t12.5 81zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 -t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="info_sign" unicode="" -d="M1024 160v160q0 14 -9 23t-23 9h-96v512q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h96v-320h-96q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h448q14 0 23 9t9 23zM896 1056v160q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23 -t23 -9h192q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="screenshot" unicode="" -d="M1197 512h-109q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h109q-32 108 -112.5 188.5t-188.5 112.5v-109q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v109q-108 -32 -188.5 -112.5t-112.5 -188.5h109q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-109 -q32 -108 112.5 -188.5t188.5 -112.5v109q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-109q108 32 188.5 112.5t112.5 188.5zM1536 704v-128q0 -26 -19 -45t-45 -19h-143q-37 -161 -154.5 -278.5t-278.5 -154.5v-143q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v143 -q-161 37 -278.5 154.5t-154.5 278.5h-143q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h143q37 161 154.5 278.5t278.5 154.5v143q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-143q161 -37 278.5 -154.5t154.5 -278.5h143q26 0 45 -19t19 -45z" /> - <glyph glyph-name="remove_circle" unicode="" -d="M1097 457l-146 -146q-10 -10 -23 -10t-23 10l-137 137l-137 -137q-10 -10 -23 -10t-23 10l-146 146q-10 10 -10 23t10 23l137 137l-137 137q-10 10 -10 23t10 23l146 146q10 10 23 10t23 -10l137 -137l137 137q10 10 23 10t23 -10l146 -146q10 -10 10 -23t-10 -23 -l-137 -137l137 -137q10 -10 10 -23t-10 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5 -t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="ok_circle" unicode="" -d="M1171 723l-422 -422q-19 -19 -45 -19t-45 19l-294 294q-19 19 -19 45t19 45l102 102q19 19 45 19t45 -19l147 -147l275 275q19 19 45 19t45 -19l102 -102q19 -19 19 -45t-19 -45zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198 -t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="ban_circle" unicode="" -d="M1312 643q0 161 -87 295l-754 -753q137 -89 297 -89q111 0 211.5 43.5t173.5 116.5t116 174.5t43 212.5zM313 344l755 754q-135 91 -300 91q-148 0 -273 -73t-198 -199t-73 -274q0 -162 89 -299zM1536 643q0 -157 -61 -300t-163.5 -246t-245 -164t-298.5 -61t-298.5 61 -t-245 164t-163.5 246t-61 300t61 299.5t163.5 245.5t245 164t298.5 61t298.5 -61t245 -164t163.5 -245.5t61 -299.5z" /> - <glyph glyph-name="arrow_left" unicode="" -d="M1536 640v-128q0 -53 -32.5 -90.5t-84.5 -37.5h-704l293 -294q38 -36 38 -90t-38 -90l-75 -76q-37 -37 -90 -37q-52 0 -91 37l-651 652q-37 37 -37 90q0 52 37 91l651 650q38 38 91 38q52 0 90 -38l75 -74q38 -38 38 -91t-38 -91l-293 -293h704q52 0 84.5 -37.5 -t32.5 -90.5z" /> - <glyph glyph-name="arrow_right" unicode="" -d="M1472 576q0 -54 -37 -91l-651 -651q-39 -37 -91 -37q-51 0 -90 37l-75 75q-38 38 -38 91t38 91l293 293h-704q-52 0 -84.5 37.5t-32.5 90.5v128q0 53 32.5 90.5t84.5 37.5h704l-293 294q-38 36 -38 90t38 90l75 75q38 38 90 38q53 0 91 -38l651 -651q37 -35 37 -90z" /> - <glyph glyph-name="arrow_up" unicode="" horiz-adv-x="1664" -d="M1611 565q0 -51 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-294 293v-704q0 -52 -37.5 -84.5t-90.5 -32.5h-128q-53 0 -90.5 32.5t-37.5 84.5v704l-294 -293q-36 -38 -90 -38t-90 38l-75 75q-38 38 -38 90q0 53 38 91l651 651q35 37 90 37q54 0 91 -37l651 -651 -q37 -39 37 -91z" /> - <glyph glyph-name="arrow_down" unicode="" horiz-adv-x="1664" -d="M1611 704q0 -53 -37 -90l-651 -652q-39 -37 -91 -37q-53 0 -90 37l-651 652q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l294 -294v704q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-704l294 294q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" /> - <glyph glyph-name="share_alt" unicode="" horiz-adv-x="1792" -d="M1792 896q0 -26 -19 -45l-512 -512q-19 -19 -45 -19t-45 19t-19 45v256h-224q-98 0 -175.5 -6t-154 -21.5t-133 -42.5t-105.5 -69.5t-80 -101t-48.5 -138.5t-17.5 -181q0 -55 5 -123q0 -6 2.5 -23.5t2.5 -26.5q0 -15 -8.5 -25t-23.5 -10q-16 0 -28 17q-7 9 -13 22 -t-13.5 30t-10.5 24q-127 285 -127 451q0 199 53 333q162 403 875 403h224v256q0 26 19 45t45 19t45 -19l512 -512q19 -19 19 -45z" /> - <glyph glyph-name="resize_full" unicode="" -d="M755 480q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23zM1536 1344v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332 -q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45z" /> - <glyph glyph-name="resize_small" unicode="" -d="M768 576v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45zM1523 1248q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45 -t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23z" /> - <glyph glyph-name="plus" unicode="" horiz-adv-x="1408" -d="M1408 800v-192q0 -40 -28 -68t-68 -28h-416v-416q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v416h-416q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h416v416q0 40 28 68t68 28h192q40 0 68 -28t28 -68v-416h416q40 0 68 -28t28 -68z" /> - <glyph glyph-name="minus" unicode="" horiz-adv-x="1408" -d="M1408 800v-192q0 -40 -28 -68t-68 -28h-1216q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h1216q40 0 68 -28t28 -68z" /> - <glyph glyph-name="asterisk" unicode="" horiz-adv-x="1664" -d="M1482 486q46 -26 59.5 -77.5t-12.5 -97.5l-64 -110q-26 -46 -77.5 -59.5t-97.5 12.5l-266 153v-307q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v307l-266 -153q-46 -26 -97.5 -12.5t-77.5 59.5l-64 110q-26 46 -12.5 97.5t59.5 77.5l266 154l-266 154 -q-46 26 -59.5 77.5t12.5 97.5l64 110q26 46 77.5 59.5t97.5 -12.5l266 -153v307q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-307l266 153q46 26 97.5 12.5t77.5 -59.5l64 -110q26 -46 12.5 -97.5t-59.5 -77.5l-266 -154z" /> - <glyph glyph-name="exclamation_sign" unicode="" -d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM896 161v190q0 14 -9 23.5t-22 9.5h-192q-13 0 -23 -10t-10 -23v-190q0 -13 10 -23t23 -10h192 -q13 0 22 9.5t9 23.5zM894 505l18 621q0 12 -10 18q-10 8 -24 8h-220q-14 0 -24 -8q-10 -6 -10 -18l17 -621q0 -10 10 -17.5t24 -7.5h185q14 0 23.5 7.5t10.5 17.5z" /> - <glyph glyph-name="gift" unicode="" -d="M928 180v56v468v192h-320v-192v-468v-56q0 -25 18 -38.5t46 -13.5h192q28 0 46 13.5t18 38.5zM472 1024h195l-126 161q-26 31 -69 31q-40 0 -68 -28t-28 -68t28 -68t68 -28zM1160 1120q0 40 -28 68t-68 28q-43 0 -69 -31l-125 -161h194q40 0 68 28t28 68zM1536 864v-320 -q0 -14 -9 -23t-23 -9h-96v-416q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v416h-96q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h440q-93 0 -158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5q107 0 168 -77l128 -165l128 165q61 77 168 77q93 0 158.5 -65.5t65.5 -158.5 -t-65.5 -158.5t-158.5 -65.5h440q14 0 23 -9t9 -23z" /> - <glyph glyph-name="leaf" unicode="" horiz-adv-x="1792" -d="M1280 832q0 26 -19 45t-45 19q-172 0 -318 -49.5t-259.5 -134t-235.5 -219.5q-19 -21 -19 -45q0 -26 19 -45t45 -19q24 0 45 19q27 24 74 71t67 66q137 124 268.5 176t313.5 52q26 0 45 19t19 45zM1792 1030q0 -95 -20 -193q-46 -224 -184.5 -383t-357.5 -268 -q-214 -108 -438 -108q-148 0 -286 47q-15 5 -88 42t-96 37q-16 0 -39.5 -32t-45 -70t-52.5 -70t-60 -32q-43 0 -63.5 17.5t-45.5 59.5q-2 4 -6 11t-5.5 10t-3 9.5t-1.5 13.5q0 35 31 73.5t68 65.5t68 56t31 48q0 4 -14 38t-16 44q-9 51 -9 104q0 115 43.5 220t119 184.5 -t170.5 139t204 95.5q55 18 145 25.5t179.5 9t178.5 6t163.5 24t113.5 56.5l29.5 29.5t29.5 28t27 20t36.5 16t43.5 4.5q39 0 70.5 -46t47.5 -112t24 -124t8 -96z" /> - <glyph glyph-name="fire" unicode="" horiz-adv-x="1408" -d="M1408 -160v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1152 896q0 -78 -24.5 -144t-64 -112.5t-87.5 -88t-96 -77.5t-87.5 -72t-64 -81.5t-24.5 -96.5q0 -96 67 -224l-4 1l1 -1 -q-90 41 -160 83t-138.5 100t-113.5 122.5t-72.5 150.5t-27.5 184q0 78 24.5 144t64 112.5t87.5 88t96 77.5t87.5 72t64 81.5t24.5 96.5q0 94 -66 224l3 -1l-1 1q90 -41 160 -83t138.5 -100t113.5 -122.5t72.5 -150.5t27.5 -184z" /> - <glyph glyph-name="eye_open" unicode="" horiz-adv-x="1792" -d="M1664 576q-152 236 -381 353q61 -104 61 -225q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 121 61 225q-229 -117 -381 -353q133 -205 333.5 -326.5t434.5 -121.5t434.5 121.5t333.5 326.5zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5 -t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1792 576q0 -34 -20 -69q-140 -230 -376.5 -368.5t-499.5 -138.5t-499.5 139t-376.5 368q-20 35 -20 69t20 69q140 229 376.5 368t499.5 139t499.5 -139t376.5 -368q20 -35 20 -69z" /> - <glyph glyph-name="eye_close" unicode="" horiz-adv-x="1792" -d="M555 201l78 141q-87 63 -136 159t-49 203q0 121 61 225q-229 -117 -381 -353q167 -258 427 -375zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1307 1151q0 -7 -1 -9 -q-106 -189 -316 -567t-315 -566l-49 -89q-10 -16 -28 -16q-12 0 -134 70q-16 10 -16 28q0 12 44 87q-143 65 -263.5 173t-208.5 245q-20 31 -20 69t20 69q153 235 380 371t496 136q89 0 180 -17l54 97q10 16 28 16q5 0 18 -6t31 -15.5t33 -18.5t31.5 -18.5t19.5 -11.5 -q16 -10 16 -27zM1344 704q0 -139 -79 -253.5t-209 -164.5l280 502q8 -45 8 -84zM1792 576q0 -35 -20 -69q-39 -64 -109 -145q-150 -172 -347.5 -267t-419.5 -95l74 132q212 18 392.5 137t301.5 307q-115 179 -282 294l63 112q95 -64 182.5 -153t144.5 -184q20 -34 20 -69z -" /> - <glyph glyph-name="warning_sign" unicode="" horiz-adv-x="1792" -d="M1024 161v190q0 14 -9.5 23.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -23.5v-190q0 -14 9.5 -23.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 23.5zM1022 535l18 459q0 12 -10 19q-13 11 -24 11h-220q-11 0 -24 -11q-10 -7 -10 -21l17 -457q0 -10 10 -16.5t24 -6.5h185 -q14 0 23.5 6.5t10.5 16.5zM1008 1469l768 -1408q35 -63 -2 -126q-17 -29 -46.5 -46t-63.5 -17h-1536q-34 0 -63.5 17t-46.5 46q-37 63 -2 126l768 1408q17 31 47 49t65 18t65 -18t47 -49z" /> - <glyph glyph-name="plane" unicode="" horiz-adv-x="1408" -d="M1376 1376q44 -52 12 -148t-108 -172l-161 -161l160 -696q5 -19 -12 -33l-128 -96q-7 -6 -19 -6q-4 0 -7 1q-15 3 -21 16l-279 508l-259 -259l53 -194q5 -17 -8 -31l-96 -96q-9 -9 -23 -9h-2q-15 2 -24 13l-189 252l-252 189q-11 7 -13 23q-1 13 9 25l96 97q9 9 23 9 -q6 0 8 -1l194 -53l259 259l-508 279q-14 8 -17 24q-2 16 9 27l128 128q14 13 30 8l665 -159l160 160q76 76 172 108t148 -12z" /> - <glyph glyph-name="calendar" unicode="" horiz-adv-x="1664" -d="M128 -128h288v288h-288v-288zM480 -128h320v288h-320v-288zM128 224h288v320h-288v-320zM480 224h320v320h-320v-320zM128 608h288v288h-288v-288zM864 -128h320v288h-320v-288zM480 608h320v288h-320v-288zM1248 -128h288v288h-288v-288zM864 224h320v320h-320v-320z -M512 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1248 224h288v320h-288v-320zM864 608h320v288h-320v-288zM1248 608h288v288h-288v-288zM1280 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64 -q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1664 1152v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47 -h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" /> - <glyph glyph-name="random" unicode="" horiz-adv-x="1792" -d="M666 1055q-60 -92 -137 -273q-22 45 -37 72.5t-40.5 63.5t-51 56.5t-63 35t-81.5 14.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q250 0 410 -225zM1792 256q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192q-32 0 -85 -0.5t-81 -1t-73 1 -t-71 5t-64 10.5t-63 18.5t-58 28.5t-59 40t-55 53.5t-56 69.5q59 93 136 273q22 -45 37 -72.5t40.5 -63.5t51 -56.5t63 -35t81.5 -14.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1792 1152q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5 -v192h-256q-48 0 -87 -15t-69 -45t-51 -61.5t-45 -77.5q-32 -62 -78 -171q-29 -66 -49.5 -111t-54 -105t-64 -100t-74 -83t-90 -68.5t-106.5 -42t-128 -16.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q48 0 87 15t69 45t51 61.5t45 77.5q32 62 78 171q29 66 49.5 111 -t54 105t64 100t74 83t90 68.5t106.5 42t128 16.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" /> - <glyph glyph-name="comment" unicode="" horiz-adv-x="1792" -d="M1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22q-17 -2 -30.5 9t-17.5 29v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281 -q0 130 71 248.5t191 204.5t286 136.5t348 50.5q244 0 450 -85.5t326 -233t120 -321.5z" /> - <glyph glyph-name="magnet" unicode="" -d="M1536 704v-128q0 -201 -98.5 -362t-274 -251.5t-395.5 -90.5t-395.5 90.5t-274 251.5t-98.5 362v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-128q0 -52 23.5 -90t53.5 -57t71 -30t64 -13t44 -2t44 2t64 13t71 30t53.5 57t23.5 90v128q0 26 19 45t45 19h384 -q26 0 45 -19t19 -45zM512 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45zM1536 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45z" /> - <glyph glyph-name="chevron_up" unicode="" horiz-adv-x="1792" -d="M1683 205l-166 -165q-19 -19 -45 -19t-45 19l-531 531l-531 -531q-19 -19 -45 -19t-45 19l-166 165q-19 19 -19 45.5t19 45.5l742 741q19 19 45 19t45 -19l742 -741q19 -19 19 -45.5t-19 -45.5z" /> - <glyph glyph-name="chevron_down" unicode="" horiz-adv-x="1792" -d="M1683 728l-742 -741q-19 -19 -45 -19t-45 19l-742 741q-19 19 -19 45.5t19 45.5l166 165q19 19 45 19t45 -19l531 -531l531 531q19 19 45 19t45 -19l166 -165q19 -19 19 -45.5t-19 -45.5z" /> - <glyph glyph-name="retweet" unicode="" horiz-adv-x="1920" -d="M1280 32q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-8 0 -13.5 2t-9 7t-5.5 8t-3 11.5t-1 11.5v13v11v160v416h-192q-26 0 -45 19t-19 45q0 24 15 41l320 384q19 22 49 22t49 -22l320 -384q15 -17 15 -41q0 -26 -19 -45t-45 -19h-192v-384h576q16 0 25 -11l160 -192q7 -10 7 -21 -zM1920 448q0 -24 -15 -41l-320 -384q-20 -23 -49 -23t-49 23l-320 384q-15 17 -15 41q0 26 19 45t45 19h192v384h-576q-16 0 -25 12l-160 192q-7 9 -7 20q0 13 9.5 22.5t22.5 9.5h960q8 0 13.5 -2t9 -7t5.5 -8t3 -11.5t1 -11.5v-13v-11v-160v-416h192q26 0 45 -19t19 -45z -" /> - <glyph glyph-name="shopping_cart" unicode="" horiz-adv-x="1664" -d="M640 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1536 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1664 1088v-512q0 -24 -16.5 -42.5t-40.5 -21.5l-1044 -122q13 -60 13 -70q0 -16 -24 -64h920q26 0 45 -19t19 -45 -t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 11 8 31.5t16 36t21.5 40t15.5 29.5l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t19.5 -15.5t13 -24.5t8 -26t5.5 -29.5t4.5 -26h1201q26 0 45 -19t19 -45z" /> - <glyph glyph-name="folder_close" unicode="" horiz-adv-x="1664" -d="M1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" /> - <glyph glyph-name="folder_open" unicode="" horiz-adv-x="1920" -d="M1879 584q0 -31 -31 -66l-336 -396q-43 -51 -120.5 -86.5t-143.5 -35.5h-1088q-34 0 -60.5 13t-26.5 43q0 31 31 66l336 396q43 51 120.5 86.5t143.5 35.5h1088q34 0 60.5 -13t26.5 -43zM1536 928v-160h-832q-94 0 -197 -47.5t-164 -119.5l-337 -396l-5 -6q0 4 -0.5 12.5 -t-0.5 12.5v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158z" /> - <glyph glyph-name="resize_vertical" unicode="" horiz-adv-x="768" -d="M704 1216q0 -26 -19 -45t-45 -19h-128v-1024h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v1024h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45z" /> - <glyph glyph-name="resize_horizontal" unicode="" horiz-adv-x="1792" -d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-1024v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h1024v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" /> - <glyph glyph-name="bar_chart" unicode="" horiz-adv-x="2048" -d="M640 640v-512h-256v512h256zM1024 1152v-1024h-256v1024h256zM2048 0v-128h-2048v1536h128v-1408h1920zM1408 896v-768h-256v768h256zM1792 1280v-1152h-256v1152h256z" /> - <glyph glyph-name="twitter_sign" unicode="" -d="M1280 926q-56 -25 -121 -34q68 40 93 117q-65 -38 -134 -51q-61 66 -153 66q-87 0 -148.5 -61.5t-61.5 -148.5q0 -29 5 -48q-129 7 -242 65t-192 155q-29 -50 -29 -106q0 -114 91 -175q-47 1 -100 26v-2q0 -75 50 -133.5t123 -72.5q-29 -8 -51 -8q-13 0 -39 4 -q21 -63 74.5 -104t121.5 -42q-116 -90 -261 -90q-26 0 -50 3q148 -94 322 -94q112 0 210 35.5t168 95t120.5 137t75 162t24.5 168.5q0 18 -1 27q63 45 105 109zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5 -t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="facebook_sign" unicode="" -d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-188v595h199l30 232h-229v148q0 56 23.5 84t91.5 28l122 1v207q-63 9 -178 9q-136 0 -217.5 -80t-81.5 -226v-171h-200v-232h200v-595h-532q-119 0 -203.5 84.5t-84.5 203.5v960 -q0 119 84.5 203.5t203.5 84.5h960z" /> - <glyph glyph-name="camera_retro" unicode="" horiz-adv-x="1792" -d="M928 704q0 14 -9 23t-23 9q-66 0 -113 -47t-47 -113q0 -14 9 -23t23 -9t23 9t9 23q0 40 28 68t68 28q14 0 23 9t9 23zM1152 574q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM128 0h1536v128h-1536v-128zM1280 574q0 159 -112.5 271.5 -t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM256 1216h384v128h-384v-128zM128 1024h1536v118v138h-828l-64 -128h-644v-128zM1792 1280v-1280q0 -53 -37.5 -90.5t-90.5 -37.5h-1536q-53 0 -90.5 37.5t-37.5 90.5v1280 -q0 53 37.5 90.5t90.5 37.5h1536q53 0 90.5 -37.5t37.5 -90.5z" /> - <glyph glyph-name="key" unicode="" horiz-adv-x="1792" -d="M832 1024q0 80 -56 136t-136 56t-136 -56t-56 -136q0 -42 19 -83q-41 19 -83 19q-80 0 -136 -56t-56 -136t56 -136t136 -56t136 56t56 136q0 42 -19 83q41 -19 83 -19q80 0 136 56t56 136zM1683 320q0 -17 -49 -66t-66 -49q-9 0 -28.5 16t-36.5 33t-38.5 40t-24.5 26 -l-96 -96l220 -220q28 -28 28 -68q0 -42 -39 -81t-81 -39q-40 0 -68 28l-671 671q-176 -131 -365 -131q-163 0 -265.5 102.5t-102.5 265.5q0 160 95 313t248 248t313 95q163 0 265.5 -102.5t102.5 -265.5q0 -189 -131 -365l355 -355l96 96q-3 3 -26 24.5t-40 38.5t-33 36.5 -t-16 28.5q0 17 49 66t66 49q13 0 23 -10q6 -6 46 -44.5t82 -79.5t86.5 -86t73 -78t28.5 -41z" /> - <glyph glyph-name="cogs" unicode="" horiz-adv-x="1920" -d="M896 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1664 128q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1152q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5 -t90.5 37.5t37.5 90.5zM1280 731v-185q0 -10 -7 -19.5t-16 -10.5l-155 -24q-11 -35 -32 -76q34 -48 90 -115q7 -11 7 -20q0 -12 -7 -19q-23 -30 -82.5 -89.5t-78.5 -59.5q-11 0 -21 7l-115 90q-37 -19 -77 -31q-11 -108 -23 -155q-7 -24 -30 -24h-186q-11 0 -20 7.5t-10 17.5 -l-23 153q-34 10 -75 31l-118 -89q-7 -7 -20 -7q-11 0 -21 8q-144 133 -144 160q0 9 7 19q10 14 41 53t47 61q-23 44 -35 82l-152 24q-10 1 -17 9.5t-7 19.5v185q0 10 7 19.5t16 10.5l155 24q11 35 32 76q-34 48 -90 115q-7 11 -7 20q0 12 7 20q22 30 82 89t79 59q11 0 21 -7 -l115 -90q34 18 77 32q11 108 23 154q7 24 30 24h186q11 0 20 -7.5t10 -17.5l23 -153q34 -10 75 -31l118 89q8 7 20 7q11 0 21 -8q144 -133 144 -160q0 -8 -7 -19q-12 -16 -42 -54t-45 -60q23 -48 34 -82l152 -23q10 -2 17 -10.5t7 -19.5zM1920 198v-140q0 -16 -149 -31 -q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20 -t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31zM1920 1222v-140q0 -16 -149 -31q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68 -q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70 -q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31z" /> - <glyph glyph-name="comments" unicode="" horiz-adv-x="1792" -d="M1408 768q0 -139 -94 -257t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q-124 72 -195 177t-71 224 -q0 139 94 257t256.5 186.5t353.5 68.5t353.5 -68.5t256.5 -186.5t94 -257zM1792 512q0 -120 -71 -224.5t-195 -176.5q10 -24 20.5 -44t25 -38.5t22.5 -29t26 -29.5t23 -25q1 -1 4 -4.5t4.5 -5t4 -5t3.5 -5.5l2.5 -5t2 -6t0.5 -6.5t-1 -6.5q-3 -14 -13 -22t-22 -7 -q-50 7 -86 16q-154 40 -278 128q-90 -16 -176 -16q-271 0 -472 132q58 -4 88 -4q161 0 309 45t264 129q125 92 192 212t67 254q0 77 -23 152q129 -71 204 -178t75 -230z" /> - <glyph glyph-name="thumbs_up_alt" unicode="" -d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 768q0 51 -39 89.5t-89 38.5h-352q0 58 48 159.5t48 160.5q0 98 -32 145t-128 47q-26 -26 -38 -85t-30.5 -125.5t-59.5 -109.5q-22 -23 -77 -91q-4 -5 -23 -30t-31.5 -41t-34.5 -42.5 -t-40 -44t-38.5 -35.5t-40 -27t-35.5 -9h-32v-640h32q13 0 31.5 -3t33 -6.5t38 -11t35 -11.5t35.5 -12.5t29 -10.5q211 -73 342 -73h121q192 0 192 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5q32 1 53.5 47t21.5 81zM1536 769 -q0 -89 -49 -163q9 -33 9 -69q0 -77 -38 -144q3 -21 3 -43q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5h-36h-93q-96 0 -189.5 22.5t-216.5 65.5q-116 40 -138 40h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h274q36 24 137 155q58 75 107 128 -q24 25 35.5 85.5t30.5 126.5t62 108q39 37 90 37q84 0 151 -32.5t102 -101.5t35 -186q0 -93 -48 -192h176q104 0 180 -76t76 -179z" /> - <glyph glyph-name="thumbs_down_alt" unicode="" -d="M256 1088q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 512q0 35 -21.5 81t-53.5 47q15 17 25 47.5t10 55.5q0 69 -53 119q18 31 18 69q0 37 -17.5 73.5t-47.5 52.5q5 30 5 56q0 85 -49 126t-136 41h-128q-131 0 -342 -73q-5 -2 -29 -10.5 -t-35.5 -12.5t-35 -11.5t-38 -11t-33 -6.5t-31.5 -3h-32v-640h32q16 0 35.5 -9t40 -27t38.5 -35.5t40 -44t34.5 -42.5t31.5 -41t23 -30q55 -68 77 -91q41 -43 59.5 -109.5t30.5 -125.5t38 -85q96 0 128 47t32 145q0 59 -48 160.5t-48 159.5h352q50 0 89 38.5t39 89.5z -M1536 511q0 -103 -76 -179t-180 -76h-176q48 -99 48 -192q0 -118 -35 -186q-35 -69 -102 -101.5t-151 -32.5q-51 0 -90 37q-34 33 -54 82t-25.5 90.5t-17.5 84.5t-31 64q-48 50 -107 127q-101 131 -137 155h-274q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5 -h288q22 0 138 40q128 44 223 66t200 22h112q140 0 226.5 -79t85.5 -216v-5q60 -77 60 -178q0 -22 -3 -43q38 -67 38 -144q0 -36 -9 -69q49 -73 49 -163z" /> - <glyph glyph-name="star_half" unicode="" horiz-adv-x="896" -d="M832 1504v-1339l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41z" /> - <glyph glyph-name="heart_empty" unicode="" horiz-adv-x="1792" -d="M1664 940q0 81 -21.5 143t-55 98.5t-81.5 59.5t-94 31t-98 8t-112 -25.5t-110.5 -64t-86.5 -72t-60 -61.5q-18 -22 -49 -22t-49 22q-24 28 -60 61.5t-86.5 72t-110.5 64t-112 25.5t-98 -8t-94 -31t-81.5 -59.5t-55 -98.5t-21.5 -143q0 -168 187 -355l581 -560l580 559 -q188 188 188 356zM1792 940q0 -221 -229 -450l-623 -600q-18 -18 -44 -18t-44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5 -q224 0 351 -124t127 -344z" /> - <glyph glyph-name="signout" unicode="" horiz-adv-x="1664" -d="M640 96q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h320q13 0 22.5 -9.5t9.5 -22.5q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-66 0 -113 -47t-47 -113v-704 -q0 -66 47 -113t113 -47h288h11h13t11.5 -1t11.5 -3t8 -5.5t7 -9t2 -13.5zM1568 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45z" /> - <glyph glyph-name="linkedin_sign" unicode="" -d="M237 122h231v694h-231v-694zM483 1030q-1 52 -36 86t-93 34t-94.5 -34t-36.5 -86q0 -51 35.5 -85.5t92.5 -34.5h1q59 0 95 34.5t36 85.5zM1068 122h231v398q0 154 -73 233t-193 79q-136 0 -209 -117h2v101h-231q3 -66 0 -694h231v388q0 38 7 56q15 35 45 59.5t74 24.5 -q116 0 116 -157v-371zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="pushpin" unicode="" horiz-adv-x="1152" -d="M480 672v448q0 14 -9 23t-23 9t-23 -9t-9 -23v-448q0 -14 9 -23t23 -9t23 9t9 23zM1152 320q0 -26 -19 -45t-45 -19h-429l-51 -483q-2 -12 -10.5 -20.5t-20.5 -8.5h-1q-27 0 -32 27l-76 485h-404q-26 0 -45 19t-19 45q0 123 78.5 221.5t177.5 98.5v512q-52 0 -90 38 -t-38 90t38 90t90 38h640q52 0 90 -38t38 -90t-38 -90t-90 -38v-512q99 0 177.5 -98.5t78.5 -221.5z" /> - <glyph glyph-name="external_link" unicode="" horiz-adv-x="1792" -d="M1408 608v-320q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v320 -q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1792 1472v-512q0 -26 -19 -45t-45 -19t-45 19l-176 176l-652 -652q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l652 652l-176 176q-19 19 -19 45t19 45t45 19h512q26 0 45 -19t19 -45z" /> - <glyph glyph-name="signin" unicode="" -d="M1184 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45zM1536 992v-704q0 -119 -84.5 -203.5t-203.5 -84.5h-320q-13 0 -22.5 9.5t-9.5 22.5 -q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q66 0 113 47t47 113v704q0 66 -47 113t-113 47h-288h-11h-13t-11.5 1t-11.5 3t-8 5.5t-7 9t-2 13.5q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="trophy" unicode="" horiz-adv-x="1664" -d="M458 653q-74 162 -74 371h-256v-96q0 -78 94.5 -162t235.5 -113zM1536 928v96h-256q0 -209 -74 -371q141 29 235.5 113t94.5 162zM1664 1056v-128q0 -71 -41.5 -143t-112 -130t-173 -97.5t-215.5 -44.5q-42 -54 -95 -95q-38 -34 -52.5 -72.5t-14.5 -89.5q0 -54 30.5 -91 -t97.5 -37q75 0 133.5 -45.5t58.5 -114.5v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 69 58.5 114.5t133.5 45.5q67 0 97.5 37t30.5 91q0 51 -14.5 89.5t-52.5 72.5q-53 41 -95 95q-113 5 -215.5 44.5t-173 97.5t-112 130t-41.5 143v128q0 40 28 68t68 28h288v96 -q0 66 47 113t113 47h576q66 0 113 -47t47 -113v-96h288q40 0 68 -28t28 -68z" /> - <glyph glyph-name="github_sign" unicode="" -d="M519 336q4 6 -3 13q-9 7 -14 2q-4 -6 3 -13q9 -7 14 -2zM491 377q-5 7 -12 4q-6 -4 0 -12q7 -8 12 -5q6 4 0 13zM450 417q2 4 -5 8q-7 2 -8 -2q-3 -5 4 -8q8 -2 9 2zM471 394q2 1 1.5 4.5t-3.5 5.5q-6 7 -10 3t1 -11q6 -6 11 -2zM557 319q2 7 -9 11q-9 3 -13 -4 -q-2 -7 9 -11q9 -3 13 4zM599 316q0 8 -12 8q-10 0 -10 -8t11 -8t11 8zM638 323q-2 7 -13 5t-9 -9q2 -8 12 -6t10 10zM1280 640q0 212 -150 362t-362 150t-362 -150t-150 -362q0 -167 98 -300.5t252 -185.5q18 -3 26.5 5t8.5 20q0 52 -1 95q-6 -1 -15.5 -2.5t-35.5 -2t-48 4 -t-43.5 20t-29.5 41.5q-23 59 -57 74q-2 1 -4.5 3.5l-8 8t-7 9.5t4 7.5t19.5 3.5q6 0 15 -2t30 -15.5t33 -35.5q16 -28 37.5 -42t43.5 -14t38 3.5t30 9.5q7 47 33 69q-49 6 -86 18.5t-73 39t-55.5 76t-19.5 119.5q0 79 53 137q-24 62 5 136q19 6 54.5 -7.5t60.5 -29.5l26 -16 -q58 17 128 17t128 -17q11 7 28.5 18t55.5 26t57 9q29 -74 5 -136q53 -58 53 -137q0 -57 -14 -100.5t-35.5 -70t-53.5 -44.5t-62.5 -26t-68.5 -12q35 -31 35 -95q0 -40 -0.5 -89t-0.5 -51q0 -12 8.5 -20t26.5 -5q154 52 252 185.5t98 300.5zM1536 1120v-960 -q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="upload_alt" unicode="" horiz-adv-x="1664" -d="M1280 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 288v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h427q21 -56 70.5 -92 -t110.5 -36h256q61 0 110.5 36t70.5 92h427q40 0 68 -28t28 -68zM1339 936q-17 -40 -59 -40h-256v-448q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v448h-256q-42 0 -59 40q-17 39 14 69l448 448q18 19 45 19t45 -19l448 -448q31 -30 14 -69z" /> - <glyph glyph-name="lemon" unicode="" -d="M1407 710q0 44 -7 113.5t-18 96.5q-12 30 -17 44t-9 36.5t-4 48.5q0 23 5 68.5t5 67.5q0 37 -10 55q-4 1 -13 1q-19 0 -58 -4.5t-59 -4.5q-60 0 -176 24t-175 24q-43 0 -94.5 -11.5t-85 -23.5t-89.5 -34q-137 -54 -202 -103q-96 -73 -159.5 -189.5t-88 -236t-24.5 -248.5 -q0 -40 12.5 -120t12.5 -121q0 -23 -11 -66.5t-11 -65.5t12 -36.5t34 -14.5q24 0 72.5 11t73.5 11q57 0 169.5 -15.5t169.5 -15.5q181 0 284 36q129 45 235.5 152.5t166 245.5t59.5 275zM1535 712q0 -165 -70 -327.5t-196 -288t-281 -180.5q-124 -44 -326 -44 -q-57 0 -170 14.5t-169 14.5q-24 0 -72.5 -14.5t-73.5 -14.5q-73 0 -123.5 55.5t-50.5 128.5q0 24 11 68t11 67q0 40 -12.5 120.5t-12.5 121.5q0 111 18 217.5t54.5 209.5t100.5 194t150 156q78 59 232 120q194 78 316 78q60 0 175.5 -24t173.5 -24q19 0 57 5t58 5 -q81 0 118 -50.5t37 -134.5q0 -23 -5 -68t-5 -68q0 -13 2 -25t3.5 -16.5t7.5 -20.5t8 -20q16 -40 25 -118.5t9 -136.5z" /> - <glyph glyph-name="phone" unicode="" horiz-adv-x="1408" -d="M1408 296q0 -27 -10 -70.5t-21 -68.5q-21 -50 -122 -106q-94 -51 -186 -51q-27 0 -53 3.5t-57.5 12.5t-47 14.5t-55.5 20.5t-49 18q-98 35 -175 83q-127 79 -264 216t-216 264q-48 77 -83 175q-3 9 -18 49t-20.5 55.5t-14.5 47t-12.5 57.5t-3.5 53q0 92 51 186 -q56 101 106 122q25 11 68.5 21t70.5 10q14 0 21 -3q18 -6 53 -76q11 -19 30 -54t35 -63.5t31 -53.5q3 -4 17.5 -25t21.5 -35.5t7 -28.5q0 -20 -28.5 -50t-62 -55t-62 -53t-28.5 -46q0 -9 5 -22.5t8.5 -20.5t14 -24t11.5 -19q76 -137 174 -235t235 -174q2 -1 19 -11.5t24 -14 -t20.5 -8.5t22.5 -5q18 0 46 28.5t53 62t55 62t50 28.5q14 0 28.5 -7t35.5 -21.5t25 -17.5q25 -15 53.5 -31t63.5 -35t54 -30q70 -35 76 -53q3 -7 3 -21z" /> - <glyph glyph-name="check_empty" unicode="" horiz-adv-x="1408" -d="M1120 1280h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v832q0 66 -47 113t-113 47zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 -q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="bookmark_empty" unicode="" horiz-adv-x="1280" -d="M1152 1280h-1024v-1242l423 406l89 85l89 -85l423 -406v1242zM1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289 -q0 34 19.5 62t52.5 41q21 9 44 9h1048z" /> - <glyph glyph-name="phone_sign" unicode="" -d="M1280 343q0 11 -2 16t-18 16.5t-40.5 25t-47.5 26.5t-45.5 25t-28.5 15q-5 3 -19 13t-25 15t-21 5q-15 0 -36.5 -20.5t-39.5 -45t-38.5 -45t-33.5 -20.5q-7 0 -16.5 3.5t-15.5 6.5t-17 9.5t-14 8.5q-99 55 -170 126.5t-127 170.5q-2 3 -8.5 14t-9.5 17t-6.5 15.5 -t-3.5 16.5q0 13 20.5 33.5t45 38.5t45 39.5t20.5 36.5q0 10 -5 21t-15 25t-13 19q-3 6 -15 28.5t-25 45.5t-26.5 47.5t-25 40.5t-16.5 18t-16 2q-48 0 -101 -22q-46 -21 -80 -94.5t-34 -130.5q0 -16 2.5 -34t5 -30.5t9 -33t10 -29.5t12.5 -33t11 -30q60 -164 216.5 -320.5 -t320.5 -216.5q6 -2 30 -11t33 -12.5t29.5 -10t33 -9t30.5 -5t34 -2.5q57 0 130.5 34t94.5 80q22 53 22 101zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z -" /> - <glyph glyph-name="twitter" unicode="" horiz-adv-x="1664" -d="M1620 1128q-67 -98 -162 -167q1 -14 1 -42q0 -130 -38 -259.5t-115.5 -248.5t-184.5 -210.5t-258 -146t-323 -54.5q-271 0 -496 145q35 -4 78 -4q225 0 401 138q-105 2 -188 64.5t-114 159.5q33 -5 61 -5q43 0 85 11q-112 23 -185.5 111.5t-73.5 205.5v4q68 -38 146 -41 -q-66 44 -105 115t-39 154q0 88 44 163q121 -149 294.5 -238.5t371.5 -99.5q-8 38 -8 74q0 134 94.5 228.5t228.5 94.5q140 0 236 -102q109 21 205 78q-37 -115 -142 -178q93 10 186 50z" /> - <glyph glyph-name="facebook" unicode="" horiz-adv-x="1024" -d="M959 1524v-264h-157q-86 0 -116 -36t-30 -108v-189h293l-39 -296h-254v-759h-306v759h-255v296h255v218q0 186 104 288.5t277 102.5q147 0 228 -12z" /> - <glyph glyph-name="github" unicode="" -d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -40 7t-13 30q0 3 0.5 76.5t0.5 134.5q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 119 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24 -q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-85 13.5q-45 -113 -8 -204q-79 -87 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-39 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5 -t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -88.5t0.5 -54.5q0 -18 -13 -30t-40 -7q-232 77 -378.5 277.5t-146.5 451.5q0 209 103 385.5t279.5 279.5t385.5 103zM291 305q3 7 -7 12 -q-10 3 -13 -2q-3 -7 7 -12q9 -6 13 2zM322 271q7 5 -2 16q-10 9 -16 3q-7 -5 2 -16q10 -10 16 -3zM352 226q9 7 0 19q-8 13 -17 6q-9 -5 0 -18t17 -7zM394 184q8 8 -4 19q-12 12 -20 3q-9 -8 4 -19q12 -12 20 -3zM451 159q3 11 -13 16q-15 4 -19 -7t13 -15q15 -6 19 6z -M514 154q0 13 -17 11q-16 0 -16 -11q0 -13 17 -11q16 0 16 11zM572 164q-2 11 -18 9q-16 -3 -14 -15t18 -8t14 14z" /> - <glyph glyph-name="unlock" unicode="" horiz-adv-x="1664" -d="M1664 960v-256q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-192h96q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h672v192q0 185 131.5 316.5t316.5 131.5 -t316.5 -131.5t131.5 -316.5z" /> - <glyph glyph-name="credit_card" unicode="" horiz-adv-x="1920" -d="M1760 1408q66 0 113 -47t47 -113v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600zM160 1280q-13 0 -22.5 -9.5t-9.5 -22.5v-224h1664v224q0 13 -9.5 22.5t-22.5 9.5h-1600zM1760 0q13 0 22.5 9.5t9.5 22.5v608h-1664v-608 -q0 -13 9.5 -22.5t22.5 -9.5h1600zM256 128v128h256v-128h-256zM640 128v128h384v-128h-384z" /> - <glyph glyph-name="rss" unicode="" horiz-adv-x="1408" -d="M384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 69q2 -28 -17 -48q-18 -21 -47 -21h-135q-25 0 -43 16.5t-20 41.5q-22 229 -184.5 391.5t-391.5 184.5q-25 2 -41.5 20t-16.5 43v135q0 29 21 47q17 17 43 17h5q160 -13 306 -80.5 -t259 -181.5q114 -113 181.5 -259t80.5 -306zM1408 67q2 -27 -18 -47q-18 -20 -46 -20h-143q-26 0 -44.5 17.5t-19.5 42.5q-12 215 -101 408.5t-231.5 336t-336 231.5t-408.5 102q-25 1 -42.5 19.5t-17.5 43.5v143q0 28 20 46q18 18 44 18h3q262 -13 501.5 -120t425.5 -294 -q187 -186 294 -425.5t120 -501.5z" /> - <glyph glyph-name="hdd" unicode="" -d="M1040 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1296 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1408 160v320q0 13 -9.5 22.5t-22.5 9.5 -h-1216q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5zM178 640h1180l-157 482q-4 13 -16 21.5t-26 8.5h-782q-14 0 -26 -8.5t-16 -21.5zM1536 480v-320q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v320q0 25 16 75 -l197 606q17 53 63 86t101 33h782q55 0 101 -33t63 -86l197 -606q16 -50 16 -75z" /> - <glyph glyph-name="bullhorn" unicode="" horiz-adv-x="1792" -d="M1664 896q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5v-384q0 -52 -38 -90t-90 -38q-417 347 -812 380q-58 -19 -91 -66t-31 -100.5t40 -92.5q-20 -33 -23 -65.5t6 -58t33.5 -55t48 -50t61.5 -50.5q-29 -58 -111.5 -83t-168.5 -11.5t-132 55.5q-7 23 -29.5 87.5 -t-32 94.5t-23 89t-15 101t3.5 98.5t22 110.5h-122q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h480q435 0 896 384q52 0 90 -38t38 -90v-384zM1536 292v954q-394 -302 -768 -343v-270q377 -42 768 -341z" /> - <glyph glyph-name="bell" unicode="" horiz-adv-x="1792" -d="M912 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM246 128h1300q-266 300 -266 832q0 51 -24 105t-69 103t-121.5 80.5t-169.5 31.5t-169.5 -31.5t-121.5 -80.5t-69 -103t-24 -105q0 -532 -266 -832z -M1728 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q50 42 91 88t85 119.5t74.5 158.5t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q190 -28 307 -158.5 -t117 -282.5q0 -139 19.5 -260t50 -206t74.5 -158.5t85 -119.5t91 -88z" /> - <glyph glyph-name="certificate" unicode="" -d="M1376 640l138 -135q30 -28 20 -70q-12 -41 -52 -51l-188 -48l53 -186q12 -41 -19 -70q-29 -31 -70 -19l-186 53l-48 -188q-10 -40 -51 -52q-12 -2 -19 -2q-31 0 -51 22l-135 138l-135 -138q-28 -30 -70 -20q-41 11 -51 52l-48 188l-186 -53q-41 -12 -70 19q-31 29 -19 70 -l53 186l-188 48q-40 10 -52 51q-10 42 20 70l138 135l-138 135q-30 28 -20 70q12 41 52 51l188 48l-53 186q-12 41 19 70q29 31 70 19l186 -53l48 188q10 41 51 51q41 12 70 -19l135 -139l135 139q29 30 70 19q41 -10 51 -51l48 -188l186 53q41 12 70 -19q31 -29 19 -70 -l-53 -186l188 -48q40 -10 52 -51q10 -42 -20 -70z" /> - <glyph glyph-name="hand_right" unicode="" horiz-adv-x="1792" -d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 768q0 51 -39 89.5t-89 38.5h-576q0 20 15 48.5t33 55t33 68t15 84.5q0 67 -44.5 97.5t-115.5 30.5q-24 0 -90 -139q-24 -44 -37 -65q-40 -64 -112 -145q-71 -81 -101 -106 -q-69 -57 -140 -57h-32v-640h32q72 0 167 -32t193.5 -64t179.5 -32q189 0 189 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5h331q52 0 90 38t38 90zM1792 769q0 -105 -75.5 -181t-180.5 -76h-169q-4 -62 -37 -119q3 -21 3 -43 -q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5q-133 0 -322 69q-164 59 -223 59h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h288q10 0 21.5 4.5t23.5 14t22.5 18t24 22.5t20.5 21.5t19 21.5t14 17q65 74 100 129q13 21 33 62t37 72t40.5 63t55 49.5 -t69.5 17.5q125 0 206.5 -67t81.5 -189q0 -68 -22 -128h374q104 0 180 -76t76 -179z" /> - <glyph glyph-name="hand_left" unicode="" horiz-adv-x="1792" -d="M1376 128h32v640h-32q-35 0 -67.5 12t-62.5 37t-50 46t-49 54q-8 9 -12 14q-72 81 -112 145q-14 22 -38 68q-1 3 -10.5 22.5t-18.5 36t-20 35.5t-21.5 30.5t-18.5 11.5q-71 0 -115.5 -30.5t-44.5 -97.5q0 -43 15 -84.5t33 -68t33 -55t15 -48.5h-576q-50 0 -89 -38.5 -t-39 -89.5q0 -52 38 -90t90 -38h331q-15 -17 -25 -47.5t-10 -55.5q0 -69 53 -119q-18 -32 -18 -69t17.5 -73.5t47.5 -52.5q-4 -24 -4 -56q0 -85 48.5 -126t135.5 -41q84 0 183 32t194 64t167 32zM1664 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45z -M1792 768v-640q0 -53 -37.5 -90.5t-90.5 -37.5h-288q-59 0 -223 -59q-190 -69 -317 -69q-142 0 -230 77.5t-87 217.5l1 5q-61 76 -61 178q0 22 3 43q-33 57 -37 119h-169q-105 0 -180.5 76t-75.5 181q0 103 76 179t180 76h374q-22 60 -22 128q0 122 81.5 189t206.5 67 -q38 0 69.5 -17.5t55 -49.5t40.5 -63t37 -72t33 -62q35 -55 100 -129q2 -3 14 -17t19 -21.5t20.5 -21.5t24 -22.5t22.5 -18t23.5 -14t21.5 -4.5h288q53 0 90.5 -37.5t37.5 -90.5z" /> - <glyph glyph-name="hand_up" unicode="" -d="M1280 -64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 700q0 189 -167 189q-26 0 -56 -5q-16 30 -52.5 47.5t-73.5 17.5t-69 -18q-50 53 -119 53q-25 0 -55.5 -10t-47.5 -25v331q0 52 -38 90t-90 38q-51 0 -89.5 -39t-38.5 -89v-576 -q-20 0 -48.5 15t-55 33t-68 33t-84.5 15q-67 0 -97.5 -44.5t-30.5 -115.5q0 -24 139 -90q44 -24 65 -37q64 -40 145 -112q81 -71 106 -101q57 -69 57 -140v-32h640v32q0 72 32 167t64 193.5t32 179.5zM1536 705q0 -133 -69 -322q-59 -164 -59 -223v-288q0 -53 -37.5 -90.5 -t-90.5 -37.5h-640q-53 0 -90.5 37.5t-37.5 90.5v288q0 10 -4.5 21.5t-14 23.5t-18 22.5t-22.5 24t-21.5 20.5t-21.5 19t-17 14q-74 65 -129 100q-21 13 -62 33t-72 37t-63 40.5t-49.5 55t-17.5 69.5q0 125 67 206.5t189 81.5q68 0 128 -22v374q0 104 76 180t179 76 -q105 0 181 -75.5t76 -180.5v-169q62 -4 119 -37q21 3 43 3q101 0 178 -60q139 1 219.5 -85t80.5 -227z" /> - <glyph glyph-name="hand_down" unicode="" -d="M1408 576q0 84 -32 183t-64 194t-32 167v32h-640v-32q0 -35 -12 -67.5t-37 -62.5t-46 -50t-54 -49q-9 -8 -14 -12q-81 -72 -145 -112q-22 -14 -68 -38q-3 -1 -22.5 -10.5t-36 -18.5t-35.5 -20t-30.5 -21.5t-11.5 -18.5q0 -71 30.5 -115.5t97.5 -44.5q43 0 84.5 15t68 33 -t55 33t48.5 15v-576q0 -50 38.5 -89t89.5 -39q52 0 90 38t38 90v331q46 -35 103 -35q69 0 119 53q32 -18 69 -18t73.5 17.5t52.5 47.5q24 -4 56 -4q85 0 126 48.5t41 135.5zM1280 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 580 -q0 -142 -77.5 -230t-217.5 -87l-5 1q-76 -61 -178 -61q-22 0 -43 3q-54 -30 -119 -37v-169q0 -105 -76 -180.5t-181 -75.5q-103 0 -179 76t-76 180v374q-54 -22 -128 -22q-121 0 -188.5 81.5t-67.5 206.5q0 38 17.5 69.5t49.5 55t63 40.5t72 37t62 33q55 35 129 100 -q3 2 17 14t21.5 19t21.5 20.5t22.5 24t18 22.5t14 23.5t4.5 21.5v288q0 53 37.5 90.5t90.5 37.5h640q53 0 90.5 -37.5t37.5 -90.5v-288q0 -59 59 -223q69 -190 69 -317z" /> - <glyph glyph-name="circle_arrow_left" unicode="" -d="M1280 576v128q0 26 -19 45t-45 19h-502l189 189q19 19 19 45t-19 45l-91 91q-18 18 -45 18t-45 -18l-362 -362l-91 -91q-18 -18 -18 -45t18 -45l91 -91l362 -362q18 -18 45 -18t45 18l91 91q18 18 18 45t-18 45l-189 189h502q26 0 45 19t19 45zM1536 640 -q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="circle_arrow_right" unicode="" -d="M1285 640q0 27 -18 45l-91 91l-362 362q-18 18 -45 18t-45 -18l-91 -91q-18 -18 -18 -45t18 -45l189 -189h-502q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h502l-189 -189q-19 -19 -19 -45t19 -45l91 -91q18 -18 45 -18t45 18l362 362l91 91q18 18 18 45zM1536 640 -q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="circle_arrow_up" unicode="" -d="M1284 641q0 27 -18 45l-362 362l-91 91q-18 18 -45 18t-45 -18l-91 -91l-362 -362q-18 -18 -18 -45t18 -45l91 -91q18 -18 45 -18t45 18l189 189v-502q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v502l189 -189q19 -19 45 -19t45 19l91 91q18 18 18 45zM1536 640 -q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="circle_arrow_down" unicode="" -d="M1284 639q0 27 -18 45l-91 91q-18 18 -45 18t-45 -18l-189 -189v502q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-502l-189 189q-19 19 -45 19t-45 -19l-91 -91q-18 -18 -18 -45t18 -45l362 -362l91 -91q18 -18 45 -18t45 18l91 91l362 362q18 18 18 45zM1536 640 -q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="globe" unicode="" -d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1042 887q-2 -1 -9.5 -9.5t-13.5 -9.5q2 0 4.5 5t5 11t3.5 7q6 7 22 15q14 6 52 12q34 8 51 -11 -q-2 2 9.5 13t14.5 12q3 2 15 4.5t15 7.5l2 22q-12 -1 -17.5 7t-6.5 21q0 -2 -6 -8q0 7 -4.5 8t-11.5 -1t-9 -1q-10 3 -15 7.5t-8 16.5t-4 15q-2 5 -9.5 11t-9.5 10q-1 2 -2.5 5.5t-3 6.5t-4 5.5t-5.5 2.5t-7 -5t-7.5 -10t-4.5 -5q-3 2 -6 1.5t-4.5 -1t-4.5 -3t-5 -3.5 -q-3 -2 -8.5 -3t-8.5 -2q15 5 -1 11q-10 4 -16 3q9 4 7.5 12t-8.5 14h5q-1 4 -8.5 8.5t-17.5 8.5t-13 6q-8 5 -34 9.5t-33 0.5q-5 -6 -4.5 -10.5t4 -14t3.5 -12.5q1 -6 -5.5 -13t-6.5 -12q0 -7 14 -15.5t10 -21.5q-3 -8 -16 -16t-16 -12q-5 -8 -1.5 -18.5t10.5 -16.5 -q2 -2 1.5 -4t-3.5 -4.5t-5.5 -4t-6.5 -3.5l-3 -2q-11 -5 -20.5 6t-13.5 26q-7 25 -16 30q-23 8 -29 -1q-5 13 -41 26q-25 9 -58 4q6 1 0 15q-7 15 -19 12q3 6 4 17.5t1 13.5q3 13 12 23q1 1 7 8.5t9.5 13.5t0.5 6q35 -4 50 11q5 5 11.5 17t10.5 17q9 6 14 5.5t14.5 -5.5 -t14.5 -5q14 -1 15.5 11t-7.5 20q12 -1 3 17q-4 7 -8 9q-12 4 -27 -5q-8 -4 2 -8q-1 1 -9.5 -10.5t-16.5 -17.5t-16 5q-1 1 -5.5 13.5t-9.5 13.5q-8 0 -16 -15q3 8 -11 15t-24 8q19 12 -8 27q-7 4 -20.5 5t-19.5 -4q-5 -7 -5.5 -11.5t5 -8t10.5 -5.5t11.5 -4t8.5 -3 -q14 -10 8 -14q-2 -1 -8.5 -3.5t-11.5 -4.5t-6 -4q-3 -4 0 -14t-2 -14q-5 5 -9 17.5t-7 16.5q7 -9 -25 -6l-10 1q-4 0 -16 -2t-20.5 -1t-13.5 8q-4 8 0 20q1 4 4 2q-4 3 -11 9.5t-10 8.5q-46 -15 -94 -41q6 -1 12 1q5 2 13 6.5t10 5.5q34 14 42 7l5 5q14 -16 20 -25 -q-7 4 -30 1q-20 -6 -22 -12q7 -12 5 -18q-4 3 -11.5 10t-14.5 11t-15 5q-16 0 -22 -1q-146 -80 -235 -222q7 -7 12 -8q4 -1 5 -9t2.5 -11t11.5 3q9 -8 3 -19q1 1 44 -27q19 -17 21 -21q3 -11 -10 -18q-1 2 -9 9t-9 4q-3 -5 0.5 -18.5t10.5 -12.5q-7 0 -9.5 -16t-2.5 -35.5 -t-1 -23.5l2 -1q-3 -12 5.5 -34.5t21.5 -19.5q-13 -3 20 -43q6 -8 8 -9q3 -2 12 -7.5t15 -10t10 -10.5q4 -5 10 -22.5t14 -23.5q-2 -6 9.5 -20t10.5 -23q-1 0 -2.5 -1t-2.5 -1q3 -7 15.5 -14t15.5 -13q1 -3 2 -10t3 -11t8 -2q2 20 -24 62q-15 25 -17 29q-3 5 -5.5 15.5 -t-4.5 14.5q2 0 6 -1.5t8.5 -3.5t7.5 -4t2 -3q-3 -7 2 -17.5t12 -18.5t17 -19t12 -13q6 -6 14 -19.5t0 -13.5q9 0 20 -10.5t17 -19.5q5 -8 8 -26t5 -24q2 -7 8.5 -13.5t12.5 -9.5l16 -8t13 -7q5 -2 18.5 -10.5t21.5 -11.5q10 -4 16 -4t14.5 2.5t13.5 3.5q15 2 29 -15t21 -21 -q36 -19 55 -11q-2 -1 0.5 -7.5t8 -15.5t9 -14.5t5.5 -8.5q5 -6 18 -15t18 -15q6 4 7 9q-3 -8 7 -20t18 -10q14 3 14 32q-31 -15 -49 18q0 1 -2.5 5.5t-4 8.5t-2.5 8.5t0 7.5t5 3q9 0 10 3.5t-2 12.5t-4 13q-1 8 -11 20t-12 15q-5 -9 -16 -8t-16 9q0 -1 -1.5 -5.5t-1.5 -6.5 -q-13 0 -15 1q1 3 2.5 17.5t3.5 22.5q1 4 5.5 12t7.5 14.5t4 12.5t-4.5 9.5t-17.5 2.5q-19 -1 -26 -20q-1 -3 -3 -10.5t-5 -11.5t-9 -7q-7 -3 -24 -2t-24 5q-13 8 -22.5 29t-9.5 37q0 10 2.5 26.5t3 25t-5.5 24.5q3 2 9 9.5t10 10.5q2 1 4.5 1.5t4.5 0t4 1.5t3 6q-1 1 -4 3 -q-3 3 -4 3q7 -3 28.5 1.5t27.5 -1.5q15 -11 22 2q0 1 -2.5 9.5t-0.5 13.5q5 -27 29 -9q3 -3 15.5 -5t17.5 -5q3 -2 7 -5.5t5.5 -4.5t5 0.5t8.5 6.5q10 -14 12 -24q11 -40 19 -44q7 -3 11 -2t4.5 9.5t0 14t-1.5 12.5l-1 8v18l-1 8q-15 3 -18.5 12t1.5 18.5t15 18.5q1 1 8 3.5 -t15.5 6.5t12.5 8q21 19 15 35q7 0 11 9q-1 0 -5 3t-7.5 5t-4.5 2q9 5 2 16q5 3 7.5 11t7.5 10q9 -12 21 -2q8 8 1 16q5 7 20.5 10.5t18.5 9.5q7 -2 8 2t1 12t3 12q4 5 15 9t13 5l17 11q3 4 0 4q18 -2 31 11q10 11 -6 20q3 6 -3 9.5t-15 5.5q3 1 11.5 0.5t10.5 1.5 -q15 10 -7 16q-17 5 -43 -12zM879 10q206 36 351 189q-3 3 -12.5 4.5t-12.5 3.5q-18 7 -24 8q1 7 -2.5 13t-8 9t-12.5 8t-11 7q-2 2 -7 6t-7 5.5t-7.5 4.5t-8.5 2t-10 -1l-3 -1q-3 -1 -5.5 -2.5t-5.5 -3t-4 -3t0 -2.5q-21 17 -36 22q-5 1 -11 5.5t-10.5 7t-10 1.5t-11.5 -7 -q-5 -5 -6 -15t-2 -13q-7 5 0 17.5t2 18.5q-3 6 -10.5 4.5t-12 -4.5t-11.5 -8.5t-9 -6.5t-8.5 -5.5t-8.5 -7.5q-3 -4 -6 -12t-5 -11q-2 4 -11.5 6.5t-9.5 5.5q2 -10 4 -35t5 -38q7 -31 -12 -48q-27 -25 -29 -40q-4 -22 12 -26q0 -7 -8 -20.5t-7 -21.5q0 -6 2 -16z" /> - <glyph glyph-name="wrench" unicode="" horiz-adv-x="1664" -d="M384 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1028 484l-682 -682q-37 -37 -90 -37q-52 0 -91 37l-106 108q-38 36 -38 90q0 53 38 91l681 681q39 -98 114.5 -173.5t173.5 -114.5zM1662 919q0 -39 -23 -106q-47 -134 -164.5 -217.5 -t-258.5 -83.5q-185 0 -316.5 131.5t-131.5 316.5t131.5 316.5t316.5 131.5q58 0 121.5 -16.5t107.5 -46.5q16 -11 16 -28t-16 -28l-293 -169v-224l193 -107q5 3 79 48.5t135.5 81t70.5 35.5q15 0 23.5 -10t8.5 -25z" /> - <glyph glyph-name="tasks" unicode="" horiz-adv-x="1792" -d="M1024 128h640v128h-640v-128zM640 640h1024v128h-1024v-128zM1280 1152h384v128h-384v-128zM1792 320v-256q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 832v-256q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19 -t-19 45v256q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-256q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" /> - <glyph glyph-name="filter" unicode="" horiz-adv-x="1408" -d="M1403 1241q17 -41 -14 -70l-493 -493v-742q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-256 256q-19 19 -19 45v486l-493 493q-31 29 -14 70q17 39 59 39h1280q42 0 59 -39z" /> - <glyph glyph-name="briefcase" unicode="" horiz-adv-x="1792" -d="M640 1280h512v128h-512v-128zM1792 640v-480q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v480h672v-160q0 -26 19 -45t45 -19h320q26 0 45 19t19 45v160h672zM1024 640v-128h-256v128h256zM1792 1120v-384h-1792v384q0 66 47 113t113 47h352v160q0 40 28 68 -t68 28h576q40 0 68 -28t28 -68v-160h352q66 0 113 -47t47 -113z" /> - <glyph glyph-name="fullscreen" unicode="" -d="M1283 995l-355 -355l355 -355l144 144q29 31 70 14q39 -17 39 -59v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l144 144l-355 355l-355 -355l144 -144q31 -30 14 -69q-17 -40 -59 -40h-448q-26 0 -45 19t-19 45v448q0 42 40 59q39 17 69 -14l144 -144 -l355 355l-355 355l-144 -144q-19 -19 -45 -19q-12 0 -24 5q-40 17 -40 59v448q0 26 19 45t45 19h448q42 0 59 -40q17 -39 -14 -69l-144 -144l355 -355l355 355l-144 144q-31 30 -14 69q17 40 59 40h448q26 0 45 -19t19 -45v-448q0 -42 -39 -59q-13 -5 -25 -5q-26 0 -45 19z -" /> - <glyph glyph-name="group" unicode="" horiz-adv-x="1920" -d="M593 640q-162 -5 -265 -128h-134q-82 0 -138 40.5t-56 118.5q0 353 124 353q6 0 43.5 -21t97.5 -42.5t119 -21.5q67 0 133 23q-5 -37 -5 -66q0 -139 81 -256zM1664 3q0 -120 -73 -189.5t-194 -69.5h-874q-121 0 -194 69.5t-73 189.5q0 53 3.5 103.5t14 109t26.5 108.5 -t43 97.5t62 81t85.5 53.5t111.5 20q10 0 43 -21.5t73 -48t107 -48t135 -21.5t135 21.5t107 48t73 48t43 21.5q61 0 111.5 -20t85.5 -53.5t62 -81t43 -97.5t26.5 -108.5t14 -109t3.5 -103.5zM640 1280q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75 -t75 -181zM1344 896q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5zM1920 671q0 -78 -56 -118.5t-138 -40.5h-134q-103 123 -265 128q81 117 81 256q0 29 -5 66q66 -23 133 -23q59 0 119 21.5t97.5 42.5 -t43.5 21q124 0 124 -353zM1792 1280q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181z" /> - <glyph glyph-name="link" unicode="" horiz-adv-x="1664" -d="M1456 320q0 40 -28 68l-208 208q-28 28 -68 28q-42 0 -72 -32q3 -3 19 -18.5t21.5 -21.5t15 -19t13 -25.5t3.5 -27.5q0 -40 -28 -68t-68 -28q-15 0 -27.5 3.5t-25.5 13t-19 15t-21.5 21.5t-18.5 19q-33 -31 -33 -73q0 -40 28 -68l206 -207q27 -27 68 -27q40 0 68 26 -l147 146q28 28 28 67zM753 1025q0 40 -28 68l-206 207q-28 28 -68 28q-39 0 -68 -27l-147 -146q-28 -28 -28 -67q0 -40 28 -68l208 -208q27 -27 68 -27q42 0 72 31q-3 3 -19 18.5t-21.5 21.5t-15 19t-13 25.5t-3.5 27.5q0 40 28 68t68 28q15 0 27.5 -3.5t25.5 -13t19 -15 -t21.5 -21.5t18.5 -19q33 31 33 73zM1648 320q0 -120 -85 -203l-147 -146q-83 -83 -203 -83q-121 0 -204 85l-206 207q-83 83 -83 203q0 123 88 209l-88 88q-86 -88 -208 -88q-120 0 -204 84l-208 208q-84 84 -84 204t85 203l147 146q83 83 203 83q121 0 204 -85l206 -207 -q83 -83 83 -203q0 -123 -88 -209l88 -88q86 88 208 88q120 0 204 -84l208 -208q84 -84 84 -204z" /> - <glyph glyph-name="cloud" unicode="" horiz-adv-x="1920" -d="M1920 384q0 -159 -112.5 -271.5t-271.5 -112.5h-1088q-185 0 -316.5 131.5t-131.5 316.5q0 132 71 241.5t187 163.5q-2 28 -2 43q0 212 150 362t362 150q158 0 286.5 -88t187.5 -230q70 62 166 62q106 0 181 -75t75 -181q0 -75 -41 -138q129 -30 213 -134.5t84 -239.5z -" /> - <glyph glyph-name="beaker" unicode="" horiz-adv-x="1664" -d="M1527 88q56 -89 21.5 -152.5t-140.5 -63.5h-1152q-106 0 -140.5 63.5t21.5 152.5l503 793v399h-64q-26 0 -45 19t-19 45t19 45t45 19h512q26 0 45 -19t19 -45t-19 -45t-45 -19h-64v-399zM748 813l-272 -429h712l-272 429l-20 31v37v399h-128v-399v-37z" /> - <glyph glyph-name="cut" unicode="" horiz-adv-x="1792" -d="M960 640q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1260 576l507 -398q28 -20 25 -56q-5 -35 -35 -51l-128 -64q-13 -7 -29 -7q-17 0 -31 8l-690 387l-110 -66q-8 -4 -12 -5q14 -49 10 -97q-7 -77 -56 -147.5t-132 -123.5q-132 -84 -277 -84 -q-136 0 -222 78q-90 84 -79 207q7 76 56 147t131 124q132 84 278 84q83 0 151 -31q9 13 22 22l122 73l-122 73q-13 9 -22 22q-68 -31 -151 -31q-146 0 -278 84q-82 53 -131 124t-56 147q-5 59 15.5 113t63.5 93q85 79 222 79q145 0 277 -84q83 -52 132 -123t56 -148 -q4 -48 -10 -97q4 -1 12 -5l110 -66l690 387q14 8 31 8q16 0 29 -7l128 -64q30 -16 35 -51q3 -36 -25 -56zM579 836q46 42 21 108t-106 117q-92 59 -192 59q-74 0 -113 -36q-46 -42 -21 -108t106 -117q92 -59 192 -59q74 0 113 36zM494 91q81 51 106 117t-21 108 -q-39 36 -113 36q-100 0 -192 -59q-81 -51 -106 -117t21 -108q39 -36 113 -36q100 0 192 59zM672 704l96 -58v11q0 36 33 56l14 8l-79 47l-26 -26q-3 -3 -10 -11t-12 -12q-2 -2 -4 -3.5t-3 -2.5zM896 480l96 -32l736 576l-128 64l-768 -431v-113l-160 -96l9 -8q2 -2 7 -6 -q4 -4 11 -12t11 -12l26 -26zM1600 64l128 64l-520 408l-177 -138q-2 -3 -13 -7z" /> - <glyph glyph-name="copy" unicode="" horiz-adv-x="1792" -d="M1696 1152q40 0 68 -28t28 -68v-1216q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v288h-544q-40 0 -68 28t-28 68v672q0 40 20 88t48 76l408 408q28 28 76 48t88 20h416q40 0 68 -28t28 -68v-328q68 40 128 40h416zM1152 939l-299 -299h299v299zM512 1323l-299 -299 -h299v299zM708 676l316 316v416h-384v-416q0 -40 -28 -68t-68 -28h-416v-640h512v256q0 40 20 88t48 76zM1664 -128v1152h-384v-416q0 -40 -28 -68t-68 -28h-416v-640h896z" /> - <glyph glyph-name="paper_clip" unicode="" horiz-adv-x="1408" -d="M1404 151q0 -117 -79 -196t-196 -79q-135 0 -235 100l-777 776q-113 115 -113 271q0 159 110 270t269 111q158 0 273 -113l605 -606q10 -10 10 -22q0 -16 -30.5 -46.5t-46.5 -30.5q-13 0 -23 10l-606 607q-79 77 -181 77q-106 0 -179 -75t-73 -181q0 -105 76 -181 -l776 -777q63 -63 145 -63q64 0 106 42t42 106q0 82 -63 145l-581 581q-26 24 -60 24q-29 0 -48 -19t-19 -48q0 -32 25 -59l410 -410q10 -10 10 -22q0 -16 -31 -47t-47 -31q-12 0 -22 10l-410 410q-63 61 -63 149q0 82 57 139t139 57q88 0 149 -63l581 -581q100 -98 100 -235 -z" /> - <glyph glyph-name="save" unicode="" -d="M384 0h768v384h-768v-384zM1280 0h128v896q0 14 -10 38.5t-20 34.5l-281 281q-10 10 -34 20t-39 10v-416q0 -40 -28 -68t-68 -28h-576q-40 0 -68 28t-28 68v416h-128v-1280h128v416q0 40 28 68t68 28h832q40 0 68 -28t28 -68v-416zM896 928v320q0 13 -9.5 22.5t-22.5 9.5 -h-192q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 22.5zM1536 896v-928q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h928q40 0 88 -20t76 -48l280 -280q28 -28 48 -76t20 -88z" /> - <glyph glyph-name="sign_blank" unicode="" -d="M1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="reorder" unicode="" -d="M1536 192v-128q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1536 704v-128q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1536 1216v-128q0 -26 -19 -45 -t-45 -19h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" /> - <glyph glyph-name="ul" unicode="" horiz-adv-x="1792" -d="M384 128q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM384 640q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1216q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5 -t22.5 9.5h1216q13 0 22.5 -9.5t9.5 -22.5zM384 1152q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1792 736v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1216q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1216q13 0 22.5 -9.5t9.5 -22.5z -M1792 1248v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1216q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1216q13 0 22.5 -9.5t9.5 -22.5z" /> - <glyph glyph-name="ol" unicode="" horiz-adv-x="1792" -d="M381 -84q0 -80 -54.5 -126t-135.5 -46q-106 0 -172 66l57 88q49 -45 106 -45q29 0 50.5 14.5t21.5 42.5q0 64 -105 56l-26 56q8 10 32.5 43.5t42.5 54t37 38.5v1q-16 0 -48.5 -1t-48.5 -1v-53h-106v152h333v-88l-95 -115q51 -12 81 -49t30 -88zM383 543v-159h-362 -q-6 36 -6 54q0 51 23.5 93t56.5 68t66 47.5t56.5 43.5t23.5 45q0 25 -14.5 38.5t-39.5 13.5q-46 0 -81 -58l-85 59q24 51 71.5 79.5t105.5 28.5q73 0 123 -41.5t50 -112.5q0 -50 -34 -91.5t-75 -64.5t-75.5 -50.5t-35.5 -52.5h127v60h105zM1792 224v-192q0 -13 -9.5 -22.5 -t-22.5 -9.5h-1216q-13 0 -22.5 9.5t-9.5 22.5v192q0 14 9 23t23 9h1216q13 0 22.5 -9.5t9.5 -22.5zM384 1123v-99h-335v99h107q0 41 0.5 121.5t0.5 121.5v12h-2q-8 -17 -50 -54l-71 76l136 127h106v-404h108zM1792 736v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1216 -q-13 0 -22.5 9.5t-9.5 22.5v192q0 14 9 23t23 9h1216q13 0 22.5 -9.5t9.5 -22.5zM1792 1248v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1216q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1216q13 0 22.5 -9.5t9.5 -22.5z" /> - <glyph glyph-name="strikethrough" unicode="" horiz-adv-x="1792" -d="M1760 640q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-1728q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h1728zM483 704q-28 35 -51 80q-48 98 -48 188q0 181 134 309q133 127 393 127q50 0 167 -19q66 -12 177 -48q10 -38 21 -118q14 -123 14 -183q0 -18 -5 -45l-12 -3l-84 6 -l-14 2q-50 149 -103 205q-88 91 -210 91q-114 0 -182 -59q-67 -58 -67 -146q0 -73 66 -140t279 -129q69 -20 173 -66q58 -28 95 -52h-743zM990 448h411q7 -39 7 -92q0 -111 -41 -212q-23 -56 -71 -104q-37 -35 -109 -81q-80 -48 -153 -66q-80 -21 -203 -21q-114 0 -195 23 -l-140 40q-57 16 -72 28q-8 8 -8 22v13q0 108 -2 156q-1 30 0 68l2 37v44l102 2q15 -34 30 -71t22.5 -56t12.5 -27q35 -57 80 -94q43 -36 105 -57q59 -22 132 -22q64 0 139 27q77 26 122 86q47 61 47 129q0 84 -81 157q-34 29 -137 71z" /> - <glyph glyph-name="underline" unicode="" -d="M48 1313q-37 2 -45 4l-3 88q13 1 40 1q60 0 112 -4q132 -7 166 -7q86 0 168 3q116 4 146 5q56 0 86 2l-1 -14l2 -64v-9q-60 -9 -124 -9q-60 0 -79 -25q-13 -14 -13 -132q0 -13 0.5 -32.5t0.5 -25.5l1 -229l14 -280q6 -124 51 -202q35 -59 96 -92q88 -47 177 -47 -q104 0 191 28q56 18 99 51q48 36 65 64q36 56 53 114q21 73 21 229q0 79 -3.5 128t-11 122.5t-13.5 159.5l-4 59q-5 67 -24 88q-34 35 -77 34l-100 -2l-14 3l2 86h84l205 -10q76 -3 196 10l18 -2q6 -38 6 -51q0 -7 -4 -31q-45 -12 -84 -13q-73 -11 -79 -17q-15 -15 -15 -41 -q0 -7 1.5 -27t1.5 -31q8 -19 22 -396q6 -195 -15 -304q-15 -76 -41 -122q-38 -65 -112 -123q-75 -57 -182 -89q-109 -33 -255 -33q-167 0 -284 46q-119 47 -179 122q-61 76 -83 195q-16 80 -16 237v333q0 188 -17 213q-25 36 -147 39zM1536 -96v64q0 14 -9 23t-23 9h-1472 -q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h1472q14 0 23 9t9 23z" /> - <glyph glyph-name="table" unicode="" horiz-adv-x="1664" -d="M512 160v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM512 544v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1024 160v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23 -v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM512 928v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1024 544v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1536 160v192 -q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1024 928v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1536 544v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192 -q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1536 928v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1664 1248v-1088q0 -66 -47 -113t-113 -47h-1344q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1344q66 0 113 -47t47 -113 -z" /> - <glyph glyph-name="magic" unicode="" horiz-adv-x="1664" -d="M1190 955l293 293l-107 107l-293 -293zM1637 1248q0 -27 -18 -45l-1286 -1286q-18 -18 -45 -18t-45 18l-198 198q-18 18 -18 45t18 45l1286 1286q18 18 45 18t45 -18l198 -198q18 -18 18 -45zM286 1438l98 -30l-98 -30l-30 -98l-30 98l-98 30l98 30l30 98zM636 1276 -l196 -60l-196 -60l-60 -196l-60 196l-196 60l196 60l60 196zM1566 798l98 -30l-98 -30l-30 -98l-30 98l-98 30l98 30l30 98zM926 1438l98 -30l-98 -30l-30 -98l-30 98l-98 30l98 30l30 98z" /> - <glyph glyph-name="truck" unicode="" horiz-adv-x="1792" -d="M640 128q0 52 -38 90t-90 38t-90 -38t-38 -90t38 -90t90 -38t90 38t38 90zM256 640h384v256h-158q-13 0 -22 -9l-195 -195q-9 -9 -9 -22v-30zM1536 128q0 52 -38 90t-90 38t-90 -38t-38 -90t38 -90t90 -38t90 38t38 90zM1792 1216v-1024q0 -15 -4 -26.5t-13.5 -18.5 -t-16.5 -11.5t-23.5 -6t-22.5 -2t-25.5 0t-22.5 0.5q0 -106 -75 -181t-181 -75t-181 75t-75 181h-384q0 -106 -75 -181t-181 -75t-181 75t-75 181h-64q-3 0 -22.5 -0.5t-25.5 0t-22.5 2t-23.5 6t-16.5 11.5t-13.5 18.5t-4 26.5q0 26 19 45t45 19v320q0 8 -0.5 35t0 38 -t2.5 34.5t6.5 37t14 30.5t22.5 30l198 198q19 19 50.5 32t58.5 13h160v192q0 26 19 45t45 19h1024q26 0 45 -19t19 -45z" /> - <glyph glyph-name="pinterest" unicode="" -d="M1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103q-111 0 -218 32q59 93 78 164q9 34 54 211q20 -39 73 -67.5t114 -28.5q121 0 216 68.5t147 188.5t52 270q0 114 -59.5 214t-172.5 163t-255 63q-105 0 -196 -29t-154.5 -77t-109 -110.5t-67 -129.5t-21.5 -134 -q0 -104 40 -183t117 -111q30 -12 38 20q2 7 8 31t8 30q6 23 -11 43q-51 61 -51 151q0 151 104.5 259.5t273.5 108.5q151 0 235.5 -82t84.5 -213q0 -170 -68.5 -289t-175.5 -119q-61 0 -98 43.5t-23 104.5q8 35 26.5 93.5t30 103t11.5 75.5q0 50 -27 83t-77 33 -q-62 0 -105 -57t-43 -142q0 -73 25 -122l-99 -418q-17 -70 -13 -177q-206 91 -333 281t-127 423q0 209 103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="pinterest_sign" unicode="" -d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-725q85 122 108 210q9 34 53 209q21 -39 73.5 -67t112.5 -28q181 0 295.5 147.5t114.5 373.5q0 84 -35 162.5t-96.5 139t-152.5 97t-197 36.5q-104 0 -194.5 -28.5t-153 -76.5 -t-107.5 -109.5t-66.5 -128t-21.5 -132.5q0 -102 39.5 -180t116.5 -110q13 -5 23.5 0t14.5 19q10 44 15 61q6 23 -11 42q-50 62 -50 150q0 150 103.5 256.5t270.5 106.5q149 0 232.5 -81t83.5 -210q0 -168 -67.5 -286t-173.5 -118q-60 0 -97 43.5t-23 103.5q8 34 26.5 92.5 -t29.5 102t11 74.5q0 49 -26.5 81.5t-75.5 32.5q-61 0 -103.5 -56.5t-42.5 -139.5q0 -72 24 -121l-98 -414q-24 -100 -7 -254h-183q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960z" /> - <glyph glyph-name="google_plus_sign" unicode="" -d="M917 631q0 26 -6 64h-362v-132h217q-3 -24 -16.5 -50t-37.5 -53t-66.5 -44.5t-96.5 -17.5q-99 0 -169 71t-70 171t70 171t169 71q92 0 153 -59l104 101q-108 100 -257 100q-160 0 -272 -112.5t-112 -271.5t112 -271.5t272 -112.5q165 0 266.5 105t101.5 270zM1262 585 -h109v110h-109v110h-110v-110h-110v-110h110v-110h110v110zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="google_plus" unicode="" horiz-adv-x="2304" -d="M1437 623q0 -208 -87 -370.5t-248 -254t-369 -91.5q-149 0 -285 58t-234 156t-156 234t-58 285t58 285t156 234t234 156t285 58q286 0 491 -192l-199 -191q-117 113 -292 113q-123 0 -227.5 -62t-165.5 -168.5t-61 -232.5t61 -232.5t165.5 -168.5t227.5 -62 -q83 0 152.5 23t114.5 57.5t78.5 78.5t49 83t21.5 74h-416v252h692q12 -63 12 -122zM2304 745v-210h-209v-209h-210v209h-209v210h209v209h210v-209h209z" /> - <glyph glyph-name="money" unicode="" horiz-adv-x="1920" -d="M768 384h384v96h-128v448h-114l-148 -137l77 -80q42 37 55 57h2v-288h-128v-96zM1280 640q0 -70 -21 -142t-59.5 -134t-101.5 -101t-138 -39t-138 39t-101.5 101t-59.5 134t-21 142t21 142t59.5 134t101.5 101t138 39t138 -39t101.5 -101t59.5 -134t21 -142zM1792 384 -v512q-106 0 -181 75t-75 181h-1152q0 -106 -75 -181t-181 -75v-512q106 0 181 -75t75 -181h1152q0 106 75 181t181 75zM1920 1216v-1152q0 -26 -19 -45t-45 -19h-1792q-26 0 -45 19t-19 45v1152q0 26 19 45t45 19h1792q26 0 45 -19t19 -45z" /> - <glyph glyph-name="caret_down" unicode="" horiz-adv-x="1024" -d="M1024 832q0 -26 -19 -45l-448 -448q-19 -19 -45 -19t-45 19l-448 448q-19 19 -19 45t19 45t45 19h896q26 0 45 -19t19 -45z" /> - <glyph glyph-name="caret_up" unicode="" horiz-adv-x="1024" -d="M1024 320q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45l448 448q19 19 45 19t45 -19l448 -448q19 -19 19 -45z" /> - <glyph glyph-name="caret_left" unicode="" horiz-adv-x="640" -d="M640 1088v-896q0 -26 -19 -45t-45 -19t-45 19l-448 448q-19 19 -19 45t19 45l448 448q19 19 45 19t45 -19t19 -45z" /> - <glyph glyph-name="caret_right" unicode="" horiz-adv-x="640" -d="M576 640q0 -26 -19 -45l-448 -448q-19 -19 -45 -19t-45 19t-19 45v896q0 26 19 45t45 19t45 -19l448 -448q19 -19 19 -45z" /> - <glyph glyph-name="columns" unicode="" horiz-adv-x="1664" -d="M160 0h608v1152h-640v-1120q0 -13 9.5 -22.5t22.5 -9.5zM1536 32v1120h-640v-1152h608q13 0 22.5 9.5t9.5 22.5zM1664 1248v-1216q0 -66 -47 -113t-113 -47h-1344q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1344q66 0 113 -47t47 -113z" /> - <glyph glyph-name="sort" unicode="" horiz-adv-x="1024" -d="M1024 448q0 -26 -19 -45l-448 -448q-19 -19 -45 -19t-45 19l-448 448q-19 19 -19 45t19 45t45 19h896q26 0 45 -19t19 -45zM1024 832q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45l448 448q19 19 45 19t45 -19l448 -448q19 -19 19 -45z" /> - <glyph glyph-name="sort_down" unicode="" horiz-adv-x="1024" -d="M1024 448q0 -26 -19 -45l-448 -448q-19 -19 -45 -19t-45 19l-448 448q-19 19 -19 45t19 45t45 19h896q26 0 45 -19t19 -45z" /> - <glyph glyph-name="sort_up" unicode="" horiz-adv-x="1024" -d="M1024 832q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45l448 448q19 19 45 19t45 -19l448 -448q19 -19 19 -45z" /> - <glyph glyph-name="envelope_alt" unicode="" horiz-adv-x="1792" -d="M1792 826v-794q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v794q44 -49 101 -87q362 -246 497 -345q57 -42 92.5 -65.5t94.5 -48t110 -24.5h1h1q51 0 110 24.5t94.5 48t92.5 65.5q170 123 498 345q57 39 100 87zM1792 1120q0 -79 -49 -151t-122 -123 -q-376 -261 -468 -325q-10 -7 -42.5 -30.5t-54 -38t-52 -32.5t-57.5 -27t-50 -9h-1h-1q-23 0 -50 9t-57.5 27t-52 32.5t-54 38t-42.5 30.5q-91 64 -262 182.5t-205 142.5q-62 42 -117 115.5t-55 136.5q0 78 41.5 130t118.5 52h1472q65 0 112.5 -47t47.5 -113z" /> - <glyph glyph-name="linkedin" unicode="" -d="M349 911v-991h-330v991h330zM370 1217q1 -73 -50.5 -122t-135.5 -49h-2q-82 0 -132 49t-50 122q0 74 51.5 122.5t134.5 48.5t133 -48.5t51 -122.5zM1536 488v-568h-329v530q0 105 -40.5 164.5t-126.5 59.5q-63 0 -105.5 -34.5t-63.5 -85.5q-11 -30 -11 -81v-553h-329 -q2 399 2 647t-1 296l-1 48h329v-144h-2q20 32 41 56t56.5 52t87 43.5t114.5 15.5q171 0 275 -113.5t104 -332.5z" /> - <glyph glyph-name="undo" unicode="" -d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61q-172 0 -327 72.5t-264 204.5q-7 10 -6.5 22.5t8.5 20.5l137 138q10 9 25 9q16 -2 23 -12q73 -95 179 -147t225 -52q104 0 198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5t-40.5 198.5t-109.5 163.5 -t-163.5 109.5t-198.5 40.5q-98 0 -188 -35.5t-160 -101.5l137 -138q31 -30 14 -69q-17 -40 -59 -40h-448q-26 0 -45 19t-19 45v448q0 42 40 59q39 17 69 -14l130 -129q107 101 244.5 156.5t284.5 55.5q156 0 298 -61t245 -164t164 -245t61 -298z" /> - <glyph glyph-name="legal" unicode="" horiz-adv-x="1792" -d="M1771 0q0 -53 -37 -90l-107 -108q-39 -37 -91 -37q-53 0 -90 37l-363 364q-38 36 -38 90q0 53 43 96l-256 256l-126 -126q-14 -14 -34 -14t-34 14q2 -2 12.5 -12t12.5 -13t10 -11.5t10 -13.5t6 -13.5t5.5 -16.5t1.5 -18q0 -38 -28 -68q-3 -3 -16.5 -18t-19 -20.5 -t-18.5 -16.5t-22 -15.5t-22 -9t-26 -4.5q-40 0 -68 28l-408 408q-28 28 -28 68q0 13 4.5 26t9 22t15.5 22t16.5 18.5t20.5 19t18 16.5q30 28 68 28q10 0 18 -1.5t16.5 -5.5t13.5 -6t13.5 -10t11.5 -10t13 -12.5t12 -12.5q-14 14 -14 34t14 34l348 348q14 14 34 14t34 -14 -q-2 2 -12.5 12t-12.5 13t-10 11.5t-10 13.5t-6 13.5t-5.5 16.5t-1.5 18q0 38 28 68q3 3 16.5 18t19 20.5t18.5 16.5t22 15.5t22 9t26 4.5q40 0 68 -28l408 -408q28 -28 28 -68q0 -13 -4.5 -26t-9 -22t-15.5 -22t-16.5 -18.5t-20.5 -19t-18 -16.5q-30 -28 -68 -28 -q-10 0 -18 1.5t-16.5 5.5t-13.5 6t-13.5 10t-11.5 10t-13 12.5t-12 12.5q14 -14 14 -34t-14 -34l-126 -126l256 -256q43 43 96 43q52 0 91 -37l363 -363q37 -39 37 -91z" /> - <glyph glyph-name="dashboard" unicode="" horiz-adv-x="1792" -d="M384 384q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM576 832q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1004 351l101 382q6 26 -7.5 48.5t-38.5 29.5 -t-48 -6.5t-30 -39.5l-101 -382q-60 -5 -107 -43.5t-63 -98.5q-20 -77 20 -146t117 -89t146 20t89 117q16 60 -6 117t-72 91zM1664 384q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 1024q0 53 -37.5 90.5 -t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1472 832q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1792 384q0 -261 -141 -483q-19 -29 -54 -29h-1402q-35 0 -54 29 -q-141 221 -141 483q0 182 71 348t191 286t286 191t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="comment_alt" unicode="" horiz-adv-x="1792" -d="M896 1152q-204 0 -381.5 -69.5t-282 -187.5t-104.5 -255q0 -112 71.5 -213.5t201.5 -175.5l87 -50l-27 -96q-24 -91 -70 -172q152 63 275 171l43 38l57 -6q69 -8 130 -8q204 0 381.5 69.5t282 187.5t104.5 255t-104.5 255t-282 187.5t-381.5 69.5zM1792 640 -q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22h-5q-15 0 -27 10.5t-16 27.5v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281q0 174 120 321.5 -t326 233t450 85.5t450 -85.5t326 -233t120 -321.5z" /> - <glyph glyph-name="comments_alt" unicode="" horiz-adv-x="1792" -d="M704 1152q-153 0 -286 -52t-211.5 -141t-78.5 -191q0 -82 53 -158t149 -132l97 -56l-35 -84q34 20 62 39l44 31l53 -10q78 -14 153 -14q153 0 286 52t211.5 141t78.5 191t-78.5 191t-211.5 141t-286 52zM704 1280q191 0 353.5 -68.5t256.5 -186.5t94 -257t-94 -257 -t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q-124 72 -195 177t-71 224q0 139 94 257t256.5 186.5 -t353.5 68.5zM1526 111q10 -24 20.5 -44t25 -38.5t22.5 -29t26 -29.5t23 -25q1 -1 4 -4.5t4.5 -5t4 -5t3.5 -5.5l2.5 -5t2 -6t0.5 -6.5t-1 -6.5q-3 -14 -13 -22t-22 -7q-50 7 -86 16q-154 40 -278 128q-90 -16 -176 -16q-271 0 -472 132q58 -4 88 -4q161 0 309 45t264 129 -q125 92 192 212t67 254q0 77 -23 152q129 -71 204 -178t75 -230q0 -120 -71 -224.5t-195 -176.5z" /> - <glyph glyph-name="bolt" unicode="" horiz-adv-x="896" -d="M885 970q18 -20 7 -44l-540 -1157q-13 -25 -42 -25q-4 0 -14 2q-17 5 -25.5 19t-4.5 30l197 808l-406 -101q-4 -1 -12 -1q-18 0 -31 11q-18 15 -13 39l201 825q4 14 16 23t28 9h328q19 0 32 -12.5t13 -29.5q0 -8 -5 -18l-171 -463l396 98q8 2 12 2q19 0 34 -15z" /> - <glyph glyph-name="sitemap" unicode="" horiz-adv-x="1792" -d="M1792 288v-320q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h96v192h-512v-192h96q40 0 68 -28t28 -68v-320q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h96v192h-512v-192h96q40 0 68 -28t28 -68v-320 -q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h96v192q0 52 38 90t90 38h512v192h-96q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h320q40 0 68 -28t28 -68v-320q0 -40 -28 -68t-68 -28h-96v-192h512q52 0 90 -38t38 -90v-192h96q40 0 68 -28t28 -68 -z" /> - <glyph glyph-name="umbrella" unicode="" horiz-adv-x="1664" -d="M896 708v-580q0 -104 -76 -180t-180 -76t-180 76t-76 180q0 26 19 45t45 19t45 -19t19 -45q0 -50 39 -89t89 -39t89 39t39 89v580q33 11 64 11t64 -11zM1664 681q0 -13 -9.5 -22.5t-22.5 -9.5q-11 0 -23 10q-49 46 -93 69t-102 23q-68 0 -128 -37t-103 -97 -q-7 -10 -17.5 -28t-14.5 -24q-11 -17 -28 -17q-18 0 -29 17q-4 6 -14.5 24t-17.5 28q-43 60 -102.5 97t-127.5 37t-127.5 -37t-102.5 -97q-7 -10 -17.5 -28t-14.5 -24q-11 -17 -29 -17q-17 0 -28 17q-4 6 -14.5 24t-17.5 28q-43 60 -103 97t-128 37q-58 0 -102 -23t-93 -69 -q-12 -10 -23 -10q-13 0 -22.5 9.5t-9.5 22.5q0 5 1 7q45 183 172.5 319.5t298 204.5t360.5 68q140 0 274.5 -40t246.5 -113.5t194.5 -187t115.5 -251.5q1 -2 1 -7zM896 1408v-98q-42 2 -64 2t-64 -2v98q0 26 19 45t45 19t45 -19t19 -45z" /> - <glyph glyph-name="paste" unicode="" horiz-adv-x="1792" -d="M768 -128h896v640h-416q-40 0 -68 28t-28 68v416h-384v-1152zM1024 1312v64q0 13 -9.5 22.5t-22.5 9.5h-704q-13 0 -22.5 -9.5t-9.5 -22.5v-64q0 -13 9.5 -22.5t22.5 -9.5h704q13 0 22.5 9.5t9.5 22.5zM1280 640h299l-299 299v-299zM1792 512v-672q0 -40 -28 -68t-68 -28 -h-960q-40 0 -68 28t-28 68v160h-544q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h1088q40 0 68 -28t28 -68v-328q21 -13 36 -28l408 -408q28 -28 48 -76t20 -88z" /> - <glyph glyph-name="light_bulb" unicode="" horiz-adv-x="1024" -d="M736 960q0 -13 -9.5 -22.5t-22.5 -9.5t-22.5 9.5t-9.5 22.5q0 46 -54 71t-106 25q-13 0 -22.5 9.5t-9.5 22.5t9.5 22.5t22.5 9.5q50 0 99.5 -16t87 -54t37.5 -90zM896 960q0 72 -34.5 134t-90 101.5t-123 62t-136.5 22.5t-136.5 -22.5t-123 -62t-90 -101.5t-34.5 -134 -q0 -101 68 -180q10 -11 30.5 -33t30.5 -33q128 -153 141 -298h228q13 145 141 298q10 11 30.5 33t30.5 33q68 79 68 180zM1024 960q0 -155 -103 -268q-45 -49 -74.5 -87t-59.5 -95.5t-34 -107.5q47 -28 47 -82q0 -37 -25 -64q25 -27 25 -64q0 -52 -45 -81q13 -23 13 -47 -q0 -46 -31.5 -71t-77.5 -25q-20 -44 -60 -70t-87 -26t-87 26t-60 70q-46 0 -77.5 25t-31.5 71q0 24 13 47q-45 29 -45 81q0 37 25 64q-25 27 -25 64q0 54 47 82q-4 50 -34 107.5t-59.5 95.5t-74.5 87q-103 113 -103 268q0 99 44.5 184.5t117 142t164 89t186.5 32.5 -t186.5 -32.5t164 -89t117 -142t44.5 -184.5z" /> - <glyph glyph-name="exchange" unicode="" horiz-adv-x="1792" -d="M1792 352v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5q-12 0 -24 10l-319 320q-9 9 -9 22q0 14 9 23l320 320q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5v-192h1376q13 0 22.5 -9.5t9.5 -22.5zM1792 896q0 -14 -9 -23l-320 -320q-9 -9 -23 -9 -q-13 0 -22.5 9.5t-9.5 22.5v192h-1376q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1376v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" /> - <glyph glyph-name="cloud_download" unicode="" horiz-adv-x="1920" -d="M1280 608q0 14 -9 23t-23 9h-224v352q0 13 -9.5 22.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -22.5v-352h-224q-13 0 -22.5 -9.5t-9.5 -22.5q0 -14 9 -23l352 -352q9 -9 23 -9t23 9l351 351q10 12 10 24zM1920 384q0 -159 -112.5 -271.5t-271.5 -112.5h-1088 -q-185 0 -316.5 131.5t-131.5 316.5q0 130 70 240t188 165q-2 30 -2 43q0 212 150 362t362 150q156 0 285.5 -87t188.5 -231q71 62 166 62q106 0 181 -75t75 -181q0 -76 -41 -138q130 -31 213.5 -135.5t83.5 -238.5z" /> - <glyph glyph-name="cloud_upload" unicode="" horiz-adv-x="1920" -d="M1280 672q0 14 -9 23l-352 352q-9 9 -23 9t-23 -9l-351 -351q-10 -12 -10 -24q0 -14 9 -23t23 -9h224v-352q0 -13 9.5 -22.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 22.5v352h224q13 0 22.5 9.5t9.5 22.5zM1920 384q0 -159 -112.5 -271.5t-271.5 -112.5h-1088 -q-185 0 -316.5 131.5t-131.5 316.5q0 130 70 240t188 165q-2 30 -2 43q0 212 150 362t362 150q156 0 285.5 -87t188.5 -231q71 62 166 62q106 0 181 -75t75 -181q0 -76 -41 -138q130 -31 213.5 -135.5t83.5 -238.5z" /> - <glyph glyph-name="user_md" unicode="" horiz-adv-x="1408" -d="M384 192q0 -26 -19 -45t-45 -19t-45 19t-19 45t19 45t45 19t45 -19t19 -45zM1408 131q0 -121 -73 -190t-194 -69h-874q-121 0 -194 69t-73 190q0 68 5.5 131t24 138t47.5 132.5t81 103t120 60.5q-22 -52 -22 -120v-203q-58 -20 -93 -70t-35 -111q0 -80 56 -136t136 -56 -t136 56t56 136q0 61 -35.5 111t-92.5 70v203q0 62 25 93q132 -104 295 -104t295 104q25 -31 25 -93v-64q-106 0 -181 -75t-75 -181v-89q-32 -29 -32 -71q0 -40 28 -68t68 -28t68 28t28 68q0 42 -32 71v89q0 52 38 90t90 38t90 -38t38 -90v-89q-32 -29 -32 -71q0 -40 28 -68 -t68 -28t68 28t28 68q0 42 -32 71v89q0 68 -34.5 127.5t-93.5 93.5q0 10 0.5 42.5t0 48t-2.5 41.5t-7 47t-13 40q68 -15 120 -60.5t81 -103t47.5 -132.5t24 -138t5.5 -131zM1088 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5 -t271.5 -112.5t112.5 -271.5z" /> - <glyph glyph-name="stethoscope" unicode="" horiz-adv-x="1408" -d="M1280 832q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 832q0 -62 -35.5 -111t-92.5 -70v-395q0 -159 -131.5 -271.5t-316.5 -112.5t-316.5 112.5t-131.5 271.5v132q-164 20 -274 128t-110 252v512q0 26 19 45t45 19q6 0 16 -2q17 30 47 48 -t65 18q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5q-33 0 -64 18v-402q0 -106 94 -181t226 -75t226 75t94 181v402q-31 -18 -64 -18q-53 0 -90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5q35 0 65 -18t47 -48q10 2 16 2q26 0 45 -19t19 -45v-512q0 -144 -110 -252 -t-274 -128v-132q0 -106 94 -181t226 -75t226 75t94 181v395q-57 21 -92.5 70t-35.5 111q0 80 56 136t136 56t136 -56t56 -136z" /> - <glyph glyph-name="suitcase" unicode="" horiz-adv-x="1792" -d="M640 1152h512v128h-512v-128zM288 1152v-1280h-64q-92 0 -158 66t-66 158v832q0 92 66 158t158 66h64zM1408 1152v-1280h-1024v1280h128v160q0 40 28 68t68 28h576q40 0 68 -28t28 -68v-160h128zM1792 928v-832q0 -92 -66 -158t-158 -66h-64v1280h64q92 0 158 -66 -t66 -158z" /> - <glyph glyph-name="bell_alt" unicode="" horiz-adv-x="1792" -d="M912 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM1728 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q50 42 91 88t85 119.5t74.5 158.5 -t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q190 -28 307 -158.5t117 -282.5q0 -139 19.5 -260t50 -206t74.5 -158.5t85 -119.5t91 -88z" /> - <glyph glyph-name="coffee" unicode="" horiz-adv-x="1920" -d="M1664 896q0 80 -56 136t-136 56h-64v-384h64q80 0 136 56t56 136zM0 128h1792q0 -106 -75 -181t-181 -75h-1280q-106 0 -181 75t-75 181zM1856 896q0 -159 -112.5 -271.5t-271.5 -112.5h-64v-32q0 -92 -66 -158t-158 -66h-704q-92 0 -158 66t-66 158v736q0 26 19 45 -t45 19h1152q159 0 271.5 -112.5t112.5 -271.5z" /> - <glyph glyph-name="food" unicode="" horiz-adv-x="1408" -d="M640 1472v-640q0 -61 -35.5 -111t-92.5 -70v-779q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v779q-57 20 -92.5 70t-35.5 111v640q0 26 19 45t45 19t45 -19t19 -45v-416q0 -26 19 -45t45 -19t45 19t19 45v416q0 26 19 45t45 19t45 -19t19 -45v-416q0 -26 19 -45 -t45 -19t45 19t19 45v416q0 26 19 45t45 19t45 -19t19 -45zM1408 1472v-1600q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v512h-224q-13 0 -22.5 9.5t-9.5 22.5v800q0 132 94 226t226 94h256q26 0 45 -19t19 -45z" /> - <glyph glyph-name="file_text_alt" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -M384 736q0 14 9 23t23 9h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64zM1120 512q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704zM1120 256q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704 -q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704z" /> - <glyph glyph-name="building" unicode="" horiz-adv-x="1408" -d="M384 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M640 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M1152 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM896 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M640 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 992v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M1152 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM896 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M640 992v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 1248v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M1152 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM896 992v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M640 1248v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1152 992v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M896 1248v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1152 1248v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M896 -128h384v1536h-1152v-1536h384v224q0 13 9.5 22.5t22.5 9.5h320q13 0 22.5 -9.5t9.5 -22.5v-224zM1408 1472v-1664q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v1664q0 26 19 45t45 19h1280q26 0 45 -19t19 -45z" /> - <glyph glyph-name="hospital" unicode="" horiz-adv-x="1408" -d="M384 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M640 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M1152 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM896 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M640 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1152 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M896 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1152 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z -M896 -128h384v1152h-256v-32q0 -40 -28 -68t-68 -28h-448q-40 0 -68 28t-28 68v32h-256v-1152h384v224q0 13 9.5 22.5t22.5 9.5h320q13 0 22.5 -9.5t9.5 -22.5v-224zM896 1056v320q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-96h-128v96q0 13 -9.5 22.5 -t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5v96h128v-96q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1408 1088v-1280q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v1280q0 26 19 45t45 19h320 -v288q0 40 28 68t68 28h448q40 0 68 -28t28 -68v-288h320q26 0 45 -19t19 -45z" /> - <glyph glyph-name="ambulance" unicode="" horiz-adv-x="1920" -d="M640 128q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM256 640h384v256h-158q-14 -2 -22 -9l-195 -195q-7 -12 -9 -22v-30zM1536 128q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5 -t90.5 37.5t37.5 90.5zM1664 800v192q0 14 -9 23t-23 9h-224v224q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-224h-224q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h224v-224q0 -14 9 -23t23 -9h192q14 0 23 9t9 23v224h224q14 0 23 9t9 23zM1920 1344v-1152 -q0 -26 -19 -45t-45 -19h-192q0 -106 -75 -181t-181 -75t-181 75t-75 181h-384q0 -106 -75 -181t-181 -75t-181 75t-75 181h-128q-26 0 -45 19t-19 45t19 45t45 19v416q0 26 13 58t32 51l198 198q19 19 51 32t58 13h160v320q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" /> - <glyph glyph-name="medkit" unicode="" horiz-adv-x="1792" -d="M1280 416v192q0 14 -9 23t-23 9h-224v224q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-224h-224q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h224v-224q0 -14 9 -23t23 -9h192q14 0 23 9t9 23v224h224q14 0 23 9t9 23zM640 1152h512v128h-512v-128zM256 1152v-1280h-32 -q-92 0 -158 66t-66 158v832q0 92 66 158t158 66h32zM1440 1152v-1280h-1088v1280h160v160q0 40 28 68t68 28h576q40 0 68 -28t28 -68v-160h160zM1792 928v-832q0 -92 -66 -158t-158 -66h-32v1280h32q92 0 158 -66t66 -158z" /> - <glyph glyph-name="fighter_jet" unicode="" horiz-adv-x="1920" -d="M1920 576q-1 -32 -288 -96l-352 -32l-224 -64h-64l-293 -352h69q26 0 45 -4.5t19 -11.5t-19 -11.5t-45 -4.5h-96h-160h-64v32h64v416h-160l-192 -224h-96l-32 32v192h32v32h128v8l-192 24v128l192 24v8h-128v32h-32v192l32 32h96l192 -224h160v416h-64v32h64h160h96 -q26 0 45 -4.5t19 -11.5t-19 -11.5t-45 -4.5h-69l293 -352h64l224 -64l352 -32q128 -28 200 -52t80 -34z" /> - <glyph glyph-name="beer" unicode="" horiz-adv-x="1664" -d="M640 640v384h-256v-256q0 -53 37.5 -90.5t90.5 -37.5h128zM1664 192v-192h-1152v192l128 192h-128q-159 0 -271.5 112.5t-112.5 271.5v320l-64 64l32 128h480l32 128h960l32 -192l-64 -32v-800z" /> - <glyph glyph-name="h_sign" unicode="" -d="M1280 192v896q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-320h-512v320q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-896q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v320h512v-320q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1536 1120v-960 -q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="f0fe" unicode="" -d="M1280 576v128q0 26 -19 45t-45 19h-320v320q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-320h-320q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h320v-320q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v320h320q26 0 45 19t19 45zM1536 1120v-960 -q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="double_angle_left" unicode="" horiz-adv-x="1024" -d="M627 160q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l50 -50q10 -10 10 -23t-10 -23l-393 -393l393 -393q10 -10 10 -23zM1011 160q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23 -t10 23l466 466q10 10 23 10t23 -10l50 -50q10 -10 10 -23t-10 -23l-393 -393l393 -393q10 -10 10 -23z" /> - <glyph glyph-name="double_angle_right" unicode="" horiz-adv-x="1024" -d="M595 576q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l466 -466q10 -10 10 -23zM979 576q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23 -l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l466 -466q10 -10 10 -23z" /> - <glyph glyph-name="double_angle_up" unicode="" horiz-adv-x="1152" -d="M1075 224q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-393 393l-393 -393q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l466 -466q10 -10 10 -23zM1075 608q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-393 393l-393 -393 -q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l466 -466q10 -10 10 -23z" /> - <glyph glyph-name="double_angle_down" unicode="" horiz-adv-x="1152" -d="M1075 672q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l393 -393l393 393q10 10 23 10t23 -10l50 -50q10 -10 10 -23zM1075 1056q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23 -t10 23l50 50q10 10 23 10t23 -10l393 -393l393 393q10 10 23 10t23 -10l50 -50q10 -10 10 -23z" /> - <glyph glyph-name="angle_left" unicode="" horiz-adv-x="640" -d="M627 992q0 -13 -10 -23l-393 -393l393 -393q10 -10 10 -23t-10 -23l-50 -50q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l50 -50q10 -10 10 -23z" /> - <glyph glyph-name="angle_right" unicode="" horiz-adv-x="640" -d="M595 576q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l466 -466q10 -10 10 -23z" /> - <glyph glyph-name="angle_up" unicode="" horiz-adv-x="1152" -d="M1075 352q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-393 393l-393 -393q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l466 -466q10 -10 10 -23z" /> - <glyph glyph-name="angle_down" unicode="" horiz-adv-x="1152" -d="M1075 800q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l393 -393l393 393q10 10 23 10t23 -10l50 -50q10 -10 10 -23z" /> - <glyph glyph-name="desktop" unicode="" horiz-adv-x="1920" -d="M1792 544v832q0 13 -9.5 22.5t-22.5 9.5h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-832q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5zM1920 1376v-1088q0 -66 -47 -113t-113 -47h-544q0 -37 16 -77.5t32 -71t16 -43.5q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19 -t-19 45q0 14 16 44t32 70t16 78h-544q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" /> - <glyph glyph-name="laptop" unicode="" horiz-adv-x="1920" -d="M416 256q-66 0 -113 47t-47 113v704q0 66 47 113t113 47h1088q66 0 113 -47t47 -113v-704q0 -66 -47 -113t-113 -47h-1088zM384 1120v-704q0 -13 9.5 -22.5t22.5 -9.5h1088q13 0 22.5 9.5t9.5 22.5v704q0 13 -9.5 22.5t-22.5 9.5h-1088q-13 0 -22.5 -9.5t-9.5 -22.5z -M1760 192h160v-96q0 -40 -47 -68t-113 -28h-1600q-66 0 -113 28t-47 68v96h160h1600zM1040 96q16 0 16 16t-16 16h-160q-16 0 -16 -16t16 -16h160z" /> - <glyph glyph-name="tablet" unicode="" horiz-adv-x="1152" -d="M640 128q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1024 288v960q0 13 -9.5 22.5t-22.5 9.5h-832q-13 0 -22.5 -9.5t-9.5 -22.5v-960q0 -13 9.5 -22.5t22.5 -9.5h832q13 0 22.5 9.5t9.5 22.5zM1152 1248v-1088q0 -66 -47 -113t-113 -47h-832 -q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h832q66 0 113 -47t47 -113z" /> - <glyph glyph-name="mobile_phone" unicode="" horiz-adv-x="768" -d="M464 128q0 33 -23.5 56.5t-56.5 23.5t-56.5 -23.5t-23.5 -56.5t23.5 -56.5t56.5 -23.5t56.5 23.5t23.5 56.5zM672 288v704q0 13 -9.5 22.5t-22.5 9.5h-512q-13 0 -22.5 -9.5t-9.5 -22.5v-704q0 -13 9.5 -22.5t22.5 -9.5h512q13 0 22.5 9.5t9.5 22.5zM480 1136 -q0 16 -16 16h-160q-16 0 -16 -16t16 -16h160q16 0 16 16zM768 1152v-1024q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v1024q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" /> - <glyph glyph-name="circle_blank" unicode="" -d="M768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103 -t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="quote_left" unicode="" horiz-adv-x="1664" -d="M768 576v-384q0 -80 -56 -136t-136 -56h-384q-80 0 -136 56t-56 136v704q0 104 40.5 198.5t109.5 163.5t163.5 109.5t198.5 40.5h64q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-64q-106 0 -181 -75t-75 -181v-32q0 -40 28 -68t68 -28h224q80 0 136 -56t56 -136z -M1664 576v-384q0 -80 -56 -136t-136 -56h-384q-80 0 -136 56t-56 136v704q0 104 40.5 198.5t109.5 163.5t163.5 109.5t198.5 40.5h64q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-64q-106 0 -181 -75t-75 -181v-32q0 -40 28 -68t68 -28h224q80 0 136 -56t56 -136z" /> - <glyph glyph-name="quote_right" unicode="" horiz-adv-x="1664" -d="M768 1216v-704q0 -104 -40.5 -198.5t-109.5 -163.5t-163.5 -109.5t-198.5 -40.5h-64q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h64q106 0 181 75t75 181v32q0 40 -28 68t-68 28h-224q-80 0 -136 56t-56 136v384q0 80 56 136t136 56h384q80 0 136 -56t56 -136zM1664 1216 -v-704q0 -104 -40.5 -198.5t-109.5 -163.5t-163.5 -109.5t-198.5 -40.5h-64q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h64q106 0 181 75t75 181v32q0 40 -28 68t-68 28h-224q-80 0 -136 56t-56 136v384q0 80 56 136t136 56h384q80 0 136 -56t56 -136z" /> - <glyph glyph-name="spinner" unicode="" horiz-adv-x="1792" -d="M526 142q0 -53 -37.5 -90.5t-90.5 -37.5q-52 0 -90 38t-38 90q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1024 -64q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM320 640q0 -53 -37.5 -90.5t-90.5 -37.5 -t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1522 142q0 -52 -38 -90t-90 -38q-53 0 -90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM558 1138q0 -66 -47 -113t-113 -47t-113 47t-47 113t47 113t113 47t113 -47t47 -113z -M1728 640q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1088 1344q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1618 1138q0 -93 -66 -158.5t-158 -65.5q-93 0 -158.5 65.5t-65.5 158.5 -q0 92 65.5 158t158.5 66q92 0 158 -66t66 -158z" /> - <glyph glyph-name="circle" unicode="" -d="M1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="reply" unicode="" horiz-adv-x="1792" -d="M1792 416q0 -166 -127 -451q-3 -7 -10.5 -24t-13.5 -30t-13 -22q-12 -17 -28 -17q-15 0 -23.5 10t-8.5 25q0 9 2.5 26.5t2.5 23.5q5 68 5 123q0 101 -17.5 181t-48.5 138.5t-80 101t-105.5 69.5t-133 42.5t-154 21.5t-175.5 6h-224v-256q0 -26 -19 -45t-45 -19t-45 19 -l-512 512q-19 19 -19 45t19 45l512 512q19 19 45 19t45 -19t19 -45v-256h224q713 0 875 -403q53 -134 53 -333z" /> - <glyph glyph-name="github_alt" unicode="" horiz-adv-x="1664" -d="M640 320q0 -40 -12.5 -82t-43 -76t-72.5 -34t-72.5 34t-43 76t-12.5 82t12.5 82t43 76t72.5 34t72.5 -34t43 -76t12.5 -82zM1280 320q0 -40 -12.5 -82t-43 -76t-72.5 -34t-72.5 34t-43 76t-12.5 82t12.5 82t43 76t72.5 34t72.5 -34t43 -76t12.5 -82zM1440 320 -q0 120 -69 204t-187 84q-41 0 -195 -21q-71 -11 -157 -11t-157 11q-152 21 -195 21q-118 0 -187 -84t-69 -204q0 -88 32 -153.5t81 -103t122 -60t140 -29.5t149 -7h168q82 0 149 7t140 29.5t122 60t81 103t32 153.5zM1664 496q0 -207 -61 -331q-38 -77 -105.5 -133t-141 -86 -t-170 -47.5t-171.5 -22t-167 -4.5q-78 0 -142 3t-147.5 12.5t-152.5 30t-137 51.5t-121 81t-86 115q-62 123 -62 331q0 237 136 396q-27 82 -27 170q0 116 51 218q108 0 190 -39.5t189 -123.5q147 35 309 35q148 0 280 -32q105 82 187 121t189 39q51 -102 51 -218 -q0 -87 -27 -168q136 -160 136 -398z" /> - <glyph glyph-name="folder_close_alt" unicode="" horiz-adv-x="1664" -d="M1536 224v704q0 40 -28 68t-68 28h-704q-40 0 -68 28t-28 68v64q0 40 -28 68t-68 28h-320q-40 0 -68 -28t-28 -68v-960q0 -40 28 -68t68 -28h1216q40 0 68 28t28 68zM1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320 -q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" /> - <glyph glyph-name="folder_open_alt" unicode="" horiz-adv-x="1920" -d="M1781 605q0 35 -53 35h-1088q-40 0 -85.5 -21.5t-71.5 -52.5l-294 -363q-18 -24 -18 -40q0 -35 53 -35h1088q40 0 86 22t71 53l294 363q18 22 18 39zM640 768h768v160q0 40 -28 68t-68 28h-576q-40 0 -68 28t-28 68v64q0 40 -28 68t-68 28h-320q-40 0 -68 -28t-28 -68 -v-853l256 315q44 53 116 87.5t140 34.5zM1909 605q0 -62 -46 -120l-295 -363q-43 -53 -116 -87.5t-140 -34.5h-1088q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158v-160h192q54 0 99 -24.5t67 -70.5q15 -32 15 -68z -" /> - <glyph glyph-name="expand_alt" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="collapse_alt" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="smile" unicode="" -d="M1134 461q-37 -121 -138 -195t-228 -74t-228 74t-138 195q-8 25 4 48.5t38 31.5q25 8 48.5 -4t31.5 -38q25 -80 92.5 -129.5t151.5 -49.5t151.5 49.5t92.5 129.5q8 26 32 38t49 4t37 -31.5t4 -48.5zM640 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5 -t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1152 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5 -t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="frown" unicode="" -d="M1134 307q8 -25 -4 -48.5t-37 -31.5t-49 4t-32 38q-25 80 -92.5 129.5t-151.5 49.5t-151.5 -49.5t-92.5 -129.5q-8 -26 -31.5 -38t-48.5 -4q-26 8 -38 31.5t-4 48.5q37 121 138 195t228 74t228 -74t138 -195zM640 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5 -t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1152 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-204 -136.5t-136.5 -204 -t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="meh" unicode="" -d="M1152 448q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45t19 45t45 19h640q26 0 45 -19t19 -45zM640 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1152 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5 -t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5zM1536 640 -q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="gamepad" unicode="" horiz-adv-x="1920" -d="M832 448v128q0 14 -9 23t-23 9h-192v192q0 14 -9 23t-23 9h-128q-14 0 -23 -9t-9 -23v-192h-192q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h192v-192q0 -14 9 -23t23 -9h128q14 0 23 9t9 23v192h192q14 0 23 9t9 23zM1408 384q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5 -t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 640q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1920 512q0 -212 -150 -362t-362 -150q-192 0 -338 128h-220q-146 -128 -338 -128q-212 0 -362 150 -t-150 362t150 362t362 150h896q212 0 362 -150t150 -362z" /> - <glyph glyph-name="keyboard" unicode="" horiz-adv-x="1920" -d="M384 368v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM512 624v-96q0 -16 -16 -16h-224q-16 0 -16 16v96q0 16 16 16h224q16 0 16 -16zM384 880v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM1408 368v-96q0 -16 -16 -16 -h-864q-16 0 -16 16v96q0 16 16 16h864q16 0 16 -16zM768 624v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM640 880v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM1024 624v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16 -h96q16 0 16 -16zM896 880v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM1280 624v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM1664 368v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM1152 880v-96 -q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM1408 880v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM1664 880v-352q0 -16 -16 -16h-224q-16 0 -16 16v96q0 16 16 16h112v240q0 16 16 16h96q16 0 16 -16zM1792 128v896h-1664v-896 -h1664zM1920 1024v-896q0 -53 -37.5 -90.5t-90.5 -37.5h-1664q-53 0 -90.5 37.5t-37.5 90.5v896q0 53 37.5 90.5t90.5 37.5h1664q53 0 90.5 -37.5t37.5 -90.5z" /> - <glyph glyph-name="flag_alt" unicode="" horiz-adv-x="1792" -d="M1664 491v616q-169 -91 -306 -91q-82 0 -145 32q-100 49 -184 76.5t-178 27.5q-173 0 -403 -127v-599q245 113 433 113q55 0 103.5 -7.5t98 -26t77 -31t82.5 -39.5l28 -14q44 -22 101 -22q120 0 293 92zM320 1280q0 -35 -17.5 -64t-46.5 -46v-1266q0 -14 -9 -23t-23 -9 -h-64q-14 0 -23 9t-9 23v1266q-29 17 -46.5 46t-17.5 64q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -39 -35 -57q-10 -5 -17 -9q-218 -116 -369 -116q-88 0 -158 35l-28 14q-64 33 -99 48t-91 29t-114 14q-102 0 -235.5 -44t-228.5 -102 -q-15 -9 -33 -9q-16 0 -32 8q-32 19 -32 56v742q0 35 31 55q35 21 78.5 42.5t114 52t152.5 49.5t155 19q112 0 209 -31t209 -86q38 -19 89 -19q122 0 310 112q22 12 31 17q31 16 62 -2q31 -20 31 -55z" /> - <glyph glyph-name="flag_checkered" unicode="" horiz-adv-x="1792" -d="M832 536v192q-181 -16 -384 -117v-185q205 96 384 110zM832 954v197q-172 -8 -384 -126v-189q215 111 384 118zM1664 491v184q-235 -116 -384 -71v224q-20 6 -39 15q-5 3 -33 17t-34.5 17t-31.5 15t-34.5 15.5t-32.5 13t-36 12.5t-35 8.5t-39.5 7.5t-39.5 4t-44 2 -q-23 0 -49 -3v-222h19q102 0 192.5 -29t197.5 -82q19 -9 39 -15v-188q42 -17 91 -17q120 0 293 92zM1664 918v189q-169 -91 -306 -91q-45 0 -78 8v-196q148 -42 384 90zM320 1280q0 -35 -17.5 -64t-46.5 -46v-1266q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v1266 -q-29 17 -46.5 46t-17.5 64q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -39 -35 -57q-10 -5 -17 -9q-218 -116 -369 -116q-88 0 -158 35l-28 14q-64 33 -99 48t-91 29t-114 14q-102 0 -235.5 -44t-228.5 -102q-15 -9 -33 -9q-16 0 -32 8 -q-32 19 -32 56v742q0 35 31 55q35 21 78.5 42.5t114 52t152.5 49.5t155 19q112 0 209 -31t209 -86q38 -19 89 -19q122 0 310 112q22 12 31 17q31 16 62 -2q31 -20 31 -55z" /> - <glyph glyph-name="terminal" unicode="" horiz-adv-x="1664" -d="M585 553l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l466 -466q10 -10 10 -23t-10 -23zM1664 96v-64q0 -14 -9 -23t-23 -9h-960q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h960q14 0 23 -9 -t9 -23z" /> - <glyph glyph-name="code" unicode="" horiz-adv-x="1920" -d="M617 137l-50 -50q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l50 -50q10 -10 10 -23t-10 -23l-393 -393l393 -393q10 -10 10 -23t-10 -23zM1208 1204l-373 -1291q-4 -13 -15.5 -19.5t-23.5 -2.5l-62 17q-13 4 -19.5 15.5t-2.5 24.5 -l373 1291q4 13 15.5 19.5t23.5 2.5l62 -17q13 -4 19.5 -15.5t2.5 -24.5zM1865 553l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l466 -466q10 -10 10 -23t-10 -23z" /> - <glyph glyph-name="reply_all" unicode="" horiz-adv-x="1792" -d="M640 454v-70q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-512 512q-19 19 -19 45t19 45l512 512q29 31 70 14q39 -17 39 -59v-69l-397 -398q-19 -19 -19 -45t19 -45zM1792 416q0 -58 -17 -133.5t-38.5 -138t-48 -125t-40.5 -90.5l-20 -40q-8 -17 -28 -17q-6 0 -9 1 -q-25 8 -23 34q43 400 -106 565q-64 71 -170.5 110.5t-267.5 52.5v-251q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-512 512q-19 19 -19 45t19 45l512 512q29 31 70 14q39 -17 39 -59v-262q411 -28 599 -221q169 -173 169 -509z" /> - <glyph glyph-name="star_half_empty" unicode="" horiz-adv-x="1664" -d="M1186 579l257 250l-356 52l-66 10l-30 60l-159 322v-963l59 -31l318 -168l-60 355l-12 66zM1638 841l-363 -354l86 -500q5 -33 -6 -51.5t-34 -18.5q-17 0 -40 12l-449 236l-449 -236q-23 -12 -40 -12q-23 0 -34 18.5t-6 51.5l86 500l-364 354q-32 32 -23 59.5t54 34.5 -l502 73l225 455q20 41 49 41q28 0 49 -41l225 -455l502 -73q45 -7 54 -34.5t-24 -59.5z" /> - <glyph glyph-name="location_arrow" unicode="" horiz-adv-x="1408" -d="M1401 1187l-640 -1280q-17 -35 -57 -35q-5 0 -15 2q-22 5 -35.5 22.5t-13.5 39.5v576h-576q-22 0 -39.5 13.5t-22.5 35.5t4 42t29 30l1280 640q13 7 29 7q27 0 45 -19q15 -14 18.5 -34.5t-6.5 -39.5z" /> - <glyph glyph-name="crop" unicode="" horiz-adv-x="1664" -d="M557 256h595v595zM512 301l595 595h-595v-595zM1664 224v-192q0 -14 -9 -23t-23 -9h-224v-224q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v224h-864q-14 0 -23 9t-9 23v864h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224v224q0 14 9 23t23 9h192q14 0 23 -9t9 -23 -v-224h851l246 247q10 9 23 9t23 -9q9 -10 9 -23t-9 -23l-247 -246v-851h224q14 0 23 -9t9 -23z" /> - <glyph glyph-name="code_fork" unicode="" horiz-adv-x="1024" -d="M288 64q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM288 1216q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM928 1088q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM1024 1088q0 -52 -26 -96.5t-70 -69.5 -q-2 -287 -226 -414q-67 -38 -203 -81q-128 -40 -169.5 -71t-41.5 -100v-26q44 -25 70 -69.5t26 -96.5q0 -80 -56 -136t-136 -56t-136 56t-56 136q0 52 26 96.5t70 69.5v820q-44 25 -70 69.5t-26 96.5q0 80 56 136t136 56t136 -56t56 -136q0 -52 -26 -96.5t-70 -69.5v-497 -q54 26 154 57q55 17 87.5 29.5t70.5 31t59 39.5t40.5 51t28 69.5t8.5 91.5q-44 25 -70 69.5t-26 96.5q0 80 56 136t136 56t136 -56t56 -136z" /> - <glyph glyph-name="unlink" unicode="" horiz-adv-x="1664" -d="M439 265l-256 -256q-11 -9 -23 -9t-23 9q-9 10 -9 23t9 23l256 256q10 9 23 9t23 -9q9 -10 9 -23t-9 -23zM608 224v-320q0 -14 -9 -23t-23 -9t-23 9t-9 23v320q0 14 9 23t23 9t23 -9t9 -23zM384 448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23t9 23t23 9h320 -q14 0 23 -9t9 -23zM1648 320q0 -120 -85 -203l-147 -146q-83 -83 -203 -83q-121 0 -204 85l-334 335q-21 21 -42 56l239 18l273 -274q27 -27 68 -27.5t68 26.5l147 146q28 28 28 67q0 40 -28 68l-274 275l18 239q35 -21 56 -42l336 -336q84 -86 84 -204zM1031 1044l-239 -18 -l-273 274q-28 28 -68 28q-39 0 -68 -27l-147 -146q-28 -28 -28 -67q0 -40 28 -68l274 -274l-18 -240q-35 21 -56 42l-336 336q-84 86 -84 204q0 120 85 203l147 146q83 83 203 83q121 0 204 -85l334 -335q21 -21 42 -56zM1664 960q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9 -t-9 23t9 23t23 9h320q14 0 23 -9t9 -23zM1120 1504v-320q0 -14 -9 -23t-23 -9t-23 9t-9 23v320q0 14 9 23t23 9t23 -9t9 -23zM1527 1353l-256 -256q-11 -9 -23 -9t-23 9q-9 10 -9 23t9 23l256 256q10 9 23 9t23 -9q9 -10 9 -23t-9 -23z" /> - <glyph glyph-name="question" unicode="" horiz-adv-x="1024" -d="M704 280v-240q0 -16 -12 -28t-28 -12h-240q-16 0 -28 12t-12 28v240q0 16 12 28t28 12h240q16 0 28 -12t12 -28zM1020 880q0 -54 -15.5 -101t-35 -76.5t-55 -59.5t-57.5 -43.5t-61 -35.5q-41 -23 -68.5 -65t-27.5 -67q0 -17 -12 -32.5t-28 -15.5h-240q-15 0 -25.5 18.5 -t-10.5 37.5v45q0 83 65 156.5t143 108.5q59 27 84 56t25 76q0 42 -46.5 74t-107.5 32q-65 0 -108 -29q-35 -25 -107 -115q-13 -16 -31 -16q-12 0 -25 8l-164 125q-13 10 -15.5 25t5.5 28q160 266 464 266q80 0 161 -31t146 -83t106 -127.5t41 -158.5z" /> - <glyph glyph-name="_279" unicode="" horiz-adv-x="640" -d="M640 192v-128q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h64v384h-64q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-576h64q26 0 45 -19t19 -45zM512 1344v-192q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v192 -q0 26 19 45t45 19h256q26 0 45 -19t19 -45z" /> - <glyph glyph-name="exclamation" unicode="" horiz-adv-x="640" -d="M512 288v-224q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v224q0 26 19 45t45 19h256q26 0 45 -19t19 -45zM542 1344l-28 -768q-1 -26 -20.5 -45t-45.5 -19h-256q-26 0 -45.5 19t-20.5 45l-28 768q-1 26 17.5 45t44.5 19h320q26 0 44.5 -19t17.5 -45z" /> - <glyph glyph-name="superscript" unicode="" -d="M897 167v-167h-248l-159 252l-24 42q-8 9 -11 21h-3q-1 -3 -2.5 -6.5t-3.5 -8t-3 -6.5q-10 -20 -25 -44l-155 -250h-258v167h128l197 291l-185 272h-137v168h276l139 -228q2 -4 23 -42q8 -9 11 -21h3q3 9 11 21l25 42l140 228h257v-168h-125l-184 -267l204 -296h109z -M1534 846v-206h-514l-3 27q-4 28 -4 46q0 64 26 117t65 86.5t84 65t84 54.5t65 54t26 64q0 38 -29.5 62.5t-70.5 24.5q-51 0 -97 -39q-14 -11 -36 -38l-105 92q26 37 63 66q83 65 188 65q110 0 178 -59.5t68 -158.5q0 -56 -24.5 -103t-62 -76.5t-81.5 -58.5t-82 -50.5 -t-65.5 -51.5t-30.5 -63h232v80h126z" /> - <glyph glyph-name="subscript" unicode="" -d="M897 167v-167h-248l-159 252l-24 42q-8 9 -11 21h-3q-1 -3 -2.5 -6.5t-3.5 -8t-3 -6.5q-10 -20 -25 -44l-155 -250h-258v167h128l197 291l-185 272h-137v168h276l139 -228q2 -4 23 -42q8 -9 11 -21h3q3 9 11 21l25 42l140 228h257v-168h-125l-184 -267l204 -296h109z -M1536 -50v-206h-514l-4 27q-3 45 -3 46q0 64 26 117t65 86.5t84 65t84 54.5t65 54t26 64q0 38 -29.5 62.5t-70.5 24.5q-51 0 -97 -39q-14 -11 -36 -38l-105 92q26 37 63 66q80 65 188 65q110 0 178 -59.5t68 -158.5q0 -66 -34.5 -118.5t-84 -86t-99.5 -62.5t-87 -63t-41 -73 -h232v80h126z" /> - <glyph glyph-name="_283" unicode="" horiz-adv-x="1920" -d="M896 128l336 384h-768l-336 -384h768zM1909 1205q15 -34 9.5 -71.5t-30.5 -65.5l-896 -1024q-38 -44 -96 -44h-768q-38 0 -69.5 20.5t-47.5 54.5q-15 34 -9.5 71.5t30.5 65.5l896 1024q38 44 96 44h768q38 0 69.5 -20.5t47.5 -54.5z" /> - <glyph glyph-name="puzzle_piece" unicode="" horiz-adv-x="1664" -d="M1664 438q0 -81 -44.5 -135t-123.5 -54q-41 0 -77.5 17.5t-59 38t-56.5 38t-71 17.5q-110 0 -110 -124q0 -39 16 -115t15 -115v-5q-22 0 -33 -1q-34 -3 -97.5 -11.5t-115.5 -13.5t-98 -5q-61 0 -103 26.5t-42 83.5q0 37 17.5 71t38 56.5t38 59t17.5 77.5q0 79 -54 123.5 -t-135 44.5q-84 0 -143 -45.5t-59 -127.5q0 -43 15 -83t33.5 -64.5t33.5 -53t15 -50.5q0 -45 -46 -89q-37 -35 -117 -35q-95 0 -245 24q-9 2 -27.5 4t-27.5 4l-13 2q-1 0 -3 1q-2 0 -2 1v1024q2 -1 17.5 -3.5t34 -5t21.5 -3.5q150 -24 245 -24q80 0 117 35q46 44 46 89 -q0 22 -15 50.5t-33.5 53t-33.5 64.5t-15 83q0 82 59 127.5t144 45.5q80 0 134 -44.5t54 -123.5q0 -41 -17.5 -77.5t-38 -59t-38 -56.5t-17.5 -71q0 -57 42 -83.5t103 -26.5q64 0 180 15t163 17v-2q-1 -2 -3.5 -17.5t-5 -34t-3.5 -21.5q-24 -150 -24 -245q0 -80 35 -117 -q44 -46 89 -46q22 0 50.5 15t53 33.5t64.5 33.5t83 15q82 0 127.5 -59t45.5 -143z" /> - <glyph glyph-name="microphone" unicode="" horiz-adv-x="1152" -d="M1152 832v-128q0 -221 -147.5 -384.5t-364.5 -187.5v-132h256q26 0 45 -19t19 -45t-19 -45t-45 -19h-640q-26 0 -45 19t-19 45t19 45t45 19h256v132q-217 24 -364.5 187.5t-147.5 384.5v128q0 26 19 45t45 19t45 -19t19 -45v-128q0 -185 131.5 -316.5t316.5 -131.5 -t316.5 131.5t131.5 316.5v128q0 26 19 45t45 19t45 -19t19 -45zM896 1216v-512q0 -132 -94 -226t-226 -94t-226 94t-94 226v512q0 132 94 226t226 94t226 -94t94 -226z" /> - <glyph glyph-name="microphone_off" unicode="" horiz-adv-x="1408" -d="M271 591l-101 -101q-42 103 -42 214v128q0 26 19 45t45 19t45 -19t19 -45v-128q0 -53 15 -113zM1385 1193l-361 -361v-128q0 -132 -94 -226t-226 -94q-55 0 -109 19l-96 -96q97 -51 205 -51q185 0 316.5 131.5t131.5 316.5v128q0 26 19 45t45 19t45 -19t19 -45v-128 -q0 -221 -147.5 -384.5t-364.5 -187.5v-132h256q26 0 45 -19t19 -45t-19 -45t-45 -19h-640q-26 0 -45 19t-19 45t19 45t45 19h256v132q-125 13 -235 81l-254 -254q-10 -10 -23 -10t-23 10l-82 82q-10 10 -10 23t10 23l1234 1234q10 10 23 10t23 -10l82 -82q10 -10 10 -23 -t-10 -23zM1005 1325l-621 -621v512q0 132 94 226t226 94q102 0 184.5 -59t116.5 -152z" /> - <glyph glyph-name="shield" unicode="" horiz-adv-x="1280" -d="M1088 576v640h-448v-1137q119 63 213 137q235 184 235 360zM1280 1344v-768q0 -86 -33.5 -170.5t-83 -150t-118 -127.5t-126.5 -103t-121 -77.5t-89.5 -49.5t-42.5 -20q-12 -6 -26 -6t-26 6q-16 7 -42.5 20t-89.5 49.5t-121 77.5t-126.5 103t-118 127.5t-83 150 -t-33.5 170.5v768q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" /> - <glyph glyph-name="calendar_empty" unicode="" horiz-adv-x="1664" -d="M128 -128h1408v1024h-1408v-1024zM512 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1280 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1664 1152v-1280 -q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" /> - <glyph glyph-name="fire_extinguisher" unicode="" horiz-adv-x="1408" -d="M512 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 1376v-320q0 -16 -12 -25q-8 -7 -20 -7q-4 0 -7 1l-448 96q-11 2 -18 11t-7 20h-256v-102q111 -23 183.5 -111t72.5 -203v-800q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v800 -q0 106 62.5 190.5t161.5 114.5v111h-32q-59 0 -115 -23.5t-91.5 -53t-66 -66.5t-40.5 -53.5t-14 -24.5q-17 -35 -57 -35q-16 0 -29 7q-23 12 -31.5 37t3.5 49q5 10 14.5 26t37.5 53.5t60.5 70t85 67t108.5 52.5q-25 42 -25 86q0 66 47 113t113 47t113 -47t47 -113 -q0 -33 -14 -64h302q0 11 7 20t18 11l448 96q3 1 7 1q12 0 20 -7q12 -9 12 -25z" /> - <glyph glyph-name="rocket" unicode="" horiz-adv-x="1664" -d="M1440 1088q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM1664 1376q0 -249 -75.5 -430.5t-253.5 -360.5q-81 -80 -195 -176l-20 -379q-2 -16 -16 -26l-384 -224q-7 -4 -16 -4q-12 0 -23 9l-64 64q-13 14 -8 32l85 276l-281 281l-276 -85q-3 -1 -9 -1 -q-14 0 -23 9l-64 64q-17 19 -5 39l224 384q10 14 26 16l379 20q96 114 176 195q188 187 358 258t431 71q14 0 24 -9.5t10 -22.5z" /> - <glyph glyph-name="maxcdn" unicode="" horiz-adv-x="1792" -d="M1745 763l-164 -763h-334l178 832q13 56 -15 88q-27 33 -83 33h-169l-204 -953h-334l204 953h-286l-204 -953h-334l204 953l-153 327h1276q101 0 189.5 -40.5t147.5 -113.5q60 -73 81 -168.5t0 -194.5z" /> - <glyph glyph-name="chevron_sign_left" unicode="" -d="M909 141l102 102q19 19 19 45t-19 45l-307 307l307 307q19 19 19 45t-19 45l-102 102q-19 19 -45 19t-45 -19l-454 -454q-19 -19 -19 -45t19 -45l454 -454q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 -t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="chevron_sign_right" unicode="" -d="M717 141l454 454q19 19 19 45t-19 45l-454 454q-19 19 -45 19t-45 -19l-102 -102q-19 -19 -19 -45t19 -45l307 -307l-307 -307q-19 -19 -19 -45t19 -45l102 -102q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 -t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="chevron_sign_up" unicode="" -d="M1165 397l102 102q19 19 19 45t-19 45l-454 454q-19 19 -45 19t-45 -19l-454 -454q-19 -19 -19 -45t19 -45l102 -102q19 -19 45 -19t45 19l307 307l307 -307q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 -t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="chevron_sign_down" unicode="" -d="M813 237l454 454q19 19 19 45t-19 45l-102 102q-19 19 -45 19t-45 -19l-307 -307l-307 307q-19 19 -45 19t-45 -19l-102 -102q-19 -19 -19 -45t19 -45l454 -454q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 -t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="html5" unicode="" horiz-adv-x="1408" -d="M1130 939l16 175h-884l47 -534h612l-22 -228l-197 -53l-196 53l-13 140h-175l22 -278l362 -100h4v1l359 99l50 544h-644l-15 181h674zM0 1408h1408l-128 -1438l-578 -162l-574 162z" /> - <glyph glyph-name="css3" unicode="" horiz-adv-x="1792" -d="M275 1408h1505l-266 -1333l-804 -267l-698 267l71 356h297l-29 -147l422 -161l486 161l68 339h-1208l58 297h1209l38 191h-1208z" /> - <glyph glyph-name="anchor" unicode="" horiz-adv-x="1792" -d="M960 1280q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1792 352v-352q0 -22 -20 -30q-8 -2 -12 -2q-12 0 -23 9l-93 93q-119 -143 -318.5 -226.5t-429.5 -83.5t-429.5 83.5t-318.5 226.5l-93 -93q-9 -9 -23 -9q-4 0 -12 2q-20 8 -20 30v352 -q0 14 9 23t23 9h352q22 0 30 -20q8 -19 -7 -35l-100 -100q67 -91 189.5 -153.5t271.5 -82.5v647h-192q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h192v163q-58 34 -93 92.5t-35 128.5q0 106 75 181t181 75t181 -75t75 -181q0 -70 -35 -128.5t-93 -92.5v-163h192q26 0 45 -19 -t19 -45v-128q0 -26 -19 -45t-45 -19h-192v-647q149 20 271.5 82.5t189.5 153.5l-100 100q-15 16 -7 35q8 20 30 20h352q14 0 23 -9t9 -23z" /> - <glyph glyph-name="unlock_alt" unicode="" horiz-adv-x="1152" -d="M1056 768q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v320q0 185 131.5 316.5t316.5 131.5t316.5 -131.5t131.5 -316.5q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45q0 106 -75 181t-181 75t-181 -75t-75 -181 -v-320h736z" /> - <glyph glyph-name="bullseye" unicode="" -d="M1024 640q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM1152 640q0 159 -112.5 271.5t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM1280 640q0 -212 -150 -362t-362 -150t-362 150 -t-150 362t150 362t362 150t362 -150t150 -362zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5zM1536 640 -q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="ellipsis_horizontal" unicode="" horiz-adv-x="1408" -d="M384 800v-192q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68zM896 800v-192q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68zM1408 800v-192q0 -40 -28 -68t-68 -28h-192 -q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68z" /> - <glyph glyph-name="ellipsis_vertical" unicode="" horiz-adv-x="384" -d="M384 288v-192q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68zM384 800v-192q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68zM384 1312v-192q0 -40 -28 -68t-68 -28h-192 -q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68z" /> - <glyph glyph-name="_303" unicode="" -d="M512 256q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM863 162q-13 233 -176.5 396.5t-396.5 176.5q-14 1 -24 -9t-10 -23v-128q0 -13 8.5 -22t21.5 -10q154 -11 264 -121t121 -264q1 -13 10 -21.5t22 -8.5h128 -q13 0 23 10t9 24zM1247 161q-5 154 -56 297.5t-139.5 260t-205 205t-260 139.5t-297.5 56q-14 1 -23 -9q-10 -10 -10 -23v-128q0 -13 9 -22t22 -10q204 -7 378 -111.5t278.5 -278.5t111.5 -378q1 -13 10 -22t22 -9h128q13 0 23 10q11 9 9 23zM1536 1120v-960 -q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="play_sign" unicode="" -d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1152 585q32 18 32 55t-32 55l-544 320q-31 19 -64 1q-32 -19 -32 -56v-640q0 -37 32 -56 -q16 -8 32 -8q17 0 32 9z" /> - <glyph glyph-name="ticket" unicode="" horiz-adv-x="1792" -d="M1024 1084l316 -316l-572 -572l-316 316zM813 105l618 618q19 19 19 45t-19 45l-362 362q-18 18 -45 18t-45 -18l-618 -618q-19 -19 -19 -45t19 -45l362 -362q18 -18 45 -18t45 18zM1702 742l-907 -908q-37 -37 -90.5 -37t-90.5 37l-126 126q56 56 56 136t-56 136 -t-136 56t-136 -56l-125 126q-37 37 -37 90.5t37 90.5l907 906q37 37 90.5 37t90.5 -37l125 -125q-56 -56 -56 -136t56 -136t136 -56t136 56l126 -125q37 -37 37 -90.5t-37 -90.5z" /> - <glyph glyph-name="minus_sign_alt" unicode="" -d="M1280 576v128q0 26 -19 45t-45 19h-896q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h896q26 0 45 19t19 45zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5 -t84.5 -203.5z" /> - <glyph glyph-name="check_minus" unicode="" horiz-adv-x="1408" -d="M1152 736v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h832q14 0 23 -9t9 -23zM1280 288v832q0 66 -47 113t-113 47h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113zM1408 1120v-832q0 -119 -84.5 -203.5 -t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="level_up" unicode="" horiz-adv-x="1024" -d="M1018 933q-18 -37 -58 -37h-192v-864q0 -14 -9 -23t-23 -9h-704q-21 0 -29 18q-8 20 4 35l160 192q9 11 25 11h320v640h-192q-40 0 -58 37q-17 37 9 68l320 384q18 22 49 22t49 -22l320 -384q27 -32 9 -68z" /> - <glyph glyph-name="level_down" unicode="" horiz-adv-x="1024" -d="M32 1280h704q13 0 22.5 -9.5t9.5 -23.5v-863h192q40 0 58 -37t-9 -69l-320 -384q-18 -22 -49 -22t-49 22l-320 384q-26 31 -9 69q18 37 58 37h192v640h-320q-14 0 -25 11l-160 192q-13 14 -4 34q9 19 29 19z" /> - <glyph glyph-name="check_sign" unicode="" -d="M685 237l614 614q19 19 19 45t-19 45l-102 102q-19 19 -45 19t-45 -19l-467 -467l-211 211q-19 19 -45 19t-45 -19l-102 -102q-19 -19 -19 -45t19 -45l358 -358q19 -19 45 -19t45 19zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5 -t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="edit_sign" unicode="" -d="M404 428l152 -152l-52 -52h-56v96h-96v56zM818 818q14 -13 -3 -30l-291 -291q-17 -17 -30 -3q-14 13 3 30l291 291q17 17 30 3zM544 128l544 544l-288 288l-544 -544v-288h288zM1152 736l92 92q28 28 28 68t-28 68l-152 152q-28 28 -68 28t-68 -28l-92 -92zM1536 1120 -v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_312" unicode="" -d="M1280 608v480q0 26 -19 45t-45 19h-480q-42 0 -59 -39q-17 -41 14 -70l144 -144l-534 -534q-19 -19 -19 -45t19 -45l102 -102q19 -19 45 -19t45 19l534 534l144 -144q18 -19 45 -19q12 0 25 5q39 17 39 59zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960 -q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="share_sign" unicode="" -d="M1005 435l352 352q19 19 19 45t-19 45l-352 352q-30 31 -69 14q-40 -17 -40 -59v-160q-119 0 -216 -19.5t-162.5 -51t-114 -79t-76.5 -95.5t-44.5 -109t-21.5 -111.5t-5 -110.5q0 -181 167 -404q11 -12 25 -12q7 0 13 3q22 9 19 33q-44 354 62 473q46 52 130 75.5 -t224 23.5v-160q0 -42 40 -59q12 -5 24 -5q26 0 45 19zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="compass" unicode="" -d="M640 448l256 128l-256 128v-256zM1024 1039v-542l-512 -256v542zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 -t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="collapse" unicode="" -d="M1145 861q18 -35 -5 -66l-320 -448q-19 -27 -52 -27t-52 27l-320 448q-23 31 -5 66q17 35 57 35h640q40 0 57 -35zM1280 160v960q0 13 -9.5 22.5t-22.5 9.5h-960q-13 0 -22.5 -9.5t-9.5 -22.5v-960q0 -13 9.5 -22.5t22.5 -9.5h960q13 0 22.5 9.5t9.5 22.5zM1536 1120 -v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="collapse_top" unicode="" -d="M1145 419q-17 -35 -57 -35h-640q-40 0 -57 35q-18 35 5 66l320 448q19 27 52 27t52 -27l320 -448q23 -31 5 -66zM1280 160v960q0 13 -9.5 22.5t-22.5 9.5h-960q-13 0 -22.5 -9.5t-9.5 -22.5v-960q0 -13 9.5 -22.5t22.5 -9.5h960q13 0 22.5 9.5t9.5 22.5zM1536 1120v-960 -q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_317" unicode="" -d="M1088 640q0 -33 -27 -52l-448 -320q-31 -23 -66 -5q-35 17 -35 57v640q0 40 35 57q35 18 66 -5l448 -320q27 -19 27 -52zM1280 160v960q0 14 -9 23t-23 9h-960q-14 0 -23 -9t-9 -23v-960q0 -14 9 -23t23 -9h960q14 0 23 9t9 23zM1536 1120v-960q0 -119 -84.5 -203.5 -t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="eur" unicode="" horiz-adv-x="1024" -d="M976 229l35 -159q3 -12 -3 -22.5t-17 -14.5l-5 -1q-4 -2 -10.5 -3.5t-16 -4.5t-21.5 -5.5t-25.5 -5t-30 -5t-33.5 -4.5t-36.5 -3t-38.5 -1q-234 0 -409 130.5t-238 351.5h-95q-13 0 -22.5 9.5t-9.5 22.5v113q0 13 9.5 22.5t22.5 9.5h66q-2 57 1 105h-67q-14 0 -23 9 -t-9 23v114q0 14 9 23t23 9h98q67 210 243.5 338t400.5 128q102 0 194 -23q11 -3 20 -15q6 -11 3 -24l-43 -159q-3 -13 -14 -19.5t-24 -2.5l-4 1q-4 1 -11.5 2.5l-17.5 3.5t-22.5 3.5t-26 3t-29 2.5t-29.5 1q-126 0 -226 -64t-150 -176h468q16 0 25 -12q10 -12 7 -26 -l-24 -114q-5 -26 -32 -26h-488q-3 -37 0 -105h459q15 0 25 -12q9 -12 6 -27l-24 -112q-2 -11 -11 -18.5t-20 -7.5h-387q48 -117 149.5 -185.5t228.5 -68.5q18 0 36 1.5t33.5 3.5t29.5 4.5t24.5 5t18.5 4.5l12 3l5 2q13 5 26 -2q12 -7 15 -21z" /> - <glyph glyph-name="gbp" unicode="" horiz-adv-x="1024" -d="M1020 399v-367q0 -14 -9 -23t-23 -9h-956q-14 0 -23 9t-9 23v150q0 13 9.5 22.5t22.5 9.5h97v383h-95q-14 0 -23 9.5t-9 22.5v131q0 14 9 23t23 9h95v223q0 171 123.5 282t314.5 111q185 0 335 -125q9 -8 10 -20.5t-7 -22.5l-103 -127q-9 -11 -22 -12q-13 -2 -23 7 -q-5 5 -26 19t-69 32t-93 18q-85 0 -137 -47t-52 -123v-215h305q13 0 22.5 -9t9.5 -23v-131q0 -13 -9.5 -22.5t-22.5 -9.5h-305v-379h414v181q0 13 9 22.5t23 9.5h162q14 0 23 -9.5t9 -22.5z" /> - <glyph glyph-name="usd" unicode="" horiz-adv-x="1024" -d="M978 351q0 -153 -99.5 -263.5t-258.5 -136.5v-175q0 -14 -9 -23t-23 -9h-135q-13 0 -22.5 9.5t-9.5 22.5v175q-66 9 -127.5 31t-101.5 44.5t-74 48t-46.5 37.5t-17.5 18q-17 21 -2 41l103 135q7 10 23 12q15 2 24 -9l2 -2q113 -99 243 -125q37 -8 74 -8q81 0 142.5 43 -t61.5 122q0 28 -15 53t-33.5 42t-58.5 37.5t-66 32t-80 32.5q-39 16 -61.5 25t-61.5 26.5t-62.5 31t-56.5 35.5t-53.5 42.5t-43.5 49t-35.5 58t-21 66.5t-8.5 78q0 138 98 242t255 134v180q0 13 9.5 22.5t22.5 9.5h135q14 0 23 -9t9 -23v-176q57 -6 110.5 -23t87 -33.5 -t63.5 -37.5t39 -29t15 -14q17 -18 5 -38l-81 -146q-8 -15 -23 -16q-14 -3 -27 7q-3 3 -14.5 12t-39 26.5t-58.5 32t-74.5 26t-85.5 11.5q-95 0 -155 -43t-60 -111q0 -26 8.5 -48t29.5 -41.5t39.5 -33t56 -31t60.5 -27t70 -27.5q53 -20 81 -31.5t76 -35t75.5 -42.5t62 -50 -t53 -63.5t31.5 -76.5t13 -94z" /> - <glyph glyph-name="inr" unicode="" horiz-adv-x="898" -d="M898 1066v-102q0 -14 -9 -23t-23 -9h-168q-23 -144 -129 -234t-276 -110q167 -178 459 -536q14 -16 4 -34q-8 -18 -29 -18h-195q-16 0 -25 12q-306 367 -498 571q-9 9 -9 22v127q0 13 9.5 22.5t22.5 9.5h112q132 0 212.5 43t102.5 125h-427q-14 0 -23 9t-9 23v102 -q0 14 9 23t23 9h413q-57 113 -268 113h-145q-13 0 -22.5 9.5t-9.5 22.5v133q0 14 9 23t23 9h832q14 0 23 -9t9 -23v-102q0 -14 -9 -23t-23 -9h-233q47 -61 64 -144h171q14 0 23 -9t9 -23z" /> - <glyph glyph-name="jpy" unicode="" horiz-adv-x="1027" -d="M603 0h-172q-13 0 -22.5 9t-9.5 23v330h-288q-13 0 -22.5 9t-9.5 23v103q0 13 9.5 22.5t22.5 9.5h288v85h-288q-13 0 -22.5 9t-9.5 23v104q0 13 9.5 22.5t22.5 9.5h214l-321 578q-8 16 0 32q10 16 28 16h194q19 0 29 -18l215 -425q19 -38 56 -125q10 24 30.5 68t27.5 61 -l191 420q8 19 29 19h191q17 0 27 -16q9 -14 1 -31l-313 -579h215q13 0 22.5 -9.5t9.5 -22.5v-104q0 -14 -9.5 -23t-22.5 -9h-290v-85h290q13 0 22.5 -9.5t9.5 -22.5v-103q0 -14 -9.5 -23t-22.5 -9h-290v-330q0 -13 -9.5 -22.5t-22.5 -9.5z" /> - <glyph glyph-name="rub" unicode="" horiz-adv-x="1280" -d="M1043 971q0 100 -65 162t-171 62h-320v-448h320q106 0 171 62t65 162zM1280 971q0 -193 -126.5 -315t-326.5 -122h-340v-118h505q14 0 23 -9t9 -23v-128q0 -14 -9 -23t-23 -9h-505v-192q0 -14 -9.5 -23t-22.5 -9h-167q-14 0 -23 9t-9 23v192h-224q-14 0 -23 9t-9 23v128 -q0 14 9 23t23 9h224v118h-224q-14 0 -23 9t-9 23v149q0 13 9 22.5t23 9.5h224v629q0 14 9 23t23 9h539q200 0 326.5 -122t126.5 -315z" /> - <glyph glyph-name="krw" unicode="" horiz-adv-x="1792" -d="M514 341l81 299h-159l75 -300q1 -1 1 -3t1 -3q0 1 0.5 3.5t0.5 3.5zM630 768l35 128h-292l32 -128h225zM822 768h139l-35 128h-70zM1271 340l78 300h-162l81 -299q0 -1 0.5 -3.5t1.5 -3.5q0 1 0.5 3t0.5 3zM1382 768l33 128h-297l34 -128h230zM1792 736v-64q0 -14 -9 -23 -t-23 -9h-213l-164 -616q-7 -24 -31 -24h-159q-24 0 -31 24l-166 616h-209l-167 -616q-7 -24 -31 -24h-159q-11 0 -19.5 7t-10.5 17l-160 616h-208q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h175l-33 128h-142q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h109l-89 344q-5 15 5 28 -q10 12 26 12h137q26 0 31 -24l90 -360h359l97 360q7 24 31 24h126q24 0 31 -24l98 -360h365l93 360q5 24 31 24h137q16 0 26 -12q10 -13 5 -28l-91 -344h111q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-145l-34 -128h179q14 0 23 -9t9 -23z" /> - <glyph glyph-name="btc" unicode="" horiz-adv-x="1280" -d="M1167 896q18 -182 -131 -258q117 -28 175 -103t45 -214q-7 -71 -32.5 -125t-64.5 -89t-97 -58.5t-121.5 -34.5t-145.5 -15v-255h-154v251q-80 0 -122 1v-252h-154v255q-18 0 -54 0.5t-55 0.5h-200l31 183h111q50 0 58 51v402h16q-6 1 -16 1v287q-13 68 -89 68h-111v164 -l212 -1q64 0 97 1v252h154v-247q82 2 122 2v245h154v-252q79 -7 140 -22.5t113 -45t82.5 -78t36.5 -114.5zM952 351q0 36 -15 64t-37 46t-57.5 30.5t-65.5 18.5t-74 9t-69 3t-64.5 -1t-47.5 -1v-338q8 0 37 -0.5t48 -0.5t53 1.5t58.5 4t57 8.5t55.5 14t47.5 21t39.5 30 -t24.5 40t9.5 51zM881 827q0 33 -12.5 58.5t-30.5 42t-48 28t-55 16.5t-61.5 8t-58 2.5t-54 -1t-39.5 -0.5v-307q5 0 34.5 -0.5t46.5 0t50 2t55 5.5t51.5 11t48.5 18.5t37 27t27 38.5t9 51z" /> - <glyph glyph-name="file" unicode="" -d="M1024 1024v472q22 -14 36 -28l408 -408q14 -14 28 -36h-472zM896 992q0 -40 28 -68t68 -28h544v-1056q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h800v-544z" /> - <glyph glyph-name="file_text" unicode="" -d="M1468 1060q14 -14 28 -36h-472v472q22 -14 36 -28zM992 896h544v-1056q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h800v-544q0 -40 28 -68t68 -28zM1152 160v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704 -q14 0 23 9t9 23zM1152 416v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23zM1152 672v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23z" /> - <glyph glyph-name="sort_by_alphabet" unicode="" horiz-adv-x="1664" -d="M1191 1128h177l-72 218l-12 47q-2 16 -2 20h-4l-3 -20q0 -1 -3.5 -18t-7.5 -29zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23zM1572 -23 -v-233h-584v90l369 529q12 18 21 27l11 9v3q-2 0 -6.5 -0.5t-7.5 -0.5q-12 -3 -30 -3h-232v-115h-120v229h567v-89l-369 -530q-6 -8 -21 -26l-11 -11v-2l14 2q9 2 30 2h248v119h121zM1661 874v-106h-288v106h75l-47 144h-243l-47 -144h75v-106h-287v106h70l230 662h162 -l230 -662h70z" /> - <glyph glyph-name="_329" unicode="" horiz-adv-x="1664" -d="M1191 104h177l-72 218l-12 47q-2 16 -2 20h-4l-3 -20q0 -1 -3.5 -18t-7.5 -29zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23zM1661 -150 -v-106h-288v106h75l-47 144h-243l-47 -144h75v-106h-287v106h70l230 662h162l230 -662h70zM1572 1001v-233h-584v90l369 529q12 18 21 27l11 9v3q-2 0 -6.5 -0.5t-7.5 -0.5q-12 -3 -30 -3h-232v-115h-120v229h567v-89l-369 -530q-6 -8 -21 -26l-11 -10v-3l14 3q9 1 30 1h248 -v119h121z" /> - <glyph glyph-name="sort_by_attributes" unicode="" horiz-adv-x="1792" -d="M736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23zM1792 -32v-192q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h832 -q14 0 23 -9t9 -23zM1600 480v-192q0 -14 -9 -23t-23 -9h-640q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h640q14 0 23 -9t9 -23zM1408 992v-192q0 -14 -9 -23t-23 -9h-448q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h448q14 0 23 -9t9 -23zM1216 1504v-192q0 -14 -9 -23t-23 -9h-256 -q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h256q14 0 23 -9t9 -23z" /> - <glyph glyph-name="sort_by_attributes_alt" unicode="" horiz-adv-x="1792" -d="M1216 -32v-192q0 -14 -9 -23t-23 -9h-256q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h256q14 0 23 -9t9 -23zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192 -q14 0 23 -9t9 -23zM1408 480v-192q0 -14 -9 -23t-23 -9h-448q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h448q14 0 23 -9t9 -23zM1600 992v-192q0 -14 -9 -23t-23 -9h-640q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h640q14 0 23 -9t9 -23zM1792 1504v-192q0 -14 -9 -23t-23 -9h-832 -q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h832q14 0 23 -9t9 -23z" /> - <glyph glyph-name="sort_by_order" unicode="" -d="M1346 223q0 63 -44 116t-103 53q-52 0 -83 -37t-31 -94t36.5 -95t104.5 -38q50 0 85 27t35 68zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23 -zM1486 165q0 -62 -13 -121.5t-41 -114t-68 -95.5t-98.5 -65.5t-127.5 -24.5q-62 0 -108 16q-24 8 -42 15l39 113q15 -7 31 -11q37 -13 75 -13q84 0 134.5 58.5t66.5 145.5h-2q-21 -23 -61.5 -37t-84.5 -14q-106 0 -173 71.5t-67 172.5q0 105 72 178t181 73q123 0 205 -94.5 -t82 -252.5zM1456 882v-114h-469v114h167v432q0 7 0.5 19t0.5 17v16h-2l-7 -12q-8 -13 -26 -31l-62 -58l-82 86l192 185h123v-654h165z" /> - <glyph glyph-name="sort_by_order_alt" unicode="" -d="M1346 1247q0 63 -44 116t-103 53q-52 0 -83 -37t-31 -94t36.5 -95t104.5 -38q50 0 85 27t35 68zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9 -t9 -23zM1456 -142v-114h-469v114h167v432q0 7 0.5 19t0.5 17v16h-2l-7 -12q-8 -13 -26 -31l-62 -58l-82 86l192 185h123v-654h165zM1486 1189q0 -62 -13 -121.5t-41 -114t-68 -95.5t-98.5 -65.5t-127.5 -24.5q-62 0 -108 16q-24 8 -42 15l39 113q15 -7 31 -11q37 -13 75 -13 -q84 0 134.5 58.5t66.5 145.5h-2q-21 -23 -61.5 -37t-84.5 -14q-106 0 -173 71.5t-67 172.5q0 105 72 178t181 73q123 0 205 -94.5t82 -252.5z" /> - <glyph glyph-name="_334" unicode="" horiz-adv-x="1664" -d="M256 192q0 26 -19 45t-45 19q-27 0 -45.5 -19t-18.5 -45q0 -27 18.5 -45.5t45.5 -18.5q26 0 45 18.5t19 45.5zM416 704v-640q0 -26 -19 -45t-45 -19h-288q-26 0 -45 19t-19 45v640q0 26 19 45t45 19h288q26 0 45 -19t19 -45zM1600 704q0 -86 -55 -149q15 -44 15 -76 -q3 -76 -43 -137q17 -56 0 -117q-15 -57 -54 -94q9 -112 -49 -181q-64 -76 -197 -78h-36h-76h-17q-66 0 -144 15.5t-121.5 29t-120.5 39.5q-123 43 -158 44q-26 1 -45 19.5t-19 44.5v641q0 25 18 43.5t43 20.5q24 2 76 59t101 121q68 87 101 120q18 18 31 48t17.5 48.5 -t13.5 60.5q7 39 12.5 61t19.5 52t34 50q19 19 45 19q46 0 82.5 -10.5t60 -26t40 -40.5t24 -45t12 -50t5 -45t0.5 -39q0 -38 -9.5 -76t-19 -60t-27.5 -56q-3 -6 -10 -18t-11 -22t-8 -24h277q78 0 135 -57t57 -135z" /> - <glyph glyph-name="_335" unicode="" horiz-adv-x="1664" -d="M256 960q0 -26 -19 -45t-45 -19q-27 0 -45.5 19t-18.5 45q0 27 18.5 45.5t45.5 18.5q26 0 45 -18.5t19 -45.5zM416 448v640q0 26 -19 45t-45 19h-288q-26 0 -45 -19t-19 -45v-640q0 -26 19 -45t45 -19h288q26 0 45 19t19 45zM1545 597q55 -61 55 -149q-1 -78 -57.5 -135 -t-134.5 -57h-277q4 -14 8 -24t11 -22t10 -18q18 -37 27 -57t19 -58.5t10 -76.5q0 -24 -0.5 -39t-5 -45t-12 -50t-24 -45t-40 -40.5t-60 -26t-82.5 -10.5q-26 0 -45 19q-20 20 -34 50t-19.5 52t-12.5 61q-9 42 -13.5 60.5t-17.5 48.5t-31 48q-33 33 -101 120q-49 64 -101 121 -t-76 59q-25 2 -43 20.5t-18 43.5v641q0 26 19 44.5t45 19.5q35 1 158 44q77 26 120.5 39.5t121.5 29t144 15.5h17h76h36q133 -2 197 -78q58 -69 49 -181q39 -37 54 -94q17 -61 0 -117q46 -61 43 -137q0 -32 -15 -76z" /> - <glyph glyph-name="youtube_sign" unicode="" -d="M919 233v157q0 50 -29 50q-17 0 -33 -16v-224q16 -16 33 -16q29 0 29 49zM1103 355h66v34q0 51 -33 51t-33 -51v-34zM532 621v-70h-80v-423h-74v423h-78v70h232zM733 495v-367h-67v40q-39 -45 -76 -45q-33 0 -42 28q-6 17 -6 54v290h66v-270q0 -24 1 -26q1 -15 15 -15 -q20 0 42 31v280h67zM985 384v-146q0 -52 -7 -73q-12 -42 -53 -42q-35 0 -68 41v-36h-67v493h67v-161q32 40 68 40q41 0 53 -42q7 -21 7 -74zM1236 255v-9q0 -29 -2 -43q-3 -22 -15 -40q-27 -40 -80 -40q-52 0 -81 38q-21 27 -21 86v129q0 59 20 86q29 38 80 38t78 -38 -q21 -29 21 -86v-76h-133v-65q0 -51 34 -51q24 0 30 26q0 1 0.5 7t0.5 16.5v21.5h68zM785 1079v-156q0 -51 -32 -51t-32 51v156q0 52 32 52t32 -52zM1318 366q0 177 -19 260q-10 44 -43 73.5t-76 34.5q-136 15 -412 15q-275 0 -411 -15q-44 -5 -76.5 -34.5t-42.5 -73.5 -q-20 -87 -20 -260q0 -176 20 -260q10 -43 42.5 -73t75.5 -35q137 -15 412 -15t412 15q43 5 75.5 35t42.5 73q20 84 20 260zM563 1017l90 296h-75l-51 -195l-53 195h-78q7 -23 23 -69l24 -69q35 -103 46 -158v-201h74v201zM852 936v130q0 58 -21 87q-29 38 -78 38 -q-51 0 -78 -38q-21 -29 -21 -87v-130q0 -58 21 -87q27 -38 78 -38q49 0 78 38q21 27 21 87zM1033 816h67v370h-67v-283q-22 -31 -42 -31q-15 0 -16 16q-1 2 -1 26v272h-67v-293q0 -37 6 -55q11 -27 43 -27q36 0 77 45v-40zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5 -h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="youtube" unicode="" -d="M971 292v-211q0 -67 -39 -67q-23 0 -45 22v301q22 22 45 22q39 0 39 -67zM1309 291v-46h-90v46q0 68 45 68t45 -68zM343 509h107v94h-312v-94h105v-569h100v569zM631 -60h89v494h-89v-378q-30 -42 -57 -42q-18 0 -21 21q-1 3 -1 35v364h-89v-391q0 -49 8 -73 -q12 -37 58 -37q48 0 102 61v-54zM1060 88v197q0 73 -9 99q-17 56 -71 56q-50 0 -93 -54v217h-89v-663h89v48q45 -55 93 -55q54 0 71 55q9 27 9 100zM1398 98v13h-91q0 -51 -2 -61q-7 -36 -40 -36q-46 0 -46 69v87h179v103q0 79 -27 116q-39 51 -106 51q-68 0 -107 -51 -q-28 -37 -28 -116v-173q0 -79 29 -116q39 -51 108 -51q72 0 108 53q18 27 21 54q2 9 2 58zM790 1011v210q0 69 -43 69t-43 -69v-210q0 -70 43 -70t43 70zM1509 260q0 -234 -26 -350q-14 -59 -58 -99t-102 -46q-184 -21 -555 -21t-555 21q-58 6 -102.5 46t-57.5 99 -q-26 112 -26 350q0 234 26 350q14 59 58 99t103 47q183 20 554 20t555 -20q58 -7 102.5 -47t57.5 -99q26 -112 26 -350zM511 1536h102l-121 -399v-271h-100v271q-14 74 -61 212q-37 103 -65 187h106l71 -263zM881 1203v-175q0 -81 -28 -118q-38 -51 -106 -51q-67 0 -105 51 -q-28 38 -28 118v175q0 80 28 117q38 51 105 51q68 0 106 -51q28 -37 28 -117zM1216 1365v-499h-91v55q-53 -62 -103 -62q-46 0 -59 37q-8 24 -8 75v394h91v-367q0 -33 1 -35q3 -22 21 -22q27 0 57 43v381h91z" /> - <glyph glyph-name="xing" unicode="" horiz-adv-x="1408" -d="M597 869q-10 -18 -257 -456q-27 -46 -65 -46h-239q-21 0 -31 17t0 36l253 448q1 0 0 1l-161 279q-12 22 -1 37q9 15 32 15h239q40 0 66 -45zM1403 1511q11 -16 0 -37l-528 -934v-1l336 -615q11 -20 1 -37q-10 -15 -32 -15h-239q-42 0 -66 45l-339 622q18 32 531 942 -q25 45 64 45h241q22 0 31 -15z" /> - <glyph glyph-name="xing_sign" unicode="" -d="M685 771q0 1 -126 222q-21 34 -52 34h-184q-18 0 -26 -11q-7 -12 1 -29l125 -216v-1l-196 -346q-9 -14 0 -28q8 -13 24 -13h185q31 0 50 36zM1309 1268q-7 12 -24 12h-187q-30 0 -49 -35l-411 -729q1 -2 262 -481q20 -35 52 -35h184q18 0 25 12q8 13 -1 28l-260 476v1 -l409 723q8 16 0 28zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="youtube_play" unicode="" horiz-adv-x="1792" -d="M711 408l484 250l-484 253v-503zM896 1270q168 0 324.5 -4.5t229.5 -9.5l73 -4q1 0 17 -1.5t23 -3t23.5 -4.5t28.5 -8t28 -13t31 -19.5t29 -26.5q6 -6 15.5 -18.5t29 -58.5t26.5 -101q8 -64 12.5 -136.5t5.5 -113.5v-40v-136q1 -145 -18 -290q-7 -55 -25 -99.5t-32 -61.5 -l-14 -17q-14 -15 -29 -26.5t-31 -19t-28 -12.5t-28.5 -8t-24 -4.5t-23 -3t-16.5 -1.5q-251 -19 -627 -19q-207 2 -359.5 6.5t-200.5 7.5l-49 4l-36 4q-36 5 -54.5 10t-51 21t-56.5 41q-6 6 -15.5 18.5t-29 58.5t-26.5 101q-8 64 -12.5 136.5t-5.5 113.5v40v136 -q-1 145 18 290q7 55 25 99.5t32 61.5l14 17q14 15 29 26.5t31 19.5t28 13t28.5 8t23.5 4.5t23 3t17 1.5q251 18 627 18z" /> - <glyph glyph-name="dropbox" unicode="" horiz-adv-x="1792" -d="M402 829l494 -305l-342 -285l-490 319zM1388 274v-108l-490 -293v-1l-1 1l-1 -1v1l-489 293v108l147 -96l342 284v2l1 -1l1 1v-2l343 -284zM554 1418l342 -285l-494 -304l-338 270zM1390 829l338 -271l-489 -319l-343 285zM1239 1418l489 -319l-338 -270l-494 304z" /> - <glyph glyph-name="stackexchange" unicode="" -d="M1289 -96h-1118v480h-160v-640h1438v640h-160v-480zM347 428l33 157l783 -165l-33 -156zM450 802l67 146l725 -339l-67 -145zM651 1158l102 123l614 -513l-102 -123zM1048 1536l477 -641l-128 -96l-477 641zM330 65v159h800v-159h-800z" /> - <glyph glyph-name="instagram" unicode="" -d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1162 640q0 -164 -115 -279t-279 -115t-279 115t-115 279t115 279t279 115t279 -115t115 -279zM1270 1050q0 -38 -27 -65t-65 -27t-65 27t-27 65t27 65t65 27t65 -27t27 -65zM768 1270 -q-7 0 -76.5 0.5t-105.5 0t-96.5 -3t-103 -10t-71.5 -18.5q-50 -20 -88 -58t-58 -88q-11 -29 -18.5 -71.5t-10 -103t-3 -96.5t0 -105.5t0.5 -76.5t-0.5 -76.5t0 -105.5t3 -96.5t10 -103t18.5 -71.5q20 -50 58 -88t88 -58q29 -11 71.5 -18.5t103 -10t96.5 -3t105.5 0t76.5 0.5 -t76.5 -0.5t105.5 0t96.5 3t103 10t71.5 18.5q50 20 88 58t58 88q11 29 18.5 71.5t10 103t3 96.5t0 105.5t-0.5 76.5t0.5 76.5t0 105.5t-3 96.5t-10 103t-18.5 71.5q-20 50 -58 88t-88 58q-29 11 -71.5 18.5t-103 10t-96.5 3t-105.5 0t-76.5 -0.5zM1536 640q0 -229 -5 -317 -q-10 -208 -124 -322t-322 -124q-88 -5 -317 -5t-317 5q-208 10 -322 124t-124 322q-5 88 -5 317t5 317q10 208 124 322t322 124q88 5 317 5t317 -5q208 -10 322 -124t124 -322q5 -88 5 -317z" /> - <glyph glyph-name="flickr" unicode="" -d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960zM698 640q0 88 -62 150t-150 62t-150 -62t-62 -150t62 -150t150 -62t150 62t62 150zM1262 640q0 88 -62 150 -t-150 62t-150 -62t-62 -150t62 -150t150 -62t150 62t62 150z" /> - <glyph glyph-name="adn" unicode="" -d="M768 914l201 -306h-402zM1133 384h94l-459 691l-459 -691h94l104 160h522zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="f171" unicode="" horiz-adv-x="1408" -d="M815 677q8 -63 -50.5 -101t-111.5 -6q-39 17 -53.5 58t-0.5 82t52 58q36 18 72.5 12t64 -35.5t27.5 -67.5zM926 698q-14 107 -113 164t-197 13q-63 -28 -100.5 -88.5t-34.5 -129.5q4 -91 77.5 -155t165.5 -56q91 8 152 84t50 168zM1165 1240q-20 27 -56 44.5t-58 22 -t-71 12.5q-291 47 -566 -2q-43 -7 -66 -12t-55 -22t-50 -43q30 -28 76 -45.5t73.5 -22t87.5 -11.5q228 -29 448 -1q63 8 89.5 12t72.5 21.5t75 46.5zM1222 205q-8 -26 -15.5 -76.5t-14 -84t-28.5 -70t-58 -56.5q-86 -48 -189.5 -71.5t-202 -22t-201.5 18.5q-46 8 -81.5 18 -t-76.5 27t-73 43.5t-52 61.5q-25 96 -57 292l6 16l18 9q223 -148 506.5 -148t507.5 148q21 -6 24 -23t-5 -45t-8 -37zM1403 1166q-26 -167 -111 -655q-5 -30 -27 -56t-43.5 -40t-54.5 -31q-252 -126 -610 -88q-248 27 -394 139q-15 12 -25.5 26.5t-17 35t-9 34t-6 39.5 -t-5.5 35q-9 50 -26.5 150t-28 161.5t-23.5 147.5t-22 158q3 26 17.5 48.5t31.5 37.5t45 30t46 22.5t48 18.5q125 46 313 64q379 37 676 -50q155 -46 215 -122q16 -20 16.5 -51t-5.5 -54z" /> - <glyph glyph-name="bitbucket_sign" unicode="" -d="M848 666q0 43 -41 66t-77 1q-43 -20 -42.5 -72.5t43.5 -70.5q39 -23 81 4t36 72zM928 682q8 -66 -36 -121t-110 -61t-119 40t-56 113q-2 49 25.5 93t72.5 64q70 31 141.5 -10t81.5 -118zM1100 1073q-20 -21 -53.5 -34t-53 -16t-63.5 -8q-155 -20 -324 0q-44 6 -63 9.5 -t-52.5 16t-54.5 32.5q13 19 36 31t40 15.5t47 8.5q198 35 408 1q33 -5 51 -8.5t43 -16t39 -31.5zM1142 327q0 7 5.5 26.5t3 32t-17.5 16.5q-161 -106 -365 -106t-366 106l-12 -6l-5 -12q26 -154 41 -210q47 -81 204 -108q249 -46 428 53q34 19 49 51.5t22.5 85.5t12.5 71z -M1272 1020q9 53 -8 75q-43 55 -155 88q-216 63 -487 36q-132 -12 -226 -46q-38 -15 -59.5 -25t-47 -34t-29.5 -54q8 -68 19 -138t29 -171t24 -137q1 -5 5 -31t7 -36t12 -27t22 -28q105 -80 284 -100q259 -28 440 63q24 13 39.5 23t31 29t19.5 40q48 267 80 473zM1536 1120 -v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="tumblr" unicode="" horiz-adv-x="1024" -d="M944 207l80 -237q-23 -35 -111 -66t-177 -32q-104 -2 -190.5 26t-142.5 74t-95 106t-55.5 120t-16.5 118v544h-168v215q72 26 129 69.5t91 90t58 102t34 99t15 88.5q1 5 4.5 8.5t7.5 3.5h244v-424h333v-252h-334v-518q0 -30 6.5 -56t22.5 -52.5t49.5 -41.5t81.5 -14 -q78 2 134 29z" /> - <glyph glyph-name="tumblr_sign" unicode="" -d="M1136 75l-62 183q-44 -22 -103 -22q-36 -1 -62 10.5t-38.5 31.5t-17.5 40.5t-5 43.5v398h257v194h-256v326h-188q-8 0 -9 -10q-5 -44 -17.5 -87t-39 -95t-77 -95t-118.5 -68v-165h130v-418q0 -57 21.5 -115t65 -111t121 -85.5t176.5 -30.5q69 1 136.5 25t85.5 50z -M1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="long_arrow_down" unicode="" horiz-adv-x="768" -d="M765 237q8 -19 -5 -35l-350 -384q-10 -10 -23 -10q-14 0 -24 10l-355 384q-13 16 -5 35q9 19 29 19h224v1248q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1248h224q21 0 29 -19z" /> - <glyph glyph-name="long_arrow_up" unicode="" horiz-adv-x="768" -d="M765 1043q-9 -19 -29 -19h-224v-1248q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v1248h-224q-21 0 -29 19t5 35l350 384q10 10 23 10q14 0 24 -10l355 -384q13 -16 5 -35z" /> - <glyph glyph-name="long_arrow_left" unicode="" horiz-adv-x="1792" -d="M1792 736v-192q0 -14 -9 -23t-23 -9h-1248v-224q0 -21 -19 -29t-35 5l-384 350q-10 10 -10 23q0 14 10 24l384 354q16 14 35 6q19 -9 19 -29v-224h1248q14 0 23 -9t9 -23z" /> - <glyph glyph-name="long_arrow_right" unicode="" horiz-adv-x="1792" -d="M1728 643q0 -14 -10 -24l-384 -354q-16 -14 -35 -6q-19 9 -19 29v224h-1248q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h1248v224q0 21 19 29t35 -5l384 -350q10 -10 10 -23z" /> - <glyph glyph-name="apple" unicode="" horiz-adv-x="1408" -d="M1393 321q-39 -125 -123 -250q-129 -196 -257 -196q-49 0 -140 32q-86 32 -151 32q-61 0 -142 -33q-81 -34 -132 -34q-152 0 -301 259q-147 261 -147 503q0 228 113 374q113 144 284 144q72 0 177 -30q104 -30 138 -30q45 0 143 34q102 34 173 34q119 0 213 -65 -q52 -36 104 -100q-79 -67 -114 -118q-65 -94 -65 -207q0 -124 69 -223t158 -126zM1017 1494q0 -61 -29 -136q-30 -75 -93 -138q-54 -54 -108 -72q-37 -11 -104 -17q3 149 78 257q74 107 250 148q1 -3 2.5 -11t2.5 -11q0 -4 0.5 -10t0.5 -10z" /> - <glyph glyph-name="windows" unicode="" horiz-adv-x="1664" -d="M682 530v-651l-682 94v557h682zM682 1273v-659h-682v565zM1664 530v-786l-907 125v661h907zM1664 1408v-794h-907v669z" /> - <glyph glyph-name="android" unicode="" horiz-adv-x="1408" -d="M493 1053q16 0 27.5 11.5t11.5 27.5t-11.5 27.5t-27.5 11.5t-27 -11.5t-11 -27.5t11 -27.5t27 -11.5zM915 1053q16 0 27 11.5t11 27.5t-11 27.5t-27 11.5t-27.5 -11.5t-11.5 -27.5t11.5 -27.5t27.5 -11.5zM103 869q42 0 72 -30t30 -72v-430q0 -43 -29.5 -73t-72.5 -30 -t-73 30t-30 73v430q0 42 30 72t73 30zM1163 850v-666q0 -46 -32 -78t-77 -32h-75v-227q0 -43 -30 -73t-73 -30t-73 30t-30 73v227h-138v-227q0 -43 -30 -73t-73 -30q-42 0 -72 30t-30 73l-1 227h-74q-46 0 -78 32t-32 78v666h918zM931 1255q107 -55 171 -153.5t64 -215.5 -h-925q0 117 64 215.5t172 153.5l-71 131q-7 13 5 20q13 6 20 -6l72 -132q95 42 201 42t201 -42l72 132q7 12 20 6q12 -7 5 -20zM1408 767v-430q0 -43 -30 -73t-73 -30q-42 0 -72 30t-30 73v430q0 43 30 72.5t72 29.5q43 0 73 -29.5t30 -72.5z" /> - <glyph glyph-name="linux" unicode="" -d="M663 1125q-11 -1 -15.5 -10.5t-8.5 -9.5q-5 -1 -5 5q0 12 19 15h10zM750 1111q-4 -1 -11.5 6.5t-17.5 4.5q24 11 32 -2q3 -6 -3 -9zM399 684q-4 1 -6 -3t-4.5 -12.5t-5.5 -13.5t-10 -13q-10 -11 -1 -12q4 -1 12.5 7t12.5 18q1 3 2 7t2 6t1.5 4.5t0.5 4v3t-1 2.5t-3 2z -M1254 325q0 18 -55 42q4 15 7.5 27.5t5 26t3 21.5t0.5 22.5t-1 19.5t-3.5 22t-4 20.5t-5 25t-5.5 26.5q-10 48 -47 103t-72 75q24 -20 57 -83q87 -162 54 -278q-11 -40 -50 -42q-31 -4 -38.5 18.5t-8 83.5t-11.5 107q-9 39 -19.5 69t-19.5 45.5t-15.5 24.5t-13 15t-7.5 7 -q-14 62 -31 103t-29.5 56t-23.5 33t-15 40q-4 21 6 53.5t4.5 49.5t-44.5 25q-15 3 -44.5 18t-35.5 16q-8 1 -11 26t8 51t36 27q37 3 51 -30t4 -58q-11 -19 -2 -26.5t30 -0.5q13 4 13 36v37q-5 30 -13.5 50t-21 30.5t-23.5 15t-27 7.5q-107 -8 -89 -134q0 -15 -1 -15 -q-9 9 -29.5 10.5t-33 -0.5t-15.5 5q1 57 -16 90t-45 34q-27 1 -41.5 -27.5t-16.5 -59.5q-1 -15 3.5 -37t13 -37.5t15.5 -13.5q10 3 16 14q4 9 -7 8q-7 0 -15.5 14.5t-9.5 33.5q-1 22 9 37t34 14q17 0 27 -21t9.5 -39t-1.5 -22q-22 -15 -31 -29q-8 -12 -27.5 -23.5 -t-20.5 -12.5q-13 -14 -15.5 -27t7.5 -18q14 -8 25 -19.5t16 -19t18.5 -13t35.5 -6.5q47 -2 102 15q2 1 23 7t34.5 10.5t29.5 13t21 17.5q9 14 20 8q5 -3 6.5 -8.5t-3 -12t-16.5 -9.5q-20 -6 -56.5 -21.5t-45.5 -19.5q-44 -19 -70 -23q-25 -5 -79 2q-10 2 -9 -2t17 -19 -q25 -23 67 -22q17 1 36 7t36 14t33.5 17.5t30 17t24.5 12t17.5 2.5t8.5 -11q0 -2 -1 -4.5t-4 -5t-6 -4.5t-8.5 -5t-9 -4.5t-10 -5t-9.5 -4.5q-28 -14 -67.5 -44t-66.5 -43t-49 -1q-21 11 -63 73q-22 31 -25 22q-1 -3 -1 -10q0 -25 -15 -56.5t-29.5 -55.5t-21 -58t11.5 -63 -q-23 -6 -62.5 -90t-47.5 -141q-2 -18 -1.5 -69t-5.5 -59q-8 -24 -29 -3q-32 31 -36 94q-2 28 4 56q4 19 -1 18q-2 -1 -4 -5q-36 -65 10 -166q5 -12 25 -28t24 -20q20 -23 104 -90.5t93 -76.5q16 -15 17.5 -38t-14 -43t-45.5 -23q8 -15 29 -44.5t28 -54t7 -70.5q46 24 7 92 -q-4 8 -10.5 16t-9.5 12t-2 6q3 5 13 9.5t20 -2.5q46 -52 166 -36q133 15 177 87q23 38 34 30q12 -6 10 -52q-1 -25 -23 -92q-9 -23 -6 -37.5t24 -15.5q3 19 14.5 77t13.5 90q2 21 -6.5 73.5t-7.5 97t23 70.5q15 18 51 18q1 37 34.5 53t72.5 10.5t60 -22.5zM626 1152 -q3 17 -2.5 30t-11.5 15q-9 2 -9 -7q2 -5 5 -6q10 0 7 -15q-3 -20 8 -20q3 0 3 3zM1045 955q-2 8 -6.5 11.5t-13 5t-14.5 5.5q-5 3 -9.5 8t-7 8t-5.5 6.5t-4 4t-4 -1.5q-14 -16 7 -43.5t39 -31.5q9 -1 14.5 8t3.5 20zM867 1168q0 11 -5 19.5t-11 12.5t-9 3q-6 0 -8 -2t0 -4 -t5 -3q14 -4 18 -31q0 -3 8 2q2 2 2 3zM921 1401q0 2 -2.5 5t-9 7t-9.5 6q-15 15 -24 15q-9 -1 -11.5 -7.5t-1 -13t-0.5 -12.5q-1 -4 -6 -10.5t-6 -9t3 -8.5q4 -3 8 0t11 9t15 9q1 1 9 1t15 2t9 7zM1486 60q20 -12 31 -24.5t12 -24t-2.5 -22.5t-15.5 -22t-23.5 -19.5 -t-30 -18.5t-31.5 -16.5t-32 -15.5t-27 -13q-38 -19 -85.5 -56t-75.5 -64q-17 -16 -68 -19.5t-89 14.5q-18 9 -29.5 23.5t-16.5 25.5t-22 19.5t-47 9.5q-44 1 -130 1q-19 0 -57 -1.5t-58 -2.5q-44 -1 -79.5 -15t-53.5 -30t-43.5 -28.5t-53.5 -11.5q-29 1 -111 31t-146 43 -q-19 4 -51 9.5t-50 9t-39.5 9.5t-33.5 14.5t-17 19.5q-10 23 7 66.5t18 54.5q1 16 -4 40t-10 42.5t-4.5 36.5t10.5 27q14 12 57 14t60 12q30 18 42 35t12 51q21 -73 -32 -106q-32 -20 -83 -15q-34 3 -43 -10q-13 -15 5 -57q2 -6 8 -18t8.5 -18t4.5 -17t1 -22q0 -15 -17 -49 -t-14 -48q3 -17 37 -26q20 -6 84.5 -18.5t99.5 -20.5q24 -6 74 -22t82.5 -23t55.5 -4q43 6 64.5 28t23 48t-7.5 58.5t-19 52t-20 36.5q-121 190 -169 242q-68 74 -113 40q-11 -9 -15 15q-3 16 -2 38q1 29 10 52t24 47t22 42q8 21 26.5 72t29.5 78t30 61t39 54 -q110 143 124 195q-12 112 -16 310q-2 90 24 151.5t106 104.5q39 21 104 21q53 1 106 -13.5t89 -41.5q57 -42 91.5 -121.5t29.5 -147.5q-5 -95 30 -214q34 -113 133 -218q55 -59 99.5 -163t59.5 -191q8 -49 5 -84.5t-12 -55.5t-20 -22q-10 -2 -23.5 -19t-27 -35.5 -t-40.5 -33.5t-61 -14q-18 1 -31.5 5t-22.5 13.5t-13.5 15.5t-11.5 20.5t-9 19.5q-22 37 -41 30t-28 -49t7 -97q20 -70 1 -195q-10 -65 18 -100.5t73 -33t85 35.5q59 49 89.5 66.5t103.5 42.5q53 18 77 36.5t18.5 34.5t-25 28.5t-51.5 23.5q-33 11 -49.5 48t-15 72.5 -t15.5 47.5q1 -31 8 -56.5t14.5 -40.5t20.5 -28.5t21 -19t21.5 -13t16.5 -9.5z" /> - <glyph glyph-name="dribble" unicode="" -d="M1024 36q-42 241 -140 498h-2l-2 -1q-16 -6 -43 -16.5t-101 -49t-137 -82t-131 -114.5t-103 -148l-15 11q184 -150 418 -150q132 0 256 52zM839 643q-21 49 -53 111q-311 -93 -673 -93q-1 -7 -1 -21q0 -124 44 -236.5t124 -201.5q50 89 123.5 166.5t142.5 124.5t130.5 81 -t99.5 48l37 13q4 1 13 3.5t13 4.5zM732 855q-120 213 -244 378q-138 -65 -234 -186t-128 -272q302 0 606 80zM1416 536q-210 60 -409 29q87 -239 128 -469q111 75 185 189.5t96 250.5zM611 1277q-1 0 -2 -1q1 1 2 1zM1201 1132q-185 164 -433 164q-76 0 -155 -19 -q131 -170 246 -382q69 26 130 60.5t96.5 61.5t65.5 57t37.5 40.5zM1424 647q-3 232 -149 410l-1 -1q-9 -12 -19 -24.5t-43.5 -44.5t-71 -60.5t-100 -65t-131.5 -64.5q25 -53 44 -95q2 -5 6.5 -17t7.5 -17q36 5 74.5 7t73.5 2t69 -1.5t64 -4t56.5 -5.5t48 -6.5t36.5 -6 -t25 -4.5zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="skype" unicode="" -d="M1173 473q0 50 -19.5 91.5t-48.5 68.5t-73 49t-82.5 34t-87.5 23l-104 24q-30 7 -44 10.5t-35 11.5t-30 16t-16.5 21t-7.5 30q0 77 144 77q43 0 77 -12t54 -28.5t38 -33.5t40 -29t48 -12q47 0 75.5 32t28.5 77q0 55 -56 99.5t-142 67.5t-182 23q-68 0 -132 -15.5 -t-119.5 -47t-89 -87t-33.5 -128.5q0 -61 19 -106.5t56 -75.5t80 -48.5t103 -32.5l146 -36q90 -22 112 -36q32 -20 32 -60q0 -39 -40 -64.5t-105 -25.5q-51 0 -91.5 16t-65 38.5t-45.5 45t-46 38.5t-54 16q-50 0 -75.5 -30t-25.5 -75q0 -92 122 -157.5t291 -65.5 -q73 0 140 18.5t122.5 53.5t88.5 93.5t33 131.5zM1536 256q0 -159 -112.5 -271.5t-271.5 -112.5q-130 0 -234 80q-77 -16 -150 -16q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5q0 73 16 150q-80 104 -80 234q0 159 112.5 271.5t271.5 112.5q130 0 234 -80 -q77 16 150 16q143 0 273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -73 -16 -150q80 -104 80 -234z" /> - <glyph glyph-name="foursquare" unicode="" horiz-adv-x="1280" -d="M1000 1102l37 194q5 23 -9 40t-35 17h-712q-23 0 -38.5 -17t-15.5 -37v-1101q0 -7 6 -1l291 352q23 26 38 33.5t48 7.5h239q22 0 37 14.5t18 29.5q24 130 37 191q4 21 -11.5 40t-36.5 19h-294q-29 0 -48 19t-19 48v42q0 29 19 47.5t48 18.5h346q18 0 35 13.5t20 29.5z -M1227 1324q-15 -73 -53.5 -266.5t-69.5 -350t-35 -173.5q-6 -22 -9 -32.5t-14 -32.5t-24.5 -33t-38.5 -21t-58 -10h-271q-13 0 -22 -10q-8 -9 -426 -494q-22 -25 -58.5 -28.5t-48.5 5.5q-55 22 -55 98v1410q0 55 38 102.5t120 47.5h888q95 0 127 -53t10 -159zM1227 1324 -l-158 -790q4 17 35 173.5t69.5 350t53.5 266.5z" /> - <glyph glyph-name="trello" unicode="" -d="M704 192v1024q0 14 -9 23t-23 9h-480q-14 0 -23 -9t-9 -23v-1024q0 -14 9 -23t23 -9h480q14 0 23 9t9 23zM1376 576v640q0 14 -9 23t-23 9h-480q-14 0 -23 -9t-9 -23v-640q0 -14 9 -23t23 -9h480q14 0 23 9t9 23zM1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408 -q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" /> - <glyph glyph-name="female" unicode="" horiz-adv-x="1280" -d="M1280 480q0 -40 -28 -68t-68 -28q-51 0 -80 43l-227 341h-45v-132l247 -411q9 -15 9 -33q0 -26 -19 -45t-45 -19h-192v-272q0 -46 -33 -79t-79 -33h-160q-46 0 -79 33t-33 79v272h-192q-26 0 -45 19t-19 45q0 18 9 33l247 411v132h-45l-227 -341q-29 -43 -80 -43 -q-40 0 -68 28t-28 68q0 29 16 53l256 384q73 107 176 107h384q103 0 176 -107l256 -384q16 -24 16 -53zM864 1280q0 -93 -65.5 -158.5t-158.5 -65.5t-158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5t158.5 -65.5t65.5 -158.5z" /> - <glyph glyph-name="male" unicode="" horiz-adv-x="1024" -d="M1024 832v-416q0 -40 -28 -68t-68 -28t-68 28t-28 68v352h-64v-912q0 -46 -33 -79t-79 -33t-79 33t-33 79v464h-64v-464q0 -46 -33 -79t-79 -33t-79 33t-33 79v912h-64v-352q0 -40 -28 -68t-68 -28t-68 28t-28 68v416q0 80 56 136t136 56h640q80 0 136 -56t56 -136z -M736 1280q0 -93 -65.5 -158.5t-158.5 -65.5t-158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5t158.5 -65.5t65.5 -158.5z" /> - <glyph glyph-name="gittip" unicode="" -d="M773 234l350 473q16 22 24.5 59t-6 85t-61.5 79q-40 26 -83 25.5t-73.5 -17.5t-54.5 -45q-36 -40 -96 -40q-59 0 -95 40q-24 28 -54.5 45t-73.5 17.5t-84 -25.5q-46 -31 -60.5 -79t-6 -85t24.5 -59zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 -t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="sun" unicode="" horiz-adv-x="1792" -d="M1472 640q0 117 -45.5 223.5t-123 184t-184 123t-223.5 45.5t-223.5 -45.5t-184 -123t-123 -184t-45.5 -223.5t45.5 -223.5t123 -184t184 -123t223.5 -45.5t223.5 45.5t184 123t123 184t45.5 223.5zM1748 363q-4 -15 -20 -20l-292 -96v-306q0 -16 -13 -26q-15 -10 -29 -4 -l-292 94l-180 -248q-10 -13 -26 -13t-26 13l-180 248l-292 -94q-14 -6 -29 4q-13 10 -13 26v306l-292 96q-16 5 -20 20q-5 17 4 29l180 248l-180 248q-9 13 -4 29q4 15 20 20l292 96v306q0 16 13 26q15 10 29 4l292 -94l180 248q9 12 26 12t26 -12l180 -248l292 94 -q14 6 29 -4q13 -10 13 -26v-306l292 -96q16 -5 20 -20q5 -16 -4 -29l-180 -248l180 -248q9 -12 4 -29z" /> - <glyph glyph-name="_366" unicode="" -d="M1262 233q-54 -9 -110 -9q-182 0 -337 90t-245 245t-90 337q0 192 104 357q-201 -60 -328.5 -229t-127.5 -384q0 -130 51 -248.5t136.5 -204t204 -136.5t248.5 -51q144 0 273.5 61.5t220.5 171.5zM1465 318q-94 -203 -283.5 -324.5t-413.5 -121.5q-156 0 -298 61 -t-245 164t-164 245t-61 298q0 153 57.5 292.5t156 241.5t235.5 164.5t290 68.5q44 2 61 -39q18 -41 -15 -72q-86 -78 -131.5 -181.5t-45.5 -218.5q0 -148 73 -273t198 -198t273 -73q118 0 228 51q41 18 72 -13q14 -14 17.5 -34t-4.5 -38z" /> - <glyph glyph-name="archive" unicode="" horiz-adv-x="1792" -d="M1088 704q0 26 -19 45t-45 19h-256q-26 0 -45 -19t-19 -45t19 -45t45 -19h256q26 0 45 19t19 45zM1664 896v-960q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v960q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1728 1344v-256q0 -26 -19 -45t-45 -19h-1536 -q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1536q26 0 45 -19t19 -45z" /> - <glyph glyph-name="bug" unicode="" horiz-adv-x="1664" -d="M1632 576q0 -26 -19 -45t-45 -19h-224q0 -171 -67 -290l208 -209q19 -19 19 -45t-19 -45q-18 -19 -45 -19t-45 19l-198 197q-5 -5 -15 -13t-42 -28.5t-65 -36.5t-82 -29t-97 -13v896h-128v-896q-51 0 -101.5 13.5t-87 33t-66 39t-43.5 32.5l-15 14l-183 -207 -q-20 -21 -48 -21q-24 0 -43 16q-19 18 -20.5 44.5t15.5 46.5l202 227q-58 114 -58 274h-224q-26 0 -45 19t-19 45t19 45t45 19h224v294l-173 173q-19 19 -19 45t19 45t45 19t45 -19l173 -173h844l173 173q19 19 45 19t45 -19t19 -45t-19 -45l-173 -173v-294h224q26 0 45 -19 -t19 -45zM1152 1152h-640q0 133 93.5 226.5t226.5 93.5t226.5 -93.5t93.5 -226.5z" /> - <glyph glyph-name="vk" unicode="" horiz-adv-x="1920" -d="M1917 1016q23 -64 -150 -294q-24 -32 -65 -85q-40 -51 -55 -72t-30.5 -49.5t-12 -42t13 -34.5t32.5 -43t57 -53q4 -2 5 -4q141 -131 191 -221q3 -5 6.5 -12.5t7 -26.5t-0.5 -34t-25 -27.5t-59 -12.5l-256 -4q-24 -5 -56 5t-52 22l-20 12q-30 21 -70 64t-68.5 77.5t-61 58 -t-56.5 15.5q-3 -1 -8 -3.5t-17 -14.5t-21.5 -29.5t-17 -52t-6.5 -77.5q0 -15 -3.5 -27.5t-7.5 -18.5l-4 -5q-18 -19 -53 -22h-115q-71 -4 -146 16.5t-131.5 53t-103 66t-70.5 57.5l-25 24q-10 10 -27.5 30t-71.5 91t-106 151t-122.5 211t-130.5 272q-6 16 -6 27t3 16l4 6 -q15 19 57 19l274 2q12 -2 23 -6.5t16 -8.5l5 -3q16 -11 24 -32q20 -50 46 -103.5t41 -81.5l16 -29q29 -60 56 -104t48.5 -68.5t41.5 -38.5t34 -14t27 5q2 1 5 5t12 22t13.5 47t9.5 81t0 125q-2 40 -9 73t-14 46l-6 12q-25 34 -85 43q-13 2 5 24q16 19 38 30q53 26 239 24 -q82 -1 135 -13q20 -5 33.5 -13.5t20.5 -24t10.5 -32t3.5 -45.5t-1 -55t-2.5 -70.5t-1.5 -82.5q0 -11 -1 -42t-0.5 -48t3.5 -40.5t11.5 -39t22.5 -24.5q8 -2 17 -4t26 11t38 34.5t52 67t68 107.5q60 104 107 225q4 10 10 17.5t11 10.5l4 3l5 2.5t13 3t20 0.5l288 2 -q39 5 64 -2.5t31 -16.5z" /> - <glyph glyph-name="weibo" unicode="" horiz-adv-x="1792" -d="M675 252q21 34 11 69t-45 50q-34 14 -73 1t-60 -46q-22 -34 -13 -68.5t43 -50.5t74.5 -2.5t62.5 47.5zM769 373q8 13 3.5 26.5t-17.5 18.5q-14 5 -28.5 -0.5t-21.5 -18.5q-17 -31 13 -45q14 -5 29 0.5t22 18.5zM943 266q-45 -102 -158 -150t-224 -12 -q-107 34 -147.5 126.5t6.5 187.5q47 93 151.5 139t210.5 19q111 -29 158.5 -119.5t2.5 -190.5zM1255 426q-9 96 -89 170t-208.5 109t-274.5 21q-223 -23 -369.5 -141.5t-132.5 -264.5q9 -96 89 -170t208.5 -109t274.5 -21q223 23 369.5 141.5t132.5 264.5zM1563 422 -q0 -68 -37 -139.5t-109 -137t-168.5 -117.5t-226 -83t-270.5 -31t-275 33.5t-240.5 93t-171.5 151t-65 199.5q0 115 69.5 245t197.5 258q169 169 341.5 236t246.5 -7q65 -64 20 -209q-4 -14 -1 -20t10 -7t14.5 0.5t13.5 3.5l6 2q139 59 246 59t153 -61q45 -63 0 -178 -q-2 -13 -4.5 -20t4.5 -12.5t12 -7.5t17 -6q57 -18 103 -47t80 -81.5t34 -116.5zM1489 1046q42 -47 54.5 -108.5t-6.5 -117.5q-8 -23 -29.5 -34t-44.5 -4q-23 8 -34 29.5t-4 44.5q20 63 -24 111t-107 35q-24 -5 -45 8t-25 37q-5 24 8 44.5t37 25.5q60 13 119 -5.5t101 -65.5z -M1670 1209q87 -96 112.5 -222.5t-13.5 -241.5q-9 -27 -34 -40t-52 -4t-40 34t-5 52q28 82 10 172t-80 158q-62 69 -148 95.5t-173 8.5q-28 -6 -52 9.5t-30 43.5t9.5 51.5t43.5 29.5q123 26 244 -11.5t208 -134.5z" /> - <glyph glyph-name="renren" unicode="" -d="M1133 -34q-171 -94 -368 -94q-196 0 -367 94q138 87 235.5 211t131.5 268q35 -144 132.5 -268t235.5 -211zM638 1394v-485q0 -252 -126.5 -459.5t-330.5 -306.5q-181 215 -181 495q0 187 83.5 349.5t229.5 269.5t325 137zM1536 638q0 -280 -181 -495 -q-204 99 -330.5 306.5t-126.5 459.5v485q179 -30 325 -137t229.5 -269.5t83.5 -349.5z" /> - <glyph glyph-name="_372" unicode="" horiz-adv-x="1408" -d="M1402 433q-32 -80 -76 -138t-91 -88.5t-99 -46.5t-101.5 -14.5t-96.5 8.5t-86.5 22t-69.5 27.5t-46 22.5l-17 10q-113 -228 -289.5 -359.5t-384.5 -132.5q-19 0 -32 13t-13 32t13 31.5t32 12.5q173 1 322.5 107.5t251.5 294.5q-36 -14 -72 -23t-83 -13t-91 2.5t-93 28.5 -t-92 59t-84.5 100t-74.5 146q114 47 214 57t167.5 -7.5t124.5 -56.5t88.5 -77t56.5 -82q53 131 79 291q-7 -1 -18 -2.5t-46.5 -2.5t-69.5 0.5t-81.5 10t-88.5 23t-84 42.5t-75 65t-54.5 94.5t-28.5 127.5q70 28 133.5 36.5t112.5 -1t92 -30t73.5 -50t56 -61t42 -63t27.5 -56 -t16 -39.5l4 -16q12 122 12 195q-8 6 -21.5 16t-49 44.5t-63.5 71.5t-54 93t-33 112.5t12 127t70 138.5q73 -25 127.5 -61.5t84.5 -76.5t48 -85t20.5 -89t-0.5 -85.5t-13 -76.5t-19 -62t-17 -42l-7 -15q1 -4 1 -50t-1 -72q3 7 10 18.5t30.5 43t50.5 58t71 55.5t91.5 44.5 -t112 14.5t132.5 -24q-2 -78 -21.5 -141.5t-50 -104.5t-69.5 -71.5t-81.5 -45.5t-84.5 -24t-80 -9.5t-67.5 1t-46.5 4.5l-17 3q-23 -147 -73 -283q6 7 18 18.5t49.5 41t77.5 52.5t99.5 42t117.5 20t129 -23.5t137 -77.5z" /> - <glyph glyph-name="stack_exchange" unicode="" horiz-adv-x="1280" -d="M1259 283v-66q0 -85 -57.5 -144.5t-138.5 -59.5h-57l-260 -269v269h-529q-81 0 -138.5 59.5t-57.5 144.5v66h1238zM1259 609v-255h-1238v255h1238zM1259 937v-255h-1238v255h1238zM1259 1077v-67h-1238v67q0 84 57.5 143.5t138.5 59.5h846q81 0 138.5 -59.5t57.5 -143.5z -" /> - <glyph glyph-name="_374" unicode="" -d="M1152 640q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192h-352q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h352v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198 -t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="arrow_circle_alt_left" unicode="" -d="M1152 736v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-352v-192q0 -14 -9 -23t-23 -9q-12 0 -24 10l-319 319q-9 9 -9 23t9 23l320 320q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5v-192h352q13 0 22.5 -9.5t9.5 -22.5zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198 -t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="_376" unicode="" -d="M1024 960v-640q0 -26 -19 -45t-45 -19q-20 0 -37 12l-448 320q-27 19 -27 52t27 52l448 320q17 12 37 12q26 0 45 -19t19 -45zM1280 160v960q0 13 -9.5 22.5t-22.5 9.5h-960q-13 0 -22.5 -9.5t-9.5 -22.5v-960q0 -13 9.5 -22.5t22.5 -9.5h960q13 0 22.5 9.5t9.5 22.5z -M1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="dot_circle_alt" unicode="" -d="M1024 640q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5 -t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="_378" unicode="" horiz-adv-x="1664" -d="M1023 349l102 -204q-58 -179 -210 -290t-339 -111q-156 0 -288.5 77.5t-210 210t-77.5 288.5q0 181 104.5 330t274.5 211l17 -131q-122 -54 -195 -165.5t-73 -244.5q0 -185 131.5 -316.5t316.5 -131.5q126 0 232.5 65t165 175.5t49.5 236.5zM1571 249l58 -114l-256 -128 -q-13 -7 -29 -7q-40 0 -57 35l-239 477h-472q-24 0 -42.5 16.5t-21.5 40.5l-96 779q-2 17 6 42q14 51 57 82.5t97 31.5q66 0 113 -47t47 -113q0 -69 -52 -117.5t-120 -41.5l37 -289h423v-128h-407l16 -128h455q40 0 57 -35l228 -455z" /> - <glyph glyph-name="vimeo_square" unicode="" -d="M1292 898q10 216 -161 222q-231 8 -312 -261q44 19 82 19q85 0 74 -96q-4 -57 -74 -167t-105 -110q-43 0 -82 169q-13 54 -45 255q-30 189 -160 177q-59 -7 -164 -100l-81 -72l-81 -72l52 -67q76 52 87 52q57 0 107 -179q15 -55 45 -164.5t45 -164.5q68 -179 164 -179 -q157 0 383 294q220 283 226 444zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_380" unicode="" horiz-adv-x="1152" -d="M1152 704q0 -191 -94.5 -353t-256.5 -256.5t-353 -94.5h-160q-14 0 -23 9t-9 23v611l-215 -66q-3 -1 -9 -1q-10 0 -19 6q-13 10 -13 26v128q0 23 23 31l233 71v93l-215 -66q-3 -1 -9 -1q-10 0 -19 6q-13 10 -13 26v128q0 23 23 31l233 71v250q0 14 9 23t23 9h160 -q14 0 23 -9t9 -23v-181l375 116q15 5 28 -5t13 -26v-128q0 -23 -23 -31l-393 -121v-93l375 116q15 5 28 -5t13 -26v-128q0 -23 -23 -31l-393 -121v-487q188 13 318 151t130 328q0 14 9 23t23 9h160q14 0 23 -9t9 -23z" /> - <glyph glyph-name="plus_square_o" unicode="" horiz-adv-x="1408" -d="M1152 736v-64q0 -14 -9 -23t-23 -9h-352v-352q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v352h-352q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h352v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-352h352q14 0 23 -9t9 -23zM1280 288v832q0 66 -47 113t-113 47h-832 -q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_382" unicode="" horiz-adv-x="2176" -d="M620 416q-110 -64 -268 -64h-128v64h-64q-13 0 -22.5 23.5t-9.5 56.5q0 24 7 49q-58 2 -96.5 10.5t-38.5 20.5t38.5 20.5t96.5 10.5q-7 25 -7 49q0 33 9.5 56.5t22.5 23.5h64v64h128q158 0 268 -64h1113q42 -7 106.5 -18t80.5 -14q89 -15 150 -40.5t83.5 -47.5t22.5 -40 -t-22.5 -40t-83.5 -47.5t-150 -40.5q-16 -3 -80.5 -14t-106.5 -18h-1113zM1739 668q53 -36 53 -92t-53 -92l81 -30q68 48 68 122t-68 122zM625 400h1015q-217 -38 -456 -80q-57 0 -113 -24t-83 -48l-28 -24l-288 -288q-26 -26 -70.5 -45t-89.5 -19h-96l-93 464h29 -q157 0 273 64zM352 816h-29l93 464h96q46 0 90 -19t70 -45l288 -288q4 -4 11 -10.5t30.5 -23t48.5 -29t61.5 -23t72.5 -10.5l456 -80h-1015q-116 64 -273 64z" /> - <glyph glyph-name="_383" unicode="" horiz-adv-x="1664" -d="M1519 760q62 0 103.5 -40.5t41.5 -101.5q0 -97 -93 -130l-172 -59l56 -167q7 -21 7 -47q0 -59 -42 -102t-101 -43q-47 0 -85.5 27t-53.5 72l-55 165l-310 -106l55 -164q8 -24 8 -47q0 -59 -42 -102t-102 -43q-47 0 -85 27t-53 72l-55 163l-153 -53q-29 -9 -50 -9 -q-61 0 -101.5 40t-40.5 101q0 47 27.5 85t71.5 53l156 53l-105 313l-156 -54q-26 -8 -48 -8q-60 0 -101 40.5t-41 100.5q0 47 27.5 85t71.5 53l157 53l-53 159q-8 24 -8 47q0 60 42 102.5t102 42.5q47 0 85 -27t53 -72l54 -160l310 105l-54 160q-8 24 -8 47q0 59 42.5 102 -t101.5 43q47 0 85.5 -27.5t53.5 -71.5l53 -161l162 55q21 6 43 6q60 0 102.5 -39.5t42.5 -98.5q0 -45 -30 -81.5t-74 -51.5l-157 -54l105 -316l164 56q24 8 46 8zM725 498l310 105l-105 315l-310 -107z" /> - <glyph glyph-name="_384" unicode="" -d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960zM1280 352v436q-31 -35 -64 -55q-34 -22 -132.5 -85t-151.5 -99q-98 -69 -164 -69v0v0q-66 0 -164 69 -q-47 32 -142 92.5t-142 92.5q-12 8 -33 27t-31 27v-436q0 -40 28 -68t68 -28h832q40 0 68 28t28 68zM1280 925q0 41 -27.5 70t-68.5 29h-832q-40 0 -68 -28t-28 -68q0 -37 30.5 -76.5t67.5 -64.5q47 -32 137.5 -89t129.5 -83q3 -2 17 -11.5t21 -14t21 -13t23.5 -13 -t21.5 -9.5t22.5 -7.5t20.5 -2.5t20.5 2.5t22.5 7.5t21.5 9.5t23.5 13t21 13t21 14t17 11.5l267 174q35 23 66.5 62.5t31.5 73.5z" /> - <glyph glyph-name="_385" unicode="" horiz-adv-x="1792" -d="M127 640q0 163 67 313l367 -1005q-196 95 -315 281t-119 411zM1415 679q0 -19 -2.5 -38.5t-10 -49.5t-11.5 -44t-17.5 -59t-17.5 -58l-76 -256l-278 826q46 3 88 8q19 2 26 18.5t-2.5 31t-28.5 13.5l-205 -10q-75 1 -202 10q-12 1 -20.5 -5t-11.5 -15t-1.5 -18.5t9 -16.5 -t19.5 -8l80 -8l120 -328l-168 -504l-280 832q46 3 88 8q19 2 26 18.5t-2.5 31t-28.5 13.5l-205 -10q-7 0 -23 0.5t-26 0.5q105 160 274.5 253.5t367.5 93.5q147 0 280.5 -53t238.5 -149h-10q-55 0 -92 -40.5t-37 -95.5q0 -12 2 -24t4 -21.5t8 -23t9 -21t12 -22.5t12.5 -21 -t14.5 -24t14 -23q63 -107 63 -212zM909 573l237 -647q1 -6 5 -11q-126 -44 -255 -44q-112 0 -217 32zM1570 1009q95 -174 95 -369q0 -209 -104 -385.5t-279 -278.5l235 678q59 169 59 276q0 42 -6 79zM896 1536q182 0 348 -71t286 -191t191 -286t71 -348t-71 -348t-191 -286 -t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71zM896 -215q173 0 331.5 68t273 182.5t182.5 273t68 331.5t-68 331.5t-182.5 273t-273 182.5t-331.5 68t-331.5 -68t-273 -182.5t-182.5 -273t-68 -331.5t68 -331.5t182.5 -273 -t273 -182.5t331.5 -68z" /> - <glyph glyph-name="_386" unicode="" horiz-adv-x="1792" -d="M1086 1536v-1536l-272 -128q-228 20 -414 102t-293 208.5t-107 272.5q0 140 100.5 263.5t275 205.5t391.5 108v-172q-217 -38 -356.5 -150t-139.5 -255q0 -152 154.5 -267t388.5 -145v1360zM1755 954l37 -390l-525 114l147 83q-119 70 -280 99v172q277 -33 481 -157z" /> - <glyph glyph-name="_387" unicode="" horiz-adv-x="2048" -d="M960 1536l960 -384v-128h-128q0 -26 -20.5 -45t-48.5 -19h-1526q-28 0 -48.5 19t-20.5 45h-128v128zM256 896h256v-768h128v768h256v-768h128v768h256v-768h128v768h256v-768h59q28 0 48.5 -19t20.5 -45v-64h-1664v64q0 26 20.5 45t48.5 19h59v768zM1851 -64 -q28 0 48.5 -19t20.5 -45v-128h-1920v128q0 26 20.5 45t48.5 19h1782z" /> - <glyph glyph-name="_388" unicode="" horiz-adv-x="2304" -d="M1774 700l18 -316q4 -69 -82 -128t-235 -93.5t-323 -34.5t-323 34.5t-235 93.5t-82 128l18 316l574 -181q22 -7 48 -7t48 7zM2304 1024q0 -23 -22 -31l-1120 -352q-4 -1 -10 -1t-10 1l-652 206q-43 -34 -71 -111.5t-34 -178.5q63 -36 63 -109q0 -69 -58 -107l58 -433 -q2 -14 -8 -25q-9 -11 -24 -11h-192q-15 0 -24 11q-10 11 -8 25l58 433q-58 38 -58 107q0 73 65 111q11 207 98 330l-333 104q-22 8 -22 31t22 31l1120 352q4 1 10 1t10 -1l1120 -352q22 -8 22 -31z" /> - <glyph glyph-name="_389" unicode="" -d="M859 579l13 -707q-62 11 -105 11q-41 0 -105 -11l13 707q-40 69 -168.5 295.5t-216.5 374.5t-181 287q58 -15 108 -15q44 0 111 15q63 -111 133.5 -229.5t167 -276.5t138.5 -227q37 61 109.5 177.5t117.5 190t105 176t107 189.5q54 -14 107 -14q56 0 114 14v0 -q-28 -39 -60 -88.5t-49.5 -78.5t-56.5 -96t-49 -84q-146 -248 -353 -610z" /> - <glyph glyph-name="uniF1A0" unicode="" -d="M768 750h725q12 -67 12 -128q0 -217 -91 -387.5t-259.5 -266.5t-386.5 -96q-157 0 -299 60.5t-245 163.5t-163.5 245t-60.5 299t60.5 299t163.5 245t245 163.5t299 60.5q300 0 515 -201l-209 -201q-123 119 -306 119q-129 0 -238.5 -65t-173.5 -176.5t-64 -243.5 -t64 -243.5t173.5 -176.5t238.5 -65q87 0 160 24t120 60t82 82t51.5 87t22.5 78h-436v264z" /> - <glyph glyph-name="f1a1" unicode="" horiz-adv-x="1792" -d="M1095 369q16 -16 0 -31q-62 -62 -199 -62t-199 62q-16 15 0 31q6 6 15 6t15 -6q48 -49 169 -49q120 0 169 49q6 6 15 6t15 -6zM788 550q0 -37 -26 -63t-63 -26t-63.5 26t-26.5 63q0 38 26.5 64t63.5 26t63 -26.5t26 -63.5zM1183 550q0 -37 -26.5 -63t-63.5 -26t-63 26 -t-26 63t26 63.5t63 26.5t63.5 -26t26.5 -64zM1434 670q0 49 -35 84t-85 35t-86 -36q-130 90 -311 96l63 283l200 -45q0 -37 26 -63t63 -26t63.5 26.5t26.5 63.5t-26.5 63.5t-63.5 26.5q-54 0 -80 -50l-221 49q-19 5 -25 -16l-69 -312q-180 -7 -309 -97q-35 37 -87 37 -q-50 0 -85 -35t-35 -84q0 -35 18.5 -64t49.5 -44q-6 -27 -6 -56q0 -142 140 -243t337 -101q198 0 338 101t140 243q0 32 -7 57q30 15 48 43.5t18 63.5zM1792 640q0 -182 -71 -348t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191 -t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="_392" unicode="" -d="M939 407q13 -13 0 -26q-53 -53 -171 -53t-171 53q-13 13 0 26q5 6 13 6t13 -6q42 -42 145 -42t145 42q5 6 13 6t13 -6zM676 563q0 -31 -23 -54t-54 -23t-54 23t-23 54q0 32 22.5 54.5t54.5 22.5t54.5 -22.5t22.5 -54.5zM1014 563q0 -31 -23 -54t-54 -23t-54 23t-23 54 -q0 32 22.5 54.5t54.5 22.5t54.5 -22.5t22.5 -54.5zM1229 666q0 42 -30 72t-73 30q-42 0 -73 -31q-113 78 -267 82l54 243l171 -39q1 -32 23.5 -54t53.5 -22q32 0 54.5 22.5t22.5 54.5t-22.5 54.5t-54.5 22.5q-48 0 -69 -43l-189 42q-17 5 -21 -13l-60 -268q-154 -6 -265 -83 -q-30 32 -74 32q-43 0 -73 -30t-30 -72q0 -30 16 -55t42 -38q-5 -25 -5 -48q0 -122 120 -208.5t289 -86.5q170 0 290 86.5t120 208.5q0 25 -6 49q25 13 40.5 37.5t15.5 54.5zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960 -q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_393" unicode="" -d="M866 697l90 27v62q0 79 -58 135t-138 56t-138 -55.5t-58 -134.5v-283q0 -20 -14 -33.5t-33 -13.5t-32.5 13.5t-13.5 33.5v120h-151v-122q0 -82 57.5 -139t139.5 -57q81 0 138.5 56.5t57.5 136.5v280q0 19 13.5 33t33.5 14q19 0 32.5 -14t13.5 -33v-54zM1199 502v122h-150 -v-126q0 -20 -13.5 -33.5t-33.5 -13.5q-19 0 -32.5 14t-13.5 33v123l-90 -26l-60 28v-123q0 -80 58 -137t139 -57t138.5 57t57.5 139zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103 -t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="f1a4" unicode="" horiz-adv-x="1920" -d="M1062 824v118q0 42 -30 72t-72 30t-72 -30t-30 -72v-612q0 -175 -126 -299t-303 -124q-178 0 -303.5 125.5t-125.5 303.5v266h328v-262q0 -43 30 -72.5t72 -29.5t72 29.5t30 72.5v620q0 171 126.5 292t301.5 121q176 0 302 -122t126 -294v-136l-195 -58zM1592 602h328 -v-266q0 -178 -125.5 -303.5t-303.5 -125.5q-177 0 -303 124.5t-126 300.5v268l131 -61l195 58v-270q0 -42 30 -71.5t72 -29.5t72 29.5t30 71.5v275z" /> - <glyph glyph-name="_395" unicode="" -d="M1472 160v480h-704v704h-480q-93 0 -158.5 -65.5t-65.5 -158.5v-480h704v-704h480q93 0 158.5 65.5t65.5 158.5zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5 -t84.5 -203.5z" /> - <glyph glyph-name="_396" unicode="" horiz-adv-x="2048" -d="M328 1254h204v-983h-532v697h328v286zM328 435v369h-123v-369h123zM614 968v-697h205v697h-205zM614 1254v-204h205v204h-205zM901 968h533v-942h-533v163h328v82h-328v697zM1229 435v369h-123v-369h123zM1516 968h532v-942h-532v163h327v82h-327v697zM1843 435v369h-123 -v-369h123z" /> - <glyph glyph-name="_397" unicode="" -d="M1046 516q0 -64 -38 -109t-91 -45q-43 0 -70 15v277q28 17 70 17q53 0 91 -45.5t38 -109.5zM703 944q0 -64 -38 -109.5t-91 -45.5q-43 0 -70 15v277q28 17 70 17q53 0 91 -45t38 -109zM1265 513q0 134 -88 229t-213 95q-20 0 -39 -3q-23 -78 -78 -136q-87 -95 -211 -101 -v-636l211 41v206q51 -19 117 -19q125 0 213 95t88 229zM922 940q0 134 -88.5 229t-213.5 95q-74 0 -141 -36h-186v-840l211 41v206q55 -19 116 -19q125 0 213.5 95t88.5 229zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960 -q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_398" unicode="" horiz-adv-x="2038" -d="M1222 607q75 3 143.5 -20.5t118 -58.5t101 -94.5t84 -108t75.5 -120.5q33 -56 78.5 -109t75.5 -80.5t99 -88.5q-48 -30 -108.5 -57.5t-138.5 -59t-114 -47.5q-44 37 -74 115t-43.5 164.5t-33 180.5t-42.5 168.5t-72.5 123t-122.5 48.5l-10 -2l-6 -4q4 -5 13 -14 -q6 -5 28 -23.5t25.5 -22t19 -18t18 -20.5t11.5 -21t10.5 -27.5t4.5 -31t4 -40.5l1 -33q1 -26 -2.5 -57.5t-7.5 -52t-12.5 -58.5t-11.5 -53q-35 1 -101 -9.5t-98 -10.5q-39 0 -72 10q-2 16 -2 47q0 74 3 96q2 13 31.5 41.5t57 59t26.5 51.5q-24 2 -43 -24 -q-36 -53 -111.5 -99.5t-136.5 -46.5q-25 0 -75.5 63t-106.5 139.5t-84 96.5q-6 4 -27 30q-482 -112 -513 -112q-16 0 -28 11t-12 27q0 15 8.5 26.5t22.5 14.5l486 106q-8 14 -8 25t5.5 17.5t16 11.5t20 7t23 4.5t18.5 4.5q4 1 15.5 7.5t17.5 6.5q15 0 28 -16t20 -33 -q163 37 172 37q17 0 29.5 -11t12.5 -28q0 -15 -8.5 -26t-23.5 -14l-182 -40l-1 -16q-1 -26 81.5 -117.5t104.5 -91.5q47 0 119 80t72 129q0 36 -23.5 53t-51 18.5t-51 11.5t-23.5 34q0 16 10 34l-68 19q43 44 43 117q0 26 -5 58q82 16 144 16q44 0 71.5 -1.5t48.5 -8.5 -t31 -13.5t20.5 -24.5t15.5 -33.5t17 -47.5t24 -60l50 25q-3 -40 -23 -60t-42.5 -21t-40 -6.5t-16.5 -20.5zM1282 842q-5 5 -13.5 15.5t-12 14.5t-10.5 11.5t-10 10.5l-8 8t-8.5 7.5t-8 5t-8.5 4.5q-7 3 -14.5 5t-20.5 2.5t-22 0.5h-32.5h-37.5q-126 0 -217 -43 -q16 30 36 46.5t54 29.5t65.5 36t46 36.5t50 55t43.5 50.5q12 -9 28 -31.5t32 -36.5t38 -13l12 1v-76l22 -1q247 95 371 190q28 21 50 39t42.5 37.5t33 31t29.5 34t24 31t24.5 37t23 38t27 47.5t29.5 53l7 9q-2 -53 -43 -139q-79 -165 -205 -264t-306 -142q-14 -3 -42 -7.5 -t-50 -9.5t-39 -14q3 -19 24.5 -46t21.5 -34q0 -11 -26 -30zM1061 -79q39 26 131.5 47.5t146.5 21.5q9 0 22.5 -15.5t28 -42.5t26 -50t24 -51t14.5 -33q-121 -45 -244 -45q-61 0 -125 11zM822 568l48 12l109 -177l-73 -48zM1323 51q3 -15 3 -16q0 -7 -17.5 -14.5t-46 -13 -t-54 -9.5t-53.5 -7.5t-32 -4.5l-7 43q21 2 60.5 8.5t72 10t60.5 3.5h14zM866 679l-96 -20l-6 17q10 1 32.5 7t34.5 6q19 0 35 -10zM1061 45h31l10 -83l-41 -12v95zM1950 1535v1v-1zM1950 1535l-1 -5l-2 -2l1 3zM1950 1535l1 1z" /> - <glyph glyph-name="_399" unicode="" -d="M1167 -50q-5 19 -24 5q-30 -22 -87 -39t-131 -17q-129 0 -193 49q-5 4 -13 4q-11 0 -26 -12q-7 -6 -7.5 -16t7.5 -20q34 -32 87.5 -46t102.5 -12.5t99 4.5q41 4 84.5 20.5t65 30t28.5 20.5q12 12 7 29zM1128 65q-19 47 -39 61q-23 15 -76 15q-47 0 -71 -10 -q-29 -12 -78 -56q-26 -24 -12 -44q9 -8 17.5 -4.5t31.5 23.5q3 2 10.5 8.5t10.5 8.5t10 7t11.5 7t12.5 5t15 4.5t16.5 2.5t20.5 1q27 0 44.5 -7.5t23 -14.5t13.5 -22q10 -17 12.5 -20t12.5 1q23 12 14 34zM1483 346q0 22 -5 44.5t-16.5 45t-34 36.5t-52.5 14 -q-33 0 -97 -41.5t-129 -83.5t-101 -42q-27 -1 -63.5 19t-76 49t-83.5 58t-100 49t-111 19q-115 -1 -197 -78.5t-84 -178.5q-2 -112 74 -164q29 -20 62.5 -28.5t103.5 -8.5q57 0 132 32.5t134 71t120 70.5t93 31q26 -1 65 -31.5t71.5 -67t68 -67.5t55.5 -32q35 -3 58.5 14 -t55.5 63q28 41 42.5 101t14.5 106zM1536 506q0 -164 -62 -304.5t-166 -236t-242.5 -149.5t-290.5 -54t-293 57.5t-247.5 157t-170.5 241.5t-64 302q0 89 19.5 172.5t49 145.5t70.5 118.5t78.5 94t78.5 69.5t64.5 46.5t42.5 24.5q14 8 51 26.5t54.5 28.5t48 30t60.5 44 -q36 28 58 72.5t30 125.5q129 -155 186 -193q44 -29 130 -68t129 -66q21 -13 39 -25t60.5 -46.5t76 -70.5t75 -95t69 -122t47 -148.5t19.5 -177.5z" /> - <glyph glyph-name="_400" unicode="" -d="M1070 463l-160 -160l-151 -152l-30 -30q-65 -64 -151.5 -87t-171.5 -2q-16 -70 -72 -115t-129 -45q-85 0 -145 60.5t-60 145.5q0 72 44.5 128t113.5 72q-22 86 1 173t88 152l12 12l151 -152l-11 -11q-37 -37 -37 -89t37 -90q37 -37 89 -37t89 37l30 30l151 152l161 160z -M729 1145l12 -12l-152 -152l-12 12q-37 37 -89 37t-89 -37t-37 -89.5t37 -89.5l29 -29l152 -152l160 -160l-151 -152l-161 160l-151 152l-30 30q-68 67 -90 159.5t5 179.5q-70 15 -115 71t-45 129q0 85 60 145.5t145 60.5q76 0 133.5 -49t69.5 -123q84 20 169.5 -3.5 -t149.5 -87.5zM1536 78q0 -85 -60 -145.5t-145 -60.5q-74 0 -131 47t-71 118q-86 -28 -179.5 -6t-161.5 90l-11 12l151 152l12 -12q37 -37 89 -37t89 37t37 89t-37 89l-30 30l-152 152l-160 160l152 152l160 -160l152 -152l29 -30q64 -64 87.5 -150.5t2.5 -171.5 -q76 -11 126.5 -68.5t50.5 -134.5zM1534 1202q0 -77 -51 -135t-127 -69q26 -85 3 -176.5t-90 -158.5l-12 -12l-151 152l12 12q37 37 37 89t-37 89t-89 37t-89 -37l-30 -30l-152 -152l-160 -160l-152 152l161 160l152 152l29 30q67 67 159 89.5t178 -3.5q11 75 68.5 126 -t135.5 51q85 0 145 -60.5t60 -145.5z" /> - <glyph glyph-name="f1ab" unicode="" -d="M654 458q-1 -3 -12.5 0.5t-31.5 11.5l-20 9q-44 20 -87 49q-7 5 -41 31.5t-38 28.5q-67 -103 -134 -181q-81 -95 -105 -110q-4 -2 -19.5 -4t-18.5 0q6 4 82 92q21 24 85.5 115t78.5 118q17 30 51 98.5t36 77.5q-8 1 -110 -33q-8 -2 -27.5 -7.5t-34.5 -9.5t-17 -5 -q-2 -2 -2 -10.5t-1 -9.5q-5 -10 -31 -15q-23 -7 -47 0q-18 4 -28 21q-4 6 -5 23q6 2 24.5 5t29.5 6q58 16 105 32q100 35 102 35q10 2 43 19.5t44 21.5q9 3 21.5 8t14.5 5.5t6 -0.5q2 -12 -1 -33q0 -2 -12.5 -27t-26.5 -53.5t-17 -33.5q-25 -50 -77 -131l64 -28 -q12 -6 74.5 -32t67.5 -28q4 -1 10.5 -25.5t4.5 -30.5zM449 944q3 -15 -4 -28q-12 -23 -50 -38q-30 -12 -60 -12q-26 3 -49 26q-14 15 -18 41l1 3q3 -3 19.5 -5t26.5 0t58 16q36 12 55 14q17 0 21 -17zM1147 815l63 -227l-139 42zM39 15l694 232v1032l-694 -233v-1031z -M1280 332l102 -31l-181 657l-100 31l-216 -536l102 -31l45 110l211 -65zM777 1294l573 -184v380zM1088 -29l158 -13l-54 -160l-40 66q-130 -83 -276 -108q-58 -12 -91 -12h-84q-79 0 -199.5 39t-183.5 85q-8 7 -8 16q0 8 5 13.5t13 5.5q4 0 18 -7.5t30.5 -16.5t20.5 -11 -q73 -37 159.5 -61.5t157.5 -24.5q95 0 167 14.5t157 50.5q15 7 30.5 15.5t34 19t28.5 16.5zM1536 1050v-1079l-774 246q-14 -6 -375 -127.5t-368 -121.5q-13 0 -18 13q0 1 -1 3v1078q3 9 4 10q5 6 20 11q107 36 149 50v384l558 -198q2 0 160.5 55t316 108.5t161.5 53.5 -q20 0 20 -21v-418z" /> - <glyph glyph-name="_402" unicode="" horiz-adv-x="1792" -d="M288 1152q66 0 113 -47t47 -113v-1088q0 -66 -47 -113t-113 -47h-128q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h128zM1664 989q58 -34 93 -93t35 -128v-768q0 -106 -75 -181t-181 -75h-864q-66 0 -113 47t-47 113v1536q0 40 28 68t68 28h672q40 0 88 -20t76 -48 -l152 -152q28 -28 48 -76t20 -88v-163zM928 0v128q0 14 -9 23t-23 9h-128q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h128q14 0 23 9t9 23zM928 256v128q0 14 -9 23t-23 9h-128q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h128q14 0 23 9t9 23zM928 512v128q0 14 -9 23 -t-23 9h-128q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h128q14 0 23 9t9 23zM1184 0v128q0 14 -9 23t-23 9h-128q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h128q14 0 23 9t9 23zM1184 256v128q0 14 -9 23t-23 9h-128q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h128 -q14 0 23 9t9 23zM1184 512v128q0 14 -9 23t-23 9h-128q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h128q14 0 23 9t9 23zM1440 0v128q0 14 -9 23t-23 9h-128q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h128q14 0 23 9t9 23zM1440 256v128q0 14 -9 23t-23 9h-128 -q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h128q14 0 23 9t9 23zM1440 512v128q0 14 -9 23t-23 9h-128q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h128q14 0 23 9t9 23zM1536 896v256h-160q-40 0 -68 28t-28 68v160h-640v-512h896z" /> - <glyph glyph-name="_403" unicode="" -d="M1344 1536q26 0 45 -19t19 -45v-1664q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v1664q0 26 19 45t45 19h1280zM512 1248v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23zM512 992v-64q0 -14 9 -23t23 -9h64q14 0 23 9 -t9 23v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23zM512 736v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23zM512 480v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23zM384 160v64 -q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM384 416v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM384 672v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64 -q14 0 23 9t9 23zM384 928v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM384 1184v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM896 -96v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9 -t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM896 416v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM896 672v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM896 928v64 -q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM896 1184v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1152 160v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64 -q14 0 23 9t9 23zM1152 416v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1152 672v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1152 928v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9 -t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1152 1184v64q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h64q14 0 23 9t9 23z" /> - <glyph glyph-name="_404" unicode="" horiz-adv-x="1280" -d="M1188 988l-292 -292v-824q0 -46 -33 -79t-79 -33t-79 33t-33 79v384h-64v-384q0 -46 -33 -79t-79 -33t-79 33t-33 79v824l-292 292q-28 28 -28 68t28 68q29 28 68.5 28t67.5 -28l228 -228h368l228 228q28 28 68 28t68 -28q28 -29 28 -68.5t-28 -67.5zM864 1152 -q0 -93 -65.5 -158.5t-158.5 -65.5t-158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5t158.5 -65.5t65.5 -158.5z" /> - <glyph glyph-name="uniF1B1" unicode="" horiz-adv-x="1664" -d="M780 1064q0 -60 -19 -113.5t-63 -92.5t-105 -39q-76 0 -138 57.5t-92 135.5t-30 151q0 60 19 113.5t63 92.5t105 39q77 0 138.5 -57.5t91.5 -135t30 -151.5zM438 581q0 -80 -42 -139t-119 -59q-76 0 -141.5 55.5t-100.5 133.5t-35 152q0 80 42 139.5t119 59.5 -q76 0 141.5 -55.5t100.5 -134t35 -152.5zM832 608q118 0 255 -97.5t229 -237t92 -254.5q0 -46 -17 -76.5t-48.5 -45t-64.5 -20t-76 -5.5q-68 0 -187.5 45t-182.5 45q-66 0 -192.5 -44.5t-200.5 -44.5q-183 0 -183 146q0 86 56 191.5t139.5 192.5t187.5 146t193 59zM1071 819 -q-61 0 -105 39t-63 92.5t-19 113.5q0 74 30 151.5t91.5 135t138.5 57.5q61 0 105 -39t63 -92.5t19 -113.5q0 -73 -30 -151t-92 -135.5t-138 -57.5zM1503 923q77 0 119 -59.5t42 -139.5q0 -74 -35 -152t-100.5 -133.5t-141.5 -55.5q-77 0 -119 59t-42 139q0 74 35 152.5 -t100.5 134t141.5 55.5z" /> - <glyph glyph-name="_406" unicode="" horiz-adv-x="768" -d="M704 1008q0 -145 -57 -243.5t-152 -135.5l45 -821q2 -26 -16 -45t-44 -19h-192q-26 0 -44 19t-16 45l45 821q-95 37 -152 135.5t-57 243.5q0 128 42.5 249.5t117.5 200t160 78.5t160 -78.5t117.5 -200t42.5 -249.5z" /> - <glyph glyph-name="_407" unicode="" horiz-adv-x="1792" -d="M896 -93l640 349v636l-640 -233v-752zM832 772l698 254l-698 254l-698 -254zM1664 1024v-768q0 -35 -18 -65t-49 -47l-704 -384q-28 -16 -61 -16t-61 16l-704 384q-31 17 -49 47t-18 65v768q0 40 23 73t61 47l704 256q22 8 44 8t44 -8l704 -256q38 -14 61 -47t23 -73z -" /> - <glyph glyph-name="_408" unicode="" horiz-adv-x="2304" -d="M640 -96l384 192v314l-384 -164v-342zM576 358l404 173l-404 173l-404 -173zM1664 -96l384 192v314l-384 -164v-342zM1600 358l404 173l-404 173l-404 -173zM1152 651l384 165v266l-384 -164v-267zM1088 1030l441 189l-441 189l-441 -189zM2176 512v-416q0 -36 -19 -67 -t-52 -47l-448 -224q-25 -14 -57 -14t-57 14l-448 224q-4 2 -7 4q-2 -2 -7 -4l-448 -224q-25 -14 -57 -14t-57 14l-448 224q-33 16 -52 47t-19 67v416q0 38 21.5 70t56.5 48l434 186v400q0 38 21.5 70t56.5 48l448 192q23 10 50 10t50 -10l448 -192q35 -16 56.5 -48t21.5 -70 -v-400l434 -186q36 -16 57 -48t21 -70z" /> - <glyph glyph-name="_409" unicode="" horiz-adv-x="2048" -d="M1848 1197h-511v-124h511v124zM1596 771q-90 0 -146 -52.5t-62 -142.5h408q-18 195 -200 195zM1612 186q63 0 122 32t76 87h221q-100 -307 -427 -307q-214 0 -340.5 132t-126.5 347q0 208 130.5 345.5t336.5 137.5q138 0 240.5 -68t153 -179t50.5 -248q0 -17 -2 -47h-658 -q0 -111 57.5 -171.5t166.5 -60.5zM277 236h296q205 0 205 167q0 180 -199 180h-302v-347zM277 773h281q78 0 123.5 36.5t45.5 113.5q0 144 -190 144h-260v-294zM0 1282h594q87 0 155 -14t126.5 -47.5t90 -96.5t31.5 -154q0 -181 -172 -263q114 -32 172 -115t58 -204 -q0 -75 -24.5 -136.5t-66 -103.5t-98.5 -71t-121 -42t-134 -13h-611v1260z" /> - <glyph glyph-name="_410" unicode="" -d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960zM499 1041h-371v-787h382q117 0 197 57.5t80 170.5q0 158 -143 200q107 52 107 164q0 57 -19.5 96.5 -t-56.5 60.5t-79 29.5t-97 8.5zM477 723h-176v184h163q119 0 119 -90q0 -94 -106 -94zM486 388h-185v217h189q124 0 124 -113q0 -104 -128 -104zM1136 356q-68 0 -104 38t-36 107h411q1 10 1 30q0 132 -74.5 220.5t-203.5 88.5q-128 0 -210 -86t-82 -216q0 -135 79 -217 -t213 -82q205 0 267 191h-138q-11 -34 -47.5 -54t-75.5 -20zM1126 722q113 0 124 -122h-254q4 56 39 89t91 33zM964 988h319v-77h-319v77z" /> - <glyph glyph-name="_411" unicode="" horiz-adv-x="1792" -d="M1582 954q0 -101 -71.5 -172.5t-172.5 -71.5t-172.5 71.5t-71.5 172.5t71.5 172.5t172.5 71.5t172.5 -71.5t71.5 -172.5zM812 212q0 104 -73 177t-177 73q-27 0 -54 -6l104 -42q77 -31 109.5 -106.5t1.5 -151.5q-31 -77 -107 -109t-152 -1q-21 8 -62 24.5t-61 24.5 -q32 -60 91 -96.5t130 -36.5q104 0 177 73t73 177zM1642 953q0 126 -89.5 215.5t-215.5 89.5q-127 0 -216.5 -89.5t-89.5 -215.5q0 -127 89.5 -216t216.5 -89q126 0 215.5 89t89.5 216zM1792 953q0 -189 -133.5 -322t-321.5 -133l-437 -319q-12 -129 -109 -218t-229 -89 -q-121 0 -214 76t-118 192l-230 92v429l389 -157q79 48 173 48q13 0 35 -2l284 407q2 187 135.5 319t320.5 132q188 0 321.5 -133.5t133.5 -321.5z" /> - <glyph glyph-name="_412" unicode="" -d="M1242 889q0 80 -57 136.5t-137 56.5t-136.5 -57t-56.5 -136q0 -80 56.5 -136.5t136.5 -56.5t137 56.5t57 136.5zM632 301q0 -83 -58 -140.5t-140 -57.5q-56 0 -103 29t-72 77q52 -20 98 -40q60 -24 120 1.5t85 86.5q24 60 -1.5 120t-86.5 84l-82 33q22 5 42 5 -q82 0 140 -57.5t58 -140.5zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v153l172 -69q20 -92 93.5 -152t168.5 -60q104 0 181 70t87 173l345 252q150 0 255.5 105.5t105.5 254.5q0 150 -105.5 255.5t-255.5 105.5 -q-148 0 -253 -104.5t-107 -252.5l-225 -322q-9 1 -28 1q-75 0 -137 -37l-297 119v468q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5zM1289 887q0 -100 -71 -170.5t-171 -70.5t-170.5 70.5t-70.5 170.5t70.5 171t170.5 71q101 0 171.5 -70.5t70.5 -171.5z -" /> - <glyph glyph-name="_413" unicode="" horiz-adv-x="1792" -d="M836 367l-15 -368l-2 -22l-420 29q-36 3 -67 31.5t-47 65.5q-11 27 -14.5 55t4 65t12 55t21.5 64t19 53q78 -12 509 -28zM449 953l180 -379l-147 92q-63 -72 -111.5 -144.5t-72.5 -125t-39.5 -94.5t-18.5 -63l-4 -21l-190 357q-17 26 -18 56t6 47l8 18q35 63 114 188 -l-140 86zM1680 436l-188 -359q-12 -29 -36.5 -46.5t-43.5 -20.5l-18 -4q-71 -7 -219 -12l8 -164l-230 367l211 362l7 -173q170 -16 283 -5t170 33zM895 1360q-47 -63 -265 -435l-317 187l-19 12l225 356q20 31 60 45t80 10q24 -2 48.5 -12t42 -21t41.5 -33t36 -34.5 -t36 -39.5t32 -35zM1550 1053l212 -363q18 -37 12.5 -76t-27.5 -74q-13 -20 -33 -37t-38 -28t-48.5 -22t-47 -16t-51.5 -14t-46 -12q-34 72 -265 436l313 195zM1407 1279l142 83l-220 -373l-419 20l151 86q-34 89 -75 166t-75.5 123.5t-64.5 80t-47 46.5l-17 13l405 -1 -q31 3 58 -10.5t39 -28.5l11 -15q39 -61 112 -190z" /> - <glyph glyph-name="_414" unicode="" horiz-adv-x="2048" -d="M480 448q0 66 -47 113t-113 47t-113 -47t-47 -113t47 -113t113 -47t113 47t47 113zM516 768h1016l-89 357q-2 8 -14 17.5t-21 9.5h-768q-9 0 -21 -9.5t-14 -17.5zM1888 448q0 66 -47 113t-113 47t-113 -47t-47 -113t47 -113t113 -47t113 47t47 113zM2048 544v-384 -q0 -14 -9 -23t-23 -9h-96v-128q0 -80 -56 -136t-136 -56t-136 56t-56 136v128h-1024v-128q0 -80 -56 -136t-136 -56t-136 56t-56 136v128h-96q-14 0 -23 9t-9 23v384q0 93 65.5 158.5t158.5 65.5h28l105 419q23 94 104 157.5t179 63.5h768q98 0 179 -63.5t104 -157.5 -l105 -419h28q93 0 158.5 -65.5t65.5 -158.5z" /> - <glyph glyph-name="_415" unicode="" horiz-adv-x="2048" -d="M1824 640q93 0 158.5 -65.5t65.5 -158.5v-384q0 -14 -9 -23t-23 -9h-96v-64q0 -80 -56 -136t-136 -56t-136 56t-56 136v64h-1024v-64q0 -80 -56 -136t-136 -56t-136 56t-56 136v64h-96q-14 0 -23 9t-9 23v384q0 93 65.5 158.5t158.5 65.5h28l105 419q23 94 104 157.5 -t179 63.5h128v224q0 14 9 23t23 9h448q14 0 23 -9t9 -23v-224h128q98 0 179 -63.5t104 -157.5l105 -419h28zM320 160q66 0 113 47t47 113t-47 113t-113 47t-113 -47t-47 -113t47 -113t113 -47zM516 640h1016l-89 357q-2 8 -14 17.5t-21 9.5h-768q-9 0 -21 -9.5t-14 -17.5z -M1728 160q66 0 113 47t47 113t-47 113t-113 47t-113 -47t-47 -113t47 -113t113 -47z" /> - <glyph glyph-name="_416" unicode="" -d="M1504 64q0 -26 -19 -45t-45 -19h-462q1 -17 6 -87.5t5 -108.5q0 -25 -18 -42.5t-43 -17.5h-320q-25 0 -43 17.5t-18 42.5q0 38 5 108.5t6 87.5h-462q-26 0 -45 19t-19 45t19 45l402 403h-229q-26 0 -45 19t-19 45t19 45l402 403h-197q-26 0 -45 19t-19 45t19 45l384 384 -q19 19 45 19t45 -19l384 -384q19 -19 19 -45t-19 -45t-45 -19h-197l402 -403q19 -19 19 -45t-19 -45t-45 -19h-229l402 -403q19 -19 19 -45z" /> - <glyph glyph-name="_417" unicode="" -d="M1127 326q0 32 -30 51q-193 115 -447 115q-133 0 -287 -34q-42 -9 -42 -52q0 -20 13.5 -34.5t35.5 -14.5q5 0 37 8q132 27 243 27q226 0 397 -103q19 -11 33 -11q19 0 33 13.5t14 34.5zM1223 541q0 40 -35 61q-237 141 -548 141q-153 0 -303 -42q-48 -13 -48 -64 -q0 -25 17.5 -42.5t42.5 -17.5q7 0 37 8q122 33 251 33q279 0 488 -124q24 -13 38 -13q25 0 42.5 17.5t17.5 42.5zM1331 789q0 47 -40 70q-126 73 -293 110.5t-343 37.5q-204 0 -364 -47q-23 -7 -38.5 -25.5t-15.5 -48.5q0 -31 20.5 -52t51.5 -21q11 0 40 8q133 37 307 37 -q159 0 309.5 -34t253.5 -95q21 -12 40 -12q29 0 50.5 20.5t21.5 51.5zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="_418" unicode="" horiz-adv-x="1024" -d="M1024 1233l-303 -582l24 -31h279v-415h-507l-44 -30l-142 -273l-30 -30h-301v303l303 583l-24 30h-279v415h507l44 30l142 273l30 30h301v-303z" /> - <glyph glyph-name="_419" unicode="" horiz-adv-x="2304" -d="M784 164l16 241l-16 523q-1 10 -7.5 17t-16.5 7q-9 0 -16 -7t-7 -17l-14 -523l14 -241q1 -10 7.5 -16.5t15.5 -6.5q22 0 24 23zM1080 193l11 211l-12 586q0 16 -13 24q-8 5 -16 5t-16 -5q-13 -8 -13 -24l-1 -6l-10 -579q0 -1 11 -236v-1q0 -10 6 -17q9 -11 23 -11 -q11 0 20 9q9 7 9 20zM35 533l20 -128l-20 -126q-2 -9 -9 -9t-9 9l-17 126l17 128q2 9 9 9t9 -9zM121 612l26 -207l-26 -203q-2 -9 -10 -9q-9 0 -9 10l-23 202l23 207q0 9 9 9q8 0 10 -9zM401 159zM213 650l25 -245l-25 -237q0 -11 -11 -11q-10 0 -12 11l-21 237l21 245 -q2 12 12 12q11 0 11 -12zM307 657l23 -252l-23 -244q-2 -13 -14 -13q-13 0 -13 13l-21 244l21 252q0 13 13 13q12 0 14 -13zM401 639l21 -234l-21 -246q-2 -16 -16 -16q-6 0 -10.5 4.5t-4.5 11.5l-20 246l20 234q0 6 4.5 10.5t10.5 4.5q14 0 16 -15zM784 164zM495 785 -l21 -380l-21 -246q0 -7 -5 -12.5t-12 -5.5q-16 0 -18 18l-18 246l18 380q2 18 18 18q7 0 12 -5.5t5 -12.5zM589 871l19 -468l-19 -244q0 -8 -5.5 -13.5t-13.5 -5.5q-18 0 -20 19l-16 244l16 468q2 19 20 19q8 0 13.5 -5.5t5.5 -13.5zM687 911l18 -506l-18 -242 -q-2 -21 -22 -21q-19 0 -21 21l-16 242l16 506q0 9 6.5 15.5t14.5 6.5q9 0 15 -6.5t7 -15.5zM1079 169v0v0v0zM881 915l15 -510l-15 -239q0 -10 -7.5 -17.5t-17.5 -7.5t-17 7t-8 18l-14 239l14 510q0 11 7.5 18t17.5 7t17.5 -7t7.5 -18zM980 896l14 -492l-14 -236 -q0 -11 -8 -19t-19 -8t-19 8t-9 19l-12 236l12 492q1 12 9 20t19 8t18.5 -8t8.5 -20zM1192 404l-14 -231v0q0 -13 -9 -22t-22 -9t-22 9t-10 22l-6 114l-6 117l12 636v3q2 15 12 24q9 7 20 7q8 0 15 -5q14 -8 16 -26zM2304 423q0 -117 -83 -199.5t-200 -82.5h-786 -q-13 2 -22 11t-9 22v899q0 23 28 33q85 34 181 34q195 0 338 -131.5t160 -323.5q53 22 110 22q117 0 200 -83t83 -201z" /> - <glyph glyph-name="uniF1C0" unicode="" -d="M768 768q237 0 443 43t325 127v-170q0 -69 -103 -128t-280 -93.5t-385 -34.5t-385 34.5t-280 93.5t-103 128v170q119 -84 325 -127t443 -43zM768 0q237 0 443 43t325 127v-170q0 -69 -103 -128t-280 -93.5t-385 -34.5t-385 34.5t-280 93.5t-103 128v170q119 -84 325 -127 -t443 -43zM768 384q237 0 443 43t325 127v-170q0 -69 -103 -128t-280 -93.5t-385 -34.5t-385 34.5t-280 93.5t-103 128v170q119 -84 325 -127t443 -43zM768 1536q208 0 385 -34.5t280 -93.5t103 -128v-128q0 -69 -103 -128t-280 -93.5t-385 -34.5t-385 34.5t-280 93.5 -t-103 128v128q0 69 103 128t280 93.5t385 34.5z" /> - <glyph glyph-name="uniF1C1" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -M894 465q33 -26 84 -56q59 7 117 7q147 0 177 -49q16 -22 2 -52q0 -1 -1 -2l-2 -2v-1q-6 -38 -71 -38q-48 0 -115 20t-130 53q-221 -24 -392 -83q-153 -262 -242 -262q-15 0 -28 7l-24 12q-1 1 -6 5q-10 10 -6 36q9 40 56 91.5t132 96.5q14 9 23 -6q2 -2 2 -4q52 85 107 197 -q68 136 104 262q-24 82 -30.5 159.5t6.5 127.5q11 40 42 40h21h1q23 0 35 -15q18 -21 9 -68q-2 -6 -4 -8q1 -3 1 -8v-30q-2 -123 -14 -192q55 -164 146 -238zM318 54q52 24 137 158q-51 -40 -87.5 -84t-49.5 -74zM716 974q-15 -42 -2 -132q1 7 7 44q0 3 7 43q1 4 4 8 -q-1 1 -1 2q-1 2 -1 3q-1 22 -13 36q0 -1 -1 -2v-2zM592 313q135 54 284 81q-2 1 -13 9.5t-16 13.5q-76 67 -127 176q-27 -86 -83 -197q-30 -56 -45 -83zM1238 329q-24 24 -140 24q76 -28 124 -28q14 0 18 1q0 1 -2 3z" /> - <glyph glyph-name="_422" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -M233 768v-107h70l164 -661h159l128 485q7 20 10 46q2 16 2 24h4l3 -24q1 -3 3.5 -20t5.5 -26l128 -485h159l164 661h70v107h-300v-107h90l-99 -438q-5 -20 -7 -46l-2 -21h-4q0 3 -0.5 6.5t-1.5 8t-1 6.5q-1 5 -4 21t-5 25l-144 545h-114l-144 -545q-2 -9 -4.5 -24.5 -t-3.5 -21.5l-4 -21h-4l-2 21q-2 26 -7 46l-99 438h90v107h-300z" /> - <glyph glyph-name="_423" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -M429 106v-106h281v106h-75l103 161q5 7 10 16.5t7.5 13.5t3.5 4h2q1 -4 5 -10q2 -4 4.5 -7.5t6 -8t6.5 -8.5l107 -161h-76v-106h291v106h-68l-192 273l195 282h67v107h-279v-107h74l-103 -159q-4 -7 -10 -16.5t-9 -13.5l-2 -3h-2q-1 4 -5 10q-6 11 -17 23l-106 159h76v107 -h-290v-107h68l189 -272l-194 -283h-68z" /> - <glyph glyph-name="_424" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -M416 106v-106h327v106h-93v167h137q76 0 118 15q67 23 106.5 87t39.5 146q0 81 -37 141t-100 87q-48 19 -130 19h-368v-107h92v-555h-92zM769 386h-119v268h120q52 0 83 -18q56 -33 56 -115q0 -89 -62 -120q-31 -15 -78 -15z" /> - <glyph glyph-name="_425" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -M1280 320v-320h-1024v192l192 192l128 -128l384 384zM448 512q-80 0 -136 56t-56 136t56 136t136 56t136 -56t56 -136t-56 -136t-136 -56z" /> - <glyph glyph-name="_426" unicode="" -d="M640 1152v128h-128v-128h128zM768 1024v128h-128v-128h128zM640 896v128h-128v-128h128zM768 768v128h-128v-128h128zM1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400 -v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-128v-128h-128v128h-512v-1536h1280zM781 593l107 -349q8 -27 8 -52q0 -83 -72.5 -137.5t-183.5 -54.5t-183.5 54.5t-72.5 137.5q0 25 8 52q21 63 120 396v128h128v-128h79 -q22 0 39 -13t23 -34zM640 128q53 0 90.5 19t37.5 45t-37.5 45t-90.5 19t-90.5 -19t-37.5 -45t37.5 -45t90.5 -19z" /> - <glyph glyph-name="_427" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -M620 686q20 -8 20 -30v-544q0 -22 -20 -30q-8 -2 -12 -2q-12 0 -23 9l-166 167h-131q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h131l166 167q16 15 35 7zM1037 -3q31 0 50 24q129 159 129 363t-129 363q-16 21 -43 24t-47 -14q-21 -17 -23.5 -43.5t14.5 -47.5 -q100 -123 100 -282t-100 -282q-17 -21 -14.5 -47.5t23.5 -42.5q18 -15 40 -15zM826 145q27 0 47 20q87 93 87 219t-87 219q-18 19 -45 20t-46 -17t-20 -44.5t18 -46.5q52 -57 52 -131t-52 -131q-19 -20 -18 -46.5t20 -44.5q20 -17 44 -17z" /> - <glyph glyph-name="_428" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -M768 768q52 0 90 -38t38 -90v-384q0 -52 -38 -90t-90 -38h-384q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h384zM1260 766q20 -8 20 -30v-576q0 -22 -20 -30q-8 -2 -12 -2q-14 0 -23 9l-265 266v90l265 266q9 9 23 9q4 0 12 -2z" /> - <glyph glyph-name="_429" unicode="" -d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z -M480 768q8 11 21 12.5t24 -6.5l51 -38q11 -8 12.5 -21t-6.5 -24l-182 -243l182 -243q8 -11 6.5 -24t-12.5 -21l-51 -38q-11 -8 -24 -6.5t-21 12.5l-226 301q-14 19 0 38zM1282 467q14 -19 0 -38l-226 -301q-8 -11 -21 -12.5t-24 6.5l-51 38q-11 8 -12.5 21t6.5 24l182 243 -l-182 243q-8 11 -6.5 24t12.5 21l51 38q11 8 24 6.5t21 -12.5zM662 6q-13 2 -20.5 13t-5.5 24l138 831q2 13 13 20.5t24 5.5l63 -10q13 -2 20.5 -13t5.5 -24l-138 -831q-2 -13 -13 -20.5t-24 -5.5z" /> - <glyph glyph-name="_430" unicode="" -d="M1497 709v-198q-101 -23 -198 -23q-65 -136 -165.5 -271t-181.5 -215.5t-128 -106.5q-80 -45 -162 3q-28 17 -60.5 43.5t-85 83.5t-102.5 128.5t-107.5 184t-105.5 244t-91.5 314.5t-70.5 390h283q26 -218 70 -398.5t104.5 -317t121.5 -235.5t140 -195q169 169 287 406 -q-142 72 -223 220t-81 333q0 192 104 314.5t284 122.5q178 0 273 -105.5t95 -297.5q0 -159 -58 -286q-7 -1 -19.5 -3t-46 -2t-63 6t-62 25.5t-50.5 51.5q31 103 31 184q0 87 -29 132t-79 45q-53 0 -85 -49.5t-32 -140.5q0 -186 105 -293.5t267 -107.5q62 0 121 14z" /> - <glyph glyph-name="_431" unicode="" horiz-adv-x="1792" -d="M216 367l603 -402v359l-334 223zM154 511l193 129l-193 129v-258zM973 -35l603 402l-269 180l-334 -223v-359zM896 458l272 182l-272 182l-272 -182zM485 733l334 223v359l-603 -402zM1445 640l193 -129v258zM1307 733l269 180l-603 402v-359zM1792 913v-546 -q0 -41 -34 -64l-819 -546q-21 -13 -43 -13t-43 13l-819 546q-34 23 -34 64v546q0 41 34 64l819 546q21 13 43 13t43 -13l819 -546q34 -23 34 -64z" /> - <glyph glyph-name="_432" unicode="" horiz-adv-x="2048" -d="M1800 764q111 -46 179.5 -145.5t68.5 -221.5q0 -164 -118 -280.5t-285 -116.5q-4 0 -11.5 0.5t-10.5 0.5h-1209h-1h-2h-5q-170 10 -288 125.5t-118 280.5q0 110 55 203t147 147q-12 39 -12 82q0 115 82 196t199 81q95 0 172 -58q75 154 222.5 248t326.5 94 -q166 0 306 -80.5t221.5 -218.5t81.5 -301q0 -6 -0.5 -18t-0.5 -18zM468 498q0 -122 84 -193t208 -71q137 0 240 99q-16 20 -47.5 56.5t-43.5 50.5q-67 -65 -144 -65q-55 0 -93.5 33.5t-38.5 87.5q0 53 38.5 87t91.5 34q44 0 84.5 -21t73 -55t65 -75t69 -82t77 -75t97 -55 -t121.5 -21q121 0 204.5 71.5t83.5 190.5q0 121 -84 192t-207 71q-143 0 -241 -97l93 -108q66 64 142 64q52 0 92 -33t40 -84q0 -57 -37 -91.5t-94 -34.5q-43 0 -82.5 21t-72 55t-65.5 75t-69.5 82t-77.5 75t-96.5 55t-118.5 21q-122 0 -207 -70.5t-85 -189.5z" /> - <glyph glyph-name="_433" unicode="" horiz-adv-x="1792" -d="M896 1536q182 0 348 -71t286 -191t191 -286t71 -348t-71 -348t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71zM896 1408q-190 0 -361 -90l194 -194q82 28 167 28t167 -28l194 194q-171 90 -361 90zM218 279l194 194 -q-28 82 -28 167t28 167l-194 194q-90 -171 -90 -361t90 -361zM896 -128q190 0 361 90l-194 194q-82 -28 -167 -28t-167 28l-194 -194q171 -90 361 -90zM896 256q159 0 271.5 112.5t112.5 271.5t-112.5 271.5t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5 -t271.5 -112.5zM1380 473l194 -194q90 171 90 361t-90 361l-194 -194q28 -82 28 -167t-28 -167z" /> - <glyph glyph-name="_434" unicode="" horiz-adv-x="1792" -d="M1760 640q0 -176 -68.5 -336t-184 -275.5t-275.5 -184t-336 -68.5t-336 68.5t-275.5 184t-184 275.5t-68.5 336q0 213 97 398.5t265 305.5t374 151v-228q-221 -45 -366.5 -221t-145.5 -406q0 -130 51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5 -t136.5 204t51 248.5q0 230 -145.5 406t-366.5 221v228q206 -31 374 -151t265 -305.5t97 -398.5z" /> - <glyph glyph-name="uniF1D0" unicode="" horiz-adv-x="1792" -d="M19 662q8 217 116 406t305 318h5q0 -1 -1 -3q-8 -8 -28 -33.5t-52 -76.5t-60 -110.5t-44.5 -135.5t-14 -150.5t39 -157.5t108.5 -154q50 -50 102 -69.5t90.5 -11.5t69.5 23.5t47 32.5l16 16q39 51 53 116.5t6.5 122.5t-21 107t-26.5 80l-14 29q-10 25 -30.5 49.5t-43 41 -t-43.5 29.5t-35 19l-13 6l104 115q39 -17 78 -52t59 -61l19 -27q1 48 -18.5 103.5t-40.5 87.5l-20 31l161 183l160 -181q-33 -46 -52.5 -102.5t-22.5 -90.5l-4 -33q22 37 61.5 72.5t67.5 52.5l28 17l103 -115q-44 -14 -85 -50t-60 -65l-19 -29q-31 -56 -48 -133.5t-7 -170 -t57 -156.5q33 -45 77.5 -60.5t85 -5.5t76 26.5t57.5 33.5l21 16q60 53 96.5 115t48.5 121.5t10 121.5t-18 118t-37 107.5t-45.5 93t-45 72t-34.5 47.5l-13 17q-14 13 -7 13l10 -3q40 -29 62.5 -46t62 -50t64 -58t58.5 -65t55.5 -77t45.5 -88t38 -103t23.5 -117t10.5 -136 -q3 -259 -108 -465t-312 -321t-456 -115q-185 0 -351 74t-283.5 198t-184 293t-60.5 353z" /> - <glyph glyph-name="uniF1D1" unicode="" horiz-adv-x="1792" -d="M874 -102v-66q-208 6 -385 109.5t-283 275.5l58 34q29 -49 73 -99l65 57q148 -168 368 -212l-17 -86q65 -12 121 -13zM276 428l-83 -28q22 -60 49 -112l-57 -33q-98 180 -98 385t98 385l57 -33q-30 -56 -49 -112l82 -28q-35 -100 -35 -212q0 -109 36 -212zM1528 251 -l58 -34q-106 -172 -283 -275.5t-385 -109.5v66q56 1 121 13l-17 86q220 44 368 212l65 -57q44 50 73 99zM1377 805l-233 -80q14 -42 14 -85t-14 -85l232 -80q-31 -92 -98 -169l-185 162q-57 -67 -147 -85l48 -241q-52 -10 -98 -10t-98 10l48 241q-90 18 -147 85l-185 -162 -q-67 77 -98 169l232 80q-14 42 -14 85t14 85l-233 80q33 93 99 169l185 -162q59 68 147 86l-48 240q44 10 98 10t98 -10l-48 -240q88 -18 147 -86l185 162q66 -76 99 -169zM874 1448v-66q-65 -2 -121 -13l17 -86q-220 -42 -368 -211l-65 56q-38 -42 -73 -98l-57 33 -q106 172 282 275.5t385 109.5zM1705 640q0 -205 -98 -385l-57 33q27 52 49 112l-83 28q36 103 36 212q0 112 -35 212l82 28q-19 56 -49 112l57 33q98 -180 98 -385zM1585 1063l-57 -33q-35 56 -73 98l-65 -56q-148 169 -368 211l17 86q-56 11 -121 13v66q209 -6 385 -109.5 -t282 -275.5zM1748 640q0 173 -67.5 331t-181.5 272t-272 181.5t-331 67.5t-331 -67.5t-272 -181.5t-181.5 -272t-67.5 -331t67.5 -331t181.5 -272t272 -181.5t331 -67.5t331 67.5t272 181.5t181.5 272t67.5 331zM1792 640q0 -182 -71 -348t-191 -286t-286 -191t-348 -71 -t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="uniF1D2" unicode="" -d="M582 228q0 -66 -93 -66q-107 0 -107 63q0 64 98 64q102 0 102 -61zM546 694q0 -85 -74 -85q-77 0 -77 84q0 90 77 90q36 0 55 -25.5t19 -63.5zM712 769v125q-78 -29 -135 -29q-50 29 -110 29q-86 0 -145 -57t-59 -143q0 -50 29.5 -102t73.5 -67v-3q-38 -17 -38 -85 -q0 -53 41 -77v-3q-113 -37 -113 -139q0 -45 20 -78.5t54 -51t72 -25.5t81 -8q224 0 224 188q0 67 -48 99t-126 46q-27 5 -51.5 20.5t-24.5 39.5q0 44 49 52q77 15 122 70t45 134q0 24 -10 52q37 9 49 13zM771 350h137q-2 27 -2 82v387q0 46 2 69h-137q3 -23 3 -71v-392 -q0 -50 -3 -75zM1280 366v121q-30 -21 -68 -21q-53 0 -53 82v225h52q9 0 26.5 -1t26.5 -1v117h-105q0 82 3 102h-140q4 -24 4 -55v-47h-60v-117q36 3 37 3q3 0 11 -0.5t12 -0.5v-2h-2v-217q0 -37 2.5 -64t11.5 -56.5t24.5 -48.5t43.5 -31t66 -12q64 0 108 24zM924 1072 -q0 36 -24 63.5t-60 27.5t-60.5 -27t-24.5 -64q0 -36 25 -62.5t60 -26.5t59.5 27t24.5 62zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_438" unicode="" horiz-adv-x="1792" -d="M595 22q0 100 -165 100q-158 0 -158 -104q0 -101 172 -101q151 0 151 105zM536 777q0 61 -30 102t-89 41q-124 0 -124 -145q0 -135 124 -135q119 0 119 137zM805 1101v-202q-36 -12 -79 -22q16 -43 16 -84q0 -127 -73 -216.5t-197 -112.5q-40 -8 -59.5 -27t-19.5 -58 -q0 -31 22.5 -51.5t58 -32t78.5 -22t86 -25.5t78.5 -37.5t58 -64t22.5 -98.5q0 -304 -363 -304q-69 0 -130 12.5t-116 41t-87.5 82t-32.5 127.5q0 165 182 225v4q-67 41 -67 126q0 109 63 137v4q-72 24 -119.5 108.5t-47.5 165.5q0 139 95 231.5t235 92.5q96 0 178 -47 -q98 0 218 47zM1123 220h-222q4 45 4 134v609q0 94 -4 128h222q-4 -33 -4 -124v-613q0 -89 4 -134zM1724 442v-196q-71 -39 -174 -39q-62 0 -107 20t-70 50t-39.5 78t-18.5 92t-4 103v351h2v4q-7 0 -19 1t-18 1q-21 0 -59 -6v190h96v76q0 54 -6 89h227q-6 -41 -6 -165h171 -v-190q-15 0 -43.5 2t-42.5 2h-85v-365q0 -131 87 -131q61 0 109 33zM1148 1389q0 -58 -39 -101.5t-96 -43.5q-58 0 -98 43.5t-40 101.5q0 59 39.5 103t98.5 44q58 0 96.5 -44.5t38.5 -102.5z" /> - <glyph glyph-name="_439" unicode="" -d="M809 532l266 499h-112l-157 -312q-24 -48 -44 -92l-42 92l-155 312h-120l263 -493v-324h101v318zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="uniF1D5" unicode="" horiz-adv-x="1280" -d="M842 964q0 -80 -57 -136.5t-136 -56.5q-60 0 -111 35q-62 -67 -115 -146q-247 -371 -202 -859q1 -22 -12.5 -38.5t-34.5 -18.5h-5q-20 0 -35 13.5t-17 33.5q-14 126 -3.5 247.5t29.5 217t54 186t69 155.5t74 125q61 90 132 165q-16 35 -16 77q0 80 56.5 136.5t136.5 56.5 -t136.5 -56.5t56.5 -136.5zM1223 953q0 -158 -78 -292t-212.5 -212t-292.5 -78q-64 0 -131 14q-21 5 -32.5 23.5t-6.5 39.5q5 20 23 31.5t39 7.5q51 -13 108 -13q97 0 186 38t153 102t102 153t38 186t-38 186t-102 153t-153 102t-186 38t-186 -38t-153 -102t-102 -153 -t-38 -186q0 -114 52 -218q10 -20 3.5 -40t-25.5 -30t-39.5 -3t-30.5 26q-64 123 -64 265q0 119 46.5 227t124.5 186t186 124t226 46q158 0 292.5 -78t212.5 -212.5t78 -292.5z" /> - <glyph glyph-name="uniF1D6" unicode="" horiz-adv-x="1792" -d="M270 730q-8 19 -8 52q0 20 11 49t24 45q-1 22 7.5 53t22.5 43q0 139 92.5 288.5t217.5 209.5q139 66 324 66q133 0 266 -55q49 -21 90 -48t71 -56t55 -68t42 -74t32.5 -84.5t25.5 -89.5t22 -98l1 -5q55 -83 55 -150q0 -14 -9 -40t-9 -38q0 -1 1.5 -3.5t3.5 -5t2 -3.5 -q77 -114 120.5 -214.5t43.5 -208.5q0 -43 -19.5 -100t-55.5 -57q-9 0 -19.5 7.5t-19 17.5t-19 26t-16 26.5t-13.5 26t-9 17.5q-1 1 -3 1l-5 -4q-59 -154 -132 -223q20 -20 61.5 -38.5t69 -41.5t35.5 -65q-2 -4 -4 -16t-7 -18q-64 -97 -302 -97q-53 0 -110.5 9t-98 20 -t-104.5 30q-15 5 -23 7q-14 4 -46 4.5t-40 1.5q-41 -45 -127.5 -65t-168.5 -20q-35 0 -69 1.5t-93 9t-101 20.5t-74.5 40t-32.5 64q0 40 10 59.5t41 48.5q11 2 40.5 13t49.5 12q4 0 14 2q2 2 2 4l-2 3q-48 11 -108 105.5t-73 156.5l-5 3q-4 0 -12 -20q-18 -41 -54.5 -74.5 -t-77.5 -37.5h-1q-4 0 -6 4.5t-5 5.5q-23 54 -23 100q0 275 252 466z" /> - <glyph glyph-name="uniF1D7" unicode="" horiz-adv-x="2048" -d="M580 1075q0 41 -25 66t-66 25q-43 0 -76 -25.5t-33 -65.5q0 -39 33 -64.5t76 -25.5q41 0 66 24.5t25 65.5zM1323 568q0 28 -25.5 50t-65.5 22q-27 0 -49.5 -22.5t-22.5 -49.5q0 -28 22.5 -50.5t49.5 -22.5q40 0 65.5 22t25.5 51zM1087 1075q0 41 -24.5 66t-65.5 25 -q-43 0 -76 -25.5t-33 -65.5q0 -39 33 -64.5t76 -25.5q41 0 65.5 24.5t24.5 65.5zM1722 568q0 28 -26 50t-65 22q-27 0 -49.5 -22.5t-22.5 -49.5q0 -28 22.5 -50.5t49.5 -22.5q39 0 65 22t26 51zM1456 965q-31 4 -70 4q-169 0 -311 -77t-223.5 -208.5t-81.5 -287.5 -q0 -78 23 -152q-35 -3 -68 -3q-26 0 -50 1.5t-55 6.5t-44.5 7t-54.5 10.5t-50 10.5l-253 -127l72 218q-290 203 -290 490q0 169 97.5 311t264 223.5t363.5 81.5q176 0 332.5 -66t262 -182.5t136.5 -260.5zM2048 404q0 -117 -68.5 -223.5t-185.5 -193.5l55 -181l-199 109 -q-150 -37 -218 -37q-169 0 -311 70.5t-223.5 191.5t-81.5 264t81.5 264t223.5 191.5t311 70.5q161 0 303 -70.5t227.5 -192t85.5 -263.5z" /> - <glyph glyph-name="_443" unicode="" horiz-adv-x="1792" -d="M1764 1525q33 -24 27 -64l-256 -1536q-5 -29 -32 -45q-14 -8 -31 -8q-11 0 -24 5l-453 185l-242 -295q-18 -23 -49 -23q-13 0 -22 4q-19 7 -30.5 23.5t-11.5 36.5v349l864 1059l-1069 -925l-395 162q-37 14 -40 55q-2 40 32 59l1664 960q15 9 32 9q20 0 36 -11z" /> - <glyph glyph-name="_444" unicode="" horiz-adv-x="1792" -d="M1764 1525q33 -24 27 -64l-256 -1536q-5 -29 -32 -45q-14 -8 -31 -8q-11 0 -24 5l-527 215l-298 -327q-18 -21 -47 -21q-14 0 -23 4q-19 7 -30 23.5t-11 36.5v452l-472 193q-37 14 -40 55q-3 39 32 59l1664 960q35 21 68 -2zM1422 26l221 1323l-1434 -827l336 -137 -l863 639l-478 -797z" /> - <glyph glyph-name="_445" unicode="" -d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61q-172 0 -327 72.5t-264 204.5q-7 10 -6.5 22.5t8.5 20.5l137 138q10 9 25 9q16 -2 23 -12q73 -95 179 -147t225 -52q104 0 198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5t-40.5 198.5t-109.5 163.5 -t-163.5 109.5t-198.5 40.5q-98 0 -188 -35.5t-160 -101.5l137 -138q31 -30 14 -69q-17 -40 -59 -40h-448q-26 0 -45 19t-19 45v448q0 42 40 59q39 17 69 -14l130 -129q107 101 244.5 156.5t284.5 55.5q156 0 298 -61t245 -164t164 -245t61 -298zM896 928v-448q0 -14 -9 -23 -t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23z" /> - <glyph glyph-name="_446" unicode="" -d="M768 1280q-130 0 -248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5t-51 248.5t-136.5 204t-204 136.5t-248.5 51zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 -t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="_447" unicode="" horiz-adv-x="1792" -d="M1682 -128q-44 0 -132.5 3.5t-133.5 3.5q-44 0 -132 -3.5t-132 -3.5q-24 0 -37 20.5t-13 45.5q0 31 17 46t39 17t51 7t45 15q33 21 33 140l-1 391q0 21 -1 31q-13 4 -50 4h-675q-38 0 -51 -4q-1 -10 -1 -31l-1 -371q0 -142 37 -164q16 -10 48 -13t57 -3.5t45 -15 -t20 -45.5q0 -26 -12.5 -48t-36.5 -22q-47 0 -139.5 3.5t-138.5 3.5q-43 0 -128 -3.5t-127 -3.5q-23 0 -35.5 21t-12.5 45q0 30 15.5 45t36 17.5t47.5 7.5t42 15q33 23 33 143l-1 57v813q0 3 0.5 26t0 36.5t-1.5 38.5t-3.5 42t-6.5 36.5t-11 31.5t-16 18q-15 10 -45 12t-53 2 -t-41 14t-18 45q0 26 12 48t36 22q46 0 138.5 -3.5t138.5 -3.5q42 0 126.5 3.5t126.5 3.5q25 0 37.5 -22t12.5 -48q0 -30 -17 -43.5t-38.5 -14.5t-49.5 -4t-43 -13q-35 -21 -35 -160l1 -320q0 -21 1 -32q13 -3 39 -3h699q25 0 38 3q1 11 1 32l1 320q0 139 -35 160 -q-18 11 -58.5 12.5t-66 13t-25.5 49.5q0 26 12.5 48t37.5 22q44 0 132 -3.5t132 -3.5q43 0 129 3.5t129 3.5q25 0 37.5 -22t12.5 -48q0 -30 -17.5 -44t-40 -14.5t-51.5 -3t-44 -12.5q-35 -23 -35 -161l1 -943q0 -119 34 -140q16 -10 46 -13.5t53.5 -4.5t41.5 -15.5t18 -44.5 -q0 -26 -12 -48t-36 -22z" /> - <glyph glyph-name="_448" unicode="" horiz-adv-x="1280" -d="M1278 1347v-73q0 -29 -18.5 -61t-42.5 -32q-50 0 -54 -1q-26 -6 -32 -31q-3 -11 -3 -64v-1152q0 -25 -18 -43t-43 -18h-108q-25 0 -43 18t-18 43v1218h-143v-1218q0 -25 -17.5 -43t-43.5 -18h-108q-26 0 -43.5 18t-17.5 43v496q-147 12 -245 59q-126 58 -192 179 -q-64 117 -64 259q0 166 88 286q88 118 209 159q111 37 417 37h479q25 0 43 -18t18 -43z" /> - <glyph glyph-name="_449" unicode="" -d="M352 128v-128h-352v128h352zM704 256q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h256zM864 640v-128h-864v128h864zM224 1152v-128h-224v128h224zM1536 128v-128h-736v128h736zM576 1280q26 0 45 -19t19 -45v-256 -q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h256zM1216 768q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h256zM1536 640v-128h-224v128h224zM1536 1152v-128h-864v128h864z" /> - <glyph glyph-name="uniF1E0" unicode="" -d="M1216 512q133 0 226.5 -93.5t93.5 -226.5t-93.5 -226.5t-226.5 -93.5t-226.5 93.5t-93.5 226.5q0 12 2 34l-360 180q-92 -86 -218 -86q-133 0 -226.5 93.5t-93.5 226.5t93.5 226.5t226.5 93.5q126 0 218 -86l360 180q-2 22 -2 34q0 133 93.5 226.5t226.5 93.5 -t226.5 -93.5t93.5 -226.5t-93.5 -226.5t-226.5 -93.5q-126 0 -218 86l-360 -180q2 -22 2 -34t-2 -34l360 -180q92 86 218 86z" /> - <glyph glyph-name="_451" unicode="" -d="M1280 341q0 88 -62.5 151t-150.5 63q-84 0 -145 -58l-241 120q2 16 2 23t-2 23l241 120q61 -58 145 -58q88 0 150.5 63t62.5 151t-62.5 150.5t-150.5 62.5t-151 -62.5t-63 -150.5q0 -7 2 -23l-241 -120q-62 57 -145 57q-88 0 -150.5 -62.5t-62.5 -150.5t62.5 -150.5 -t150.5 -62.5q83 0 145 57l241 -120q-2 -16 -2 -23q0 -88 63 -150.5t151 -62.5t150.5 62.5t62.5 150.5zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_452" unicode="" horiz-adv-x="1792" -d="M571 947q-10 25 -34 35t-49 0q-108 -44 -191 -127t-127 -191q-10 -25 0 -49t35 -34q13 -5 24 -5q42 0 60 40q34 84 98.5 148.5t148.5 98.5q25 11 35 35t0 49zM1513 1303l46 -46l-244 -243l68 -68q19 -19 19 -45.5t-19 -45.5l-64 -64q89 -161 89 -343q0 -143 -55.5 -273.5 -t-150 -225t-225 -150t-273.5 -55.5t-273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5q182 0 343 -89l64 64q19 19 45.5 19t45.5 -19l68 -68zM1521 1359q-10 -10 -22 -10q-13 0 -23 10l-91 90q-9 10 -9 23t9 23q10 9 23 9t23 -9l90 -91 -q10 -9 10 -22.5t-10 -22.5zM1751 1129q-11 -9 -23 -9t-23 9l-90 91q-10 9 -10 22.5t10 22.5q9 10 22.5 10t22.5 -10l91 -90q9 -10 9 -23t-9 -23zM1792 1312q0 -14 -9 -23t-23 -9h-96q-14 0 -23 9t-9 23t9 23t23 9h96q14 0 23 -9t9 -23zM1600 1504v-96q0 -14 -9 -23t-23 -9 -t-23 9t-9 23v96q0 14 9 23t23 9t23 -9t9 -23zM1751 1449l-91 -90q-10 -10 -22 -10q-13 0 -23 10q-10 9 -10 22.5t10 22.5l90 91q10 9 23 9t23 -9q9 -10 9 -23t-9 -23z" /> - <glyph glyph-name="_453" unicode="" horiz-adv-x="1792" -d="M609 720l287 208l287 -208l-109 -336h-355zM896 1536q182 0 348 -71t286 -191t191 -286t71 -348t-71 -348t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71zM1515 186q149 203 149 454v3l-102 -89l-240 224l63 323 -l134 -12q-150 206 -389 282l53 -124l-287 -159l-287 159l53 124q-239 -76 -389 -282l135 12l62 -323l-240 -224l-102 89v-3q0 -251 149 -454l30 132l326 -40l139 -298l-116 -69q117 -39 240 -39t240 39l-116 69l139 298l326 40z" /> - <glyph glyph-name="_454" unicode="" horiz-adv-x="1792" -d="M448 224v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM256 608v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM832 224v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23 -v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM640 608v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM66 768q-28 0 -47 19t-19 46v129h514v-129q0 -27 -19 -46t-46 -19h-383zM1216 224v-192q0 -14 -9 -23t-23 -9h-192 -q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1024 608v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1600 224v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23 -zM1408 608v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 1016v-13h-514v10q0 104 -382 102q-382 -1 -382 -102v-10h-514v13q0 17 8.5 43t34 64t65.5 75.5t110.5 76t160 67.5t224 47.5t293.5 18.5t293 -18.5t224 -47.5 -t160.5 -67.5t110.5 -76t65.5 -75.5t34 -64t8.5 -43zM1792 608v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 962v-129q0 -27 -19 -46t-46 -19h-384q-27 0 -46 19t-19 46v129h514z" /> - <glyph glyph-name="_455" unicode="" horiz-adv-x="1792" -d="M704 1216v-768q0 -26 -19 -45t-45 -19v-576q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v512l249 873q7 23 31 23h424zM1024 1216v-704h-256v704h256zM1792 320v-512q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v576q-26 0 -45 19t-19 45v768h424q24 0 31 -23z -M736 1504v-224h-352v224q0 14 9 23t23 9h288q14 0 23 -9t9 -23zM1408 1504v-224h-352v224q0 14 9 23t23 9h288q14 0 23 -9t9 -23z" /> - <glyph glyph-name="_456" unicode="" horiz-adv-x="1792" -d="M1755 1083q37 -38 37 -90.5t-37 -90.5l-401 -400l150 -150l-160 -160q-163 -163 -389.5 -186.5t-411.5 100.5l-362 -362h-181v181l362 362q-124 185 -100.5 411.5t186.5 389.5l160 160l150 -150l400 401q38 37 91 37t90 -37t37 -90.5t-37 -90.5l-400 -401l234 -234 -l401 400q38 37 91 37t90 -37z" /> - <glyph glyph-name="_457" unicode="" horiz-adv-x="1792" -d="M873 796q0 -83 -63.5 -142.5t-152.5 -59.5t-152.5 59.5t-63.5 142.5q0 84 63.5 143t152.5 59t152.5 -59t63.5 -143zM1375 796q0 -83 -63 -142.5t-153 -59.5q-89 0 -152.5 59.5t-63.5 142.5q0 84 63.5 143t152.5 59q90 0 153 -59t63 -143zM1600 616v667q0 87 -32 123.5 -t-111 36.5h-1112q-83 0 -112.5 -34t-29.5 -126v-673q43 -23 88.5 -40t81 -28t81 -18.5t71 -11t70 -4t58.5 -0.5t56.5 2t44.5 2q68 1 95 -27q6 -6 10 -9q26 -25 61 -51q7 91 118 87q5 0 36.5 -1.5t43 -2t45.5 -1t53 1t54.5 4.5t61 8.5t62 13.5t67 19.5t67.5 27t72 34.5z -M1763 621q-121 -149 -372 -252q84 -285 -23 -465q-66 -113 -183 -148q-104 -32 -182 15q-86 51 -82 164l-1 326v1q-8 2 -24.5 6t-23.5 5l-1 -338q4 -114 -83 -164q-79 -47 -183 -15q-117 36 -182 150q-105 180 -22 463q-251 103 -372 252q-25 37 -4 63t60 -1q4 -2 11.5 -7 -t10.5 -8v694q0 72 47 123t114 51h1257q67 0 114 -51t47 -123v-694l21 15q39 27 60 1t-4 -63z" /> - <glyph glyph-name="_458" unicode="" horiz-adv-x="1792" -d="M896 1102v-434h-145v434h145zM1294 1102v-434h-145v434h145zM1294 342l253 254v795h-1194v-1049h326v-217l217 217h398zM1692 1536v-1013l-434 -434h-326l-217 -217h-217v217h-398v1158l109 289h1483z" /> - <glyph glyph-name="_459" unicode="" -d="M773 217v-127q-1 -292 -6 -305q-12 -32 -51 -40q-54 -9 -181.5 38t-162.5 89q-13 15 -17 36q-1 12 4 26q4 10 34 47t181 216q1 0 60 70q15 19 39.5 24.5t49.5 -3.5q24 -10 37.5 -29t12.5 -42zM624 468q-3 -55 -52 -70l-120 -39q-275 -88 -292 -88q-35 2 -54 36 -q-12 25 -17 75q-8 76 1 166.5t30 124.5t56 32q13 0 202 -77q71 -29 115 -47l84 -34q23 -9 35.5 -30.5t11.5 -48.5zM1450 171q-7 -54 -91.5 -161t-135.5 -127q-37 -14 -63 7q-14 10 -184 287l-47 77q-14 21 -11.5 46t19.5 46q35 43 83 26q1 -1 119 -40q203 -66 242 -79.5 -t47 -20.5q28 -22 22 -61zM778 803q5 -102 -54 -122q-58 -17 -114 71l-378 598q-8 35 19 62q41 43 207.5 89.5t224.5 31.5q40 -10 49 -45q3 -18 22 -305.5t24 -379.5zM1440 695q3 -39 -26 -59q-15 -10 -329 -86q-67 -15 -91 -23l1 2q-23 -6 -46 4t-37 32q-30 47 0 87 -q1 1 75 102q125 171 150 204t34 39q28 19 65 2q48 -23 123 -133.5t81 -167.5v-3z" /> - <glyph glyph-name="_460" unicode="" horiz-adv-x="2048" -d="M1024 1024h-384v-384h384v384zM1152 384v-128h-640v128h640zM1152 1152v-640h-640v640h640zM1792 384v-128h-512v128h512zM1792 640v-128h-512v128h512zM1792 896v-128h-512v128h512zM1792 1152v-128h-512v128h512zM256 192v960h-128v-960q0 -26 19 -45t45 -19t45 19 -t19 45zM1920 192v1088h-1536v-1088q0 -33 -11 -64h1483q26 0 45 19t19 45zM2048 1408v-1216q0 -80 -56 -136t-136 -56h-1664q-80 0 -136 56t-56 136v1088h256v128h1792z" /> - <glyph glyph-name="_461" unicode="" horiz-adv-x="2048" -d="M1024 13q-20 0 -93 73.5t-73 93.5q0 32 62.5 54t103.5 22t103.5 -22t62.5 -54q0 -20 -73 -93.5t-93 -73.5zM1294 284q-2 0 -40 25t-101.5 50t-128.5 25t-128.5 -25t-101 -50t-40.5 -25q-18 0 -93.5 75t-75.5 93q0 13 10 23q78 77 196 121t233 44t233 -44t196 -121 -q10 -10 10 -23q0 -18 -75.5 -93t-93.5 -75zM1567 556q-11 0 -23 8q-136 105 -252 154.5t-268 49.5q-85 0 -170.5 -22t-149 -53t-113.5 -62t-79 -53t-31 -22q-17 0 -92 75t-75 93q0 12 10 22q132 132 320 205t380 73t380 -73t320 -205q10 -10 10 -22q0 -18 -75 -93t-92 -75z -M1838 827q-11 0 -22 9q-179 157 -371.5 236.5t-420.5 79.5t-420.5 -79.5t-371.5 -236.5q-11 -9 -22 -9q-17 0 -92.5 75t-75.5 93q0 13 10 23q187 186 445 288t527 102t527 -102t445 -288q10 -10 10 -23q0 -18 -75.5 -93t-92.5 -75z" /> - <glyph glyph-name="_462" unicode="" horiz-adv-x="1792" -d="M384 0q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM768 0q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM384 384q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5 -t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1152 0q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM768 384q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5 -t37.5 90.5zM384 768q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1152 384q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM768 768q0 53 -37.5 90.5t-90.5 37.5 -t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1536 0v384q0 52 -38 90t-90 38t-90 -38t-38 -90v-384q0 -52 38 -90t90 -38t90 38t38 90zM1152 768q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5z -M1536 1088v256q0 26 -19 45t-45 19h-1280q-26 0 -45 -19t-19 -45v-256q0 -26 19 -45t45 -19h1280q26 0 45 19t19 45zM1536 768q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1408v-1536q0 -52 -38 -90t-90 -38 -h-1408q-52 0 -90 38t-38 90v1536q0 52 38 90t90 38h1408q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_463" unicode="" -d="M1519 890q18 -84 -4 -204q-87 -444 -565 -444h-44q-25 0 -44 -16.5t-24 -42.5l-4 -19l-55 -346l-2 -15q-5 -26 -24.5 -42.5t-44.5 -16.5h-251q-21 0 -33 15t-9 36q9 56 26.5 168t26.5 168t27 167.5t27 167.5q5 37 43 37h131q133 -2 236 21q175 39 287 144q102 95 155 246 -q24 70 35 133q1 6 2.5 7.5t3.5 1t6 -3.5q79 -59 98 -162zM1347 1172q0 -107 -46 -236q-80 -233 -302 -315q-113 -40 -252 -42q0 -1 -90 -1l-90 1q-100 0 -118 -96q-2 -8 -85 -530q-1 -10 -12 -10h-295q-22 0 -36.5 16.5t-11.5 38.5l232 1471q5 29 27.5 48t51.5 19h598 -q34 0 97.5 -13t111.5 -32q107 -41 163.5 -123t56.5 -196z" /> - <glyph glyph-name="_464" unicode="" horiz-adv-x="1792" -d="M441 864q33 0 52 -26q266 -364 362 -774h-446q-127 441 -367 749q-12 16 -3 33.5t29 17.5h373zM1000 507q-49 -199 -125 -393q-79 310 -256 594q40 221 44 449q211 -340 337 -650zM1099 1216q235 -324 384.5 -698.5t184.5 -773.5h-451q-41 665 -553 1472h435zM1792 640 -q0 -424 -101 -812q-67 560 -359 1083q-25 301 -106 584q-4 16 5.5 28.5t25.5 12.5h359q21 0 38.5 -13t22.5 -33q115 -409 115 -850z" /> - <glyph glyph-name="uniF1F0" unicode="" horiz-adv-x="2304" -d="M1975 546h-138q14 37 66 179l3 9q4 10 10 26t9 26l12 -55zM531 611l-58 295q-11 54 -75 54h-268l-2 -13q311 -79 403 -336zM710 960l-162 -438l-17 89q-26 70 -85 129.5t-131 88.5l135 -510h175l261 641h-176zM849 318h166l104 642h-166zM1617 944q-69 27 -149 27 -q-123 0 -201 -59t-79 -153q-1 -102 145 -174q48 -23 67 -41t19 -39q0 -30 -30 -46t-69 -16q-86 0 -156 33l-22 11l-23 -144q74 -34 185 -34q130 -1 208.5 59t80.5 160q0 106 -140 174q-49 25 -71 42t-22 38q0 22 24.5 38.5t70.5 16.5q70 1 124 -24l15 -8zM2042 960h-128 -q-65 0 -87 -54l-246 -588h174l35 96h212q5 -22 20 -96h154zM2304 1280v-1280q0 -52 -38 -90t-90 -38h-2048q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h2048q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_466" unicode="" horiz-adv-x="2304" -d="M1119 1195q-128 85 -281 85q-103 0 -197.5 -40.5t-162.5 -108.5t-108.5 -162t-40.5 -197q0 -104 40.5 -198t108.5 -162t162 -108.5t198 -40.5q153 0 281 85q-131 107 -178 265.5t0.5 316.5t177.5 265zM1152 1171q-126 -99 -172 -249.5t-0.5 -300.5t172.5 -249 -q127 99 172.5 249t-0.5 300.5t-172 249.5zM1185 1195q130 -107 177.5 -265.5t0.5 -317t-178 -264.5q128 -85 281 -85q104 0 198 40.5t162 108.5t108.5 162t40.5 198q0 103 -40.5 197t-108.5 162t-162.5 108.5t-197.5 40.5q-153 0 -281 -85zM1926 473h7v3h-17v-3h7v-17h3v17z -M1955 456h4v20h-5l-6 -13l-6 13h-5v-20h3v15l6 -13h4l5 13v-15zM1947 16v-2h-2h-3v3h3h2v-1zM1947 7h3l-4 5h2l1 1q1 1 1 3t-1 3l-1 1h-3h-6v-13h3v5h1zM685 75q0 19 11 31t30 12q18 0 29 -12.5t11 -30.5q0 -19 -11 -31t-29 -12q-19 0 -30 12t-11 31zM1158 119q30 0 35 -32 -h-70q5 32 35 32zM1514 75q0 19 11 31t29 12t29.5 -12.5t11.5 -30.5q0 -19 -11 -31t-30 -12q-18 0 -29 12t-11 31zM1786 75q0 18 11.5 30.5t29.5 12.5t29.5 -12.5t11.5 -30.5q0 -19 -11.5 -31t-29.5 -12t-29.5 12.5t-11.5 30.5zM1944 3q-2 0 -4 1q-1 0 -3 2t-2 3q-1 2 -1 4 -q0 3 1 4q0 2 2 4l1 1q2 0 2 1q2 1 4 1q3 0 4 -1l4 -2l2 -4v-1q1 -2 1 -3l-1 -1v-3t-1 -1l-1 -2q-2 -2 -4 -2q-1 -1 -4 -1zM599 7h30v85q0 24 -14.5 38.5t-39.5 15.5q-32 0 -47 -24q-14 24 -45 24q-24 0 -39 -20v16h-30v-135h30v75q0 36 33 36q30 0 30 -36v-75h29v75 -q0 36 33 36q30 0 30 -36v-75zM765 7h29v68v67h-29v-16q-17 20 -43 20q-29 0 -48 -20t-19 -51t19 -51t48 -20q28 0 43 20v-17zM943 48q0 34 -47 40l-14 2q-23 4 -23 14q0 15 25 15q23 0 43 -11l12 24q-22 14 -55 14q-26 0 -41 -12t-15 -32q0 -33 47 -39l13 -2q24 -4 24 -14 -q0 -17 -31 -17q-25 0 -45 14l-13 -23q25 -17 58 -17q29 0 45.5 12t16.5 32zM1073 14l-8 25q-13 -7 -26 -7q-19 0 -19 22v61h48v27h-48v41h-30v-41h-28v-27h28v-61q0 -50 47 -50q21 0 36 10zM1159 146q-29 0 -48 -20t-19 -51q0 -32 19.5 -51.5t49.5 -19.5q33 0 55 19l-14 22 -q-18 -15 -39 -15q-34 0 -41 33h101v12q0 32 -18 51.5t-46 19.5zM1318 146q-23 0 -35 -20v16h-30v-135h30v76q0 35 29 35q10 0 18 -4l9 28q-9 4 -21 4zM1348 75q0 -31 19.5 -51t52.5 -20q29 0 48 16l-14 24q-18 -13 -35 -12q-18 0 -29.5 12t-11.5 31t11.5 31t29.5 12 -q19 0 35 -12l14 24q-20 16 -48 16q-33 0 -52.5 -20t-19.5 -51zM1593 7h30v68v67h-30v-16q-15 20 -42 20q-29 0 -48.5 -20t-19.5 -51t19.5 -51t48.5 -20q28 0 42 20v-17zM1726 146q-23 0 -35 -20v16h-29v-135h29v76q0 35 29 35q10 0 18 -4l9 28q-8 4 -21 4zM1866 7h29v68v122 -h-29v-71q-15 20 -43 20t-47.5 -20.5t-19.5 -50.5t19.5 -50.5t47.5 -20.5q29 0 43 20v-17zM1944 27l-2 -1h-3q-2 -1 -4 -3q-3 -1 -3 -4q-1 -2 -1 -6q0 -3 1 -5q0 -2 3 -4q2 -2 4 -3t5 -1q4 0 6 1q0 1 2 2l2 1q1 1 3 4q1 2 1 5q0 4 -1 6q-1 1 -3 4q0 1 -2 2l-2 1q-1 0 -3 0.5 -t-3 0.5zM2304 1280v-1280q0 -52 -38 -90t-90 -38h-2048q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h2048q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_467" unicode="" horiz-adv-x="2304" -d="M313 759q0 -51 -36 -84q-29 -26 -89 -26h-17v220h17q61 0 89 -27q36 -31 36 -83zM2089 824q0 -52 -64 -52h-19v101h20q63 0 63 -49zM380 759q0 74 -50 120.5t-129 46.5h-95v-333h95q74 0 119 38q60 51 60 128zM410 593h65v333h-65v-333zM730 694q0 40 -20.5 62t-75.5 42 -q-29 10 -39.5 19t-10.5 23q0 16 13.5 26.5t34.5 10.5q29 0 53 -27l34 44q-41 37 -98 37q-44 0 -74 -27.5t-30 -67.5q0 -35 18 -55.5t64 -36.5q37 -13 45 -19q19 -12 19 -34q0 -20 -14 -33.5t-36 -13.5q-48 0 -71 44l-42 -40q44 -64 115 -64q51 0 83 30.5t32 79.5zM1008 604 -v77q-37 -37 -78 -37q-49 0 -80.5 32.5t-31.5 82.5q0 48 31.5 81.5t77.5 33.5q43 0 81 -38v77q-40 20 -80 20q-74 0 -125.5 -50.5t-51.5 -123.5t51 -123.5t125 -50.5q42 0 81 19zM2240 0v527q-65 -40 -144.5 -84t-237.5 -117t-329.5 -137.5t-417.5 -134.5t-504 -118h1569 -q26 0 45 19t19 45zM1389 757q0 75 -53 128t-128 53t-128 -53t-53 -128t53 -128t128 -53t128 53t53 128zM1541 584l144 342h-71l-90 -224l-89 224h-71l142 -342h35zM1714 593h184v56h-119v90h115v56h-115v74h119v57h-184v-333zM2105 593h80l-105 140q76 16 76 94q0 47 -31 73 -t-87 26h-97v-333h65v133h9zM2304 1274v-1268q0 -56 -38.5 -95t-93.5 -39h-2040q-55 0 -93.5 39t-38.5 95v1268q0 56 38.5 95t93.5 39h2040q55 0 93.5 -39t38.5 -95z" /> - <glyph glyph-name="f1f3" unicode="" horiz-adv-x="2304" -d="M119 854h89l-45 108zM740 328l74 79l-70 79h-163v-49h142v-55h-142v-54h159zM898 406l99 -110v217zM1186 453q0 33 -40 33h-84v-69h83q41 0 41 36zM1475 457q0 29 -42 29h-82v-61h81q43 0 43 32zM1197 923q0 29 -42 29h-82v-60h81q43 0 43 31zM1656 854h89l-44 108z -M699 1009v-271h-66v212l-94 -212h-57l-94 212v-212h-132l-25 60h-135l-25 -60h-70l116 271h96l110 -257v257h106l85 -184l77 184h108zM1255 453q0 -20 -5.5 -35t-14 -25t-22.5 -16.5t-26 -10t-31.5 -4.5t-31.5 -1t-32.5 0.5t-29.5 0.5v-91h-126l-80 90l-83 -90h-256v271h260 -l80 -89l82 89h207q109 0 109 -89zM964 794v-56h-217v271h217v-57h-152v-49h148v-55h-148v-54h152zM2304 235v-229q0 -55 -38.5 -94.5t-93.5 -39.5h-2040q-55 0 -93.5 39.5t-38.5 94.5v678h111l25 61h55l25 -61h218v46l19 -46h113l20 47v-47h541v99l10 1q10 0 10 -14v-86h279 -v23q23 -12 55 -18t52.5 -6.5t63 0.5t51.5 1l25 61h56l25 -61h227v58l34 -58h182v378h-180v-44l-25 44h-185v-44l-23 44h-249q-69 0 -109 -22v22h-172v-22q-24 22 -73 22h-628l-43 -97l-43 97h-198v-44l-22 44h-169l-78 -179v391q0 55 38.5 94.5t93.5 39.5h2040 -q55 0 93.5 -39.5t38.5 -94.5v-678h-120q-51 0 -81 -22v22h-177q-55 0 -78 -22v22h-316v-22q-31 22 -87 22h-209v-22q-23 22 -91 22h-234l-54 -58l-50 58h-349v-378h343l55 59l52 -59h211v89h21q59 0 90 13v-102h174v99h8q8 0 10 -2t2 -10v-87h529q57 0 88 24v-24h168 -q60 0 95 17zM1546 469q0 -23 -12 -43t-34 -29q25 -9 34 -26t9 -46v-54h-65v45q0 33 -12 43.5t-46 10.5h-69v-99h-65v271h154q48 0 77 -15t29 -58zM1269 936q0 -24 -12.5 -44t-33.5 -29q26 -9 34.5 -25.5t8.5 -46.5v-53h-65q0 9 0.5 26.5t0 25t-3 18.5t-8.5 16t-17.5 8.5 -t-29.5 3.5h-70v-98h-64v271l153 -1q49 0 78 -14.5t29 -57.5zM1798 327v-56h-216v271h216v-56h-151v-49h148v-55h-148v-54zM1372 1009v-271h-66v271h66zM2065 357q0 -86 -102 -86h-126v58h126q34 0 34 25q0 16 -17 21t-41.5 5t-49.5 3.5t-42 22.5t-17 55q0 39 26 60t66 21 -h130v-57h-119q-36 0 -36 -25q0 -16 17.5 -20.5t42 -4t49 -2.5t42 -21.5t17.5 -54.5zM2304 407v-101q-24 -35 -88 -35h-125v58h125q33 0 33 25q0 13 -12.5 19t-31 5.5t-40 2t-40 8t-31 24t-12.5 48.5q0 39 26.5 60t66.5 21h129v-57h-118q-36 0 -36 -25q0 -20 29 -22t68.5 -5 -t56.5 -26zM2139 1008v-270h-92l-122 203v-203h-132l-26 60h-134l-25 -60h-75q-129 0 -129 133q0 138 133 138h63v-59q-7 0 -28 1t-28.5 0.5t-23 -2t-21.5 -6.5t-14.5 -13.5t-11.5 -23t-3 -33.5q0 -38 13.5 -58t49.5 -20h29l92 213h97l109 -256v256h99l114 -188v188h66z" /> - <glyph glyph-name="_469" unicode="" horiz-adv-x="2304" -d="M745 630q0 -37 -25.5 -61.5t-62.5 -24.5q-29 0 -46.5 16t-17.5 44q0 37 25 62.5t62 25.5q28 0 46.5 -16.5t18.5 -45.5zM1530 779q0 -42 -22 -57t-66 -15l-32 -1l17 107q2 11 13 11h18q22 0 35 -2t25 -12.5t12 -30.5zM1881 630q0 -36 -25.5 -61t-61.5 -25q-29 0 -47 16 -t-18 44q0 37 25 62.5t62 25.5q28 0 46.5 -16.5t18.5 -45.5zM513 801q0 59 -38.5 85.5t-100.5 26.5h-160q-19 0 -21 -19l-65 -408q-1 -6 3 -11t10 -5h76q20 0 22 19l18 110q1 8 7 13t15 6.5t17 1.5t19 -1t14 -1q86 0 135 48.5t49 134.5zM822 489l41 261q1 6 -3 11t-10 5h-76 -q-14 0 -17 -33q-27 40 -95 40q-72 0 -122.5 -54t-50.5 -127q0 -59 34.5 -94t92.5 -35q28 0 58 12t48 32q-4 -12 -4 -21q0 -16 13 -16h69q19 0 22 19zM1269 752q0 5 -4 9.5t-9 4.5h-77q-11 0 -18 -10l-106 -156l-44 150q-5 16 -22 16h-75q-5 0 -9 -4.5t-4 -9.5q0 -2 19.5 -59 -t42 -123t23.5 -70q-82 -112 -82 -120q0 -13 13 -13h77q11 0 18 10l255 368q2 2 2 7zM1649 801q0 59 -38.5 85.5t-100.5 26.5h-159q-20 0 -22 -19l-65 -408q-1 -6 3 -11t10 -5h82q12 0 16 13l18 116q1 8 7 13t15 6.5t17 1.5t19 -1t14 -1q86 0 135 48.5t49 134.5zM1958 489 -l41 261q1 6 -3 11t-10 5h-76q-14 0 -17 -33q-26 40 -95 40q-72 0 -122.5 -54t-50.5 -127q0 -59 34.5 -94t92.5 -35q29 0 59 12t47 32q0 -1 -2 -9t-2 -12q0 -16 13 -16h69q19 0 22 19zM2176 898v1q0 14 -13 14h-74q-11 0 -13 -11l-65 -416l-1 -2q0 -5 4 -9.5t10 -4.5h66 -q19 0 21 19zM392 764q-5 -35 -26 -46t-60 -11l-33 -1l17 107q2 11 13 11h19q40 0 58 -11.5t12 -48.5zM2304 1280v-1280q0 -52 -38 -90t-90 -38h-2048q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h2048q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_470" unicode="" horiz-adv-x="2304" -d="M1597 633q0 -69 -21 -106q-19 -35 -52 -35q-23 0 -41 9v224q29 30 57 30q57 0 57 -122zM2035 669h-110q6 98 56 98q51 0 54 -98zM476 534q0 59 -33 91.5t-101 57.5q-36 13 -52 24t-16 25q0 26 38 26q58 0 124 -33l18 112q-67 32 -149 32q-77 0 -123 -38q-48 -39 -48 -109 -q0 -58 32.5 -90.5t99.5 -56.5q39 -14 54.5 -25.5t15.5 -27.5q0 -31 -48 -31q-29 0 -70 12.5t-72 30.5l-18 -113q72 -41 168 -41q81 0 129 37q51 41 51 117zM771 749l19 111h-96v135l-129 -21l-18 -114l-46 -8l-17 -103h62v-219q0 -84 44 -120q38 -30 111 -30q32 0 79 11v118 -q-32 -7 -44 -7q-42 0 -42 50v197h77zM1087 724v139q-15 3 -28 3q-32 0 -55.5 -16t-33.5 -46l-10 56h-131v-471h150v306q26 31 82 31q16 0 26 -2zM1124 389h150v471h-150v-471zM1746 638q0 122 -45 179q-40 52 -111 52q-64 0 -117 -56l-8 47h-132v-645l150 25v151 -q36 -11 68 -11q83 0 134 56q61 65 61 202zM1278 986q0 33 -23 56t-56 23t-56 -23t-23 -56t23 -56.5t56 -23.5t56 23.5t23 56.5zM2176 629q0 113 -48 176q-50 64 -144 64q-96 0 -151.5 -66t-55.5 -180q0 -128 63 -188q55 -55 161 -55q101 0 160 40l-16 103q-57 -31 -128 -31 -q-43 0 -63 19q-23 19 -28 66h248q2 14 2 52zM2304 1280v-1280q0 -52 -38 -90t-90 -38h-2048q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h2048q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_471" unicode="" horiz-adv-x="2048" -d="M1558 684q61 -356 298 -556q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-180.5 74.5t-75.5 180.5zM1024 -176q16 0 16 16t-16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5zM2026 1424q8 -10 7.5 -23.5t-10.5 -22.5 -l-1872 -1622q-10 -8 -23.5 -7t-21.5 11l-84 96q-8 10 -7.5 23.5t10.5 21.5l186 161q-19 32 -19 66q50 42 91 88t85 119.5t74.5 158.5t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q124 -18 219 -82.5t148 -157.5 -l418 363q10 8 23.5 7t21.5 -11z" /> - <glyph glyph-name="_472" unicode="" horiz-adv-x="2048" -d="M1040 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM503 315l877 760q-42 88 -132.5 146.5t-223.5 58.5q-93 0 -169.5 -31.5t-121.5 -80.5t-69 -103t-24 -105q0 -384 -137 -645zM1856 128 -q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-180.5 74.5t-75.5 180.5l149 129h757q-166 187 -227 459l111 97q61 -356 298 -556zM1942 1520l84 -96q8 -10 7.5 -23.5t-10.5 -22.5l-1872 -1622q-10 -8 -23.5 -7t-21.5 11l-84 96q-8 10 -7.5 23.5t10.5 21.5l186 161 -q-19 32 -19 66q50 42 91 88t85 119.5t74.5 158.5t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q124 -18 219 -82.5t148 -157.5l418 363q10 8 23.5 7t21.5 -11z" /> - <glyph glyph-name="_473" unicode="" horiz-adv-x="1408" -d="M512 160v704q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-704q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM768 160v704q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-704q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1024 160v704q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-704 -q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q0 -14 -9 -23t-23 -9h-96v-948q0 -83 -47 -143.5t-113 -60.5h-832q-66 0 -113 58.5t-47 141.5v952h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h309l70 167 -q15 37 54 63t79 26h320q40 0 79 -26t54 -63l70 -167h309q14 0 23 -9t9 -23z" /> - <glyph glyph-name="_474" unicode="" -d="M1150 462v-109q0 -50 -36.5 -89t-94 -60.5t-118 -32.5t-117.5 -11q-205 0 -342.5 139t-137.5 346q0 203 136 339t339 136q34 0 75.5 -4.5t93 -18t92.5 -34t69 -56.5t28 -81v-109q0 -16 -16 -16h-118q-16 0 -16 16v70q0 43 -65.5 67.5t-137.5 24.5q-140 0 -228.5 -91.5 -t-88.5 -237.5q0 -151 91.5 -249.5t233.5 -98.5q68 0 138 24t70 66v70q0 7 4.5 11.5t10.5 4.5h119q6 0 11 -4.5t5 -11.5zM768 1280q-130 0 -248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5 -t-51 248.5t-136.5 204t-204 136.5t-248.5 51zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="_475" unicode="" -d="M972 761q0 108 -53.5 169t-147.5 61q-63 0 -124 -30.5t-110 -84.5t-79.5 -137t-30.5 -180q0 -112 53.5 -173t150.5 -61q96 0 176 66.5t122.5 166t42.5 203.5zM1536 640q0 -111 -37 -197t-98.5 -135t-131.5 -74.5t-145 -27.5q-6 0 -15.5 -0.5t-16.5 -0.5q-95 0 -142 53 -q-28 33 -33 83q-52 -66 -131.5 -110t-173.5 -44q-161 0 -249.5 95.5t-88.5 269.5q0 157 66 290t179 210.5t246 77.5q87 0 155 -35.5t106 -99.5l2 19l11 56q1 6 5.5 12t9.5 6h118q5 0 13 -11q5 -5 3 -16l-120 -614q-5 -24 -5 -48q0 -39 12.5 -52t44.5 -13q28 1 57 5.5t73 24 -t77 50t57 89.5t24 137q0 292 -174 466t-466 174q-130 0 -248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51q228 0 405 144q11 9 24 8t21 -12l41 -49q8 -12 7 -24q-2 -13 -12 -22q-102 -83 -227.5 -128t-258.5 -45q-156 0 -298 61 -t-245 164t-164 245t-61 298t61 298t164 245t245 164t298 61q344 0 556 -212t212 -556z" /> - <glyph glyph-name="_476" unicode="" horiz-adv-x="1792" -d="M1698 1442q94 -94 94 -226.5t-94 -225.5l-225 -223l104 -104q10 -10 10 -23t-10 -23l-210 -210q-10 -10 -23 -10t-23 10l-105 105l-603 -603q-37 -37 -90 -37h-203l-256 -128l-64 64l128 256v203q0 53 37 90l603 603l-105 105q-10 10 -10 23t10 23l210 210q10 10 23 10 -t23 -10l104 -104l223 225q93 94 225.5 94t226.5 -94zM512 64l576 576l-192 192l-576 -576v-192h192z" /> - <glyph glyph-name="f1fc" unicode="" horiz-adv-x="1792" -d="M1615 1536q70 0 122.5 -46.5t52.5 -116.5q0 -63 -45 -151q-332 -629 -465 -752q-97 -91 -218 -91q-126 0 -216.5 92.5t-90.5 219.5q0 128 92 212l638 579q59 54 130 54zM706 502q39 -76 106.5 -130t150.5 -76l1 -71q4 -213 -129.5 -347t-348.5 -134q-123 0 -218 46.5 -t-152.5 127.5t-86.5 183t-29 220q7 -5 41 -30t62 -44.5t59 -36.5t46 -17q41 0 55 37q25 66 57.5 112.5t69.5 76t88 47.5t103 25.5t125 10.5z" /> - <glyph glyph-name="_478" unicode="" horiz-adv-x="1792" -d="M1792 128v-384h-1792v384q45 0 85 14t59 27.5t47 37.5q30 27 51.5 38t56.5 11q24 0 44 -7t31 -15t33 -27q29 -25 47 -38t58 -27t86 -14q45 0 85 14.5t58 27t48 37.5q21 19 32.5 27t31 15t43.5 7q35 0 56.5 -11t51.5 -38q28 -24 47 -37.5t59 -27.5t85 -14t85 14t59 27.5 -t47 37.5q30 27 51.5 38t56.5 11q34 0 55.5 -11t51.5 -38q28 -24 47 -37.5t59 -27.5t85 -14zM1792 448v-192q-24 0 -44 7t-31 15t-33 27q-29 25 -47 38t-58 27t-85 14q-46 0 -86 -14t-58 -27t-47 -38q-22 -19 -33 -27t-31 -15t-44 -7q-35 0 -56.5 11t-51.5 38q-29 25 -47 38 -t-58 27t-86 14q-45 0 -85 -14.5t-58 -27t-48 -37.5q-21 -19 -32.5 -27t-31 -15t-43.5 -7q-35 0 -56.5 11t-51.5 38q-28 24 -47 37.5t-59 27.5t-85 14q-46 0 -86 -14t-58 -27t-47 -38q-30 -27 -51.5 -38t-56.5 -11v192q0 80 56 136t136 56h64v448h256v-448h256v448h256v-448 -h256v448h256v-448h64q80 0 136 -56t56 -136zM512 1312q0 -77 -36 -118.5t-92 -41.5q-53 0 -90.5 37.5t-37.5 90.5q0 29 9.5 51t23.5 34t31 28t31 31.5t23.5 44.5t9.5 67q38 0 83 -74t45 -150zM1024 1312q0 -77 -36 -118.5t-92 -41.5q-53 0 -90.5 37.5t-37.5 90.5 -q0 29 9.5 51t23.5 34t31 28t31 31.5t23.5 44.5t9.5 67q38 0 83 -74t45 -150zM1536 1312q0 -77 -36 -118.5t-92 -41.5q-53 0 -90.5 37.5t-37.5 90.5q0 29 9.5 51t23.5 34t31 28t31 31.5t23.5 44.5t9.5 67q38 0 83 -74t45 -150z" /> - <glyph glyph-name="_479" unicode="" horiz-adv-x="2048" -d="M2048 0v-128h-2048v1536h128v-1408h1920zM1664 1024l256 -896h-1664v576l448 576l576 -576z" /> - <glyph glyph-name="_480" unicode="" horiz-adv-x="1792" -d="M768 646l546 -546q-106 -108 -247.5 -168t-298.5 -60q-209 0 -385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103v-762zM955 640h773q0 -157 -60 -298.5t-168 -247.5zM1664 768h-768v768q209 0 385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="_481" unicode="" horiz-adv-x="2048" -d="M2048 0v-128h-2048v1536h128v-1408h1920zM1920 1248v-435q0 -21 -19.5 -29.5t-35.5 7.5l-121 121l-633 -633q-10 -10 -23 -10t-23 10l-233 233l-416 -416l-192 192l585 585q10 10 23 10t23 -10l233 -233l464 464l-121 121q-16 16 -7.5 35.5t29.5 19.5h435q14 0 23 -9 -t9 -23z" /> - <glyph glyph-name="_482" unicode="" horiz-adv-x="1792" -d="M1292 832q0 -6 10 -41q10 -29 25 -49.5t41 -34t44 -20t55 -16.5q325 -91 325 -332q0 -146 -105.5 -242.5t-254.5 -96.5q-59 0 -111.5 18.5t-91.5 45.5t-77 74.5t-63 87.5t-53.5 103.5t-43.5 103t-39.5 106.5t-35.5 95q-32 81 -61.5 133.5t-73.5 96.5t-104 64t-142 20 -q-96 0 -183 -55.5t-138 -144.5t-51 -185q0 -160 106.5 -279.5t263.5 -119.5q177 0 258 95q56 63 83 116l84 -152q-15 -34 -44 -70l1 -1q-131 -152 -388 -152q-147 0 -269.5 79t-190.5 207.5t-68 274.5q0 105 43.5 206t116 176.5t172 121.5t204.5 46q87 0 159 -19t123.5 -50 -t95 -80t72.5 -99t58.5 -117t50.5 -124.5t50 -130.5t55 -127q96 -200 233 -200q81 0 138.5 48.5t57.5 128.5q0 42 -19 72t-50.5 46t-72.5 31.5t-84.5 27t-87.5 34t-81 52t-65 82t-39 122.5q-3 16 -3 33q0 110 87.5 192t198.5 78q78 -3 120.5 -14.5t90.5 -53.5h-1 -q12 -11 23 -24.5t26 -36t19 -27.5l-129 -99q-26 49 -54 70v1q-23 21 -97 21q-49 0 -84 -33t-35 -83z" /> - <glyph glyph-name="_483" unicode="" -d="M1432 484q0 173 -234 239q-35 10 -53 16.5t-38 25t-29 46.5q0 2 -2 8.5t-3 12t-1 7.5q0 36 24.5 59.5t60.5 23.5q54 0 71 -15h-1q20 -15 39 -51l93 71q-39 54 -49 64q-33 29 -67.5 39t-85.5 10q-80 0 -142 -57.5t-62 -137.5q0 -7 2 -23q16 -96 64.5 -140t148.5 -73 -q29 -8 49 -15.5t45 -21.5t38.5 -34.5t13.5 -46.5v-5q1 -58 -40.5 -93t-100.5 -35q-97 0 -167 144q-23 47 -51.5 121.5t-48 125.5t-54 110.5t-74 95.5t-103.5 60.5t-147 24.5q-101 0 -192 -56t-144 -148t-50 -192v-1q4 -108 50.5 -199t133.5 -147.5t196 -56.5q186 0 279 110 -q20 27 31 51l-60 109q-42 -80 -99 -116t-146 -36q-115 0 -191 87t-76 204q0 105 82 189t186 84q112 0 170 -53.5t104 -172.5q8 -21 25.5 -68.5t28.5 -76.5t31.5 -74.5t38.5 -74t45.5 -62.5t55.5 -53.5t66 -33t80 -13.5q107 0 183 69.5t76 174.5zM1536 1120v-960 -q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_484" unicode="" horiz-adv-x="2048" -d="M1152 640q0 104 -40.5 198.5t-109.5 163.5t-163.5 109.5t-198.5 40.5t-198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5t198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5zM1920 640q0 104 -40.5 198.5 -t-109.5 163.5t-163.5 109.5t-198.5 40.5h-386q119 -90 188.5 -224t69.5 -288t-69.5 -288t-188.5 -224h386q104 0 198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5zM2048 640q0 -130 -51 -248.5t-136.5 -204t-204 -136.5t-248.5 -51h-768q-130 0 -248.5 51t-204 136.5 -t-136.5 204t-51 248.5t51 248.5t136.5 204t204 136.5t248.5 51h768q130 0 248.5 -51t204 -136.5t136.5 -204t51 -248.5z" /> - <glyph glyph-name="_485" unicode="" horiz-adv-x="2048" -d="M0 640q0 130 51 248.5t136.5 204t204 136.5t248.5 51h768q130 0 248.5 -51t204 -136.5t136.5 -204t51 -248.5t-51 -248.5t-136.5 -204t-204 -136.5t-248.5 -51h-768q-130 0 -248.5 51t-204 136.5t-136.5 204t-51 248.5zM1408 128q104 0 198.5 40.5t163.5 109.5 -t109.5 163.5t40.5 198.5t-40.5 198.5t-109.5 163.5t-163.5 109.5t-198.5 40.5t-198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5z" /> - <glyph glyph-name="_486" unicode="" horiz-adv-x="2304" -d="M762 384h-314q-40 0 -57.5 35t6.5 67l188 251q-65 31 -137 31q-132 0 -226 -94t-94 -226t94 -226t226 -94q115 0 203 72.5t111 183.5zM576 512h186q-18 85 -75 148zM1056 512l288 384h-480l-99 -132q105 -103 126 -252h165zM2176 448q0 132 -94 226t-226 94 -q-60 0 -121 -24l174 -260q15 -23 10 -49t-27 -40q-15 -11 -36 -11q-35 0 -53 29l-174 260q-93 -95 -93 -225q0 -132 94 -226t226 -94t226 94t94 226zM2304 448q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 97 39.5 183.5t109.5 149.5l-65 98l-353 -469 -q-18 -26 -51 -26h-197q-23 -164 -149 -274t-294 -110q-185 0 -316.5 131.5t-131.5 316.5t131.5 316.5t316.5 131.5q114 0 215 -55l137 183h-224q-26 0 -45 19t-19 45t19 45t45 19h384v-128h435l-85 128h-222q-26 0 -45 19t-19 45t19 45t45 19h256q33 0 53 -28l267 -400 -q91 44 192 44q185 0 316.5 -131.5t131.5 -316.5z" /> - <glyph glyph-name="_487" unicode="" -d="M384 320q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1408 320q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1362 716l-72 384q-5 23 -22.5 37.5t-40.5 14.5 -h-918q-23 0 -40.5 -14.5t-22.5 -37.5l-72 -384q-5 -30 14 -53t49 -23h1062q30 0 49 23t14 53zM1136 1328q0 20 -14 34t-34 14h-640q-20 0 -34 -14t-14 -34t14 -34t34 -14h640q20 0 34 14t14 34zM1536 603v-603h-128v-128q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5 -t-37.5 90.5v128h-768v-128q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5v128h-128v603q0 112 25 223l103 454q9 78 97.5 137t230 89t312.5 30t312.5 -30t230 -89t97.5 -137l105 -454q23 -102 23 -223z" /> - <glyph glyph-name="_488" unicode="" horiz-adv-x="2048" -d="M1463 704q0 -35 -25 -60.5t-61 -25.5h-702q-36 0 -61 25.5t-25 60.5t25 60.5t61 25.5h702q36 0 61 -25.5t25 -60.5zM1677 704q0 86 -23 170h-982q-36 0 -61 25t-25 60q0 36 25 61t61 25h908q-88 143 -235 227t-320 84q-177 0 -327.5 -87.5t-238 -237.5t-87.5 -327 -q0 -86 23 -170h982q36 0 61 -25t25 -60q0 -36 -25 -61t-61 -25h-908q88 -143 235.5 -227t320.5 -84q132 0 253 51.5t208 139t139 208t52 253.5zM2048 959q0 -35 -25 -60t-61 -25h-131q17 -85 17 -170q0 -167 -65.5 -319.5t-175.5 -263t-262.5 -176t-319.5 -65.5 -q-246 0 -448.5 133t-301.5 350h-189q-36 0 -61 25t-25 61q0 35 25 60t61 25h132q-17 85 -17 170q0 167 65.5 319.5t175.5 263t262.5 176t320.5 65.5q245 0 447.5 -133t301.5 -350h188q36 0 61 -25t25 -61z" /> - <glyph glyph-name="_489" unicode="" horiz-adv-x="1280" -d="M953 1158l-114 -328l117 -21q165 451 165 518q0 56 -38 56q-57 0 -130 -225zM654 471l33 -88q37 42 71 67l-33 5.5t-38.5 7t-32.5 8.5zM362 1367q0 -98 159 -521q17 10 49 10q15 0 75 -5l-121 351q-75 220 -123 220q-19 0 -29 -17.5t-10 -37.5zM283 608q0 -36 51.5 -119 -t117.5 -153t100 -70q14 0 25.5 13t11.5 27q0 24 -32 102q-13 32 -32 72t-47.5 89t-61.5 81t-62 32q-20 0 -45.5 -27t-25.5 -47zM125 273q0 -41 25 -104q59 -145 183.5 -227t281.5 -82q227 0 382 170q152 169 152 427q0 43 -1 67t-11.5 62t-30.5 56q-56 49 -211.5 75.5 -t-270.5 26.5q-37 0 -49 -11q-12 -5 -12 -35q0 -34 21.5 -60t55.5 -40t77.5 -23.5t87.5 -11.5t85 -4t70 0h23q24 0 40 -19q15 -19 19 -55q-28 -28 -96 -54q-61 -22 -93 -46q-64 -46 -108.5 -114t-44.5 -137q0 -31 18.5 -88.5t18.5 -87.5l-3 -12q-4 -12 -4 -14 -q-137 10 -146 216q-8 -2 -41 -2q2 -7 2 -21q0 -53 -40.5 -89.5t-94.5 -36.5q-82 0 -166.5 78t-84.5 159q0 34 33 67q52 -64 60 -76q77 -104 133 -104q12 0 26.5 8.5t14.5 20.5q0 34 -87.5 145t-116.5 111q-43 0 -70 -44.5t-27 -90.5zM11 264q0 101 42.5 163t136.5 88 -q-28 74 -28 104q0 62 61 123t122 61q29 0 70 -15q-163 462 -163 567q0 80 41 130.5t119 50.5q131 0 325 -581q6 -17 8 -23q6 16 29 79.5t43.5 118.5t54 127.5t64.5 123t70.5 86.5t76.5 36q71 0 112 -49t41 -122q0 -108 -159 -550q61 -15 100.5 -46t58.5 -78t26 -93.5 -t7 -110.5q0 -150 -47 -280t-132 -225t-211 -150t-278 -55q-111 0 -223 42q-149 57 -258 191.5t-109 286.5z" /> - <glyph glyph-name="_490" unicode="" horiz-adv-x="2048" -d="M785 528h207q-14 -158 -98.5 -248.5t-214.5 -90.5q-162 0 -254.5 116t-92.5 316q0 194 93 311.5t233 117.5q148 0 232 -87t97 -247h-203q-5 64 -35.5 99t-81.5 35q-57 0 -88.5 -60.5t-31.5 -177.5q0 -48 5 -84t18 -69.5t40 -51.5t66 -18q95 0 109 139zM1497 528h206 -q-14 -158 -98 -248.5t-214 -90.5q-162 0 -254.5 116t-92.5 316q0 194 93 311.5t233 117.5q148 0 232 -87t97 -247h-204q-4 64 -35 99t-81 35q-57 0 -88.5 -60.5t-31.5 -177.5q0 -48 5 -84t18 -69.5t39.5 -51.5t65.5 -18q49 0 76.5 38t33.5 101zM1856 647q0 207 -15.5 307 -t-60.5 161q-6 8 -13.5 14t-21.5 15t-16 11q-86 63 -697 63q-625 0 -710 -63q-5 -4 -17.5 -11.5t-21 -14t-14.5 -14.5q-45 -60 -60 -159.5t-15 -308.5q0 -208 15 -307.5t60 -160.5q6 -8 15 -15t20.5 -14t17.5 -12q44 -33 239.5 -49t470.5 -16q610 0 697 65q5 4 17 11t20.5 14 -t13.5 16q46 60 61 159t15 309zM2048 1408v-1536h-2048v1536h2048z" /> - <glyph glyph-name="_491" unicode="" -d="M992 912v-496q0 -14 -9 -23t-23 -9h-160q-14 0 -23 9t-9 23v496q0 112 -80 192t-192 80h-272v-1152q0 -14 -9 -23t-23 -9h-160q-14 0 -23 9t-9 23v1344q0 14 9 23t23 9h464q135 0 249 -66.5t180.5 -180.5t66.5 -249zM1376 1376v-880q0 -135 -66.5 -249t-180.5 -180.5 -t-249 -66.5h-464q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h160q14 0 23 -9t9 -23v-768h272q112 0 192 80t80 192v880q0 14 9 23t23 9h160q14 0 23 -9t9 -23z" /> - <glyph glyph-name="_492" unicode="" -d="M1311 694v-114q0 -24 -13.5 -38t-37.5 -14h-202q-24 0 -38 14t-14 38v114q0 24 14 38t38 14h202q24 0 37.5 -14t13.5 -38zM821 464v250q0 53 -32.5 85.5t-85.5 32.5h-133q-68 0 -96 -52q-28 52 -96 52h-130q-53 0 -85.5 -32.5t-32.5 -85.5v-250q0 -22 21 -22h55 -q22 0 22 22v230q0 24 13.5 38t38.5 14h94q24 0 38 -14t14 -38v-230q0 -22 21 -22h54q22 0 22 22v230q0 24 14 38t38 14h97q24 0 37.5 -14t13.5 -38v-230q0 -22 22 -22h55q21 0 21 22zM1410 560v154q0 53 -33 85.5t-86 32.5h-264q-53 0 -86 -32.5t-33 -85.5v-410 -q0 -21 22 -21h55q21 0 21 21v180q31 -42 94 -42h191q53 0 86 32.5t33 85.5zM1536 1176v-1072q0 -96 -68 -164t-164 -68h-1072q-96 0 -164 68t-68 164v1072q0 96 68 164t164 68h1072q96 0 164 -68t68 -164z" /> - <glyph glyph-name="_493" unicode="" -d="M915 450h-294l147 551zM1001 128h311l-324 1024h-440l-324 -1024h311l383 314zM1536 1120v-960q0 -118 -85 -203t-203 -85h-960q-118 0 -203 85t-85 203v960q0 118 85 203t203 85h960q118 0 203 -85t85 -203z" /> - <glyph glyph-name="_494" unicode="" horiz-adv-x="2048" -d="M2048 641q0 -21 -13 -36.5t-33 -19.5l-205 -356q3 -9 3 -18q0 -20 -12.5 -35.5t-32.5 -19.5l-193 -337q3 -8 3 -16q0 -23 -16.5 -40t-40.5 -17q-25 0 -41 18h-400q-17 -20 -43 -20t-43 20h-399q-17 -20 -43 -20q-23 0 -40 16.5t-17 40.5q0 8 4 20l-193 335 -q-20 4 -32.5 19.5t-12.5 35.5q0 9 3 18l-206 356q-20 5 -32.5 20.5t-12.5 35.5q0 21 13.5 36.5t33.5 19.5l199 344q0 1 -0.5 3t-0.5 3q0 36 34 51l209 363q-4 10 -4 18q0 24 17 40.5t40 16.5q26 0 44 -21h396q16 21 43 21t43 -21h398q18 21 44 21q23 0 40 -16.5t17 -40.5 -q0 -6 -4 -18l207 -358q23 -1 39 -17.5t16 -38.5q0 -13 -7 -27l187 -324q19 -4 31.5 -19.5t12.5 -35.5zM1063 -158h389l-342 354h-143l-342 -354h360q18 16 39 16t39 -16zM112 654q1 -4 1 -13q0 -10 -2 -15l208 -360l15 -6l188 199v347l-187 194q-13 -8 -29 -10zM986 1438 -h-388l190 -200l554 200h-280q-16 -16 -38 -16t-38 16zM1689 226q1 6 5 11l-64 68l-17 -79h76zM1583 226l22 105l-252 266l-296 -307l63 -64h463zM1495 -142l16 28l65 310h-427l333 -343q8 4 13 5zM578 -158h5l342 354h-373v-335l4 -6q14 -5 22 -13zM552 226h402l64 66 -l-309 321l-157 -166v-221zM359 226h163v189l-168 -177q4 -8 5 -12zM358 1051q0 -1 0.5 -2t0.5 -2q0 -16 -8 -29l171 -177v269zM552 1121v-311l153 -157l297 314l-223 236zM556 1425l-4 -8v-264l205 74l-191 201q-6 -2 -10 -3zM1447 1438h-16l-621 -224l213 -225zM1023 946 -l-297 -315l311 -319l296 307zM688 634l-136 141v-284zM1038 270l-42 -44h85zM1374 618l238 -251l132 624l-3 5l-1 1zM1718 1018q-8 13 -8 29v2l-216 376q-5 1 -13 5l-437 -463l310 -327zM522 1142v223l-163 -282zM522 196h-163l163 -283v283zM1607 196l-48 -227l130 227h-82 -zM1729 266l207 361q-2 10 -2 14q0 1 3 16l-171 296l-129 -612l77 -82q5 3 15 7z" /> - <glyph glyph-name="f210" unicode="" -d="M0 856q0 131 91.5 226.5t222.5 95.5h742l352 358v-1470q0 -132 -91.5 -227t-222.5 -95h-780q-131 0 -222.5 95t-91.5 227v790zM1232 102l-176 180v425q0 46 -32 79t-78 33h-484q-46 0 -78 -33t-32 -79v-492q0 -46 32.5 -79.5t77.5 -33.5h770z" /> - <glyph glyph-name="_496" unicode="" -d="M934 1386q-317 -121 -556 -362.5t-358 -560.5q-20 89 -20 176q0 208 102.5 384.5t278.5 279t384 102.5q82 0 169 -19zM1203 1267q93 -65 164 -155q-389 -113 -674.5 -400.5t-396.5 -676.5q-93 72 -155 162q112 386 395 671t667 399zM470 -67q115 356 379.5 622t619.5 384 -q40 -92 54 -195q-292 -120 -516 -345t-343 -518q-103 14 -194 52zM1536 -125q-193 50 -367 115q-135 -84 -290 -107q109 205 274 370.5t369 275.5q-21 -152 -101 -284q65 -175 115 -370z" /> - <glyph glyph-name="f212" unicode="" horiz-adv-x="2048" -d="M1893 1144l155 -1272q-131 0 -257 57q-200 91 -393 91q-226 0 -374 -148q-148 148 -374 148q-193 0 -393 -91q-128 -57 -252 -57h-5l155 1272q224 127 482 127q233 0 387 -106q154 106 387 106q258 0 482 -127zM1398 157q129 0 232 -28.5t260 -93.5l-124 1021 -q-171 78 -368 78q-224 0 -374 -141q-150 141 -374 141q-197 0 -368 -78l-124 -1021q105 43 165.5 65t148.5 39.5t178 17.5q202 0 374 -108q172 108 374 108zM1438 191l-55 907q-211 -4 -359 -155q-152 155 -374 155q-176 0 -336 -66l-114 -941q124 51 228.5 76t221.5 25 -q209 0 374 -102q172 107 374 102z" /> - <glyph glyph-name="_498" unicode="" horiz-adv-x="2048" -d="M1500 165v733q0 21 -15 36t-35 15h-93q-20 0 -35 -15t-15 -36v-733q0 -20 15 -35t35 -15h93q20 0 35 15t15 35zM1216 165v531q0 20 -15 35t-35 15h-101q-20 0 -35 -15t-15 -35v-531q0 -20 15 -35t35 -15h101q20 0 35 15t15 35zM924 165v429q0 20 -15 35t-35 15h-101 -q-20 0 -35 -15t-15 -35v-429q0 -20 15 -35t35 -15h101q20 0 35 15t15 35zM632 165v362q0 20 -15 35t-35 15h-101q-20 0 -35 -15t-15 -35v-362q0 -20 15 -35t35 -15h101q20 0 35 15t15 35zM2048 311q0 -166 -118 -284t-284 -118h-1244q-166 0 -284 118t-118 284 -q0 116 63 214.5t168 148.5q-10 34 -10 73q0 113 80.5 193.5t193.5 80.5q102 0 180 -67q45 183 194 300t338 117q149 0 275 -73.5t199.5 -199.5t73.5 -275q0 -66 -14 -122q135 -33 221 -142.5t86 -247.5z" /> - <glyph glyph-name="_499" unicode="" -d="M0 1536h1536v-1392l-776 -338l-760 338v1392zM1436 209v926h-1336v-926l661 -294zM1436 1235v201h-1336v-201h1336zM181 937v-115h-37v115h37zM181 789v-115h-37v115h37zM181 641v-115h-37v115h37zM181 493v-115h-37v115h37zM181 345v-115h-37v115h37zM207 202l15 34 -l105 -47l-15 -33zM343 142l15 34l105 -46l-15 -34zM478 82l15 34l105 -46l-15 -34zM614 23l15 33l104 -46l-15 -34zM797 10l105 46l15 -33l-105 -47zM932 70l105 46l15 -34l-105 -46zM1068 130l105 46l15 -34l-105 -46zM1203 189l105 47l15 -34l-105 -46zM259 1389v-36h-114 -v36h114zM421 1389v-36h-115v36h115zM583 1389v-36h-115v36h115zM744 1389v-36h-114v36h114zM906 1389v-36h-114v36h114zM1068 1389v-36h-115v36h115zM1230 1389v-36h-115v36h115zM1391 1389v-36h-114v36h114zM181 1049v-79h-37v115h115v-36h-78zM421 1085v-36h-115v36h115z -M583 1085v-36h-115v36h115zM744 1085v-36h-114v36h114zM906 1085v-36h-114v36h114zM1068 1085v-36h-115v36h115zM1230 1085v-36h-115v36h115zM1355 970v79h-78v36h115v-115h-37zM1355 822v115h37v-115h-37zM1355 674v115h37v-115h-37zM1355 526v115h37v-115h-37zM1355 378 -v115h37v-115h-37zM1355 230v115h37v-115h-37zM760 265q-129 0 -221 91.5t-92 221.5q0 129 92 221t221 92q130 0 221.5 -92t91.5 -221q0 -130 -91.5 -221.5t-221.5 -91.5zM595 646q0 -36 19.5 -56.5t49.5 -25t64 -7t64 -2t49.5 -9t19.5 -30.5q0 -49 -112 -49q-97 0 -123 51 -h-3l-31 -63q67 -42 162 -42q29 0 56.5 5t55.5 16t45.5 33t17.5 53q0 46 -27.5 69.5t-67.5 27t-79.5 3t-67 5t-27.5 25.5q0 21 20.5 33t40.5 15t41 3q34 0 70.5 -11t51.5 -34h3l30 58q-3 1 -21 8.5t-22.5 9t-19.5 7t-22 7t-20 4.5t-24 4t-23 1q-29 0 -56.5 -5t-54 -16.5 -t-43 -34t-16.5 -53.5z" /> - <glyph glyph-name="_500" unicode="" horiz-adv-x="2048" -d="M863 504q0 112 -79.5 191.5t-191.5 79.5t-191 -79.5t-79 -191.5t79 -191t191 -79t191.5 79t79.5 191zM1726 505q0 112 -79 191t-191 79t-191.5 -79t-79.5 -191q0 -113 79.5 -192t191.5 -79t191 79.5t79 191.5zM2048 1314v-1348q0 -44 -31.5 -75.5t-76.5 -31.5h-1832 -q-45 0 -76.5 31.5t-31.5 75.5v1348q0 44 31.5 75.5t76.5 31.5h431q44 0 76 -31.5t32 -75.5v-161h754v161q0 44 32 75.5t76 31.5h431q45 0 76.5 -31.5t31.5 -75.5z" /> - <glyph glyph-name="_501" unicode="" horiz-adv-x="2048" -d="M1430 953zM1690 749q148 0 253 -98.5t105 -244.5q0 -157 -109 -261.5t-267 -104.5q-85 0 -162 27.5t-138 73.5t-118 106t-109 126t-103.5 132.5t-108.5 126.5t-117 106t-136 73.5t-159 27.5q-154 0 -251.5 -91.5t-97.5 -244.5q0 -157 104 -250t263 -93q100 0 208 37.5 -t193 98.5q5 4 21 18.5t30 24t22 9.5q14 0 24.5 -10.5t10.5 -24.5q0 -24 -60 -77q-101 -88 -234.5 -142t-260.5 -54q-133 0 -245.5 58t-180 165t-67.5 241q0 205 141.5 341t347.5 136q120 0 226.5 -43.5t185.5 -113t151.5 -153t139 -167.5t133.5 -153.5t149.5 -113 -t172.5 -43.5q102 0 168.5 61.5t66.5 162.5q0 95 -64.5 159t-159.5 64q-30 0 -81.5 -18.5t-68.5 -18.5q-20 0 -35.5 15t-15.5 35q0 18 8.5 57t8.5 59q0 159 -107.5 263t-266.5 104q-58 0 -111.5 -18.5t-84 -40.5t-55.5 -40.5t-33 -18.5q-15 0 -25.5 10.5t-10.5 25.5 -q0 19 25 46q59 67 147 103.5t182 36.5q191 0 318 -125.5t127 -315.5q0 -37 -4 -66q57 15 115 15z" /> - <glyph glyph-name="_502" unicode="" horiz-adv-x="1664" -d="M1216 832q0 26 -19 45t-45 19h-128v128q0 26 -19 45t-45 19t-45 -19t-19 -45v-128h-128q-26 0 -45 -19t-19 -45t19 -45t45 -19h128v-128q0 -26 19 -45t45 -19t45 19t19 45v128h128q26 0 45 19t19 45zM640 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5 -t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1536 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1664 1088v-512q0 -24 -16 -42.5t-41 -21.5l-1044 -122q1 -7 4.5 -21.5t6 -26.5t2.5 -22q0 -16 -24 -64h920 -q26 0 45 -19t19 -45t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 14 11 39.5t29.5 59.5t20.5 38l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t20 -15.5t13 -24.5t7.5 -26.5t5.5 -29.5t4.5 -25.5h1201q26 0 45 -19t19 -45z" /> - <glyph glyph-name="_503" unicode="" horiz-adv-x="1664" -d="M1280 832q0 26 -19 45t-45 19t-45 -19l-147 -146v293q0 26 -19 45t-45 19t-45 -19t-19 -45v-293l-147 146q-19 19 -45 19t-45 -19t-19 -45t19 -45l256 -256q19 -19 45 -19t45 19l256 256q19 19 19 45zM640 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5 -t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1536 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1664 1088v-512q0 -24 -16 -42.5t-41 -21.5l-1044 -122q1 -7 4.5 -21.5t6 -26.5t2.5 -22q0 -16 -24 -64h920 -q26 0 45 -19t19 -45t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 14 11 39.5t29.5 59.5t20.5 38l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t20 -15.5t13 -24.5t7.5 -26.5t5.5 -29.5t4.5 -25.5h1201q26 0 45 -19t19 -45z" /> - <glyph glyph-name="_504" unicode="" horiz-adv-x="2048" -d="M212 768l623 -665l-300 665h-323zM1024 -4l349 772h-698zM538 896l204 384h-262l-288 -384h346zM1213 103l623 665h-323zM683 896h682l-204 384h-274zM1510 896h346l-288 384h-262zM1651 1382l384 -512q14 -18 13 -41.5t-17 -40.5l-960 -1024q-18 -20 -47 -20t-47 20 -l-960 1024q-16 17 -17 40.5t13 41.5l384 512q18 26 51 26h1152q33 0 51 -26z" /> - <glyph glyph-name="_505" unicode="" horiz-adv-x="2048" -d="M1811 -19q19 19 45 19t45 -19l128 -128l-90 -90l-83 83l-83 -83q-18 -19 -45 -19t-45 19l-83 83l-83 -83q-19 -19 -45 -19t-45 19l-83 83l-83 -83q-19 -19 -45 -19t-45 19l-83 83l-83 -83q-19 -19 -45 -19t-45 19l-83 83l-83 -83q-19 -19 -45 -19t-45 19l-83 83l-83 -83 -q-19 -19 -45 -19t-45 19l-83 83l-83 -83q-19 -19 -45 -19t-45 19l-128 128l90 90l83 -83l83 83q19 19 45 19t45 -19l83 -83l83 83q19 19 45 19t45 -19l83 -83l83 83q19 19 45 19t45 -19l83 -83l83 83q19 19 45 19t45 -19l83 -83l83 83q19 19 45 19t45 -19l83 -83l83 83 -q19 19 45 19t45 -19l83 -83zM237 19q-19 -19 -45 -19t-45 19l-128 128l90 90l83 -82l83 82q19 19 45 19t45 -19l83 -82l64 64v293l-210 314q-17 26 -7 56.5t40 40.5l177 58v299h128v128h256v128h256v-128h256v-128h128v-299l177 -58q30 -10 40 -40.5t-7 -56.5l-210 -314 -v-293l19 18q19 19 45 19t45 -19l83 -82l83 82q19 19 45 19t45 -19l128 -128l-90 -90l-83 83l-83 -83q-18 -19 -45 -19t-45 19l-83 83l-83 -83q-19 -19 -45 -19t-45 19l-83 83l-83 -83q-19 -19 -45 -19t-45 19l-83 83l-83 -83q-19 -19 -45 -19t-45 19l-83 83l-83 -83 -q-19 -19 -45 -19t-45 19l-83 83l-83 -83q-19 -19 -45 -19t-45 19l-83 83zM640 1152v-128l384 128l384 -128v128h-128v128h-512v-128h-128z" /> - <glyph glyph-name="_506" unicode="" -d="M576 0l96 448l-96 128l-128 64zM832 0l128 640l-128 -64l-96 -128zM992 1010q-2 4 -4 6q-10 8 -96 8q-70 0 -167 -19q-7 -2 -21 -2t-21 2q-97 19 -167 19q-86 0 -96 -8q-2 -2 -4 -6q2 -18 4 -27q2 -3 7.5 -6.5t7.5 -10.5q2 -4 7.5 -20.5t7 -20.5t7.5 -17t8.5 -17t9 -14 -t12 -13.5t14 -9.5t17.5 -8t20.5 -4t24.5 -2q36 0 59 12.5t32.5 30t14.5 34.5t11.5 29.5t17.5 12.5h12q11 0 17.5 -12.5t11.5 -29.5t14.5 -34.5t32.5 -30t59 -12.5q13 0 24.5 2t20.5 4t17.5 8t14 9.5t12 13.5t9 14t8.5 17t7.5 17t7 20.5t7.5 20.5q2 7 7.5 10.5t7.5 6.5 -q2 9 4 27zM1408 131q0 -121 -73 -190t-194 -69h-874q-121 0 -194 69t-73 190q0 61 4.5 118t19 125.5t37.5 123.5t63.5 103.5t93.5 74.5l-90 220h214q-22 64 -22 128q0 12 2 32q-194 40 -194 96q0 57 210 99q17 62 51.5 134t70.5 114q32 37 76 37q30 0 84 -31t84 -31t84 31 -t84 31q44 0 76 -37q36 -42 70.5 -114t51.5 -134q210 -42 210 -99q0 -56 -194 -96q7 -81 -20 -160h214l-82 -225q63 -33 107.5 -96.5t65.5 -143.5t29 -151.5t8 -148.5z" /> - <glyph glyph-name="_507" unicode="" horiz-adv-x="2304" -d="M2301 500q12 -103 -22 -198.5t-99 -163.5t-158.5 -106t-196.5 -31q-161 11 -279.5 125t-134.5 274q-12 111 27.5 210.5t118.5 170.5l-71 107q-96 -80 -151 -194t-55 -244q0 -27 -18.5 -46.5t-45.5 -19.5h-256h-69q-23 -164 -149 -274t-294 -110q-185 0 -316.5 131.5 -t-131.5 316.5t131.5 316.5t316.5 131.5q76 0 152 -27l24 45q-123 110 -304 110h-64q-26 0 -45 19t-19 45t19 45t45 19h128q78 0 145 -13.5t116.5 -38.5t71.5 -39.5t51 -36.5h512h115l-85 128h-222q-30 0 -49 22.5t-14 52.5q4 23 23 38t43 15h253q33 0 53 -28l70 -105 -l114 114q19 19 46 19h101q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-179l115 -172q131 63 275 36q143 -26 244 -134.5t118 -253.5zM448 128q115 0 203 72.5t111 183.5h-314q-35 0 -55 31q-18 32 -1 63l147 277q-47 13 -91 13q-132 0 -226 -94t-94 -226t94 -226 -t226 -94zM1856 128q132 0 226 94t94 226t-94 226t-226 94q-60 0 -121 -24l174 -260q15 -23 10 -49t-27 -40q-15 -11 -36 -11q-35 0 -53 29l-174 260q-93 -95 -93 -225q0 -132 94 -226t226 -94z" /> - <glyph glyph-name="_508" unicode="" -d="M1408 0q0 -63 -61.5 -113.5t-164 -81t-225 -46t-253.5 -15.5t-253.5 15.5t-225 46t-164 81t-61.5 113.5q0 49 33 88.5t91 66.5t118 44.5t131 29.5q26 5 48 -10.5t26 -41.5q5 -26 -10.5 -48t-41.5 -26q-58 -10 -106 -23.5t-76.5 -25.5t-48.5 -23.5t-27.5 -19.5t-8.5 -12 -q3 -11 27 -26.5t73 -33t114 -32.5t160.5 -25t201.5 -10t201.5 10t160.5 25t114 33t73 33.5t27 27.5q-1 4 -8.5 11t-27.5 19t-48.5 23.5t-76.5 25t-106 23.5q-26 4 -41.5 26t-10.5 48q4 26 26 41.5t48 10.5q71 -12 131 -29.5t118 -44.5t91 -66.5t33 -88.5zM1024 896v-384 -q0 -26 -19 -45t-45 -19h-64v-384q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v384h-64q-26 0 -45 19t-19 45v384q0 53 37.5 90.5t90.5 37.5h384q53 0 90.5 -37.5t37.5 -90.5zM928 1280q0 -93 -65.5 -158.5t-158.5 -65.5t-158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5 -t158.5 -65.5t65.5 -158.5z" /> - <glyph glyph-name="_509" unicode="" horiz-adv-x="1792" -d="M1280 512h305q-5 -6 -10 -10.5t-9 -7.5l-3 -4l-623 -600q-18 -18 -44 -18t-44 18l-624 602q-5 2 -21 20h369q22 0 39.5 13.5t22.5 34.5l70 281l190 -667q6 -20 23 -33t39 -13q21 0 38 13t23 33l146 485l56 -112q18 -35 57 -35zM1792 940q0 -145 -103 -300h-369l-111 221 -q-8 17 -25.5 27t-36.5 8q-45 -5 -56 -46l-129 -430l-196 686q-6 20 -23.5 33t-39.5 13t-39 -13.5t-22 -34.5l-116 -464h-423q-103 155 -103 300q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5q224 0 351 -124 -t127 -344z" /> - <glyph glyph-name="venus" unicode="" horiz-adv-x="1280" -d="M1152 960q0 -221 -147.5 -384.5t-364.5 -187.5v-260h224q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-224v-224q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v224h-224q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v260q-150 16 -271.5 103t-186 224t-52.5 292 -q11 134 80.5 249t182 188t245.5 88q170 19 319 -54t236 -212t87 -306zM128 960q0 -185 131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5z" /> - <glyph glyph-name="_511" unicode="" -d="M1472 1408q26 0 45 -19t19 -45v-416q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v262l-382 -383q126 -156 126 -359q0 -117 -45.5 -223.5t-123 -184t-184 -123t-223.5 -45.5t-223.5 45.5t-184 123t-123 184t-45.5 223.5t45.5 223.5t123 184t184 123t223.5 45.5 -q203 0 359 -126l382 382h-261q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h416zM576 0q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" /> - <glyph glyph-name="_512" unicode="" horiz-adv-x="1280" -d="M830 1220q145 -72 233.5 -210.5t88.5 -305.5q0 -221 -147.5 -384.5t-364.5 -187.5v-132h96q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-96v-96q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v96h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h96v132q-217 24 -364.5 187.5 -t-147.5 384.5q0 167 88.5 305.5t233.5 210.5q-165 96 -228 273q-6 16 3.5 29.5t26.5 13.5h69q21 0 29 -20q44 -106 140 -171t214 -65t214 65t140 171q8 20 37 20h61q17 0 26.5 -13.5t3.5 -29.5q-63 -177 -228 -273zM576 256q185 0 316.5 131.5t131.5 316.5t-131.5 316.5 -t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" /> - <glyph glyph-name="_513" unicode="" -d="M1024 1504q0 14 9 23t23 9h288q26 0 45 -19t19 -45v-288q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v134l-254 -255q126 -158 126 -359q0 -221 -147.5 -384.5t-364.5 -187.5v-132h96q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-96v-96q0 -14 -9 -23t-23 -9h-64 -q-14 0 -23 9t-9 23v96h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h96v132q-149 16 -270.5 103t-186.5 223.5t-53 291.5q16 204 160 353.5t347 172.5q118 14 228 -19t198 -103l255 254h-134q-14 0 -23 9t-9 23v64zM576 256q185 0 316.5 131.5t131.5 316.5t-131.5 316.5 -t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" /> - <glyph glyph-name="_514" unicode="" horiz-adv-x="1792" -d="M1280 1504q0 14 9 23t23 9h288q26 0 45 -19t19 -45v-288q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v134l-254 -255q126 -158 126 -359q0 -221 -147.5 -384.5t-364.5 -187.5v-132h96q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-96v-96q0 -14 -9 -23t-23 -9h-64 -q-14 0 -23 9t-9 23v96h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h96v132q-217 24 -364.5 187.5t-147.5 384.5q0 201 126 359l-52 53l-101 -111q-9 -10 -22 -10.5t-23 7.5l-48 44q-10 8 -10.5 21.5t8.5 23.5l105 115l-111 112v-134q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9 -t-9 23v288q0 26 19 45t45 19h288q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-133l106 -107l86 94q9 10 22 10.5t23 -7.5l48 -44q10 -8 10.5 -21.5t-8.5 -23.5l-90 -99l57 -56q158 126 359 126t359 -126l255 254h-134q-14 0 -23 9t-9 23v64zM832 256q185 0 316.5 131.5 -t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" /> - <glyph glyph-name="_515" unicode="" horiz-adv-x="1792" -d="M1790 1007q12 -155 -52.5 -292t-186 -224t-271.5 -103v-260h224q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-224v-224q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v224h-512v-224q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v224h-224q-14 0 -23 9t-9 23v64q0 14 9 23 -t23 9h224v260q-150 16 -271.5 103t-186 224t-52.5 292q17 206 164.5 356.5t352.5 169.5q206 21 377 -94q171 115 377 94q205 -19 352.5 -169.5t164.5 -356.5zM896 647q128 131 128 313t-128 313q-128 -131 -128 -313t128 -313zM576 512q115 0 218 57q-154 165 -154 391 -q0 224 154 391q-103 57 -218 57q-185 0 -316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5zM1152 128v260q-137 15 -256 94q-119 -79 -256 -94v-260h512zM1216 512q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5q-115 0 -218 -57q154 -167 154 -391 -q0 -226 -154 -391q103 -57 218 -57z" /> - <glyph glyph-name="_516" unicode="" horiz-adv-x="1920" -d="M1536 1120q0 14 9 23t23 9h288q26 0 45 -19t19 -45v-288q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v134l-254 -255q76 -95 107.5 -214t9.5 -247q-31 -182 -166 -312t-318 -156q-210 -29 -384.5 80t-241.5 300q-117 6 -221 57.5t-177.5 133t-113.5 192.5t-32 230 -q9 135 78 252t182 191.5t248 89.5q118 14 227.5 -19t198.5 -103l255 254h-134q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h288q26 0 45 -19t19 -45v-288q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v134l-254 -255q59 -74 93 -169q182 -9 328 -124l255 254h-134q-14 0 -23 9 -t-9 23v64zM1024 704q0 20 -4 58q-162 -25 -271 -150t-109 -292q0 -20 4 -58q162 25 271 150t109 292zM128 704q0 -168 111 -294t276 -149q-3 29 -3 59q0 210 135 369.5t338 196.5q-53 120 -163.5 193t-245.5 73q-185 0 -316.5 -131.5t-131.5 -316.5zM1088 -128 -q185 0 316.5 131.5t131.5 316.5q0 168 -111 294t-276 149q3 -28 3 -59q0 -210 -135 -369.5t-338 -196.5q53 -120 163.5 -193t245.5 -73z" /> - <glyph glyph-name="_517" unicode="" horiz-adv-x="2048" -d="M1664 1504q0 14 9 23t23 9h288q26 0 45 -19t19 -45v-288q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v134l-254 -255q76 -95 107.5 -214t9.5 -247q-32 -180 -164.5 -310t-313.5 -157q-223 -34 -409 90q-117 -78 -256 -93v-132h96q14 0 23 -9t9 -23v-64q0 -14 -9 -23 -t-23 -9h-96v-96q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v96h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h96v132q-155 17 -279.5 109.5t-187 237.5t-39.5 307q25 187 159.5 322.5t320.5 164.5q224 34 410 -90q146 97 320 97q201 0 359 -126l255 254h-134q-14 0 -23 9 -t-9 23v64zM896 391q128 131 128 313t-128 313q-128 -131 -128 -313t128 -313zM128 704q0 -185 131.5 -316.5t316.5 -131.5q117 0 218 57q-154 167 -154 391t154 391q-101 57 -218 57q-185 0 -316.5 -131.5t-131.5 -316.5zM1216 256q185 0 316.5 131.5t131.5 316.5 -t-131.5 316.5t-316.5 131.5q-117 0 -218 -57q154 -167 154 -391t-154 -391q101 -57 218 -57z" /> - <glyph glyph-name="_518" unicode="" -d="M1472 1408q26 0 45 -19t19 -45v-416q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v262l-213 -214l140 -140q9 -10 9 -23t-9 -22l-46 -46q-9 -9 -22 -9t-23 9l-140 141l-78 -79q126 -156 126 -359q0 -117 -45.5 -223.5t-123 -184t-184 -123t-223.5 -45.5t-223.5 45.5 -t-184 123t-123 184t-45.5 223.5t45.5 223.5t123 184t184 123t223.5 45.5q203 0 359 -126l78 78l-172 172q-9 10 -9 23t9 22l46 46q9 9 22 9t23 -9l172 -172l213 213h-261q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h416zM576 0q185 0 316.5 131.5t131.5 316.5t-131.5 316.5 -t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" /> - <glyph glyph-name="_519" unicode="" horiz-adv-x="1280" -d="M640 892q217 -24 364.5 -187.5t147.5 -384.5q0 -167 -87 -306t-236 -212t-319 -54q-133 15 -245.5 88t-182 188t-80.5 249q-12 155 52.5 292t186 224t271.5 103v132h-160q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h160v165l-92 -92q-10 -9 -23 -9t-22 9l-46 46q-9 9 -9 22 -t9 23l202 201q19 19 45 19t45 -19l202 -201q9 -10 9 -23t-9 -22l-46 -46q-9 -9 -22 -9t-23 9l-92 92v-165h160q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-160v-132zM576 -128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5 -t131.5 -316.5t316.5 -131.5z" /> - <glyph glyph-name="_520" unicode="" horiz-adv-x="2048" -d="M1901 621q19 -19 19 -45t-19 -45l-294 -294q-9 -10 -22.5 -10t-22.5 10l-45 45q-10 9 -10 22.5t10 22.5l185 185h-294v-224q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v224h-132q-24 -217 -187.5 -364.5t-384.5 -147.5q-167 0 -306 87t-212 236t-54 319q15 133 88 245.5 -t188 182t249 80.5q155 12 292 -52.5t224 -186t103 -271.5h132v224q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-224h294l-185 185q-10 9 -10 22.5t10 22.5l45 45q9 10 22.5 10t22.5 -10zM576 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5 -t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" /> - <glyph glyph-name="_521" unicode="" horiz-adv-x="1280" -d="M1152 960q0 -221 -147.5 -384.5t-364.5 -187.5v-612q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v612q-217 24 -364.5 187.5t-147.5 384.5q0 117 45.5 223.5t123 184t184 123t223.5 45.5t223.5 -45.5t184 -123t123 -184t45.5 -223.5zM576 512q185 0 316.5 131.5 -t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" /> - <glyph glyph-name="_522" unicode="" horiz-adv-x="1280" -d="M1024 576q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1152 576q0 -117 -45.5 -223.5t-123 -184t-184 -123t-223.5 -45.5t-223.5 45.5t-184 123t-123 184t-45.5 223.5t45.5 223.5t123 184t184 123 -t223.5 45.5t223.5 -45.5t184 -123t123 -184t45.5 -223.5z" /> - <glyph glyph-name="_523" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="_524" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="_525" unicode="" -d="M1451 1408q35 0 60 -25t25 -60v-1366q0 -35 -25 -60t-60 -25h-391v595h199l30 232h-229v148q0 56 23.5 84t91.5 28l122 1v207q-63 9 -178 9q-136 0 -217.5 -80t-81.5 -226v-171h-200v-232h200v-595h-735q-35 0 -60 25t-25 60v1366q0 35 25 60t60 25h1366z" /> - <glyph glyph-name="_526" unicode="" horiz-adv-x="1280" -d="M0 939q0 108 37.5 203.5t103.5 166.5t152 123t185 78t202 26q158 0 294 -66.5t221 -193.5t85 -287q0 -96 -19 -188t-60 -177t-100 -149.5t-145 -103t-189 -38.5q-68 0 -135 32t-96 88q-10 -39 -28 -112.5t-23.5 -95t-20.5 -71t-26 -71t-32 -62.5t-46 -77.5t-62 -86.5 -l-14 -5l-9 10q-15 157 -15 188q0 92 21.5 206.5t66.5 287.5t52 203q-32 65 -32 169q0 83 52 156t132 73q61 0 95 -40.5t34 -102.5q0 -66 -44 -191t-44 -187q0 -63 45 -104.5t109 -41.5q55 0 102 25t78.5 68t56 95t38 110.5t20 111t6.5 99.5q0 173 -109.5 269.5t-285.5 96.5 -q-200 0 -334 -129.5t-134 -328.5q0 -44 12.5 -85t27 -65t27 -45.5t12.5 -30.5q0 -28 -15 -73t-37 -45q-2 0 -17 3q-51 15 -90.5 56t-61 94.5t-32.5 108t-11 106.5z" /> - <glyph glyph-name="_527" unicode="" -d="M985 562q13 0 97.5 -44t89.5 -53q2 -5 2 -15q0 -33 -17 -76q-16 -39 -71 -65.5t-102 -26.5q-57 0 -190 62q-98 45 -170 118t-148 185q-72 107 -71 194v8q3 91 74 158q24 22 52 22q6 0 18 -1.5t19 -1.5q19 0 26.5 -6.5t15.5 -27.5q8 -20 33 -88t25 -75q0 -21 -34.5 -57.5 -t-34.5 -46.5q0 -7 5 -15q34 -73 102 -137q56 -53 151 -101q12 -7 22 -7q15 0 54 48.5t52 48.5zM782 32q127 0 243.5 50t200.5 134t134 200.5t50 243.5t-50 243.5t-134 200.5t-200.5 134t-243.5 50t-243.5 -50t-200.5 -134t-134 -200.5t-50 -243.5q0 -203 120 -368l-79 -233 -l242 77q158 -104 345 -104zM782 1414q153 0 292.5 -60t240.5 -161t161 -240.5t60 -292.5t-60 -292.5t-161 -240.5t-240.5 -161t-292.5 -60q-195 0 -365 94l-417 -134l136 405q-108 178 -108 389q0 153 60 292.5t161 240.5t240.5 161t292.5 60z" /> - <glyph glyph-name="_528" unicode="" horiz-adv-x="1792" -d="M128 128h1024v128h-1024v-128zM128 640h1024v128h-1024v-128zM1696 192q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM128 1152h1024v128h-1024v-128zM1696 704q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM1696 1216 -q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM1792 384v-384h-1792v384h1792zM1792 896v-384h-1792v384h1792zM1792 1408v-384h-1792v384h1792z" /> - <glyph glyph-name="_529" unicode="" horiz-adv-x="2048" -d="M704 640q-159 0 -271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5t-112.5 -271.5t-271.5 -112.5zM1664 512h352q13 0 22.5 -9.5t9.5 -22.5v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-352v-352q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5 -t-9.5 22.5v352h-352q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h352v352q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5v-352zM928 288q0 -52 38 -90t90 -38h256v-238q-68 -50 -171 -50h-874q-121 0 -194 69t-73 190q0 53 3.5 103.5t14 109t26.5 108.5 -t43 97.5t62 81t85.5 53.5t111.5 20q19 0 39 -17q79 -61 154.5 -91.5t164.5 -30.5t164.5 30.5t154.5 91.5q20 17 39 17q132 0 217 -96h-223q-52 0 -90 -38t-38 -90v-192z" /> - <glyph glyph-name="_530" unicode="" horiz-adv-x="2048" -d="M704 640q-159 0 -271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5t-112.5 -271.5t-271.5 -112.5zM1781 320l249 -249q9 -9 9 -23q0 -13 -9 -22l-136 -136q-9 -9 -22 -9q-14 0 -23 9l-249 249l-249 -249q-9 -9 -23 -9q-13 0 -22 9l-136 136 -q-9 9 -9 22q0 14 9 23l249 249l-249 249q-9 9 -9 23q0 13 9 22l136 136q9 9 22 9q14 0 23 -9l249 -249l249 249q9 9 23 9q13 0 22 -9l136 -136q9 -9 9 -22q0 -14 -9 -23zM1283 320l-181 -181q-37 -37 -37 -91q0 -53 37 -90l83 -83q-21 -3 -44 -3h-874q-121 0 -194 69 -t-73 190q0 53 3.5 103.5t14 109t26.5 108.5t43 97.5t62 81t85.5 53.5t111.5 20q19 0 39 -17q154 -122 319 -122t319 122q20 17 39 17q28 0 57 -6q-28 -27 -41 -50t-13 -56q0 -54 37 -91z" /> - <glyph glyph-name="_531" unicode="" horiz-adv-x="2048" -d="M256 512h1728q26 0 45 -19t19 -45v-448h-256v256h-1536v-256h-256v1216q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-704zM832 832q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM2048 576v64q0 159 -112.5 271.5t-271.5 112.5h-704 -q-26 0 -45 -19t-19 -45v-384h1152z" /> - <glyph glyph-name="_532" unicode="" -d="M1536 1536l-192 -448h192v-192h-274l-55 -128h329v-192h-411l-357 -832l-357 832h-411v192h329l-55 128h-274v192h192l-192 448h256l323 -768h378l323 768h256zM768 320l108 256h-216z" /> - <glyph glyph-name="_533" unicode="" -d="M1088 1536q185 0 316.5 -93.5t131.5 -226.5v-896q0 -130 -125.5 -222t-305.5 -97l213 -202q16 -15 8 -35t-30 -20h-1056q-22 0 -30 20t8 35l213 202q-180 5 -305.5 97t-125.5 222v896q0 133 131.5 226.5t316.5 93.5h640zM768 192q80 0 136 56t56 136t-56 136t-136 56 -t-136 -56t-56 -136t56 -136t136 -56zM1344 768v512h-1152v-512h1152z" /> - <glyph glyph-name="_534" unicode="" -d="M1088 1536q185 0 316.5 -93.5t131.5 -226.5v-896q0 -130 -125.5 -222t-305.5 -97l213 -202q16 -15 8 -35t-30 -20h-1056q-22 0 -30 20t8 35l213 202q-180 5 -305.5 97t-125.5 222v896q0 133 131.5 226.5t316.5 93.5h640zM288 224q66 0 113 47t47 113t-47 113t-113 47 -t-113 -47t-47 -113t47 -113t113 -47zM704 768v512h-544v-512h544zM1248 224q66 0 113 47t47 113t-47 113t-113 47t-113 -47t-47 -113t47 -113t113 -47zM1408 768v512h-576v-512h576z" /> - <glyph glyph-name="_535" unicode="" horiz-adv-x="1792" -d="M597 1115v-1173q0 -25 -12.5 -42.5t-36.5 -17.5q-17 0 -33 8l-465 233q-21 10 -35.5 33.5t-14.5 46.5v1140q0 20 10 34t29 14q14 0 44 -15l511 -256q3 -3 3 -5zM661 1014l534 -866l-534 266v600zM1792 996v-1054q0 -25 -14 -40.5t-38 -15.5t-47 13l-441 220zM1789 1116 -q0 -3 -256.5 -419.5t-300.5 -487.5l-390 634l324 527q17 28 52 28q14 0 26 -6l541 -270q4 -2 4 -6z" /> - <glyph glyph-name="_536" unicode="" -d="M809 532l266 499h-112l-157 -312q-24 -48 -44 -92l-42 92l-155 312h-120l263 -493v-324h101v318zM1536 1408v-1536h-1536v1536h1536z" /> - <glyph glyph-name="_537" unicode="" horiz-adv-x="2296" -d="M478 -139q-8 -16 -27 -34.5t-37 -25.5q-25 -9 -51.5 3.5t-28.5 31.5q-1 22 40 55t68 38q23 4 34 -21.5t2 -46.5zM1819 -139q7 -16 26 -34.5t38 -25.5q25 -9 51.5 3.5t27.5 31.5q2 22 -39.5 55t-68.5 38q-22 4 -33 -21.5t-2 -46.5zM1867 -30q13 -27 56.5 -59.5t77.5 -41.5 -q45 -13 82 4.5t37 50.5q0 46 -67.5 100.5t-115.5 59.5q-40 5 -63.5 -37.5t-6.5 -76.5zM428 -30q-13 -27 -56 -59.5t-77 -41.5q-45 -13 -82 4.5t-37 50.5q0 46 67.5 100.5t115.5 59.5q40 5 63 -37.5t6 -76.5zM1158 1094h1q-41 0 -76 -15q27 -8 44 -30.5t17 -49.5 -q0 -35 -27 -60t-65 -25q-52 0 -80 43q-5 -23 -5 -42q0 -74 56 -126.5t135 -52.5q80 0 136 52.5t56 126.5t-56 126.5t-136 52.5zM1462 1312q-99 109 -220.5 131.5t-245.5 -44.5q27 60 82.5 96.5t118 39.5t121.5 -17t99.5 -74.5t44.5 -131.5zM2212 73q8 -11 -11 -42 -q7 -23 7 -40q1 -56 -44.5 -112.5t-109.5 -91.5t-118 -37q-48 -2 -92 21.5t-66 65.5q-687 -25 -1259 0q-23 -41 -66.5 -65t-92.5 -22q-86 3 -179.5 80.5t-92.5 160.5q2 22 7 40q-19 31 -11 42q6 10 31 1q14 22 41 51q-7 29 2 38q11 10 39 -4q29 20 59 34q0 29 13 37 -q23 12 51 -16q35 5 61 -2q18 -4 38 -19v73q-11 0 -18 2q-53 10 -97 44.5t-55 87.5q-9 38 0 81q15 62 93 95q2 17 19 35.5t36 23.5t33 -7.5t19 -30.5h13q46 -5 60 -23q3 -3 5 -7q10 1 30.5 3.5t30.5 3.5q-15 11 -30 17q-23 40 -91 43q0 6 1 10q-62 2 -118.5 18.5t-84.5 47.5 -q-32 36 -42.5 92t-2.5 112q16 126 90 179q23 16 52 4.5t32 -40.5q0 -1 1.5 -14t2.5 -21t3 -20t5.5 -19t8.5 -10q27 -14 76 -12q48 46 98 74q-40 4 -162 -14l47 46q61 58 163 111q145 73 282 86q-20 8 -41 15.5t-47 14t-42.5 10.5t-47.5 11t-43 10q595 126 904 -139 -q98 -84 158 -222q85 -10 121 9h1q5 3 8.5 10t5.5 19t3 19.5t3 21.5l1 14q3 28 32 40t52 -5q73 -52 91 -178q7 -57 -3.5 -113t-42.5 -91q-28 -32 -83.5 -48.5t-115.5 -18.5v-10q-71 -2 -95 -43q-14 -5 -31 -17q11 -1 32 -3.5t30 -3.5q1 5 5 8q16 18 60 23h13q5 18 19 30t33 8 -t36 -23t19 -36q79 -32 93 -95q9 -40 1 -81q-12 -53 -56 -88t-97 -44q-10 -2 -17 -2q0 -49 -1 -73q20 15 38 19q26 7 61 2q28 28 51 16q14 -9 14 -37q33 -16 59 -34q27 13 38 4q10 -10 2 -38q28 -30 41 -51q23 8 31 -1zM1937 1025q0 -29 -9 -54q82 -32 112 -132 -q4 37 -9.5 98.5t-41.5 90.5q-20 19 -36 17t-16 -20zM1859 925q35 -42 47.5 -108.5t-0.5 -124.5q67 13 97 45q13 14 18 28q-3 64 -31 114.5t-79 66.5q-15 -15 -52 -21zM1822 921q-30 0 -44 1q42 -115 53 -239q21 0 43 3q16 68 1 135t-53 100zM258 839q30 100 112 132 -q-9 25 -9 54q0 18 -16.5 20t-35.5 -17q-28 -29 -41.5 -90.5t-9.5 -98.5zM294 737q29 -31 97 -45q-13 58 -0.5 124.5t47.5 108.5v0q-37 6 -52 21q-51 -16 -78.5 -66t-31.5 -115q9 -17 18 -28zM471 683q14 124 73 235q-19 -4 -55 -18l-45 -19v1q-46 -89 -20 -196q25 -3 47 -3z -M1434 644q8 -38 16.5 -108.5t11.5 -89.5q3 -18 9.5 -21.5t23.5 4.5q40 20 62 85.5t23 125.5q-24 2 -146 4zM1152 1285q-116 0 -199 -82.5t-83 -198.5q0 -117 83 -199.5t199 -82.5t199 82.5t83 199.5q0 116 -83 198.5t-199 82.5zM1380 646q-105 2 -211 0v1q-1 -27 2.5 -86 -t13.5 -66q29 -14 93.5 -14.5t95.5 10.5q9 3 11 39t-0.5 69.5t-4.5 46.5zM1112 447q8 4 9.5 48t-0.5 88t-4 63v1q-212 -3 -214 -3q-4 -20 -7 -62t0 -83t14 -46q34 -15 101 -16t101 10zM718 636q-16 -59 4.5 -118.5t77.5 -84.5q15 -8 24 -5t12 21q3 16 8 90t10 103 -q-69 -2 -136 -6zM591 510q3 -23 -34 -36q132 -141 271.5 -240t305.5 -154q172 49 310.5 146t293.5 250q-33 13 -30 34q0 2 0.5 3.5t1.5 3t1 2.5v1v-1q-17 2 -50 5.5t-48 4.5q-26 -90 -82 -132q-51 -38 -82 1q-5 6 -9 14q-7 13 -17 62q-2 -5 -5 -9t-7.5 -7t-8 -5.5t-9.5 -4 -l-10 -2.5t-12 -2l-12 -1.5t-13.5 -1t-13.5 -0.5q-106 -9 -163 11q-4 -17 -10 -26.5t-21 -15t-23 -7t-36 -3.5q-6 -1 -9 -1q-179 -17 -203 40q-2 -63 -56 -54q-47 8 -91 54q-12 13 -20 26q-17 29 -26 65q-58 -6 -87 -10q1 -2 4 -10zM507 -118q3 14 3 30q-17 71 -51 130 -t-73 70q-41 12 -101.5 -14.5t-104.5 -80t-39 -107.5q35 -53 100 -93t119 -42q51 -2 94 28t53 79zM510 53q23 -63 27 -119q195 113 392 174q-98 52 -180.5 120t-179.5 165q-6 -4 -29 -13q0 -1 -1 -4t-1 -5q31 -18 22 -37q-12 -23 -56 -34q-10 -13 -29 -24h-1q-2 -83 1 -150 -q19 -34 35 -73zM579 -113q532 -21 1145 0q-254 147 -428 196q-76 -35 -156 -57q-8 -3 -16 0q-65 21 -129 49q-208 -60 -416 -188h-1v-1q1 0 1 1zM1763 -67q4 54 28 120q14 38 33 71l-1 -1q3 77 3 153q-15 8 -30 25q-42 9 -56 33q-9 20 22 38q-2 4 -2 9q-16 4 -28 12 -q-204 -190 -383 -284q198 -59 414 -176zM2155 -90q5 54 -39 107.5t-104 80t-102 14.5q-38 -11 -72.5 -70.5t-51.5 -129.5q0 -16 3 -30q10 -49 53 -79t94 -28q54 2 119 42t100 93z" /> - <glyph glyph-name="_538" unicode="" horiz-adv-x="2304" -d="M1524 -25q0 -68 -48 -116t-116 -48t-116.5 48t-48.5 116t48.5 116.5t116.5 48.5t116 -48.5t48 -116.5zM775 -25q0 -68 -48.5 -116t-116.5 -48t-116 48t-48 116t48 116.5t116 48.5t116.5 -48.5t48.5 -116.5zM0 1469q57 -60 110.5 -104.5t121 -82t136 -63t166 -45.5 -t200 -31.5t250 -18.5t304 -9.5t372.5 -2.5q139 0 244.5 -5t181 -16.5t124 -27.5t71 -39.5t24 -51.5t-19.5 -64t-56.5 -76.5t-89.5 -91t-116 -104.5t-139 -119q-185 -157 -286 -247q29 51 76.5 109t94 105.5t94.5 98.5t83 91.5t54 80.5t13 70t-45.5 55.5t-116.5 41t-204 23.5 -t-304 5q-168 -2 -314 6t-256 23t-204.5 41t-159.5 51.5t-122.5 62.5t-91.5 66.5t-68 71.5t-50.5 69.5t-40 68t-36.5 59.5z" /> - <glyph glyph-name="_539" unicode="" horiz-adv-x="1792" -d="M896 1472q-169 0 -323 -66t-265.5 -177.5t-177.5 -265.5t-66 -323t66 -323t177.5 -265.5t265.5 -177.5t323 -66t323 66t265.5 177.5t177.5 265.5t66 323t-66 323t-177.5 265.5t-265.5 177.5t-323 66zM896 1536q182 0 348 -71t286 -191t191 -286t71 -348t-71 -348 -t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71zM496 704q16 0 16 -16v-480q0 -16 -16 -16h-32q-16 0 -16 16v480q0 16 16 16h32zM896 640q53 0 90.5 -37.5t37.5 -90.5q0 -35 -17.5 -64t-46.5 -46v-114q0 -14 -9 -23 -t-23 -9h-64q-14 0 -23 9t-9 23v114q-29 17 -46.5 46t-17.5 64q0 53 37.5 90.5t90.5 37.5zM896 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM544 928v-96 -q0 -14 9 -23t23 -9h64q14 0 23 9t9 23v96q0 93 65.5 158.5t158.5 65.5t158.5 -65.5t65.5 -158.5v-96q0 -14 9 -23t23 -9h64q14 0 23 9t9 23v96q0 146 -103 249t-249 103t-249 -103t-103 -249zM1408 192v512q0 26 -19 45t-45 19h-896q-26 0 -45 -19t-19 -45v-512 -q0 -26 19 -45t45 -19h896q26 0 45 19t19 45z" /> - <glyph glyph-name="_540" unicode="" horiz-adv-x="2304" -d="M1920 1024v-768h-1664v768h1664zM2048 448h128v384h-128v288q0 14 -9 23t-23 9h-1856q-14 0 -23 -9t-9 -23v-960q0 -14 9 -23t23 -9h1856q14 0 23 9t9 23v288zM2304 832v-384q0 -53 -37.5 -90.5t-90.5 -37.5v-160q0 -66 -47 -113t-113 -47h-1856q-66 0 -113 47t-47 113 -v960q0 66 47 113t113 47h1856q66 0 113 -47t47 -113v-160q53 0 90.5 -37.5t37.5 -90.5z" /> - <glyph glyph-name="_541" unicode="" horiz-adv-x="2304" -d="M256 256v768h1280v-768h-1280zM2176 960q53 0 90.5 -37.5t37.5 -90.5v-384q0 -53 -37.5 -90.5t-90.5 -37.5v-160q0 -66 -47 -113t-113 -47h-1856q-66 0 -113 47t-47 113v960q0 66 47 113t113 47h1856q66 0 113 -47t47 -113v-160zM2176 448v384h-128v288q0 14 -9 23t-23 9 -h-1856q-14 0 -23 -9t-9 -23v-960q0 -14 9 -23t23 -9h1856q14 0 23 9t9 23v288h128z" /> - <glyph glyph-name="_542" unicode="" horiz-adv-x="2304" -d="M256 256v768h896v-768h-896zM2176 960q53 0 90.5 -37.5t37.5 -90.5v-384q0 -53 -37.5 -90.5t-90.5 -37.5v-160q0 -66 -47 -113t-113 -47h-1856q-66 0 -113 47t-47 113v960q0 66 47 113t113 47h1856q66 0 113 -47t47 -113v-160zM2176 448v384h-128v288q0 14 -9 23t-23 9 -h-1856q-14 0 -23 -9t-9 -23v-960q0 -14 9 -23t23 -9h1856q14 0 23 9t9 23v288h128z" /> - <glyph glyph-name="_543" unicode="" horiz-adv-x="2304" -d="M256 256v768h512v-768h-512zM2176 960q53 0 90.5 -37.5t37.5 -90.5v-384q0 -53 -37.5 -90.5t-90.5 -37.5v-160q0 -66 -47 -113t-113 -47h-1856q-66 0 -113 47t-47 113v960q0 66 47 113t113 47h1856q66 0 113 -47t47 -113v-160zM2176 448v384h-128v288q0 14 -9 23t-23 9 -h-1856q-14 0 -23 -9t-9 -23v-960q0 -14 9 -23t23 -9h1856q14 0 23 9t9 23v288h128z" /> - <glyph glyph-name="_544" unicode="" horiz-adv-x="2304" -d="M2176 960q53 0 90.5 -37.5t37.5 -90.5v-384q0 -53 -37.5 -90.5t-90.5 -37.5v-160q0 -66 -47 -113t-113 -47h-1856q-66 0 -113 47t-47 113v960q0 66 47 113t113 47h1856q66 0 113 -47t47 -113v-160zM2176 448v384h-128v288q0 14 -9 23t-23 9h-1856q-14 0 -23 -9t-9 -23 -v-960q0 -14 9 -23t23 -9h1856q14 0 23 9t9 23v288h128z" /> - <glyph glyph-name="_545" unicode="" horiz-adv-x="1280" -d="M1133 493q31 -30 14 -69q-17 -40 -59 -40h-382l201 -476q10 -25 0 -49t-34 -35l-177 -75q-25 -10 -49 0t-35 34l-191 452l-312 -312q-19 -19 -45 -19q-12 0 -24 5q-40 17 -40 59v1504q0 42 40 59q12 5 24 5q27 0 45 -19z" /> - <glyph glyph-name="_546" unicode="" horiz-adv-x="1024" -d="M832 1408q-320 0 -320 -224v-416h128v-128h-128v-544q0 -224 320 -224h64v-128h-64q-272 0 -384 146q-112 -146 -384 -146h-64v128h64q320 0 320 224v544h-128v128h128v416q0 224 -320 224h-64v128h64q272 0 384 -146q112 146 384 146h64v-128h-64z" /> - <glyph glyph-name="_547" unicode="" horiz-adv-x="2048" -d="M2048 1152h-128v-1024h128v-384h-384v128h-1280v-128h-384v384h128v1024h-128v384h384v-128h1280v128h384v-384zM1792 1408v-128h128v128h-128zM128 1408v-128h128v128h-128zM256 -128v128h-128v-128h128zM1664 0v128h128v1024h-128v128h-1280v-128h-128v-1024h128v-128 -h1280zM1920 -128v128h-128v-128h128zM1280 896h384v-768h-896v256h-384v768h896v-256zM512 512h640v512h-640v-512zM1536 256v512h-256v-384h-384v-128h640z" /> - <glyph glyph-name="_548" unicode="" horiz-adv-x="2304" -d="M2304 768h-128v-640h128v-384h-384v128h-896v-128h-384v384h128v128h-384v-128h-384v384h128v640h-128v384h384v-128h896v128h384v-384h-128v-128h384v128h384v-384zM2048 1024v-128h128v128h-128zM1408 1408v-128h128v128h-128zM128 1408v-128h128v128h-128zM256 256 -v128h-128v-128h128zM1536 384h-128v-128h128v128zM384 384h896v128h128v640h-128v128h-896v-128h-128v-640h128v-128zM896 -128v128h-128v-128h128zM2176 -128v128h-128v-128h128zM2048 128v640h-128v128h-384v-384h128v-384h-384v128h-384v-128h128v-128h896v128h128z" /> - <glyph glyph-name="_549" unicode="" -d="M1024 288v-416h-928q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h1344q40 0 68 -28t28 -68v-928h-416q-40 0 -68 -28t-28 -68zM1152 256h381q-15 -82 -65 -132l-184 -184q-50 -50 -132 -65v381z" /> - <glyph glyph-name="_550" unicode="" -d="M1400 256h-248v-248q29 10 41 22l185 185q12 12 22 41zM1120 384h288v896h-1280v-1280h896v288q0 40 28 68t68 28zM1536 1312v-1024q0 -40 -20 -88t-48 -76l-184 -184q-28 -28 -76 -48t-88 -20h-1024q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h1344q40 0 68 -28t28 -68 -z" /> - <glyph glyph-name="_551" unicode="" horiz-adv-x="2304" -d="M1951 538q0 -26 -15.5 -44.5t-38.5 -23.5q-8 -2 -18 -2h-153v140h153q10 0 18 -2q23 -5 38.5 -23.5t15.5 -44.5zM1933 751q0 -25 -15 -42t-38 -21q-3 -1 -15 -1h-139v129h139q3 0 8.5 -0.5t6.5 -0.5q23 -4 38 -21.5t15 -42.5zM728 587v308h-228v-308q0 -58 -38 -94.5 -t-105 -36.5q-108 0 -229 59v-112q53 -15 121 -23t109 -9l42 -1q328 0 328 217zM1442 403v113q-99 -52 -200 -59q-108 -8 -169 41t-61 142t61 142t169 41q101 -7 200 -58v112q-48 12 -100 19.5t-80 9.5l-28 2q-127 6 -218.5 -14t-140.5 -60t-71 -88t-22 -106t22 -106t71 -88 -t140.5 -60t218.5 -14q101 4 208 31zM2176 518q0 54 -43 88.5t-109 39.5v3q57 8 89 41.5t32 79.5q0 55 -41 88t-107 36q-3 0 -12 0.5t-14 0.5h-455v-510h491q74 0 121.5 36.5t47.5 96.5zM2304 1280v-1280q0 -52 -38 -90t-90 -38h-2048q-52 0 -90 38t-38 90v1280q0 52 38 90 -t90 38h2048q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_552" unicode="" horiz-adv-x="2304" -d="M858 295v693q-106 -41 -172 -135.5t-66 -211.5t66 -211.5t172 -134.5zM1362 641q0 117 -66 211.5t-172 135.5v-694q106 41 172 135.5t66 211.5zM1577 641q0 -159 -78.5 -294t-213.5 -213.5t-294 -78.5q-119 0 -227.5 46.5t-187 125t-125 187t-46.5 227.5q0 159 78.5 294 -t213.5 213.5t294 78.5t294 -78.5t213.5 -213.5t78.5 -294zM1960 634q0 139 -55.5 261.5t-147.5 205.5t-213.5 131t-252.5 48h-301q-176 0 -323.5 -81t-235 -230t-87.5 -335q0 -171 87 -317.5t236 -231.5t323 -85h301q129 0 251.5 50.5t214.5 135t147.5 202.5t55.5 246z -M2304 1280v-1280q0 -52 -38 -90t-90 -38h-2048q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h2048q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_553" unicode="" horiz-adv-x="1792" -d="M1664 -96v1088q0 13 -9.5 22.5t-22.5 9.5h-1088q-13 0 -22.5 -9.5t-9.5 -22.5v-1088q0 -13 9.5 -22.5t22.5 -9.5h1088q13 0 22.5 9.5t9.5 22.5zM1792 992v-1088q0 -66 -47 -113t-113 -47h-1088q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1088q66 0 113 -47t47 -113 -zM1408 1376v-160h-128v160q0 13 -9.5 22.5t-22.5 9.5h-1088q-13 0 -22.5 -9.5t-9.5 -22.5v-1088q0 -13 9.5 -22.5t22.5 -9.5h160v-128h-160q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1088q66 0 113 -47t47 -113z" /> - <glyph glyph-name="_554" unicode="" horiz-adv-x="2304" -d="M1728 1088l-384 -704h768zM448 1088l-384 -704h768zM1269 1280q-14 -40 -45.5 -71.5t-71.5 -45.5v-1291h608q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-1344q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h608v1291q-40 14 -71.5 45.5t-45.5 71.5h-491q-14 0 -23 9t-9 23v64 -q0 14 9 23t23 9h491q21 57 70 92.5t111 35.5t111 -35.5t70 -92.5h491q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-491zM1088 1264q33 0 56.5 23.5t23.5 56.5t-23.5 56.5t-56.5 23.5t-56.5 -23.5t-23.5 -56.5t23.5 -56.5t56.5 -23.5zM2176 384q0 -73 -46.5 -131t-117.5 -91 -t-144.5 -49.5t-139.5 -16.5t-139.5 16.5t-144.5 49.5t-117.5 91t-46.5 131q0 11 35 81t92 174.5t107 195.5t102 184t56 100q18 33 56 33t56 -33q4 -7 56 -100t102 -184t107 -195.5t92 -174.5t35 -81zM896 384q0 -73 -46.5 -131t-117.5 -91t-144.5 -49.5t-139.5 -16.5 -t-139.5 16.5t-144.5 49.5t-117.5 91t-46.5 131q0 11 35 81t92 174.5t107 195.5t102 184t56 100q18 33 56 33t56 -33q4 -7 56 -100t102 -184t107 -195.5t92 -174.5t35 -81z" /> - <glyph glyph-name="_555" unicode="" -d="M1408 1408q0 -261 -106.5 -461.5t-266.5 -306.5q160 -106 266.5 -306.5t106.5 -461.5h96q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-1472q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h96q0 261 106.5 461.5t266.5 306.5q-160 106 -266.5 306.5t-106.5 461.5h-96q-14 0 -23 9 -t-9 23v64q0 14 9 23t23 9h1472q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-96zM874 700q77 29 149 92.5t129.5 152.5t92.5 210t35 253h-1024q0 -132 35 -253t92.5 -210t129.5 -152.5t149 -92.5q19 -7 30.5 -23.5t11.5 -36.5t-11.5 -36.5t-30.5 -23.5q-77 -29 -149 -92.5 -t-129.5 -152.5t-92.5 -210t-35 -253h1024q0 132 -35 253t-92.5 210t-129.5 152.5t-149 92.5q-19 7 -30.5 23.5t-11.5 36.5t11.5 36.5t30.5 23.5z" /> - <glyph glyph-name="_556" unicode="" -d="M1408 1408q0 -261 -106.5 -461.5t-266.5 -306.5q160 -106 266.5 -306.5t106.5 -461.5h96q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-1472q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h96q0 261 106.5 461.5t266.5 306.5q-160 106 -266.5 306.5t-106.5 461.5h-96q-14 0 -23 9 -t-9 23v64q0 14 9 23t23 9h1472q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-96zM1280 1408h-1024q0 -66 9 -128h1006q9 61 9 128zM1280 -128q0 130 -34 249.5t-90.5 208t-126.5 152t-146 94.5h-230q-76 -31 -146 -94.5t-126.5 -152t-90.5 -208t-34 -249.5h1024z" /> - <glyph glyph-name="_557" unicode="" -d="M1408 1408q0 -261 -106.5 -461.5t-266.5 -306.5q160 -106 266.5 -306.5t106.5 -461.5h96q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-1472q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h96q0 261 106.5 461.5t266.5 306.5q-160 106 -266.5 306.5t-106.5 461.5h-96q-14 0 -23 9 -t-9 23v64q0 14 9 23t23 9h1472q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-96zM1280 1408h-1024q0 -206 85 -384h854q85 178 85 384zM1223 192q-54 141 -145.5 241.5t-194.5 142.5h-230q-103 -42 -194.5 -142.5t-145.5 -241.5h910z" /> - <glyph glyph-name="_558" unicode="" -d="M1408 1408q0 -261 -106.5 -461.5t-266.5 -306.5q160 -106 266.5 -306.5t106.5 -461.5h96q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-1472q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h96q0 261 106.5 461.5t266.5 306.5q-160 106 -266.5 306.5t-106.5 461.5h-96q-14 0 -23 9 -t-9 23v64q0 14 9 23t23 9h1472q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-96zM874 700q77 29 149 92.5t129.5 152.5t92.5 210t35 253h-1024q0 -132 35 -253t92.5 -210t129.5 -152.5t149 -92.5q19 -7 30.5 -23.5t11.5 -36.5t-11.5 -36.5t-30.5 -23.5q-137 -51 -244 -196 -h700q-107 145 -244 196q-19 7 -30.5 23.5t-11.5 36.5t11.5 36.5t30.5 23.5z" /> - <glyph glyph-name="_559" unicode="" -d="M1504 -64q14 0 23 -9t9 -23v-128q0 -14 -9 -23t-23 -9h-1472q-14 0 -23 9t-9 23v128q0 14 9 23t23 9h1472zM130 0q3 55 16 107t30 95t46 87t53.5 76t64.5 69.5t66 60t70.5 55t66.5 47.5t65 43q-43 28 -65 43t-66.5 47.5t-70.5 55t-66 60t-64.5 69.5t-53.5 76t-46 87 -t-30 95t-16 107h1276q-3 -55 -16 -107t-30 -95t-46 -87t-53.5 -76t-64.5 -69.5t-66 -60t-70.5 -55t-66.5 -47.5t-65 -43q43 -28 65 -43t66.5 -47.5t70.5 -55t66 -60t64.5 -69.5t53.5 -76t46 -87t30 -95t16 -107h-1276zM1504 1536q14 0 23 -9t9 -23v-128q0 -14 -9 -23t-23 -9 -h-1472q-14 0 -23 9t-9 23v128q0 14 9 23t23 9h1472z" /> - <glyph glyph-name="_560" unicode="" -d="M768 1152q-53 0 -90.5 -37.5t-37.5 -90.5v-128h-32v93q0 48 -32 81.5t-80 33.5q-46 0 -79 -33t-33 -79v-429l-32 30v172q0 48 -32 81.5t-80 33.5q-46 0 -79 -33t-33 -79v-224q0 -47 35 -82l310 -296q39 -39 39 -102q0 -26 19 -45t45 -19h640q26 0 45 19t19 45v25 -q0 41 10 77l108 436q10 36 10 77v246q0 48 -32 81.5t-80 33.5q-46 0 -79 -33t-33 -79v-32h-32v125q0 40 -25 72.5t-64 40.5q-14 2 -23 2q-46 0 -79 -33t-33 -79v-128h-32v122q0 51 -32.5 89.5t-82.5 43.5q-5 1 -13 1zM768 1280q84 0 149 -50q57 34 123 34q59 0 111 -27 -t86 -76q27 7 59 7q100 0 170 -71.5t70 -171.5v-246q0 -51 -13 -108l-109 -436q-6 -24 -6 -71q0 -80 -56 -136t-136 -56h-640q-84 0 -138 58.5t-54 142.5l-308 296q-76 73 -76 175v224q0 99 70.5 169.5t169.5 70.5q11 0 16 -1q6 95 75.5 160t164.5 65q52 0 98 -21 -q72 69 174 69z" /> - <glyph glyph-name="_561" unicode="" horiz-adv-x="1792" -d="M880 1408q-46 0 -79 -33t-33 -79v-656h-32v528q0 46 -33 79t-79 33t-79 -33t-33 -79v-528v-256l-154 205q-38 51 -102 51q-53 0 -90.5 -37.5t-37.5 -90.5q0 -43 26 -77l384 -512q38 -51 102 -51h688q34 0 61 22t34 56l76 405q5 32 5 59v498q0 46 -33 79t-79 33t-79 -33 -t-33 -79v-272h-32v528q0 46 -33 79t-79 33t-79 -33t-33 -79v-528h-32v656q0 46 -33 79t-79 33zM880 1536q68 0 125.5 -35.5t88.5 -96.5q19 4 42 4q99 0 169.5 -70.5t70.5 -169.5v-17q105 6 180.5 -64t75.5 -175v-498q0 -40 -8 -83l-76 -404q-14 -79 -76.5 -131t-143.5 -52 -h-688q-60 0 -114.5 27.5t-90.5 74.5l-384 512q-51 68 -51 154q0 106 75 181t181 75q78 0 128 -34v434q0 99 70.5 169.5t169.5 70.5q23 0 42 -4q31 61 88.5 96.5t125.5 35.5z" /> - <glyph glyph-name="_562" unicode="" horiz-adv-x="1792" -d="M1073 -128h-177q-163 0 -226 141q-23 49 -23 102v5q-62 30 -98.5 88.5t-36.5 127.5q0 38 5 48h-261q-106 0 -181 75t-75 181t75 181t181 75h113l-44 17q-74 28 -119.5 93.5t-45.5 145.5q0 106 75 181t181 75q46 0 91 -17l628 -239h401q106 0 181 -75t75 -181v-668 -q0 -88 -54 -157.5t-140 -90.5l-339 -85q-92 -23 -186 -23zM1024 583l-155 -71l-163 -74q-30 -14 -48 -41.5t-18 -60.5q0 -46 33 -79t79 -33q26 0 46 10l338 154q-49 10 -80.5 50t-31.5 90v55zM1344 272q0 46 -33 79t-79 33q-26 0 -46 -10l-290 -132q-28 -13 -37 -17 -t-30.5 -17t-29.5 -23.5t-16 -29t-8 -40.5q0 -50 31.5 -82t81.5 -32q20 0 38 9l352 160q30 14 48 41.5t18 60.5zM1112 1024l-650 248q-24 8 -46 8q-53 0 -90.5 -37.5t-37.5 -90.5q0 -40 22.5 -73t59.5 -47l526 -200v-64h-640q-53 0 -90.5 -37.5t-37.5 -90.5t37.5 -90.5 -t90.5 -37.5h535l233 106v198q0 63 46 106l111 102h-69zM1073 0q82 0 155 19l339 85q43 11 70 45.5t27 78.5v668q0 53 -37.5 90.5t-90.5 37.5h-308l-136 -126q-36 -33 -36 -82v-296q0 -46 33 -77t79 -31t79 35t33 81v208h32v-208q0 -70 -57 -114q52 -8 86.5 -48.5t34.5 -93.5 -q0 -42 -23 -78t-61 -53l-310 -141h91z" /> - <glyph glyph-name="_563" unicode="" horiz-adv-x="2048" -d="M1151 1536q61 0 116 -28t91 -77l572 -781q118 -159 118 -359v-355q0 -80 -56 -136t-136 -56h-384q-80 0 -136 56t-56 136v177l-286 143h-546q-80 0 -136 56t-56 136v32q0 119 84.5 203.5t203.5 84.5h420l42 128h-686q-100 0 -173.5 67.5t-81.5 166.5q-65 79 -65 182v32 -q0 80 56 136t136 56h959zM1920 -64v355q0 157 -93 284l-573 781q-39 52 -103 52h-959q-26 0 -45 -19t-19 -45q0 -32 1.5 -49.5t9.5 -40.5t25 -43q10 31 35.5 50t56.5 19h832v-32h-832q-26 0 -45 -19t-19 -45q0 -44 3 -58q8 -44 44 -73t81 -29h640h91q40 0 68 -28t28 -68 -q0 -15 -5 -30l-64 -192q-10 -29 -35 -47.5t-56 -18.5h-443q-66 0 -113 -47t-47 -113v-32q0 -26 19 -45t45 -19h561q16 0 29 -7l317 -158q24 -13 38.5 -36t14.5 -50v-197q0 -26 19 -45t45 -19h384q26 0 45 19t19 45z" /> - <glyph glyph-name="_564" unicode="" horiz-adv-x="2048" -d="M459 -256q-77 0 -137.5 47.5t-79.5 122.5l-101 401q-13 57 -13 108q0 45 -5 67l-116 477q-7 27 -7 57q0 93 62 161t155 78q17 85 82.5 139t152.5 54q83 0 148 -51.5t85 -132.5l83 -348l103 428q20 81 85 132.5t148 51.5q89 0 155.5 -57.5t80.5 -144.5q92 -10 152 -79 -t60 -162q0 -24 -7 -59l-123 -512q10 7 37.5 28.5t38.5 29.5t35 23t41 20.5t41.5 11t49.5 5.5q105 0 180 -74t75 -179q0 -62 -28.5 -118t-78.5 -94l-507 -380q-68 -51 -153 -51h-694zM1104 1408q-38 0 -68.5 -24t-39.5 -62l-164 -682h-127l-145 602q-9 38 -39.5 62t-68.5 24 -q-48 0 -80 -33t-32 -80q0 -15 3 -28l132 -547h-26l-99 408q-9 37 -40 62.5t-69 25.5q-47 0 -80 -33t-33 -79q0 -14 3 -26l116 -478q7 -28 9 -86t10 -88l100 -401q8 -32 34 -52.5t59 -20.5h694q42 0 76 26l507 379q56 43 56 110q0 52 -37.5 88.5t-89.5 36.5q-43 0 -77 -26 -l-307 -230v227q0 4 32 138t68 282t39 161q4 18 4 29q0 47 -32 81t-79 34q-39 0 -69.5 -24t-39.5 -62l-116 -482h-26l150 624q3 14 3 28q0 48 -31.5 82t-79.5 34z" /> - <glyph glyph-name="_565" unicode="" horiz-adv-x="1792" -d="M640 1408q-53 0 -90.5 -37.5t-37.5 -90.5v-512v-384l-151 202q-41 54 -107 54q-52 0 -89 -38t-37 -90q0 -43 26 -77l384 -512q38 -51 102 -51h718q22 0 39.5 13.5t22.5 34.5l92 368q24 96 24 194v217q0 41 -28 71t-68 30t-68 -28t-28 -68h-32v61q0 48 -32 81.5t-80 33.5 -q-46 0 -79 -33t-33 -79v-64h-32v90q0 55 -37 94.5t-91 39.5q-53 0 -90.5 -37.5t-37.5 -90.5v-96h-32v570q0 55 -37 94.5t-91 39.5zM640 1536q107 0 181.5 -77.5t74.5 -184.5v-220q22 2 32 2q99 0 173 -69q47 21 99 21q113 0 184 -87q27 7 56 7q94 0 159 -67.5t65 -161.5 -v-217q0 -116 -28 -225l-92 -368q-16 -64 -68 -104.5t-118 -40.5h-718q-60 0 -114.5 27.5t-90.5 74.5l-384 512q-51 68 -51 154q0 105 74.5 180.5t179.5 75.5q71 0 130 -35v547q0 106 75 181t181 75zM768 128v384h-32v-384h32zM1024 128v384h-32v-384h32zM1280 128v384h-32 -v-384h32z" /> - <glyph glyph-name="_566" unicode="" -d="M1288 889q60 0 107 -23q141 -63 141 -226v-177q0 -94 -23 -186l-85 -339q-21 -86 -90.5 -140t-157.5 -54h-668q-106 0 -181 75t-75 181v401l-239 628q-17 45 -17 91q0 106 75 181t181 75q80 0 145.5 -45.5t93.5 -119.5l17 -44v113q0 106 75 181t181 75t181 -75t75 -181 -v-261q27 5 48 5q69 0 127.5 -36.5t88.5 -98.5zM1072 896q-33 0 -60.5 -18t-41.5 -48l-74 -163l-71 -155h55q50 0 90 -31.5t50 -80.5l154 338q10 20 10 46q0 46 -33 79t-79 33zM1293 761q-22 0 -40.5 -8t-29 -16t-23.5 -29.5t-17 -30.5t-17 -37l-132 -290q-10 -20 -10 -46 -q0 -46 33 -79t79 -33q33 0 60.5 18t41.5 48l160 352q9 18 9 38q0 50 -32 81.5t-82 31.5zM128 1120q0 -22 8 -46l248 -650v-69l102 111q43 46 106 46h198l106 233v535q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5v-640h-64l-200 526q-14 37 -47 59.5t-73 22.5 -q-53 0 -90.5 -37.5t-37.5 -90.5zM1180 -128q44 0 78.5 27t45.5 70l85 339q19 73 19 155v91l-141 -310q-17 -38 -53 -61t-78 -23q-53 0 -93.5 34.5t-48.5 86.5q-44 -57 -114 -57h-208v32h208q46 0 81 33t35 79t-31 79t-77 33h-296q-49 0 -82 -36l-126 -136v-308 -q0 -53 37.5 -90.5t90.5 -37.5h668z" /> - <glyph glyph-name="_567" unicode="" horiz-adv-x="1973" -d="M857 992v-117q0 -13 -9.5 -22t-22.5 -9h-298v-812q0 -13 -9 -22.5t-22 -9.5h-135q-13 0 -22.5 9t-9.5 23v812h-297q-13 0 -22.5 9t-9.5 22v117q0 14 9 23t23 9h793q13 0 22.5 -9.5t9.5 -22.5zM1895 995l77 -961q1 -13 -8 -24q-10 -10 -23 -10h-134q-12 0 -21 8.5 -t-10 20.5l-46 588l-189 -425q-8 -19 -29 -19h-120q-20 0 -29 19l-188 427l-45 -590q-1 -12 -10 -20.5t-21 -8.5h-135q-13 0 -23 10q-9 10 -9 24l78 961q1 12 10 20.5t21 8.5h142q20 0 29 -19l220 -520q10 -24 20 -51q3 7 9.5 24.5t10.5 26.5l221 520q9 19 29 19h141 -q13 0 22 -8.5t10 -20.5z" /> - <glyph glyph-name="_568" unicode="" horiz-adv-x="1792" -d="M1042 833q0 88 -60 121q-33 18 -117 18h-123v-281h162q66 0 102 37t36 105zM1094 548l205 -373q8 -17 -1 -31q-8 -16 -27 -16h-152q-20 0 -28 17l-194 365h-155v-350q0 -14 -9 -23t-23 -9h-134q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h294q128 0 190 -24q85 -31 134 -109 -t49 -180q0 -92 -42.5 -165.5t-115.5 -109.5q6 -10 9 -16zM896 1376q-150 0 -286 -58.5t-234.5 -157t-157 -234.5t-58.5 -286t58.5 -286t157 -234.5t234.5 -157t286 -58.5t286 58.5t234.5 157t157 234.5t58.5 286t-58.5 286t-157 234.5t-234.5 157t-286 58.5zM1792 640 -q0 -182 -71 -348t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="_569" unicode="" horiz-adv-x="1792" -d="M605 303q153 0 257 104q14 18 3 36l-45 82q-6 13 -24 17q-16 2 -27 -11l-4 -3q-4 -4 -11.5 -10t-17.5 -13.5t-23.5 -14.5t-28.5 -13t-33.5 -9.5t-37.5 -3.5q-76 0 -125 50t-49 127q0 76 48 125.5t122 49.5q37 0 71.5 -14t50.5 -28l16 -14q11 -11 26 -10q16 2 24 14l53 78 -q13 20 -2 39q-3 4 -11 12t-30 23.5t-48.5 28t-67.5 22.5t-86 10q-148 0 -246 -96.5t-98 -240.5q0 -146 97 -241.5t247 -95.5zM1235 303q153 0 257 104q14 18 4 36l-45 82q-8 14 -25 17q-16 2 -27 -11l-4 -3q-4 -4 -11.5 -10t-17.5 -13.5t-23.5 -14.5t-28.5 -13t-33.5 -9.5 -t-37.5 -3.5q-76 0 -125 50t-49 127q0 76 48 125.5t122 49.5q37 0 71.5 -14t50.5 -28l16 -14q11 -11 26 -10q16 2 24 14l53 78q13 20 -2 39q-3 4 -11 12t-30 23.5t-48.5 28t-67.5 22.5t-86 10q-147 0 -245.5 -96.5t-98.5 -240.5q0 -146 97 -241.5t247 -95.5zM896 1376 -q-150 0 -286 -58.5t-234.5 -157t-157 -234.5t-58.5 -286t58.5 -286t157 -234.5t234.5 -157t286 -58.5t286 58.5t234.5 157t157 234.5t58.5 286t-58.5 286t-157 234.5t-234.5 157t-286 58.5zM896 1536q182 0 348 -71t286 -191t191 -286t71 -348t-71 -348t-191 -286t-286 -191 -t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71z" /> - <glyph glyph-name="f260" unicode="" horiz-adv-x="2048" -d="M736 736l384 -384l-384 -384l-672 672l672 672l168 -168l-96 -96l-72 72l-480 -480l480 -480l193 193l-289 287zM1312 1312l672 -672l-672 -672l-168 168l96 96l72 -72l480 480l-480 480l-193 -193l289 -287l-96 -96l-384 384z" /> - <glyph glyph-name="f261" unicode="" horiz-adv-x="1792" -d="M717 182l271 271l-279 279l-88 -88l192 -191l-96 -96l-279 279l279 279l40 -40l87 87l-127 128l-454 -454zM1075 190l454 454l-454 454l-271 -271l279 -279l88 88l-192 191l96 96l279 -279l-279 -279l-40 40l-87 -88zM1792 640q0 -182 -71 -348t-191 -286t-286 -191 -t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="_572" unicode="" horiz-adv-x="2304" -d="M651 539q0 -39 -27.5 -66.5t-65.5 -27.5q-39 0 -66.5 27.5t-27.5 66.5q0 38 27.5 65.5t66.5 27.5q38 0 65.5 -27.5t27.5 -65.5zM1805 540q0 -39 -27.5 -66.5t-66.5 -27.5t-66.5 27.5t-27.5 66.5t27.5 66t66.5 27t66.5 -27t27.5 -66zM765 539q0 79 -56.5 136t-136.5 57 -t-136.5 -56.5t-56.5 -136.5t56.5 -136.5t136.5 -56.5t136.5 56.5t56.5 136.5zM1918 540q0 80 -56.5 136.5t-136.5 56.5q-79 0 -136 -56.5t-57 -136.5t56.5 -136.5t136.5 -56.5t136.5 56.5t56.5 136.5zM850 539q0 -116 -81.5 -197.5t-196.5 -81.5q-116 0 -197.5 82t-81.5 197 -t82 196.5t197 81.5t196.5 -81.5t81.5 -196.5zM2004 540q0 -115 -81.5 -196.5t-197.5 -81.5q-115 0 -196.5 81.5t-81.5 196.5t81.5 196.5t196.5 81.5q116 0 197.5 -81.5t81.5 -196.5zM1040 537q0 191 -135.5 326.5t-326.5 135.5q-125 0 -231 -62t-168 -168.5t-62 -231.5 -t62 -231.5t168 -168.5t231 -62q191 0 326.5 135.5t135.5 326.5zM1708 1110q-254 111 -556 111q-319 0 -573 -110q117 0 223 -45.5t182.5 -122.5t122 -183t45.5 -223q0 115 43.5 219.5t118 180.5t177.5 123t217 50zM2187 537q0 191 -135 326.5t-326 135.5t-326.5 -135.5 -t-135.5 -326.5t135.5 -326.5t326.5 -135.5t326 135.5t135 326.5zM1921 1103h383q-44 -51 -75 -114.5t-40 -114.5q110 -151 110 -337q0 -156 -77 -288t-209 -208.5t-287 -76.5q-133 0 -249 56t-196 155q-47 -56 -129 -179q-11 22 -53.5 82.5t-74.5 97.5 -q-80 -99 -196.5 -155.5t-249.5 -56.5q-155 0 -287 76.5t-209 208.5t-77 288q0 186 110 337q-9 51 -40 114.5t-75 114.5h365q149 100 355 156.5t432 56.5q224 0 421 -56t348 -157z" /> - <glyph glyph-name="f263" unicode="" horiz-adv-x="1280" -d="M640 629q-188 0 -321 133t-133 320q0 188 133 321t321 133t321 -133t133 -321q0 -187 -133 -320t-321 -133zM640 1306q-92 0 -157.5 -65.5t-65.5 -158.5q0 -92 65.5 -157.5t157.5 -65.5t157.5 65.5t65.5 157.5q0 93 -65.5 158.5t-157.5 65.5zM1163 574q13 -27 15 -49.5 -t-4.5 -40.5t-26.5 -38.5t-42.5 -37t-61.5 -41.5q-115 -73 -315 -94l73 -72l267 -267q30 -31 30 -74t-30 -73l-12 -13q-31 -30 -74 -30t-74 30q-67 68 -267 268l-267 -268q-31 -30 -74 -30t-73 30l-12 13q-31 30 -31 73t31 74l267 267l72 72q-203 21 -317 94 -q-39 25 -61.5 41.5t-42.5 37t-26.5 38.5t-4.5 40.5t15 49.5q10 20 28 35t42 22t56 -2t65 -35q5 -4 15 -11t43 -24.5t69 -30.5t92 -24t113 -11q91 0 174 25.5t120 50.5l38 25q33 26 65 35t56 2t42 -22t28 -35z" /> - <glyph glyph-name="_574" unicode="" -d="M927 956q0 -66 -46.5 -112.5t-112.5 -46.5t-112.5 46.5t-46.5 112.5t46.5 112.5t112.5 46.5t112.5 -46.5t46.5 -112.5zM1141 593q-10 20 -28 32t-47.5 9.5t-60.5 -27.5q-10 -8 -29 -20t-81 -32t-127 -20t-124 18t-86 36l-27 18q-31 25 -60.5 27.5t-47.5 -9.5t-28 -32 -q-22 -45 -2 -74.5t87 -73.5q83 -53 226 -67l-51 -52q-142 -142 -191 -190q-22 -22 -22 -52.5t22 -52.5l9 -9q22 -22 52.5 -22t52.5 22l191 191q114 -115 191 -191q22 -22 52.5 -22t52.5 22l9 9q22 22 22 52.5t-22 52.5l-191 190l-52 52q141 14 225 67q67 44 87 73.5t-2 74.5 -zM1092 956q0 134 -95 229t-229 95t-229 -95t-95 -229t95 -229t229 -95t229 95t95 229zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="_575" unicode="" horiz-adv-x="1720" -d="M1565 1408q65 0 110 -45.5t45 -110.5v-519q0 -176 -68 -336t-182.5 -275t-274 -182.5t-334.5 -67.5q-176 0 -335.5 67.5t-274.5 182.5t-183 275t-68 336v519q0 64 46 110t110 46h1409zM861 344q47 0 82 33l404 388q37 35 37 85q0 49 -34.5 83.5t-83.5 34.5q-47 0 -82 -33 -l-323 -310l-323 310q-35 33 -81 33q-49 0 -83.5 -34.5t-34.5 -83.5q0 -51 36 -85l405 -388q33 -33 81 -33z" /> - <glyph glyph-name="_576" unicode="" horiz-adv-x="2304" -d="M1494 -103l-295 695q-25 -49 -158.5 -305.5t-198.5 -389.5q-1 -1 -27.5 -0.5t-26.5 1.5q-82 193 -255.5 587t-259.5 596q-21 50 -66.5 107.5t-103.5 100.5t-102 43q0 5 -0.5 24t-0.5 27h583v-50q-39 -2 -79.5 -16t-66.5 -43t-10 -64q26 -59 216.5 -499t235.5 -540 -q31 61 140 266.5t131 247.5q-19 39 -126 281t-136 295q-38 69 -201 71v50l513 -1v-47q-60 -2 -93.5 -25t-12.5 -69q33 -70 87 -189.5t86 -187.5q110 214 173 363q24 55 -10 79.5t-129 26.5q1 7 1 25v24q64 0 170.5 0.5t180 1t92.5 0.5v-49q-62 -2 -119 -33t-90 -81 -l-213 -442q13 -33 127.5 -290t121.5 -274l441 1017q-14 38 -49.5 62.5t-65 31.5t-55.5 8v50l460 -4l1 -2l-1 -44q-139 -4 -201 -145q-526 -1216 -559 -1291h-49z" /> - <glyph glyph-name="_577" unicode="" horiz-adv-x="1792" -d="M949 643q0 -26 -16.5 -45t-41.5 -19q-26 0 -45 16.5t-19 41.5q0 26 17 45t42 19t44 -16.5t19 -41.5zM964 585l350 581q-9 -8 -67.5 -62.5t-125.5 -116.5t-136.5 -127t-117 -110.5t-50.5 -51.5l-349 -580q7 7 67 62t126 116.5t136 127t117 111t50 50.5zM1611 640 -q0 -201 -104 -371q-3 2 -17 11t-26.5 16.5t-16.5 7.5q-13 0 -13 -13q0 -10 59 -44q-74 -112 -184.5 -190.5t-241.5 -110.5l-16 67q-1 10 -15 10q-5 0 -8 -5.5t-2 -9.5l16 -68q-72 -15 -146 -15q-199 0 -372 105q1 2 13 20.5t21.5 33.5t9.5 19q0 13 -13 13q-6 0 -17 -14.5 -t-22.5 -34.5t-13.5 -23q-113 75 -192 187.5t-110 244.5l69 15q10 3 10 15q0 5 -5.5 8t-10.5 2l-68 -15q-14 72 -14 139q0 206 109 379q2 -1 18.5 -12t30 -19t17.5 -8q13 0 13 12q0 6 -12.5 15.5t-32.5 21.5l-20 12q77 112 189 189t244 107l15 -67q2 -10 15 -10q5 0 8 5.5 -t2 10.5l-15 66q71 13 134 13q204 0 379 -109q-39 -56 -39 -65q0 -13 12 -13q11 0 48 64q111 -75 187.5 -186t107.5 -241l-56 -12q-10 -2 -10 -16q0 -5 5.5 -8t9.5 -2l57 13q14 -72 14 -140zM1696 640q0 163 -63.5 311t-170.5 255t-255 170.5t-311 63.5t-311 -63.5 -t-255 -170.5t-170.5 -255t-63.5 -311t63.5 -311t170.5 -255t255 -170.5t311 -63.5t311 63.5t255 170.5t170.5 255t63.5 311zM1792 640q0 -182 -71 -348t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71t348 -71t286 -191 -t191 -286t71 -348z" /> - <glyph glyph-name="_578" unicode="" horiz-adv-x="1792" -d="M893 1536q240 2 451 -120q232 -134 352 -372l-742 39q-160 9 -294 -74.5t-185 -229.5l-276 424q128 159 311 245.5t383 87.5zM146 1131l337 -663q72 -143 211 -217t293 -45l-230 -451q-212 33 -385 157.5t-272.5 316t-99.5 411.5q0 267 146 491zM1732 962 -q58 -150 59.5 -310.5t-48.5 -306t-153 -272t-246 -209.5q-230 -133 -498 -119l405 623q88 131 82.5 290.5t-106.5 277.5zM896 942q125 0 213.5 -88.5t88.5 -213.5t-88.5 -213.5t-213.5 -88.5t-213.5 88.5t-88.5 213.5t88.5 213.5t213.5 88.5z" /> - <glyph glyph-name="_579" unicode="" horiz-adv-x="1792" -d="M903 -256q-283 0 -504.5 150.5t-329.5 398.5q-58 131 -67 301t26 332.5t111 312t179 242.5l-11 -281q11 14 68 15.5t70 -15.5q42 81 160.5 138t234.5 59q-54 -45 -119.5 -148.5t-58.5 -163.5q25 -8 62.5 -13.5t63 -7.5t68 -4t50.5 -3q15 -5 9.5 -45.5t-30.5 -75.5 -q-5 -7 -16.5 -18.5t-56.5 -35.5t-101 -34l15 -189l-139 67q-18 -43 -7.5 -81.5t36 -66.5t65.5 -41.5t81 -6.5q51 9 98 34.5t83.5 45t73.5 17.5q61 -4 89.5 -33t19.5 -65q-1 -2 -2.5 -5.5t-8.5 -12.5t-18 -15.5t-31.5 -10.5t-46.5 -1q-60 -95 -144.5 -135.5t-209.5 -29.5 -q74 -61 162.5 -82.5t168.5 -6t154.5 52t128 87.5t80.5 104q43 91 39 192.5t-37.5 188.5t-78.5 125q87 -38 137 -79.5t77 -112.5q15 170 -57.5 343t-209.5 284q265 -77 412 -279.5t151 -517.5q2 -127 -40.5 -255t-123.5 -238t-189 -196t-247.5 -135.5t-288.5 -49.5z" /> - <glyph glyph-name="_580" unicode="" horiz-adv-x="1792" -d="M1493 1308q-165 110 -359 110q-155 0 -293 -73t-240 -200q-75 -93 -119.5 -218t-48.5 -266v-42q4 -141 48.5 -266t119.5 -218q102 -127 240 -200t293 -73q194 0 359 110q-121 -108 -274.5 -168t-322.5 -60q-29 0 -43 1q-175 8 -333 82t-272 193t-181 281t-67 339 -q0 182 71 348t191 286t286 191t348 71h3q168 -1 320.5 -60.5t273.5 -167.5zM1792 640q0 -192 -77 -362.5t-213 -296.5q-104 -63 -222 -63q-137 0 -255 84q154 56 253.5 233t99.5 405q0 227 -99 404t-253 234q119 83 254 83q119 0 226 -65q135 -125 210.5 -295t75.5 -361z -" /> - <glyph glyph-name="_581" unicode="" horiz-adv-x="1792" -d="M1792 599q0 -56 -7 -104h-1151q0 -146 109.5 -244.5t257.5 -98.5q99 0 185.5 46.5t136.5 130.5h423q-56 -159 -170.5 -281t-267.5 -188.5t-321 -66.5q-187 0 -356 83q-228 -116 -394 -116q-237 0 -237 263q0 115 45 275q17 60 109 229q199 360 475 606 -q-184 -79 -427 -354q63 274 283.5 449.5t501.5 175.5q30 0 45 -1q255 117 433 117q64 0 116 -13t94.5 -40.5t66.5 -76.5t24 -115q0 -116 -75 -286q101 -182 101 -390zM1722 1239q0 83 -53 132t-137 49q-108 0 -254 -70q121 -47 222.5 -131.5t170.5 -195.5q51 135 51 216z -M128 2q0 -86 48.5 -132.5t134.5 -46.5q115 0 266 83q-122 72 -213.5 183t-137.5 245q-98 -205 -98 -332zM632 715h728q-5 142 -113 237t-251 95q-144 0 -251.5 -95t-112.5 -237z" /> - <glyph glyph-name="_582" unicode="" horiz-adv-x="2048" -d="M1792 288v960q0 13 -9.5 22.5t-22.5 9.5h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-960q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5zM1920 1248v-960q0 -66 -47 -113t-113 -47h-736v-128h352q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23 -v64q0 14 9 23t23 9h352v128h-736q-66 0 -113 47t-47 113v960q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" /> - <glyph glyph-name="_583" unicode="" horiz-adv-x="1792" -d="M138 1408h197q-70 -64 -126 -149q-36 -56 -59 -115t-30 -125.5t-8.5 -120t10.5 -132t21 -126t28 -136.5q4 -19 6 -28q51 -238 81 -329q57 -171 152 -275h-272q-48 0 -82 34t-34 82v1304q0 48 34 82t82 34zM1346 1408h308q48 0 82 -34t34 -82v-1304q0 -48 -34 -82t-82 -34 -h-178q212 210 196 565l-469 -101q-2 -45 -12 -82t-31 -72t-59.5 -59.5t-93.5 -36.5q-123 -26 -199 40q-32 27 -53 61t-51.5 129t-64.5 258q-35 163 -45.5 263t-5.5 139t23 77q20 41 62.5 73t102.5 45q45 12 83.5 6.5t67 -17t54 -35t43 -48t34.5 -56.5l468 100 -q-68 175 -180 287z" /> - <glyph glyph-name="_584" unicode="" -d="M1401 -11l-6 -6q-113 -113 -259 -175q-154 -64 -317 -64q-165 0 -317 64q-148 63 -259 175q-113 112 -175 258q-42 103 -54 189q-4 28 48 36q51 8 56 -20q1 -1 1 -4q18 -90 46 -159q50 -124 152 -226q98 -98 226 -152q132 -56 276 -56q143 0 276 56q128 55 225 152l6 6 -q10 10 25 6q12 -3 33 -22q36 -37 17 -58zM929 604l-66 -66l63 -63q21 -21 -7 -49q-17 -17 -32 -17q-10 0 -19 10l-62 61l-66 -66q-5 -5 -15 -5q-15 0 -31 16l-2 2q-18 15 -18 29q0 7 8 17l66 65l-66 66q-16 16 14 45q18 18 31 18q6 0 13 -5l65 -66l65 65q18 17 48 -13 -q27 -27 11 -44zM1400 547q0 -118 -46 -228q-45 -105 -126 -186q-80 -80 -187 -126t-228 -46t-228 46t-187 126q-82 82 -125 186q-15 33 -15 40h-1q-9 27 43 44q50 16 60 -12q37 -99 97 -167h1v339v2q3 136 102 232q105 103 253 103q147 0 251 -103t104 -249 -q0 -147 -104.5 -251t-250.5 -104q-58 0 -112 16q-28 11 -13 61q16 51 44 43l14 -3q14 -3 33 -6t30 -3q104 0 176 71.5t72 174.5q0 101 -72 171q-71 71 -175 71q-107 0 -178 -80q-64 -72 -64 -160v-413q110 -67 242 -67q96 0 185 36.5t156 103.5t103.5 155t36.5 183 -q0 198 -141 339q-140 140 -339 140q-200 0 -340 -140q-53 -53 -77 -87l-2 -2q-8 -11 -13 -15.5t-21.5 -9.5t-38.5 3q-21 5 -36.5 16.5t-15.5 26.5v680q0 15 10.5 26.5t27.5 11.5h877q30 0 30 -55t-30 -55h-811v-483h1q40 42 102 84t108 61q109 46 231 46q121 0 228 -46 -t187 -126q81 -81 126 -186q46 -112 46 -229zM1369 1128q9 -8 9 -18t-5.5 -18t-16.5 -21q-26 -26 -39 -26q-9 0 -16 7q-106 91 -207 133q-128 56 -276 56q-133 0 -262 -49q-27 -10 -45 37q-9 25 -8 38q3 16 16 20q130 57 299 57q164 0 316 -64q137 -58 235 -152z" /> - <glyph glyph-name="_585" unicode="" horiz-adv-x="1792" -d="M1551 60q15 6 26 3t11 -17.5t-15 -33.5q-13 -16 -44 -43.5t-95.5 -68t-141 -74t-188 -58t-229.5 -24.5q-119 0 -238 31t-209 76.5t-172.5 104t-132.5 105t-84 87.5q-8 9 -10 16.5t1 12t8 7t11.5 2t11.5 -4.5q192 -117 300 -166q389 -176 799 -90q190 40 391 135z -M1758 175q11 -16 2.5 -69.5t-28.5 -102.5q-34 -83 -85 -124q-17 -14 -26 -9t0 24q21 45 44.5 121.5t6.5 98.5q-5 7 -15.5 11.5t-27 6t-29.5 2.5t-35 0t-31.5 -2t-31 -3t-22.5 -2q-6 -1 -13 -1.5t-11 -1t-8.5 -1t-7 -0.5h-5.5h-4.5t-3 0.5t-2 1.5l-1.5 3q-6 16 47 40t103 30 -q46 7 108 1t76 -24zM1364 618q0 -31 13.5 -64t32 -58t37.5 -46t33 -32l13 -11l-227 -224q-40 37 -79 75.5t-58 58.5l-19 20q-11 11 -25 33q-38 -59 -97.5 -102.5t-127.5 -63.5t-140 -23t-137.5 21t-117.5 65.5t-83 113t-31 162.5q0 84 28 154t72 116.5t106.5 83t122.5 57 -t130 34.5t119.5 18.5t99.5 6.5v127q0 65 -21 97q-34 53 -121 53q-6 0 -16.5 -1t-40.5 -12t-56 -29.5t-56 -59.5t-48 -96l-294 27q0 60 22 119t67 113t108 95t151.5 65.5t190.5 24.5q100 0 181 -25t129.5 -61.5t81 -83t45 -86t12.5 -73.5v-589zM692 597q0 -86 70 -133 -q66 -44 139 -22q84 25 114 123q14 45 14 101v162q-59 -2 -111 -12t-106.5 -33.5t-87 -71t-32.5 -114.5z" /> - <glyph glyph-name="_586" unicode="" horiz-adv-x="1792" -d="M1536 1280q52 0 90 -38t38 -90v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h128zM1152 1376v-288q0 -14 9 -23t23 -9 -h64q14 0 23 9t9 23v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23zM384 1376v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23zM1536 -128v1024h-1408v-1024h1408zM896 448h224q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-224 -v-224q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v224h-224q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v224q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-224z" /> - <glyph glyph-name="_587" unicode="" horiz-adv-x="1792" -d="M1152 416v-64q0 -14 -9 -23t-23 -9h-576q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h576q14 0 23 -9t9 -23zM128 -128h1408v1024h-1408v-1024zM512 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1280 1088v288q0 14 -9 23 -t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1664 1152v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47h64q66 0 113 -47 -t47 -113v-96h128q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_588" unicode="" horiz-adv-x="1792" -d="M1111 151l-46 -46q-9 -9 -22 -9t-23 9l-188 189l-188 -189q-10 -9 -23 -9t-22 9l-46 46q-9 9 -9 22t9 23l189 188l-189 188q-9 10 -9 23t9 22l46 46q9 9 22 9t23 -9l188 -188l188 188q10 9 23 9t22 -9l46 -46q9 -9 9 -22t-9 -23l-188 -188l188 -188q9 -10 9 -23t-9 -22z -M128 -128h1408v1024h-1408v-1024zM512 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1280 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1664 1152v-1280 -q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_589" unicode="" horiz-adv-x="1792" -d="M1303 572l-512 -512q-10 -9 -23 -9t-23 9l-288 288q-9 10 -9 23t9 22l46 46q9 9 22 9t23 -9l220 -220l444 444q10 9 23 9t22 -9l46 -46q9 -9 9 -22t-9 -23zM128 -128h1408v1024h-1408v-1024zM512 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23 -t23 -9h64q14 0 23 9t9 23zM1280 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1664 1152v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47 -t47 -113v-96h384v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" /> - <glyph glyph-name="_590" unicode="" horiz-adv-x="1792" -d="M448 1536q26 0 45 -19t19 -45v-891l536 429q17 14 40 14q26 0 45 -19t19 -45v-379l536 429q17 14 40 14q26 0 45 -19t19 -45v-1152q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v1664q0 26 19 45t45 19h384z" /> - <glyph glyph-name="_591" unicode="" horiz-adv-x="1024" -d="M512 448q66 0 128 15v-655q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v655q62 -15 128 -15zM512 1536q212 0 362 -150t150 -362t-150 -362t-362 -150t-362 150t-150 362t150 362t362 150zM512 1312q14 0 23 9t9 23t-9 23t-23 9q-146 0 -249 -103t-103 -249 -q0 -14 9 -23t23 -9t23 9t9 23q0 119 84.5 203.5t203.5 84.5z" /> - <glyph glyph-name="_592" unicode="" horiz-adv-x="1792" -d="M1745 1239q10 -10 10 -23t-10 -23l-141 -141q-28 -28 -68 -28h-1344q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h576v64q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-64h512q40 0 68 -28zM768 320h256v-512q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v512zM1600 768 -q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19h-1344q-40 0 -68 28l-141 141q-10 10 -10 23t10 23l141 141q28 28 68 28h512v192h256v-192h576z" /> - <glyph glyph-name="_593" unicode="" horiz-adv-x="2048" -d="M2020 1525q28 -20 28 -53v-1408q0 -20 -11 -36t-29 -23l-640 -256q-24 -11 -48 0l-616 246l-616 -246q-10 -5 -24 -5q-19 0 -36 11q-28 20 -28 53v1408q0 20 11 36t29 23l640 256q24 11 48 0l616 -246l616 246q32 13 60 -6zM736 1390v-1270l576 -230v1270zM128 1173 -v-1270l544 217v1270zM1920 107v1270l-544 -217v-1270z" /> - <glyph glyph-name="_594" unicode="" horiz-adv-x="1792" -d="M512 1536q13 0 22.5 -9.5t9.5 -22.5v-1472q0 -20 -17 -28l-480 -256q-7 -4 -15 -4q-13 0 -22.5 9.5t-9.5 22.5v1472q0 20 17 28l480 256q7 4 15 4zM1760 1536q13 0 22.5 -9.5t9.5 -22.5v-1472q0 -20 -17 -28l-480 -256q-7 -4 -15 -4q-13 0 -22.5 9.5t-9.5 22.5v1472 -q0 20 17 28l480 256q7 4 15 4zM640 1536q8 0 14 -3l512 -256q18 -10 18 -29v-1472q0 -13 -9.5 -22.5t-22.5 -9.5q-8 0 -14 3l-512 256q-18 10 -18 29v1472q0 13 9.5 22.5t22.5 9.5z" /> - <glyph glyph-name="_595" unicode="" horiz-adv-x="1792" -d="M640 640q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 640q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1408 640q0 53 -37.5 90.5t-90.5 37.5 -t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-110 0 -211 18q-173 -173 -435 -229q-52 -10 -86 -13q-12 -1 -22 6t-13 18q-4 15 20 37q5 5 23.5 21.5t25.5 23.5t23.5 25.5t24 31.5t20.5 37 -t20 48t14.5 57.5t12.5 72.5q-146 90 -229.5 216.5t-83.5 269.5q0 174 120 321.5t326 233t450 85.5t450 -85.5t326 -233t120 -321.5z" /> - <glyph glyph-name="_596" unicode="" horiz-adv-x="1792" -d="M640 640q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1024 640q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 -53 -37.5 -90.5t-90.5 -37.5 -t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM896 1152q-204 0 -381.5 -69.5t-282 -187.5t-104.5 -255q0 -112 71.5 -213.5t201.5 -175.5l87 -50l-27 -96q-24 -91 -70 -172q152 63 275 171l43 38l57 -6q69 -8 130 -8q204 0 381.5 69.5t282 187.5 -t104.5 255t-104.5 255t-282 187.5t-381.5 69.5zM1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22h-5q-15 0 -27 10.5t-16 27.5v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51 -t27 59t26 76q-157 89 -247.5 220t-90.5 281q0 130 71 248.5t191 204.5t286 136.5t348 50.5t348 -50.5t286 -136.5t191 -204.5t71 -248.5z" /> - <glyph glyph-name="_597" unicode="" horiz-adv-x="1024" -d="M512 345l512 295v-591l-512 -296v592zM0 640v-591l512 296zM512 1527v-591l-512 -296v591zM512 936l512 295v-591z" /> - <glyph glyph-name="_598" unicode="" horiz-adv-x="1792" -d="M1709 1018q-10 -236 -332 -651q-333 -431 -562 -431q-142 0 -240 263q-44 160 -132 482q-72 262 -157 262q-18 0 -127 -76l-77 98q24 21 108 96.5t130 115.5q156 138 241 146q95 9 153 -55.5t81 -203.5q44 -287 66 -373q55 -249 120 -249q51 0 154 161q101 161 109 246 -q13 139 -109 139q-57 0 -121 -26q120 393 459 382q251 -8 236 -326z" /> - <glyph glyph-name="f27e" unicode="" -d="M0 1408h1536v-1536h-1536v1536zM1085 293l-221 631l221 297h-634l221 -297l-221 -631l317 -304z" /> - <glyph glyph-name="uniF280" unicode="" -d="M0 1408h1536v-1536h-1536v1536zM908 1088l-12 -33l75 -83l-31 -114l25 -25l107 57l107 -57l25 25l-31 114l75 83l-12 33h-95l-53 96h-32l-53 -96h-95zM641 925q32 0 44.5 -16t11.5 -63l174 21q0 55 -17.5 92.5t-50.5 56t-69 25.5t-85 7q-133 0 -199 -57.5t-66 -182.5v-72 -h-96v-128h76q20 0 20 -8v-382q0 -14 -5 -20t-18 -7l-73 -7v-88h448v86l-149 14q-6 1 -8.5 1.5t-3.5 2.5t-0.5 4t1 7t0.5 10v387h191l38 128h-231q-6 0 -2 6t4 9v80q0 27 1.5 40.5t7.5 28t19.5 20t36.5 5.5zM1248 96v86l-54 9q-7 1 -9.5 2.5t-2.5 3t1 7.5t1 12v520h-275 -l-23 -101l83 -22q23 -7 23 -27v-370q0 -14 -6 -18.5t-20 -6.5l-70 -9v-86h352z" /> - <glyph glyph-name="uniF281" unicode="" horiz-adv-x="1792" -d="M1792 690q0 -58 -29.5 -105.5t-79.5 -72.5q12 -46 12 -96q0 -155 -106.5 -287t-290.5 -208.5t-400 -76.5t-399.5 76.5t-290 208.5t-106.5 287q0 47 11 94q-51 25 -82 73.5t-31 106.5q0 82 58 140.5t141 58.5q85 0 145 -63q218 152 515 162l116 521q3 13 15 21t26 5 -l369 -81q18 37 54 59.5t79 22.5q62 0 106 -43.5t44 -105.5t-44 -106t-106 -44t-105.5 43.5t-43.5 105.5l-334 74l-104 -472q300 -9 519 -160q58 61 143 61q83 0 141 -58.5t58 -140.5zM418 491q0 -62 43.5 -106t105.5 -44t106 44t44 106t-44 105.5t-106 43.5q-61 0 -105 -44 -t-44 -105zM1228 136q11 11 11 26t-11 26q-10 10 -25 10t-26 -10q-41 -42 -121 -62t-160 -20t-160 20t-121 62q-11 10 -26 10t-25 -10q-11 -10 -11 -25.5t11 -26.5q43 -43 118.5 -68t122.5 -29.5t91 -4.5t91 4.5t122.5 29.5t118.5 68zM1225 341q62 0 105.5 44t43.5 106 -q0 61 -44 105t-105 44q-62 0 -106 -43.5t-44 -105.5t44 -106t106 -44z" /> - <glyph glyph-name="_602" unicode="" horiz-adv-x="1792" -d="M69 741h1q16 126 58.5 241.5t115 217t167.5 176t223.5 117.5t276.5 43q231 0 414 -105.5t294 -303.5q104 -187 104 -442v-188h-1125q1 -111 53.5 -192.5t136.5 -122.5t189.5 -57t213 -3t208 46.5t173.5 84.5v-377q-92 -55 -229.5 -92t-312.5 -38t-316 53 -q-189 73 -311.5 249t-124.5 372q-3 242 111 412t325 268q-48 -60 -78 -125.5t-46 -159.5h635q8 77 -8 140t-47 101.5t-70.5 66.5t-80.5 41t-75 20.5t-56 8.5l-22 1q-135 -5 -259.5 -44.5t-223.5 -104.5t-176 -140.5t-138 -163.5z" /> - <glyph glyph-name="_603" unicode="" horiz-adv-x="2304" -d="M0 32v608h2304v-608q0 -66 -47 -113t-113 -47h-1984q-66 0 -113 47t-47 113zM640 256v-128h384v128h-384zM256 256v-128h256v128h-256zM2144 1408q66 0 113 -47t47 -113v-224h-2304v224q0 66 47 113t113 47h1984z" /> - <glyph glyph-name="_604" unicode="" horiz-adv-x="1792" -d="M1584 246l-218 111q-74 -120 -196.5 -189t-263.5 -69q-147 0 -271 72t-196 196t-72 270q0 110 42.5 209.5t115 172t172 115t209.5 42.5q131 0 247.5 -60.5t192.5 -168.5l215 125q-110 169 -286.5 265t-378.5 96q-161 0 -308 -63t-253 -169t-169 -253t-63 -308t63 -308 -t169 -253t253 -169t308 -63q213 0 397.5 107t290.5 292zM1030 643l693 -352q-116 -253 -334.5 -400t-492.5 -147q-182 0 -348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71q260 0 470.5 -133.5t335.5 -366.5zM1543 640h-39v-160h-96v352h136q32 0 54.5 -20 -t28.5 -48t1 -56t-27.5 -48t-57.5 -20z" /> - <glyph glyph-name="uniF285" unicode="" horiz-adv-x="1792" -d="M1427 827l-614 386l92 151h855zM405 562l-184 116v858l1183 -743zM1424 697l147 -95v-858l-532 335zM1387 718l-500 -802h-855l356 571z" /> - <glyph glyph-name="uniF286" unicode="" horiz-adv-x="1792" -d="M640 528v224q0 16 -16 16h-96q-16 0 -16 -16v-224q0 -16 16 -16h96q16 0 16 16zM1152 528v224q0 16 -16 16h-96q-16 0 -16 -16v-224q0 -16 16 -16h96q16 0 16 16zM1664 496v-752h-640v320q0 80 -56 136t-136 56t-136 -56t-56 -136v-320h-640v752q0 16 16 16h96 -q16 0 16 -16v-112h128v624q0 16 16 16h96q16 0 16 -16v-112h128v112q0 16 16 16h96q16 0 16 -16v-112h128v112q0 6 2.5 9.5t8.5 5t9.5 2t11.5 0t9 -0.5v391q-32 15 -32 50q0 23 16.5 39t38.5 16t38.5 -16t16.5 -39q0 -35 -32 -50v-17q45 10 83 10q21 0 59.5 -7.5t54.5 -7.5 -q17 0 47 7.5t37 7.5q16 0 16 -16v-210q0 -15 -35 -21.5t-62 -6.5q-18 0 -54.5 7.5t-55.5 7.5q-40 0 -90 -12v-133q1 0 9 0.5t11.5 0t9.5 -2t8.5 -5t2.5 -9.5v-112h128v112q0 16 16 16h96q16 0 16 -16v-112h128v112q0 16 16 16h96q16 0 16 -16v-624h128v112q0 16 16 16h96 -q16 0 16 -16z" /> - <glyph glyph-name="_607" unicode="" horiz-adv-x="2304" -d="M2288 731q16 -8 16 -27t-16 -27l-320 -192q-8 -5 -16 -5q-9 0 -16 4q-16 10 -16 28v128h-858q37 -58 83 -165q16 -37 24.5 -55t24 -49t27 -47t27 -34t31.5 -26t33 -8h96v96q0 14 9 23t23 9h320q14 0 23 -9t9 -23v-320q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v96h-96 -q-32 0 -61 10t-51 23.5t-45 40.5t-37 46t-33.5 57t-28.5 57.5t-28 60.5q-23 53 -37 81.5t-36 65t-44.5 53.5t-46.5 17h-360q-22 -84 -91 -138t-157 -54q-106 0 -181 75t-75 181t75 181t181 75q88 0 157 -54t91 -138h104q24 0 46.5 17t44.5 53.5t36 65t37 81.5q19 41 28 60.5 -t28.5 57.5t33.5 57t37 46t45 40.5t51 23.5t61 10h107q21 57 70 92.5t111 35.5q80 0 136 -56t56 -136t-56 -136t-136 -56q-62 0 -111 35.5t-70 92.5h-107q-17 0 -33 -8t-31.5 -26t-27 -34t-27 -47t-24 -49t-24.5 -55q-46 -107 -83 -165h1114v128q0 18 16 28t32 -1z" /> - <glyph glyph-name="_608" unicode="" horiz-adv-x="1792" -d="M1150 774q0 -56 -39.5 -95t-95.5 -39h-253v269h253q56 0 95.5 -39.5t39.5 -95.5zM1329 774q0 130 -91.5 222t-222.5 92h-433v-896h180v269h253q130 0 222 91.5t92 221.5zM1792 640q0 -182 -71 -348t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348 -t71 348t191 286t286 191t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="_609" unicode="" horiz-adv-x="2304" -d="M1645 438q0 59 -34 106.5t-87 68.5q-7 -45 -23 -92q-7 -24 -27.5 -38t-44.5 -14q-12 0 -24 3q-31 10 -45 38.5t-4 58.5q23 71 23 143q0 123 -61 227.5t-166 165.5t-228 61q-134 0 -247 -73t-167 -194q108 -28 188 -106q22 -23 22 -55t-22 -54t-54 -22t-55 22 -q-75 75 -180 75q-106 0 -181 -74.5t-75 -180.5t75 -180.5t181 -74.5h1046q79 0 134.5 55.5t55.5 133.5zM1798 438q0 -142 -100.5 -242t-242.5 -100h-1046q-169 0 -289 119.5t-120 288.5q0 153 100 267t249 136q62 184 221 298t354 114q235 0 408.5 -158.5t196.5 -389.5 -q116 -25 192.5 -118.5t76.5 -214.5zM2048 438q0 -175 -97 -319q-23 -33 -64 -33q-24 0 -43 13q-26 17 -32 48.5t12 57.5q71 104 71 233t-71 233q-18 26 -12 57t32 49t57.5 11.5t49.5 -32.5q97 -142 97 -318zM2304 438q0 -244 -134 -443q-23 -34 -64 -34q-23 0 -42 13 -q-26 18 -32.5 49t11.5 57q108 164 108 358q0 195 -108 357q-18 26 -11.5 57.5t32.5 48.5q26 18 57 12t49 -33q134 -198 134 -442z" /> - <glyph glyph-name="_610" unicode="" -d="M1500 -13q0 -89 -63 -152.5t-153 -63.5t-153.5 63.5t-63.5 152.5q0 90 63.5 153.5t153.5 63.5t153 -63.5t63 -153.5zM1267 268q-115 -15 -192.5 -102.5t-77.5 -205.5q0 -74 33 -138q-146 -78 -379 -78q-109 0 -201 21t-153.5 54.5t-110.5 76.5t-76 85t-44.5 83 -t-23.5 66.5t-6 39.5q0 19 4.5 42.5t18.5 56t36.5 58t64 43.5t94.5 18t94 -17.5t63 -41t35.5 -53t17.5 -49t4 -33.5q0 -34 -23 -81q28 -27 82 -42t93 -17l40 -1q115 0 190 51t75 133q0 26 -9 48.5t-31.5 44.5t-49.5 41t-74 44t-93.5 47.5t-119.5 56.5q-28 13 -43 20 -q-116 55 -187 100t-122.5 102t-72 125.5t-20.5 162.5q0 78 20.5 150t66 137.5t112.5 114t166.5 77t221.5 28.5q120 0 220 -26t164.5 -67t109.5 -94t64 -105.5t19 -103.5q0 -46 -15 -82.5t-36.5 -58t-48.5 -36t-49 -19.5t-39 -5h-8h-32t-39 5t-44 14t-41 28t-37 46t-24 70.5 -t-10 97.5q-15 16 -59 25.5t-81 10.5l-37 1q-68 0 -117.5 -31t-70.5 -70t-21 -76q0 -24 5 -43t24 -46t53 -51t97 -53.5t150 -58.5q76 -25 138.5 -53.5t109 -55.5t83 -59t60.5 -59.5t41 -62.5t26.5 -62t14.5 -63.5t6 -62t1 -62.5z" /> - <glyph glyph-name="_611" unicode="" -d="M704 352v576q0 14 -9 23t-23 9h-256q-14 0 -23 -9t-9 -23v-576q0 -14 9 -23t23 -9h256q14 0 23 9t9 23zM1152 352v576q0 14 -9 23t-23 9h-256q-14 0 -23 -9t-9 -23v-576q0 -14 9 -23t23 -9h256q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103 -t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="_612" unicode="" -d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM768 96q148 0 273 73t198 198t73 273t-73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273 -t73 -273t198 -198t273 -73zM864 320q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-192zM480 320q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-192z" /> - <glyph glyph-name="_613" unicode="" -d="M1088 352v576q0 14 -9 23t-23 9h-576q-14 0 -23 -9t-9 -23v-576q0 -14 9 -23t23 -9h576q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 -t103 -385.5z" /> - <glyph glyph-name="_614" unicode="" -d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM768 96q148 0 273 73t198 198t73 273t-73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273 -t73 -273t198 -198t273 -73zM480 320q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h576q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-576z" /> - <glyph glyph-name="_615" unicode="" horiz-adv-x="1792" -d="M1757 128l35 -313q3 -28 -16 -50q-19 -21 -48 -21h-1664q-29 0 -48 21q-19 22 -16 50l35 313h1722zM1664 967l86 -775h-1708l86 775q3 24 21 40.5t43 16.5h256v-128q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5v128h384v-128q0 -53 37.5 -90.5t90.5 -37.5 -t90.5 37.5t37.5 90.5v128h256q25 0 43 -16.5t21 -40.5zM1280 1152v-256q0 -26 -19 -45t-45 -19t-45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-256q0 -26 -19 -45t-45 -19t-45 19t-19 45v256q0 159 112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5z" /> - <glyph glyph-name="_616" unicode="" horiz-adv-x="2048" -d="M1920 768q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5h-15l-115 -662q-8 -46 -44 -76t-82 -30h-1280q-46 0 -82 30t-44 76l-115 662h-15q-53 0 -90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5h1792zM485 -32q26 2 43.5 22.5t15.5 46.5l-32 416q-2 26 -22.5 43.5 -t-46.5 15.5t-43.5 -22.5t-15.5 -46.5l32 -416q2 -25 20.5 -42t43.5 -17h5zM896 32v416q0 26 -19 45t-45 19t-45 -19t-19 -45v-416q0 -26 19 -45t45 -19t45 19t19 45zM1280 32v416q0 26 -19 45t-45 19t-45 -19t-19 -45v-416q0 -26 19 -45t45 -19t45 19t19 45zM1632 27l32 416 -q2 26 -15.5 46.5t-43.5 22.5t-46.5 -15.5t-22.5 -43.5l-32 -416q-2 -26 15.5 -46.5t43.5 -22.5h5q25 0 43.5 17t20.5 42zM476 1244l-93 -412h-132l101 441q19 88 89 143.5t160 55.5h167q0 26 19 45t45 19h384q26 0 45 -19t19 -45h167q90 0 160 -55.5t89 -143.5l101 -441 -h-132l-93 412q-11 44 -45.5 72t-79.5 28h-167q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45h-167q-45 0 -79.5 -28t-45.5 -72z" /> - <glyph glyph-name="_617" unicode="" horiz-adv-x="1792" -d="M991 512l64 256h-254l-64 -256h254zM1759 1016l-56 -224q-7 -24 -31 -24h-327l-64 -256h311q15 0 25 -12q10 -14 6 -28l-56 -224q-5 -24 -31 -24h-327l-81 -328q-7 -24 -31 -24h-224q-16 0 -26 12q-9 12 -6 28l78 312h-254l-81 -328q-7 -24 -31 -24h-225q-15 0 -25 12 -q-9 12 -6 28l78 312h-311q-15 0 -25 12q-9 12 -6 28l56 224q7 24 31 24h327l64 256h-311q-15 0 -25 12q-10 14 -6 28l56 224q5 24 31 24h327l81 328q7 24 32 24h224q15 0 25 -12q9 -12 6 -28l-78 -312h254l81 328q7 24 32 24h224q15 0 25 -12q9 -12 6 -28l-78 -312h311 -q15 0 25 -12q9 -12 6 -28z" /> - <glyph glyph-name="_618" unicode="" -d="M841 483l148 -148l-149 -149zM840 1094l149 -149l-148 -148zM710 -130l464 464l-306 306l306 306l-464 464v-611l-255 255l-93 -93l320 -321l-320 -321l93 -93l255 255v-611zM1429 640q0 -209 -32 -365.5t-87.5 -257t-140.5 -162.5t-181.5 -86.5t-219.5 -24.5 -t-219.5 24.5t-181.5 86.5t-140.5 162.5t-87.5 257t-32 365.5t32 365.5t87.5 257t140.5 162.5t181.5 86.5t219.5 24.5t219.5 -24.5t181.5 -86.5t140.5 -162.5t87.5 -257t32 -365.5z" /> - <glyph glyph-name="_619" unicode="" horiz-adv-x="1024" -d="M596 113l173 172l-173 172v-344zM596 823l173 172l-173 172v-344zM628 640l356 -356l-539 -540v711l-297 -296l-108 108l372 373l-372 373l108 108l297 -296v711l539 -540z" /> - <glyph glyph-name="_620" unicode="" -d="M1280 256q0 52 -38 90t-90 38t-90 -38t-38 -90t38 -90t90 -38t90 38t38 90zM512 1024q0 52 -38 90t-90 38t-90 -38t-38 -90t38 -90t90 -38t90 38t38 90zM1536 256q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5 -t112.5 -271.5zM1440 1344q0 -20 -13 -38l-1056 -1408q-19 -26 -51 -26h-160q-26 0 -45 19t-19 45q0 20 13 38l1056 1408q19 26 51 26h160q26 0 45 -19t19 -45zM768 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5 -t271.5 -112.5t112.5 -271.5z" /> - <glyph glyph-name="_621" unicode="" horiz-adv-x="1792" -d="M104 830l792 -1015l-868 630q-18 13 -25 34.5t0 42.5l101 308v0zM566 830h660l-330 -1015v0zM368 1442l198 -612h-462l198 612q8 23 33 23t33 -23zM1688 830l101 -308q7 -21 0 -42.5t-25 -34.5l-868 -630l792 1015v0zM1688 830h-462l198 612q8 23 33 23t33 -23z" /> - <glyph glyph-name="_622" unicode="" horiz-adv-x="1792" -d="M384 704h160v224h-160v-224zM1221 372v92q-104 -36 -243 -38q-135 -1 -259.5 46.5t-220.5 122.5l1 -96q88 -80 212 -128.5t272 -47.5q129 0 238 49zM640 704h640v224h-640v-224zM1792 736q0 -187 -99 -352q89 -102 89 -229q0 -157 -129.5 -268t-313.5 -111 -q-122 0 -225 52.5t-161 140.5q-19 -1 -57 -1t-57 1q-58 -88 -161 -140.5t-225 -52.5q-184 0 -313.5 111t-129.5 268q0 127 89 229q-99 165 -99 352q0 209 120 385.5t326.5 279.5t449.5 103t449.5 -103t326.5 -279.5t120 -385.5z" /> - <glyph glyph-name="_623" unicode="" -d="M515 625v-128h-252v128h252zM515 880v-127h-252v127h252zM1273 369v-128h-341v128h341zM1273 625v-128h-672v128h672zM1273 880v-127h-672v127h672zM1408 20v1240q0 8 -6 14t-14 6h-32l-378 -256l-210 171l-210 -171l-378 256h-32q-8 0 -14 -6t-6 -14v-1240q0 -8 6 -14 -t14 -6h1240q8 0 14 6t6 14zM553 1130l185 150h-406zM983 1130l221 150h-406zM1536 1260v-1240q0 -62 -43 -105t-105 -43h-1240q-62 0 -105 43t-43 105v1240q0 62 43 105t105 43h1240q62 0 105 -43t43 -105z" /> - <glyph glyph-name="_624" unicode="" horiz-adv-x="1792" -d="M896 720q-104 196 -160 278q-139 202 -347 318q-34 19 -70 36q-89 40 -94 32t34 -38l39 -31q62 -43 112.5 -93.5t94.5 -116.5t70.5 -113t70.5 -131q9 -17 13 -25q44 -84 84 -153t98 -154t115.5 -150t131 -123.5t148.5 -90.5q153 -66 154 -60q1 3 -49 37q-53 36 -81 57 -q-77 58 -179 211t-185 310zM549 177q-76 60 -132.5 125t-98 143.5t-71 154.5t-58.5 186t-52 209t-60.5 252t-76.5 289q273 0 497.5 -36t379 -92t271 -144.5t185.5 -172.5t110 -198.5t56 -199.5t12.5 -198.5t-9.5 -173t-20 -143.5t-13 -107l323 -327h-104l-281 285 -q-22 -2 -91.5 -14t-121.5 -19t-138 -6t-160.5 17t-167.5 59t-179 111z" /> - <glyph glyph-name="_625" unicode="" horiz-adv-x="1792" -d="M1374 879q-6 26 -28.5 39.5t-48.5 7.5q-261 -62 -401 -62t-401 62q-26 6 -48.5 -7.5t-28.5 -39.5t7.5 -48.5t39.5 -28.5q194 -46 303 -58q-2 -158 -15.5 -269t-26.5 -155.5t-41 -115.5l-9 -21q-10 -25 1 -49t36 -34q9 -4 23 -4q44 0 60 41l8 20q54 139 71 259h42 -q17 -120 71 -259l8 -20q16 -41 60 -41q14 0 23 4q25 10 36 34t1 49l-9 21q-28 71 -41 115.5t-26.5 155.5t-15.5 269q109 12 303 58q26 6 39.5 28.5t7.5 48.5zM1024 1024q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5z -M1600 640q0 -143 -55.5 -273.5t-150 -225t-225 -150t-273.5 -55.5t-273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5zM896 1408q-156 0 -298 -61t-245 -164t-164 -245t-61 -298t61 -298 -t164 -245t245 -164t298 -61t298 61t245 164t164 245t61 298t-61 298t-164 245t-245 164t-298 61zM1792 640q0 -182 -71 -348t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="_626" unicode="" -d="M1438 723q34 -35 29 -82l-44 -551q-4 -42 -34.5 -70t-71.5 -28q-6 0 -9 1q-44 3 -72.5 36.5t-25.5 77.5l35 429l-143 -8q55 -113 55 -240q0 -216 -148 -372l-137 137q91 101 91 235q0 145 -102.5 248t-247.5 103q-134 0 -236 -92l-137 138q120 114 284 141l264 300 -l-149 87l-181 -161q-33 -30 -77 -27.5t-73 35.5t-26.5 77t34.5 73l239 213q26 23 60 26.5t64 -14.5l488 -283q36 -21 48 -68q17 -67 -26 -117l-205 -232l371 20q49 3 83 -32zM1240 1180q-74 0 -126 52t-52 126t52 126t126 52t126.5 -52t52.5 -126t-52.5 -126t-126.5 -52z -M613 -62q106 0 196 61l139 -139q-146 -116 -335 -116q-148 0 -273.5 73t-198.5 198t-73 273q0 188 116 336l139 -139q-60 -88 -60 -197q0 -145 102.5 -247.5t247.5 -102.5z" /> - <glyph glyph-name="_627" unicode="" -d="M880 336v-160q0 -14 -9 -23t-23 -9h-160q-14 0 -23 9t-9 23v160q0 14 9 23t23 9h160q14 0 23 -9t9 -23zM1136 832q0 -50 -15 -90t-45.5 -69t-52 -44t-59.5 -36q-32 -18 -46.5 -28t-26 -24t-11.5 -29v-32q0 -14 -9 -23t-23 -9h-160q-14 0 -23 9t-9 23v68q0 35 10.5 64.5 -t24 47.5t39 35.5t41 25.5t44.5 21q53 25 75 43t22 49q0 42 -43.5 71.5t-95.5 29.5q-56 0 -95 -27q-29 -20 -80 -83q-9 -12 -25 -12q-11 0 -19 6l-108 82q-10 7 -12 20t5 23q122 192 349 192q129 0 238.5 -89.5t109.5 -214.5zM768 1280q-130 0 -248.5 -51t-204 -136.5 -t-136.5 -204t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5t-51 248.5t-136.5 204t-204 136.5t-248.5 51zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5 -t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="_628" unicode="" horiz-adv-x="1408" -d="M366 1225q-64 0 -110 45.5t-46 110.5q0 64 46 109.5t110 45.5t109.5 -45.5t45.5 -109.5q0 -65 -45.5 -110.5t-109.5 -45.5zM917 583q0 -50 -30 -67.5t-63.5 -6.5t-47.5 34l-367 438q-7 12 -14 15.5t-11 1.5l-3 -3q-7 -8 4 -21l122 -139l1 -354l-161 -457 -q-67 -192 -92 -234q-15 -26 -28 -32q-50 -26 -103 -1q-29 13 -41.5 43t-9.5 57q2 17 197 618l5 416l-85 -164l35 -222q4 -24 -1 -42t-14 -27.5t-19 -16t-17 -7.5l-7 -2q-19 -3 -34.5 3t-24 16t-14 22t-7.5 19.5t-2 9.5l-46 299l211 381q23 34 113 34q75 0 107 -40l424 -521 -q7 -5 14 -17l3 -3l-1 -1q7 -13 7 -29zM514 433q43 -113 88.5 -225t69.5 -168l24 -55q36 -93 42 -125q11 -70 -36 -97q-35 -22 -66 -16t-51 22t-29 35h-1q-6 16 -8 25l-124 351zM1338 -159q31 -49 31 -57q0 -5 -3 -7q-9 -5 -14.5 0.5t-15.5 26t-16 30.5q-114 172 -423 661 -q3 -1 7 1t7 4l3 2q11 9 11 17z" /> - <glyph glyph-name="_629" unicode="" horiz-adv-x="2304" -d="M504 542h171l-1 265zM1530 641q0 87 -50.5 140t-146.5 53h-54v-388h52q91 0 145 57t54 138zM956 1018l1 -756q0 -14 -9.5 -24t-23.5 -10h-216q-14 0 -23.5 10t-9.5 24v62h-291l-55 -81q-10 -15 -28 -15h-267q-21 0 -30.5 18t3.5 35l556 757q9 14 27 14h332q14 0 24 -10 -t10 -24zM1783 641q0 -193 -125.5 -303t-324.5 -110h-270q-14 0 -24 10t-10 24v756q0 14 10 24t24 10h268q200 0 326 -109t126 -302zM1939 640q0 -11 -0.5 -29t-8 -71.5t-21.5 -102t-44.5 -108t-73.5 -102.5h-51q38 45 66.5 104.5t41.5 112t21 98t9 72.5l1 27q0 8 -0.5 22.5 -t-7.5 60t-20 91.5t-41 111.5t-66 124.5h43q41 -47 72 -107t45.5 -111.5t23 -96t10.5 -70.5zM2123 640q0 -11 -0.5 -29t-8 -71.5t-21.5 -102t-45 -108t-74 -102.5h-51q38 45 66.5 104.5t41.5 112t21 98t9 72.5l1 27q0 8 -0.5 22.5t-7.5 60t-19.5 91.5t-40.5 111.5t-66 124.5 -h43q41 -47 72 -107t45.5 -111.5t23 -96t10.5 -70.5zM2304 640q0 -11 -0.5 -29t-8 -71.5t-21.5 -102t-44.5 -108t-73.5 -102.5h-51q38 45 66 104.5t41 112t21 98t9 72.5l1 27q0 8 -0.5 22.5t-7.5 60t-19.5 91.5t-40.5 111.5t-66 124.5h43q41 -47 72 -107t45.5 -111.5t23 -96 -t9.5 -70.5z" /> - <glyph glyph-name="uniF2A0" unicode="" horiz-adv-x="1408" -d="M617 -153q0 11 -13 58t-31 107t-20 69q-1 4 -5 26.5t-8.5 36t-13.5 21.5q-15 14 -51 14q-23 0 -70 -5.5t-71 -5.5q-34 0 -47 11q-6 5 -11 15.5t-7.5 20t-6.5 24t-5 18.5q-37 128 -37 255t37 255q1 4 5 18.5t6.5 24t7.5 20t11 15.5q13 11 47 11q24 0 71 -5.5t70 -5.5 -q36 0 51 14q9 8 13.5 21.5t8.5 36t5 26.5q2 9 20 69t31 107t13 58q0 22 -43.5 52.5t-75.5 42.5q-20 8 -45 8q-34 0 -98 -18q-57 -17 -96.5 -40.5t-71 -66t-46 -70t-45.5 -94.5q-6 -12 -9 -19q-49 -107 -68 -216t-19 -244t19 -244t68 -216q56 -122 83 -161q63 -91 179 -127 -l6 -2q64 -18 98 -18q25 0 45 8q32 12 75.5 42.5t43.5 52.5zM776 760q-26 0 -45 19t-19 45.5t19 45.5q37 37 37 90q0 52 -37 91q-19 19 -19 45t19 45t45 19t45 -19q75 -75 75 -181t-75 -181q-21 -19 -45 -19zM957 579q-27 0 -45 19q-19 19 -19 45t19 45q112 114 112 272 -t-112 272q-19 19 -19 45t19 45t45 19t45 -19q150 -150 150 -362t-150 -362q-18 -19 -45 -19zM1138 398q-27 0 -45 19q-19 19 -19 45t19 45q90 91 138.5 208t48.5 245t-48.5 245t-138.5 208q-19 19 -19 45t19 45t45 19t45 -19q109 -109 167 -249t58 -294t-58 -294t-167 -249 -q-18 -19 -45 -19z" /> - <glyph glyph-name="uniF2A1" unicode="" horiz-adv-x="2176" -d="M192 352q-66 0 -113 -47t-47 -113t47 -113t113 -47t113 47t47 113t-47 113t-113 47zM704 352q-66 0 -113 -47t-47 -113t47 -113t113 -47t113 47t47 113t-47 113t-113 47zM704 864q-66 0 -113 -47t-47 -113t47 -113t113 -47t113 47t47 113t-47 113t-113 47zM1472 352 -q-66 0 -113 -47t-47 -113t47 -113t113 -47t113 47t47 113t-47 113t-113 47zM1984 352q-66 0 -113 -47t-47 -113t47 -113t113 -47t113 47t47 113t-47 113t-113 47zM1472 864q-66 0 -113 -47t-47 -113t47 -113t113 -47t113 47t47 113t-47 113t-113 47zM1984 864 -q-66 0 -113 -47t-47 -113t47 -113t113 -47t113 47t47 113t-47 113t-113 47zM1984 1376q-66 0 -113 -47t-47 -113t47 -113t113 -47t113 47t47 113t-47 113t-113 47zM384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 192q0 -80 -56 -136 -t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM384 704q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 704q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM384 1216q0 -80 -56 -136t-136 -56 -t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 1216q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM2176 192q0 -80 -56 -136t-136 -56t-136 56 -t-56 136t56 136t136 56t136 -56t56 -136zM1664 704q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM2176 704q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 1216q0 -80 -56 -136t-136 -56t-136 56t-56 136 -t56 136t136 56t136 -56t56 -136zM2176 1216q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136z" /> - <glyph glyph-name="uniF2A2" unicode="" horiz-adv-x="1792" -d="M128 -192q0 -26 -19 -45t-45 -19t-45 19t-19 45t19 45t45 19t45 -19t19 -45zM320 0q0 -26 -19 -45t-45 -19t-45 19t-19 45t19 45t45 19t45 -19t19 -45zM365 365l256 -256l-90 -90l-256 256zM704 384q0 -26 -19 -45t-45 -19t-45 19t-19 45t19 45t45 19t45 -19t19 -45z -M1411 704q0 -59 -11.5 -108.5t-37.5 -93.5t-44 -67.5t-53 -64.5q-31 -35 -45.5 -54t-33.5 -50t-26.5 -64t-7.5 -74q0 -159 -112.5 -271.5t-271.5 -112.5q-26 0 -45 19t-19 45t19 45t45 19q106 0 181 75t75 181q0 57 11.5 105.5t37 91t43.5 66.5t52 63q40 46 59.5 72 -t37.5 74.5t18 103.5q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5q0 -26 -19 -45t-45 -19t-45 19t-19 45q0 117 45.5 223.5t123 184t184 123t223.5 45.5t223.5 -45.5t184 -123t123 -184t45.5 -223.5zM896 576q0 -26 -19 -45t-45 -19t-45 19t-19 45t19 45 -t45 19t45 -19t19 -45zM1184 704q0 -26 -19 -45t-45 -19t-45 19t-19 45q0 93 -65.5 158.5t-158.5 65.5q-92 0 -158 -65.5t-66 -158.5q0 -26 -19 -45t-45 -19t-45 19t-19 45q0 146 103 249t249 103t249 -103t103 -249zM1578 993q10 -25 -1 -49t-36 -34q-9 -4 -23 -4 -q-19 0 -35.5 11t-23.5 30q-68 178 -224 295q-21 16 -25 42t12 47q17 21 43 25t47 -12q183 -137 266 -351zM1788 1074q9 -25 -1.5 -49t-35.5 -34q-11 -4 -23 -4q-44 0 -60 41q-92 238 -297 393q-22 16 -25.5 42t12.5 47q16 22 42 25.5t47 -12.5q235 -175 341 -449z" /> - <glyph glyph-name="uniF2A3" unicode="" horiz-adv-x="2304" -d="M1032 576q-59 2 -84 55q-17 34 -48 53.5t-68 19.5q-53 0 -90.5 -37.5t-37.5 -90.5q0 -56 36 -89l10 -8q34 -31 82 -31q37 0 68 19.5t48 53.5q25 53 84 55zM1600 704q0 56 -36 89l-10 8q-34 31 -82 31q-37 0 -68 -19.5t-48 -53.5q-25 -53 -84 -55q59 -2 84 -55 -q17 -34 48 -53.5t68 -19.5q53 0 90.5 37.5t37.5 90.5zM1174 925q-17 -35 -55 -48t-73 4q-62 31 -134 31q-51 0 -99 -17q3 0 9.5 0.5t9.5 0.5q92 0 170.5 -50t118.5 -133q17 -36 3.5 -73.5t-49.5 -54.5q-18 -9 -39 -9q21 0 39 -9q36 -17 49.5 -54.5t-3.5 -73.5 -q-40 -83 -118.5 -133t-170.5 -50h-6q-16 2 -44 4l-290 27l-239 -120q-14 -7 -29 -7q-40 0 -57 35l-160 320q-11 23 -4 47.5t29 37.5l209 119l148 267q17 155 91.5 291.5t195.5 236.5q31 25 70.5 21.5t64.5 -34.5t21.5 -70t-34.5 -65q-70 -59 -117 -128q123 84 267 101 -q40 5 71.5 -19t35.5 -64q5 -40 -19 -71.5t-64 -35.5q-84 -10 -159 -55q46 10 99 10q115 0 218 -50q36 -18 49 -55.5t-5 -73.5zM2137 1085l160 -320q11 -23 4 -47.5t-29 -37.5l-209 -119l-148 -267q-17 -155 -91.5 -291.5t-195.5 -236.5q-26 -22 -61 -22q-45 0 -74 35 -q-25 31 -21.5 70t34.5 65q70 59 117 128q-123 -84 -267 -101q-4 -1 -12 -1q-36 0 -63.5 24t-31.5 60q-5 40 19 71.5t64 35.5q84 10 159 55q-46 -10 -99 -10q-115 0 -218 50q-36 18 -49 55.5t5 73.5q17 35 55 48t73 -4q62 -31 134 -31q51 0 99 17q-3 0 -9.5 -0.5t-9.5 -0.5 -q-92 0 -170.5 50t-118.5 133q-17 36 -3.5 73.5t49.5 54.5q18 9 39 9q-21 0 -39 9q-36 17 -49.5 54.5t3.5 73.5q40 83 118.5 133t170.5 50h6h1q14 -2 42 -4l291 -27l239 120q14 7 29 7q40 0 57 -35z" /> - <glyph glyph-name="uniF2A4" unicode="" horiz-adv-x="1792" -d="M1056 704q0 -26 19 -45t45 -19t45 19t19 45q0 146 -103 249t-249 103t-249 -103t-103 -249q0 -26 19 -45t45 -19t45 19t19 45q0 93 66 158.5t158 65.5t158 -65.5t66 -158.5zM835 1280q-117 0 -223.5 -45.5t-184 -123t-123 -184t-45.5 -223.5q0 -26 19 -45t45 -19t45 19 -t19 45q0 185 131.5 316.5t316.5 131.5t316.5 -131.5t131.5 -316.5q0 -55 -18 -103.5t-37.5 -74.5t-59.5 -72q-34 -39 -52 -63t-43.5 -66.5t-37 -91t-11.5 -105.5q0 -106 -75 -181t-181 -75q-26 0 -45 -19t-19 -45t19 -45t45 -19q159 0 271.5 112.5t112.5 271.5q0 41 7.5 74 -t26.5 64t33.5 50t45.5 54q35 41 53 64.5t44 67.5t37.5 93.5t11.5 108.5q0 117 -45.5 223.5t-123 184t-184 123t-223.5 45.5zM591 561l226 -226l-579 -579q-12 -12 -29 -12t-29 12l-168 168q-12 12 -12 29t12 29zM1612 1524l168 -168q12 -12 12 -29t-12 -30l-233 -233 -l-26 -25l-71 -71q-66 153 -195 258l91 91l207 207q13 12 30 12t29 -12z" /> - <glyph glyph-name="uniF2A5" unicode="" -d="M866 1021q0 -27 -13 -94q-11 -50 -31.5 -150t-30.5 -150q-2 -11 -4.5 -12.5t-13.5 -2.5q-20 -2 -31 -2q-58 0 -84 49.5t-26 113.5q0 88 35 174t103 124q28 14 51 14q28 0 36.5 -16.5t8.5 -47.5zM1352 597q0 14 -39 75.5t-52 66.5q-21 8 -34 8q-91 0 -226 -77l-2 2 -q3 22 27.5 135t24.5 178q0 233 -242 233q-24 0 -68 -6q-94 -17 -168.5 -89.5t-111.5 -166.5t-37 -189q0 -146 80.5 -225t227.5 -79q25 0 25 -3t-1 -5q-4 -34 -26 -117q-14 -52 -51.5 -101t-82.5 -49q-42 0 -42 47q0 24 10.5 47.5t25 39.5t29.5 28.5t26 20t11 8.5q0 3 -7 10 -q-24 22 -58.5 36.5t-65.5 14.5q-35 0 -63.5 -34t-41 -75t-12.5 -75q0 -88 51.5 -142t138.5 -54q82 0 155 53t117.5 126t65.5 153q6 22 15.5 66.5t14.5 66.5q3 12 14 18q118 60 227 60q48 0 127 -18q1 -1 4 -1q5 0 9.5 4.5t4.5 8.5zM1536 1120v-960q0 -119 -84.5 -203.5 -t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="uniF2A6" unicode="" horiz-adv-x="1535" -d="M744 1231q0 24 -2 38.5t-8.5 30t-21 23t-37.5 7.5q-39 0 -78 -23q-105 -58 -159 -190.5t-54 -269.5q0 -44 8.5 -85.5t26.5 -80.5t52.5 -62.5t81.5 -23.5q4 0 18 -0.5t20 0t16 3t15 8.5t7 16q16 77 48 231.5t48 231.5q19 91 19 146zM1498 575q0 -7 -7.5 -13.5t-15.5 -6.5 -l-6 1q-22 3 -62 11t-72 12.5t-63 4.5q-167 0 -351 -93q-15 -8 -21 -27q-10 -36 -24.5 -105.5t-22.5 -100.5q-23 -91 -70 -179.5t-112.5 -164.5t-154.5 -123t-185 -47q-135 0 -214.5 83.5t-79.5 219.5q0 53 19.5 117t63 116.5t97.5 52.5q38 0 120 -33.5t83 -61.5 -q0 -1 -16.5 -12.5t-39.5 -31t-46 -44.5t-39 -61t-16 -74q0 -33 16.5 -53t48.5 -20q45 0 85 31.5t66.5 78t48 105.5t32.5 107t16 90v9q0 2 -3.5 3.5t-8.5 1.5h-10t-10 -0.5t-6 -0.5q-227 0 -352 122.5t-125 348.5q0 108 34.5 221t96 210t156 167.5t204.5 89.5q52 9 106 9 -q374 0 374 -360q0 -98 -38 -273t-43 -211l3 -3q101 57 182.5 88t167.5 31q22 0 53 -13q19 -7 80 -102.5t61 -116.5z" /> - <glyph glyph-name="uniF2A7" unicode="" horiz-adv-x="1664" -d="M831 863q32 0 59 -18l222 -148q61 -40 110 -97l146 -170q40 -46 29 -106l-72 -413q-6 -32 -29.5 -53.5t-55.5 -25.5l-527 -56l-352 -32h-9q-39 0 -67.5 28t-28.5 68q0 37 27 64t65 32l260 32h-448q-41 0 -69.5 30t-26.5 71q2 39 32 65t69 26l442 1l-521 64q-41 5 -66 37 -t-19 73q6 35 34.5 57.5t65.5 22.5h10l481 -60l-351 94q-38 10 -62 41.5t-18 68.5q6 36 33 58.5t62 22.5q6 0 20 -2l448 -96l217 -37q1 0 3 -0.5t3 -0.5q23 0 30.5 23t-12.5 36l-186 125q-35 23 -42 63.5t18 73.5q27 38 76 38zM761 661l186 -125l-218 37l-5 2l-36 38 -l-238 262q-1 1 -2.5 3.5t-2.5 3.5q-24 31 -18.5 70t37.5 64q31 23 68 17.5t64 -33.5l142 -147q-2 -1 -5 -3.5t-4 -4.5q-32 -45 -23 -99t55 -85zM1648 1115l15 -266q4 -73 -11 -147l-48 -219q-12 -59 -67 -87l-106 -54q2 62 -39 109l-146 170q-53 61 -117 103l-222 148 -q-34 23 -76 23q-51 0 -88 -37l-235 312q-25 33 -18 73.5t41 63.5q33 22 71.5 14t62.5 -40l266 -352l-262 455q-21 35 -10.5 75t47.5 59q35 18 72.5 6t57.5 -46l241 -420l-136 337q-15 35 -4.5 74t44.5 56q37 19 76 6t56 -51l193 -415l101 -196q8 -15 23 -17.5t27 7.5t11 26 -l-12 224q-2 41 26 71t69 31q39 0 67 -28.5t30 -67.5z" /> - <glyph glyph-name="uniF2A8" unicode="" horiz-adv-x="1792" -d="M335 180q-2 0 -6 2q-86 57 -168.5 145t-139.5 180q-21 30 -21 69q0 9 2 19t4 18t7 18t8.5 16t10.5 17t10 15t12 15.5t11 14.5q184 251 452 365q-110 198 -110 211q0 19 17 29q116 64 128 64q18 0 28 -16l124 -229q92 19 192 19q266 0 497.5 -137.5t378.5 -369.5 -q20 -31 20 -69t-20 -69q-91 -142 -218.5 -253.5t-278.5 -175.5q110 -198 110 -211q0 -20 -17 -29q-116 -64 -127 -64q-19 0 -29 16l-124 229l-64 119l-444 820l7 7q-58 -24 -99 -47q3 -5 127 -234t243 -449t119 -223q0 -7 -9 -9q-13 -3 -72 -3q-57 0 -60 7l-456 841 -q-39 -28 -82 -68q24 -43 214 -393.5t190 -354.5q0 -10 -11 -10q-14 0 -82.5 22t-72.5 28l-106 197l-224 413q-44 -53 -78 -106q2 -3 18 -25t23 -34l176 -327q0 -10 -10 -10zM1165 282l49 -91q273 111 450 385q-180 277 -459 389q67 -64 103 -148.5t36 -176.5 -q0 -106 -47 -200.5t-132 -157.5zM848 896q0 -20 14 -34t34 -14q86 0 147 -61t61 -147q0 -20 14 -34t34 -14t34 14t14 34q0 126 -89 215t-215 89q-20 0 -34 -14t-14 -34zM1214 961l-9 4l7 -7z" /> - <glyph glyph-name="uniF2A9" unicode="" horiz-adv-x="1280" -d="M1050 430q0 -215 -147 -374q-148 -161 -378 -161q-232 0 -378 161q-147 159 -147 374q0 147 68 270.5t189 196.5t268 73q96 0 182 -31q-32 -62 -39 -126q-66 28 -143 28q-167 0 -280.5 -123t-113.5 -291q0 -170 112.5 -288.5t281.5 -118.5t281 118.5t112 288.5 -q0 89 -32 166q66 13 123 49q41 -98 41 -212zM846 619q0 -192 -79.5 -345t-238.5 -253l-14 -1q-29 0 -62 5q83 32 146.5 102.5t99.5 154.5t58.5 189t30 192.5t7.5 178.5q0 69 -3 103q55 -160 55 -326zM791 947v-2q-73 214 -206 440q88 -59 142.5 -186.5t63.5 -251.5z -M1035 744q-83 0 -160 75q218 120 290 247q19 37 21 56q-42 -94 -139.5 -166.5t-204.5 -97.5q-35 54 -35 113q0 37 17 79t43 68q46 44 157 74q59 16 106 58.5t74 100.5q74 -105 74 -253q0 -109 -24 -170q-32 -77 -88.5 -130.5t-130.5 -53.5z" /> - <glyph glyph-name="uniF2AA" unicode="" -d="M1050 495q0 78 -28 147q-41 -25 -85 -34q22 -50 22 -114q0 -117 -77 -198.5t-193 -81.5t-193.5 81.5t-77.5 198.5q0 115 78 199.5t193 84.5q53 0 98 -19q4 43 27 87q-60 21 -125 21q-154 0 -257.5 -108.5t-103.5 -263.5t103.5 -261t257.5 -106t257.5 106.5t103.5 260.5z -M872 850q2 -24 2 -71q0 -63 -5 -123t-20.5 -132.5t-40.5 -130t-68.5 -106t-100.5 -70.5q21 -3 42 -3h10q219 139 219 411q0 116 -38 225zM872 850q-4 80 -44 171.5t-98 130.5q92 -156 142 -302zM1207 955q0 102 -51 174q-41 -86 -124 -109q-69 -19 -109 -53.5t-40 -99.5 -q0 -40 24 -77q74 17 140.5 67t95.5 115q-4 -52 -74.5 -111.5t-138.5 -97.5q52 -52 110 -52q51 0 90 37t60 90q17 42 17 117zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5 -t84.5 -203.5z" /> - <glyph glyph-name="uniF2AB" unicode="" -d="M1279 388q0 22 -22 27q-67 15 -118 59t-80 108q-7 19 -7 25q0 15 19.5 26t43 17t43 20.5t19.5 36.5q0 19 -18.5 31.5t-38.5 12.5q-12 0 -32 -8t-31 -8q-4 0 -12 2q5 95 5 114q0 79 -17 114q-36 78 -103 121.5t-152 43.5q-199 0 -275 -165q-17 -35 -17 -114q0 -19 5 -114 -q-4 -2 -14 -2q-12 0 -32 7.5t-30 7.5q-21 0 -38.5 -12t-17.5 -32q0 -21 19.5 -35.5t43 -20.5t43 -17t19.5 -26q0 -6 -7 -25q-64 -138 -198 -167q-22 -5 -22 -27q0 -46 137 -68q2 -5 6 -26t11.5 -30.5t23.5 -9.5q12 0 37.5 4.5t39.5 4.5q35 0 67 -15t54 -32.5t57.5 -32.5 -t76.5 -15q43 0 79 15t57.5 32.5t53.5 32.5t67 15q14 0 39.5 -4t38.5 -4q16 0 23 10t11 30t6 25q137 22 137 68zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 -t103 -385.5z" /> - <glyph glyph-name="uniF2AC" unicode="" horiz-adv-x="1664" -d="M848 1408q134 1 240.5 -68.5t163.5 -192.5q27 -58 27 -179q0 -47 -9 -191q14 -7 28 -7q18 0 51 13.5t51 13.5q29 0 56 -18t27 -46q0 -32 -31.5 -54t-69 -31.5t-69 -29t-31.5 -47.5q0 -15 12 -43q37 -82 102.5 -150t144.5 -101q28 -12 80 -23q28 -6 28 -35 -q0 -70 -219 -103q-7 -11 -11 -39t-14 -46.5t-33 -18.5q-20 0 -62 6.5t-64 6.5q-37 0 -62 -5q-32 -5 -63 -22.5t-58 -38t-58 -40.5t-76 -33.5t-99 -13.5q-52 0 -96.5 13.5t-75 33.5t-57.5 40.5t-58 38t-62 22.5q-26 5 -63 5q-24 0 -65.5 -7.5t-58.5 -7.5q-25 0 -35 18.5 -t-14 47.5t-11 40q-219 33 -219 103q0 29 28 35q52 11 80 23q78 32 144.5 101t102.5 150q12 28 12 43q0 28 -31.5 47.5t-69.5 29.5t-69.5 31.5t-31.5 52.5q0 27 26 45.5t55 18.5q15 0 48 -13t53 -13q18 0 32 7q-9 142 -9 190q0 122 27 180q64 137 172 198t264 63z" /> - <glyph glyph-name="uniF2AD" unicode="" -d="M1280 388q0 22 -22 27q-67 14 -118 58t-80 109q-7 14 -7 25q0 15 19.5 26t42.5 17t42.5 20.5t19.5 36.5q0 19 -18.5 31.5t-38.5 12.5q-11 0 -31 -8t-32 -8q-4 0 -12 2q5 63 5 115q0 78 -17 114q-36 78 -102.5 121.5t-152.5 43.5q-198 0 -275 -165q-18 -38 -18 -115 -q0 -38 6 -114q-10 -2 -15 -2q-11 0 -31.5 8t-30.5 8q-20 0 -37.5 -12.5t-17.5 -32.5q0 -21 19.5 -35.5t42.5 -20.5t42.5 -17t19.5 -26q0 -11 -7 -25q-64 -138 -198 -167q-22 -5 -22 -27q0 -47 138 -69q2 -5 6 -26t11 -30.5t23 -9.5q13 0 38.5 5t38.5 5q35 0 67.5 -15 -t54.5 -32.5t57.5 -32.5t76.5 -15q43 0 79 15t57.5 32.5t54 32.5t67.5 15q13 0 39 -4.5t39 -4.5q15 0 22.5 9.5t11.5 31t5 24.5q138 22 138 69zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960 -q119 0 203.5 -84.5t84.5 -203.5z" /> - <glyph glyph-name="uniF2AE" unicode="" horiz-adv-x="2304" -d="M2304 1536q-69 -46 -125 -92t-89 -81t-59.5 -71.5t-37.5 -57.5t-22 -44.5t-14 -29.5q-10 -18 -35.5 -136.5t-48.5 -164.5q-15 -29 -50 -60.5t-67.5 -50.5t-72.5 -41t-48 -28q-47 -31 -151 -231q-341 14 -630 -158q-92 -53 -303 -179q47 16 86 31t55 22l15 7 -q71 27 163 64.5t133.5 53.5t108 34.5t142.5 31.5q186 31 465 -7q1 0 10 -3q11 -6 14 -17t-3 -22l-194 -345q-15 -29 -47 -22q-128 24 -354 24q-146 0 -402 -44.5t-392 -46.5q-82 -1 -149 13t-107 37t-61 40t-33 34l-1 1v2q0 6 6 6q138 0 371 55q192 366 374.5 524t383.5 158 -q5 0 14.5 -0.5t38 -5t55 -12t61.5 -24.5t63 -39.5t54 -59t40 -82.5l102 177q2 4 21 42.5t44.5 86.5t61 109.5t84 133.5t100.5 137q66 82 128 141.5t121.5 96.5t92.5 53.5t88 39.5z" /> - <glyph glyph-name="uniF2B0" unicode="" -d="M1322 640q0 -45 -5 -76l-236 14l224 -78q-19 -73 -58 -141l-214 103l177 -158q-44 -61 -107 -108l-157 178l103 -215q-61 -37 -140 -59l-79 228l14 -240q-38 -6 -76 -6t-76 6l14 238l-78 -226q-74 19 -140 59l103 215l-157 -178q-59 43 -108 108l178 158l-214 -104 -q-39 69 -58 141l224 79l-237 -14q-5 42 -5 76q0 35 5 77l238 -14l-225 79q19 73 58 140l214 -104l-177 159q46 61 107 108l158 -178l-103 215q67 39 140 58l77 -224l-13 236q36 6 75 6q38 0 76 -6l-14 -237l78 225q74 -19 140 -59l-103 -214l158 178q61 -47 107 -108 -l-177 -159l213 104q37 -62 58 -141l-224 -78l237 14q5 -31 5 -77zM1352 640q0 160 -78.5 295.5t-213 214t-292.5 78.5q-119 0 -227 -46.5t-186.5 -125t-124.5 -187.5t-46 -229q0 -119 46 -228t124.5 -187.5t186.5 -125t227 -46.5q158 0 292.5 78.5t213 214t78.5 294.5z -M1425 1023v-766l-657 -383l-657 383v766l657 383zM768 -183l708 412v823l-708 411l-708 -411v-823zM1536 1088v-896l-768 -448l-768 448v896l768 448z" /> - <glyph glyph-name="uniF2B1" unicode="" horiz-adv-x="1664" -d="M339 1318h691l-26 -72h-665q-110 0 -188.5 -79t-78.5 -189v-771q0 -95 60.5 -169.5t153.5 -93.5q23 -5 98 -5v-72h-45q-140 0 -239.5 100t-99.5 240v771q0 140 99.5 240t239.5 100zM1190 1536h247l-482 -1294q-23 -61 -40.5 -103.5t-45 -98t-54 -93.5t-64.5 -78.5 -t-79.5 -65t-95.5 -41t-116 -18.5v195q163 26 220 182q20 52 20 105q0 54 -20 106l-285 733h228l187 -585zM1664 978v-1111h-795q37 55 45 73h678v1038q0 85 -49.5 155t-129.5 99l25 67q101 -34 163.5 -123.5t62.5 -197.5z" /> - <glyph glyph-name="uniF2B2" unicode="" horiz-adv-x="1792" -d="M852 1227q0 -29 -17 -52.5t-45 -23.5t-45 23.5t-17 52.5t17 52.5t45 23.5t45 -23.5t17 -52.5zM688 -149v114q0 30 -20.5 51.5t-50.5 21.5t-50 -21.5t-20 -51.5v-114q0 -30 20.5 -52t49.5 -22q30 0 50.5 22t20.5 52zM860 -149v114q0 30 -20 51.5t-50 21.5t-50.5 -21.5 -t-20.5 -51.5v-114q0 -30 20.5 -52t50.5 -22q29 0 49.5 22t20.5 52zM1034 -149v114q0 30 -20.5 51.5t-50.5 21.5t-50.5 -21.5t-20.5 -51.5v-114q0 -30 20.5 -52t50.5 -22t50.5 22t20.5 52zM1208 -149v114q0 30 -20.5 51.5t-50.5 21.5t-50.5 -21.5t-20.5 -51.5v-114 -q0 -30 20.5 -52t50.5 -22t50.5 22t20.5 52zM1476 535q-84 -160 -232 -259.5t-323 -99.5q-123 0 -229.5 51.5t-178.5 137t-113 197.5t-41 232q0 88 21 174q-104 -175 -104 -390q0 -162 65 -312t185 -251q30 57 91 57q56 0 86 -50q32 50 87 50q56 0 86 -50q32 50 87 50t87 -50 -q30 50 86 50q28 0 52.5 -15.5t37.5 -40.5q112 94 177 231.5t73 287.5zM1326 564q0 75 -72 75q-17 0 -47 -6q-95 -19 -149 -19q-226 0 -226 243q0 86 30 204q-83 -127 -83 -275q0 -150 89 -260.5t235 -110.5q111 0 210 70q13 48 13 79zM884 1223q0 50 -32 89.5t-81 39.5 -t-81 -39.5t-32 -89.5q0 -51 31.5 -90.5t81.5 -39.5t81.5 39.5t31.5 90.5zM1513 884q0 96 -37.5 179t-113 137t-173.5 54q-77 0 -149 -35t-127 -94q-48 -159 -48 -268q0 -104 45.5 -157t147.5 -53q53 0 142 19q36 6 53 6q51 0 77.5 -28t26.5 -80q0 -26 -4 -46 -q75 68 117.5 165.5t42.5 200.5zM1792 667q0 -111 -33.5 -249.5t-93.5 -204.5q-58 -64 -195 -142.5t-228 -104.5l-4 -1v-114q0 -43 -29.5 -75t-72.5 -32q-56 0 -86 50q-32 -50 -87 -50t-87 50q-30 -50 -86 -50q-55 0 -87 50q-30 -50 -86 -50q-47 0 -75 33.5t-28 81.5 -q-90 -68 -198 -68q-118 0 -211 80q54 1 106 20q-113 31 -182 127q32 -7 71 -7q89 0 164 46q-192 192 -240 306q-24 56 -24 160q0 57 9 125.5t31.5 146.5t55 141t86.5 105t120 42q59 0 81 -52q19 29 42 54q2 3 12 13t13 16q10 15 23 38t25 42t28 39q87 111 211.5 177 -t260.5 66q35 0 62 -4q59 64 146 64q83 0 140 -57q5 -5 5 -12q0 -5 -6 -13.5t-12.5 -16t-16 -17l-10.5 -10.5q17 -6 36 -18t19 -24q0 -6 -16 -25q157 -138 197 -378q25 30 60 30q45 0 100 -49q90 -80 90 -279z" /> - <glyph glyph-name="uniF2B3" unicode="" -d="M917 631q0 33 -6 64h-362v-132h217q-12 -76 -74.5 -120.5t-142.5 -44.5q-99 0 -169 71.5t-70 170.5t70 170.5t169 71.5q93 0 153 -59l104 101q-108 100 -257 100q-160 0 -272 -112.5t-112 -271.5t112 -271.5t272 -112.5q165 0 266.5 105t101.5 270zM1262 585h109v110 -h-109v110h-110v-110h-110v-110h110v-110h110v110zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" /> - <glyph glyph-name="uniF2B4" unicode="" -d="M1536 1024v-839q0 -48 -49 -62q-174 -52 -338 -52q-73 0 -215.5 29.5t-227.5 29.5q-164 0 -370 -48v-338h-160v1368q-63 25 -101 81t-38 124q0 91 64 155t155 64t155 -64t64 -155q0 -68 -38 -124t-101 -81v-68q190 44 343 44q99 0 198 -15q14 -2 111.5 -22.5t149.5 -20.5 -q77 0 165 18q11 2 80 21t89 19q26 0 45 -19t19 -45z" /> - <glyph glyph-name="uniF2B5" unicode="" horiz-adv-x="2304" -d="M192 384q40 0 56 32t0 64t-56 32t-56 -32t0 -64t56 -32zM1665 442q-10 13 -38.5 50t-41.5 54t-38 49t-42.5 53t-40.5 47t-45 49l-125 -140q-83 -94 -208.5 -92t-205.5 98q-57 69 -56.5 158t58.5 157l177 206q-22 11 -51 16.5t-47.5 6t-56.5 -0.5t-49 -1q-92 0 -158 -66 -l-158 -158h-155v-544q5 0 21 0.5t22 0t19.5 -2t20.5 -4.5t17.5 -8.5t18.5 -13.5l297 -292q115 -111 227 -111q78 0 125 47q57 -20 112.5 8t72.5 85q74 -6 127 44q20 18 36 45.5t14 50.5q10 -10 43 -10q43 0 77 21t49.5 53t12 71.5t-30.5 73.5zM1824 384h96v512h-93l-157 180 -q-66 76 -169 76h-167q-89 0 -146 -67l-209 -243q-28 -33 -28 -75t27 -75q43 -51 110 -52t111 49l193 218q25 23 53.5 21.5t47 -27t8.5 -56.5q16 -19 56 -63t60 -68q29 -36 82.5 -105.5t64.5 -84.5q52 -66 60 -140zM2112 384q40 0 56 32t0 64t-56 32t-56 -32t0 -64t56 -32z -M2304 960v-640q0 -26 -19 -45t-45 -19h-434q-27 -65 -82 -106.5t-125 -51.5q-33 -48 -80.5 -81.5t-102.5 -45.5q-42 -53 -104.5 -81.5t-128.5 -24.5q-60 -34 -126 -39.5t-127.5 14t-117 53.5t-103.5 81l-287 282h-358q-26 0 -45 19t-19 45v672q0 26 19 45t45 19h421 -q14 14 47 48t47.5 48t44 40t50.5 37.5t51 25.5t62 19.5t68 5.5h117q99 0 181 -56q82 56 181 56h167q35 0 67 -6t56.5 -14.5t51.5 -26.5t44.5 -31t43 -39.5t39 -42t41 -48t41.5 -48.5h355q26 0 45 -19t19 -45z" /> - <glyph glyph-name="uniF2B6" unicode="" horiz-adv-x="1792" -d="M1792 882v-978q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v978q0 15 11 24q8 7 39 34.5t41.5 36t45.5 37.5t70 55.5t96 73t143.5 107t192.5 140.5q5 4 52.5 40t71.5 52.5t64 35t69 18.5t69 -18.5t65 -35.5t71 -52t52 -40q110 -80 192.5 -140.5t143.5 -107 -t96 -73t70 -55.5t45.5 -37.5t41.5 -36t39 -34.5q11 -9 11 -24zM1228 297q263 191 345 252q11 8 12.5 20.5t-6.5 23.5l-38 52q-8 11 -21 12.5t-24 -6.5q-231 -169 -343 -250q-5 -3 -52 -39t-71.5 -52.5t-64.5 -35t-69 -18.5t-69 18.5t-64.5 35t-71.5 52.5t-52 39 -q-186 134 -343 250q-11 8 -24 6.5t-21 -12.5l-38 -52q-8 -11 -6.5 -23.5t12.5 -20.5q82 -61 345 -252q10 -8 50 -38t65 -47t64 -39.5t77.5 -33.5t75.5 -11t75.5 11t79 34.5t64.5 39.5t65 47.5t48 36.5z" /> - <glyph glyph-name="uniF2B7" unicode="" horiz-adv-x="1792" -d="M1474 623l39 -51q8 -11 6.5 -23.5t-11.5 -20.5q-43 -34 -126.5 -98.5t-146.5 -113t-67 -51.5q-39 -32 -60 -48t-60.5 -41t-76.5 -36.5t-74 -11.5h-1h-1q-37 0 -74 11.5t-76 36.5t-61 41.5t-60 47.5q-5 4 -65 50.5t-143.5 111t-122.5 94.5q-11 8 -12.5 20.5t6.5 23.5 -l37 52q8 11 21.5 13t24.5 -7q94 -73 306 -236q5 -4 43.5 -35t60.5 -46.5t56.5 -32.5t58.5 -17h1h1q24 0 58.5 17t56.5 32.5t60.5 46.5t43.5 35q258 198 313 242q11 8 24 6.5t21 -12.5zM1664 -96v928q-90 83 -159 139q-91 74 -389 304q-3 2 -43 35t-61 48t-56 32.5t-59 17.5 -h-1h-1q-24 0 -59 -17.5t-56 -32.5t-61 -48t-43 -35q-215 -166 -315.5 -245.5t-129.5 -104t-82 -74.5q-14 -12 -21 -19v-928q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1792 832v-928q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v928q0 56 41 94 -q123 114 350 290.5t233 181.5q36 30 59 47.5t61.5 42t76 36.5t74.5 12h1h1q37 0 74.5 -12t76 -36.5t61.5 -42t59 -47.5q43 -36 156 -122t226 -177t201 -173q41 -38 41 -94z" /> - <glyph glyph-name="uniF2B8" unicode="" -d="M330 1l202 -214l-34 236l-216 213zM556 -225l274 218l-11 245l-300 -215zM245 413l227 -213l-48 327l-245 204zM495 189l317 214l-14 324l-352 -200zM843 178l95 -80l-2 239l-103 79q0 -1 1 -8.5t0 -12t-5 -7.5l-78 -52l85 -70q7 -6 7 -88zM138 930l256 -200l-68 465 -l-279 173zM1173 267l15 234l-230 -164l2 -240zM417 722l373 194l-19 441l-423 -163zM1270 357l20 233l-226 142l-2 -105l144 -95q6 -4 4 -9l-7 -119zM1461 496l30 222l-179 -128l-20 -228zM1273 329l-71 49l-8 -117q0 -5 -4 -8l-234 -187q-7 -5 -14 0l-98 83l7 -161 -q0 -5 -4 -8l-293 -234q-4 -2 -6 -2q-8 2 -8 3l-228 242q-4 4 -59 277q-2 7 5 11l61 37q-94 86 -95 92l-72 351q-2 7 6 12l94 45q-133 100 -135 108l-96 466q-2 10 7 13l433 135q5 0 8 -1l317 -153q6 -4 6 -9l20 -463q0 -7 -6 -10l-118 -61l126 -85q5 -2 5 -8l5 -123l121 74 -q5 4 11 0l84 -56l3 110q0 6 5 9l206 126q6 3 11 0l245 -135q4 -4 5 -7t-6.5 -60t-17.5 -124.5t-10 -70.5q0 -5 -4 -7l-191 -153q-6 -5 -13 0z" /> - <glyph glyph-name="uniF2B9" unicode="" horiz-adv-x="1664" -d="M1201 298q0 57 -5.5 107t-21 100.5t-39.5 86t-64 58t-91 22.5q-6 -4 -33.5 -20.5t-42.5 -24.5t-40.5 -20t-49 -17t-46.5 -5t-46.5 5t-49 17t-40.5 20t-42.5 24.5t-33.5 20.5q-51 0 -91 -22.5t-64 -58t-39.5 -86t-21 -100.5t-5.5 -107q0 -73 42 -121.5t103 -48.5h576 -q61 0 103 48.5t42 121.5zM1028 892q0 108 -76.5 184t-183.5 76t-183.5 -76t-76.5 -184q0 -107 76.5 -183t183.5 -76t183.5 76t76.5 183zM1664 352v-192q0 -14 -9 -23t-23 -9h-96v-224q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v1472q0 66 47 113t113 47h1216 -q66 0 113 -47t47 -113v-224h96q14 0 23 -9t9 -23v-192q0 -14 -9 -23t-23 -9h-96v-128h96q14 0 23 -9t9 -23v-192q0 -14 -9 -23t-23 -9h-96v-128h96q14 0 23 -9t9 -23z" /> - <glyph glyph-name="uniF2BA" unicode="" horiz-adv-x="1664" -d="M1028 892q0 -107 -76.5 -183t-183.5 -76t-183.5 76t-76.5 183q0 108 76.5 184t183.5 76t183.5 -76t76.5 -184zM980 672q46 0 82.5 -17t60 -47.5t39.5 -67t24 -81t11.5 -82.5t3.5 -79q0 -67 -39.5 -118.5t-105.5 -51.5h-576q-66 0 -105.5 51.5t-39.5 118.5q0 48 4.5 93.5 -t18.5 98.5t36.5 91.5t63 64.5t93.5 26h5q7 -4 32 -19.5t35.5 -21t33 -17t37 -16t35 -9t39.5 -4.5t39.5 4.5t35 9t37 16t33 17t35.5 21t32 19.5zM1664 928q0 -13 -9.5 -22.5t-22.5 -9.5h-96v-128h96q13 0 22.5 -9.5t9.5 -22.5v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-96v-128h96 -q13 0 22.5 -9.5t9.5 -22.5v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-96v-224q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v1472q0 66 47 113t113 47h1216q66 0 113 -47t47 -113v-224h96q13 0 22.5 -9.5t9.5 -22.5v-192zM1408 -96v1472q0 13 -9.5 22.5t-22.5 9.5h-1216 -q-13 0 -22.5 -9.5t-9.5 -22.5v-1472q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5z" /> - <glyph glyph-name="uniF2BB" unicode="" horiz-adv-x="2048" -d="M1024 405q0 64 -9 117.5t-29.5 103t-60.5 78t-97 28.5q-6 -4 -30 -18t-37.5 -21.5t-35.5 -17.5t-43 -14.5t-42 -4.5t-42 4.5t-43 14.5t-35.5 17.5t-37.5 21.5t-30 18q-57 0 -97 -28.5t-60.5 -78t-29.5 -103t-9 -117.5t37 -106.5t91 -42.5h512q54 0 91 42.5t37 106.5z -M867 925q0 94 -66.5 160.5t-160.5 66.5t-160.5 -66.5t-66.5 -160.5t66.5 -160.5t160.5 -66.5t160.5 66.5t66.5 160.5zM1792 416v64q0 14 -9 23t-23 9h-576q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h576q14 0 23 9t9 23zM1792 676v56q0 15 -10.5 25.5t-25.5 10.5h-568 -q-15 0 -25.5 -10.5t-10.5 -25.5v-56q0 -15 10.5 -25.5t25.5 -10.5h568q15 0 25.5 10.5t10.5 25.5zM1792 928v64q0 14 -9 23t-23 9h-576q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h576q14 0 23 9t9 23zM2048 1248v-1216q0 -66 -47 -113t-113 -47h-352v96q0 14 -9 23t-23 9 -h-64q-14 0 -23 -9t-9 -23v-96h-768v96q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-96h-352q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1728q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2BC" unicode="" horiz-adv-x="2048" -d="M1024 405q0 -64 -37 -106.5t-91 -42.5h-512q-54 0 -91 42.5t-37 106.5t9 117.5t29.5 103t60.5 78t97 28.5q6 -4 30 -18t37.5 -21.5t35.5 -17.5t43 -14.5t42 -4.5t42 4.5t43 14.5t35.5 17.5t37.5 21.5t30 18q57 0 97 -28.5t60.5 -78t29.5 -103t9 -117.5zM867 925 -q0 -94 -66.5 -160.5t-160.5 -66.5t-160.5 66.5t-66.5 160.5t66.5 160.5t160.5 66.5t160.5 -66.5t66.5 -160.5zM1792 480v-64q0 -14 -9 -23t-23 -9h-576q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h576q14 0 23 -9t9 -23zM1792 732v-56q0 -15 -10.5 -25.5t-25.5 -10.5h-568 -q-15 0 -25.5 10.5t-10.5 25.5v56q0 15 10.5 25.5t25.5 10.5h568q15 0 25.5 -10.5t10.5 -25.5zM1792 992v-64q0 -14 -9 -23t-23 -9h-576q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h576q14 0 23 -9t9 -23zM1920 32v1216q0 13 -9.5 22.5t-22.5 9.5h-1728q-13 0 -22.5 -9.5 -t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h352v96q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-96h768v96q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-96h352q13 0 22.5 9.5t9.5 22.5zM2048 1248v-1216q0 -66 -47 -113t-113 -47h-1728q-66 0 -113 47t-47 113v1216q0 66 47 113 -t113 47h1728q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2BD" unicode="" horiz-adv-x="1792" -d="M1523 197q-22 155 -87.5 257.5t-184.5 118.5q-67 -74 -159.5 -115.5t-195.5 -41.5t-195.5 41.5t-159.5 115.5q-119 -16 -184.5 -118.5t-87.5 -257.5q106 -150 271 -237.5t356 -87.5t356 87.5t271 237.5zM1280 896q0 159 -112.5 271.5t-271.5 112.5t-271.5 -112.5 -t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM1792 640q0 -182 -71 -347.5t-190.5 -286t-285.5 -191.5t-349 -71q-182 0 -348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="uniF2BE" unicode="" horiz-adv-x="1792" -d="M896 1536q182 0 348 -71t286 -191t191 -286t71 -348q0 -181 -70.5 -347t-190.5 -286t-286 -191.5t-349 -71.5t-349 71t-285.5 191.5t-190.5 286t-71 347.5t71 348t191 286t286 191t348 71zM1515 185q149 205 149 455q0 156 -61 298t-164 245t-245 164t-298 61t-298 -61 -t-245 -164t-164 -245t-61 -298q0 -250 149 -455q66 327 306 327q131 -128 313 -128t313 128q240 0 306 -327zM1280 832q0 159 -112.5 271.5t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5z" /> - <glyph glyph-name="uniF2C0" unicode="" -d="M1201 752q47 -14 89.5 -38t89 -73t79.5 -115.5t55 -172t22 -236.5q0 -154 -100 -263.5t-241 -109.5h-854q-141 0 -241 109.5t-100 263.5q0 131 22 236.5t55 172t79.5 115.5t89 73t89.5 38q-79 125 -79 272q0 104 40.5 198.5t109.5 163.5t163.5 109.5t198.5 40.5 -t198.5 -40.5t163.5 -109.5t109.5 -163.5t40.5 -198.5q0 -147 -79 -272zM768 1408q-159 0 -271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5t-112.5 271.5t-271.5 112.5zM1195 -128q88 0 150.5 71.5t62.5 173.5q0 239 -78.5 377t-225.5 145 -q-145 -127 -336 -127t-336 127q-147 -7 -225.5 -145t-78.5 -377q0 -102 62.5 -173.5t150.5 -71.5h854z" /> - <glyph glyph-name="uniF2C1" unicode="" horiz-adv-x="1280" -d="M1024 278q0 -64 -37 -107t-91 -43h-512q-54 0 -91 43t-37 107t9 118t29.5 104t61 78.5t96.5 28.5q80 -75 188 -75t188 75q56 0 96.5 -28.5t61 -78.5t29.5 -104t9 -118zM870 797q0 -94 -67.5 -160.5t-162.5 -66.5t-162.5 66.5t-67.5 160.5t67.5 160.5t162.5 66.5 -t162.5 -66.5t67.5 -160.5zM1152 -96v1376h-1024v-1376q0 -13 9.5 -22.5t22.5 -9.5h960q13 0 22.5 9.5t9.5 22.5zM1280 1376v-1472q0 -66 -47 -113t-113 -47h-960q-66 0 -113 47t-47 113v1472q0 66 47 113t113 47h352v-96q0 -14 9 -23t23 -9h192q14 0 23 9t9 23v96h352 -q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2C2" unicode="" horiz-adv-x="2048" -d="M896 324q0 54 -7.5 100.5t-24.5 90t-51 68.5t-81 25q-64 -64 -156 -64t-156 64q-47 0 -81 -25t-51 -68.5t-24.5 -90t-7.5 -100.5q0 -55 31.5 -93.5t75.5 -38.5h426q44 0 75.5 38.5t31.5 93.5zM768 768q0 80 -56 136t-136 56t-136 -56t-56 -136t56 -136t136 -56t136 56 -t56 136zM1792 288v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23zM1408 544v64q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1792 544v64q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23 -v-64q0 -14 9 -23t23 -9h192q14 0 23 9t9 23zM1792 800v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23zM128 1152h1792v96q0 14 -9 23t-23 9h-1728q-14 0 -23 -9t-9 -23v-96zM2048 1248v-1216q0 -66 -47 -113t-113 -47h-1728 -q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1728q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2C3" unicode="" horiz-adv-x="2048" -d="M896 324q0 -55 -31.5 -93.5t-75.5 -38.5h-426q-44 0 -75.5 38.5t-31.5 93.5q0 54 7.5 100.5t24.5 90t51 68.5t81 25q64 -64 156 -64t156 64q47 0 81 -25t51 -68.5t24.5 -90t7.5 -100.5zM768 768q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136z -M1792 352v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704q14 0 23 -9t9 -23zM1408 608v-64q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h320q14 0 23 -9t9 -23zM1792 608v-64q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v64 -q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 864v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704q14 0 23 -9t9 -23zM1920 32v1120h-1792v-1120q0 -13 9.5 -22.5t22.5 -9.5h1728q13 0 22.5 9.5t9.5 22.5zM2048 1248v-1216q0 -66 -47 -113t-113 -47 -h-1728q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1728q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2C4" unicode="" horiz-adv-x="1792" -d="M1255 749q0 318 -105 474.5t-330 156.5q-222 0 -326 -157t-104 -474q0 -316 104 -471.5t326 -155.5q74 0 131 17q-22 43 -39 73t-44 65t-53.5 56.5t-63 36t-77.5 14.5q-46 0 -79 -16l-49 97q105 91 276 91q132 0 215.5 -54t150.5 -155q67 149 67 402zM1645 117h117 -q3 -27 -2 -67t-26.5 -95t-58 -100.5t-107 -78t-162.5 -32.5q-71 0 -130.5 19t-105.5 56t-79 78t-66 96q-97 -27 -205 -27q-150 0 -292.5 58t-253 158.5t-178 249t-67.5 317.5q0 170 67.5 319.5t178.5 250.5t253.5 159t291.5 58q121 0 238.5 -36t217 -106t176 -164.5 -t119.5 -219t43 -261.5q0 -190 -80.5 -347.5t-218.5 -264.5q47 -70 93.5 -106.5t104.5 -36.5q61 0 94 37.5t38 85.5z" /> - <glyph glyph-name="uniF2C5" unicode="" horiz-adv-x="2304" -d="M453 -101q0 -21 -16 -37.5t-37 -16.5q-1 0 -13 3q-63 15 -162 140q-225 284 -225 676q0 341 213 614q39 51 95 103.5t94 52.5q19 0 35 -13.5t16 -32.5q0 -27 -63 -90q-98 -102 -147 -184q-119 -199 -119 -449q0 -281 123 -491q50 -85 136 -173q2 -3 14.5 -16t19.5 -21 -t17 -20.5t14.5 -23.5t4.5 -21zM1796 33q0 -29 -17.5 -48.5t-46.5 -19.5h-1081q-26 0 -45 19t-19 45q0 29 17.5 48.5t46.5 19.5h1081q26 0 45 -19t19 -45zM1581 644q0 -134 -67 -233q-25 -38 -69.5 -78.5t-83.5 -60.5q-16 -10 -27 -10q-7 0 -15 6t-8 12q0 9 19 30t42 46 -t42 67.5t19 88.5q0 76 -35 130q-29 42 -46 42q-3 0 -3 -5q0 -12 7.5 -35.5t7.5 -36.5q0 -22 -21.5 -35t-44.5 -13q-66 0 -66 76q0 15 1.5 44t1.5 44q0 25 -10 46q-13 25 -42 53.5t-51 28.5q-5 0 -7 -0.5t-3.5 -2.5t-1.5 -6q0 -2 16 -26t16 -54q0 -37 -19 -68t-46 -54 -t-53.5 -46t-45.5 -54t-19 -68q0 -98 42 -160q29 -43 79 -63q16 -5 17 -10q1 -2 1 -5q0 -16 -18 -16q-6 0 -33 11q-119 43 -195 139.5t-76 218.5q0 55 24.5 115.5t60 115t70.5 108.5t59.5 113.5t24.5 111.5q0 53 -25 94q-29 48 -56 64q-19 9 -19 21q0 20 41 20q50 0 110 -29 -q41 -19 71 -44.5t49.5 -51t33.5 -62.5t22 -69t16 -80q0 -1 3 -17.5t4.5 -25t5.5 -25t9 -27t11 -21.5t14.5 -16.5t18.5 -5.5q23 0 37 14t14 37q0 25 -20 67t-20 52t10 10q27 0 93 -70q72 -76 102.5 -156t30.5 -186zM2304 615q0 -274 -138 -503q-19 -32 -48 -72t-68 -86.5 -t-81 -77t-74 -30.5q-16 0 -31 15.5t-15 31.5q0 15 29 50.5t68.5 77t48.5 52.5q183 230 183 531q0 131 -20.5 235t-72.5 211q-58 119 -163 228q-2 3 -13 13.5t-16.5 16.5t-15 17.5t-15 20t-9.5 18.5t-4 19q0 19 16 35.5t35 16.5q70 0 196 -169q98 -131 146 -273t60 -314 -q2 -42 2 -64z" /> - <glyph glyph-name="uniF2C6" unicode="" horiz-adv-x="1792" -d="M1189 229l147 693q9 44 -10.5 63t-51.5 7l-864 -333q-29 -11 -39.5 -25t-2.5 -26.5t32 -19.5l221 -69l513 323q21 14 32 6q7 -5 -4 -15l-415 -375v0v0l-16 -228q23 0 45 22l108 104l224 -165q64 -36 81 38zM1792 640q0 -182 -71 -348t-191 -286t-286 -191t-348 -71 -t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="uniF2C7" unicode="" horiz-adv-x="1024" -d="M640 192q0 -80 -56 -136t-136 -56t-136 56t-56 136q0 60 35 110t93 71v907h128v-907q58 -21 93 -71t35 -110zM768 192q0 77 -34 144t-94 112v768q0 80 -56 136t-136 56t-136 -56t-56 -136v-768q-60 -45 -94 -112t-34 -144q0 -133 93.5 -226.5t226.5 -93.5t226.5 93.5 -t93.5 226.5zM896 192q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 182 128 313v711q0 133 93.5 226.5t226.5 93.5t226.5 -93.5t93.5 -226.5v-711q128 -131 128 -313zM1024 768v-128h-192v128h192zM1024 1024v-128h-192v128h192zM1024 1280v-128h-192 -v128h192z" /> - <glyph glyph-name="uniF2C8" unicode="" horiz-adv-x="1024" -d="M640 192q0 -80 -56 -136t-136 -56t-136 56t-56 136q0 60 35 110t93 71v651h128v-651q58 -21 93 -71t35 -110zM768 192q0 77 -34 144t-94 112v768q0 80 -56 136t-136 56t-136 -56t-56 -136v-768q-60 -45 -94 -112t-34 -144q0 -133 93.5 -226.5t226.5 -93.5t226.5 93.5 -t93.5 226.5zM896 192q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 182 128 313v711q0 133 93.5 226.5t226.5 93.5t226.5 -93.5t93.5 -226.5v-711q128 -131 128 -313zM1024 768v-128h-192v128h192zM1024 1024v-128h-192v128h192zM1024 1280v-128h-192 -v128h192z" /> - <glyph glyph-name="uniF2C9" unicode="" horiz-adv-x="1024" -d="M640 192q0 -80 -56 -136t-136 -56t-136 56t-56 136q0 60 35 110t93 71v395h128v-395q58 -21 93 -71t35 -110zM768 192q0 77 -34 144t-94 112v768q0 80 -56 136t-136 56t-136 -56t-56 -136v-768q-60 -45 -94 -112t-34 -144q0 -133 93.5 -226.5t226.5 -93.5t226.5 93.5 -t93.5 226.5zM896 192q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 182 128 313v711q0 133 93.5 226.5t226.5 93.5t226.5 -93.5t93.5 -226.5v-711q128 -131 128 -313zM1024 768v-128h-192v128h192zM1024 1024v-128h-192v128h192zM1024 1280v-128h-192 -v128h192z" /> - <glyph glyph-name="uniF2CA" unicode="" horiz-adv-x="1024" -d="M640 192q0 -80 -56 -136t-136 -56t-136 56t-56 136q0 60 35 110t93 71v139h128v-139q58 -21 93 -71t35 -110zM768 192q0 77 -34 144t-94 112v768q0 80 -56 136t-136 56t-136 -56t-56 -136v-768q-60 -45 -94 -112t-34 -144q0 -133 93.5 -226.5t226.5 -93.5t226.5 93.5 -t93.5 226.5zM896 192q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 182 128 313v711q0 133 93.5 226.5t226.5 93.5t226.5 -93.5t93.5 -226.5v-711q128 -131 128 -313zM1024 768v-128h-192v128h192zM1024 1024v-128h-192v128h192zM1024 1280v-128h-192 -v128h192z" /> - <glyph glyph-name="uniF2CB" unicode="" horiz-adv-x="1024" -d="M640 192q0 -80 -56 -136t-136 -56t-136 56t-56 136q0 79 56 135.5t136 56.5t136 -56.5t56 -135.5zM768 192q0 77 -34 144t-94 112v768q0 80 -56 136t-136 56t-136 -56t-56 -136v-768q-60 -45 -94 -112t-34 -144q0 -133 93.5 -226.5t226.5 -93.5t226.5 93.5t93.5 226.5z -M896 192q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 182 128 313v711q0 133 93.5 226.5t226.5 93.5t226.5 -93.5t93.5 -226.5v-711q128 -131 128 -313zM1024 768v-128h-192v128h192zM1024 1024v-128h-192v128h192zM1024 1280v-128h-192v128h192z" /> - <glyph glyph-name="uniF2CC" unicode="" horiz-adv-x="1920" -d="M1433 1287q10 -10 10 -23t-10 -23l-626 -626q-10 -10 -23 -10t-23 10l-82 82q-10 10 -10 23t10 23l44 44q-72 91 -81.5 207t46.5 215q-74 71 -176 71q-106 0 -181 -75t-75 -181v-1280h-256v1280q0 104 40.5 198.5t109.5 163.5t163.5 109.5t198.5 40.5q106 0 201 -41 -t166 -115q94 39 197 24.5t185 -79.5l44 44q10 10 23 10t23 -10zM1344 1024q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1600 896q-26 0 -45 19t-19 45t19 45t45 19t45 -19t19 -45t-19 -45t-45 -19zM1856 1024q26 0 45 -19t19 -45t-19 -45t-45 -19 -t-45 19t-19 45t19 45t45 19zM1216 896q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1408 832q0 26 19 45t45 19t45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45zM1728 896q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1088 768 -q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1344 640q-26 0 -45 19t-19 45t19 45t45 19t45 -19t19 -45t-19 -45t-45 -19zM1600 768q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1216 512q-26 0 -45 19t-19 45t19 45t45 19t45 -19 -t19 -45t-19 -45t-45 -19zM1472 640q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1088 512q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1344 512q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1216 384 -q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1088 256q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19z" /> - <glyph glyph-name="uniF2CD" unicode="" horiz-adv-x="1792" -d="M1664 448v-192q0 -169 -128 -286v-194q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v118q-63 -22 -128 -22h-768q-65 0 -128 22v-110q0 -17 -9.5 -28.5t-22.5 -11.5h-64q-13 0 -22.5 11.5t-9.5 28.5v186q-128 117 -128 286v192h1536zM704 864q0 -14 -9 -23t-23 -9t-23 9 -t-9 23t9 23t23 9t23 -9t9 -23zM768 928q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM704 992q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM832 992q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM768 1056q0 -14 -9 -23t-23 -9t-23 9 -t-9 23t9 23t23 9t23 -9t9 -23zM704 1120q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM1792 608v-64q0 -14 -9 -23t-23 -9h-1728q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h96v640q0 106 75 181t181 75q108 0 184 -78q46 19 98 12t93 -39l22 22q11 11 22 0l42 -42 -q11 -11 0 -22l-314 -314q-11 -11 -22 0l-42 42q-11 11 0 22l22 22q-36 46 -40.5 104t23.5 108q-37 35 -88 35q-53 0 -90.5 -37.5t-37.5 -90.5v-640h1504q14 0 23 -9t9 -23zM896 1056q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM832 1120q0 -14 -9 -23t-23 -9 -t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM768 1184q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM960 1120q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM896 1184q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM832 1248q0 -14 -9 -23 -t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM1024 1184q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM960 1248q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23zM1088 1248q0 -14 -9 -23t-23 -9t-23 9t-9 23t9 23t23 9t23 -9t9 -23z" /> - <glyph glyph-name="uniF2CE" unicode="" -d="M994 344q0 -86 -17 -197q-31 -215 -55 -313q-22 -90 -152 -90t-152 90q-24 98 -55 313q-17 110 -17 197q0 168 224 168t224 -168zM1536 768q0 -240 -134 -434t-350 -280q-8 -3 -15 3t-6 15q7 48 10 66q4 32 6 47q1 9 9 12q159 81 255.5 234t96.5 337q0 180 -91 330.5 -t-247 234.5t-337 74q-124 -7 -237 -61t-193.5 -140.5t-128 -202t-46.5 -240.5q1 -184 99 -336.5t257 -231.5q7 -3 9 -12q3 -21 6 -45q1 -9 5 -32.5t6 -35.5q1 -9 -6.5 -15t-15.5 -2q-148 58 -261 169.5t-173.5 264t-52.5 319.5q7 143 66 273.5t154.5 227t225 157.5t272.5 70 -q164 10 315.5 -46.5t261 -160.5t175 -250.5t65.5 -308.5zM994 800q0 -93 -65.5 -158.5t-158.5 -65.5t-158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5t158.5 -65.5t65.5 -158.5zM1282 768q0 -122 -53.5 -228.5t-146.5 -177.5q-8 -6 -16 -2t-10 14q-6 52 -29 92q-7 10 3 20 -q58 54 91 127t33 155q0 111 -58.5 204t-157.5 141.5t-212 36.5q-133 -15 -229 -113t-109 -231q-10 -92 23.5 -176t98.5 -144q10 -10 3 -20q-24 -41 -29 -93q-2 -9 -10 -13t-16 2q-95 74 -148.5 183t-51.5 234q3 131 69 244t177 181.5t241 74.5q144 7 268 -60t196.5 -187.5 -t72.5 -263.5z" /> - <glyph glyph-name="uniF2D0" unicode="" horiz-adv-x="1792" -d="M256 128h1280v768h-1280v-768zM1792 1248v-1216q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2D1" unicode="" horiz-adv-x="1792" -d="M1792 224v-192q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2D2" unicode="" horiz-adv-x="2048" -d="M256 0h768v512h-768v-512zM1280 512h512v768h-768v-256h96q66 0 113 -47t47 -113v-352zM2048 1376v-960q0 -66 -47 -113t-113 -47h-608v-352q0 -66 -47 -113t-113 -47h-960q-66 0 -113 47t-47 113v960q0 66 47 113t113 47h608v352q0 66 47 113t113 47h960q66 0 113 -47 -t47 -113z" /> - <glyph glyph-name="uniF2D3" unicode="" horiz-adv-x="1792" -d="M1175 215l146 146q10 10 10 23t-10 23l-233 233l233 233q10 10 10 23t-10 23l-146 146q-10 10 -23 10t-23 -10l-233 -233l-233 233q-10 10 -23 10t-23 -10l-146 -146q-10 -10 -10 -23t10 -23l233 -233l-233 -233q-10 -10 -10 -23t10 -23l146 -146q10 -10 23 -10t23 10 -l233 233l233 -233q10 -10 23 -10t23 10zM1792 1248v-1216q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2D4" unicode="" horiz-adv-x="1792" -d="M1257 425l-146 -146q-10 -10 -23 -10t-23 10l-169 169l-169 -169q-10 -10 -23 -10t-23 10l-146 146q-10 10 -10 23t10 23l169 169l-169 169q-10 10 -10 23t10 23l146 146q10 10 23 10t23 -10l169 -169l169 169q10 10 23 10t23 -10l146 -146q10 -10 10 -23t-10 -23 -l-169 -169l169 -169q10 -10 10 -23t-10 -23zM256 128h1280v1024h-1280v-1024zM1792 1248v-1216q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2D5" unicode="" horiz-adv-x="1792" -d="M1070 358l306 564h-654l-306 -564h654zM1792 640q0 -182 -71 -348t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71t348 -71t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="uniF2D6" unicode="" horiz-adv-x="1794" -d="M1291 1060q-15 17 -35 8.5t-26 -28.5t5 -38q14 -17 40 -14.5t34 20.5t-18 52zM895 814q-8 -8 -19.5 -8t-18.5 8q-8 8 -8 19t8 18q7 8 18.5 8t19.5 -8q7 -7 7 -18t-7 -19zM1060 740l-35 -35q-12 -13 -29.5 -13t-30.5 13l-38 38q-12 13 -12 30t12 30l35 35q12 12 29.5 12 -t30.5 -12l38 -39q12 -12 12 -29.5t-12 -29.5zM951 870q-7 -8 -18.5 -8t-19.5 8q-7 8 -7 19t7 19q8 8 19 8t19 -8t8 -19t-8 -19zM1354 968q-34 -64 -107.5 -85.5t-127.5 16.5q-38 28 -61 66.5t-21 87.5t39 92t75.5 53t70.5 -5t70 -51q2 -2 13 -12.5t14.5 -13.5t13 -13.5 -t12.5 -15.5t10 -15.5t8.5 -18t4 -18.5t1 -21t-5 -22t-9.5 -24zM1555 486q3 20 -8.5 34.5t-27.5 21.5t-33 17t-23 20q-40 71 -84 98.5t-113 11.5q19 13 40 18.5t33 4.5l12 -1q2 45 -34 90q6 20 6.5 40.5t-2.5 30.5l-3 10q43 24 71 65t34 91q10 84 -43 150.5t-137 76.5 -q-60 7 -114 -18.5t-82 -74.5q-30 -51 -33.5 -101t14.5 -87t43.5 -64t56.5 -42q-45 4 -88 36t-57 88q-28 108 32 222q-16 21 -29 32q-50 0 -89 -19q19 24 42 37t36 14l13 1q0 50 -13 78q-10 21 -32.5 28.5t-47 -3.5t-37.5 -40q2 4 4 7q-7 -28 -6.5 -75.5t19 -117t48.5 -122.5 -q-25 -14 -47 -36q-35 -16 -85.5 -70.5t-84.5 -101.5l-33 -46q-90 -34 -181 -125.5t-75 -162.5q1 -16 11 -27q-15 -12 -30 -30q-21 -25 -21 -54t21.5 -40t63.5 6q41 19 77 49.5t55 60.5q-2 2 -6.5 5t-20.5 7.5t-33 3.5q23 5 51 12.5t40 10t27.5 6t26 4t23.5 0.5q14 -7 22 34 -q7 37 7 90q0 102 -40 150q106 -103 101 -219q-1 -29 -15 -50t-27 -27l-13 -6q-4 -7 -19 -32t-26 -45.5t-26.5 -52t-25 -61t-17 -63t-6.5 -66.5t10 -63q-35 54 -37 80q-22 -24 -34.5 -39t-33.5 -42t-30.5 -46t-16.5 -41t-0.5 -38t25.5 -27q45 -25 144 64t190.5 221.5 -t122.5 228.5q86 52 145 115.5t86 119.5q47 -93 154 -178q104 -83 167 -80q39 2 46 43zM1794 640q0 -182 -71 -348t-191 -286t-286.5 -191t-348.5 -71t-348.5 71t-286.5 191t-191 286t-71 348t71 348t191 286t286.5 191t348.5 71t348.5 -71t286.5 -191t191 -286t71 -348z" /> - <glyph glyph-name="uniF2D7" unicode="" -d="M518 1353v-655q103 -1 191.5 1.5t125.5 5.5l37 3q68 2 90.5 24.5t39.5 94.5l33 142h103l-14 -322l7 -319h-103l-29 127q-15 68 -45 93t-84 26q-87 8 -352 8v-556q0 -78 43.5 -115.5t133.5 -37.5h357q35 0 59.5 2t55 7.5t54 18t48.5 32t46 50.5t39 73l93 216h89 -q-6 -37 -31.5 -252t-30.5 -276q-146 5 -263.5 8t-162.5 4h-44h-628l-376 -12v102l127 25q67 13 91.5 37t25.5 79l8 643q3 402 -8 645q-2 61 -25.5 84t-91.5 36l-127 24v102l376 -12h702q139 0 374 27q-6 -68 -14 -194.5t-12 -219.5l-5 -92h-93l-32 124q-31 121 -74 179.5 -t-113 58.5h-548q-28 0 -35.5 -8.5t-7.5 -30.5z" /> - <glyph glyph-name="uniF2D8" unicode="" -d="M922 739v-182q0 -4 0.5 -15t0 -15l-1.5 -12t-3.5 -11.5t-6.5 -7.5t-11 -5.5t-16 -1.5v309q9 0 16 -1t11 -5t6.5 -5.5t3.5 -9.5t1 -10.5v-13.5v-14zM1238 643v-121q0 -1 0.5 -12.5t0 -15.5t-2.5 -11.5t-7.5 -10.5t-13.5 -3q-9 0 -14 9q-4 10 -4 165v7v8.5v9t1.5 8.5l3.5 7 -t5 5.5t8 1.5q6 0 10 -1.5t6.5 -4.5t4 -6t2 -8.5t0.5 -8v-9.5v-9zM180 407h122v472h-122v-472zM614 407h106v472h-159l-28 -221q-20 148 -32 221h-158v-472h107v312l45 -312h76l43 319v-319zM1039 712q0 67 -5 90q-3 16 -11 28.5t-17 20.5t-25 14t-26.5 8.5t-31 4t-29 1.5 -h-29.5h-12h-91v-472h56q169 -1 197 24.5t25 180.5q-1 62 -1 100zM1356 515v133q0 29 -2 45t-9.5 33.5t-24.5 25t-46 7.5q-46 0 -77 -34v154h-117v-472h110l7 30q30 -36 77 -36q50 0 66 30.5t16 83.5zM1536 1248v-1216q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113 -v1216q0 66 47 113t113 47h1216q66 0 113 -47t47 -113z" /> - <glyph glyph-name="uniF2D9" unicode="" horiz-adv-x="2176" -d="M1143 -197q-6 1 -11 4q-13 8 -36 23t-86 65t-116.5 104.5t-112 140t-89.5 172.5q-17 3 -175 37q66 -213 235 -362t391 -184zM502 409l168 -28q-25 76 -41 167.5t-19 145.5l-4 53q-84 -82 -121 -224q5 -65 17 -114zM612 1018q-43 -64 -77 -148q44 46 74 68zM2049 584 -q0 161 -62 307t-167.5 252t-250.5 168.5t-304 62.5q-147 0 -281 -52.5t-240 -148.5q-30 -58 -45 -160q60 51 143 83.5t158.5 43t143 13.5t108.5 -1l40 -3q33 -1 53 -15.5t24.5 -33t6.5 -37t-1 -28.5q-126 11 -227.5 0.5t-183 -43.5t-142.5 -71.5t-131 -98.5 -q4 -36 11.5 -92.5t35.5 -178t62 -179.5q123 -6 247.5 14.5t214.5 53.5t162.5 67t109.5 59l37 24q22 16 39.5 20.5t30.5 -5t17 -34.5q14 -97 -39 -121q-208 -97 -467 -134q-135 -20 -317 -16q41 -96 110 -176.5t137 -127t130.5 -79t101.5 -43.5l39 -12q143 -23 263 15 -q195 99 314 289t119 418zM2123 621q-14 -135 -40 -212q-70 -208 -181.5 -346.5t-318.5 -253.5q-48 -33 -82 -44q-72 -26 -163 -16q-36 -3 -73 -3q-283 0 -504.5 173t-295.5 442q-1 0 -4 0.5t-5 0.5q-6 -50 2.5 -112.5t26 -115t36 -98t31.5 -71.5l14 -26q8 -12 54 -82 -q-71 38 -124.5 106.5t-78.5 140t-39.5 137t-17.5 107.5l-2 42q-5 2 -33.5 12.5t-48.5 18t-53 20.5t-57.5 25t-50 25.5t-42.5 27t-25 25.5q19 -10 50.5 -25.5t113 -45.5t145.5 -38l2 32q11 149 94 290q41 202 176 365q28 115 81 214q15 28 32 45t49 32q158 74 303.5 104 -t302 11t306.5 -97q220 -115 333 -336t87 -474z" /> - <glyph glyph-name="uniF2DA" unicode="" horiz-adv-x="1792" -d="M1341 752q29 44 -6.5 129.5t-121.5 142.5q-58 39 -125.5 53.5t-118 4.5t-68.5 -37q-12 -23 -4.5 -28t42.5 -10q23 -3 38.5 -5t44.5 -9.5t56 -17.5q36 -13 67.5 -31.5t53 -37t40 -38.5t30.5 -38t22 -34.5t16.5 -28.5t12 -18.5t10.5 -6t11 9.5zM1704 178 -q-52 -127 -148.5 -220t-214.5 -141.5t-253 -60.5t-266 13.5t-251 91t-210 161.5t-141.5 235.5t-46.5 303.5q1 41 8.5 84.5t12.5 64t24 80.5t23 73q-51 -208 1 -397t173 -318t291 -206t346 -83t349 74.5t289 244.5q20 27 18 14q0 -4 -4 -14zM1465 627q0 -104 -40.5 -199 -t-108.5 -164t-162 -109.5t-198 -40.5t-198 40.5t-162 109.5t-108.5 164t-40.5 199t40.5 199t108.5 164t162 109.5t198 40.5t198 -40.5t162 -109.5t108.5 -164t40.5 -199zM1752 915q-65 147 -180.5 251t-253 153.5t-292 53.5t-301 -36.5t-275.5 -129t-220 -211.5t-131 -297 -t-10 -373q-49 161 -51.5 311.5t35.5 272.5t109 227t165.5 180.5t207 126t232 71t242.5 9t236 -54t216 -124.5t178 -197q33 -50 62 -121t31 -112zM1690 573q12 244 -136.5 416t-396.5 240q-8 0 -10 5t24 8q125 -4 230 -50t173 -120t116 -168.5t58.5 -199t-1 -208 -t-61.5 -197.5t-122.5 -167t-185 -117.5t-248.5 -46.5q108 30 201.5 80t174 123t129.5 176.5t55 225.5z" /> - <glyph glyph-name="uniF2DB" unicode="" -d="M192 256v-128h-112q-16 0 -16 16v16h-48q-16 0 -16 16v32q0 16 16 16h48v16q0 16 16 16h112zM192 512v-128h-112q-16 0 -16 16v16h-48q-16 0 -16 16v32q0 16 16 16h48v16q0 16 16 16h112zM192 768v-128h-112q-16 0 -16 16v16h-48q-16 0 -16 16v32q0 16 16 16h48v16 -q0 16 16 16h112zM192 1024v-128h-112q-16 0 -16 16v16h-48q-16 0 -16 16v32q0 16 16 16h48v16q0 16 16 16h112zM192 1280v-128h-112q-16 0 -16 16v16h-48q-16 0 -16 16v32q0 16 16 16h48v16q0 16 16 16h112zM1280 1440v-1472q0 -40 -28 -68t-68 -28h-832q-40 0 -68 28 -t-28 68v1472q0 40 28 68t68 28h832q40 0 68 -28t28 -68zM1536 208v-32q0 -16 -16 -16h-48v-16q0 -16 -16 -16h-112v128h112q16 0 16 -16v-16h48q16 0 16 -16zM1536 464v-32q0 -16 -16 -16h-48v-16q0 -16 -16 -16h-112v128h112q16 0 16 -16v-16h48q16 0 16 -16zM1536 720v-32 -q0 -16 -16 -16h-48v-16q0 -16 -16 -16h-112v128h112q16 0 16 -16v-16h48q16 0 16 -16zM1536 976v-32q0 -16 -16 -16h-48v-16q0 -16 -16 -16h-112v128h112q16 0 16 -16v-16h48q16 0 16 -16zM1536 1232v-32q0 -16 -16 -16h-48v-16q0 -16 -16 -16h-112v128h112q16 0 16 -16v-16 -h48q16 0 16 -16z" /> - <glyph glyph-name="uniF2DC" unicode="" horiz-adv-x="1664" -d="M1566 419l-167 -33l186 -107q23 -13 29.5 -38.5t-6.5 -48.5q-14 -23 -39 -29.5t-48 6.5l-186 106l55 -160q13 -38 -12 -63.5t-60.5 -20.5t-48.5 42l-102 300l-271 156v-313l208 -238q16 -18 17 -39t-11 -36.5t-28.5 -25t-37 -5.5t-36.5 22l-112 128v-214q0 -26 -19 -45 -t-45 -19t-45 19t-19 45v214l-112 -128q-16 -18 -36.5 -22t-37 5.5t-28.5 25t-11 36.5t17 39l208 238v313l-271 -156l-102 -300q-13 -37 -48.5 -42t-60.5 20.5t-12 63.5l55 160l-186 -106q-23 -13 -48 -6.5t-39 29.5q-13 23 -6.5 48.5t29.5 38.5l186 107l-167 33 -q-29 6 -42 29t-8.5 46.5t25.5 40t50 10.5l310 -62l271 157l-271 157l-310 -62q-4 -1 -13 -1q-27 0 -44 18t-19 40t11 43t40 26l167 33l-186 107q-23 13 -29.5 38.5t6.5 48.5t39 30t48 -7l186 -106l-55 160q-13 38 12 63.5t60.5 20.5t48.5 -42l102 -300l271 -156v313 -l-208 238q-16 18 -17 39t11 36.5t28.5 25t37 5.5t36.5 -22l112 -128v214q0 26 19 45t45 19t45 -19t19 -45v-214l112 128q16 18 36.5 22t37 -5.5t28.5 -25t11 -36.5t-17 -39l-208 -238v-313l271 156l102 300q13 37 48.5 42t60.5 -20.5t12 -63.5l-55 -160l186 106 -q23 13 48 6.5t39 -29.5q13 -23 6.5 -48.5t-29.5 -38.5l-186 -107l167 -33q27 -5 40 -26t11 -43t-19 -40t-44 -18q-9 0 -13 1l-310 62l-271 -157l271 -157l310 62q29 6 50 -10.5t25.5 -40t-8.5 -46.5t-42 -29z" /> - <glyph glyph-name="uniF2DD" unicode="" horiz-adv-x="1792" -d="M1473 607q7 118 -33 226.5t-113 189t-177 131t-221 57.5q-116 7 -225.5 -32t-192 -110.5t-135 -175t-59.5 -220.5q-7 -118 33 -226.5t113 -189t177.5 -131t221.5 -57.5q155 -9 293 59t224 195.5t94 283.5zM1792 1536l-349 -348q120 -117 180.5 -272t50.5 -321 -q-11 -183 -102 -339t-241 -255.5t-332 -124.5l-999 -132l347 347q-120 116 -180.5 271.5t-50.5 321.5q11 184 102 340t241.5 255.5t332.5 124.5q167 22 500 66t500 66z" /> - <glyph glyph-name="uniF2DE" unicode="" horiz-adv-x="1792" -d="M948 508l163 -329h-51l-175 350l-171 -350h-49l179 374l-78 33l21 49l240 -102l-21 -50zM563 1100l304 -130l-130 -304l-304 130zM907 915l240 -103l-103 -239l-239 102zM1188 765l191 -81l-82 -190l-190 81zM1680 640q0 159 -62 304t-167.5 250.5t-250.5 167.5t-304 62 -t-304 -62t-250.5 -167.5t-167.5 -250.5t-62 -304t62 -304t167.5 -250.5t250.5 -167.5t304 -62t304 62t250.5 167.5t167.5 250.5t62 304zM1792 640q0 -182 -71 -348t-191 -286t-286 -191t-348 -71t-348 71t-286 191t-191 286t-71 348t71 348t191 286t286 191t348 71t348 -71 -t286 -191t191 -286t71 -348z" /> - <glyph glyph-name="uniF2E0" unicode="" horiz-adv-x="1920" -d="M1334 302q-4 24 -27.5 34t-49.5 10.5t-48.5 12.5t-25.5 38q-5 47 33 139.5t75 181t32 127.5q-14 101 -117 103q-45 1 -75 -16l-3 -2l-5 -2.5t-4.5 -2t-5 -2t-5 -0.5t-6 1.5t-6 3.5t-6.5 5q-3 2 -9 8.5t-9 9t-8.5 7.5t-9.5 7.5t-9.5 5.5t-11 4.5t-11.5 2.5q-30 5 -48 -3 -t-45 -31q-1 -1 -9 -8.5t-12.5 -11t-15 -10t-16.5 -5.5t-17 3q-54 27 -84 40q-41 18 -94 -5t-76 -65q-16 -28 -41 -98.5t-43.5 -132.5t-40 -134t-21.5 -73q-22 -69 18.5 -119t110.5 -46q30 2 50.5 15t38.5 46q7 13 79 199.5t77 194.5q6 11 21.5 18t29.5 0q27 -15 21 -53 -q-2 -18 -51 -139.5t-50 -132.5q-6 -38 19.5 -56.5t60.5 -7t55 49.5q4 8 45.5 92t81.5 163.5t46 88.5q20 29 41 28q29 0 25 -38q-2 -16 -65.5 -147.5t-70.5 -159.5q-12 -53 13 -103t74 -74q17 -9 51 -15.5t71.5 -8t62.5 14t20 48.5zM383 86q3 -15 -5 -27.5t-23 -15.5 -q-14 -3 -26.5 5t-15.5 23q-3 14 5 27t22 16t27 -5t16 -23zM953 -177q12 -17 8.5 -37.5t-20.5 -32.5t-37.5 -8t-32.5 21q-11 17 -7.5 37.5t20.5 32.5t37.5 8t31.5 -21zM177 635q-18 -27 -49.5 -33t-57.5 13q-26 18 -32 50t12 58q18 27 49.5 33t57.5 -12q26 -19 32 -50.5 -t-12 -58.5zM1467 -42q19 -28 13 -61.5t-34 -52.5t-60.5 -13t-51.5 34t-13 61t33 53q28 19 60.5 13t52.5 -34zM1579 562q69 -113 42.5 -244.5t-134.5 -207.5q-90 -63 -199 -60q-20 -80 -84.5 -127t-143.5 -44.5t-140 57.5q-12 -9 -13 -10q-103 -71 -225 -48.5t-193 126.5 -q-50 73 -53 164q-83 14 -142.5 70.5t-80.5 128t-2 152t81 138.5q-36 60 -38 128t24.5 125t79.5 98.5t121 50.5q32 85 99 148t146.5 91.5t168 17t159.5 -66.5q72 21 140 17.5t128.5 -36t104.5 -80t67.5 -115t17.5 -140.5q52 -16 87 -57t45.5 -89t-5.5 -99.5t-58 -87.5z -M455 1222q14 -20 9.5 -44.5t-24.5 -38.5q-19 -14 -43.5 -9.5t-37.5 24.5q-14 20 -9.5 44.5t24.5 38.5q19 14 43.5 9.5t37.5 -24.5zM614 1503q4 -16 -5 -30.5t-26 -18.5t-31 5.5t-18 26.5q-3 17 6.5 31t25.5 18q17 4 31 -5.5t17 -26.5zM1800 555q4 -20 -6.5 -37t-30.5 -21 -q-19 -4 -36 6.5t-21 30.5t6.5 37t30.5 22q20 4 36.5 -7.5t20.5 -30.5zM1136 1448q16 -27 8.5 -58.5t-35.5 -47.5q-27 -16 -57.5 -8.5t-46.5 34.5q-16 28 -8.5 59t34.5 48t58 9t47 -36zM1882 792q4 -15 -4 -27.5t-23 -16.5q-15 -3 -27.5 5.5t-15.5 22.5q-3 15 5 28t23 16 -q14 3 26.5 -5t15.5 -23zM1691 1033q15 -22 10.5 -49t-26.5 -43q-22 -15 -49 -10t-42 27t-10 49t27 43t48.5 11t41.5 -28z" /> - <glyph glyph-name="uniF2E1" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2E2" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2E3" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2E4" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2E5" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2E6" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2E7" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="_698" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2E9" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2EA" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2EB" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2EC" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2ED" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="uniF2EE" unicode="" horiz-adv-x="1792" - /> - <glyph glyph-name="lessequal" unicode="" horiz-adv-x="1792" - /> - </font> -</defs></svg> diff --git a/site/fonts/fontawesome-webfont.ttf b/site/fonts/fontawesome-webfont.ttf deleted file mode 100644 index 35acda2fa1196aad98c2adf4378a7611dd713aa3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 165548 zcmd4434D~*)jxjkv&@#+*JQHIB(r2Agk&ZO5W=u;0Z~v85Ce*$fTDsRbs2>!AXP+E zv})s8XszXKwXa&S)7IKescosX*7l99R$G?_w7v?NC%^Bx&rC7|(E7f=|L^lpa-Zk9 z`?>d?d+s^so_oVMW6Z|VOlEVZPMtq{)pOIHX3~v25n48F@|3AkA5-983xDXec_W** zHg8HX#uvihecqa7Yb`$*a~)&Wy^KjmE?joS+JOO-B;B|Y@umw`Uv<H)|NZ!E#)6MA zsq5L5E0&$_-1gQ8#JvvJx)K2n+43aH^THolxn|wQDOVML8UCjjv-GXL^t@%#zaCt| znB!i?ikxefZCooiNM(p`NBYnu%hs&;>s>da><dw!a^u=duUPl(Tfh1MlUDVi&h?Di zrtn|x{ME*@&vkVCixn9Wz}TUdmzwkM0RU{~2J)JiByym5#tdu#L1u>d0W;5qQ!4Qz zJxL+bkEIe8*8}j>Q>BETG1+ht-^o+}utRA<*p2#Ix&jHe=hB??wf3sZuV5(_`d1DH zgI+ncCI1s*Tuw6@6DFOB@-mE3%l-{_4z<*f9!g8!<Nzo$=DrKcxk^p5OY&(bFU8Rt z;gl~Z%A@c!YZmEp@%ixloAN1Xkv~Q0WTyM+o65?Un}fkZ@K9VHCQ73qY%pE!oBStr zNKbWBp8Q+nrEoa<9xN}-Hi`D+(iX-WfD?e_9=JvMw)`9Klt`0iNtBfo_3CBYs2))l z>dcoz@f1eyoO9;V<o%)zh+8Ba(h`gUPQ1~VDavPpD=br#g?M`B!{xjmZwgcJ9*oDg zNSAx(VHIP+5SL~R=(BV4X=rQ&oQM;30S?4Tx$nU`MS99#=+ewb>5yN|*Pk0}XYPFk z!g(%@Qka**;2iW8;b{R|Dg0FbU_E9^hd3H%a#EV5;HVvgVS_k;c*=`1YN*`2lhZm3 zqOTF2Pfz8N%lA<(eJUSDWevumUJ;M<C?`d6<WSZKIGd7Yv&l(mHUe(YpRWUPG{zzi z!I6VSltbkXh9NEohn_Z<CtnWY2#yre+YmM)msbLu5|u%1%e}h+2ZLK~Kr_f8Jr{8Z zM|xX+e&Sn3ctn&%`3oGiAK_MtQ}9i36vlJk<VRd<J{<MS<O<6;**C$Na|!tde9KHB zc&Hw{+XwTcSruTbLbz}UuM#eu>ocT>zZ5W08%2JkP2szU{CP(((>LmzOmB>ZOpelu zIw>A5mu@gGU}>QA1RKFi-$*aQL_KL1GNuOxs0@)VEz<hv`#6Vi@+){#9VR%ckHRP~ zIhy-~?+9Cy52v^IPV4VHeK3B^YV&c0X#r=7%0=5myQsXuJo&O1b5d9?JVg1aZMnKd zeT8M8;ya#)IR}3TZx4Ov;1T@{ct#`4O?Bnc4VDjoC*q3>%g?77_AY_{e55-&2X`IC z!*9krPH>;hA+4QUe(ZB_4Z@L!DgUN;`X-m}3;G6(Mf9flyest6ciunvokm)?oZmzF z@?{e2C{v;^ys6AQy_IN=B99>#C*fPn3ra`%a_!FN6aIXi^rn1ymrrZ@gw3bA$$zqb zqOxiHDSsYDDkGmZpD$nT@HfSi%fmt6l*S0Iupll)-&7{*yFioy4w3x%G<fIxL6il* zAL;CPmlvcNOiwT)JWBNz#^=J%$itb7n^I6$h{E!IdLwNlpFaPlG`X_F2wR16YbXxk zL^t#{>VEpx@jWf@QO?itTs?#7)d3a-Ug&FLt_)FMnmOp5gGJy@z7B*(^RVW^e1dkQ zkMHw*dK%Ayu_({yrG6RifN!GjP=|nt${60CMrjDAK)0HZCYpnJB&8QF&0_TaoF9-S zu?&_mPAU0&@X=Qpc>I^~UdvKIk0usk``F{`3HAbeHC$CyQPtgN@2lwR?3>fKwC|F> zYx{2LyT9-8zVGxM?E7=y2YuRM`{9bijfXoA&pEvG@Fj<@J$%dI`wu^U__@Oe5C8e_ z2ZyyI_9GQXI*-gbvh>I$N3K0`%aQw!JbvW4BL|QC`N#+Vf_#9QLu~J`8d;ySFWi^v zo7>mjx3(|cx3jOOZ+~B=@8!PUzP`iku=8-}aMR(`;kk#q53fC(KD_gA&*A-tGlyS3 z+<Un12s`3D5<ZeTGWSULk+nzCM|K|RIkNvq=E%!Oda1^NzYZ)JSU50mVB)~2Kfm** z_y6>m)8@1~El#u3<TN@BPSvS6WvAr$m*XSHhmIqT!;U`3KOG-9-gms`c-Qd{$KM_A zINo;r&GDAwO~+pye{sCw=ym+r@h8XYjz2nHbNs>as^j;LR~)}{9CG~D_9MNw(aQga zKO~TeK}MY%7{tgG{veXj;r|am2GwFztR{2O|5v~?px`g+cB0=PQ}aFOx^-}vA95F5 zA7=4<%*Y5_FJ|j%P>qdnh_@iTs0Qv3Shg)-OV0=S+zU1vekc4cfZ>81?nWLD;PJf5 zm^TgA&zNr~$ZdkLfD=nH@)f_xSjk$*;M3uDgT;zqnj*X$`6@snD%LSpiMm2N;QAN~ z_kcBPVyrp@Qi?Q@UdCdRu{^&CvWYrt=QCD^e09&FD^N$nM_`>%e`5*`?~&bbh->n~ zJ<Qms-HbKwWUS?U#u5oQ_>(9*nTC4`EGNEOm%t%U8(?hP3%1b;hjQAV0Nc?8hxeG3 zaPKiTHp5uQTE@n~b#}l3uJMQ)kGfOHpF%kkn&43O#D#F5Fg6KwPr4VR9c4{M`YDK; z3jZ{uoAx?m(^2k>9gNLvXKdDEjCCQ+Y~-2K00%hd9AfOW{fx~8OmhL>=?S<V#?G+9 zt!8W<(#_k$*nGsFi82@TF}4upE;2E;I0d(Zu_YaFfaM&dTUrB$I?hEI-D??Jj{M6% z0&Aqey~o&!xo|IId3~O-3sL5UsPm#OxC~<#PsS2vg#%2h(_k@Ej9r5Kmm+@c7RD}n zn6b-I_Z1r%TjzvB-u1}yC6seDgL{^-YY=`7(p|fivFmm-mPXk1#c(?q+dQ4IEnYa3 z|7GO;@|#eYtY+*c)Uy?^Z_6-tGxG2F8Dn2T`dbipEAs3N!6ELp`Hbzd!Xa$e7C40M z>SyfsZaC!Gt-z(=`WU+-&Dfn0#_n3e*q()q-CYLpelpxsjC~b#-P^<1eJJmK#NGc1 zV_&<Mu^z-fFqg3h8{i&h?4fqJy^KA4K4XudoV`04`=*Jp$57T|?=ki*_`mfIV~?Y* zZx_SuXKep;xL(HoV<X&)jD05rw+`-SjD2?@+((Q(v6e9~KI}=9`#qHPy+e%s=M!*! zj6H>XPb2-)pD^|e^5@<6_cHeE7RC;w7<*1(><1_>^E_ievcm0P?8kubdDQj%vyA=3 z3HKCZFYIRQXH9UujQt#S{T$`}0_FTN4TrE7KVs}9q&bK>55B|Lul6(cGRpdO1Kd`| zeq(~e`?pp&g#Y$EXw}*o`yJwccQ0eFbi*Ov?^iSS>U6j#82bal{s6dMn-2#V{#Xo$ zI$lq~{fx0cA?=^g&OdKq?7tBAUym`?3z*+P_+QpC_SX>Hn~c4gX6!Ab|67K!w~_Ac z_ZWKz;eUUXv46n53-{h3#@>IKu@7En?4O7`qA>R1M~r=hy#Got_OTNVaQ-*)f3gq` zWqlf9>?rCwhC2Ie;GSYEYlZ8Edx9~|1c$Hz6P6|~v_elnBK`=R&nMuzUuN8VKI0ZA z+#be@iW#>ma1S$XYhc_CQta5uxC`H|9>(1-GVW=IdlO`OC*!^vIHdJ2gzINKkYT)d z3*#jl84q5~c<Fq`%MLMKo`S=>0(mMGIK+jJFO2k6NLvlqs#h}}L0klN#8)z2^A6*6 zU5q!Nj7Gdit%LiB@#bE}TbkhZGoIMXcoN~QNYfU9dezGK=;@4)al-X6K6WSL9b4dD zWqdqfOo0cRfI27sjPXfulka7G3er!7o3@tm>3GioJTpUZZ!$jX5aV4vjL$A+d`^n- zxp1e$e?~9k^CmMsKg9T%fbFbqIHX;GIu<72kY<qrx0Uh5=fffW614f8R~cW5xbEGI zFGF3+o@IPF;5-lc&VP*Y73&zkU?t-hzRvi{Cm6qI3F8-UWc-pXjIT{Geg(?8(#iOS zpE14>ZMzEPZ`#55myqXbyss&PdzkU-kng%ZaGx-qUd{ORDE9`W-<*I${1)W@@_xo| z#P?RjZA0Ge?Tp_{4)ER51-F;+Tjw*r6ZPHZW&C#J-;MVj3S2+qccSdOkoNAY8NUbR z-HUYhnc!Y!{C<Retr+ed#(SP;{6Q<@4<#6Xcq8MFpx#H*jDH<<>@9;sxqIIma{CrC z{*4;OzZrsik@3eKWBglt8Gju9$G0;6ZPfp5`1hya;Q!vUjQ{6qsNQ=S2c6;1ApV)% zjDJ4@_b}tnn&43HfiA|MBZsgbpsdVv#(xMHfA~D(KUU!0Wc>La#(y%O@fT{~-ede{ zR>pr0_Y2hXOT@kS3F8L=^RH0;%c~jx_4$nd=5@w@I~NXdzuUt2E2!)DYvKACfAu5A zUwe%4KcdXn;r@iOKr8s4QQm)bG5$uH@xLJ7o5hU3g}A?UF#a~+dV4S9??m7ZG5+_} zjQ<05{sZ6d0><|ea8JQ~#Q6It>z^jLhZ*lv;9g|>Fxqwm@O+4TAHKu*zfkVS4R9I8 z{~NIVcQ50g0KQKVb`<_&>lp7xn*Q?{2i@S=9gJ(JgXqP;%S_@4CSmVFk{g($tYngU z2omdDCYcd#!MC-SNwz*FIf|L&M40PMCV4uT<VAWP9QZ3K(8Q$Tb|#fXm=xT~q*8pB zKf$Dm`EV~XDRecH!YI4yF(yU2m=s0&*mNevk+$X|Ce`j>QXRtTUT0GMZYDM0-H5Up z-(yk}+^8)~YEHrRGpXe%CMDJ}DT(-2<Q*Mi(ir3)bBIY}t#Bx_qXzC(CXGulX*|-W z_AqGz+A*nzNt02=RQRWN!y*5S7a8_|{&$Y81rO_la3GW5*cNt$WmnkM%QISPt<<wt z4IN!;U`HAKaCXiBGd!fGSvfpjBX@<H{a!<LxGI)tZ3(-0ESc!!E#c-0Zg{9ZyOpPc zu~@J_O;4W6Zmo~h1k`k(CZbNQMs$|N5;6Q?b8BOLb)eK_Dy2HGOJrcEqaGfY*Dh5x zN{LRXr8(eIkM|~8+jwhBGjtM@S6;a2!j+Tp8gbEr?D5`Mg`BcFYueS&v7Y8R^*&$y zob&ODOC|4T#})?2Kdo%BaHh1AWCR$}fWgSZXt$Gv$Z<5aD$enU#3ZM?H4;+%ZXdeA zucWn4wOh1LxshKk8@m#%5pCahU%yp5cHq)W4{()NaP`2YSMvo?2?@ASPpqXAT`_*v z)k7&RJ)j-C_4V&+`=T@&Y#?nU-As^;W3wURy9`FfXqW2IAv_+7#^NroAAQ~G>W~^` zjDf-D4fq2U%2=tnQ*LW*>*Q@NeQ=U48Xk01IuzADy1ym0rit^WHK~^SwU449k4??k zJX|$cO-EBU&+R{a*)XQ6t<lT5x3R5ijN#=Uww{w-I`L>~;?kuP)y%}DA(=%g4sNM$ z8a1k^e#^m%NS4_=9;HTdn_VW0>ap!zx91UcR50pxM}wo(NA}d;)_n~5mQGZt41J8L zZE5Hkn1U{CRFZ(Oxk3tb${0}UQ~92RJG;|T-PJKt>+QV$(z%hy+)J<T?A6}N-j;St zi%gMo&%b5T>z~xmNJS#48TFsM{-?LHd-bxvg|X{pRq&u74~nC4i>i16LEAiprfpGA zYjeP(qECX_9cOW$*W=U1YvVDXKItrNcS$?{_zh2o=MDaGyL^>DsNJtwjW%Do^}YA3 z<vxR*f5exU`l99;{HL|G+UOai(-C;1*KH>3HS=f@249Y<c{olDhH@Ei))pN&pe^Ff zGx#Qc75}M#tM=4jNnT#t$*<LJ=+u6%E&l0GWlJ8+=FfnVa67G1-}`75M{^jGK*#$- zZOs)ja~eW1XKP4h*DUP+#=>h{jnme5ZRV>tcdeh+=o(;eXg_-64c@tJ&As=oIrFZ& z*Gx&Lr>wdAF8POg_#5blBAP!&nm-O!$wspA>@;>RyOdqWZe?F%--gC9nTXZ%DnmK< z`p0sh@aOosD-jbIo<n&)Ed2@o&q@Dz^$njmJRJ)PDt3y8!eYTJBQhuB*$-bnp<0kr z!}n0C&npvC?o%;o445>je0ec`&&fWsK?xPdf*L)Qp(MwKKIOtB+EDn(3w-9Ns9O~i z7MwnG8-?RZlv&XIJZUK*;)r!1@Bh4bnRO*JmgwqANa8v4EvHWvBQYYGT?tN4>BRz1 zf1&5N7@@!g89ym5LO{@=9>;Y8=^ExA9{+#aKfFGPwby8wn)db@o}%Z_x0EjQWsmb6 zA9uX(vr-n8$U~x9dhk~VKeI!h^3Z2NXu;>n6BHB%6e2u2VJ!ZykHWv-t19}tU-Yz$ zHXl2#_m7V&O!q(RtK+(Yads868*Wm*!~EzJtW!oq)kw}`iSZl@lNpanZn&u|+px84 zZrN7t&ayK4;4x_@`Q;;XMO4{VelhvW%CtX7w;>J6y=346)vfGe)zJBQ9o$eAhcOPy zjwRa6$CvN-8qHjFi;}h1wAb{Kcnn{;+ITEi`fCUk^_(hJ&q1Z=yo*jRs<94E#yX67 zRj)s)V&gd0VVZGcLALQ|_Lp<4{XEBIF-*yma#;%V*m^xSuqeG?H-<is<Gp4jPsMvl zy(i?PLjG33kUe?vtWz*#g>7=M0Cq%%W9`2Oe>Ov)OMv8yKrI^mZ$ql{A!!3mw_27Y zE=V#cA@HopguAWPAMhKDb__-Z_(TN7;*A`<V!-bvnypiC;;A<s)ZWy7roDM^(*wKe zFRBbV>XxrMefxoz4{Seu)$%$=sPf{vT@Pf_T`RlrC#CPDl$#FnvU|VBC$0(E>+3EG z&3xsml}L_UE3bNGX6T~2dV6S%_M9{`E9kgHPa+9mas{tj$S<&{z?nRzH2b4~4m^Wc zVF+o4`w9BO_!IohZO_=<;=$8j?7KUk(S5llK6wfy9m$GsiN5*e{q(ZS6vU4l6&{s5 zXrJJ@giK>(m%yKhRT;egW||O~pGJ&`7b8-QIchNCms)}88aL8Jh{cIp1uu`FMo!ZP z1fne;+5#%k3SM7Kqe|`%w1JI=6hJJrog4j?5Iq!j=b=0AJS5%ev_9?eR!_H>OLzLM z_U#QLoi=0npY1+gHmde37Kgp)+PKl=nC>pM|EJCAEPBRXQZvb74&LUs*^W<qi`ikb zULO&OgAOE?{8Yo8e9fP&eyOZA+Pu&jv_x+)uMRl2HkY|<PdR<7%te>CT5Q%L-{O+y zQKgd4Cek)Gjy~OLwb&xJT2>V%wrprI+4aOtWs*;<9pGE>o8u|RvPtYh;P$XlhlqF_ z77X`$AlrH?NJj1CJdEBA8;q*JG-T8nm>hL#38U9ZYO3UTNWdO3rg-pEe5d=<c84sd zWQRT5%`;6#i}quSQNGk|?_N5wAN*7|VF*u7b<CmleU4))Puksbr~r-Zr})lR$WQcl z0q0#&W#)RbB$@93ApKqQu3A4&eoBD$^=Kjy{W_r=@GXcAyFithN6dfGG&DTNpC!%Q z-v&ejc41|R6RwdY4OGg}W1lMki51+cmL34l6^wCdyfmKwSEtD)+f3Q6Y?sw)Gj*CI zi}b6|!q7)zGX2paNg*wS#A#x5c`sKue@%-?6K0wvqckD=0~43bGcBbiX+bdf;*Y5> zw3Xi@nV)1`P%F?Y4<N_71+4WAj)%^g1&pq6EY4%lIS>s9yVPgPYT9d#3SLD{*L0U{ z;TtVh?Wb0Lp4MH{o@L6GvhJE=Y2u>{DI_hMtZgl~^3m3#ZUrkn?-5E3A!m!Z>183- zpkovvg1$mQawcNKoQ*tW=gtZqYGqCd)D#K;$p113iB1uE#USvWT}QQ7kM7!al-C^P zmmk!=rY+UJcJLry#vkO%<X&wMhX@z48d4#x$RV9unDpR5F^<jPm*#*xx`6FaC(DhT z#3rNcGx$kv;@)Z~0OANH>BuM>pb)46x!{DkRYY7wGNK$v=np_sv7nfHZO_=eyqLSK zA6ebf$Bo&P&CR_C*7^|c<!)Ox)fnUtm?B<24<QbOJc=X^B;oVC5fgtPD3vK3tC+{8 zWfZH^2?^L^<BS3Z@SM@wsdznQ9$SV+Ib*};vG+PzPL9Ah8!?Nc&}*<rjuz(Mp5ul+ z;BOvJvyc0Y#K-U$$Xht4%<w_^r0N<ZPFkc`;!&Uj1{y>A>zl^hJ7z0?xu#wFzN=D8 zxm(>@s?z1E;|!Py8HuyHM}_W5*Ff>m5U0Jhy?txD<f<*#QXUV_l-I_2Mu=7^T_+=O zrI<wo4*<R`2iI80r~*}dE@&{M9I|wBXG|%;7p2>x{jjLGNXs}(CVxgu9Q4tPgE+Hm z*9ll7bz80456xzta(cX+@W!t7xTWR-OgnG_>YM~t&_#5vzC`Mp5aKlXsbO7O0HKAC z2iQF2_|0d6y4$Pu5P-bfZMRzac(Yl{IQgfa0V>u;BJRL(o0$1wD7WOWjKwP)2-6y$ zlPcRhIyDY>{PF<r;4(jy?Kp%N?>LvIr0!VoC<s-_2bN?|P6d;e4@#tzTqN;6&52j$ zL9K!Yk^iN0oK#a%RaKIGbzDIoIan!+0}vWdT67`CwzZ;fPBHD3o=~>e;c_}dp>U-X z`pii$Ju=g+Wy~f|R7yuZZjYAv<!8-$PP~x7F<7|MfT8{I*mxeRuChzu*;<-uy$7wX zfW)8-sv=vO6|DVW?SSA~s?O!A+`jwgdFE@wp&3S}!?MlHO>4AYJT}Ct-OfF$ZUBa> zOiKl0HSvn=+j1=4%5yD}dAq5^vgI~n>UcXZJGkl671v`D74kC?HVsgEVUZNBihyAm zQUE~mz%na<71JU=u_51}DT92@IPPX)0eiDweVeDWmD&fpw12L;-h=5Gq?za0HtmUJ zH@-8qs1E38^OR8g5Q^sI0)J}rOyKu$&o1s=bpx{TURBaQ(!P7i1=oA@B4P>8wu#ek zxZHJqz$1GoJ3_W^(*tZqZsoJlG*66B5j&D6kx@x^m6KxfD?_tCIgCRc?kD~(zmgCm zLGhpE_YBio<-2T9r;^qM0TO{u_N5@cU&P7is8f9-5vh4~t?zMqUEV!d@P{Y)%APE6 zC@k9|i%k6)6t2uJRQQTHt`P5Lgg%h*Fr*Hst8>_$<LHwKtiL1)`Z1k}WtCHTp*zG) zd@VPHYT9L%x%kdIFV=dq86xx&$bIH1Zqhz^D$`DQnaar#F4iXSa**JGiEagk!B1Ru zqf}j`tAlBt6VD8mxdq~os68b*H#O_xS!$-F^!N9SyJpGFGcsDA%NdMS`o_HX^r~mB zjWxGkcg{RpFs8EUv14}XPvuB#dUDpR^;xx~)V{7E(Xh)L3rc?r1?=S;$BgyVCF%&l zFuy_Xs4r#_gboRee-4jmoQc^ehr?t}CM0kelFwU7%lRN{rjzh-BqK;@Rc;%?knZq4 z?ccU=h)bPzW!2>J{ZI{mNBjN$^2t?KP8*6_xXu5xx8ufMp5R?P(R-t`{n6c{!t+*z zh;|Ek#vYp1VLf;GZf>~uUhU}a<>y*ErioacK@F{%7aq0y(Ytu@OPe;mq`jlJD+HtQ zUhr^&Zeh93@tZASEHr)@YqdxFu69(=VFRCysjBoGqZ!U;W1gn5D$myEAmK|$NsF>Z zoV+w>31}eE0iAN9QAY2O+;g%zc>2t#7Dq5vTvb&}E*5lHrk<o?&An%HnHnwimHSFY zP8?Z+cWG1wcZw|v(B!8vX4BBN5o`uq^+jO9@}GlL?8!jh7Q&l=w;&D@9uU#(ln_n( zJ72U#{LF9cuvtrE@pkXbdGls^+vBmI&9;MoL;I}_Xc{+G8XF^D+P?iukw$!q$lw0^ zz0jr|)UspMSTOD_ceFnA)I+U~a&H{iKlXcCLc4A;Vj=;>rj!I1b0=@+&c(qJcmok6 zS<GPZ0U@7op7t1v!25ZiLn#F<sK)L<i<qmv>ZAuQ496j<&@a6?K6ox1vRks+RqYD< zT9On_zdVf}IStW^#13*WV8wHQWz$L;0cm)|JDbh|f~*LV8N$;2oL|R99**#AT1smo zob=4dB_WB-D3}~I!ATFHzdW%WacH{qwv5Go2WzQzwRrv)ZajWMp{13T_u;Rz^V-VF z@#62k@#FD#t@v9ye*A%@ODWm-@oM_$_3Cy1BS+(+ujzNF@8a7?`$B^{iX2A-2_nA? zfi2=05XV^;D_2G}Up$eFW|Ofb^zuE)bWHkXR4Jm!Sz0O?)x6QD^kOuf<UNV5WGvfC zOct<!4cuW|m8%OHs?V-X)nS122bze@LS9Yu`tzXRX#rfMv4X!6B?!~W-hTr=v<5(z zRU#8AlYS9P>R`*v0=|sS?#*ZCvvr^VkV!zhLF3}FHf%+=#@ae1Qq<4~Y1EGYK$Ib1 zg!s~&&u27X&4Ks^(L3%}Npx!_-A)We=0v#yzv03fzxKZ8iV6KIX5U&?>^E?%iIUZ4 z2sD^vRg%kOU!B5@iV{&gBNc9vB)i{Wa@joIa2#4=oAl|-xqj_~$h33%zgk*UWGUV# zf3>{T#2buK?AZH?)h>10N)#VHvOV}%c|wR%HF|pgm8k`*=1l5P8ttZ1Ly@=C5?d9s z)R>B@43V`}=0??4tp?Y}Ox0$SH)yg(!|@V7H^}C-GyAXHFva04omv@`|LCuFRM2`U zxCM>41^p9U3cR>W>`h`{m^VWSL0SNz27{ske7TN<Sg(~G0be*C4|__Qs<mm_w8%)^ z?}L4T+16fl^uMdxZKe?4t1S$ZUp2!o_GpQgr^Wg!gD!bl2b82bF%f#7a2^$D3s9)6 zb&ide8nHa*q)+0%g`~$pefr^gpmInnX<ZP7Vp->1dTpM|P6Hn!^*}+fr>rJ*+GQN{ ziKp9Zda}CgnbNv#9^^&{MChK=E|Wr}tk?tP#Q?iZ%$2k;Eo9~}^tmv?g~<gBF2E!d z>PW^C$`N)|awe=5m{Xqd!M=ST?2~(mWjdOsXK#yVMN(qP6`q#tg+rQexf|*BeIU)a z^WuJyPR4WVsA<ziZ9SGIqXyk(0hNQt%J-j<&BZ9m&0w6`VyIog&iYuWAU%FJMvhDo z@BnM$s{+HX7{T~AcVgogi(AwHuBHMT&d^d)q@mg%{q7s#Uyb&}7WDr@3QTNjiCN$0 z#j}zo`K9o#TE|)CUWaG9!@$RDsV?mwab+_Pn0#iN+Ia@oI)f|9#>Tp2E{*y77*kZ9 zEB{*SRHSVGm8ThtES`9!v{E``H)^3d+TG_?{b|eytE1cy^QbPxY3KFTWh&NZi`C?O z;777FMti@+U+IRl7B{=SCc93nKp`>jeW38muw(9T3AqySM#x@9G|p?N;IiNy(KN7? zMz3hIS5SaXrGqD(NIR0ZMnJT%%^~}|cG(Ez!3#)*o{{QjPUIVFOQ%dccgC0*WnAJW zL*1k^HZ5-%bN;%C&2vpW`=;dB5iu4SR48yF$;K8{S<c_w99R^X`sJ&aw1-E`?@ZqF z($w`|+q<M;f5XMvC5{k3?ed8Y9buPZNghg=Hq1Ondc_zVzi$52i^i7Ont#z!dU3F^ zf9+DG*kP@RmW}i^tMV)LCYxDR_&mwOgJb4jGb^!R^q5d_^$T~L|FsLIlo|YDex?<M zz8J%pkH{U!UXaI=jZ(agL{ktw+FLN9g$4;v(nuSO7+*A$#@XdRZ9-|8jc>Y`7mu6c z@q{10W=zwHuav3wid&;5tHCUlUgeVf&>wKuUfEVuUsS%XZ2RPvr>;HI=<(RACmN-M zR8(DJD^lePC9|rUrFgR?>hO#VkFo8}zA@jt{ERalZl$!LP4-GTT`1w}QNUcvuEFRv z`)NyzRG!e-04~~Y1DK>70lGq9rD4J}>V(1*UxcCtBUmyi-Y8Q$NOTQ&VfJIlBRI;7 z5Dr6QNIl|8NTfO>Jf|kZVh7n>hL^)`@3r1BaPIKjxrLrjf8A>RDaI{wYlKG)6-7R~ zsZQ}Kk{T~BDVLo#Zm@cc<&x{X<~boVS5(zfvp1s3RbASf6EKpp>+IFV9s`#Yx#+I& zMz5zL9IUgaqrnG*_=_qm|JBcwfl`bw=c=uU^R>Nm%k4_TeDj<s++>y|&K2eKwx!u8 z9&lbdJ?yJ@)>!NgE_vN8+*}$8+Uxk4EBNje>!s2_nOCtE+ie>zl!9&!!I)?QPMD&P zm$5sb#Le|%L<#tZbz%~WWv&yUZH6NLl>OK#CBOp{e~$&fuqQd03DJfLrcWa}IvMu* zy;z7L)WxyINd`m}Fh=l&6EWmHUGLkeP{6Vc;Xq->+AS`1T*b9>SJ#<2Cf!N<)o7Ms z!Gj)CiteiY$f@_OT4C*IODVyil4|R)+8nCf&tw%_BEv!z3RSN|pG(k%hYGrU_Ec^& zNRpzS-nJ*v_QHeHPu}Iub>F_}G1*vdGR~ZSdaG(JEwXM{Df;~AK)j(<_O<)u)`qw* zQduoY)s+$7NdtxaGEAo-cGn7Z5yN#ApXWD1&-5uowpb7bR54QcA7kWG@gybdQQ<S- zDdN%;r{I|pU-aM?e9q_pBe-?*L>a&cxCKxup2Av3_#{04Z^J#@M&a}P$M<((Zx{A8 z!Ue=%xTpWEzWzKIhsO_xc?e$$ai{S63-$76>gtB?9usV&`qp=Kn*GE5C&Tx`^uyza zw{^ImGi-hkYkP`^0r5vgoSL$EjuxaoKBh2L;dk#~x%`TgefEDi7^(~<C<r}yHnA5! zj8>cmE)UEw*l#i+5f-;!v^P%ZowUbhH*3Av)CifOJX7KS6#d|_83fqJ#8VL=h2KMI z<X$qRpzLMz@O*B|3fjxC*>GYTbGm=Q=0lfc{$IDTn;IxIgLZ(Z?)#!mln$0r3A(um zzBIGw6?zmj=H#CkvRoT+C{T=_kfQQ!%8T;loQ5;tH?lZ%M{aG+z75&bhJE`sNSO`$ z`0eget1V7SqB@uA;kQ4UkJ-235xxryG*uzwDPikrWOi1;8WASslh$U4RY{JHgggsL zMaZ|PI2Ise8dMEpuPnW`XYJY^W$n>4PxVOPCO#DnHKfqe+Y7BA6(=QJn}un5MkM7S zkL?&Gvnj|DI!4xt6BV*t)Zv0YV-+(%$}7QcBMZ01jlLEiPk>A3;M^g%K=cNDF6d!7 z<aM&9@bguR-(8%RuJl`s*}oX^lq!s^yr(~-b*t}2AJw`ajkG3X??#boRa)-xQ!0y} zQvE{f?$7WZsW%yYl=nOmi~R%I4k0fIp4kYU*KwK-JLnPSlu&%jx{(IIGpy})j-Ni> zq1_(l4SX+ekaM;bY|YgEqv2RAEE}e-Im8<@oEZ?Z81Y?3(z-@nRbq?!xD9Hyn|7Gx z-NUw`yOor_DJLC1aqkf2(!i=2$ULNfg|s8bV^xB!_rY+bHA;KsWR@aB=!7n&LJq(} z!pqD3Wkv<EUKw=QUZ{YeNjs!1)DE3)xa{mp%FNA)7E{?JXJ2MGou?yVULS4?IGusE za6J!4pgDi?kcM_0y6pz@LpT57oC=&XKj#lOKV-g6pRa~b&`twzY&7d+(*cz}nmFc; zUG6sS0f8^TDzbqAV-Ci#0Q*4v!S4HIG2})JIE}Yg-neehx^9VuT-t-Kkc%%pecy(D z;Qzi2a%y}l3_7f%?CyY{3^<lfl!0RAl%uAbhqB#Tud-x`*1H5Ya(l{nH;QTRmNJ9o zWPkOXUdo1jB}S(U#TX2_A;{8+*V<5uQ+f$I87N9BTf#$2%1{bKYJoGfM*x|>o-Goy zx1edGgnc}u5V8cw&nvWyWU+wXqwinB#x7(uc>H44lXZQkk*w_q#i2O!s_A?a*?`Rx zoZW6Qtj)L1T^4kDeD7;%G5dS816OPqAqPx~(_-<DuB$fss>jZ`bo-MR_kd&sJv{A^ zs<mvsogQ1M8mv<Ct$f8}u@yT_X8yXC2EC}uY~H5r4-`o@d)0;ATiQp>@18qv!kD;U z5Evv$<lBB7#(Jc%96X*_<c5{DXipqiqGV{lSf@y=%UMwtHg|ADEjAhIx56>C*bD~m z+x@>Oo>;7%QCxfp-rOkNgx4j-(o*e5`6lW^X^{qpQo~SMWD`Gxyv6)+k)c@o6j`Yd z8c&XSiYbcmoCKe+82}>^CPM+?p@o&i(J*j0zsk}!P?!W%T5`ppk%)?&GxA`%4>0VX zKu?YB6Z)hFtj@u-icb&t5A1}BX!;~SqG5ARpVB>FEWPLW+C+QOf~G-Jj0r`0D6|0w zQUs5sE6PYc)!HWi))NeRvSZB3kWIW|R<H0_gGW<At4DglY{@PXl7rw}vx_Wdy?mi( z$0>^A%RfamB2jCbVX(Fn>y%#b1W%}W%qc)XVrwuvM!>Qur!Ooy2`n@?qMe3$`F2vx z9<=L}wP7@diWhCYTD?x)LZ>F6<rMsfn4>F?z8naL18P%1T9&P_d4p;u=(XW1LO3-< z`{|5@&Y=}7s<hKtKn``g07m^x2L<{euW5rmHASWqkln<y;G60p+yy1Vd~}>x3t1Zs zr9ZBmp}Yp<Mc)Mmn-+i$m0AeGjn*2BBdF_u2nszt)O{Ez!VomrO1NW@UJxJXIYe56 z(hC@vJZyP9IiY;2hPG1)J9VoU!%1sA9yL#0JHU99x=C-a^r*#L2#n~b{G2!*mf=7Q zwzqIf9N*qy6``%-9z_FAX#r2LDe2LPf*507Fz(b?J9;sPV>HLq7lwu?CXL8<aWMNg z?79jh7{F8S&wB>$Q65$Q29AlDCBJSxu5<ljsZ?&yzAt$^o=0gUg23{>;p0({^4skD z+4se#9)xg8qnEh|WnPdgQ&+te7@`9WlzAwMit$Julp+d80n+VM1JxwqS5H6*MPKA` zlJ*Z77B;K~;4JkO5eq(@D}tezez*w6g3ZSn?J1d9Z~&MKbf=b6F9;8H22TxRl%y1r z<-6(lJiLAw>r^-=F-AIEd1y|Aq2MggNo&>7<AhoY=YGA|J57w_WG(=uW_6WAD9gb5 zkllb$JpC=Oq9dIg7R5r|P`SqQ<dYuHB2Q_MK7ytAJTZiC9@ZoL{?AIDD}Wz7W#0sZ zofkn1HXLlFfEDzB*m2m)4H;LeihD7}U0eAVJQu^P`eK4ns&w}cgGjC_a|3kkqQ9Uu z?lK56cLJFz-7|;5K>Ln)S~iAF1;-4`A*9KlL*vleLO3vhEd(@RsIWp~O@>N4p91SI zb~+*jP?8B~MwmI0W$>ksF8DC*2y8K0o#te?D$z8nrfK{|B1L^TR5hlugr|o=-;>Yn zmL6Yt=NZ2%cAsysPA)D^gkz2Vvh|Z9RJdoH$L$+6a^|>UO=3fBBH0UidA&_JQz9K~ zuo1Z_(cB7CiQ}4loOL3DsdC<+wYysw@&UMl21+LY-(z=6j8fu5%ZQg-z6Bor^M}LX z9hxH}aVC%rodtoGcTh)zEd=yDfCu5mE)qIjw~K+zwn&5c!L-N+E=kwxVE<t-E%KoZ z<y&iLhOY@duv$!~)enS*2k<Lm!CRy6J|N;BcsF`0Wf&G9=@jB5xd)5m4ImV!$et^Y z=;6?K5$zg_9uNnA#Dfn1K!>ewN#vvx2WGCf^;C9<Gj>^mmTlYc*kz$NUdQ=gDzLmf z!LXG7{N$Mi3n}?5L&f9TlCzzrgGR*6>MhWBR=lS)qP$&OMAQ2<kLLosVT*|81)fcx z1+w&8<iP=6ky2!#yku<s8D`Xjeg0*f26T&a!7wAh4DXawVOI&H0$tS4$mGs-tDyy1 z`?j7-f4=Li{u1fKn;wA}t%hLg^jDmT`70Scfh4r^@|UE|58Rk_oV{SzFQ1i;EZ)_> z`$23{zM%a@9EPdjV|Y1zVVGf?mINO)i-q6;_Ev|n_JQ^Zy&BnUgV>NbY9xba1DlY@ zrg$_Kn?+^_+4V4^xS94tX2oLKAEiuU0<2S#v$WSDt0P^A+d-+M?XlR**u_Xdre&aY zNi~zJk9aLQUqaFZxCNRmu*wnxB_u*M6V0xVCtBhtpGUK)#Dob6DWm-n^~Vy)m~?Yg zO0^<MG&&0v@aU{0YUeT3jttrL)l^>+v~`x6Vqtjl4I5;=^o2jyOb~m+ER;lNwO$iN ziH4vk>E`OTRx~v#B|ifef|ceH)%hgqOy|#f=Q|VlN6i{!0CRndN~x8wS6Ppqq7NSH zO5hX{k5T{4ib@&8t)u=V9nY+2RC^75jU%TRix}FDTB%>t;5jpNRv;(KB|%{AI7Jc= zd%t9-AjNUAs?8m40SLOhrjbC_yZoznU$(rnT2);Rr`2e6$k!zwlz!d|sZ3%x@$Nw? zVn?i%t!J+9S<yDxP;GX2UFING%JN5ldLzG`I1!{O<N*T20}&)36oWiJP)STb(%fR) z^L!VP7mPfVZXa5-Cw8K;oHguD-UZ5Wxf}o}JGCe1+hXBel!A<K`TIeUe$XhF(j)l{ zVT(%{kUl6fAPK+$ziJ!!iu$2?1+-qoqn!edQ^74P`A!1JDd3h;Cj~^h{iI+Md>F@^ zO&TGun2&?VIygfH5ePk|!e&G3Zm-GUP(imiWzZu$9JU)Wot`}*RHV<<B{3C7B0`1| zb>-)vUhc6J6{w&PQIaSZ_N<(d>`C$yo#Ly&0Sr5gCkDY(4f@fY5!fLe57sH54#FF4 zg&hda`KjtJ8cTzz;DwFa#{$!}j~g$9zqFBC@To^}i#`b~xhU;p{x{^f1krbEFNqV^ zEq5c!C5XT0o_q{%p&0F@!I;9ejbs#P4q?R!i$?vl3~|GSyq4@q#3=wgsz+zkrIB<< z=HMWEBz?z??GvvT54YsDSnRLcEf!n>^0eKf4(CIT{qs4y$7_4e=JoIkq%~H9$z-r* zZ?`xgwL+DNAJE<u&KUS1Ezcw_$pu?O>`VB;S+w#NvBT{3;}{CD&@Ig*Ka2Acx)2Qx zL)V#$n@%vf1Zzms4Th~fS|(DKDT`?BKfX3tkCBvKZLg^hUh|_Gz8?%#d(ANnY`5U1 zo;qjq=5tn!OQ*-JqA&iG-Tg#6Ka|O64eceRrSgggD%<C_<FE1olnRcjTgNn5M5OHQ z&ChI}{m3J;@p6;=pWDOmu{ro8z!hi((IT*}WPJvM6IkXOn63i^qh*JA3M?7G0qW;I zX>%QBX$t=6?hPEK2|lL1{?|>I^Toc>rQU7a_`RSM^EPVl{_&OG-P;|z0?v{3o#pkl zC6Y;&J7;#5N#+H2J-4RqiSK^rj<_Z6t%?`N$A_FUESt{TcayIew5oWi=jxT*aPIP6 z?MG`?k5p%-x>D73irru{R?lu7<54DCT9Q}%=4%@wZij4+M=fzzz`SJ3I%*#AikLUh zn>k=5%IKUP4TrvZ!A{&Oh;BR}6r3t3cpzS(&|cEe&e{MQby|1#X`?17e9?|=i`sPG zL|OOsh`j@PD4sc6&Y3rT`r?-EH0QPR*IobE@_fkB8*(886ZkjkcO{K8Sz$H`^D-8P zjKG9G9A`O!>|!ivAeteRVIcyIGa#O<6I$^O7}9&*8mHd@Gw!WDU*@;*L;SYvlV#p( zzFSsPw&^UdyxO}%i)W8$@f}|84*mz&i2q@SlzMOd%B!BHOJ<(FYUTR(Ui$DuX>?85 zcdzl5m3hzFr2S@c_20C2x&N)|$<=RhzxI!}NN+yS16X^(_mtqY)g*Q%Fux5}bP3q$ zxQD|TB{+4C1gL>zI>g~-ajKMb{2s_cFhN2(I(q^X!$H(GFxpc6oCV9#maj|OhFZaI z;umX6E*fQVTQ@lyZauuv>%E)5z-?zQZne18V5A}}JEQmCz>7^h<Bd<fQ(OP%nTPmA zS6%V-@#lVf^v}OCKC_D2scF(BL(TlBQ96*dK6|!vDvF&o;ni)GU6t>0r)!zhinBG6 zMQghGt!Do5h%HmAQl~%m+!pr-&wlrcwW;qw)S$6*f}ZvXd<qjrVb%W}nk2)_bI{5Y zG!_yC8`{(bRk9VmUS!KrC7aBuqpq=4M_JWaSF~DYu8yf44}dUv?DXJU_%E%-q9UVR zYqQvF^8Tag@#D*@s>;cHw=xm|y~mHbT3yX>?hoYKfy--h<AH4|v7u4V$?TsF1POK# z2l~Z#K*f|-yBktn5A>+6w9%@_4ukf0Et^zr-DbPwFdyj0VJHi}4bqRetSNR`DoWd( z(%n5>8MQl+>3SeL-DB@IaM{NDwd{{v_HMIO)PKO}v{{##c@ihB0w$aaPTSP4^>n3Z zC8Il%(3dCLLX$-|SwWx1u7KVztXpzNhrOZQ78c$jd{B9lqsNHLr*9h;N9$i+vsrM1 zKzLB_gVdMCfxceejpIZat!MbR)GNZ%^n|fEQo?Xtq#Qa_gEWKTFxSL4b{g}<g5!pg zqjYvXf#WmS590a@AtEIYrKmWbO7RfxSqt%u{zj-2;vw3t)Ozvk|CUGsKto6|7(DwJ zqoP$jyJ=6RTYq*jeSO7zR(m1Ew81tMmn9k-6|A8-@s}tK@%iv#C$P~FZ*8kkTBv7; z8+H#yhOrc23fr;54Q3_NHJB(5rk3vMnC-7<X_?i~ODk9JQPqI6e6*h(y10E&qN{mg zVod4SToO7Foj>kJNd{QcoQ}HUP-A)Rq;U(***IA*V_0B5mr}Xp$q{YSYs-b2q~DHh z?+muRGn~std!VXuT>P9TL_8Km9G{do<P%hMjBo5*GjY+zxo5S6Mdsr|^yKI_%q!Y| z0XZbr6buPibb|^S7K<litua~z)i$~wp_60LRE(@oJO87pvtH1Sj+)>qRb-W0B&%d> z^3@hs6y5jaEq%P}dmr(8<wosc?GN8HeS4DK1fxZ+I2DI%mrTh(d*k$)v$^5U8{U6x z#W(eF!%R`r(0sLsle|8!;EM7q88jUgGH&h4`-Qy)R=WH?{fXNApJ)$g)3pbF0`?qg zhmjp;G}bkq(J@l@(gj3xY+Luy<wQe1DU<%W_S}<C@`*neDS)*xzoPg_5JC>=f}x~^ z*{I{tkBgYk@Td|Z{csd23pziZlPYt2RJW7D_C#&)OONEWyN`I19_cM;`Aa=y_)ldH z^co(O-xWIN0{y|@?wx@Y!MeVg3Ln%4ORu5~Dl6$h>AGSXrK3!pH%cpM?D|6#*6+A# zlsj;J0_~^?DHIceRC~0iMq)SJ&?R&if{fsdIb>y;H@M4AE`z8~dvz)(e}BqUWK^U~ zFy`PX+z*Bmv9VxAN;%CvMk(#kGBEMP;a-GgGZf~r$(ei(%yGqHa2dS3hxdTT!r>La zUrW2dCTZ!SjD_D(?9$SK02e_#ZOxdAhO%hgVhq54U=2$Hm+1^O^nH<>wS|&<)2TtD zN_MN@O>?A@_&l;U)*GY*5F_a~cgQb_3p`#77ax1iRxIx!r0HkDnA2G*{l|*}g_yI% zZdHt2`Hx^MA#VH7@BEN68Y_;<n>sAcCNgCY7S&dcQsp*$+uW7Dm@$Vl7!YA^51bi} z*Vy8uTj{neIhI<oWy=zdr_{4#i`<hHemO&-p)DvOH>L|PhditfC1Jeub(uy}w|wV5 zsQz)04y;BY2$7U4$~P{k)b`hZb>gv1RkD)L#g~$*N^1N1GfNMS)4r|pT*V<&KE1M9 zTh}rzSW#Kcci_#(^qf0gTW3&QN&zsW%VAQ+AZ%-3?E)kMdgL)kY~@mC>l?RH28u;Y zt-@_u^5(W>mDdt<VOYVgE#|A(uSEv@fOheek;xg_<rDOsNOl;gd)!Vja8=hch<)Lc z7C5x)TTnfC{Ty&89Y{5hHW8hoHQfdd*GFRp`xtUQY*_1G@JVU(0G!G`VL(RRLkFlN z{KC?h9ZQ4a9Zk#WEDGc4V`(K#V+h=(@|#!+B^!*`vNb|*0ka$cyGgGYw7@VL2ZaV- zNiT>qoe){#t;3NA7c@<nTry$GxFrj|q+b7*)5;f*OrAfZ+;0o|R!v@aTd;Wd<7=Pa zv0@Zt<2sl-2)ia3b{+ElCFYvhG26s1EhzUG*DY&qzkD1ok!G#8o0Rcq@I~^{X&b(} zZ@$|+f=ffW_Mv|;7CHE_1<Uh0i>{WoY9bYFNoq+sj&ru;Z`x>4ddY0y*`HRtHFEN% z@mFkp=x0C6zDGgA0s|mP^WNEwE4O}S?%DOtce3At%?ThxRp@`zCH6MyzM)dA9C7IP zI}t;YUV(Jcnw$4LoD4H(EM#!{L-Z|&fhNYnBlKcQ$UScR#HH>scYBTf2u|7Fd8q$R zy5Cbt=Pvf^e}m4?VVL@#Pi3z*q-Q0MG8pGTcbS|eeW%R5bRzKsHSH#G<xBsM(j4bg z-=NG2#y@lA_H_D~$e_|wjQMoK?HF?(ofQZ4Vzj9Zx`0FjEN<3>(#$9hj9}0O7lXsC zbZ7#UjJM^FcvdKK3MOEl+Pb-93Px}F$ID&jcvZdJ{d(D)x|*`=vi%1hdg(dd-1E>& zoB4U&a${9!xyxoT%$<MsXn8+O$mZ77dXQPXHRf6#s%oUnHFa~!q_1s$>7gFp{M<_q z9oVnk*Dcp$k#jA#7-pZbXd=L8nDhe<*t_*%gj^Vx>(~KyEY~i&(?@R~L_e^txnUyh z64-dU=Lc;eQ}vPX;g{GitTVZben7||wttapene^dB|oSGB~tmAGqE^`1Jxt$4uXUL zz5?7GEqvmLa{#mgN6la^gY<w3YLNeVW;rdeNWK%R0!+U2r<S&xzgpz0YBw4K8$3mp zE21?;R%76qA~z3I)m~||6<Kx~EuGGQb&th9nAE?Kl2+9hCN)_pskOrqC_+*xeTUWE zqHK~(U3Rb6Ywt2io0Jx}^^QAit`<e<Y|lls)EbnHN{h>O#}`eXyUJ)lFyTO8*iL~P z$A`A_X^V#!SJyU8Dl%J*6&s9;Jl54CiyfA`ExxmjrZ1P8E%rJ7hFCFo6%{5mRa|LY z<hHMFEJ$i9a(4NfssfTL+q=W*C@s0HM3JX0^+}R%=`>k^x76W8M0tQBa1Q(&L`|!e zrczv>+#&b<PjV6d3K}eU;V9%YaKm(oM4Zr(@CHXFAW14hML_r4VqzLAlt6`n@RklP z#|_mXnz0e!Du^;&6ZEY1{U4mQ=<0hS&2o_k$4F#Jm0_q5ahCfmSKsn1pTe)@Q>2bt zuD1Bfoe>oW0&!ju$-LI)$URptI!inJ^Dz|<@S1hk+!(n2PWfi-AMb5*F03&_^29MB zgJP7yn#Fw4n&Rod*>LlF+qPx5ZT$80;+m*0X5ffa3<jfYcXFe-W}$P7vS|L2z8h!H zn)b>d-;F72#5un;L$}RfmR5&xbOf(KNeD|gT1x6bw5t;~j}(oMHcSzkCgcpbd>5UN z7e8CV*di9kpy<pA5`bF4mH@m^LV0vfv)0p{>JAo1YyE9XtfV1Q8^?ViwrKgtK$H60 z%~xgAifVV#>j>4SN10>bP9OV9<ecvBe`%dx>m`EA-H{bzMimEQ_3@VZH%@KZzjDu` zRCG*Ax6B^%%dyLs2Cw{bePFWM97<M4+}X6j=rFmY8D9@pIA_cZHBTuS=i*BuXU_~r zW>50@SIoZoff4mJvyxIeIjeZ{tYpbmTk4_{wy!_uygk4J;wwSiK&OpZWguG$O082g z^a3rw)F1Q!*)rNy!Sqz9bk0u-kftk^q{FPl4N+eS@<V2$_#KjDfHky}YxE6v>0p1= zhaBFdyShSMz97B%x3GE|Sst~8Le6+?q@g6HwE1hJ#X)o^?{1!x-m`LlQ+4%?^IPIo zHATgqrm-s`+6SW3LjHB>=Pp{i<6FE#j+sX(Vl-kJt6sug<4UG9SH_|<oE5<{&zK>( zOb(+Vn|4R4lc8pHa-japR|c0ZAN$KOvzss6bKW^uPM$I$8eTr{EMN2N%{Yrl{Z`Y^ zaQ`-S_6omm((Fih26~Bjf^W$wm1J`8N+(=0ET@KFDy;S%{mF@!2&1UMxk>jTk49;@ z*g#0?*iga;P7abx1bh^d3MoAy*XQp{Hl*t(buU@DamDmvcc;5}`ihM!mvm36|GqRu zn*3}UmnOSUai6mM*y&f#XmqyBo>b=dmra`8;%uC8_33-RpM6;x`Rrc0RM~y9>y~ry zVnGanZLDD_lC%6!F%Jzk##j%?nW>JEaJ#U89t`?mGJS_kO5+5U1Gh;Lb3`{w<-DW; z;USPAm%*aQJ)UeYnLVb2V3MJ2vrxAZ@&#?W$vW)7$+L7~7HSzuF&0V95FC4H6Dy<( z!#o7mJKLMHTNn5)Lyn5l4oh2$s~VI~tlIjn09jE~8C#Ooei=J?K;D+-<8Cb>8RPx8 z-~O0ST{mOeXg+qjG~?}E8@JAo-j?OJjgF3nb^K5v>$yq#-Ybd8lM^jdru2WE-*V6W z>sL(7?%-Qu?&?wZNmmqdn?$FXlE!>2BAa^bWfD69lP0?L3kopYkc4>{m#H6t2d<oO zM-rb4RHzpm9{wcAncBP-$z|oGYL~V;7`yL0TF=DQi;45@lX^I`Bry3%;i)|iLA2cJ zD&=p*f?xWH#B@1NV?V)Q10EuyprvuT-(Wa1PPQ5t`ay%_ybUO&1G_V*CdltU4p|^e z0stB?-MU)8M=g!9-Je~j??XyE_G1Xia8c^kF+>LIEE47|jcI$tEuWzwjmRgqBPkzk zM+(?6)=);W6q<2z95fHMDFKxbhPD-r0IjdX_3EH*BFL|t3<dcW`!>))c7d~8v;<l` z9cOc4;PkP*doonyEp+5!5ObxBj=3fstp#093;Edqi3}76dI?B>{wU5p8nHUz9I?>l zVfn$bENo_I3JOh<pvs`XjXA{<Axd5<=D(OZzz!%s9$+)Sav(kLD=%GEmyDH<n>1^^ z+un~MSwCyixbj%C?y{G@G7mSZg_cf~&@djVX_vn8;IF&q?ESd=*AJHOJ(!-hbKPlb zYi-r+me!ezr_eCiQ&SetY;BocRokkbwr=ONG<Ls)CKFqlbRTn0h9dn%08(unJL%2r zdUhMy)*6q|wFvsNtu>zW2U@X=AUvS^E9eM^w~aztd4h$Q&kF;6EJ1O*M7tJfFi}R1 z6X@asDjL5w+#QEKQE<MUI6g|txm~6wNm96XWpM0snga=U?c*@C(Hv00gBVrANXdvx z>5V48#ASm?H7u5j%nDqi)iO@<OOjW*K|ky8{5@_LH+T{a^`=0Tq!!6Vc7wal>a1@F z*^R+bGpEOs#pRx9CBZQ}#uQa|dCH5EW%a3Xv1;ye-}5|Yh4g~YH5gI1(b#B|6_ZI; zMkxwTjmkKoZIp~AqhXp+k&SSQ)9C=jCWTKCM?(&MUHex;c3Knl(A%3UgJT_BEixIE zQh!;Q(J<0)C`q0-^|UdaGYzFqr^{vZR~Tk?jyY}gf@H+0RHkZ{OID|x;6>6+g)|BK zs6zLY0U>bcbRd6kU;cgkomCZdBSC8$a1H`pcu;XqH=5<F+D~<Z-#2~djpJ<Gc|QJ4 zafx3Y8ElfZ<^ZG1&3Fb^`teMqzPjsAXz8)fn!7rC`gCl=cGu)r3S4c*Gq4a$RR9I` zbygh!Z5_qV1J=<UxX}I*0+Cz_Mi`r=u*<^PZD0p*gr*iS3Qw#7?l#<~E5s~4t;h%> z+$oO3i&T_WpcYnVu*lchi>wxt#iE!!bG#kzjIF<LQv0?E-PesO@fZ7xmz9iuWAu|3 zUjMrd=^g!Fz4{lU|I&`I87urHks0Z;=G^n6O&xEK@pO4lpNSRFihC!e7aDFZttcB2 z3@r9X-Q3(1DE7CGn(_C4USHEw8(7d#?yroD_&xXD{D}7bV{HxP<(JMVUKpr(p!U+= z9sJ&evE$}6Sys%)<pH*0b4aPV4;?_ABJMbVL;$*lmZKAN<_+7HxW54NhyI}iov<Kw zuLAu-eIuHO(?9yvSv27=w*oMg@D}-2EL~EX7+*QpynOa`+6SjKRm&AcZevS(bIF2I zyU|@;6bm`za{K5>qb)`s?|OclRAnzUyW5*Py!P@srDXI}&s2lVYf2ZCG`F`H-9;60 zb<=6weckNk=DC&Q6QxU*uJ9FkaT>}qb##eRS8n%qG`G9WrS>Xm+w)!AXSASfd%5fg z#fqxk(5L9@fM};~Gk^Sgb;7|krF-an$kIROPt4HLqq6+EL+62d@~4Hsy9nIU?=Ue4 zJ69;q+5+73nU|TQu}$>#v(M&Vx1RD=6Lu`d?>zHN?P7J&XWwsvwJt|rr?CZu+l>m4 zTi^VLh6Uu2s392u(5DLaM%)Dr$%h3hRB>V7a9XG`B{ZsWgh4IyTO9R~TAR^h^~>ko z(k|Hy#@bP}7OyN92TKE%qNZfyWL32p-BJf1{jj0QU0V`yj=tRospvSewxGxoC=C|N zve$zAMuSaiyY)QTk9!VmwUK&<#b2fxMl_DX|5x$dKH3>6sdYCQ9@c)^A-Rn<Rh`_J zlt&lRB~G?jhWS|EX+K`5L9k`6=#nb!W7EW?%ai>9vG?s)0)lCR76kgoR>S;B=kl(v zzM}o+G41dh)%9=ezv$7*a9Mrb+S@13nK-B6D!%vy(}5dzbg$`-UUZJKa`_Z{*$rCu zga2G}o3dTHW|>+P_>c8UOm4Vk-ojaTeAg0-+<4#u-{>pGTYz(%ojZ`0e*nHo=)XZS zpp=$zi4|RBMGJDX{Db?>>fq71rX3t$122E;cJ(9elj+kBXs>3?(tq=s*PeL^<(M$8 zUl;u9e6|EP5Us-A>Lzvr+ln|?*}wt;+gUmd>%?@Wl@m%Qm{>Q0JqTcxtB`ROhd6TB z$VY<7t$^N6IC(s*<w33DF&yc0dj^+WeG%4|-2BQb-dJ|q62>Z@x2?Gi%eB8%(hYaC zKfY5M<cZOXmrbZWEjps?;&|+w7tV1eFE1MrJ*_siZ1L#0#n&`(LclZD>-9MeR-@5h zZ?V`qr%%FlPQlW5v_Bp^Q?^)S*<!9Xnk@#nh}mp1o88SOEB+m*%{oDLLdJDWayebj zF_P1%goGu~ua9Y$&Q^=v^6lf6jW86qp0@eS4GR`uX09nN4hD-tBg~gAUa;ZJ8|Jna z8zPe|mipRIjUt=vcHH^U9tgxn+Hm(pY=z=3R%i3uj{TZ1>%Y#Z$|{!Lpju=$s702T z(P}foXu(uuHN!cJRK*W-8=F*QlYB*zT#WI-SmQ_VYEgKw+>wHhm`ECQS`r3VKw`wi zxlcnn26L<!RA|D|NB?I8;L63FJ8~v}K7sD;$+XuL+<N#lUCD#$fb`H4r?HkfVB<kr z;(F9QtS+CxIxw-24Zzi6CHA+pfsJf|GxQQ~rvIxEo5uj_8Jp)*Ji9qdbCkPu;?S{L z1o!sZJjbvRCUxVYC=aekOVe%v0YWg)|4SMNPhKaD1BlcIKrK)np@qZ^L7+8mbudO^ zk)b#%tQ_22KM*Xfl|1h9n@S>*U;F-BC9u{Csy#e%+2uD$He5?mc55)ot>1w`?lr$J zsrI^qGB@!5dglADaHlvWto@|S>kF5>#i#hCNXbp*ZkO$*%P-Sjf3Vc+tuFaJ-^|Ou zW8=}1TOlafUitnrTA2D0<3}&zZz^%y5+t2`Tk`vBI93FqU`W!zY;M%AUoN1V1-I2I zPTVFqaw3Pr-`5HcEFWuD?!8Ybw)Y>g7c0tt=soTHiEBxlY;RlQ`iYY-qdd94zWjyD zFcskM^S{_!E?f3mEh9waR7tb6G&yl%GW%e&Sc5i;y@N)U5ZFLcAsma^K?Cg^%d{PO z=SHQq4a|<d>l`AakzEY;A{n6Rn1u`7v~#ufV<svwufaWPWUTy=#~N`~t5mkQHi<fW z`FA<Y1Jl}-+ID`GoTfY9u${~ohdoPI#lsU?#@1GhX)dh`T;Fl!{L5OGP6ViAJJBDJ zpV59(s~y2U9&jfGy;BDCu?YA(0fISrdk`{^7k>*6GZ$`Ef)d2%6apsU6^>QJl0@U& zq|wIBlBAgf0j!YaozAgmhAy0uy;AjRA2%(!`<c5JH7OZ1B!vm4rC_yi#=>#&e>`V` zg`MfSf5gWvJY#?8%&|`Aj0<@aZ;-q#tCx=-zkGE|_C4)TqKjr-SE6po?cX?Z^B%62 zdA!75;$my<*q)n@eB<^dfFGwRaWB25UL#~PNEV>F^c+e2Be*Df(-rIVB<p_t*#`AH zc;HqtB-Eb)2q$!amvSS&+P%HPVy$^LX{{`;eWuA$Yc*DSZ@Q_huGZ4@Ol^6kHTi6f z)l#v2SgP9cn{FzvJwA<;9+pk=Q?{bop~CCRjxV~_Vy(RS=87WAacOi}RN$2n`jVBy z^{K+DkdDvjzq*1DbPuc1pgxtv0iuigUWFeWRH#nH1!2Tz!W@wXmBa>Jo2o*an$1*1 zD$bsU<F&S$>C-BvObdmkKlhW<59G9{d=@bAu8a05VWCO=@_~oP=G3SmO91AK_F`#5 zwXLRV<oMV%5Bd3(h00X#=;XZmN#kyA@TE>ay<~JYok|rdQM-~C?dcq?Yfz_*)fIte zkE_g4CeLj1oza=9zH!s!4k%H@-n{6aB&Z;Cs8MK?#Jxl`?wD>^{fTL&eQHAQFtJ_% zNEfs|gGYh+39S{-@#MrPA!XpgWD;NLlne0-Vey1n0<q$-*MVI^cPLanp`)hUq)e}| z7juu>?=ww18{L)7G|$1kjI(sjs<Lds{1=|%D}}i9=&YpHJNDt{uDfMF=i}tCvhSk1 z_N?Sr`X9O`HhyGPjjeiuRBSKx6$dOll$@28(#6enVY7OA$`8RH91Dz_R6j2{+B+L& zz(TPanmNN+96IxiR7w}e=-SNWtg%zRx<l={!0#K~x#yw__jF9&bYpd@^9*0xOd0*> z@|a<tA^Ht;SLo>lUMcx*04*>=BWHv_W-t=rCAy0q6&*<vg?_<Mk?9T;qojT!H54tn zjcQJE0gFXPL4i}{Ab#X)g(-*L3C9g{qGk3lO!z;%y$4`h<+(qs_YCcQv@Cf@mL)k} zNxbC5j-$lRUd~QPNKAGdMiK}KWFR1{Kp_MOlodm1fl>;kW&ImkwWTe$lzHJRZJ{-{ zl-mK6+j}V`wobm^^B&2Tl?1r=yWbz;v-F<#y!(CT?-4K(($wWtmD6<j=8=JM78lKn z9fjlZOoMjWml&PHLwQvDG((TE9g7Z#cWwm0Wj{3E?2<NugeKbCX_Ln>31MN9?trDG zMI7;9U7|UsC;urLP%eH1h%U`LJxT3oM4=gpi%X@lpVR9N6Q(uhJ00RWXeL-Z*V(O8 zsIyyVUvf=RXLBKX`!peifjIMvMs1YT0n$0*B;K^yZf&HN8$N%e=EgOejqihLPBT|< zs)z`nNU}BOdT7wYLy}R10eXUksn9o)jG<NYa|-+p5j)^0ybo8gK3wABIrb0o2pu+4 z2ecT>)&=qteGc|XNI~h5R6UBfaPeIHbA32@*>orZsCB4`Q79}A<ha)q;YIYpFEUx^ zaVX0?Z>=z@najfekt-_eTg7a}Mcas^D1ELl<oB8@C{n9M4tgW}W3F5YY*Z*0FBmn} zuvaY)&M>N6(y28c{ur|tmueFvIDOQxXs1)_lKrA`L2-^^VNC#miFvO%l6w5uK2b<E zsxRS3%DZ8>Fyu?hyNLCjTCNRRVW^i+GX``giwc&TpV~OHu(yN&o)r2$K$1kjh@>iP z^&`?sCk#?xdFX+ilAb(;I7<$BQ#6j*jKsu%LEhQKe=>ki^ZICepr3#_2#pE`32i4Z zu%eXsgL)3x3Q-^OPPRhm<^!TEPoek6?O^j+qLQ*~#TBw4Aq~M2>U{>{jfojVPADAi zurKpW{7Ii5yqy6_1iXw3$aa!GLn|$~cnvQnv7{LMIFn!&d6K=3kH8+e90<Z&?075z z1r_Q*zeWit1|lMnj<hnhBCDkKn;(q04?fKL1js!s(I_kx2CW46P*4v*nxWEAq3IV| z?)g*Z<FSyUK9noj98da^{Nq^lsGOZJmS)W@c!8P+C2C9pGBDaoui|U`)Jkd{ECFad zh?~(ybdHOyBsRbubpmi70eu6@qz1_S&<k-Sup<;JGoU0?7nBT`kocElgoj7E*ey8p zf()Iv(;lyM34RDlM8Rrdde~%}%>Zq5K%6YfdLv}ZdQmTk7SZ7}>rJ9TW)6>NY{uEZ zY^9PI1UqUFm|h0Vqe60Ny<Ali(d!LXQ2?Ng)gVS4vQ09X71=JDtR_jaI*exKavGUQ zhUmD|tb`09Ac$JbHUUQ9HZ#5`%SO=<HcJ7QEJ;j+QFy~R_!~KeN~3Jo8&DyGAUW(t zr~b%F17HlnnCn@sBEUx631K*7(@YRuHd_QGcU$pEqu_^TpNS<vW0mwm&>=wCFBtKb zXtqOa3M?2OEN=zDX7z}2$Y{2@WJjr?N`au<Ajm-(2?4uW;f9P`kW2z#jy;0xlSLca zNnj?uV1bZ`k;=c)0NgMb6Ig+iiNd29nW0wiFvx-&5KCkcHIvN(gx*HrH!|ihz@N?T zlDvqb%h22=_(sVltI;4=hz3EFe1c09-8QpRw3<Yh;PAPgd--Nj5#6vcFgZoRELrph z>MDVG9kSH~FjfJRNfsR@yJQp4cQ8zaFkT4>5XQqSVt5c}`-A#Z=3-_mGZ^)Hqayei zhJ}wgZ5UDln%)!;Wz@u=m(6C_P@r9*IMPe7Db`CSqad3ky-5-EcG=*v8J&{RtLJ(E zw2h-ghGYcDtqj4Z^nU7ChgEXO0kox=oGaY;0EPqeW89T6htbZg4z!uU1hi;omVj+3 z0B%$+k$`oH5*SeoG`Ay&BAA%nAUjQxsMlNdq8%;SbEAPVC#qm!r7j75W=A)&a6)3% zdQq$fCN;@RqI!KPf<D>l9l=vmBFSFpD1cAxb@~K-$ZIlIL3W}?#3+|2p{|vZVq`YA zMbx|Xl57kJVwoetAo+opiewCkCIO=uBLEaG+!0U$MRdReNsx>+PIJWN6dW)pfeZ(u zQ8ei-Ht69)ZV`qv=vmorhOkF)Squ;)8AUfh<7A_xI8FGHMRW>~%o`1Wt3|8IMrM%& z8)|@=#ssro9=f9HtN0F#O085{Bf6PJnurfzS_yg?qqszmnQIYDP{N=xqPfvl;VN<X zBtysuidNKlPkQwSxNO8ro{)JGpCejxE8`^OaIj}3iwQu?gHDAbCxVheQH5)!kygSt zv&+awwGSrR4<~LH-u93E>sK^qpoy2&App~Fe<a_!*W_?%#5a2n$z$h|7pw5_y4%?e ziLR?3)Pzcra!aM<;el=9su_?-646cK4wO&x;|PILf>(MB7KCI)$p1!&YEB&%$9gTk zmvlt?t7!>_paNt_fYJvw^~LCqX{4opLy!n)md7}<_s?`gytfSAdoScQWTy&Tbr&~( zg9myGVv)l|4-umFBL0)Y(d}Rvt11)(O4ij#zeao~K$vh~JDn0_@3RjP2M0|79T&9+ z?>Vx&M30Sb15&<{RtpeYUf|n7n5GHyc+-FtA=7H$p6Mh=&M0O!so)tze7#WT>pp|x zfWae>0++DfscU2%>|@oiCQj+6O827)1}KsN^a><x=wCzi9Fge*mxaUxIYx)13FH|B zMp5H>NSI*4?#ylfG-{q?3MMXX$dUH^S6Ni=Ve1d0(janpz@WqGJ?cG&sewpq294Qa zL{huwuoARdt5F4Dbh#?<2ruzSS{VeDAOtY+52t^xJW=!(0f3P&G3Cs^%~Q~~Wq{YA z!QrEk#>oXK{sc&Z7VB1_>fA1^#YyU1Ff<^9G(!V0!JW`n@EDdj$$2SVK6*7$!BvXP zmAC;h-W75(Nnzpro3CE9eV=~Lp7yS(vXnk@$<WbDju^b**6AB3nWy$#eeKmfQ_Yh$ zPHzo+4MQ3(pSUb%9V<q*FS>g3{R`!(UG013==W*Hj{-*F!ujl+np%IX?E0*I&-K^u zY1z1I!`iOu+Ll`UtL|F6Vb?~vk=x9w6}eE^*<<q;?v;04Ul#M%*04VQBMm?DQ#-8R z4ja=5yhXO66^La4D_jmP4_(_JlBqNYDFNk@`w>)O?pZQ#8YKE#b($x>w$3E*F0Kfk zfnyCo#zOpX1(P2yeHG@f<zm>P7}}~GB|&S27%6=@G^V=rmeTB$(w9rC6J@uQmcAMq zQ=Ce?Z0RkF_gu30<;5#jEW32il2?}$-6PZ?au16Y)?kUFy3L?ia1A@%S3<!frNW<Z zQ5rf@nn0pZ#pp!YcRyZyWbx8h7Vp`+Y{d%XJo=XA1?FlRYgY(u#5sScFu0*BmQji7 zMsg%X%1Xa`Y%qi6^2Un<APnBiSo`$ezN2F<e~6`zy$grRgTHMpbwG3e_>G-M`{qn8 ze+|6jh0vqfkhdSb0MvIr!;;*AL}QX^gkc+q0RJ4i9IyOo+qAyHbl<c#_9^C%X0HO_ zzWRasvG9WWu_wJ^<z<(x6ie7m+8V9y=vEiAM=N6NCUr-QS4GFT%{MJT^P%PG)9uk+ zt-Ap~<uLz&|4#Bnn%$V=AkIO9(hZL|f!dN)FQ`^}MC5BLmOxTGVts&gG1uw1kwNFi zNl=HK2Kpz^(sXA>I+$VuZ3UT7&iIG7640a)fe&>NOVU@xZ*YE`oy!JGMY%j}bGq!= z`R5xY(8TK&AH4b6WoKCo>lPh6vbfu1yYy0<seAE}gNxP~<T15Nue<r+;&le3G@dU! zVRt!g>2g^t9bDbexN!A`*$M5`u&}WqF?+*m?ZoW85&MFmXqQ1J{i;_Oz>3*#0?lWa zf?{tv`_JzP7D3x2gX&ICRn(aR$#>;ciH#pO?<*}!<}cYh_r{hb6*kkXSteV>l9n6i zwx63=u%!9MdE>@2X)3$YXh=DuRh~mN2b<cf=#Xu1I#^T14?_P$OO?ksHYhql7O_8v zg;YvixTDu7NO1QGs+knK<B*6=1Dg=mFIvVbfJPC6pBExaFfK~Tt<Bm3jc^~(qjMQc zYZI_OfJLd$CPJf^?<K=GdcOXXMkqT#5g{0bgdntm?vAwjCac93i{taGLroLXfS{`u z-_kB6JkTHoSU4AZ0xu&E^_l87)i;Gl)Hl@!5JRt@+{+OWM_1iD^iL_R-C?vbAjKb) z_sRQ6|3ThU7;c)>QFEH&_nHWfU{q+4<w4yb%;Zj0qteU!?%j3GhM`Y)ynXE63x%`I zlbkkl_WlK%`i{&MjguFpmQ2omKUi57FJX7WAlzzJH%;CNDZMal!|l87-5?q-y!+Vi zc4Y52Tb+~4!rFO9`?k&<`gHP=)S}74v_QNp98x#ow@x#=6Vm*h$ZQ?$;W-_+h`Bpn zH%*K2!C=D&z*TU7!vqZ?^yEceBccS&2esJe&QDZYTGYXxC7t$|)qgjSb7&g5G@4sP zHvzzM819*(u;*fv_9OHM3zSw9Ata{l<ISwZ?8!U!59+8#(Mh@j?T#{perc<gk!N+O z`lr%f-Y5AwInY7F70{L<{V4T1>=t07pt+Jfj90Or;6JX{BCQrE8bZe&wi3fwEXHRp zz8{VAmxsWU)3nT;;77X7@GCm7_fL1p_xKEG&6G~luO;Bc3ZIa?2b(*uH7qJ!es71c z{Buj4(;Jds$o78u<3df_2~DLq`e9*$SGmrR9p<NOKaBabz7+pZ{Dh4lYirHwpm{HU z3hd(!-CV5N=j!%hIpDK~3twBB4GBmeS|AzWfvm6}DPEG|lUoxlypTiijSA8Y1hy)w zB;2lAnwp8))Eg48P%ZwwxU1Oy^A`3^oiKhvS?$)KaeS5HoZ{?b^VT+X2<lP&*r`*; zmX$O_mj>2OoVB5Q(KL3M{1>eq+;+lHK9N?xvyBPHni<#j$sZK{QrKEcdR9+eQD0V? zGPaq!#<-c#a>t4bt+R#Hu_|}dlIGeve@SR!d((u)Ga45+BuhHfA88G0cPrw>>(`ID zZ;aIyn|qmhuDXBthoW{J(WN+`Yud=y(wvd0rm&1*4<oZAz!P8_BUcs&>>6?#8&)Fz z&@V=a0w4)F{^!&W_l6<5xg|-0F!~>aCALbeVsZTd*)M*^tr*!)O8w)mzKThWyQW@X zw%BFs5_@CIic5EPcTJu8=CmynV;``)3}gJ`Vl#VY_3Yib@P-KvBk_%!9OVu<T3_4Q zJ}Wh6Uj5jGl>#8tG|Nc4I~A>8ch-~X%M@!>yk~ERI|QEcwzgI66IaaY>gx0~lm<@f z5-k^OY#SGC80Yr-tDRP(-FEJ{@_4LHsGJ=)PKZ@`eW75-r0ylN%0Q>&*M;@uZLdJ$ z)rw7Dt5ajr;P;~1P>jID!><(7R;w|Yf}qI&8klT?1dTfc@us5mKEe;qw;YKR(cp-D z6NmUMP8x7cM%~ytE@l*Mp^oN*mCF`gRNhw3gpO1PVi_^JzCJo>#mX(q+iJ(Ts$5=! z13b4<S-Zh5EO}|}9`O40iCKDwXte9Sh|o8C&bIT8Jh*Fj$0VQARU$8Q+egBv%Mn3X z!Qcx59^alanM&*z+Y<Ues(;!yzdbUx)E%iRZJRjj?gi^^S=Qd=jY1Hx%nT_;+-3+^ znMLmi8mcXddd*MHo7*sD;)HOdVeFi_yL;|o^NyEPowy?3H!$HRn;jju;n3DfLNQpY zqxhdU>5gILEULS!=)SmZ{<rUozc<?661*oo@wI@|+#<T*dh{>qsC1)$8-4eADGR?v z>~4k_SvdvPHAC}=4(!I^OLgQ@9EMDE7d$PvJbi+K%-H<G2&1&Ug-+g;!=q5Z4@4oy z0Kx|tUslEKy@~N5qtgl&^V3QJCBYfa(l=Oa8&KA2+$T;zZ}2m>Th`P0#Ea|Jm6zj> z?R)<Djt=~Zzso(z-kZA+zt7)&vz6Y(tF82|`)4=v_sZ9)L;Sst=d){aPlWQ^1%{oJ zi(g9gzf%{`%^Q&xF~Wi-7H`-s2S7+tJgON)@FNH^jqwAO(+Q(4a<$<K<6&oS^_MJt zug&xmldPHcv$;ma9<e4wvW_55HYFyk5xZgpyU$>(YWtZoIRx>AqzlG1UjT@6ba>yE z{Wf<5moh^-hu;ptAtPG}`h$4PWcOn>vy`#bH#Ss>OoAEE1gIbQwH#eG8+RHG0~TJ$ z>`C`c7KyM^gqsVNDXxT|1s;nTR&cCg6kd<-msrdE5Ofk=1BGDMlP2!93%0c@rg~4` zq)UFVW%s|`xb>;aR@L^*D>nkSLGNmM?cv)WzHZy3*>+<LzK@;DO4hrim(_e!_Jv%d z6Fz%J<#!SjuKW1t&KC|$?A?EL*Oq(3*05n5<oc7&y8YJ=T=~t9mQRZAT3OLJb?4Gq z<JC3Ox2<CP{tRm|3f4YSHz0Kk)BX9{xwYV6xysQ~=ny`qa$Gfwx@-Q!+1jOkQ=NBf z$CH1YzW1rVEl=)0t9HRWn^!W+dROBs58n3RHGLgaIbX~8NlOFk0<NbEeT*+%QTbS+ zjV1nV$vv0Mn$WxVs;Qgq4$EeHoy)Iut~mPU{g*%V!SasEUCS%RPua0}=6H48hC7EH zk@-H%b*K~}o1*GACtx_ngO^d~_>*xAJSX;>))*XRT0r9<#zIpug(}{rSC9T$42@gb zy8eb6)~}wl<=or)2L}4T{vum>-g)QaKjtnp5fyd^;|BxHt<f7_xUqfX29mzKzMGPr zFI+W>x~2W^YbKq1HfB7@>Hw@U5)?b^H=uNOpli?w6O#~V`eG;`irLcC(&Uxz`L_Cl zS8r24e*U71o@dV6Soupo-}Ttu*Dk&EwY`h4KdY-k55DSqR&o7nufO)%>%s-Es^5Q_ z60#cReEy=$4|nW)bLh=|4bxW4j}A?qOle+wjn88oAeYb~!eA+EQ;8Ggp-UldAt$3M z7*E590amz>YB9L(z?Xx&?I37XYw?Os-t+05x6Z4vkzBE6-hrbB=GAB?p{DQXV4CKg zls@_wh*&XC<3R(CEZxg8*Y(6a>cIOq9Nss7{=UQ7Nv%O_WxSyBqnH{@(<>A&2on@z zn57W4Dh*E)o#rJ2#tyxV2;C5#rl8%%As$4qB=IbMt-z|jnWi>>7Ymq37;AW!6Y4nx z1Ogx#!WVdA92mEipgUxzy_?ddg|x)KOCyK)P5v@usc;0sN3{=0slt4CuwaxK@20eO zhdp~Z8iJ7GWrkq_-X`~(eBpthn9|`tZEUCIG<FXgMLh*(G&pfz3`a<Ki+iFUHjfBn za2~?J_QLeBaKDq%gZ)6pk$MNyiuBTm!Q>iFpJjjxPVE9I)#z3Q$3tw`a69qxjuf+~ z*?v>d5~pcH-AQ~0)8PyIjumD^?SM8!<YujmU(k2xcdb41eaGof@IN|TkkCj*=Dp7k zzhR+W;JWWvD?x|pV5R~OT8J;>Wb>KZoD7hOlc2nA0_(eG!i<ZX7;CzC#)NOOnG<T) z%~G#jZJts)34rm;-#V#wig}g#&C1U67tfP3Q`U+Thu$ZWLky0O|6_g4*zx1X*6jUV zf-PHcxJeyM8OD}Xx!qM|V-2Yf12wlyp1)-c@AJp8o}G&;2WSM87$*7@|BstjLE%DD z7+TZFnkeFS!ZgGirKoZ~cDDKyTgx^t5f*LO{EZFLP3qGNmrY*kwWv>n>}Ru}>6)>5 z@*}T`Hw{I^-?PS9>(<Shea<;U?_^&P+*@}~YbtNb{)*kiJ{~vjaClr>#UFBQpW72* zsfj(2+_9@5x+57aN!`e`f(Mp_I(D>}p8)<OpI&$A%j$b~e@p$vc?M>@&g^g+X1%d{ z%X5boE<E(ymUZ8oeqDh?|B7u)x#ezF`m$IG7~A(=zVzmU<`Ux}lVD!E1<7B<TQjB} zDv;b?zVQNp*Fr!99oApvx5%a~lE-!>?hEoj0CiwTh9)#8^?~;|wgor_=Z1BI9_dI{ z&t*f95n?ZgZ5CnQa!v(p|JT?y0%KKgi`Smi9k5r!+!Mkz=&Z$%CFl;?AOzV`YBKrY z0#Y6~J6&dA=m>T@TYb8ukaV4z^Z?V<l9Xy&tR;4CEXGJPR?F}$JJbTm&uNY$Z;Q`l zA~9RF4PUi{@O|hTC9i<b;9I9AMX64c^uS47E$WgWDh~7xpBp6apy;in*&vIzoIO_l z%#_v!Xgc82N%kHf4&r$&VLnCPB91xHe&j#p%13@@WOqBKk6E<SryKR5#s<U2jx|nC z&z*-n&N^Wo|0Vk(KMA>X*MCKcp13-ye1*`gAj_Tm@r{fpm?K!U@Xg2AfndEo6jZN} z=XK0GRNXVLW2c?}B)rH^yR>u}b?|p(W$!TkQ<TY#AFJP^aZT$28UX8pkM=)U#1Ick zkn{$OZQ@U>TAgu1AIG>MFfNchMQB_^-AQxRE$Th5-E_tBP@v(Cy|ojjP5LEU|JrM8 zVF5;$><F9!()BUBoBfI1Ev11Nu>Hl^jlHWDPChrTH(vh%bARyj5#TPb>omAs-)4zN z9?9(wybd0$Z5s+}F<gh{I7;dJ925oi`S=N-ZN-&{uKtgE`g+faR+iSa%%AgBX1?d1 z?7f7WaQq9<737Iu3RU4oDZS;1O^fbmYg^(`D$MpRPyF(UKVA317eFlx{_~4b7WVFW z4eJ9tsT22MAB|u=g&r1D9P4{hA#ZpnClm;{NaF1=h%u}U;x4x(`<|^#GP_*TPt`uj z=(f4#e~|)CHd6`8kFXn{pBI1O2@LKn5#&;*SiO;19075H-Cg1`n$_{^Fu1U1<?wNq z7VbI(zC?}<g@X4pJ|oJeB~NN|kv!>iytv}-8U`IC<{6U2_NqEAkv;7lys5Qcq3EKt z0-!^Xy3idllgZ~qX^QTe=i*oGUCJNk>Y26?+9U(Ks|C81S{-v+6ebc`c(yibQbuB% zxM7mk>}dI-TfUi5Jqdu6b`4SqF)y5humuCaHhssdcR(jKf5ZGprx;Oe7VG#G6TA1+ z8oZLl<+ey(L+$Qsck^4fi{I|)p15MX73gHFUU!l${lN{)Ht_Wb%j#<CYLJugeD?U3 zJMP#*v}NP``He!iC2mv94Gpt9%vE}qOJ8N~m|b|?&|p?<vhXx>UE6cZ9}Wq^>+1wz z9TBA@%f~tby^0YWafmn&8Ppjn1Ng{d;S01WImtMzV<`!zU7;+8e-Xko>qM^OfOZ`Y zEZG#vcm>EGF??&G6+v(3l`X(xMn8ESv=@LdMfdcxFi%g1?0HDPG>blldR`OLlWN80 zz<$t+MM9%1K~JT@#aBZjOu9*G{W$u7cqTM|&a1)0wR8R^*r$<&AhuCq1Z{-aUhc5P zdyaaK{$P=Y6R{40FrWmLbDOCijqB(1PrKlnL)Tm|t=l}toVLAZOXJ*~-dx|_A&o65 zskcpT@bs+d@ia`f)t8ivl{(t%H?O?;=^s3O^GXqopx7E3kz06f^UQq<>gyNmo4Ij; zrOxuzn{WOqP75~PwPXC;3mZ#YW1xy&DEXsl~)u4`-v_{*B<S!-$Ig>%R6xNH3* zJElz8@d#i4`#JV(ko%x;u{LMqLEEDmwD*(ccB9Wp;u*9I?=sC7g>%L{%$4m#zhbjm z)gK{LWQvE1>_yl|4T$nYKNVZ<)vza7FKU5*W~4)KNgN@;SA<9&ERxIfA&UZnB=r%N z5YD4fY$9Mkzy}!G+`KU<CJfhQQGh_2W0A_!>y>3l(FS<N0~Cz1mx58QH4Axd^9>i1 zw)t)*w$E4#ZSxfm3cZLC(o3aQQ7uHk>_@fMTHoM0=quh%mfN6%{`O($pyzg0kPf=2 zjA%M7bRl4BhV5{{d4HbnTh`HM&<gjN3kv%=W$Z`e^_r00Y6SpwKb4mCD(Z)4C?tx= z`KX-#*9u+lh(<?iI~uNH65?WVcc7IES~!dVUpR>YKw@N~47e7NFGr*9<B4Xn!7P`C z4LdYduN2qJHY~A;Te^J{r$gs6V?xP6HvM2p-_bDKd%_S{@$<BH$g<4FZpiB`Y9?pK z0nj}xQ~NJDbm)@q14l1h<!J7H(X(L^FvWT%ZScI<S+QkHMQ7@0-^#KYn&mZRD+x7Y z<gR;nrFlw4bB*Fyb>Yzi(7XQl-FJb4hPEKOC!K2x$nWy>8=PJYE)T$=Cqe(n*ChZE zklF{Ms}h0Jd|@o;Gz(~b;9d&c#0O^j{1?tF5dtMj9dG`|j0qZi^aF1r{<7KC5hZ`E zNX2nxJYEr@>u86|tPjTDet;fLn1R+IOm6&3b*}TOyN<p4?y-ZOw+?eijq)nceQx!9 zF1<VA#!XeZ0HYw?;kP^o`NQkI>pIaid@W9c9!jIfiJOgK-aw=xb5Kpb)`E9x%CU82 zEQg_v`e+tWYClJHl=_EsSW?LZO3)o#ox(#2UW9|V7I8fYnz5fRtph`u)dywWL9}UV z*hdU9-BBK5G&}j~O6&dSdWDIpFX;&Or5wNbm^Y+A-x6(K$$Of6JTVl9n0gFY&=T5p zZX?pCxA&w{J)eDSfb?Zh*LT#AdiPlB;A%p<Bq7W(aa{j)HJ#0%)g=?gRG9EDg#n7r zMdJ!BkA1LSHx=!UeGUUkV_RL?km58Crwq7P90nCp33RRX1Z{{!1>|-`Aw6RP2mYTh zLmL~zM^VS0V@*4LkOEG~nQR)<V(1f^1fm5Kg+gLui1gBt^%q}%mf;vP*kr}<!Q*k- zya`(hCk0Xh4Fh*E>HyRB+;*KWli%QqKt&%16HWyMXRhtwdCgyoTm*5#itgp(Wap66 zyr-dgKgjl&t?JLMuw}!Boz)TOa2|37p^FAcPmxX0apWmfp$B1WF_@-dsK+?1F6~yY zEwi!-))Q_CbOP%?p%bx|=d^nLBig-_$e!nh19^Ps`s{SNq{nnW)V-qnz3y+Ipd7HS zsb}z<Esk5=f)PZb3ECG4Ig2px3-625(}?@L-;2{1-~Zo!|BJp?Twmxx@PBTJIR%Re z(MX3;x<{^v+Dp+kh69X1hp)wg>%!+}y8izoy>Nyyj4m_br&8TGFcze#gP4?v*NEdl zzGBLM4qpvdu;5vC<mb=7`rk^Zp6)XvB|C8ofsPJn1YCh>Fi9^zXU;sW`>pPi|NFD# ze=<sQOFyXEk?~r>$xI@7q9B4WPsw4CAO~UJ(S)s@u41E>#9<Bwg!2FX=>D>!?<Ced z73<(hU49huS%eXpA9UI!*?DVYlclgqX21of{@2Fn7mTQ(TcmWD7VPj~nvW|nqnT9o z_Hu$dLq~pHKQes?&k9u=h=b@z<!}@<-D?LS1LdV~=))-4T^jP_oWi>=*N5m$%^0E` z<0RjkAj<jntkEX?s8M(5QB5~HybXimn$xGg2;HpR_!D@+==g&dL9KHY`T*@S!<X=g z4rcQD+B7W#LFH#kD%h`SarmoqNKGpkequFeALTMSL>02TN9RLX3Js+GArg=Nu>E5z zPa!vMuMV06#7$1dLbwv+VGT(5V_&A~Uy3T^+|y~Q2>lA|=hZZ)ex%G`rhkN54C5gq z>w?qN=A+LgB0-@s{OJs7Da|z%dK)uDH4?m5Y=K(N5KWL)uqDxwBt>QmOk(h~1u6_s z>9x>G_+@bJhBQ;(Rr?20>Tjn}^Y`|rQvI3Ua5$aGq{HFf4BhwAFVk2oHNbk)hmAri zjQ_!g*-c^AKM>A@je&H)i1PsJ5929F<8bLXvONK<o3lUBL*4;8FM<`KiLAf~s-eY| zNJf$okV!)Nq;5_N^iCk1kG-1;vw^Uf38%>4;-n6d;Zm7Q=G|k6Fp*AY!b1a`eoS*c zF413z6`x;!NZV1k5)sv<jba+WM9JofGjJc|L8)pdeoYe|k4D4$WS}2J^(h1fok?X! zE)Z%4XYv1x(%$CyrIB)JBAwwP4O<!IiiE;x6RWsfYDOy?eFNXpN`=plE}4)*S+r6W zzaUH;T`&Q3_B`=Fr6y2uV~C^0^dDAB@;H4aCY|n~aG|*ou!U*^{LNm+MWflQ?q=7q zL#KWGC{Mr>;-Dqjt?t&|JLNGSA2yWhU-RYC^oiWI1+idw;6*>m1&Io`^iPgF6c$sN zw9j3KFYs@%*HNz1Jr?F^RiLV%@DyQ^Dnc1h&59pWKhD#AMQV~3k7}>c@gdw=dyRf5 zHGNU7bA_hHWUnI-9SXt<YCy$e7oYYqZ?Ac}M^UZ|wQ`N<U!h>jM~LT<QF!MTJVd$9 zSl!dKf`Fc+mMQFWv=bHVG>>U5!uS#{<o!rmVUrdOvS5f|Ako0d4T|BcMYMo=njwO^ zGEh(RgvG;oe(6DQrrzdx&SN&INu+*7Ir%L3avjLdJ5!%9-8@#0JgHBp?zx%uB^qC> zKSOhB>l^nUa&S8kEFoAUIDG}(Lr#|uJCGb%29Xr>1S4yk0d)9hoJ7#4xNbi?5Dt?N zBp45evje1L)<Z`vghl*}Fd&pvoJS^Tv4xXtwCLD7dg5ykS^`N?DIMh2T)s%L%gB=+ zYcANh{8GyKUaYU&lMD}eS&0PX3oeL3xe?Hfm7XTnPoL(wP95wAN+|#QAU-em_G_?B z{o;0~vmZXE(4z||kcNIM_Kpa+m7&GGMCbHkm!e`pc7Xz>A;&Smy9J8MJe@<k!l|t` zQtX_c1^Y&>1#HwBFoYPv$=k%GOaq!kd58)tzBI~EkGG3Rqy>GOTce-p>jH0rb~c(K z1|9q=$3)Vdgcwyvy&>S3p(f~O;~?XK{)Kch&2!gs=%kNH#-Ee-i}S+a@DNWR(Xnv< zv7kIUUD(c?<t&Uq)Lc0kt^(j9G<@Q%>RS|JmPeXBC6cbxUl6qRxl;fFAiK%!>EzFa zJ$-mz?G%WqC+P-l!DLX&nfxzGAnLaFsOg^Vq~gaW2QQ<(qixj#J=;Y{m`?kHkfO)i zdxQ*`2Jr3iXdj4QE%|AlQ;|<Cw_QLv^TG%)Eu3^19@P*teSQK_cS?!3_b?PpOrPNg zDU(eVmzZAI-gAcMRBj_^`IJ5bd6Nm=goKJl^RGSUYd~ZagkpKGg8Z3e=p=q^^yQxv zpS2Vudw@jKJ_msT=JW}ZIK`d4o}Z*F4;9ciXQ*S%!8L0RN*`;#z6b&A_I}}tgR3d( z<LZMyL7W7{6j1-2Q|E%FNYCC#8P1s_{s+CfCi^w*$HUwKLEWjnd64P{IyPrBV!<6$ z==o8~Z6OPXpQzb3<}4kD?Gz!y<?(BoM})$O>Wx~pKrr7xu<l@1AcE8Eue`3NFwDSp zmx&I<t93eDvK;MPx8tjKts5Z2EFA(c=_s(R@Gtkbu>NnTe=t-AO)iha6xDYpH}>yZ z+FD^H2VS0x4us;Wo_95^kElZ$>j2HW@wyeLi3i%Q28NXxQT7V1{iHY}Llc~!Dkv8* zM><6X$}-pv0N#?+N%W`5%}K0Is%8kCOC~LuR6+;gtHYPi9=dqUoin~Q<Br;k52e-} zxq8t<i>^MhE;TSIe$6dEI=Xs(`oTlj_C-3c4K<znz_wfNzxCkO2evhZu4P+#mUMS7 z8GYEd=OM4f;(ch(iYw>T+wJvpu4Kkn_RZVg5jE+RF`XNx?0xmaV~bW?v}wVTXn4{5 zO&2X+*pF%!%qu@3SLRk-npU5?`f_cV9;|pa#ktlD9VuvRx;TK+fWUv_$vC8-@TcO4 zN_-D6?7|-4!VWMEgQ}TUe(c3w4{eyxe8C5t7pS0MFe;X@U&B?sVDIGR;u>?mPyb2F zV5WLiQ2mX&1v=E#B`oe9yk4Y2^CFRk8*rV6<p~YyuWOHV4Q7S|KY_E$w4UX22xmuY zE7aQp&S+f)+vfXTKUGt!$|J13W!l#|C#tu#OoO(7zZ*Wdx_^8_nA(-|tDKkc!0r-p zIEj}+VF`c`Mm1zkg^G1T?wp_6c~C!{vq1>k1!uW{m47&7E!m%(ANz&+i<mJGEE#Jo zx#pp3N{r(g1FD(NYXN*a{;@fm9{Erm`tXrWI5X+PN8T^Oeqb+Ma_g;^AOmF6vwnR~ zFaWEvSNQbi8O+J5^cuaM%LdE!8ui%A1@rdwb@Oo;KWrUv)+CDynmYfDx)}qF<8rkc zTM`Tysjh1BzgM@WlCjQwwT#BGC8(-k`k&U#0u2En5J1b1YS19#^8YVirF1-)1&;^s z+jLNk|MzO5DFxY+|E<F3=scXXU557eFCihx+jHdwN>xrB^ng(;#RLHnX%tfsjJWM- zyBo5Of=eNl8*;gm`o<fXBX1wcyrDvK{Dzo*gJn*6hhE>zE0weGdP7~Iz5$$pI`$C5 z`U46T|8cnpt;J+VO?%~H_`Ph??bcn%Jzu`2`z~tc^<e+UI)xP&$n(S3h}kuKjWp;t zq(Lp9T^N2|l;#Vc7k;bo8XAG&cR_IEuUn{03Tcc+h$jxoz|ziMC;p|K15W(Q-9eu` z1cFiQb!LZu=gCHrkGjo2R&`o#0GQH#*HO0HDgV#X21sN+15zA<W#P!S^AsN!9J7Tt z+O=BvD0h=PG3_TO&%mknQ!PGrlK<&Yn*QiRA|0vw2uBkIz<TaOe7t4oD^e$8O~BUa zBAXL|;{||*8KpMNX$SM(afGbLEi|#IQosu`d(HdDVR27CeBGkS8y;Ho==+}>PoA?r znJlfFuxIeRC?a>J?C!EC2Bn;dnhn3XeZ}<B)b#BWH>sbjb-10*a7A?aS00$P{m0wm zO_v_`nJOwO*k6S$tHR@xmt`N`;fR%l>^^ZvbfRm}PUBtryK5pTwRdIZgj<#_irORP zr7I?yj7m&+KkD(;PKtLXmF-s9=>`j_AFjI$YN7_w1g7hD(md1~ysZj9;u_Y4i3Ssz zgRH~g_UH9AHR4A!67Z@2zch=Odh*4WzWc2=ekK0-ueW&=xy{z7Gz9CSbv}Pk+4ST# z#ZxnW&!Z1tS0A}`@LT_*wh{sv=f-Dy+2cPoUi{nzYTGjx)eit9s#G5^D0+(|iNBlJ zV$vUX35MrZ8K19VAN|i75_}Z#DO`R~MZQy~2$6gqOvN0Js%d70SzJm|ER&Jy5k>-I z!fh9^fC*zr22w0EG6&Uqo`eqC7_L8gi(#?!A>;y86ak0F7|oHQIhmW!15hHkZ(*|o zF+vd5r!A(imA-b0<VBdKUN8Broi>}qc4-&FS58}j>!?PW$SEg*;W8H~a^e%b?2`O8 z*`i%!x17FmIo=X;^83K2Y3Hja(b_rMns6%ts^>=(bA-9V<9O1I>564?R3a}v1yYtH z*l6T7AY0T66-95WtZgaP8(}|MBGlfNdh@=~Y1m!IA7($BPUtE`qT@h@;M3Hd<i#;> z;_dtQw^?1x7-WaPK4XDxuqd5+qVz|PQlALGw|x}&MFa4RtVSK`(e|RtFN=u%s&M?) z7+HD3$diG_iYZuX{0ijc(*2C7cTX)p*3LRRtn3r@wq>%<@A9jY)yX*d<YnruVRt?I z$Ge`gQt!J6+3gm(PBfc4*wmp(6%VoXQ(238uWF6NtRH+}jYZghv3233VHW%Om@(>v zSq7pIH0)jCA$)wa^7RfPVlWXzzoH}vzHmu4?W&f|zEC#fi<;dYS!Z*G+=!O(wLx7} zkfS~!6{@R-(Uw86L(mJl7`6&&tfK<xNf=^6_XCUu8ur|r1N#l=VNU&`*4XA+3xiN6 zAyXvFc8Uio)C-HD%2=FzR7&R+I79uTG#BFHe_)iFOrH+GE=-4{zrkN`W+aW+i_JTK z`>Dx<)c+WIlqL)3pSX=7*`N5ysyr`8ap$bd^E3w89)ZgPiCBi|f{Ji<FG*G-wEW`% zb_kg0x8xoGG@_?1|4(>^U)|AMCk%95n_gVk3|_XmE_Z6(keo8NCgI|@0sfZs3_s1} z$KK|ZCF;AE#cQiOrv*z^HWTBHM`H8Hwdx20FDq8lu^{(Q!@5s%Urrmi_ZX=7)j%7* z2x#|wO+pMI^e#2DpLkU+erWUorFxiNlu1s>XIg^5wIEm|<NqYAKz#tgiff*fn0f}X zH6gqc8Y>joek2Rd2IsPtNkBRLQTFsnoh4v_<(`f@uV0I_G*I9RD+?L~j{1bx`#0ta zEeZiTNBzhh^|GEN+1vl7{w)Wm!`yhLKAuC&Ve`GhjRo0c|E^`tZXfkQW;&_kBLS|M z7!XYb?!E&&=u`h5Ld{_dyivFMQHW{aI!yVS7oS=ttZ_4U4sb{P=wmO6wCrO3g8Cir zRxN0ht{}^=kNOy`2fdgiLzr_8?$^fWMSdbcHb<)&+4+$`i%$>mB*aF7fv0tiFWhcK zRThLy0Mtx?A6Q34Vn$tJOcHkv?-ldg8_%9Jr8YX#=C;}%u*pWq^?L5VVi61EUkC^@ zTi3LAgna%bC9aB?Qos0?XlUZtnp9cISx)1AbGeO~JGb1<*DpHId@iRrT4e7+!$h07 zWDZ4FAXQ;*hdB%9)8U`#Aq1XW1`G)sm$Ol@ZCv2#2r5~I^BXuYJm%NgOkCQOAufat z)Mo2&C`TDc7EDz1sE;V{`=Bx<#5gYrDb+@@FE3>Yx=pZB79-7UjD-g%Z#qc&td6cl zI`S1u2Q2b!m^1LOg{LEV_eV*@<A9-FlOjf2A{_Fdsdcgdw<b}6Vv(B@15RD*qXZab zGz8+iNI;_yFaPJU0Fd^84av-oyHqv((`T<kWEsJWhoZ6M2FCV3^Go3`Syhyp=btsd zxqVDigWb0^7+bvc{A<V0Sur;${^`L7hiXh#(O?ohffEn1D04pao)ia0rM2e0htyBh zw}b~@4wbr7eO(CW9*K`nSVF5~Wm9&qZ(Gwov9=@9gC3;1kqw!61?mPZ^p~ldeCE1| zU%qZ1WK~HVUALZd&%bus`VF(CFRt3JUk=r?Q)@PLYuCi?1XT>cFW|i{!+a94itA#8 z2;?I%3?C8LQn5B+Ac|?$1Ejde^`AH_B}3`>#H=np*@<fr*CMXd1TQ1CRL&_K($$Tr z#96|b_DJ2yxA|JS1veOY96cL$3I;DApFr+o{Tp$trocX&{NOP{5|u43!pI<e402%B zM4wvdA3SVLdTD48)&_a0Vbti;D;JEJRo@*uKg@iV%FDJiOj;5h6Fqm~l8eh@<qff( zwMR`clMTp{LPbnGy0$0Qfae!4?Ku~zmP|_hX)FU+Y9Lxy@0&2br*_dQ_S8c9X<y=w zgbcF7+%h57T|aBgf|aWl)%J{^;H$5T1_TLMnM~3Zy_ze@gcd9{qrOIM4%wrWhI}}| z^)eJZg}gqrX-UgLOI%0m0vFBe!nteO62g2-2>XDR^y^=fZDd~Fz;wS>e@!M7JaPvv zPU?=U|2$6iw_+;&j{0oiARgl1!2p}_PMTg!Yxs?H%{HmJgU62_ghA}_;}{7x*brZc z@>!rSz|M}1YPdKizI;?B3~2O%LY`8A1SF;-m<K*j4Cd{v=p*LXNG=-ER`vGd2a~P6 z=bVdBB6x60efFOZY-#f%gsmbnc#&DXG<(bOgPXoCEZDgD0<8iqLb-Z-`rvV;w+SF> z+Oxu{+PYOU-V9O}bVd$T!;AU2M<2*KtciMEC29!H9V-u9ZUJ$M-4<ShIr|tXrKWbT z`D><Y{dE@|XmMIgt<IJM7hSi$OEY*8QbKz0rpdFP5g*U$mOXsszD2Ww2EfAxXD!-y z<-^Ogg%RH)dPz5o4-q9@fFG7wBX$sZ>#Nb$5QVy@LP8HyfiyK->WR(e1g77J;isq@ zxu$>@C(@*mf}RY@L8hJXBrWMOEKDqt3i8iwFSwpR$W>G_j=iMN>(!1>S7GdmXt%UH zpfdn%XxP3S<>d1=1{yBn9c@?(YZkyN<fb)6_r;Xwbe|ytusrg}0*9elmW$u80{>N1 zQx^M4-32#mo8SKR;r8t_CV3=RwbSNzS!Jbd%GS0L=qT*0!ERw05x~DzSsUKHYQ||Y zuwKD!+2nux!l3~g>0-F=;qnW{w$F|jqXuhZz#N`4WtzLDj_MYvu(*X@fb3G;s!oPE z?QMW|e7J7#=?C#3QWQRp-~(1;_=?J(Y^}oNmHRoN$^y4Pv2Z8cL)EmwWVNJh@>2ER z)el6y-IQ`!2h2{kx3}jwTf$_!N75)(mi|n=?Ylj_>QzqjfMiO67Wc4{rOcF4JS+{j z&z%duf1`r(U@ZlI{F=sZFnCG<!~p^JB1!~*BFX@ML;u<X>Jv}cN<(cA|5AP8m+HUK z@vG9%#_zOu)ChxFSxmKsBSSO9XX%g4SU79e4=G!|Cgo(;VeA8dsRxIZ$Eqhj(brh0 z>Jh)P2`<<#u_i^?L>%2jxXAxZX%?<7l073C+~1p!t{Dj_9ZxL$sz|_G{C#{Hv@t=B zP}EsMr62u$;U#=d%MRJHCiNv=5OI3(_o-A=G_9B~AsrRui@pzUDE@tHg#6PmWEuT^ ziPt|@8=kjTNmkqdOlyJS!m{E9I87hqn;%9rT0<0-L99QeURoy<D*!&^jevF<)nUT! z)AT`#`;NGXqh(2qEP|?uDsodxOo=2AYe<esea@j9#tIvJKYiZzn2+^IY;$9FpF?ev z)9LIFvOg9dc`W;v_gY$RQ2(-ltrx;ivNunxyZ?UFkLj$?i~bkZTu6LDXb4KgAR}J| zCS^>K-&OxH^mcao3^t~WeS^K<Jveis^lW@^CYHAs)lZm(ZMfyW``BVOh6XH+7pDP> zH`XC|VCLo6*duA78O!ugN@5Elxkhd!CmdSX&*f=utfmDFD9PkBHMk3&aFB&)R8NL4 zD&i)OQL<Xj-mUIXo7Ek=&j#eS(SsGU*Jm_3HgtV*TZUGd%*5$1jGR7EySeR?t_=>O z(Z_o2Zs~o#^$zu`{XU~$I{T&vAH3;ofJ*ZpJ&JR~s{J0}8cw}`t#a3NvWA?#tMY67 zLG}{Q{#6^CipQ<Ze@-_<(SF1-XcPlg5Dr;dBvQ`!fSM-T6Ddr(E3Sd74(BL?V2X$P z8c~UHEZ`K6=k+XAgTAh2(4>$*V2|W$g2v->Y9+4=(K+K`;I4$BFUb9!Nrk0B*fL+v z_lcdO1uEs@|8I@xoKCB{68@q=)}90JCVF33Lb?M@bC5mog<2~vPXXzk7B$|75Lya& zL)t=%E&Pk`S-PznN<)4iAI;NU!@f0_V&wOND{4!~b@1&pAN$<uq_`Xeq?hMesa(bP zh2<#7d$>Goqzvq>;o=lr=43Xx{tUtEaN3B>CWZ)Uac%%Y9--wFCA~Ek7aAC_APm}b zpXAnlNOIF+;t%pPlAxIkvv1neXa8*XxNLX6ZDDR(+U5bi-=^>US$+3TyUFaf{gSPI z&A@*!TUbRQ-p-3$KUDc=Hp9j|c+t%)Z{KNid2DyGia&p6lgtpOkDeM{Qy=)H&22V` zFBRKM=Etf98a&;o2pD`R2ctkyWxz`aTDZXBjY52aOspy*2=?xDIZi>&&))8y?Pe*( zt;DkFm|`@cFI!Kx=wFn7fh&cqy-f1RZb2KRCK7JNBsApYHWk=M5J&|wBQOdb+2_^g z*;b(s3o^wX$sWZHhUhNh^+UU2+hPaWw)eN~kHy66akHOp4#cDm_4zDet<qcA&k#)o zZ(%Dv>K1Mqx+sR1`nMz9wwQP*hL>=&Kei3+FtV>|yg%{T(6f`N5BR!MdXj8xHG^3) zqCJiEswQF>ZLP}3Hs3ciKciD63}0Z^MFL6+`V473sGm^=U1^Mx3`Y|Mrl>H0pEcT6 zg^H5MH*WeRUNMs9VN5fcZQ=>}GHBs};LS}+P-y~P#IlYJ0P8ym@R(0L;jYe*1D4ll zwDy~vES0HtyCCI2411OeiC>SA#1wX;8DRXzVihdy^T9BjrZUmN_=b)~n*!R4%Wps~ zkbFH!%W;I*pJZ#8%)c_#RUtKlOksrV!Y3i%vh>?b076sjL-)-NtH_t7E8;OBZOPa@ zAofQ3jdT&<%k!kzaG)7qW3j4HcvQe1&&jd+f8}J3!f+>UDx7H_B8^6hA&r*!PDQ-B za5jys`+BVIUd>7lmgi)Y&fyh!`yosPQAwyIh?7D-h2#b7);pTpdfDrCm->#&W_JPe zRvi?=>OgitOs_62y`!|JbhXf5STOdjJDPjj*#EK7D|Q>bl1&L=hPkN@2)(QE#vP@l zt9uJeTG&n{WG78N)aYu19%#`y%8i44oVsSwNLRxgR6hF`tsw;8VRy)COB4<JUN!)% z`7vdOk&d?<(g93fs3|PTNPor1pPNW$3<O#_EA7})G$BSpfHV?TmULI#dE$qx7wbVk z&~JAC_{wwIBIXAzc0go{b$tyF-B4|{28H;j=K3zuV<qS%wMu_t^4i`@H+DSvdn-bD zuywne$M!>`B4i4SsLAa4`Y(WRazi3X`V<D0HSABo{&CC0QDLaak%OFWRGk<uFflyE z01;aMG+U7Ux04Vvs_$?k#S`3MQCM3<V`66JXbM8S0++=Hls0KFaFn8zqPUf$OWb}n zqfAxQjNk357Sh#sz!o$`29{*=IlAY>v!fMiDilJX?r1a{9%U3-*f6J-iKJh{i^La~ z$yJ?ASG(MP>=IKImh$g9bD7xJqR}YghlfIHszUwEmoF2yQ`Xet0HgZCGNmYge2TvH z+d^IF=q3{GD`-m8K+R-7AdPA64e{l|c4AofbmD)4hUvwM1bw^%@mXLok{H%R#q;qz z+gU3h@JZH-G^8$-2?T_&a!E51(fhSa5<Hg^#*~Dm+3yoyjfz%;wzVmN5IR8li+ca$ zwbZv$1+j|?wXTbuGq7njBBY45qR%Et4o^v_!hYa|M)ro&WEKN*NJUs@_M!t6LI5%V zb^o{*FLPH_lzI_f#b&eq@tD<1VOkJ+2ytha{ElMVVN)iIbK7^?-LEs9AK$kfBlRpq z&WOB{-VPLkoxxbQ)#})8aqPMdAy#B$fM7_p2T*$qp*<$k%A*nEqb2)XX2TVMsJB6n z5Ki1z3o9gX8w9zYljMj)X!33V6x+^OOEq2l=?|Yh3Kn5AV(XaYn%?@QTiAGN3iTCs zuiJh<gakLKH&e=^81|Pbwkz%K-#_xZ{l*gWWfmrw<kG4&bARQwUujd+tDeM*C@Rx6 z11INo@J#D)1GU>Q$w^j>=mA9b7)O1^G1VKyM1v8fOAgDLfFwlSN7aDkBbh=1Vofi; z{_|sQ`!zOY>fWC264~Y0Y;ZbE!j3Cqv4wlfV?E8SiTe3tr;ceTaXo*JV!Oufp0KT} z!>xB&7aARQo9It=F0Wa;$5j)X(=fKBtv5LhYKFC6eJA)BwZ>zny85O7zI6@a-&ln8 zLF2LorHz$i{9dO!8mb#Jp?&t4L$8*9&!)KTkLxQVHBP<TbHssoPog_o=3mwwtV@KV zireEK8}FR5BG+eEpwBwd2D|})dgY_Q2B5wZpn|mygmgKlra{EZJY7yZo5m&j{9qWx zE8O;n{BL~oK~`lq8@)eqn$(vwfc@Bh@IIfql=_C<GvQxvrlJvD<F@~Ktm_eQgEgD% z?!({nJfYs{bi1wWf?t|!m)VqMi`;hnutTw3MQL$4X)*D9kQ|C9QC?8%5fYb^YdVz? zQsUed5GFXu0nNA15B60d*&xu`$hk}#W*VjtA8;PMu<BbF6azGv020K{|Bw1ZqtWdC zv10y}5;Ko6{LtwAq04MCsz3h~A9KHBmD(9qVYA)#%{FDJ+rGu72veO-r(0bfUmjTD zVh<=zyDR&gV%uVOFI8-tf#30zO(9x_m`T(DkVd$qEgFtgKq!btMoKn37>8FA!bZwX zC$1xtlq<lQLY=`b$iAR{i+x7zq#6|4gLxJ806iIiFMa3*aq|oN)D*i<J@8)_7aOwS z!EGq}F8Ih}GLQ%XWitXT(+PcVneKdy3?-^SzY91_c)LZK5>a{pU|8*e#v_V+#E4OT zjwi(7(vGZ$V!mG>tD`=FtRvSqWZ9$*B?GPmVd1ek!0@{$s=gg&_gx>I&W_E$e<7Y+ z5K(_sDS$qH^8rKPSita&*B->#;u88_rMf;Axsguitwh`|=XF8(EVlU^L*PKbu#TN~ zwj8|9X*SENE}$egSAG|3#!^5By}_`$$?RM3+{=QMMid7b`V01GIvvI+&E63R2wQNp zn}sc$*2c&2oUL%!tO4~7wk4n)tpFT)D3<_3R0r=|=}&0KCf!VqIpm|jC(z<~qb-#Q zZxk@2wJZtt%hiN1;J9w_Hzt9B+S-HzVkb8@NI<A}HToJm^>l-+0XLm`=_dDWyDqXB zn&w}0*`hmpYVLH;R9>jKpb<gg%(l4~ypf9Bxv2Wi1($A_^jdk{jE)tJ3yj8z@$T_c z8pf9^GfJvD8fVo`G05$eQ|sE}Rc<l;)ZEge*Uvb2`dFWIa*z!H{ov^)c7(Bt?gtnf z`c(L2$WYOKPWGOvwyF}VUQoZy5E{)Hgi*&oBTW%MqUJG((HKy}{S^(IPRAg^Fv|H$ zl9Jyq6L(c0ooR|lI@rqIVYg==f-`A&7$PDJcJM8`UEN~$g{#{J2S|)Sh!2=9q%X+l zTa9@$6Qj}xoq-Ra@}shb8xQ#d7qWH5Fx|!X@^O5#+Gp=BEK1u8r-Py>gr%Tssmku7 zB4?i;DJ=yE$<pA_cr!K(LE$dAB>6)n>a-tiWd=_(RksK=Y6Abz5;b5mLI|>)(FA9o zGzACes-Q@1Vend}5C)iY7*G)}1M%Udge?eW(1HnSXri;yq(~2bXQq`x;Yrz#0k&ke zS%JGlk~lDWC_ny*-Pvc@4#dzy&@`+2PkV%<dXwH^v75aAd|e%wGxk5n&Qaf;{{H>% zOIv<3)+u>drFF184*~^AoZL$_J<;#J>d$8hF1HEz)8d7HT$%mI=(a%Fw_CitukY~T zzCPh-wvU#V(e-YoddEiUO$O~Gr_8a91@$Jc+rpZOpW6;!qTct6s-1GiRv51Kzn!ku z>d;8_q{~ie0yF5Z-59^<YXgm6;>#vLXATUx*cq!zD=G$XZeu&u5Te*HqWE4IIDJ=3 z;X=s*MnE=AeJ9|E8#P5YEW>Y3>i7+gy{D`72zWgEJ6_;p$$k1u>hqEMJ4WhXT+1`J z2UoHdw1-mEKE?MEYBN#+HGKNk5c-SiJgPNDBrxIO3hq2zQ?Q-Gzn`%I_?VYp&dv2M zvIvf0jiNBnpf1lm=3_A6ApuPS)>4!*8O26GMgpxwaM6T-up7}x$fShgk;qe5v^RIo z>TaB#z4r{2{wUbivuj#sL%^MIIAif88=Zo8VO`(VhtJ#lK)G7`AVbhecjuza-rrB| zo4s>x>$20;IoY}UyhY=kM#Bz+WZSjeUwYHVtw){{#_rt79ybJJr`6`3xa`^N&f)n! zT=yimh90T==dW``)l)vNIle^QUoEWPPd=w1q+I0(zj?aa4;5EaZaQsy5FJ4LeF}5{ z$zg##sP#GwKG2!Ph}IYe2=jqBViZeEZy;=DiXR5O3_2O25<n7mLS%x7gM{b{!w(P? z0=P&Yp-_Zku~0DH4=E1BGjO^O2jfzI(jhjuT#kfkV(O}@xvr^qFQ2zJ8jM$!1h$Ot zj<|y+lfhEzQGyM##zq{lAXVH55h^k7{GA!f1bGUfVFwO8ST?yXJa<ytqzSR@)8+~l zp&)P=VnN~Zl9(XuOf8SNJ7P77x(<)xt!S+7C=bNv)>Y~Q9y=cg)D}9l1=&&Xw&3<J zxv^Z#!N+`F&J80}F^ph1oN<Eyev!9FzDcM(+C|I^#6zZb^PH(6wkFw*`RZRKqg{mf zl9?Qlv9lV2il@|KFqwkxNcZ?Hfs(3tFuHf%^1G+H=2l&mnrKQmT&@OD%(q1Lj~dJt zZ;!_&t(d>l?g{8))$`(k@{a1p3a{ens7utuI^2=vshxrlD-kY-br`D+hAM=))3(PZ zpyB3*357l{^D%K-(OTUkjEoJ4X>x<^UfmPAA7hlXG?QgK21ybCZk1lxS0Sifv<291 zEjcA#Q%-#E!a(4PJtQIWk)#atL{s*GU*JZt07Zc#S!1%fwV7fXkwZu$LI=?Jii9b& z9N7&))d3Vh8fPHy4GD@Ijl7yD&?%NGuJ_OccYXkIaDN7{Ux?ntALbeUyb?sbz03s# zLfJD@r)GcJGkZS!PFErpG3low5RJ#jCL63{qLHqyaMc*AVNejQp_b+{ucvHN$a_^~ zK+n|6Qz^l#n5WiWi;#UEURyWC?C}74{5m0i9bm^jS=(82np)-?!p5j&Hj8-6#y5q$ z-cZx{GVhaJT^!E3OK(B$?9)Oq;h*nmgonr@l}$~5ny#*74^BUz-dtT@>WZ;S_3r_} zQNaQi9BKB}jHzND-dA1Yeacj3_qnU%q4vw$L-Baogt=3ig3Ri*h;4T_HQn8u6~D8% zu3dIG<o3Gdk3PPnwu7!aYL~z(vic`K@vru8PX5cDD<+oV-HH4Z>R>z7KUO$}07IDA zm>ULZ#zLtQpB=zl`Xly=k@2w#_&57?*Xi!kJ;wQT><Q#XB7^vAZbYO9VC=e@BB$w= zeCM@bbhOZTYK^r-`GA32`zalZp2u(lBi6_x;2QK{K1ATnNuwgnM+j>Y(diU_s7c9> zJt9<SRq_L@|I`sGzL@a)eEtMmbk<oz6St`^Y~8@ZD_52yC@@=Tv^I=uWsjPGpIf_f zW&OBT*mo~lg=MYbyP-r^ccQc`F>NLo6(QTdY?<&%(7s~gGuhxX6Ia@TxNd)1c%NSn z1vg!?!9F%t+BbteRT}T^ikFtgySn40Y{9CQ#s-^l6%*Z|a#r=PT|QRt>uzZ1KDuU2 z_UG&)_39e07-r|Hmy8d@CawADtYBN~ud`dnC6l4WwkC7cwB?%@#G0C73m(O(B@{A= zKYo4MwAZI+m;dFW_8z_0tM6&w{t;apJRSqCB|8-3|G^xy4{cteem4EFg?KyO^H>jM zvPiWhJ7a++c1XQBBKT_Aev;X1adZCx?O6i7i}=MPVM!{DFhM1no>Vgi=FJObSSzE4 z!cz06q4?jt9&?tl`>Ym||8Lbn@fQ|L_G8v#F`IpVs|l!&x&>B}_z$1B(XGyIsHAWY znA8qOJ=@^)4xPoaU-h^g^}_jK@kTQ7$?aFf|5I6D)sIC2%qiC(coF8shYu$ie*)ue ze%G2{U`NRIn<&=&^cNmI;H`MZjd<VtpGTl)=d%@xm^8S)U#_Sd!ESrP&$tl*S{-6v zHU746f`VFpr_y3|y~WgT*zL9|aiKMuO$cfv3I?0K&4hn-UORfaaCwRu*tY+`)hv{_ zo+*t|1tZXVm`h?2D?`iK0qSQ5zZ8jAjTN+=1dHD!a2XVDzEj`SD*T+ir<OkZ5SuV- z7Nq}?K)BqlR}v^C0$dxa>~?#3I1s@KF{obqiu%g9@l{o^DS=Z{*u!j)-EktzHk%L~ zUeueNeuutfbuxAHnCfe9zB#!P8?xVF){CM-QK}``94<NKcSM~>{Bxq4Q=lI*@*(t$ z0*llTSuC3*FY_i0Esz=DU(#!`f?@wi{if=Z>r@~3asMrB8H6RvvkTcW)vbP8ZeWX4 zzxps+&i<@^TXl<*)K}C$u*vFs=c>O<uva_OepgZ3^mp(p%~u)K{5Z{k!@f>W^5N zctHJ;`gb-C%!>u<(kED#4A{XPx$+SHa}?%+(O6P8P)JhxL-2PKS-#1p!TbB=d;5nL zMMOs=yP`{Yvn%^wn}ki9e$C!VtI_NeVz`$Lz%L_RchA@F7J^6AM{gFM+M7MOSKOPu ztXH`F#C^w(VO);r;56Hd1-i|6n#b*T>ceqoYd9adu&Oc+x`?PF5k{oi7$_HEV@K2z zymA4)N+`DI{|3bN<-4D@&N)YxIVoqR5q@8N=Kc5COtz?XZfomYb%y==nU^drYn<z` zI3Op&P<B{6a~sQ+UNZCgGyf!<FPq<cb%olx;??)eGQVcOpliX$4LaBZ+E?D^A8Ad0 z$!te|Q9b|E{Q0*=&1>>b!5Ctr?PZ$sZJGC4(Lx<*GmYK3@9};69v2?xCz*86!x1fq z9-^Oe{|eU+0lSwM-%%oRlZiDYBcsgabpN8BFSM>vThx{{TLd#395z2-=dk<PIpPuZ z<pJTl*?Ij8p`A|kz3!1Om%j$N33*p`(O7_$HaZj{d(8I+3YLqBkKR<!&s8!;0~t<( zT2$=h4*{AQK~EBs8OR;j2r7DxyMPde5d7m8ikXqDf`1f1%EuaawIs?)<5jbg7ubKc zZt@&ax@GmMOW*U%tQ)VsqkdT1kaCx=PIkQa_T-k;%Zv_NUDdKze!96~#-fy`B&>J; zUPumj_0A`QOXa%S$dG#HKaV)PHrXJUqTZlMEURp*D&K#c?PX)`>TojQ>yzh(U5ggE z+}3v2ww-mQmrPrgHX82`E)7LZ#<AlK=E`7c)MO|tH5#Ks?%>9*S)OrYMVHZ2*%Ix2 z-f6n^R()lg_{@W9puD-%bs!$vZY>)VYBn{#u=iUtgZ1U*4oibOw!C4kr;~&cIo+d? zul5rmlh}%uY=)i|^mJ>IyR&mweFZIu_7x~{W-C@zr5Q1cK^!<Bg2VsRurR(%#1S|` zYwL0m&n4nj+@}zRJ&USpS#4s;Y~zZ9eWH-8pMKL*o~}gQT@TmAyS%oB%JQGR9;s?< zv&fG7)Gha0<)CBiJ)eEIvfOcv$yK}kEft6y@!BTI=&RB*quy=X`dubI8K|g`#<A+c zu_@eH+u-wTbt6*E)bp*YyT-5ZR4if>y+OU~frPEZqXZ04#L0$|tY}D-NPT^J>z!>2 zLk;VdDSg7vTYSmL<dC(};Tc!neZw!R<YrH$rN>jc%I1lCVSm>+G7BEY6w@(XH|*G{ zSt~)o`-!M-5J4aV2N@%gOd!0FRF<O!#m9U6kFVVF==S-Mhi2~V9_NvnLHaK?Pkq7` z?ix4$l@FtxOlVuZXU{|-{Hx&dvoBa)FB{YqL)jDM%`FJ%Sg7aMBFHANwZS6*`ZF@< zz`_T&TK=t0D6~iK%+mJGy7sE(5;H@Xh~q{km$Yo`zM^*dUF)Y^!|pF^zd29)n7?|A zpWXOlHr-seY|FBeud8b&T+}y3VB@5*h1DnfkCc!_-QhK%%f*~17U2n>IBn|vW}Drt z-eWVGJOi3H9hf$!nudR8+Nmhg011-@!@NC3DA2QVhVsnWtq@_vVUsn7Lgo{)!})lf zHnxUxXX|Z}q6~&9Cutz=WXN1iJCP;&D8)pBPR#N=xfBTp2pd7-lFF5XXBc!;f}%nR z1Ca6zjC^CAo!5Zpsbiu(lgpE2dZaZQmR3Pl1Nu#$p&}HOO1KhD0hr0cDxiUoC%PDR z<vY-xaqjh)W9Ntqi&lzo{m`gwqUSlshu<Mn0MU*M{Eu}I3!C5LlXoPRT%I1kYT1-Y z3l}!saoa6Bc04k3>z2y;b(?1FUenyXAUfrc`fgeI<kr?$mpEtk;|#$A*_8eJpAUy4 zmAH}r{@#(p<>i%?Q>s#3O>1`S`<Zm<q&agYx4JF*+s@rKMs&j5T(hOqaa(aQM;$-| z9D^wZApmG+;hWiI^rue`?IP1I!JXZ;240!}o;^=!$?hKdXuoiqc>d7)!ab-ztxcdp z<papg0lTFn@a$IMc_Ueo={-{J4E$-b`qaoyzz>i(oNgfzqrSy+Qa-h~$kCFl>tV#u zT0yo>Sj8|%X=Z5eLYl_j3H$wFA3GlQ`NIC8!J3ZtWgQ*Tf>iySj%6K(I%;b=*zAUs z@a=8sq4nu=XBezD!_2jBtet7FSq<zc{oYYi+aZ<tDIe|MD;+iJqS}s*TK4Ul4kf#( zA@v8k`-65&H3)aI+mb)swd<$JD{Th7^<YiSL95<i%iflMtG2xz`zX;}pP@;NXp2f_ z1S=4OdG##z9~jV(lIRI-$FGkYP$?Ftb;kfo9l{Z@1K8co2_kr~c7xu*T1`I^<->Qn zIF@m`p^X#2_+Y@)f(;Nc7NdxOl%T-$NRFKpzZ*Diiyv-9$byI~Y_VA7@fF$<pa<7J zoQP{X0=7?b9<h_eCQhT%F7PE0pP?tn5BcCd?VlbSw6_&I3|30yiO*>z<qF(Z+Jl-p z!EjK4ygq-UK@Z-SsswZd)TuRrV3meSy7YVSGVUO;uoKyjZJ$_InD&(mR2(Wd-~Gt| z!w#K3ZwmSVH3CtGtc4em3MWDlgDk0#BzL1+sV*peC+<=2B<`tft(1rwozi04<T=BL z0BuCgn#-8Jg^!fW!tE9lLhs6}C5IXQeCl1`0z}bZHo=n}-dHfmSeis|T~RQ=o!&e< z8i*{9SP~TuW-)o)HXE~7miQ#eV@WvL^+vs~Bvxv+xItC8N_@_3Ze|)&BC^WTP?^9? zE`wPwSqv`5JOP)7F`vnpU^axC^qE7xu}PsO6qZdESu|NavrKitl2&XB&XO8eyfPB< z*#tqaw-{_<>4H|Dx5g*3@-my-zW{NS^+s=4LU=S;5ULvFYRU7E$thNp8*A(h3CX5s zqQ~5@=c+ot#VX*Ndavjg1ef4*RI#r4+51F`-Xy>#L9~eMYl6w8mrb%>5bZT?ljVD6 ztEdNv0*uOqR@o*xU>7I~%q&O{-x-<HE+joZ!4UArf^lPfZK-Qh-0e44me(y)=6dR8 zR5g{BJ&;mTRkZ=xT*+W5%gpv=uJVAlCEQeLvbhB-ELJK-vC?ySG_YZczqVF%d(4-N z>#ny*Sp3}O21M?Rd(O98C84<|F{P!iYQi+&Y*nsLu5^Ihu$V)k)=GECZL$l#xZCMb z%xz~?w@;eYGR~3+M<Sw~*&RV=NZQC0rnlnOAzEyDbrCZHuFGga`<fBS3>_}0ce(?P zl902^TxqD4$DQx-Ouql3YC)>Mv?0+^0b7X9MdejK@03cTh{%+U%}ktHqQF-^C6`xw zO``FD0}P~L0z_&PDjancf@m?ZGR0TUYN{lM-RfudpltLzU;yJ{R+GzQ*P|q&zCuzY zP@pguLKr`*Q*oFilK?v&y$CF+j-b`jSz!_lC6mW>m+2px;ND~mcq=BCmMTz-PuXY< zOa5z2j)rQ{(LTN*&~0=Yh5whf_W+NhI=_eaPTAgjUu|FYx>|LuiX}^yT;wh{;oiU% z_p&Z@Y`}m`F<A5%LqajdV2A>N5C~v?rUXJU2@qOB4H#QH{+~N5*}@@#Jm2%V%+B2D zcW!yhdC$u$WMz8Y@Q7Sm;An!nZCaUSSuojY3}>m>9D|bq{)XtxPsx!lnpMKJ$><Xt zF&<MH<b>l0=VE#0Q${LhbVQ?(avB~M5H(A<6VIs~Hmen|XCr57cj;wDg~y7PjIZR* zau8CZLCaPfRJMsKeNi~1P;*LSAkgMF^Q=afBekooDqXYIppZJ`(kv}2%`0n&8lEg` z4=C(<tB0oZo?G5*zM^dDv)$`+EyUx>+1ET{^|A%kM<y1x#uz3LMy<<DY7ZaQj!U#` zY-r{E#X$#N%b`-Wx2k=mUYAOTQF%~~)=IKqH7Nk&Ngcc#Ga&^=*dTTKM`3wmBT>#z zXK7m|9Wcfc3=~;>1jcJfX#rU|Ppz!j;7pMyJxd%-z##=(QTY&BIZl!@lVSAb*KE2t zsC)F&?X{LH;g7;@GHG<l)t9c1ggN89H5)GN%OVBr+yD26iP=l8>Hi9oIy36f@s3g3 zRt#I$TBG}b-9;4UrV$&5Ij9vP)Y;Np6VLT3k-c!=P<<;z&y-p^C+_T2?PjhnuA3&) zZg_w4<Qu0qYc+&_{k1z9zTWdijqi((MtsPgOUPBXo9=$qIjp?KA_P8ufT$hG$~JdA zhqt47vCs~9=DDGlrxbV@Sx3ZS)=8#gMk^~|fsjK8G`t+Zh0s`6_`HyX5VAAko@iEd zOxp$Ui_zMov96iKTPO_CG{j;+9i$>iMx50MTey|GHd-~Qvv|JOonzEpncEx-<v9y0 zm)_IUoLbTpX`fKDtJ|72x}kJTaZZV+r1abUZ5J+SB#YkPw{?DJ+ZDpeCzj|(4{Q*g zZjd)i#+K*VRd$(7uXh?c?f!s0nV*<k;HzKKRy(C8tPNSU=2)~eD?dN0CV%R-aLf5u z?t6c<e#sMrzoM;k{?^QfaFTAI4p>PZbcYu(#|MF)Yep>~>mY?NK)j*MDlofYp2?IA zdWFjqQYB^@4u{F4kONMK_E=?Xxs$LThk3UpU19S{Nzmr?e_{2qb`9sV2yanqH0d@5 zKGJp8aZ;((RpJ-E(g5Ey-P)#3bab(6W+bgQb9J5E$fs<9fcfNuxIvFo=h1Dgwcy+w zPuTU(HesXi2ZPm;XEiGog3BROSUdQwi5UwQ_J3+1m1G-UYluB@01JOMr|AGf`7CDG z0ig`8Ee4)kL6qbPGy~CNdwL7bt`jNhr{b~f<0Mqx@25+$lS$DH(Vxp|&m0t?&qQTw z7?k*9V*W>p{DU=}4O&dJVTtJY(^>`^lPL~F6O|IFf&j!DWck6E9}tqnN<Dwe|6YM; zr^bTbf=2j?6yN9n%~Kky00C^<Jd&E%25u^$XWL`jwmn9G3&|zpw+nN0Mi6&@S<u*6 z0Af?8QAcjvE*!|Mh<WTJwP=x$va>z(gl(B;1+U04#Mx7H@PM!jr;8}`p8X5AFzRgZ z`H&lBbVagpDgs^cAL}3%1zD$XOne$PNmH;OFF;TKQt?TS2u1Xly;A5E%X>i&LS8)c z94WDnS|omqYiN=XeK3B}x+|c@HmfZ(WQ<~YG9AvJ!q|jbd#I*5WUrl&T>ys=H|eYa z=2P;fwY|sZguD`qxdX)M>uI;{{E0Cl55B`!K{}wLHeN|4VH*Yn<g*Q9SfO!lQOZw* zkafssaRZDwfN)YdGl3=tagv7+=p$jsl@<eJQy@3Ndeb-M#@9Z5?ZzpYAF_ng3vX+5 zTH)=3$}HtU;ccx>BfJf$tm5E77<2U`gq<XGP!UiZ1jtdRA)$etjOxr~aJ9IO?-5=C z+DtnAxJGk)x)g9UQqsfg4C>>@HG1qNC7Hcyb!M;d687pf$B(PUZ=T|xM7)L(EmRVw z;~E{-q~ZvOOr2pdE3KGuy*wmJ%9P@R0*A2yuAhIFS3E2{e{lXEPa&La>y?-<Bh!To zj_su>W>-8zjMwKGjQ$BzcAdCp)p^-It?U!LP5Hxpchm^Keq$?$57$5a!Z+()BJRD{ z6WgCQN}23z-^iC&TytVqsnMs6p-*RQ(ixw2F8vzfP=&GB|8F?{vwhrLatNCSGk0hY z#-0-r+MT6XGIxqGf<)4vq(!0^mfU%UhXXyCkz}3fmG;0s&`8l>X!W^JfDuz9HUo@{ zuuFqpp>Uv)!psk76{RqQDF$&!v^n_EC<X|E2=v1&tF#A99!*E!2roB2%bSypGmBEC zbK0u4x<|}zJLr=Q&sQCz4|MEa6Bt+TWjpQ#->T`}V@{zZoqC)oA7_w~`M~N|5Q|_k zJ;Up>vyh*=Kjn%>HQJW}(v6${w!9Z%lq8ZlF>@K=Ek<&|<By|*pF(_rY$Mkyxd0o` z9iYKb(<Rj`7^r%@G=jMiHcZZD2IK>IT4DB~B~Y_O;v9%9bdID;FI$4}a;O}@l!+Yy zZ67)fU;`NEa8WOT7DH7N_&*q17&?q>qwQXMcFgOOnF<0N*-^sEWbzzvC)kr_vv+i5 zgPm2{O*$B>IAd@{>+WUK><(pc@%$Y%QkK)@5Tn}4^Ln|tOsDsh=f>O`Mru?jc?N+S zjv9?oZ;e0J6*s%IG6n*@)S#6c137i!nnDgDIU_YINmjH(${tUCloc<{sdVK)q-C~s z^SX%F!SQCb+A?8SAq-ab;ILesL&}?2F1w-0Zdb;3_7dq1y_J`mAZv20%2Kk(?Wvhm z?B<C)R5iaX)~Hbj6pYoRaxz5C8+`t%vdIxuU7*mXU^r*)l)CwAl{!kS!UD{$Q^4vV zId!-6dwv9L%yh&nL{#kpc<3eJ(>gJojYahs`X@A7)HA9Qm5P}EkW30FIDr{C1ON{u z1g5dIMr=}b5GjQLE~kiOEsekhAqGW;iWew{c8QDP()f-j!!>b}0<_?aiq6~yI>*3B zi`CdXW~Cg76+JS8SL=N!|F26HjVUaAW#N(;&=GruQ@h?1{-Ra%60++(*a{-;SN={& z3m*yJzP9zU)P6F#y&<2IYIRcSWv>_H=QF%ksji&bymFkwB+s?s!OWBD?KvFpwAYaF z6HB9tl5(fq9jdFlXQI1E?Q^gHx<AmO(>ncuVOg#lH7*|HYd$Tnnm)HD6gV_v+Ekb4 zp_-m+TC}!*?8^M?Y`$XK{JN&qk1Sq6xYYg&+mlym)o2Awb#46$jTWSN#;OI(jOptu zaCbaIeUAorw`cR3Q9bDuE~l}?)pf9WSllS}RTN5{AmKP8TP%l##6<JOQyPLORCq}3 z29Olcq|6~?XvRQkK(r=z?ty21f8f%m1Hq?-8`FP!2rzXn{?MP&50cPbf$7sH9>4O+ z<9w~)>KD$L^#-v&PKLdn&JjL-V;0%hPd@a%E}(nDen@49b&%5#O-QsX6;-7Ym_{)3 zVl37&u%3X?ma&!7b)K&CFgV2vc<B9{78(wQr#j3(TH2ANbd_J1HEH_%S<^?==ncz+ zeMd}4A#*Ruof{fHebUScWBtpYyJ&g6Bkq)0TbpNeO-SZTTQ<*}P+RJ^@*sL}wN{jA zBc*NU)`WPe8U9wNN7};jNi()IGtt_S<HvPW8w|D*seQ!g)hl<B`$w;=&0@7)hf?{0 z@B?v19OSjbTBBT(*S=|Np*c9RBX3(TVHwuosc2u(;x?GFtLy8Fb^3K}QuDC(rOS3S zJK9EfPa4}&uG4W-oibZ}yvoPeMsDt^^%~KA*sCv=*{Ta7OtFNMSb%~446ZwhBsYuk zRicCis~|Pe!6av(|HFW>Wds-QvlU}1h5qyxV^(mlpUfHjzhVqKa?A?iY8<~>_=ad! zk8dO`rvOwQj>Y9oP2*Ot9wKK_hBC~WVtf!r`yU%(p%oD8e+cg4QUi%h2a{}O5}EG* zZ-HLS&Y#F<df>kWd<|*0G}o#4taLmE^k0-iGxUlg8Xl6I@jpH*%~?tx@JuRJn#pu1 z@%_I=rNM%Y&`YFTCG|8jY9=GAaO%H4EqhwG9gJlaZKg1oi{db>rau><CW$!BhD_L* z!AxUl|8p=H+efig>VdE^b)^5<Ul0$5R1wslIVkfQ|3k)r0e^1Jf6E&%5M57(#`gVt z`hbi~E}aIM+#)H*C}0!CF-T~}1m*#-7eOT@VVgJ_v1sWO#eq0Sn5yAvFu@5#qrFV* zskv3ck;nH6-{1Jk-G+6$z}U0tweuNQ)hRF{#tdHwF?*QpVdUBg<&D$lH;2hx!U8?{ zL(K32nf9j#-@TD2_C7>%>b8}?cL9itw!Y(Bo<jblcl>r%WpI?%Pj4J{j!bwjl?n=A z?##%PqWmuA8zS)5vCxk(#bC(9jFU0xQk5C=7R7TRzMFn&JpLe}gI6mL{C!MbWW0*I zJeV8RWO=t%FK{h(m362pOLR55=AN7W`u2&T{v&qlpQUo)8&gl^+xyG^_=H+E&E8{g zDtj>Tm&AiGOuNYD{?mSBc+fDm!jX{TQ=<NBX1I(RCaM@z=LO<@=z0bj?tw^8BS&Ps zWpKq%PSXQJW43asCxpYl?bRCJ01QK+v3)Z5Yt*8UtkyYg$mx>#IZQaQl<E7eaQF#e zqGvS{U^h}WdF!;(YbU5y`Or$<Ov~3!8(a|_2-ANO`{+1?d5FG4Ca>l|>^G`1^D^SV zM+ZBRqk?)b(96%pKAv6kG#;Gx_9RUJOrL=Ch#REmXQRXa?RfD@|1DZP<I73OQy90S zkbCMS;6<a}H!3?r3qeKzn7zdcj-$08UJ(+E2t_5b1SU5djr8?PVU5=T=|@1w1JhT~ zgyxT=-&`s0k1&hRyE&EAm}FI*omB`|N~Uf;kDHvcML&66e*U`2`Yk!qczjHA^U3k- z_iuhtxzyzbq;+zB&os|XH+iP@6ifoRmD3iMw1voX{fd>OH<>K-+Z~L-ZeSdCe_=8y zv$DF<yii`Lj<w4ALrX*6qHBtT*CRPa2-bo4PL7ac<OqezavHzGGCDcXKcL|v8UfgB zv>gjbD+f$Xn5p?QtF#T$_pgT|@$@QGPJGo8D>TeAt8fg6onA*w0M><Bp8uk5?6i9- zWaFv}J9P=+`GkC2O<{*FalOY<J0_90cfnq#wWcePcT+sG13<5G`FMJ8{8o##W_(TF zO$%<f_xF?X1;X>p@iDdM_^a=-IIAa==ijmLcDs$P+!j}iuEj;;q_SK-hF(6t&u*(3 zU!LE)pqCz!$h##W9aWv*rYjeIUm+JxEFjgC8ezyBN-_G-vS}?09R$E(jR6BMU5U^@ z(V0P0B}3^eADjeW+@$S6T2jX+!gXXQh=c{DMBthD%*Mu<g-;f$bCeY^HW{l(WcT;Q zDr&=p4@FtG)@`y|%oe)|Yb;R(>wk<jIeCm644SQ*Lc@k-t<920<U$7w70jIMs)|^6 zlp|}olQF@{O#wFeF)_ckjiT>`k2(;0!J{>|O2$aekt_pC0cNlWBQj*NqU$H3%h)ui z?qoV$6o>@NL$D;;M02ATJ{}%ng;dfcXd{fw1p6fDH854f8<GVBq~2IA$qJQYJA#Y_ zQfyl{H<2a3|0gf!b#fQiA$(epW73*TFOJx(Hb{}lWAvI|2_FxX$+DCS$k=DECaML> zL_5c+rAD;odO-?4m`z)jE@0QsIP#m%s{3yxi%G|qJ9mC592Bk*4$?J5vvrf&4==v> zL*Z%RPT^^~#-wiB-EW#fR>F=Qt#Nm25b;_CbGzR|l<+O7jV3LT3y%tNHaS?@`}o41 zF$uNZ<kET7A%4-(^jayOm&JAc`sr7MeRG*R_b<v}%6@d?4)I}rc;w;_N$pwBfcmHQ zwv-tAVx;U4#K1>Fw7Y~77Aa>jb2bAph2cqyb2hF{`0@kc^4I@JroH*5@Ck{3%HA7J ze{=QfTZrXPG(~C3e0zG=<=@}#yeD$(it9e|@}t3Eyl(l}7SBEY4FhdhBIcb^!*gCl znFlPvfq4vU4akQLkM!yPH0F@Xp4CK5WGsrIY#-Z~%66Yny0cS6LL^vZ{#CoPf547v zDOQeSMJf?e5Ldtea!LXg_#yu@^rU^*gZ%^VuaIC)(1`K^c$#TLNtk$0pons6AR0!$ zLUWQKxeJ{spst%xMbvmTKy*u_|1@&<2(Jsb3$Ne98JRk3nUx!DJ=x2tx%A513Tb^+ z6{A$>`g95<Q&rWVnkYPIwQ@b!&Gj&ch2IZbH4Ohq%|kL7qc4yP&<$S-+&CxHHY_5H z(yH)BOYV;hYm2<Re!jdwPI`bN?8W<^3mgKn!%|EOF-72@FoD^W`ZJ(r8R5!E(A8os z#zJ<ArY<)-sfh`v?weLslGjwf<99jXtGd?|md7iq+{xfb<u0ZvU8_<tPY)x{kl1B9 z-I9Oc$!b5j`o)P_ZFE-Eg@)s}4Q(&{`{EobA%5D+5MQ0?S)tM_n+#7Ok~2WgViSH= z!n~$_SoqCbH^IKOb}j5yH<7$IURnRhmcOJ@f7vo|*Yp;@ROn#-*;M<caG;MKK&puQ zrL}+j<La&N3CG{tS~PrWR~Xaw>2ZR_y#^#BMQ;Q?NEWr8Kwqc!wGt6zh&EFKrvp{{ zN~{S=Y!iu^0Jos91XK~^De&WAO?3BQ!NF<=uyq~mg=a<yj1E?=Rj;oT{#mC-oVK)R z?V4h#Q8-%G9*?(^-^Sa^uy(R#c4+HE_0Q45aJlmxlXTd@Ey1#qmN4x4_Gg|Yc9}Os z=Ke%~B+K-=x;mX+DlIOSzKV;XEp><KHOsn+TSDDZ;>r(~#oOa0#k@s$PSzc6DGpZY zT%MiJKfg1}p{soS^vIIw;22}*cuMOjV++=yo`T|dD%z@Ov!(S!t0^oRsA=_x^+YR- zRun2H5=~%|fM4gQs|vMD>7n5f8#?tsN@5RaH1W^l8V#@Kb6(2f^@31PSCF5~CtaD} zHvqx#ExV!o0Lk}Jze|zj2?JMi!xC>^ZcUbx|8oD`UrHT5QaV&bC3|pDTvIB|$&v2% z6%>eP4*a&})c8hn-$b+WaF^U1-Y9%4?aZpl@s?;DwsrU3yUt<P6q65z3bAMOB`-#w z8Z>6`1&HKhr(r4L3qt&ZY~Ue$d;q9YOJv}hM+5p1Omb%T%HEakh-=S^t}!cIW|NCt zvYY;N*Q~sC1sQXeEuA^!svEU*$tdANv&&^(v#x9Tve5*SsoPZk-nva@m)o@7>0Un? z!Atj^ZD6Nk^lh>fKMh(sMon0&1|FKqIv6qslh=z6Ed%72Dy!IIOJsI&k(zNe{r5j` zk_^X6`ZxFWKTWP6!%seNfB&|pQNmWNqVSmX-rpQQ`2bN0Cje~8WfmX!`rCUhuDV6| z?tzm(+(*>4Rl?Uf)zvuzW2UIDP+k<|WI}{Ib%x>RC*r31(n%p}+BT+-9GkW+IrRJX zl4DHYwrN6EI=PMW4E<6fuero2mvA4UMJq5i)7)epXyn;=e>z3@9f-LGcf5hMl*Uci zj^i)l8w{96&a4mrQ~Gl<d_B4T#TQ@fVl3&;$Z+9NT5tnn#-3IItdi@wkNUHjL)f=W zxB|ztGKYxwPhg|$@4Bt)_DtdF)gk?h+O>lC9!c~%TH#{M$B;EW?N3ttH6-F_R*bkE z%xs+9eK>1JJlEyUi3|T4SYbBZx6y2}B_?h-TH3hruKPE(H$8SVQM-|~4Xr_@In|BW zVgnhInnHim#YFuiJF;qqG`&6hB@?p%o1y+ku}Y5rxPFzA>{ANai<mpMN)Ah&m%Jo- zTk<!_-zA?&{vr7SbzJUcp*q;Kj0|)r6iMop!E}wy2z^yL{narTNvVnoVH4i-jNDD9 zM2!2QMokq~%nwo>BNe-q$cmhZ(g6f}5CD+Sf>5JC1{YNhE(3F0!pqbX3(RwM@_N|c zFzw=ol!l+B7sM0Mdy|AsMx{H<YCyb#Oc%ouRN_*U#w|Kh&v4I<xMbG6hFt%Ynqft2 z_lELnT+xDAZGhS+q<JzQZ)6b`Ph@ff8{D21^~2Cvoj^|v?WuLlS?ZQr1g}qTH)zR6 za+Ss8;yIgC=ak*1FgOglhY7jET6kq)qEOkK%Xih~C34IbrCnXZ#`1%tmN&?>Ql(76 z$#hO*p?1?0eXP0O(<)bIWm(nM?>D&fvK;|!P?al}G1;T~4{9s&3~cWA(L?15m&fK{ z)~>Hj3O^K`+eU6-gO#NfAS4*o;1-7UNR|0&(@~!?n_WwQKqAZxwy<UOIm0U!CThKj zdVP5By@#(ohk-AQ56e7iuOn>rJL|JM&?c06U%ORPS!-dO@oAf`H*?OVR=v)~F4S5z zN+5)YCd&}E8gy1RrguKlTO10oX1m^K%4>6G=~)DM_>yi%EXJsGuk#kUP6`2@0mFH& z*Y7NFja4Y}-Gp?I88a-Qs4d@6Y3k4^;uG$8HkVZ>6{d2Ts(+j_*H>Op!RM>kkox{2 z;Rsw5Iu&f8xr|1}tTY4tlHM>@EiDGFo?bbl;~Fu({1Z6Pa>+DgRgwURk+FuLorv&p zv=R76sC6XM%S1>W=qad%1G_wM3Sh6nDM0zsc0|E!6pSFE;zY!kd0?&wr8l1tn`~l0 zKjN<7P2T10Tav&7>10G6STwUFdt$Ckoo6!J;)Qlku~Vxs*jOESa`jr1$`w?}mAukM zx|OzkuRpal<Z2r=X=!4`?F`f5X=<t5y=q;Sb7VuSIarwG_QW?F5k3=+2p_$cidJ`x z=qfa6+k>^<P@a5CW!^oxmeBa7v59DjNox%hL&GpDv)nAp`CT&g_Cl>rsm`;T<uWs) z;grAxF*>czAm!Ag(3+p`9y^Z2s;Xjy+&E`xnc2|LnIxpPt&XsPg6uUf-7ft7w~JT& zfw+4o-?d@ch@?j;51V6l_vA4*Mm!^38vC%}t2Q0LXa*LS0U5%JS+ZNQ2IGMa4z4Ku z1XMXlM4({XWT3mXmejMX4KfvQpFUQG=p6zh1P(#hx0TaeK{z8y&FKjo3kEhe;iDcE zfcF9NrmRd+z#75I#zyOzI${$C4z8egkGJ98@%p80<JU|cW-Vw??@@)r;rU_jo?D(& zU#AYwZx4H}>)mt99&dA=tEGF*_>L9oaR=CWYsR-P*G_o6S+z$z#(P~a{(6#ymX0~h z+zw|!lNvkPaUB%ja-FB?(Fv**Bgd~HFZW*OO%_;My4Q{$zEnTq*A43HRN?uNFg=hl z(mS><pm)SyZ_}ZPIy~6gneFDnlh=-4JH6GN*DM~Axh}obz2j9YI9umR)%Zu62A((b zEZ(Mk=ZLv$GkH6p{9^nahJ21wkUwmmWFcY*Z$n+v>Jp)!boM~Ci|rMz6Z8QFl};xW z+VC;%K?kAOOY{Zm7ozQ4hK7!RFs<OIAU!GpmllpI;Su8(Ht@v8MltN#2yehAjN&M2 z{2H)}i04Aaqn!)TU}`&!mSJL}_emJ|l00-Gi-Fk*^s+fGSJuomHI1=zK33Xc*`hTx zGxE9eC|qWQciQYmB}b%TRaN0^M%k!RI5>`B9d6c9mQ-&9ZPv@IOdauhoi;5;SiiX_ zWHK;M)?aq=IP-A2oqKccL$m)pH~*+mz|;ySZZ3~)-BsluH|nc;xl+!#{ao9<tuwA1 zuk(eK<l6od+H`?T1JZlM`#}9IV4&z$g9_t$7V%+b4<Bddjhvwa_dPes{;<n?iGl<e zm7L=4l;FnV3{nFlW%c?X69fVdpgRjwQpuN$aZAe>QcRBNG&Y@@wdtJbh8!GYyZ)Aw zzW!rQ{z<yr$6V!0f9yN2fvtX&Y&c=i^fr!ls7Dsa6c{>;Ot{z+k{O^#r%wLyJLxwd z^XJOJx5<aJB=zihi!8!)G2R#sO|%1@7}M7(nTVFKT5^$OkK``NBa%L}hm55sW1~dR zLdKvUS$-gX)7(xLDi`7)kuQ@%k|>eNf7|~5`*>4^z8HR_EXsbFq6_{Qh=&*U_cl%k zwM=iU2Q-PXbe70@^dA>Q@*j7JJAQ6|4-hly6bGu#Guf4I3#=NJmMq+jRMnDLMGTM8 z6FZqoQTr`j5OI0-s_>JgLyrB~1ISJSSW>S5iIM8Fd`kT8G)kmiG74kB5_qw%knBSo z@oyzBOWuPdb_$`9K7a)3Pq%~9W`D><G{$o<wXhT{y#LcS`!nt1=i1NN`)PZ^!-?%r zv^)NPy#D{%KF@xhv$y#qH3*%Q&?V{1(Epra|M^dAsb6~Fnd2BV*db|w{?nDuoFJ~z zW}KuA@PAlH;lTSrv~x+Z6po!fRrvbnRv$Qhz!2V}pItpkX^Ap?GR&C={n^v{|A%9T zTnw@Z13Z(S>*IUiM@0O!f@)4ww;cr6QD5gESP1B%!6;MicH!*-Y@P77+wB?U{(vm~ z0JN-bp*I7tds}$B|2Yv_ml9GUw621L=mG8zKA<Ojn8DU%>?tYOyL8Y$OA*gF20al| zE!BG<J5s#_@O_9m17Z)-sFiR4wu<>;U}OpgXwsPQkfX7WgsEmUAWlI(Q%5G%c5JA@ zvU7cnaQC>*j%_XCf?T?a7#|JPH|92fQQw$ue`M)hN67HnNs*fMopiZ@%<df5vaN$I zxLD1ikt_DL(dCWlqoY?X+1oyP^^%(kpB6q@QBdnqbx+<k;pned9UZx-rv1hhShnq5 zF@N>w_PtA1jc&hb32b{w#B}vxOro)&kk4QYrL#`LlzCOWDbu%nMm`flvZfG|KV$j$ z-FNRE&whE;GvWRhXt!eH;b*Q&eRI=I-{8}UJ`2g|xFh(1d6<`@`9woMA|kP%%i+S5 zK1F0WhSZW`Qt4EZc`V(MZsAXaeCedS(Vb5ELclEaS@QrmjTB5H)0hpPEE5EQNlSt? z21ITlh|EwEWF@giEs@COAQx(+_op}^iJXqHgK<wZN*lX)$U%4xp2|b?H#)dYpNq$H z0iZarGVp+aDZ<<f`<XcTVga^Uw3{TkMp!H?=3gDMk(XINB(w^x{0|YokaQ#+LXu+{ zeqa?OjY0AN{`c7sebyJuB@cvhdiqG!?Mv^y_ull5!L>Da5asPlpLpVlbgj@6s?#6S zYL9`li=n^zx)AA&B=wJxE3xcTD*N=wh_LiAeKO-y5#$mc`A=Xw@xj(!AZfrCg?F2! z%%%|*5?(3e55O%Be>hdJWqz|Y>@NYc35+My#uxNsQ%rG0cZ281FRKs`l-S?BR7$Qh z-dVrO@Xl=E(CcZ!zjWz~bC~pbD^8Y^*o%J<{*O3DPI*%37d~UUCSH7g{XNT97LQ$? zYDwS3-Mc~fzXjb-ryofsKuafo;|MWb{O%<N?~=78cO87Pt@Fw^+)bONJk*tS=h_kL z$5lB8z9GJ#*#Eq+|K#CKTE#p+XZgRV4M;Rj;^7MzWbcBcY99FEF=F^p4qY*WN~cUs zY>5q#oGdD3s3+{Gu!C$mzxRqo(e`nj_uaPooI_7+V3f_n$&KXNEvegYzVOAmOI2;f z%Txl_vJgS~zx%NlOt`B5A1jvKoKv>6a#W5%c<Rya>B9YQE}Ng#F-&RRe*ZmNFS`A= zffzY&T}2~NcH;d+T}$M2l)?WJg&c4iEkTi+0V>Z^9RNlas=*@uckms`6J|+}MwkVl zE*N-dTsD!&Rw6C9;`uACcs{*j*L;_2erJQvcU_02%bc~Ubv}FK!A+YVd~oxo2X_nq zIxLJ(Kec`BV~&r=1*4{GtdwIw_4r|;;(YY{D^5OnWS2C@x2K~s>682AHEryBn;yjZ z4?M8>3E?~8cUvB~Zsk;R?@dJv+4DFYRsX`H578avc%LRj22up7SnVaEaV$dP+@Mb2 zq4CIrhOk<yv{1z(Our$0M;S}sp7uM6nufM0!ngn%cx!p@vSq!3eq~q1bVpf1%^HWj zw6@D??qd5hrm=g>SI?M#gOW_%ee~$=YyOXUUtt<iC%mzLHQTp(|Mxqe+p%NzoE>a- z@3Q5iMlTbdyK_ZVk=cxE)U2`ldFI@H5%zHXu&HY<pVxo%+TBNAxkh-BgjbVSLG3Q| z&fc-(xt%*CU>iR*LHY$S&l*@|^Pwk?pbS!QI|E{fuLT9l>Vn41g5I@&W>ri?f&GFo z2Mvui(Ha1iNH}VO&g<oAc2F4t;36nVu;By+uJPrN>aA?EjuED!@2g}wMSvNZckt@^ zbBcT{_aqY7%7ddWm!=M@i%rJXYvdmtmEHZ<%5=2wE#Ya?`{vOxdvUPHUc~Hq)u^&+ zVxd}piz@JUQn_L0+rqRxfv#aS1_Qa)SFTn?$r9m8tB0)&yDHj4Q)OzVO1NO^@T(S# zL(0QB&KiTUe&dAnr^5A~AR?Oh+sP8L@Ls*u%05spT>iM4%=WoC#%#@Vlnc)Y*M>(1 z%>k=bX=I0!#ZUiZtZ<om!sHvKH6b*Pb?J1qnKqKd_>{s3P3^i(18oF$Y@`P&pb7q@ zvO&%Rinll&IO>Nvk;2BP83HY%nxOt@^RQ6}1388?OVhV+Wsgs0?25ERVP|+&EE0^` z9;D*zmtfJOHEx^cUSPX*CM%hFt8IaM+BUL@o;Mw^gE?}ONuG9OHsL}9goCExOl6k9 zcBF9hZPPbzo-Rz=Cbo417-4=XMb6q`w5^}k)dn8)rye-Nvy7(}Gh*3HgK@Lu%)3+n z3oI%!*v)_P(IJ#lCcqSZfges}9(VST_vZX!8Iyu_9WRljFOkeF&%DGjD#;zAuOeiL z)kL;tDxm*yaTD@D7Ic(j;`>P;SyBFLyqBneU^?`pM<(c}IK9OD2nZ!U*T9lL1{g;P zQHC5spChCsLWwhCBD+2mm(S2;iqgWTOcCcZWEYknl3hS(8+Jq-!Js3u!vGXFx%%`X z1<rY#5E0H57emGxizFAgB#MO@V8aq)k0`{0ItI%VD5Y^YgQZzwsR^Ya(VjS14j=&z zuOmZZCAfLs-w7AokmIuQS=*QV^#HH|?w;U3gg;)tyC~?8dDm?rwpZ+0M{sd$lkmWO zf9kdQvVAJg1;5@$^5<Dhp4=DD@&l2UEYpgBBG>GZyXL7}pT{gaax|rmpxnPf6C{R0 zTib|2S=j5#k%yaW)!9?dat0A<n-h5qz>=*X;8^v`SQ&KeDAp3DgrAcLuh@xA;PZBR zg`=d<4p03_tdo51mGomi;T*5<g_jgEtzNRvOAKE0Jp=w<u8?pmC0fj|G^2eDF#F>W zBR30JjLniAk}JV|c8{b_@+!PN3ED$3pu<0a5gVJRMq0Nr)(md5j3YKqt%Cs={mM&V zt(QUujwTQ>MqnxgM4FbD0^omUM`j%X;ov|kMM@GAVteUvCTv*~XK!V8i8e-rGO=_w zoddypK}UkYEyU(oO|oKfA7hGR%Au_RIi%5mMX8P!NNn^DF#hO?MyUXe5YZ^CBuAyz zAaoLmQ4tEOMf%#4pPP{;jWHM)?Ifp@kt=LAg`7AKI~*z{W3ezw)pVPUQEMy~jk*Wh zTB*WpR!FsEi}0SsqLk?wqmj|el+#Tnl^ko>maAr>%xuC2=oZxEl4o@~9aI9XR%h1D z(rWcqJyENP-l}^|YjhfkRH_Dq0Csag*5}@Ne*Zr;M)&xhr-|1PuRQ<G+rMH|O08Cz z{?}JZP1#YVT`xyCt6rLZf_z4lGL=H6A&XO+6pBqLMa#=CD3s5?tU%BF>|g&-ss8aV zHQ)cOM)PgI#`o!W$Vm6yr&5JrWzH40e<xEJzatxrs!>ATw{n%~Tk@(&l_f~OwphL< zCqVa}HZY$G%oj?XR`mrDRG?uJ%%7|Dde!ITbG2SC$p5Y}8a2z$XEq>ISjNkZ>1)ov zgE4B@ZHNjMe(1B_iMB^&AdI3IXEcx*C<ChmAe{t;8VbdzMNHR!-Qp<3aNy9ga>hj7 zB70ZAgoM~V!p$$OCVPKo`w;0RGhZ4!{v}p2VcgvrJjUJQ`tKgHL2`y{a5*?8l{pSS zVw`E_9ZV7@{DRZbcUGeBT!b+Rqb4RXao8LXXKXTqpXO606l_ghxNxwE%@d7RW<mR= z`Ue~sncxs==LN)#+F)tSK{bPR4D&8t8w*GlT10``FvwwlB#ysPGhz&JD2%@_?;>#3 z3UEXjf7lI6*9ic+0Pae`^tPR>QL2SMsL3oEYnGOP$E&ou>S`~7xQVo(=)(GU4qQK3 zr?C@W$tk9f*D9E@M03cl(WrbDVpAIxG#Fl;5L{*BOWVj61YAL>qYM>lvf-j@87tpW z>Z<TEtL-a0inJWJT{N-LyHnEWn7diH*rO(u>JvtU!o^7M2?;aC>6H~*p<IC^KIXSq z3nyf1!YQ4xdHD(*!;D{g@usdDOJX`pA>z?_@A_f43oiSGu}SQ@oNif|jUiqc=UP!8 z=>_F32*pk3PFPZ*vcpA%CN-p;Wxmn4U-oTG7E0BO+K-oF$b+b15-I&yI4^>TevPA| z*`O%f1ySQ{Y5ZqvdO^$W`%*F%#Lt9hQ~Pdj5nk<{#WM`}1&EZna`}}EkJxL5;b(RK zf@)(^i_(k8hi0c<Zf66C_GGR_m}0xxX`y+9`78zFPrs+|feg=SCJzHmc&5K)>S63J zs|Oki5QJx-ntFo~>>H%pY^E}xqM$b5MkoYvA@~kW?9WyLsNftU=J84%FU=uI1-qz& z1e^PwZW2CepU0^YenL2@YGH@)Zu1jQ{eo)vbm78VWF|Q$<=}w5W#K|%AkIaL_Q^~f zi|eTOp-#ROKBVnH#1e_)P3HY8s08{;dZ}0gP%Po!hLQr;BV~334uMWAl-Bd--#Lr4 zPP?Qdr)gAseNmTi<y6A2UQ_0)sPp}@7raYnOLdJ|<1#Huzeemp2>QDw`*c6`PC1Bk z|3&YFAt(-S5J%N3gxme>D{!fPNgp+SjP6|uarzfLH$e)iK6*+D$1m-L*m8QjAGFH^ z!4#H29_}tYGe9>0-gpLnEkFNVf|O((Fhz<rz-WqL&;*)(OzX^m7|o<ozzNd-Y|Vci z15-#I#Ozokzv|tq<d#{fQP#j=G1nW1^aH2=ke?gKQYj2&uL!rf9AnQtcdWxzPdb4$ z6r2bwjFi!Dg^6xpzRR?~N@S;<{tLer1^rUF37IabB1Cl{c)J1W(nIM)FSJ?pBI`;N zMU+C8mf!^F+&}we34|P)JMvDXXF)QSJN4wC&$MNWzT9l#4YOy1%hvdW%RF(Su{;8f zu`&~5GXH|F&lbq7>0>mN<XxiO_%2<#)xM8hx6dLq7~+=o!c*%lalP&u^R)hHCMFa& zS6)+TjvJ{=dUijh*4JOwrq8ks@q@J{vSEUq=4YouF$=LEF4@JjTC@t`d!j%x&0q4$ z$YY#2m*5IWotf^V?@yH7M2WWlZ{I%w+rgAy_-1u9b`)c+jrz_Jj)?FTKPUynyhSiG zAUqMa21vOO6rdZ^W9iQ5&%QC~kvmnTDpvmR!*UiYPWfN8cv-FXuV<GFt28?1QpT#O zANc)6HG(TgaHVg)!IdMp1Ib4Su1Pq0c6nNgE3@>{pkLJV{|+nAL!+nm@Nc5q(1;$0 zM^XlI4futW(0Z&+Dmx`;z%>=+F$`--08{c%b07caoO2rfcx&P4E_cI%*(-V`x`@j; zY3;gE`&aF}^~k{oo~)8NnyMR&zN(UV^8aqFW1e}|cCqmFEzbNRLwxxa?}In<yqDh+ z_6T1*+q3^4!%6H=-@lIb?b}Q^bu??<n2Fu9rabn>fKOla<+Aw3N@!C?SkfJo8^8o_ zI-fw6;_#rs8M>Q+4?{*lf6ip$gGD1_2)F*3nIb$OJoLNYv87o1MtGo;=rMVHc^Mg* zzJq)5cfvzNlfHv34fMZg$+Pso7znVXSU~|SIp>ji?}fH(>3^H-I{4m&4?q0ywD-t7 z&`*A`g)pImWS4M#Zu;G9Tl!s%h6&iR8RREo0+8h2rQ~oF4^Cf%UjrF-Vx~<}RSZ*I zE(2MIVn4)+wu!iV_&KCBJ7WozHtAvFJ})oAL?hICnfWHzmC33lUvkOkcX2xQWGg~> z@BaL}sp{L$pV2vjL?679*l!~z{`9L2m(0`GtD8C#ot^Q#F%1oEW0p0nz3W%&ub4Tl zv7>Bsdu8sZhQ_w8CH3p>X8H^Mu<rIxi1P1uocCCHG<U_VRW}^IMfm(Ssa^PT-OgF& zHf=*wdAvENxvQ%==Ze*9cb>C2*;raREK{(9zN$DD5BT3H_a=?1Nud0!pn*^pUZupA z00^Tj5tSm3ES7<&%$QX!=9c9_0)sU3X6E^ShyF8t!uA7Cb=}?d)XA@&a=V}EW*W(c zOu_RclPZ>-{Zx1NQ$Vf%1X5Uw9d3Fmy}|)ud-_SSfJENUoGgFpK<0AjCt1h|evE%Z z;>VXe18_1@Fu#N{v}Dy$lYcahh+FBgOa3nO3B5w!-!FNJjDG1I;T;eXh*@fdciwr4 zjDCtq-A8v`@^_NF?=`aGOWz0iLhnbEgMcy@d_;QkKk$7ipcWA}i23ZFsLEMr>E*^m zNiljMCxS`D0CtQ<Gz=bkKMmiYre7M2rvpy~U_^9<GE^YqT?}?F;Gp%CT8q7Mn{fbJ zZj%4`83c>Rk`;cwZFtH2PC&AwZk-Esg4y{wTFw0ENVACmqI*lPKgx2}QEvCVye^Z; z7cdw4Cy!~hT58(tTvkqTwpOE+DP#Ggikowbz?sCpE1Y-gkZ|y`3z*$+64-JWdFkBM z*Ij#OYe`h^Gw4gVEuZc6IEwvFsdR;*#px<qlI45kp@L-rf4J3PSfL&6@l@P+D4Fjw z`wjZ|hSeLE&Kp~6FlfLX+E_BWeeV3Nt%&WO{n0hYNmoYh5!;#?dTtE%s%etNlC?nn zT#w1l5(z~CLBl^Nb)XlLMf3?9ti`Ze(I`e4EEqrn*~(coGn7*3jtxx^u^BRed}y<z zd$BCR1#lS*lmmxF%R^w&#cmOMi4zj^6!9$D6aZ>I9Sj47n+C_64wj)Xcy{3t;pT-^ zp1g)@-ZnI(|2o#{s+>8q(rfAp^75*M!p%o28Vqk=(~!6B6Rq}RU(=z=?xM1(WkubU zhnjpJYqg*F8xK`aD#}}&S2U^mP@|C3P(crm1S=Pk9!@{A(q$bR3U-;imDb8&gx;j0 z;T429XfFCd_&s7}e*eKm7kxl#5W7<I!Gk~1LFP6}U%n?$TvGX~ya{T?;I1k2A0#&% z^jB24XD>Zh_&9LS%OJK_PssaKWeGE7bk2mF(NjBbZ8CnPRDNY_y0vqvSTwEU)@I|E zO68Zv=36_MNF$?~kh8xcr^0{F%jpBc+=KqI8uz?&m(F%qRQMx)?AV_(LB-(KX^Hq` zc*ZkN%k29pbUyV*rbJ(s3^CW0uoy3ptf1(|FpOf9QHdS+wI<@yAcjwBu(VmQ6c=8m z6b?EH45R2<ETZM6QfdcJ1gU((;~b<8z#Lc_57IK4R)7w_r53>0DOnSoM;S*<`PnH@ znU-mbX3h<@cXoy%caE$qshO~<IxmvFEY{qSd#q=qtGKwcTN!S>gkdgW$q6rpc|}mM zfW4fn2@zHg?ak<`h$MyQiiQ`Lv=lS5hhmgJXsl0?YsZi4E)8$=c$QBnnXh9F&2c*$ zo}1qk)E{n2YI&bMPp&&}lpO)v=eQDNTY=41B&;b>thIE#&z#?7w)+at2l>OB;qvN; zop}qqD&bJPd~C*5L)|+2Gh=x(#-YO)hiLs$8|GplsgTtp7@+wT*fLZpU7<aLDhrSl zP%*qk?*|hJ>J+vUEW}w38eItqmZNf`rIh|C45<UGR&nt3V=fHH;*_2rdrB=dRhR)6 z%{aHj`HvZU$IbWbsjjKsuwgaPgmU$}HpHU&trI4+<_p&jTX$Z=!%g+WXFPd%PuEQH z@VgxMF2_u3ombag;*o<oPil$uNdG21psUo5pIDpza%5%I=uy>G*4gvtuv2ThuDXc4 z_`F(~o4xr#n>-TrA-kYAe{7|2#8J7Z{f-(gd;Ga>&c1)lWrqs;pUj`koHIS(pOU_D z^8LS$#%g*dRg)QD^LVnOJea-VNlv(W8>d}4abi{VBvc^g{(<%>=A~8;kSobx+W^dd z&`(FbE}}m!n<$swWH;yBxQ58)FmSG&`4)_se1oQtH<eTfCgCu9>6u;oagR#y4*UV% z$RlzEQQ?Bxx~K<?n!O@`;$~!w5)Ni~?1Ur>CmCdnIwnIbM2*apCK_K0`0o;qZC^gB zrnD~peLitnc+7HIOQfYaR@<sy)=U1mSNP`UWRu0l^C3%V<v9<eIyzDhJcXY!r6!WC zOitVT=T$co4QK9M7Hk?N>=5i$KjSiQ`sTL}ZLR4Z5zHCAtN<PR+r3&oj5~9hfz$DG z&hq)@m|1IDzjO7U_fFl5dC)j84=%u@NM#`aWfPQI1kK5dG{3abQ7#}$#*Y#J&u0YL zfhhkq#comz2`orhP_Hgp8!iCbA2hi*3JbwPhoi(witCA1ZUv4)duZcjhc39FxT~fl z;4`a9!pL%+6Qd!8#b8nCVGZM}+stDVazMkA|BSB~lhnzzNqP3YUBTv6;~LC9RkfKz zvdW@08lICI+p=MH0vY66Ccn8#Z>>{bMsjN!6PEI-ku9@ESMg(;v}J0-^JMuS7w0b5 znX@cD7-?=8W)2tRaCYfAMyrX35sT!5f6!STjzv9;6_lBvK768%HD@<*NHttQXnIdk z?y7^F`IN{L?uU%rCUVHqK1zo@akLs-EoXkZnBZUz#7i_Tpn#3a5+TYeLYd_#dc{U1 z(h#`k#S*5uBs;gUF*loal*U~7`L0;$=f#;4=AN=BEs2&1-}$2Zg%57C1^v#VI#-t> zJzRMAY0~-3eWdazv*eQV6Mxv<Ob@t?xcA>e+y^*iS4k<T<b)aSynL5CKaYHIcER1; z4Z;t5N4B%9k~1)y-+BSr<ca&X?VqIU&wp*=L+6Hfd`-Mx@7VDT+7Go0oI>A#R|fn- zu&3e;qG3vLMn`=l-=NG{P!dW@q#yXDaL&2329-vr{@Uo%C`>lC=j2i0{4mP|q$wR{ zgn!v%CnO%Y0uBjp+Bjf5$TTk4KkHU)cFe@~QB_pz^SCGfJ*?JQKf0@!=#AcW;GQ7N z<N;nCa^Lh<!s|wlR$;OVu`tU9rQkDo;hht6$!K<l5IA%9H-tkL@$SF>oi;maX8SBB zw0v&=GnX)%`~NoZ44HYcOdJ!a{DCi*(Pc}iWH`|I(H=k{g-Q{v<}ma?m=r%QWf!J} z8H0%E83q-u1cZqn?7c^L{#>B=FH!3BvbI-O&wt|5F=H-$V*bp7Etk-A)B;d}v8Z?J zB4WCFFCq`qCkDZL$3!R|>lU7)++0^}S32aEDj4OA`8fRuuF~3gDH32)EFsOzy=Bgl zbuV3)$<mete?bq7C_j2P9b`U<BumTaML8VgAwP;H+bZ;DBx_LwpL>8@b(Z6hmq6?u zdXVtQzxf91Fn&M9rzk%aFfXVsQ6;NGq(q#$=}<**)WJ{ZWib+A-;a)nqTVnf6_5cn z4t)>}4PzEXog;w~#$Z1ki{Lk<(<R+^_7UQBrMvR(&(FIr&+ZK5Rq6eNXu|1f5e@QK z=#wQehrRGnLGIl-His{}T!+0FqvTqJuBISlw-+2L$l0Ica0GHIbU~cxWF7~50|^?( zlsX)^WbQq=4o5J*3OfV(n!=DhC6i@4d|pnew5%o<S(F^7v<N$PT9NtE<8$W8WI1-P z7x_kQYbx1%w!l>qh}xw}&MofCb9!BjRB5?P=tIsR5L1!lWmvIA=!w|rhUdd}Y5$nj z@Zd2XuQLzdk4WtBzY3^hY>D1*R4J-QL@7{T4h1Gs&|F;1!b2qrcn-4Ri{yl`y@Yd0 z*^pzgBXmX3x!4)Jdgi9a<fKV!myi<jo+HFJwRw_6>QKc`rW~P=gL~>^9sMO=stc>u zp1<?XHLJ42AyJQ@{e|M%2J?<k1aT9PQ4(sSrs%?<9w&k_6Rf24mpyymS=%&g5lvG` zdW2r4kAnIJc-?yYdygHHO43=!&J?mhoemzrB##~I@1<ta!4e_?n*+8PNeV{eREqo2 z9s$_ZGn8=yhlyigW-JJ!eoL%0dkE@~^6X(G$2=OQ^J*5#F(&fRg!po<5A1BhcaJl| zbyGKC^@2@P$$W;=z0p<SIye&MPh}+ZkjF1jTMtg4yCxj8sxzly&5pt!_0_N>E|DPH z1|+>G%%}<4&@;lb7~m`>2842kdFnKRX;3oaB^xJ=tNn^$zN#HJY2(KGHZfn-jm65O zv2|Y|sE=$MDk`P#+f=niuhp-qLb%_?NizMK%8mDJtX!j)P1?vF8!9)6SVmEIG{8bp z2aE9}WF=dHrxwk=qJ>v<Ou!m!qda?v*+Xph8AT3NCjw<ns5mItXqNE6BW*Bs)1Tfd z&$L}?RW5-#4RbmjQQ=h=CnAx=4z3O}Gt6A%SbS%m((KTva*Tm3;~eax1>ZKCOv%Yh zo)At7f2FjnBAx2Pw<Jw*Q}aosTBDqzP$-?sZspr5yQ*8MRJasV6j{KxP;+yS_SZ(Y z5&!l~p@%AQm=k%?ZzwS<^X^>iC{psVaa#f^a&N&m&A4FlmWM^^S9%ZFIKlfmIcYLA zle~cwab?#R3c6H?C69~O?j5+5(Ku}I{&=DcPF1X14!C@Ld06RKKXaA|hyZ9WLm+u1 zYU9HRsSL0LRFN&gn`8*8j+(;EIWTVc&J}Lr|J??}oqO%vFY7Pd{Y6}OUwA+M#qNvh zzMOllm$Y2A^8D}4UwIj6VU8R*BHYKNenP=LIsAo_?BrvlN&QmChJE`sbiAY%o;Ws{ zJ^8}+nDF|rXml9KiJ>Kc>Yu7U7@IPDQ1zHiY1R;GVYn5!>kiY=A@hYZ6D5!jXKm9F zjgDUbX@8jR^5dZ3&mH;m`~C4Uo)bA9>NwaLyc_};espuXotf1sT)&St6D)?TGRdDT zPCw<2Figb7ochV#|KTi>N(;hPVQX42l#br<AW+J6M163_=u*R(t3+Hv$`zr>CNgD1 zvWp5s5{;f&-4$_d+2V?%|A$k^r5fdYhRjiF3}qc7I;+Crs?HH`C`>$a*KxQcE=)hS z=pzx^E@g3}=pCRZL~ZT#1ON~Xut5lx&eUcc*{uON08|U3d`6q&Pp<)B?F42E1NRRy zJM%GAHH^}96C?Sr?6UqhDb*1YaDnW1aE<FrV}H7j&ICXlFR-PPOkY1dX<|VVpm2$e zB#s@UbtW#wF&^zy)Bz#q!v2l%g^LnQP!S_6f<S|dh4_{JB^(x(?C!lys6Hg@euymG zP<UJKZeX4lwugnKpOUL03%TmZ0wAApOyLIckVE7dddyYf_C+}NQ*tF{61<1wlv?_C z+>>TLszQtvMYxNSj>v)_3QAO@Im7ql<stt7c!9s=HejI<Cn}X0>1+=foE6>vkVT=e zML-E2DW}+g0qxjgNR(UI1)Cq(jDO_2P2H0>Z=T$}>HXxWlfN2Uojav<O6g?xc&6DU z<(xs;629Kn9BD0Wt8IUF?a6VA8qc3Uaso%>ei`8=j+%dd!-BCV*E({dFq=<r;y!7b zZi(iw?y;*^j~%ya)wqFQ;W%^S#J1+C=^3_w6;Z1kaZ6c;a|&h6XQe8=&1b)Mp76JW z%Ys>jrOQYQES*I7_41O!tkCj<#5M2QaG8ryvdqK7=gu9TZr8csspKTHAy4i_ol!q6 z<&!|m64QwpObHr;Z$XeC@yn?D)x@T*VtiL!l|DIvw7dzSd8F_dSYno+%Z(I9k_YJj zv|M0aC;$HDo7~;~Dq$pkFC_j<8=icM@OSfRWQ@v%95YffhmKT`I%QJSENWZSf?);l z!poo|oEX;_!8Rr%>f(a^n0^QrUm-z17`_DZ-=T;mxdE-G&1&Sa35xRsy&xnq5mJN0 zK!wb!qvfZ98jkQ>%^p&%D|XmjyV>G3!aoc_lNykvoS^23*1T~x2U{uIUmA95?=I9L z*Jlw~^}<NGJKfD!S5*iX%3ax{vnr-wBg~DgM<+bn1QcF@*6Sth8%q@KaM5n*o0~yk zoF!LG&(T@bT4no#Ehd#h8TlZgi%wz!j*^atnwd_I5#T`KaQW+Itwl*Jot6?yF3^3A zE}4}XGuLdG#7qdfa;NE(L1X%N&qdA34xJqItk%L9nRB2~YBJc~q4+79r-u4*QsTEe zlMyZmxwQh{5noS~^60=orM;p5sSJx?UCQf#z<{BY4*odch18QzlftD_yafL8A^#!d z!O1O!|69c)9wfDN^bbJJ5;P<sbqWF?D-qAb+JHY^{OwP&ef>!~T5!peeSTkrd+Vf# zRppW?oSGxi$X>^L&`5?#8hsNQ=(QGe0tS<Sh=7NPNb!Cm5f~VHC_)i23-mxV03=1A zmKPIv3$RW_nIDyN=Gd-{T^r{!9h<jq9>E&-C`W$&(dQ$TdnBh+>We?VZv27Gv#S`x zZY2OyBt_P2SMC;6st1M5LWQvTL6yp|2gJf0<7BwUm3uT-o3rxrvdkMw@MpJCqwJhC zsZ*&j?k0Nqf?0WWb$PpuYUTD_yS6LUDAXx#+PCi}1wHVwKmF-3dLTu?Q9A&nV6oSo z@k-UhPdpYrmPL~F=$s-#*jh4}6K)VM{Y!r-HzX`A;+Gyg=WM=6{lGoW=DZ`R5fm3e zUJ!qT%nyqa{2SQ%$wGES$NUcb69&&849DX!S%_!9&{1|m^t$s{#zpXjSU!ThAZ`em zpMkBPEKH+)mURqx;F(k6X~?W8PDi4?A>1LBv62%KdYqIl(To)^r+k4rkHRibtuKrp z+A+}kFuI9BP}DF9=o3}v!~q124L~~#QGm2Yp#;K80}BN8x{HW(2&G>btrLYno+H9@ z35Jh4PFn1&B4`XL_{g>k=KW^r+_+su5K}zr`hwB#F1xI|d$y4oOH{&}z~X<*=X;n5 zfz3sWma*%`tr432PLpt_&gu7BDvm9EuOiIYq6=p1X{ncj7rFYuMO!}UiUBs)BTs*) z1o`Z5JrSoV`*u2pM+f-Tl<-D7;B|slWs{gddl4xwg@uU$RM2QL(h>#HgZf$A;YVLG zl0$wIQT7Opo4-^W&Ft;P9i#4#aYx_(jN}G|+H66>&7adGyzLmnne=3yCCIN}dz^55 z%q53NnLa4o_=l&E4%Pk62f{t%3gK|tBrIdDXQSypVUn<cDPjNuATGVbX%^<8rNj0t z@^H~SN_J$GsuQJG39Rs-z=p$+M?i>Q#)ZYSK&Dbq7n*`JDF?m)27D?iLX(kMOA%T@ zfiG0Ffqf_p6^<=Uz=~9Qb}N=Wa;dfq39?xAiLF(tr0^|+?3lV+4bD}=FZvDP!*|ZV zleuo#==FO+)Lay)iB4#-+S-?Fy@|QJIIp+>9J{11)nNVZ*TGkL-3_oO9~YaG97`l8 z*{J|YePRu82%1q-h4#rUt33k4Y)Nlow(4E0rq3O23t7Bbe$|x$vS#+eW=Ftc^%IBu z#`5&R9&0=M)JgGTyx2DFr|X7BOXMQjAPG%>5=Me~z-O<M?3Fw!`Ge#`$q&eT=Rlnz z6&9I<tc921Tcd(9$vL1LMi?V_>XC8J2#zo#gSvuEokmLq13>Ks;moLJ;z3yyYjIm? zg0+BGvYJ>*qa~#P6T$wBIE>PGX-G8vh!q|}3>8NeL~*NpU@c$^L@~tDK^DVraY>x& z?bc$O#cGkc2@KvrDU$WVlNFHR@nrPQ)cb{S2>N5OmC_7h^vhB+a6Q4DaVe_5(lU!# zw4+1&r_Wz*i%LbWS3HQz&{u#fCNW?^PSAZ(dZ*GecfnPx^t#xIhor9}Uia*q{^*2( zor4b~3k1>VM86!(%Z+PMc6V6DU}B5XdIGL@P}a@}*xZcN_4A&%c+8lK56{0owQc&0 z+cr&|vU&5AsnfR3n7%D_{rtmp-xKq$XXeNZGS<WKQYnu_OT3&xCL)lch<^YMWywja z)nv3<jV5wO4c|g8vO>Nw8Bf?kHe2W-ikXB#O|-cKR7uZ5(TT(GVQ1;IKD*BA^?N;j z@0}ix!ATR1xOEQ{YHbdiSq;J%Z=uHSbC@*_zsJ8-uF;r^io9-jp=FLI67~A6TB9W( zn-kh*Q+vJO4pAtKQNPEeH5!aIo6)4#n%(}Fki*jDi6SSb_5z#Q<DrMiH+wv~iYRhL zYe(W?;0=I83g|2Ft=JNukUsUip&f`jFdkJkk5<e0MPPHN89PfjD!e*t7Re*|>lcAS z@#%&1i23tyME{#Ci!?+UvreNCDv`Mgsb5hG8a^*#cNk6fiCMnPiX-Hp+aBztPl4Oh zyHn6D*0IHn$3DB=tiNbPC^UlpZ*J0?V|6jJJs@Q`rA}qn+Rc8tYS7vYi29IOYhBsd zuG*5FF<(~HWYziASy7zd5#-z)PSo2q#2&G$?fT0GFSTxP_hrrNTFu!t*=E!SBi0Cg z2=SRH$2YzncHm7u96A(;d=Z&(Qi-??nsK-hIGvf`4q1jA<d;frK%d8~gq}<LT-YqG z@RSGwC{+l@$U=^-f${Nlm%>~oib#XKO7tb8)6w1$r@c;e$bb_`&F~Ni2jzvZn2Fw$ zz~B)d_)khj<Q2VL{NKEWq%!0|)LE@NB>ggJGS~kwcJ`<lGurcOybTQ`nVs-hTo#W# zkX@Nq6DkPHO>S$EEhn$FG)b)C?Be?Rg4{?f);@1;dk*(~!#;TB_6ue~koujG{(Beh zUbt{KVXkcLp4__g$fK)QtXTahxoGr)j=G9-8WhCenK&*7rYIphp6F!0FZDa$cKI}A zbC$PH6CR9|P9~in$MVcdqgHQm<%JWmV76W(Ra?!jyjZd}yEEKSQq&abG|$;JC;bSc zi%r_K<?^Bk^Lu*Jg>o|C*fHU5MMZZ-d!_K;<@%9@Wx|6OFrky`ijgBLxNotf;yC;P z19KdM9L-wjp>Ck8BG5)h!T0r&<gXaPc@TewfP<L{Buy!}uIONlaV>0%+sf$hTN2Lv zkjxKXirD2~To#O4g3+K1RK6xdDPT%wEeGp9$`Bglw<U0z1u$iCfD9r(_2B4)iP0vG z^$`CP7=Zn8FH=5PXb4(x_h@E>rgN{jB|EL-iaRh)`YmW(^uJ7uLBa*m(&$7XGI-Ke zN;nA09{>_C7UNiom=;}hVi~*+tXPQjh2p-!$Alh2G7T7~LDWZk#B@Y`_||eS0j5c8 z+}MXS8)x<*jNC9-9f5cm&Im-bpfa@rDJ#}aeD&mfrlGy%ww*gk?W`wa<Hlwae@@q! zT;Wr?Bri<P>$f&eubjT!agn2CWzTsF$9FQ<sSIv=)Sj)_G<qzufN-O`UOrcN=gl{- zyXG4H)lA!3;z&5&GA%PR9;CIi*^=4g6^;tW-vSRGbR4q%Ff4YDy*Pb&Ztk<z{UbBY za80TA{)A2Vy|3aGTl1&oCBkiFVxsJ*rP}vvxtwDRmA+rqr1R}ohq1vP7}jt@QI+tS z!{RhFfGJCDGqxnJFNR+7=Rfbc<{IH2jm-CRH*E?42NNiZZCxD-78eJ<uqQWe^gF`2 zVMjc+WmTwZ)HPGrZFgSi7_sGodRdOn2Z~!)$TRhvb7w7QCoE21IBHaJqI~3eZ`SyS z#XPm-YfnvNaiQ?ltN2q>Lv-MyCyzdwe%0(XgSv}M>Fy@F$&>plh^`XnrC<3lF=|wT zxwE#mprEjD7ST?yA%cmit*xpe>+<l&ia+O^lb81(xriQ_?xqFwPgRw0VRiLn&D>d> ze4^cc(iT%F0-o}GzhxHDd0~0Nw%;391a(%WY$gC>p7cuGwE}l#_6uJTU3%q&Du-Sv z1BNQ6(xHc+GOV2wta51Ju2zM;w9pK?-$vo<7hb5Tx!}@jjIK(9#}tXZhOa3(4AZCt zeR8mWs=yNvM86y>IS;<HeUe{G4oUt<M^*?FnPCw_t28L18bJvp{h{R|>5hz*qP;<Q zsbfSh8E+~_!x!b9jCULP!7OGRfuR)TpuwU)D1uEQvm-&mzw}BJRDesF;usSk?aD+N zm;{U>0}qHi0D~PqBaSeil!iUQlCV3>8lbEi7?siLw38X7Ay0^wp7>Q~U9X90Kmz9u zGh;-Yf!@kam`UQ<v_#9gnupIBR^_fSa++Z#@|2wKRLK_R@rDt+9CQj@8f=2NW>aU~ zKC^g{E;aY>7jX`w7r}f$FY=D2T_qmcXkvb7<8v^QFe+0lBwIdIEMQiJi?iI}QvaG9 zFIlAGEc-(x;`Yw!xJj5VRhrI|!-jRvUkNW&`eTdRs$1-4wL%XTJcV-aZoPtMmT%{l z$~8)|v|`{C&B}j2h3Jt^>K>w12|Y-kXd<Sd6}PjI_kJ`LQya^`c&>!bQUbiuM2zE$ z5%+bO<i^ZAj0u%WzkHA`2Ik!Xi4zi}$jD8T#zaJ67OtB_Fs=iZHxot~M{BDE3n`s| zG5CV)5?3*aGdO&?a7_)cNhVHD$eHk0T$tizGU9I}T3UX#<|D1OnEW=Q$6ZMtK2KUw z`orgq&7rJFRim+~tFxe}p(*Ms@^tx(!+XXQm6E8bXN$4GP?r~J%PP!L=a5Tunyf;l za@%fag}!k2ZsxrFSZgva-@ZGPHMX=?$SY_pC}?EwD=L}Zv!p(;a89+pvNhLi<Bu~x z){@o^3kMa)yc1@AQr+OtSdp@3X;viKQq^GB+Vnn$aandan*^6!WLYks_m^=YuWFU7 z<Qd0hY?ik|fJfCQ{D!>o?z+mdio*1I#~xKh1Nl9@bD{9rvijuq<*AxPY@W|#D%3Lf z|LDW95-oJ%uc7PzKjz*$Fsdr;AD?r})J$)wlbIwl6Vlsc5+KPWKp=z?2qjWO?+|(s zVdyBJ6hQ>RtcW5iifb1!x@%WfU2)a5#9eiDS6yFsbs@=IzMtn#5`yBo@BZFDewoaj z+wVE&p7WfiejXa<gsfyi!lqOY>4W`Z0o=tf#%Y#8W@tEJz+IKR>U~HRPH7}){FA_g z2@RTRpp84qzJ|6Tbl~m%2s1O8`iyqZ5(?E!d*MNCf_fBIp0pN>Y$)^p^{g6c-qdT) z2G|`q!rdp`_EOQ1xd-;oeZW1skI7UsOBvE8XfB>qbJ|9n@GEyp#)N$*zuR$;iHTMl zMb6o*mJJixJe)xE3Q6_4>)`+&0VYGZT=+r_+-_y*&qQ=9TDu^?KY|vD9{9zI3DK(5 zME=Du$arMS#9PPZ2`ya}-Oqi0SJ|R6){pA<?<@LXXXV)6xC;%-E2hpbl)_RdTJa4} zG1C<3HsJ*7`=3>u>P<F}u$;#sYUc`)=Md#Clu0NwS|EndUjF-S=7F&00m=)QA@blu z3*og#A<KwLLZ3)_oMeUt0Xl`qet|_#n5LfVkyh4{r_xHl(W>}GuxC!H>S(E&)JRvc zK(%pLIt!%_Ggh;J!P3mN(C&zQ%b!{2zgdp>O3i+p(=nue_40cDaryCg10&jdx17tO z(^oG`_H-m)1cDqwb`64b;Smyx)_@t0hzGhdMCC4<9`|!TD8jm$rK?L{m%e7ES5xX| zjVv*(Fl`#N^Ymjk_TQ;du2gC}db*#$3;ZWOD(u{Xf?=5$H@|z8nKTK#24ycWnW{7M zAKQD&^LZK7DvgHE{3S1zo_>f1NH&P+M;%Csfl8EPu7x`aIkw>Sb*g?XAd3zsX^HUS z;UC1y6~<^aDLl9k{x&4~;8i-HtfOnX;mQ^KYx5>mteILiZ%SkHXs&4RwL5E-R@LO( zM6u}hNxwS1`A=KMZudb^r4d&kLjbo*jB_XUZm7xw()$Npp75WZModdD;0bDHwr`R1 z_{sVCpn^HUU7WwBZ2nzSn$~Q2(Y)xssf8Q^yiQfaGpCL)?csqTYl$*OC+Z@HVq^XB zOye(GF$~=Qgsvvqt>JX}F)?~g{W!WMD}jH~8i`yrp|6CFShk_1l1@(nOjnF*SpCVK zPZ>c(Klp(l_zKcZz|<cE@}QsDmHp35`ee3N`xeaWb0WvRfmbgnLSZjAeol1X?2jjQ zJkNz%A}vbA8R=gWKc0;f-_pwG0`5rhYCt|$-Z%S`No^td+?Hs0loDvFNyccFDWsxH z*A6uxdHy>T@YCZ0<co~tcgo?_bF!s}w@g{R?vV-Hx;`CG`nT0HLHhm=*PdP4Th+g$ z+}O(yR|zM2rD9@rMMGrAwd0>yA0EZ^D{lW`$b84Z^U^;j-tpQBvB00=t(w>;jRGNw zHbmPcyBkeUMyN*Dp&<=!4Z*9_kr2sB-A2w*DIcMAtDSr>qu8;Cw5OT*sv9K9fcGOK zSm!4y(a2K=dfsK5;!ihJii?WuI$xqIGc`8d;YdoW%gL@wbJ?B#*wjo{qOWdT^k9m- zk==Ptc1~SdlEaZs=lt{%`6zA(m=DT}5dFZ2(yka(5~#H%rX*T@<G7#Q2$NAKF(Ed= zVK7<a;uDkN;-JuRU`L|7S6*YGmgv#M_~_9eofsdN;Anxlr$q{NeVcgG(E8dyQdX=x zxwr4;AwRnz_n)wAuM&w5_aOCHA^aUZ_^K5b6flM`#l<jd;X+t#3k!;lFo(VuX;F}M z67zu)9Tr!bU_%5hhU`Fy>>g=_aAidv5RVz4Y)D3sGFSTS2r^}yJIAKH`4lg%ntx|R z@g|#cj@ugfX#OhfWp`jJqBtUbHkZ4DSHKDHin0O4ELt|2GH9gHaP!L}3}X%RMu9^v zuS(%Jt&VKN;Q3N&Y~gBXg}t%bWVW+k1Gq)5L#s5@ZkEsLIw^XNABqBodZ8Z+V-=0W zNfK@`WLS{B9Hl>p2R#J6Cms(mA4-IIVD5qlOg);Cpn%vztqY4NIw=`LQ{iB&^7#Wa z7a&uV)>V||WdnY{zt5auLkdb=`8s!>hE*dQPt81k<n)M=gi*#Mwn@6Q@B8gz_q;e~ z?Cx3c56|cw-+29qrM`-hV;csS==cW~4I7=uVjM1ELwurhaDFu(Ql=*LN`RNo|LlF% zTRgdL{qU`cBjz-uFM6o4`R|h}0tZvl*ukUFrO$cgmI=P{snfSE-CO+9l)8+*?qp|n zpE=bg^TIpfeVn07$QGyNLrYPGCLlV8)`I<ya8U{0C5Q7<)V-{mN_9YaD_i@B=cS>I ziO)fk1BII*_SGJx{lTuOLY^sHz={3|Pb?n%Yie4$M&R<(ilKI}PV{R%0}AWba;7QM zlhO+kSbd)<)y`7?fZ^f#8IR88g^8yYJUP*(>zlFUnxzNtoZYl6N1f{El@=@+k}>b# z?4Dj;?9=<Jw0dtMr`K_5xkh8!z+!EZZE2m#)Yy=uHYWCp7kgLl&fPzA<)Hkg$EUOT zk1k%A5WApm(5dBfPRv_rpWJ(VX@7sh4(_v_9$|97mu<X+9h(fRfZ~eGyJ@7@_fYB^ z*l^341+WfaadJ2ldI_N2m=NytV~zgsvRACMZsm?0E7?YP$-YTZUJ*stEXUIt7fX(> zS6nw@ob*rWHR+$@M%;ibXjl5MM&Dm&83`?45etEsp3Zfah6&wn{SbZWiSl#g2s8QF z!b4X)kx8BIv0a|9d#)&qO#jKn1JeLSU&g}PO{iQL9$?_n`%N@9{Doli;kV#$3Nk1^ z#U4<fk@*HUNQj$qlmV~{(<reE*h4ZKXge{6b`Mly@)`lI0ocDydLcbE_DtoV&1cSR zUi#p%CoQFHF>_1qX>;tNcxH3ovQtK_!)Q;noSJxssaap?qI9Elad>s5bi2j#ytCs3 za>OCS+>#mBw~`ecHs)WC{zzU^cx+5Je#R3lT<va8J|kT?v*+)lD{f}t&9nCZmZ{Ig zN*&H3gAN+)$s0GHVe)-(rz!VWOqDX@@B9FPt4omi>oHj6;g(tCOO%@6wkpq&GX4R1 zbtJ>0R7-sa=3topyX?tUg83mJE@(3F#$*?KY=Y=`;PXg{F}hsA=r<KnNU(fTxR}%G z=(G<DQHyZWATib1G1r+Y4ibGX!RPY*CD9fM+Y%FPT-e6$BV%wxfkilOmgY1v&y8UT zr!{k&OxMHy%f+y;Gbza_g<Uy0+`{z4#Pr0-_{i~aptd#;jxM?JO^etI!&da|yJDEs zd!{@uY{#5(0rNr*VGB@<4!Q!pMM&d*V)$vDP@v>60uXOmHR?c0m~v#F!u!V#*&AI! zFCAz1AzPG%yv`L)O!?wt1!<A0w1R>(?ra)UJ3BIHo!{9Yy?_5{>Guyf`FChX$Fc_I zzkl<0r)IOI1!<ein?8_!clQs{ylGF<(aqRq`6s?-`=^~rE8zZIU07IMIA+YKg4DEv zn@&6+y?wfCebxj2!#1ZDj2?Zr^t%r?G3|Q=Y4R4!|8SG^!+Ql3SD}N`*7ukT>D?xv z|1Xy@#d)U%ppGeWtaJ{l2B)wBCoHNdN?uM*O~xylSFjm1X(4SGMWdi;NKxSuf(5t$ z(yq)xWA3qIH}GW;dPcJn8YKu5f;{oiO;wizg-JCFwS~i3j<8^y&6ATjN8`%x<M=0* zEI;lmm3F##Z%Rx?s`UEpX1B5L>e@W3ZTPIsDF&xo?<=iJvK1bU>vQqQpAR2|98e;? zywn>Lli7c4!^k9)D%NBa68o3AL)UnD;d+hQ!;L5&d5@<^J+vey>4Buo;w7UeC9Ww; z>UC`7uuab)c08w7zw+VUfg^7(8}2hqI@xh>QPckSg{{)#cJ`ZoB^^z5>Wnx}rQ)|t zm9Bv?Y4QiD9p9(jwKLujJIq}-HB>Ae=~c1k&Xe~rE;Db4B|o4OT`5J0Rv@-mt!atz zj@X>-1Cp1zVgT55j#C)|HMfmO@q}V#n`2Twx+XYdZTw(Y`5GfTH>Yk!<SDb#w0_~9 z8H1+Q%$KT-<&!3c4oDqm7iN?)xPg$2kNHpLi{cUePB`(%e?pBXcU|zWsTiiR;t+Gc z_h*k3Lzzf3r_bXCTMHowZxHOR(;;}=1omFetl(pL^Y0AzY`o)%CwJa_LjS5*R#;l0 zjW703=H5AHIMKELHKDwK_6Pj=LAD?{TWL(>#zc-pZW=AdnU&ctSGLmPRA#Yl%*st2 zE5@3|99PQ)1!p??$QLg?_qS8cq3YGk^9J=x+wtQaLmvIzOJ(X93s+Gg81?GDFTVN4 zi)CtqLG-vQfkdF``vU)J8+thXfiD0dYXo1A1iUiY;}P;M1b7IG9<XMe8#L14A7-|p z$!NHDhK3+l4Opbk4Zc;*E!o^8eJig(V1HxVr(0Nf%ct9<Jd)I#A@-Y^;Q5FZez#<j zh5B!sn%l#)CPGHv@+qt7*o~_!2EIiW?iI~Vd=u7CO!*zkJd6=u4dlTiM-EUCyJT`v zHjF?&jGY9a4MDc+(ABoQIGu<y+;`+NWsS#=wR~c7LhLKCiHTSwNRG<Dmi~!}{Wk_G z?LelCn-rU9FLorxP65v!v~xqN0@8&*6;_igo(WW~+uqpzM&tJFjY3}IcJ9$7bQfAk zZGp<7qRK#<!|^%o`@H*i#i2lDWk8x~w?CXw$sM_T_4Y3QoOn2R*~ngmo&@Hhj|}JB z^<wa<9Q;1%{e54upmatWB%Ogj#(Hk<$rT-;6(>)w;9FLlWY2N_j$6R}D_C#tuFLyR zQg?8Y>?h+f4n;=rDT>*O1&SreUa?-W86MDk6bIlb(X6-=xcVo7u>QE>DaBdEvx-;o zHejCOiI7E?piCY_R(m?>8YV(eH+fkc1o9v@DE}J~P!EEwJy^lDDl0jm&=M6(WjI1} zhsug1OnxZaJWem}2`>S^DmBPMa~QOGSg}|L3CHQ+J#ajM_k+p-<rOY+nuBI10{)OX z$glLKicY(R^ZL!Asu1u~xg2^8@AGT80B=pTGF1gqzoASz<}6k#;%%?<akgi9<lHn# zP39uAwAdW)Oc9>7#qsBCaS65;S<0J2iW7)(J59wVcB6%k{?6%EJ!OsS@Utz_$(y8; zY_=t%V?5*DFrIlzZ{ki!YtM2>w{6Pe9$-Sq>~eHS?^dvtrb=lv8>;ST64@AOhk#MC zHzd7!<k@2<;J$jV>sHq5<Bx^cpHcEFO_6=)r9a?(rbn*b&OXl=&Ai`i=9xqNuBeu3 zS#d|pG5iT-c^lJz$MN@W7%Flb?8$<3n?_^9&jV_;GsR-F**vl87&cr;wrMnMx|nRk zInc8REX~N~Y1OeFJN`Ve${6Vo^EmVd>5P!v@j9C-9X0WZ0+LTk2bC|f@z1F_*7DLz zruI=vvH$QnNO|>oNZOsqiluu5BhEgp6xpgOR(aQlPoGxv0hs4a`qNCWlU_c;dVlqi zTDma!WiF=mlT6^9KFbP?yQEJ)%wpTyIW&YF?FBzULCQyRsUJR;KJU0*`iv#~`OnpC z4l-gG(E_)Pgd|FRRmT4(%sYi_RPEM6;$3%-Z%5%{n>c_iJhrLhpPL>N-gq#SBPHg9 zDzo{9P0z5IZB?7kp52`GFuR8^%q3e+zbL)g1bTBFEEJU4yBB)6py1I-C^!=N&1nNd zCbKBK(G8K1;))gUZ+7rVPAR3Vw7t$6-x$fJPaG&+8+m@w#PTMtSUR>8IWwlE8>A1U z(8^i-@18xi?eGFN_%(Z7r8sxBlq5ZS&Db~Cl-F;l9Je^~taR<5acm>kyS*=)&e>K> zn6*kON8)>1LFFjt>#TO+!OahJ(gx)D`j_ncOO%}4G{JPx7gXF@3{UmqLN~)yN9>Bc zpC>`rSsX-oGVPMHLph6`su_njt$XR&Kiz!upPqdwyjDEi%D68N9r}`S(*JBYcVz9o z&$k{p(E9wnYv-(faNH~R-S=Ja_ctH>=)vYCYu{Y{=JESp5mvRUOUK`Q^Y~KX!uq*$ z+wUr^<Rim&?Hbm>XJ)0&pP$0-5N<hK)lpwqFmoQ5XyT)fV{RTVbI}xm1zF>l^v=I{ zJj$bjzVt*|k!cGIjUTvd6KyVeA${ty&7gHGB<#Q1y14zTyV}$4`fA-A?XMQk9G1;8 zp5EWF&#>*j<J^aKm&+bWr5gaq3CqaWv}E_wQmU8N=gR8`u)OQ8ufDo#+4BgWTl}kj zkNtID$Ns7k&mTc^{b+-7S(Xq$+bK^QA>JebfrN6kWh2{r0A9OgK<yp?f4-sc-yMyZ zl!>6uv*5?N2oX#x;mx`pR@Uo*GrC8yA6OX273VP`NcBT5$Qr0j?G(M{{P7piqRt*) zN=el73s(VL`SV{oUT6>g%o)xA9Yvu3PritOk*PmT7!2X&#aO|Vk=pG~2a{1WGXR_p zgE>l4UMm$H7b0r$wzikJ{oJv(mqs9+QS`6EILDZbuS@=&Z5%$wIA;~Ut2=)?Dwi<Z zQf6PuY*@Lx-u}3EDbfsHx&PZ?OS|st^+vXDc>M7V8y|a2de7gte_wyolz2Y5-{hoV zNoufec(7NxJ*CD7ZahunGQ>M#l7ayb)Ka^pQ*2}<f3#z&^uCz>^2^dYOPAi<<Nv0V zT#q3oCxyOxPfkiojwg3gPD&1aC*~rg{mrx={Ym!?UwY-PrNh&%xF_zP`q;*3Y1wsY zy);0?w&xdCWM~~($vL?yrbFIgV3_r0MqbA&Ipo=*`JPALTZ-|w3$l(0e&Scku;6fi z5}YST8x_lp#^l>uj~;F1rK7F4-`>hvE3z-Vn_W?n%^t`Kao>fq*aO)WY&#u0N+&ig zJ}Q*7oyn@G$P)Y0@>jpY5>F&PG#&KoJ^YRX^+K*%Ss=<$$y_-}L{UXErgc(E5-&jp znr?_BbPwuI#L%IiL?tQGQxhLhEFNIO&2PPbbo8M$OJ>hnvg%;{q2Ii5`}B85i|$0V z!QOX<^!@rRpKN0Z=T@CRx@XJQ<Kt#-*)Z8P%WO+F|KaxD^U4a;MtfYJ2PZuv38nlK zuRXu_$eY0lt2S?0H6i%etw)}$EfbQgc0+8SVg9mrw!Y3xV|UzrV#nBPXG{)zT{g=w z+wE_7v%GerHKAWk`&n<2v6Gh#gY+dppOXo#9JM+eaY%gt2OrW9z~zSG2MCJ<5;w`) z1w!X9@u(_M&MN?JuZz?{84uAjDwn$iCyoe+Ln4JmnA9NAmB{B%5!BA*UzLAeb;q@T z#Z+P}e~?b;fP1&Y%bk8Lsx~pj8Fd1JU{1&T_$%*1w~O?2EazJ!MXZ-!qe_a^n(}b6 zFWyktU*LhL<I`J`9bT0WRqRpks6T$aTyHpTV5TNM92m$*hsF>I$o|_piwYoJ1MS+k z4@{;Nph^J0Rz&vw*R{6pWnO9y>5qG@xbr22mF}0)L#gr~)}4H_qp>6$<~$925GmFS z&0^K?9>3KCfKji9ml=9*)MPGa_6R~d<|%laTO_^BzGM?4)z`l!wMngf1bd$Dc#b>y zn)D5~h>eq4r8agA3&T>^5wi5Qbc9S$4}>iqA?)E5ky+fW9UZ(72IOS8<1gH;@(K&j zloXa+bBDra6BOoL3kUoHL_@>&^ECv-8f4FE#sp1<FO^@0qW<@bpi_>A{n>?AMziib z$qd)|3UYAtV1Drc0u&k(6_1!N+06DIJd)YHfVjlPDl1-ccwBwGrPxwmkM*Bj&`JO9 zczs)T=dI|h&|7Ak>vWhY=o3EevYFqaC&{<?#mBJ_G?3x2g#IFz%}Uo@^(z<Zsz>Tq z)3qak!8J0(ysUS8nYK5}M38q_I^SDc7B9UZ{n3JhIN{&iL_m^m`s*5hGQUi*X#Er` z6bg?OrWdP`5fltDi&4H2EUat@&_IR9LpUa5W4Rg%4tUpe(;Ger9WZ1j`qB}QTf#b^ z3yJPJRD~)R&xINrsUgCROu=#5G1XI4iK;2pV}O@}KOO%07*Vf-`?EeR$EwxqVsv_~ zH78B)v;dStjN$1NIP~7JcXh{s)q6EbIU@q&-f?ixy=5Md=FW1>?>pa><xU%5e(O0^ z!aE}?uE{VVq98;eQlT(cAaH6c$5{d0V$twOQ|D2~t8FS|43A4q0YXA566(dMY}n}$ zpsUh!99w<E4XewRT)%Ki{0E^>4E#k(Gs<^oc+1PZ8N16fN=wp54FANlzWFAaH=&b{ zfQAnN$J&Hh3yED}MWOIH7)ogV@}!cEsZ;SyN(m5WYD~`QDI`rOS`C|IRmP8uznuy3 z6YU4j3nT_Wj2)#Thq^tT0U!@=r>Blx9f|3`@u^wA`q~sTeE7h|h2DfqiUHkf@F7ED zuYDvW)BRyvr)4E^ilw7Jav_Gs7aQ@|s+U+3X3)W3FWt2JrdKY!z4Sq+^g^o5V&0dV z1qHkqhFbheojd#ItY@|lQRzNyUi9L?d3B#|Oz?MU#uKs^g5D++Bss#_E~hJT&JrXc zz?^emMMC_0k@h`{lHJLW=t%Jn&Ha_?_9*|MfFDXLc--MM6MEpA;3i*GXw={t1haxc zP`O~@;Da)-23idkDiZUq^f)0+6fq@S=PW6PuYLV{sqOpMudQ0PYG8bpASTE6ZY)hl zG*aHwjnBOO%*LsCJTs=3HujEB7KN<%fvc8PNnxb6k3uS-^=bnQO7TWH*Hy)gvgG8l z85Q}%i&JB8E8I|<5bHDvy5v-s&E`r=ju8y8&IB#)g!{#$77yo#OK1lAl0AaH(6h4> z(VSQ$yN2aB^90#@%0m!-u!JJq(ht2_FagGX;(L(h1it7V^eiZib?`<aYl4Tk9WX!~ z($izui3*kvd~rUq$5Ld8>=sRIu_INiKC4V|*i)2yOAx9uOS);1I@Ox3+wfauYF3K4 zOuA;4)LOn_QC(VE-J%WUtrDkDYIq@X0)YDCI7@<^#YJY=;(>PkSyL*zZ_nWm%{ET# zC5_}x+2RxIQr_V`A6&?+38kflYBDbn563}g9u_;~*cxbq6e@C1CRBO&B}a9MFmZHg z>&!U}3RApc!IDO{B7B9g^xk`|r1yg^5$eF`>Vbc3h|%r%WXnmGaS946*%m{#AHL;7 z=?R!_dYl?{EfP$pnC0-+&-WUwd!@fx$VwEwO6D^=?VyBEslcEkgpa6}lN3z`4yHZX z0PJK?bdvJ0Fj_W+No<Mt$q$e=t{Z>&{9n%>9*>{puinPiN$s+-au%71qGl-(Z(C}l zy-X=>xb4;D(X;8Ib!?q{o3`-fx)3Rmbs0h!^KMx*b`G$h3KiVGf3^t&K3Le`N(YJq z`T??m-Xc>Hm9neQeEFW!XjHi*jq+ootM5tgo!)c20)egr?CPwRuUfLyNo8iMvLbTl z<w^*@9VGqjZffPtJua13H^;JpAKuOEgBnkp5^QrfZk%gNFr9AiG?h^sMEiq0*Hi(M z8a8KnaCWsfP$1CWzfeEwBG9LR2p|B>7wD>#prGjauD7x7YW3UykBu=V=6-d>2Mvl# zTMd@Tw#(HL(Xa4!u(TMqUOM{n)hmcjWIp^F%XAv5s*(Aoy|L%plHZjaTRM->L;jn( z(Yu2hvm0`_bA)sevFNaIg4T5+6&Jg&Yy|O_8v!qQUC|6pyf#nEG;`oi7ov(2?tsOx zW$u{H1LI1Mvb{(D%T}Up@bb~XA}v#AsS~tIo6y!hUe3Hpod>3stXub!RwUgIXogZk z%z6oQ`n9kwl4ZuhA>I2=`@QF9hzRu%%$g3QTQ>nzmM@SQ5=@t%DGc~QxEVaeP4Jqc zE{Al<Qp|>b9FSjsl+J($zLMM^QvCIE_uhN%b>{Eb2iB!!>8wMCW-XNs%-qH6SFXIC z3q3(Y{R#O1|M$bvH>XTjkfI*9XHkN54q(mprAzIAYmU6KiOt`%2|=Delpg<6>)oYM zq5=0I!8m-lQR)EeDAT#pyIcQs9D(S9f?ZOoh&EIM?{pHpqp#BEz&<PCE0zWeq)b?& zqeiA-m=${tJXj#*0R=x0VpN>v%nL&nrW6Gbh|z9nE=Zz&d4Rf@@`|1|q{5LbefQW~ z(y@Na-`H2D*4*%?Z7cqGjog2Fym_fl%A<Sbod|Hf<9+;hgg-jYGsCNa54WKq`4%)F zb|91>@S)Jyb3{)5Cj6+>5ufz_Gs;=VK3ci$ultSBF&OH3*5JvSrRY&ov&|RRcDKAZ z(cw&Ty~QfLtM*D4J5(^?V^3o8Thg=GgEmxl+BF8F4JW{^@$+qnKJ#x0Zx>;LPPL%3 zDdoN=vwA^5&Z75q_c;@~T)1b`pb6d5zaIJc$>lpxad^4*pst56UgwNs`X^hT+<WTf z&QYmA7eqX^>WSqu4jr1Y{0Y7^+WF+oE2$aU?qR7TA!Y3_<4<X}`3aVC$5YIhl(Ke; zJR1Mf*$shrjY+cfZq{;lX-9Y0lR|eB27=6|$ac3rkQlC<GouhmB-PQ->M?r;FMCY> z>^ypYr$&<lBD8bZmGkMa`=a~rJOA(J)pvGk<8cGcuHjeCufy((?!WK6+^#>JXSqv) zJkOTO`5Ya<bmjoq2D7fEI2h<=Py({7+vWJ#<rI3kTX0by7ZVqLkNHnY@0{H7$qpS+ zmS%dV%@d-9cm)FEE7AH)h#t6jc_!l5#!63oc(?S|y;kGi&$k|CmQ$waYdbz6dd<sL z>&wv_O*k&sroHp^$Wtud4XmQ7u&@r=;Yy;MG736DQB|-W<WdRRm)6N`L7NKJF)D9( zI^UABYi3m~0Gj;!fP2c;YM+qam!G)#=gP=@PiVE-IJrG^xk?<eis2T~15dP4Q?z0p zI>j=&+b6p7iRe>0zW&L)D!&`j4@G&%F8+)rOvC}XxURy=?4n#mJfM>!i*&PxL}F-W zkK9IO;HJ||)yaiLUj5NCL14o|7!omTpTvmD-|p^AUS5hQg_f_|cA5JFKL-naH`m7n zI=RB=4=O-BzC3o)x<?KS>xBqV0Xqb!Tu66N_d)rAQ6f+M;=QQ_1*y{N7hRv__Fq%6 zbo;TFUW#~VpBOGkZ9AD-z}0_ob4dyNou+y3y<bG`$AK}aa9N2yOH^Qv7($jXk_!YH zl%x~%8-T$eJ2HY@lqCloi8{1XI0ns?@J@kKh!QIJ9KLq*eJ{*9%1kGHBUjuvtN+#{ zZJN&I$SCjvVK8%?%00WTtbfC}P;gR3euDmw556S*GcGC4<=~8k!!sTHz4M>Bady!b zsk!m-lN*MHO8omWr)7?;DG;?sk|%t|#<?nI*Y6!sUe#7!QC~1-_VPKo_dhE|Ud$Sq zm#Ik{#~Vk^TY&12e0^yR+HNc4iE=_LQNVbGVuNL(LZUjsXgV?nSOqY@a)mIn;RBT7 zZihHbZ>pff(gj0?OGPsDT8jDC;_neTvuR;&>6WRxhYVu;z}Q4(tjcOss|yB*Dg8?( z$7qdB>%TlPefo(nCH$-!{@qcKb>@6!)v8ydFK_+LNon%-`Kw;x3K}$`)|2TElxOd4 znm1NGzMq5F+ilxb_8P59T@woAsifhZH^I;PSC4-=bhbE?ZX%tNzI<iXq^)fKL%+Wj zcP6(x{K;}*6fejs>xlhm1xPGGD9ey)#?$3zhFH_?bxWu38Tp`)Pc?nRWaOu>(v7H@ zlDf9o9vj%k|G|rRTJ#G<8O$^XX>W<(?povI(@G+4a&HDuP4}|f?kLjO$)v~`g&X*S zz!hZRIEaPq;YHFl4|uw~M=0fi$Bt7-bx&?hoe~UINb3*u)8{@Rbbc<x8ueDIUTU`L zTl7|`nKoFXyb*B^DPoHLvq&|hvb&3`XOzL^>6V9X8E&&~9{n*uB*L8l|I+P0y*hf| zNK4U>ZwhW$9hk9v`s9A;<}&=58;4Mm8R~;!)xYHW6)Fhbu&aL56A>mLqh-iT)S*Hi zVh9wVw0xuvlQ9-lBDsDg<J`*?b-3v;;EK^ZFW2*tiyF;#jozTScu`}}W0h8Zv?pSJ z#OtlMJFlqlAvXAs&6=2$81L<_Z{*{jzscp>KH@D7c<cGgm0z<4!K^u+VsEE@3oJT3 zK;soezho{l>Zu={LF`@K&_guDLmGUhP(n_=q-cY(TUG*b2<bX-;B9ftC$~pJ@jecF z;n|L(0?xy_gN_~u-Tn#I!)Rd4X*ej`LLIV~3j+B(Drkt|#vbtaJ%K{fkt4Zbs#4L? zB7GBNHPQ^}Fg`O_O;Gx#r3KpBeryfvCn@H?t5is#mT*f)3r{}`AZxq=(-l33-fi?b zG}ok}egy(C$>3?^S5*O33rKQWp`|kc5{)N;`2O~X&znq+_Ev|<bpGLX@MYi1Nt|$J z(E6vgbc8oPF1}(+PdC0IKE9C;Z+WV%Mb6p8-XNGK`a`2$@oaf)|J?Jt5-`#8yR)4B zY+3#k%~=|Xz$z!NvQVVuXNna~^~#>3VnupxP#M8lT)F{tXa(Ls#n=<(4Vni86uEij zxr*|XIyD@2Vjt;y08EWu4f$<!T&&q7<=T^MR)dRO;zOJx@>gMAVxChP$i+o2Wl3vT ze{-rKhD#EJ@$K`FxbsVGu2WcMOEg|m@UuFOGA&o#{-?NP{RjMKe8)2bxiy?IQ7L@~ zEfdOxcE*?_JT62j^u$+(_uY>$)saQ&N+fmRWYqgDRx#?5Qhg_K4@cvaa~1tzS?^#< zW`Xyt7j(Wa8^}hmNx-38$$rhAWADKLBXMvj6bUJf)Gkm>Ad7i46SLo^49e>yI{B2* zb1>K990uf+PH-K6bk+q9Dnu<+IR{;@1H7{%dPl))ptQ$`M*zGUTr;9ez`u}u>kM>G zdt?g*8%I+e)b4ngzX&&rURUgJB1?hOLAO9)H9pXprr|v~f`#QgMR(BzNda6c;P(@r z03L%p=H<{f(h)kKOoh=j`b@ino(y9E)c&-jn&BEcOpjEmQv41l;wO9}o`;I#a@++C zlTUGFbVU%HM*z_j)J`r69t!#tAQWWU3>5J`RR9)gdB0CAhvqY&gwCAycq!YK3^4~= zgvuc}i__2?MdiRTvCB_ZqTYCjI#r4M&?vJKP&BlM1bzo!Ovr*hl!mHR9HfHCSApxH z_%)>}6=iY?K;_1Ud`+soz)RIq6(jc}KB$j;D-mGp)GFlBi{i77)ILjGfMX*QP^lu7 z&l(5Uruqbjqf|dOC42C;y!70*CHgVZ)g10+)+;q3rPx=LC^ij82I1Ce|5%%_=(-gn zxbM_f6&oKe&TDW)Mnrz=9GeeJT~4&Bm2rjyl}4AC<kCA6Q{yc*ohs0P{HexaT#7GI z)aMv@PTNasG&=jU^2Ld6yV^9^*&7fyNGqL?pI4B+Cbw+np6hKHMV|5)U||KO_8pzZ zL=~`N7~k7jW5G<BxTx>ISiqiVXrP|R(u;|{6mGadqmF3^XjRN+iBC;*8a(j{I;}cU z@07mRjC2VJi8lAJ)Hr=VmtN#c3XOwZh76tEVRBtO>l&%?SQ8V{lltr9QoY8)prCou z(8rpVof99&zo$0yyxyFi#bTw_FYdbQi@S>F%w;NV(uQP>AWGk<0n_p}Cn%M=l&#W1 zQ?F8^1u*a8faiGcX6C%>K4w4c0nm)O${1f#2u;08%PBRg8040<3Uf<^7?%ksjlYiN zigUAK)MicZBsK!MG5oz&H;Abliwno-ox*RPpL%?X(#a)jVzRVWpmSMAb2e^<bZSVQ zT3s1>;|)N>Gz+l?B(pIZGYpz!&J^?7uV3IA#fDWGz5!-lJEpLB;|`NorHQjTszjmC z-ebKXp;DtqKHLSOI69@rx=>|QXD6f<WIC;b=ghfz&if0p`}M6+rY~swK$q+^mn^R4 zxLg2r_sZk=!}ShJvQDdUBqeFI2Ad&C9g~PxW~||~0|LIxa*x@ojkAczOaOEfoiZJ` zMw}-(t*K;Pz~oF~@upY<Z{Ttg1xrrvK#g7v*Ixs_PWsoV8?;v55Nk9T5@XFp>q?ta z-5z8G>m>ry0eLfV$5^$`?5;@f6{yy5`LRZHqQn?YqRFDyXcJv_HU9u$kEVOCO|l9r zGPd;AyA6iW43kmImagUdZ_S_Xj!Uu#)}(89BpZ5f$xs?i(<{xDYZnP<%WLNGe%~&u zMWwcF>dSGPjxSq&{P^-^k`Em*VFd=2jvv(TNui+u&2AetQZ#Ze^;sFGR$5FqCvh8{ z`du#s^Pjs_ZwGu6VGOC*xC{(QwLV`|1K0^SVH%s+ssr4bxwJx~&e7|W($FlC%?8uJ z6}p(fyy8F|$MyZ7qGWMd(e^1woB-f1t5c`f)%Qzz-EQBPpX%Uwdt%=(%Pp?*dDze) z=s&SGi-0^1XD9X9Sv)Tgqgz>RGUTK9NQ_N9Lq83GlELp9$zvM%ysz-gU@o*P>@ot8 zBvrYXgP*h<mH5xb?Ng>~k1U<Cy|Hn7WL)XIZP)(>+C^6S?vCHzG9{bO7&w3J&?jaj zO`h0T?TZ<HNIHE&`rw17H!khp-;<k37vXYRiurx7sVz)*8UU*9&Mg@<V#fT+BjXB} zO`bWTu5V#7pyy1kqQb#_Mg>V?l6?;3_||BI3Sl44qHHcOwkQ$U=jhB-M2L<Q<J1jw zZ7l#R1iJ|4`9gubXrK`5GU1T(4Pc#dI<Ce|tRUvlC&-b}GS}a=<#Q{4jD;xll1@pl zeY|<|$E=8rWJR<VS@2V&=OreS8`-3<om-olB#Zb-sUzf=!frVKx8&Wh%*ns!qz}&V zjqqRz6H`fm>SD|0j}cLI<<ow96<p}ZFq1^jYBhyw1F5~uOGsa^#N0bIz}I#^jmZ&> z(l?ECuyNw1O%tPQd(WNgxDj3x#L3bUEsH+V89N2YUfIe7UX1~7qNg`14158Zn<A0B zOZJy#rUHj5d@-zo&d*}F&_a(>g(zOWHZZB`0%GAORjEQ%lLEDZf_T|T3sl8!I;#U` zLC?`F!N%B3r}6U1%@mY$MVS)1%M?`#QxH<mBG}_74#^i~=jA{p`1muhtehMl)Uy;n zDk6%b;96<}KPp0`@yaqX_y0wCc!;(Xwe<g&^1!L*|E_(&<97BTC**2KCecg;|H+$D z|NBz#|Izz_fOJU|u7xfs5l;uhl<I-&K{Qn+3B6DyAykc+ePMx|{$u0IT3hVtxIEo6 z(o5F(n3(kRn3x0$`x>b|q%`cV#bNea923nMVrzz3v?}Ns3Lcz1d|VaGZ6{zYv(1C0 z+pqM%ZPX1Mi9n&bNM3gq;|L#;TA-r{g+kJ|O$amzg;)r_FfI5sH8n9)NDQ}1jp0aZ zYk2S8a4Y8yvu1fU+MIZv9M{m5?SZ7OAgFjHo=><ikT2zcX@D~W$;%@GeG7g`0doMz zVJL{~1B+6^>Bx?N1NlS0B$s*YYK&MZ+^&$qq(y;2J`Akhi`c2ew>|nRVJ|Sf!+aP6 z1uA_3C6dCF3pjd}fa9HiZMXut9<ZdG5409Ri(!O?IT~ej0V<`XHq<E4h9tp8bwq{m zCZHS^@TBNjt1Ih<(8JCevYxesCejNDyQ`{{Js8=ki6TJl6m1|#;{gqd1UE8#@gFTZ zA4btJ&Z*WBMYs(`7V&LIT~P&mIfcBV>k>Xpb%|a}7jksHyp5k|E3{*c{y2Oi_|PAG zh`OFh4RBc&G$TqC@@WrJis+;irPD*bRt2ROlCzhji^!QyY1+f=I%C1(1tSq(+8Eti zlHSo+GH4`rLZ(DJcgdJa%=4rhKoU48cD#7g_!Jcr?WTl_Jqf3{>OxY?6EV_v%-xQT zUBX^UPkbEd+B+0ok7kMsTAXo&M~7hU^b)=q#~N`GGPzUHO7LiUnVon@I@HOJ-Z=_6 zDirXC>;@!6f{D&`N1+2C+EK9_`LL3i+Z(_!_!&XEfd~XsfPsT%7pdMLl?I|2w}EMg zTKqJ4TXlP~Q?0%AR;}8pcRBf(9XpU=*4aMi(;@xluMTYQmB9vauS}aUf6bctGp6Ou zPE1_?*wn17sgJFn!PktbDh-XS0y`;{vcC<Ud?iE?z*a<VCnr8okRq@ROlW0A(rdkI z`ZO7lT+6e#NTr_o<W#k)uqg)l>6PhqjmsMA(v`xE#REiM-7hCt#Y66{;ft@pA0iz} zSjM^~tb=&Orj}C=FhH${=v%+Jm=XiYNEry&a0^Th<d!2;Edtzx5SdEN)}zDl<K}MO z%uU^lr03(0NZ;MQ^1UMykKi4bGXs~n=<$DSl79c7^xgw7Wt?K!8@_*h5o_oPtHu1> zBfXyf>(lt}6&c)%y(v8>eTO@|xAJyoIC4Z9vg7-^8t;(adGcQAk0)o`^A)eWqB?S) zQ*`rc;4Q@;&B8y9Oe4?x%k#91=@+#jfR9jyt@?H-ORah#q_>7ARkh39fB@D3W3KC1 zv&<;a&PF<|bGI<`^2w7}d9$o<rJLb4bS{!?Qy*2R!}uXz)&M+KMk7iLx){>Zp~+O} zUY+{il&BYt2mU@3DjYROmt#gF2W44BEOhDDq81nEf`JhYWw1aXHH381y+hdo+Nrn* zGQlg@BZi7}u929YwicQ7X-uy$NOoFff3r_rJJrtqMjMfes@&YFTw(Xb8~1JAcjLtB zCDUgMmLV2l_Vgvy?TV}I6+)DKArj)lxMkb-GKVQIL>(R~uayoQSSqiWaPQozjwv<E zsyv>mWi`5;Z$A2@%HvTz`RJQFbywZnQ^%PNos)tAUBF@Ka(SRW84X)B!CJ#z22<*6 zFILV6JQ&l^M}Q6(c)JH(8`__uVljNax%qswO+r-n#_nxVZllNzLw7H&?od=O-96Om zbXsXk=-Lv)$T_oU?p$e+)PA|jkP`P`MC@VW<$aO9N$Vf_Zu92v9$KHI@}zrIS8hh> zCproGM>Y@@;Nkzjs$nMc*boqi&}q(}iu(OxwOTtA8vYwi|HV6pd_H97;{N}6O{&Vv z+WKw$`|0(`$?H%5eIwCdqWzc4PO((~o43<hcEmZN=<uxnr8k12kLJgWE-ij+*5|Vx z!yEih-@&<u9_ldcdO0KG<y}|yr-A>=5~p6-pOh*OVS)S?o$2~{+?jdTqg(ywmH0_V zD%`WDkb2Y=@4*P`b`9v^k4Q=o4#_!czsI0fA<MD#M@Pvo0;EA622=U?l-~JuMb~6y zW@cSeRF~g7C7u^LuC{mT<*Py#LzuR(2nA=4X$aM3R|3h|RheBIY8W#+c(r}xE7Nrm z>d?iXC@_o9#e0#hy+pL-V29`mXdqPPkfAXtkqjNQ(vnVrWf-TBTXy%VpThV+J86Ln zRRp#Xoy1s_v=%@m47R+Ohj8Q$<>ge#i&R$ZM_w6-#oGB=d2fN=puxe)0#QAxvb3tt z?34ue^qu+z%BH$Vc+`C9wIREv=|ts@$wfJXgfPG%Cg$}+WMsYTKKgCVO_kpDSCH5n z*DH-ZoYw0H+U>qBy;99p<%HK14i#CrAf-58b<^}83QMISvAK0k%SW;FnwhQBcCpDD z?E`46QTr&Aji3|xKw?*rVpx`w@f!#AEj1H04z&!L1u};mB|_q9*O}dIf%q}x+2Err znV;|_NIW5zU}}w{6RO-*6RHmRLV;Rx#SL)}rWC7&h}cK_-4AbHnrwAW+coDF^$^2# zBO-Nu7o<ty!UJ=M73`Ykv?hz;s+>p@XQJ@X$hVgiuNT$^GE*c)VO9#;?@nOf$#J9K zcAdcO&UtQNnXqe`S-EqLWJu4H<`178%;gmQ$ILyD!XBEoODLoI%RG#1>xFj%yd<Ym z9%Eqc`0+AH%>pNI*<~C9GFl(tM$4k0N>uX1e^R$82$DfY?lLM-#^|M8<&5`68_?lI zW}+zONRW(_aFD}MYD}OJQ}BB<$_SQq*+!ufh5XaUDxBptqSQY3z=64ovj&epFgGWg zTZWn7!2B`N{S$6Fe9V<rl>^`4k@*!YL~GJViIz;0siMG!tc|X;FCr^q9f8_xFK39z z5-I2WGH22Jku|J7vluFZ*S4ooyO$OX$ni<9gm>i!MAz~GJ}qp4=EO~Pa}SvReqe57 zdczL;XeamLz`=%~C#On#NLyEMNr9EkdUd?r>nI3mnhinTd_i3sNUt)y6hfHK+!rb` zXLcy8qjdwaxZ47?>pc0=yE*06Id8mCouwWT$QWb>#q8{RvOJh3<Kv+xVHkGugk{Xz z&8*N%pcjI!1a%g=maG~fj9CmdNQ!bNg|D^;W0H)*au>vil}EG_c8|{0VqtyR!Zfb$ zil#aV30<T=MLw!)x@RT3Gb1u``aX4KQ=PJ9V&pgeP)8*Ku1scaVKN`;Xu-OAP&OAH zpI#z{w7}~eP&El-Ip3sbWwA(5!7!?oijw?}f873(bOaWLy3;o`)wx&R6W%s$_$+r_ z)5R7x?Z|pwU*wB-r{>s_eQu;?G-UNINjDl>lDw0u-0?ouQGHIr^Rfa<9+R@KVF55$ zL9={*3VN0oWRD^8lK`fee&v8#z7vuJ@%hSBp1jjjG5tlyuC>Q18Vqs$7|RH0l1ZNm zcn$F|c17tRF2fKn^08NkuC~t5i_27NCz>~nt>0*?pJm%vf6W%dgjK3*wLwQ-N`Bm& z1EmF$*nf1suS|32`aPO5UtWmc96wD{?#r#>m#GBxbaj!3do&}3wU^WuVW_?y8pI2s zTz{EnS^NRM;*w%=E!$ICnC)O6Cb%YU*N&b)YlL(syKls-rDL@>OpHyH6sk;<OQ*Ye zPI;40+FYm%=lLRIxN*L`s$8Q!@*z9uEv(8h#7HTL%UseTcdvMMZDzlsn)uKEEU*}s zuD9rN`{(&27uRGk<n@=wx<d9DE|d{SDDMQ#2v?(YGy&j<%-?qr|L81$QR^qtFOxMQ zZCw+=PYItDq}0>-CEeXEy{d`^M~UA#L<dZWg<nT>iWpps$zpKvy!{UCw86PWiw7no zP1=|^!8E%nQV=DC`{xYobKtLT=B9rU^MRz0!mkt$p_Ww?B37WOaq4@$`j(`Z(L4|u z7aU$2XykeahldZ(`+yr@AFJ9n>AhtOq}`zrQ8GB^mQ*fv?g2RGft&C8cD51mja~(1 zv7Mp-OGapv@?00KVgP|-Q5U9UB8o&0sS$u?X_TP|<xb>8;v#u+1bLLF4)iOV(`qOG z_+Z!c5$&Z+J^^45xIOwhq5%T9hKM7@C1MbZ>b|+VoTKeK8Y0u@9{9WYz}&h`iDnS0 z1p9#HPkMre!2^Q@b)ZdE4>-K`c(s1Bwkij^n>C^KO7(@AnH4X9D%FNwGE}8QZ=0Ak zKsVaD%RDF}FhZSG{l*(P)#W+TyZN4VwE=#$v*Ot4NfV^|$IL$frkh)qoiq2q_`z9= zi4aTeVofm3b?k6OJ{xI^&#BsGGG$s4rH^Pm&BYomHehAXa>Pbf3|N%&CFdmlC=^Bp zZ+30l--<I?ea*=6(ks540Y#-W>!od%UJJ<jVT^eg3m%5aKMtBB8CT_Mo;wIMu|gS0 z@e+O=;d-UuM^5Ymg&=^dC<=ifiWnwq@-XDDf!EGE7X3DO?fgM+1*%J&-YOV|GL2Th zHO9hno>tpe*)(UenI&eMUaJ{~-y3b3542idFMO!6?b2KL*5!Ij$J_G7Sr+|rgT<=t zsL<=Q<``~>G#0^__eLIyF>AF3{@EC_HF6;~L6xdO(3hF2gbH=ySZWa2+&dbFKp^3e zwTe+xxh{U56e!Uk5YTuaB}C^z2aFt77)hW|=r)j$!9=k1^^Cgqj;cXLuOmT+^`K4t z++l<i5YNbb;quW_N7pE`rMIPTpI^TGd1gY^4U@d~{#C3Hd}V{jBK>9Xd(sZG!DMC& zq&w(71cMWseA~<k`_i<0@$%=TZ+~<@sTN-34}W{(jo-2%U1pS?_09&ZkuzN}Sm~h+ z^7}ABfy3w?LXIFQw){Czm5^1Qaufdtr~Ra9eq=x5Cf+*RpaX`HNhb(KTY}4L9Wu3X zuy=sTq%)e-0lkqoWd%lN-~8L>_!yk3%~qR#;naQ4Kj;5Z<<YE&SDDX-Q2nKcyIUf6 zw#711FK~L~B{CXfRMUqJnU-iU=ol9>%w`pUifwy#_ugmdESS=N;VdElD$UO9S3EG< z^u$wyF14y!M7QiyqR<FVm0!bx8r~mtYT!JoVq7Z`yGSF~e4=rX<YukUN*{C0Tyvxm z+pJY`Kg&FgQkZKc#VQ?;Pov$*bW-POblE43EKGe|mRDuH5&5zBK|1jyCwy_7LBB=} z`h!5MMP&b#U6*?uc?iC2XlazbWX|-l>!sd&7JEVJjVu68>}5{r%k;7QkgHVkQADXZ z8=k=_bYU2mRIwLu>Hpw%&){~rumKQyKkbyHtNsA`x-_(n6?TPamdyb`avHBdMaWsO zt5<eeRe|+Rd}~256nf~MSfhLavcc7XbL2Hd$5Utr)PCSC3YgF1R>4Qu4p-qWPhP7B zf;c!c(gu=82Sjrs^=VKnkxz(6PJYhqfFn&1ZtFo|V{lk7IIP3JxOp-Dg$;}AhA&y% z+%e$T(q+f){QQ`(@z}DZ$FR}yvGhOBT=(|cwQpbd41cdAA<jTr{`Rfo$L>GJjgY=W z7F48EVCw|7KC4`_@Q`%j@Rl#?a!2Y$yX(H(a#*@>XrZP&i!IpCZu?U!yMarHK0e6N z(~Bq3GZ!yrav56W2OndfA3OH>F)5v`W5%`T+s>~Qbc+^_KlJwUrEeab1kY#e#%sW1 z1)*?#;Vn<zCSAC-X<Es(j@902i*IG>+n&4y`=>8%LZ6ul2fRa=XEk^i@E2CN;a!ad zLb7BsK+ZYv2%?<jDzL>eA~Kv}WS~~$IVP{89HcxWKO`4m{y;*=fr#%bZI^yvS|Imm zr2~&|+VuD)mZcZ;>Dm6JFV!%e%N3J6Cb{2B()Y<@u$s(tgI-<Y&rZiF+O)ioN3>N9 zYAPLnm)GYB<)v}Ukzx7_?)1Z%r`X|56DMriG+|=o?u6{LUY@ub`ylx)dY7v|{EuBO zy=x5J&t4Pf>6Mn9U~?HP@q!^W-hrIw@fL$io(saV-<S#*aOJYMYNz!cvKAC+aIVRf z*U0k-xl~n3N|J>c6`NQhcNa(eFK6<(5t8fviTe2ViJK=*+{_BKX?>ElzO@@yBqSvF zNz*#g`_<o6cw@f9p>dQso>?*!OO3<vOwq1U{n7`XmHv75ED{2(=?QfT&pq5}V&n%~ ztQe<`O;58};>1{6cAu<(q3FiE&KoQp620ZwB10gn54_f5&eGl37agIM_uR9RZ^068 zmiYOw@^LW?KR)u|lLbf_jS&FekOCpqT;|9%GQOuQbSsl8$8G;idiH?_rDs3iJ|V<C zo14mR_72F0?Ekd%bZSoifHbz>BZkLUMlL=mwS2y9+vhCwAg2mVXn)s30E_tpJkl$y z*fSu%FhyERIvs|x90U!RMSV_0WD!gih+;(WMJf=%Jaz-H^c2Xf2DK-8TR^l&9k}3@ z<wFMX003AEu?zTO7AJ#pFd))vQmDjtr1#>a?<-kgq;!0Yef+X4#trn3C^E&f>#~#I zcUa#^@*U$?-+p$_eD}hN*#47Q==?rw`4Z20{bwrngkfNxc=j4&JIW*9d1i5s<wq`U zfy2qOyzP~OrDH^&x`0dUo!X~k_w+J$IK$6A4GiE4ST2{7Q~me;@9YoebE-5YWBFWp zfmH25>SO+*FW&%vPA*H>)gG#i^0hLJ*21Q<1YGUj9u$uxPlPzLa=~j;p(&6w0j|L+ zS^q(P!zq4BFh?|wXqPN68A-trBv@WZOt~0*LGpUX%neqUQlCHr0C5Y_z0Fa9fobB% z!=ooNa|I*AKjMjt_oWnoH<+YZzIDfBUOJ{)wRz_x?uOZXVw|AwGx)<zCh!ePW*Rj3 z#rs*TRgXc!Ek2q&K&un`8y0Y$1VJBXvkh(CY~=Jf&p5E+wFmq$I`5)elS{NZq0}&X z{X5dT(v#A=@2p?{4ogS!-1K)ozRCo{ZHo6%aXj_h0QG{k8^<dnXH*NX-88=M%O^RD zQLjyDI^yn!TlP23;k+(EpOntmezMsJ7y&$XP``8cGlwZgX^n|#I^@E6VwHYDiAEo) z{b*whM^(@d`iuN<6ytJ%xD8jCvSgaUI_Fl0B8AX#7+Kc5KD@5tG!1hILEeX%#N<UP zKu`=XE9U%Zgvo+giofpL-+!?x65lNSvrzgf%oaju95IMF4t@T%bk~c@-}dIGpS$UE z>7Q(WgKmaY(sufE+i9hOTeI~Wzvk|}?8NQ&OY<XhZn`;1D)U@q>px(+-~s6w>BC6< z76Z3v6RTLE#1*I8Xj~zV5_+VUWov?40ZdQ`)<E<kUYy1dJVJ@WshoZ#jDh83J3v%D z=0U%UmBTuSeue&mvI`2UgEKIJ-zI$~4Vd5CQhjiiPPas#@%6IeO{%!s!qIA@E>3ig zD>3e{*bD1=6;7)0mX&HCJ~?{D_r2%3!Ka(|&r8Tu_sbqTJ;Au=dIpjraHH>dSNigj zf@NRW#740JEOVmt7Xxn|v4qS1U0*eLL?(_%RXOvtPxs3lS_1FKLO&<;PUBP-y_%mq zLRXfVTr)E;{?$`HU;V(7Y}}%u(md(;^_LVM+&8V0#-aY0&r)I0R}c{s$Y&EKQGjz| zFc4@EU|0#>8?duTKq@c*n$yrK2BItHr(uKi#^;YecUbyrX6-eCa82z@W;^`c@zv7n z_aqq}kbe8=R^qWA<xjADcITn(vsYZd;?RpP99p?;<-BdTh=~jLuAkVpD`QvN#Pxd@ z&R@mUH-E$WpMI2hQ?|0>LW^|ox{6UHZ0e_fW>ZV+E3cF8L%B&lG2y*^3onlV>?GAh z6;vKl>Hz=(uK@)_A<5SwXz?m}ivrRK(C1|69|uod5tMf1oQo@D2Uq6FA=L|rV*7?a z-aPI80(N)FXVSS7Pu=tBU0-LLC%njPkN=|rsYT;lM#ZIvLbFHb)y}A%J8J&k)vpdH zy!gVDF-vb*^H|PQc7c0WeD|i^f8fTJ<n4Wa;;7sIxut#?Q!Q=odk>ra!*Haxu&~K& zd3<nc)v}Y1vIpy@R_ETcXj}W*F%#<teYEv;Zt#s~&!URa`9~GT{9`0^?H_xDMXj3l zE@NP~)-8ZFd3M4PmBOmO2m>Uj4$PD=Lq^=Jk;J18h({2%8Y6D<cv8YR$=tkrVh2vN z@~?Mb7rc~0bQ%`8ly8wOm{cwY=Ec9FTu!naTMNmSM*T(pbUqyka%7!gCM)Jes1%eG zCzt~ON{p*wj4H41hUcqG(%_LBYq1Jkp{sPV^le`M{=yfm2tRoby={^*Ia|ii;I%{C z8D%M{cCE!Wx^MQ-(n6$VdmEOj(CCsCTjm{^e2rD}&G>s~_s<h!>B6=z^7_BUrp?G6 zT%8{iUzO1R?6G4n4fFL1>0@-x+<f=+2dHi3+%@8ASzl7ExL5H!W>sQbsIx~uaN~w| zd9+gKA|&h41|$UX>Y>0*d5PJCqE~_#2Nb#j&t^)>Yal@%pFk=(qQm9f+!=92Mh841 zSWLm`=&O{olfYx_X7odvtfHF`HL0~aU!x5w1^AiMGf)EHb%IKE6_qZg`_Vx>e6@1% z-b2TZAG~?d;_{3bp{P(~mc)XYQ^T8g-?Sw>MX5E$*wZ9?RfRp#Y}9JXt3<8Q#97o; zRVJ53uT)i5T3iY2#hmOBb?<!lPTj~%BhwTWXuW+IKW)y<``&nSZ|~wnqX{yhPd9t_ zpn<dJ4Gb;Z-1yMenq50zd}(JOmQy4DuRYmr<`=kJ9p_p4+MESzR=-m>B0DEpqtnIf zHLAHY!Z&Z(kYEAn({H@z&V$$Ml#9zlp^B!ay|cz7s?~{%A2(p_%&EmCB|(%};H_S6 zq+DWcS(Rwwj0TmqvdWZX5vwZAu7trW7S0(_H(^5E$k`rMg4vWftv{>hwl~f?w|Czg zCS5_Hn&*`_&6-g?ux?O;G_7CF)(0oQuxsbeKnjQS=W5Yu<W3l!3#FutQP!LZbtC39 zu{YE<lRjQGKBFXGn;WM$38kUl)Mt`lbNK{)CIS6^8EM$O#RwcVfY{_vTHtbGc;Omf z?8Tgc55d%c+Xt+gfzRMDQ^;`-gxLpJDA~mX4*0T}UW9#t-aH11R}Me~NHx@MP%#8y z(=zZ%qtRj1mv6ZB;}t8vdHDGB6t$q#>cy7%YzsSdmLWT!Ev3+G(b#j%Fj>TBSu>f^ zpw__F0smj++=867(&hxO&!GQv`Y@|iXYj4uzI)T`@{)$@R_&ZtU{4vVwD&FQYmwg1 z8n^EB%;|Sbsf>#>R#(-GavA!}UQpRrsZ6q(f+PCnmycgQv6sdOggjw+{)1!E-!je1 zukU5hTC;C;s5Cr)iK5A3InI=)RK>7+lB)_bbh=jWP@7HX=rcB5nOA?)_)$A2*7Qo$ zaO*4G0nXta8BFNAV*bedf|`lLQzA#l<YqbIre>Gi!P#<wY7pfCW&Lj|=pcD$%>y-z zl9w(wls=@<?2OIrUh_f;N&dAMM$lX#_r#C38MM5ljfajDS}Y7rQe~BqGLvi;-M6$W z<z&dw#NZ%=L@c8}mteH)qL7gu>q58<jux#%J6q*5daR0yAw`${R~m`@Hg+zf$Qn_v z)E?_?779J&LO;HfeA6r1#|W8}+K4>ZI?bE1^#wBlgX7XKVt@AV>*=n26tghev}h|K z49Acbsu>qTZYYI_ssb#nyBT=J<#h&UrmM7CxM&D##>LSSBX0?cmY>wwAlHA`)f<XL zAk6=Y&OYG>=OXtB?`4oRisQZ<Nl(aVB#I+L(f?UHjHL+tGiRbP=1)HzjXLkzK66IX z;r{QM<`x9zU*&dkz(3tyYW@G#UN*>4=|BwuRxG^w2{Z{!MGYh`{_h${bV>?josn9j zE%O13HdTA$f7dKrUr7PbWp}i_aX0z4k>3ABV~{Kz<$04j=?Dpb;8r?+FhzH<P^<-k zm$7DN*U}e%F-*35C4w<Kwy^sR@LrGpv`6=28PzKSG=uU1$yrAf1;K7TIoAn(;FT>U z-72GEc6M{Q9QHYionTo|*EUFRa|#+Hd(T-CE%&e%V`MQsn!8EJj~<3v{KOC(JGYlk zTS+PlJll(L@ke=%@=}~dR0Y*tAx}4P1V4<i(5b$^OiiJ}rv6Y!F1^Y(wc1^Y>1{3Y zb3@UnR7HAX#~FtDqpEy}j<mI;C+?EZ<P=OH`~h!$S0c=-ig=%VJe1h}f$ZZS&$mPY z!f-mB)6ydI@r3QzjyW~ziHj1GEg2?JzgLu*CE|i$`E-ijV$m#`z4$Qvsy<D0SR&!r z*vP@iL9M}R#wLeNZZ<dD-5$$@+8d#4Z;iBy-`hNGVVhfpfJNO8KZrUTbT8Os@}~Y! zyY_S82YtKtJ>iG8i15RE?NGR0)(x9MQ3GA`4H;@>?i%F*Q6<NTEB165!f9Ahu{nj_ zYL?t8HGWvDO8DZ6&Gr*&C8w}h^=c)MbgW7+>un*M8VW`$=60JJjrr3({3V6f+6E?_ zXIK%zv(tMgdB_cUh$2^v;LFJ&wo?b(l~JYZ7aDC@IueOP0qa<<g8W0++-`xRTqw13 zSrp^6`{!F}`ac)WxL;~3N{O-YLagXDc#QD|qqz6(Z`p0^3w9ePUx!OC|Faw<seA8U z%l#!<fY<1Z4$))q#^>er^N)+%bc*@!y_d=@)A1hV&Y`*M#|WlEr?!!7C(z4)c>-EE zpq9Zhrvcs%0%=!;NKYN`75gBWmy6Ja!2^<^UM_akntdtFmX5r6)5ft0u{j5?%`6>I z_8Ob^=9_E;Rk*tL1*t8+QZ&X2yojLM7*3UE?-lFP9eL!k$%uQTM~$PkXW<=RUElQT z;DW~SBP!<ml<FN{G0Rudj#Fq1^#lD3YS0<MBpAUKssqlFoI&Z=Fy(=bU*9)l#(iJQ z?@=Qt#DAd&D&eKBCn^mLYBP&N8VUSz5WiqTmY1{+<Whg+wDc~&GIE-w3rA_48!o*^ zd(o0iG4fK2@_sQS$Du136#l%Aho(&BCxIeZ>~LDB9cdLiEuuqtzg9Xc{ra;Tr)D(_ z8f{rHH1<T!ZKEeIb7nb$m4&lrsP$_~L#tSQJKy$|G({T0RL!i4DThq8-;#cVTC-t? z^vRfuzxvha9r5gXotoZpRFA5FyvUQ^Jvo9i-s(XI2HvwmDEsik$kq=(WM!Bb6zn*+ zlJ%1Q5V>A@gRZ519o0R9v4Ahw=+5h5r*Q^hr$K^pAYa45O%)_JW!dBpq#2?hMh1s_ zNS)-d1Kf}l;-q2RVAu!lE@1XRlIuK=%E9l9sZEZXH!m)^HfD0b9gq&V<JhUxszLc% zHa8BjY4tU%WxrBH=-e~DzbFNPk4fNG6str|`r29Vj4@PG@r4#o{gSxS(zqq{p2o&w z?p%AY)@xXL-Jq~q+GvPjs)jKJ1WXD@>#`}VRPuER2}!z+-;9AM#K$N(^$dr~Cf#Vz za2h}+P~E4?x|v+~@r{7BhipAjgAC%wWFrj7Ir%bpVMBI`Q1V6<XaQuRanhUy^+xAB z11&r-f;$CM$czZe<)9Vzkq8RGLupef`V_>Rmv&2a(w_6W!t!PHqx-(kdM)E)4Q#Px zP-b~U!`iXZL$g`dAA66kU)FZV*tHD}<z)>#*n6!@*Q>d?xtGqR)#);Cnba`p7RTDL z4Q1sG+(W%5$K@2jXmcy<D2H=%<EHc;xxC__mec4>{0MJ0?lQJ~u#~R3rEIzM7x^I# zQlrkL(`qx)(=)VMZL%)2K%*(RKo1+c7JY+ElPhpPBBke;u550~+o(>)t6n8<h zG6t+4)Zi8<@Km1S93dAH)Bt80z2_`rH~)n#Tr|7=Fq?DY_kVb~W`gv!bjPlzzvuW* zf1lUa#9ga)mk%G=-{IJH@r6U&XY2m*>i#jmf8nW1XBHhB>5lJLC~XT4=89`r<8QxX zqo(%VG->F%p(XKvpA?60yrrwZ%D(kcH2MUE0zD1Ak!E1(kZ^knV785N)rA@bqOc%O zP!I=&sVE@{{0sZsTw|meq5(^x*bM>FMr&&o+{dHyl3e#>)E@J@7ph2zpCI6rl)!;} zbZJoGMHSW{k6`f>o*oHDoqQ^Sg`fw6_kl9+{lVYw+IM01=shnk-1Oy;KP;4Pf8|%w z`){vX_crtW>O5O4g}6tS!BGCqqg|HrN0IE}_;t7Y8@Ic&W3<^nELwHL?hAVtzPM-f z>iO5*)3WYu>3vWS+~OUsT566+u-JE**QM{jl$JF!1d)`aqi?&xr?lc75>`tm9zoE< z{APq=n1Sfb#C?%N6Zo-hk325iZrd06icOGWI__c90jj(4mX42>@#7+Kjgvd>V#B%h z9UpOM3VF^}hM^NAd+v4UC~`(}NOzE4kg^8SU36W<8;LqX;upt~5M<c-8|eOpg|5&{ z#(qrmHkF-56ZAEOI9axzf>_!Mid`J8y?hPsg=j2!n+uy7P56f~wevR;29`yHc6Wcp z7?p{+Jy{-iw$DD)WbUgnRVP?#tmy^Jq>2%{&!hX8T1}V#BPJFihc&5%`_^P?;+n9K zze*Ja{BAR*{=e$p13ZrE>KosCXJ&hocD1XnRa^D8+FcdfvYO>?%e`AxSrw~V#f@Tt zu?;rW*bdEw&|3&4)Iba*Ku9Pdv_L|PA%!HAkP5cO-|x(fY}t^!$@f0r^MC%fcIM8V z+veVL&pr3tQ@lQ(H{B5hU3cf}4x7V@V;L~v)I?6_*wq6t@dtRqF(&Zxdh`_-87jFo zg{9(bQc^a6km*oxBtb82j0+|3Gt$9d#X?J%2b?W%t;(wOlfeAIqtZ25;A4nbqKVe@ z8qq%asL^OLI8WZ5S?G*P@uv8q)`9n^>;UDX_ULuK%KXB_tZ0`vF~1;IzRt6IISK77 z-|gv)Eyz#wx}viZ3-c>|-7zgy^wCu<G2Kyc+L#jMEX)*F`keedqu+Q`63x-0mED;c zRlW@8IwFawaES7|6T<#ESvjGKnQ4oAAbr%cIBjM{C?{)<Kb+vMn2u4IUg6B}Rb^y$ zSB}DF0Cme5`p*EL5~+wGN*NPJ2Xn%t(M6%NB?H6@=a-L-l{5b6c6t88V;A#X-*84F zR1M5yHu)b+d<^#DY^>`W4o?X0{{rKZ1(}3OoJ%xgbRfJ&Tt)B>$;bt~Ya)oH02^A> z?zHL{FI=YWUC4L_u%Zs96<+WowQSBTzrv!*aGs7Lwv$2y=zHr!2B#q>)@n^jG<&zc ze%{XG;hsiMezkXY7Y&E#ncsi?kFPxOhr2$1aeo!7dhU;Gm3R31ubRC%u~1x$o<2R= z8k`#4%yc`wIbK)1ExM;C+7=&Q70n)*)D%-t6q_iRE0U+rIPYg$_ijm?=dI57%-;XT z{{DGazWCW)*MH=B>?8TP-^D$-<^HQvZBbL>I~nhcugb8+Us*55zK~{%u8P0)+2_6; zKQ$`angE(21O97%3H)Kw^?{5e3Q?J>K!-R4#1|JrMzTtP{cS}&H-*?hL0I&l<9B)i z6o@xu<10Ov6^e?+7tRS`%uDbl8>L@f`0%!E4`2B4(2c2kKkj|(ycU=)HYFA;TE8$q z!RSrw$;uu&5M2;nyJlvhWBAIBoSaoVU)Z|&#fw(@lk>v)QC#ne4`vi5x*f|iGwWM( z&Hnlem(96g&CKF7mzmpEY}>YC<+g1<aNNIfR`tBxx}}R3mfBrKHKiHpMU%4ZX+;Ui zoV72EFUa=?3es$bvXz6Et(|xE_U4M<!u4IHg){S#%;so+cv;x#VytRjW~@J4JSi!z zB3xb85T1#{0)k(aZc_Evb?2TQ5HAR`T9IPLELU{C6s2_91{{rsXLp)A3|Dsc)95*> z-E18(f+jMBv@km*uT?$Ws`}>>XgO8h2Io!Cra!F>uk%$gXCXL2%;_N?C)hp_*NI3p zLO*9c^P;nL+SwtN{ng&RU&-&_%08v`D05%sR4GB}+=id{&fc$1=bESTv%dZrXyY0B zl{^}LttWv8RCRvzoLD`v1a|b__0`w<=ggRC@<{)xcgob>IE|eDZEy5ZXQ)H;UvvRJ zdjbx$K;{Ty_n9R3hq1t>(ZxW(1Ldb;KSs(Ir|$s|xUMuAwG~zi!?c<HSLuSwmGtFk z63&;>^=p=Xxp=9N5eEhR^|KX^olF;(A#aC4bl_-Q$^6);{6eB9CdQM<Dun4LzZ49y zsftXz=5aIi=2KuY5UYVO)Oh&t2OsEw7&M*b80_DRKq<{6Jv+b?;qh^02eR--VjN)8 z3x%MF6ftX>8S1<UgwG1QaM+9W73cRL1Q>*_Np2I_X^o_%P!ZYABl3X2mGHCDR>zQW zM&Suv;SA%DgXBtCBtD({cutV6nQ`n0z7>Datx)gle30qL!MpT$DK7KGg=;Q}xGrCL zhbpgr$I8oHkxSNCrWGK9?4#dNFioHy99v&Fd2%5?fZ)kv93s_6;?u<(n9`0*t40`| zB(GDt>P$EW@i}5Ty~yEd;=6Jidwh96CF)-;PiHsfms7YL@Sh4?@@vou0_@DgLsq&# zhhK2HffFY(<(4WC=bWG-{d9<+MByX3&V*<_<Md1@RfJqAL?sBnu9m9>x!eGAnboY! zVK$59QoQ{50z>REr`aUTlM(s=hgAsum~KePrdLx~Ny(-!FvJ~G-=7XqIVNI9;pqII z$6`<EtakgcSotw5Z%$S|e#gVHSf;*XGw7ZC;`rTar8yw#y0Ome2RM<wW)$Xu4$>h} zUU)nZq6Cr^WSIYowj~UDC{{Lwnfvzd-?yE;CcnZ0a`CA(tXe+0Mt6$8THSy5Gk<^P z?*8iW0Q+#?e&O={`%X5q*H{4mUmH89JGBO)3O_&wHUI?r!jI1{DLMbgtO5wHLJg~P zGaEJlV5LoKmoBp`3*P!%#3>-bN!<A2`|!`l+Qxn(zum)z<;#0m4o}!{@4c+=-h0RX zEZ;Qt5r08`r-zNOksfdjUKslb-#ET*sC=Xx=ojfubB;8QlFq4DtogZ!`$cCa*ora* zS&P~m9CCEdnbSFjAO6HSAE*`yFS4CD=JEjoZK6-PkFC?*{FKto5tQU}<dks(hDY7e zUya85<K58`Zcl2_=m1G4oSLp5lOD%RH$z_03!h)`2LmazGFJ`+iU>W00}QqoFh(U5 z_I3)fCvSpLkO+H)?~@-H`}}!1@Vqe~6-Nv>$hb*}RUVB()kzcIXv>RX!ILKas<E4R zn|Ex7{L`2`+t_a5A9(UAaG~<sj$uyFsSA1u8-aAD=#XcD>?#Y8)jb>rWA^~=6v($U zWv7;bzCwQyw=J5D9yuaR>)f;J%XMt|KlfcEXDhZ1Mq5|NV~=fprP4LWRr$)+$K<gU zt)H`l?6>UT=ltlgu{Ty{aMm<?U1Kw27aTkK<ZbQS4!+;DoU!Fy?;qUOe%q7eP#!A} z!HMbgc#iM0T;TraX*%D~lOWG#v@bpV1NO^<OL+qkm{@x~L>#cPR0)3*R$@YWTsR5O zIA6&3uq7mxJGM^9vKoEz&eva;clwN0t5JN%h%MXW@_N4KSGXKsT6H43YU$D{@tvxr ze8cFd?$owzGFd;+so|5iQjSx)d+x!UG@i&t8RFUl2M)N;WFt$Gv>s#A2-r`dRf$Bi z>AxOF>X6ofSS6jCQVeH>63_Bk5f4s)J_dd<Vd#kqqn+h@UQ5eZVb02`iYyQ1o5cLa z;h}41Uk~>op~SgAl^4$0uxL_c;p{9-qi0y?<J}9BCA-bW>N@4$dG>VPyZ;IP+7B1L zH0+AXb|$CfMJ`#pILf$q_uUtd_-ge+T1HGIX8whfFFttPFP~?DOJ@u`aOZFC{&3Uc z#a=jNOyaR{(}54sc%S$VvZg_HCpz$Th0GxOa8#?DCEGdhE2#WZ5~D0D1?v+*oGL@y z5~4St@wFK#p0gJL<uFlZ&42D*MQXDa#6w~B_wtGO0&7D2v%NeaKG9_M#F^~*PGdMK zH!5s&=G#qi9<wPiK7sc>8!tbqFgW?1{-==hxP0QN{{E++<P5ifPk~49qWsY-TerT# z5?MJ*R6;L))Ba~kFhO5crB4VJ=nJif-d{02Ca%~l=D7E4-n_@1BbtljVy3Tn|B$s% ze~ZFe-EdA#RA<lDSLBb*sZcKdhMg4AF!jLS#4Dmco`L?wsx!Wia?#{8WT_l5+{*y= zddR3K#{Q`P9V`>Ft;7OwL)25*Re+~}0H_}6{CX*0oRXs#@+*Y&tIGCWw(8|;cD7%( z`BrA!|Gm`Zm6GqX`1)k_`wVMT-pgz#XJ2RMzOIw+u3x!l?^F9u>>b`S`DOn1hN7`w z<d!p`E}z|J+_3GPn=Xs&yylhFy9X0k{<52srP}35Z+Z>U@^4~_>H@!av%5N}n6I9m zvS)bjSNp!dZ_o1HYhK1z(VlUf-X{s&m6#W&542T6n!zXlB-zx%Zsmv@<^mME79>ML zJ3<oTRfJ5q56-ZXxvvUpnY6y0rf-D-%|-DMvaFOmCC~3m@+wg=38g7Xq5NgR@~os% zlhb1CF^h(+A71p{r}D{9?rCqo=MyF>cXrLWL~$buQ;TKC1C5o*G0`w)>7%&%^hp`% zPFq|?O75ft_f)HXp&{OU^dVM<;wBa=KYGqq1O1V8N|07y+)a?xn6F!hKB9F>;pTuu zgG6>AWXypxT=3$F|H{5PfuwtsIfqT6p!g_fblgBT7%}xo@&{5J>HaLZjs@h<gP(aw zG$_QrGrD^9=qh&k<T|^24P-(DgJmnbxh~FO3xty51pb_~Bpn<Ix53~_iL!`=KliSj z?PGTuz_2mMy?Lu*V>9%YqV%e4vbA=;aBYfUvbgnw@=pZFuUNz%ud1nDwW_*iEIp78 zsn<uW1}+$Fd*nlM+0op1eSDm4azYee!H(SeeryJ;{BptSj&*RB$1X_A9{UH}K{4Cy z8;+Hit?@M^c&t8`)5MzO`$~)wmn=EvdtspyRt`<Xus2}Vd}z{%iNLfLP|qc>eHMX_ zOssGM6bn=xAm$numq;aA5H6YM&=B$gPUVSqYj_0A35IkspBaRNOlh)^@*l)_*+1`L z!t%(vaBx-6*t5)Kf5+~Ue^q9Vmj4#xvhjRVG@E003zJT~Ab(+ZyY0;SBD;<`5~t*q z`YYmL8HL&7%l&ydRY_6&al}`hiH{qPhc<?izFbW|9?y6(Gu^&yy?p*4duhx{rB^L* zWJK}5+1<~}7yc4+=&yFS+-Aw>Zr+qvu&HZRLV_`A)#~k&iZ*wwh>!m-<CS>}4xID_ zG^|!*hXR=*3CtZ5mh)o)CdLgc0m4fdEPG&&LCBw^P{FgO_mH~-?9zsr#KP#mvO2hc zvxrHAjG%kK*wcGJjUx&SASDKl6_f~UxKWN0g>ATjcg2IUFv4DDhIegjnoVz(j4U&g z86~scmKM9#o8d5-jErZ*FY~#vuc(+mH7P|el=%H6<q1=V1)(}E&h1-vINtBd%p{+G zvuDRV7KWd-fdP6taNrhtSlQgXk{)}9hkLKLtlex93lfZWi$#iYC5fZ3>I9dNlEq>- zCKQOK&1)^5DOO{2RMC>MI;)}kUHOZ5ySHYo%3v(oXq_V50rfescC*N3;p{hNyS_($ z<_6j1L5esaFF)`iMXdS*)BRx;MfGCI`>FhUYz4v5ql<AA`x=$jO##1+R*9)ti^-o> z6V~H?*!<h3+oih-9{q@*QnhM=8QPs|_jBXVeLkIzw!`S}@e|E+Pddk+|EK%E%FnoJ zx%k`4ZLM<U>H|}6V`n|7DZcb6R+jmIa+B5D*-w%hIi}vUr*BND<zwsAvXv`etK4Mu zre3wUvmQ>`6?@Q1GX~hzUw=5E#tG_8d-|q?Y7r{^tJ9yvIzVGg7UAc>DpVJI{$37J zKpTy)c84=_2JI+igw)j%EJDmdjF=*-sZBi{Y5Ne1L-ndKJ{HihqBxqi+G{X96iGlL z|G{@8Be)RJB-ucc0UeJ}_x-<BcK?f*ZY6fC_bCtjV{b4cJtfKS_yAMNAHV*9w=^Ry zJ1awVsem-&z8l7r^sE6lX!~4(+bb;N6}DTU=Ul0V`kP?sHYynND9bd(#96HQ&ebn_ z3{Gb-=8v~4Ootm()<eN5-Abn0r9S8;wJtBpN%vRP=D@}{xH9W2nwJ!36$YEjR4*#j zCe1O#drFpSiXjt^bccb*tBADoxE>rqMQFffI}}py(;M-K+BG>`$TJwnFg_$_(V_dU zLeDGQZ8H51d<H5vGYF{^bSrQj$a@l-1+PZIe{>)NtVcac%BMhudDsp>4h$Wvc*%4@ zB_<3{JjklBxfQ`oWI|$avv5WXcfRUy;5Gb@BO}I239C$V8ZsbNLdEKfQiTN%)(<qe zBtpufaZ+4VRc2^*hB>V`vnnc%4~>T=X>a7EQFGF(W|S5SHevO_?5Ko{=$M%3jD)D{ zgRAvU=plb*cVtH$vDiI7+ZVNeOUnF!A*G?{ysNXPic)d*;@O3vp^l7r;epdB;?oO~ z;?y*vF{5l^s_1`H6|*O@bgGM2bJ)b59V$;XrevjsF4pc`iDl90@lh#JtZh-o>?o5d zYIeq=HqH|^8`4>|x5T!IS#D%eZE=RGdGV8`EsjD9(N1%LIS@VjeEBG)kpFh0{8^hP zJw;8yiZf29$oLm!1Gf?ltM2PuuqZx{B-E7iYs@JhQQXAA2mQw3r&xPZW+JwBFm*)p zlny~C5zSLD`3o7iGvs22^zN_>I^cC4q*_4q(FB3rQ`|0j?2=CMIf5W2Km3toWM!vi zlzI=WCm25bfy1AalAaOtuDWsT+2dnRS<|d{TCMtOTt1GUUVG81S8Zwhs0QwPHSlL2 zl6yOPQ0GZmbFeV0cu8}`dWEfdIH$JCpPo~+ymb<0&)DTuEJ{tY>h-wVK8~Ayeb=g2 z!F@Wz4|c=GODFXP0G$2^7||CBNkB(Kevkr?=O9%lQ26Ma(f}5Hq)bnvvkt6}G@~@5 zCpaQkML$Sj9Q}2!bu^*H27(Y&q1#d!Y^YE4CPuN}&a=hXR_)?K$rrKtYxmE(`Pw)p zdhD|ca$}N`J%-q6Dd`n)9m^K(T@j;qNrGi#Z}EI4NT$cmQqCJos0+Lpu)rd9YxVMb z{q|J3!hW7)oXb7OYd+RTUGx2>y@&KXZBekLD7MHKhskO1B-JlWTi&yNZ=+|0$Eu$k z%}m^J@+>tyP^pl4lir0r`Z&<3I4dJT5Q855Kx$qdKm#EG;>&`pqBlw}67LtCL#LKr zP^n6%fyx4~<*FiG1V-UfAAC0&yp#+mgZ~~%Q{JqsuAZojX+>h9)otd^YNv~T;V|kw zjnyf4Jm%1wlZ@WA+aFxF>u}bxu>V$;T3G1A0dHd{&m$Qi&%i$XYT9{E^}!V4#yOG@ zxn-#*#kEy@H8v^5;jNVaaasPNc}0*Xu$t$x(A-sHcNlC;aGKT_T^V~)Ry}at+B+@{ zjds-~GH+I3hCelX>Y9z~a!p)de>>iD{Mjp9Ci%J+`P&&nMU~C)1Hcf&Ir}!q*G++s zxLxQS5{1Pd?SfIV21sPH1yE61Ks!KUYfG?<LlFV5<W&y7)xDv?Tg1UBSI-PnfQ3O? zg2{%Bl!KA>yMm_;z`P__1pOuD?$VxJ=s`*pE`x!CslJ5wr<Cl?w#?8M`M``kV#=xr z`<W<L=TyM*bQXJJ>>oJ+y}lyT%s!BB_805*;dH&79sLC)5WEie6Y2K2gqSDZl`=kM z0*kfyQf4Jw$@R<^E!^f19mUqN^*m>9sQUf1+|tZH#@W+S=f*-K_N$nf%=FprKVRyI zNz0rU^-RQ=91A7V@|>)4p(%P_cE#O=ljT-lo>=ZH&xX9AZ*opnkX1|7Iq3zH*P5qh zW)$#snXJ%ufp<Zw|D_8uxgUNHB$hHS&TC1$mPx#zhutCnedet9CMAZeite70yU&}+ z%2OA3Jx-r|guRg7=!%X_lUK6^>GPsoaB|xGLx<#c9?O}`6n}NPQ^}BrYr$x(!G2%> zr!KVMK$Rp|rN>f;J5Bo(?6!P5qU|vT%3c)Pch0badE&A0SC%xadgP)DLtKPqj?|r8 z?o4ln3%Y;A8_*G&Kvo5>0)u2`c_B+7F1@WH1_DY3yFQvf#;ko&!`5i?`K#NYoc!vw zZuhEF-$IndWj?=Jt~XTX2><-lWSdk0{(V+nEIZ#~zf4?zEI*C=4Br)kB`oTJhvkp! zW~`O_65UI;CT1r-cp*$5nG6r}itnyY&N8{3ZmY-W6;2F3Z*!TeoxgF(pZq>$PRf<P zu72L;HX&k@5H!SkB{?k55U}Sp2s0D2^Tz5#bH4sZ>|iJ)<Xh#(&F!oDp;i?$P<_|x z^-u9h*@;`1x)T%K7d{vz8i>rNwdGr)EOmirSOj@aI>%6ZNkal&y#akd%Z!h9PH=pX z<p%Q&8l-UZl;z2a@k7tg1+6!b3Fva<q7V=%bw2dX;;-HJ+u!fK_R=q|UANR{sGWPq zhaa$o$9%OFIX^vo*q)TSXxoeehfr2FbI0=Du|;#=o#EoS&vmD##TWKZTAMH{*0+Iu z=k3Ec-+cJ(YyYt~rK&pd?|0t)$tQQOm}|>unSE4#rHx6xEAD*#{#Db`j(nTHb$rq( z`SIDCw`IE4UK1Cdl({%QKiRpYvTI-Ol)2E3n83%6*X4lQTMw!im@x|=F;1LfZo~Bi zz8NanVFA(DOnN3USPvw4gNFtrRu0qgkpyHaDRvGISd351$@kpw`x|c>3KfXn$u&2; z`YH>)`XD!_1eR6A#F*dni;b15*+r!}i>5Wk&f1YAUQr*cES(1_$e9xt2lm;#X>q1N z^~f!^j11l7%FB=Wh5XVRZ?du2qN$s&8EW<jdzf>$xAD=en{wJ`EcLpk)nsQzwbcYS z`Gd1Uxu1V+O&I5g%~#~+ly9P;rmZu+8N?k8GcAjx>r1RXidKDjVTGVLT0Jn;=%&b4 z;Rg2DM0S{X%2U^#WXLMY%5+<^EuvA1%GkN&g*j1>MX_d^W76@)P`%T088<qd)>3<V zb^i36ycv1b>Go2a({ALKF?KFD>=KXUSYGYYJ3Q7Tk1Ni}n_TnL=PkP}eZH%SJ7V22 zNmh?T@7kRtc?vyJuFI61o{T@EJ6rOw6X){5n9c#d;0Ek*S7H2tlnGpED3z&Cv;vSa zF%Afdu{fd=#`T$~KS;8SP>%}g=rPh(qP!r9DH^uY8h5@~kzlghqids+!c%8YwPtRg zpBPMh53UQm?!}(WIA2w`YGpXMVoJCwB|bBDQB<7UXm}4v=IzL^PMtF~nB=H+N83#a z)$d57Y|nX>TZ*nWBxEG|@?BYpj>LtRrdlofq=r;Wd8SR0(sQyC60&pBCCQOlX-REJ z(p#*)-3yQ~%bk~!kQr~dvUqFdWm_=^&YauN$6lVGU&EvSYZy4!f`Oz{;h+$3V9B;B zaIj<P^K+xYK>;o02H~N=!ESD}J8h-5^cocoYSL{%o5NvbyP58+$p9d*FRvk~X$=Ub z2Ipk}2>f&XbGS231p<qmfS`zgz^np+$(1K~1j{*dfe~+5v2{&Y&n~It4}d%aBH&dF z+-@+JBxV03=!uqu@KzyY3WA{^hB;b?DIj^vszeagCf&4w+K|U1m`lBmhMGoWQKx*5 z{Oi>}FPi6cOn+?AjyX?&<~CXM`ez-!(c^n%-K7h6Hs)HHe)q>mS?`Y}S4F6yJZNv{ z{?h5q!P@gT)#`PHs~cwK7U`ouDNLH`&)28CXumgfp)=WFNSN)*w59lQ;%<@eNHWB( z;4HB)EeiZSeHrV6mm!lQtzc&11LE9u=UrX1aMP?*^-M*vpV|PLc`fWelWZH9{J`%M zerZ`{23RdQ^CPZ4aQlQG&?DU6o%IWH$X3#vA(W62?Na2jp^HF=uF6HqmHu?hmG#yG z`BM*eOqoC5?w{kg&zn`-ad1+}gKuTIj(s9YpMF3I3a1?EsGAAop5<3l9GX)2z?+#d zNRfO{{>!0F?;Kpc`rtd84l&!onPdH9{rnpK!?DR@lcgVy>BxTpA1z3+&zo7_acD}> zgKuYgKKfj*|Ma*k`|StwY7TWyn=#*>3&|$?{F!x~hbaXr|C3(-$p^0Nw;n8-a=5c< z{yck1;SuJ5q2+fsZ+e$3HamFo7?&?%+qlfOefbl1lTgOs9qi<fg;g+FUd@ieR{2qN zU!qfVuiRK$!suVP8B-4nv7v||k`mnC%u;688%XUHc$SsAplk=tY@9A;mBDq>BK}bP zSV!N%Eo;293od`*1>x8KkdwXXWuZBXda7=zaJ%IXKYCJFdh$1!Mt*y1V_f6{$v@*z z-^sD2{Vr+7i<b++<uAVb)fZEi^v_vPlFFZpzbP0B2l=nY9%WygxT`E$v@f|lsV?my zpaIp^yK-J=0M9QKmUSzD4f}MPbyw?thxZ!C2iqjbec*V6&?D^wzTsiw3veMwIi4_L z8J3G^FYu!JQ*JOd8l6QOG9Xbk1tHC_YM+jXF6>jV`Y20{@<zePn2!?a*Nc!|r!6fV z8!I>JRSICq&Z6Yl^wHK%S;Vm{VXvZ4>(mBX$~nkA!t_dmJi_9%^0c(_i*qJt=OiWP z+?zc)Cnq^6=Q}yLPaeN9>tgwx`_Fsx>V+|#7jI6UQl9K9!>`YmT%K5B8@Tw&8Bxhi z;p54R9^BjCYLgqPTdJqFP30rAztuAL>ayZh?V%MJ5PlVBFJa!g$(8b_tHeopS^;G! zq^Nvl&&D<3;D%|wtQE757RN>x)b!L&^0>U*EtunDoy)$wG(BO`vPBh=)dq0!I}c{Z zr5BW~6n|e?R8(2?)#AbAyu9SW<otYG{!NI#bmWDOdyZ{hzV!aLzb(HJsR$_lQsUM! z*F&@QuD>kZxNYBoUo{l-2Ltox2TJG9myfNxy{BQ);oi>mE`510-d+FPV88sw+UkSx zY%s4{&0kks-^g4k>kNfQ2g^GvF1zW%#X%hGK+&Mk@9w`utges@Qk28R^sz9avHSDn zlE#U9_&CUpkd#0$3$77pXRdG+A+HS>aAHI;VM6I}830cLF{KlU3}L@sKJW|c1&ytj zU*5WAa%a!}Bgc*%x$P%xMQ?8({;}wDNC>_uHRX~yE3SI}s!5SHlCOAu6Q%288_%T< z&>TfyjLy=t@Bnotz!;F60oD&mrd&BL(<{=?pc4Rg1Y{n)uH-wn&Xhk~a_cKcrp_6C zWOUBdr>}2qwLce}yWFzd9q)&}>f^=s;G|;tJJRyFf%;XWqpRu%;_CAqJSUoyvllx1 zUH}<a3RS>A<Z9hMoeus2G3g`-85>A53Fm5s9PM$y8v{hG1t?dc1>}<q&7esD^#y3w zr;}MaBqEgtEM{S_0YTCbRRjCzC{JCiLBT42S$1r3vlvNeDMwT!x{UdXOEP>O1U%O@ z`h1N(y~$h=A4o6sT(IawV+E^xz*Cty$FjQi(2bJMnqZGHvYerTc|{fdQL{pBABPLm z`V_+@>((5s?YLt_#m^EG@^<Dw6NkO4v(cT@ZGopvrC4T!7}mUmzJ}DMvfK-5Ctq@2 zf3f_5d;{@{*lqUOq)EAxt20Jg7cOkQBfYvLoL-s$(z`Vt!eMaTp|{OStIiF2FJCLJ zoAzM+D;Ah#t35P!43gb?i&6e%f04VqLLZ1O^O|$3=f`s0!*lBXlDasxln37R3Z*Ho z<*^xi9JRAh6kK6Io~CAzxwy_6su0sGS@lH>ayI-(yx(4*81yDu%FC@$8S$Z%8YhNJ zp`~;R4$V~dPG`0O5dH>X04mvw4)m}Lj1BP$Kwj7dAV=`I{a_A|5QCH~2C4)D)EmBn z%7evN71PkL^|n5#skpJSF|b<SciYy6hL|d&W$dTSu+Snvs5{>By8&r!3Er2im7X|g ziAS7ZSqK+sje&V{XU$zuyigcCSx8FM!s`x`p)9I0v}Q}AI3qPPGp#{t+_ENA8C7O5 zjotZ!DaJTU5QW~gK%lp&GlZSPC@W}*Gfw$|adKLL$5Z5+O6vvj-PCU_fxmO?zyV75 z8XTSrd1O{!wPc}r1WXntL63%)Wq{-1io(Zc7E&ro4K!}h1ZXDk*sy~@e<2g~7_2r) z&t@3~bKV^nidnhyXJs;$Icr|NU)p>}78;vrOt7qdLz<gEEtav^Y56b?&C4wD&##=e zN`6ad%x$3PhTKNs?H^Z$>;_UBRLp!(2j`r}o`(yqxwEOv*>ejs@{S*0p2Pb~@x^Hu zH48pp!0Qd9rig1UN>=(tG|jw4tV&5sOQ{l{&o>HVe&NWX@>##-waMw}$+i6U!zBT$ z;p9594|3nhbxNlnDfbVuW+^$nBsR7rJvrmvM-~#e;M_O{Jh?vtuZ+tb#p{w`2gr}T zXh63STn#UnT$x!C^9ork6B>4Sb`wJ$FeC|?tPIxED7q{QNAi%vD0A>E16flmB8hfr zD)>WLegPte{;ct9Sthtuo*0*+=pExF8yjV$%Sxs;Xd{cvY}QL@?|@MdZGj5yrymyo z4MgM=JJ>Q;H1Q7DE||B(Fg6u#apjN2cE@k|*avLHC9e=}a3AMa<DdYi!w<P#(~aUE z42BkiD8DCi<40l{H%wUz4<E+0JNP<oVh>0Ho1%B?H(n@7TO|ErL3%|m{Y~T!xA+4+ zd+Sec%BAoA?QOR6O*Z|fW5?fOFvE6B<7e}k!z2V7^!(6^>}U6#c<2wee$F>M%O1bw z<dYv?+0`UUn^wO+`cZ1hh$K8INsDflgg?NK+ZQ5>GKiT=^{mMt6|@=I>t<mWyB^K~ zOANx4C0u{rEKw})(+d|0!Y)xixJ!`u^}xCcj4L!;5->ls>ga$z-7b<L3s|8bZYbow zU?)dmiGWxEHKLeEP-I)x!+j_*VKGqUNk{<pOh|mdl2Y7Q3b&r6MTJGB$z}S|#^Myq z;wij(%HsR3f9rbu@$;dY0rwh!RCvcY3Qrm&@v-jomE~5nMWz>ssm@rlIo6pf7EF({ zRm^N|<~R0ScU@2Sb=S%BkJ_V;QFaO0p(3RSeUEBa?L0yGMiV67R^ZeRI|1d44$B%a zmPiy9Ed-#WCc*z)pbEB)=qu0q7VWFFq!Yh9=3JS2QB*&zxNv5X&uN%nJ9e~oKC}iF zgd{^CrXVTDpOaJ&6W|ZIZ0l$ijbG2|1)J*>^ng!P(|ZxKSvVh`+Ko?^A4{7ubH$vT zx{i*z;#KSC2E`PM*MxswO9~S)?G-o8>UCnTP+^1?NR=2@%})+=u1CQyPX$d<1Kq+A z%vs`_k3#@g0Dx=aWuOH7=&5nj+~KJI;aOdBkq8SjGNqmgjW4?p6wyWJG*;+~6Y_I& zbMq65^%add(X*g29bUBK`#W}gUrd`QN+07Gd(jaS<KX3vn2GTWtBHI>u_U1x;E<0H zEa(9dY{_VMYlWETaGOkSN1|BK+C932Po=_l$iJ;7aH9*0Mwu}Vx-iR`*m(q*>n6aY z3Z+oO14HrD=-2vh2YOHi5-^!cm8Gr>YIa=PT`1%{fNk6!M@R#{fA#FbPKml)6~P20 z1`0*f8q`8xKe-Wgv%<12JnQQnyXU{?Qb5p`3iPpcN(X5cJ;>$v=-S#Z(JNZ_zB#(& zYdy@KRJwO;-RX|}^mOn3?R4D907142<rBNuCBA$HACS7^gteHo+Ox54fO=D`D-lal zkxRqjAUK$cgd=EIKg6)Z5U%w|fAMVJRQLERkJoM5?pe=f!N}*V%QR_Y{=t=>$qzqz zTB}j9g!`i#Uv|z~v}l&|<r@#&e?NZ^>IamZg&|n@y+5C0C-@AF;Dly%K3Yn4d|@i} zw0S@>)vg&21d}bg6rRfie$4_Ve@V5ydj;9v-77!*8A=y>_n#4K++X|ocGk1~^SiVL z>vbec`N;R6hI!SMe`d3l>?fwb{MAjWtfl<XO<%ptrjL%^thM@d-zmHhl5Byt4Lb|v z5d%*VScmCSr3;}<M5m`bOYh5~(=$)0_(0K;HMRLKUUBm7D_+d6U9+TU{({|4z4p}Z z1@d7{7CUeG1qZLaM!xu(YY$$qoPTaFZrSnk?Kiyl-VNJ-zN4kse!;b!NR3RL*K(Kk zF<*UfRr_b)$!yzZkpJ{q`&HCd<?L6YeMl6rQaZ`?PMA%ez*$p?6l#_AX#!aVX*0AK zQ-hRI2)m3=g$3n}?=+S#$hmaa)4O&(&2EN+0~=X711yt`v$eGZ=0_pUxDFU&>FCm> zqdjdEvu9U88A1W&6Gx<VUmXFoN-q;tr7kuq&;mjsFQ?%Gk}b)Dp2=+R79xaCm2*gJ z-__%4T2B8JIz1P?LZC^*oaz^yi*6N!svrk+#6)pBk1*;;t1qg4th%T^%`r8f-JL&m zR1t*{A-FlF%s-BAm9a>w%8{gnN#=VHsa?*bB4?V>_AimbaQ4Kn53gAksICqyTN5su zJD1&}$mz((kWj;@r>z00&nlWd6UqA4QPPQ1{onQD=~bGSDuBTM6;91O2d7F3(W2s9 zLYn8|T-Uz<Bv1#VQ}gAe`BNwK=Kp;^j(!P{)2kpi0e#k^@5)a1BzsHg#jzu&d-03Y zp));qChp{e;<^(AmS>|(uGlC$j(HT1b)7sgrKj;IXEZj>WT+fM&LD1J_OR4Ls*l*q z(0*St?x?Cn66Xlq2=RBXfAIcmuf0F3!jl#b&CDrGE$O=Fk~`|^*v=7bS7u(Zditi- zwW-ZL2jmZbwQJY=ENTCiKfZAN(wlb|t*M++%RhlqRfYV#{G9wl`NvUtlN<7qoXx9x zBKzeX35|WLYW%Zc^=lYDzVEu5<-IgK<MOAzyzFWDN8JrGXOH892>1gx>U`KST(A29 z7zKa>5}U&3kmea3T`C7PP8?q(!vL&C%aPcrM^Mg1kzT=ZU_koGHY{==3Tvr$@}meu z(76{7H1?;&I71DJE<MRiU^7P?a8f96CrjuPOiSgNNlxqq)xvQiL_AGMb3xw}oeN`_ z*unhB7sISmTR(U+YryaFO>HUJbY5U7kF&c?($w^%6EDR3)04!Cc>mjVaVxT%7K77Y zh?pqBk>{-y%(hC8Bnm!1{Hf<A1t8BFtxwo=UCH!@lDMP<S59DZ&g85_r%^8$%`o8a zP0ThY)Xtc~f4<vQId8GgnOr&F^k`17vWt5xE{oCUYPhgvos~yoQU%8akk7xAErBL; zSAr{HYWu<$zLJ0U`jX{Byfe;oeiD2g;m2q0Z1pd2o2`0NZeU5_oN%sWEwXvy=Qt`I zaWQcfFkr!plQXLd3#&3)s`A{D#094%@0qP-TY7so1w%Q-PLtK0<SH$lRb31tL#Z(y zOQJV!vAbc;kzMkqzwtG$nUaF@f<>0!vV!feb#LkwVyxaMx5<@y*LL}%dvho98^~G} zG!Mgm12%DxTp%-y23ElgP>F!e<8u@r#M`blW%*7XNs4jC{))30i@_o{144R^Rr8*2 z&`0<U|6ab|DnDdNj5ADQu+Acx<;y<ubl#T1Z~P{O0!%EJ=b?4Vrko^~VqGOG4BEb; z44UEKJOLm86hyo!Am)Q!HsBasU~C_J&}j9L*3I~j6J}|s9Ib2^t)GnG<oYLJzI#*^ z@*7pcqeqp0r2Ia)Ek@XLawMj0^4y&Tf{vobbOk%-?s)Ra9goXjGQ;DKZ09e~(^$oJ zwp&%MQ50;b(f^;fo$nd@|83jZ9{K-C+e4*fZTJ5z%<S3oe*-a^ze)w(O$vQe1nYvJ zB*)<$Ysr7#Zov3|0_pQNY?%LF!?=@R+%O7ICr%Stt8aiL;ZmFgw&TQSqwYf8PTiHd z1G+<48zDTSi3ftw48#0%{!f-cl*4{6&dSM|CMS(y@ajcHD%j%&>p*=TzY~ufG2^DI z;q(2Q)BlV7<N5SjzVW+sz1ci=CCsEPG@J8hxXNd>uRm}~M}+kHr>C!dWnn&ErK*Cu zE0x>r%5_Y=!9E*3GS~n^U_5eSLiybZxnwPulF6?oQ?HO%i>G#=8S&=)RljeYeqj9x z@a&1IUpOl(sV3iSmhVvVt^C?Gs8pfKH-G)@yI)IBZS@Byro?W5#*<K263$NwAh|O| zHw`v1F3|1JU7_2rJE*%&cc1Qix~Is3FZS4m(<C2}16ik^u<sr}=L;+26At*byE<NH zp2y=S?${@R)kBNB0u&@YM|?;yAJ2eIz%yR{c>eMGzbgOS`0-~wIj{%qH??L=S2NXR ztHxf1SHsR<bmBtgTBuz2P6(?p;zE^L2N&(Y`kpz3ojD$h1=gA5=ari$!<2|KrrFXl zsJo7SE%)pHHb$+D9X)#2$g@PdZ}fQjqP*UyCr0i%%DzKwi!PjSRmwUmX6*F#fRC;} zFwbT8o!z*?;Xpe1T!OiY3FW9|F8sGKYF)}6l4Koql8m*v8oGWP01*m_=u=5KA9Iu2 z1jw=||H*uL2Ix42Vj~)o*!-aWNL(}wc0Ts=<D&gw3o0j4P;n<ITbyoy`#?t;bjkr` z9p(HL2T#KBsFYZbe9{vO@#&*HS^n!gMx+!(Xw@8k<`MbD@11}A<|f8|co>pw0yA>v zFz!3P#c0_0114N`D=T_$``GdAPi)`*1iPhsjS;ks*I=%!9eIAkj-xhnU5(igD{-f> zshbOzynpf4|Gb7RU)uk6%gU84Z}%;`lj%N}&tEE7O~uhZ@RAp>z+(@yf;-KIp8I}x z!DI5P^955(tf|OqvWk_zW+iuA#iVDpn#>zsli$mvI=7$FZGCgP-e?YHo6X_93;UmF zwmN>eWA&Yr&E}k-$*7<8?giVAU#2(g{Ie=s13AS}aA?3%B=_Db)9(y}j{!}bz<8*~ zJ?g%B6!NI+Chq$f<~O#PjBK3i&fUL_9~G&2j~%7mH(fB+3jam%K`7{~!1cNu7L~(+ zy=h;dw&bj>vBtMm9KnNrBUkX)?+a+$*pYEY0AHsXIp-+-6y9(hF$h$CqJVmdLqK&a zaz)CwldWB7-owEOwgIH1fMZBlS);Sa6aa|k1qDt}&g~oVTYJssk3Tk>_X4fr9*@9T z&wOZNx4r$Zl4;pQ*Tg=hzCoX2Y{;`c@qPYdySUmWO6x80W2*PAyVU04t~7VT^GVy+ zhnU@kPx*$lr}N4$i@LL5fcjI#@d_-FBkZq{^@S`jHYmR$t@{QVp0)EJjtpP>CVHKC zwK@aG`T<k;=#N6!kN%<V4fKg0<~}@-wjldSkZ;4r%?-C7inahL2>{8vN%%r}=W%B$ z(_Hb|gBcG?AUFkN5Y~VkE(GrtKO*q7;wN+fJOUo29}*gAigXo;osss59xv!U`MCtT z0Y-7tL3UXoH<G9z{;ZqrR6sUVoNd1cHI&I+7p&q;$?!N3uAwtrmOGDX%no4MwBE zYcw26x2D_tR;zm3LQw{z$I14jT^sfninHcc`?<&9(%S_|Fgz!CeQEma<<nfbRx5`e z7D31>*PGWbp4^j|Y{)20DOhSxob0p(vRs8Wo6THMV&gai%S?{*q({<HS3croFfSI9 zmzx7&R+BHvTOOMdV}n!S?DY~$wAsXDNVYq;DV=-ba$R<ttKj)ZQ<iQ_DC+K<n)P=K z<U<&-hZGNNbHUJ@`?;7A6_p(wC1t!}G#WkhNuGu2Hk;p;W=pqPv#jYGDc)#wdRA8F z`4i{1EOjSm@ECoJImY2}$I8*}B)i)@z07H|Fji3D&cVUa!ocDkW<zO7Y1^!@L#!_E z8H};GV&kFdpNLFJqLA3MCuH|Hdy5Lw9-Z#ASfk@AJyEVH$y{$@l0$;)v7F3MdvV8b zQm$Ta2;@wiR9BlEj*m?a`}2L4xCfbOU2t3Znk9<_&bE9L2<iyO0S^KOrvvVyNMD$w z?n(t0<N^UHVB!%svr0r94jB=Zo+^cC++q2Q%*2P_ycTi*kb}S@@%^~I;{n)}Du_z$ zup2s(9FxP#<bNz%$s*|`Q;Gt)(Q8)dOQpSkEXW_bT)uQ$T?xZMH!rH+#y0b(_xw@s z6s0x3l%)-0zf5#UGsiaIz*XP{Gp>Z?zGt@82bgi}jd`<0OI%h}?mL<C!YOe5Vr`GJ z6_@etZ0}#D6xw3jtqJvGYgV7XWdsD8avq}ihl&Cxycjh1B8>wImJ5vIN5RxqA_FrH zs@2572~8G=#8x69z5(NV=>~rmtP)1KN?i~;E|k*J)1YM>DD}XM1K28x)<y^k${_Ud zb$yi^7~_)BZZcu_4qrvEuW5nldr=VelHQ23*c0z>-O3(Ze>l-?J=9$=Cy(7F3C?I= zOiomcQC#KDxT_pC^QMT7w4}n6kv>CmQNZ``#3MQW;Ul8Q=rkAw7UD+1DS2AAFt5=8 zA(0!o*<jI=g8{uovVY3nfse6wezk8VZsjl~Gu2+_b;dcZl7NWP8><fQNb;pS(r%Em zlQX6W(G!t2))h%te2N=qOaH`*76bm>B50lJByg6<YMBR82m}id;C2x<VRX2lh-w<D zpjDkL82k`8Ac2A^PVl7=Cz3vkRIUkmAQNf2LD5~G?$Zen^o2hgyz&+b5Z6FDj_r=} zA8;4WfDMgkag$!M6<*jqk`i^OmMxUm+!^N+#KoD?*1WX*Wa-HHKbPM>e69S~^~sLO zw|{F_PIhXxNfa*p$t_zOL`Qkrd0#$!O=hMi9nQo;ugPP(9?98#=>=I?S8aao(^>ZT zhF`y0oHk=sMkaa7nFW=1eN=iTkVoP4?m&{jrHbrYIKMKwrruJ`EsJt?C59YnzC*C! zQE}jx$A82GV{%*XJ<Ei&H|4kZq{KOohC>Ultl`DgiwiySp_^I88y9q~t86c=iP4J! zOUleNTViVGPR`iymr8w3ZGBv<)8vY4j&06#i|cM)Q)97u{jKbLX4*CPHTjQ2sg`&c zEnW%xe1QwPR>j9#8~m4DwLLeN$2j6+6B4ZEl*vZl{wrR(WvDeV%`t1Tf8LPXfbq*b zW!1kU{S_xw#h^f!DHf-&ED-(&wMYUV2B<D_F@vbgDxpjt4}Z5HH4SbKkv*_ul>-?j z6~eSPWM;Y7&#Oer#)Pmg3sa{oS+olnaA``?^re-%BGFb@dQ7QI$e5a!8S92~PqrcW z%%9*w@2k%r?vR+n>=#QrVX2g@V=IT<{4WbG{r+p;zjT3mV*@q6gZa~+$nVMWBaO)= z(wr-w`rxy_AAe<bYUO0B{PsVMoX21A`lah)ylvNfaB0tzH@EQ5^Y5w1N^&Q7&5|HG z%(l$T?2Ob{8yow{wkMinOSd0o-dl5)<!rXSChwHx8>~0qngDl_DX%?Ehd@uOH~qD* zwHg;Z@OSyv7j9++e|`O1ksR-mTZaNy$`}2WEw7hQ^6Gt0{p{86?_I%@+xEVSsR4Ns z&@>7TC3|*7(9tHD?tbWIUj@DF`(gVBa;IdW66dL8xw7<s`%G5MfNC`<F<8=#ag+Rm z;^UU&9&ncMzj{78;4F6H4;DH9?OptCdG^VpWHVK$&MpUG^gm}&iUiBg>2&(=`%gnh zzCs1%*%DQD!bmw$!sq|PoyLagim<*d!1{JI(VBo(P%#kG@j!@A$c(}>yt)?AcAAc2 z@J=zY5+y+c4O{4OQ9sO*D%dbC07Zs_2{OW>#H3(>#ID;VMJbP904q|7Nu-?yy<RvP zCfmo55Gq3|MSh5?!!na;W0_&3&?n#Vgx6-X2fR;|*#gsT?ed>rbMn~K9OnSo4Fk@c z)L8C(P5yJcZF;~~_JlV8LqFap?nsI^<-%FC;u!KJ(Ug!T#wSog@j;JP4s(1%Im~fR zISKJ%T7pTGUs8NphLdtl@$8n=Zd<7rjaq-iUuw=|`8UZgd>Wmb;xa~$zD2TtZ;eJ9 zT`9TIpR$UZ<To<SrT!!9(JF7Ltvgj;Sd^--I(QI<&GU05Dyh(JR|}HgxQSJKb3yV8 zH?jP)3zBm4GZ+=5Pc15>aXdqZN7Igq5s^!a3Kj~lCj;(!JkeM~M1#cqv_}Ts%8;Hh zH12(EWca<?x(o>YY~)7fzL!mxZ`r)XY<Wa}<B^kbY?Tsb1&`dpZ{STEe{plfV3>E+ zt0PLtbgAx?I7Pm7M1JY^N97k^h`WTX8fIm;KgP;mi1REbqDk8un00no0QaC}BysLa zx3F|qR+-lT;-vs4*|IY6gBc`0&i*HwK019KPci|*!?%>)e^1Fn^I|@ak*BfZi{;nY zyPtP_#<u-n;nREyEokzy^7}tO|NNh`G=O8;&;RsI`;C+Gqin7GZMHGBq&>j9P|C%d zIzDS(x!~yqYn5Ecf2Jh9=^Lm*>{(AS!%FC^F4wi_dSGSZB6y*CRQIgzW!*cvk942n z8zGA2hoCFA71%OBmJ$;}uWT`($E@x(gc!ZDg-~`0;6^B1i7*L+hrI!1y{A<NoD8f# z$Wg_Z2Xjub0;{vDd<#rdio}G&1cC=U5D1h+%>YTqa2d@@6zTCo1Q!H`o@u428I<xU zs6?y9jHkf791l{8EXt^ONs_{1QLsRe=FF1p9R<nYj~E1lWFQ;p7Nf<YPtHhSPE(B4 z5>C!p?{x+;^E?Y0l5?UBS4;X7dxD;~Fnwu*TU^wrhboN7w;8N~lBoLGfs-|Qr^6m6 z2+l;l%xXx>v088$i^-UZMLaqhS4nhP%WM4Bgv6RlriFS|_PQ@RG{wp~{yIG%EZUUo zugVZZ>+5|x4?i${#-&@97wLlyF}@Rnc9YvxVpFd7iqUC_a7yKjN)&H{44Es<7~^)Q zj`cVli3wAjPDi+ke<eB{4r&G(^L+=LmWGqf`5{&q>t?a>MUOv_72z=D&!M?0i14E< znc=Akr;1+YFkp|BV2duyO}yg#tJ$<Rk;kXn*oe_$H#q#!f3f&8y?*1JhfPV$@07nx z47J6YY(mPN$!cZsdimX`oLFNNU4(Gh>WZ$8Pq0S2##myV-&$Vlc3FA#2Kmc5Q-#L0 z5<DDFFX)`P%D&;Mn=2<@I=INR<iZMXS&X5kY1d5W!gU>dz+Ga;S1VUEFbVF#@!6v5 z<Ro{RD8^)`NupO+pD-1losd|S<{Vq>h!ce<yCWVSf7}yxI?VZDAFTfCVws<ILdw4) z-8wA{9~`<nwJ^s{;)hyT0O$8}EiCFREM5ysI@tcv!g|DS(!vJa0-mLXjXEp8RtuYS zX8wp4hD<8|L<`$>$wCeIJWPazJe&>?M~T7=80Km%%z<$p*1`g0SAVL7MV*HckBHJs zx(s}m8rCDeNedfv-)7sjuu&Jww`gIL&drZ#VT&%8Kcj{1y2*k7-b6p-jkmzhX%}o^ zbi&7&51O0JIJbx(G##NnXf$m>H~<Q>1emZ8;TqtN9^B958d9Djx*_BnRC2c=rLL}j zV9Q`vN9VAwzIkKBH@&&9ZHq5ZToNwy)%5iElvhK(!N^c#aATwm85+=@KD43+_=!sE z2Spn}bbsG)&8Emue=i;uBBlfKE<I@7ARZ$fBR$OsG=M^z2d&dN$XyFN*7Ff=0PlV| zWUA*Q7xYmZ=FeL^r>3@Y{^Evd%Nyq}q^SR(#-++v4WW;ybv|7X-&TfSF~Z~hqFWjn z9O~-t^92jb3X7GG{Lcz+#D_%iDb#h;r4bw)Q78J)4gJcsQ+e}ELq&O7k#4+U?Z~0# zRP)d?btjcIh&tMkzE|nCZp1Ysmg2jxAdDb1UP>Qw(Nil@5796-_C%V8A{eLk$e?ey z-#6SD@tqmkp-Ag6eRz96UgAwV2Fo`**xVNBZ656QH4hIDcD0NsN&5PSyILbd+CUGY z76PVohI(+=cY3V92^Mu{U`eNd>@YyM5+r&NdQSb`=CjHyRK85tIXpZ7y&h^_vkFUv zUH$(}2}KwwwO9I-(JDgbZz{8>2Orrt6v2Ci#-ZE4`p2Kc8wN^9z$xJ#-EN#QU9GzY zwu1KRu406);cgXD<OXJhcUCjAV3%eCx#?g|r~pPY7jv^1EP7rR3x6%~ECJFMNgz{G zm=8?NH0)k8SSIsB|0J8`uv{3c$!BD8p@<d3WNCnvvLFkA=q!T`MQEMC4g<u~vHVVD z)vN|yR_a(iT);N4>1+m@36aLx@U1YH&13UfBU`{0vPIbGEn!R9GPWFkVOFwLY&BcM z*0Lt-|C(6~@Y!cN8*624EW+AZ2kT^AY(47+^Q{;9l>KagZGa7wAvO$?up8MXcq8A! zwzBiEF}?ueliS!RyNF%PwzEs%c5o-#1xb?2pt`z;UCypxSF)?v)$AI!mtD*DvHk1- z`xcC{UC(Y{H^N8IL0ITM%#N^|*|*s(>{fOgyPe$uPgi%byV*VLUUnb*4!fUymp#B9 zWDl{2+4tBZ>{0d@+^s&ro@C!=PqC-j57<#y<9wDq$9~9u#GYp_uou~n*-Pvv@Id`C zdxgCUBf39hud|=CH`tr(E%r8hhy8-R%id$ZWWQqXvtP4g>;rb3eaJpyzkxN?-@$Xy z$LtU6kL*wE6ZR?ljD61j%)VfMVSi<RV}ED=VE<%avai?)c9M-TnVsS~&N$}+7rDg2 zP2ome8aHt>x4=7)jl*ytck(D6&0XBhW4MQVc`T3P@jQVi@+1y^3#>Y)@-&{#GdL_q z@GPFqb9gS#<N3URL+6$k^AaB5r98+(Jj}~@Ij`WAd=j6`r*JTKcr~x#wY-kk^J%<+ zPv<lEOg@Xx=5zR5K9A4mjeG%L$QS8e=ZpCgzLYQH%lQhvlCR>c`5L~KH}Q46nYZv( z-o_)m9ZC<Kyo;~r-MokQ@;=_r2lxg)$cOkaAK@GMCcc?(;amB6{Cs`^zmRX^qx>R% zG2hNF;XC+FzKdVVFXOxU9)3B$f?vt6;#WgcbuYh`@8kRV0sbw19lsuQ|Bd`6evlvH zhxrkHGygWfh2P3=<G1rW_?`SNemB2|-^=gg-{JT3@A3!ugZv@>F#jHZgg?q3<B#(v z_>=tm{3-r4{{cVBpW)B)=lBo#kNETa1^y!cF@K5wg#VPk%wOTJ^4Iv!`0M=V{0;sl ze~Z7(-{HUD@ACKfFZr+d`~27Z82^AD=O6Nq_;2`c`S1Ae`N#YZ{Ez%k{1g5u|BQdm z|IEMOf8l@Sf8&4W|KR`RU-GZ`34W48H<q6gbfDRAcn7*o67(=fVT2`UvtSXdu<c|Q z9D);ep`!(t;1*(_i|rL+g*YKzNDvZ*BmvAg!6&2&X+pY?A!G`EAxp>>a)ewVPskSv z1n}a7VxdF`2&F<07AV6)nNTiN2$jMlVX`nqs1l|M)k2L>E7S?~!Ze{lm@do^W(u=} z*}@!Qt}suSFEk1ZgoVN)VX?48SSl<NmJ2I{mBK1vwXjB5D>Mn~gl3^dXcgLoh|n%{ z2%SQguwLjEdW2q~Pv{p0gbl)=FeD5MBf>^uldxIXB5W1T6V4YdfD*|zVN|$CxLDXO zTq5icb_%<MONGmX-NGK>a^VW$O5rNuYT+7TuW+rfPuMRU5WXc`CtNSwAlxY2BpehD z35SIv!p*|Bg<FJMh1-PNg*${hg}a2ig?og1h5Ll>2=@!$6&}#-lRA2uhlZryk)f_u z{ZOQNu(i_|>Dw6T=^uzlop>G=hlZO6&2(vs^bQPf5l29^i0xfHy~g3rCQu+95kA~$ zpm5jFFz@fy4@P?XH%1Iw`}=#Fy84XDy?<m_=x=Y=4|R3)HTMXu{T&jP%AroN6SR#H zrA3;Wdxpi~uHJ|^*x%e{ZR_9E*Mk5>8^<5?BLfsCb@jFMZ?+8dG;e8Y?HX+DiJ;Db zNb|4(OEsvfP9rr%DX^!%wOefOY3?xNW7-Bf`}-n8=8gS5BfXI(w8x?asREN09vRSY z7;Notix^ta9k>g_%^f0sLt;yRf47k?w8BdRgI#^Y`qt*&$Y8Tb%PZdZwCTHso3RjD zh9jGYn>r&z1)7!crmnW(PBY$h^fmQF+J~)b5KHE8WYD5MD3qa14X+;=8t!V}BGR{5 zy87CXPR*x<m9+Nu=GF-HTGPg^wn)EWpsRIwWH6#1i1f8~^_Y8`2b!p@kwJZP8x@Ei zMmr;IUBl8)XY*i0YVC}mhiJfTL&K4Qrk3W`?oG{uZMODibm91uaXcVWr%MCP00IC; z|9}D14<)xM$;Tg*lC%dYvL1wz74>W!>{q|sHvXV|f@z><WT=S%!`$1|rv)u45Gm1y z{%+-N+b|MAm!J?zZ0hQ3?^m-8wGKu=cy#s;+q7&dRG5%a4Vhb-`^JOKgM<B>l%BMx zL8TQ&H9Rt4Rs#w|C|yKwgysx&ZH+XwkM#6dweV1Hb5D;mvbnXVxwrXrv&4?B_F)l( zV>{-^V8j^N0zkuPm?+TN(?1lkqQCmO`Z|=hOX$zOh_SV~C(_r}Jg6VUR-wPw(AwYI zi}BX?Hh1(zhRx&sH8OCzAE|u+_u);E$gmBcJ}^K<wqn9s+X01<LA5fA_CQT=Xbs;O z83Z}n+!LieclHl<Z6)fk#{`IPY6V;jTZcDw4FfLKjwc`^uvea}D#SOT^1*(gJF-Q@ zY#uUdO&+ohcaHS73^k1mP{%p6Cu%M|n-oOS+1%4^QLu}OQw%hqKoPc{uD)&nr#ixh zfsvt3e3lI}DKZGO0SwdEt)P&uK0PWQ=-guI=t2!GDjcZTfvS;ufP?xZWxiV!5LYX* zk569p(X1p;G8?oHHjaO*UM*Qa0yNaxZ2^d3(ohGt34=pJLT6i>u?5h8&g&CfB0W8p zR_fMvbnI}%+=*dqQlVQ3(tI~4p^*WTa;FZ7Qh~GS3`9ns6{8g3I4f#o;OtCP3~+dV zOGLkE5Ocm$8g3ry9?}D&qR&h%gI$sKR%~L-1i9)wkvazZM+Sga`nn|m<Vc&oCDIHG zA+(MRV<-o<m<O5%6!_^H5QngTHPh%cwE$4N0df6<04_e#!~1)HF$TN3Bg362u%pv7 z0vt7nVk2lnOHV`s@OHHV&5yKpn=p)M52mmEbV%uTXGed32ax}19AKFcAz=(7Tg>S5 z$Z!*VDdq_UF-g?`b*n`UDt(1{1I*qxBo6ft0@QF(vKf>RCeQfFMj(PULWMOE?d}J_ zbO8R_uq3tgV~i~tI8#dNIB3%Y;rL;|>o9hC14cmlAjZBK7!f$n4BXxcq&d>lVgz2m zICn(sN*625pry;IKB|yvpry2_x6OjQ!=3#@==_LrXrybHM$AY+MK$VMu~0=KSYi5s zm1(6^mJ|AfmXWR=%$5!#G7r$YV`}b2?ah6y5q)o@t-EX3(oRi6E$bs_dIal0r_%3Y zdvSXts;z$n1J#6f;!2$veO8PLe`iGj{?2-)Q8Ay%Z&8CvMxz=gjH;ARNeyk0p>8Z2 z`kv+ix+#D%Z0+rDq3=>=qg8`<1>VdXM*4@<rdiaWI?njozCL`6npVQ<-?PQ6p#pS_ zb0P|)K6Fe#gY;~UY#yK~iXcW812{0G5FpY}FB&4XV}<AwdLx~N4y^72&22`YIR!3^ zL`zUMyAn`P3*frVh(5;Z)!c&_)D|%*4MMUW$7y`0As`?Vb+J$;z1RvYW}+-jST4H> z*#IiVra)PRWx~p085+Ti#PsbN09cQ-s39aPFSQPgY~4zI*A;1vU;(89iOR8`2@;{B zAL{Ii^t9Q>7aFxSQM5!g0lfl-M!JSN(W8Svb`e^5Hn+9`L20YDf&ml&IV(m5kh7u) zK~2o0AgIpa-ky-yIy6+O2W$dmnpLby9jRc^A*_xrz<bxgP}fk?nFT58blmu&B{r27 zmx5#f=|_#yV<kg-@wMZW1C^<vd?nJ@69G9(I71C65Q`?Zs;ikvLF-f$N-hbQ)g_;v zX%WV>rj<<vkTZ}1(52L6LS$r6=x7-bMuysiuD(HG{lFGsaHK`(9^53f47U=Q95J1q zvrYvMwQNBX2RfTuFe95v0~OJyV>OOZWXSXNDEchhc(j6pqt1Gw_b9G3NSBax3s%#S zmWaBvX%FIN46}(YO7!V8)R~4hzzv9MpmY#`<H!p%7M?o>n|t-`plQ1Yh32+CvAv|M z#NN_1+ycZ7Y^)9gFk#Q2Wmvf>QI4K|RCI=zvQ2m%8JPH%;L17Stvbawfz0jSG-SXu z9qjLFlQ1zxHlvwcEwr`_b#EEKqSik$IJ98|ivq|2fJ(o<9cZ~HBGQEx@ZqijVQ7Sg zHXJt4=B8_7L}(f5;2XQ8O_8paerz22@P`Ct0lV_;m<}rDrnq2?`T^r>aF0rY)2pz( ztsnG&vi;CHzpUK45u`Y%Ql(8uRbFgUS2iW0sh^?(bSb3^ja7MwE@8Tq(WRU&6^4<% zu7;ADV)S)$31TWJQ$;B~Ql<*ZR6&_4C{qPxs;Cf~g2hUX778Ipuo%?@i-T%uwJ0c9 zj7-5|WC|7|Q?Qsal@!y3-j-0N63SG9YJw%GCRjo_N+?GO<tWkGR#&W+T1WW;lrKQ} z0+cU6`JmZ`3*`$?z5vx!N;Q>I4p?)>g>sZ?&8yc6tS?auu2)h})>5rX_)S#0r9Q0P zsqi3`5u{p!RBMoG4Jt1vYf#HNjVcaN#UUy-M43XADMXn<lqp1+LX-)1TAG4k$`Pg< zuw#r1<p^s{ZBX7(rf?aCq+NMOnab!z8D%P?Ol9<<jH)T4Y9Q&1#H|#fYRV`_Iprv) z90V6Zf{P%*MUdbkNN^D(xCjzl1j{K$1?8xq90U|W00k~Vu|1#+BY*-S<pt#+pa=md za6yg`0Y!*_B2<hzLdB>fL=X`ohzJoxgo-PqjS=8d1PLTUR91*UB19k&B9I6XNQ4L^ zLIe__5~?IXl>{gU0Yiv@Aw<9sB47v+FoXygLIeyU0)`L)Lx_MOM8FUtU<eWV3c<7$ zE|jB`a+FezQpype96`zvq#QK#AsYG+4Lzh4=t4Pylp{zvLX?BXJw)RkqHzz=xQA%m zLp1Io8ut*5dx*w8MB^T!aSzeBhiKek?~E>#BTP9k=(tdha0PlBIdGvI7<7av2Mv0N z20es9$AxmxpoeJCLp10i8uSnidWZ%+M1vlpK@ZWOhiK44H0U83^biethz31GgC3$m z4`I-8p&Wz>LWBuIzy$4qvWPN20_EzA3Q$d98u~B|eOSW>fpT>^1*pC-0YI1lAWSGB zOt2KD@ekAZhiUx7H2z^4|1gbzn8rU$;~%E+57YREY5c=9{$U#bFpYnh#y?EsAExmS z)A)x2>a+~hXf3Q!=X{_hptiiGRJ*GaE>NR2wML!!ftoVyeYtiYFRw;>uGQ{!+Pz-8 zPgC!;TD`Sey|r4swOYNkTD`Sey|r4swOYNkTD`Sey|r4swOYNkTD`Sey|r4s8qy5Z zY4z4=_10<i)@k+DY4z4=_10<i)@k+DY4z4=_10<i)@k+DY4z4=^=h~^pyAd)y;g6% zR&TvluZCd*8iozjYxUM^_10_k)@$|FYxUM^_10_k)@$`n)B1Uu_Px`zdZ%giPSfh0 zrhV@;?R%#Yq*Q2?PSYx#rd2vkt8|)H=`^j<Y17ohCU)u+&XpJ*?D)khk@U>?v$(?k d0m<LsAEnU!bjr71rTb6jPcEkW8zr5O{~!CE|H}XX diff --git a/site/fonts/fontawesome-webfont.woff b/site/fonts/fontawesome-webfont.woff deleted file mode 100644 index 400014a4b06eee3d0c0d54402a47ab2601b2862b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98024 zcmZTubC4&$(_Y)Q?OXfSHg9d)wr$(CZSQ{8wr%e%e)p|<|9eyQq|;BjCzE7qGMTiS zyqFjeFc1(Bu<scV{67g2S`7pS1o^M@|B{%PitIl_rhgT9|G}zPyhB_>RO}xo^G_%I z2O^L=ATW7lM&^H<^*^2eAN0eSJq3(x4DA1L)&F4euaO6sK5joV1E+r+DAqq4sQ>Wu z0|aVj?P2<lBLV`}^8f;o>5hA?l{GgpFa`oP%>HM?@(=7t5y$lA|Hyyb+&}%lcF7Py zVOq>>oZbI%cmJ;c1Ox<gC+dL}Vr^&i&(7os2nZ4b2nZGl<hfAW#=zbF-|c<=pDz(8 z9SEMSfsF|eu+6`IfrtPBfrTEXMOE9|IXMFX+vx)VA%*|};keYrWflDM`ERs<ruhnI z|C3YlyiEVerC;V9?%78A#`^kupoY*8Ncx8V%EsJaWE%av5I}J4K(9dm)tgQkXZQC{ z_V+Ig8BX{2&p{MqL}FuMbz$B42Tm3mLO?=<)WKQ9{RDWkLx{yQktU*mLx=&*O+ncL z{qvtm+?6_TKpHtUvotw*=kF)E02S%+ALsXn@>&!PmnY&6cmq2?4Nt?RBbj#@*S#u% z($dm;AKJG3Yv)w@yrS19dscW!&dp@T$utcaiktwRu?l%Fgn7##v*Q%&IaI$|O!P}5 zE!tXI-Ss#N&%~+2xwep6)=D=@bER^nrNZX=A{Jq3H3E=sm}xcLG|pUA-88}8wRPyv zPnoSTxscjcm{McuVx_s+*=h#*Xv3UB1T}&E{uxPi!CD1QZy{>6F_-GvT;_v+@h3%S z3~p6JKLUMaO+O0%W$iTHs4{|UN^?L;ts#@G+64bnV>gujTO1A$SfkJKhUN{&{#iBu zbrz-NBAI4CWjjIN*&fwVu4RubbB`IvgcJ!WV;{$}bpWy2K1lw(2Xe|eWcN9U#V^J= z0v&sgD$Y5Kh^J4utKJ8w`)YkScnEwZDG=2~oYvdtqau)|6HAhwqW$r>MKydMdi-xf z|IPEi=Mls`ySoS4Uu8Lk>GP(?uENKw#l^+NO;vrl>caNS*3!n4J~PMG6%1?`Lo`8D zP!I`IikK!Gm+D~0Tx5dT2;-4lEPJvvNz@R<kiGAHgN!1hJv!-#1G$r6x8qe+Rh<L= zZ2FVXxySAGdCO~->oxn4bK2&F(-3ukKoTzvdLw9<pc%2CJ99f4k^?qCIZN~gP5g)W zCXprkEjs?tvi~qSo)627XGEc1SF#jUkVi>r!ZsOd)GFakMtPqh`I$P>j#E63N~^t! z8t)N`OP-Ey8cNVPKsgcS6B*&w9LA&4rPERq64J$9K^)cnN)EQxZgj#nJKXDP(A<tW zNAO%^i{5c@MW^MU(q(av?f$|fWL~!<)mXOmPZkm<A6DM#kt5qo7l^K+_Oow65wm}{ zS{f5ktlc4#l`z5vk%Qn_TMX<RVPNfwI5O`bOe@#IKG3aeyFS*Ix7^pC9`!FQ$un+} zXUau`=QZ`6PI*3`&kd_MQs8Bh{kIm5A`@gd==frCHKz$*aQYq(vXPdt<;B;YXY}Vk z!2k$13Q7l%vG;ZcTZS>wtHNPvj4d!y|3WE|h>aXutjp#eR1Va1(D~!1cD@#G$XK@| z8ScdxW>*_WC0A}fCWQ_Gk+039h^tbyU`-AaRQXE3C@|xuc#bIvB-u`7jVA9qExYjR z=L}OyA;5`@PuJUM+d|rr+H3CQORerU?U9!{Bot;XUqe}i%R=!=DIcZf5IBHt${UX7 z$u&nXerDE=@3Wd|0@Hz$q*rpVDJ+Wsi!-OJ!$UKaeXQAz3oz@z3unQS7l<)x)linz zAH493JdOfC<S%@|Hvq$>{BNrjX7CVfZBLDtgiqO>03bm9Y%opN;dZI*d!CgC7s1So z<Lj{I=Syziqvncna0}ky87F#D0WYU597rB$E?Q=Q6*XbeA&#y;{cR~Ml_X8swSvzQ zQKlT0QI)XTQkbxZ-fhyZC~jjs#y0Px*fMdt7H84xHp5c85c~_irovV9scGH{9{%EJ z-LsS5zQdL;M3*lT%fM1ngsSCmYi6%4zrFL|&A|mca#oCIKIV*Vr8d-w1C~con4?p* zr$_56r1PO=HN0SH`prXAax)$EY_02zCKipD3>x$n!T6vhxG4g7BozT_i+(EXciSh1 z*W<L!SJ(XL_SdE<`{t~@UDdAdUz?>Kx5dLayUw$Hadz3+<5D}%BZCKe`cE4yNK&2O zC_2B@YGbYTJ=@>6O14_I7;gA)sBiMP<Rr*bC{!3%h|0P^fuZ#J<E|t_1l^I3=GWfj zGronMV{|yF%HGx(Tw@_;se3`h`M`3Awj`nz5|n3DOFI=Z>W}zMqr`$mljy|@#K)X4 zywlOE7bt(D_<9aY(j=81rYh}wpQBZ2>BFX$_0y{XD7Q1jV-(PFSPU`4DYgBSjuXGW zB&TypZ4-Ia;ZDv{*YiZ4BK%bLvA^d#3^`kw)^(lO=^V#PS}I{JY8vD2<6?gDUgByH zoos%w5n5SA70~&_wmZ}=sE_CH+$5D%I~M^tEkJ<ZQI7BsvH)rso$j0Tno$9{71< z@V}SCAhApjLIvlX0Pxk%zZqkf%M1LSF2n#NI}?5xPC<wONXcqDK?%thB2MI|yK>=! zobSQlu20xcw~DY&-wOel-n@?qJ&by)A02bP=f7VUb$6h9A&zxij{$poi1x&>usk&q z)o~Zd^jeapPeoI1Jmh>Rc-6+ws~2@GiSZz{hBgw^soz#me0J4++L57M=6^+@00R~q za2yth-1NjYw%qz!q2gOQL3>x?qI6L_n5iR9jUE#0ppndAXQSaxXgAAg+?Y2ZVSq`= z9KUjbab4|QH-zBoMtL>BP)ja&OJ4O?2yYF#*>9aH4X@u0(otsJ5@}kXX@!4~Fy4Wh zDN>w`7i{CSlIi9?H2YDBB_h~K`_cJqA-9`a@G}pVc;w6b)PGdJz9MqO5m<F}FzeZC z*X!+f?>S;`wb~72i`W#}dhh!aglheCet+(79kLz+P{)7XRuyhb{YxtDFZ#1N?6e^# zh*vvtce7F3I~yiY){1)rPtn#OV%8zxe}b9$IU5=66PVl01yCBSd^dXUKhK1G0R|IV zcvk_Ac>q2IN6uR13{;c-_cRbEqYJTB_{Fr4IijaD<!Mrd*>P_s&jXx0$`sG}^H^o5 zz-Q`#Xift$p?Wb<=<h|fA;%agwD@EyQB>fxuzXVyNKg#>QnXBe)ocjuyk{hgW=c?V zRs~?RkX9n-Kuh2ogdASyGctZ-79U~PP*d!u<<~CRR3B7LYtxF8T{?!Nye0d%0n1-I zI4RC68nKpBKg^rfqiJ-i4HXbQx4>=dyxjLao>lA4TIu938pOX`7jX~@WPeN@jr_P# z^lTrnNnS5FJgePCzFZ$yZEE2?4_z#R){UKOsw3qqM;Tb8H@A2_3MP!1!fsit%Vn(B za_2OfhiiPV49y_-YDhUHAURUHq=tlP%rx5l^&mD@G^8z-Y=Z-tIt3L`u!>WVQxz;^ z&9LZUjm7~;VIecrymMSz9sAiMQWB|u=tF>$?NZ<_+~80;Rt&KJZ1cdqEdhb%EWus! zdJaxE0R*U{g1~6{#~l&e3R1mY+6nb{2=-5{7mcd@paR4GV(zxv{CelE`s$Ei#`XXd z)c6s?t)+nM8@GOItmYqze$tkR-@pNBhUdU3!dN9ILMYJOj4^aUvZMFQFK=P@cL1r6 z@U=sJ<=N(Bq`QQC3-wJHuee;+1OIT=^WJf^vichJbLK-(8A>DTum-ya`_|C7PvY^V z-X#zAoguBv{!+QTW6rx3-!1S_UiFDt_}ti$D*F?fI@AHKaETKn;7R7C5HXlh^h{!o zsrxdvVOX}7A?4Tr{6o+@q_3pMQZTg)Ea1)Q8|O#l$}N5<%GqV~ZE>N)M!~x7JUKA5 z9t(l39F)9Tiu!T`O`2ZQdW$v?+Qe4m558`xNHnv~bX8j4G6ay*PnvTLCWgm@K+IP1 z^SI~_P^NN)(Qy;<k<zK*cR2v-gyV~0AQiU6wxyy&Q;~4VPY2c0%y=4xW{-2|PdgcS zF^4oiRfuFIl`54OCbx8sWdU1~MGK^EAw@<m&+%NtLJHaP?ChvFc0Prg|0QDUn}NwS z!<dmX9)(xhmx1?IbRx8JB_Ic-_B+^~(lH%&-IS2K5SC}&5UN3FTv5U>gv`8wrCM0r zdu^7~mAS%W$G8dDhB^z`1T=lN-^sNz%Wcwkz4|)K)IQg@u1iEb91XhJ5xEwYDfvM6 zkLOfT>Goml>)dkK7RrcGd}4t$1w4`Vi@x?8r-Xz-T@erhoTTvYj;62sm##V72KMKy z7jCvo37#eEob8=(e^%k-w*#CwiWcoBL~yaY-mZ;3#7$hwrE0n&Z&_iqW9;qZ8h>;~ zOjAz(rmb4$^7bp}HHOIkg&1oXJz&O9f5ETRc`KDiwH!c>87$<tuS#eJC?p^dhgYAu zVt3(EafN~hd+X1VAY`n%z{Y)(fLV5{rB`+<wj~`@By^!un~RGmEPOd+W=!KsUT#|j zBKNXj>jXR}9R=#e{N-{typMNosUZX^8aPu^3Zb=_A_|$kJ2>CKI25a~u?@$|xUD0E z3rV0H2Dkhmtcz}Bqr1R;PGC&s1*q_(cw=w!eh^JIxmYy6ip|~R@0t~6h9kSKF8k`r z-rmZ)soKb2jgHIODnmo-1=6%KLu=Va>yJSJgYnC@P2eB{+<2U~g=4b-hjNb|x!65z z5!Z3c@32#?=kl#m5f8>l8a@f=Wi6&X>j+N1+ruaQG?CtDV~PXb>@WWf2Q($z>z7U+ zMBlz(Z=2s-T8$d;Ue6M3l3xRuVhSxm5<B)h0`2`38hI#DD>s{3BKIpgmi-?-oisza zkmgcLp`Vnlx?L~qe?(H=WYV)H)PPR{pA7{5h`m_l^X{d`q$MOR49YduCf{c>9PI^G zU)!twAe$_^TtGrD{jAw%Wfw1k)5`DgJXWP`-<M&JiUEl?X^Z}aEST#t+_N(Efbm3| zfQ)w<x{im3hyL@G^^sy#+bz2isgU@vv!l_bXVy1Jike{>7XNQ20MryLW6t0#t42k2 z0hnOio5PA`bpihQ)A=v&;|;YU&l?F@fC_Npa}OspB^Vr!zTb{NLwi)Hy`}19z@fr? zU3Jh7xd)*wL=El;v+()ck_u(iI_w^muPd_R6?OAcCyxtX2(<rJlA42`$|$TR(<12v zl?Y=e)gf9DYybQZ4qK@b25;3f3J3V%pD+1&^X-c5Giw!-UTKF8n0O2YT{F}e1ff;g zjuv>vAWE-tjbs3u$PJ&jfGp*j;7`8P+@e0HF88@NU#6<M%>t?jH*EMz0L$My9PHiB zRVebeoyHC8Wl&pm$IT(G**{Utw9Bh)HAE_^TCH*ta-8|<-fxJ&aV4hWUSV75)+$)r zdIu%X^B9`Hh`wv*IW6Ho^#zL)v08Di99QNKyQ4Ex^x@3G;Cg6K(hX}D-{D_(j!D<f zrg1OImb>%6g}xd;qA)E>mv@<*$ZX$rUpcaK+~5kxF2pAac<y?O5anPPc95_H`9WED zWYcOrkRsxK*XDZK&y<t`G6Il`S?diP>=%N>3B`6+-EO>fzLHkzfcD>r`}fy+!N&}- zUH9`HP&unio@p<rTM?to8R%4f?^t#A%^89fSe<HHA*j2--tcIx4&F)}Hbj_5T^cj$ zFF|M9UGld?S>V+24r=ON7xE68a7?3>8!kAzHyK4Lb=YbvQ+HBn+||W{Eg?GVcYQ!l ztSPK!t!;<OEKu<(Mig8rL7&)3cSc8nLm>Un>i4P0$ET?I9pdIh^EU0+RcYthPqRm& zPB}LVBWJC5;`qzHr{VN*QZ9;5?qvVIY@^viP)2>OQxb+mdkWDzLq#%PR5z67y??M+ zSjDiw%<bN@u~pn?LjN*Av^%NtrcHn!Nw6uYgXzv?BGZ`2z)Fxe$ldvcV_=rTg_zy$ zVt>%q&n3QENt>Lwj~Ps8*c{0xvFm@csrU=eyiH}Cpb=6h0&O92O%dTc0WV%R`6~bS z;QT3eZTz7V7f#K|S{Kj{_}e_u;Joz^)V0uvH!H@e3WnVKG*Y;R5RQx=UKb=?4!qeb z=_DKa-vz<<!-#l3;vWuQzB@*wKRW8KV4p5-yPE2&L*Kb)=K+0M>$?}ZxrbHii^hC> zLN`k`gS9^kaeye-(%)p=Q!i(kFa)B=q#!VbG7-calS3zKZMl8Kg`I^HD#h_iN?($! z>66rNVaPiYq<@#JX$rYXkw1$h7(yVDzNky$V^i%H!;0ZYI+ZXhW#@zfK7#lXMnh2Y z^3kcr0*7W=&Ss!urb<Hj1^*_}&FMjA<THPQl9#Y$*+cYu*aovR?$_?Mf((Pm8WuQX z(w)kntkfo+k6L5XK5?x8D;A0y>d>4di6HWv0K><1f+uu%DQIF7AJcpusQzmE==J_e z-fwZbee~KU31mUe(k?U$jD<>ni>OKvN0|-t=m-(#j;6O&G~<{8=r6^gv3$D&K-xY8 z-A~Ae;#6^CAZ`&J{>W;EQAqsZ`r@~1+yiz(zXcIDK*GBO!0caA&f@eEcUcd0SLAp% ziK^4%<Sav&1Q~{(M?V|7JN5KxFol8bvWsFaTxRp<P@_bvbtFhUWu3z9*86lWtML^9 z#C;DDG_0n~qcRzJTDmsFLJoSEBE47{fQ*99v9-<hflt}`b`N31iUd0<*@^)wW}#ZF zeN&`IPbaM(%6@frESDf>9xfj7AK-j%&m}#)l$Krz(B|KAu~u{JsH3mYsRF-@7#pkE z;OJGjbEEV%#{Qt8>G*G(Vfh9<)rQPk1eaSAEZC<JH+7rv@BMOTV&D6H6=!m4Cy2N0 zv|$(o9`iC?32N#QZ^#lPAcP3m4FpZra9I0|#4}_=Jqc{i>J)F~PoR(h+g}tl-VX($ zYO0R@KF7}dH^^v=pHnQ9YSNiTJWm+f!v@BwqQ$Y$ei`a_1{_|I<o%WKWtavx$ZQwI zOg7hf?IM>-ss`3Ry;b`bNIE$R<S=!Sz_quS7=Ad4IYK%YO{6~+p_215=-{lh|Jzts zhzbfqTR7AoS$atY2J#p*VUq9n)4<=kbhJ~Ue*SztHP9nZ{0}cB&PUIC0T7o9IAy9k zX6TA>nb+z+c*ky}aexvI*zKtJjccvTTZIqk!Rw!$+NgN&BT7q-IM^YM>9lAFF3qsj z{Ui)Y_-SRrj^=N_HhESJD-ltQtL~Y=Od(%jfPRpq8P9`F;O6pc)s_oF{z{=|n6er5 z!u-{h;{bvm_L%5agg+m)4aA0YAb@K`Qv~Y<N@s{Q`gwf<!<S<>LWx~sGmt6*V!|?F z%7PdL2(eqp+Sq<ETerEWz|-@#q-oRaFR~bSgTqB(jupzay7hGo<6ObuZG(&7ALd99 z$H#pYC!xL7{N3A^>bvQ;>6xmHK-4tnG6El;(blqDJ+}Q2=*wlRYGBr%&K>9+K^{Aa z9GQ#O*$%Ki>UYmph71RnuwA?#!9vfTIuG|p%N;AWWwB5C+IE2*>xGPGkT?t@?Dvhd zt%Wpg_71*1_@0kBba@@FZN^TvjpVY+rkq1<XZfI(GvRp^mRM6N3mBbJH&~P>h2gtm zJPXCjvMjf7K+`s#pH$0kv}>*SPOV2H-e;NChSuuNAtqhRtEe-DVqBG7vr*enVEmVd zAv-&^RqMyAthD#nN)(w!Yp^GI_VB1e$~skiRlP3K6DJObNVTJM{r0E+{x$grTNFbh z_uB<D_Q7y0vO6ERP3?(oz<N|y&4vw@&eU8j!8JuRvp7--FcrN(nj!~;y9)~vI=R4v zeRIZ~?_AH?p>sc88W7$jtTI-pPGD>}Uj((F_<lBWCkEX5^#QJ6*rS>m&nMmhI4lhx z;SZUOC;SP$w;q=0ux8Ozq190iFGeAoD%-HBSfOO9W&PK~Tem;KeV~3gA0dW>Pv6I1 zYNn)N-+Qq-I+AJB!=V9uxeoR-tL7t;-ZGy%%>9l;tMtQJm7z}(vh)}z8v;!QqkT%c z`Pr;kXU{<7gZGe(<&Zjp1|1&SGt0&iI1JiBIdPElDo}oD(oS=FPy1_j?dy9UkEB(@ z9bfbpt~myqXy`*o?NPpA2S*3Iq3$t0QzT^=d^GlO7pmjpsXe^IwU{J-P?mtkdD4jT zbfg}pfa66t&>R@<Z8=Xb5anVZ0@*VilInharGReOmWG@F2H~xj{4z&<dWfz$Rj<py zm>5s6DBCTElqWD~=VAB5A$Y$g3nSX4Ol}s9ozugn47sFrns|d)D7D8mh1^h>F8%3W z2a5TI<EH=tQ)DCEKj~cA)7>9W)%Rgrt<QP4pCAdA&I{QZu#15e8|E9mu`%xzEc4J7 z2bN9=Pr>E1+L(i!DwwV@xZ@VytBSnvu3ay?9Y$%KBd@=bFp#4X>B};lBl^>;B5%<i zBeZ|TmK^P85F)d8Jb(ax#IIL|Q6+2FY)??77PzOwe*DZ||6qr}zKgeYIX2Udef04z z{K#*38@!fTKYF;k7}>>LW8TF<a-2r>DeNLsW?@@;#fCxMm!*pX9lfHt)uuajgiV$d zT#h**{Ipyhjltvp#_fvwZ6(9T&)Rb;VTsa~=gJDe$;q~EJzFO3Apn2EXrlA~F<KE$ zU^`~@dRT9`NnkY&k5rnw^8IMHpQYj9vL%e__KV3ACO7`>^1;i;H_jG>WmV*SvFHky zf3twjY=>%B`6@dr95pk37;>@x#zI%UP>yJ?6%2RCAY-s(SLIof9c#sG+>FEDjD6gU zD+r3UOyZKt5Q%XW6oZUQHH@|K!@vgu>y(j~#NpH5x9l+GPE6*P91EzHBE}krNo7~5 zb|0;8aj<>dJDCakJW=LK#vk^V^`8D9UP$2lLk&K$X+Ag;(w#ZeR7?dFGzJkJMi;Oc zoicM8#T@0|)<<i-CzAhA*$y=Z&Z0PxVqZX}e-5SKoUi5o6#BA`Ohg#o7yyhaf~(fs z#T;g5k6S!wNA7;0NgV?b_ITlcXM0r6ZO@u5M|~?sv>b|u?YyW0!6Ew$>Y~pX2XU`J z<A?^&Uq$5?i*YZnoAh-Ted7EF>DYoQ`d*fm7~YwxoZtL1W7$X*5n>+fi8oUqvJri& z6nm&FFcO9AAX=7k9_;yussklMDtxu6t5OkjY3tvL7s1PUqGstoYssPT_ItLMXX))Z zJ03DK>_IPJgIKX7x8Rw<+?!kIc9MEA5hw)}5-iqzE8VFOr%mr5VC50inCtJ#tAQL} z1%tXg16rH5cZ?pPJcaYO6~hh*gGh%x5*s)RLDozXG<$(Q=kn_7fh78e%R|8C^X%4F zm9*vMr4{4*^7ibRo5iK-C*+ed7*^J_i&Im+>V~x=%ybD)(9wLptciZLN_)YB5O^v@ z{$Ja{Qtd!!GiH0^v6Ue$NG8nsD)~)N*JjWChU+1?Ny%198}eb+iG#cLFl;OopkF>K zIJg1<gZ|vZb($pzH4ltHj`;8bzidD`^%%<T4%M5LS_!KXtzD_a(AWgymwVbW3(#<V zb?|8f6xaG-z}4OhlEIJf&0ddWw;~%FTaylOxMkzYCUX@vrw>zG{!THV!AKNdnO5aW zt-47+g@#B%3Z{i<S43@LH(h0Inj&$ed7j?=wQjuhVH;EX$@l#HxJHc-AKQe3cs&}8 zKZ0?Y?sDCYx`Z^Cx|^s^I*iw(dRAsvhwb(3U7F%B30k*WJfGLHM0klB{ddN50ZTy2 zu60p|v$n`?G0`75qP$UL+d3eSXk2mIrC+O+ew6RiZBe4b>t%Q@M`87PUsQr8-l>(V z7?crSbh@OEA$m#}=67-ZTp889W3?AU=1tjMdw;Ne(Izfm0-RQ+6jH&8gwGA_(Q}sf z2cqudmvKpmxhIPXLGEOm41F$3^s>mhI5{xLs3uHjw&8hlNfyhYWJ>LMMzm7Au8{{4 z-78CWHW(hd0`W;PqChl|g^3)t!&RZbm@=i00BhlV_)wg0=hMU42F)9g3L@3ao5I}H z8I}fZ8eb0a?<61oj=9=X+T!Eq!RN*aH=0Y9i8s}rg8IT>C(zNJ!Th>8L<=0PZ>~y% zhz0Bh?ag(U19g*K4YsztBIx+FBiiPs)+@S)uF6ph=|=6xgUL*jcixtPvskp*56`B0 z={4aNiYE!<Z9^;P?FtUwfqZfE=e;SI7Sl5M@xR%o_93*!yiuIf8V_%Sf0hYrsorOs zfkK^9_=$Oho>i0tq@Z1;pR-k?I3o>lQ~?sYinu)T9ag!9h~z6;ikT8&2oT|A@)-z( zaQOIKXY~=W6~KLycubCWOz(G95I!BBDB0Pny<_|zlgVmqx-mrqM_VmHhiBtJ`$Z5w zCPrd45%V_Ko8gYvDbKOB4l<(Fy#)}+&?NnmY-1A}rTwO$s?$(4W6U5%XfMI)w58zk zbnp#zcaX9eQujFlW$d|exgN>CX+D9ODCFX{GoRcYei!0W`_<p^r<%M|<v<e(yc702 zsGRtzx%b`iJ6Q_e<y$l}s|#ePV@&Z%BjsRpVnr@IOe(<8;=%MoTEI_({<MIx?o+Lm zy@uf}-#~+3XTroy5iC8qqX>4DPA4@ELI0BSq?GTP9{qy5{Jp>{!$ilU=1r*;&BcRg z$*q-IA(UIbR;y$MuoVtrm}_sru-Iv6QF-Z$*v_HQLPEzhFGyrl8>MSf`fNpzygHW~ z_QJA574ufXwN23TR!mhNU*^BKQw@5<dJs*_=x{mDYt5qy%uW6HuIrYQdUw=BHHG z5Nt@%wEdaq4{)mv_E2B_!pNn?M`+Gf3%JA^GCHQY{6Z+#==o?<Xc<Bbo+i#H&0;Bq zPkGS2-h%5-AG?Sl&q79mB*ASe)Q8HRNCZz&>VMBVKN&I-5tw2=+-ea|`(iVDzDkf` z_o4ZdXMG*j@}fOMk`);6@zP0?jJx<T&}h%~U3MIX@wIi?F2pMP=$Rx~vVOhG&L&fI z%&@+nxx`#%MaLBV6tSRtUBbUEE)aL$F(n`|>g<s`On6!kmkg!}$!cL@ukMVLxBMmu zb!1M*xy0+mHQp06f;84P+Uf>|pqYLnuYp;NEjq=E37d$523+{9c|=_m;Y=FC2zr0q z9ABp`#xa?^D8x?{^m9Pb8P5(LYi&GbahTA*2ISmx(8c(0gM7mGV0*-m^P2+5>2y*D zK>!ty(}TsN$-pvPyv8MaFTTJ&O7I6s@>;4;BIl36G56wWqHwlP{~pWLHf$Uy#0Puy zeV;G?gvis^Jxj`$>M5o?zm}_}UVzVP!9jt89Pwn(1x#nRAN`d2;9sJ`tk0AOz$1+E zH{8RxgaNe%M&|1hrS+*9C*P^Q=fDJ&p_?m6QWaQ!V5kK*vuF%HaecM^I*<GHD<U0! z?Vw-JqXtn?NIzh8dAH)H@;erWZy<Fp)KYh>D{f1%Ubp+IA5m}APs2n1ZJ<ms?xXTU zi^4!7YC{p;W!{O&ivrx{EGU8I0-<4j`OT6#reRd25lOhZlnhn_O);6DCRAY!BVw=X z+(#m>u)J^J{Rl04s^nuyFN`DfFR|@!RJFA-DyQV<_<Z>xaV4SNKY62@hT@DgkLAq~ zhG+%xacHfgNfA`ZaU>zuj+4n`fU3TLj}&960XK1bcKm{wvmh9SVn*;5QgF*KxDXp> z;Zr51Q6HgH%jqJevB^Jiu6LMSlE`WNR1ubZUzzA5+#sU+UBVg8!D?yT@>=FvY+EEQ zC!*yn>I=^d@TLt~CR<gUtqRoFCNQ~7N=pIB-^L8<acJDWKFHZ&?KRH4)Ny5uD2jgY zM%%mcr$Pa};^OuI-jG)kdhz25dJsHAKw+&1ORV8p6=`~X+8JAll7g@mftT@A=1n}& zS`B-Wabu!RMcMspnE&J;9&n{hi81DZjLUgcD}3#Y463ka)ne`IfTV`gQwz_WU}S^< zjqQ@hMq$hFA;ADZdMa+Bq9q??N3;zy=KMuH=wtb>iEKJXWgp@5P+?!Jd%4yZjSDVZ z`OkMD7`^B2*g{%}qlKpgf7Zmo0$lvg7&BQ)Aza@3G~b|J$Ysk*P8I&CB}bAMZW-~Z zIR_wi6Up0t%hZXSOGa=}k*;=(xjt200^6TTRMf=`GX0xknXv$dY&r<S$cBtbmr7c% z8dg|Gq^Mx@F2bUtEF+yZk1<t631!(_Ug~#Ax?D6mLho7lP8N}6B@gO8+*Lr+ENSv4 zuB5UJlOjqmA*=e|#@?#T<v6V3Dnd3r@b3##W1upO5V#EzCbXiZxH&?iH=J-^5ucn2 zB}#&TJ1e5^d2#7qR0GXTdW>T#xsb_X8RNyA_$By$)d>6vNs2f?oR!rfdl)uT3^wm? zQwUBwSI&b&0r(I>$MjJH`fi%N1_>bz?&Ie_?js~TGj-`X%$+E9%n{r<<}`S$e`-p) z=*`trS)6S1Q%@D>CURjquWCtl()2l|<=i+Y;!j1i7jdhWpckp=OwWUJ0MIi}l3TJ6 z%ie2wuVKrrw_6uhff+-6)=_Nlw(qWRJwWbgGK?~1p|U<-iQ8R_>vJhnE;jiLPcBi1 zRW@hF{B?5XRh6|AR&h%$^yWc*ouol%@U#QTr4H?XOSYZzd|Vm2@o@5F7Ops_jl7Q) z_!ybL>GEq;&gio9wM`Qi-TlKa5EY2IY0@jteHNx%WR6`sJuJP1f$&aYFSPnLp{u4Y zEC0QDql)X^>kq8ecE4t_gb{C=2=3N2Gdry^aVqO$<8QdOeXI3e?r5`^^}Z(42qSR{ z0UzZY8>scj<Q3Y!S8{2XYT0fr-jCd5x}ByHUoWLG^n=;fe(X!*S|UwonOU}J?-rpV z7aw{2Eqqt?C_b!);TwD(s$Q+B?s#*;2|+GN{>$7ip(7LQ+vQ=uIKkHj_~tcpcgSP5 zl5+MbW(cv;e_PPRsa@@MkrcgqMx5Z%N!L9-bn~Ur<+53s7!rjk3?<gI`54Gj8$Vx< z#FH=xj^&gwp9tEbR7;b?boGuRVv=(J@npfqC<2Pzxwv70W2GpI_pgzi!gkLW!0I}8 zxIVF;fVd+FsJhSGb{^3Eru&>KlB}I?)Qdv;%ICl2PJN$ftp)ow;+k%4wA>Ck$|vtQ zY_;32dscrw)Oop1ekSSV`gS{<%RUw@3VxU0lDzU1SQNO$YkfWP$ke$i6f&=S)<#|) zlsaMpADLw$TU8oa^N=>@h~Cf?=Nn=+j|^}w(v<M}Vw)2monZgX2utG5RxHF8Xm(ia zHnGZQoZass?GbP*e7>lxqQu54&1r>x{W^6ldqjSsVb<$rwy}rmwYQ01Baz>U?dDE) z6Enk8YWv#EPCC25t@EorUGU5O{POaAz%~D^imu19F!K|CcOQ6u9A(3jzt&6Lx23hJ z_sY<?&e)V;92O04dAWT%JTH#=I$b`?@Ht59g-6)^nR3Zt*5_Tp?#8~B$8UUxR*b}! z`px4dPclxQQo@VebG1i%V`-qyjJwO68P%1TVP6p5Q}m6dzH1N{3}tqkJ?{a=PO&aT zXpZ<%AjynF9+mkkedZ`cxg$3>^Wy`DrdJCS0duxEW>Bp16>_r;eS+N9O(hQNv<q6X z!LJf&xRsc1VT%AbwVR2Fc+<~Vq5hN%Rld6L<JOXxJRR~0!+M#JkloN2s4cuNCVWy0 ziv5I+{c~L3PZvw#<Zm3B=|n0pGBOK}8tiVX7R1vUr?+-vhHTooBFw)Ck9F(OYz=qI z`rp#>jVv4ZBkPTG<kV;HThDl>)KZS(quq)nebe34H)H7M%ti+!MZpA9N4oWcss21+ zAQwnD0vc>}2(d1Q#3z7x%6;?j6E#S26$>I+F1&^X5Yhyy)jZx2)-|Upucn@=gqJ|1 znjL{ulPOb0eXL1wk8Ah>PJa-YixeC}tZx!&A(kWBz|&k)2zfAfgt^NQ;Olk0Vk3P% zSYd$?<92$LGI`4r+F>*)w>2H8@J!QRnSiB-i2PD1f4t*yB0TW=VEPmk1ex?YExNMN zI9GtnDg}xUYG}IWCAHvEm4{~@{-51el6Asc*;aKov?K-kv&2q9S;tVToYnO+c-B=` znQKkgiC7CwY$Fiqj<-%#M!D%}%W?y{P=lzvRFF$pViFDB=NX-O>E6kM3WCB9`o^B* z{MM$j4lm`~NPO5-ia@%@awPiq@h@2GFf=ysU@*00s(yk}5oIaOg0TGff)nIUWYyxN zcEn}cZ}y^F)#s&R>KDsgsBwSUKb9_R?p87K-R`$x<v&(_!f{Ea&rKI~{B?g%D=_d~ zd64W5YlZ_|sO7aDtJg*(c%A?L)%)}JeV)}X(Bh#p8;+JH$GHAyDbbySI?o#O=)_N+ zuMnUV=PkduWqz)`w6)e<-X!{{1S&#fM5ILcdot{?<F=}<`;Fw{t9R5;=7rWa*G*X{ z{2{CkPf|0ZW}HxWpkuhm+yB9lRH?)yVF7<xa1@6>3itD)iTviK$x&+bcHFT*Q!eFg zNcceU!8YQz_sVsSd;ERa>;c4~o)C6(H5wX?RrI-;Mgfj(au5r*P)ju{uKG+ds!M@l zW?klvU;Oq*8pDCohHSQ24f7DeFk&%(PZcU>rFa>O6fcD4U}U3XS#+b?NZOc2maoDf zS5>B4E6*}7JnfMM)^Z2!u|FFCSETDqB*+}eo{nd-W7`sNQ!;2e+6~Ni)KbM22iZWB z%yRrZnm~6U0RBToY0kZLy)+s{VKacat74^qa)$4)&Ph1*?@Ov-g?MMEm?8Zb;eqt! zLvhaQgRdzKuk?`*j<s|~EvD~@G-lqWsxjRi`d;sWTGI4d_%EqP%A?pgKrW~zC9?5v zG`vd^SmM7(5XDoa;styK#oZZ?moG*5LzknMYVI7J7PDCzFnQi~OGIosrr#5bX8Q8) za@v{@$K%>XV%Juuj*{CsQsj!V&}8J|X^iw$%6jIW)vwOI{HkFX{!z0lWlKgw@5_{( zOMVy%4F^D<rpqBqH4z!T8=XiSh;~c-?&owaRpXoQk)sN_2w{&TNj3XeNHIN=_#N#d z9)FCDYAtAMHQx0o&pm6}LykKhN~+j>sc0R@>XubIc?i6ec|UaBw?M>gea5yPFzj5S zT>m(ee^IdLw=-~?{o7xKpf^)qkrM(2p!((az6XGrED0(FM33D<0}i-zg79zA=DNXS zEsb+Zs~m#O<|j?o&r=|HRfL83{B0M~P{4zigdGU_Y0sk`&i#!eN@q9FI$Eh0D@$c= zHCwJI_FH!WbsFo5orbP4n^#UY>8;Ped9MS08=u=>R+PXtTkh6>nUbtX-mk~TlT<&} zv`4nQ78`LiHas=DuR9r3LjJaDID5~MGzV7ac6>D$N#lJ)K*b$#vtKZ<$~-Garg^@I zP>8fe%19Y_zr@ojHZ~{hg_(b+=~elZnQQ=ZFK<0h^nP0I2;dD#pcOcEKg%FDH|FA= zgCO~T$_6o8I$2SShA9w6s>(w(SXOn4pJ?h|oFzAC(qSCg$%!_$fG;Qnflw=yLUdWW zA)3k1AMBe)===<r#fQSmedPJoS+6)g*_XqEY!Uo8wAwYMpG)K%Dtdd9q2xbJ`ixu{ z`KF9)xG7RuKd<pBg}3b%r+KISzC(RFwk%Qw^X1u3n}&m;<#IHV{R$VS$89#ixd)~j zO<7{4a!EjClZK_a2Su&oGZMK?=;a%;AUUSsV_?}acaM0iu0G6tP_f;oe%JjvVbp#Z zyf@SJ10R9R^YG#5^!8_6g<M&ifp8c|jG|1!gy68#om(TaW9U@{^ilwYff~oM0Ui&_ zT>HMKi6Z+RK3K-|6!Nf$WbMb-SFwgWqST%&t-)@hRVSed2jSKYbX^_BIu^IWwbNF9 zpJnu1Rn|Wqa>o_q$=jWj4UQukG7HKuhoijLbIp1FaSe$CRlFxs!%%g2>DL85wjvj( zy86kPCL7BS#|tDau=B}#QE|ffG7?kw$s+S;oe~>*PDr08^U!7HjxX!ohnTQt-D1S< zv>{kD2r9{5>ItH#v8$A+WSK86m8%+ql61HsP9hz+9q#mvT0C!ly1bL)-)G``ieJy& zd%tNl6e$!ua=U}>dM}XA>NTG{gA*PE_J3EIFWC8k4~p(C2wkZV>yfP7W~hmm#ntLo z8zO~R9Z9@lS@sMv$@L065Op;&QPR1FUw{cSF>(@B%9&rewXJ#8_cAc=o6*#1DT$<H zQSV}Mt;v;izf;q)K~g*{K<3122P-sX-@|4FC>xOzeycmC9E)Kw;29{@u_qV|P2(ZS zxS}xa+vYYvo$*1@$w1$QXeJ2ZsA|VX769oq82C&5=~|MRo4VlmF*%RSB7`4{P#pDd zHVO!rfZDXw4$Zpt!Il+oD?D$1+{uEk#nJjBK(eeJY%HhD`*}7)n_Btv{`Im!O4a(D z%EQ}+PvTbP=WADI;~|5XOqn2(kOqamX)kKHqw#y&_tnem731aRZGz5@?m$TdETNl9 zYS>UXk-v4THB7I;csa~%`a0{~6#Le+(<x`G6xoU;d?%J-@yWMW26_M)fv<#&;;9OE zAVAo|xSk}(+N3^jE%=-$=BmtDQE4*bB3l522B#*Yc>mw=byX1PI&dDx!XDsGYB|_m zcnJe4os^9}S8d;{%WfLBg;;#j0-p7l;vBtSuFqcnEiu4ur+K*sVg3u1YtU+w(t}S* znYH047Q2SAnx}fb`rn$h^+M=ct#RG8&mx;^A;cRG6M`R-O{L-D%KMi~ug2yjTfo~> zH4VQ8Mvs>gE0<^aSeNJZh7>i+(1$u(`q{(nwWQK^YY{7>(QcDGjqqfWJw2Vyf}@0< z*0q@`%Zi=ABF2bB1I%U^tnxIB&zV$RNhKpCH@w6qHX=p|SL^r?GC$PTAhC+K`1sxu z=1&f_c)8l2Cc3u2W<phEB@~7TVAk`Ycfm9vv+cMfx`c34IN`HX_`0t`R#eVoi6YJX zrcaYK_*O#~kk)e`H#HsVwL9HEQRPWgACldr#ubhQs^dGVY<#+9BBq;`KA7~GG1uHx zWEhUI_EhNMu#3F1c^w07p0Z<zo2R37+}`ix95Pw_<S5!r?iIBBZf)wgZEL~M@;B%w zwJu$y0C-3<Oou?mM7p90V0G@PPR5DYmK|ESoR6LL6A#)|ie<kX89VAM=%S@_ffX$% zyet7!lSuaytgH#cT8oSRN*A6Lzyo(D{*H!~TeTJ&Q_FdlJ9qAj<4w*WXspwkGKJnB z_s1(!EyEj?zRl}AZVm&hT%X6m(>@J%(6;VRUbf0Btl2F`Y)VYf`m|vxeoTi>`gW96 zdvwr9$IR>Y)MUHq$%$rM=IkMf`b<@d5=nY#^q%C`fbwITF7v&Kd~K}4z;F$*^rQ0@ z4Sj#ac5hQzCLMN`*^3>aRyVd2a?)5z3k(T7strykphhh$nsZ>Qc7_&FaAzY51H=Kq zn4HbEn!l9dl5~X1xNQFng5l~P)~B!E-}j`fMweF^Ns421yno{$UANe9e-h$<ORWH) zwdN<JmYgWRH?k%y784`R7sK*|U->_dT3dQTzRcqepkzHk^z<TZn0GjuG7O8EpCyDT zui*0X;fXLTVd$i=z?0?HqK5HH-mdCl;ekUT{jkntTpU0)Wy>|s)HyzqDH#~EbY*nE z!3acTnuFHKm4Be2=5dmGaC(Z~Y(EH2Sh?kod(}((&UA6`XTR-YOn2Lq=K8Ed9J;;w zkQ210aTLZ=kK-~tSZUlpgbb=&zrtSoh^z`D-34aSz#KFN6OkBL#w9Qm3&c|6wm}xW zpST@|N0Y+_&$;v!^lp@ufMv?cYmi{r4I{lR1#NwKkwjJrH|5aRv8PE^P+iKQnnsxV zp9t{@(G&~gYy7pdSBcci0$eh7${KG?ZP|P5B!Hh!V~Ydjpyepjlz9e_y56W~f?UN1 zT}>?Ii^u;+sVa<|K{^5K$KG$V_fNK*c-!7`SKC-ilQU~8d^Yh?4bl^Be3ZK^lT{8= zS8p}8Foc24u}xec3~k<W%lw#`Zr|e|d>@==9w{AJZg;u$Bsi94Ws6U%vuicdGkP86 zxPP_v64Oubdj3pnSIZt6EKDi*gaANFtS^9aDeN6?*l&Po^l(+nHNdVjB*mkA<#9R( zcBb{DRXMY<aT3K;yatb*{ZRo_u%&Gna|W&?!a}D0kpMlhi8&s{^i$g_hsCBbAu?H2 zTpGvd5Hq^_L|y+paUeVqlwk}y$#4m74l$7>=mRP1rN=ufcI?i2TqDX}okf?on<4}r zl;fjdikvb6STV!q@K~{=8VjL*l6Q)k40Kr!tD_9n-j}cIQH4J3L)rJNMja`rb^JJA zOox=e;F?5I3T<R;(!s0eP`@Kie=#D_kwQyzOJ-Bwiv$wdHE9%#a1+RyRT5L?DgX45 zHDn~ph7<YZ)D1;~70De`ZTt3-wYHB`E*-O+w7Jb9SJ|+4={T-NI(2TI%Il`WY`%3` zTf?)jOQ2xi=cITkO}Gl>&fsrC0_^(Yus3APsM;-FFE!Cx%+-tsa;5@zPj%AVh-)t$ zF+X@&4pt>X7%PsBv14&KggqdqHG1W^!jSt~HJUay?gXlvWsLkQPE0grR#Im*_Tl>X z$Zi}x0nE$Bk%)~}`lYFe!RX7JuD=ox%p`whlQ6|bqgsXfHaF81jT$YIL9{f(HSak? zpn0T?m@}WjLFh8hI=OyV6rERA*m#w}U1h2qzjXGbsml6#Jw&N*zdT-dd=15Ie+EtT z*#y<ETWzcEuXmI@5`4V?0KZ$Oz5pRkdtj^vS)hH~`!^SwKO%W%B|HZLA8K){Ik~KV zcYh9B*YtzT)`M}}bleXZ;~#I#mhEx@GUXK=uB~nzl&;158+mze8_BV?9yOsY(!8|1 zrgi-NW4YzYA^DxYa$;E^e1Le4SO&^o^X~BV)DoXT=S^zYS!HR~MUAD1iqOz_CmbCs z^}0&AmnHhSx{NOAFMVID@Fg64r9yh$mg)@>E+H{;eR8(c31v!LGR%vg8(nR?iWQ!X zgB&?&SyDYVk5FD=GAgy6YMPzYc)U?f6w91AysneldB*ZfNwqr7o)r^k6yycj+5=oG zIsm{uOIXjQV$7>=Gfq1Zc(Qc~$x7f?D4x<e9PMWOF+*|sC}Q+kSoL=sGJ4|m7+9_I z&0PCH0@U@^igkK;CZ90Rq<g%#t`S9t0Ro@f{m2>DB3DhOeHps*Sz*-D^I+uTCI|L@ z!^~0YFTBJ!r7pCmhdi8L0w%yf7id5|2Cex45Bt0=AS`Qc>_st%GM2eiFurXA8)&vn z(v1_c41I0zS)vsNN<jmX;j8M`zp%8FXBNQ*=(y}DGe0#Nmgg1FTV?sr9OKqkQnBH4 zE%VZl%77oF3$e{^y0Y748;xzDI5gmG`-tSMbJ6y;YlD((_A4R8tKKx8TjZ|drvsU4 zxI`(`Z=}N~4I`xYyz_?JBkN!4!S8WOq;4Bp)G-~s*0$TSUeT_t6u*r$wV9ul{zblK zUC8(4f1hp^l(y-$f0*mc?Jl*y?=^e$JW_0htMyj$_>O%C$bu$RG48L{WZ2&C)?)C# z>17e@z3yu<pho91+8m-Y@9&IE{U!K_6olS&_>@{by7YpJ=5K$JiT#A#la2nF;S3f; zDSR=#+R(v$PoqqAEtF7EmCxP>bl;Bz4el=aO=r4jf0+oz{lpsf`JTJPo^$7U#Lirz z*rL0Ew*_?NZcc0iwo4?}+q1LDEVUGyv&xom@Y2<247cIV0>W%XhlS_CXn+GXfhKB1 zlkLEMF9fYoKw9yoIFBEbwmtAoO2?fPtK2%89$@3BqiiYqJ(gJ#O3CSZtS5)QCq#Td zD;_7RGd7geKFUW=+l}kCIyx@xSzhNHB=BU*rOC2NCU#BeGr7%XUc3<LJ?Aa$X19l9 z!3ASI5lh-3?PIQ7x_I`G*0N+cdp23XGyum~<yf|@+gz`i9)ilCoKco7Tb;t_LUOIS ztc%QN<@Wt8yq;3CogeGg&6~waFW+r+Tz`qO)r)_L+^>KTRu(22MeP|OfeK}h6Sw$9 znybF@fKbPT$!GsTdDghElPCbj>FE=w$Ot1AM3OO`xCeU~O~LnREf(PRSZF*d#^Q?o z>;6J)+eJi7qg3szm{M%>vS1BMpTSV>egNC$?5H3hAr1~m4Pbo}?=89Nzi~9tHbPTP z;2V^AM16l1wX0b{vq4OIUpnQ|fwiRQ8kTb|JSWSTROq@C$lwruW0aX#qk-YnxK8H> zHw!#`jFjBf=_XQx5f~Oa{a_)-ei$&AuTgrk;Fu{BoqrAlS)sby2vM(P<hWoI?7&71 ziK>>jNt|rNgh>#=@{8vwQ;2CN+C+RNN7dj;t?yk<b<X4!I*6+KxRa*=n+N~<uQ#&s zN8<<2bE=+7NHzkYAj!vXMxRI7VhiZ1-A5jsfl9YIXAT7QZH>eFtlMtesE?J!WjV9* z3rus4%J)WW(aIZ8p^48E4n3tHQ9k8b_cpaLHU+paT&KQ&zhG@L^d~+YM|w33YEs); zo?4rq3Nc<hE0ql1owNNup?9pXzoSN`9`zQ)HeUGtR>CzHtF8B$38y_<uAl9Okq1qw zEdTX+(rVhaW!TarOK4Jc=;mJD%%b7NH#6(E9Thh}Nrc0pQib}>U>LwR7r2++O5|Bv z#$sZ13Jk+K41jjkomNzn@>A+j*<RV1qLrKfRUum8@d7)uAkvYEb;YBRBYZ}C_H67Y z*CV(cW=qC3`YQWUlp^Ty9BOVaRajKe%BQ=1)=~2qO0Km~<y8?gukGi^DWj|8w~_x{ zl5n5XXvUeaQ-TfbTLxs1Ys{(2S7N3R-$a=O{Hvb<m(+R<66OqPkq?kCU+t;Ho)~0n zf8?t*CK^y>ifN0KeIZ^$OW<*yfL`NGz?~QZUTT{3buT*ARp{p{y4spA`#PCdq%(!t zgVbI=WSZrJZYhdd&(h!^D?ghV6EWy@F=6~$$K`8cR2A~~Yg!i~=>Q|o`GeD>@AK1s z*Uv*oP}N%In7?%8Abm7D=%i3{BPIHITKaU$uuS!$8KP0af*C~(-(~u;_{URw3*`*_ zdq{v!3xx93adJg%>3)ftaFArB(~d`3U&FxMhmx>t4)wF+v~l@12ZgHeOpe<MYqVUD zlTo7q$Bd~p*2Kygp^kIVfQsAec8l0hIqR`q;=D`l!Ai_=8FPhDB|j;;9S2nq(o2t& zuVsWN6$W;tsO%0(?L55#Z_=u*PoRlyrM<ImcYGcEsxI&1<|W&h=}dJFKMPX&^W-ZA z0i&a$BB8*+kUHUEP;}Bq#$nzyRih#(MBJ*X&HVv2u~T(#iFn5`Hb-YcV#Y;ww&oUV z%jFGp?bTq2Y7%URb!%e8MqcNK{i|_I5O<HT5!i)E*X{+S6LYQT_WB$?PniqB3C$=a zc-J0AY!um5rVhzVUM8FoYe1AwHM}cgq^F07=k7%Lt1WA2w4sU^;sS#qs}d>lk^&}8 z>}dr$wl6ypRB);DsHO8~b^1t@aoA=_md7tRbz;K2)jSa&9J7=@>-9u+J;6&>r7Fe} z1Q+j@6rI;ze+5kFhp}4Uw>xg<puZVf$+@W?>0GSfUi8Zhbz}Y@6}@->kHZ+jo_eNB zh(V%q_s&vwdO2BFfGpWxY$G-%v(_2hc5_AcDm2Jepu?qKUkzVEKPk4WM>j+2dM@ow z8vq`m^&8RJX*`fav$SU)?UJt_67BmEgZxsQOvV2JJV3+0J-Z{8?Apzzotf{|zIMm{ zv!jhM>cxsvuURNkE@|ysfs8o<_zT7QN@VBJQPZ3}3lcCuLXJ*(Vf-n-Y6LJ=XrD6d ztc1sN0qxRH0G(w}9yLBmu9JSRk?N^2Appkvq5mzs20=JsXT)mCPH|p0tTyVyWvdgg zFNy5FhuyPMb=0E4S|<Hfp+!omxNNp?o>_06JTmFIA{Aep?DP~m+37hq-Z^Hn+1lxt zj<W!Wvn~X+Aq{9r7aQ8bG!0T~rSn5-5w1tmLUjx)V)~ewbW!zC5}kDj(Ju_pkTLq1 zR2W({9GB&l^xFZ4RyrIj(Ry%#<2d?=YJ`rbv3XomRHW%4sd@-<f#dl3bvRkuxFr4f z6V-);MwN9&Srgsy49Zk$90~#}TWgGJU>M>@#ipY5E0K9@)7GY0>x+%?jWiTetLN0y zEVe7E>1ZOYDLtsHRm(ok5FV|sc~;NMl_AU6R$a+j>o`YW3Kwcu3mdMoaHyt8>hvJi ztWh>ls2=G!J$JBCIlEm~jLh;lFu<KTpB|hM{Ste*(RvRSB*iuB&Tzgfl#SAZL?!B! zl8P8Dk>vFj6jER{Lt;v4rIl!cMM*%Xx!m-4piw}Fxh>dAv%`Oh{%GoMl%m&=Avcrz zha=aWj=EV2(W6)pt)ZS4nWhCY?9WY&>4|QM(#Dh+q|(i4CW0erg?KV<k3xk4fPRe9 zBc1YnGHxl&uoK!YcLuf;$pde($P_~0p~fB=A1IU73|Yr(p&}L=in}##5wrJ484XZW zMvJ3_xI2?hiBUlJDC3&trK2Re>ggqHH&GZrj>>FO8onE`P~>Jp5+Qe*(xghpone*3 zu1DM1jR5gVrXYiMOB;=6>H$|z)2x)cOke3Fn~-#fv72Fx=vyIaCjK5x7wtYu7UH2y zLT24kfdm$wx}YVs4BMkNA>nVV1`C;nts)i#B-$)Wy&Zc9@e*t@B2jO_27`#O6(d3f zQ70iH5)l(4vDyrxo=5_+I*Bd`ZwZ<GJn;cJDq$oFJTc0YFD(k;s-P%JCW46`nIsaj z3T9q!kOd3ToAf+yvg!=TZqp&Xpwkh*L9UjnMFANNW^56$*#I>Pf{sW51Mjs9JdX%( zA<eC#DA;hlPOyj)bV$Oy)uOYB2QHQb9s^-6q9z#=GqNEfNzgh>>}GQiTJA7Gl{)M} zh#*o$5avbfvtlA(tb<&{U~yv6rqjDcLB!Z>auT6hXE50Xt6vJsSTIUh@ClI6sk78M z1cEWI$09;bEVu<?hH=73FJhxfmJq%2ub0d+6(%t8m^L1U5lKy=RT8k^B~=1}o(TqQ z)JZx-|5+u20a@(4OO$v7!>yMDLC~9Yl2At^On5i86XGx%Y{aA|c5HRqkDqve$iyKc zNpB<Az#B!0P8CKjt6<U_MFGQN5{OlA;msK44$GJ)l3a~>n+=_%prn2e*^$A7B%LVg zWb8%&7H(uS14v;QdcBtj&=W}%3<iu4j4MWh$Rpk+@_HQ>^t`B-iD(fdyIE)BbuN+J z1Hjl=s|20iY}O0NVkM%7POR0$TLmwSrGY9}IG_Rm2jl^`t3p<t1(~ccW|CflKm(6p zUU?@kRPlO(panW;1`M~=2Tjl;izL|eJP}2cNGvdt3+|N=vOyP&$ZzLi&MYvEC=lh5 zX(Ayp9m14f5)wtf9yT+Y2Rj?9c)`UJs7-R&+*qu3>2+aIGK&TbgU&-=>v>s+%nl<F z9U?F2yqMHjgGdi>BRP1Tm*_D-F+c#|3O2I|S|Agvju6c28f}K4-G;3MQTwF;jYKaR z&B!iPI|xqze2HK&#K2`YN;<t>M;x*q2|8Z3>7gbgv0;-zr;{WR!>9^6WaP0KdH^d8 zVS^|P-yVJh>H%cIL|dzaX{L}ypaNJ{SQG$?t3+72Myw~i4LU;%adVx$%IfB&Y8}&# zaGi09w=$Z^MKvKyD89a^kxS)QYXQue!~|#K*taO0lHl@apQF%FEBv{_QmUi6UQzI| z=)?FePs_XaXv#qCyC&Fd>TkX!Jb07dYA@b}{2r1=Hc~BCd~D6bXn%C-9nWb@rC_bG z-gs<i16euK$yI)nQ`6(9hSRkJD3@R=h)TZ;e9LvLpXVs<ujY7e7+h4E5Pu>|kjzX! z{0(PIY%gm5;t%KYP}*An+WRJfV{)o)schzsDjc(KMa6}i>~*TltlOR8WL2ggffBez z{#Ok(s$B3f!*-nPLw`W;*ECS2V!nLOO_<ze3;SM4#TW!#C7V+rSCD;&VC>Z@re6@? z_~N%!=oLKu5c<FwDEnh>buSvwSa@ilceTLf3Y;3y*eQdwYlAQZRPiL&yIL~}Uiw~k zk*Ck;F=Z3DM!pQBXD3jJ@sy@YK~m`>Mw-nmD+EQg@t_%5tU%N!(B=0-r%N9Ux?g=l zed2yPK*f&%-H$GZ0NH0U#poRxOM@<lOzcZqd7r{8fRW#gpRYDLCKojTMsv{=tZtTq z)-A}ITqVcsoZGoc++=PFx0qYWxcQVov1y!y3^lA>mT4EL^ow@$B$T*xrLR{r(-BNu zi3t!xUR+Fp7e0N}9g8;KEcWf_nA$7wxdS&2AG+~?<O-x`tnD)NF1X{aI~MdBy4KD} z2VByyW|xmFElS6d;O0elT=n!qp?7M>jy~~bP52Q56fT^HE^BP^L~8CXSa#ff_m0%s zZC6}6HP)1Bg1^|*ORw0rR){m%Lba~=sqDg2^A_GDY`eQA;%RC`>se$;Pwjqjv+yAo z<fd|07p<iLDO}4lvJFVF5yLKNR5JWh3d@v4gNP^1X`1-jJ`*k404OStQ67tjxr7(| zZ}?-4wIpDarM8&0%4e!0LH~*BhM51Lzc9sr!XJy)9Q6C&sBRdWitM`R;Nl%OE<Uk% z`SPVZcPx2h$rwM%#-Q{7sWbVitg$vpHiEk`=KpcE?|>iw2^{|F1O6x^s;(QIsPOiO ziw`Wm=*Nq9+_ZH0awvJUw`k)s$839Z8eDMHKnpdgNI!_BUBgPXNXota)ag8Im-lYP zXu`=S5$c#Ru>MfPZO^0JQ*Xl_y5~1(zx5=V@WQ>_ht~J?)cyqMjq72}nVEilkXn6b zP?ymp`-_q`P4pNDqG-w<vb%)8w@qJvI=6m$8~S^;WLit{W!3l85AnaMA3Af(FW$B5 zVp4_fE6Sv8nWipAU#yFu`_wHFZQDUb*-VL>$F1Vlb33>@xcyw&=D&a#f06BR3^}(H zmpa4Q6HG9d$!ONIZ^*FgXohW5A>rbrQ|4ltnc-&SL?TYQnaLn1i~6Xw6)1#RaYqv5 ziXxZ9jQN8*Lu(}(;|y&?r~O2z&6#a>OJUwMIv#N1HH-H=aM#imMrqBWJqH#~)0=nh zH0!4=KCoxe8cAqqx@hkMdls*eAf@ga{AG*XX3o_L#D98Kb9~{dE9OMCSM$Pnb9BxX ztF#xg3<NFneZ(RFp7e_WJOn%DpdN5O3-2QXxxzobgpF7`Bz@|2&Tz<<1(a`hFOMd* z&b=qKWZsgS`tMCGsWa<m&(@pkmZa|O@7Nffza)9@+}25m??t<HQq~p<JNGs2Npyti zqtKDq(->wCJlJjwJ9RBSVgs}Y{d)jsv+BYv13Jv}Hr}V^v*_?X!fW?1+PP83)pHRp zLBA|9>K>+eLYA~uT=sNALP0$W%JdK^exfs(E_=km(v47Ih<*_Q(N989y8_cXbL!7g zQ-M9di#k<inE-P!@=MLhmrV+wK=IoXkQvHQzaIO!`g8nw_2=qmQ76?YqF=2*Tx?+Y zAAC;pF`hPmW{+^Qa5J+4?JQZ<&@6Wp1@wzr8;CKRk`;c}nDoNUPj0*Ois8R+dF!bs zug8}dx@=}c;pzETu0POEbUll*OL_|5^wrlWRp?Pex6Ncwuk5*Q(ef@l{fdXTJ$VJu zUH|w~|F)&@w83QSGT=++9$J5O|M1^?mSh+8;OSnaCQz+j331&9^k~=eMb|-LYevcT zQ625++BZI!JRkdh%!ppUJyBwLjnzRNt8?d}SQ`Bx5Jz%3r$POrq1CG0$mL+T2@JQI ztI!P2mA_5OhTrRtv5ynwERzhV%HS+1_Mhf7OA(l>xZRP5S**amTB`oZKQK!7WL!IZ zmDlV1z-YA<?S78kYiah5Kec7=(RTjM<*u;a{fF_K?2OM#)DDAozZT^(6r(9=usKS% zhGS~y&JljLRcUr_sQn2aR^GIo%8e6P>3)M{L-%V2h6l@rl*#YLhM*Bk)7r3FnQrOd zxmsB9{jh6q<Zg!TfUSazwmPI}H_oK$#Sr_D6`V)1E6A9x+I>m1n_Ui5W^N*NwjuIh zDv_kvrYJ=-3Ht>H;g(Gc*Y{4IG`XhfYM*XWShh{Etw(b&O>|=Qkl51O+fq~29J<JR zY5VLerY)L|b_svh#=&9H21@<X284BuGw)h2yjK}hIa*5=tm<7pvBsa(wHc;1I1o-9 zUVgo44$eLmb|jW0n<jXws@ppflNZm7FK*6Mb*kUe#?0c7$qkoZ-f*PBWRLXkQa7r% zlq1HB>&RV-l}mAJ*<LzIxG|zB$D}o*u-XSCs4@*Z*vgj!Bv`bTTe^+Y6j~_WoXTj@ zN}i&vH!A%lGDYQ&nG_@Xj^ERI$7c{9-`0Dr!DtaoODyr!@}1XBo7&s^^Xco_Bk!Td zqy|rGWY&V&3vRk@;r_{19i<zuwANaM(8Q+hj@jA1xf5F!*5lmBxA(H4iECE8nA;_r z!pef1Ivv$OQ$EzTvZH<Z?5R_`oy~sl87`W-c5Pd0sJYeQ^2BT=oyEMdwN{Bt!1Z$! z-Q-Bv<*xR1c1>F{yQYFKdO6j$mz5UH5H9OeJR^BrqBbCImq)JXt=8jaZOE($K+EIK zc*=uC)4OH&$jE7TSg_$lm9cgWTO&GRuI^0ksb9KiYi(OC!kyVp*^H1yoEYj_e(}0x zZB4EAu-zqDf##O$o360nC9n7I09t=ybhc<EI*aJC`jk0+8#WzyZrk?GF1O8IB`kGV zMn0o11TkKYc!>awZ^`QQRhApfQSlx1PdCr&2)6hg!LYxrefHz?*Bo5hG1V19m@G9A zGgi!!*My9s)hES_vU=xtHuX18X`dVjHn;TkZ(r~Pn)`B9_|)yCxp8oup)A8O_L~Ct zaZhO$BP#oDALAc8HviN9vGtApMkxJGdBrE{E8L@FRPNkypFCxyo07Xs7D1pQab=r^ z=-#qZ9dQ!Nc%c_eP*E6~SNVlex(`>Md8}xULT37sP1M2%5WXnP<h@(Uc1$Cl|I6E( z0Jc$_ed9Z`m(G>6tILut>#!upXKY!LZ!58LIB^o^PRM0)Iu4MVKth5Dp^$Ke0O2O) zD$tNZxp@h#+5)BA;e}FKXiZCb3oS?6mjbc1`OnO*4j&=B@BjNgh_$o3v%531vop^# z&-46#c%*0p;51w2hak8?{yi)cPo5NG;)|lla(H|4m6a<!9`pw;dj9$5X7oq@K|PYb zQ%%s%l55|I;cL;QF?_A~;&tStXQezuu4{fiy<9y4J!*oiJtVU7znOAUGx99#7z-M! zc-gQVi&bDk0mUJL{2~ENQT&h~(@JANC<2cW$pw4MuY%;AM(r=Ptm4`ax-^pDGlXfZ zlwwUbL5B@KLDC?_9WYwS_@dP2?hm;bq{%3;01yd_=z3%N$7GjD4N)eCprrx8z7qHk z09q?JOc>Kt6SG&l{pcpHlmZ}-lVPS&85{;Y5Mk9GhZqr%A{xj4Dn<yE1yd0zF^67M zjeT{{oCD)EVCxQtEpaC{ZfKIUmDawN3xC?X<4@PDfAZob<=rzp8fbu=C351yn;zW1 zp=Fw&2|8l6tqTfPS(w-540`BnvQFz=HpTywaKrvD4{v#CPh-#SAGG)0={0!y@tnij za@I}n-FxYeK3~`p*uJ>9cH)-#oi+0E$s3k{i#|D_Sb=hN>&lb+Gqn>Haxk@WWbpmY z%4P7Tl=$Iv`Fw}A!nVHoiN8$V^<-b~6T8nUpEbj1V{|NMseR-A8}GlouNha)9<6Da z?_BA$Je40~ymOKN;cz_&|7qSG7j`!E?7D2?+S|RXPN=Xrq}D};-?{se2mZdW*}r{Z zam|FybEnqGD_7r|4Mfh_w%kNs!`O*FTSQRd1Zo{|Txv5Gbb^s+Ac|xhTf`O_DWTFg za`NH#X!rQ}u~k=HwQ6Zg?>RU24-E9*_X=2i?z!io|A3e;!@?b|&^~8fEO5<B*EFqJ zjaOgZ+JUp32QM2Y-QxN09xcT#Q?Y=pFpm`5>)?qix0UoTI_``5>_HnA!vfJrG-6}# z__6%cH*b``e16-u=Yjb~;Cby=+aKO_V&~2iyXIbbR(mmr^s2`V^r{nYojCCp-1w&a z>{B=+C<Er_x21lz`J#7*-npo1(xRDxp|&=h=fsKZ8Y0GMFi?e51<?cvbt&(N;UJ?D zB<lIAU|lP$MLLB=`GRrSj?Gs$YDCA;^T{nY%qTPBb;Yt7dk*iJQKoEQS>NHoB>wK0 z<l#?4u5ZbT1CIfPlRY4{zdLA^2R^>);6*cMUUX2|$Yqei7s%w7PUQH4LMqk(gY+B9 zn2C}hcm}8#3?<14jMkZu2w4(+7D-DWCDmnc9+28d(Fx^RQUw(O0RxZ>5zK)U#vDii z;wvF34*ANp2`ULOLVz*LtgAvBV9h@FASRK2A1TA9oP-G`ugnUNpaZ}JDYNn{9Db82 zd`Nxn@YtFnii-G%Z)6bjL5`kV`(aNyDY56Kldwmj&d$zvOmeW_D0!Kl!KB2zmd`_i z`)7(#u;<((TU8v<PQ%+cG@{vlm)8<wv>|y8dfXY`-LM;}*V2?)#xuM-dgOC+@x(5S zMw0vP?GDD_flZLuzJoCg9Y*m2Qw~XBK?$+qsx(o`LU~04=)1gO%J~rhBIi$O_z{@e zP`<KLzGA6vLU~Kffu`qJOZf!d68T5|*7N4hU{e!T(2YYM6EA#MP3YgN%BpK>s>^o$ zAq*DGIv9}$6MS`1i71v7Rr86@oMqRy&Fo!H-uWYFJUfTP{gtcu7Iwu|7kd+u6@7)G z-e&QM=4#-x1xSb`SSCLSR)BT$;GEU#ez=;sR(@*sg0}fKz5Ems`#~qPmQ7jLcJxj9 z+94nPM^M|ja%JbVv(Fy-ApH^)*YB7V@kG+^f@{H-<HsNLj;}BNbLPD%<7*w66LC0p zNjQYV|08(S4g2(vxr5_JEc0W4KL)AhJ71SSy8R*f<#TzU*{?I1`Odd@^gefB_Ur@C z^{#w;^8UP)`Mn!}bp7q1@O8RSeo6l5^_|!4(-mkAXrXRJFUtNVx@yYA19>a=m#o>i z^L13l(o;6>Z|rZePn&NTXe|y-^>8@emsO9oG9(<Db*Npaio?hOth#n2FeJ2u4P)%v z3>NI)f*T0$?v0`HQ`8=zRDd?d%xLIB+O2nqE@Nq-+*_#C+VvjV6VjP2Ityoof&i9| zl@;7PM%F!mD#xo-8-mf`Il&;nma%exo+UslhccOUA#{P>uGNy2G9$W`-i>amK{vNS z^ceK4(OFTc#>l$o6jhGu63$_GDE`Ely%k$Frsra-v%;Jds{%NRo%nlTF5!|9IWit` zz|1RlA4`V$9V7`0GSDlVuh($y+A4lc^K!Gb`_=<ia|K*U&ZJ)<8>r^H@@gq?@&^Iw zYK&$D&H-ItUIWOP=}@IdJ_7c*Dh0Po-pkHto^hbGdq(pXLCNt7*=$$xrR2ds6cv2{ zxF_*VuK7}aJTopRm|J!{|4~R#L$VKsq~~J_8huI39Aa`{To`^}I2soLiSCkn<vUJX zyLX}6S#|TS@`gz*uYy%4PJr2Y7+tK%ISQofOK>~*E4ZCWUitU^n_ih#+p}bL+c_al zbLHQG`1fDsfV*s#F>t$n48li`=GGu^>_#KCI=>d#I@E>mTlfwX1@PVY2}t~-7t629 z|GuNI=j?#Lup&Bh`Yk|r#~tZAF>b=~GoUN5jo%AZ;Tk5{`{>#^H`mwCvr5G}q4&{O zAN}k8zn=kWVep$Xqb%&Y-~<{Uz$uEp2#sMr#SW_&AmS3M7$;O`cr;4TK^*Y1UDT&P zG8Qp9i-mbX?qf8fQDlG3IL%<H<YJG4t2B&6zif35?ka$6p_wjS1N0^*-5`hs7L87> zSqbyGKjsf#4@F83l21pHBaeBE7;Xc(30}eTvH4UKL7u8FRYD4TWQwfFj=9%W2bFyi zcv#v4F>+sNeSSD%DwWAS#$H`lDswG9n(C@c)#qfB6w+pAQHxc%DC6*sk#j7uT4j|H zt4&<S6okFWz=&NknJKsgF$&Kb*dzpOAwN%UM<;K7^X5=pXy)pfzE;{N8TE#mMU5R7 zl=C)5ZxQv3jW6GE#SXG7TFCAN9gT}>40@vkDydUo{!gz0#)12MAWfB3lwsfB=hMe~ zZ@#$~i!ik_XV$_FeaI;3s;Z_n>q<C_g)7lDPs>kNRp}%n3!eg(E4r`$^8pCoS_$Dw zER-@?yNU*B#BQvCus+3>;v2PC;>*Txw+tsmA*=T^l5Fw1yPU-AjA^o(2~(&J6eyS9 zfmF`eQeVoTl+A?af+Swb2mQdC#fnXzi}KG;lXu>)EYoAtiqVATgPyEhNw{FlR4KKT z*d|F>xvDdv=2xQ{tO`?hBu4bzxD|W2WuY;!W=I0I$eYXjVR!N<Q5_&0kX+I^b+}x$ ztETr)+Iaba%QsHy(2dh2jUO44XxouoSJ~9eR2LLf8r!O@-OIbJRc*$~f`V$Mxv6s9 z;_IIJ=~LG&rpd}{RV})kx2taIMYAd^XI(UP%`AO|{+8>my9I4#t+{P;P1n}i!dTGl z4%QVpoK>|Ib#)cBRZd4y9X=K-tlipGv-!4FM>kKHu=yw%{}t?67l}b3%hWmBkisKL z+$GF;xRjw>pt=HQW<1$184U*c=UOdD5UR)?Oom8MCQtSgl;0i&MH2L&TA+VAln*m5 zCNM&z1brE>NV2q?g@nvt1QKqdD2V|s&sl&nwk%8#$bN@inWaQwfZTWhlTr3yGRhS? zn6Wlrbw0K>-wx=eDJ%L8kK21c>=8uJL+m{LgaNZ3RcnReZDNDo`+nSGd>d5!_+abd zzOL5d6Qj!*CXUMrK1J3KH=-g!oVJYkF{l;p(&ZKQJIdHE;F_TP27@5Vq>Vw3B!70A zLT38A8vnJ3>d9Gj*sQMx9Y<d&0Vz;G32+pqH;y_0{(3WLW??cM4ciZe>#z@|hsip2 zD5hQ}q_}P9gN?l%_QuJZ`ZrB!DA)%k?<?p#?5*#v_jdJmk?o`=02rO7qqi}rms6@6 z$M1*<d-sJ04qVu==kV4grn-(-tgBnPH*M-}S#5o#wYaypxHWOOZ*fr>{<zDE7Gu1O z++N2HzivWtU72KBvi0yD`nRe>M>e)xX^R;-NiUAnAB&aomSDmXm12~beaIJq-laFD z_~Mf_A?5AiaABKrhDZ{%*|3Ev4GMhpz3+!yoX*l5z;5rp;^RPbyx51+fo6-2bA{f& z7awYvf?9`GoDLGLD{b=jBOiWvWS{l72MMHxrvyoHqI@1%y*nhLoe~ek{9p%vYu!f< zUTIs|ike2{`c&+ySep$hzENxr9v$gUk*q6}ilH9Kctpwl1l5u0AEJ_q3lyaGElr?< zOcH~}?ORHt^dOSA6wjxDq14iSEVU1{X)Z=AG9p6k`$vV*iSHQ*_PqkX6xlGL%JzQp zrb%UiPwDii!<LWWTji3Qx)X`+Z@=WJ!+nEP-(f!S;gQ6?r5NoJFTxz5$M<iNlj#(C zKBGl_%8T0z4q$e!92R;dHBpfU_-ur!(xph3u`Ads!_#awXP&eJlT#W9@PEUUd>92B z#X^zeXqY&@54+m2sdN&37DHd*kAT*r4+Sdlusy^XuYY9vTf&(E(dbQk_Z?U4zDoRx zgk}Q;19vWAG_Z{{vhx-n=0pYR3~$K+<SO~t5mvyFRzdbS_H$4z@5$1h!hPTZok9MU z0jwhK4HO0)&OpuxA{@x1@{fuFv#+qHbTEnU!VxQchkQkm0W=s+iVTItfs+*7lQ9|z z{W<I2DJqx75#L$-`q8$ehpC@YZ%}`vzCaR`2^51$&@!60Iyz#X9kD+$|482Ik>}5} z|Nr{>GvyyyUyKND$#`3i!eYX_(pfPrhu2Nz(x>v$^l6TtF8zNaKRnIx;bq47skm+g z7>mkhe;>%!^k1VZo_8$$uQ3jemHI!GQ6B4H?&sw77<6<%5#aLNf$<9DcYHHXQNO3Y z`hWkG{BL?`)-NNkzZQTD-#{Qb+}o%HL~Nt+?IXUd2J?TVcYojBcM5C5XdJ|8r5BP@ zdF4r}_sjH6kU*m(=D|t)AM2xM=ut!0Gf6KVu)Tvx(y!>0QqZ2BtYejuuFQQtfLtLD zgpkmY$nuzD+iNpM2Fka-5(w9fI46<Umt~I#WGEzT*|%96Qve`_8E8YqIvZ1Lv)O5w zEW63CfaINm=9fHVDRSTyB&`!uw}PFYNn)<bRK{LQz8jbHqLgs;D^?tJ2@h$wkpHDY zmp(@JY451};3TZV=AVq^7W;JeyAtSb#9uv>!In^P>%&wH`W8EtD9STd{d-A;M0*;e zifKh!OcLpbNe!m@bJC(09R&Sj*XHx@6e2VD90V60TPips-~);XUQS0NmH;0JW2;~^ z9F1c`W;7mgprg?ysQCJVh=WDiI-dmchjRZwLjL_E-26TLi9~;@$Lmd|Qc173Cx!Qk zFf<7S69b?pc~AorUi3dw!vw7t^bdGbUX3&9)S&GE==W-|BAD<Nj@J?WA))%8aHtm& z>jV~aZN6xnv}ZW(i~Eq6gz>hgM;SCRB$G!zOnAY7mri*TINstE6`d|8QmNF3M?fNx zOs2d;1H(8|G4n}|E_H<8qXG{?@DE4f01-bvnac6j!VGh2zU?-p*sd@IM#hGP2Lu^= z0nq<3!Z&e5xxNpV>saNIQ%c!V%CnSGB}SG^A#+VAr5k<$Y#d%Nh~(@U^uL%0lH$f; zjdmm#F0Td5SO?)&U9HZgldE((@D@tc>U8oBupb;4^YAf}B1h1Vl4XayLpSzeQZ6GZ z*MDZpMdf^3a-6!%SO?);{BY&I`_U7~O~G5JTw@)EGnBHDz5QUnTH-3**oSesW>8l% z5oYeN_8QI)A&zyBiJYm{!w!Eos;Kz+;QTQUQ%bpxp>l1_Z?6#?6XIA0QMpcA-7yZs zW20X#%7F_u#$h}bq5cK8lJ|&9r3EADmQhDia}Vn`^k-u?78&1A-+*(o_x#?S;B;@B z+;avnG7);Na?k(43k2t$?w#O!R-$`u&6V?eHa=Z>n&wpP(2Cqxt>C5Rqx2}Ye5)s` zk=M0?Xx<otmm0q%A_A8nw*B;g*Q=1Ed%bA_k;LI(E-y`FiXohpo`GG^g0owUcnlGm zyF)p5v^*aW6Ny;IW7}{`LjdGq&*8FS5m1#GBW!IQ8Ud{l(4Td3n>g4n85#2U!4zHy z?N?x%`sqz(bHCXPC<Q%KLBFMYatqwj&|I=u-GgSStUoxT$T|)*vuEX;;FztJzht1` zJp52?0VAz6Rt^hqEm=SqP*#+Oa0%$whB@jnN&(2ai9_*5Ki$(IX@Vv&lEMJINdS?k zQ%I_#NFC8{3O7MN7~fSVQ_AW?UDpZ)H;FWXvffY6@Or&NBm;wg^XDHuDi1}eOtq~* zF-~S#o^QyK(jGcAik6n?&m*mUZY-<^bx!vY_McpRT458&5FKY}$nxoa#?C-fo4Vy> z_a<!OC$m|i?{IzTK=&p(d72Y2fe&x$hAHeCK&HOKeMtCKvX60`8nfI2%u{oatVibQ ze43)~&OlS63bv^mByqV<bAm~w6C+CuNQqB#92Z%GWcdVHFVs>Nf{KQ}za}--K*7<e z%|;Anq6@6b@@p>MVC)=<*B%t6N9($#_rVs$xPB$sFlj;+&^LXkdHKHO%l9!~s-|}Z z&}{F%rI__`>Aqj~O~)DK|5BuN#gLx92H$Y{bow9o(&g!Ul#@zGg1kk!G9$-k`z)1@ zbis{8B~g7F^E%@&{#szAF{FYDVv7C2+4AB3S2jz;E1}WxV%lWj4Q7*tWdp4%H{WvG zN=#ZSQxeu8(FYHIeRmY}|4{xj?{{e}R+Bcsb;Q^7Z=WA4HsF|Dk`4c06j%A&A7rs) zDe~RbP>b+PAOL?As3R<Ud1;4&D&-F={*fLU)0NSZl^2+RzlsTGEMv;LIbK#Li@z{g z&@!*30RO;SvdL!-oImc?DGyD#x%~VCiIs<bFz10eKRC29G0+ma{Q93CUOF>*|A8y| ze63fwBj?<^;rhF8*th=P4H5ShptpNoN5{P3KNnr_fK9KrJ#fLIOQ%-~Lgn;Jf#!{i zW^8H>XgO(I>*@)+-u&#yoJHH#&YBnS&Y8J(+rruX!@nyBehccjhrgQd9DNnGB&3R` z6FKuUCXF3Mpfmu<xkMIH5dL)}qmsg7xt+y4mSG$r1?m(KJJiAiv=_vJzUat;A;p!= zF)!tpi&)GZcJ8a_V5)S9Oa7f0WfG-J3qVL5yppCe*oC;zfT}piHdM*)l^<#!Ol8ZB zG`m#nEV~IV3zQejH#*IMiej+A{&;_`Dn~(6bkaSojq;7rNx8}_?XRgK^BSASM_x#k zP<4c5h?r1A1a5@jLAL@A7zye}nX8~zfMyRwBfpqWQhr|WW%=cx#_1}k))ZX+z~u#+ z@wHA_r})xLSA1!vuRZ#iJoMSowP*{N&yIdPl4$+43vamLLUaRko9<Psx?N6lHF%>> zxte_XGQMnW?lx$+9`W6dT{k;{@l)*m*y93!F8_nNX`Hp=)ml{-xSSeXS2_Mat6QX? z+MKDD2Hgf#6>9&tb<-2y{c>#O&-fwYF82MalnlAjMBju-mmK<^)kHB0f+zk*g;(V~ zv{7c6_V2es!i@0mDlt<5e>lJ?5D>mvIw1-vQAi4+67i5p!h~8GbtAw1cIwdkhf;6L zZ-a`r>EzoWHR>9iTt}*-dUz3>@?;WJfCm6(F*jw`MetaR{iyL=IhR^NZJ>5gmy(s& zd#J~V6(7|J4F{+m@w{|6FOBk`_lDA_7Qxf!IpguurP=(nC7X`oeTlG>jkF1vd(7xx z(mY^B|I|H(G7lkvk?t|4v**bMjJ=!L%9OgF+oIcU!WVptrq$`uZwYoLM$iPCNRBV_ ze$!u$IwX&=qi%q*QUA&PB%c|_pAIGQAAS&xe-)8Bp{~{0sWNH-mew-9LA-_Vgb-{1 zFv4u8S_d=HaoEw6$)ZQZiQ8)?Vhj!L$p`n(X<Plk@?#%=bv%Cds(8GxqTzzUa|1=h z-c=MpeTz*6bo}pbS~R&ZF?mztTKTILCedv676%qyeQ)QcUu+6Rb{@BDi;IiF-(c?& zZ%zE-^n;sC!2-8nr#A;0ErSBFzx~tFh>hCY(`;B|nQZ~V=P6v&sMSb8_;J8$D{l$4 z#-&XL)+}0a>`$idEb75!R4p}`+Je7Bj<>}m@{7{pC>koYs5xw;QVtuc7dnaRYP0|U zY8E>2#4E2o_R!n!(x3e8Mytfu8*8O1S4E)0?r=$KpV%N-%W5t-_Tc_X-wlHg{jb^z zI#cE~&-8#tUeKKX+(x<n>1~w*oR%)+oV>*88HWBtV^qr>w?O{6C7S2Uz~}$FhQw=2 zNG>7k2PFy{=ZN(KyLDvzDeN3;K|#k<ioy^vgwlA_8aMVTid>l&d58OO<*DoWxy)ze z`3)+^=<ww=yH$&3bWD<l1L-k@^_azk>&IGc)4@sdm5jsCYBVxnyOMxck6D5JW3NOp zzLQ^}i!F@9$m*3ux_9i#<$U9xrEC~e2iP+3G`K<-w~_$XVIm5}Pg2D0dLuH~&=Zg- zOAu@nal2?-Sl%j0oY7w%E#x#-jxK=ZHzwY>Yj_@T+wlj%i<2?BiYj|!NAOAV790sM zqw%KQyXy@WpmBkN_f45)92}8PK3VwlV~VT_PaWg-umhBiDn)guL~T!794sBy0*T<a z1_|d8W;}{v!XqSsORNVlcLN~bKGAce3A8pFz*q8(3s}3u;*i_qHng#`3*;NTVkP+Z zht{G(+lS!al_G3ES*L3T6Ng%g9{{T+f~b6_Z14pQfB(B7=)+t|-Z+n`lE%pmoqU|i zVDBc0k1>@4)%W=^;2Th|FW3vyNlPiKv%AwNdq5{zS;}a3izc4AXOId&HeiPdcSWfV zCV5F1m%-Y^vN=SfNj*XE*8-nn0nD2De5x;nqUh#GsN<;j;dMOX^im1urjzLJ7?aGH zDu()pSuW_g|3>{qtNof7c2L&ep}(Fy>jvGEXW{r-t3|p0J#<W@RQ@6+$D=Ymw~iu? z@EOOxM+!$w5g-d-MBR&L#yJ}v`;b`7M%4oV`%Pv3!KcaQ=DG?v(0)a7Zbco^M(Zln zO>A|1LRVSXLUx_x66R^LnM!_p>J}HsA6^_PFKwOVDp*{H6?b%quFIumldITL5G-q+ zr5;qU?vo^z(}=Y9Ad+;KQoYnRYOl%=tgbxTtq#Q}miV}Y^5jJ}8>0}$;96)<z|Q}1 zI~o}Pp9RZt06s*V5!=*21SanmAn(@fu!n-DPEmTNUaJKZetG>0)6zg*EG!EZ2psuQ zo9zo=anEsIUsx!AE(UC%dtUmcFXS&&I2|COWAY;^Vh)&TgV*HUCjC$4*5IaL4+Pp% z6zK_oY$AE#xC11A{{<K(2h6esMEP?!P{4J7^o0#QY5`m>0#OCrkw5>^hKjV{d~$*O z6We-)G>Xc*<$c2*hR1^*^pOmab<JkQuXC8?8;@o}{!!xSuEIi-xfH}jA9h|%IgvEu zBtM|JI6S2yJT8v*Ot!r<n`)P}OehabHHrq~y+&5k55{-jy>||9W-f5Tsj=lv&2GD6 zUV)`JC{@nAKHzSwE=v>@oMqPR)_IIT*V=niM%RY;d-h-+t$gGQg{C(%k=gJ!OOKr0 zlFAxz$dyQBsIXBYsc_LKKxA3i3y@R|W9d|gSxXE{O5iJ`R-zwImUm>tLnK<!r^9qe zjp5F}y{8a_@opFtf*`ic27z|~pTs{5mnIo$0*Ho3EQ*!c4d|PhhU*%@(fx$t&;)$Z zh(rgXJ_30ViM2tWEkH}a<|FRJOy(Vh6%HE`sS}CPmwa=?siA2=GeBp$O_q}{JhLBI zuY@*qkk%Aj1;Fm7UxlBhWtyp*dDhIjrg4!f!M@%VoV)(KD{7`M>Wb5Uz5o89GOdB; zwb1H3c|QmM^8+6-A+14cDEsIE`78Oi@c!4`g<_(wy{)R%7pe*C-AjW-6LzesU*6PM z-t6mE<{=jQkkNZl-8#Qt-PqIDjsE_1`+Hhu=;3wiKIgnECaqdMjX87G-h16$2}aj! z;`;W+j&L`r7eKn##jJuiM+LDDyB#mXkRA~t^B7(^O@i(;B|pM_WzrW6B}0vAD%561 zX&R+zlqNWPOw>QUaEPiH=SN!xZI$)D_sLk=t6*di^lXeLYxDD%6ebj{%f%jJVjneb zpc?qY{-_0GWMDxT2QX&>mI*Bqri!uQ=EqnY3IPyO5EjoG*IC&SJkJa4djG|}RW0)Z z;{xZ*o_D?{=&1^JuQ;p?YK;IwSRAAeujmd|q2uSz?>-0Rn%9!}Yc*h5;0#n$+8b)R z%jYZsPtL}tE(+fqW|7#Ti#7y1Dm%x`TD)XVd3Q~Ny|N<UgbaiLFw}@k)si$sJtLl` z%4JNFaM?x3_ek+JUxqE6IG~3p{B@F*cgm}#@s!{tQ-asdB<1(sBQS*OdIO-|li$P3 z2LxmhB~DSt<!_E-|64kd{6K!jxleJf+~)*SK2U}X1;3G?<gAsceaDaQgZLMF*CM%Q zEAe-2$1^*+VYWD0pm@>qsL}HZIjRC-J|FYIZVdtj1Ra>x;1CUFy?oR0eeqb&+2=e% z$~&q)yU&x+xIagyW8NZLd1w0i<NMa$2WPEWw^6A;6nf;FlKYNJJrT7cu3=R*cE~z( zr3>EzZ_yoa4bRW|Nh>@_e#OrLeVvlUDzJp`GK)pdB;>@7<$p`HuiC$DPt<HEG?=4% zw(Vclt~k8F1WXQI+cxcK`jIrX;K55Tn$zjRjBT#YITu~}-~we}ROKZqiw_aNwg89E z;tP)Oei*$!B4S7t^s*_`qG)nCi9Wxa<0qZx1d0hSP7=1o<noD_K~LGBBS?Bn0WSg3 zAo5*adgdT8QL#9Zh%o0Vzg?u=Xd?3tAi1wA=U*OFVI6~2IMivlV{JT&kq6Yz8pow& z9i)BeWRSqcK*h71{-Hr*U}!m^7fw+duTv(!-K)G!o2Il_8B3|z=&%@AseZ!x`hw;A zmJ6Ut(1R4spsQHro%hHovz7-vP}Hw=JhF9&OjWeE??KPT>ZWNvO@KGlI(6RZ6DEme z6}VQuV!a4^0I$V$D>>!m6uV?)u5Q4JrB@oW@DT(bq-tbSxcu>02{u0U6G0U?Z+dk0 z7Aq9wB(F8-6GnEv{9p3lX-?24EQSG{8SLumJ`UyqR<JUP#0rw%$~QjN-6LOS)(tRP z{L-wTU_Q_$Br8Y?(7I*zv)4!&{~_*#qHh4&hq;o<@z#g_Xdl71=1$xzKl-S#&|UxF ztMa3-%74PYw=hF%e$v!j1_#qDov8QvhNjTZ;6E^Bc4z0$hWo+b0f!34V|7Tw%3#%V zBpM~()I(prR+8k`Ktz((VvVj*lE8~n9sI$FRn4pMSr9?_>Lh$cqmmiEds=*T<@xB* zVHJ?xp;f`(^Pdl2LyuE#hi(fZ@@u3Z^yHDx$ECtWQ;PW-%7?Ew)AK<*mWg&zAn>&# zp3hvJR~so;NiebjfYJgZ3kyaTV2pQ=X?|^{Ax6G~%2D-FUc$(w<<CeXjC^5m#~5vp zAh9DB(!=7YXJJU1fW$_kLCF^eKAH_OedeL7s^Mz;3(p%Lwu3$ftgB65WRmOIWHSBJ z^e^b652ru<C>p&={&Y211-(yzcTTRn`)<;I4W|;^f2$aBJ}s1dJd5rt`Qknxu^-C+ z9(q4Lc?uX;1bzrU?iiff$UGAooQj6GSLCmN9<09puDifoFz#n+TbX%<vtxeVx2lXo z)qq2-6dr{`>j92DwK-1#wM8;kZc8hOXTWOdlrk!<wQ3lHe%TIrvs@={-my$8Fd8cw zF?;M9jcIlJS2rrccH`v3Zy%oAsO;SMRr_j_Mq>v(g2;SK#-^cux!<mV+GW??fB&`1 zv`{%|mCUP`?6$(oiS&MtfBR(Y4LNzc{~mH6eW7*tlGSF(Dk|SUksAV|IJU4qCt)I0 zkpu`^FXPRTdU;jJK&AW2nZyVdaDwo8J%dY*pDw3{+p`oMpGYe?q}z~}GuyqCnNt5w zXA@B%Y=K>keFA4IM5Sc;|DiJ&Mc}6jWbN6Y^+S9;oR__{BE9E~mL0O5f<*Tuox#%@ zr7@25ogU>&ovbe<?UceEpkt?LZzeV7PuAxJ<Z!)4mBVs|{rqx_^#gKc0KXk8na?@R zQ*J7Xt*%+$7fiHC>_mhk0T9_E1gk&^W^o|L?To0L7|qZK6_;V~BcuGxCxX>ty!CxO z5RFNr6<o}&{mCcthqnq^ooIX&EIq-K6FNO8&i9N|BKrNGfGI{`MdwAJfB#ma#cI42 z6o2xQwJgx-$Wi`;eDAA9QKzL}8|g2Zv*BT!lY9mYGL#c?9->Q(Vo7)uyI2+byk4`} zVj6{<pT0?)CW3?{O2Kq8C~36lUB*sDxmVbii7bcSrIyXj7I=_WmV{%dZTWCU0wybA z4!ad83eD-~<>$eA*oOvW%srAmjK=LgF-BiGv^}^XxTk(ofBo)YkiHV_?8ZBLf=sjg zd>Uh|;;ZU#ZhTc8z8+pXv@M7(>feO&Z3xl_g6JZ&vpcw9Si2~?|HzQ#F??AShgo`* zUoG)oRhAfrd#mR7_wxGouoZ?g_;uk0$|17mLn}ybIft%fKJO_U$gbDRwS*Q`$w}|c zr$9yHBq|YolD(KJ#D3Q0AO}{Cy}<)H`d|8_Sen8?S2m5t(62RvM5<J@xIjW|MX%gV z-xtIyrlQr9(!*phSPYVOyPWirxeUp{=ou95ZBl0aMs+2`RDu>Ckq~2E?EaN1Epf{! zbW=IyvY<!GDM8<F)M{Bu5{B-Y=9STXL9#d$gh_o$uu8*afs#B~0EBb0z^Kpg7+kB) z?v?+kiUU|CTATkCKZ9#kSpelxGT$oEKgfTdxpALEBfCv~rw$%Fr4x7s(tcXn0#q_1 z(`X;L8*Z+v?*VTK7Wo5<QLuo3MIMA+c_2MNCzlo$-dxySxD@seV`Y8l=DMDG@E=Be ziD1N+q!Z8!(6=cMz@DY(ikk}ym-d|QyDz~0wW{AG&lP7FgW^D0Kp4|&+e_f#)1N90 z0qqAxx4{@5(d~NxQX<0Rif}&Cawwm~T3}gzNpj@~;#g^IX3!}Det}h%DB`gaVb z-YA5!&p;1)DgB;s>5gAqdUm}}cfVfXIXhj^SM|VEr3QlwhK4oQV<<lw33{-(q(9uY zqG#ut<|qDWptVl0az|Ztf0>1asbP(k8~-7Cvm)go_7q?N7BqPS)$?!|4HXXLz(F@M zMSJsH3`aR2f>bgIW~Kjhib5Ls2gFHH$qiSGn38jNZW!^ZQpM{~J{r^vBS(snt;Ad? zI^>izQIb;*(NYSNr8ld7o<{8RIsDDh%L2u6!tDmB;y@tn9p)4|V*DCWCS|x#2Z=M6 z$x@n5mRdvynk6PmAmP}4`Z9rg0)ap=NV(l|qFDaj_b(IiQ&#N1F$XwfnG*Q^0p(f0 z&$oq+=-hYZHKhf&ZTjyt8Hvdi^y|ZUj<F!~FI1);G$J|doCyegz)yv<B6TR9O~xp~ z){TU2VlHJd2#6wtqk?h{C5acF3C0xDhK-H%n*rLOXBng9bfXs_eJbDGwgNZML`b*e zLal5EXAi7hs@HOKIc+yGjLBNyE*AE@R14lVYjw1fb<0kg2CtZqgy4jtUfw<am5ara z;zFC5VU0$^Uyc}Tn5KneS(br-{XjBqHcAcSMPZ8|z6B@;dcN@pHqQc@p=p@hWH1O@ zT)_^5!L&(l+P<G=(1if9Jg@5G^bl%~wwzb(@RfZ9<pmo8arhQTsIWQJVNHQtS66cA zA=Km@f+!i!UV&@6;_IJ1a~P=z>$FCrjxFn{oZky-NFdo8;7(Dv8@<SD<?C?H!>Eg0 zEEz8q#6KSW!){H1?qWTFTDGucdDpw5aH&y}FMC1(H3n4ODT;mz=?^Ovp7pGViM<%x zFz}OOyaLgS*IVgul?EH?vTIG4rCY6rN+pS*h3L0_bwm^{H%b$Cb$1l77SlT3Y|_Hb zdxOE*yF9_}x>&e!X7$8zRRxyk?~sg_3u42D_GXc@7-nlsf{}K_TNjqCxWG~toL*HO zt?!9X3cA3GTRw0-j9cSjZAE3oiJo=24njR#<<cyeRaomE=dhW@fV*N`L({Zz2j28N zj$xu7;rzZbGbdUPnE<V$#Xyl`L5Hg%><&nx)lnU4ov=uKXM52*Yt6{u0^sc`Q*f9H zXPt-RSpg=Lk;5~g;N`&Xz}A|*qVRy@?H}C_N(7z8_Di!?ejQ_dY}$91U7k<Y)b42b z&+Bl}X!8AiQf)bkw865qf_U(8>!b3mW>GYNjjw8r7aOGob3_51*en?@!+BA%Wv)m- z4UwpU%8R6RUqA)&S7A!B-AxfWYB9nxQeP#KM&oKE)6HzT4rk@yl7~>IATf%-t89NG z|4gINiNBC^?@B@4IR0lE+s`aItw#RUyQI(k0r-_IstTAU3hRv0d{O8%N^qjtY!>B( zp@q&x7I3d*7A)!KBxA22&X<hq1W|s{D5<>nir!IAbamYEF;_}{$+Dd>_vvI)%BaRj zd;4%yS0C7zeo1}^d`lKAdC7Qx#zdX5TSNCt^tzWWk`v%AdCz~JKhlv69k>ydeY+s$ z@egSz1Cn+M&}e%e>KRf%vRfT>F)8kI_#)u|K7f=U<$$6i(xk`G0a{^_rn9BZjfZsR zz4)YITRTr@7aVwOtB13XOa}mL3&`(#!ChAdCW9k0@1Bj0Z1l<R68!FW?&Y0Focs?@ z+>f?;3+#Ur*XLp1HF$IGVpgX!?{~3hfpur|&OJ_kB{+8(>)LPD>DVP3ahB`+kD)PR zJ}5`(GlLnv9!e&YX{1Wa@1PxY=vXr8MZGkAv(pKC(XXI`y+qblR+hmclhNRmZw9?i z<=0>|$q%R*uzp*AiemnX+A%^+C745YOnf3Rye$y*hiw6iAALq~Bn4R_p@0QDC^~B6 z(TFXEflxg(U022U2?%LzD~ET`)PQzcIp$jN#_ijTd}QXfi|5?hU3RNDReGs-W39%_ z>5N?)-%j{$ol|=2tew3rCp;BXnitj1(r6k(9W@iGYCO`Ef|BOi&hiO7+vJ~E(G)5X z>Ex4Lg@>=4a?a#xJ9BCf3{j`RQxR|ofZ~pO0T}ukel^4wH=Uinqols1z`#NI$AD%H zW|zMTeB+Dw96AmF`86~>Xaq-bm4b^wuqD)ZNo?eIuu9Be-jvKxb^+Wh2<ZpvOQAOl z5#dGZ;BN&%?iK9b()z)H4B~^r=aIc{Df(`v;x={oEErUNc#O8yE}a3Ud*nWQ96htt zE_CD<rT(R{peRPBnG(T{i=2KwS6Yl&mG)+JGa!tW#PX*f)oC|}QO--KiPRiaqX?!3 z3h-z$0tyaOMtxEy9immSnCL3j1~7#I_JYAI1^@{kHk>gkVTOWmfREs<6p@(we=^m8 zsqmQempb|9I-@}^r|?Q#iukf%x0jCe(_phfi%HWA;$JU-ars)#q!+ZdZ{CszrdR)~ zdb<4K<d5(EnCbmpwVAO3zhTgm0f910R+d0-oa`4b`{XQW5w%(x&|CkqIQ_Xs)I$h& z(3kY}9X#0A2M?$F6x&aJz3XtjgSmRMr>!>_Q8W5G+u?iE`;K9?lTO<T*=ql3tXj)M zx#}?&k7@!PlD~GK{_9t9)39FFKy_jax!`xa;^8X>BOM{mv=0Zyt}^4zUs=Gaev)+L zB-xQk=L9LTbBZE6=(lIATIWH(|MLt<WxC7M-;}HW$XcGhJw~mdoME3CRsq>Nc5A@? z5p^Ec8o74zW~;Jgtfl~4&fEZ`&$F+qeZC!g1P6(cpIGis-{*r?4DB5bh2x4G8V_Jz zLN)3Me*hT30Lcj0?E>?WuoD+G)wOnZ)J{&{d74Up?yB$JKB=|JDTYnvU})YNGqlaF z==;IJb9deAk<0G~kk^Q<b5gth*w!~~OIK{cSapuK?l^Z&jakEQ<{9Jmg^t2&*Q`JE z9IW251w6t#S+l_?HZFK3y@eW8_c-x}&ei?9$GH*JlfJ7tzDM2P%&*@szoQ`-weo{M z8msn0Bdf_q#%zxaZe&7Thh3U^>x#q1$aOy!qYT=4JK+-Jc#O>q2yHJh8xu%E495x; zL|>Z~lY&7WFE3Fcmpd4AyF&dTmrQKD!0QSz{c#grWwDsT+Q!6XC0&+@w=bNrE8q&1 z6gYcpI((u_tL62DR>@V>S?x1vfh38vpkaV*<`!bLLHC62Yyb!PUC>tH?P{rS06jp$ zzi9|=n$!i0-L7%~f-ZPTK@h?%iG@C~Ian61XtqkW;@Z+?k2BO&;pd!IVT-!vkH-B3 zi7|7lIE>ksH&TNS+HFJ|h7RlmL*R@t`7cyxjMXN=?a@SI4mI+}TTj;z>*HYaO!;q& zMxaH}3bZC)b!U}JvKH!jt=1*_I%;~I1tlR@VAqU=w@<Wmm0Xr+)P_wKOBGFLS|a@y zr42}XJlwKq<`%)~<btI$LQYR}X>GAhvNl(Q%Yx0KZ((8!guw!Mi7N;|xyxM)yC!W4 zHlT*<@?sSF%vy$)*pbSq7StN6sf($rs5_}gsb3IY6YLp}SIHt6S}lkKM)ZG_MSrRh zFQP8rTUgac2xY<IBVJ{&Su-p#Xh0-RutgwXv!dK}u{z9Ygo89{K^>u`^LYt6sS1AS zCH)ME_k1`&z%XqQOms>-wvf1_EZkur4vSijfLe}G3wSpbS<Gfj(bE$rreEluHWU1$ zJsk9DxrqP>Ry%0p4dVj7_I7W{I0HWjX@fgjS7fsmt##Wj^E){pUy?{bo1~jqeueyZ z`Lio3Cg`kI-GuV}FtooMrPIc<rbnVPd9Nm*p9XxkqMC@?W^+esifm7`w-Y$)c21A( z4!+J`;Q8c^TJwPUs6P_<>tuN`xPS5<`MT1|LQ4?%<$pS%sTepn9;&mIjVl44-Bns< zds15@*u~P2yXlf9cPLcU&^00A0tTC&uD?AJxxFq;|731O6KgWDO%)4|Ju<Jn8K;}e z=h01|^q0SMEOo4l|N4TxjYa6%MzV1?gDvu3ip|go%<}JpqPr4khOgnGuLF&je<AU* znf|eaT1D-q_EWb}4--8dqZFcd%_5<vt~MCKc)FH|Z;aQHd3jOPJEG*}1)~AdVOl_B z)QgQ{h8sWHttAY&Jm)1{1Ve_SUg0G@1Vm_bcMP5!kurTx>1Vj_1;^;2^ebV9-R=m3 zIcJ?U)VM)@Y5i*8UA)-i7HP0pW2hP*1IM(MSZ(>@#g*e@7A=^w1PyCdkGaF`9pS>F z@T93oQGx0H1q<LCE@4sQ(`nb!jf;&glx}(oeoWFsuC*7g7OI1VeN(`bg`tl2P+?)H zy(3fzW-W=<Rv9%*04=&bpyWT>?V!@$QB~D(c=_`5ufXT>56Wz`7n~zsSmO+~EPtWX zRUdmVy?%T=?w)Im=t?FnTsJEii3DdILz}4Et)+kQ)}%>qO-?WTbX!w5XR~qLO`AT) zY2Iq(QJN9t&GJ8hY1)Bx^W<+QKRg><9qN9#8{cG(Y>c-Coe^+AzRm~j<zTnsONn`N z|4m6a`X6wL1|64b!x@DcuL|2Ti!vYv@Ig3V_q99ph0Ur<18|o?uXTJ;>Y`uP>(gI? zZoN)t|Dwz(9}^)c2>-)QuMy>GResD{fL@`=R0&p_Z9`{)^etA4sS=*&rLU>XjM2*2 zBxU(U@OlrnAlPWmfxWQefE)pKK=xu`fW&aeDC5f>Tk+GPhS%(VUaQrZpDC8;IB$8@ zBgt!!x^4A7E%F+zJOpmh{C?OXH4Q%S>kXFQ0{Mr6U@W0$8v^MtlzjoDV1xGo{7>^0 zqcLkJ9Zxa;MyXD+hA-7J#Q=leD{S^f08?|CfPnM_U#O%SDl-Y{*)1SM_~u)=NDTf8 zd?Xh>^8je*>;zuH=k$66P70$^0wD1vf*^RjP9GW}2IVW>klz?zQ&JL~;2fPp@Pa{b z^T{+=r)3$M=5%I<i7QwdPh2;y^or}Qi!VkKvcsg~Xj;N7)eNii7K}P?#^*&!!NKBE z>;Yn1#SF;BXjouuz!v7CAnHK>;x?@TDeRxiKa%Zig=|OqxZ`@T006KsJsT{LMft~U z6__JC<Ojh*@QM6mS)^O!H{?HqHu<A-<lhrM_TV`^pb)dA$saMlmH#Zi2?Y6{@;~IC zgCcs5{B!xAU_27u^=MrBiujo5(a5Bf2jf2nEa)#7XL(FXivMW9Yq94Ws~x<%pS`nx zXpWCI`O=Hp;XCc=|6U8%u6+-Dkdy=IM`>>l7)U2!vf_^WZilWz^0DjSle^NVcG0`i z7x%zRPTqCo$QZsCv#51BFP97$Z3gGI#2-R(5tfc<yOmS=fQOFN+OhNm;;nRupx9zL zf~L%Q<D5A+%w^m(ydTR?fBZ3+3g*PRVzDlHig4xB3+ETrc3Jf%7PlO0(p$S~i{@WA z^~%$qg6FZikKSA)#}>W$k&Y#4@G?$AJ8|d$_bN~Mm^>tw{GPWReo8)X^!-VC*mrFr zI3FYZWg^+g*G#kup*m8&G;r%hk6d)oBk&Qj$?zB{U*OOK_?Y@H|2YuNUYG}5^05&u zh{S!vT(ziQ%jdz^aycqTm-j*)7#xX|a7ccA06vzU(GP0IicjulFJbRN`UH-yY{z{8 z*tsx{Gm4>iSB1%P(Mv>cQ$p{#ghjmpJ5D2MQ6ljWNQR`*{M81KxZ?qw#1Y(uAUe$8 zGng|YUczGE54u{jJsK`543%`oHwrJVY@1Fq*DqbN^CRojiW>O?`Lpt>gy>lsZ~o~0 zw&>CY8k4c2WWgIRtgD(bCt)q{a^fFhe89$;pK#4*E6ROC@~z(-GTDqQ548cCOG_8| z>q|VlkAq!c+-=Qf0Pkz-@>=H1v51By%Z4o#g%?g*lGJE!hCAH>t){w$*ZEzA0WDut zsL=$5MAw@3PV4w;+M==gqk*31&DtAo;QaOU)A!3xPhFv9PsqK=P&Ce6r>%Wy*F#fX zl^%~tUnK??R&`lh2@b6Ct~6w{Z$vsdVYdzuD&kn2gtL=SeF?V@9y77>fksuSE*1)- zkH!QDhaqm*80J%8IbLaN4~>p9SXU8835MNsO3Fcbc-}P4qJ4cdj8{&+_DO4dxZ<`4 zD?;ryW0l|Y;#<Cw+Il@}x=Zf8+rYX^)pvaRnBQZ%Tx%)c^jrBK`<92wBaFsg!T}8@ z8rMGt=qH@T#msn6I?S4PYk8I3x?Xg|S|*-nSlV8*$XYxHG}>GoYqfHGfmL$yNU>n~ zf;7#C3z)t>&Twn}Y<JZvX+j_TRn#8$wYd#`lXbkO<EocS*gC6U-)${iVzh>AKo4q1 z%tL_cz%gK`S^d}^h=-Lb8cAYN)Sn2#pwH&BSUso(=|{R9k1XyzwrQsCfvH<M##>py zGye@{$d4Mm?c-;@@mZi1!1|>ZT+j%;@46N)+qkfj<>f^~>64zis0YA&JHNsp8%9%G z6^vSZQS8ux20k7Mg!oylV3aL%Q)@+2NnL>sfK$|Q4PXnRYdZFpFT8Elq|3qG`RzCT zDLZhKj&p!(egP)yDi-uED7a5v-mtB20tDlk>fyFf`cwj@QQa|Wk9};F9)4vu%6IFG zf=<4}sL@(gyg;P1ndPKT2a;w<Vu;ORFym7~!8Dt~_f-8>varc<HAoDmv-h!?jbJ&x zJ6L68Bx#U1a5DoTGe9?&lV;@|B>>G+beh~VgMy#Iz;`I%89aqcFrrX!VE8ju3Zw># zA2Oi1lzLCaE<GGQAG7MDa1ln73SNqu;!Z)6Uj@`aS%3V^s69g@>QPnau&^HR(=e(^ z+gN5N8lS=u3NqZP3elazYG*fx=UtMlS+Zb4%k0^an{T{+^X8*d*Z2A>SFWA1V|iWO ztiXf=@`pv9wpc9KPEViq2%ymnGhz4c=e=H^AMLRJ{OHg@kH_zyP?BhmEZ=<5i_FfJ z>C@X{qMp0)oDJh>GtC&X{`>@sT#*haU<q0Xo0GbQKT0pae_V2CJ9h1YVtV^>SPB0t zeJ+fqcMN^L8{SBtH}o;Q1G{xAxU=jYGT#>>NpuF%fhejrM&>6*-LlForgUxv%8~?B zwqSLaEG~qJjSvS~V()tF$y$uv7;vCCPreNG!>F}`54;YC*A9+*?RKwYXt1ogX+d){ zGb>R!y?H_Nf#&kEW-zTP0e`$9IkYNy&J^BY<fpE9d&N}VXyCzNe7?D?yqPI-;QJ{5 z-e@k9=ae@$mxCXdHB0HWRf#|2y+13zi7I$2xH101_U$jkFE#R<VP9F<J_E-a(>G?W zDsO5+^C*_Pz9pO+Cdv;qNEHZz2<Qa=Ab2VPOzBkX;6gmA6bRvQhv)G4-AXru5E9_A zz@x}Jc$oy9#F7u0=|VAeIRy{$Xw;9#>Z0f{=dcE<P&dZJ^c?0u!kQsZ0-zkVc8&zE zk|~zPV?T|j1fX!7pjjfM%m`@u#K9v1Ua`U6hE@TRA<>Sr;P*gENxUn`)gEYzp&14Z zSm<gJ$d5A+18xk@L6KIZaNq*j?2cNVL%J`n5~EC2;W{+u`*G4?MHkm$VJ;wGu^&zQ z_ztx(;DNX}fIzd*1iT0hlVPPP@B?roVCfJ<NfB}3Xx@q%l9UjyyOGtn5I#xP;ZVKf z%KR81U^cv(*G?o)a$D3!xLMU3t1aze;Z1ri!%FN@#-wv<X!$mz1<*9F(=rwYKnNf` zKhE$p4SApix@iHYZ=p{gE;a$Z)+QQ_K=2pX8OEv)o0>QcXD<jA>hvO#Dl7$d^9B)U z#}&}PU+6A^Kx^T39HZwg09c(CD*$$_CJc<MPO$6T_Ubq+b9-5>o~5-0Yp1rtRS-kd zg1Ml~67u`pb|Zuwr{|4y;jEb5R%WMxr^qNeW@#YcG&U~-IfjL>q>3$NtPg0-bg@TM zCRBwPBL`@!uIhrzDja$PM9<`Gv;#s5w3|vm`^@xRw4T#KT1V4*8r%c57LL`j9HfOZ zQLBGkXP`NTp#??*W2})jX|*g3fetc^M$iDW0OM9W<eiS7E2IgnD735%i4L9LQ?Wqm zvQ|tfi4+#yn~)ME<xW=T2LKJ>I$?pu<FrM+5x>?bLIcYHKTZ3smjs-vCpgN>Y0;{? zaC}Flo<W13=;1X%d&qj8U|?WfcX&dyucToTi{yk=@x>-2Zs>Jxcg!!kMXdnsA<=A= zboFPIHnns{$LqshpN|%RU~-w=%o-p8&VY7JwBE?cbAZOevKl>VUmdN%FC5CZicV93 z+gzmc^X2UL^Q_jkySJ4>rgCRhxVcy~fYv#l61#1JUqgEUsI3F^!~)60GYQsHYSYr1 zJtm|;@(mLKXec&S6hm6C1x1qG1IkJmlVETF!NqDECOv=_V9;8$0*6XMbH$9rAPJOV zOb!4HX33;ww2);Pj^=^T>@w(Ei?uXg&^ErKh<l(xT&r=~irWecB><QSYYbk-ak<u1 zRZ5$hYO0(a8nXtnTE5O~ni|q@RR!%x-Ilg_ZR;1i#GqGCmswm8YFS{i{zAjk46WmM z0L^g*hy!|y1~3d@+%yY6g`5VOfYHb?MwX*dy8-6pONIi6-EOfOMMj$DHuIvk$c}n4 zdP{}g0sxwgsAn52ddEV&d3>-$YhZMu-{0x8vb51u#yJgky{SX6Xt@Fn=M`wKqHaRi z^3%F$ey!7NFT!-*YhxYOYwI?>c-F3R8z^#@9qCxHWApl^Hy74SDTUA<M6Iro3`J~E zm=~YCd%_gfR}&~e)hf{Ir%f>wM?7x5NsW)kvY0@5ksMt`)l#k00_;^34AB8>^v4`y zbSTXD@GR|6=z!5!f(8mN8{+XG2mE}D#q&GbVWdzPUqwcfR#59<9I;^$1Z68BG{<h( zb49@P^r!n;Y+m`-@?o&3yY~76yFx)mTzA1m2U1>8MZf>nuNIEmc*D>?(4-D$J@ZZ1 ztV_2}+Bv1!^bvgsXszwjcTXz7s}LnKCU-PP%RRcCBlNHmd?ja_vGAH1`or-0n$~5! zaM6d07vHwLLofpNH}Bjx;h#5s(Omq+$J75pp9{cs_ewu{+chcHY?J+eeH0i95)GY& z(K6PFx)+VK0~WqC79OM8ey!AUtbbI|)c|uRM`}H^;(LXeh#`)LEe3>J9>>kn89PcV zREW1Y!ZfR(&ta)3h6x!(j6KKP7;aoNqo&tWSSFedmUonvRJf`eHa*nSk=)oGnzo?% z&{=kG_k_sonzGuW+Q@%D*!hEv6TyZLkL>N8(Rr;r_}oTwx4HvZyaV2=og1rg>YY4q zHoGh{oIbxZQ5j!cRou3*vt>zhP$;nr*3xjqTUqICu3UO)aPszpM?UN}Z+s50*LKe6 z-K*@#gLsGN=M_kIc!k8Wv{4--;wobgi4%PCT0&DC%CmCD;+zh<ln@7BfC=rg@Tqjn z1?ay9*+N^o-cihffWzba!tSA+MyA*%{|>K4gR?~c$EF#r49D5swLbYDMy*C(Ztpb2 zyXMdrtVr1JWLjr1Gk@Xm`>lhIp$GK1Ohu->EjDy*Sy9mad8fQv{*}dUtFT*jTG?H| zYwca^-uQ~XzM)SopaEP;jaYY3G?h`FnrFZ`#dc{TGlK!uVw>IT54lbflMIV~Qw*{9 z4pD@d91=?|vFFl4E>kEISBCws1_=M7VucFR0h?qeeoVv2S?c0aG(f9tZ6x*^$?}<) zAC{^wjTHU4@@s9#m6}-9Uo|o13TeNt{Bu#HwB8J;&UGNUt`ksZx#!aVxb)Kh00X7< z<zW6kd5uYQh2$@*CQhsZ=8#J?f!p@UN6YI=i<}_2VS}7>(mnWsOO>)RxU50qiK_~` zfzxc2Hp}9(QT5&RiHS=ml0TH*)D4r}o8$pf8ag2>Jb67sn@CCCl*i*OeNZMCf1tm6 z(<bm*t{lpHv|(1kAnKB#a45rShZo!Ij*|QZ9u_P&TZ*bn7qmD_nwv_TEepm~6<N&8 zq7ehkr{F#Nw4P!Yl2=Mg3;n=V+%wI7%@|`u1=26xW>2Ah)QMOA2w@u<5NcaN5DhCh z&Mh1yG1e?`3l4^`3n!K{<3Zvh%*F}XJi+i`i6gGV&Zd^!_Rgp8+_ps7fQ^h<e&1*k z<o6%fd3BG=?+YfQ{JxoCwU0lpWf?OlK_@}U<66J=aZrLzf)e#4)aP!4Hc=DXBfoCe z>A2<J7>(a7=X5$VsO@1*7Q;8+7|rM`s8!Ay49Z#gb#&Hj{N@{js{8$vy_gbF52b>5 zT*Jc}M@GO%ZAp-0)S*s{l@Li8LwsPzVIqk$pU3K-lwW?l_t&S^9{p_ZK{Q{6mdlq7 z+>R+`x4r<xTPiG~Z^4p7CvQ0aym7?>{|Ty1?8(%9&GL`m-TT?mwYz@#%D;BL4hnC- z1vp;a&B1Zwif6vD^@fv&B4V*ns$iRODb=Q3u6i&MbG~nsAOEP>mP8(!23(u}1*0=3 z$r%pwVEs^m|D%Qo(g(4^f*<B^I0%l1_f$XE!=|W-*c*%@{0_XsZYA9Sjf?2B@K?d= z4E16gP)Q$}fF+E-k79xlqyhv19|jWOdP1&&_BSO-jEOI@*2Kh><CBp!olS!2DQlrq zl4#4zjSAeyF)36h9izLBNp&IV*nmr(oK8LqbkD-G>Ox0%oRI1yNqT`bkMp`PIGj5i zHVSXp%wp8~=PmuXVj<;1<u>x~Aa&WZ&!P|f)F}$^yO}A}WyEI?uczUqORQNyr0TI; z2+fT&8ucAkLV<yoeee;aW)a{VB(mT+690PW+ITBNup@}Vg?kFpX{-bfhOCO?)jY-i z0Ubf4!|=Nu$DHFzJ;l6K<!8XvwKUIb%0j0oXToarbT>?J(mJPP0zAWrfvr;xZ(ims z&;`!vy}FsB8B-Y$4R)3_Ypiu9b5X3kw9p7SQLAI2z;gx7M$v4K{>PlC)h+N43G|#r z(1`xB)?jlrgG6%3S#`i0uI1=&5+8e`k+KGN84_vXrDw6Gkf(rQtp<Z_!K7*pV4Z<U z<)#9Tm1t(%gm$M>S9(o9;I1~?Sx!Q-CPV9OwHpeHnitg+vOrVP*xOk;(P;2%p<U9u zATY3&Vqat3;*h@8UtO$)jH`S}Ywv|R69yll0c1^D!WH}@?rU~Br6VI=H8r1FPhE_@ z#;i_^XJe8?`llm<_lX)cSOrRWj=>*dJXR7!dM_Fkacr%KcCk9>!A@(~D33l{qFO=^ zPys_@NV`;2${;yL4xtlRWydNyya$_pXWHyy$Lwtytx+iAEgr%1MCG40ZkSzNeWGvU z3Zx_U%cl<gQU1N4?FZn{A3+d=rXBpboR&Y9|NEW2S0*0>i>FPfWH`aZaaaDPs<INY zg3i#ev}4=60!dTuKn1fdYIBw`tjpO9<}ZwdwY4#qrm&>7^`V7@;|;}yyZ$-kpKKCb zKK~@I`!=JSW%b5lfz>Zx+f(9yX2r6l?xH7}dv2I4I6gb1Y_93J_R`+g_8m{1vlTGO z2Y)avah+g5y#O|~v~4vCdeosB*TWUdch#e(qcXJh7}3+6<5=UYp7d6?ORROzdAws% zROE{5t2x*7eA!|PrKKdy7f<+Yk*4jzYo3tDq|7D2%%g$QVrN9=+@mi%fAqjF{efS~ zx20cw;(k!VM4xyy{TL{@-@knM!fy^9{Dy6j-9z%(tKJ39XThZ3q|4;LzPkz>83KRt z{6>COS?fcx!%ifpZNO_UG!|7kiYF)^Xe<^WHXi`=am8?&#c8$}#G+L!()$?!X*g(j z!fPV}{*XDGWOsTOE$>~md{(pBvROXzrsQ%-$3XeolBvrVtz0nIx8<eFx3m>RUA%ot z$BH=%5|!NKi&rjaiTLa+W6-##)Yl22NawlDB`jwZH9S&}gzDI$6_<3taLdg3^SYWW z7Dp}ToZh`-+cn@P-P>BcwBRYw={}Ob1+Gv5c;~nvYK#@r_ROue24;3uT-pz4NLz~P zr)`~FXpzP>wYAll%sV?d>!fL$HecOQ(Aj;~qPde}CKI#N#XH)fjm6M0^Wr%z9ua*$ z^z~Qpj;5**tU+Rn4aqKlV=3ZEZYA+mM8X1!&pxpEEch>I%P=xAf7?2{K^{tfF?%cX zo58Zo-`3gm%-LIkd*b{Z^1py_$NY(4@+s;Rn2LU`YHy#nV@IBxi4n?b)cBw=X-w^> z3GQN&Dv@c1WK$tBeek;iz2G%t@R=U{u7Iy$GO=3L;cTq=WUS(8%ZfQmaRGBwteDBP z|2qpipcWCdVP;f?kySqRouwTmzbk8|xnho#-$z*+sF2HQQNqqFRvbh79RX@7>|13} z!^RAup%=eLJQ$C@{o-64zIYnO0M(vb_FcRIYIHsDekXl^>f^o)$>cUFh9g0VIEJOM zxC76vR0Ip94l)|i3XoWwkc(nVgXFXMaI}|1pIX}}zxnL#^4GVW_>pDjA;3Sg=bi1) z-FS*JnoBKT$feF8-2*kkg4o36y&XYtzr5ZIepPDu2rPT`u|M1fw6{M2%33dt{qeGA z<Zt(W;M$Vp1K<Onlh#`bB5lhjPCA>H|Cme$<c487w!-TZ3x{a*`grU<sVr*ZrBR0x zn}zKPk~qpc(V#o}U$ZMh^YzSCf%34?>)G41-hGa{u1nugYic%i^xW~M_fHOcpL>7H zY2<%NJq_<a82j~e27Z~d$Jt)@Sp4_#$3O_U!NtQYXeyoqDkIi^t&maFu&jeo8;Ito zGhtXra4AA^A8X4bEHO$w$t0hXkN>P+5Z|Rao!031B(oI-bP((?xg7Eib#ojr<S0!2 zOg{dcD8N+{0jD!39hxc2vZ<3;t9)V`YsGsPF3Tk30wb{g6npySV!s0_`m{|AGg2y( z9gMP8vF{lVE-S2-az9aidz2WIo-;;=kIGN(^j*>7YFw-a<9LP%<6pO8eTynea1~H! zjj@kC>McGZ!4Owez{k<#=D?A@K92Vz@e~N49MF+kIv`<)Uf^LOtS<!5Iq%LS`lR{B z88aGr$(Tsqc^<R4>=N_hot2e47n?6B<D${o9UX)7>961WqG6M}P#$nCuIyP>bjKY< z%X+F7xqz1us%tw-z)M5gZJ<AqsP;Jvpv7r{1x{aeBEj2fQ>3D#B4VQL{7}iJ63_S> z#>>A6m5p~gu~#T~6AXYiv4<#Q^cC2;6YBSYu|(z&|785JVhvHTA|a(Rm&_0}v;jJo z46AOeNW;t}Rd_qp5K=q_f;7v1(K>h8L-qW;rs^4{xcqWlGq1V2%M`z*$ksADUUB>S z+g<kNIq`}+qRNRFh{ZJXapvgVHhc5@=88Mk-|HM20<rb-%kZ1oEob|h?zg#`7Q`y= z>$}(Kz=?aJ+U^!~?f*yHcfdzgW&gi>-+S|>w>Q0J`lKf_nVIxXfRKa`dT60{2_PL| zXkr5urKl)T5gT?aD7snuT2L3a;Ln1)xVyHs7a()_-}~N72+00)KmY$fFz?;^%6+$- zbI&>769Z*&=?HR_*glK7a&$buXKoKElE}L~AsJqgKU5P(FP2Kt>A9d{{)Kxr*@7n3 z1v(-?mv&@d2GXwVL+Kuy>A-2c3`wM#O$4gJKqV6TgxlkNDK@RXep=ykg~}XxX_&4J zmnO3Ndc&nvfx^c_v_tLSEk=XU!s8GP6uz4CbxqEk0Ec`A(>nj4L0PM^q(LcaA<h(L zW|x<S1H-LZOKgyr>10Id1)q5Mpm{izktGVY2Q2Q*gQ*eJRBACr@puIbLIEL@7DPWm zjku>lcqhI;$s6>={lta0XyS>feU>+wg*6a=TgdV8SP7NI;H4T8kewi2ZsJsyKaS%; z;sXT7P3s%Lq8I<oN?DE6&a|M<3UYWg&%Zfry;3KkHOlst_n?2J)3_}8Trl$G8-QZ2 zJkiWBN{Jr#<{BjV5-{sFbkC*NOnLjE&nx=w01H6N*W0&$jSit3(IFB7qd_OA`s9i$ zK0z;_1Ly@3g2_8~_3=}{r9jGuDjl7wvlLxUv(V!Nav%drJ+B5*^c0fPwVBkOXXbT{ zNGL75UE@;v+4;v(t0Zh*7Sr+I3-6$Pv*3Yy=jXtZ^KRTCJj|&lulNL1UGNN<wMS@w zJ<%5fIyUGhW5k~@N*JSoG3x*yNCZPM#>`ZsuTP?D{`?0p>G*N<DTsYWKM+WLcf*R? zf$<IEqBLpPpW1QFPy1~LgD0-t^A8u>j%v{AB_o@h2R&;uI_84kDJ2!8iU{(6(U<PR zz8LW6qPhXUbJ%fgCK#oA>E2|v<Fwq7odkGIN@o67II>USj0y=3{EPz<3MEAZkh4?@ z-}u~5geN5)?UET^(Mg$TyH4l@-XwIC1kaixiL}410I|9?8aO_!<B4<0WE)QeGJd@z z|JE}H(Vx+N^yh<&9#)%5Oia7!Ob?@EQkUkIN@~i>p<Jfv)yU+qyu3~^7CmM%F<m&D z>4Hbli-VRA!v8_#;~WRI1yY20!=v6?X8MN?3Zmg^1^!cmM}mWf2H#pUM_M2ST>zjS z{Qe8iCfOTAofg0o0R<nVAdzQaHcM}=rV}JsOaSfZ399Pc>{?YAoqc#xc_go)X4~&` z0@ru0ER4rW%N@18Hu(<Nkg8HB;0uGmOCWr;MziQk{A5+1-tgvFwK_Dt{BqsTX`eXk z$HE#)0S_5v;(iw%S^j92nZr*Z5P6u+q7xpPf`3vzfgXDAR&?^{s#QmU<5rOOmv`1Y zx#543$^Y3fcE{u<j|8>Ae>YSeNB8%V0-zi?j;{K{A69Jq2>txg#-bq;I|8C!nK(}n zyH_vOCP*VpL^&`hDAAMswTM3r*c@Tg6sIXcfNg>y-b_4v3)rTZo}wjO+R(#{4@@-T zkCk9<&_7_7z_Wvi8LZV-qkmUxwGzFgXw}MMi5?v*X^zF3!S7<tB{+1ZX@-B}<2A36 z#6aBk0WTSJXoIgL(iCt@faRr^frWGV!K{mnN3d9{sjk*&C6eM|$-DSsxG!}E$Tdqk ziktkM5@62{l*F22kz&?}UgbIiO~t-)jorEZwb!<1Sv6(PTyQ}gIv)p1BY1Nn^Qq8I z-gG;WW(s7Q$;9@D5<=qyLkTnacK@NNL~HxdFZGt_`&Fvg7crHJ)v`H9b671x1=R)l zwfUe!JpZLKH^;sB=J@>}-%aE$MaE}!Oy$jsTzR>bSvL0Td++;NVs(S)dH55%@kQ}9 zC6b&R$u4(6flxDj9-LF@ZezX+W#!?k=jO0_^u44tt1`zGQCZEaA9!H3)uJi}Coj&I zxbW;l5SbHc@Ueci6yXI$l@ljmV`)W|D!_$|qywF&CONJ1(w<8lLHq8d9V3?74ZIy( zxr>}SD=)ocDHw4f|8m$~J-mC-aP*16Za1u4-LYhGJHU&ngO7i-dY!@U;Mdq3YucAA z0S{cr)sQ*rPA<t#5}sSYMG_xcG_mh@UN1Z}$9gh3=7gBV1S>~X_C50G888F~QV%`c z_X4;U3_0`YBYm4*z$tX;a-trS+WXMYXC4J|bUL@9A{Q>W|J&~mUQvEK`ti{-ryd5% zs&e#gPDMq|Kz@bbeNX}7W?XcSdJ+1V?M>C9tVx?-FE}x2Q|-X-+XGI(-c6HGR;qRr z<2+wsPl|swDaHH)_h=cuk4~_54+yw9WO?vdflmkUN<kZ=F`i&k;C}%fP8%2=@<!?l zFk}t5>CHFa?10A9=U@nWiX_|&4LD~oIt&J{VgAvV4G-hI#pqgGW-vSqTyMOA{?^xV zXUBdqu|GIqe8~iC)FR?rh!WUtV)HQ|q)h{PbGihv?SMkuCq{n3h?`nsxpqfR4E>M} zz;zE_X5h_<v1^Y->o2?ek;|GJo<5eSx{NlTr$pJ9?9>3G4va`nAm>yuP(DYul~0kR zHfJB@;anW`_dSJ!;OFz(S59T0m2q$4`E(<7gnErSO1)40o%$#BDfK1w72!c$G*Qr3 zL#}}J5lvDT=LRMm4T=UNC5dW?rw78K3Ys^JNNkfO5zqSqM{Ukf*ie#2=^%oV5Sc&( z8#!}AO`8)1T&Mu%5Z5c1EOo&eU^HXmPFf@CED?oO%%#!fg7}F9$}VB%fCx+-s)kWK zG)X2O#i=o)2Gl_2<zZJ&pIw*hnNT^XXw_bSUZp)$Fny?8!pJ0foSW6t5&!@v)zQAq zSs@RkVMPLuME~w8>&$M4#E4vOtwpB>|Bxz-yq#st5{-?!Q>L@(G*19<m#Of&!bG9G zF_-PA&P(JNjB<-ILWlExpo-OTYKfeaF`UMxFI2Capba<HI0s9WSy?8f>8G`hylksi z?Nj7RIhZ}X?~uAQPe<qe6gnK2*_~!9T(zdE7M&_uI27DY`^uuF3}+l(=RkwHB=USE zf6Oh<rdI(-{!O6^n^sI6QaUeD<x13O1A8AhvU4hASQ#7OoJyB9VDi{%vRiPUkZaw| zCRJIL5!zyt*4<=aY~3cU#!Sysn^ba!g+P_13z&2o6CKvtp6n_1dufA3tLO8E+Oo8? zTIF}wI;`O^RA~Op8l@TuCg35>fLxcyR$w0~ljS=AUV)}eG5SO1d|eseqLIbM-1TxU zEtAXmIH%|vWy^KP3rg911?^WpQiR^t08XQjav&F~IC!Z+2b8I`BbAb30E8=xJgy#( zv42x$Op{HbHsNJ0nBEN``m<ysU$C|m)SAUnCaBbLxRVIF<#A9#&})eL9g)WXL79yx zBM4e?g2tC2z1GJ{aK96}Y$DAcG1s!ZYf1BlT2`via6syvKP{3yHkVW8@dk6O(Bdq% zsbxAdt&yl@S~ah*`xH_+D>s8qxjEnENpAGphYlatomjdb!WL&kQ`xTNtFvrvb%PDQ z!Yqd~w)SoGIeHuY<4?&@MaQs?LSEhMt8)4Cq#Mfe4(1yDqZ>vhLJ?kV@)lzb!ywOc z&@|(*bIQ$yYK>f(XE8`Q15`0`MnXf4TBDONN>FIZ&v%R*1;XX!VE}HK*<E7L)wR|F zP&Kp~(B6XVl}GoZfBtU4(pP|^XZ+o37Z0m-%B2BA*y0#J;+CMZ&91Tx8nS%d{nSA0 zEf-!TJE@g;KF~ll2xaVf6G6;Xn>mRAlM^*GZN`LxS7LC}Tp=s~i2@Nv2#<f0Xha$! z+vy22A;1dhgquiIT!1!6IFn#GOteKcbf^Ya6NMUFktyie?W%iVw6=Tb=#}FKnF{KZ zyA^>zU{1ib`}XIQdz67W%>n10p53?ab~WbNn>tsHZds}vbw53O<>=-m>M_qWDs~HH zTzh)(KWA;Bv1KNl)nY4XP~wc{IYP$mdz=kVjZrLZ8@&>|)w9P{TVQPJTs3+~w|2~f zb;>=8z?@)!6oh(m$L6`@j`*Le;qX`uey~;3nhk|#c8*>(d9Wj|Q7AGeeM4961EUp7 z8FTBUiqTItq@OpP)sSx+HfxpWw?o9t7(|VuCQwtT+0;DhO6pFspA#$;T-Aj{WzJAq zLopE~)1ky5Dstj~g3&S2y~JaI$b|$QPf=x)78Epnq*OwXh9x4bIRpYa7MSS}o_5WE z)!|P_ZXqDTi2EW!U1GY82N%!@qU=yfNGE8wBy?;f4`&*6a62#?40*X+Bh%0@!os*| zNsDoVTGt4rv!o#xgn+e~EqXZvBmqTv;S4CRSIDdk18J*+wwBZ?FJl?iTQsK(x?DE1 zngO)OP~_)z@VT0+&-@IZNHsIZXFWdSue0)xp#oTiPTv*}Z`@Jt88!Ty8mU~$I6TbI z2L?~MZnVZ7kb|9lr`4$fPQ?<1Xbon63m|56D;NWKjpn2>gOiQH*=<uakb%rN%M*6j zl``Ok3MtXMld~Mh=p;HQp|x5c^f9y?0HqN|%Gk&Vhol68VsJvEGHaByS!2V)kWnk) zj=$%mZu%D^{WKMK!|UCtf4MKIqbr^QYftM`J@q54%C-VdhNlia3=}*4oDMV6$uocW zQSzM%Kqf8&64tK<$(JvgZ$y&?d*clD0<q~EOpV1oVFk67+D+X}JxTTU@AULLhh@)) zEGP5>@$F<h=SEM&W7qvk{Qq}(UIZ5N=s*_P&rFg7d!oE)*Sx--`pKnX*c0w@>~Vxs zSpv|}e>?!{|1Q6)CtR9JGRevH=e#T5>0Lf3Ma|naxn4qrOT+jvy259Y{ndc_VnKA# z)c>Xc*bb=Da1Wx0H*catFQL-1n;L33o&y$9>je*j4^h9P-l9Ijl-OCI0d7zTYA&+l z*Y6}zY<Jy%8K?it@A+H$$Av&X|F_rsfBAf@psaF6)Kcj^=XT(KKO}!^!E*;l4e(6# zn`;5;zrT}&3ffKy`&<%yL<1mazx}n~qy52)Lm*u|ct*fX5f1RbsMhyilsylI3G<q@ zXSGZ(+l5C-i0l|%pj1a>of%~zv&oRLGG+Fo_tUy{=zWL7Ioxp)bf0vzI~=G-RIqy= zz2En$pjwwiNkO%)6!=L2$H|kV!Y86`9h>&OO!iZpg4AdPk$;JN52hUnUjjs5F(AE! zvJpm4EGqEq=kwwW;xr~Opfte-2?)MnL~;t#XUgEXs+P5t_}IFp65ThdwPjP2Z~#{= z2l}VHHTAiTU)9v7nxE{x`)x3!YFw~#O)ELB1v6SlHEn7k2PRxOzisK>q2zc=>R9{o zMSGjuS1h`<@CEeg(t;|dqI3L?F~=TUeynYNW%Dgd@p0(hrE^xaH}74vyuJC>Ma2H< zECq=#aHEL1$eYr}?&8DaXNSE@rsPAvt=Hy<`BRpR-gV!u(e&5XzZB?uUC;!J1zx&7 z`Q5Fzes>O2Bx85v##B7ev7vmRA|FviQcYup2%D&wYDvOmDp?DkPBo>P*wcP@s@75O zNY%Ri1wq(r$}_><V8J9X39kKgF+vEYC|{)xCAC_R)K&UuDqODylvH|_QrC-6dHDWP z(T~YFt7x3X288EeLHZoZ#L089WXBiNtYRSjfC1rvpHf(S@c{}>glfT!XaQQlzB?e2 zCx#EB!DujhD(FGA)><p7WoXEzgPvfn4?K#0z2>+X^!jqaqyC((UQoWj`+)}@NNvl6 zR^A2V`@5fg_SsYw>hf1>PpH)=ApRp~ZM7ft1Z%ZVgX{3IS1#|>)&^1c)7n~5rh=pt z3-No)aJvVo0;-Pe)*3xDK{gH2n8J%fj~6pPl-MIVkHHl1L}DdAPs~Gjb)P3dJdfcV zp~KQX4_Ar+INR6REdhJ<2Wpn<KA3?H|2YNNqgTB-6Q(X%dL|CSZ@z;52&`=ro<n`; zbJ#KVie)timt0yra!Il&bu070TJ*=coiC)GtWBUdf%GqL1OAE+uhh8~t%}@z=((Y- z*ZsxOux`TR9a#rf4O=s++?M`^nf*)`6`=C)n+z8G{9Lda^1|*D*;^nR4YV(YDOE%I z#NslZc<day4}m74BW52Sdbm*rO2UqzZD8H9o@?iann$%heCLX(ho4;w<&8tYEnw${ z<o??(f3x~3#n9r#3dA(c12yRP=S#uR^?hG1U3^<kNq5=s936W8sm{shk>iW!WVH;E z8#X_3aO2kfzw?H{C96y8fxI=tYjGKz`w&5A?e|(B?7^Bd`ez|RnS%icMF|7t1Hv3q zh{u(nK0|<PDM`V#uU&iXYpHEJ=Gg6XTI<~IJuSu#L)P-abLh7pTetqP$JVcVY!~|B z2pXn*e)FoQt^WX9hEG<R2*!ZonIk~KxJmtM&ph|`4oQxCcS~!nTk4Tk4y8X@_ZY7K z=%eeOML(k7Z+LXir4zvew-!OL`xzibKTzlKj#B(S4OD9y34m;$QX)4fm@miiL&wi; zgjHo2k3b~-oj97jeIT>HEVc<@4&PhSvv_e2(q7t8I@wxMP`T1-iB@%(3>|cz_$3Y+ zZkRIXW;qzY>)5efH~tZREaQh&qrZqB=%?+kZre6v<~BOJXYrEZ?TgW?2bPu>84UOu zl`AbC7A_P&=1qepuDoV;-?5#$j=ggudJY6ufOl~^>Y1@^+pF8R5w!8MV<ca(XDJo# zm-(1a9wwkB(Of_fzG4Q-WXBvctpPOAZjC2GfLI#=%%RadX#pUi10kMhC9*{7#Hm>> zh*J`DAVCz@*f^%@O?0CMqKSCyD>#kJ3)}Jz-B2^N$W1fP=^!Wd4ZlW`JfbY-^@DGe z{^J;T-`~nop~Cmj3;f51_OPYcS7a%IyWiC-OscTI%G0Fq{u7j~-TpqBwA<S;s8Xi` zJKb@mA_Nwp>r76%EMPBf_D|%LupDifIOO`dql`u{(^jd|*IYIx^%=U!>7yBr-47Ol zc@Jn!Ci>ADbj>qLFvIO&puv=9jiZ;)&On>b;5C`#dU^<0@WPiP(ba}A<8PkSpi%+a zuF+J9eWX?@_Ia|e+i(sog7@IoB19zDpEA&J)RQqF%{UUl?MJ$YnW!*;6O%Vjp1gS@ z{quNek)<e^1+uh|g!JmJ?w;;G3Lm=E`c0D@bIo^5qp4|knCChsZ362^p~>I`m?`CX zY04@_DTGP(Byqi&6pxsmOXAXZPF}x$GMcnWw5yep={8DLU_QQe0I&AHJg|tf>`8mX zGV>X`S#a*%(<xlmq)iun^rVQ2mn1K6>a_T{GX}gj;}Ozea?>R861C*<YsKTampDPl ztP|u_P?pe4V58Uoea{F{Ob=vX9_57aew3dKR96P;k8A=F_h||8X8lLepD<A(6p!qQ zpbbTU+;2s_R=*seh%Ly29iIrtm&h?mVBwQ|(DoA8fsdbn9i1gX-L|0B=d%X4>4G@- zhW<m6=Mk<DL<@z(fVsv<<x-7;E=(FUNu1@=X4vrrPW22(%@R_1oaz_$6Z=auf_pwN zP&I9~>-T8O%{g`xo3(k--|pwtyrawaCHlinyNY~P&b4|2Fu!9_TYU?{>(HYQztLlM zXS)^7Ef4Mk`Lm6@GxyC4;pdyO_@!Q1uE8m_&sNyK2phNMsG?S%)U#IQ1G+-<&|!sK zz~#=71{$lB*%K}h1_9BRE&e7vp@xZHHjd^nj~&9H1fTFQ6ne)3%!tj~?n1{vp#^;k z&fqY}XWmIY?M72w=qnc}go9mRp9|<*c<kx$?;07!TuQ0&3QIiNS5iZ$QPebQI<**c zq}!<LsN1N6R4?@|@wP>Jsh1dyk{KIEaWj&(<!FOB<2)cnb_czM#I`qKCW<(*P@)9S z_OzKZ_=NN*rrFPBxYLR|ID&IzfaeT?XC&ex*_602CPqZN@G?M?Y_bOOM2u<fk70V% z#B;?$4Dzvf+8AHJZx*OJl0XSU3R;UvNR?csR;z(ZW&$}1rAn%ms(@S}k*O6j*_kg4 z2B?NQ9aN9A*`QQrHp`^IcEoB`$f3~)<%-E>GgPXKMwPM)$JG*<!61{U!3Ok^(J1G% z5c5k~PCl9E6;d3+aj9Y&PBZX0qLRv#;L4XZ8VzO%)M|}EGhMCLnzTTv1X`2kPinnQ z4<r&Ll*<%SP7Rq^2bT5yTcvl5?s@^Zbg{B!2ljsh6>_y&p8DY%xvJzCY}QIyR;rbx zo&}!+Ij4|uDzG5AP9|HIlr_Eex=jAsTQWQ{KmXxNh2qN}lx*MkD%JOWD)(nUYGvGy zpGjoM1Q(*sKXMBFk6^7{F&yQ6FIDj0gLipF7Lt5xG=2+C%T%hA4t|E<S8|P7@fAP{ zwc3RYUnsO%#TVcUrB;ii|5WMqswK<dT3RFHSgAgB<FdP;UPW&*%f0ASujv4l!6%>u zAI5e8fs~@M{0ThOkRAFeVEW%<i~vq8MoZ$52k(KsD~_VuP#3!G=nAm+SK^03514i| z6F*8*V@BO4>SNqDs_(u55s)(=!sOsnQjFo#fc;#avQa*2G9EjZ;<2+8&q=@BuQPKx z5AmlgC|eT|E)b+;WD{4y8O1$w4hnwzh&?+X)*(i+2TN=YDquvgzsIkQ516u010XTu zNsgGj$MC<9ful*$5V?wk4f@EKEMbp0!u<fs$#9orGgv8CqF0n)I<YlNQ4o2e73DX$ zv}y<>bw!ugd~p9w<25P^VC9T#@@TaTmLwYe7L`ijHUhI!FC)hA$^^2PjE)Wk8#F5X zI08b260F_26PnnTsJ+w$S6D7>DN-}cW?_ph1H&A4G@>h<N>HXet!F4=&~}=FBWy0N z*o2uY0D@tUr2?Jilz@@j!n5;b8VE;sU$L&^mPlA*ER;Z+b*&k+AK5LJhsV*Yb2_;I z9cCDS>zZ(Tq~^x$m?&;oIA&3)!r}mcI<KFJ-)Pit`!>9h02<@gk44GmIt~kvezZgb zd?f|MH5&m|C$yapw>TY*{c20kZQ8#t$<dIZ0ZS+M?QcTwc1~v@Wbo5V;B`x`gDhYe z2-)eKzoFUa<owip@QJ~um0D=84B6q%`SVe;!Db}RcG)Z$^rSwLxg|!U%>bU5|I2n5 z`P<rom;(3+s3(yILpSh|I4w4Y0O3Z&4TV66^YpJa0>}r}VY68|i(i_7EJx380lvoG z7aGu~&9fOLje8d(QOs*WA2vSw{BLN6&*sg$o#Um9gyCe&?epdV9k9)xzmMY?8ed1b z54XwJ=#z|&%)s|A6?B1rYYSkGQuNb}DGh?`2z)v+atYYtufKB^7(D69mYjy+%{4_G z=(>r3U9qynU0Ut_Z7+DY#+>XJvC_`ZPyGp4fKu=281L3x?45F`$Zwo^be>qk3>Z;e z%J8eNz$E*qUb6Yo-qVd~(%<mhpcnP_k}An=QiULAW}Y>(FGHR;K{X2~>oK2^jrpAE zv+>v8!AHQwbwIEX7PO$_d@M?wB*HWq4U<Gkko@S%|Ml&XNkqYI^QG5+c)ip(JvrRu zJtFveW0C{@z^}b=GKr_MchH@7>&S%*M_TPQpf#DaA)DZzv0vwPz_%)+S_Eyj-?UB` zGhQS69XBN61n5y45|PzRS^;$>6d_(g3jj$m2r0kbIWdt#d`BMGL>Plj2ejajo8PcO z8#fqP-HaJJ)~J8hZWudO9}hylq=bjO;kV3A1yWP$1aT#Kx3F(~w<a~%vJdRpXO!r4 zG2<Ha{2F6StGUrIsc#Yw{V_w?jb(<IUU!i`9sRB2^uoO%Hq&YX8fG-|>r0{Fg%}A( zdI4z`wG90PWU}A1j?u|XU4V}ezke@ze<1G!a@j?`e}WoD@RNSin^hCrQ9!iciG`_P zzTz=)wBWZ05LI_#zKE$@OepYTS&|w0^^e~rwJD+sTKdEjQW^(r(!Z(k%c|9XyD%Ls zS83o?(4?wKpMO(};41|2mA?B9Um=LE1oC<OhmKvCo|51z(&W-|9O(N!4q%=qbapp; z9Jd$I(8c7D1d5J*<e2L**$C~F2=;qi$Af^VBn{l|V~@|ATK9)L3fnszb#rGd?HX$b zcH6jBtB2paeES`5AA0n;a?aXNQKxs8M`E>qyrUYv^s@O1^zH4o{32a!$+aH?4qWoq zduTWM>gBF`zZ?R>hkJiG*1K;#V3eV(*(1hwPM`4fU(zytPMp^ylpJ$Ydd!(x2{r%^ zbOAOIl7T>G!x{5#IyQi56rCaMRE)4BA`AUjH~~G19{>IC=_n3;haPPOTD*9DeKlxH z-Nn55d-OO^rS77m-o7`DdB(msy<R$s12_RJLYIKqANe<I=zAaRIP}rwPdxF&q>sRC zbP4)u1AzWRU<bM>H}zq*IrX7R1-<5M=*>1mFQ()_G-vQy@r$r4alafZ_DNya&gaR6 zf`p?Vz=P=B>v1L!m}jD`kiiRgvC;G{9+%Mp^La(DTGB;VesMRWq0bBkkiGAVOC~D! zFPqXj41^v#04#Tc({J3f_R87X8f8OkqO~=aH=?d?=!nI2tM0yM&9&1e)wh(iH<#rO zud5&0v8ZPCeXy_KmDT${1@eF1b;;B5Q0~$@%5Oe$JNn{Ii3NSVdi!+4P<35HJl2@g z*wN9LbM1;%+ovw5t&f%s5)-zaZ+{?SZxXAT1mQo<HyZF~;a7>66Ce>RNrWU?DhnUI zAx@ta7ktaIW;_9NCIfu!m#Y7;7j3@(`HuTKo<H-=^A5-Jov<FP6%$@dyNab}Jm1;* zyhB1=L{nJZNh;y0q{97ZbLu%E0R%r5D@&ihU%Z^KUy~&{!TyDyK!9@!PI{uuPZ4~A ztZK65ABve6z2wA||3tm02lf8*%I87*(GSoEX}`9!&<7tJ1?|tn<ijNM%722|hyMhY ze(gS)k9z*GQFLRw@gJZo|D^kCw2kP5^LRe{H*Uk}xTnYUiAXUXg^Zv~A{ej(V&+aZ zH50^qBQ9tI5rUCq3IVlX6)zgQ7ciLXiJFRCQnNm-Th2@W`VtGI;o4FUJ9J27PHVMR z8**suDz!X2KS!;a6KXT?AY?S$-s1_;jA^*TF*~n==iOdIp=bPv!6swnPz#flQ=F|< zsX2K#e|T|YZjplr^oQt@vxm{4&$qx`$H8hmLZs1IEB5R>FgOy@x^>#<rX@36PHVBp zCXp=FHQDS%i?ZBLK3=fc>j@0j>6WU8IGv@p9InlG8$3E~Z0(A*-Lpql>2xaE>8+2n zH_w{0aWG1u8UMKPXV4+iJwjhoVm>!awNsO*1=K3)O6n%!ZzJd@o)hqY%+zuC7}O@r z5{{@{6Dvk87EgrY33Ht0h#{ARsP33?7fb|0L~EOLOOlI^5qtrB89Y&@i-qETN{f%8 z?j^2}AXS7~q$^MZjA0njIOaSxczWL3=(c&~&b+!C-`CZp{x;HNFPk>4%*A*3SZVn@ zblcmdb-MR&tjk;dsapLncf;Yb&Z3fuB}JWOha24gQma4p)E}-GSCqFPuV`Gw;d+!) zS4xTpeP#1N7o(k4W;c!W`#N}6nW@YdBsVFodk1s@)z*{fMRWkYcyjC3lb{lGg36PR zU1WgFs+YWV&|4fSyC-jq66ze4C7wgz=0l#+Qpb$$h3H@2gKtUdfpSdVJ!KI%p*?3z zPW!~xI~w%g$mQSY8}0x{K)AnXohT$tYPq9P|FvBHwZ8F=78tCDiZMC&mgbat4!)JT zAI&=CDXDbKUf4auQCjK=dT_?QIb#$M-x{x-<jJ-Ud|J5gV%quVWLX@g#Z=!QYA&^u zXg3B7L`ocu(CM=&5tH~bYWGHh{!b1BUWBk6WEvJYkWieD3af^X&^K&M#Ep-bqA?#y zBW?xA^cyw$aAjH$9*KgO_=PS|Vbi4f(`NabTU-6X18rrI$}!6;@`Blm!;LMuC%W3~ z#l@|i^1$FLA-s|Xr(JlOm-%FK>1&uuKcKakd(*p1gSF_@q9MhRreZi_ph)aweN8Rc zIeJuQG;o>IxnxXaj)vAX#w>JTR(^v|d!(UO&AKglQq3j9Ee;u)YEOVo1!i**S{ae8 zGIo3nmvtB{?!sj>fX4&zil7C)=TF1~{#bnE1sJaqsu9maM+6LPt+0o=fLcMkdicD= zzXDBGBoZJaL-3?7AhWPWt;Z{<CdUZ#5KGJP%@N=uTJ@nQQMC^k1yrk;E*kR+i2Vef zALm4k#8sHikk<Sb*FQM&uG<douBfb7yLJUo`E#{9)`mm*gU5^+oR4-6T75~~6Ad*( zraX5|SI0E)#D@&?Ap<83o?YD;aY`X)kr;zrlD|vt*OaM8kF835-BwoKKD?sbG-J+8 zdSX?_^;^N4uPNoZ+3O!OOM@YIw%O>)A6bUpwwBFrzN?bS9=*`PSneHh_2I(4=kmwH zsgu2)38`DgKk{NIT-i0Q0!(3`<hho(FMI8C47y<xqrP<P*z$^T)Gz!}%Ycu@lLzes zy67+LJ+gtk`&Nq~=a!X#n+gANo>IC2e22S2-b7G}cyxrm>U`g`WoIeo75t5y0#=X+ z4#q(u0VCU9K@qu;n4}O3aRD1ffSn}TyCSd<*<=>LkBMRhCPL`uCBrMD)v=%Qf!)aB zVWKt$n;OGagSCr$z`ysR?{2GYFq&D`Z;X~reKgt9l6>@ed@7Nvg4y!gNqhgg{5GIs z3_Xi|4a3nkWHEW5-LUSv-#xyuvU8X(r+sk&9@yXSRkHznXGWE-j!#pU%rS%wYJSc3 z6@T43aW7s6_33qxAT_5IWfKHigjjA%+(c`gjALL-Q&j|o(#H{aO|yvBly)g2DB9xQ zCOVcO`{@Eu3=vg`jTF-YwbY~nI`!epu0FhFOL0eK#OpRFK|)V6tz$!enNep{XaOd& zDux<HCZv#IYyTZvI-*u1Ro6<hAL#HkE+18Aa4RYdj2yzEl`58z>W5|nhM~>yJ>Fv| z*P5!8SA*Qj`h+oF-qtj|y__A{pe|7YmIX`xupoDd#*k%nL%`fT$Pg&VVJwoVdK1q= z27vr9t+B-e;gA!W0ECcMJX=j0vKtr~h!+4pLw8kUI`eq}C)|T+<e>tF>^Y)+pr{*O zJQ?61L;8a-I73{*Pf$e&vK-M~F^iycT7gnE!Ny2-Zhd`jHf@cD?fLokaP*5}F$Eqh z36Ydg3Hs3;x)+_i)9mxuimL4$veXdt;R~SkrH4V;F}Uc;Wr<gIHIf2%0>{0#1IPW0 zydx3~hoWeTBQM|X$j<{`U6^nmb2B=%x2>6`<%|xlfA4kRz85&|-27>(X4#*{KE5!p z?OWjbcH6e^MEnxTS==<Pw69@-XARIIL{bd%3<Nn#z;=+F^hnN9tX-{Bs8GeXdbdi( zo9V7I$I!9>4ZV`22CoP|Si+|%r&h`yM#s$z=P`gujIVF{9qQ~bPxs2s;U%19f5Mz- z)_HdYnY*U%33$NDz`*;azCnN1JJmAYgu(%u_DPaH^!f*<XMuKl3i6)2`(Myuqj2{Z zf3MCC8R%Z)X*HYM_tKScec#{Dn|DftIf2k0_{)jBY}Axi-4`_a@mDo*nVCp{3l=^u zd%i<k(R^P*tv-rQ?mdWJTgU<Fm2!=i8~W~wInQ1-V#HO?&Y5<o=}O{`8r9l@;gD<N ze8AtZ7dVclPG^+&P6$e9v;)0Lu(__^EVoLpl0o^DIk@`U_$q_zuX2ULM3<0w=H4)K z(dwgnP`@@`S4MyBoc@ZP@D*yP73gqn0XWZ^Y$NIuDQkfQpLIXD{sO@vOX19ch0v3~ zkPTMjdp(cuwFT3YY3(y%mc%m%5z`4^ghtw6cF}3BZeWQJZa0EF3^Hz?@CXyQH==cJ z<cVm{L*^pBBvb$|UP8F8WJlg3`FRiLS!~|Ca;*mdRUkDfr~)9Mt&N1OmcqjYx%cIm zt?ul&#zU|#S*cp4sVwkYECq)Pa_-HsTD`fY8XvA`gU?uTMW22|)M~|-%srTEwfgeQ zai&*WS?ISUIWF7kb}@3faRu0FlrxOnh_-Dv$tA$)w&ihLj>Y9-<#O}NGCH3wut&Th zi$u;iguFbP%MK-S<JG9wsGw<u5!7Cd1Lj~l(*any`KrDVD~xguI$gFr0b|H=Hpw8| z#(8P{2*Ld1oPxHmKSY%8MLXemHo?0<0ZW9jNoc=yfz^Ml&$htkyJ6b8`r2{hRxJV% z@R8NeHVk#v0*O@-JCn5f{Z{7rGu7k_zE&2NuUfYKz{=&>0l&aUkUm8X@H;{@h#RQE znA$OVVu4?13VUL_(HA3U`og>m_sVcN;-(UGp&lr>*Gl8M_4M_eI3b}@StrgV(#dmS zSbO3`Uk}+K9RMO11UL?<WU>$cnDcTFH87SgCd#+dzUhfJ1@Rt&+mPVw;h7w-qXE)6 zvv4||omk8Xv2mt%%QMfQAD@9}&%|{&xMkf$Fb5L2Hxfj9AOv$JLW&f5W{c8vXbj03 zbI7C=tKpCZC!RM}15}Kn{GttP9J5TOsJNAkml`hP94{dl#QwsRkEJdfH>&Cz2*0Ts zHSV&@9$p8(sUC>~<3?701J^waE*nTHr5;{azEZ2!t}I{oFfPJrSC(D&@MUEywcNPN z=o16!Ca#}%)ZuSkO|?+ts2P}hpeSM6SJ>ed1QUrkFcX|Tjevk~j**KJT=j?>@WSSC zT5HyXm(GE)xY&1v`7@MOT@j?}BDPD32#scdgA7I11qbrv2CGVuqxWtYWu>1g_`Z?n zYsVAZRP;9j%PPRBK5=_3ALAR($dxMj1er{3lXuGBS6CFCa=FYdn;^^5s|DbbF7<<Y zhaUgmJ%JRJ<6<_sjJ`Qykmns(U@elf;ZYmB3h+R|ca8XhwYNWUZRQ@pMQ~ro*YbQL z$V*qmSGIp89A!evGs3ld(znv<2W9S$pD2w>K-!j}4CKp$084w|1zSKMPRxLLb1-CP z0|^P2;E7SNIl=OrDUt~B0XP-7fqNmkmHp)&5VLUStgmY>-}O}teT+VieYI<A_1Cj@ zx<J#_moEEp*`-&5CiwQ-Z=;8q<V6_VGecfKLytcJ8k9#{*6!Q4w&kc2J$qt2edhJe zVf5!gq0k_Z6Q%^;b!q?58JmKkg4YnT!*B=nD0PI)4jIqYX;<zS#KEuR2Qo@czF#Z( z>-nBo3Cjq;4%G}^0bPvlf+D(p$Du&<5-GZhJQswu7fnt*?+8K|w8OLiO)Zd2A+!-~ zOd(ygecNL|1*(Da(6;ud?p&Fm9VP9-6a6~y1H6l(B^OKG5wvgEU=ODLiz?tMm3$5a zGvz8>Nz1U-@<5=xby!OY8hft9D11qL;eNSa8W+JJXz!GzalrcLC7vJ}5kX%jK@cTG z%%C6IjqMM?-k>dLLwG_y#aZCL2)wNr#WVRm7Ow9&fjR<N1nf5xVtP|JKP4;`kfP_B zQ?;<SFNxBN0$wC2Gf1S}os*+T=L*m4BowECBicet2;c=OsILKLKm?JTaF+Z}Ab=L_ z>bVnD97eky2lLhz-r2JYTo;_z96;Tlf$M|wn2O-sAnL|t3fBrn4uh9Snd<}1^KsqJ zz;yvZ_HR9_l>Afh+h?T81+PQ{Q4lWT>(a$y>LxD0d&bQX7p!LSsMm|ucL`b$`=|XS z@PhLN7ci&S0HZDuH_>y~Ke`_O2S2Xs9KU}3_|A17*A72(&&Z1034<p@$2&&DM!STu z`M5=Fjkz&6I6Abd`Q=q-M$NBZHm7Y21Ed}fFk^?$Synf9<e630oDid-CpR#kHcU2# za%jiM6)Q%LTE2W#`iy>tw~QUyI59QF>@{g{P2iBwR@(%Enomm<TC>}-b2j?>p~b$e z!sueq1fUe42bV+&v;0dA0sHKoff75E)9{HQvt|uRHEZl8q|IjF^>A-mPD}74aL*Fl ziRt(RvB5VcfDU*#B7WuR<n_>f{q?CcV?fh!Of<Klr;i^;OBOP1s0}<yekm!U?PvaZ z{5W%8p8~bv^X0%J_Tlko0ec^t!kl0(p0(PjnPjGdq^SiWO5}nC0MC#_0~o=sF^J=M zF(M)^Ko-DIkQTr>(|#TZ=7r$o#!tSWp2blXPuda@ZB<aI!Nq76u*y*UtWU)NmZ{tg zDm2Abi+vYeyb^t>^YKbns?YJMo*kSw%50^}xO<}koBF;&HLLR#f#t8aNgb(9wxYZg zT`sj}gVyq}j1IzEXr~6f++YFb0=3HpnlFpU9D$-;lH=>q`>HIdY;umqs8q|FA8X<j zGI{W1LK7MXz1B#}<BhP@srSlq(UJI{4Qitt7+Z}IW3CQ_>g}8fj+kZ8je}!+_S{Jt zxlf<^{i`8^yhS60m>?+(gPHf&OL(36gEGOsUzFn{&$E57Q$9?$5}!5r>j_kzPJnrg zo%bU&tguPw(HXe&ARRn0hC)P=pAsxJSPEgH>D&(!dBKvPBzc-ru&-m9uDktIvb`Hn zq|#YT-O-d#kLs7l3%|Zvx>p1eW@^v$dfY+gy)%NYDpQ-pRdXm6_h$ib!Hws(5tuGZ zk6NQ4;l<2K+KMJY^!)@NFaiI{=OxaF1@arOEkZhvDHt41<AR+_XAI#Tmvvk=2ez!= zxPBO|-z{(Z;vZkM$#+v{70N6AwP*DG>t~ch-7fiNuo5J}%FXg!NTGNPtw*J3{bLG+ zZnyjy$Uqxpo{{fX-C)Sd%gZvXjo`msdX>C&+_+Y`O1}$erE{m}RafWj(ktbgckI|K zSK>sC?ACqzZk3UOPrvcT)1)BLf)ng!gni6`QmGnh7&VfbPR*y*;K6x;PdMtoJQHk4 z5!EgdADA`}>rOjB2YVom3zEZ#UIchuI3e*w4;vV}Xd*qVWljtJk23W$=6EbV3Q4cG zl$;hM=PW+P=83h*fAG3+Laz^uT{JP31m~pp@T{2CE5K5V{06#9NTaFK6e%YmN8%Ch zEX95$A-H;jgnba`@e!Cj0v{k4L6MEg3Lv<@5hf6#WFfkAGWbH638aN4N@O(BF;V)J z-ZU0@^Q=LZNkBGaJ!7=cGN0ZrV}qNv%zmhQR?MORG{X$Psi6JC#aDNB&d|e=K!J{% zob<ogd?y^^U0*bRF}-@!O4R7F@#uiY=5ZNIPjr7&8i3}|+$p=8Y9utED;}G_;nR*V z1fWZr`cdvNvA=T$ZG85AGKYuKYptBp;~6|mxGFQndmtUe%%6mVbUdNw^&~|Gv^4Q= z6BKn1^x&|E2l6gdzy52b-azj!ZW;bam^t?G=OEXIs<IKq%$eDM-nWimcrBQ1^s!6n zE*qv@XMw|@MC(Jhyz)GV1IGs^(Lr!6I+EhiVt5lBO7)^~Xe}&)3Xl)%TC2@~W{a{B znP@knp)8aG^EiSZdf<x_31Y&`(ztJOOh9l8#76-g<$bJRDMG*<jG~G#XNsDLy0^HA zA9sTQX5r{)B5E>6FYLwKlUJ!rXhumZPj4(&)S~YpNC3?pI@|IgTOR^!;J};%aL=Ij zHG2WrQ538UjcGEOn-^`o6<$-ES6t8(*MQz+o$1F1eebfGo0BaiKMUPSijUA6*e;W2 z$rCF<D6Sel5jBr*TH!<xZb+?z8@2Z3lV=Sv`;D$FkK#HnSUF=}qt$CPd4N@VOFVU` z%hXIie@3_%EA&lEf7=jiiCij5)nZP69G(egQ;Vt9)L!Z->J{n}>J(4_D{j+D&<w_b zHQ;Kp3!QP!ftkY~)F&iRZ;tC>$fSpyu%{jq_SHZ%<}*f(6);A8OB<sxb2LF4b3v>E z7^9&`G!ZW;1m0X6iADV-{X%_z#O!0lxfsXd>5$j#4S9otGzCwy#gUkx+FEQjnv9%- z_>1>R0#PE#@^Yg0V|>+;Xv7JGlhGU{P)r#%y9VGp2T6uGA@2MN`{<Y*CC?uhHhGD+ zWFNi^c?IG~J7eWV@<QZJ(Q|jfS23kCL!wEMaNP@UdR~QeW+spG`pA|K*$prVo0AE` z7$=BuH5el86#NCg&y?g%v!A_U#E2`7%(*GKY4Vgi*G-(Te*J`rlQvG7yeWCpoFgRf z+1WS2nVgQJoeV3Hu(X6>rI4lxD2nh00UqpUOeS7$GU<76S0&p7wwf?~!|P9*{bsX& zE76%G<;b2pV4zS5g40J_PHUD%?Y3xKE|1IUaUF0vbvEK?#G!e#P;IuF4N8;8<|<TK z+*X6i>T!BDN>wVpsL17T6dGqbgCUp4q}Cg~+)V!_v(n{q%B3=yKIC!oYQ0WxHtTt< z+TidUb-6TlXDH-!sJEDvPA4fQUGH>iN<$%sQ{6^1h9RLyAwx5e#Dpg#Pd$6!0AlVR zjhkvVX_nFRK^3SRIUOBC?@pf%@<9HY`RE1o!aP!9&TL$w?>J5C3@VjDqf((VNXuD3 zT0zC;1ua%RZyB5A76Vqlm7JV_5uO5y?L(Aq$ur=G7>)BR7K3){Fu#8o`876Z4dLpr z!Qz!bMy^p<)E0w>1a)e&&Z4$*rYd`Ow!JE{J?zd3@g|K&nH9qITYQXz!4<U@6|e*? zN{jaD%%xUmvafDjJ=l<4nQfrS7b7hKg{L5Y<eNi5ml`mi44eu?|MbFKAjh)-+4-xd z#!}O$d3YwcjJk@tS@3d55Mc#}-v%MT5<_@h#yLEjmLbQ_4JpA<sEBbbj4DBvoL2OS zWd;S%seux*o^$j<K=4G*C&=ha|3FmGydt-Y{X%L8%KXksovzB^mwa8z8){CCoH2aL zl;-?OS6v;b%}zLtcB9kc%`VHU^cMuAyu(yv&dD1btp%n)c5zKjL2fV{Zk|1>IfwbF zZXbFP-HQweNj$b--vje@&6~Fi!0QHgjvu`J?Wa~OUAp2au(f?|OLghgIvMb^CVrMC zT3Zv`&xuy}Q`BR7-|kkG%v{nu2|X5!jt8y(3g;Q*dbQSQ&kH2NzHF^ZqBI%odEwfs z?AAbCq^Kd-YM8lWX6i|(36I;c;hLf#e39IAo)nBZaRS{ZEA1?8E<=x9qiriJL62>L z{xizbwzg8{dweA1xW50}K}?aWF(2x{^mq_+qr<5Q)KThhcm`*I4ER9}m_|{2Gz1c4 zGRE^-z#KD|km)xP5KllnvC$B5>dyH>MqkLs`FOm_Ma>CdP&3{jo)AMECiKk-T+Qgy zMUCRc`i;1BcwsaPb3G>e6A`i(<AU)_qRwXwk^jO}NY-8<#`W{KLI(bMjQDetA1Bx# z;kvk|k%dCUNl%c6Hb^)ZaAFRhbPe(-5kW*Jq?N=AXd%%|c7+B!CP8o$J1{}KD-cno zjYNiTin~C<D9PkW1bIX{&jn~*ra8Q^5}fJ{su=<MA|bR+Y{mF|Sl8Ti=l1QOYWlO` zQ6sZ~C#PdXF8Yid!RzDm`b_nc%O-EOZnZXD+I7ji(XDWgLg%oAEZMSk?IX`pfNAe| zYa#mJ`0+hA-pIZy_N_(Mg!Mgsu>m^ea$q*sW{;LxORazRK5@u;*nDbG_@JdYbxm&W z%cgtV#BR7U>Utz$MlZTc-!V6S7LTAi!PrE}F=K`ML8+91x-$1Ym8pD-$*Qljcn8(p zTvU!ew;FA_I)Is0v%abJree&O{PnN9Z@dwGSr31jwQil)TO9G0gg376`-+QwUs-A| zyUb$^<OZy<@P_67^5Hj5T)o+LxpmlvEj3(@#;w#z?0)COsSBqsp~oypT|RtxaU$M! z$?;0hps=$FeCw<XE+|Cbyo>)TD}e@`1>mWtQtujE1{DXvgw9T&89%<S6lltDz0^CO zB!&-9zynoP1qFqL(}8-7S*ZlDs;ayktj@!KCj2*bYF^%BU@Iw^>NKVQ%FEH^6&2%E zv!*lBu@=i2b66(xI^+2s<8+{LfqN`C?s3IrK8;DvO#>R>OkIlaT8i%q??vALP3qDy z<yNhn3;IJjWmc^WNCN7azH+Nd30OtIMG_5+1lFT(Uw^%_a?4c}cy^WRoP03+C;T<3 zBS*;iRP>Ke1?IYZcwCO8E}^zi`=|%0!_*(r-l)?1M7T@)IKmMS#D{_D0_X@wO9<u& z(PtB+WGo*Sk_T9C<1;Qn3B@%aBtVsr-fI>!65uyq$spF?V<kd)PJlZ}5Yuq+0ZkXq z!|>B+!0C$w906K~nN=NB=uI{Ym=g6n{Ur7DJ+0L}Jgfs!Ns9sMfl{wE(PO58ST;#f z)Aq(8GY6GBD)o$N5D%W0vaJekULLC(#!5r^phJbD)LF2uwR)dHxJZYR`Q=4ygUChj zdO$AnfvQ;{6s_mssiABRo=KpB5Bs?#=h4;61I1a6K-9A`#|7pq7~{SEh!Edi5#!Mu ziJZSgDyQMpzX4Vv_k<S=t!T0slrk+a%4PDsyR<SEF4@d<OI32vU7D_<y7ZS-fR@pM zgFvl7U67UWfcKg3?abh#^Oj<=$I`zRYp4h{1h2#ss0Gxe)Ya52;yR2MR6_l-zPPXo z3+rxr+2zuR+=P}53d?RH4idhYhK{DqC`E-XR#<@X<g1GkTuEM7!38VVP#R4#agKo( zG9>Bx0{I&ZMSp?GDXB8@9<$!*C<9MiB8fy#eNo@&&kB~;>l->+3ySI*Lhd4Ghg(0S zYeZ2LGh1C7^aZ-=yx`ER!YpMDxKg9aDwNAN?Xs0>3wP~;m*j^B*T$rqclonMMypU> zL483%J^gS|WOCP{n#8=B722}Fxdt=)Gd!P5S~V!(lbvvlnf7T#omFL0+dSP_!BA6q zokeZdx~=-f*@0}}TeQ`<L^}I_M)_Te<y_<?>(z9Ys}yB}h#Nfw{_^4KvXaum)Eet< zMQI&)k=(fueZIJ+cJq>CWg<M=&GUt7_{e!5j;7R@C;nDK3FDj~E9uRofyjPvh>es8 zW0|Znz(in52pU_Q_@}C7h#QH_<`Z7L%tX~*VygPGr3BUPdUq!PlvZ0YI%_r)l>+(C z56kV+Q8@54AL$rZ75eNsX=!_@bnSC7a0kwT2hrYFOIqgb+Bxr`tkD%(?a<khTDe_z z>OLuyci{rJXL)lb-f-WySMLF=gEtWUdIPWDFbT}Z1w?zcbMIlobVM8373zQZs0^fC zGipKq+a)|fI-w`l1HbxWjQA=;Q$NuQa~|I^>88#irZ@AVJK+xpsuop&hEc!zq7SEE z4tx%O9=EJ!+JY!bqFV9AH#`HhQ_)`Lp03~e;{6!MY_ea@l^~i!#CM@Eh3Z7Kr(cT$ z4;~sG3CCvq3W@{7m+=9S5chH1#M29;E)LT)Fq}F8dW$$YdO^<7i}dO)(Sd^?a0Ia? zO<Ne$hTAkvAg0#$KtYs~5~}`(F#YdkG6-cmak#H#<f>&O>8FI-+#M(>3EZt8fMuK~ zXgU&I1OhokiI6U|lTc3Hs)5>48L=AtPdX^fx}i%~mA#3+1lrfVBWHJ%YL{y_4Y}r# zC$~3VBa^I<$oqaxM+F>R7-`GJ<L0f`fmUYE9+JOMt8L0Hesag|mx^+lIGK#j{WUF? zfpT~?@zW~21KGq7Z1Y*7Y~mGXw)EDoUO=0y5j}C5O?bs5WpSt0QXa4Hu4IlRa2f)h z;sCgEihg7M<eJsfs^(Aa>KP47n%7)2Ou}<h3LR|8zR9;gviWaY#$FMG3Ls;bvr-nW zWUX0_@^SS=Xg_*CqX~cV5rs{fEd@|M>&zCxkDuV54~zr%z*7rWS1mX&wR`oJS9FUG zPK!bi^F->${qDhAf&7-iwS1{WsbCeUn=O`*4ah=O%iA#ZKQYrp*U6xwSgBOWMs|`* zf>Pi(x*Cn^*V_{I^?YPck1}bAO^`tYh&-Qo1Ytuw@rs!i+7o{lG7thrN#l{pAJ37? z|0uV~=ceuo#9lv3)g}XQ!dx+J&PS8_UV^o~sa^?n1pPGWqd7S7k8+`GvKCOU$Aq#% z+MJIkpRN_k_NMj7kRXT5PW$NKsLWnFhzpJzOq7pk+7eylL^UHB-ZVEK9ojN=)w;(g z!gUpWPlvXS1PuD&FKeD#TFy0=R%^1=*1G0db0pNHrkZi7tJh38ygoS!HpI{T*s{Ph z_)qBjNq4-loQ;IMf%-`me$9FE(ENThJprLQB4B8W5SK72#31Q5f|trPV6hAGMxui$ zV#jg<gxAF*gp(nfW}g0kB1a=?Q!%k~GR0p;IO-IY-yU?ha($Mt$>j967v#75T}E@r z;>&e8g6*ARrdNpMr_1CQwELYVQ<#+bWfdV8*XeGrC4Ldaf3@x1XQ&~iv0=Q!>)?Z( z@IOY9M5yDiTkIyambcm*POFvIs!ce-A*2c+P}?i!I&5O@1qE$ZyQ#Om8}y>u%&(i) zwvHSYbLLsH+~vU=TmEB29P@&_iY0Wo$4I<DV749qq}0z8o(=$YDhYaFEe@VeGE`p@ z;=Y&a?e~q+E3AWByb@BF{1?1F;aM%pdx#$p{QVg{O+oKG^vSL7i1W=m&(Q431^ZLf zJ9yrKm#02lurf0XrKnrcC-dMPG=(R)^PbI}c{Z|Skda1)TSypT#Q2UZ8N%~&R7krj zwG|JPPf=Gd97+P<?NwTx5j>{Wi|=p(wHkFosZ1fUOh}*hx5QD*SgMOqk_5My5p{+o zA>v)RAGAcY5y5L06xE@L6BH3`TOxqE5-F$81<hPABoLCUQZ9Fu_yb_jv7Hrj+gc)J zu3|-2#kiFnT@TLs)2)w;D7BB$IKd`#_J$v>7<>IIbH`pcdu(|{PPwh?$`MP0H63He zHJ2*rhZePsE&@uEi`igvn4626=vs--nQd3eCw#Nx_ksA7_VvRrcZ`@jF1+Z`uAZ-^ z)Wr69{b0{+0PL9i+U|+L>S;4BU%Dgy>eTj}$}G1zzhZ8aR(HvMhBoIY?D_2UVk0ot zpSKo_6=e2A_b^nF*}n3bFex1p@kk5;@-1HYOoHMnOWMe66zBd#KXkD$%(>`AaO(Gb z=JSVT3@rA?b-=(+3duc#qU~#;cIpggIARAQE2cJ?%R+;OCr8eFVjj&*dT`;>lMIT= zoF(Iz?%6-5`_clb&y?*?l(yu|-!tbtKL#fssF$k(4yaN<f1yU>9~_rE4NKcOZPz%b zRO86DvE@zI74Dq1Vn}iKQ!~JVCl+5~w=8TQ^5C+$_sm~moKilatTAN28h&!V!2_L^ z@roFtQR;lpyMD5<?;$EooCinsT0DahLkBJHB_2EtGRd?at9YUXtyr{f#no4@02}8# zd(}H+ZA(#^RgNYhXgOCQbLvFjqc_m^og=OScV9ps46rAp1hjn00>rz+^wR*QU#%ar zzWw)^)qij1(ev&IQ2Npt8shr%9!8k|iHZk45$j6}rj7_I7yiyQL=+;?lCcqrVlp3i zIFp$XK>3O7f#460&<$C53dtfq$`T>6jFNtXQwYx{xTlTc(H}~O2;f>Y0#Bot!#>NA zx*?m79NE0|;X9w!mx09~3uR58Yh>9Yn=7jx)W}U5qfh_fq$5BID$yyl9i1B9REPHI zJujL2?m3K30q*dUnO6#`l^_Wo8~vfE80j$p#e|uML9!|9jQa@s`N;KOjjp*7Bsb6A z`67@Wv7kP4iCW<V%#@9rj{xDi_)1eBsF{XxZTsH`99ROl#FN32V3LUD9jH5Rpj|a5 z^qgFRr$=2NPv$dVRJ25p1;g|F2>UL?x6+jm$tN)vGxHhwFeA!tokLikxo@7?#|~kG zE+*&-{?lPdB@GUT0VWOLASs-p@F8iPEqesm!5CnFL^jt96a(bHPzjP|r_+p*u7U!1 zN!Z~CJ5m!;cO_%PhQ*TN5l-k{1YT}iURk-k4VBLl)`cr@-}@P_3k3vQfD(ti@a-@U zE#g>3Jp=_xFeC7Yf-H}TA(Amb7z0s>68C|SIDb?Cf#CEL=pa0ouun$(sd|4T;)l=q zfz;fWL&Eem<IsB#Jc5orWTyX268bJ32SN{Q{`SE+U}?m^Nw?{J^w!-!qWROGCAph% z92DR41K2tJh(7}V-V}{CMaPU874i8a*WU9GdiRk`c=kizfX%+h=+Sqhw?5be<i{g^ zAqI*+*o1yO9wBxWinA2+L;Bm=asOW?V2aabeGK!4f~|A`el>!nWF`=M5?XLhO@vou zU6Igfkycz+Lab5z;zoswNkjzrBoUGvj}s$K4u&MYwCgoY<eAb7kFZ^F&QMrX73WRn zYE}>%(nLudifI0jKD=bvUBNPRjf)O=l{r52=007PrgGJ=BHl23_GYizoTUnu)jJK* z+pHC*ZvFc$d+>KEMSoZtP%3j9$Byf8YB`Hm!#EnNvTDZ%Xy!_p)B{JvJMQ(ANLx#l z&WD`2@g<`tJ62aYv+wL^+w{ByN(!<Ys)q*#ckHVg9#l`huW!$y?w#*3m!qeQ;Dv_L zzV3Eujumnoy=592FK5A&_kDdo_zzrTQ(M}y+NagV(XH7v*YCgYUI258TJ*&+1?%#b zICzGEJnwhdOqPNx2Dz8Ilu9T+3ORLb=-v@tPpxu}PU-z{d}8KWTeGWrwFVrW+deb( zj839mf64VThfEu~05xf<Cr_%~h5DYmB&Q6h&iVTz>z|E^3pnu<hz?xLeCgGXEOdeY zm=h0>%_kTNda?+Jyzm8ye-9Jm$s%Cy)quw|EUkM>eecFQ4nKX(jrXWtXRD%RHF8@# zGzI?osQR8v`WsAjgrvtp#R;&`oiEWi;F#2{scT2GR-Gi@<;s`n&5}H@74UG{Sk|Ir z3tYWFQ&4-`<Ntc~`OhEhscl-dtY}Ds|LO}9TAq08)hek{4{K@|jY?Yi$~$krQYBTZ zSx;6SqgKl*U#4F_O{T5Py25thc5zY_qQ+wyAU!z1qoXj=1&_mOkXh_Ao*m<v2z^5E zE!oqJz7_(QVSjt;$t@tc<>XdWMB+FRXuEra0DT?O3T3|T?m3erAr`acTTcET=Ds_y zi6i@eXNy+77h9HP$+9F@xyX`igJs#6Vr;;eX1eL7n@)g$=p;ZwPk=zU5K;&!<eT0w zxl2L{9LYfr5|TK%H13jIzPU>dY-#w-%u2RwxZHj3`~Bkw*6!@=?Ci|!%$qlF-upaI z6WM{D(kdBY5l<ayVOj(m5H`UEBuF%PsGi!foPbr$#{sKRT>RFpuAIJ3MICZ4hPU2> zqe)9idMC+ZL5CD*tn_WHwpgmy`6>+o#JW#NvKahEOVT97-3JWxpei4{=Bq-%w2D){ zs?}SXI?gw3+0w)oG;N`uTZnVP2iWebEH19}wHu9J<G8Oo&r1*ZDl2_(y20>Fb|rnN z>*+0tz6)tIHDfJ8dkV1Q|B{>R3U|Ygc3%Yn_zD~VUjYHIhMskNX(Y7t`0=Go>(b-k zb=n=d2XX%tD5D?hia(CKgQ*jbaS%0vnnX2IbE$>Ya<mHEK;1#@r1nwwQAel;sfVfK z)RWW;)XUUs)M@Hn0qd9{7NJaR7t}7oKY^|>#Nd_@&<}LQI7%0zZFWEY39u77f}@L$ zsA3L)?f?>N3TWIS9@tGzlqZG(<sAVAm_R^h2Y$|rRyZajV-Elo@cMY8MvosdX#yG( z$OpV8;1)R@oXH@eoo)@sdx1%pPorMLxZD!j$LJh7z*hj`j!p(lelca%pJkHurx|-v zvKqE%iZyV7#%xMwPwSJJbNb{I{aYaYeKy@>)`D$nzZ%@7#dm*ivhgqLk|S=g5gxxA z9tX|Z?8sO^pI5!|vO-Ni0$068XTxvRx%88O4QZ^#2)tAQmZ>Y@2rx(-Y2m;~xRpht zWLF5jd+7AhM_3?!%(@?BefAl9_LPWOrjG8u2>*z_XJ&Ne7VvfU2;lr-0|SiWOPm<i z0)M3SDEh~i7p%+tJ<U9}b`VFeHKeidK8Zw+PEU%(rgW`dueT?O<zNV#rk6;-R4z@A z(tt+5f(#XyEfpu(4d|aeNhF5{fZeE2!dKX|$XPHM9#%6JRwCu#y`s0k*w@80XgzaA z-7#FmRS%g7r0{GA?yvwqe3Vt8)F(w8SY8DxANu6glVZjPXrNY{1PU2^`w%Gm?o%<m zHt=m)-u8J8d;^{aUe$K^<tMpdO9zb_!hqTfEC(zsgfD_$K7(IJ?tou`lt2BcADAg+ z(Pz$#-q*kg@)Cmok1_8k()~LJ1Nznpig?ejr@sbG&##aCef{(7`KLl2Xz_$9^O(l< zr@-{~stvEbwjtvg@F>PGhk8#Rf!?e~<L!1zN<4dGc??T2m~laT3y0WVbb%I`KubA= z3l1a2`Kf@2H!=^w-@^1|`m><vd;^#|zCT<UePjOkU<>VsM;Fl=FeOt7ufWi<8O-lb zKe74XTrluGLwzMT>o%AQPmdmT9!xrWXXTg$(bI6{fH7blUDnYXOr`Zp$IVy{gYaXe zzNm7z=`5(7ckhNLW3)j`vHu{tznGHi1TQ~iha?B+{D{r=du>>`lZnSOc%h3J8NoRn zPrO5!{3d?d!S$=poc?0Zo-a<H39sS~wgLL;fdf|o4gY)XxkIZSd!udNzP2|WTQ&2x zr^dW<;>1sZKkT{p)2EIsT=o8v_m7=;hh5$wE*-mP&)8D-+L~FjIvy&mWTJz&Zyy|C za&jGW=A<)Q*?SIFMTU8crqAXCKKdA%o5yzATa5dk%b{<&?gCg%Kw2TR#R|A9R{eOr zl^o!gR{b;_MhAH1)?seTcMo-BJoMe_nbO}Zm_9fUWWTyMvRk?N#4-94gVkz?I&eZ- zhmX<k@ew?Zk8l_X&m!S{==jh>-+lMc;x~%Y-3xxx=lMVHj_j=}v42cqZAt1zP$byS z2!7fO#8aD{_-f0e3Mn5|N|jTUR9~tF(dD6tGLNRlBkDYZnoZ587E#Nnm54%bL=<{E zqS1S){nRn)A{r4`^y4H)pWT41*GxTs0TZA2!!C&ue*oix{mKvD_ZkBK<gN+pvl#eW z{G4$Pi-6x)X}_S3=IGR?10t7z^=~wJ10E3;UW=&6wWyf8ul)?Ymp<BlW~ej%uR=9b zm4<#+a&A@)nW>t&9Q|&Kog<YCfqoG%ECTw4<S!QPc)_{ofu8X{4qnumu`{}5?F%og zUHAO+@Upvib#HgeBr}uNXJTrg#1W_;63EIlq}QkrH=ZG}D$PcDYQSd~QSCe66QH4z zIUJb`KjJdqej9$cXwja8zq`}(?aFjdI{qo_p7eB&J%55H-GhJQ^V2=-`x!m{!4pFk zb&pvzB%@oN=+E)|I-#`S_KYkE2Xux)Z$-A$n49Lw=YdRbND>)MWkAKq7!fTs<;DFA zEJEXNJHdO%?y-iwm2qCojVxv~Cf?t6_;4Eo54YWae;a74$h&qauc9IkJeeD!e+uP- zC-W-67JTn8PS~>GFk908N^V6(E?13@zxfS1#`w@oM87Vh^B6?ExH#Mq-?cwa1kD&9 zkQKZ{P>B#pG0g#=u*nfuWfvasbNc|h=Yx+9k2tVmVe^cI%kLd_;J4@RpL%HoXS0Zv zhThZQ&ucb*z8R#PTYmBI&W)RnjhVi2?L_MgjXq8D$NS4>mluguhU8vPO*jSFQs%|? z-q>~M{lK{88#XQ<7kGaEp_gjQ*;JiDndEDnv-rbJXMuXu)`uV2I%?&#iD9QzuN|zv z|GYETX;A4>`qXs1=1f(^cv<X!9NQA)V2miD!a#gdrUIfW)(a4j5sXC=$FgT*XC3jd zK>P}zj}RwyK@ec#G8HR}m*FgS(2J!O#D^~lM86hv$OTpMcWucX-vORWV(!IBB9z%> zbkZl^6T~L!WR;BN0ejNyV!G#o1JOjqa;6nhNls=3pPD397hsG&v(j75G657+Xw!^N z-qnR`kLxYy;|~*hn<}nGPduQRfUzh5{?j^hl&e^`8@+ZnVls7r!qC`MboYN;Yuzs3 z#5dr_yL2e$8<Y;`m;OJmyWmGd;NP(~PC2nPL$`Di#_rbHCA8gXX9z>@6t>KXXAg{1 zU@y8r&xaSlRWLr<?~eCiJ8Wa3DCX73>-6#W;1BeCFb1~4b}$-*m9#n%(w1o>AvLW8 zVXd7F+Zif4gWeyBFf8%65&4GRPXZu39a7qSO@z|xSxS?yr73L3i7Lr|kLIEp>K?@D zQydn{^KJq~{p*K-U>y5T56;9y8U}BhYrNRar~yNOVjm5RrYrTodL=M8IUk;8cpdu4 z;W5L8Y5m$^!%+C29&n;xyFaWwFCkUv1C8E#GAwKZg-=@bnh$h|IsNMEKnP$HABg&k zkfH9M{eI={ZTN0OgHG2F0!~n7E|->p9Bdp8FP2Hm&G1e5u@>EI_|;5UvjDjnAAelj zmrEaNDMi_Js3mnO0Afxc(__9M1vico?0_0;XE7)s77U|1#~u@KdoiIEh%LrvF%}V! z7C?Ypjl7q)GIXe^2{%Nz2~adG9ocUZZ{a8P8!07vx-#^~$T@{fqctfqJUXdDCYLFs zI!}heq}9k2oSc!7RN#SKw?+2dwo8)g8R{GJp^<+515MuyTds9Z?>W|7TSi~a2e0!f zA2w8s&Q^oga0r`7g~D_ZON(_htrOF%R>JT+YZsfvdS1@5$&U2o<a9?e%gT8@inOyl z&qyLiZ@cr(+sYT-F>jLjN+=}PXO@&^2X|yUgF$EZj$n3aN#@WYpWD|QxjVLR5Jj}C z4son4*xE%&W2*`m*(f0*P)CB`+tq0kZlz6jFP4M`$X+|{?lGYRV%1G}uL*Im0lVNL zorv2rf&V5MyErPZUib2h-+Zr@4;j+GX`VCX2GzGy3|?24wDMVE4i+A~X-aM?O)VPn zsnx}?uB514-*2HVWg5QuUyIi7xci-J7ZyEbf^RzXTFvhK+zqe1!i9nOmF_Zk@b?*~ zw$$;mFOSTBtN-l!FW05G<C5`|ym^S!r4*$a!+BPd)iQIT4!zdwI#H2gz8vk?IX}&o zhE{Y~@cv3qaeK_a5f|p#5RW?q&xCfplD2th0Z+q-m28wOZ)LG3bONkFOc)tKR+Xr6 z(y5mhn=!cX%CkSi_MgwLtXjOhzd4uX(#&N|rI{+g6ii-s>cXjYlM5K2$}DXvGpBKE zuDSp6#Z@ruGKT~cC)9eiJ`ncRHW6P}71PSo(#oe*6b|t_<R~kYp)@7Qq-JGl4y#^o z%}A58$|O^Y6y=2SlRNxFm?0256blRc5%}k;tp$$=KC$RdpEW5&d&-U>`~(b3w;g@| z6d?F=(V2_@&3PD@R>aHDjDU9&>@kc;+7x840G$GboRnpvJGI5y=nhT|78o5|zt=?R zMnk%2SBaK(&wzK&7dv!$vbDbxIdapv#c=ct*cMznzdj?Qe*W5E8>A_bgkhtPXtneh zTAN}3$<b+gvsn+mYH@%QI}d)M(bHTOY@naen^O$1;NZ?H*K%q?85yCPoNJheqi;}b zn2QkG8VcwsUgM4T>P|sjC*H2c18CxXmepq9y(08u!|?Luwl2^ZA-L~vYvr=7pKm-4 zvY&`hLXX3HKTPW<@I};@5|Rq)M6CJ=pgp+h>s>0{F8F7yu$zOQO56v<wt(GzLK1Jt zbh?Ko8^9(I1H$nNWuw$wtJy_f0N1S^d+_3)FCH9=PD|ds53GhAL|8c^UNm@}Lth&T zYD*%x_P%$Q2&wTytHElLO?$YjE}gn4ET+xng6*8vfSS<oD^2KgQAFbmQ5>wYW5ra1 zP!e7gFEkU}c@j0MfY?A@D+DjY%O`gps}SileGTH=*6&(##i`{Qov0%EU{@vB-wl9& zc^J3yhJ;5+a6=O4|H;F^FrewAIz>Ng-MU%&6!poDD+yI1{ejFiRn$Pd=Nwabk5>bO z$Nh`?;V$B*FcEO#@g1)eOJSS&_}5r{tNQKz+d8=#*xp@wrIEU^NvVx)PWU#cv!Jg- zy3D2<hu^++1yLu`9DEPtn3PrhpjO$@l=!viPTSJi7(L`asT=-w0XN?XIc(@AZI4d| zflj1izpZ;CaR0gy&_`ZfJWXPj7D$0)=6rGo5g{ACxpCIW*o`rzhrzDH|6wcPZt(VD z-+U<$=ZaILi&{33J|8dF><O_OQ%Y8^zNr<l?1#qsPF$2Kp@LK`)j)P;jI`)hq{IGC zhzRZ>Xx21RXp(e`)Jzd!NL*y%1sW`q(|{rrM)N0OOGHq<_HX+VC<df@kQ)tfpFQ{3 zW9QBScVstWU0N5-IB;ObA~Lhm*Y8-i><&8gBCf@Y<OcKCgYbtZ)ZIwY%>?Nj$kQ1X zEi&lfAENK92Xof1hkM{JrN_Q#d$?3+a>S6csv$#EFalz<q4eRlFOHRBz75<E@wmVB zqw1*%)SO%OImAATC|;D|_Amiq>U4JMVRrAFrr3Z2#e`8Y1%Xp<G@(|b0(iFz(M1Xy z7LrbE1)^>}t**kD27h|~19-I0lJmRk#gaR}*u3=P(WL(*rt6jd+%6IcDfWSn&|f6{ z=`jW<-}Qa688sx+iW(3_z@JbA+mzVXCjJn94o1wWADt4-IQr?b&41pj62@RCG1b6{ zl0_&E9?`p!+aD%}M<S+F>j$91xqKJA9^nxegkmgdAHdTn2DPCmwy!Y|wc$9b`B&Ny z^_hQ*FcEhnLQ|5yM_9dpOO1P9XP;A}E*I|6gf{q(XFq#s$<~|3?7{1|o05UzrM8!L zJ@IyIR8nCK6@aR<WZHoP)0Vu*lnR}XTy`I)sO_GCk(BgRgH<;VR?Ukb{ehQ(Dm8u8 zBr+PmiFHFTA9ugfhe7MZrIF6A2SX2+cGDf|MUdkY=S;p+S(5KQdj`5RCjaR?dU2aX z;z2}c+u-i=>EIJW{E3UdKCgbbO=?C7CEJH|pI--`5aLf<{3r7)eS;<QY$@^QXrv5! znkC!1T7SUT^^ti2)JdpDyq;a%f{xXP4qpcDC*X(2H-5fVhA9i#{5&#FD8xHxE%=U5 zpK-#C>s_^BRwcm~KY1Abd6!PL>+4Mif%XZt@Y#-y6P|fnr+Zt-XxuS!qa)mX9zrWR zKFqF;*M*><3#CpVmm&)5@d@0P(d6~TH$m-jFsk^s;pggf@FPizBu^@R5q=b-@&BZZ z!1bb3nuij1gu1Fk&qWo69|<>J6sRDYhn@i0o$Vt;z9_sU^8HQoD)}~8J|ysvoj`CD zUJ)Rcx04OP>>?=%dO_^tNBM--B@ANpKB5yo70*<$UJ`w`$2$>$4YL?e7=yRRm{F>; zJ7X;`3SRHzBR6;TR&)Xhb0+QUibp3Z0f#Lk!Pln78^DUM-T+Z0!~nxyO($^NV~(OC z2fXbq>sR^JD=HRkIeO+y)Q;o0aFL_^xTA<3_U)dM67YM;kzJ2{8+{zz80jdYV(;QG zeXGMeVR&7@8i~`<beCHx!p2ePrP@H3*j>;CXNl010GkWDwjQQ-!-+R%90uy+u7;&2 zW>jxVm1fAS#_S@eQliQk!`qtc%c~p5gaQ*P3R4sxKXnHFJvlYmYNS=(Avs3ou{o#i zYA)Ugk2Jk-eC?o6iF<d>l$?f|B2IcJZQNI2jJ2|P*sh_$s`g;Tu%eO8OJ?Rjei}yK z%55mfkyyqss)pHf<8tX0sO>hP^+XUOmQVsR3DG?#>+FEwj?7535do<X!5KmkOd`%L zA24d;Fk~3-PDhlFnD@+&a<oQ0FTZr-<Q;Vb0MkFoB1+<QDzUhzFPC0jGEpv7wXC`N z$keHiT!rYge8^9S5m^VOca|@I2VUL(+v9NJmO0Dcp^F+NoYbD<_675HwagwR9^1#r zC=B{7yrD+Qi)%u@0v-;DWr9v?-aU48P%}3jX9CZ`tM4qIunFva`Vae2&cv@v41Y|( z3(^r8u^Y7lTCoJ|KD!3}I!%=l!rD%p5#o=A{IH5Ig?}PjNqtF4eS1>Eh46RpbqecJ z<6oG7(%egKu(o)J7E(rSSYSv~UB}LSM}ozjgDqz$n@f#x1wo93P0%8V&ja?j_6Tus zZiow$IB$FfgEdmIXS|8<_0KUnKOF*13Y|^?kLVPw3LQLxFF+Hyh}!Ck0aZN%i-vfE z&EIcYxlTXio~Q2_qStL0@mX;l9gYF~!~1W3TF5urT3q)-(Ve&XrY)H|u}`L^9R1TY z)fLBeqWOQ2`gy653H8H0Q3V9F3;_$!S6o4c7)DzqG97%x{gvYh+(KeSjW$wE!hChr z^V#bX$rg!1DY<@KqEw(D4)lnL8lH7JhZ#)WDtrJ8JfPQEQY~g@XMLle{qsz^VxD#S zea>M_SLIi%(1=nzcE2-0FIG#L3H>6hlAxy_`-JhXXYbUc0h9>M?>DG+M97H{hz{+$ zuy5Z5Zsh0pM?>fmBcX)=Ci4XA3>xv>eWCk5N8xZ6mM*4aMxy1ycnx;mZm>&mUw7Mm zUWTZ==+Laz+6sRNfEqXr9z_4AftmpPp|urIpbuC9`ao*VB@qQft>M;4D}zs<HHxZ* zrmHx85#EF68V9D&ET+$hry-xJLU5~{;nl`Hi=uk7nD6g+=Av3{>}WHp)fb=XKz!Mc z#EBEi8PWQeH%7wiUf|wQWoD}0;a*tBgg3t2-b#Enf%6#NsS|H5;oUicG~(9prxV^! z{mZg^A^0o}McWuCxHJu6E0kLnOK|lHUdP3XCSJt%YVJgIXesf(Vj-9}8Ztq|+<9Xm ziP0pXu@8B-6VKHWAVkt5l9M!Qm~Tkc>y%b-g9*{b=%3lymI4#(PbWuj<iZu8TH{d4 z=~R{-+c7)YA-*cjH?fQL7S#=knGrb*q;$WIFOt&~SD3Gu$f^GH3R-0m!Jp`E`0A>j z`092|PfYc8st1xfdtA_dOQMF~5Q!h;Zp7@A^QmfT5ETI;pam(wiRgT9&>sv16Tlp> z4Ez^(9b5)i0i+e^^I@bk7r{w0a#-4pJu$moq5ugKr)DA{4OT$#8-X{SkAdsBW80a< zF0|C*gR~U@BjTNnLXNDHIH|_i?Raq!I~EJ;Tazy~?cu#p#Kz&NE(oyr$6Xxo#GXT| zKE0JOVSptUPcW7|tUCk4ECswl23vQT1d%G>4Oj~ml^7@T27#5_At<O0J?|IZX~3v` zsDLIv(BmVn8D1m_UKG(XDcx-}6`|U5MA5=Ud}o<Z!xUn>GW<jUaUMRAiL{dy%WeZM zD&{}ISIoG0pU5c0yEyc7`$P`Gjfqo4JY1vnA|4fhHTg48a})<bIWUkn6U;!jQtS}9 zl?+D1aSJhc!Ea>z7+KJz1SaA05QSa*6k-yL1a8WK%4A}Ri+T}x#$hOO;%f1Jp8%JK zeL$kDIKO}ms~3t1J{7yP$vzr1q@YR_^DbSo575I>jK)&MsPw#nn+r1Y+ZQTE3PBJ3 zHpp_Mr2AdP7OrJTeM?K*l)tS?nScAzq4ZB;9S_Ea{RNH2=+NlzOrr`%z6@wiCl)0u zQ+SEYl4@0$EDp0)FXMfUGKoYrm`-<DIE6>a(9$faN@c1B!37qZL975qK)JsjXewhE zn&r8a!h)jA75U}Uciy4TF182d^f2I?+<Qa@;CPm%0aIqG(V~gNGQW5KB&9}{Edi`L zSHW3a>GTk#L@aOgNqL~xnjIFC(r!+XNyQe03H~f;u(Bx@y=|}~S<%O;;FuDxYM@n_ zEi)L^*6XiX8zgp}B_%VpT9NE<s=NVK@{+QbE+fxlPG&BPH4LNBWafe-*!Jy14}JTP zf95Q}OH<dTNmvg4BxYFp0?mp<vh+>xUUgQfO3N@(uJ7xNa|19vbOIO-+8ID=s#N9@ zZyLw)Qd%V8<zls1$IdEc<yN&t#Qawh?a$AZi#3w6R4|m$RJ*foYvdg3{N9lY6Z_jY zCLfz5GSO)gd9G9kXht`J*3;AA6Ls}sab@WIc>vfWY?4w37?mnpDM_Q%^7sDhO}dF| zT%PUft6`)gz5aDu)lOcLtTR?|tk;kbZcM3^C>(arT#g%&o)BiMRN}l8M^TPRH*n_6 zJu^R=o7bmzjVN<&`xRN5NmH_*A5G_HCnskW(9FSMMs1o*Dlw*}N~B7?GF2?Mpiic% zp{0F&uAHD<<CG?AIW5Bi4#-5AjEJS}X&Eggt9@!yDln^)l#G)0SXiy6%vYljs}y1- zvl{+m_?=Q6qfAmMm6jw;@#@kRhs{n)=`2o;^5GbiW0KgVu~lSd`T9xd!Xy>yL>9Tk zqSh)TQj66fW}Zw<woP@eOx3YVvp1G0Y)WNlnp!0Vi&in!n+m7el2f&;E_LPBG-cVw zY>`SmwNg{LYCenFa`bG*?b@!>@?!n^-ZZ`b*y1I}jxAXXU8p0bEJcG##ti8565H5_ znq5D<fm6|EdM!&9i?r%Jsq{LNV*8tc!q@jJ3^ap5d}tnw$wos`tSIEfA`ZK@1QNYQ zCu3-!DXox*Jqkr?25Oe*2Bo_HQkmR0+wZHV>E2f=N*0tCZ<)kOfQZ)WOfrRRSfBK> z2E*<`hmm0nmfm5I@2_&%!Js<Sf0FcCmSe@QP3X1FKiemlE$gRN7NX+U3p_&~3E*BO zoOv(e&%2wJ0<fZX^%hxKOp^E@=Rv0o56}I*Y0IQZTcV35ZE0%S5*<@Id-EN?Jqz5R z^wQQp+%qYU$?iXA)u0!rrA{2*)Xya!asZxu1YWuF>bgbM)%N@x{Lm!w=p?SN_vl)0 zrb)?3O}6}!0Yj(FsXR2syLjUCq4mAJX=;X6TZ_E|dkqf^jq4o5{BorcRM1*#2KMGc zb@x<+5goh1H0z2GD}wlTG|zikvRLFh#R*vXhPJWVxXrW9An4o)AlHcNk6*cLqMlfY zY!-Y1zW3RN4WEHx&;W{YC_49Mr00cdwN0%CD`(X@QpplO)iG4CY>t~se?X$wzqFp5 z&%rC_m?oDw5{?6^bFCXbgYWft+wX3H3mqM-hWK4=>QJrEQKngl<pm2hWey)x+ruy$ z{ve3c#;;#LUI^_vcP?@lO;8aDRBzm#Od{aQn+RWr*J56cqeoNt2sdTusajQJ<ipp) zJX(usmDh+8JWp)oYkd={uC-Q8m{7%sswVhqSqk&A(N+)^1o{!d_WcM9`)ZHiWDH4s zq?YPYrj*Vy#pDq^ieN|vHx(7J^&CYL`=G>9^e7@K4n?=t`g#;0+SI*_!1jMp9tJIK z|9>hEjX2W(v+~fL<uRdAOZ`tuBL4MKF#p#3u`G#WS0cJ$At;{cwYs>gOybeR74!UV zV&@X~AM4(h>XS|;7syV*Gdi<eo|!3^r)a_VR(B@MK+(16HuxR7$7!;;wrmZj6Y#4U zEbGHQ3Ty_9Sf$b1(b47>*&RNw&8I;}O)&|Z{OAr7g00~&2!%rM$CeiOV<-ed;V^7P zXLU;pP=~m18*B<k+uFjrc7@w$TNoT6L7_S0b`5cF4><(&q8E{zVq6%ah@`<I6`g%{ z&On?n`!qg>!HEh&G+I$9i9g+#!8$$@`*njDjaV4&pdfZ`8|Em0v3jvcMTCAG!Wp92 z2uj6-v2)ZY>cKZqdh82Wc#5S!+&^wR7W$(I!RG@GMJdvQ!Zhwh_yJ15&OsGJbxP}$ z5qV=iEJk&&Rrk7S9Pt{0#9BHGUZ=gQs@Qw59sN*0^Vwrrq1CugLh6cZg8qb}Ggx$l zHJ(tdqg1#ZMRMrZfo`BG2!1JWMEntkz!(e9;vY@UFy<VX96Mr7L6hx|sBj0hTkM^< z0lL<L8jSA=G>M}FU5HF}+-rH3iZo#W6fTrmLR=Js+f_v`6g2=FY!YHiG9yhT0~%1I zib}M#5fQ)26m|kv0sPLm^aImw>~OK0rO@(gsqz=)@F!sFKpndToXNDjU}?&XQ1Mp- z>Y5a#IK-e10c@Ei%n@|22_?#m6$1BDQ38He68ff<)NpDlvAXO8B=mQNjb0;1oTZ>K zX~5tRHm48ceHWAUB6fG>B9_bnV!GxNJZ@t@q#FCprcV6*X(q9B|9+|1q_CP8`PQwB z4467*ep%ON&TYOeS=nF!{mztWb5^XFGi^#iv&FLJ`N_Gtlb>HRjj0(~RT^rjLhK|g z1%DYhu{%Ujaj}!5x6#~_Md>V93)nVL4BsoO>D8iA17KfJ%!?<#G+E4hTjVO57G>5q zEpDpM6tQ>t`*Mu9k0(&Ypmlc*<L`z)#C5h4#yT)l9x8}?fD-W^8K{yPKSeyBd5G&y z8w-dbH?dIFvZ%H}`X;NXF|#&opeJvbH4BVA1b@1J+2upy520@y1WMX6|AoJ;hktkq zUVickko^S6Tl@11^Fd=`ErFYUHtW?tzl#2{q64$Q`|x%66UyQ8vipJZ&>>j2_2-A0 z9)KUd^cej3__RmAV?^C?u$XSV8saUv9<==?{Ah!t%Ye;DaQnKjslqx%M=O?YvLS^o zJfW(Cka`wP2WafX?;SZ3k8HxpV$tlNuEY~S@W_$)op3BJ=I>REX*bqo^-<;22x=~t z#b7BN#*x=_%6~hhzG(T~c|lOd<4M@KOiS2tA&Q0mB9oQndPay^5$&X|V+u-vXO$J1 zG~vS9$?QfqWmYJmfy`ikF-%@H*#Q1Rwht?+^7E_m*&XBW+Pz`-UE}*LoZ8H4>$Gh1 z)P?;zs9VLdA?$r28e+mI%l4nU;E6aHdMOE&_U~Ux0_uF6ePmM2;wrnnYH^Kh+xySG z#M|xsOV7Q(O?J!JL>XruH3;=uHO(8fag~QI7hGy>z(s2kHu1@A5M+FIG^R~fY;mV# z40hDD-5!*L3tv2P<f70fq{vO3*PLKadFP-Vi~2>Vev5Vt(wR&;e8tAExG?O1^JmS1 z^I=B<Iui$Wq7ktZa0w_)y;Ijb{nLgGKRvxBmA7Zw`R<pAvei5?AN|j2m;>y3lO3B* z({2Z<-@mL@TZED@KS-(<Y^(daZrg(Df4?SG!SeP#>;8IjO<LNTlYd>;T`r8v-s?Xr zJA-<=1C4`!r|2V?kt0g|&(HXJ#`FGvzvSnhembJu{&sfu+uOVMr~d!D{v_h<uK%nv z-7%YfW%eEayN=}Dxzj6jp@i?CqDrwxJFW*w2?WIfkJxQjig~Npr0a2)YwIfcWX7Cc zR#;y=IyWaLcXV-mVOhGFVI#MSo!s?QNUIFvGdhF<%_AE_wRx3Ri^W=*R~u>^*&Mi4 z9M+YIKa`+5L7`cE7Wyt^w>RceUE>x4sMIFBPef=uDtbWYj{%MeY2ArIcM<UgP7dfn z0@5W+3P7lKkx4M)W6<Kr7Ao*mk%I&H-7YhYmc`OJQW9I_R!Y#RvWOE8uPm+V)tsJU zo4QRgSJYA;4G*6K->cg`MaGG?PAv8eV8gY(@c4p0RUSCZdIF!@@*VJ!y8<tJle03w zGZ|UG;Aq?JnU!L?$`fAU9mIifQM$j7zkBL0-cneXlwaSnWH`vIncg;KJD9l6(6?)^ zrwM)ILJ9+N;bw6NwjjwM4#EgX%b;fQ#jHde3FpqX@Z%*C&j>7;8^o;sgl!5xb9h{p zt!iA=0awUZi&b$$^i%16zK*LB;%(1tS(K(TP1!#49&w%W_My@G-g7fx*t>7m;G*qQ zOu95KT;++j&}wWR8vXGGb=F(!%SnfnH#Z&ZwWWZch~4Oq@dWe^&+Glm+3iy_qHQyw zGBXFx8PXicr>W|Zv-YKfr>AUZ%j5e%f)20?&7uRT$=HuEhu2qvm?dBrRK`1zrn#89 z63>Yk%zp~-MR-GobQzu_7`-?u2pDG^mYOrfFh>G-dy*k{1si`p=DVUCc!_Bw7W8mz z;mM;FreF;RJ7(?MH)}!ez_I&gdGhGRXaMh<GZH>N?(Ty}tr=AwvmP`QR)7!=!A~vP z9JRWlNUsG=){JkXOOuSg+B_$%jFJ^8ZMy22Kc}Gv49oGOCFpxwGH|<>7WehI;5*^% zg+9)@q_0c5@4`NfWqtjueVV`Sn-!hfxYaPiM8DO4pfX_hR7np=>x*tsD6l~xHXEGA zqLAc>GQeoAiEDkCRmwA=+F7-;-mJ)(9-(w2WPNk#`+T*l?S=4?C)m$({(Qe&@lap( z0L}K!zDL%B83Z2>^(4^g#IGDUJDC;y5!^x;Xo^wSA}klin8o0R273%O$!jNC6|q$T z9@e<unmCNk920)6DW8`?_g>mk55x5>@QdiD^(~Js0}p0L8>a3SSGLrPTE|C!>kdUK z%`Qf*k$TgZP^1-w#RKx_@Yu`}E+j2VgMF(eps`%2R)F%PRIF5Pc8REx!pPt5KLZb8 zk1r?hZmG8|do;Xx%8(hh`j+dhV9KF2jH1|OwmCfdG?&d~&Q<1?m1L?^t*OolRW`GW zKdkViyg>w50wx~j?TV<ym}a3hKN?Ns;EzGPrGOaJ8#btAVvr=*WXZu`?Q{nd2b0nz z1W|+Oa5EJJfMBS{bG~9T2*klo7>5oA!MlTQ(@j%wi}_XKHS0$WTc;m3L%(j==#9#8 z%lVbkfUzLGFnQ*_(jv%Jk0^ANOCDUaQ&R3K2r(PXQzSuGeigHrXT?*+#di9+>~zpk zQd^9M>e<i$h7IMr%O#7=`<RLw)Wlk_v(%es1fQf>$8V92m@{K2d=Q)%I%Cl&>7C<~ z9FXF3)K-~n&&*(p3vTd=!UeAANP3K`pekRbh<*a@b$Y8jN;yooEVjb=wk$JPnbW7Z z#{Bi4SRe<P3au=^y1)gkbc<41P#xRz#qH`P2qaSo64$LMUAf?7+~G@@2ZR6`wQfT0 zyksWmnh0(te7cyIDg48-yL+;#iU$hB@0`)WjY}%2FA=7Qfqpbu4D`?5Ym`nIm|wi0 ze@g0vb(V4J=Bj9sKnVBF?B8qOpKZW~|2Q-sKnjN6@z)g-BA8+TElrazXZ0@0EXVUe zXiQ*8t*?nL3k(S#oPCAvYZ%3Eo^W~bT(~1NR3L*h?{UzMS^L^!c4)#cVPD)yyx2~n z))7xDiB<Gzn3xJv8W6BnF`%!JPZy~O6d~zErgS9}L$DO44%Dwu7c^k~_&|L65V|Ia zwKzjiU);jF+D{f%Du9<>oVa)XcGC#M*2d`6S^NH~**B|xy+wlvRf?hSl9%iO<-q=d zqIyJ|s-84D4Q8=ogS5(nqK`;I9hKs1({n1`L{zCZbVgZ~>8oWexqW3LblWupvVB9v zx&6+c_w);T;H5(Q>RKOjo2laH$qD1&<0I$nL%b5bIL|X{-`Ih<3os#u9b8Qy!+P{! zMImU=n>|&V)#@Cr1%8Ud8CKAw)fZKO8OEgO(!TROS7{TbyU{SMbmrBz|HYpJhSfBT zh3~jLeTz%+te3F`zUQm$#DU?TVJRw^@Q;RDYwi>oIh~Owv2Gd0^-4!4;@HRS^63QN zP#xKn)(My}qjd`Sp;ob3p@V-^=(I{ES)pT<gMdV;*d*5i&#Pj?a=ql0JfM|tR!Aj- z(fKaD<cd_vDA-RV`t1rW)3!vRja(oP=V}#8lv<rMQlXGZHFCNRj?(GCW6`JO8mUyt zoR({J(Q8R5yhB2Zb(**igRrKSP}x+OV29uq+&K76vAVdKE^o5}Aek@^8tERJBvir% z3EnVawY}}bS3XcRp!lp<BG%oj7r%Z`s}%I0UE$z}$s=k+dGG@K@r@-*-T-Ql3)Cd^ z(Jgi%n94@G7Jf7EGTf?Bsx@FPd;sTzVQ`Yt?wdYtcZQVlFL?w0*dsq!+an_sO00Jt zqBX?R6(|b$^Rba1nxME&c%wAzim3u|haOho73}|BvYVQw$NmvLet)x4mM7AxWh|@G zr&yi3y2(ugoqg46nMy78DO8L)*VmA@=Xb9&a+Fk>C)WInq`TjE-F<UoMjyO+^9bZO zYqbpumR2yVT&0xrQwP;eu_%=?K+6Z~b*wrqS$}%Ru2VRblKEGu*uq;hJ22eh8{%;k zy3xkBayFt<LP=N?@0NgogmDK<62PPXbQ>mg(I)!HBTWOK4YZwxpV3F?Bhe;w4cegX zG_W_pFx`fQocIPwhNIJPqF6Hg*yl|kOm&kR;di<b=|pcb@e*JPxFuY`9gqMw-?Qt| zXMqY{ZES6VSAi*W)EZdE<bpS08T@ZxLb1REuV#*7>TXfV=ddwK<0+H`KNv=jRDn0q zqyLSvJB6}C4>p49x9F5uR((Z6aT%zbI?59Bve}m!hI(kYyH|ktt|}K(FY^;8!o*h! zNrkC?Ml9qN)a;dj0I&fJ%~fQj4aGq^uF0#jD~WnKmIh*t4zx5U@Wr%`sLj}k^K*J@ zz~v4E+^zt-E-*L{7#wjgII;l!v1=F94_Ub2NTl!4MT?I<`1MhC-O<I1j-rdmXYjvI z-F{)!stdQDVjhF=19%!Pxv+oisJ-Xtq3|cT4!cwUt68unHz=8R0A4$|@!R_R2H6N( z{nd>J;k5(vB*9!TcQ3f_i#Bj4og%zGK;yUjC*XH3SO7>FTFHx#0`&X(D9i+_foj#o z_KT}n+5CB94_sKX=>2;qM0p&IJ_C9!%X-&%?|JDycx`{nl#-Rk+niGt><8leUb+Xx zPhHT0`ponj6nlWsMIF``CSZ-|V9<9d=Kw3f9?5xAO!*zHK4Z$|0jzc8VFW!SD~o6; zRxGjtrZ?OIe*sdk97y557uK(TVLixIu!_t)_o6d3KxVbd(?+KCIRk%A8;OExKsMmr zh3>pelth|Q5VCXnssSyfV;^$5?4g1TdI^xe{0hqHmsef}2iK1uw|@P&@zIA<@-njQ z$u))nBo~F%T73ro-HHMuaejuHWP4UdUW(qT)S6kP!)){>C!4iOYXW{4Px+}J(N>M` z+IxVASJLUOd=kQ%M<%Q!gq>ue85LckqrW(x#{4g>cG*N~qwOZ~@%`gBj32)Nc%>P= z(xk3c>z1aZr1i>>8Z-M0yW4wLq0uNYmK#qk9E6S%qw!Sn_Thap`@aVN{@QCmPOnIW zI%OcvX?*k-eG-=}PRh*CYLmGneO|9zpR)L_f>;KN>Vzy`D^~h)djTzwzlL)I-*(40 z6=V=Epn7Wszjb(#Lo}fgIfywg@8rlOppz99rB;sF@)bP&<WunU*IUMg=+pTQu*ccA z5Uu@Rg|9k1g?%%?UON{aTN%Cn6m0!^^X9KX=qZ{HjqA8%#{i3keeLia;J5cJ^rpjA z_Lz?Py@&?Pz<rJb*wn`5b5R%&y4dBj8<e!kNSUybI*odpk244gTtvOjU+gPFs}2;! zGe)fmcxaRWe<q6OK7^N(;m?)8*8q==G>l!G3+Vptp~Y%5xIHiJBctxaRM$}&^zLJ@ z&#}#`NUEL)LKk=If(z{z6<_h-MP>h9X7C;WTZ7S`>@(=+3!^tS0su}k`ge*JjpSV7 zBHB{s=oQ&9wHzGGc7rc{ed!{QPkTK5{#yOv-asMEXNUkOq=QAUpFIjS%yn0x5+JIQ z%Wm%o)h6I+OQ|GkA>wLxB~U!P@>H@s2(nH+kFl{)`=eTtRY4lrZpDB&1Tq`ZE3#fv zVLm^AF$vK{KJn~_Io*7+E)Ws-ZC30L7!BnLG%y7XkHi_f+ibu*Yfm=2(u+{G6C_JE zZJo%#qx|v>+a}O=HZzuFR?%zVC+pRSArJxefPrs44w7^VG)U+Lhtv8>Wn8s#E^SX? z70G)2ptcPvT7lB3`d7U7q+2d?&flL_B9*bF$`NZmgqPq;@Y08C)_e#uK|hfB;b*s) zVCeN`7cP!{7~NMqch$PFqUbC9yp`+6_I~>~tyL+c=`DwBeNdLws+qLY$|_PbncB}c zs2DkZ?SMY#9tTFXT%?oBTMk%JI<87Fw?v`{)qc88PU9*l27E(az9z9i^xA*MM}gSf zYNXOJIu5`)YfcyXT>cCRFtP#0g=P}9)2O8p#c%>Y?asjXB#5vuxBvKuZ<j~SN8sPR z@VhXWhZc=P2LR)NuP?xd-V*(;jG21w?yuqB!QMvMG+5Gp@?`r-5k;@q^n+XS@GkJk z{g1;r(V2JeNrmP1>tM|lAPek+r{E{iVH=h7{Pmz>spuqr2#+fo_b={kvYTL|+%6g| zteGGdQ3UW9Vu;Qs&70gJD>ekeSQ|<f&4X?$z-B=n95A7>vy{$AD*?-FhF`(HbIP>+ z?wui%EmUNGzu3Q?Pp<Wj5UXU#oM}vnYOB`Zt<UmKsBxB;O3?Djm}Tz$=<Nef9@#rP zB}d#RZ`9Nj)y_KrA}K6A;(<%>>J19yU0V-^gT5eVJp4w<XofDN^M2o3j2J(_4R!FQ z0_m%Lat4*8@Di_^X3H{YUaJr@6Nk#mbJA+2`^LWegv&Lo|8X#8{vbFTzOm*8b>+mA zxGX1z;~x<GHL^s-6>EQ@`6)mQKU|pLVc6MT=(_<uO`sI~T3<2kCcYK5+{I(BAB5jc zuN}>@qid%F{lV9d-3HG-nyP#f{_e|7xNkhiJOT>Ag9o-WFTG>wfw$f~ux#_P*_-d- zEc14)8Q;D=dwcu%HM{1`Sq{W|egM@cpTj)~EQ?%gg^#VS7+wM<fB7dcxpY|N?eIiw zeWFI+?MEty@=+C3KjQDkF05J++k#7t3W;?s0$Qht(Nbu2f`QC1433c@BpZJ>KxBSc z!4=raq81Uwj<p}}neuoMc(~*%c<zyxAN}&7@AEX{-U9j;{#F5A1j$wvb>rz!^N51l zY5ismpR?<>cl&y;zd32-qI*_6@0kp)(U-VOcklQkJ*uQ&*Bj%9-~acG!xjU6(UIPd zg63a_!0*w7GZ8E?2PRi7KK>kdYS`p{`H#-u+_7rp_+bM+-E@{7c-L#M#pP^aUhp%5 zaRF|*t7*7tztESsF-_?d*U65hNZ8Gc+5p*zh>(p4&=j@d4NFm|Y67q^Bw+;aXEJ9a zg8oZ<zabpcc@fHGMDyaGQMpWb=UC^(duf6aH2NQVdse5pqo3ihPM@1>wF$1T(Wr8| z?tG(PNrp$sBx!Xl?X{Lpgg+KkSF_)OVst8a`hptf(E98_ft7W(?DBMnL8{e{=$$vH z)a%fI3)NgWG@@kb#@UA^j@C(j82earbpe-zA8h}&p!x$aWm?|AeuZ*#RZ8`1M~|Kv z?8*u$<T6u!p>67u!unQugW_%@@{)ekW7HdHR^<hV5J43hE0AL)?wDB=S;XNWS|J$u zo2^#@G1C>3k<$~1;&hUU&q4Arc{MSMD?ybVMW%r`?6KgBNfSeF6E4vj61P_DGwQMB zTMQ=#mw_?rJBx}_6U}xq5K)a5>^gAt*u8t^F9>GK*<vpTw3edjlfxT!+`Tb8NhDRt z4H?i-RbjAbR8mQ>ij%6;v{qbIrM7AnBEGUxYfS-fdGdzVfB4gf^$j^HASo`AI(q|V z%FI2x&%eK`%x_Vt(Q3~nYu+)SfAj4Ap?Mpcp59cmecM}Sw)v81vD9ufq!~2KT&p#5 z5oE6N%w2KYhxJ4AJZTb{%&d^`v!;djY+Re7MWj!$?$HPDy+bBi5DbMXT3U9^7-<oU zEr1!*ch&UC{*saaDlyCGoOA;zv9!F3Q`0IXFRiRnr)s&XN{uyT;L>?Bht`i9SKrWV z=TkIl%am#`jNZ~T<J_*@=hd<zub5#(ch8vCuYdi3MmVKU&8DX&zPWwdHeWg|lSsv! zNku=RGR=dj*~4|7arHxIwt(|uy;@=B$7YulO7oKyYPK{~hWkwF4NCO62xE^0sl`~r z=JXr&XgYADd$3zz;*{CMI{i+JHiI}c)Y%cYucRGHCYu%jZu~}!*_VbH5-crHGGv>c z3kY8x4HPFaK(sOjpeM!%{&JvXL@Je0r3kLw|Jl-IKRk16YPy&eNflh{9Iz1_cn#bu z)9BN^8m+{Tui*@KbFMB2h?HUpC&K!_qFF_rRd7R!)1_4WDRZz+CsVqXZP~HDIatzo z`|@p5iVW$aM26nQy|wV8+%c<9PM`X~q{`%IQ@^U3;Z|j@=DC%<MWR%lIxw_PVQ}n{ z@mUg;gaho7!ly=NexRORI4nyk)#dFs@)GPqRB#qc)9L~dF9(mNRTX8+WNpqtdcDlp zYgDGFVA$Pbhi@BSlVy?6YbtF%+URTur1V}or=hT@X8e%!=;4w1xyIzlc_k0f`uvHV zHa}r9M|p{c*6akE&^W4cmu+rm$Qi@5^u;hPxgOVoe(A4RtH+GBh>Px+V{k+WF|ia* zHxeB%C4|{<dzhgTkPH%n9937`rN7dI$!}BaDA=M48ifhrb}cqNq(eP(BEfF$qat1o zyhMY|kT@q48L__Izs3Un7Y&Nqibaz`R8Sj1MTnqPA(ljURh>!nPZhpptDzWhB%Vea z{eY!fZ>qBp9(?PDs_Wh-+=z1_eZtuVapodaxzqPh%nsdT)c>Eg!zgTJ{>m$Yjrpsu z3RdUw>sMZpL~Q?A)7*3G>^iSu+yAb;^k^NGNtIx%Scw3d6l<lJwf_HA192uuNzGE& zCKUY&5bDy=^q(~g-5XL839tzbf^s9-M$7(##sHl>Z)%K=05UblPYKcq&}w$kNg7l9 z=rUg?dh#O5WsYnFk1JhfD4aTkcytuximb5qAznwQqClsdJPv-~Bs(RYA|pR|Z9|Zl zeGUhYfLwS1Ho^-ug)6h`oYta!6tt?M3-BxGyV*kFHpm5!)S-LlcHv~p9u;JoPV}8W zCUcaN=-?0$RF}A=>tkW0rg*WssA&wi0ke??(fd;Ac1vbEu{Whdf>kP&X^Ff71QS(; z;H0&;W?HtBlr(Bv_K)bRZ?|ATNP-0BGKVZ3SBQ?knQ0XO!ccOYrnOa&w~HyRgXk6G zu}lej$vhCbom^aF+8;pN7w7bI8cyRx{{cGlUs{aXXgDb;dT;bzsZyswmo&Pho9Sj- zM-m<CXrgUwTeOY)S#NI->uvlEN+$c|7fz<PbNCXFOT~c>>DTNpiVo>z_Luf3`^)7H zX`*acgG%L#&o_9Zmb4@)kNp-g@r`gitZ=buN}<z08!tvD(8jahcDw9&L@c5ytxh2p zL39!!X1zV67o*OjLvy=c3~-=l40D5$>e>;L&HxnP5YHapud(rXm}C1I6NMFGdw5id zp9Sqsw}=xFQ_Mh+4`3w;tm;V%j#I$9-A_Nlsehk0?Qz&%oG#ZhY!c^G+Er$yire+@ zkKjJ=Ex3=aO@Q?j{(uKQ2roaTeY`}<0HsW2<m^hPxuBqrdrb81CxH90$2#qK6Uyx& zh^{?sgW_krcY?-C8pd-&{5a;zGyG~7(05h{%c*@XzYDvR-~z-a6?O_^1-Go(RoDy< zy?^(*Q7K8DdspU^_AP%8OnLu(Fcep~U+tT)y|9u~C9w<#q_mvo3k+FFR@tFvyF?xI zvp3f}y!E>~THYO4)HHTz#T=JNy!AVv{SIz@0yT#C$v#RkqBE?TRUx)e>@$^k24s!~ zqJ8VWKQV3EiSNmGl&}={57Yxil$26nDy>0(AQ_M|HsgipKTUpUz>Nm(=t+2qSr$DB zGTFm8Ob>yVaV(J=Hr!|xJ<f%Q9l8imj&#Pw(AaxxFlP0t71lQaKZ?|HM$zL^_!|7L zukM<h+c;o^cJ830M?Ck9n7G)KYYbF+XG{|-R+NU8gJB&^`*&~>918d&pbCiUCL8X_ zyi+V$yA^&u^7?OnGh(Y5+#wTpu46?4E`yXHYuf>%v!f0yqS`68{F6_jn?Csjl%t7( z0>|iOAPfF6dIvlo@7M8XwNxcFBKAB_Ft-ElfEzp7=FmzvfYp>^pdi==3$39Hb{|@G zVvQYdz>$tQ>Ea*_d_+mlr?I1zTr3?f2eVCHo0dF#c5+&+e4@|hgZpgB;0Z_7fWnO% zn(FjYMGa`(E8=JXPPx7ju`DA`p_lr3j)vcxhMDBbez^E-t9{tQ<kHgQg~RMkO-F6{ zhCr=Tx#;!*VKH2%lmotTq!Iv@m2zNxSR`WVQ&Li8k>8F)OCd%sqQ%pUydK`Al+coq zLfxkl8ie<IhGH+HSkI7b$k<|=FfoXBr!ELg98oR7W5wrWSfbzn@a#h78Q@0?jP%&T z6s;Ncg6Bb390pp?Q@x7h7vT<A4|iUad4r3Gjp+Rm$Xd)KDZv8mfSh1s<El~c>1L4o zaoLDri`yR<wW_f(Sl-AW_Fih0WgR(kBui$M%49kBOlX{Nk8#zg#`0hz{dC#HWKYpS zcy0fFAU>F%pFF9oVM)ckQd*)=GeezuD3?*efiP2YPx%t~4S7i;Y?4`JQfYQ(X0}u+ zO_SvmNhC$r@XJQ6B7M5=4O;XvYL@~meF!pm8wzVW*sToe)Ebc-v3?koD4+zq-S1)Z z(F&?BP><O88UTB~0rTcJcN_#WANj){->w-4zlRTOfAwdY`SK41z18$eu`M{Hq1tHN zeErP>^jE9Dd3W!~KfL+!jaTL$ZLpd9c;V*2K-ymentt~a7(Ti8`U!(p4=ORM0N{qK zyC>dXiEh1sMxR1asHeqP3fv*F5lJVr<dskps;(gh2ODu<GZhiyQT~DAY;;o;tC)jy zV#0#BptiQE37xu~FB7|!0zAJv$(9`M2!cP!0P!Xz2&@r#L@XX<PjEPDVjS2$rN|1D zKveM2z@NA8zP4k;&bQ{RzgqXldGP*segi+cuzK|c;QGx)umFbWXBWd-_``E?3GZ`& z1;=i`-`%vumM53y{W5>~ojb1Wn)lYu5x32`{n6Id7vM*TdY~*mr2D}mQTS08t%N^c zg^P~>VorkE$%g9D7Q@qx;SmJvz^wskh|bY=!0nD67{`oifA$6Te*Ny~cVHZpM;--J znO<PsJ&{irGh?F3O%o#zfg$vo$ftC#c-w@zAIX01?Wm&=kK~;$YdXUmwnQPCEa;4` z5x*Kg`#R4-+UV7E`Xn|8a5?ChD(pTWVreMyTj_4kptC_EoawN-Nsaz!Dt?ZJTaL6u zadhX8lk;j|v*+4vc`m~swWc?(?w6vi(Pb6^H81B9)6(iTb=O~14MJF94Ytxhs?{xY z3p$XWh{p!Mn^g{z&1Q0_G*XpH{eeoMl5Yfn;YBU6gwY)>YQe`N>8rB@1T2BwDhGC> z$;uJFJ`VCGtRzuC<FJ=5J5l%%ygXs4RUjzp1!BYp&%lm!C5zRevHM3cDj#eWizU1| z@{rS`)})!zY?>y-sS}<bqNQY$tt{7LacgxZOKx5gyw`YVBUajJywlj6l$UEU>9lT( zC%4Qt+b}tZD;=C{n60s)d^Bp0lO1DI(;tgn;#Q88YQtr-of$z}hPo-9xmMYvPw~6z z+*!WTn)Kmw_FdRFXLx!|sV~c2=kllMOZ%g*(!W%lVGCwBXP1SwdRcef03MBEJK;%) z@(ZQLHb7ny>Y>!KdPqq$S_0_j*TW&tMAy-qZ>6mgY#9s`@E?GEArb}(F!L6hCzys@ zM&HGaxZyHt5H*STAa;x5_)T~pOORC?O_<!fg)M%f0oUcdL+}x21t!4_Jbp~=b`cR( zhPpEfi=7A;Aevk4<JEjtl3bo<k(yR~uyWHE{yFu=+>ohuCjK0(amf7rZ{OAN=SP1$ zvo{EWzx@jsYg)X&eUd3FNoSU8`}fz%iz~E~0JX`KWzv}y+BtKy3bQ$=1<&=GXvoV? zvM|z8YySZ&-(RuoHp^gBDA!oK_rl)!gYP=?*GKn%X?)>J_}g!iU%u_h9d?DL!rTn# zW^*t@VZN&xCcTxe&<4#9zW&<>%oQ4~JO%L-88;~I3fYIBhuBCm>*28~;4)$l2pl$l z!Gbibo|^`UPg2&6x8Hqn5gWnya%2M!ODw*KS5qrvvWmGYtDjl3=9$%37ag?kx;poT zm6QDrxx|t;Y*s^Vir8eC<IY^PPn+kTFuum3T71`xtgNw{-deHz%-S(7YmPrxLyL6g z;tY>PuWEEUtEXg3UDc~c)!jb6rXXD>r4^&stQkFK&6-oHCzlQk4bJW}a(IJRsmrhQ zW;pVDxs~bpDOMUxZ!qWOx{C7B6?|aK!aF7m-m!jCX>r4>nO;v#PO4O@b@@m6)j9xz zgPln(e?hO*<Wheq(9d7vO|-)k?ALGFJ#hB8PISagw`!F)$l4R2Bj|n=?&)*8i6$et zODGfzgBv|OYj^Q-g1$|FSkre}!&9~(P<zk<9;l7ZL|<u%vWD0-o!{N29UKOk?Ynlh z!%OH0e!yjD?J5*Ga6Jp&@x-q7&Zh=<D|$k>8~=(u8s5~B-CUT55_15pzt&bawGY#y zeg0|d1QKmE|5a<u>#EQHpb2{FM>(l-#B1n?K{J6@2Z(_uTHJyXeCN5yh=oIfCp^+d zLfCIJiav2LI$i4ZaH>wnI7H(|ULQV^$w&qiSv27Tm7D?ByNX?iMx!H!;|jyKEJlOD zXaS{6|HyTQPqHU^+_eAZ1||5Oz!WMTzW?*jV|I4<WFQDdy{H;&X9w^Ul>_2BzcCLO zXzp?|9>ft5HEU<Q<VInXbcS_)tmDFn@TBJljDzTH#y$hlpT&ByIdPPKOrJ?9Rg1&} zJROv0fOwG#UxR682W_LXit*8z431#O7sW0Rj-8uqw4*o+-E)|^WM{|f@f)V;wvBmM z&{uzW%r@P$4dZ2f^B%^qUnnB4FN0e!I?L0$w};+`f{`{l%M6Z~9c-2vHYKNrP#D83 zG?C`%3L8*3i*?|OEIOV22u=krHqOVl=QqOs;J(}v94<lq-;Vm_XJidF1bfJGFnIIJ z&e)4gMj~8dh&Cwd6^_mj?j=Nyln^3d+eV)8c7|{c%x57OVqgiM1YZn+fJ!Q5OE@!p zaVVeErj|;W!4QQ$SCP=$_=Hk|{vjMDDMA0Cd~%yAmNFK?`AD!h>IMa_wI$u4<Vs~* z5_|zoZ)#8yKchoi5)kfz0XPnxiJ#?iVf{H9%`<yr_L>@Eac|-^CZ3Tn8V2hM0yO@K zwIv#)1Z9({*|T@=p7r27JO_$k!Hw}C1Y5^bH|XDo<{v-(%jx6uL-7Fk)1JM|w!M2I zlfZdUg#Mq89-?lHho|5v^Z;l<WA&}KR}~hndK;9Y@meZ`g|V4=gJLdqmWy%*+?s$} zhb0AAaT1;A;g6$#HZV2%75w8Iuo!Ne1G4Cp1t(5`c_&Up{~O*H{gQqQUYY}zfF*O_ zCHk%Cmx5hQOxFv`W3s41it3acbxM##1kF!}rC&sTh^1ecIJ2=k997p`SJxb^BPX`* za{;Mut)in%XG1+u!P>|<+7!F<9!^)skmPkREe`D0s@JxoPHxs~IdpnC7ER<v%#Hq{ zDEA_flBiKhQ?Wl7?Dj>M1wbJtPyQl+-9AV_Ar70GnWV^lS|vXXoTK-^=b}Hp35(to z7jXsCc%?RSACp8b#Y`|Fp_eLh44^n75si)BM^80HH^TP}Ig03=%s<eTDM8-WhQ0*e zbt%MtQGE_TY{=ah@QNTza(cX67t|-y1?5S>?FXJL&|G@t2-CND>*niCpz+$CwJ?)l z8-%BfhS3*RoGa7S>B`QncmYO7Px%oX0$+neKhmvj(F@};XfUz1seTdwx3{&vd~Euf zL!ZuU1fX%|r-#-|Klbwb!ekJ~ZivfIgmspV%0&EtVDoKo_;kb*nZ4^rME$_c6XTQE z6o*!39Qx~_w?{LPNQC(bJ_bf$wcKbE<CcqUaMK4Lz)d!>TrOrWiP4hnML3Jz`UyIG zF*4YZ85}t>$X*JLq!)z4)QvT3AVxo+gmC0R{KO6FvB%Ju6nA8zJlF~Q_U+SmJvOqN z&Pp1dl|XF6UX%u~wvNfl;(b#bLjw;-yKQn5kHOgtzyXxBhi1afC0oy@XN;D*-N9*% zzFY~LTfcbG?%MqT6!|QJ-h&Nw3x@S7^VGW0Fggu<SMO}Fa!Q2)^l66Q7`HZ`*!vQF z{&x-UEY@i<)z-9}X&WXC*NBJTd3@axTa}1O&oGpsk-xh4x<mI~c@q?E8PIa_k*|*b zY1#1MCio&qrIVDjZ6uYT+m=suu!x6w$^&@T=5#5^Ub2!#^pz+eAPO4rVs&zFQ6Pw+ z?`b#Y@na*B?Xec=iBi;at-?)}=b2##{1$eY^W-V!*XPnHW{X5>OqM8f)ndOUTjLk2 zbCr^0qf}xsr_gg>H^b+NfRo-j|5fzl7qH{i`SV`|9IyiJRagtpz%S3OSaA+mKnbvr z(3xAUe?}Cih=M^;N^zdZBR~A<=>CS}0x6rN-@1JHR(%#LEl4)>AN}cJxkq%Ah*KBz zcoPoIS#b`2+2e(<;8tpAsMl8``u%dOjR&9@BQb{|s~;VKwRgufI8l3|ZZGlxqLYge z8qwtDqy?pEJtzv0RRy*!#Cn28ZdEmx%a&(}nA}pvad%+P9b?b#+%)};<qrb!-e>KN zWt{D==4vbWHbbt-ISUqL?P+e_Gc)qhtT9`6y}GAk*W#_c&(gp2%a2~pE&)uRT=2Mf z!J13=-7#&`&U54LT$loKNBzdiRW+twH1S&al_9@R(YJc=Xfw{H{k8I~i+8o}d1cSm z#<@GsQayeA4ko_fdieOoC;_~Z7B;&{bddRf)qM$k8^zi8&g`Z8T4`n7vQEo~WJ|K- z+luWti5(}7bH|C}-1iANNr)lj;D!WJAmnO*aJD7Ta1|P$C6pFOxf@!V1m3ok5-60m zkZAMG%*u}Kgwnq6_x^t0msmSHv$M0av(L;t&&=~Y|1|MyL12rBHcM1iGJ#$lG`OL+ z4kDJbKYvRv&p{OL$8LGtwM8MX%SvJvN5bPOFP@mJ2)hzWgIcjz#qjGtyz2ck(z#C` znmhNQPXR+haO+^ExV^VT6F41juX0;VW~ZL)<2CuK1Ac?n7Vs2SJIwVOu7kI$jy?t& zQE~l?m7W;HN~87&pQqW$L_VxTTuV2$k?md0K`ju%2w|vid4NC@T@4})JFs>S>2pX( zqy^b0rw8!Z2criQ1SXHLAN%qlfO=S^1Bh5Ps2u#DXX@0RPH;m_qfWY&*D*A&UJnj5 z+Vt9Zxywew7uoTCMrAVdyx=jandqC=DXm^`KhGm(N?KCXnU@#f)G>cu0rs`Ff!^t% zm1;A$Qu-yWplLPpi_RgL&d$t`tUvA-t>B1;hqOX_y|h<KA1$<VcAK`wVT6;x{(C;i z3kX^OX^lBg4-r4Q(N~cD2dklh_P!M-E7H@v%??rH6%0x+U3#QU!9rFYQ9TSk1`LEs z1@V1GQy(fR#)(E+HR7Jjs8(tYwF%E~G@H!X00E&7U4qC2AsKEBa{&G32t;!<@5k;l zlTkXHaMPZDBgG2-`4FHd&FM}hI~QkP0D|s4ogn=#tujepSp+RCR+_*_otdsadrbO5 zXEy7A=4EgP&^v335}``F0RW)0SA>cpbuJ@(3Z>UwNVoN-AIasf7?=*A8z}FaxKP@# z61<Yt;L_{H=mSRhmCpQ<bmebRANrflEX`I27R^{wRI+A4r^BIebxz$}9a}qZvfch4 zrpx=EY!CYd)j~~063W$RTEM{f*r`4NXAGk@2_@6mkq23UBxva{XWh;dXL(p8v|i9u zs{;ZWv*e{6MDJD+5vu6Jh2RAsl49WH1SC?1a!F{G;Nb#0;w%(Or9NQo%JXSC6{`Tj zAW*O>PV39-vIg`@r2@c!eWKTl<ciAC^)3%bmxV%Q6^{DLX*%PxnY|PD*+W7wh%x_K zTP;tfOoFg{Igahxbr46U%$P9+N0(fF`I0-;U2Bz0i9?}Nt67WSVtS+=3<=4q)jEZv zgi)^TQj4`3g~=EjpKjFfi%v*ioLICz+!+ACq|nsTheG8-5WOjeENp$NRroQ+nOg~J zR^o(R(xhE4<AlyX1M1J6IB5x*@_r@#iTFu3<Ab87kJt4OY@;+)kLF^Gr!^hT$IGZS zi1N2lH&frE_7OXKvZE5ng)kQmPXE??gLX8(E&g5Mx7`n*?78^+B#{5$_xq=1zkhL# zuf4>}GF(mqY565$tQ=$q#4edL7X#g07oGs+K<c_UZ9wFlZohmt{sXsM962io^nlo< zGRl8B-(}o6j_UIWKav_pEugNTZa}O+uGLb#NSS~Ky%tpRY)M=CjHOZpRt<MOV(mlL z8;QKv+x9^@O;+lVmm|GmP{y6Z462O3CbD9$?q)7Qmizxej$;8e1xc)rkO=(Pnhf|o zu8{uJJ}ikZ++|G!d<EfvjG2J)^kO$;Nq*3bg3Z&WUW0<yK;SiS9pUGMe7@;!i^5<` zTAlzwrNyS!6dR_#D)I(H((=iDWYOU+dFi3#JT(PS+vEsUJ!?`d;t$m|ME0G+t1=5K zXN)fmN0T$sGKvbWQw|v7eR2!Y_P{P4ak%5P*KWqFohTe$g&u?|aP1bt>Ydq*qUh;4 zJzV-crO4*=Eap)^BK&;L@||$IDeQqOMyzXc;EH(m(Gk;cJ}#@o;ueh)&3rW9g~CA@ z>JOu23Mo@M<;JE-d@6^Dht7z{{2+16M{}|^J6;7(_kJsKF7t?WM9m=W>${N1C09ey z%HlzpQB>QEb;0u1fXY`ItTWo+WxZ$Bxhv8H<4Awq@I)!CrKj#GFggMzi^UXh7z_4H z<qS^ggU@vO0^@$QPW<(4#@p9<M#p(ibL@&+3hEiQv|wCowEVJp0XzL@Uk<y!#ATE7 zFp5CBe=gI%cofh%fge8$W?;E5+*)3dukeI@<yYrINK%2xZ#R03VD{KRt5K;mwg$$| zhS0H=_k)bJOuGiCcLU>W8(%ldUOjZ25j`8#Q&pmhn_4$WM{y46tKHIPvqis0&H+jT zeK`W(QuY9wV}WWyJnU4w-%YfmLf$?-Da4!-Yzh)1JrRj^xqiwK^?$ja(s+*qaq+!& zcNlMn4u!F*8{@?tMEdP(D7fayYv$uFgbAKNn*_oIzCgmdYayoLeW&yxm&YGST03`V zUpSq8R^!v$uhDQBbokgltl_H8*R?))G)L|`a^w#_#Be+~BKMQ@jAS%iI(|mwLb9y6 zFVavK@<(EmW>ur!lf3~Ki%RurI1U}PAKQlAxuElPP5(7~Gc}2zE@21{+0S@xj|Xq@ z=U9O-X5}$U0Ez9stcC9P;k^ztKjI#hb9z!oe2M22#uFENN26zI5krW$LbJLm+1%u` zI*s5DqqG)n=Qc=}eUVq(b$iQ!oi@OTy4I3Hi_0zYc|$$^O541N9XlplIDw_rtCy6H z1~jXDa)5DO*3lS$Ij*JwoRyjMa7dRgRqC!_6>U&FJ>+A~c<WE5@7T*h-`M2LSYFI) zL!FZsT}>UnNsAZmXcs4o8m`6!lu$p=Ob>CXLBvCyV9!%F#HUikUmcQYAO>bZ4TP<9 zOfvdvSiVA9k@oxgVA9Q)fN;~$X+&&=vPu_0(M))aX2{E~f!qN8iP5^O;qZdR#=y`R z<J6c{(2y|*@C&_y6WcSMLr>~Cl}lmm+I+Zs+rIF`ROlX%AB}qRy(R7CMIy_qR4VY{ zH$$&@c4;yNR*z)qIR__*9$`K6dY;Rpw^m92xVCugs2BjOM%4z&+d8v{crBm}%4rHA zaJ{GV(L1^hZ7=Ux(C7r#aC~?uzo35F>h3}%q`_CG7oUFNMnNgvF;n_}fUd05@;^m1 z1kn7qi9JizQXPnop)hJHUPi!DFe*7mNZ4l!_E1s+<gpE~L2`l}=UC&ifgBc)cxw#O zy^8D}?5~a!fq)l*^#EJHlX04-<t;@ekV7FC7f=i0OsO;?vU;!MgW9qk1?u5k%$xnI zv|kynOoad=Wg?Xj)`Y>+*?&ah99J1sfm70fP$|cy{G1LP{S9D%Rd0UUud_KUPoH1| zX8;ZI)Lu`E<0i-fuZg}_&*)1v>4h+|qdfD0uP_n(#HRD*x8(tq^o_+5^tYP-x?OMa z1xFd5pQCW+0S&B(ge&OjrrQcCAB@&Wv%E!2g}0(0m}0#(k#G`Z*i6J<Qq!fX4+ctL zcMwFi8?9sVL$7QM!Vus93p}H2R<&~{U}4E(a7g<5h?e)OovFImgSCaXS%aV=e+=Tf zz0!Vg%-=4U@_f=%FvdU1DA+^NjGyUQv*Bc<22{#ppqr=IndoaxL-GJ(VS+Mv0is6^ zO@(-^mDrl3ur`NCMg$^@BJmv(m;ZJkN5g2=!w_A5Y@JIJzyiS>v<3tiByJigOz~oF zBt@Ss7`B4ZkeP6ArG;TsypA)$CxK?E@p6qxwPEUPpaQS&G@Come-9<81=WU()Wlas z=zpG3YO5=0sUlpI2R5j6*D?!F7W<%={}G)m1I9-mmp*PB-X$${nkTGx7B~-IX$Boi z{&86Oqp9w&(rhqmM1_?;yYeNipvoBjOOQVOlV_yorr&2?(wdbhVGW(+^Q^3tl7`br z=H=-T&Vr(BBcm$jeh&7Om(#@>=_%FR&Sk&^EXy+wOkMaatS)e_pI~-6%~u{aGJLNd z+4mTUU4Xd!7{SZMqp7T3N(KQd$LG{>y;yQerNyur>VYqeVV=Tb*b)l6kzj=v-LP7b zJpAH;R0dXJ>^pD!!=HBS-2TPR?g?JLq3zIzr$EO^Z$o9|SNrzqT=`=+4KLBt>GX&# zla^%1ww)L*z`_?7`F-~2vg$5JOP+TH_`$pT4jkC`?#_Sg@YH3Tf4~31Pd|Nda+@|V zv-PO-+HAmjZ@mAFA9fD)?f*V}=XCXX>8aMWn}R~ut+rHkaGbr^Z5Us*;I<{TZHs#S zW0ASTPDQ9Fnoq|O4<1B)jLW$Tz&IHMCE1&z3E&kkR)drg&lX{kO%ja*0&<AoILC?& znk*Q*Gg2(x^{3r;UshhiSZjvd_4?}|`TAj~^28@rS}dyfLri<sRB%-yGInTtUdt92 zb7NDoDly7p1S9%S4&bv<Lv6}<X4S!it1f#$C@Qx6MtbF~w?Q^9R=@Vv%Vvo5^}ulX zlJ+OUcSq(9pI{Y+4k-v4XAeoP2_;*~3qQYTPUwCA(L`@9+1rPp%%NlZ#oV-rZGEk> zN)IPvdExaS?3oG@g&!Oc-6}G54&3fNFE-9~@!?oFXx0>{83k($Y#o1Wq>*J*ngW%@ zkFM~Ut>U#%p*Ls}I)A2kSfprpQO2)JXbn0AycU4Lt6|rOtbS5P;Pj%#B?>kJoGy&^ zkD7R|f3z?i>hsJNmqyfc!gVfIjEZcbpmh7)=ucrTU`23t@H!Zv^r#(HpmxBmkdkr0 zWJM-|J4hUGS#$7UP}Xb8*)z$_BsZH(>R5vU%8n)y@f>(L-M;nhN{3RXGc}l8sruG> zO>pyQXVUpTuP|H9<HVrNTaBG#aArZ&=aY#hww;M>+qP}nwkDp~wrx8T+sP9@v8|nV zYv1>++O68%`{DGdb8mm?TXpa0?th<OT4h$Ew0y1Hve(5xcY}DA)<9o;W6TE1+){kC zLgxd#d6N9;c=y;-;CWrSHfC3t2YB|5G4aI`4^Qif%~Ez1mw_JgQe(A%GQbf{^n<nv zQN7|mOD|jdpvNBWFBUtd7oHGLfXeAxrOhg0VbJNfhLvGG6j*BFDNz(qP%od@wccz{ z#*SP*40ClV@|MWe@gF7#ZX<L>K(sW3*xydMYL%wnEf8l88wnXm4nLs1$VF1F5C=m< z^0OsOTsTCI{6`A{st_D%kTm&^5=GJIW^Y9UkVbiu{i@sYG83~Ws2;<>qZe*P#G8E- znL~<9SX5X;dKeQTtz6N(b<u)Vi<(nzNTy|JYv5hDvr@4dbXBjHWETm$fw$z_=MslW z5!{=RLfkg<oz<`l%`)C|@Xn@-cZ8_j_1cDhIp~%fAt8i}{{T$N(KNc$wl0=5T-G1k zwffSaqC%A{3A9np#WE!gzw-3^?rNX*59yGs6a1Q;{1wzfE+smA=UcBNVn=a5aS)I+ zNEsn`#WO#$Uxo3^cH0_T%*7^hGX4Srn$wXMSB2)N+w3fa;s-u`c?{XdBA)@CYJ{<u zpZ=Dfhl~4x1}KmpqA&f2NQ}^;44afy7D-?_<6lR`Nd71D`)_>r))Mh6VdCMgMcO#W zmlgCpAM%=<uT+z{TwyGtHqSFQctMcQj*>GCZR~HrO(EF7dpp1UIy|O*d`jiF?{_kL z1iLIm-L>4YyV1XBb&_g~0#eCdAnMD8i*VTrp|`PkKI|1gfG%-7F4~ly&yMp6J@*j^ zgf%n|udr@K609@35ia==-(d&*d}L_dE}ZIJ4*uIfC2j>*fw}99)|254Hj4T&b3Rv# z0$21kaI*T-bA#ZnQ`R-QX|8A3&U@YXWKfAy0>@^B*~B#zv2wIgjsurBM#+4jTPdC_ z2>zH!lg84RpfJejhbqpwUihLt$mrnM#k!Zwb9I)v9bL!X8q?eJcfyu>K&S8F+K3wz z&9wRHP<(CyMfQ7L{*N7ws%>_QU${8E9;Y1_51SC~FOwW|5AY0mFUQdvx0B*=RFe@5 z8`tuwWr;T)>lFQ%7KD;nSlchSy0N`u<@yHKTzdR0DGDiyDVD6d(lsUa1z(;68z8@> z3bLPtSQquUnQ!nMxj5FXSXI-#d;V&v^wf&W8PO&<Y5tU%Bv_uC6GD?7-{vCg-JvlR z761G*^7I(fi}@I}))4VqR9L*)l~PvRfH`sA6lDwrKe@w=WN9xbr@i=4j4O!Xr0V9C zx;AbJO;q=4d<cT=Pvj>0s}Oh?TMy`5Ow!K#9=gNsf>B1mqqc`#*k+b^Ux~g)Sd(nm z$5~c5?)IWe*|rJdwI;g^4V#6z`I*J)kXp@d*1Ee)XS0j_>tP_1(oAz4)XHck^{Fg{ zie54eQLKMM6jii_f()4k++#RJ8v)%kOA4IUmLeUDx@D=_6YtP)UE4eUGU}LmBMu!& zT7r>6(6m8f?%+oSHAYpGAB%lSSNV9)f}ZZhSDM95%IDZIpR4m_F|>g1^ZSC13-!Ta z-q;F6=$JOw-XwGt$9C(v$8^b!qwfRI)A+&i)b!aeI;-lLE~8HoK%MCBvKUR1CY8r( z`m{Fiw=l*xz{E<02Z?w4-{XIyUQ<?g{Epg-C&jb4{8&x(-QkleOubghF+tMGgzoQ8 zt{R`dn7d<PQ0Rh8Id>C*D)}wPoQ$Go1EL*$TMoB6D5=ANd~KUtR;v!IxSJN+jziV| zmS!+_d%q7SKA*o(Wc3?OsotPuLo|Q3lkd7rk56#)xw<@NuWR=0$Fj*tjV_0DfbnvG zyBwIM=Pwyqi-q7hJm3~_Q3PQPi0d=`%7TrQ<*K}ZdX7op#|xOXc|VtU!aK#*`rgWE zGC$RqZIx3tuxO3II@?ky=`?k#cmQ)xwDVH2P*AW~bk<R6AP|>DdjC6o@PHM(I8eC5 z8I&o#Ev{7R3FC&q{x{q#q1_uPteoE)z%kk|3)1)+%QR81$CeQ#vJyHUzr9c(yH*S; zXHLZdSwyZ2FY-5u!p3V)G=fi)m>%RoZb#D%+YQ&%(PgdS4gXT#p({<LlKRpqKy{1r z$<~Lg+*`F8CQensZmlcp(Hf6C`iCgJ4nMVe$JCm#{gbeozfw{}U#ZVVZn#$jxyIMV zDdmRtM>qULZMb`r%^z-PN@ZHb(2E7iv4!K0)6>CNc(zsDhH6!AvTZT6rmJPP_DWbA z<{-5uZf<k@JBz={U!h<PJ53kFb4e4XoOaO+D(THTb0e;Zd|MBh+VF-$I~P)c#XK#- zZ<mef9fgU}SqtiFM&_;r#RP=wFJ@ouAK)>0^$XDPj8qJcJ-r1G=wU7Mmj%QoY9+Cm zchaL}2pl7Ue5Miam&AHWELLunG}Nr4fjwI+!$>&!F36<1!w`^^vBS#M7O*wtpkhb~ zEvWUsQ{$fY?5Z6jlTxrWIZ*40yeg~qvSdZlw3RHZ?DYe#mEFCqeAIk=soNfQ9;c^M zxx={MY5G0Nt;8gaG`^j<Hr2vw)7%;qu9Y^GaB8kVx-0qkD{D&ZWUZij?sSs{8@mfM z?yOAIUYp%6r*1wB=T@g$>$24K&1CQYUVIAFsI4tYsRF@FEPdGmIC~zQRn?X4RF=L} zl@4f-N7CE;^LI?Jm*dDB6YfEailXZa(=H}RB7Oo(tBBQu5Q|j`4MiDnWA=4TtMFR} zMt*{0eRU)3hU&l-s(TSv=c|cD)S3>473l<nfa{2(N369(43;+vg^6rGVr;7OvLXg{ z7mV1^23oazmQacClz0H22F<g#bl2qj-lHp>@#AB`e`g_X_5Y#im(eBKSc#gnwTp&~ zlF!RU3z|d$#`<YY=lATiFXPNUGcMLTC$$<S|3YAPBgmUo8S7T9&Tk_2{$7hp73ha- z+r73Ae7>ZKws~>EdQ0&?#A_%mdDaM355}(EG)PU;IQD=d;9m%u2vb%`y+?bO5_m`8 zIV$y4{W($SWX(qM%LY!3X6gqGKBN#%7!zxm^O`try(?0&7mbvBgjZq2pOqoTcsVT- z&7z#6kAgeLNQ7mu3sVjL(hw&a8f|c6pk0G8A+D9}WR#wrp%BJ4oVNaL50q?waq3Ru zjIZV!x-p53+rR10fh#AXu=$cFzYbzK`KgI{?H3}W4@@;m@x+7P@!|~z!W~E_Aq(sf z+EkvGKl!ZWHH+dca#Faj9VQk6x}J_9hib5d7S58hx&31bZCBjU==_BZ-a9(jqxo?e zp63aJgUoMKgC5w{Uik1&YM(d!xravA`p>3$!Mft4X}qm>=9kA`7KHEje0f9Y41r|` zxjx4SSs1bwYiue4z*ovXTXY$Lp+*zL`iDGXa0ABvah3sSy!4qSvL<u@JbDlO)<nHB zw!&&$#1=XooJvD-QC;z=9bB6;?(`tf`WJ6DZ0yIuL?zYbDwF2T{X|tkr0pr}R210> zi4oE93d9LC*i5>_a_+(tc$zzf@x10>&N0em3BhB#c6tT=^LWnn*6%L>WKwNc)t+rQ zkvX0nkc1p}+fPDKlgnqO9))~2p-lM*`z|BV$i-YEE}aSNO5b-3KN@q}DT4K_e8v@J zcLrrGHc51`i^5~-k|M!FRatDw)EcxQZ_+9#A36He4}Vxf4U7Y~&V>G!-fxDO-rHqT z49hO&!@6W1nW-*_a65r-gHijG7F%WJ&PnDs4N6qIG_BK1dj2Ij$ls2GK=nD86DlE} z)ch#Ma*jpZxhi_$I$FNdDtsm{(_*Kc?$L#rFgvNyqE_m8fvOEKtffn6<|f~ZUFvqm z)b^(V^&w#d3JKzS(pSqET;bRPbt9iW%8Mcp$(^51!Dc4_W$#ZX+`eD*3W!IIiy+2l zD?Td@N0H288#Eot5>7@&Mh!*DRkrcz+R6#ivD<e^bUOZnFK<-~IuGk<j~LMe>OeX$ z)r)yslFRGsKoOETT0CzL#$Jp0YU$Am4w@A6o}`NGmU0W;>aj3~KVNevfj`oz9VcEu zmN1ni_8b=S$d9fU$xOiXxBPV?NrQfa>+JujpvU(BTkFc>9Ve7{^%xEVZFYmkgiY&j zF)B|@7A?`Hw_iK|4j~sqdvFsUeY?8O0~PTv$~ZcgHMsBHX89__fSgS@o_2p`JIv@^ z`K)BP)XgRa|6S1?fC@WRh3PH4+TVd?V~LjU6~amUI6>4ADv_EatsJgD8`DD_XAqUO z%F6$^p%QDu9t|r5+m6z#o3+RuUS|I$>;3Wj7Z@63K<~Sn$mCiBUATtF_1hleo)I?u z2b!c*o0P!UInl@<>?5-xXl44EbtHN8Yj7r+J6whffhCiU9Q1rvT!eE<Slu|;b|gBf zhh%&}_+Qi1Eq|e!+o$kZj*yWofdC>6qqxD&WC{NmYTtXg0En8yr=}tO&trS7RpmF} zm4iOSkheF&p*0^;{Kzkz%|K8Q{Z5Ub0pn818f8dO2Z(;g6L=R>%s*bN?Ecy!x04*X zJ~yLj(YU3t@v#Ih+f8G6|K>o6oThpgg;KcB7u{-|Z!0-I?DD~R=h7DTUM}}~*L?x2 z#~f`_w99r|T!csB9MikdVOx{FE@#Ibd7vzPR;Uc0M@=0Z&#zhLW&yD5f8!s$-yg}D z`15IuLN;VTcpeL^5P&cy)Em1tby%qDy_X$!o4H_6GX?W0sU5{Gp(~6Tgd-2JlHS6z zq0oHM78<Rs>NAiE$jba(d6!?1zqlIe{F6@c)m?u52=}_ihpo4lLROP&QO;Sy<IPR< zlO*k(++1$&+-1*3h=Y0`m3fV&$I*XRJh+Ft-hKl5$u&$RWoRZT7#P42(AHiPS29jH zFO5ijhXD0Ta`g`+1+!}pWs6OdDtdLXWEYFyaYkN)@cXFP?2LJnek)|BO1HHHg@~?- zb9)(GT-={5T+cBe(~@1T_EmjXxnKNR$0P1{z*Pwe*}ouyUyB}FlR-tXAA2*l5_a<I zTlu(dGnrkDJZvutiI^o{azE`&1;0*EpVD`Xc^DzXxfR~%7?OR~l1&ARkd^dJ&Jq6{ zm!HFipSunb4}s?4GL1}R(aQJ%jqDKkODH_4XWjTDu2G6qpG%rZ|2W<u`#%uIh}zU* z29>^|q?rb-fC3u?Hum6}s)Tmt{n3h{6<NFSyavy-hrR1oyLXUwdxPhR1|AB$c6#=j z%2rVSsNo{%Q?gUBQ;P+L7%{eKG5VpHnGqNJv{ufIG<O8(7T03@wC^9H1fED}J<hK; zxVfKi8z;qi4Aunm8$1M4O3D}Zf1^pmc`EWya1{#K!}KMHCXqVCyus#Q5udbr6R$0L z^U3GW)ezl;8;ipmA2cHC6Q%>Sd{7)xQHHS!S%gy8ZU&)D*t)a|wNOZ$`f=!i|Ni>o z!3?37a%L9klEJSXt3OyDo8)`&^$AeAA6X_>bdmEw?6{i}Yo5Di2$~{3=t~y}yxZp4 zxoj2h!xhm=u&n(4v;?VJRf(n+^c1LimCvDbfEe!M*<4ZLuIQS(aD_^ClPjaT<DlWy zC|bo(S2YP9Sz_tK?sm(U<FL)xIBLYI^CL^C(*+i4E7Eie->0y2u{p+(<*hh?%h%(_ zK#dOnhyax5Z8}}xp2j=G<cV&=vpa>*;58Nz;x)LbTgGUW>?McY-p>E25LQQBjC%U> zM%^=QTm=pXCbK=zY1vHA*;G3|)tJCu9-V8Dr{89Jn`!D*yp+F`t|$BthDSB>Rs2s+ zZPgOX!V$mKC-+a(zw>0(LJ;D=ruj%HIB|Rsy+T_+hf_6Qjdn-4M(g+BX!QLU&dYob zTY(fG%8A@n(HO;B4(^NR6WB5S^L;1hZ~gO@f7(dGGtW<2Ykj(DLA1sfQ%L&WP`<%{ z0Yc0O)&&#mvRFbG95)zsGQIadoZmYjTYgj_KWb;&l2R{7DSjeQr!0QTl*B?8;c7BP z720x2N={`-XZ_B*VPy(!#u6j8@Cpe)il?1c<5QdFlVbxmm!4whdzVV6-<=bm@JUPv z*na4<Yzi9$*r%Ae%WbZpWDZJcRhrstrds5#-z7h=0rI#Tu?dD6Y>&(xb8K}*;B3G0 z%6Yo^-@om)2Obx`rMD+hQ@<ZO4FWy`%8ozJYf0}Lb>DkCi#iSk>NwusJ*@e>N22Dx zonqnruw*?;pna+wO2w5>%jvD@TavZq^rY-c><itMe9Yo8^H((t(TdG%JA6%M{zywV z8(+w#yGTtRUIZ%l-0m<^m4QQ+A<lf$JwNY^!NaWcs8wFbQFm2}lu#4Ut2R&RwV(5b zVV{Q0!k>HB6k+N8O+$ApOAu5)oZd-O*-2pwt^oc0$s$ehCgF^23VTTP8AltR8*&y@ zX{3Sf@nyAAuLnCzB98C!h)-v<nqZtF%`JMB<vh<M6$N}vt)ifphSp0Crqg-%eT1k$ z2~Ouuwk~2P@p>0ObGJrxV|e`eXmX}?F@SmP`Pkq)tk}a4{#7otu~VQ+i4YY*KcJ@` zf=7@mnTkFSK1|$ss=)5_=PlK_x8`Huw8yDd!aYt?fK&#)0<(F|iDfE1n>?v01h44d z2Wq#&*Oc4T9$$*Q3xl2jJBJW?`AoP)+xs`TvEV5j`ClET-h+<Z=qvvNgVJ#YFo}5n z`ZI`+(B;DRCCe6^JpIRhr5iBcX|ziZR=woJOH@R|*+IP`@Ei}rqr$YYPb`-S@Hl-} zhp|4{Ki@xRBi-<+)P)ah!*S$m65P5yNoP>hXJDtW*g>m$_rKTtyg+W9LQRHvN%fB< zwg}ZR<scTQNN1lj)Y8oiyw-FG`lCidyGWl5J!Pn*pA1y&1@qXw??g|t(@&U!i{|J) zFZm$E^$dv=FMPftAMWBgviTYD1M@Bqh~>Z_z`aN8%2ugfmIWXlrk?}X-m{v@I0SmU z?iT@oLMxczO-(N~wV}#1bz81VH8upLTQ6Ex%2I~l2R1@ozexcHh$M1aACKc?DwbV6 z?puFBKYF`#L7U_f@;ZH~c+gu4LMXE5s+W=Y52u5qh4Uh-5;6tsMM^f=<!hn!f-KPY z5APD|5vS8nBLCy#`=z(G2qze|M!^#4OQ7b{16=Hvg-+>?L6NdpqBO*+v+=?4;;Qq< zO5d?>(xm&yk4(g$neRl&W~{Q=V!I+cu?a`!Z~|M~2Ku1RTp*it${|M_{{1}^6aP|l zqsXiKYe5wp))f_G!x%wU?|-rYF0@+M<<lxuIWyz}LrPE5>qQ{w`ezR;XuXcRGlEj- zJrJhYv9mija`6^MNF&d{{o`tFl^$KT>>nNyfjEyKRK%14g@<aYAYb#(uz+XDl@E_< zWxRlTg?qjcdYJoiK4*9JcU9lJ-MX`rIB>VrweM}>od3JkU`wdw154l}2<q@8Uy+T{ zU*6}I0uoV|+rdEBJEfo#>Th+A32y-zT&N$i4k5(th4d*~>pKcBZ#rz!x<SxE+WxAS zGCOG};Ton*vURb)cu!c3tShkP9At8$)M-tAW@A;<c_`GU?)Nd#=`nZ<)T1Y{SS!dY zHvqJPapz{(nP}%!6my<4_;G5Xpg!JG!$=ENC(jZ1qc_fpe%>)e$@xayog3zro17Sh z4_m2sCTc}db1WZ}+>C^~bgj^j@#$yP3Z~^!XR%ObVf`HpgoE0R&nHeFd-44E0C)B< zjVM_AP8$n)6f>P&1`?WA(BeGpbf2V74}Y!Uf?|PUQ4lD?oU0N<nKoywdBfr~6iRL) zG`B^p5H2FDQjK~QHPxc<cu4*x`P<3MT^Sw|NQ%A#6?F_8X8wXrv-+VIt#(c@gXZ{; zGWE%JvH1K5X#q65uwZ$b<nD)1i$QA%+xd^To89>cUpT*pv2jcr5rgVW7ji>ZjPw{= z09}|c@xBHM&xf|1<oV(oTW0-eA5Iw!8l;nSDdnMU7T3T9f*`DglGZ~ZmjZK!9(R+q z=0?66n4+?AT)C<j{;iXas&64JQx{o@1v7fIhW3(hE`DG;>h__r<;lbOq+6kp6z!Rh zak@|q(|V<7k>YuHHcGvBDwHp&CV!jj&QYy!+`+-0x3f`5kH5Jm@?lXu)|*E87xMO% z>FoZr@B^JP8~Gu<ZkjPjmT15Iy!a&Ng7Yj%k1-0)u#!iGVpI=iEhcY0FlCJ{(kip3 z<I9+g03V*)h|c!}@Q_*3F|YToAG~S3<ZMcBW5ZO2aedY77!0g(SyN<d_<6}_77hum z4%W0_Fxv1Ya#{V_FI$g!j*#Iw$B(j>GhZte780f!AgQHB6E|7KC&ecmY$HJ=?<?i% z;&!mB=)IW9+}2tMmd6{-C06a&7cY(MmD5>OPON5Sa@+OxDNJpI!mhe8s!VE8o>vVW zDLkZzK&(EdtJ0jn5oAfUS{utL<D}uxdBnMR#IR4vP*jv3(Vej8_wP}GOu0xZdq_I; zn=Z*kaHldGRa+1srs&)yC{ou*OCbH_=+dbL*l|Z_wD2?j_5OUE-Hv&S)$3j5rl!cA zfhF+db!5iPl<D#=fFKy0KyC4B7csu7)VnLqt#{9FI<A*JWN}q2PYC$G1;Om~`_*p! zm2)oA%l75_%XpAjfTijyI8Oy?kn7K@R%FWZ1bxWIPVuS%^SOFfX~~P)&9;%F$8$dq zDTpgUIy;=^)dA&_lY+vOPm0WW+GPBe{b@=jG;;k(Q5ZxTrs_ooz&to$8ltdrVs(D) z9)fUC+5joms$O551Tg;|(GbK&z&j_mwh((Vn7u1A-<yy(#0Xx<COsyIb`-LLXA&CG z2olmxVh%rMU8F&;eZ+KS2ZF!Q{BEb8iH&KxiuoF~eenhFAWe9&N%d)0pkv^&``ju9 za|fFY=Xb<_SnFW^fZj{f>;JK0sQ9pnt@r9g)paR(*m;RNw3oHo>scyh;qdi&Ueddl z6GS9FX$2Zt9Q#Ft!&^9nF`~z6N&}1Y7ll7eF@O<l+2q$|_K|m<2g6OsU%fuQPZvR& zt)?_y);~`@bUk2IIluBGQ#`g<!Bfpg#l)<xL6JOd4uY>LJAM;m#1#b5V5wHn!P~I~ zp&O_>{Rt=6$rYknGe4aEnVE3~wisT{wlYUs4@%kAf}h6UL2F>AF>eSn7yL2`k>lP~ z%H?`FodpY9Am%XZ!pTal5IgAe9$SakZJWAS=1>70+bL@;zRTdLKh!h!728;-pHM)K z60cIB$O#o2j?VvrHYY?L*fGV;J-r?TNu-{{A;NM?EXr;Qf(tPM`~g)%tT~3{>%}b= z)?h%!QB*V!WnrT?M6PO=WwHSLR98s(rD%XQ#bUEeT~G4*VNlFa?7$!3O91;&iIkN7 z4S@yKIgtF1iZ#i!8Q}au@sDx<s>y#CzfiWoQ1VQ6D%sT)gYUK2RL1}Qe!8lCUuDg@ z(Dkh<Xu10``eYyN_Eb~W%HV(8FZ<$kE0SWAwr-@>z*?kX6*3Sk=%0&W8qjfiitY7# zS|aE%cYJtU`_jp(igde#%Q0SLQgHV6<c56C#kv2ImtThIc)m~O8IK1<`s-=*`B=32 z;JSm>Kgo4@x4)PiBZc>|)gs{YO~G9@{A!&?KkZR!982U0^cF{&Z~jzY+)mifl<-j` z3We66@JaEvr^H1E^Q}NE;&IrVrn;#A(Hev$iT;;B456MqC0l;q(JnHxKqV!o2im)A z2@3>zB-7iKj^xjBf{+1#SYN=i?KcPZ2Ns6FMfH!ee44xf3CeS%(YX(HNWUx{#yYCa zz0rDBbeKh<B6OSGP~*V0oTYu6;gP66C2pFbS89*HUZ~Vt!Rv?5@;OFAVy8u``f<zx zQ}A3HV0Fqm8rCzA-}x~f1zWF{!R|O&r@>o@BIyFSo(sxqv}@??{kUsl5f^7tzPz_U z?(cqu9~GEdb`U4#LBWre^vx_IMB6MX=p1m@ti1h`5b0?Fe^C8^dxa@-eZlGi!!%Wh z>TnMHLOBBY%y-6fA3afIUZ4SAWIm!+-54175ZeevSF_&xQWQo9AMubGn@NY^3m#m$ zM_7UIEgLIF;teZh$-lEdt;wfG-snS0F_*K%JaU=W<kx7kP{99*hS}I&TrN+6N!sJa zQQqw;w9wEcblV@iHn<Pc58?FMU|BaEGZm3lFABk!6itDSFiQ=Zgc3>48o|g5E37Fl zexM%cm+P?W<hy)yz1tNiVbQ1Gf}2hVIaC~Cvs$qvQQQ`n-4rD6j36C{7wj=`S{2}8 zh%g)9`TD4KDBF?4J-a0RT%D-jaSeTXWYTIQcydJKx0{r|pDve8S|@OP^X*!$Pg1EB z^^&gw)`ujo<2Br!3_?SQP)K;M=jaztDEo0!&^RWp#os0=B6PaE;N-Y}@0AxMCvfSB zu(A&==PmRGn1;vQ0p}OB>*e@%rt&(-egFq1_9CjEq)o>TL6j#~txmn$UL`Zl#-5UR z*Z~btbX}lpktV87Kn2416yyrcm7^=zmeiI+mQerEZL5}imL!(2AL7;^%Me1%B#m%% z_Vc}PqOqDUu3@tHTtq{Ol!MihHOQ1rnFetv?)h@vlw&9v43<EN&Oo@3E4DfzR@zXd z(uQGTNIJ=4nhsZZ3K>&Ix8ndQrASFZYsLvQa=k&x5{9vkjk<6^pWHP87tNU<<#jYv znbf(9aSU~ix?wq%gfg$xG5)z_n3hZzD7^msX3Hfi57UBWBt(qgCYjsFr~$B(UaklT zGvK;~>r*jyCsP=hU>vuZo*4}lZ2tB?E#}T`S?wGLf8*?6&X>;<+dwZBNo|=5OQa&R zqKgRQM7WHziA-WDXc_lfJJdi<y>HfY^0~_ymDBepGuYnQZ$AU;_cmAMqMRnoqn|IN za~5cmttM`bMh{(>n++McGkmb4wQi_r&0YN68-%W1mvG?TRPjH;nShV&IOWU&^E6^i zN9yQlA(pw=hwCN^d^ovaLCC^_V3`F4scH>)@R}j$Krd1guI5t9g8NbUw!nfWY|Giz zU^SSQxYY<*gGv!08%d{c{u0<wxP{f3yLB&Vy33ATI)SMfDT<gsykW?MKvn5q>CEmC zqok%mO-#iVmW;4C=~~2oe2uyG*T##|jMb)Jk@DM7S%|93wgz14Twi~sZ8ioGGkWbp z3yORQbnWRE3);vfRE5%n84Fj<LtU2535PUcd}&QeFuBj>ZFsWX_(j~acSh&Lb9Um+ zT(o7eA1e2gH68;%RAKj8K|nw}vrP<54Gj&Ac=`5x#Y}norZph#-64_MjeS>sihqB9 z=LIGGfge6HG&BY|0|7Dp1-ts6eN0|v`}_MRZU}#JVq*uAj0alLfcU^b%>26_t1e@M zCWKV$^}rjGMH`OJ2Cgn8n@k&34ir1CC+LYJfQuyA7b6L#aIyZt{z4om>XYuSQDaf# z+igy&mf^4L>g?QEPMTV@*f)4fq<j@qT?p878)&}aQ&{;y;UmCn;_H^npCN1HfmM5; zN8>u{ah)-Rb*R5{YA;H^=x4L}?7bWTJM#gafp<|CtL8URQHJHfb(q8bfIkzRjPi8E zbMR8VCO%i53l-dWqL7W)!85X@iGZepxh#AXr{ft}G->vWSuNRN5^Sw(N`&AoGqn9r zW?ij-z<zc&G*_dhw-1U(6n`GH0!!?sSW6!D=irB^*<E`YmJV1S-(_;Dhgc1_OO4rC zg%q2^=uocmcEp-QcHQt@7@vyphhi6`3(AbI24O7C%XlY74D>1>BhXKWad5}>P%oBA zee$ustjIrTy}3#J#9{C~Y)5W=Y{|Lsq2}=SZQL~v=p;qh+u$8)mV&;8?DObZjaP?d zlSB6~;@#)mi!BFgbrwVU_U8reVvKW{6N?`>pSwu^2S(U{NFC~>B%(N9H}Y74d)g)3 zZJyx0)xE9r9{sy>F>AL-$z3zT{X(7kOKIbUt*QE8b(Ac`mrjq_)4BW?`0gpA#!?^R zkwYi?Y|@*RgA1-ktcN#ujrZ5qnNnSaRw&rL)@L3|>%ge;r`OcE3{eEXz}`L0uWR9$ zs+ecrFX_+T8gJ`TsFpW^kRx`87d^oqHBq`g#R&IletSSyj9WiXNXv@G^Ckpvi9n&I z4$vcKCa%>x*Oa_^sk>$?m=jV<eMfxt`wX$C+)W^r;@mE#EAT#X<{EyA;<)jHi4*p7 z{E5HWBjD*x<YOpP*Stfnw9CY=;QGa%Z6<)=^%>1}dKxp*&ViPG*)QjrQ0uzjuF1Jv zXGJC_;B;)tT=x;mtF7=;xK9G%(raUopur&}_j*-Cr>VT}>l7Yvy|L{Je$yw0GAkws z({puNd#LNzjcUrfjpn^`&F~20d+V89lIo*6Yk@bmJ9{8c-w}?4V>K=O$21DbnD_uG zx`U<3DoZZ>w^<D7Y6ojtjodijApr%eGh#<Xf>kZ?h1vH@z<Kub>sRmWeMk51_3XW$ z{6b#<!8a&VV#QAkN%kM6l3~F@xvHG1)MqfYk|Pq*WTQWxWE%DV!e=Mq6&5dml`umq zYwb+lUP)?=jVOCLQ_mF%m#R9EP$p2%*EOc8mSi7_=Bjk6x;3lP{T!BBK0TT8hQG7M zfO4!9V@SgWk#9LD;m%do!<<OL(r90XO+K@bv45}y;qbWIQb~|NjF(vOd2j$CdiD|` ztlf(?ok&;wlQIugK=btgjz+pTT6*@*b7<L)JQf)R<ykn{MatsA^5bU7E4cL8PEL8y z4*%86l=KH$LG#~$-B^$4(|ErpT`S~(yg(^-xiKU#v=n&YwnHh?B)qg<v>f#CIbAjt z6P>vW21pQAs1%~f%33&g=J&z!b^+caq?CVV3j<cs@%3&iaCy6!Lav70EP4wrp7YHW zd@sbdcV3pD4BixZ;A)K=Pi1rkKCUF<<13Kk;y@1(`^SR9Ky@L*KxPI2s#ZVQBIaDM zwjv7rIGeD9c99)%pxYEG1fo7>*9fQAU+`x8@}IG0l)>+R6Fti~k1A0lx}g3RIM5<g z*z%ZP06%8>(;_7glACnP7_}~@6adqq0^mZA6_}&IxmpA;=6qmVEhr4nnmS-`F-<Xr z_pE2fYrc#$V!-BUt{8Krxc~(svN>5tm1q#+j|T$?PMrAf4f?AwxMiXNosq8}vUMXb zO`+a0>pD>$lj&N#?|pz-XI2J@AsF-4AGtIctJG(tjw|X1J|rzDx6bg_HqON@584r< zZc|Lq_EOpBkDkrB*Ct?F95?v3fxF_~cBU9v>67Lk8?xJUOB=z2I$RMtdpWW@?E7s4 zRz7b!7l9HmnI44>nA{#J4u~vU5rpqI)&d{OrzugpP&YRq+=%-DI2Ppa{1HI6NbZOV z7w~^1K$(ciykWeO6D3!?kO0V*xT0^)d!C>bR9=OJ1JZMfd0!X>`KADzz8Szf_T3C~ znXIct;U1pN3BZlOVRmTmN3U+a1V(og!1vEuG_X4~b@D>*III1~NmaGMP};d=`%K4p z_yPRB1M`8-@OGgG!g<>(#&uv95$5idQ|kA=?2g4XXfLnm;xA{ydwjlu2#OnDX@CBm z6P0spi+!#h{kf(<hZ2xDhWTfd9frMcnCUuAA3a@hbh9YiM5;H*Qq5l3Teu7qFf-{? zFj!M*t_e;wNE$O~pYh~9YI<C{gK>v3&y2fMW^`Xc_EpyySuzem+avva!P373*kzO% zl_qADVt-W;Q=It8RE7v|s-@)V&Q^_Q!@4(ySBYEcx6a~{oy=xa2p%K;wjYhRLrr=r z77@>iBZKV3){V2?f=e;$Lo@GGbC8v0RKa-^SP_sOL=)`tW?($rhr}C{%F=MY@l1lx zHMwQV;v%(cmeSo`3ck-X3-R*wmleSZnow{;6?L)nx(bQ>1kkf=1LpV?$&=d&9N#JN zkT#PDdb&ZFdgd2!uipR;g!@BtTbKl&Yq0T2rwVmnRLo$2S7<LVR5Qo|GscYw(+M~; z`JqA6@F5E{9PvdTIJaKm%~eOj=vIgF?@PE0r$f;@ym<aAYL=jG=qZ;UEV+?&N#*Z+ zw@G{K8K0DQWq@5(e~BiJ*e$}0^9PQ=U0K%Y;Z3a<9Fe~D>@2RsvD@tE+Kwr2f|e81 zE+oC^^0xGLvMDEMoV3PPxY<;up%>MRqbW0p9*sgXbiaTc%6nWs6u>0DDT?#%zDM^< zh)WBOgN6$R%B>l^?#f*+M$b90FYcN2Lvr5_mcU-jgn7qtHvRI#VQd#aI|3gl6Qly; z=ds|hid)~BrR{SQz<~EW=pexLp5a05jgbFJ^ock~2EP;0Z}f&|#DG67vF97}hW)@h zW2^9wR74!uvp97M*E8dsI<v1te)7os!Jx<jt6PlW5!NBvGwIk7g}6FC89oO8WdAm? z6}(FKaMUjl!Qz(1?*a6R*!9>;kB;w{2;6uscO&$Bo==Vl=lyuYwL=8lCv-==e5ZFR zy!huiUgZs5Qt=-RU1QtKdIbboKn$bhhxrV3AJTRgj%B^?yM<TZCbrBphVh!hBBsze zOSk@m<v_2<+->ef*`D&QH_A62X}V0M)&MAU{=7&Be%INeD`-&=u2<GJE*F|r-FJ<W zuV)qz>8+3{x3agKlm6|5oa`0x?IBu!8}8&wv||)m$zgk@UH3RJ<@01ORv*&UQkbKZ zZfy{tOt4F&Jx3=#pY~UA&gvR}OT30%#Xtzm^tUHcX(ijzM!xP7WCy{w+cyKNn2&qT zcNFx8dVwhWAp8I`>&bKdul$mGigY4>2IPmV;MC7hI5-4DelQSxN>I6fxnfGvt~II< z+GyW)v7Ak@;k<Hpf{^Gy^mcPHMli>wz^R<2@y`;CGj<-SRPrt(_rwGn1Hl`JVH!fg zZp`inHE_ZK2MQC^24OkLV-AbskJp)Xi26(3u#nfWG2BUnzb~fiV$i#^n2v}7beKx+ z1lsxor7CUR((g;o&<xuasjH5wehyyv9$O5llb(RD`tBf!ut2TseQu9B+0kr#+r1pK zv^IVndH_?e4!!ZD6I~qu+HKk^<!Lol<^|Z@(j9K+VeW<hyp8y;|CN7BcMxXSkGKTm zP;4~ghq;LGekNgABB`=saO7DCcclS@r@cR5a4+aG`Wfp|wTU)Y__yS73-hugJ-rxX zd!kW`jK9r|+Zvm$e;YC)9aEx8n=8wiibmj^OJOneFNw-99Y$$q0u8B~T*u|2Qwc_^ ziA?F6O&xfDs$FgNV{eo%z9Xq!Z8YZqe#kjT^DN<5TzWfd+BMryU`~ry`E|vdh^vXW zyy^w<S@Qtu8kaD$Wl~10NS?aE>WoEq=slB!NlQ#ikGxR3$aC@ytiRrm4@;Gf`0*F6 z2Rn6_6BSmEXX&E2NVFqL?KGOhnypc<6EAf|rP`0X;wmy!tPo7orDiH<Qsye={HQRB zrcEVI2D>VlDfB8)wZs14g`Y`>YFE8D+t!j+#PKjUg{YS{_IVdIx7*Li&5~fuqR0}m zzAGQmTp66he@C8Tn*nY3D&PF|^*Q6OM^3**Z@4PFG*A}3z6qH=LB+^39&TZ0qt}o< zv;8z6To1+@-PAISDX=w5+oqD&QnP6l3^Ou%8n;{7Qt4ue7$>LxUGW)DOnrV+Q}yu~ zmBml8#~&{K@(ZNfz1w~c8dOxWpM3%^IG728XeIX2dU>7nZYF1`OEnd^%55d~kl?|r zrbMt@<3mVj`9Fske-zcjr4GSpLgNmM)<fIEWDOkGL)b#=46M|{cSE;^T^jnVfx3t0 z7+S1BxQA^UdaQvxh4vUatU*16F=0iZBp3XEOjitpP7nPw4B!Mo2n_)ka)PT5pvexw zO$`<}V&Nf3^#5^0!-F5|Z*auJL!{{c?TC^MSJq$Qh?9+A(VyXnkqv*+-=P?C-yiI4 z&H5;A4URJh4nb!r0%fZSujiN#<>xpM!UhllAr@tXx~~U`uE&^(fCUJ*|D+F>0Vub_ z(MQk#q}yR?!)*ZC?Fh9IxB&5XX!~#-fO<Q;eFP6cz8&@|+y|iFj(8OT1rYN5Nf0gp zQ1XN$h@b#SdBWs|TL82?5pp9i0b-ue`r#S?HBXjqh#eS*khUQnK+X-lCr3Bb4(xTv z^AHpu<c9c#u^W;wD1LAZaBu_tLh1{{|Bv=HJ9q-Pzkzrm=|(UJW*E`{RNb(>aQlMw zLhlAU40!;$ZunmKKS2C{3Ir1lDFDiDSYEh3e)vQ81se=G0NQRKKM?#80|EsG^8m9q zm@hOR@LveufdPYkfZZFy7lu+Kq(6+Y*i*&`_Z9e#KVdb8jqnDPbi*f|AZmwW9Zj~t zIYy=(UABI-4c9o@Y(egZZtlCc^IZk<?8~>aTm^US+qd&v1^Mjjw{u*DyzgVhnLtl! z3W3R0?}N+l`?m`a1VZf#c`_0NS2@CzIYC<7D)Pc1j{Ulk<b{bJHM!B`MTj3my%FW5 zdgRWHDFak^;^f9z05Uu=a$`>b9hyV;bA#OM^}k_s)<zLF+HtyohKTRL!U&tWF%Xd$ zM}!k6P)sMf+;~htm?yG+Od6oX6I(w{1CZj0t{=M!Xz|3?kLLiydZO;cv;k^7ad+Z8 z06CtRJF(Az9#6uZctk+R3le`!JfP?Wi$6{Qko1DaA3F<Zdcot5X8=UMp!CMn0jgfi zePMgicf%dVJODhmdN20Ah`pG*5!Yjg0H|A$7b9Qzf1^JVzc+S^`Xcp#=8q!~k~cED zcXA8+BIb+V7iutC2hg~+d2#!I^hfOr+a0|Ic-;ED_<ewWAq9jAjOGCpZY^G1J|MnO z0>b)6cL5H!@E`bJ1pi*tu)tp4EyIh(2ksaCchL86z+T_2z>9%2G7^eXCUbHL-jP)# zjB2qFPJxp4zZG|gn&MbXlZ{aJl4(nqjo{Ye8cUmv@Ey_31@~sYOF^Cm`DT_&;jRVy zW}ZtSp9TG9j!TjE1*}+=-+xt!Lu4x#z~vVFn+5O%p%#Q(8S#ayETc-T!p%<=xnmH@ zegP%9qvA?UfSTNKab>7LQSRUJr7A#G?pXOU7N9J5^h~J>P`7g4%Ty@`XNgpd&RQkH z_Marcxm?1}d7_BzP(_efj8)>kSunaeb*2m!DBKxIUn&Ds?u?-?qX9~HM%9+u0JS^g zYRhne;+?<kOLc(iow4?1JV5!*=&Mp6pnhlkRT&ge$UBmtR062v9Yat?0hIEN$}P13 zYI(=ymSF<LynpMLY5<p)@H1LXAcRUWA>4oAQcgO!-c<^e;jOA<AUqGbB5C^zMP;6O z?HO~!B<I<$%QFzVTgM%>p@-*WH(wHowq-r4&E}|dwA5}^t$+IJb}32PSEayTxbHfb z@3pcNI6&mMj$Kyp&X!uIqLzwul`Ztzutj8D`R?w8!<|6o*d9uyG`zcc6acwajBAYE z;U$>L%BmSps#5EM<@Hlh6oBoq_MJzXmp>dzPu;e9VPITpQ6E)fS5=neh_Mzf|DBY) z#kE&CI#btGv20oVz$`wm-JF)0Z~Cwwy}$HNx6|Z1(m74tM11X7oZ2WjT8lL<#~9R> zSih9ljNH6;XSqOo(dsgAQKi9?&xBt_Ofit<?(ODJY}SwvvTVaxY)mWMcDVC_mTx=o zkkC;VnW$Jkz9@Be$=&|>%fO6p*<nq-z`~17tLm;vU4{G*_1LYl6yH2J?x;$(sOTkB zwEho~s(y_<Tjzc1E}oT^Z)k|V|D5Dxfdy~0T>q$JkM887nJ=fm-`sDDg`61e8k{}G z`>9v^#``})6gz_nC!#`fF-pL7zinD_@~BO&Hr&-;HY6hwgPf=E>z}Dv{lVdNssh0F zy~uE~+JE(Y7O0nMzVfYJdwB@!iqcsR)DDx}4^K}Te(nE4A-r||;ZsxDLNbQEa+z<z zHRIee=UAQFVy|`Dx^28kRW;%<tl`x%E^fjWUOex(B?y0C@o?$S<IgvAvaazOepK$i zc-Pds?9kbxJ9PP|d0;zh{j8>mm924D!y}<ZxXG8TL5;Up<C8194uIv^GRCA9Af`Es zqczIVf62ZR$->qE`j0(cw%8g>VjGXG;^1eHX19qvnK|DWGdK8c;mYF~m^km2)N0G# z+acU}PYg(|{q}wgT&0F;lYKVrSRjl7lNxi@9^vdHWg?@vcaFqzy6{h%&cHL9i4I0^ zunBdDzvHr9I&{JlzVJ_-=$SEYuwxP7yA?vg4<$dSM|^QS>cupPrVuR(napy9y@iF& z*m3l)U$td+VLy|BqiP&^Sr`Z9m_Yn-#`>yUkNa}-cG~HjZ7dSkG6IELDI8(8bQPDi z->SP6)om(@U@EphzTquVyJbk4Yq$<6@~4ehvUCsYYDLX`=Y(f>B2;}2z7bE!i$%n3 zSG^`2y*!wcqk|%&^;%qCdxm+4;CJSFXCtSu;x8C2>3D^aJLB&)eeU{WRiT+Ob&DeR zb*I`{|G{yg)xF5QO+9pX&p~$!%Ki4k`{t-sMGw{RX&VmCDT&xCq{;E~y>p(<TJ(P< zHd@bHy*SQ|1)U}Y!`W1$A^2U;M|q|x<a)SGZvGCLMa^MZV3voO8wc=n-@cq{D z^@7Zxlcxr&EAiT<$1R=ya&VTe?5cKB+SJ;?BhciTp~O27d9o?zY&~Az<hOBf5<N`U za5P0`YZc?0bqaE`ubO{}v<ZP@&vr#EJ0?w8@t!6z8H&Tb*{kwJS}&kX#Z-YdX&v}2 zO*O+vFJU&3?8N<tZt2(N9)9yZ+!N_FoED#X+Z%k!Wrg`FLNuKgN>jCZx9f;keo|<~ zil$7BWv7x}^->yY{Ab&M<bEG~_*ww&gPB3t<I6sZp?@G{bpYGyzlG~X@s$SN3ge>C zA-*>H_b7*h`X`Tzw!zGC_{SwFmVX8BH?Qx_6Fpe6KXXQc5g>dSC)2|FIpOG_Llzjy zAr$P53h7~iWY=cF1Pr8$`&G+jxo3wPc;~!T87GXG?<5SnD0jz}TahBLT^$)GEXNmS zTvo5fSW%e6bzGAxBRu$loav+!B)xs7kP;2VL6V&p()C6fr8XsJrcP4kRFKHKlD)mH zW36##Qqcxkl!!j_8!gW6t=5$C`OF1)2f#OTy04qFwZB$z2q<!mndpgy))(YYil$Kb z{7QKtIf=tfbC%A53+v=qzRlHJhN4jAB_b&KsZR$W%g{b?f9%19xF&ZF6SO1?E7_SO z$Y<hk8+Pp&>O;t&twuT<gchV%|C$^Xyeujl&)yN$dANS2iDcI%B@iMW<Bz22+orP` zkCD$F%tmrnk7*RD6EL7BCKFnK_dfa7kTsD(YF(qX)^_WWp}#KG`k+C`HNNxLJYh`4 zpDjI}R1mNqvS`8AbTv&LaH?XOc;Z*ha5SpZ%hl>~;5c*ENEE=ZfA)zq*8CZ8#0$}| zor^Y6snM;KG=gJrW{*Ad{?(bJZ<rB*0hvc|zVl~jifx_tLur~gr|$uBAiA%iefOzG zzjb7{Sud}eJR<J)?8R~$;rtI?PioMwf&<JXY69)!cG+frnONFWiJgVA{i-9Gw+ZyT zlUZF4{_$J3N4>6$y=Y{*8|KT-!_@pPpp&x8KY|ZxgYgGfzq(Ts9l~Usv<xvcZ!;eZ z$)6Qgm%G_mzOr(H7N^>*3=Q|~qX4|Ok4XkqnWEbrn~>>AO|v9ZsgUe*QZ5OCj3PM> z-8;ci^6--vmFzz01Gd}o;Wf#`_5Gks8WA$8zsiy7sNra(XlhjC#pzRGe(!U)Y9_ub zE1dDNFqVz9dZ2PJmdb)jKQhtg4oy4Nv7?dQtWt_8Wt61MvvAVlsKnHwpsB!F`N_k0 z@iFJx14n6;v6O!r>mnTlW3Ad`5iGU<q=RXoYf0M^|I1-a39F!uY$W;{OnH7Z4TQEB zb;Iu?o@*@4W^%PV(<ZMQ$3r)&`Dr7jNplMpq|WN99tls(ZmKOJ7#;`*iv5Eb-1O&o zLP=&YBZLvmAX;cenz!YLte|<yY!|u!s-hZ!DugQfe>7pG)U0YM`u37CmX*QjNW-B- z!<gu|Vq(5Z$Ofwa>1H4e7ZZ^~5SNzA!WcIu+NT&}ucK{65&jgGHL9m-$4V<C71N4} zLi_JgnxaC;+Cr1TVooDYWgJEc*BnHW!7_#xp*4zum6DQ_4n|_ZGKLhHGK|EAV+tWQ zXB>tL|5vc?zk|>Q;#x>%Ldg)s1dM-!%YPP<L`D&bdW#fu{I{VFW&|hh#4YUlznM}Z zS)oW^IMq)TBwFvP;h*&21BEU!NF<%ag}7=$su2I0|LOm~3=`GQ-{9e*KSW4^1`6Ax z)zCMaS(!?0Ml(oys1~xg36ja-^s<;WQ;daSj#)-zN`VehU;n?xoT?+WM9n*_YSc|{ z!A)z`yQ1nrStFyf_Uzv|f{<7BQ*Y_j#m?OMM_XkDs~xIoo)WJz*6ZZsoP}<cOb_XM z`TrxR0X4nQe4Mk=y_o5teJ^jbV>QiF<5k9X{l5jPOl+jaRu*E8bLP8QGBqUD665Mi zu%~&7yewF+|5wyQ{C>uAM{Am=%FBZ7y81Y0xw|RTL;ZdxN`;*5w3<9;xwt9QRXu6O SdSQM28?+M|D(2r_;{O0|uQ74} diff --git a/site/fonts/fontawesome-webfont.woff2 b/site/fonts/fontawesome-webfont.woff2 deleted file mode 100644 index 4d13fc60404b91e398a37200c4a77b645cfd9586..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 77160 zcmV(<K-#}|Pew8T0RR910WD|%4FCWD0*06X0WA&y1Odwc00000000000000000000 z0000#Mn+Uk92y`7U;u@35eN#2nJ9<0SOGQyBm<Rj3ziT71Rw>81_!itTT%&fM`8Do zgetlXfhX-f>pHa>CezJ5a<?i90vY8R@%CK=bO+o;s&1pQ|NsC0<fJi1+Ya17;MMf| ze65NU2xgOQl3<%8*pZ<?WLrW}5hN&ERXi&wt++r?&@~?9V1cj>+CKJB5E?t-D3Q@I zv;Az_{%F*wqQWVk+*x^)@=9sx>ldws&U_`?fwx|)6i0%hGq@6No|Wjj+Lhc2#LbXI zik@&>S#lthOy5xS4viawbfqcF5t#22r#4c;ULsQqOn&iMQrAORQWXh`G=YxhM*4YN zTfgWxZlU6?d>wP(yNq!jqfNVxB}>Ww7cSen4lE1$g!lMN&~*PN_7ITCO&u%|6=U~^ zD`NV@*N5j%{d4(V*d&F9*Lp4o^=-wV4E$&&XJX#);dbqZ^8pUYCyEa?qdKs=!}D|N zZKGn0G1#bWFe1l-8nC}AR*a~P9;0KUBrGsNR8Um3F%kp&^sGD!?K|!B(qItgwkPpO z4nOg8&Z#<)4^Bj%sQjrANfD$Zj098^i(7$$Vl;{o&HR7r?C&hE&b-&}y`y4mHj%mu zNlfW!ecOyC;56fuZ7e6t7R&P^z1O9)e^Pe=qGENxwk%7Q3&sYU;&zJz+X!u6Ex^F$ zTu6(Z`;JIR{;Knn>IcTcKbV%&ZSxB`P>8MADLLm#sD>oQy@;IWvGh3j=*Qa5&VIQ& z#BvplZofSw5gN50lul%1ZW|#duBPzgJG1nxIGMaB*-obI9wC1%7zRoi%C^%k;Mn?+ z?pUuq3@j1^4v?E3B49cgqW>EY2?-#3jqje^;JgycOCcwp0HG~LNR*rji6bO_n_6Fl zxt$OawF6EyR#iAg$gdotjwKXO)cf75+S~gE2n>cpa0mh<1W_5Hw7c36opP+~qRPFS z?z(HcYuX#9GugKj(K=EQB_0sAfiipahu*36k{xIzyD2!y5%vK1@c|DQ3Q0^$kT!Po zBklXM?*0ZWJJ6;!hoDZHGR|mrw+{{o{_lUy{_6}+Pm!l|BNl}Q;&@bv@2Wy(0-c_O zab6Z9oUWgiKYRW)Vv0%P;3X|rT9E6xVx&Q%6AWJDG0oX-H5vJ?>5A8;PEnm%C;H~y z%@URb{E<@x+!!CGA#@@j24G?{>Gvg*2lVeVHM;^7(Pnl#tDV)(Y|gCiIh;CbXJ$WV za+~#V|9GDufDe2U{2(L<AY~Gi91@gT6ez_MBzGuUa#N7#b}O4&*&NT-arRB3>>iu$ z&FbBmZ9gV+TlVF2nNyNeYL2HloUh~eKdpS)>J9Pm#Xd(4%myqFVno%qUa9n|Ua803 z8#-)?GmgDZL7HHzH4B_FHnRat`EXP62|?edFIDRb!q%9yytA|?Ib5`-)rNGqg%GbH z-}d(Uw;KH$fouQ<g&<H(QnZ?oIc|Uo&0rN06DZ0MK++&cc_~fOWKGHqkQ_DX)T9$g zC(@Cc9P({bJo@O{aVBfLDO}HcJMJ;X!)^Hon>gEh;fvK+gfZPMGsl{cktu>gD1?zL z`z7_05U{qkjReFC1qI#x+jpODe!iG=?eIufIBbyAS`i6yq~pK;J!P{R?B6jf<_85Y z$&N8sKi05v?h+0-IZ#Z-(g8koZ#f{v7%?Dp!<ehXbz`TUk?zDmv-w?16cAGHe_~Gm zyOMVMleE+ts;DTqB8r1~>%F^s91LTw|BvSLb7Oj@878i9HK*kSp)6{%ZXlv-PQ)RD zE`x4f_xM$H9{@mn{1`uWwLbR;xgELO9FcMuRbkvnQXmT&j}ZE~*Z9?u0F(1c4Md6G z%ZpLJy?$`%3V_^=J3F{;`T31Z7#Ad=bomK731~(`S)uLTR8OErP908ueHZaDB4D$q z{GZri&j-sW%|A#W5to*SAH-ai&E<8<FcVDFwe|5Qwe7mSzRi?SVs33uznJT`-TjTf z0vJkci$4MZ2%hD0tLFuXFf0x@jCjdH^7{6^LK(;9kw%FS6rW$;8&YXn9r*vZ)P7|n z6>6{%v3LDwPh%=3Mm7wrS#iOV1$&8oKgshx_jMlowl4ED4$f#L1!t6C1g9p~=ODPt z5-F*yQZ*RmNQ`~4r~k{Ouxs3@+Z>Q5N}1kIzW_;y+Y`2(U+=Sj1(9)2Vkg!}$DaT~ zSw&5w0~|KUc7%a7st`^}4doR9Pl!$j8b%9FcqlQFIssg|->XC5YmQ@}VmJj+^a&GW z;TT&?6ewkE94j()E$+}^)|h0Xjx{@?P9)U!BBDsDj}WU<M!Ik7GHIwHirHMVXaqM& zjP{^BQ$tSztr@tM$dE{&9^yrJAptHQ*XjKsLQ@h$s%IX=orz=P4!PuWD%GcRi6}K~ zls$3A%8F&8`|(+k3nc4YJ`iS3IDm1NjCkrJb_%67b#3nn;2E8;VB1bjElL1f^bkh3 z$8zaJ0)QU1TGKF-2O}ryrHF3q5z<F$OoG~mSQHz|jG{rRo$+T`bD1i#S<2i8$6ndB zm^&ia9VBRCe?7St8PA&&m@wXUNIaKTc7K6mn_mE2S&9Sfqv&@#WI`4KgNp-BgM>31 zAtcV{=d|bI-bs8=m>_-=CKKcXWW_GX0~^$^=>jcb2lM)283`*Z!V{7?x-M-}_~|s` zV|lNhxg(2J)xt(s?g(|g4crMAX)o}cuastffHd9kY=i3#SX1;l!-O06F-4v5y)!_N z{n~32h};!G7bhd5ytZSkz1eQ+sUW)X74K7DJFF%9?n#Q!!7ID?F7r$p*h2z%vFq+0 z9=`hOhOu`E+Rawmf`Ea#sNtl*!}&#cW`0Ouz3DI?ydh+i=s;0>PiQfT7Z<Te!L~-j ztJQjQ8jUye-HEod#+-Bya(2w<snTnmyM}0GQ!}^Qhjy7cgmHYUPlE54ODUCAT=JGd z(j}w0y8^^-9(!y&hTBWH|GJ&cj-sTFX|s><Nb5&{NrifyJvN$GS8p)u*$aFX-X%fz z8fs)jDIpDjsc?6$$nIZ)v+Fy!@uu5jmqa7!|I2|*kigzfdYtkFw*+voUXMdWc;3Qz zYAJ*@Pslm@4WVGs(nUmA?s)NyAl;HjRmi~Y%dZZ^nW_Y-@7CIhoGv8|x4*W|h_QYQ z<MH}E{|FiuLlbYNDGf%1H}lHpBwD2XD7R<9>u*A>rw!Z2oWMZd<tFmO+GXy02I}JF zj-#LmY@o1aieJW3c$rRHr(F!Ln+G^xEi;*Ow5Gw5dw)=B0p1z7S9N1K?}fOose~I_ zJ{{^UpDskA+>TlLANQLT4}czIhYZic*axDrD;QpTldic#?)QnYZQ#V&@GPdWKu$ce zkR96D(D?F+uOEL7E{&8{@#anN+7VOiE7M#=o-3l-Qlfm(Hnj`lCvjX<;N1eImGc}P zIfq1q23S0QB<*mCfZhipyXl3dlKdo_(zgrVEctLByL0)aRMXBH-Ttp)yZ_WqYe|tF zU*@4;)#eID=!hTcSCgMs|CA-!(RT=~eyOCyMAVSk!pq$%^Rswq@*cQ(TXI^ehX9#d zQzf)Vo7@<4U`9OSg`<VDtRSMnR*NhC^C2s(miP6FxY&)0IeDLJ#ldD`kvKBm26Nth zY&%;iWhBJ-iBz42mTRqplHe(tc8|8HIjjMM;~S@Y({&T_b&qt9$lkL+MdS27d_(ry zJX|y-2$xcJ;S8m=>E*=es@n8G*SbT@I9!qVekl|qYka=BE@A6$s=C?(x-c+DlyNW} z6eaQe@Drh#XmE?Ex(!VKoZcdgD?X0w=CviN3tmmjikMECbJNHMagMY-l@hQIzV7AZ zriQRf5j1k=Eh_KlCFt5{BiAK6a8T){lxWsNJ@?M~+S(158s#PwDXC&%gvLuu_&~q; zp5%18A)<aV?NeQHS!}vmHqbNCo=V<tw3tt$R`ol%XeMV*HIh<Ph%bHl!P3TiuBGx9 z7eQLmCWjeQ^Wxcb*vk2w&jSn1uL0?zQ0nVLp`;X|FcS#0wnQxZaAI)Kx72zmryyN` zN%uY)01i=HLTtezx|d!vQa*D1D*L+xLq5PB9)qXh7kE%rx_Tb1eOVDb0uS}fl@`2x z_0YD*5SkeZ=M5Y-EnqJqNDB&Yg(i|jmq{@HMKV}niIpuMse2AJz=4@O4J>_>(Gy@` zHu}fy7?5gdqUqRaZ9G+VYFVjT`f3hBTtJLx%QHo4W^k7Hn4dbj+U@EPSKG&~pSs!K zvyPmU&Tyr~vom3Dulo^!F^FVgi})a%1Gn9)rTvJRN`lw2KOkz(aW}5MO~dBSW@edL zwPwp4)N=wJup1<soC$=LNd1u%=B#X`mi4*6z=)S9B`%wC2IJCqKLhf<Z9RSqf_buI zS&?FADLl(pgdi_gcoy1gskD+L=kt#YLhy2a@rD7X)9^VH-ZqUe(I)<xUv^CZS;8fR zlhU1g=0$&rHax@WmB+&E+a-%7t2C{Pj^Ch9-azm3PT~0op@OO+TDKVAPegllGgA+Y z$5)F@u$5A$?K5*c!}S@bk=U~hqkyLIuv|$FFpaA&pv5gR-L34;<Akj{uL<mfFx3vW zeNL+{V)gbKj!t`oNtv^JVqbE^W%!+6a8qf6<H{5dJ`yL~cdRx1+xk*QEYXLEB_2~* zmV~>;S7@U)OkZj2gQGo~o4#o=@iYEeNjFZoLvW2r$?(LKzQYnI52$jlzP&K3-Fs?@ z8TYz{a*Ip6o|)y)qHif|*~IjRGj3tOR55>Cr^87ZMJVZQz4x-c--DZz!bJ3J`mBFt zv$MzMB*TT@cUYc?%vG%XC_t5juJ=v#VIpp<4lLv<c1N=vmK=%9s1ta(impDDN7)Qh zf)Rs&?!9Ng%!wkDQOkCuViYhcdfUiQMf7>W$%%|VH?JfU3&D=q@FkudiARUh(d2N+ zWLd~2X5t4S?fb`JHk6Khs0b;)4m))>Bf>MuG>~md#IxJ@3UBxJiBI@&t;m6*b~tLF z>Y4m_C`-#PTHIv21B#D$$;E^HZ8uiYUtFhV*G%O%3~-xR^LiE@?1e}-zAdW`mbEM> zF-u5dt!0p?EOIRw9HXESaG^}g@5b$*Gd<>1m;%N!sdSMt*}PbmYdWd4wf_iOfHlC+ za|MYGa1MylQ*%_SxCI*3>pCu7wYNkflt8fcEw)9s%#j8m5R?-^jqs5&y2-XJ@J1PZ zvCEQxGD63Ll8sRsnbjBI1u1mJ!>4@OBQ%73++6qLsDSXuV7F#t5G=NzBh&|HiRm#q z*)7%<tD(7cc^4qtUHgZa;N&u|T7Cktga9rn<LR#R^t0Cj$Bf|C9<`y|B~qnJjhL&y zD0z?qV39bMwnG}%$JUMdx6Ai>le!&>OD#^0421Im4)tJOE2i~}o^A-DsEaeX+t0KZ z{sQInfSneVRDtp{f^<>g*rTZi2sAuCI!Z9Zh$ZFSky>G5VCcOA>UPbn{DxunR4-Zq z0{Rr3Vcwm`(344N37c0jkQV&${exerkPtp8!}^!LNFtPq`QzzulIshDd^c?rMzvmA z&&_^jixC$vO7ZGm0Le*_7u+*exgqHorQCbdJY~!;J<f1o+Sd%;ajl*JI9~SD+0(&G zILzx$%{lgWFw^Q<khFXrirfTtKap<36baxAGnav8fg%0N-?!0zmHbqSARnjX9s9w` z@*qmno2C1N2N4|KHP;KlaHfwGx>gCi-!q5HtGLD2^A9dP#_`PVfh~Qf+*{6P<x=4_ ztMi@nZtSB%IN_1^gPAify9m-a6bV^pp(8NgCb+$?Q-m)rW@lUq8b#t}=Y0;4$SiAz zqLk^AXZ!{f`y>OoKUi6l2P%*Hl&QKAyfLqkaIKd`D8JY1@={Zhq*1z<be`@5HY}h~ z<7!+8@Z&rPFI`TqbNkI5r8~_h3>ZjQU5-VVG9EdQhh(N}S^W*!YLJe?QZ~`l?e_yw z5+Rt%0P61dAXbLEnF=K$2o+w?V3$raPx6eS5Bi3KtXuINb~@n7ggV*iUfP^;*T3fx zK<d)$3XMj^U#`~?<pBszW6VO*7xDeWIB=s}X|JNS*Ei4Zi)ZDn11j#1(arrVgF~yG z8p00YTZcBZRFXfsy0sRCCGvXv#XtcD7-~UqLQXaCo+30<o~7XQB9?tHbxQPykDivu zoPIO*9@!4jdwl|RSN4qH?oE>(YWg|IErMMW^{br`nI~*hvLG+;Qa(JTE9Xz2mD|`K zWkMsBLSxbz*}wwmYD`=a5~IW|zFKINTi5zYJdLXS5AlQ;aj16QewJ%pn@7XW)l@{k zKU1m8+14)_#x2y>CEb#Vl-cMv42b@BrfGab7RyPY#BuR=W2k^v0h<(f44SbZ&kQd& z1c7+0f=Ev<uI05I@NZp_o)45qU5a^+--GM?hx;RihaWeok7CW^B$IT<9I8^=Bp?R9 znCX9A<FsKn8_h<e(XOwuk1TE>a?9UId@{fgyyLhy>XLZ>Hs_gVQ>JLK39^$?US5+# zF8FwgP0>wLKjyriCrA1t{C?ppovgaV><D%=iew>1c~smv@h!4uR$(`2`$DeE7c~B> zpO)wsEU7ZQ#)-uJ6()96NKJ8Y@H7-Z0#aPGy|SvlSYbSo*fbFCmK;D$X{<=pL|?w> z37bU`XR6OqiFvV2n$yv2RQ}kYO5LsvtCo2WW6I7VnMg|XEFd<Db@ngXw434%t(^qG z4~vxPiFu9ymsZQ!=59)E+n-2WP{mTCLS%f?6g0VC=#C_7?wrfUz{W1Qns*ZFmdYE# znG^v@pm0;Y{Uqf49F}~2YR=a5Imp`Nr0EmG{t=;VhkS!E(chS!x3~OmzBx04IldnO z>+Y<O)HQJeQ~G>{o1b`B?Ku<L-0aQjLb8@tS|PfbxXW#)OE3FOZaZnK?sc>6B<2+= z&U7;n*3GsPjMqSY02HvKv_gCJS?}Vw<e$`$loWcaGD_!lK;L2a7F-!k`-z7RQa5k2 z^B81$=%5v-pqpj0@;BZuBs5)KmBGI<Yl<9bB#aY193=pW<uUF_=R(rhgCK*?NPGwa z_)acc$xU1fOqY!NV%qK6=`l@b=JFakolaIRUof!%ES@VC;VihMXx(%-v}`zybACxy zUJXv=%e+nUvcrG)*pX`F+}VRIq6u%@uEnoW<iT$cTWx=cplazoas!<FZ9egSdNd?W zn&B54eR?3(@0);g0E=iMBA$orbvL(Gds4%@-KGGzPZ2Gig+w`4*4d?JhDpkfOYBT% z+qOaw|5OjC^H8-Xhf5dF%FL$k3J!5s1%%d;^D||(v!xOQ|8V4nCXkkh+meaqOp6$3 zaI>nX)lP$9Q?8>7cln_TCYaRXg*#;^hb%1uH+IT+qb<MNcO+M-7td~3^HraohlRL_ z>i5QUIEkAPwUL<vS?3Xox<5-zpJQ~pS=fWVa4OY@j{1oOi{dTqgdFb6(2-_LtbD>- zZcK{joDF?6iF-BK80ny(qch>Bj2#sVh;E9olq4i9E2BhC2h@ZuNbOcWnAb?Aj+ol{ zPjg%dw*~)|Ezvu`S2h4n_?1nG-8izHMroCi)H}Y7r8gOC^D?nEB?8ux%nux4T`W2w zjmomxy+te?pWb^_g#G~wZee%3vH68gXQ75Jt@23+IdVE`poA6wl8hR#JV_HpwK4Eu zBw$Qpa>tT{f!Cet&Rr4Zc;X#7JyIEVCMr=i=zs(;dVe1C%lLUbh~NS0gJ4a3_SBi0 zWKV|KrD<BO;~B*yuI?GTD<d9DNXSMCjl=20>g~RR0H=-#?#LMUi65trDJ==U20Be7 z%Xwp<u0_BlL1~qe1u|Whm<Y^l<+iAhi5!|4%{Ych8oYV3(gj)<M@qAU8lncST-6>j z8rGRuVi><nzV`5+Ny0%b{YP?A03ppVpcA0mOe?@qXsT25S=7=K*YZ=pHT_>6*eIn2 z4sdTqnx|BWhY_zMYaCA7zUpjza))jPvt-vupa&k7+<6n*ist$5<j5x5h8K9_aInGm z;pJQD7C1*5hW^=PbgZJRqp^DIu!GsA1_?N{v_ieEOHd*}GG&Zq1Ll+qta;`hc;}}c z+G@SHoCcLjOTvg~!C1wtLhOC&5vDc>`NN|BwO~KBX%LYryjwYCD`L@BOz&Y#&6yLk zrl09#3<5$~a4xgYhziDTTr}+GvxUZ_irgNJWb6?<cc9-_A_fJ|(?1}eC%I205|3d4 zG8s<)IUkFyEV<vGp|%g|A)CeGe=gH`f0tVMuO;#{_Ep67Ui!;*pwEWlXP_wpO(m89 z=R@_*8dmA42b!MBO(A;JCXk)H?b&=5M8YhlY4v_mEG4n}d`}|2``HZS&mCx7Gb?|i z^88_|y&9b2=LCR*o&|`|X*iRD9$XY810gY)Op4`UHgmQOdJ-#5!cwhb=h7En3TxoC zcFStS-RX@RRrs&yqmnIx!)jRiM|2;R3z)yarj=rteu&GOz>^#5mb!Oz(fO^4&7G%H z5^GS_GXIRAC_Q6#bn~Jjo?A1S$rmQJt!U~*P6dbvJ-70Rj*C#qoAg1nM--Cz!Y317 z=u#u7#!Wgd*X$9WGk^)j?$&fleixkNGkSM;Ai$K^JD4}R=>kur91A#{$yq51$wX5{ z_^yQCFMy;I)XX=RX%FBGjUjh=$~M62v?QPtjW|Ux>QrIgjQe~*2*&>nXZq^b5AiNL zZOI)6wC_3KIl*(?NODXbHzum22a=JFGaEv41mKQ*TW=5nCK7LT+EZuu)vXw=D|?|q zMZe$WYg*z7q#{n@ie%~;HG`r$nwUvewW8XJl|HLR?P9D;g~!gQW+^ITmZnEFJoC&$ zpqK!kl`d!W6<q|V>#u8;k_s8NrGXb9K``UKExyy)qZX#Ac7FthR3Nwo1`lL3ODL!o z#aVG+vZ|XXb=~EAEWJ7~DkOX|><)vPi!TI8y2~t+U`4!!=-3qTcu*Uzvm<eYu&>X| zU;vxoFY7w$fXLF*)+alS*@;#LhY>_6%d`y63v$W)<B#?@IlgzIeOm$ghDNN2%LwbD z`QDrt&bJPnzM&l}<<i2MXr4<*-dy0vnpKL99jN52f=a+vByd$~nS5g~%kJEJuinQ@ z^XoyY+CS)4<~^r0jq>kPx*5f^bYS(x#$=iQiEsSbWTj#TRZs?$7t8|iN~L%c(PyNt zN>cc8olk|i&vOa$9mc_tq1qTUO?Q~7+#U@N=prKaG!!!T;ppICO~e}UM7l3dA&J#? zf-}{*xAKAEE{qjsE0aKYPnTB6aq63DUe`n4s;NtDuJ@l2EaI^^NCY{ITBxi%Cb)05 zg&!!<K>x67sqr4))=f2=^B;|&U9nAtxK%O?JrH(qLN-KLYGA2ys`5Pbca_F5=9yX0 zI@KWOZ;?E|06C&Ni~*hajz+-M`jaFaJ2KXs*J`w}5c=M_?075|63ZIOft^DH#ZttH zbQl)6uo5JL99BwZ9>Hda#W}|*0Iy-0IZ%nKCgAwd#WqiGzSaX5Y^gk*)brv38S)wL zWOF?u0W-yO7LT=1Ezn{_pw#>#jSuWwI<R-ix?XDeiPM3*&{1V-V2q2GRB@68J)-)t zlt?C{Z;u<H8*9XgOhn#Sp-=mU!ugcz$|_1Tb6XkCHl1oQ@IcwvZgM=X&c54+h9im1 z(m)A_Fu$8)1_`2_9|f5|@3oygGKxgi)20c(=zXuSRCCSQQ_lU+BH{~D%?hrSq%M8m zZ(82X7~_ZeMsF&Zm8ad8hqnK>mbE(F^wt}}lf1z<$?f+@!t&&enhvFSp|oAa+s9!U zHXe30?GjS`pv=ByF^BCWSWJbRy2A=eiD6-y5fj~pEXMQfgpkY{A~P+|N8}+K%cVH8 zxAHg&eBe|%Q{GUMi~=9Hw)OFF98FTLS>9sw=B0b@E4xqqW!sxF_VU+f1*fUgb*|_4 zRz3PvJ}t!oYhpH4pAwRi(5Y}*;!VBKPpDx3vfLzB=tRMJ8;%jV@j>6aqg%i<1&#b+ zk^D-3Kdxp(KRuW4k%?rmuP94I<g~Fm;+#YtlP_E4BcgxxBtZ<t`y)$8E)hDFo_Iec z=G#{Ea<>&g0b4>O%zd6?@oyO6liO1^U`$YEO(w~dfSW-)I*JFbc95RKnhH_Ueo)^V z5O<-H?_2BbD+u?V6s?hlkNW{&D{7-4R^P`fkDgL0;{mp{b)#&5Aruay{_1@GD<`i@ zS^hSgHnz=Q2J4n}WYT?K1Ba~KTmN}=+nAMVj->#wyKf}M<5@kRd1_Le5osxl7MTWO zkkpGzVMHjsSp8MXcS#7V+PhkS79{jH0@}OoIU2e8CV!dMG+M*m)+daUL`I+W-4I(& zUB!OpWEez0R`B*0QI%Jr&CRlbeRfkm!A=eXZTHE;D+5#BaqzefNU;B5|N6>RA@|Ob zujYmt7m3<eVqYtztBslHRrLN(U5Z{Czw8M7Y&8ct2|&f<G3F<K>)_czpI-ihZS1NN z{mBusZ?O_Oo54A_*Q29z84jB*6Wst#IvTqXn1FOd0WHRQYg4!CYPDfB?VoaEw10XJ zM*G{lAl|>>gn0kjc8K>kTL8Snq(eBCBR95iHQy_<hO;0X;(i?}g<?udS=@Tovv(`} zwawWA?+`7>>TsDaOw3GMV`td+(amo3Y-6~SVgFExhSbYQt48O)0=vGOBz@93V1J{b z%hnjMkz5Lb^ba^Q<`P+L@G)XOzkbHOO0N0Xg0Ihy$^3ajb3G!GhUm=0X6-0?ONj*> z_f3DrB8?gdNMPm0cL=p(y+ve&>N;XLt~MwFIj|UsJns<6WB+W8-IyLPg}oO15Nn;A zXX*?`q_n+^0g<rD+TdOSMN>s7HP%P#UtYbBYu|?p@^*>8)y$gH5q(rM|2sDE3?Nr_ z6;wk|U!eBTYxBbDj4oegyx`H4PD;~E0DDx)A+w4$lWIO__?$4^47wxdhTYj)uj=EM znyJ8s%uB-ov3ip%{vp~EGl-_rGMMKEfwnp}WIi3G1!!q)Mb=!*J@7~jy3`z6D|<Ga ztUy=F%C9C7Y(pm!!Yq;y5zum2AQTb@Ar%@ppD0}8%&jt*Ex}MY4ru7_Kn{`=Lo^GT zjXZv{p$6IRS~qGXDGX))w8iUXM$7zgTIwe;HL(lRUN#y+gUtoVGq`SJnQtQ$e!7Ul z!m@FguTtAUA=RPx&>(ulUfoM`T~yvcgH%qlR3L>cQz}3KH_#K=7el_UiNveh$%U8? z_LGuK4xOlJQHD;H94v&y2_rh?&Qj5;yNIP~_>vbFIhO?$;xT|Nf?1iDP{&TfzW|C{ zCb@Y`IIq*W&G(5WFw0|-!FC7~@WzQ;j=+kc@=CQq%FR2Z@=-e+m0g92{YkV<I0@Nv zuL7#r6isIIKh&@X9HB`P7@D$j8@qBL(o2Rp(T_NA*?m#J?7fYP=V$vZgenhnZMA;I zSO(Wn8Zcz|O<oAK@IOSw7-UI;Z9Imka66%Y26>JKEF#;crZ%nQcFJ%ER9s%lZuHyt zzJCQXZKOUpq-8^{@!U>*5UtJX?PJ5B=Gm<zb2r<(6Z)JZ)wMiRMlC_Qr=S$%hWUD| z(&CA+6VD2OLV*b?d$<NRV`m+Shp|+8tw#q~A2KVGg3b2X8UJAiwwy4V0UL&-{|7tf z93Y)VMPWynWbw|J00!5;z+3OCO#r;B2H^^>Y497K(+_9#(mFzjTf_-f`njzVGrbu~ zIo%B~2+9wdNd~?$Ckbz>{gcoZ5?p1VB{W_&eWQl99s=eyg47Eg{UFjXJqPm>4W7YD z$9-*oALJ8xuo5PzsHx8)k^U}Y)`AIEyYYQx=Stt&>pC<tabQQxPxPjnEfW-#@|>^1 z<1Ipzi|(09mqxhhS;O1DqBDH|#e6Br<PM79^9HQC>h?)T?##hqzUdF1q6jPRD!uP? zbWjmu@AiW4LERk~L~lO?LlBOkXS8(lwDr(C^0>rF%Uwqug_tr@MLb@WZA&whtoIbB zE8!EYJKqhOTZ^g|%QMT``HvY}F|fSBy?KOoxP^}j7bAZUs@!njJZjWwL(^eq=6+n~ z8%LxAL!~qu?!w+=bz*cNLZC~R!u8OxQEj~wJ<eg9rkhl8JgxR#dgs`o@vkV$-no`X zeo@=JT5nGn_|{|RhTxPBaz0X}dt$n|G^Y2<c1@jegtHNYY$-g8R+M|ho93C2u`*DZ z0QyuJfF~V9*bE9*bh`qOyw36!oriBwtzlJ?$>TO)h@b)gBEo@zQDyI4YXo5}-(Ea; zYM(shM=smh)qbs|w%6;$>GU<*xxL%3UDH<j9ll^V&!)bP5tsNSxxKq`*B*JJcG2_> z0vH0D^OBr9<J)j!{~7m%>a`sG=$rh?)7@YIo7tGXb<&x^?G`z4x$kihn?<R_f2kSL zx#rF6Ld6-<=M@yE2irE-$B_eJw`JVHwng>Wt54!tl=`j5ks~^J>k@Dr0)P<4=`SHK z9HqZCbCIW(RVN`J;D75Pe20ytLgS&Ts0!l`bX*&cR3jPU^U~6tO^zfhGHzeR<lZ2W zy1ru5+8i9>UZ*DYv5=CgnUBb27sKfkX_*_QW8g{ZJrxy%`UQ0*MHZ%`jL5C?){`F! z&C1heYOrD0xYm%Mlg`aWz|)=J6XL61(PaYmoZu*Oee#}dZ#fyd`&CdjdPpQ^urvhm z*}68VQ1kadK;l>pC^5~>n9Trx;doyON_o9|l{4Dr69cU$EWU&B<4x-^ZkyN@g+6xh zPwMoB)w72E_{3`d-x8SCuyV~Y<7PBtbGlz8b|q|+<4fOKPHB=WR`~8S-zT@E#MIz^ z=alPCn@!+HKuGW89YXG6E7SeT?x%L$Rz`6^7@OU(bxT^EXsU2P?CnJ`_xORo<aJ?z zm(KE2SY(9mg;ltdm`i=suhKY2fmp{&0vTyZ1gwJXqa~UP$NPf-2N`uzDX~SK;F3u# zsM?k|oW^_(F$dNX%SRG`&Ys}rW;W}{fbKmZ7E8cJHj+_h9$Q$k71G*%z!3D8UytBw zqFi3;&5K?XYlW1WonVuQ{`o}swR80a1qMf5ewO2;n|TulZx8|jihYiM(K{<?C5s3e zPJM?_yqqw<<^I&A?Nz_Ybtz4@-CM?5U?QJ44Y}eP4nHqjL*#qRZ1I9A@+25da*6f8 zntWcm1d$10nY-e*5Ci_FI*6dnBI(wOo!emv!fXQaJ{#U>0LS5ZqJMxCVbRWeo-#hK z{zFi%iIA{N#Sai5nrc7MZU}T|<(}BnT?3{T;ZumX`1pI_wN=xH1(7Hxv$bO9qbFvM z=4UX|gWc*FmBdU?L8VP}WEBU@DdV#;!@A>HA=Y*PjwWDlg|GfH5>Q(U8=Ya^l!UuA z`@jrShkPR|fU*HMN(H2f3L_iHxXfRx)nrwvq&6c~8APszz?(uMOM~~;e4-k-z`+?7 zfGGlRkkAmSbZh-=1DfW@EUpy$Y!T?8>kso)AM7dJxn-C&fjmLF2(TVpFr4e2U+g#7 z+4k*TetXy?4RKO}&ah^a69N0{Pzn%X8X;zvwD}fTRfDp#XjmKaqHNo}UcvD?D4zpu zpg)quKs{n;XPMnk&6ayDlWEX8k|(r56^l4OXTtD$NJe@v5fJxV4@4v5kU@+YF81KM zB`3Ckcdb1#4>KC1$+)+jS|{?MNO*>ms=Mx+CI?BKk~GjUN$;IXX{4>cn`P*Fl-e82 z)6I{U{cqygw40B6gQ<c3uxaPRpEw-}Fk8z@xG(MlA8sn`s9`np5z_79u!#`rEMo$Y zn5^jgm>97V*DIRULB6*KLPT<GL~ppC5%D4<A5>`CR2Q|GilRB@t|Z3gvZLw#C-?I9 zy!hb|Fjj~seB&a|1(KNJ>wxs3916gZ*He~34@x1F)sNqi(l*9MHd0)QHWXaHyE(K7 z7cKZ-J*L4?vm!Z3S1w#G4ti~Cddo)5wN><Z4?H>F(8-aiB*r&s{6<sh@6X}>%BN!A zf<XS;sOpq``p<Rc0rYS7q5jXlyb<2fst*ScYP~c)r0)K@<E3w9RZ9!zcx35UnCrW^ zF>XYqSk3jA<$0DOjjri6<$##L%7TK|6qVIW0hR0*(fg#o6fLB0H$oz`;1a}}DIS=m zbyp1H(H}*@XgRD90l;D@8c^gVE|w&ON1VYZKqwZG5%G1S)>4fd>}E_<AO_E>8%j0} z>CWmY4@fF`)8Fw6=$}2#(#%l{FRR_s*mX%Ry$HHIkK6B%!5A!-uyP}Uc?5jE0|so# zJYf39QTYezJ;eLe`Rl1hBpc|f(m|4R>6nc&+U%5MHUVSI^MY5$rR0aBG=BCa?{*tv z8T?`Y(3M|9)vn`N-fV}=sLpm8aiki6a}XqLI<QCI^y6H-S6lY8sk1`~8Rcy8^nDN0 zQpH}MGD@jQHr`nLY&oImuRZbq`-2gTU~~ul(~<8LFZCCc8G}%)*=l8TBAlsTtPM;V zjl_4SZL7<Ez@}Ify5h}F`xZ^_F?he#7JsVQn<E0a>P~HXQxETrC1SUhA1v?k<DkX8 zn|`uFo$d)>|2gmVR&_R2s(seFN2Y%r46JqWZi{zMzO@6d9I)pcW^+TATpWS22)!K7 z{@c%I{Tj3rhq(T^vsRbu&Ze%9K%2Jx;;cHVUtnV^eewPNOqD#*TeOfPRjbx2AAHc} zt-4#2+gs(Qnd`dLr*F8*$-Dx&zg#^>Qus?OAzM6)zDVOgj)gmgIpO%m1%Wz|)Je^w zE56KO{+Rh8zqjowkH|kGk|#&d2je}T?Zi<w6`L#?<7dllCOfu=h}84f&g;3^Smu?5 zF!J@`3i-evfRhfL><f=OaaXBv%_plgjIK!w@C-w>XYJha&VyO4V8#=E9bh(Tco8rT zPe-~LXJF3m-dlc?;6F}7;88&8_{fAd=8#U#frP4_L49h#jzVGc!5lN~#ic3g6~oWV zv^sIRNvi<uub#mjXIj1pGS7AzY-TLq5YVHd&VO0I^M}!{?A>D2sp=g0o*CI#Z^KCv z#FxvQ-B_rBq7Gjt0mKsW!!`BC6<r!igt+`NP|^rFGO(ya&58Qd{|x;=@tLMWrJ#oS zn;13}MW1ES075{E3kK+{-$9W%l;4NL0_Wz(wCH&TT>$k3Nbv~=i32Sh;2_&#wx~G` z(eO_m^%*b>b$6$%N#e-yrUExgrg)Xbt1_?iT*?_%W<73Jkye1Kq|hQGIg_l`b~tzn z`?hTr4-{}gX!g?+=y~FiGlIKtQ3(zu<j{0puaVdwe1pP*U7cA%IBG-JtZu|32PVtO zhYRlfz3|GkcC5>iP@z5*mQMqJp{b_?lasFliFvhEL3A?EU$@}>?(xy?0}JwQH8W)@ zgM%@<laqGz6|l74C`(^RwRTp^%84_55(g~R#zC}0;8n(r*n$$2Lv;w5tI2c@todoq zEnT@_<^|5y>G>PXH-ueM<_`@adULW)`<8U01d5R+zQxRm%!F$xyv|chrOou44}{FQ zu6YqRf~q96u+ODLO0G^H%4Fs2B8k-be>oiK3g$C0AW6*^ms%)ZC=G0PHVrTJK#p08 zLXKYE*x7xsPgH(6W4>d;@{V2knw5LvDa+k`?zu!b?IaU>6Z`Pq6UTXDmMjv=q=0+& zbV0gTGkOq6NxG|T!|+7LG~A?B1pV4nGi0U@Nzx9T^F)#<4HAstN!zTAE&*ige(75b zE&EHBUNV4MV+@np3f(yUgLS?vS?RQ1T-jfytki+QU-&E97h_7L+8iXKTrxUZSLO`W zV$?#Q?RP!b+FLOvP6MA=R(dp(9y_!AD3@k>PN&3w;8lV1W+;Df)|ucTc-JF?m*BR~ zOsPF17R8HH<!EjN(_)L{Vau~3*2OH$RN`E3)FFcKKtOX&C8pLGd<vDv=XB`gSZcco zY8$iD3BOTqHM;$Zof??Av+Zw>Wkv%j8E+8z^ns8d>p9D}&pP2~Dkoz~<@M#QkC?n$ z&e?ks$b<$?W~FX=nO!(W5x+0$ryG2dx-rUj?F|2CK-5Y)v02RT)wWJ`+B%|S>gH%j ztfKJtZwjIKzq@q2O_0W5goIMejlWX#_i4d8d`{b6P$HnB{fI(9u(`CzAZ=h_p7o2O zI!*lxi_iiR31c$L#i%^U6{h{zleCsq2#-&VQv#A)oq+%)VO&84x^U<84CMIggs<|k zy=BH+=Ey;ktf{G+F3hldr`GGNcZSEmemrDYNoc|SQck^RYZ`Xo=5O44Zl=_nqJ53m z?jA^dWvppdl~<{u*c`_{q0Ag3%_vJcw7Cau9bggfCgx23cwR=Xk^w6xrQHLW>mJ6~ zoLc6EiL#W%j~X5^KVItxMGgd}D4^Y)9{5DysmOKYi5BuUui;d}nD6_L6YasFOjC}# zHczo(ZSUG->j%o24td8i_|W>9e3D++Qxe`w@T9$cDvUBrFU6PyDH+cIXb67yo5J#3 zG40794Me%jg^c&;B&HbEF_T9x&XsSefG`7I4C>qZhx=cAaV){D41BBnVE){<2L>v7 z@O+e}#wYA`9CLORgK8)rap0>`tBHC{KGDrK|BkwuzlaI=96JbeGJ_Pwi(vS%g;$GU z{Zx5S_h+a9Wo0lHhxZH-?es7(>U}TAl)Q~QXj^ng`9!-l)?P)w#v|is_sESpW<Lr0 z1$8reFz?!q?<DF}sHmP^RL?<ZdIb`f372>Z=t+AIf!#G5rs&Syz>JIdC**R%{28T7 z3V@q>j&C4r)}lPRp4ColvW%S&W~ir4e=5v=&{fKhhgb93U!Md&2bOjoJ19Yb8HK3L zy4q61UjHC7w>>t}Ha#-tZtH%1W3Rmx2ar!UlUNLfmEdH$tN}_H)_jlNOi-NOoqi9^ zg{k`SIGQU_MC|n7T(8vT(ya@_ty9AnT&F$vRoQmT4Nc^QnjT{!Vf(8~JI_I`92Py) zsKlD7l)2VxfdNW{PJnQm=uIU-Qee^9h&$N%C=>g=hc&|xSDL-sJ+%mnhFKt;XD#Gj z2zE4q&{%)2*@^mvO4vZ|*FE@S$1}z1{Oo<oDk%~Ob&}S~|6&`wRJjvInj0t6e7R0D zm%+K_Mc>{4vd%e)yV|NLF_6$95=Yw_z4vQ4lC3tBMDGfINUylPM{vLdC8$PvGww3M z#7!FCN}^#}-qt^>V~yZ$FrFzti)i5lP8Wc{b)L^3ngy~Q{tIn0A4raVvcVtQ$}w_8 z{3pGv*4Hunp5VvTf00XaophUX0ZP<L4|Bew+4i%hf4<hZRo4Z~&&~A(5S}u$^%Qbm zpetxjdS{lgY%nKW94}>&+jLmekkfXZY#_;M=VNVsAyL*H&%BP~bR*Q}dWg0oT^8Hb z+8?1G&z0BSPn^-$hiXOPI+G&__cnoUIy{k1=Mc@&b;oJ3rj6kk$$N!*-WU(H*D=bT zr0V|Tqw7^x$?|Od3@g!L!cOqQSF7ZW$!NRFDNm;|d2K~(*`%*Q*3~y3q@}A_QE>1T z_6D(LLad5BIEtTzyE_8L9|e!)^p^N1XG>BwZkhJX2IjpB!BjvAu5P?4wikmTJr-d# ze~F%~qM?I`uv&gYSC`RHUPM?eSZ1ec==@HA#jy~*aWwx=5(dFZKo$AuQ_>Rp!25mj zSZFWpKHMx~mgDF1I61Y+^zJP>M|=fW1(A{|-QHr~ANxVa>i9KBlioZk*_GScI>eu& z1|bw(XKH?{PY2&7|BF?JPV1t%IM>@CuK1MYhZAS<3|$8;R~lD;C|B%<PD#8oy2Bby z(#8&Fa^s#ee7BNq4f1V`peH?BZj|kwtW7LG=8Pn;2+>GHu9HNvEw0;77(X?22w1IM z%aiOB(=+-KA2<0vs~0Nfhj)MhXFr;#l`0{U>G=9ec~qi63stjc&eM9u(Mj>TmCs)n zqy~jI(kAj;bc_&x@JKEnS@BxtC^T6o>tw<V*HniMkw&=EbG$9=ae2C7tr9mI(+yjV zgf=%zCc{dIFC^fxV=u$hc#IJDy;~Q=0il55wvP@wfA_?yqIxkJMt1n8T}Ol`+K=i{ zuR>E#!UOw>4wdD*?dko{h9uAd6M2~^-V^XtQ<MBH7#=&8XvML5zRVF+n#d(Y;iK1d za|ED>B8iDT>SuRV5`lF@KVqR6BpM!C7IOSK==Vpw&g(pxj3)fUkzqW=b~T@qFwtEZ zW+hV>@`(tZVIO~PD)HCr*ovK<9kXxHykgqU{en1fN;#jwg4p7qn!+cTEpyI5hH}vG z>x6~8sZ_AKr9oJMqy|Y0(OfufU3-I1W($>IBOJ=s6IioUUS_%(HTTpfCmY%9#O%-* z7Wh}nGS9alcExi=;#_~8?TAqrbG4o*nahwsLFg1}QWPF4TIl>4u;pQqh|II-98+uo z(Uzi8j9bgxoMgNzDV@owyPUubP~^g*#Jxy#7^83fyfvKkIEl$Fgu-3GXv<u4;LwKH zi{T5}0}$rdKN$5^bMe^dXf0KPCy66P_Gj{mx@}$b#4=85S8N&fa&4(v&Epc6qr<}^ zGUxU}MHn|VUS5Z!17S1=+62XSRjKVu%O__)XJwOr$lnt(F*DBQ=eW$rxSMZa)YoiF zdQdiAmmr8F_ar$^ijWkGS9WJI`A7oKkCU(FLsIYGoMAUc4xumOx$!gcTsW=!tww}g zxi5MQP|WWyKmEPz9_R<|r|I>3c-G_7y!TzN53|0z0QrgQ7caCIUOD<eH)ErJ-lYy2 zM;kmy7jJ%-MvC$w%r=xdhH9BAny*e`+H6jFx{24rL&Rf8q1<FLTVyweS2SJI3eh#d zG-=H;siwgy8i3ts(eL|kxg08j9UeK`&er*DS#8+2iXOLXXd)8Hi^YngSKbRD`KUqg zieYa0*-EZ_>sHrJxMO^Wb*kGR?`kWpC;A=J&>1(h7!{7l6brcI(kLf%V{TT2<75-6 z8&zYT42<gcDUeCxm9@a*8#pK_^-JFm8P1kD+f^sm92m5Z)w;nNYJ}!LR@bCJLXsU% z&qd%6M-3S);A2CiB2imnxFp~86*(DnWYUI-J(T2FrXb*COoNc>7ft`=>CKA>vVv&c z>9c-_$@t1_qhpRP6z0#+ww!e6an%ezStolEC*FwaLF8jo@%>hTO&IniscS@-4Xk^{ zrtKJ5<UQUx>&7a4q|Ll#BJS?d+UDhcz~oPM2|KSxUs4*+p8fP(ywu!Bkt8%c6sw78 zWyNMQf4$PiP-wJBw)<i*(&#qCj#aw8k{Gvh`@yobOw*{J8keY19`k%(Q0CFf=iD>J zFrI&zxy$w&L>{f?;zPdE1W50pp&X*=#w>q9F<vL%O*D&=8;j`V0^40B?L|s8GtTIh zf(<LpszOj7$zCK<y2erpL_#acO)c%R39)km+Tdc0QVfVzMrBY1pF1~;x`M7r`4g*e zm)mW%!d|S@O{I$cifi8t#Vadco0X!Vz*{*>o{|y964+OygHpN!b_)=H+o!D;6hCIj zaWcvUbE@H<eZN=)xj=cEW~;NRy28oY7R5-)FFp5|n;ZMe)~}2~ejxs=Fb37koCx(9 zcyeK(AV);McB^c^08y;2=~`S|kwh$Lm1)09;uINOCMZEKjUq;3+!qsoVbR8#z6cEr zi0N%IqikJ*<%xJC?E8}FMKyTM!;mp}wUc|9kk<tPx#2;DF+N^0pWx<Smd3(DWtux; zmZE<1)`Ka>&Wtj%YJiK-AP$vs@i<*4hd0{uunqN#iOC>hj6>gO$NE&}#blR<?wGX< z{V4>dD+`i|#RqLfDYEs|E;WZS(Jd4JuKXL$d|7$*@si*w5&^NgZ;jfd9P&&PAfyK0 z@-#u^rMW!<3dHgDRD+nfKzz(tB&HQ<8g4F2+(~@yQiKAa_dwrJf`<O)K!uUqP><ui z94fJv8;W7Fm&5f0wrfBP&fA}i50!tfqm-BJJxQ8`3E(}OY!{os0lSV{ys5m`k|M-t z>{u|5QPP|UW&x-B%aYvU?T(iBW85A*9V0nld}B|2ByRyeWvN&^j9@JKZ@!Qbsb8_^ zONlcJ=M0REj)N6&mU~$eu?2^f;T}P5TkRP+t4-So4XIQpAtJu020vP`T?2z@1x3Vd zvJ1qX!amg}mWG+-dq>E0of@wos@EzJey05Ent8dE>tKl|t3mre*_a~%{M0D|w-9f} zC?w+b<H(8Il}w+F)ckf?Pcd&Dyqxo2JnS#WkhlR&$|$*jv=mo==&$IPkx`K+Ug{eo zF#1h=mot5MH*}H7Jc!O={?<Ug%)0=~NS$ntF~>fEz#g9_ATATsZS!`bnjtFS^eH6s zdY{~Fa>v+oy@j+DD2O^9u(yLph#W_UVr5pQccN(|L%vTj^!N}UkkH#>=UUua>^w(f zJbJADK(RUlt4b}v)x_UlVCbm>IDny<ehz@zhKLO0O%{p?r>O(zDGhZ+jkL3o0&`h0 z@{No_wWBu{*EDzEFzZK`(=~~~dX2&bK`<Zhe9)klpIfKoK%B%@_s`s)?ERln%@%Q9 zKvw~raxISi(p8L}Pwlq@+5W{OIVb1HNkVF7U^NY!A{5c44q14}_*(DuAc0lxPcJ4o zGvC}S$T-dpYpgQj`W_~yz91)=KbmDlc84U+8P!Z+k3tZMCAK}YRENE0bVfDhuf8~u zF2&CqzdSKLK8`TUN5&usHv$2AKuUR*uYSVia4BaW)GTN1*;yM*E+n}uAFBoTAeY81 z-aMdxYZuP>()oMNe|h|4Dlo1x#xHR(r?t-E^1H#SqLUK8XTlHbx)y<h;N(jdzLoJ! zCcsiW=nmz_H+Vzlo_SkM^p?t-4I|CE`2dci=R_EJVY&XPGhD8xcV?b<a>x-zJV%;W zKH0>$zqd^jvt0{Zv#3t^*dDNRu~*%VWSu<Iq5IMRgIDDMyqyTDFg7&b{&WY`$h7a! zcJRo%Fh10;49o<G@6bCS1PHy8DQ2%DuPO<mGCTtN4ru+&Aom}BCqfv)cdQGN54vrl zNH5|UoW0ZRw*!!`;dgOOupJNZ6v^23Z|-Ld6A&9iDa^GtH`^oSh(g9wkoZ1dEfrhM z#7ulrzEaQekjv3*6j6hH?i^fHzDOwwpE*WhvAZH(;9{mhI+5M8Q8cl;Cm_l_y>m|q z51|7P!|^AB8yP?XE}H1sStdAo3W_XgHx(MPwWI3&GkMs-JB@+sRef+T-$|bg0qg$@ zcvks%*4}As_(r{2#p-68|I7JkSlVNUnAGeZE@BMm>Ov~4d?vr*k9=pVw`DKNYshuG z{&rknNQbtbo??Qa3<vsT*>K@Uo4zmWL7IK@zzE~4tS9XEc*vZt)r;Y|JJv<;-Pq|0 z%OO{|+~4Q~2Y_nK%zLWsoY`7QB;R_zdr#gJaIYRa=Xj<Puszqhm~q~F{nPcJ^~p8* zY%;hrPmT=bJpmc{K<yK1hH4`W6Zl72zf_ZMp+5O0-mDI<%_D=)LqgA~|9VUwb$PY< z-E4nT9|G@Ty<wN7?l8wpRCa>EGnV2kj4}%4b7WKja_3cjMco6HoZV~yG2pj)qF`7L zVJc{QADVF*X?0cOT;3WMsv=DOy3n*h`BatGSlLolhrUJwXZBrl<;2|=MZwM#05d?$ zzq2)~RxsboSgg_(FUIe6>$S#fx_X73LiM~S2ib$bO1gL%8=}nT-y8|%NqY0{0f5ps z`ihbDjgrz?{)Wz#?J;z;zqWa=h_}v~Uwwh0e6)CN<68v4cmhg&di-qj$o@o|*H)MN zhH~@QV{>G4ak_TpTan|pCJ~N~V4rVQwtu+3Z0kP<JREA<L-ZjDPh}EuLJpI*J()_} zT$~<Yp2SRYvE{9Z;bbvQXdU-&t9Q}Sxl3sV>cpe!WQvt4J6;&li^~|lB(=48NU`r2 z$5ptqRbX95wQEDI>V|^m?Dw++2AZ+`PnhjdQ-wp7;&+p8j}{AOe&HW^M>tULnR|Ok zuD>oM_4^m!6*k2o77=|29Aq>saUVY9U>1M`Y;3hvO+r$Wxlm;ShBD?sjWJS$x#CFt zalGMd2ttrizow=n(pRG;iN|8%w`f9%viT0fnpPY@C_nri9kzc)_XwUrm{EN^M?~~8 z9KsqptPf>CkY>~*A_I*VIO4tc$c;w&m!_F!^Xs=YV7%&ksTIJ23`_L&b#~lbrq5XC zwJVsP@(gweY7>RvwgO%>J>JhSGf$I)DB$V(zS=M?Nr#PQOVRaGpb^N&Z?Kz!PpG`j zY2z{z2Er-Wh6fb0NAky>3RpbR633Wj$86{78f~M+Q_WnU=k|wC%-kU%`fqsdB*QBV z7l{ai1U_VJ?Zx0LjOU$ViklGOPDxDz7Q<vzA}bgUg2s%wuo!PJ?Zsfk7&tk>{@2g^ zTzoYk-lO!p*rq7Q`jeoGlGu3*@oJ@Ulo@R(vh4SO=F>b}N0A8?-ZIw*>G5P#o*45` zoR=`K^ynmrr?zg-<sU$y3IfEEApzG|48P;Lm7kWD9F5aX>4U}@Yt^%@cxh{CkoMm5 zoPXV&&8X3vA}~MBUNYsjSVrfKEPHdn=5k+U5I|P0`W2GF@sfF;XNZy%{u&bu&Q8i- z=V|l^j+gs)0&%@NSlY-OMMQ(3T%oOEF&Z96qmn4Lq!5jYQghe9lB!h2%iZ)m8(i9n zQU3Xn0y1<|34=SAp9^4;)!bVf2iYvJ>OpJ1qf4XeVnl2s<6=0?EM1vtT&$b1{(Ngg ziP`1QcuaAAau(eR)Xs)Je2aR_jJpp)irmA=VV~$?#P>g8-w^PChhYw9GrTaM=nm53 zC<$un+#*J`K`QNg-=oW9v|YuSD_BV8lzPB(|Jl~}3*`%1sRC2!;!GV6;0|><r<u!z z;VIJ6vcTPLxJn$3MM4j5Q@-}>541kSrttz3llsEV32psoEb>y#`{&)#REmCm={YP3 zkS~Izr@rF*wXZJjgaYCHsz`u-g(1b@h09>l*8)ZPyAQk=cp3W?_!Lk1+m;~P8*K!4 z0ZFiI>Zi2PkyUz~diHB7y()Zd<(bL?Dhn<@{q^^L<@~-4$mL_}__@FWXmHolKV{8X zmtDCkNPNtjG0*go`N(BIsa87)*ry2&G7*|kQC5h&l5AHtZ5%aE5u`I4Cj;AF{i3TJ zcoP!fEU41C8?#|4RP34arDaw7u5&RktJ~QYgl2R(7ZZT|fW!VA{8YQHd(t7WicG+# z(LnD{Opce;bjQ6R$qxFtUgJz5bgkxTAoiq|Uby)>LlXGRQts9Xg1wpWOPu`;5<fCj z$?92n|J$iXaU&ftg^I8WJo3F3uV~S&&ib15!zq~t^{yLhWplY<y;PHI)$t8@gSD#3 z{{cC)@x9VykI5w~{<C5@Sp8bcAnV9GXPpCfI1FwVH<Qo1)YxH@fF`|O9E!ozZ=7O# zcL`5V;RE{>H@|AnueaE;&Yr*p!z}53qVrc-7QXP<yK%#gdYv1vj<Z&8&b#Sn#wRtx zjrS<4>LS&p48sckL6*~l23wsvl+#eZ@qD?{k}E!>@*~j(GCw3uZe+c6>cFUF(NmvF zC7+C~{t{)_o_?MERiAN})$tgb3cTL4+0ux5*#%N=;LyJ;H-rU?%dzP961Dfy#l=2g z7sV9@3e7L;bw(0rhldkSXDLwUl}hx5Tq#%^zXWR_Rz@Q6=mT7I_Se|Ta?%1L^4NDp zU9)o<tU6hsBW{P9IKhAmOl=}IL>r6R3XU9B02{=iu1H`}AmFc}s^F;7ukNi;7i&ih z)Bjxo@;ow<PKZC9k)+^N4d>7%<QejOhGCVty#ilH+>fz+n`CL9A&@#?$i4;Th0(zq zq4@P%1npcbS*gTbO0&BD8R^ft-;ju`#KWw9ySA545D}A}9Ns}CKAj7;@tFi&)#MX0 zP?>BsaJb-4lf%)F2=;+n%78RaK%c^)5i9`50Me|Ahl4GHEE$<Jy&ddWH=ivdKaew$ zxCDQD9v_SSsTNJnhu=fLjWNzRqZMJjuO3+`vrWGuYw8oNVdh^wG^a{&O@-w@qReh+ zHjZnG9m$-S5S>u}8Xyn}nlhj}i8BndXM!{V9@ULn(5BO=r$<`sYbb4v3~;t~tLvr= za%ox-M$LVSxQl5z$uH~snh+g~V|q}Z#dTK2Q8`78(k3U&FYF74k#^;r@~!y<Jb*HW z-}Kvx-u8?Zh3Q#QDS~P)8qN4fPOST)+7~e~nRCt6%9pL_lIPfq;Cto`ouwyyt!sm- zY*ecy?=kY~s*4>%rO(}G_EA+zTka?F#8vv(l>5w`m)5p>zc?}JARmg2a;0vX@8X)$ zxrGwVeI2^a3I#e75dbX2(7D|AHX2wrq@S+utY)mi8fBX&1q}yIO&OsTGH`r?G}-iU zHU*Hj0#<J8lYww1Q$5IAEg;{X@!-FHola`TNuIErS@kbtU|B(NHkvK{i&?HSFe6re z8fXV|hW<rRR6C({9hdN8HrDutS41Pl2Tn0>KEWC4DbARw|3e#iG>jy*FKP&EG4~32 zmoC^Zo2~LJm+tb7QgYY%<Cf6(TUkXaM-b4J;!ABW{d_n5$L{<K@BH6WNx15QQ^uFL zai&Jq*W-=3_>8DF{mc~wIt63q`c`uX!V5sy>UWxeE81)SF@eNm%^c75VZ*KB>B;`2 z;ddS|3p!af%~7->3c!l$pDPw;A`&Gk9-}fE0qJzh^_pOfN2QS6w51KeW;$q2Gwc>K z#ui=$hJHLy5Ccv6zghsx1S)re`Nq%I(vb2=FrXH2AtGRbP*dgt3ry$(6*dbBHmpzF z)DwFHCb+z<U?_<K+l7)71;4pXaSOCzP{;<%*x5Z$o9kL%U8CM`cJ)`0e|v4OXgUq` zRSb0oEDAyChiW<WUiYgi@1@lQHzV!9L$79~1)N>C5sVNNXL5^sPFcLNv>-LCj}*in zB%n`#2xa~aM{dQ&bC}^Iii}(a?`ivB<3!fj+0pGkwBNo3JMsYP=y%-A>orw^cxry` zw9KZ~+_i?Pr}WmHpFW3q)2ZL~;3*u^Zz*gl-tLh|@GTvdJNwA=0|P7Be32N^D_f<C zE!o%=T1T!ESovmtJVuFz#9z+z%$?yJ_x+B4avi;9UIh?R%muy;--3%YFDWaI*fC6D zF;Epnc77U?Ix9>*juK7AWtCz#4>hE>(_0DNNN*N>a1aA&IDhdw9bkWyB#<|~n11hB zccL`+<x+WY21=51h@z?kg1z9uhlT=u*dZ!eX4ig^u<@|!rdRoaU9?n7Rv6$gMtS+# zY=aw#BqEv5T-N!*K^nV*ML+(c)C5-zEZtjT@)8~}arx#D(=Pjbm&|yzS{{ur%FBTS z!Il4AlJM0~Dl{4clq4!Bla#@U2pLj0(E+qfpeDYe;1cCvXKxTx+>tIBq9mMF%!i3+ z7PVFGOz=o<sAazTX<w}(fs@z+Z{L9c{T8=nDs5u!o#aF^nxmF=d-U`k-MP_-4d1q0 z8=t6&@7TZSsqBQ=?OLZq$K0C6^a^(9dj1YERcwb4;8JWYcWW|M`FPpYkvpB(fHeIu zFoiwTk$8`O-o4tZ0|U|53~w?5Qrm5R)Ref_J%8TxUw@s^a@QaGmRcs3*Pkzc8{fF6 zov)#*DCf;v&EZ3P=p|xXW&?#T;mkZvV=zZDxZ1pGtM`6LlI3YcP@KYJW!`^$I#Ap< zX~i#b7?;)8E!vN#8_j?ET<l`eH&XOIuUu>-eeG5ewfKU|_u7UZRra6A9V$XI{cMyD z6jD%T>j}|h1Ft6zzWU8PYR1716h*D<!pk3c{0sN9eZ)3qt#i;9D8#T+$jck<WqtuP zo&;6c;OHq4(?c=kX^V3uL-nk3qZ&ve_cPe$4%*m*9w`Zwi%eVC=+mhw%x!866r};? z16A#Y8w1<foFe@G#o0-Tr&0^>x5hTjS2M1bZcwGy(MXMlwbkF7HBmQnT<So-juTUN zm9MT+XcWeT_+@eG+>J*tKi<85{MeCN8$Q(z-qr#~Oz!UG+tI~i0b9dl{Z0yvB||xj zSfxDrQSI$sY5BX_?~8CORUpWb6c-C0RKtn(ev$1}t}+)WCwF|-FPf`DGZX;A>ao}8 z=Sm1HyL1Zb9^CP)S7%I4B=R6z$X4V04t(CenRdWvFj$>f{tW5tn$OTY+iH$z=lPtr z8Hs8z(9U~uOipdHt>#->Odj?#Q?Vpj2!j##rSZy$6MhZfhoyg#kxQPix~=gT-67Rc zMJU*dnv;ve*-$zrf0y}tug1L7tTc1QlZk~_Ofx}@Hic3R5ovZU6*mP_5IUbsu`{i( zWd@q@?zuf)s*8!Q8KT9eG|RKUGzP*?L*MCAe%z3Zg-%N_D`O-kGnP%U{MPApJUXQ! z6<U*&=m0F8ipOp;PL{^h&P7L-5bMz;SJkN@U=ZB{G}(#5hx!{2x?kGP9AMSzfPJ(s z@k?>v^u>OgO2=!ar*yf>Yt8mk!+9#p4YSJoDfdZ?`D-Lm?uLxs_J(rRaWjcjl(l~; zK?+iH{>VLBM7RoSIUI4S@8WhIf6qhQZf^tPol8<4GKO~FDaOszF=U)$eMFfuYdkqW zz+DbI#5nz-fBL#YQYm=$%cDC;(`mGQd(AgAp3TY^G|!J)7Q_n--a2QRRtGJ8K)4{? zp&DP;fJ#t$7p1e0`iG5`SUZ;~VMI#JKc$bHToof&lELh9>6+(v@NK@y&Hh32(2g=( zsSVvd5#}~IYKc<t3ruZ_p^E}8Ny24E<F)*tQm`kSh<5iRm<dqO89p^=3||U>ssUrw z(x6waKfH!3`oiD<_5Zy0<6z!{&xf)jL%o2P%Lo|7Lh768S0_TN!+x`?g3bM7;bIK{ z6Vm?g+BJTCVDQyJ)<q~1C~~43dopchHTx6_bUt8qv%>=e?_>fj3~(wvuFsXmya5;| z*x|VcAa9N&-KDBKX7XU7%%a%*bg{X~pGvPJ-}~dLNFV;?TIB!)5=)iC)QW?#9M5Y5 zz$*|;0d4KA6yD$OQZgQ-<*qUGEU<Cm$HVoAm4`+i?+d}BaFWcqEtkDYC1S_xAE1@b zJ!kb+^`nhmiFtvLy?)OZ!`Hm^6)=7+P{+^Ls(bv#n?j=a#xhv4TG~JB2yaIrc+GAu z%G-S<u6<7%ZwX(%KVrBWO576RiRVtf`F0z!Bw}nkdg;z!h9*m^hj?CHA`z116~knh z4=WMmXA1#kNiu->uZslsAo76}LL=}fX=+YRK2vu_!3<Xb(+r_}mjhzB7zzFYh`?9* zZ&zMiaT6MkKfk6iJ_JGlOwcbq(P>iu+bq88_~6K6d23g`7+NXELRGw=j@D~xdDR;< zSpN0LOT*?Y4Kwiy?nVFt`{lej7~*hC>vfK=u+_JN3zv-9agadwoS08RcK&%sH1PV6 z%ii8DEN!`?BSa!z%+aHV0XS@=QCjt-G4=C;tI$J~uAk^!t2A#)+^CG`?VgGcm8PJD z9h3cJL^kJWTc*5x8kyHj(HvdXR``B_E{4}Sw&@Ox#uCibFnTHl7##W;6`Dv`*DQd~ zzt1<Hk$SH7RrFC#yk#yoBs#i=#V*atFxI0@z2$1T6YDwu7iX#qdotRrLya5$8>>$l zy`tr!xYPUpkWSf{f5Sj7i_}-tF$F}i2YMV^5W%qGTd++fR^~PAav?M(Rhe?D4Rhk4 zHzj$00OwBGN+>_2Zdq-K9wJl|`a_LPZF2iA1n!vKw0mMxPE?E?>|H7uedv-Kc3`Tc znERrYG3s7Oo#pO}({__iZ|+swhCx#{SD8=QiDe60DB8|K5d-C-&7B^FbZ;?Y&#M($ zNP_3Qd(pu4q<+gzfPGdS%Zu5$0B^FA6+DYRBgg%sZ>sR_zEnm;BJUd|H}5m9tk*8} zC_fdxX19`qisj~A-_rG9A@!WVvHZZlyfGzJ@APp@I_R9IsL!~3k_7ueI4AQLE3Wlc zsJ2%gb=#nVoiKlk3(I{VD^xFu?on>(6QJU35bB<OhGV<(dzWyl{$cGsrs$(qIwN*% z`00L-;>a=XfzR!b_H+p_jZ;uafnByQ$ZFzeFCn{<h#Pw^8>3?&<!YVK_sM9Hh2mS| zJ(3Vr)%rlnjd6=fXEx4FmaYH#AD5MFaq0UdSvc&zWMhBTU?LZd+E|fvuB7Dwvz-&9 zuRfg@2O<Nv`+MSr#d2x5#T4mKoligetS~)>FTXjn(nbO86K)<>eWp)YTN2fr4;#I; zuOdnA*$U}<M@4!GU@6mwa>^3y!5y|wZ%gt2Spw?1r~Xs#>Bj<<Gy^-JrMr~A>$lV% zOegfQxuQPduw&@N;gU{38I`@@s_{4=;TOt_ihJyWm3kCn_5?TuUw8;s;?(fd+}bD} zSR!4{l&r*?O*VJ_ETm@WXJ(YsE6toKRI1fV8&wE&J`FACU3z^38-{PADv@nR2gSA@ zmNAJ_%^i$9yRo{v+qLC~{I@2mg%vs%mzhz6dhtl@;cB|QY#OF&{<%y6?i>x+MlAdP z!SMKxVdz<^A}37CtcJ<7rLtm5aC`Q=mo}}{tLCH*Xp`pAT@$~J5N)ar{YBC}t_#wB zlImumyV?Xsb{vY|>W4+UU`1DHZWeWT;5Z>iR$1piKQ~KW_7y9eTQawn-6dbFZFl6l zbHiG->gi2dKiqcWY@V}|IitB|q=-+-49|NU`Le1kvnM&LFB^Ro01Z@q<;)xF%I7xO z-d5{+!?gc)RT8;d;?ZPO9xPvV>Q>6_qvS=+D?%1Jfq3HKVUJlZOf-#h-B8Oh@*)wf zp>D75YFjB-bJh_xG>!EE+aSp_bLCUYHr>IiqVf!TnJ5J;iECG?hY<S;t2!D)ZK^Z; zh_H??3h-3LD%~H~1*v)`sWBe5Z}BHH{}}jhig$d_ixj1)J<3^!JP-;5VV*>&ZGs*@ zMqi^@Gv{UkUbjpVm1gT^CmIz%)EFjBH@8MGdxDJTl@dp%im_D4Ld4O|(=V?dX1LXQ zabx&hE=(>-5wdPx9=)X5(pRBtl-4Ni5NH~T-D9L7$ejA?u6*K(CD=bDz|dU%gf`t3 zQO3ZuZYsH%Fu(%jvnLp<87GR3j?-7JXvC@GpFR5k?!}!<mKxkcX9!XR@tc{9iKaPL z?v(21j;q+4L8TDWnpD40^{#o$1;mXeY5m!wX!V@qQA#W$_IzaNsk#Mv&C2nje0f<& zw8-b~(R->!NfITQtWVex=oEq$Qbdv_)@$k~&IuRwktnFF{qbwn&9`6Nb>Uc41%a?M zgG${LZ>@pdbjP58^&MamShIiV3+(fVYy{dbgx)RP)TyehuE7}!6jVY<YNFJeF9V$l zg7`|nZZIsS#HUz7#P{7*x}HeYXdTxVMrEi1B@EU!jwKA#FA~PVwZ;MzIwpEdY_S<+ zOH{S=3b9_iM|RDMqRJ0bK{Sdw<V3*)Nr49W-UF}7uaWTRJh%&%Aug0LKOSyW<(}Xf z$a)<IXw;MCu(b@IT8KjA7^?EA7naWNhp$6!7SisQwIBKJNu8bm_>Z%RegiAp?{fle zrZ~A&f3U?pW+7v@D4I(fNcW2BgHx@`=twsqOz=~`E=0rvH0O&X{@H$A%i7trVZ2A_ z0-AHLX$VU<kig&~e`L?NeZQ=UwVHgNd!4D++^|uSH{NnB56T^Rr~Aj?h|Gd|s})jV zR?b}rb~JRnmY$%Gi@&f1GWrPZ<OKS{E#}AQuBcf=j@yhG?Niu5swH31LMTSo6XV@T zYh`~WCImJ6)jcP?8)`-@r}z7qx<~c&Z&&ybIp{n5DDT>&kiqv@&@*~q_hy|<gVa2= ze%~?b%lTqb^fq*+=oBz)y>-?`nyJ1?Y7xt?`{T<VN^h%(Nz8Gmxwgc#5GaKku48`# z$C$@e@5n&(x~tpK9nqpA0%gr6o3U1vv-hdY^LZ#3`U&!T%%@9rq|f1)S4$IR0hEwh z3SK@acSK318Is!)72mmic*x!g{@TX6;x)WyS%60}rW`U4$kY2Ki}U-Q4AXPZgCTJl zfXZA6p~0TKtc;EsL)}m%Rf`Q~V(Sz21@}|6ujSe&wMd0bVrk4QoO8A0MgzdP^fu%K zrmi>NyhP**=B8&I%%g8dVJT|pQ!OT)J~x!odB)G@6&^!F&Xx#i;#~kuQXG?@y9`0` z8jmoU@C*%0W|Oo=J$eg_#%Ba)iUY57W}7z`OL!oVThJ2as~-$ZUM^d+rqr!I^IFjX zWBVC5<Q5YLjfFEd{uW~%&6k<iam)v@cdWT8LqqviJh%J~T`5J49m6@ApDC!%o60b3 zlv7RXA1);HVnI`w>Xt}pViP5L?6Ps)lU5J|-On4|x5|JRH{|v!IN<tw3S8nyra|Fa z9sna-mpH@)q%fW;#thMNA6qS)b9WnV+jvkb?v$uVMQr;Shviu=FWqZ4cNt4!u0Q)Y z);LG@Ne2Ng-wSo20hYIN-n^M?IgbxTY2KXp0TQBynP7$@g2scMk&noSpUb-Q_9k|r z%t_4K1l6K-DoR{i)m{T-ZU*#G1tn?;KYQzLS1X$kvoQ;TvRZ)H+(WDK2^P{Muh%p8 zuOZv+HtP44qy8_YQ%A=jK&dp&j2#(8xFN$*8f2axIA~`-gVRcW5swg}vUNWTT4|xb zL>PmIG^6cHduk;ZDTpT-w*`2b=}lq&|5<ixMRbBJY}N$D0sRwWI?2O#;e}Xhe0(zV zoP`qWB*HF5d4@e726&RkDlJS+nz>&VzP9gpLxa=Pdj-IB)8~jZ0xqAXJQ<(_Q1Ei` z&6%0u5p%gQxx6o&7S&E2IIwkfqP;HDzf-DTa)fHDUASDWrJ7-OUX|n<qN(9?_VZB@ zh@K2u#hmpm#neQM0Clxf-47UzXNxw2gb#+Ydn9}27IF`!GTP7Z0RA=eKDPK9g)9k^ zhiDIDrW+Pwu}ap8&p#x4Z*`S#KFz59q;^AWp^_-J!aUqBFz0ytC=Ns?e__<8*ixA= z+}|SLZ$O%csH8ql2%9<ImQlG~w&JK7eYm#(Gl3|JAbK8NDJaseWwZ!=MKKp#1Onf1 z8l6IbD0%ypX3Vz}#Yph+&&XMX#4kum{*3@TEdQsJ2yw!zJ(Z{Y1Z396e>{3@uxM!@ zW_&@H(PqGBU3px^=npz&)a3oneUBfD$JMVB=SHsCO|dRb7o{ys+C!t{MTlnUx~#vf zb?xF@Q79BkjoXBvQfjTMxl;QQ$B)tPFSYPn%>=h~4pdKK4y21jI}=0Lw_^g0MZ1>0 zMaEQ9al_sGXftG#+bw$q{AO5i7R1BwHm9v<4_%_U+g77UVKY3f)!YDfnbb-^Sf=9X zzUTJMO~iU+Qp!wX1*0>fkuR76^az-TxMX^$BA58{Kh%H&A7|P+L|>&H(ZW!uzBj$C z!e7~-%Tr?&eZCc;mcswvsPxK}{4kIt`JFHVrJ!^ByWpEmM<rhSmRRD+12xm#{#0%| zo4vq3CIS&urUX3YjTV_HGYC^=Rx$JC+LI@&{nU17lFG_N=Jcfe3+`@)Bs6ZTwYMkS za(!YFFz@6<+jwvMR_Y7E1JL|col0j(9~e&liJYf>2C~*PgS#&h!5i+1eBY&9l<Xu; zm5h0TyCRutrhNr!kGTNok;7>Se`3@5A=D2})4dQ=Lbi7ELpiQ@aGf`O>dG~-{rIee z9&s}0(W>Ca(zF2gRl|+DEbG<jyn!1@ofh?d|2ys7S<Rftos>jMZCmj6<<L*8t#jU` zBt~rISc;B*radoT7`x$9=}hdg>=#PJ)7>Vh$6hE6ad&nj>*K!(9`EXsj{E;E(NN#n z<rX?(Lin*}8diuham#0JiP@P8uZsn2I9AhzxU^!?<+LytgJDN4PgHDAJwc3Sld)Iq zPuHzaN}v86dz;nFk<GHXXU6a#YZ*F+;rZ=J{^v8~&nkKl73P~kBoY6I^Oe2Kifg`5 z4as2U=Cf|~Mn`xl>qq}mP(>xZHN;%~eYdXK62QEvGuyRNb#<HG2v_GV%;@nSnf{pH z`ZClM)aLuS`t+qoiJ}tzs}G%$i}V@}C3ekkP5<4^Yan<BhVMf7bw-ZqPB+(rYgQ>S zGVo+VAqX@L`QWZD3X+OWkpnnSEM~p>rxKihGE`|+4RwpL<iup#O!BVndU>b$8_IQ< zXVLJ&lFU1%8B25DCl6kvrxKufD}x$0RaH-&sQW^h_|UfME3G87B~QCKWo*@@Dv{b_ zK&puaMu`OVV>T3LX9e_4RexXEelcc*rgptnyEP4o5c4fo4V&CB9gi5nAQvfLMDcsQ z^VG9qF&i0{BT;<jgKM5SJ*_{7_vBD-nZcf8)^ocbAA=+&@fMwZ(?V}oREOdsO~MSP zygi^qc_b`JoYx3W0W8{vu?csX14+$z`$dh95vMkgy?Eemp+nR|A1$Hy9GNVg&y~{J z%(7VxI^B7tezDBToy!;4jHr%1S)vES;+|vAEL3=9p>b8BYv<tIDo^X?RB`lNu|*u% z2dsc9gyJs%*pe@as!mhne^nl4I_&j{mST<kKh?ymr%2Lt10>nDRc3XEhGa-0g&L$J zwlZr`49qW!t<P5%X;}OI%^FS{?zYZ&4IHp?HvOEsUwL~!GQt<yz0LDOetVPd+I7`9 z<(V6Nh7R0;9Jy0wmFX+vqVIbKmg+MXm@r_wcNbz~L-W-Skx|(6D|*JNTvVTbEe{;& zAA9&+-U6e}zBbj-uEJxHKf*YcT7G<jCd6eE^Bk^3ot8OqSCcYQELkg-y%V^3JYY|E z??IYkdNzEvUEDfxr{r4SPZ#%A)@M!e(#)l4W6Wzx%P<ANJuII3iSo?SG}<g-Z6ebh zh#IE!QbQy7+bm<@3}&LN8p>K8Hd13py~UzBx+xJKWsC_4{hGpMNf*5q8{KjbHZJNA z^jbTY%}}r_Ptz%g(<ByL!FNfbR_wwkXz%u5dbE2b&t`DC`*>^#edwhcZ=ca_8*&Y? zl{cCt)2II&xO<)-uML|M;dle8ZJ`~f2E8$F(2}$CX@l``6R_kU5=z#}+)tXXCsrYe znIg9musw++6$%Z}mo$XJ_)Al|E9#NL$|hRc+nIxrC#2?vrCE*+;Lu*%7Pkduz6Aoz z=6?VG_kH4)EQP{&Cn9sBZ{MzDvB&+fAEV#BeS0nl=WFQ5$W%&MJ7#9;mhXj**J`Ir zR+6|Jyh86Q(e`S^+yNbNO|Dl=uOgcpW%Vze*S5RgyIE$L{fzW@ccMx4@;YnlkxA?5 zaW003$Fc~VWK36SZSMTIvt1ql$(QxQ$NOCkX3yfdDS|@b>U(Um*1NaC9boQ^vC3-J zexu%o-s!J9#DP10tv9j7EqX!0@7UK^!6&TF4s>Fljo2K6S5MV0n9Cm|0Q3e&Q!rA= z<P#BLp%EkHQMlgz2>npX9Z$)8+E81nn+%5I`6XaO5-DT<n#I5(7uL<aWJEE_e|T_T zb1F5(VPia-b|3o4PzxX1>|>j8V0%P3hEr&E5R&YWX(0Rh&Q}B338(XS`fzLR;O0^i zd>Hn<8c&)sFK*C4k~U4@vH;Ce=+&!2e5nwaToqMrp`;65!)&i}-NFU5JrG-atd}08 zK?AM@KeF)*dP-jqQZ@nvt^QL%gXO>D3BQc`kD#^uZ<wh6cW$<p`+ni!o<~a$EB7dM zXiK>_*#iOk;S?;n2L=z$7UxKT4FBS~l*jqV5r<K@+b^)1EWf9}T8s2~k|<7E=>3fL zc?yV&`?|@ewX^2-Wh-^gXstuOJjO5YEOQBWd8of5@oLxDN$2purs%J=pL_ArjuQT~ z`pGQWzw#ySrGw631ydqhJG9;XUw&X4AwKL~`rM8aD$d$;T{udabsN{W56yK?!3~Mk z4%MMZK8T74XzxsGaW`k;61Y+_7WOR4s*$=FT3yC`ppYc2Lt3S*<ZTVM*6#GJ!}4*W z7Xw%Ur-2EaOmb8CkKapTAD_(anV$Za64NC}zKMk|y3WiRca1&|N1HjqD17pE?RF3I zpiXX1c32Hg3ebRHaJ$mS4#OGID+9@UD5Se}16&`RAaewayHUvcvv3QC4&A6TxtVdr zT$1+9pPc9<5*g9&wWo?ZQP@+(nR>wviCb!H35qsum>>o?g+x^38-2Cux#N_m_E3sN z0tqF7x<Xg*{;$}r78H(Is~UF~g|{xi?H$qLwB?B*p`5*^FBD#~ZDdngeri3qDp6-- z{o<&O(J*ww#vHL-hMxrmqJhJ47Qrp<E>NdRLU5MqF$v(<tKh_8bpLFLFZzS=dGc`A z`Eka3e#g=HR8L6L1&pRP_XOAbv>gd`g-)XXqjy=ke8ct%L6}x@&+K<Z2xv4*YnB$G z@xdz%;V-^G4M5e#Pad562mKk8zDoS;0a`*aTZcbvd-+P$iu|Lk8*!vLP<E|v09iK2 zLrSoMy10Y)sr}3j(#hu}wYQW=U^fOuX^pR_DJo@4!K2uMSqM0Psh4|JyP~TpVyDmy za87nFLD?ncf;-)H9cYIrcg%ylY<;3C(KO3>e05ej2PWVuP&-WV7*Xz-^YdpaeNVp4 zS347URKFp(y4dzcf?Euw`K@p14Q!Q&zAE|}u&1=ZO9lazgiD9wRd%-AyvB^#t4>)o zn<PhMD}*zZBul_nTG1H34!A}p5k_@V_N8hrM^2lp+m^~T#<a=^a%CA4c6Jt@8Fr=o z`US|WGEMsGLtD%#&lr5j-vMPud14S4r+fG*+Pedey^t;yNqcPCF{h^dphLh0mt}=> zTIh5Ujl*cs#>u;pQp2VJM{vf&6*oV<xG1M&vBp<~UOl%B2nwLY*(UX7@Kyu6@y(tG zw`tFx5?mX!hml7Y&h?r$B;EzEB$BvwP(254=XE$YztkSR?44VlB|cz06%L**_sb;5 zO?1<jwn8aKs%lfNQRC*uRb>2Nj_6aiBDkj?Gq;%?$-RYrP1murR10)yKlB$jpRoq* zU7O+1_k{A7X`)3)<rRsadc~jp<$M*VEuX;|6xDQOhNr*Vku3c>%S6uynj4a-7SL)p zY{A_GL;yC~rxz{!hK~Zb)WIvKeOgsCpI)x#cu%$6yq%wB#r)V&9!U5b6c7uI!s=B! zB1wDqDUsYUg#?XSz_9olF7?<K#RuV(5<v5an=<o+mZ@R`D{;Dd^H%Vi3xRW9#y+1P z9<KxfR$IljrSHWp9~d>xcD{h2wDDc&ny!|Y+GD2sBK(aaW{CO3T&3Tvuj8CNjN6N2 zc^<8pBeum+YM(Y_a(^QMr^u1Bg5DHL?aMT55*qSP76$I$#wd9XhZgTn_04@GZH^3E znglJ&eDjmkh${UN9h6h?id^^6oQ?kIhlxNE{|n1N3f<tO2&idgCz@@7pC@r7owO2j zCqe_7e}Tr}sk=d`xn%Px_V6wSh$;#BUwO8UBYIyK8PKa=?hnuL^n1M%>R(~3Up*`2 zijvce&z>hx^xV344M)^U?$&HBi@N=CsB!yR$aWt@D4j$@85l>8CgVft*s;SQ5ux&v zuRW5-qk1%jf{J!1qa-^6yn6<S8!{Rc6d*v4X~&rSb>Hp>aAVR%!xZca8VP7<010#C z&pr(kf!0j6UhAS}@7lX}z714Y-k-Mr2U6J$%r9TLNgk@iro>GrLVqrvwAd_Anl0%1 zNXlv{{r)9TfBC(>^h9tn+sIz+UU!XPOV+D_OXveoVLr~j@2jP1&!}hW_$mEMQ~cA} zyb|tYM@Csk%p{W)s+AS^SYU_@HzktNfMc>tk=jufPq`bxkAWgW)u9_gl_#s{wq6h} z>tG`AhC9kff1(D{|A5GBWz>?bPhM<^gF2Z}8KFMxG&N-#7Wf)HTQ?+ny{83(w0{iY zX}{%0@LVcF^bQm!$DPJOmJ9`JZ{7m9kmpTCW4yrK5Wa+krveu<e1OxeC3f+mJ|WT^ zw69&*!hofp*!GDhAqnsv?|nLWX^9sv>Ud*Pv0edJrHe_c_J+3K;Y0fGo2K7-<nv!( zeq;WknpURoMs{Y8T{TZ`6#J}0g7QDb-O{(ZH>^3KpC?_WFK2zB=YrOQX#|1ZRY}N$ zsjg3wbQaq1zOBrX2Esqh)oYCB=NAGx(#X}&Tlw5RR8wig^q~--1elwg97Q}g_Zmel z?@kHWkas)hZA1u-uXWbP<XgoZ>dM8_271IRIjYHLUr-uPBp=?(Ras7yfm^#HYOSK& z`wvMb^~2LMmRw~tZiUa+5rruoQg&l_>o4?H(nG{Q-Ana{or#-gdml%+`dImrvbG{( z7p&tb<2KF<yvm!se7dRj@(EJ#a?weg$>1iyEl$<3+|T(cr$3H{GD2`gSx^hn7h3?N z-7f#2g>parXHTO6Xp+A#C2Zuc{Zdc36GglYx@H|9PCaBM{&in*V!%HPSi-P^+!JO5 zI@rugFRTlbeLpC5i#EQCqt8&7BKWgRe%EPME#GG`?dVxT9A|<C2^5$d+!z%64F2q- zeHTciOERD|1?jbxaw!O@+o8xKd~|6D>p(!G9fnHgQW#ss8N_Q1c&3xd57=V@14Ul( z;Oq|aNiyHKuw+(mm2ptbABVYXT46HV*GPgdjvGBFxMN#vS0!oI8@L~%w_{iUf@6pe z!J}wU#&NgP={AWH8DsoS@;|-{eIIF4Xopg5(CA$r`Op<MJM^fZmW|p!C$FsX1cO3- zXT{k~t8Mh|$DZr#%R_1`cC?(&)UmIiK8gCL50^*qz!ra!D|%UMKS$`?rYd`au@Wxq zZ8zjW$5g6bF+}Aer8`oNRdVp45QHLPX!xcMY;kualCjh;XL-I-zinD@hb0t?Fhb51 z&z&PQ|6;-D=v;9yZtVyO?oM4zi9Eh>>xj-ym(=xp)QE=7Xv{$V{4qbf+kT65`SQT( z!ZyvE*xJEVow#eKj@8VD4<6E)84uEj`&>;30OfqZbRZDZHBUS=J|IdC=Y78387%)% z9dc1B&9C;GL0lCl^(lD;dekR|9TQ7r*scadjrLb$X}myZdUYo;Torx0UU9+a&q+K6 zK4o6kXer21DjvD?6l{8}e?ow4KMQBv`LY4j_lk?k1I<U)?pm7~8iKwsU=p(j<$n9! z2kMhKqdXxx1eJ)+E0U>r+oK{PaH?B{SH*qzj};=~S$xWpk*YrTFKJ~fRkm`kA6J*@ z(N}Xe3Y2H<DxE}q?0SKDZY$x4g?Clci`I$j&1R_-4@bWPB_XN1h)i1}{(~aXXfl~* zZD{|}d`59%eJYhmhzO}8dYrVeA8uC5w1Ami_IN=cm@$p6qNQyCx2Yd%u?h*-f>sg` zd_4%nK)XGK!B0X5uzJQ&ykzsh$u(ATY$O1^q0w5^ggB79gS0qa&ySdKa40%KHcB;6 zSuzO;!>CpsnY9ilN0f=q%y4Dq;hn8qwyJ1qlNKKx4x-X<p)p~Ej>>n%%9B&MK?4XR z6VrUXNWt|*BRA29)zaX!+<gAfaI)SUD(cF_3Y!OD<wRXt^OTH2Es}s-Hp>%fR}Xm1 zh)0bC`jGnm?+!;tk`SQRu6~VKx=N|OR5wj=Uc%_QBZ4r2r{vhfwQ+~O1RC?#%j#l_ zFq%tNZ*=in4T>4nmTeIZUgv8d7i+Y-Eo94Z+TEXj|F2#QO7z`i_A{c#-IYcf6OTsE zROZjR+n1d=Z<Ab8;{rJ60~yhAYY^%pfdbEicuZe9ulO8LoQh0nz@;!D+}>%+j1JTn zd+6vm8?`#Qp7VM|4Fn(<u1|UTtBWttnBZ&;EGhG}N4|#g=Z(d2&hk;V6qp-Q8S@Tc zfv{W^3rhJuTZ<b2l%}C+a3<A&K%uPhts1Ve4X&vGB(I4DI>8W8II^OkLUcMnV0%8i zr-c?L`(fwaopm_}=js0UIS}xkC!hfcsZ1Uc`D4(y%EXaKXp!_}&7Sgy>)}~Pk7k*v z0R*<I!4lTrFP{lc<^nzN_)})PFjT8MS4t+sAjV0Y3o^L0^7;boGcl0>+iSy#a$v~R zeX^24%(kxlnZBzNfrHfi>tqOoyp%v43|w(75S}?G)<W#8EOrsmqpcD_OjyEksz_U@ zA<1Usg)MY^`0t<D1c`3BP|l=M175IV!tB-f!$(h_W`zJfE2WI2<!Ju2uvhRzJ4<HX zcrk*=UKa~ht6h>apg?N;OE`O0+b$p?Yc&Fa4;>M((f(+qN5a0fa6{?2lCvuLHUtJ~ zs?$>|(7(8KG&DIi>SSt=D-4F6OKZ8(PI2i%r5OSRluhu66AmjYKYItpG80XMn@&o9 zR`GQZ{5deuBqL;2oG;ZZDUr_&L2EFS#)4iOjE8~wMjVvio6QBl+}v)l0*m+ix|BR6 zq7j@*t-zf3jCOGVB%GV-9-qnRuVe{8<ac-VxIdi)<x;@?7;}&cL@}#I6D^=$@kKFx z{d!~Lkru|CT8tTJ(DYl}5grGW#?K_f{hcNS07O@K7$#<A#U<(OAbu$cevT4@SWLHc z%bVTOg_e#=220&9FDRErTMU>>Sv@<-AIjL3V*mP=gMK7dWVl_LqBz>zeAM?E0)b*m z(-tW@b|C-yqZl(%hEkVNw2uUR%ev%$PwfoW32O$$RZzsii+!`7Q&yF){S3^1cz<&M zQOa^}ud$yq9;5$y=a4dqMi8Wo()uUXucO%AZcab&9@l#!UG*^*LMtD{)wQJ!<FGp_ zs|t$k(Pd9v!rj&_&eBePwZHf>^~{{|qje>0#VA_7t-GV0Vt=7IO_^w2S|1KGCn=&7 zIiMqlKFliD13Y7lJK7x7ntg0O;-~v1`zg0pU=VC&Sr_guH7d{#*$<^ee(Eg@iS`F% zHA>;eTJ<4O1GTx+rl($J0Z@RWFJ@}K3xQP1SdkK<1Xw00W+4cO!<}9e@|b5YYCH+E zFWSfJrGrx^O4gG#;Z|M={+0UQpTC}7#2Ib8d!Ua7GQO-kqNNQmX*UEU0pJe@7AE4U zwf@t!j*X40k61-dQ|KSSc*Zpj9>=l0*@|=`jumLC5r}r@uU|vj7K7zem7BeOK_t37 zhCmC^0leiNW{O-pQ_NwEDVnA>L($P+o!;NhiVSBkC^Ts;Yr+#e1qvfIbc<X=L!+x1 zX!Sk}JC7k?wg`}^s0$!7s`D+^00lOcXmk&WL|q%{Q?O-Dhp6i74UK(+eT|KasOBQA z-p<bxBT;UXx3U(xU`9Hh!x<0+J{6%iIV5aW+RSBs%*)5jh!#o^VfrPLiE)0`+@rJ0 zw6CXL4{V1+vGlV84Q4AZ4z3Hi{^7J`LSeh;6Flv@C$u13DgZ|FnM2z<t&Myfgkuf2 zhT4FbPP%{gX1*cd|M^TZX?}q`TMfn7#qDtGga{^@(Un2IYi{mtx{z|d?Te@V?CBeB zCc9=Ae|8*hYHpM*&FC^8+%+g6?9uX~^wr(OazlQH<cdTDvX{*wU|dw6Cu#}4%uryG zFlidxPtBU=okr|ya%Syy$(i?}+No+>C$AnegCRn?NkwemQ9q{hZ80)DRKKV55>n@+ zrF_6xec$!x3-5M?t7hpcw?AKqOMFRL_1?t$qmqSty(Mj6DiAf?M7yNXV2p=OfuA`f zBa>sjholVH6rcqddf`ip%Fh>sbg|fg9}8rHx@*{h-8b_G>|28~r~`VU8QhR8o~FUQ zVm$X6d{aD^e%QJ#Rz-<y56q;h0_|x*Ku=q?*e*3b#BfpdrKRF!i$0d&Qpb1a*s4V5 zF58RH-sBBfP9w1=PNJmBYH9(Zk<h+07u6vKnXz<BU%ld!%VwogfWRR@=A*nr5|m0U zs1j;n$sR~5_RS3A`;CE}PVUQav_N;4R<v6?yWI`l>f)Y+bL?@#<<Urrq-=q<+MGeV zKh*wttqCH!YW*RA<b>8df815HKiz1(<-p~CrfcD+F|np^Vcxs=+ty|2{Ww#AoH6&% zo#cyzwgikJ)APFGIg@CG*hvi-ht@)l>k0=EIZLZ=Unl@u0cII6x44LJA^Z!4lKC?+ z9iBtCzQH?K4wgx1B&ErK=cc(pgvCHGS8NR*-4R`eCMk0^@ZhL4ck!fIkTYX0{Nqgm zXA54u6v#2s$LYCGvvG4HO>^;rGg?keO=<Rh@bJbBp*f9zRf2wp=(Bi!dls%FHo6F$ zr~^jT$*fVYMQQFbS9XS36h@{DLEGn^RuF8c3lz8p?bhT3Rzz53a^fhoz_41nWuB7j z&9IObGb!83Tyy{$iupBMUg08g@)^*m=X0|P+;{FBP~<g^s%AH#PTxx;X0ym%aeJ^L zOpTq~U9X54dSF2Kl|)f!m&^p_HfFM&7L^=+Xsbu5HO6&S1UE<I@a^lk#1($`3c<MJ z^)84bGx1Op3y%6!R>~o~A8voFukYHJ1yE)-pw)>!Y}+;oIY8agmiMNa9*?C0;5E;h zHZt=0bU-%>p5aW6&N2xd_SY96bo}-0C)BUNVo1v5@6@~jh<6gp=2vF&@wdr}H$BYT z{4PCWcnu{5WIqkMf5GmJVYAB1Ad)%YW&d!Hr;EKvkJ70OOUUK-T=0;^+mHL5gr0<q zB%rXE<C{YzmN2N69Kq%d$;QeB=7Ry9-EMStqr5tRwN1!i;j@uKgpJ!_b8wR;3n>C3 zEfR5KgQKbmo0CAPN#e)o^I~h<*%Y~*smuj4Wl)?JMmXI8iCS${OeonAC~;6QHNP2d z87I7@!9)1R!d8j3ifO>Ls+-yplcA1kmC*3XzXVu6ap`AXI@6oLTU$`DRye7g8L|tZ zpEjfb+C53hi6{uQV+PGfmYNmYK&cfMz2Hn@A#As71>D9s->gk`+WGpOc2;8bao>Iw z+|m*+q}t6T$4O})h=stm(t^*S)}vJOojv*?LbHPePzF;5I;L%%b*y%a&;$ig1fR%r z&(EdrJEy-Frq5agd~+-oM}-f|I^f1|NcM`aXW8ji6?K547g`8XK4#|3K%L?MWfbCz zu0Te^JT~LavfwTq1(Ui=feqFWFM%nOSd<F{Ga^h*K9pX9Z41;Cn0F`E!Q~Z|sXp8K z)EivmLp<y8g+u*IVql&qF81zq*1^`Wnt`R(RGt_o$RL7m@6`tLO-J;YdmVVVBnoy0 ztQ-0{?+UAY=$=Nfjmdb*%79@IKv7BC)~E9)?SDPbdVEp&Z`k}^LhNSK|Dr6jT4wrJ z`T;cNsUw{T2Q-t;d4Wl+2bGvPBir;f3=Ue_tqw&Dr`hJfG93rvw{Vju#@iM`hopYH zrS?_0YJC+N+oKRycSZeR?g<SMZTBZX=oTTYKp+5sgF_%PL)jkr(ExB`t;7gy6WzN{ z#i7_Adnmp`G7ib$YeR{H@zHC6_9u*0m?Ewg=|evVzYJ9JQC^Vl4BDNzq-#s@=556@ z&gpIJ>Lj|`ofd%rjvvjgu(Vy^JZUHZQ6_h6WNlg9F`pn0bGzs>?3HLw0ZOK&|M5DU zPKimPl{Zeo*d(cX7TUPF^a~>+90YH4G<wg9xp3p;>8YBWFps2b{&?jK$gEYWx3(D1 z!<<HbIw*X$Lb;Col*?}E>21adU``7ytCf#r&HikiojIc~8C+D%CNYW3!UMh+0Xdsi zJa%p$1_QS`eLF%c*M|;d-cycTNT3ng2n@+=H5Bb2YKy3*W@TT9jMnMqPRxN}#5li# ze0*p1fWUan)K^A~Y4FG;5kt>L0VD19O>3u&F_-A{u@MHIcS<E9gArWQzBk3&!xyAd z4Hy~g+%~;ClzTj}RZEdMVB9+->e0TnJmI^0V)0=rO?PJ0vAVOUPhak5s4~M34*5kF z25O02RuL8f<kvLvSd|PakLhRcQ!tE6_q>Q>{_BoGq=8f#?NIsMkGNodk7Ylh7DoD8 zzPfI@YFNx}*s<GHl$AAFAR8LIBA}8EaENnrnY7k4G1x%2gwFv6l-3YOX}~;6+r&J3 zrpbEK<pG}#3*Mql)_tH1VVln(LS;JP+>LL!U@enFT-YvoYpfdnBm?&Bf@OHevw%+U zNR<H|(&&&Vfy>BWjHA7s0U^svMzgEe2yb+DSJl{eE#<^>v`hffK8eg-Ib!p$35ZH= z5}7G;Zk%*q^70w$Uk`XiORbbdlm;NByg~_?BxhNeLBCc$A7><$B}~vTOe5~&dmARs zotTzJbPr_fT)?GJloLIi(i>qk;>rz=9}hSpoIKo}ii>mnOkQ42-`w&=W1Po!xvcF- zEnhzAm-46a){EHM_yRk8D~DsL$RUfV1i!Yw-s%fDz8_C7(k|$ygu(YpZpJvgCa5gz z5rLK^>vQvTkX<$?3u_0KNH*~diAHfFDBFo!mU)+qkEVP3!7wP3Uf{|L*1y4G*7)n! zqpZcO4g-UdfaDhx0NmOOot^!(ktSw_&U!;}Nr}%A5Eb1#&YUEYt0*XFT+&5E=|j=< z9|0W|t=$~l^XX$>=y>)o!GlGDE;{5K{rqWO_{J-W&Yzw!e;C)M$@9{JN@+AeU~GqY z5Kiw*B<7HqHp9|Xm#W1QE}fP?(CUxm4>Si|42@W%F=%{!XE;1D$fP_A?m$ZdjhZhO z$MvEw3*)8HHSKT#$bZ+I%5UrFk#v%-aEB0KAZqEQbl_q|krJE>MX7oAw<gf0G+@}T z5sAIEB+^WW1fBUJtA&59=YvXiX}%1>Z0-PRqgo|BCn>&`IF=Y?=7<!?;b1uYRLbGU z>?)5<=Q#D7yDqGNhr5l|ces8J$>Q}~C`goaq;?B(t0HPdZ@otlM-AqfX#@VUglq#y zWsHU;X<;Tgvt)_3&m3ev^ZX7iX$`k*O%m?D+_2dep;STdlq9yCR!B#D=d<hzh!-Ts z4+{(G`8plP5;0N?6?0thS|Ai8kx**a+lRf6UD=DqXV^Gk=dgw#7W;bNlJca>R@7LJ z85N`5m3X>xbXYH-LD6v6GPDl}URyDKQhV<R>zb^W8M3^|hoU-b4nq-D5+^lon2;PL zp(ocvSOQQmHb;Zou95p}Tj@NO8%~3BV^2n9QToa)l4ofo^B7W2=o7O2Zy7hzS9+Qa zUv#>;B0uVSJW<u_Y$mmlLM%1xr6T%$5?t94+i6~te0{_4v7%C+oW0l?=(KDPJAWjJ zjM2OKCuDec;qpN$#?87h4_|liW+ZQNmv^RCr4=L!$mvua+eij!`SSz%^3zNf+u|1d z<ZM<4V_cx8yuLS23+#%T4!88hxl)EiDc4=@_`Hll?$H6$<hDICmFVelM#)W0$>_+F zhC<5xXSd1N+X}5uO<VfZuwW7p^sQH0a6-<BbPT)S0%^)z_VHledl^{j(eW=S)5fV| zM7vU_|4?>%?u&Sz?xr+3NE3!%pTXIOg(K;@F{1e<)9X;eFV@x8p{La*u76dWsCAC0 z;3<~x07XE$zic`7(5?15A?1C^k-R-y@)9btnLDSgvH^s3d$6>z1M4mtq?T|Iz2YM3 zA?o4=EdIQF9Ci+?4{lBwn@bE6?KU%Y0AxOc_BM={1iR09FGv=mecTfslJU`zg93YT zOo1Jo@g$P+4GQO+;4Q?&^kJcoTaNzub94*cZc~hIGLFQb;6R~&lI|MOw~CDqzYY(N zjCe>+aKWO9$K$o$5FXMp@zCQ4CIsQ>3o`==r}2dIk<GxyW{mfE;}0q4`V0{PS5PNt zf;VRJK9SlWmI8Ajs5=$>aDmk(QT?&E&SMTv9|S&6XJknCMcy%W2@rdP%wEgdul!cz zeevkyGTT7sO3FwDl~dss9`+PIA%681n@s6mWE&6(nC5c8(lsyV9gs(PP7hc92rczs z1*EYX;^fJiOiBZui#@5-C{m?XGQ-G^>`gnqI*TpO>_G@HJQ>KO2~5KWF-$y0DAG#q zt@IR34u<YSY^YiH_QOt(`~zP%?EpF|m0UQ@kKT6(py~?^WHl#Dvft0;rtbAf_j5H( zAf7TwM2;n%wpUPGaCvf~euab5jp-?IPA~_>MfZFui753z0sPh|B0G^vM_P~}qobEq zrQ0l5Oo}5#*R0Y-wylJR92l8TH7-l~!I80%rumsuY;$h{jKzA1WRep%<LT40J0%5f zJdY1G|0<6Z5BJ$=b-0eUKW~haZAq4wB5jcOE*<9-SWNTY_r^x0k+rpvju~>|$Mtgz z>Xr+=pZT<o4LGwQqR7j8PLpHCdD)kv?F5SXPG*4TGUNRr{B9c}HE31DM)-zGC(vYs zLIA`v<??fOH4)wk*4O2W@XAA>auYs&7%qXV9JSn}5Q%GN$Inb@Zcg!Jn~;z5y>%z8 z^3vmGU7;TFwL<%I6im0bLCFC%Q-^5POQUw?oOW(4%3o!?IS^&_RtF+&ldlJfLJ~Uf zM+45QzIfJS^;%d8uD;1{8XM`_dH&`30P?~}5KCuNoE&~*P6xuc7wzHzhfi8dI^1I1 zK?i^(IYS9uox^YP70Q<vn7PQ+ciS<I-6<O=qjb2KUa~*2fpvh!rkDdQ6hQ%<x{5ik z8H=L;r|CIP*4cy|Os)38y(8K|GFf%n>EYqMHOIy;UmhPlW)g916w1eH_QvJjhlsxs zzRRIMb@u&1a;aLGnikCh(OuI)>sTNZU)6T+O%J?}F;*Owza|+_T<_`~#Wq-@lQQe; zoozSdrLkLV(vK&*9zm(eQ8rS$3sVd2QGM&{l&w>T>}7wI?C(<?TtqE*Omb91UAfba zZ*pvTz1@OXrOn$)US@CZqYN#I=?n17u!of>l~^;=Qa)VPBkGn3IpP+HR#54sm{HY` z+mRkD9%1=qq|fB0SeqliDuv(YXIAV~ZgKgK%|}d^D44=pDbsI+P4mHNj^!aETG1E; z%18w+gU}@LiOGOh`t`J+uUxQjskjx;D#*6=jSCkq50sTIXTH*TAUTuoOfr{&8gQp5 z(IZ+dDQS+uxbwB$YU{MpYSgV6Js%ppFk+MQ@*7}oqcGrMU7Tw&lSwJMSnWmIIA)e^ zM6u4dyCpc1LsKr^Z`u`$#G4rQPG{dIe`MWotu39|N|QZdx{AG7JZ#+T$Dj<OPa0Wd zV#!EsUrYf*3I<*qUB}AO9no(Ns`1C*k3Or(x!9<v{7!4JSqY_))U`c<n$*MN^%rJ^ z=T4=p>;p*7UX{56pUxSdX5*+lmX{xiD172Y)8r^qOtsfs`JakDoOQx94|Zfum+8Ls zezZtV@&Kz_v2H}f%*thGFWQJGGO015Xk}l@lu>S0J&{A?_VALZ`AGj98-GQO?`Ion zey1g>LZ#y|HU7rnV|vAv3w8~GK4I%wfbk`<Wxgh3A)tvBa7jW9%Q(=pU4B6=Y@s$Z z!!f0pk}nmoTU-)cS_lkD^$Jup&>UB}`S4+3I45lSh<H+2tlv(OW%~feYBM_A?JWj6 zN08e#7v&jt)8S!cBqU&iO)T7$L-h8(1bXl@w;^$cF_W10kKkR?i`f_7ZX<Y(wL%V! z%IC(wY+bm%OT%8b4|EQd^tsl;6JJ1C(EJy5?3`y)`n7AW=UvbOPsXH|CU!dYm>*7q z+hO`l8Q2kJcgc&M^(|;weL5bf!FXvPPq_skm5O+LD_)Dkv9d#P0VRZg1LnA0ds|x@ z9@udrnhD%^KuibLb#T>`9o55XyXu1r3*6Q%0o~}MTRq8ti@^1h*ru{v4Dn@&i)wLO z{w41mvtC!Fhm;x_C*nwI(|N*U>hvW_IEolaZFrT!<vr*_FZx|BCc$?!%|zbAEXJ4@ z&J)TUGx@pdlp?lzmK2|ZV8vCbkQVB}4SY>HA2U&7A(LOnqvi2eC;=E(YKM^1`El#k zQ}QEbC`U9$-j_)}w5QbIh2(D4+Jr@t1`hn$ssHzl@?M0Sl7Qxy%a@DVJVYcuZt+M* zTgMhni6_ZJ)FzV0xF>J;a#d{z1%Moi#u59?PRq~TzJGU00Y8Zn<u;dGK|F|$PQkYK z4q8h!5~4)GDVY3)?;S{KJr{y$*>P-B1<utvoI)BhwkCO5k?md^*#;M4*sgVb<`7vc zY=2|?zFOODOvlwFIiwZirgOo@%&J-MPx3M8-?~{uG7e0(3cvlo%AzIsqV;1HD>t17 zR+L{Za&t*>4R9ORsqnewx*$Ff1j%AY>`r=>#l14Jah6z<{Y3dmuGV3S_LkZwNdFL4 zgH)oe?3}!rpC6S)$#jo=`r1deGnOa~Z%=e`N^B385_1AP<vZ2#+%qXzQTgbBU}txS zI0Yhay(aJ9H$E5cW6`aGh+d-gKqd?3(RAumlLr!gXS*D+B!Yd2?65Fsx@f%<DTqf@ z`T@QG$8(h9c{FnD#{dl=6aas3>J3fuNIMJ8rg!Roe5xQJDC_U?_s{tY_J-Nuwi)+f zWY`BH3AvFA+bwfZXCvY)F-@=*oP4jXFR69SX!cT+vC}QbE^8!5_)9F^g)w0jJz=Z- zj9E~}LB=d`lqDe%*8d7mP6ZWuc1||eUZutZKJf0wtU>8^+)9T=@YB7`DX_^3FP)i+ z-l}ZOlBq&7M@<==uP0j=kQyv*To%6Pj9eXS-qE8CZ7~I<Xl=A_wLT%!8uYvH)oM*J zLm)h+>F59R2j!o&fVtm}T)n)zyOF+NOMiR^UwBUR5fNa=fSkCVa9152N(|@>YDi4> zO%JI&l<HrUl8HSB`G23PkV6F&sEP(nKnGXeh0Z1FiMJNG{l|ua^x>0c6qkRajwR%$ zO>Wq5=AjE(0Ms-6Kt3n-O}y}A4gOiWEJ6fSvzK+T!b<fz`2!n+=GbpOueD-c9o4zv z$dQuXi${2lx?fl8V9x|d3<QsoB0aRvpQHW(dL}*I1qd8K{w33lNZIq)acPry7qCP~ zx>$J6YU+fqO93Djd_VvMQB)SN#!#r_D+d_kI&~iIvSZzS(4M_ivYX2bq40%5HH_M* z$^tksg4Srrsj8}+r(w65Ms@aBOk-Q2Zcf*zcyvzRM4MRH#VQd_I0ORy@W$NX!*e$t z0v3rCeE9YlhRre!e~<-Idp><PIQZBu$Kkn0j-3S6c@fouu%Az7!6W!l4wjLcaq}mj zXA%m}W|FG762b=<OZZucv|zG1k7)BU>cWJ{Hro9peUl!p4jv$vgDAsPKfCX;7=1yl zVD}F<8`K3jl<0sMOc_W<QW67+M~VsyMg2xf2v~O^Q^QNra)up_00iDZMN47&+%n7d zC%);i%8ZmHc3aq=TQ|OfA5*SIvueBn=xT-(hg&u1XrWC8bS$B{;a>lt(rF{w;X`k) zw9awDr~6u`W$5Pfn!R+azh&bYS84v0w}<X<U)OY~|6VFSvbeGNR%~eT`4?K6QKM>D z2dB>*Lf_-4s)9MGaRN8iK=~Q5i-NDXC$tjK?G_&6p5gi(t6M!~9vq3pNGo2^m%7E? z>R~VSM}-qMjC$2P@HQ!V(6)!=L`dX!M$6Ch;}dq}`uZ|%M!hK|!({mL?*qB+E}bdi z2o%QKl~6Wb!?$t?jpGD+s%ZDfJc>-pKeI__E~mGcj<jTGbXz2y<sRb}T?y>svS!7Y zusJ3)F4{W)=5srbLX5AK{q_nHnrrs;8QkXe^_70lKB#Ib&#-wSRLkR?ylTBoRU3f< z>157=O}yQ)t+ZSJghcUYG!J_kE8*RpAE}H2p%*%;JcBuLsRFkF{z1=w6aoc*p%r%r z2~2&v#X&v7qc#&8uiKzycKF>vbrF;+Rr+85ANEn+GiKgDpXB0|8&bDimk2NgQpNxn ze+{HkULf-<_n7Ne(RYR1SE3so6@q`V?lR(FK?xt_cBx0HJUI&wlgc!1SUaIVy9<dw z@~0arbyhI@tK_uP?E`yT0W>165W~)bEVdWK?t&E>anro9=REA^l2S{WD}o3I-yMc) zHON<ZV@B+R@EA}CPyBx$Ju1TRr8`9u`-26U)fhyBe!tH;7t{%oZCpXNzHs=m+WpJx z8C+OcrwC-X|Eu*Yzeo;iHv9itV&3fES94el<g72$%>yJ~x~)-!6B6-+T3?r`y=Z8V zO!akq*TxVy`3(ue*5q20roz;H@kvO+I>w7{OMSbH3d~_IE!AtI^LSQqFvJ4Fa>~ws zOhb@g;DiViL=ZM;Cg{79Q>AfzaNnr%J(?J}els|}5TWs2c#c!wp<}+N)i_mc5wZ7W zemAhVwjT7ER#jTZI`nqNuM6Z`ZRtLRzY~Bz(+$xG;BXs#^j`+y`4DGI214ERq58vL z3MK1bq-Q<%Noag7-KE5Z^8Qv1UNPj8x-bbMdy|$ohJ$T}bI>`+59*tyv-HtI;PvcI zo|H+!6L5#jX?qG?N~|F25cWDvxT>YndE_OD#dU_~)dm2+`bXvj&Hq-`fuRDm3+B=R zYXWOLZz&qidpsRa@kdJ6rJ;C3PHHnP%c>iy@9_{<EjJf4If@@b#*!#h3Sin$3?8bl zh*%%#zG;e)3wH=n<vQh^*TLKHfHvc}DH0-65g;QI7Pl?vj|>QpEUqGU2?+IsT<#j` zWPWZHu#qxyaxzb1yEcMbmQ;b((h5=-535UK%USd1ii`NKG-F+nKC~31jRuTxdElq! zfocYDIvNB=U9Vcu=-9|45-<vj^!i`wPAg3}`dKm<Y!wPX2Y83-HV1K?;R&~$G@sSq zH91ez(4w4u5mb|qa{OjB`kylYYZf4M0jl4!x_!HF#8q6<b#l6!x3P0pY`E8|g<*&F zyxy{o4SiN^+VRJJGyi98hK<cE7+g1PeU&Ya!GtQ~nh-2SNSkp3=OkHpxIGR<6`1_~ zUsflc_Cx9!vQn98hfYMb`Z@Ijoslm93qu)Vva`Y1_;{V6KsaE*GD*y;EFtmH!UJaA zgtBf$@$)#|V)6Pb5>b$pGVH3D>%Bu-UOz|o_*Q1(?DprNv9bjF7brsO;7Mik{3{fR zIjt7%It@V#4hzHeobL+%ymqLi)X+54QbM;#AlG{5(X)B%eE)bGzOJ0squW<MfJGbo z5*l>0&_+)V&)k&ZlVcwHls)yDF-7GhRwz{SlA71SeGBHRa#<J-qajEfXi*_9MlTIy z5T5q*)vHIegPt(|@6^#dY8MAMHgp~Sz{KjR#TKsY&<Qx))p_Lj3*S*kwEZtywUA*m z20aD}%g^*rfvShXU)9*8N}8Ea*OH=1u~KuRT0M@CeA>K0Baw`(tc>suBaw4;>+a^8 zyE`uH>D?LzyZSD4ir1++>Pr?$R3{gKHkcZf%5688(jxLY?;7mlzH<iy#bOf2ao2fF z;y_v~S+5CVXPA!N#Vo=8ziyuf`-zx)4o|o{U6aYGAz$MnJ!2R6)4BqNnu26$YMU~o zN9;nsShXa8<F(*?sjC-%Y1<*Op@J%IRPF&Kn*tO2JI@s=>c#ftUNg=wW9_cFMZljE zbDsz__PRp@cT8%1DH*Z(;yfsZo>_26cjDdiSBqYf{YXrVEem$b+i-;W#F0P&cizO% zpK!&@xt&$|OSqT7p*}I|w}A1)Ov}EhX5s`eaEZ{)j+Yxf)L-k2@t+|J2|508##_3& z!N#qw`E-OWV_Xf@2|(3x@m;c#;6p)5w6Ac@P+@O;9(k#3PTuN~dk;p2^C~m5M$q`n zcuap(cA~V<Fw~n?ERm4A<lg)9)M`leK`30w^wfCb2m1S;z6W@8P`wTHuc}g|Ua8mR z<p5K^?`R^CNU+BJN^T=xaKA3XB(1Un8I5x?@M);=`$Ej-=J9(}(REr&3v^u6Od3gt z8702$@hX}NjK5I$Jr;ZUV&EcGxKW29R?|{_2|ika*71tc45fj^icYb!R^6@w=`H(8 z$?RwX49|n%%?-m|soB-R3=xbwlBPyc!_+J)E9k8v^&-_UFe$b=J33aBrxH*&DrG1) z(n@TXy^f2UFmFg@UdD>z<#{E6V7!wZG^fW|(pzO%7JafdOZ-X&%c+Es63hSqUL!oo zoyiE#N#9>D?yfR3EkLnsvow~=`(VoKP~trS=1V3$E-C5F)tp#%Osa^*X0dPC3!RHX zM_t~ojTX`?0`iOI*n&`bxX?+CZmCva=4&l}Q;fxA(Craq{Q}ryRkxQe+Goa>C*2@1 zPKy2YtuRm_^Z*E<&aZ-pNR{oVT}WoI5}prRv|7S=%N^py1zaw|Ad%pJy(^+zUlueI zVwk2+cCQ-$f{KzOyRP=Jh{bjxf^5tLEYx^B>>5N9cu7tIEk+Z9>}4!3iCk@h-qU2X zP<faV#7T07gU662B#IG78@x;LOSTq-)`u~~aT1q$N1q|Dqq4O@1?{`Q5xFxstr5(P z%>+3&RXfPER%PaAAh7A(j2^#CyZFwKZ=7^+l2SZ#n&oRS1XbWI3xcA+g0SYCJwuqw z0lq`Ao}SV699L>VoU*kH+D~c2?VpULl4)!(2N*|mV?75{qY12aHJv=!gz<&?Cryez zBL$AD4emjwM2Hrm!{oMw5TYsQZG$4moADV~ArKBN>X*)(VZKrxm8ycdnP08+k$ovU z%{w*|#qZFcvM7#@Z#veL{Bc8G{rSh0?Wy~%+qLPfK|PLo`5I5}2V%+zg=B<&_{zoG z+xxbS*Y0R~mu@dgewfFq#iV*u=qyTtrb;6+#jV5h5NQkH|5|=uqI+Yzj2>NY2bN+| zI`nor>!afKKV?4&bXr~3xZl;F-)GgTO=}M778E9qdU~I6vmfOp!&O69Tv^`QyJd6r zwuU!pcB145xvW~3WbX(X6cL|PsTNk|tWnHEjvORy1jLMMz-bKKceKX81rj6k=C3;s z&G^iV$q6NS%SRurI6yTzd2uPUsH}YAjI2)G=RN(j#_Yx2Le_!BUR?gEQ~5Yu2LkK$ zs$H5td%U1>SNXN_(p!Hm?71sf4;Z9z*(qK!)%f52$1TXr8%s-|6fkEriA>VG?j}$9 zvQtpJWbNProyDFlZL$@B1;;-3xZU%Bhi>e68_H36S>?2j0Ak@B;)!{tLlRM%2%FBw z`auBC8Ivgpn2$os>qKBYV3LUJnZef>v$3-91?j*3H=fA{k-H^kBBfc07Lyf?`#!dk z+0dv*UEEZC>R<EX`klk-$W$1_EMb4r?B8A{c+{Y)k>@OSr8JmDa98lcwx9A-gh3Sj zPVeG{tq5mo-YMS6?BXV>ie#Ap47xQ7xHPSQ<o+ny1PJHkB*FdTb421o-KKb*@K2v; z5vo{GpUh0+vUivTieT*@agJ^sQ0|CxRG}Ah+tnvP$5eX2y)5lAAI#`VwJL6E-iZ4& zUzmL}xwsddoV(YV$7;pWt1YV~Y5Az~rM>A2fbzEiy~0qEPxGWkKaZ_zYE#=I?FR%$ z`X}qka2xh9=8he`O2Zg!>S6}k_RZB{TkkUOvE@H&OK|}lr?Mf8h(Ik~SvfcNDxH>Z zFz|tqX~j*_Y~(%l-@5#^wC$?DrIPl(DCsw6sl2~mtKY|&#{^g9*rTM=E-w3x3XBeL z&D$R6Yov?=pRNn;BM+?e`1rwNT?Rnl`2+5kl8tc#i*K597G11%OOC*4UDHDqD;=6k zHr5L*?Jp-&qRZ%eR;uAfBX9-Argcvy;pJx@^m>V@b@JeJlB#%ROq4E)sCM3S+)ZZh z(Vsvs<IXAeb(OI1$Qwc9DH|{$+r9VzDKA;L2%^ScByCkcQ-59+-U&x>(E-}a6UbJ? zi)t=*-PZ9{NTKsE!OCsNmDboQGZLu0htOgNbTfdX+Q}&4&m=}8vBXe=XnI<hva)Xx zwiMc2qR3?V=CYu0Z5&LIe)dcZY7P@(5X)^4ZA<0Q#|fx$b58NLB2iyaXgSddEs-1B z<FlVbRnG$f(VzW~PwGA1hW2^XHSvVe3|VW)?bIKN!%i+`9Ce#%ts`~0<=T?qprE;} zl5o`wDwEhDq4Yyv-@7x6)lu4TgJF6EZKh`$=Pj0unF+Zq<%#q32LJ+V3JW-it6B%d z!|0Bu7M591lw`~wO{$!+@nBR(ytU9AXwjryI;&*()X-K=&2|D~s#iL9nG@@cd~4Ey zE;`ez9*s}k+L3dBcN`#a08|pn(jN?fwk3Mbf!T+_D(o;R*bM3(K?f!?uaXl`P*GoL zF5WF#=^^2b(e}M1(NuKysQ6|x>ucAv-Yc~5wEt#<(A_qRo#V9!r3<UoHf>PQ(T_+p zvDb$fg~Kxb)%*&vb!|;U&7}tCp>S;~S<9`fi_$p`0m5Iqo$}%pN)cPc^YgkcIkeX% z^WiLVfJnG$--9^Gg`n?Y!p+vm-x-%%zfK;QZnOS8jze;IOttTF`ARb4c4HV6{^UM* z%?bRR?$#0HN*;nEb>pN5w>oZFlNOzreHv`^dcxDLwCP@1JD#@Wv3j)Xvlr8etTDh~ zH+qA1FPfNN=bV$U$_{&w&l^1_REHp7O4+<!7y|gnhgHi0DNB2H`&*@i6tLBud8}a9 zEif(9Kb@MvUV@tc$6Lv50}oTikRHN4`2$ZQUqr2~opH!<8jfvJA21Hh8p6{ll?rL1 zVi8K6v3=*x?f5+1cldAOVSxb|Ho9A$_JO-zM1|fQgj#xFudAHk%(gB>=1b4=r+>{F zJz}v137f{^?qY}l<ehA--m-V?mx4{=7@E@tH)I&rhNYtjg@6@+?IU+lT_Ldc*vpuh z6!T{>eL_mwIf;h)#KP2$@ky@pJwsMfjkzVxOw~<QqFqBM^(!M!_c2kfWNOMXzPWx9 zCdWc2*{N{8*<<ig-W)W{8!;oWmYQ)L%D7CA$;_^<Oq=4Kk`@S?h8Ye=T68LE7?vJM zwqWQM+flWmjW5swNJ?dY)@-@xhtePDv5u9Slz3Y(ltiP}j4{H%Q&dZdVP@1jq73-i zGhi1?n6Q21jeFRqN)aYCvw*S0+3HG!Ubp=9v`4*<hM|V*CiV87c&+ZTC2odsBO~cY zv!OSZr2lXzFF5&xo!82?N*hOc(Hp+@IMF|rr2nwUD?UjGw56{2j}YaBi&x^iRNQR} zL28)yFJg!XriimyMovfnbr)m)1oyR$T&}^3h|fCfX(hUr;bv7Va2Mu+N$ibqApe$L zdA9479|-qCz+-sAGK$db8~4f<4OU{{r#4Ojzhn#6*^SegRDk5>oop1wSB86Z#E4XT z@RsOP5gsq4QI%Q#rAz&e71cMl<RcQUyKoj!V@Ge1-tcGpQJ{cFKV>|C^R(y%bQy;I z=SraX>8v=nGuK(Qwce=wMqWCe%!=cD?vBcuIAC&p;8EwnXh!KY)$5|VY9g~bYoanc zYopFCEbk`%)_U7iNk+F+dH6k@OPRtu!f<i|B;3)mfH-Dfo0C*YA68q1idzA)#kyej z!saec4=~PExT)hfNdQ@%mkq~6H4l|H_HQi;93k^<b~&k!99%E8OH!GgvEm{99*qAs z|A-8MV(A{qT(v7FN#|uRv7KS|%0_w{&9@NywR`Q0^YBo7^R;)4!!%`TEv2Az!rP!; z!d%}|OV=wWyh`T#Qk#NP^o{@>W|{B~$mW6rG`^P9mMg|(`OwEA(}UJ(8eEa{%8cMe z%`O7PK5(|??Uy0VT<pH}LiOOd!+mzzhE5>|B4)+wy5mxdFml#Mz~8&TD!I`8A0Vy9 z_LYq<N|ogE;x-$YO`87kZvX4bev7MpeHc9P?T#I96XCGnnCEQIxchML{Jmx+?cV<o zZhqa^SJhbVRpGpB5kVoj%Z(w^tHS;LIO}OdD!F<RGs)RNxP?VoZ^{Kq(oXS1IA){- zw~{Oa<5C?G37p1{gKgG#@T^4}4#Dv#$!Yy}OpFEJW$xoQ>v+(tyYkaA?dME-0IVQF zq6on(<R=<jZa`1oq-+cC_x~7gJZX`=<5^Y6*UF=oI)^xx-f--Ib)$${<r6sbxh6j5 zb2Lz6SfN?_6PCr_&+iN65s!Uz@~m)=Ewl-Vj(I~ICh8x9dZn?-3JzLeH=!x2JZYaD zsiHPdXJ>SOc)SW|R7tuYcQIk^a?H%$GdpFj7aqHr<uG*q(Chpj{aJHucFS2s_Kn`d z8ha_OOCFMv_E{Tf0raCCCXL&X$F_y{8(5z1*5R=}R4J=4|7uF2`PkR5k0dt=pudx% zfZMlk62VntYnQQi@zhy6IH0yZ%<F*hD`(L*cP1nq(LUaSb<69v?)p9(!V2A1TI<1x zO0)%|ks7qPXM)SW<voi2gFa7-0p)4wwp~D+$pQc6QCZgahWV%W*iYK#+Va2B#=Q(d zNv=*MMkt-&ngm(?Im|hu-b&R&2sF-7KuZ0%wmG2Tb*?jTJq3a=L8FQ>3b^DfUK#a1 z1%xQI+DKBV)IxZTwM^89h-xhu@a^wm+Hf<i|NA#}Vgv9_Ry2GE5p{>4=b(#WY-J3M zntBML_NYog>eV&+tKxaMLl*~)Q9x2sae`0zr?5OP9ponQ9Z5$f0xfVrUsEr;ZEmLZ zzu3Y9W2TT=H9Pe@c?1a<<I5N?>8hSkmdIs)AmE+0`hl$i@S+5i(+8GNE>~;xS&2k6 z&H+5_A3=)xrPCLtkWR;}m6~bAM3wdqP9%TAHz4izE`}h|E6c!V97&vKp~gD3BR}D| zq)>H7mlts>H9RPj8PD3TEl9gcM4ub4xZqVWCTHxs&b}jAxdIp?eZ+&1i3cr|bE6eJ zNt(*JjbP4uHo}2$*i)qYnsq_zoNa9ui${ZSJP_@f-1>9)PibQ?0?M|6b-x(+1)Y?f zW*)*dZzB(^lAMws+SM-aZ(W6Kt~@AzN$b^?E6^ZY6htkSvC|S{q45O2aUJTNyWuGr z%RE(3ad~f1UNkvN9Gem&2`a(A@g-jV=Jt;wRv&hR94als=IV3Vc`+hRq#?sJ#t86S zRV2}$%8OgA%)m{3f!~o&zJGE8J(=}OEs+NbiN829N#(8n-Yby^$|$iNS!8W!ucpP2 zh@1sXVW7MuRhd+mt_t>)L-!~K4+Os2<%%7S9VZ}2C<J&82pP*2wF>qF1Ij&~sytX# zm#$Hiq{;({!UaqYDMn3;hhD2bhQhpsaK+vjh3_!~%tE-2YOpH34hR`f@__ApPq7XR z6fA=70*d{S?l8&Uu&>Iw0?@tlh%6j+?umfI=!E>h!V0uVbN&)Fz23yK*~(I-)#@mv zhx7G~E2PjyyG+L)KSpRHeo7bg^1U$+^^}&D0vrpJw4o4iDNiEJElS7|{c#Wtn*zy$ zH^+50mDecSgrdLqtL*>omLX6;f$9i88pDAxlnMZ(CKMSbj&n1u*@uQ$EbBR0gBN_i za~iADLC8Zzc5udg%(^8Mn6m^kxHlhvlwT@%L+j=^&k8)FB8(p!Cn86|wejcDAqU;U zqr?!T=T`OWv#H>7z$QF4L@jNekHMRviw=Qwu5_My=y5gvw<2x#jIX>(>)h;pU;HRu z4!v#dCsv@do11eI-U8dSM)y7v4}B_g)>g?C(}x2VBCw{Q%=c~lx3{eZ@BI9z)fV)r zId5^Oxu?3(`Fp{XZ>*3Z3_K2^e_eM6zd&IQ@FQW2#Ob+N*I9jO!J?GJd?V6w@6ufM z2J(rQNelv%U*DODS1a4gBJGim|J+X8o`Nu!e3$2^Ij1=2*1ZZY#d&6sq__z0ZtVVZ z%b@`1Vwk_qejRWsHAN!<@&$7W%XUuQIX=*1$>iv>QAgDw>wv?W#}9!x{`}C2k$JN= zCaTH|y)81ceo_0D%K(8}^kLz-mYD0%z9}`;ALHZM>0euyk$Uf6X&&!%s^#-yDBrCf z8c(E+J?KL(`pMv&4DAlE8BjDo3=cWxRLd*^?lAzOuhp#56oxs`%_8+?z2M1E?yRO= zQ@i!sAJm+GC?7C(H2ZVUN(XadwV7^Fw|nXA{04o^3?sonr2X>u?#Yj!@t+x(RoTJ& z6TPNhzMN7k7=bS~_a_Pxq?eExi;EG+OK7L}E$!b%_;Z0ZlUV+=-j-PWd00{RGl<px z2h!;cfViFQS4;hh)M6rys3yz9qS1TSTkUNVmT=qJ8d{>h;?}k=%CeTjT3gH8S}klO z-cE{TlvhYs2G32%Ul`E}R@0~Cc;<7H<bbhQ4JqW-93Po)!9ymgHrZ^LYjjC)iJ@Hp zE<itb8dJHRhG>^_E#ihG;W_N+Zn02X1Gb;|^{|d`gISN$vPb6iA3F7=ul4nrMeB6Y z*XQm7VkWpe4VXpfU+eMFaM3VIbb24aSPZAFLbS5=tS(aa?fUf!E=9uP#EzhpbuBPY zQ$oYO7;OpS+ttUSoS^aIlk6G?U3Qcf-(;O&w|~pSomd(FQ2*eZ;`*Cg4Ht~+R_;U7 zG*1wbjFGjFzxOaEdd<yspujZ7CeaYoMWM=>Cv@3C?)J?>!L=pYD~CkOjz=7SenIVc z)*kS@Lr_avssNX67ObD=zEWqrym-PZ&h#5;d>goL@yeXy@sc>Kw{M&maZ0mb1Dq7= z{6`er;eHH;iOH33AW#bDI1sRT4|Q>Z>!P*U!U)Xz*6@&^wfdQ-jg6m~)r>vHwx1K5 zRNTV1ZZdGK61l%&K^-sQMq3SCD{x-6wMMlUo5U!}^Zmj<$*ePHX94rG_1O*t>`^JS z0mH<^inR_zOl>sxm`6LmKR7YhThXi3RM<WsJ+V<Z8sihHDaw-GHNmK3Yb$4DQoi;8 zhKL=*5<z^1cI52+Vd}1HLSuUW?d4mQ>B&PllwK#Z)ue{h&rb({Q!uxKDj+GFHFA&Z ze4l{Gq>7VX%s=>geYaciqQHSuR|i%1y&m=(u>|Z?eHwv{KTOxa_W2G~&0f2}jLm%* zObOC9Xt+4r4eny%jmM5f+OPs{yf1`J0nyn(g$@MlHp=4b`?ixdO=}c9>CAOGjc+w6 zKXIuEBgQZ>Id!8!F3N3K0v4%h$g1*YXU0)~8k4uWS8wtDXRScS>lk&cJHrXdZx<s7 zmZi+~CXa74juB6hI&W^%^i5u=ouWHFp67CcFt#oyntd0%DOmO73?iaAnYvHs46ure zeyJ`uKr_6w^6{F#Uw%m{`j3@h`h^bb)}z6(5^=2*!OC6?Q7BY~mCpm%1mB=B_h4WK z!t-}BjlSN-b*-e}uho+Uuh#|GqX^5|#V`gEj7IOZD0*W&xyWrysc2*J^(}A{6oxvI zd~E~i-Y&<YxmIRqO5*BG_<ouo?OV{bw0->aa*E0_iv+lS{OF)}dP)V5I@OJP>2nDX zo-+~l_juI0*DOc3Ae~K1WW1WNb{8dL?XhpZgMSCsd;;M7t=eohrFscoVM9kddRA<> z4j_DA^}`RQ{cYf{w?(O1QEZ&<S{&={K4-V-5OY&+n5@5S2b%+j^H7iEt{XW2L+CU< zKoy#g@_=PAVieDUAKV?B;0YiO{>*yN*Z1H?2wk-`wgXYdgN!d(4dHe{W=Gps5=uM& zs6F0!cNRdrQoq~f{&Bh)TmuqoOE7yfbaw4920bEo4KRPiPTm)k1NFRe4X;G*ZrTQe zN?<d@H~A$mEyF6EQl9RBE>$c1TWqgUorX6^!WMtQ*YhxV8~87K$A$rMu#mwxJ~l?O zz78iaDhNkh@=@Di*Caawo@j|?6aYm+*ZilMLlU}{gtskV88Cs}0V(j0gL#x&Xv&e1 z_7lIvR_c`sNHU&qLy8%+cu}=b!lm%&IhqnaCVFS#fUS=zl`Ct>yo4vk6u-(>U!;CX z`L&M0P-kEF5JOLUV)5e6%$A9xs$tc)^R`aO$RP00^a`i@enBS=l`jHG+2!qwpKr36 z_39rYrwrQMtQsmXcLJxux%04r>yAqrqfbnDi~EUbF~ChKf6IV++?TO?nIM~O&1Fiu zAuLZP_NZDiPKs>~!Vd=GI;gac+@dN+$6(;}cwKYSwj*XlT$m930rI*Pqr^r@f}Kcr z^X**{tEvE!Nela;kw3UMBNfPkRf#U~HFq`1uFg_FH~ZEXkPoipFdUIOy)&u5ZW94; zCOIbOR&{W&9kirDMstu9n~WP(V>?NGyCGbU7_L=z!W*>ZeW-*1VuHU9nR+_S&CWS_ z9^4@yQrXnl*Ur9^?vvj9smcmYKq-kZ-<L#WJ19Z0SaP&mLirL-{$U*-E0*JX$IU-j zfrk_R-!G@CO4_~EoOagTw!=2|Y|^A>jI@VOCAy`-Pzor;FIKC~AnIxkg#JEFRE_du zH#B0&q+aZPUhF6-dB+q<y^6=!(1c|-VJCQuRU~ydm67Hzp`7BJ^iJ~5s3#U~y5P=e z=(LfZH8I=h89TmSZZwNH>%QNXQ_XSDMmyplN_Y;5q}<BzUGQDz=KfhA?j6=mM=e)P zsK(V_$q7pj<EF>yR-|V~XBWrh<I_HbmEBh+`ob9*X}u;5`wYpM+oR69HOp9-ZoCuL z7e+p3Y+=#jr?QVQ!{!mfcqx06<s0HS+}wRWJABT>ISFaFAU8k6$!ku*yc^EJSGK*T z=KmJrv-}|W)j{&|Q29k__J?rgrdiT*(u&d(@*R>&7U2?b7&pUyR-wDvz_&Qyw99Xw zKbNE0@4L&_{_7xztJ>$S{4*m;MhQDpY&H;4L4auz-G8eDr11qq-w*6&e^fA8@^>Br z!b$u0v@3qp9<*DRuxmmcu?6CjG|@3k`KVi=D)YuWFKW~JOaVbnFj(b%KK&4}xuml7 zF64CBx^)%E!*m~Njk3gPT8+5sHpJ|qDdP~aq;(PO9%T5M_-^B_`~<+cm8-v=e?OG8 z*~-cl?h1o^ZZvONyYo0m+b^TgXw@OB-2?`GgGoNA*A^e%{NH5$<O$?|U*i=BS@tud zazv1KV>Z)T`L)kW<ptQR`1i&^9=|*IpH#{Codp3PM_3t4Vl?h>06IxI=<98b%6lU} zd;iB+CHAF5u!l=cJK>D$!T?2$D0_BP5;hA=VVhZf#%kkFlZ?@=RQAxazhDq`AhEds zgq7{P%O6U_+S`NmGG>G^_TNOB>Eo_1pG_M4=u(X_vqNHs79c<)55!(1c}OC*V*}wO z8{dE%<iL>PE)z|3zSu&W$!s?u>Xg-9gr~?|U0uB@mjb^C5Ev3=!e?GFI*zjmb|Q4D zyu~u@<vmS6W?kn^y3C(MU!3uGV06)#{O<yW!T+Xhovf;r$aNk{ccmgiXq%RWv`;&e z8yo!Ay#y|Vg?EWFEW>3=`&LVB1jIu!OhXiT)16P)2N6vDfmM}z$}e0Zi01L{OR))P zfu4}63BO`^8d`|I>r7G-zM8sey-&v|J?^%A((R=D$5wrax+(Cr*S?+LTU!C?AKFm% zThH_E@opW=^W-w@Hdz;)ORAL#zf~Aa6PkSkl2;ipB!Ak2QaYfg45d#1{WD2wx+u<) zA5zwZN{xUE@R2E}ozxcj?YE|}u?71ENSjIfgV}DJQ@1F~XP8Usa0{iV?=qWQpO2;v zZ%*CsfgO2a=)0Qsufd);lqckn+HkfGu_YUS*8xkbMMbG+PZ-5pIx5W9xDWu(4{*Ae z;<JYYSKc4q`urKk$VSWLbojL)16Sp5<M!I8f+DiZvLZv)fNzM!Z@%S1txD!(hL<}h zOU;ttUtjk6o@k+V>MPsxlNSsOfn>me1GePI-i?ZjASVHTm#mzJl7?24ui?0DtQoTo zs!1+h#mj{W!Mq+g-|#}8<F9jnt!9ZwFxdV$<JG(xvMS21AiS9r>Zy>e5meHZgrj4= z8?!cubAI>-pzZ=nX>G6<7U{7Tq<C3=t?E|ODTztCx5k?lufQt^)Zrdz7yR>q%Fdj{ zJ6-jjMV`da96|v>(2xaDnTc#7lvUN*e}?e2EZ#%xDgF@TCuW;Nd)!MzhF#ilBPbjN zUh&S~9u>OfdG`);J-nG1Jyp5fYHt>9{t)nNR%I0Sb;<Z?N-HGK+GNF5-6Og?^|l31 z>+PHh2|qcnGMo#QJl8w2aXxPeRIhTR9(X3!3R|_iCoR%=rf{e*YNuQ9J2MWPNq6ar z4!pI1Hcme~o3T7?Cn}71MA!X4BthWHg7F$S4~b?XA~449yUJQg`8$lGAYb32RT5)I zYp5d03<e{ZD+^IY3HqRL51UwI3w*?_%%{d7cr&ABGWPjf(DX)whu6tV;1R)ZVE~=x z46RFkw%fN-sWu7RS;J#${L?&ymrPyg7lS3C(h<^BQ_Hqe=TD8ESHreqW4pqm$7Rq^ zgnf=v3<4ItuDlRj7WLc;9^T>mRD>Vh_R)3Wq#$U)jJeROYo@y{cnAjje|rbW=m_5v zdRhre4peW9JI6TY%}C1-uZa$T%TOO)MRQaN5+_TXK*8h&?#~4G3<`vF_JKn4B}QuG zWJA+`gV)!p1{Mu(u^pqXhCo<WE3Oyg>acn)1(OF<trP_FA|WEC#v6^)!$QNJvfu)` z%trHzE`Ez&ha8z<W2j*Z8#|%c01XiIJ0&9tPd1QDoA*6J4IX4zn3>^k+Q143^xvVp zbL#KqOr9Ywh(R))QuiPaAe%G_qZz4~f;t^%wO@@YTXY1Mi1bq`U5>vt73?g58&5gA zGXtii<F?;5{<qkcBVM7NY=ax=fo;4O1e06JWuk@9pT4LCSEvdHJ?J(#b;nS=8{MSj zUzg_%*~d-p)Nq0#2TP89aTJ~0W*KgdR(%v(;9H;}aj0dBuA?eoXCP^2&GMjp7F6sJ z+b>)TcZ5eX>j{;)dPC|}Y;umdv*NnW%@a{b<Ro$aeR5!yllpIG9;(%vXUR`9%NafC zs>J%bE9HM1yc^v49`?q&f!})o1m8}dVgcOqEpVx4TXOF@ru2`4y|3%+mhgT=W*RK8 z6(O@ep%JM|2AZRqIayLNy6|@Ka`{9v@5Cqi3d8uB4@<UZ)q+20Kg)7o`qcPKCJbP7 z&7tw1o{6Z=?Rhta)Fv*UKK*p{0rYj}G??F;%@jws3c2pxI_O>&O^R@KgztCSwA@*G zejM6|)v@<Y*{bCs!BSV7WW_tPBlO=1UW0lZ*UE^4wD83SQ5C7%40K$hS9t=vTog}A z7cn%x$2v9AT(bTQ@Rw6qRE4+QAl;J^9GdxO=XSRqRymJVYgjC8wrNeWT<Yky{tw3P zC~l6PK1H2L51}AoySML2=V^ss6d#klEQ6TZ;>YSADEAE&J1%pcDX={?o<W{Tp6Mvu z%Xe6zXQzEMS(B8pkwkFBG?%P<8tQ6sC{YT`w^~ZFRgGF^u}iV_D#dsJLw;}~(h@SS zta5`R>m(r#j7lDc9prji1zFK94xnCq5@^u<eW0DldXB&wvRGl`julC#i>O7aSZC05 zUNoyxd;YU#6dH<5$q{+ee{cxV;hLJs1^_YMsC=+b2Myj7GTY!a-XaVP@^r~n<B~z? z-k0)~K=%I;b^9Aq<e~jg8U#+ZSjI7;T22Om{{Jti+^D~JruO}+l}S;r!<PpJmcw5I z1de$&kVQ-Qs-dMIFTA>;5w-WnAY*kzmT$khfH&2ouL;on2i6_id@}sdR_6Re<qOVc zD@csJ3bi_5zIUJb0crBYdE5VfcWb!ilh$-V%QR0#T`c=NyZZmh&89yObLJAEVi++k ztAq{mO23ld!$F+1r>Kn5@%}+F;L<HUhRJm8?75jk!o#)9jo^Pe@W2lW>77DhvpWU# zR~PA$Lq(#_o)&Wd<$LE~$tH=!EFUNI+jRfk>=llRTR6cNap8$|?)VBVD91|dUAvex z4XE1lnX>E3xizcj@L_rUw+d)z`dP94nYb?R{>wC-2Wlp;wi=T(-|~XCVfGxN_6vh? z%O@zB3xze{mlYEogz~r)a~g_R!$<ULS3;TWhTeXyl7-1*Z3x@87>qCdnJxh~9m-+< zUmHO+y#4ztJ!HJx;|xB;xnC|B?y6|d&&cRFbVA{Cxacs%4@gSJABt?8;h}6>RY)}U zb}k9K%06AjC<<$gIWC|eRg^(GEI}<5tiQ&0=7o96u#nP;%kfs=YF1SYoL;_|fqk%i zcYjn!!PA&59|J*g$S^xB^IAkIuG}MgpS-PX%t$xj)nXn}Snn`HfyZRcbwbgi^)=FD zs6EYAuv}C<n^7|W6N@)*3+-_eLClar`;6;P{QL%N?%cnkpXVX_t$SZ}L<~n?ldaBj zYxb|-m{gQrn2@4<SI}=M7%n&(n3$}kd}mh=P?bYv?n<A%_MCX`HcqnB{{IKxA1K3q zb_e`lnLDD+o(`)1;8Q;ci<yg8BN$Em%fH{(UW(g)Q|ZVSq0enc{|y_?xJrMY?NK!_ z2L>SJnQ6K_r6wz`$U7Gvh4EHB^h>UCRfN0>oF8QmleUAP=ENiR0;ep?5Ol1bMx<)P ztE$4zlNy*+vINO|PA7Ftq~gOIq0xAyhbD?C3aK`Ca&m7+=AbkI7Y(t#-b~w4x4H>u zZj^{xVV|S9z?36&D-|;2K51ql2!9gKrM(;xDaXF~J}@LE+sg!Tq`<t`?Qu$3T#~2S zNdneWD5cA!H@IFLD-W$F)xg#ngT!}~*_!HlWeft1A^2sLhAqqC&)u1On1x}w1&((d z1J8uS{1_ml?Y&DJBCASd4Z-FeOnD+kiOto!593gb032?JDG!Ixw)5Qe40o3;Syj0I zHca|BBDf*Zq5m1)e9lG{{AxX8^ytW-?7)ER;DJS*?e@sdQu@%tdLj8dt8P^nJfu-$ zs^*O6@Y?YA_irol{4HQ8e94U<hsf@sPb~0?bQ}v7V^Y$G6Q9$DuAT*oFKXm&?La)2 zQJ`no8k-oa`S&#R{5GGVpI1FvjoOluIhn4VG-FGsw$>(lp4;Ai?l>b_^H}p9?N?P7 zRV(TIQAf_v`BC%S#^2;KEadAi;3bMhZ=9n7j^D%HhYl3gyyy<+^p#}IH+p>p4I>>- zw{&}XL?ScctP8us^h=)3WUiI)AbUe~H~o+&(hV9zDQ<)?dmhg;tZSyNkSKf!btpCc zm31j1>wLBpRv`YAS8^1dobY9?6!C7|e{PfB>sVKWPadRukA#v!b(vRHhXx<1k}NVz zA&n@DOMSSa<cjt^@AIRE39?)NO;9P~b29PUY@#+L;23HRBV0;<XE9-3963?!8Xq1P zE}8w<M$Rt^_6m7?c%<a`CXPOO)O;l+PN|4{kOn!lwPV+bQzuF0Bj)<R+_NT;QAWPy zjBd8zxs{R<8KDdz8cN25WdoUIk#8J*EZxvp)=e;D?0*7tIbG?P!({_7ICc1VtmNat z-oE`p<feka;y`GAR3ybE3P0}8Ehg-_J*C~mMRS`1rf$!u+w5j1B!c$PgkU-4&v&Pz z-{t#u$da7jmebPWBy&!Au{V0uujw-`$%#R`K85mQzMt~3EOwdvLFnSeKX5HS+Vvy; zN6vQ3r>1CaEZr1Qc9y0`qCHF0z6pl^ZoF$ia4Lg4a`fI&`~0(aoLagn+LQRlq|N5^ zAo?@Ty_40YcT(~JErnoFdR*_*r;T>$0D)ulk34{L2mpz=&?+f^;>O=4ZRfvdPTZ#M zx~)lhvVJ4y<ttTS?zg#qf>n>s?eeeZjjL=Y<9{s&aT4?=5{ZP?qoUOTkK1S_$(jNz z*h0Td6Ql>gJg;ZuO-W6E2>{ur0Ok9R5*P^K&cZ-$X5avZT%h=U!L(!^9B-Jyhlz~s zj9V8rTdqPRthzZZx1Lg6)q<1a1_o5keeHD;K_r_i!DZ5-6g0+b0Q$R*b|>%Z>HMFT zUP}nh?9$2{7&Z-IJ2+%5cq_Hl;YtTzhIJKRG7Qe5N3Q_~%5no`Jsq7tz})-WD7O9m z1A&SYcZZZ4FE5lR#{yqqy*2uG&M%%XD>_(xw_5yI*1|4wb;yuWmVlRmS0?QP++|gB zKYxLG@PAH&(tK)a1R7t+O?NXfhvdf*9}gpO7D`)n|5rxvc=^<huzd&z+If9Vg>t<v zDRSt(F)N81wHGM~^QBR4wIX~(AHr1^DPA~pHr)R04E<xP#3TEX52!)xJAIcg3*QQ( z!U@kCxza!H8`evI<nVybxWv2cN1vbiO=sO{y`T>{UL!E`&pX(Tml8^17>keUn3>qx z_9L=9pXlpN>w0}2baie1xNG~4aEF#*Qx>e4uAb8tATslC7%o9xQ!$=jE_X*CVQ(cj zt}IhkSE-cMl?pfKZDh11MfN=`+faqx>Zx1Ou+!y=nyU5fY>MsY@k@|BGrB%#I&fMy zf7hQMyJvp?-Xrgd)H@t_M6Yz)-%q=y{(RZqbke$g)YT?gIsND76uQQ)aAI{;TV0Te z@t9P)qS(&4Bf{aTRn|ste}4HEdCt|Ps-<e!%wS>evg+l9%YLdZI~68eRYJi;uE+=( zy^}oQq7v`}YQUPoHF>1bgKy<2UAm3$u`IoWwkzme$12f8jI200yT!cXn)Vf@plwr% z-BhJX%=S6ry14`6?As!${;kAcOG{^H#qcJ>TwY;4qze*QhNm77#{DRX9CcvsvmK>v zXHOd}i_?jQ0%(1K`;y*ys0JjN1KW}kq$CXAMaKJE)9GT8$L0*PTpikq$arjiTgC9c z0MXNII<gMwh`jbJwGy-4&1VCjTxx+E5v^xg6Y2?$h$jcM9OWivDdCGpOnjfltt#P% z98ufcaB`)igMRKBJI-41w32_rNnNbn4Iw=FVNicUw@L2v4q~`NRd~>k91iyVMQ8uU zLx2A$raTpYXSZbU+t<*ba!q?oSJJLW2WS#E{5i8%_eRN_EOSx@h0EWSdPq0Yde526 zMsj0FOZ@-%8sBdjQ?B9TMqw}+!xpW2vVoOo$3v<d&eu9C1}~wVuhr76{aY<pn7|-# z0+a;6&@&Wv=<Qr}amPnTE7Y4rSSj7$o&zV=2IIFTP3v_T#+TE>n|?*Dyxxe6SAQ39 zr}o=50!rC%N7bOy()6@2%<7C^)zpoujsV|rSO3JAl$Z*CT{W0^43YrJ_Mn~?;Q2Aj zd3Dkz=BEy?I7rBkCljCkJEYP;yF5|ucJ(;9gp94ebyloA9_F{nrbSsP7Au+WbZ)t^ ze9qsp)l0SXl?>D$-RZT}Gb)M87O3hX+x)fy_TH-_BOCf2@VMIzlF*J$*=Zt8L!(BR zTETTx2nyZ7gQhq1?GWmDTs`;EhQ85}V+55CSXm@0=3d%KPU~pyaU2D~hiJ(>hp_C2 zqSERdTekq`t%i}cCBccsRay4VLGDNNIGk-8UXIXnAFZ-=7uLeIlanMi33<RbSyoW9 z-@eqq&;tVYbVl<iYJZVBsIS4=xaSolY8o2B!tVf2K(N1{g_hBKcO`IW5$r;?Op)Cl z@mxJXk|Ho2xQpR5DODB_stkAW1ScS>PpWqwGzZGc^&=nRnea|NaiXT#nC$KguRg@; zFjIWnUqNM&XRbUl%s3GJK&>n3u{D$lGy7*ta5~oM@T^4#>P+7MLU#X4uda)UYWq6k zz3wU|dWDqT;HmmB;tp0I3qB5^%}2CY9sWZ~qv}cWPqOz#a<T)m@Z%5@SKsZ%0_|dz zB%fC+!d_~U$&#%5`}XzEmiC?yh=n%b9Z1tyBQZD&jmwNRn2{CM0~D9ki=y<F*<hm9 zsg8F?u2mPx11#{jpZ+j236G$w?5p(J=ghS+3&GG)nI^~{qMT<tA&<~qcA8k`>wYkt zVfMKTxtqb&36J<(y-k6*{Go<MG(#q6;+Y_gR@BqJ)0cdXtghyOnRhwrqkX}gIgKy1 z?MQ|cS-iBRnlb9;mCKs7R|SPA)O9jxXBKbNeL1LpAyzjV);pa#NT~_)c*mF56MhKe zvcIQ_4|WAt+}N`FYN5*w`*)8d8REoi%Bp3WzKAheUY%DYQ`alRywRF0sV%EBt=0!2 z>|<^2nP?XLx;d4Oo1rBJAW<qiC4pa-5AFb{IEY48D6`f0hMRNv-?wJWI22^{o1GV* z5S-YwXKgh=KsQ#_iM7cgDUsg7USO=&BW;Wm+=`{UzK2J#Yl0!Y2IB^K!*%GWrWv#3 z)F>;<asJUd-omZXMBRxsFM!Kpjk9+d?VjuK@`?G>$YLuQ?P3oWpZMX9ftu~R*EY_5 z>qxKAn}=;AoSJlH)-f#}#G4B4{I$Hh2uEFMx!joWsF~ooB)hs%I&K<pdF%djhR3Oi z=|b7yaX9&=e@L%#EjyttWB*RSD({+&r4n+pN?jFW#Paj#{YTatS^3WGbz^x|Yuj}U z)XEsYL{Xp`DUF{xG+vyZtxG(7=U7FR-kL5HNO+@i<hq`zYRUM{n#Q5=h6VfMEt_5a z$z@eGrT?RNtYT}f7Zc<jpkHinpBU5Pu&;G?rnqlR4wQO9E7T-j(v@6y>H;M`>RX{u zppQp9s+yUpG8&cB;`Wa`y;aBL<&N%mu$7#ct}8v<r&RMYVt3u3cX+nFpYjUGVR@M& zN9E<gsONOXb1TPls^p5w%JC1DTk@+_(OEHz&2AnnrYtC@G@SQiB%xKzQ<>{IlaZZ5 z=Zq!ATK!0?TvF(_71yry!WnJoSz3fFUExbel3UtEw-Cd>$K)?;JKtu#>k<m+pCo`b zsKOyzVM2v+@9WlN2e(@F42;ZH&w6j!l86L>ZqP{YrS_#AOR!cJRfQ$C&JWVVDMyly zLYXAKMK@e#{8`quROGJhxW@|h21{q&-^sT-qBk4wAa}2+LTLUe`D=yE%`~!&m;dQp z^Rse1!g_VVt8}YV<!{&dKPVU@tq3BdDo2Ew47MG*2RutZz9$1bY3uX&oK!#hnVRof zZ4GS86%~}%vHpK+rpB;C((7ck2|&_Nwo2-Y8Xo%fl&p1Cxga+pLkKE~0H~(PY|dr8 z)ewO3LA6O3=8)jX5#`LTLRx%B)qW7$ndb-D;8C`|0<7<4V<}ijsNW+GyTFzDm1~B& zp6TW@q?1cl!yr9d65aIj<5<97#>d}~=Kb&KS0C0xZ>O05*hZ^(wj(LXfpj?Ltv2gj zo8?Ha&UZ5`5o>v?l+mGht-Qj4$}B;K*S85};;G9chJ`QG=>2rtb9JnpBl?`eIEl08 z=F8#vJ7>(744v9t$Nn5!hks;X6vl6}u0eqaY>4|9XCt>DZ~Z{tULNz&c1aGSL$$ev z65-Dm;A_w05pn{E{A-9!a0?dI)PUjhOP!6*ZEg-q_%@``%^}1Idxd&YNmfpta)EM1 z&RUkbaOAbpSEY9-TX`D!9r>%W4Jryw`9t|r#SViZe<6Rv*rQ|A?vR9|{=&j7ajm`3 z9#wZr`#owb!W-}fozU3pz0hm`9__JPUUN<E9OQN9i#ykw&ZoaRec`>*ob?Iu32|rp z;kgF3`_32QV@_zB`;`4u!hd$xDOa20WWvcA?On%R#~mt3*&W9n#uA)vzN8Pq<xTTC z#WJzdv4M4X<aUQX>kp@@8H+}ttZw5(A?hRnQ>%D5kf1xQip0-5#VERy0HuB#4XRgf zb-G*_%N++ublNIM#GVdz$~vmkTjRb=*K(NNEugEZdHhGvZ3=6HEjCLRzdeFE0oX)7 zxkqdEzTys>VMG}2Y&qaOYTX-Em=toaod7orjI7}FYP7j3?FLS4rMtiskCPWEIKdHW zkTR6eV&dsj%fKEjVTzk`^Y7?1WFRaVrU76Cf;a{N8y;#fUq(YJxDqy{6sL(Qzgr|< zTp)2LI~YSUY(&;c()klTBjOkFI^I@rEht}`=}2MBxg?|{J$Jt&7HtMYDna2fN{boQ zP`M?VbKqnur#jT(B?*1#y6e$2szFjX?!3eW28EfE_<A41xTnF#z&(LKMRhE3EfE*4 zwb-M9g2T+ix23N2EFqR~kK#_y{+T+y3);nyHQDUBR)9$~s+6LJKR&A}OEQDa6pQUj zI8Sr@(;mN%vh`3R<$;+K^I}fhC9Vy=76fqa26$u0%ubRfVF-DRdw)%fY{sewGae&M zjcrcMl_u3b80_@3G<Oos5ZKk3-l)SDhL#w7Gm+rrmfC$Qt!35!eTV4&rs0kF`d@>{ z5Z5feEJ4dm=;L*?TbY`i`5n))QA#!1CwiHc51K$u)Sb^-%!#K(M9x5?C{R{pY?G{9 zI8Ny%ES#_@NnN&NtLCIm^Zw7?Sr#}eyUL#GU%Li(pajnQ?EiJ*rHbr0*CYGnEAue| zWbHU}Hi41@^`6J98-3-YuMD5!(ezb$i}Ge;kinU_E6UXSAt{Z>rnBBLo3|CdTj#P) z>#+3d*L^d`u1QC%+jU)z+jxH7UWLk(m^2EVnVWHB>E@UNxLY1Rlq`Gft}!F=UNfri zNks3P>pkmn2PCm2@}SA3!t**oDuLcZX9^2a$-%@x43$EZhDiO6m_Xzq9#n4qn-$u3 zwrt|f%dPMg*kK41v0d)X^U18T!x8iYdNmW93$@Z1@d$f*-xkI3G13H5CV-D@o?KVa zpOpJ&g7BCCl0`|`k#s4C9-;_@IFM4PRB$Q-SxuYTi<?GfQocA=ms!GMN@;QyHAcjZ z2~IR+tY=6ir#z6^yrm6<ya;ZJYRqFQc7&Zg;itS6ATRH$<Kk8J+iGENE^J2yi{+Ab zL9=#Hzyh-J1lXX2TB~ZRiTmm~w@G9{;OBPg#)&+ij|*x~@9Q(QuWA|(i8aqlX!L0V zO>}&+2B-&RZr>_BEkOW6iu0HSQT6zh@E+HVE_|mVKdIxxk8`>1o!DGj-sSrnCDQ&I zXOi=DGG0uOBRfl;Fg`o7AH&WekdqSmQ&UOR$NU5#A+Oa3NQXY4Q`HpCe7r)w&$Y$1 z9#KxO2rMM47A#8d%Paw{pLz3Pjy^%6@B;TDR0rTw=z~q2&(;o0mcIVc?FS;mN$jhL zoGYn2JEhaS=%ril>EShyttwvSo-rYb-8%qn$t^8EcVb>;nW95!=uZ`U<tl=m1KCda zLofP`?=DS$<8vqY&I=#447T*pW-E4@Q}`$A%$SG_fdUFt>uXQ+NQ_LD#8ldFQlyV_ z8HXb>1RRuE-_{gBurj>nfll`}UR0XDDRo=S6+Sd5ZX@FnDtDj4vPxo}(%t{AB*>(d z)<zHdRCrA@1QRSGCJpr5w0XAAS8W@@0RqRsfzAx)s{U&sZ@(a_gbC#q`Ya0)%aYP% zzN(F^Hkn6^M_h7URWc>E=s3(*NbiN^unI%{*&L$8QE%m_qn0VNpTH{VTY6%{GUaZg zuKcylw5TpaOh234XZoLP(=yv!^^_y0E?1bU@>yW%9UfOlfx$j<IP{LD6Yga(8;ZKm z`tEA3$wKqZ<Nfmu|J7BMn7?s2rC@s>Y+qzNL&<0zYOH9myL{1h`)?iN&`dd|p}^n! z7iWqFt?}fCgs5W3CA=oLvS`R4-gv;)OrWhPdkYsRW^eYJf9z13NEw#vp2vP{7nYM9 z@z^+`AT4w1v@^RXA<f_2Lu(mgtA)d5a*+x_$3@ae)$`DbZO3u7*r3d=ka&U_w!kYn zCRUyt%ScRN!~)6=vi0ZSYD*&{Fk_Wvqs3ccVrb1k_Afcw^W^c|N_iYHo=r>qyE^1G zVw`VIzDvSXlD}vkciQLJQ687Z7k>%5uqox8f!!zyy=j=owihOFIgy-@n4H}nMx$i+ zNr1riQ}Ca9vDMU~rRM_Hb#a>)6=&YvwCPqv(OUE-VE<R1Iy&tt!v0?CA>CHS0RM1( zorRg7`C$_of#;R$EI$ml@aH&?&=3{}=9!!PONO3bm9Moo%xB_11kiGu5mzo%<y+R| z9K~bmKA7h@DO|oht-AiPa=|Vwl@AAAhz+1ZbW%X%yUmlvW}O5$@ptjj1pz!?NyzSk zz^xbsVa~v}!p5Hv7|fys__HX%fe}NSRe}TDFDa{BQq~TB``#adKYrV&3d6s*WLgaC z?W<bbl}E;qgWebsk%0ShXTlHJ@wZBFU;QF|`$=Wb^=GE6x~{#<k`^{C7X(M{JDbtV z;top-#y+V5ByQgv-q!8-49YA2vP{8SfRS(ABA`y6KJ<x=@wTsJ>(E(|W*UN<LoJ)V z0$fpgg!CnW>~m%89UW)1r-Q6OpSdONsqpjp2Ot(n^TqzQUf6`KywCiL*z>t6&C{%i zl^o^l9z^GW2ADjOt;6+-<squrFzHVr4fJ;*^_7OH-ky8=f598TyhQ6$n&a%#axS3y zey9KbLbiJ2<|~CCLK^CALFy{@D|x-I3;P|}C~E-2$P@VWck5s6Gi2N!-r^QkuDf~u z)cG_o+`7WwS*?JXXmgTwgnCNe1u*Yn;S_K-8#kYtSiRmhq@J(E37+YB#cl!4Zxy5e zn|*AjhHC-xgN|;VayJTjd1w&bM5SU&rSwzlPf|5-r8pn#r!G{iVOtEVky*QFOGraX zU1LDGFXf4I0WBS1&%C&zi$%7MVWdZErde{D0m_GkOFZ*M!@&q;j+<BTRs)byP_2dh zbgV}!B=3#TwD%yiit*)Au#;k-&CQ1%P~gW@MO$jpg^vy14)o}~5Ev@4g|$vqwx(K| z@_PZMRy@kp_m1?o46Phd@#f0@;H}tq{3??_o700gPU@8mbNf%Df)ojAlO%vJVB4|# z!Iv)_4v|irOC6tG&#RJM4QX0GI~qE7h=JMuA3TD_^oOVQ)A~2khQrc_l^?v{rnK3p zA8uCecWmc<M8(NLgeClWc8{0@h%1D5BrL|rSZo^)KHV>B{T(sGCl4f9rw~S+mk;$^ z{DUY6{rJd1(1Yq-c<;e!@mgz;u;U~(pzH-z+=z%j16r!JPW}TrHQZXizX1Y6<^?BO z>fEHteIFEep{Lq@NJZn`0j*X}C-YA_sZz!L7^r+oC9Dz@*r6B#%+y0JUf{XM+K%O5 z%i3qnkSH@DwvS;Aj9W0tm<|xay8t7gsAFAfq1ziNn1Nst8}HI`b4nqlDr&X`5))(f z2xedul)Z1uE9MQZ@9iBK85=uoc&NO%c>jSQwHz`$bH)`l)%uP=gGf}ueTlDLjo?s$ z$T}5ud;K1)P$#w5?b-M*wYsf7Jq>*bN=t96o0S<2VG8A`>R3+Zx-H=ZzDv3TI}~_K zKtLVAwuzKs9gFZR1mcOv5vZ!nbzL3Lx~ZL2ELrwDN$p|S%de~@7J19UTnUIAz$3Xb zBA{fs!4ZjJMc%bOP?dhKKW@dKc3pQ`#P7^m*Q^50?~bvs@PM~rDTwCYGo3SZGSKnk z?+^E_RQ~<jw@ps(_%=Q5oIgX|xzpZ|fd^#&eKGJEc_KOWXI9`94$8}#AzCp5v2l7P z-&eY^ca@DTW}{M^2!um?OAo?69S*_%s3+fJcK|^@U^XNy*nC67YuTOc<S^+iK3w-B zsLQ>`_rlfhpY%0L9PhA9Y0^}0ZSl-pTiU5kN?3J{ed?992iu_-l6d{b!&^W!t97dh zt7nGy_wxIp0OCNv9gF-c`XYb@lTt1dK~s=an=7sdI8z6JnXxl+3Q#O@-IZ2egk}Z0 z0NvAKnfBV9U1WS~unHP@bWsc3!=yc;6FTAu1aU(z(Z1hH`ZnY_K+X}&rnLV!+k=fM zuj4ibZPja!&x;?05_)@ycKx-r#X}Mc>+MGqt@<C<cH7LLWca*=4QxXZh{%WMV3%kX zmX&gVv81m204};yryfSzZSL5mJ~Hi*h`N<-{Fz*jc?y=i*)zjy*cy;DGh&fH?}F@e z%}#2UhUDpo`c#F85}D>D(qX?TwE6ZjpAfQr9ybd8y6PZFl%4DfeL*&Dg(7b!f@w@i zj2)gy4>kF`dEl4hKLCM*hk<;r)>UOKhti_VXkzQ<J<qBJ6Wg(0(gG?cfP0;Y5NMqq zGjhD@&<C@5B082;KtAVj!rOdG+5aKz`L%$R_ScTO@0jhUv^O#kTF#xk%?VwCa332b zeFcsgfGlze<WvuXmgU5OWwUkKF(-5mgv^#$`)%aesaPaBL~K5QPsQ@;>IEM2{_TZJ zSRGrEJGS)UgfvCVXd%c#L9NT*Y8S5)TFE?oI%csOp`rt<UIU9yo9}2ft$+bxl_2%- z{D6e7`Af2sslDZdbsLa{oun^f1A44<nt*qHn>cAC`KWJiqwjRGUIa5yKXTRWOv{SP zW~}#b%gqQ$4{p!(NZ1vb%^hjkaaCt$>W$?o(}$)MX&&`08eyybb!p7YG%R6zo*-_% zStPKyoB2rXYf2eo)Xqu>0XRU3bTL7ad5`M*r8uKfQO+qS=MBMea{fHE!s)9gRK)+3 zGEr4UzVlRwsD~847orT*s|ud!(keteAq12X;-#2i@|3Fuxm}VlUf-fCJ;$r{s!4na zUcM4f{b6{cyC;|9iA2y;QxZ}<J)Tc15??OG%j2kuu?M3FxVvQamnZV&33P1yIf=Tb zWi_VcEXJ7<Pj)8<dIfs8d&E=1UGf*Irg-BEy$Gt?Y6?VMfgQu5OL1=^3aO8$O@6Bo z6N>&f_wc(a05#XI2<80k7E^_AxkZi3@j^aVRxL^>^7Ob_S6Y5u&tBC9%x@o1<u-c? zo?E5F={-DedJ$0%B_6NA2xo+5cncZ#5RV5wk_GYr-^!%J1;&)xmy7a=KKh~=`GOtc zA;JELLj|xe;W-zenTBh5?s=JVtxPLEJENM)lChYtwOiva@Ui#_1<z*pZtIW(+CI@w zJPU&{=pz(z$;x>b>UV_z88<K@0iuu-*kQpMSxtbrFn>v6zBou;Epp^(tqoxe1)JWq zLX6^&05_3NIkO?P_-9EVGV6l`X-`5QxvUGiDtpMPA-yKLM%)l{sKHaApYP%5ZFJKr zR>ta)V`zM}lFFitCJ;qEqpd{*mMenOLQ0?}Q6evK!eo)<B8xa#ZrS~#wuFEro;nck z?-`_uR|dO$4yHJri<)7<B?;q3j)LW=>(=gmy<FGY*y>#4Aj$-=1%U@W5BBMycfgJo z<+z#TBC6zRsx;upeL|I~S2LO4tnTCPTW>U3X1UBFiyi*b(lapwM1ODEl)b=m!Cgax zs)TUQyg_+vu%c_pH&Y-?uFYz}stxr(**^XGbNVI!@#-+!DRmLGLAoH_IsJ$&UV9oN zc=#`&-lj}j7GUBqFRhj+iQGTJs9DV^hS-~73XFG2d*ZER&16FeF|U=j+1>c<+K}2u z@Qh@I5^9OOJeK2t@fz}^Qm^YU@G50lL$OYCNhp3U<hT(;Jb|BYbq{yXsxRB0MNpe@ z0^Ij|f(zSY+lg2tHUi_~>mL))Y2Dz9MFs%#?Dv?0Jg6<Ujexap<!x~9R`VAHbXeE> zV$n;z&Aa&yk);<e)op5M_2{T?spm2*J6E2}fy{WnNtS2Et=Nn?Fmu`TQShct6HoFC z_aLE29#90bG3c27C%ge?OQb9<$llB%_<Az&^VgW#ZKbq?aS{qzi6}*Sl2J}&mUBd; z5)mA$qdnDzJM0H6W2;b@Um{{~d9fY`NepW|#))*2EMJ1~LZzmT>Mi$il9-nupzPd` zE|_1o6$aDR|F39^B74{v`DgM++YxH6-RBhHc@PHS!WFHDJ0Vz%JBr2|gZvgl3P`Au zDrfd`Es*{@GD$nKf$(JG`c#tFSn9+j5?tM87gVhG2bG)0no@J1-);F2$1UzJERG$^ z!aG&4y;ZW?-}$i+#C9!vg{PA}m2OW7If4M4@@s$}5mm11m5`mP?&6aY9t7@-65;<S z`fe9wdbUYv{u2!T!%$!CtqrDYt&m}g6}L8IJ}Nt(?*a+hyX$+ER@&gz{&CRFQ-TNm z8|sGwTbM%bPw#A9c~_q4O;TFHBrd7lC76<%_^+pNY9hAGnX%n2@7*~n?YFC^2NbKF zNGk3+oJgq2W@Dj6aNp)qwU%$CzPV8&B52j(bWFm@T$fgt39zWaPiFPn;@Z!^lzjy~ zQC+W^OueP?zDTc^e49LM=a?mm%R|#P2WbB#V01L7HGA7B&qTZzB=19=g1uKVO@*@R zphlON%S45%mvD}knEqBtG-W`PAhZmLtVbgG!JWp!1yi$KeY2<Ah4YPq_?At`V&q2L zDZNW{-MG(xl^U9|nN4vyC2<>LE02$&Il8gBz;kB!3emQ*ocX3=7?L3q^K^<&Wvva# zUN?1o&rq%0<AM5+US}<8648n%LX-({4Fv)>|9-~Q#t=VNTzFlgZ$^f1XC|I^HBYD3 zZ|f{GmD{RpOjP}!*2A^j8HP@71^HEAdZ%1e<m|)bWEzpJQ}pTST>7tT#@_oYT_{jk zoYC=^^mrvQin?FQ<(`=5GG{>kMZlkz$!CV7NNT&wbm>j)`wods5$ZPfMozvB+hbn3 z$_4P*vb^oB@?(+J>#Tn*O5jA)U&jS5EAgRBQEY)vkpl?AWaR*0b(6cNAG|xM;nt>A z{bKECm@DWJeNT{G=H|2U?!oXA4%&&swIR$Ie`08u3B~;4AJYaBj>ma2FZLvTEi?nZ zt&lAOf%g)qqT3vOmf#tDkbYdp&o6E1+KA7wzyu&(gd{Qpp3RivH6z^TzQ9}$flyq6 zYgn_i4vfEaculM+#+4LLYzDw7UielyW-I#?baRbryb;>S%a<b~^YgC4fq3=m)jwz9 zLL24(m{5GdLv{_`wy_g&(?2h{ChfJ=`^xB&><D>uyJsS~XD3||t4~R3@K@<}WEJcd zjW53+n)c0Z-w?3!@hQ;xFr@qIP$O6}Klwt(hO-f=DT_4=<CAk@@?v{*s$+(FW6n4G zRqGT%86W-Bk})YEqSz?Cjgyg6`OfhygcmfB@*m_#_(P5)Zg4*&!99Y<0%=>G?taDB ziL0FtwWGmVSeAtY#6csIUoe6elBkN7YK0{o7b8l^^Eh9nyqRV$=kLVG;VsUJUdArq z)+Y*#WOc#*?BavacnB;#a{um}vLlgYv6Hr?f$}OrTFuJcg~bzFQz~l=q4l-I?6iRN z=txez1Q%4YvL*RNorE2g7WsCJL4xMUV~SGWS(G+_;s9jp%)6^u+_C|s02>sC4g&o2 z%I|?6ij7Am2mcvk1Bg81^lzS*kS5}6^LKTOy+2GyT9mVtZk&y)O({e#^HrR2*0MXl z8}__A>JJ4CkL-_(?hL%f_GccAx3dwOxZNoM%F*4Ts-LBd|GBq$4tIQBeq`Tl1Fse) z$-Y42<H1u32>oo<hX)rN2FA{fa=23-?^+(VE@FO&>k7pXevXu7dHH!|z2d*cX8Ip# z{kDk+QwQJGz|@gMRJxTHo|TnN72+7l0D(^><t!x;WGsK~O!Pg@Lj9>NgMu;YJ1l~a zd+L1`ge=mW+&!(obC2F`jEOzRx=%?v_9TC*?$U7b?ZPK%CTolz+&8Y-`n^Xk?)I?~ z=KYPj58d|7bo2leFzOp}1-0l6CmpT)Vq7_cs&apk+wKi)XKGK}+AVSn-2Rem@dINL z#q5j2H)&&SE7Ktrt3;Pw)%1zZVKF_?q&0DYi);pejt{L4Z139!)uW>&5tWg&8q$&d zYQzag_heKG!Vh)=FQfGN3H690_Uw-zsl86#zSUmA40w~A>_V<O#=+cI7^&L4$&Uz_ z8>B_ic2YEP&jVFGdTLc<K~hT=)?MV4(9{&~xoB)5)mNFH*mXka>!J;94=7^~+UF+< zNCIV!sC4bz6>ob|mVG2|MHFKDu|Ju^*%g7ytnQ;hp$~Z#vu4}=nz2JK&Yzrn-PW^p zH+tlfj~$O1lh9a4wsxVi)&APsEmuCjxvgJ*nQPCZl*sXqh?JD>zp8fba>$!$f+iua zDk*`p2pw`s_3YAOK;`VJmL*L!(4BLWAx@jU>pj&oXv8I8fgM#d2C|Ni^?6o&433TD zaEK2G(`zg?uGZD9id`#v6ZZ7RMb4L8z!TJ7+0z8d)&qHN+mtRU9Z`CfO;5A))xZDg z5Jc}0?%gNsRF(fzT%s_TS5+r9`;@*qnIqw7&V@l0CCWuwx5}I~Vzttos}wd(F8f|_ z=hf}gw%S<N+bEKz0x^ZB*j;Rg?$jtW7aXFeLDc4$;*HXx?c0mM!U7YmR#<CxDPG%} z+-^4Gt7WAKQKEkCfN@~!VKhago0YB9+iVfpT6-a$v(DaSr`fp>2n@nfyOw5crG$6I zp%;9$_}WhPcK~EzdnHly31gpm*wJT^{Zg}@pq#})IePD)ShWX2PM&-<`Pq@P5r<El z`!qRUckG<bIi<Cbps%|S94-f$=;fmSX6<kKu_*f+MBnC_ToVHvMNn*=oPLW?yoAf5 zL8~XF_s5TIowB6EAP{`T)+x+g%Z-EDF^}Iod|)Yi+3omwKg&uxe8!XI6+P|P$OL~% zH@2}&<>mcNLB753es^X2f~1W|_^o1I&Auz<&NSHfmi1H{v*L*{8t1yQ(X;9&T25C| zsAdqu9a^S%sgey+x6K}}eIAnt%=gsI9;-#y+M;z{!1t|v+YOnluowS5*1R+1u|q-Z zY(re*qbEfU&Z#NaE{kF=E&9jzM?(Cx?wr_!^6p4Md|E|^d5p`g(|Peo=iEB~4ErRF zh7%`>ScUd>AIUQ&yLs~hR#8eXxw-$ENnYvG#oGz$Cp22`|5;lZeLnoelWrEDoY?Ec z(XHkg#iMrUtNv7P<aa3YYk0?k)gaIE)W5A>XIFaLyts14F>4KdP-E~eX8OgQ>Gl%) zOhDwfUV|;&&^PdKYJ_j8vAdjd&7|=9MB=uz3vh5tbn=1119BAlk5zrjBxh|(bdW(% zgS5kTt=-EE9B30N*|O!$n=SXX{aVm=CdFh(t7?2Sw@}6oIiU0VvEDyjU4ME7cN-Yn z?gAhY0DuS@cliIKOq<~k2bjRxdd(nuz=i1^xS-IfA=UUU1uG{kdYoc7`|b#Xrw=OM zt|W`z>W0p0&W0?4wKwWwL*|76731rYZ=NsO_g%q7tY|A9x)Qe|P)@2D$T|%l(#JfX zMB-BrUsE&?I}Xm)Oh+HAu9@BMv+P!1{UJxQsW_L2%A6&z_W~WQXK`JycUZaH!W$S8 zTzU&#h(ecFu=@;$&b!xo{p?gz`F5c6Y}3l{@X8Q{hE}*MBl?Qrp`5C<r=<qB0_GL^ z9}MzLAoP^J6~0&hr~bOnn?2oiH4f5d9)?}&SvuV{bZ2omXvCr^qpG2(E2-N&9yMgF zv{0`)Iom&Ub#5f#u6tk{=ldR6tPrcTKOw~?UEH&h-*BU0frh(MvK&0r(y^s(YkkW^ zaC%{PS%KCh)_N8CI-%c_iAWiG?B>-G8-wq!WLcaLM{2QQ?{dvP@$dI>&A3HC%GgKa ztTc_@6Pv%<EAD{&)Z6Wo0=M%ue5lzX!$m-!pW@)p;y54uc#nBL*TGl=aY?79GbQCT zCXJQFRb|nmc#PhhD<9SyR$B^JxDG$kB>q*5q>Gt1sfz4Kot5m6GO^s4?rjQ(CK~6i zdwsMs1Mz*Gz4wgQ^`ae?U{VKF1<utyeQ9Yb_M;+EI<xC6AR#!Z^Po11+@w&YC*V|+ zbENCma&~v7tb<Jw07Oeqc<<ZEX>Lt|CtO#jtqE;LlZe@7ico^8PsAKnrVR<Z=3Ioc z)5>7J4wd7P6D5A~O2YX{c0+BVIFD-`b~(KTMT)m)-DY;4N7<pycQWs>F!3bYEvH=O zw8lx8O++`GPZry{(&MdiRr(Cd6gpAbgPSotJJJa)tC;IL7~y*Bulimk@o|v6LcUr{ zicv)C=*D{m(wCNa$8TjNv?_26*A5mpe6=lfJYL;+*rU*5RQ~NMZVZ*>ea_pNZ_vui zp4TYz-2v~kvV*4t*V<gRpQm0Z?;cui#L+XLW`gmtTBIy8%k76cgQu^mBDxpQZo5Ex z^S0GEET%1@Ubvli5>d0agHj&rli=;pMSiD$>gx*yz$ZS@6+m89wm$!o-B&dWfWRd) zBUp(w^adi|w&%FD=xuj@46e86BP{5DEU`oNIO&#!omY;}Pd&uD;)WR9NcS5z>*GDn zw#CdEIxEo);gg;yPUWmT&BAUXT|3#V;Y11w3M+?AeFU{xVAkg<Oc0mnc5EO3i*doK zhn`wyqx`mYCauaXf0&~|II~T%`_%4$zz`g_=IYkUHgyVg6A&_Kd*(IpmE!&c#%mD1 z9QXMR3Bh(izP{zzN@pE*J27x*w*mxE0(IIlTkCL)2fss+l~rw{mkOy~NQJ*{s(d=k zedX%(N(gkI9ys0MGDV)Gu%2u^>s2kg)2)5z)!Pu0FclNz#B-?$E<REH7NDjY&v>Vx zRIcV37GXCe?rjqKeH@89VZ*=wZEG&XG}9j3=QpbHwgb3Jblr=TLi>CC5Z=!p^Pag{ zJ)@C-`z!cKp%?n5;pCV1cl7<~lW$I`F0YVM@gi%kPc>+=ycJ=&y+f5tkT4rhuZsO2 zP^%<_FS~nj%XM4964t<9X6s)fE|7QRc_i#ODI#xJh&waDG+HO*@{^)RCZ4SHZ`tfM z8=&%M$gBxl<n57yaIScxBz{(GS$WbwK{}^TFi$Q&Np`|}1+mwD3~ZKqT_y6q;CO-g z)ugU5pGPwFvJK6z9``h03o&>3p|iOUUic2NB0~0l+0H!Ij%(Fu`Z}fizb5rLM1#qf zAN<)s3GuptNw~=3G(7BVoI@h*V86&V=lrF?-ZvJ|i<RuXfP3$ih3%(3R-Y@cl?`B* z;Zy_oe-Bj<Fzb|#ps{l*Ps)!Q679UObZU*^)OqMKmd~Jy!h9+PC{%aYY^h|n#bV-a zQ7_2LH(FbpX?+ixXv3{-UWuDmqfEAKr$6KYEH7_sYXVrsMsx<d6#|F06T7i1oh{Gg zdsdpNx>z@iPDW%5_Z0mX&NDg0$dQFsz0rFIT#po}Z_E^|Zy){2{g*c?4<9<jUdt|K zo#H)b_M@!|J34hxTd;MVE6-Z$<p)p%$?ng%<#m22dz`tnx694S{hkO?r2Ov6k<hax zq(9C~*x@H_qfaA^$N(ESw0H`D^baH&PCmeVlHZnI9=WO?d46&S+ZKtB*RI={eAX;^ zEv|0+zflqCpx>54(@xJKZV&hT28|^<RCCh<l@UVLt!C^XJu~P<SXh}iq*oUW`Nne; z3G9^r5}0W!C@L!=K|?OW)!L;P?YJwQI*)>%(^pbnZIM$^O~b&S7<wy%3@)W0?sHSx z<`5y;MqU2<Vp@1iw|!2?;!ckWfdM4z&~RsVorgu94C!q_enhW~f>3B9<DxF)Fl)2J z56I)Pr^IjKw{DGp^FQy+)~#&+4~>a06;F7-`6OMF4A)GeU>Yu5D5g*Vf-5?5YJ1dp zePd7h?(6*{Rv@AV`yI@sDV;hD&+cZRo~S6pz4B2W>hK^O^v8hSDyhm_!_~E)lC0r= z#4TWG_`oqKI=_g+1%}d@oEW#lZVx~$$j;q?+9y6^6DYEu@$b(*ET*ZkkyS8`E>WNE zuYc~_FN~yfRVub?qTZ2GF(xKEdz?Kyq#g-T0i_nTkYvM!QWY2_q?H||u~M%Iz@)v! z;-^MHA`*$t_<DwhMMp8?P3wI;Dmqc$6*UQi)p}Q&!J`Y+pQw=dL>7w<*Gp=CAKV9D zzVQDa3?B2({|te`TO+C0$IRgnyjljg?%FTFgb+DcO-7xl+lPA+;KAHC^8OwI$eEC_ zoZ6}6^v~iOw=0STXoj=H!~b(cW+5Rj*Tvd-#@P#d+_?16J@xKqFg%GB%<n1XIG{AJ z2UP4$%BFg1w_Zyk8N+`A;v`I;E6UuDzad_whFmI={A49u!T%*5po&hp7|e6K<i;d~ z&CB>&8<vzh%e)u_jfN(LEEB+V3@26GeUyH*BhJp^3cXwr=(6d6^vRvLj4vTA>}^@X zR`WtFMQJ$6w>hlP$ud00$Wwk!2}|3l#BkFmhr@!PhX;TvkrmdQ)^}r9M&I^hryi)D zOFzO|K}rzW#=50&H`KSh^I{;;X@~gs%S%ksU|q-SXUUFmBy1^%ar_IpqQSA!jaIQj zAErZ(Dr4_}{7bKCa(aIuku&JphqfHHvwSe)-$t{F4Pf*KTAM-ynNePz_IiCHA=Rl( zkFNM~A`8D;-WgJ|j2iEez)e5x$M6q^xF8d~A2*il3*iZeWK3inNGn*=>GxD{ox8U6 zmmfQwjNiLgwa?GnGmnOAK5F`>S6!f6_XPp^(SnyzRDSpeH#xOMojjXz1(l<tYEJ?_ z!ZWW^(Wi<+Fpq94`43!i#9UZt9l$$fWRZv9N#=O=<u>I$@uwi6p;$ww{h(GIasiWY zPNqh$6O~Kvd^tH$Q0JKT8e(BB{eB806#|h*7H(LOfIm86E^q;6E*~BO3n9X;L*ZtK z0EFL!S`Q@o-0y(;z84DW;nv-rT-b?fwzR8_a(2>Un=$(2z(zC+3ME1y5C|W+LJeyo zy>hZF9VDmpB<#ukT!}YJm8~`2bNBOZU&IW)(JS@!v7;4swY{exit<n-lFz9#%rBv( z7<j%NK1zayVRtXf@>I@gyIAUmMv+dfhbcfG*UTOs)P+I(p#t@!OC)kW`bXDpV+m32 zQe6$9zg=Zq6+<8pcMx9c%DT+}@R6RcS2o_NeM~}p`RLNInW(ciG4q{L3=Oo=aBe-4 zhYTGIVi1%aK0s>*v;G!Dwo=#E#*9J?z&vE@7DUWXOP%N5XL?HOGKFn#1;5>TO>PB6 z=Y2&>N5EH<<hPVap+kcSX{Vy|Gqbe?JOxQU?M)d``+=K+ZiQ&z?x`L%>oBbrabh`Y z3qxPPeo*Rf*7fjVt(nSzz%lTYK4RCYijmXYY1Vdz|C=^58FgO>oXI<8Y90f)FEJ;1 zuo*eGL^zva(I5q_x^62LE?U6y7-n(*xjw;K4$Q;zRFIk$&Y#Y#1od+^r|Rj;8V%R( zAMK!bqgD(btUxLF!RiQs_TYCHF{ly#yR%@@XzvLFrhHm=vXG0ahWAyo|7r8L4<2Ez ze|z{{=d%7Hs+SNo3y4_vAg@jLp+s0_Y<xD)Vy4M6&(c~SJ!7F{_{WAXz~W+8PyEou z;@W2x#7rO+3k=x9fnKl-e$a<+lsim#pVTmAw_ga#TAGq9nUK3f!?oSFCHEqj#0ZDE z5hxpb!4{g#KGf|wf;I;~!oh<Cr309}1^oN(h3P|wj$j5E883&R#(lj<*C-GH20sL( z01=c7np$bJ7YVsG{cZj7xe0cAOs~f?>{_c^VWW_Ex60Z2C$Kp-5+SFwF}5<X7bod( zy}-!LrRH0t%aq0*Da|RdOhCq&R(>mTn4YdOpVi8d2WxACwK?(wTJ7cuFiuCig@(&A zgEey5VNpsJ3l760&i#KYjuu+MEUHha>Cb5GPYvig`Wn_)6$d?Fr%%7;Fo?knjuhXE z92|_iS3L4g9n3qx%6nV0z8;+X9Mfem#a_2Z=g7|8tiUaM3_89h9Nd=mR-qOdPaZvV zU54|#wa3x+G{%ohMtw0+tXBb0%6Z}wKu@K9YxnV{Tkk7@xnrLZ3<SUwM486Q_qX0a z`HgEVzZF$N)FZ{QCf0I8jJRdiEaWd?bXyQuPh{)VntLt&K=NS=q!bxxU?5TMyf3f~ zq6Vo#f>`btN%croh%9}h$fRAg3r~5fEUv2F?ew`DbVp<e6t{|>E<opp>%N4HtN`|X z@7sX+?i$ArIa94w60cVPfgw-I8luvbr0HO2z`8%1FPJ@_r1J_O@NdWYBKMgZ29G*8 zg7`r;0#-}LBc_p9t{=9DpovLw^l^_%g^umqc`VVmgF0SNL3I#*-`(<CumQO>pn%^z zi(q7tnQSt3*xDWcb`3V2HDc2J3z^5Qt+0Vh)Ax4k{O!>ek8cZzfQqim4V`ZjqnQdx z(U7G$5Q^v!FpB8NO^p2c?FoNVf63Sv5>6lX`~{ZOCQ<z*!ou13wzha%9wNG>I)--3 zMF?UJO4^h4Fp!i>B9LI@M}J<rq~Ia(+`jb`Gb2Z`%*XwUS5%;4_(8+HCz|s3?=%!i z&cjJ>zM(bsOF*+^DaN~^NI7L!8ku06qi~X2%kd{V?eTHWTz%dFj>j}T?yx{aH-F$- z!1EKCceWN;HRa}>-su}K6gHFpzSEe^>d=ybAhaqe1GDJtfb)8{M;7W+JOM67IU?ua zLt)M#dW5c{id(*Z#ZW$)lHIgp1CiKTLjR9q%rtBs5W<q#pX1$gpPlQQ)U({XS&0^> zfodp9m9*8I8?rixaawOBIU*p86`#rCg<WGTI=5PK74)D`K|Iq1KyVSnFODMnoW|4* zpO0tDUB6p(5wmZ_SDYxmLC3f=VM&8b951m5k11Yi!HZuL+3s@_lgB%eUlSt!fUVUO zOO;u*2?=cx#Dtn6xNmCd%{?(l_MT?O&ZIj=AkWziRa{m+v+y5#z{NRPV>U{hKX~5E zfLHS{O)aaXH_{p(*qNT9?nrW0s4@z-krW+C>a^}W```%c;^ru~+~&Cz2JH`=4K;On zcWOd(h0Fit9Et`(k+84Uk8c+bhV@)!8#7tqj{3DsT<*%cYiuKP|8vmGf0Pc(ugn`1 zM-vX{V*f8|=Fr4KS}>OKauv=*xoCw%*cx#;;r>_a^PkdsvqK$>9XKFBtjQAq(?b{P z1vHU_w&I-e6^br5qrz32dtawq(GY--UwtDXe0r29F*3MMhmW1F1iG<g*+BY3Klyk! z?h8&XKY5dCc1J_>{Q~9EjEcD;1^ddH6j{7%L#klChR8DOCnXZb_w0aTTWQ>@HiwDn zXiP?u3auGPPhGwKgofVdqYaHs6`kSkBHP?m?b0!yP~g=H4_grO9=VM<ud<}b69)VJ zmeRsltep)MxqMabueIvKLfG-S9Es^5Q+|4UF2XeY)9+MJzJq%%^`&Atj&R<2%q_lt z0A6~mvkLlL`qn@dN~p?MU~cYW5JXpSYYsVIV=Njvu`=UXVF3|{%0|w-(x%s~6k^md zy88A+Ok?(8^>rfBomA;m43jr2Z+86zdY~WEfX1T?JdSS5b7@3(9@(KUv&Ewa!}^=C z@YNGDZC5VIdon8r*r%-S%XE?#V(@^K#Y&xm1eRmh3j`wSy~_nT3&qaE<jRv7o1*gz zxzjk~Y{{)awVctU%%ml*K56jl7W$3_r%xB)^Z^3Lz=7V`-xT(Km)U(Jx#;?4ePyQe z=Ut}Mw+_4SIg_M0c`~UsS4`r%|M<6c<x>kycKV6N+Hs-MIds`6X-C(I<LEG^&hLjd z`0}NvM+q%3N#y_|$QK!P;K>s)myLbJty^QX0>P7dsg$8M5?956AuVueKNd@&q@<iU z{wH9#P(yNZyYcK9(-X(fa9sS2n?gJJYZl1^Y5s+NIMm;-Nr;DgC(%R&iSdKc^2)05 zoJ!ymB4KSnA+JJ}wo}rAv9bIhk3SzDd%oix0p`Vm8f$&-q)l<USB1sSSKcfyzx5lS zQY%l`>_h!q62|<IS$8vhV}JO~!u;fGJ$(yNzW=t{V>?-?G{EKJ8T<Cm%oT?R`(<P{ zZmNHg?UVc3uWVyr<JT;`$*JF@KotiMH)&0ekv*Bq;zx6ytu$u?>gR<e)^bOV2BzI` z^hoNeLmUGLHE_Novw~AE+MOzqrsN6FlU)Ti@(+U_9DNu3PT+X{oV@M3q09HZyp4;? zMRKlCKT~qb?qaw7k5ASddlpfEt#FCS*k~$Dwf@u=4`R;!?~cs;SqxL_Zp#fX`$Ag$ zI}3f*ZGJgnxiqqw9ju})!f32phYPaT^y$rxAJ46pN+iWGvjvy{PRAzAd?&kO$K<S~ zFUq><=lmw&r=_zjry990o;ft^oeJW!XNQp~8D2yN6oL*2$1klFP$Ib8h(%=6y$c^E z9SBn+mem4qOQ6W_fJ7dc+W|!Uqze1UnhX5!>KaXmIYQROG)Lhc^JPHsW{!T|yE_A6 zez#XoYYNvxOabWejv!Qq=aqb*JC@yc=qcimvtdXUlD7<&z`5{xu03pdPWlw0Q(pS( z2H$u`hv}~{7^($k-^O?$Ww-;zxGtJGm8QVrTqp_$|0r&6L1|C<Ky`FFDU(y!P}Ws8 zza}cZ-}XZCcJ<xua12I%vs=Z(!5X@wKn{xthv6Ju9Nf?CN`Vy^m|-x;$YfNdc+HAh zt<a$hjhN(d95619S2A|t&G*7FXk?8oLt62qzgE>jK($AN!?Ap4JMQH@8Aa9@G|DGS zJp4edx_k(Wm^5C1aS43oT;+fJhE^3H;_VxsF>s&{C0oWLQ`GO^BkV@$i~8dC&)6ff zs4b>Lq)GAG%<asjzAIc7DJ<<*d3o|WBkbRE?e|foyr`JvP+mOo^4~@NFY8d4o~)Ek zd546$^N9c7o{~Q)f#_wx{Zsvte_}cbY94o`Zega4((YWiRLA#SZayrs=1O{_yNyO5 zBIl{RqQ<!}ExgI$Xo~=cmio(a^R%-lLO<5&=2{4QDYCK4HhZ>Bsr8l<t_?GFL;v z2HkYai-O%}f;ID0IKIR7j<CKgD^Ja3FdIl!;M}?gw{}u;LDoc%4VkAlk%ki6#hI!9 zAr|NY1cpQ@ZaS%-Zet0hVT<auJu<f4tMBR^cXK)PPu*GARzO~j<5)@i1A8lr66*U4 z)gzYYVxO*_iS4(lSM?Sfpm`N50zf4Vv8pgGxqG@NetTqudeU<3(=QR-+m3AWf1bbP zolK4X7%F1h*<{jR2fe?b5}Xc2W4&ho*Y4pJVUr@mbJ(@wEeZRbb_EpYB$W<1HHKZ> zCM>7Si{DTetjkQUS>fL#IPk!rKK9ZN(LMOWTgTRS+&l&<2}2lu&Ljd{n5CXs$yqo5 zn^z=R;gf%{tX`0uapFcLMTOSc*Fn=1R}->PsT4QLd)4sht<gnDGKWJ3<|#wk8;*6& zl)UHq5-xW+LB&}V$di&wnz0hr|ACyiW9sXf>&fTkWD3zq%%hh)<r@s~j@RQIcY>4} zR8UUkko^dEVzQ6B)S<khvLb#+r_eUYHp*WD5Z*9%ZbVB#K~9d5-eozszRuDZsncmj zR%ZmXKm7Qt55gnB(cXNY&*$3U*1TR0GNpWo#hQEQgh3|oSP+DKwBxK>QD|9+UZIf7 zZ%2H-o#7)_Duaq<WneAF-lMEs1_u@Zg*L`%^@Yp2uP$_TFJA4k(H?pH`QWu{L8z4V z?a2iK9-pr-;metV)HRvxZqaVno_g0^G3VOaT3gSZZ*3J;IW-T<g$Q5aHshQLWnSf8 z!OtO0<?0(52F*>e{pm=d2+@aDcwKEI@7mRmkxNQV&kr<4EvuIpZ&B+*8=b1Q+A`6{ z?Xw2DGjT72RG(eFDe)Z^JT@+BcyGTid_zHArdwk|>N2V0d_f7hdvAZxF|CzLd+`P` zK^0(6t?>*SMmW2|JEzqrAij$^5(E;)fIwnW!(Hx_qsq6@aV%EaZx^3DD)5r}_-wrq zUX<LcwHQ%jRHTYu6p<7)KP0nN*bp#ZIoFCHX35bh7W@Moh7v=<#Rb=;m9#0Rj0^ME zndpR2rduL$EH;@J#tIg?sNFns9$JlUPIfL0gR+4xC!p)P4Ik8;mJzw%)%<-qiM3G7 zg1jUR{JF?zKl=CT%Yz#-J@@m1D+?~gBS$@VPm`G(M?Qoj%wfm?=IWaOV`L<tTpq{S zfQRm#__godyZVaZXXCYbv|74ni|clbVxPSC)VE=J0yq|_d&rx84nk<sLAQHtL<9bY zu7!mzgB<iHDoeQAPaA1#JEEOLAeJMU6M=x89Ih__Xpn}Gx;v9V8vo$Ju?->g+bjRt zs}9U9vKC{UYi=(3%kOp>mLxwqi|>i1f$!Xx-^IZGV#j;m6U||I1Henb!|L9nWSK{6 zc~;i8yupR1TKTWdr8>9FCt8jbb7z|_0=ofETo*4Z-)Z|UgrzlV%04Kejtf14|32~v z%XS_L+w^xmH(Y}>z8~4(--vnf`hF?c$#EG@O928G0&}Tze)2hgJfheOYYm*>w|is( zhNj=vZ~4QXJD;`3TIh|0umt8o#8Qbgr*?9~txe5=meI2L63T#{my0IyUp}>PJYifW z5ZzK1^IvhFzs+wAKv*JBT~t-xFnPb|zIGYlcC-t3*6RJGbjn@jRn?ak?P=c&hddQS z)8g@Iu6R9TF?KgOiYR9J3hYhlYxCNKI+G{bstUVF>WU1N2KQimdCmwqMD4t$@imfe zj__3uI=VwEFFrX{$3`e4Wl5BLl}jPI+TqZWlWZ`kq%$_L*>1;7N0((PHcn*?FUyP? z?bMFf#j0v*)tcjX`n0X{W%b23a(vN(kl=)r<kpmw*5u>_nW*Tlp6uNXgF)(=TFq0c zLvjk%ltSZ4o3d_nhuYSDwJpsfTH{u`f4kbqcKX&G8%(mSLIE3c`KKZ|#g{dn*uy#C z9)LJj2EOXJc&rC#>R)7D%Q};Mcx_h!D4(}}tKSX!P3n1pE2SwT5+%xlwV5Av{i=nX zf_~nwz83q3(TR&HxAdg9#Y+>Tlvs{~ukSqg&(UYA`!@i5U=V=K+SYm!u*OI*l^nFs zX=_=SJu=4@7UbdY`{iy8U;Ec}|5(5NM^{$TxsHyrfmvNIOFT;MRAg=zow&GJv+d^f zN=-IE;OBDPjhq|vPWxhNzVFjS9XPdoAkD%j<SSkdw`z@xbbXBT2m8)}v|>g<N5qPd zq`l08`RSrBubpF-M5j9^X0&^Ah$(roz!)d`3HeJgH?;bbMdj0N`skRq(v;JG8*AY_ zBl-mE{jcHbsE<?B_({0HR**X23CCsd_!~GLTA3(xAZfezKt2ZcI9r6~!0R+vOz-{h z`-gj?z)CLu7s?ba3N?I1yw6@;bJCq-okJ0R>ERm(*b+=Y{vkc#Nu?AQb$@#5Z4R2s zkY2spNmV+O5P<2JWdDuB-HZ}p4nJWsXaX;gu*7NZdBr=}*KP(;x{3JbZy?z3kdr8j z{(<??8zu{%Jy<&}**}kVYeUK@{GXi*8)j`e+1%2fMe~*HW?mf{qVR<+P1lx6%jZ4r zG;b_%m&=l&X<mU?Tx?k0v)tR0hLM+-4|^>-f3BUf<-_~!{pVJD6ygusKR@**+z#_9 zUupR8uaaG&#iBsBkip|rei7U`8GFp^9aXe&t^7^>*;pOdkf8-?`ozgo>6@unIy&#s zKvoo!R@uI<sy!{ey?t$6Q*b6gmyL~yZQJ?cFSc#lm`t2ZCbrF)*tTuk6XT2R?EJg6 zRr|2#vAgPapYDgQx_!>QMiy^b`(7xJK9Pg5Ifgw}#EUkT$JQsde_T;h7pswSZdX`o zBSt(hd087`3w@5%ml>7RcLn^BBO^zV(9mOrW?HmyHMOy3adL2Lc{&>mzfYG}-gIUR zvQ(uPmV|mCv`7+D_a;#4$`4*Z79Nbok%`0Y9Sy^dOFK>k@$5R(jS-`_ET71?$G^1j z#hG8oLeZ3y!I<d-7X*fz{RRnpVuk&7cZjCJw#U||kLI=l5y&BEc71&(*k6qNYM-^> zIr!2KKxMG`e%y50jm)j5zrxdGk|6RbETSD?hO(x>^k(_Cb8uRYT*DnIqva{A%}LW! z%?zE2exenF<@3*R@AmFSnk+t(IaEI3HZ91nt3`wm?IQ@KIu4F2GPNIFgW1w-^5Tjr zzliSakOP*e2+4~lXJqpP?xT`+QJ^t(OKNuLq7nQ`U_{~f^uX0Vf+Jt<trnBasR9!U z2m#1zO7(~e`QI=dEN`-OW6V{6eYYh%cFDQ3$H=t}Jeqqy@n>zdIy!v3*TE2yxCq+3 zmx2?LZ@vO7E!oLXgADFuhj0Py?`ao@9K$>RJRZX#?8>k$SNF?|r3xP5aU*ScE6enB zWo2B_tEVq_xcR+Q;G}N9c<1B3U&`F5BT65Q(LlpRp!gFOz}T3DZOMUSZxE8V`)k*N z1pVct^9@hQl-|Lh@LZ@r5e~>B@eQk=Zv)hL&FJlozmJ^-vaz?bkE?{3W4|B?<M;Ng z>9Wl#rhXOZA@F^c##c(~_f3A^44sA8$3F=Yvq)2`RJ&I76~~@H!P<-0mJstYKMk^W z-sKgB0TZBoVR*UQdEOeOoXp@X?j7Q1#^VJ=N6~R*JeikR;1#*8w0Kj3_tfuvYGkcg zlALYL&ie#>9tu!z{eYXNOosb&YI;j2*As}Sbr*4<{#7@5yMvCd+RmfXXPZ>?LQ~cW z43IOF(h6MlNq0h_;<>zwepxd2Xo4-M9|&lgk_ExSSZyl2d&6@uXGa3mru04xOC7_2 zeTxN<v*3~T-P==Qy!qPZJs9(1Ktia*2`(R&$Nr0y$#Wl;%Z(+Ue$oMKc5xqZWcOna zr|8p~p|h{a64bv0zv7Ya1K$J02yL^>LP5zdtLmE+qnSt>7%*McATI{_ggapmw$ba4 z)47KnvtHpDgRN8Gd6DmD&VU@!V-#;qkolx`T~Nfvh6ST*^iw;4i!0=K2GrR(yB425 zx1z7lCDO16g5L&2!UyWzO^JT`w>I_7nVv$&xDn16db~&w(;2%dxz5GWS!@?W+l%RL z3d>o2*5&Tx<n(yQ9<eys)G5#rwEN<TqwT2X%=IMI2o{x&vz(*6`?f=kID2aqDvJs9 zy?d^?lL*rCreZtrIIKptFv)9G*D^cu)#>_q9OdM5w<CD`{Y6Qh1HGoiReb$Doa?Lm zG`jataXIjwpL+7oMG+(^jjti=&ifhC(t~OR^Q}6qFQxOZm(ltx+>!~h?hpmOUgYmi z>Vw5{pBc#t(lo#3iIUn=PL(2~eA%106>GSzBJ4=nWSQ33(9U#p+#cGAG;K6Cc${!w zp!zL!oX6Y<YUxw#;m25(C4RWE%a-r4`~(vD*0v0_21yp0GR<j<m`PrN`2H;RP1>K? zPhI&O*L7gLVKK|yzjQ0m;&LnK;Ar(MF><sMS9l<$rKL%j^_StP<m~oEof8NkB$Ta+ zsRMWH-vJuB%{zooRgv^__>(?R5;3<n&SL6~oZV)0mTR<ZEHIC7x@e3yO8I}jF(P)3 zs%KhAEWZt7i>18I+O4Ld6FyC$%e^z+pvXz{l~9jfQxHf$)q$Ogb2+$5*WC2&13Btc zb|lHGdOF1yW+UPX`?*(dB8OU(XM|dJ_Tb4nu{2yl-EaSin=LoZjtvhQzi(aj{?xA2 z*VWyZZK&l1(=@1>ty>FcK=r+|ygG0RWE?!6kGnY(sWxIc3{F3!r2vugB~K?sq}csb z*>s$l@E7}ykdc*@i7ikw)1dHV851~GR7?paz>g7f2uen=i2HLey<W(MOGFhBVKZ?z zC!v92b>l+Me;22Ebi^j89XnvHWgModvFZwFxteCyK_{Pfc`AnRn$l{Z&4W~^yrj<i zbZPvY34cn1HNEZr#G?B$L;cx&ySyawubZFDh8Cw+5q`#Z*!i3lw$k-;m2qr{&7jDD zbK6IFZpaD3=qD{Q>q~P04i4Zpid?a^vu2|4`97BKQtU=SAMAT@hYg!+U8x>1a5l(k z(q}(LUBdg{{}lW_cLmPA9Z<LZy;4Q51GEE}VTlg!jl>(({PJO5ffHP+-XyQbV#q3g zT;LT1k;*N|TQC}{og&qHOz}EtP5mBAdbb~5M<8m&Gg_RNN?QpvQB7oRPq!G@8=J>B z8VMwEe~f5`<bqC`&}I)gx62j6hAYlk&}-of_yvvrX}3F<#sc8G(-}xizQ1SCVrCOD zUq-dy@3XwP%i(-%ji`#zVNM?3IFjZoJ^?q(9T*><ZZ#&y7s!!5D0$b)+j^8DOLK2k z7-7UCIxE%{md8{FY?+A#^>3lqY{!Q7CL**EZwt*40;t%UYAGeSk~8_lQ|*+?I{(Im zM6<j8!4M8F4qAQV3JAey{wPn}N@0l94S7vRudwth9G4nKMg(MK?nSE-3AS-cDVe?= zt53AlBLuYV&%`FdQW(8?R3NFF%r_L<z7`jJ?;6m{ocxF$dzJ2i4`hP!a*Md!ld3hx z9mTND5OHVLP2l2GH0*|~e+7XWRCgEZF{C!wRpt^NUiD?L!_rB*wuSKSVhy|rm1l<j zt_Yq8(eO8Rxn@nAr31Tp#D_ff?6_=`IA!44)k`$s8?->Iwe%GQCFR)G>y@jLRz)B3 zs#dSsj8h|<W+KsKkyqzO^zY|>R7nSjZdgw`zOOz|qmmt4pks!F_i1;7XUbJ0Cz(oD zbOu<rp?18FA3R;@mu;nP9mfH2sj9z9H-U^)xJHp}G>VKkK|Bnk6Kha)c7r81k~>!B zER=eoTxlpY+10w!Bfp91QnDKHMfQA@lk!iHeX7{aKbI{xi%wg_XiI~7R5UWI*rr`y z^!fLsU!velyQi>BR}f)mg6~7VNUHx5Cl^>S*vrI`Z<0SPWEZ9&R|YV50^yR%glz0C zj^_?F*>#p(F`47~xliY!W(4pzl_dS-b`I^$h8ZYJC?-nae8$odxYcTT=i}WQ7mjw# zgHPv--!4z-8`0NNptNVs+m^UC1z+DSj!*7;(4E`?{$HGn|LQS+j9Ru$Q0Mt>bebJj zeHFCu_jeXCcIaMY8*LR0P}}X-l=Xj{ULfjIKh&6cNM6Gwm|=tRs{v=kVXMiX@6%dx zLr+l#>wYSMIwgGbo6<<=B7&|ga_(B{^Vooo`bkYEn<DVq+?T&qlNbC4uuCz~U4*y{ zm3UjJN<fRG?zo<_afc{|6<)2>k}vvDj;g377=`jAcR>i8tPZAUT~)gNk>lRbaFvK3 zWD?)<RZLoGdDF0z8AN}K_y{jTOAw-SrP5=#R2cv(7f8OwW`3w8uA)ScY3;|i4g&Bn zt;p6DqkkRu7s$A_hH5F*9(9gDEryw6JD3JlCu+@#b|iNz43h1)emv02q}K{Te_!Ab z&?Tt2aT;FZVAcNH_L&zNgbfqWmA+X5_6dyg$}U2b5p3l$Ug|93+_e2rV2q3t^C}m- zcG`-lQFoTo9-o$UdT*Ta*Z6dbZiLBX&~=cSJOA`%deFpWN=%L9C-J_pi7U<;4-_uq z!t*!d@wsftc)Y!ITvnFfSI_l{|IBAt5FmO{=RAu&7u`Q%+U(XV5*8_*H)rt@wrIW+ zd7^h1vL4Gy4QJ&zK~WSs=KW22$MJL`p1(Yi)D)0TOc**|33s(~218sBH22pF;+&qu zhXLtQ2JhZdoDzb|qX!N*IscEI1Wc3^MzKXwsQivaxBI{Lk+*-_QOAxU*rIdL8VUZ{ ziq*gTr%Q;wYk7GzQ2ubU#YL~QWT*WwKiRmFT(L5+Q}ZVC&!;gU=HrAmhKjKFvSG{9 z8E^gB3-G#wt5V-B7^Fj&k)6BEa*vuKO|38LHXQ7STxviM8f;0YF%<tza{Oa{adz%0 zXhFt9(T9OjXv7!kc5$`m&ih=_`~_zlFMHbr>4LaDVe;q?lv3x8skl7JoX=$CQ<GC~ zca8v;@7H*I>Q5$dnY{d+OuLt=6)#YesFT(Z!;@3W#F*j9AdR6S@TTvC6kCu--xuKO z%(~|<<I;;5rrr4%nR#CR+E6ggQH7wLoV#xB<=tCppD{MZS|x#-j_6GHupx&PZC*t@ zE;^XZ7$trv7yKJ>I@d0!?Ze^g<`QT~8HQx3YR;=bu2MQm^$aQ*E}bi|yq7K?87K)e zIOR1`-F(r=sugj$^Ap%yeF<I`{AqYIEXlMUwWyidaoY)Vp4p}8(oH{@Q_d3Lwh88% zoO%4d1&dl(3!1`nCo>iYZEoM{$$&hb1?k`=>>__`<5w)(jrLeMxqql7GaA1fgXZW_ zjvEU2!V#?mf)!f|A`)i0DSej9*3%r)yLVD@COY^44&(BZIhx9)@DVSl!MaX4p8KKq z`fH{%V$bXHe%>x*f>;tBe-NyB%F~m+M<(j^NpfhL1uyMtySiU9cTqyg`L1$AnkFsq z6g_0PLKn?PReWp!6$rgew@b@KNcI;?fa7)yDh+sN-vlFNb@|nwtz2Jv3>5G&e8d+0 zMCAq-v8Y+|q9y(P|LB1B`C^m}GWACf5Ja1!6V(gpsp~!%B}ww!q3$(WywZyIjim!W z92<}wiR&_v5hXwOdws{{;_Mwm=RE(ty!y<CG1uwa(!SLL^D+*xj6*gC4b=E0;0$7s zZM=8SB?RIqx)Gr#fq)=}qc^LMxcCuqx4uiE?6Qz+@LZ_S$b*JuBnQ*<0AHW;P#>3{ zO7313dtvL9v<v{Kodc)waBo(cYYO%{O%>Ss+|`jZOodR1h8n+I1VWOEFnPHv&PBLo z|3{e!zMSRyk!UU&*;xx-4>t=TA8X}|NUNAA>}1A@a7(gcyTggq!|Xi6)&Ako=o5S2 zUXOQo-+_dk%60*Z#ar~Lti@-T#T;J`U16m?8+_%l+iLiq_V+N3ZgWJrYDjU*$!)(2 z<)_E6eG}h?MP0}LQpqIG<`=jx|K^w2m{etqeH&7+1yp3E+52@f>Ge&c|1`!taDLo< z?Ry`q?!;wX3uJcBLmiO8CU-{@6GP)Jkq67jz-m(rI6PuXlqD)Mo#Yn{ChH^3JoTrG zN{>9^G<QriSF{DVWzsZDz&pkA%I?P@!aK5tC&oK(a^yntCv`W0CUfRku>kZ2n9r(P zVNJskC(vRmgm0vq83Mq~zJPen*TUaG+-9HenJyK%_2mtJdY=h$hfP<scWX&zw`xfO zI`3`*zgeJu+wFIF0JNmSr@?z1^!#WOD-Ec(KX`x83boyIK%zPF)iZA<d=wfZC@vOH zP)YKH<`A>namJ?W$iA~csmYBI6DmDi%%vn=XSWpGJ$OI5;gcSJwdPv?1Bd?m)mrlW zJ$qNanNc{sn=d;)ub>`RBE8-p5O^f22~?p-NblrO5jkR>OJA>yzx33)aJQXOhx}y% zAT(BNCoiCnwv#i}>79@jCv4(F$c?~cRDW&gndWeF8Ks&EB9o7GLV`kfQjS*W)b-~v zA{NyE<yv>K`xZS&V+yB)1>beuI_y<R*cu36)+{@ajN)E)z<WrJg?0Fzk?9qTQ&vJB zsYDe~%y>WiYqJKXzKy?}t9UZbjUEgSe|1tF`&$~7NYRvxz?25tbyRbAe27dHI>nK= zhFZv@J7UY@v$A8IIK8!;uFzE#&-hkIK)?Oi_omncEP)ih?^`@WT&zmKMw?T?<#o4U z0E8)}taVbxW+J)BL2Gb<yF2g&3QC{&DtFLYitG}AH9pZrEhki7?aakmA(2r*f4I2r zPa!l|sG2O~%tN(9<no^7^EKr+L4nwD>l_xbFzAvr)iZ3VB&Fx9X_9~Bil+GY$LJS= zu(5Qq>zQjyj)t^d=5&>>cV)U2<M*#?mHO+y=TlWDil*<dD2;T77dp6utWrCE9&UnZ z0YEgi4+EIz={#a)zT<_XIc+5Hh^vJT%7RHk&6e$$^*}d|6=;^|J>e>0aOktkZ67U0 zzaM+qMdXXE-m{SRi^~!+B(O4a@kAOIV1Yw%G8S3NU<t+%S(Yy8oM(wY1th*V>ieQ{ z@`=%UqY^ok@;kyO+gKB^0@B;C*l44)wZBY-*1Qa;46fTrGvSyB$(NFN(RSU!j=aC& zs@kBXkRq>@lPtu5@(S57qR9%?Y;QP_pGFKTOPJJ*b$G#`g0o5Lpng(K7L6wc3jJYE zWA0}1YjK`yIlTiswHaa`F{!pLv7c&OHR$c#KB35I#*r8{HOF<>-pm@HUn(9)gb)Xs z#151Dy*9Tqou2zX*1y)bliHDNv75X?7#8Q}CX<=cF^M<MW6^V=&H${P*5>lxPJYRL z-p&K{r<)xG@b8_zZd9^98(9sDS-EqmV61Mjgy?!Lw?{N4=>gDN{UaJDAK70tZ2{p5 zlnkJmk6~^j0Q_QM{ws;j60EQ7!~I=!pN;eDmxlL9lSupqM)~O5%<^qqBZ}TU5>iqk z^EYF-dmkjr4syM-(x8IJ>>X(~z%px4wL7VW#aO*`n;mmvcfSd%z?`X+%B-wS231>v z(KrLy%EF1C)|2f<f;+90o#o!9gqvhKOM&YexR=)uMWl8B*SciJ42JaHa5dMrw>*5E z35$#~9)VjnVylbnQv7s3OXUi`B}S%VL!(I9^<m<f7HU7rn+@)fAH%I1mxT~@UluE$ z%Y#k{tTj=@KoAK6xX+?_f9ftd*xn-pObVu8#&cpfeaX66G^SVio~PG@E>)G_4>bz0 z;Zt4&XL26;b3-Cs&%rH#+VWH+|IFIZt6OJVs}Xt1WQ|SF3I)v=1O12#J3fXC^gMC0 zmpv6?TBJm5Yhi(*-f+Zo2%wfnq>>3@0h^QXZa=F2ow?#!WWk+S@+?L|NjKAE8<$^| zLkfCH^7vpF7x&a36OtmKKNt5TLcQHU-^bSKx7K|$sy1u`od2T$QkJv0L!HF<R@L79 zg-R%XtW+107!(2O?REI`v7mDVrJGoVI(bJzv<^w5yA*J9naz75-?QS~2E6G<0vF$j zb*1q<T-ZT-9z`BKQG9$tW;EkddjT#e(Gup{GX<6qhed<B`$f{CI|J=ipnVA6!UVn3 zs>krb>?h=_O48fmctYHQl!rtQL>13-$W5(BbyiJ}MoRrs*1IF91XV7YsfBa{aVl2s zx57pJzH2CNk3p4**K0Gw{VaQP^R_d?eA^{SWqYY-VH)tjNX6$lns%fag+BmciwTD; z{eVqUm4Mgr3)3<R>4~grHgkOhHM1NIlmK)DJ;NPEBY=^bL5fof%EdN2GAc*tSba|5 zd%Da_mCezJ-OR#}B5eCDOYKr|h*?#syewp!p-?V6K2h15S)NpCOho4^p0%JDK5iEh zx5E`Egfd;y$Z2-YWKQw6dL`Uh+<MoUsz4TZk2d}O)-W9BslKl|t5Td+%;k@4m=oj9 zYlU`n&GAKONjm8hQB6&u9vSYwAe|Ec#Y2VlT)f5p{g#gJL6KrV%q<fLq)G&p0$I6= z&`X-<=Q0}A>8l`BJ0L5q7U<Mg;=S<6=fl)QHao^oqswb>=v+RZic}Zm1hu}UNe`mO z=LptzGSdq5EKUf?`+YG^;{mRZ>MEv&WAW2kl}mE-NCVt17>JK7Wgxm{we_u2<8t}k zhE3`2yO=e>c54;}iy6<Ce4RC(-ZM^hbeeFHo2vchSr_8f3%#-5s$A9;axGMyk=_x} zdS|ei6d}~~N&ogT?;BhrCSANphndIyfE}^fF_+`@yE1q24n*DClN<lWb?2x}Y7ut) z?VZb1vM0ZfyKFqhAxvz1QJDV_*y3{)UaiV#Un{u%ni=Xm_+gQ89Zq|NT_dT$w6kJ{ zJt)w@hbT2Vn?v9@I(L{F3B7%Uxv@!&t2%6=syV&njY-cEG~cM6L2;KY%?(%i^XB*Y zkG1=%#tn*bRGBT>mEDa~O){1F{NO2EspIQ_)1BZPC>#dQK?im_j?!XC+>TvujUx`O zrP>n6kf(ZfC;SY5DVK1NYw{0LRH(j&?q7GP^!vy~O?pd-yJBaRdj5PM2kMk9%57Lq z8{48QQJxx3-?aAE)fi{#%_G-5f|VtP;dT|evh}ysUl}sn2)6>_4#d`5)A05UZPLX1 z02wc&a<SM$9NUJMnfI=D5vkw81hs?Xh6k=1P!?SAj{}WnuTlgQdSqf_<g$>b>YE*| z00wzTjq#4xcwee33dNraE!<1rf#}rrLC>Ne*Hz+OPOl;ShcE&{W3yKE(nV^p6KB=` zRMYM@Oo1fB_Fum@?w?s^yJuO8^%W-k<eq^ClIG$`sXl_EJ_4|dF(d(pS8$PrbNPu^ z(frquSWmv59a~ZGY52D<21LA^wr`W01Za|ICa=FOcYM)@?e(z<I#n9TVqLk~?m6VQ zNX!VhGl3@i|7sEsaP64IOiB6HFg=5h^oFh3yBV(BRSfcEQcd7+yZ_Bn3O(lCS844Y z$wN$wb<~#etoew!<#aaL?83L~nr_~Dl=B}45;CxR$v~UZ{i)+WqJ6<XlzmT#olcpR zlx{yuc)polW%wIR&qhr?R=c}97E@Wfr3^7_GEGI}1e0J1>>^AFHd7i`>XSn}I49ca z=gHReK08-Pi5@6RFtZAuUM|6SAmr9D@_T~cKyi9ccIdqOV(_+7_q`0!Q~}bIJ)p&& zW{@X%7USX^sK)VIDH$%xZw&JAFK)XGZ*H5^hV7)=SIL`3%j>^td5j9#)xL!K>sfi& z?cYH2ZOjQlvHR&piRSs_6lh@}Fy1D3bWyLXRg>DSOkm@f2&XQ#-T~XVg*Xa+Hzzm> z(gA&X*`GJTi-N~5ukS-Mho#wx7!m1QlKQ3LjFDcuw^Q0VZ0*zsb4BrpU(-i{iRjxZ z4wO`zbg%Kr_q%?k8tX1bhjnJ%E;{f`!2~Od6BuwtlWYrt-E_9gK&;Y|FbP3`P{}?M z?*aFreO^3N5_5SLsoPEJFHiDa>%XbLV$8Z*TJ?HoymC7LVZcg7WTsE-x}QtvjkteE z)emmI$xS`a4?+LBe*!!~@gDlt&DDD1dMDe?TRB)09>_d7wn*<dC0fye!heL7ulcWI zKU_5sfRM$$_`k_#%DP<S;JYK<Wz9R+Kz}%B9SdMRH8)J83F6o@&<5L!DZk&~<vK8> z>B%%mKS|5ch9vpQtJwXuLJjOM2Z}vQpox06_V}qN{w1Hf;cu>$RMe=8G?PF*FVnZ< zlGv3(nC%)xH<bOcDiF+&Rw8cKg%~p9S}k`jR;7p>(B;wJMqlj{ebX1v|JYhFlX+7n zbOM7NWBYsG`uS@hqD#v^z^BId-Y#pPr(%W@#^g(|t?qMl-|B&F%?8!`c&j(aaz0d{ zGRmQ$2!<3KgmgVe;%z+tR>_L5{q2jsae_f=KcLhRe{PNxD2qyj1QLQAg#pu3`yOas zD@2DAgAQrzZLUC)(Avl_%KNLYno*aAk#w*|2=AMjyPsokxx--ms^V$9V1_pjI3=1Y z#8SZ|$E_JsT`3M5xPrvD%0an8oi56j=9s90h3n8&sNajoTxSRe2822S-r=;hF%2DM ze8e+Kre}(!T_RZ$(U4rL|I%ZzEV<ybD6n8mg>~EFNNeM@N8t6~7*%c>!R!d8lVXBl zVJWn=l4EWf;4AzSakR{LSO?S*SHc4=Xh6ACdK~c8lySDg_f`pkFa*>HU#k^?Mk*9{ za)hMXOej0CYjHfP@rr~g=bzpZWd>K)z(RWS24$;J{WoGXRRr;k!7#8hjdn`O-U8}5 zo6@7Qu$vlPAwxkd&&~X!a5-rWMK9dA?DB9=jmEx5D3{D5oiT{fXLI@`D=Ux#grhuG zD^+!nEA~NcC)v7i@}e#|#_(<SKfN^V?Nqw6<=Ko~bt(b*kFZ~V2`EvzX99Vt1LU)0 z8dV5?CtUs2eCO-#s@IayU9^h?96{`>t9O%4YG-k=tCW>)%JiM~S<pu5216HzDrY@I zVIi274Ef2+Z_U*1@GUdb)XYEx;;N&wAVt`Pt6?)m36=L{+gB#2qtBs_aLxItL#?<% zLPwra3|bt}mb|NFOT}gLVOP^=p*dtKlY;TQ?B+RXrf7~l^vn|BU&aE|0_e*lA#k|& zqAG$CnSOnh0_+B4{&Fe%G97!&p{9rYRm;{cpBuaAI$=V$DqOlnG^T}HSFeoomI<K6 zutnc6(M>cnO!i>TNad-?#I#}>v((J!f2=gHwtwVc_EHLQC){JFeq7&ps>W$Ag5{AA z5%-n%)m`Uk9s6B0JIB6kaJrH3z;!O?qLioid$n=1i4lrqDOhOBjy_{)&~}-)5yfq~ zDifYQW_zyMSN{T4L=Pc#ME$CI0va)*OlfjUkgHml<^y$ie%U+w2tv?6msX5G3P$2| z#}ZAU`GSWiS?V@OD{M@e!KF@7;%AG)l_V?oK94RRx+$P-W{4>of3`BKkt$%=Cw)rH zdIYbw;3}9c=gIK<(6$4kYGoOTejN0P^d6Erc!4g3XYGD<S47&V@y@wU1I^{;Q@tOs zuJbV$*AKkl+{VeBXHsr;S(y#ge+$>qwO^ERSQsi+-!=}GN!)X>w*ji{P1H>wZ{UH6 zX{an&UKRFSLBQ>AVwy2F&Q`XK_T!efPgBi&dArxpzkCbg)<c?_6Vt6~$?FJL+$1Nu zAEno6m?L-)BqrhzOB(Iw7`nSpT`vgS^7OBnMSo*7Be5Xpl7fxQ#YE&!W8|7v71C(% zI11-p>}*sMQ3d!ynYcWix<iFs8?LSg;__?O8*g+9R99;zJa*kAUgc{pNRO3EFr9Ji zp~rpa9SgBeMGKVz@el82`r~0}o;V7)f-_ANEgy|u24IATY7Q;~d#NHJP25yJ*jkc> z_|npYGkjM4H<CR+;CT6#0g4-M3=V8t+spfw04{SpDIC|C5%0CDG+oK9N?N{+9jX@P zJZuEs+m2GH%21PRPkEWYkgoRZei3u)Lf8tGj=<#^xi=39mNkq-KJ|U!6^I@kqn44Y zB$La_D?=r)S7-$ZLymm-p76MIrIL94Z>_VCfl1lDfoX0C$VNv<hS_7hgp^D;*9i$K zM<`WeT+u~dZ74KV6ArZtsBW<6r(?(MbAiYZ17@2)b~Q;4p=3%U8vA#`vdJ)jM}#uL zM-Xe2fbq@WXZiyT=6cl>A=MKO()qiafz$U5Uzd^r!`sw6gjbZ`=$i^_!5*E*mpvGd zg5%DuZ3wI<OTO<g!7y^84S?CO)|aEm5S|*=FdD}VWx@aouYLT>xm4a&5e0xsqmgD* zYGLt_w3+$h0%!yaVq;0um3t$XEA$yK5Pw|pv!C9zSh@wc?lNT5)5EG6KfIzyluy3k zUv3{ba}*4FG$(pmR^n<ZEP!kW3WQJZqDlkG?NtIUqUwGXE4bXc@V>Cj0s#eCNQ4~D zqf!&>E;YJNTW#siz8Z?A8ZLGxgC714l~`@O#>4Wd5=#=oawdMM<77yT(2db7k@4Wp zE%_OM$dm`us47x}?QgqM7)?HZM=$E)8)}u<x^Y$6HrJ5(2V<fSMCLXhsWZC@-HG}e zB+IkGfME{s{h!15{W%6dL4<5spr?7HnJmn1DJznX^?@}|T=)Bzj80+7Ve4=Z+8c#z z{U6(Q6W!*0Z!mIs+I>-P|8J5me;Vs-QgJLa01hjt`<yJGW|Cuf;MFmh@HbqDkUlhN zO#GmsU3hWy4AG)FJbCP_fx{c<epWyFZf)|EF)Y~>-GZf4WXYs8)21~d#k7r)eGs%T zoTM@mjdY}?b}Wv#jHbE*Kz`zf{tRkAt>Qc*%XqotdNs+gjp4Eba2n*ly|eRwCt$ys zh~nX>+L&#zD&EyQzPT7a-T4FSO1;b<&IKtjfrbAlppEY|+K)W=f(08x4LSchxPcZ; z&=#FTV)*|ywEy4&Mhf@OGx`^f5+SB<?;47n4lG~U<~vN%(2=v3Z=f<|s<ee`h-%TY zwYzuFoO!dh!;i3q!^fLvZ}9);5Oi{<jq6tlnd4Z#MGm5u$yY0vFXC4zI<aO>VpmLE zI=62U*W>|>NHHU*R5SE{tCw-<<`9FC;fk<jlhS3H@cy3_JpKHG;Vtx~i(=e~v#=l+ z>J1!6_8;hau))x%lmF$sfp7&pD(kD96H)c$S<Rp@m&`Qxjkvq^yeBs>xIVbZT_~A3 zq=}nfv}2Lwr=d1$v7i?b+##9FLkXQFg^h;+o~eoUixID_yyG_rQYZ@APz*{54#pA0 zKa>pR#RSC`{ME;>CYUt;d;KKSEM)0R4s_P8I^L$4pB(rX9NTKK(#8fN{R*CJBK6fj zg$x42U%7H@19J?CBoA$x)b)Wp621#55p_mM7E4!7(moooafA6ECF-Zt^1qol{;FtA zId&y37DAx8Lw|yrU@Kx3nm!Z4dtT`gHi}vb$}j&kSBP&eGZ2SUb=dNsnEsur&WEKT z)j_QnLZ)5KOXZBcM8xs9Gw{W^CwZ=9$>@IzmDQpcEd(2W&^0pw4EE)QCw7R^@bLL; z`;jKBD-xYQQ2yd6a!O3cQ1R6Y?8$v6opn%hlyAYLdyZByBqP$wt`$?@3G?GqjI-WI zFr(&N%W-LTiVx^1Ho9CEPW9Z5AOL?Gi|-iXg08;`9bHFOX<@)jh53F(ufGo7X8;-H z0l)YvMmC@|H(*Hq)5~Lc+wpVu7B-~+C=Jcxyn+Svys26)m~PyI-+W15v=_={`XO5l zHTRU5<6Q%(;GtU{_)M$_Z@txr^r;MoqLKj!*lxsJ-o*}P>e`FX{w*=TWA)e>mkquq zR>aObeoL>tvlW0b{B)@!*Q#MRNDVE1iwYTY0jEF7nOpwz-CzpVB)}t%DHnxnklM&j z{5nE-m_I0{MuyF@X{w^ZXId;$ZzxX3PofMm&=br2L2ZV2EG&HUL-^jmzMYczD$O`Z z?tN3awcrjqUCwXxK5<+SI?>|?PR!D$t||ghxxLKVr-Z6Dw@24}CgX^Pq}kM_7!5qg z%Z*9SS}A#;Gxrf6Yzc??{fJaAfRlxa)hoqd(HC=<gf=AkIIcoG=V=;RbmAv%u6WD> z7O1`LmWceuZ0Io0(jzpSr>;rS>W?x`vcp>fVVJl1r4thU;2&FV>(dCwX&XK8S-%w< z9R&H4wYnRLSj%_btvh@R$#$Oo0`rfNf}|CtyFYe$!fDRQ{TCn#B2oP}ys`rt2n8pY zPr*hy=n`c2!FY)-Q6avwsaI|ld#8}B@=2^@?xy>AgA!eO(n7ie<cU_hF>tiyp6B?7 zzEjdImQZsbH{m6+$_l~!C_p?uVA-?$aetr2!i(>2oJ8*9svS$rL?LjaYe}8@!`*TQ zq#ig1wLj@;6j;-piPNt2DLzE!!*!-C3&;{_h7O&)YC#HO4{G<&N_9zob7B%}yt1NC zn%`Mm`%Yl-g?yhDxiV;rXh^>0f5my?!*A)t)TMO`3`(N+D9}1!YxNnLK)>@{8hpI5 zD`Qq^)g>Q(N6@}yx=%cj9sNvX@vp)=nn6ncK;7JEiZgd^P2j%)6VR%zgBZHuTvAw6 z>wG|E*}P>alW<G|H7&0;9<g5Vr^gDmfE3Fg=7@aKIbuF43dwp@=y(kTDgBZ%Pqv%J zmL}QvQPYNSvg8Q6{wQM*S<!A(DC;E9*&QFnW#(IAz$GzO`@LGoLA)<oS})@EqEeuq zHqi%hX#I^UuM%N+Zw)@%fb;zEo^NS^-IC6L_QtTum|kb8?t1BO3u?WggTD=H*Hg!K zOq|+1q2IrV<8&Uq3Na{oYS&^LX~?5OYPM`F;iC8YydFNjdeS0*-;cb97uH$j_0M=e zEk7HGdfHtb3r<(Z;ab(6S!=K{ip6oEApOMW&w1ZM1%-^u(Gpd0aaZxmzoAH~OoFkz z2dEC;Udcn&{+5uSRz>tK8B}_gAdu^xWy(?U(@8_IgZ{Dg_YfH_<f1UEcKW6meAmTI zi9p`$NeZtJm3{28py!+Q{MaURhvQy(M+qsPF&td}a#6u9WmM*ELcr=sIzVHt(v>i| zcEU*ZONGosHYDv&Sy(wA_rub(!|ZW;oHgD9RV~OgubHzEy>?~?K2bePVezxt2%>;P z-?ra7<4n?x&FYaE?cEGI)-)$tD$5+muBu}U?sPHFKe+hV5?aCTUXV`J=9AHC=o-*Q zXUuT@<CqAmX`>-0>M!)m+!o+T(oHaeB!5lJUF^EcXIqSUNsvI7$4;|X#{w!e5pUJ_ zak1J+C*mxrK*L>l)}}XDmB5!T;U_ev;jCB9B2`6t)Wa`7=7pam>YPepUHy>E1}-i| zx=cTq2|P}#Ey5pcy4D8*2oic4dykynV%zxoUkQ#ZS%}$Wd?mL`_nI;G*Tm<wzvA`| zVZ|gWr+_}WUr`!wOp2bL2X}3(Q67=0G29#e8iZZ`e$1ZFrSZ8}FJ@~Cmgl)-;8=Cs zPo7;udw#i)U|?W~S|Z{CHX9(0_uJ-`M!Gnmdta5omY5^|_1DPcP?V5J?X0>EF^KJp z_vh{DE5H7`9RZOzA<X<ghmb2*Gk77~W+-6lgX@$0c2!%CSrh<fAAni)*&m*4sDGpH zTMHjXS^MvkoIn2)hhG$e-qd0EGWwK)oSdA#C|OmOPD1=21}Fa^3&>ku0+?DJ`Ocwh zS7jB5f%YHF1(sTSKSuTtezZh?ey859@nDV}*wx8We3^(^>c;D^k{15Qf0gLJdBw#% zK4AOfnWngIHTLC=dT)#w{3rZBSpE+*HU0+;Htp>`-fzW8*#W`aU5e&a;9&m+kS-Mo diff --git a/site/img/dcos-logo.png b/site/img/dcos-logo.png deleted file mode 100644 index a9a23e0e12313f11af4e94b3c2e236f55a4f7f59..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 17756 zcmWifbzD>58^=eF7z{~iMCtAvC80=2Be5?f-3=pz0U{+LC{iK<0@BhjV05c=ZxW*! zJ^IJ*k8{uK-q*da*E!FL=lQ(Pa}x~ov_ND`WB>pF^i2Dy;eGw}z6=0~@2_6Y+f4Tr zo}ZzX8lZBFW$XTi&`DKS6#%G7CBL>My1ys&*0%5i04TfvU-0_9emDRCPY<6xRW%N_ z*~<+wHd*+x4h;%8jl9tcnkg?cJ8dazL54}S^lz4#$z&Z%VZ8mMt^CGFSEUM~<w!=z zwDs`$IetVr5E2?EtqEpTVXVT~KP(6g$azm5%Wa^7k?%T`W?m`W_R)LMM@KTyua5>= zSk<o2{_5|TS!swB5@b)Q`+K_+OLQTO7x-whD3&P8={4k{@U}+C-L|=@N&awre0;ch zxBZ*m*!Xxx-EnQNy#*bI;E<vmW~*ZxS{Exc!cK;`xDZ|*-dS60v*Oj-+O<+?`t@8` zn0G)?3UkPQQni4j-$+a>UmqVI8}&PVWlA*@b3FPaI9Z+mxb(?*F!n-&F~!fLyGCe) zp}fEQVq-%-OsIS!Dm(q=d9z4q&+nZ45ZFdIGrYk$^7qBq=wQI1)hqH&lz&~_I)zGD z(Q9Rf#@-BHTN~9lkecN$f#|)paX>~Q=7W4xQTGf#9#X>_*?jS^L9Re(nYRfgtDrD5 zXYV5vHIz0{SG({7;~T=skz@4U$D!>1pL0^aNVyg%d95^q%0A`By2wkVgH5TCCq~jh zk3U~HU?ooLHoj0}ffA<;z@z9%%F$rqN^<`7-QDXXdu8QICnaXf*?;S@q87L0Ut8B2 z4C1|r@mxD{fM^&+e;xTjZq6YNda~<oo7ngDI5#UT@{flq>sLJ31YYXHb;+CNCIC`G z;z*oQw^r3Tky{nx_a!szk4FTn${=-#3t`{2yc`HT*6poWCpt=8AY?pyN^vUzLcEG9 z>2^}(17b{vwjN$=Nds4oQPt#|!d7`^g<WE&AtYN_vg4)iCnk;$So88u^+SE!4<WI+ zxt$mOxZ=RaurGSoFMwh^IhpydEm=F9x@^zaCU=i3{an}9#^<8msQ5=0E2itzq2Wfy zfAwBO;8H^m-4BIHsH)_22b)pimhoO>s+)?eJn%Z8ZN&U7`(F<|oN#}v7{l%HAm1Fu z`t{7WW4sjymH&w3As1OzIVOB15e;%^1?IrOuE;<;-!nOkh1BMWyBs8e)k{fUniM=n z2sb)1@&F_3eQcAiF{|l-l@X@@_cHbooZ+?cyNR{7$!;2t^3|Y6ZB9iuGlewyc>#GA ziF#w1_B4O~nEt$fFnF<*;cI7TQ#R4#&J6boW4SrplKqr*>I-{>x+2ZJrr+GuAMXjs z#GZz<Gc^QTt`6wyLzHCdb5g#^pLwM1yh!6xR$3@G>!$beW4Yb`^S4f!0GSYl?Joh# z&%Fz<b<JS8It<wyeAs)ZUKK60d?tH+lm5i11c1!<T_2#*FK{yD10iw@1l4X^ZTv0U z1blF<JUZig!k<=^HwM!VDkzW#H8@-rc0JCMJ>NOzz&crdMd?*w9QgLVKzv6SfBc`r z>92l9Wow(~a<<aA<rvwMrF2Ip?u=nE8u$ucwBBU%AY4So{ahy1!w+%(gceRS{O2&l zchgb@C-mRuf2`58u$8%-gZ88tsIpDOWeEd-!;CWgm=>Rw^+#PgElkFJ>^$Wu7V)7* zm4vK+^UlMB<@S<~15U#aOjfOdkLgXK6ciDs<x5dZ&Fnq0gXgykc{iq!!}QY+RKv)F zdUGB$uReazR`Or`ZD9GoPBVS8oXZ-k6j(MFNk@dMPwqvjA!TGbYXo-q<6S8Ox}F;K zS%P!dpf|C{@6im^NA3&R*9&^))tSI~qb7xI_`p>76W@Ty1Z)6y`J1#VP&70D5)T|f z0PG001M0P&W{@HpK&^PK$Nlkaf>o3XR(x;5$kxt$RGK;zdx;C^;L%j9YB=nxgetB> z8w1N+>~Oz8E&z?L@*}Kt4$QqRP@@dbVEsdnA%6rT;V(l{;vtrZ$|A^+_K#-HKFx9L zjsWMa+Y-ML9SCV+zS@7`4L)F2!%eNy>8<R5BQ6N>6<`0}Vn@NC>%UxPLH;AmLE+F4 z-Az>xVk1ECy`@$&I->}8z<*70!_%<&1Gs3ksm%afCcp*O@3H_74(qgGJkDlm?YDKN zy|*IlRlb*%ES?!$R~-O26!1-EGGcS`JY(o0F!j0TY4r{jPVKDYNA%SoUE!rFEFWyM zU>SHml!CdlyekzFWW}QlY4s{Z#{@Ez>IK&y9p>6iJ;Oz2=Xj69`vvqP$blV%@O<Sr zfC=W!$)F7H&Rt-?k4t0$&&@nGVBle-2Q<WF;FgTkd&KKy2|W@clR7zZXjCnKze^Tz z{jT|qo~CAnXa#oe6mj%;q277;BHQzoZ36JYY}b#p&g!X=;qTL<MR<@AxO7vh)(FZu z`>H)6*p)wyVtc~ydU>?_)Yn7bq86k^t{J6{SRL(VF-&~Yi3eICK9qrjs}=~+`-f3j zW>(}lz*J!U#N)~B>lOFJ<BQz^wjhsG%6#>Rx}y%8iw8bHy{6Wa$~$3Z`0G?ayc!9T zVblCEo<g!bN($+P&zgv)eRQip-mo>;z<&li3qDP#8eZ?jM0rPxd4rG`o{l%i;dunO zyoQL`rA(!b?fu%aoiG6dL<6!Up@R#q+N$HB=R5|SeYpwt0Upc7+a0cS#YL#28{cTh zxS;PmV!W?cr?ff*E0>hf&AUVnRex)$o;%(-$x)}o%%flnn=ldnsslWj!Vy$B$P&I1 z&rS$WDH`0D1}?DQw<}1K1preBz>RQ3v-lYF1ZgdBEQ&wc+N=@L$r#>ay<2<2dht!0 zrC+h>0B5osIT3j@k6p@?;U1{@xjny6cE|-IUiT>ftcQCO>h4}N_(?|TDojbh@|V<g z14gD3)5x0a#w{jPJoxMIt|BzBgGm|I)F|+=`Dnt;_dhkts4)JpJ|&mmZ%CW%l#eQ3 z>}?yL-9+wY72WNcbtiW3S2EC%A@-uGG~^bziWv~>dYcAc9Q=?nz<iGjh3<3vKh*@- zxk6Q;uL9cHAvgpvGD~SJD0}(FiVgL_3*!CWks#VZnUjSC*zbz<U`-QDhsfcxW;C}8 z5C3EvJ3h&0jvJqD`_q-Oo?#Yb^0q1*VS6!O1#7O+ueYt8xh#n+aq>u<iLDCnqHIV1 zmeT~ev59$XbhTaYCN^x&$s}GVwYLcG?&ZpKH1aIzMqIAFhh23EgOG>KzTRqY+#gcn zT39mjA&|!tt+_-|`?bHI?*(_)J%9z6X-r`Z3m~5hweXq`CrQ08I(aZOR{M0pd-0u0 zJTLCLk)d>K3rNHv68ONX^dX!)EX3bpT1GY~<gPqw2viwJ4bC(C8h(zp$y!+;_)0Xi zCfx7vjdCR?Z!CDlsJ+8i<%i(zo{W~hUhFt`XJg&Ze`TxWhjW=u=4{6T`pC2CLfyo4 zcRic{y8TI{)~rpZ6O@qgw>RtKsFEXxWr!s?E&RA8IO5m0m8r$4{J5cikciqJsi`CJ zb<{|gC~xt&d3MMm$zJb{y|{W8<`H>lSm}Yt){e;V718jvez!vnB_}<wn;__;39$kI zvgTq;uaj^QyI<PhXPUF=9)n%S&YAwl-aw*`mIfgxXXq(oHGDnHGGwbFV0Jb^2k`<| z@x1RdPe2Bd45x=ga5nMQc!?+l60rWTzw_&Ja`8sYIyd(4>I)Eky$`7kvbz?eIM+~B zfWAG?R9qD|Hy<%mb%uUG)sGU}s^dI`X4zVpu^vP)K!L=1YrR3`L0LxkgdrZ~8*w}! zT8h)^_b_>9bJwBO;f%>f*F!OV7%9ZnQ$~O8Xik@O-iFVt;nTbG*PTx9y|fZ8;`fo> z?$DgzkZ{D_vjdy#qh0kN+b3kWHkrCN@=%Cnwd|aY;8n@vp=&n6a#Far+Qj>|>c-Sd z9p3Cnt%){In_3sOrsor0mS~*||GcBeaImL<9(MM6Epp5H@HX~bq<LWujIac(a7!l9 z4LV71{PM!L^}-Kae==Z`dB#e#5{&F`W<yB~6#5YXS$BS)#Xe)$TUH0QM+`2|{{vzc ztRCBRtR*rq_0F$>fMUjj5TXJq4oSSToi8(^%vg;dlED1>X{vCJ8P@GyFP&%CVFc)* z41v3JOq|sf*m$v@FrRKXjD};8m?tNPH!<`AzZzgMe=sCaUtN;w&JDu}Ev^@x{M3)~ z=DD;pqK;dQ4q{hJg_1=AeZc<iX~CW&kpgR{I(Qr2Pik*=crAdy<jz9+^Rslin3x_9 zkCZQ_j75&QL+PR;-R;sF&4XS~4@!e16{R;klRDqc*sJFSBQ(f5$pXiFg0tc;z7kuZ zbYA@))-t7{>g&_M2#CJuXX-^XH59T%`q3<h0IYY9Zl}ZSRVVBBeqb-H4y{r(tL;Km zK64Fb^`1N0DvPLW2<cD&))u`uO}pz_TL<!<1@(Bjk;=<JZ<cwj-KeiRBmLxzO+n!P z?WGgX_^Uol<W_r)>~<L?r^4VX4V<3P(0hyavEigjhACkN<ZFtrsUzF_g6YZ-&Q|Va z(+(+B=VV{#CxM9Up=z*TIihW2&MEY~jooK@AAQg^!?4us3XPBZ&)z&ACPRRT-4bu_ z=gT^(Ngvr+FFwN)-ecw+z9f>OSy^>D4z<oKwbmQfXJ>n$r7|F?IE`U0UTOY|I=(9U z)fb>U#t_=r21VCicR|BEcvC2sO-1(I{5U0(G6sT-5@dx2-n%U64}DKf8Qx|4G+7{B z8EuJldZ29id%-9?Xo!bry`PHmUHARK8I=bAB!ClN_{S~&o<?+hRCk;b4wTE^Svrh} z8$QbEID^v=dsQ$rN=@Sx9EixnlyV&Y#rIE=0*}K&o%iD}@~L58svq<P;jC|8lf~Jo zZT`3q$SU_>z_~sI^Kg%o{j-Xzt8JLj>h8Drj$<n*3+pe9nBN{C<}yEc3U$eaKLmux zTEc^h1dZIto=C^1cJF_HjK<qX#ILM-;THJvoxy)e!vPHn9FjwgT>f*yW-w_mqSke0 z>fVoabsaff4$lzB+owuth{4~3)?deK?)(>$jdJrw=zN|CDO+u`au(cS_*oiREx4_L zi*~q(DzE*)(>19abKl~)j@$%E?s7B^%#maq)mnjfZwO06Zd}{Ur|Fi;h-FvY<CnKq zkr#m)N&1H~Sp@+JrCn2R681GJLjR>^TkjtzPSdZDE>Zx^FyH>ndb56=it-{rWq*1@ z6rV1Q*$H`;Yu?gSrTmrb<9gKCccP`6^-n&wzNfPi%>jH+%8|DjL`&D%;qe`4uPkB= zT$u(*g@!9@mbESJ9wiUk;Wd8~YM-G&)G}c*#ycjnuj<oi4Lt8oOs##t-6cvdnS7hz znr_7{{3@5Kr$Ro>TcmwP`twEWs>NcNR+_KuS+&4B(nVvQPiA;(owhC^RbKcj{l!G_ zB=mH|SiD~#d$qNVBK&9M6(2sj(S7hQVgdIJL{~;^Uw0IP#X-eI9J^{jG#m~NX?ggu zchOgaIkc*~y0wxPyWYTrAnf=XzS}wiuArj19-`c2w+QOF%nVOwG8QUTm3n}&EPDz* zuql0>80fWBH!dtVe#soviu^3lG5HP8&zS4gbDN!a>uXlXpyx4FC3UX`g{RG06<Jpv zBO7hHK8f&|2|QVDe8IOhME;C|_tVXUy<>z~9_bQ}xwz}s<Tj(r-E~6xyv8b4{rgZh zbJok|k#8At|LPlP`Agdui7^E3S8TC~ofp3&9k)g9@=@o=7S5zIeqxwZJsBkbZye>n z9g>1agHfW$<^a7Ps^+wQj$9m1)mthZV!oK=xg|28uQmy-wr5b}?@tTWAOEvMEP^VZ z=KgANIaTN3141QLns!aVk5c)pfoL>07+Fd&G{e>mMo2|@>AEBjzr#ScMla{zo(nHr zZ4#r731FeD7Yta^mHwTZOS1|AJEeCWZ-%Fe2&c7w%H&YRdh#QJ*aP#{?*7?)*MC{= zSr&{Bwy`l`WUj8!voc*%aW<D2wAZc4`8@IWqA5QW%h*YRNo)e(q5=6q$1QwlCH&7Q zDxMsAdtiYyOIT^!3-JCmCmFMv*w;Y`K}<HEmLgt$ytR+DB`Wclf9X@~AO0=|Z2hkK zb^d`9wVIazbBcZ2yAHi(Lz8S@B5X*8bF2(lxiykiZxfoqv@{euSlXL<JMCfe(!De8 zm5+@l8<2G6S*J{4ze~XE3%~p2yAE(9iV}0-L@lUb6!J-M{1iJ=#t8th`sZkGRWlo! zzn|$Rra2~auuf_?>Rd4R+lxwpd8z^lbe%x#kgKp@_+|2PQWZ#cg$;!8?GDZA8NX0c zY1Y4U!Ss2g;>$z#mZDq}@cyf@A>5n#7X;*N;2t5hB;jP>t`s<0y`FIc%Zq5KcN8q{ zp<s2)=+zAOwmf^jIBj7L)02*A4jLFbN<^+2Sq;&AjukLdXW+D&J$+s$nMHDJmp2mT z`pHN<O%E6TabDC4Q1G(ZvV`T(+?s#CJaI=@*rWZpH#lQI{4Jkhn|`xx_^Wp-tLLkV zItoFuh}D-TPZlY#Q`$*9sN<)2mYfkX$NZc>wLyL!OWrWg+tn4QjRoya2PIgT06zR1 z<v}Z7Obr>i*f{wuT)37%DAF}!oqZgePXsdq<f|%0==#44rm(&$_GQpjsr!}1aO|)w z*gdHa;LLdsu1yji#w$?0h}8Y`n|r@pbhnbfqP<$Ob!v=Z$D#p)N5Q$wjD0w{YVz-# z{lDjQ2PV^UsUMRAMQ-KiI$G#ID6l<=diP(w&9!bErLBViXPc<Vj+fOZhQ5o+(o7v4 zKMqNlz&e5qJ38d<5AnZN1N^rMgL@gjT?D|MMUDTsu6U^!FTsk8WlYK5__GO@hg&xt zGj|^bF~P@%k0^F*f3o;kS_rZU=(q_BhTPUKTUCKR`cP>V%63lH?pD0a`9#cV6SV8i z64Jam{CMSEi$ez!2tq}>37P~H=xYq=o=-11Urp~{-l1FCIF^SikzcuzddKASaG`l7 zT2F%I){&6;knlxlqaENYqqnt-ZK{ea^v9ws0S^D(bj=HoD9x<C?p{6Y_K~6Ne+3yy z52>RNBp>dnbLNilf69VLHDykEX`SRBqQRL{^%H;M<6)>oyR=>VP~e{nVA}nd3lO2D zR5oAEi|H}0n_^9Bp#i!N(0(Fs0EVrEz!*S*nZ}Su)wh{t=c~HNrL#!P3F5tv>lv4i z<;%wmpT5%Aa7OeUNFp(l_z*!#f9wwpd6;d<arNgPqP#oV36wFEByex!{+rPwhlgL( zaDHYgxWkuoq&#bJelC1D)f(+D>~5?w*U2@Be*Ud8gB?Z&MwZA-RB8Uynyb8;#o8+k zgN7GO#aN4-e1Kc91c*ufGl<lKJ!0IV0YHM6={X79hi$!7Mx<??)akV4<iIb?CPKb_ zCj=0n`KGtx?%zKa^~1VV2?+;o)auqtztHBgo!}5na<d`<yr^f1VOCw$<TyP%h;rx5 zx1>F~TzuB59PgLs0;5N-EzG$ylO5M?oVd5ZHhsk3|C9KUFM;~Lt4TcYF;%~;Ra$U! z9b(}n-2E0|RjVMuhz$WiVtf&$pV>H*5LWIFfn3%)9ok24j;RsD8!_XLi?i6Yd49Hp zEnGPASpB^l>JZssxQ2JrQ@QEh;+jTwiGWV{YO<ZN^ttM0vXDYjSj({#GN=MCs<-7h z79TS5Y>L>{xbKYjvX<C3DO!{se6*p#4U`j^<Zx$9LBrD_kZ-C@EP5SH)JXC~e_W<B zR`*g{!O`6Zs{EY}->lsZXq(he-v7dV6W+TR;1F7~zy1vs7+GUE{#Wsh29hCR*+eQ~ zS?U3i!R+Tu1KN)fL;?ciWGb_`#l$D!F}Ew*crtlEnf;l|wb)*w#;jP?0D<?c*T>rQ zyL{Zg9eWA!!#O#k6{{vP{ZAMRek*`;F59>7C}ukD0zG+{(2#?tOSetdZZ1Bsl}PGi zjHBJ0xn^bn<@iriFk5==il%+c-<?KqqDYk1TyopD?wR0PL9e)9fhZx01J3l54I`ia ziWglBtGCE%s|`a6j<t_uD*J!xJfVwiAMc2)p2dt$`C_wr6!-Cv=Be*m&%Fkr;WrA6 z%0C}69tPi_6r7Fq7R8+Y2H8661*@qByN4Dx35mp6(hmP|VS*_V#GZS)Q0A9*Go#6U zhI-XDHR7lARe}8mV16EVK_ereMc^{PMLgbKWuV1pwdI$2yuPyIPg}DO`@4YzTn<9g z)4fX35&yA2($`X>UN!YRlot=*NPqKZcWk}(Os;?wL0ILQ;Lm585Cs2hR;G-hDGHKF zA?xTU8z1G)os{*N$uVdJzGVRIUGTj;yZ-#u=J;y|DI_oBNF1i>I>z8DZYK2Ln^n@V z9W`eaV-rVwMSD7_tr5>8heKAj+5nBEOk{*QrS7xzjP9E$bT={XH$4+9c?{8V2eMd{ zs2&O$Pw#Z-knjBcCg3R~N<aBEqgc?EY`u^tRPa)9tniz37Y$a9078!-=v7P)-&u<@ zO8ChhqMmuu#$Qeh-~OBXO+=oQ^(*U3*+P{y7wkl9o(Thq(Dq(=N>^o1&;|3v!KV{o zCdeN_)BNCo2qQ7j!S$U3?gmEok|FypYeu#o6HmSgK<}Z<Wu^}?`g&bA)&Smfdsk+& ziOY)3FxEY~$DgK`XQUm>WYbk|GflbRjvg_h|Lkrtq--Wr+_*=%q)#!tlu=d+nF7q3 zal?iLzzn04v&;}Bk@D5@#nRy^j;c{q%}mBkmCQvl<;s9)a|Fv6<^m}?o<h6v+8aaz z@2V|}w70MboL04V`a5hrl^n2Efu=u=tLwl=6tzdMw^_IJg!!%VON}%Z((b+DKF0?X z_+=HQ%v5RhgvjQHIqJ4))h{9lal)Jhdi1KzfyVLG3>??A$+~~|L@{EyF16-M+`6rv zpUNF6pOx94+*5#3xs?C_s}7FhXq(~>AMiN=yjYw@O==xg&>F>kIx#kDyF>>Iy)iwR zKwIWkN-y#{-79OTmcTp_7@6Q@EqHq>XWP?!J^pX8tZa1&Z>84`U~b3S@;0GWtJ#VO zzC#Ch4sQQbnvriuRC+LEE*A9<lgz444H1^DWF1v~C1U&~E+in%fS$6~KS?Paz-iUu zGH3BA?y^+g^?0s$#o~|h)66r<Ous07538tg*X_5yop?`1^$jY;Nmv^<sC^1H>w-HO zFs4F{!T<>U@=(vs3$xVUq+I^M?pJSf{~Wc8THw#DDSi+hl1%#BK<v}>uFbeiUAx4~ zf`P_+bOri{ikWdc*pm*`-KQ}l_PnExZrw`Q1);{9P1WBx#*m!qQ}H9gulx_HHHzK5 zz*E)g9|=1#yb$SVk-y{GVKz)DmXI&B6Al^+QhzhNA`UA2RiP@#94`<^&-vI`^VGF0 z<FkbD9tdYzTpsS7#-LNtkRoU{StS1K6c5f2z>_3vHm@*UUn{@wTfe9X6>J<^O*9pb zrVZxMng+FV3e=p^NOJk;{R%YyOvpOa<z>TI?3nz|TLTz$$0#s-nml4&P}_DOCD(eL zH{Ug~!dT?w@vhBWm~Z(rO`gCq@LV7OL1Nh!vcDIlzh#xs6J|2h<wELuKC-Rv#mM_& zZ%08?NbjGLz)IQW($gE+kD(rs-%M}&PFMRF9=XNEJ@1*bt4E1GAC~WZ*<idMnsB|b zk@bzQ^y}eStoYP3uNc`Ev2t_+eKj(o6i|s>mP!tWbFz=8_LYOlMnn&iFXCUmn&u6B zz<M>X_>`hs^#|3`{<{2;*n>8jnD`rYSO-#6$Zg76KkcnV@0)p1Sn;eFk=G&wTHH{a zZOZSVGlRa`EepLR4l-g40GlN&YRQ%-mjBH6*6_>`Ha@0ut7O**O7#in<jH(IaLs13 z-7{rEkOb(VB?y4dPMFbCVQEE3R5r<gpufMQsSyL@j?^;xQXyr$?kk)R+5gk#&MGVX z`JC5ivdEW|2<FyzG59N0xGda$6u|P8ys9}eY20mf^%wU|eEMwiab~p<yn4}fr@!Dl zWtd#9K~}Rk5@xeRk*@H?OoqsyDj&c4E?|mS0^*Y?*DmsIW{kv_1k6<I1yW;8SCXAa zyamd+Vnq*_OA><)$s5Oze0pxm)no;NFF!ZM<qCpZYart5F%#QhmedD7`wZM<M6FCu zNYjg!Y98%Um)R_oQnr)@E*=>y=|{>_0^9F5zk)A>mL@k}vz<txvC#*ZRK3B){ZCWa zA<$q-=V`w`K|o>V`O{owkXZn<GKDZ?a58m#Xm=R{K&xk@ykyI*a0se32^CmV8!Ttv zVD%~P5gP<0evoi{kfOKkpUWR--F=})JFs1|EPl~#gyf+S+QL*Vj(c)+s+YK5DH?jV zDw)k+T0CA}O?gGxo$NGreZt-^$)iXLOyi!A9;u6omF{P$S4$Com&V5~SGsU5yuVW} z{dI~!y=E5J?J8>wJ6rq&M=8o7M4x5)lfM)mDCATW7w>3ny}mwlAu7P6#Pcf^w#n-_ zko5&l{Jwal*|YV*nh9R?iYMUPqwyUqUd|wUj$!{(Qub8hpTsbo7AT2#<JwEZEbUxg zr6iFIp;fQaaZ`L`(M=WaCzKpq`KFW<uD!r}zt_&c`X<u;zR(-N=Z%<K%B6!TUKa<- z6Cj?<eva0Md}wh^yL^>ad;LnTS@Icuz@NarI-yx^Cb*M>>A^7*{1c!+HIKF8`>?rp zj&S?l1QYIs&#LjE@J&2p!+Qgg#E@lfSpOCbA`fGehkZT=wc&NDIqbj}K4h44OJzJN zYOa3{(*??;c2pn=Reyj?6hGfhM0(4f5zof*r;e57M<09}ya48Jz!dAB?>TV8X5Rv# zLfcQV&kZ_;K7@jhPX=o@Y1j^?=^uZg3H?tNLcl`^U%=!1>tw@ft!0$w@OCVyduz@s z;MXg5Tb{%qo(D)HCOB(xdenhcGhDJW=#|v&hX}luV;8K4SeF^6;fKO5twX7mFh(eM zcE8jc@Np4QxJ?{(x>-D<#MVOaJJ#sQ=NajEua`QwP^<e;+M-O@@gjf>Vsyj*Zl_4H z%2ZFVn8uegP(274^%@-NK5lryn9=q3xxa}~vt<QSAe7FQ=iRv|C(%8iL*k#z+5uK- z)ap?NWAO#OKtw!gk>QP6%(%7`Zda8h#~}W_pPGDuG><!6bWN6{2bmbcoH4ja7K9ID z0IVR*x*I=YS!YlJqXaPGQzJ0GHV9eVT|>+nZ@&&dqcVxmbgY5TrX&cz%UbYSg9&l5 z=zWeN?9P4uWcO*BMgVzY{*G4j{OMFqYIh}~wU-hrG==6epIU#^J11p$M0O|<fr)T8 z0@fH7)0~}|g7u8c?b!j)uk<#Wj^ZV=s%aQ8j3uLhQIoR)_JR*LxzB}+_a$Z{fXzk$ zXUn$-&Y`qZ0*v~C&p-6G(9~AUz8)e~{(CQ{#i51bS(b-m@T%QF6dDs*$~Adwp!XKu zt)Hnq>*L^nM;-N^D;YbwvsZc##pkDxOM;etTKu@OA;5}X1x<trPLGZQ_LU=v5I#Tm zSm%nwR`#q9*+X<6B*uVID~cc>Xy`TNk}Mu>j2g1<$_cC7e%`RYI8%a}kgn$F)N&FT zof`5o7qQmBwY9ycv?x>`nDS|Il^7*eniBpZ2lr;?czUbpDDmlmFsXFyhk?%Shxb~5 zwxFN<yiX#A=7~8AZI*sNfL;Ff3+2j3s__BEby!<O10T#A5@l>chRZ7;n3b!bM(F%^ zu!0A-=Q4Taua-AChq~9H{?N==M{&U&W-{jLMF=^Lz6T%DWJmzx#!R5d70GZ0rY&D= zttpJ*m+ibijH;M9CsT-PK+K$NUpXA)l7?n&(vpECwF}f6{)kNKz&3yVYYkSlzz+`z zTQhg!kC{;urmg-E%GPA;QYv4@^Ui+aNSedOEHdOmGPrE2rqLHk46Nbw9&<CEmU%gQ z$c?p~&2<E@myp~m`uV0>H_-5%d<KD<{G)T4FF6LF{?Qz4bd%Q<r{ujXeD?yLFPMC? z;uI&hvh!%o8I5QnkB$@Ssy3Mv_OS`sdmk9Q=Y))p`$sgt7&=3d^xjbkDT+xn-Z!zp z$K`#iS>PnHHBMkk`Si$Re`MYIV5+`J$=dM&hn2*bS+m&iYvYXu-V;W86Q?0Mn31v^ zmtUxrX{%0?IVvYGAkzImCAIY-w4h1DKTOp4*To7GIn!bIVGq*5V$npw4%y(4ExftP zHvxh0_+42UgO}xxk<}v}bQVD6mw{K?dAl8dscT?X%BF2H8*M*`rTHh_0=|>N?4@wQ ztD3^#VBE8=85*qZnAXfASggM5cX0BeFej)ugSu#xlvFmz*{|P=0M4E>$)RX9Q1L2b zs+zJYE&AZs28<g=zjGzFwF|<s;=y=n&=P`vmhelTPs!2VFKY$2`qVS1y~Dy~jJ-s% z&IaYO3OPm$*TW_*V_En<Dqyz7m@@6g{+|zGLS4;s<v${)YWq&iovz*{9&aT<6O$b+ zmljRcZ5wl(g(Bdk>y?MSrk!|^3a7Et8)yUC7Sgmxs=XDj{V7;yYi+~XFgR-DkC0`B zP_$lB=M-a!SR#H589v2P3|$oS50@K7rizHFGFGboIEZ@$|3dS+AV-8^2P?0UTbhO< zAaV(DIZbw|_cwlVfx@xdVtjZQR!r!Ca`?cxXBEGbX&ik|IdaFL*~bN{k{EIiVs`6o zjWq49Ri?zZE+s1)-Yida=-@IG947?u*s>AL0I>%P|B8){ZG(KebENUHE&&v>AD__i zs{g5b)qPZNR@59|_+Rw<?iydU2~S>?x9DKK6`g+#L|}P5wV0N+=hQ#)G1EM?-cedz zL3e@v(J31JJ=e<-kyf+)=eO5+%g7RKHcQ4Tey}eim`x22THy9Oc{S=Fig;z5J#PwC zcwAsRP0^B&(#)BS2Mv9U5+H=7CNa+ls?Ec(%M}miB&DCPSEvTlfiL-IzMQ;C|M~Dh zWQLO7Zj7AA{bfm}9dosxB&Pd&hr$L-2xa*Wc8$lrV|GG|wJ1x`Zj>Ye|2sXr@JEV% zL>IfdHC$Xn{E^4v4B!?5wakSddP2Pi&0ZnjBV_8xso04ph4=i5rWy%7%><c)PgC89 zmtXJ*!ksi56V2yx4MA)^h70dzE#MVdvv2yJKMC%jT+xz;`9toPl0{0&Ty)4o>n#n{ z_BAIHw_JNwF{+Iz3bmgv>g71~*Z*?DX>63fPN7Xs?U(L%&*8KSUDVD3@<J4IqWt;E zdQ_^q!!6hFblR!FV5K^Qo98iHH>a8u?wQU0yeGkT^>QnKq8|efuu*oBN~)s@`Al<T z)><3sT%NCSq*>!@Fv~7Ykuo+bF8OSf=a)63jd<OThTx(8pqQF-nF>v5ioz>=+>xUY zDw_-^ONc#pe_J0F5;Be@#x*It0nMb@<(+%_>M<5c6XwM<;)9Q1YRmbK;v7~AW6)-q z9i+TeIgdP0Kvj+mSG<K4iaqv`0hZ7O-be+RVb^DLIIRYMuw)uO3#_;dgGGE0jHU67 z!D#3fLvkpAm@RNE+svzJsyocpe)c^=JuqgsTSoX`S?6RQHxT>j4y!oJtW9^gGqPkO zJc8e~96u2FdRBN!Zg}<QbwKO)GauGm_EcNbnD@C)W50DrpREgEvM>M0bz&+a`nevo zWqc-ve!O?$dduCey3W;<X?g6&dWpQ#oD^oQqQQD>`5z$nhq%PRt$zhtl0aP!UsqLJ zv7hhC5~$mAwem5t|Kys=!8Z0)Ra_5c1$*pu)6m#>*$4aM4~{M~%<^z^*jBjgO*(nh zZwq0s&>#94Ojqykg1BI~oa&>iSM08Z3|X_1kJ-R|lND_K6K(qcz8}BY4_k;UAv^ff z=xN|U09UkLG=7t19ROtL(P@gV2C|y)tA?Rro3cnJe%Z*9efttrqUpX-CJBzsIeCgt zZCOy=RhEN=rb6}Y6ju=AG4;<;jzz8eoV2H`-J~mu^6YW7Y`W;y(yp1+INTlyOgmfM zwq%O>g5JI>{YoXi`rZRg{iKrO@uzRpwRpwO_TSCW)SV@=e<cgWMdJ_y_GJ2E(Ed6q zi^EEe<6piEMT(0zM2IW>h2DTQqdfdGjo!`LFP1EZ#Hs%h2zQ5itC@cT*%2i<UhgT{ z7dLDOQNtBl-OkCAS#}SZ?&tK`U(Pjq3rkts&V4IE`R8lc9P_MYqA@2Rw27GA^!tT{ zJS_Le;?<Pv)M%(HIIg_bRbee#eY5r!Qq1i5uNM7g<Y4EUbaip~z)BWHDIZM-011uj znQB-dmD}}XVG8nE0LrfX_ZT9({iLGz$_+rnv%eVIMQ~l$Vz09msArmdl+hcb=%D#2 zaXUO>?0|ZWuKm%+<03rBzVLtD=IQ3elgupl^p3ZD=_^nrSU0y>yzx`^D+fWgHQ{p2 z%d%uoPyHZ^zwB5aZMCpjP}7rPK-CIDkgx0cN9@D(2@C$7Z3PS@7h5T~M+uyZzx3oi zo1#d1?H8($xWpKlHPLS&Bpq_~pAq}f)ovIU_rQ>dIv)Brj;XbaI)CS$08MvYh~||n zq-q<HR%#)uGNm8a;SZA#misLPFZGR7j)^m;`{=D*K$M{}xQ3rk<2{LB{#k!nWTD(l zy=1+$MR1pHF`3U&Q3eVtdSQe+KRt!T%gfK&()x&|ge;EvHtM6sKVfZ&zMO-x55KQu z4ZR`D?<74upIvONr2tExa56wQ)t8AqUw_isuG$jEySGryc(TRsr>~_!I-TAu&=`mp zGn@(L;{rbjc$n2gF^5w`4C?)>(b~T=lp&fqzn#bnX*42YUyRL7PcM*)Ga+-de`utZ zC}#3DGlTc)mCAq#J4VH7y*6V08BvZY>R5kkKgz3$7`h+Bw0h4Ien8QyuLj4WXz-Ei zO2KX`eUP<)sh?bTu8p@(V2!VAwYgoucG{I)cDoX+RZuDrVm8d<O;u@AfcLsq_UjVf z*1kJZ_V_1(j}?Bcg|PBhN<~X0IyzV!#C4KtX_d=+v-9SDb-6L_|E+qtM4PW<VeW&R zJ0L*NZ&_s5PL$X1lfbu3d}+az+vTG>HrwixSqF`5kn~56$G-{Z6a@?B_a|m3`n^jd z|K=Rz-L7s*4@;8#W_c-4{Uzt$jib3g?wjAx(NmLaj|aoMdpNLwy+7QhzrM_dTchCE zbFh;ApXeOMGn1*?|EI~~e@$r>4}&eIDIN0bf39X9e&r(raF@w({*{8b_fPHjaUPN( z&KzG077tG33wvt6^b@E*T`E^wudw3f8h-V~S_CswTxKnyxmQQ|YDz{;TEFQ@>I3MH zg1w6c17)&xhy2CB@wAK^Q3e+0tXm#<cH9$Ff}@S@!*^Y_ciNbRWlD_1-#G57scSEl z&w~4`9A+s_+Jh3KM03xk1v6{bEZPZKMn4vX2N!+?T2KWAH%Fwo&i*rce>U32H~yFZ z$sH%h3kkg$SNIW~a>b|b@A2nEH(Gvf(v)r6m6=V*H$bDjogbOBd;S(0CEX)_dPkYw zGr7n0*W@)b@k-6x$!nlrLafrw6rTH?y$cik9B{9h?k6yvGToul2kyGoroj(3Z=!aK z@v9wgrgs3oMY4PPWB)|=S`_t(;8fdd7G*w{(;U+xhoN~t8MnEPTW~+r-xJ=yJW}F3 zCoV_|<~H$Q^mW2|VK+ScU#3oyh(yl8{^dl+H#B|wvNWZeOZF5v2N?hJXZNw&gW>Ne zg&h6~Wdz4@OIVo}4|d?q{1=j?v<xLKM~$G7^h-BOS@hZ{ys0kmZ%1xMu2I#aisvJw zZqVl|OvWeQZ5{wt{&Mb%uW~WCE2?gC2{S^MJ4ec$*WoxzQh<HnJ`)HKk0Ky<y#f1! zP2VX>I6Q+Y*WV@0@ZzYEJW*iF!k1hDad{<pkdK2H?Xr1;*Fk>ij`wPwq(s3x#bHL| zxs{C5P=<PHZ-iRT&+YC6`QnG>Qob`3A_+M-uM#Ps5da;o^4Ts2>huFEhHA~^(TQss zoTL&N{1*|i{gb|OW6n%7+*x`{_-%3nqrghf9m3Xmq=L0jJh{VN)929;7g<}nQjojr zeRjivtsCu>sQ#a4xus5}Q-&wz%z+jSb4$ODFN`sOYN>q9p(D1&wRyhTLbg<+A%_K= zxy4+z7Kvx+f$!bJpgsX$iM5Ff1|OMu5fDelo<O`}ko1)Oh%+AH+QA<5t1dO^a?I+{ zqGcS9pO|62%WUqgD_fJ#@7#{g2NxS-FV7E)7|EZ<7=TQ?)odP0mzS4sJ9iXDfm!uG z1&22J^?;%rv2r=Y7zGhO6il(9oE6<9<VFXR!DDZrSK8jG@6b0IcHEIwnI&H_(qk-( z(Ba8B@p6&`rfGR{bGWB@X_ZA`t*7Fa?p;`l<}P`FRU;mx5Fru24?whfcdQKss2BG0 z;$GCa8w^0*nV2FI!Ja*zgr3yXZGv6Rux?;iKQ)!N#Gzjdh4j;tnSVc0vl)NSliSru z3S)Ur`hdZ}O*#25$`8LFbaF=O-QnXRYBdkpjc5604yqy`wRo2q;|RHj^+(^ic1XlJ zq6LwkM0Db%ktMlRZ(}nk^Izs?X$3iy6GNx^RE1CyM5ZbodqAKC<HYye_+a|;(;1HI zr+!E^$;pY<sj@%vY9ooc>X_B&RY#mZW&&NfGc|<}>`R}lg~odGa5&`2ZAFO=HYsKr zbg8EWFI^PNboOh|p%YER?H0{8-jgD}m({C#rcU`tP=b&>mI_*5IFjn$IuHJo9VcBg zodu#p4oTpI(V%nL(g&=L55+uMpk3Xe>o7%5KRuZsKsOCJl2(Nq(;8|RPvvQ5W_0&l zjBQ0qb=J=I@32utN^|1IGv6YIk{R<~5wsKIkp?1%T~S^?NKmUcB=y;0wR>|UyX1us z@l-onm}PvCaV0a0oL>L#O!xl8?8>dUB>HSDjf&!dWC;+>8Xmy%mk>!lL`oXx+Vgby zE#=2lMHr>am<$8fM<(q)Y-r8b96s7_T-zq4ft?y!g7AJpY1CC^>C5ejY`;_5a^x7q zyw}r~t(~|BT5}^sZC6nuy!Ru&_YDW0+Heok`I1Boedo@WY1QtD&7JOYB5M_^=~z;f zB-L}a0P5-u+_-JW$;-nP`3zOxR?IvA^W)w(MC%73?xW#yB5Xycs|=2iI}RwHA6=DU zr>Hm0xx2)owtQpzw_^wVO^^&pxay0jd2h3PBi~sfxALnP8xVCY=F!9!x72Fl5jRe2 z#p=cM0lX#pJO;}^o?uB9WZ=tAJI?)DeRMz6;te`zEpN?U;B_HHt?-Nbi^MUMpFzy{ z8cgvx23sM~6=dZH89Q(-+g{*iMQV{D&1G#r%D+M?erCw^u4PJ*0_K-YGiILZW@JCZ z45vHl=A?QIOTV_P8%@4Mve?Zl#;}p6Y=hM8BE!edTV`byZ$)INblUbrw>clA)_tv> z;swe%k);)d6HG#ywyc4+Eo%j;CaZmo`ecyr>*YBI-$wTK@i>B$&WWw?-cz&@;5>DH z*KlJ^@PEiqt2kh9r!4#XhPRg$<5-H048dn-9|j};Zoq#o9DigQ6zBD`%*cKz;`1iT z>vM@bvg=;#*LuwLG4X1K|3}%2k1%e5PPG(N$sQL~bj1#C*gk7y)g(X`@Ln6-vm}qF z^NV!90k-!!Ld!;$r-eBFdr<#Pi`0Y+vJSKM$6XR2-pIpZ08stT5(TJ7vUY$*yD>rW z+1s=ts(vygXB0R@+{@<E0xi-BE;`n9T}2o1DSH{Xa*~>B!Ytj{pHWG4mFkW?0tw2O z-N(I5Rjc-S)y%#-RVa8G6I?SZTW9{di;y!_4UBx)LgTx$VOl9)3VPIZT`}GjroCgx z1=G#(dV-$Ft!j7`KmNiF8!=6Wg#2NG50QqXs;LE2g5!R)D7BIU?UQ#7pQEqj&bJ@$ zKZl@>KRje_{n6kCr&6!%MeCry6^5?<W*lBDv!BBPDf*v%ruo8Q%#iolnmMh3*!FwN zK)e=huwrKAdYG)WZ*_)KugRdgWT;!u!I2CTY)l^3hTyc)?VAB5z2u=MWSc^*{<b*% zHnwZ$47QWq@9s9dM_6AAmU#V+^2Y>#q>~eCOx^b~T^F65Vryod8{c5hT-F;Gd!++v zjq=JZLP^AKz)An46xK44RS)7nLRsOQ%eD4LKn4nLLt=AR(kP@Ry~`sRk9Yp&yzxW$ z@8L6d_8S!kCzGY>RI-2X4ti?2=euueLiwNIW8#D5j(``@vv^1lZ%p7`%?8{EP_TMp zK!i?ztm14?k~yR|fwHX388=}MR5w{D?d6*P`y+Xjv$%q)Rca@`r(wS(^{mfSr76JN zkd{-D|6}uaRmMWoVs`_`>09vFV>p1Hr^3eLhH77B^X+S7*tH-t_JXCU1eHyKGsS-7 z@^6yA8CipA`C|xR8IFCZ&pH|4m~pW=d2TlFk)$lEnk-svgp}?+SvK?TnO(AJM`pm} zN?6`2o*KN6+GuR}TG_m|LbN(gcJ~z}LRWi%ZH&ak-hQ7E_d=CtQCv`L-p5}{<%b;S zfg_TO>>jB1P<4wnMpUTte;Hh(U<q`xn|%V?hRLu(<X8ARgVw7zU^yk!h?EDbwmGE8 z4ZVS5(nx7lMkCHtnzD6O|KrQUyKy5|DO_Awxg7gKoW~^fnATkPMFU8uv}>r*9xc%u zr<xYo9x%Ja+y5ukfWwBs^<ilUF(*q;LTjLTqCyHOeY|N4Kw%vsAN$V@Q6iU59%rrZ zziSdu`y@#HCp#@UCeaXcL^@3Zuln|@vbz_hJ5!G~Utky#KccQ654!>&$)iMRnaznz z2c|%$g?URvu-ki_(a+48b!M-*g{6M5dgk5%gfiubRJ2Uo`>=Z?j3gy3Sm(ca4s6{9 z7Xy&a;pRv;(EXt(kR1bKf-wVD_>n=_8u`+%+e9b<G2)(oY6yyDn6;nUo;zn|hU4@- zDF-u872;e6lkj1K^$#YVVZ}U<@fupAeO-UN0%r*P;+LYTS3F#zem9{us5z6m!S>Oj z)p%<81dJ(0I~i@~c!<h-2#Z$lmyV$dhni60-bsPwjgs$Mb|oF}txCi5{dnE+?c$*n zsMX4Q6I-A=E1fBcmwxQQoM&1W;hYn45H+=I(0z#g=$yBlYEs)mIR3!UY~J3pdx>X$ zY*D4Yc3tWEY`m543bSl11OBoyp%c6zr7vSx!URf5pKAa11l4ID;9pq#)5Zn>aicE5 z&kDK^ojyOcm`IJU+y7DP));@Bw#9D!VRVIf_;CGC)VS8`0JI~1l>Xk+R8tRRgP5zt zF|TUy)Y@6v*3`go%e`ee7sVCjUe9dy6Vr+2O9EH3RetPg6>^RAy#G}#qc27RivdDk z_~Bdu>~S3o8ug|;5Lao57Gu?>hN{3983F%U2OB-M8R=DbJI3)wPBv3Rn(|`NtEd0W z{U`s`2T-(Dy%XXpjBNA>9%2O{Q@oM9^`XY+;;JRp_LD;PmYtZ6yBhhHu4c0k+lr5c z-4)LTX?jZWyv7iq?!#B|XakU1o_6Qu%i^%T>V4LA7@+#~#CzG{<>(o!`;{Smy`O7s zFW*Nh=kzaYc$fDbYj?`3liOAQ+dBv=WTf#0tH-DzSTpnOP1Dmk(f9oW_LG%#MmOeW z({o4ICw2o*U@rBh3cCn?X^P|v<`Ggc7G+h94}xV-f0?z%Iq|WRPg#;1iMwY9V_ml4 z0Hm3z!PWFzYIIIwhR@G3av5$D<vgEnGx{+-O}g_{Pixq-C~<$$umrF8<N72}%m*}_ zPpq|HF-_<PxQeL$`>eAr^w@pPuiR*gup@^x`|j*5^=i6EVEcjOqz*x){k#tes{6O) z)%3m5$m9#6!G6e;sNauUP%UrL?z0@;F8W6bo|?m|?Jqw1xG}Jg-|;pZ2g2xtc8H$_ z0sWKlVO)|pY>R!~jqR2JI7OQP?ypw(JR3^*{=C?7{;F)Gzf26Nq_ZtX^~)9LhX*AZ zXBA9KUR8JSY20z;oJs?@VjU;1#bs}oqQ+?#GH8nX_SkM{>2w>KnpTbXiZ3KzYy=&@ zY$W4`2bjPJBPu?At*g+5sfC&vCj4Y(;Pq2Jn>e@a!&5=pTInD2C7@Kd0<z<{!z+Ro zTIu}NuF_m;a%eo+QZ!jp-dSqD$alK)of^1t_G=9(!<@tss8RaBm_nYVoW9l99)HfC zDYc64aMo|Lopi4{WI%~p;ev~%{)n`P#zLAYK{ODfA2)C=6x<#SCr-B2^V!5diF%)E z0W^)@2o4N@#}gM&16hS2fgLSYF<yI30^;OYA7$<;o{gd9zs&sg_Ov-(k8`|w6e#og zrm5O)!mg-Ye8T!5s|iW;ABhX(zjb!spU0>S^gI6{brW6TpXS`8RjNuW6HkjzV1j|h zWVFLZ@Nx&6Y&IUCn6vm4v>c~BAAUJWURFBLa%W>XZ&c7+Ok9xID0Ky{61#RZ#mcD6 zin4_90p)ns+J*_$wce#pV^%Nf04puTIJHS+0#B%~-fLs_<5E&s_lgAWMS+5CTM?)G zK74);Ez2V*#~H6e)GGm^NPHKfV)DcrLAZ1Q`GZ=erlf3Tf@OM8A>-wsOJU^y@68L` z%^CGMpuk1e)XX%?x=n~NUNTRlByLr|%2u|RrsY`^Ge?hswkU2kjOKABc;dbt?2rGe zSAc-WFhb?Tt|h<yX4l_M{@ts#Ukjg;JBkpJ9y&Pj4DN^l&r|8^%t~xD+ZJZ2w;ATM z>I8AZ+tdM6+v7VMLVlr=YT7D-Pay3%shCJ`)t+mXI_^dJ5;5+DV?BKTfg%xV+IDbd zd(?|O?&e2W$BNh^t_YDFZ)T=<x&M)7ajel@yP@H28gnpNPIgH~pNljmrbHmce`1Ze z;0R=61nm9DMUQ3h4Y60S9u5Gv4H+$I{rC00+Z(on(ijOc;TOAF#XP&LS8dO_i@sR< z6M&Gi>u~scJ=w<iRycIbP~D-62BJCu`*+{r9y&K{_-<j~d$+MqzX>6$M-5Oub9Y|c z{m+;!vx<s`ksj<>c)&Uk)88CJnejgy#z7nS@TG8CI5ixsUKFjRWY`fN`qDt;eAOQ# z(Ivyngnfb!Y-xXaG4iBS<_He}GiVIxSvoj<-OyVV`|5Mj&?E;7o)c<4lK!{p_t7Ou ztB^7|bjb`eM;rP}QQQa&aMCwamg$RRAjf2P+A?54-;WOo31rzzn26WK^EY8sJ+>zV z3)KtL=E{<n2MwtM+Q!{AM4S%uWNP8E8{5G)K@|N;N}?>vu5hquy~+9oJ`M#NMNPz8 z6XiSZ6ro}oQnc&kR<a4>SMSyOgL6Kf7wcKg^wbmF;BDGU)keoBIo=Ue>{(hsYxdF$ z<~QOnid%u6cxG2D(XbWK3MfET@m$N7(sk<+tl0m@qQSPYy<nU)9a!+vK3iv$gXO3C zTeP_<kO%gXBkpVacakWPPu-pg?w`Hsuoq<cb;CxRr0f_6&5LY(5SFv*7yHTUUm9U{ z3%|d2EXeU2$#F8zHlY`)GWD9C$9qSJsBGc`#lAnNwNlai{pwvR5@W=3u)=wN-oc9K zVx@#GW!eTsPG*nOZve+ggOWyG%qs|&Jc}tWuOBJ%&^C(^Rb|aZW$}?A^YUC}H`-DF z4;nVg3{(9_lU&d+GQ@Xo>_f~#L=he$e(1s-Yr!iGYPmXB(1d8u%LvbWB^coWAwSZC zFU$G`d4b0`$rJv;!L`-KUmY;T$c@~8_rGOvYr|8uWH#sfLbB8atGa&(cAjfDB)zEX zavJ1-Q^LIkIbW>Sliypn*zJB^58{*i1HJwiqy$_0U^p(z6$M^S2H<;MCmGSjvhlO4 zGVc>V-T7kF&Tj6DBP2$7m_sb5Bb<+Sq`=DeC%u_$VaeDu>h9m~9GRKCe{2pq!Ez8X zZ9=$tNP;>?uNj2S+MJNehyr|YJ`KKb*$C&`Iy6b$JI^>>bEy$NytcvDk+G8yj(U*1 zAH%Wg3|>ZP0jVJlAaN1F8vya-fdxJMWKmyl@z|Xr*UmnK^IenqT>;-9z&8ZWH#}K$ z=aH5x7wgRV9`8V5KRjik-YmB0$k;YE>F==WySslpAT;VEGszwxafPfPRQGiblO=tw zs1pk2Lh4ECgCf-F|Bdh&r)3^#@bTIj%>Qi8*O`HN&p$YfeyBO21s$g;guJ&jZ=Xgt z<elz$1cets5k4*mjDBxE>CQLiF<z%m(wo5+of(`HI*f)Bj>~J(QM}N`yyC4Y2%&cT zn77K{-RHGiXK>Z578@4ITT{AP>F$E_*$=;G8+NTJb_`4X-;Sdn-aRlbUwB&y{W3!f zNShaOr?2UZ6K$1i$vmlaHQ(Dk<|#nDXLCMl3=YS;MrQ#+%?XV=(tPTOJTwu$N5X4o zZ17om&<PzzJxsB@6Lypu;oC7HeX?FnTUc{!;&41FONWr1$Sc6!MrEjBm)FYJ+u(b< zLWA&O>e~yU3BG4G_}<pwaJ*|wR?5Ng`LYuox)UAyb|^!5JG;GOpUL*&=~MLT*y3#r z1>xw&sRt$$@Gbz_#W_~3OtpiVXM*n>WrOdXi0vcp{SzTPINsVJG{KjRA^85!dhu=X zE`;Ogw13<^`bp{mK)AyyA=KdGNw-+}+TeQ^V=ExsHTqGS^*Lcv=d;20PREvEN&mcS z^b<f>sCT})X0{V`Ume}p$5YFOXH3(JX^VF`G$(wLb_ftU003S`(K))Wb};k2MhDxR z@BIpgVd?)2=V@?4V?Zn^9X_7LGwQz0`QFFi>Uj64C->-t9StGj<7EXqN8F#P7snRw zW7rSL_!ley?j4#?q~>J2vdpp~e9tiK=3>F|V7gumTfEP)MH`M4gbt$t;b?$j5dprj zfNy-B*pJMx!DoxcupbR49Q`Dn5IO=BM*w49Jy3_%c&eDA&G~F$6!t@s{{;?p@9>0T z9U5M1YYshBht_yvgu{r88G3fMu*J)vua46RA%{(4pxUBF_-u1NTeK>+jJ*2?La5CN z+XLZ4%=rM{c)&L*bEclWEo|}n;GA&Jn5P+s+Tny6d_4C2zy_Z!+8#ScX8vLCnCBVV zoRAlRKWdfuT@JfE=Il5s%Lbn<-aPDwr~Gl>m}~{%u2w;qCHnv$cW&<*`Cyiwj4f>O z=0ONC9A_S$P@(`c29MI<Q)k=Q;IoA}W7qKef7~-RJM+kdQVg+xZOr55!h1ZU<)F53 znXUiK7Pc^7s5v1ZJp36T9McrSN3wTZwy48-Hs`a2C1dC4`~M2qGLL+Q;aEWk=hNYH z9QAOH{xw_J!g67c;dtM;90ei%ioVm@HtgD><6_&02mi2dOiseVaRnuq|Gghi8aI3V b<~RQzHUVIBMvQAa00000NkvXXu0mjf(qq7< diff --git a/site/img/dcos/exhibitor.png b/site/img/dcos/exhibitor.png deleted file mode 100644 index 3bb1c7f55410c3e162b619733f9f3258e03e5155..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 116526 zcmaG|WmsHIvL<No;4Xn+K?Zl%;O;Ji4eqW95<FON2pXKh-F495?(QD^=G(oycc15D z@63-m)2FMts=B&QclBE*TuDI+6^RfD3JMBU`jfZ{6ck(%6clvBTlhaEyY@w6P*88! zti;5Wq{YO@m7MI&t!#i$P_%KTMn<gC473Br#zsa1BlL7gPVOpUVUa3E?PDFi9sM1U zj!vV@?|OP``0v(W)ViQ@l{@P}D3Rp%V5Wi6E*j-ATp@XEYyNI35C@GdQ-K$DN8&e~ zbd<aoC`T(N5oJUe7|M5Sw8_cX0ds5tHgOc$xdBHU<Zis5eW7|3f+{Nlra!~-z{~>z zY`$zo_F%)qQ=^gV(jZFnlZ2zz;=$$xKl=LO1Z38$oT1!kGPANWgU30*e}34h*SuJT zrLExx{82{l;1VTL_6ewvqC1pi)z5}yXC{@Q7OaLkpuWO{S&Pq6^o)LIX7)INt9oZ5 z<BX4AH=py>u@VN6Gut3EsON`x<vX@_Y?(aBELgCx!s23%sZ|)?k?}&I8GN9w-l(wK zK2lEoF<+*YnwE=}yd1Boy)BcmnY{^+$-~y+k9nb>_&s?4T-pL%jLAK0ZS0(RJp?HJ zQG@r-^<TvR3i5wcaRCWXXvr&)i`hE?$vK$VnV2aAk;uu(`JK$nc~!(E|CjvFKLH9$ z7Z(R!0Knbdoynbz$==BVz{11B17Kzau(C4#sln*%Y3E|>!D#19`A;VQ;v)`pHg&Rc zaIvztBmax9v5CE_ivR`1Uk&~H^G`W}9#;R?lAZJas`W=fz+ZO&EKJORfAjuB%Kuj> zuacDq&_+w#$`)wn{HF~;7Irpf{(lhs?biP_`5&a(|3}Kn!}*_-|8eJkDft0^Y49H! z{nK3kDE(tEK_q^_zilswL~)=0$I*qMq{T(lJl-5FBkQTFw;`O7UGTnrn@913T!o?y zQw+;~G)z0l{3i#mb;_@+ar1)G6ut%asL5fMNj}wiey=f{l0j<Os?y5At|3JF)q*ep zw%Qro+W`^b6aQ1^8_%O^bkemo7{7<h&g$C*P+9AYdqq^%*bBqVO^T4eA54RC05o<s z6dbMa8ziN_UGih^tr-iN=l-|*e^C`dqcAv=|DE>V9EIay@wMzQ@z_-V#{b`SpkN@x zF#lZ++VKx>+L5>0u4i=PzJCuxBf$@Di^UoH?+hEh!Hs4K#k92W`<!(mM7ed_8u9S+ z6W=bolR?A47_+9V?e21da`D!rU(xzsQZ=x`9vmDv9M{ek$443ZUaKc0CK7Ni9kZKJ z6%i8>evOTN2cL+#|NYZzruzqzY+FQHtjNR$hyG}i>G1v&FAIFFOkUSwUR54)@(z8T zg8!~TwlERSTO_0)Ofrom8O{Y17)YzvwMue3^T!VM|3_xa2WS}wZO@m?33Gfc4TGtE zj>M>j?~?yrLf^NvEGASMHZ=r=gxh9>|HTphp^fmLe%xXHug*d%97v7GT)Fgt$-LnI zaTxd{9{7J*_!VB+($IdHyG(tV!wgg1Tb>xc3aS(vh50pk@V-$2jZmH`^W65}ItU7^ zVrrCua=A7j!HI$knaUG*b1jnB%JOSPNmm3W*!)&nGr49lK@j!ak@2jy_VQwhZay_? z6VUqE8BkHj#cWIA6aNc?#a^NMB&GGo%rgDJkT_7AZ8kd2`!I_a(HFB@w?2-sbQ$<a zi8fKKI-ykpxd1CCs79hOb#uxL6>#a)eHj5dzS<Lkvb1ZSM)P_h#gUDcmeQsO0Ewt% z7zhp=Pf;r0>`(PwRq8j&QD3^#!Ha0ao+(9x4sX3EPzz~*lxoPS1O)8$itw)L?cZUC z<H&-KduAB0(Na(*Ppq#LrwGkr&spVp6L=75GHl9F^_vN=FH~7<b*fM1d-ux;EDVQ) zM;3Mr%e<JC+FUuiQVk_al3hq%gq3CJx|U&k=r30Kp8Y1H7CBx_f8%Eve|)vKg^Avs z`Ou++1HCfmGlsdeUPk#mEUxQgrPIsmGVXkERy;veb;LhGzc_sKb~H1)+T0Uwc7CY* z930P1@ntPKe7qUUUp^(j4e;WqJ>J<Uo8#d_45ccvK|H80(2H;2AGKlfSmZ9UDS7_} zL1>^SqN0uJ_UPg$>o8JJcT-^TJlT=r)@?ajHFLWiwo$KN$YICSGf87-E#XADm~m9H ztvUOmr{7_#iA}47|B!yRF`*S!f0K&t)d{SHVf;eNQI$~57qcIow5PKrxX4PTk$Oi! z)8#D!@~z43c`<dOjc^Lni?$jr?eP<m_i>fMF8{B>jh!p$jl$gwkZAWHh1IP{>DY2f zCJUa~`ooEsQngJQ<8Grq=zO;~qROeh8GasYz{yu@&egf`{G^<f&CW);{f2|vxk>3n z8x$S66QAxr371UR+?=O{IXW<x?ESDTb*OYg4cY!)Dk$sCWpZp6s*${HB6aK7H$oza zl_vSwxc)+pRtPih=JN5BBJ$Ae^{(mw(u{Y-D*CG_Dp3SC?MH?zC-9446lsj!-FMD1 z$GBsq5so9tq{C))nzO@_O85c=d=8s*ymvy8MLB8}3mm?i=Xd(t6Cd0rXXlh-yUs+w z>JS;wLx6krVAN)1GJ?-dO<qqwAvX<rGvzb()3-v=sOL%3B`{lCJRHG*`yPYn2#N1T zsW-$2lOPATC(R-b;uAk7Mew`SveTc}+=9?JLw0Mu1P^vP6Gvu?zqxjeK|OCbsT{u* znnJeT&`y^-sbl(i84`-tnEI)_`gEuvc%$1=q%CkDgs&@!);CX$S00}XW#FfqoG`yf zakzHkVSy;fyT|W5!<=M!EP7esE0xpU;_@z$uX5{@j9_$)(+cpxCGP^=iNA^B=IKLN z`RcI#afLWP9)4Iy6XDd9yJS6trJNxuCQCr^xvWQq;H~#E%1WT&HY9@7eZLv$gIw3` z&-0`;&*g2N8+{_%1eE|1x~Isj<!9AO+aaSl-1|NDSP>wlXu)ew(m+m%ac~D1b})f? zR~wdsl~=|5x+%PY$5`4-!Q7T~#D}PN9XwuHM+c!2&(F5EPERtqV}NvL9+iRHs**I# zu_=-Csq=qCI4guw%AVC;YT6qgvVoM_YlTN8I%$~D-ducZuv4ik(Z~?{8LwOI$XbCy z%OA95+rNfI80`DhlvB#2rEM_p5kKEM#BlUoOt;ei=CDY$f(lW2sCFm&FwceS6oa8} zrQTV|zPM}rso8P#D5826rK>HP<1Jq^o~Qk3zo&M;M!W0q^5-oNtRNpjiNS2XarrEf zXh_7RQhG-4LAp(P@{QJvd32Dy3~!KmAtf)BGIVyIuCW_#@s@^`K@0Uv!3Vtz(l0Oo zM$u{6`QlbZm5)KYa$*$;+_y|Bts0tTPA}<oC(I<jNeBI0-^zXeDsoBlvqTr-;H1Tw zgCfz`NDCS4Y9!tlbF9({R@B3Mq>&dD_u}=EY;2~-UA$cQ9#=K}o{q#(xp%JG5@S@_ zoA5^ur4=UcNM?u!7ogDD8tk<ey7${M0fIz-k3i!zsp6t-`c6%f^=m=l+IJ1@-A}Nf zhnX=f&qnUJc*^5%=SoKVg{*!)s`<XGej8Z1E>ue-bv?uS89H8wZuz&pJP*YT3y8u& zL#wM9kytu14ir1Xt-THfPM)0_#DiXlJ4bqrg)`6x)vUUiiPCnNIAtaf3+Kb4#OtBk z-L1}kUM9j}YL}FxXr>*LBgl~G_M$eYe^#E=0e(y}%gL#aXL9aNos_p=%Xw)d`&Y!* z)<xNga6UU@S@z+4yn3I0y~Ds*kNt5pl-2RDXHG98Y!wfiAOnMSe%2Bm1y7Jc&t^Jy zM6Sl<te_q(ipH63pPpn8q91Q|7dgF9b;E-kH=Pm|*I*)HP|zaIAjX&DrvY3-xAB4N zlyns22z~T??qq0;b~XYtvyt?(&G1eohA>Uf?*Sp^Mo`*lW(lx!v%7o=&MT;okL2(L zeLOTn-=Nci?|A@pXB)U5#-v$~;+>5l%4;VoC@v^pQ;{n{D<9O_ZLpu5OO0`PN%VpQ z9oM77!S+dO#_x(r5M$w2!!gs{8xg#S^wmRL_b;`gub*f~2YccsMsKHz@=hbja&OaJ z8swZf!SZQvS+$V#)<hWuKraWY>~cO@E^?DJ{c38suUO=w?(-hmnQa~i`#w={vmXzP zK<Tv__Lk90-b7LR+OZeLa?-Gf;?uE{rDDWLUipvVdD=m(9i<X}!(Y!)aUBCM{BI^7 z6bcy-Sc$xE1?aecWFQ>{d+#nsk1kXlPe!%^Mmkd(N3RzZblo45>_uLzF17=oKYJ~K zdG-7?<-`YeCzUw7nhqPr%Y{I6CcXJ3Zbw)1-)=lEwVkc8(O+8x1!%&_$jVk}t=tu? zrq+KA)`%EpR;Q1>_5;<h#|X;rr8O=(WNZ%+o%UMrB$3-n&p-!fvnoAoh}#&}6P5@K zcS;8%BLHj}ib~lX7(aR|_A)I$ocfq)R9|XDi;<P~g&w+n`aZQ>S2B?eRw*`jvA6NF z&vJ&tES=fpIvVUfm?kqhozU5Bv5PKl)1kkc-3a)_4Ln}VULl1|OdLvDmpgrH17T7^ zORqz#BX9a}tD!&umq^9KgEU{k*c;k-XJ?y!pi=Bqt-j;);g%KeB9jZdRiHojPh;v& zOOA6~UMyd)uL<Ww5kHnZIgaY3CB$$U_#!tp+eUD6TQrQvwG7y`r)4|I_xX4$Y46a* zw2r+mO26g2@b-SErcvH*!z4o?X=)#MPJ8t=CCxAMmdkWa)4l=mqhbeXVYZX~5)fZ4 z2@tjwk%tVIm`FV(PZo5H-fpiQv+z@XrKr0~6v$jE2{aV}QaylCh!s4#1UHv7Bp{2M zS3Wgo+SMEpe6NkQH44N%(=_>Jn`a~x?Mf)Fr(2S_-OXV6sAoCR992CA49!!&iEA%} zm%!+XM>4@jJAtp%?MF^+enVnfhM6%Le3!BH&NQQXme)~xCrqwdn1UVqhL5<QOV<ft z;gW@ewIZpTUp;O!5u_k-e;)7Jj%oHRSNCi_6-Gy)d+bum3^B4L<G0<Va`)wZPbx_> z)r-c~i9lqFRnz>|{V;sL>!z2fsatp_0qB>=-I3i=&cV8XvW6=$-)#Ho%#KgsLovx% z`eFv))=8l4_nt(8od5j!)R$Xl+_eF8|4l-@nPj!5oU48-Sz#+Z{9*+>lc$G2s}q;e zco=QxxsiyY0~-D?z=P!6N9h(@&JfGVAKS74PfrDlqn>r~l!ow;y`1`#iP{Zi=G3Uj zx{ugDX7I=QDjMkszUXWMId87A<ZDOzX1*nk{TTKvGT8b4;}{~Zd=laJ01-d(I9OGv zfW4L^8BKO1Kp8D0iMe^k_~QNqK!f0sE$m$1{OOj-g<i$i*4YkKuvfCZ2Bn9^vxxHP z!AupU9fJ=-&Q0Yn7rkNM$dVb2k=&@I7Qa1ws$^Q()yEHM{Z?pn`q_+FC0Vz;;|xdf zy-R#qQ4^Mu&KJ~@72%_9=ZP`Lf^uh0o(pGfRn4sYgXg<BY_dRSCyJdj>MLL4FHEw6 zq6Yq?756XLxA|0iqda6YZqM&2+)w-F>kD%)q<62wK?Oo#rpxkY@o?ev6jP%)lQ;EB zwF`}C+eLm;F2wjL0a!W0SL^y=ah!Jt;5&gw=a$<piX|3m1NS~JkKra>hZmQG=p(fD zZX%3S)by}KnMB40vimPuZW%8e@2MZiheisOJ;Yd{(@nN9$9)~36~20AV;8GDC<mUj z>#XU0@k$7|K%m(aE`_zCkKRnJm?2kdKN?6@eE$-c$$fc6{Gp*aX2x!+P*LRPZ#a7& z7!lWCWy|;CXB~OeOrh@~jl=t8EB48f5KNXuslJ786@<KSSR}D%E$kx>98TLSC$f7? zY#k$eRfqKAWm@(n-QY_qYjdF>*RjVprW-g{CfIV2@oi<aR<nZXQdoc62ysxj)2W87 zXiTWoMwmT&!EjojR`)8kC-+ZX1F@euUOTyStmG=b=N1((jeD0eNv$UH$d3=m9s(Eq z1uHa=nE_UQ#)|HrUdx>h#93L<`~;z$tbO+tUbrGnKo_5vJ2XxzH>u^Il(1V+{tf$s z_M~fQ=<8$nZH-NzQ#@e<-s+QQRcU_}EFZD?%T3d%0PND5v;-n;rk%ftR`&2p%kq1z zUA=AxY`KY`p`jEdoAj$#5Oak_N{|P9V`CkTNk+-}SrPT7qmM1>M|ue)(~AC+2bzY2 z8+C)X4cERtC26k3V_HN@a4g~Bdr22!Pju;rq!GSxaRU+2eHa|HIuU2Hq|SqyLfaxf zf_BVkU*M(BnBG2^#8<7KKj@>H&3Nv{81x_X6_`PIn-)(L5<5H%ZRA>sUhcY=>x}%a z#y`}9K%tVoITm`*A0?>{7j{Fx&V3^O=C=hI!_ZdZ+VBv3yxjv2-cL*6tpwXs75T_0 zWFSTL^f}&Udx-k`O2YbI5jYvd=cDS4&S4$UgiBkq*4>Z<pt=LP3cSsH9j<Et@CEFo zNA<HN^Yp{%!9&T{KZmL!1+?AutSZ2Fk}+Mr3L+2u>7Hh3+j-sXIOt40$-$<M3<G5O zA~_duL)<<uYac?1lk`_h6&-fU<c^Z&UvNC<;Q^DE<tbk3!P#IcUsOy^l3sO6`*@=g zS=H&!So0}NrV5SdM5TYG=E<Dj*w_Qr$2BAn(WB=wtvF}cKXdm_yFwVx#*MDQIVKU{ zwrq|Hfhp06AMx~Z2J2uawTnO<8NH^R)$$GdrS@cfQvYi{kX;M6^f_2(6P(WkLeeeA zj21cdi<V!O;_{j}cmURcUESz>yiIqe-rHTyDmK1(sHyI3W%0bX{j{eSP^S3UOE<9H ze2C9mD84y$KKV`8J3m!pd~(bM`>J+yC6R)qo5s}II%wD-^YchWsiPmA*7F;e@<gc9 z1x9>M^+P$z*?pU=YwIy}#%7U*kB-=qeF)up^?0bP6ZM^$A~zwM!r<1@Q}yr$Tc3|6 z+fmluJ{QWj{ltY!B*^rY=iM>P;8k0N{RI)%1E}<IG-%2=W`5jl1oyQyM{>G-QM>nh zYz(-C$@agjD}6pfK0U|^-}dQ_OHr?CwBC}XyN?Dgq|}=vM-AeI=h}%`f4e#?4;(n< z>Gb5^_KztCdV2UnJ`FPy6D~Vi2P}VJ2BVipqxP(m@Hw_Hk&5;9O_nj-Qh5&MPAs4; zEA#q;eUAnsj!x22Hq^>^0JByo#We92CQ_wkxd*KzS-eW-N-uAl-GMn`(WW@f@07hT z$z%eY7kCP>tP6J1INo&R@U+oC2d$ioIdTK3kD6{!IC^_;U=9s{hTfwCftxH(Ng~XN zQ9T9Qlo=Nx#EiY?QHRPaW!tXlG8utTg$s-b*Q%25$0Sz~yJ-}A2R@I|o+yWZOBv%! ze9NsIcpprpguN4Wvvi_X?=}v$kT3a?^@=;_-9h)xB_{@lG^kxkf3?oz1JaJhYke6s z8>7U+eA?I;?|nb$_X}*nL@%stmq}Q75hE+=V`4fxo?J&<&rq4pEMg9iE?LU}aajL3 z=2w)E@-sv?0aifB4?48}sq>6o61cYL!RT>FAF8^pjbDmo*X&;Wu#J&-yPHTa^N`kh zjb1zQt<`$ZW%$!{kh#Ew;}2LlO}-huGTv$875c+FgF}xlhgGgAUjQG?hn%d^Rb!^P zm{ZsM!$d0KQmepL`<|Ac-8MY;D?G@nhtQCN!TVsX`tBliFuRy$No#4YcLjrIcHKu; zl;K44LKaYSm|efnk9Z~tzALe#Y`=r2`RY^OwA^hQXoW!r2MBH$(P>7gF`-lp%Ujo~ z#J%juiZ0hfWu6{aNj7FJ^IRS4%P5R^0*!bUt5~}MC*(}q+TQs&A(L|Bodbl^PU^D3 z*eb4Fx8!Y(p%0a%tHo4G{xhj_x&{V3(=!wAT(UTjsd(<m>&Om}XE%SN`FhC?N5W*> z)D`6JO=!9J-{2%~t5FyjhF6pmg%Q9T-lWsV8`?B6+MHiDNT*;N0S+)A=TDm#5uimK zeJ0PweC@lNoOk`VWa`Osj>F?qylwGZde6zGa5GQUEDNad$@HTsAYFuJ$beeHnrASk zI_(I+rLkhfN)eE}p!;0fs5S`u%95g-J8%J9HkZf?>$;e?ujtRn#<z^%8?l-hN;!GW zLUBibAA|lw)+^n3RdL6BV>}F{&^ayhfp~UQR=o>EYL6F<z@?FB7|Wpe?kgn*25rll zVv*~7AYoy$lhw4ly#M$yZyyKiq}m;uD=>ms5Qk1DVB*`>#cAU^2zF10;LoUv44WvL zS+d010DbFoX5(4oLzQ=N_-r!BADaOs@ZiIkO$SpDX9K{FPgPSh_v9h9%hqZkx`I(* z1##299w9ODvY-b$xg-1Zwm<vxZ1hQ8>sR>_8Npm-W7GjjG?H!Xajq}y7qy{cs8J_6 ziw9}n3m_WCd}vA9Wp!F5sn((cdD?ZCNg49s^blBzL-ft;xUz0MC^0EiN7}>CckzX* z71&0aw~lDQ<K`h*DlW@I1zD$G2`JrFyLg$F0JFtv10fG`JLePHE)^Bq#u^gU9B_aX z>c{5DiEpJ@2AtK?g2bVC1rz{Q(l7T^^a=c68wsD8G%fm0-CcQ&h*3vW{1RkN`a?tr z9>W}U`+L-&p&Ku2<=g%C<Ip+*x>HAzj=<$mTPhNFxuJR_Fx(9R5cwm0e?fzI@~*fv zg7w5YWQ?I%;f1qSAbIW277Os&7OUYAc9?e9Pde$S20k%A@Qu)0@zKzdD!T|}o9@U- z`)2sWL_ZGLbg}0XAL5@ia7fc98rWv@!9!N)9rmdl8rK7WDq!#g$COFPiwB`H8-q>6 zI|!uNF3pQFh-z+sNiEmJFO2;=sSX9Gs)?THw<wwyBVnkRCTg$TVx&=S6Qh+YMpA%9 zMj>0AMCYupYgE=g8K6QZo{<?PjubTW)A!n{#XM&8<K)b>u5VHPOWTB4Y&J;M)-e#G zf^v<hKb`HLoyAnGQ^nAlBS)NALH}bD@NQz3t7p@cZFKtpCKjVeHElB@2mX*ubJuoy zdc_cWZOHwj%0LlOT_;ZXoSh_r%XA}g(tCS1`%RXQvf4JT9zV^likf3_?Hdv9t%u?6 z-5{SmK^nQ&#UpT@_aDtNY^i=3BEOIvy6m>t6wC<f=?8PKUg)FVpgyx~TtQMl3BGJY z0)Lwk-)%;k13u;+Mjlsox}Dlc%dh<W3HHIr1z6Vk@n9}i)L+tRcZoe=4)CZ8T=Y4E zmNFyGO9@=Uq+gkwYkmk5plK(kzYUWW8KXT`xp4u~eZ`;Du!+Xc(LK?4=6%{!tW-vX zBRLIW5p#*x0h(8P*5<dNjWPDoaG)NjWXo(F9@UpjVP$+PFtXEWp~4zc;<f+-g^ndY zKb?GRsTU;XIzkP}GDL}f^5(%M!;dx`mg7LF(5A2QWvA&fr?*)=N0yYcL61qss!W`8 z`|7O3e!_A*u;wgqzjY;XeY)#0MTa;oP4c|A#50`D6r7g?Tkb}Z=m%j%=Ab$#weTKZ zio8r3j8(upG7?Z(JIokmKP6X#&%@kv(lecR*L}}RBekKH0cvO&B3DB6hcGK8?K!L# z7>P>}BFLuAI`ykie^y8hJD8rK5fP|GHm3P7j?0K4pRBACl4_H(U%WV4a#vh=yQXa! z6Tu{l)ecpe%;aIkS)4Q<bL_o8#|8{=<!n;}020$ves%ll)E5p5GX*+JkR%ce#EGz4 z>-(1ILGOWmA{G?=(+~V9oBVgUz3ZsV6B`U>HMT1^au5|j?_!d-DNH245HcV*q3COv ztzR08In4I>&7{h`b!)5;zRQit753NNmPI|Tz)Y~{QOq24@;uK2tD_J_#hwak-q!l3 z>V#@2yjbzRpIoG*R8mnv<8DEz5pAn<^eH&ne$=fMNII<9sWv}FynQ7Y(!dREAsJcO zBan0=IW`}DAR!CJTia?#o_-3bh#4fBHWL6|TUI`n^tMtc+2~7?HyK8H&76|~2h|z< z9G~N;@R#*BB)cc8=<_RV=co5pCh90^dmD=kKmvMvc>N>YEJ(}Btj{#uq@M7FC0k1q zr<J+<)BAvCTbw<&8ty`-`ED`Bwb^eSrQ$>GBc#6T$mZ=qf+-ag?TE8T5p4%u*LB?e zURV4KZ^k<;3tcrq$i#LDZc(tFhz6>-A18)*e69Cp#I?UKQu(B}L!W~g#EmKim9#^0 zBS;;K<$1XAj1kxFB>Ng>fDpzr`O12nONV}Jpl1<rb*SAyG8Hd0M$w^>gaNK~%`QZP z92ck9mfYkA--3$1Tk1q87LTlh^@(+IalkXJ!S|0PP`X^&LmLLct|7IPLxprpH?;|j z^X|RydX!R~olV}Lm9LXAmWs_ut$fA747U&($Lu7_2b;E(CwdHB;7~N?x_Z6B0bx(@ zR&(lE!u;DzpuF2-tfTwL0Z_|bi{A7x8S0I>P<kwtOEk}8dU1*HW4TXHI<17%oh(c> zY+|iz;h~Q&am9}lu3EIag^_$Ts?L`v+XIwCyXExu&%AL@b+XS*?;zcv(+q)#`UTUU z%H>8b7!TVGQhz^~?7*RX7rj<E$xx=B5w*0WceK)uA@yfpz<*Z$sPFaW2Jib?wG9lW zLfhB)eZ-_kuu2yJNQg35UlWr%ZX17>58hClbLvmL4L*x7ENe6?CX~z%9}DZ~yi>JN zTfsW;kgLnz7uV}~9hmoQiS#2l7-NY}XUD?}_d9cR9k>3ExZoyhm$D+^fxAs^eI+QW zBHtWF=8tIWH_zG3&Y1!#et}&g0EXU^XEBTy)v*H8+3ChCfA2t9j(wLiWJnyl5N3Zb zzc>z~@JhO4x$6LAuSVS=adDH7$(iwo;<@U~=cpRkh6Kb3>Yn#kjqC{r(6Pp2Oj7Or zUfDhe<e-jb6mMMlc<s)6!gxoDzWeFWw6(N0aS*Y)e9Gr;ts}i_YR6+c(TO=H6ao{( z$Hd)1$mKnddpAD!p6<!9C-tBZS-jcIPDPXyNs3P}5A91~-cacoQcuE1_iDCpC-?!y zl9@2BM^aSph1)!M>ql}(ZA6z|dX4;JsyedYAA~P}`M2bWOde(ilbT;X5m&907w73x z-2v)U4;1KAu1WbO*V8^?l^a~K)B&7HHlJlRJC`235`>IB9gUNxtXRbsJ;&i%W1~KQ ziJw^ScQ?*!nN`*>>HC?b7pwz9X^p6rdSiInug+rp_UG{zUw`NH6jb)w=urXQTu%Gk z>O`*(wuAf8zA!OEkw6IG7^^XbX$)&?vH9!X+uELq=l!tGT?JQ;R)J1hkL908q_2Pu z#wFo`;=gTGEK{9T9SY60wZRM2F@6Ca7!yL<2}Z}voT_@0@)YAW9qR|K9r;{0_zAlo z8V<UPo?3<m6noX-*Y;wq$D2!r8h2Omj3WqeGDzX&A7%M{0X#gc-_Khq_>&oGYS>_v zr`*wUO4FJRAdgY(&2c6O)M3KBaHETPd5|*Q@E0C~(fIB_?;fTIIo5k7(e|T`;O*!n zJX9J|yBQ)R$i}pGzBTVW_n!mh=p=>CgY4jsnXY`+aY_|Vg82xw?K<+iG!zG<l6!@E zrL_}lIjhjr31S-HVe?V*2hWqNPQlc!EpqP!<|5tTS}(9AX=u9LJE1+_YjfPwq`qUb z4HO}_!E568MZY-#e716d8XfT+u9UB3vlnfnQivLj!`yCG)Ac~=$|Q}v=uf98Khv+q zo&dQSFGf^P7=v&}TT8P#OV~ZQ(FT5zB{8C8^xo857pV(rv;@3C!z(g)q{rqVK6Ga^ z6+&80=8qD~C36<c$of)ZVQ#G28S(aP@vC!8>ygijmzp1lRYCifa#N+%+T25gaboOF zmhh#>#l`cSIz9d2XHxflN=(u8=|=EjyhC#7s_9o*DJ4P|LsDDuqxk$HM8&T8Dd7ah z6bs~!i}cN6Ng{~i-x(~hOsqXghAIqzKOJ&MxV-7^?I8U#7XI+<W%jw|RnFvkjPVR( z;T+kgs}`kKR(=upW*28^Lb*4rQGW-z1k%frGq4O|vJC;mZ#UnE>9k252b})OGF@)V zNE&sK>2e8o*RVaSMF_@XU{M3+f3{4fwZp!*_{bJ}Sa~vS;UF4hyd4~WE(TJ%*PGC2 zxhW<iZL^PT71TOB6d>`0p_p#R<$}l%i1eGKEEk*p-WsuR@VW;uQ_pOhgsf$5vob8S z^fY0?amTJA8c*yhPkDnuRD+XMYzE<LpG^$=Xs`5}R(QtBxA)GwIOcwx-lDP5W>o84 z<(@3RNwGd@{=Hk|;(o4tYZ}VIOogEb+YH%_B_$-?bue4KUapZAC<4&Jlp{*{CaYaN zdbE|QmZhE3H~Oq-ho!n70W)sy6+1`YV9ewhp^y~oLM_obACl=+lOr#ZvU!QDNKTD# zq}p0rsrKgXbDP;37-i)sd99ChfWZcl&RWx6Rn&fSeb3{DKPDf!9L&#|W(hODgYGUm zuTiWE`M=uNF5=8dJ;BorN0>*!^`fQ*8#q6IopgR^Y}-;Wm+yZ%B-}!K5>Vj{r8FOO z+PY0EFQFg{jLF+<U{dER9sJZ&2w7D__m0U5cKj3kxrh7ehFUmgN~9#D@0X5wjF8WT zTFE$qA>@#ach62VyHK)A`U;l8R;%NKf9DD16G)ipN7hK~mCKoPk=0x{;q?F>mqP6@ zQ#s;IeYq1e5{}=SgZYj<k7W17i~(OFDK_Wv^_`#G>R4|LS9!1M+N4nTKJ!wcIe8(t zX-0RxdqOHgsD<M!#<0}S)3(Y2+YT->25yTa1tUK$f;o}^N{tiy;3I}oty<_se}R6Z zQ3L2`g;>0GZDft*C+HwM$Fkm+s9A@{l<AY;hH{C7_Ih{Jfc*=O0k5$ryOV?1Y^U7C zv;8^FQC?kbR_Z7ML?L%q9*qydOiD@e4)rOkdO^Ko{4uxgnm=^!DO~QF1wD4l^I&Br z*($f~9*$<$0!>*y!E2iWh+PbIH!ULdq7V#-;ge4~?!NaOR%giz@8=>{DI54$%<~#Q zdJz1!vx%y(su)>b@pPw2{azmRRO?+bZZQG>d-cFkN;%`4c4pBW2ot{9uba$${!JU} z!DKL#*3c6F<)`TQuUm%=vLp+3c1J=l->1@kO{Kn_g0_Wv<_zW{Mf?`!(0<f$V5!HM zqs=eqdvNp(>IKdF6-z}BKd7nTGWL}b2@&bQhKqlcsNbA9dnk!B&2Yp^_aF+<PF51E z=2%QE6H?F;Gl;u@Zu^mLb2K>A)y5BT!rf_+n`(Vz9evuvdYo;&bXffy81}SDlmw2K z+N3RZJif)W(2b80wS`03R%}XS9K{<V8fqdFjEqj3#%6b^(krE_j!h#1RqIs^zPL$6 zwRkXOo1pVPxn{`jA4Cq$<@de~#$pku6}oS~sg)H>A&q3pD(xd8)0rV+8CXj-Lkevd z{8SRpDv`f8WEE9KElcvINkr*-T-2xP?7bQdq!gGeZ&1T|`To)3!YF-wRG(vsHABy! zY4;Z?W6Uudv9EJo2VgpBtwz<p)my!v@zU8_;5V-5iWlZKRtVp=Gw&z-=wG3v{kuPX zZDl_So41d)NZ_i+g`XF==pz7k8tzd;RjK6pn4JVp?QSf83#0H=lJuWuMK8coszPc( zG(e-<AKsONxN7kCky&qC1)PE(J}XVawQD)d(%TDtNXjqmc07K#<9^tBDk3ePBeK8l zrpehmmzW#Z)IAV0xR|wQnd~||3rTVEJWh^qR7Fv3E*KLTFOw{E%UN_cH0o{YJ{V-8 ziy=3EaxNmAUZt(3Dtn<_oZ|<LxK5Ajj*l%c*-z!Pd)z&KnR9-@_664p>~nv-Au_Ec zulJ#2UwPi8WCST5@`L(FUd1FOYmgHcsD*yly?@fG*L{$^v6Ra!>J1q#8q%?~Iv~lJ z@%RDQ*pzj2y5d$-$c$ubObV}Br>id0*NOZ_PIh$2HQIz?NBDSp#0a_|72YCxL69R0 z{mPp1g-%y7e6vWm^I?w?a)cIpJO6WWt<X_NUqn<}tNT(OPsg^@h~gWenO%)1!t>3e z6lrVGn6IsGmA^MZ0UDCz$uBA`{}zK}^#XszEX$M)#?F<k5Y|$?3(Omxosh-L6TL6? z_X3eDsMX4i-c--e&jt@X$*r^~3m=awmIA5#Uj}$bmUvWWui*9;Dxvr^m^!7H@Ki4( zjg9J*NkrzDw~bxWmEHr`66m(RUAD9f$xu=pb?WAO6pAl-XWeXcHXzqY(QN<@$I+f2 zKY;x%X{I!FHNwlT(M5scMP(<dg$mLN5LR5@!WZ0KUblWJN*Y;R84PpATv(|RQA&po z13W(tiBdj4u>LBoHVpjjc~LgdB2)T!xmu;^uA9nQnNSDPH7yTuy+`e8c=1K4Xk*9U zIk0x&)FfAGD!*E9xq3>t(odVVYYYK=vuj4Tdg>}oc5%;QD0E#0Cpq34likXiEggnt zH>+gkr=10_KBH^6`=TOHLn4@EN8T2f)m~ve(<Yxgcip+YXWi#C%e!_iYwI<?fB<z0 z?yJ4GSI^ppi*Ia@rCx4{96Xj)7+yw=j3Z7pHWqU(ndR){x6px%+Y)ge&QEV#DvHD_ zU2TSFHT!p1mT$S+;>1{-BY}D#+jB`09sMDTP#q!cKAKzyW!9$TB#OBMLlW$-T}r#~ z*Nx{)dA)p6tGd53XFO(zANqV>i3~E4y;($l+|oA=wGXT3U^uTE+%x9;hIfIZg>J&9 zmArivInW{Vz>7?Bv(b%yOUZqMuy=bGK}#qwt3g=L|0UKCwF|-URMex4>N|>@*xj@1 zZ-;g=lX%HN!$d%D4hF#ut9xtMS~K7J<j5E6)QdKH#d>PkB!RMD)jk>B(8}&+ts+X= z&`M(V_3|%uQ@A0Eg^^uZt9qwk_yczkm(lPNO9&pvkzQEKlDrd{D?<(hAGZLC7k<!% zK(t&m51_R7E+>+}8&J?k+IO*?wH7NP$E1^#`rQfgZdN*{z=)&h(1T--3Bl_8vy@+C zzN}EERVc_>|1FxEL6zrt&%?O)09vW;3(iHfHK@R{u`H`C!%RAQF_yDO(B+s%F6?9{ zqE(6nfcrLHw)u#Pf1b^$c6fz@mW1=|?Z{~q3D4o#w7Vzw<{WnDQHIGk2gtEim}~T{ zwiEsm&3CgAg3yc1)BB%>_#Tv%N5gXjSi7VB>m)o{FVpUGy9K#z879m-b&EiKU~aUI z_c|(D{zn+{UC!p>-XX1n>P?PWfd2>VcH?C`l;&kjl+!p?7@??@pxfE<{<cXwc9;s$ zUq$Q$X`>>*<KmSfu|GIYfpj5En$-vqsafORzTg|x1p}iGb}hcVAp~fw*}!9~3=4S> zuh1)i!UlfW1fWeb$D?wFSz3iWIEpwr{D~M(xK2Sl5iURzaK|@A5S_8r>-%0hOq2R? zW_z>OLSucHg}joQ^k@B;$XWTQ$FAY(W=ob_pBgo-jFR@ef`mcLLi{QD&qp!-_O6Sd zj?~{o%(OtxrgXU;kbmM}zIBR|QT1byF8eQ+_{UNj6rsOhKY%i)l@F*Gth(QHFJXtb zr5v=Nn>buu&yZ0hU3~bvn6OhW&s4Uh(AM?{WQ9jH4^D7NqAL3++FZ@kdA+dvOY$OX zq;x(26e2}zEc!(b*LwlRoX&T2#Ed-Agow<uKg*57!eb=DlJx+_v*}Fw7EJvUxl~2p zqKludkj2g9_<6Z<bCX+B8Zo7!$@&Qn=MqBUGJcp&P?pB6HO40qv*$#AZISZ~&+g`O zQfcXGl5{^k7fNi;j<pcYOr`|4#r~q+)<$gmNO&&5|58v+cRb<#<+T&L8s9(|kYVrF zvLSHYMR{e`sQRks&acBJ7_IwZB-e!!@bkqOwGQ=Bj#ETw&sjL3{fYCUqGK}$tuOg5 zV+w<&WrZa5U{sSYl8T9#Qrjj;%T*j}925Q~*)svWb~lq$n|?T#&>{$cMI5|nJ{n*v z0`)+UFl6WnFld>3-wTFxfRPcg{=k~+XvaN_JfmTg(%uQ%MF<~Zy|UKSEEm?NQb?RF zir4EF<xj7n!8#g(0QcciS{Aej+yl2cIae=-T5oN5UZMSG05s4X@%NW(XH`wwGq(4o z)uAGm!IE^RVw-<~x3fu*T%_ccl=zOgn&ti;Gl7EBVd@}&FmjopVrO^YkiSdiCW0X} z&$YglX}N9#qUKa<3OCP-zcR7WI`AMdSNia*;M9dfV_V|9*^c4cT8<e<NZ<>Fs~fu3 zqI*qV%aT8!|N3b@h~FR3V+i#7V7u;a-w#ctw&2Xz@ODHF)Ys9RF|haw=V9R~f0>kW z4FQXC(011iO!G?oR}V9g8X+kn(+ATvFz_rQy6pq%j>Ib?@?QY*4nAmCshF;c0(b#D zCL5J6qzFkMG~$jb_Z5*p2VF&jqmpIB&i_vuLm<6ohmatth-v>1qX`O^EMA;Ir=!7{ z&UiT)QmP`o%W2J`;bAkSc~$nE|IWpM0GSpmY^>!9)4Kh|AnJ|xf?5nTB-HN-G?kpx z^>|#LIRo)|McIFaF&w_UO@G?LVpr45RZ0@QF}tL#TpIt0W!?6>EB^j_=-(I1|8|6O z;XrFdrdi0tHUB)nj;Xe%a#k}(B4v{KkMbAcshL@CCxIO+80hD=wzCI68ky4+-_Dhu z$3w$2B39Vz-7!OE02{k}N4*%rl<j;_7um~PZjg@#V0bE_Y8h7`?bi!3h!x=#KhR7K zz=Y8->CChtg-eS1cp>{qc$~<-lcs|#LoCP`359v6uZEM>sGKJh+!>6~bk3(XmLAyj z=#y-adVei;J6$p%5;RU7kLc$<b(A%GODtq^TKtuw{0I`(lbk)79c#{~k3<T@1Hd zT(Qqh_vR|^CB#84@9)V4*~^F?#K#~@ONTDypdsT7Q4CDtt%T9wIzYBh_`B&^wxLlq zC|0yly{w~OR)3)ANJyf>Q0)#5o#!e+n5F2bs1miMzPTP`dz1Z@=7J?53?bo}k;ipM z`_eC{>#yZ^2sdM$aaSH~3Oi4Gmq4Zb9L@&A$nY(pCIfI22%<HCOk(>-+MIy~VB82+ z)f5pRk#xxktTdEiN-J@*2^><PoMRJimO2eu4Ej>k@y*75`3orwE9EODwl(P3>|iA} zXPE8lDTWx<c#^oC1wqR|9VC=i9dirvUY(RN`8#%Y4X<&$p=wx)P1tfFOBlK<<;<G1 zw5g|2%zfrEjDHOrZwZ3_Ge}61yOjQsB(k`mWNF}^;FukW(0`uOQWOLF$Te2on2&AW zyL>K4>w2zjTTVN*>UdwZoXH=RdAtJxe|Sw?`9I=<HzTL&LOPy3L^_NmoAInoaYk^~ zd@ak<ADoVtYg%ttp9a<l{x%m0H{%)^HR`YW0o*b>X#gg5*&kOzan~B@7FOgJy<e75 zkg5NkEwj-4Cv)i77uKWX1x3(P77V;T6Ra5?D{}GXvORs`BE*t=Ww=89NmzR4Ar;U` zbIM&;gS!%%gw&K|e_C+d`ch6AMEA-F4f`uU34R5`=xmTLnz8K5ZBBkr`SxP7mG^1; zVVDupE0&;Gb2MsPfAkDe?kI!m57kYL@Cv%^d|3LY1s*8Ba{L|ez7P`79|+ttFXPyY zP8(B>5KD79@12!krcZ?Zyta@gY2x7{epzyrx5cxqlWEqXl&NB)mlaAFK0)&VT}&+u z^bCg4t!z~~&NwzR3K&QDoU6Gi7y`5*(6y4Xn&kRbP#Fedeq1PLZ2vmj7o1NAglpv1 zUbESm`lL$imEY63N>_D<Pz!|PPSAz=ER~}xLm`@GIGxXE<v`YQQ<_BDLzCqfDn{jM zP1cQt<#>r6O1?xt7WZbc5$pFxZbfC`080F$!KVhRw@O-bd)4MjvXFR}0>*0w`lSf5 zexMD4d*@ou38TFsXg2%MA7mJP{rCQ)bpXC)=c_zQJV`w1%VJyU!>Vms(C`mUQ<dDf zrm>T3I8lS3(^oq7E`Aze=$=u3iE3j+{i=?;{_|y|uRY(`|IU}<MAi*eI>{$Bs%H8m zD9_Jh`_`G6Z63LEnR<=ZHnpf`(4RaSC~wtRogA!)!_zM2*{5?WK-hw9Q9mmA{ef7X z*OX~g<gNjQE*}2nz~*yms@mhl;L-L&wyTZejLL^rooo^vL7`EYOi15>g<nWjc=_fE zkF#n_an;7IfA_M)jg$Z0B#z-@<GIC1=3<1m0Vm_#Z~eTYc67Ocr&&{cn=@bnp6=>J zam&(65?2_<zo!5HNMLc``O6ARzqGxDF)|wtZTU^r8|o|+m!`2i*y`UdQXQ{08m_lj zmv_=*ZMGaHBs2bjP3ho7!@s~ki60dhLRQh2OY@<D_oZ>N!R?s&HSkGKVh+jJv~<O4 z!4twHCojj%zdKWT_NK2&xSBtg$uB>?qYcarF`qSK+}mD@NB?2Nc#*<AQD;Ao0qJpY z8PE6H9iNi9_9}G6EA3qq#E%+8p4p7e`8_%Np{P{C<LL}Jg)aIBG{0)zupX!B24JP< zB$3rm_ns=Mn<H@*{ZtFUV=RZ{mx38x*+csF$VyZi@!ibC*qOF-ahEOv?_`qKLZ-kS zJ;L=o=)8NkEV}8Z)T}>O<M&sVI>LN^&Ag?C7S;KkiQg23>fjRnSr7YvLt-%1@3S;^ z)}QxZx`ej(MuzRHe%Be)&DZRF^Zm`rg-$h;6(=QbJcq6_t_hu+RED{@olXEwkR0OR zzUnRf?WRZtO_}TP^gV7be;BEhA&H7hrtA)KCNfk1mY5CO7|0g~@V6tIVN=-9UR_@# zF?VTZg5#pm4N9Z()IA>=g-%GGkv2Lcd~}e*Y_bwzM{Jtg5FKh0dDoYT8#!YC(#0#= zgl5Q~9prdeoF|Y^TW%>_`m4_wIQJ~UR7X~tFyG_%t01#M5J=mkZNZU*V`WPc*48QU z+vr*&q5HF4FuaU+{uA0_j=bSIWTf}oMyS^Q7<$>0GI(x5&C05}3>|Gd4O~J-CQrBM z@h8ALJoMRT*Qhda@B+uP_{CEd+vFRAtZ9PHh(sa^#x^9l99-Ia7gWZrZWH3KN5Pu| z)jO}N3@|5Aoy5R%59B%DF~Q#2+OTm<nW6!zs9+>p65tKt4AnN}Zmg-MN3eMU&lF{( zvOZ8L5+H${OM%@O>6y_6$_r|ivWtiLQ8|ts`$u|HkPA|Mn*1LnNfM31SX{XeM$NPG zrI2r}Z~b+Dz$JPcU0!%k%<|%%`nV&O>uNB~e?1?(L0U?)5m5HTT;1{u^>IZ>i;evj zbcoeFSMJxz!r%{gCVzr<T!OF7N`ar@bt56#02_Zw5^P*ceRV)$N!FbY_xrY~j5cn! zdeN2A(b;g|@}uodvW3vR$JQL-W@q$D4KNdJEHhFspDEao;6_8gLuJGx?!n%k__Y&g z%Q0vjwVNRLd!w(<zk_hNfy|o`?lD>_j}?6tr+XlDOt{<LQ@rS2MYBF@(N*rxbb)w- zjl0XqagR-M>NU2jFp!z`!A7>T2*=q^<G;x>Jr2ZtKJFq&mpX)EBB*f<-s&)Mwq7G& zEyM>qo@!Un@A^F~guXqRz~izt2%aFD(z?_Y;!IKUTt@Y{ogFi|QRr7ySw)`Lx-QI; zfs%!`LQ{S(8#}-ue)!!4{lguFaW?GYR0nwYK@FzG)5EBb&su5Ouh+KYGHat+b9d7# zWnb?xZf3@2P{n@iW`$a=!Sv%wW+JT>@YhK@cm2bYE&hk6yWj4nQ^KvXX-6C!pbFq- zPL*uZdOEBt{fD`OtNTy!!$yfU#XOWsjN(IXCq^*koEASU!@zl8Y^pVW25z(QAS+qx zYiu@bN8a1lKI*m|ki#o}b^cU)nzY**<@*OSj%B7u=X}2(<9Hq9;!1$l-&7kpjiuR{ zP7vA>_CDJE6QgdOb-kYANn9gTp=6{|f8C9GsLFG~RSqa!Lv>Op>Lge>wU8t2)+f}W zEG6r4Tl@p;B1{gV`y)PCDBk<JQ;xl9HZ9)}h?`GXW{ka4xk2BKe7C*u&Sp}Tdk#xF zj{}jUP<y{df9mb5@yLS&trrN&@0t(7)@Ei5;D|kTYQ5dQo8o+VL+f>l=Gp?19=V%( z*pErmDTGw-eqU2ckzGx2=9Nw6$IE#^liy-`<-vcf)p0oun*ddOvngNMb^&JB_g(%5 z49OjCL#<Z`U|6hLAXU3L4M{#BiR1&$BLa1QZ|Hj52FNKUCa+sv(~4~O-mZjg?=K*= zsC%qKD86qXtVtvC>T((`@l@iKtDEy9eEO`i_!!J0a$w+d>TslegRbk~rgT~o=h;7v zpSM%ex%Ki)b3+S&viW7+SAXbqK7wbszO0#S{OdBbH2>)x0Ym5It>7Q@lkfSj?)vp= z@8emBkb%`82vh=_=&cu}-cB6NS5{l<X%CY?i0=Cc&}}Aby@oGsYEDiy?cYKDEp;Ld zvIF>Anu^_B9n=x#FH^>nVFda{3#I68e_Y?oy5#UFoh>=Y_<a1tS}JupLFxSM9#gO% z$Pvp)_MQyzrGi6qd5eeJ=mE1UwaDlDrNcO`8fj!eJ_LgqRGFHyUggOB+byu^RPA;x z{49>)4YG=(WgxDag4lR}7Ul2#aO>x$!O2R+#>_#_LDVd5C&6Qhfu7wY+vwX)$SuYD zm76oIMf@*h1CcKV0%iLbv+$R;B0xiiHcEmBZ{0kh<%TzxtBjQi%^mnm;fj5-FtvQJ za<RKE;r!m~Xk!i1PZ5FfzYDvs4O01VQ@BFrpMsvuq8srzrBnfGyUIAE+6F<-d^!%R z&;AsQmYpmeuYSjq`U0dnk%3XLid=C0#-RC90SJRxep8y4OWl};ThXfH?Kqz2OgHJ= z-HE_~4k7(6#=qVL(Bod9A2C2@zsAFJxziNfuxaNV?!9Ba*6FgXxQ1Xf(bR4d*g{;{ zPy~&-4s*8G_2<nGX4UD)ow<t3QLyundRIn#B382AT5I8EOnbqK$CUQr0MZM?-!a2i z>UDgQ<q~_8s{;mzZG$DT+S7Gi)<P)yHrfX)yS<82Uc@hrmnY{3a!PXE?;h+JB<~vE z^+?{YpN*RIm<YLj+Ck?ei5G4d9<H0F!UYT~F^1gB2JG)YeJeo$;!MxAE%8=2_q%$C zLm#b!i#moz{JV}>FTK{mdi>mvxS?JN1VuLw>vQGQ$A^7V@k5-H)iJoND2gJFvjCHq zutt>#*8}ho-05>h_s5Y1I8x#IBZ(v+M|7{T>g|M+3F9zWF2s+13{o}9Ng5d&tUJM_ z;53{WB2e%_t*r(1I^_b~ZdFb4RbD)KiQ!6xA_X&H<q^D{aQcgCFN=vSVREzbx@uZw zmot6v+xd~wl*K--9v9zWPC4!o;SxeO%l<mm5fBE0U}tf!TBpt;?KmPX(f<X<?u4%0 z;Dj_nFy^HqveoPH^69j4FWLajbZi9wrkT)PkORh0!AGp@G^yEQt-KA4(#Lar8*t&D z7&-ry;*fW{<J%YXjnTso9T}OC9l?sG_agbhWo&iF)T9&#)$^Ig`Qb7|*92SAH$AM! zsj|QPXJ=Ty{C%q*cth(j2o2GpJ)Mk#E6VgP!=+X`715p`C9XJ*{Ja8I``ofr{`*tZ z_R6Psyef!2v{J0jo=WI<35kq?PfqAE7DG;iNqYFWg>WTQ__$Vsx#=p<qC3v6GkUH? z8cA8PDP1}|KzIkT#jN`*alaHQkj*Knwf6O=i{TC~4_UMs8-}h7M6~U)K!!s&8?w6l zMC=p{0Wqui;WqX!?tP5)&+oVlXD3zK{o&d?PuU(<Eu^NX1Z;3<`D$IQNp<xFn&ugx zyYsPT<TWxSvYm=*o(9Z&<kW)p(9%j3^~(r!;-|?o#FEU92_^bd)#KX6#l?xei6$`F z5>640(uRLe*PSKsyj4hxD6XAJKoFEN`SH|}g#Jmzg1Mw$S^1z82xN(KZiEy5Dv<mK zu^r3q`e(<%eOT@TTc`sK(MtQmdTFybY$2EaxuiyRsPLi?N@n?*;HgqNP29GuptCg6 zkFok_kayf*5#>Zo7CHyDPWz4nZ0^Jnf?89rJCh~yyhHi+a1n3WY&dZcnT|0Hu}Z&| zyar$rI>5!jw=J6r7UTXf_CL(McTiJZ`vt1Lq6jLAA_@YcqI5(=ItU0Ty@eL4NDZO) z76AdJsx$#<BAw8a&_YwXfV2P!9YQbC2_bNUzU}vY;rr{}nLErd43nI*&pyw7)_T@@ z&OV=@L9$Ku(&0k~`pJ2*#XK%8<A`v`P@Z^s*Y>B6^h*Qug6ig6Ol96|eRQT(EXg5} zNUFA)#JZBRvAC*fw-mtsi|sMb=e{Hgf*+k^+-k-{1B^epFagV1^ba_ts;-#l<2oor zv(LDs7yAItoDyM&Tq}G%|LO@)Rnz~Qb;x(tl@TR_&>>G&K6((#B^uV?i_AAOKJBGr zOHkPB`MDpc;d_5v%H%|($@0NcW7N%{)tp@=TW=Td;$U|`x@?7GOKkPK+Z`Uy$n87R z$Z~6Z_5j~^W#_YaW?I9R&fW1E$5^1oc^jobwOVCujFzs|U)1AfPU({E23Ae_l~PwP zS+ZS6ITcxB%k^N$E<J3=uCvWkz7PSeZ$^ghS*P!SPd@jVdFWA*d`(lU6Io4;_l*ky zeRjUl8wUER6rByTUAzSnq_;J9F>cGb>Kwh0Z{+dpE}9v!gh2%)Z+)sR<I#^;RxtXg zK6%%v*y-dB(y3*(zM$UcM^oI{41<!>K|{4=?rsK*=I5{5bd}EXohJ}nx7<hnC|yh4 z`=qCwXWAzGlm2DC-ecU0Y}0^coa7zT!QzhC%ILHG2Y1=5{MOYaz>9s|2M=qhJ*=|e zIcBhc8DC+2?VN^Ua7RwhC}mE2mtc_%F3BQiqzAHC6JVw|tI&Hmvl(wjqMgyw>r@<P zk?2eTb6qUgZM}d*Mx77{Pr@t7_glLQ;cie#vvb!YYc3X3_wnwip7N*O5?dELh!;7{ z7qnA15zaIr84u03k&gPzGH6P5&q`9~97gbUEUd0XEMG&5jgOf{*fNq{cOx4X#M~r| z!F`QMnMmxjJRjB1U*j`1wyP>eb$&7Zdl>3e_y<;+8)QjKTMGMYpQhDLp|}&jAACog z?mBA#;q%?8bLqh@yLW0KQNGYrBs5~c=3RW^dwE68;&a4aTSR0OEURPp=MK*lw8S_p z+IZ{ZA`Q=gXIZql51PY9%V8(*yI`XBE9x7k;P-%hXPX;a&rd&GN-+00myC>nv8E)w zEm<1s<go=N`h6A3xgJdy{5fB7QJaRkiAuZf6XJ&;7dw(J>!H(SaS<}smd4hw{@Acm z&e9aHpIP(d!$dtC`_pq-dv`N#IV42KV0qz6z*j-pPj7Jpe5YT6Y|iY|J9t0BeGro! zPjN7s>OAGqH9ZS@z*BvZ(sq@H&tG7wbxhB{ZJp+V?R#Dt4}EirJRnW7m+6&cq+FAI zx49@nJ=wr4bTO^7gr{`mnVae02V1tghw>?X;)2^s&qEd7)iUmC5f=T<9#)=`PI(qQ zKOX0<M{ty70~d~IX*;>zEX?c66|uFB*`9O}EXyMId$I6{L86@gMY>o~SZM8Cf}J%a zK7~YjhqDp&B3&lMZYF{F``ipEVI9qbDtG5Jy{-xzPSP0A__^OReMk3go==FOR~s3O zrKa$<UwbGz7tKevaqn4b8M~1IK~gBAtfNO9S2hFi7qYGdvxTt_i_YqX)jF|X6YjH! z4Qtc4V9|Lz<Ldz!lQIjOnkvVoS@dgSB%bG7ACoqJsIUPWEF@II^%o*gXNb7fjH|}7 zUgvcvLC_4YDLr0s-vmWpjW*5V&#R~xmmW$u4TVN*aX_dc0n}v>MgMIx+?*HwMc7(l zv#`sI3ytl^+|w~Mv@A<jL!8ZMwGHtYVLc0AXqzw6tecGR__{$Z;<G4-F&A1I^GA(% zP=Q(P*Y^%Q?LDnG1mDHq2GJ_q8ZZ$Ns1$fCbkp=_=o`)0jut1y=%MbnQI|wH{s6Q` zV>xASNqoh5x2)cMGymyjN(QIeRA;+I+RMF%k3=7yN_^P+5@!LrqjsWpjO)V(bapqO zr$L%ng+|Zd%uez8`qDLZ8H7fD`OyOyT8K2o@tXGXKGJW}bZ0jV#rSzqzCxg0=1D~0 zv_v2F2yZXtG@0rwd|(lL#O&hDKGn{`b&X2pViPcLnpxEG#6Xuvy<xG73fJNXblGWS z%0ammtabGSrc^tCiJ*1nqb<Fl>+Krvx+@B`M2I(V@*uHi&=V1wxpAvnpS$VXQ&#bm zSo-Zc>A~%15@FqDH~CVjYTVbD&-WhmhzOqKsTRC^c*Wuz&U&c%{9G`&u<HZuaO=z} z2cHu(G2bYEtf3)B=nM6OB{A}in-=F&VBe{=RD$rj0c{JH8_p;{8MO;~QEBQc6ywmY zB)I*JT;bl~6TAx-TlhZxncx^Q{q^wg&!mI4^cJI29rZa6EM8@FEy5X9<1Rf21YYn@ z!dT}h_I#;kV>UpGeOt&SBNh^vFp&pt?iT!c&w+8ee6ChXz0_sJY2JpPxXxW@6te!Q zg&fTXyis^7rMY+ev{G4FDzhow_B9c>`2%mJXQlVQhy~jUIsK4P-v=ugy#{s#-{IZ3 zE|r{U)>XKaA@=HlE~wn@lhebO6&IU>b7spFople&SfYqkgXm@EDkuH8Is0>Jiwm|5 zX;VHw(a1N|7Z1IC&KxAaWblG-x5P>8jQP1Sy}R*!;hMilas`uJrgbS8+B)%mcN`~Y zAGQ8aVo3DH*9V>j=X<DEJjv<#CV35$A!dom!kPtkOnv;;*6jxlkHET0u{|awTHdCZ zRO7}WzV^@&h#ch#=1}De8JFN0(m&{)>z16h6YiZB2m3f~4JDQfmoHz2d&?BoMtsoS zJ^^g033yX2Pb*r2wH;OVu&T%~elk_B{fRHVTE0Q%WIgp4mkYPNJYBXtPx`3``;O)$ z(&e(N`4$+WTrLDWN63~WQg<lZI%0+mb1q^t+8&;+ArE~Qmjdf?-FFsZFPsPGi5+ZO zWv!WtIJ|WYd1#S-*Yw5%2ca`Bm7;Qr%t{2`oST?n9QCqttB6)J6d8zq7`qb778k{1 zrZ~qs(CZ^N0eIijqkgk|fjvtYd62J{^2h0xj}*2xin`5)8p*s_v0nD}D<_h@?fKjA z&eAC@;w>%LGlS*^2x}@IO8_+wLX;c2Ys))1cT&xP*9e2#m08R3;mhU4)u#z|`7xp- zy1$=bF3rJhZg`IRAfSBm->udr8{wwlEZlot=oucLAQ|`T*%u!$EBfkHdU%U1OHn@s z*}#iqJLprCY!fk|d(WQ?Noylq%F?cwm+qAyALwdS<lK42)?3Y`@88|Q;h`J5zAIEt zZpw3u&SyH}U<vb8%H?`HV*2*>QyOvgIG>Z|D%bm_@!q-#U&GC+<`1zR+Jo%>ZB=;D z+V3wCwr%SV5H5b@_E{OqXiqnc@1f$ywU>0}z%0L2QEMvpNTyi-KvS<W2mkHS!^gf} zLt_7Rq&hQh`=ReTHQ$p4t2uO1XP1bZ{tTm2pF}ddyW%eGs{krE-=|l<>rp}3$6}rJ zJSq~}WlNAdhb0x4ZOH#id`Kl6?Z0)fBIKp$_xQfCI!YW-TUyw$wJOD6y-KKkM@4GK zv~i!4+<_B58YmJe=})IERf&>#+)O(j5fXRo&(GB1iI%X}STqS+<1aE~*?Z&~&tuv< zNo|$$+syn~^GePSTiGCW`tCi?!-f3!|BaP@yPn*Xeszll0o0VgtkNY7t)K|gW>0md z4H5+UFLAw&hBfuuE1$ePG3yHAi3}loSG@1sxh!h+K`!PYDPR9DH%MRTl>8Dx`Wg9) zgI&2uy5(6l?$$mZ^rM?mDrV2a^i(K$nf~j3{IjExKKzt?wKnl-HC|&CffU@vtiT+f z0^RM~w>MiC947311NKsH?KXb9o@{ZO%-;^#zr35Kym&4;Hum#E0F~bAsr-17^N`wc zjupht(QZ_pn3MnQ(={H$r5?LT3r-Nrh`|$)F5<Qc<2~f8MrUM(eReV`1PkT=p1Z&9 z$&rEnqu?~rG_EjsTU7ej)smROAK63`EPVYZDckR_ug#HN*FlB;{77=Jzh{GvhUoIC zF(#96asQq!B=Jn1ml;XeLx0`IC+Wmc5@`Dx_1Bq63^<k6*jSgR!Suft-9P@ZrRFO? ztgMIohyO9)P2!mrW!y<v5q}>ioY+D!%u1#5-;Unj)BMWtm7f`sJo10^Fr=!TJQGWg zo%ml{68<1A#C+Nb-v1G5kJ#(#9MeTbpOd`TzbDZ1q=ZjHqp0tC>wTUXtUK57Qv_Bv zo@fZ9MXJO#hM7U9_RG$n5dI;FQ>t_L^)vgb_Ux|L|N4b)UiA}lnlY~TCv#saZvMGw zbcp^ryVlGUygucAFe`Cbu@0L2U?aXU+hzB|NcU(fqcoN6z2b+ax?Y0ShC&?22Z}uz zd31F2E#kh8;^N{9kv%U<hr*<TFQ!#j>n}^{=v<$ioV;}Xy5*vpj11Yq-aswfIt*4; zb~Tii2X3H&shH-At+*NF%CURwOqB_oOy9M1ap8Ae8uaVUQc16=F(jy+s+Lc1TyY%a z_Arj;i{%M6se4Q!`hHJzoKzjFxaaOqlfM4`ibrzKH98G0Vpym+>1C#Q&R&Q69UFH~ zK$O=T<zQHzu!506q4a%djBD#3?Xs6~s0Gf&jK3rAKf~W%=p6BDF<72qaCnlFb|!xh z4{;|#*4sq8qAlj|$9;a>F4N&pzk{sSn4)73^5i5Bg*!FV)+DxCD<);Y+S*)3_Ogc$ z05NuO#Ya6VBIeb)if_-jdXqt=g6zFxul=ee*+~|0J$3aS3VwubOq^&@{?qBLdNUPs zRxdgm_}NGU3o(v*b8K~YCe_nbT-RAal*egxyen^8=l|H@Qt6uNWV+MLcPqyV1%vSq zXJfY{%`SW?t}DLpX8}!ak~n1Mi#Lkm?Bx55u&JOW28N|&QKw_UP(T)wCv`W(*XUGx z_(045NOYn!<?0rkMwdaHUY=JHQdHtQmUgG~HDnI%557IEs1o5SUVrEK^}=;&oUXm& zo0EWPMSQ;yO>b<)5Ah^rx|{lLBPtJGwvfm1;g~r8I3phPS~X5X@vru!zcb(82mWE> z$Klen&SFII)iT^{nVYp;ZB78Qe$`P*ms(o)q4eRrPE(0i$>Ls{bu!ob>LuL!F0Ng= z0fQYXo6#j5y&;JZDy>tdDRh2M2yFi&ky`rI#Z>+}O?vT-cKJ)Xhg0s6WiB7LQX)Lg z@5H<+DLE<{vRE3iV+sC+#j@)6?Qfi?r9b|rb6v6I6i1gN01=zSFh|VL-EOlnYkzpX zlV`S+>;r>2CsjRp^}W@#((iikE_?14530F+zViz`l>+{-WApv8N-F<UIyA?FxCCg$ zZWnfFxstPk>e}eqJAq`VeJ-XyEW%I3X}iWwTs|%_{&4gQWn=<-MD^2gWB6lZT%wT0 zR;981uvo;iq=<GSOG)!!O19G2RpdZ|z4MjW_LYV{!Qh+2$1k;U2I3tG$0{)DJPv6A zf$n+DqI9f_Io~ql(e1dhE?I8%qUW*ob<5W;cMNRvH+<9h&$aqDmhq8A?^-^8{+v(2 zsPe_{WfJb+K6+QpoCCkR`>0snXx{=iFeeIPE3bCEGXF8@xUBK>*|>aC@vtpQ$JWyF zNtAD=&9@S7W7Od3$TKJSj^|vKxlceUm0)Wx(GCc4hT|52R4OaGC$b)#I_!^YH+C2P z_%ZUhQ(7j}Vd<EVsyy5{_3Lq<r#0;mv7ig|r|OH=1A{J+;+wxOyMwsO26Sv``C{X^ z@D_Y4MC<7Wz&40J^w@NgOrf7cOt&7!xEJ4blKB^vRaWn89Dh;emZxa%h@<I}DyIFg zp_ktqRje~XO`3)O8-NeHMhRatw^s1UhwdN+=>)#=>ldt6n))00WZuZ$kqqXTYIE1t zsd_MXRibC2t=x%Le@Oiig@l0t87(kV*x?IDW6or!zSWm^y|4Lpo=wQ#Ap=o3MNc9M z3Y^8pEKF>|4!=Qkj6dvz7;~XE@-4&`1ADzU`~Nsae(TkVQ{<x0jIZM60`9qYG}eBj zYljdPB@8!q5h8?i<Gr-0G23gKQYTKHzVzfjfBB^-853ua(W_{Q8fOycb;wHY3h{Qk z5Ky?Vx-!+dvrTo7zEV~O`JpZvr9NFBaKYh3R}?J;n{(qk!@L55xlX+EQizr^^U)Z7 z4dmay>Xc4#GjUy|5ODE`2AfjF1~ahlL>E%{dw~(>lD&NFN}06Emta(7MP*PG$fK(j z?2X9AmkTle7eBMa^mGF^Yn0+LdsNJ~OctY*L)H{%%Pvyrwb8nGQ)}$1bKL*$i#fW$ z@q2#Hp*jaFD4pbl%K1EcWgw5f@2=&04vqjj5!<}=Dr@j6e8jGs=X3bZs>{c>hC>2Z zOE$VTQ-zF#9^AkDUlT+;<}TT39aCJgk)tY2J955U6*v8KVcBao7*=by<<uPeE*p@G zwkxA~o)OdfGI}9Vv&s+vrP1I`6coc~!SppawKW@1t#f4;mp0(rK=nt->ga;VN1$Ba zSn*YG-sDiv)%Rr-@gIjBX)yHkxkJd$Z$HUr_)~YkL%S>*#f^v*pz3ST#rcl5KPiDa z%!Sa6{K|BKwM-1!Z)7lbDZ925|B_q3{w#@`OB{Q7(g~V&gi~c7N6XV1+Iu^z`1?O7 z1JCXdd~AhX0d0iB#63ex(6W{1y&T#^a7Kxh2aHzlBQ&RXAg6a2wcxn1|7k!C>;njo z*eYS+%e0s&v#xh~oTye_oSQM7eDJc}!rR`o3~l|0E>{3}jeT|g?Fa~YMcXRfQu1f+ z2fiEh%?(zRp9i$#_(k~g0{3&RI}UDa-kZuPem~$3C2)G6+m(Q0F0BneHOmXexLPvx zD=ZorNd{`wxzq8Hqu_gPi-g{S;-%|*95G%KjuMfSfg_L0S68omTN`cqaG~lm4@8s3 z3B)!_!LCbw6D2veobOl;d3$c4>oH<uN$x;@$^@GKbnv2+5Dk}N#M3)yXhB{ASKfzE z-GH)`(htjx2880ct-_8>s4%Y839!ILqoOV2OIXXJ$jRQ??z2VElrSx9ES<$mO}uGX z33?pkA(DlviFN%F$5fjW?HJP`lVNJrVT)m=C$S)ii)2{$RR~rj%7uqq`28wX+^SF` zvk^+Gy<rF^kYStcE$qp84Y}o`{}V)M$SRbiHN=&rX1AACLK%dlOdmwjxlWFhaJjwL z5(XD4b{1L}AoN{Iytvr0!ELTC>7TPyQmj2aE2|32L%R^lRc1^xF*RlSO&&}U$1IlZ zQ<Agkh?Le5&x1aib}<TCZi!^*9om6P%iRLZ5Px@m4TC&AHWy_yUG;Sr&>HW@yliNj z6N2LcY75!MbaK01%+cAr>L9u8=i?0ADU*UXt;g}q6O{uhnHO2%BO1%*ZZj4ef{U#u zkC_cQV|9+$W=77cr{Zszel9{2y;FGj20@?YI}~tYlvXNndF6Y)*T#IsRe?e>24{f@ zpKIxaKD6Y%Xa{QM&l`(CEBbR2MvaSZ0vcAfydGt0-p0yS6!G(Tmr$j>c~RUlK2+MJ z1_`T1Gt39sS`dUtt?wr%{1VCZ#r0UP+OBMp1Bs(KdzpO)rRe*5^l`>J^*|acE8Plt zoeXEiBw3|a`KQg8ksDPfgs2sU@VRXOpoGv$P6F^}Tu}>Jw@|k~>V;e|=7!AIWG=S} z-;&jg+gTaknBaNBZ-8A9BYC@tWksoqo6xrSj|RgiLo4Np_2OZM;hB}4bSRiXpR5$y zm(U(V@N!!K;F%?=c0MQ492ZCcVjxw8ByCgWzNml*7kJ{*Og0{Hq^8G*h}XbhZ8!Sc zRG2}4-h}Skv#|`pOU)acI`EKbfzZ6R+1bsiq$H!_2YXHkH`bf|lfgnP7TN^Rh;XZ2 z?bRiLZK<~CG1f!T!iw3%ZM?7DW-v5y-%22Nwq3y!;fo&e75r(+@4d&2BKXx9_VVMi zMVOSc*L@ZEDDzzH1O}_MIIUL10u~#BNP)tMQwZ9ReIeK!7!0PsaS0=Icy`$KaCc`> zC`1+K%SYc2fL8~`P`W|FRu{ZHlOh0DgK1W<bv`4a2+<$|ri&@D>^m?6@y(agxSe$f ze$_Ep<kS4FX<%VOAA@Wf2kAI_CMPEsyB5gL&yTIWNJYg{T>co5G)UFP67~FW7r(RW zxwQberEmc+P=uJ8G6mb}U0$r)&hqQJb2!(4O+}b!(Nnn<6bR#HR0Ur`R0pt#6*MNW z%@#FMBT+uW#nja?AMynstKEBnRO&i;y7E+di(Ehv5oK=@!8t&c%HWKty{PydlBjv& z)YnVbWyFQUJT6A?ZX0`CeY@Y;Id(QBvFpDP_SXd04U);ek36&XjtH@G?^_D<_qoVL znx6hA)c-j&#=8$fyf-DURL@KnpKVWnHqEq3*FNCQ@ky5C7yOK*h|I$B(mTk-7eA`P zghHMkq=7gOSwH&D?Ygaf<{>@b`=UhT7ZVmvC_hX0n_2|7OHlZaAe;vt?B*V<h_86t zNjQv&UpeNa#T4b<WvZ-UE*EzyaDxF@li)V7*LnWqh8VE_tl=1D>0F#$`co#-n0Ni^ zQG6M_2*>8+)E9zwQ@keDf6AFZCH?7LU9x*b0#2L@NdV{0!4LgxZ^zqYG~87Sv!i)# zU6>^iXeE1w|HJt(3T()BBKWXG*~utFQE$y%KVBWs(TTH!1|(3RVmqfb(FM7qB~7jd z7%aGdd#5a^oi(X5N6)SmB+x0Yy`4B4kWB>c=T`I*wmSve2$zK1Sjxw;dKFU7`;nDf z=F+6+KlQYKr(8h74sjN4UBMlw;&SaP5O<r#58#6Z@)d_Jn8iMH@1~+U&tr_dS?{_v zPuT-tF9fW`FIJVR#8dD&uoL|q<SLvV)E-IImJkor;)vCHd_~)ozwjlLt*c?Ye+$6H z!)ax7IS?(<cHd%idfo}@dfU}pj^~;=08}yLqS>JcVy?|`FV23U-v$I$qKn}0w%*dV zLPT_@ISrLSw2^%7e8u7*X^6Q+?ssdW`_>GGek04kbV$sYM{#9rhjDn>fYrNyRN}@s zh6vl?HNRk+a>%40RlPJrCF92Y5Xc9^qjSq9tM{B)W(w$QZb<v6U(%2|R9|xthIO=t zE!8b*-Kq6H1Y|h(txtqyCUegrMtE{xnxYI<@vTk_gS)!@F%XONo4y*n6JwHVI<j3G z7;AF{;i3scMpS-BZGdBA?DS0YF-gjgyr^kiOgBJzx{xwvRcM4+AEgD7OMqxB1hr+| zY2y5&05~UNq=-u^fe7E!FgNrZu;#exN+dm*;G9r-7Zn!{mIzsc6T>67)dp;VcXMSh zER$+Gt^O15QZ69S$!I0*`0DI3CU!hDgyo@YQM&8>=`lAkA?PQrj$(vSoP}x9SQ;k9 zjdaXjPk!<PH}})r)8R`oPatmy|5N?)eRp_-ezjM{rNO8?u^RVLCl1YLT$JzmXc$&( zbvbPmfYWi&pL}xl`whA=`K$9NG2z2ukuj^3?b;vCXiRAC`m!ppc8CFGWNF2mlcgaP z+b}m#EJrFPZ=LW{n@&He2qfm40QqSziEOuiaq<>S<PAU@fO)mTy#f;InimCb4cX|( z4U+)UINi3+l9GQJs`cFag@U(tKZ-ApZ?pyzxhO+en09ll#SLJD!8;~~@866te^4zh zv6xDk!!&b=)S4<-Zn~@(d?U#SPo56CeMwVXIJ$M@L5oU7lcU}8G<~vHm=bda1xHU$ ziDOI=H1wuMmYPNhwb~tv**IEFWp8zE8ZD%HUuDN;I9YqRyljTD7G~(Rl%c`}%1^MF z*j&qzea@`@)&gqbs|tf{ojp<o`CsLg-zyt53RQibEO?<MmlTgy-6m>Kk6tbcxCN|Q z-u2$tExS!Jlh+~tN-f;)sVavt04|9~Hm;`UK~C*@-rigtNS&gD@fG;)UV5?_Vr!KB z4Q*^P-me5S<asC~gb%EB70IA-YEN)^UW%D9T2Np!Q!vvUxgRJ!d-7)>_K{6E_9IMy zV8{!osnNDDmu!vi^}B}H%!q|$d<1GeCn&c+2hQP~y>Q#1JzA~bkBReo@?V+uG@tvc zbY&<Mnsh$GZuXIwn3%igT#DRfb}FA9X@(zB7Br2_ln@@9q=+TNwvAQD&kUxnjr@*M zGv0;?KM?o%pHox}^vx^w9yb$Jn+?<K2la-I$p@Bx>FIac=Nl3d64s4^gM;I2=h4G6 zvBgUFq7K$}R>!s$`tzPpx<y7t!YV4}#|jUZiFu>qkU3ZvJS?&DHBvfG+Dssua9O5+ zJjNi3(wm8n-ObLSL-3komBb4vv#f6<FZ4;#sVJgCMNFcparO2_c<H!$@@<viZjuGx zlW}btA_CLBUM7BC?e4mW2!yI3f|vAXPxzfZ7j~P;=f~;73iqY@byjLzOiZo8#_lnI zdh&GBeIf!sdXq)(RU}G;)3t))o8PqCK9WZF2W-E&!>-e%fKjvW#L-Y0nBwH>H>I0v zLuq|Tk7<1NvOR9DySw|F*pI;gv$z&25&Z+TOG`BwRYCRITW&iJvtD>S1kcdFvq;h0 z8zQqJL7{z248J`0D?k)cU%>D`Yxo+!e?u|TO>QqPnrqiIiG5l1j=PT$U#N~D1*cC% z=@suQ<QI$b)9f4R@aWeJBuSabEH|jm8EQLSCjBkN>=Pi)v5inlBFz<Gpk+__^bF~< zx?i-t1NB{bv;*MQ?z2?L(bQSBk%}MvlJD<PsHf4-6?yqNYf>v~R0P|Xi0wQxu-$4N z!`oaPP7aG%0<>2XRsuy!D!jMU&nYvAdj4i_mXY0yWU6IMA=b}3`^x@`(_SWyhc%D{ zq^i69*w}2QsdmQjVP(#0ARLvPX+c|7aRvTEFvAm^RalYJ5OsqKsiVya2621wbMbu7 zo%p{fiorR<Oa|b9<Ya&^GQ;_Co_l%PzPh?np`FgfWA148QTclYS0u4a*M1RNt6o9N zy<=LbItUKHABaodT(#pIDgy4gsz>9YH7}T<n|=9^Jk%)M7m+@~17W2vh4I=<tmx-h z4XxFD&_Y$a>?$?Bu42z}=S-i)f%!{W7zX3xg84yC{B1(m5H29_>&0Uh@OU?cYO!}$ z;k~)1H4^$YvsZ(?b%9D8s>zV3ZE0+MJqB92xfI^RttmcoyZB?Db4+WfJU-14!5rH% zSRa8-dMAPMbkq|98k%;+{`hH=bDoyB^ceYQiiTX6>B2&5^TCvlWMT2r_@5~E)SHps z%>qSO${E?^jH9N4yP>sUOOFey7Hixj+zmBj?1IsIU^S6Np0cuceZ4t16-+iY9KuLd zBXxJK9<B6mrjb5u=i^$=`V9aDgviT+!*2krM^SRN2=K?jA=QyLf?ss3yXmO<O0b-r zz>I98q)*y;rrI)<aW3d>$&HY~Syk4w7iL|CaO<XecycFnA+-Xk^QjWc_wpjiJm~)N zNE#b-ERTYRi|Jgvt+@h?-xsV6dpCJQA=c=7&P@@`d)5+daljvE6Kzn2#nE0yT94z@ zVbSW@8rr<N8o0<3t$Z_PZIlU1)*BHCgl(SblHQ3CRdZ|Up|2%G@kozL`wG~^`L*g$ z@Hok;#7~ZWe7!q8JzY+?YB7s{+gK<&1bZOo+TTufAu(9@Bm<9T8B8F(KjRB09cn*B zrYQwHVTL#%`zWuFNu=#Nc567v5i<z_Nd1jJH{4=GYBP9X`2Y??KE+;A>_{O#$(Co# zR}IY`e_S-J{feg5dv8Lp^*N%CG=1l`0yc^_bs7m*u)^?Y8P@zFY-_GoSEY;w8NV{c zx@Z^F!J%f<@ex!E_@HvCptRKFP|8i$t@`uU>LAC}iz!A<kJ-<X&ZR#GPv3*R@K}-< zou$V8uIii$OekM<l}vD)xwsYF=Cgd@qHSPFFi33AId;I5#B6^-%7mEe?`m?MJ=t|k z;(k@fy54;&ny4m3-hMweg!8)O8upe{2onGIFZK5C8g4nEdG-<Mlz8ib_z?q>6dn1E zPm#p><yMXAF(sd0>*Zfy)|3)(eUsZX;`aMPqI<f=tx?&x^BYH;rIr6J+x|1iIv(cV zG4j?6MJ$lG-!r+E@2B4y^ilYiLX0@r7U5cED!bbC<!Z&2Oo6G~7NN3Xt?Y$c)lX%& zG(OVT5GGrPO%IeZc_ctTY0$*TNhPAWGNey)dIxQLMRaFx)EqKkKLld3G_eAv99Xha zS9UNc<K6#-GZ$%lXh>JvjOle>v^Wo`L7XZ|N@NEDy_9iwkn}ezAK36yK@A~j$+=G2 zQD3c1(iO5__qikH4-LrGshM?N@g91Rn}XCbIMQbM(INL7*<&c=pH|H~WK|S8R@?@u zv(unN5=}A%+Lbl>9d%(oRosbM!yzA~*TNj%W{;%<3aw1jZ3bkcThQf=`NKt93-AT` z7Olx2^@h^LHj=!nbxmlLuR=(ZhP=hYwoegLR87YSKIaXHf{)jSGx04|@>oNd<Dr9H z?k_L`9d?fb*mi1t`wrW-QOU{Q@4`mk=9Q^weyPcwR+J@5w{ZCU5E*#oHP@U{jv+>f zse$-dwpiZ2xpa<{?1VXh$1pz^wYe`2Mf#h6Xt!>ryvCXZFfu|ZW?JVx^0G#0!4g!g zn^qEye9~qhIF*Ft_+D#{OdC{_b-p!Qg@w;COXb$HU!~vdFTM7Mva+(b2rc_g1fJJ` zbqQYY14&7Z+!2WV@>_9@Yih}yvKI3wkVj@HcA(2L?CxI^#Hba&)AF=NrKp2y^BFo8 zrw~3F`srLh89*vIC@{cd-i@mU_Zg}GT_UR3apr#4MlNDWTnoktotqi}XM8sCxF$cl zr5Fq=q1M{U*H>=T0<OFxnG8>!_P=r@8vF|Kluhh|l|zeutvnNLh@n<f?px@`K{#A| z5MPR_Wj7~l+pz2L&0f;mLumEdX*LA59n2UFi!BY7yKXY4!0d$`h4bnALsCVOY>h=z zwx!M7xTo)Tq{MtuUc@fuPp-Z0d24%X8GYQOT|XHYc`e`4xd(nE&ydFy=4$!gG}4p{ z?|K~dOLyfioNAbO7b2tbeo(>^bLi}l>04YZ(+IC#&wyggvqZQgJ|&N#Ra@C(M3Tel z*;|7E-`S%V4`Py;1n~n;OZQw^!O?kokoT(*7;PD!dD&0LXrgK4M4VS?N@b7rTD&*r zq<5#1GxuBd<}Z<z-(E$94b+anxQe@?W<FcciZ`o=qTEws<21gj1dCW#eV)!oes<8! z3+#1ZQCAaL7_06zt~mU(rSuX|j1^>e-TPCzO$PAbO^ue??~`O-m##+~(J$xopI=lU z%-6-T9+k9<Qohk>9QzVTX7<?*(5drk2?o<n&?xuLf7<i`OY6kmd=ZLFx3;Y>eCj&n zz!_7z!_sljhGs_pX@9>`#iRu-Yt^mFaVnAYIbhXe^F>w!J$pyL=0W<*j=fznQr~t7 zrQ@r<G-of8ZaG+r71Ea?s9%;cb228W&fF3_$sW6EMX%ZDc<8LZwJ!a2W<vW(@IF&_ zXeP&A8GZi-mIVo1?LhnL6w+x`t9GMC!?fZ)I4W&$aKIT$q0pttaZA}TaU_i8(PK@h zY&2#BP*u%|cv*_+iLpb6Un}$WghCwGVC={uK{_G4Lyf9nZX=bGd}qPlh`uOcTVbLN zmZTz5L~?W|{FuoVtsB?-&}ek)_y$HuNkxS!nQ?Dc?RA@UinH+_=*m`x(6Hs;&Hb^S z#VVC%J93U;%0N82^P<QFqUgd=jR*AGY%Y~?4d+LYbsv<^S`><|+i;m9ZaVEY(HnTM zk@m};j3=Ku(py{q(Obi(i8Y?D3U|EXUh`NjWFSww@anhRO9vO0mUnk`5)%`VkrsG~ zB!=+$y1;fxyRU_hFTQAEmiWPNK4v6on5jP_e*na~a`2-^C{M4`iukHCH$64BD@ehL zcU35VY^X)y{&a(d6qdtEkR-%Dx%I1v`nY=}qV;Yv4id}1x$uqb?X+z6D!a0n5S<LC zXFNu={R2}A^Heyo-1vt#NC#xO?2*sPkMR95f3!TGO}n`s9NhZS?BSsE)<(HeT{lKv zCc_^;fovAiFc16(N&o$q_7Q_X?sZa$i2Rm!+uFt$Gr#qdCM~a?oH!c%A0OdFlq6D# zlra9-fL+fUSB?R{(qFtI`LIj!yk{IQQ4f}I4#zT)^-?h$avN^SkP>uG_@&4Z(U^G8 zuIxM^5)gyi5=-_tImse?><Zl(qyDjx=A_l-`zsi2;90K=GSpU$x1$8S%t;gKFO<LQ z#bMJhF&#$N>|y+wf`3`QDbe;728FrKm(((%wOSazgH}McY=r|U5nMVmafh}W2EAuV zU6`K<#Zf5p6gRE+^*yLv$ZK8*Y}ZDJN0b_tW+v%jsABLIS1E|Ljh8*?VP*gFaC?q4 z-pnZ}Y}Az;#A*@~>Oj1|YuQt&zcNcl5j|FrL-U*n*Qu7XPcqbHlAh@Ysz0CqNLI5P z#E!}uwaHB{dXZB`QIY?-pp{9k?d7C3!%`kYJ3B*ep5Bl)as)1So+E~fHBPCe)EXO( zK0EL93`5g}{@nJZlA>cTIk!L8`hv=u&2##SKC9i~ZeLjbGMHY38c?jG#SM;vLk;t? zS7feIA^<r{t(1)b^r$NPf_2(4Kr#OKlqTnx?0ks0<h^y|g9^x)M4%z7DA5Xa>MhPw zUxAGDA!?zfF<%$+Jh-@~v5o20tN9#4e9)|}xW2ct*kv1Q6L@y4!L-YQXK{ty;;pP` z`PXF?mHF{D0B*XKh|>I_cwMe(Z2-u%FGsuVzzcYeGWE)PxK><L8z7M)P}&+;z)275 zw#-(uTNpb?#B+jY;>t!M*lY;`8&2#ih$S1j{_kbsRevhSJm1+3z&eRp=39YU%awr{ zA%P<furaW;t@+oKQWw(s^@Jag{K)m4+}>GPKXQGa{&sz9qyKV!hu8R!hiFw{>zL-8 zqGWCx>z(%E60M9Snk$|U6SYeLfJa(Iv#g(#?$E4liW+J3w(A0lW4JKedsEsp>S~9B z)~PO|VtH=_px#h!tYhvH>XAmLj*L_1>LP#Mf$uUwJ7(mmrg|AKl=ecNR0L5_z7J+N z5|n+1c!;&L>CUJYpSD^HkD_l*;eFTes-6{zr2Om=Hhh1dYsfK7!}r#c#&xP!1)z(+ z9N@ylukM|4W{s@}*WNbSNZ8eb1z1;-VRb``Iu7;<uw{QPEHAb;dDbmo$QiB~Se6|f zSc&Wa^}~`BQiyK_tH6w`u6h)74)xO;4Zl#My;;@qc_DlGV!~crW}c#<CYmZ{`3GF* zw&|+Ky7rL${h2$$RXyyV!ATb=zk)k$Naq}T4!_imZlKa6V^Y(fwYMU%_GU%&zTV-S z6B7|kSvEA0ZO^L(fp*esya}0w!G%zV)TyE7SPH)9(3PiU__e4SUwmd{;q1r)t;AsG z8NtO_SmMYOrP2sh7H(5_jPk56%SIrd_c7!Hbed>s89FVVO=85WUnk0^rwS3a>jBP9 zzH5i+VtInekPSeiO<zj*?8v)eJy;2cs_zCYw|8Yb-7}79GDIUzn9x|L+}TcZtNHfG zj4}prr;Fo7N{&L!)24GIGy4iBvC@%Bv!7jT@~J<92`TJoLhDo$p_{;S^$(*V90brt z5wQVFxX3wo0ciiJlxg2dL37j2V(RIOYj}D7MW(Jns)c^Ua`Vm-E&f_*)(DsSK{eD` ztl(n|L)EKYG*|485Pt6IHsLb<_&eKo{2MD==t0`N^qC%>X;nL5{{_&h^k3aP|F;^e zEqc{17a0+8PuX~>{-_CsF!nmn(ARQMbf04G<G$J3u6v%0T|rXFZ#xhzb1!X7X?sE< zOm*PJ6bvTgGb&oua&fQeTq6uwl{thp^RXXEA!PC^1PfqB4vN1A`L;`Fi{plszvqp% z>|K~pEytwoFgkz)n>BHJhDq^cEEdDYa4y?<1uhVnTN&PgGG{FBL3gI4W8NO)PLwMj znfh5M6p9=A#O6p$_8BR?xV+CQg-ug(&Huw=!C5r0VFJ?#-%f|13``a!!l1nlltzP8 zcmu_TVBSZj|M&X;jM1O%l;t62aabiKC0yaxuU{4J6?A~_ZwENGzU+}WI3QFRB0XtZ z)W7~sDrnga2${u)?kSXU0i!`i;G3JoU8&sCR=9|iJO^-#ibK=&Yeg}6Owwu671!4z z5m<QmhMOEwsr~<<aeE&Ujs?|-DU-I5-$0h>{@M$m?&&Yl;?&o@<VfpijipmkKq zPy`ltPKXOI5w-3m)rw$yz@=eOmD11QMXyA$iFI)FT)@An(LYBD_(v<|-*vfg=)|MX zc+e8|hV-W;^6I`66pQ{Au|}B^C}UKVzoa&pX%mLMwpMWIK2?Sq9#y$WLN3L8Sx-gD z0yF|0E{0ulx~JXaTk7;UY#=x9lL7~0B!|Wzyk^u2D0j9WY_<=gKo_j!4D?y8<V6c1 zGc^HY_SOu2nniHuds!bkzK$dwlj3E0E;)<S^<6oV_8t=3d}0bI;=B}IoL{jEUCxch z8zGPdz4=x?{Dm2T)z$8r%SkRQ7MWX{xDUeJ+0{u(T;H$tJN^OR-h6p4XG>W>BmDF2 zEx=-;+SihCPoa#8{oEJv`41he_-nr;@H-~}l)AVrdqeYdO15+#ZzLqG81{;@g3N7o zp*niHRoNyR&rk`7Bw+$c9rB6sn^TfA>C&HOGSYm}{IMfZ+YQqPKDDp@6JXh48YW@} zgMBM;re>xl9*wV)zt8mB=w_@neJCr>lh`hy7<ySUU#_Q;oTq+h!E#YtfrkpfeK(&a zOW#%+9O&|{U7^cx5#uX)dEkRGD<46h9<LOG{6nNPL$AV849XR!k!N=H7&v<1coL;k zbKm2~+kKZbVihRzt^IS!<V#5CQ&V_~_OKN80J=;F)w>|FVXJ!s;T#Bj{57_4*Enkd zdYD$E5IzufpibA1^x;RoxjhlPuxeBGqKztsu-3q9HUs{s1!O+_3{c8)${p;vv6X4D zQSNSZNH<D)wr<%U0b?rTcvgYVN?L_Ij`&Wuj<&XUBb{T)5<BAZhnYb%EFfQn40V0O zEKC})H|nhn4Uc!fZFT;_Lgl*1NZzP7uFD#omS-zIwp0oCDIxoqo)L1<ZHir6duwv@ zbButIW=Dnnv4O?*Acv;?7e#0KcB&ug5+#R|d$8|C5Mjyh*>%gaRAy1vKF_2D*I6^N zW@05cE5ib=m@OxGmzoj0Op3GMWPKZr*-4aa@42kI3}YXGM8wnkmn-31I+?aZ&q!PN z*VBmbLmAjWzAE?$__tUsdh_$Rtix8bfE{y2NbPe<%C@Itxu1$t2SK31M_Kk28LC!l z%l>5T=kWCslpjW@Z-9zxS>FTS$Nt-eg&~W5|7F7F0WTx3v1r!dLQ_dfdwx$&XA|Q9 zbOeFD4!?zJqmy8ohO`6SWC(fSZHo)t&N3;k>7F&!U);DEwh=Tw`3Ev9afj;)&m|tY zX3-keXtU82zjFt!Vi>qM#Uwx&qSacV^K98wo@pkQA43m1$f$`{Kgg~=#*(lkJ5rFX z!QkbW@7g-8JUu;sZvK#nysgQ&_oex9YL;^IwG#5crGANXZlk466Mh@0W^PNMt}`oj zjpq9$-9&?opN-B{5aS5aHjnY?UzKz3Kb7-HKdfT;o3HA>OXpL%)`PWvNCqY*kdssC z&YI0(R%tpfL%F@s9C)|CNwRA<cdd%~XQS%SmeXwEkxiNusehRrudPd;RuV;;Y8!k1 zDv{kQiDj)M(V-*X`eF9qO!L$+^6jk$=Z_@luODZ}um6^;SC{sEepcW}R$s)V9cJ*W zjfhI<?~>QQ$W)Oe6(ctBN*)b{=}|q1o7gjTWRlJjJ8%>1-u3<i5dZ$`LGiD_*WvzO zH%Z^(w^tvU{BC)U1I>k!h9807?MlOgv%HTZepkKHHSFXou#tQ2D_h8pgITJ$M0tt$ z{$QwzFk6$IwA3-JOXY!oN!MeOcrW`n!}9%cKSu0{%iwBA;a9~?wa$p=BqQ@d&Z&H& zD`&?|G;zF?XFKCYvi68|c8=c8)zIg73_t8P8`$RR)#aPnnz`9m-sY*YvoTQOH`erA zz;-s-C`*on7TpwoYBdD$ZA@qTbKxMo`rTYjcBtaxbj!JfjiexU*ILg~EA1ef(t|*K zt?m`@5h&^xwAiL=3WLSIxT98R&T&k``nQLj(Yau4tjrVzf=Y7;#Co+0z3=F1G{DWj zlgEznL8oQ++{m+Ec^$)s_Sx4uCeNncf70lvG8us<wmjwJuwAIx%NqEPw&kM+Dt>6; z46N*39BpJSvb13Ods(u$k@|hQ7({7K;LvBimCyfI4Y&5;vv&USH-zf@RwWpf{H$<V z_9)aC2<`BZp=QgVW(F5<gYJQti$|JDg^r-L)P&joUIwvjHO#M3Ej1M=VGIwizIIX1 zbx;lLVt~sa;{u@N8}E1`;s+GF8^D+Z*eCPl3c72t7V0K|4W6<}JwP*627&vyl#=u0 z9nSJy%P(Vhd3t(UR)m(K3TzLVl92{^XYLr$%gaH}44xXyqD({+9P{%5Y+iq&FQ9)| z=fgF(t!6{i;Y78KZ~-c6Hn;Ck_iaE|PrGys<+((Fa>Z~^{}mw3W)ol*x*(|kAgW>L z%9q>8$+IA9Pa#-adrv8s4@k!yEJhJnhSn&x@EK|Kz^~Pe7Vd}vz3fi9q7OfYJdXD6 z)~uq|@bG9364LydTn`R(R@5kkjhf9)IlLylBtE~uB|}a}CA5rfIN-TM+d`rUT-4!1 zzJ_bP89bF1T=v@sGSDC&PlOHNn^gwbBZNq)s9bYL_n0WEPP>ccI2YtEjMk8SyDZZ% z0qO)A6`Hl{T<y#)PnmUFdf9RS>o5C?dvR3UeyP<B^IpCdF4NU8S;XaG;s8fOV~5|! zP}X`S4u}{>k{({%>3jT4gdC1o9v_9s<<|-ZTu3oWBNd4hm5s5!qDj*ZyP?W92Vmgm zUm#v=EMT~sJ5o+;`Lv>ln?r9m<t97z=XC|Q^Og9&05AJ$3SZX9MrOrNCmZyu^^V$6 zd>9!dV1_<*DeFBUj18CEIELBS1CO3o`2F<7E#JDM8W&t!*F3a3#>UG-u%+o>1JdHw zul>UF5~dyni6o5<lw@b)r1q!7>^FJxzQ3wqrnU`FwGzV5l#%<8AMfHy>xN)(j5Q9q z@^d@x<25K=@@oh4jM3oQfMw0s>&_%4z;Rk?dz@vO>c{f?M_sB<vxk9vxMB!>T%PY) zi<|1XfA=llpadb|+wx8naX^ZgYyI3hMW5cFBUfSjEDupHR5W23tlTMwV^^If9H5y2 zPh}^x421CS#ej7~hOR08BNkS==R%{LiZitaR1{M2nSCoFH48s96m35SngJ6lU>Q24 zxmyg{j>{|cB-?mSoC+YSUQ{8&>rO``0O9r*31H9=?2~BUph{Dn&Sy?2nCrFG?23!q zv_-mk$}r_Y-s7>g#0CpwZz#`LUi99M^`ipu!j=NKMI67tV!_7&H2Sl+oojQL`I5Mu zDLYsb0KUIDvrG@>sTjw3hbA+z(+Q>6?UWG&Q<LPiYv#daU&8eO%TJZ7?9h6h0$p+V zIr;!;l3?cRa&(82^?*Q;8~f`^pE9$u&Z<xmHjav66;GQ#JRS4oUvnZ|POji)I=x&A z=$A(KstGM)fhL_in~}odxtS@mF#HykS-Qa1&2sy3)AWSy4R0MIWuhm3Tu)cP5s9Dx z8VdZ`?tGT`hNqF%sHiK%cIR5cKkd#t_w2pB^E*0v7y3q@V-9mx-8qA~^)GtT`@$T4 z@YPuFzauzY2x==x5s|pgJ~p{u$5n0KcsQj<e70hm=z!%n8gPWv0JDelzY!c913x&< zvr)#3S*<C(RrRP#jQlr1|J@!Z_{+F{7D+6Oi2b>e*}>cayn>@$yFyo=&o3tq$X5Ua zr5u11Q>x8gx`C^^Mej9b?BMXLYI13F3P`d2Trs@llgA_{`cgUPQDgJE0h6ye(cu>e z!9H+xJt!ddVu|T5S}-ybn>5K!+6%@OF&!%Ge^(bJUGqy{`LztLqutZ#=Hjw-?rAdi zr(D^QBrTg){qk3iO!$1}N%mN#62eW;BZ;T4K&06nu^0Bpw*T(6q+k+u`#r?6ab0@A z@<8nY;m2CQ?pNbu=n%Q#Z*jhU%(BJpx>wQu^#Pq<-CP0x>gEDM5X+g{i{M5+HvIeQ z3Gnia-@@{nhdK337|^0|VcN6X(7^|Gg4H>E)Fci0E5ynI#7Vp_zWFrm+GO>}+V7at zUC0{5&=|kI?FudIo!w$D)3xm&Tc=>m=2}@wl7^w3^Y9zCVYjyf!vn+WhJ_Lgz@p)? zo`JWdSHh+8(j#(H+O(cjyI{;c7ub%VAIWG{3k<r(?zT!7f*xi6P)P4OxU)DKLt0t# zHSl|GVCq5ayf7su&Hj{<Jl_?hEks3|e<`QNNP{g(OE^i*rPjsQBP++6Ez4cBz&Y~^ z!F;w_iD6k=K^_Pkt}PfEK#6Vc8ywa5F^HW$#<>aD#WDkKP+>QiNQPpZtPXwS8{iAj z74nsmMKpRVA>5$<pnkN!YaFJ@v6GyZs)y`PlBu%Y8AVB!nspiVyTV4rx<p9NDeF<A zA9?KGPLVH9)o(U)c7f|H@4uz`9Y@SrENyImL`J?VE{UNlyQk<xMX##4rxvawf+FMd zB)YF*sVujkLkeqcj*jgf)U4Lg<mIli)ta|Xx7&SJd%lUdZxc}(Xn6Y#C)}A<)HkaA z!dyA;tO?7c*61+0H!A)MO2i@NaY#s?cFd>NmfDdE;3RXUBXi(87q08lyqNSzM`^}U z=UQQvMbRZu^#Mo6CdipCsSRR>ct=MkYsI?(YpB&_5z-4MitX36-XIq0l3PR;6NK7e z`sp-Lgz?1F^U%qsQyQhr69$5M)C7!`i+44UgW^RCSg}ycSr_-}tk>AOHuI|rDjc=N zg~M>N*3~c2uXgPBIIqGNCyT?9XA%A10mDz=7iHzV`ai{@Bo<*?96Az(L*Ru++nPFh zJ+nB0J$TB-AG3sT7ApsB$dVa><<XyN#0#3Hc=Q%n2GSDRb;+$uwb;UPb>0m+*Zv&9 zJJ6NPtPlC@ol#)3BL%HK$X=FOXwdsHqPreJiZ;|ulPfOJ{aJoHS0M6q40Ums>3#kY z>r6G7bY#TkMrzmLg}Fo9rDxann+&p-?6h=0B_W<xKq1`a864MQh#hjaAkO#20oCZ` zUSr${C7155=;2H#6T6$2%a}Hort1Ji=t2zCTNfIXv{#+4n7N=eTLgttZ6D-s6^0mL zP(lVR9^*H9T+?EN4+mlsXV04TE|*%*Q>85T&Bh8hzGHZnb2Y7ms-DzcBFcb&dUJe4 zZ|<V7rt4Ts7qO%AAgSUOBC(V`q3CD%!Pu7EMi~_1Mz?zaS?+DBSyb*d_V8tL>*n+F zK?QpQI&5@{d)ElpKzsYRB=}B3?#mahx#>Za-m?fz_C~Br<sY=6$+wFks@O&su-beb zdKe`@i|7g39NRj4tp7-?FuW(I;W6((#!Cga!Bnj@Z{w6<^NB}WZ8fLuNI^yv2hlDO ztF3C*D;@?4;F2Tx6WTkajPPklFr{g_5FzD;cGB>s=|IaoCz=c772AK9Y;R&eO7ob* z6+LKG^~%-oGJYwdtUS#wrl?0{_EBQ^ygSc8djSBA&$Q%%PWR+fr$oer%3;@+#?pB+ zsM*|H#%4x-+~fBW!zb6r8pKW>h8{%reJ;DAAsSP(mCj>}G!x=Z#$F-0<fGVrjTX_Y zQ!H8u`>^_UhWOeXOGs^jDrf$T;d7pR$6Xf|sO4aIMZUgJnmlt_r=*foVk~WKG1Ek; zzDJ`Tt)m%(mQDCHvE#hL1vI?H5k)8(Sk!6x)O~=D^R$+Th#QHp;u%TGnw5!WU0%fQ zX<$<<Xz=ht%{2jfd&&4YV2lAx>A;_lmh}jfW$ZJLS=G84n>gx(TkqT$?NiSJ^e7~> zK6Tbn#`Jnx5Atb!HCuM!*({tPsb=~;r%fet`LS_g&aIi9()ZH6MDrt$@;NklbPXYp zVx#mjj@@Ew%Lm9udF{KN`4YSlx|2J@xe0@=Zy|;GDCLs{78E)NKZ&cD!#blKF4M!} zGn@;u<<B|1gsa_mns_(%xd#y`Tof|%vLJVg5F)-TyXEcid{udMc5*SYzv%PYBF3{_ zC!f13N$hL;j!|@Js(#xXCd+^(nTG>cu=D!bCrJM8hP~ayO63iU6w;6Q8B_lw+*}&| z58;Ms=v7h#m$3;mNMLh0MPDun9@v=nroorS36jX=HBnG1DCCW62xYJ%w3hOMzp1tu z2GAPVg0%P&Iuwi!1~tA*fx8R@+~a**H<lti0E;xV_7o&d>~mKHg@sE#apx0J;Lye0 z-F^O?ej<k085ON~k1(}5R=e1rr%mAqc=PKaH1r4ZKQJk0#3T39j~omln_=AKYI<CQ z52zBRhi|~`*8yoIiTRL`e%jFu-Ad*ytm@DOHv`~KFixM^$G~Q3X5Qb7G+FYM{l!Ge z2|{lqS09?cv+}fK<lulTGL%5>o|>?f1r^>2`EWzxgWYQ@QeI4$^wC4(nJ+wULb!%C z-fJpj{qZK^$DE1?+P_L1#m4nQd`i%LlFyWkYt#N}Qe6Ap(Kuyjic2||mpHRF!h4V8 zLL*l!|JCU5M#of$yx))aaBl!|Q_u+!DM69RBz!PpE^SAeCSH&I(<&HZS(l9SstUoX z9lRQg^f%s-bNlDvG{~aFjIsj$EEkba$=Le5UmR5PsV%S<g$|BzSN9ssk(9aW3UB{1 z5g#~25<5GD=2VA#?xdfO@q(N@B2@5y_|!J~tey@Ul_fzJ@|wt}dc_SD0t3lt#EAsz zd)LZ{WJCd}Oh~T(OZ_G4JlFvtkNd=@+LkRCyA2+iL55Xz)I`Wfa1EHQkYtiy-IxEZ z58*9{tN4C%5)ZxXR_Q=4$jek3CJXF-s+^%N`PNcE>~}}~OC|Y?22TsV@0;dBR=IlI zd2qV{v)&NkWhd5THSWt>US$jXV4LV=^c!kjdc?;Vdbr_;mch9p`J2LnAl6te))|ID z9Sqv;dAGa#yrOJ;ufu-5JINlUapG&6h}p$xftA_=(X=iI|ILqPyZ=M&G5mj+d&{^c z-}ir5ML=3XL|R3qK~X?*DoRR7H%NDPY$Bx~C8g9rI!8#y5F{r|rE}C69iw4m|AC)x z{Qf_9a6h<T_x&7pUDsL1alGR=#|Hmb4wLu0jJwF*$KzSD*c&n*wSpm!-q6jYZbdj3 zbVQ{f`gM)YKsa4tfuICb*%p%iWv+a5^BrNg^}tY~U~K*<S5I0}rAG05Y*N|Lyp~bk zfzfg7iNAd`j5E2ijG<6heoe>doF(mWx_hpax4T>|$Tdw%Ng01qcS@K*H^1H%UUTG; z3$H&k<zMEHh*9hq@ImXx`zU-#s<c)>DDgN|W&YLY0|EzuI$}vf9gauxwbFXV;)7|b z#bX8NKkK@FTQ7ZUKZEX@B7Z8Up)?#v{EKWxG@AxHAY<|`ThTmmy~|%HZO!i$E$P&z zgcVnfpW`+MRGhjOJn$-@s6zZO-(_nkD%Nau*mKbN@-evG+SlCW@PsA-Y4nlCp%7VE z6>le1Ug*QZAhLLDv0nZPtB0Mwm{%g4YqtHH*8Rl2WZl$0(YXV>Z>>J|Ho*G0XwFR+ zDX(#~lXK^HMF4jBoJqDk71;+d(!!qQDgFG+EGRi<hN`wRK-M%wl7&if-H>=`f=jo@ z^|gg=hDX?okm(s7{28>b=u1stRf0R(=b6QKGsD{{;>}vQENHvkaq@6!XU_`Wq=9NF z>rDND!*GWS6GksrJyB<eFtEyhY~~C2w3HTxKkOJ6!3K^Kb1{?Ky~5#Kc({cnf-_#0 zLVuOh-2T90thb|gRR1Pkp!Zk!k9G+=Sag!H*IG)c)OyTki`RLUyx_8t!6th$@a%yr znm!b9RWZr5DY4q+3RdfC^uf5+qWsJF!DkXl`#P8LhVxQ$3wHYxZrLqwr78M#Xm_Jv zEb9?#pyfs9{{~#5{{dX0;ybm=F(yjL;fzX8^&Z~+_)b2t$ZVJ0rmT^m;+B-~yt+D0 zEzHU#bGjFFm^YT!{cVTZ?U<1sQrWdh-JpI9%UVxGA}nHp0j3<r%<N!!ox-<DJX9gi zh--%7Y?aae2SPmKf=WE9(BwhH&W^%Dj4m>xK`;3K2gKrg^rq29d-M5AJ>N&V2J+p4 zJ&+>V6T1K%vCN~x-Nj<wjIs<H!LQ?MC8J35+Iz_p2_=4g4&Ty|eYV)xaEQiYa9dS~ z%P_z?GdNQh^B{L_Wohxt%SNoK%G@@M*qP@zQ1icV71jR>S2@gO{xzsrg&1%JWT9{g zh6q;=Oh{=<qe=~Ab)^(vRvp&1)EjB^NW+!JT6&hWudYV<0(ZOF;DaU}u!o_J4Zq_P zU8v6t;U8#ELij75|AcUz{~f{u_yqk<krhQ85bFq9!aayIrQ;)RtQAZc0p7ieCYboH zGGc%{&f_<4G?a4Q3r*|Q8E{cjyTJwR3zPq{(8d@%UU`GZNZc36PA5sjWhB=scg83L zfYnb)6*oKs18P5!1%73j_<WA0v2H}*_8Gr%@iI&HX&`0s`)gj4gJ=U&wy3ycdh)fo z3RUy-)SUngbU(lxre_+&HnvDl{Sc1-w!RU*5iWno?tjb{I3s;#EPD0%^ntQ}^^Bkt zT4*&+6t@UIo?R8|J4e=d|3r6bb@k@IUFg6++`yBV-$1sQ-Sju+XEVB7I&01@UUs<m z*T$f?<NqoZ|5$vti3O(-4IX9HQ#%9p;)Ec1QvtL-N(?a;wM13X=K!ln#+ws@TQgoy z>|!JhWmY45Ze3Ts`?KX2>H*VP49rK}zkY`PYX1#vd((C4Um4{zNm(kjdl7Zbdv#h6 zc<FAV(}BmVcR_pZV?p=hQ3tpFqWL<M{E>gFS3!2}b;#p-lqf~~>ehB&u8$35agp6a zQThn?N2S5B;kc~6gVufgilG!e)JWF@)>NFI0+KAj?jBe}bSgKCAqr*K;><=DX*J~> zEM7>Pwu{ow8!WCAZa;8!UuPZy9j~-60@mm8o~uQGWyj{X?t#EHedYB|%CiP6Ao)an zHR*;sIx#1)v@zU?ZE7z!PYQ6Xrb?9R9)K=l2OD~<4cDQ58hj2K(8iY89>Ue?-rQw@ ziT4PLXLf6~gMgpQJ=1jlN4ttfln9%b>-`v0-O5%e=0nJyuJ3(`qt%{8OHWB&Ev@kq zR&B>>|6jmNz_VTD(J@P2yFKEuiLHtJljbnC*4R{_@0Ov_>ah{j?+;`B$ofGDD^Y#X zXr(*qcPhN2wcr?Ak<#Sv?Wk<4jQL}byn~vp4b>YPU}r>a%<46MtgcK0_qjEEZ5nZB z|9&=3qjlv(kULXFB`LU@{XjaVQ`w}=Wjz_MHR$?KWKvasDijrhIq27EaeQ4^q`vW6 z`y1ooD4O~PvEIJTw1c-1d;E4AtRcZBffNoM(d4_fHFhWFV*xuzd)2Y?7>g_dao-W7 zCb_3{w8OA8pkeK-Xgk_pr$L{n`o$skUdQnS-Rc_OFkcD>^QytXglCc<@9Hqpu_^D< z@}yDBdlzJgYHfl<Ac`LzBbyz6vu2R>ds?4^DW|vkZ`ol={Q^5UO*l78#ikv&^uVmb zyoaYjB_pZyme|_f^x5%n86SN_KiKl9mHABwA71I*JJEdLzWHQqOf^B_iXDLp_T&9~ zqg4(~Ha#At+JAOCco+8wS1XB1N_RdQFdb!7<w_r|+!4bIX^)E;H+I3BQTFr^VT7%= zZC}HJ5m~8WOMHrw6XWmOBFd|ZMn?$80`=;fEOX~Ix>G{B{h9TjtO*H6c~s<A6<?ev z*Qy^V_eUreHG<~XlD_jM@hwEe3qcD7fy15~x@R`Y!1$b~gYa^nFJZ`KT*bB)*Io<q zc#`j?$JogS$VTaD)+Bc?Wly+Sw+7c^Gn~~K*Nc&@b!mXg%6NN;nLi#HJA-XgyLAUu zj7L=$+8YEm-{1LA<bIUi=yPPU=cHd7HqlBrHfW}^+nn&4N8G7kJ=0d#==08yQJVAe zQ}|D-dFviX$lCI5U~4J0jzR+9+q%~TRY*4d8QW<p_cZ&tJpQ+|iEU_i0o5@5U~P8l za2sf^YWN{T=IJ@E3tJ_dc6e;?XS15$2RB~{46mxJP116)(2YO-Gt13Bmv3lLo$xJU zvaut{f3%26zlgNzWmM~jah=i$`n+I|opxajruz53%d>RMte~x-w;sz+L1KLS@q=1Q z%k8m=s;i8?eP^ksHf;YM;2!@R|AwX`{)>G2X<59Vy1Q=tD80wI+t2XX>j|}*sFsR3 zBm>ao0lErNb)?*8*!QX<5BXN`Ws%)SK*F|=XKc5mpgs7VB^@J}XbhUDF6KnrgT3ge zAxtY<U)YL|@K5YCepelr<LWY6M(T{p+|?oer5eI5d*oS=o1fIq3KoFk(^#?bkM{jX zm^q#zbBEkIzPctlKO#Vbtz8|tbz_aCt}?;MY5-UY8OtvAX(en#>2V~LGMFt^XfPnY z)$YUqhN%deuxJfL6L4S%g4o{Kh%j(zbGhj;NL2;)9}I864kN4^7!}P{_A7)|WN<{) zIfP@VM+?0krONADJ;kc7kfH`;H<VV5?#}Au&dQkp|H4TPI|_onVa9oD6HIuf;N9A> znt0gi(LqzrGov8~!2zpUj@>{?%A^}}?R*mgcFVUMCG-ZCP<{6y#VuQ?u%gD-oXIMW zR~FN&-UxP9VSdmKfvK#pT$*5UoHQQTZ>+d9V<im%BX?R7T8Cvj;Ppo`H|(^kz9qS3 zdSZS(xei-eb@b*fz8}9knU^BR(Qj=HEl(dv9TJsXB>GNFEcjB_(lZ7X94Fh+Z|#O} zvD0d3(Bt-^Ojg+BSOv1&oUpSo%IllfAcPp<)$-6#lEHr2IFpDwZS96{@KTo#%e2F% zu=wtA>8<8kW#ZP;E;ys`LVdhEb*24guf;P*XH6L2)5_Jh@uKHO$I)%yG`Yr&;L9x& zLpRo%7DB#f<aI+peJ$Y)l6D>t_jP_xK>p|VDd3ypXkC0WG@x&?C|;{2`6fGzTeT@C zfeuhiDG7z@*&g&@3N)xf;iCfarj1dFb*@edLPui_hS2rM)tR|eI@qX){bD7^W`0|| zb{xCU2hoA`<A&iLLs7>PkXl5&B79zJd*1)G4p|-C(&fzq$+K{dTTahTqxy79t=#^D z%Bm_4YeFNj+{MMkW)vVyMPws%ipcZEhUf&>jMILDz<zI__{O`)p6|SWu2Irl1Vn#= zowZI_IHtG~3^axQ)r=@&VRYQX1kjoAUm#EbeoddqdnzER{m=@0)1cg|QQpE0HnSa@ zakr0@1$tFRIBSk*xB!t*VvB(P4OU~i|GVD-C@!{vJR}*1x4k%-mY$Wq+<MzQOLKz2 z{7}PlKrAostm7N#&DsA!0lId7GM{m(T?n{!+I8q5?k~H+-e}_e2gUk>mG7K9@zF_; zrD(DKSC+Yt9U0W7oT2)1c3*|hFa9D_m{Wvm9^X|#4zlmRz<_Tfbb{{~G&vrgnUB70 zuKlZ)?p1!(`TMl_0WmWl9_5iEO$1%PTn{b&FVakAOL%NC&z?WHCG15@l!flA%V&P+ zo4>F*x$kgu2P0mN{w%ax$>|ebATGRi39)S0DA_uSk3VCPD8a&{q_Dzi1J~doyc4_o zFl(UHZ`i)LF~vVkwbFa@t3|Od5Q5xOt-Lzazx^$(_vgZ5pEc;CP2(y9)}G&Tw!Ki4 zd3W1RO@vX*09#};KCxFfMBysQ{DJvQ3@r1ncYW)`X!B2!hRh0Z^rT44e*b*vXMq9W zml$+4aF=mH<w?@WucTC%CAVn!2Qhc^M9*!zWtA^hG6UNm(NNGJtUC12!><TpL|G|> zdv-9VHh_6L8p~}~rDxPoZ9cH(CX0(z9gt#X@i1~6Op}8TZXa)N;d8IALyDtd{xfm= zD<3saQ+DOY|CLk0JWg{e(&~TbRO35_J4*%MYAl|ia|R3D<5lSy?-cx*wza=+WMvA- zlou_)s;Yu3zw!qM;mL3`YZhV>sKH6IIw_E%;=LX|s%%lIKdDuf)O3?$@4g4aje6Y$ zEAcm;OAawP%&Q%7N_UmuXS?I3obin<+^A1r%mZgEv%zP@)6PVm3&AZXEfs{BsrV|L z)ew5s?hFMhvj0hJa+fuOXha$fVt_^*wjgLhFrpZ7RBm#EyqeM1Bix&o$Em3_*o3Po zt=hR@h`-T!`rQ^)ntp##-`>{NnU%w>4eh_S^zQ<+upu1D-v5=&<VB4|`1_Z>eL@q( zfyPyMLcd33@1!n3)2>Q~h~2Wh)oC`>Eu1XPa-*nMrII?nm>bomUBaUs;(a@GM)zHT zy-1M0Z!9k?xvpNR;8PX1aCPP4T}=d6>0yr0AXz%{O?A;cVR1oni@d>o%?dM~`^nl0 z_f6?<^5h6_(}zqQ`9``Gu^oVU7G*V$j@}k;p+w-`y)q7le%?!-;?HWa23eH0cgL=3 zk2YU%c;hF|Q<$W6ZAYv?jfo^px8R)yPl5XQ*E4%Wdp*7^8WW{^dtk@7mga*W|B$v& zC`=P`@tfx>ijg1h$QKg@I&B(Dr67e3OmKBA3NegqkrKYg)B9j!VdW{HcW~K3^k~(? z*Cp)Jl4thoZDzak>w{lEX+a4wELC>Sx3<0f@Q+5#I8CuC5y9uyUrzyt#39+;<=@;n zO<!|{W_V2a^dKKTNwtBGt9B+I(pzrR?P_>LMx|vC_x}V_JUsh0deZH%r|jt0+&p#1 zXvz3ay@*^*o;}?fyMD#083o6&&+c*#50gK&(6QB2q`s5swO-QjZF?Nj@TQR|Bv*C! z`J+E?c68`l7fCvmZf>fxq(v8swyEAwa9t_}GOC8_h!ST_x;zQL^E{M)Q7YpC=kiBd zPUv^(vzxL97*eYoSly*w%;sEnUG*uobI&VED<67po@s$@JGQM0e<cv{?ltVWFNM*O zsSa4YiWCgf;dP!SH#Wcfk)AWOY=MeF1RWz&QTt9inbMR?oae(f(G3Czt6-k5lX680 z>k)ZI(DrG~i`(Rvu2Fq@&1$E6adWDMi7{}@`jzy0x6wJhiEB14aj{N}jn!{#6@iV1 z0?kXbOK|d$YlQ=_;*cKAqPJ;ZR0A9;^LJW?*7gfB%7NXKh?Q?e0!m-g3C#z}e&d`n zU7ACTP3VNQF2_Eat&va@5f_MvFe3+AdLz~%jctpA3f10_BMP=-RJ`rEFT!K=b=JPW z{7FO&_}_hcPu`I){V4=5;7$gE0jl4%BHtJrH@nO$1$8F?E;||rybV#!*Jp2afKa7J zjM4jNHI-v!zuLLYu%Efg@W0mZR=2E3prolu5^L$|y49I7ATmHX7i?z;(Lw+>rY<^6 zHdtHdPh+wEJpBC2dyjBShp;2{8asD2!35u0owCt=6W`%&kSu#>x6Q6JYe+ZUQT^P? zznlL5kKLn0v+zRFXB3+(F|=d+MKA$=yrjz;=)B839C%)H<%N*b1)5g4{N06}`oPer zY<zjEdWI^|^qiXpre0sgKg1?jZ72|_bQj&%2QF<XF*V&i#E22KF`m;bIlbd_J3ssL zcRNEM`;C5r3B~(rsL$su3$IJ5oifE<OTs!27rfh7Ufe+}a|88?%ltGSOf2oA4w^<1 z&-<o4(mX#mM@NWHx=d9a3s2#T$t5Se;bAS?>7eMAHr75>m|U_1S{Ehi;ODXBsL35~ zTrp-&`qE5Q=ZW~m8m8PH*&Uc109S^U59Sf@7P|x-vJTu}I%k6GxYE(TW$*qO5P9sn zg+EL*L%T_{6Q7&UDv0x45$YAB7H9NlJXrCmNl{J*J^9F}cl6mk{(+dthxc~%Ok7ry zP1XmGrP*7xMKX*Y(tJ1bx^YCzmm=Q1X}ZO6&J#uN{VL>6&@ru51Kp;>NJ`g?$6Jev z*Hk-GUCe#aEcBFDy{gNSt;a%)Qz+rJ^!79ZeTe_r>xJVNE#5rg@F|r$Avwsl&s?z> z$TxpA>RwC_>h-6cW<D@$jLOw2Kk5%3LyR9^wloLo-VW3*e>+vgTKnSCqTP))YGcm> zvk#yvEVTC(NXRwno^SO7)UW05lMJcX&AW2mH5)fm3Vd|6P4$Q83Mulr(1*6Rdvjrw zUhi(duncq(x@kBB4?8+%1z06s(Bcv19Qp9Va6o&lZE$EP^pJJS^=5_>Hn$dTJ!TVM zet`PcH1Dd*8U6+NtzUTeJF4uBw*#rv_n>~+!P3&J(p=nK&e`U(nJzs1WHI6I;XLUa zg|S#5RT#{5gEBaJC@WL*lj|EZ(t+h|SfyG?{zs^|;0NvBgkYK)!AD`n;#wkVSdOea zH_=KN3z4$vD>5F><{4#WIeO1KW7kOC?yy$Qkm#6>kPZk%csn^cEy;3S*DOBX>JJ}v z`ZFIkHI=TbGLHJ?B$)Ez`<G@4PQb&&P7_VT0^+$CZKC^{QclDAvdL-e3(m$<HaB%h zVhu{rS&?zj(pu$)b{+4?Wk#|0hcI8sM8?T4H_<UKkVS_a#@6VBpKloRRSu*tRx@J{ z8>RB}ZI<AQ|L*kvuQu?14+}S?xA^(~wuawQ_;r+gxAIt^_^la_OHI<D&3@H*-+i3( zt*faC*{u)r+*@^p2;40OA(sp&M$E{MySeOWxxf>>P3_a_v|nI7)2fw^<$v{aS!KqD zQS%ZkPv5!(gcc?2vR&mu)T_)SnFoFVpFO=3*G^86wpi!3a@vBZ{m=QGS99(@Y%24h z0!EQlnaIm4W^Lzs9I~N%#rXwBF3IQ9>y$JQ>7<ZrM7D6#DI&C66XIH&hDonbM5J*I z3=49-tXiCXlES5}q`Au_l}K8{oop(6<63-t_q}-0v9F)wsS*p;f1WpQe5Bh;VMzS$ z<%+33Bn0Bk<lEcqb^9piV?9RqSU4zqKx;1MlZv;UJ8{LcT;L-`ZsPjG$cez0I^V<> zng4h~5L<G;3UB8ceG3U{cnlf8J0ejj=%Xme)SDeE@U^w`3oKJ?;<JiO_N$N%6c6jH z<2h5KfAwN~l=Ph#GJ)jw+}}IKBTFheueWvc?D}6hU~E^+l!^UJpFa8WqMhb#WAsZa zaUL3#P5at|JHl_rT+1Ityj6Z@Xw8THdifJQ-T3YulMHA7du7)iEvMQ5M;Pkv$11h5 z?IbH4TyJpod{|yilK3$ezyu5K6g2kpyV5I`e5GdPSD45PnOJR~&+o?KKCl|LP@MCk zZn7t9C}=(mSux$egw$-E-z|Py#dcLs%Fc^4vz1KX;NefW@)X``1Gn-ZNrAkf9AUgB zF{jaC5O8zewDmKCMlk1F+ac7Oro5DOywQG{L9lnVcGD$<cZ_XlFgnt!|BbakLZ)~c zpebC?^0Hd2VIKXmy$Rta+IsfNIf0wwmvexjLD{%4002^f-0OJo;DHLLX7wobO8l6l zHKq2oUn*SIr6a4Af)Vfy&V0K0k(E*~Tff9IgJOH!muBE#x~`nzDmy>@w|qK^@^d2D zl*yHq)m6DFa~UHe@Nq?Lt+$!UQi=^RG4X~^<Kb=~F_qKSyrO~v#rpcXsrSgY0(I!4 z4Q*tFEi0FPRb4&<tZ{s_dbB2$l&e$1D6eR(q(p=pR}4n#4COf;{Z!GRC?Fmsk6-q$ zRX#I~`y~;eG?%F<Zh%je2DK%RZ0#$_+ebW&sm_@vm2}3xakS*0JlDU?rEtq}$A|lP z<H`MV0u0B6lT)mv_HRnCxTviCW~l6;UMi??$aCeH_jy;7=nI^_TeOFS_Q~0@%<Z9< zifb$SZPx~V)N#%gJ5L=+PM-CAPEnKGC+(qFOy}NV*n}nuVG}W5@orJG0{};m9i5ws z;8WU#$w@=FgerbK@cA_}Um-2DIHY0%V*wY#RlGCa;XzRZMlD0uG9wwm_JV)pIc_Mt zogfKqbTD%-xr;Imo$c(ZYl>Lu)_$o-U?-3^q1pIV{Y8el<eJ@v<h|M)@i>C{jDkSJ zEZ$-}9<_2YrUw$<1hG6BDk_Ko3QCkW#NOD;BjbN0VgH2M#1~JInK80isI;o(><2}Z z@dLttU>PnITUr6@a)8xiZow9$Yg8d}YJFMi4*zs<KJwGG?Hwv(C#x%g1pZfkRD@dg z#WbO7pNR-RdE4~#Qd?_9hhrR?WP9h}wI}1E{?j{i(ulX?woWQ*@o&TAHDLG6C<y{@ zei9+iKiW6_b2OJjD{!Uu=)tX_yfUJhT6vWCCG<yscR*Q+bM9<A&+Xf$wKNB_N_W5; znef!7#k#f2U2t77bsxF*g=Ky~_0MV}zfU`^qSHs57V)U3BNnm-j8E%S#FC{^uf@36 z&e|OTF!s}uYT#t4<=y^c6Jf$V1%G5wxRv;-5AD(9q!Y+uw`<g9;BPUYQXZ!;wbak6 zuQ&1I_?U04_?4-Ddw30tPeFVejrPQF%i0>BkQU_P@|zc*I*3cTo<tJgU16q#8Q8f$ zD+;PUz8&O9P`FoR-oR{Ue`rSW?b~zG$<m*1>>2SJaTw7Q7*p^1KIQ86*SMF)R@bYY zDfX&7OY18lc%f1tf!U!S-9%MYLS?l&$0T*zV)VIGfbcIhETbXo1ih`9@ewA{RQaK; zU+=RQ+ALwL;kvbx+ZnWP4+bdg)>lj{Z3$@5im!rI<BhR%>oOCTzq!+f){M36D0izL zM!A2K`4oSU;>eQ2=^T4jeyT5jx=EV<ZW0i^77rtchf6&Hque4tE9(o;peLWr-Hfq! z^06p$wnTQ`YtxsHRAgg7H#>MX)9JKOhzMjtbgS6nWN*)T;&U!y@r=BnB<j+^`$ahq z?VVl<Imr<I`tviJMc#!q9zHqJb$iaW%S)+7g0B&LKU+whtkZd5rR0KzjMGf_g1*aZ zlM&nDuC{4cxkbucG6yX|ZOuNieXLWx#D-TlfO5-hyIvhAqCY%b!<IQf+_BU^5EDVM z5XBZaNa>8ODB!N>dXg}c-)ec~N1rz7!SpAA8%ouz;lgtlsm8hxQ7wG3(1W)!7HmCh z&#l~Fb-ae>TeyF!1tvky&PQ!ds)WJuQbjQXss#L5P+d1utvIMi)whD_$v`_#my-r* z6yogMzV2iTF*GXexE^3Oi2ce{tD5A*MP|4|6An3vnF`c0ytH$^`UHpgxuZVwlH}a` zMX@jLQo`iRkV-m|WvA52R~XUPbysz<s9%i_Tq##<KF}_u(0N19s_n5!`MY6PI|lvd ze0(H3--xsIiR9?HQ@e(KY7jOb*byD*Q+m(dKV*(PcB%2(I3n1HuNWPrq!v9wdi$|l z<iv|~GPdO4q+b+))<k5F!a4c{u7vl?_vVy7@nmItX6fBlMeaYdXBzuUr<Xq}H2P<O z-1X?lt}WlJoNp_sl2nZZptCmCMbghF1C!)1RMk@gZGg~xz?d79*#&jYD{mP^l7`^N z<ndmaPrW3ne`tD=O2KMTMz%#hjuidr$(LZeg13X?<z+gYEw>6Ixw*r8P(R0Hv>w}b zX5S-^7xLMu;!bay|5q=7{N>9+dukg`2!<0hYFeiWy@=_!P+HMDVv7FU$*O91?qxhO z>lKuWC^ivhekbKnlIHAfa~&41AU(f+tK#|-a%b*PMOLKkTuhWbvZ5nR^vzBcbJIR^ z(fqyZCC`4gA^`fFG6DgsdF!3&HfxnVABK0i9OS%XIM;9GaIsgr949%*^w=u7hMu)^ zRMPL@E3L-H#<Z=eFeJ-<calL_5zA_p%E4F3D(;%B#Of;wR{78k2jeSbTs84e`<YyB zJbLJbS~D?mC%cF`087KQWE_lT-cl}eD7<R?+zqptB7djKZtAncQk8wYA=oJYgZxHc zaHjC5fgU~)QJ2{?$w&s<qsZ$EHH^}t{Ew<?qlYItq};)w^}{uiAxE%S*?d=KJ0;}{ zw>kZ%^J;T{+FKHjC3~77_eDdn1BrBlZiOS9cUZ__q^lZsj*cegx*g9rl1dJMHhj~R zXU5*<50@J2Oz?qMK#K>wFYNC1b%1t=OMmUl{bwtGzBdJ>Cac<dsTuNB2QS;8RdF@? zHqUN6w4}Mol-k;fAyU2VsTP$3$#a+YpY9$gh;&hp2S?K!xRLMeh{7dnvW}g^tHWtU zs+oHF<7$5KYhcY=Id{?{=1GILsacbNG|ByrRf$@9V@w+|rqLN$?n3yBoHcQaPNLJF zeu*??v~0}%R;72RvpRU3WkWQ0L&~XE9R0kNNyss;R6GRJ=9N`P?|NoYu5WjG=v{I9 zn;CS68FBxDi&;F{5;n~997lPtJDP+1ksEK!?8fCZFTxboFu<rHBEILencAC$?%$~~ z@_FJ@6sE5<ub>hzhh<9;>aez%m`=^^OA4_@hnC)B%+X$+5`phV2lOt|=SRMd2#uIs zQoCf8>y2<@B?%`exAHVt;$p2bdB*?SxyEs_6V;u~&UM>PZeEXPfW5qGNxf%CvudX! z`U8t(WIp`2B`>&tU5)39&!9uWR!1dT;-f6ChK0JIJ~HHpy=Qj+npfOeD^%a`^fMRA z+wnDxJ<C=DcS1x`mwlSFUMTlGlF&1|f#s6rT$=}rj@XKu$WG~Fa6&mLA6@^F_WC~g zt(~wy2U>^uYcVQ1mC<7%AD431N2X)0jwUF-QL(|5*sZdC-AknFL*A>2XZ-+->6yRF z=FxnQJAUH$JyY_Fr%R-{782$1-q)(<ItyeBjdKsoY$<<xUEPefgVnqi5pl_Eu+c57 zk8*U@4M}>@5B;D6W?o#V(FfeFOj_+^zjU*A-<ki;R;<s@4<nq|6usVj4HzfxR?D7N z8R+VtS$D<fi_i1(CN<vIf8jy1G_Scm139K^_8T4N0(vKS1YG`D(+Okey5L;44cvS5 zt%I$P$E(CI`L}KqPXrpJRmUlOXKX4K6NAZ>e@@-NK1oq&<Ik~}bz)biSY$H!Z7x!- znvd`s70+U1JacC}jd%(f)L!!`Eu!`=O!UR4c_<1?+olP=^>3n}N}Jk&Qk{t3O*yxj znG&8>3Yd6Zug-q@LS(x?I^KBjiLi$~?WUKAn?|DsQ{iZTUUfqTj?ZYhfQb{55YJ5F z!q%$A&7VuqGFkYEQsI4a>}EgNp-_pbmmjGV7hSv%$Wrf;t$V1BUCI1CJLZnMxzU1% z9*8^#4<{s_e$Sc6L{+RdXp_?sN$78i2Xfys6fA6CthRi>Rcl|X0nhhORs(h^--Exr z%^Z_RG#MZGXD;|{tII@p)nz1Ky|@xXD|HH^z1x0*+1i+F=v<ewuO|IONe&PZfKFMx zB5OZa+Mwdpd(&=3v??}aixUww5xPCkg`wV{qEdH~o-eD_tKIGQz9~F=HU|0j_9;MR zva<Wr5`n>Bo?-2fH;IUdmZ+-zp)?sTW9&T_i|KyJ|0<LkUJZR#?k+rFhWtj9JPy0! zS-AJfJFY+NXRGYDLiQ*Jjd^=OPQ`tK{DFavR54K%RAk&Xaop4haC;G+^zvXdeD9TI zPcrH(T($Wn&G6V*pDX@;OUXR2qBl<4U#&w!ESj#syAk;^k%~pUxhGke*Px~z3XMV{ zySrcYWoes^1UbJJPR=j0119X22nj{m{jO3Zti=~}0~n1wAN{<dUXA(RvUx>R8fy@t z<z&(le%-vH`8+Y<Yg}3o`|G)eLFb@L^66T(Jf0HTRd|uYp^Hx~xwP0k!iJt)kXY#4 zN(y^YNsR$))Bn~*npzm$i=ec>brIOIWJTf3iVcswJm$ma#osnZ9#2**^RF+zxC0u| z;xP(B;o6MEbm?+70cyK`{g;oIlzaG%{QA=(;ZkS)5(AyeL0&=GxNQV?g6|Zgt$Uj@ z91n~ItOu!`S6qo*t}&^Vds(6_$yhQ#+fj_K7`Z<?xML~^&jo~|a!ydUgGP@#F^=wS zj<HdLRWRLDoX&SAHX4m-XKYlR60@y2hFf{e8Tpw+-+SI*f5lZPNPL&gvJu`u15S0C zsem_ab=JmGdVzs3x!kFp&c^$b;lKDLbox4$y@i3UHdmk`n<b_w?LHlZH<=__VSTXU zaR1T3k%4zNB}6>;099jILa$sRQliWT2aZ-Gtve$Z*{^*<9k2YT-&|3fe>AWf+r%o4 zRl>}>nrw~xDmPU9_EukSKl6@Rr-M`IIis5arkPje{nG)WExGLc(MiWYtx#f%tOmR2 zIThcNxUmWFt#nz`bMl|Y-t>*y%oTZW3oVZVZGq;>NW`dds0WhTJ$BcQ<H4;<taDCV z$k=sEjB?$k*ks2h26apw&j0yPV)X|dK=6G=>*nKzLpsO|%moZM-h;lS68k7}0lbg& zoF%KS+Bm}NV}<P{k5b5>SQvp<%ah%u+5H(Z5P>*$s|J5HUwB8vY&=}6=@_vAk;F~Z zWwx)0ZMK2-;1AR>l)z0-T$M{{nI<Cn94<S?qXydA&7s(xD*e(Jo66)Udf9rMTl4 zi3Pre1KT4uj*bf1j?uT$j+Q{J(BpTT%$Fj1GDR;}pOt?%o73VGtNYLcJgi?_vTu+6 zJ?f!@2Mf$!I%iV734%rJfT%w#<1T^jWw?kq6%(bf9`w^nr~_5TJJzuP9J7({PKOt! z>mp`1w|{94oG`V^KJ2~dtF4G7Xwt-#F)ejKfM`?{bX83WgTgJutJE^R8w{ymZ=r+4 zOCeLC$CSq$mErY!My-2hS=?}m?9gti<72V?j}2}cdqS+8{&xGFuI%F-TauVa7B1XP zqp|TeR{tHbnxj8}!JmseTC7uRL(nMSJ!ljJyR(Nh><H}!V}EL32Eg)pJX%o0M=kVl z;8MjPdDeQv0saDYTF_e*^pI-5Wr(%2k)`!W^6~qq1xw{);70$|G_?#g73);gu@?>g zF75UABc}Vmpng<o8bfKWEQhQ6;b$fQCZ`F&hGwFf?n55})ADxoh%B9Vn~Q744j^Nr zJg5=@Vq#?~AGjdC6&D5qeXIw=XT!@$55je(0JufmM^6Bj&udfB4^#arU`30iRqNR8 zs<N!CkKa)Y5WK##Hyaq)Ve@V@Dlfj<WkR^ghc$yAQ#y~fJc8kPns!<c-CWkzpj`pr zfLKvzqaZW4*bu0V@8Ut!X0Xu_ZsVxKJ=zq~u<xm@UDybBHehYoy^0w2CXkX+p6U@~ zZt`uf0Aq@xX@@~I4}z!~kQA7MV#hn1E+iR<Ls-&p@J+PIN-1?`7lLYX=d&bay%FPo zcY9zA6b`|n#Wr3ObK%&(TW{sn-t#<bd0ilUae^6Q9xdR(p`|dyeb!?8#7a>R8^(US z;`fI_jqZ7{G=xB28xu=+P1m>h{z|bSj;M(*!=)B`m1;7&7!0JDe8VIl2|muZStObL ze6WSo+E@l{C}NHgc^7TQLPNVR?&7@0e51AZZ&Bk0gAv<twkG0U9NZb%{tH@Z-&vK! zKUiiHmFYgiOicq8J8VI6SR}i7+YS|?nFj3gMloXN3mAVxR`IT1%Yh*eaEY#Tf4shc zIBG;5qr;K`@S=xw#@T9V54o3scGc1}inl$QIzl|f4=zipc=~Eo93TviP#bs{o|vI2 z8!;cZZn@6BAG7V<5x-}=I_m3;v}TvQn;`+*H}$U+R!Wtbc;s|ux~y{Hg!(d`4>RXh z9}~Bz*W_UojD{cDZEY*Ftl6`tE_GDG8|aJ<VjXA0EH5Y*&~{OO8%zkfaO_o}FF(u3 zuW5V4BK&o$6aN=AXdw;Y>_fkODS%q-TyeS2;p$DFK2RG3G@a~qya>8o=!E!)x60fc z@U_}+1aPPyCP}7wGfozJApS&^QPI1Hnit{rFQ;J;uCm*SFu#Ub34;9vUv(Ls3GO#* zAnpB}eizAnY+Msp&9Qe+qq5I#CcL2Jup|x<{X(Z@wneIsLlt*`JN)(<cfjst_<S|- zRj%{+-WEDCj>uaqA*-SDNE%4l6@w=91mJqmOKaUb9q?B1C)H==5ElsJPJ@%@uMaPp z256y*lDcDjw(iHphKMR+Yvc4b99qI3x%}%0q+~<k8-qbGg>uLDaC>R+&QQVRr-#B3 zv8p$+iKI(bziLigVmOZNCpmtZpNU~rXErBE0>{w(Hu&NsmxsBjo{B<cAD4;}3~%V+ zG>i30Y<(T$;LtkAkg2Ar1e7gPVHFkSr}}BUaeY>EY5gj8*4}U475CN=GsWIK27W4= zYFjS2wF25_SCyXggnme#Po$E+1zIkw_d15a1QP&0JrPl0!Q9-*aD<9rA!z?9+!4qM zQ7`cQ=#K^`+VX(iCC<6mw*LwqFD<TplkCwA;~aae{iqe+kbcCbxJ27w=di+ZxSd6H zXnzEO6B7r4SdBn(?<R27Opj+nN2f^<Q3@9rfY5%6<L^`(j=(*~q`F!|=zavKWr*w9 za$<HQx?26<0~kmS6-9f{Z*|n{1&iM>zqf`vw8f<-=6Hn`S@A)~d);83@i6Cx<95bS zZV$R4`NJg}!5fA^+7FPGX>R_6(u0=jYlpGx8k&Z}d&&@IxM8Y4$p`-HSv+0B&dc{` zLjaLf5^55@<AW8MfK14*<3Tc)39vsIZmD_=Ms`C(`&jd1n)9mj?B>n03W1uHQ#|{D z1K)(r>?=_`Ui>F)V%F*uoMsiytzz=+x%raXz}45AD!dxKKU5}ujEU&jp@l&(Jx!Br zcQeqAQQNcbI=7khy*3~G98i|TD*1WLG<xlFZW*gG*6It;Kj9(R<TCV8Y*8Umm7zKs z-E`oreXRzZaRMR+1Cgz-d-lAi1sm?Lvw-xV4f>^j(iQ<lQGUpLmkEp6IM3~3q-#7K zC2($JZ>9pO4)r+zZ;Q6K3$~;3;f9ND+d44z?IZCcCH;p)g-32n+0T|JStLOddun~F zq5Fe{L{2+D<nYa>r*53}_33Y^Ps#V~n57EM@wsoOfO&t|glVbn2|CaEDEkIgBOw?& zcPlNf?ngnRX=gy2^5|wIfP0F&)YN%*!pm>{_9P~bIa8`!;2<JY8;fDNB^Usa%PZ2& zxPob4>GR<=nF4Ph`TfWsfGyW63SUCB()`{-{Eq9J)zBt&zpJO^I+VL;qoxTm(ddR4 z(2to>Zm;7u(2rrL-zB*JeBjev!6!14`Oqs)mbpRCak$G*>~7ZYJ|@kV5Wig+1js^9 zYq2gB=75Xq;bdY;<P;JC2py+s7=P16@JBMtW!z5N`ivIfUS$$um(6e0@tD{U6kBu! zh|}9~<?j;F*nV<-U&Q<2l+^X(nE~`Ljnw|e*ksXAJP>2A{2m6++?YY!+1%K+#0e+? zH9Kq3{=j#h&_e^BPVHpqafR2;y19new7Yml3Nso&ID^@tGt+&7JR14ky%R4XrHwgq zoDF721Vrh-UWQhpD|0^=#*CaD%xnmKD%)NDLp#*d@#`033GjEX4jfebG$xHDHf`7T z<#_eFVmG`&nwJJ#e8+F|SGi&?M?pEhqsailRQfTA3tdq*5CvWYzn^3%e?t8dx3@+y za<37T?()JPa~aZIzqqCm+2Fua>YvEWx|v|Kk*h?}h+}qvSI-cVBtRSP^Dp==(fheA z6ni<m3T|xP^Jof;DtxLtd3`?sgd?M_YGQ_NE_RCpH?D%3c6l4Yo3I-AI!pmuqK`A$ zc;@}gNE`GFp}Y~%bLmETvn4o&nub~x<n`i`OC!E=onY0Nw<(G{-sprc?WFFMP&a_o z36WkDBqV+s1`DxiSQvFDc)w_7>*>IAAWV6qPPY;}%IIT1m(fZ`+Fd3L3_=Z&qtu$f z(9YSY$@O+e*=F2tqsDhp-Ptx4duzaY40EbQ$sUsb?3%0MZMO_t%=-fsWxKr|IGnW~ zu8x=EG&7A4>{-Eq?9Y+AKjSjLaitsD3|ZBqe~N7e9CvJFt^=ZJo8zln1${QXk0h{n zC66CDwgj#higs4g8O3~}W8J6Ezp*ttwAr>Y!hdDEy8*#VR$z1xzCM_+fd5ap&tw?; z9Ijo&r(`!Rb-ex#cRXURf3|$-r~~Z>Nn0OuS?x&Wnoo!yK{xQHuG?VSKB5|eCeDgi z&1~GI)z!rH6On(_KU-D_IlcKFPEHpf)%tNe-vfzjyti_)$vpS=_R1O>{C=cyCqzZ- zHxLw;Ji(vlO5qLLEeC<!IjH@a!qCEu<qj1jS1-+Hhwg?bN5BuQ@*Bi+9f<w@O#Vjw z76pFrC~_{=WtA?uad+;ucgUIfspu&|x|6u4V?ZQ@y|?Q^|Iy<obO?<5gN2Z=em?7e zRHpu%gq+GtvD^v8D)w8S|Gu1uc&`!G+S?l_);@FdY>NN7m-<Ql+2vS{SMWg~!}h0< zdQJ-lJ9|fQJPmksQmdss49-#VY`h*KKUeHv)8@cmt%<^A^ZS=1vVvW6&*Ivyr{jnE zlzVC@No+aEFaMH{9g@3?3CAA2q&a%b(Cn^Fle5<u>2@i8rS0wR`mB-sOD}YAr~jq{ zl7F^g3&4Fv(^bUrrk_5@oz;DO-wsJ1rC%7Y!jo{Jo`%wQLg)Vpkyure4XFP~QY9o_ zuPK!9dxbLeqgR^d^NjQ+f-j+44MS-t$+K|;7W)&W4*IB4R9&;r=3kY0(ibSHpYoyg z*bgoO`e}4+$AHVm-p=h9t!LJi|BFv#|9_oN^zQ4rWCENW8IUkjTH{y>6*<sRl<KnZ z>Ed@?`ZuQ-h=>2KS9H6k+daxEp}1v`hoxjpm02ySkvR+X-~*rP)71YeYNAGLBFwuU z^E#r)5-VLwM?kcmJ6|-Ux>WfB3e~Ee?A>XOj_PZw1tNCn`Dc6<@-=TL8@rp5)z{Y# z>o*b=g<g|OBGBH-ym6IEc36xlAy7zt{p+Eu*D~w#eW6kxN4@s3?}ItYgHIK>v!*Cu zOoLU<PEK#Jt7126#X@Bje-(;q;>AYIzc?Ib8P@FAjXf<H^?;$ynz$70k><g{`$EDn zz_UbPZjz);?&lB78TR{<Q$-?$w~gW~q1D06-n+#SjLlf{7i#!IYqv;{U{Or>;%B3Y z=3V?{*;PUhZX<rEb7+tZ;ww<~o6rSXxUFX?w-E8s*yj$>cDAmt^@Dk5Hw*FSXO2_i zKltc4J~#eC3#D6xSH~X|s_38rB;3W`cQlAv_pLFMLQf<rhHENZ=02J5-RsD?M&WH- zEjAU-^X!iL&xOK-)hqW@uN+LM617}Z=HSY>zx{_shUgE~@Kl)HR3vG7_7??-xH-u; za+T{Vb63%|X5#GQEQyoKu`V<PgmX1E9$FVF*r*Sed}Tn$PvO)s!&W6eO0a^w=Mw{` z`hxG}q~8dDfN=*Z@icT)t2q5uG$oyo&|0SzT7Q1lNv?(uzUJD>318a{P2VnX6+O@y zcraqdq_Ell2KQ7vB0HLukJEJ8?WoL4ie;lT6@%qidAFTR<&_S|m<ay0^jR_n?WYXx znC7i{cGd`Sq?4VTWAEzzCw!Nj3Kdb8<kQy-D7D!ENAwmq^-E_C<pc5Jx8t|Un%#rK zAJ}F5kiS#`$hT7NT-T{!>gteajfR0#)2h3l6P^lB;lVDK6>B#xv)FPGiuxiUCH7d^ z><U%oTED^jl?fV*RJ^zK`qOFq-gl9jz{{+iNNQ{D_V=;GxzOJ=FLJwY^hhiIHmZVt z&Q)YCB6-2{$sueS{e7LPS*zD3$|IUBso%LbXXbOvYI=pZ*85>Oc!4JVzEX?Q2Nmz1 z^9C}I2iuW`yY4-tc1zI0BmL|^fDlP45`*@-(p3~zNhaGdcOapvrn{>I4xJ1*hj)E- zl2a>I+m2Zb8BGr17f?Jc)IQeXO}Yij@swqJtp1=#7caYu8W`IV$!#jx2Wmf806eGG z{Hh+E_@j%*OpWl{d_qKz{52vScYPflJ4wq)P5!<@dj=P45~=J6!$x~i!GIp@b@AQ{ zRpzv7PF@!$zXx|8(3)YD)DYb{{c*B#3*=v1dsIvdhZRGoqLT~9_?06}YF)ZIG0bKO zmqKqlAWIiB{UXHCF*$OL3&6&2p(zY{QvF(E%L^3+w^ugW){O=EGn^<)SdNdU<wZV` z*va@8xGIW0XD@TH^QEmeqg0!G_DaLY5s9fBVs0#*7e5E3=>DrmUKXLmtAGO_MI*GZ zrQb@V@%&4s_3spXyu(X_5)R4Mi9$-#-|3OO=+QBMNE7I}#&=({`l?eJ&U)L=eU8ZE z@<M53`H%ka0t@Gdf$9BSpA8#B?ehVCiV*6Dfo*Qn3lV1S6$+7&<2AWaneC%ZY=oj~ zAu+wMkM?|bJ@z(LCQ#FSLM2gh)^l=M{k8X#6uVtkR#uF+dMRo<u46vvOJzQhxEPeR z9h{j{{Zp}hbh3esyJP6#3Z$u_G)J#T>Ds2Twx$)t#MhUGyHAwz`ER$LJvCR7bG}*W z`oaJ$3{Se(njHLq=C{)HlxPbPsZ6`TACs6QQtsYc8IRL#O-8%a^6G2FCLVT3{|;3Z z3i$q8<l|4rTeL31J64C?hPnN8CHlon95NaQBqA}&9Zp7&t(8E9C8`eBdhHlj*zZ@0 zAIPw(14cqAs#K#^B~NAfH!~K6%jB>YzTTCUJr_ZbiRk5ZUB8#>`meZ&PLuqcN$ZA~ z)@Hg2-2c;@E7k5Ix4?u@3A`7T(4EI91<Spy@u;-w<~tZNhK^yr)LSYT`EV}TKdTPk z9QUkLM*sdPJm>X~K<WRNG~}S2Ek}}rb&BX=5%STYf7rduu^46d;~LHTkM3Y+ePr|2 zL>bn=kE3Q~re2?w%{X}zo){-GVE@3~Tlk5zxnJ$gV9#9gIgxHW9_@1_C5CRy+ibP8 z57|#J;HTToRim#`XxcdYNL9g$_E|Nl@YZ`gr=r_wFD1Rxx3{t(d&M4>`+%boq*b+h z?e`HZd=X7O-W~M<+G*kX>6+Nor}%CM(2GdLJGb0})hbwYvK!v(xtI;G@t5}bl?#Si ziNE)ftui0aOJS;gZ75hgFws%{gjY)O&JyBB5j@h(O}_*jt=r=<M{|+f5>rDLb08-% z|6?T%*vpIRFF?kDu1+5n4g;<%Zr0t;aC>Ih%f8O_W8&2f@2H~J9(7Y3cYg_|w2U^s zq#p{=w;@qmeqJ#%2vOr-5L?t4z!!k+r8^HOs(GWHA|CkJ?oB14c#*0bA&$$v0^cT; zFhC}yycs&R3g;S{f)F>#vW0Rco&0`BNqZJ#*+KGIt4JlC>sO&*Lj7N1IdambZQZ+d zWJzlbTdawKX!2Gw2o*jY9eSJ<*h-zJDNP<VQ+544M`#*DIrUElRC`JB9Txu3l7i|m zQ;lSXodn@+1k%<Zo((xrUe5N+!EE0KiFA89?5pwT4T-a>K$_1pM7ukAl=2%{<kvs* z>vjsp#cL13dj=YUif6SG_4O*t4m3!D?;!;RXC-@|zOrFYd?1M|Zb)||_pG8(k)lnN zoZ2OqzE{m#y(CiU=-SPHkUv+;k;R7o(_HVOlGJe3NxMXEBq4o#t!2BjPP{s+D_-b6 z|HW6*<H*LoG!5zM!hD2hy11p-B~?Zd9{%eoVod)Z5|L8K9?Qz^hSd(YL9<(xb%!;s zmK1uvP2xsh9<0c)>yK3iG=4gMOD!(^Q2SuiLtKr3#+>e~ZL)EN3@_tctbJ`{gfA;| z8bC!&`LhMgon!+IljyH7Vk(xY`JUtizVQhJq66(>Zk}VUT2yA&Qj?e5)2}TAQ1I1a zOqQv#(hJUECOMal)T4?Y$U`0o#SjcB%OQqRlbMSA>@%NqOr{Z^SubYOEetT69K*$Y zff>gFzW<^nKPuoL$rsM4vPxysX%6ye)z*DVPjWvzE~?oKw)|-b#(Ag3`=9k3d<o_B z<&+hh9yv8+V9brm?All<&r@k$Ak?M9<4&W8JW-Lq{%zsF*K-nDrUwfM&M1|)XYQ1# znCmBS72$Yt(amjfQ#_{#UExmnFB6CF8>6#k2hW*`%l{LGYh{-HlK!pcl{3|et;nrj z+=+`LKJ7m)4#W3QVDVeGx(lAo#4=<wT(sq#bE{YO4R}AcD4aI{b*JD$=VB_~)W~1k zMt4J^6w(3((!Xkl!T#ghXpo&Q*X%U*69_2|6?tM8@$-0JvhCI7*lC>=)*53#6Is+b z+SGLOlT!O91B@56Gf4HzCIZ<cxs0+3f(;@Tx5b9-xD-9ogJ0FU5a_InzZ3fCdg$k! zaP}Cwkd?N(#3BHAMG(bMcbF|L7c0|+cGQTcm{F94CV;L&?r!<hTT{}vW2Jx7zUZ8N z`Ye*4kMFjYTQd#$ZBfP`HR>4?zTg+)5q4!9u@$X!A8%dXW-7Vcn@gh|#{N2p#vzoP z2Q1Bv?9&cwkLK&t!4=tsPN^zHhwJ<>qo%AT%YMa_pF-sMAgnDKa{ra4tD?o@%ScgE z^-3<$1fE+@xI4^%cIPZjE6o$eCl1ee|C{g;6a%rn5&|W$n+LV*fafK1?N5oHzQZJV zJX((Dr{WtgtDCdaB6qj$U01zE?wzaRCUaM1m#zLmaK_V1_i9VyB21?>ud#qxS!Axc zsu>1Rd;80}Pi%PItX_4v?B|i?I00zT-~=}}Z#+p56t>8`W-Rh?I^cqfK%Qo|slyve zwb|MCbOzM2bjWQEi)_h?+L3!fhT2!y;x#fnYEQf(MOXZtzd1*r<6F3H^&G$5tZ4c~ zx2W-SdTibiGhT;gXP9pn1e?!&opDk`%u5d172a!Y?rvm$;iojhb?c8O8BO7sY_R^) z^-B(7hmepM-k_<dEazLbPPK&7zlg3XS-d74DShls#x}Hmn{H<@mknLs*fgEhl=>kg zJEe`(qwgB_CGa+sI3S=<`xPq2Em0;JG5q}#f>Z4E6IX)IU9{JR_FyP`K7eM{-S$`F z;X2AssTajfV5-%=iu146S`g>WknYJ0Wu45>$_XWxznWMn0CKGv-~AvcVJugY@>ZED z-n>FR&Wc`}iul^b(R3o7-gy4TI#`uN_gS}Q5Jt5(QyF+yy@{9I4sBG+$DLfEs@iIm zRCb%qVkn?Y$4Kx(RsGeQu}88x<e9Z8(&Bc36Pz-Sy#3?9IUB}LZtxotIPl-$4oocP zEK=A+zUTBS*+rg>eP1B#JrNjh`#rKL62^-Z=KBW>Px93J#Vts(5p`0Yix1@q$K#{Q z7{lLp%OgMCXSaL(vpW=~^W3aCeC%`kXd|zezG}D#ZHtxu4Jx`V4!S0lz<yy0ZxMB0 zX(IJ6F3%*Xl1EEcHR{=Ell^ahoW7y@g-1M}aB7rDe7E)DIaTG;fE+>}_qKnKVC*FT z{*)SUs#=786w)95iHWG|vae~}k$N&w4y1Rg8CR7V(eZ^oSAA}7?9LuwjEvKi1KRUN zU2ghBOJ`2_?Q(BrPd-G65=G`RtRa$}!6COjT%@)e9&bwFH!Sfv{v~X7*06DFDIqr? zd^q^b1LN%_OR_w_$J8HECN|&gEc|tPHu#FDJ6Ie`_so&)R(qI<4!F1Sz1W`e_{a89 z&L9%HU`4HP6z=1d^6LI{!Y8^myH|$$?90jQ?*!5aE^6TLj$D)ld5P+c*WNK;u8IuJ z+rH2HqN}&wbX5dLHw^PLf`f~M+Y354p%rLXzrx-^pGg`WLWle^NGpF~!8ss0$1lo@ zc;-qqhC2ZsVr$Wc9?49D$Q)@stv9aLKutxhoAl6s@6Xbr0_{Aa;eJ0h@xHjn)XRp{ zQ-e#~g<|Uy9se>}<m&q@p=cI{jyO+ksF{!M;Df?@y?^w--xM+v+}fWTXnp8tXU8^I zp6|J_*yB)0_JO%d{tfw;6g$SAHMhlNn!*5^9*fz#=bB2g{%r@%16hcK1Ox;ew%Q}s z@m=tD<y@cI5`O#CAg<hEkbG@J(;+t!rfO2($akeAg|AED*v_SdfiO9T6BXuC-18<; zYg2G2?8`m!Zqg;+BazM-%k#8G-xQ4<9hKpYhlhvhKp|n_sE@^wg&w!~)wj;@n0V_z z!3RU8^QjJYg-#<&IhvSVm*t%UFw9W%>e#ZBx!_w-;Pbp9r9#(^kZxqG)3>W!g+4#K z{tt6+85QTYbPETEU;%<#aMvKg2~Kc_;K72sTkr&E+?@a+xVyVF65QQg8<*Q;pYz_m z-*1!q_Zy@C!03Ktty)!c&6-t9L4%R6tXE#c^_fbSN|`4=H%DoX_Y*8fHK$K*24HZw zH;J7igir7Ug}{8C+AJ$u-%I7P|MBm>g^%PnBm4EQ!V{(mu;}yQ1LJK13hMV0imQzr z-)EynE|~HgaT=P!`Ku_^wfKs?+3J}BYcTS_mi!HXhUfl^Ow=Dn<M~axoQGMS3I2jU zjK8^{RkS6;u8b|*ir)ma`onY&zbDZsndk5mVf@&O?E3Gp2K?X865#3%cXxMWP?JRJ ze>vm7)tEy3qsA2VZMtOEwoS*D9YJECDs7vwiRT&_Gw(mk^8SmVew~4l4?u*J?~>pN z3H~k|k;*JT!FB)xlN57u7mgc_(;1MB2GGFCfuJjvVSU$c!vpfZLw|mHwx=+8#g1%= z>h`}^`{zB*!=8-U7yy<}<L~WNctwfTTHL3G)%hVuJhtmAwd^Hz__^a>zOrFeE{<Md z{MQs@E7_a>?0@XLPItWTX_fiky(oMcKe21FwsoF3p2mP#{#Bd|{tD_VeOKfVHV7rh z18?LDu~iMe&u7T%sGWL6qHmui1h!B`Ht^0Ql>~~uh*%($8KOj><KzfIB-wr(qKFS= zA(b#j?8be~Dk-wQ_#g;Bq7FIJj<80o)Anphn%ZwOkemK(2902)T*x2+EipDDJ_XCJ z0KTVc)rXn8v291{Y-Z;5t^qr=<_zMGcCc$5l6THDnk>w`VMc=h4TRJ(g6?C%7hJJN zadz-XNMwy#Lgd_toN*gT5aj348$5I8jzCd**T7e4z}5qMsE-IIiOj5S5?X@U-tbC@ zW}(z~Ve~&5g>P`W2G9e=a^-EV(U58*lMy?T2RQE}_9gz6niTZ0`kiE1tpjTwpDIU9 zf75QiewU7p&G!T!jIU-R@c<5j5Nee{!Sc`}9uS%%BzlH04Lk7TbXP)N;r@MUWeG!M zl#d!cI8Pe>=?z2X7xCUNR=U`PXb>V)#_S|SD9dcTA(I<MhtG)wR4B3{K6WxVE<x3m z>@pclSKPz*R7_tOuD2p<4Duqfdr-gjrFSsSdM4izfTafjYsJ#QS~0L<T_ZB>H7#e~ zLLe}hdLFi0HETn@j`h1#)Z%_UK{{v$T(v)aD5Qs^&h$ERI4{KaXS4S(iky5Z{dekA z6`V%?x8IH#^c)*Fl#AyfMRDP30U5ge+S>vU{zt$}O_;mhV#RGPq$}aCREoWu?C&(< zXEL7Ay0*e#K5aO<vh1JAN-c?$0)W`Lu@TXI(_zvQezd#_?pl6nc-T6^l1A+y8QB7q z-K=a5aj`GAmuMk5uR`I%LRAJ9Vr31~LmAtPO>Z1WW;9&)(h-Rrf=cT)yA?m7ywyi2 z%aJLuB15y=nY$BxW->#lpDMKf!w^7D{8diMxVkj>&7O6aGJQJJoGRtGPiW2N!|F!T zugRx?FC~c-gQ(J*8~DADnkc!6ls$Gr)Rj-rNqJL;UP7K1eO!&MKzM9~JdS_{DNL4h zdq#+Mz4%<2n4{JMqg-!>qAH%xCTR#KIc=F<@sv#T?JIj!ea1LZ>CMla+{gb~iS*<- zrGM9`t@{q{?(cxb`PJa$5_A(5Ek1kn@WW00NC&L0Azng?5Ve)~_)*m5Da8taCjznW zB+u6MdJ#*ApQrppgTD8*8lNnJuW&iV6us-Ka2NT0a+Z1|=|VRYzoqZ=+D=#Br)$$X zU9M<Xi?t=DIyO)9SSVRu+vwT%F-@8F=6si!jxY5W6_I-1=}R<S32B!v(%pDwv)}Rl zS-^!z-iis7@xs+An<t;=Vlv-)`Pt-6m^)qT_CT#NFJsNOSoiX)cY%OU2y4-5AVH1P zy0S6`#AFqp&^JyT!8myFdg?iYqm)(_${$qLL7ut7AgH?$HY-C|mNn$ICgOsa&ItRV z7fv;kuj;sAj&4#G)$TvmVaL}G@y`>cw%Zt4glhLVjXO^V;s1N&dtLkWcNvGZm)AoE z(NoS^FXO>x?ve+Mw#IXm;1~HD$1tPVvO!f8e^j}c5Ap@!mMFI@bb(Te)*1o`?uhKP zAM8*amu}q6?X;q)XRS(-$5z gPqQaFUtIHIC5J)AMF{b8|BiA>aOq%9v@?DQUi6 zZiAX{bR8R2?6|$XRo96dg>pP7tG^FB%6|e2X7m*rBsOf*6Gx<v@C0Chuv_Rqh$0S- zH~hi#S^UbXJ(hMUndMP>|LzC2VSY2J@4&WW3^1Z=Rt=N<_ay=G^4#}#B98zZES*Us zx4_!A2Z!e?P=Dn=VF1-FmwC`%rC@S$-mA4HTP)Ae1z=;={%h$&{l)G+C5(q&@;v`M z@);CQ&(0uo>>sk^(0&p|2rgQEEDQIn&)MSot&)zT><M`mK%`;*r}T8NWv{LL!}=T{ zV>%{uf%Q(K{;}=-gy*>JTE$kgr|iGy)WHa1JO4KUA)&VQYhV!UDr$I?IbMY!`U30$ z(8FR&V*CwQjM#6#ir9apG+wbV{J#IMqVT6ox)${l)!;#6Mt=#?c&z#K#4+?az!iPb ztQq?y9aQ`Llz`E|@0tpWiD08OIq`sgIJhq=E9D>WE$(nGdG{(3WIoZv3DTwtb{+)@ zRQg+F`h%H-h<!LAh3vDO4*g0YEGZIjK93VJGT}Uyx{kUBD|&Q`@l)BmPf`f$A4JIw zq$G?_sSU^Mc)$7L<-~}S;*FME!VMV2-%=@>X0Iw-O*Ca8{9FO3_UED@D<3j!q2y9r zhBI_UNawyngjt_`g+2pW`RUL`AZ-Z01MC|uSYcau{pN@T=@X6KSdFgydjWIn#_LjU z;;Ii-;E9R-*7!TS$MNCQxHm^|kt$8HH(pb5jNB)E#?ZbKX0{rPt~tbG6lA^QUGfjI zRF!i;9O5*Fp!R^*H$jc+)OBduDFI|WRGv`1O;*Br>f_cL0~--QWBC};rM@~CX#k2r zFHcxqUG{k&2{Uu(v7KbU@jyE=`^5`%E`;K$s6_!@uCP8AH2xZ>J)l0);17_k)HO70 zU{6_L9SeReH<O)}i{N3AZc@*DJHd}`s>nd7>QzhsCgE-3#^Aar;jsDcfh*=H81|Sz zhiu}-Jg!=ktOO7bGv|iro5-=Fj%T~-6hf+QW~nCem$S!VCZSO#3k)Xw7fjS2pjqs% zFFW_I8l)5YtpU~N7R7%V1##@Xf3(55_APv+lLAdSdCZLIFGGoO-k`AE5`5)EaB@W- z7i0*sCq;?pAK_iywReNzDT7aO%7{SHCp4Izmun-tg@;`}MSatlAV$40S%R0iB<R#v z5`JXAxt!XUg>y-Uj|@ddH!UMhP90eI<3d8h6Emb7olB$n^%SSms|jM#FU@VXTAFiW zywnvVq}2VqZ?7SJd-d<IO6%`mk;Ff|KAwJwg8~fRjlh@n#CkUP0;yW@z|mn#mFko# zpkGF|1`y9h5)uvCw8>X@L9#-K9W46ki5QSVpph;l*st(Y5%WP%B3M_DDK*nHcCp*U z$hm!MOxUO@d3Yf%)6a$&#WINM9i;)5LgBe{eTv)9LZ<_c+&kGUYEW{;p50C%m>`+p zn%0K@DTUY_`=uj~)Wj5pigLXJEdBu*SL_&I1>*&G`Fc^)XI4KTof5R+f?n_P-_Sh# zd*iE&)|}tg`w|qXQ)ZEoXLBvYftlYAx!7S*Rj!-5hZ@nq<Zi(E3Q(V8CtBCBUkDrI z-}Xy+*Q1g!jEYRkbt6mWNS?RRx~gSFC7>N|JcpG}DLRc-i7}x>er`i&BWuEQO{geh zinD}C45TuaM83*F10<1y@Pw!siF}Upsx|yiSt8ic6;aDuj>{Y#v~*mylhOHG_XQiL z_c*UBRNCp7uh)wY6<}CZb-qgxmmK1z<qL+<VBkgXTp`^wDE5E-lUny<L`CrjR1Hu1 z_m@w+hD!Uqx_vF*=n2;RDH6W{IzYQ$=b4!5k`1HGkM~W)?GP92Ej7`eKnl}MO=c?B zdwh?IsV^tC&?(NPuOq&w(UD>j26&nhlO*FUGlF~9j2p+{%uq||<4x@#^ew&?k|OFP zBZ^^_3}uv>*#xWH+R6)Q0;$*-;=>G{xQub1p${@!Fu0ZzYcWe2*IYR&M6xm?o2T#7 zAlLjL=4Z5g^RE;Ja+s5B^aKdG9f@-1eR7`>G=aBrV2rY^_8IJ7&m8{KC!^$xlO&(P z8GQi{ho+-O$ry3ZWD<s2E@fG<qIQ$anoc`PGTE1j9h~+R1qoGprk-1=`@%7OJRoN0 zkQkcqS~O0+mVm7R)VBz!o6Yd^DO8DZI*O>3m))fb^!gsB`9#zO_M9BmAnaq@f}Dfu z-TJLv?VKzmy?UtChGd(-#)R|Ycx#dtD%C?4W&ZJH*ni*S8U0QOd!C)0iPOFUvLGQL zsXy>Jkt$c;k#IXf<;By3=q+}0T-Xt;vM{JKvlnBiGOL=BU^t!1=4(Cjnl|33(+T)L z5C9fZnw}Y(<IhhS7)fK3|7oPL?(<h`EW5wA#*)m_zlS|F2aB13Kr;<<OZP&jnltyO zUK&;2Rb>-z`K>&X?`+xeX$wxsUpuj((eB01uGmx((H3M{oT7>ZtUbpvWW4xIPi#GL z|KsY#L1tA*V)rid&*;m*yP#hvPk7-V!Q>w2FG#Y~e4oCXp8QOO*^?R9KNWtB2M6u{ zPR)3^-C(Fr+e$wVVHlp8VOW~_QfbZqm6yp<Fu3-&jtKe#4Edd48Ty|Di*;4^sYaVa z!CHDx$aBdbK_7APzvv6TOF{6vsE{rl{%(!sDfPE*8GM%qHM}dNta$_kL;_l2ad2zT zWuR&u{>xa*gReG6NQ=MJYWu#_wm5QmW_zx8OB5wOWCr>>F5IeufCYeefa?ZaLj2V8 zYGyWGZ^kQ(ZYK5_CiAuYljFHo99KWws{kLg1Rr12GIo8Ku@POHN?*;V%rV?PdDuDo zY`p+AIM4T#J<%;z0oUU0{FJ|5e!<KSSUb3}Fb3JPB#V$1v>lX_VT6<Aa=i*=Bwe@f zuMC#x!jLejpa#qPfK*Z|!pHBVB<Lw1bYGE#@8%{2N|T29lD+9+BVT19e+%<2#p>z~ z!iCUPOKuV7aPC!z`RLL&y-vV>*%~l-?Gsj+2-czCi)ta*neT`dB;j7UVqT_pGQ5ca zW5hzO2VYbpPOz{e2QXL8{a*pZmdiF?g&3`T8|Z(Fn}v05Vh?Sy;>-$e+Hm#n5bk&z z(WVQW8Y%;$oZlgSj7@0eM$SHcta^gh=xo^3b8zcJd9Vu@>YSWNu2Z~odTmX{%kA&* z-py3B(*#T2!Dac<U0y5A&zai4(SNvlUvUgZ<iEWDAlnz&$gLSU5q!lv?0eVP>5;)? zi2N4;9RImAVVBVm>&rR$7%g1tH}OD{_(AYVzos5_jx^zQ0bHRdsBJt%-CC?nBaTpm zKZJJJc!jePfYU3H^0@!NRRw8eUsBEof0{(SnEl-4ptafj!e*!9eY@PhE)gFoQavMn zGl~^5O#_Y0L^WFcm2zCAfcE?e@1$Gx8a`h@s)cKS)p*eS7wqd#3Kt6!1>e*@{#t8o zonS79LH5~3*7Jl-6oQ7xHLOYWKTnE_9kT8&w9sCX{xnHO!XyMgi1kJ)9!*#3(`TAD z1&P9Tjq_o|E#!NNKufm`qEBeV;}U>MMaF>Mb@IdT-yIM-`z4rUAu-Hd5?<~az4P_r ze0i^iHaz>Pi6szJ_wG~yRSR7^*8q5m;|xB+=oy|>ei<F%X%=;>C%{-$%gQboOMBCQ zjQUouW-rr<TQ|ntw3pK0SPFfSN`O|GBWYnR9tZcU$?0i>|M{q;WVEnN-v`~;K1LU~ zS097PVuysH<uzbBl794Av*(Y<5-n}7oUwzwq4_2_M-zZxp(bRcGL#h&jj^W$4}DjD z`q2xL<Y(8|j1WSaph&fqR9$!!uNam-m#wg`#dGwu76(MZnL+_C@eU>sHpvvC)OP!m zM}Fm$vf?x)Pk=J?V%}Urb6Fq*O~6Vi=Bp@ljEzm9^=|ma6}FfB*v5xY1(M<X5+2SN zhg}pjMl58JMP_5uUJ1Wy23ll^m$qqqtULB%eIn6l);zok%an;uzDN}_kWkroOe5zN z?8l+wZ@{IdPAuDS<&0Ng(Af=2e=uWkR~$XTgS`1*K4bXrJJpO<3iSVIoK#4yyAba} z>-(C{Ie)G=t}CJ=h8K~r;1^132b522B8V7$JANG)xIW#Be5Ji%I7`W%XtG9g9SLer z#DNvxis<WEF3|dTn3Kxo1(F3=iz+qPP~IQQqI6VLg-6UG4CB}UcYKEp1~Qy@A4hq! zd)XT8T)-_ThGxg+pL@2PSAX^9tTTD)%qjZR@2h!xNzBr)(D?S97Qdun2Lkz9Mppz~ zw6b5dIzptMc|*tGG%7>6G|+enL3<R;85^&*bwzCK5r}9?6n^c=_qOW=X%|OHTzesX zE5v7{2{~EUSi`^M>dB>K!a5fjt8HffdOYz)tHJcjxMZizw$&9*$_%rjF^2gs?1WYW zR{cZgxOB5lviN=-*L{`;G<yBJX4J7jl|DH)OO*+%eyQ<>sYJXuWnJya<2GE{2nQej z6`H4@5~&_6q7@(8bh)wN6g245y2_qst`#Au3aL3^;dIqK$+Dr%bOb!wbFKz5bs*BO ze!QXu7v$SHAZa$RKJ=QdNueR(9`P345uk`TACbI$_tKi$=iY%p(_B_7HL3H)mt~gi zkb>4=F7?K27xazV17(f(xtxwo9rRPm?yqWiwg1&AYzGBdoi_HowU*z5A9PVpT1PIx z;fkxLZS1*T9@x#)Cv|mx(<{yx4|Uoun7cM``*R9*H=L9SC+1OAlLhd!g;|;x9q%x# zd)e`?1<8UwK+4}`0fuh>Wo*zO9c8PaTr*Rq3SJIC{<jvMhP`k}(CgrLAF;;cDvwQ; zEa4h{6>w`%^G7p-0u%azzzx(Lt<3VIDDs+JZhJ5HQE7Z!0VXDNf~ZeJJ&#DFiSfs` z13xyEv1x(qGlt4ua%8>o*V6Buz|iDhP??LvU(&HliOK)LL-9sSm>*W&7<P^IfA~BY z9t(J-J8J^N^k1RK-Q#`}kG!T-%@L2s5O(QzK#O5Tl#-o;&4M;XN=iF!7<A-1lDMzL zyN*a3hZG;OpR+zT<SAN_B;rwIy&|y1_!26d5&2eMoXSNP2s7{pZ;2NwwuFbjG3Z=) z7YDEt^EHS+x}BRl3|pbUf%wPAg%vsteB7pYXXpl1mZ*1V24Z=01$HsoeNSgH8pS`q z`+tAx^Uf!i+u*tl8E6){^&XO3@Y#<ysZXvMv8PSPsi*pS`YQNNpC~`2NPdBq_}9;} z9GMOA#Ylr*KYJX8e5!H6Cf*Rjn^6RWh!@X%T3`9()_0%!;m1vo3heDHjv=)rxz(`4 zhTD~_Y<phvWMf`BWzm^TNNK3+#C8R^BgQ~PK6kAb2zR6qRpC~amiOP8E+22e7d5d| zc7hkj^<AhS4>Z~0=sOa|kH0OdZ**;-$q50sq&brm`GNnB5&gfT1Rgyfalz%hA@v8c zNp-J_{X_On-1sB@>=TlKL4bER)gQ(!JiJv&`}YVtvm4CheZ`CAlUpwi1#;ic+}APZ z-lq{0_O`WUUi0wsMhT}(@l0)2soqO)fGeg=O;wyw_C^6NPJqNdo;np?&8YdI$jW^> zH0+DYC9Y_{CcZqK*&3d$9-nkg_ny(6ppsB`CoiPyuzbZ5q7PP%DysH8KB1cjUgBYw z9HI0DbqrkmN9uA7TkyAnLmvsb#I;buhB{hZ#GajW@>gUwQx_xZdT$$Dz%#G?aXm|H zL!sNKhP->(TCtg1it8qRzo}VEw`RTEk!IncilBKZ=8||3c(PGd8D@8CXQ@;#HCJBI zM4rA>(zjF~{{3QYZ!(hCyN)vbR;s+Ig5uOwNr!(JK?gCs&Nb3XJ5IMT64m6ays`34 z>TX=4%RP#hRok!%*W;C}&U#_`S~`dX=zd*#tvb0#)K7}EaC%Tt*rXnBkD()8{oqD& z#IFOK911IBOaV4bF|EV5pQF{F5)1Su$1|<7ub!^dQlRlTQPU0oyUqSK<aNe_Eaz{R zHQHcE#bsc3&iPUa(C&uxAhs~@;32D8gK9M4GLLqBrQl@<w2RmSUQ}2|%~(r#y3g=x zYk!Q{D&feP{TP3pDii4-S-h+H{r1eVkWuRJn(kWJx{qbQ*-cAV)oAZP^E#5IV*z%) z?Zw?ih#^oW&x#(8vo6Q=8qq7wSv8ycVx~U9h<&tSw0S-<rnxRGan!pdvcpXrYm>hZ zyH9G22YqeNO?GqmF>0{GUSo-)O`_R>7EY6n)tc#=>6sM<UM;QgkaCWI4LDBv0?`M? zT=(#|wBZsR?ky@yZY1dxbbu`6#2)Fjl8B7L3cq^;FH;THcSlZ-Os1X~eKBgG3NyTE zx-kSqR!;S#4|K$O%2-!ZY+^<!I?nnZv96oz80*$xc<gI$37ti_ABP|L9%evQ?C?zV z{woaGp)7ahOF_KwO3fXU^T=u5k3O0L+;&ZTxr@O5mw<j)=tq)gofHD}IDdUr6?<FP zn#cKa;6?)*Ezi1{mY0QiO*5IfuE+&`P<g0n(|sJCIUEZ6>guH9<6|tAS5e<&T^l6E z4Rg7k9XImaVYs!Oyk(!h6&jP1-b289x5wyxy&tFSchPl(Gu^|hEe;#^wI+G4T+VIK zE%p!lYmajE@!e|Im1mNhD^!ggtoU=Jh#@xin=Q5gU6D|@Ka8?EWJ(z?*5Eqa)0!_z z=H)tvbDQOho~Rl&o(VbFfFbMD>K~G|+M)Ccxkwv&H~Fn~F7Hto%h8FIqq9SJjp)bc zomG8fo&fVI=8js;rl|3g{6?=^#W*Inavzz8^4gZtT5=#AjUzhOM~pO(XUD15=Yhb; z8zgI{fyWT~;?}8>kn+lMi}=2;SF_flj4mIZU*XJ=FHGp=I^RC5Xb-46nNx<Nf=JRG zHf+`a>^KJbc6+CaH(-7>D2I|muf(&a8X~K7=u>mt+5tO!QvAkiq(OXFPil7{c6pgE zZdcIUDViJ*NONbtg?)L8Sl*}<zYy*X+$?a^Xv_zyzBjZy2*{Sji_%Pq_qtScS|k82 zB2O9+-R|RftzPNGuT{j*4_@ZfpTSn7SJ;&$MRgs*5;b>GenA|a*6eCrZ`M+EUjOVR zmrv)?>Gw?B0k7dVXb?l{vi_#iBie)NL?}B`hXWR{4v!yq>!YB<2B(+~5FQ5g{F-iT zHS717)(;A>AMOjsli<qV1pz@&d5$GU2M2Tp@~UvRRX;42Gm|NJOAbqSPlhJ)QuMQ$ zrtrl_3v?wZswj>Yn^6{{EhJG^oOW#P-ZlhxskvTm7Ptiq8*xQZAd|&L7lZ^`G`Ufg zw0JVhfzn5J^lkVBdW@JMad@9Mbrx_G58W{7`egeV=F@!i)^K|Gx7~d!0nDt^{ZA<a zx#kvWQ$b?^*J#}uynCxF^X+#Bs?34!Z}AW{2L?ssYHU`K`1XLdhfJ<im^&&{Mu&KB z^NS{Co6zp^3%lj6fPWab<qPPCHGv1=O?^a<i~YoFb2dETwC%TiycX1#Hz0HS2i-@I z?&;Cv!;u|!fl$JWW4V&4=;3sM2H^5#3ycY>!}o3@B3sCO5i&sku5|js<j=w*eb|#b z>a5cQlltNGIUJi51r1#ljfv8MKR-rVzUMh{P57?GQmvHvjRnPhv`2Du7f73ifquzK z^*r7GtL^iM2KSXqzB#>p_bC$kqx1PER|D~CRc!B`pDeP^M(N|v@VpbaKvXOB&Cm_; ztd7_I&s*1W8UnA%?KT)ur00u1q=7=({)9pTp~YszA8)-Mj|>dBcV?#J50~5i^HY>j zU!ABw02uw>Ki$xMFrp8T9PVy*0Bf-4OL_ZAz2Q!><DY#OFoHYYh7T<de+Yj(5m+aF zyy}T4bdyWx3tR)Eye#_Da0n!$e?Xi6<#v4<D4F~DZ>Ds;`&#bjTKWVYujNK^n9h%` z&$c$x_}!6uz%L-Ku}%zy)BikN1V5j%j9!YTGIH<ZnfHU-weI7dcVbIdm#8bfn32&b z*o!M*T{p<m-^e}hn*<@leOi^O!uf0UT~kCKSjcJrA5TtxtbC9Ko<^^H@U@jGZ)Bdo z_OS`rPA*2kkx)P-<SspPGL+&E4qY4lZ(r7?#ki;ZU!(T7Kd;`x|II>%<;}aJ{$Ngw ze>W$_*l%9Zf3OK$=%=Lto=3R}(nph@>;{t0zo1S0-@fb}-!IbgELi^n{eT21H^F}s z9weLgcoqKW@)-?>|M}r~fxwI7<9$EneV<qQcs=a>Fz#OAvIBlBFg;%J74H5|<^_RX zO9lBfMHl-se4?>!&+Z3)55s`(3uItLhJ?T$5Szp@D(UfZ^5y;eLR^*S5j+I?6$AXQ z(mPFZGO|DJbS}le1a90K3`YF%o;7a-J5H@5LqnneG*0ux5O{G%;q1bse>0H(Jv=^3 zzO4d8+*reDeD43RfsbfqfVb&BWzZV|YnYz5l!!0Ae#7b)yx}{4T%zR1{!drK_&^{0 z2Z->r48hY7PJ&4d^{1Qil0c$1zwUlA3_cRA*RjuA{TFCsspa&?yDba8Mz|%gTf2J- zE0?ciVE;HX<ji!CsAlR9OJI*WOiOeg^rt&WET_TS05;qq2lf6qk<KLGi9Cvm`V*jg zfiREpG?55we$R$(@C5rT1%St?Md|}9(VzUYjU4o6@VVl@kOKeo{9f3QZ1j(eG<`0A zn&(zNv<&_0G{Ltxf3$q<C(CchCSUvG_+E;DE&pl22z;8yo$c&wCLK8+ek~PirPZuw zQlImv^lhNzoCS?~D~=r27+O0DKU~q<&xyGE&)26)nK~9*<402+G%GDc&bQ4(hd~8U z{eO%eM+SU{5iOU?F!*!Lp7M+9<jR|So|h}8E^}Oqk!mHnA3&!hq0*jtRxPpCOWgd0 zC`j=I<-sj}^$pIK1Pvf4jgLaLQe1gX-FqD~?`7jcJ={m@-OGry;Tg93dyaG|l8hjo z3hMIitOzq@we6&&eg#p?<ne2-X4NhvbxG@f?i&hGJ7raiRjBhaHZ>cSJDUiU9vsq+ zpDc)Kh(5RKyO<k3En-h;xt+q%;wk8FTO9F+C@vlkM=>-pmvUM@v>T9^cihqnL%9rX zXzDO91weX)R7z7OgG%Rwrr~(XVQ_1l#5y<X$tF6OQMM{@)VHg%Uouh-6!CHGiI6!n zY?(N6@$i6~`<RUGqU63mFZ#B1Z*Hoc);F@)<`2Q>uPG4KL|8gK6TMb(V${RFE>J+! z*EVw9sTa+=uQj=0BX@n&+Hy19Hfv}a{7#TD%bEvHk?$xJEt6?sGDp&nEbI!P-#Pbp z>+^mIw%o^<7%%dZ!XrzeDBu0UA4K0oP|3RI?|czltPUaCgI0DHIJ4XP;<5{0mq+aO zUmh*xJkWIq&50R#>BPlvuHS;UPJ~rSC|7SeTj-_iX_s6FYPpy9Ih!o{+I~=~%6!rg zn_*z;Q~dDbw1FLmI_GBL{y~_?aDHtPGTcocsYXHe=wLBrT8oL5b#9neW@{5|$a%X< zZKwUsyX%DbE~9lO>d_h=i6dE%U`CT&c?BNEn#ES)9q7*56|`C8aRo1YE0JuXh7$J> zS2c)wdVQzG?>!%*=FNnR3*-S2x<0H{Su{UV$1^i~dpyeO@av}Py#Rb;b%>Pa@GN5D z-UeMt`QNWpIMi#mJ)Tq^r1g!SVV_>y!@W)TC{Cc$;=%~GWpfoRzO^r(c7Lz9=nHfe z@7V8zJgCV(y%|hubUmZI({v|e^0tfkiP${+;Ks<L+Y~t%PMmGe_aT?N{A|CjQD-hD zJe_Id?o1rK3&^2I9#lT+U#u`WB{*#XQ%uBRnq+VqQQvn6tb0S$%ugT4uF}2ZP?OD1 zFDsfdeS!^5^KSyd^H~I^2|_)+03mAdy$cWVYfCU0eh{i}VKuek4e{}Hfp7dr9PyzN zUAD!KJE?nk>mUUC80WyDZ6&@d@Rnhxo}wp#V(q33Orjh;P7-tBpXyX$!VcExik`+= zE@ilaPbST8#ZtZOJmZ~FPib+k+jX}_GuElLrxDnPnQ5hOp)cON<GDPkC~YJO5_hMZ zmt0}6-q#ab=14g|QfZcCU6`ssF@E9oUN80_Ry`t+(D?J=irkpxkDm76eS%MuI%3^J z56Px3&dW3JtejtsGc-!~5uG+V*Rd#uo5cM<1lp+=UD?Ud=q@`Q9m`)_pyaTgogenD zXnwYCMkjvlV<?4>oX4ItT6loRB_7MOP(gNm3}rwYOPQb@>^vcu2NIsF4)?p!)U_h? z?rwoGz2ME8V#=8sul0^^+Rfo_21SZ+fGQm;DXjTDq8&~|iX)0aW|(i)Iepup)?lwM zgR(cWJ^f4)Y(2-~NQpeIlpksQ%vv1DnHC$dp=q(z(zGVgj6w3M^IrpCcZnOSs?Xi{ z&YT@5qLYg#F1J-%^5d&M3MUQXePZ2gMQ2KS=_*54!Wge3M;lOicU4NYH8tXlMLzXY zGHWJgwi#rpJS)DG;J_dMF;+2$gm1%pr%$x*#TKzG5h7b;Gf2}h%epK2ht{V_JJ6|^ zSy!dU<FO>8wx2?PhI*7(5#^wh(jB!ldgMQ8tK}N1<DbaDdzOkKue&Rfr54Y)s`%jz zmrkncHN_(Lutff0-TnqY)n6<Ja7sD#-pdxfw%Lc94>K!xyhHu6RwJkxWN>vU78Vx0 z-IIazJ8vNQm-suWnhPGjwCglsqWuUTR_-w2UM|f{SLnHoumhm1%39IjMD$u(tc*RF zlye?yyC9}|HCe+k#(ChXvS`wvlh;3PMTiSvI71N>Xa<o9R2`<M8Nu21ZEzT(rms)x zvkmTmx_ulxm`a$?DC3xzZHFuY2`X>~1A+X*eM4=L1fTfK@DK9|Qh9WRu5S`DoooHB zeAckk$I)_;{YzNKc}5mXA5n}oi`pim8IHb61^nVnuo|T54=T?wg|rIl*%F3*b-#b7 z2kZkO0hKNKPT*&_w|dGkH(*hO(|v?R=yH;<>%HdDI9BaluqZ{CGiToML^J4G+rjg! zBApgk6ukeK#}8PY6#Xb_q&fQqg8aLL2^{~JUw(@CsN>eMi^NiPxZJ|Z;!aP6iIl|S zyBd_Zdv(#I&y8WsnhK5%z#VEs0zRKIx*l`pDMMhC=J?m}caUCWX*)&G<%ygP@mB*2 z-L6pSlT$3h$`up7t~d|T(o5*;N=U3T9xs|yRB?`ZkQ~e;QOE?NHpb+|CbG)rTi0+- zw2)8l-_CG{;#@O#IFR+GbF~Z{18HIulC16VS7z76dhF3W{rFK(U37pft3O;7&p?wa z8p_ivXEdg}r|C3IGF)}EL1lL6eTOCqR(nW5=6&2c&Qj&ppB`o~+sPj95*9OU5JuS? zEQabq1_l=NSxETfi`QL78rnKBT`_@Q;ltmtrIF%h_k8z!Np@8~i`=507t%{t0B31> ziP8la8O&AezIVn+BpWIuMtG1RP1d_VOqF^2fi~W~F4F@Y*1&$`Q$H1Rd{me8)wz?& zudGy=#;EwsjWl|idW0Ux=^+EG#s>!a0o0r}f3>ynP3$uIp>lf0AKj?%xpt#g)f=A| z<meLPm0xB-G9RNdww2X3(gH(5qLSof2<sFH-vz#J=Kt=cK!P1NWn}J&n>k}(YBZB_ zRe9e)`fYUFxe0z|pM{F1)77Qc@qJtw+t?>FG%e5EyMG`#AAv-{GWB!V$A>e4M~rA1 z;)hz(IKFJaHC>GPqG;9%yR5kmO*+b<)B|~0#(~{rb9@o>5gX8TsibfU>G0HW6WT48 zXp_n{aa6Q(uUf4?&@_>$wo)+31W|{h$n<D5XG`T&6`Ne?xSxtjr*1npcfAQwTBBTi z3|KIfeWsJ+v^ydDf#~Lh5@4WTL$|5J-EuS`E7$ZPPp?>IsXx;sZL2|v>jH>b(zpjq z0PHP@W{7_^pv1T`I_K29V9pvOXxjW1KfHNgOF}PQ5klPG8J-)-vv+Wajm!fo*yL@O zY6@ZY&W7uNjvo7}i7jTJ>uS>2tylW%p>bHUL(-%If>ZLy-EzrKhYYErRQG~*<Gxk< z$GA<K3m{4=IdlVSMbk_K#ysf~h$mQrWg}%$uEVuD6bFMFE$+^~#i?!*l4kETR%x1l ziv7XpC|mKO+$-gDgV~vrj<HCk8ly84$A52ht@8DV=$iN}5XW=8vd@3VX6#&VkM+pp zCB@Nt_Ub|Xxo|ztz`@5ppz-&e;4_Cwpa~maC!MGxqh6E1g~kg!5#^TD`4!I;iWua# zyfW{EcTl`5(5(8-8T9!rry7Nv-lR*+0=EvDD(R@AHpe77-2o@_vzP@urZG$#FGu@J zq)ub2K`XoCK)9Vy2bO-vjabCY!bD%-u%&LxXxboRgBSlG+;x}R?3t_+;@4zkP1Z~u zk<PFbSqY_D?h_E5=JX})FI%dT5MQ^~MLg`<6}`hZ=A$l`%Be^MoI1ESiQlrFx8LX| z1u`w~_jg#xT7T5?^YFxQpV4Bwzv0GX(u6Y<i?bXG(#bMhO)T<vUF2&RJR+YVSYkW% zz}>LtNA=7{4w@>)45be^&6{edJDJTj($rb}zJT68UwIS9=(cgn*goLi@EN;~wY$!~ zyzZQ1N4F5#F~Z*YcB#(|_wuNpF7cEx9Np}Egq}<;2e|k(`%?EaoKZn{<~+gC=w8(b zzsQ2wsTIj6(bph88X3MANxZPlBjY1joisO?r52}o{6{mzG~8=Q>whp)t<jVj)vNDs z%XEN0lfy?!3673#QdJMA;Zo1(N?@TBKHo{Y&)gNOogq6P%r-CWObxN8wUx@7FS%su zb3qvxO-sskxa=%?j?z3z_*=(}Gryh9$m?$TZp~7Jlhs#z0G;d;5bEsnkp0%o=t6ic zN9>@VlN1J)dFj>0UB!sC`|b8R7Fg-_&3g`$o#2NsR3`uX9@ebOSvNEyPBpc~*z3Kv zcQzZTua$%Jj-BqcaI(ffr!k6fA2t1`G3TBfVkWvV8S>x~$Acch94@fxVBV1~9a4W) z-*W0S0eB7Bk&kCD8$9^}YnaJvx+##wEE^7IHn({M($oQ+#fb;1Y-8@?7?-nsptG(m z)EEH2np(Ls7kiK)Mad?S%-q_Cca4bFk&IlU&Uh>4zu)U&vX&mmQfhc+>qMODO;>kc zV}fKL)<eLC|5Nf-@5^^*eWTtO$ZD1u`LtTYs(R{0uTH&jtC;&^y6%Ml>$_R!NlV@A z?q(^ttz>?9z~!|a89ZEi!L7BZx0{Y60B>hBVugD5!##nCx-4b#Se66*0u${|f{4lY z&5%XpH*xRMZz63%DRfaBA$bX^=GK+=HXN+f)GU+J(PUAtE0?#}v43o7t3&A+zF?JT zSa7*s0~I$`>+*l*=8$_CIQNb$;hVKc&#LK8ad&QB$ePMroFv{RCTzi~DJBbjt}_`} ztZk>%9L72d;EY<}kWmpcnO`6#YO{-EYr#PFEGCdN*PcI@1;!o7*Aaryjy`LIYBcFz z6L@-soJp2fnR9qkJID3}B4imF+6=-vt{vcW?lRt)gHIFHA|01wBmmKU-V;n;6-{IC z9{c>LY(}-{MPk_z$p0u^n)zkyoYld#-n=6eWjjFbV8)@tPF=^$qi2H#RHX58=0h_F zj<OBdtN{YIXx?|{vxEg5YiH80-pKu9|G&&hdhCuHr~d}nAdAPXViuQ&8N>r!O%{Yg znZ^mwe|soqf)E(EarO3=G1RwZ65|%2yp8?cZVlf|Y%r#y@Pj!{MTSAD%u;Eg>8l?~ zuy4I8jA3;+SkKhpa0*dJG-MNuf)g7x1}Nv4Qe6%{f2=NI>CU#Ftp{Q=W9*Z&U73bj z|4L-SfjY_XG6b~E)t;D4qT(scQkgXp6$w}S@p%0(#kwKH1F^j}|ALGJT>fD-dW(I- z%u9(=c!1P;o-pGz)B@3gJK))!P<TCn`j+Et4wVa!*qPr9t2ma#F!$soSZJd3j?l;u zfYX2HA+SCWFEjSmFVTZrc<ioD<9lsMAOcl#J#aPAJ|5M0sNuZ+BoxiYU%zG(m0oMU zHQ@k%7y4~ss&0P#(2Li}5WAxmzuw{g(6QMJb#2IHiB>q~<+Z&wiO>|9Ns$)zL*`h3 zx`uL&1~ItuPvEpNXWcQAd0Eh7Y%28_;MF({rnBtsMKWxW2pc-p-59~O-}#_xaSjS9 z--STDMkz}i@X-~Uf*lj~1g+u6fLl?GvoOj`8~~j067PwT-+t72$45*b?W>Wvu_@TP z1~9CVQUXmPpGr%p2{_PEHH&<W8$|1K5PXLx?w&?`@Q|<HFXCv=I@IxXX-t-?Z_`-G zB$(>QeWstA(U~ZKPN49}YFVkgL8r(zm`SIcl0Q+wapkp-R?-K<(u@^vnk*yDa~y&~ zyTHY^_hZ{UW5ZNCMpsT9#1<WgrN1Um#0!gw*9rj|y$xO3sC(3M=Rjc%j|nR`%tyK( zjkqfW7Y>R^ZfaJ`Z$oCbu4WM`+5yb~hV<^O#FYrwe-c$`{64mU8{Qc!cjPGA*cBm; zkUZ*87r-fNp8?#8W*w<~^<d?2i#n1XeBfx&QC~<^c<q_WkY<qj(y3s7s^!ex9U(^O zOGTdfc0AX!(H6FnL~qsxD!a-m($wAR#z_!HS$b1(L;W_EHB##7J{I)H9=JC&igW6? zUUce63TholjDZW+&a45L8o@=JApn9C3oq9$ZypM`14NBtA}J7(xkdZj&=k|@`^R0X zRTcY5szL-;M}=qui;vEeZ8x*Ew;-U&nX~Fh6{%9KAG^11q5fHAXpDJ!DPuu#(@%3o z>psn&lWeB9Q}xZ8as<siv`XhVjn%`jhp-`N;!ERaar>5}n88(%_<USb0)d|bs#x_Y z*n^m+=I3n&f{wln!#gvkY?mnZuOqSsvE7^+3s08h8yI!k(MV*KoEqQDgY)WSlN^J+ zFF7tl`zw6e`7%k$quLk3i0yv7a1MD0xGx)u6HSHb316mvmq-llOBE1{`7MuP@<!P@ zPF1nZUH0ee{<1!C-Q9EthKcWmCG^rx+!v-e-@cU(Iu6`q3&4&`xaKThZbUpnq4aFe zoqL$^aVYxssFQv>&BL&J$yu15qB$2socMj5Uti`@w6y84SK%<$L!?sZ6De?-faCt5 z!J{44rtScS%Pr{UjW=n5OBdl59M{a<sc@-uw1okKt>-wGK$P0YccEP0^+Ro0Pn<6I z1s5B|zbR7#sVuf@Yng+nQ-G5NH5f~XM4y_pk=F0WBiUqbi}EeLCoxP;imwmP!K8j% zz$&S+nq*H;KLVcIAi6$!q8I_c2qaZ|x3GYa;(p<_P}aE?g|j!^8Xv4c=%VWm=`ZM8 z5*Kc&dMs#1#1d>aXCZ}D<1!WyFEcV7?1uHp`MQ~9o=H#gPv%rqd^+k9B4nDIPZ^{z z9nUJ3EN_hJnA6R1TWk|6rH$ru(B9b2+*=0rTrRr+=1xp@5AiOyj)&e%{v?(h8jhfH z5Ej<Pe%NKc1U}@%l})uq84uA!`3wq~=}cbo_bqN&+qo{H_vJ)=C8*uenS?Q;i*-_& z94w$4O*`rhsUcoRx9m}|XR;Z>_+P|kdTTTby|;Lf4E}u?SyRYhvxUj-ue0^#dOB@~ z2*xtgUo$5foiyB$h#}16*Q!3_FF8;?vJQ{jABPw2TN+>xo$F#v)=(gJu-1$7dL;@< zp^QuD{_F~6&8%TYPY97P!T$EjiOlxLBluM5!b0obi!Wsd^@ttEDe2Xz`FEhz=<#Ak zm^G*eTLy8dVWYD-uKi%HvBV?gnqT_TeX-8uK5s)=3m-rqn$xNZ_3M^mxjlMgP}`BS zl@r3uNH=G0gofVNjH{omjHS^NAM~;!8EIP#RXb<k#dIwlqv2_jouH%)CKe0S-?R}T z59nMUqAjW5cq|$8+^lT~Yv{@*0xnUUQAm&ca|tcnzA8sxSXgl0l<N1xS%g+qy~dGj znErBfj~d#gr$mXY@z`udB#D^3z#zt(Z{R2F{*Fr&1!c(a-i93=5$9yQ7TL+q$rmIS zRBl^JQnyvIF}Q?pH^Ku>UJnQq-iCT`^F~1-5%786>pTn%N|w~?8!Dozq*?E%%uFv5 zF@M%ZRPJ22q>egaX@caN)p1&!(`692$-MKl$4<U<<j_-2sIV&Cna>S>LCTy&B3%$? zs<AX17yGqww>Cs@t5OgFCBC=~FhT{tM#zjQTP=o@2CHj(iUN(YD{0iA$`oEOI@<=T z-F!*8U`zDbTF6r>MNZ~j7$C%t*kzqcf#2~k*JiKdYaxvq|1?GoamucYxHYL~DqD9$ z0OpKes?9-1rZTqT1EoU&Bl(?<)E*L(asN-h_FMb=Fvm4~2r{PS@c`yms`HVyCfX=J z@@CtHPk71Tg7Dve92^A6EF>V`iA!*F^p2o$konsVT|0Q4_th@Q)bTJLpRg`4%82f| zCm&aS8Dz24u_p<=0d*mbgxaySd;u&nvpF@wWR=g-*Q{Mb?lk*Dv=T2Xnoku$yn^~w z^x%uj8|LcjNrOME(7h*<tzB{$Ps}2I@hQrJ21#ef_M4B1Mm7B{wSc^0zmVQ|K%~00 z=xCSLVAgVXMh=7Sd6>6!G(%N(RhUN;TiR^=9dcNF(CC<}nYW{X2A|N(zURbRR>~y6 zEj3oGWcT|TL8sT&KrFy|AA)h0N2>e+H?LH)<2v7UzTTn_682cEX~kjuhHE=nM=t)L z-W^b_-RWF|dv^C(mnvtcN{0ShGdE{CtFLx4MCE&do-N!n^na>!d;|hol{5^vlv=kU za-y#3;_0pmD{*hx%3W8ztKVIHvtO@RUyfev&FyV#nvQ~5*}85Ug_850yz%R7EfUqh zy;7cfIo461`0%l~fudyO@H#yucd*-%)2eB`+)Dm6qiy6;i3=bE@#uAW6&z6cYL6B+ z9npYrS+Qt^ea^CEh2J|or{SI)w_++&3R?$1ePkDJq4zS9f_lEb+|;^7@ytwp2|JKu zO>6=41NeAKMQphy;>Qq=Y5+(}t~AajNAcvt!^hslPkmYivYdl%FZfDUCehi=Lw2b^ zP4d;BW^5F_>ZReNUqh|T?A0#b05#ToipuhKmM57eE*-Pl%hl}gF+U+I)Q}3#=?Y_Y z>zCNg9^(O$YK}O@knUPeWaq^~>bX8a9m>+RMPa*oD;4Q9=yPN$>c}n@B(<38aEF6p z%#E+3INGTb<$60P*vy(Aod^|f6CK{**XCJZ-|Sp_;V@k~R%-V{0>7?HEz*qYa6&Jj za+uiTmt0HfUwNEe5x5iMm)2DxPwt__naLA!ffyO_Bnmve8B9<1sb(aUEOynLq;T}k zih_0gP9(F6Byo8PJ$@xmFI<UM{5Vqc1;Cv(sFsWmi_Ja0;ES|Szi87%{W#X7%=L{$ zDp~Goj`G;>CXPORm+ElCu5Zh~F_>4u2i(JQq?Q7Xa!UzlOj|?Oa8YQruTeY!=>*;n z1dd%r5kLs6Bg7E;g=nT=jfoMT-Wrtzo>SKwDP+8@&zl3c!go1SUJJg;ylZ{&dJ$nw zGcO=7g<PgY){@VJgHV*99a(y%WW@a{2r%$H8Tlngt#XqMKBiba(GL3xT$&2)gIIh$ zBh#qRQ9ykny^IWPhO}ZDWB6PQ?n6zHG+T;i@{%)vF-yz2-dv}y?G)CsWTEkx_5mGT zVWz00IJ8>pQtEaQe41*t)E?;gW?lbYZz236KWCzaniH}^3?OGVz}aVz%Ti<i2z6Qf zx?eYh%g=6ZTog0eD{B%$mNa?61JhBiZj+q}9Lm3$sPmhbvTG~hx{GYvp&-XJVWyu@ z6JE&ouivxEj#-YMiZ&TT-DF&kS;;1{8zDRJD>`vCDGQ8tdB(&xsJFLIme~qa(Tnhl z&(67#nS)-Oe4}Vu;S}YF7m)E1(~?u@!68z|_I{&}k&{P9%qqIRP(5X#e0RfNPxS&C zao2XgFLPgB%<LgAqiH3+QOxZ@_#c9R7JMIDDSh6N!CTm<9!L0Jy~D3Bn#K?<hWt^A z4?2Uy3R=I0g4nd--qgZnefWHf#KShU%>Ig$2?Gs&YRj>4tz^d0vq&E<)rkJZO!0Mh z!z#@_Su;s*4l=g`g-Pc)O6s)|+2_FqI|vbnwrg{*kr6BCuu@Xv!J}ok2lYn>@hc~h zUY!bb&c#n-wb9WwWWC$7aozGCg>q+bjFOmNnfGwB1$HCD$vzxU?SA~}PFthR$v#A5 zT<L=*37Ek9fwz$aC|^OJ&@`F<+K=~*RF+KN_*94F(;Ezj^0jWFT9i=g%>C;qyvHoK z(3PPGJZ4og>9;n9oW>Pi${qzmdIT3sBgh<dLlO)Dt<!F+9NDag!S9m*Q8GaPSy2Z} z-QLYm2DF)%%&&ByxmUrwXmwn0CQ7zG{5KqBHdeC~dsyUL`}%7WT-z!$+!cKPtqQtv zNTSWosikeVI$Jom4MW811{)A(ee<@%%y^4F5zMyjND%{3EQ?Qb-inbOKj6oX9>J<u zO7``*B@S02zC+C?xNw3J`cYoy#yUH($b^pK%zP_2WR*=84qy74K>9sm2Pjx$1B-SZ z)wn*>oNoRCvgm~X)7djxmIZ9<8as59!wjoEc1zq0R{d{@(|QRsOBP=!U+Uj;CZN;~ zrK=;|9F4P8$sXvyjnuQ9lk(yds}(`b+m6v+8#Tr<v<Rqo|CI7R6YX@giC}1Q-)(&9 zjfN^MQuDA$2W1q;Ek&7pVk2^sUQK3GL>9w7*}Cf>D<9O!NyLtVyw+Eq*RXQVlv?eH zx<B0On8)&!gFMNW2!y&}-uQ}>v3pk{TkEsxog4lwCiDqg%3CP>rtZD+pK}tUw<n2| zbK9V9XFBYM+e0eXB$NtZHwU{4anID6KHs{~=0EkSce2pkxN+$_p+;<n25^UcXIsMt zmqe@Im$;)^baqXut~}~yI`DiW<%q~2MHdcq=X)<kQz}cx+kiE?ch1(wjE97ly;9n7 z5K^DnP7wAX`O6N7ZZD5<KcD^BWrFw>1u$V6+$xW52YhWEi-hJbukp{JDy8qcNvOoX zo~=w)KU~MNav1De9-G|exdD`2n^jp)s!x`vZ)Gjjh%M}L5llapPVOyF6?YeDUn*kz z!Z+Iyof~vIdcv8TdNE4O@q~KU^R*uoS~>3VoL<hFaGAQdp+^%Ia;EXf_RWmk^vXqv z2P|+(mpXJv5M3Z^$18>RncH(aCBbH$m4Ga(AMs`~2+qhbI|E?&gkGOVTU=^XyMJ%# zsWUm1m2G#d%R&`T(Ih%X{6(pUe`36@ik*J?s2%H<+@Z%6)jO8XFGG&cUogeg*RxrG zmKi=;(>hb>f%i_@g_S0KF>tPE)s-O?MYTVnbo`3}Pg$rWPFMDj!@ZNz0EryM2z$k% zyHE{=Q9{%YP?JjD50v#+Qd@U7BRzJky&3t@>QhFF8s~fZ_l>Ca>j0v&2wRJfdWiwW zlsH=SBJozgGzDxEHqwt531*qsYTt2wltf|jIl=JZpH*|nBZHdS?PCJs-qh$#VcZxI z??C)RN^*w+T`F1Y@vv`ggZ=GSG#FEmp8d)zwhfv2V0=G-S`wF&c1^NWhLy!*(c=vf zKlzGMfWIHuBG7+G!*OjJc5L228Hvw(FA7OBQ`dMWDj?Q~0@Zi-b*`-_!?^@jm^}tY zPAvVW{>UBYO&K0k6b_)ctoT%+iH-(mq+vq1jYxC4t-pAOcW6tu*&6`LC<~*d*JYh5 z>gU1#NuMi}u2jHdSBIxV#r3v(k_IIjH#oX18GXEnm@3-xPO!~!OM2IMVI+TiSiiB& zI|LYl0b<lhQ2Jh8O6zzO_l><v$}>fR18F?;ts=~O@?SNr2Y;Iq%_tXG(n0gdBp32e zJR?DK0x7%+Z}-#d)B9K`WYnCQnOUR1jAO93n>F3nhKsPazu*NX{6uq~jqDs`6G-c} zx0vT7>|yxMYW`@2#+Ld|-WG%WDMe_7?ErN2b+qx)$gk3NExSOIEmGiW(T%FRaleLy zIvddtuZ}N^M0H9S)|29&XAL@BY$j4CWSRSH_=?Q>hW|JSS_pFCO!)e^O`U8|#>0s* zDBak^7(w{u@Xdn#%-u16V<b)g&Pr|~9*fZMV@yS}vTkD;zQ@W^)zno#gm@i*iCEFw z$yLzvR_XM`+P_KG#t?XSbC`dG2yC}M?tAqY(!TY(R;`$X?O0()2qjtOuLwd}Oh*{q zy|;{G#v$XghJ$XW;N33@N^~XsdiGQHr*$|j7d6s94C*h@%sh+Fk{eBEgf{Eiu&_rx zL4R+>Qc|lQk~;a=?NZY10GarYwaBPJ7Y+}he+16I1QgyVK93h$BC#ybs!e|jToPcw z3Ja6>kzRik>JUGPYXqpv{;W~^|NXUi6(P0jc&#NbBR`0jrM>iNc3@=65f*i$MR0nw zbK<dcovC}Y!6ikwn}k-sc$n-1_TE!MC&+n6``>;0T$&ET_~jL@Hhp-{HyT;?#s?Su zst4)5-oB)`q(lbC{_Kp^c`K9UORj8&eIr@PqG#-=s9bg2CY5-uQoNRx_zfbd6j(MC zubHy{pq6$~k~2{sb9R%H%~5RxqaVh7Go=ld(g0_!tJa-3$l3eR4z-;%nKROX8=&tN z*7V^H|A(!&e2Vi8+BSa)!GZ+{?h@SH3GM_7&H%yP-GT>q9W;1w8{7%*J~Ozx4FkcK zXSa6ieX8~!xWC<fb#))-(H?7p8Me>pFCONBHZra!Ck+Dc6%IPt;EmksB;dKRKUKB0 zBXx~-BY?EV#`Io(J-x}%hcU8iVxl6yPH@FojLrqEQG4QebqV7;FLNxBsX>NhtvgpQ zU2S*?)OPg#KFhFd$-=CYdQx?xMp&bEX%GDYO(uD9wY@4<tr&a^{k)s7>Edw`-uaxK z9s^X;^9@PUOy;8PjcstXy3o)D@tJ!t0=zxg)Q1d}3=bXyvJ4E<_p~}`k8*VOT7B8b zT8(A%M)%U?QhE6n98V078Qi88VfPkc;=1I&m4(ETX;KwnG0Y9k&1@CwM>7{AS6fs! zcl!7l{N@OI7KVQ)(?cf0xp4QWkmwOhE*{>3GR4%>E)YTS^Z}aPo_iTR<b<?ffj=k3 zJ`y-%>8S=d;O%;!Fr5}AKZ?R9)DvH3ALA)ZtX;!&b2Gc|c8Qli9jybH-CRm!r;_*5 z-jT@?8HvgQ+QJ?exQ-uI)z;BO&&LGEV7AYOiH+ty0myK9CjZz*F$iX4q}<3d%)6WS zW>v5|uDdz$?quYVwdqK6BvLr^EM3~?osXEQFl~~(Lfe<*N%ecFz<htPE}5>&NLrgY z>-9hRX$$jy?io$~9db!|%4*-b)_~;z@34y5iVKDF|GFmHz6(}4CNyJ+NX`aH7RQtB zx@tta$O6Tm|5jH#AB#co0tE6Xtl6KXH~sglP(H`VI9*<dzrw`3<lJtu0Tqc?pITOo zLAtjPegutvhmzGxBqyv^Viz#Xb}xFF{3@&^@9n&b{_LT08yH9)RjKLJl6W^a)2Q4g zxc2<5-I+@;fSev`!};ez@8U<XhOtk_tbY=tU(pZ9t)O*gmNH~oj_Bfsi9AaGlO<%l z`1x}F5wNNUWkxc!L}!QdcLfpOoqDym{~@v*z4?8X7K6xM`7~5KtUU9NOHh=U9;ytl zb`Z?u;5N<b6fcl(Tx`jjc|iSUzfL^tF(U=^;xfBFDmWQ-&#gj7{c`T}rN^Goa{l&O z0{n@!r^&w+rK=ZwRAY=;(g+YcnNmWqi0gLU$vwAv=0e-k;XAQ_?NND|g7^Aa<IAUu zEHW-?)JVVp>-_W(j(MW6ELQZi40Tbj`%CizbsP`wic>hHgKkZH1}kHWb?c6K%$E^S zfMM==h{KXyeAPVChqmuv;mdObV>(v9+{~_s`L-?B;(%nVo7s}GXBEFV2Dv7)CXAx# z1|MkNEbLjf6QAD{^sTFEy!Hs)UF0MUX-#I_f5Ty`_)HozFy+Jm%4}got^(vhE0Nty z5LHMf`0Ys7B84-Wo@v0ddUfwkDcSS7hdWiKFuuMajvd4nQIkWmoN+H`d~sAAwg(ke zfbE9(w4VWXALM&h7Ar!}T86VJb>bR~rxO}6M6^ePWNVn`%d~|{Y-*Jcbz1C73ZBj8 zc#JM>&(yV<==TTyUl%}9ep2WKI*yvbRl&1vt;NFRu$eCT>@PPAQy9scPRX!HL=@7> zheF%hZTm0oWNXNOGlD({Jy;P*Hc^;|#gFi*e32N?sK<@C**c_K3%<0CD_yQqIS=C} z;jNtw%YWpUD(H>7<uv-ecnx{S<UQ5BZl|q0%&i$s><0lzz_hw?2+M$JAuf?b8&k{+ zL-hNA?zx)_c&DNSh7?gPwhMmurBz-tGe<-r3vn8GBbDnOuPtva*wOcq;|7D|-tc-t zbpyg%iIw&>iLGi1J++N{u?5`wfNGpw3K&_p(7VLdSo_PYF!qU{=0uu!VDS@>`S1sX z+1SvBpI@|gx#P+=&<C5}UJs;!9@k!z^j3fP#+M4x+49hdQgW36Q1<ED>fo_vM5Or| zqx!(M;6zzepU)NmpWc3-c^2SfGMn$OvCJZyxI>~1!gB~VSdnF#ejJTK8+Y`r>~MjH zq)(@u_{Yb{BKCx_j)D>7b(akop-B?krbUm7N8K4q@6t$iD*#wR5u=|FtCt7(>K@Es zN)R-m5&l3D-#Tv3os(b1U1Pdmdajs;P;%hN-)lN?Aad0A!A(kkoTkLDqSLIQG<m8~ z*OJ?w@?79#S0!2l5ur}s-4B_i2_nJg$|5@CD$E<_%K!O3nS4lIM8l#3g#2`9gCjQ; zQL{iGo}}}&jt>sj!JPNIO<TEnVY2%L)5KRXWj(5!3dHH5nB+o1;jWW*#_-O{TN(2q zsTD?^k7W^@?x&6(P0y;k`7^U^c$r{<ZX5p2HUA;ch1sZ4f;&!bfgKAIEfh@mv6xKO zlwov|JDuLsE&sq}H(cu))$z==ZE^P03HzoP*IBOwFE6QAUjpYCJ<Fk5Z0^Z^XZFr( zHp>j#m2e|MSCY~lo;mqtJHI!{XJs#g2+3mwFAh4f#(VH;6sN0s>C`z_E}RpQtx6bR zKXYDcI!{Bw?|%z9(_M8W?igB23@!zG&Sh;3)p+s^bQ1dgiy=6Jh*>h)wiAS8#oezp zv|M!m7&(<)F)^P%R4JFcf4?3INRyJe{uo>X*8BB`{aE(30sQP-YIf0oSTglAc4A)> zoYiK!P;>crpvEO7JjdctzWPHhh0WnbAEn{d7+$JXqdM(3WGUHWdj`rW%R(p7S~A@B zgEPCV%7}<^#DJj({Fotj<5lmtkM3jn9>A&HuIIU$jJ{s^Y6qIa`vPzsT(Vs^_=<d) zuyeMjk9M9*-fEWknK4yrjnfs>^}LxS?#GdPr)Ux8)Ed=dU?=<I16Z}+D!WW;*=&TS z21=;Kw4($Z5<tdoQBb8%ry4?l5>D%=c6~&uEIlDbFSX?O(W{13ls+Z$tLuWq&1|pm zLKV4#j%*-ejHI>-sZRF=CY)67rycKozXTf_%0$Y|BRy2h;%7k?kxwP~zn{}yu8C9{ z3*hG-1Ch$J->VB;I2P6S8VEi%Erz!rhft#2W!LOsap@E)L6kU1D(E>NAfGWWvWK@K z*ssI)9c+De2;~@k(r+YkPu9<!g`%Lw>G&%xQqVUN{d48oi=@Vh0)U~HC}H{{MJGj6 z);QX1!+86=knpzip}U3{h6h?O-56SpXnsDSb${o7dQb%j6{>~O5Nn&g6hqK4lBL(d zbGuB?GvE19cm76GSI9%<(yiz)W&dbkUvc;nS;*}zTlZ#csb8X#8oLd!JQ5W%d^0A= ze`Z~_FPu+>Gv_&N$`^5j@@UPQjQ>kM(yVOWpRb^%Bqf;f5VTg>!p&vU_wRFq4$~iU zJ5nyMoNpJxDr8BFXP+htrjY|OSs<v{BY}SXPCJ$`5Jj;V9pi?fVD2P-(v+yOKI7va zFWvlML7HzdGScAKU;hF*8;2k%xhB#Ca)n1d(VKU7Xl$a`+ND1mddPgkTeJBD`@wQu z>Nh7C!5x}CD)>Smt79ufM?3W(Br=sQRo!kC_<<)TX#yqt*I}jRI?C?;Weo{q0%hen z<3-7UnFh-1@WbyrA3Pj{l-bYthmYs$$g)`vH)$W3OE{bkl;SAg>yv*r^7;8x_Dobg zU;TnO)j4{$K4a9O9yOXidNFK{wxcg?fPZy69*MqdAGtm_WjS}QqRT$`KCkbM@BG8< zSS|W{mzTJV0Xg9D!zo=^J<nqM`u^+>8}ZFj<7?#ml~Lgmfb(Ae%`Fh_T>7w|fw6NP z5)<<1kUm>WZgFtPYoGtoShpqD>{$l<<3ax@d07%Vw#%HF54P_lM<Xl%tja|Qvj%tA z%%A&N?BW>^RFJuP%Y@n3{P`j9T9)2*OgL;m#(%PG^zyiiF<xSg_}*glxXiEITdB;! z9l!}3;MJYUoSW9pP1;mM8|$r$QpCV0Uc0V%gFm@5MJ#m%OVkm9neY<8pAPTCTdpT} zW7Zvu>v4?edOZY`5s>ioW+#3w%9eOLf~o?uNlRUZKIaiUF2%KvUqr^`0bn06s6LZ- zUolu#T^8+Io*j4C6nAjn9KubkREz~V8v`A@i}E(|bmzka6+z|3>616{`z!DHkF$c# z#o8v38sp^z`dRslYH=EgG26-bV{B&Drj6_dq~-^o3USlC;#7BGAsf0Fa|1-b6Nk~t zKS!6Ru65{AfIt464uW+94jZGF`UxHjJ2Fj)8iSi%uSoPcr7~kM`+2y#zRQLIF?I*w zKeqtJQ2}q0x{g^NpAKU?X%a?*i6ZbZ@QOIC%?3r4XaZQ>{g>Kf9BrZo>)K_A`TPe5 z<!>}T_S??n`4;S+$64?rN*AhB9p6`bFwZ|dVxjRQSzmGt;<AJgp|SLDjzbdCeD39M zIN;%G+m!A)35qrz<Bd`&p#-i*wsr*1ZF3DvW&{-V#=m_kxa68O%%TzR@2e1XHJRLm zW5jIXDc0xQCMnZU-**U6djVw93lhLOgndIu7ErsU&#%x<d?J(O#eV83$`7Ll)L7A& zHZG(a*0lDwz;i*MC>}}=vX2@&u{6>da*u#_&H7MP*pIe8SJ^XdOp}(CsMT`I7ZtEA zx~?n-<>Cd-$ENN{`BZ+kLf;uGZn)!v_s6}4$M1JZR-EQ)&GcOkP+07C-nEnOm8VnO z6Eo3pKL+{+T-;dKwH<deF<O%lGy<Vz7C>@NGP&HOMIk&S4EOz*nX7(Bup7ni4?>$6 zob@H_Pv3@%?vjsz+?IC4r)Hn}(f6^CZBOPa?s*dD#EI506_^aT92H<k1@_{8)kE5> zZoJhqY;cUP7=-?7qP*Q%oG^J|+R*Fk|3L|}c|d2MbOMP6WSZiB2h!y42MlP<tk<lF z<i72b%F3EYT*DANR#W|Lf1E>5QJ6K~z`l{7!mAn9pkj}bxUx8s2O9>MLl;w?Wn7;b z-~CPZyp|bu#Yqyf97t+rI1Z$8io>fTAzBf;@UK)iIo-*Y1$E3)#p_v0d=#@G()bio z`+1JzX}MEVIRs_?4=HmIF1(R=@2d_PAxF71(jxa%LB+EVeL_;jrz7TbOp>vg_o$Fn z=UR{PZ)B+BE&11e{Yv!Rc~;Mzn{#b*Aa<Lz{Tl$iL*^5PtYd-pyb6Y)Z)>e$Gs^~h z{!b@ezOVqLUDlg4NpS~=m0<^7gph&QN)&?7p?*cKFYm-Z_XnKUcDyZB`#Xqt>q}8u z^?L-P6$&>cw!g<NgUkl2G6B08K7Tb&)NhQ%C+&%n5-<v6b5(@D>9Z@t4)~^1cIOXq z?D^tH4w587%Y!$F(id~L_FJ0gI<)y??NN`zi_iGsPM`;*j<M8Eju9_Iioa~NrAmQq z*=+ue^;l*+r-*W|2Z3t%C9ti6oc35QhTrOoznW}R0%LCQVv@!f6teYSQ_{M$kBQQp zmKueh@7@}yom{KW5!U~m{Z{w&ogFIsxiz~04jwU_6`%jw{^#$xOn#-hgO(gU$Jg+6 z*l8Jx;e8{~Jm8f3gS)aaB7v4AgkuiVN~oNcZ7dKmOmA}5edsdfmC=%zrjxLq%@#2g z6A3$F@J2idlUes*d+hb}bAGI>5FY`a0V8sR;sL3~%22v7)wdiM+)kc7b5KOL3*vd= zrx9FGaIm5vCjb7;7)xL9zaOAYXZ1JXp<Q77aY1BF;MoKVGnEXq_hx<p?|jdFu-C)Q zbX(gNa_KG`J_-9<*xD}ptvl@P(&f2+VXRnHy-h0B<li3eZRC$wHEc{EiF8J+!&(T2 zB#pAa?A~Ew*+u}v0GHSrlHZ<iP7k%lwC)DA*v7*Guh?98t7`r^<YB<$kJDwveVVaq ziI?4a8o<~ntH(ue==iXA!jPtLk*us1M0zExWVt!%d6-jUc}o)~*%`E07gt!wH9C!~ z-6I$7XD5>R?j0sWhTPCBKSP^$Y6@Zcv)A^x$v;=#0a<_=yBx`&^FDg>tq+8~C2_+j za>09#WR#;7I}I9$Prw;c5S+V+<Xv@&X;q16kyDxfbQ=Z)*Z6!FzB;dy<>F`9+<;?b zn!yHII}^3|FzDX3<R?$coYDuuD^ZOg(pUyVg#f<Z-|$t8beCTH3Zv>4INH03%p2Gu zUb+fVKrFjo$|723up_!n)`-0_2I?8ETM#A2TjSG$Ky$@aC`c^p?6tr}c)Jz1ao%9j zIBrnyk!hOA_+Zf@_6?bA+1_4|nZFK~e^qx!s)<ecj}z}a0JhgZ_=z6K%|&?cltS>H zcq{uX^enNsp%70f7y4}s9~(xm>hLVq`t++>uHQn9ZA@>#t*)v$_gG>F==4<*azMjf z=N^UUxB3s4al5e{QmUfMgAg--O_cwwy5{$M5G1~9e`yK}nbE(k!BmgrxFBWHhqH+9 z9--J~&J)E_L^qWA7<73_gy^xyI^H7RLG|!Ag3Z7JYa^(KrRC01&C-kvw$i0WElkMw zhwKn*)mD$9TBK0IQ-Q26S+rZc+tBe+`a*v|&-(J#=s0BsfO&Pz=+1k_vcO8<6)^BX z#=22@@Ah~A>25kxm~_+0L@BW_ZF1N)N6R3UW)Le{)1;v5#^;FGh$YFn`Vx!{Sx;!$ z>+g;vu6!0aNw#2n=zgt<vkDlnw^~Tqd5|@Oc3GcfUW;;DB2|b>NDMf;hQoV*d1g`Y z4)-pRiEo^%0N7i92EqA##5_bAGFc83MW3%bOpZ@QcoxW~>I}P`Yo0h}v^~6%m(Zdh zPK`_axN%25REmLbMWU1*i4v;5e7g$fYRYNl07(Ynd_)p~jd)pX5IUq+$D}rc;$?sg z3>`@05xX|Fv|0MMLSQS+Zo!T7Yp)#Bdy0{Qggg1Y4@i_g&o98Vt6~28!zmZ5yJAM+ zHW|GUGWoVfR$zY5z?5zxWGkKJhu1BpXDmn#K&%tlAhMCxwMTG$6*<!KrqMc;4$?m# z0ZvXkr>DDOIO1!*2tVC_dB1}3j80yBhh-7qB6h>n%mk0Yt?<;?KA&Q(8DCs%2O~Z| zaahff#1RT^82rsUm^G6C9qVvqJR0yV1AxfZnIpDL$8;hWcaK{#H|PzyJNg#mGUf*T z@dtms@)5H+ohM9@mm*Trr+0nDzMP!IAWY|vSlk)92Gyc27^`MSHV{UVNdNW=qz$SK z81#_HmOGIULd1MZffeO+&Q53vBn`xMnEjaJ?6TgIXGKMqrVBC)(&bd#`2aLX_I*^u z*P#BNnf)W&PlQnsX6eLGGj8y`Efbok^A<Y8tC?v_>Z*_4eNXswrgWfea}IPJ5jE1) z&p7KBzF&9o>fg{hPtj70nQP0-NQ1-QJ!7tko;TKz?%50Ke<kX~9+l#3v#N69><QZ5 zZZB#Rqlxr$$pwG~lT+^7i}CHFc=*NH(MAp!F%ZU@exBG2xBiy>uBAj(op7T&Ryew$ z_iK-FGGc9<JAh?F+&?+4QLv3q$GfZuQUPCVyGa@jVX5>mE1|W@SWX_R#yV>8v9Wx; z+50z7?9V(hH1<BQWA0*m<0Rg_9qnC9-zM7$a%)=J;+Ft=U-vze$Iv!)mSx(OpT0g0 zn%x0Uv{ob@oD!QhJy0%3uD`f~>*K*hH;xDvuaBZO4q5{O%qD6}`9HD7yp;g?;c}g- zy~6uzhp-w(imTBUo@r0%?~hN`QUE7M#ja*dBP_{YrGUQu`E4BBN3}?DjJz)+X!GWp z=L)Uh0w*eHK~+mP=#)E#5w_ny_c5?z2F;DA6m@F<;Jb19<$u0t$i2e%nLl~dyZ)1@ z4qT?}9xuzq$l^wgQ2OvRx%x_XP~FXfbmZIP5zynoVrSg+G8(mg41m7jkOo{mzd#OJ zYdplhhWX>~)<@HICWUWHpLI?|QyLpOG@yFv=ip%bz$d!=y0^@M)nB2Ixt1n~q@MSj z2%iI`=0VST>bJ%81>|Gb%8YJGZ=LUf|1wLR0#aS*)cPo^P7BxK74`XE0$DG_7cGk7 zo1(mhMA^gQ%UYyLM@xUxZtf%dkbor_gKp6u{#DQdH#Wv4o>NF|Nf3o#=Op@kyp`P9 z8EloU!8-$Gtb%A?e`A@a8OBcwGj#MCVwGo>QdxouMU41bZu0L7@_<sJ-#^_yr~^bE z8IIg#@e6)2?w+I!rB@S#wmjihf<nDq;@mH`>5XL9lA=Y8!4I@qSQ|6=*T%PdWBcg{ zU3mtEd}m!NG5bG2_cvnmJT}4~su97#h_}#3uS51~CCq>N!+xb*`ff*_<JXgaafcMZ zx&V~AscGR7y;KifgYG~;idBA+JyG%VY6wep)%ezu;-;AI_EYRS_ksS@CaoqUWs!qn zRbg~ULT|liHPue@+jJ4R?Doz`HA|N9vz3bv;w$;^7-Clf{dI*4bqqu0eqqXq1lS>e z{nnTSMDhBT7;p$6JLqP&iWI)y7yWpNI=uQrRq;s0=k@%nST#jRCa7<HEAEXv)y;f9 z|6<Ea%0L2|adbuGcI^u4Vak1xol!j|S+WuA5=y2#I@li&5@?Ra&l@>t?%Cr$rOKKJ znP4q8|Kde$fZ(GOx;XHS&ky^)C8#<JP~6n=4oveF!X({v%gO*66^ouDbfhseGZ(5c z1kH>K|7tqOA>eknQ>`U1tm}F5i}636v#~Pcl?MtIPfnGE`1?E}o~0o)>FiS_r*G@& z0p{F9_bq(Lo@0I*%=t<*quskJK053_M5v|gBH}hnzZT0epaa}xNK^^b<D=}yd=Aw1 zg}-a22Z~AxoBOC^_WUN(Hn^2)aL^y!r9JH~HplZ$@F#$vwyQwr(p+}N0S8z1<NOCF z0WV^X16=au@5)<T>4xRxu3xw{#9aOTA2o_SzX(^xG;C{#H&a)RD=p!DG<SOb+D`WQ z2SZrGiPewOuMeNU{GiCqNh_c>NZwnt(9#pJR4_?Zw=`v>HweT5*O`pyeMJuZtqxY; zr7uSdEt)?n$W8uOgPU7Wpw1p>Z9^=)ukyv4sKbiA$z%hfUPx22fZ~;Kt9^$qyT|`~ zEx~o?WA5ne)nQ&A<>S7np(#hHmjIu54q0z`tm-@87T0ihk~3hDDTaWXTZ)n3@_b-q zGV67}q%9Q>(}2K2_bR6iu?tR)>>j5t&(z+B`e7NHK<qR_dK4y+tbyLarTB^wdT)~U zAa^S=cSqw9zb0!=r~>BVSB~S%`tCN$CfE2<U`PIwPMkq#4FPRSGj@-l!ss>|y)j}` z6*&@t*~aK+ZF9zS{tR=xnuFQV=pNBX*<F0%%}d@Fld28z=YoyoqS}XB0-yWkj2R%7 z-SzfB_f6+n!8=m1(~){6g+hFH`w*-081eEKEso5_Jg%0?GasMZ?P^kEPjidKbs*4q zu+fR|)#&P3F2Dhu+}hBq=6R@@SYig^XPm|+qz(#NZHY*CSI~&Z5E?$VZ}eph&=R`^ zIS`h<DLI(K_zrZ~j^gmlw6wR<b^aq#nXw%dXsm0B_PZ!qJTvQs+?{ns`w{8gATv0d zCfN8;724NLeA)J72Is^=$ls@B9Vgdb59l31BC=_fT5sH3RsWG;(jN<i^8Hrw@(Z%i zoBxF1ADq1Mc&E4QH5IR1U_6r^x67jE2iK@Ojkib6m1PFLK!fwB9!EyUet6G|cT@%T z>H#3%d)B4e>SuiD>?ls=z<Tw(q&Pl(t<$5?M+HBg(9b`%-XS9BE|c(yPq4B`0%eXf zjGC7BCrReocpQE`%bbL+E0;Q<aF^7~bCBP=va`Ga>C1nb61_5bs#Eq6;U9;a{%5ZI zVNV9fadhkxv);dp+9f^MxFEVwPX?B-2YR5>=zX8CHz1L<C+9;GWnKTrvFb|=lWNXy z8QL1bBnPV)#SvKjjarW1h!4$m5-MGzxKv5Fjm3gEg8rEsFKHGUxZz{CZ6>vawWA9L zlOK!1*GRiVHz7Tu-e()>S-bNF$3V14op~)ATAN6g15esG-y7WH>ebs2(k>~P9C0F! zg+E%O<NO|79J+ExDzCbt_DT*VOv8)Z3l#+$_e}ZXhZ|W<T{H#}0>deR&o`ut0Ft@z zUJ4b5M|Z>a%K6XNgHt$$Pm<wjP5x-o-1I$58MDD+yi8_9xc5zQozd?|$gEd<;e6!i z$0e(MHJw2)?6Y_Tuxf6U$^$lnaMt@{(BG9ukgWkGRJ?EgGo9iG0t9+SL2Kg?SGevU zD{K0!;#X>l?(Nz@Y5*_)X5hnKc@xE>cIJ&U7aQDa{#-?$eb78tfrL>(4329}nX+>0 ztW52Py@MY>?weQ+_|^UApu0zkk&`AsDUSdNk)%6M&Oq|<BkK*CHuC3As1P?hi**3P zV*$wR*y@05qZ;^ly{A0I-(G5@Y4A30{kJ>|<IH@NVwqT@wITKvu0Qptu~hFC>>=O2 zQzuFHSg3?{4Yf}Ewm&1w>TxXlasU_gc_hbUi8uTFQ@FBnYb0y*sawOaWyU3dk)jz- zY+LQGOA(6Y4gDVpnpM~mD+(npd+&Y5q-fg&S=32JIabCnva5I3XOeebvKIEVlyU~+ z&WlN_jReVd>(Z7{M&nmYZd!Fj6+E}nGd{j|`{6fs4u@YAyW+`AOZW^^0F_ES*#?=5 zjfXvpqT;eJqv8J|6O<<O{hHsb91Dk|OV@Sz15PU4W@t5|FCWh6PQ0rk*=lHVJ&qf0 zB(0dNikE7jMC+LtE63~~xEU9BFs~QUaw~FzB|whCrEB=agqf!4(ZBQ}g9FaE{h`&S z_zQoMDm=iF$lN?LA3}a;sbR?aZo(Q0!D_)BsPsiUrJQ7J`=ME5;AF$^EwPy7wPI3& zGpS1MnQOB`>&Y1x-SYKwi8)$(@DBclg<%$-L90wEp?BU_kS@~uW*pK{Ju5VB#V3AF z+>`l83N>1&OSC}7$vBPAi|zbszg3V1Z#%h%3lz4+F?4TI5<|O-5}5+8rtup|nfcVG zZycC+GG<YUWoFHE<1qAKDSAGF-~aNPAoJPXt&Nj9R}xz`chZznJ+8=4jw4mpUBYUL z%w*|$u1tzB>xuuJWb>Ye!E(}0J2aS;X^(_s0C|ZL6@y5FJVOlo{7^Y;%DMFQWP~_A z4UPc@)E`XU*(u*27a@=e*UZlPzl`xjB-R(}M5^=dZ{89V!HAyvGX6tu#!V#<K+I?p zoXGQmX?O1`&8^I44Gs+RTABn6ovp={U}M!zb{+m#ctUx*cFNl--}k9}_zQ7N-Fmj0 zFfJV#P^LRk!z<r(_it=is!*tE;J~+cN5vWYy_+R2OZ}Svs_%rz25LWXCR3?p?wQC& zaIy+aC{z4OjQ$5o<Vr9gY`R5AQ&mD>9TIkYTX{)>pTMB`)=B!b$pn+K?r_(QU6#Is zkfettFAg~UpXsTqoLM`saAn*a+eq>XV)m{+-4vgd72fVsvg%qC{^Q2{gq{)|3o}0` zf4Pu)&7}QUpI+Zv`DO3_>BZG!ah=AfC6w9Mc`tMOjktl_fk!Y&9W9s4dI}OLhQ$Aj zJJA~=g!yy$s|b={A$+9!B{+*;?V8<~eG8~WG{Lf1rj#TSdhLqZB4+j#QL*q_l7;^M zazJn)fdbsY<?jdEIk2TwSjBr3%W~U>EE-e5PblqKqItuYy>vr9k__Y8#wu~N1_1L| zWb>Uy_4FubgDt4jVVq;<KL-a*(PB>PE>>pPE$f{V0esjHg|KD#fbmY%W2ddtNC5Am z{Mb?9^;+%47Y=w6q!OlU_67KHt$#~t+Q2Gyv|K|%jl11XVhkF%1;J?C-w^-GV0o$; z@b3KMw|RqRDST~ntn|cYtVj)+wi>~n)0Q@jJA-tmnh8=C8hhg7o*8ftK>j0lcETTM z-`V)Ucb{;vJBtHrYw~Z02hPMtxwSyKfBh)N7)aez7vyCBboe|F#HrH^yxFgR3w8|= zxJ_GVRvO$a5qJHuBS#m?@z1UEM-$1CFdmxJ78h7W@wWCm?ub&*x-fkjJ{_ahhf>7H zTbN!EvR&5)p66`PDc2TEwsw*JvQb^|ob&P-79-w}g&WH}z4%_Cy207LDErm{G?oq^ zGzqDKDbMblg1s4BCo^PT)ej!#IBXZ${{OzSw#8QzD?MI{8V?aQ|IctHF!SI29^V`* zjIp$9^j;4+g-o;i)23zjH5#;gzU@mbL4<%^^k2s*nq1{CG>vmD&|Sm~oShp<<e}y? z*p}v^|8p_11*ZQRT!Fr|pew8t15^1FqGFa7vTO-*y>Ftka$4Gj+lh>&_Gdl)>Wu5I zPPv(^V{6d)(;Fk80s*-<VEu%P#=K0F`^WhDY#jXkfekeL`|7?H8!7(V&N}>LOt<Yx zCAdy3x+b9fWV97-`J2SDkaa(p{-v%iU~j?BOteR{F|j4!TL!+X9Z_M($E6ydf32_N zIEIU-*?-4ChJu{H{SPFc%=>vci|lO9&F23omf+Y+ZZ1`J+Aee&wUqayp894xv_!Bi zMGxq=SsGE<)47JMg)Am$%+~B=Fs4s<v>9GyzxWC|o(ZN^e*lj?A{y(sbN8$TuSFEK z|2lA_Y0Z}}1vCb$6!HXiSWeEI){F^tcDOu#=Ay#f3+SM`sPQQaA@f)Z(Ixhtw#!M4 z>j)T-72L^nokYuRgBL&h|0FZ_3JpJ`A)!M&3F;@~t6@63=+9jf`9f^mB9^mX>F%-t zjWq6fp)+%ggD~K?D7>!<&NCopdGLq$)cPDdk%zSAnh>}YI{^mR4u$+zK72v*X_Tog zwX=A%YyV@PrGfZ>G-U#uJ!vq?IxY>^O7mjX^zbru<_mo@mw4HCoO)u}u^WK;A_%j7 z>j<iM%d;CN`Mwah9_wJ+xQ9$CsP*X?^=Dto?#t7Lx+<=jl<BwNYY!?k%zvF8_v{ip ztQu+#X?}}~a!CcaohSfX&kaPGvL`R+#N_Y-Ko9u6zw9B{Rh$GO%Dy2Uh0jf+pX;M^ z&S=vwke81e?d-uS_O>QKe!B+hHAU?VE&lHWTJ)<({Oz1+rJ?&0i_@Ax+-aw5o<5Fu zb@$=N0OAv0)A@GKzlW&lp+Co+2awh{DqLKG<J`u!B3SSBS&~h=O4=jfW<h`?oK&i$ z0w>7I!k2|M;#*-tkCk@*h9Q>;_14=T_f1F5eJ(mM&N(_=x7pU=><3FDK_kuA{*K;w zFrU-?$*hogJxC}P*P8<Pjt{u9Ir6Y~9N(LzH%~mxlRz^O;$x*N+o4eCn5YTMCg2Tn z=;*tyJg8l(v}@w+is)hg$!ukxi3)p9ihLxf4k}(v9@#JV&ND#3i*q_WJxkcH=#d~D z+rqVPScSnM_8hxQ95R7uAnwcjvU(~1YVDZR;TqB`!0mb|0Ogp{9$!z;!J7bM7b+h> zvE24pdqU>Y2n|r%D8i&jD|>EcE;?~(zk@F~w>A0Of$C9JES7hfi5SEUyGr0|;Y-+o zP}{mU$2!4&J&{S{R!a-l7I8e^Y1l&qFNVsCjL-sGHek|Wzsb+G>*`e{nIO4ED*1mf z4jgdE$VBm!Yqty^sw~i>yc4T`?%{Z$qt)8Gc|_L6r)U=fkbeB@A}2iEY{jz?5!vG? zIwrS9Srg3FTrITiR7bp&G7qbGI64-rzH^sql`Y!(?k^fyKhfwH@RhHnCHgp$k;V5{ zXwL=ejo!)hWu}o>*DFt#kuWW2c}8trs_|~{qT?rY<r)KiiwY@|$VVX&v87Oeve%1+ z8w2c&s^A+kjdA|QXpl|NUZ@rdBp+K`?xyiQIB|-}03i_E9lbOVNb#v54XypW+hG}6 zEGK$?w-LRU65PPijdWBVC|6FOGM)Ukv`FA!3v~z{HI>e9cV#Yr48h3m8z?sZy3&as z5yhJ+cOz`HRt-$b<zwp{V*K}TuLaX4={|hugwT;K%3?f5CGnv^QraVvD#UFA*D$*} zu4uTr)+#aAgwC?3$`6tbNXqb_Iub%1%|(-tzE-}6{ktu{Nwp26$8Qn<-is&wj+WDw zgW;BqCzPFGmlrT!^26shy^;S%uUWAlbjCNgvji<?2F^^7z^_=q87Oat6f|<n5;6;} z+sG;aw5mK#x@3lTf0;sKuw!B<YwCpX+NAS)Vs@21Y~9RN)3bsDP6w8xg==w~2h6{! zymz}{Qv|7my-1QcK?4E)aWK!K(>va#3sac+{2pRYZe`vV#ra~3T0E@RsTzN@&d#C~ z7dw{m0(HGwibzwUD17jtBHsP)YG7A^*$J_%KK2<udIKeQsh6opSO>69q5_Gu$iJM& z!3u}yGl}$JS6#ta0cWVHdU=pn>gS;JmcE+Zz`ZS|L~CR|TivCD*&anGKFJnku$Y|L zXTqWfQzWEK9_8s)@u}&Vm7lV9Gp)^ziwwI=#{T_eO6AV{T2W52gl@dg)HCzJ-a`Xu zXu|X@kE{!jq_3wDjOThc{O<V2oxEdVhI1Wd!NyzXrJ!H}xE+53OtzOw^bqr&KD)eo zx`sOW2ch;S>k!=K%f{;h>aSmRuAR?5+8pxBtH^bn@zJ609B!-kTITwqAM3YO^H&E? zw>6WMd2gNPwQZEYQ}ELFx`Xq_$#&TFnjsb0tlF&j-aKJ+FXX0-inXov?7&6H*H3R) z?HNkr>e>4R$av+$kGRnS+_XYI8Y0C;Pf5%fYhBlbk{0VBk%41O_qu|`_RQ`-3%3?E z?it2g*sagT$!y<i!W=vHz1}BL_+D;|Z92U{+-!)UR`E{w9RW1TsSfot$ylWhbwqwn zhtzCly&qgq#uG0~sJh1U!>~fRH>UkN(J{z(q`^pW7WtgF%D;r_4^}v6+h;S=VP%22 zpFBt>byBXi)dI6PkonRvL<WDR8P^Dqm&|)!@(2Xm*RXJ%f5b!oB%NUH)7O<G4T3rY zS%7G~q%*&qAUk>A9?8lp6Zdr_tdM{~@ElH&_TUu^q<;f_q84Rj<s4Y5?0!z)*(pCx znyB^fG4o~@4hU0k$jwuMt}b~#BTNscV<9-8J?dLexCtFV#MGUPI(LRH%h4Q`&CTFR zo=xwI0tap8H=35hGgxzF^V;Y8I@K;ix<YbrpL4Sbs&>|8zL0t^ie$3>J(fGL;K5j+ z6wJizgS~Z<(hX}7K;wvSEK)O7&PYAR)s6lPY-dLK!}YE`wz?x&3zIer4i@gQ4scE? zV~oRH8}}j^+;2dq36mr==K_S&n~`T(s5}#pJFN8pQicZSVgr&1L@ldK5$=Uxx&L>K zaUAKEa>IYT%n;!AGQpzaG>a+WK!pC|Tn?R9mF_uLzZ;{;#fbROsnzHA!)G#djB!P> zPBu9tiU`O%Y%+E6fxsnea7ffw6qlLW20jGH3uXF+QzuJox<dFb%!;QL(cghK{==zG zm$y2PT-ptNmxIv1gQdMI=g==9x*r@yZZ1udU5fX#JK4TJv!0su&N%T5J&w-ukP(jb zxSq|nl2k#l9d9E!hL`N<s-HkrDXc##>xBr2j~5hH**jzW9D-;xPQ;wf-w4+HtY*K( zTAMR|g5z3HQjT~Ltlva<kQ2<B(P^)iSJt)WYAx;I$Wo-f`h;<e-k<>QcYr!^FYq^| zBSXPO*Zn5<COfC@Jh5dO-OdO{5<bOy=_mDV?w2QNRc9ToPuQ55m>hit(d3q)QU~l0 zq99e3#L}p7051AVl<WQC_2j(|f6os3`1C?zY*;i|EB%{`I$_N(GX`HQD85#dpJ7<x zamYVpU1-cy!W-rg>+&1Tyw<R0%<h?%EYeYua-(Ml0pYdxAx0;|fa`zKMRieMolT_B zu*PXFGg*V}Ub)wuLleOduk+noxQ5RU`r^!e85km#ij+^EKg%9qpY%`i)FIz0^N0MT z-b%UHYKD>PHf>P;J09ZpdHl}Ll3h2t?XVqnjY6?OwL{ZYy6e|rh$9x4y?xHt_s#w7 zazODoZE9rzt0ukdXNp5$Zf+v_zjWb${tkn}Q)?otad2GI?8)g*=lx;Qi+eoESqa}v z=>-NF-h1Bv_w4S|DeOPx)P1l#jY<F4M7mgZRgnp}!<l999vm|57o#ig=5(mJj7>dt zxl~)NbI%=;>50O1{nTZ#pP%G<E&Qir2R<9L?EkU)K=Wp<4VY|Y2*}KQty__8=qyl5 zv^Emd0~P!9C0wQEhcyV2b0<3Nd*J^lfBZX9Mwc4wYzBpt*A~#4jgK)l{Z3GME>n^$ zG=){Zz|~u9#>9s_HE;reKG}4RktP)dm}0xCDm7-7M%^}$lMR*!hZ?MMAkFI^aw5F! zE%EvB7Vxcwp_u$&f2xA@{9t+`QLG50(R$|i;C74tF4Pgn5c-1%(_ziN@wv7%{e;8@ zn@cYUS6;x3{tZ14wx&@TE6d%%@VQcTY{~O^&(v5=U|H<+$0jg0k%67|7rn?$#)@Pc zZ)s0~MHd^pHB|i%!n3hs&o{+++8^RF{9S-=VnIuY+1tZ3{47$sRVn^WEi}M<siOL* z_xPV`$xK?nGCc+TNP?9sIm$Yk_peYtl79j<*XyrVKZX@^xA9V?>8SKl=;VLuWuZlB z%xTjx>f>ru7qWhI(vNg)R^2e))V_8=Q}v74O{Z>t4&Bvg=Rz!orYq6~8xe%K2Ln() z$}37XcZRK{sxg+B^T!3045Gq<rm_SZ8!L@$be6^yUnUQVa<gC$ITwQ5mCut|2jwbx z-e$#!4G&Gj!>$Dw0j7Gc2Zss@v$T~(Ll%e+r$4lq^34aI<XgA9|1~*0Q3g*+e<g?5 zYhpQ=zhhT=vUi~gPQ#Y})dLs08eXA$wTIU>{%hun$%^^Fhwu*e6<|3({yXPQk<aBo z8?5s6=Y}4zAZ3gvDL|jDt~w>B#{#5koDNR*c&1P|GCK268yAcc#_8hwu2Rj?uWRG$ zpM{>~Z-=we+1~)AxW&Z@U}|KJo?NNRC*B<b(o-vmvNq0-D5w8G?C`vNll2$yyeHDL zc2iz@`B>Op+C-`>^FAD<m~LBsgxXaCvslubQ1(4DO`^i+M=<I_WQY$Mw7N=Q|ILC~ zhk7Y&cM|vE_Gsy3eolXIgR8^^5>l~r6;v!8Bzhd*da-DixvgO>6SMBrmuHf~6x)mc zDV;f%Ve)@@-~LZSZ7?2AosqUVH#;vC7cg3mw(jLaH0n5XdE@hv{Qpln=>N$G5%9nJ zI7;C4>J+{tZpF+K;_(03oy$MJcgxI{H>uH>C?vy^GhyQmmsZq%%Y**!Z`D<xzA2bX z&$}m0{i(%%2G^sStHr3b%HSKaoI885aRz()f51h(?>n;lYUL&1RnK<i#luB=rHQIX z-S)09{S$GtFDQRtP@a^=<vah(%v_Xn(ME83Un+>v>#srcM@I)KGYPBvad5hVioB7T zymHBArPF7qzR6Ph&Fz9Ab$sDF1$tTDuznO|yRO-D7uz2nq-b!}db&tLsKO*MF5Bw& zO)#&5sbZsk6?r#>4ahtZ#U#{GEn=bmvn9cfYU<60x*HWZhHe8jNW+{MntsXHO4<{s zf=|)P|5V6H1@<s)X!QrNaDEA3d~uOtwPyJ+$nym+<0VhhjMi`_<NSJc>_Sr$etf2R znBGmSNv?E4LwJPgBTx-sC6H5ywvg6+-*+3>RC~Am1C0?KemDLvO;{(?uxv|mc?16# zwe5a1jioaB0K|xQAzQ8^>;C=IGtF7s-Og(9xt?t~?&F0>c=sp|HT>SI`m^s|H;!@n ztOx5tx@WGnun;^gd}IVgGThOm1`&9<`J31Y)%|=eK{J?PPIK%1hF#k;LB^x?L8kvv z0k+hY>ixGLXP^l~N%ClA6DG|V&$fLur-o#2>R{A+wuyfRyRRSHHV7c*;n!jQuPTFO zn^-O5KW4*mM!g!Jrn%`hCa?1mi1!|gO6tF&DLsj?!e0|5(>Y@QmjhZ`x}d0NVC>cJ z>1vqOk+zQb!;oJ#3sL4Zf%9#h(2vpgGSdb{qLYr<rTdnZZM?RVzM50<`tOYJX=@fm z&xxiLEDq%3^(R&JKV(-~GS*H5*#u9QETCofO|QOm^XvM1-agRedQAQx%@iBE6y9tr z$f<P#knL0537Y%h^>Jg>EL&OZ*TzMNef*=V&xl@vCer)Tsh{6boT_M`Vja0;+FZ3u zRSxeen-^~k03`cXRdfTqRvT7(YDz<^p1(&^5i98_e%VPuj@=;CF?g+o`;>MhRlirq zjx{x}Z7))^n~ElFwH<7t_qiDB|A1%NOqcB^Co(b%O&W+QRqRnv9_bFg83~8ic7-5T zNo`gwe&NGAcIvp+V}{r!oNU`HjZG9UCUJL{h(mw4(y#a$_|laQN4LFZ`6)EIkxj+R z<@z}^^6F9l)H?_y%pGlG7Pu@n-Xm6j^*Z!(k6Io*wO+8bb6C1d>)&(S?{4!R0<I~k z`XS6F*Hm0tXTU^t9|&23cvkLc?t~cl*sMf19F$gjCU{N8m_;nRp4WF@zTqAcP4e~( zUd(H}VvVxqSpA=>Wm+}jcO`fSC&S02;qMt)GLX_L(y2_sHbJIxCb$xXdZ;-uGL2c* zmH6oI$p(D3Z9tg9D!s(2LdQd$up*XM<bq%M)%u@5>V~HHjMXrz+S^G2dBuRB{dpT^ zRv9`ohaXbC#MASViuJ5Ze;|vNL}Xt?xyzyR3;Z^=sa+_2kQ)1Tm$J24QaqO-d_!qx zKB4&gzQbo1TeQSF2$I#Qkmc~1CqGvEy}R^WeTkJ}{#5*(4K&u-+$ygH$z#L4HDtf@ zDf9ae^O|xiDE?uz-gc|e>*UmwZJ?*%ZP_HwQURLzoUNt<0TmTitwV}j=5(`?OvXrc ztXOlkVaN1t0gB0&Xu5mt3F$O%x@Gx5X}tq#XD<)kY86|<G2=$oU70Q{oiQdSRvf6A zjZ2W<3DcK2{^w&-dgQamB%e=Q>sB9B++=E0MM;dJ+X((s(dOZEj7u1<!sND;mgfgf z?nn+7GqQGk`%@2C@<?8|lf4MwFo5ifw(AZ-5#fn1<aVw-`E2D?)LB;MPYKf#y7hTK z137g6+2%J7nQ_|D2^LYD?gxtdepFQwbXA!g2_=Dx>N`&eVM!TGdbRosV}cacTt$6f zu^8cxO--*-B7E10|LgA@qf}GYElsdcH)^fM2uAeraMO=IkKqxjZC|1!O!E0Dz*>Vp z*b@rI-)&wQDEql2p*+f5i@1TmX!AXfcp+P-pz^6a?WpnIKoT&JKKCg%OHwHyY0j1y zyX9e4ZAU6?8aco7C6`c;vF8Ai1l#wF0wMaP$d^7EEfa|K7y~_I0l`KbNMm;OX0EFX z1TOQu9}L`2N2C=m;E^eX6-yWG>X(lMJ;ap&pR~D#8Xnt0yomA^0nuWk@0ss$&7@pc zV%Kkx$h2r`;d0^7qr#{v2b`ixjD_FYy5-f15H&LjwuhVLe}(u&YC*ibyxIBuN#ZMU z=3H_3I}iQz0VO!f38qsv#(Z5juYI_4&qajx9;YL+ieIBv70)!Cz?H65t4Z!OEVXx$ z{n+TgPST`hiL;Q|c{#)68GboHLExb6eJVCb{00N0id0_=;V%-5XGMHpY7OWY^6z}r ziSP$iq`agJBLplIvXWj}J@0@2{>%RHZIVfNNU>lwdmiZS6%&WyBYULF=;-*q3p>F9 zZi?<byt|sYLY~YQF)PgRbIJsSVOAkS_Hl2hM02$vj5grzt04@v0_I%Xgg#QNAwP1m zU$!Ukr8b&=p>1B99h;XSAbG#3)#d06UGP-$2Q()Pe`PlhH={`^{t@_|Q4vM|Qwygn zvQ0GgoHfk$m-{&0hh4+6FR5C#$!+z4lT4)NgS|gVh0SOQiDo)aT{RA|m@@>ZZiY04 z<oD3VDq*&#S#_<gSQ=FWNhp%O{vS%PkUqIQu&g)?MHzbWQzW@UI8x%r1B%~aU?lU2 zA+(`eXx4C^s<feK+?@r#0G2^&2ffr3V_6mN!cs;5yd<`AGR~=}*5?r`%bx?<dXDgy z1^!H8S2#%;z7PUWQ$Yq8-;L#UCs{|DS2m)2BaK^ws0|7XxoNtRPT?)pfYueqWM#c* zC*2nzKjfK<_!g7%=0sL?j9nGM^`Wp(O9-=Lg2=M1)kUIb*RI3%;MX$IJgxlW9u(>Z zozqhXOoVJ8*+#99D0g9P&>DLm$2Q}<9TB?B0-?4|$+ySh&hLAq%mn(_3TEe2xTw6D ze_ZD>Qx>ERYk-BTuguEkM34GJuNckE+b^RDGNDk`GFd}S#J$%<{rht_1OkeGwtEo! zuhsquN;cvAK`5hHv`G54x!x2{fpFt_B3cpS)RMsb^qU}gQzgRxeXfF;zL#j+Vg#nF zj~{%WRsMx$-j+<c{eCq|FE1;F&ZwYzI{J2mWqgS*OSHwiyELnL9-l0Yr~#&}0f=+d zXyi1AT)R$#e)Q9JPgx{5YZMx^kTuX*$W=I1$n?u3U$$u8yH8?R1g-Y7FgsWxc5o+Y zeYB!;Ju%^H;|2<BPSAm9lBZ7B`RsTEnYsU1+}aqVSPJG$ST<XM#;Q!dWui1Dvn>b6 zRH=kF@4X!wH3f`$W=<HA=GN~Y%r9AMO>U;Yz_gF2kaR2{LBacMS@*fkm77VIc&4hB zQ@)UMT2h_H0n*0^3()R1kx7Z`cx9V>{(eb&o#xi;zVpvkRvW%XQ4fQzS|2Dk>tgIb zP+mDNL#-#Wd$)Fr@xIk*)^3E|x!CgJuRLtldju|AK_LNa4?#AX&gE>|Orp2bP2+;B z`CgXGl@E}I-%FP!3+K#&PH{ps(w3noFY|nrf%c8<LZ^CNUv;JokW!{t{J-b94kXRT zTkfsPuI6RBemr$En$2v%TF|uvweP-|xS_GQau;QDR}J@wvSs9<p7C^VPPM&=aC`0r zAVGO$r@Bf__VEy>;`eU@+~hJ=I~F{s9F7P*#+3B~9;)BFAnxA8jFqiEG0^R|6P<BC zQ=7WJ`Ih9r$Hz|d4OhqeO7+LaNuH}OpgXi&lz76HA{ykN`>D8x4I3dGajMhkG2<cH zP`ub;Biq#q5pH!N3K>h5X>`aPg;Ckg`+?XJ`?mh#NynZj%}2VnixU@WixYlYk!70{ z|7morao(M}uvmph)@T;v4weh=P&{Vm1ZdUHq_FHM2&t;{XTVzdMS{J--lNvW{Gg}? z&_83$&JldIyJb1_rw$99SuF*8A~tjEGMx&F=z#1BcZ1`ngP`(Qv{}t3fZ=yU8|!}Z zC)-5hf^3PjfD9K?3Z=Ml`%yj;w>66aSc3hXI+znkaUq)D{J?Wz0jVFnt7vKJe~t9k zyv+PxZ;>Lsn*9)$k!k8wBI7un$rCbDz3W^yHr2}0zl*UKS3H8x+bJ@EiP;pbAZOa` zuf{djh0;6Z{J9EPcK+*jnF~iRU$w<7$y{Wm880eJ)=7_BGf86S9AK#ec0CVFnso%v zD4@Rv*q73zI<+Ij>RdCt$BbVwt$an4b3pr10_h7yS%{FTm5Es956o9wG0WHx*s9sY zq0Tex`MdAne;IBWH0^}2*uyoZ5X%YTQ_k(_ZBRC;V(N+OgZXTN-LRK=q*)yf=f`I? z_ACR?-0y&61=qgFAp*}~XYHxXT+*h$1lTIB5thD7iqX(dXF}4ezb4S3r>P@WPOxo# z^QCB;R|xg_h*Wq}N}%?=?I_gJDy6+j?<{!`yH$o3R?4%zMSbq5XCo(&tW)b2#_b~h z8xb?%5#fMJzVV>F330fz>cR-SU=dDYO)?QZd9CQJ1JE0Lo~YWfm1+B~?x1G4kiQ(* z_9+J$M|QBG>ZUk=B1-j*ZR0#QJUuCy5pV@|Ed>8coSZ|VS%)a(HlTepRl?Ku&G9)* zha(!7t3p9fAdHV2KYdUrzyERj#Z1-=-plrr)ZXR#;o|vpw2LUd+QjB&wfcEyD>iL= zXR4;O{HN7<;X9Yf9Zy&l?}}+m?EN0G=W(8}*hws<kANi`P~i+9>)&~R={jEZF41CB zWF+xq#1QjuZQ-Qoy8W_TKP}(Rt&i*KRoPjf@yK1|1KXk7TrpXO>^0d=d$Fr4Nek0E zJrYyH|EsyTjH+W<-iJ>JPJ(N22yVe$f&_v)1b26LLU4Box^Z`R2pSxMJ8U4h!$vmp z-{-yech?~+C!b#yA7*Ctp6TwXu72vNnyQY2EAK8me;b(0m)gFM{EEQHaH?%=x}~f5 zsBr9NV(yg#?Cl74@KZ3<nN9Z(l(4AbbTU<4b&fTv)nFcOmHl{%4!QQwQFi6^CsbP& zqG$E0go^(*nVkYjJ3lm)W4#cJqA~=QZM;Gzo{w&M8N3o^ala4a>Zuu{GCwmr%^Xq3 z9!O(}J*n!BcPZf-B^BHp>O-J{kk*QdPb{WfkY0S`cJg)g0L9&uH+%JMqDHXkkFV%z zm^C*E>45$-v{s@F#vKtYhD&Tae^PVF`cVd1EyMs2!8D^#kZCXbht;&!eH9h7L%Z|S z7t=F@0n;Uocj>McJH6I{HZ(jr%n<`QckEsSyqRDNcNx?{_2{N#u26qxXD+*W1{s}g zX9h*vx?x>Vy$d1vF~wnsRZHlqtoM&)tiT(Mz{qRd%hFLB7D10GbM^Bf&nAkHB#_TY zf6H4CrpJJyP068O8ztdlxZw{f>mF8k{5f2nZ%{!|#0j|^0rkUv+>mk_B*O>W#oWfa zE_9Ka9PMr4+I4d6#f*n;FN&Gj=RGFySSne<;Uey^>xMD>)2&GZ{l?ybju-s-&(*ZG zBU?lYI+fvy%O$nKr@M5ODx)8K?3u|ng`3|LD|Be^X-eu~pqwm}4lj}$tEkO28xAJ? z{MfN8-)<tnLQw^C^BoV%8+M>i;L4-z5hIyM+_|Z#{Np?Y<NPBks)aIWLDO0!SK!?& z-Zkfj-Ma$>Q5^)^(C;wi90Oe2J_Y@@on#fuCDUAtD$1HNpIl(UwcOKL+6G0ey4L<$ zD>V#~>VfhnTz4%k6EmD4KG#<ciB0#Tb#r4hb8<9#!<GG`)mfE;CTtwjT>2_SD1-d6 zErZq;@9_^Um$npRR9Lnp#jV+|BKMX06L~HYw(Zjm2I#MfD+Wy1rori=0&H~>Om5Z} zM&rgiq10z~q^yM}+5Es+GSA_nSg;Hv*^-sZi-<Ag5Kr=8)x;y9#0pZu2iXHJ@#rN+ zl0$o<54}WNxTz8g+lX8%ig>m`ril@p;55q4{ZYQvY4})%iDc*ew}n3T@zdqU+7-MM z=9`D~KVmiZ4iExLu7|xTN!H384*Bw}82>L9fU9^uEdu*%w7l(Bm%C~GW%{+NDQjkX zYpiMeaKZDAXg&lh9Jv=qVZ9L?c)ez}C#QMm1Yx|PXWs;zoSpRT+FU1HE2xach-F^| z*mXVTU?y{Rx2`u#Ul&vLus{^Q;x9%B<JT?_5~CkXOcy$I$^H8?<Me(lNuP>vw**}H z&92EQU_2){w@xyEisBwtBr?iP*}p&LzJ=de5iDKioDf)yM$ynf#{I(_OpmEhN?HFJ zF1jyv->R-BPfHUOK6`RdjeKv~b~|1KD5vC8)hwtQxpGc^SQTPs-4prg#Xu<C3`d66 z{X`ImV8X}HIF>EoT*WxCXeb)g6qwYU5F%*s(E#2Pg#b-({1egFv>!Q{P3%Ih;kr2d zahLi_t0K6mlG)5=!Nu^tRfMW7V&J~ZThSuxg>Z7yl9;~IUK+H|Q4t|elU18Y?%%LA zM3ui=yaf6xhtik#;G(VNSXknIB&(MZ-N>eK;(&GjdMBTg<*MYB4||4^eg<2LdI^`v zuW02&@IZ8?LWTJ>^RjSy^5e9&iz3`!bS_-=wOKTuk=Zy%g=V>i!VE1>9pDjfjwZ{N zR6<|1ylXrbW_&RNt9-Fyox>T>Lsd9QiGsN3f8`NTKnYv*G{MX%1M1p<gdvVs*=^2n z@ltooD8U&F^b}PlqA?lV3z-Q<6*n{?gcF9KY>f4A5u^lcM@>?)&Ccy?S!4R<I!xv; z^Gc*c)dm&kDmDa4KhDQ!<(rJj(Y~gi?WgtizX6sf6Z9!2=vk}0o~Ceg6TQzBe}$p> z8RZhQ$(Kuvmf1OflX{?pUI&5k6-}daoC#xLKsS|ab3t$FT~?kF3i8h_zvxc8%zOF5 zi2L<=9?piA?QnH^HPzTrCKPH!F=O+3c8;Hj(UGq`@}@cOjv{1#+<RP%%P^$*v=#3d zxTlQ+q}2JBf8?BJrWw~kqe{t=H0@@tV*($Dr1CErCID(3+@qi{1ZK~f(D2u#=eHDI zLvPOyOSeeHMLTx3N|<ztsJyocqXw`2?>Z%xYxEifmA8)TN3M&_RcW4rzt$~3+bOze zQP(53RX%?wOCr}Bm%SYA%OIBXKks2n8({j9v{=B`otIioIaPcq{eh4MDjazTJgBm* zB>2U+pfp25-~oPrFjmRqXUbryq)59Uo$?~uog?ik6ZkKUh^F;Wx>1h7n6O-(@FAJ= zo~0U`kMSw<;n>=aLlBy0xD3ec6kb%;dk%_O3ysYn?+y)$TS?a)L9$S2x|bPsNZbo? zJhNc{a)>Vl3cHk^(RCCWGm~I&1k>EU4u@rmhS5wLx;54@PA0DnMw+%j6#0Oy%S&$R z(>b?}*&>#pn&Y4*jZKBfDS=qb8S8M7T|(4et^W!Ig&aFv-^oc`@Gl`t;(k*NM9@qe zL=1<euYlhBlv0u3g>{lQYK9p_>_|3XlO6RO@r1^pT+XwAH<RY&4)Z+Xun6q6pRoN8 zl2h91$)>5R3nWdL4;Tt_DKurup?5}i;}^<Iup3jb2#$_ERK3TJ`BCd#8WTsGP-8H+ zhj|1lY_xqV6B*%#Mde$<0sRGj*lNP%L^ltHp=Lg7GDd51+R&C`q*%@hM+RFw+SRZ- z#km4U??B}xqpjoq7+@teMT(Sn#2CBLXxK7QDW*}zWg>yENzuCmTjCJ#eM?1szj(QF zpW-M*RJ9T-R=PgxedtYi116aw;EnjwK3r0pDz@<zv@%{rCNi7X3W<IJ{W}ORJ^prr z&ikZbFxOa(w0Z#={cWwgSb%L(ZhUZ*l<q?D$WA)_E(Zxp!iZ{|HD&c-<Vs0qK1V#` z0GnJ;cI-0D2P)Vo-Z(>>=D@>YZ1NGO#zB68IL`hha`^fHBT^2-_Zw#>13zbrht@_m zt>3?l>jNu-Vg(XUsy8iW>5cju-jiL|#$YFEG7IqLi)~NG&|ju-&a$4z)U_PTh9Du0 zM4;sCji4QtpUeVl7<qSkFf%=kmp-z1aIz$wZ+(q#)SPzB-(vkFV`U=AMD4y{T$5#H z*LBfui=NP_YooPDfnC#Xr`<3ZJ`gQQm<*>cscf?!_U#SA!2vIG;&)4NgtfwwDwZb{ zg-=b-ipX}sV5|eZ^F1*1TM|*_1q|Xpo;~r-8xQIwb}k2QAF>#KdvoN^sdv>!Sw5%^ zXg^h4&C#okV)|JSRm2`It+81!GOHeCzGuHASbcuQ#m#CXmJt}(@*#3g+SXwJ#fe!r z8))w6zGPC)xWduu&F{^xs-~7aQ?iuMy1UqRUQ5>?7lqsso_$4MGklQd(Jwg`(Sju7 zJ~yc(P-vQ8wl$K+K0jl1@%pl<<Nb>2m>iGupuuNgIQL?~p;lW}(wX5h=_5b|kiycA z^kz|@Fs&@pY~&3wapQ+z&-4y--_Z&Y>0y=*G8KM8evWA5b`m9$gXlSSdE}q*s#aVU z^|VnnxO!GBdy$DSucS6iYUt+*U0U)sH`SGAz94-|`H{4kR$Z`!M_3n>8x&k8f-Mz} z9BnC>olE#)w5^`XqR+rzRlIfujuxO%=fHL#p~~GiWB#(7KaRshtaZe|3YqsTqxk$i zc3+s}z25Xet`>GV0YM@&Nk^)&LZ`q0m9>&MfSq~=qzeGOu)e=+A?Q`RUf*qE=N6z1 zH)~jMmf-go`S>KdbjSg{xOJ!G&Cn5_D|<^gsZY0e>06<DZpFL(Vu<Xir~xqdd4?4i z!!!S>p2=h|*Sp08CnjJXyOyrrK-M#DTi~4y?Mz0(!3D<zXS4Dg1D@&q{*bcgS)joF zjc_t^=E<5$BJdMm*(txtV2RU~m&5W^<LWHVlp!TQhU3Ffs(l{%(#Tc|`p?|zL*a0s zlmx3{$6?x~^R)i!XsYr$*388mV6D&U8FBe>_77c~wPE%wuuy-&C-@)qCCRdq;H!3N zY{$#Rf&n|1h|;rn1c`w(ND-|?TKKF#L3Det8BPr+*4vfZ{RIKVE{h?f-fI2Aj=}7> z_zfQEXNCP8d!BvP^8GTwtTxlY_<cUw>%zRo%4Gq?BZ7KLssUqytNHXiK2N_Nb?nFV z>Mb(^X?Mr(HYLui68p$JjTHF3bz?6MzeN3PmAK3Pcr0GCo4v0)+B$gVS#i8_Y>)@6 zU1*lbQH*MqTbQe$IG}$ewN}J(=oxe8n>D@%{x%=so&CMr$>UzqXJ)+Tyj`y!(4rRZ zS(M)4T&bcBxCwJT7ID$v_L6*((pIQ06n`2gG1QF0iKK8%H|?yPhFH2O)dy@M4=34+ z`&DQ-N<VCxpN84ZIX2}q>=B(VC5W8{bBaK&?tmSn`nr>;A5SRESB}pM6>p<b|1?Yq zzfKg1H6JD?`BE@K<>Ffgo9b!xzC@m;hC0S|lHZ+b`pu{Uta3pe)`2~=vXV_Kiz#fo zLE6305k7-*=`KA;Sg)Z<{#~eYYr6GVxB31+cJ=VW4&xA2_`_CjBaguX&|@c_S<Q_@ z5Y!r!LA)w^ew!kYTrN4aXo#d3E)`EXGH;MiQ)>OLkv>Bl`U_F?dhrhqS9fgrIMKjE z#V`8wa_jiBWHqOD&@@U+P%r_uDiu>GHZ8OIa;%a&1uQ8wNry@wO#_^qB1)|2R^!e| zdh0d6(qzxWw+IE(Yg23bF=Jw(RnFj{+#>Zwf^eA^I($+!#Msi=hSK5LZZ|cd58iV| zYjh1^qqe(j6E}MFcSV5TQF*p=)Q84wO`CsBWx3?B{|H`W*7wgmU}yNEt1Q}46cX68 z$x1d~eBe_3uCza1UoUwoFOa-7e@xf{nOqZo+*(cZI39Z=4r>51R}Y9DbDBx$bzzCg zSf;Jmg3~nBI<;Jxqe6Fb=M)(*JBL-Ogau*4ov>-G9_hjA4(%Tb9iT<(hPp#4iWVt= zzN*x0NV62;grtibW&4)LZ=CvCx2-Lw4EcN`=3Pt&l&1Hh^QRV!&8=91!0EwXTt1N@ zibs;rAZ7K(jdces@U%JqeEqr^Zb*H&l^7;<acg5F;(M9JFrh7oJ%x9*Z7yQ0`r|-a z1es|%o2Iun;lc^S)w>aLuOaFYunwpy$+PaK36T@`W+8hjefN*)J(?cYpCrQdD<}@F zBb`j&{qu|F&F|D&1X@EUmQ04s38ztvYht$vwn6p9Qb+AqSj3%U?W@Hs*DYst%?v-^ zY_A$ZfkaxZGGw@sPLn|1soPbTPoy?ieE)Mg|9#+B1CNQ3)L4u+d_Hb9Myc;%Uz{$K z8PkObZ}gLZ6YD(H^%2S_Ff1+p(qgIt3XwuyABbTn+u2^KW6yZ+Q9NexgF&g$fhl%# zvq_?zVnOw7C4wrWz~S6|a`nof-hP?lF@`khvCgs@wd>1n;enk_vcm#L<lc<#^b#*T zq6Gn^@cwNa1Au8BA;`cazdqNFCO<f7*!osL<roPR=4wj9VM(g4)WQ&`6jqA8QVlf7 zf@e)+%3VqSegM+Z%?m_Ikhc%cZjov(qpthtyMLVUNS~A^-;m#3=dJj44(?8IPbxCp zHH_&J!&*Y)f!jty+GmL{s7*sBub6Vh<7Bj{6y24zuf^(Z+v{N2L2_vO+aE_)XSMzP z7A8<u^)U-`?U(IZm!&E{ElA`ao3~jeI#$$r7#&}ics^PrG`_KBGcMb4!$-94$-u?F zMj}=6?mLpvO?2rj<GL_MeU@a@iUw-W=?#@^y81LF<7y*3!b{co&pJ1S5m1&^reCFY zP-6?<5?1v(<%bLnWyM)(E;^uzOV+K>nC|f;7HOg*w6al9C&#oU2wZ&Ropq#3UaIj< z#huDwSu`jmp`}C55AEy7SKbj5N=DPj7Yq;02oIxW&f)`y;iP`-Wkv<~cg_Yk$NCT` zAVOkMvx>?jX=pQXDh;#^l89YRUJP-F&MGMDAerQAT^~U4SEARe?^+;sm9qC=jA$iJ zUA?hQnZ^Y!9^JwNG4`|bQR;z}-l3)jr9V0xkcziDXd4-%pQ}90#J>W!%NJ{{a1dgI z?4jyOBL+mPN>H%ndcI;|d`;UOQ1}!xWF=!=W4Ccu{=EgeeUn+#h*YxEoc{W4l?k+} zVH<;<(aDSNZ#@@eH`Yf%3T)ibRle*L_w4>%Uw$c0P8$Mw7UKS^R7hUdDjjGqGkDcw zy7dDw{qp_d*U?=T_FB$m=Mhu&HSYP_clcj06V(u-Z-*%DHa5!<^(~1=m#E^Ee9djO z60GT2e@V+~<qn0J)aZp%>-V?sw(14+Y?sOaY~2@?xGwCsLZt3UF0^T)&tf$2h1a?9 z73t{yoCbEj4w{<YcCjLN<C#zRmjc{X%zw9f<vRP^gYnBx(vl#<msxUVOMgcCJ}YGP z8JZ&q(kgOeOyifyd)7>E3O%c;&Ac@#=$ZZXOYQKke$*l>eQx$I5J%ni!%S|6K!{x) z4o2}vSdlTh{j11))r7fG<p~bs(s`U$*`9i@=xP|_yC%Azv6sW#ei`EP(?qeXoWSkC zK;a*UYz2km>(hg|P42eru#|}TG|28bf&u$2{<sp_U!64PIaDg=U4ODg5ej_XAFBoi zVTPnYq^e<`XL%tlt4ixLivP2bLBfar@lWe7B!)DsBWgW~3jrX^0&hg>au*_byd0?% zj~CA2if_y_iY#vPC<=FR3HnAx^5!{dK(BKd){Xcl!;4ui4`5P0mUH@Ic~Q)b1^n_h zo}E^8KyyS^El;3g;^4kEGsEP>T_OJ(@y7l(uc`j4W6>z7z(lBCyFEZ5TD}SYEFg8J z?X#OLuIpk8i|loSlvX93Bo*)_Jlp|cBmNaUY!7Y-mE?ZpV$g@_Sy~*zD+68lBmG?Z zr3I?q0(ma_V_sN2dKKo-SQ|p3xKrQIYci*KT&j(1c;Hyse&94Nk4S%mHM}!2$sCdd ztl`U*1($HbZ+3-K$8o7$qR+bh{H45=XYrFErzambwD%si$2E5!QTVi(;c0hzijbbd zaRlh!QyQ?VsvsO{?p^Mp!5}4qB+*~~i-jfEjd3-!#Xs17Xp}3G)l_!l^jk}umdRU6 z${2~a@7UP`LvLbk)LdSfA-<<1&MkmdoTIT{v&a23H|B`T?Iaxx&^g^GRI<0v(dVcJ zUw%Z3#;J<I?KbvAS?}GmI^4quu5RQx91F_awdwWi8K^nR#0JY{#JF&F&n(MKJ|_@< z6<`?jdC_hM9jVfqV=}8oY=nsPswp<{0cP7z68$ad(uMD0i-hgWFbT(ufI^y>oADxK z_l$Z5-$z5X+5(+ef+(qW)KwA2SV=rRfMaP2YHGm`;Ootw{;t4;+!xn_ugtf9bg#_= z!`5G((eN5tKn<jm-H}BWAzX!pqZG~x;Tvqm!{7O3gLsdW{g5N5rX<FD(`#h29fcAT zyStcMyM+ec{rGWwyP!}fDV<de8(N%@0Ra5z4OK}(b5vHf!3#0mB^IS-irMC(D6u%Y znVF5mTfAVek1V=gdjAM(U#g%>*KLa>{1UtDmMyM5^1E~&GYeY)tE}v{sA2w6Yzr!7 z!by0EhbVy<cqF#TB{&ur-!9Brr0N@-t_;JQ17nNY#rsHfRi>QA5etr>5YO<`l2<GO zg2H3UN37&ZEYVBB9^&)>0ekqFr7v6p2}0ByQPpk)X!){^<o^tw`{meA<!zNW*09D} zPH$i!n)kN6?U1Q`MpH6%M@m974NE9m-87*?-`4W{_IM7fd4Q$2O?SeL)T-xcQ99s5 z<Bd5HLbw@$4ICs1zES46XbwzD_7sn|k{qs&_gkH;NOy-UWMjpK%(Jpwxs}j%A~_-+ z>Co9Yvgx&8a!%sO_c5yJTo%*2e;EwY@Lw#1Zbd!Q6#vv3kYfm;N|{=nj%A}#n&@UJ zRV(Bj**n^&t<YRt4woDMal}JQ1CuIIxDhR}52kN@_pQm)$4jo-&*3{4{acGaQDE!o z5N*Rs7D-ytKZ`hf+Rsn=^kl=@vViA7N7=LZPJNDbVFyuX_79wI!bv%J!p{6YF!0T> ziS~LAyfWxfaH@xtk8=Jv#Whc?t0f@vOW&TIu0z3Fz0v1pW~W5jlF^sM9>T}l?Ohe2 z$}njRZ4G`yZ*+O7O^6*K+FKr=yQ~r6W<rWzKRcF`Dy45@BWr?SYNRjdi4;I?W?mL- zYlA3Wn9+sqQej==wl>|1dq;lsE<Rig-=*PAK#tC&+LxF7W|wJh;{xo#3G)TZT5Mm} zBiz2rAM?JkcsJvTnz6F7ON=qhiUDuE6tv&&jQgIP2Q04liAdi|a?xyPJ@Zc!uIkrS zU>|4?9D8Y(G(T+avUX#?lo)VBtr{PZNaW?@`w}QK#bwARETK+GK2!!@-@np(S?xdX z_zE{8LoC;bj{&bR`dHk%4Ph!X0*MD;q-{WFf^U@4zL~ATX=?qBj3aZFqoivVL}Ib5 z?wn<nV10+X-`i9%`Khj#$L=tz?*3B$aKU9hY7SsJ;ov2d`ZgS$2L#}&!=ZX9Q**kG zmu??$)&Tl&Mm8Gx$6-VZiH)mQlngmDZ*cIU3R#L1_s5%y8o^esdcDRC#WTfdC|X{e z3T~j}_0XfIfFtX60nNV{Go{@WF?vgBP}WYPz)e!Y=8+y@3QKuu;K`k`l~-hqj)MwL z?dup^K>Oa}wNPGwc2qzHDHQUpY_|-326joCcluV8_N8wXK7{$f9##3;4l}{Z=P$eB zQtfL6tFQ@WxEM0@!s&HXZgTiFp4p#NW*NGlH6bCPs?R6%p#A;*`*j(I4wBmVzFYtC z!Uq%_jzw7|3iGyF_t7#F1f4X7c%i%&<i${NN6ZWIjr2R2{6%E68Do|jQ^!RL(vBS1 zq_}>iw<v!^A_)IM%k&_MYI7}p`pJg2B3d*ZN6dvbH)jnl8T)SyLTKC2qmiU<Yh&f) zcpqur+9o%Bqc$<t=S5sfxBNh3?%;U(JYe$K0{IL*L?J&}@ovaky00J^ub(Emh+r4@ zLmu;7t3%(SY|k%^pPiCKcVVYjr_Xgua+Q~e^QDM>P(>~~pXt)ARqf!Q&91KV`*&H; zE5(37ELnc7&1-+hggFF`(tnLHeu1v81mzT1UEubK>zOI>D@f)48>H$0YQ+AyM-)K7 zWF?Ibw(K*!`pZu=qyNV3bA?uDe|lmW%?rd~%dq6YKf=UWav^>9gDV=%=Lq?iCt6DC zfm4&lx|zOq2~+yR(Bzj4QTgI%5g<LYQ-9qukSOIBSd)~<=NqgwS9SUK5B%Z9J(y!5 zDnN)5ZD2y(Ld<=RN(uJt5q>}aC|N}<SL2bf)e3)KBm-tCJp@IcK7VT`zX2vfbl*B@ z1C2qIDKXXlmySI<m5=xvF(M%a25g#3%K^R>$BWxvtXjO1#*|blYwi;Dg`aVTW3DPg z)ohEXj|z2F-CU22|2kG`c6W(j?nn-Fq%=T&$hI!xb)!;*+-Eoip|F?66T6Nk(DRp? zh0ji@ga3Z)YEYe+;n#G6fg7_|lTjv|`=_HB63wQ=#CgC%=bW%B+UgjgCxli)zXoc` z_*#j=_>ikQp2)NhwtO1|!y_iG+z!W%-C;*w@W>JWpJRcf$!gyq|J|SMg@41X@08qj zP>FX%t5%!Fi^QpyBi)2YEp&w1I3suoNkLAwt-V(*IZlx1bh`<Atm<;2Z+f^dD?V$w zWsQVNMUL4`IL|2P5({5<;kG#zVHj>FTUuPq&E?GMz66X%>v?i|q(Yfl*a(k)v9LvK z&vwDpZf@ickg(O#pxM|!?cRXDgS$g>P=`d#NytR%8(BePDABZrH8YC}$8r3K4(-Kl z5s?M=%}A)WHN=}J8tyvX2E1d7Ma@JD5Z3Rda|1P6^9%k#@2JLiHAF=r({&0VO!;_U z^vr?1NSQwm?;{^r<o*pk_-h?-3Q$|DI<ZyLa#%xxEzwSvgZ#&=*0eM=nGHkbe%@IS zWP<vk-*F2DrdG=y(xjtJZt-<LLS3Zo0OuQavKrrAH@Vxvk~7ppjNtyvwCLuIRe#8K z<HRxR=AXWaNwx$6=HX)qX{teggGHOc-&Zh{js?w0fgMx*1@d<A(K^b;u+n9i`8R7% z3D^94OyB0wggNm=(4y$ZkF`e2B3^jtS9ock@NY#9aqCQTa#!>HLBGtGLI2oegHRIm zEjFom!{J?OD$U0PR^{w|XocRRllCK;*m2d2R7*4|v6_MF6VyCVE-?<Q%mUodzB5pi zLCX;mW113FhGq^9QIYxSxy#y;CXb7nbp56~`lD2n+X>ztW|AhzbiQLz5arq}z4*nL zVsef?6iP-y8qXEGc^k@yfcWGBT6S}*nw-l?PZ0DS%7-+iyxde&?UtGCjq}lTg|Lb6 zDMCcqJ(1Ab>ASm@J5-C)_c6_BI}^y3``uesZw4o=Rz#E>c^Fr%MYsRT$^B?Q%1nM_ z7Y=-0Gw7Mh>=OH+cY!z@&QpT@Kt?%UXgayH+oS9SFCt=?sf<)qY%SWphJ0R%*9Ei? z-ck*K6}`;BQK)Z3ODX315EPU}2`Z5ZZvyF$6?W$91_0ehD5HBgIVqf9>q8Q0Bn4h3 zR6!~eZSCzD7L~3B+3moW^s<XP&`0@!`qR(bs36{db+T1tkaJUab*P~CA!-Tlz}9r| zVn6Ix?(szxiE{2*CC;l$OEsL>v5@aGQnubJXw!2y>$X<!0pu}DKu1_d8Y)|H&hAJb z`%#5(3Ex+;HTg&;^#87}yOPMOyr&u@5;V#0XwlTHAr>M8P@5nuy>|yt8r#@WGJi>G z5vll^j<2~zA1~x8MHqz#!^4z<ZhRn;VJ6_v?<{4%->Z=B@WZ(;4p=)6NS2btVUZXe zB_)rv40Sr_jG%4}4do&Wj+(Dvl9eWzS@}wZP4}iF{rnyXXr3+~${3?({q_(yj5Vl) zYrlwlnxwWV-cI$Y`05?uAA0?hsw&ikekZJ;*sYRoz9gjq6)^BY*~a4F+pdwK35Kb* z2Iff#W=mudfmzRgp*B4$A?M$6KzaEpij12#e3EEWsHYeLjK~#d-K3jy&4yDpJP+4V zt;8{!lkz`w&$l)rgRkfGsf`D2MSHE`#okSr4kg1t;h1lM?hS0sU%Rt@ZMP4P#=$b9 zDlU1Ss~>x=o-Wx$b_=Y*{aq2D|85yoi3tAz-$9}Vh)Nh3xc|l8KIS5C)yXluYjaHB zp})_)Y5Oi62|?aFh?O)cFnm<3{y0WKcHJ~Is8Lz@)jF*WToxvW*oWTsD9pwzfvD!O z3q2jLf}(hoJe^e>cTsG58ur_p+@V99K!%c9&Jiwn_HH?!xyqJJ>zdeCn>izHlqF4= z#zQTmeFBU@wX0C1XRnVar{ElEW-_I1vRuB$;wXd6Ut8cGN>5QC=Xe?+7WYE8EJY`4 zqyI{S{d4tLy@pErpe|s)1BsJ58hA{z?a%HRGw}QY=e;>lFBMXfs^R+Kbn@Xfd{~j@ zPS`k*L*zCoMrN7#!1`|f{oz6_97f7v#PVJ)Sb_KzkR|1tRRatafV=S;xW=_By_q5K z1$I!$_Y+~nbma7xroAFGHQcBWuXEG}>r$MwefzGPF|+V24mpBZW3ULW4SRh?_qPX6 z1?_e?Fba>&ySF7rZWAsZq}I%e=hgnsKQevzkro3|6h~IgTLi9%(rygEdZr_hJT7K} zpxKoO!@^|c)s@TVl=yR~G4BCQs{ujo=U9|P@LBb_D4UQkgkH0|ywQF6!9ZK~&OAMK zu)Un$+sv_U#jiS{SA}$bVKcAVwKxJLHnway!h8@HRqYMGY(MKu3>a-yHqjtGO5#$! zDd30P%ED1bClBaQZlU~8G2ZmM1epfhrDb+%Bp6u7!|$0zeb7_5&KJgpBvQ9meI^sc z=SzMn^=2pLptn_~?{Tsz!;iT?_Bn_AsH^o!#7YsSZH!+f?)7rOBpgK^5u!8LWNx}^ z9DpKvR_5zD6Jq5JwZy~2Q+976`E?|&d<<pv?GB5kIyVV6*352E`0K_g<&SSF(=Em= zlDzs@8EGX)%~WfBy@!ohSPPOHEEUd<r7bBQ?!0s?`OW)+q!BpW91@&yPmr?(Rl<W7 zH3GWzr)dR%Q!PH@@7w!zome96yzwT6N-{$Kumy933Grm~hhls}LGO<``l)wtn2f5o z@&Q}6Hl6Fx<eFeBd@8djy(clD9Fdv^5VB}Vb8u&@c(Z3nb!iYf5QUswUXDm5m+7)N z7ohQ^F`MZhOZ5!NJtw!YgtBt9^_Bl3P9=f1=xN_|<hMyb3mF1#dejlWo=)VJeFpsh zK_N=4=a5ywn9}PiKQzilUWF4|ATc;PF0uzZJiIvb)h1U;DW<7Z!l`=m4A1=&(-x6H zU^`tAEA0xVZY2ER`M@76$aNs4kQy%J)4|%SKMdxrJO>1q8<QKt{_8^_9cXGokXP?$ z0t3pao{RcFI|p$LL0U25<NZk$#B}w^eC|H)TY0kL|93_H7eWH0mBYgKuYWL$Ak>Nt zWIC9RA59PbARH1|R!~dBW7Qw<h~!LTZYyiKo<>+Lm3#U=9Y!53ZFnAUyXn+d(*JcB zU1L{nVE|ev0k%;iY3|{+Q0rGu{&wo<$f`EF&uhm8!o`U)`l1Mz&hGbDiWc4X!rpXC z2O`k(U9tU;<0}>eTUM_6OEEU**$Go6JXF8UbPhEFlq3CQ(EiwLXeQO8xA|L9nGNU) zYVDTKIk!ns@a(%NrJ=2Sk3*O3GmbSrPyuL>M!FeR`Sz$DHL*BQJ3gCvrdpZ3%v?9) zUgiDU-j>Sgi3DyFKit^!E8RMv@i&(<yx2zG<Q{iN_MvAK53@duHZbC#O!sbJUT#z@ z&yUD1pVnyz{GC0j_W*BIRCT|#@D#kG(_#6b;Tb!p+zu{03%IFfY^W9ti(P2411<Dd zuhFtIaRxO=@-86%POki}D5|o%Ycz_Y1rp~fd$11jvBiyi%XD=UG%&zr%8Jrz5nT?3 zMd||o;#BBTgyv9I*BN@(|17ZOxu$~bK+2R5;Z9s&stfzwX#Q_$(k69*A{$0($|5=Z zt~}wZ1o_@^43dJa;+-%@-~LwK{TM_)vnDmlo7T|$cFO;Iv5E+qu?BBA8fd%bO58zi z#G0%VDR}q5*;Pm__(1)5_i()tOSlxv?aRIDDTh?iUZ-_GzAi;f)J26pzOEJ(bk<V3 zlQ)7Ut$Rj$zc;BVPtoTMb`-hsm&-r3-FZ9Ox*Bxdv8roBVVUXEW6~**fv#gOErp~z zcgL};;CV}1KeLxC&NOgRUZkrw7{N|&;tei?A1C&L>Aa%Sh0{qVJh7bLiVM#actE+i z_eW}~u7~M7&ADIFj%}b<@c0Q{>*<~m41d!s$afw{dVeNpbV}DrHwO-%!5aG)y~9WZ zbt^gC(&)u5T3vkd!0U6i`|%!-$ua$Oh!ozqaTwopo#xmiSNOGuP_T)x)2TlFUJt$V zVLd(G7RTp-25Ih)T1R^0>xG-qy@D6n%lrF<i>gcybb$Lif2MV~n;sV=UV!X6ia^|i z^NAA7;e6<fnWw&{Dp8AKT3l^(?qlcEZBAI><GyQ^>-YtB;0_!ghrmAQ+kn8maXL6$ z&nFHmQ>v4X<l($W`z8|W>Hx7+d%%kBh}967Of0eA1MuzB!Kmw9jkwUtL^Aaah^8<; zE%Xf##JuL<O{Ug=$iY7CS`!So4*f*s>rON~bBjWk$DkWiZT}b-*WW)UaQ~hlgh|iq z^x8gunfPZXpqGls8?n+ytKzynaT0iCIGp<1_0H%5m8PQ(3GCZ!eUiM+MNei0aO<?m zxvI|}g+3@6U^i;3_}lvKd1lJ5E28yl&5g?e`V61W0P1cYye_)l>|XAyDL6!)n9&*j zz1mS3Gl~%*>XN&)F52N3C%*idE%z3ONf*s<Rtk8$6+IKhoQBu8FnNIV;FI%1>4Csn z)9TMh-1^)><voLvVv^*vzT4yE6^V`avYpsz$%hyjcl3{YO3_WXG50(jmu?t7I%iG| z(mXG9>*dDAfUze}g-&rOb2{Z?wVEI9fCwqCqokMy64jm}GAS6_H^S1BHFjcsyxxmG zX!U<OW-)vy>)o+dN!C41pjZJHX^YROsBI1nbKaj%CROB(cbw}X+TOIk$P<{!zAhng zKJ*<%SiT<6U#=Oi9M-uf<#YX56ybQ{47^I{P>S@;nB1eon^Kt(^rtJWHPjZvIcCaG zXt@<U`|g_#<1(|pNLQM5N6hQxVEy$bqL+@m#E4*;%kQ?eto$z3iOC|~hRTNfarg!} zmArYU{WF!x<bt4pYD;*4aikECZ~%7DLbK|}(S3YX(XOBT5VLFbVKo31f1BuRK9{E1 zRC!|vXRgUme6B^SHNyN~s8FFmjG|Y@R)H}IMP(+`zQFmkvxbVN$ZC{_>l(?siHEfw z;?+A=H#=bHms<$dWUByoH97!2);uBW0cV=cblsBp!Lbdr+U;%+QfPb19N*|Dg>t)` z!?9c8UuziVetM?^O0psNCO1Jv`rvzS(`F4QGBz!oI8Pe?{%S}2reClPj%g61>S!eK zp;*+|SBAYo_nd(5;!rMYEU~HW5VVU)CYYei>#~udM!#+-=!H%EIC6b@Q7&j{$`3~E zE`Or7I&Vnsgnhd8(-dnOUdQOXzzLbhu+0!{J?)F`faBEJ8@&MBs2}KVJfJIwfS=lT zEbf=V89wJm2i{MuX9rJ5Wz%PB+|F%a!p<PU`$P@7p;5;oQR8|V(gmxZ>U|7P@4B{V z+s3`XwOluX9e!~9lVF|OHxWd-9i2qrAT8HxqsrDLBbQU0v6qc7Yay|%x>5IThfRo2 z*OhCbYHmcLZYA14yBE5TQ{yCYv6jb5C*KkU>1{5^iMaz^Cae7%-Huj><AOh&^jo8N zV^=<0a^pz}(#^d)4Q(}kwa>OnTiM<CxRJ3~FWa$ldvyL2$ozX?qRiv>TDNU{<So6c z;Oz?We#+2w<oG%$NF(BLp%t~sCYoPUee^5+c_D9aBYq>D8V(A7#PsG`-%etY`;1D{ zfE$#&`%VWiY{<5KFMrpwyj;G)_te(zr03$R-11J?(8`GyA#7TWW480k@u67?yL%7a z<dI|0myp60*EU33<fU!z8&(ACgJsUbdb<|eCz+=A=h(GR!aam^VaKW(9klY>?OWVS zRZ1q%rR_YN8&Zk9gy0fcSIyI_U7wTT%QJ7lqv=SBz_L9;gh1O#ff|r_&AwdLKe!`m zkII<i=EfY-Qac#@G>1Rgc#R$^7>wd&(*M(}5c_3iZ`$gobZ`ft8T<OM6T98D4Z1Of zuV(G26?%%8KmTC<`aBu<Xeb!mYIN_lyXhO;nOt3^yrfD#W+?fsCviIVe(MCAUJ9n9 ziDH<ITjq(&lUF@*#og<cyrIbJlCU!(*h42exBK9m?IWwelmCU(^_<dLADBSlw!E?( z4dA)stf8!Ge3D?V$A62u=yWf8N#xG|c!s<;ej(U9P8e+{Fix}rJ1}c-G+A4zl4@+V z>K%`Vk-W~x?K6V~yz8z341Z$5&`CT=s^8SO-?Z(7YCLH1r54CE4!UqSqBR23265vy z{odi|r$Syu6KG69OYAl-e%6@LSzt<AdZ8$4JZXNYc3E24o#vy{KK?{~48EK@{mAVx zEW^1jMk~P)+aGgR!*A%YCIdI=dKl?;VB3kLYhHulbr?>=cW&%ya$qeJY<Eq_2?UX! z2m{cfRS*goRaPqw?5>|Y@)_~f+Mr#-Q6GkMz4Eb{i0}6~W8!+J01evjm%Rt*>LgU8 z;aex{mO_94I+~}}=rBvf1?AO?alY`|!pvDuXPfTu<h1UAV--f&&zvhx3@$oXENx-{ z{x6=Lq+ik#vco1Rj+$No_(2}apnl#-*F?!n%#L_3bhYJ6mwPp~h)UvCG~V&i4c<Ld zXM~}H^0CTsWjr7+IEkW3*O@%3<LRiWl;)ri%wT)3(q?|*&Ud<9=tN`NvQ5!sa`L+A zA+bSjMQ%u)dX>W%9Qefj6K5q+Gx)r??Yi6-m~?&f9h~(zQUFv=&DrKb5L3L)vxnM7 zSGfHNXc+f|9hZ3WK>@GY|Apdner*p{Nv2=Wy;yrp>LA+iq-pU1J2!1d@t$4rrXwkZ zrOQzsrX28EoR%bO`ybf0jQPOF++%lm&ss)19)8XD?3CB#I*ao#PrL3UvYj-nKy)Pr ztF&n>>POqCd~U4BX!2b@YC5X#jPP7FTY9*)6$sm2R7r3qO;bbtX|Vn;gdoJV_1ZNs z8#>W}WBU;}`*g<a)wZd220HHSdGNiHRa4e!AW%xKM*=5A2Qc{%XIw9SIc8u{a^B%Q zLQjxE+Og7YMbRK5;l3S-5IqtBDdMjfntL|npeH-EK^D3wnPD^!K#;D(cZO2n^UTN) z?HZlkJaG?>V)VxF^;&f0Ta_)Ov($&EPF3fY7is2}#VUf7k0!SFkthdSopzU%r@jK0 zjGPBd9pKzMw_+C7wR_sf{z3b#Ye8y8;hLq%=qJ|W#U~z^Y2>yX#D=HUi;6X)(L1=c zuGqD(4JFxGZ<bmtRQpfo?gz8x$(xyvl&d#sxlOu~^RAmr*(XPw5_m{9YeXl+{q}`K z^-FsrjMvcR;Oxf8eC14lz>`*D_9wh!?!eT8D*~dMZ+xKkUQk-w)N9pEPpZ|+4$gHy z!HDeZV7cs_@U>X<trJ7t?!u?@b~0$eDp6nVMB~hA>zFAiSKdaN8NgD;=0)^+(%NOq zrv;nn*M`B2l%9%B{Hr&{RvnAYKKWWw2=kIcuF|6np1hf*0~ul0^AHcGrP}*hS(xSD zn+8HK=ynOwk+o$pMJgkyd%>pK3b%w*j~#pppRg=?@z3Tw`mQ+jh&xVhKCc9-X+QCK zsdXP2k==I)jdFNFfioYgv73}{45Q!Je?&;46&+eIM$)rjuN8mnK~{=gaN!=H^yh6o zLtiG#p?KWoZ2~h5IQMJnx{_BYxQ}NfruM=3)1wwNP>s<HlqU~2J0;Uo-ULXWu6)YS zmY1c{$d4{t8OGrPZ$5Q*xIT!<3X@Z&byfOqaq`hi7cB_-SKgPS6B;j7V>Ie+yb?#d z5==wW=em;dx;@kFW~ix3ZC{|@p2E7qJg^1a#$hMg`u5<T4z}v}4UD~7r=WV$uK9W? zYqP>6W$%kSa1v&&fcN;&RCzHIb+rH9U%Nl}aoq;3)%&KgG76~M%X!lAFt^Lb?Hyox zDyo5<86UXpQmUUY2XOgVdi_ZjP^_VQac+5by4Ha+*Z$zY7_virmrG!%ynL*!B6i0x z45ZWb#<gwU$mcofOm|m5xn4OH{#52b6wL>_eBPo4lmYAdWe9rU0v^(sm&rS7pFWtl zb?$ndJucpUq1$S~Y{aXP*fb<tKVEglMv+C7YX9gfaHN&TyMOxqNg!Zl$0vj-{Z>u+ z$eUT`QmLdfBagD38xpF@^4`OXi)zSwK#^nZ4ph3#+*q|zF;FfQO^4t(JQSY%)p#i% zaTt%v-Owzu5=@Ul^e=K^4$YBRRc1n4Dib=SdlTP!8iq56iPjm$rc;N{@c6xKW9JS8 zl>JzMgLBc2J2y-}2bXRoS>!ocdcC2FMs)&sGr-V!1#h!-W_VeUz#EWplQHqS>G2-J z?e-MM?#Yb6_M<Nx=lw2t_X>VWlIn-^M`VV3?ur-6So>Oi{Ed$)T=cBmxwP^PJ&n3d z_7^wPwhu5q<h2;OcQ>tKMqst&tEj0a-GI7jdHCpuqmRp%3x%hqollx%c8lG$8J^Ap z!)P?^IW?)p_dhHvsm(+uS&Uum)Fh3Yp|3&h_>E2Bl^K17tGCJB>z}-~I&3%DJ5BCJ ztA;VodVMPwiQIQ}?bq7ZKDa7}4$)e>%$9oFAo&C~Yu=1ac<;_|Z<AmBb?kBRvgzr* zw&`LbZNgn*5&Hq9trZ+!g?!k*^SHNG_t%5pT07C)({vBFcily{r~MJw^z?j=eRJ5F zY3xXDiT6vfu?k4+mQkSvEfI-A!Cvt0@&qd^t}4TA^av<-A1}AA)I)wVy}NVfac26! zeRU|(AX4dGP1JMUej;!$$K82lX*&7XvU2x6G|YE&I1}zbbH7N!w|)H$L0l8u*-6vc z5tZFRlxaSSlLL*KMW;`v7AtE=Cg8m(=V?h{XY{pn@xl3kg!f&aL5AyaRDA2&_0CNQ ztE_TZ@R)8?rO!R%I6F@?g>}-<Q83bu-PVzB5V`nTm>Rdr-oZk1fy%@e9&F7S_6iQ~ zHtE*$O7KDaD)XhHmuY0B=aQn&?<rKFJ!p>Q0f)T;6D3-OPq)fCXFrv=z1?Oa8(v-A z&3zxycXg}-?W+L|iLH9|KY}Mvo>s6EUA6o?1>HDx%AqIdbRvCkR(RU9kOes@a?YFS zP6aw0zLBJDQuYW`3zmAdI%#adhtC1W@H6;24(x|#Tvi%}U)zJTYKDZ}G%6Ao+V1WZ zm#%~o#ou1c)}jrdJY3o0U?nEk?5uqt?h>K(d%|){q@ykUJZyikVLAOmz;*4d(Rpcm zG2W8<lB~XW&p{p_U_hk!WYc=`#6b4}%chfWqu}}{!w7%lT0$00ZGo|;FcG5GMNlJ9 z2Tg($1#@*X1TufA_FI&AB;j}8D-PZatfoIY9WBiJ$9k?7yuMi-SlfJK5G9k2AM0v2 ziht7G-pGpr$-<}5{4jCE<uQCgQI8^iXEOlqMLBy|OZ`U3j{!s%;pb|89c<ii{o}!z zBFg8U;pTLB`s$pXsCEC1S66dIf>(r8Wgpz*J;#H**{LOKL(!op>0|N9=yygyifNUA zw`%X!A@P=QnM0-6jIPwuE9H$OUC(UBz3dCQ4dTv3!cWch+;hcL@(hpgpL`q!T|S1T z+XI(HMsS5;bgnZgfB9(kzh?b9+1sjeNP;ctGRHI~{i2F7g}ddZ8NM4(4~^DYTmE*| zOqp$;@t)drqviEq;lnX%v&OJ>4utX`mt$tHeRa%E8VswGSK(hlu$s~gt?4W*_XC@9 z>lAahNI&P<9uy`#QfO!&0tkU*TR+GUq)!9w#_iSg5Ve*bG_Dr}1&zRXYK$yHkV@wr z-vQ{l-C_fsP3-Q%TK<Rh6HG8*DHD=4$7|EGUKL38Z_Tw3P1~aG2dYQ(&O0W#4ez7i zNuThW53UVV%D&<2rP{q;yJqH@^jR~LK;K{O=NvJp`r;kYs`ds3FtG<Q=4t)>ZLsqE z-G7|JT-8O{tLEw9WO(bSsJa7v+qM)XiP2}glSFJGQb?Xlf#3b=aS|Ind{wyxijysN zU>tQF=;>7)xBC;BsM8v(l0S70-`pj^G4@qte0gx*yRaW^c61wW9MfO-ee^9LTL?#r zN;VcV#?tqS@kB2l+d$_iP3dV(_Ow&@9>%h$Le2Xx!8lnz;1eoCR5g$|iuu)6n@~FA z?_3XkzS*?>l*a%yS8m(>fVD=i9c=%>Yr%>9bfvF#A&226zbr!?jG^}Zi%%m_%WKZE z2PsTTc+=7b%h$2hX1}F3!zdwPEdDK~*TXhF;FE={Ats>3pf`roV+Ea<giTVXjGiSo zrOk{>V$Wyu-%{5Nul!`1-PV$_$AkXqNc;7;i}EexK^QJ?WGH{jQ~u|oU*uZ|+j<U? zV*EC>16cSD(p7dAyt;I!{@KR?5@%+?#~Bk?;sR;r|J?4MP&KDodjGp{pT7f>mDM}E z=8V7k?(bb$pNVOu89Xcl7&|0Mz`5tJaNx4@JEPi9b<4PmR!7<Gza>T!-~Y`;)9IeF zx`7q|>&y4T#uM}1s?`00x1oVJd{6FTnuCwjVJ0GE^QQP;yron$hHw`ZUz-^ImN=V@ z><U60E=iozX@`^Je|W_rm)Ub`x{GaC@2SJAVufYc=g}PbP)(3sA8iUK=ie;X5rOdT zr5AeO-`-9Y&;^=ds%b-Sv|WojxM%8)_POVJl!Bfx&Gc?)gZk?65o*Y`#=*CGTH4t5 z#UTn)WF)r<!1~p7xw0kXzI62&*1rv$P$!1MXQWRvq1~ueKAYOw@Hv8T%(GEP8i^wU znshbv$^Sx5lK}qq11M;%vi$Ye%chT_@0GuEhV1-yUjKJ7@o&G0@ij{M?~U323Pczr z%&HmK5&-*q?DHQUUC0;`cBZxPAad~A+4}FrEFiqRdPNgs_WS?!;Wr2J3y?@MHJ7Ie zp5LvP|C8M!uD{(|6;fu;KZE&2>XrZYSS79q*8MT;mNp6!l@@k@{^pO7WdHx2E=0s( z!P`N((-GEj<%aD)k5xi%6h2+hmq?P1u?`C}iP(J$jh>z!Q*!*>pJtoHFZ}c8A0Zf_ zqTiCRF_WH-Pacuw_^&Ygw!eJG{r8NdkhiOu_8C19;B#lb?0M1-gRdd4u1X_#HyI6F zhG^;YL8?9MToNAG-y@*^$X3@ZS(esYIvI8I9!YY>2J3^}-KsFLl2hK}rkC~g^=@O1 zVHOio#-;?nr{exA5nYs|Szfes*EoG9v}q>vbo6Nf|3bt5&olIytjqj60o-py^dC-M zV>F21V>43zNvXboeD*TJ{Er{~=9co<eow^aheQ4J*0_&<LH<69$%<A8>wWou4tZ8k diff --git a/site/img/dcos/scale.png b/site/img/dcos/scale.png deleted file mode 100644 index a7efeeed99fc5e18c448f226b6e381bd4043789a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 102808 zcmeFZWmH{DwlInY4ess`Y~#V*JxFke;O_3QaVJ<H5G1&J@Qqt=cXxODNT0ra`}E1_ z@qWG^mr-M|_FAiE&6+i9daW=;d5O31c<^9gU~i=)MU}w7pku+nAe>>}yxx(3nOX+} z17|fC5mA&95g}1@u>E9iWda696KABa&mu)n(`R6yuirOJ_YU5{RS5`;RMKx7ZSQXH zZ69p!(9cZK)?URyS%p;T1pD@(quvrRlH|UMv9F|)`ok!efE<PeZx^K{JGC`qz6VBo zQoLpcV(tr=y}5(X2RKMbauimY&z~{;W?B8L;>f;!^E+fGap4Z~1_Q|lRFwNo1wnB^ z&H?<af;S^U7;oNCA(3cN!%6WHhP|!DhRO|m^!CQ|%dA&AMZ8vHVqsyb`pI7P`j3rj z)q_P)$^yF27jfhcI!P>b50?@?x?NF5^>j#Pdg24vy!k*oSU4omLUfi4G?K!^<aP{Q ziDD@2h=Wr%mmO|j0SU+PRW~#Mlp|V!!ivJ0$%Vj-4h1DBDq^2jiJF3d9STA31$GIp z#Af|SKKV+%Omh`=Cv`blZX;W3MgwD8LlZ_fYr9wSf`Re6alc+#n>ZPexLI4-IC8u3 zk^SQh?$_&Iw*h1%|9Hj8l8;PXPLV{!*1?2?oso@^iHskfgoK3G!T1xmlBoDUyTAV8 zBQtYyvf~B-TwPrmU0E4z9ZUhtTwGiLCKdn-3&ZOh436$LP6loaHjd<f0Qny{q9%?; z4(4`F=C(E@zu+1e+B!S&k&*ox=wE+-;A!G!{=XyHIR0~5uM-6PdIDf(WCHvP_G?$( zUw63`&D~6_)J4s$O>7)r$KYpXV`1a{M~A;W`rje{t*gfWcI9B<`uCpy_T-;Ec>%v> z@NYBvgIxc(`${i<cwWH2XwMH{@A=RI1||q5B`T!i27dU(U2S1D?TUEt!ip*A9b}=3 zMZ?v!PW{Xi$&-Yy-!rJCX{J<5wQxW*7)7#u)jgiF<$){S3Kua23JrME53w*lhU>#Z zWo3PY{xJG-Z>yj#=?5jbnv8mTQ{EPF&{S`wLVyGp3-+5vj|Fa7nLjue|GS6(>M<na z2XX!*S3L&m7vR5XSdn1Ro6tOK8H~Rf=l?+ciEjT#tAFs@|MAr?uK52eGYA@XY!B}J zk__!Tl(B_<UBg8f<5FQ7$sk@eZBvRCc^Ta{@0ZlxDzcdR0W{b9b01R$&8+&H(Qz&M zxHw@BuD!tvP*RFIv)W;Y;u{gIAN|Yt9?6Mb%;d?IGR+e&h)+vD*=Ff~BQ5-1B?;h) z1E+3tbyYOuFy5*JEBNCnz?p0o0Jliuzx8mQ0AHu91L_X%%}bFUd>J&|EEjmb_Y$_@ zCch&Hl8fB6d^%OwiHMi#&Ig79SW<srbjRsdGamV)+q6DQ&@nd3VjcD5A%Ne#@oajo z^c9UL6c&?CZ9q(;9>;4U9)mz81xF+Kb88re`mW`ZUA@sHGRMQ47<eChTmx`IVmzKs zx=-IZ7kIfyC%AbB)E)}=G)1QwoQM+Sb&(h^BOY8{;G5&&Og{ek8KYE1sIE@!6R{7v z7NPS-cte-}rESiW*3W!wpPY5px(OSRc59h}iMZ!kcY@DWg|tbBH6^~KBB9-LUKZ_7 zUKchymV|c_>{t=h6j)ussLwI$-)_%H{(|WdRO?6ddFvg1_SQMw#zI=JGiX>&mx;?J z9JHU<V2`z~8jO<juojs$%TJ``3Y~B>c`glrg_RdMX}bZc@qHCORTgO243sSrfy@jx z`J)V<R0cH)qbS94z80hD;_J-E>%rv`KC)d$IPdTSskX7$<wMPQhD6E)Pyv?cSwat& z_nQrDPYP#XgI<o?w@fF`K*<?G&K8DujODOHRq-HRr4}Pxaf@B$DBQ7$ZS%>#;C+B0 za|zA}s$#=2X3%;JpS;BTVh}~Ny_r2zbOp#TB{Y+22$K7n%Y-F)x3ZeHcP(uuq>MwY zjdGA?@RkDlr(-Bfv9b;MpI7X40UI!(Xh?jsCj`GuV?zkI*7p?=&a?;>tT!O@-xa~h zF3WD1Pva5xt6iYbqbqexH_a(0_pX%6F!SA+`Pli2G+mLqS9%&-@ZMg-z_ujUM9Wj? zYQ$>`<KJv{c=K}WxL+(Bd}UD9A@T~n-{W*494tR!xZAVuHYRDiJ2FwUm1@27k%KSX zeRyn*^msexeddBz{z3Qi$@|Zg*7peW;yt$BFHt|ZuAheHwiw)NkPJBZQ%^-At$%VS z3%DRX2J$c5^y5(JG>vncOIzeY#c`6(MN)RHB0<t6B}f;N#AS<+pGaNgQr4N1yh}== zl=7&%lwbxK%Z{KLYYtVC4!7z#j_X~fQn~}58Vfh<212Y0-#7JCW-yem@~Z3ml2IR- zX)(VOyd5#L>za;>tWwQzg*mp+s=Bk<td{<A%n-IS7%wO8GuR%r#920ISF)@b-=%)8 zY-FDo!&MW&MR6gc)K8fkg8*g^3fN^2tl$LdbDOWCiSSYn4nmVR9JH#~i+zlUuCZX> zYX+JZuC9x-+_yp$8o>0rc!j#mk`PP%#g?#P_utiPulM-s>SA}>^>n){h~$C{DWUhV zNb8^~EuFJFfBVx--u0TG3zEggaNa;uiS?35kFl`<&EZkKMc)S>K<%epO#1%)E>&Y% zHRGcEXmD#H0wYABDXT)V;$r+N&Hktdu9GmHlEH&Erc4Sf$b~Zb`nTqJ*V|D93+{{I zZH!z~@Bv&i(evYoJQza{9r;#EJnS6JbfFIq=i5I&k3wD5W%9<yPYHoS>5zZsH$gx2 zdY;3J+-|48^>?6Z2Ds+I%^mf=xRG(~Qgn?lt51?-vh|MiTbZkJ)N_yoK(U3$x!<iT zCf5;58>oEN%WT6{;L`Fxy|U9e|ITgY6T8%v+|Uubd~zgcR=v606{?g0!NZ@SAh}8p zZhW|}(5M?uWAh}}suuuRHy4ZkW9F*2!uj;`BX?dQ)@C~Ki45tjic7`MOc0vgXEKHQ zDk=q1ZGK;b_f=!iW7`Ag4GONCUug*j&r0YGSjSlcs>s}dy_z0<1(=2RhDP)%kaXvl zw&^HTa+uW(S;cpKD75HvRlN5j3BE6lFs|oKk4FaVeR9_zTmy}!AaN`~m8E&CQFp55 zKxg`&YN?a<iZErWUwoU?lWgD5pBugmHpD&6L9ev5UW$wLi@?@1!C?9t73Qc6W}`0# zCt%Ni=Z-_yC{z)%VD}<g<;6MTO%8J7T{Bt7XBu>}ml{w?5cmDK&YUr-%9%CXIKg<; z;(u*`8cp7l+ho;0X0IjA5gqK8(YH;pY$IEhP>RS8bCn^q8C=M%{BsAcf9q-(LXGK7 zN#lG|sb2)Vk3AEzGxaxRI!cqFlZvvFABvIJmW_7tMmgMg{rX7ECz5H9KM)Y{VYh;F z3U{+JLvOw{rERRL4j|@kQQA-Lt+&;~52{|wm3s0G%}f9h4y%;hHE7pw#1Cq29-Mf* z<RB;n|J>}fU=;;h`&rknw}xMyX+xF0FLUc|@K{6?b9}u;Rv(W(?-+rzm5i4V1AX{z z*G?Q1Fy#}cLwC-1%7`mbrqLyfLTxXLX$78*7O)-d1Ft(?p2=eU^1~U)uHNVk9}#a! zT^XXzg;RG=HN}3QJCIQ#EG98Sv#I(b__)N<1YY%JY$cKLlSj`FS3T*{2bJeWo%3E6 z$5y^M&pRvL<!OCY798CI7cg+y&6$^oy2F)Beyd|4#AutF^b>aEEy~u+$47-i!BhJw zhPh&yryIqR>w~=;U2la<(wO@lUw+Z<vdM!4?iP>XjD+<bET~OE^K^m%QwpYZ!56`6 zh4XX$cY#kR9~^--?K1=SSWxO(Z(7PS2wIXfzZNns=^=xRB+_7bZX1=<Y{l!^7Lw6K zSFPMtlT`$xF<=CIYmp@!>{#M^1^DC0Fh&-~oBAqWE@GEx!pFFm{0S^Qp_A~bP4-%w zG_TkVIijyOuUc20_pe>6{EI0+HqmI|v|icGVTXM2KWy_Lef~MU`C$8z;_X`1k2=cc zlMkb8eDUA(o?~6N-GzvJUtkg57*cgt^;x6DGBfrXD+_9~=M<CY#RG_%?WT#kG>Rh) zQDQ!Su4pezN;1qRf9tiFa>Na=rMdd%u__xaMMUd}6uqKSNpt4oP4;M^ljxSuh86QL zoK_P6{pj@Ifa<rl$zaknzS|jwc*C@5HpUjmQKQ4qZ?(!o<ZSFGl>0QSyrGEdMTye! z#YZGs>v_kB1%|rvd08^w-W&{q<^FQCk?tsg9JuFWl>NY^(@B7Y|EcVmNX;Y4$Q&kO z!(=s{QFv26PkEheAL_$1nvJrnQx|6w!#kL^1&=Faw6sERHP&S{b^V_KeqJLclXfm% z6o1u~l&`vSWi@P<pamYZQDXISTB>qEyTyqjLi+I-s_Og}U?yP!5I%ELZGFHs%v)AE zMkM{w8v8|4!6j?TcrWX~(Am=HtfA&grS2q2^yDLvi^ci+91=x9+GA@y{fn16TKx(< zXts#=$$Ze2Swzy>XEtufGd9Ro-WV2q4k)`*3EVNkWaQl?azLziWKSTTQNgZBrY9KR zhpHo;y&qT2ZQIST)WH=Nls8x~;9NOudOTHGYHJu?KJM$dFm*uT%#(AX0=);hi%Uz@ z)%wuBtzK5VhCFLHKP}`fxu;iAU+_GWMH-WbsW(<rYCYw*$!%f~5K!+dy-uD*@pAF; zE<ES4Sy_&^FwuHGyK$a-_zZX!xILyG!rcj2Kf@u;8j@JuJd)uuFYVt*$}uKwD6CeK z;f+fqfKq+Z<ep2wwmnelyAueOuctiG0@l)q*maNgKin|%3y2;<Y3CMo6ame{CqB0b zOcI`aMu!dQrUVA12}H^@dFSoQM19k>)CPRFA?ooaV<E+eaqe9kfqCauPxh|2XH-FK zI^Z!tcSCYMjHM?;T3&>o&EZ_1i~1wl$j(`n7o0{&%9rot<KPfm2Lj9M@_r?Vc+b>< zd$%0k)#F0EkWf=t{7jSXk!R*14$bAAHUx(6IEx(c6Nwm^<RGYc_f|(j*^VCt<wPZ9 zgH|ehSb!Avp+F!k$Y^QjGJU-BfegbqZ>1L^qAaXZa%W=v;}l+&Z))UAans6J1pKq7 zWDId#+J?QGI?h-JA%T5g)lwaR$dVA>)xQBHwp3rJO=bRtesbk4UUxR5nYY@N7DK<P z>JRxMI1jSpbyhBnnA#R0Tj1FzsuK~%nqw~20E171gX^`<s9pZEU(gE|^tp1}yuz{c z{xe3=UP>hhe%B+fpoiq@jp_7cPD5mAYQLGupIGoz$BV40w1l{`5M7v=R3Jc3ORD0u z@lI5LH{Rn27mCE#zUjso_G~6`-^jhHOe51rYKwIKd9|-C^GF!n1Mr1`b2z|zyF<;0 zG?pT-rj=n~q}3^-;+t7_^=L$~-S+gdq$4(0+ReHDDkAB93qY?_72*SKy{ly*|MF$G z#Ra^hZjm%xj=M5<K=TSEyG4Jxe(MU~63ZB>CVQqRM$bfn!)f`>c3<@h?Rl3#rsQ%1 zm2)|R6g%7d;;LHsGfqyD=!IN0l3jw<LK!t%1mcr1_~LW)9L~;EYF3oK>1sRg`yCle zPy+5EJW^zeE^lh5nxw`}Rct#ZEZg#+rB^D1y{mH}eYcmcgCgFQPG0m{Y10lsHgx8U zz>qR?eb_Z0M4?q8`F^&nlHpk)t1bJvjxrr379Az+2`3_tq_s1_g<;pXXjq?_*9!lt z>th!@u=I}Atfhidt7AcrH9CuoY(w^gJ^ttD`M>~s8LcqH^MET$xnuJ1Z!1Dhy$q7& zu)T!R6;$`&>-8`5@k&>^>L&Nh!J}YsVsmk2Z{3O_jiJ0X%8^0#AEfiIdnr;+!J7iZ z4hcAzb{_)hS!jmxed>|HadUWE<98>fZ{ZzpQP!)%XE(+MN$j8>tmS(2i)1g4E9iJT zxd^<HHBzfqF@|ZrLqcohSukAdb6b#cL<^kVj6Ro<T`MX<dkv6XcWTo!W9k^o7nATg z+Eu5ofBa9UiNqfu{~F|y`vUR0u=z*f^n1i(1=86~R1Oua%ll;H<B8*V8MPb3&*+?f zS&tW49pVxm{%<P16_yMq#B5P}hBxg9-{&E(u_2SHh#5Om3cNtutSMne3&9S~1<+gb ziINrRvF)f|_?KShin%hyE^gOX1wX>*B${KYPU$V~+ukx`acGZ;JF1#i`)XIYDLmgP z0GU!oQF2s#XFm(IQD5nR?4Z2OhtIs?j_`6FpghmOP&JepT{<DDJaenwR=U#@9Yp6Y zdfqF1nCq<!9cO-en>$l9C^T%V&qCgoO>f94e?axlIGQ$33RkPEGR&|y2q9}&Mt|?w zXihZOZ$=<)Mfdn2bggrY)R1842}b5-tKh)|Ho;!Iq%WE<t1Qq<yFsig$06HvIgq)M ze1Y%n{ukPlHJehy4&JCr^^VK{-#93Q1JXEsasLOWp6Lh<b4EvFSRYlwkl-IW1q`;& zULh&z3>u7^YYo$npDs~D{eYuis8^`BR(x02r?CN2{YBka-MC^&y2EJVA5Yx8NQfI; zjC7qIsJ+O7=1&@8`Vs_Fu3`}cP95T>OH8y;9hIL=1hKOUI&8!Gg`U%m&hKeQPjQb# z<?}gAmR)fqE=#w>_O<ie@$H2|AuC2qD8C(hI)bCGJLS_1rm~T#j);3}=Xp<4{zVhs zdi~ObD3-3UW0N&H0(1@f&5Zme(qC5*dHfg3-NngH52D&cQjnmGO>>+gZ#rM73gLC% z3=7>Qo%njtdNz-4DPE~;Djr&3i78GKztCtkK<hxS<cA<iu&Yc~8q_YZuq1M4%h;*D z?`6h%%5PIrGvR4sA|iXg{22T=jk|9<`;MshqErEa)hB5p4l@Cg&UDh5-Ily*Lm}!o zr94-GLEJ1lt>2;bT0H{l6Y9@HJ50qmUleT+m+AqO>@T@vx~NF}4>ku#6eE_IhjX zxMsciZa2=@FgoLq9fhG1mCfm66t9Jgi8(vq%t?a?a7TP1qAi>HRQ=&|hne#DJ7e&I zt9a>}QOdiu0y$CK$lU9DFxR(cL2q*KuAgJSn6cC2-`IIdtpg`^bL%!2s85GDVm7BD z_<SQP&IGizx`dO}z1~L8%Q;;z*3@Noo;B*`N95^e3(Ro3oe}2P<K^oZ=Mrb3v9Fbk zkZXbn$<p1tMuj1w4ZV&<YfW1aQ8JKbHt8fxH~RY)U9uz2_jY`Cjjs>Pk}KXdndQ7I zb84!~wKri19<tIQ_&|AJV>%B<pwhYyNzh&w;TXM36qUf_-km_Dq_L=DNP2-Y@<~+C zSGc1yM(k0yBd21C1B-+G*pahxtsQ_pb@nBmVA+*{J5Io#8TmhX!GBcZRVciztXqA& za@K|gm^u@|*7?H|q`)TFncz;<V-T-=B_K?DYg!5!M;0jXm@R{w{hq(tz;=Ky-+EVA z4tZ_71$oOI_Pt#}V5BN%fT{|&+Y=8~;R7;fINkZhsDR~FA(}{#xVlp|&G%7YrC?%D zOQh5CgYcz=`TCA^>mBrZB~e^b>5<%GQH+-fCrUye?ebllUT*{p6*Cp<%0S7TA<_h% z3?~$Crjo}|{v*Vhx=dXaW%^Bcd>{$B#CE05LY$r}m0@Ec$d=+X71k;Ao;8|yT2#07 z?ihTru^BcyHBs)->vp}88d6)(qW9w}k$9P_i4IW>ewF9?41*1Ca#B3^CLyk@&m`4b zvCJzZ5_q)>^;t&pq80OGVHrpDk#~ffXUR%%Qu^7oM_VC>1P}Yi``=U(0*eJ~y(1}Y zY^CtlUCa+ERqzwU1m9<s;M=Qbq+Id>LmJsyW9^n7l$Q(5W_t|36jmjY1nEv4d5y&G zEPHWVbaa|(5Utx&0qsshtk15L7dBxFNGv{{YJ9W|fv>0RmKI}sUW|I%>*})$=FnGg zJcDN>VV92)J`Oxk{BSvG`fLYgkHd_E#eV}IIM1m(+u7-ww^|lsKtbCHhdmLvYok=M zq{}qjSm0|T#)Bh+W>1s4cGjSy6I8CPI5q3v!21y2)fknzoZZQTH6Y?vt<LCB9i@0E zq;Nb43sLn9|DbMqu~&AR_NTc1uiVJ$Hr%Vz_pZmjg;__2a70&&rN9C2K6`_6S0Lm* zpY7l_u+;)97r{lSWtYTf9T#ea{@m9W2S9Vv8V!9kO^q<0dasUh9L4<diFDEML(_5P z;o%fT+qLAh%sbX-*;a}v&zsHFdf$E`rvUB@Yt_f>tZ}giMp_gcB0==oBT~=WObn+O z$;cx~s9**ia-s;Ur_+`aiE1hmzfh$zi)|`um4_l?nbAV|UM2nPcLKy3blTd7x+zQd zf~~#|0R3a)iH@9r8D;rGp_vI9t-ArZ%WJ(ssWTX0=PkYD=gqTGUy&h5f}5Y%3=PsE zLQ`VTcb>p*K6*~qa;WTS>Avx&tXO3mk%2CT7r2+5@or%^&z~H`+I^*rQOjwIgAc?< z`x#S4pD?}Eh}R)xu6EcZ3+#bi3Y!J&TD->7ArVhpOZj(=r7gD7d!%HJ>s)i4ywj6D z5p)_IKUcZ=lLNn%&|AJ?-Rwx}73fD>C*`Fn468uHg5u*crLXR}>0gK!)6iiUvFX|C zmm(*K4D7ymQxypf!;n#+%Gof*Tw-JJLSSIew|OW0_DfB0@J+IsnjDlkBEb;1(=1OX z1h6AdsEu)*B(a(?`F&sc>LOWh(+@J-hA6@k#Qk3GuMkk{(~0<Qeupmf`PjgvF8Q0w zJl3|SnA<WVT&xN~vJpqr)X}@RxBk4*j(eli_%%@y^RP3Hb;Lfx`Qda`nohZ@T{}$? z!Q{hUYLpv>t+%TRIPYsYxC78VMtU@&{>n@d=&<aO0RGr=TQFQ;^D|$2^JvBGq8*x? zfynHmv*|phnaMeokvjCw%E}}e8B`7>|Bnk>AqjqmH@=(LC)T*yEiI-FipXLp?Ch?_ zJ2QmDd7s{OZEv-ksM&IuoZR_4kw`{HG{r$CXgss5xp^s~-Q}kREhfANxXR|A?Q1fV z&3m9N8!w$mOoA^i`<$+CHDIC#!%=4nMC9{&I@3IX#RPKwxZApFI5tt>eScU<_o?*9 zs-Mqmi~z>)+Qd4eUf`13G7<&>&I{3B#~5OUME+jD9T@J1uP0U`Sj0NA@wpHB+85*C zym#<sobXNn(H=G>+l^QF$It=^R%hxL9af@cp;Fcn<>U8|_!n-rr%<kfw^;-d=ZDU3 zKVUsK>pWa`8$Hr*kj)f-hIs~$sy|0N!+lP(MhRX!y>T`$t|CX<iK*|a-6ifj2+Y@@ zP(}_n%FK2<jBKiU*>`3d)kn;F5@L<E-u#x;w`Om7NHlH0`4mD>@8K_rz|4*j(}F5H z+1p1EjW`X-b8#1PQFs;U0k5k=k1-qOb-YJPJCY`%lM6E?%h=Kyx%WV#J~=72eP&<E z&TpE}-Fn5dW{pF>Cm%Lw5EbL?lxOcm<{dR9_#vAYBSs<+i!8j;XUEp}=Yd!cGD(B< zc-fa#Z_FPT_8*Mti30T=ZrvW64WD(@6aLB-GN{0(9p5L{ON9*pdvtlVUI97a<z$XC znp3N2)n$_#YN^kOZs9zJT9v$Wei-s@=Az|nM7owNaGs*g>7co7(Bp&cvvcnC`HOLm z&}X{?_H&+ee=@{^bL9po15WWN#-3A8V?@eGgQjCw-<R~1PrZD4r%;afZynD)X**D# zoCf$77w?GOiST@@anxAm)qdLS`9D5l*sR&eJ)hisLEuK8d)_0qljz;N-Fg|YG&_Di z04n1}ygZEp(CwRov__8ZgEVGurzf(ubc-lo5d2%Jo)UNq#TSFVPa>}~<{Zs1fBw9t ztw`Mju=*VHUSf5n?K!#|%Nu-u+vCoQK`pt$jT{TPnne2APEELNj8jjMAu-Pa%R||q z3dwbUIEFQX-_ane4<fq9TGSJeOxMsXBqprZL|wh})!KY9dK&i%FU#a{jM-<vaZtD< zI=mC78GV>FdM#&F8g9pyL_*^2M%ndAuA;$wOMr+vfMS;tiarXsG2$6YjWDRxPX)r~ z*fKjzeojms=EPz&OPGkN)V4-qo#%E%$5IMXCz{}IS0TKm|IdJ+AcSiK7&*Poxt9TW zGqvx_2c+f0KB99X4~`(YZ$Ca7dp6Vb@-&Z&4h1n+7!dT2H$6UzD7j=E;dIm^oy{g% zi4h5SkPL3%L^h~oJ&MGK+Re3oNVj&Rm$pPY{T6zVpbod_@Vt2<ppWQ#XXJ<PPS0vN zy&o8L*KI^7%S)FI6z1sMr_~-oWl(edO!P>hHB!Ed09Z|9W5$*YTz)nnycrZgN0(Z( zYR_{+Mi5oq67Z4le?Df+m|XiFJM(BPmwT<fY=&PjnWXMk-r$lCP50*c(pbjFdk^76 z?lp`|dk|yNvfj^HG3mf-M@kZ!JDl&>Dz~4=ktJRGplz}CbA&4(T}qz_;wr&$vAr5e z{Mc@DzPki(rnNMbf5*MJ9<O6(9GX0(z7!$t*05Qdp2kCi`JM&pLli}$N8@dSF$-Ka z<{27FOzkE2d$%h=vGbVJ&Lpb0Yi>ySY8LTHe@!Rw(}8ox3SiqjCLIgYU!US2Uql0T z7sU7^mOk6rPy!0e$wCUlJ%l2+xEQ*QLa3b2$cD9B#~#s~HjBVf>lY+;ildcA0}tMI zUR12a`J?v=drsKsAI80ZxriW;mTG-?m}rEy+4$JK$h(Dt_S63R)=~Zr{&jD!CHZMB zS8VxPMEqCuOCn1?I-iYZ6BrqErjO|`yelWm;>=k4PA)-BC^_{TCZRapg8k|L*f<*s zkxyITh#_+@s-Xrwe^;bgB;lJG5G|`#kY8NS@{k8;Zo5AtI#5{EEDIuIVad9A4sa{U zqrsIsOsSTgk0wBgCa(%p_(hi00jSlSqqi_<636$2_~d1alXoy~hDQFvI+i1q591cP zE9>q%H~oBn<yZ<kW^njE?2EN!eDc`1D;Ubc6OOIbt#Svjdi&b6L`JMMc-wC)_hqzd zKVxjIa;;K+i7TkNGwaQdD^_Q@3$LlqVVBP7zpjnss`RU$r3Z&ir$_Do*ivwAv>$lb zB7((s9-mG4kXJ*i5I3yd6#jI}4bxNi%d+~LhWLt}7rEC5h{gdv_izB#UtIftIe7jH z5+sPZE#BlCz`q0b$Ng5CSD1Pj0zB0J>igFV|Ak*=F-Lsw32=Xt!~R#r%PV3Sv28Ry zF7kg)4*WS+Q2+Tg=e^)*!^!gZ=!2u(=0Kt8)wuovO7ETj)uOB3-Vnc!Cd`t6z_UJ) z)bEBk75l|ZVincO{|$7*6V$F?>(dU9oWehtLSMsvh^H>r@kNvRC~KnJ*(C$}H*@s* z^2<@4zu1>*+XH?9N--A*&*yTANO*WFXe&^}MN2WVwybnNmmT+#_N&OU%=$JW?npOp zOaHKv;$XIxy&;g`Caej4!%*|N*DdErtT$n3_4e}eEFtb;l+i-r<_abgWGIB3NuB9D zMxedts*O7zi+5%-vWXU-vA>)ogsmrX8FjSxgL3ayC3vE1!xF6l3xfC_P@YG~A^bZ^ z2tvFD5noFSq+SAAm;A|xMQA{j{(iiLv*iopC33nEZsYGfsvH0YqDuyY$@iiC&-~Z* zRhcU9(ODQ+74~<l!9n{rIuyaJRdKGt6Zld`ggwlRnNDNN@blF_U2g0sj<($jE};z_ zyOWYEa^w1K8XNc`H$X`rMUYdDS+6L3Tpr8FYiAfeG7>Xq<2_?zA%blDgO<ID11^ny zUw+07dF!``Ze|j6nEqxL{=qI44N#woy&?uTB1RW&Vsh;v>dU!5eJJ@E7O}Hzw;9oE zM+jI7oxp-Hw<NaN#1dVvLijM5kRRTw7)}=xA0OVK`Uu`xaeUIK)Sv0H+%(SD`OJ|! z{T;rTFnwK8q}!-Q%_k+X2D;-GHozV-=DVKY10iNib*11}q#or^PY+z1)$eldnb+%O zv)(F0-Nq)6jHV4SQPGHROw&4}uRZm6<!#lKLg?oAL=Km`W|?+)ZyW50iruV|m|HP_ zOi9eXVE5FWNE<9DiLfC2hM)eqXWF5mzGv%z?Z_OStFQ&@rvn?JyX@Wk`_eUPNr)^f za~rm0iXnj@T^ucYz^htZ(NpK5Ye^Fcw8fJ_9ZYe8sixFi5J?2VYO&hkKZNLjOjH=& zeI5Po|K5KVVPz@O{tgoTOV6HzyXP))fGsa^*N_thN;Cs+1aLQjgU^0aBQh*G02t%; zW@Lk`&S24dXz=(hkf!~uwWf@|FZ2VULP*+4$KUYRuZ6QCebqSt(8Zl=ez=sz2ULs> zL<zuBAxUV2xDk}nCNlPncIB!fQpD1i?9C7!(h02NHH7k(4oRdH;UJ=uKm*s^HV01} zglqHds-)lxx3|ZpdMim#iAE1#_W1ajxs#YB(Z-iqY4+yp<JTnnMdHUqP3Pcx`ME!d z7yBBO8;73dy|QGTe@JD4R!4|j&Y&Qw0r?Nj$|wcUNaiy%i#r5|YFn;<D+<)zAXtrC zgH0WHz(kW4^+lOiw&}cq#dUKreZZTsaP{^wuzI^iZ1JOGN%`dmn4M9Onyi9jve7J@ zAzO&VE@UOz9t&tGQ_|7Yn4r}nX#}rQOO{^LQ2-?sp|U-Wcd6IpsL>7ye-0<d-iDk? z!x_N{@u-b}9W#q|mpn>tc5L@{+A6BTte$EL5dd5@MG{L|n|nhIbN*jstRJZ$%sHV4 zIY`O=UE&ksdV<>ZWnaNiO>_-^L!Y=frWL@NI<PNdPh{I0Q^HEDcD;B;9*L_HC7j1v z;x*RAXtc(s9ut+o1y@)X?ENi`?(HVo=WkrtaU&y^0Ogw8nk^|7bo!J~oERsx2neF^ z%UG$nOtcQ8T~?|mLeK|-_R2$<(Ky}JqK`UUC=*?Tr1`0ZI0lsb|Cz7&w-QU7SCw!w zi4TqoqEjC#<BVs-O27CT@@Wmx=6Ng44em?$j^JwW_agaaQFf{%>GcHch{0rt1J?P- zZvqIyl}*7gt#36Q(xu=GfhKpPIZV2qKF<~0L8gfp^B3WETm_YGx|=B&i$e~agCddA zXF(;r{))>X<oJg65M#L0?Md6HoL@0zR+c(&x~L?`+e#<7)?L(NV*Xa@2tp*(PV}ua zqWI<2U5_V6hn8$O2utS$QJ~0{h%e-RdRU(e3-5K)3_5VTL+{?e02wj_>c&b$?huiY zUaArxZKOi;sOEgxdJROhbi|Q|+I=))Xt#&M?!>nR^6t%8-<?h;AsX^Xuf~bIsaX<& zcFL?Wj2G3dMtXxfvcUpN#`r19Ry$1Lzvyhl*DZH4jt`zzI#Bn-N0Wo!k>d=E{24Fu zBX9w~A9mEp?r1;9jcNNHS{02;hQ(gyo_zr%pGZUv?Qsy?9Uz(E2T>fc$$s4UY3;`{ zOk0~QhYcCMlTgz!<2qXfPS&(F%!brXt$l;PE8!jwv&LKq`hEO$Jl4Do8P-NA13BoE zI@0{ZD)NV&f>aENa;Cy?N;*}LHbo_}{O){c9ELP=Wx5?X`8O_qGlVNyt$p1x1FsXS z%8z!ui~Tw}>d1zk+XPPwCs8ZEv6BAB>PdXnTw2fk6|n%`clPhd$s3+WSR<xMyE|zH zf<xh=NRz=AJ2Zb_1<mvsIy5oVzKj1Lam+=D1?)eQ(uB6?G1>E{?zKf#17CAPAxNK# ze2BvY#;R))OFe;+9r=9!_$}+L3_P5w+0&V9(~sO1f(%eGN<*AGuMILfoz4A4dd-G_ z%r<j$Jh=|$7iLq2Gill6@Mj9cuppJ`*zCw)sD+b&Gt%vxs*(B@h1kEJ$EqPj<;!FF z3xDakju+I;GCV}tL3k*u>8obq8mh?`7+S3QbQP+OAZl(FHoR!-BIn&)!iRiRxQQ>} z`59{zGbQQ6DzCUx_gp~auAsp5S@q&uh_<I^>XC)bPS6q#yTorzpjX9@4X(+2vU~Tq zD7@SKWYE!++Ie(m2e1T$T;~t{+<Bi@R0}fQ>)9|u2^Cs$lMVcPX4BV&NSxZH?DqIC z+l8V+|J5=A%);$B{g)M$d<BIOyYqjb{*P9#oBaQn;n#+G-8g6n9D4Gff4O_F-aB1f zoGdr#Spy~W<zZ9w>{BM9_;FNpm-Hvf1Pt6uk9D{A^HZfZKdM?Or>tbth?fTpc0Pq_ zF8u&()zI~puSj`|tKDrmYZO!DsXdT9M0G^sEsFleHK>mPBkZd`$*>cBMM4kG-?)yZ z+e<S(tO4ha6Du$_BUZMRCU|IAw=JSTDPA_8!UtPjSVN&;!cb%8h3;MYup~>cqurBS z7HN7Xfl^_~X1J+K>RbCr@+4Uqq!YN`xugwrP&OYs8t5=M<$u{?4K@&!iN?xBnoZ-D z^(Hv8enxf_#oq}Ob|{$526jUhJHF|v-S^K%)%un;w6|aIl9M67fo!NLU-@#R<D0NU z)^M2UGg6tsBYFHEb<4kg`g6UeBr?%4isOD2b0@)qHmIsPs~?Qc)MIYMz4v0q<K;j@ z&lSV!uSZFw{G~K_LE4BYj%!dk1xZ7av>frRZ{yh0lSC1jaiS%MDoylbI;liUa>E8s z#YHOlFw6NK(=iadjmf-cQ7n%t&`a}sO;<JRhDcCqvyHN7$}8Ol2y6{viBuGOiR5nt z_Vx!xL%K*8n+x&L<Iua=-JJjk)GR&6^W)YtCNC9)xTc!Z^&cri8&JQrBGfG<kaGdW z<#GqBYCp!ylKEe3_M36W-iLY@W~Gb!bDWe*YCaISeCp9moZ-lg4^ddG-ZHysaRL!b z5N-Ut39v-Yz5b+!7Z<nBtg0Qvv;*#FPDnfus-NNG$&8CcAhpGqbu#2LE_1HW00>MF zu|KC%>Mz*b8k~tkJ$?5V52c+9^{d|xNz-)4*F@oKmYARySYK(3lw+*sA%dS%yz*2r zfuY4dzv@h;oan_JUkP@`l`8ttrud!6ZoNH-DEBPbDX4PSDpftQArjBpKiNu}>`Rgp zEft5T<@?}66JwHQluy|_5k2Ock>uRpt?nhmmMmplcO~iLlF))WrNU%#ViQFxR;0J` zQlg#*PJ2@boN||9fU^m2${r6ti5dr6NwpFTCjn(}-9dAk**`oA#;v4?yge1F#(p3J zL4{W^@goF%dMBi8B7V;<)ifU<;3hRzeCX#~%T3zYQyJwMq&4e^Y^BDSUd@6{Q3^eH z+PfSt=YzPCfJ*qMr37x!MgXrzfW4e68U6}iJSfXo>Mi>WF@N&Fq~J!?4<Ehp%K9n< zN}kVy9d|_N+NQw7iHnw8V%R$_QQuA0o3oG$Q*`slf5fBy$Ex|mro)O9yd0f`PUkL= z`L#LUOSa>8GH3Wp;wBhVg#s?2yWQBTOkj742pMxs^SsYy@udo;nB&mTROI33a*fwW z2<trkW7G);GZC9~=-YQBg`wAswUz8xZe)={g1vMw9|$-z_$rnBxl5I6Sp3}%rR%4# z7Op6;Bof8J=h_wVN(~gM#CflCV;Z9To1!<&-(kcOXADbu%_d_IRBDDCg<Pss_B6<J zBNma4-lRbpPdv+U#jTz4Ex*X{3+|Q*;^+cIVgZbUv_D#opNarPUC!DfggREXSI^`P zUiL_|cB{gIjLR3@Mlka0uH`pTvk$9heEkI^BV{&%g3B{&-`r4+NQrzM^};>)GyDmD z%(5SEC@q(-b++it&4;*^$m*CA{Xm<lq`Rq)ayy*Une)ApPm2$PZ9ziiem97$mOCwb zx6ZswclAWb)vo=KR9KBvct`CcrH-%V{rk_jj49=B)oQ{gcdjAxG%1fcK}Imx<ak#= z?&zc6L?0u>NDSSzO2QoLehFl|38;{zd^Rl<FH}|ovRKkS%8Cjdq=?<52(DX0F*Ng) zD;_+=?y&A*GEb@iVHj+M@Ufske<U6T;5hm2&&=8H<&<HP@XGY`X)rD0!JJ1AT2I6G ze5jL^o`mE?2OOh&G6nAw4SY#-$+mbb30F!>5S}zpNJ4wmt->sN4dTUE(nA%a+@N7M z8GvI9LnloU+;QwM8rakLT4e&23A#`1z}|V;Cw(<di@_eW!0szmdb97-Mhcptt!^dG zNGs2jgsd+?i5%}yw6ayj9n3&RrDMrUGt`C$%G!kLQxY}VHaq8mioC~z&4Rof&4IS5 z=OHC9>-HO^03rm?deutUB@bU^a&c8tTC#kzt!9*I)%n+4HBwRWmdwj_nOQx|<|xM^ zd{jUc0kcMyz{-B!(|3jW#~rYfIM?#!fzS;zzYv$6zeWoxB%v-94F6CJ{s&ZGmFiFJ zY4r;ycwxwf-zn{ZwR7}XV<O9Tq@NRQzQ$6mDj{8V^hwqPR|v8pVXd=jMb09?Fwa*T z(^Lvl5FrJ$c6MEkflyZ6JT*04R(X&2jzvj?U%8N8xQ0*1?hz%&q{{En1UNWH&{~6( zQS)RY@y3o#&s2&elbooxgoFbB>M{ZEy{S;hm%XosQA;#rRL0sQNcHl2xeSy2BD-=V z$08A?H}YFJS%N?H#20~-QowrBV0B(j+Xk@M(1Qjzi3=6x9g+HWc3m1+6~hYxWpSZL zQRrwhZ#)LK6LlewIGI{vl(tg=Kk~BebMtY)-8!F}l)zf^h*0wNUUt@TQu$vE<yN0x zk<(|#A>3D2+6x?_-wF;DUDDN$mJ8NdtNK)1ZyqIp{aHw?tzuT4%@zj}7)7$1_GozP z{bBf8XJ`%l6c_oY9oq36hUgNGsH_f9jx|96yX|udDw6YAOT})En`8%0{Xx{5@$PlY zu^i2vKG;zd^mLRW2H2v$XE`$J_i&Uf34p$@W06GXQlqN8)7_UUyhNl$8%^++GllWe zQ23|%P^v1N3|~yW@-TZz(V$k0vIBT*-ciHp*dy<dy;gC~e4hz8g_mHi3LiYsa(Hx5 z$T_clzuzxKO*LpDnxXP>OSgrAxCb*fbkS_h>cMpQ1b+1OqznQuv;7UeypywpgymN? zF7a{DxwIV!`r^tmV?_b7rQ}=uM4OLm<I4NCgntVv{1H7{q=YDV2K|r`*-`&R(ba1- zJBTyZgbO2>9P+4+F$<8u6f~+-c6ZgIm+`RVG$4w;Z)?IQQSfrL&k@ERlU6(bf~%v+ z9JoLz50`47O5LX=Z;}(SsSzLeO?~O-?oR^jKped5@)#n5{?{t&@hS=}dKnE7ucf%z zu2!c7xxEw|pgAt<;e`>AtvK&TVdj2@r3l1AdNFh@aAquK@-*C1d#t<%nK*9Me$y+p zT)GycO>s+1dbV4AcO}Y}CJLYX;HXE7r80Wny~*Xl*Rn3$H@%B1v<gU_tj&&O#7N81 z6Dq*L@#!~pb$QsQHK$%LVAd%B7z8jZyN+xy>;}Mhk*?2yXU{kBlad(pXs`g*fvt;{ zlB}@Ukry(Y+(+^MwEhs}{g5`HeIf1ce`y|(_TWBlVO={2GONZ!Q7<TmSkx;N32jjp zyS8~?`Y8!p48=P*`V6%iHrH9P%gPhPj|OUQyLSfM`bdd5HGnCtc3h_-5u>t_z3ck) z_OM^Oy$Di=h#cZ3sBUJfVeCkZI@(Y{#vp^lh}jqyLp%0@$;`;Hs8|UCWgWBulgill z4`V60a)Q;zx0epA>u1K=aGaf4J_+&s^z-j@CKcm4;2yL2{dgLv^VqEh)wULT-9>|U zJ3~pcG7nwxl=c|=J@`rQrl`;5z=zt&0`1JKdyYvC@M1df+&kbb`<A7IhybRjPal$^ z49U@8)?W6#Z<-hzPd>*+eEEEI6>ykwnHy<uJdl1)G11E@sx%kLrEvX~ycSZ1c8k`b zJmM20Bxe_TAtIL2q!1#M<w}_Jm?mTTiP#h3=AjsJo2t)W^Cjw{5Ig&55wL-dRKG0I z19B$eO-L-x33Yf7x!I=3o19$K7<L|<<f%;iR&Kj;m&%4O`_)oyNKgjIxV=X<DVBa5 zBV*>FkSiVZ9T09CXGYmAQn*shQ|Lz{@GL3pXPrXEV$BDD9#gDcR?PMI9?n+<$u-_J zs{$f&a$-z@586N!9Y$1<iBt;d6MWu;z~q1%du%eI%h|v)IJKvD`5daH<!o;|*(>ZW zR3rNI<wvorTw*AE)&79IzU)W^-9!1c9P3p*kJ5?YX>W<@<S^bD&c(^()>bBQ=AoQe ztl!*PCY!8J7-im+bB!-|{5L9Myl1YF#EJ&Op30TouxaUh*4LwCygi)afpr8a{PLSy z;Oh*Eq?szNZdmKgBM{q4o+mM^8P!cAfV7kDr=Zf)xZk9@zCu6#O8Bdokf8s=UBHC$ zd3aK=U-NpRg%1c~K<_*VgE3HR`-C;gd{8W;&tE<;v{6=dR>jZcmx8puXm8lI{hZnS zLK_%1DVDGkwi&0Hb<Gx~y)Fw8Z~%s|5WN2AlyGMV*Vuz%{Q%re(V%3=bHHO^VT53t zSDx#cVDO$VR%$d}9Nvi{C)n9*wBq$4GoeUgbI~1W-Dc8bSsA>)tI;?$`%75azuf;d z;oTZi-N*GRpB8N(P_&j<k0)?NTBX)F509((88Jgh{m8>N<_F)hxqpfbWPDA3)X7`# z)-Y<JpsZh*M9J3P+5^%))62TeaT?*dzP-6M5qaIuG5CBjWUn0c=XRS0%hb7U$(^3@ zKlz3Bfv4J(8+Vr9s2ASXpkdo1*DCBk1B<Jgu!n66y#YQ;j_dEs@(_NLMEcLTPOBjV zaIYc6<ZYXwU<Te_9R^jnS9FR%>RG>8(tnP+w1WQSf4&9qdKft9HOvRiHI_m37hwM; z>a~+z-zr8k^8U~H|2sL<QC`(2_NJ&~+Allq@0y+J>s#g0kyn2u&_5CU6`K%xJ;<aR zi#}}rUxM1P?W(^j-)Nbq|0)Ik7cYNkQbBNcA$hifzlqfSfvFnhYi<93C*1#cVhOGn z!Dy>_k6suv5riK!u;6j!l-Gg`nWZS!4Z9hn3Y505oUi&_{&&8A_D=q|zfFBQ2fg{M zPtvk1m@&4y_(52WBaeYFj2bO|IG-{>K1(Z$=f6WJ%W5}8j}27}Uid;p&^zLW^^P-g z%DP0aG9@d2&ApXeL#N*(5`Oq-TC!=W72{lE%;+B<a`+uGe+*=k*=~v&t92BOcn`^n zAC=#?RfWwR{td5^JGog$&&`HoA$}w`PrYx27x((_*8d;(5#(3Jp^b;x%ie4Dh?UIs z&=Qzdr!qS%TpQ8H_wN~q)>}FS`&@Z=lT#F<lpb$NH}-jLGE4FQ^_29#t@8o(>W;X! zZ}X=QKNNXC&e%6<r=i8v%Je#m6802C*i3kj<#blpWm}t<1lJKV{=hEn-T7N4{RfgW zBC9=6n=S>GLAp9nwuMR@T)hRM8M0JcH4o|syBtfdO=cgf>M+Cm=3mRr{*`|J%|-h# zN&_mm1|bNKJ-ZS!O|2xRN$wkU4yBC-YjrGpmGF&N1W~cInO0s7v0k?KF30b}H-DNa zQ|Y*_$O9hT7+gk2x1rB~`=S|#0IiGg!lY4Gj=Zt8R>XBTO++l>uk)_Q5a0@@$XS+p zmk37szbByoAgpY;uB*lo*Y$y+Yvw^JY`6+7JHN0_H=ay3(z=FN%mkW_F`%TNR|Q1} z)tS<AVLsD(;gdg~&es!<aKL_sIo+rJ!?>)%yD@y;QOw2?V$3f;yW9x`nl3B_6uFjz z?R6hOb0lyXGJ?JyuLcSJiy)UP#sp6%vz+Y~-S0)pDV%n*-y$)h++a1-(swWdKJfj{ zgKLjZ%2WxN@1jSzKR%eTS%K6MO+NaL@o#$`OCK~aLhd$HheLNXDbh&N`aCIBy|<Do zHJ73Pl3>BiTI^YE{j*5Ma#yNIJz0s{$EF3)(nVOa)nKx6Z|`F+f9T5$8$?zb-dkT9 zWCY*qw$Rgi_M4@A^$Rp0X7CKMxw^|XQpAowF+{sAtH5++Q=qFWBJ16IE$4Xe26)Wx z?0IduFQk+eph+vPdPHk<sC~U_u;Y2JAfG#y>Kkd`_?fRrYbNpZEFN;Z#6^nkecMA> z<i)mBMW`61om!9vW;}V8UxY^!r|s{^<F5==Hsw#iXofAW_WgS?*&+RnofE;UoH*xN zaV{QZKVHvSZzv1v|4DCrR2*}QaMT>zr{grM{UGeCU-T7eC9L(>z0N7kdT$|fwx!OA zbuhlU<;`m2wnsveaP<2*m<1~TW<e6CTZa-?f3RN3KMC{tbU`ZJMC}~<u-WdW{-ez9 zt4N#8x4AqPk}KvqjY5M}`jU~OKOmn6qL$7fFUx${b(?;UmuX^!O{mTdDZePsj!r!9 zlsJ|`R6%yA%W(Jc6Lrd21uc1vz_wIV<r`MC#)*tIPVED2codwbITqA86~wkh$t>@c zz_|`RKN1sHAJw1(utdDX8Y!0TyF|4wxxM?7yv7zBL)3@L0d;|mt1zq~1RI>|jN!o< zL1;BR``<>|C(l-GrinY9IH(&Z_mo!$DrVy4eS_-QL^v5!y@;;l3#ZB$bSm~2qV46h z&Vq~`x%w#z4(0-p)`i4bScoxDF3?|3_p*nOOLvQ85$?Su=+afzoSOU6zfFm+Qp22I zMwlSXO#RJ9f;%t{q@<xiYYf75$y=&@;%B-sre&C$m<V@yl{OV5P{-j9oF3fA>nYzh zb{zk-w6~gYiSak0(8gF<e&M?V6ecEl=DsQTG6eTwX@rDkAC@J6MUJjcGPFHk1-hfu zZ45@fHA>W)Wo*B33=StAe1_NYsw;X=dtuIaW-<p7*~Bi0^CZCXE?fA-kYC`eVy24( zwSSs2dV|yJg~nnQ0}{yyCdUoq(c$9n==6^ha-02@b6a^N?OaH5{FRcr4>A*D(r<RL zHr<eXuz{eT<vPUMTpW#s7gwr&4n;#VV8ea=pq4a8Ld%JSrXK$-u6R}4!!WuzIlx`% zCmz)8_Nge#Rux7(t*z&{?eyrRGHRRAd(b;IS-xJPu&6pbXM9J5dC<2dOR{nDd+<%c z$YW_^_=MxT$k(rvwCgw!hZbxzMspGH{Q1QGswL&uPUh~7Ia9Sp#HD3zd1LmALO0%k z{AwC8F$Z=`Dt)IKUQqi8X&Qj6Al*&C3-KWxr`Ng0{gvAYAHj4fU(~VN>9Ec>8oyUJ zOdV&?k;KrwQRr;#Y*jdEJD#!MWJD0v(Ls>eTIAR6rY-1+H!n<d$iXXLvX%K3g-osv zym|wUZ==KT5ifel5G$>eyyc^E6N>SXyuuHsjzi=&jl#&pwaX|GGMlLd7C*&>leu`h z+)mnU=FEO+uB>iOao~(us#}ynF@p<UeYO6FRz+U<_C4)sAPV-eHOMaI8qd1FERo{) zI?#i;KQTOh=Fa_l=o1q~SKMJlceax?$Vf(K=$%v#pZRoGqe(7AFMHXVR2wwR)#-k{ zxU5_>Kyo!2y<t#(xtaHx=ecg}^iMXHJ{5RWsXkYh-+3zafUptWr27w`*iNbZW+T)l z^tKcFNmOw~TiA2?w_zrOHSYN)C4Bhu6=Ix%vQN07Vp+wWYBx&*#1<`@m`GSLRID%p z>UV}j>y^i0Yeh&lp|JZwG56zLr4St3`IQ)rN>UD~xw+)WMvlNF0)#Qeyu8>O_Gi|5 zRnP=5rVpMmBfBLiY=$Ys5ViqNj9CyWBw3x7I?3ye^C@p;JQ=|lzcpvNDrt{;ykZMF z$~Oey1k)AMn<b8F{grW+p3D*8)@<vAX1&nqbuZdy`t9V@jPcZAJ90R$E2y09mvWz~ z43a-<<aRkU{q3`Er=2bIz+HQBy~c*U@=xU>S~~&?8piS$r;Qe0UMFv{kX8I#%N@)9 zenyNfkTDY*Gx%G=2ZM&0lXQ>!5ZV0rv8C-sn**1%A+gMgy3}D~3iPl>PDd~fvzE~h zKTy}{YRJ7NdR1P}OdO(YSW6MQgFgAUDRRR5Feaoq@&?UvSLL|}aKiTu?i-I7S1Td1 z#P)_#*va_U5o3^~XvAv>__|lIGVX~Efn*yPgZATm^|@J(G<$$?1q2CLi=#c{1Ts?Y z@1Q?BNoWGwnYc#$58TGDH#N|VkZRZB;D6Fo8e7n(Jg~=|L>_MX*fN*z-yz<9f!d}< zi7-Fs<X1j5IuJ;InZ++GBqqLiYFig?GX3^j6R1A4y%xUDUS_GQ4An<L!QStCIFf@W zyM!3DK~&~0Nh@hEV)8_{RiIy-1aDx};cJFoRdK|Kxz))-6J_d2v*u*<gLs!9MsP!p z@JYo)>?cv0sMqRZ3Sr6+|DPpBgBW>fm)K+YyE(3kVFVy=r%O7dn`5Q30nOo|2yy@U z!^l``BM>p{RCUnRccvg!83haV2Bx6U5C-}hY>A-|!cT;u50@XTdxXv?Ev&+=5N^7k zz=HsHuX@T33H&KcT2S!g#7mzDEF$CTB0TC{#+d`_pSj@=&!*iFwc#W9-2db2D}dr! zwrB&vAuzbR1$TE39^BpC-Gh5@cMtCFGPneHm*5_pPm+7@d;h)nf7R5~%&D5`K704s zvU=~;O}>cFM@>;A{Wj|T@?*@-8xqPO?1xfwAz{K>b#Z9=F#6Pr%yYSdwcaJAY9c!z zXnxb9JCn|pUS{{y@P;AfC;Ze=#65&TWg9Z*oC$QFD6qqpqnM*M_E@Y1cW@9MwoFHb zp<(Cs%J6V{-cvKrp9(z9sluxI^%9R6?%f&pPEi+a(b+P6X`qFsW;EsSy7GKS$dwkE zY3y<=GiQAaKTs%3V3;%lE)Jnz<==lHQ-}tfmY623NWlcm;5OCBPUtS+OMah*yg+^1 zYeNBX!aur6%nyX+1AGl`VrJ>tFAU%RAE5g$y`L9A3r{2Qx>0X_ac<9~d->+7X~V2E z9j{+aPx{u{+~Ip1o^~iRM@IwAt}7-Q&rIiUWj7tvP+(=gtRD2<XLq#9ruJf~vzvkY zcyP$>V5XiV%ijTuR?DY^U6@rkmnaCA+7LYgBnR)i)lHSFmZZAg8xoT<vU3HRcPWN$ zGZ|+MP{@gN?QL*WsumR|p$<8HDZzE(WDF@YG5NU!l{l}?AWn{mhhNr#DXA;hHYpC5 z(}o!WtcGoyGT3XN?PEiX;Atg3(WPEOeha|6aKJWR#L`;~kusZE4Fyz<{~y6Foiu^D z^z%-2{-&o@cuS<%0K<xTqjX(>sGO#<Ah5b6eu4^X`K@aIIxuXoKwmD%mc(MJqq26P zJP4;=X)`c{sbIU0M1c}UKOm=@=nw>fIHa$98~KOOSEtSRIgLr_3rh~`>w>SMtQTEz zAWJyT+n`?a=NBxtRH-nV_%7W)hdy)vn5&n?eXrSXZZH@7wqg6T2iV21Xdy7@Ji5?= zp#f)oA_tJvGqR8EW>4=DIVODHyazX0e$A@S+QZnF%yl8LL5|P}5gnSo6FcY=q`G3m zYWQxG4>3GqSJA2Cjc9>glAjOqdS9JSe$)uxQ|LPaFPDhB`&(~B@XK*`qo<^vgGNKr zH7&B3%!@m%$B72-21n4(5{IRyy%K7Zi?rUOky9J4+>-y*PC#Ot2943a)dErcR(Gdc z-3kGwJx1E`iCeko;&UFx=fotcJ;|)nNA?QkHixi=607;6m#noV6AUxEaYFn?4n^l{ z(<TQNtlYLX5^!q3eo}tCL^r5?u>~=$VE;Bd8VFpeE5HD`9|^7ED*6JjoC8{p7|jqY zSg;d5B@kYOub)H3zJO?qWr3fXa!B-^6D67^1UO>U=aJkJl3$FrVK}3b+6*VR{!#AD zVx-0G3NFornZU+Emrb_tXCp=%sD7nG*48+;<po3`MiBF|WG3$nlMWR=HOBa~!7L&} z*72cDqjtXST91<Wt{C(wC9fEa5daY)!+;b!>N~btQgrM;`fFQNx2n%o;ArCm@zY%F zh+-Ig4$}udopZQ}$yx+JWc@c}K@IBnnY68;dCy)Alb}<Tn5<^8_I_Y0Di$E%@!XMr zj)CgSHOM*Vg^p^)<%n;QHLn1Ot8Sz2r(!lPfRWb{(5A>GP(+y%x%P}i&86wn_rria ztK9kOlkP8Uh=wWr$e;>W9ztnY5MHxo(W1}}j0IOyOR<#L8>xxSYIB|nwxf%B%s7ry zl?aohpfH^YvJ(+X|3c44qt4$)ODE1NPmXd1%YhR-@N7U?lM6C#0z-=wPr2v1?#5FZ z%>-%p9{MXkKfjOn%~F*2Jr<RBkYg6iP<MX@$d>sv*?Wgg3ii_q9^qKW)1}D?+_R!_ z@d=*)l@PBugU-_u00|?$-Ia#7nHAbqW5{vIP7UTOe8+(+STvWU1*EmMcOa@=HDBdX z*QL<YaRjo~P0(o&+@)8~n`QnmYwKuo&oEH3`&omefcO||UVB)K;XR}mSFwL!AfGbv z22$jLIli@rBdY9Idzb90R?__lWN{*Vv_k{L$dOA8hT={9KXQIs1u$!2HyXWeZrI!k zQMux@yIgt7dnb+ZsDmWhr9#+Kk<n%o_<7cPtCKuccAacq2LiTV>0uWQsctKlQH~!| zOaT>7ACUjkI1wm$R)ZAt_%$wAqaz~@+W&MtDnc7|BDd;fg%!=9>A*2j$sk({iy<8V zQ;bMZwfC5UA~*c<3~%lf`W4Ou=Pz4rs{ws2*5uGFtda}{>yu1|4UCh5fi+goRQ1A! zRau+E9ObBxf_6JG6HlWGRMoAU@BfmQ!GQq(4OOBhRW`hU<Dm(;Uj*m7Ub|a8>*rR@ z5(#)SNq@gUW6dAV=HHtn!=H^V2kR!pu~HV*Mg9HiWU#)oRqov<CJxIB)R?_`BfBui z%e2y-++&|}G3*oj2P)p|0yCzia;Qbpr|q!4+UO2dJatByjOPB^>^33!VIzKZ0xQX< zrB?|kZL$K>LHMb)f7J||(T^Uf$$}DDUS+i$GEiVI7poc&9rz2o>_7vB7j$|gj?*P< ziQ2sb#pHeGl)|H>N|z|Y>uE{Cv)rnYG8nS{^|HmYRl9FRCU{%X$iaeSbs)5DsZ`6A z6P60QCI>bk(@e<F%HqAgqa<&eJNXMz={`$BQn}4se(VP{fk?&Din3v^km9k2nB$Jl z76Kl#Z`A9aF0wUy^YNt4KZRdb?&%l37HcnqJR9X=^EaeHG*_3n+`IsY3b3V@W1KZN z(<VL$6;>7gLJGG!CnNXjuG)CM0%j2Yv-M6;Q#>WOPo%*`Sc7FeUxBkY_AR&^Za0Ea zu-KQ`>R7M~ab{Y(>DF86XeN!-ZflQ3G|THe@!ZNOILz=n{n|o|2_7|W2(YbW$aijt z_dhEq$a*)K{w-znV*;z8vKz`rtS@;j=zE<%>7W|Rlhyk1-fHu7=)1g>sI*wiD$}a{ z+tb^HOJaLVH(`+;KpUkJG`Z}HtQNQ~nn%&Hkzw5_f!U)s7kH(LzNa7{vMlwPy8}k> z7fC@AF_>dDH!M-j9v*SCzUS0fuV6;YoiWT8HN-=EI3jy|qMESr#?Gc#YQC>oluZ^p zZQTP%v883h(TjCYFI%gl>e$hlpXzO=&Z3j^Yf0VC8_Raq+Tx!Twu;Q^Y-Kw<ny5=0 zAVK#+<KtCW?M$RZU!IO&a&6cTN8c!W3Sokss=8|nKB%0x^@r4{(#$3;Zf`B(azupy zQxa(0+LYS}^0YbPjUI!`+B|M0q9z<Q6N-E^G9uo}Yhr{ssVt&1CH7Vw7I{(>Kat1U z4juJP0+$**HJWs*S2)Ih9QN86mWO1YT@5!|<=Y`*hWF@y{9!4R*hrwNLTkF;cH5ls zF9aPemS0K&0jX((fzhp4$ZjPeall6(;pfcr1vGfLpUGtv4u9_t=d(V1>MZ(2<_PMS zx3)v-*vQTwqGPGY^N=RX6Y)kz+5#(kh_~UE*^#P^gl95Dg)y~atD7_+<WDgMS-pmC z4ni>W(ix4f@UkzZ7!Q#!gA$(t*9r>4>XuF}dfZp7r5rD~oG9MnleBvc<}V7X15w+9 zfZW;vrDEGM5%hg3G-f7_r&4Zvz;FN4rNYn8clxG3Cn6y#R{e}u%uw7GCJ1tSMp|%B zxB>mB>)?tA>*v=YdhHOPcef9bhBOB+Z=$;g0hY$+a^(W~^VO~kM{)0`frND(T&aw) z-Qel13)9J%xI6>5N<_#IJJnL!5qXX?Abwx0yvXyE4c)How}itw*W<|iRntMalSyG# z6awAKviJf(le<Eo(fw)6d&GNvm1{-xgb(+6rPSLx6?vI$&)am}uh9FH3FTt;ow2y{ zxHnjra57tVe%ox6cpkQr^A;>6xMr3j+RBZI1z7eWMTS?8kl&vT*pSTRVr{%b!hpJ; zv*<teZnndS*$wP`>MLlwFOP7iGklfG2e)sA?dYQkGEk5)|6q2#&sP*UCZ@1;F~cE3 zGJrO21`b;PGBE$MA7QzHdpRXt?$rlkA?4$sC@BW4w);5Lt4|jK;NKltmlP=U)4F)S zfz6Ki(YpbZSvJ^2MU`!&i7p;Z$Na@qw*yb?i=UzQolAFomRGpWY!(Ftiru(w`*7X| zj>fQqKia5K!x<fyMfquSBT-%q<w4x8(rk<<%+flE_ZoB3nS3>BIeI9tf$1)?k(NrQ zH1|GYA66ZbGdRD{@zMERtT(YOmcw)y^{K%TK>Ylwz|n*6{N7XWhl;A8ST${wOACoD zmpQG6)BfcT=RZUNZv~AXb(BYSN>Tif<0>v@`V@*8gZABHsM|glCO9&*gYd${u`41| zjj}N15#?nbeS#AhN^wM5t_SH;;}`D0mf4BHdA$)_6G`uWb{4w5wg89UWm(U`3HG=Z zxaW87eWej#)}oS02R=V^gi?Q2F1+Tt^p@=$zC1y|j+LW9B)S}q`@rGJfvCflC7ceF z(OKhNPC;^Bc7htZ`(|k+)OV9nPf28hYK~zrIS-Bi-p9rL#X)r<49#RRcAoB*Xsf*Q zx$Ue7cg}w<huepVMw@SeIx+)@xAcm7A}axDjrjb{x3^`2(62bGJeGU?YMY(%rwS>$ zdHLGswbxs72}2v4lW=b<uSdm1DO!8M#50@0mvCsHxQ{EV|F-77A|=3!T<|`nxgkwA zs&csSDdghlEAl<ec>iXK17dHWX=jA($S$PYmG~5~fP&}0Vepm7@2s1c`f;1ys(vb$ zD(VREk|ev=Der_x<-X)(o2RrA*M|l3DL#l1V}RWxkV+y@i*L+zT)g^tRLrc6+0$vr z3#=;IVrPq{!%vQrht6;YXvlj`5Wm<w@3q4LU)g56IYKnJfO6dDuY%XnCz9`vnn?&) z)AYMdQ@Yk&UrH<YT2o3p)Q~WrI<*U5bad7Pzw^^8WDTh>om3hVZQkEgBTMu;mFZ9- zAwV=N5HE)eE!~@Ks)}rrSSLsekwEwFPB4CQ{#_<CFH8`n^MpwaFeh{;dcZ8|1Z)o$ zS5>WFX=1_!8+X2lQBqSvHJ)d|MAm%TcGc39^_f4cKCE)Qz;9tjo12DpR31%4yBxND zq+7auG`nC=FLmQyN_bvDUSt+zK+`X;2q#!p9)Ykvfgu$va^a@IOT*f13~E~tA3*?w z-6UynD}0qc=sKI>NmEOTl}oIr4LHs)yHITmD(GnR7uoGd_y$cMCHZ5G^TmPbBn18_ zD0<{)qfzZ~_;`!eLJ9HJ-IhU*!OcZ?T#-Cp^hDhFP)D1$3#C7%pgTz8`&7^i7aIxg zTeipbqf320D!x^^w4YK~v^I!VlISb2fv&QtXNCRSrm!A#Ym%onX=Hmu5H5XK&K=Rp z^b;}uPq&^?_T3ybmdXQ&SedKb#PQLs-#X)7C)QfU#FghwI@ep)TfepL&++t*(!()* zteQjAqBGBx><3Ry2qpV^Cm11>3riQ{ibQ`F?jDs~@D-WC1`MTFVhuf_ClnHi7!M3R zn<^7SjcN-YE~c5j7jAmh;YiYH^=$wCRIAl|zTix0c6D~bMp@}(<m_RQN6CZlu{CSm zb&w7n^A?3Rlt5i8HZr#*2A!2K4ydQ`9f?A#G!*vVF&!n)MMn`LM5zs2`=LFp<Vs?V z816E%?*OL>onm7_dI3wO3biWrVHHPXcy83wuU%-3C~+ber}hH1-SyD9$v2yHY-Mzs zAi{LPi0v!q0r^7uty)1mS$nfbIH(!V>aw6$T8e2$!O@<KEc=?&X)edH((ggGTj0?h zliRp#uxn($Y<_jM#c@7eC3+u(o~h#8Zr5oD;C5)cZ|PFD_oCf08wk2`&w|?uopCmE zoW~0dY9nLQe80~yKSIHslOi(OI3Bu1S9s2b(zQa1H{>@vWynw4-^MkU&QTz!P8OAJ z4u$jG*;)4`T~(?wZcm2ERsQOF0mqNo0sUy1hX?g5O&BFqTCOe6!{8E7_E-3HfDUTj zpGSvq?0%gP{CVXo0bXenPWO;tH53;NcS4R}?OOO;h~aG9HhXADcLR~Ht>FjbxoiWW zk3r=2{UA1acU_|AN?{a~B$;ZxKa0c7I1Dl(Sow=9r7}_zXp#_A`q#4$Cfsn&HdCwA z<5MocxkGFB4LO+z(n_~%pb}B+vFdfV*p4781LEL?6f4+PYnbgkm2;Cc1RVhvHQ(*O z=a5gQin{0!2@y&r2R?_$96GyK7oxtmzvw`OR_mePg4>er=ryMLf{c@HtE_TaTlKEm zx$ZGSc5rkLL>O@!0m9|(1K}z@GB=HJz62p*(O$W(@;H!<5J^LJX9Z~~X#a;L@2GmH zI};sB%IZrpj10g_l<#vm$#|qn(y~t<L-iJ#6s0K~DIl>*szC}qIWnurbp;Q<@n2~z zKb4=MI8sv(lo$b^w`CO8hntR?nqHQP%t!QVR|+F2C!qLCEa^@Lw*_U-rkoe&3L)Ok zJpVFG+x2)BbEQBVAiw}nkQV3BM2Sbd(SNCYYc0A4f|`dL%d3KrYew7JF^7xv#jCWD z0+ho6gnZR}6;0_G7I6qu6WRR<5P^L|&o=Ca$e#if$>xpu1|n?q0mmG$J&zvK&`yI& z5?s>bMY!zd<p>pLKs*W=1r9$dWKdAv`g4m@@=V@WqXA5VOLcS0Y&QI-qOMyqYenE{ zCWydfTTGE^*kA{)-V{}`CoL6flB{QD_VI#CjA8VG)w69Vl%8tklu%CgtY)tXWkR9r z3cqkAOv;~a@fJ~ja*)}btNRVD2y%K`r0kOJ8qmf~C5$<8W>ApMYT1#J4b33+^d4Lv za$!Th)0tAMKy~$lTNv+8xTxt}N+O0lHxj!9>XqGbaP10hwzdau;;2C+-}Jw|?-+6| zqd0|`BWo)~VGKuto{%vzM8@+Iq}jSTf)>#1IRw|w4>K%81LQ<{?oTFlo9k?JBH&`N zWm`c<$M1Spxi6^IyVJYu60YF+>$cnW*nOe)q9`MSD+)39Tbxd=vyR##>|_awCAkrH z^kYsL&#`d_4HARBK>TnIN2X3aPaD=)(vdR6Y)NL@h`+pQMb<f1wC{>jnUl#1WHxi7 zCMKq)AE#lDXMd?^43V`VAu#VGWd)g9K>vn@`$5{_cPjTsNS}0NKo_r|bh}4j5f||t zH#`yPLWb~!NfDy?KPiJFQ}8KtHhqdRc!4Vw6($UBWPhWAyJ-Ggakye~Qtq`S-~S0d zOMz9V43VK;YpahHRq2taThPW7MOKcrfPL@Qp7;_&#TN;`x7quxM}5EJnej6$5d*B! zFu2>v0tIHfVJF-@?B&Dew$qyQO~v|bZhow5D%zKaLIG|~cA6Te&sOj36o@l>74noo z7CS{CK3p`tTjsH_#@6I(N1T`Sg90`DPGDYm3<v5uU@Q!7vhqIWdIy@rd!TL<w?8gZ zsEG_`;ATa2+G(z29=Nl$8lpd-(0iNjr_~Z&67yC&BZ{Z_i!IzQ3hCo^Vl_-RB}QpT z8Qp3jon_hc!H<;qwKCSWn5p-4ij1h?lIPsC1e6@$Sq7+}{!qO$+_Y_VGQG%{xv7v7 z(ISXr$G#bF(4<JaReO*;Vxma6I98|?H*9dVzUshzev~FU>^Y~lVo#s@y&QLgAs0vB zj`bX9ssAE4xMXdDaLdQ3Vu~g69|JqFIXta5zATovK?Wfy1BUh=G?vTQL0w5`CN(t$ zllx={1RG`1pp)QEgLXw8rRvK$#miH0ntIohm(iCgZ0N0L=W7~iv)h#Xfu-c_ksFS@ z{{8uh24RBroia%}v>nIa!Anab!O8bH<0-gwSK`&~_WHg^)4ye-f7caRVt{rda_d2i zQD0z<=OHy|SCftU?_U_#m(!CeD&vF%mCI@cItf!+c#1+#HRLrx$UUQzI|ucHgPGLV zB~xL_sxHZXtJVxM(s?D}M&#t6BGrsbfu4unjfenh+5d316xisiJ8^V}K@LNJo6EC0 z!6}o3JJMX~uvw)e#xlQkM4DF%;l%s0!JCojNAO~+E5ZPw$5FWEmI^S++omhLFwCzO zSU28zI<7hn*(wYyxF<^9;qEh#K0RCw>zcH7>jhL~-lVp=OQR@i@JVG`%Rbaqi4wgj z7A-DSB4psZ)i33Dalf`>Gui*4TGDj`8+L^gCl1T`i6u8c35R}RP{=1i2~ipuNQ!e5 znm}7R79D0>_^#ZD5N12zLxWx}+qauG;k;-TBuq+1ZT6I2ggyQ)ZfMC(#H`#g1vfXI zGert+d*)X53${qpqs_x<&D$%<6LF)doY_MFkBj<oi7#HbTbz?|uz`X=UmYQ7D`+x{ zTUuXqR!U>>X&L%_DB{?(P!b=dhp5S>11lfhSZ5vb@IW>D<hpqr>}{=A5Rzx+{#l-q zb%((>bV3%p1<I-}lcRMJKhR6i(WbP2CCSHN%-=DJ{E-%p`B7?4xT^)I=a8U;eG^37 zzmFng_<7;wCR`=kPDCEXz>sZn`7Cj_R;Z~mXOxu$6_u1BA9rK*O37fZo%G$em!BMF zIz9SX)GRXXFM-7>=CM|-<9S!lfqJ_GwE~wMVrR%@5i#4GsE(;^2@`tz4mnLE9Vm$@ zmoty&O$TmN5Rm~(OuEex@2H<knO_(YzZUoy{cwEz8b@u(B-8G#t|&a0KhhFe@~fsz znQY~E5R6Nn&F2^>s68V&1k3dfH5xW#gPE+`AFGVWzV>Tb#^kfPRgO3~5L8=cNQgUN z(lTW5&M)AzCPgn7=2Qx2wKkaBha*^FA`Nm&QSj2+JwY&P83_84KEWX!FGZQFA0;9N zj53H=Z1q~D6JV>>KlLrlR2tkgz}Lo$5VB(L3d488%J-n7sQ__2%Sf)4=9?F2OiY(B zeS2fAx5IN)_i;ElI5ATcQta##Zjx9o6R_^vIn6>Rb9UdE!NVcvZ>8(o`xOQ0O~kE4 zPnMz@U{S?o-gfI;4wKR7w;dtS>BF(0J#PUilb%5f4HGN>UeH00?x(gOf2Yg0iD^4h z)kxhW#URtsPI&cH6HuJ6sHpZ7B6{0LpOOh1GuVV7R|-2gkc$A}uFJO1Z#dFr{Ba79 zKjyW7*RR%g;z{EJ;>Q$XIS2h1B!&~~k9Xj|UIW3j!CW7WG_dH*tQ0QbI{blfV}bJx zGw8F0Ex5g*Q8Yg}o99oys>K{tK>j+G7XQ9XZYCDq;r%QUL}ep(TX*ptqMM}_R!ZE7 zUdJ1a{G;&ofI>%;{CK*Dl6GxwyIqF!VwG4}O5$6-Rd|i%1t$kgbxbMYd1F&~>-wkq zWosPMeIe=!r_G>%c$g7-g0(sC67@LrEEOiULBVuFIz$z#pRL<ODYo-R@c}*<aor`c z5hg>|*JFMs@OQ+eA({gLU|Z)sRLw85nZhKOf_`<l_@-$vt5-Y|#v^C92R{PH7Ca9i zm1zhUvFV>9oUYm^)PG7>?3d1g#<b5)G-Zx-D87CF+1=5Z94how4pWM*ophz=YF#3W zE)N*6BW42bU07Cpa^EY%spPeGdf7QbTJx!9a0`%;rLVInk`hR6kcl64ra=N1PtUC4 z88$lAI%v1n_ltYEOOBAHQ`CQX9^ox4o{J5q>The~W##n|TKV17^j8)zsRjytHzqhb zBYv&v?KN9USAa;tttSdYPOqn}O1@A_2J8(Kv4&Y2q8Gdsy?FqmlX`m~Bgmm~lWY4? z1B&cdylMO&#<>hJ^y^NincSt9#;+rIvYNekI<UcTyfGJRk%?-Y3N4~Y*gOOZW8I)d z+ZpRTqMMB4<ZpwFCR114$3Z+kuZzT{@{(^)JA`LBpy~K?Q8PZI@E|8~gBd17Q#IFN zs3^@9WZ&{|5Y}>Jpl}l;9WRqIm9#NjWVUF1A~ciQTeS>eV@P?U>3NT|=3R0OPZ}}w zkXsF?r~q(GyXqCtFz5dmfVRQ9wd*JOtb+L|;I~=Ih6LNNfTId>BzvEO5x1`-OFvUN zR98Ov$*>-so==IL^@L{T3d5y-6A_0_kpxa)?uZ&Q?uY{rRz_4KGhfPdqmKqFwNT`Y z#)%V5!gw28;7;GiD(&hIC07+pR5wJNtVkOWCqvwgMr0&RYnWd}Y@V*l?kQj(A8=(g zGRFqr|D}GqL6FzrcyK42{r)qb!RRdxwx|l(5%?1AhhFq?&VE10dYZ7F(Wtmnhe)|* z6P>iLlSb&`lk67)%)p9p!w1`K%g*{jN)8Sc)aWrQ4h}neQQB4#<8kXF_~BEnwB3Zb zn<*^q9^U80AgB%+l6~|L+o+R;_dfJYHH17owA-%nutg89e%%{rcab#oDY}-en%Wr= zRRN!V=&&r>Q@<wfZA?WDBE$9jeTtCNAPE}Ey?M@UKlfxc#-<*=9x@n!Hn5<z{&zrM zmD+EK#2xEH#8nMUZdCYrcD)XV;W!$p0~%2^%pf4TcEP4#u{w8j%TvwDB%=v_NNC&f z$j~#yZLl!rr;^4KHQ14OTN@zH&W(5%>{0`@{BgoOfPwuk!Z&FEq}jn5lgkTdo+F8$ z`BuUau-Ep9G(D|VS3{MNOs$css7X=0kaQ0=hgXU=2{>f012s5jcxFCUYz8r3lBTC~ z5bK!hZlxT_$?Cjqb-aLiak@-IDEkgXwQxcY8o4x!hcT$#ip()IdPMaN5s6`o@TF-7 z1ve9y$&9NVU|d%gd}CLVIH>-8KoGURFAOk6O4+`-4N>6u3#!1o+OZZMyxu}V{NAaM zEhM6_57Ff5Wj{*P!M#ziGRmU&M$tx)Y_|h}1KbaHe7Uh&520RjeeT*Jqwhbe32LQ0 zQHl|He5t`Dth!o62oN)npN=McN|nVO)0OWU+hXYx61F~z7`j)OyZUsp0W@UK7px|W z;IiJ!2rDuI%K;LQtbLXWGL$cj;67Q`HwDIP%fX(8H174M*loMvQA!M5Ofy{uX)!n? zaglJjd>Dy|JLxk|FJ@!Q;2|z5@~ZKe<mRvs4mSK*sOEO^1=G&fnT$LQuvumRrT-l| zMTH;ltMvOhC<5Q;iLFAeQCU_(PV8N+<Yrf<HsIl<ql1ZX1HydZo#r>`rpuaWGnuWF zF9AgE>9%<WM^zDxMv5m2uEIS77eG#;L~;uSpT^r1jlZJ)-^Ij<qWsSh`?3Ik!n+3P zWUvWZc636V<@vOc9V<}BtB#x2C<|W|ZMCXL1E5X<gU=^@Z?-;nsg_ko8INl|z0J?k zLrj$;BPM0_-uITf%yv)3m~NII5aocK?u&T&$VtdzfT8ym;l*}00Y^|$yrzX})l9YD zk}<g`m<)KOzV9=!_HR2?DC2q4M*H4yA74t;qWq3RyS_u4R#k2KY0{r^d$emqcN44f zd{t7FA&HBX>dKpQ*W-!lV!dN1n8r+g*e6K{yaAV4xHOW=hZLVmkB}vP2WT#XAr3Hf zYB|q2^?OGYiwKi_RGOs8$X<iZFyN4X&bp|V7Rb(`kYov@&Hn(RCFxGALHI3|{QZgf zQnO1b1GN8X>BdyyeEHxe`&&-8(-5O;z)&MwB$!G3!8++AsHx<UDz-_NFEk}DqMpM| zbim|rLd>l<jQ~o63;jhZDl6V>Au1U<8qRtZq`Eb=i^SmM3wt}l*0rXY={{LEa-it9 z9r;<VQ}?9xZ(YP1w_0F))f`9|eNRfE*i6{68TB$V$M;D-Qbpqh=6H72M$GFBxVXik zC!Ecq85Qkxn!g%o{(1Sa$h_N|VycobMT?$ytyJmsNR}keOs@|G!a6|<G6fif0b%Xb zm%L#HCAa%memS2>H_&eug2zlEx~T4vsN2(li=@}Paw9NO9s55YNdV{Dr~ve*>|yvU z6sqLzh3}$r%EYM|=4Lr)n8lO_Kf9v3twKgJ&YqCW1Dcnj*tChzt;$<3%zrF2oY-UL zI+u&-Qw9pt9L^Pc8A}yf^c01CYc)C>r<oOt9lh<Xrl~P`^~Nk-l!-UoE8|^Kl&vI& znU9sl%?#1(@JWj3LN3h|WUO&W(&WWrcxyTKpx~mqg5aKqolo2{sNAHeRhLA<TU0>z zvuNJa#_+f)n_)IwP>MLlEjN}K;*erDA+>R>)od_95v;5S&w437Ua3_PC04$+%X`Pm zkXbAe$j(jJ+^4IHzT-s$cR5sI&fHXvYhl+6M46gnVzEHIN<LkzE)bZ!)c~T%-5OI4 z?syU&D`(yPcaOrP9O&GQyBL*`(I1B%PMqyMC@zdqDlS5g(M9!<H*4a&hw+#y;y@oY zX%-o?0aD4vFfu}kDx8RHld3zjXTOJfNHm@yH^$Clvc#sU;xiX)BakXaR|LjX@`QR+ z`~hvgb9{%gyCUz4-6lEYROgg5ak8CD8vM8V2YJx}IN8zxt!<!esK=QMR38><DX(Rf zOOq!6g&N(=!~0G%gJ;$}$xkd-Cf#YcUxQdwxU?7d>*fji5a4>zS*v6R$$o2=R2YA` zh2Hr%9c-OlUsT9IUkKH&C@ON<ezcLD3P{#t&q3z<SrC5RSF?qfMocd{hTB1Y{DC&6 z{QUO&4m{jolq7`kb_uRKdX5<Z<qIp+2EbZ}>nKLDp+{DF1Otw?f^o95M>I1*P+|a^ za#EQkfh2=c{If~4_8vn9`%BB+?MYBp;}mjlqJfQKZ~g2UHjf#Gq)06KcwIFT17|PW zzb*4W>)nqQDcg5Nq^ayT-7=a`HdxHOdtV_fk_vH*oL`vPgM7ED4rfZHd7v>PdG5KJ zR*g{?IWIzhF)9dQNAreJ&Z465t?ri~+jqNFY_P98jD3cs+NFi3Uiy2x29;qJK-#-Y zBKxID*qgCTU0{s(;N69tS<U-W<Kdwa12H42LyVe3QzR~yJehCwEbuY)UZk<@^kkjG z)`>jl8u0qL0@5Ei3#_}qynpOO;PA0z37rdS+u3n&;>EIAvYnvgbcA#%H=}`+kBz1| zf`&j5HI&9}+(QKEzA_jHgJ$vFe?t+ai?!m+vqjCj(8Gs%s@<M8*}17LNi?oO-zZUD zAxk1cT2U$QqBcj5?tb?&1(coW@1kgC65G{0CR$`rJPB_1%9{<0{$U{oKXOpP!-#Ra zWorJdWBE}1AQ`Lw?!THINhVo9qkgX2N=gSEs8f~w=4?#-w87`&)CZ7EQneg2w292t z^qgM-czE&B5k=t!?b|`;R#OhB7m+>FgZH{;$Us?Jf2^I#79#>d0Nehy&v0o@ff_|M zs8HOb=e8Lu-w#4(bVt)`qL!C^Vn*uKIL|b1Z;5RDU4)M%NSX8msn9YiD<xfs-C9S| z(w50HuN^Qn_FO>3ontaFG>X$>x2)bHx5()mVv|9sj?9i;W0umSi%FiAHh<}Xg43e} zFFNA}&SXTs;l|zGG(_|?h0(XH>qI2le*Za^8$&81!|aF!XSG9&Hmrn4RAj3NVV=A2 z==pPEvYy)zaE4q6P$$GIMG}hTNNt2OZD_AvxOMWBG=C|lxBf3jnSBM;a7XvUv%pJv zyZN{jHfqqHLm%e;>C62CORqk#bHVd@xCDD!{Y(neZzZx5X70DOpd!f51PZv~K*C9e ze`S~wq)lQXGE(9WiVM^;(G#Ug!y9_bU3XWs8{-kxG@KLr13#&wUcpr_o(nmLL#_>y zE@&I-)ufG}OL2vxUORD!n9%YcXvqj@q@?_~Vq~&fu1Xh?8gxdhyh-2M0O)`OQ|CsI zpR3E%tUIfk@d?`l$|t6B^*JEF82#%;{&Wy!kG%2hxM0$k{%X(iD?s{~5C(9$3aR=) z)WU2S(oacXAwQ*hLSg<=HCt@BS_I!9;);V_E3%-BvUydyHi&9)$xZxC0o;hZj9!9t z$goEK4NL&P{|j^d&%tzQe^DC|^`88)l2sC5VIp@Eh0|Nkt%d}Q?^=Vi!9cx*j+J7X zz(7>nzuqWZ9TqKpE~YQ@eml@@IX#3`*#n{%9W%EUeZ)LkF37a(bWj}<`Cm}g?^2&7 z^3P>E^sj+7CoYZ&?!}&Had54?V7rQ}8}(m>fL5^lsnPSId(8MryQdm3kGl<1C-&B0 zB@DkfP~OteRNqbP>L&>$jHrpe4|EH*>?DUtU}_jKy&U2saq<BlM5X{y(hQ57i1ZVX zhjRV*pq}v#oKu-XPsyeeXDsC36$lqg;(pYJ^K0v^C-Ipl{>OnzreXwI5hK1PlC4@R z5TL^*4C4OQGY^$bAmE0$jZr=fX2cHaa9IU=YA}xlbE~CK5&Wm9W(`7bvdV$#GSu(i zo*lX(D=LgA-a6n{T)}f;`O-wasJRwFAj-59Yi5=+ho)SMRT>e%mO)&ZA2U+bFt~%5 z62Ub=+tzk2F}|zkA~&BF*^_b5It%N4O9diY`SU1dx9J}(&5`+8QV?pJ@IO@yE*)=B z-bySUSTKlZ^*HMG7&ih9xa*q{iDFnDl)$Cx=`@cOim_v+^<TUbQ%yLIla@p)FBSfo z?<Q1nKr15ynke#T7ZVfY0h8jx;Nz=%IE3v~1bdB|*+h#kH94f%`u3Q}hY1p>cKTcG zdv%w*Z1aOR-b4U=F4~J~Z!ARMzfmL{1$S9)rtgekLbo7l3Ua~9EyF}|YnQoN;+kbR z45%@PC_!e99{<drkH?a0Q@vQ&s-9w3+dkddw_kLrLCE-4`ubYr7>086D^c<zMN&<1 zwPe|&=Z=gERg`9)!+(Jh-q<CJWLM5<`Z8%KicgnBH1cOr`~`*bpFvZTE;amY*WjZ$ zWSNx*pS@vf8NZ27=LFif&kKXNwc`uizIAl!N+TaA3fq9V48e*_*R9o+R0X2Z#rf{0 zS)!gP>XbkHkLcmo!|(!K$J^QV0_;Qly^K={v*iXUM=O}kcqnmPB)F|l5&E}{i8TLb z6XoZ(r|RD2?O*sowa9eU)XFqJ%@uDah~X^<TigsC=6T^K{a-x>|LPq?Lwmyh6+Qal zAyx^i3)N}4eZCxaFP|En^`R;aDfpkidPsSL<h*%XaIoKMmQqP{p@uR(kZ8wk8y5Yu zkpBI{93P0fYT2v7;O`sz8&>{HsQ-NBS9$9tJ%v$LmiWK=nEpgl|GXb%y$yZi1IGdS zU%|kikl^?G0n8h+q}4qv@js{k*W7fUk>6N*UZM%be#`BjA^zio)>~g{GnJ~^f7AZ_ zGxc}SWj=2Tn15y1`MYQMUjqIY^!h2A>X)Sjr=t4L^F;n_>I8l_vR&<1LW3s5u_-hz z`9HdR71}@!H*o!iB>r8f|M|H~G<zS^o=h%wM$x3t09TAqBX;t$2q<Ebm8#}{*r0zU z3%|!zrSQvy9b7cuv7_HZNdi{qz9hk*iciFg9Oe7A5yewW{r~hq1LjUkkQ8Q&@0pXZ zEYs8uFT~$)Xo-pYyXx}a1oR);rcUFRee2oG1P;L#7&)m2n#k%J#f*%YCkOs@U)kYc zlnD~(7ACWQsiUd^njNQYKx0`fr2Wr<t_y=;Ewls^yKb6;G-ra%U#Z0D4^ebncM(Ui z@4Fn1-+re>wy{p_;S3idsor@c5m+Xbft!?Kne_O%-1}%;3+bkn1fP$pGDG^Na7v_M z%dxee(1-$c|9rrurT?g3R*6b94;<1N<&uM9qS|N8TJfkN{!;$q=+sLaDz!Ix|7L%! z{%?*h0>4OcU53sCloYnTbqKLP=RH<E{GQzpx3-9Iuih(m_=#=B6wu4V*OMy9x2atr zo;Q<jT=C;r#d&H&2l*qhyGfD7THU(1#x3ImDj^fxTV4`6^E0_twCOK%rR2W%TWUlu zfW*vit7oqRd>#4=UW-+oR47F#`1?+U)kphn$d~kR|CSXZ^ym$jq@Ek`3ybaOpNs7z zRb5K=!>#)F$S!?X3y>2AP=mYQqA!O%fLOm|o-GNs@jWj#AGY#()GcKwWdzSh6(SD= zYQ#FUFBkB2TkDX&eQD@|Ngm#VH0fAw(4qZDgQQHf4k(tAWr156Hf4XhAtm@u0RORV zIR2MfQQSAHt3G3Aq_zoEFQDM0({g9!;++>MW1;r`XNUSah`)8+Fl@M8Z$kNx-dQ0) zmQg+TAy&{cfeAG`unDRBo0sf;+sE}*K@B>H!M%&TVbu=75iD<p*)ft|-2Os5<(mkW zY}g%L0%4Y<=9=#tW>aG^Nf)>ikHg`%b|B*FsN%F-hm+zsY{Zqf#UEf@aU4THl*XbI zhRNY_nV&uZKbZMUjATm?#-<UY`w3i`*uQT=dhiHcp0UnI72;WDh`lxze}Sx3-;^#b zCXAebw-J+AYjk}0hYGO&ppcKu!`V79TWr@RMm~oC)$_HPAXGNXTWP?;fWevWd91`r zy@n=njCNEGp#xGf(ZpMvl6OqI`%bbxS+594?hW-_#*)s$$cSFWZxM>pIc)h1xpG9w zs*t)Llqm7)<W!imANxF9g|*!LZ!Q4XXR2MsSeu$zQx`i<v7)0ip1T*kOX3IYPW)5t z*Pxl!?gCQ4<Tkuy+%SrDljGu&A#}UG@^5ST=JZToRtvM+gndW<W*uh*#YO)*-??w6 zCO=I&G`jRrE&L@G&hjF}MZ3Cpxb1SnzT%^TiE!JpFoY)xp0Z3l%hxV*fGvzB#vw(E zbdR<|GR@xiEqgFR%Pok6G3}n+&(#WzDfgPmiUT0kY$56r{X~cUHdR~%8qN`e_XZSp z%~k80F`uDosPP^@<_MtYI^}7wjxd86)|*d;la)i()K?#WFttLZ5Dal@)dHwN8X9`Y zM@1zVPMlgHF{jQw8`*ixyb2-<8`eth9ljAv08A7@27_`b{l4m?CnmP(dNt7LXbE%d zBGJ1l=jX=^a*9T<0nWv-BH8uUH{aub?e!M+<6=-2=I0`%QIkxX3#;l!hshNirY8^l z=DOl}5xN>KONke~k&W?2MikDlPl^0F2cM3|IRV-=NI_V8_kGXaFRzl#1a{o{10{~D zH@^OlakIn$bymwG&THU~77_}1sG%so;?ChVX(g4aOG^DwzHes?vt`$F9E<!A28>p1 z^c1<ovJ2MDUkuHKi!{&IIQwqDa8u!88TuALc!R=+T6e-#7Ham_Nkxo>Sv(dN;+B`P zP(}Qh-@S8%aXN_!*AHQKek7r!5PF%B?;nk5v&V7C=pnYhrxm6VQj7|6idOQc)qrLz z-smoy>RbMKnk1kK*8EHeZR5{ZONfMSg!Krl;%o6@TM|N{iJhxUjP1?YqRYe>DlE{2 z>XEq+f?{>D4jBq^g=0-Nr7IH5o+joM_DP!>nvLPN{0YGm<f<u}?*hSgc8s1ff?J8` zCUxQ3e5c%r6P1D2JG^*(_%o2_kE1#G2y*0-DQGWNUGs;;-jA1T(hg+tk{e!_6Ho}z zE(pGp>4v9rxK-`b7x+;iO$x_<!5%AGtapwmfljioqf*j`KqdsoACQ6LdE6`+phg4z zW;wvo;1bUdYXn^x7&PCF?V~RTN@y3p@qmYsGt~%z8!kx97}#H|GDBaeDln4XODK9) zbqaJ@m0LL!-D?jOm@A_UNr8aY%tOW3KNLN_kvO5?r^V7t$<3HlR2KZS=cW1)i!C?f zy`V}UxKx8Bucby@c13zYAh>uU;>fUfM70fB@J<pwvP?9)F(#ch+D@S=cTU7@!3o1? z%!|q$=)ykDNuE7NEcM6wdP#O6#X9+-#68~9JLvwohD-1!h35BO>c~oWso?sk9$&!4 z2>~T?nja-;wM1uX9vpLz2oAArB?fI^AdR@KGb2>`upPo7qr=FJXY!C^#sG|zArY_$ z%%V(5fcr!#rO&~VZG$PP<ssxmNX1%x=EAH2)=*rNoQ^USc92urgR=lss~P_q(hLun z+-Q$?R)NP>Ie6DxY0Qu1f?-q}Uq6FIcT|QFcgcL^?*tSNj#`Krjiz?ewn@`2qaru< zQdOVGaWn?#%{fYY&L8{?dC@7i>a8J_w@8?Ceq>$kXsd%r*goXf3`xCJ$&2`4*A#Fj zC7CalX-`P}gR#R$?`?fGA>X3$-hF!H<`7qzeSGN86Y@Nlq%Ck@@gCXkm-iCVJ$d=@ z^MgM8?~S237?2}RU%^Xt4x(eoKR~cL(HlodR9;LnwFG+*bb{v1L0*?}(p~lddpQ)_ zAj?oK>5zKQ*E#TR);Ix_<*$R;qIyE3pC7}C0(NR>@H1nBh)qPk`-KHC2xO%Y2+Jx1 zaD!RT7n;!}oU~7vb#qE$!YC2OqLS#Fv#Rww1l~~yK#dDf<KZy$uWRNKgi>hWQSZ?y z(ublSvo(AR(M;?EBTF%d@uD`WdppN5O3>7ddYt7gl12sC>Tj{tcQMFcMv&Oc^)lwt zULB4RjF?~~BIZ2U*DFr&+Rs8C;dFfEUr(%V`eX?_PmSQ6Yc(ho5dmlGlYeju4>;>S z{J<h5KzYi~&kf%`6)b+?$d()F4+zDD8k-9-Xjd(Z2lT@U8RARjuU0*uyM@P!>|BN^ zz&nB6F}3PPm8yYcvg`lh!C6oR`$%F_wmmlW$MI4}y{oi5Ne2XBM+l+>>B@^5o?0TI zzeXYTL4`CEh75&B+%7y(UoF^#04IHV^Jp(xabqV?jL--J3n+vUAk&pL(Qisz13X{G z;4gY&bH9At0$Um?vbDo19j>R}9aSO@^2Y)x-}0x~MRvMi54dHfY*ys=*bZS1l67Fz zNCZLe+x+HwylyTkm{iWK7D1sEn~YE7q{Iz>#@~t9uJXmr6z3*h>0Pj_X%pIo<<UFz zH*=dP+|}~ToCH>oYv)O3u_V{@#eQ*33&61{pxIht$Z9e>U18V$Y`FTxyisRleAc`5 zX_09<78fQ2Kbj5hbb;<Yu@SL-qt;q|?T7VJZ$9p?R09D%5Og;z&b?8BR@~<>V9!C2 zH8T9g(J{#rUy_OGvnabi)LDY?PB!@kiQlgD$1CnnY24CNDX7WhHd?w{WD}4X%@_>+ z;f$?ffd2exEg6`R_XmnMmDT0d%JmB6T|bQ0`(9!NFO{i+`;ll#B7|T(oksG>>5f2_ zFm_=9{4rRKN%kFLt9Ne^29anN@%!FA=lX`KqI3A@0K~|>K=vcpv{+fUhKNswOsE9s z=I=6)V<yTGyG$!No3jvJMiUDHN469F%HgN?QC>#f7IaMQt?VT*Sh1GE_Ch05Pf+!W z^v;wJjW)o{_b|J+m;FN)n9MhRWo;7Bs_Up@Hz;~0c|+JxlI*;+OidU0LnpAG03FT! zx%aOa4+$~HuO?$Ovo3dgrbFM;F(u@O1?cKe=oVG5D5u@)bZYI7`6fOsCCA7RdroEH z<hqo4scG<GHDM->rLCu#OXskn-ia~=ho9NqaiTL}QhXDH6Ik58A?m?c4@4ElFqtXL z<t(NQhS4?pkZf)Ql9iJ{wh~b))zzpTeYOH=R#0B$Sp<Dh-NV2Xl_Asix2SWVRRu4D zg1Wl@P~2I%f=0bR>nyl{aS!}No6h`<%6f7*O1z-#<-aVg=HNbxqMka>HLJaSK2jCe zMIyzaK+XCT5Yr99x4D@tR4Ah+7-(IWP#1vaN(lU7l+x3t90fHDw^*>)f*a^F6B(i- z4Cqw`RSyXuV1~>lNJ5M%GiqPfWZVb@7pZR_l;PI5vZ<V7&W+Zy(-USmV)ei~`k1#S zAdFj-T1Kx6gsC)gCFmt@HYZmvO@Jk-!UEIga0f%qw8FU<z>kyw$g;=7FeQ`Nu!N{A z!$topYqU$evOgUZXTUQ(w~1rZMDl0Xh#yjk9_qt_8+JUHCcMk6rM--BiX<q9ADR5B z^gQr|mg!$0MVCDEB)(foU-BPbscapn1=KNb%;_4S9Dko2%p7WFv83PtiF;L<FZ}5R z!4`UEUo%NtAbAcS!Ehr=eMv3A$jSnWaDda1E+ctdqMRJtHJ=kgGVCfb5}bEd88o_r z_+2%N+80Qynr73wol<agBK*uH>rX8e$Uh0S)gS;d=_CAA13Ny+1`?E}TnMkbPlf_d zfU@3>!k!N-c5h{d+0$S(7p`Mu&C^bFB^`#Ml;g!sS|FHS+XXe?Q<D!U{uj8xbm8?M z@s92&m7AQMs&1Vm6)U{>XrZB){zYo&E6zVbp<a3PDfqAK`r!>0FnnYffEHkggg#3* zg8tA5Q`}LdARjkdY*5_fTBC6tdUo!ts=9!>0~4^QDUyCJTyd|kuv|jx@fpf`P(El@ z1TD_etgHY#W~`jX0Fw34dG^yx(zLal<~XX+1ND;sO&}i>A%-&gT6u`NHNbDP5-xfN z=HZ0Dh~@j@u7l)Eat6W|cD!ln?naL2!5$pX6bb?r1SGysy1j_WF?mJYH6|L0pxSKQ zfgtbJ&6*wM$J<$}Pfb;~E?SP*x&FzqtH_`~3v0&(2WI>+-Up0+PBa7>Q-<RWVMpJG znibDyAv2vo+dqp#Y!GV78VyO!(@-hy-8@Lh`);IXQuJ^&VdVsaVrK9oL}$0MDv@yF z>lI0Q`sH@zD#Rwc%EsxmQ!SAR3q6j=2Xv)n(uVmgn9Bz0T*X)xp}+J~-tzO0%jVkT zeN-qf^)=lNXaKP>k%e={G^vKui0DRCRVHYd4E%URPX?H||Kh6B4ikw_QY8mXGBI*t z;L5F^K^_J?dU{v$_;(^I`s;4;AcI!_x*IIN`<<~0mZi|JIKU^#S+$I|?(vU<GIV8B zJ>~`=AZT%WTDA%Okx*I*36jZ>L+*l0jf+OY?;+_oH8~@pwid&r-hya>HO4;O5h@Oh z0F3mkiQZX4zmH%h@eK0sZc%SP?Hw9ly>|=7iNy?7h4b@35(R44sRXE6-cW6*bYr05 zoKJl03gd+5GnWM`(8DfgMNReePtHd09+Al~g?TEJB}6#pV3t?!x0QyMbYfMx|D7it zg}n>hc5-T3Q2PsYQcVIYyZQjJUN+JTl=-tRsbly};^#?OmqL#NsRK0X?R`H8L*<&9 zAPFz&32(ut<yMD%6A2jtZ-a|L>oI@#(uFvJ)pL?4;0j%u5XfIeGpgKgumc-O)S}w2 z8(ac)MB;Q3i|mByH?i;|l2bZs${DkI3>yFY+it7Z9OaZY-U;Tj;OZu>HA9-M_%)HR z81Z*i1rX3$LcF{Pn{Wl9!xK1%z2sRNFimW?Z(G&axvtdQDkybo<iF2g&@D%*qTQP_ ze_f#a8%OhdA?k97e&Ni-`S3f~;`abOVg==ZA)Jqd-p9Eb`#A4C3Tqh{P2U#vtak^R z1Slv4=>zy_O=@LWwK2lUxj+VJ66fjp`n5R;EJGtqNw@A<AnBiu-+n2d-Z!Ht?dTWc z8oSk9wEVbwdmE8R06^$^-wYa!K*g!{Xf3g5tW)NV?H0k1x?Q`UQaQgO_6v2gh<r!4 zF}H-M=kpgVq-y|}bO4AF*+dwO$^8mvx<pB<D!>d|_p5<npr=bQ(}Vg+m|ATiZ)z9` z|JZO$9d0(U>8sTb8deok)b8DW+)+*N>-MfwyUas?qG?UKGd_RC=)cDEe<v6b@h)2% zwgn|-e&E7{?$5X_Lb|I^gRAG<r||M~fAGo439xHWxvaIx>m;3*HqB?9HR|OCYsAj= z`H9}Wg}<jOjXY@VV83_F;dU52F|6|PaCBF$jEgC&r%VGr@3_Cn@%MV#20PS-wP?#* zz2Mf~_Ba0d=MT_GCPxkoB)MN~Tvo`$d~UvYp1bf7w7O^sU$}PX^}+r1m`!Sdx)ZDT z#PQu<@G*Pm-88D(+}%EK=d|V-{@!Q#g)Vb~%}ImV|KyHVf#lZsgFmWr_-X5)90ndB zbo|6^eu{ilS#z8F2Me7<1+`X1^nRMt*um|5QNABTc;a0;&?2eHOrt++rV36-_!CZ+ z5&1=l5olZimC6>EmFd|_;mZ|xqPh6=)?he+xqoI@Dv_PERSmD$zy11C2d}`j?VDIe z<X`5zYYY0|gU<6wLs=1;j=d3whgVch%~}of??w0ynp!G*8*y+gTcKvCMk*S$e^*Nv zq7nQ-WjGBP*(~F4)#)N0+ytg*yB_9M*?9Kymcl;$gL$e#f?YRi`1HOWmLgze4z?dM z632Lmq@Mgq+ONO(Zg)M}{5ZJ2z+&O{=pZ>%TM6B{;4;*$4W9yL_zrP?(#1t|-9130 zNE!<ro|9`_oW(;tDEHN5ctM-JQpFE7`eWMOa$Qv&u&{zy&4-}Uelu>`)*6DMf{1iw zG|8O)wm)L9NZ|((f1NcZYMG^>BteIzLXKI*gAClRf12;@_~pgBC;<oMu4QwmM6dI! zuaN!QO6+r%v48nEP62+KcWH8iI$|jH<mQ^1xDkTdIOKbOhF5(AYuWpdBUtl*R=FcG zv0eWP*@yQ_LHu;k*XrD?K;@Xy*O4?24ygu_zwz8jJ<zk(7p$44ns-IJj%H5dT8YyN ziGP)m`87j3O7my7il6>{;YNs`?y13gvCW%vH1m({-(*+^m|V9HYo<|c^80I^Xd-U~ zTDkNLfA;0N%#l^c!g)K_8<%=b{`L#}pi~4^shzu}Zt%6BhSYx}{w!}Gy2Zz)zn7pD z)b)$mJZr`V**`W46?wzvAq=tR5BmRxyS@!Iz<r9t@@ut}QGrIRsHp^pXZ(Y&o4(n! z6fy$MzZSg88~$Ed+S@<=XZqKf?|Kv5C({29U*7@M)U)j^MFEkH6oEvVfHXm+H>FDx zrAn0|DjlSEkzPUvr9?zQKnNWoCG^mnbV9GvLTI6c<cojx-uvG7-dpRetdql#*=Nt5 zJ-hs7kKmuTQ4lBH6K`bEGv@iN0de>Q0Rjil;QS>_As$+mM;xAq9LbCyw$4Bi*?$r- zAq<4Y=<_Du^?QQU|KdTK5}vRm7m?ouZJIl8DlB?iv)aY~9!tw*+6Cu}PT6Bt$$vV+ zn{D_qnzu#Pe*ff#J7L4e?E|8|#XpvvLh7ZUgVTb)e2Mv0DN`gR<G1l+sgGxqu8z4{ zX-~+DzjcDjUK9JC&&;2@XX+F6hX2ljoJHbOCvs95tXF?``iS)Kn9z+|*7onBb~8kq zc0p2s=T(K!KMW7Sa^h1TxOhI*f83ma2f;K_0j^Avr2iU@TZA(3SA?zlznkCJ!N2qe zd8O@tG;D$bnyin`=iXrHE;>f=_iV#|zKLyafc-q!-8N!0qWOmq{J^RH%APFiT7!R6 zil1{i<effv>yPML^xyo+M?>PZ3rDb*9mKcmZ%Ize{$B?#tWlCl|8~TN>Q$wEa=Tz+ ze<>u84w=hhIovgc%ZUm7RXF><ik1Sz)Be~bUWAR%z-_N+R^VT9WC<~G7Jl$|vVZ2* z4`PHi`ok_z`kzG!KbOro?hKCQIi@?7Yg|}X|Ixw<gpD_c_A*QP<Dt*8z1B^`>i+Ic zKTEsBnQ(qQ<n|}>)5|gB;sO13!zl9a%UQ}&FF0d;B_3^a{vMngH}PXk!Ak7;dscna z{wSUk98+&)YFg?$5B=9sCHSm}0Jb&&C7CY0G?#TCQ0pb|mqrgfPqzf>{&=YDq<!uW zUe4H>AgJ{u)c&w%l9%{K;4+KB2eQ8UO3AwuzqzmW(V1+B0P0L7Nb(yWQpQDCW3M!` zlvHH-g4PH^>=yNzYoG7UJii=7l_PP_m#t_2EW7O_nXfyyYLJCixD#n&t|fq1GiBPO z8_og!gwW@<+1G++|ES~;z%S5_;-tgBhBlaZoHC?=HSpNP4Ly;QqWHAGalfk?cvglu z&Ye#hRy%0gg@C;`olYl1g?nb?P`0N_&fCamp^TKWg6Mg)*M4X^xuBz|Yrluv43C_E zGqbU@mGABfdUc_oiF-evs<$l&WkL@%iNEn*(o%0iF9`~Ke~{|&d-{LVC3J6Rm=T*{ znAl*N6r|lC4teE+84NUZ!oKAO_}R>_GWJ$am<pW7Zj$fkIpZ2x>#!<5`!6f#D1-{* z{g<jVwXd&KcVm)z1i23ugD_d9=yDSKCDh*B&VG%3X#)T8hiX>eUFOrtIGC`7_vYS{ zq8+C>Ocg6ZIww!T7eAZt(!}H1g!WMiZO(yh(T(ZuyJdDuqJM<)@IHacP4v;?>cTJ6 zz%qbl!A5jz%e%aGE4?`?2>S_f@OBE9xqF&~I3D2N8vYi<E*js3kY+ChB)vs=>>X4f zPhp7{vPW<BJg+<LuF$z11VqG&m7WZNy9{v~H>PY@ES#2<p*V7j6u3FVX-hiJ^w{Ez z4SG5<22wf>tNmblljE1Cc}PMai*}WG{o=-j2ygt`e+2j=@>lA%Ua^M&-{6}R0q(_A zrL}_Th@4eNh@JeC^-dcHm07X$w3=Onpsts>HXAT~dnEe!R&5)1$m7SDy~iF)&%Qq4 z(_jau>WT8NQXV%iQ>pVV(D1F=h?yn24Dl6CRi&&s=3Su5fccVceJ1=~NzG@RE(SR# zV4hefi*qQB-K)V}r<Ym<QfChEk-8;r76f8Vq;o@~DSJ7A+c~L98YJHv7QY|^<{cJ4 z=yWN9>+*$i3}=qwOu4;e4iiqCp!xv0Tl-_evh3nAZW<a*LSB{CXNg;@jDt8SOi^RS z)d7x@i$w<74I;v`8H;3R7K823O!o*pI{x3Y2G4{U#omx#Ot?W%YvM0QMZ5xi%Duv9 z93H%VeAcn$e)SPVmSAc%xy8unn)ByJ>W`F_MuWp9BFGB;&EMlUroJ@~D!C_lA^H*X z5tG(S=?X<Gp}aem-vuMoQ@@W7YQOnzz|5#-c|^7{(5(@^j`D<_elS4Qjkk1}td@e{ zD{IaJUAcJ$<<Pj`tjsjJ0_`rFR)OoxP@`keweg~t#?3Y!%XZpbC!d)gutc@nc+XHy z(HlQ&$Ad5G8pK4bwS6O`Jdp@a8WUnyjd}`}B)64SX6xPp?83x4O;~MD30VFn71005 z%4CM-r}meMvPOk*78KkKqB0daLQd=NvwHI&86Sr>IVDUp9s*lUlp=Q9yjJ}ABF^Is zrn26NbT6_(?Jq>o!rn5(@a4|{DtE$;u04jSMtiu7=Gx12{2|b@ctv>jT~cqG_qQVa z@GIo2#YX8Fjt?FPlXCH_7z*o*hF1h21DOx6uN<X=G#Y9=pyX>|gVXgG*GF8hqv2tP z-}+u`f(*}FMaUE%gWffYrW!%*3)<qktUvo(UDfqAQc&|U%>$)#0%jDZ8RE`PRR}M1 z7P3qq`&ny@I^12DMK-9HUaab@_M}B^=dZ#o&Oaegvveh2j?LN8xYg|a<6G=UUxJ!U zDCbO$`d{Am=s;dX$sGP1S6j$;NXk4ZM0jj9?&qbc<CL6E5a}zGFjrgm#N(|o5`|m0 z*p~*ZwH9YiDJ8x8+0{+40A{DJe0=if5q>K*&L6~>DC&h^&j1&jcD69?{WbF>leq&Q z$trZ)ex+(@GT46yaJuc)@Aly3&d*ne7sVFogH0Z)iucERG;n+Bk|+Hl!!J?wLKlS% z7Yp)H+p;(1eOB^ncvrpvK`1aTo3-IWU6HAFT2#Ka*NiI2n9@{gFfO>0(3CU8Lq}nd zfFb^&Il67-(q9K=>em@;ENtYzRAjRonUXDfuAg=vTFl6Qz1xmX6uT*5)uI$$gG?$J z4z7#ke;-XxZv0-x<H^8`4F9*7tbS>)-n24{o>Sw~v7c4jQ&9D{fP>=3Cwz}?$qB;Y z_nb=WyBEnK8MBM%pHM87*L={p)v+EIJ@Zqv)8s*{+oeVK2aqE=y)=I*zNn&&ZD3>+ zEJ?X42k6r$v0L#>zmKcqdOSzZkidG}H4OoAepHs~6%HYFO`cd%y8_`zQrp#n^iXa- zdSygu^ymELB=kEjhT{kRIf;5LdRc693D;C)l9cXQ&e5ghmYF1}R{^f(#y4nYi*sJD z*GQ;NR-hs~@$51QSE6iDMaPGF_2z-*2+Au1h`%{c9JvizSj>+?U!7Xi0#Zm8sJ{Jl z1WE*}traWfo2S6W)KBBtl4hJ_c+o*PS06up54NCVcWRK<@SFMj6Wt}ix<?$R%sxL0 zbb5*ArP(8z{J@AN#~7Kt7U<;#M7z7&Lg#tW+jY;=(RHT;{@N(aIl76PN?x2($N%&| zH0L}?W^cxQ;`y<Jz<Fsjdl8eZ6E4g2bkE{}K#4DTCi5n;>B4Qs0;3f$Yt+*Vg1J53 zJ<0LO+m1%FC4|CU!@9Z|S9ATq58fP*yB!>XlU2buw|?>Ip`?I_Rjj<9CcrN@Fm$IM zHxr3DgwvJL`#f-p2u2jGg41}|0rubbM+7?yP~n+wN1QdW%P5FfZ-{iU6?WA*sb20) zV{BK33<^D;1Z1B+D-9*#M(=s8bo#U(`P$*Fk*zp|P}7EY1H!ck+7Fh$^U`?NDUUdo zCvM{9JbvB9dr})3gp510;gP1bDdaxtcA`(dNwjYP6B2ibI6@h_UtU`(py!SWki0I> zKHxMvb+T|ztar^Uy=y!Y_(?!%;Ky=>`?{E-j6sWJ_JfzDZGtV6x1gj_>b1#jwpSZ3 z%jC~~h%SR<q!fC0ma$NFUeQ7^5qimiBg8A7Hnb2#UD0FmY>jW_7K;4-0P^%I^xKL? zLe_JTNHFQzqe8udC#7Gtt{jv3r?XKSg)20BD%=F%tKnVKo#u@>LrjY|ypjKu%63wK z?W_KrKl;7;JFuObl`@KDYt>4f6*~r&C>}QyN>ETGz1E{;(J=7s)uWDq`P5A8w$Q1C zjx0}vPL7xZ`<lH9&%)gfYbp+X7$=qgTHgL~acNR544t>%1(W3he>c@&T&aP`8kVfx zduRJprfD$;(rX=rQujgYJEzt*9QmHM4PxSCd<Mr<nu3BhJA9_Kd-8CNw*eM`3OT2> z&Rsxk=l%V(;?j`jbJ9(3kTJZ`A_XYClrv?5-Jq)&<P{+*KQ!<;5X99Z&Z*|paT30! zwB(aNcqjvkZvlcfu@NhYgF&NAgY`k@$@`NB7U`MCuR9`E)*zVCm$}DRo=;JO(B^_v z@>sUh4%kHura$OB;BLj~Dtv`epC#d0pukGJ9FNXMcrLoD+EZj=9L@~{A!6M8^ktfs zmMs!2WZtB+dh^7?rJWP}0H;AK*!jCb_jV<{!Lc*aZML*{7wI<DzXRV@Ja|V%meJ{# zn-?NP!$b{C>3$@_4U=*+jR(r}L9(7A)wL+2u4ws%TZ!?(%a9w3ZAnU`jx;S5n`ic1 zri<5#yjfM|tW>md>#mZSW4<2Fy0mr{ijDBq)k;3nZ`2C2B_fV5l(YTNOvCNh`SkL9 z*Ndh?Nauq$p&RH?JyY7KcbaW1#yl|h%X$*3qNNMeA~JWmE4x#;Uc9)e{NP#mrb!o< zZGO=%{S#g=-;Gv?ejnq##>A)iB$ZmeuIDq+<M&6~h7rfM#l|ISPq~FrKh@ud+z;=i z{@CB|&l3{JUiPy{rzaVWi@)hKuBK{!@<nEE^?YS56K6DswMU8*ud0FrbURHP?6iB^ z&^dQ{5H5rBH{EJauk;!~Pw&F44bBkB8G%$}O%1BCf`+{%NQdfIOabdbskJ|`d_U|a zyQUA*q)!`UTr|a<(j$<#uMV%L<<ACmdM|)O$icTVm~9O$znO2X^QVh&WXk-#cJpm{ z&xe5XwfSB+CAP3(OBR`Kum2beFIe4sC&y@|<8#`L^B<Y~xU>Z{AESrHZtO~9&nTbL zV;AY%CT|_y2)juvf6!&O`lxq~bN?2!e`t_=e?tc}!y=ky;k)UxeBc=LCZIQ?zfVIR zz?0+GDg*EfQU#n4DaDI(K^iW8SfsH~=L+q=#jLzMIAIaRmoOL?3CT|mUP}}xtCu`~ zG{K(=5U(4dtd#j}M}*^jS)VpGH%dIF-O_`suczG5Jkh<oVtdLjs$sF>!mVTcaft@1 zn~Te`z4?ljTgPg8H_NdF2c`E_n5P3G>mG!VdulQDYcyO9x>D80q*0mFcZ1v31#aus z!6KX*pc~Pb<zuF8_R`=^EFoI}fBLX6SKN|gdX^IvM+nUYISaYgsl)hal*gAR37Y1@ zS$CMtcMJ=IyR2tl-8#2T&6|!?p%uRsC)UXj2|KPZj_WkHx^43$@|rmLYBIl9skzkw z9<n3Le#duAFp>$&c-p;ek>n@ur4V?qxl`?l=lmvP6E6vMHl~mSOvf)_z-bj^x#FA5 z>cDwrOKAR!cy}^Y&;#r-mMZg80rHX01`768zmsP=`s1>&oFK0Ba)Ea53af=7<O1`m zpZ#3!P#5YJ?`P$#s#l(`c%UN6VjL?zk2f-7K$sZJ_D@Lf{M-Qz^k)F$gnvAb{^KU( z1L$eze7Zeuo5ii}%0iiplZ1TxO3l1IgGI3i6-;s09~`I(P5@`Wrnf|5wm|npNPFw$ z^x|>?_kP~6tIS{CZ}&Q4nd20Pc?DJPcY^Td3YUO9C37k(rjJ65%GR$@dS$_G&c?xe zZJlRZx(=T(c`Wb8PX5=EeM}tdT}^S{iZbn&M`Dnwpm)3tO#I0HCe3m&Ayu>Ua__sM zOxxqEhJCk3HAvR9$IU$Eq>$W=gl#s`UI~sk!Tu`JbMgo{f72ubb9Pp$>j|e0tNg7f zEzSzb8}zU<>$6bNh6}6F0G0S0G21mKibkX0=`HuDL1dz4(Jt~;&3cN<n@hB650<)% zZqLxadirzp@_Vya<|~#nDi@QRsBlZcRwcp!&6}cbUm8p-VC?(KMoXp{670ec$^tg# z*5lO1f^f$H`$CfsF-r#%qrBH($Q#ThCHIBBZ4XC^Zyc+t9QOv+HGn0~KCwz3CN&U; z99QS9FEApQJWSC?jwMfPSx=8=-5?D_Ub!F8*0QKiP*=(js1})ZvM`~}%g9A%pLOSM zHn8gb<bu^Z+qX33Efc#Lb3*#zu`;L5;QO66C(O&^vv#ok`_OUISKn@go^(+4BU?I4 zO$7{h9pWb$qD(y$N>kTGL>BJuS4M$)JUDNKwdWSBsy;gy6YtKPskXNFK*2eU53~bM z2K<!gq$kJ4xKK;Nj@9mo#Am6&*h%rZSf|+$i1qqkz`5nCU*=^Tqx-MWOmh#ZlA@K- z1p}?yMpuUMM2(qSST3I)SHlqp^yMV~UHqOy&d$$1a!_wv2d(Hk8}I#APNUT1A7wBX zWP_5Y<$i5$He>fmBHvV_vmU0Su3>+(SfvqLB5jMzzljM8HA0pKn|7}ci9_lz?5Hnx zTy}Jp*-g~#)}J40Ke~R`ZJHv4WfMBAb0mAroC{apo-nFk%`dt}A#X!fN>U<6jJe&Z z7pu|!XzH4M4f(h^>AX9S(tSgo{Y+^pbVBnPM}kJyb(!s(iMb^iDHTwp`4n+EdKOj! z0d;|3$LnrzFSt^mH(2H%(W|SzXi9aP?)!bi;TIQsPrZ8kElRbg4Q7yu0fsR3QZ^|r zkpAU~=~79?tNW9+S|$;am6{7=SGFCPKj_~~PjS`!z)a?xsFvAH#Mzei(LWS%2C+zH zL5?1qC!B-vArXc8nvoSMa60^3s-zs)AN3B=@MHQ_z`F6)_&NZ3swi?gTx+*7Y)94W z5#;ivd|AH9F8&~#>f#JxU1(4~y8N6z^X!WZ#;llXN1i9CUz~Vmec9lbQKpxCzF+3{ z_;<jomviVTf#mx~8f0{|9-`1iE#|9*5_loU|8*-k`1a=y2!g4`fJ}NsU{sivmo?)p zdvcpMq7tBM$R;HjGJiWO4w)Y_bprZ3KX^-%*82*7h<;rXy%k^o#?SoriQF5%aGQAm z=yQcFinzGk-yl<^Td1)`jN`pvcQ9~xR55g^gE&~~G`n{>Jy_(D_l*L#3}UKQ?aPT{ zbCK;F%tnSO9gOdqueH>sXlM|4zcwG#*nO=$QrK`}iKGFrQhViJs-=P$ok5ZyDbXt5 zD#ht(U_`ozy$DJs<|4Rlw_^p~iI*pY1Rw`je>v4npdfWXadhgADovVK&>6Yg-jvDe zB3~lP#_UqIgzpXI<67Yt4r2{#UPEEviH_xf3D$%e+=A~EjBcp}a>m9Dx808jdaW`u z@HB7t`(h4`z1E`fOCYY055|aFzu(UXu(sG8#R-y7w!)bJ&GLRLX+8($*PK^j7mbXo z)@4ad;0&=%vAOWlG&t}g`IdC_nNu<FlXYLC-n<BWBo$bvzA{7Alj|(Uw=8ihkN2RA zy~sj9+;ncG5|JPOg_WwZPIWf`^V~<;$$T7CRAP0pd(WW6+GYl>{6~fU0_%lPsD4}% zn~F)ayKAgjrnj-dA1d|FMjp$4y0XLWm3MvknpnK)wDBD_)Jv|+^D*CNc7O>AuQl`8 z{U7?Ih8XExp()?)oWnzinJd5DEBU^+fQq{7YE39*At3NRN}}CZO)Mnx8l?*NwZ)pZ z-J7?}vp2|c9==w2K>>Ttb4LBv;|j6sbr8R@dPl+KGtno;v_C4J@G6=fUr8WY5($iY zbb9X=yt13AtnMoX-IwinMgw}0qL=EcrVp2H>xDw(nSxJt-|ac|lIBWhE2@^sM`nTu zlz>}Gp$0|G*3)S$nZ0Ez8Eq+Jx7&4Sjl768=a0IR*BOQn5@KT~E8jC*yAj~m+5~-m zTIIhu5okDvvG?jG!=T^aYkxtL{bsiucJj4hb%ad}#dMkO?*22yQ&0Xibeje&iv}zl z+qEbre7KPdK?s7H?syIMeJCE3WVNpvd^u6%6yw0+Fljr<uEUn5{_01>YnZv;E^=k} zyonoPJl9Qezsi{T)Z=v2d=~o_v(1&zfNA*p6tzc!E#3y}P^bd$39fo)aFOqmE4@CS zc5N4vJZeB(r23i2WW{yj_wWTMe?%EtM2vs`0=d%H@+Z>XTtRwxk8o-;H|NlcgPS(( z!${P$(LXS{HxbQ~<!x_Cn(bd*Gw-l2^Nl>6ebU!S1C24G23a>H^|j~Lv|A_#0^9Y$ zb*9`=@TW%=S{~~D4+Fyt1+{b{<z;Suz2ULd?prM@Ys~wZPV{{^Usx}<wx(ynTZckD zCkK|~wSH}#=dH3wY@>`yCZ;9t86BuI3U09+SO?M0d~?P;Vi!!FR=Z#L(m|_X_3$$H z248RA3jL#3MUg!8$&tl@uzMuA(&?@A{$tpwj#nWGviJ4ZOYWzX$3+pJMKy1%o{O+X z7L<MQ#)KO7#-$Ggn5Si);8t?e`-h$18rD0nyoNq#;?_}^^Jhw<wD3p9Gu%6hnoXEb zPqDy^V6HUmmKav8j8Q~@&(apF5uSD)%e%F$LCkNr6On-$=4U7RjyxmMn9<P4X=RhA z&Z(L6(4c`XtN9v!TQG)h`ki2hJTT4AazeyQN(eOP)jBCFMaVwIn9Ks{R0<I8>$;ft zQwO|aY>C@gtcPJ@*)PFX6@h6NitZEU^K@xja{IS@76vSOl67P$lurseOHJg5a^C9? zc>R0=QZ<@8mK!}4bx>?N4LF@ttgU*lCd{W3xccCfAl?sK#O#BX2?83(<986T#02M* zZ=90<-Tj6~A0P2=0^+B_$WOH!d?#(i{>k%Yb0=tZ_i5tstsmHr2PWTH$s7oDuQ-Hy zH%wC6=sLZ~M=zB(2H7edF~scMb7BBFzJIeX#kn-07w_no2HSY!vuMn!K<ugXHW2o% zl^Qg*T$e>xXFHl)V<r~1b*3Veh|g_NLpQHPHoy43z5wi97S3^!-a{$Qr-zEmMXH&6 zZX{VT>s)cdTW{-{x(6f{C)PfPLsOj<$-+H5B)INL&ki`1<p{LQ^fZ5`^UM&o?<G2y zVbU3drGP?j4{Zv~xRX*y2Gq}Sl2EW8zhiCKVG*4>v{YEyK(E%EYv!Ik1sgTrm2gW% z9Ls@paiHA`_tRE+81D{Ym^+<t&ulHG5YybnoXk>cP%a~YJR9=pOjDf4G$7_%oo*~` zF;bw61Vu|rv?J~4jTRgaH|J@&{YVWM&|d>OhWu8{$fIjDWC0wEF!UAD4K-n0vj1ia zxK{I%*GZw@YH_Dl=#JaohjA8S@@t{>f8xOZ?EE=P2wuJrM~t*+cqI>+n!{gP<aY0f z=uGz`V*I=m-yEL@P`uZ_Ib8MX$ESzVH+tOg$S6xkx|2Z*wnR~4x%@aqh0CkjTo%07 zP?PqW;$QXCo-ZdpmVZ_3yG0i_#ZatO>!d4!skRpO{AB+vlz9AF2%`OmTQhA$oO#E} zG<qj}*vy2w#qSw*dCH=P{REJye&Gg8^_pmZtn?KNh6#Jg+U?GI;nO8Mdy<p)=~w21 zyu0YLR$u$<&&!b$FneB81txmwE8EgUCV80(*RN(_wj*?UKzd(Uqr30is=f=BofN{y zwOB+k+dS+7p!+BH#E&I!iC=V6?7uktIRp!wkChqUHNBuYowC4<XAQ$Ybf`g>-OzDC zw{=E4Nn_&d=eP{xw-cVLcPCn4=>zT#n%u`xp#9C{&b2i(`|$y*H<gIz-t;ls^=q<j z2YwK0<jQlOL-V`)M!ksBJ+HnECnc`={hUEXy*VK&d0br0@Q)7`KL(rL<RJ`7(2ic1 zRGA@qwrrx_T7C`EvtJ?5v5mzqY*dbRn5p`eX2!t6n6Fz`Pe)1SIqMtm?$arG+pm=8 zd+ucVooEx{Gb+1{hRizZ)$t!iKGi)hdERk*$}_Lp-59)<Y}|Xh=i#4>-;f8P8BKz) zX<2rhmkVk6q=KR&eFBS%OBnH4G(OkbB$Y)@ng4Xj)n2jWZJF79r`ehBVT5k{`c6OT z2%$grSHyuS=P>)Irjljf84@{x+91posKiKvFdq&DLU-FC{c>+(#?eT>`H0nGF!X_n zBl_b0$jQRaVATp+(^;^L|DbBe`Tl%&f357RZ5;i(iLG16WTINw5Eo);DmDt-JBzyj zoVJwv`i+k`W4p86;3V_Yok=Y{!<Z<MX1V2JDmhNoH3^SduOocyStrP7uBmvK#MrXv z0EqL3_-{8Ak=jFik(n#bxP3~WUbVXKM^%Um`yc_Go1ad)dE^}<H;=cER}sTJH>rcj zGp*ckKd56ep*k`=6La3+E|T2@V89w%%_j|Mz%v(2hw93}47NKqt!C^q2W|&F&n|9J zUGSXWQ0*g-^qVb#@#zz#-3XXy{68trvt$Ls%PTLdbob1JTD=jm_ws`qOyA<On_Hk? zI{>}<3`Um6s1r=SSg}3WGPv5sFIgc_FGspU^F^Klhf3kvkNke*%S8)FSzK%h<Md(w zY@VTT=pq`H>-#(Fff^8TFfp$Zk-29vCt38>X@Gm6N8(f3!KTk5MK|ws+$8bVsYdN? z|8q?8(;qHhCpqRYl18uIafw!W(0FWS3_QoYsB*J;SszPj(&}n2Jv(n@-LQbV<CmNv zA;{>0xt3CP!OS-GJ#u;OBC+f;?Z%amj;meMo`RiqHI_)-nYiO?z0dHkB;X+MJ3~7} zL}-?+G`Rfp^^t6I+pRzG_g@=L=E=OdX+w;rS$DpFeI8*;$KfdMA%nzRKO35lL2c^o zD_^mt7rrZ~e964$M0#xIbko>3h@))BXgn7TVfl@t`YW%7#0IvsjYldj&}5nStIzVL z#%E5ZEdIj){AWYp?`<3Z*n@HEA9>hYQQq~?62>clxY7{sNH>H0{E;WFzBVs;tu%Y8 zTn6u}l7ZfKa$C5ndB1#&N!etKh=cwj|D!c=nlF)a;i>evuEFn@`~^fRB0glb<g20A z=!X@BS}*@0t;~v0mL?&zr8!#q`|ImQ>ogLoMnzQ%Llqi=Rj-L`+kQ1GBOu+A_90@m zq5OLiu9TX>Alo<lR!W23qQ4mnS(=TgU<9Lim&7ZhW1qGP^3no%?&H*7VS%%hOUmnV zvSoh`duR^lY@R=)@amGZ{D%c01Jw9`Z73BKSX(mXQ@i~>(KOeFm|2=t8+ArROIbA_ zV?JQVo!GD5q^IJD&&|!!M!)N@rc3MX`1bjUB;QS)%elSU1yB7$J`y*|N-DX>CC&Wj zn(*%-{bz{~yAzW0HF;~g_0ud+m5(#|$NMW@(N|?am4%;e4&<2Q!ySfuuB-Em)I7WW zq;H*DPLgx6CF8`*MPq>5MoRZ(*VDHNSwek}*lAP0QoYakZ$JKDzJ7!g`*PewhlMHc zG^cPgL!#l@wMsV`&P^%1?VgBcz9lv7>UQYl84ea~U~;PvW=$#s4tIa~*4O#t32$D` z)aR2cZM;wJz4kgI6lDsrc=Ltpefzi12iN&N52Xn^(0#UNTO|A)V-qFYBOAAUYc29e zq9$(=zYJ)2KzT3XcfFJ{Ucv$8RjYid5I1wTO(g<#aaNXK?{C8`We`%On>R8OTf;O} zVg<&Ylq#rkP=oZJc-CQP<1TAHsMAZx((3q{5?xlt8`ZX$w9p((d;vg-oYg%_1pbkr zzv!6%{MvYHD4BG{oT?~uNB`>OAKzOqyVj})qKN~Rx;}p))U7Kem*yJUj@dqJ#Yu3I z-eDl-h|vj%KloZ}lrZ&E;yV@C24sQ>30B$&LG-tEe^>u#|4-$IXS2P9C#J_^tMt~) z|I_3Nq0s)QPB`0juP5BGUDLFLa~svBr72){RnyH*P;B+_MXjqvX6$Xz;y%-d?d83| z#Ko46Cyhc%K*L(auiE_sJz+nI5vl)61%-cvN9`k;*g%%>m-k(0l=SkRRjeF9Cvs*d z=`ru5^^NP@>oHH>m2K@!EJv6~5iCxsQG4r8R_iz$^3nHqS$}Oiy0Ktw%B#p)V}G=| z85rS>(DuG*_j$!d!~5p#=cC4ajNB1wYc^Nrx?bB8$j;abI~@lj|Cxa=2?4?~lM^mQ z8D3*jh_+2}*9-Eg<5`FMwll9BM|5V^KK+pEF8p^=9FOdvBVf$geY31S^6WL;hV`yJ zVz|t}q)>3Q$mHkSw%A+GUfmba`GZJf<0;!DbQ4QX4IxO?>Zd0?v0+H=Ju+ADyusl} zP~;*)QvPvM*0g6;i!Jx|Rq0uCR}Jmx1dv^_G4r+a{&>TE(abKEUP@5y1ET@%<Ru_$ zuw&T!`(VwyhFgx60zZ{h^m2HWKG8m?r_>x+rwZdrkJD(mWyPfYs|-q+5b2Pp)vJiY zV#10>{xRg}IAnZWP({Q+Xjz@+-}tj5osaK=bEC=ScimsZ;-p{OS6`M6tzp?=PHWI4 z@agU{`ZGGJ#0F*x<F3AriLS@10Fkd^L=Y>OMnz$g-u|oiSv}Eo>%?E4j19Cd7e==N zqTV4yv){hU(&`QK*Onnw6nf|*wn|c<#Buw!$(E6llB>VlGsK5-(<H0zzMm>7)#Eqz zB&owp6HPgUPOCJiQv{TiY%>XbwDW+Cwk_lTW$OPX=U<-+ik5~Vg=YAQL@$4a;E&pE z)Gk#}dbr0vuGg~dc^}0DlBA#<J^y+2n%cb;5oc3HY9XT~5snDXP!ZG(@x1=X7mbZO zEqPBUyLy$4Of!~5YA!#dCC>oq_;3=@R!u8^&eR3J?$b_xXH7JA7VCuiT}<^~z5mx6 zUlmJz&IEm4ZLX(+9BvFXG>I->?Wi{?B)KmEt>ihvW0ZdsOs=}c6vq*ilUN<Ne0&>h z)n5R50JT(o9u(Wno;#T^s`{L4gmR;`tnjONG<Q1juoLxB#!;@6x1jXTkTHcQ{{Pek z|Lwk!yR3+?0>13DC0H1k{N1@a$(OpV)0?K7K9n($#+W)0p%UpNP@i&kL*q|T*#AoY zzt}8=hXv|QZ>q6IPG{%5Pkmr?|K*77?6?s3)Ayd4(EpPKY9Gme$qC@CU70qM-IqJJ z(%kE{fL8XIN5%s5-#Vt(O|9Goi5UOY)&F>trffNO*@nbFaL4od@#-3la^8!Sihy%d zt4dj_KiCwp8wA-=ReaS`qbZ@q|K`KcBZWcsXMiNf`toXl+COmv1t7JN2Z0aYwRO7m z^e0pYf1r{0`0W35;nBIpgl;l9Ms*~85B}mx|0hk%h2PllMp?nYV(Af-BTOu(qxXvf zWO+5o`Y!_d-$uuO*dRuLkW$jFe!68rh<)6b4G%<DsQtg`|Nr|La{OhGXYzDz9p5yP z|DQ?z`=0A$64Dmj_J2TW|Iz7xQnEpUN70=x^1A(nTL1rj;-3s%#<!}!l&b%KF-t+7 z8V`G;Qcbk}&kp|keTdMnR+qni`-}ShAIJPG1aAWeN<_2%N15jTe|+BjY9*v#oBDsU zj0zuuN4M#&F#V4`{dtHNRD9(Mg4@Oa7rR0N@t{R0FrEXxhpR;IK5)}VHzI5O(C3oE z_g<E9X{f2(6d-8n3~x_y?W1Vw3_m&PtlMU-Uy|}X&GOUf(Ov~*nt0&UZ?Au*oj^g$ zdz;cz)E7m$BswJ@caabZMT#)p5xYt&93ns#ckf}|`#@Md(rX8<>7_HM@d52`IHrkg zJf7Y{P1V~O1fAEyG>DZH{?{ulB$#-`Hn`}v^8emO$&!;gA-I92^i$_eLXQ9a>Qzu6 z+!s~QPyJUL4GqivRPdr7S}uBdb=Q~Qdp8KfgqFo#*_@3K8xA~ekEuh4fEN+BZXCR( zJUSr<EzPN#=Un6zgIhz<a_<=7hPYA(<QB>f*b#vH;R8w0X?W*{^kXeBfW@Sam*oDf z1<>oq*Jv;D%y`ao`Zd^pl^Jo7U5{c6I&#Dud0`1a*cqi6UB~EP(#YQz9y527z~{p3 z?vc?S#)z?nEDHlWH-1`Y6vFSjVB&Nej(QCf=ilh%lb8->HAh)T?2Riceq9)@hsD88 zGD9=6O3h8oh;M<|I-=4+{egq2?h<ZiT$Eq|&ur*^{Z7FB8Ry0~UAwgbgIWF*)_0py zpuuwEIt$YD8n|8}h<S5&uYa_cV5hKxLQ}A7>by7Wyzhn~Q!n9+SNx?ZSDBzaIhm85 z&3W=(&1YjVe<^pcBn+XD5u?!hJD!%I)TBmfvb5`7>mLb^9c~5p%M_|_LoZvXiQmcC zuyJSi3wtZs`*7u<_8ix`1bUejZQA}YLx8Z+Y4ZAu(YXnJIZwv$O@_0{h=z;Ka_=wc z9QqEr(|%wf<v0xk65O2(5%bCUh>rN;w{So}T!}IY1qq}f={pxWpvCrjfjjMCf#h}j zo7^mEw`mDzsDED}sa!74-3wE<bEp6PkO0(7_(JHk*1Y|K1PnLIeGa}pU%^rvv}Rxm z;9E+p4+>VC48<LO$$Vaw?GBTRtOv@51$YHA)FPuqcw!=&+=1GTu*(*wlK_bR`4NJg zYB9$m0+lOE9~J02GC-dB-T6i*Om6>P=k_Hy^_+XJ<H_Ms`V^dJ-usHY|KX6H#kZ3? zPSAs8Z=4#Fc7O=N&4Y#n?6#@s%>=|DcwChijs4fLZA=Y|%O{wAZL^QhTL+Q#jPj5& zoW;owWk{{e89xF&B#Ag()mhwc)i{|!!lJjFrs5f?uveMud33j7JCq3QT%f_-5LJLg zq($}{tJo43;PJd0xm*oTL_O??snfwh6(t^YH$=J#Lo%7(wG;(pYj;<LFH%?EsX*UC z97PpRMcnmXFd?C-TMYOL+oLzP8~vi*X83dnJ87<3)!Ptu4)7*)&W23GTZv}&hC#E@ zL^hUB){iRgG@yx25ImHV%f%pAk9$x!oiP$Y<9|3X<6=1^IZ=u~QM@zxWQJt^WqmxI z_ntH^Qk>Hdl`1STA%*K%KADN=bU8Pa?-e0%9vN87EdaGbwX!_r+dSzO0^F#O$!CWF zgvnm$MhTI9-8shkh8G3qgBLT&b-3ZSuF2p}-h*R`8*IpDEp%w?r9s<s*NJpwGbK8( z6eQPX@5zKb^#a4;p7-|-fU@rjWabR$4QCH&5T_wK4$i26%uOzFZPz}koTk2}py`8t z&wxX2#6feV1mk49@CAJ4PJ2zxIgn~G%5ZiI0Bl!vh7lp>=lXL@f&Tqa@p!p#SkpOi z)BYFGB&!`z)oaTkD97G$P6I8!&o2?u&Tyr6r(*})#GK9?-T()9@2NKg)~x6`aF^pa zfgL*&rc#x;wrqw(9@b<PM^dUXVkHF;-AO_A412Y`^oIWy-;ioG1qKFtRMA(Re`z;a z(tCFT_pc4a<+Sjj*AXYoD>E{tan$I3pqxE<4X3o1^~5;uRXflU7Khc52xAhmO>tVr zuz3Hd_VvD)kJmjZTB;6`UaU%&3|BNR3VzOc%PapWyz{YS&?o~)_1HYbjEG&&2u%h2 z?9B>jsWx=B9({EkBJWBq%xn_e*LyM$>Yvi6pnJL*FbS)qifd&8)Fa8HpZzFA-Zzl} zAObFz&dQSNb?)t5=<H9SXVoh0c+4g4IdbdDMT5Ztxd7xA9)~Bql`3#M)x9f0Iq;BJ zG}RT8vgC^21Ze8ld|3QWkp$RjFM~w{pr@8l3pKNzV4_rY%;tjpa!1b{Pe<Sz&L@0B zj_T|WhqpRNcH8dK-{myXGBdQ4Ks0X@>nyS#s09Yn!pXT=ecmp)KBqnp>SZPAG!G;s zng8_e5qWSu?40fwrQIUtvTx5$lf6&QaWB!AmxaGi^tP5Z){7)9K_PN+u#3d<hZ3RZ z*j=({c>9C7b1%vy94`P{uUhavqvbAlO#<5#5L@lX1_v#YLabM}53A+&-*URCFg**{ z><((a&|yYNY*w*2PwoUB%rQDzEVp`JkJBt~Tdq^^@NQTj#RoNaA^RpeOZ6mi-rJ_@ z#WjQw*mfZMkonLYr8O>x9vP~O!CW_Vw8PpD0#rgK&PN`Q>mb~z0$;aV2Tr++%{fMj zccBX-P>ptjllKMkooD;g$&+nF3Ecg(x#?cZopE4i(EaH*+5I;LOdMJc*3omJa9#jO zjYFG$eZ=$Yov6tR`DP6>F+Q?6#&qW>sWIJc0PDs&Bgv5uTx6`NQ3PFG3O-?O>OhZ3 zoY%%6zZBPWoYXGd*@d2~$8*P=?_#3_0aTgjL?57-o5bNp+rW8QCA$E!&Z_P}KInST z@hQ>MlBVM>I;Q2hJuyyB{;0o8N@VjH*GFXQ#Ja~*$R8WPGMVNMRjwhbFI2wuZlj75 ziSjs#o|C?xXbXtx11_fR4BT7qhx0LD-LBzNd?EFvQ*ohi^S0bPW7fL-5OE#pRDJCm z@-vORFt<@ToRs+6x<)=E3WG)*OZuNR>Phjr1R^1U?}ksD{4e-xr$6n+amshr0@xJW z7~E9($SQ=v=NwMC*NfCc(f%y*Yz3t~j*zCET142O#bB3Zz6SDKjB-T0HdX_bOQep# zjflvIKZBQW5~yeZYx17Wd;Ku4cUTv;`xZ{iszddzcJ`e-w3nvBiPiH=pO?`U^NsIb z<UI7kK$;e9J<;$|6**2G8;(6JNH|@?#c&>FpR}&`XGDAN38<E@KEt5a<zH$O&#|;w z_~Q&?t5$d$-hZy9u)v<})a*sFgzquV2i`b+co_~3>|_{Jkvlv;St7}LImDt$nKUfY zaMDApeX(zBkkQVDgRhqllD^l}Nf^GX3WnP+BP;PmN&$#dGM?y@?{>geBH;*K%O3&e zGi#p@0oyV1VT$m%k97@<N8+aJQ>16`Xo<Z!0m@AKMn|9!RaWaPY(csCjvTsL2ddjF zzC7%Qr5pqY0tVyo2#(QNF`3<|?zOF*Nl3$bjzpgzIS`vCyr`0Az5K%TsH^d0u~)KQ z2@T2Jf+(NyMQ_3ue317w2k(5(Y&~qQo&@_a;8G{x)4qb(cMUu2fw3GIaX{zB(^yJu zb4kpGiC6DjVEs{e7$M_V%vNZGw=;zl&H5|KfJ3_zKA#SBAbk*qOq>>MF_5u^{ZTO2 z8gk6L-7e&RM&4|J8`BBShQ$eEv4BA5y}L4Z*ClpccCSM;FdVIs-x?tvY{H_^uw;?4 z<q#yQ5p%^{TG~OiAB1T3+kUYQkfz3wAP!>G&jTbpKo#MsfC&^2E(%S+c~QF!H`~2% z4lm^Tw~30!*q)F2H6EE=T>opFOxh5aeqTkG&6Aq$))5)d=*XVXe=Qf)7jeSeXq9%& z;|F9SdYz?$_k@`*cabi26`-Gp`BU(iGhg;Jw6y6&vof`831_#uj_Z1-il$Yw7?#Jq z?>?cs)00B;S^H5gD0*o!ZRaUXOGsIX&5UkC6u)x^*ikxcJrE_)gx=uo99ZM!y;~l6 zm%x2<Ca4;ZyN_sPZ;~LKUllha-{Ow`Gze%bfx+kob%|BI#hpH9G$d3le|SY6|1+?| zWp*rp6)6#j*>PTmFY+>;O^43Nscywo6;7CJ6hBujRAH*dBN;KbwHx<o0lo`hUb(wH zQ`J{qAr-yX=UzRyLoKFQ^<$-uMpv<}wExjZ<cFC(yNQf8<)Nv!d=Yr$wK(Mmt^BJ* z>HXz|kYy~v!s$T-svjm5dxsMuJYNsAIB6ZrUMlAsV1U&7Qb)WL+)pEethPooSrYrM z^3M4$UsKHBX<==E-{}UIab~oA>#a<<vmUe&p6!jzwU!7J759+U>US@=ExMLdSz`?o z1W12@oKQn%dv%WF?w66nDu1-L!Oc4s&q&IV#Eg+P-RM0zK|7{^fj6f|B>5gX&mQ@^ zc#+D>Nrlsy4Z}Pa$+4^cd<RdX;kS&XsVu>M^XWTVFXpIp79b$W08ylHa!+huPGZkM zxo=xjn{g-2;Q9@C0xALZ#H!TZtbNe$PSfrWi*}FQQxfbOX-e20$8J4Zy=k!1%p@t{ zLhkUx9SFaakS}l^P2Obp)Nn!G%Xpv)tv>Uw^`D~*Kq|0J*X7F;b;kQw>dHdMB&@bd zi@k<>>jE;avtIQk6g}wgd(qiro0h#3Igq~i-O(}y$>RM}|LOWor-oNR?Z!i<_y&yV ze(VQL;H3RIXKn3B`A*?PpTo>Hua#%)`mQ+8w|_v#nXRRX<yb`Y*qBws%E)i?Ydg)t zc3V`VH?H|<Y-hmr^?eG~#U;-+@ojLuv*Xb;O{?Si+}bLx*A8`rZVD+dm+0F-=w^8z zC0tZ)jn^~f?x@3Zy0mmQ>R}+obstZOsGX|DI;XN|Fm@@!F#BSYEW$ir{PWbU53Bw( zAz}pO&b`#F9(LOyYLOHC;Sa73PXzodo>WDhK`mg>;yU>AU4_9*g9j%YzyJUexc%*- z!v2^L=KVp8L73C!gW<8_ISPBgvp3G7iW-6NOjdK1-j$nhS|_}_qAcE&AzTA05obb= zRgHUk<bysC%>_4_9yeBc%@M470FlrL5L;`Eu+B}F)-u-4IKv8Ohy<L-3|&@<tZMq& z<RLawy*_G35o=i|8a-^SvyfQpITGq~2Dmff@2xSp+7fmhDq(50e#i(gyRMD=9tyT+ z{Snr0z*IZnTOz+Y7rT4(UMi6sq5D{8&;BFQ|6C*^W8dX9wRC9%L+DYS_nQ8BW@O<A z8w>xPQW;1wdjoEK-h9umCJ!`C^H;M_hul}PEQRje*v$XV@k<)EG8A~Do4ntBNTYf= zoLs4=RBE(!BO&Z25taQ|&c2f2)avy8fW189k|PH7wBo7jMdCMfn)k(Zg&1^AwEFn( z%~8x~);^wat$nf}dN;F`tU=}Z&tU$?YkW7K_<a$#AL-oBvG;tUAQv8?b;$2XB&6X0 zr+ocF6@ahuEbrZD{B3*bN2ahe$CfG#qu($ezx{1F(J;qMse>STrXJO&c<9mu#`Z=o zg^EjnHiSPVm43P|)Q$#8h`QWuI*gKb;RoHQ55f83*ja;C>0SDT49Pu|yy!~&CL`e{ z&L56F0=8VQc8wmI7RQ!ek@9k6uX<OjZQx=F;?zvDkc@;q|BUo=!U>GrDi)2LL4GHi zdlGUQ<d=0!L~*r@#|2pnfG;?L;n}6v;V5`eFcRlR3mINpz6dqvs+-|*6eztauLZMQ zmuhkz{4(j9GqJy#=bdrUkw!Jt)S5yuMbr5{S@o4kh5aLhUw<r^FI$__s+2JdTNveb zO0SJVs?@%F<+klt>ehKYcDxI9hl!<8R~9sPd2Da7v-<2mKRYvc{wcTrq?Z88X({C- z14Vn&tQbFY&%kS+g#0?Wx`n1G{r-bqtHV<^iN`)<xYc@{WkY$HNTMKLlJFn&mG8qL z(6AT@U$iL=)$D<_O8(&Ih5JiW6~t?75nkcvk09HAFzK0UA<2nI?7|vfNWiYLBqq;~ zIh6u4hoqcYLUjp(+ZZ8UyHVo6++c^IobSQf(uvnu4Q<{vt@}lwT1u5kW@)yR|2p~# z#N*dPN?(nA->jG7|0P}aSqj7^wG^D%x;30f6qa?y9nhx*cYewo#b6V?p5NJ2_1V8S z2M`~$fcQsKnnm}Hn9>WlU*Q>q@=rltid#*(p57rGyT(ZoXH~7#4Q{XZyrwPt#G)T1 zvgh@J8GOBMyo4x{Sn%<#-RoSvH-HZFPjg>C*xtNT!WV&BH1S-b?&ywyT@c02y0M*! zIxm!4<k%GJVg|J0mEKQQf?VNnhv8S=Aexvq@4*4y_=cYI*6D828K^XL5(%V~J>?jQ z6_VXQOg5eDAC(Q6Q00G$UNJpVE=7QL=o56Z)`8^FbvZhF2-&PHUdEWb)XvH72v=dJ zR*!lsCU-{aYwc_y6FID6+Kahxi`EEdmwlIGeW9~7B^mdW7_4r!^^Krq!4$6n*;Itr zfrocw%o3Z=9+0Lfl~Abv1|SbtPDg=&*O|b@mU5i2baK91)ufeJxKU8p1iKcQXDs<) z_Q_&nvmmg26(tp>w|<d9ihiAb%<6aUvcl~<uz%H}Ws4~isl+<KNLn9gU;9(+uGeB) zeI7j!w3O*3zBkNL^L&5IV$p|lq!ZE{QKRaTMb5Q^5ka8kEk3bb-8)dt5%U#oy<7F8 z29JNkl7(eLZe}EV_e{q1y|-!rs@9z>e7#uhjBs{qU+T)CdSVnvp^A5Ok(ig)fQaOk zfP;H|zVA?t$h=Dc1)r9H%u1pkB>gPB0hhck#-B$Y<kcZV3{gDbK;XG6lWHJ}(;wnO z=k)mmclJZREw@sCd$YXxv!U_oA-iv!N!{w%Y3E}0V$=Bg_wjB%19|K&NEhlM@3KTq zd1HR#M6N;lC9LoccK<*kYc8sfe{i_~B8BcYf#eVXc{3h`?C^`1E?138v@4Tt`B5%t z)0vh2rv;XwkX}?CullL<R}8E924|V=`4>DAB+pzCs;Ba0?v3wHs^=SKoGDcZGsBd< zQ`RZB`w<mVW~56=y+RLMn6ChS>aaUlJBset4XBozR94Zp)z>&axW`o1v24(rav*Yh zrv6c7R{r*NjysA__nGR2&79-FrJk8R=BsZR%$&eMOJu4tHD`N^gZ`e+{J#`zd(3DC zFnI1T*9_KU9;upYTlidFmRW!FyuH3!qqyFA02^ZB+<8YFZXUoO@4rXJ$_#qm9vQSf z(*mD&Vfp!OFb0ve+w!Fj^_s`c7e_LkQudS3(IopqZ8Eof;5qzJGb?EEcyRVD&1Nf) zx$h16z&(*?Hkup_t5^M7%moC)(eqVttB<_1$r0HLZ{O4cKhn-H!F7OVWPtp-%t<MS znE7SM2zct9=Uwh4?Uiqp@UNzJZ49?yZZmRegNueBxp1JIZ^lChdSv}Yvgf@rs5x#o zoJNHmv?x-GedJAD8?czCrgXeOUTgM%pzfUo>KS$~w_xuKf0_cM>2mELN7qhWAcPKB z??L66A*Upix91=h8_@BleZOfQN96C#1AW33xV`X{9^_v3L>dK^KZ|7B`Htj-X_4Jv z;?4ug9o|#n45hDnpFbK{@y2z=`b5y~n!GLjxlWMJwUCqpue@^u5q<Ah8y_yTv*}9; zU{xR257Xu2&G^WdRjC#iL+L?<Aj)0rrKTmJ7ve$l4GawvKbxkrJ<oSov*)HziJ;}% z6-7fmOHsqryO_NBHqTv9ljfC%pg3<C|Dy=RWv<5WvlCrT{*e}~AMbRcXP>Kh4$8;J z-&li=D^ARZfGqmaub<PC05N2gPc~HiiM{-HUtDWKzc8&2vk1%yh(jOf@B(zKY>E$r z<n}izYvzv*JuG1cFHKk9E?+m>U6mXW@!t6pC;la}(>x33D+SmK2k_^KK;)Pr4Rr{G z#zGfAByZ|X$M}&%&(I~I`Lvp_u94Fp68DE0^jTrV=@uW#K&d|4jsO-!4bercUa5`t z-Lu+~?MLym(Ki7HdEehMP_>LQ#9mW*u3LTl`CvRr>81|OzVb|E&nen-On|d^-gQzN zcyy_DsMbUJ3J1o#x+82-JC5V2^+&6HT8AC-rS70pJIF`#35pP@p4IT3*$--Ux>d!a z_==sqaBSO#`+4G^4Ks75+czqPp#q*=tXPhu{9EB4&n#aOtTI0PY%atx*fp*uBNk-A z#4ct0wd~sr(w8RL*8a8TL9KCtg(J<8$|bUCQcZH-x%Y>a_nmcA+K#`Qcix?1)#mce z-WqcyfUp|AcN{OgQcJ#Tu9a_gKKXE$JzY1L=6mDj{$`dh+XXXr={(|&Td!G_dB#tE zQ(?nNvFlq3$gRq+O<MQApTvnG>D<kzf6jtVUuQY9&aMyolyplBo|idk!N3g%jptMK zu+6mqRnJPZm{dUegid3rP>7dBARA^)L*98rBu_e<MCw?|vw_a<g)a%?#diimGKSe# z*}oGMWqhQ4Sup45=k^J|4EC}o#G73;G4TAUR_U$@I<<Y_GRV0k{scV}l`G7;%)yeQ zFZD!D2VQCJcW)NbKeF874v2gwaB>RmAl(_Wx>7J81w!T5I4A?dhMw)pXzSGRJX1Fn zOR~z=(u~hh8HoE7^P|S<&Wl`$z$?r_tV9<ydj9vb`p2oafSr#=Kv8#-BPpoZADEEb zxm{}q{={c0mZHMDP<w6$Nl)w+Pt?4kD<d9`TGE@G8@<z~a+6+Z2x(V+mwPt;ZRIv` z@&(hb&P}zTS5K9TjaB*XhEy_N1+X4;8N^Q^l_tYz6}=5V$M;P#+nlbr+eJUIaS>8M zed7dVE+1`6XOIegv6+}EI!;Sg2qq6KiF>8Z^<cQ#@oUuK8!iWv<Jf+2us^*#P*(N) zs>(;~_<5S1j9A=dUee=gF9#K>Y0)}fQ?R4a_L^?Smc0ovq+~@PlRm!t`7!;PpUZI` z%~iNNYtBXcX}pW6`|Pd!*~<k^DwWwrav>x@UovYR?FuE}4&X&Jd~H<0W;g7FVXSuQ ztRfggicF4CKZwh#-<5>iXXcqkw$&upBiAn>YoSiVsQ^P0!)m_29u?k_j&Ghmv8moI zD{T8`EFTjRI82paABU2uAhlE(K#dn-75+>0dR7_sCRce#^8IzSJez?nLtSF`=484_ zn%`T6PO9sUCMs!N@p#e;dg(@;uKNF2`^NCPw(ajGP1>MsY}>ZeIBA^5wr!)aZQHhO zG>vT=jq&dE-qZG++yD9WK2JWZz4yvmbB;Oi8<=xsF<ZL4qi}zPGf(XuEVKpD-_Z)6 ziL(Q9@p+Ml(ilV)a!&`oH?si&JaaV&+eGdhGQs^ld^j9C;-#j;LQXCU5skEiK-xTR zVT)FW<U8Ki$0RP*uFbQh9?m(;PuBVt!HxDq?CV2)(<>tt6!bm5lwCqj^Ov2|>4xtk z(b01)m_y;_(%JOnldnLTIRgW|cZ7~iDaLbgb-er&5p>9;-kAzVK=K<xd`k$ZsP&qM z_l&`l98K;Fh$8T=JMPlwTvsqDRyi;-yaoGg@h~gwgym;(bcAJJ%otlnGe!Q3b^wAQ zi2?F*{CJ;1Ux@eh3-@<++9~|I{LwpLcu%UO!QBo-th}-eB~qNYA#C}5j2PvC5bfyk z?x&kq0+eDCNX#DaKGLaPL%ynK^|D>vvn;_#sJjc5J`tTbwOydAwcQq8<8{h6@eN}T z9sLyo<YF~Gv!HPFHN&(sB7oi#j*KkYh{trkX{H4|PflYFqjPx_p%`ybb(1mJhIW^+ zeSPcFsrLI9t=}RswPPEzIU!d$!PVIgF1*X3ISlNh5MWDVdHeto0$ocj%_{7}EMynA zDZVzW5FrQj1z7|{l1zLxMFpWoP06l3X46$urO`FM!~tep6QpaQS?)V2Fqw$P3Q?=S zYnsA1;nb`Mbo$B()^kCX!LkT_9^>CKqpMorHeBqk72o$<{*%gzYj23^qhVC!E7=Q( z2wr~g4gE>_0evns1)j7iPT%-5k!9q1MMSKz&qaSMpv@*1o<54+yFT3vpW(ea63it3 zZB)Xs)Xup~ntYv8i%h6-SXZp031Fez5Nab_%e?1BG<R_}rY(nnTBEH5m<Bu3(K4U! z1)kPK5_raZmJkCw22oN<0+0s=q-%VdGD27!Zf|@F4ePwCva&2}9I@72T3aQFGj<^+ z#GLeF>rcXs(6E`Z#x;^Kw1Q%>6_YvQGMEbOk64F{C4PAkKPSux>1oKXSrx3X0T1wC zJrVH3(XckM2HQHuSCV^Y8-xWg<i{--gvpS*`fEqaSDfOeULSepNmC<Q@{Q`D>@;N> zRBeDV=}nUh$n%aCL-fE0(#<2&+`XgAtySU~(3#lo*d){n&Ec=nNYbXxvrjCN7+07) zk_mxnKiZj*ET&BJLeCPc7XEJaA%z$<zMt()pPM%H$Yw91wegjY+5R%WX2D=L@iJx- zHz!lTI9y(=EEv<H%&&yZFSol!^d&QMyPt0k;O~Q2ZX*Yf$7<<(E122|Js1vwt!>(E z{Z7mYY{1b&hG(mg0>me^1Wxb95*H0-<-XKH(LvPIv!~+4qPnsU)lvq~h%liGvL9{A z525Iw0Ab{)@_NBEbFz^(crIoK=%nI}Oc9kesq~xQZmuV|UR*Bco{1led?&dND1B{x zI31`<|E5#rRd^7lc<sG3Q@Cwb(+;z$-%qOly3pu?H3$N9E7G29fB8~l3;brc3k?iA zRpiRV*@e8$AfZY)Vj3ThH3@9}N{1g$CZe`BS9v%?D_TbY4JunK+Y%CCo@o;zlo?e# z593+;?ybb==d6XIRO==sOHdW<k#SjCt<Kaq5(X8-3<E6zNK0p!c9gjb9S4zmEIo{l zEBJL>ACVW0@Q=T8MeyRgymdr*dD8QUBT+*A=XleoydRkUAc4DJy|B^~q}Fr~G2+WC zZ72WnA`s;zl^ULWI*X4mkP7X$lhhF;vVfsCcvo&ka(v^<q<HF#uBUh=_wthwOm87T z7hcao_?UDw@MPR-N#hBI={I@3dLw|mf0%ni5QoEggt4YeHD_G9I{VoJ@~s*p^AWx& z0ab@blY@HAZn%PE))YE=-+L};UQ-iZS0MrUNA)c)-*ldTPU6Mlql3szb!Ms*%tE5q zEDF)mwJVkQN;h~pum=h;H=}FIrM&!8`5}lAoOXXsEe^^lPBMDuarY~u>Qn<MM#hMN zR0oyO<T?(QrY!rG52%70d(nLjDp<9IU%>e8%boRs^^WSXk!Gz$7%WqlThtPVKU2fo z8ksTNTu-B~@>nyxp?zy2Rvn;~JMnRr&AxPIlyl31bH#6s?^Va{E4!2ZI=muR>p8Vy z*;}5lt={mX70=IK{R~#9?Fjet>mI;ZUccUj<FIqXG!6}|HFEHi!(6z-JN28GZNLjf z{ZjM=2QtK|DTlvL__q+$FQIrZp)u82xc|L3+5ue)UxhV=ConZxZvku&l+(y`koYD2 z14#UQZJ#}pi6@^zFM_}-fdT#qkz6{vV3f}6_{KbJjAB);>FERFXlqDSL!e9^mq$1m zW?dqKXfkIgI1}}Zo?JgGBjRE~jLGe+i5NwBAHpv6oZ9khjxibRCD*}_lx{;ON@w>9 z)3f+T5;R-q2tyjqL4Ao}Tw_5+I<O=aj<b3xvNd`n3YFqR<&E#mN%pgM4~WlL<5`K1 zOzTaNe-&%|OjLP*naBaRO1SN>P+jir5$Hw_AxUaxTY9&q1UTp)s<uk<0bELESOMDc zqsxZuB}rYt`Wi>KN64%B?c6pAG}Zf?Df*uk1bPI7nogt44H8ihldG##LWlUQ>Svo5 zDSoV|cBr^opHJgtvmM9*bRA1(p<DCs?``y2LRqeef|=zecnP44P(-Uo+eRSpk)wN$ z_4$Ad5ik=wJs2Ag87<hn{g%pGI}UxYu~jHLP#d_?e7k<>QA6;TjFnS@H{YKhtkyA0 z8M?O@Y2*JFz(zU!2f!XmSWq+yvVQ!px<&^s($kbGVbwI3g6Llw*hB$p&`ZUVd(nY_ zLux$6dwm{Y!!kW#IRKF<{s?h?7y<Z&I#A14MAR)`pQngVCTH=m<biX#$NE_URF6Rj zn+w8`rUOUTO{I%!e7}XUe=2}-VX)#vk3}Qav_Y{;75npZrIH@@;YHP;tN&yMW7D|7 zclK(U88Cbi0<{lnNAm;<Z!eHfbfYRBFk~SAAcWY!pSU%qKsGg~4XEPfHSw*kwHyh@ zvOQ{ymXv|33wd5<%J|l^vq1`-EVn#|YP4+u@xe{INJ{YdfkhNPmX_AR`gD@`55-9u z6fg=5K8)z#bHJ5~<YVhv{2wVnbR#QDs2vFtX#cC&jMKQY;(H9Ae(?KmjPxJ>&V(Ge zh!8s|UmFiG$>)+bf!xK7lCo#gh_x0$52)q1e>22KE$df&_os0Eyzs0M158fm3qgWE z-Q8@MRZgH@mSz-f(Yin<PPf*$EY8j?<-JR2I!6^8MYU3I+gupt)d*csSqHj)Fz1zf z?(-$;VQJ%U;gWw+5YQM-PL`Wdn~|sDmc{<KIYc9!t3`xZwCcT+ot`AaodtF<&Yh!O zo*qHi1&foDuNJZhe<Rdb<BAdB5C?|UDlah5V#OhfXCU)npi8Kn@<a~Br;0+*pM^R# zA{s6(==x)+{1pl+MXD+>ZarEwCUcdbU%q@P`qqm2Gi~-#+2d&jTtta&Z*Nd@eUz?( zhdpChA{Bs*aZFJ`$!J$6ONJ9y^tyxC_gfL1>QRL)a2Sf`q3*8iFX$l`-hQ^xXS(Ka zFkK=<7*&1)k_gL-a>J)a8P~Npkry$xQ=#4WLGy0}_mm}~PLk*<v^Sx!Nsk?38}I-0 z6=jUF(^=JvRMzL0iYR6SCOn_z!%%nJirGClh;mr80}Tl`i$`YkjnK^bJB5qHxMBLO z)jZ!+&f((uYFo$BzD0`xKF>kj7wf}f5yINxV;9NWBpkd2Cbbp7qN_okdvWA~y(M>m zy~NUd_#)N5UP-0P4Q45rc0ve7_=7AhvYsCbPGDu4+jNJbn@tV%Ba!AaCiB02lzSCn z@NPU@S*9pZrS_dVzOO_2s6j04&|UbUM7^zD>+I3A0@8hyj{D^Z4-wYU&!jy0dmwFP zP28}UZ9)y?1R)(o#o-ZQ!QZ7IHqFDPGd1{8D7!PXhqokHWlC|iOkYY*?i@vQlQe%S z$v}$Ri~%h_R)m==BFklwCr4$229KOBL3;|7lw7`STB_)?<W=bSmj;!>Ou3?qnohcu zNiD2Fa^q1t#}JVZ8&Uk9cMA@yXLz#YcyW?CbTal+CpRspWzzL8_+VtO-c!WdAJ9`R z+tq}B^xc`76&S*bc3A8cO8UvdpFXwoO`|SA<TQXI3!M9<CPjq)s=)IAD}DzA-p|2O zw+h)C5y(8=9Akp1!dRwnBzw^x6ViUj%N)_Z`t{Wv3Fm?a8UsX?n80IhL@;x{YX|k+ zy^N4AUNEAaihmJ8-4!CzyftuO2soxuX{T}#GcZ7STPU39lK3A)gF2w^-rm#D#|tce zH!6*TAZA&z1Ub!(C^u|D2ggeg!CtiM8PSVbx1Z(&0wa>y4e}G6`VA75n}BROvU4fX zB&J1({0a%ZjNMbWt~_6^h?pkPE`ua$U__Z7^*tWEl3GR;{}hAJM`ncq$$kboMBR6} zxO8`Nx;7QUa`Yb~W-F4*c`O6_#eydaGo>;@_}Bv^!?kSYD*Ve0m^34~V7XJJdfw9- zp{z}ZP}f4yqBe+tvcPMBRAB%e@p{)HjT8nLF#=lNris^&cOquOyQ6&vk4B>qD2nbl z7R`~Q4Vj$O@-U;X3ELvZj59ey^UuBu2dgO<lF>Kj{UwLLFzA0uJ~gUf736>#K9goi z5>p16hRD6UAdn%%%v&+H$xUZlX$WF7!-@GYPU!_~UgPgV;Z+Yue<;TrF;kOf5N*XQ zzc0n+V}N&Km%NR5;nR7^nd`Bb9^*5R<>O6)67bf5qr*T4VldsqM3CVAH8rY5$*u#5 zGJBoO@pl|V$V`*WsHJ}0CKPoIQr5ZUYd3}i_^=iXNX>vg%y%unv&-+~2VB>K#L9DT z<607;R4_ruHx-By8pny{$y^<nwC$?|kNSTjm~MFhrrlb6g%{KX*DtFCuWzkkVJS!( zF({tNx0nNaU%!%=aFo(M1D*L1%g68zL5BQlPRr(%)V$avF*cg|#+xba3GWqzVAovf z?VA$u+u6_aQ_TZfxn=gP_Q9{yztGYq03$&jq9&_)y8V>fe~c3F5dc}NiyD!HbA++f zqY&{f-+ZyQA3$dUu9Uy#x3m11=NAPY-C&_wqR3m@rP;+py=q-dXp-b_Cso(cb5ReS z+C-e9Z@Rdo;m?Su!Z4<0Gp280=@Lr?vf-eK8wE+QGhA*AD0=E=_X#?|nI9+=z?sh` z5nEVJIzq}SdixEG;heb-%vjFfgy5T=EzN@CAkt6c^*<N0<RJ41V<WWNkjVPb1)t5w zwmgg<B`M%p5$B6y>g&~7gIzpvt(1dG_QG}9X7~`3a_*FB8cl6sY+6!G$8=`iqEwgp zOXf3@Alvb8IQdA8m!?&)w5eGjU1+OV&z@=3O2i+<jX^buz-b;)Y{v~{%`eZ*jmozE z(&GOlUwbb0SCr015?}UfZ2bud*9;J(grwC>`xF8%sd;O7t9&|E7HczLAUM`Vll0M^ zLs5vntXd!MvC!gNF3S>c>zzR`>+z;+8V0>;%Xks9@!GOe9Aqm>Vd`$}V_^GMjef+0 zBZ$|Q-Ctzk>)X``;xsp{Ja{akJ`S4&S0UK!>$)Fyo7r5=4wyY-h!aS%_R&a|kLu~_ zj*ki~;969GkPmq&GEeNB7}8o-Oqs`DG_|QT-FVhKZdo%eSj`un45n0kio)W2pZRzD z^B)bF6w_V_-hZU!F?Usr<m9PVTG=w6$Ts#?VxywOw+TW}+A?}<i=<lm`a1-!e`^Ow zWfbUV64V(K;X{0^o(<+KXE7I=C@ycI@?csYb-cY=WhL({#6XtRoeeEJGfP4GQt59w zU}6%RSMXwaKzB4q$exRtMAc*nT@o2ADdNnjeUpU%qoKv+?3-Y3p!hKxMUx}dwwV09 z1{wBJ?*(JlGXr~SguBZ;P%j^iAg+8@mB3a}IYK))yn259iLR2803Z>uQ?+}HOmuym z?T0bdqE>8KNH5`W9_*+Ai@{jlg3p;b%J2g^k=6{O8_|DLJAkvq6%c_M6v*d-79G(? zC3Fsb*0d_};~2g-s_5pSDopxf1v~-a>v0fyw`O=I2%_*O??XwU8A;-=Yd+ne*v^u4 zC<$fXJ5QVX$RO`e`XBXgKB<|`mE6PR>{!u{^JN)k8&F2?)a7=YH~CX}d1m;x%-s(0 zSZTZ(7!`fPLqttK8B?GN7%U{A25wl0gGvc&P=J~RKdaOaJ*lAz%j|qbo|v5X{rXy4 zN2ePxA|zgAjTKS3)<iK*TRQJ|Q_M)@Nq`i7p|V=4Lb^aNc9{}DYD|4M3f3cLMg%CA z`8FhLSLlhbd`jaryZ<>PuI?>-9(j?z%h`Et(LCTe+XNaWWTKlD|8aQQ`Jq>Fd-DXw zvN{=XeSK@KDprE>jX@<CBKo%7+owyE^HMv5E(WR<$bBdBB)ZV$@y{XDpc#I;7yvA9 zAq5iq*_D30Jk=pt)XK7!?_B}DDSEwWB+EA6aHpGUXeAOqFVT;D94rhqBU7zu5A0N) z(T^xZ6-+<ooxaNLQ`m<6$qnyA8P@$fGl%Dqo1`6yGRSv?-<TuPOOQFnMZ3>tTCW)H zrw)@`tW|{nW?ZHrJqZxPjT!nvn9x7qL0gQt2zQz!u}3x2E+0t8)u4@2z2Hv;NPux| zMlAKZd-1|4!IOK{-Ja0eow<UrIG%{z9?R@EjG2J#-S^NObTOD;Z(TSl6Ep6%orRdM zb_c*+70m&3D9i7uMu1oy__Rkt^pE&f(W{h!UNp0p1z+!P)xEwr2fVr{j!S;-N_!sh zH@&Ce2QrG$6E=55^{B4i9-~5xn>_y|Y|#qA+CabJgRk}>*@GLGn!_3>I;#zs`}4KC zd0V1u6je}`tp&^!|6eHj1san8f7<s%&*VxuV=bkHWq}31sNl%Mfb-Tr7p2v0p4lQc zmM>NR35{Nc8rS~yWOZhYW1qc4&m~;l`GfoIjv#WxLt42p#tn871Gi`+pi6?~3ZV3M zVYz7TEOg=R<==O5Of$dGIYib60c<vCd+Quj{0^g{Ub-N2&S18W-8g8Aw5ivz$ocoT zLSnDZrpcZ@FI`!>XfLv?wG)+nGy>Ww^#8jKZvO_OAPosQQSzCTh5#1{l0QDp*D-jk z@=*IoRlT(G$kagjw-wiDfd$)ua(|$H+KMpW_{8*KHHYgN_UUQXiMm3eta-}(H$nly z#0zT-fa(ObnmxLbbZ^jj8)|F14PSA)qi!oWEB)`YQ(*dA0l9=5&}GfKt^$0@CKIgI zr<Esk=Z5DhB=O>;4&<?orsGCPAD#|3*Rvj^8L^cqslUDO6pXyU$49=vF+>3GsL9%l zN`8(7ZR+E1_joc0r_<hnJdJTZ13jfdaRsbkSTzZKs<vK4x0;zyh{CXE(?xVM``S9< zfPJ#khUI?a=k5>K(&Jix+<0*Kpa#iXwsrO+`uLO9NlD!LLZ{n$8RI|qZUBs1F%a-t z#Pz~>@ALihRou;tyu?Nc?{i?;(pg`v$HQBuBZb!-$xV<I3h&_H;OsKW+#e5SE41IX zUe%P%-4m{$|5lIw!gSwxH0E%_XCPmFm-&4;LSTecgWB4zjc=gRp5HumVLfyoEy~`s zojhIa`EN2zZ(SW{PZ%~|jItpkBXj%1Osw4WG97k}=)vubrgRa+seDsY`FnJrJQrMu zTWumvFa-_>W>ypg7zomBfo!=?vrfa?*Qv$Q63oTzRCs?m;CtQS21RuuMY?O#HOk<N z_vnd$7e4>|Ql(2IRj@g$2exM6&@<mU>-*;ITg#^hM|2h|kgl$-+HjMrzTVzL%lM7o z0Qie$Py++Jno3^e5fG-HEeCON!|Du{Jh<wvCWrLoeidUh1>->F;V}k*%^ajCm8Hf% zVIImPRWiL6ZChJFhzAXkX*ftvm9Clcqa^DSA2zx;rIZ+%@<^#_lg0id$-_2NTgO_L zUqimlB4QL((_v-Ik-rjPn*ZI)@-l@Wd~BdwT%diMmoMvyfrCF>b#KTO`6!iqxG?Gl z#5OeA5;wL)dO!%&xvj(tXN;&n_R>igg({7`7T~dS(@V~J_v3M%j^OcBKy~ITp%D=< z_W`gM-T}ALlK)ViXh0)cGYbeZ?f}!k2=Dm{&pSlH<d9S}+p<Y$(SXs!at;wgp;WC* z{dBfK_*#?`&)_C81(&vj99W=v8$}Kq#J%XF=7NIZjRwt%6$ee?VVZu$8WhNp)%zQm z>Idrak@x(R*E*%5fO(DZb7!7yH!h=={1Sf?KHw_$*=88OLYf~j!g)LmmoTx)o;9<G zxcgLt!<eRWm>UCgQI`15GMFN^V%p%7ile4Q*{VF|iyA|78TQ;9ADA#S_$IVqrK@`i zZ{X237@544SzSqXM$&qFC;5{3zXcWkBD8`&2)t+WC}n&jpt`BN0?a|nOgq*>m1>2| zJ|hc?nx`I*;UIrn9~{Qa@~l+}^5cf*V*Xr2H)qf~4O#xoZonbLlGrp#w*T<WUO^$6 ztxT&jcp7W*``?!FuQvf=X99qH7;BTpekmXb+Kan-i;)DjCI_`P6^gzcF4hnvYzxtZ z!7!|xMpJI^>9~}Cd^R9qn??X5i5{c<kdJU7;qG^H`zv~e17Ou<=s2k_FXpWbB>1{Q z(SW2q4K>TFDQbXeAc*gnv$fn+NklUF7&;uuZar99pg*;+4<RGu*c6@CV3R$onf7le z>8~w$(@X$g`QZMp;QGaKBzq`&k0OS@23la6y<lhTqJ^Y#45F&pDH{w|ovI)3dC1k# zs+vtb<F*SE10O2-_K|_~uK?uFvA#$aItV^C)mz!AABk2-d=1?2qA66KOSBJf8o)5z z*_DH?NQ1*wil*kP<a8FugJ*cYsP(_EuE)^e3W;CJyr}5myd@!GlFKGC;sfL_@L&L) z+5&ISMGftMeIx!Jy)6a>euwej+>L@5umFDk=X&N8W1uyP_?>_u0m%;_<bgw~kZTm# z6BoXqbw~y)b`Qa>cF`3uj*0m+O7mcSdLLWUDKP8m^HX7>Vtyc%H4v=|WUGe7PKnuU zg7<h`B7c`7xg-zVaQS<PeAgEizy`uYM1z^b=aSF4=+>!PF=2rD;9~+acSuDD+c1@% z!72!?929iGQt~Yz({#bC&Nxl8KYv78F-u7nztS@ofZwU9JSmiGS1<P_0O6&ix<frm z4P+F#f9R_D7apFM=RSH-`rJsKc4xEbw2-T^#9y5v`kfA&Ami--QVl5N>*)B%mnGGF zK1kNDL#u*s6KdWtJ=IUyTIK7bsWpo%;2ZM~0x|;eW0zeaNZ!9#nDSS^ss8Bc`uW|l zps!?L0Fulg1~{tSjj6Y<kA{^sP^+Ro_rCF43SBJ%;W?nOnn;xV-%vmmOt=R-8p_sx zOR_ZhHkGPcd!2HQU;M9j9~f%j0={pM+IdxfV@3@E@cvFM0>dJTKMB=e(xo5>;Ht8$ zpq$cw@!tL0MPLCNN+YYMVt;9#e>#5vbBPBG8$L7MBQ5_s-2U63UNUC{FUCoD-stb> z=pVOD!@P*lu2w<OiwFgKjMa^@79an4jNC+kdsEn}iC)C!6Hoe-Nyo`V+AM)*fyh6e z`k#+-As&+<vb<%!FDjeXQy?pp8@oKhFTdiCaRC!xz^s@$_@|)(x67q+XN_}LP4N-K z`>vP+RaPn72BjnZak`X=9-SMMs2QX$mdSG&WWNTLe5evH!qx2-=Q?+DvsS63;e&d6 zMeW_<NNsZMU89x$VXa2yzz=cxOOyO}{D6=Tfc}bQ&Ak)=cESY_#jm`TN=$06gy>hB zFkQD;$aS{S3w8CF6I101OIQ2f@i5{5DJqzB29@Icj&J_CgzY>p#v6<v=5I0GzgXpE z9xqc769TAX!scDn%70@9a&N#!Wgkgtlz-#CS#JRMng9P97It~GcIgZnG2$nK8l_K) zN6ogkE}Y$3QXs(7&APJxAn89oYG?wSFz(Uqa|1?HNMpx<Pg8Tp-0n^R#<I3ABK&{- zl{<oy7kC^7^abdk->LxZ*%&Df=+P=Vm^zwL8Gk8M`3>j&<Ang$gL&}>ILsm}W)T4= z2<a9q8~3_>pWpAF4E;C4`nU7LDY*H>sSfK{*$GbU=?MC@JcJH|9@@-QdB}VQ==T44 z)30wdB0%K$xJ##uIGS`3TmO`v|82Dul3s339f1V+{i@J0Tfy9BHHmg<E}YNguErTt zMrZL@oGaw~h)E-i%BK~rIfWJ#7?O_vVUh+govOK1_pH~RuZF{P6;JQ=@7rkPM_38} zqGdTeNa6?~s)|C3TG;~Ek}Yf6TBCIh)z6=yfyr>O$F_Z%uW-ZB`cuZzjV{-?k$GLu zP~IiQu#fH_OJM|R5+fv7KJd_zHoUWv#ip4J^QYg0{Frb3GG7fv@XtlxQ@_iTwxpHs zDJmrVVGYYafEW7%RBPOv_txBC8DA>~2q1*)1ezcMvR6J;HK#l7DAqJ2Z8?TR{5xX* zANp?Nmy-jZ-v$IP?fOtgGU<mkeQ+|h)ukFRqlgn^h!zmaJdQm5On_UC0*-8qIyW)w zz7_XWA$<{|qF=e>w^+Q#b)eR;mmFF(ymkf0_Q6Q-uQk-*fT6aNsok0+f4<7=qpjjL zuj@$I$~{nch}{kj($A%%O!YW!3evBSHCDJAQ#5w0K4>!ERw~&#s38ZoB1inlDc^3T z{8@*Vp?0mhhP_Vs^_&R|@7`?l)C?&o{HM!>%<EWiyxQFWlprGLeJ~9(Kvhw?WGSGv zZr2UDpEiv`%Qi*{D#R@;Y0aC*h`o9fP-xvbA&bUNy_R5}3@@cGn1jjV@dJ{~sJ!IW z<Z-rYZ%6@>@>RHfC(og{4}vwr;7_G2eCZbW1UP@41@Oh=QTN054!H?Pt9|C?eYT-Y zdVx+&Wo|X_Ac(S(9D+STTm(KG<&Q;jfFL3AfH*;YR)dutu;T*r+>lv@S_!Ul3O9a= z0ImU|oKZ%60$&BAD_-?iZjv}_2_u<^Bwf*Ds`FhNBYNY0ecF72388OtSyBU<oene- z1k(aI4GKf^x_VX1@uZ~y@ad^bpz(C`iCs;u`;W5A_zp;!mm2pR)dj*%5oGh2Y)4Ts zdtxAzSItdu8`!S#K(U=9h`e)mRRg_12qVGjH6Tk8kwNdpVPq(44Jf1b7;iHa;LB3v zPu=;9N)b)x$mb`0YI(*X!Ty1UmIdWKPzb3NxxA@-;CQwqpq5Gwdd~(J5boB580j&B zLzp+>>okPuAN3M{vVeaaAU>hQ6VO-8^-;?3mSfJS@57w=r1(-ec}YAg5+TR-Cx)j1 zL#EK2fatGLa?U=y1XVQ)3fhZBY}4(2((qb0^^t0W*RPtReIo<!*Cg#@2Typ*Lvz`) zv>^vJ@C35Ia~_Oanx$Y^jwx}PLv7lv9i>1yh17R@2CS>iR8t6_dh|u;nn-ngo`*a} zz*-A<OW?IG>1b;FlF_80=6JfSFG&>eIeePy1S67Kjng09ioa}353E82*rrWlr(W1+ z@>r+M(X^Ten0_@Z^6YOo!4h)gg@_Sl{Ltpsi5g%uMz(#CxagMrnkS#Vcli&q9{GdT zEndsksZbvZx0!uZ_g^>n>@2TZDV{bN)0VN{Uc4=sKeW)AdF-6c00|kZAW<&EF{V-y zTG`S7r2nGr(8#P5^X_y11vYuU7uMj9?wxB6pRod|w6AdmniumX3NQK^>ct3~QE%Vf z{4J#$F`8sqsduNI%+`Nov?3v_<{p~5Xjg1bp`NsY<oGkEEXmAbRsti^q;@ERuom?$ zd8R&9bzk`28@v|CZNF)x$6z%%^mBMt(|=m&-}2%ruRIZ8!u1bkOghg^w&cmUl10;p z`45a#`Y8`e9QEyc-?mYl*Gd*l!-Ujsn27Te$sE`4uaoln9oUlg1jDC_Un<lhO<Pau zxwoTdE%{LScXa6Zg;^G<`C1Nxc|YXKic}YZSyXUQe(R%C8et_R5D<=2XrEP*G&s&N zl&Jf6Xc?cx-4w}UKfsU9VXjR23aL?O88^{uW?E}h(Pa)>nnX!ar${yL4*DG}W_+ea zizK&C(1Onr_ofob$-MLp^arK#M0QE*L0LS*uS=cc45|7DeQG1Ym><%Eor1J%3w&)R ztD5t1=oIO{(c;Rz3Q+;?RcwqErQ@0R8k4vWbU=@(JTti>ohbS@W2(aJsS6XXT>*C9 z035rJi5pc=@jkBvQt+$ew#j@ERWvs)y(c?I$bt+tCU1H7u8x!B=BPxvP{r&_w&bG4 z;4K~vUvNHp2##dG)T?bTmkqc}vSV^4Rek8QdLL2Gr$KTqHOBFw$@~V3RMo7sK9mj> z$a`d;;r<<!H&eJn$QVgHwTcg8@hUgYv-+ZS8UMm_fJj^#;B4gnA80|oGqcm!_$h-j z+q)v$V$gODY9S%wGy0abBOAo*4gw$aUI&O(R4{k0!D4;_vSt)=$}s5Bx3lu9AuHF{ zmIAs+DE^CbqvklfSxY_9Z)q_oGYzxDd28GG&r#tcm$X~Cp$DS1jbTJ`p^dks=sT^H zQOaBSNEr~;4jD@UDTCiI!auGR=wImB<h=I3dJ&R*T`<$V{R;lrl?Tj~_ca~VEsxNZ zU$_Co*X*$hn6O7gyx^u!cU?l)etYu6R)nA9i`Et0W-c;|Oo$V^C98e(zUcLeQZ$)3 zhahTQa)r`R&P7)!>#BwlS<CDGIY{xBHi5)0#|5@3(OY!V&)ur6kDpsV-@b9VWX)ab zb{Xg+UIT^KW(bAmuy=(dp|gL_`bYQsF9g^E27nC7;(1OrAzBrabndOgTS45ZWx5#u zlBolPZg@5LK{f#jgGmYGe7nJvUi$y0qUQ+rkDIwi3CYLg)NcQv$Nw4(;3qOc4Pj87 zv@t(KBtJH;hzdcAjG#>ly&E;<TvLMo`z0?xJ_oNn8@RF}n;1S2fO_qq^p)Ao_Zq_r zYIXc7qCd;V{hQ#t`b3H?m<7X!t2ucP>d`Fi@frv+kXwyp{ug-pADG0aQ3b6a4K&A1 z$5YxI!ZdW0W%*}^?uUjm(E)x6VP_8cd3MhKJ`eECYA3*(Y_+gs{a1j-NbgBdS3di- znx6TEBnlsaV-Y-7PUx1oA&qOMc&g_3x>WmZG>HnL|2*emae&KnXkzmNTm#%kn6QvM zmlid_pw3gCND}kWDsmw{H8AQw`%Uq29^~ZP{MkRCZTUF}AV<N!>I2x?;3r_fxD#*v zCx;I(GYt|jAzcNCz``G;0I`X$RLKxg9T%6YT}OE+sOBXmYyUjfOKfbQhPUv#e{^Pl z+LBkW;kHOWKmGCI5)ejhAVSprmo)<XABgu~>k=F5;UmNj3|WFXb@#PUUb1*{t%}E= z{|k3jC{M<s;{1*vJ}N)4Mu!3`6JT?1jtWoB@gz!M4-D`anOyGZ#oI1_U(s8_K_A$~ zOgk(Ww^{_qOf}NX&eu587$(mUF;k4F44+AB3htb2)D(ZoM7bvT4X<W5uBO736ROx` z5kj<xEd3IUUuMPVE!>52a#jo(c~?U;7yQRy>k6g01arsqVRQ<_55xV}q2~Vb>aWHX zX5G4*q$n~~jMLuULkINq$22cGmIL~;!BR(!Us?I`LB?o-mxMbv%L)cusGgw4!;U7; z2(gGYncOGOpsVk}3!8W3e37aiR7A&0i8te{0X@MyKMz8ls|D2z!ZSg$DpKs(WYg}C z_7qy7raXeQM=(f_PU)W2v20YcFLFNsySq`$+^bF6etLCMrFPJN<3a(y?D*ExGr{$h z%V|xa!%jDdG{@a+ehdxG-G_5Jr}SV*!qpaaB?kDQsaktaD9u)Ruk8!;=PQsbnAW^` zGvua^*B3f9J=crF8zpkDr=)f7s#e+4$VQyUUGPnxA9gR^9yNZ6y}&e`EONn$9k*y& zcMYvs8QZ#p_YHZf69+z@%$fH6!Cmc<(5KeG7!V_BF>Pj${U6=Hz(HA}a^V+=<D`rn z@)U$FbK0LbWCl*^pjZ9ExZPlTP{+Xh9Fn6r<hidvvxJD4*K`Ww&L`mB)V)#1=A&VK z#if5dSqii6hu7q0538Z(1twEnnh`#br9FBwoq+^Xj|}|X>2-}#3Rd|77qFUW2->v@ z7tvjZ#w#E<PaB-Q*)cSSBX!hx7v*9q?-eEXyc1P7m9r*9yJZK>>ux3Vs5^W>_jPXK zRG#GZ6Pt7ER`}IqGXFyzp!cIqx!V#GKJ^F!@wgIhw2<!1UK$=3RZ0U-oy%m1!KXp@ zn77Mk@6U|*NX?(n(#w-Fzz#nQ56fgFH_Z=7bQA-F5KsH{XUiK!2YVc1KxN2kpMvhR zsnKfL^ndU+uv}h1p<jzp9GH;|-q@H`ZXLaKCtXfeuLEuP8jN%C$<N_kL{y*Tx+{E$ z&9r7OjP9xsEki&S(Xt_$9PDe7H_@|L1mBReEim*Ngzn$^*M8Hh>RpYY2JK65r51dA z=LY`$81FttsMqr8Xhow~V~|vp-V^aT^)z)YIPBU20pVmsD}gF$m_w4k0zTeN!j{S4 z>;iuUPUfVcbVKurf>e(-pMK|FWv+51DA_J^+j{xaljOU-`iq8dMbl$cHeN0gthVQ5 zZ!hL(`;$VrM)vUskE5-#o=c)U4wqp5i`5lSfrIh;bx<^i4XuMB+6wrh7v67@3^SsP ze=|bOiH8GLX17axC&){@Rdf{^pjNTtHbP6A?2`1{*P#Fb?n@potR8>^VYdm42QEq* zb>7P7txPc)$g)W$BU0_U__XEA>R=7G^T2AAfR1_?wX$9}JkAUZ-?mX~Oxs5dMgXIL z3ydJ3H-*M=JVa3Fo+YczcO_CYm<N25dd+D;Wf4B+R_43HjyDb2U1JuQK=s<qQUXr9 zjN#0aYBb|RaS}{H7H)Yh*o_yBJiD0{XRK|eK8k0YC;@ajalF5!nS6261=VO8Vajb( zSfnz2LVEmw{OH8sD2TBL^LK$gKJUbYwKwAE$tY<hVOCFQnaEoKS|0;X)y|QbWhM{@ zZ_xpH8ulhhNU`I$s<DJaAmZ@GqwFZnn4u1bQ}w=TEgiP@fCJAN7zL6;+gwrk0~_4b zkzdDM;tXPk>9{+Q#%IT-wJ*T&LmYm@mv)7}KxT$L=&J1?7Y~(j)en4^yWqWweOF6p z85l84&=&TfB&1pi3hp+Yk3@$&^cneaWDHz!HL`>~!Ijy4o4wF3yG0elogrfF1gL6Z zx=(}I7wAo`_l=^a2h6Cppr9CdP7t9aZ$icepg9Tb`bwzCTtXA*qN93MXa0RGob#b^ zbi~L-W1VVQEO@^GXv(!pi@&g2sLh89;aNVfh+{XmT1Z-lE)Wcwk-LUS_37%BI?ksh z>z)mzfHbnBT`d9rzNo7S1R6I(sSPBf38LE@ChA_Q;6OWd#=8|(@O~uMjUwP5;KWlL zIQhm&D<J_Wsar4rb<ytn5DWos<6FMuyIc(uP`v19lFEcinW)!X55c9>vXg<1(HbT+ z^WHd5b=RPs@cAJQCWFJcm<xi=k6P}`n$*v`Xz(o;HH49_;0noK-$)54rwf)v#Sj5Q z;%azn2lPSmFl2?V>75a2)ChKu5HrRAsx}EQCDvxc(YEhCsILL)GklvHQ74UH{rOT1 z*y!RCE7c69!}EF+0@qWR+^z4JIo!1#$g)6;!774vd2Z%?;-}ER7S<i@Af*EZYU$~r zao2Y-l~h^3=giRA&ax&*bA%X6AyQr|$CDnj?M&np7;l!$!0?9gvd%vb=uW9F0sAWO zs`L|ok(4INyhk}-tW@^*BQ_f34Y<%fP7Rk~%q8ymgy%SiFw}!)XwcU4J*j(i8Ej4) zaCaF|J$jc|AUtuA&dqr(OJDV9<^<EWsAD^QEPz_k7+ZGZL_z2gLPt&^fMAIc!9C>9 zsGBlk96{QcE1n4}{gPx;H~1<DnQI7Td{(fC$E^~Ni*6??B<OM3GILzHnvFgUPamHt zamgjt+D#N(sWsXd-o=r!O;q7@N2KojRa$Jh7}-9kJUipt*)*O|iLR@&PmlryZ-X9G z=3^f;@o^D0RbZNr`h6FoJ(|}j3*P*wssdDace}L)KL_8=szyEgZ+vu9M!%wQieIOD zyh6Pst3_^!5WPs_YSS#G+s_;h;GbhmwbF+SR$65r5q^+s(BC}El$?~>S?pfd>6$Lz zNf@RoG(uRe1~Hm4RIe6;T*G~1b_1FT2^St>>hrCaNTal>n4Iy+b{b4R^3{gFDx0@b zjZC(ol*^E)lXL*vGi})21;}XZ>PD1uMLgd5WYNQZ3J_2dN6o&9#FvOF#1do|0T4)E z;wM@IFW}5EH=Pwc9l}JnZfQeF))eu}^%33o)?H2Djq2l1UvTMNk@hcL7&_^VHyW{n z7n6H{wHqrL>U)Ikf?--zYP{7(r7rV0AX&38ooj)XTedH`hNwt_Jxo?|?WPHW<C$~| zncc0>zsRcyaP1)rNW<Hs=q@I(9#n>%RrOshu?09Z_UbYCW%ac<w^Y4R-YrD~O1OSN zmx)y24AXGOvK`uU5)<1VAhi}Ph~ge%H|;f0zeu2J_kT<bC}oNZIst-<F>eDM{zM&W z)5y^U@<BJxM0I!tCu>21AG^stst`UhDcCCY-da~bjeH?4nU;R0XHPU(L%&?gf9;tQ zPqZlw&bV5$8uy*|dQ|a8Crk6ou&@++J0hIF>Y&@d!L8O+UuD_5un#-DU%+CX6$yWK zIU>QlsDcZINVk}zAA`4qMbwC{d;~-D*-9N4+gJ;Fj?=?*l!5ik!N(rk2+C?eb~|Tp zWu89RGEv?6Zk01RW-HvD>OV4b@oGoMFk6}gA8xeM!fsMM?uK2R0_}<!uWrO^q;?9# z@y(IoISRcd$Q9;5U*FNg5!dIAdHt-KO6@s~=6sI(Ktv?ltk@cH8R*A2tBU4Nbs9q& zkwH0@$a=@_ZocPi46U?C)Q0+zm;o%A>^*FQ{>1ZdsyVyweRD?>oWH0iN=7rOdpmGh zuiPP=+c|w*4m)O4B99sydndu@xE(Le1VkBWR`zPZ-*p*G4@M#iHCXgGyGOfvAi*i^ zv`Vw@d#1S+JLXQD;&vZ-Vc+CBaQgO_ZCoySRz!IAwk#kS9GNoqdE?Eae(fMee;3yD zs*O=cwB0(~aHg+4lahddHX}69(*?~+ChDUN&44`)5k;;i0WsPpu-(<Oy4z6lv3;i8 zq5E`g`LKB!3pkka_IO3|qeFOD6i5~J$K(ifH_m0mY^<2LL-RC4(}-=Rkq+d?QUfzR zckLqtg~_6^P;%_={3$t|zX9u1a0fapIE9%AV3<pStao6;crvG{`G%+9m4&9pni6ac zwfsKgPPrfS<z}S1winLE``T3aqqU@&MeGD{wxX<LA#^b!daq3CebzTOqvg*<72u-| z5I!qQ37{qQq++}unp~lcg)(OUK3#7}!wm^}l7ItO!Qps^p``Tc9rkeoY_K##U9NRn z?^p9f@c3dO2>Ig_&LKX}T+7ib+FHort00)tJT*4)vj#TDFh)CLAK`gKW{zFAAe$MD z>6U!&A{jZv-p0;NlJA7nrrI5#@FcKFMJhC8sdYJWB^u7?b-FA`%RS{BkpuDfS28eH zNXja~RG)bYdVByNL}l(i2%`<dXn79m-rCNdXjbnFWc>`TVQC%`Z}I)jn}xO;i<0+m z__|(FWu7(M=>r4xJw2#SPg#jMUvi6eL3dFWof{d449SvB5qqo%-`Xn4ejb5B61Hfa zy6#X=2EkJaNPug{mw#d$Adi%!Kv0#?5-@a#0_CigTlI1r5%CN%izgw#dcEnFT+nqj zXZ?K6Nu40=35LZ{88lJW@Yud6yPN;Z11#1CCS;xNcTD!{VcU^f%fGNu*jX(67&cB@ z(lbaLbF&}V_lCP=yWdNEvc#`DEc^hdxppUC&7JCaYPe6iT8$3BxLk3gi|?}>ageR_ zrBZ_Vpy7JN&=v6b<*V8wvV9eoVELi^Yh!#}60ju88y=~5wcklb_=iPlImcuIF7B*} z1TDWkIxZlD<%KSwnRcpXV|9TGQkPz&X2k*F)`sxG=`Oo(W0J>^mcLd+N&d!lIHzGV zJ=?-`okuW_S)D?CQwlVrl<|g@XgDcY)yL6GWE$Zjb3IOG)VZgRs_2mDa|7jw7M310 zWHr-pn4*2Q+{cW~MXqg%%G0qJadqR;S@W}@v`_=(AmBPvl&-NH3=H~{E<y;s2(H|k zi_+?*s?9dmcyh<1LdhS=l~pgWcy|xIrTkY|WmfM~w#)q12KBVeMpdl`9?kbRvV^Zb zR}T2RftcXbrVzlo&Z{>7LV9C}c>CVUBdCHeO6YA-zSPM3uUw<4WVBlC4`WwPjtNF1 z-ZjZ&*po<keu%CS+P0<et>NZ(Bm3)w-#ltgmGuPUbK7JqW@!vntjVWve#K@Kcz|u` zuEB{|u64RC6NB2u1%}BN5i5s9ivqO9`Su#3SnP}9PUo^M=GR5X^!dl*AilQMfd^T0 zIe+Mr^gsdhsbTi5k&U@9a2rNQ9^}9ZqXR*C;QmNI|J08oOLgJB6zwf?c$GQv&W2h( zy=OJ3p=V4MJ3=?TZyKL}8oO4d0*-`q(QR}IBWxQ{B*FS{(#Gb*MJl+Xo7zrYd?4lG z!s9^L*u`=MWn}dm1}~$Z6U`TGBGDG{{go>-6;vqP$lbTSqYiT|>Lm1Ghs%cUz7`N) zVLN8H?Q9n3w$4i8+LHPVQ#6Vx7nR!pIcvTXR5q)H5^^2YanIa7@q6c<#e=0QgomN7 z{^jZ>gG?D9DzFnjmC@Vbg-2D9q4&A8p0fN9lNYX~Hm%zhMvSc|IReO1sY!<tbYBA= z&rqPOb|&<*q&MM!wo?`olo*r>CV5rhzqTl{LYc>NoY39)qJ+QsQB8*C;{oF}la1_x z<FR+In>)}EenB;VVVzVMSVXZ9wqv=DVj}+S(_r6}$(Q8nvkJTut{BDl?DEw_)>n>4 zX)3$sG1`~BSXh=9ekdKo?pGI7-o}U_mZWt>MhSwnC?kt!CFh@^YH<EaVPXTLMRQ}! z4Y0R_+|ncN2xwlDP!F;YZ&}r9&@!A}-OhZ@ZR*(VFzMhS9qrfU6h^X&)(+&b6)@Xo z>Os0pMO4YLw<Pc22R&k;ysdAHv|r@aFP?7%r4~%4O<#)YuleRrcOE<5UtMM)y<k~Z zDeFA4J~9Dwh`Fu(0KbeoUNF!^f!UU#sM6<B{N=#7si5b(^dK|S9b*YeB(gs;YxPq( zzhT)sYRgvdPsG^DD)Hv^rWAc|PFmCk9I);445h+r?K+;*T9|+$@!1a8+v-pJgNIow zkFyST<?1{^g7`jpWq?kbcHw&CPebo&2rBI^f7C(*f-F`v9>K2yLLoyT>lTMZG!%fv z>+sTfhZ-`@mdWt7Qh?kSUv%G0*47?FRbNh5px0IK@Y(qoa}8!K0!0zUar!w%*XN#% z)BR)zbM4su`N7#$u>ulpsxwlhk<;_O!Zccw%y%zT-F!7Z6mK(#v-q2zeAYmqCwutH zREsWnf7c2LQD_C$eYrBpCvY{MjT5q|{Z;z{xpttW$#Mc9@C^!JvW|iLH_Gu}EmaEf zrfqEq4ZC&opf>i8CtX&OmhUdeQh@FD>u1dYZ6z_pSLb#{O)n(hk-eVAein^_L$;4R zvZOEb6f+xZghnpra<SBmDYb&&vwX|*#KnHGnNCw0(e4d07}J{f=q;vT>d%eKshHe1 z_Kd4$s@q5g)_eq&g4WoiRgb<258iy&;n!1E=1{Va�uSuvyi}pk&UmwDds}JXv_j z0?2!&Q@%l5Nwa)9TfRQjiwT>EL=9+?kvMVNGOl|w5J%Qs8Cvy%e=WiB!2Y!UzPRLM zZu?4O<T~bN9BTKL=Y)3w1CP#JFRV_xKW)5^GD+Bb0rx)*13VGHO#0w<AeMAtesvo- zGyuN4vtMunb>VnDe#@S#p~s=Bu|R<>Nh*4r(*(M-L3pACY}{JreJzCs#S?!AvWq*T z-$C%FPBOpR5+DIV^iv|M{M-5aK(M=TfaU@JzaF)U@8pW8^52O92I7hFQDberZWC=R zHOi;Cf4HOTy#3m(EIB^F+;Iho(oGPRPJi=M)L`hIHsr(Nb6KU;wy5N~q5_%y)^r`& z`DPxe3^hjdOdAtHM-js9KGK1`1{H3ZyF1S|6exZwbivNN)%O7}_Q%OLqE}Yzgj{!Y zFb!vUL+2?$_FrWIg*U(GGkXT;MsRYY+7I-z>A@lgh*>|9+x?gGIrmxt_9z%dUw`s* z6vy2P^|<TDAUI|JOk9cQX{>&I_w%?-NG|7R#4Byoqz{8#SPOXiAD~)`<2Ybqwe>sC z1>0(xM?*liX3?}FRm0h!!=uix#_|Zz>})mFg|1RV1~cxh&-1X#8X<Da9f(P<TL=mL z5^Zls1@latA(pbhe+?7cGX&6=-f;uP8uC{M25L}XF~>XYSceC=lV@AUKB#2qEF4C& zes!)h^DQ`uhtg!0EBAi9zPeF378}t`2bUdZtP@A;UjM!lpGKGO_J~`zL9?!9p9U7N zJ_%o$h+?`KD5km?r8HIJd^56{G1=_sgA-t<bZH`2f8tg%>+0OI*Vz(WhK*`57aDAV z4o_3BnnFNTDH*U<_lvt=N+kQN^&9Jkx0?|HhRk%L8v3(yiN-Z_{rBthaeY3u+fns6 zdtUbAoYQ9Z^VzKPBmF0C%0<$AFS4zmv3?@1>$vktE^SpWtL+Y~Xu*{WYb#tJsx|lP zrt^>qE`l=3XZ_pLowE3IzvU}T$4jvlWAJa0ab@nubmsxjm&N&MM`tIFKFxsOaI8E= zrT1R%l)JaW<IedcWO>RQmATtk(`L%?(<N)IHBzM?`PjywPapy2fn=*(bQ>F71eQBK zc1O4xk8!<!ee-QVH2OT!SJZE$c+uSxptnU5`FUoNZkJyaSsV%RVHT9@KYYA9!Tf@4 zYSi=`G=Rnx8>{Vy!6wG?JfM7z02RU7Dt|v6vml|kUZKSDJePdf#zhW=K8>lkcbQdn ze&>f933eYo9@AM@biN0r5r!!-(Bwvh6O(6*TNXIrQcJc~DS^Ay=yF>@&|p&cJRUYO zv}f|E^*kaM-AR_u3al<ed->ut_~C|4J5oGf^?jP=_wPmObVcWQVJ9)p>Ij{?5vYh^ zRN@O{^Yie}M8ESsKJb^5Ez@rT4ZpXZ@~naBUux7qZC-nu4*MbR^zpnX?Y?cZ0xMpy zc=@D%qwPD2BWasDrt8YsU>27n&nU-5^}~}phrZHQ=~8sw(+wPSSWk4@t>4D^<PH~( zyQ>svXRgcR%44hx^7`4C`)&C{?>%PhDHj(R)?E!01iBgw)dtp9h5CIqyDb^TiNjde z`RQ`fc~96=smu9X&l={@?)Qro0~AN*$Hd}?PB`bSkdw5ClP}25w=#8G0d-5!R9pq? zmVC<(eRKlsrBw7=`}S)YNnrgIg4pGp<W|q-!-K9+-uPcl213l4vOSOoBV1u_pRRj- zXF;|?+>;sBE9M()b0|9agpQq8IB#1s5*wYqCz~x05v|yBFZ)oVLA5baT({EtpW-lW z1=ue~y7TO%GK~%5h4JBF+7sAkkZk!%e1q!VQVk>Gr#G{jrMfWp=Vib8czG-YrwWy6 ze%&fw=V8N6Aqdlo2lnxSy*B526<xh4_Y)dc5ZYP2MGg+r*BK#3e5hvgGJ~&?C(bTy z?CI;44S)pMk{mO(-*R{iQ)>bznJ?Kh-p}>(#1Qe2w`Q~Ht62D{66}#Yw<`my@3$oN zwxD4`ly|mp_gzoV)Fa+fcO*7n-RC^4xQV6KS}Kl~`)%Ppuerl{U-I8tHk{VAQK&nw z<u=n;@^U`Z;p#76eLDAKQnU<Kw{~i4-iqUTeX<=G{X9gp7V`k%xE^pl$XRo({oIm# zd*qIX^?bQ9z~=V0aL-IweP}=)>vZ4?)JJcAmo@FX>%02qtMSmq8v(_qhc5oPi_mQJ z&P}HhGTGZK^-)DU_rB!jYYLcw)Bq9HmUuL=TT(vN4a<kYV)iRf%`;@J6E<3_cXs<L z6gah&PQgLwt~O;4n|RO>D`+w9cg*ZQs(ZI#D_*C@j<>l`e*Dj)*fu?3+|?})GoZKU z`pW0LTrm7WwJsMFOh=EH5ou4u$-^sHQ4{V{d;xcY_M3|*E7TdxSIydC%|{fP{)5o* zZE}%WVeQr^wq`D>SfmQB(~MB9Y<lO`5<8m`hYtNvZWe7W&#ylwJd^m3mU+FkZNN!$ z-!X+#;#Fh8YdiI<M|T3VYpY+bAK!iTS3BJo!_poL=*81aFxw#YOQ$QK0U1P)<({9v z@1}~6>pn}1q<ZkfiZ*vYFyD+?xr=qVCnF{1g}1BYyn<lSAsOgTiJC!&yPdF-LETEq zo$e~Lom#$6cUiNB!fnHKw-pOYiwS$WYU?qQS@(}S9dVy-HXrcm?P@GH{&tb35V^wz zM01@{pX?_4`3f7pN6ou1-u*ofH~VsozSVS)OI=$AZcz$n6EW6vag`M>Wt+AD=1sqv zBlHn0cu}j&z6<J@S}~tE{w5M<R5-WL2ClNL40I;{8&Dn7>nW(E$g=f|Wj`6KO9q$( zF3d<9Mq;8BVDlBLkL!g`BHWla!_Dl_=fY@x;@z8|TiuUh(fOyU2R)Xpk8_oq*w{s< z6w*9k5H0)$1J-GUczPo0m!U=fkFd7@ibGl2hJ!B<!s70*Sa5fOX7LaR?gV!NEbi{^ zZXviX9^BoXU?I2!*N=1WIp?1Dy>I=0)fP3Z)y_;$_tV|abmOrH?3U{NQY^Y2L9#)i zEG3t^h!8BwKF#v_ZTd!^IHoEc5Si-={BhD7j4|bu8Tv<C#y=nW(iktxo$rh%Ap2J* zDLW|Uf{WZ1sYvKTw=3US*a&%0AD`=k_mfc5P5gK2Z12;e)+6ZCO2YSJX+0KO*Js9U zS+Ymj&J1g#_@vPb8UGqvGs@OJAGge;t$uV_bp&fp!LtT6Q4Ybkb^$ny@1@2kqI7PA zd(J#B?E&>CzSpN8ahf7T;z_d-`$@xPJXmHvEbldX;ZT+`l6V%%6j}2<ayUAyYb07= z7(rkR-d21uemcC7J^55i<rL6Vr7XRquf<pF-k0fjz6H>#{;m<P__$vq_-s7+U><pL zAL)6%G!IYX*%g*wwp>mn80hu}wOMQ2srvQVwE!jUI?6KM>SM)F;fV?BJAEauZ6|Q* zdhi4;9--ei(h{n`@Z-ZAVo3nnqUElEq3CKwJi`z&I<1WW!NWgM;etJUpI&c;5nZH| z_&3#S$vysv@`;(<YcO_gTtIiDk-<L@osJy2$0^Ra3}8U-Zb5HIQSgkJF(5!_4FN)= z*K@Auanh>&IA>b*g^+^5s~6=1%=kv&h~Ch2Zlbu}Lec4DTyeXTI4R_)HBt2`srAVF zE<o@ioh(MJ<UQoTeLGX+-Vv1G`)NSv{&^r<H=<Ok$x_+bR>>~}{h=ct147YKVLo0T zNlxU$dr%;RUYK)$SFQ>e5kdBCdWihSsWKqR8~hw#_a?c1CZK9U_QXqT)LCL{2#We* zaRsv4=w0v0`U~r=w)M(QigS#wZ!wm68AM*@va<f=T+)VRIj<b0k>NOa`Fz9(Rj5)A zr}w*yt=eH0p>OlL=Jp=#!TC@v!2WD3;Eb^S;>)0p6^rHi#@<6z&I6X%XS3$#GYwY& z>2Pwp3YV4Nu+!~@QA2aoIggjs_X0+1lz|}Je`@tiu>MQq!g|eaxv>hhTkaIu{Iw27 zU;EJBi$sCHn+L*RE)>FdnQpH}XC~=+ZJz8@ufH9*Uk|+L2J%-vc|U6iUl8=f4m{+v z-r_jcc<A<f>t;R8xd6u~sr8<JM-Q9+9&r0f_8EpP94RAI=PRd}_00|XprSFk^7GTl z;908l4p(ZJ$6lyn4mA^cc3N_GYOww;z3Y1a=K0O|Q_evB_hj_Pv5z$m-Lz$=2+ZyR zu4jyTWrKUyy~a<me6I6nY1(Z73OuZB5>Ka@kkUXcCcfvhB*7cRfGFPd1gHpUOfs&A z10h?yDtY_e7u74uwJK`A^ZjenZDT5O+BIO{1w&kyv9mPAzQEpjF;I9DBOtmXz|+=S zkdj5mH<)#k$rJ;x$q#&p_))$+6#0qNulNj(EG1cJ{p;{iU{t_$;0`sYa$2QPGx4`x zG-a0kd#^(d_nnD#(N`kGvG;X?^Ul_;6Ez*fcZi0><5Vj_q}1cylb9i^^)Fp;f>w}x zJ)W2x&-+rpVQ=&p1h$wB-75YpI3N1D;=&91e#(|Dc#Y09*v@YPLVCOIptZsNABQcb zLxGCb^QFp=tNGWjM6VTAA6~70D*fEE&(HvdaealR_P3I9rts~4-c;MBI(w!gyZbeL zK8Y^0Otf)E=B4iSTS$avf|%ys=p-rlY4~IiPvj2iT(aDL-+g;j#N$Kz4ODS3nOW%V zHJSJt4HLdxKNjJEv%*VzV7<7$;3rYx%iQ%u#Z(4!*Nd}f$uW!9t3A);WUhLuUYFO| z33{GC!5{g5{}4Z~zJ72~KMv0;(x-jCR#9ko=%cL+CeIKq^&Zo2juRi*61k1p$-Lj8 z6+4#-saRC&-MN=ftXR2oL#__Aba;M5yUpd%H*B7#mGErR;?stTDJdL=55ZMB)_xnx z_MoIZal_85UUE&WFDC{a+~);{+-IV*4Cq3z)=}R<o>YMznH5~Xtj&_vheUHn**7Fh z!9c>4kdw#;Ti&ks9{2IGr|`!}jP_}o*-oy<%S=6xH!`={Xwl??Q}MFrGjQ%Z!Mf34 z1g9<``N)RWi(-VWy@v-D7TK%Nh!L`=Ax3nqUZ);SuRHOm_1eV#=YKZPb;gDHS75>a zWU{>W8EVTyfa-oVQZYScJj&zl9)B5D|FTbWvVVjcz{S(ixsAH+>8ooe*I94PYwxQ+ zHq0L>T&e^ZL@)hb>oIGCF`+xITRul`P~ukV!@*4P5pCKGpiap-Py5C>Dt;kVOS1tX zq(EzhdvA8QS*0`AL1btk_#>NB!#)}Vm4Y~VQs01ogLDXp+g&YEvRXeU+jAe^SUz+0 z`@6bU$xYO7R68?>z6ai>d;0on-Xbd(Uv6b9KoNciq%CP{`EJoskbE(<A>$b&=oT%# zy@n;+qk46GFgZ`r&aSVfBy3h}I#CY{=yz0CUbvKmFl0Zc?euJX({FIOgCCOyCJPNC zF;sq*gp$O)>iR4g&4ykqbSkmt4<tV#Zq3y(;0`)i-Md?Uim@eKqi<<xTP-E?P=A1d zEN&e%I}n!tAR6^U9v?}RuT<Lqy*7Ztq%Hw_K-;lp^Y|KC8@RV_YP~yooYihE&17t( z6Ht8;k<4n7)kjiyXH>|87zI7oJUGef_NV%fPwopw4ghvBPKS#NRwQb@RL+8wU6nPg zb=8dx;VMKh63wS)rh4C6mWsePKFFh=&Ao=wKv_1&3HU)3?oWBO?02q<;t?d}JKN+D zeIN4;KpD!vLyt1Q^Tu!#CMH5X@6*@B?i08bIQ|CBFN1VB{ceP9v0Um{GYP-Df_bX? zAB%-fCZ^@{dvluW-PE;zpW6R(P$c^Dabu^JC@CpDk1uzmuMrFO-P#<>FG&Rchduq@ z9U5UUQ<z~8_cE9jHUER_{|5Mf4}5jRePGWgDh{0mjJCM6R9oog`Nh%Hbn2JOHKTF< z&tLTCllty2uv4&#Y)M&LGpV=tb4fHth`?nWz24iXF<v2k3$tExwXe`%nMRqvVA4wq z%Kv&S$G<dx%zfPatwB)gioZM?q*+;K!8m0Uu$QP=6Y&@3`Wt-xW6<E&5u}(0#qnQ? zTD}Ozip$&!69Fzntr}(jfBx!#jv&+l;l>N+jj_^fUGqk(^2;_W8w=eQ=3ZqTvB7{F zOC0(+EtB8EdZKxu0X^MG`YYBv1===Xx~!P5GX+ZKp(6OPDwLc1Hfk1yaWewKp~Z2S z%Eqfj39_msJj;1w!%w10hG&fu;ti{xBvGn}z(v$E4Ba*#`7z_Hxap7FG#F-+>i9`c zuk*|G3_}YfU@JVU!#ve?8{hx6JXKP#ZcnpKSSiO5KIiG2qr_K(=Q!>AT=bQhnu^yP z{{#E~#9Xk0;lqsv3+p9_PAva!ShWxbMqWy_&E2&HY1K`b&r%#W6l4vfU*s?{m0-st zkIE4HS-<$Ti2F_gI-z$0Q$01D)vsarw(Ab(3bif8=&s(uOYP54Lsc+bX11M+=?gt{ z(@G)=C}9??3AC83N|&8&lmwJneij^}Y_9|+DiE<{5e-jCX6H2vfT~WuL}vfJ$(FRh zH%NuLwq<P;TKKR-wo_lO`u)x?z1K79_z#YhV4Ji5NAGCv<c=Uc-(I5+p^{Wd`ul7R zN_4zGGF#{FNI^V&r6&j*=+ZRtVX~rW%GyD+_C%z;vqe4e1{WoI<9K1<o^_vMd#!IN zDlG66>K}=CQu=H`*s&p%RE3|l1m6Tu$A5_4j#3GeEa6lkX9jb|bvC?hNrD_E1K{@n zJ$IV9##U0w^k6CS58ZIhDJ)<(Sm&mt7&CSNg`3m8ko~gymiNoF;xPe;XTO)8mlT(* z+-F`Yipd|hv`4b9Zr7NyG&^Xg6AqDRNCtV+P0yx<Q&$oCl8dR;+w0nF`0`CJ0pL63 zZ{^bzTDi$l*N+AQO$XeRO$5r<LXqI|?o&hIj}v@Ewv%6Zsd;qc@l(wUQ4u`i!b!q0 zx-6CJw%FSqT`}1C!gKp{3|t$)HN1=0lI~bBmA@{wI6!oSS`k3ABm1r#MHlU0G_EVN z0Nopdg#-0->$v?fBGepvjp{6#LDZ=}{U2b~dH`f>X}~quJo~n#B&l*{%+P$4o}Xzr zPn58eI?ce8(MLs5c8!2yRw-CaMbm=#VKCZSPra1ZOBq99@Yq;Kiw1|1$u2YvU=9;2 z3<SvXr^hHGBN+g{lVUBQFH-QjrT@`py+Z=NVGI39f@D2UhKPY&z{p{*qe8Of3uiwz zxz_rcKBey^FeIRx7kw*Zy(9hoYtw2588ofjtbR+3Xp;QOTQ!PBmpPX8$4aK4TDzP> zKgEB&h#l?HIkvJ%Q3tfC!s;0GEK6ZmXRYF{+I-!^F-@=in-*c4i^|BK9(a0Bi;l+? zI6Bz_!ukcVP)=V#wy{wIz8saNSqpHJGM9Z!TgV=m-4}@4r-hC)BVMd5=?NKfqsb;a z^EJ}+wroZ3)1kcaqtFs;aBEBB(4oV{(5v?Y_5-YabAu54swbl<n1dguN-XJ6<*f(T zb!#H_pO;{WMqj<qRdA#0l|{P%IV(OJ&S(a#r${(@ez`F{7|DDQ>I0c>*49_EZ~s2S z9V8e^Y@GPZX_m<X`=@Vg3Mi=p`$$^h7M=d%%#-_COaEa*V8!SqfL(?ecwp2l9Zgf* zwc0ckz-SHy3-LSGloXUHO;hf40_L>+yBANTW~Im285H?3PRB`2`$8HR+(@Z<NiFqu z8z|^EtG0+&ZAj@toTwi8T3+<)@4>Hd@%MZir_=jp{EQTGd1^swj^g*PC0h{iZeX7t zCL2crv|%6NuhfW9QU8f_{+FV4rtw9?i<#9CCY;3YSOW`%mYym($F(yMPXpf=#r4TC zCo>w!%x7B<e$y%80k-hIkkIo38~dig!g8eeaTe9|ql5v*a?Qo`DC5TRUXj_~T-#$t zMhg+wYW5o}@?I*O5$bj^zLkMmgAUIS^5VwS*Az&4L@Ru>(9Yro2rc=zb3Lbto}~X$ zqyJ@VsO@+#?F6dVBDcs09baKhXjJqbIcR(<@4V?f;s;gvL#v8TQSxqLCaX)mAVj6D z%hoe&Y@~;jQ4S+(?fRR7`NuBl${i9Ok248^pn1k*27oJ@)JPDIW3@`6SUWkFJHV?) zQ~|_fPxy^9PrIz!>pRCk@)CcW+`k43z3|*CaIVGG;nYUTnqpRl@CkG;2or$~RXl`X z#Rht6irN6wix2tO`h=aNzCcTZ6CDc+Ot}9f9`_!eTF*wdRy(0LZA{`#wy|7>sfD<e z0{NIdp5tjGwYNIu)3+In;QYFTcni8$EjoLF>!QTwB1JptJx?LlCV!_R{(G>(1%KzZ zqr<VSint#=q>YO;x|r3W6EI^&o)JhziqwTMXo4gz1JRfnd}^Jtvnjv>!BH?7`sWIZ zRYw$HeZogOlAgwmcIs-sh?qs9^{6JJbP++IPLvnBnJYJ-Y%?jPCAC~0k(&zWN$K%g z2POY65&Gx9$K0?NqE|~>z5w_o0O*^gZ0TqB2eA?~F%Octz{l8xLwic`Myp#f@M3Lo zHk)~y7pf&avHv6fL+<jYw_(gC6R3^+8>sm!^_jZvdg|;NfT<MgN;pO1%a_U*aNWPp z1}(1KuNwa)B?4hSm#?gIhDkNxcC`Nk7hzanyLCm2qD+g_EPuxSyRmpj)E~)`K%HOy zKM1En_ZtK$w&BIhS3dath%hLNn=|mn8GZ};Ym5IIlr1#Duh^q=8)FMYj-6Ya5A~y& z%u?s<<AS`@>)VIn=FR&R|8emD`+I!%UJO3^wz?qb*5d;#KU{{i1(ig=oV~IjZhYYi zJ1FA!u_kJ@PyL&}HO&9j4?U)%5A*bsd2!<?<Sj$B8GL+Wy<mBeSSJknHvYc!2R$Qn z_d@ajGiZN}8p&T1|3AOy6>?d2#|(lomY*1{wPRqBQs+xiDftzx^2)Ari$P?k+-2kc z#Bu(6Hh+Lr-&xfqE*lo&H?nP?r18T9yObco<?8u^)w5g~fuL2;|Ib9eAj|2z9J<XZ za~EJu9{s3-X>;3U6)sAk1awg1PgUTeAi8dRmU5cN(zwE>-0)`@%J=BK{$Z1+Ri`sC z!u?-wO_zBIk<<ib4;U+h0(u*H`=$vgwkQ<CsgM5qto-Mh4m#fw)PXS&DAMR~W|#IK zF8hD4TE`XTk9FNu1KWSf`hSlbP+u~s%byji$o|Gt|Hq(e6kw$e2)E`|a1Q<xEW5$? z$)(OS?!H*zKP7fm-sj`~>gs)Vz8<4*05LB+?OD}M{=zpk<Fg^ZggJ}8Qm-%38N3OY z-o?y3EZXT&@|8!dwleUraqoKzb(p~AJ8WQAa>^2jl0{(+r%+h;AOSc~D^806+>aOY zo|7`yN3S_EX{MBmfNL*Vf}3&U(+7v%pqFbJp;BF&$x(G_@+I@>a32fr{R~1SyD>Y= zmdk@wEF*m#ki@7{6ZA_#JDS6T?Ah(2?+vR%;O*kfcn-`&%)E2XnYD6#T8EwKg2n%D zot`dE&O|#tBCCSKCm`8n0L};<N_0#v{9(MdXNvDf^28}G+mR;XrEFsKu4LDCAbfsq zwOfE>vBC;3=#&r*sc`Ys6)*sF{|H7#YM#1qMILw;Dh)ejtx`Y~F=z#1ehhZS2<<ae z)*S<vABo{b$Y75%5nd(*U=o0(@1?=inDG*=%uU2SW}IizadGlecac;WA)Q<a$?{Df zf1uM+hq5ErA;NFRUKv=gLr?0~%d<UeTATui@&4MG|0Nt97sw-pZ_gT+<95=a1>1@P z5AAjE8|G=L81!!|9peLOnsBYbDul{f#q`BZ=%nSJ92jF2=tbrgI%uoEXd33`$eucB zoCQ*6f(aE3Y3I9d@nLO8Ac)C~XvnlCk>yh68ex~tiVu<f-OB#O)$x>Pbhzot=7=<z z%AvJ1frWyt>5rKGF!-oa<cMkoN_b$xvqq{NL5W{Uig|kk;D9{3%Dh&pT!>8EWXz3c zlGW-_vs;M~>ruM$2u=ZvB!60#q&r~QH%{f)PdpU1#&PB>{!fu?1@O)%>wX^j+Yw#h zgNG=r^WBFDt+K&SVGIj2E|(3|Yhyez%Q%2&I1IAk^O4Jvk^15?L>Qz^D5t*77Q&&A zh4jIS0)5>|=pse6l}}&K%~<mo-}Xs1tql&WrM?zZS1GKCNj^NoV#f+}vBV)QuKo%i zIX9oEY4Aa}xtY4p3-?NvkMH^1+Nd;JE%p;9;)lsQwvYn!p+$&fMZ2A<%wU_txu{g0 zv&M^i^iXbzkCh+|9_ZY1rzy-uw@O}ct_6kjFb^M<^z5;&u5>=%M*Lc`h87S3C_Ms> zKe)hh>ZvW!T`%q&E1U8m+N<+(`RcwkFv_99(RsJenRX*WDV%*DOuV*7k^N;~m00#e z?dqpOv*5m8?+evq)jvhQqxK$vRJjm-4(U(xHA37u(*P{Hy`HbmR^>f*E?7MS=Um6U zJIDiwRr4?Mge5`BT)EzE>Bl8C^Q<F;!k+6?#q&wLupctP<oG?)iW!X_S6CCXQn`xn z1&$i(H2v+<K3Te`B_1HG|Ck+t?}k=l(taIe_*<ncr*@EJOrPWoI)ggeV>{}Qu~Bf$ zP07oGK02HOB)D!+Ovy=0Y0a!%l?3Ok?;qzLBg4MOAQU2avI11O<K)CiXBz96&X6_+ z2?4_+YpzXR(+t<QOa{sb@7qMeD55IDJj=NQEb7#33uX^!{;CBq;eSg55$zU!Lcoyf z1e>IT#%V8e-C;yF5AQyIQXm@)i|tYsl~8z9ZDabAE$0eV8GIC<nv&#fUpbrf>JHCm zcfZ|Vx-f{9WF}^|)MZhS;WRLwJ>75OpksdAMjVB2*pfG6h_dfYv_zFdKQPZM>Dy0d z`4dYnGcYJV#-X0RQ+;K|a^`-m-%IedXa_328Za3DzFgZ{-0Zlt>0OVab6lU)Tsb#U zh5L>TR{Jop)cQlS1~VW}LzL;w@S&B3{_A%THOuo2-ti*Nz8X?5d3rn)Qu|c-S)FUW zXQKLppgx#CP(#NUP0nWZ_(?bNn|F_^m1AYB)NL^+L8rzzc%0a=I0?FxuXQLYI&8H) zXHU}QA-oHCXpCw46n|-njt1(S-Se~YZK0kB9O>%0J4Qky3gkm8*g}Mo1`}#zwI6g6 z`iX(^7PC8dVQjRTHD5Mpnm))53SE=Ngy=fr*$&+@x`IgsnWZUYN<%UvexW47&)!eR zAzt2*CN6U|S@mcYw~<jgSm3ZhX_ECI@7`qa(rCbsmE%=VAUbGV5%Zb%Y}wROkGI*4 zetG^ORQyBoD0${k*lHziEF!tyjZu1|+=G`M1~S8rhN(V!3-0x-#1``~^8BSFNrx;y zO86#AUD+eVK0}UAT2-1*$ajJEj922(S=FKltYwiHW?uS?kPjZjw{;#inRhVngNw*L zI@k!fH^m7kXR<uV<sYzL3S@n!hqQJUsUX_bc&#vIjPNd@_qc(3t$y(6zReP8(Dz^K ztUv_>z>kzw$A|Uv3zwI2jNVNqnWw>f)HEif7rk5h8;L9!!_GB%s9-N_+t@i$R~-Nj z1)MQA*@6P`2lfNgK1Lf;4bg0m#4c8E&4y0dEWp<8ocprk?zEQZG0glxh=8}<wAYPx z>glT&^bS|H)|^`BwRdR5dZYu)d4A-z^1KtX=X3m-kRjxVsXLRTjr$^}%GDNfH)6<q zZtDb1C%mm6DPq=}?W}P+-5|r-*PxvE-P?_?+FYca3ukd@;Iwb$ZW<`JehF;gte>>C zz{GlZ(&4*_+z%vfG8?EelpKty0d8c_g1QLEfVxA#kxf7U(o9cgfkHlA(@q{X#)^R2 zCtI@~%}*5f%<r*dPW&=QDB@lrABw{A5>!YEI!k@E&WO*C?;pR!P|A9~h_o#q2WZ9r zWsI!oeJdYEy|=igl8#|)JfJTz52mwv)n=yT3Z{X}QnhW1P81FxfU4+-FfOPTU%|6- zVk6GHVmU9*iyYgp{nb*Ckph;tQ;$UjlXk3WC5tSMJ!JW(7b6`8q@T3)B;e37!>D*M zvt=YnA*&c2<y(s2IJKO*si78>F^*-nV9`RRJR#WuRRHlbAq*8JR4|E=d0vl9V&|}| zulfp1R2}6Q?uw<TVeKq%)yeh4m$=GMlZT1L#d2%wNo}!t`GjhYKP0`Ze|L4WgX`E} zjMs4FPhN_z&by9g_!KL;=X)fPx6kLXWIK_&#*OOXe_E*q92n1zCp^7}IAS4(g$vnA zm>|5NothQhx;5VYM1jj2Ff4TyKa5&LN({$z2BWPL+;n+vbzEiw!2yLuXxK!Ayv01K z!*Q3NM%1xD04pKJo+w1nPMh`^_5J5Pgxd;SqpmiNkW}Ixk^ES6zD~`#o96=fJT^hI zUvmXN-F$vr3LNtEW5IjD6NidR9?xtXtu4QQ>5GDV0|zZ^Z57m)E&YO_rRCyC2Rpv~ zjbtCzfa>y0S~?rQEG~^dl+4)~er=ztM{{ml9DsT`WX*Hm#){B#@%@R%BKb{B_rGO- z!2@>iF?d@P#NdW;HC3op>MYvkaniHp_B?46_wJ!a%Ih8T8gW_yRkO4Lv5f31JSrYt zv9cGgDO<bW)rpAL6U+9I$o=#dV|Bn^p;>(`%$up?>;yzXWKZD}Na5sr70l28WOvv5 ztGR&krA*rx1nSr@DhYG2#Xeu7?8iS5U)uHA4yiM5QVcKXs!MQDsjhj-h9~#4YJp9# zzS7B%3O_@n@+3q-3KJa4YglIezA|<BG^K2gRV{4!2A4_ALcRy(jVx=Ep2B+ky9uA1 zo<rnA!o=Kz-)x>2n5dcGLJSQg_)R_Q$HhvRb+AMTHt3G|!cGwkLIYc&KrmU%^hR83 zMC!E3n*tz?fz()5*ak$>LnkC{>09IkF`zGE9sN7?PlZ7tdq}BRhI3Ed*w4oImE4CU zYHQ0>PhWd-Ho1rAwp~rgSNRQ5XlV#ThUJEW8wad8LP<G24r7|@x;|K*3(8gOQj2)? zN3ygk(2+%fS3*V?oEDTyKlNUh7_@(^7L_xJJEgWUr0gqjV+)@teGI)NcJI#43<^^1 zv?m>ZKcv^&;|den$N5qbxpJnSjF3ejoDgmB<?g^)LNL?Oiqme&dJ~oY_`f4E4J5vz ztdtmnu}99_SDaWqz2f*3#my-<`qYwkXM~-`0lX#d2~jFg_6y~{1LYOk@7L8L3G>1? z7JT*(zfu}T{`OfHY~U+iqT$a^)7i;^fy+kCKjV$0(bH=rh$+uWV|$q}WBb3@^_v7! zlu1!ga}%}eFp}SQBZI$58@)R7R>O{=5H_q~qVn0$tX;J+>auDTj<)=1%ul&-UV}aR zh$;4&7e7f9fYoZi^}2eIquGMj?_lNI+a1O%wTg7tiMM)fbk#qQ2fK!O0xTVizsvW~ zd7Wt*6fkDS;@6Q(PT(|tA(Kxjs6g=j#HC5~W^?)sm$ck!7rR2%3zw1tq%`f0z_r{3 z7eU@d$ZL2~UszM{lLB8H`%P~iW*g)~h2bvoKwqaYB{By9;9sb1`+D-@J%z+QMsruF z#$>F;Q<An;bn?)`yTcXOe){*~o)L;9kVnnJ4G1x+wIGl{Wpj8K;cUF%#|2S*0SR&; z>RDq<C0H<gVTlAM@LmfLbVvoiUeYsBJ<|i1#SiPK29)JXk?%(3i#jQvZP1{9|9PY~ zJIb-cY}V%EdB}-SeAJeEsK~T-bZqOwo57^cEOwjn<DzFzTepMp$~rZdX>~r_KM1W7 zf8+``;6)&qi|<zTT-2y0={Bub7h9AL%v+A5mJ!*56^3F{tG*n#-j1Qp)3P6ETgj;` zR($z2V?Jou6?k`!#^*ho_9cH_n2+U}bhHDmi9#*Q9gHY*#F2fCJRzWVv|oVgebw}w zPU(Qoj`ycCs(+8!Xm?igu=IQfLp7u0bvu+2b%kJDEJi?@8xBp5kgkd6x#9B+)W3$1 zcbBqc22J4Bftq0(K@yFAj0$_?!<kLEVa3!0y$1&k$wp9(xI!_{Lw&Iq=TVy6uKrt` z|F46Uoty4`0|L0x2JEDy!o|yX9k10ZmtCx*rN{&c@pMQdPn8u)0Y499_@LC{&C3hI z5sas$;U_A}VkR{hfCi?L9h3vNdt1WQ(0zNr$&oV$OJJzv8lVq|L#DFN0FohUCkh8) zg;b(C)lA3jbQ!PBPU8=|Sy^4XTcB}acATxVbFf53sR(^9*IQ}-b=WU_y0Ouf0?L;P za`52&d0Dz_ZOI!^Py0Lb%#6&>#esj0;Gw-@^`XI<tzg)8*{jCr-}qS|R+OBwcq}97 z>!U>DYT3}JV=_F{81(sP^nh<_BfPzRF{}d~9_-HNfc(Y{2xGy{FCllDK-4b>UAc@* zJ<ZKeUBmu<Rn{!X?!mQet)erX2=!cAHMknqSPq95f?4mkFbj3#!dZ0V2{)5h9hGP` zCurxIAB*0uw=&Ep9TEBT=<=Blm7^XogY0g2p6~RA$6J4UCuF-XYuRrYXf*JQqyH6C z^0gKft)%C?NN5-Oa+tEC)2a-$NC6D;6-xhDOqJ_lf?2G^7=O%CcS{wf`nJ!dgVz<K z`))N<beaRIgBNi(6YGrRR~af@tt>4br*wwjWwAy>Y8kGO4?AuI==#tzC7BXv@=;u! zx<Era>AVrTzk#eFJ`f%=M%_@ZtdWF)V%SoknafJ~s&Wm=-^cp&PPAO59WK4p_R^*z z_dAXeu@72Og@Oey{MUQ~hPl{gI~AgaCS1q0yjDHKQ#acmzNx=`%$T@c#8%CPES8Xs zK9l`h`EjTEu!N{}oqkg*4NV7F2>`<eJKdvYFo!3Fq_+pOyD+0KyI`VYE|I@D9On?P z=bvCpZr?2WH|8{SDb?BK`VeA9P;U<`Qf;;hngj*#_QSgO|2$nHVE&b!*s9IIcf#p3 z61Y}99ZX%a#v0<C^4(Q_pPkV(YD{y}9%`MXe-BGBNGeE)G;QIepg_`F8uTuM;mF~& zXbn=-P`2k?^mf139o|mm^ZNJL!j&3Vw8;zq=Rd=VuaK>feL)piHY=?(-ser<W7NK1 z8Qh9^KSzTg6PCz!Jd?+LXb0mMaZqDmi(x(_BwH_O#X&bIJP8P0H6An9w@E^c>*o65 z&~8;>uSQHIri#B&sPP+nC~c?pEi<zi>s;w)z|`0B`P#>TgBOYl1wJqiW_~DET^ZA$ zP1yRq5tKpovC}aW-WlK<g?%c-d|U%%S~=bGJuK?{{4c5TrN??Hv%Bxw)cm@3QQYCV z6I)`l@o+Uej@e{zcE%caM_{*t5Ph4fcuWSg=^xUn$8I<CKCM<o4Q0NaG1*Chs9%be z0_jScRnixfePj6BA-mlssNU_iA>(^pNZ;=w6{;eV#@)Ie3gd$UR4B7<48m-B^k?}9 z?$xK--OAKvq?0Bj3O3ldzAGXQFxIk+9@yg;q2*y)D6Vs)WpB|_x#E6f^#Wn#>r?7m zJ@*e4<`EGD0izuYb!3T!2UGInvcyJaIf^Vg9{^y!s2lT?Yefr$*K~<Eukju&hNdoG z>UFGKL*9>&#dQ;8F61)x65+kQPknTd*{a_Fdpz~YI<&Q`DUir+slKEPGh67Mwo)hv zd9W`x{j?i(6F7U0{pry*gYI}(2Arn*A03?arY@?}w19dh*?WpDAL4%F+2&OY_!`kx zkv%lB&=0w=H2jdPsFkzKo)u^XSd(quRTFGGO+u5mziqMFXr3HCM`gE5=Pl44xncCj za-m*tY|&!4%X5_G<vzUW8rFF%kwIk%T`cFYDdMSKzm_HLw}q<4owOUX3BeQ5FZcBl zG4VOPzoybXv%13|!$gum>hBm%{tOfBQ@h?=-pu(BLR%5cd^JA2Ai^bhQ+0*3*=TT| zd|4%P5<l&xtuMl&HzbSM-*8NHa*pbd@11h=BQr5>tJwZ<^HS%xPE~ZLXK?gzSy#Uc zrMLTswAi|l!m@S514zSF3D+rW)TKY~T)l8RmD(8z!7gv&E2G-+G>4;y&(++!7o9al zYj5)^CDLX=YYmNW`BFQd(m2-1N|P?VOIzt!QpluSjxDbAk4A(AHO0FkPWd;Kf%a?l z*Ha~f!z|i~_Nyl(AAbrQndC&;14slxNabYOwYp7@HQKuSE$!uVYn)3jA(=PDpncD4 z6t8E&xbFV^fw7vXX_NiNL$LnU)uvMuWHk-OYn(y=olW2XBjcWuq$!fesm!cx?hLEy zdKu;+%D9WEDafdl&09T+%Ep4Efw?1T8tM8(0rCD)lwIGe7%x9~`$kShbB8!es1=FR zQS^8f@1!#N<e^!n?4+AeK7x~Q5Z`hFnU?h?XhQ$~YhNkJ2>X>qN=$@#Gd$vT_ZMqh zGD}jVX{FtV6g&w_E+WrLupS_nkyRs0p<uTF(Xb@x-qDY5@g_j+d?dnhRr8Q@zh!RP z-neSyC6%z?0E-<1^}wrw`@`&1g(K^C`AOn5a_&^f7_h(_wr1hIHmrfqcN6uI7y!_= zd7$R)Zq%JgID<r#eCI>@8zOg_YJsk^h$defh=MRUmSh0$ayP35C=|3;=$HweEA-*{ z+rrS(t8f~Fdk`L8vMMpNI3leAoVRSQVEmV2ZgmeM1WYIDJB?%W4vnl@J1LVqYCa+5 zE{M13%4_LKoS;2GZl-A66_p5Kc6==`whCD;3AUwC17M^ajoSmtRB!QfhB8LND3;`i zyk<h7w5feNPjZjY)3=-)3p~8{z>vH1PxUgW&>b0M`tl>jtx@0KOT<Q{s)DUsE~3I2 z&3-9EnEAb{;?M;fAlfbyaix;&0?6OfG^th5Kpl=O*31xOfO`<*wdpjtG8(5^TusfQ z%S^y>s=3-=xv_Be%*e88T5r+bzs1m%yW;*TuZ6`5q{?=Wuz5P`k!^0zjc`WVUwG^S zDQBGCG|b-PN{;-B-XCX!6ipW~hit(#qU4Pl|9a1UX4y7{Q=-s9e)hp{#wok#=+U=z zp*<dgpCirOP?{OkyPzwln}2Vq?Fyo7*BawNN_4iG+FIGPA~~27wb2;`%Se7AzF+hl z6Q?d6As$Cj$C9{?zn_-Mm}fDefzt6XFcJ@6xeP`_5=`9tLZ*=pe7pI6zXCB{|7aoY zyLP<N|HK|Dbi>q%dm2L*tsd&t`YyJMT6|+hx0US!3D8`1(jiL3(}&H1<>l_7JpWS^ zcIAo>Z~0bAG6yd=+*67%hj6DLYt|rN5h8x01@76h@@X$yI`5O+1L&G0uB|D&A!Jbm z)L-wCx)d8O{QEIA=V0uNLuGcObR+gW@y+IuCeD|<Vs(bV!NQA$b1*)K#GdUJK~JS) z4%_COEU(J5T;0HAX1#==L$ATC4u4>P7s@97{I=bEN~BxF*YXfO>pd6I^|BB5pEO7~ zMh}`3%Z$E-U-FMt%FDgM=`1h->nX$m$O=76wjC?(oNJvtc(vM%u&5BF*g>2^?K&oX zzLiB~ZeemlF-TJoEV$W5G1yX!U!%V_go`e6KAx#RU)+Z~wSg`atm*NF8!_t{ei6A> zL*!r~@6#-{3*Pe>+ggp#yd<6&Uh*p+d%=G~I2%}<056|fTb2_4H9H4b`4o@JRA2_Z zn>*TU2+i?TjT`C<BVpfXg^<HukCl%$csksnH{@*ti8l10x5cY2U4CnDQu3{Y+;})K zj(dN?vTz{36ja>QJL0kBRei-ZQX1hxP?1R+g>TAE;HuKNQMt`L8_wbL-6zicjc>5I z6MaDNQSAM3Z{3-5SV1*u_<*XDT{l{2-<xS+{s#V#`Qlp>9b5ei^kwdYuWGjYVf7|Y z(B#?HkF+q8q~oyEA@&i!-Dd6tm<ToyHf>vA-mR~?;=y$z_V0)`!I&XKi--f=tgiQe zRSQ{lf7Xj072nG^v*TVoZXS4lC<02|k?s3k@6TR)U@*n+&@$C?xY^uGXhYe}n0Vli zGVknio>yO4hH=ra5`((i%<j!=B+>GQ^t28ZXw6pZR{A+osQRRexbN_=C{p-Uvn$wU zlPA~K=rzu~u>C%vt@KM}uvzn+ol_Kc(nEcBNQAN2h1omvjX3q(Romf*(2wl?!TPx& zZc^xBw~WibWy_T%8%cUhCGbYCfa}%1P*TGyTi@B>#QXJ9tju^Uh(6vc{{y^=A+**g zCd}k`Q}hk|F2<>e1rM8dOQhbFk13;v1BWJB9(SBJfiu=QV<<a!k2dqeHC3#?3I($} zs1@3R?gV_0{0SSt-)le{S`pJ9NIWN?A|^nQ58oG=G8h?7L(pi+F)8}leV^SMI6r## zAFVsobG6#+rlQl!<TguRf$|^=F>L)`>lhbK{6Y1{Nz~VkZX~B8hly`kAahp(BfG0e z4Mx4+etlRi=}^RMG=kOf$ZmmnQ)BmvuUJS{;-4m|X*kGU;b0Rp{v6RV-QxDO91U8s zQx@Nc$a^uTcQnD<rgdVy(1KokG48Y%id-Z}?*0$0Z$}G%f%3t2j%3zejELduMJqAL zsy24kBR7CptYfRx<mXmfifB(Wm*vK+2d{xx_9fWDhD^|<T<shBAz1yo4<bZ#Jq5b` zQSI{TP2ap~Xw-3hg6pdG<b@ub6}Ig_lb9wcFE$>TN;KAQ*QSYd>n3`&Y|jturpJ=+ zUuPQw=P1+~SHw%RycAM=e?4_Oye5T4I%_pw5llf~b*s9766At>JI*H&!R*AZcG4Et zNFf3TUjEYb@P4mMrh*PCIK$L-3W<`G>aGXas7ReOA&=VE)t|T(OfXUfUcERA{I_EA zI0*8&Uk`hO&%(vb;PaVM{E}uUn7<aMJBn|h2OD4=q#Ra$noUO@zSL~{x|^$7s+3cm zd4CUcWKANJHW+56j#l70lCvtifIe4&651{l%BK@^HSq2N$1z~op_csV7irICHzCNc zDO96tW1Y$IHvq%0os*)B3HQq({i64)dnDOM!7{61c=XOgudNRitIOf({6|pz?73wV z<bDn9tF6XX^Qd8=xGyBhKPh0Nj*M~hygJ=5JViDg)021`?$PCeZ#Hhj<M^Bi;|1=h zBez)e8uPT+!QI6k*S%pYjwixZuZN$E?ui}gq|M?P0n!Fp69GE|q5h}u1AfiiYevxb z_fhSY8DpL11)I9rP}q_(7yRry>3;I67ifqSQAUayr0@t{o+TEmp}WV1|1i8<rP_`6 zYCYG#0a8{yrPpfpc*QHeU;R#da3yl3qnn~w;Tfy87J8Kqj_$Hw4YFaeR!o?wER=1a zDjA$H<4?Od5NU10Iofddn0RA}HS!YqQlrmge<>Gw_KF1mdvW*u<Wkb^vjIfx;O$GV zJ^%IAM{+=avr~vW_hNmjq~JojmWC*9d98&ef$qE)xn`5+2VL&(g{&b)=UI;naG76w z*g|o=C`M2GXzY(;r?2@>C?b|2C$b_HVl3YFC6u+my7>E_BVHBBtpPK-m!le|5<gR$ zf}SE8Ty~AwqmXJ64~TzLkq%^OebG8E>yX35r}1%=A=zCwBM&+VTvCcqX}yfqJaRwQ z-n*2)KfNy>bZ4*h5@P;WN}vv){+)liMoQq{p87|Mi6A8$u~yDY(KHp{r!(FVZjzAf zWm*Jk_t$5^rSixfMzj2TKU9y?Qy|Q|RQ5eaulr1P);_Y`Yy?u9xcj8WyylT$2fF1c z@%&WCic>G7H)yRewbGwz)tVoQ6Z{-mSGtxt%pz-ZoM60ho1BuK+;)1(${%^8<FRL< zdn}aHDYuw1aPli7qd)TEwIMrF9*4up9$f(W_Z}mI5RX&gKweLsVRZE!Q$@+bsB`s^ zFoWWqN9>qXs%!lSCO`>Bsdsx3Hf~Y@mRxr!=iAs+BL{{etl}Su;;mg)J*3im2dm*h zH1S}9K_~@|D+WwBQHUb36d<apmX;c!@&`FQnU=el2&2R0RG}A76{q((br&5x`2_O- zO;3*;iJeVceKhT;*J;a@_{Jw){@k*jDGKBb)%%aMQxGBHSMNq>3KB4q@}pw#fPGlM zX?G-wD7*qbv2tmuP7XHI-kU_wMQKr$h>9{BV&8?aQp@a#8SA_i-S3Ln<iO8<1KlfK z$~zamhW3Wl3cya*Wwaw1Nb!4xrg#Q<&CArZPQ<=7B$u|+HrZgp@2w^YyNn)Q29oq9 z2ZWULJLAgb$N1Y>Epy9xi1)y_kfpQeu?*}{ox=B4^LbM<qK<8RPWaXOcH5q~Q<pp` ziE48A>ZwCm$P05FRS;g3H=)Ya!3T+j@34|!RnKUMf;YI<{1A0jfc|X$yzhW7mUJz6 za!t7BO)c4Lt8kq<Ls~+ngKHN31=>?`0ENhxVoI+|>wx3;WDU_e#(|XJfXeFcR5y=S zR91tdQX?k*wl7AUJ&O(7G9)zWG5RP|IB?-uIDmu|>4R+8yzRy<x}ea}$=DzPT`M7J zze>l!0Pv&>Sx|}nzH4bM?24JH{nDJ+9Z(<NYbyr3X6kFU;DOgN`pq?k&AVeQI{uI& zVCE~q$>)c@F5Q}-V9f@hp0U`Y9HxrK+G356oO~Vl;_E$soah-Qqjgs#(B)Sl;RBO5 zyEq=vm(UA!%!lQ7;KD6l6QI<9mjD7%=m<$?cUO|CiI=7Q%E$(Eh7n=&<^GDGSPTT` z&F;o^>y#XL=%J75ZERoN`nT<eg(xd;tIqfGE<vV@<8mIh(t<H<3~oD3P~O$gzomM= z$WJXbQ2LzkiYW!KJxlYZ_SS2vesjt=<ibX%pvaiO$!D+T2#KM_@am(#-q6N+qop=` zxkQZ)C4M2WztEWgKL|O)j4)2-$opK-Y`e<6$OwI2u7=OVDwoU&6v<Zt{aC`>+1lhR zs4kNEhay@-_9q|huf8~3eAS)*L0IVzo%mL@>k%9@DuD^s){A^P<~g`>!9f#CX=M*; zG}aj@`N%LUZFLL#0Ze00GHL!EV?KDRfvfc>v+j5$__#p0w(?)z)B$oGIrDyi90epn ziu^|Yyx6HevSD?9#<|rLV=Z?TO~1h`#QbN*N?!m(%@~<v<RwrIm6;0_#1HLsRAUKf zQD$Kpbmd^wdZ*ji3LlM@;YBgmdq!Bf^s8&s2E(4xNtoA^<Ews<$7OFu3UV36!st;b z!MMi9oae;K;iEXG*mArRB+dDkj7BfSAMwXddR4E{>Juve$E9<prXYtc&u}|jdn!_` zM|TWAJr;uqrU<r(7>qIe@cGKrn}S<QJQ}18Ej`w``Ps-2z*$o(1kOHk*0QI;a3y=j zeMJ-6g}J>^uMID9snT<>EhTuogr}^~t;;gvx3yXX8&IS$u1_?))c@o~x7KK*>(7!C zeVgVq0#5~mlE3|u84Sez)Z}acljpAs+;5Laogg5bc5>bLc9d&=jYK-{jHP{(%FNq> zsOC5KRqi{eiLfr+_DXu0G)y=IIpFlMHgNq7sJhufGP*fWyU}lJM3|WiG1p03ETZY0 zUFO5q-EhrUKkTH-g@zs@NM1M5J&l~rSES7PQfpnHQ<9ZELxSkaiq9%CQPC2kr8>rh zJh`yNTAN6fCl0lf_gh`J#gO-K_rv+EF4{kSHQd2Wk0qBt(a*?x*MgUqTLwb`p={){ zwmPv|2nTt((Wd$F^y$f!_~^?4TTi8l6g}U}jGn6Y`y<6q0M6|Kn&S#{$G>J-n|1;j zAt8p374SdMaA(rdJ^I6;6{%3fm3vC!S<gZT@}8|7El`2vcv@Rq;?c%JODK|Tv)*Ys z(7DH`C++(gWmcnkgKOa{9sGT7TY)jWnHlCGngtF$>&M7W+nhaN$WDd&hO!)2{eed< z(vm=7%eEs%|0t!+jW*Bs<}z2mnndMj#b=_~N*izJYkonwmUiu49v1RyW5QhO7mARk zde@m!k3U!2hJ=R^1ldyKRlMcCB3c&qz{uO5NJ>;!E_EU6m`C596lm<dzT8bXAGz<g z9IsgqH}1{Ol&FsIb(C54c`qSfuMco%cE$Rsa;$9CavRzX+wiGJ1y7%fj*&+M|K{;W z%DUsxww*<*?fM3vu}*ny-(-<bqG>SeSlE798==14h=}szod@RGWIV4l(rJqhWAcXa zgn*ln7ns!&7UUhV;4LLM$7&BTD!0fkb)n;Sc1OYrDL9?rPXuPBa7Nr`z|Yk(<WwZE z_RBJa9#916+CTri>jG@SA|av0;`x?C1Zfr*3EV5fa9&4M<3$u2<9<J|H)5fzhM!~8 z3mum1`;5KL!V42<Crt7bPRkJ|uVHI)S1%X>g_8ROuVzA-0<r8rl6DEDOIKUE3%E>P z7eSvqPSy-|-}no?q=gr(;j=uCiS-7TxCFTTh-y<(Rm$Rq(R(Bsf<n%%Fym}xQDlcn zcTO#PBXv<KuL#j69)@_%x2jz;j^%J$KW}{4?+=6^;9Y?zMaM%vRo$KV1C#>i<345y z+pV-Rutc^BUT?~S^P^!Pv8qWzefW{GpfG$VAlx|unp~?U;h`nJ^`edbJqRVR<yK$S zWpIMM51Hi?*`2x$b3}U@SueHSX~#juL_dJJ-H@T9Q^1~2WRN$gxvKk9=9zOk*Pvy_ zel9k6OVy4znUIhIebRxmgw`3>!An*~cSM?~j9yx@2!F*=;Fn%!sds5YrMZznIeWV! z$6e-fnbGW1)WJ!Q!&-MK);h^FSwQmHTe6=mVQuhHZ9tIdfwP^J8>xFYqMB}YLt7%W zPnjmhXV?DK^FA;N$y<n$H~WVPPHtA&N&S;^aADAvMb%oSeTIOW*i|1J#a86+M^-`? z6AO=0u2sIulJfy>B(0V??eP96V(ug6jd#i-O$-aE3S-qqn(M(}G`(Xr9$AIR+(Y`Z zDL7XYpi2xg2Sw4P1i<!9GwDy8N&GKj3Pr23`m@1AF05bBKJupbm9pJ&QywGpLx7oC zy%Ot<KSx=9j>2I;HPDKa!5h~6jw|^R;(<&!)}~|up6Qvr%bq$V6rXene!B*25`ZmN zgaaJ4E*i%)7PH7G+JLw~^h4R<VUUL$HJ?$}sCh~U<=PKIxvwIn2Jn-#t>C?cJ6Hqo zR(U%)>0Uz8k0*Gff`YM#hyvSkCbhRn#KX8^kE0vbVmbu)>8j0IPf+ay=V3G6H~)^G zTfcvWCCpU$WjXvjLseJ-wS0$S=-cnzy7)CzL$PlU3rCiLXOHui<i+C%7n2$sCaa!- z#cFFGvQ}@6l4t$mo0aLxSRpQ|#i6^jeS6wIuJ!NU{#vs+X9ym3>!s0;DA{V-BS~vo zf*eQ!&Z0j38iX6kL?X@FIe`0E*+>cxS<ox@@YGuWS$=R@@x{woa6zu^dLgT}RA^P< zbhQ%|?NeHTNa+IO*1|y{`0$d!okear>-tm50Z55?38`n`$=^=!RLlC=IeNxXOUsil zE|ur(I~8PhAvlH^;JxPJqTP0zU|HtK>THWx-uSY|$5GN72Ha__3@^1b8jRl{uh7GI z8rebl-k743CbgZuo#nBAJMPG9f{nW~#dSc)otBwQ<^FL_%j(s<O~WZW+BO{^Moq7g zl0R`k@F(&hHRVrh{LjmM{&j+&=6X=>EOF717<m9R=LZ3mBu;ew$Ga;W(l7h9?!WU? zcwxp@Su5gIfqko(UY|D3_1A1rf1p+QTG9j@aFDi@hhap|YGcjI%LgF&^Tm5uR+~aQ zM_$VC%*_V1+XSB4M0p*lXzYDlcPkegQOI>Xb{DqJ&n0+FR93XQ>b7j&U2P3lq`;0* z<2O{7`je6R6Qw^k^qzM=5)m|nN?!CyWW1TMW`DoF+l<j=6`b5wAy#DMIz?JV`Y$$D z#~2708Z4h7(>AEr?)sTwZ6qG!j6Ya@&Y@7#|C5eoMX&GG!1{MvmYP<^GD#Kus$rZL z844XEiGu^Q<d=lIY6Ib4i#aG^rVMY9XOLIaUkz5f_6uN98uq6_dpRdg71b9Uw*y8% zOifkg8#fcez=(mvT&yX2?U+t_TY1?h=x2Cf8cR-AqjhM(Y0_7c=obGN&0deimFF9_ zVJLcREF45j<|O2c*iYi0$&(3bnMAK)V4_!*3mLouCiV;U>t5*lS+-19Dqn3jbm;+= zyVPr~Jugg3j{bK@Vl`WOE6Z_~xnwruLp_3Ypo>(;^+Ka}zf#=h1CBxvWjA>#`hkdY zLeZu2JzRP;VZoOE0yeR*=TXa_4_W`cKA9L*%=%JG%{4mVRI@=G`{DE3D|wj$6II`* z^ayR*)|~2FH656pKb7`7{QGdND&*Kb)?imCx+lbEgcr4q8XzysBF*Qym(~aU=J)$~ z#poK|_vipcb7|@HsCaP+;W5_r3Zc&Rbi2$Ja!(<|0-kc1O6i>ZeXUfaRBtX49dDS# zBK>I<QYXx1DyiJvHtlO6AIH-9XxSac+wL*uUgg3>9mNL9LhBi(8^~XoFGx5{3=&h6 zaI-CvfI3j$U5t+0l}}(tvagAjlh4dm>01W+;FC(h#b*l)7CW=E)oPZje_zk!S}HYg z7{uRXL!v^i7RML=VXgW5JueaA?Q5z-LqzEsC(_dcZ30cC{&K0dsxJm)89z6iNWiJc z&kT}Zr@iE;0oPj!Xjnz}%2c0RJ|ZG)uh#MW!4-HhX&dmIh&db^Lsx`hy253Y18-FJ zDld|`-6fho|8>WIK?ojk+^T|We)H7KcCGcZsjC{{0Es$5Lmk^egm){Owz<=9K!S~K zQu4-U!St}<)al|Md}V9JpOkNvXiH{QSeS^;)RQZjlBgXAvRc{EY>h?(kan4<NDpL4 z<2OxqSPeT4sO#mH<9!5*>^L(M;X!HP6x;rNY|bE~tk&HW30$lZkULf>%eA0~Q_XJZ z(JeWA`rrfWEnk<MsBDrMdnf=mWgaveLxbZfge9hWV?C)pTs>-1%TSo7V0S=GJWK*1 z>;?<_4zl{gYNZrtpJ0`y@an`Vw5{F_+HM}NR1)L-OP+jaVcL9-^k6T(HNHqxo1vwF z2e@)mH1R2n+5CK=Mz!NV7q|N`?Z)ySQ7VY%4ZnLp23w{?k2}Iz9v5I;Xqe+WbYw-l z6%T8>70m$QQgFFwU7vVSE_sgbbDApy+P<qjHXl2?FfCaNpBAZTV@=KG&Va;146I-7 zF=*}j1FU1K#k9zs?J4x)itg<+{Mo#D4N3Lu7+vXA%gwfE<Q$H((_?|~HX;&w*Y<Nf z*7AougC+;F&D1}Zh<oji#jQT?b;%N-x!7Nz_zy07w<^ZS{yUoS^7sF>_my!`enH!? z(g+d)!b&3uN-4F{NVnwD3y5^b!h(d-tq2&j2-4jkOLt3mBeBv52t2!s{@?fae1E>V z-}a6<GuK>a&iUQ<%-pt&JWiq_#b#VDa9qdC-n(DF50_s+Fr}^c4jp7Y=&107zFD+V zfSi-yP4hZ^Y6I@i)FiI7QBg&{&l3HPheJE<&iA%gaSCy~*bsIJ1<vKl4!&OBn7Irz zVrFt>bttGjNjTpQ|2^Rbpiu%_jHLy7KkhK?6C<7~sQ6hVBqg^Y4I}c*;pUpJ@f4v8 zqHpHafC}~RK_bFb*}s0bXE4P6D63&T^7F;$&KD6YSHXgz8(xAtD$zmc46=oO{0)lN z7oC$8LeCw2x+K~HY-%%du0*`~I&z!-pLgn|&A%)qiP}p&&0DVV(!Gmft7F(bBmm3! zfp5K6R`#gz19>q&o~oX*R9yKR*6O|KB~{L#3hiF>R{NlE2=D+Cnio_uwCPW^N=Nt6 zz4JB;^(Iu)A@9N%cG(li9t5hrv{j@DJ$Zn)nATj>I2`N1z`}df8lUw&E&q&^Stx>s z3_(I=?!6hpRz#X%FuWgfkFLruc5ldj^ozb@QI|39k^}G?JV1!`*MT;&gkGM>v^leU z%YB@7a)M4peXx1O$6lG?D{V(BjK62Z>s7FDErDn(^bldbe(ka`RY*{swC)>QXDjk$ zB^(Z$ihx{Z>MYN+Zhy?e|0+3TAcx91%k<r}`bHmOPxL$(pb(3Rsqmnd_!?CtQv+<P z!gZ=KO-RTiBt?iJQVo@>LZo=v&ttBFeNJR0KYm<4cq%U|vce%P_bDM<W1Pz{QRC^$ zC#1Mk0!)$@2=~l#H_gI$l(Pk}YWu=(Gu+Lia^a&3@r;bUH=FP-kuX-1P2Ncbdfdnz z%0rT+K(fSZ*Jd_mavR|8CY7q2D4P6`+vIB+y_nSuz5*-xT7WE&d~fvuEHX+#HKhGE zl!SaTndTVFZ#Y-MJiZd)A?!7Y+L8axBCZ&<QTn2>qw+R04ySb)Pb}(#FuFJx&bYhn zy1v^zvJg|Q**-9<261H_2D+XByXo}Zo9IoJa-Va;bdGxYjye(}!7p(BBu#IF0S|i0 z=)TbDE#v+)@?wFqw3G-{!gGodYN}~f4uO2g{&v1sZodgec4div)eiSkNb}FTQO@Or zhRuWxbOj)jscbAjetWe)oZT+JCiR-_m*~hCKk<`U7_j+rUqqLjRLFhoUX`Jn%V`ZU z0O+l@Q^u+wZ566&c|P0pE6E(V#9iB2t_mRUKHZuL&+|$LZ??gmOASK99!Jnq{TGK_ z0I)`(A?)^6iu07t8y>TQSvRncHGhePH4XcU{jH6Iu#q~Yp4Oj>X0^Y!a|C`AHhqns z7d20;gnTz}7l5s-wWb=B)PNZ1WHiwNl)RgPva%Kwnn+11l{@G}yxao=$HU*Y(2uAx zVk}M^3(QB0B8D05bL%XgVn06;aNwiyWLiwLVtEA?#ET{_0gr2TWm3zhtMhs3Q^H3A zXw2SP#U45dDHpBeAZ%!^D>&}3q(P(1GxbnxEw0fssBy~-&>80|)ysyLIgL&)kh;EL z{vGotLsTHq|J+XCG(p#@LTbqKd>Kge?q>c?1A`K$>PO>9?&~lW`=}Tb)5l(X2n>8J zDOr5uNAu$v4UWz)wy!e|6knjWTE!Ju0?HCPjMZ8XPh@qA_*uw^J{|dsGo;5>{9y2v z_}G_q`XD`;%XXNDs!OXGZ-Di`{2KTRvO#Y-gAeYUutRuFm*MNkExu>A%zZOKDHO72 zJTqpQfX9FH-KI&vtN9g*d%sNcXPna9AQts93CNkJ93o+j$r}x#nsb8;6X4sC1`v_P ze(KXv6VXvKlVk8`H7zQ6Vpi53KoBI=?y7e%V^*ds{HJPeTMPi>nn)7rgyDiR7GMCQ zhP6%pb{Y=?k-E|;qe4sO4kqzGAsZ5hpYJiZztK<2M_pc88W699%c*kr(yV%tW}N!U zGYSq)Mk5MQ!nX_c&!21g$TWl}rOl$e5sqvwv7^%ym=!ve(y7lz84;Qy!v9?cW3Kau z+i+m%{e*6fqR}We7uLApT;)G2M=>VWVWmqXp0E~Q`Q{MSI;DH0Xbj~IJyloeT5eW7 zO%0LLhd&OiBn`+(snih^&@g8~eulnH3M#|(CtDnDBm;ShwX&F<Lo@A(!{mfJLbq8e zoD))dX<fg9Va9x`iq;rfwfeV*jdF<p98dH!n~?A%gC5?B9mGw3eKodZqUf>`7rFKd zGqFFj*Y0M(XupBx^kz43|KU-u+mmv$bj`58jRl-^$tFlJL3A-6iYYcF^Iw$|=8@cm z&sX2Ri35|_Y|6JsKDe{E&Udap$J+owT${nl@G;QKnpXWjmjc)wXBz9FZhlc3wUpJl z-jxFI-!df+hJY?r7;}Z}qe_nrx3^RJmorbUx203~H*0ScH8fZfQoJVbs%FpAv1JDM zudN=)3>8z>ZmV)o3NiA04>kaMj4%IZY$6;Bn)p;ZSVwmwdMv5_m>?M0zT_HyC}Tj^ zz-MO5={GmIeaN-<Z8}BJ+gVJq>IZnrqc`+g)UCG7qTIJxqF!%{m})zFQa6_@$It9| zWa`+;7Z~#}a+RlC%9uaPkpzz^9QI1XLZk9a-U(_eBPzcP*)eSgM`O*6G{pvzn7zL4 zT?RQoB4O&<5PO8<LgBcW&e6}g&QV}zrD|`KoF6nB`?t!_DG?cL9_!NATG|ivni`O6 z(>S<wZ1p$n(E=e)|L8aY0BBOg61Kf{o7$P<`AuT@`Hj3|L*(G?banN&_&vgUgnXCI zIfkUjHb)gvs=4FVP-O#(%+X^zfswKbXF}wA`EU(;@HE|oDg(=p5AOoAeWHe}`7;Ab zaHL4_PqXcOAQ*p&aKXv`m&8R(h$rHsg*J40d3N2_>7$0hI@WDwe|2fVnptn}lKnB+ z{CK2iu9c#J=JDUigP=j_NxSDckfNvNA0?~#=)JO|P}_oYJ|icKY5dphJv)yq-JRo| zFPNsay2Y4;qdt<kDqFI1xC84>gO-)Ec2epNf?r}EbZ^@!X8nkm`3fJY9g0_f#PY#P zobnLoW(@1~++}4-6MRP=T1z*YWH5xUUk6(%cd<|@eF=5{Z=ysqEx02-`CynE#BkoJ z6m@P}3^x2*mrxfC*TP;I-EbGC-CK5FS<Lj(uUySBhzsYWlN&~sS{j)-)G<YXT3r!* zUuLXsB@~n`N00EDSn9LM3!oli#bXJGM+Utf0+|#rVaMbMuuk)eHD#NdR^;X#I##Sb z{5wt#+55xa?v_AWTV&Det_N$O(2UdUR{gwzpcKt@rSb$53bEu*K1Zx$>J%UOa7x=n z#%>b?WkwI)Mif(iEb`J{%M@ru_E>b#`K0n7Wq&Qj?gp8AM8{N|grS)vUhf966iBH| z{~bYn@WtZSOf%E$t+ld%QX5T{1c~Yh2?HCvp=UuGGm=kJ2V&GIXxfpzd8`@QT~$7E zZml)yaVn%E62TnojIKsEh*uv~j+C15ezs?!jC=llG9`HWz7pHrxavm6AhKrPG~C)K zD<<|mb>MNHh9jeH6%f7>EINI|XpP`{%7OuBuSEjnw`O99wcSfcf2$8oa26K88-gk- zp4<N=4?dQT$VdTLNTiluo6e#4Z_kBlTYP10iRcb<jbQLp>x%1HwnC5z(X(`go`WC& z)~bXWL?-||{e0M3Q|`+4J5WPX`8jmB@>AlJMrR_Av&l_>(>x(4<-?o%h|3F-R^h>_ zC}dW!;$47Cqt1<G>u}ju#P_B&kcoD)tA?{OVZmpejO{KStoQjtTU0lh#$2CHn*tf> zSy_71?Zq?s6d7a_6^E_bEClegl5qm9%N=jGy`AP%s*O@)<fiB-KDARKobF@B>bNWA zv^#8#L9B~h&niKf>zJ!!?wWr5Z14P%>A^9Zs&ArOK6`EjcX0j<-U7@EVwNxu`%!|H zg0UIM1iW@kaGcS`$jHL{4?uSqd9*S0g3OLs5X~c%Iz>S4B@U4=T1-82V74YZyA;+a z^6jy7E|L{2_Ma6R5=Ra6wyJxQVVYNmGWSReKYF&#A{$4!a;%>%@ZM|!#$Ct^Qdzzg z_G4j9Ckx!?YSj;I$|+v8m~E<#S_9%K^^V8hLpB*;`Jke}w-D#?nW-nJ7C-1&07+6` za3;aChG5wyu$0CS1D1_dGvs+#hm~;LJF7wN%}K-iEx?*!1>7c9Om7(Vb9u8@;!)hR z%>oBCZ>>6C?0Da%>*$h~pE}A|2`4!c=47^#%cgGC6Kq4r<GK`o*JV)p;g*&z7oEyX z8}XAE8&%T(mgRA>-6w1?VvnV8#2EB@r@epApC#03ocw)l<X5Sp;3BJvhvSo`MZBCN z`1><{s?~(EK#qsPYL>z~(Ipk^OC>MwOONPvWipWk3fbQKb<nJ&-=e;f=u-M#t`kp_ z<eg_LS;N-ow=2hn7hbnR7I=5S{x%O(lo=*|NJSE}Jr|%2x<mVFBH~CwUF!W;wK&zO zY*SW-$1RU1m#C7vBt*t`ezDwDQkqJ6UZkV#JgwlXFn~PR@PqK=EC>hrbW1-#-jGRi ze!Bno6j{O~0RXDOL4tO_6Lh9uUA{Q+Thz_7?@ngT<8#$L&`+^4$p5=aO-K5VgZh%I z%%E(ALUwJ|LP?FMThr?_FAh5TIgtfr$i2}vF+lT3!Xwt|SC~LE??#vlj&X@Gn{tte z{m*heeSHB_p6_F8!9^0kLgl4i?6U202`maRT#jjB1iQ@Gk6Gj_B#K&lGIT7`5;;PS zbR6)Sjog45kK=Z+?MSJZN*Xz$RL5H%jd`|bEDAR>DzIuU4S%I|8?&TJ=}S=LtXRF5 zbEJ60)OoB{CR(jRYBxnwu1C~tpx0uw;qm@@S0@neza{9!`DyS4q8z7cc`?i3vEiRm z7i2vcgoCXv=guUB5D4!N<KN>15bY_rvwl-zSr$dUXDJ9%EgbhJ5l-S$>nV?BrFc_O zz}k_(WlMW8{WX((c{Slcqq4!TE!)gdYl+0$($bRDxS6;t;8%aU_ym1hD{Knrd6Rk2 zxym%CXg7#-v)S#@{U4M0nPiLK<Xmwx+zME0+U~Iv;|L*c^>DKKWBX6|eO;29Ajn@b zx7*}5N$;O%e~1dr0rtv}nU;=%t<uZECw|Zg!t9#yb=r^5AODSHV_5b5UaE7n&>7w| zWIls^qiy0&OR(s5FAHS3wLt3)y{8&q*9%pnsiZ@LIj}}*X~)3_{HmqxTceq;iXShg z_-;x&mRNl+A9Z1?4<hKgfN{nZEpk2iMQAm+V9io}r-~HzIXw1uSn)kbU7su&DSJ*w z$=2v9SR@dC>*KJ5uT1_f#WPP<+s#U#hMdpJRbzwqqN2H8GsYAk<5`A+>g5%8uc+6D zU+o4&)G`?v*^=oKWeKECR#_gFpxrN;BgnS9x8iIu#ysw<udb|=8ya`yu9QXWi^15h zDvogCDB2Q-_RVf%9$s#Hr-74wDD5A?Oz-TSnW!CMU;?m83BTVmGorsZ%B?uJczHhH zxf3!@bZR1}zm)!a{OPU;01J;q{{A4$SD&LxpSHK5JFw$KXhu&n!1<zROaOWIAqiN) z-_StSu!B3DhQ=>~qQO0nZD4V3S3o{Ryb#t@A8~+ZxC}CyFgyXpjCQ@=AoB5b9zUou zPAdw=ZOmwB6Vf@Pqo?IuW%7PKCH;$UjMAjt=EsO6$X754)awko@S5U}t&WO?W(@2i zRLhe>1t_!Kf6h7NE2ex4tEqYNfX#WR&IP6?<{O?sv6p{|@b$Xl;wpqbvftx9094Jg zFS)?(6;A7N6=k%iHulUt%U`)HTmDHe-t)6F%KPWp>c~9xNyFj{ql7<3Ko!Y`ule}6 z?=lwjSbgqQvn?3|>To=sP*XwRXc`g=28jjIfOxFVql&p0Vz%@>z49*`au|j`#|Nu| zTii#Ts@$u#a+lAJ4pzo9{q~)NB0Z?6;-<x!_>yPWo~~IGNKaEocQ+xb@T?rWSBv8I zSD{$noH6T+u`qNK$E$eZ#^vMH1_vVde=eE95-mp}=7{EY<}dT9__tegS3d1ooSv=P zQh(PHI^!^r;W4aA7Jrb9+v53~Ne7(j!)@Yw)N>xwF(hDK^!t)h^0ZCwbiR?wIlaM{ zTF&H0bs^;C3y{dy#bYDr%GuW_r2D6tiofN>u3JKmZy;uC9DTV@@R1fGEtUb4t$&v} z|8_A@Am)1xa>7EHUMvz2kKK7hA{T-iS9ID^KsB)iqaL4j2TH3ORK?Iov(9hPvkd`0 z;|#jz1?<=1Svoqj4>dD#>7Q82?E8w0Rn)xSo;d7K1qq39HN?ePX_IzPNS?Tt_rI5I z;dL_1cerxWk|n(};LtLK=S9YF!i1Gt<22jvd+O<C^zABgu9J66J95<i!pdmRl+S2! zCT0q!A<sp-W$~*c8ShQvp6s;uB7!=uCbADAyC^%Hcxd7q&MvJbDNFPmOR`o^<B$dq zP)sC|)Z}fX&vz60zJaP8C8H(*o?Sfr>3VM;CJCKmzadBiMS9N^xFYhiLd1(aQma$I znk`GilbjgQ@~G?{dnKYU+dH%N{K|;tRfklB<<7jfg`eAP6H+=8lX&Mt4z3c7>SH~G z=&-Gqh=T8OJTznk0$pQ(wY2Q-rPiLEJ(;p^1rK$9refS}sFHT({dOz$*fNE`!Rvg+ zYUCMn@TZM1WEbp$mt3UkR|ls7>pKnHsoD1LE!!jG`;MgiA207vf55;X=2DQB)QYKA zK)OqXYNDRR%lP-)Bm@!Cyb2<LqWs7Z57%6W_i<`@*4MbMccKn0I0xw7irwN^y4ZmF zP*CON8E^xJP9qjxUL2bb*JcQBpCz3Zi0SCEjmLMIeEThI9miV-^YXn)oG&unaqtnn z?ESP=y<rkq*XXCh{i5mw?(8uzH~B*xq*FEjbRN2PV?D%U{|*w;N-fC2)0GZZ2WtBK zpuw0Qup|`F`dA6h(KcQeM?QtSo}%k*&q$GMlc<B|;dUOyZiOA?RFWGRL=@H=Z;Lnf zi3?mMonGiVU;SRO1bukX@#QeYf-gyrwmzJSj}_%L63;jA`Sk%R(b<0V%=Ab563Npe z0P?dW+!|NAi=M+vONZMJ_2nabyz(E-&DYJxrHkoQ)xN4R_Hl$lvrcw1sKCPm=iV)) z2JOx_UvDV+I5l#%y)|y(K{FoA@RMVutz!E1wP;Uo%)FR+U*fx%sPXevA)Gf>V3G}p z3|NJU%f+S~FvMYW-@-7m5IF`7o%r`9Gxz6E=AVd|?!Eb0euRfp1WJyvibUvdoA1V< z*FxWulF$u6P=3S&uw#!sk|-P&6ck)N-1d3pOHP$~et97``9tOEfkVT*y?g4E=A=R$ z`5ub5(+?P==jLj3LDygqVx~TwmzqV#EVwOUzWvBt8D4P5_z2++QAmq|gyHMuGi|lu zqmtUumz&>Ev73ZJ9eWeh90KyALPa(tk+cas8%q?8%*m1e;I;VqI|M6J2}>{$u9n#* zmWPG*4>mbgYH4X3Qp?M#L&Z9&_I?>u6oad}+(b~)oY>UGCq}RO%$Sj~P)a@2go|r& zYmR`?VB!O8fegC!LyJuXrsL!_VuQ1Vd9iizomIlMlIZI%@$lYUXrmNC-kiYBAYz=A zRf5tCqKP&3L?T5dpKWs>RlHAZMF4Fw_&$TQvHWvfv?&q`)PUw?ONPlH)ZH2ccFO)q z8Ya(BBkytmW^u(hIgNv+Bd+b#0?eL=YRRw%qt&dJfY@_qoV=phRzwe-a&O_rLfQLm z>1!)~lkUznnP8(wA=wFJycS_kwjqY6A62hXR;YtMwfiUlanan9V&RDX8>=bdpX6$+ zErSw5h*f(l_|*!wy00=?W~F?Zw_mZ$y$A7V1s>ieVc=#dy@9goIpv1feS)`-2TG^V zPJxdgbq&Zq+1EsiElvgiR(6u0+GrOg!MAbcO!x0b*P-l`U`U5=*NB^XDKeokR3b-; zjT)9for{%8JC|UW?|<}p^BQ0%VAksrL0W|#1S23D3#(J@$1iG>3U(DpvC5|VI#=52 zBIpCv@NLk`7A5Cj=wTK{v9UUfR1w&ykOCpcUK95ucqiygN&6UmaRmIgm$$*D4jbwp z2{_0f!;2!fodw0OL%^+<JQN(+sUFy<r4>#?AmXNK+|uGEj}{CuM+ycCyAz|M#hA)M z$*tm7tL~rJJD#Fo`*eSI!pno)*17lk>Nup%pZE--(I5(FEP&`egL;9W3gV9_R1;Ws zlU!Hd8h4|oGIB6Nq;PJmGU2~!-RjR%=f935e>jI%PH?D?yX%8S`VWWE-bUZj$<83r z4=T#N1{bZuH9=69NR;_w)O;Ak_+N+7?j`wm=Rbu0G2t44e@yttgnvx<|Cfn>tHZyT f@P8Z=;8&QG`)R4B{R0*l$e)6Ys&t_g<kkNGPO&pU diff --git a/site/img/dcos/services.png b/site/img/dcos/services.png deleted file mode 100644 index 60d7e6f0bbdda7d6ff17ba1ab506ed13cf38c0a0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 102032 zcmeFZV|1R|)&Lr-abq_&8yk($*tYS;HX7S%lE$`e+iGlEH{C~jcc1T$`}dA<@+T|n zS(<ZA&$$xvRayiN1`7rV2nbG0R7eg82s{J`2&4%L;=RWmWI7uN2$;!KQ1Gjmpdi6l zJ8L6T3qv3v@@Rb>9Y!%~@&R349i4$uDhe1oC%NF@Fgcy}@s8e({*K{}PMwS-4UG*9 z#0^k|E}$IQ&U$m$FoMS_+JTZTGTCu7UMUnau5J=@W-?3KAFe1J@iA&?u(@wQwx)Lc zve2NQM2Jk}iHRuQ3ryY?(S%t!-p9-YjvW3TKt0mF73JQu{$T8&i*()=0o!3cC=d{& z@C52)&|+M;A#k<mV7Y$J9v-OP8TE4Ku(!(ejEwYE6U<fbzdn<0xH9sInSl>@!HzwE z$A3vaz$Af*==dt3cs?RAH!TaaWIEIV^bIuFOlX0yXDo@H-uV=~5>ZbafPqoBnElPR z0u-7hODo8?=etk^A`>E02Kxtwk6>VYLV~s_l}Je+(1Spz-GQ!w<vv?J6V1G%FT+$p z#a=~9l0)Cxl2+HiTF;Qy+0y16y+A-*&K&PeOGA5I0%uDLD*%TxH{l;6INsa8y6FfB z{up9!&P}Ky^_4)-+Rl)Gnf5a+Js}Sa0RaJ*oq-XDoRIK8r@#NjO=x0oZ^J=H=j7x> z>%>HBZD&l!z|PK2N6$#d$Vl@(f(GDXWv}Z@V+A1k$0Gk)N5~MMZ)a*_Z)$Bt@M~RN zJ!=PhZbHIe8~yv|ALlf5HvP{|R)Bwk^$sB2uP<~AwDffUUi*D2*RNiVucpq17Aiuf zmWEb<_dR$RJ~Oa#{W0NhpZ>GupQft*XDSOj+n;Ct>B~Q7a?$;Q;7<_!16_agzT=Ar zhKugsxaWZ>#r@$81jGj<Cd9Ac41Bx_t)XPN#Cx8}bMJjzKoAU)QUC=`yOppVv&%wS zcOa=>IISl?s(qENeBzN>vc`Jj(SQABJ#9+LTy2r;owj1v`I(^~{Dc3DiUJrvIDbOW zm$!M3vFb_+G=hP%41{q&2H+I1e)oVkbMW&ghh>=<C^k4bpN|nOBF+^4moL4e?(rIU z?>)dUU%f$4vVi{GG;sm3R9}l})cgIzpU02`Lxf<9$87yJ_s>IoKwy#UJ5u{d<o~+< zpU3=fng8MDHxe@WAKm=lH1YpO+yn(;1^+`%@Nt7ZO80Wo@doNT-FHu}B1MpXgn^+Y zIOu@q;&=;pg`M!$eB=p%E0^(i<A+J$t%I_KW3vZf4lS^Qb$36@s;U;!6@CXHW$Re% z^ao%TvLz?LT}Ah)S9PKb3I}#JvVDB|87n*I%w%iuH`&!Oj;s@PZ(PaTC3aw|(qvC` zN%TYiA}!j+%%6gd7@XWD6dX@8Vw)s(L$@84OoYv)p*w$BUKAJEchQM=(Y|wu^HIm- z4e{Ygc(-1A+OlR<e}dc}Z0=P?Q*?oEB@_f_87<U<><a6w2)7jxZ%SEUn)Y6q3JMt# z)(hw^pkZq}LZ*_ewBbU7o$=q2_MejQa}v<7)MtzxI_!d)&$yvV<$YC2_HnOO+0KqR zP;rDtW6m|EAAQ0pS_H!d+?NqJke0PsAcG|Um2bo0D(78<)GAd-LE|VU%4ah)$iqC& zRR_c{L&QQbaT#J?<fccLa+wqrQm!9h7!lj=)N$6{Xnx+|3)q<@7@!pIMx5I+A)=n< z-I<5Y!|Vf+DY1CX8yV)jV)e0BTQ~Rj;aNqxr7{Lj|EwT>9owcD%uJ@dNwIjdB?YKD zICYOz!1I>HQa4WxBh#tSWi8LBrD!mEztCejsp6?ul<Tq4uyj)8?NqPj?SvuhY!~W` zvx^4O;>;sRUtAlyMXcardYpCncD@Rg!-Y}+7m&B!rWT>JTx*O(2!f`#Z2C$L;2{JE z4CxYEE@>DevfK9v`MYw8^Cko2aRDjwkt-TKr-eB(C&eZetv^2W+ioumQf{62Vhi(f zeE5Wne;b+R@7;r7+oW~^om0((0|%jy0we3*%7Fwyl$MUG8&!qY+uO^E1e<WRvdzVw z({OGyR%}P%h#zJXdy(4)0b_`pU_{R$m_SxP?O#=8hHiS4tL;~k_jOLriK%0BT5~sQ z3b!|)fIUs9cc)pAV+KcnI=xqcZ)c@y$#sS+$nAGMn>7iJL2PCJ^HHiJO8+IzN|TPP z8JQrX-U>7A;TG}hMYY+9pA;m6{Vl;@$FV<!9kmiC4-&Q`O;<q%atU*Nyft;-Cf$1X z_TJj`ZOMWi;<Ny6<<r@uO(M_*5I(arrc#?O7%kaGGG~5j{}iN3rNrZ%eg6&-Hd0j= z(%?~k*Opz?b*kAZD_o`*Cw_O67~;g#nkOi`n6~bz|3jS|MlAYfYGQs)K1=4B?arlS zPs=6sYsIH8%Sey=$Sv(QgiS6BwD+3O_L<+7gWd4d^)Opa*WhYj0LsCdnI7TZ`F-M_ zg5eve7BKVWuQv(+9Y8|i&DdIXc{${FY<2E!pnJ6o`E=hi+`(Q^RYiHuyrb~y+;T4i zWlke{Z*z~2G*YnxB{O`1eQ=Ae2PIz67*Y`gO9|~S&lngdz#F08I`W?wPaZKDo+rr= zPj|d$aLuF8SwKPEuZSf$&~>{X%{EnAuSkx=*Wm8>w~$WH^f=qeK~q_Je%3JWpM<@B z#P68EyEUf}xI8Vd7<Px#4CDzb1Z|0KacU29LU=yWxRi6+j2#W}-UAxBUtQHJtnXlu z*;r%%UwC*`%S~#b|M^M*byfe=7GnjD8FU^g^;COQOB1F3JC{t%fKane&is^9(%Twh z4>p+=7Dq2AhZ>1f>w%m)BG-*d_pSY4obq(I7Z#}&TuQ<0hd^O#&H!-t*Id6ys-}cs zb}eG2j%OB9nVWz>+RCY*B+b^s$~#~Pj(L@3nxDf_zb#4s7;qsg5glcXH|9RNMRYt* zI^SBAP-{r06Q5STyw4+{4^N<bByKr^a+O4ZjXsouPDs*@Lcw4sSsf<eo6PXiAv_X- zWxZEYW(%)}eEfN;o{*9d2HQ~kaZb?@Ag-<Y1`L?DPv)eggfYJ<z*QQ03I3Mi$JU&E z2;MJUs=kpAQd;9TRDn~wjnNk3f>6#yq}py2-;gImn5)d@3R?r_5VUnAXsrU;lSsGs zv+DNcBm($uy#IFR;|)`-PI}nG)4?uKWn5%~jj9)0iU<^*%S^_L7;G$1rJ2|=i`m)Z zK|=7lg2me=s=0TA!tO|83A*7qbh`hY6V2op|N0kE=O<f$_k#a0JGg5y4w;t_%>xh> zsRZjmweFw-z+2@}gXJ#z=#2f+*22}LfoHDNlSZQNZjtf|)#fxeF^wJ~tgGZ{6!_;i z!fjL((|*mCr=QvVeSDWXdDZ9~6fEex@7ofb#b_xBlrcEZ5ZyZz@MJjJ#iJXCd0Hee z^J&NuuVa6C#A<O}30I9277%!Dh$>9{bO9e4m1f+$yYj(<m=6|Z)4+_RrN+`4b^@da zm3&}&+z+gcD4(YQGMEZxo8#III+0H@3B?HSD*~H3NY5AW#4NgA@K1JdcVZhF%uaNu zd<i<vxp)LDV!XpPTHM0=0{E<Ku1{*9JxgLsp_J^p9p?v3CBS;}{J`HvBy16If+5e~ zDjfFEepfXs$UaMQ#a4p?YQ4F66iL%OV8z86#|$Ov*VrZ1$ABt}fo3oUq9&ORhGtDO zfci1xiTM|Iv|ad%*@(VUoJiTLI>ob)*>Q{5@_yUT)-;4QU<>xtmg~5oMK=lvR<6Qr zb!F_|l9G{TBa~kDw+}+)4w|Jd;d9WW`$u>jL#27WRf}&-4&^MJIM9aNhl}7pWo|d^ zN%c1Q(=^mMws_Xe9dQSKkq=Uv$4=yd^e}LNoTH5mlH&K2TJhYrw@kY({V?g_`{L<w zufMFMU*!HJUsF&a3T#Te8p(|st@7pi=CF0dLmyq&#DZh0tTZM9Iyk)ch8?Y0x|xg! zovv3@8*cVh!Ad^M(PvBI{L$F2;&Ky_oz`dSRnAkOEkmP5$hRb!&4CLz^VNfA?rc)V z%itjtV>hSnLxsbje$tE8H;cPyz7t3cI-j+Q!ST)Lht!*~-T7f-c)gcz?LTkE>2$iY zF~VY~*IyBaGjHKsZCI~&+$N=NXr$ti(HYmE;;&%L6P}5NGli0F6}TxvF7CcKU!!_F z8}lxNykQ(5<Y(OJnQE)Ps88QOu4+p6ddZ{)*GCQ$cxZR9W^x5&Tq3qnTqWqN!6=?6 z*J`w&BZef_o@_3hpRCi1qaktV#T+l@pR~@>f%l4c@`GfX^@2Azt)wr-e3NVMd>n;V z69W_6gBo$Z$7^V~?%n5E-^?uZ&MN?s#0>W2Mcb=^)h7y+j0dX7U4n?f0w2$%i_$&; zS$zx(;F^~*A6%Z(PkhLB$?)jgnq+7ND)(%tmjS86;SBTDZ4Yej#)Uew=-}ZFz)bHx zi#sDo=6#}<7GtqN7{32eT@^uB<ABi|*%=$VZ6r1<vF01;m@!kwdFO?J0C(jlxW`F+ zPQBP(h1QA#K)Us;Lt8foVA#xWCkMw-?{2t>e0tqhwku62X5L&#JgPjiuVm563~BFg z<<{81UqS6nZR=WQl*$~G*;?ZhF*v@*el0XK;?(!WGw-Y1udk$&_w&t2KZd;9;M{Dv z`dl5tEZg>**U`WXGD2)6a0goaBNen~Q{`i3p!s~+pfcXic`i%!FNKCo_~tp(z(4aV zIca3hG7|%tBafIR_TXljT&_OOy))ugf(5@)2-Y3e)X#aovqtzAG^<$%TB#5wNDch{ zgl%e%kr-+VRYqw}JWg|>DH@4_qg4RZE>{`;N&|DYI9>c`iQ0WkV{jCKZJojX;ig*i z!*RKbrmstzdr6gs^*u<B@xgp*B5EY~;}?@H4pSdvikGZb^X6?soH>id@a=7kYg_St zN`SZXP%}$tON}#^CEgw%jp~B4JKWTz_~*!yZ4*q{+S%aPnVqbH+JRy5zUd_%o!(P} zhqQoX*O}FpL~E3W%8vZ2{O`lnUcIM8g>$f)4hcb#ybxh&{)R{L6{ZggmRZ>^&$V~M zB93|wcoh5$F5lf9x-L;UZB=ojOID*P_2;K@4SKqo@!zF4Ov{grX>R1+xZO!sIV$b! zx=f7hppSrvQ~Jb&uWpv39L91r%zPygZ7&Pz3m%lH`4KpR&$bk%le4m%6t>fD!B<L0 zF?466&gQYNcRup-HeBC#x{9pUl5EMb&RU*xXB5|hM9w^c&paC7>Dmc9k9HT*Ku7aX zzE#kjLSp24mK9~2H=Kjs5+7kUy>|L2DtR6cd!>N(m^op>3t97L%;gjtjL*3{r+QVC zJJygpguWlfaXa6lcMO)>oSo0NtU$6i8Wyz!!WF*4-!~lUWUKUq(?#XWo+S;lRj~uE zp()vjIsN8<7B^psYle(u+_S!^oQz6<nWBwy?e}f#D0}{L^y6a-zNs<m>k+4*GAMdT zXT|n!Bi5zBdN=IMs&szYK_0Vs2SoZuqo+%wshc11JEM5GH%I|#p>MF6p_<o$<(LF4 zp%>wQ3D0D8lp^bp+Qts%>&&+3FIjx3>(SYbBDfmKz+eQR*5%@K_!~ajp76Qi7@6XC zSCUww3ad3kiX{gs2CK3v_;T3}(58o8$5&rh;=N%oeCSuN2Js6V>6yECzlTdFyBLdS zt56uCOER8aqmEW0(+<;aKr_EY-{NaCBz%fst``||L07qhts4#He_Z)AF+S8c7hkEi z*C;D^9cy4Qi}&Wt(04akpJzi7D47{zTdgF)atSQj3l&1zzlnKubGp`qvI}PF{0QGU zRjOVwV(rT!J(@@nJRaJKC^H;wabE;=R@Yx?Kjy4r3>(vNn?@0%h#TsE?0%rb{(a01 zYtY;{rJO(FI;2Y@MAEvt89$@VgWm2N{m_ofBgCL`Jb5jae>lj^$AxPMym-3OLD1C< z4`*ayI86W7?3h9-PDs(+vne_$C}v1K?njy4dG-7`#yX-dVd-Aot7s-Uy)SRm`DelD z?V#mc&nWK1NhRXaJyuCZw9L9)uLz@8%9x6He8w3}clVu?B0$hskr8uu;VK|6^<dr= z>olG7?ZtD79tyw~EfEd!(Q0$mu`*EKJ<Hp6^#kH6L^&!{eHUueoI)Mhla5f)_fR_* zwa2I8Igl|}pBB}Ww>R*~RU2Fp@f!R`KHPqia~J~Gh*~HsGCn>aMVixi(hb|!qqWya zinjFPTb7m@jH6!BvWc=oE6T%Rr*#N<>dX@X<Jck|YrgqX5z1kQCNiZUt2@lQMVf24 zzu0<M@PbuCJNMZADEl&t3Ebw|u})$DvRgoWhc$|{3c9l05AVvESp*qCvx<Xr4APm9 z&b$_COUwhxO-RykN^Z!|_lyOqfKRa#9c_o<c01S9fR$qd`MZ7bIm3IWofg{~BBJC& zTtbT1jx)@UNZ6umM!Q`Noki-S*C{#FfjAZ`+L8+ck4PchKmuXLfSUw5ei;w*Q4MwP zg{fJm0s8?uPl?-emRwOA`(Q9mMS)Tjw+{c)BZKGuO1&nqNTNWU$0@}mQy$r3PMlg8 zY5qDL6H`|XLbQGz(>O4(ps;yHb7hGP8;v2o)@}cpk8@ia62t^0nR}deWd?ggwSzag z`NRYy++W-xlgUV!MgWBwb4Tff^XB;NiXOyYTZBQ{tEtJ7#^tT$=S2Ayr>%m(y!NbG zA(D|E77yf(R7Vpd{2MT+)$<W_nh9T8&cUa567P%a2nsVx;w(yM>IxOLOL@Mn>RQQV zq4|g;SocuH3hHp+(J+^zA8|*VbuX6>BtBkG*b%ogK;y~f>Mpz>xrS>mYEss^e~=_? zZEq=x$`C7Nvk;0{^_1t&u8nNF2sC)(p+zW^Cn3v-(~Nd43>oe}zx`pRS^G|a`1G@m zHFX|E(p9U-*R*DZ5dq}n3qibLc>WY05_S55`Jq~KK60fQz#~8-53O^iw%v$CHfI)F zE|9M6MK<St_B8`Xr(GB=oxiOO@Y=~7hKXIk-cV7>GTc<-SbkKwow<5(eFI7LE00eO zvYnbi=#21$iZ3yc=L<%=q7mg$$O-$h+PQ&UE;9Rtmb%&isF<?r(t~vV1Ek1ys)e$O z>{j5qv&HOIOWdu?Q?ANg`$rE~u&!mz5nK&K3`YdK4@W+5*0a>ZvhC4}p?{ThK;T#c z8?J95ONd(l*lIr0rkUE=I(-rXO6*y^FzMYe_2JiRP=?3k<;zi`nH$p1J&Tv5glP4g zR5D*;(76;na^PLILYYRJQ?L>xhH1+qdYLwA%g%n0O8X`-QIdjYDb$zR$XM}w3+T^u zvm6q_SKn+^XdI#x2SbuQ0tpO{gii+Im}MPZnhq`rn%U*@Riu=hs<o^wJZU~H9PW@< zYze_ZS9#D|=Sd8P7RXi?(X|Tm4o47}2UZ(Kwk-GipEv?yL3ourCRf!z#C$hLIMH<w zn}vMj!ZOxMB&fV~5@d`SOF<+BUR&V~Ww>t~Rdj8?TyukRpzg@0v+A4B3?1#i=F9i! zZ@1(n<^Rcl-k?v&!nGpCX>TO+*=(c~H>9s{rFVaE6zSCII-2_yAl>amrp3>iui;Lw z4{axrKg1yw?8SgwT@02+_A?#fM|TbZ=2J6m=MN08w53~RuFtyBZ6!$Z7h~^+<m@`O z^lI0Ixc0_Q5$eZlPpSK*m$gbcM8R6Zv>1-&-k47oUAV|$<H93>bYDrXPjS3Iy@1L( zbMQYpFqc<>HbI>nT<u0KrJ{3<*5{}cJ6U*@!0ug1%Ed0o;>G0~l`(>XG(DN8Q^8OJ zAL-$6r3L5|65&jEVc@jPu%og$p4Yr&@dBDL@~fqw5(pk1{4xni2YEhXLP6n^=8?kK zP?ut)m#X&N4C0j4;rO9HiVU?=Ee^6(SyQ#G=SN5gJM42Ou@#l8IDK*UMsky8%4tZ! z;g$PqAk~D?r%sARo+@)djL})KM9EHjvk=^;$Y(%srp5?f`tcUC&<k(RGz9x7bF_Rh z=2L1+*_dt26y&4y_!8vYF#AG!DPBa4c$KBZWmObTJGi+LHT^tog2C?kb@x)79Glo+ zKut{zmbSVg(Bev<(CAEE-DqqTdd#$=P9B~`29p+e(siAKv=hLa4M$>5h=_+BPE}Rz ziIomBQk`S4xx7ToT;8l%3)(;bD9F4`DRkG!`pd7u?nB5zXAz0!0|FTaT4d#wnU%Yu zD}p|KC~S6f8h{RCM4t7VggE_zC9Q`IRDbqD57M-6iM&+(H`+&Mj&Fn|&I5B!+r!uw z+YO?y*q9-&$Of%i5^9t_+BYISL|{3@^u>pG$DU+OWAgBCx$`SG2fVmvyk<{;=VLb+ zF>jT0rNCV7)M#>Gi}-1-*PHMpUDhQkhEPOiJ4E4dC=ru48`ynbsA;rTUuEGWPBHIC zgv`8^n9NR$qPQ<nOU)U;7z{UeNhY4YX%zarHO3{V#zg881t;GMo=T!+8-a`u@B@I6 zUe1ek=$eZYkUxDS6GOsPKLTWuoTr7xsexqX%io-V-$h+uSSg>+H$IVv9p%l&@EIMd z9(g|;0y|y~ei@PmYaSNvL=Y{)Qm?yhT?`+Yu{c3_lSRmiOg*vC?;w|qzzIGF2Aw=G z?g_9Y8cfLH^+a_f3yg5#E1NgrQ3wQxg*50$Yn7GM7*Z*-C*uDtS|<jUB(_34!WfdD z&mN2zS}KSCU8Xh#h9Y4^_sTUbw|%azmmtsn$1Nh;)u%x_ZtsiLAn_j-4KrKZIv&Bx zLl}emnruBwQp=(tolI)T)@;58BhUWfQ^qubBbosLTYd((TeaoMfIU4^e|$8tU7Haf z`$5=Kzbt~M3XxgeS{Z(#N<thtJv|~lJ$Iy>s)+v5N-Ff|aR1py7m1%+6%%c={$PzX z55ts!hHy6hRzd==BSj>{Q}roFs7LymXUv?RI^CVYf9mZjL9gJj=_Fsc63y%-q0tn} zh}>p-D-s;UAKV-C@DCn41c{6@-0xpAXe7uXAJ*TkdJtF7tyEu+Xi>l6h(11Zb2+Ue zU)x{y)v(ItS~>p!gIS#z75M_aQ|^<SuJUfbP>t-(zM_SfifDHL1aCAJr<nR{0A@$U z$lB|k$Dmw}u5EF7HvRV`45Ar{(6Rbt_s;vKm5GtZ9j*uUp1ts4UT!gQ5y2gdjS&yV z#7DEM+K&fBnM<$03D4X((|D)A`1f04E(GgD>~bjrAj3SnH(Pe3JeFP5dXR6E&bN(| zwVE7Jcc46raTYuYLEN^yo0SYs9j)^!T+90ORdgtwqw_LFD8-}*HOp>lYbPKBv`!n7 zd#~Ot#~K(d&smfd1qp8V*lE+OH_C+`B|l38#I(1zvsC5O7l1eKczt+W8p-J!jz};R z#=mbZ7~+E~Ayf>1*3E6_vFY3Nr`(~v?(|wS*PWuxg-;E@i@>!x+w$s(TA@baXy~4o zXjhvKv5qFaSk7Q&XvymXY$T~VJ4ymOw!I<6zz7qqXlkC}TvH5u$r$vOUc((hji(b6 zX7cM{<sT{gEwYHEk!9%@IRJcwxbK&IN&k6v7cSke^x$F6a}jcm&_Jo=rh3UH<`PcE z+)Z!NdQ5F_rnbi2g>6NzHO_a2>Xc})n`n=A5*|<eGm3=195t?NiU$yrpwU65qNToe z;WQOOPv~kLTqr4Zua5<41~Y4+YIK=z%T(YBIa`TStg^NcjY2V3pz)8fDKl7JZl_#L zojR0*-8Kq}Ix0TTZ!^ppW;}%RMc?&f#7eY#Szy!KU?_|-+#3VgSH1L4+;YOFrtATU zV4{Yw<`N;Q!O*+jwGMkyRw^}B)hA^;r2RT~b2RiaX4jsGiDqND4*o$idpDI3!8fo! z2j%iv#q_`Ramx<dvWbkJ-ke|5Lj(h29Cb1p{RR1bgp$jrwcgINm;tP&Pkl>nXa*^J z?g)OyCEbNM9+J%Y?=iH^FauUzl)N{J-X9N1E&EhyldoSzH|kSK_%ytQD6|3N&!Zg| zB+mAO@Y_jm8_oj`Y9Kv{0Oa%a=^yj5x8H9`Sf1BlPzJ5#ah6{YT3U6T;=dyVP-yFR zpKc+~B>VvRkpNUlf$o;W+eh(K1oeZOnAEs1;ylLF)w=-%Gw3ZuX6wen=4gooqXgZa zyyuqrC_1TdU0sCYQ}*nBd<1y1$VF!g?ylz?Q?%W=|AQcH!~$egsiwt^e`#|~c+;)A zE2`e`QzjvMz3B4h&l6)f#J!Ar2z*Ok(}RN6dCZ|&&M4PoNa#HE6#rhTp$hBne759b zFYUr9s<!uo-S>LjGe|F|TeHWW{YF#d6ZRnQWdi0+2Wk(J50HpE%rk0uZ0;*4V1xM! zmX5UYa|CooKhR7@-(5J*1htGQ!=Wwts5V+91jOMzlIH3Po7{4X9ev9<bPL;E+ayL> z^*&}150Wis2{jPn_mmsy0M_GTscu?Hw%PisD)dBP{3)x2ofMLt)T87UakYziJoQE? z)3S~)i4UvB+>SYKqRX`{RaBi~mFLH~uHsDg0?qy&^O&oepl!=Av&i)xkVj(~^|Nb? znggp3e37Y);ER<kOcMPh>QnTr<nSjX!90dijCw}5l{iyt@^fQx9WPCkTm1vIlC>J4 z*F_9Y@ksaC+2Fo{k2wXR2$js&3-x*}6bMp+CMF1MDYOI>V!k!0Ec3XZaT$=9nu^qb z4A}smv23NHe6T42xAh~9{JSNfrrX<v`@2?gl$Zf*mo6vsS$!~y`eK=BTdAoAFAmgq zKb$z5T{_8a$A*RoQP7sY1ciS}dYDmZNb>A*JM#!&^h2_;qK%=@@acN)dNh7qN>^4= z3I569o2ME01|d(>qC0et<yCFOy?(u;@dlQ|#FdVlCj<V?kjBuVWarUuWuri&yt+D6 zfjEI(vX|UEPRyzT0`JFviaGvBG5TP=ry*qha*><{1h@36wmWSAMtZWx3^>CH8+mA& z>AuY7!g^aJE(@e2pA_^;&Dyrm32D~Q0YPI^zD<?L1aoTlDDEw0BtO(8l$`a9B=Ovp z#p;fkit~n9%r{hLcQi?!EH#M?;4Z2H*&nfpi&fNp(@=Q~ftAD)>{{O=E({q+D%5$N zwHpW1F{Kvh<%&W5<&u3d&xpwqJOh5`+f{jkH+^(qN^k)^x`vJFc~+MiU|&Xr#MBVc zBo0CiT}9a0;Ln7|!RMqqn9N`SO5ZI^N5v!r>)d1RvdD====cX_zATMdA`>WC4*5n_ z(ifo!_R7_UALJnu*NK`pA39gwV75Ddz>`Vxc35JbnM`k{_$uSGFwV<g5*J^f9Lv#h zbTnJJ$x*n}?J9hA(;v2s;_(7+SJS4yYLlftk=;OFF05pIV_4uNI!rlC%w~ZuRJK-F zHa^ci?mhO}Mz83sn+q1KC{bgCGS9q0x1g;Q=Kf$dLprZc-FYjVkta*%<M&$;Oam@> zl9+q(shKB_I0=CU<yl6;WB0sj=wYHZC6XNUx5u?CuQn`ohj`Cgp5ZYLnZmhp1&-pj zI-7!8!EVa|oIFN;Q%}?QBSk>s5U_*T0R?1K##u<T;G;q+D=uKkzTXTQ<7{k6?Tnxf zOI<U|uCA}F_QAkQU)#EtYNKD9)~4AI`OOX~!bSo-P4N8sxKhr55EQ=!RXtDROLyro zT)U?g8VC*OE?!rW%<Vj{#G)PEI>p~=g;)`FNH5^Lu}&)3MNK{UR^dqHUfiuG`t2Hc z_>Ei3W0epZgjvcAvjW`{H|k&nI2A+HiW_eXHc&?3CM9gCgQXKvdK366XLPjKL_m6q z!i6#)2@4u$I(A)Zr)NW1GS#5}s9Dk`&>0exJCw1Z8k;ajfAh^g<e7cX|2LjrGG@_p zVsh5w8+kWaQR8L*PWRdb9g;t_z5C^{l{PwtE-*O8<+5CG@Vr*gp^xTNhjkas=vj|j za7qoK=wllUSIEj@es@6ME2a|2ItXlsn~m#lzJ1hF(%aN49JSECb6S~RAgqn)UzfzX zuq>V7po`%DIp?zOGW)w!eMj2oHXy&AaO)Aw_thW}b)p)85!u0h51sr9NEdv`vZQ%x z_?sL2Cwo*a<bAEuZQ(1%zhB3{ww@t<Kd~j{ipbw;yZ%*sl@$*~FS;Ub^_edA?-2iU z7N6yN0oVMGE{}gL`&-ETeJ!;RZ$YX?X)V%sV0_dm&wqAk*4w}CvOOFH3A6Sa+DiRx zs*e)c`;{m;4ZC#y?c2XkYw;e7uT!;oM}tWp$QUhN!(87&nDRj0YKeDs@!n`g*Xw_o z``MQ-)qDy|R|9@=17t^ZEg{Gv^A=-vymV+E-1y3dx)9_CW0pLfh|slTxn0YEdkco; z2DX6^KD*~uCOnGn*(0HII6x#RQa=8I@{)FIqmvN@4cKZm9AjoYXUN%wQM;-nzd>^M zgL!1dDXrpI`Rk_ZR<c1^>_I$zjd@O{k|$jJ82o%=pdPW8a@fdwX!ssuqF5X9UOBl9 zrBEa^Dn$vp0b@cx>VvDZVr$#s(bnGXCqy0_lyBc8Qu_58VE$II@~g@M8+wIJ$r)SI zLZU71dyV_Ch9>ZeGA+N}a(vo7;fH13n#ij}B2NYO4EMGiw1&|I2fWm^81I6{yewGT zY*Mb<6xO1^7#i;DyBW&>HZ%{^*OxN!s0GIo0tQI@XVp1f%N;gkwLn$Ij1NQ66_n^t zb?OKLqfRzO(ACEF6(=>zO|Q;;!uCi%Urn8S>rDAN&`+7?wc~;un8d%DoM=T5rwP8j z?S<=VE)TBhS-5@D_8m)rM-^4I6v*7s!kBJtH6;5lQsa-yn<xT%FM!ys=LBfO*Le`C zHYPs`gbM?NOWjM@-1rd?A=x*wS=~3N_$OA&cZp2vp>5k~<2#Hg9E*S|^MPZP9fY$g z!JgJfx}#Tjf+eUsUt)IDYvX^OKRvLuoKX&4-Vg;+;?izOpb{ipkhU#ia%&KiKevGe zys1X$yL8ZRq4pF5*_v>2QS$544ZD7qh~|QWDwL(zfY@@0CuEZC3#Hea%Op{6lq~zg z7&18q9HBUu#}lW2X2;F9@<PHY8Dnweix0TyB!t#$kB$VkcJ8*n{)R^2_*(lBFLf0- zo*lhXHv*cX993`s>4aopRQTW2{U>IA=bAaP4#;`tD>p4!QhwR-x_uPg5Af;<7DMet zmJqyWw7`0^X$)p<S{$3(r^rd3)Uot56Avz|(1^ubCkz&Kc{RQsx0M_}mY7i_prQM< zEDecU7a>EWOi2qCY|o2viQf|}^@2TTEA-FcWLL&7Y0~YE?^dAM*!fw}Suyqe`iZtR zmliv1u7p@<=LxZQF{2OMdu=X)NX}B(8s*n0+NWn+`aX?_?$&zNz{nVngB>=C`S|n# z#;j$X`!fy`)wu{BTn6`|<Dv2nQ}(g{6G-xRrZ@7O2NiMZJDxO)cDX+SC8<D-Xn5^X zw3)B0{~+A<6=sh>{VVg63C*P1(bp5ChZgt2y>B%*B`MmnhHW2AUB52be?~YPuGGUE z!Z738{(!qt>^mV0$}tEi7^7<r!=;P=1cbU!FLf*N1HeaP+Qf=eULa70MXs*tU%xG3 z{%XnaOd=zsP#jx=AEzvFqIU%E(d~_#iyLR@1^Rr#nDPQSi0@;f$g4`**So85V(*#z zn>P991$J0cy#teAXTFndKbx9kfKlvtISOK6)*o8o=2EE1itZ10Mvh=K?U%VI0yuxb zrW~>8@TQk2tI%|6mbyV~IV=^$gUDYxV2+7i9tU3aT17Jyi(hzn`XD-|*Ki7@oZLJm zr$yCyJ3n;8cX{r}d;}?xi5;#Sr$mbyo+qZm_Dn+NFCBk0+Vaf<a2c(p?*VUv5YraJ z+;Y+k;(&IUyVBFPy7Z%^#zz)pA<DeKwrGiFuTu7}k+6TO0cMzY#RtCb5LAYT@oKL4 zu>oXqdR(zJhEl9DVBy6mdxkQB_GIbj+8t+iyV+Ul<MwUyt@AT*YnBi{uS?XbIVxMj z8KWy$as{_qV2s7;CS6TXy#ex>M=1(A4|S*V%aXpFlSE-?(ex~{)&M#BN_{R)TPybs zK{Z!Jd5{bXOKID;o{zozvLq9$(p6dMQO{3uX<Clp)|=w9V!3gFO2&)T;r;`bqnD5! zUqv9QO3CW~!ae`tDH}H!6+MIcz1h`UWfEo|V<-Md*+d(*O62&yI;ewrG|t^P6U>&w zOR7uSO;G5xqNSd6&prP^1p9k%1e0l|kna8iU_ItGw{}N47nLFATtz}65xT(1x7o2^ zdZ*;qU6K~g9F)Dt1R!SLo_YLoy*MI}=4J&KQOymTLe$D79(A?Su9X&!jY7I_Q4Qr7 zi*os0#fd~|GAO!xDduu4-N0jOQ*<Jh%W7I>S^`8BZZ|>qi;urF+`q2kFBeo_{az7R zv)!zo5afW6Cg`5bRbf@d+81^dJZ6YW9%p`uzfE2aey;%@OycIez22<zNC$&BoFu0w zP#YUkm*1k877tW;d<7%lf1Oo+sBihj^=n<7<SKT!)8&hVvkYTy?8%{^0oj};+Rq+z zRp+AsLNEmM4;L5{&D>Q8Ln%vzDFY`P_@oGLROLcdl=c=@h0O|4D&n#VgDrJ9JDmT` zba{uv$_v;CxqjQt!hWL4LWXw8kKgRJ2(q_V3|EKG>|UHA@OuxAMwCq8CWJz+cF!yE ziyqZ*SvG)I`TEkXE96F1j_@bespdNPr&=zJv5bC_5u|hj_#2(3@(il@pjolSAl92K zOPhjbB`c809X3W@?D+8`kFDAppiaNp4U&ahcRXn{cAdE=o`y;1SK1hcmR1nXy-blc z3!=2Ep~^Fwc*qiMd}7hp*K1@2Nz2H5Bj?UlZQqUq)P-{++Q79v@8c5^(HQev_c2*2 z<xT3^=nGxUi020`oXmu<?T@55Vo{qAn3Kb&x{kN=BbNV7U%%UWE2qE8-YYndsS#|e znOP+@^aQRmhif2v^9n%a)Sd2AagoSV3*|rrYKK;lHC9V4)wG|rTPgYVj{32%;c1a= z;&+Wa*)rBTLFM1H!CA&wKb_D#OgViYFUG^j>btz`n*nFPw^Id=81y@$rnx6!g}st3 zY`W}draTeede~^oAJ>066vwFyBb(>FRD5|Xf+fLshiv}z-1c6gsuhuLH(Jyv<<NCK ziMLeeCqpLbxyi*c*!}eL%|c5%3Y3)YxBz$W!~b!?YB|45EVZqGb*p&|1%)tiis6mp z9qL&p>~_;Dv1AD2#M{~t<%sWu+yuueM;9qa&6CbQzS|`c`z@^I-3F~IVpf)bf$tnH zH>F>fmqrK`mdL~>n|w%o3{y6=@oZ28*#s5IE?Btj4pOg$ACveGcmK!nuws3Gcwk-6 zIdqiWVM*CRaAE1H@1pu&xM;-=_-P5e-km|?zl-sI^nNaVx2C0zMi>7NvGU7!r+VKp zb@pUK@Bd}Y|CagB75#>$|DpIdg#5{k`u~QTAfs9=+Q4^J(Q!hlvmdsZ)4gd{AY|0V zdjF|;#Xaw2!s(tH%EX#a%#rZ+*#^GNBj{6CJ2$0yB|XN_`Bs~CwBt_EjY)^M==F(C z+&GEoLN)%P;hf;33UFBgHM<|TTZ}HZftJik^}quJf%;tX@}_cd!H#>5HI%TBy<hev zOvKr8<TObgd*heCvn1c}dtH5l;VNBO!n?ax2qLTucK_5RJEwhM)`L(176NyEa*DtU zJ=E2X97G~|e25Rk>xFt<n)86qzZCi9kzbG%wzZlwZB!GE`{Tvz1;|mzl1iIdi8#iI zUym??ed3MA89uOI5E6+QO>LUB&53VWTiYWSaKFt`72eUv%70OxS$R+zbUYJsWj;s0 zOt^_>aN_ss!V#U?b8!bg@Q`83TAuhNcn7e7@v($w9d#xQxPex^QOH1%GkSAxm7B{0 zY>=&svl7hixk`f($}0&78zU>)rQ#O|<$ITAv)H7v?VnTe;uyqb?vHYqlq5E#%+1V- zZyGOAo*>%)#V0qR_@xlyIC~x5(~<Sze1>^6160wv%MOKnl^#@_0sEy-CwB_VRzaL+ z*JVxPwPdbMo<Up6MW<yKg~vFK9uCs$mQY<an0F?fyT{#}6;z`!JFxckY2?v5&GF(~ z&Ef-#Q=w-l4S66;XszGGkhmu*{iqaP-52Azi1D6Lt`aOvXB<YIAK<tqKHfi$qMwg- z3a5YjJsxF(^6NoYK)m0toN0l2hgS!i<e@oq6&8Uau9>y4b{GbS3Zn<jq&_Gm-HZSI z*6v1`h_$_Sd8X^_If4zY4H*{+Nn`m83{Tsc<ca=V-8@Gd@#kU`J#z}!=CMAYu*}b$ zDd83->z$_rtQ0m!;98GQ`AH}x#y#B%5V1K(RuSn8_FY%^lv&uM+*k9$>IPredCspu zK}M%pmxOSV{%R-j5kbF4FGMSF-bpTtPZI>b40l7mTqlOg_4-uPGU@i7-?xWSmW_b0 zM??M|8QGH!=FGbd<Ji($eH_tQH0s{2mdR^T1eC*7d06e$5cJXl=5ty{mJ3D>cyPJS zcwl{n&JF9*LhLSGZ@Wz62VBAajXZv@Ui0v{9c=t3W{zv7pB)E*&BPHxe`SX(ieEMT z;+A8-*n|)48{K=wR8^j{pRAir`c5Z6(MR0o*1pc*Y>6=hPL5Q7xBBC0O)iqe0}R4C ztqC1-_|4D0wub3;+*`w?Az=mFyFK@q+FV0*326*XkIC8Pw7zFR;1)ZI37^lW=+&Aa zffUBJ_oT?|xi*b<up5MB9~3Xq_G0uLi|MOG5<>yxO4vy*@s|iR6|F{QMNS^Y_>z4a zv|*m%?v_0=!U>fyK%yl1_$!{5vtg-2*v8YIZl}K$ITvoE^#<jYHvcBYk?wvTUDF^A zUv^w8n~M#TmMb8rR`C3OuVP~o_#6<=9EZBu(>7kwaeAM-hpw}f#k2G<1$@xIm1hxX z>3PKWJ8w<^y)yzU5Mj_Sp0!~DY2<bH<DP43w`AygbJ~|x9}KCoI9{rqSNqxN39ruw z<xb5B+ZK#-X8w@d>eODH4;B(@`9wWSW8r3HB{5fJJg#LPwSb5i(l4w|g|@z;axCQo zfyQQY%_S-S*uh`}%kpZ?S@K4Wrj0w}*Y@E@t3N}I4{r0d*xSo0!UbP}W@B6gF8xa{ zD>241$gW57w~98WPt-O9uPhj98>#{=HWvUt`GQC^V$c4T+d}z3LdmMcX_@{3_Q>yE zSxD$2<rBMwa*qO!PR9$F{5~FtyGH?qZx@(-BVM3yBuBNi@aJy#;6ka9GB0P$Du|1V zjw*G}AkAWnu=+U(qXmqgc>kma-kYW3S(e33YFgC4LTN?>S@SQo)VX#06%R?BsH+~F z-Rte4tIEbTi9J8poFI}o6$#&_NHi3*8xy};z(a7}-wJq>0IS}7t_kwGsdQg0Wgpt@ zJoJoh>b>p8JUPUe_dt&PniT@(=n)8tSp*e|#P8StGq1fR094!S+y0I9M(qO+aszh> z-S(^U4N&C)SDOkKlHh4_Fv4xd<QVqjH2`YP^`P;Ql8MdM7*<RDnRg*M`_`c&>H+lH zmLB2-RKe`h#l3bL`>KucC5XH|UeciSx|hWLCNZ&w{q?F^kY0?geIO+1ThK)b8Uby0 z^!VCD#Q3U>sKDgmeRt9_z?;im1_3ei{+RvbK1}gq=>b=e<|;LR`&az;Ph92$@vHU` zl-~~Hm-!7xt8<<JyJ^cDIQs1k9#76qg;{<VSjg)owl=czNtBC206dC&qQ0Kfvj(To zp+wTX&M{wUePJs648<fFq)>%^3n}d3giA8h1|5WiXnu53vC4YK$L^BZ(H>>(l~<<Q z+Bc)s?c)AU)XoFL%XtZ0)eV72Vo346(V^FVkgcxfW99I9j~#b|J0Tl*EQsL>zy5Z` z@zV)6lwBOoR_wZd>#@tAT};cHTP}~t+*+;kiNwn@uJtRXF0eHLtnmhe7x^}(Q%5gK zPo@Q#c~VGMm@~1_YCyJU8OCMBhZ^VlR4yp%YmgEf?l4?V)-A8s7!dvB3|vcAT-M(d z&FA2EI~J~Mfd3b^X0)@a=0K3`Jh>$V6zpOB^8zml@u!L;H4+lSLXnA2$=zOD*L?65 z?F==g4%3t}s~(My3LWqY?y5=!ZT&#tXb`zffIvtpo%%)OPB)3J_K_#j-Oeze5r_Sb zR+=_L68AdgkQ(%P2U$*)Jy7N9RUG3@%av{9eESm|?QY<6dF|1CpHF*W7Y~f!!N^X- zYV9lAIGYZ0h)-)qe&DM;<fGSwGlVJ~adNB$7;Co-yJi@6PnrnXALIFtVtq2CY|<^5 z;l|_mB<@LQ4}GCNPv}~l_qRCW=d@oC8xKbR0Wr<j+mAe|Px3VV5|w*72o?ROqAAaZ z?z0o+tNg>ep2Na7cUSjzCy<Vuk!#K+9P<z}wFxmbY$c7GgeM7|QZS#tcU<pj$4F`u z;hm=k^Q!pvKsxE<J#suE-?p&T3>~YkdxYOhB&2jF2)Zo1*9zy4%Vl06dQDVmx1M#d z_8SuxXhViMdy-Db6HfieJf9VDFucXuL|*}2Fy!PpcyPfW%IgL!?Qf-2!HcbKbFmdA zRW*vaV?eSTclgM2(jVj)VgTMG!=k+HRb&Q(6P5-A^SL0Y%dy<+(uSRW^EBIqy!n4> zfA0tUiv}8YKtl4bh(%X0U+}fQeZr^d0d7W>I-FN@14<Vothj(K%B0fRX>@qI%152k zIuo)Mev8<PZZjx#(7p+G((~%5@gzcYY&W<OFAND?7sCubOy35$zSrl6?1ybkB7hbr zCc8;?d_;(hXZ)CW-ubhe9_|c6L|NIr`gwe=_I2~kGr0e8a7j9&bLNc`TepwN+46YW z`9Q)WIB~Au#OGY?$2RTnFsS^3L2>~2U)<dg4>D<fyHPnHLcDxSTk`xUn43AoKNrt< z4}KkmjC2SL=os|7Sg7Pe9hXibQEk(|xFO@VVQhxN1$iV8oi!d5FO}6+;helA#C#PS zkBri(5`Ss+1y!F@i%@XUZs)+8h7`%pS%*r_VM3PC<p0{kR|;QAu|>bOSPRlA6B+;5 z?NM=A%xw4E!u7G7stbg~js~IYX*Y)m<LTU@&~y^+jkcc*Hxcrv(NVC#K)hHF!{Rrm z6H745QXh7R;a|6s#TN^mrWRTlsvqSk>st_ki-@uWDF|7N)m22@b6DA?_^<y3fQqwt z`Gd><6kB!7IO*|met<F#qhuX-e_iHLbYh+uA^J(@!MtuK#Uiz?RKY!-09buCx1CZ- z*OU}mRDHRyUGG%;cH?{4CaKRxb~G|ULeK=Nkw;`a)AjL=%k)}DG&Kdr=^UnR6$A|* zHFMBiiR*_+(O&$7fm)P=PT7b%d@jTnK~9OwOH^X3MV<6O?BC&%%Nk`ig<~h^|NhIG z#`uSm-jK@>xSRCLB_XEY$CEr+#4JNrXpKwvD|cLcuaNdi78H*35r<*X6z`4jt6S>4 z_3%8Xvy8xH6rsr(R3meU`VzM@+cdV3amm+~p|x~@>!9)e#SZ;9ai5Cz{%H5dpfkw3 z#X$9*Fla+PO#3exE1x-n4!e(~o3vM0|3?2m9Q;2i*_lu8SutGt6{$bzlfO2zn15x( zuud7vqyK7f{%I{f++SHS;O^Dm(DW}tem8E^{C=hA{&$!^;r^dZH9v1b^9E5ZqF+YN zF2O{lym3QQQ?_Qa<BgiMiwir<2fD4P-WAsyW3t}_#vj-|2Y{tWMKzR^;2dBY%R1*l zq0{Yl5@#v`KGzBMM`+mq9QeQLC}Zp}{gE|JAe|U&u738|^#>$le(hX;{ERAjdvO3# z4xb+7bRgg%t&0!yIhZe%#d13c6P~_?Ijc0=v)$gl_L7paTxsae3A{|Rd5+*{gW&h8 zp7;z>&n4>q(*Vm;^brZ2ZfD-5_aL9?M&GSnupd67fvJYo72K)3nit_0+$L5aR;PGu zZXBg+z<RZwsFGuY>_N1zKJUhGJ+|w6-Si86UEAX6Ei7eHN!v{0p_}r{dE#$xKHiBc zu!H6Ua&gJ#%Q|i^>L5;CO$U&deN2fiDijM=_|j=r6(fzeJAYLu6R}E(@1QL*HIMyg zq;812>8engy{q(Jo)Q4{D`xV|C_!C2gu~c4^a619ef@lX_tgsg^Palfagb1=oAKs4 zsZ)354OQi-4S&2o>k~h=xjR|U>-?ME_|Bp6T_3*o8?L*Vs^Cq=^9mQ(5-0mCC*a<d zN@>0N(5)@E=Gt}DIqY}lNeStofkrMSC?6tF8lCOM1bti&S(_h(RWc-{J`rI*JCDY% z*|@w-f2s-l86E9M6Z9bc2V;s8<l=XMzUcT{!y(ADTb<v*1Dzg1;7AwQJ{vij=sQ63 z*`Vtn^y%DiPH(1NFdMbphm?E*X)n@;V`YMTVhQ&iB50E}4H@$g(UELev4|sl>K9ya z*_KQXmDew1NS4(gL;B-}{uD$oVF8iCB363KZZN%)romnyR}}_*3^L6lqQ=^*va1=T z_2fy=Txx!KzeXkPBB9<*q7bzT)mi!st_!U>s<Q8W2=90<MjM#-nr$y(0Ry0VyCw*i zS%NL|SeDrB3pRDDa_VJtXnZLcai4KSu1`tz*7Cc5WPWp#@dTOf<{X-$+V+CqcM+iz zx?T&}!N)JUvb&!8Sn_^U#J?St6hsP*xDEla_;}M+pD3@yUl|qRW}=B28bul5ol^AM z&3yC~6IE`OZ*R0{_Dg)nVNsMfw;_LVRxQj-3D7$BICVyVZf832m5SOVc#lUF^4aOg z+!#XUGj&E14g@|rcd!a+|D8nPH>bP9e#LzF?pObt9JixANa>!(`m>IhH;&SD>*E=O zVWo&KQT3jlV;AKYnQyI-GG*>an`4QUr!sW9L5&IlBHPs}3}mneN7|jm>XuvPpci?Q zU^9Y;W$V5qekve6dHI!xbog6b92=93GU8dz1OjkDQ3|Sei)I!^ALmcHiGJh26hFQf zJk>QIs@1nuB$}jU{zEw4&kh3NK$R&1;2y=0ezTsn<*pklC(8Ur%n2|Y#cI7~@GO6b zl!}iwr~W=pUvz}F>0iwG$USHC%=-56m0zKWNch55rrDBhfemTX>FsVqow<bi8jqar z>knRt`7|P~w|EIDwiv^fF7hYltx_bCL)a44OU@{$gkN`tzjm6?<S+Ch@Z%*$+aHT; zi<(u-tXJOiZuvCG_cA8n_esOoMtrsvz|h#9ZUn$0s>`P62+kt3Ssg6^U*h_H=)Zlh zw62v{u$8oI?3kBdmfahM>o2>K#QPI%{?YXLf!rW__|>mOa282vv7@M`r@#%u-0;&( zn=~$WF<9uThNfAml!kii&(Oq6n@zp8Ic-A{>qoOhmY~dXnNt1F0|7ROzc4`~$_H-p z{@Qs{=ELJ`qgmuYW5cp~OpuIK-*K<@{tI#HT`c<x{ZAk0`}snv7+j4AuhK`N*D7V+ z=l*!F?o~W*80ORu5<r;h6*{yPp51<YN;nx_vVE7SKW_LJO`p$7l(SJupppY*Uazbg z><@zlkM6Qe_Whu+IFRqZ&I;oN>iRI6y~;kHxOvdFv4nSon;~uebW(XbAR(BR^J@U@ zgWabNZ4SN;o!8!i>u-BK;1d6<w)O^*BAD3c*Sn$maj^@#{%-oz(J@q-UeP|^Mn73M zB~)lN>7fV0pOzyPGR!U<@;jQ9onB^f-c;cHK?$W=S^H$}*Msdq##ieha={CL4h+fT z;!6SGH$JD6RT5|FxpK^6%cD(gmz(3@txW39nlq7&<Uy~lR0K!A{Q7DwIRQrB_qTjk zs^`=FRcY3XnhEjm54vX~p7m6Yh_%;zxiRnb^>nPCg?-2H5n8c}2t6KIIjQq6TQQ*x z(%1@1`zF{yn2f^&mB}_{Zs{)MnbdMpo5{GiTQ~N}L-lj`s8H#X#iKJbgSvcEdIQub zpy^V7N%2rble1$ObY<2W^SuSax#2H!GFdGv{@<wr$SK<7+CLWEm@&0{SX!G$wFIGl zD^)B7<y}hvnU_Z2IJ~WHs5{e$)zZ#^44DPpRroqTK99|h+*12F9=AkXbOv=qaj)=) z8G^F)VH#vkpP(V0dJ2~MXmosfG53rWUpxJ9w)VXibnGD3R7HfLjRjrgiRp@xE+w$l zXPJN!88dzkj<pTt0Boni8w^eMI)8_`ExCmN5E>!AfPi7z=ir5OMZYplu0XM<hNUl} zA;Bvjb}qfdXJ745&=YgWw_qQopFV1}X}`HOkh6XuLm=r|03iC_kj*G!=axn80y)qG z>DN66hsgQeD<cSHu~h_JH+YX`?t8t>Po+0wiEWteD9(^=Ii1gt(eo&9x12J<yEox2 zOQWpXd@(!zVZy_ssbg=~lNXBqRZcFfSh-xD{QsfrEr8-$wzlCgxCGbW9^8VvySrPE z;O?%2TW|;gg1fs97CZ#^!QFz}PmX=(-v7Q;yQX?-YI=9CUe?cg_U;xMY!LT*2Slhv znf<iu<Lr%@8jRPVMowMy>7@^ht;bsQ8$))`h4b)m3ksepehR3P^P6`+)|q~MDOv(l z;Tej(CJ@G-x1cqpKQh0|cHn;Lm9J<#K7Giu{g|4`6P|640&yh@(TCyd*w92>`TXRO zq_1W=$esILrkLaFS~ff5cYz^me<kQ-6k_?GZdb@B%GM+;jm|R!FUz&EnRUiNAWGl^ z7dAD8hOVyj)pdXMkQ9_Bhfl!lXS8tFi^<~al?UU2;*3D25if=Qextd4C#cvgpd)|A zsbWf`;z>`qw{yl54@0F-QTH*q5}S~<p~x#s_63n~_rZTX6zgFubp_ww;O?_pCc|J= zull;i0W6a@;M9teWchrz3z_w%?65*2)={6((eo5LD~m8}-P_zW^6^de(U$O0NgXH3 zhZLa#C^dUb3Zk=23zlbO&5y)OVKbTN2OXH~Zl;`^?kEgGKG*fnFWm3E1!5HgcWA%* zajB@O75lor-9x^z$gl2x{5ZMOYFYWa94<tM^Qzu?&_JdOSnsA4_aKDpOqr;Ls@TNI zuP3h3=P^n2m|~eyf-pe)wo_nc7s-`qkTvy%gRTASSk5EcxaCjSxsui%%}UHqC@E#f zJ<hHboD7@gAE8G?kG5oO=bB2me~KRc%(6N$T>aiauWw5r%3GiMChWTjM}}#yrc+(b zQ`AKDzTuUjsqdWHr}6eu`m!~DJxz_-Dns90`!hfDI`gh^m@WroslXiu2hZJ9E}Aa- z4K@dk)yFvPG*-_HDBv#HPu&AS%mo0VFR_OV@3kRaDyiBNdia#6pVW@-VLu-4+upn) zY9WbY$p<tR*eznd{JQJzR)5?3JfhudSy9B#shWEJQmH+-<PauXIo+MavMffELP#2h zzHRz1eB54rHTGP+F}xmLZbDx^qSn2#J2)V&EJU6bjPiePRbB_S4nBybqeuJMZm$Aa zuCQY2T6MUz7~66FGEsS|bu?ECopIrvtIQVR&zEcQOY6H_ad(BXUua$<z0rei%KHK; z`t)u=itOf%;5`J&_j(!{g)_Vw&Xh!g5bQVAqz2c0&*ip=N=Dzl|FpFy%<ugidVBnk zkfj@2PJB&D>cK;HG<{jlpJ|o9QJ0SIv_0nS`qs06APUfcSWFyWYzW!8U>ImmeMS7e zAogtH(?NNLCum0U9_{5$@TD97k|@M~t?gXb880$x)6qkph|2x0vWmrFRqPfDrPn@q zTaQLIA&E)YOc5p8*8%Bza49CW0s4XMWMHUU0oc5&33so;gs*I%W;9pQF2S%Vihq;8 z_XT!C<rletpv_!zraYCy@gn>2gPd>I%&eXltWep-;Q}HG|BsPMS*jm#{=iE&?<3Bc zo62`>HJPQt<uY>Be@Qs#s>POQ`Hy|D9<<t96LpiI<7;fCn`V99E2xxv_F0n3998SR zna^v8DY%yT%{sN748q`F(Q@#sg)(Q2{>k^C>!h*(dt>8-E01N)HR0njy6P;(^nnKy z0^w5w{!*w22Ir-?C)q|ix*5a8Mv|J~d}1={IPrzS%=4tjMz>z~K{OVHxJorl^C8Gz ziH;{uB~Vp##>}`I`-e;@bB8FhcoX2PO`zcCu*)AsKa2vLy>XTuHMz(DWzPl1%%92l zX^_K{81uAhpOAi3ovs@L_jHb_l_)I|svkbYTHpw8W8&*~IP7)@?pgcBfA+iDRqg8Q zIdzUV%sZgRU*B0lgM^XH%gprC^RED<e|wVRN{<uSr`9Cd9BzMt*;Ke}-!+qmop^`p zc`BsGg&7xI#Mq6B8vE(ym&C>b`Mmdg44kSVf(ucIoRbcUfYoB@!|SMB4(N;mA-`wJ z+dJ2p1JPN|y)uAX>9|$T#4!EPJyad>(>IN%&7AhgPuJmzP3xgWjZ)bjG`9JE^R6e7 zcMl^z2RDZ6ciM(uWu@M?JE9FH)-^!-o+60*NhHoraA-iR`ku#m%-PN2&CJDLg_f7; zG@j{qG|+>y`V50-*pAOKJXDjtScgzpt>}1_UAEZTowjzczgeM`v1FF69HAIxSm|el z1Da$4{Hg4c`F0C_$Ae2ixv<ZrIm0p1>Sl`Dpw6xrKD(4&wwf8NE^9~`ah;Eb@S4Ge z60fZ4l9iz?O0}x5+M7l6kjAWOm(!D1Mk$8i@r0zd8wbJ^j@_N)*#-{vSCQ&Qjaa%O zh4sE=l~F$jTp5Lv-a(a|x!Lc^P6e1a?kBx1Lyl><HX{Ly+`fP%#U-yidScindl(`Q zEkIS{<8obi=b(0*9VI3Ae3Kd5_>jlu=;l-NCh<J^Rg|s#4u5t8Ly@5Pe8@2FPEpW$ zGFK-9%mZuL6&+3T=>FLIEN1b;RDTXw?R3qWyC?#hQ{xl5Ifv(m3Lrnlr?3<%h@fcV zShBv0^jGY>0N|OrrMa=)<kxl!`&}{KdSbR*AwOuCR;WB%_sNLdM<A6lz03784^JvR zgmD{poPW;8n|%wr`<NboenF&FKM2-tGf<_%Z93$6eHi+C8itwgv@<u}^_8A8H+HHp zz(XC_oBtxre-(OyddDu6^_q%QfXxrka^hiEV%jaLU9UR4UBkaqRnke>99yndkD8^o z-Jgd&AMKoydp6!BzAe9S3MB60Er15bbgrOLYBzp~?Y{fm0bF5Pa7PzYj|1|gmd?q# z9TNq?N{03+C4`H?qY{7aK3WisV1|~|&@QwVVv-|TAht`^20rW>WHtL>4R#6Ab&V5h z734@xpvn<pBb%BbBcCisfo$L>Pgdfbdy*#4ZL-*fj;5Q4sgOQL_aq|0Qw(kv(P<_$ zn-SlS6CqB3#X!~&U^lonPW0fddnGq)@8=f9dueW&7MNsZ+xR1k^->}dyVD3^$hN*B zQSQdU0|*n%VWQZ`BPQ!%(Ti1npx~HcPKi_9ZLT^(v5~zhw2v-KHE$~vj3vsD?EQuB zktJSK=dor<P+MuNLz^42t2y>u*jLu@8PS8EE%M4Hd5!E2#Q%x5K_EQx1xma{UmHzi z#L*KrFZ{UqiU`LN+N9T8=^pcb@j-Zg?mTlyh28y7WbP8EJewcXRm$z{k^8jGiu8_K z3lX)<m(bUYxVAx{aoI4U!(cpV@eB0c&OT)U?EqknNVqds2M<1<DQ_TcM2+j#cu$Bp zYd$@uHqlBE^lOi?fT(xIB_J3jOr9X-5v6IvMw`@6ia|)}K>Rp<>V;A4ZNe*Y)h@94 z2JMv#+@<Ij@koX&+vWYlzp=T9oZLx@Yfj@!dU@|^NiA&haAe{OhwFQmxfUH(s-f(0 zVZD;uQ3ciYAPU~B#isFQF(<eXj1jMeO8<lTqXlh6$8XF(6r)S?HK^T-%7MZEbnSbn zquqa_P?ng#1=QBcwJD>_qULM#&|USET)b|tasfXq#V}i&(&~CuCs=M<<|~Pge!kL$ zzJ72JJsEGXhySBaLhymm>ZH&TE0yseP?-j*+zP@SpB0<!B=NpmauYQvyUJ`;M%$z# zog9=6NeTBQJ-AkOVg4fYbF7fAhljh!T3hc{prV&N*yNH%!cfqf(6pgUg|)%TT3_^- znBtq4#|HAL8yB5+52dD)qXtoR(IR1PqYBX?vWxibi^Zw@oBYy=r&D{@^kJSPR}r_` zc+`~Yh+<3-b~RRc6|llmw{7`)?YZi>Vs`0T$9iVYtKzuAc8&37(d*FGmd*{$9;+OH z9g>#^NXU*y9g?9sluKM-O}e?uW=oROfP^O<xtSWxy5#M;a;DlCz2V#Sbc*(n?^SB1 ztidL;0Ou3uL@d@e)i^I{Fqk{FIa?->=!*cM`tWiD%czSY6-ItQoA|!*jFAwI#T$&{ z_L6B922cq79~UzVoSsU;yiM$km`RiO*C8zf(&3eBVi6m;dSGYh(5Ms!S~KRSPpwk# zA|;b)9DkM4;p7!I5>b!PI(+^0zEZmp4*Jell=0{6evRME{HF0+3s)5Vv|jc&a4BaK zcpH`FCq=~0XB<1#7!U)p-22>H(ObPmX=y$8cggI?TxPbmjWF$`q|~cG*}h}qE(JvW zTy@RfW4KWxge&D){96EgFoC=Rg>`se0%{c)`KE5N@MQXS<a7<K3mkwK4V0CMSHf}o zDV-MeY|i}Y?xX>{t|mX#Vb*gGKm2_yL*oCK^{X_Pr{MH-61W%@oJbbdEl=OZth;$v z-^932&i{-oOJNi%Wp4?Ww$3Ds!Wr9NK(W(OMR$}5F*cKt(-3^yNY(Cq_MVpVd@<5G zg^X1NhcwtU-Rwx#PQR#TH?6aAOS=wj!4>-f05~5l(^&7jtOclA_C$60LA&PN+;4R) zMv^<=s?<1*d6j>Dr=R`yy?+&^cN}}q+-d-=Bl&no_b_Kx1?0s;Sm_xsyfzX)D=Qh^ zd5Oig?NJU=20H}dT;V5ld-P0v!zsZWG`Lx9*<+LTBWMqyc5`p->`0dseqYUO%uD<z z#04$9-&q@e6%}P$sf>QQBU*H%S^oc)iNhWId#v+BZLd~F2_kMU6?{>6kpD_6WGADb zMpbKJoKQLK`-J(7yr=>DcGoutlIiG)wlQOiZfD}jB?4-OrHV66wcqPz7QG#8Y%Syr zL{@BlE%_fBcN~`ruLwiwuw~Pb6s<`(95E5hZz8{`+n-?6=y75kan&Dkxt%V@WCudI zp|O;t`Ch4fvlb1BWxI&rognDR4KwJO)$f3Pux;J7ubm0{CAU)7x{=-Fpd+^THQV3b zokXJ=4LTkDrA2=i*LG|l4M@7$Uj+kY5^Vf6)uFly?-R{^;W6ell!#(+P-Sb=Ss2As ze!?60`2XXQ;{&ArOyaUP$96mQlr=$birjTzd+vOYm*%sW>Xm3Gj!Y^Vw^Xleu^Nt7 zSKo2P3K9t|Q4%>1pEHQ618bMd#qg(SbXed&dmA?2LPVV{mi2!}-AMC)3Xd<7lId-R zB9MgC^Ekfo?>YYv1tPQ)PGUnX?3oL&FyxC#_C%+<jmFt-GY~!&W=o_5h$YE~W-GCA zkeW9SA^8p<XGh{Z7~W&VYEym)i8i!wE*XDg4N)n@OalH+DJ9sh3pzk<Z@i0ppWK26 z>T4s7k7l%+6&X!o9<P9~H`+~iKa*SfM3L!JT;~3Z{_d#K)`RZE{MWtzVV`AHiEPcN zmj*^r-05<d47_TBje|7bR=1u=sZ$b^RKV|xkrTp|(D=i<jVrJ1qC+Re@Xe(El3}+Q zxq~!+ggg~_r1^ft4}<0F#Rmk?>}Lu!^*L+iljdWpVq&w#0V>eo&D(;ngmCTebG0HS z{5?19bt2xW#_-P;QpO+x>tXf<*{NYU?e4i!ETDH&Vh=SftZE`QF7ql2r7?h;Y}3qa z201Z<vynehD*I~#EUN0p0<F-D*#qZ44n*S5?FTtKTs#p($vZ+Qt_rnqR}<%?M|#j$ zr#=S*9QAaF1n#Eja@XLc1|<b+0MO@=QN;3zYKl0(z61exmXl_;6b_T#pzSm!_|wna z;>()Omv|=1A9&v*dQVN-NQ#lbI63%K1d6E`{F;h0GA<O5tmvRZl_cetXF6;m=)g0S zHhPD;cX9&b&xjKI>FgYg&J-W8KoG7C-IAuyg?QJ(*5KYy^<B~)Mr=UaNLo(9$0?Cw zIYwfBkGG?k2uYa*_`#rZiplD`jo5#hszW+lNQC8-FU_C^OHj_!X4E1o=js_LgYt^f zW;Ty6ObogPI1I&buIT$n)zJ?zljL=~_lJ8f;DS`cY2>x%^SE9DPJxI+n4CD2>})p= z3a4NiMl_ZTnZw|G^&dUyiN?sFJ9WSAY~&K^>@+!SkXWpN$gxX(H@S%0X)~n7{B~pC zzDvX{o%H@_1O!@(eTNaDlm~pBHcn3)kM7qzWe_FOYM~;Ez8k!L?f@{6g+4fTsz|_H zYTXx(UhiJGc<pl7>Oj#}&W1ov0;BgW`GfwAEQeIn5@lBvDT@geUBC#}X*F}?$5gbJ zW+eVH2V{j*LgFbcc&BM@I=1{GJ4uToz09Hf4-jRQB{RK-&^JtbE2zQs(U^NpIy2kq zXLY+0zE_?eO#4`xFX>^V0(AJ;o<YT}t?-*XE-xhxFSTgR@oO<79MZ^gzXdy;CFEsZ z$(5$A;EXVBdWYjwN&ajxHZL5Z9X-gp+r4$_>%@qzUtCQ*fYDgXf^Q)@&Y*>6*VmEk zJy1Te8jIj)*u$=cn+#WGY2#*(c+J@!Irz?1a(dM(e~nf_JDe#he|P7?cAoeCWZ+wP z<-`$glHY0-hJ2?_mrOo)C1|mH)J1LX=O<%Qf%=03(l%A1Zmi^)`4($p?9rTx8Sm2} z>G>sI$89XcaGN6J;&_bj2p3L_bwcIl#zJz@z`l+(?gmz85zF3YA%3(^4{ECY`PFG? zzi`k#1#0_+)%6^R4P~H9@GQXLksIH#gyH85N*R@+DFGZqNCC)__&#LMu^=Qes=<K_ zaSMdtx;jpXJ*mZ7Wcc`QA%F}LG6Lvrql9$O(!46oOa>V@s!IJP?XM8@IHsR@iOE7o zqN<_{JS(mA)<9_y2`2(`3i15bPd@f8u4R^*8{i)QTts7EbP|X^uE1f%uO9qFT^9V< z5kwg9O?<h&*t}mJsJ{4knlRZu@uRHAnq9_|(Znss99x!i2Nk_s4K`!eZi1*-!s?=E zL9L*c3+_FOVcE0&QQF6&pGmhfp@#R@he%o}%}|Hry49LuU<Sz8*g_-9MusQm3sDo& zj}kg8)Rk}}qy9Z%Y_O8kcL9xe=g!qs+-R0)3C*6}8#MY3!{R35_E)ZR>xvUy<)JYi zhuJE64K9{FH{{kD;JtI!_QnT>w?3sIUzOkw)Rwz*<*rIZR6t^qru`QMqUxsHzXOOK z2OuFqxw@$#T8l&jL*2wlG91!emQNf!;<m$erbZTD&Ug*V$jE$v1C`3b55S0WB_<_) zJ<xBKA_$X8R{g5m{sIZ8IhdPUh)$We6odVNQOs(Z6`g<ziv^#zv@7APDjHWzK*i~( znz)H(rerX*f=<_hA-|jXfobc#HI)hfn586v3d2<OQt9MQj7Xk(uMX}E27=1k3aKY@ zhIdCwUtU5QAEYD8fweHF-i4ZysVZ-%X|6E&Un^50V>G_&AF}3u{LVA4c&_%sxyEB5 zQ>zY)=5zF1(E&SPX}uk+xY1=NJ1BcoImpSR0(?G~U2-(-<Qj%zQ02fLA8aVo+}N+O z>CVUKuVCZq8ooH0<_E5vm^POk*=ptGp?;DYZ2j4<g_jxQ9d#-)7MNqYWI4BaBZH#v zBx_>x`c~8t_Q2+WPGa}$TRH`)o5R|ZZc#jEmO-xyD)hL5_xe+AxV*ia)z0q<=icdf z681U2qgl(SY`dnl@ixPuozSr6N1?_nt48k&O@YhKKnDi<ma_cR`tV!x(k!9WhJq8k zP{YUd8{kni!4uFKkNGJ{Ik66sgtQY)H=k^Pb!1F^07+ZQ@usdEO=l?ctuZl|-8+b< zU;Dqv+`;8+3D_&Kk<Rk843>gz^`<xY(QVTK0sZCv`<8Z}@Ld{8^&&tuHRQfFRQ331 zMwC4PloDUQerN4Dagf@hDFU_&4C5$~_g2~++%Pe)=zxr=7b$9Nqxl?@vofIlZIi!c z#|V2_0gzS<>KFPCxyas~|Hxr4!Dx_Yj9{Wyq!rh0ppI`Xh@G<tVegB4pWw5!bbJ;` z>};Y(@PoCF0wgwh+(QCj`6kGVlwiJ~5F{=V*boZ0kXyFS@ppRDO+l^n`1Vl;1GwU0 zJ)(LGdYZ6AY~B7ccz(rH-V1)~zImS71kN<2Z|3+ee<XF+lODBsQ>AD(17*@A&llcx z!}Yvrh6Q8%mdO8_(MB&Vc0%BuoUiMEh@h+aWI+b)tgX69wb{y`FTAFoYU7>E$za%? ziw}1d)Y3S?K<hHEqoE8@`37x31`a!VqRCanu2~yYCeBI_x0CwP`!#kpa=;*O>}F5C zKfwq3KA8POYv~`FUn|G-V{($8IKM>Qab0hX2j(BMm(wlfn^R#Qb-i;ZygwCq((d1o z8Qb>AXGASqTq=DD2u-m!Ng_ZmavlrI@*cgK=f07d&)AyAFboneWOli2n_7HsEN6Hh zV;I38J4@Q{ihI+b+b_zEfEDN$RnfG*yFeMpE?bNk_jYOg2?_<8v5A=Wo|c0`%T>{~ z3-Te;pg-I6XoVo{JEcE9PJ&hinQUT{@5keY`MVq6jN-#9567A0sIV(kBD<>t<;kX1 zfsl;WOJ)f%DS}OPsZ3+SO8B_nDQEXDQX^*4DFQmhj7+iR<F+FQ>BLVd%-AbbkmDoy z=Y1JdLG@5|a5D^$_jrvfa3~q{5nviyp%5VGiv!yZJ$=BiA9BI=1=9&7k#~sd#|9dy z?qj0mE>bPK<}`e^-r*HB#7seGn#A&P6Ev3DyguiaBg+&}tpJ)Mh8RiM9AkvBZbR^O ziknuQ1qlu5JJ(}n^y=^7Fz{0a*3GtXQR5QKBO=T;Vwu2kw^S6E5v1b!S~p$xMZJ6F zB6}ECa6ZXgCq&Y7sWsq<8;!<Pd@xOnf+$P48f#7Zj|A~GN2H<-d?40?E{OhY?<oca zUgavBc-hO$E3_TB;rvE!-9MA2>Vxm+ugPfOzk!Lp(vF~A4AR6A431%XVyhgR!<cJ3 zPWIUM&O95#!Hg2&dlrl{I_}le0yrWbvBRFNG`t`kDspZbYV4NEG@mc5G+%=>^qe1V zYMy(DH<mb=0@+pA9rC?iko9iTCF{L>4WIGwHNU5w$naGkTzYiwt|R+gUK0YBJMlTq z#|JGm-ZU!O_E<z^p8Z&2oo&4n>em=;JAdQHaM>xdQxUUe*IQ$&7%^CXn_77bvwFMF z5CE5LrI3NO7Nb957jO)HUb@;M7F#9xZEH|OrnQr!33Bk-u1DVRPJBYH@L<4R=bbpk zc*?f;SFR@oz62eLB?UE?`d$KC@j^3rH|&M#do?<GvTKPjYBJT3Nzvo0Y6&xPHaCbb zX>*}wlHL;;ChA#NiXQarij88HZnM&AvzQ+xb~ZM^Op$XJEFKp)B!Yerv(hAJ@`VkI z*ypowx`!nKukpm%-@9ydDw6#U#l*s^BU+!*m;`yf#x!n|MP1ltzJ+sul#=gV`Yff6 zzPl!@&qmEqG#v<rM%iGfUt5-0a#nM*BB-Cllb;MdzNG$B`bO2o78|`T`#9}<e1JG> zs{EZ{|M^Z%w+e+wgO8zvCSq(`TRbrxpw5-tZ@aV)=U_m<pf+QwpgtZ*l<I`rahi36 zRNl1_oKai`TEQ_~UD$;7A<d_oFrMLfk6I=`gK7vTJWKiNt}mcK44R%hr8y9V(6od~ zJRZVda~NN~^1of(0m!s9_&o(mu8Gs9G(N9urFtA8rcsqCpaF=gyr7U1<JH^sU}}u+ zLc=rQ+^AMT3><2j*%)ZId^Xo^%|0u@$XXI|I=rxIIR29CDSG%5R!G4TFg*R-7&<HB ziMg6O=MWSIMTi&=Ywb2b9keRY<-2n102C^h>*v<QS<QwB)JncP+ErG^!VE3=x&*JN zq@SYJl3h!PCQzB`Z>94`xcQy#1Uiz%C+9(}Qnf$JRS6Lg+}}NL8qY4P--2#d#r(dY zi%mOj+LED0)mqux^zFI}euW2JXUGtgS&Dq4#KC{@MeKY}xcoeQYPwmgC)RrC;P8T` zwzN|Hk+BDdSU(Qa^Xn4BLq^Mz@hes%V(_@tXe~x^{q@m<(&j|!BZF?7zM>y3SN7^n zriE25%Mli=$g2hRxmW2^UK*5n3hJ^Yp?=Daiy=cjtVwx$>ua=HJ-0FMh(<x4)Ngv9 zc{?3UORe#Z>QQI3cIfMNI{U_=iHg}l^JDS+7+EASeMN}=m(Gl~dxgiVDfS+0)$6B_ zK5RG?v^e`QB|r^i*TT#DS;8lHjm}Gaf<!4SnfC_NlE2uNgA6+_xi|>D(aTWy!Vy}O z8bIScqAa<<A7>2b5*l+Q-laqX2YrHr2cjsZWz#e`c?-8rcKmMjBW@l%9Ph8xj|9rl zOFQPh1@^Sdv8<VN#~-0Gb!YfuhA@S_J6ySmGM~&oGq&Iej?gCXu&z2dr^;IRo+#K< z+;U(Oc_3x-60&w;9RfD(BQ>9g6pl&~+E5@ahaTpfWIhBo=j}7q#!}bppKab2=r>^1 zaM==`{DcD2F8i#9&y$Q|f8l%6R<D~<%Qrvv&5M{*5loV-X3c8%ax0JS?2$uJR~C+d z<M;}MU6`AHS{;BtKZk6ON}hAD*3Hy-iG~AN8Kz$-JWrNxUu;K_QU&?kDg54e;7E|b zcwSyk`gRf-U!XXPtS^?-?Gt<-XTJfeqD>9IPh$vDj}|`cJUxfgaz>0NB}&ZrwdI=E zdhx*~Pzk&6)@93Vg#1E6KkdSsSIN5&(1@(Rr$Q(JvM{Y&zMHe6*vk8;fT56eFgftf z=ZNDQ7FM5?Q|_Y{_aY>%j)x7;4tBxUibveiqmCzQ!gI4l3`<+v>BQ+VDftACwd@W~ ziV(SGxS~?MbRM@2b1@$Y+SW|#{t?Z@e$N$$hLRlfyDy0mYSI}vXxMvAx+V?Km1aM= z6q<Yj`tu<Y;D;0%EJfS&2+;?ZW14Tdo^C{B`nGVs=`vOZXH-4coF3fq&%Co@;<p2J zhzBybQk)&KuHz#92`~JP9=w59Lz<~<V0dpSXgk7)1;vfPvQtp5w_hMm7m11EzBzr% z)1b~sGRiZy;+jT?t{6T4XKK!c74W|I6_x%_^8A3Le~U)h%^ps{QO}r%azDj$G>Ae# z`F-y(n^QTw;)H&jWP^_8rlS3`7MQGR(2Fm)LMQ#X_W65y2CTzX1SEe!)61Z_kGCt& zvK{3N4NVgv(^J{SgW5wvHSt;`B|dQv0sE;LYeNPnv;8rDDXl}@=<(W|$@$xy!SC4? zSGALxQ@1c_@{6)uh+}phZzTPj4MM>3S~RQkkY*zg71Z{^(FhoMqtTf1q#sPD6CY}5 z_y7m!GkM9BMy`o@j9J+hXCWGf9RvgRi<N_Y5svYDNQ>!HDB4$@d=_VYtzW)3H=Q8O z?oMD)1G)HS9bhuSJ0`hTUXCbLecy2%)6TMbeQi6gM4O0sC^rbd83q3|Oh^kCXZbYr zfzm{NfZ;azv|(e{t`cGNiR;Anr?+ojE<(=i0M4E6g%5oa0a<+77|QT=D>^9HoSoJk znfiK;5D9xD0weAaDbY^2gf>R1&pws!y}|5{(6!=0K-}obxys2WE_jXseF_b^&&6#g zeRH|(V0vU?e#&8!42J`kf@_SA<})XL!!3y+E>H6CW3t#|!54Cce<E<vK>?5XI2Iqs z$Yf8J@3Sqf_QYJASWOb0<0*p{xu+^V2+N!l>ca3ciX27f-3`U>ZnW_nTVCR#=8&*x zBSvlTI$@3(52qywii$BZ>11YXunmwdkPTXx<u}yVT7XL{I;Mj{aPvp|5SvCS;I&2L zd~x5}5{(Wqn@R-1ACKMlo3;aeBtm-d&gzfw0`MtFpx$k>rNxs-Z%|Y&ln3$Iu5CLF zyGnnih>{JG_QUzCpCv^QXc-JK@Xcf1`br61$KnEfs#067;_-woCsIkiVry9g4Q09z zCzFC}E+8o9NNZk;bEWQjQA#f-A_F4j*wbgCgoHMbj-;Npo5LrOGa<4Vev6+8;`&P< zubWCaPJo>yatFj<Twqz(JHz%O)EOo^2+gL8?5_U0+7<zEH6a$+a~J{-=in`GmASct z*XR0vVH*UatTkg-tzEu3xg9b0;b+6si1L#wBVoD|UgO^u@-m@Psef1<Zr&U_ZWeWt zERoXmdSB^M-T^Ux?^Im?H%;`U)7b=-mP8W;2M3BjC8K@$+#VrW{B9^_F2_r(><e+) zP42GW)Y4-pZ1q!R19~2r!X@CCpZS-@-74zB{kG)U{oJhSF?2m~>u!YamNj@XUw5)+ z0I_WXI&Kl`6cDaTnP)xtY|c%62RR4c`;pBB`$f2fE*8!AMBPIlBF=z&_V%5O^Nxvj zbcm0X^c4EEdi@nbO$T|wh6UwINa*vX77Zzb8o@RL$I~7T!fAg;1o|@>6{(Uhuw_I( zklku%oULPpqcdic^*Ik0%~=)wYl@p5`s`af#-G0oT4j?zlmnX%N?WYfPlH6$xB+nZ zGp;RFRw@G%o%CciCdK6>{)N~1A?uNBGsR{0lTfCv<y~CmAsfHy)jTQ45QQPMXZ}Ks zql1K$950p}+v`QpEh5;Pt3CV*Ltt1f3F#(8J;MkG*YG{~_%y&dkwQndC7^`w0nAjZ z?NL$R`1pyC!!Lq<Z9bOd7i0Bkr+Z(MeH?nLU18WE#beo*o|IT=RHLW9Xzg$i{mV@1 z#~~6+<H+nAcL^eTw!}bl*C-yCS*Y~7L%w&g9WwC912_9JLZPhO-j(kbwD&WVvJyD* zQ4n$r!H+V0;H)gU7DjGrL-Sv-)AwrcoCYJXBevt;nF?!!LBxz5$4g-_4|TFDsOiB! zOwKSEF7x=RP(#mdnSU5*;xg4kFK0<`Xwkh|6Koknvxo@$u8WhJmKT~S<U)YN;AFNs z(SFV0BKrWsW1BQf%49Z9gi`G1J2?r^WmHM#>@(sJJ2z6-{-^lDTmp9_B)$o$MGs%j zxk@i`gy>7%bura;e)YDcc-&?HrGA8fGq!7mlRi{8G|FM9zkbJdsWPH1ra+P+a9Kl| zgnDj;>~g_s9K4?HMxPSm)R(FE!1WDfCe^a-8lkD)Za|k5P-E3)VS%>{uF)eg7zhx^ zIbH6ASW|}W*AQsMQ<SCOUZtz!X<w~E@it92;68iT9_xHFQ+t(<hhCefcpS)k-R)|0 zng)3&ca4rUcr%?bf7<=!(o8F@S?}@P;<?y9N&{Dim9Js^Hyd{e8NH2e6Y_ph4bMe8 z#3OqplTY0pq)HU_jPo-$auLGiy-M#-^~?;@lyTQ9L$<waQkZQ=%J*+|Qk?wld$6Um zI48@8;-XW;ywHD;8QdeBe-JpaO+ARnNJ`*XeHhjEl(^BsYvj_Ye$VpOa9k$I1h!Ef znWHnqnoL7B+wc`fAG4>shayst0)4qBU$kKT;SEzJ`!YFwRVC9IF<N=u3Gkelis*3E z76+5Oqr6wSefwjb_ct66Ef~n??JV>)u}1v8>I-I=PO9nICQM6y)o5B;FuJVU*9b;N zd^t2Sq7mqiZOc81V-m_K+OlZ{@M>g6pA32fu@^HvjA>Qlp28QEO#9ZtyjtF+(ZKhj z_Z?26r<PORT2-Jd>mVP8EK#76eC=e;VyqGqX%}zT9%UZd9VZ`l;pO3>s81$L$J>{} zwBLMe&qBnGT#=ti93@j1Thr32aJr!|5`eEU;a&UYiS|IzqA%+KgKnA_Pl_}4T`zmg z5>s>iBvzuC0!<=I&t`B~oY_aAzO8>rkQF<nI3>^k)amwfUQVj1bgia4^#fWS{y|3l z8dFuPcHT>5sj=6QN<Z%DZx{b}DKs^v`>?=uJ}oDz?Bot@r`<sj$Z1O2Sx=^!soDRm z+AZQ6+2akC8C5s;e!yuyp%boJ^dF#Wo9kOiixEE*BWWb=v8Z}(C-M_U<jk!UydN&k zs4IjUE9FuF?KR+gfsy>rhsSQ^3Mr~pAww5ekGaeZ<NDL*EFD7xug@|`C13F3fYmGh zS?k5ZOj~7kZTKD=gMn_ecV8~VQdZbx(oS6O)v)Xgv&&ZR(=Jkw5q@?TQNSB~_07>q za*=$=pj@2~o!5I*ya=TOVCQ?BM&6c|7rGJ?9lpXSXtp@o6&^Ojnl*9}C(7XMOcJu{ zB4}Ef-~m1Gtw7^hl-8^o<tms`tE7^hvK(X&%GsDsFXc2d^I)8uD{Q$TtJ_Cfp7PYn z=VzY_T7v?u+gilR(wmfppnm9Gb%g2qfq&!uHmqI>RJd@H-`s#i&BvG~zo3;AUpZ6Z zNZ8w$vd}l)VOX?y;{4VO9MLY<t1+~g6%I@I(s@s7Gep;$7p=IdInY-Ok27J42HfU^ zUK+QRFntV8RI_KiPF;>*NQ7b*R8AAiq=dOKB3F2<g(pQ<^pVRV{jJ7wtN@7Y25NY{ zp<hqW5En;pj2o-uc$J!u{Rsq6vcRMzhi!f?k%$GFp3+q2rhb)rN|7zR1eV-uz%viD zAsDnuk-~Y4x-Lrs9ye>>d?2VqN+B{tkWqNi^=ocH7igu3_ar_*LI}S1pr~u+CvBC> zIZpFynF@W5T^ig>b^dwI&I{NU^vL+@Ddte3twMrm5m7YFspu4$n{joZ@xXcUHUr=H zt8tmyQHoc*Dd{+dPKTW)n^jpx0n-YnVOh~lz<8MR-sQ)v(Qmf#0)Uyj|K}*7;cPly zXnf4C+-v5&-afZ`vl*;;znLLot9+Vlc(0_8gWh)I>`&i!A0S1bck$)aR`|~LQn~sb zqWxkE-9``+9<j#2dip(4YKFEs*zkE1U+XX>9u2>8s$E2#wKW;y)R3~@;DK0iW6<8d zzmAuV(o>JI^JT|e?<xq`>8WYD`CV^n_TWPg3s>dyGgas1PyE}<xyl~!l3%obzy12v zlLcD3An{yf8ZjJR?uj2tJwJ+I_(dkMP*1P?{MC_MyZT0tdYR@Y+tM4s@Cs^zTbvt) z3(SX^+XfGnC6$yRJ{L!p`OF`e^mmDBrwd#*Yil9moK=!3pBH2dPceA8wfui{Jg5!9 zar17N(<8uRfKRLvn9-{?5tHq$WmVw8DdKBUZ_zt#L^JNIfvyYHcgyzubd6$bng)Gf z`F0vncT}g5^R+CMK}hVg*W*JNqT6vY=_9m0spplZ8McSg`2%-<@q7AWd1uc7hpfat z^<DB8yrPVB52QOl$vX6M97_7`JR@lM@a8T@{7$7JKA-T+2ZkPO-?gRZ$fuKjEex8? zrlUh|`eus<2BP}qP<zUv5uTL80zX){pElF&6-@-sbQjo_eR+FiK01D%7qW}a=1VlM zT9b~F$<8Twjk6g4aKylXi84<}$zj27j+R)$rE~Psi7q0ef|E=&O+Vn8e2Dbpk8SwE zryGR1X#w1I7Cj_SnMt1==oV+EM5{l|xf%~C4B<B3^n@JezPsgpHwl&%H<5tI6Z?Fa z$U93Ys&38=A001LtVBRa=~)-d$U$yXRqFZx()ZKCaBQyfT5fV1s!`f#vM*TeNcg#5 zF~;yurKUvI2v)$&mBO#<$xbghozBG0ZbHQt%YMC$b9K6I&E1qv;PDLo$V(|?Bl$p5 zP^WX((+6B);o!YW>*L+$LvP@Yh&@8u1nl6D<tYb0d542=pSzg4BC4uJ#8X}=S`xDx zCk>e(eD|o3;-UTlUgxXvqvYGZ8<CF8E0Zg)#}aP`I-(wbKfelW$UEe*I!)x}%(MH( zx@M~tKKakH1Rokwjdas*ns$cU?WY#aph2exKPs9}qIBzJUg|Ct_-ZTR=IrLb9WPmB zd|VwyztB3BLgXi9VX4Kg?6rVAKE`T0kC}KR;~LuX!(nk*c<zd|e0xLqKzfqI)pJOZ zOZa#^KTA*4522d1JRYp|zIU&c;tuonpt?dHrRQtuviq&^GEQ;XDiP1QC?C{59gWPj zp6~sSGAuI1ZK(6PD-ho72yEv;#Nm%>MoRp2FuwApHR0siq_aN};tdL;h=0qk4TxY{ zj6m<qWeWI3=WW43=WVMa7crT~4TiB?V;W==)qP>i!BnGI<-{jX6oe$Bk>ML6DnnY4 z&M>)7XorE{b<5!H-U4DV$s(xztZyl4pM!P^qY#7RDWoeH3vEpd;GNW?4O;pZm7Ixa z>$Te1MTrnq?Ai&rT&s}?(QleQ3M2O(?<(mqPQcM5)rP9f`d#CGUjl~-t{zFA1l>t} zHs;B&a_Afjzpa;}7{oP7y=I80>z|$r7*Emp85M4CqLadBUgZ<0U(BNpV|c)qo=+%= ztkH{~CL6++vF6YT;W`F0if?BZ_|a8)7BHHig#Fzrj=0y8p}MxQtOJUl-dnbreFkon zHY<4AY;e#=yM~43!kyaV(sZ=VBAmiu7+$Y*Ek;(r5&4feJq;9Ug==wTPfj)Or<+}O zgD)sRA5hQnfYH5)wMk3nEDu!26kbB){WXcsKZd|k#D#Z9BmW~67~ZeBnF4-2b^I#q zLrI;%;`<u+GPq&#nq+VfA1539^#Bhauin}dkLqPgL9d_0&dTQEyK*&PvK0~Q4Zqci zu<rSr*k>pH(wsJ3u?wp5t`@j!3|-hy#5NA+r+-z7`%zq^byAF$YM0_q{ZRS~{0Xk$ zZm?ED*%KWYr!iwP3SasUoK`YDe-bKAA-4*!jh@cqR;GUenZhOM|4|E2g8KQ?TwU#O zPH=?5;9q>ezfEgUYG?vw7~LQh{aQN$e54d3XXn)<D}Wh26xuF{6d-Ccu;0V=V{o&o zZuKI*_rRfz%Mk`0@uelPCA_*(p(0Q*4>?(7R|mo@cMDK@H2zlZN5Wb#aw<fPvhvWl zoKS)0i7HIOtq#S215Yh|jntS-SVfC^X78k_b@=xAvqZBvMeHw*+?pJr^0{?7cs&9L z1Q5<(OvV2LXy5`nc+ZZ+<;*5VMB!D;xco%?f_%StU`(K(ng+u{o~l?JKFsXTeViB} z`*BT&KYTKIsZlW^cKa4BJeUv$eMvlH1_v+w5Ha{C{A~8lX$+C$q7{l`4Iy;g5Rglr zq>(#rLl^;+Pr}_OPlF!Epm_a9bPPs!Xvt#EWJn|!fmnMG2&;6@g#yoOU5fJ`RNhQN zFQq3{OPd50TRL7{(Nvjm>U}{Ko}Y{EgM}ml^`wvF^GYHUAsltAIhO3$_U)60Q$VCW zCgo*?Q0^o%y?^x5ajaE~#gj+(>xESmiss0WJ9GE^W{|X1^Hzt`W?PaHTBz!+V3SbC zp4RbH9rC#VvbMH8^)lI?HPnAT1AE}Oma7*s9rP$K^WTcDGoSFec%*IOxb5Y>BJa6~ z*1e0HLIGswTtYjjzmw4Qqt8+;EH%u!V9J(EU;{bxws5&gu9dB%lK&7bI_L1r6#Ldj z4bM7;qsRN?*<CDQu8$;Dk|u`Hvq<J!J345DMMSjDxo)+E@q`Spjnw~)<Bvu&7r?tK z?(j8?_cxf)p{JQZy+>E9!T7%YJAu*z$Q%(itn)giy5pIf3uQrRu|+;EYiUnWbFZ1D zA-#&{?||@6BT!oQb?D#tU9-6U$ibWezP%=L&e(bazrX(1^$O}l5=sA6WB#X$|7@fc zLhVWO$t&`uz7o=yJE3}edk5TwVkH36(E*WH6=09)SDkYKkjMBd8vL2Er{)N0X>ql- z1*hjQz4&ki*HC{QXNiHL;(_p!p}xlhKS~XtC=>ksE`Y!5QDKzpPHm`!glsm1L==Jh z5<*W3#cOZh{W0Wj5Y>7D)I{M>aRk5gxZO{vpWGU|Uf*`AoTBS#59J*Jn|In@rTF>< z<@AXDnxmX6z_wycN=705U}WoFYCQ<9cn$ul?XQA*+B_7`(vndlNbyM8$t|Di>v#>% z9C^cu=0wat9U3)9&^#N1FMgw)Fo7p0ClM?OELqT^meHhgZtS!GWbe!!>8$Le@SQ>l zuIy|ieiqt18$NCLVpNRp!CLSjyK^#)F8n_W4uKK!-3tp4Ni+yFCHsqOwsDV7=6ewl zzT*-+yjUh8o~U$z^M~g`*@p=_H(z*eEvU(S#0R7@t5-28*4K0que5QQ5HU7P4rp*T z#p5<#S1}qtFjQ6$7^|?`9K$&aDc>*HfEh7(uiE=8AuFUGFi1^78X0F3i%q$tXab-G zg<Oyn3h_lKk`=uI+sq8`VuK(uDn=rz{7*WS{-umMp=nA)ARIE(@LD+C^w{b{XVfpY z$P7Ezn6D&#x%VjGb4(zXN+d}2F}Qy_p<9iV8A}YDHqUjO$FyvOJN4=O!6$<{?IJP@ zD3Rbn_62J^;x8iFKkn$Y!I&ikny@SBx5$xcJ9|Ak@(<1wvk5?kB$ipMd&4qV5ww<& zCJfq|*J?2(z;>H*|02nXeE44~`{!3Vdw`rCXUBw~rsC^-6rjkq&u?Op2Z7Kg><pAt zWgH0!37a2ekwyQLm-Xj{l9>=*iQ-GFMYc5+1Ujxefc8V56hBaux@fUjObo|n!*M8G z9k`6=D9V}4(;|K+6@ie%1M?f-PeNyN@g*YtPxJVbI+K?8+iZuYzl*;H3^{KA3<mmG zjZ`!~)p=+shPALrma;=ADdSZ1R><C`xY#%U7Rz64qgFIvnjPYf*Y5(Ykq+t^<i*R` z&Sk!=l?yCgDaXo6U(<&2|IgZA#}Y^*`D*Z=c?<MVf2%%{0tqb*;@tub@aGrO^Sp-P zh};)6;AF4jM+ynisuMBS?N9TIk|{3sXL!}0|Hmpx8IhsCYJP-=1{d}3+u@o5U}Uj( zM%4f{il7aNdY3ow!zrYa>B=97waHK)c#kbV9S5x8#+3J(P?QW|{!?H3x0C{D#EmSe zYiSzf7{h+wvXCv1`I}iG+#A8yjAOtv)kHZaC97pfkqEbri;n{muO^_W09}$+*VUnB zUrX-;B;NkHrNCTa>Ur}29Iy=PtL;-NkmLNJQhjL5-QVhsx(LTIUSp$e883>z({(^t zn3O&j2i4A35IUdT<N==CqlM`kX6*l=T(3NgZu^I%M<}QJ-|mc=3_we)uho^|w{|vs z>_f!G>kBsrz_^>*KMb2YIhliZ|EHq<?^UJZg@^<tJs~+g|L)|h6RLINQkK$`eiU7` zd7+FGC{u%+(LBDp|AW?r<4fQE;<(T$+E6+_V%Ub+wvc}3pP2i9jZTgVfEhtOy#Utx zLr1<r0X|75b0FO+X--FOEkliFslu7svY~#G&?$u?A_6@gUs;|PkT^=9MUwqHQEb`= zT#$@u{TFBPFWZp*Z5!XHW2awjBTWygwXF>e8$0&IPQQ_G|L85u`ATv%Dvpf+6W@>b zw3AAdxn)2aA-_^QyoFY-<?U-Sz8_pbdxw8zJCrTitI`&#SmPhc3_krC9{%f7HR6pp zxWt8{1^ZQ(;bptGoqBB=zvlNr<k)}At3n5wnvOZNF_9rxuLUN-qFilEX{g2qyC#Ad zdjssy5<1XLxqgm+4*Pl2xbZ7Nlo@v`2Yx&G=<tWAMhM6MBiH;p@^ybE5lZlo$&z3U zdbPAX3<o#2H!SEF^XN<w>qz7ol@}x$fSOr1F$M`G>HA+E5>zT>t3I#KUDPA|8I1nj z*Kei>Ce>?q2swI;F|Xa3pedQ0rXh4<SbJu@h1YWfP$i+gIwLTIU@bmNs&8-BAODUf zhp6I=Z*e0S+*0~S95@QyOzFG)7pK=!IU=2BXfXD|2_6oN^sfc?znqL1R9=3*I0=V* z9-edQKmYry!5lBtOiG{}CgtMlIy~LKnTlg60tD?Mw>r2;?h4W3D6`tjBROHis<P{g z!P<!Y?HOM&-#sD_CXjkYdh_Fdsd5-4DjFK8NR9n6UnEnTl#h!y?6u%7F823Of^xzA zp4z3+v?h*gk&K!@Hp_Y{0Apk0)Ovq61;BKA00PV3<h8%YPZC%n(f#p5?>>Hxm~g5- zg#h*$iGh<d0sW`^%-D`jL|6mM!J6!B%!r~zd+52peYYs85fYfRyE^SZZQn=*Ln)Vf zU__$mdu$K=Ql<+#-|6?MaG=z9?4%g@df;4ne$eRd=yi$zOfZb<UzHic>o6{bZ0VH@ zEoTNeIHh=9u|Iwt2!2yFHkfk3^^EVQEe7r}LG5~0TM`$}YT$c^=lpuyecpNv+Urs` zqwNt53Lb4X7IzCF6|k&=xc^Q!{%xw!41ta~5#f=-HoTi1#mOaZh;xG?5&eHM-G-n{ zOg4>JGf?fAqb)Q1B6zpl+SXlS<xM8g-hP+)??wH?DyL(+o1;LY{Uq1!`nTjtwyyf` zR~7?neHkM`(<k{O&`I4}WM84caH3EIv5jaJRO*fzuST<I`fGeVcDTW4Jmqg0{3(gQ z<naR)*FC|5-V{~$GCD(Qd7&2<l(d5}7?Ifhff5xD587LI<Tadz4pV;t5fUP&F`U0T zS$RP&y?+R6|FVN<$UsN5h$1m0lPZiES0c|N-SJ4JD8)DtE4eMMth`Cmb_ee#W?b?a zOv~39CIJH?m&(M}5ZPy__<TNY(A-}WMvS)8i@EbU6xKrcN+8FHo|NM`%xZvryob<4 z`F7}sg6l5eXmwlbPmmlnw#>EGt*wQ&Jtfyjn<f8N<UbwQUv7*NPS2^r@nVp~4V*pZ zMzA_1NDx-sU-wqay9rBs;QWr><%><fv)~!&<>7g=ABQP(9ExkK2q5Wm01Gk=ym?4p zG{AyaAsZ#a+J>1_xmQap4T+1GjCgQdN))rbxakm3qvI)KPDRgBi)Jmhd52+{^Now^ z^)#M;RQ;buU@p9#Qvw%mu*ZU}ImPa;(h>oC|0&l!X_&YV*;n8dhU3k5!MA&Cwp{JH zIl{llAb*&L_o>c=X85wcBnjB~H={NAaze|9;xS$7>&14RXAlj(W*}*BP>(AakNAwv z5o1H5DwoP1U|-yBA?}EMUl20mt&%ep3EB4|)=i}X8g}D!F@LDYajop7!<B6k{5wef z(;kTLro@Q|W&!S2#1LN9v+YV+k)1ccDqQ1+k`T$4t89<*QTX%KmHD8xwZpKYPQ-@^ zZb$vS8s#dNGh`F<LF2xgR<vw%*0`Rs+CP#C5jb2HXAKygxH&7Z!%*Otv9o413Si!{ z*3z;nHm@nn&x(%4S~eilXfQWBR@lG+>e}X)(&hglzSpUOpcq@E_#zfU<W>R}p2XL0 z#Xi%Q4z`6;w5Uz;Dq3$y^!tT-5I;<=!r&(MB{Xp$XpV6nO5*EQKUC=tEu{t2OTZd< zuLfVAIeh#W)?>|L)Zh9|z+kw7I`eG6Pw>L7ngqqJ{h=1;lQmLWDp7R!5h%jHesf+x zkniF{i}Zl3j!kbZ3h3szs2$H<YN^rdobW?fU+X}KwLm<o%3wt86Cu|%3UQWOdJ`qm z`6YR^jV(I5h?K+YvlvDfsBlRoUYX;$9DIaq7^(@WKY$mce1S<nRKB@cOGSeNENZ&2 zVlD<pt#3$Y6GJFe*=<~?zVHzP#{XApCh>by3P%!w6n)%f*IOFHG8B1w1=c^?78^x3 z<|w2N%*OEcn+%(#7Uik9*_fX~&T1Qri>rtUuaXWeJ?okMi8L|1-nY79$#W7eI>22& zAGN*TQ%zt;;3okNo`sw$dmM7~a#R3Wtg&_hK~~WUM~{t!yIUTUS?U7ygihd5WF|Xb zX7RiP0M-(I^#k{ngXiQU*G)a*?lt>HV-<Cj(h;aKe957Z)<dtuJ^!f~R@ZIG;n2N8 zM=)2i<@fH98xA+x??Y-%AbSm5%8@qEUI3=We`c<CGl9yVUT+s4?Wce6m;USq*Nj~U zXL)jS9djcI2|k--NUyym;+uce_1_IDEC{U1L=H>{Q@)gyuRiB*CIpxo*OtPHy42!! z{cZbnB^M_r@rcM!A!phMT7?MEVy!!I;wA)=qZ{1u>aUL{qyg;`j@-yY7a6ZVhH1Nt z8y}Nn)5AwQ+ZjqZm5eK7DFYlrAfi4~I}1v`9Jxsfks$YB#TB3ba5yBV0+paq+Fh)Y zE4a+CU%`RuPGb9uL=Lk)vmd8^P|LRaX$hzqeF%?OY*!b|AYdfB$>nKkgz0<@7IG4R zg9#sMT3X$$m5@hmjh55xLku^v?-{hllv`mx?^f$wq(4P0n&c<e#%spM1(kc)uDvH3 z&PO5o=Gu?*KOGIc$g6rhWC43)*s7leFv{y7vDyMS&Y=XZO{p87WidDer|B=ESg93G zCOcer{W!7ub(W3DYq51}u9J9nrXNJ1j6#h2G1#h|909dI@)NpbZiw9MZiXP&TG<KC zt%!W4IMZR^@76*4E3MQ{tM2I4LibAM3W~jpZRAF^L(vC(V21L+Uf_!jfriWwVgXwj z4?fY<ptauugD`#2PI$$4%0rd|*G`OxcHL-H>4;anh<O;^Lw7u{RrFb+-<(Xdb_zPj zRLMQ(#}BdVW6M2xE$T1*CFZYhq4xNU2ak<Nzf=4sQ0p7jb(<;s@Pfl1>G{9C{%f0g z1^z7a8BPGA1X{ZJ{IE0+<1JbHY|bwr2|Y`A|Hxd+Qm`PtuH)_i^E|+qIQB`GKQ=TG z%UC;I(c}qcteO@*l%<E?6GxewT3$>a>r0+OXx7>P$KG3p#jz~`y9p3n2M+{ycMI+= z!QCaeySuw<kU(&UA-IR2L4wN+8VDL>@H^RO@BN)~zJ2bm`}00e|7e<irn{?a)v8tX zR`vGI_{(p)U=$3tdRjIk0#{!!_E9TfuVM>Z<&y&v<+($J*3z~6+rq%|MFq?d{$eFF z!coaMSt<~%#Z!gUgD|vWei<Z-4f0)Hv7mx0H9lDX1zpFOqyicWgyYg9d}Jqbi)%$6 zJrI+Wq)OICZL3a|bnr9sVQ{b3)B=br9J<?6Ff(E!+&Gext)#pa3`w1K<YL2rJLrGf z%McQnh~|sN6H2?{4T;Gg?j@IJYF!WQZ)|L}RII=KIao{%u@EyjDmD(KJ4!x>B`iHB zIa+MsM19l)f3_)$9d)c4f;Dn37`j`hZVU|Ck8<k8M?b&5dNqp8YP62Ps!C?oAeJ9p z`ccpZzLaEy0>~<(=9(6s1o=p6O|>(0k<iDF3q!#S(+vMDdUwn@KoH3Ri#I;{2t|N( zJclPdc6PRP<z_f)+`zoyu2Wir7;0bZSe6n!2eo%)CQsLvE;@ZDKC_ZAqWGG#-%RD` z|Mn7MgL~0-H9aTBPae(FKTy{+ml^@)e&B`76Efs>pr9#1Q38PjTk$wv*fBMFS&r{w zQZvPWubUJpVL<G31}&KUV^)$8_uoJ^QnxYhG`R1R)hKq^;UVJc#WpsmY~J?X?JeRx z1KDb>y9PP~hI}3$aCqve&H{@(M;)Xmrj<n3V_-kooODB?bJ<a}aj0%GG5Fgzcm@k9 zRo2g43|fE27BW~?Ft!|LAJ|w}m2V$fDl|4&kUTZ*Qg?Mc7Zm#bHK~PE{vMvr*0YRA zMd)KH@hA!+ysA`_y(|Eyhp9?h-&l5vF)&W?E1h%i*T@nP`c7|^^M8uF=gOIB!p(`J z5%YQU$WcUo-r_$H%USEv;!bmn1t580#v#p_Veop%0R*u^)g>4_3hjrAg61QDc_r+A zjnR{xA^G*RW*#9IXsEf^xH1N3y76ot0?s7^Ns#3+Xj3f-mQB4Qfkxx+T5bRey-s=n zLgRS>980RGmlrQI(D!X!Xe)27LFhFqJmgXArPoqb7|Es2ecKgRVgGFzU?LhVN`wh7 zC({{7^x7BjUCgX;bbOPwcM$$Qxbkks<Ve(2nkkjqLdx9XjAj0VL~}A64Mxa7()M%b zvoDq3*YoD*Uj!S7Q~r}#XO|+(5&QGi5O7!!Y9=`IzBex6*WAoTeb>RBpUdXts|%+N zgwv<}b#HKOAx-e%Q5wn)fojM0BJkO3()%nXNK7sD!EbYaR<!PwpeAHFOJWae7LE7! zf@bRii_Ls@l2qC^npd%g`ekKKhit&3W_BD(vcS$x$ys6aP~^|s?6kbfh1mDC1J`LR zINd6pRtf-)IaIdq#3z)13fzqFbQv_x_UGZw{V4p{Fqp=<-GNQ7<Ka!OQz=DvoYYi+ znxT4y10#kxHAT=xV$%Cx6{><eV|y<FKob~t!sZ4bA3l<rv|JHbT;1UnI;k+@VRVzT zMHZ7-xvMM6Z6LUH)w${?lfXwN8gm}Jq#di-SJOW^co0=m^K2vZgtN=PO3=L}J?x`Y zDQDFUx47NdYsvtrN^wqb?wc2eV_ja|pV%PYlo&XE?cliFKH~w+gv`BcyStlLYM1I5 zYucVYT#0;6^C2vb@1Fs=HebYc3@j`j9sKw;prs!2^;crwi;En6Z)#Umo}VHg9G$;b zu+hNA;p0+_5yOUUzLK$HP9zmTdfeFUyQ?ZC(+=y8*mlY+;otd1L|`+}twf&!9IusG z=90R(neFB~k$caO*^n=VC1-57*LMY&YhPUB`>YGME*f$%GC6_o)Fsi^8T4LkqkO1P z(Ct(%JId?krAraFNr0Xc25g&|8M|E}%4p(U!_EGo4Z$AE_t_b=PgR0Rb5>p@x{_;N zyu9Yu`>8P7{Eb0w`2GN83IQ22`igN>uY2c{_itbK*d2E`+wm<Wu8ZKG+j#QMI$nSh zNXI?f@Y{v6DQ<2OWMz{^V~GU4Ps%W0y9*I=(Bq&c@^|!^-;_sx<{Ky(rKo4%SQRZp zl$gjaa`la`7@Nj%;BJo<vG5X+c*%_oudPaVlwPgqomWw)Osd6owgDIPU-QkYL4zJW zs0h<Hth>TUimou6%`Q@EZQ;6OBLSO!7`%_GS`K21#;rKit`iPl(RH^N)*AQZcdG8| zI2X6)C`g`ZoVmO<h|^_7zNU&?@ngjVz2SrLzunVxRm3r6cgvASC5%fQcDoF0nVrG! zz{37K{>OqN`vqKy<QB9F$dB;pE*24;qoO6k&e=f?2D$FbV4S|O+HKpe0Ijt6_gcxS zHC)E$QAR$N%?>n*JNMWbbq;O(=ml3m9151l1#=huar|J5g3hIBB{Rv_Dz)g_9fg1f ze@MsVaKFPxiO8|Ltt#AHNHrX|=;2ZhE5gp@C^88wK(Adcu0(eVvVJ;~p*uCB22>X{ z&F!KJ6=Q)>km3_XuQL|6bJsR#-z;#a4yVR<J)m|y^g8vC-$^Q)vT#T^s{WwdE3pwH zHwrREC_&zfT;n>_<h$QcJY*(aj~}gU{T5%6Z+}dGl@EvIw0r8PYL41IH4KJ%%$>v6 ztdAKQ<BtFgxWAxsXi|!j2OaYlqG@ZsLSOdvl{m+ZOD?sf#jhZ%v?HpZHp(=O*TUI7 zG>{mbW~3X!Rk}D`wHuHR*xSh!R^LdFr{b9&P$JbW<M%so*x}4gj#vC46YzA{{dlNA zy&WH+6Xa0r`Scq7xDW@r`ddxMin)18dc(}^`(ZtQxF1g8?nmd+;+K4HgT^j7Y;U{3 zwU7$t0@++r7Seb}uG1`?o)BP<kJs?~jUE)99Or{%gsC@<f^Iv4vt@O^qA?uwYQSs; zjf#?cZg<@c{reV@V*+6`0K3tB!W8_&l(mW!tT^HKLyZVj9qYmvg_Sm<dk<_}rx*Q| zlL{;YdkQwF9r;btEea22q-}vcB;PeJWJ=Y#l9CP!cGvNk<qicYpr}U8;XqNZJIQy9 zXg-$4ScfQ>62FzD=#zl*={JCEE4ciEH?DDS@t|dq1A_?9qn#oEXrc?wtk&T<%)IGT zLdGvzRFEd12^)|d4ky{C${UbzlIL=~?`#ax64b(LO5A#5O3!e&bBAfli$>ZVgd6zl zlVH=_^t(Og;VTHY(ZW}4P_EBs(HhUN_4n;TYgdp#jKC=Qv<r50y|+flju!>L;}kT= zAzN2*<j8wpoPleB^I0c@evut)_j=!`RcH^FRRv#SaAV}!qr1a<<b3hRp?mYG6;L5a zNGCrj?J~Q{H~;gXvOY-taFxZB>L{qFqmr@y1MzHcrw4D%LTusfBPn-?cSv|Tw79l; zh(zl=Ds4EAj}Dpf6vTuiRGi2-ZAMM}JDtT+80<1K8!WmO64q%Xk@GbQzz@3^Ah~Yy zb*NRr%H$|StK+t8qycowdLm{A#DWidMa`&9rc!8?on<#tnVctG+>F$ngQF)*-?p2S z`Ppi|@O`@l<)}{V*S^iUiGhhiYPjy|uteEW<IND)+t|hXggM%v`rUG);1t_z!)$U2 zm)4CacA{ly1luz};;D9?Q6)tCHnBD!|1~<O-ft>ay$ow_-k+)SGeP561h7uB(3OAf zWl^ZpjE_au!~z^R^CDm=P_#o|%Iw=pQVAXT%eXzg^Y(XsMR%a@w?KL_%*C#I<R>+( zOVvWiLwUTce~;qx*9@N3@M@k2--pCKDuTu>x0q-=3=nT1Eh$xJ^x(btiD_KL`Py`@ zizb_qlRUsNpo}o+K13!Q<TxM4R-H5731;EEkUH}q%J6E!__iE|QqSJJ+gKir&J{P4 z*UaLpio1!)XP5G040zQ;WB(X}Kjj-kPAr=rWe=;hRuIhOaK^U=Q5W%i0%Gqge=S;A z58;<1^)Zg`_XyD$=M@6a?K9i`tfV|nbBr0dI}$%Vd|z-80iE6{5a}9kW|4aE*~xjJ zp%H@c;D^rJZiMrtkthUy__dzEM6E!`<Tl-BJ;9KAE%1P73PH*;@Z4%^0e|Og6579F z_i5d8>!ZV}OI6T0;<elGxUTN*auwFR74B~jo8Zk$ZB0VG%#Y;kJVc~ja!*g*0yC<{ zaq6S;E(2H^q4x>JJ$G;udV&h~sq3nN1g9OSxw85C`3VyDQ}ET7Jy9)3%*4J&C3`gM z^kRGh49MJocVHno#R|n?GA)upr{f^)IH8)MEe}#%N|zW;u5aGeRQ2i<jmSb`DLN7( z_yC|R42aGsxw^G>wO1c%t_$x4jW)et7L5xSd6RuVDpThOP*Ht?Hq}bu+mEtKxe2kS zve3~SnxJ=7qx+&FzAJZCW&=Rxh;og;EY@7lCpxUt34Z5Z9<C(Ht>Lt{=5H8X2!^|7 zo!-A0s195+I<FB9t-Uy%9j#}SH0u%Eb=47v3(pqZs?HfBpgxCC(?xA2NYtN(nTqlF zCkBs{+eTwLDdt8b8EDr8xYFi{)?~6-N`i@c3d75OB4(ShoW+W(5qzimKJKA)o3i8X z;Mh{V$zu)&rlyxrjKAPNv}6Av2i!%v?fm{@{zMtF)5zc^+x;cq!vDIm$KmcI=luC$ zaf`xBF~mbuznRKz>C#xVB7XnebKDS9qUgM(CjqbK>GLZ?qjNm;hx7QvAB>_1`?U){ za4^i|Y8HEyH+@`_{)~T?qrco8J>t`kEia4sqGX&{@yvRb9r(;|kzzc+&8a!bAb8)p z37QKE!f8&!2lsRL4ZiiQUMQ@U8JXbn5jbG%LlGHX{62{l1dG9t#6_uXMtH-Hl~&|` zu1hwQw&dc8?|3I7a;#qaoG5o_>~-u~3&EInNCA$((yaC(-TaZWaff@@Z+uj#|FBa< zeIe@xFjH!j%^1yENP3(nIWK$7LZXQp%gqZ$*Z2+u-Dnj{h(8R)s+tZje-l~pPjMRu zfV6eri;`)re%04`l#J6c;A1of1CDwozBcRRAd3`BT6o0&URJ+`k=AKhlPA%*8utLV zzv<sJ5(FFwe(3z7j><q%BeBD4o{k$=Kkt;us~OX7u(I||(PE%YbET}gm}Fy~iwb{t zkdC6!X+)7Z-e$amA)AU=s@|X`9Ykkvo<hW?(&lccZe*yrL{E;RVQ-D=22{=`oSREk zOE#5stHqFt7<;!7%Xy{tWlp^A>j_sw@X2cDJAENNGKxlW1Ih)z7oYgac{%5(gx#kA zYg8hIiCXiE8sLLYr~YE3i;(G0^!U6r=q*3#@-(1K$d$5f3Lh7I;{{ny3k}Ob>f{)) zNRt|cRQMZ}xRmpj-ugk**QV(+0Ex)@86sN`Y2b)yB$)iqm2tK^oFc{3#$8e0+fAM5 z)TCz^-V>YC=13ywHzRQ9UGfE>v&2UKLzCx3A!(GI;}bF2*g@zX-vtT7+-$NwP*q%+ zY4}2XjCG4;w5Pc+grX6$kdx*oq4r65C#LU)n;Mg`SSM#C&|{RvA&>QU=-#;!acCmh z84F1`-`kECeVhy=JvR2P3v)b{)0AE>x@E9loWzlMe!zMHovJHtxGB$ZxT#PNiEb(> zq}I$GR4>ihURZg#s@fQvfzZf7ja;qsJ8KZ4ASwDI>(p*c=RCzD!0y(bPj5NqQ{c_k zy;kmyU!7k1VjNu|jVXEjX<zj)#r21-5)wB_suQR3C7t8SgpCX5$q=>8o8mo<SMTQJ zlhu<$+NF%Op`S|TwghOm!})Y4l!G$8P2$uWz7o$q7je|r;sibEV`nncOY7OfC^jMi znsvZ@<0`jD4E%>=Kr@s^j$Twh>D$v)G9lyH5;;bIaV)`5Ryx<)ztNWLEjUT{01%P) zZX5nNDCgA@kY@g^7d?+rt#Nc-T;1r~`?^zIg|A(^Ks7K1f}M=-Mt@J!E(PLr-?556 z;0M@Xi|+#6$ksUat*_oI9%xbKcLWjAXv4MVt*v{82}!p3@G7h|>!p}}pGIh))WjaK z43D)xU&EywAkuRTp4_|fEIB-T*+HI357QtRTAi;J2^`PR&#zW0DJ@36x?na)V;aI{ zW1S3jmovdP`j(em`E2jNU7H-QA*(s?MK`9XLKoF_&i$ERrUIqXjWy8maTMi*aT;XU zd1FgX^Tcxc`>iz6lI7-S3!>GY_(eTE%H}5IX=n9mc3=egb;?!0T2g!3(DgwUBk|9? zWAUpFUlXX8%AKj7fmtYD%In_}->qc8ig8m@<sjYQ((UrD)Z+1J{4lMTF?-Vgg#WaH z-3gpFbr?l-a{E~J6(GjPOgc<`!IqbvRrD#eUDdO^Sx)?IwJYm2;hhg4ic(2jl}gqO z!s~UEg#OyRhR&#kz>R}B6{txqBlJx|IU<|yATNn1Ypgdx+aIxmo3)tZvUy$I&CF3z zF9k9?L{^wh8`BKMq_Q7=<2^Lb6hIh2HVxd#^OTmsc^;Z<j7HYvOZ|2dDXZ4?#59>I zn9v5K9}%yhJx7UpbCayYJe1yeo2|O6u|LbF%dVMu^go@TfZum{`!q-w-Jr->V!gJa zXu;l9nrOf-mi*n=itRhxYX-aEc||G~2&Ql<gIWE0*v^CjRW<9v1z)pd3V}(k+Xr@u zAxq6l+<UKHT!7vH>iDQ|{Ep7KY9hUqZZxOMC-vfLqsK|5F!~N?^@6_I+>Ur?eBP(e zl0sh{0wFABj#VCP;gW1W!x>QYmVdZnMd$e)!i_KZX=zCxu(U(1F-vSP(Qbz4`ACaw zVbm?(=SSHj1nZ}+<y`L{D39h_Zd=`Hv<PS-f~k3O_}s%tGhfI+7+w2}R5+mz>x^{Y zt>!hfXlm|+WAO5;3U^q6yl3lMZrhTUj2E$KjUQy+w1d2xjh`~321u41t1{rQAAcMn z%YYWB@2u-8@yMTv$3S#AMrxt%%Nznv@q>Ub%d77PLG3CE_6O8#Qp1<Nj`y|=#;Nl@ z=kObhOTn+k)O6Z>>22F$5Fk`9`QA%`74XBnQ}g+i|LXZ9yEThRchCA@fB#OfrEQUQ zw<ix@*1+_`d7qX)c)Yuq+R+Eh3;DkAp6>HC&z7jOz{T1I%BtG!0^?P<N204>r;=bG zK0;DdS6tu2DIg7~t<O$AJD;fC`|G{nwU%smpOsB^%uK1+dA2F7c%^e5blfo^xfce& z*6V9+!4AJ~_+mbt*|)vklj!?kzTYp_^P-cgBgwy8IS=W!8c`)c3R<LXwbxha#$u30 ziIfv-{o!Y@H>zEEJIc>!FOcrTF)5PHeTcXuZ5g==1{TvB9@FRRfZ4dESJO(ThDiNi z@nrYzcFH6t?Q}$7<P_UAP<R`a01FQHtJfy%b(yXiKUwJ5Ziu(wp=sJz*bNyZlIq_W zsAzGvzk7`FX)c7**$KO*E+pKc6AP#^o*p7A@_IPgur)eR@ur0aDE4@@9$KX@_%$f{ zU?~`Y)B$F&@N!FwqZ-L@F>+l-W&l5TQU{ME6OIp0>x(m(HH<0&e99U9>i!>T90jH` z>`FexDhd`%P?7U$)iYp<rU33H0hqpqd=SqFlY_?zIe0dW1z<AE7|ec657JnMrp;AO zGQCRsr)E%1RWj$SUH|k!jNyH9x64N=zscc4YE8&$$gl1UnzQfFyqwZE(=)zKP)fK{ zsS-oQooe}A*Cl&2?V0ZO{GY+yV7QU=U>RR^d|xrYAN|h*GFgJLj61Q*+p_)sxY{~C za8CCg4YAix4{oO*MhZaQ9KK0`l+cSx#jBP#or!@+MY^fsE2Z~CMK1tFd^WZ%M(XS_ z+_bwJCBkk<^j1GXsswBQVx_L4ZosTjgVZ7IB!~T{xMGXdK{7`L@kCBfr$y&h4Id8C z9vT^db2r_GwOOjTt`apWoyk>oEAP4>74Ia6vx_4=J%BUxgg+!4EZk5WUMk@P+C6tt zAR33TKFk$BHZTHb)x~1^81P&^5!9qEJFGdEYRSZm{<gN?01c-5D1K)l7|b%|!?7jc z;Ikig!~KSxYPffH8`Xa3#O}~n3E!4`M!k{>dn8OdHkT=P!DXzTGbJ^&F|*x2oLxdi z)A&Rypc78_(ZDuv<)xD;l9$$oFZG^m{%w<aYl!cm)9vLM&37Q7SxSDDqv_#Z%i{J~ zPk`#3U(%~3?UU8SPZ^{Gp<HrHx%wZd(KS2-4pmI{SLGkjqQ8dR>G{ch`g}y1v2+$< zpo5%Q3baR2Hu$m~rboN|47i>{<6bWcIGmNb-S~CHlfSVKEnYAC`R)Deq^4o+lAxUr zmsVin^1{gT_5hX~G-9a}_-O0yF2K48$HxKH=y2*}OsAIwvz7yeq{{Kop$7wPTo&10 zZ3m-h0ag-2>U};9doK4uU{r&61wqV7lfYZ?DAv2%X<q&YzxcGS0xz0r!=fO+;Xin| z0ZMaBqU!Zmp{30?gw-g=zCfsjTfsjvrh0kHYb|e-7bQbMqlM!Y^8@73!M;WELkjG> zX*J;1)sp<JvhB1J;G<}rMtH0wJYRHaBSLAzVUG(96x@~l9en>kFhrjXB42^>!%l?? zxZ3!ZA|3O=$@rN;9)na0I%U|}+O~4wHhYVW2|8rIrhncIdiLa!XkvywFB#m!i89}g zb<5tNdms0mqx4&}wxU?L2gfg?cJiy#H&p1e>ctf4v}ZPoJ`_nj7Q;3p*7{H_8J7tz z9Lh=(lA#HSQHJhfKccz>LOrvrf)_$Dn>RRgNnm|me+?8Nb`48hGw^BWjX2WyA7BPq z2sxsnrV5xm({C)UB9+z!GZWxND8nZkcKX}1`_BzKA{Y=@l%k5wAXjklS}W!!$8eaA z4>n`H*{-3Yi+4#r^HkU$i;k^#n7|zn4a;!ZGVOoEw@@j?bBEB#=~Q$@{Dnrbts^QX z%}2}~*SL27*ERp+Pd0HV3MPXXyYROG_=g=61^|@Vz@27T{72U7uScS!g7Qn@A9VhL zVgBKlpvWEtlxI^vuW?oSkNf!RH~u{M|KHZXg!ligw4xywzX;V`3=OdVQlY-*>5ZH| zDUS`mX9QSS*g=Xom@vfH5;_)kGp>^n;0RN$)q8tax4CPcAKgm6PN^F#KlW)HoN{}d z=6a4N@7s8bxVmX3XKO|z6TS`EaGAxw#vKkOL#D_>U+wGbyAPVmA%rL3Sq9ylneI(A zKNRiXPUYmd1?~quVf=Lq|3?2Li{c|fJc`k)|83lVpvHWIiv)q7uAIA={MHV(Ex6oV z)l(aSGgIdm|56vq;NXY7v!yQlO>2;Dz==5PYx_+cDe(1jL-=3Bk_jRv9n)tnIO#F+ zROl7^`T(Em>Ij!!HhBG={1j6PW{*L*S{>~F2mAB)U5ai($HH2>|NB3JjQ^<O(1Y;Y zP%RI;I|gn2uQPrh|KJ3LBv15vTK;Gf^1sGIST2-n6nngOcK%;y{60R@4h4sAn2q)R zUOW8*1QP;%MvC(ffqr~^e?9i!$=`q5Inak1h5Jm_+W&6$`L`5)i}1s5Tv4@Gh?w}F zUgck>UNS=^5ip+h{dd^ne{CkJ*iZxWX_Ev!@qeEb40o3al|;|3!qNYmAlN=Zg-SB- zgn{snMAu)2CP1OD-f419;E#J3<qQ!mo9INR&qT~lkmmczqmh9RItU4)Yk0|C`HOwd zEVQ&kY@FGT{Q4*VtIi#o38UzF#8UfZXZPtq96773l~YEUyCy<5qMu(@Zs=XBUl#;G zC_VZqsIpNx>VBOBDzc}sIbBmp=1sB@5I1OQ7SkOOpj+U!5fd5zqpqRq!oV^BPDLQp z(8ivZ3|(KM)v|fLwCY#N*nhvb1O5<Irp|l3c=gY9j4`Uj7U+WsNOF9T>!Sw!R7^W| zpnSI=k%<OHqbbwgM{4pfx`WC7FsIg9eQO!lgq~D7W|$vVV#x~fG>@A?v#C(bT2J2S ziIk^P109S)n+<}HquI#)H_&9|`N9`H3vG>O9KXe~{^PrY=L~s?8>5mhICo@^%!;Q@ zb>dOQB@0zXb(r!Ue#a1q{F4ju6oZue1*)P-Easf_1eEfaD)CPzoNRbY%qBHNW`Lc3 zOvl0yHAr^ou;4Qp$H!!ut52r2{klNl)yPEHr$c!^7j;h`1_fT`)N8$%p(c<h+3Z{a zOygi|J-UAkK+<%ti%U|z;Kew%EKh-Wrne|(N}n_N%+i6VxjAKSt^vulHZLbBsWIfP zD(JXse1XuE@xN97K<2l!JNG)=zj@S6k_ZJo{MZ+=o&H^UM>S&&w-#Fidpnv$-dl!s z`)c15vGdKC6_SuCam2;8U*j%NI5d2?-P|V?sR7?)|GGPuuO<sZDa8DFmu+&)Y@3Xu zb9RHT*m5<|h#pIqs}9~yIMJ#I9|RG+s73{vOEp=%#l!<);LLkjgjiCsSs?Neqn(|f zr!+7GuDGedZ_Lk!(_rOgC|M@`UzHH>Z$rdf_Ba>>-5cD%kU#8+3>>jz%2MzTyHl|$ zL$iCl9m@MNbPF7fmd0A*s8%YXuf;mU<lo=vAg6oQ%tj23nidvBI;YsHTPI5G_5AV; z5o}xCFf9FrmQ~tlEcv>}!<d1AI?y%H6TPkX`c=)zOAp5=ZdsECY-o*#JlP5Jw6W#5 zjBt3uMTd5q-}s5i{+ILrQfE{{m`4hL7oMN@b3f%e4WF$BUAf|6q>Mqkp9$<Asw@O^ z{V?s!Ci2=1dm>j&1c?!=D81^DRSXxs=b3~<q04Xg^N`*Go9Q0Tf$`cdpYH==3HCd7 z-j(3w9ePCUl%+$<FNMe5IDV5HAMoxyNyIN#_z}O^%93K?DJp3{$EoH7A?kY`#U1!? z_bCDh#Njlekwiq!<8M;awHp8d^)$D#Uyyc{6jLC!2aRa|X~;x_00;Ofz~-evsVw== zLe>kB6fW(JjW40LFBr>4a_^))`F*Uo?FtSRtqvx=Db1zD$XaWh>KT}&p2F!@a%u9O zE2ZZW79KW2+BzXN47+SI#N4=jK%|9|boE$o%}i#DzUv%JFG;H;X8RRR$Bg3g)v1Lj zIg(og)5jO9%`3iQyh&O5RBT*`-f?0-p)vhDs<s9|biEcGw97nG&-o157q#D$*}rY5 z9>3+?Nmi@hJ<+;cx?j8MdGkq#=(imgeFh*7bzSk2kJ$%v-*9#XnXqzjTEIoBmeuQp zReqCaN|cTdPigRMz|3biMV9V_cZ}{I&}6UEUU7S;*AzWep||7X2a2^Y<*Ev_E6@~# zGw)CmW-}FbG-N_qT&`WZ8R7Xkws0ItJvqEGOrNVmTtN1yPIVob?IV#Ltbmz#RxhR~ z$~>hJRk|UAo;6gegR@O8$cg@cO`|KcGxZcM1~vK{HbIh1zl&r-Ol3Z3!N_N0lI8w9 zNdKs#dWn9$8;elq4XgdWu+Me?y@R=XJEo#6#-^GYp)OsY+sXLKBsN+!nXWgGJuTUU zFXBP@C(xHr$PX(OMOeHIjv!I48-2SVaT0Ae%X#^A{EIM)?2<<RM}w}`U9uXpMaSyz z_qdekgd*5TsJt*NgiFCR@fcnv?mD<Vbm-ESVIo5nI!^}NzCnA_J!cf}D5@K_f1GaL zAoy<fSh>u?E<KD}B36!%aNY=pJl1~Bd7er0{yAcyv3ERDPAa%rmCpLFxlo!Uq}AK> zh(@PNvFq}3A2Qtgh+fc1=ly<dvL+%)Fs>pKVN2zRWQcLyes$q6^U?#zJ{GHOyo%Dk zDZK>8l=p_(E{IrCEl<MO1#TIFMwRX#yLi3B-+3-|(RGAn7>JC}BtJ}Vf%@H156N%j z?NZLz+q2JiMweZMeA`T&@ZYzK$g2yI1BFgY>zP_sq6B`9;3HMKv2k)EnbPo+YwTw= zB~F}|MkO2WxWlIgIG@M9r(3#VZ>oNslHLQ=_a^7hQHD|L@OIyPh^GZMyYLcaZe`cF zq?4rdE_9-c@T=qntT^+tud8JR4LoZi1#ZXKvZ(@1Z>-8z_$A$u&fIR#J?e$*pt=dQ z63a5pF2_w44lvmGo1rmr`jb-PZ!OyS&n`ciD@s;DQ>>MBTW($5oTG*|ih7|=4E4t0 zayrqfJ6`TDY^VD{B}Z%`()NTD4QJn-Oo~@idr~~Ov<-66AfFk~ur(1ab(v0~)upwz zFGn%<+sDk|kx+eLl$}7aBce;MkJxzDg4s)R&s^$Yvut<V=AuZ3<<bBqNDY+A7=|0X zbfZ#a)a=qH@C}q-6)?zCkYr=y#0rE46zH4(H7QCHhb%0bo<JnMzOYZ<sw6+$?8Of% z#gYDSE9T>3z&hVHZ?xe)`^qz|b~)Sj9+3G_`omAi{v-@LOiB;LH$Kk!LmwH1%TZlP zg-O$cRnepk3%i_bTLitF=r?LZDo=v}kRqd21*??eUZ;aYOWW9Dw>X*U28E(hbf1*2 zjOC}8A&2W?&zLV#$oyG(#N`YJ%98Nv1Ow)&JBf2tRK(s0oePFVBW&?WjbsyM;+~nX za!o-aU&$Tv($nLAjE$a)oRYpLCu^_dh2c<?Lvh(RyCv>?*VG_4K2YsMd<hA3#)U^` zEMHIOH^lV4e%L_htUpYg+(6d;R{Cwx0{l_CH*}pK%<n`u^aBe~bzhyfNFRTc2;OGL zOL{}6rCk!86lj>?4A8K9I;ww9ti#Uwio7;3oe?F`{C*011L3r5$WA_c>#1i<n1!NF zrb&S=ngc6_lA4vr2PXMPBf+PAYBTG+nq#Md1_=Cwk}Wk1e+^&9KBWat<*UoX#?xc3 zEgctFn$ebbHN=|loDdTAiG-y4l>h|7&{n`PE7CzbS`lRQpI?f(wzRldk+l`IQC6%~ zObf;y<s%+VU6GG7bj(og!S3`g7?WO0&;84_lqeAWcrvW&bseB+c%$*lp2le)5ubD* z{J`$a5X+JD1O?RA!Q0h)>@$@0I%Qerk?Nx{Wt@b*3&goH+SWy<@!EGNJjOhm2fkx_ z+MKCwawn&<NjJRowlzAZ8!k?r36bLv;m(7k9TD%%6M(x?f!*nvpeq29TFqbNR}d^R zt5qYJt0yz;)3bXVaCGJ9H)bMppWM#E^y|9~jElsWOBeY-9gxFz%pxa3U%Sk7t?Ofh zMQ<F8KFh|65xR^|YdV6!snO(Ub)JYzdx#6F*4k&l`L{}0o;PBd<Joh2Uwn~ALSM;~ zwT*!<oW6oD;LGZOuiTjFb)bIY+h*L+Ou1(u21Yq71z@_6yO#P}@uhr^eak74Oz9$N zY+`u{KjwJ*yy{)ap^Y#TwCMMM<fYxUSQq)Tv>Tfu;6^)uXsN0Sl*k7%W%0V%{3eb( zwGq*4;iqBK=!2wh^iYJy@37KNS1YwwSnIlKKsG=zp{hpn^O`}Ywhl5h^)(Thc<Bib zo!|k>Dl>R2l^C6<swzEtLK0>$Etrv}G1pMp@&%x_j|-Psp(~b`IeJ*1@FS#3_A!-r ztT*Z5CNTG@6u(h>U_6{*&O;M&IQTqLPP^&m*@83ObLn7ERMHiq*;LAPN>|wND6<5r zX}pM1*>3dU#KJ8RUN<XgA~v>vWEF(0CKYgB(sn-*3*MY;19@iSKgSVzgS_Jy`nnL# zr&zVgmCds8Ng#}lK9{%;Tf9LdBh+Jekh15{?auQrkJDu1%Po%=AD6n0vC6_<MBrCr zR`_r!P}aNo%kCW*FwHglZwx}Vc&jnr>4w+DNl(aO*Y9^7%l1CzXu<9)KqtK(vaPoQ z1d&e4@Zhi+oXViMODU0-Wj{FouREz*5wH1-gcC-<ZQv2j8+Tt$;mJolnu(bpyF&ln zfHmgMk-X<Fyrk2)+Nv%z7yR=jFaqXXl=k`D56vZ?m3R*n>?>ps{PVpa;mA?r=ht2y z?|Pj3Yav;0DbCv^oG70X6CI#!PaZG$1wZe-mbB}9PtAg}gvH!GV}C_ZB#h|8HDv0U ziiP$nO7(-~YZErUowi3;mx1}g&mBjfdBGOnY+xChm2FT|@-(w~_zPK@39;zGu5mlb z`U?ud9ZlY?x{_ucM&t9eaHwoCvQF)>WK{#_a-<S+e(dfo7_A^~_-6HVN)@CwKWG#6 z+}bTfPiwsfJ;7~kvj;U*040c~PU72%S90;`mP1W{Ad;>W&(Qe3$q=nN>O5K(cSlj2 zXia{J#mhS(W3#<=R}h`t_non+AFoTuzq*S`f=#jVmZ<T(9BypX6y_rJcFtZw9XeSG z6YYUBlkA34yzi-60}bsi{D)Ga<!nFE(0)adzAVy;n+wpw3<k0xfCv3e>f6A1abT0L z!0<4iL-jR$@kT5;h9@9m=SQ}<8M~3oJd*ysDhm5M2mZp<Yfb}J+Kya41HFjkPl%G# z<=Z(9DwIxFNx^<aBb;6z?VREPQH<O~Siq;cB2Y(@zo54?<LnBf+D(U7(ZH3<VfY8c zpyy+3XwJOEzPfDBD+{rSh#D^P02o0C>7~GQdU)J)e)i`V&D2oSjZh^lqISS73_{ZJ zp5`n_Cl!yL$H4THi=mMBD}ef2j3rkb4vL;XVD(62`|Vuy0!Ec7OQ`z%L`7ltY=~Qm z&qAP42~tG5kfj@u*>7cFBMvk|nA~jE(4JGOTL*}bbUo6ry5lXOmSSdg;Dv%Z?dHPE zi^sY;Blu5D*6(~?ev|(KV?{Ik)$_%7skj=CYd97%yk;u@p-58U#M*k1nfD!c-AIVh zqAg1p$3t2_D5tk`^fS!$s9h7sj`>}%+51c2X+k?jA@e{Dj-mL4Ce|GRT&{Z``=ISE zt;`09CzFb8=X9QWIk^GVmq!_tR^L~Iq~KOmOC~HYCvhpEi@$mS=#{<Ad{2ER;b<ki z_q8AhDU>pep%EcxW35JN<o;CLj|hMV)*nw@>B8t<)^}Pxv?sOwRurD0T2vDm%1MFU zt|UdMe~$l3oVsI%lm_(fn8u@PKXo!88XYA<iP?fB%HL|7s*HAQ7)`&;o5ZeG`e&G2 z55|1!N4HxNi{n?S!W7%_wJa}MKX3_VqesgMmJKSkEFlRRebJl=yH6soki9|X%7tO~ zyx?NW7nmwd?Rn4mWvDKVpRVQ|tqFX%WRmpVz-_d;UkT>|jy1RszXM2B7=W&p5vJIm z%@zP}mADNKVv+X$liUy4Alrg=3N6FYtx>`mySsxAkBIot_C0YRRD7Q2g}yQ`=<VVy z<=SS*YdgXdLjiQLdzP*3dWT%r#?)9pPE8AkML2ECU(3BtqfL^O<@G=m-me|C3S=9P zMb&3t^xuKnXj9HlQ7|zx5@ksBVD1GfL=Au#KX^ro(T*0)%rlmNPg7CJ7iEl&C4?f6 z_jrohX7-lCMv#j8Ge4|k?Yf!VFGx}h8_V5B*wKs^*PxKqB1{z!LW2T!^f5&7E=cvN z7tH?iae*PcoT*QRHpBdV)Na%oIm=3c`7*pASsmlRDc)K_uPi=e5rng_gPt%g%f9cS z;5HY{<BLX0R*12;x9VG&DQJ;-h%ki-WDf^<C}60~cp?^9zcDys-3zQbff=cn^4weD zad6uRRQ^T$)Y>c=ShXT|xW@Ye-}CXc!qcro?bNAfV4?>erzCIYyi3P;-P4-Odgmft z2oU{kP#4UmpUiS$H2$^y2t_w##y1MWf`O#guhbv7@rk;8y>HF;6gy08g*L9+{OhlV z3)QXLhTtgY#*Ystg<Pw75eZiPNPP#rjaLr_mXtF2l)iKwkXyVqvohU|#kr;9zcF9l zzqsj+f}N7+#q(<?haY$9{y3><d+|%UtD@7+#WRhjiubL?dva=uk!Zf4fw!fm1HUa9 z`5VWQ$x&Gjh9uKpn=0I+ZYfNi-6R$0Bz-x}Qo#N4juiwQC%vS2m7a@+cNqGC!fibU z@x$$#CF`NOw^xo`P3=I-f-1fA<CTPg2*yS$FkKRd<KHXF%W%=K(wEY@37-939YFU= zxF>0mb8oc%j}ShMXlLlh1mh-+H9!ao1DK|lShSn({OxYNA&_0uo619@B(;2xEu|ZG zl3O=;jzDS(XIZ#C$-`w2iWhb;jlQ?vv6liDJ=d5hoHce(E27e)fkE@H%rtkQuIS*b zG4q^#88CAKDZ2ksQWH6yWV0A)=#KG`os5N$;dP$;sIeMMV;L@%G6ECjY|k_Nb>Jpl z#ig|M5)$XT9?rV$3tT9a#N7H;X0g@d9IrEI6>qJ6-lX&*BLxQ#cEnuwigUbimv)kz z=Mvc;+V|u@0Y)jU3utm}8P&3W4+nmG5o*`_mU;p{TKgoT+43YVpg2>KIbGra5_8OJ zuU67UaO8Z(ZAhji<>B#pr&`2Q>*Re+>1R83qxJU`FQ3|Np^YhTxHs0aqNLhKd|$gb z5h=ySz1iJ0>~d4a^C+ZKnzG7wT|KZL)Y6EC|56~XNHv23N9^fQESdTX9M)FZ6)H70 zFR2_xNDlBF{28B8yWnuH8WGkhg5RjlYr>Tzw(?UWykaeQ_*K1m($F$Vr5wsVd0Knt z&@~=e*C_2`HR>OZ8%CTn_Q{*!$ZmaTu#BB}{QIPgD^O&rNHEv|Ar3@0HXKEp<~%nq z>V?vWpSCwW8aj6FY%n$urJuA_(v9aQyq1JSv{!oe)3UaX|2;AfQ^8A#LCbv15Wdn6 zB=|jiA)jMVge0von%Dkr9Rp;5p;a!7tR0EWV9eS+XvDVp<VD6~74J=;GPE;>?E<Nu zTgo*I?W|{~SQ~w5r%Tuquq_E8a&6oWZ^ajiJ^mz^wat#OFBdG6NZjh%lj?A#SHQ+g z+wrTN|GgIrcdvanF88z1gYPh-x4-of;PL88#2@Qu^D~D2&PNuU10v5_;~2RbW?QPp zG&jZ+pibwiPW4jx!?*(Qn#Hmi#ckeptV<&btZATH&T(;`vxd}V#M8<*Vm%KJO^1j+ zV;vmd)7&}j{2Y_}peg&Y&Q)#k=y2*}DAi-yn`=2-g@aW5VaIOfd#_G?dC$OV*;F03 zJM`7h*csM$`?lFOAI})Tty;9_rEux{Fl24nIP!+K;5p_W=UTl)a*qk_K1d~|pr+*U z0HEd=+lW#9yhS))*+f4)F0CIHsZny+%q%5dzKvK($v65T*yP68@FVfaM6N9wrD?y> z=H4pOJZ8OoG>p9kb7>HhCTa!k)xJJoR9p5lo9W@I**pJl0WjPhU%r{OLCpIdswK&^ z#wW7bJP$c)6c&P7SxCkt6QXt-=5KW<U>e^Yzz(Q8&pD`Hs=qyw+VOrUU#`tk^Zy7F z7+A`te)6oCC1p!PO@FWWca*(biH!D!xXtMqacnyc6JRy6ye3G@?~01gf<wW>fin`` z@hQ3o{k)VClMoy={*fATeJH1s4p@o%7Lx$Ho*G?KzYk#&tx%pKM`;Z({7BjYFN+G6 zq2;eu^u=6M@Hfsl{P}SPzc_0K;3afW^Fq+q>9eB*-aNg>J|4*0Efd7aJQ%I{{3J8s zl@-H$u2smsuN&}^H0JsK00pD;cH=XH|3g@UPLJNBNTT33SJ)k{(#zCw(YL#nbr<M+ z2Y^1t=JO9$##fCPp1W@-QI<NCfQsiiP)BDI5t468wh$P$uKlLqMv(E?k#?m%MW%F; z&s$5`h#K)6XN)<8?^e&%_+!^g^^&rL(c4;?ZQ->V@rXlCGQ2oE;OCs^PJAj`R=sI) z%R`DrEcS0tdIgE3BPLd%Slr8_Ew7FQ>Q)V^24GR6ql|uG0=e@MzeN19*EJyA(_m6Y zcjRssWzty-5v^Hp^jjDp5RzV%ahTek)RLB&P>5P%|I|Zoo<_?x&F{-Jj~3v$=wj;@ zGy46^<m)MYQv0PX&rxjbRW<9Ljv}({dvZrDKzxp|oNTtVm$!2t>HE2DeJxaZ@fr}$ z)EKT}9^zEYHdVf4$PzQ_KwG<T^5-7Jw2wOK6)J;(5h};A#QHxY{2%a<@+Ze-E)Q&U zQOGdQ&+*{$!nTgKQH+g^xwv^^7M<B+VeNHN0A!~0xvEBbiKn$W;h*o+U7KpO60{3F zg|)dU7GvyezB2p_;Khyb<fNgMwrN9F_K#c)(vLuE|7g|}ZU)?s;AMN1!~&Opq>x|u zG(Li&(#wQC9M=lzR(ns>cyww8%TN>UNO_aS+qfwxgCBqNbH(RAOvQ0@<C&5@LgL-| z+~#J)W~YV@R(i<I+5L^<?9Yg=VE1VyU&Q-gnR5Q-oQ{D$DDTksDNXXjKSsME1DyoJ z9=+UEPe8Y;C&Wc<+b14LlyW1#?oN6^)ttL`#2Hg$B;Ry781QMi9cZAg#1rL~_iZ{; zAdk$0sW<X+(7azGtv>9$-_re9JY=CS?^OnbHbqVA;lambkw4xz5Y~$Rxm=6F#d;G| zkl~lO>9;<84z3fN_{@z(^mN7p=DaW=7C~#yKabW&li{>xRMX~XW_usU;fui!`dq!4 zzUJC5Fn#V0Jf2UAm7i~f!A81F4-RL~$lj16sviWtB67A9r%ALn4wZ;V$j*tC<>ykb zthKM)AGb}=X)x(oO<OWSl@JeoA|IP_HNNuFA~98w4`xLumnk!S4D}^u1)w<<vV69- z{>tod{LyqEy>RHbyB{nW_^^HKz2AMz=PTxhdu=p#3*yX%?KU7xAAYI9=;QqO$%a+P z*hcHk)Dk#>u33xB|AJ#JXq8D(4l$(1+nKV4dYF%loc8JAjuwCx`*(`s11lorODzzA z6TS3917fw-M)Yi|tN;0M)>7{?83%e$45M}ppP7Pq%qCxayCgZo5b2ditGDwy&XCNf zu+VDk=mi6JRukRi+9u>;7bNI+Ybl&~NqjDaD$m(Zyf4vqs$wZ{RI-*_a?>)I1aD?b zH~Y~m8@I<!w;~?zz>aD$mC7MYiEEEeTbZ;X-@8Ke^T|lPr4Nr7b9MGMBRop=e32Ra zig4wFT&L%*ZiGX<oPzM8kTu_6HGB-B#$2q7I6Zkz#7hL9D}Vi1oGFhozM8X*zIH*n zN*x}dkw!37$DjEX?1QoS6#u<#7~<jfhZ}(alQ6ts^KeTJA?}1Ja=q&A<HAAwbR*ww zeb}vj3R)<;?(y<C1-+OsXh!l2Y9f@8Kg6`niA3Afi_^t>E6<>JW0kcD-0KqPFj?mC z&^poWeEfX+yrcZE7smMs+X>u(vOlI1PnXiznpe{SB#Ss7tFw8iwOcWni#$w%a*2$A zns`n6PX=v{FZ4AP(8%dEhbdWP<Y<pq2+T$_v7Cp0*3NDMtDD$o67)_b97fbV;Oo23 z=~=#p#(|J3b%6HtnPt%CVUcqP=*2p(3Gz8y^c$DLUB<?;4<1NaVH%6pW7GULhFo@w z1xwT^(Z>rS@OLSRbNtaVnPer(Y`pQ&Py0ZrBj{hC{)ezqszA17J<j88gf{ov&5q|O z<>O-!$Va&N-cXO~toI(9_a^O3LmbGz_NH9hJ(fEi9urOkThG5|Vze$A{-}_WsApx_ z4(mPXdv(h-%xXXxH_SgMIBy9a&KiEZ#P|AUtr>L$9Oi9mSEqy6&wFAV=Zv9`hJOu? zz}S@!BL{e`zr`nH4G1Q_b;)UXuwo*$zWC&Wb=uV+BRuq^<G;_0NG}i~KWI}HAGdJG zHIGBNUv0jWrAtHVV$1~mMI`I6+xpWqK97ox326f8Nnb^6ME3_W(L#esyw*%Qe0L`R zx#zv7q%YQ4i%b!)<QR#MiCnPFm{AEtObvYS4^Y+L3<rn$*C@1^PS)sXEFFTZNg}43 z8P6(`kWH#v$8RzM6`br=YezRWj?#S;Qt<2|AI)gmh`+p7XSN)cpD4l!Y2qYYYHD`q zAa*c$boc?D(@cDHI-bEVv@_p5<b1bRn|@BHM!&rZa_~1A&jPmwh+wN6Fm&BdUU^fv z((`z-jouz<Q7F6L3}C-)c%E2D%3y|O9o%6S7KD(vMPsRExM9@sKCxl<ReA{zK$}AU zxYGL_@bUMOe)H6Fg4MO#@|33S2`057XuLAU-v&82bjBA3aDM+t*l71_Gi6gt&5f6B z{7nzM9e>Sr;t++GN5CK(o|nBIvPD4nB05eR!;S(pQT;NEeyKyY09Nz+;k8h--^N08 zlS%qp6ipj5Kh4_JBVQmaf%^?|0iO>%FN+rfhWN_79K?CC2am$#xr@FSF$*&Ye}_T- zoWG-vk6rVqjiX(3lYhP{C3Txu;ALqFBa9QZr+;q5TfC14gLc4Rhfa5G_fqQqj+J(! z^ky&8a$PNx__U|n>7Mzzix(k?7*%~B73HxE#arVh!1!oVsBtpl$f99fJ`=TGyNks4 zgbQe_`d^U32W`ZI#pwX&3ysW&qLKFJbA5>O{^*ytUPfR%o)1QNc?pS$Blm%L@tMOy zc=5agMrz@f@7Ws^yWpLy4TfY$1HuapBBuq;T%raXzbj(}uIkz-M<s_bJNaNI`Mttw zLU(5pY}c++IG09JC*Q^_cXDUTf7FN0P+uwj$WMp#%Wlp0gkVP*3YL_Pm<~W>1%KVH z>a?H2!C1I-cWg`SXcb_4&p|_7D;dN>;f(Kj$pWqkprBh&XMko5-o)mDW>z8d4>wFI zFNPr3Y#5iU;Kq0DNPPAS4YNg6t7`&wChZBtE=$8^GciN9CNbN&=GVTCo$%`-)9_`= zsQodl43_N2^=Dv4YZHJIUbUui|2!t=3jM=~+1Q$MSKt|P&9cndeyIIW<V#DbS!E&{ zvP>q80hd>=4Pe`|K<m#w*rnz}Z#(4iDynDe#S?q3U&r^;mC@X8i#rOr3NMEYP?AUE zRBSh;$}hDr=1Rkj@n!!h`rn{OaIC~n%4hB-lfKi!0Qjh0K^R#BGisbH*@pq-4(scJ zQcG`3YnHt+!?HPmbU*E<*F}M;-zY<XPS)JX-+!g#X(xW+drMXF+qM5AJ$i{0A`)<8 z8Xt5{Uz42@EoFDv>BflMGvxGiEB4c&UPgx8ll05z-7+Vp8$<#Jx^`VFL^{3gm(6s> z$Hxc?H!vhSGfL?A?GdvfBBC;+=tN{@3)$%rF)=tDwXL6E;y5r|s!2mGEwi;Bu+Vq3 zB|1a5x;V@c<N6~rIM=!(RSY9MAaJB4OsT#ZdOrc0Hf#~*=btrY=hv)4xu<vBc6*|g z5T$gVaB=WEw4%<X{8zYjI1Qg)Dj0l*`Ef1uh3-@2q+X>Dr(FS|M8lSJBk!jTZeDoD zKXbo*$y8gED04mv2??%P+E2#5KRlMIs?QJ1lfd;s>+zGUy~x!=G+HpOY3Yp9+ws^5 zh!{P=gxDomB33xAQ(1=!OReUs`ay#ttO!YK%`Om?G|qo5>j+F>jJMD4PlmSFi6gj= zu<fS~txA=vwaurAd?Ay{)lD$J<%`M|B1X;2zz#|4Mp#80)-~Rb<vm$@DQYRcg2rna z3+=lD<>bAQV8sV0ut>m!DmtWEYxC4Vrz%+P4BRF}Lm|`=*>E_Y$VfJm6K1axA3Pon zEv;Ckplk`MGRe5*2(!Jfd2H(P+<xz@xFRX&`sR${;Qjyp#JO_@T>7YUX?V&7xA${_ zrrFu%rY}ZTtbz$Lvb0AlTHw>vs&840UR9mklkIDNXR;#>n1n!8mjv#-+04H^mcPg( zC{&TTvhyN|hsBets%oVUSaqSXEJ}@w%~z*j2PVLz3U$h=8VJW^HFF#1_Z&4E&-9o2 z3+jZL)SjNE>*nxe)si|ucx%Lw9yd3^L&HroYHf{2M-NPc-cRMWoiQ!#dOw0VXZy1^ z8qn%*tKqD_7g~Sre847fw`P$p71{8&M&Ma%dPfAxJGIAW)eEXWU3>}y5GDWHef!Fa z#`Gm5G||;Y4=X8cXft_5e6*=w<A&2z@u5{G^TYyi{s$tE+-E||b|4Av>~+DGAFl&; zOy>Eal~4?>*9-B+vSfJo%(ty#r>|;QKmqu_$O3OnRg&SuqJO*UpPA3sA!4)uqJ$B= zVrl_o&PK<pCZ1Dmzfjk_*ol9@&;AWDvuQP2!GxVn2^JhG9{~LwbNc&=|7WK!81q0| z1q>w_FuqVN)BmAw@GvQA<|aH*eVdzp`B&e~YwGJ|EI;DzS~^G^wbYF-{<lE?1${ZU z!re(xX}N?RU;E-1G4-stw7nl)8gxizgEF=znAp*=Bw;Vi$&gW|#A134<>VDfgapt_ zDN39>x>^{ObfVyVBZ8BXk_;x!(jpeAr0e_J6?ll=BshN=aJss(1+|y6w3<@?H(W#( z(l7SGSQ7j_?%UPKJfR<umu?m+S<xUYL?pDqh=(;6lp$LKc|{f$7Rh`|Vkz*GZ|M@o zywp;iZ>JylomtbJw!=bCH48fDf1x$M#~s>l+N0el-qGd*G|K_i;SV>6bV(C+zL<X3 zRVxMwCJ2g9sK6GsL@0s3z23i@boz0ZQl~rp%2R9Y;&f{>69MET^7N+Xz+t;(@d!>a zB*gHV_~esY>vUDXhL--S8h7SDTov&9su-yqt^|MBLsvw_oCv~gfXO9~`PlQXcpU$I z(DNZQ_buqj#I$Y3ZjF34Oz#wCx?w{8K`{C)3Nt&62fJf77Olkp!`L^5SC({bcgN{; z+%Y@sj_q`uj&0kvZQHhOI~{j;V%s*plbLzuop+x1$M@%4b)9|oKDAffwNSP0RTTo# zb7=E7I}Z+lm1fO7;`OS*;Qu{zzCQ2yj~7f<f7DR^Jpxv--}tyRane_S*Ewg#VfMr0 zCG{y9ofEb()auC%52i7>Riob)?d8HA?{1od$3h+UBPWFQAPiZ-^7V()G{o<_Slbiu zL&zlB2E$a(XNrq=hyXrG8o`U%Hd3?y6bt;j8!y6M1lFEY2AY4o>u<2Kr;DLx;2EbS zR8XtWv4z%-bm!xdo_>l{hrY7TSX;yAneZ<VJZtVCTuBza8L}j0Nr_pCRf(IvT(6%z zIn}&P*}q4oRztuqBL;Ws;|eT4zHhbBpE{w#SEK}3je)m7fcxJBMU)Ie;%-etb`P!+ zWuX=yv4{g{sE9$PR~K)u0P-e3D7UK(%TOsd$^y)eF5#a~<?-eNq!yX*qJNvvzZzOJ z2>XjMJHFa^*r0#RXYNR7Kk-gj(-_}A3C@zr4?Nx4UbLUPH^vI&GiaCd^u^+(5C_h` zh~szIP{EK+#EZOqD|<L~%=Uq^Q^3VFh=~H#t_pe?KMbJs{KBPC3ATy3t#9zPvJWqT z1tYYM*(I#fIdA!p5`@gdD)2iUh_p=h_52USAbGOEpMx_BG&*d~mTdLq3tNBo=06JN zeY$sXBkhmc8{rQQbo`kg<090>Vgat<>MVdo?!2V&n<MbdCSkw(`=Xq$dY@hw9(d*} zBG>b@ba@YGr!y}3rr2VRQ+j;samn8J;i{W#$V|x~Y3D22^ID~xV)lLG+6$^}TX{!e zNU}>fw_$aVv;zWlhX&l2CLM&I>j4Y8d2e*v@%9{H66TJ}Y59Kx;P1u@uxgq$7q<c0 zA4os|i=#31^D{#@(@a#t!cEQcw}C=~*XBlZXU9$bBq|-LtD178063bv9vmO$EO(T1 zBtz;i#<+Dmo&?jGtpLHp{xz6(V^SstD!#s)Tv`7WePFcDJyn6=2g0vcD;OIH7p>#W z|2U)nu}(r+gYQ|s=wD&~5ZF3V7LoE(v-$l-WTW$Es~@lDmoMn)9FWEHBm_1~a??yO zQeVN-XhYjk992#iewCI(Okj>JnIMU|OJq$>230sckGJ1^EK4att*qU=0x;_iCLc-+ z7MT5Cs>T8&Oqio-4KL!43|$u?{5tukm;24M`wv*YbxA$oifJ+DD2nx8MuU*_=T2Q# zG$$0~$N_q~6uV70fqd3i^^ZEvGa*m6?p6#ARhMbCM+=UedC+|VgAVIl9v5#dL<pq? z>eVZawYm2HtwP74e#bV^^HRL$HhXIeasxbjqgcT2*V9mv5Ci%nxjsGJVvDKOgUI^o z_muN*j`J@9Wsc}lsp`l4V{C9{7Fy}5v=Gtpp5l2>yjRYULx;1@zYxXZj>H?dTXz5C z>S<W#;j_0H0Bx{7jBz|t(tmtynvKIPr{2>orkJL4Mkv#e{O_3sO!8)aPx5|x+1LkO z3xNI@*!xja6BM&`z50MvpLlteUH60QuA{yxbm^*IXAC*P=g3gr!<+}FS4qpw?VPhv z^i&^Jregms6DhTm$Vp1fx!k@JV+zLOa^fXm_kS*Vtc)gR^x3|?ML=34c_m-v<Vkae zh#u0WzyO><j$=;>3P{YkOeMd;NeFxS;U`oP##UWm#c~Wtd9{ruS`85n+RC1^4m)TT zu4iR+oci4UIjjEXJ+!!?<P=?DKP?>TUPQ+>c5t~K-Dpqazwd?<(rhi}7gbclE*M9K z1o1|FNAK_VS4VY^*J5_D!!|LSZBo-%EVP9W73awbs&>}Y7p`0I7JgS_juS<LYaY*< zEW6k}hndfK%ZZ1P>7Dnu8C;AsTbRP_DIlP&Us9XM#pEb-RL#v5OKzJatY`!_aryH8 z7+lG$P6?VoP9ce?zrY{x{i6#K{4{gIG>}({t_H>s%?uJ$8+LTx;1Q`!PCXhDLhcSc z5}K+Ut=6h6*Z+WHU{*18!&*0*h3Yfc>y<2v9X`w}vZ3}a3*uWz?fFOW&5oYco(d%U zaKSgKvs_9{*EXrAiuUlq=AYpIv-$P6|M8g&<eCUq&Jq$5Xc~UwdS{;sFR!*GoeATA zapb(EQ4ka6u9BXpq~h@2^SEGlqm?=x%jm`8S^Dwg2f>%8Zv3w@Zm{aATMEu_r2>a? zpEqASgo88s2#xl(<$&W+kpiR>Rbt{`{pA651-+LP=5HHIBZB_y4$Q(2e0BYW5yh>E z2qYaS>KP{j9TgT}n<H&id>mM_;t$2(lF1EN_I2shRQY=^=KplQ(7}Mp1Kx+(oA3X; z{MVP>-UPrdWc{ku{I7ZV5C84|`&sMEQaFD;3SAgT+(;3evV)veb9Zxtm~~=7|3Cf$ z-R%9CfcaI#h`|3+e}C&iV2BtEl-v^{Q|mh_7QlRb#%La~HcV;kp!suqy<1vT=FJ$@ zOD07AR^R`tVZ{g(g(fUx!yoyj7D&L;Y%{yz<Kq=r_@7Ti=z@~4v9Upfn!r@W?G;n1 z;Qq%G{u>v-hF$81UU54t46OGrx$mICz`$}^T53js$1(E+50!{>b93LZ>%sgV?+6t2 zJ^>Kcj)-4M-yJM&NvNX!>6SyD<Aq3*t)r#-^Dh}E9|$*NHnmNxzgHpNmV8np`ySX{ zt$$MLsn2bvLm(!P7OlBhTZImBWkeKy(d(baJEQU9H6QbT$t)`s__mJ7=%w#@z+F-J z>v%r<=TddQgJ!$<dJ$0!d5kqs$2A-7dgR=+a8Z1$I>|XvgWbcwLP5uSdLjlNxoR6R z{2gxn=}Z6l`I{IB(N@3U_;(;`1tA)l`Rr1tMMvtg?u>Uo;j{GkY+FkQiKGP`hQxk5 zQ*qjkAf?7(WMZ^eD}>3vnY&`Ss)q;Pq^i>Sqd>~PP`NJ-e7I{YiTS&PfNYcBeutMZ zzb@gs;+hcplzXHyEWtm26%v!^D#G_PzrJBij81filPAE(>e-BD8xJcPcCqG#o8*ke zssHxF6BS&8Gx~*w#27@UN8Dv)z(tK}_~&ZAND?J7t#ut@V#heYcD$_IEhC}xq7ypR z9>Ef3z(p`g8v6s|G2#|2Z3!;g$O|nKm0aOr(Lgi(!nexVCCs1w+CB4=P%}B!^ZL?V zg39}q&<~_FR?W<PXFM+&HdgLDT5jw1N$5=o+OM>nbe84RJ%D;w>xSDa+y!mjf8%1G z`ki$X%JB3DCV6<X8m{Ru`H7hB$?X}TVhbd>tITHQ)B;uz!p_&FlUPt#9%fk?9v-_V z5Q5Inxo6}x1oKzC&VNaTNr-_TMrLOX+sgb15={=E<%v<$3?Ip%Oa1AuBex6|9OVXK zILse~ixap{o6Ao$*Pi;)n}eWpF5q7d*8!1xt?whPqj&EQ(JMK#R&C9F5>F>L#FE?N zOH0LyJBPeCsTVOg)Awx_u_7*HB}*C#o_5$2(oHT5Hlz>4yO<qpH|4B*`?tXP6|7g~ z#BAS3%Tr1s{7<@K$O!7mVZQRzjA&M&|3)dq_jglz9llZZj-)IuD6qkBVD+I9=fUVb zyH3~$(QYC{o2?$ItS;7|mKrrZvPxayy%Au6w<+|nhQ4(jh2UC<b<+soEdz(5EFQZ# zxTgeMVBhKa{USmI*7G^y=K8;qF<=_Rg`UIk^9PRP!X}Ck|G50-&d7N)KU0JYThwOt zX@S_jhO$1!P{%B&a~nk4S_I!=y=N%)+axbJS`arUA6k*TJ_*u3g!hH_6@&M@k(46K zqNF&If&e9Nam=2>5vqtD+&MiVvED}!Oq;&kd+sALH(-f^e@+-~5Z@5)RK{?K+5Nwi zpZ5>^-%yHR8(trF+{qu!c*&*;3XR{k29xSIxW=ZH%$aiTbEptZ!Dsc0q%bYApfeOP zVFgzd1cZZu--?q<y9qSkK=dA;b>(k<T44I>5*Opbhwz<h6jqVS*r&Q=HO<a({n7@Q zuJ8aCSd%+o;in8cDeX;uT7Q=!CF3SQXa7->DM=`61b8$Fl~<=zbz^4Aa!%D$(2XY^ zrR4UeL+ba`VW$7AqW!J?K0AP9?A`ggmto`m8C6gt(D()L2;f;DSgN}_vx8!qm}(=J ziwf6<hr=OuVoFPs=ak_VC|#=9mnoVhAIuD==uL7=%zSGA(lA7Br*vk*VB@1uHfWdv z@VPIkj5jM3mT!3gA?XiPAC->lk9ny9n__j0-PlO`z{6sAj;F`LArc3xDvx`@Va2Ne z2ke=iUhr-&{NEH>a{~jb4K;^zn+h+&IZv#C<pus>HGu8MkghD`;1dV-h)fROP=Wr% zG+kXmLi0l+^mJSkcW@tvcQ;9k1<A;){PZ!9kkvYlX?8PIaXm-@#_l2acp31m%pU1H zvMVomGRLhsPFs^-06gnCX1Y|-VhG?y)n5yszpg||l&AaY_d#K2wloG2F<ZpA!e5=d zg7;#_g{VYeZzjYxVx&o~T>f}~4;kCU=+z+D@N=B0gF6kI3Z>QFZ?w^)07eDkkCtF@ z=cRVKnwmSb6s}hCgdjfdG*=`j-B`bt<#YGIA582q<%wteFd}+V#drT5sCQ`*(-iS< z(rfPN0J%3(yw&6=8WeND8soNx!A8;B!@GYj3%ye=!G98JmjyKtI6PqaB4Q#H)t7Xp zMT&}iNr16|m4-&A*C%_X`qFhsLm;RS3Ug_nHA5AWh|9yP^GT8?W`kqcyd$VVi2-se zBuA8zUnlT}fQp@fmW%^YHYUV+NF`F57S%5#O4YMj2wwWDx&{>)k55pacMgss8ftyj zq>IRKbd$Nwy3Pg)e+0~6Q!%o3i=O{iqEFG|Ry9<{t>)~zQQQEiCQjcSTir&+;hYfe z%QkK?RTRG<r<dQx%f}zm<~4lVs7<mfBJUxGSIKTx%v<N=E9X-FMY||%Fb4A7-^Vr& zNhuVq!NRA1<XTkHGeYxu>C9?aRN2yy(rsIrqn_h~r^+3^#)0{kgzNv1F38CZSY-cT zUHFf>LrTw3$g;qI(pqjYM@e0N*I0_(8@-Cx4fqf3V#lC`RtcCkdyH|WWmxd3bzmmN zi`Io=DrT^UVo+6{c+Z;Sh+{G9*62$t(H=3XwM(^P);BJ+W`QwW(t#=#qW+|G?Zdq2 z?hn2zA;-%Jdz84lE{?S;FT@DK#j;w#G~E`I0v;AjA$=dSSs!|1(tP}qq|mXMwhc46 zJ%rtDs|OXt4&6q@XhPGmk6ELpch!awn4m{?iHCit_NomhzAGoWeMYTJ3h;5qBUF79 z6VVKcqtOj@fNO=F%NbvA{j580>0sE2Eg=ROE~KUaEWC_BN}@z#ZW2O*O1{BD-N{WJ z8n+ZnNN5M&q}}<`#s{e6dPBUZupdmcv0LbAeX-}8IP6-dC1z2A72`^^_yrF#A`kWX z+F=z#K3R3hQDTu6cVHJzKoICZ8}Sd263C3$WgWZ04evDdVVHy1*E{${V!+goriskv zJO7dn9jgx#S@vp+Wu?g@`pV$Oz~4J7a}v*+He~CJDym?PF+2<6@X<lJs)BX5N<$T^ zzT<l!)eY{TM7Vg$NR@(uW=0@=#}<vp@wc!D*(=+uf&uTW2mp@l;tn#jofx+>Xz7HR zuU`Us>}Di}sShH1|F;biuND(`JXox)K>s)NknIO=mRk?TklG{d^TaF^(0yTSq~x$X z7JEmpM?;_k`tG@SYv5NsvW?<o7WqS&nw$G<cQPAdTc|t>Wy17lO9uzx;e`s6dMh^6 z;|f!^F$Gs8V0zbVWirF`LGeFs9???uc!T%`^_8~<XZQ>mteXki27lLvLmjTB=}YEC z4#r8eVwnz^JrG5VeXvl)DmRa)0syOzHQ835+vIQSiAnxR)^Ng8^M6fhf5l3|{c2J} zFD}szQCrK^kM(p}haGB_VQEYSC`HM}zvH7e&0-0^UX*2j3jHcU^<~*>^3)@S*8bL> zrFBpIcnHON3Be!5v5co}n)RoAYEn#qE#zwVX;}jr>SzL3-+<6g5)Bc~IFAFl0O$7? z%)U|^RxUcS=y~*ji_Iov1KO|bBi=*{T)L`+GU+7HPupAfnRk(O2vPErLZT~6)H@(u zz?C2)ew{y#tdflVjdZoAv9$_`JlG$~{Y)$&)Lc@lx1r{U!TiUBEkY<Num4I!^ksH7 zeoV1}9it5?$k&%oN)ezUbdr}%<`B2l7kx=3_a%K^<(oZ>z&f2!5d<E~Ayx`n#xzCH zo`nJlm8hDwFH%<u=Cu{PiZZB8B#!y%pmcEBXl38k6n=sVz+RsIs}MM01r_3izGB8D z(|B+6$T*;RFuyZlD8&)!=jAm=t`ota-+Z+FcG1!_z8vaWDnj0F7e5FmVHTJU3g_nH z(pZv<ug6Yyg$hRq@wHzi&#kFRW}Xi!<7?nvDSZ1+mc!3G7<v`rHCwX+3;ijtcM z(o6xhS!};uoR*SXv;S0AFo%*m{3N%sa7SyuB0JU}%d9;Z3a`z3&3In{)d+v*Z?&|H zwYql8u_+q<>p0gVu4)&O91ailOj8$Ip{!klfdxP?M#)#!91zbzFV7Te#n6dLTbrn5 z9RkiHsP0}mjjA?h`rpJ8eeEM~m^f~;-tN0KeuLnT#pD*<YNDS|i88y!j}73p!AFde zbD{fSi3cwNppu!UmC6>IzElqZzCKvt)|kHrNrTidB3D{GAi=TXNoVv~&9CgixP!FE zz7X?pjKp)Rr9B|7PJHtlvZ1t4j}Ezj*~KTkLq;?-Gy=JZKc1-42Oja5ifnx}Ab`&7 z3L&P9t8p^keH7|kibj%GAn;Z7r1tbnpj@b|d4iJXVsQc`nx#2);xu1Bz0|mX{Ek~& z%R{We*Hsn>atrm~`+Yg0)Cl~MT_zR~=9WfHGVb2sgDk}0t8w+;4jl{^_!jAM8A!p@ zBKy$5TpwTn83Ja;c$tg+B@98CVO(*gb=5zQBPgD`tp0@eB}2h_#+ChpLQH9-gBy6| z&Z?{VH4BYJ#YP})Ff<f>a<*RMChr7j%C3s~!X>kRNTUsYK>>s@%-Vw=zph;qHo|N? zFE1{AG<PHv!|-P4M-olH!N=@pH&8L{4w@9Vx+)|Q?z!b_0dgjh^c3>&Z*1m?hLF#C zXmYCqimv%OR}CjZ(BB5v-VQ9Dcf(S8WXx7?o7kz>e|q*l3JdC`f8$-Qle-vDv_EEf z+0e8el+@^c##;$}0thz+)6*eb96s(ILs4G$UAksSqj;l4Y6B(`yOk1x2Q6Q};%|J< zSd5A&LU_^3$)Q2+?0k4~;wyQ*2A@sD@F%U&PGdkuZT@z3iuW3l2a6Sjgtl@0tu}Z4 z{>Ss?tEN`xT(=Cz(-dFRkA7thMgPhl&z5E`mz?Es@IZ6Gm+5u3uZ7;hiV+0QE1P#q z%y3Q0IBK{qOEM0DW&+HuV!#Yl4YA-%c{~`qXSh>MqTzdkb-s+hvu-@bw0yl%;pmWT zEW2dtQ}5Fa#pB+`)99BhRMg%xrk|q}n<1Z3?@N7qK*#lbF<qV&e&_*j+|)b`>RW?r z55H$;Yse2<yB*m(-%l|`C_gEPZ<5mbY4@dSR@dzje_NV^(~uNnQpySnT%~|T5FNXS zj5?5(0%X&7ow)z2IEMmxtwc~Ag{cj?pKtpoeI`Nz0X2(gY#S_3Xc}hoPoo4#>}PwX z()#=zH2wH0^K;uCR3OzY?p27%t{r`&tpZt+F|9c@wklfHc|=B)@MnnLO6J8@pEmp5 z_u7mM+i8RtAmw(^dPrcBzO}Df97Q7gM%<!Z(22`D@kNK#OJNn3uft$diS$1!Fnj3Z zp?E9J7R4ZVU4Ef@iUX)i*5+{W7l?TuSk*?EiDcxgU@YPXJgbF=(;``HvVA%6fW&x& zJJs%PHtzH*Bf#3+v8CCRWxgA6aE6sG@D(I#Sh|6?^qRD{wWT*v)JXqXwz(@~@kfCD z8$YXW^%cp@HB?A9R~-&;jGNKRS!+fYNTAfQE8i62LpJdnX{A<GJ;|RV)4Gb^UpFkI znWDI&2HkK1HR66Lx8-L{eLJF+e`-;sypI*38A>=5u0HqgZ_0TRa82>d)t5w6EXt&B z4(1=(@rq08uZ-JFBsUQdO+i5j5Evx{^Mm`M*b!2$4te?AU}<z(f;tgCHq=cOLAS+P zw!uhRxPxb6j6ua;elWgzDt~Cw9RC3#Gj#z<d*}M_7HCfZCF-1>mT}s7Vyw5ESP|TA zA8*rwNnpMub5XnE$-h%`*88mz6l7E=g>yRiSH!~Ia_GIZ&1KdNtn%-v&!UTT&yW5# z6m4&;98afPN~nbrRL4a{etq1`x{u<K>x>6y5u{uWiyOc|s7s0%sPEU_gnq^j<#T+V zO%5xwHr45XdhCdu^3F88PLbl+Nyyah*R%hu@O@A#Y7MuRqiD~dgNzzq<&t<zEF}Kq zLm4t><@PbO8aIW3@O=<0mm9XKrf1f)%}r{K2Vri%hY@Lm2%a<U#4?+>OFa8>L^|ju ziH?r|tQeSW<VCS{v0e~HslMhfdorWjtNGUf=W+_|ixZ`c8HY^fLa|sHGq_`}5*W{a zaAQ<v1eZs~MhBPsp`)Z~e&vajRWm|6=5J(<pn<Q%AA^5G^0rN@j{i-y^AO0@tEfQ# ziK9&|RyJ<x@V9en4+2Rgh>S%SR~`$j2*;)`I~M|C3w-}3++Csy_R1A&$!LQ?r8@st zWvqsv5jwHddTdyvo2SK_(v5jX5V(;qMSR^UK5Y(E--;CEStw6CVu?q2Os)-C%7TwO zghXq9X)ArC3nidY2h8MGAZkMd`<uxO8l`+L-w_D(?=gtuXjbkrOZKxPHdv}rJ&z9| zq_TJHG=tFVv(jQSIvoxVpCqi?PB(Qm?u{RG>KkjtYkYSafNKaWWoTB%zl@cmDwthj zlln;`HiP_vSZr4b1ux{2Eqe4>t=PV1Q^j%KWG1TzsYId=DC#GK3w4QetNEvI5XW6v z=^KL%kI$O=RJ**Is5098s1#@~hkpHtR3zZ1OL^P$sn#Zb2t{e&rfjsS;l}VKK~v6P z(-me;_n}6UP?4O5P``gFp}nIXw6MMgYSfcAeyE4tn1D)jc<~B6<EeoP0qywBGLKp{ z@|sn0qb_ZmkO<}O%_u@a(zB??kot!|hV3iN4bR~=$TyZYoDs#!L1v9nZXRRAPb0@d z&3DFsQe$?@dV2Ir&*xniR9=QenY20@`XPbNXtr3_`Pg?GM_biuvu_i|{Nd`W>?fRU z!@`hZnxV&|6Dx0LFD8_m0vjXFPppl5RdgqhD<(_fuUHP+fJ$1+70@R46%{l2!q;qr zeUGg4_#;$KPxz(U8*0-yZneO4E)DN^M^3N!&7tSwKwyn@Gr>Z9<8l+%uF{^)dhypJ zR#~|p(5An4|DeP8OotJZG?h0arffygAdn?9T-0FrIPmH8q3<W6aa@+~5o7=QZrH@8 znMcuq0Vv^1y%d9H{OUjvoU;Gfl~uw3yXU~|VldvR%@$C><KqD}ZB`c9&~`#S_e_tJ z_}E4@0X$2)9}8f#K{`w)X+%(!yB&AhZ4oxGtdAtCjNH+GH*Y`LbEE|OZJ2{Nk;f~J zp%BwdBmw+jJBjh>i^#G0-O_(ZH{eHpV=bxxEn$}$heAVf57(YL0MF-<(bLmJ3#eYd zgRBc8iR2`|_f-QBdjp_q)v=!hR(BOU=5pfSsyL+ipBS9m6XD!?faDgDS(_N7R0=7P znqmF9nvDgM9~&;N{~Ixb|05u#!6@Cf6&^C_X~=c_91d@*G&B<byL4M=pJ<-`Bl;3N zm*3D8VIP)KaEEPFk+USq#>$$)X2#O>VnSVOmE)cIqOyC@bSoK|HAGF?fJmg}pzqsF z_3hyn@*1itV%6wckZvGT$et1~+01YqE%HZXAF0NSYEUBJWvOA^V+WHcV@%~!aQ98D z*NfqG;(HP+dJ0$yiL-S#c8V-Ub8dv?QThT3x=0c%0_$PV0$n2%HknV_ryT?~4;@7@ zA)Fg_`9R;w_LGV;6-Pca4|Z}#=A*tjm9WGrq$2jf^0THzw{YgW9qq~T>WVPb3BlM+ zouADr_08npDZr8QhZe$Q)r-q#t5Hxw8jF_dYe!B3Yz2HjO@1Ass<vGsvD7}lRteaL zR;uUSd=|M1xTI)n7tk~sOYnrsJZANcdA|C6?SUwLwYL(6CBP~!3?=AMiau!mbi9<d zzul}<jW2*)Gon}BPC0%*BRpa;$q{%YQd3RM2bZA7)7O6>jNW-qvW@nP)9hfmg@o8d zCfSoj(evzQY?Dn4A7w@2K_GyqxOxK9R(~;C+S9$VxpCa)qWu^BCKQ-z>I10$xoH+T zV^{x_e|d@v_Z}*92!pNjA_h;mrH2CZReFn-SBZ=JkL6@`{>q&RAoysH#rB60A;_ZE zD2!CPL%O{I`ee<JBkP5p87rGra#Ok}cQM?@9kBAeKd(P5Xf_zEyIv`N#kU~wc)?=l z?N<;`2tc*H$1Pl8WBuN6WVrY38^x5()i^fjZaJWV{3=D#%kxbVEBYyy6qrLP*C!yd z4}<7gNUa<RXzX8igiM3j5mv<&T=-JC=|dME9XYX`76?r2(JDgT&p&I*j?n%V`9>D` zbJkLNWg%UuW*lQnL$?Y#Q}2i0{SZLQoLwx9i<(lDy>;%la&sYLrG5MD;X=un!xgWV zc}SZ9(U?NUFfmKY671KpRJg<0$6rHrOV^;dzP8m|?z>2IyuqE|dz?M=4D`^V%05fG z->X-s2wn!~W1WZ8jiOM&!BNg<(qJM+J_?2G;mDjGzBGN+m>AivC;@A(!4D0^TYiZ4 z^R+r*_ZAA0UQ}dV_4MpS4Mit*y=b5iaddSFivuSBmZ?@?EjPrQ=zrE1Ch!^Tj-s&W zYoO2z^-EZ{QMa^Apo=gIUMVXzFPU=I+f&e*#sPH4P<$o7v|Ti1J-(_m=$y&=j9lPL zvh3&60*pdOGZ<lqW0it?4>iq8IpmoTinGT_V15p@!<>-RSH;aBp5<hxT(e#(nih3< zVl)4_9(soKyGQU>aGj<XlCJ>1LPpP}3bIeSdNIP6iuwMMh!EN3=f}f=T$mm~BK0k^ z&Qq7Z8-C)|2a*ml`39XTli<^KJGVX8lnb^!WuDW(NYVYE@Wr<X4t63v+QZ@DbWkbD zM*&xN#CFR2gYcpI``vIM&8p99t~*7i(j+<;-vBL$@e5ABjV&Jz-K$DNUy*v1jQigS zJ$$KN6Bu_WW!aq3(HxIgjGRgCM|gSfgcmSnXkHG1vV=IcJ9zuI)&z^LFZDe@CYx%a zcd1*&UyPUK)i`A9J0JcrjU(kEF?Av)0+L0|(&>v{^wGF^a3JHwj;C^K_*5pASfY(1 znFE?-STy_g*Zem0*utc4y+1&0+-D5jCi;g<0u_R)?ZsSZWdlNxe0!e0<3_6ETCg4< zJ9Jxslz4cnt6+b@lmS>k>eTRlWVaj_q<XJO<&d@l0UJl73n6+w+VfMtj=^RP7LCTP z`^Eit#~E$OBlAYp+rr`SN!>d483o3g&ZE{=O{Q8LTrtR%yEr7j*|$thCwX1>J7x<L zgVi%tl=+)ONS4CYqaf(}aZ2+_?pe2XNa_>HfNnaRw<fy!5kj+~=Abs8k#m^PpPNQI zOY~AhjDteU_HUmot)C%Hw2pz>u|QV?m<izLRoO>S(a4<rZ6xB}xMn`m366Otn2h4V zMP}8mrLdX|`360{-$pghFKd5D9aK}jj_|uQYC-D}teFHkZ`$(h9$T{f1Tgn@ygLE( zA;~uO>UOThtnCSaT^;9}2)`U!Xd%|M4f_@jU!b2u!QJMAEU(o`CWl(WRvw#1V-yOJ zZ2bmgvMkc?Kw%`V<E{-AuJNC?3`P^thHEBXR_1Y7sFYB|6*B0wvbzbGmcDOG(AYr% zYX}gBW)3|>8UIe_g~JH~WtI6aip$+UDK5p+%L-f?6>Igk1Cz?lhU{ndT!W#lmo!In z{ATyPOF<LAe661uMnmRQ8!#ghjLD@}i%^5Z$xE=>0@I|3$&zDKsgw%$95yD4@lfLB z9J4Q>iU}Ms^+)Uv4iJr%I<s&GL!YBb9LbC0(v`C~Nhw=#M3FwvKc(#R8Gl#NP!^7- zNMj$Nguwb-kMw#Nhuh0M;Ny>_NO*jXG2eqk%Pil1p`f>WOYI1F1pynbn!Sj}A>I*$ zKA;(Keo)u(TN1Utcfzf)DM!yl3cqQj;u)CaP!_UA+BGp3iIQJ;_nhH2g3o8kP$do} zFG0|xn<xv}&539r@;LPYzgo%Yg9u(pi^}p8s;QmX(*`S~I!@F$04j=S|GURiCd0>K z$;L-s>WOYhO;X*lKIt)klaYr5%U2{L<>gg!*KX+JRdNqT+ME0F6pR!3qeM6J2)gLg z@oY_qK?xzPB!u&8Zfp`+Ooi$F;40=aZYw&>kBL96cE>Tlq&Mh3+mrB<+-Fxy!$BWn z#glv{CVKki8D5X5NHsHJ8|#F8wA!y2dJ)`tI&j0KN!(LXCPZ27DXkn45(Nc~tgSfS zbXtXQvQSyqc=J4eX#(MaPhcI#Ko)=B&7KzC_`Ll?Eb<x4A`=HxRHP&gQ?!J9(6I6u zQ}P(h!=<t|=ji}}iL)IC@7V~2f^5{UZaSuq*yAozGI$01_&yTj9(b^PLgMa8?}Wu& z8|Q*~6xO?*3FFqS4x-@V%A@vRVuBJ!#65N4GovCjZu+rfa}?wRJf~jC6~0pEQbg&7 z<d?~6?-n7g?HCEi^F*q(Ubk+N{yQ^9W&@o}S_UzzpVH!VD&kXPVfFKIlZC0Z;(pn( zx-`>b<<qO=X`H~#E96^^U|)kjFu@&;nAkpOj#0q3ckn;80B9r$9~+zsG`bjrBqfmS zr&6OVe8pQq2k~mBchG7@P(@6iVwZX<%1d@uehC*J`k1rwuj$z!8})6R;C+3Z!|F@D z$D|ORgQ3D+_3kCM`8`YUKsfM_j+~rU;EctL#;%0|zgSkLeov&sm=Lti=ALmimLPC^ z+xhnKQyPK0ZEDA|%?b5XAcbHd{8CWxA|_(S3o8DV6K)8cn-{eh@Ps8^kX>pAjrqji z;7^TbQCTGBu`30)l(DV&Z~GVwMN7X!h#+cRGio+gEvgU9411dNNUaiyTTw{i-Tby{ zR@Z(>tSnG-3d6z|)R2U-&0@F>?Fc`oKb>KBbq6P&h~nW7I*XDZgw3wCK<jem-K5mE zUQChp(}a*-xUXCg{kVhHn3o6B?V(t9^z{%h(;_)c%hhE#ZFWp<$l2)%^KFme9t1N` zYyC3IND1G-W&x}&IICzbRfQqwnF}SX-byt6P4sDQcDBdhXsF!VH?&s^2UcLS`*GaP zT&=sCN4<1jqTG`I&C(A%!aA{JqO?IWD7FfCG`4k+uFNy7hXuZmAV!oY|BplwY%05~ z61j-3c%$PG*<+KVI)n0dKY-}#ta<BIIJA<dDp?r8eMj}p)_Cb%U4RpKMJ9<)Wl+1S z1MXo={lirP0ofd|6q&xx;CLF7`EI3ZFw11bC&}Y=9eeYAV2Uq91h+k7Bq>D1>#qan zVGtAp5SE+qpDc{rz*44(6C17chv$X%k?WLu=pL8D63@M#2^(IIbU50|Pr7mp*TU?L z1&<e>9pSUf%{guDzI>juQ93CX7;Ug$VGXEm(w&yUKO$^%$oic=7H$9{D2p%mC&i_; zzPYYdwL>Zt5wl+=zMNii`E>EL_YuM~sqTIhj`a<Tck7go2-LjA@ovOBLmcpU7X|dK zwDbTa1A%xR!JcY_ipFfhisQI_tCdunAu0;83Mbg<abEs_F$Bc9fYAv!f<petaYO3i zfe0M%pwyV48d*arU2K1U(Y==5A8KsPXc&7Yh*XWgDALL*d`jfPUhv$5MPxw|G&4*3 z!;8tsdpf9Vq>ajFP0+`3qu5MQr#90wCScKCxm@pXac-Q~o?Z#V4eW5#qkq5nfPLDu z*g}YvI*<3^+ZqTzA`y3NT$yF^WOyozKdw(rn{BssnG2p+V1LLR!jou~XmeR)5Kl|b zTPP?f2;jCB>z+??{Qyn$_3PK{CwD=@^u33%Fa{*Ts1LA6E?nQ~J0D&$md<Vna~=+p z&vN;mCWh-T5jiu~@<Ab~ii!uzPcoNwXEI-HR>Gd7V-H4Ri*Y151kat=5Hb;ya=l;? zNSlym0087+SBE_~HIk&XltX))Y&h7IS@thr2tmQzPQ=z}(64OH!DCN1w!5Z2ih}Yr ztGTZcUTl@&CZn&&cyMq_Le&qQu5bRn!iX2Os-Tru4qJ(<fVUQg#?>|U$B`u_CreAG z_Dl0pYs8OW2;A#mzcihn`oh2kvf3ub7f+R0y`H_j+PXik)QT{q9+@j^zj_|7);=1? zeW*4EyiPtHYTVWRc-^;Q#Lav-x_vp3sChe=s&%1$Y<gLn=3~45_;EA2%Z=}jkOlh` zSWZ6M`w77e?hY;Y&MR}>Dy+zmee~tUm32uP&#WE1-g?D1ziDM(_;IE-Ug9)~`-nJJ z=it^^!{qHmyxn@gM1^fkgfR1k`q2w>T=O^lDvKUcEqv*#PH8jh<5*OsB<z@Z3D$LS zZnLVmV&!4mkSRJvJDY2bGG#eVi`8REkLy-=gWAl(fCZzPxvj_fjrxzGH_J2O+4@^n zF1cgNbeF{?JG-}~$HF@-AN2>9u^S6@mfH-E^VnDu8e15!$Aahe&yg-CYj4V~k&LB7 zX2PeUBZS;;t(}-^d|C-0!NJ0q==-VmUh@v7%w$#ONXWVF-IfWUvalb)PDSvWVp!^o z(+|31u`4gSnksPdCdVGvJAWa6)M<5@f8C>U4DT$3o`QNGh7TXXay~uZWIkIy9$dKG zERG#HaOr~du^yVgN_k!cdgc*It0u*IAF&x--jf2g*Xwb|HM6Yb^mt)O$*<f5c#q;z z<Ta@b1^-AWe6KDLL@nZrGF*P{jOW2?GyBq84Q#0uZ$g!qOIxzADTlg)Ah@|j6weEl zdi&BJH`O|@jO~@ocYxx~PWh+R%DIY(lbM_8K>PE}Dq(5a>^(=hr2YZtmQ^c>KyjVR zM{>%=7)3J~()xXud%T!exca0waT8hfI<*4_!EqW^J(*Dw1}+Ayy@kyo%Gs+43+aMV zMZ<`;(p(Z&*QI1#wySEoWXo|Dp3|?I=gf8vkdw3IpY*1Ji~Egn+BDuund{e<b1CDO zGG=7SuXq)m`965NB35<hE}crS^X)Z=w~5=iC=&B52=ot+3CYJS<yT)w!``gQ$?B#C z?483lw_>_vU^Y+L$VX=ydB8CluT!0FXe>=kn@^sAqVD6E(>i-zrE#BK7tg<)rcfR- zSiE*DVD~`_7X&ENnu)UL0>rvEsxQ2N6`21V7lfevF8!B8E;r#f{RqBWHIBT_PL@GI z$Fd(;y>HWIErLA2!9(@t9`)_JffZ+S^|)O#U2Z*F$H4&1mB~0KLzj%}UL<Ro&ysTG zZ!R{@>M5+BdOo?$VE5g0!a<nPn=}~z{Ko0mcu(7Yf9$2PwpJ3c7;3}Wz()+8OhPJM z8a(O}e*g=iGs2ai>(f_$Xv|O=z$IPAZ5#R^BhvJ_sDm*pMkn(S)0byIg|Q#gjKk>m zD|0_svgUFvW8ERM=uyVJ`EBb%@OVmEFkX67k-ci~i;(`3gUMbWgj=@!Bvs~*$41Y& z@1$Ln<?griiz*z+pWSO1e?s9`v4A1T$<)r7dFdNs{a#-WS{5F<G(a)2cS(70=s~$^ z59n9i9GeiUWL96PXoW#qGkxkO9SAcdv@+gT92PB+(eYWqc%0KwgkE~HwLw_|l_j}E zT~l%24CA69T4DzO3;2JaA3<p!gccSllhslBXJmR>?9feu*6$mKaI^Hm#-!&~(-KR? zlZ*;*z=i(7f_lD-yJfE1SfPDhzF?%J@;ab2U``7&IO8hthIuddvoX^0Qh&FB_!=b| z@<_=jRXRzOnIh~`DJ+v^wW=xe-u`j0g1RY<VQ8TJm|%psRMS+Re2YiU$ef&@z)oEq z0+so4Tw$$&Y`YhAnoy6NoxaHkzt_AmrrU*m>Idt4+mc~ugg{!SpIXwN`cma+nYfS7 z7+ZJe>Ls81lohH4iIWe==$4M>7FH_Il%c_nC&sITpOTNR&+1>)26JCG5t``S<ec5* zRd!zTmMir72gM%--8V@H6hHsOm6U_RnMdi9@M=0!tZ|q9Q#9||0od~Pr$X4mzaSp$ zg|M@&;lnB!!~dee^L(H#IW9%$)RJ2}0?m=h>1eFs+qWT1v~4~ci$clfG+PM+30LA~ zs;9E&B}lqieL?3cA@C*f0b8m|cl>8sx{FfT?8AjiqrH;7IOS#Cl9z47+^pO|3N+GP z9aCwm_Ah1iGzIAmFn{E9o@bGvHv8Gik-WC1{3P*fvEAVP{^NoJ+My|T%-<c%YXEXr z+)$0kS|D`R!PRh|(w%Dgs)mAbAIOVS#m9y(P=^eC%PT(1SC&V$R2=H)N}gvNH{a-E zw%BNdbLNT+Q_Y2J>WkQUScMxmRmz4-lt9+XG`rNdF*P=`tFPv*Po!>W#Ok)6KJs_i zI=W;i#O;#zn^#5}SCtCX-<%kj+omPRS14Fysi#{vlZ}3U)w_guD7xLO|4p{Xq90!- zbwO8I%TxFadWc?y4DAkB8=v2ica|@%ky*ZtefCu^sWgvTp`W38Y14I84fpXvyuW+x zR9rGDS0MJ3Ie^{>*Ex$AC`hUz4}wNmOUEn!=jh<yzQgzK4$^jIT)*E^yZgj;bMeT* z-bS<NObJ1Ovsxal0#_llQv3-{AR;NhXW6MX#Ys$`{nltN<X4w@rHQ9{DYuO`@-j}C z8LG9~YqdU!&~kxk;l!~1mC*L?uT-fr-YeMoj0`3m`qt7VnP$2qjow%@P{Ll5=4Jfy zWRuysslA!>AXWBE0~g`hTFv>~%kjHwW`lgYtab@Y%?c*t?^A{*UCv+#N?jnY9Tv3< z`Ykevl~XE~m2B*-vrB-|*9{AG4xCX_3(r`656gU~*h;6GrZ_#1ZiYiTw3y4f7PI8w z6S&eNgHE;Y$|84E(<LZ;Y|2ZyV92Zz3~LfX3|}`YQrm6vlaENgwHe$rNvIo5CK-1& z)cm$HzpQbvPrk(v86%SsSK~IUxE)U2m6maf9y+tASZgb$o%S>ki6{{nD@Z3XO4pDn zztkt0H}9xfo21RkE7KpQO?EN!?2gHiKO&De`SJPIVQ5c8-oPY5p(1)b-9l;#XDP5{ z1J*mr3-%Cceh0Wu2DE}7{`#O0@Lr#)%y-(mvzq{4yfKrW!MzMIZ|cztaXKrq;Gb!- zUNkehTP4Q%VQ>k%bd`<%v#V@5oZ@?a{4fn^fC_t>S}!=cPxtyI!O|HwO}annPe$kN ztZ^HxuIl_+iNFdlS;_gxT=TOdZX?i^O)2h$jLhn!b&-shcyIOZ>X>TL0IO|aVoz*( zH3}L6w^sJ2d-faC+8oE<TqKSnEeU9eCOe1qi$F(#EVRb*Dm-8a%cc<Qsus?%q3Bg5 z_K+%L@}@%@HZ@c87B6z$WmQ|9)&7UmOONsjIs<uJYyvqKqlUjePzTAr9MG4PTK>+z zDjc)_{*_)6WB2EL(=-paINGTF^xJGGn!acKP7TkIQ)P&mms6Q!qJ*ZLu5Bz;I%jS~ z^w2{-Y68p0&5B3*79~JdY2x_N6l`qdpuUH8VzTw%Z7ip?-+v_fmu$PHP=gLcHv5He zBBBET<^E@1vz<~K&|YTD7?>G<wYt3!4zpV;nH$7kP;0yuUtW%hEN5%f-+&#-TPPp7 znX_eo8w4{J>sX(Ic}xY20aP71a&gGTR{ZGX4e_QgB%Z)8oZ`QLD?*~hErg%?kXOfs zis(-vLW-`U0zK-iT!Wj-d_K!mlwZlicFFPmC)t$6Q<uIkDP3W<@Vokt?S8GFIu$wF z<qb8N@3efTp6;R@2#$N$K|;m5EM>UVIF<|)K9RDew(INZk&)BOA2{ciT=X4?>GCfI zd}H1_vjA8rx5OmK8^u1j$9XnMs-KLtIMq%NMF>fyt#porY2OZhj<>FFyfbY!j!jD{ zqK!pfAd*r%NU{I)+A{JSWb4rxk5BsK*KX$1wSvsPA3h|^YBrI2&3ubjBwfqPavwX9 z@wq6!UXfM5d);0~hekj^!8p8)__pPHHU}3p?gY*fB-VMzzhwRagjQAf_zHZQNa^y| zmQf`BrOWn3`&{_k>+RmM2sH8e4H(*~M6pJ@ZD<cPF2~S@SSPs4D@<f5#n{y2tbqa; zBxs+7yoHCFWb%5jtc*xK8FP~kU=HiESbzmN!0(bVb=GQUT(MMKi@0+}j}$IpI8xp} z!pHK+;cRjXrMv1P>OsVrg*(k)@p(0reN?eqyjqjI@yu8_M9fE9lC|HX-YrM0BCf%h zbk#IDZHXTR4hVT37`?c%nSlCu5$3m9+^fi<!dmrkMe3uOVm8wKl&Fl2yp$?tTEs2; z({DPmEXZB$R+nLisf#nr`?2gprEM`H=1Q6tGBRKzGY#3X)2nX2zL*%e4zJfYw<Q2D zk3hre-v@>!{nZyk6I<EIktrR4yf5&{ahUDxD~A}+0GQ6eU93ruMTUo<e+jcI1|?`n zWD{N@V-};fTCM5%JiOf?wprfAZhG;&IQZ_Ngq^l44KAksq(8CFe1NIn<#i{~qCG{a zAe}nF*e*|zkEz21q-$uE!ccxmZ->Hs>56xzqn$i+60b6?>;O9%Wq`U(jp;=1UbkkB z^CE4|97|?RDidSejN8!0w}Ai0N}q}Q>&~H-Z_u3`=gkxKL2(fMEzM+E-0N0iVfr>9 z2oweG$68_iwk30$dzYCd)(26x@Nh82qs28CRxJ6HV+7q;Z@ENh+(1verm0VI?F8cD zhSa4EFD=Fhg*mfWafxvi3|H(_sZ<0%I7h(3*!i%=lGph1A$teCyNaG~OVKO7GR{GO zPNPnwtLMazZ1EM794BLhJTx17x{<ih?<h1r$g5OQjMR4;!Blv`v9SBM%uZS<1A0rN zR~A(2=>W^BfOp-y4GehxX$?Pk6H_BZfRAGJxeorAS&;H#@pP_m^XRPgj)8@UOq%Ui zZRDV%y>**SN5>z4gbqH4HU6qY>%dzBMAwXcPc?cvEf6;!p|2SlO&Ch~ll2)@(ye7r zR$ee|iOJbLo7+f$KG$KKrJ0aJx4DQxv`AtrgHgjQ#kbEMvE9&Etw>`7cEu%S#d6a( zaNz@_<mjMGAD4rhY7|Qq6OR^esQUA%Wvvq)#(R`1D^kC(Z(Xbm(`yDQ)qCWpR3u-s zWr(39zakH`jLpX?F`4#L>X-ebT={rq9wO#$CLgTA7AW4u^3$0FP%X}`En2y-ZEL4o zoKUHe>sMfK1I95-{asGPjhp%B>GJA<AKL#G3{Xx<N;FK=U)*0nkw0!0I+)GYq-mXJ z_vZ0C?Im{FRFtdAnmaT}&H`nVh?Oe~yjv>*WXoW|TrEwV2MZA=T>y_NmdwD60{u(6 zz|9cZD_nZ#vCQlzWQH-d0;DTaHHg*}dDXQOx3<;!?66Dhoi7P2-_4(W`7lL=9heGW zt{IB5d&_IED?>(ll7E95>{yhyjK&?4q+(tf_knqeD)`xKfkDW<9g!QfvdOj_+7ma+ zkSC<x?x5U;1)sqkEapSh(y_$#lGf?4I8CZW2*obj-}uNV$ftH$MTq#g`V3n#Z>Ij5 z(Pi3ejnPwJ-}_$J#1R)WIC`d%&UP}j#a<5&Kg|A51O0AJMAfu36p<S1>R8+4<Mn|N zvMT{8yX>u6`9k>W+(kv1)#^$8lRDs5_&wOdt?N2JkE(TE?(!0crf<!vghePUaABmQ zvr(LezfKK5l4wlkT<5S!O*&2zCPzFE518A4XuY;qpDwN-L3XM~0lEAsXe=H-J}|N? znI>#{D&zLt1U(h7hr8TlQFhldu6Y8d&P*OhS^@cV)~rz&3s4`wXS(EYc9**;<dSi( zQrWkc3*S}S>T8x})MP6}RrFanr_%72*6jPVt3~WNVV}<Fyt15|oGBcLNW)4+(tGY` z!A1K8@*iHwA2yUi+uIoyX=%2*p@}rXU0ft^!vDx`t#^;qRH6YM0NpuJV4=r_O$d|^ zyoi?3?`IA%;dsFIb9f|tc}{z;-ti|LEPRK>)$G#~D#&M&CQOM#toj_-*F~@%`gm*P zd8Y0iD>s%L6}L9=H!Xz^ODNsa6_BNi>$T-yo*4iDTKa}FICD?~ujG=^f)}+te)oSv z6_><&{jDXlke!U0+{VSi#Do?LT^s`cxSG>Yv`OOr#RFguvGE?pDSZH8ng=SWkB$y? z6Ni!Iu9Y7*AKFf&W0jno7Kb+;up>JV|B7P%RuozuV7vCj))Ed0u_`^g_&?mXlN99Q z*c`2QOmbs==L^!S?9GxZTO{_{{*cwOp|`YptJUL=J7LFv%Mt$^A8d2fKm>6KY~}6y z{0zd=8Y=<Tw+ISjgW49()=WF5a+u7_%9f2+=bs<E`^l@cuaGfmQcEc3=q9*K(8x!8 zk{#95mVJ_8kds@vO-zQDu_i&@UrPrB6x3JrIHFnoFVQ500;^{S_r|i?o^JAI%~#-S zG#0SA9V7s4h3g1+8Zbx`Amom6n&c=EV3qdi3%#rs7fZpv00Vy?`>rq$uwcWy4UoUk zkM5lW9y}=sMD(eERyAbs+0LqQ=m(HSG<+d9SG)=W;J7Pz;nRA;|D{^j9YC&HtXEDT zb}|1wfBqwpE?=}HK=={v^9e2JyXw$}ii*8|<$#>xUB>(A@;-U~siyH|yzs5_AO`=s zyJ{co{?kp4tDYG8KYxds^nqs+{FXZj4g`v$4di>V@nsWdh-0&XeXOpxm!Mz(BSW9X zb}dLQ(tq`8IUaaxt4=N=@Cps{M=X2^E8-$xt&5My&*|dU$yzohB>*#8%);z$?*)cz z;Pd}-E&n#uG=?9UDpKh&V1eECZiXQuAxT#6e=@+qZy=o4sxgmEN@BVDXzcy>@BjAt zM`gU?`)d7i-X,dT#Szfgf6fj!s|9GR)^67tI-=BGbOzevVqX&3FoqpN*xyKF~w zalPT=w#bOQfJcl<h1zr%#nR{uKy9LMC*eC+v!o&>JIYq}9-I8(`;7c&SF4J^p^&H# z2M4!-DEGzyTh`byC@>WXn^zE@n5pR@SSjl)GF9)uz?960xO3XMDJWe+dBU$I<rH1D zOGFZjx*w-cPooE}xcuw6K#}?M*<R%^0sjrS7&9>7TWaZwWnWT3&#=dmZ-7=@VwRi` zUpd2l#%jO%o%6Cuy_fx#Tj3I7;COq0p@XF_nfQP86ZrU0AfhgE-0j4$fkT{B+SLTX zmTEVdUNq~kfr6@ER9pSi##vrNzNG9+<8-!XmFP`9>|rDl_pRQSq8uW{@2BgSJG5_J z6`Jii*`p{b$$OLGBYXDkTwmtGC1qz9QEcqhXZuavZ3_2{1&lq4PLd#6RpeCHD%NR& z$%;#6=qgEB0=+F@lfs{Hh3s4^%4UB43zAyVFw2+X+X5b3VtDT$T!#bcWo~6KIc!>H z0$7JAlvmn;^!c`{AL5=5rN`vQn+}U9Y#H~*=FS3*6uu?*3J&w^Bzu{YV@XJ<2*Z@N z&zKFlqw^d;q2mCZv-$-l4rVkklm*75bvTNaiir@T#c^c>vvPW=k5ut9Wey;HxxOnc z#=Fc3DaE5iw!!*+$QjuGH7uGSWkea&%8vrT&ZByBn`j5ijB&IcP84;JHx)3~SE-bW zRW;J&3XdV)txJg0vqlWsZB4PN<3VYl7q1raB9De+o&Fw+=A{&!kWXQowWKoKGe`k^ zU1>T{)V9xNy)AydcJ>dT-D5IwHt3+6maU_hq`<^&s-RC?^uf_QlOB7tAsztdy8Q3x z;EEKeeO}EDFo3dC$^waX_CQAVJ@T&=G2Nzw76%kK(-3!8FhmHkZk(C?9(rZi*^M`n zuT~uTp0PgQR50FD?37!sI3h9?pSmOCsTg7-cI{mE6abCh8fWU$MQ0i-3MiAU=42r5 z#vtLz=v(I!XlL$ijvedgZd7p{luJ03+xg$CdhbeqR~iU8j?F2?cN8Ols~upF(E35I zM=WSnuOyA<Pi|<#CeOezK~5$u&{!;+DFN8cayq*i`p3<iYDMZAI0jj9G4^Ev;i(}L z-{cg&QTv}6q(X>CvCxysX3$oesoe{zHae$27Y;D96mh%JOixO6g%NfNf^=tHU(X<o zXE=`={b@q}IgVX!pTu6N?;f!I|FlfFL0w7EV3OZY#pL55uOJ%h?Z~52EvZ`naCZaw z82@y4Dc~~}=Tgn5e+r53{g7#$!0j(4nl6ZjO7y6=e~{0RFB&Fl{nRYVoL2W`>ylMC zDsV<gwNR?yQ**6oolUc_mbOoSL!Lp3ey{q*Y@Y$KO#w8CWIC@C{_1~O$Mqkd5QsV= zlkEN<d*2z*)V6i2h$1K;O_Xi}>C$^q5s(%ILZmk#w9tDMQIHzx9hBaC??{tgLg)~B z=tv1AKzPA>&v%b{?!Wizz3<-+fz8@$%{k^+W6rtuM=aE^@p^w4){2aBhKPbJ0XakU zjsk+62zVAX|Kq8Nj6tJxjRx0X(zrM{*^kGKtrkxx=?4^()ivC^w;UKpt@V@KwNtyg zIMt%UrSHGIo6Rvzb3H4!TsxgnHsP%r1F@DBp;7Mj@7L1yx_EjSU0C7v`HGhT836>P z5l=?)=aOza;MqpRs^<XH{nuy+zSpwp8jo0Lfe9b~j#oXUShRP!mf?o=+a`h_D3q0h zW9H?;6H1ckPGSp~#FQe}Jsq;UO29tuT0ZTfAhh4`^hBPyz~+c;238bVXMLHY6PlSY zZ+kWTCGN0IzEUE_x&l9w$w8+!V+y_`$MtlU!jWq?25_Z?Bc1V4cG_bV0Zs>5<14jF zwcgq({T6Qo=5Nm3gaHd#J0mrUr*h|a17?q#J{M@4J;7&ccR+mu0)e3K@ugGavi*$D z58{EXB;y>6==t9;2>WrXP%;5u3-^5V{A@FN<|6o<u>52VcW7z~J=#l3(-z6A$)>2D zRJ&jKx=-Llg}C-fMWuchiss|JqcmilspjN0ZN#4wC?Fql1v5Bl>|$Ll#u#WLDem>e zdGo-Tmw!#A=_O%k#+o8f?ykv5r~Kw?#)YtH;(kVn1HTCwHCuEDH7u(WK*A1_(n?S# zRs216UyP_pZvLd}B^bL7lJ=LGzQ>2L6pr(9h>MrK*4)n_18)+{hlYm6;C|Zta+$0w zu{Un%9Rl|Qld<Xhdhz%#f=|1<<s(BAGRXI`H_^SHs6zdWKNdk~#-Fw>t$N64&ENVv z3HV7HmS0|Ya`O;7){~!TH(d)bEWYoIt|~?+Bt#Xc#s?l5*T<b4hsotl-NpOukG<U} zu&k!=ec4aIe}a>mBXJVCR{T3D#PTO5y^?3je`hJbtWH$UC076FwQwZXZ5c9fEUIp+ zSYCVn%dcFlmeBR0FlVASsuu|TPxqqZ$3|A-NwOu+fBN=cclc~+@b5cpsp+}9pIXLq ztU9Kxz8Qih$Nv_#K80c}|ISmMo7jo@92Un2olJF(Hdnjz7eC?^%tfm{`*SQ?(%1=| z&KkUi9p&E=z^U)>F?|_T3QvUi;H+(JrY6>U{EAsVzmkp%uF#|YE}_+qG4=Arf;f4% zwmu}hx!)@+X(3b8{C1L&i_2!~%FqRq{cWCO(y`-JfA;!f+g?R^IbPGLg7wpbj$+9C z0ysL!AVu3fK#qkKs2-R4oql&GyE;d!Tsghklnl7}RN1ju4H)^DuVgoZ>cB_7dMT2n zGwz-qZF0tL4n@~N>-10<+3&!>P*il)lA83*m<rhO=XbBKwo8+Phg8#$i!6^)o=ICK z4XLO^s(dzlhApmt;1sJ|zgpwrKAIlHRv5K(^-52{z5!&S>~-XLlAcU~lu@j>Eo?fR z*6w#j5=(EMVJ9STo{Q@x_L=oKa4tJIokWC(zkUDBW{v)iKrmTE=xu`88s$jKlvZse zvqf^FG1*;NsH;^x%Xnt{jc9T!oPmm<j;q+#9BN{|U8nVLOZS2aj}zAaA%e`4Fv!+9 zCsFWR7@0Bq0AiYn>`|1!19hr95mT$Xad&pC5%0q4vbz)j*iOH!PQK6#RoHsb^sD|l zFnLt&LjjeofkOr9Yr?~?+!U`E;&-v<s+xJ#TUHU2-|yHrmoq*V&OaDgf3{cE#S#ej zLXDbjDXO+j5CkJkfLfOR+|Tc_Toa*ks4iA9RO*(Z00JNP$1l3?634ms`e#$ETMo3f z(;;a%|5%J$)R%`8Tiat_Uzm9eK@jS&tKeAi<&6x#A8+5rmA4w2S((A#2_+{bZuKxR zKWK5;-du|Yj}zp-vBYWhzU(knmp#+`j#S02EQJl0z3|eOnXfbdYmyEj+UM&G*Ljhz z48(anJ<Yi?jI4*$$SG`S%F>7;{!+1<EG>kVTbub&rfQoberdL5nQJ*2!vV4Vy0J{* zymFnKZCm$3h{A`to833C_WUas?kR1mD|ooIvN425p9KHZFVSDgRemA`)r5+ds0od$ zt8o6n#~Sk+797K~HJ;b=QsMDqsr>2Si7(cLHNGc{+&2PMV2V%PBpRm|Ys-kHC1GsR zIZAVV@a$JNjF>*N+10zkSg^a2Q;!rv!V^~w`|<r6{TETL@to^9y_$1f{2&D?f)mmp z`uIZ~Z9#_@EeBeYMf~~fVUyQF!?`2Dw{JmFeCrmF+WH*x#E%nY-2Fn!e^Ma>)+dEe z=PMB|ptCc_UX6|J-abnYOq*CSeGJLP*5T^CTdzQfqNrGMqE)(Zr!jNGifJV||D>(U z6d7rq=6#uycXsrm(VGeD{n?GvxYpgEEWt<W%>K<Z?TpNA>}?eE$b>*>DXZ6}9QO4v zCRX4lk`|o7SIM_J5=p0&3M*61OI~}$*L3u>yM}`W{{STKJNP~3p9N_cW*Q)YMD#)# zfNfV>9V_$g1|I?_1)ZA?UYgy7UtPX&Gme~8qe5Z)xVpsrgPVw3;yBS^OhV(wYz^bS zN&Dt2(@LM-sHfAbRF7};UGu;@K@X42skrJVsq?ryJrXy#TC%BcAXZX3B`B9nMv<bm zK>p`~$2Zko^KU6F&{3bm^}lgr`8>CEuK1hbUdRO@0(uu?=m^bXmF+CnXp@EAo}0V) zxebe`W!WQ7V>H|pF!*EyeLW?Y=fx96)WK$x^L*6fvB{H$bEhPIoy?qu3xgmMKsN7Y zON=_uLf$N+GkOY6TNf9_6n`vOHzDX82KoatKYf%`c;5=MTI{VP63#r>NUA9jboX>U z6Y)rz=}%m<qrEUcDx9-50eo*2iw&nV&>7jz6p2GD@v0-4P<-v!ODY;7+{~;23!Tv= zfSPBgEjYb6j3<JMUJD{R&CXXTRc**_2K97tbwJvELHgEQ5$ft;136T`<e9T$2D>N~ zSvpg_*0d%41J0M9&wAtfYidlU%yg&5k4TI4|3tsy8~>tUItvSSf$wXyI7&Y|-LDam z@wIU{ycw%FAI3LE)+0l+^`$TuU@RaK90zY#3L49sgILyO@9!FQv9LvJyNU91f0_HB z`sk5y_TyT`P|eTi+KFM!2vb`saL$v;eq8?!ri0uY(78jIJ~y`ecEtb?Kv(&AP0q+@ zsnhr|9WC8ZRxGmYCdFKV^KUT|Ywj-O&J0iIV{OU2tamrGr_J=6hbcOp;GH*vrDc2* znxqX<dROk|<MMJFG{kJmsm-4a(a;WkI$m?a78cwJc#oCEe42{z-@DeYV#{z;7;fDa zO+4PRcqhmQH}`B0>d{&y0$nT69qwGORNHkLS$~Th-fJjeOk6XInl$9kyK%HcJhs?u zv+Jxyg$g#?Se79nyasgQ-z~K_PHE};wD2liva&Df<|sh*^@|2~M@zBh*4qL7uX=q0 zI_ug<9@zYzPq>7$$7<d9EzT}(Nuk=Gaz<`mhxTYt)2wJGVMFe0XAe4Un~W`nUdNM} zSY**C1UZgO*`8S{4b!=*;8AN&U_Mf+LYyK~utHG^<sB-TCoc5LON#wZa*e=;HyCQ) zk6rteSiwbhCmKOXFePjl(hbIGJ*=#XjEXIY=%Grtj~OO=ZqvnA4c&y+#%~&{h!OnZ zRIREPmIAJ6iGAO}1BbS`fVm0@Nu5Sk^Sgl)^+TTcA}JpX#7b`*OO@4b>K9e2bMV(e z09sPm+JgsrKJ~?fM;r-m$-?2D+6}J?14{0fy3E*?iX|}{icec)!OCaG^GlK@=#zh^ z8<Mj+xCW#TF>};b!gh6&!K=%*K6Ay(L&g;j@z}y3a|$<AQZl~qW`hc{R66XSi@xKw z`_=-Tirqqc<5n+pu)l|cAJzn|52s9FvNYnh3VVr423UCNyW^O5%zR~{@%(vx$w{b) zqt`+g7p=s!af39Zv0^h!KtKj+Ii5iZ?$oMsksR%-&s=ES8kl|6l*Xr|yp$mmahF|K zbRxx2Mk%|%K~lVQfJ5kk{5rptn&MN}Djvd$#L+=AkV7i?&E@cJ-bX^f$aMh|L-iRc zT5NvnPg-{oSY^FqaQ$;ZYJyMa-=u|Khf=tNA9^OF0mL-Z7Eaa|&Fj;y5CUCuEVu<l z>dynWPgVV>&qy-QsLU)z8rlWQ97pLO>_zdjSN8`sCEz<F#htF=jY_n_G-wO{5Q6CD zJ@gJ#iIrO=v#?%qd&^B8BI0P?5nz8&;pW9#9m*7Vx76zmEgsPtx+1YMqcbM&`-KMg z+e}`3#A?vkfo7ig!jv|~&lV%(=k`>T`sQ2qJ&mykB>{1E7bP;lxtisN^Y7M(Xa?7! zlU4L1bfn{m6mdW8nem&BE}NNVHRIYta|*l^z=!p@anZ@4$gWWTOE%+CfYofiHiCV= zmCxCu1N7rNpJyhiZ{Diq-YqvD<)?6UMGbnl2+xS-$@}$asflK8v1%)&mIc6F=llt6 zgB@4%@Ndyu9ptXIst4yyEn5&!jPqc%%3C7dRaI4Egzd6_K2Q@4E}iV>s&qcvw)I5| zqzG->qO6_7ZfqV2F!?khz#O5b@&#~$5HFONQn!<rK`}afU{6Xxxnjr|B`pFg6#(@s z%7^RQIW4D1AJ%%72AaHRwPJM<#dl*-xGGji@ELjzkvJfg;Jlc_Aw*v{f%+EmN>5mx z+G*=Gf?8tmBSt(C^5jfNOtvA3quk*z=OJvwMH&tFTXXEKTN4@%68Iym{4}38DLBLr zWwE6^Dxh0_n-YR9{O*F8hu_|>ob_ghi>MN1;P(xc*2<yKGt$R#fOT2<V7lZmi7Jrd zW-~8Yq&Z6l>P;SDx+N{I>0@aeu^eK!W<!dJS^WosNN~cy_$y2Q=4G|<ymsZ}g9c1$ zXN{v4mK{Zrgr_;Xs7DP?G3tVI56(8&?f7->R(GqGdgeQR4j;3b|C7J`G<Hjhf1@_% zVq~j+&4;>t*OHVMc!g+(<}|y<2rBx9w=>hbf=?Hmjf}_YT3}sLk}llDHYFW)^63e@ z;ZVVZcY?P9aN{*NK#OmdOYgGj;k5c2hu8E(0@jvJ1C1AIG~fb<rjMUw(1lkXWyLNG zmbsr{v(3Cl{dK0PyFbfT|Ma=2jHWC2r2SH2DnFbv(8O!*a57DYxi$UrqlvQZSBq?- zyRH@tP6`g7$udh1oz}K{<sa0D<ySS5ND6%qxXR1p%gC;VqH0t}%|H+E1PbIk`$N0a zvaVO{?w_2`buznrl$qdIQwB@AyvVNcT#7&|BtifZnUD<BieCJhQt}Y?*pW#LzczJ* znezRNmV5u~Yz%t;gM(vEwOrF$Nvh6y)n&16(>R;16|#lfR!Jkn={RIXKydPe5{(K3 zP6zymQ!1cD&wuO#aj3+Wh(rjzT`Qk--Og=53q>+rtF6IcxWe}YQ#7r-ZS7w|y!mMb zM^mQF37%1EmSIa>vHUBSCn|S);3{9|>3FdUApg7aGr6Q3iDGpJN(j-C|DzrfTDw0Z z*}q(?^|7qQA2@pglzvY8q8+lC{b3$5p|h_-Ye}@sKm@HkEtXNWZqMT$c}Ij<Ax!K7 zMCW=*Le839O@Y*^kSiVQ^si_9TNuBa^EaRP2}Fb^KN@{iIN_Rc0FJu5zSFC7HZPSN z7eYINNZMoJd~U|W{3t=It@9Y|{4_*^Wy*}3`N<klef~OAeHYg_hp@se*YB}aGR&}2 zjSoyce`FN@^#E+9-Z6O(|HUB<LzV7}j-0vVZ0T9Ur3`(leKQX<u!RyFb(n(ExY2N$ zkZ-q5`YFZJued?8`*7VwG-{s*&jVkTK6Mj+O@Y?>g4CVJt32?`%D`}`^rs~jJsQu& z>3^s3O~}7Dducrx#z8xbAu~3t@g!q&RC^dNWQ1JL20IBdKkiC)mZm?X9obpFOo)mR zo;#o`6i01X<%lu@CaSOK{BsD=Bf%H_JfJVQ*_t#BQ2I)coNQ}*n?B=^Cy=o;FDwtj z*|4~H6g>0gO@HI*P<}LNAZqwV+l2T~?$@vS;efGx-olrcx$2GOA{`sw813RYWh0OR z)PpWdivA^YiKd_u%**-`@h{B{I+@Y$Sf_`B6&{2Z1dAM+q`r69U@?R;X*2tWI%ELe z_5X>o7xB1g_^PG9W;;$=I^1}uzP!gtI6zY-0l|as&-mFKGnM@?ErzD;>r`81;kt4F zK0l%XcfBX4cG9U8_+KCp=Wh_0_$vgGvHYHK`74mgzxqjNx)e`~@*nEF_e!oZ`Y*SC zdvSZw@Ii&<h1fMax<5$tzZYS426vfHcEhH2Y3~HpA~0ib>{3Y$>s`9tVmMyR{LfQ? z9ZV%Q?Z&W^3eAHL8ZyBV60<!98HDSW5^dGmgixc7jDL!}KYuo*!(cfTF$t=!%oR`8 z6%eV}$`2Q+(1{9v1oew-wR@W?jB`UnuVtBu_laQH7v%TFKaIt^?eUXBR)6+Sgd<_* z&RiI{Uf(~DvxppbKKh%Y?M03FuaxSqkAv7emW<OJ+kZi?qKCNHJePk*kK*r~?0;|Y zpE>>iViHNCTD5J%u@8-HHjq_%sbUrfvUHlqi6wza&SGvHojssh*@g}1ou6$d(EXO& z*s8^fk0(lUe%Qa3wD%6`F_L?!80Z;VdR<fs=OCIPR^5NsIcyl_v3IYA$MRPa;w*So zR$#|k9JED%O})iP+RCw9#^$CKV){;RjOhl9HJ<!cwf}Eg?){M+n-SBOpf~ozZWsL} z%{0)@ZJDO7;n<PQZeynEvVwo{1kvjky703^E)q$-PW;b4&YfbmmVytP5FQwoU!CTL zE>^nz<|W<pr%e7Y7mc|{-tI}uV;SpT-to&-(>H9MzwSpe36IpDZ06@JmKCx8qHb`A z>%Y0<kBhYPDY*6KZ!`QCMtHlBG02KqmDvaV@{s=-51Tu7j5p5jay|cN)Rfel!Fqf@ zH9PHp1K|G~(;)k)jeMOt&i>#u-G#nuVaEgrtnTU#KzguUyI4iK7|A6^kIlgNrJdtU z#b$Oy6LWpds8zw}$|<5TLw7iGwZ8!^va97LgTU-$o1(VQ4B@3-U!p5B);`@e*PRwS zpse9Ue5{Q%{~->$rnCYdCF=LM-#11*Ty-dd_9L^iQo8Z`<KM>0jd=e33AI_N9ZKyz z*yI>JqU7cJh-!z0!nF(C%z))p$y0#GF4rSz+A=Nr#G1V6<f$CRhCOtffX7719#>>J zI+0-q)UYSEsWDRk@;c<SXD~adPGr+OkE!tB20ifGdj5HAYShwBH|J)bgC!4hLREh) z0jY?g6)<o+0-qG|;<Jloc=|=|{}=nYA;)m->HI9V_NS`1voCHmetTuZ7H0_DuYF%u zv~b8CtB(<@evznp@0^D6@HnI1{;nBD%9Z$(wkj$+xG@Y37>?ij7zUuiFBDc%d~M(u zfWEIWHTiXQ>gH^%qgB0Ney>OHn9mO_<b=_pM_)O_#3H$#_75pRUO{ihr}gaY3Xv(B z+QIGHi$YK58wO(JH`HxwFUSF|yN|FASXov9uvc_U0deg~Q>waMeV-qQIK>>6=MNi3 z1nl>EVMO3tUW#65kPK?r3vG8cUdq?S#q4I$RJk<#y8Z}0+7mE?fKek{G!h$mk16Tg zw9HG+T-VQCUlh{C5p51eS-AtY7s0~0@iOR1hf|sJ!UoKORy}4<;*}?o+2c6UDwltF z!`zr}&$3kGDERyUjNVF67KvCjN1#h8@;SmrmEaK#BiOenQl2ZH^b`kgGcH>VP)nSP zP^5j_N<a9}fJ)z9smizL0yP{%HeK@`R}oB(=7%?)%6J@WhAK6?67Q{#p6^AMt&eY3 z*0*__5U<Cdx17>``E|{%v*CNY5R(Ex7UtW-cuf7@1zO~|pdI-#0{JRBmK8Aj0#qin z1B|u8*{>8o-mdWUi|3c+{g9h~s*TwZVPsf)UBLwG>f)`W)1itt=tIN7;uDv03maw5 zmaEpB&b1In`_d{_l7!`(a;)si&Aywi3-P_KHeMMpKy#m-q)ltt4oYUUhwJiuNPBk< z%E(L+SWs~`7>z2Y_R3aDmSL4Q->>c+%2kp)sw^`tj|CxR>~7wYPd;u3mAm=8gy_E? zKyui#)|Mz-J?2Ay167(MDxV#$cB*>U^rEgAMdztLg{yfi@HIMmt@3%y#Uo&Q(AZ+d zlkR-+NHq#mSRd!A=I9y<zUqCjQhEF;p}ZOK{!|O1rAvnL6vuWPU$RlXtMn_UATGS< zs5*DWo?Dxr_sUq>RTaLKq%$t8>%7X-hpHZdPB1fFNalVDO}<j~$J8kJxy6;b@TKpB zWDkk1+CJBZ4YWCWpduTGK{_WQV!H4GtJ5uz!jozR#BnhMkuQ*Rtl}}6UHuesay(kr zjvJ7SR)<(S6yP&7?n&qjo)>%BCalWPZl6Ev_d26bgaJHIlj!!o``80oVCeR*-4nax zkrVr?ndxNPD+{`=J-9aMzK6&L*UI(`_z%AI$;LAZ@KO=Km-=>Hjl+cM1j+}AQSjPK z@X(StLY}9U<n*Gu4kQNnnIzrS&vz=%?^orx>_zX7<W83Fj86}dYVw|xpC3Rx4kyon zK-R8|Be;&~=lY+`W`Du^h2X4#!$nL=e3Qr&2y6@La;X*)c3C@Z#m-N%$@hz_;E!83 z#@-S<NQ^G;$JIA#@YWC1W_F)izSf|Kc4A8XkUd{csL(LK!F!UF-G6#|f(}~kSYAX4 zBW+RHsPYT|kJ5tw<h4(mW=n=AK9>z9C%vsUVaXe(4~#Nh?N7^F%$_`*=O(j{ni*7y zhrBA8%sUk`%k9gqLwB(XW4faEV@DhECAe6l?S^{HRzHy@tRKI6H>M+)lU#T1;(-dk zE2kOGf~*))(3?t2Eg}*x?HafFUN}_X|Mm_Et$3>+KgF=X`H8vG7eV-9c7>6i-hdFL z5*g|iLcG%V*O2x^kzx7iyy?Q;zq(p2uc8arCykAMu(zc-#%S(jH+nsZiTQ>LIHB&9 zwqF~R$xMedg1Mg$D`Ms?+dVVWzTUZhu%zlKBg1hT(xn#f{!K(72S1$5C2c1Qqu(B4 z+%UVyU6IYAuuYpk>aaEOWhAdJvD&+O$jZ(vhSa?r&l9PW7`~id3ah(0-1T81TB}+M zWTi&vC672B@p9Ojrkk|;otdO#Yjuu@`rYl{%3@DS@W(H>YjzDQb$=r+c&h^TWqA$2 zo2(`c=gYjOaS;30SpdebPv#$Dm-UH&w|FM{OL+8~B)T&c<vCl-x8{WN_v2HWcHO<# z9KUjGuFX$ULPs{7r?BE<K9Q$o?aR0NY7Bl<XQ=+Zb-VKuHOR>jI3cpOqsQ09j&rg+ zV`!PIA+IpTo}>eUr&NPa{V*v(FGZc=G(yO(Nk^c~QW>eWUF2<5TC9z9sXuD#v?oBI zF^3Td1Z0F{a&~Ws^&nnkj&SXMV_WI|!z#mHBC4M0ja$ien|s@EZ~FRl-*=h-&$+9> zM>85OKX&!cW@Y2GUaX_K`r<TMd#o9!3Y;gEHg#O$w|)Bul3>_q^5JSi9sZ&1c{O_1 z38Te?z{s4<gL*X#ck#T=kcg4?v&^(G!U@Qg+{De2Knt*DSI;$TgnaH`XMw_|>&UWD zRA1wvTG8=W-IY{M2t3LnoGztN=fsmn)T2AMnv?oFDk`q-yQ72o<6vKvLPX(1x6f)> z(T$jZisPPTXAyxe%$q)eY&P+NmgZN*nX<YkC+8~Eic}#9gpLX-&A7f^y9?X${g$I! zLmm4ghv!-+GqnXJ`OB`^<JQ~0o6f6I9w*2ode>Dm8{{vsys3rxrv<>(C25{`-`Ha^ z!)JezWSVF*%-`3k=jR=ZPOQEM!sgquZ^V!j@W+_h{pc&F5wy_(L|XE$((`x0TvZX6 zWl&l1e(gkkjcl!iIcPd=2$ARIx_Qo@Yu-qHDjGN{9&TyS%alAwm018FY@EpC^lH2} zT=Mi}#%km?S&(J*kC|2$gk}w4x^IzC*r-KJMaiheo65uJXjD&Mq{g0<bpi?5_bu6O zV90pW_A%{O$;|$=n(w;VNL`IpdkKTBB_`d<!~6B<u0!C#q1P9_WsEMbm&L02sVt(L zdMj3O<BvKaHXxRDmb&lS5#t~l79s-JUAYY3bdKlUZ>T2+kx0(o_9jD&Zh;%Tx>x<l zm~>#UeuWJCqch|)QwD{P3yeo6`-_h<J9Bk0JoOsYPXXihL=Bd-f^XHxMU2cOzJ{yJ z5V>Qk*Hdd|vvYc(=IQFyZZ7A8nibx;T&#|YySj;|<L7Zho?U(-)6v<d=kB=*K;tAM zb>k!^g<BInZXlz8@uvQ%)U$;MZn5&zDfRM=Vf@5Mb;xO^u4vw@sMxLp9lys14key= zlW!Lg`wKghm&D?bL8Zu3&qiI#N)Z;T9}|mZ%Ni19nZ=^xpxi4m1vz@_w_P}s2fhYs zmjr$k>&(s>F9A{q!V<@W>~-M9WV9?(`&DYI?}-vDR169_(~sW(TP?2C9Obsu@J6IT zx5sjpqKwL3bc7@*Yv-!7gN2p)z7N&(#sgJutyOM5rxhIe+64jR6z#<vB%8cWW(9CI z6e)T{LsyTg`5lOl7r9-cRJiLjYK`J^xF0;5by|@yu%)^GuBqkWD<?U;wXIEGGesqX zS_6R+Rk1@OkA-xq$fbByvi$8;xb-XklA(kUYE?}z=EG!+%XY2N3?btHtxg+Ff#5M6 zBq<yu^m|fSZxvrb+R)H`n|gF~rPl*5GBTFLp|xNO8_MUnN6~YR#Ml#_P&xRsj#*pa zuUZLT{f~Oq>K0CVtNT5bs1;xF55O|6sc;TR<iSm^n3-+zUyit;0eRp0ajp|Oymr*a zPuQ6+PIgOYOQewge5K6w+h}j?uQ?Hzm0#S1$Z*BE;;{mvnfpt3<7sw?HcF;!Key3K z63^=Zu<NmP74F=d-YjxRKY00KU)n@_wItV8@tvT`gG9#77J&x44z%hYnI7-48?B<U zb*9gQ&BWEx`N_TqsqNa@MZ^fkn%WhhQc-tdb_JclYSZ-GGH}O(-D?{5Cfg+1Yj<JC zFvW=@Y9nrW7p9EJS4B@FlZ@f9T5;fl{hR?@H^^%3gb7Y-hMDEZab7Fg?Rt`b&Y6Um zpaPJKgeYuEg~=4YOaNRqUaLDsS7MvzE6h*2aIT`MdJ;l%^uHGpEAeM73+(RC%!nK| zPOW~tXiCv@#OXL+<1E`OGzLX${AR{j;((Q^=^j>8n))<7YP(w6uz;l$4jp^ygW-k9 zWKSt($Dj{5VhuuoXri(%0p43i!}j_42RT~`$#H$LoUaR=TZi#iP78g!_jggb>7ltb zZ~1-0e&$OC%of-nQ`IsaR;N;w=o9;juOKL3e{19*Itf(H<IRysoVaai`r7Mt5_i4P z#Ibr~NZsf|P!fJuf64ihQS+sP)+c{d<o?SAn^JCna`{}9Uk}UE5_^!$$&G4G``WQE zg3XCDbB=C!zLk>!M`wHW4Ie711qkjsi)LV`mPEf0&T0h^_Zi#j7wwIQ>)yI~%Ws+L zdeDm=^hnE+Im@Du-9#GHWTbphM)iJpMG9<$$Hk{RjL*3ZHzv{8W8wr2WfUn|h*Kr6 z2pW6X_SL0oH}@H~f8~+<qZgKZylD7qx0hBi61LTm(aD*?niW~1umdlB$Fs?;KXCD1 zNb+OgI@YL83-Lls;9Uep#!|YW6VotOP2>$ukM^gYCy7qwmc2Q5eVX_`S**Kbi%0g3 zUg2=N)tkx<3-D@ahIc-)2w=VM@xku?t$-<EdgGli<$U<vOotq>Br^UA3-%0W4jQF` ztT4r$CwB&uyD111{AwIr6haKi7?grye7bSHAISH@IJ16qod`P?z!%x*UNOvqH0??c z&hykwf${t9&5Q0}nAcmx&(4%T4Ch_bJOy?cob{|_b8}dX$xDU&0Z9Z*GM`JUCu;cM zJR_WJ-(gsl6kO+7NGY92)UG-@X273N%lATB2Cz@jXGcGp?y$aRza|^6;d%D8FQ<Fe z03OGJ6niAm!A8nYJzf1XCG+x;^vx$jj<2JIuXM3;dmk=;x-r)>$N?YWXStWEA#y0X zoItx_J|wf9IG`ThUxio{C$fk$I-e8Awt)PT`0MTWOm8i0*l#{+3#piv%PmoLzjC3K zE77!c^1EWxR=1yhomh-?%G(#Kiyv!6<<w2p6yhrymW502Ccf;@-=!+-zOe-Taa(Yd z+yv%r$4>H~%mgtLE{vWrWqL(pqrW>an&q>tJ%I5z&k6KthB40=K&I75x|H<WtD-SX zGV^~O1M)X=e}Ol*V<me)q@3LJK!|p3YA_RDnm+CH$qkQ)Iqds#O3)%!IjIs6U_o|W zXnb9BC42ZdjipNd8*|02*%p3D3pphRpp@^8+T%n}qo6*2rHJDjFSowF=Zr>Dx=*65 zZ5zaO^oEqQ{D(Z`KB_&^9Z0{|NOZnJ=n)ZbiRN2nw3xux6dPxU8jpHK%m`sCkxOs= zIQH1B>SYWi$Bl}4cUBAdtt|D;7`8ON0k+3g{l4+2x|gM!sNgZ(+A0rqQ#m$Y%oD(} z@5Z?m6NYJC<u1D<I91QY&g?9>dAhIF9=YDa%s!=2n;|$g<)Yhw5!#$cK;~gksCYe_ z$C4YJeWL~axX4?5ObH30RD=B5BH|}i0kDk)Iu(Fhr3TjHCJ%U$r<Z%Y_|IhQyRiAU zo~L?6jbNhk?0}}02Th1GKcu=X)8DQ2#}5K*ktl!vh-P|XKgiU+NFI_{-wdzhfQ~B^ zN-~FbdMoNOvnCDG7gm+SKD^IIFGN==X~jV5{jYuX2}1$aB2kwK4^YBjoC13n6~+hs z{zf7_f-uck?8U^HK`gIuTJu+ZJ7Nx(Fj%DErd3lS8al+2iACPqIM02qU^UK&u(iY& zkQ{~|*4J+m?RR3PGk+x$*V!z*P49jCC^n8KR{J4>jUrBhxS=XyzP{V#;lua}C)r?q zobOK4=RO_!*RHd@@lM(zI(Bc7d)du^YJKk^r?BC9eT<yU@hW!TfAaMu{ofz{UDN4B z?AmZ-`o4YGy-Ac|72_G$Y#t8Vb)I6(GO%@VO+oJ7)Crs!-ETO)kPk&oE>rV9us-;b zjI;2E7$}+6^p=-@cSDrUN=r9Y;E+~TEo0n)U2b)4D>9#q18S*}OW&}Q5w#?xE96$1 zs#!twtE~IC$f~z>v7@z`c$FFl1!m{hKUlNa`x#TK&77y>%z%Y?p9!zYuy)&WgSCuk z^2L$AIm~}PGG1L4lciChoJ$Vl{&vk<KHvr;*v9Wcyje;S!!}X*Vvs)TIM;tq={Vhv zIb^VLQ7`7uZZK1p-r!??z`ZS+>Xt>K1F`5x`)#HE^KnZ3LK)Mnv{2SXywlK_w69O! z?Bl8Fpt^eOQeybuE1S32ZV6a+CFV}M5hnKw80yTqZHtXvIriU<et5=E^aGdQ!t>b` zpXWEM40`?nE8Z;P>@|c5ZtzUjeVaqq{joxS?Us8QB~MR@o3H<33x9oi@entFSkd{J z2Y1hne?m#@&pvry)H*Zp;Qmg1|9#7%R~M@EFjGt2f0p8xEAKnkZ#@jT%>Dd7bN1`i z{|@2*p@ks5z1A8-d7(tTBB)XZ?(G1wYIs4WbkR5s_Z%t&)>2cMhK1j5kLAY!&V1#< z(<N0eMf++1R-N4{Dy-N-SqBQCM2(qhx>+UW)#9kdYmaJi%*6Is6HV#VJbTq3Y;qwJ z$=yq^HYcz2<uWsvu71SRD{EHyqj=yo3cCAiWrF^Hx9*=d`u85V+li7I>7FM};@Lf) z{FxqRW`8MbKg}Lm^*YWLH*v_$%8|WF;$SIcANytV)z^T%z25Y91vLtv1+_ZEGlo*! zAI0?_v+zBgmzQH@0X_|7(pnffOd*C-6`fH`yY?n=ZAz%AWYM1;i3!<ktdt94yE}h# zyg*f~yWRJ&R-nel`gqB(it~}T@0`<0UxUr8SW-<Q)o!Q9#_yP%S#7EAAsX^W+SOb( z<mTVWV=Hf!nm`Sn<@N{NOSajK#V{@2T+c*qNPY3~9dWYjxX^Fg6|PR^>k*QL-Fe2A z+XChv#OBnLyWxRryqQ|YO2-rwU1d8<i!5R~J70-(XdR{hp>>On?jjx5U%IIDv+B1h zfNn5gz+fhH*=v;e9Hnesf$YyY7zCJ))>!NbX+ph3H6<jfbioO^I}?~=gwsTDnP--l z&>MOSL!EHeZ+@4L6(pTfKH#*FB|ji`FGn9Y)=|0`&pVei-V67fk16mqY%hO~hW5h+ zQ5Nswu=Sn4O%`!77JB@)V{K)yqR^$Q76kr!_%Wv~UoY9LD6Hdc=Ne8cEyeVA-gN}- z1P?@hA}8UwW)hBCv8z=jBU-BoizGGk=#FR7ouo&kG)d_)GTDL?Ly{WlUc>X~`Ez67 z^rFDGwoHD3R&u<;?om#^C|DPq@y!Odf0y!v>@5cS@4w^v@{(8Yj>dXGU3;o&$jFow zy5siFoJ;MN5`nqA`B?=?YCP&-tf|l#R+w^DVEX7y*2^?fH^KXssa_N6;=jt>*e7jj zVZh2k`pyAU*z;@HoG$}f*QPbY`KuHP=4RQ0_7D4xb%f&zg`XN{er0;j8ep*&U}7Yz z$cV6MdP_}oo2OvA#-c~qolzrqRtTq+XVUuj5bND`kqWhukT8Z~8zrQ^)Wa?KL2*{b z@p%COs&E~em;@J$S)`(rDbwh_l3LaD;O9OXQSR91W59w_PK@+^1X+U`Ew4jrupW;B zZXloogM`}8ycC#I>*eV;mw#PCYAcY%a-7FO^l^RXqDcE!P$$N|7^%hf!4w|$i=xk0 zUClVNIC-1rI}TSCD7qJI+E{yWKY5)9_)xcucT+-j6Bi&c@FtS%K&GgNGV5)T83CtD zEG#TRN2OYwWS7_U9w4O1#|!u9;l8f6*=3LUGzt^`>;ihhf$flbfw>ebj-61_%LqPN z%okv?=Wqezfl2Sdv<44k%?EU@NWUMXDB?yhQDa-aDd03Sq~BdD0iW}4`cc*`R!_y7 zxnb<=on`(x6OP25$non_U1t-%tVYUNBh&r?n=vjXC`%qMOs;o(?Nf>S*27GAPyX!4 z`mCqYQ9j-pewTABYb&p7*keC~kKiaFp{*iSi%GjZuO{u!#%Z?u-c>d#w+G#0S+GNY z?TtPEXP{}eCH$6v5c~8vTi+5vRwG!iVrS|RdcPX-ho!~<yz!DOQ&LOi>D`^x9*8&E zCHvD=(s{hiw=F4M-fLLJqRlPmFF4Q7>WS^$ul>W0=o=0(b`yFavYpfD#UM(LJTnGY zil5IXF(Z$D(YyXWAJ*qr4X7Sg<oaThxl4-j@SI4BHJ)aDY}dEE;$R5k=!2aTwlRM` zp>))uUOu`|7==ypzR`9+PnHn-!x?_sPZKHSC-u)qDfkC;;<gs=Cp{~?4K(X!R4K6O z#ttE&STj?|C@Kn@LYW$xSWY;fT~vo4Dqg_9^7p;`FGW>YH{N>5!1&aF`UfH2r*65x zVFU5WSu4RyG@C@<P-2#q1PYg`zZQ?LwiIv>;YFmDFVOt4<o~vSrj4ufU|rOfxhM9x z8kjQ>_|Z|kUXk<kz=}mg1SSObX~(9E0rr;AO(9JSTFudlVh9BPaLIl4cfb4}o;90< zYfut4wy=ZCbeGb-G}qh9{7{8z=QyFRCDiGajvCuZ$1>+ZMtbLh4Zu`n5cKvz$bU)v z|2%$@k=0*7qC|}!@?rWz-}3E>wrgIHBIk^-@I*cmh6;#<=A_?d;GZD-?-jN7x+qnw zxQytK*!je>z=J)Y6}bn9J&U!A-e1<lD?d+sP|^|FHbRIO1);CtyhsrLIInc#Z1T`X zB{q3gu#JbYX?k{sE`Uy3L&%sOKBV^XzlTpAg)1lLax=n1(W&+dPf_VcU&S=IcDiJv zEHORNl83fN)8t-}<nuebCf^u8_zFM#AQ<AaIkqzyg5Z2v@fQ1<3Ee0*3DWKZU>>VS zUr^flxp<%0p&e#yaXl6BNb=pK%Q(2#Z!!GO#jBvEPvO}VFnD$SVi*bF)WnE|)~3h_ z8P;~gU_84?5U+_CGrMyqUX2{7(Yl#^+B#8YaI`WQRV5Ocy28x2vyV7uQ!5mrr31*7 z2Y~AKMg#<&h<h?rWcY2CS2#(l!lf1>at+O%L$*C!p1x~pB4DdO)*c+1;4h*a6!~o| zl6rRtZkouRjw0NPNRJMxLh=g(4`<5j1g3iDtSrU8+VtCit@`!%9(bLo(m+f%h0;jb zyyijMTOJbDoJ1X6GM;xhEhHSb9$^n6IuSqGis3>}X-Ai)I~*>Lxvsq7)e+;&7y)W4 zf%+ajd5im@^|>+2^!4Ay@Z>x0wf$xXmlxKSjTnHq<ILntnUR|Y*Xfqsfp+Cv{q$2Q zfOCSpg8O{x;Xa)c0@Pq5eKb{TcuA-3$j5KtqqcTrk~xfE*_5oKb>}&r;bamy75y|V zvB$dq(IOK_O51yM27TJ)Z0iQ%<unhdMWob8dzaPR|DOf<&&4d8_hmx={+baXt4U~O z$4XqBT?G>2f}pVK7;Ux7#F@Y9R41X<L*?Wui|2Lw{6u73**{2oC%o5hA*A6%iQeI5 zq4ek?yhS^Esy#+t81eF#h5zgStQ0oA^KIpW!v|+V*NM)bWbg?9+spJYe1o6OZXrh! zD(Y)oG}#m;(G_*<+v}z>Ljj8p*=jbHB92=Hr#&8xF8TLvC(_~wH1Ju*RB<GY)7#vS zWS-d>!gls7hnU2aKUUKcIScv<(MCy~x`SICy46}Yd|QeSGPB!vqkJOK?Lg-1mI*`T z-BI=IREC<OsMhXcH^8CKnPY(~H{E#58Xu=F9gO7>`^0=?azPNK&NQWrvab$MgmGD? zmN%+C9qEc^uw=-Z3PBH5nQlZm?`K179tP(#(pDagEZMXG{T8x2bU!J&P}md9Q!^?B zOscU3W?w-w3;KMc{?!-$H6_<#o8Gni2-G{gJbk;gSIi^!gWYQ>ie5UJUO9TQ*VK#X zvz^Hl*?r`#BPLcaN9nr(ip;#8TgUK7JJ*J#21K3XLps+V*;R(eLotKSu6#R$^hbJZ zvicBhZAh8rhTLHlr$_xQfwa$6#ZK1o3#(wKwy*o1(nxRT({*U8Z$tl#+4`qczmQge zN*aHtm$PzRossT#?B=SsS3GRAjC@}yaQ{}5l{o2o$iY@fL{rCFrWZ}TBjQf7GngAP zt&E>6QYm^jlYTFnNWMdCZ-sb1Ry=drZc_OO(^NUJ0HfC>iM&~}?OBNbHx&5G81CH0 zxk5#Xd?M)A*a3A?Rl_NCTY@|9q&w8#vnk*^O%jRfAd^6!bOkK1C4sJEjZqiSja;nI zo8b!u-G|a?d$ePc76{ITTy3WK_m-S(<z>pY$LWYYz-~2C4nfJ>9L7D>!pl0PMTgC{ z4t7yednyt^iHS1v$jJ$Vw*84NYYhR*EZkNuPh+<1aZ=q}+*Z_12r=SrAhr4Y_lo2; zglNsZi3z6;yA!J*s8^bhCuV?nVTt^jfVPv}6qlFsaN7G0&GW4etaJD}ci=^J%x9p- zYS#RA21-xnS6KTyx+;4DH9J5u95b)0rChUS@Wg*?#i*4F(VdO}48R1Xdt;RBy0^QL z0v*jLK&wFVL?qD+CH2tWCCW^5lYjD`n-UjJ$^!<B#e?vWQ6N%eWd1SOQb?+gIK6w) zc70sOCA<M8#vh%yqHZ&VExPjJsN0({nX2OU+})@c^WcRRKOov12OVqF+X9Dm92BQc zFq$x*^|<*A7u3#KIeW1_yoQ{VGyf)}Mc$27B2%~Vv2<IMN(%%!Jz3eYQM{L;kDkPW zU{(OVl_-rO&@knz{xe=cw$eQtZZnGj->$Bumf~gjmft0g<0c7(%Qj8|JOz!cGfs=m z&X2Y2rsWIU^ShNC=ndz^=#?(p7&(I5*l&P3u5i-%=j2iXapv7s7fvhNry?aj5i*PR zQGDjZ!M&z7yVx0i#m!rP;^Y&_1K-ueTF9@i^eUn0onNa<R<&wtgDU^MLed80nNtE8 zF4C}%$ZxM=YWKjsPZx1sFV`6`HL57k5Sz|;KclajPduVwzIDzSGTJv*?czoNaq1W( zFRIvd<!iuvk2%WR)~144CQ$R4gI=5zuov?wn+A}@8}=p@*i#bPw4By)*F1<b*9qxm zzNK5aJKhBXLOgi;>xAJCbx4B6_~?~g{I?v}+IGlafz)th@-P^IdHcEG(AA<TFDdXz z_LbfExOS5@p3>lOTf&tGgS@#y+wkkRs-EFEpbTNjK73pyIbP?yF!*?^C)vWAvhwsL z4ApP(LsAd>{%I-yEA}6UH$xH&fx?ZO^mXL|H~Stvw9TJek75fNXMVUj$qyN)<1Emj zE(I$B(wT3o_H`n~kB7VDbj7fa22Plq-u7}rz;3wJ8^XzU!h_kl&f1r3usmuy%ZAIr z6{B%SpiXQ)veE5{XN=m8V}@(D%n*=z?RKW$c)lEQskuKN^<;`rjlAM;PaPwiyf2@5 z=A;wa%9@)L0$+FP*m_PoI6gu;IZ_a4l~H(=b%vUCIf0|}wa_Ckd375rOD(?Lp2$WZ z{55=>q?}L4eoZJxL~u)}aZmNL7QA0C2Z=p?@z!G{2E5kP9O<^u2Yx9$9~;k1kG)D! zC_)m`atlrKLo2d<q-e(<7{ehwtBCKP$p=u(eRrrzCoABhY?#imQ-dm0YmcQj8Fhz8 zknhjDb~&FVrYoe0M=W}rV2^{uZ4+B!vDK|!!ka>g-sEiTGkR=m8n9sc0^)f~=ia8X zd<`aata2@qoU{>@z=52uVeqUwI5ycPo8+k3$#GuZ(JtPk?C-E!)wio=uW2}W>cJFZ z5$K>vB(qSrwDWa+FR4$k$_^rWHoP|GSjNhKYS4ZBe7>fWi`l42L?n0N06I;5!%d9_ z$q}+0#8<cxQp4;L`J<w4iBRMGeYU_#69*11$7{X)6)u~j0dG&d)?;oV1xr&QQo@Dy z8~&Vw)26>+N7Kivw)O8XTe&P*d9G*?*)*OBKvn4sqU8aLkiya326qLb@I4_hzQDb; zg}Na@3EaZWrBN46q8RKi5`*3^C3!^e*(tjUYNc<gQ%X3mZd|Hh*Kv1uAox_P+0O}u zCY1}u!Q%T=ZRbB~XZzZ00&TQa4rdOua~C?YLv{}7ol{D71wmIVEp%vD3$}H{v?A56 z9zrrHsrlQ2c3c25qI<3V*j^JKPs~bqc7OE3sw(Ww14+Ff1d9{ZXWCDv4moy(9>^S= z;=`Q!uj)W)sCkgR&~cTHBs)F$7&Dhu^Hhv*g~XX!Tg14XVf%-Mb72RiX#U@=;n8hp z6*W?DGR1aXxiY^2D$aq+#=x+SwO+zg6uzu@*$Qwe0lMzuF&%a_o4oD)k(Y$V-4hkD z1@-KQZOi1i$cc8S_<^c;S>+N#HQ2~@Cgy}aB{->E)G~H=)0q)YF;lVIVS^}zuRe80 zy$)Bqv^{B;CcuzXd)mG~Taz?GE|7*dwyA)%CG#3b{q2StRmT+mBmnA#Q`>0c?xsl% zOJtFW1WoOs6SQ)9m|!IvrXpCsq_^RY4z02>dR}+(;QWc@8-9VL6O6CPWCKUTZaMqv zoFi&5j{XbiWp?gD6m@21Zhp2FOaP0HQb0uolPqW`levM4Q(Ll5L?|EO;ODM42?|DL zI_vBzu-cP6N+6f#GhxsDu`$=#BlVNI-wC_#`=<EYq~JWG>;h|yaP87;WoCo5XX0d< z|8%ao*V6OGVYefg*Lmspdv=yWybT;zi#D_CS|`X!k(Y!j9|cmX$^leov<@nBs^+$8 zQxCR>%%;onU@T1`<U~%;!XXDm4>t9KkT;HMv~zby`CsJqvL<%4hJEy)yDi|VKXsC1 zUkGia@>ABWb=q8~ol7&n-G+w{&)sdf_pwv4UmS5(RMTfTO`-&woL|`Xx3|o(oAP9H zw3Ha}uq1G^R4*1?UV7n4`P~X#YFy+bsdJ0rm8G|8+-c9+qK74NaeZsm?p1&H!PxF@ z&7E-vZwWnDqQmgtHY|@5FZL_iov#~@h9*>VkB)hKttayZ)Np2b_%j5cX%wwOH~XIc z7}vK0yh~_VIGm|I-B7vVjJPX9lz}%)t0?beZkB3e@q8E#PpU+Wpd^9{#mYr;GT1(O zxgJ)3brxT3-Dxp>VME|*-Yx~1uBwVD!#JDgn8%@4HfYEQ36b<IE_lWdyh(5Tdn2PZ zArkY%#&!cyi41F(uo;R7l&CZ_&~iHKx#J=j*)U#o9<;6oh@q7?7HFUAH8*I%yte5L zGoN2HuQh0mxI5^alOwg>z+UXxjUnfS2INw51fWyn=i#Zr#m-8G8;B!Y3o-kgABym! zH&t)dSc*%zbX+z~g5rf)jg{hWO}E(z&o@={7Mu1Ds%kUQ3-D-ETKLCsi!qKugYt7h zGNVAd`L#~MX|HRJ=@Q^WD`>Fho`XuR*{AV6<Mt0VD)s3SC;as#LLmptg-PAczjK_X zC44Ofeis*zYhvl1+3d$XWb2H_G_i}hY$Kh=fe*4~{bsUGv)gpoN92A~<r1)6NUC9* z6(P$SO1&^6p8L69Xf8!S^~G~TJGAQ=8I*qf?nwyUC#2d|0S)hA`-?)iA6bJ7LxonE zwq(%*(xVQAB@G``5}&=h<jr$6r=xXjZS4-9hqQ+4)a?_;O6kqCm#+m(jTI!E9}kuC z!y1i%FV6heR>oj)eTn4ZnF_M^Y@BAbdGu|EgK6&Kh0q3~v-b?Q%2YWrOWBEN#OWLs zs13jEyKtr0Db~5Bv6v}CmTJbQ%U5cJL(&SC+wwY5gAN<A8)B~a*BBK(`F8RcDoqYL z8*OL3u|AuUt$(pTcV$oqNFR(MGr*LSaOO`;X>K8V;;~xZuPmU7+?$n-6!~;?ShZ2# z|5FkvxtM@pO-2UNXuH3(GSNe6+xyCq0ZawZ9$u@Yt#{hv++m;Vj`vw}6!$v$y#W=M zly&R6({Jrr<R#-CY?fNK4yw?u7iKlYS?*Q{dN*kArwrehT$YPTf>V6=ueKAe#yluq zABc(JV|r0Tdtv}z%g<z?9c_D0O)0fdmi-!KD{p^3AqTe5tQ0;dO9=9}E6km0#?Z#s zf*7r)DeWlxg9=DeQ#>bv<RsnoV1dP}l7~ZcXww^p&BORy5O>#it6#;t{R6EB@P2*H zJMB0=_a8QVKCh>Dwb~e~7SYkF=MSkUYtN0ZOow+&=MK8|^-kJSYwAYRIamH5e2z@A z3&4yEBG*imNaUxFUfOg|B3xfTdKf?#h{387)?7GDMBCd$pY~p6Z?t3^kb)g-xE&iE z1>=l^179xW&j0W(=CV=f`*_%1ycW0rs((0#scn88aMtxgE{fHwGcB@9$g=O_y&9rz z>@MPX;JMq7)#spins6kWe<!n=xZ|Sr#`uBUhS(GxS>Y<CqY0mfQs%kFqLwLeIub4e zL?%C*S7Gt|HB5PT-+uf;Hi6z?bdRA{tMOvjr^ft}D_=;Afuo#;0ST#IdM6lNR7DC| z(~RnvwyGutIVxiDKH<@mk`oUQs9$*wc2Zl@$NoZsJ2Gfxvd`Vg70}TvX2bTeHJGeJ zMsc)CZgY;dmN@+iW0G(d_Vp5)skMy-CJ>Rk)n!tEv`GguF&pu!y&o0Mo^VWzEgV}t z<*7Byr|{|SYVPrmkukNqPQV0NBQ|Bq&<<GSxApMVuWo)&)V||d*l&+=87KPDtANjf z2z@s<nYyE)ny%jEQH0luUnDn$*a?fsP^G&<flY$^n+q7YQ-PC68wqbRq{?g^6TGuJ zXj2cFYAuc&)y|ZAxxcm2A6W2UXJW1(hOa=V;BF!&VN>+hiCu6&==3NxyhcFB#RzN# zAT#cRW|aM?WQy9N7KbMC3m6NOwZ)ak8}=W#ca6PqUZ~5W77>@;=znVMurXOD?x<>} zw^&JL*nG;IU25mE?Oq}Mn`EUGCy6-MX)L3gNh=*GK-`gmR;U*6n=iv(hkJSm`L!Lh zHug>0c6TfqVmB(+Z<>3PFK-FQ_>7=q8twJ9#HzX4gUy5P+EI3Fw4Xvv*o)oHj|1hj zzHiQ&pTO?HUkeC}scw~v4Tb`;v^eW?;;Ba2V@Sx_q0np|%Jz+yC@$l-VEaV$WQJ9p zCQyYE?x0xgJhNXG=IpMJ#4n_b3LwXEY7f<}WxaPe{9?4dhdG$b(?Eqy0Tr+=Gb#mk z11u&GkNjPL!*0F3yD1*`%FSy%oBtjfC6_-pUYY@tQ&@zzR@Wo@mLfOfm9^)!T<7H? z<Ey4Rt;H|nJO^FuzqkxqzspsB2(0eTQS}puR<DY1oP|dl4)fU=x*9@wdt+#)uevG& zvQJww9H><k+-n@{_v@uWRxY1lG6|k=2YV-)tpu8doC(AJnUNV<JRNQ7^1a)w!~1%* z<93X|ie!^%N^0HSue(QbE0c$uiO=Il+8PG;43t|IV<H<^@=G686ikxR(lRYjQg^g| z6H??dMQ^NZ<?|dsrf;3FX4@amHIV&BY5b`%L~}m5f0Z&_D=ZRp>wLS=qxSv}sJKhv zE@||+`oI<Mr|d=h%T6!k%_qOmEl?9$DnK;&L~?{RG?YDN`Wn19){`Tz%%jo1D}sZe zYo`an_PXZ@ds}yfSlWj(;)oMhbJv||%(oG!{z;G*e})Yxy#w2|``S(hEP)BQ8yiBu zgYE#O<H3U!&Kw*Q1T!VuKiMb!VPHv;rqAcQYaw7VhFARFC4T>>y(^E0a&7;|7U%FH z5@ng_6s2QGc4np|MV2-r*{Y*CEsWtXgc)->Wl5vOQpk`Kbu5#zFAW(c99t!_jfNq_ zG-H_=#?0^0dlZ^~fB(Fn{PldE`?>G?TJGz6f3NF)o<ZhPhb~NvpPl*8F_!7KBX82S zSvNW!1jU4kpr8fEMy&rN`=nc))m|>5EUApE)oN#(U?rRH#$5q62kWC-i0hu`Db-pZ zt#??z#wKvLj6HG%m9Lx&oYt)M&p?FB)Y<a0M(nfk^fk-ozgXPvB%+CeW~!}KW4Uj} zcBXUM3HJWTnheEHk;QvWwyoR0W|}mGC%3z8acefQ|4Lcs3Oi^Sr%laY>&edv4ea1Y z!<A}(*&cAGcL5XYr-zq1M%JY2>Yr!5QL!k|{5v;yA?Q6Gk2eh%b_OU@ju*uudhNXa zJ4E-<?2iP>0;B02^=2hJHmEpv_+$5Tvj9#nA*pUSJNvxBY1**q6D$(zh?2qW6K7r7 zn44>B)A{mmm6M$$FZ1#7kFl)<tutK6dTpolUB9ZCYpxU%XJVUy70}ruarGwvqw4=E zvAN+!e19c+#$rpWyxHN_(KyvE(_cOMx(5{>J${U@sOV{mW%JjOxq7TZF36ZwUM_cI zmDo*NxBRTv=Dul%i0HPBFLie>RQ^%M@Er-9Kb8J~T{U7cgeJ`_Fzee$MD*)1Wq3W3 zmqU#K^WSPaA<Y@NQ$zeZsAup%wXv)NuE*f49)CJmBbIlsC1Q2n=qwBP+H9mz2dLe( zV{!NM4KO{!cO$0nqekB$Jg`oAcd2U6a^j$2Z6_HN%E#aPim#8R75DdM?z{;X%pvLi z5moFVP)zFf>cag~&9O8S<EP0h)wp5<INSBy3sb--&Ub5GGrLpXzUk$-GCQs!W-gC{ zgGgQi&RN7IC?j<)dJgl}3BuqSCmqPBT!+Hi?LS`cK;khul7XY-mM#;IPkdG^t*><T zy1a6#=E}r6Hcv%;$+?FHaZzxJ_C~70Ge68J^5R9O*+PAnT}`^t>;@|2ZpklpK0Eka zD+8>ZhYvQad4My9=4<gs>#QgQ^L%aulTNKPp(P?I2*HGPzv^^}0Oj<N+W@M^hWZ~3 zXFiX&$-4}p%>z}k`S63bd4dl1nb*U;Lk0DBbVrLoee!})wBVFw$+ODq?1&L+y{%uW zYGPfMk-A`#GYJjbJ*lQl;OB$M_phgBZ3p!Xtnc*-)#6WMtD&+%gz>OiQLbi?@&%I! zo)=kiB;6B-p>eLZei$Gu_&EhWS{39qgG#`Abe~#p+yZV7zLH1}+x<2tvBv0iQktF_ zYH`v4`mkU!l@PBzG{<2fv3Uyer61oZOqeikq|7%m=bR!m`Liq(!UOHTCwqGH1TaDG z1M_i8jz6j+?yifM@-w8p2<oQ{FSlpgdR{;0GOA1@$U|or=8Fr@JK4V3(|W@RJ6D(Z z94YvRL4h`jpcV#PNf`U0qOONSP$e(ka4~8cfiWB)h|)~U3}=fqgL9r2c!tAy6m0n9 z$styW4$j?hVc+D3+K57Q*IakW1nt2*e<mXHV()6i64Uj6|Ik`2ETVb|EovgA@@{B* zLN-h%JK~kl;5f>wruVNOZr6yl5VRl<=xuzm@Q&Dk5a<&J9O`#q#sX2)oFsDZltb<{ z!sUvm`Ce4nbo0M`?0Ex3L2`&@b@k#j2_#^RG#2v11ZJ28zQ_6IVx}Z^mV5WMD7DB0 zLoX?+DD&|R=Xa4dSnSLJUk3|1p)v0m;n=>XrD4V6^HHQEmmQ-gT1s;go6a9L%7GWU zI5P9~ar`%0o(t`z51uwujgN3j1ygv8S0yxpU1=z(uoA2Rjih2E|2&ZcfL|F)G~S9A z9)5W&_U`Obp{3S3$IjtWjHCus&7|*^Sl8Q-dDVUql*VLc6exBU<e@N4X^VcCtk0XP z&eat`L4p?X?uur~X)i@ke%cSV=3go(vVlK5@$zveI1bFOCj=wyJEdNop%>*FW0?1E zpcZmAHWY&v_Vo!0@up>G?RQ>bBABCcq&E?H47iK;;S(IF96b0Z_xmd&+D2N(<Vew@ zC33CrD(+&napF`!W6V|5r~|qoFX2T=9{2pVmy?Yb0~5k(T@L-;1{n`GI_CuC<gpV5 zlo1316-`dB4jUTGv_?G%LSE{r+Gd>pZZj!_W6$isE+&T1t{%Ujb%TWJ=_{K!Mu;c! z2ejL|Vtgf}eh&&(YH#LNs=R#OhP64h>abz(jKr>W6|M#pV3lO-5v`<2=FT+~ml{#D z`P|_qyM97;=Zo^<lHr@BdP&uRr!Cv3y(?{y*UyKVF>31ycPxJJwV11~oyj)1T4Cku zW=?#{Dn-=Q^A0c4{VUG3jdS2$oI3-CQ2r?rsQJ~gkACw!2AR>?qf8L=VJ&7eDvm?B zM2M0@OW;o3v_LcId{-Q>SExLCDqoIVjq2-4BX5TIx5alXh3!-E^u@X9j(}c|BVGg- zZpo!+O8Ugq6pq332-mlg@td<ETnD#a-u_lCMXfYv(8Q;x5F*ituuS>b{?;@hNyDVY zZN1>VVbJ4i{(*hz8s_N6$@a5M@<<rX`k7xGk>8<AlZ+?gcZ_9TMtWJ6vz)7M4Qbmh zCX8_Ez!941ypHil>Fkmz)`lH_LDY|Vu{!UEXyEX8#UmaOm$z|3Wy^;hgvcMDcqI%I zhyMYmsn!1Op34P%Ud4yX###?siqHNm3s9_4SVbFkk#(mq8zBy+f&@dkJ7A7-w^OTU zCYgLYSl{|o46Bnf*;GfnEc*|%s{#1FR(0|`ompI(A6@BpN<}?vE}M-g!6(%r{6emv zy#y`L7(#YMz0t9a4N;I_BpOzQ@a1up8S#?**MxfeFt;p7Vq*xl2i?_Tn3WBD;xGvk zG>$Pzd#S-5RW*%ww87(nZ>ilT*x*3w>f*YLA8!2^bb!~@JD9#TYb$KakIN+(x9dMD zYTJ7FMtmHo&Z3@nmyJ$M$+o$8%3iITgsz#2aGO5yIEl>pEi`oUo^EtoIjzjK>XBJu zR`;fh!E;b~NKWc=)mOzi^_-%k9Los26YwuQ5nfbCmEPfu;D7GEXmT8OBl}YDO2)?O zt8u;NM*t66UD|-OUB`Lurs=WajF7&v28_TB>v=Vl^Ql(kKhe@@S|0ydk4s4z$6yb5 zF_NYcrPYhOU1~_4l#*4X3P6`fhb*_`nFCe7nsU)lqff8`DF{2ZM7}Hpr#2PbrK6#A z;_-evEa<mnxa?@u8+zd4?N{}c7E28COMMxB6m~Ft+pI_G=bJM7aB8ezqIk*X5P8RV z&6Uz;_>0ID=%yWy`qFjfg-(3r%`(eCpTAb#{DOr#?y!UIrjKP;h4nJ4bk+b^O6W+_ zLE)`sz`*`T)mH)QvH}^F0QCdT9&QP5^#=xi|CjmCE4kdV>abAYM4x_7ck9}k>1Wmq zdk+l!;!x1RWg%A%fnx!%-fRdzU?;Tog~Xrat9Qn4E(^KpA^LiGg=9oQT!PTntI|?2 z(%PMx%R;VtXa#2!V&tEc<rS12|F-_>*%c2Z*aG%XFW-GsXzK<&fQ){3ERcnLckJ>y z{~ocQaR3Ow?-9EUV9x)GP>ildME&rC^!7vh_a6Ta%RnCgzv$%ej{Sdi?19O?@@JM= zM#AZZyv}j&<AH&Jnd;+lFn*LIL6;yVgon`Oo&6rDP5+v(g13A<Sy7P}(PTzG85ho* z=<)!oI-4GMdlg|?MsP7Lpdc#ma$h(}R0Eh^z(0462yOL-0W4AM#oxz-2rMp@U=7eJ zeCTdFp{+8}KsE?-4HC{01C}_XfzyFQ%jmki^DdEQoCrV@<)m^Qg;Qc#+!4CUNKIUT zfN#xw2(1G5;MH|!L<yV^R;cC{d+Etz2;oMQ!o62l0GYamz|yqO)1AD4Nys@t@^jqb zW4c3)AgJO38>H6W(TU%r*yX`a;?OG@u{}KoVwZopzbf_kY-SlS?zFlB`|zyMGem<e zglBJs1an7QJ;G?Y*u{Q(2sMVT@KS_{Cb2)H?*35Kvv`ldw{+|6p%EhH;nakS6>Hz6 z1=5&LfFtGNLxYpWwmi8o+ha#W_cQUXm-`=xl)V_!lhZI)!}68cl<{L-memwpA!bk3 z*59>{Cf%ul33{ExVPO5KHue~u#c{zO+%ET-P)8LvQMi%ArcbID)S!B-ST^rugiqbb zT@&=%NUzrt%PZ!q^{z$LynOGUx*+*z?vpe)nE120?6oQ83sEYEM2*ky4Q_Y`1&3#E z{kEf&yBY3u;q9_0^i<kO<~u5<IK4EPXGk384GCtGlf$1YMrGGfFm$m*Rf~nu_Nr58 zn{QTv{&DeZ^>(z(-6yA3<!oQXvyqm`O(fe4jm5(@a>lzz(tms#0I>5C*L)6!Kc;KN z?t~xdak`Epzy}Qlw61{!$;Jq-lVG>;rM~b1gKi<Y&fw9dzLv<TJWdvPalzgQ_?b^3 zMi;*p7n$<;_2`%0^4%msySK_UgRimsB@O&gTJyyHP+s4D5)>p`C50;~ISVY`3NVJ7 z&L;n20<MlWR2LF0gH1omG~aYBkI`l0RGZ)h`Ra=05GF`KB<1yGXN2gqm4t0w89hrP z%BYLbw_j^@ueh^!i~nrV<HQV3B{ghMdg$EMz4}_#Uqt$P@u@+yE?`?>fFhEY`Gn09 z4{%PicjV13Qm=;wo}-XshQLH?;D~4KMDi*y-04jUx+VsZ1RW2`oI0@{4GP%<VRnRx zlOb|nT=nJRiYKf*W-ruco$Gd-n!JEX;QR1cl2%m(ho{fweR=BN3*7r%`n%vuqo+Yx zVrYi1FJ-4({)DPL)=Z*7)z}X*`hY8f4z;OVT{i#Rd(ck$U$ZS1s-|=i0ke5=`iO)h zXK2qz%!}63=lYet@n+4<_0E20ULJm^x=BV<<EV+sxmw`k(zROa9q@K7_}Rseb_Sw_ z7nVSK>-;-+Cht}-a;;WNv%1Gc1}G|L(OreDCn*^N-0AY$WhDCck=A=k$eF=HxQTC! zfG^4ZPhtF@y{k@V{?)<TbZM1@WHdYU&6hl^8#5gC6o#HUg*8uPVy>qUrStdbjhoMJ z>J#lne~jhH$%(z2V?Z|mMlEnZlR-4P$<9Q6BO9((8`w+21JCT=w!RG0_`=SAp`T&3 znLE+)Ti3&j7lbGpyFtZ86XhAhEeG=)Z(-&)H;9&kNyPzdC(GHxYNr1Fux%GE0xwNn zRBni3gtkR`+KzrZN}ZFp>z`yr?L7x%-}Ut1mCWld$SF6SY-$Z~?o-^D%vvXt@=f4- zcT2~a-s^b+|H1iDP3wJ&tn%{mc1X|x@39s}Y|>j}`ERF@kdE6`2JK&2_4$fPuo!VZ ZG_TgQCDzcT?FZoRkd5R1f_>h9{2yayx_$rv diff --git a/site/img/dcos/tasks.png b/site/img/dcos/tasks.png deleted file mode 100644 index f5bf5ceeec48515930b990911480122c2b2c9885..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 108349 zcmeFZWn3Le^9G8$yIUZ*2X{$uclQw79S#;OIKd%6aCdii3-0djF8Ab>-F>s0|Mz=; zmoI0SdAfVLy1Kfm`soRgmlZ>V!-E3>0YQ`y7gYoSfeZ%$0c(VT0^V`jcUS}g0cAB4 z5s{Y=5h0PcxA|yhX#xU56>X%i&muub)o);+uirmHOATl5q8J<;rl{XG*51?J*FMzV zp`Vtht-XeWvIef)36iDIQD=b=M)FX}&|ln1sW65mAcJAf+eKl)Mrp;6>yFVL7o(Yq zko^i`XJ#*?01FQO28ER>Apyf@j@8F9nk*yB=ZKBOnftRBNVlwCd701jX9zCvc}5@0 zfUU4@3@E6#NF-X6uoAq4A&52D5ZV4uUS60!X?2Qc2sdg>EG$fw<7}0{pLcK9+*t%A z%pv=|5k~JJ<K8Fj<5Ivyw98AWoDECOOe%mZm<_grd;t$O7o8*P9!+Foay@~pKrxhb zz`?1V|Nh0U92}NCLpR8;J5#hAg%yQ0jSHR`9Rfm7RKza10yPmHI|z)<6XXh1@txJv zn<=39(#(|A9o1!|xs7bB7z~VU3{4nZt!#nf1p(o8<py3_nK&AdxLR3SJ8--5k^Kw7 z4ZQw&n~{v<Ul2zNJ~DL~c@hyDdlM2ihIb51Wc+X>BqY4{#vi#AML+zfIPgC{vQLhV zw%m-2E-o$%F02eT_NI)?TwGj?Oe~BnEc8GGy@Q*zqk${EwZof#SMuL_L`@uw?9FT) z&1|eme%5PXXyfF_M@IIup+Elq{hlVSW`DM1?eL#&0XxX}^9ds}0~6yPbpwm?{=CaA zZ{})ZsV-_}Wn%3BY=fWq9Sbw>zY6^A(VtEJrKrZAMcKJH|61}dPySPqm+@y0{?em= zi|b!^f%4*q<7NCq_WW=Sg^;))Ac7zgqC(29phwHFt|}_=98UtqY3Rbj!akpw5~IXB z<prtaKO|*~A1M~p%9m?)D7Il&HwCyaq7CK2?Ex@aRf{r8urSQ1Yh+MNVn|`oGI0FR z&`3Y(ONzrJ_1m(xH{Ec#3=4w2jaqtg;cK{Ea=4lAItAmeQ7@i6&V5&|-~)!Sw+7}< z7L=8lS?OOEGn54aNhJshCl7p(@yF#iy$p}tRWO&oJ@^a8ilRNkPD{d`;Fq$0fjE=2 zXH*b+hAe^oLmU6b`PQQ8kJp!wb0Pf}0?LD;KR$=lDr)Wf*LVC&I~W;)TChx?mQ-U1 zf_@9}A%tQ2WC^WDR`>f}Wt2eRytAg2h!FWzB7d~jjs%3%`ZGpr{VA?L9{xv&<Emf? zcy)2)ec%6(=&ulcXdq;<BcY%Ee<lA{+$UO4sPIL}n9biq%E5t<<29)*tiKQ9f2*pF z1M%;*q8PCIJ!H-o2ys56u!sL|p}#k|An7BtSsSy(`FqHbppTFd&mCF=<nMLj|5uye z>csyG)y5}_@W}UHGhy5fB9MG-tc)+D0$g});}{IvAJoJ|E!x?YBhB@`q^T*AbWDFT z{bOen9k>mD1Hyod%S{tYibI)rSSXT^msbnFo(>^`?dPT}&;*AnTLvEDUfSb2q9Bs6 zBoxHkK`G9}n26+?l#n+{|6{3sD&vAEP&3&LEd7P-V=C;oKgM>C_FTM2Ut53@v4??^ z&+v!kJ%2l)HtZL9U8$|JcCZ=V-a1BRAx)n4a?k3*=*mL^X4my*dkjBm)m_Hfk$~Tv zj7%iuMNY@~N?iH_uI%xWpa1TcMVna$KyU~F*>`i~rxtjK@W*fu2;}$fn@cBh<pNEW z&)QrpzuK_s6p$gMYW(A4L~WgK{w9Yr1}ch3g*K<EH?6c_)8d-%2$vgs3>~1_zBdV5 zyL=t)E}abn-8>K`crawubu+`&)u_PjV~jOxaysTAD&qItIXP}F{vezycpX5Oz1*U4 zdLZp|4dvBZshv%5ht-Gd6&fqj7nK%AiQ8A`_U7XPZ+G9_{`10S4bp>lVB^i0xpn0A z>Q3V53aG8!d0;|lm(|L^$*=QyTmd3kw(Wqb=rHq8Ki;nIX+!G5#T>q9rt{$`o_nWm z%2$oegBQxY%@Semw&~n;gLP69z2cm4==`NFkEx{$9t^5GkwJYWD6A`$6fOAc_$Fn# zRaL6Z29>!pSf(~y&T7ZoQg+T7Z(PGidJEf$JB1xj>7CE9W8#aACG#2fQ~s=|m}_Lu z#-k-8!)Dq7aPSo--mG59y6;oIv1#-9qvwszht*;0CSG-eiApwe8b8rp@jRoBa^$P5 zADtj!LqkvD4=T77HKHMn+Pb*-`p0(U7Z*2&uaz(DQrr#@#%<R*q>rc;^K*7d^?r3H zph|E`O!lwhF?PFf!jo5;)c+bU`#@w&zg-!>lLFrC*<yl3eXXDo(c2{vs|~1;4C)mL z&S^@7%U{x(RF3SuJwy0)*q5AuR{50MZ4#XYV!m3ckL%D}g{8Z0lP;8YKNXB7Z6?F3 z{+endjfes_nU7hl)apm+Sa)vA5R+X7giW~Bh}MwZQ|K+zW58`SYf&9>&!z&y-0f<} zS-j7AuppZ&;!N{4LTv6v!{aVq-vqRJi`oP7)rb9%kznm(zpHTAGisRx0BkQHa<j@t zuXnB$4dY#n%c`)@f|@$-ris*k*!<Ahh=l{Qe2Uy@{mzJ#xcjr)le1EwGuc51rmOe9 z^wo8R<sBp1-Kg|R7<8Jp0p9aTGtay0^d|eG=PQdV59dORh&|u${!4cJBFOQLxA?wy zW&%O!Cj0qRy#B}O$Gw>JSa_fBOo>Fqb%-X)ILTO+E(i;nHyj>o_0=?A=x=rcn5%a6 zML!{@@b_q}l9?^q<*SH}ydb{)?4H4N?R`GbA*K49%6t;yEgY8qrqa1g%<hUe42CA2 z?m$>Hk}xyC;(Bz}mc)*@SxiPuIyh+i<%_oEay8`ch>)!dLi#<-MrX;aqVJWLo<W%% za?2*s`_5pHY+_FGO+FtK+Juebmb=B;NuN`g;m7N-9z~n)N-wZua@1XEw;Uv0YeCT7 z%+NnbRohClwKPtcpH62)76A0{Td02<XA98yy!ypepV;qn%^y(>o1DHXLLX3KUFO7} zucv;LB-%37@0z&pZGGUI+)OYO`>?4m`r!g<IGxC1+JzgdL`FC(%^Z=|1hFEQ%~9m& z%C%TV(kkwQlPuasj`!0OVt8LXD?V0_k^7Tmc05iH>xr9t@%4i7sztK87Jz1E@VPPk zq>5#qNztYFNAECGL{f!YL9rV(i)3MRH2LhiA=hj?Tn8kZ-pspw)lu`v(5P7Roaid> z^NRC1x?Owc;+kZohrtJA{HC4$rStRixeT%M@(-<%cX#i_Yw-$IY>%dDAOUofl0x|i z!y&Y?>nYi{XAcfBn3l>Cy7vfV(0wGH+0^|!Na=(gLj+6=^z@<yH;0S(Zvf{k&MjO@ zZA$Pf1Bq=(76>>}4IxE24_z~!Wao3}Ge=|k#SmdCQi}~59<w$a??KSZ4<BboKsU{k zsxp)rH&Afc;7?C=zAyWe4n;kjDE0bOfaXA&xN8v3qI8$tp%j@)aqL%_z4v3*r}L5w z_cDG3r{oPNDGQ(MYn&}(+rMMbGwB68;k!n|9DROk*ue-is*P|M2`Zmv`M?6#669ok zw+19l1+J7Wi8^+C>@?2VS`Y$`?8dwM&TM1()10eDLV4cuHMA2jl@>+m;bT7FJ#~6V zK+1&$eSWgLj?t|pC*fAL&n!uP)E5<R$H{MRD0aUy>jTC44jFbRuUfnc4f{D#?hcwy zUt8_%<9M5t>u1B>;!m4C?jQ>oOr=y!P+rtT!LfY2<i3v6&Ckk$8h)Fb*FF_O4@rTS z@adm%QFU!$VOXHUYEr7PRWdt4H+M!^p~DbOO!Ao+dUzbkUkvO8q+E!8y7PZ^Lj{9S zKG9qkiqH)+U@Ls{eQzDDZd%o#bf-UGcILaETC^{?LXnTV=nW;S@l7#^r>=McpqjU3 zaGpqVY<`ivpIC2f+Lfk$yCi$>21gmnL82-ZHPRtj_o6<Nk>`#w<#17D^ejO>5@}^S zeUnvWgHo7tDmx$*rpJwa`KB1_n*$E~%>Iw*xP^TJy10AQc_h+w<5elNGob_Lr6nev zwuLQ#oPmaLs$0tdS11bU5Kpr~9B(t+#n@np*yxYmf%yuIXH)g`*&u-$2ST(hO6JG6 z8(rk(s^22%^J?{nn7mrt8I4xyhfwM$pyxSeCHs^u=MDEA%!i;CTv!b5C8c2|<V$R} znIV(0IIF<C9~?1ec=4!bDQvm52UYsVScg=_$FI>?#<DlQz5;OZZx8DPZgx(Y*yRZm z4EnxMo?q#)Y;g*M)e##uLIAQ%+kE<TS8`q0TjO;(zl0aP!h1g)(XbOrb&3vv5xwL; zS+8b8dt~XRgtZphIK{_yN1YQXb5$^z{s7j>8a$IWK}qrM#RE6ZRfqWX9{G0dRTjqg zuzf(COH}oBRQtKZEl1;SbCdP_1Q*x1U+QkhX3Q{ycgkio22;IKQw3hN1m*M^KXSHu z({AqSSdUo&3-x-9j>|!w$D>SdkS5MA3*y7{mmwy%h3ADBn#$IWUaJC4SjZsVip1MJ z@Ut@<KM7Q^GZQR*nobZA2DH^xzadSiO4eR-aY1!OI}u9R&o&FYRA%KuWx)keyWM8= zUb`QT*!Vw>aTs!^C9>e0;Rg<&7R&(-I0t+A5p4QEgZH<6{N2B>y7w$Gw)q*qJZ<*J zTCUqp-UWtMMvT=sao(o;nSLBdpeW(=;(0M0<2tUNJrg@0s_3viViN@W5kp94@neWf zZLK{I|Lhxk5P0BdTI6}mHt(kk(d^Vm$s6;9641QXxnTcY+M08+k1tSPyZBI8@{wuk z00Ay0$-(pR%v@8D4Nb3K7;A>DT!zPu#yI2d7?5-)GBQjVp}RcBpKVti(&l@Usq}sc zb{r8vKjN+SoUWfOH1zt|A<X0%Z|;nwTquRbUc=KD<+BsU>x(o~>o=o3R)$``^vtWj z#_C==N~EUkYsEBn5HO|H8uDBePStZnf2$;NA46uc+()unI|ws20=c2KlpCAuD<15+ z!JMo<)WWj-5Mc&8(()M5k(qZDn&OhZaHTjuTG&&p9;Ob9w<`rzM$}r9Jmx+|oQ=0j z+o3rLFwzy0Olw`;XvS#Hv=PRiRe=|kDu6lG;h|uhEC)RDXXjCpcCtZRgoAE!c~_`o znC9t!w@dAkF*e&Qxb9~to#4HgKsOPco*~sxOzGO-$lzoogs*XEKk(YW;6lE<rfNQ? zTcsMWY^P+T%(%UL4e+9@9L8-vU`TIA2}@*xFJk|wf0zQhAZXVmTzxx%?s2vQpw?ja z$bGu%PAR(0P}YeqUV){_nw`q-?w&{&pawO<3?6Bxe55F(9x?5AEm;cSe#wKrJ~QkX zSngHCI4I6Z+Le}i2OTy&?UL>Z{m4=k<+g#LI4TYs#?P=IlULUov;|Vba1T2d=u&&& zO;mpq(qW`oT6}=z<JO0EzB`*1-8c+=-aV$*=|Sw}Gpn1R?<{6{^1XKg9+NZiiH{)r zrL?f)(DvHnNN$$ZRRk(vq4;SETMrMs_rMVmiPBoEVkRrnc$OW2qurkYG;i-`oE_TL zzfld$`SLZmt2zk4c4qR!Q`)Nt;^T03V&FM{P!{!ZK3THEEUBw5-Xh#_nlesjEA8+> zhraTuIB?s(G8>SKe|sIQvZo5^p;9FN3T38t?cQ~Af+W6p8J>oFItTq62C6q*b}~`7 z<5TgF8dsVyGTzmG5Ox(^as9EzWz%4IxR2>A8G$ZK-*&9rdD0w(^+PhKQ5MeXugZ$? zQ(0?l$iOWUQYSuon0TP=M8dL;9)&Vr&Ki338VsxF=*=>0ksJBmFhea?%^LbOcqGO; zsfT7GZ;<bO4SPE1>DQ<dBt9&NjIMTU<g#G09QHgCrxvnZLJRRO?O2afEbZVdy7~B} zxe?0XVFFQIuaY20N*Z0m#nO}4;D??Ma$46Yg+#vwFVut1{anSz&>1(oC!FWExLP%N z{JIhQs1Fqwt=)+Ym&{fj+ykoo2n9@}Keml|O_xqv326<aGXibKaf+8J2#fqoKW)AP zB>Bf~kcXYK=?sWPeYrdFgrmil33BUYf^}p}exos~6_u<<MkuW|<pm$zYS3zjY)?T( z6iT7kGI{WLhWMrZGgl|o;w2*7SK^fX8obZpU?u2t4_IkY#3cjZ8)@E-SZ^b61J)H{ z252e8#6O-5G9m=CKz;NIq+B~Bgp~<9_S=Jn<+N_L7m}>pYkRyx*V0`HpC$If@<r2D zrJ$*`bYD_W?(`8M#@(&MRU!Ihnp$->tdK(M0})M0aj;9I4f?Ey3i6{(*SE}jIiu(y zA8g?cJ%+W<LgPoNZ*=f$_=6j6QO?%;g>@^mDeH|^568hD>ZkRU^<XEY?pS40-`cD5 z(rxVDN05Gx$ti-~_O+;YoaQzzv~KuPgm$QM9p%+RzM=VvFO1b4AD$v%cSlQaJfez) zK_dP{pHuQ4ewsx#HM+64YYmksRF^Y|R#$|nUqq)N7=6Bws$ySY6Uun^V7~33@EWh0 zVdtUeLFIXx@Y(OG4dg61)%QIOY^#EU&?`XqViv`{DJBOxL}r<Qumr3t_iOkHtsNN! z_WCBp+zFY<yIkHd)WXQvm;MpXh-c_zZS5=tSHJ&tF+d<s(tP$`tyWran+~L>r+d9K zd7&LBKeH(&GI=wXoBQ5-I@Q2gN~YA}?A%oXK4hX8MUf|UyVFLK<O8Y!6%_iG2YBy3 zGHellk#atlH5M;-41*9*=`6trY7cuM?!S{WxYmY!+ZWNX6lcM_>YAlm#|fa<?xO4X zaga*T%^=q~=n9)0kyL83Qu>v5V=HXiJp`^mthld7P25Mljqy{b;lN5<%K*ThMKL=1 z>-QE)`e-!#-Uem4OTcbFpk%Y5FcKnSR*J;ccOPog>%eN%9hT`)^IMeu*#;p8DQhOV zfSCFFqI>(O<kZh9#CzcC&e^NjmxJ2(Vz0II(g%#R6jDXC1m@FP_z{LqNkJH&gx4xK zI5w~6qnT1AVw2~Gu8B;MHkN0%#k?N2Q^dV3uspN#yXN$|57_*+9c!k*7+7(xUM>oS zJ)uerD_G%*NAU*-N4&25>&SWTsNMFy(I}jUrQ($(<WlhkxYV)l6?!=`zXm8Nsikn| z9nfjf3!P@hA1FptR{Gx^XDcLDIYRF6al$EpyLa}f?3_hse1B@Fx?V*2{Q4A-8`v_m zE6k)UNu96id|f!b<Zh)e#i{<{pka#Q0bM^8Yo^hdI~vcQ%Ua^uHtp>j-ZA$e)n_-# zgy663hf1L_6bOjcM5w2GR?n)$9)S+h(v2)E($+N^L{U`<-kwc5!j52|VpexnpWTDE zakHG>{1OZ1ew(U}+(wML{vGBF%RvnJ^9KPR%B-%6{cpV{yqX*=9)ELMJz$m&UMp++ zJawFh4+siEi;e-I5aQbzLc>3%6QeD-(;YY^>J~I6{E{HDF0UChd#$rpT4BB-XgI)S zLZp7pY2BXaVJ(;Kk#_<3P+S<9mw9@k(e6sNEBiI`sQl1BtS@jJOlZH9EU7Z)ElUe@ zQWVpO%TFX`hQj4xw@}+Dj;tHp9EpbBq>th5Gq2o9ack3(Ooka?rxP>4r36SPZu_58 ziAfrF%W03>?uS1l$$m~^&5$wOTIT*>>69FVQLhAweF{SV=?nk*Lf7d89@3Qt8N-u= zU++RX;=Xv(C&6k6PxxELzCG;P`76M4%Pq>bVxSzL4$;md0l#eDSMD&vd)Z0SuQzLr zRsStJ--1$;qk%UW-bjg#gy_e0dxWDyuv1V}`(F3aKD(`>!@WbzVC3BrK?5hh;z^&_ z)$qPacF?_~yUQSlb*1<S5ieDJ3q=?fmltdRmr+t{F*N$wE(Iegz!gOMv3Kp7wB4c> z6YC1#a$fVgRiG#+QUhl!`E5mVfP`J&2V(a!x_aMX?(@d-qrvc*d~CX09Kb8A=*V|P z*yH8PdDhJiwBF{dLcz3DZL6w3Z4Xp|*%2A?iW2MMun*u_q<YlJBp1Y)B7$hFi=;qg z$>vU-^P#}w!1iFz?NI3StR8Lm5)p@7=#o}UKu9s!DVJrO@ndmpE>%~-k>nMNDQVr? zs`?Ilf+r@=<|h{l0YXZolXe&AX{Uku2zL-xN+i@n{^0%Hu;<+*yCuO0V(R}*<}b1O zp4?jQFtAq?8Q&Gom2CBPs=>ptfkXvWnvLKTBYa8B@J`#Y`10m3J-qBbMkpmZ>%=Zj z5%xi1H5YnjNI-qLgs8S!wbHKW%VktrCE=l3n_;mR-N0xYULYdwv~3(3;{5!4IKVmI zr?MhEB)yFASewj*g?#t;43F=w`BNht=e|{6=F3kPQ?*!5RMMUO;v(DG){Z-k#VO{N zMC0qs&bt~t*s$Hb1pGGk;FFzGL2sjW&X1#T<R?<md|ThuS*j*ywZ`UL$<w|IdcdXD ze=1f$;e<;+_7WT9n5&%?@Fe_EdXNg5M1A409i|JFampBPaLq{XMcG`7J0^`ES%Kbb zR1cwJGd-ayNuvIYbso|!3OiNE4g2_#I(J0h=RIX{1BH|xdYfMU#1D9BF{RKiwk{G{ zc(LN72epd3<b9S)VIDKDp8Xn63h1Pg_GE%m=rUq5yrbf6J#uP($*A}WQSAL_*2{O# zgA1L)0;kwCTSrVJeTVoFFJ0WUpAX_V$tv!?e-_R9k>rKcAzPV!jp2Ednamjnz}`P* zKC7r`yhBIbXk><poCNkGp#KvYiaWp5Yt0%V7vW|x98}Y7da#*)%>sITtkVx({SbGO zG|!7D7EJ%<?TjOjY@l~P;$?HYrl{KLEY;sFhO%zbu-LFFI!q*h#{A|(<}P7(2>qLZ zXek%~mY2XrE-%@m6T5C<$9DB(y}7h?X<3~y>*CT7^8W<Rz?{6T97Lj-XWg8aSGZ6B z{T%5vIT_xRem|257XIue&2vpHD=hBxWlhjvQi7%z&g5gC#9OPdsY^af0M$}5`M{kH zckhx6PrrDlgJz5kLZ|U?%+$z)DR?iPZfMSAmoedFP5ON(wvmY?Q@q4SyNPvz0fGoY z^+YF}ZfEslr8$V=2;bQ3xv=bP67quLEwjUtL8+}bPTv5lUt*2vYD(Xb3fU+Ds=k;d zfv;$>zo{{9wHgRXCG2(Ydr+@)ACZ$`)71^%%I;zJP3;PjVO=r*Jfl6CX<D-1&yT?b zpWVWb?d@@vZ~WrkWx9s(G<L5<6*qx?p!etjExA)tVR{_^?s~~ea$I$L)+s{EG|mPR z*%n9Y)W;~XHV*Fvt}-Pbvp#vFUtqjn%DystrA6kP>Iuu(z28thjV)_-`)*o|qPO*N zE6DZYX-KiR6Ms^ToYZT80!Q1OY+Ot(g`TehA3tZcce;uBd09izoD8eTn3z#7s#yIL zWj`-6wc${t`0f;k_vI5O7^<J?7N~b5rZ_b`>^o?@*~kg0yDf%~xIBT<otKQ#=H`cj z=eKu@Gme`zhDUNQsB*HnAg3Kl=bm?;w)9GT)a1*Rxm0C{qFmORo3$Wg`JmQ9umPbP zPr{sckRmF&@+D_?9QDP-2W5&p8a;YLldZiC3h>$N6>kgy=W@YR45CT?b-I;8*TtOv zEi(!R*MiMbP|`3zphx-A%$0~lh@VDvhbC{xERMiFH(j-F<-G`mQ4fnWWO4XCT`l8{ z0zxfAc9ELH*Jao;Me^v{FO>$%cb{K{H0Q~?U5lO>ghe$-zaS#*;f{ByN8cwK^Rj;O z#P2?m{NEkNjAh8ix<`c6b}AI#`wLsP>G*E?&WKdBO}BgOrXUUy`wpy2Q>QhWZ6Yjl z(^~zxOvIQ03v-H#@`i2qeaHeA0@C>&bgZ&*hBQsApg0NK+=%>9fw22%_^Yu(cKfKQ z`#9($tN8}>p^wGg>z<gUZ0rSg(OG`26TQ_KbTdr@7APADg(_*C4zTdmcs=mga^-MS zs&D${INo$sg2x(*>uf=6V$K(RiI>Q&Xbt@qoYI7a`ymZ&zAN*kjpf#AKckVmZ;k>} zQKD;!B_`f5klDwI-z^3DZOM~oQOnW}&e2K@WW0FzD)QH=u%udsv*?FbUtrXfX27d; z@RVyN9pXyzB16D9UghltCNd1V=~un1Zb)}8uliPDSF2jU%W*1oKBd;eVUll%0e9CE zIcwL;XBf;&nk|oriG9t_Sm2dK_f_Xh38T?XWtpweqYSC<T5GP49j>fYLkhI+3RJ3w zFQw|y2j6p!h{&T!3}E(ByoZ=LeNj3oEH`29#`I90^mSols`a&}DdlD!BZO+~4ld>9 zTQo`CJ4`_~$S}kJ&Zs+PmXabWoC{lSJ~^k)=~t8By-xjnFBJR|KdO30EE3krbwM%- znr3Xe_oW_O4#_AetMS?<Dvl;3JnI2<73ue~#sqv8Px8U&GoU68*iw9km(<4@4VSsP z+jB23l;VuV<Fc+05Gmlj0j6youSeXb2l*`=_zN|SQBDUQuq7$Up9c=jZEYthIo0xC z8x?aj{iH#S&ZS=Vj8f|wR~;!=e_0;o*H&O1o2ZXhWLA4VGH`{twsAVRttXl}puA15 zJ*u`(tRm`>xU6Znc!`LEVV?$zV{k}I_D@Bw-2*kcZ#L8;ikF3n$wIRXzV)X#;YD=F z-5#p%r7AZjw;7WSAjguLRo3WYDq@rV$~JeioR9!#rM|yn_?53`FzqTSS7PT9DTQvG z@UqDukh+88!wjY|UxMA0Sy6P0NTr_=N4rfTbN}*i_u*t{P0)!|giBXhEy#nZp44e7 zx#OV?thihXEA+h<-N)Pr+FDa1J9U;N@_c?%FYcHfu|x*5Q(XKVDUw(4l=H8~jIg;a z;$<Hk7Q@<+t>~Gk>^W&@uA;QGBE1-UQ!a=4P%$x)u)9>Ub0o%{4{LY)^N~lpoB0c! z1rZrirA04v`p3g#xPnr4mdG90{offlgFU$ONU4O|r`{Fs^W|4$PN`4itqs>URla<T z_gKtZZh2axvm1MOfQwDgdLR1j4fE@K!5Pz2@a@VY92?T;*~LYa^agiqLTnvB`rJWl z@`b%JJ38+%--()dFrp9Xt;N%HDyG-sV#77!=*gLrOEkYzer(OuyQ+9C<cUs|`Ld>_ zAmw-QTnKUbW(g9S>fnsO?t%)2g3-zSAXC71jtUy5*dFlT!5ZnSq|dXZ(eq2vtad`0 zDNFYmfsfbEi;a>GsVjt((fe`~OF|c+l>zBiC*H1Z!R&B$zQC4f!C<!@dN0ufqe(ko zJy!CSfoH_`lgg}1Not*AcpO`HKOWY;MO3ko6|BA5aH?AQhss1PLMGMF^Y9*fYRqCt zC2UFdDwQWyowU}sgtU@mCJgr^$TMH9q`p4q(m~Gj3T;$YE)goaxG{Gc?q<>OAPnUw z_tLavX{QFZkuyNC54Ss*BN)wO5}1vsNGY*<#oU&=W>c058h5fBZ^^>q6tVBdMOM6P z#j`UkSlCH5hs8_Du2C9kR)3JzYbBJ+8EuDNAWGgJoy7Q*(;h$e5=gHYn5iOj0ApjR z(pP(!ZH|42J61=;>SnP~!qY!J6EVFZc$*<&;%1jsyc#$p|4Ld?P!_FUp(5X4khjCX zdchBU1qbW*eH0m;T)%k1%%RuL0sQ6jC0iU-bh@)u<*FoXYuzQ?aU&#>>@Oyq7KUl^ zPS<OQf+9a2#IC0KG5G*{)sVhWPohRtnnE0JO86Cxt@eQCIoXpv%`MHmM6bqC)r)>z z!6B%YCyJ}k#P1-+JmtBtp=jH0K~c8XU@P{{uro0j>T*Y40hpAh1l{s-<IyCXQZ*_< zq%n1%(@K#Wt-A%_ANde_H~O@g;JrGcM%c<hF1@KZ+b>XzCBr6et?J2be!o~U1drTS z4)Q%06h6}6conIJKXl%a_;2RI2ds{|U3zm7!lLS=HzViX*M%OT_yMwJyw9<LP3Tig zS&^z-y+UoNSw^jAf_@#GWD|R^VNt`i@j3!s#FUgvb4(%wTcnJyQNen%V>EEfC&edt zhAh8fs=e2(W^p={JzWzPAGoXLn@*xn^+9tc@~m)p+JgA?^Y`SoX`i8nfU`>nK{o*R z%@OygkKLtuv+)0l*ayst3iy8P-CObkGHYka`@vd<G-tesa(tMdr;z!X62{7>Y+uYS z&rU{v_G9$E#cujnzYiEdIPwGcYk5{(KF-q0%4quM4CT~TIW4c5!B^;*)62DeduP<7 zdXSuAG+u83I9+1<bsG@Km2uGnQV^PKMscYHeSgxG?vTb^;r{`LQ4j8W;$n+z^Lu8H zASe$eknbehrfwbjYwG2XJYoAcAeqUN(!BZq;5&VT1M;2n^yuo||0=XUM6f0cBr~<5 zv#tCed?#5*f@4R$C9I#rghX-H6z#z?<<8w{_~K}(D5efI`@aF$zj#A};K2MP2gthE z?`c3f8a_$AI`@maE7ap55KDZXwpA$(l$y*53o$-Tc{WlyuHXLNd0!8b@#EZT4<^RE zW-DLK>{GfAfDHV%NSBojO-F{zvN_Wo^59nlM}BjU%~qvqIAv;?_<o(KsHC%6FN=O~ z1as=3ZDuRMNRBnQWu^5PVv#c$&0uE-r(p_Ro|e6|@%IxZd~s|nde!)uj~;VNt7ly$ zy{!G(yY>g`Yg009B{y#mmI-Ytf+Vwv3%YZpIMz}2d^)<^NxDBim>Yj97F2cSTU+-z zQy_x5ty<RWx;}U4LkY&^LdA?zAv9vvhWsamC^zzrHwtoYz8|fw`ErDIq@f1Qt4u3o zG&DE{NDI;-*64e-h-Rt48f)GwgS2<J)$N#ZAr~~a#LXA7j|G)$MFm+LuPg|9Nvo+M z$V0OsF`h$&#nUQ{`^L~7)COcWXwl1xPV4)I?E>XhO5fd~vlA+RNhrg#w(rJAM_CeK z%|{fKqVBuJSYkbnL^yuxaN2uVM%4>_StxIPq8s{7yZ7uBp?!{D*1i^vM~^0&q`E+a zkYxrJ<vf=-_sWI!+ZNA1bqlyF*;oxEl@UH2wmFuA@Vvg&XiB&W;O4<+wca*b?kRF_ z=Nv|D^Q1))Lc>_T4MNL4W+k~_H;4VKY7_~F^T@4#v!Dy(RNDEyF4WFYk3L3DZ5=xX z(sHrT?&%qgK)@B8jp6|lIt3}`P8c%Gq&dmvR=7S0B4jmz<UFB<Qw1T+sWk!vlPSW! z6CF+ThP8yU0&hPOJUW~PzM3gbkB#A5{JG(`inZIA5$!h~(b7hoT%A21bq+-6k(vvj z!yelX0yJX6b#6pR`L*8)9oI=RZIpoE$m-65^`0O%Z4BgjWVB|$q+sDjoqyokSaJE9 z!SU0O{o_n)hhBF&(*t*6LnFUFj!3JB*RL=e2#mpBSkJ(20Y=}+Ja1cz6Er#N%HRTt z&B-m{uiSq~f%P6V;`j1f(e&tl`&L|9;EW!pXy0jK>w+s`Gb3aqQVU8G&%1}r2V9g& zc}Q<&b#ZQEiq5M5a$`Zu|Eg2?VQ=@q;@ahKHomW@XPSW8mw7<~tdT}(4wQk&+SX3B z4^fM7(0)rh_)RX(kc;aowvQe9wze$&=s@d*ToJ;)HO}oJktSBHRE9t=MSnpIcWT0+ z<sYWwmz`%2OUWM-S>SM=We)=KC<_8GH`|B`{Bp*-OR$UlS{QEK^gE@dZIP=WG%n@X zQCrxMw*$`^2O~X8hZx!yfc5S)C6_=KAX0{&w`v!~eoxIJV0y_Rf>9JQGljm2-PP35 zQ2DZ4Hj_boU~8L~njpOgkjXx;<<x!u?FcVlL(SJ^OU#0FnfXLTubVx;s#q^pP3hR^ z685<rRHdJvrjHZ}ree?)372BvCZzDC(z3T}V`|MAkp0hGJi!E1{aicK`@(Hf7sQlM zW0E50lVcX`iBdKF)s$(~WnL7%Q9YuoTxgL?Ijps+I_0@}&^MVf4c)5CB}mgE*LbyX zZuP0=yR%ZgCrn~j-t|1@69*N$9BR>}J&rA?Su7qoCd~=~IyO@WzT0-P;fKgl>G8cq z{iShV`X|Wc2`&q1GtrFO60FzOfWK^->2I35vdfGyPk9w{PlPLC^Em6}cw!T*ZR35| zjgiBxRr0p^*?((mlX|~B%=#D+6Nh=IPh*sdY%C4cdQ?ax)AzO34du*KINBXV44mEy z2V0#kK5SiEaPVFS>-uC`Ci3E(hI13}sed_R0(Yhul#dzVIicWy5YIjL<tVhf&Z6;T z8%C^pNRc#}lbv01jYj7?8~>sSvAkOZ{cRi6XWjRHuJHs)HoZi&US@4kqB2`2`X$9Y zRnLj=bgl82T+v;mX!2-LMtCRwBhA6_xO8A;4rSzT=Fm7Bgfw$iX@yqPV$Ku>ppm1K zQn%}T@;cw;B%<G7xxzysE`GhJR!hwjkyw&MWt#ou-a>5{kliQex|Wh=M!+hAj8k;p z62v93ZAg6@r4zXX*5cZGCU7NOTrsp%SGLZ0S!lOyj5$vA$@B31;B6yGbz9Jg$f<Cj zgjI;s%>0x&X7A1{_!vOfcM_U5n$d~=39706*%I=mxQ*I`onW1a<QZ<NA_o)ufPzE# zk;9WJBM14nor!}`Ps?QWUS#XJTZ<kNFNlU-8$oK1HGA1pQ3|r3SL&z2cBAhyrO~GY zM)5CA96K-BJ{4iq6x7N*E(;e^^^KuuOz{qY9|y@q5UO6oCBn!P>~RB&6Rx`}ZGpGs z{@lxjX`(OtJ452B&%DT|&C0$)o87{^AC`6lpRy|7QmQD;!;=2EVCq<K&n^-k=#8&Y zgAC7c43PO)yHE2EUf{oG{rEL7LAI3C=3pvD4}ZH<X~-!)qIbR4U|jG9K^#$^dRgAC z5kzhKYZoW<)HcStd~)mjLvqR<+c3KSv$mbHQCe(leaHa5$D7KKplOXuSJ9@7yy5Re zj<?hB(-&tv)ad86MBRiK+a+;%f`;_6&8d+O*w?Mp3+_HbS<`DObN7bKqbi(+9gn`B z%$iY<2dGw5g7_Hqgw7{NZ5c~@I$V^~%3B4*$Df`XRPv>4yQ(gq)(FXj)>FAH-J4MC zkh#db)i5WF$66WdKW)=5<<zq97j}fK>iG@{52*87w1+6;TF0_NY4H&NcjkF>e`6Zd z(1CpQuU?xc7xJ7^mz~3yP~<PIA3V;0{7RA<x4xF+NO`W|m>AT;>C><Jhy2ImPiPlG z_-OCgy)7Nr_DDh1&2)Q%j`s;Yqt4Qu=AJ4+=TG}10cfb+XB{wX@p>R=y43C1MF+N$ zM4wH~{dKQ9=LF*hX_3dP%;>|hgM+YiJ4&6dISugSZUokmA6_g2V~~ZiT2L<Kfjrg1 z9-{dwb6wcdM4kaL9%qE3ka<R(4plcC{lU}lLwJYoqnwuor`Er7yZ<xkE>16OB;Sbc zV&~`Qi?+nt+1@%N9JCy-R^MkA`!@w?p<hmCW(p&tG`(|cIe*Z^8<SUn_czAz@Obyk z#Fv6<Q7Erk1km7-c+EYKRUUd!*7$OA_G!eG-T$(y1?}ICP<&fhz1rbg^FOU4?JJN# zYW=NE;$xQ2Pyg1EQjfe&=<iFaU=VhBsnhs6_7J4vuglP%Yp~!W(EnX(Z6){zNAf>) z3g-Phe4yl|qfPf`xnIWJ_Xg;V*L!(!asM77`|~J7-a})(`frUe*fA{-^Y~!z_<P8= zpA381YbU#kf0E9>d%B<o#N1qO@BJRa{gdzi?ci|3@Yn3mpTepx17eO3hkpDX;z;u| zD4L$_>G{2oKj{H6i?az?zosMp6ylyT@SuZBVOeNs&Yw5`%2@;69dG@4UZRXqEbsS_ z0$`+aqO6CFE%kSW_)n<}FQEQd>*+=O5{UoVZTSqPpXWphS{WGP{~liaS1bO%uQ-+v zyx?R^o>Q*UX)K77+q=H?oL9(a<@=LF#JnD>k<nczK`z*@N&*J+lH}H&@NI<CM1!?P zqaMffmMeGA<a254-guXhW%`$^E4@XNT_d}$fA@U%V8C>EJ;za7!AgPWcJ^q%gl^?J zyMKVAqV0jZK;Li94#4uiyaW^Rp>C3aEyQ(Hxls9u2=lxjulKk;!C}$|(O%YGTq1WD zhwu~WLOq(_5#hjNvef776b>@H%l?p1jh<E+QN9L>!=Eg)nH%8xw*ftg^Vw`kPQ3i7 zzI%#b%a3a>@-7*zXq0WwSlB}d3pYAXLE^P`oKH}_Bx^_37%QKM<BIkvZ5#Vh-#Z<% z0<<FBH+vDMX7HFkwF{;`GNVtcOj~g^v#X9=vlCa&gIXlFWa=?hPkj$~bW8f1HjU5x z)ae?kzMluI#y<<X2zci=&mRn#0khl(AE~(gul#Hg;2r(GKGYHSW>kcFdyC$jB64tP zXleKjhrNr%xeB#zAwxM({2UfqgX8Lr0hTIhx8De!I>bPBX&;TJYGd7k`a?i)r~my6 zA~t7+>Wisa8Ynmu(>w8<C?{oIF~Et2HbX(BqwRh`FbwRcCZ3HCR+tPXg9|04O?Nk4 zYcHg>32ZY`QFVLv?3HEeBR07EwUMLgZNY%&ackdAqOS>SWsFF1VMdfw+KlZX{t~W6 z%N(kQ2z_lY=a2}*d(e!g8BYtCsdraxXQw=sbY4)S?S%=ZhPKE8oIJCcd!ATms{rtj zq<Dr}F_s-~-q4P;(R{9SzcDLD^Yo7`Pc&%8vQygadaq;V1c9v)oLsxbGtub*;bSen zWyL2dx5N0D42M&<mEMy)>PFgm{AuX+tf8oQix+^DRMWLA^=EqmdjT)#_gU#ZZ?Yap zNY`goY1c@9HPUk_Ktp$P6K*d6JR}C(s#nmY16^#jJ?0^(Fv*KbOF{NGo+@-c-$U9O zO7C^*rPF^6hQ;9-F9i6tA^Sbv^tyPoez2ZKyt_Jy6e0yj#iGDuAee$#UPI7x9eRah zAQnvGrWo;FLqULZ>IvvXP`(ZkF%AZ&Vbtz6NfL`2yh|GNR4vg3JUydK)G@l}5p`FD zQrKR#5hM2mNM_7Wd7|K8d?wj+;=r|$EN<J+ydfe$`y4g=hBHy&lf#RwK(*YOdd{^w znRT!7Q00?{zUrQ$D5RVI$>HMm6UNhZQ3A@b$#hQ}b|>H#2}-P_TcCxyV_QP%Z2F?0 z9P2EBzpVKcH8bHA&*rFRiMB%5Y+b#^j)Dk5W%Ur#^JPD;eUEwpGX%xr^zx-1X)N-T zwvP=hLpH|e9Q@8=Ls3*qCUeL2`p{{Q*@={0a?JaANX(VIFvyO;?Q6^M^RgC<6987K zwZHRh4mB9b=|lCCCUf0~M=OIgfEN=l4VWGHDwy`wa$?^G=--AV$7j&O=wD|KeQ&Vx zn}R13l#?K#ufp*nf<BIR51T15xvL|%KQ@WC&*3mT<NN%e(bv}bEW*>;J;T`6{W9g* zFa|3kb`ah6fbZT~{#Ar+^c$z24iD01kkmmcvhcHvH}+oO_)5&8wqDq<F!(_Yh*L>g z{}Czj=A@*FcHiplQTxoXp5bGMG(9{QCp(-qnhz1PBxIOHG{82pHn6N!LabmIg@8x9 zl$*@6oM>Q}t6rw5)kzO;>f{OPCSkVLg2q-`Q#I3spw2AZs0-3kmml4TC^1B3(R&Ju zfc+l!Ni|~Uxz7{G00)2)zf`U4cD=!_=KtY<^$EFF>p7E%iD68p!w}bK7@RgbrpR1Z z9K?OSNK@}Q2WoqlLjS3*?#jpRK2rU<9qg$In2XNM1%vh0wn4zr>znod(r2^!0tNe8 zAqbCye-W|1TFs@~17Hi1(&N2PF!j}Y5<<qoCx@|G8VZ7*;{JwOcLs?tfu&#UWf%Hf z%NZ$M5KacVdC`nusry(v{o*SA>xz0q1w6OO2Pj%X2bwNWNb9wYf#tc@Q;1O=Dq%Ax zK7v$jRGbOyHydvaZ?dCfUTH!ypRbQrisl>YPsr!HR$lVyhu^0SLfz~KkSIy3^Y3Jn zzuB}WML@G1eN{j<4-}VxFLC=!CO#;utULUn>!Ri5drJ+%M)PSTNt-0H>TI0x+L=p3 zzvR$|5^pLnLh==DN1;-e*_k!#elhD-&CfiYU<C5!1n<br$%Q>2+j9=UTIU^5n%elk zkF5rl0mnr{B?mHMTI6CO^V^Fd<_r#aT?nmi_X{s3*0C>iJbkyni1Lm#;!Rr%iZ_4Z z*v+oDMleRn#Ro=dofi+jM>W41@3h-MV~C;~#Q5Py47UTOJAV4R;vh{v=EK(nAG`8G z`e<M8lxs3H72+ax*M}lk6Hi7i2SUB)OCwjOphixxs=#~)Hcpo-57*~S&nIUCP9)-C zpTfXnwll!fZWzoo>U+~xk>uShdoTVAB!b!5wP_%Q&McuOVdwTyIrK8$^AmlW`poOW zCZoZ~YrduB--7#cQ_!L_e?FV99zW-hZ6stPPp87hHz&-5Xgf*3lzpR=tc{?zXkTGt zhfr|nez@|U(2um}m~eu5+Y#9X6<Cg;0P`(u{Hm`ib(NOd`G{hx8S=3F>0&NM`kb`1 zZR;d^C*(hj2gq7e^-LT}ww`XOqFEzA-Ph(j1PIXc0vE3ohzW86&$p{j$7gM*+#etf z_r|uwMrYOZ#2yCT^s}p%6#ZDP^Ue&KLb$nz(fKBTd>z%CKBIhB1=S!{X&HFGGmHs) zbQ9;$`${J%cxOTHYO$aD&@EwG7nH7B*BSMy;}3F{$C+{q;pfkXN(ZV%&!cATqVPUi z<q}SGiAukY8KopWR?T4mTJCd%HwfoLc<Pl;EPwpl918L#E=@vBj)uPX&-DK9UJaDa z515|kE)I$z-7Uh(Mowp|w%x+~3*86ZeU@0@{}!UR^ZyJ&#!lm!GC?4@69wgmm7k=| z;qp!&(ULHGdzSNs2jW;c6SV{Om2$2=4`#CUie64AMteO6GJh_9a7mE2!@Qs%!p|6; z!Tvt?7N8kWJbbu^zwgiA`eM6BfBmJ1aa}k-hSJIqx<Mzd>C8D{%_eEPG1P__LW1NX z`P|HH`aX#>S|^uYs!3s);(RA_NTqW4QheTxD8l*$24OE?MG={6ZVdg09Wu$|(6`(Z zfqNauu*U)PbLIBCg_@0Y-ZV21Q=|6jD=y^&Tnibn8qM;V9&rEdZfhlIH{z+Txx7V0 zdW^#jYUJW7vo~vH%bv>wOsWZ?6Rq+3MaD={$LktHs)i1oim6qNWAfx3yIG@&v`Jk* z9MXC==vVb+atr*YI96J<DuYxqR;P@kXCuHj)82=r4)R>EDV9nc%{yv%F!t`Wv@bzE z3}_vzo&I>@)msNV6*<xjKFOXAs{I-Om3x5FJL=yyZ9)AEMwX0xCxoBR9sPrlD&Qav zL+ztU%HUwR#oyI&>TN$rqQZ>fQ_(bs32#P@t*Jj*Zt0hN4w(5Kl2ExsS5j(ww-*iL z0{O+8H!{~AP=>u<L-BxgV#M|Z+OV(3@Y9i?Bkka9uidq?kX^J1BGf4;4l^&55K(Dx z@)grZIef^VLNaZBZ>YZ3<&Z8-9h1Yg-03qH+c|zvqqx_@u=U7R{hCZ%SN)6^riSLP zi*_f;x`l=Z-n{11ru-Y3%Gvka6a11N(h5=61u%+Ue@vK+cZxBKQ;<r<+kb(uyv{^o z<7@`7t+-mF7nAe_|Myz}_F=bAkHn_x)-hSlS?Pu&UlPuoO8IR%iB)8_Y@*`J%BIn& zU5x_5zCSDL4~IcBn@oWBb7evQ;4|5OwQg<2>&VJAjK9Lo8p({bD|)VdH%4-4UNdES zNlEl;pzrex$22(yXwy0g1G>b5U@F+G^0wXtU^!b|#nis_wi*X`YSUemBrsm!6twCK z0Y|j?v@{|@UMOH9K}MBYj9XZ^Yq)c2l0xG4c0#SH>2oCwUISe>m7zgo6LOH&MbXqX zUsKFy$I{=QkN-G>$NMty<~b_KMzDR&7bbSJCV@ra?Pe-{6iDSTk5>$3rG4o7+J59& zE5Bx-gV1}#RArDUJ{0u6K2p{?!3hU&W+u@Q3uz^5^NmH$S>O;1fjwn?+&F9t*u1(w zgu!P)>!6WQV*$KAy^l8r^$sd{rQ7|$*sg}Z8RfDJ+W`(O$oDW7b<4NH*+>Xm5V;=r zqXngo3O1$BbiJ{P7z*(q$K-K`a5>k6sxK*bvNE}Db9#)ZyO%PM%=HPZncJ-Cv!9I2 zm^#2o7cJ2H66eW(^K88m`fMw2)gZpu=q!vQijzrX2J|yh$oNcjLXuxNE$?}M7$gkQ zJC((uh9|hIJ#1g5n(<~j%UBLffNDw3^NM+z!?*?I;hXcgU&Dn)KM>aNrp~%yK47#$ zD3}2w*2?b(--D*?bS=7iCNC-`foO=yR9YhX5bY)Y6@)pv@GQE8GD@w$u#z-5ESCR8 zHa|XSf{-`fYAGl-wp=-`d|?q0mRAZyP1Ebg`;ypz{f%(VLw##fq?3CjwIY*<j8;u& zyK=fLW%HR60XKyvirYP+>|M344jsSml~BN@${2bEY+e}Nvsja`zQ0k_M5ZM67DWmd zHLf?7npJyqcq%5Xv~u3$KBprtE}oK*U9Kyyu)^jFDMlg-WF}Ma%*n_4y1e$Nvu}j1 zZm!l6h%|T-zI<0?A(I5_MN?)N;|t^<@)p5=d#;DIcKHT->$~G9OlS<hW_vvpmsm{H znfEnx4{m7@ot_j=oi%kUcV1;PAG66iZN08hp_1W26=+f2YrYo_?>_8sj8k*PGf(VL z(wYz|4!<;5Z)HTL!()YH%BB+e)4s))1a#RNsJ&M{MA)~Q7SFrfR@uq^?HC<E(esKs z(Il1UuUQA*QnH`hjHQ(tCtrc{q#2C)4P|EL#7&3QH)oDzowc2vw`^Z56fBt9<!6;q zVG9h8_Mj-;c67W-K%E5cL2YF$G<Nc1xRUqYSU5#PuMUtUYvl2+jlE|9a@BQMA3qsR zVaXM=pUZjg>A+7S#Bz6mKkL`WQy}LJnO9yWN+^-osTb@v;RlU41(q}g8Vz3A4ltE= z%oDE8P0`ezTq{N9NvCI;Fmu!)O>*O(Y)^+vBx1sOUOV!x$UDVp&Dls@K23R@0e-c! zdjvl-F%hRBMcIN0*yTkTb_uWSu(ov&(n6bM34NQ>EiM+Sqkr28{-)!zP0jtt(SQA) zB#$+<yubbPd{Es{Pw3yXP+x(HEQ9t_rrg!QOP;IQt{x|4Q0fk5sJ;~Hk{=4#BTm0E zz}s7}Y@27{*Pi}cUG-PNT!^qmvrkBW%l6#RwqIb}JTBXt{ulhKl>>pa)&eHU7T<0n z{vL7!%%d4SK3g}i{wk(Fb5nKn?HT7*U(ca_4*|YSM#VFrZVC6-2<MNt@?FFN@&RfG zT)}?}0e$?r<NqJK{Ac_B^Y-ml@W37aMdRk*e+1>HdH-bDwo32v;a89TkE+&cf$mMK z70b%s6E^>CxE%mVz%w{wZf+j^)dO{VmR4R_8NuIEb);t=_T>d|c5x-B{)w1#$n<Ys z>3?egCR{!d`&x74)QpXn;9nm#SbC=4fPX2Ct3H5wC2Fk~IN(Phnrc&do(}=TC`a~9 zj7C>b&@drRS!ejC?H@sCta?)+qc|1My?zPpgkeJ^_+xCpv>26Qu&&Y3(S`lwWIXx| zPnF6@hcP`Cr+_?TqDVqM=^=jh^etfIsb2@w`^96mq3~bz?q_DE1L8ztob$dNH6Ouf znwM{h+3@bUzAP@OOQ7GYPtG&)5HoJ!nP$_q^X|t2^UqU-|JPXgMo`$Q`S%uDpU+r% zR1X`NQc}WViCN^ll_pQPgsbja>F~`}TJTWXRgnnYq?FCdtyLY8PUCZ6IV4=nKL>|A zlKQEY5Bv@I|DbZ*rGmFjHSZ36Dh3(S_$*>%+SN-$_#nn&eJxuUoMd&XtP4vk@I=CM z4oHjitCRi1e_dq$BHI(~wNl7%WJ0d*kS-qLEgO1ohog@4+ddSj;`VkRR)(oiI+YsS zRB)H|${Qtt^_e66Ut?YG;M;`uPSq(`rM8?#g8AI{hbAtpQ|X>&#mHdG*1EIIG`s5V z6x(6$iHZ>?50AmZ)cL}kIg(Sv|M`Z}8OuOP8jF5m26y#?2rFR?tFvD85Tl^DQpENc z;ZZdztWZ&MsE`gw3WNsccA&BA?KYtf4iDw^)Iu8@lRYl7c2q6oEe!tg14j|ZG8shb zd4BaBLZC44$3|(gCdfrG=rl_81-U0mGSdv!b5ne;C<4oGAJxN4cKU}HKR3E`m+uAE zEwAso;~=nwpE%jtiFj;I;}-hF?-YE!o|fXk4*Fnmq)r)YWU6l3r?_c;q$UXY$!4Y5 z*Lt}z%fkiQ?bKP0!{c&CMq^Bd-nF=_VEI+{Jnbam>px`A$D0K``uIk+$vz-7wf&&M zi%(^8k{8<9q?EHDKE=maG1hM=@R%mhvtBJ4nP(T;q#^fihBT#cUR1tBfa#n&-$dP6 zyaCk`v1Z|)`-)um(#2$BxY^!d#+Ni8G3U{W^&S1Ap=N#EzGIL_8E@VH!`D|p)wLw; zCWHjHgS)%CTX1)G3GVLh4#C~s-TgrD;O_43_Q=fp=iZt5-&?B}^oFx`ckimM{;KNR zb#QU9$qp`#Pi$;7W3(WP%e9e{9c@hRdHpm_i~Q!?Hd@;fZOi8tV&~?MBC84i7dytQ zmaU~Qq`10yJWp8<Hp_gA*S4pemEkT|8?>9_a+MdQu{?uyl;dJH9&{V0!5L)PrA{5L z!nwZz$~t7DtTsMAfOdJ?>v8h*qG0jremrl&>2;lTbbRYgqOC({+FxOEds}flAC|LJ z6h2wrS5@cfG$BW>iz3^jnxMAQ;okOkX`Xp54Lh?mgE6bV_%}uWpYDMh$Z6`{U7y-( zjzeKB5=G4M$%%!6axVGRwDK--rwr%vPTytuIb{-_t&~)xGOgN}<I|{Gam!%TW!;&R z$RJ7f?@NlG^J84X8TfGIZLs(q)hMp(xPnL7Eg`qeI-Ou(mRYAOiD?H5W#%Ipc2tl| zwAteF#n9XCT4A}5$m*2`DT8ivwj!`L8p@-Df8X5>+`jlci1ef9dZ5jB=Wc1%)!V&C z7U@)P0Aulc=H{s@5msjz7Z<mlp`n7^@r-s6%Zks)^CNOfuCI;GQwix-m_^8}FG$bI z(Cd3zlJt`-xT-KF!*|SIBkk>I#pqZi{95`fk1#c4DnYt;caiswht{!tIp!7FAsuz1 z3<+>KJV5SYV4FU8lzWXh&qW{62$hl3#O#_p3A%xa%&^qxrcDuYomjUWX#FCX$--jF zi3G5#f^9p~2GF|W-7FW*K2|b8oH<skvjHJRlyv^#8Vk)>o?BR08|Uk8pB(xS7?^ss zLVp2fyQw%^t_tu3dG$s9Cm3{Ylp0Tr_}t>wkQvW?Taqpiiv5MfX}J4ntWf-(#@9+i z?4HSUNLl%UBPPyn%5Gdj;b;*Z+GEpWTl`aa`H6&V&8Hz)L#`C(1(Ox)^!u>#!vi6( zh%7FEN_4@7$(|D1O?b~X+)D6OIH}FX0x^<S68jD<1jsX7qL16)SZVzldo=m)Opwwb zTVsmHY(Jn|Bfa-~<9-!Xl+@o&PhU=iWQ-evTo2N<p$aG0XvlP4q!)tkE4i1BZ_5>H zdC}G~Y~m-m?d71%L1w3mj1`?NKOGgtB*$qgFK+G<Y$7SeB`O=;z_f-+X33f+FN+q| z2be6E8npZeU{p$vpqmuURloXzjR;&xN@17TOL4=?+{%46RghhBm{@hcA1K!Ae7KBx z>50pVwuAOn8G`V*c~o2A;iy8HXPnr|>owzgg)@+Iw4&uneCkY84lg~PC0nXPO&XNx z8|6=gJ}kX9t~qWmAT<-WahPY>h!r7*Hy>K>a5>@BtUpp#Qu6U%Zrb#VU>##ygKK+7 zeJ*&bDHv+sr`3CYl!50qU>W#UmhHklej?f1BLVZIq$FW<%Mw44oOesvqUzHw!CHNK z4MGm4e?sZre6<Q|!LF>I^8H9*-^&}&lE}!F%a?>i_XFLzajfjUx}<?|xw+2MUX%7O z1LDIC^>OZgi%5G5N*F09ovs*WbF#nx4Kg6~H7N~ebbP#2kVL?-;BFgt0n_YVDFK=K zthS830o)D@`)uY7w$Q9jV~N=$JxQmS`o`%9J&QYu8oecN471(Z#=F>Jd(UrRFKXb+ zv}R!orndF*gf_l!T(AbgVlRa?>cggJAqt%n>`}DKQ15Mm_o5B)_NPXyVKAcN(##TP z_A*tCAmew@m%e^9wi)E@deDey@$9>;nrS*{4C^|{2`_(o_g)*9*?s)SkuPi-ZeNT$ z+d>w=zFR&$LPzwU)X>{h(soaBgV}KCfHZ=^ZeC@pU34W|z@mY{q6eW9?ryp%wE$G5 zd!py9H*W6y(XMV3ACxnN!#?Kp{buJ9jrdt$Oz7nOD*NV(xFH<H_7Z<>o-d|b_re%z zp75QBZnXNzQZ85oH*p)A*Ncss1pO4;iO8_4BQECG)=HlzxPVf`mBmyF1>5bQr+cVT zmNScW?D@aJY27Z6QlhUK*C4}1%s(i&xIRL#+j}Cl>y-1vbxR}5MN0hJl_xVeT_9G2 zo!ooYr^(nPkh1Dd*v3<g4O2*O+*B!zN2&>BA1+W9?0h=+hD8IJGcIql7<PIyMBWDj z4MY(30xr&efM*UrFLg>EDL%eua#BWILlRj0>_^w!Wyub~28Qny_Q48v(Gl10kDM7% znNehZNW;r|lgA+Vb~ZIJNepjbEd3qVQg8Qxrwp1MU1$k1T_A|SYt+*w){90vC;4T8 z?ZI3n%trI6rw7$HQyM(}(~T;5!e;S+J)E!&9O*VJMmsy$mP%J@)B^u~&#`(Y3R18} z3}{~gQg9;*`A_;E7zEsyD}&KU2WB9w)=pvS12A2Q<Fei@o2ifgmejkW>BbmndJEDx z&ND+Ri$l_?De-J-46;{oGK<#1Y%*BKxo&cjgg9={xxTR(ybTktXsmI>TW}LbhL)D# zr9>}FxC);|@O%c<(D~q;w;+}j+3uW|T--Q_DDLP&hhi-hXWc?D->;fD&WGLbl@M)} zXtsA66zZ?Zme6`Ipo={cs~c6uGtZ^b>#tBM`fVja%Gy)oF7qbIBgXaYBmPDd2og5% z12t1Bo6#;HvXQNBfvx3G((&$d0p3-}%;+v7UiA5VBGrs9-z+^2Y$k&{QLyBTFSrTz z62R@hvG3I;*U&3Y3TzF~zca{b_BcCVs5_;s*t@!=IF?6#(kh(SC1=;C2=*XW@f)97 z@q6lSbZE!?c60sxDaCHu(xZqk39&getzyZZ^>BYht2VCfbHkNJ?cH97w@v17oY8-v zFx_Ynt=sqWm9tLkrY`O`b<;9#N0bpK7z}q;R}+2%{~X)jmxENUsw<O<F3oLX@GkJ* z-N3|TACUa~Zn8^mQFF#ibPo@*fE!c09xpaDP^8iY_*RNk-vX8sO`6k-<{_2s*CUl< z%8W0><3&c;I&B^(Et(*sI7a-B6%(x{iA&rqG-Is$iMX(D{JFqs5!sV=LKAA6rg}m0 zMMJ}oCh%U4Y9Q20peouRcN%lq*);a)j#+0Uxb<d-!%H`+w9a64a23Yu)<oPJI%i`- z-&Y-=2eicYY7n{HAH+=~7<R<eB7(;FA&b&dO-shD6737kJKyFW7YghCc*YozYz}P@ z-bv~%9CLa30|*6I3WUA}Lkd<430eOeA%m|Cyx#iv!Ze~Qnc$<N1o>D0oNr(XQJZFR z5W>+B{JK8cSp27z`Ss9!`h=L4OxZ=a<Ko$>%*dL`yk7%_*qj((*st)KY;6fQw{pRV zrR-gYR!-NB7;HBpI$7U+<*#yRP=h~8B>kS;rlT&6wuJyLJiSTph`lsvzd5rXGYw;d z(0dJ2MI_-kCVYLkY*S!!DFnZrhqB?His-Ylzx>+b%GYdZ8jex%?GWbS9>Q1U5FP#U zo7X5!_p3m~musb8j7Ion<g~aF^IrxseO(?l!6uirqREnqPP+$1%5iLrwX=wePcf-_ z^oJ~zKihQIJ9ik2vvWf&dSJqRu+D`+SPKFI^9<4NZ7KyqL|}%AXCHE3TxDfXLykB~ zNQiC&|2M>!&F2nPZ!WdHq07zY%5e&2iA$mT0#s>}({cXt6E~vn<5dz^3KOi~3UCBP zSnQu~^HL$HP_;V>vD36?*qMVcBqB=&l&MMYbMLHDHJ*Qy(9Rt;$9+zIcBUq5Qp8+F zCJYi&i%bP{D1OzX-{6t>ebc)s0N|QWV;ugd91!*DerFJU>FW}8=G$yNDhg_@pgJ-A z6sYneOW4ep*vGut<&y~1?WpeiCu`8)gOvHNgHd=2ezJv5yKS%U(H*S5(X~ADT~&s0 zDU!;f49PLzVhi|?pMOk#q7V${*9KAQSLR(xLMa<WaaUXt$P<a})UU9+nMqW5n?;TG zYw)p_$Y;r=l0%^k=UYHb6igHYS!Db<y3^r3@$t)0MidW!eqzEfbGOMWOEj}(&#MwM zn7Pcv8#)Z({eU4OZ;>Henl4BX_kEfe4HG&tFJ#qo6&|_7T0L5?ZKU)&b}@Ss6ekfm zE)+xzA18PS9~O>ccn$(A^x?5!b&YMlH>aC{5&-Y>|I$nteQ6?@qHxi|E0&!@i!OhJ zQqlCHYB&>6F)7xim{<mhMDLm@kw~E}TPW@NmQzv2R|Ep0&c#yvrU9<yVHj<U5^Hxd z7v7y$32b@@R!0qL=)I_+&g+3}LQ_?^PaAYJxsX|1U9iD<-+@gJ07yzL{pf9ixtCW^ zlELtK8!KjMt4E~uQyBgLHen?71gXG}a>~Bc?CdU)r|b!4Cu<J~wbc5=_}f`^tr08K z{&bbmpS_x*gl>E*2(**7uN%C3yi(Hm3A$+$5B04sHYU+XSbisIP4*_Vy!W~TN#XSz zHdY>|T8XpPt{|MgPB;-yD121kd9~_s88MYA=V^aLOb2L$g9{`)kl#xyxWawB!{*C^ zXfyYet^TG(kLE$FR{-a?vz`cVU(xO(fSOJgd+un~Pz1iT@}sGQ?J;oH#W?dyhZfkH zm9J~7+?1joOlntP?q(`Qf`dUle$i;~;5tS4>)=9iAqG^`=RX@qG+&Z124>_N8V8$p z+UYv)W4lt%0suX#&UL@x32rRy>#aOcT{fhe&ieT4X~1zXASwZ^<80g8L}Vt}>|~rq z9l()fVjlDM41?yA0>|UCtiNg>KCn8i&tz@DR%%PrC5sr*r=LqlCz%zw>W2P$CfVbn zvSa<jmi$UEeiqcg06flktqkh-*$^u@7t6Oc0#B_$0nKMU`d4_8!Qp)GCS5En<Y?SA zXUZGvZ@ISXaB=ixlD4+FvT-+uel@hYA|#tR<o*o^&nq670Ak-viUfkYvKDq|el(6z zOG5d`a3SIgdsniy!Fh}j#Ovhe&frP<C#;6MR9!5e1F~0D|J{SJho7LLe1sc>lR(%@ z8k!20?n48gp7~FCSwzFo7eXG*xN-WP2<QidgUgAERc(4O%&lDrjJ-SK*^8?ri{p=) zhp9*G;6I(B#*T>iK>oA+Rswb4t|@a%@*0ek>aRg;DXy&K-y%cSoAU@ow!PiRA#A2+ z8J4(too5BJ$xAmcDejcX7@96ErspgQ&|iYX?XP!oKw-0Cz<G`Rf{aXDEg_gq-%Ja_ zJPaGJop@t603Hg8VtbqUOeegP6DD6m`idEu1I{!nQR)lfyh#cX33eCW3B&t1Y3WG% zBqIrwA(=ZB_fFD9)ob%wd9tSi>MyORN;V{9*Vc;n>x$W-PR$cjv>o39E$_#GE-Kg| z*g~QqG-0HqT7Gs_;Ae!EK@$Tqh0V(@of1Tk+;#Lhg@WZYGiaX~giZYExsL7AKt;QM z`Y7CN0K~kU7q!Fj<PcsxJO$bA__Mp<zJt7O5@rq#Rxn=`$W**^t=V%&S!RZJN;g(H zNU1EnarFH95%US+KbpfA5G@*vU{El877`5C$9*Qu(6&N1vUHeF#o(Li3a$(7yOb;G zY?7SMr#*RO<YnCz+rq5HryXTf6pH9byr0ZQ%{8FGp^<7-PjHUHKjV9HA^t95l$R0I zs4}~S;i#WYe0!110gqLYC@A6Zsx9QP%qY)_m@L|*SI5ZO7tL-0$j%||Zq0hN_UjNt zWL&+c!@WTBT-=rj`AniVLDLb8YDRupUySu$3@yc>!n&U`A9eisU#p1Wb0+;=aEEjT zlxZ^nfQ%W98R^e=IoQs7R&GAu7s>>&<9*7qUM=YG@bqX1a31bzZpnR8L_sl&*jctR zt;C4_b4$KxyzGE&``vGvwa>Z2jh@6ys;>rG8V2+I+x?jS^!PmRwwwA&`1%YVhWNi{ z@Cm|!#CI=YK^hBa$gmYfj(9Otq+xU9kU4i9pphDlg~O~5K2t_51Tk>NGmB<Mh4RN< zH{*a~kzWYui22M9o&chNk-3cn-*EGAgW$kXY43HDqg89}IaO-MGFuNer|G>1WQk1P zy_RF^yy507mKHl_iDU+QHG&BQGtKyW`QmZdLlSVk=Q?@vosLd|>5bT8ej@q>B+H*X zKXDrxHngRg&+9%2j>_PK2Z+8}>+z4<T0Bbx<(pWVqx-SLN%n_;t`4p#+hDZSvgkg? zHa0rXPFIQ$5HRQMMQKeV43#9?9mhQJMEyDk4xYYrnC#c9W&Eej6XhedQ4%y@j=!RO z(;h~$c-x|s+No$%G3c!@_$?1=D}gHJN_76@hI=)k<<pggYqY|MEXS&<Zq{r6XLJ9~ z11E#K>{GsEW|`4<;jJl&5p?am@&h0A0{D9v<f9J=LP5U?vBek%SCFbH#Sa%GUt+!8 zjUYwTGmVDbgD1Wf!n^aTg+C{?&u=lOwc}xTdAUG-V?O%ga>she*5Zx1mYx+Af#RML zvTKf_w|CGyFm&McLZsc0`F(G&tok7GF8udxU*T5Xs1O=@NfesiYS(9HuGZW)2XhU^ zshZuu!)h{A58JoJCJt;~7oY&2lotr@D+96^P0f5=oALWK`{o6@c4imSGPjJ2rA$Ah zUd$Az|Dbmb+FZK7%YDLE)kQu&+{+tGm-pD3-U{gBy&`=nI}T+oF0IPoLKOFi-<jsE zGmcVe2it*rUbI0Q6DigPej&T0yspS~iH5rUCqgcVVvnz3p}#ASH)6+@4O}JT)BZAx zzs(^c+{$T#U~m1|6hL+xs&UG@dDlyt?Qm?3B-2oRB(k)7Gd)$-u-*SC9|ve<s!4ew z<Ho?7Xof8*1B+))XovTDJ-UHN^!JtS1*ZySA(Zn)Z&8WLB2a@_W8QEa<k`w7)s#O{ zZ>9fiTmeV(YfN1?HjE|FTj2SrS2w2EU@{WB5fbeOr*i+)5CPHiBhu!tKr_usMY+LM zZvIT{M0D#7*`@pef{JIiWP+IZJEFS?oUj_k=~a=AwWRVDpFt84ckJsmdcEHVNH6ni zf)TD1Fe4b?nd+-MLJL=O{vnpdGHworwf%Z~(K?CXJAx}U)!5&w+M+jQHl~@Qbz|J2 zm)R|ztAC&`uPSz!_nt(Pl2@pPK{UQ#KXePw;ne7|ue}2m=Upr0{c*ztDApF)ArBF8 z`%}Le`vzYgu8|(^&mVB<ii!5uz5fhFcu(;Y@-C^0fk%&&{i&Xe?VqU~*(GCzziEkC zdDz=Dx3y?$d;5iH#jG*fHzX7~wkVRQspp_3x$8uaO3FMZE9liO>#t4X#{8Lg<<*EN zXuyp#`7V-Ge?yk=?WSx*MVOT26Cp{moV*rsuI+BZ?d+)rTo2>Ojr+TBXyvZ<Y^y7C zqNhjuB}Gt?{^L&t*;?0xbg)EG+`?CMmp57*UxG-D+YLe*=_`EFDs+--9nnD~2uRa2 z4>)a)=#<v`E^rmSCGnMv_iifcE7fom6!*|u?`I@;AyDPc{v*}C+^7=!(x}ohD)Je6 z<c{*ET-}uzw0kih&`MO0MhsjJ6=*E#Vxg2(FrKaJGh@2$LA5VM9e|)EjDC?o{VOsk ztIo*?o0{4Se&>63*q(Tfo`s-}G{%b+8JwUn(2jFbi5jbC@@DJm?}+l%#$)!8H2W`2 z?OzSd$6;SCc~ML~p)>2iy;rKgC_9rDnsAZaqOu(A>RI<JD2G^LDsR`YBQ|*9JV(zi zBr2%RQ#f7uFVtRbw!X4hj!F+k-QxJw+~Q?Qi1%Hmr8di1<o2<TC~uxS`qiu$4o7Ym z#}^$F77^FZE6_EHDc-csCvEZQvAgULyv+R+Fu_W+^87;BK9CX>?c@eIum<h*=Ju`j zm@i0o&)Y~hbitX420wWNC4xE1{PLRpw5{-`j_NKkn!DC?){D9qZItV)_KC|qM3reX zEh|CM#{9=zD)Qk;;c%}@N3j2f$bH>FVomi@bv5lLgC@K#vESUI+Pzf5Bz6I{bfXb% z=jBz)In)pVa^lefIH_%KBYgS!1$et_NTU`HsU(9y`KloRFM@rMl#qBW2e0Dsb`EZl znUaqBPhSBI75;VeT1PmpgDoOS9QUQ?PHml0#D~Rp#e6a&jGs8o{l~h)D$G+0W0ZK^ zA@1ibUr6nxx4Ux;noX!bAX8C&Qd<<Ud^DD_zHMKw;8u-+uxacF%!KVfw6>M<vyP4q zs3=)}bo=~_;=ELyGC|m@B4|y!Sw=(kqNUxMoO5TtcK9{{hK64Tn&p*%k6aa;G4sif zq|JF!kiKmzpT&=GeK~diTU`QK<<VJhjk-2I&x066=6%Q~p7aD?;86n~L9BO<2tlxT z{7D#kmHs3NDgZ$|D+5Jqkkq|Tek>~ctUETt#WK{bV9T0wnqEHeAk3!SMvFsvTuWW$ zw7@ZXMg%PhtIr-0x96#~uKO-}eNNjxv0|Xi^l41%WJjc$>ybMpPXuVhLK^PahbwfU zWZPUCzheP}GorDo>$2$+J^GpE9qpcZ)p)FmpNQ-R%Aou6O+@Vd_6MmlWN?DSR6w8p zM?rajGF4iXd*XpIroFa1K+F+4z1R9DPXF)|Y)p+=$(?#9J)%8ge55ESjvF=hh|!$= zO)*Wl#b}MxJd)|d?MZ)YB}N7FF{2($)uP15`-$XBsnpxkJL>JL2kOb+SDi=$f4xj& zJ{%%1hRsNYV*X{+ZB_vXn0bSU$~(#%zsTnqx~PD6IV6)-&==P#Q*tl_=V$)zxGaQB zqTaQWP9+$F?-syBTBS6;{0?+KO#On+IU#EC{x$vEYryB};$++Pv>Y@tN2L<-`RBzK zp}a8eD<7$FqZs$&B4s>Uq6Gt(AHPg#oFyt3V8o?5PU$UHVjv_+D2G*_=+yk=wkD@< zt4k`BJXLFnhCR94&yE$6q!7$`Iu0Z&gOTssnGfD+b@E){;tYk33IaCMSGdwSz2y*N zW*soXfZ}9PQAfc6530>ycg>kBVI}?mdTauC3h$F(SWLg{Y&;}Ys11iBh?5z26H#!# z-H+miTF>7rva>%qU3yhjC~Z|mFG$a9g(EWSE9iC0{`_p@N}dK;6f&v={CZL1sC&_k z2@kL9H}sCxh(#^n$a>GVcGudZd-BzBcG|`D<{Q;)zaHX8jjG9kH0{|^s@TEC6%ON} zw}v&p8WmFX<l6)I&>K(LwoPzs`meNN@h8_5CCeB4F$}rdnlBr2ANHs&?N$OOw%-Qb z#iIU+ff1BkhKH1u=}D98*wt#n+mbnPO#@dbrpdj-1+8gipzbxyF(p}zl^4^=vl)rk zMGn>N_$btchj@TkY*`wARpO)IH8Cb8-7!!Q>q)DWL>Dd&)ZXVvf_Th6{G7$G^MD%7 z{jS<&&n{x|xz<QePdquqcxT5v6S@9WzTCagHOgc&RoV-`VjQP0F>6s2s|xdI+aX`V zuKj(AAe31DRWI~q{0e2Np;iC$)wo9uk*PIESF3L`|6hJoM{xIL2XnGh67P4?8({Q1 zTEz8WX>3;$z<rIp8+YB-HUtXj84X4t>LfvwCH@sUB<xmXO94f-t6qNtw-27sRn-Eu z%ytt1k52na{B7%OIs8&4RJD@UtNHx>2FQ0QrR*qi(pfFaOPG1|L73L;&wL8w_M)UU zH-1I*gHf)#Xi!pRqn0aQ+X<*Za(B+UE2f7R8$v{|&xiD0AafM~C>F52*q$pJdbcCp zv%Y9<Ox7A(4J(9e-`@(*v9ebV6|~peQ5!`?Pq&1QW|Fpi6l!nLfEAdqWysYfy~CHY zYY^Pd2Sr`|$qbn#j)UzCn;!SV?ZUnbQKgH|{JSg;&UpMJ(V%7Cn@Z-_n+RhMqttlg zK5lzqZVhG@z?U4f&Gg~};%>MR%WBGpO-%v%Ke%CA<6dm;cjbcL=EG6~7~*3@8U^WK zp_vTAR`4__S|d&f2W3?iiR1>EQ;B5~1aZCX7QX$Qgk!zE_mvYu3AZ`1ndy~N3oE_) z)c=e1_w){2e%2-d&ts$N303}gXR0~whttwT(#E|K$rq|;1g#5f9%XkdN&zovbZy$Z zN_ShKkqj&~U&m*Ki)M+7jA}kB(9BqE92|}NF~^}dNi9*UoT4AsLj7Y-p%V;A?DEc@ z{!cWID6}1IpWz~%!)y5NNJr?8Q($1sSerPp&y5QW{4}72$yrz~5S*IQ8Qm2#Tr!&; z`-HD;=cm_LseuEeMH2PZqw<|Y^ZnPGrfxSiiq*T;eKJ7KipMK-Wv^O3g39fDaZr*M z`M6=%2Z}PM#+MB3$<H64V(NxDa0Ln0L8(D+`Ft9a<$7r%W+#NVja3UVg*vZDpJB~B zeuMV;=eh#jUGBi&PD>Nt?xdt@;z_WdXV84Iyx!1?RXcnujlV;$-A69xcMFnl2<u4D zbcS04?seupLrFUvz6=mQ;UC?73b?uC9|MZN9NNl76hJ?)LK&^@20!hS-d+-VqZL98 z0Cuj5bPj9kzMzP&x(l+sHGMjHJg4Z;kA64p4FH>h$2`I|P45urK_$~@be~mlDu}CA z@85b8h+$GYolg>Vb)#Xy|HjV3DNY2c7CO=xM-X3XvvXV$pWY`XXz<%5`uPbRPTwy8 zT?42;&Xmgozn|jFxMQCPdKL1^>&gcIJK`2RrA$F**FaFs^}PwZG!?zRan6HihTl}l z!c`l2NWyA6R!u8ZZ1-$(jSKTN3>L~rS04^m0vZ<PH14*w)D@e0SIhHpM~5_>a@jdb zuI=lO4U|9LsAd<a;><+~<6|*$th4NQE~|)8V1Y!E?6yM)b)X=&glx_9rl6V{x(755 z8W$T5Yl?`OGP3b_I`DAXSWa6j61(Vx?(tDyv3z9)9k{OgRu`UqQcU4Ht<!ng6tFp- z=1ByJ{%~Z7E?zcMp?(jDzU~~mc_LiBQS!!YyV}<)1jrJ9UWzxwBQ&(gC<vf_*G&uP zndjjmlj~JzAX`>N@F+l)M9|ZsWjT4SM&48_YyQ3$x0foxR5%Jh>GHD->Qu^V&n{T^ zaiB?y1@@_>U8bKyK=5czB8AH;F-Qg|tRKS}p^ow6t&J%28l$MQ2%>M*oD9g~c)cj@ zdVf64!0I^tBS46tQo$Nq)i}eJ!Q7Fu;z!5%>14a#><tzDS<P|U?x{03kBWD3Xaz~7 z>6}AEh=bS}@!+LV_*TOa!<RNY6up?RewMH1tiIY`#Aa^WXr5uJwpZ}~7FRV%?2b){ zQ*|#0>2Ut0FDn0uAo!_oN`^;661<1-u;JHv9u-ykFP$o=fzmwe37u|O=V%@6{(Nt! zo_5w}{Q9cTBxAkbxtR~Op3Gi<S!#N~d%bAGL%)?qSw$l-F!423Ax^5ET;e=U0N28k zIwBwUiO8D)A(jgdxOP?Nt@p12d&>DUU!xAjEUs==qK$Zbljbae0)*q6`wF-gNK}(1 zJ~+<V@NKmgXhZ~025=!l8#+V-#%qF)=>%D}L~xQ}8caBy;P%Wt59z8$1uay!JJ-|N zk1s?+6FL~3B6at&BNJ<Q{G1Xje4(R=hTi8#!M5*|Zqg}9JE7yj!V$o{D_1L9{|U9| z_how*@7rZGd>JtJ#6FQ|fsEtGz+7AF_9IP3h?s6>LRn?ysA+;v%coAz=;=N^0^jl! z1-HVkKB(nR6sxN4_X&p1!g*rXe9Av>(OvQc(b8n)mBXOspcWiU0s4^<`M$p3ESP#s zj}lqT09#F1?@*e~8XcV#&NXoFZT1r$HNSA;VS$!s@WDAhdo#5P6mx#`>{TvwJ^9ZA z1xI({Rt&p8kk7YR5~7_6&I3yv@4(zJ69NU?Vlit*m4(DRfQ_$01Q^HqC8_?-5Iq9s zL=Ng)W!w@LcB%cs#=pEPkQ8!1^lqV=orEf;SR)i*)!ICVmxyk^o*;TqNI8oQQ*r&o zjK8UTE>iVv%nQ?fXP^W~aJuT&NUXhj;087O^)Zl|#W~BuL26eJd*U~~i??xMn_1nt zqdC2hf#0Zm<UUN6pe~kGBeBfiBS}m|bS$fWm$u!B@5^C?I*J<+AE%o$8;&>ECtnnA z6+}{T=@ym-R$gc{!rG}HxDh%_sB*(?y)=$jWwzwMaJT4|EJ{k8dPyLzZnkFPc9gK% zG8uM}^|y~C<C7yQnM)^dz=-jk=y+ZuvEp<?9dWV*cz$t2#=BB<=h@nN_uH(8vsO;} zM}udL8as(kb+bt(llGqI+=Vm*2qJ1Y|5Aux8GQ*fx4e8ZMkwPVZmqRy!K}8sCUm)7 zvk4ezgBAZYB#scFR3rG!{G4#VqBqm^N=_=b5r`_!$s(+FXL<D&uPm<~w(r$|g!|HE zP1wIHq@}K2h^?^iW{_jDf?35eF9||qGX_r97BWn6sXhD-D~%9}_zl>H1@DuaD?vt1 z?%S2cJE||YxxY_32Y}ZIGkcKO3jGECC_vUZNCC-C%LXp(&Q3G;l$$LKkYY232W(JU z!W5U5`qq_#NSpHFmBJOSQ_Ueu<DPKHMdV(knm1aV`%1*twOIO{cX~Yb@u*vCky5pC z)P=5!EQ_0VkSz*+dCh4ik!E5uG^hUJ|7<0V6IdJCTQM}46RO&}$8I=ky*klc*-}1| zF-q+qNWtTVui!DFzE^PMJ~9hKvTib;Rwy>s<cZC6P=n;E)5Im>?4MX!0=N=jHehu= zom=lvA-LZv*eAniii%|ZD~FIB#itXF`||XJXPYCI8_9iE88tK=AwQBG-1sXI7vOae z%>7>GU5S=}6UF)S#l}qonLvoHaaVam=G{x}c-~UR_o`K|9;QiF)3zG4EJ>wRI%osz zofMuq2{TpArRh}8{VE0TIssV<G+J^#Ddz-{3q?`ZjvHbR2g=-0A@qo{n);o8=#T+r zf{R)i`Ko0npGX9ZgxRd(?3O!x-<auiTe9a;c+~Q%(5)D#1XROB(DNPc^+}=CbG51< z&s)48R@ZZ*lcZ&tK}Ulu(^-W|<}t}f#HkR}B5sF9CtzEvtyB0UlAZtqwa1J){y;zP zuEI!GHk+scD(2&&mV2(>05?lrEZ120yJk8G+p`2iI}{1nXoD*_;{t;2jc->ETtcJS z!U#<t9>~pa-`J%{?Ld#8@3%wxiVOX9Td7I#p$^D0nqtr&>_~*IHzVzEC@a~H3W_2k zFpJ+)_Du!$nYT-|mWWk*`-MEd5nru0`e|>{(tl6dUrf?A4H3#!RL!?~bCZO2x7+;q zmYG4dUM(S(niQ496QA!ZlA6`CCc_Xzs|;hQ9$o>D8{#Q&Q2W&pTiw|j$kJkNnWk+% z$Raknuf=hkn)y?55=@Axs_SfR+vR+bXi}(eaJb#vCi<_akZ3kDNE)l6-Y*3s!Bs?G zk*(4gZ}I+Ryf5D(k|qeJn)}uL$J$QN@Z|>O><Mgivc%(u_v1GS&_+m+vEG+LlgaJ6 z)+R#QP=k6WCjt<}+#KviJpqf4_fP2Qu0>c$XY{#MM61vItq|<4e0~q4_X+(j$h-5g zCeSriWjc+6i3?NYTH03;qqt|{Q`Bwt18_IR(>F%bugqFaq}WObGnA~j{i7m)%ZKP~ z!z%dY_*Ko?U=ym#?~oehT=eYUGhG~b)rv_+>&T%DDi#!(YRcl{ODz%aUU^`6?L-_$ z_Up0XMEag8tW-8Dj~_eD?{TQKbG_H(mOEOC-}}`5sq~c$pOI}Xs+=}-)=&Ccpit>q zEm4Hm<3nET_iNnmyKFXdk%37Qx#hXgM{rpJahv2ahPb<8_?BPBj3O}D0ijoxCor9# z>EbhevZ@r~wjaYUPNHkYF)3&@HurJ;H^1}}FEyid0$SQ%LtMz5{oUy$B4|PygH{1~ zyz)d7Er*l6b`ysv!XJ^(guS(BB<98S2shlFr)wIyshN0QTTkkVia*!vAL#gO0={RS zxUkX<!&`E^dwbYfLVSwwQ1!~z3<_tI<z%%Q<h~83K)uBVq0qupM&eTrV1m2;%Ca1R ze#z55Kl&I&d1AB9IMI!(uprCww1_Kamx2ZB&5#}?+<PG*9#%&Iea2JDa!x_Dp6FG) zu&42_NB|$mDe(o6*aGLs-%u<7l|!;)1$$;|Y7MfzOOa|j-y_zfXU9-1&N=BAXZ4J4 zJeZj6s^AMfqYxR+UU1I+!T#{InQqvvfJq#B-@7*L&gTRd?*fQuDx@4Xa`<gc*FhSw z-VPYd){&pFXSdELT5V0tFu<z$cIw`m=S;10O5<ya@*$2uq8bC2gVhG`)q2FnhtOnB z^(HA3uYQxmbC{E*3zp1xiEBUO>(jP%^n2q^w_Xnk&^w_fSd`0+>D;P?g~y>p=Hv|a ziu&h*FU<DHEN48C0hbCfhOusy0V`RhEI2<Fk1|{ioXL}~(j;FV=On2&*GB8b;@~^r zP4MwV9BZruFnNsKA{;xc<e56PtElFMKGFuK5M-NZo!{aU<2RSB;ldgOk)b=i0Qe`Q z>-E8*U<#mqz%T+q3vxf80yz;Gnvwmd7eORtOH27KS->GSk4oTjUaxIl3lhH<Y)vCw zH(us19v&7&B-C6MCM<|ay@xflOUcUHp2U+}^KlZV`T&A@iibiyKTh-j!NIeo{6bIo zx%C1wR>Z=98dWk8<<dUt_{X}zxzM<|6@T?H&lVxOJ_PDzl8BVo>Ib!s`R{N?GkwW1 z8=V#8c4NT0P$IL34hcFZY_HSFg5+xZuW?M*HQ-#0S1{#SM;Qf8m0-)J?T3JfTZ6$% z|G?uKgIBDUl30E8aNa5pke?_=(PAY6sjSs3MJAH(yMpBw#Br|U=H|e2N@-@`8k~3J z{w_T=p0su^o@MMqa7D~1%J9i9+YKgnj&7)+V6>%f%jBiG8^%`v3%w|cSzJhX5*GnY zYT8&><kn_-_89?G%^7LQmh+sZ6b|X%!7!BXb{zP5^NK25^J+wl7mtmmL+-{11uayd zK$V8=fg>+;%QWsc_lgkUt=KWk`w<AsU}%Ll?iUc+Wt%-Dcy?jH(bw6HUp%g3la`eA zEgrV$WxLE9PL~oWQVm<-6zQfTMDBvvDnvCN4ruvbTU=z8;t+j-6I^6j_ADW?SivdR zTlJ8{S2L!?#301_S)ZD}7~eEv*4_Q;I43%^21q^veBHG|256+o{FOBJz8Ri*e+MK7 z>#pz3KJ{u%0-vIYD$^b*^chhh+5{n5#b;oXMD1?Y3#k%M56{eQ&(o%2GqEBI<ekNr zu!?1}Da6JbXRS9Hp43;&d_M_oqikFt{c*e;m_HN0DJv>(E`JHOVIWlj%%{MkU_>DF z1>;jth%^1rSpa@nByki6UaJzf*>eCjMmK%52x2HHDCN}Te5b-$RzAG>NqXU%V}nE| z0*r9H3W6w0<gElOw6M>U2%@%Za+-4f{>Bn=r#7US2S-XQlis7MRMrFr6~<#jwE6gR z1=CAcaC#k1c3+%!xqMZn=pqJQnuzSr6xJXXxt>2wT^zU&!<Omyb;{hdn)59B>&QsV z_U|lQ+SB2l>qTmp3}xA`rRG7*s?%PLId8YxH61@nHP4Mmue^`Cojhi-yMeHe5WVQ_ z_B9*mDS$N8&<ts=f=0E6?ek!TIqw~^3MLL+U+jNPd6=Z&uH9&ej^3=fwA&_wi8C7g zqK*_<v^{BRTUjaO<9*2(st(J)7EKJP5fv1bNdfW_hVu1c7j8cMoYa}Mr@e3fhK<F` z9Tu^$<QT~>z6f{VT8hC=ikBT*q5hcmd2ZSz$*%?qdi^huv@r~k{#aX^AZ=#z5^hA` zCo>!JOVhhjH>>GjrFxqYC)d8up4(Z6=Zho~WvBA+)}yH^mj}TfbhJ+eAHbNkxu}Cw ztY1%M@ojh-?KaH)+|AW`)O7I$iMw~9Lok#EhjWUNj`Rz!Xe{`~PH3EkCd6@3U>fJq z6Qc`*y>pG%5J<TW1851O!*g_m)UD>K<HH))XV>wRz<-?V8|naAkc)y5AX}O<Y#`tj z9T71c0&#$_8_DP7q;H|slC#lbHE5!1A&gq1aflL1qtP?8%E>Fza~-F560v2$bBFVU zODS@?;H`T8$$WBP?-St@Xc_{M_Mo&z-~;xTzO82iVR>HY5T?EZXX<y*0NpdFGWsI+ zi=l~4rRbBMHXdya75mFl?pCEhtk8@`(SR9z$9)W|7rku)>R3MxtDl33D5jIcwRK`R z*27JQv%uV~`IM2{(?TzvYAgw6eua#By=96C%%nmL7pxIC<U`COMACj~Z699V4cpFy zu~S-_?$$mUf123fqL(ctgl4VXZMfJ|78XQ)w{Y;Ov4lr!>Tc1dPg!--w_~kOEbFVz zOkRnPK@a}S3IRQ%2sBD7PLSqnHtXrGbLS7tTG!9mj+xddz~Tb)r`{V5UoGNaY$xnr zt^O_B&nJU#V@NvZy`oBYMn2tTbRECKhOU<73O-OVCK$IJsw&yj%q$%_p(6pzzO--& z_#+B8{LjdiC)?X)YSLtbVd_N+78D+OOz6y;3o;&I2|ogd3;}6jcsGa3U~;S>7o2z~ z0^dkC3j1T5ANSJW5xg0PM{TQ@{P8{yeK;PJLpPG(2|KR7UeehEu-@Lh5!ir02mwO) z65^D$pvi$8fy<*c7}ZOMj<(9y7kjT3EKVX0mAW4>o*Hj+tq1C-^V{1%RTWYv3VcW8 zx4St4$vNpGUXlszfapdmA@Pg?L2{PP&V{^h)uOqUtEpw=Mij7?T7krM+<71*`a>FJ zHV3?p65nQ<cP@+{N;pIPRIJC&4XIo~kL^~r%17^2%#Q=t_tIuF)VQ2oJJ_HRnps<g zjx$`Y4OXoLl%g_Th@8sRC?+8trF=I_9{N(D5jwUt!8(M_f6yAW;(#HVha`(Vlnrz% z1mI4E@N3TnR~B1G+peTxcN7AYI;^_fkYtcwQ@w7M2xt-91D;b`sU>V+&uL;szx;#G z@NX<MUun_}!q9}u9R1&i$9lmZFlL_Icd}_E^eSu)C*i0wyb7UbZeoHH;=4kfQm(B{ zHhR-K3xdHwgs`j)ow#_IoT7smV1H5X0-;O)l8PlX8w(AM%xiGV5evgly(X0mH4XCk z(f)##J2Hn~89EfwZ%79<ugpB1tg1FoynlyIf=W7*Js7Sry_c0eNls585tV>~Rbh$a z-Y9h*TUx6T<vjs~^tnJEs5h7$5YlQNcPdl=U4_izRZh+nZmT8qN;Gz#-qIcXt@_du zA&{FJzduX>ba0>}#CrBv#szAk@=cgZHR|4*zJbP03ERqwqA|VV;cXPC9b0j0Ui(2E zkOAsD6V;WyKQ$?Ow6b>a9I=xk<{1?lw76W4E1Aa}tY(pNFSL(bM-;FTiD9jnb3P8) ztz1V~A=KFHe&ga)Q{$XLMHa9$)5mvRRte`8?x3wTqyNqT{(g(vIu_Ul)?sRp`H|Lk z;o3E|a8@eU@g~1JqY<2XU~@(t^S9VceN&55mn=aHR8m<I-%0(IaQpNMfB7sEewoMW z)H;wB)phEr7GBu6YY48<RISa6Z58`3T1)q>pzh?wfb*~U=_d{YYRV|4XH5gG`$>9R zwj}RTtF9)?ddFAGiuvUStsyse5Cu03ogri+jXxRB149sLW9l%8k!ss%-UgLyqE3a- z3qL_oWrRU}rq99P?c#!EfU0dsgfL1<N?EPbfGR;!iDrh;EdhEa5>Od0Yxp2RFv4yV z_?{&hz#+PB;@fEgFeEaJ&n_2+IfxIdOV;<QmSK#rrBzsQmxlI-i?b2NcO8wuz-a(N zX}l~gzpPYuQ2Xn03=eOIboNc=ob8K)5oBg8502gaWI#_WlH&i=0?;i75PIODz*R9{ zfZN|Yy3FcFa9@G)0N!_q^a{ndOG5Bc8$RFj?ov0lKL1|lX8i7+HF?iqGZBvXCw-U7 zXDOcS$2*E5)m~C6g{>flRR~ieVaRH-#8q>*kR5tRiCWn6)Qjoj)MyCeR5HAjw*Osx zTyHygq^4NfumX4Ig5pw?(gik4y#F3g1<XxB=Cd|h2=2}61k+%5NV*D6e_Gu-yavKN z7uVo?kojxs8`%#EZS@uud(y!cP&-flOg~%m`FgJ(D+GUoW&hlP2V|dxF`VPEmOmt_ zsRYtNJf+G%i~Ff0!8ru}<cowM=r>?^h!Kn4MCPlyV2wb(R<J-cys})cz74)#%~XUZ z)8zuu<S)6Lw}-W6lNHx#n2<>5fUE^2Hgiur{=w?!)Xf|Et#m>VWf2{uWe*#&XYkI2 zTI>P>b@#;aFs8ecX?yo^kRcyV4)wac>Yk{$7lVa|VfV2RU?%l((>~V)C`klvG1PCn z=KKvTCrfSL+mC<UQgb)PDO`@$*PtPl#8HUWB;cR_!lBZbwWorY59~ui1^X#SuzV`? zYpwH3)$1;!r&PrY7Cj-1eH<~nc5!{e9m$YUIa;}^fxk+?2iNII)Bux3JilR?vDFKQ zqE(xLu<1D!w#NgZhLlLqe5*!1T&nVr1>t<&!faS(k(;pduXBd)9Isn+d#;L+FjB*K zX3k{3an9URaLxwt078-+qpb>M{ev0N%Iv#FgwaMD0EkX7Nr%UcKAbMpA3aF3c`>AU zd%Ksh*%FKj8GuDd6rw|^VIw00@9FUcM=lGCiIEKm{7Ohbg(Lu`N=L>4`ugP*+(|n~ z$B{mLYMjOiC8(Q+tP1A)-6MttuX@A^<0G5XNdQVu`IAFZB}(M#_C?1&!Ua;KM&tg0 ztNC`s$5j3=BFg4O%r;qhn*E5SapW~ZCVgzrr;zeZlHZ8QQ<=S<4G+?P5`U`^K4BQj z%6&<+I>Kiq3IH)q*r5-TN~Ar&-cls^;wY})t_uNv>)*8?Ll(P8J~;@;E3XjDpamdq zhYvz-P|<$J%^C@Tvlj*7LhP>rDKx`H1L;40u%oYq!{YB$qM|r)k_tF8B4hhoLg)V~ zB>;qiyajUL&{C+Pjk}=9&I~TEan6$Vx$$m!@s<q3nMLkS<j)kP;D4VGrn0Rq;Uiya zIFx|DKh5I^3I`ZG-IzAtCf1;N+7nQemDZbV0}wnvfVA!56)_S@Vy^0)#K_1;$+5;y z1;DE~10WEbX-3zk>l1W4e{SrT7X(Br;Vv)cYB4`9Ud3=;yB0|iqa$>qa$Hl(R4$pn za-x6SZ5w0=Rm+yR34%9@sD-*9d5<M|#ai1uay}kJrTT&=M<tO8jfyVn0#<u7sI0p= zQ@H25yVaE2*EjcZXBP`4MdQ1n&VNb;0|1Y&ECj-eWQeJ3Zl)uIiHf3yq7eoS3JNM( ze@WE8h6ZI@+3oAUxX@_HIj+-8hA;%YY7MBhOF)w$h}vDYlkqm3y)QZUfJe~`;LGC` zhJpU5CZ|+<%V_H7umQ9~$SvzA%x<v8BXC#Rfc<x3Vz&}$b(0C<S(MjNY4ozfH^q-- zPz;A505Ik#g$x?b?9{w9#I5&cP<T!&rg830Pum!e8K;^VO>1fY7;=*!pEoj@K|upz zrm}n&V8O1-`)k8T5Ej_Mc*fs_`Go)dBgU+_XmjHdwQ`_XBIyF4Jb~i%PVgD+OIA%$ zE|%U=qL>vYZf$-L{d~8tEo>8m&fnA1OkYZCH-VC^Zb7{wgxb1%zYy0(z3z_a+bBeK zLqecQ`PUj8h<#7CXyMJJ`+xB`JMGeO*Pfq@=<r>dN|eMGQAZAZ61)o+HWi~kVSy-Z ziK?KO*_TJ&EfpLwt07T5ww5@wOke8F?{MT8?(EohF!Br=$PoMK`R9z|V+8f^^aP6M z3XeNU`pgo>!xYyq#<sXJn(ff%h9Z*U$-YD)0=6zseJ!Y4n^QJ-t7|hZcr>=lg~XJo zhfkC)Xe)$GM6*X=S~@dZ^%&3|igGi-o46o8fEYZw1Xr}p2T}K5+j7GU;bH%=DMrN= zUTl-mR0;og0&!p%omH9cNvAY02}%z(zJDvnQ+LMCdd0YM!ubbt4nf2+1zH>V4U6=d zqw}7-$fENe*3vX;g3sM=eXW1+MnOQ1(u9g!q{u$E5DKA2c>?R2p_d!I13wD+j5KWt z$HyyWi|0lxTYKV%-Vt3b92N)m9s1-ogD~UB4VYD$jZ^*6^1rp!pFRN5Wy(0Gi3H7M zHS*659Rx=W8J7*{8R?ZU_vXwLLfhs-ncH!nAZ|E7&P=wKmIj9QmLLT}n;QuB)S<#{ zYO?u&aD9XG*`aNxsF4Y2LY@z{j*)p*7{iHvas!TRe`g*1Yv7Ks4^LX#v86>7fahy2 z#izzimIg`u=ESz8N|4~0aB8E4WYcl6jDD<uFn|=n=}+|hPfPM|KYcUBd~$6k+O5)= zKO_rBk{w_`JlYC0Kt4T2zg$i%cy#!^mD|#J`%9m@RTX0ChTbX)=^uRGe}SEUcDiq6 zd~z>uy(3$`f6T^8lrNfLjyo!2i<By-BQBuwE_GB<O*6!-LWoy^s*@zi$N%GD{5^Z~ zr_a|!z&F{fp#>`)1p2{kMz9GV?66$br2VfC{`u@Z@y4=e(Ll;ysO!IP_;Gw&f*?g! zvDila!!-Wu(fvnfHTmO7T6x&AG5?Kp_Fu4|c?pyZL&tIp>+iwme<|w!bUv7!VFRlV zTQ+9@G04AN+B87OGIcKXv*Y}4i~FCII2e4)zU>mYIrG<s_@8n9`pbsd$I0c<5Ngl$ z|C{~60O(6=-^%z&MQ_do1;-DHyh1R2K6$>P8=&-G_xkU<f5@SEFZVKM)b8Z1`r~`) zvUtq~A6)U#5&$!%p0$Mk-yx|%D_cy`Ps8YNI62XB<WyKP;;xw&u5Hn4Syevr8q{4q znxy~l^?#H+Ze~bmL@bn7KKShx+I2fjO-x{QGcCAX3(_$mos84=Z<_oc^77Rd+DHu< zka9({)=*l(zwoza%zGC#2wtj1_<s-i@!)+}eamPtSoifT#wzYI)iCcB+a^o+Ww+@6 z!CQCG0O`x>7O5)0LuwROpc>+hVz$mVc#Rb<YTfB(w65qK-R$}GX!zCZW?-BKJuffg z+f76FiAJbI;14OMO+=F(pM_4Jx}oU{AsCvSz3kjCaOozVa3KK|2TbESC+R!oQWJ3^ ze}R&Ji6h$`vH2+1)*feIJax(GCk2m+|10_~+nW-y(6-aG!IzIcS0;|f144+Ht7m5R zJAjQy3MU+aR}5HNUP$3^Mn7^P*gqheMURlNe2ILwx6TU73U>^rWBj?3hAXJhK}Sb{ zEw@i2v+2B`j!kXhjh4Vu$`fI=i1vgT^+V@38w~CLOVGZVQX8e=0~v>U7NPg^g4<yd zQesft&%HKxADTXo?kjIXKOoMu*0y=CngtcgfwJoKpg7vWD%ZaQo<FLbYb%^Y`?89f zlt<)>XIC5L_^4`$j+gyTx59<@JvTtBpfH6_|KnTP<NZB@X}g;mztII4JXEWq5x27K zKOn@1NS_=NIu0+<qL~wN-r7E*0zL8niO~LSoBh^+&_temXuY7TtQ1^h=cJ(ARS_x< z6avfE(fvv9ul3NT2&2(}_|5xgD#(X?`J#zr!_uYB^`sKrug*XBF<#1k2mfv$tak-v z`mo)OUv2dc?@Z;PQO5Kbs8nfyX*1|9eBwfhT<EvWW*ROm?}y`PUiaa7K%wgH#vVSH zm~#lHhE<=3z!(e+9!`!mez$mem8Vi~K)lZK0JR5Z5-dBwyH2BQN7wymCk)m-=7sd9 zdg?Nt>q~L5vf^kgLx+~>6S3y@%}<86<RO0C*M)*WnQVhT&J^prb~K-qZh8)`BIg4& zAdNRVKBk<q&2U>$RGBTN=q%5=xb%+;j+|;NEe2Jp1XK2Vh8y_(77Kn=!C#>8)=*B> zJA^nWTgdL8JZXKP+kXe)D;K$4zug<Cna-iL;6j(B3oVHi#zVrZ$>aH{sNx8&I{|s! zi=XaAmkk~>WGSTVm33!$Z%sqv8EKeSTKg;3Ij--uzN<B``#;|=xnTg&s^q@&fZG;- zF}H+(^_ct3Bhc@!_bs2Q7Vo2O<jb5QR!0P!_c7mSonEs$8h3itN4b^nAQ@8K;0pO7 zS&W%|NJ(hsV~hT>+o}kzQq?Z1A^{lwT7wRgEHgyR`hZxX)p%7Xy;{LvlnA`;)Yz29 zXM85b>8wHqy)L0vG~|Mz<RCd$sgFB|s-(F!<V2d|X&*>~|0(fwVbY+06|;O!UX=>J zIzqIkD#;SaEirsC*%ls~-<#WT?UbTXzZ9!4Sj<E0+2qelX?Jf>B&gj-ip4607L~pC zB@#1YBZsC;3Ic&eDDwN358a||ZocAM21_U!E+Di&E&bt1hZLUchJ+~KxkpVQil9d7 z3=ty$GdCC_&^q>XDmN!LS#KcQmbH(-oA@-zK6YNrujV{_Th5$?>Ho0zRbh1`Teu-O zL4reYcXx;2?(Xg$+}%C6TX1)G2~Kc#cXzluGk4CJ$;_O``*h#B_gCG!`mb86mQ<}@ z1>C8H0Z4Kg)o0DA@EPu=+X`x6<w4KZ>Ms+V_kFrh*07*{L(m04DF>s;RFtz9V}w?g zRTQ0nVQOEHaa80Ahig*~2UfpnN*+M_$TWg_Tf;Q?CbsJ}?K6V<f%a@i1f-t(MXF?D ze?4w4of*hc0ARs`f=DDiq^4h*O$0c=X=z=ee@>(_vB|k8hf12Upyt-_f6`tKnKm@P z&^ASo#bg61n$KR)DO8$$IifRtE#b9ehkc0>API}+WKd178ZOcOY7RI6ByEVy)at&a zG$Q$N!}Xp};&q2NQAZ=#!OpWPO<J>FtskQp(0TEt127)<XT{!XEaON|zpK8(GYQ&2 z=~i@)+NOuFtn#-7iV=>GxIIB8*jAtm+l@*^XVRJ17_=m$e8~PCgh`Ke|2lakqx&uh zsQ&cNm!BN)^*WjZ$5>t+B8WS6f7Pw)hwu|TTuvL#R&ZOgWPHk;ViGx`h!GVjLADNb zw@n~hUE@Kzm+zwXSDO;gUptm`c(naijLB_-_L?ZPOsB9wU_S^gQq<83yN=`@6myCJ z(rFQp^I~F0y|Xe-g;pl4LPKd+i|p1*WTo(c3NYcN{LYFY{lp=9<>uBs<umwz^Z9ka zF7*n~a(hJdD)y!m<I^v-JIAFp$@BGZX~F%)JQtcM7*LASE<}K)RR&C+j%UL^GTYbE zJ(beA!e59`t|Kr*H-HI;SO*JRG)+=$m`c}>354)tsB94gF-gXdcM@$*-MOjND0F0i zhU7*(atOy%kV)D(me_o&4$B-(Nc;^;B2~(<d{kav+8oc>=>NJrcFqt^?Bra`Xh_UO z7<N_$=GdrcFEj3Pwl9!K3%CisE=Je1DCERpky0dZO2W4XOG9@03usbSB5QQdcFf>< zTHy#Ff8D&gar_z*Wc-GPqY(5d+TFTQdHQlEH(<VMDdZkKI~cWSFI93TIK`^7(4;|4 zK7>F!0q==|<QVy~vN0Sr!=U$msi*acP1PUkNBw=P!0N+OH~mVmXv5t_p?t_z%W%v~ zK5#q{{h17oKhdMEyI3Q|=Zd5KYGZ_bO%RLudlM)W9NRMX2e#w;Ot#9TO<H4B@>L7) zaQ2Hig6~=@bO!}RmgD*-XvKVrcO5)1uugs}?L7nfDmX9F>Mo{uW5}Sm#I3L5!^LN? z8%OjX9`B+)JKI2v@Oi}07Se1|Hr*3<cmi2@?cn8tZCz#{4n5=`ialjk9{BSu2pS`t z$4G?z>M;;be1MQ)w3-q`!3gC+)jM3~?G2m}Mqhch_MlVoU+ir5caEQbUD8WqJ)Asn z0k4LO-x}d0IhBU=%au`8XVsyi^nEE*=PeZ6&?ONQOqfqph*-*5Va8_0bguZcH!i-t zq(B0uptKbpJQ;Z*1s^ekhCKg#gpd|<bt>mF>$B+*KloDmY0V1{L9BS}duwnRjw`1t z)ug@gYEgG`1r*)c0t9>{Mb%E9-gbt=UCyextm9xLj5;7fJB(6>GhVX{*pIT$3MAiJ z>k!tjTA`;dRcr*lL3WjbU-$OCj?XBvV(s+45X(GSo*V^MP686K${noI8Ig9cE~v%u zYnA&-z`D+jaT!=p$7+P0<N=KEeeZ_ac4Z#CU`I?Yye0X-WE*(a>-Q_jNItnuUNm6- z?b6q?yq8PcV>UTozO8{Sz2hKVU>f|VTK81PfWAEybTi>2A#~+=<Wy~n_r^O>+<!Y} zc>Vr8wyM8tm)TN_z#DCNX0|Z$LPxRCZU^dC`oOsUOJ>=;LnRt2g*z9BzDudQco8+) z(}Ug*3SB@*0z@uj8WB6*gB0=xijZ)DXv*BTDiaQj2i^N?em2_Cr@awnY#b2^^5_x} zT^`{6LXkZ%gtvncxkQ~~KU?t`XW9E#xm;FfG5EEeQor~@Hf0qOG&n6J{~)d-%y^~~ z6u!dlKzB*n1edq+3TEq>DB`HF{kF5TvbJp%z)6xU$oHrB%e${pa-VI5W^K7Xvwop; zw9p8MVm87j>R*vNn|2#_!*wXHyIQ4ETjh8jQZq0|jO<)qv+T=APX=r-YfQk<ysGgG z#^6SJvBC_G*>dPY>lf)QHZTGLq7|3xOpO!D(tZO%qg@9R-764#K{byw%V%D)IV^mQ z&@j2S5dblXFW(Q^dg<8=%8os*x)ynFcT#IUPe3v*`rp9c?_LY<)6a<=Fc;m#f_tMZ zq!QRfKD76u7lJgVB^wf+iL7_xCWvrWB1RPH`7I2VK>IXkBjv1;M=y?2<RLrqq1OtI z4~<*I^94*o>d<fAgaP5(;bdQYvZID0jKhUa-K|KiOF|W7Sg=^ye6cY30M8mYi#SfP zzd#UyVGK@qHhr#rEVNj!m0~ychkk>qL7C>7;<$PRoC=e;(JgpKl2YRm1Fl!s^gYSo z%q_UESzs@K=cz#kq)DkHy5AaB#~Za9fyIR!$C9tV_4>KYmk1AD6cVpe?IwXaaq-yo z>QvxlVJHjJSv;joCNNf`h}aS3R8G&DGqc$~zj)I)DZUod?P{i86yf`KcRScn;cBfC zBA>+->A8*;P;Kybj)a2z2ZkU4e7tWspVJjd=f0?E_d<sR-0(7&I+D0c>WS0gqD*E3 z*(_4yJ#|7V%(;GOj@-n=Y%9IL*BuOIeNcWH3cb=}OQUgz$$UxNke}wfqreC#<4E;x z2Jk)>nHQAf{e9+xz2~;h-<d0hm?<Yeqnse9V>gF(^0H49G?UkA$m4ahZd_B{a<MP! z6@=Ft$EgeBrOfDi{GpSC?Jd0fsB<DsA;|8CJYJ}i2&<cBCA3)P=zjmks*^-+iGAxX z(Ii(&99lQHp({?%o6{X~LZj6U8S)PMu7z}&5;BEcO0BM*Zai?~6r{zR6G!5u2S&t( zL#=UOZne+x^W$_*{ER1}z3)A#*V_&5v6YU!ZsR1XEWSZhM;px}<3ht6?=qebHHppA z%9M<j?46fNVw6)EI2s)Rvi|3A(g!O)0UYg~1;PUTVRLyMa;^v#c@&+9l%eyzO5+zy z|D2C!Bcgru4Kw{rX+}NSFtL5(bE<x3RA%$wudyH#rVU*icA%FRbe(ggZ=|z>RbzCN zR;-|>zs_mE%;t?ES|=m8ofrHv(>s1b)p<%|4$nY=)?gBHKC({&-}aK@lX~{CVx4Le z`ME=i)loTDA;;(F3wX%oj3k)`!-uaPko3&MwoY{SEa9pQTi;amST1;^wnGK`EbV<5 zhUoE!;QRz>trQ5oRWPJWX?@i{I$)iy8#Vm^OAa!YeTt;jPBzE=)^;3~)<!hADhGU& zt;RaW9`n}0Y;nH^DEV7LD}kLSOz&2|#|-G~96W0)8L*y_&58Id-@HUoFpOmHkGAA{ zRUfQuI{0Q#q1P@eGBgSkSjge>Cl*!C#gY}>Me3Zox=lFik_y`sK6|iV%*>y+6COWB z`#?>BL$+30%SW+sM>s-o%AZ4G{-Ps(F?RyuTG-iyXiU63Z+k};ZIf(UDRE<P@>5c7 zFI!FHgK+E<(WgH38mQYgM@(t&S?l}KcQId5Nqx9eoNQ$y2xky=8#9ITLvumzk1&`Z zG7S4G3E?vv-UDZxgqiwr;t*B{jCGN-etyv{CGCsBB<T(m)cH0IUTpiI0qQuscp3Fs zzyd>Xi<lY+E(VZ;s06y>M2?8^Ob(5~M^V=^nAx<pwR>Z3ec1o8>f>EP_)E9|aO<Nb zwBCHlSYCB7j)gG_lk3dksMwRM;Vi)Tvd0I9C{xCP0b7RbGdEKgIvW9>tsHMXtLa<7 zu-kAJuR9P*evGow*oT1T-<6veNSHbN*cR%2)ip>+Ri=qJIudO6K<QOeKd9d-cC1D5 z-PM5f>fME&)U<xac&of`7<QY+vU*?{g~j0GM5NdS-6|gT=-a&|(6!=YuJjkQSmS+5 z3}HdHFTIk0ZXRXw&Mq~28IcPSN%pSCT#AWxap9vGYrIkHLH%mp0s;tG$_R~C_z^X$ zR9*S*nqRS^ry)&GcjmenaiO=YD0*46fgUJ^E~HFAyAeHzJ~g5Uwxn0KFs|Q|_|=)L zr{XZ`t#QNa^3-g!`JsL!a-e`?mnn6@dD2<F{|Nb1X10Mkio>#E&U=eDM=+K941b1R zbv*L}N8fWl>iCOj!d*Pp*^Y*QL=IeNJu5EcYX1G^H-V}j(craIp0SwX{`=4uiS-i5 zc#XU{ac~Feed7$`KR7r>Hv2VP>cPA`p5Ie8gsEszieB6{3EUTs!EdyE8~<Bs<iBAc zA1l%I<`Bkqn`KzH0Qns2h@|tZ;F@%~XWOf&NBoB%u$A0Kf!@suOxCw}aoPlt?U>Cw zx1gmn7JAGzZ8L$SOKNWIu&Y=~X~gjS)yDj6wQ6?1j>u(q{i)9u>y_{PTk@WP?o!(9 z=9&@>Y|>c-h1Z`tE`wB0GEw35&;buwGvYotdBB%2LG+I_yr=6#j<;AjL<rB9e2(*J z<ac3u3ert*TV<-7IBW#jI%fkCY3b<+U*LlLQ~K<W*x!GGqZ)}I?yF+M6&`@hNFvoK zR*02ieoep49gCA{-t1X35$NJmxN#4>XD&{-k7mA~cy)^&#*jc#U<Xe3`@U%Wi75h= zRG}3(b;0}u|L=J49e7mJCFV8(7|8S)ZOtfPUi>v%GYaC2RfH4reVFsLxMed%(GdEg z({mbFZQ!qjJ|*Jo^(_V28lL0-ceCl=qlL{0-V{c4=HLg#t6W^+ivG59C|>^lMaoM; zbRnc}OyXJLdodp>zGuHY_l4%DwtQ#{Q7VWqx3K$&{SRXJhhXkcg1DBNEF)w0b3KaL zUH*eU`cDLY0`h0mQqFM20}Z$>Dcm9IzxVOKzhr?U?pNj5&bgu8t(OcH{e!aq&lvE6 zLw-@5ZEQY38|7ZC*CdK=>E1l$zbV2GZ=pU|L6-VD`VRv4-=JL(!v3Ym7LX_q>xCi_ z_oE?Dw{>gvM(EhANs?mb-&}(Ke9mtd>C)>%3d_tR#@&8+q!eG)pJiKa&G_R`N+x<5 z1vxoI|B`_C1sr98pD_VLyzJ+o_&R?MxyD=urI?}GynRxxdV)Xj{jb#g=NF$@$S+MM z#~Ho|Zi-)N|7J)0XG;DStiL0g*a<mrKKsGO@VD6Df8GeV@CgKj%~=S$J^oj#{XRgW zyUnoi{Qo%|{D62k2EqpSUtQGy4A*ZL*Lr|PZWn!fKmO;~-~qz)Fd;U7DlhQg8S=*n zy8evix7}~rAN&IFUv4L&0oj8<fyQ65pno0Cwx1~mE%%G2FaHY$6A?f%h97_V&s#-* z9VZ_x2|y#7_InNX{{;ghAt0B62mepD%zyCWr6C41Le+Ms{(C8sKQI8G@(BTu8IOke zS<&F{82#U0vQU4f&%<cA6MxH-{{^5wAprw<KP_hJ9R2jO!Ndwgc>9jjS7imq22)$^ ze}2Yh1zsob!of+j3HN^?2XID1`>~7%68`xuzjp!fE<mPaMN^JC_}{<88-!g70Pw&< z7195~SHtONHpnUQ>xaK@Ao=f*`9%JN#n#c{f6D6|7NGNUD?~fK|B5dF`m8OGXZF>v zRKE|yqJYjl;{ARdl7IKCfkPbR{LRxB%GZ16xIK0D`w{gLI)+O|Pj()xAoXJBvN80} zwo`fPbQZiwpToV&s-gGBG5b%k24rSK>_a~>18rG{`efgu(cdk^a1tYN(7y$2N@3>q zXrfznPJb2HHumsAG#rC<vakEpAC(!dt1HCH309k^**OWkz`8d3P{QOz1HWfDDje4G zmco@#Y$>$Dh`y!g4Xoa<C6Rxps;_{5`pF5{&dIt**1YacG>S`q8e*C08%_K7XDB7< zBy5Mqzu~hI4-lABS3_1nHr?kah!?QyhDadHWZAuHg&k?%+r%OSCv+Z!KuBb6MT!v? z+yBh#&C$QmaA{b^q^`#Ta-dTek<P{jliA~3I_(UaZj=*=EH1E1+F&en=p6|7n)iMZ zT5pmdqZRHwlNTuaSvZ~1d(UWkXcyI^XqOyJ+CV%!axh&wqcm3Xr<G<9Tu&9w92gi9 zbU%MxeOc!(7_aqP3O#WVcM)A8mA##qhA5MbEfq}{*A!S?`>2Lpjj53vBc~s<kinub z!c0Es+{Dhb6`ODbuD=E+9AR4_P2PWc#%9idkNS&F1qdeZ5+E^X#X&Lqb7l02gdbE3 zts+RAt<!9`rPw7->lqj34i0b5#afxjO-zl{JvGTppQL^#Z+zfuyLu`eldMV8+DG!H z-KcMabSf6v1FkNud7%R)?gbz2GnHA<wP#FNJcH&R@kXnG*rh_%uS;ugZpV7O$U$yj z^@<~CgLDIZF`W(hU|f0IMrC83ZZwR$eABixX+4j$?C{FTJzn->ZgyGvAXjK?;e;A& zbmr*WKv)#Ez`;Goq;UIJx-gUD-FN(6(14d~P65v<;TJMRM=Is*SV4JD{URs(H$PSW zqc+fcfVec?g2-L$MM&#>{tVJv^f?s9C8z<<1CRq5UVSz!IX`X-ftZn6)EVW8bjnoo zju);ST7+xEs+2h<_e|hKs(+RU_b`Ivb~Oiz)x)o4XI96n{=JZq@>*qBJ+y2t9?%4R zRJxXL)hB9}Bk`(|M%0a6okDLV@b@-Tnp1Q}yp6=K-LfHR9@=_GK4;s2B0%{g{UT4g zX|&;;RKUK9sD-5LK3>l!6#hixy|rB%i&Z{OFCS|nJp;YAZ-s}ZY29DvM%4)IZJwzR zLO2mT47fkahub5pT2!${-)snN3rlw1orMl0kMju4qH97h!6tdTI0@n%`MpVV@K~F% zI=Xuz^n8;N;ORkkC)6{$Z{(mg-F^pB3@Soam(e-v<qmp981*2L9#xxUIbXSicZ-B* ztJOcBPU8r+`&#YAfza(>+w`p(;;EXu$H$$?lkhafi3Ah;wue>@zxrk;=e##@aFE?a z5maREN9~QGnl%2FwGn3;GYarPQ~=!_#r#flL>fER)~4p1<ZPAsQNbX0g*r3o$pZbQ zRGwt0T~z}Cjk+g}V`eM^P!gj#GZ=>5#T-nFMouZFcxo7F3GFaDg@ZdhP4`J&4Bg@z zDwClgX)MG#?Y17jCPvXLa-XeIDT%i_CGba@Ioze4d_8PhBoBkuZp0wN7vAuB6~d?x zrx$EinWPr*WOcx%_L1_YqP3y~&+$+&Y5qf6j@xuPJ>U7#LOqHh!P)lr;iB!x@QY{V zSOrdAKI1FeooG9EBL41oXR-m)YmZSUtQEw>^|Wzvr@iNev}F~a+~}l^UlBN@kW8MQ z|5U)?_qC!7UI)K*7UF{Qr@H?_=Yx!q*@FZhZuwA9>GZ~@VoBT^`7<{p36AXyEIuVb z8wk;EcbW&Gnvf*G`7|q6q555yQlklITndBy{Suk`xdi*``K+EKZT&7zoToI<HweOf zs?N{_Ey)&x0nBe<ihI&&ra5o#pSS#G?$>d^1wn558dc$f(I~t1^O4z_9txvHR{U+J zVqE3(YMJnf$yM;Vt>f8+wz0TErvx2A0^$1aVy*q#8qtu;;Vt6(NGVYIBue2Wc1eS& zXrS?-)_4>l`{P;5>X<?edC{Js@m0de&7!L$=mQs2XktEXr{VZ!!xmk6f_)$k+h3A? zEU854@tS9pXb}a`Z$Tc&ALmS#4mPZWn6V{gkBjOSqf$U+qR-qE%U?%_Cw}oyjlXmP zHX2PWhx9<6b>H4b?Hb37RF}i<WDR>3We&m{qCCQ(@vUlt7^soXl4%Y`bLSxLyKhl) zby&StbPiW^e{Wp7gJvk$DOm)k;|tLriLW|<PGM$%h1gE=xw1UK{Jm3_Ac+2x3ucUi zh9eq0haJ?9{<SOzvxE=Zs1~#BtTs`4u*_Le{h}ZX7k3-5L}<fjeHXbTG8WLnJf`}w z%*wfGgGwDOV9?k_<H`}}G*>bArUu5So%B}3r`QD`TVmTgyPJCXK=7Atk2+>Kb14cf zv!3IWbSO#Pn}EaJdZ&=v-_NSZII*Wr5xP_^IAUnDiO>ZrI}rVl-5>1mZt!023Tb&z z3(5>H-vl^~+`X!wci$82u#<M(Up{D*mGM)`1(K}!>6X*rDZu^B$e%y}>@95&2-^F( z(q|1r2A?CE`i?dO>&s`%3ag)kNj>YuTBJs^v#W6!aaAMN>~<7ezOgX?6ZA&u`D>|z z66=>k^fQo%Yaa1&vN!<E>a%kzrj!JaE@^pzgt2`8cHLs^6Fd~dZ?M<MzF<fsCZI!9 zzLROa(BJuy357*`=F<_7dvuj@RNm-elq#IGFwaFK3`Zr*qWmiuhohxRm<3hM3ZMJg zWK(5uqcgWT$Ez9$XDpy%K?@K3lD}Ef){P<Ar2TxO?jon-%MF2Y?fx9T+?x!Be8tSP zv?lcpg5=4={QZUzRkrH$^izI};w{T}X+<f$+1$HOS_|HV<TgDvgm7U(SGF}K95is# zIt4G;EkTg4cVXb%g8eM0wOgum4TI0Zo4Xf1gr+HShWV_QRj+NMuJ|3miRW)MP9*PS z6CQ9nJ3^uD{2+Wk1~&wQYXyg(rX;I{#5<~}o-C%<VJ+1y>Fzn+t<?j<LZvhH)67h* z!?{Sq>K6i>ydOCmWB9J$+*X~8k~Yx5KtLY2TSi*1+Q*+-$Ddq`3q7y4?xZs5KbEV} zDvoc?aA-CUe0T*W{GjZ+PiB5Gn$q(q@SIEQ;ZAdX<`rO2S8P*K`|NT5*djnhftngi z_c~Su5sVhxDM=wYr)Ze6#H={`8C38|Xb=@{h;O-WxSfwJYUbJEt}s(skj%}-%()Q6 zm!k`ph&)`)6pF1R%|nFDj6~b4@Xi?pVV5w;%-*^;%iLN>`i(pId^NytMyll4L#cQU zqzk^tuT5>HtKYM#84O4cZ0qQ3*3^@uro20X?sYK{>-A+v9Qwe)WK^Rx=C0L@JeVt6 zne{Qq$}cX#od*5bj-UgnsH{p?JI{dK$vdvZoo-`)6uM6K;jdIXjLHDc?9`+9Jr@A& zN5s#mgw=|`pydL*My(YrWx3QTUcsL|YW+Pe!r4j={QfRvpexUL+bCau+?v8>ZtLuT zrJW)HH&+}S678rUuGKRGP(WQhq|OaTuz=e$>G@Vj!jlQEC<s>P^@!m@jLNF9AnNp9 zU$z~}v$;E~8lT8SvEAjqA#TEq+m7>kdZ0Fs%do5G$JRU<SsB6S7owxtT4Pf%x!S>O zS52r~#k@(KAp}P|Iq(|T;${MwFwFwIJ_W6JtWkOC!jo>!m8Loa8FDPv)9?F>orY4Z zV`8R(+5!jggshJDjECIg=i6qX$pci?`W|?#eiV!~i0o#8hbwmcsMTaO2#-GAy4AB& zDT^(=GT@Ra))KKaxSBsS)Hf+>yE)oL@g%Z>#I{I@y;KfaJ<K^kCG}XM=F%y(Wb^Sb z&4++UKtS9*DlV(o=A$oHEN_kgmKuD@QCLNdzphABd~^>22_jg&QO1Nm&WryhqI{<K z-9Y($AA$;XOR_#>{dc#L0+1A7;1$~mq&FuEvqLWZ-pO2Yol-UK%Ye$!gDsQ}Ie8Bg zq9e=8GuAKkzWTLTAX;}$4qQ>`;P%^rah9}tLnmmul%Jao2&`2u34Wl_GV5Ld*o^)} zfmA5PD<?Jo8v)7s56JL=26N`k5Ye9eum0e6(%7Ag4nf1;YVNp-4%iW|lr8zpl(1W$ z{XTk(Do`%6!?BcTebeh#H4TcPpy7~oMWS(-bMo5Ry-0c;Lm)gAO|;CAyUfA)*1--w zyqc2dR5&nD3N8Ak=(V_3rb#4hJ1}vfPiOTUZ^HoUHJY5TJGP^pIyZN$3dRy)U6V;x zSW-5%)xw-ADVoWAbX}|B5#JoMHhs}E!Jr{$c~w*CW{*&9C{M({5Q$eYGDXLuysx9S zVZbG<x;Hql7@=r<1k(kfU@Hn#z5ISGDQc=9A3EqO<oEC2dCgKl_M;9bJ(!H$=Bk#- z%%&8mZ`()9EpRLcTwO&(#bRQ1JTO;bH_{br*>Ob45$C@EQLOHcRd+cG#cOnmtr;u5 zl%iM;#47J@p@cn|j%sBv{OElnjTAEttA%;%nG>JYa}oo)+#XKNI)~73=&?kb8ZGHo z(ct?NWe4Cgiyk0j6!IJFKZ#agh-j^7@d<B(Ynk>jnZO8$m;G8OO{@BXJru;mEIgo^ zygG0Q0;6Ici>IOs&F~vV&Yjaz@<KPU`r`og_h*a0sNVKa;j~kX0|+1bpjy!rgVaf+ zKX(*%CEZt0RbxXOOi8<ykAvreR9$Vq<F;dHu;BvwVaGZ*1`JPFF(%4vnPD(lj55E# zel>@c?x#pzY@j|%sxFT)blCMZ#sWg1(h!zvQoKZ5y%<z4;=vWluE-SIY)_|X$~**0 zO^6}rd>J0Q+)jt#`~>ytyyr(|m8jKxfh83xXf5^>XLttHA^G0iVyhTJH4%g2C=E|R zb>4BcCrmSd$}<J-Ff!@FZ-bbI!ReIGsy&0-(`=1KrI`^(JfSeNjx;tE<C6ruA~H8j z$i4+OZ$>)&vXCe`k^FKnru=OK*$BDy4)6W2Q5k1|!A!CE8G>_wKj&x}7QS!3pCS0w z<D(n17}vQ3IyYd48MNHM*dm;$V*=a$mHmff9VQ-JMoyVjiT3XUoD2hN#=Bg{JQ#Iq z3!UvukSfIs!&fOiw1v>yaxPKYSWzV$7*(dDcJ3rMJslb;c`#@uDVZ;mO~0@tdcXK* zxZXW5*@hXA7Co0Xn|oTi!y{rT0(@Y?pW}Q(7Hu3FS|8DStehc`z$qPkQj9fO@?;}$ z3gMdw5?tU_^${=DGVSOG7?xtR>Swb)a&mJsaxa@MerdhS6t>`m4rT4zAzP$|o3E+6 zl6~X$e$kCAIIqAx*Q-sa$RSKPKVEAtXqgFbx)kLS9eb%*@W1R)=ZS*lIq4{0pO#AT zzE_jWolT-)k!5fc8jxw?TP&=X^UQj;;Q@{>;9D3#wuCnWtpm$ZY@?rFHUvX7-}*q> z3=JcA46`+1v`2^L+maI&3MoFN=T#tWmX9+H)9;E-xWBsw+18`D;ZqzwtyNsd_ojuF zr&UVx*nb_%X;pU!1?h^NAk(n=Oi(T$aDYN+n4~l~H95K@OUhFB0fQS||G7bA0h;DI zUag-T&CR5ynJ&xWyw#!uOxq5scOsjg5tO7UM6L2G)MV^pe*bAoI*DP%gWXDVwtj^< z?dlCnS(-?pMgz6wwo!l_EFTo<ewF5%r=4bc)x$bEh&a_&BmxjO$quVrcx*|jmD)a5 zC><SFrF^(644_0jiaF{oTD@pU?EHj{L^H}fnRXtC(^38$12t+|g?4J3NNi3g(8gh4 zh+V{Xpr;xIS0EtPk*6TU`c2C2bB??1Iy`nr#^G>ej^BI&-kx!BS=nPEJAen|vaA$m z<}!)ckSDZ7Gh{W+%Kzxw+BR4K_9gc*I8V__8%;+5$R5`^D(ffZEfz9JIe+GFG=^f5 zT}H#a8h_HyKQCnhyxTh7m+2)otGAioy*6WvF#fo>mH>-5(ufBJP;oj?%UFD}eHm71 z^`IpNP~+;xZPvv%na0v#VuHM3>rXsQnI^5hNG@Re)5Z0tI+FL#2H<QI-E?A@MYvEl zHLa9Y&~zm<42UO(;@#f{5S_cawi=+oCE|F@={3o&mVT#6?Jrs@sFi%{x%|Nzeuzxj zB?yU@0%ttZT|uEFW+Pt48pgC%=D)ER0|6Nkjhnqn_Oy>e88af<`8CFTh;ybvOk@tT zB_4T~at!|{-gQuJ(}_pFO1>0?xM#Aa1G|*lVG)8yejQB+(hBF;4?Z234p(R#fy;qi zyIy=+O|1y!-3*E(oN<-u3pWT{qIZ!&X0bgx2@!cU%<K6D#6Z><{h3EgG&?C0-fQ+g zj;)4ozmjT;;P-po0M!1d10vN%1k5=hs=k;}imKhhWkwku%1{emw^%w$t^hDqM?n>1 z`$jETQrIGdUhNLu4l)e2@}=&~b@DkX5&D5G2~BCN<-)bW(J$D0gl>xjwWymCb9Jod znQ0xb<F81Nj<+yv2tX?aZ&r5UIa7o7mca5MW|$f;+N>}gVEo-ZbiA50`mogztLLWc zpOA$WvshabC2+;x586dYP#NCfJBK%c0sHd9Q)x>SI#R!pDm|Z<808s@q^=guw2oin zMUYmzYMURCgA7VyZF|(()X|h<A^TAK1@f&d%6Z%9^oU_nvWI;o>gePj>HrC`jINGX za(~P^V94*-{gnjG?L8H>NX=X}cf*VQEvMyuOb^)HcU@1l7FmqFC@0CXxC>SY0NbE5 z$Rjjbi0|hY=}nCB(nA~K6>x&~7M$u^>aONCdaJw1V7T!CUyzr(PlS|69E{Lf84oIY zs(bSnZqQf}*p@`wbx!%*_4+L{x)llM(LAY6@fZ&1VcWY-4~9N0M}Dm6Y5605+iD3J zb&yc%cSQ%)!uRib{S)<i4$Bgt*VnTz8jEXyIy@QRo<^+6NQ{#+<3K{NxE#eBi!s_o zY3oD`M?a5E_ITs7HC)<u6z^I@U&XiVWGnc(N1n>hs3h98b>aGW#N(^yYl=4sr#4sT z6H_cLe9}-qS_&<-)|*Gz@GBQEInXU#a387V7w;9(Ts<uprn4R!k0|&W)as;=)lz&_ z!?yIPW8C7l{5qgZ@ogE6!Uc!zd!23kyY@w+t8}q^0$Za05<~%Hl-GyeM?A&UA|tQ@ z&U!?zew_gTqy*@#RquObN@f<w^TpN#%1wtOxTJ>JeV6ga_<8wUuEXXl7qYojbI3&{ zCfA}Wf$q>*pPph@_V(2}$w~uSx2=AR=PLa@T~^RL-f<Z+o|q14F>#d$c{TinHYT=P z7s!vODloWvDuT4j#c`uKvSX+Y?(NLrCRwOLkSI&md%Dx_kK*s-fJTSZQrB(1T*C%+ z0ICz6bNO`!dqTvCCpL_yooFm*CF-QNI$^>N&fa?{y*!}R-C6FOY2FYG_RWA+yxt%G z)ojtocn^cXSYGaUQ4ykUUV3zN6o9SCG2_g0`nv})DvWD>h5vj^$U=9761w{B643=+ zRHbV3PF&_P`=L?$IXPos7pFaFs!(?yWf^_a$E;T_Z#d#k$%57MqQ@TO&#ZGl=}9Xo zCUE(-4$V-F(*~<Kz7H`Md!R9vO9USMX0v8VtaHjSzfkfRnm0}hfQZ>wnfB?O|6qza z>N5ps#y?rQiSfLCAQ7W_i2V`tG+NVFoBlJXbv#brevaK=?Xo`}^BDsD#Ave2O`3J7 zYD4v}&48d7p?ne|^iU)%5cWdDY)&;sNy=_1xRuqeu$sXC^)VkX1d|B~{VO)AM_VSf zTG|e~=ce)@bym+_F;^7gjS?3Q9Y&DN`X!;xvRLZ)ilM$@7oJU#B8EWe^l+=lGnEG$ zjX&{Z{qhM}*dWxR6OCR923%=kz%U0OgKs;;r+Yixr%dK$(~;Hkl6NHwvh%5b2Ix;% z0hlUJKmjkfuP(H-fW@BcUKI^vpVJdQ{rt~J{WNAXem>ABnpjYH=!;Z9a4~lxGDhkx zD;EM&US=zJC(?ThT~aA5tA?O-BGrH4o@-(6Tj2E$tPhsYEwQ>hAX#+&+zm=<J58`i zAbCY&WwOW#=Ne{f6lC~oqrkuY>vUZw5#%S9gG;1%O`q12q^lEbBvng>PvVs%w!pVI zL!Fk0xw0s5)Lq{)om6fp`6efGRh}J)1J|6nnqA(zfGg>rHcKpVKN#DZ&6r_EVTGAD z->X?};s5)fH6u0>@@)<0a0cvrLegE)lkc2g_mf2o%&LiEh&);K_V1PWYLu_@aB3rQ zys%geK7pVa1l=`pibp!S8nhGEfWP95VxI68Dis&BexJ>trX6*Toy1uUmo}E(MGc8& zEk?n2c-A7bOf*XlVaY^^`cFhlzt32WJl^53<;)h;Z@2(jHOKym`w}9ZV<O`6#+*G; ziQwp3UDE^gIGIAK1GrUqe%jCP`^ojrHm6_^x7<3jn0N9BnpDnmu_M@wqm$Fal*-&E zosQ9<4^e{4H_Y7s2aoCvfsH7A#L=Dt^=M%qda_J>$OgQBxmCTr2@WzW{}0$ATX$@^ z-TCRX$r7jFhk(~(lt|nHE>=MOI($f8Cy2O9L{s|H1873Zd)dcY{{}C0f$PX#h*`$^ zx-AMQMtGm(6;#IJ<n{)ZcG=OOxa_}6XVwJ-?NsBCxvYOW4^`h?Sc0B<u|wUa9ft{7 zw;cyZ)U4WW<%tqW(`Y)Q%S47G#!kEAj#`o973Vsq)XsKy!4v5+GEM+?c81N&V}`Jl z5-~G?OUvu^&Uib1DV65m@t%(7%^hZZWwQMsl{wb$Z<||duW`L)R(o?LDDg!>Grh)d zE3L08cVX}=k}kh2<fGVkNPCLmPYF7eeA?P6F_9Jr>-wjJ<9mb?B>|~KevKA5MdE3) z{#Lq%TR_H=OX={NXVgp)4Fuz4@2YYK2$Uk^lR~z!ma4c9ki@PSY$ePz1-=fO)wBOT zCln|?A}wbxav%R9_&;?Q?-i2C7d}9#xVx?5#_WWW$r58_O6|H=1dxcf^(~;h-FL~t zaz51Xi)oi~Kd$YNzVX_Wc*}s`f+oklwOE{ROQ6q6W<E735SEHfCBSrLPn2LBrN+*c z2^*ZF)iNC5JXM^+-2PDuz%cB%Le2cXaYyfJ+$|dF0i6!T7RzIDi0a)67##0qyP*hG zS@gG3xFKPHj*$IOi7AlV&Y6b!4-%&IK7!h$baxi7v~;)Prh29oLuV2aUX!!x=#CRD z>a-Kp{;o$+Om=%aZ6@*r2~7tI1YB337Vb4QDI(2J{bIb<58@I`%5czAZG09DzU2M) zCRXE&x(lS&yB&8BkRi#F%N--9hrK!~AmGPrEO&VL)9~r92!wB~&;t#w-O{nK(uO<P zBI3%-oudi}J)D^TNOJc3eEc~VicL$!WUr$_OKq&oRsf3$^6cClROV{#N5sQ&tbUa* z^^`itSi#@*pHyJarmoB+f?@qphN*`5o5^&Jw-5bfS+t^Hsf$Zm5}Hb_%utaI2|UE3 zlWm8PXstq`wSa9r&TuV9U<n3vWGX$EbJ94;jCG;VFgWSHbTjQwkn+Evz*7H*y!%)6 zD5;Vq8-Sx|bHC>hg8Y8q2{uop-s+l2$5F6i5^<8>49YlzI<~n-3c&s87ggZErRSk` z;uhJN&Lu9Jlt{r+M-iho;CcR~hMH)@fB&c;=a@1yIx&$%pS`C?UQA{_EgkhSGM&`n ztK`;qX1}fvR@Adw3&)+Pe_25OBDjYFoy<zAB>c0U(YJg+gX1)XwBBk3^_s=s%%B>a z_WSLnY~tQk@@ww&yJp0LWLl0_5pOM=epmkYH<}Sl6#Dibf<S{AkGy%J6KnG*ab!NM z(ag*$I@)y*Ta{Tb&HV$B_8-#1O9lE#bK*WBm3VWz$YS`QSKy9cod;(aPz5F&QFWH4 z&Jg*_uWW#SJp1#@d=?=Yo3pCAbSlXNbt@?Q0P27|%|Q`p7YeJ%_g@#)ccA1DfX8eD z&=*y8X|gPgS*&vYg416t5g&38fceqNs_TF}wQlySS@b6t(FFRZMW*Bs7}1Y_DLr0V zV{de_$)3B-`|t2k{^kg@78ii#PMX-;w=}bc7i*Rt;?w{z8|BX$_l_YYr8L7P8Lr9u zrw%5{WckoXSXx^-L5?Q|{SOlgh~qugiNzLxlk*Rw>eso7WMTv$K4+=<FgBI{Pk;5> zH|R)v^Xa>>FaJAZd{#0ft;zBTHLG;uzryd$1&Ftu9N1v|gYx+uDjy@_&MaGomUT|e zfA8@v2pVI3L67CXqnnca&aZ#;*!Swozb_~Ah&mJV8L$oiK1u&SF#m$^|4U$elx~^r zexESYe!Mo;8W=0UK-^>Abl_wyUIWEt2RpFa5$ig*R8$4akAdz*yX%3zwO<TawskS~ zRbIgQy64(Hx;0n{simi9St+fMAt42uYGO_aNA8^xpT%9CWgZ%Jp8V}QeTqTVG@_Mm zjI4edKohLL`{Ug?{dCD{<2m3N9(%ITuuc`MpS$3nkBW@X`BU0WFNfcH(_>EOIlNFZ z?#-n8USTSUFUyrL7JF$i)vm#?liPBEB2`{LzFZu153HxgdVi1dQ1sM4(SGXPc#%H+ zEbPZEaQK78W{pHKEmPX7^*qsZANdrntp%P$j(ICL@fA*;9P%+`v^Z0P`IYGf=vn4F z+RT~y>{g@`_W;ZWZ@2x_#pG%AjmF<qnnw7$VR?;k?7N>5E(7c%%<cXIfp%=1GqSBE z)`ql(7pIF0+RJ$lR3N4QincEv2JE{~d+*5)2|S3q4sPI+!;NKvtt|<lt@mkzBi$jT zvE-3Iw0(!8vOCChEEqrKX9Asc>z{9ONOu>+wv8E}^4?5&T^ri7<W9O`tWzX!JIqXv z{GQ_YixzSq>byj9Ao_0mQ|-<pLbSv5e7@G0Nw+uJNCs6}=<zesd7bLfq5aZ@b-{J+ zf5WVRJW=aZ*}PD1-i0|LX3Kv#OI~erv`fpJ3lin=kw)r>%8+1LGP#X8^=MnrtR7oW zQuDFg08L#p$Szm==@Oc&(c&xR`M@=$4O0i)X@pN#1GrUm)>*-~5u2-W(+^r*+n~G3 zm&XjC_B<UZ*!2B9@)>+hr%G5&B7fJKoIkZ@hIv;qK(mg1#{E!Y;oEKZBbB?We?%7z z1=pq2YUkrJ$5u>YfoPzeI#sZm%`Qay(>?ODj%Fw>E6O<c7b<-I*6eEE2+#nLg=VC2 z_eKSb<HMkgJ_pT}n8zv3*6L2AcL@kE6|Y3Fr=*>5G;O|$-{>JHow!0ca@{Lk#xO)| zaKhC$Th^mDSXZ?ZB0VpnHsJ_X&R34YCeEatbZshGWaNCs_NK4QT3zpEAmD7Kc0Gvg z;XGBeg8`Y?dHN^KXE*bmx)cw;rQCFoaPuW!VX3Z>do=(1!)PK<k}8);&ejf<u_gDh zQ_H#yem&#$7b{f}=O<*!vX7%D5v+9bmRn1PhWtMgUT|KqX0TbDK*vnfOI#olKIoZ| zn>QcRt~TCggVRx0bGa|ya6fA74QdR(U$AxJ3dP9bNPP^hGbpS`C9$7ZXv$<)IG;y% zwzEiTM8(};cxjVMwUwCKMf`Q8VC4Uq7TplsBLk#G6XiN_?z}E3mwD8^ahT&l!QXDD zw?v{xW;9zYtxaZarxm``rj&Paz4uDG!G<Cp?Io|{He=s?HDu&0eL`EFG@oW1i8}TP zGHwk+a02X_9vt^)?L=aPP1w=e&|TEBom^F^lK{nQqq9=L@fRtbp0DF#7TdKS_;`SL zOtVC2!dcr8uHQNG7DmdE(&Y_hPj4)hcoQ273w#b<)K-$BE0Y8gK{2ki%Jc*yp-!9* zWx%~Lb&;Hwnp>7Rt>Y<cM!Uoa$s5)FM0OpT=I4e2sye26L_m#|e=WdL#2|}jLdilg zjnEST9$neNa_V(#1}PwJqwG6jj>_eiTIP4l3{H91U2J5l3JGbw{5Ix>hH6D#Y_Tes zdSri2<$_5g68)Mn_ZD*0yZ|=A)$L)^Gj-PWLR6&NI@7AZV1MpUa?!QoQOny#cjbf` z>oRMURt;aG4fW;iou9f@-<?aQ_+H7!9WE42E|41yAh|2%jl}MXT0Zxc3}Fz*??(YM zaApO<Nq6A;O4T2SKX_G7g{A(*d;*Bfx*mwi)V<zy<8NH<!k|CeKYteUnO*vb>$6sn z*^?8Hxiu`U)%O8zC9rY58wyDOY9dH!ysK8UVRLm3!RE92K~y6oUp11y@|yv`_>S6Y znKRPe%p16<m=Ts3K=<CmK8#WftLOiq@))>bHkFl2aATo<LvLUb-ac!L#%@L@_Pt$^ zWsM@t&TL7%RLpQbeX2*!jQ_-z7}Cq@)Kemj`@`FkA_^DNVp(I1-!bDu%;`e1BgdgV zGY^<_FXA`vDmr_>UXa4bCorYcsi-AtS14>c%D&RM)y|GtthrnE7*;s1__;Mjly|VS zpyot2ubX?)x*8TZketF<%Gb1<v`6O8TXc|}bM-gj!f7b9%wj@!97)VCN)m2X%(sWu za77NsGz51E?;5y++QPVJE8fKIZA6arc`aHW#17*~V~}>&?~O02$aK;G+-=yT+Gxkc z8>~z``RGZ`f)I9LMBYo6e9NmZTF`J%LPc~KeN-NFw2ascei=b=(3sok0Y0nWtYOVO zafh*s1VMu#+F*PVn10P}@cJ~P=u9-03$kJVw5-WNxIK=eoULEaSY>r0`#-H?d<=Ze z?bqEcjiSBl5vF44W`K*yyadLMp+_XYPOcYyKmK$*nB(RRg5uWCCoRB)zmZRpXW%oW zAvFMp!|m5${ryKE1Ufr>OiSYDwNacNVi8eDNM_;UFM9G<FlS{e(^Sr1rx6ky`lq=j zKBy<T&2w`w7g!VlQyStpiuc>xV!J;+QyxzjSaGOewS{70<Ql7spR9L`IU|N3I1Amv zw!ZBdSE`l}=ytA?^+Y$Cf-|T3PJ$+bR|#q*#)p%pN*H`4Khv=i!W5I}bi;Y`ZcBcw z`w-Fw6D5$`Fn7+zft3tuhuE%Ja^CGb>go;H$tFU#hE%m|)4l<}KRbnCDrdJHQ?RSM z*A6_NXco3`QjU7w#mEKx+?&AG_%QD3SaBm08Lq#w7PIp)Guwt{HAp)=hz{2TX)$fQ z*U7D5v!#@0<LTP;es4oTR43|lS~iUh-|<-L+LL#pE*SnYjzH2?JadqpeMV%t6<kiq zu%!*1<%L}a2+J4VtYJS0_Qc@rtM*{~jv*tS_CI+Yzg_w?Bbq2W@!S5E+C(ap(wR{{ z(~A$?d${IJ`|7rjx&r>v9N5K={}me#7_dN3(LK=J7{9s>FYBrI^W88stQU^C>eT4K zz1A$XDtT$KKRG>Tudyzaza~c3JylPXWb$<95|ubt)oA(R8*<z>iG!7al!FgjoyswP zuAA<yTeMe%yWK@XN7hZ5vbC6vbC>MN)2{(xcZ;WJkeQ(~Yi{>a=CXerd991pG+(hh zPdCuI%cSJfQ9M^ofeJs8^s@9cd~xspQo~tib(Utlz3cC~5Un_zusKt~{h7uI__4$e zN4*XoZ?;9LBds3kZlxYcdF=-G^>)Ke?zS5Pn?A#G39!)vRF1m`>jXb&qe-<#p~@^x zgu~b0B~#~>*d84*K7wjTM6X4~QChR3s~Vb>J#vN`mPRXt^!wnL*b`(Q6?QML^uy5R zJWDifRX%b?dc>JrPPF-Q;K$)b*vkwzEL+QwJx;NHh=Dm5?QHW@wniaCm(0YmL-Uj8 z@_SkbbfExg{-A`ZT6noA&tS!Y@pag@7#nhxNV(z0e-JeUoo}|9qIQ@+{ToM+a?fK> z_s8#U&dlvWtK!wx%2cmlUT#{4TT)HPYso__fOKzG<_+sN-hvzQVY_;*_AY0kAvP7I z6jU+F86pL%J-Mc)6xUmbv#5Q@wz|^g5fAYya>bJazb<|KN$p|ou>_a+88zl8qbmz* zsa@8r<?wN4)LA2cb}!q0h3`4I^5d2D+@0uQ{x@n*!=@o_(G?D|EYytPCNrU!&*NYF zPEJe2{HJq7$s(jtJtM%0`?Ff6Q*NW@9om=Isc$4yQOZBv5PX-&Wby)rH8eUh#;q$} z5}mzBm+lB?qLC(Z(jmB%Y`OEgXH!8PFI-^V39Ngf$85i}hVMiD;;!h6f0uT`_HHR! z?U~K4tlu^kS|6EK%L^C#>?C5KBg(Lw$C;A7t`T8nv60~6T-@2(i_@q;EVNO$u{2S? zkY5Dls{18sgQLh$gY+hLa#6W^$KqF0UIqtXr_BYC5B&6_Wwbii{T$`w^v+ejqm|{o zJe|}r!CIn-=qNGqi@*saVoV8FM~|`$Ol>x*&uvDB?llE%Bvg^-s)$jGm<B4QMw+`h zVsOB!L9UE&-Dm58G%xQfc1TFlfm$jqL;&*PsYf7*>e{F@qbPVW-yTPl{BF;7t#5`V z1?^F)F&06WP7ybcj9PL6E6UsN%xdMIkT56BxuugMQ;j)&yezp?nPa7`m+AD`KUXVl z#Gu3JR8QA>KISjXTNH<#Oe)e(*R4KsI*lchX>*SRDun=_><GCv9O$!gN-bn#X_=wF zqd7=(S2{!&*HNxh`eytR?|SS_dpGd$N41|}ytp{mG)L{!`;p`3cTOPSgUn9Ob@!k= z-zfR3=GyG0dO;g01a|y6dIb9|hzgQ3Zeb$RcuFI-@?Mt0;JWXv=~BRPep%k9>v(gY z#;I~;t)OciP*F%qXFaE@)dqgX2(rjN94}jk_JTq8Ji)2%`4rbcbrX1nObxRSQV4r( z8NLzT#4sUwaq6VI93lqIN!(aQm@ZWb9@T&!)v`6_<_?iEXY?+XRpZ&#pqN}kBqE?+ zUQQ&`shury8@$!_&=;g?XPBT9yY`o$X(*v~SGSj=C=p>#20;aPl{DPcLmx2!mPPo3 z+%~C|>V@j|#eiDHiLIK`Ml~x16Vonv8_hiI>(WW*1fcDHzf;!<HXa8*tt(Z_xZ|xS z;kr3DbfR<YWgHd_u1`*@Po$>@MHQ<=`LtU17W_nKaEkV9!t7<;saFo*{P(P_#a0O_ zxwh9^tgnR~^1zCcn^&gMsstkxLymU|^0Wt`*}?QGf~$32SFC=DL8!WulQSw`0Hs94 z*T@$JqqfSaV{MPWfSLc`?my-?3=mR4u|EJ^%En_cqAaZ}cgJb}HMZlk2*FsK(*FdP zw?5Zf#>}_5b&x#Y<i-;yAe?QUc)plMaQD+?wfPSE0|juVUVp8p@*OaHWb=oxK&h^f zV@yalHUeJy=ut(iTk4d8nQf}fd~<Ff;{)y4*$wTIiavV;i|dG1l(6>D7zA)$@2{K_ z0~DQJtFQe(UV;R{FM<Yv7R~i<`3Q^ik55FXlEAd7VAZ^-!Ys{6)g9|=bT|qWlUl6s zjrt$OsaA_rt|ePhwi(ezj7^NQ!WOQlT2#>A*o1=6Cd@D8lQ7&g#2lRodec@kqWxJq zioYEPj$TY%+<}VTtN5N_uk`D_Dg#HSkO8gP#FZ`EobfPys<1duFJfy1<gfObk?ncV z10`~&lj=1XvbMIBFDw?<cKUsC^Bt=>%2RLH0u3|1+TmDTYF$iLScXiqB;dIv2PxWK zFzU=HkLy^LUMJqER<z*oFXt`5M(AUI6B^%hA^^@^FUN0I3D*8WvJTdSz6#UV8O0VR zyJ$z*A5esO&oJ+Li|Xvcl?S7(QrbO?w@+*x67cXAgc}t_M>~iWc3E$>LtLNn2=)}w zj)uLgcvSH!C^$n^|B9L*r+B5(FKIlX!dLlA$f?lYM<~wJ{qP!Hyt9$tgqAoQBcNn6 z@H173gg&x_lE-hJGoz#r=Cfr#!~I(Mmg~2|Vrh&CE#H(~Tmxq2aW7zLcS9bNRPvj! zGWE?M>GiV`xPATRCw1Sbe&4P8(|jJib`S(A%PI0Ufq>G%<(n4MI?p)HlYp<^?lT>{ zt|EmLW$F)=Jf%ujlS5R)VkMY-^4$ZW(1zV_zfHHE2IQ*FnK_>ETth1j!MrJ4O<2^4 zuwKeHY*jS0<>(bNnf>&f-H6rw<{cS9!y}Ag*UzH3a=};38FN<IF0jZLVu~`8Q;da? zX_<}m_o(ITYyAWGn?GHQ$lKc#^4>6XDs$j)=~#?Nbi2W&j0<aGEFE1n6riX8&R|}+ zyMYfB^@XgH9LF{2XSHN#rlnlbCuFkveT2k(t9s&StT<BnBBR<9#64{l=`@D9PUV5V zmj7+Y%>f_#u>Bp2KgfFJ@-g+pv(j;N_lPXkn{V<8&Sx4D2Q4s-1;<k6rp+tEvp`yD zGE><km=Nio;n%;cAiP`<wT-xC*!^V}(L5F6{zHtB#|w(pp*La?yL_IK%f6bimZkI& z!o3L<%ciG&#sHtk4rQM~hY$)Xj5<#KTBw7sjmTWVHw9ERw#!w+b3k>?qKvuVAY!nP zwvmG)H*X!c{<t27dd{eK@znk%oP=)V4r0~)1>*(GgZzWiW}meCWt~B%7-YDh^?J<0 zt>0E1SABh^^vB15hM*dINT}O0?*nW<RJuG3nj(0rT2c{?U`<m8{UgL>U@X6U1wm1~ zdW)HwEDzUPAr=?)<2cu;gpVz0(s?MA)LkK8xkPF#3OC4RW>}Si9OzpEgV@2=TiTq= zeRMtKwVl~OI}bhsjitTy6+1oj!OB(4gYQvWOwosos;6%UI#VNT(zM#BDBT3aElWrn zeoCxcvT?g}Ho$U)gzUF8ENS{%+Sq}3!o;^<+nq_9K2+T1N)mD7Alo=7d7Pf6<DLCm zi->wf{cizq>GeyfPugR16)daT_ektRWV;0xoo?3S1MTl|md^{&<vysaM3=u?<GL4q zZuw@~k;tGL$K3#nhW!ZJyi48LQQct{%>a3k;fJx9w~0e0%`nx-*X#1}FCpbDHYk-u zh}OKl3c#l1CN#qM5)Fn5id~qCq3`EEcb53{6ITMiseXAkE_wVu>Ku&|P%oVsQ9|<$ z2DqOYU-G3-_x~{Vl~HwONxQ)vf;)sD!6mpua3{Ds!QI{6o!|t5I~?5I-QC^YxhFl{ zJ=4?E-@5!_!F$fTrS?-*PsuJ}e<ZqPPe_O?IAa_f)b?IzfWxLQvoem9`3qabWacA` z%7h3D&*v&khDaz@Y~95PkY;XUX&rggyV(B4P%}aJbT829EY22R@VR?|*FUwdXeJEz zcMiH#Gk8t)ByJePf{s{3@)C8`dJ49Hq9uLS(I{`nOZ%&=dP6g%02`VWDwhBQGl5P7 zav487qS0F89l12M^Ai;b{}og=eg<7oPufI((9HqriT{kL{L<S8Fu?DqY#dmzfx#(G z-Jp?HLGbIyGQp7|rI>cS>H25>ADCGA>DT5SqKR}1o3S=6ee&6IZ+J*Ir94tG#JY<L zoz`N#?rPZlcv5ykOIxOh*NE2BBR4cLfKD4BULAU%RHeo%0;ZvnM!7Jk_?IL2AEV^6 zLov4DSJUU(@EXm?mpTHbOttt=4fRxcy#*5&De0$5-*4T3Q4Rk*Z#Q)Z(4tCrwAkFu z2g5EEtJ##%;|EQRW+ga5bAUT5&Xj`8rj7w`H(g_f%s{XfB=7|VApd@DO{=MX)1m*h zHb7)3@ZlT6xwc5uRR11lCG%;k56p-XC%HDYN%2_r0Y`t4wnVEAY{*Nduvq}CY-V&~ z0EU@62r}&>lF(ly;y+?BOwqtd1+%xbW*tG_=!^~&rzQfVkwtOG_@Xr3{}G!SuK<;* zfwk)Tn1EmUZYasBV`c7|N?ox>?lxPnWGN={*uBtV!+?*^nOqO*&IdO%`dZ>+Y_+tB z8xA9)-a-{q&DjJ<{_U|yb_Zr9@z2_7lV#MVIo+w4>hlAvkoi;oJizyRhaHUVhQN=f zgU|(GDrpT!M_#<aTkJ*v9!Ml@K|!Hx<>2zxu?B_bW_QfA#%lq|b?g!~08}CSD3+&w ziHYP={+8{sMd9bf4k%(_#CVs#LF=FUI>HYOE(Mj@n!&eFVx=a2>PoI8ERM7n)=-&o z2u3Q?^q+yc!F@RBN?I44B*0^Lg{)hdh6=hf{IKG#p-&K*yOT{`!Yw{kWdpk)ddU86 z0;EVqkk-*cN}zGsh>n{Vf&mzt!#vh_<T>U@p#fQ$r57Ks7d@mLOK2RNEiE8N#Z9CW zur0^Z;H8JB-~Gr=jQca+OQ`CC+CbD~*T_Rgu~8MEkDK&?h(bmkaepQOL|@7m=^rH2 zm;jVoW@FreHh{?bL)qbO*^RUo0I{!l{!HVLL%zM9v=I?Iv}s$|lT~{LhM>sH6B5Of zGYpwEvZuh(RTb9Hi_Nmp25qymiTJY$ovO`!G)1mPTI80iz49%0fW5p)__Y8Eed6v) z1LJZil3omueLqL)k*Zq_=3V8%VAy!nV00qW%MX1MhhNNc{0tH9$KljjP#pLpP7$_p z(epk@DAC&{fBvg$yXwhrcKUy5yMJv76Pyp^7!!{>33^lPEoXQWFaQehASQ17A)q1( zc%jYn`|4YlUXB+U?Kc#ymJ+lx6l=W)JY)g{tHnmot|Hxf{)K7&M*V!t_KG0i=*HR$ z{hrpDtij8mFF{DQupnjbogt04?DtQDm`$+z*;9P&`~*m@*fc{`lhr{Gjp8>$-xJ41 z6;bad2Jd70cIFhvls94lYel<>@=A2HuW?)uE~UM~G}-qmPwqVSnY+0V-(zrM=r<(~ zDv7dG&!&aX*E+(~rhd*PE^kB>_@Xi7xQX&Np!<E|<J7-}c<t_Z%te>gi08KsY7H?d zmsD>;LJ}WaxSJGf8>D9o^^m>k%5X447s8@ClWW0_CW+tYSkdDXV~Tthuh~h<Wxh9x zH38hjT{J7<bi(YF&Y2F;>ZWf%!B7{M=mB<vi`<UG`$0VCZ`>dw@nZl)MmCZ{v81BQ zVz7jSZ!h78FlXn|JA<NZ*;Weg9ADBx<Z>F${q5~9`D;jZN=}M+v8qXR%so#c5#g~? z8SW+jH<tK(BOPz&*XQn1fpC6VOlH)ix??fIlEGpvx<t$a<V$BxfKD_yo|VsKRl1;* zf(Gg+ICCiF$Low0&Fs^6H+G6pjH+%{(`*=^$~wl*{Um4ZcfHa<&kz_Vt{|N?N)xg* z<$zbEe0LJ0g`939>D$YWZPL#pmupDq<{xCnuZMZ^fz<k{Fd6-rWT78DWjS7kO8%>^ zUx~GCV0Mk5?;?b#$~Eq323tjymWIgxo&gA~MlUD=4%6t^oE4&ST=@Dezw1F@QJK$T zSyj@Nq`vY@IqjSi0Xw*x+-yI>;pdq1>YM&n<Fhrns)#XS^KQZL6Jz}Tk)DngsIP}O zxe#t}rB2mqr{gW9;1P^A=6$qmg^j4Mp6BN1zPEB{?4LfNhBW<@4y=*Zytc)IR91v~ zpq@m+194qcGt4_;t*-8_1aKF-TK6f|Nee;&^RRhI;~JT$u!j)s<33|3uAP?`#nfl~ zRR#ZvX=uicpb5D@SXSap>c2zjE-Q8<5YT=QYy(+|(PmxuSBygC+=g^%JMBvm1nTSA z)Q=56V;9U;mcMXzs4JlvrcIinLpe<Qb>aKY)I+#F!1)n=1ig0Oan!ZASph1ZJptE3 zSiq*nu9XTv25B-V8FbjSkCGsL89A%<i|qI5>_>!$$P#rw{Y8V~`Jm!|ih}zLNoVmh zMx_48VE?AZqh6;dN*xF_mV`UE0O_%Ga~kBmHupikD*BDpjRzEzRKAWJ^E<n1oAgAC z?0%{5ORi+VH16oUpVuf)a2av0j-bfF5+(e~B8lFViQ-i~X&Llvb&y}@bi~#5hYB-m z^4zq_RnQ;EsLtTHBuy@FV?!hY3s5}Lu`X#2N^7Uc1yNLmBa7_)0uOc!NQR!LT&~cL zBZ+p=A8sze6AS5^&cA~?5Un4OW_~5quCK2t3)^6Wgh<Kdv$;?;Fv{fDvUDunJv-b5 zaYB+|<7}EX^*a4!e*(?C_;|FO54=c10gDzjsl*^--+$jT9~@{VmGODw=^@I3hC)NX z?7_jtMaL`lX+!O+<Var5E-5MbJj;UI_cnEv6Z4F2E#S(AO~nuHzSb|L3@W#6f_MXT z{CNiOM<dwjknnsO$%V^)>g0`&gxv(CmNqc35<_1a(p?|*rZ^VH1~BnSluFc!+hq$a z(;%7E@=wzN4!c2GbCTemY7qz&mvlb))qs>f)6}42#)}l9@z*#tQ$n-+%?X@YA;w|- z_5qFqjj`O<yGy20$)56q7kZKcly^g;zX(P=WN2t(BmpzbG#&>OA)x>^uhwI`v;Nm( z0iPlMEJKM;^^J`kbLBJ|>P<w9SI!aALreKIY6EQo1>V(PeaA3q6`Cu}BOncoq&|?> zgOR4x^SLf=ka<F;d3wOs7*C);K6EY$v^KL~sS$sAGVmaX)Cb1iep7(f*uQy4tz-IE zvzqf1tVN7ak&5m0%422wgy^ap<2N&Pojy1^Jq?>0Xobnq;QpH29su|)SMKA6kK5=w z0~_0cjs&gNW)<b&hc_i`Y!FO7JYd2<vMLG&Or2d_x|Z@0<f8JJ7VOr$DP-qBiLy6P z>e*;O*71S@9Ubg~QiE8fXXwX|$Ja-iQT=+Q-VlJvBU5dG2o8{*m!yumo{<K4M_xVi zvVA+hg_wWYUB+slHv~%0z$`*Xv33<eyW=Z;V*GlIL08ebb)^9wQphRrj^OgvL@=iF zFI1T2Ri9B1Y>t)6&aX?DqUwnvNA-WBZQV(9N<N1#0KJ6IK8%;&jft`QA|WGIxSLHi zCx09l{1$>AZ-%(sp<|v-S^x%$&+b6Re#Z*gbiNkIh^tFDYLKmZWP$P7G3C@T3K*R7 z=L1X&_416B^@EMb$3u1ecXhwk*%%r`u}dW(R|jaW4}wn{{UEz(QAE&JjP;H9Z4*rC zvOJ?WIFIh63SY>V<NlECA<g#2Xp8hrGuOtVTR8huv~Mt2#dE+~s2T^;h2_3yjX94Q zhJP^&kRY@4mhj3xcE*yRfCtHnKpXu=-!wR2-UAxMy;}{17W3%uM`rvS^ah6s7@03( z=j2poW4z+9BfP41X}Lk%b=c+EHM(+j^C+#e`Qw27FPJ}un>HJ-7f`8iyq=!6k5A(? z`W~XDbfmy|$G|)1pgqLj@OQKO-$9QV*l9B<eW2^4MggnX<vh|TXb+DxLuOCDQ=kM~ zq(WH_1>oi_uDk#HBY(Hlg1`@%jZ3~+&+MyEYd0WLT9<eZj|f<L=GUs$Jsz5Qz|;OA z-~VGMd^14#VY-c9Ki@to?;TxvqUP66&|NMzs)MaADsM$c>_-3l8~=1XWefRQpdv;@ zN~uN7SCw8HS|1a6f%m5Vew_Zt>wp*jwX8yq-O9TEqYD8P$Ouhu{;gim05;w~-}JZd zLLeX4*`dHWPje>!^-xj(Un)!MxwVtPP$G@Z>$A%?Cferp0VAe^B~6;9u5}{OXcDe) z>lZF%75^(!Hwog<Fp2CNW}2j<ktAS-D-HM;Uq9=<8VW3~0zy~&v%_&{x7*XcUMcnS zGxoI)%rq<69`l8s;GjznTfO%-!H1|mwCNmd_R(MiL6r_}D6N{&-oQjZ#FH-Rq^Nhi zS4m@l?O&M`zt%1#hc66nGG!~|Dvg5ACy#X8s9h9IRJOXXc2v`Kecgu}QEIKH2SPWS z2xVDuLUZXGHAP2EP9e%PP3_A1Iu&Lc)%*3?ACs#Os(rHi6ie!uL^moJ@M>#cRCxm6 ztbMda--TAo#b^(EnJd8dCjd|i28@V-z&AOCICsdF<eG2v#)Ud%mFfC*>i&7i#d+uE zj@C51$Wf`*ToI1Te(j{5;x(m+ldWQ>4bHPW=<R;eCj)RasQ@fo<Xq})Q<lFUzT%#` zUlU0q`j^?cE(c3LbuV{#HUDdNgD?dpdUow95W$R$)v|K7h@+#W@87SNL&VP9`VCO1 z-P$y)6ng;uCdDBRPdp>T2@dy?Rsr*`DFKrCb|@r?1_m=Thi*bi7&KW{Ux_QeuG^yQ z`x<FCKoQaApJ(>1b#oNUAKAP}a&b*(mGYVZ%vbt~+<u0IVYnYyf3x`smaJ5iRXr$| zf#ucIVi{qrjz!@53<-{1b@5W1sXHY!Y`yGpryITTPRreEM<`sbTGPl2czMP5o?Brh zexrn+?sF|(wR5F57Dqe;R$Lqe9;6bh5Iw;DCg4<UF0h#-8b3^m9=PbSf?8b&GE2(e z*bBzq!hM9@FONm6kIl3_q@Rc9-T)iOe=rXPULX&C2ES7i)SwnGi^e`1)jdXlc56rJ zG6NONFWkj^m1^amqZQl2Ofbbssi{G9(aBlf3NF+dUEAkYdVp=TS??thE8|LTuG>I; zKzLj$8Uo;&!F+9c(6cY?Af8{EsrmP3UpvmP90S7Y+G<yt01nQJj?S$=+DS3%4?HC^ zneg<>Li)PA2+SGNBf)NN;`%^lhgi~&o1IS&MDU@pFo{j3VBKHK=`Jz}>pl#P5~37T zVVDk+&GfE?r}iQW9``fUAGPu@C%eM)yzWZ4zjdr6dO#vzM^2!Y`DsK#njo!t1?huq z)zI9JHFnL6iX{MK^;rUDw=$e64Lh!q@s^O%IUtF0^`Qh2a3Vjgt_E9aV|$rM6S>oq z^+j}y%yC*jXUpgbOyamd%}oF@iXVq%lI=OaO|K*D^&9axJYkTCFMO}c)a)<SOO(1~ z)b44pX8yzHk&*y{L!Ltmn>vu3{ET^iaWOl3F1R}Y`@Ne|LZwuBWLd`T>sC;r@y4b< zhP1Tx=ND0w%2_oIKwx!o80md+d3?&?-l*9^UzPXfq+Qh_b}P}z`pzWE!^1;=Le0*s zB@guNU9<(w`sOh>E1eg(bAz~SBRJ{LMN#)Gz8o}XeeqmFEo7TG>Y+#9&~=QVH-&1g zS*3Xo*&E5Tc%0r_bpoEZEo%+8y(^W=C7kD-izj5%o6mA5$GN=<`m71J*Z#DLg^mPr zaqqN<;0Yk_8*r(gtR7tuW@(<#p{UIct@w*|xdwcUbuV;>UH3hocNPGOkgp|1E;-?a zy_VCZPjSgFD)7zQ+?n;~XZv^zDo-w#+Gbglj3QDgA*_K~zKs2GDNnX4L6l7C_Aw7b zQABOCsN4o6QDg2LJoOKr%dYu1V_!x9MIkJJF?Yqij+;LKo{7dsNy@A-?N@1%&yae% zMjsnp`Q=+bb2;l>ak?<vy+mMz6lZFHzN%23;*Kds()N~uFbrt9anKL>gZ`MeK*vi{ zD0BjzI2-r}(nrgHR(|N~^FWC(Q0{i>V!NtIlC0t2N|;2M2)T7n_@j-GO#$4BAN7P} z*ZcVpT^VX#7jZ39J~+J%OKsg$wk1Y;<?Z$s1QO$RoxnoWJ+bAb91UO_`1tAhYAv@p zQz?DTX6%Xdm9OshxU<;A^o@4l83gZ|Z<}0Nb)6sm`jF?kv^(7<!y(wJUX+(>+r7`U z!+kPCF@t(_c4B!{ZtC1()`&qpmVOfLwzjQr-H(c@owE?U_Bnb~6chAD$-uWy7{0Z) zSXs1ZYI)#pK|-eXj`YrE?rv@X>+L-7otsT;nVG}G?zRRP>hwFrrgT~m9xL(ip8TR- zPx!_prgj$E5QAXNKGC^j*YT(;!5VZl$$(hwZH$P;x^O8}UCvnvK5MV_pl&^OtVw9{ zQ|QLj;u4C!_c+yB|Jfix-|#{~lIN^9|KTMpa=8XUrFlT;v-6Xftnx{05MZs@r%6jY zWWLQaG%DXdhT(F9@HpQEV=qttB;RViRfe@_b!X9eSc^JyBhqv6adWv#<@r&;UDp{* zo0nT-scnkcO6P7I%-QyyTjleq81>T?mi^`3^y8j*;C%_hoxaa)5!`C>6B*g{*Pm<# zYM*Co<BNFx7HM|~@E(S9?$Ks{AhfRD6aQfLw)IoEs5)@pEoK8L`y;fMH2?!Fs2R}w zES(IBR-XnTK6V76<iOZGnBvk><fQTB2x3$vz3#jVW033_naQI*Voj6%0@f8DyF4iN zHaa0OC<|UY=k?4z9r2VlzSt9A>rO=I0*|G<H8_%_rt?hIb-g}sMX0;$n$SksQED=E zXrML$mQ#dpG!`Y{y*jpq0kjO!#FHX(F0P*G7>MdG*Fbhb@UfSc@8o#;xOv`Dra9U7 zX*AzIA*<So*gpdjDY&m^r|N|v!hiLEcmhd_J<$d0TW_m-7|>=wGgbXrWcPYNjmHh? zeDB)DI8OK}4$hzy1Q{$)r8mV!(O-jm{W=n%{$?i1rUe*ds)s_%%LdkP6;$4Blv3y> zzRu8GpRDAh&}Qtry)bdT?mZ!|sW;*jA6Nn!9xF>Zw0(P`YY&UWtnH*#+vm9=AjYH< zO13uYZ*C&?FZM;8?Zm^u!+oX&o?<9X`d>R<&Mz@DKPov|UtD{Kc`4KAq&0eA%Grf& zpyT<eljt1BI%28c<rYh0vFs2O_@Zh-9?-qP#dzDCW@|1wVwQghcR4m}{z`Pv1oy4n zMGFpVZ%{=nNV=n^7>ul~@qL{WlPHUAhRL9?!A{}M)5X=&g<8AH2(#*8yYqm?n#Kj~ zbb_bJsYr4OXfI0cYNOboP`y20O>>!;-L{=7^_~OD%XSO!uEb@oA-CqnEw;~&4*J%d ze;_wnLC7z5ctTJ6m&JBNEh{TqfS(-!E=*~*d$a4Aa8u~H;u4=mBV73GVHI+auQ0<_ z#xH-@FJGD~_^L2}Ew7;Bz1-=ZMiYa3IKZn(i-xZHIr3)JJ(o`(b>CM#&@oyJzZa8E zy#0Q~MKrO!^eyW`?cI@{QL$;vqLtjxeccp_xrWcDt(~#A90eHfjr)%f&@ms<T9jVq zZ*MINEOhIoLF@Q%T0OKudoy2xiBVK;ju`Q57SSw)H3$jW-<X;?4v3bY?yu@SpQBKU zz!#k}IxXb2y1pK0yW4}6W!`eNO4Fv+**rLj#`5-C!$h=~E*@eD-bGhlMy)?>FWGGI z0o?ga6?sV<;@|uyWXgSDac^6SQjRyO1QfJ~=1GB9*WJE5ZP7Gcl|F<C0q0m&u-NKd z>dnzt|DF!6StnL&JyQyo(bSrG7z{Xr?+b>h)9*gtP;nQ{ZTWa{u1$)%fF>rRc8@2# z?2aVN)hdS+n^(#|#vpIAwJnzHCF-HB=N4b$p#AY%C->s52nIo7L<W}KQyfdWE7v1= z-~(90(q49ExWkD*u;I#x=K-p}XNJUkM&|&>l5N7wqy{mI3%Xun!CK1~+Z2|$%FlCe z$1x4!>-VEgMtYKjH$5Y|T<p#$md#fL6sT8_a)zeNCXw#6)pFBG@>Dh_>l^#5&hW0{ z2T^;NMJ`KLk>M6z;FkZc(O^c^aS2Uvsbu}7eO%Gm91h`|xwtpx@(@PTsS-G6iqAt< zL7$gQk7kO8sTCD3Kc<L7mZ_$x@Fp*_$uKKwt6HW8HOHy4nGy62fO%flZGW~X{SG$t zNW~+Ug$x_sgzrJRpX)2hh%wd$uX;J>tBh|wl%rX(t%<%|xVHOaknT8|x9ROl<er@t z+HHI6`poUwvA#yr=M!XM^8(qjutpH{Nk;l>{Y`ZpVBHeep%uE<!6`^ru_{VwmyC`b z1}1Up8D%c&Ve*<%*fyltR3!Rb9pi`aBeBwH+?~TEpTGE?eIZ=SK`8BF{W)|O6kMA> z|153<Bn=$4PdXrVKLNkI=)$bFwO-}NQj%00yb3xs4Q4+LXG=#iQD)S%zkQ9OwxxkZ z#~KgK635jPP4fAn^#IOoT*4tNhQ5riMXzm<!IIJn3G~a0NaBb@#tWp3q$})eMlI^$ zyiyxciSnJ;e3cD~hQ>7WI4>iShn{!#F3~jCcJ+!9x+3VuJoiu<`R(gz=Bvr4yNHwQ zgu(umrU`|jFE<*&0JpO~8jm7R%pxsjiv|~8D!bR)t|#P)*<kcDu8{9ztC9QsW<@V> zeklZDFMpDA?LEN4%0IGYeK;_H6+nWfi#8V1jv%kW;ro$gnj9u$l*Z<kAg^k<L(f&O znNsMVmRnxN8v$_QcGX`$vNm5OrP>tENtbf36HY@~+WK*`z%rrx7+EFp#>|l&`AXw+ z8uRXgueI9LX<wThFDQGAtA;T<W`?jD(jM+Q9*&(G^^17Cx+)H4AMOHs_?)fKs}qDq z(Ldn4$G|ChN$y%l7AT98)bSt?rXbQ?;XvB=V|yqpBXzHq8X`Dd-m}zT%?-mVey5J9 z4)H01RBrPlRKCn!za2&-UP9)!AL5$e**ekFTvp@5!je{|{t{=I{*uz>xEPFiBe?G6 z4f<WMhap+UeHWPXCV~6|dJeK%NmKgpTy;q4+;_TW6U~bO`=e6yBvO$2k_EaASq*Ad z{Si(Y?W)}&8ID+_pZclsodrecnOqSX4+F7oPzFEog{z*;Ia6{R=tcnuub6YWyTRe> z{d=GF2C4Bh&e4AOa+8jq1i{Gg2DDb4Ew)@LLpi1p>n|;&|2vhLF9HHs?#_Vi^ovrE zZ3A6kZxCg7N2ByENqNX~+V!pivinC{5kRt<IWsts$lQwc=vk+D2Tq8M<|ZUvV;Nvx zigEcU_>ERvjjd~~4S(^ej^7N1*vK%=pJHNumA^g$G=fhj_L;rW-~$F>FY)ExMmi1e z7!if?rXl{*#?Mns){$;0^&)IO(>f&=_@{-4nJCSX>?4mW<L)9`WI5MO2nzi7W)Cie zP$ViPgW}I(o&kmL)3u{o_szeL88<&JPC0Lcz(SbQa8tt#Wa_F1hR^7K$U};!o0KLT zx2kX&E(m&*{<M*2f~4yC`GdpURkh9dXYZGQ(xtd#r~2U18}?fv$)T#F#S_YcBj0_I z8`ri(gneH(KrC&0w$y;Da8<lh#cRoMGRyq6p$~XQv3ad(SM7?%xYP1}l_G0|NPd-a z;!lk!1pVq<f`?#+I7wbAk_DAR>?QD@GA2*ky9Xc}7GA7(342zPq{M2mKjW_3YrHZ& z)o8xGe9#<c)fkhj#j8mPIOI+0QF-oMarR759M?#Wj}H?MshcVyTpAr+{&%K<DG{)y z1i-gN`^!i_ROf>TR$NzMc4B3fc&FMz8|&JQ-z5NOuXIg!Tt4LGnStdjCHaKkckN#! z?v{`y)>X;)NOoih*fmcyuU;)?!^Q5an&`B*Ylh34L@$z%KKwe*9)3fvk=IU&9Q<-b zQrc=Vdkav}WAsZ<se1V;*!`$G4;B|q4zj&iOV^qf#F;;^*0E$VVN1-s{6LWLmPeI= zm7kP<FKD{O!$~_ls^yK-#8Af{6`uHm(3T3o-KL)~BFL&UlfgMIWQr(q$LpONz7dQV z-md<3g05s*O~X!i8eYE5Thh_nKU$MZ&)#QoOCFx_S2X$|Rv;}^_(u^k`cq1pXn z(&XmTeQOO=j`zHtw?-t-u6Vivfgd=gq4K4Z?@C@D+OE91BjWI&wUl`k0tifxdzTZr zo+wVkQv=A~u-xmu5zLmY`6}crS6EKdH^ih3b6J3$KAwSg-!;H|jmu3NS`L7Iuz?HR z8NxUj!phwWGgAI>wz+5KkdjWu9c9(>NOX~#X~5ZMtZ%OVbM;D|W~<cAgD5s%@Q3{J zmCCIcZ;2>KZ+P6h>ftkk4De8?g%KhFBfVR7xi4Eo5%oJK3|O~^u)_rPpaYlEsq|u& zoB2h?V{X`HR=5!`_{0~ln^6_cFS_<P9FSjkdATd_W?p@XGhMb%dY6MT1lJnq^d;SN zE{+&U<(m*P<hAt{Od?(wt&9%;loUQxbYLPbYKwIm3y|n^2Oq}n+hwFpOr66QXNR@x zt&DO)cq^XelEvpP)~R%8ua6Aep+_rF_JILYlx0COYojPmH|1IXv}WZWiXH@y7e$Hc zGm?J7m7gcJZa&KD2<&1u!;iGS#2eM;i+PrW>m=2wd1l!?Qtlpdz+E)jt)O32iE9^j zL^DB8ff-ZpY#CtU+2Y=$r1VkLGWc!{^VJV5KjQpo8Ix5iA=MWG$NF&H%K1nU{oSf} z_laC%=el187*y67dS3Rm|JReAygM&}##;{Dqg6^crZ&erqxtl&@5W@APnp%)+0OP^ z-0vlO9u$(ZOoVzvsNI{!Cl32!NhuA5@Jvi4nH_<5QP^y)C&eTuLSeokVlh$>Oy!5i zc$dIcAzHK(>l4i6wXH7{-j9*F(1}xo>BjX+w5zmIl1ukJ8<FvgrB?eCb1Ns9n$3jn zfMKCyM!Ds~Cvj}`h-%%CT9(S{yTkOh)(bYNYUa;h*b02%)=6u&1hITk5bq)>iGIK| zyCQUy;=LAi*0N-)lc;hfy)PXVfdh6H^`=|vQMWz*j=I`CTcYPIw*Hj3w1vDf;ax(V z%7LxKSmIv!IMy6C?WRSp4GSW(!iD}4DAS{j@_atI%pP2O6?K_+{=Hno-|P<sYdi=_ zv(0-wQb%t(xYyADWL=XQ_%cC=NtbF11~LryXt}M=y`^UQ!gw%&KQ$LO!mh%*LQy>+ z<kp4Yrk4xQu&3Vkg}jT?i$4PF#&gs$pQaM&{A%jIyO(JmUn@9BeUgqn-sZ3zyp>~7 z>{BWFA;Bi|ImWBJ>AO9_dp}y+JMOImW&pw6mCP#NUgRxf8-99aks+5yuY9Cq`PVkr zCv#x;p46Xsu}R>}@^_;hu{Q})X7s&LarRae3N0kM`iA&;YOJpbF3w(Z?|1u<`s6J{ zh_$o@Pxb!j1ps>C)~aoy&e#weqxUl{pfSWS@mf=Pz;Q`Y)1d6}Mnmot(xXTliG7X0 zR->=C3V-3bh%D3PX3W;Dl!y1~<~F0!>C2xrzI<`Wz^ae|4NB>F&=GYYhqUYQ;&Pi* z&SwXWQ(SzvJQby+Ui71}xTFffsO8quVZ{{iA_o;=#(V&|e6aC!R=+QYPT6%2^zSvT zs$20uy<Nxa!7$!T=)ZBKY-?Kmz+|-sQl;96XWzq`#Fzi<gyIHTVqz{6wDbo-n8k+= zU7%C0bas7P7)(IoMX9a1TX~R7IV8LCPU2)~UP5jfL_P=q3xt~@{$?tfwqF4cGNKVd z@vxr56eqxHi%!hu;6af+Y<c-b)O-^}pLAgC`KK4J4dfpzjz-!$#t6F6C^w)$Ads|6 zZbAC)>XMJQgN~3+)%&QLxwC!4nM*F2IB74L5Oxy7x$#diZO9ak7##RLS&)_yWEJ}C zb5j-<{y)?2n8fW_7d`tnzqxXMy!v;-`}e6P{i<3{T7cL&`0L;Q^XYmlVEr4NaMaVU z-a5b1krJhu_eaXm>YpJ@uf7{k_`V(5KD0492Ld^I57_fNV~;~y^M8%1eTTUH?XuCm zjC9N&7*^^AH3}Q+oS7*_9*dUG;p>7c81sKLXIl&>6~@v~<6KOC`E6KPC?Gd(3NJi{ z|IQZs&-&{_(GQ=Vm=^B=r8yI9@DQ{}AUEG-GZ948bv+TeTAJ}sMm>KUzHEs1BX^N! zX4StFEZu;K`Abt9zfr#qB+*H!etH81%^#WYPERumMlyHiCKP^sW1+X}S@D?nBlYj& z3x$P~)noc$!S`N6t;xqr@1y|l3HYTWntU&<3T>=2XEJ{uFR+&PTRGTorQX0=eGkZR zL4Mt@`pJVrBm9PU&QCS`X0IVHLhQDCc{Ew#as}{PDd6FyWAig3_?85#^lZ2#%69*X zX&pls#G+AlC4=a5ivFoeyoIt73`5}lHP16pN>QaHKGhbPNT@76RAgBYBuMU*{+>;y zKVayW9m(fIMgg=_q%Ny$CjWDLfBWB`VY5RK;Hi7G*(vzjBR+3NXFTgUjib#MZzcRW zz0gEN46)Hz4n)!KTd=UhDSFZTY(nt!OSZfm<D^&&TPTj^UrB1;tg1+*9jnEfQ|D|3 z8=dHrap<umF`>a?Wd#a85D_sI5evIy+MOvlz>pAKG`Wv2H{7M!jlD=WT@5#k{o#1( z2*T1EbA2RaaZLF=*7)am|2(qo>(Ibw*vicRt2g7n(<8FlgVA)xodtQ!kD30YJN`O* z;39PZ5y;tEV}bIYA3>&||3>eoDenf+7nvzsTxvnjI#p=Fn?v_)BrULF{33nU2IL<< z4FYRVM_<6dWx;uulbv1pREo5jG6W0`Y<B?s>LFO9{ywwcXP|Qp18+J$k4>+Y#*E~4 zhu@i^C;_EteKoscZ8_9%!oTtXH~&8-``2IT2f-nkjgM=@#X_fid%wmYj|%xY9xcyi zS$HG-b0`q22UGxs>2$x&WXRHDV$rQxgn%>2iF$46CGzB2DdXrb;`P^oX94qhVQ`cc zlpGdPRT=Xf8q(nYkEeBv!2?N+mo+W7^gl@>8JFjLY!RVGDu}V9)HJgQ)SP{IfUuYE zjGR@%)q(b2i<oSl4OhbY$t}RTw5|8g_~%D#ozt22a;WLl5Mk9C(|a?2vxpI2P+(&W z+S=$To&RD!xE98HR_sf}0j90`eb(fK4<l`@8m5U3tK$9&H)fMH8bc-MpCzvSUxQX| zf+KTM_9Ra<^Yc&~udjWleu-)$GwFOb!T|}kHqX%EYUT*gl>Xa+`g6eoWpzY-d&`?E z#g=~hXziEr5{z2h<V$N0X)V_g{<Hz!JV#<0d-0<jU!NUTR;L>p3my{7!(&7L9Gk)8 zo5)|5)kj^RUAeW`8m9%+!oniQX-{|}1F*ED8`n3;Gn^w9MDkoXDSf=+1nrmZsRtCD zMV9GWNF&qk+mpVF1h;Rui_bqh1=!q+U>!zc&r~|w^3LgpQ<4I(;*1nI#d$fsyY@`S zr_GaU`d>wp8BmEdg~G}e*T-Aq$P3K<rG!5~h2utDFI2~1U>}=S6duAO63}%1ET*ZD z(4nON01bmZ6~oTnsTjzJmeqa^a}Vqr^ldo0`2=xI+IHxFEDyX2AHwG=*87b`fZ3zN zH01K+jUry`zL2<F8U_<V=r!u7g(jgy-{PxE%}1P|I?Q&Z??IZRA@yceoGMF|_NSwD zm3O{-<*XLQt+Iion5*qIie{Dca+(Tbk;gPvTh@k6O^u}ppYDzwmhbKz$`9bRyiU-{ z8YMcFGy`7L>#|G-xK&UH;muwxn<*O-R^rU7vmuz99CNtL4EJhqPJPpW`YDi)FM|GC zyEAlF1I*tfv7P+AEQ7z<+;JDnrZaGh@d-Z}rGx_RsBhGEG)@5(NKPEJQ$dt3)OtY^ zTu@YUl$Gpt$Pi7px`X<~$TLDzrI>g1puNZ@j1{Jiu;UsZL00|9jg9*bN}pnG=WVgz z?ixUQ<MwDeHRV5bSkiP5P$*BXFIY1on$Lpu2HJcpQA-FQ*XOQsI_cS{!q`5EY3(i- z*4#iZot&?yn0MZ@$12n8FoD4$q&4g6$;qEciHFdVADc?JLc>x6Nr?6RTP6H28)3iL zJCu|ONj@d?xAW~98SKD_;P?o8m-xtQ6i1C3Qb*UBd$nXeVcBzQVRXKeV3LkQnW%tj z`dQdW=fxe!`wuW-J8>KHtw=n5C4P%UpVgOZ)v`2SHZO_yu1x6_ZAiESY$Wg!1W;Kn ztNC-CM8Xua(At0(^%2|csHv<uqZ;9suxWlgF`;=^TdXXrs2r92Uhy0I`GCzdb|WGB zNkK0z@RUSDBzHMS`z&z%4TyE(Vc?CZbya+O&6HfC=pA`7`lQnwM{kc<D65WcA7@1A z0&bYPwkq`$skIi}L29ey=apjuA+#QXj%+H`WF%tfnac#i;^?_&F`Gy&go<A%L-BXW zdPUrla8aR#>T}pm`h_MGlwcCju`BehB+46xicZ8W^_c<Uv0EW=ePD))6*EeDN1W;g zg-E>tGJI;iOw_v4`XeD;<n$$=UN2PEOBP*=0U%cHBzGR|oFk$Cbuxs%ziV5HXogbY z{?^5WATeIwwA{;jC91!-#Ym<{l-R4N<Mb7GRQwTY=nLT<Wy4LtSGbQ)9}Ln4)t7Z` zA(B^I!ok9Or2{&92ThJg!Fve=#t~wnm+U_G_a39r-^OVa5{XG%B(p2i`a2Qy7%E{m zu0|He-rZ+iU#zPz5_np}G?#VkExh0R3-0`hWxw<>+i$p-?&;LAeL9FVciGK}xoJM3 zr7&Mq;Sgm+7fz3@s8fdzA*3ie7JRO<5Dwe7ulE59weKM-w-H})CN}TmWc4ySnnKaU zpzG@tpB)x_>}029aC?rn+F?RcJ%AF^1>h(s2$kJ@PEsOAX0~8zAa-IVrAW@^fbujZ zx)Mkk&jMoNL$I<z{kPr!SGosK&(nejk{CY^;-=O}AzgoQ(!6e35*oE|P+t!$3GOaJ zpKvyDQw^Gh4~~CFYg$^s;R?NnI`?q`xuC_~w0xUTsmCMcJvRZ-h+LUk0;lh0{7eN~ znVhoccJ#L7t??{c2>o{UpjLoprs8@=VN7HK`h4R9xPs$8#THp-;~q8bmsdK~kfzN( zeMuc~x!o~g{<6W2W1P+733{V{URQeztk|qdan4GoH6nP&M_@^eli8B&QsdVtW)Xo5 z8x4T*a4t1<q$jT1&^2D~=!sMsBWeu$XISir`IkjpHWS3a>}+I31lNYjkg1p-paa;9 z$Ts%n;2t|aT;HA`JBPP)jmt!46whuAus&l9vuwQNC~0ujVA6Sg`_jz@G*V@a8zRAm zlX0;FL)o-7;^VO)+-kF*{b6)t@pFWICr8x!BA}0xA-WcHSO40E^FJl{{5^0IHva+P zF2`FA!;gm@N+F|%4WTWm;;QUOM%s(z5x2~cY&lgiDo+nW^MHr<uq5v^<W=_yw*C5R z4pcaKCv138j(Ayd)maGpdIB9OKABKXqv?Fftwcs2P3x^P*6TGb>9jQGrNnGBi(ao} z(NldyGws!+N;IaRqIRO~(^%5Nqcsz}Ql!`_v0~Dl2yN2EjulZ`0(<=*4G8}wHo_I{ z75*scCX}<Q)ze|44a;V_gs1lA`ACY7DkSezkvye`ke3|-#8lqq_c+QgQ~jG*q&qif zU71+xM0xbI!<oefgCj}7P;JKiLSnt{VR6hY=taHf(B^v}u5+~{d*$3oyCTWVT4qC6 z-cuWLPf3IS!aRd7bUDIAxDg%D*H_e2m0-<(t@Zg&$E{sGOCE2^?Ph(sxIp2%GpCGS zdh3rX3@E1oEMA!7!MD{s2A|li`Ix8ry(&m>@$-f~F}kVzU02KCKN*X^8N}+cI)7qi zBO#~FBKuPLQbB*&(vdPT^*<zr{{{!?jAZZuVnEwVWJz=@yJDN=Ph0)3wkZ%XmPLJZ zu}zj&5bZG~Ps_NH{@EnWYL7t2n$D`6n8U2CWof?bB`ci+YZ=${a$IPj0a*MWRu=g< zuMa;0&R7aN0u)j@pMXHlZ1E|r_gow8oyOw--)Hk>r1-_<xM=4z4z$b5s;~VkF6eRC zC+|71Rnz~;Ost2109xK9T#Dz34WU*YMi@^2n5u{s1L{Y>L=Z#yZ~vsOnNLR&_VbS~ zI#zW~RrL2Y+3WXW3Uv=3$UMmOq#za2I3qh3S<0(?zJ?^npn=Ip^B&Vk6;M&}Ii3m` z|8$oW&16LAJ=DOXDsh-${50ZCI!zc@Ym3LF<?A?yGy0#R1C($q9H65g)calij{~6N z4<p!}Pt5Y;NqWveVtk$svRaOYUc7QYiv#Vn-$9X{l%}&tvigX}*O#7$8qX%ty}9Xx zWYJw91`ij(?~PP>e4gt?dN~G<%)y)ua6}H)Yso$OZ*TmUy(B;m<S5?j*~$OHh%%9O zG`vu6?{`@69&SgqOXb;xg~$lAjYkXAM{?_l==J9Z7}Hn?6-&p#>22`X$imZm`0W|) zRPG3lH8r3cQ*kxkm4Rrn;9*v@p#a@(&wkN4uwIBv^Ug|Jiet=xog)lAcWacqz}*i^ zu*Uky_{874rqV&SFz{ZC8DR~%KmC^h*F`j5*WYD*9lrFhhImPQ=k*s@jJ=r5Np$=W zeovnZ*Dvd%>BE^L;@U4}S`~D$VP)F|vRVKzN+CBzg_Tk=D7vMk{1Wn1YBfK7L~H%~ zHSkAw0lMP0Y_W0zU(>dvnX^M59-+FRW{j<9`dVddqY3oCaKo#?9Y#s{PG(<(SsNAM z=q{6kY?Z=eHQBQX{6)>e2LZX8uIy2ZUtqhR&&SuUqES}156tspF83W$J~DmJqWjUW z_vQMV4XpjA^{p+$+eiQW4{Dck^Tn4jc9V!<;1PX&`R%-WS>OrkG0L(*``hokN7bm` zNbxk3BdRvt4G0Hk>~NkAi6g-D_ViUtT6=n%bKtR@g2s0K6VkKVqhaph7ZU4<ep^7S z3h$^WnKsbgQG~*F8|^7owxjbz;0RN_NtuPIRFqUG>XKwv$ys}ia|4fplxIFU=^f?o z$tunj{LF@ig%Ppc{bKH{gFTEwe`*RVuH}8-UN&ou{Z~!yla0C#1eNOD#6Lg<iw2Yj zHO5>ism*VjQ;X_th4Whs+?L^-!bctof)hZ)eDzjM$u+-@GupS|G{61z_Xc@V-6BS5 z4kSSrK}+L_4JdFwUKrdSxZ>Nq`eQ)5R0He&GDo1PFYQen5j2{j{ztPjop@+)H92JJ zCq1hY;jjD*p`l@4Wi<*T$XQZQb#EE<9fc`(X;uAYEi#uT6g-fF#uUx&p8Z|`>Iui4 zN{JDjc`We+Y~GZBX)M2{T|twhhyU_5+pVFkb1$q5>{9<2Nw^M(wxkro+@tpSftG=r z#$6f1wx@wVI5G2N=x1D#JsYgVl_pTkP`d09UbI9Nhf2;Y>T-?OLmyqgj_;ouFbK~E z`d%br-y5g~t75M9RZ3W>x%{`fqmKuYWJo_*>V9Sat|y|=6FErZd@+E}YuS1o66#qj z+Ue8@0Gu;Ob(UOXVX4t>%wK$Y`Jy{|$>SqBm=-xtkNWJfIAL5W(9sY_UaAPfOxP`% z>8B#VjOaFfm6;qYDUrNwZ`rXRBlDwnWg&T*UaEIllCg*fh-7M<dXq*U)h_!NH4Tp| zX5(|RWI;VnO@V%v+Tu3$1}=nMw9Y(L%pNkm5?9ZjcnkwCB(m67XZ;}_0~+n%{w$qt zvSOP7)2f`!We~it9Xfph=Um!YCo`Wa+t^^ksQE?fJ%}z>7vPGuV!WDA&S8We)6lSr z?QA4hMbhGAvYImugmZS&Tq8KOdU(iO2){#yv4c}qiiQDjTox{Q^lqFJ?#{JPvEqEg z$z48~on1>q`pMwJMrH~EMC3<X&mH|??VmI%h$%IBqCgP{GO<TJnA&xUdn@A(8HY*6 zMwq~#YCg!gS>9i(otXpZ@>IWUb9ZjJP)QY!Rjb12p6;yNM#%8(5qB)%62Hev2*!@J zrRZK$E-RhzO}YHCy&cas`Dx%nIe+U@R;kvi`jx~uNuPqxSBm6`q`m8igVT|j(Ax(R zUQmg0)6nV|ufmxeFHg~w?;-T;h{iT}P^n}xT?*{=`}f#e&+aB6lXE$xkPCfdZa*t* z39iJ=SbL1VUUIK>Bel`v*EXNrFBz>`pWRhiZQg0y(DH^FdA=fvCQenFUlKRjRWZlw zO<BF-r(XGN-@FAAmdWg^ykqBF>l$0*JS^F4Y&jd)tFTGIMu<Eqi+Mu&6d?XYfNLGk zf@8sUpSt$(ID6D1TE11MCSg-~%^9DnaX$yJiKpeUTgSG)_}QpTz1@Zjb6PI4>yyO# zqc|DV&;_H>$4O-#iE23AQPH?Lg`l(e(X+T73(?FO7mub=L?NTmE*G|7EgpdxGX~;C z?`GoDEsjh+F@~h?pmxC=^5ekb1{4Frp@5l*uP050uJP{W+)?@ukE#KKxl?<04gv@s zU_ha7VXynub#HwZ<s42VP^e8uC5iJ?yA&cKXD3WVqXu?IFo!|AHNO6%A(=s`c8KVn zjwAA--w51Qb6m5A=^7;la6>C1p}V&h{@8xEzl7;wd(ly2q9?J74NSU7$v)!46? zwj^03H*wU_`e0`RvEJdsIPT2{bOqaY)>f(N=UN9oBjgSzPB1Jjs+)~_>{D{C&{q9k zdo@-=P3ISIg3(-U$Bvw8YU(RcofevP@}34%t8AO*K6Tu%_bJi8v~Jt;{5CQZOQ8Dp zOLutI=Np_P%!)OV<rd4<M7iz-Em_(37|Ry?#~T*+xq>Yl*Uwt=sF{)T&_*Ytb<8N< zj$Tsx%^4EHv3xRP0Vni7$+DjEeR-wD#^>KNBFIxuEJ{PoCg_P+!o|83D&yjX8drm~ zj_#HE4X34@RxY<CjR|_{MC2MRjx5h5vr|>HZ$5X(-!MUUbm#^H3mIqckOq-bS~UrI zJAAjpN!*`TrSCuGx^OwIj7gpC)wn&398a9Jnq<m+#`;3p8z@Tev?MmS*CykP^NnP` zuh;QS{Dh5-nnf&9ijyGxP+c*T&Z6vw!I*+Z&=J5`vg<dk`aM{2!UX1<;tF`CP$H0S z;$ep@jkO%IQ)3vjdmzmnO9l<gG=gE~g{THPU6)P-y$6Nz(X@T{`1$q?Jl(N|XL>8F zo=26Y3<RnyB34$HrtV_CGZ+*}C<kA2yUYV9AQNeyvS1;J;2AXUfJtZ6&H5l(4;@M8 zc6UVDL9;G(+Mp+`!HHW9I{rDAk}xE%#WH!$!%F4oZrIWWLl8fAI1Q7^V?R9bsFLwA z*;n%ru`8AlR;f;Q++07DWp3nah1o)fgz7)qq<amUm!w)<ET(5=*$b*|{v0Wt%-d@E zSw!-7G_)$zN-aNS>|CxtuHr@g@UP~#kP=w{v{c@rL?q`2#d%&xs>Ix(vpa|Rw+#@L zof>yHi5zowyh326;F&+E2Z}AC#`Nj^5T+H4jz(q_(zbV#P-03xT^IfUk!ZAboQyd; zR1*%LPK6oN7O;U}#V9g8Ehkw=))(b#RV$4bhj6qS&O}janLN{H!zrHWhx<<6`S4+O zI|F}XtW%X(UslI$4)M!C&=*W&D~nCzjo0d%(m;2kS*=!lkvG?tUJr&N;Du?+<PSM@ zgly&M{O@sik%qnvAW7R{66d)y7L9PX&7~~^Aee<+#;O)}I)jFyk#z+<RzZ#jlEy=t z-)fu@KOU){sUDJ{??2dmLQLK6Owg|(^aW*ba>3Xx^!Amh-6(Ni#xh{~7MBJRmw|Dz z*mO}Sc0R`KFY`0Q3hCy{wqEk2?`xXlj1n(fq1D*qWyVt;59*2!MHCsEgpVi8y>!|M zG#2Y-cg9M8c_J%YTZoEnoolPeR^)foi;W8o7t5k=D}6GM0`!yB7(~*Z%bBJOL}ObG zM++chdo_EPUzzDLWe(4o_)Zim;FZp~4`BU;J~gZnrWrL95+=^r-)^OoM|;O{(&u@# zCgUd-NrtD5#M8`)jgzfmbsK}|JIL^2%qlN~dvPM2&v5RjlzEufDsAo%K4bNfc$<AD zTi+<R;B!?WPc&DFn2Hp*zl_rELp{szB{lzuLA~S+uoYIG9!9v25~<rFeAOrbe1G0I zje1vuZI-DK?)5s-Mr2fiTnmY-<bFa}K|dKh)W^EDrw+@YB#fhc4BmQ?tB}Ogd|B=# zT=^A2%SIYup1K;TxQW0cTrwp>WVYP9$Do&(*A<ah`-#{HVpS`toiweuMz>7EVDaM_ zevhsB&%E>dfkg|C4i|PeBH1_h684~z%>FIfK7-K#%Q&mxldq4DMdt3nB0NZK2bj7K zk}dkkC<gxL(}iEj%hQ&J?grx)KOoJNHT&;ePm?b*ul?ASB?F7$nGi-E+TU|=(sMMg zINF>UJxe@H;M5<0N3&wj#y5)$yyJk300d=v{bm*FoP#fpZ*#pg{S<nmuVemETR7lg zn*O$p7M_HtLBSSOo2-B>lt!t$9rjqj&Aua=3ja#~@F1#_hoA5XX+53@fy^!aDN7A8 z&qDy!<}OyjLbVQL%hG8Y(oDU{y}fR*P>6KC?kCI;p-T7#ky!A>h*06<=qT{2y?P1= zyfuKP$5fl^k(0-lspza$*`>jAOp$h#(n1<8f4_|lmG?(<M(BdN%8FxW%yot9ZMO)R z#_@Ns%&KN)<CD{eu5?ovmt*~y`#W}fYH{czi&~3#Q7s6}GfMq8)Hefk1Td!F-!vI6 z&A*C92J@TlBM*34IwUCbdKX2<8nwK)P^I&7JxF!MI2s$(THG*&wWs<(tjtKgxER{y zcH>>yIOhsONXQn1j>2bRL+CL88@2U`-Ig;^KC*sE|2+z6!&`beOesM9A&M7e6X>cG zPm%|GBvNa`vNS^GJe+nDqodpO-X}5H`KVw<hJ8{1k;lo(@+Gl`=u9>{*Off}^;GC$ zdcoOH0JJmqv-j2`?R*J@+%Ea$_USDR;g4PA^35SeQtp{Ug!argO{(kjLyAR<(R`yA zE^T4@G406aJ+IuEuT<YWcFwlTkrB}Ib0b>RRCO3yvqzZqt`hOfsv;9?tLGVxCi>Fd zE=H}&)^os+bid4_aWNv@+py+DC{wG0hK1Lkno`nI^xUSqHL5ub(Na;7C&lpFk`;o7 zdR)p38GR>2A)zn!gr^wy1-QOjSe2vd)4v=F-AW#YAUn{SHn1||S9il*thp&=Bf`v< zPA(r6j|26%ho4cOcng$PJfPjyEV~RQ)+tG>%RV315)>>EDuE9c&vXVhjIepJk_EMt z;V4I-JlEio+rwreD2&2;`W4>OnTGC9ic?@u7r}qOohBSsjz`@c_`#)$ut<UgOFkuv z%5DUU5Y3NsZonrAJgt~=D8^}*WYna?xX;Bx7*1(rhtFjA%+=qfZP-*+6Db_&*5_8h zq3y7W9)cWbyZ7$1&pWS4&XKKPV#b#TlZmO&#+5cI2z$HAB<#%Cu3E#p?}Q1SoiVRE z%y1L#B}I)E*BgMt!EZNw@C(zFcQ*Zz$l6r&0V-b&UTu6n7RT=xOfwg7QLty474#kC zrZnibJ`^+^!X3G!#bQxb?ewK=LCXV&(1PQnV3X}NmheVk$77b#$3XkddN?@;YQR*S ziu7IlY{P@PamXWbenSz81@bq4(!MTb-`?AhIKVZFGHrY(c_#1ad1WZtlm6^VF}>pM zbtc1(l>~+32@i$DbgB>>CFMylQxFl+ASPhPk>pdrfbGD7+CdbX9q*cK?rw#Qr!C9) z;|><pSxR0%ym|8{{v){&lZ0xOOyv%(F_R~wpTN1aT%T3wR5dDxDz`luxgXGt5tEv8 zWT=vXXktqpb2peZBtG@5_ehlwep@~&A$6-bEzZ{vYBsvf$`yu)Aib*`aEsV$;U*+( zCq}Y}{<d;bbVJz(>^(Q2o>kfm&K$c+CMmp|>LHz@0a~?qMs<lpu4E+?nN=~th3ixv zpIMfM8Jn0Swo|#nMiyjm`Wf;&GO?8a<c%%8KapaYO?KX<-Hvgg$-$1>Po?RbPi%s` zfhDwwcZ}tQ=!S|k!3jQ4uasCP_u2W!8a!XNkADOYL(uP8%|S`H%;35>39mN|_6vNA z`i(r!f&$@%tBH59-SD%;j@f*cjg*#}Y4fYo?u=yJsab=YF^-?=+!5vT9TZcsziX*? zd+aX5MHK&{)ci`c=HSyxZ6Tr==lQBZ2gdU_i8TadY5Yh&oWU;UN}o8q-OVQ-K~S$z za!5jsiLUXjnsL<hHRTAdH)g>3h|UCJv;L@;v`tn{{6ebxd9a@k+sWl+%HihZk#!8i zlEH^Z2(EKG`Iw-j(Ty~b<~vMC&H$UYse7J~%m3r+EyLp4mTuudkPt#h2!Y@U1PKIp zhv06(-QA(l#seg{ySuwP!Ckv?hsGTm@2_+2-S_NszIVU>`dLq}?lsq}S+i!<7^7?$ zE6M??;kM|XC(zM+dd022Xbc-$#e~qhy!y5m7Zz>Hul7{ZdM_yW^x_<P_JU_#Azrm` zIYK5lnxi^B8GfFQkbJ!SQSNRRL(gou2~F?h_dX#x9i{&&7<)(iZA#Dzb`frKpO)a= zYZ<whOfuew^u>CVJLHIi8kf{E(nIvC8ceOh+O;Cmr!KdLA*rwV$s@?EO-e{qBtdEj zopsNfyBvJtf=j4^02k0313S|+Oifw1l~GN8yhZ52^<~64>W`u6Jx0&9?Ds^;*;2&> zw2E#yZ;)(pB~OEb)GGC!JV(-KV5a1s)QFzuf7wfFmo;W&WC2W{63bW6xk*%pl<CLU zHXWU-yYxnMKVS-_fO6hGesst^3--F1bAz8sRNk7|+V)%X@!gaxU7Ip}=<D=4((o>R z<)(h|9Vi^nm8b(12=%z{iXms8m-rqyVBY=-kQy`XGgHLR*X|oEn-<<w<WLo`wZ{PA zkYkL|Qi}e469B{qjf9DAmF88(T8rBED<v#x?y^9DJZnL_cFqH4jzqyu#hu9TdTk|o zp&`IyzQA=n3RT;AfZc+aZD4OQJ!W#1H+{Hisb3eSQMm9Tb#BpP6!87}gmw<DWEalh zd*DWQBQ0&O)yVq75xgAHR1U-xEPhB-FDp-UB$~duJmO*-E^bKlA>LsS4_BYJAKxsT zz%K2eA)@U?J)Mlg50d14DH~_TAATcw8-K@zouD<vq~zLxzG@I0zRdbTLq1jY7oG_^ z)>J{3j6?m<`wRuGX%vahHO8VGd4ks+P+uBoRlj6^DgA;)h<z5hwx7kBxQuF+!eEhC zIkvTg1B4>u@@pLtw|Ypb9J&_Rt8!MWXK@^+GNt^Ch6~4eNtc0M5DS+hBe|wZ!GW-3 zFt!y@471TvCb9XA<!N+$iW<$tGD*|Z5dZ3JBT_22?ARrCbB)K5L2gOE0x|tnLxDtI zmOKnzGDwAb6g$#XIOZ3=kQ4tsBM?_B;iWfN;v8jmwpvZ*FaqvP;y%v_1xOb<5PMS} zY=_&f36C`MzS|7L83~K7@jjev^Q<_0_$p5TP3Ww>z=IryR#X`7);Rhu-3{y&fl{9r ziZ`Q1O)@h|c{5*8F`k>Wb&y1~KGpR9Ngmmt`S$g>2ZUoLOxxW(X=YFpU||L3n_OAm zk8&kN=3+3%m+<ORi7He*&{GMBwA}@;8Vt?+3>HuO`327*ZYx&ansQ+=Sw7j3P<bJ! zG8Jp<p|w6?y;AG4ylCsA>}+A&Q=$jh3X&uZd?tSzx~yb2MJQ}3MqDQk_FjhrrI&gT z$CR*%v{}p`pZ8s6ly}IdU%iPBytEq^T9qG>0rz{wzyE&y*w$Btu@b7|%jbRUB&a7* zXC_5ZBB_wcH~kBJtJeMqFIb$@c=iHsuFBmx2|o=<*o^FLn&oqS#shv=wG@pT9_Q*) zz#C+Z?WU6SnrIE<Czc9`)x!)nt@ut{{jg1M(UMkb42+YrX}|^EnHu%}Je9bIsOd|x zN`{zq!b^J*cV<4p^2Ja6@vOJE4s0Ia6BHuHOP3`y|F-v#U$Y(}ADE(*Eq+;~mRTo+ zSOJ!G`tOA7z3;OSj~Y-q2Jvp9`%%Es)8lVyK~xz1GYi46r{joI%y5_-2Rs!zdsilQ zn~QoD;nHIkVtGD>04)`*jwj7VXet;d6}#8<8Qz~;rbi*@cz?3|ogLcz+Kd}r>d6gK zy5zN2pgTd|OzGAh;VkLBt}wxPK&Za7Jg_;&YO_r4{_Eu}j+tyUy2URc!!MHDK+NfO zBtVFfZB;iln7C4-3|_D#&hX0TcSpG$n_C@+RzE0ObL-<xw~ga4<qaT5&Ekr1mJHB@ zMQDHm<oLc!?7L}|$hGie!7h347_gw>z(15h21z9RQq^sUBPUO62?{~q<x!bt7=sFe zVcGHM&c}>`{Rt5J5F=?x`x}qfMV{8-gEIlBC$eIZ<Y+SeLtF3o@Hd&`H;#i9zr%Id zacFPLXH2WN8sSEr#Y9>nL96Jy>ge(MqlsbsmSFu_c~aiZ?hSceGi30iDza{~+UpJA zy6sEBNW)zLk!ySKZcsYG+QKB2UNR2`-lSmR5I-I6a|c+do@}yjzyx56h#w7VK+=;t zE?he3^UJYcLG6I9Vj;;QUg4!R>j0nEiaq03?GbMSQYPB{5MA__-#q~Drc8^_ZN;0B z<-QMQEUKhM9DaiaqmR#~p2)6Yz0N4^st#q4R(GK9ro+z8?^Tt3TfbZI6$II|b1Qjx zHhb?O<D$8CK1nXGyaa>z;gp|+bz>h%84<lc;my?@QO;rfbmJTM0zG>2(3djmgmF%- z1hQrs^&N-pR+u?3&+zfwE>_~W_swl9K?UEE{7=7qz#QrXyVyl*mOut_v0S=<%(+3p zkKBzXkM2lSjpwja*B*-|O8jldQmq9=L*A=Fw}<W4lh&hQ#|3r3<qCD0#@4Vo)Uj@| zh;~4!i2D`MJ5ZsL`sac(i$U|!sVYtif8n(6?I3ChBATcA#ksyzS`PAQ3O%-Xu;h<O z+iY`)!gro8xV{(6n<?II?`h8IyXcuvaE@*bfA7)KOPDU_xYi}wVlGFtZjzjJN#o60 z64JN@^zV}!JDqy-UwrbqS&c9s?-}mr?}2n=EzB0&dp)5(TzE1^pSZrb*@oHE9p9<= z4Z}LNSOEPLVW-S&diT@e*G6!#JWP;4)#_y&3r=`}*I9qXW?H%CrN%zqR}+nw^vQI` zyI-igYr(%BIu+$NSPRA4m=KMSPL^q=mz<5)d<3UD4;KnxrwjUo2n$=Ns;o)f-zE~q zT_8|N<$^X1QA%(R8QIrYE_{_Qakol+adNh>8wT{~)V;}18Ua<8;gIS&T`u1oWyVCJ z$wf7%y0{dNea)@)ofXvUaCmXAllD`lUkAJKGuMxIv<}LeMW*$;B*X8SHo7gV_X}2f z?}mT8N?u7=l1Evh!yQIv>mTrL%WbQ|$~ffubi@(9wA2sQXfNi?z#iV)!@`zS3!Zs! z+0J&;(VRFg{Ot%e>bYTnBD5}36lj`|2m^lhH+Z<{k1dlT>PPc)FIWlGYW>VPA8iXK zdGin%oBpvNb5S9RR#2zlhi6eDdCsfZb$us!qL(gPTwsB^cQi-O7|%7uK2?+7<j6UE z8u*hIGi{RyZj`KpybCvy3I#CiQ*E`Q6`M@Vm1T{Z(*wuB9sTYF4%WxIr;Ipj*P&*# zl<omJ(uUidx6xVU6SSM(w!Z#|@W;J6)!Ce^G_21Xef%u|sncRoM81P*nF)sQ{CpUV zAyQSRcj}-wd!e$WZPta@q$m4rUwDfI+%Z>7^6NV<ct*o7f)B0hd@<(D8v{|TV8gI7 zlpq(viB)fuQy%BKalDt8>Y4a0D~myk33S3vm7WxNT;UOi&Ky;y6*!;mS!YM@E9%?n z$IVrRa%}r9F7~>cu+Z=+yL4hI?j=saj%|)7VGj7MRYFKZx<q_t8Q;S*l0NHc$g`s= zeFkbi;l1xvjd5o_d%%;_<dF6TtUW=L&bs)^1Y^Hof0@3lLH0%2^1Q)HpPTBixs@PL zC?BZCNRS%a=*W0V&rMSc4T|~xW<JnR<Kx2eF?U5qH#@mmDP1{q`7_j^WYln`+M_Cc zdTz5l**WMwH1W;eYx4Sf(8lE$|E6um>SUHE)J0k2r7Z9r)o=`?cAoTyF@uP`>T~k% zvWC!{oVxvz4zp%=nRzQ(jVL`w?1gaXw%Yhb-AuS+%QjkzUDYd6G*K&uVo<#Ml}aji z0FZO#c?=K#6grgXTlFgg#pn5A=Vu$%-f@Ff;MRYsF;Bmg&Hz&4rzG1hJY9jRGCQBG z)i1i!036FzN#3!GtODPBC?t7P+c1cG#5NwWoe~q?hJTC5hz{2C1?-t{BFH#!c)1D# zIWK-ccz{VkR86S-kxTyJSzp$t6V}I!Ok)c~&aotAt;M7#Ytl>xJ6Z;lyumI^pz@68 zZb6ZQlREcCr3uC;aWO|R-)=$KK+PG+x$Vw|qBvhk3_}6$Enp9JZeE>vag96bq1Mq^ zTc3#i@c64$$1eT}q^l_K)ZDVA*UcSZn4i{$@fk^nz>%NTxlycA`?4Y9{qV%SV{->> z^@ARoQ(TF2=AuTW6({NFrg7yJIi7wiuG@3#;47O%fs|1|jAuiOehZ6FAj&D;{wAtY zFqdT|au|r>$TS%NUS?F@y1%jO{l+ons)vbZ`LEh^XC!)UPSMXCub|X40w**i^_@?9 zH7<2Z!^`?wkbMC>A~><ynIi=p_+@ZiToOaXUKa8e3t>{?MXv~_y0UCsO%FGG!v`!i zT*qsuX>N1!!zCb&`#2zPBB5^w?ZFq^XsebtPWCV1W6csdXRN=x5hiMH=X7@UuzXQF z42ir}cS>tnym!3gV(;8%&+F_;1CoS!C>=lJW|c&Sk78lvrR7nXi~LyT85I!CuHml= zXyaSN{qN(VTWKXR(~dce@T}ODEoWJV;{If-oa@W%ZD~#qWj+Vl1^uYZvzr_28wg=+ z%)w0Vg1Mkbp0?<cI9)B3?o9k*RBCSZ8_?j7>W6K~-Iank{Lvb6Q_T>qXVY)9;|b!x zr-@su+y5rN{hv%519EFe+N+)TIXJWF|H=Tt(G2Z{w})@WibDob$=9w2vvw1QK+*U) zN9yYOc}qIq+;Wya<V4{gyA>LXsq19n*oFT&%sIR=*!s$@sm@1zHBb!-yvx}(VlGUz z?sCwui+s{OCv>mhp&o%3+W!J_!4oPzi0o?c(yT25RmxwC0XSIt!>gl#c}GID0~1pB z<8LQXeem+|VdIeFq3578&uSYCr-?xaP`5=6qR_3K-N$tcW$q_q)$SOO>}pc1B_AY; zkO-r#k0a{xK$YZ)cYqhL|Bj65oOof^IW)oHNb}d_P$LBAo*9i`IrJGjL;nXyIp6s2 zM?(YGeHPg7v2zL=)yl86ijd>Oc7$C`(irA6?aLcwse?X|spsVv_RTK<Zt$DAK2P@% zw6OQUQNI5%I-$><QGG&4qoaRaW;jQtjvx{KVN6a+q+m#LV_;#?SZ|1&aV%2b?|gzU zbSx$+&Y(z3fz9Evj9a<0{X3ouH5w6b6!&FqNOjR^6(qO27h#f_yud!$qiTzdtv1#W zd>j+bGkC202h;l34E`Cg9T{>nKFEP=(Ny!Vk;>~Kq1U?Xw!F{q+<1l1klFhNxn_$z zuno(N7{O4vNNq~C@C)e|ohSF&X^mfqtaEC8q>zj+s?1DorC{6vUEYGKA<NQ7)uu3+ zr}Dbgzpy6%cf<^8hz*a*`CS&NM}J@qjS7g<rmH{u@^=Vkmm(Y9VSjkG1Xzr;g$7#5 z1EoV}+E?E^WBYr9;y(aru&Qjx28#vqf)$%r1A%!2T+^PUT<wJzenc%L`u#sQP}X|` zr`-|@YyEE^{9Jg;L7pjM#_NA>IoSE}4_hI)(dvI8D~Q880lMb$8XW$sOVNML_y2j( zzJk~vn1iiblIiH0f>X!OwV<e$edD_!Te%!+6ms~bGL_o@<9+{H5IGup6aIai8QU9c zS?`1b)jYFaq3?_e+|9&(TBKnAuhsiEW7DgB@k($)jYPi`u}ME`6Lq#M|N0kBrb<pS zJc0S$f=%{czyAL$;oo<LD*6gy;<K}`SU3s2?Fsu8I^owhW*t7&AKLQIDq?yQNGQB! zOI48B1WUV1wbvTf)<ml2P-UHs`I@P>e4nISR#(rp*@k}n4}bXAT9}~p{e+2cyVykX z#i6P>(!NU@P<d>@`#njF-vu5lbZoa+#r|*9gG@O@Az~IA3ky*P1FeR%kQbQ0f5w<< z9o%cKC`ZA8>i@YM{w8rYVT+LA1=d6U=Sjf+PzQ%nK+h+1EUiTkSUoWF4$WqVF6DEF zmEGGcwaYiZ{qw!Q;oL3zh~qz$h5MV|B&)w~LFMRd+b6vwPf*~4TLO2UD44u0f$P9} zykn!y5Bm2t`?rULe|(h#hY7W<MKJCnTP<U-0ZLt#E~k<q7HxtaE=TS-+y8%r^k1{; z`w8pAIr=)rq}TH4=p{eZk|QqS=FWPTJ?)DZM5~U1<$v*W{<`7c#=#Kq=dE57F{4;p zgVzoguk!1h75g}^8d_^sDwj3J4Z-cC!gQ&qN>2)hgKdbciE|&`1k}2rM{#~pJJ7JS ze*2oJOmHR?S-emEkV^r%<5YNwm2xuY0I=m%jq5i0l-OeQiGn)b@;yiG$>*2U1X-f| z|BrjHXV$RK0U#<C{034Ko+a+S&Us&nPkOr6_f_ozbN$p*xzMt{OOJunE){Kh$M_Rt z583Axse9tI&3{A-GrUAznw<Z%(1e#{J_qkR@~toIz#+t0wc=Zxz3e0HNh!NlgG`(V z)<l_L7q@0>vtG^Y`YAviu=;!;_G8(j#e2!z)V3^Sr9QR|17jk2HyowUW`x_N^X|k! zjwKVXbk)cOcqRcT6&-+=s%Zz?p#R_>d_4lK88~AU*W3EZEPx6`w_w}Wgwz%nh4X=d zfue_A5Z*cBxnP!<Qj7gIU<3-sKl-=OZ+}*bsY$NBq$Fu^v1Q9v=nKpr85xsx<sIAF z?}IG+)`;u4+1T(b6Gj3Cb$oeKpr1}GxW`hPgGab2gaYrH-8m>6xUfQvsVZ4X3o3~f zupX3S>URB*Li5aA39%lN?bk5iO%*<v_7y9u2Qz`lJivb-wt_3^z(lMLd<wA^_s%(z z!J5s;H^hF-i<lPH6h^dt8@b|hW#m%@{#52RzpLJni?@&=qOw6(o`lQZ|G_5R(Yzrh zMp*)AdY3E{Zo}@tQH#(Np~$wsuguQiH>X6K+@yDdvah8x%SQW70MBy>+BMFBtyNna zl~&#Z{(Tno%JLj0(M{vrORMKV8&?K&{86<ID+<I}wmxBluEQ5wN5uhUO;sUfD25up z(2zM*v>pz5-ta38&Drm0g{gp|8IjFszv=ilTXq+RNfG8m{U;+qF#=fw<EkGwguj9K ziw0`>TBxC$wg@tQHrX44<WM-NL4VaZgh5z5vOVHRF0~Z3Ry$?=6`Ti7>ARut9&@4u zQ49Xh7{liU60*4<k_Ozh_Vwi}G@CA!b-np;o@sxzjZerM@2`?#T8soOzr_PGdZ=)9 zQw|jJBw1Kgtj(QG^TubeI^uT+?)f>O-0Y^gU=o6k2}YybJJod2TiTWx!B?X_MfMCk zsnJgFjH%k#-Z4h9;4bJ{4RZ5vK9l)@Wd|zOsJhJqFNGBz73U5z)fIp45BK^o@-y70 zN`sl0ak8DNX(BNB!p&9a>GxX53r8d-$f!TLaGiI(?NP}N$A4lx|K>z<RP^9eu?RR8 zgCoGVSdQI$ES6{83z;Ikj&(4!N8te5V>YT%zoFt0GR$(K9@N3tO#s5y?9T8qW$jq4 z9wt1VDH${L!*KC(Kmn#Vq7sAg(wEfMg<xi5fbasn0LItX!BPQ|Xp>u$>0N6tzH=wD zejpdCMhbo8xzmfxtc1kYTajhIj98m6jf=i$RPdX#E~KXUeeE*Dz-EsE5^KI+Ot|J< z-1$eJ$l#6?$YZuNGZVVi(`bWC_W1<zGiJ=_<pdolo{u$e@f=RA<8a*-ZaAoL>18eR z1z>9xMxxJ&04&OPius-zrhj?SGkoMFL!^W%XKgImr}n)s`>|E@bHO^3QyS&%@Q;gE zZ(T2?q}CymUY5#q=N3o7!SMo~N&PP&hjla@-cN{qp;BE#b&j=pGN6zFK0E9JauNss zE!q0<UBmk^Hn)FX{gMbLpGtBZa4Sp;{G#+gVuv6?-hfnUXLT;<_k<4AppvWFcZ_FM z$M2t~qDx1=omB*@sC>k2h`A(+#^vPgj<W5>E-LeHd)dNQays`7x!#7)*B@2Z+}AMT z{jvHY@*$rc&SJx{KsgyuCes`X$E26DT77`6yZ{J|=lDnU?NB@4PbB6%Q>|FIvcJ2~ zDJMd42$vrCs{51PFw#B4s7p|oUK>#20fiz*yEOI>rafi_8N@KiIAESABN!%iD`N37 z85D->C7nJgG^wp*wYlCh&ti)ILMurR;phx)ii*z9t_<0Ez8OhqhvPPCJ$JuAFz^~P z9zf3fLsBX07wS!SaC_aV$br^BhD4#H2;nTpM_sX6zja;p*eHVb$-+}AG`9U5l*@h2 z8h1H(9@elboZf20gd=p1?SvUVUd_KA=HB@xJ7AaJdD>fB6q7nzxA+sUq{;#iz{WKP z`q*VC);agVqOaKFijU*sEovzGc_Be4Ih3|Dw`zagrhj&y>pMJWTzuRHH8y`df$vRW z|9$@WcU1D>2O^>QQUdC1Y;fGcJNvS|m&FzbSCdUBIa;aoSib3lKk~{tZ}M&|JRzd} z!gM{VB$~S#lVFGPm*Gi?p?NowfkZMOck7?aZ(b>7a$_B_yZy8(Fr;ldVaJ2=aNsq9 zLi2ZU=24%AB6OV|oW9yEd!*CYA^msYf?aev{g)QNzxV$?A&4F}n$rus@N8a!0ByW? z9K;Ie-Zzxb5oMkq_9s3oT9l&3K%>HJd|dC>$NZ=0#rFX2@Gi024f`_7jx-vy;iA2G z5TnbhAEWt@8)v$UR%Q5w{|)nR_lS7BdwA6_TYzI?l&?Px(td$QscL^<P~77FzBtq| z8SW-^+%}&}AWvQmEF>?`$Q%E&^MWS<uh3t)O>i)iVJInLDJW|%8lRAL3#~&;oyw## zlKo<HUA4-aHZ*oPnLk<ago=i<T(hkIqc0Kg^?muueL_u&eiD?QmuQFL@_#odpvqM? zDVaO%iL_yt)B85Xs(;V+-#5KKSAFS_;Ssb|5)YaiB*a~sang7krBXhxSUzTf;=keS ze{;jwyB(dGsA0Hc8EBS5 ?Pw!M{M9&_XPQ9E{p)((FGod5rEoq+EX`k!aci&yg) z*Vb_Vb$pHm?O4<X>v?UKYd6OJ=MMI_=N8cY^Pq=^;<^`D+`||Db)5d&7vVT-oletT z_CeZz<`89ej#9&E1EY5XmAZc))qlR7Kta8MDWzAx+wEii=PwC;tvr`n{hc#Bt#|&s zw0GY1IK#_#iC{8ltN*M3lF&WlP7FKWo#wD3TQDN<r+O3GBxSl`F-AF1W^>S*W{;-E zWr;X;$R8l737M@5Iqub-taClO0-CX*kTH@Y?ARkzvg(Q|$PyQG&z|-B-IEAgUG=*H zZ5a_qaB_P^1T8X~tFa*=bX~_)c!sX8rrsGcfS$2@Pavgi_$~=Z^^9y>0|bE6d%Ko~ zb!ad)vm=vCy3}%JD!z&WY?@T>b$*Dg5ut>Mt@-!;kK)n3DGKler*5ATjlr^ZP0vPH z+0UDmVq3iwkG;+vij*jQMoVYPSEWB>>z?-V8cxmJseZ9NzB=%BSmr&Pc`9rD2<Re8 z#3%Um3tg=F1^&I^@x~$}=S(=0KT;UCpp1aY21hMUS^%vlKCwFQyK{m7L&=_*%RP=$ zP|AR{9bL@g=@)S#bZp{s;majuwJ4lpjilJ<z#lbZo1+b{1(eZ(hr1y-{Y@Y;#<zHT zO|C~}LS_JZ5rIQOqdf7ZAt@cf&8na3u6j{oj~vRd)*1da<<*>`x9r22XTcr=bNc^w zK>v0+cJE$(y&-PjhKuJCxuVk_5rx=3p4vz!Mh4pQ0x9bsx9Gv(J)(+@qAiWw%*(z| zt@xFjB%{PawplJ4H%B01ySQ#U*4~c%&EN|iR=4ouAB-=DRaeWyK7jnA0Vh}P?~y*s ziY1PaaH)hgB4<n^_WihD{k?K=^?{(_4dz$bp3)|qtz$aC%rh+JX`^<mdE@m)k0PYd zquIHqJC)e^2S&>)ccTDG1HldZLk`<cQrfk}P$81`Fy>j5Tw2`-x5+PRv91~*_HG_a zG@JrqvWdbI;gV%T`d8JM#qDiaEh`J*=%aj!Fu|gX;97fByTjrVDKhs!!Ogdir&SmK zlZ~(KyiseQc-J)*2ZD<iJ!y7;xMnRFH+;=aViR0nqs`v8oT7Gg(}O;5IIOtMef=3c zjo8SBO*9uqI<56J?ro=qMV(G3Dhx90TCPv~WV5-<pgz7f<89sj9Nrn+c~EpKy<B*v zmPL9ZFnhty=APtqBBtV-rFTv(J;I?!`xSQ?Ewp;GP$E4r6pe{RIluA!xWQ_vl_zY4 zWUbS$V)LrtM6}pYyQB__<Z=HCoGRoSo9ds@dI@c0=EUg=&#JiEHa(q^>+oGAnMrQ$ zg?)fGaaLKJ)tD0!dP{{*yspw@>eHnsCS*+;x@pu#R(Tk<<9|6~Hh3FZ681T~rsUh~ zKs=mNj8CW4&H7oAd(5Xp1OLaR+4It-f=R-ZQtO8Oxqd7Ll>-esE2zB@4~y<jUfDC| z?(O`dT_v;+4a3!d)OioWqzG{aZduXl4El-1de+SR`CtZ#pX;1>Iq%+(qQ7)Gfzy^X za`sP<rhY3NaZ1B87cWvxCtQ*E%nq60`7_Ix0WyifRfaOBS64^x1T-}XN<k%sWkttI zZaDwtXYF_qYhtcG()W$^wm-JiWm=Tf;bgQvGV?qJk)<WG;q`Zqj|cBMv3{#XNGR(> zLL=;_Z6(<6f!z()ou+&wQ<14sgU+6;H)R|gJcd?Fg9v0CXs(z|1_7GNz#J8R=ic8p zgFC8cKP^bAzYa~(7v)$hX|kd85@F5VNw~cu?u&;tl3-QOEX3NEy<khSKf%<EL>fJ? za~gE9vu&1)(*M0ZjKLrQ^UIk!J%V`uSjt<~OE*TFAbvUp65@OkvchR^WRd-0z8q7A zP12@Zyd0tZ@a*{4g+@sNL5g1y)^JB7tQN=4Ch}SBR(j|D?1de}J=Vw`a~5JUxrnj5 z+buVnaWay_5l5u<>l7pp%q=Yu$0Tk)g7o+`Y689R$#i~lT%LBC#Uc4GlShN1yaHMP zS8g3JpxaNZ-7W~s7cfL+!HTze+$l6{Soh4?Qi%NZ$=tk1x-r;94Bmq__<)i74_$E$ z2Rrc%-`gjzCn|cHuG@zi;9bQK$_=W4=Wf{0wKICRW5Rz>aDBaA>?)|e-^GakXa_8j zguVHdr|X47U%FS6r15ZjabO3Y%E%JmFD$YeM<38|=5KG?^8>8Xf5kyJ9;;fgKM1}X z8rZC$Jul_Q`7(Smv+oRpFtvM~aJBA<|I(~|{4x^PI;_yKREQj^ISe&yu1%^l`?5l> zl9SuNF&5N;VXD_!$3q||0o(C<qD`f#5LV}$)^w_6!*71^!ULNL4kG7rOpYxU3D5m0 zue6=SgQxy=Jnt0ytGXAe;wW|HVrVw{oqH#4QWDlchdW9WwX9Wz*m2dnMmyy&(s(;Z zNgU7sl^&~xdYyU0!FY<#v+<}QMH!5ruj9HjilZI(vm+4phg_IsBx9GB>x`+sw7l%7 zGBL8A3l=^R92`>--J<n}>j&x^7QkimHDx-nd0s0kZ_DQd-E$RH7t7jJa@hy|my7^B zt5liZ4<$qZ%MB^}j0G=tT(;S;uV{FFfF&1Hfc(4Jut}D3X*nu4U+vGPh9Vx@a7a(4 z%Y?se_wFis$%_#PXT&88?us*hn*3JgsufGjbQv<#Q@%{1w|d-Q*C3*cLx>jwTXqzs zf#9;@c5qe`1nFwL>`1=2Im(f9jJ4P8^q|e5^TGXy+sS>@$dS&EW80`rpswZo^rGIY z5hG}<<Qec;{8C09|Insl<max1?!%guG6D*{uKjWSQAAdoHk|vFh2pwBX-oYYH_EXI z8kaP~`4;`h^txTc)~7DI!<93>6`n#-UR3c@uHuqxG6%f>ct>l7yQ-aYdpU^xKpE5Z z(%+i@wU<4AH}SkK;o`MnN{+8C@r>9@XajFtarWjGq4ARaZCG1H#JAZ%riKN*m@tjH zsGc~g=%wK%*p;?4><pDdS?dc~k##gN8Mggi$mwB}>S@^)!D5VNRh>_xd2l4r_#hp~ zW%UC_jl0GyaevVLcy{$2*4W`{s1S8Rz+|YIIxf3Zm~?z<zX+{6(XHhNc4SQ8I&p3w zfr63~Qbk)U9^`8Eot`Dg^i_Ds(q3Gjerm0AS@rim1}Iz6+ZjMAr^+Gq+();_lr5c3 zjtcvs@+&g?K+Z6+jhsHd|7P_HbRc`d_+j$}T=)E(0kHe?jQj8LKj3G9hR(@W$tBC5 zy#ZU&xdR(ox3F7vY8bp@m6!f8-_o+R_vIxPwRma%-Ntg`!KAL)#TRdw(t`%woa(e4 zGXE4^J3CH+PTb6U7<mc2#NP55^&APWh>%tNm|IlZ^8IZ=1|pyMaBM8*r~^W4$#JnZ zd6PWuqLQy>3y{>L7k|>}>GP5@g~3%IXSC%F_3PIAc2&({Wsf6c+D?L`#Am6TS+-O> zDEdrbBgY4`O!}j9+wC8cAK7s1?YlT)a>D$Bx_A_Zt+kYnjztpU#3sJMgR}|%-ts^3 z?1v&m_0?NEFDE&seRP*K-;T$t%%i?{uv_j@38Ij0+Kx^`de8l~w3&*{QE$!gH4qO^ z!`b~Dqz%xsG7ZI>dl>CMvxT&}Y_^-G_u_MMTl8n7cbDAzBJjT+I_6J!4%s@p-&y1J zT50RxJbv7n1v40bd(H+^^X#>g&(}#$dG57@l2^lSn^V$pfz6gbWV!ys^eh}~sn6$i zo}t^*%Hcdzgkem{-)ku6&L44d-&N!4st^sRgWgq-pPsMsg<yKj&r+tt@TMN9Pb=ge z5Q$_1dRF;hh$?asvFN39@~&Ijo=5#B&6=9E2pzZg7{M-DMD5e=E)rlZ=n?t)AcUkp zS5H;JI=}tU&=Xd`d(_4BaM(M0O0)3O*5zx%{-Wz%2=iOo$C3^$zqF2fKApqMrIW07 z7gd{4kAsZv`i_jB)wKKvm+HA~rv3A$8_5-$vj;P78~m_f)_0PpaP0s7C1FqTiT<Js zuG4(b!O~%2QF@9;`yx<6LZT;o5n_IT$8j8Q{*1<t(Pd9sYxEYWX%;Bg#O8y6Vfdcp zGhHLmdY!2~1Jr^3@l?=ifzT+wS6|BqspE6Mg^+b#XgQDHMI5cW_HW^@I(!($_b^hj z&x>!_+0nl+VNYEIxVMzysTvy@$=Vco?!SUvc9B${$<Y{WciWra#N;|x)(?s7QtXn^ zN7#6^$PqaihwNt8X0Xgo--RpJ6Y)ftg8L+Oj3rplb}Hn<)r^FT^h?qT8sE$5hZys> zUo7aCdYF9iQv&I)-kT4UMWG+%Nf9XaE>Qa#JKOW_4W!b0`#?>!eU#~<A<`+Iz#E$^ z&k)`}&F1?E=Y~C}ew_~B&#)Fbf9N1mg6m%XR`fQ?=s<K_kj^a@YFXg<K0IIqK|Uoc z`}ooR7tXVG6>SWmOO_GI8G4Ee2P#!ay71&Ix;!2o)3@m=8wCOfYG72mfOhCx4(~po zJ+G+;k!?!IxqJ;3&f6N0fqAFI_eKAx=ya;yLn<zB2^;dfJ~B*FV%*aO8jm{$ho(&& zoA7%>n{b;-+<yhG49UoTl+vQ#oIY!4&e}Hkv;XKP*H^41(V%pyN^4}u?JfBzTcYLS zaw&bsH!Skudota)z`Q^G+I$d>8EV5BNRHg>;S<UO2Nk#5!$Yl5#iSuPf%m6eonEvq z4MozOqPq_~!mxRLV^5&xB5F|g1(WXw$)k5k9T@QFkuJE2F?YQOjaqyAT_#wr&lGDS zt$F8*wQ7dvmC(beQv@{B72PB?@s|WMUl49fm+Q^#>iP+rbWC0E+yhc8BA{cg71wfO zS=aR4Mn{Byk%ZdmX~(mY^zJinIC-!jO~#|lh3>NPsH#mQ<2M-rj%9^o1hvEVM@>w@ z&Cd^qvnP5v3y>R>ip;ave%($@EHkGo9rcmr5I5KbFk;OVdewHv@bY-6p6VqZ4aIuB z+9TQ}82!p@)$6VIvhK>}N?AIy^8JVqZidK$2Y`pc_WkKTh&W}>+1OQ!iK3T)QAASq zrt9SNo?cd{t7Y6^JBGyIDm!WUItv_V8->XK*q{q&DHxj^ctz+U#gh5H3jMfqW={UQ z76+4se;-1XV`LmDs&QO`I~x2H*cFHxMcu{t-l({w?WIsBM?m6cq_Yd50cLy9aubvL zB**Ks9kbAo*PXXo+#9(^;;K%goBF|0!lDMRpKjLCDtxNNerBbN-Xpieit+e8T+5Nd zg~-)P*sO)%@!i^iH`VS8re%Rb8iu_2mklivQDWuNy2h`9>a1_(nie>z=W=49hvAUZ z$=^wh?7Z0TzM5;;LC=j-kgCJZmxbx?E=z*!A(-UiUwzkhFIjSVu8nzEX|eei5`V50 z3XenyepuimbVWb#u9hi@J%4ife8ZzMrMT-WIv@g*wZ0?a0m|FPcsnDsY@S)09q&<w znQ*A~k0pO;3aKFFZkY71Bnsl6`&o5x+_N>u@-@nLz`~+UBY=ox#{eY6`~wzBa}}V0 zYoqekSWnuObjSJTT=f(GKDM@XPgMkf)blr!FvVC8*87vVy^Shs4wW)p9y#5Or^ThU zsM>Cw@CZJxaA%kC9wRip5`>+7&gc$rbcfXEVY3z7+AzMXIASz!czGHv0_*+{N!14t zh%JvwHYPOiT6t?tiMunFKYa7|Yh=+KZAd$Z?NwW(z&z%%F9`(U%m-R2Ug5TU^s4ds zZ`H}Yg?)w(vFV{xkey65XET-q)Q&!S<L{;16ifS8cU~LY{Z~HrmKbcXGghtC$`9UT zT{;~%+pc?O{2Nunx;<p!K4(cTFm(F#+O-U?Mh!2j%C&WG%`n`IQ{uAwt>Kdm+a1wn zk56)MAdbScBkrQ2UKR+3G<;8+74`$Q_^^+Q=o#(!9cowf94%h$ATVfu`X++Xn^JJA zpXdO0O}_|M++u89-f+E=&Zq9(Uqas({z!h_s|7Pr1opJ7y4PHsO@*7<i`CxaMyd?{ z-c64_&FQ~_hTV1h-+)(vq1zGOL@qrkcKy5rKDZN7Ko_YT{V@oQ`RU+PHd6VrLYLw1 zO>;9jG{Jiq@1_P{B;Gh=RAB$W`BG(Ob=YDzjcu00CXJowggfD)z5?+TAym1z#evD) z_Es`i>RJEz+rvvj@<r{ksQRV9PU%r3b+aoMI->NEJSA8R?J8;h4WB0tUWp(Jk9P)c z?<Kab$qHw9lNgVRxI%I?IOEzC#2>a<Jjuz(aA8}F4wD>jhuKSea!*-4qofLy*Hq~X z^PFCgd^~`lsk!OmOJj-Cc$;cA;0AM_PFEP*$?<Y~AVpMY>^D8*xS=0NCgX4Cq8Pum zjwt`_IQBk#v4BJ^sDL+~_i^s=M)!&*c3i@;m%{4c!yFL8X*B3vu~)F#Tc1Ipmv-ns z3GzuD^`2C;-ZidYM~B)74f%13E11)3(42LJlj+R5iQ!hp5p|yE@=ef+O&){S+!D2u z+TNv*?`teq1(p%EtlsH<2z?6YD3w7Rb+>D93wk*-bkLJC_T%wn4SLUu<HSm1?R9xa zqW>zAZToQ$G_gUve5k&n;-jiVcdC^=x=1azS(YG2_VMEj6n~;o#nt5?oUI}Gotw*Y zukF2=8JQNi7-h<JmVW=A1aas$1f^&~`J+gM68DMh_CuB@=iAM4fx&%}Rv(hGjGw1^ zyr+vAy8c2ZZ9e^8&C|YBuR7fBPwe}#S@)G?$_nXt!XY5ff?=h>n%p+sA^P!s>@y+T zbi3O1S2QKhhLjR13x@0BfxpHpZu!RO?%4QTy;!HZub%2as(KgW9YH4@Q5E1LeISUr z<2OB+a*!SRawdgz+sVEyhzP2Xhkuh0w9|4u+2)0}vUCsRTwV@jhFC;0u&XPYV{d9Q zElj?DQonTGTNRq(cUers)b+`*e&WKqBrOMCC+}_B%NOv#MzsfD8`8)UmY)`vA3tpM zf-4z^H_kXb17WV;WE-vtkCuu+1KN?dBbFR|%h{-un!QHD6=$6>CtQF+D8%?=+`8?{ zqyB}L&C|6JD!Uu8O^naZXMQreV;GT&*M*#X#5~z~;$GJ@T-T{dNmkt(#EMwoa79$l z;BYLNIYPv6ymVyOzgLOt@`%6cCD<1I6d=Lm00>KsdczOIR{>M*jj!+ft3kIiE>ZI3 zdg<BK-lS9Ks4P|R!Mpwq;w4Up03KTGqxi;h*R&<1GTUt1LeRSCKPru>^U|yaZ{%Fv z?c00UcjQjY^W#95FH&1|`VEFi&C7l#q^vARY0~HXFs7b^QF<Qz<T1+IG1P6S=&E8! zx`eWoDHiY#zpc>MIIC|hXs15VO{*k?bkMxcXDAi5z2au&Fu+}ke?R|4@34uPI`IMA zN^^72K6Vn8zTN7r2#2gTgV!DN+PN?*cy2{hxd^XxT1?7Yp}V%P<umZ;CAqS3d5IyY z#<dSe;UxT6Y!8U2oBTbFKkjh4AzlQiaswCYCG;oSCqo|K3<|^X5`s6Lw;X&O58TJl zpSY{NtE0$RrUMa34L4g!&D}}Ge&*CDcexA#+eCFmKC)(o@h8(n-|@^+*mZgR1-)ux zIt^rM-s*26y6<If>B=Z%pw8Y5z)6JEtO;qI$IO#zaX7!!YUqcGN2AjX_^4evf_^Wj zG1|5SEDf1(9!SI1D-*&n*Y^>vJPJz4W<MSJ_2)adSY5ohb1&VsuYX2zS&7+%YCPi^ zan)4afmk0noc6qrIBd?Ewwy#gHH4s;p4$}F$<p$eLT#xZl$<gf!9`T?F?V?PI{MMp zW#6_xn*9Qpk<#RNNboNpSSTGr$Egw82Lc6cm7K6K^!NwGaj!c|qd}YE>1^-YUStgh zW#o{O1I=|@!a(l!o7B)3v3g&lPp4KOm0IW@q$(RV>eho`;FUBVRpm5v5^qb%Z$3!L zZ$2YE_l*=V!by^RhDVraWG7HAe~?!<d#!U>vY;7actp8ByP!~(ab-bI8kZHS9DwD= zL=_{Ag(md*Gx?iOXaPvst1Z`j_gkNhJ|<cLqG9goTZc74b)cs*pzIp&K`VFh^hdWZ z+ad=CO<rXiEN{zBjLYbGJ_|blz9zJQ_nc%)ur*<VUcgzxG<)2l+W`mTE@pg&_`T}u z*&;xvn!n#dYds0A5-}`iWU}I|6zk$|)`lc=8XnPUJhQdDY)5Pvyt8x6z}qoK{?0X4 zjDg^QpdfX~Qa@b9w;kmPpInNCzJ!PYbfGG=2^RC$_3a&zPR(Y19^#b0=0`NfiF%v` zQSRjk1>VWrM$zk8MF=H&@h+o%T)cWXjN4`!#bp?qwjZeJ%c|hH>azr?#aows;z5rZ zYhy9nY@r*4e!Z7=S9`j?@O5{X)lHbyT?|*~1)J=_-i~^HYm@b6S2i8DIF)<O*D>pJ zMno~O3GX7boy3S4b<I_C<&8U1PhD=)GVKoZlm;3E_E2KXWathN|5ozH%DX)|zZ6O* zbH_Nj9f5X567!F-$23yGx?hr{F)i4f1Pxy`)9Z?wKUv;ieOQ5c&9|B6?DQ;BDY=Ud z<x?Iv_<0WV)i3VxnAEpE)joY~Tz&GPJgFSRE_VYibIt3vv0NRkUc^5bXVo{^r}G#f zPT<C3Zh?IJEHXX4C$rG&e?2G=-}MqEJ%-GmJAoC>b9anaGTq!{2B%wr!+Fh{G=<ga z<;@0{%CpxG2G?lj%c>NsY}&_3=CCD3`)GRCgYUX`E|d^~<XqDosm6AGm^eRDa%qdS zmsk3g4uMWA`6A#j`ZAUD00`qxZ_r(or48bn_6*;Nh{w5Tx9-54(;QvRJ#7_ofB5v6 zVz|<B97ic~VMoI5U2!+W3tDBbIibR{(7wmjfdXX04jLr1*B@BQnJYqScxh4+VA?C% z)zT~4hU1i_s~5-AY(q;A?X4HbtHy)T5oO-1%5LGwf!zV2<EHzT4SxI8?DW$~Nw!0Q z(lJm@H4T&>lEP+s0vu}D8i7*Zzgq&WXzeBZjpb8Oik(#)@=;CjRP7{7zrh>jq{b`X zap}}_WcxTgu75Ps^6vYUPU`av5ko-0wDQp)@fAL=O1T$L@10m(05pzdnO~7Om2ZHn zBYOn9$$d=?kHs$WE4LmD;=Fr#Z#?n{9iE`7k%}sH*E;xsBQG}d@pk)>qS7iMGCi4n zCLw{D_X?Bgf$qM%{~E;L<}go<@xjtg3`X{Nz9n&9i(+}~W?WngCX&8C1Byzm@;x{1 zLU`wc9y7^ITnuQ@Zn)LH<!txLc$_^UTPP?&4UaF8$s86=?`|39jc~5wm42VE%b!~Q zNR?)s`jCiH3uNw%zkRjoT~%zgR(v4D6~ZY<!|};`GRaTmN#e)0|FgI&`|72<F0m4H z_vvpQgfsfKdZ)KKY5IibM>R)jx{|s?LG3Y+dDtG>38~Tp!VAj;lnWZ$6NjHr?Sv9f zuyIC;iB^xZ`eWVlXvO!ly1iAxDea4kl>+9_D=f}P(uUnvUE2;74?$zUSSmuCBQRsz z-L-pRB;1*JE7pZPMeOq>?kyeHC{2-uSR@wa{5v{H%#N`una8YnM5lsM`ytR_yS6|| zHTK;Ztv2-d`+CT5F@LNV#<hWcs;Wk;*GF>vPWJJ#k89m-DtA9;C>0OElEGdd+00A5 zEW5pEw-3dOPIpq3!R#u{oRDna?#s5IL#Y30`1vEG<mnlt8xA65mey_+U(sn*=o@R5 zb0j|qas|B7s;>Z;%r4$510H(0+<^R)LH5dzt%<+|O1XGbk`6gy&(0^eeK6*tw}<C@ zZf}?CugT>8eWQ^Cq7m^BMY&xFADRQ$@cGqF*v{QV3hnFE&kZY#N;V&^lYx)12XiKm z@*Br-L1lGfDweXF9+I#fb#J90nc2;Pp~Ls;&2p6imd&%P<tur^pP2ME+5pxKdkt1j zt&VIJH5u&Lt0oloBK%>&opnkWAOd;T8;7@$5BSrvy1+a4VJZwf)Vd?G2DuFChE=bU zN9H3HJnY4papeu&sUMm3Ob%1Rc<g}C&=eNYKxjmRiCxAZN3Oc_mlq#>1dh-m)3dL? z7NH>^y%EUdLKxk=?K4=Fy&#EK=CACaSkW*woL91BJ-zDw{iI>?DClVM1T{+o5@eZw zKAaUvI@s#JkUCIfBNH(N1@4Z1#o7wCb-uR&tW7_ddaU!dSEO>+1qw@L$X>r=etzq= zs_L0m5f-#NQi4ICy2rm0Qm4(mkH1o$<8%(BZL2=oxTmiC?h%gPxz8$8Z<DIO;mOr9 zcP#Z?E0zC4rS+NBqlLrXh*(Wf(xBBZi9E((My-eZ!#3~QlbzRGn^$pzps>y<)$tgJ z^Xe*xi_U~2of7qtq3JuJ3B}JTIy7m6*MU2dX;Wn_+PyMS?O9x(ZF$ao+m35i2QmkT z%jQ=+p0_hu+gg3-fR<-K_@hW*`Z<SBwr#g_BHDj8h~-PJI!SVH{<NLAwBPOvb>sUK z6F@7DDUZ{dJAb7X|JTWs=ETg%I@Ls{mWmwPrsyN~y&zhI8mqG3_!AY7lc|qqY;AAT zQOpZ?PD_Ty5A%bz+WIN(Z4wK>uoTBF{YQp9#gN;`tdrP+OS?3Q=24_O?jdct)i0ZY zOQvn2Ar6-)?lkO@;$UAU4JAxr^3{UX!Z|9J4wY-?lUiMS;A)gJ4V$=$nr~^dX?@xb zB){NN)FbB#*!%R1(m78_Xp}hNz++e*k80Wds%{Dm1GuK%1gVpVTlr9FQpcX&c5$6Q z;1?mc%dT<g$mX{05kXk8`lc!aZtKx~BT>tSo^<VLzgfu#esk3Tmq+HAth;!cr1*n_ z&mUzR_@rON9%2(7dRTbBS-0Y@c8jYiWcRF&VCjJJ)re2oo|G3wqU^H^KNXS~Btl&u z2_8TskN52HKO{CsU4V37#}7UBy5mX(GjWho3H&AG&5Z?jSWD4Dq$Gc_QkCxcu;huM zKNdA&59G5FunJh&V%pf}T{cokXpo0T>nB~1lW$t}236s}t++7jncybAUllhZ+`c>W z(6P<S*=Z)WOR}>GN|JsyJVjT)?C5ldmKq1GSnhA0#B{uLf5yYq*fe$UO#Rq#;|sTv z+#QP>jN;#`oy{6Pv6oq^VJ4|x=XhDG8H`AqSQk_u;p;JO8;n(n-yf#lzqS4y`#ggv z7StA)jK3w&<Wp7Wf1*0i{J69q-u6Y%z!PD2jO=8(qIea;UwZKYxI!cN<XF>Ie8+b; zHm&+w=W)o%nY_u;uNDkDsRcbVvT)`Gw;YMwT|IQ_t>nS&PG=E;&bVUK2cx|=3&tu( zF`hw<e$~|Ux$NK@?ns@n&O0$C$16tN`T~d)Bud!|nkO}pMqsA?nf-DwfwLp!>+<Dm zh)6$w!e&{JMRE#*%$dUgm#HIjTf&uzdwkXyykyYx-R%;~zijNF=29rI!DtzwZ9cu8 z^3H2!0gA5;9r^Ay2kuKXzDCt$v`1@}TXR;q8&k0dc2&DvsBr2$H1X57mTJ-MMTEuk zwF64uwa)2s(EtZ&49ThaWr(M2sd-vgM(BvCPOR_4P?pJZQ*Lp($eXpSa9gWaRdu(9 z->cqs6f5XAjPT9IrH-o%PCunST(39=noZ1kXFj?VP|>S8JlN8>Gm8OfNhYv`9a&V* zuxV7QBqU<VQvjZHY0=MZA9gDWe<4GPzjh2es5jo~xG``F=9@2sCYHOmCokToc&aJ3 z|F}E%oSfRe$}737@qK}93X_8A^h-G@`XL?N_B7k8*!NMmnI2Qzzsk8Ag!WS?d)1RS zRr7BaUg55;?4i$lGsxv1@G={0TqzZWUJgYVqr>e>{yD;w(#*3E?Y9EYe22;9vzal= z8@tnG<YV|)%%_CM(zsP)g!Y|QBNKg@`ZFvqbnr5n17tY8c$SAgS$+E~WIA#VU5`q7 z4(zH#92{%tu*01)&*Yh!ZIA-1Gkw2+CM$mT>~2u(@#JNH!X+q$d`j!Pa~S7JsoDB8 zy4N&zC#n}5I!!P~p0X*kB_oxdZ_H3cKQg{d^I9T!>|tOS{4h9c)er$<F1#eHd34UL zn5iJWUtsvcap`4(Kl1YCTb8q=Hj8nuRZ@<d+{xs8rdw}&u-^{jQ_}RDOii9k?8R09 zN;>9X)bpUv%g~%X`ZC2>!*l(7PHvS+F;ha8{>z@dr$@-h=_nJMC5x<qGC}rjx~(H1 ziJbn?y`JalMu;m$@v`Korqqi+d+*ty;<eVHbI#UVUUtYsYI5ssq3(LR4%fw<se6s+ z!O$!LdA_oT_Re<?>jW>(x(&-|#wx{v&!bhQGeJ(Vy6umVti;y<nF6~A;PPvHHYsTo z3}($I*Lp`M^2LY)wN;j7{2)&LjID7Do!Jk3dnBp_HYa@HK=*VnpX{ay&4R&zFU(sY z4>obiRS(fIQv&DwYpc;bn+QnMC^{>$#Jo?{D!fOHSIW@%_4CnpmL0oXS1u-W;j>WA zdtT~O4w;Z6dfAsRn7eANJZ5JKPFPj;{dM`tBRJ}oSr*ten*|(LoC}%5aam35C#N_i zO=|#>kqENDY_{vgT5lOcwVS)gPz$*WgKNf$?qbRG6aPsW@5Il)#qPay=4d~aT_Fz2 zMXzX&mg--$>rHXV#Cf++mL=oMSW*^SRFBlFt}K}4vEbcZc$Cdxdw^KR){2tMwI=28 zzNUYaPeJU(tA@p@%#lh2_OjEfIx68YmHvWzA{U7!LzXk?nK%OzBpe*12aj{VHZ|X8 zh%aWOeUkcEFAQwqyZ&1Bi&UN@n#swd?YXKPTp}bkgiSW<g*^;=S)F)3)g`k;X*cJ% z%athdbuQ-Fa2J!oYFu1Nor9n5{>0i~%VH}vkoI#!im-rN%$i_{hSLI5mUhScic9|b zsB>T%(0Jc$wWLK`Od+}TyhjwTdBmn}VCjw7*<-cd9konJCi_#;S>;lVVaa2Lb=XsK z<=Aq>uIjBWQHky*DCMk;NjvbsJH6mRqXV>TU2Ua3tL@L<HPxajc97Vbt>spmd;}QY zaWRJ6#AEk{#+mjz#pW)qM%VGWNQ9&4968Pqoa}vN(gnTJRVFezKQb(HLK6E{KXjV0 zr}~pJxnf>yIHKNsMLF$}V0KV~SMT*a6+b97&bN2VCOsdagzNw^P!?g9K2?w=pkfTv zDg>s|0sDldJRX}L0*wvH<=VLWk%SvT{CUijVuKtQH<&=rxUWf0Y8Qrp2BnW%R&ng0 zkUybz43qGd#r)F(YO+hirPeR^pRLvBUx-HZlbMy3B6f9l9(2^z0fEMAZdzl;lw0ps ztV?xpRfpfoV5e=h&_#GQ%RWJqJV$xeW}R;XpX6+W<G=um#I%D2&tmz>y4zF~K$)V^ z|JUAE#zom}?JJ^`f`E#oDBU?A4I<5u(j7w$-Hm{NAl=<5-93nOBSScJ4&5dFAD{QU z=RD_q;LHEBe7JwZ-Y~QF+SgjwUUA=ZO;#tKs=+RyJu)F#H@mfST*BtDlwr_d9b-S` zuKI4V!DLU?t%TFQKOt|)ArHS)vIfi@Ap`B|jg}y;-jT<tFo*ygdgc%E()0MEU2n>> z`mV?mPn^qP5^=JX96RVYmp+cp9Cyu<WXpGHT&`vEWQ_7s;RaWFO#G;DOWFlwMvOEO zW2~s4oS&Oxt`BU@(jEc>g9g12Rew;jXSe=rS4}SBEAzr%K$^Z$8IF++t-FoCUEgxs zZgOgbr(4XllF(hT8$mZ#Mohb#Shtnb=ClrTOblctMxr0iO`LwpNc83Vnsk*a7hSKE zI$oY~*knb-WYn*m+f4$Fwaq>zVLPgHH=P?hCCTE{*O(AD!cVff;s!`TBriiXS+cc6 z+URwP1<bY;7di=w+|LI>ShrgMwbhWgYX+B;t${D$KMbT9O@R@S>&PW!%OxsD556y* zaDZ`Kz*nRt;EpFE$7vK^u3N<WS6oHTRK~@SG^<{s9G9j6H}}}4?QA038qP>DBLdDy z?~_r})Tn_%4#{bT2RUuwG^{j^VO!HTOf)Q}l&(|~*)o*OE8mC9nrD~UNmvcbymD>i ztfc}&s&AB<z2;indG<ekyg*_A#HcH1f`QS>Z(#UtARW<!U6|6(jM)9F<-~kmwU*Zd zUO){Ve{C8GK5ofR5E7Y{YbnefX!1-9gj%w&qzEz}yUp%(r8J@dg*=11zvemhkgsVT zp(QWzsvxgnH^x>yN<P_*OhMW9imx3blsgHvc-@gVLIDzXt@i-*N%tY+v*c%aMjrZQ zXKxi(6@PotRM#(Uz!Zw^@$TMz<X$xQ1UJpWXQ{D{EX3pxsoF2+EEkCe{pE3uzl?j# zp3=$sTBZP$eE1q=j0HBfMr*5m`(7oyX38U}@G+84X}Ity5Q$9XmbG+lN6zODM-xb) zTBjr}^6uxq0x%ma>bnFiGYO<*cuUZU!g-W%pW_UmoaH_`>sP3nw$pJcv^?ymJx^Ov zVQ1EryjaFRSl)*n4D)CdtEnuUW2N3#{`Rx#Cby=v8l1X7TdM(BFU`b_RP5nmyo7y7 zXoOAV!cl9GIvZ{p1cquLUoB-A%HCEcZSzK-11_jy!7~j3Xv^yN4S-3y#7j5YcJm7a zZ1&?*t*2k`1N;P%$T1v$vW-d0O(F+>Ir7)`294Jl1BP|SP46i^INb&W>2>Puu$^iM zRllL~6aRL$9CO2&bus`;j@zYwShE-cGL}9VCFK=;a1Hrbo)B@pl$Kb!tKy}l>}Ej+ z!uPhFfVY3HsPvci*km2RCa6n0E2Bpx!t&6H|5%P67PlL(ms3j3rv2RfR0r>TNG<Qx zHMmPHLnE1m{lzT9Ai(O>diTp|uN~b;(IM+ZWxaASA#fG|mCRxlR8;=;<&lmsyQ=xs zT5mY>g-843j+ENcTy3P^sIgJ6j{7d%g3-8T-FQanz7kV^#X&GisEDx#)*2DW$KkAE zhj<`^=w<YAwTgp>>o#Ax+#GH-`eEukaxlR7Z0kp*2VWK6WoeJ(xkc%<pyk+k=&uc_ zkyF9K3s;7#l{`n<Xj&{QUBVVui?Ha+mKmIQ|L^n~oAy&3&u=igTN-bwLE~#ECMR>x zudEgAh^IJT(MMX)X+t*`9=L!y7@Hl1$!TD-FG^b7HmwAmeWqEUr}qO&16R(SdNncN z9%gIdkn59!AgK$vbM;fJ*!pv!02&t_j_6!2nwe?pSlUF8rdsNf=zWC%s`cdq25R~X zZLI*HcFk7|4D>?FMpEFci&KiB>9Zmw^GNcrIvFbyV?=xE0n9&aC7y(M8ACYciDrVC zaE`Ehll}gceGuY#T6k%ZGc*{6c?qr3Li4Ps?N~|lQDRupC9YbIL9)@UWR$%O!X;lD zs4`S~%3e#12>I4Sh9rxwd=1V`8)66z1r;*2R|7{%Z{+G7&IfQ_sXI6*NO;Igkgd#7 zhD3TQd+d+c4LclILH(ur&<nw|Hi0%c#JFVJeV6T$EqVsaCweKxS?cZWlJ-+Ilcx|I zyq_*QDK=LZl5R7z{3Timc7<gUSf!Wa6Y9Lv0uBBM)>lPpKG&kk8~Sh<##FQsD2SK9 zTi%#!NFBR<7^lK)CtEwmA!qz_wYLGiHmTHlv%jbWTOXngDH}J=Pdusi>PW<LEcfKQ zL>dc1%(u2=R|39Z(#I770uEGD3h=dITpI(Wl&8Vtsu8pKvR3{|(8W=Y_Z|vh*D|jg zv(Ai7>iVp_N5O?_2DPuVvYmLIxIW)}T-^Z>!17Yr=<%_Gh=@7$d`CZ!`E3kd7GAhP zmWg!LPCUT8%qtCT`J1n>YvAO`W%+qxV_Oq^x<T2a)aH=?pav4ADR)$DC#JGJWOF!K zpzC(_;A;Vs+8pjwz&h~h;qhdV-)Z|~N$KVK-sL;*<s?KL%AwwU|H+HEt465EO)B|X zbtK$PE9rCxv!eEUi!9!_#!RBytSTecNm%sn{8i(sO084d&+9mWn?de~-JAJ5JcoCH znVU>n5EU{Kvd_Du3rkm(HxCitIK^zhqN}3vB8k%qF3H1g>w+wk#;>^f1f`mol+98) z`jCrfWoBh6n>oKaIF&<m5@n^gf;0Ba>SUXpXu2jWm^hZY_?^cOWgv{zYDc~NrgrYV z|Fma-JsLyAOyP@ecz-31-2l+mBaE6W?JT3Ins%n3^G4Od0|;tAn#vX)rjgL9`k>h6 zPGRF~8M_e3uwvcou4cDa)I#ExQSdm)G07f%gui*^sAoE@B5V4sX?_(NZR}|MVYXp+ zF)QzQsD9ed`ggJBXvz6wGL+K*$u~eh&|tBq;_*DqI?YBa=?{_LX(Bx|cSa06m6#yG z+M!aj#@S;ntCK1xkXFHm2~UTqZBGUrbE6Dp1?e)47H-dioZbeeh7q&Qg-WdDZL{+_ zCy%rFP7zLRCa0nu_rexNrk%<956R7&bmvcVt7<lC;&x#7;*J0+Gsl{R##%FAO&uO9 zFm2oy#<9`3ue*WOYG#d)$t?}xV!G-PtfvBTE1+v5<-=Kz2*DXkX!1egP-ErU$*0^G zt|bx3!4F(;aF-sWRRn(n;ZpywdX!p`1&S-is-D?n2!oIckff3q51=HDa}J{~Lr-tS zjQHzE&1h{7gbx;FML+k18bGijyx3KI$jI0kE=XK{!BLjUv`*)xN_#(n7mKZ$<9!c8 zcjG9SF6B}+K&kRg<WHks7S7N_Kf`6_Ag1GDq`6V03Y4eNr))v#!VK3CZSTES`T{_8 zmmw<?;*Nr-$}GQ-?E4&huuW<gUW(lH`T3Wz79gdrxg`8WO3a)=5L30@lg&dVy83kl zqkU!4Xz9okS|SB1$t$5vt1DQGI~r-yku6p2HoV>=ycZETLvAnim1(Vb?&0TMqldvm zn=**q``XFd5RzUlhT?De8j<2g9cA)<fPy6@qn$lFJa`F>^FfS}e*Eb4ko$NebKTLt zrZKlkb+5m9#fa0bcCH#$nRc@!0IuZTK+t}g;Nod&1Z>GoV<_mr4uGbJ@q|RIyLxFi zib{ncMohLpa^iYZ^uqI{Ym5KoqOR#oNwG)Oqfm{_kyD*SIw#DUiIs#4+_n9M;-!^A zouQaTCFnARjs~x@P<mih2s4|dti<%^B*S=#xXhu0%2U&M=ct%IPWC!KN)7aw&o7LO z_PJ;VFW=94tm0}x-Weay7F4=A{Gj1xbGm}3jAW8R)yQyS(t93>3|@>Qk7e_3w-#uI z%!(ezl>15=<|!wQ3W$Pfql5dQITo5_AM}r#?H%j_!QertM5zI{7HQfxU35v};2W9^ zVOZp0OiANA*Q4R{FKi%L$4^J>`$w*ts00&Bty!B0h22A%#YWr>*T%XA*gXnJ{XKw4 z2qb*$34vTj^ww}}9bcSmTm7W!hP}2<c=(1)V{L0ezh|yqRCM;L=p?tYspeqFwpi5n z_`2+19!vD+{iRF3gW=C<-EIwCQC?baIH@L2@u?#du0PtqC-Ff2zDAA*&T389o`D#q zbhz57G)z~Yz3g@1dYC5s;_9ku1nQDhPr_4&3F}`3)Qq{e1hQJH=chO>&IeR5Q>g}F z^rD!0tkWQw;hssdo|qq&qK$mq|5VObwCYy#s)_^ytU8y<2OF_d4X%<HP52%N`TC`j zB5TV+_|ifhL#UipZ{%%?d=abz0H1zm{9ePU@oly2$)>#-&kG`QnA?e@w0<=C$>$gb zFlK^jrx8x&^5%K+d<JN<C}0Sovo@iuzQ1(#p9=ub8SVn79@h*Zds^r09j5&&uzv*B z?u5C?P^iKfQ8sQATc8zm^jca`Io{TKTY*m3JoeR9+Oh3Ysa4e}A#p~n`nW>$YbK5R z28V_Pm+MzF2V1{X(b!DssZ(*R6#zuTM6EOsqSnAHtC&{T%DPz=prLPW0-`@t24J&K zZ1rZ}S5%_nks*-v^N`|;3!{U?JU~8EU6%9+;#Btf@(hjz(g`)neR-s{j)R@I*%Dq) z6JW0)LLu%Wyp6?LRNUbg8B;Gcp11Wx9GFoV0-Ov=&R~73Y&F6d@ZQxPnlR3t<#Hh( z@%s4~;-(R7_7a^X5i3NR=y#s|?-yD0_)(ln_W}xO*PyI`3W}1f6tnNhVZiyHh-YM8 z4pasiB|q9JWaIP@+n-nRHiwZ|E7tp7Q~*qfs2DtGXw1ep$5|IoSVn}5nbU@5m<q4= zBMecr83Fn9?73OTdhLM>bcR={yqn$06m;1gTDT4z_B_Buo9?tPb<))o_y;@F)z^m4 zKs66}S<XUEcX_FO#o8tG&3Vv*CQLj1xCzxR2n#EtH{I$pA!+xn#QrY&zgALriDa|A zmGCVAyefW!UEAHRp%d*35iieWj=gl}p%R?lqp|fZZM5_JIVV7<hKV=jy%Mfkf9MVN zaI=qXM1i5*4?19Cadkd3B{0Dnl+1npBP3EaB#xVcwiAMBdu-$cpi}7v;BI}~_dfoz zQcMv?!d%icBX?ndKESikA6a5D7bMV2U7OeGC&hZw%c3;{YNuY$MgeU<P7cwuym38= zPUt)~|F*{X+s*yu>;8S22YJT}34}XozhcrOjw0`rW*KLmalpD%FoKk+QC=!reZq?+ zA1cv#(=tl9;CG>*thyR?LU^_c!B!4kN_FC;_|ga_IaqTJgeG^xI}L6w53P!3eytFD z`?{S-*_uZlB{9pqt9=z2`C>aePhyJ>mWa7XhZaJ}s2I@EyKszQIB<R760&``H`McF z48M_bRx^^UzO~$T004C7L6)$QN!;R%SIIg@XB%109y|+aGHMD&y;jA9Ka$YD0aGY| z`D~b*dlTA~9!77=aJ7ViThI*+lO>F8Bu};~joEoLHV?_zLN*WAemMOf*dc!XMurpy zX%J-{=wA#+U7r+w(5crExtc^wG;8x|dS+2g;=1J~T)I+bHyE!3Ef{O3e=o0?d4EN0 z*Ikky@eo6)R-8PC_G%70evIX|TAgSy&b2okibxc&wC-Mpgf~Liw#NVufNl}uCJnol zERk0xsK;+4^)dvU35gtX_%?Z|CC`s18eEUcydopBYh&i>jE*IG<khajift>%#kr4O zQnv+Y(fvp~S!{0&sc`Sx@1-lG)YmNXz@*2Yf-X0eyiIFXVZH3y@bTiZ-aW6pEH16q z9NbCmzEY?$pP>-xSS_;rgv=QMK6i}k{=BO>X!(N!3DsFTU8QAn`6!{ns-I-@&5Y20 z0eYia@DM&=mUF`y3{~rF?Xb09a65<?cWztW$B9g15Seh7_2gnyK7exc#W#9^s$+f~ z&h=f=NWAH)?X%$IBfUr_A=jj;eY{bG4Y7A|f(fKl^Ljq;hUN{q`(_pLR0V!A!(2*_ z)Tg1YGH@RwUVxv>59d1Cbr|Q=rJ~r)*fwqnE!7{!NeT(U*4G)-+2c%WL!A0PIwH<o zB&bi>OHh2@<Ne)-q-$f0Di7Bwe6o@4g?IU@R!R(n9E9RMl#&J?R(nSDS2Oj-qr@q@ zQWHYz1!o?Gw7A^lw`w4j<mQzbJ-3>oFim?^_3QwkYL3*&a1>Q!!x>WM>664>KAvEa z8}CuO#sXJG|EJY_6y$rZdXnGoY7j!iTf*6n@udu@lhXNef21Bccym2`WZR!^elle5 z#B`67!XZj$gjaSA$CwWb1I)OcIi=bc5m8J|$0HVML|lo~sRo0SeA0gy8bE;+{(g>S zT+;KeL(2cL)KNn3u<Jd#gkJ%!P%Z8t!K~-j6ei|K-(<ek37ZESuG`^0c0~IAe4Xba zDWU5JfgFEwgy|S|ZUJ5LJv))Ea?Jnhb^q5I=0_fCO30Rjreq2O(DrR?+wu()Sl2}Z zKbwGx*@Tu!RH)8}zR@Q+AhgjBfxomXqLI-W$Gug=za$?0`jL(;#Mjq%TgI!;leyB5 zDj1(eq@zPW_JLFv=61(rJW;0}gSxo2brfsY`6fE=XVCwO@?StiaD=xywlyf%rUf$7 zk)2>|X=dzd>C?LMpwrv<8U8-8*SDY3s6#~H4Q2=V|0a~bz2AoDx0v*Dw0l?tKPR|- zQyNbsZ1(F}jVtY(66v+dMQ3kFmHKUU5ow$Ba++LBMZAZ1sVUtO8FE}5+fblh^b8j~ zVKLGbbxqkoMbWLRa}S36DXv77fq`AbuLq=Av`%}n9=zz|17+2Qs=O_r;h(Hia@?kQ zLMdieva}$KQb#Drc&-2$PC8AYVZ=s5k5!guu?4B--!!Qr*4*%XZETmD%OkhUk}$*Z zZ%aW~05%4@)oc-Vnn#<!F~XwQFQb+pgMsNXzMXcfOw|Z1SZS;!4L!+<x@y(Neaz8; z_J-#oM#t)au%(t(2_1>i64xZvamsl$oZG!W3b<PwL7ZUBWS%8LSicU(wpHr(OM>(y zvUOqh;pGiV7REc*SEvnh76ch!n=m6GhY<G5*}nNZfy~R__vs`dj#uA!A+)6KV@^Ic zx*oFEAFQY7>3lkh&V>as+z4L8#xvji!S_Q_WdS(~C>dm0E4#OQy0LlKi=1^uxS^}J z8sya-x>Z_~DD2>G*ZA4)>*M?P?vX(y-V1%mRj}X4ChGY?YTL@9c2-+$yPDFL!y)kH zLVt=<qUTaF6Qwp{UrVA(9;Hn(=I@gIhveh}1yde{(3wJ88tU>@-}(4n4W&G+95ip- z$}tHu-+vg4D;gTHK>S^*(cZSOshk<8XlB^w-{!)~YOS%Z9ar@X$J_|dD`%Z6QF~;c z=+nP>^FN>P`<3R`-=E@}#2&F|M`^WBV2U@>xnb}w?kyU06esunW`l1)n0RIh2E9=V z3?A*CQl6Y!9}473bm(_wcW6}ko~?0~Ze0i&;duk(SPe>^VDA!hDve<X7?My8uw=d5 zH~}@BbmW0SZn5HFt!+65s;OHSs&?$g$yEnrD5kn~N*@UEd4H*!Qizc|Y-@AH8q85M zZ$h8B?0$s1$;iYyDkB<SVK@7fFq^$k%V23f>Z$^JNJHG?osFu~9P@MCsAo016cbh5 z;irmW;*b*t)l(^gaQfycxWvW=v2AyM<oGvt)%V$t3HVh@ilBw*j~~k3ZgSt2kQ@jY zn{ut3U}dIBhU-t3`@?A9a7TEqQ|=LTn=;+pr&~r2hU~{_KB-P=(ee7p8!K2%A9v>& zXv48|xvw{;fR(jAPIK(5eT~ho4X2dPBda6;MhqJYWW<g)Lx3l{_ECt&8IO}H|2esa z%=%%WBqVyws)cIZ`}Ba8m+E2O0mxGnGR?xJaAK6a_e!mZ^C@qUz&88dlnf-g`fx6) z3hBn^$LFT4bCYHw?s2i)CrVH@d}ZbaaxxJ~E5iInMEPq@=XKHpOhDHQGv@1EFS^*> z85Yf=c>8(B*uOK+f4o(?Jo>2V@K59bAah;*S~5WKt;LNHte$7!6+1_LQ0c1#dW6HM zG9pjHW@h_5z+E{&1AnQAv$)jW{{a)tRKcjR5|NenC+|uB*&G>ZBNJrD3R?LtZIKyR z6^tHN$lW2nTKE1YWFygGgYicJn7oi&r{)z+^uo4q@lG;VxaGs(98i%x)?j#x`u!z1 ztKk8)gxPcYGqIZ9#Zzcb*{^u>o^3LxX#$2Rm6C{N0wPOjR$A~D4ozz!^Vt`yi3_Ao zJyt`_3i4`GMjiJCQDsltcjzZpQ9!LFkYFFBex!#ga1D6g8AAAa+Xu66q{Skf{)uH0 z#7T;TzDFx_WqB10@-(a2aep^*3{Q}ld~}x&t;;Mylir&^4wYT@GzEZ+2$B*?Wmu;A zfjhY~@QWWH<MDD>*A_UGr6aq5c?#)4@3=qrWz1W%plM!SnaL{a5xwLnMJ_7CiCo#0 z-NS22tKy&J1a@O1q9uW~P|IvDb?zZfk5d~k_#$;CoBf>*eHSa+O8*%t8rZ9NnuAs7 z!LUURP)-pI15EZq-ch5qcv4)oaCeP<o)=~}3@Ok(wJ`wbnj7(;^eFL(^#KDNlS~`8 zkBF5GMn06J72P+$Ma5hvy~1rP8WUofDf(FMGV)bQXMIbG%7T?z@pf~#cb`m-psPq2 zD@uLwQ?43w<!1yNu45sdO|_2f|6n2kR5?q;cA7$=9E71b0q~fX2EMih<=tnM_gZ#O zy`<xjs=zchdwLrGY;9}i`<Cy)i5qZb&T5Tf``kWu-Xl%Y#?jyXbz<-0r5-HmFeX(b z{%A+vNG2_6ZNAx7^}&K^5`VcTA~z>xZypS1Ct)m3Cc;h({muySL)=i>pGZSr0fzC? z07v!ouX8;kBK7Am;S$z#rnt-%#czPRTso8XpE)#sH%Z;*_5^c~9QdI>tK8W)b(B#a zJ=xmCdYKfBOPRKx@i^)r!<=QapM#3Q4~;0v0BEzp+Gq6YP*G)N4dKOF7gs9Qty5>( zYui?OPXb3S8p#YNQxrj;tYWPK;ow0|H+HF!q@@HuXvL~vM)FXZq9Srvr>Ts-;$-&| zr&(IE^A2%BAcdZ!#DXbT1rrc{v&*aI_ANEU!UFX&r`*7ch-(cU?ciq3-Jlw6yx@BV zJg()Zx8bU3AX6F9vuAZ-l`&%fMXY}_<qxE~{ATODliFr@9yx<JgXNai6WkRs&mYTN zOv~iyT887za7bFYbB>rft_O{cL*RihqY?F^I__5P@gg>d1bxf+)oINk`lPDolrbU^ zgLMGT@xmAdd-bWIINY?IxckIg=?=Qh`wwsouOJo*MFnq?M~Y=NU`^Z_q3$)nnzeyV z>Gg3aWPk4Lk}NVdf~t5i3ZySvne$%8XgexQ!Ko+0r<DAv@-hBw0ef5}tc`X+T`HR+ zLTTi^KQQG@;9#&}WjVFf$(8`+KxRMvBH8#(%*{bSD&3qP!#_51nEIhiS`Dr>M{m?> zHnw1T@b9A=kQ>bomj`!LtLzuwX09`(lAoH28p2HZDeKHbZ4O9Zal5xCrkGoPU42$* zJ|v@Ed~w~DCIEwZ7KBB(R&o_CXZHlntURdjtFSv}eDZRt1C21iUOdvoLm`uf*3l5W z10Sv#$qH$%e(M%jG@;qucMTa&O(<%GEB}ySZOS92#zeP0pqz4u{bkT(vHUgGXod^c z<XD(o%c;>)+me>)uoDx8`O8Qc#oPe<j?-Fx!Cwo>^T#6A&B^^)O$B`OM!m=3yeVJY z?!xTb1R846Rq_hS3~u}7rQHl=&zO5RG5hYW9#BKimaOGQ=qqeYM_Lz_P;aiB=nBsI zWG^0jEstSX38&z%Fl}x+m%wdC)3HMv$Nx@cdvJAsX|TtmecEo2(IUHc_9+uM&10#7 z195Jy1wh%KXH)>uF)CNKC-B6RYFvdI7kRm2!Ka)*Ifc)APY8+z-_oePYNjf+M32I* z4UL)g<>`wBtt-SQP?p{RgsoyHGzZt7oXM!kZ&Hr5$huj0hC$ItZl|Gp-cmT$3?}BT zoT`U7NofZi$UZ64?MATZZdA9EZ3bMcquuKtoyGb-gSqjfwpdTPUx2xTJP6o1?t>H& zCt1}3+9ImCUn=28Bm&y+1V&*^7c!C(JzP&-OkGue#k6FcLBWb>T~UzKbZ{Vy@GzpZ zS6okhX?<89{yxFBIdyZx*H&_qJJ#d-Tw8aYrR|{;kLpkxd)+iV%Hm{TS8{OjEO<(j z&PlV>C0PZWy1ZB!8RIg}&+%6J97{IVIjSPFTSv<eW6*pJowEYxj*XS`w(=-mv6bAK zZk9cKZbb+66MAs2dVhhH?zqQ|yX`jx>(;sN!O*CabPt|nMfSZ?#~6|#zsa86D-p~) zc@!X7_(E@_NX^9WIQ7gu<PvZkk1Rsb-4fQqEN+R(trMNc77ExSoc{1G$xjNHc`9NI zsY2?Zpa4?P7cKIC_{pODMlS53>v5xuOd3s5m$Bf)KexjB{IxaAx6{8TJNvEYcZjK` zs_G$zrJ?aE4P(lJTbIQW(@(0F`Kl&9uL2!D1w~zHwD_E@Ik^A~kux>h%y}tT#kpe7 z$WPnv$DDn8C&&();mKbKBM=>ZqU=ADY5x(6YASmtN?R9KhCV{!EMU#~6jkwkT~OS* zta8nZzNO@n<(hcYdO^-CDBJLYzyzO881>~u28R;a?Jivl)%lHy=1d?k!P%E+fSZ!X zX*<dXYP>wZKo<FvqNHEXD20Y0krPU|Mxl)ViP27%f!}nDMl^G!h#ibibJa3cD@sFU zR!;cA?>5HxeNe6tY(^zWN?PRN>|{F&L!1K5a`H^@iaC>^`bfQqcSmIYEyJC-dSkSr zK<*QLKUgh(H>Ll>b0Nm6$>f-PI^iFHvr8$B(4tC?zCPs=?5OeYFe8RFrb=1;IGJmU zLD##C<v#LPyt>SDt)}uW$Pkve_w3eSG-f#q&UAL*b8(LQhLz^y35ZUm3x&Iw9odXg z&I)meiN-xU%PZFGAm*v6RZxhBb>?ac(8AQc#19FQK0J@3;}aP;<3hRv(0j3h&1eLK zHLVXW&X3i_JXd`{xY8T~&+ecngo)$+LX<NZx8*{Aw;)X8{wr7<ZhmC?R|E5ZsO<ME zlWe48$~+q~;#V?+cSvBZ^fA4>aGL#hb@Vo(ch*46*LDTW-j<a3{SPJRPuU$~k~;6w zlNH8O_LGxe!#sDa>6;QFiY&2(zP$bON#`vP^RpzcTm$Zqu#Q%6_@|O3f$)E?{ogs} zCI-a()ROS-T@tE1L?vkK1sdGCenefU7BTPU=DtRJmxN#M2^px=n?Aln#d|e~d8;t2 z=DQ@+d`@bzMavm-mx^a!Am+FCGo0>{un-ZRYpvw6+@<0v#C$U!|IuAppg$nE{|5wr zK=20yPyc}64+#E%;NBk)`~krq5Fq~n!5<L(2EqK4nwnY(hxlC~5~36A`7cL%`zj|l zv7wEek7uMln_smtiBm*xi^&AVWm`q=i0V;E=kXB1PM+@lJAx^}kLfHEcSM4SSWO2J zt)}@rydx00hX^G#AKVc|-9z$3OnaigxGPRXMkTO&a$Ai2*Vr#alz2gdDB<2AIH^fL z;QxEk^HU)Sq@1VGyr+wW3eG3y#jMhe9zI9A6NSo~*{pZ0_S+NhPGa`!1H)pJ;DL0D zb7vGG1I`j{tg48}1__06tJvx3D2K_U;ap&jSXc?G>7@zU6ASe`3wf?v0$stS7bc@Z z2F7bk+?~C}QC9Y;+6M;8j-gEerymrrhP;oZz99$WF}Tdd$Rxq`k_2WMjnOec@KZ~> z93a7e_n7>Teoc~p*OXW5?8d7}_tr~1lo_(bFe9}28phs*#HiRDd`N9NiE}cHD;?-P zHT-?6;Xjo(C!H>j98%b)<-RB9&wKH`5D=T3Ux0+u&_4F-EvAVolEu{fu`{%4GkiuP z(WZLk)4W%)zXpjnnt%kiZ4*Kw6?T8<U~$%T2an?<8<5cV^{Q&~w<f2!9z3y%>`Q}V zu?m!1C7H!GdBzq(yfc`>-7V|I+veih6K0gVx~t=P7#X(u&7X$(IM!x7FviO~-ndWg zwm6^tgSRLPv@QdaWc9A(*=UVmR-&oql=Eo7M{MWKZ4tp+bb?Actcy$l8vdmKl^Ruc z{qo4}*!oTEH(5{iwirU_)sC!{<hgfS{;df>xjdv0RuiJRdf!WUJ<I`JYUMVqBgO<h zo{{F+cn|^3x756+rxjDT35jn_Zp`Z=6Rb!Zj3{dM=d|(zo>SBF(hXg~l@iz&T-WDH zRe)H=Ubmu`_w_KfZ)bMiUVHV#JU2GM<OgZbrMZ^NTn9_!P{Qe_cJKZvnqbybQa+Ka zmqs!p4BqjMD(c=#B2u$)18G~<g=&+xVNfuA{9p2oP96)3JV%ZHRxTmWf~d&h%oSw4 ze_vIk%cOkUZjsIdy+!faDf#?3nL7PP3j4EeB1OEsbaH3-y+7Ir<9`@@@J}~j@*XKW z&*Z)DK*4zPN3@qW*Roc<*-7>I+B0A%(k-wO)Iw$NO+Y9i|BF;0HwMO#I%g}!<Cixw z54zV;Fm*l}SjHP0HQio%+)EV9dQ1?N`W+mn*S02WVxs6`J&x6vo&9S)%$PkmYKi_f zFGmn^Z=Hr<K+Z1H64e^Y!x?RQ%Q6Jh86W=7rjD*qh|K(RAAWmhxwMA}5XK)pJ3fod z*}da$;t=}Cm(tp}l_d~<bR$79E7{b&e4U2dX@gX0rh98X<FU($pUNgI9IACIZV+8E zw!e3#Wb^3RgOY8f`nMXlE%f&o76yB7x{XDMm;+k`xR7{bzeW)L!<^;S`%zHO03QWH z#Tvj-2Mq@olF^V`2xs2ocV6-pnd_Rocj^povEWW7#qO6go2;W@j@xUdqd9_E=!mL! zVkL?DS4O^qMr^dlQY5-5+moXM2P<(@n@9b^!by^+g}9)DPn(-n#I(0Hn+pE+d1Mf% zr1Wrc*`9fE=-c`);FwBnWh106nY}vg)O)wHZhG>!TFAJ9C9y2_FHs}}pW_e{UxjSU z>mv7J9BnwIB=ak(dIE<2S<twTY^F9Jq+SxXahE2MH_8$930Gccd*Xkh{x46PZ{dA| Zm4HL81G^m7x`+6Y5RrLb@=n+1{{j9sq5=Q_ diff --git a/site/img/distributedlog.png b/site/img/distributedlog.png deleted file mode 100644 index fc921cf46ab12e510d46850a4f34bfd1dec63152..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 76557 zcmeFYRa{)n(l3llurRm}?(Po3-3boC-Q6X@-Q5Z9?(Pr>5G1$<ceiiIe%}4=eZK2+ zcjmV)*7RCkU0q$(UG=Y8VT$q+NFQ)NfPsM_NlA(-gMmS^fPsOx!oh%A)K<ZXz`#CK zT8M}!N{NU76&>wNEv!wzz$C+xl3`WR4lswi)1(+1zQReucf!w<U*onp*PsHVsfb|$ z1CineF*KA^MSEZv0;yUsh2{e_YUm*_Yiwi1gfy`bG!#7``TD%hI{42zGMx6Gvb;{8 z_kZj>{dj;BIxs?jqOL&%D}6@%Qj<c<%0iPKh6M>13l47%X~aRPy8%GofqmtmAk1Gi zUtvw3e|+kC!>wCsaYhDH{5V#-;w(=yW;We1f(pk9CNvF|Zp8vMs}szk8;Tryd0`=+ zy4z$CowfVK%AeP3ON9a{lv}gZ2L(1lG|j`Bkl=r)_=hUvyp>O2?39Yv5wN6_c%`V9 zn4ZotNvSj+@q(50)o(U3$-Hg}9+2&mk_K$Knt`T1u6Vn0dOnksL!IOofs~WZpXSBZ z#reZoE+>DL#3aZ8w2$^!NqfbSNB0PsWTsrD_l$0}c3bDhBoAqwVAkgHV`L1%;2G{~ zFbMP2X_E*Q6^!1cOd}>2>bu}mn8!Dx3>KeopkU-Ph@MRl@!t1ZA#=&2(qM7wm{z%r zE<L!acUZh)H6{!8oSMR<P7A=>KP4``>L+^%MEo4~9Y`)YLzJ>bURrDxEO|O9#JPx1 zS?d$H=MqS^4%-5WvJvu{3Pel=M=%K2djTH_)#mLi4$w{q%8bZJP8hW17E?rzZaGC_ zF;B|MR)VDp1l9XpJawhjLo>r&v4fY{MVx9EdUj!!LkY3AF`|cEu~!y=NlHLWC-$Sl zvzl3@9w1=yFU|NPqX*DTLwWYVWlelogfw$DB>D*VNf=!No(qUt2tlj?IUh6$Btj1) zFhHS#g%xHmfu`)Co<{r;G`_*pg4EUnc@7oY3v@>s>VZoA3WP&ZBE~I*D;547OKc3Y zBorMBy8{a?qNIqJ2zEh?+Ccal0^<|8BDXVkS&(nQ58;%&3`NdsxWf?fT(@Z~>;P&e z0`&lwjlmkEV~D4|yz>uS;Ce`A{T}D8Er?ZtG5vDqCifiAV3_^Yn+{Mw<F5fFq~f7S zTftByKPCA{L^(*HC8?Fau>mV3VI5IWNef2I5RpatnV7|~y&^d&5c_*8hO7tBzl<?A zrcz9(P4LSKnj&5i$PCgLA=SESU~&L5{B?sShm#GhYRqaEYEWu|7UA)chkFjTDje2Y zsB{3f1XXa&L<^x-{Vv-u_S^1!dZ>1xZauJDOZSN00=`7AT(5vGSi#UYQ6SnJoGFyT zC$<2E5%NSpG%5qY6Her-R6e3ohPebHnPZsdfZ-OZA?*c&LoAn6Jo!O9zGPk@*^H?& zPHBL1)OF%@wk(BJtj(y*D4!#8YovF)chHK$t882Ww&DuaI5o+b>zI->-Y)7cf0FdC zJRX&TQk=PyIi5p|L#IPZ>h#1~beVA4Myaw-Wm07lhf!=wTH3$GJd2#0oJ+4Q4%%l5 zSVVCMa5!<ACdAV|q&cy4vTWC}X{sy<EvYV%EE&~tnuP40JMuUZI_k9YwOY2)IzqC2 z{VwHH;9#z8E@8f49^{O#oAM){@+ZOp<)5r;W)E%;ga`YF#!HM#teru`uX+6j%*xn< zkt&fPk&Fncm@Ran+BG)9{ejugL>QPfT(kq!=u~Vpe2Vw^BdXVmvg#|Pixu~>@CC-X z4XPGVGvGy5(gg?y2q##Mn3!~Yjpwy271p`<6&>=eS`X2eE#YP1Dd9pUIb#?}N@~Gn z^_KC6w!b3ENG;jeCD^ln+cwHrR;OOdVwrxZkMv05{$5paTFIa+Sg}?rP?}ec`aQ1v zXUT<ZLZL+gPIaqxy{?n2jNG`sRi}FAypc_QQ<Q7|t=<XpNeoXL4=RtgOS((23x>01 z+fv(2o5D?rr@UwRP1*J7(e|~?b;8ZvksD4AOfpOu3<iuF(iah0k?c+1{!_6<v8IUo zh%(F@T%|~zNS3I>sK+RrA>X0ap~)dTOe^|my$&ab;jJ@@WD05u3EAw_v9!n3z4WH^ zE>>2i?NqhYRnFpcuXH;WJ7zztFXpIb4n6r(@umlMt-tD4FEf%FlcvjhEeEv5d4}3% z+7^Os(Tr7$o%%7Yjx5_pZ#z!Rcxl@a+f`ismhhG+mx=0$^zz%uY)9kBA(3&BtwZ{e z4JE!CMQ#&I)F!l{EtifRe5?P?y3ji0F(S53KDyg7j`tbubKa5WLX54eE#AuS6;wCI zJBT}Z6z&+N7{{3XY}{=4?3-*Oy=1+kj^7=%Yh7!+onD$Wbjan=?0ZHh^}YLLjDws5 z%`11`asqOutA7#Q-d(KkzOf1M`V&p#!$D{Uv_s@TAHY;1Ke|xcT1?M`7cmCO1uek3 zeVzWK{tX`?4bBCL9@z)*9gE_F6-o-SKA|4gD{~)zKG`SyRxU@8ofE1eDm_&hHUS>K zHVRMgwBb7bahMcshAI75`){Y=jkLB^s#<Avmwhw`YR78DMm5Z+%)^FVb&yMH#}^vv z)+Wc;!%_7qSLsI+tCIv_PNB46c;RlLjA2Cb_L7T9O!0pbHPX4poVJs$)+jtE31lBr z@l(CoJZQ*CikOpPaT7eF8HW}2^yO6Kv@_=Enb3C-2Fg#=&oucdt}*5dMGk0Zv|0X? zEVYu(puGSOr<^A$MxGleyeAJD>#QEE&Wv%4ZN_oOZYN$n2uumgbe}wLZZa0`?Ch89 z-ANmmU5%b7CgnUbV4OM}Ejbf7DD7V97s#5)?)sBM=_@^GKz0Wy=>DRZ!<fW4akgsN zeX|jhO5`EvYRkIbWVd_=VcT2zDWPwsuUafKUt2KkRrPju?)Ls9Yfya<)^)@6+>LML zprO7=w27?U*dXPt>ig=%Ds;Qxv1a4GX{p#jZ)ag=e&-fTcU`iN)|2b;lUBCIYi?g- zeywt=qxSCb%gPHsY$ggDzACqd`}sKWT3%PbUPwlnbD4Cx!>_boj4@w{4(^xs_S<la z()-fwBpxIN8+#i+A5|P3MpF~fR)rji-H;Ba5S{UC`sNq)mTcOS7Sx`TaZ&R;9Hyk| z6J}rX|MLC4=iHBT!kT1eVTO>N`@Q0O?cRW?gN&-o9e0k?N#$*&nIHS>%^#1OD-5h1 zMs$5L-HCQC7wV@O_l#VdYsbFbfLpZ_g(^|KRzH`6Uq7t67HoCyI(&=zi?(NwLh$zO zey{Smjqj;n*_1EKxzU|vzD!;=*Jr6`$+=aYH90T2Ho7t%bS-YrIXz@QbspVm-q{Jl zxlg|^9e>e3yVP?aYPP#q8Y^gyVI@)!81)GE(K?si(>!q6-B;^AdRwDv0_gv7JL|q8 z*%?d#|Iu?KIP-STlX@5bSm!KhQ)0Vleg7P~FS$@|QVpYzpr;VM_eT9>wu{%vY4+;) zdh|nMXK_P&bGg(_><nX_=*8e7&xg<a`9bt>U}rc`Aw0)YaM}Cx9&lk|?l}4Rd7;kk zab2tX$=8o3iHDus0AYz$9ZVQ4ItF%w;1I1u32eFz3@A0}xs&E^>hppa(7mlm#E(|u zRB8(eWP=40bY2aq;xU)zh5Y!{w1m}va+3UEN-~C%GvgNtXTf}-v*jWGA=1SonD<du zdb)LX`lEUf@)NdyQ<$cm00B$Mn_!~N7U7PYRk<bdeat=VB`H*pLCIw#$cxRgP}OkO zkdx&xva@0MVr*w<!r*RW4{~h5!1&yGK({s~&R>A;HrBRIJnsA?f8XE%-M=?8k^uj{ z;%voFq9LaU6tQzO0kSi&F))z`d;kK0e2&JZJj$Zt|8xia<0mn9cDCnXWOQ?LV{l_- zuyZtHWaj4PW@KVvWMQEPy+QBfVe9<Go!-`o^j}2&hmNR;laZr^y|aa#E%2T07ehN2 zXMPfr_ksTX`xl=k?iT+$lC9G}(*jM9@%;%SGXoRjziETI^1Zk6C|bCiSZjz{*qGQl zfyNNvV&>xe`~Ls$$^VY{A3Zhyw<jAL^MCgIk0<~1<YRoFz<*5WU()*93KEyV2R_Ds zi(cSEvw|fC7?=>4l&G+(JNR)Hth=fhR`8T*kpDG_uBSvPiJ@2`O(^tD3W;Q`p>S_t zD4ZmQsA4Y+Np4`cG%6Gn6rHnZMF~*jV<A=?S^L`qMVg6QJw1Ph_r6Dbr^)F`np;Ba z-30aa{aN?h$|quQ0PsJr04kD?Xks=dDpLMXlK*-2d<=Er!WR{WAQSqpOG(ThGp*#Y zZBgUD34wpiCF$>m{O?AP!zd(1aoEMek^0}%{lgL8VE$`7f2jOuFpRX4r?EwY|0V)X z!}5&xUrd7oLU|xmjq^`wje`CUzDq{V?f!EbpnKpUoMK#9TXiH7#edEm9PrNYf42bt zAH)A6kpGJk{)p#7Z~DG)eSpPUqY!*9d!ftSaY`gSj_X80pT`F4wT`9Z{mG2axa^iY z+;<?o9b3@%v$?gJub>``z`D!4TC;Cot+QQitJ&?SmKwh;RVm(iI_pk3S#Flu?E6-v z*XgO;>FJU~1KD^m5=XkngKqu1p_D62KF6v><XxM&(|X+2eJCYj2yYGyWo0ALsT76A zQWzP<A~0{bQ*d~j<cg$|;>*;^gFB1HcL97;T@QPBVKFf&zZ<Q_ANF(3ShJm;Q6iWn zu<ZXtMDa@8#b%KP=typq7H7vR6^O$pVo<S}Ww|8h)i@t0A57;3Ek#E~CHXwvO7#Xp zPreQl`Oz)tx@9e`)tN*nVNXP_Gg67%rnfrndscC~pXVYH@Ti`4T>Z#hcvQo1#kyGs zG3)qenmDJ9bHzSLHkC<uEQ!`E8rE{Q&SZQi-KvUt|Lyh3j#|AURe|83vEKc{P_y1N zDiWWYrL)YK|J{OPO(80^8snz~p6Cqtkwf|;V`hsLQb@$Yq39A&VYQoW>C4qC<GXyG z#v&NIurDz|ElGs@p6(Qd)|-zvN8^dqDicr`(I&h1D1+7mp{SSh+ICBO7pw{J(~Wcq z0avbMu{cl+tvdjx3=k_*Mkc)h7_{o{wlPbUj9vGA_1(TNp7px<qVT2K%~ks?Z!m=Z zHIW;zV;^_!*8MmaWV3kG006)<qtvh0KR*kKK*NiNaqnk3?Tpb@UrzvkWN<39xt$T# z=`~C?q1pU^!(lUzbv~RMoxnB~eW^%z-(6;;!7x*`8ZFauKzfrZMWDfq$)H2Ux$f<x zRX!Dal;^+64P*7ZWR^ppqUTwvTBWnJVzu04BM#qBrpq)MdEp0I8^>N~tPzkh2C)YX zvj%|nBU%3m?N}jGH1oBN*5Pa0)t+e>-i_Z_Pru4Pd!?I>Z*G2>@{&J<k$sNk^SB%l ze0w(Xt#E-oMW`A13ZRmY8?)<v*{ii*ao>)*yqXdmr_Avt@q4}q;k(}m`i`q1Eb#cJ zAXn|vMePaV&Zju#Z{xS?Pr<xyr`Kz8O;IV(6PU{Hn<TItP6*!L3e&gi<yL#C)2n&K zc~<T_w>u2JfH#-sEC~V8<|CdwN#*!_&?WZ%_SV(}6pl_dGyL7Gg5YQ-x05V$19u_@ zAM21GUrZnW0#lmgs0L{NWJzYUq70xTRxD8@4tO9i#7KO3y32sYiFP?%!66zNh*9V$ zv|FqJ&oJFX$g!6Z;u`}QnO$@*`%zv9Aqxr?{xAx9fm{a1XybxjgiK`5$&ugNQ_d<z z3qvlwy+n}0HUHfT>2Nf`RVGRUv&W@zFC<{0<Gc?cUeM211yOnq#2r1q*KJ}L8f9<; zzR&B!{`V+z$}N)$@0(wr9Upd6@Q&mUW{TvN^Qbg?eQe$~IHt=#Gg50eRrDxB7adl2 zzp{vd<m+0X^92^)9x0I)Ze=o8w$B98hf3vpq(hMqKH%HCV${Wh=|qE8+A1Ba2pmm9 zy)P<;>TaB|8ym|+_G9m16f^QvgudU)FLfh<FKVqI2BD`&2XwnG$7lmbv+V24f07i~ z=dFOYsv_kg>1>&*?pIG7KKB|2qoyh5DXHqHNW!<x-yGie_D&MAJ71oO1bsV}LNMS@ z=rrpJ4|ub)?45#1#HIWo5N{RyzE{LS)j@+#^byYw(y-8eu3|$$h4T26=6<*8Cb#i+ zu{}H)Gy(HX&vnV<`-^+a$?wMT#71Ts_q^dCxZaeHIuTzXQa9@ZZK&Wr@_jv@4p1#s z9@J3P^EzQKV9V2m<3l2I|0tBsW?_g9cgCV<c`k%VNvV*1-7u@Aa&F)Lcvv3wgh$Bo zXFQ%<25LAC3LHf?vGL_-v9<s{8Jd-f-JZNB=&c_`FcGB5v)4o?Tn<}bTVZli`M>rB zwZ$2ONukYG>C&>APB5W7?`L~4=*)U#|Fyj+unL$|zP?l_1ZE1O4H(1#76><1NwRIy z!#}I-%?>g{JILth8Xma~LTSB)AL$<PIBgR-Y<{O+pogj3SLk>94&~d~Vs51nyg96@ zeb!B?*oh?;i6^)cjQzPS$oG+n8?JT=v8C-x-G}TCUbn|_JT8aWDdw`b#HMJc)9#oC zpQ7V^p&+`&3iV$q!ax=i0k_D7Lq!Vb`gT35ASa{MWKBJp&d$mz=H9cpX_7=Rk?CQ^ z-4E>!H25gahXld+0{tOj@@ueD2|_YF8KCC79~sg8W(d)Q2r`O%$FVjR_}gQR9X%OG zs-R-p96lbCfxnU{6JD66*aWc2{AVBEP8@!(eyk=e3K>)u2^YqejY~iuM1a@_6s9=@ z^$pK8kA3)fTA_mgMxJ=!hKBSi-2~Eo#Jl8_rNE2?6Om=t3}gqPQ0dNVCVIA_u+iT| zmrePrOpTTC5B4mo_QMa}v(zJI^}f{)EY^!mSAqI@pP&o=<kC^pZZKfVAK#<MxO$pq zL>Bt3*&wjLk_rIyz(4ZMGQA33O|Vk5Q*Z&rVUx*au|JgtLBE0aZlt5Y6CXcY332~$ zS>sHNaFF-quVy@LPld%XO%#p@quKIVc!SgiBMo#Ma4iKv_8x!Marg#rInYBW*+Eb1 zer`Ziuc#OZgbV%Z$B~FfrSuW6+Q{|UcL_V|`kFDg5y~b|$N8t&P$0xO!XsjBc=nRi zBlIpbCS~aEw=_Av$j+z8+uyKNa!5!fNGJ|1zc;(wLk%frlV#eV1^;e&BJe_AE~b9N zmGjZvax4nYT%ac3fV_ZJw0gMRohZ(=^9CYy&(N$D%k{2K2kIegk7b)?obirg`&cC2 zR|$Y0T4A_x2<b`A-M73UzlNVrIQ|X<!!%Z(Ao%o|)4o9=g#HqC<&ZTdJ{|U?bzcuz zws`kcZJyoP;f=K+u$jKxh5*?MZ-RnJAZv5%;cLPXD`;N<QdNF$Ry8qcytMW|16hKI zpiV?K(lKPM+nMVzBx4&Vmb2&;ao&uY-nax*XGNE9HOf(cLpH<PUFgXS)39;b>Hg{% z|7}xPGX#l1UA<WaYbE>k`Z6hkRPT=d?(>ewLe%i|)5L^jz#!qKYWFMq8&jz(vj~s` zqpIU*oaKEaj!4DmnAj#KOi=&mhu~K;%H+i_aB`3GnVuJC&xbI<PO(p!g!&+-l9PQ~ zkj47b&;G{L^T1(9LklJkn1%N1%*^_eiIbejxX}|o@MFJd3b1lS2t>jA4aixA`(u(H z(1t!S%uc4OS=Qc>)UnqH^&X|DEi6Lk2K^#vdv10Cc-l<8{V27xHO9~P+cDzsH6{)d z6lngR0Hux_FN%TX+PPhiv{b2@GEqQ|KOpnr@DZ{GBy#d4*4`qS?sWuxR#6eGeh)zc z5&=CeL5;0UVL|uEEklOS!-by<w@F#V46TFMA*$5iEG1x=%YnTt0S;)4*DbI^cC?{W z0VV>O?zDpT+9G`zw2Ibkt|yl0tbF~k8svbm2ER3zBG=~-1RRQn4XsFaC#KhaUF3vd z$?Vx88LYEqd)_E0^EfF|Aw7ABr8NJ>8V#8sHN9!m3Jl01n0hm=7@#NL#xr?*kEop+ z^%3%Pi-S?YK&~y{^hP@L;0IGzsICy{1;hv#bYdz@4H=k#+Y(p@9+2zOT^#1xD>D`F z5%0XYN~1QPIv|AhcMuVud29fTz{=VSoKn($BE8*HB$}k?bd@geRgHu!>I#D*C*kv- z7~Se&6?o3S>Yf1C!+s3%P1xP~TOw;Q0SEWy3xxfJ*cHs~!9bXSa5+x9B-D<H{z`<v zlzSmeOs~op(NCBytRmU&TkySxld0B0EdJ2(UvvX|Z0MCi;)t6Maddr_$T?xfoF}0Q zZ9<Ve4cy1tiU%g=ENxah6UVk43^}*5(}GrG#2i9BuJ*3>)OhWw-fW7aT`~dQjmj@T zj6BoecCxEP<idX$+;ohI6}p*&@0=+iB#pQ5PSm3!7+;pji^F1u3=J5jfyRfPXYNtV zNXTWMYlg|B!T&mZq4-q{s@{7fGAM+HB~G|k=vi8N@R66a3kGK_+xt=~p$wNb*`Bxu z63ZsU3;Z;|cS$27aV=u^J){Hj!vW9nPg`ryg<AceXjyE;SOoI%hfIxWL?#fjan3sL zvWqo2`pJhrkqPv2Ay^@y6Sgq*1Be)21ZevCV{fd_);fpUz!&t+;7K*+&RVvSr+i44 zLJR%k=8VQQIJv3JQd{~5aQUE^lwcI#sEicNV^v|Q?A?<|_7dZ}-c?0EyCMpz4g<WS zgw%xz%G8l}+yw8yCJ%nc(q#PG(*ObwIux9tk@Y+dRG{Gy3JPOOcyAPVZ;YaCMTB-u zTdl!zioUN!;KfBvgEZpf)$`R<i0aqCUtzFZbYv}@=wwu%N63OhC<wWzF<SV9(gM^F z7|`sAf{KnNIoH{6CN^L~nhb{k^HmR5`>czsMX{Qoa~8M?4}4*nZw)kv4E{wHe}Q_z zfx>t0zd=rj_6d7-C<<>sO*N2~9*UWmE5;0R3kOvqNcGtsAwtyPqW5YmS=*M`>X!yN z49@(t9+e@ri&r#A91WAn!V?iUhW>^jTrv$PQ=AQW(ga&!v7LxhnHIC()3|WF_yuP} z{6*vpyM9(4s4!Z+Ys@#OjWT~&LzBssEqbPQk>1|^YM}s#wPKRr;?rb_q<rX=7-abf zBItpQP^`wT>hz*NV~cYK42F<CB>Sz~BcBY3Omwk*t%#)jyViIg;W!2Qej$KgV;^<a zyFG9cB>&0cbsKBl&&HeHI@1@J7RV+lfvnOQfwItTgGh?H)@XSZ(?S{BTnerw%7%Fj zWP8(KV(ZM-e!X$h-?lFuIiLD(5I|W-2!k#QNv0Q?E*%x>k5)3OswO3TEZF~CLIY4M zxSva2TZfcw8F*Zzb<$aWSN2|iY#z#QanbkjK)Z99#I3JZCn9Y-o9XeSwUzUWIjU?y zAw+N1a;@OHZsUqtz6jMiSnYm0U2-8Mf7AENT3XlzTamITa+v;hDTUyl?$g-S!j3!_ zF0*aS1{^sJhdjCZeV4f*o({*w4zw~)x7--~FRUdN1_C*1JM#10&KuizvT6!AuF_fx z2L9w4m-0-t>g>!+0bL!Bt8kfmp}n7j6fstrG1^ATbImfDYUN|hQ7+ErO)?=g_#Sh+ zC!yclKX)@Ylt7?q*A?Q=rcOes>1CE-51L>4Z{so^%-_otYpFAp`{@`_ps8|){YsS& zX_L&EbI;*&LHbc%)vKr_Z&6{C33HNDbmw@)uVAyzP|O~UX5(TJIpK$ka9uqVxVIpd zv(0H8>mC*NaMzDQ-fcP@IeVPy@H?6>9WDf9Q6QOj@oD1ykHA5a7oveDEBUqkl?Dfz zEiGIHBi6k!K2x#i`?Ri`l<a07pqM#~(#58(FWtE(@588tnsx3o`rO=eG@l2VJVvA+ zyNT1L1R*p1z1@qakq322X$EnzYH8?aE2-YX3bSE0ZkY-ip$GOOY^bpi?QNE+`LC?8 z33lD|>9yiN<|A!ljoLO$Drlr1TnO;$K^}W%5zg2p1--t#Dn?01Iy!N+Da&lfgroBW z_GtCe*rhoHr1T?NE=j_xtS0GyPQnz`-z%-^SCUr1Q#xtSC1h!3%z_DFz?|Pv+Jg8R z$QAt$MT9_;3<3mwC>?+r!bOrO9TDws9^i=ow-Wxnxd-8B+d)CR|ENN&51auKIbDPF zzkoYQd62*6NWu5%A53m35Db$5pTk<C$s7UiFIKk{1&UgaZ=-7Xm81-EOf_+s9i3I1 zvZ6rP*T@=F4G+)XkVsj?;67A|`*)d)7zMi@Mn^DI2OR#4&cz-7w0^ptNBQ-ESMdUl z?H(CKv9klm2Y6$S0Bj^fS0iYL5ATyNj1{V0^V!LtU)??RI!z*#h%P)o{6j^EC&fyj z??)f_OU`AJxS8R2n-N(TPSP2iB!(-Cq|H38{S!2*F4qWYn`OBO7WQRmKXE>Hx7bx+ zdz^l|5cm}JPYg8zp7E3U!N@|-8`j~MCVX1;Q1jn>ZZD4{yJHX8zxXzpmz%;2Zhqmi z53F{yjzXVm|Cz|Pz*A4=zteFz+LBS{dufJ+@NSD^)X{gw*HBY`+Uz=JSyyfYREjud z@pe6ki+;{<S*WD7)fI*JxkK$-%Zs2H<=zi%mqN=?LDn^ZFJEh$64WpB_Q-Ks8O4QA zG(2r9&f(**{BF@B*9n7Tve)=o|3^B^MHP<tM$%{f1Xr77NhAUpDzex#sFD$5ijx(e z>I*cAvAsz=liLc-Ry<MUKm3WQ+GqGQ<T6PK!^0iqK4*VOs=F@h(5aQlV6RVQvHGe> z6Jldq0HrjTBR>g>8MpHhz&#?aPTR@FvEV+CRb}&&Z!Vdk=`S`SS2W=&UJvrU?xqZ` zwl#8}ct1OY{S*scq1Tq~s7l0@z5Eki?%t}9$}FPOyq2zUeE!AZuuqcttYh0NEjh|> z6WYP;_>UL9*Sa60N3MpD>O=UfPe6Xr*71y(;hNW!HS7}x2*5FA*W3?6!5ySU{va36 z#bFb#u?rM@HAf-$J%o^jpW>&idV862S}8@~8<b5mMDJ~x%n(~o=i4_-iW0$clvbRt zI$h--ZjEDydQQS#4)^gpiE}HCLZ@L(mb~_hCS;W_Tj*9!X3V5U5nPNJlfCvUQko6Y zZ40~yMcKPWDSF}Q9QAoWr8dHPMG7|u35q;dDJ}>233iNhRpS9!P#sCwXM~ZSZ2Q!# zUv$Uu6-jajgys|$xNWqyTyxixXu^(~mIsO$Ur!Mj*~kr@UDWqWlt!i7t2^gABHL{{ zlvMfAAGT4#I$u4NmOt0kQS;<d+-vCi{GwuR1C*C55IL-to(h<ZVV5h@p@S_GRflK& zmXG;URvWD#cyrw5GN?Ywc#`onMvE3x{%e%5zgV^65A|97#nlub;+KBjKq3u!RhQ2o zyMa)CqZkHLu^t{;XsK4Pp;~F4u>wSIB6dTQ>XK8DQ}ytHX)=Fz8+!8xY1yT+s3NVb zA;<lmedFf=EMpLGLR}St6)R*3$L0Jf3NvjkcAK4=c%yP)qs>fDiwXpGDoA0_Wyrxn zsTfXoh>lNWagR;XRgi3F3h-Z)m+CHi50&hS%*oNj_F&L#HZG6}Ck5U&S`E4k6ND{5 zO&_9V`~KFz1g!aLZ#7w?hN&@f%CX|qTWjW)kqTfI>?saE-jF<8O08h&O9uWhB$v&m zbFgPtaiiz<P|j>SBW<Z>FQB-YJuvwaNMii>t!vduVXNp6u+RN{I4UDmp(u>Ewr2Vd z{};`GZ{(YUSma-JB*WgGoeGTe60gE2Bh^OV9EUG=0jzFSZ(mN$m;)50BW<(}>M%+t zG_aMPxh=0X3I^uv(%Ay?d}_xg6b{>@i328pJqB+RxtU&Po7Dpv08%|2A(NrL(D*4( z=aWzJDyE-kS-kGTNu}V!LIxo&XmDw_HEMb;;T{e=S{JV4c#6<F=nO4#jfGrKWuIkt z&r-RaCX%YEbj2)x@u<lB%pVd%s&at$#UvSiejov1VJ!*w`#GEosYbZk2^2~Q$wrg7 zDq?*J3b+xG#R4v~GVXgganBD?0JLora269bLijCwuIYifRD#zrM~FBo=B7ARQ@XG} z&tCy5Ra$5l>W0<|xf8PPuVQ&DMq%fy1#mmFQmyIiR;H>q6`Uz(aNh^O5Gn#xP4o9r z$N=O+w-l7rti%{;scEcHH-kRC(18Z7SUTOR9hDfE;SZ|KOi-`9bA4OWsi^yZ2Amrx z%=xL;22}KPSwG&NDwe}<Y09#I^B>s$o>L)X$3u464RnV+TK1HU?QpvcC`6mDM69Jb z#iR!q`5P1k+)&d8Ha$O8m-b?m=nsa6EjJA_`VO^}Tg-slJ0KIJC##?Tb|#4hakVx> z#o8?RUD(xaN#r>a0wOOFd)XUg*-(P4@j@WHZWk2z{TR`%In|K{CIXD&6w_+S>h8>> zL9Zob=1+W8HRfZyUu{eCT?TV%;e)Qr1XhtXDUpWR%*z_0y2r?xS~^HdDx1^sc@M35 z*AJA@YE{r6GpiJ*tJp@R2=hs&4-9qRn?WL|mzu&kk+s(2&_>hfR29CayXPt<7aF)w z5O6LHTc>Reo+O_Y)3AOF>AsPu)CX<JsZMaOA&!!xJP@TyhSpXxDdl9Gx88|kk<jsz zf)dfVXmfz=b?5Sl=RcgNoBGjDe{u6HIMMup#oJG5IIn%)t$XQK$Hcy-q<<+~IR&zy zsm3`}=+Wc;w0sQ5x=eN01{f*CGT;cmakvi6^iV3plpAi@&Wh_auaGcgiG?q>Fy|&g zCu2eVXy<hSBrM4uMfhFGDST%-vs7uQgolkR(bi!Rt_aV;E|0i{wTLB%z{>C)6MQ2} zzC?JuYiB#ITBZf}@8{)jTu6!r0q0()G-@5u6t;m;+GtpzrqNEfM`kUQv;&Yh@<4i! zLE$Ii$dGvXeF~%2FR&1Dt23YvVPSQ+M4#N>Oi5x|gyxfnpa%NiM05=S;Lrh*ln~TP zhR5`^**|MWiImc6h3ZDlv_R_yT|R;Tt4KIsDUgO`Ev><Dj>X^Kd4Hpq0!T!Fgnk{< zsIry9kmMefC#BfTh8D4vB(R95*hgC}`AnH3`P_c?p(%<TN?I9r&?W&OgnE^rp;mQi zz9JimbQ|&sW*0tRKlh;Ec2+CYo~s0pKt33)CF;PFCpHyDP*(l7hUQ1XLc`?~eJbBA z(Rv1?`B(?<hoU!&Q!}Q3YehqAv`QAI;Vidhk>?3<C_d>4XnN(9GF}+jT$Thw%MKs_ z0v3fZQ7VB#OgYg+5=a*?2Qo`#kh+TA-pitZYz%5_;-u4ayw+&=IIrXZv0~&84;?d+ zw*k-Q44p1~TXw|01gmY@jhgbu>zp(;z-jx`%8$GcYLRNns$Ya}8AE8~hsfp4Fo6_< zEa>oZMVU;d6KMjLesL5_iNYjce$9+d6oLBbVUtO7gj7Mub|&HpxLgh>F1k)RZrQbl z&=pFRJn)J@V%%ntQ9g)+(t$;Fk+jylVbV_~Lx54j91D=DqLfT%0G?$PAY{8Uv)P>7 zbU}@D+~pI_{>^aY<8Xz=03%-J`z7v~t6M##;tBH~+dGEy2EwszrjLCHU5zx0@gp_R z<Y4pR_k~bkw>qAFJZw=;ghvc}YTU}$c(l<A%+_s;-7AIc(LWPM<aI@cxUEwiCi{5U z_xR#5SA5%-n$#)X%GI|H*Vpl{o|QS-62Rr?*=qz^2|Wg!Xvp#4EjNYtI71j>!&-np z5p8UBIfubH(F<$mHLwefISz`0RFC^e((273jGk_R<nCOZ@c2ImFnpfYs9o@MNr)x{ z5~_HGSDzVde=3csRbXDwH&<v=^SW89yu3|x2*<_mOM|>klix?x;yNe@rikEF%ovt> zRQsG#M&8*RC}R^>+eFE*kE=yeXq|Nwrn0k|Rt!LYEQu=81ca_Xh^L00pd}Bx&+zt~ z34t&%1!V|tnFLwGdTjVTrI*4RjVJ5w_}vDul?voZeWG>uxG`;wrmAv`U(#b-D!qz} z%{#?Y%2;N#*D{%PKcG`>aA#+6LeR2EA+WXaxXFJX$YoW8%2YHBgnB8q3Q@@6NNbP( zg#%n0Oz!>7Jd(Kg^+s~rE3#x67o8~K61y~t9hriHI?5UNO|k)T%iB=T=NBEs;U`?< zw+XA;;5>mF6yn$N>H%(V_xx{?MuxIFk%Vrb?A(w`h2yRwJ2pA&7&3tLtE>@08)mi~ zcmPS6_3g<r?-Ig*d}AT~i+S61hn?s^qtErc@xiYn*DMrCXVwO+WSU>|wwJ6Ce^sS@ z`fl?AbS@$Ken$Z1)(--i7u4gkr6a#dnheE{JWoA6uF}Ff3%l4oe2zk4HItW*BdPa( z#ZxX8CokQ?D)8~B_f#qv@DJF&KA5$&!{)Zteu>zfCPsx?D;7bNPk-2mK>=4&)Kf0N zE@}m@0MbLvS87u_?bFBGx}ik1q<?wg9Y0>GSLKw~!K2zj&}0&U=My{-d)RFjED|lf z6u6^Ssj6dZZAtRX%uXx1q~2(D+PYU#vt380#JstQx8tmiqh{;`l1nF^^Nh&>Byj>> z+J=_=!6A*swNfA@muxh7LO!NeuHo0$D7LIWiC1)?tX82pCY3KlloSs~<0o?J<C6HD zF72uiGR7j|vBfI+9$h`4Z-KTQe=MmS0k@G{2RjMx>a<6nHBS{v3tV%L*ifs{$G|Ry zwF*0h$7V2#n4ls7`M36O)u@c@-nH;kVdRp)Ao#fwJ+U(NpYX_nY1~J0rJB{X==6GH zXR;fsYBGD0)_q?rRJ-F80)|}LN2D6@RpKss`?Amh29dHK0%}`O0;&$f0*U-Q<W|qz z&Yp=K#rFOXk8V4U3`<26J}nIU07hsaxb@>kY3e!{L9ufa#WW~c>wn^9_M;s<29`H; zvAE6UNE+lc24QSE03_MLsaP23P=)>p8^t$duN~@C`2JnX5t{_{o^z+yZkiaQIA8mF z)Y-r*#*H@VBjJhvjV@!C;ek$y)q0(dyfC*p?}01?sAK}|BLd3*5waSafs%=JSIzxR zaeqVD9ZQI}mph_`!;xPgl$j62Efwg6Pd_)g|0(Pk`2?Y8Z2BWJ?Hm0@<zEEo2}C); zryH*+@^8QlOhGev+|_x7?Kw>Vga1kfg67Exv&sHdn-EYz>W?wEdYaj3>oWgyHw#1{ zKMNG}n!mv;`TiXt0AI%cXk=%xI+=hJ93Tk`8hr+wr2L;6l7L7cm{{<?BIu8)ppHE{ z?uq}z>2;Z4^emdM7&YJ^i7dT8w0_>LvG`Xo-kJdx^x|}PH?{W86a|PWHll3uf0&{p zjipo|JXmdaOJ2?4e6KVKIF<zk>#nb^iyZ&rN=FV%bDYQZIJF$7=IWi~$4Jls2FC80 z|Ai7r0+-*VVbhJv@sBi`(Ry%M5-1gZoM12dz0-n|1b(+!Zep?D=<%mC`;ke_6*DCS zICucHLAR}}VBLi95}h0Jhgw~r$$rWj=64X#Sntn@X*la$-0W)cYw3dJB`UMqKt1$s z=@Rn<JtQ$ftFW^$xf&sjf%6TND!J-@zKm#f9ZRGJXIyW0tIp)}VA*JuL}mc#%N%H| zz^8m4a|s~4n}F)NqCVMv&<#I6o)?Hv-wprDgS`)mn?QV5fvn|DPXHwGn{D3>{d;Tw z4G3ON?tXjaHMlx?@Ta^1Rt+jJ^>WXlO4?5~WDaxr9|c9hZvI&A`gY`1bJfE6mzA`i zcc5y9(XGJ<NRFvG`1xkZC8!D=l;A_&DtH%xf~8#FYe9{5X2Q*9Q1%qkm2t&GCnCoH zaL9KWcGV&Q47>KO0nuycCn`I8x;sOeYY2`0cF0Qu=OQrE1#4vXl&r<E*>z{XFLFb# zvwrem;M4CRJqACJ>G%4e+-UjB|8p|6inu*t4muzd6T$+=Tt1WQ3(O`cVa#serjW%$ zesOUDJ|*V(G5q82kax*!@%=`X?8{P+S$7cBA*wjrTwJ{BeZ;If+x!u7<kfOzVS@JO zsb*L{M}Qz+x25YFFHx19ObP=#ntU(e>1~Hi^&2GxQJ|b`F5Qp3XrVx_@4L_M<u6sk zW*~X5-GZ15%8DrjEv2#edLCU54VolfhJjB+qFip9108R@@H*4Q$Kx}{!@1Hu@0ts` zm4^md?~Uvh!aF#!xNMa9W_zDz&|G~$xy68V%%8Ot#zy}Z!R(A%57LcWfSD>8+Hvnt zK|=?qnA0Ll6q!9B`cdRvfdV2?YPW}?L|(v))XLQZa|~uWz1(^gm_V8^*QsRIawU^Y z2k&a-#5eZ<DjSi}0!>-F<!bX0^VGYfso*N_M{(oXW);c;NZ$SLs-ozlK#NCp1N|Y> z=Cw=ts@&yhekNBKl3KUTxo7;W?TDdZ04hTV&dv9Du@?NO5tfu&KUWA`du1QZrtfC+ z2P}ws7f9LPg)1@R=IlPPXy)@sFe)DgApcYcztH?X4E+p9{xgkM3kJm#4w)z%dEd@> z>Gv<*z-n_$hmTVDZ3R2!)8uOUQ_&C480H=%A5Z-h+H*90{EjM{kLi5`=o98{SL?1y z=d37k7d2q?COI;i?0Od$nK>5SgQZp!L8^9$$S#F{@L=%f#6@TszXTF9=k*JyK;gr$ z&e;7=Hw0M0JVX}jCwJTHAD5fw>S}L-4THR3-i@$@A^xffiRz3#4y4+5D{byoB4z+c zg&X(TT@LtIf8rY3Rh~3nyEx}|FY^J>;G44w2JHHs<9d~9FfngdOj*`|s#V(r0hFku z{6QHp)rH}+ZiYr0jD#Oqyb`L}-z0%WP*tsaxSBNv0l*-fZ%{KqJyzA76qGAsJD2L5 z;;O4TUcPbP>=Y6~hbp#`2j-X#ghVucoPf`1tnl_)^!*Sk=(ucNo0CfCu))1xU#bO4 z3?AQC*2w*9Pb7;K1vdX^xJ3C2bM~QzB7OGX|GTAA0u~!A5?XYsjCCVoVq{$DcEF?b z7wgSptQTvDHEW%w_&G~t)a&{{XOb|SP+adi4Lyr6*oD0!MEU3HWbzy`g*k_f9>JO8 zv5^Yq622+T2eRJXWMKuqv|~LaJoZGZ-;GFz(MiV;gunpj_Y;EPOC1ep*Zo6W^Jkm9 z&TU<oyIn{)N1U_k4-K_ON$OOsA}^NSnmp#Dht#XPU!FoOf4gt;J_S^a@HO=W0RT=I zEL^t6|LtSS1v<QB0)?FMT#3@mRo52*HbcWq<xW?T9HTtfAJn>fJT3>-nj6SZ=-RhG z5vjEbsgkgXXjVpL-%Dx|IL^dC)mAoeh#r^lt#1J#J9Qk*l7lpjDyvoTMPo8CPuUl( zud4?}yQl4^iH;yp6Mr8WzA(7we6xS>!1?>0vR=YCJkTHipnbX7j&Z7Ehj%GnnQfOw z{i@Y?Xs6flE~~RhYFk`jxK#uqP0F8{B~2>hjsyhmszXum3knOooU<==UWGVYVI`){ zf7YX;*V4(hb*al=kpoL>{iB}T{suGi`zc27RU^BlSfow!Hg~}J%o8mswZWIULtql> zleV#QC*m;YdZ^KpYg~cEpl)e=i+00D>eQX^JS$zU`3m^eP!h3s^tCt=(h3@Ql4EM` zsDFBsyWXjdzwWb7jb+}i+gc}?M6;#Q$}(*+#mvdVWX_}daHZgIyi94^(Cq0l<?)6z z8v;CX&8=up|M6+A#9@y`m$SFlrJT$6QL510!NKZK!7W!cYV$s;G)^0&YMP>8V~BLD zpIr{=G0NwqRA~3)@knj7^gfYO$f0{MgM19}kNzx>;}fA+Sdv-*NyV_yP`eP9@5w^; zV&WM7i*GLq_*M&#^SWh;1KpS5EjKyBst=)^w_590zFw!nNaVgRj{^np@POsg1SKYC zPpK{6X-!2cN*ZR%N9Q%SoF(Mj@tYpV_~~xHMX#ZwN0!rM`ay1wx`9X{yCw&^)Pcj_ z1a(FdHM3i|Y?6RrWcGcdHE)&7yEWR&bw7dK)dgM3i#GGjoyiP$K7g+PyM4&tvaZ4q z7D)2%E;3q%b%(qYqjx@AVU)%rvBZz=Ql4EqJ3s2|aHN5iV~9zqh%cTR9yD-JYQ~vm zY}n-D7)V%&7>&6D-sb@$F@n;^HqBBick2=8^fU~sO@MmE7e-KW%58gSOW>?ypx<(^ zcFUh7PaQy-9x%vllTL;xgDB**Q0DTTL6@9&FUw7qnM(FFo*b?D#ZkAtQP<WcsJq3b zwq!2!sSjaLm$$XLgVv&#Ln$JzUV-RmDPnJ@?r%bk3d1!&eS9KcsvoGNWWqja#Hbg% z`<ZiW2@9YcfiYM>0~M=!SZm!N2F^5#dfGDlb8dn!JP)cQalkduASrE=aY7+(oFpmr zI(Ld^CPPS3jKcK6(W%<K(XwT^fq5CetYIer;n!%w*7P^<RRSK#K!!5oL5E)GEk&^* zm+h!6T&`)tTW_9X1%;L1x?D)7iAbFI+<I96o|Vo#V513!+k3HKgsg-e>O-(Z1xX<+ zAlp}K<>t^szR0EjS5ZXEX)C2UJM%a|1-s9M{<ZV&N8+qzU8-jNK&|07$#@bp>TcE~ zdr*}QupC~)<oj-8XQ6Uq8!Efm=}|$^jamv^QfjDuts)H^P#{=yTp2t%1&SQZ1O}lJ zIpbML^nKBCsXRy`;Gs}%73wUxi2QU6L161nj-PC%)Jm-ppg0^cF%^Aut#fPI4`1>b z&yH+p$-E*`<TGtFF17YN+pk)O0xlvCQG5-~PxLbIhn84whlv9qw<N8qkh7bW-RYe* zNO++%%~pf9=UbUSsr+o<(y|%lj75NifOXjNU^K8<#;Y^n3p|}xgJkf^0`-Qc)|y)Z zU|TL|c`52$-~yv{{z86cF@*8OpP<7!?+wG!*hV#sm4C-!55$<Cw`*5&*z-H-`gNA} zJv<&!SB|L&sGFf(&Q%;-p)EI6?t4pfW*+YC=@Z>X0g97a*^psQLM{QU4*3Imj$xZy z$E&<br59~qJg=_G(lbDIj3c}85VqY<15ykNhi{)};t_4J%BQlRjssI$>+g<T!~t-{ z6CmT2e5~N*f7f=|^EOjhR^b=3EEB!8DDth^;Q=ay!`CE$L*Ivjt;2cxi+V4&s{?<8 z<sVt5yo22vrcG8UQ!)MHH*|wHrP`T(KBnuB6tp4kE{sn6rL@cHmFXItT4jc7o-&5o z4U!Ey<g09*4@E7E9iS5oC<pb4npM?s=4&!maV(hbxH4SipbBW~nNYdYPnlp5A*T?- z*5rzKXKb3VSqp!QJ(OR5weS2O@G`HfExNjk<$^x0X{pw;k&-r5gf(k05|0wum}NX+ zm`XBxF-?+ImIkZZfyZGu=IPb?cVeH`4|E0hTg~;h_`~FxqAB#5!6JZaobXU1mmGN( z)0aRgcqFTYt6o?+KkLowEwiI-P|{jx58%&IZ@GylU~8&q_`9i1);w%<<&30xJ&Kv7 z^@R~=^wh1G!0tf{ng~lWr<H;r1ULpx$dKoNo=4FDiA?IyfWa!ZhAChUX~Jcke(#jI zY(kWVIX!DaI<o7Ymp!S_>D(Z$h06(B_)QT?nXU%?;Z2BJy3%|m!1yNbmCV61-IOUn zI}KEWH}I=EXW+m(I8G#FAn${`=ao!FW;f{cuM6xM*w1Dkn*cc%?99xHhE*-kbrt3x z&eCMyr%4iIy6<IAo77B828>zU*BIoWAWZ4|`%>I-LbxPi<aI=#KLl1q0j`$J_PA!q zCCEmEf*U4jM&L{_QyQ=iwdfQrYlJ8QUst6ZSZ|%1o}AmE5j=VP-gI>|P$aPS*)CW( zf;79lO@As<>u|l?oAAO}e}vx=fKW6Ij>0yfV^%eET))ft#h+bRTPo)4U?F`7-;<w| zECy8Z+yB$%_<e|#NWfcF-|h!p7OF14ft3?pfPZ+Txj}cioqm|djoLO0W)L(}-pZeJ z>U2ouLPcVSRc|NJiV07qw^ytAOet*Ey@|j$sn5gSkELAKu)=Q_)T+jY+KtlA3RhNA zf!}?Div~W-_m6HjW=HX7GdUKvw7E96Z_TMr%QyT~&`Hswfv-1{>GKB%Ko;%xfR7f0 z<Ij&dlmLQ#LS?W%Vz}`4D|lX9S$%Mm0;%gc)0-?MX_NI5iQn5xbJfvddyQ2(&D18= zaQUzMew4km9Q?Ik&Sz_ymP-r;Y4~_n`6-~v*G$GlYBHUC(J*wq-YuZLWG7j03^sL6 zsh3x8;5U>0wgGsp8PFz8q1fmtSE^cCr?XxfblvM2lqgI3Q6ui>a>ineauJ|Dk${^d z+GaTtuvMk6zoph2*A`b<Yp^<|lf_F3%1OyAoHR$EjYT3oh9r?H3rqhI%CDst$Vag5 zJ4}#ei)x6$=cfrI5_I@uU|uMW6sF(7#e8##r4Pr_hVv7)0ptY^usZ-^3h}xN5ATH` zEGW*i82<dex)q4;Hb5;^DvnF6*3dQFMqA~b)#0WE27H7v%<|Y-*7r?9!HkSop(IL4 zWBo2uDOveQe7H*ufD<=ZeT<rs;-`c`OrwfvI;3E?Ji-MP-i-huB#?;`&$^e$mb;dK zmCFq&3dwX^U*%wGZ>#4W9<MFfYrlTfFvx|<<cdoz9X~)+;=;-b1QdoLh$it<>+!h{ zk1}#)Q1R8vqsdNWHA}Xw^0RHtqa3Yknoebto8|bp4lES9^%C&9j#RFCj-BtD%A`4B z%(i>ZOF*Ea-8)S&cv$w>u<TdjD1&PBL#xLpq0hRWY;~<XS`z6rSvZYCTh4NPn#6+< z@u<=C6~gtS?xiuw0vgaX&R;X?HtQei3|>1LK4X4Cl9=0=E}kM_F<)6m)gxxPDk63~ zlFKTj-r6kC2-Gs>^O^b9QmDf{#lRU4w^7vKdUDP+!pKLOJYo-lgv%B?)qPd3YI?8h z0SS<l&s}Lv+D&p3NO;`Gn15i-0v3EFKd){xqg*rr>r4Kz3f)t>f5N|;MJ`2%)`|z~ zl>-gMv*w<0yxcA^q~I=3=v(54*#LA~WignqTlwbSNgj|XXE41%Y4T_BfV(~6SXUKQ zX!=IQs2Kj0gL)I5Ns^zBRNL@JCihH>CW4$$`U2q=CA=|Dfa`*5(q}y%`q{G#JUdA- z3QpgnW2Y*T^MgwaN86;s*0QN%>sw02JY{yxi-WP_^=a$rPay0og%+=Y?1}SB|MyAd za=q>2b;Two2{ixRhZY9?ONAPZQ~cGZi4%N`w29rx_Bf&M0RN1^CFl>*C!SF(H(eb# z@Lx$4_dAB-8U5x5qrO?`yYirj1tS=W-$orTH|vijSEFlwNhS$EBvb5E=U-~H`V6xc zpcVv%f%8fbD}+qppF;E<f3jEKKGDr)HN`5w12%2en;{g3+YC)1F&(;zv>RF`UPu%w znuvF7!UQ!=T03vL_mv0R-~>CtRAUxkM33VZfte4#G_|6}n*JqjMDab1biIsTu}|_^ z8h2GjRBrm>xKK)ZIA>YLT$*~bW_lqj=W)y!Z8FX#=p5LBYE6VjiY!MP(Wt$WLSEHm zgL$Qzklk)NIUsALhruQAFi`Q$=rIpL<1w^pvev`Nu*^c&8_~v5-#$PA_4J?0DA0!w zxndOcVOPGOqx+{2u!W@FE@$8nB1Zh-{50%4Ui2_T+R1W80dreR_EpnzhYZ<rbbnxK zdK0(0isV>~AO)`-vbol%JJL)L%T^{+Zw2jk*C9>4ZNwe4Vwque>kxkU=fRP`ZjQHN zp{Gk&(5Hf}j=MF{U2%*oE`B=<vdjpOk6th`V8NrM+X)vE$(7?NZCHc-J%_07DL;bj z^`u#BETviOLF%7^m^5*q-|KUlE8|+|JE)c)26A$S9(o|dq-A%<w))1qK}<hvKo*1! zlx+d-UhYUh-<W)_w$yZ3bsSIgIiqhJa*xn<QpZC;oYx48NJuS`5hKkfYZ8GCTW<fX zquUm$f;Hk?p+%nmAxX2GyfiTNZe3wTd2rBY^P70wMb)y|QwR~6>9e~7|7*+Cmp6j_ zf*9!urzxDHXl<<Nz#`ZW@gkogUA=FTlzg{1t(o>;s&@tcDJL=}3jn35OpI?TGXAoj z{O+HLW57#3?~EGi^gHtkW2Ex!A04)|VHb*E!8n(1ZKAqx%DB_lsXA3@+hKg}_RW?- zo8yTu*5gNQT<L+>Q8wud!=YM@N#j(|aa-M`j_`py3>D2Rkcg0>mjQjt(6bedB6v;3 ze1U)!{&ZN*ziVjv1KsRwZA-CeGA}%`U6XZ++cs9nmg0QfuXv~UtZ6Kos4FgiP+Bs< z^s^-DOQgDHvumqrg)*sm?0ML>#O{%tzH|@;mP|n4H|3Q{mEzZ=$s_fU4&<Bs*T|1_ zpB|pF^>_Osv`s~CK%eY5k_V86AdfUJ&mHy4e4J9vH@2X5vrSTs!q8I_>u5c_QYbRq zJro|U4y6sx-B|no=z7bjI+m^r7YGDRa1XF?cMAk}3vR*P-QC>@?oJ@MySoQ>cMa}x zn{&?ljdAZ7_b2SG-o2|!R;`-zDPHFh#C5AlNBP$n2_<v!)8<dih9@$|SYP<(RY@dZ zfru<$3L9CIa;A6=VZO_M%WRfv(o<4jhJ@|juy`x`f>wD2ohk&4iljUWkJRW)Bh>Xy zGnz<WPge(4UfjnHMY__YfJ(FK<w>%YipG~jq>K(}K9c>Bf^kpv?rh6yMr<Wynx6&{ zkWcZ|TRm$8aW9F!F4QOt$CpqHMByiT%it$n;jqO`(zoj4Z%qar#=2W4dVR=>O=~WG zG6!31J2pOCZ-^UB?cg_?nHx$+8}54XPCFLVRL*nqEc5m(A}Z!-uv-ki?2l)xw&>9= zF89YY!t>3;$y4LYij#JnEHog1NFMBJ^kFss^1BmDP6k!#nvc`DO#vT=i%_$p0d*jH z=zbGX{H`5QJmJW3)D!>MIecaI&9rH^-hIqV<R$gt$R&kIH$)stt5k@R<9=fO=i3Pc zlfiY#RQ2g6Y)5qkYqfz51j{ef6Y>z^Z>~ejNW31693jbQ@5=*G^oWIU`b4bcMkNk0 zvf$6+_@Bz}-uO>KJ9=rffJ~)eABRE6R{}59OOhY_E1z(~KnL1=Y8p}z;{@Kx`heD; z+!O@_>qEJ7L(Q+*kCe83f3NSmOB$2h3+C2MUPOQ`lU+%u3$|q<orQsy0--ssPZCoC zW3k>WeReh{@q4a_8eaL0hM??GJhA*uK`Zqk^vgnr3jT>_+1@zs&+TNzB2}gS@2)b# zN?%b`l731j&K@gUa&J5w@|vVKJ@+K=x<$3I-POb^4pldWbm~WQlL0o8Az|*Q^73{D z{i@P6CyI;xv>qnoi}|U#dH>6`lO)4eck6s}W2e8*;uxL`)8u$}A^S$d1uA6yw7-tW zTB+3E`rur3S5T+!UdeakD!h{Vw4cfT(!WX5b^Olb^76)9e6ZglshKA|Y)+YD{HlrD zGhB`Cn;jPM)O<|hwL9FK?PY*E4RS$Mt@vPOv0P6SjQHdP$$#16MEb^I)7g-IMtH0` zIv9(RXQj<})J#@+kAP2e@!Lk#7T6HXg$29rP%4Gk77L5!NBB+xVIFpz=`J;?s+9cG zP->d_b1io6kKxkZ*1OCdO1bX+f#TD=_pa;%_0Z4#lC4<wL#Ds=tBOQ|+;Oo)shSBc zYnv2V>t)XK&4BggN#*O_{kXPQj6d{kisiGwT!FUNqw+?SE~%FOj?KwT2l-f?Y><t` z)BN%7$gHI2uv__LXb{0{*=fx#yuqFH_oYqpj<b7d>CCTm-`dxWj;a<j|0GVR&|);k zZu0m-&J|S5(J;^V=|T{TUmir5yE>wWj^*j+z&(a?zH++?uz6HX)sh#RbRw31@Jo?= z@_z(9TxD@2(rbHhIM{SlIpn;&zCbt7;?4~kkL-<1@LJuu##XUIiMHCDJd%pZqpQpE zdOzKecRBCSx_dqK*}Ql?8HEl604wzEBc*^eY#|?bJtL$Agx~dcb!v7KKFIQ0=}eqF zrRPoI*y|1g6o2jm-?Srw*YpwT`m6FpKd0E*O`K^<dAiR?JWsR>!J9Y1C9tMg=-NM@ zcX{vJt#+hdfWNJm@M==n(0(HM8pVjLPpr{!&WkU_=G7z*he1Wuu*&#nTjJyH)~l%K zp_*Z!fq&*5Mue_9d-?2R5(%!gB`;&Y#q$yKw3g~UWBDU|)Z5EE{sn=*WmB)MXXr#t ze`b$y5#w0PmxjC~J~0BuJJKYv)>l4ozKw5aVaba%-@!b9^vBjz02uxx+OTblv%3>F z?%c`|M>eP2dOO*-AERm3sV4@qsvAp&Xqf7BGWfQRXEcDU1_zV5u^xM#$<x$s)gg`+ z{g#=ZmY1wt#JDD%ScHTc(Qp9BhJ0;ZI#Zr!<3BQjG3ttOj1+)qO7En9yhK<x8GVJa zv+e46I=am;JPqrG4Yd{go0Qu(5gR)(!M2m_aoyP7w6d`K+10>A@0>}i8G6MtkLJNH zIs)&}2aQf%%Iopf+|%ZT?$<2$2S7z%!rHP!r2vipM~3UqEXC*j-!k`E9e;n8EUi)a zXQmLzLV_AJontlLI(RP6N-6hrd0uy2`z5YNvs*^}7Y-w4o8mm}CIox()w6;}S?{3& zcM|r4i}ex*xX`GT$|x0#Xqn6Hk}XeMd;0L_nQGDDR-y<xCNV>Zexp580Q$^?q;<73 zxpOY`Upm`)P1BLNTQkiai<b#AbHc>NqfY`xZLjH48Rzk6R2nnauy8VQMoA<2rs0ZB z!}@0o`(v1fQ^^A}^c|J$FfZi$@pR-9(iz4c`!8uej~hfx9?uBH@@4g|{5Tx<_zwnM z-})zB2RHrqnq%YEnBcyYBg*>HO#F`p(94=<?EA8up?;S$D4QD&)?X1v9wpA2abM*J z8^Vw1z!(Q3YsY{MkyP46tj%`8!t2rU<Ltz)uC?JroufgX)jh@G@K0}iak*8QQ!dO~ z1^a=j+x`c^{Q2)mt{Nj4<L(M?(feWrRIE^R1rL`QmhotfnK!M+$O`0Wp-=9Pe;PaJ zBdVdO-A35vN5W)y7?epTBHP%gj=%ca5sbTPmbVl%>;tAM>)pf`Ez&-zkK8X})(Y(o zi*VI<6WmMfIhhwu1BlD{6STAu)L;5(BXrrHD=JE#cLY-^61{q{`8RTdZGXvhkmu`8 zOM3R}NR@K4ZaDpDwu!>FN5sEZ$2ksIvaj$mI1(i2i8SSj+>@R`fia2oZBaT&P?4a~ zXxz{%@O>6*NI!o(&KxyhOdT5$W%$!=xBZrfXyDD?&jh(K;C&rC#6`yYLQ-Wzf5n`k zJ`%3~;$~}h%NPRNKachYIh>00LVO;l;j4kf6-vHJP;!JtGux-$vcrt+nS#%}2<Pl# zfJtwM-O<!XaSm)kM_%#+@7>?0>Cx=2U9cJlmkqu-b3Q73Roo6W!`ln^8ef-tK3`v& zl(+hoQm?h}juqOh(kH<WM${uEaY1SJm8A%H3nQ(A6E8cst}5PZTD5&E+hZ!3tcE8Q zeFSX<5`*%tHyH_blLciv50la2^J_2P+u-M4soKuGq_i^cYeWB5&0DD5rEtAHLMZHg zQo?804h?o<=#nzd^78+#G4eFh!AcJqy@tzwqE!hO7v+67CgQkSqW=iJZa?~48vm-1 z>-=7Ix;}cgU^~>faG2~n{`qcd!5-D9oAehBS^OC$V2;hTB4{a4y^wcX)gL;$y}cH{ zytEfIoXKzC_Bx%`!YN?xy)ryHF@$h#5H@%IeN&W;;C=Er>-eSVF?~lm2pL!}^!zE5 z?30CIzz6)@V6Ov@D|Ue-MSVFvpH|1XrH^;IH(pmAnF@+VwJIl{qq{WGx3^~YGsBAT z93AwwALjScs<GCydm&MzKB7}7s$!Hanr>}zI-L<#beu=XnaJX!GA7fwy{yg_;`NZX zQc@?9kjcX-5=80#{5{Lceq47axg1V~<;>JL{_@aLfewZ7lb>_ZVbZY7Fei#wc9?;X z$@K)J?|D)FmPUK*fUr(OKcQt6&&2hOtvIlM7-!e@Xhz(0nbOscT&lNoiB-;Z^vxo8 zRDqpc^(Uh*RH%bg&E1yqH4048Aczif7y+YMXpTN#j4ih~rpgMTSCjgFB!UuYh~Bd} zG>d07-wDH!Km2A`+UN|NG|x;)0kxDJsbnQN1ID|_x{-h7bjcKsO<M(4Axq7(GP%$M z1$_VI?y0x?D+VnS_6Skyr|?+5C^8^jClODDTAO8K>(eQ3x2S}-Q#5|E-S?&X>}q%F zwYl7g&F*d8J5^uSpxnf<Y_Y#0)e&d#hI2WI-I|A4(A6~jwvaB}pA|+q%*yl8y%b?^ zX1cj&?6T`yMNWvQ4Z@k#4K6$TFls47@@Z|kBDYfPk9ymRDw~DyB`x=onX;@#Fgh)) zqxAXS+S$yzw}3RLa^KFe@}5r(DeWKax6eCxW_XNbVue4|H1*B?hJu1mq=7M_rZ=6O zE5-8Ui-X})J{&dgfu~~~g6pKyeyvejOd*;-kc?L%k+V7*O)|J_<Y&4qaGq^w`|ugN ztB1f7oOuvfiw~DItXpd=wyw@RH>GwzCu~Ay52|7gA#`9{$LK-06niP61|hVz)NpyN zmt8)F6FWYe&?F;KYTjxn5NIvwF18gfYs$CHx$eDTzWHNnPtHjtlaPGRFTCI5r{lZ) z3GA1qUThuRvYQtD^*#!#SD8nViP;TbSCG8@WT*8(vs>V;zPzzHk3|{Ynu1@ze-qUR z3v-KlS%ZAtyGC3yk|5?-eeUPJ14>_Wu3q&v5ERWoT*aeNs;wx7dg)ob{jp9zTT<6d zU|mBA)ziT)|8@F<>5W!~S$3XOrFL38#hz>6<Re-IE^CUw)rfi;kI&UY(w;=dRa!Dp zY?Ns|kCzC&@6H~|weW|h%~$lRqY@NmtMpzSaH)5u!3WX1-cmjG^^;}uNP8fa*qMJf z<l-oP1zUK>Q_TMadvv+$agfuykkYVrDW{BdO#s+{Lsg0lVHQW_i7!yeB{G#tel~?R zO?;EtVkIL}R^Ul{J$i{P7`;dvX9CPF#J>-8#&F<ZcZ?ur%`>c<qifUpSoOby9*(0| z_qsw6G+ZVJ1mPGdXU#Lomsvb7{G%fiW=Ak_N)YfZ&O>j#a_AK3E^@4o=`?DSZR5Im z{4@yT26zbR55xTvyk*j3dV*gWYa(q*{1l5^`e{ErMSn`85yU^Ed^4Z_W1!Y@J?xY! zcR(jIM-b1C5Yf<S2vJVhm0w)Q6y%<g03WxqzQG1ADQ^b#y0_*LJJ;O$nA_ubUDI6M z75S;A*rm#`31!sMYsw|tO<W?mjUJ8gS=Cn^hUQx`X}@G7Iz_^{v3A`xah}D}Hta>H z<KMM9!3{Q;i+!_^OJ@CdpoRWJVcl)9nH=AdL!*Evi*u@)4*~02&6<_reMcXOkbRw^ z3w%FOqv%W7naUw47W8J2S)W?#g?F^=HSc%m^^<Dl+uDZ!fjZY0oGks<Z};DNN#MLd zHaCIY3@WQVmnP4k_+P6y?!Qt4R~D7pELtHMy<WWs&|6pbWziCeBji5J-e9FL=-T3Q zKj2lUp|B%Q_%C2n@UBhWcMBQPkm{%2-K<1OLl|dvd!5Q8Gj!1inXXi^e5%o3W{dM% z(CldME%76EJbu&yhuW{`?POj@Z8oOm9Fiu%w^?<s=rvJh?A`nd{TU7OC#=cnCrH!8 zG9rD9y=#ftbQi~f4oAF{0Dk}8hn=_~IvZ`r1Y7%XxntRga{cXv=GkXrBIeZNDM?cB zc@Qm2Afz3muiK|ooQ?z9$I24T3OPO?(JCq-OTmcLn8Y7jtkO};@sZ#!d};$j7y=qm zug?OuWW2MPhXg~_;djPqXX(Ssp|C|Tw{#!jZgr#)aA!)@KqR3KdF>_4@ifYc2~A5Q zQ8_lm6+Hz|V0gN#T&JPyPdz0#k{fO)r3bVd>u)QhK+uy==vhvmRA*sLBfrr8TikSl zrcHhIZfiS^W}l}nV-dU~)NIkWWhMF7nF9PcsZsyTV_ri`d3XvQkG=4gGOLnE*?q$p z{=RWPN+*1FgCVW{-~J^9NwB#;X)WvedV&lWvLa=WrxkyVe)}t7trKb#w31fz-)=t9 z%9l2tJUuzv5APag@C(Gohef8rJK!yWU#L)>T=30rEXYQ>@|WthNNaKlBb`JeM2Im| z8ubz74eF#*=AYU7i;y(CxWXL=Me5fMAa!ihmd$vcHW`03N#7+x{iT|366&LmQr;yb zUCBuQ{?=cb@G>AFniEsX51J`qQ#m=#vb;&G=IKJ8l$lLU3aeclIj)rTkdFNjaC5v? z#?sQLwr6tv?bgC|Q&AsfzEr^5q;kfXafpX9?#~2{&1IbT<L<@W(HoF5?{_?n+ke$U zNB&|jNO*~oRH$F?7@BnhGc46z`Xkiy<zc_q%RCEJ%PU190#ndxPYYtO5ehTVR6!)% z@zYfyIIk^acu&&Ou%U{kK9f(G+HEXLQ&FI+3!CMdJg@{B?j7hXAe1kbXSKaO-Qay1 z?T%7iLFQE*0W-~EB*qepIfX7qOztwD{2j-|^DK-ykAmsEGys8oup~R+wb0(T&%#u; zxts0O7Q{0!-CV;r-Js`mlhB}IQhAow|8iVG(&1np+tuyB>s5lvpjGWKsc|@yTR1@9 z?xW@w1KP;xngB+oxm+L60Ty$8GGo2e#Sby`YI+`ZB1Vmt-f*)y3+1VOrFM^p>b<>I z^Dx-!#w+A)GFFjo@zRpwIy0|}MRJ}S%jM?;HO`V$5<J&9t&?cH@0TAZf3mSn?h~zw zm+6HEa>AF4B)vhjg&-r44zT8JpUjiH%?)1AxyzT47qB$JK#GIw5Dq|etZfgmdGegh zp6KU5qUxyBw~zN?)2d&W*D5%xSeG=kVPKG}j=l_V_5MX@JQj&E#ZaVsKqwzn1$)#< zw0ng9JqKGl^piSYPk5UriTZI^`X?V83<Ca5^^EM>*;z*F;pi*N_SL8GE?k9yK~L^w z$zN&i;uY@rcQY`OQj>b5hMP~;slhnX8=FD1ISrPt@}4yly#p?EaMgeU+H)I`dw(ox zXB*KN*SV;Unl~|4W`)*CnfXtFi}=6?>uzSmwCh||S*yOEh@&H|9p99*kNxokG1v@{ zmhSEdE)Tcvb|2)TQ(x816)TVB-CBd1H}IywmaLWJx@kzBPl9=-Mif6_A`g~v|7A{D z`Ga2>_{NFRDMUteY-J30M1P-l=dG76$C;xe(0u2)pgL~U$~B(vBeEW&3J!}1@$f4x z{GNIm=6YT@BWF}L*B`8!^D{ojtSW*C@9YE%PPuW2ep(s!hl`(4qyAXz>uKDx5x5iq zQihdo?Tn74;>mv5FO!0g#RZQMR|r3C_}bN4Xnth3aUkzuqWLd73P7HA8hor~C3>2d z_%4qm5fcDqRp@EeE`V|15@ky;ce)w9u1iBPXz=mSW!qA|)xfl_6oCa*S<E#06=XHI z?#ce=g>!8|kfEFyABEAYD_^n#ddHumV(6Q-6kBnzE^@d1#|aS3K&{S>-P#578y-i6 z^$9YtrOS!R*GbNFD&=lk+xj*)@))DkAnQn4q*+mQ-TEY_&v@QWN`rO|E~>T(pQ>#t zP3Q%Q$dZ0_OL*5#w{SbXU6@v9A#%`TzmVq{p}yppYWCpGeFpCo%JF(0hq_a6;lF!3 zo`myeLxCI3!@bz58qmtnE8dz60`ICzQemejpj}=8eaLd-=|wO82Bq@ZfK8xTakN{2 zv1xyElZ@ha*)lW~EhZn#wCY074>M2TcPnA?e4+YH2Q$P|Y_21)IW9@;*>sc}en+TD z!}J2O!S;wbJ$)SzdC(sA1SDB7fI<Svyj*G;(?zNkG&wZ5Z;ah%FxbOAnlCgYzng#8 z=07UQ{=4^WRli4z;YVM2TW-*gtX<#+xCK3HD7K^_1A;WLTc43PGUa_yd~BaSI5G?K zy+E1K4qzdfT}VsuwLQ=MGI|z{;U$xk6&3z|^?WjO$7&^lN>!}mk;-D3!*$iGX=fc> zUx=WgOE7tXfvUclKP0SDM5H(@<rC<}vtp;t%i(n$(t3ZQlk)Wl`AX_A=dgg@+%>WZ zYM|C8-+tV6;BfjEYhxgw8%Bcl1!RgtE3V=B%d3LKlvcwC|F_q7dW`tR7Rr<&M2ZQ3 zi6yWuM;GQKZY1d`BM(Qum3p63PwgZ4DOY_IT)d=AD+n?_^!2_?^?9qU&G%kEjzBlL z-f0-ib+DNQ3ywg8L?KXKTVoXVJqzyJmWZd(udajQBsjuCBQ1T9{Vnz2Lx-1B{(Y<| z{21+zVRNIj%2y}ek{0{LYD3npLZgNJ01_L){j}XnHcgLe!k|_VytG~<YNK`n<*zjY zaLANCq-SllxGYGu8u+Bn>Ig>if-28O469CAl-9C=I7P-s4O&9`M2|=!La1#>l)BEP z&zRLZevrhnE*tHsa(>1uUUgy6?idEQ$$P64JnY(J6;dfA-&0A9<ga*=)|N$C65C|5 zz8U;!Y#=Uo+mjUBnM>moSD>jw=|fqoqY3*qPzcDV4QRj{@eJ!x{P8!z@lU$Ye-F4E zK)bubIBFKzpU;k)kbb-q1=)Tj{kSVjrdUZo42e2gnOGuAsSuy6W4Up$1F1EqItH); z$auSm<;20x^mF<NDd;OTpn?5oQwpSFIX#=x#$N>S<oG9hp;`X$xbh3yx#Ds}$PuWO zzk}uVWD(H{A{6y(5F3XB-7svVJ`|+WDjL|`-i@sND`pqs0B_W_!%LxfL#dIwf<{Qu zL8-5`Qr241)DU;mSYK&0=rT+YfS42kq<E9JkyUN)Vl`<n-&Qs}heY<EPlFeb3?)*+ zWCPw~DRcZJcrNCfWN}3R^~@_)%@H@SuG~Ov+l3fKj1aJ5P(t=^0mVsTCeKhvq##}a zKk+^4rk5IQ{M1e3xgeph2^rsfwaM%4f)fgiAQ(MwpJX~}A(~7a-yBlsI+^yLo|<36 zWWY9Rt^iadvgGDg{jHkUPd+!4BKfkyp{2x3$LjJO;HD(T-NYm@`Y<|mu8@LyWvGC^ zn;b&*FPZmiuhLkJLQA#2Iu<eNLxX&~1*Im)*>eZ8k_6CmflK|N$>e>f^2LMK)ks(a zddF5%n>0u}kZ*rv+5tppn!V1%LV*dHkFW$TtJ#+^&B5Aw!B~Er9}+JA>@UX+d=8=? zL?Mash4gy0ZPBnly}@>YR(|&#RFT|^LvY0N{ng@jAO!*{lfMgPCZv^mLr;Dk-+|(h zxa^52q!Ix@@u_-Cbq-g9tWu3fWYZYFL}V=1dui<J10R|eZ>wfT^purr%bJR&pL{J& zr#r1nvj$TKqV|Swi-#wo8?Kt6EnaO$dj-r-b#za42SK=2G4CwBjv&KnjJCV8Z`JoN zDu##M9E-Jq7Ru6OHr^9XOkO7@rVEw2(q~5XhqyI~hG`Xkp9p;;h#Bm9r!N<>$#;u` z-Ni=g*(J){J_*tdCfd^Kjx{BK>FIghJB8ECCyj`FIvNbomDyEBJ8>pMQoS7-m}(n1 zt-hU(&ga`ivGd9c-vL&SOq=7a!d4_U&j)f3PRpDeZYEh=kAq;Qi`Bwby)mSog+y_r z3$6+%gB}n-MmmX9GI8*D+Gr47pI|olvBU0sgEwUzX^n9nna`=(<KYrkBOkx9A*mM7 z56?4-Ne0hXXwxT9DgB%(ko;AUs^NS(ial)zNV%&mm)RbUD^~3+-d;5ZeDKD<K<o}D z#3|q1CYDsRtBvc;wT}8`I$%;UrG%xi6Z@8ug1y;3^TjAZ@%IAa2@{BCbxGV!fXgVE z#%5VGm=CcHg{c9$&jGfOccdTy)KJPgwYxBcF05=QY<LLNR4CZn+|);u=f*YZC9F#` zV0uUw2YeFa&svMr|G75=fb9HU4E@hl!0I68{O;bsqon?y`{SCFFI?*J(!dhvDICCt zdrJBRcQxjFw*mf+2Q*uD2bd#PK<CX*0X}3QHh}u~Gm;ovAb$Ypkt15b&HwRdP>=$u zVTq2fZ2#=Hih-PwfaI04$HWOKFe4AcGQ8Lgr(D4O^GtxgmIZ=Vtrn?Nf&X7i%{frG zKMDHoKfVJ<9T`9by4IT^;{HRJTnq%h{N(a`CEkDE&IeIUWOIM6`v6d^6dWy`nu-8^ z<xhI;_LHe}8tVbT#I+0fseY_Q&9G$)g`xnVa>0z&_%YGDvROcuMg(F~OlQ5_^Kw>2 zTUdXZ+p>Ktj5=Q|mMGKHiUdWq#i{HImkWcLZ9J8d1YPq%Zjq(85gX77cIB(|go^?H zgwYz??=x1n(XSj%&+$8BNxZ+8EOFhR7vze)y)+M<|7zaKmcOnv+U7}lI;Nu0X>gfw zeBo}@?vKQy?F&N>U*ZuG?D-$D0w8yAU5!!<sULiyRQ$>bJ@w(~Zn%F;-EiR#VKGSz zGd%#?OOtZfPgE>k1etxipKguLHyHW}Fgb6EP6drvJ&xzf{CvErn7Tp<I2~j;9dSX1 z)QYfL6u4OrZav1;`vV7=4iYl00G~5~X_sm!lKW$!ab1eqe2u@mgocJj;TkjhT11$X z)$`oqzJ_3D8FyR0A$hTCowVZZ!*ZGSaI|`&sNphm^&~7TtbC=7+@Z}?2DWP&LYez; z^4}EsuL9kpY8U{?32+AuIzlFUam;(}o-q8g56{Pf>_a5@|3ljd0^C81dnG!8z+11z z21e|~6L?+w|C&H&me2AhQMrEr88JTWz^7|0jo<&%B`ij~7ya+*>QC=Q|GUq7X%Gi^ z|G%R$(2T(#M%DWNkEA=;56S6p0D^+Y811JunEvN#a{y>NgmczP{hx9`NGKcNULq&x z6TUarLCbg8%5uByZPNeCS7ZkUd@IL>qygr?4{S&08wig%3}A{<?7t0WzE5|+iv_YD z2^%ih|I;LV^aTiwAtSrP5$YHC?@bv=34kMo{;mj*{MY#ed>c@I4W8R4@3yr3_YdRI zM&7$0yzjfD>f*xoJx-?dwnCT52>yJQwC?@XcY;YRCqboHdc<)DFtkPhufo{t`>0V> z<r_>ls_}L)Kp+p;JUHH-uej%b?f3r=)~bk&ic0KywU>}99R5xRqF(qF159s#SL`}J zrO7eltCynJ>V;-?+8zgW1saW_(6CZbI;nm?V2y(eFP$rr6O@Rjh%NQ}+F3;h!oVLj z0+4~93~EHGIo=14;7)sD=;8Ua_82luO+xy%htOa8)-qWn%v$)yw-e?IQd(J0Df=sc zbAcR{OezLIDB=kB*>_uS1Nx*`M*V&)Qlsg@@0m5>nDmLP=5vDpdX{@YzQuOk^Dd<s zLUX+|rT*z=zQAZK)mQoo3pPItXcEQUiL`W&7cFIW&JA<?%ko7wa@br)hLnY<gejE3 zt7(!m0Inx}r<iuC+n2(J1VYzfgG=KX)zAlw5&EV}XKz9!CDU~?0sN4S!?Q<vWtaMn z^>%S!=k;uH;Dm5zN|HW7wr09A0_f8V#$%EE{vXAENoTy{B{CnU^gk$P0Zr#&)Fd}! zER8)05D~I-TXk`Ybqa#j?8hIF$<}+27Aw{jKKQcY=r2~~1@BgQy~SmD-IFalMd@qS zznCpB03%_L_eY=xQzWwa<Aj7ihJ`r-2L;?`Ki{8XOn>HDLEYLmT;A)Y`3_?TjFB}Q z(86Rf3w#G%{?H=;XOb8Y$n~e{YfeFE8b|jXzsOT1Pd@LZS??@ccd#nwA`^=wQ$^u@ zBcoPMmm6+9t~<TvF=}W_;B=430{Tkr;7&w`KQgNE@Cxbj=tQebM-uy<r|!ydcD`2| zrHyj4SU;HtkQ{#t42)l54AR$Bt)+<xd?&RKdaR{F+7bIcM$%v22Ux$kweeRTmMD=? zD}>q)i{%w2k>aZ-6-As(-(#6?Hl9?EL`(s0Nbar)fD4>2kw7I65Exwni;%%qZ|L4V zFsxf2HNM;~@5afXtj+^l)mGzOeF%QgZ+X!?RDf41K_C!L{N?cmiq4`q<v~Pa^yO}y zUi#PcOq<iG+SLO9O-OZL1Gugb4Gs3=9HM}IOFy0f%?Hs9Aelu{<1GED)D?6E7^&RY zzoQ6v>E3aS#oGY;cXA<DD9{|J*UQ^d)2XH#*J{sWPHe^l=^E~x&CI>_+xh(xjI`~B zT1${<^q1=32W&$~fggl<rPLPXn7&IlIO2zP`>DiFZ?DtG5re=SG?uJbsl(4KvI;bG zKXDZ6AI%QMES5_%s@aTQoj3I|Sv+=^JK}%!F1NzzagG^}1Rf@M-&O!6xRd<1Il^+A zP9N~hTOu}S{lZB0Yl7p_Db~NIDq~pvRdl55@@G^dSSc$vo;3|5F}si(s9I%vuGQPY zmermkDt0flZGFuNtd?E{isj0ZY!%wHX>(<X7%JM^p`n!W`5`POfB2C8Sg-b4HSJ<> zI8a<b0W-?cB~X!R_P|uf`~pzcc%cx*1_7e%D2IoDRqKUFS2!3XogVM=xxbu+)892q zG#q;(s1BrHeS&rqcu0DD6Y8JnI<!dX!PWj(joLs6Gs+Pk`1BDi&G!RzDytv>?!>$D z^LxRyYV4Z%Abi|>T+Tc^{tle}?Zghd73%bNg%Ex%jSw&tQ@&(jk@tmV`BNDVJv$o! z7V?6{0u{+kqP|ha*DxN45M3RV(1WM7XLSz=^awz%b0{S2Cq14XGoeD4*x~gyhE^iq zR{!fAfde&X2t#HT)42nXb>jdoHq463MS~Axf&1N=(X(Jm%{Zr|*Yky_>(v-LDIJ}( z(P*-6`M}ONaWt1+J!6obl*MYZ<8LwxDbh#k0)4#x4WRqD%?Wg?-T_@|Xr@?U3@d^s z$8t?SLp=RwL*=NOeti1lUH*!VxptJ}E&RqWL~u0&51W5HD$D>#HRV(tm6i$j=azn| z){6Z3hlkDOF~DEg@woBQbE55W(Kn>E2Utb*S6VP=)krY4TyiM)U*HinC}?5;ui2cc z(^9=9HG%hYm1wAM!ACkhSr$-b`U!xxwFrKMa%NXm%U12&Jq4(wtbL5fuCq!i4;o=V zL~B3L;}8J$mLWSQfbE%*6ni!PZK5@O!_e^ld~=UeRb$QlFL%vqjv#as@W~k4_%@HM zUKlzx9U>Q7{}=1U+mC<&j4+R)Ozkn2;}%e`8JrGBk2yUZH>b+HhDk%b+G<ZO!q-2p z;ae6MZrkr1!4HcKjQk!tAbdfHzrxidp<UK6uD?57-WQj+oKcWTfsM}ceqA&WettZz zFa(6Eg)2bRR$w@iIF%=g$Xu?2;J)DtJ=%10j5?sCnjspAV;H5oBFKGzxLEUx1LLrw zHp)?k_oY@&kAY23!~f~;J58C)%<JVw#r1NC{JTQy04E7yGVfo!yk4OFr|E0vYb(>y zhzgTodL5oH!5XlFmkd7FUy<`GXRSNGnWfIuy22ORYoW#^qS-QiV*#6^V)EV62W*$d zrIdh>7;b^|?+=T{ebssTV=d)&57#TdN+8w;td+I4FWnY<N3$g#=Jq|2m~#m{uVdxH z&bNH-F)5TQ^WV|fEk%p{Jvhw2k~Hi>U1PQjAngY5oW=4)A_42+1RL{D6X;$l<%NC( zT8*jf&g?SR%bnN~wT2{*hlzgED_6%Sp+L)rEg>kA)ur83hRtL-8@1XagW(HIAV!!& zrjRcU=skKXM~8fiPJOPI%4pLdXTKb>t=s74)Ya@^zyR8>QuYdnm<SS;9*Kgna!D$M zNx1!f3mzLFwGBjKV8j6Q@)2O+#*JaFT5<PxogTm>vL_lVgwE4T)ptth?E-eX%*Hf6 zeN7;aju7sPa-^_^0(t^P<C}m*8e^9~ENMhfa%UAP57u$EPFLVUl%vU8d&iv_xAD+9 zEc4g7AC1pa?`&Kql6TL1%p_zfk-tm)p#n-79D2!2%#`5-D`FGbSUJP@h4Wo1j`SVQ zpnI&rB<)Ptus}VZV3PP~J_r>$u7EO~AQVJrb6a0@$m6LE==N=wKY})2{89w}vWx@0 z`KIyrCV#}D^eCI26Y@>BQW>Di+*5lNaHS6E^F7AZ!eB@<)3?peR_&66Ujhe^*xI9# zW5B!#h#dfOtR>?WitJ^&e5R*`L*ph}O^4wN4u^|QR;en}#EE|4JQdd-n71&VF94Q* zq^Osh2WMMAN<JTku6BBob2T}4sd<|x=onf;Cvc}318hYtRpqK0A>qCd{ue|2kBe2X zbk&um@wDp7YPBZ1r8u`=$SgPJ!mzlh0kS&_o$f+I9O>;Bp3Q1=a+k{Ji-G<fol-#$ zcs;cfRp=4NDm~*BrM3i6grn@BEcDhR!x^v-;(rQsfSFz}-6tUIO9EorL12{q8SQw) zsa_-f0qqKAfD;kUiUJ6R{{c9HreWi0l;&t$U?oR?7la}%!^de1|G{cZ@~ro0_oySY zy+y7G`JT$ccv_2SegeYyh$vpNlMJENp+j25$oMrPm?}OQu#Kn5%W^h59B!px2TXyH zey9O3+E0L^8W6U1OZ`$x-RM~bz^r)`ny$MEg$QWKX~|&FCvA)$La`+%M9!h^yt21_ zqkX(!(6RtpLb7G;r`MB8oUUqv)o~jX!!V3HB?BCmCe>^NA}ymsKaOf(dq+X0<+wQo z7-X~{?$h8BF6i9c6Q9&RlWr$2pRrb%{~b>y3g=0tG#plmHyWc)O07GBLHi6d-1c<V z(yw>&9hUzENmXBGRya2fhyWLR+eS0F)hln|`*CUPVEgo!YE28dXvn@oF9NHzhwIls z#vyl&ZkO;KaHI&w&AAPMXQJLH$GHN`0VMwrTO7D>bP`g&TvQmEMW@`4r`OoWY^AY* zS4Kxd-<{oyZA!1QH-b>I|BK*$2y`w}`F~tkz_MTmbXXomTd^jlO$WK*5kx-vK$@*T z7?}bZILCmMk^DCKj(nW>+h<6tA23Y9WX$F_oiuv$<(g23!qg?-uvS<$`gO}lWyv-g zETQ-x*J@&boGE%qZ#F`8lrA1N^*h=+t*R>2UPQJr2Jh42$O*@2t(qi~hD2%Z#|C%< z?VfNrEj|YQu;UmIShdrQ#Dk#!Q`>IpF)G1rg*1MS6_sedR8!bg{^IVMa3fwI<hi`$ z&=1H*Zl)<Iu51-+DLmqL<)1^P?RI~r!!i~o16vm!dddJ&XY>$8$Qaj06<U%H{LipI z0R>$ml&>M=g%Xl^hHnB6Ha@e&iR!*4Z@)E+9(L>>NI#igDuE3$V_*<TJ};K1wLQV- zeI|@@#6aSWLk2(%2(XCncE^vtLE@%~w)CT0Kb)CP)Y+LW1R?qcHe5*PUFISo$`NFB z_9n8dA@<6Md`ZB8`I(tH)dsQW0|uS>A*JsUhi*5dzMS7T=p~4|PpZM718W&aTwhS* z{KtDD)+-<pW2Pm&wIx0B$G43uWlFaG<wo<9Bn4v(8{QixGO<E(M>Q?@^9$guvh2uI zWTnw=f4<x)Qi-TTzwlw!cGn`8)ga5u`ThlBo{54gnTG#o43Iv8A(H_OLVV;;tPaL^ z8>D#*w}dIhNsjRCTQZ_a<FGHZRp=3dD`ly)GY){nIgZ)GM-{fg6XXKLvwG+eB<3;d zqpXIv4K%y|ZBtotY}A6oq_xs}`8sFV+mOI~lus1L+%9_S_YM}X8_eFUJ7&emH@r<O z#MC}q{ItipM~S4Dr&6KSPyWTqaPY)G*cb?5BY9n(Zcigyu=S7_reU2!*w<WowrlWn zm)>fe@iye*zq_2FaEhUt65$;x@rR+I3#CHA!%)G%g;Bh+K_d5t;T=-xfXn6Qfj|<X z;D>E@RG>DJq^JFtpMMuxP7*ZehSpX`uPo|49M{@T&OD~&H6d$QAmM#Lfl5;VPOl^y zzO{4nISdiPM7Nd$Mp-i_k||XeWO~q$$Kx=z8{{XfRAlJMpPOJx((PSQ7qB@IYLksz zq*g;$_xHM$bG5}iO)}3<b_57Y<sZsGtspb30BjTa4b%?;B!XNO=O<X44n=l}!IU^G zL=4`uO{E*lmp~{{az4r+9vI{AmZMcF7k7CHok~bUvY7%shCJ%5O9y|-BxPQ;u=7yr zPex@tu5qoG6|GotS$YLgu7N>-{qP9}!jG>XBwOlY`sn^-SmCc<cj+t2IbF}%L2az8 ziM*$^<RG9u?Y^Llds<?P=d=#Uo(TgU0y<zvNBAn=f|{VT9vK*W!tqiGy#p}qvIDpf z7g~{dq|dv+<CyMyHUh09YCkSJBsaUFKvwFLPsr`>uV2yRQh%w&QOuE`6#%;8xhPl% zEV4oO70+Dh>Ysi4vxhYTq<Tkc%qd?>Z%Q)42#^~l3;Ow?U;x9*$`r-Mws&^`?65Ur zH>{K_Kj&6TdGPvYh@Zg9L-e6&1KVF1C2y#-{J05mx90N%6U_?Wd3O!gKIb}S|Bc7o z3y>Nx<5Z2fs0R^Ga87)~$k^XHA;`GXY{Y=fg#-7%il<f?k2k)bJb+9rM4#Fu5}^NO z^m!l}FL&%%WpWCtMu|vH>cQ7Zrl4okFyvZ*4>ILiV2N{^Z{z$Ihi}k_G+sXpwY2HS zlC;EKRhlWU@U95riDn(y@DnzoL803h^fO3seh?heSFdZNwmD+PDV&8e#T(^{l4LnB z(cY?%XkKIkqO*E9L4FWuq8Y>&eoh9IyZ?l7`(Ex6nL_T!=;&F>I%?D@RwAiP=({@4 zPNEC_C31+x=jA$c%tZt^XFon4n86x4zU<thR2U{Xhnheoem<lSDM-7$;qR;3UyxEp zVO9KmKp+?rLeK=~1EmWQNN$7+Io2My3<(nWqD}!A5Gb$#3=<=@P)hjyf4&Z)z)y7u z*7%6_e!=^75Tam{H!*dZV*i!F14lMkxrkx@znjRzfZP&DdTVEBV~2oN<^$Y_zn_R* zppXy-24YTR0{{p)oUL#?WY^wWh7B(A#~`CXPJQqNd~SdI0=)|dWSN3MNGOoKi@YQZ z66C-GBOSsxc-Mb39L5)1%{-BAm}zC5T10=P3Ggv7pmAU(a-k9%{&v0!uJ~KemeYO@ zVRd=09cE8iH5?66C?+)i8>Pp4V^1agb2ubDb6G4Be;2gE;&gAq8U@2_`mw>`#1{+_ zdx&@&JS;s34IGBC;>I;F^I%^2rp4tFt`1X|ET2Kr*wn`-Ik5n|XsQ6HK%vhNB7`3` zR^Kd`alU!6l&Q5b*(IY%(?t5LT4D0{PdQVxlJ8UY`lKx{uL+bilxH6@7Y4H1Gd+d8 zS7%5^Uw1Ze7sznFU*uWY7@D|L-y^CJk&Plu25sFyAe@wIK90wJ22?#3IHFM#5(8^P z;D=OQuEsXO0zV{(FovR+CD#M$DtVkg09!$IM-K-S^Uc|6IE~#J=bA=Sgc+he04p*a zh8YKTQ5_xpBDU%%Bi8lbvKulXlc);a5%;4eUk$QEU|6hHzDBwwTIHep@&ScD1z=yK zOqvX?nZbmJ^qV;g?GXcS4Lgi)pDyDdvwV$}F*6QzEP|ZKjt=<0vIvM&AYxCD<)3f* z1U@;adKxw8@!n>8kbID)TO|dD|MsPiF82yCZ-2sW!iR$d_%+`L^%Wa!Hf0?!V?9Z} zq8to%tvcKs?N;vmUEc<*ya~i(`}h$^dE^OesVU+66O-+$>euJG4bESuUcG|wC`=i2 zkqxH66c6Q*D&Zi2L}9IA4;-i0wU1I^e6H)4wT|$u?JfBkvW4L;I7YYDYYZd0^ly?S ziDR=p!K<&A1@OA;`w^+<0EtH!I5H<cliS^B291W-s${Or<n{Rn8Y-&(LsRN^q=};e zeD{@73WIrh;@$)C{3Hg2GBwsm&sT!l^Q0=bgUaRkGL@vu9mcyVgl|SPT)<P&c-R}5 z`SF<{yA-;!r?Ok)YjWAL)oB`vJ!z!i%~VP5vk*KDm1i8{&-J?4pCCFw9FmSI{#Fdd zVK0^ZC2axkV3t@ipX;hX1snmhyJt8HD?Y=t?!33gR>tA0Z@!+8+sJ|@5D`Sr9gy<; za5e4zw>>X7z74d24)noR0b(dWzI$TX0txhPvK>62A&+IXa?c7x;1dGZkESqR9i8gC zi7kqLh4?8-)@^NrQ-di%C?dnk9e~Z9cmf5k4HE*|hR35s1^Ru6FevK#9H@83Qo(<# zynnjt!&|dU+h+fBfW?vT)S|-v;Z&e`{lksgd_F+?kbPn{J{Tu+hFXf8L4|B30^wj+ z{1Nj-gsit*7V&(#$yrvL$*v1$Z!r`2TCUd7ddwB$&z*+&CnauQL&<(MCc|jrEdfYR zNy|8TE;(v`hV;D_dLLqi)ca9@ac2Z{4g~%Rz}ne#+D{WWX3+Z#La8P1OUDj%qvoBH z-NiF0uzcmEG~ctZQmS64nVO?jKTJ$*K)LjRfraP-9w$X7HvG&9Nb95F;!1wUS>@l^ z{P{7fq_M<Su|!-4=Lt1fX0qDU{K3^J9Jfemc())7oys;QumtjZRSYO*4XBv7wU7}P zU@+2wg~VYo;alJ`1@d>b7oEJBNo|?mw!T2>8eQL?J4)OPr=zJtfSW)I6YGbnBP+-G zfN3;4m>x~$iB`Kt<9$o~ti6_T(%b+a0tf8H8BMph*2NzC@wtI?WTN>E5dJrc@*%-| z28Ur%)x><S0N}qiz{r8^%(Wx+@36~<REY)wtc3Cb7R(PbX(sG{3YPl#v;leUYvA;& z<ndh~Q9vHAZ!O}RSfSmqR@#Jv`SL`~ZaIoMiIK*BsT9R1cNXdFz3Y2<_UT(h_{Gb` zS=kDvb!U%C6Ym-X?Onfm_5vVR0Z#3@J}Rl0D%U(~hkxXGq83%RSQ}K~=ojMXXUNMF zjh+HF@y5a+nuBBq4+z|uF9*^(?6g~TTlt3DF3u2fHP0g1y<4BC?*>{rF8N$UGjs;K zN_I_-KYjCV-SpOdzQ54Qf;Nf1)E|gSk*5D+H{I`n?5TexLeYa|6u?nXxI*yvb@;>X zDz207`Prwm>n*&cWZ(!O5`S&|?Z^hcD)BknsleLaJ)lDz-JP9sAZB)Ev9CQR<Z0Iq zd^Oss_+nY*O02#6>H=TOrMkw-*6tGQ?^U@I?tFb%!X>*#eBq2tvo<x(4v`ys5g8qC z1P0OK{8t&V8y=IZ6*X>et72td5KZ%*Y9N_n0IN;g+y7(xOXl+J@``IbrTrXQ&QEG* zEu8B~^50p-(?(O}t4}abllmP63csYAZg!*Dy?5JPt_E7@2C$3_dizy40=SvG9Y^UG z?B44{hL+8UtErqLqkKK`ICZm=;4iSnr2sFokT$u}WUtiJfSATNGWOWdO2gx#p8RJQ zt0#hiy!KQ5|EEqO_B8*k3&xg;$9DJaRqZBTi}N?QP4At-*uozDGm5+0mP?f=ugSIL zt5I!^CA}+=_TiTc3=`MNR&xuFi}$1DT=d8OI%>D(RaIQ_xkmS{Lsg28L2$0=n!H&= zXIib3-b+91On1eTsyiVdGZ+q~`g^5*9h#j-&1jZgh9=0Ff4a^CfbJy61M3j26;KWi zdnFoI7!h`aw+-uFl59Ub!Cm1zcB#ZHA4CZ}0e;OY&>~o!9S%xb&1QcWQ$-uCz33V{ zTJSud3r?@&(j`(Ug~?_Ti++CH_ewIG>j=KwrQi9jU5z4lcsZ-F@B78UqmUQxTdj<e zu=}668}CUuFWH@uB>D@1sr9qf7F5WOqdgJN2~S&}UZCVZuZ<0cL{``(cO}c!7>&U! zrux(V*Bl308?08%*gUi5Hxx^}*^SVdQ=KEC-_)&VfCQnn@Db-QJbLr3w@zuqQ%b7c zGZ~+o^~ti!s(RDj*!*9(R(EQ|0Otvg!>jA9mwsI9d?4vcG6Gw`c=6tOOuV@6Hhm)d zJMGUNv5$^jgV@%FUv}4lp2s6szQ4iIz&FzNGO@||c&7sNk<pP8!CCvvwQ<jjD7N<h zT0GD-aaSwoEjs-hfIjuGt(kPpv9TiNs~5*1=@iyrm{*OyN<7GFHruz5vA(Sw%_Z5= zpgGN%SnBXCgUdEqnKnfj*sQx<UZX;{_`)<{nlLoT1qoh4=#vngc$)c(s81r;!c1~s zreNV7*ilfQkGY}53;Ya@S68~}=?HOjy>fWPa#8bxMfW)xd^SnKk52+wwb6zU$Y?}p zdB5Dmvn4$G`A|Y|n*GQyVwD=<rYD-1+X{XAGq{<@6~4pu0SSxA0n%_-DgKDX483|1 z@oZJv{#Qgkp|%HpHP3V&F>FUVKE()@Zg}Upu3$V)Lmx4B@xgaP`LSRa#^N9b13Zm( zkCfL3a@XYDHut-q)0x#b8TXAYYQTtE-9)&g2&|cYY9_%0&xI6oK0?Vad1MB;q3#H5 zm@6E|0Z;kohJAlV>E-Oy30Je(C8hoK@z}sI2faitFKGBjT&XNwJ}d-xmU|Fr^tsN4 zY;~^4@%I3X3gO3ya8TUQGCiIUCj^pFrmqoMLU#lV<J@CGsajo_h!?9L96I#|Th&(# z2^?<4<<r!PxRa)8xiJwXlcSTY)!!Ms?y(Z+)Wm)fxKws12oOJ!n6YN^KY6jA_oQDv zefjMu`cu;$g$9axuP`-|8X7^~H}|1<t!?&j9Tj?Y>15R*gigCjs$tEoWZ98J+#OPY zbTps_=|g;9EzPO7Ig>yDC<aGYBmm3&6(*5RU4VDuIaJNj5@iY&f5j`lo@PnZe~kcb z%B!H}KuBlkCuyxt7lhk(BzrC@*e0hE3_v1?{@6O5#d$4_|3JrzN1s)dIE+~$g~)wx zW7Zs7Un~TcnzEoF>^C(U(1P^gfW+(vl{eoMmm3mRS89s3H;w@G-KiMzG*NFpOu0ju zMu*l1x1t;3>ydfyAPWs-J%Nt}4)FnC%5#B`zB<ShTCHC{xop$oymdwBw1~|~2P7NB z7%V`U7;R1974E{zJ9!iHhkzekI*e9kWg>iug5gU}dAX|e3B*R$EA>tiq#XUSK$ePi zynb<>mUBNCu+Ckx6aoA<1WCWHMG{h%Y_ogPCon|}S<i+X2$R?T${88hLIxqe5P{z7 zD12S4=URYplrLnlS#@*>p>kL^1jz^VVZf4*AH)h1AdrkqNXF_6vRvkw&vWPNmLh>= z<Wq!9seRED|BD>vgJJafblr3~2Zph*QuR^KEC6^c8c*{cFFQb~S0ECJ6Ua@_&DT%- z&l^dO=<t3u+K@>OWg-0N5>nIAp&b|x55a?UFnI;lyq|+_m6IYIasjT2g;u}<*7;f{ zYO?(EWov=p<DGIL1rSk(!sN4n`M@x4-i%;#T9EYZPzW`!wqtA%q6A{Cpid44%v&oM znKMELeZnps9j!Hb|ADL7pu#wmF@%8RQ%T_vSO_CiqW73RginHnnW)!bBNEQeq#&iq zCE`^A4h*SS0AT|;QYwZQ#gq&N3)#o0&>?Uso_i0)@$cNMD#?Ium=Et*1cn}ui;MtG z@FDHzG*g1FHa@_VD+FxyBBpVS2=#mb_f7?>p?jOhEp4}QGmnDISa+%R2odM9Z-4D~ z;Ef}&tbe>6+9~tvT&yuZtmsgYuPJm^Gn8P8MgnboPv!u#XPWB`K|+OyDS+>QdEb9y zaXeCV+s|-*o=bpxA7;+UfEOGmPg17x1|;lR%x|q7-aPNrtnCO>V_}#GA>^B@*dTyZ zIBBUF94rK-u+CGIWv$aOuY+|%;qSw>@6e_<D`hjUvA2|`RKJhcfs-^9s||*V`a!CD z^QAuo0Z?BqAP-<)pv!Gu1tglLS9@b94`Txr8@-{ZERK5yZd;M{zvps?s40@Tc~_P9 z*$B$=#gJeSd^MWv<Zg~;Ka`|*t5s2!($7NqRtjVTGsYB<broK;Gbos1Ps+a9rwf{W zHmsiG1G(2*I2>M$I4rNSnp~>X9CX<KG9BypE@f<DNEWXDvS_kC{^9_Hut!UE@NF-N zCK&Q|%OufK5`UVNogc3culp(|Ot*u+&s6^K9Dj4XSIhZAD%fdNGab^teNbp8Z1NH| z=5TQdm3ldnZ*a5Y2a1u0wA&dS=de=#Lwrh*+m-d15QZ&>2-!d&F#lSg5;Dd<$MmYD z$Rk6Z#iJiIoIw7oV2?ZwmKRbQpZmM_b8Cdw<MZKpTC1>;$63b}i+4Yf*}{&K8iO~X zc>iOkRm4s#ECRWHSq^dVAH`P~@5izMinmV>5yEySAAEb<qb3jajf-|so$rfCh5h)3 zkvp9i_gI&Ug^ITTa11Ava;@s{H%8OV^;#O9OpD|BILn@oxs6}#l#BSrn;kjUQj&a( z-ET?ejG6I`Q<!#2l_QI6bnkMazYs1rJD$z|(H>9Vp^@_|5<_laTHfjt!lY}|WX}95 znM6jVKIbAHmG{M$%$col`EN@gPJI#Pt{BXdpU(}FDyKEnSXyOrB8`Sw+t3dik>dxv zXW+!jKhOZPy=(D@wFH6ZB_XW_W0qUn{Fqx@95%3A2IE$%z_na%&bZtgOEY8fFgK{T z$XOWA7^-)xTjm>AtE2F`HpvY`uh8Umi`Q#$BwjvQoG;&`U`s^Nat-|A`GjIPUOIdN zVE!CV4zwn05<3l}kcmU84dhhxTb#+4s(NI5UKny@2<#BBHoWA<b1{4?+8@J)SJeap zl8auTV78S-v%RL594y{oao9YtZd~evXMd$_UZF-xZ8YcrIZqbvNiMgCX=4a$5>_Z+ z)vbx_V&6M*{Tx0qdAA-RbJMs6g^eM_wCeQ@SyVq)zJMa<6ZtHMOUwyP(ZW|hILP$- zj@K=)akktN70P;H_-e})v3N>UQs8tjvpg1#j#r7>yLECsWAlNA^=6r5sI&m5Q<RFT z;h}y<3(3S7&y}xs)e0s89bQ&98Fl7yH6E@F=3m8q=0wGi!~}GXMk!~`ebu25fwf5` z0|Y7ciV!oI6ed5Jk2?tu6Sz;jq9Jg0ZalH}TqmyBci<v1XUJg{YDrJd4oQ`c)?v;l zAyU!GxO1w(-qwpC6uoq%-?Ci1SPRUek+GE)*nMoK0`+y~Lu}?49P0HXPy1Od0a!DR zYi_5+Q{zYH$<i5vJh~91Tn{81xU9`=^ItdyFK*`N#~E(V5Px1$ZR*y&S1apwnl z@#iNsRyVssNasQuFllsa8g&u-h8&o{$_t**#(s8+<&4>u<nvA<8nto+Age+Wn=PuS z@tWrRE!(Csvkd|RG9jWH^&$;O0weQ*EfW6^O=lSpRTr&c7zP-6=<XJf?hYv_X+*kH zy1S8<4rxINX$7P~x<Ohx1q1{Ur0(Xs_x>bi=A1L<?7h~zo_DP$gDX?t4j67GBrBcG zh{_^KUb)OwDx3x1O1nm5v(lo>AH@-wDWI`OkVKMrCPd=cYD!(k;(uS=qrOZ<6c*Ml zPo{xmLPS6RDs^16diIp$s5F5l*%)TI$0D%&<w&JwqsK1~<Ok&j`_r{3{g%I@7u*=0 z&&^h!=6JAWK8i1WFriIKFz5KA_Fwx?DI2JWMy;hH-ntcOj&Pg1f6nvTGzcBV$V6Xc zUUlve)mQ6F3w$>6q^F{{?h5I9?wF5q&U>>x^i!-T@HYh(HxjBQrbDn8Yhsg}$K9X& z%$-;NpMw4;8a`@*bOeZ6(*6Bq!Z~^O=HjvlN);y;M1eF1t;Y-8Kb$u2e_Pde{>@+J zXU?q_240{%)!bGiJ?7+<(Ne)^dS9cK$NlO*Wl|0SqPQz+GfpwyJ;M)a#wGeM_k~eH z%F3+9bDiB1l2xvxhMvi4N(&t@MCyI^9m&i0pv1m)?z6T$%d1{E8y9_Y!2>}Rdi^ya z&wKSiSG7P$;y!D0yg>c)i|(-2H$dk2FH;5zhs-)%r@J6sR$?;H;vPSr98wmBN%%xA zjX9~p*06c98nKQhCD#3OUwuvd@!vHaQn!(iqnWag?aS0b2)(Y<Y3!RM3PSE~e2o!$ zChxCKI=0Y{a(i*d&EXY`@j_`0=o^?DL(SRW(;fahw?`2YN)ysoq!uIzwyght*GzBv zbkK!TVIseRDL@nMY=Dcw$bEN%|4vByvreU=gA7Hx-$!$XPudck6v_s;%ysd;{|@A4 z8XOk~&*J|!{~Fh=W+{43{C)KPpxiQiEe@Q@^`-Xz_X41SMG!&;a2}pl5>>x%8u_7@ z^dRo1X{11!n9%5V;HTsDvoE-R#Cd>R*ICzdJa5zGR3=O+jW*<TL?$eR5Ld`^v+-~- zaZmJfa7I<hSml?gp{<YJ+t_I~mI?D)CM}~+ab#YEhkQ)JY*uLGvDIH6q#hZ+8_#{s zL>`8aCtBX6WQvGflr25JJEqgeBkG{Der`@Jk(bA)%pmf&7c+snXE-3f=9+FGn))r} z%lyC_mW;^!=5#hACBCVryJLc^){}*nMhrl0Nj&?gUXfa5(en63vC6CuhFV-$f5Evj zS(TT(`@_G}_B7gF64>GSj=viGyt-$8IjPl7tvBE^)jst{+!IWMQAn$iDrV9*;<~-A znotz+I~BqrxqkB9Sz9YzsyX}WyNBP+$UGMRj>e;2ep<xD0Mo|`#bWz1S(w>zDpD<a z;-ol%6Yz$|YJ9s%jRhSDjNiYa1}0?xMJ8-Li_-953yeU781wU_@w0v7G*U*J9Cg`O zSkB;xmBAGKC5nPcGqB$6d_+qfZ!t<z%wrWZp7+AnI+<n0pk3tK#5>Rz&k!ROADbO6 zX|HYmR+VaET!Py<VmlAUzFxPeR^V01^r(X;bHiwY)g8-7tWX-9?wTfB9Wq|Ks6Ouh zpSG$G@eW^QG7}SWx|bZb?n&<$vT0N=SH8#(XwjJ}3;sOqkc3nE!|>8qH{D&DW)lui zQ0e<I9b>^FpvC^fGnl?f+UH}ybMxQ7uV;uqBA3Tl=#$iAKAw?%6#FP>`z7bp7%VB; zUAv;yXj@!p)*~Y`kQkwgjI=JstKHU|EB{PT$>++f$69_|^jBdso8HQsO#vr2(M<fu zwVw%M&|p=>L->4Z&Qo+&w^EuvY-pi_5|srb^P5qNvj6Af5_g+%txqe$I+QM&bwmyU z265qjr%)E%iVr-hgZcDq=@lAbR6+kFdfqH!J^NHXLd-c6`r|gh%*~~*r+tO#0n1k> zxY=uC<=Il5>leH-Wj%Ij_*=er!hsoT5B~k0PJc7nwLAO_GItf{Q`>1w8KI}0v}B}g zKX9|-n8Cr>6g{->{gOD{V3W1&=6r7V=k1?aFgp7aldImA-QT(Pi?tzMZO{u@yVe#_ zFI%GPJZPZohsXUWa%-J7t`2I4xqMhg(8=YUhs6-x&mX4jyi_M^zRB3f;xdtO#F{4= zSy>KuKK^@yo1wVU4%9b9mhtX_ay-1o0^;NO?_!W3K&1?mx}epF)!S^OLHad+m%h)b z^=Ix4RNDL|%Ba!xyKaU59!FyWGL^=TAs$lHh2J(-tJ?v3ll@wJP&SuddV*<c!<YQK zL93S;Kvev<IaZ^+xHz-7zbH^M8F%7UUWUV0#_0Y~*8MtOvj_7@S6<O!CFMuY=&w3= zzFI`nSxEz0ch;V;my~Ua<SRze{{rrdpJYltE&^Q&=|VS#XXAr;0knnqS|%n!ot0!n z`Ku>?NIB&xy^Owob;9>6GqD*So$#7eH&H90R#2fWsliyqy^HRY5q3NK%Q&;2)Q_u@ z@#g{g-)ZT&4d>)Q4OR&uhVIulPqaA|+9d<?(Fr<?jm|H}S&ctuRVGmAlZSR8lRBK! zHUKFaF^)!~Mb|e+%#W$N5tEbDNmsYs*h<9xXC9uVK&`LAu{?j;E_CKoPnm^$4)3)( zqp9i6_<3X)F+3uP(fyvO3i8>f^kYT%_m(7M(uzhtClP`2LyX-LC9x#jllb@b>CazW z+s_l}Wz=_tr+3w-D^F26ISXQQ41|U3z*&Yf;5D63J4`daa`Vs<L-ZEA@4t5Ae;00K z9l$OqWbC|qKc3rIQFA%Cec%0n!bZD07sda<Yb(dT8-3!Ym=PU}r(0&h8o4Tx)S7cN zNatkyM<(fk#~I{Q^sSSld>7}3Qt(~f8>Xb|NON<G{igyJY?{Sz)f<5+)ERzS36h&3 z9Oam9QD~e68}X+yLX7#zWJHsht{3eIIz#iK1J%WO`q)7GXZL<?36^djdW|~y;%pKX z2=*`Gqc^6yCC-FImgCcM#!iM4iT}PX_F=L)gAAoabLV+8cVKXN#_Yj2FX#CSC0qfV z=Dm%tX*8)E{sQk^r56PY)CwQf@pfmcpQ~R|D-!PqaB}5%2PRzcEo(Key1(4LVzREN zzr}^Jzmc+56V#~6s)8F{Tn;vpgjVWp?hmABt<-+-GGz4oaM-Th5TxT7$l<VOwi~(k zdX3Xqn=H%xeVpbAGs^UZYbUETip;<V6N#PsPZh`&4G-^1tj=}e0c`42>KD=X7X9Ym zO*nr2jr4B)xOMpZl|uMUU6LoBk@eVpo>K6KhP{8@MapemUVH6dv~Ey-XnXKx<w*NX zZy&xZIUHQg0n)w2*bQ9+9<+xj^p&sMwCYha&hpmv4ADoI5#O4LYGY-6qmj4#yvK{J zNhVAV1WAMr8T2J%JWuTvrqV(XiqINg4@mo^sLo1paUf8NySzI7Q;<8n$|4hwB76DY zCSHn%2xI09zRLS;a%;D(9Y%x5`;?vCZhFF&*<XfEA7gMsF}~mX_X2AZ9NB&S+-73t z_8<<e=$h8W_Pj2lv{eBoU&E>Ld^4-8qKWONY?EaD1ItZ%_tx9}5gegn=eNOFhIUUx zy!In<Y4Arynd_LZbEh6F5>wt%{lri@IAR+C&EV)Q?bO73`**{`+0>k@Rq-3h{1z(| z-&!t{xO<YtZ3MRL-XvnwihB`W4scI<`ql{FZuxo5LD0}38)03ziL=6BJp5z+-D5S( zGr=ND6IG(qbAn4wr_ob$ra)x7>2p>wGD5jB88T@RAg|ZS@nRuVK5%4-@(F_^(HA&! zHO7t5eRs+PQA2hPkvxyhQ0YHeQ)1OjPNTW#q04mpj6K_&*n+QqD=YaY5!!9;Z?Ra2 zrwKzHipZxu^+_L#8g!(HFt)<y#7{p4V)a(!89$pD;;3X0{#W@%DVT?`-6MWJexNo1 zg_K_Shc?sH1xlrtddKaDrC}Wo)t2)e<=ybDn&2&cH+GmMA0%en8w`Gk&yR}dgh3;u z-Z3>TWbR<=c%$Fc(+D$t@R{Z%d%;G1wDeNEguLMaGk}t9mPKLF-MT!oA>$(~UG=44 zA=PWIB9SA7k~PAAzDXZ43bBp^ONRC*lN=)B2rXxt<U&!IoPC%ZgLe_@8$z`6_<uip zW5UZO8|cf75P8b*h&6N!RUsojr$>kmrnlQbo{yKo8XX<hed<(|<5TtLVd(iR;X7M= z&gL)vLB+-CO}5^d=x1&2=Tuhi;=5bauvor<`V-8fFFyv=ADlfwYBFdFWgu`wSwkR| z_y+R`Jm$uZeI=D0i4}DIT!1`|8Abprr>5R3%v|bJS_`<OZy&%N?Z>({ush{e^c{bO z^6zFM*mhX36Hi9yaK+*!Qh4bwTygH9J@%j2b>j4Wz{5AWo!c6+^Mb0XKP`JHUuO@> zYmy2%*B%mAf1aiMo{C7qieMIe>m5c;vEe}%rXuu%BfF9Y&IEg9!qUkmDD}c?T;ZNr z)hESHi<4n;G)?%$PR$6?VrJVl)3m_UNU(`g^j}Jy7sC?QW`%F$qa0^a<tGZs7e<d3 zgVdR32BADrv$)Wzu#oMPy5V=pzR))aqu%~?^b$c|QN1WNCoCvB1<PvQ=z1MZx-x3h ztVr~1N#0(xR~*W79KPuF|2h2)BlF-;dd<Bni~d5cMnQW~9y=Q+49B`)!tdjXnM3Al zZQL<qs#)0`d@aJa_a1?Nxb2tUH%*_a+&E{vdFxA_+tGd@jl<7rVI+)g0(At4k?KVh zx(Su#?qgyg?7mZ~#5U0EX#N9zhoKlQ<hpG__(#RTmd~Ad!o<>MrV<30RDnvDL~@0m zJJ=ovn#I>e$mKTKtlbJ_uh%;GtVc;kMu#f!TRJ%uhU|4yey(~}>Xfsbn?nnWgh8C` z&^>}rz#!4lXSjOUF4=2l@3lFo)YFq~(?wzt3?p*6J?&JUoa*4|cM~2bfv%niXWh?l z>DU9)ZUs~F@As41Sj-9kr8Xc}68V^IlT_Q(529E(2*ky0E+v;PlzKxN9>UaNIt4Kf z4Y4B;Fo}4;FbUlMJPAeCRG-fb^r59Gv)b_NwEX;k$;ykE5I~Xm%tTs&5biV%{18O@ z7XAPJ2-qQ7P*siZHI^p=TeJf`O;~)>Nz$m=|E7jC!?-LOg$N0hq5{J&;Ftd$m|`M$ ziy`h5HnT*j_Z5WuPnhTM9&4M2bS^u;bt+!9zx8Cct;;^J3?Tv#O_{qzop~O|s#%Ng zPy|Q<yC_4CTHGc*=Z*G)xMO&6EyEC0b3mLR)^a}m*;BMyTq*k<4SWaYXxN$Rwlhk& z!vY=eP(vZ{iQBI-D0=lwt*cpFB%AmXawwOK>A}g!N8Rd|AM@*AJRhSpL~6+(9KY<8 zP`xQXFEv_9fO9mJGJP1Aoo#*Q&`T(M5!D40!h^a2s&ifTc`Vp$rC!qJQwEYxgGC^7 z1c^fX^52;sfB8y|7PY92w4tPxKa8@~D{y^^_AnY9FC`nzc6PP@GEzP!fcv-$^<JVJ zd7yq0a28y2A_nnKi^vLHn*~_4*7x7v4-RqqkyjP%(ZT<=(s7+GCfQB;Q%72=0QnA5 z;$hAY&3I}ozZNJKyQ^kk+VpWKKqkJcWoFOr;j*{U`G9DtC8%b58OGgF^vn7F;@QQ{ z^49*z4x8)`&%R4kX?YeG=;}|&<YD3(W!ZnhW3jF^&{YllGEoR~QeJWA-lrnU!^`Q; zX!R6&<J;WnJEqdDkQ;Bx;!k6KQ+aE<)2f0$2;QbJ(poE;rm0m$N$rki+l^H;T?EwR z&z9LMw6?ezJTdqzL~B^9aYfq|n;q-pzDa4bkji_8G%-Y+VuM5-P$%4z)Tvm&jkkRK zqb}*Q^NO^(*MSQ8oO8TKhU{>Gt)(XRUWg#dq+Us9v3F~re7RadajWW27osbC))U2- zrL2&|k=vuhFDYZ=6K+9KTLv|WP5~kBy;cV-iKHb|U(1Mi<g>an{d3q*wx9eUad&+q z1854%LHQ=5cE9vs4iF}7(}m!y0H@U?pokSItb5NT9xj((Yq$BhzHIm7H)``q+1%Pn zpM*GH*z^B7PY_&He8l3_ndp7yniakY<!F!P@EiSQ3tPkzcGw$#SMt=TbrPt?va8;u z-Z3z1P(FT~vi0Iso?JuY(%ouDpeToNo4)wWpB0badU8KUfWp=Uh<b*pYLAC{^<e1l z?IBsNn^8dT45AVzs0`AkV^>EZNEDwUMyj&h!n+1g-E8B@_}>7aL=BwE3y7VHib_3K zrO%mBi(4d6#}%xG(^4``u6{UY*P(%8(+o5XfpbIn#b-x@CgcUViuc?QpR$vMI$0<E zEFKJw#J$^a90^4**{BQ{#8Odks8m3aP;Dz)>F>tlL%-oOZ#f<&ML=5%*0Bnc&W_Dy zP|dyoewnYgrWeDM5#RHK-KQzNSGt8GK24{wYQfk7n^B5(f%!_8nnx&Vu?Xl_&anM8 zzveN+T{f7M?IIAX{Ft^{Fb`p3akQoa&MW!`D*J2Ri4?+VCvu%Zcl!XmpY03|2D~+% z*)tPGIF~HI9eB03a~_{C6hJ~GknwAt+s%J^ho=cRkge(ded2XM@1qsznT3@d0MDHL z_t&6!-srf<<N2yo5%&)y(xrr7u(bmyK5TRd)iOmgkyS$c*{L+^QC;_UH%1T)C<*N8 z`{$Xu6Dt5o(*iFBJ?k}{ooY{^=aRP8z#{bQv79URHyV>v227?aco{I9!I5i7E{qH1 zgKW~5q$IaL9vxmNu&+#388p0dH*B=U4ZJ-YP(Cl_l%aRO`!mnxl+dy3)aiX<{Dqjq z__Lp<28LkKo3c$OY&WCFwKCL2VI4Skej1=Z&^tZR1@6T^V&8amB=yEr2IAXZFVu!I zspiPNZvT_f+UBw$^AFf4_QBCA-y}*5B#_%U(t$Ju17rZNGzLTecQFEl?qnD;#y+@j zU2<Q7>NX651|<;+-;BH-VlHU@j&=bhqpTv^d=^j4lc=62RHO?x`xl}kcM;r+B?w)5 z4*;G=39%R+Vhx>;cYNs*ngN|qEIDwVt1{3U6%0GIHQNI;PWeV}bY<bwR(e~HC@k_a zbt_I-GcDK!$SK?&)jVC9bzopCHgs-@%;B@!6sQKpSd9XSz}b;cwo}E*SqDo^Y(M-C ztDXOZiFYd(mb-*Iz-=4nP&i%cma&P<>@xA7G>#{W4b0g}YRdnTTf?LZVXvB9x_4%Z ziM`>hcoUNC8`qhD-;(-@`GM-z7ObYvzU;eQ9rtcL9DiBE{($q5AP^Rlq;m1YeM27( zBZrzY-}PW{GUUFFL?`?Je0Ys^GfZT>*6~jiEXlJ(BIs^;bO|eUt1AS=g>)I3uKNHm zm0lR`_wU=Mzui@eKe;VAVOg{xNs3q4qgmX&Rr;UMAky?9YeYm&G;+2@k1lNGnJ^nN zqLf0!XkWdTT9-$Q-+KtB8N0fv$-I-NPPnW0Xzrq^0cL{M>CeOqxKl_da70XkabG{{ zbFm}rvZ%Xej}FQUuuJ_a4k;vqR;=uc_=WNXHm^t~Ik&ZOSSaNfAy*4+V9q)kq!D0A zWKf#gePmLx2H+<uC$flVG9MgBPXV4gz8<fiv#dMJGhoP?A21j3xGD2|j>H8RW{;pk z852y77XmCv!qVq%e}8zF!=ivMZMr{!9Bo6CH!S{z|7lSO7B!k*xZO)I4voj$p2s?i z4uLuhQ~)PAJ8$vIC<YsOA&3#e{pYATM3na=q~GhB!}f45^Ktul%=7atWRPaRyfy*r z5^*4S0-y~Wjf$oF!k_Y@nIUw85qp8D1{e_Bq%K(;ak;gCb1d2#DP=q0!{ZCO^&u1S zuvN}2QwqGpEkY*WR^UPxxGGGxd~F8_l=DEBMlo|JDk8vxj>1o8vA43r6%&2NN#30` zX|)?r>o)Ilbj1<wGPwi2X_Nl{lD=ZHP_WKb5W8^6V1`s0etAmBuAdBiWt9Q8GRJ8- z)45t@Dx^J88q+ik#}OJx_(`4j|Kd9CE)8X<C}4bky#vuzGGzn@gH=gM>hCyNd4vf* zX@-Sd&d)3n9VOiRQ34<O#DvReJCEz8+)q&JR9;VReeoFSytqet41Cz=LUdXHoK|jc z&y{_3e{*sG9>P+v@jOAK)uD{4bp>1KN5F8|&-?%&;+)kzHxR$mF|lLef8IQSQc$3> ztsJ<5TMq`=2=4-*m5~I;5-BwXtq+xH9a4^n7%?4f*e1Btgw*Sx=mH}MB}qi*GX*|@ zS+F2kI-TO4E;-bEOfLkG4YE<<zN9|opHe+goRYY^&>|!q)bj;~Uh&)tQ2orUWKil( z1Eg4fzRxDw?|wjQ&6?x0oUGQK58MDJfin1?k}}m?re0@Y*SP3LrT&n_)Zo$-no(lW z=lz!<Uetb;#dA?lSc30-XJl_NPd=VR8u)Y@>VIIBtq`A(jOzIow#hoZjxs6)z0R!{ zEE!J7$X~Z1zX2<iW*Amp&c?dks}~p1SpTBEkS5-OJ;t1D`0N3Ekaid*-9}e1@`IK& zREv+nnwL$1uMvb~l^<-a|Bjgm=Q9r`A@KOm6$PAaej9E(8W%k&1@|+X(==ELKFQ>P z@sshp<L{mW`?bd1did`4Q0lge(Cot`V8&9<oVQ<CSoD+syxug*3?DG{!9f=2sGVse zJ^4y{vO0{kAx<Amoa4Q+1)j36$cYG_2V<}|h<fUnH-azMyTgM<#)4A!F*={e<EjYr z#T$Nc*2$4?*lX@|8zbK>dMI$M8ZrUr?ce*9-e=%}_x?+Y=wUQ<kj6viRY5fC6D}#1 ze_qE+xT_C}OL2n<a9c5YMtjG$Gix2kWGMc`A#hgDAD+ms1utSi$Pm9uBnMj3hD37- z@XEydq-`ME0=SP_+;6r@_&<y&(RCDdE^}i#^1p;1)#JB^k>>sS@mhCb0Gf|O1H3m8 zi@=pFyJo2(Ps5Yog>_kK3{0O`|7mbCXp6<2H@`$D=;i?<#!G(@%M))Ve?MGj8g{vm zoASkNp?SViPd$=O+-)0%MZy*G@9#;Cgzc#-uiu4jD;Pea4G}(#I&HgBJYz^m*z*kh z73V#xgmq#UnUMc=ckHO6B=J84kn(>a!PlUZz7>0oR(WE_(Nee|`>4-N0j02;rPR(R zY;o*KL4nuXl@#tW!n>y682Dd<BLJ`V{Tua;bCrwwk?5P*xpFBG_ZJ8}`$jyRhyE~P z68I+_58>f@j~bY*iB$8&3^>GO8ZbHd@y`DNcFKQi?9IY@3l0`vvD&|XKh}eB!}7j7 zC<X86%gOt{UW45aKVCQVQVNSK+}()F`{D4NAHxgK;jg3ub)9#Yxup3&mp}`6`GvL@ z_9H1r<HXmG4BN$FB?VjRM^=Ea6)9bx_TsdEnwI{Gik1IGN6aWOXcWeFc4dqGnZo`2 zSqC_c)lu);bcfYjKwFB@`=VNRF+Ea=t3DV6<W*w1+3D|z$i>tz6nX&@QE<e(dFYLp zZ|ZIQL8Hj>*rnT~WzP1n-G=#J>#BbSlBie%7q&oz0zVU4q%NY@@5yPJrRCLj|L@Ax z0?&JuKM6Wv_CsEGu<*B)GXv0&q>lFyS8MQU-hJc}{{<bk?+N`AxwJkpHv)IgAoa;= zeqAeP)zuB1$&OEA;mR_}{qR|5)aB~^-fjA!YoPc<SMDQ+R`jmMjd!(>bVh>XAN@ZN z`VKo{K9qwU-os9;js#Il%hvGm-C9@3U+eawh}~apC0&3Ku^(<W4!ui!dshLNm)n%$ zu#pfV$ANKj!`uK=EbLIVcOp}T(&4iT<}bNW&8(-2pIw0Uvi#ZKjQtt*Axesfv+mMJ z@&E?uEqVs99u_@A!0m$%%HA}t3xU}k%3R$Kl=rB0(I_%81ZrHA{dDnQi_`?*yb`Oq zrU{6(rZExQPH}li%S`Q*xE8xfEc)Y>M$Fj13pVC;>D8cbkqBI06^6}lUB*;-{qUSO zuwUiLmV?p+2-;4DR8dI?F8#_(q=|Z9=4kFRQF<^r_=aRzP0XW!(5=5d5oenU5({Op z#WKB*Q3`ylW%ELl1yML}!Lu$j);~$vyH^UPwJK+wS(g<a%KxOwZlGNt5o9RpRUi>) zcXQlTr23eppU!!uPQzEd5*PnfHuI^G8wbk%LpGoqzeW(<N#dj7_Dy40Wp(#dc>^o+ z0e@*cz8TN~@y!Ia7jU6Hq@m^ce#*iPy{HWBAKdrGL7K|BGoNm3@zR$9<|AK?rJI8r zAdMloAnQ(_)&Y}Bz6QYt3mCk5AnKBq7{yR}B{`WxbcCyl(HVG)W6QymcM=(<fPkxw zJ6USh8`bSi$5BFtI7E;C{fkK!F|;bY3+!0heK{8d=3RFnhW+S$vKoQoV=`r16|TKB zUh1;Z^90iacz<6ZZLIDmiEKA_D^N;oH}<1Yd%;lI(!5}bnxl=qMB?;4nfD16No#FH zOAfOiiYTVy2-z||r^eK;;`-ul+@IYN;z7fnN%8S?(=i`E#-^pPcyd9toxxM$XHQDQ z$wXvf+F3f%vh>^Jh?dy{1NpA9P$rIrLHnlHw_&W2NtnggvEe>k`z>Fjq1+{9$emlj z-O1Md?4~hhZi=#@QAqac00e~DyWA>qXQdN^a$S`)7F*kn6i9+84!H>|8R>$H^(pvF z$nrW*YRs3~w&f?2g-<$iuP`DV^Qv<jC=Nzq{XYx|vlchVAe(QQ{N>3z8!kw&a9GBs zq+`w%d@M!_U4AeWU#1R*yOtR;<Na`@B)+zDxH(rZX48B1eGEI#`^U3B&ixSISFLIL zE5dHS(HRHcc^ppnvt{QQrR61%<GX6(FYk^y@8mO&<1xkzXZ~$t-wTtHqXXo^M8(xL zi26sl?$^UZ()Af2M9QR)Q$i3s3H9^C_+R>mC&`^rE0TWF6!nBjbqU-*rR@R}CFiZE z%2dCZpC!<zIm5bCPQDSFT;$*|{a~U)iWG+!BWJ6yGs?y&${RGi-8%llT4bZXdCF?J zx3I>YB$;28b0$L}I(Wu|XC<TJH~!K=nF56`|2X(v-q6bz0b<|UZh02PDZBJfUEA(O z7%vXCelp?b%D6y~-2!en7+rQP&}*GQAj+<BRV;LG(=d@k$dy$0y^<N7g*~LN`sEH& zo>kanf0zozmi^wXICs}1Dr4IzMEG`l>1^uN8)#^)Gy+@cV_Kw)k~x(szc~tGSY?(; z_zH}23XS@$NHKnWvI8|zb@$n*3RPIq`#2W4br8`j&zKXuqAA>8%M~Cz&oE&`ryb$S zhz9vfqbbPohiiogyHY|!))-+n$4lGWg#5N*M{IgM$!X?7Y^;3P<Nm)|q9DPg%q~3i zT4C}7QsaiNRcE_S6~w=NOQFp6H!zLaq4uDt|JJZOaQ|l>6n~>3s;0~eHV4y|He+!K zl04?16wJ)*<L>rWh!?%l2|`EjI?bA#;M?G|D5KrT!noqQ*W@fbQWdA!MPI(lc*9(v z0l_e`hN|E<BETFCXrRJ|jWjlz&Zz4QK-@!ujAU@|s)l9bw)F}^UyLp_Ih5bp)SsOz zvIs#716$GyEEf1^2T&r&92K`?34cS7(VO$in_Mi;eO~Wh))PZRLgo<2RQupMn4Q$N z2!^L$38cm3kZjx6Mtnbyc8B5et%Fs~9`}@OkU{Jp3uEmN>R`8w0(-94qJX2uOZ4a$ z$wS$x6zl;|naLcGDCbpHgqv6@BlY3XW)%nv=Qjtgis|G}R+9^_1`3{KA+x;yZZ?Ah z<Hvp-rV`>rSrGIyM|Si%C#!V7K})^21UP4r#0^^tv|DaDrZOHbPWP@p%)Z#4jWk=P zs`wEKw9+&=aQ$JX><Q9*G}19qi3mVnJ_Ek7(j<Mh3SKjti#9s506E_tudj0*yjoyt zx8ANK+&Sps&f7B{M>Feg-Pn||9jf9KS&WA$2M_NupKY`)V82y6HI!lb19E10Mv0An zzTy+{O{Wdfm8_I&YavJwGU*r6dM`)^JKWOI|8Fbd`=1O7*m~|ZZy(}P_2h~M>fC!o zY+3M@8fnd|RVb3GZ#`deyrnKJbNam+))uy{N6e|oz8uo8o=arOJgowc34nsO8uDCd z-KeqYEZ)CX_<Vm`0(VF>o-Koc;nELEIk8?Z;sCr<*i<~tg5y$SY}pwv0dC?c`INPc zkbDGx(V#5Y_!`&4?miiPoedliuhFe06Tg_0#be}DdF@ZZ88eZ^9bTqhK=qiYRY!%G zNcW@I+$WG14KObM-m%=HreVomoH>|DL&N?>1wD=ncUnuZED+CibX1ma8x9MB2%!ou zc1EmUOl*e)k4{Lkt!0vUwS7a8W-9V@aO?Wgr$Ev-+o%)nf0GKKqRis8scfnXPDn86 zQ4>hQ2a0a!Hms`((-&H$T)PvLdk3L)q0Nle_(y}_g!wsx!>|IerXKC7?)I5k+#2TF z@*X+lummFK+GzSO8do$fF-X_8s&Pcd@Ll~iy$HE4OLPiXg%0i6_k-3K#R+w*foOZA z*Mycw;V8HQf}1+Hs5P3cxDj}~99W_o{%;dhJ8N5FN-`MR&QOHksw~WU3Jkol=uFT0 zyGk$0;5{xiHo{~1aO+;Lo+WkGd7ne_e)N0n1`WRx)(-|z6~rX9L{Xenv72OCf)Q&6 z*QoS<%e!G@X__~<AqI=7>JoN#(8p9xWh^<6Z0dW&G&Q1SW6fw;=K|k5D{6)6mh3SM zb68QrY2Vc;#{{C@DL1B>V%}GNbJDL0rB7fSla120-6&(_L~>?yQT~}j`?)!FS{YNa z@d(iDY+jqFX1#-_Igbz9sa9m(5yHf4h+F+uS!zi(V=c!MPXCsz{vJh&WnpIeM8{nv zw7homsiC8_Km~!%(M(y1O-_}U&Yr@bzJ?#qQnt~d+XUD;9|ep)FCplCneQWTbE^E& z?y!1>;56nb1rkbf;4nck5-JNA;oJ7-$vkrChiB`24#Fuh_2%E+WDFFN-yCirKXy}^ zmYVyFZY2hpcjd~;;diJ|o#?-+i%((-U?=ZA-jpS!9)Fwkj6OraAC~>Hw-}k2!?Ne( zzzq9yxIe9o_0sz5Z^j$sNDWd9-e3=xi3Il|N)GYy(TTf;To(5TlEpMENWdj5`V?=u z6m3zrs!)GUzwgXAJHD<xfSscI3Tc4bjY0TuPA3=1-|H&=VAUwQCEG*Q(UCg9a0VNT zSgWhV{@c)>O<^|!_H+BabeRZaqQ^VX0oEnQk{6)<vcZmxE=T%<G#3KxA+3uzXS^A2 z4%<&?jG=JOHhAD?Q3b+KZHrLC5}+Qidt<t=thrh#+t9fkx|<k?o@oFMDvDp|3|h>h z$tj6~%ySqIXb2?BP)jGK4@2w>Q^^@m|2}YLF`&s5v1^Ins86I}%G2L!T@U0Kl_B$g zf6IL$_AQ*?uG+l*Z<RG8BLs3^*)Qe%$sBeVN2Ss*t8h{ZlKOb0mILIFNQBYhS{`z> z2#ubYM30m9ztbex@^2-B?_F|&w8#I*W7CCKqjBchg(0<Ej$MI<9=6n-?ibNt64P`E z=3p|;Fd<)Q>}ESBjKmGevdA8An`!}RI=$;;PD8Dj3{mLs+St$(qm8!q1LkzZ^&}e4 zQSaN>v6fL7lj-1%xpLdfEz^d|EHLi+dThbqrNW>M6c)2FSuS8KKGMgD&&)j2=(JiD zh7EU+PxH;Si*}q&fAiV6Ri%xBA?fUBJ~j3_ga(wG9{KfG>r&KLBo&D?)MoNAwSRa~ z>AO)4K?}m(9=@leG-~Gu8N2J;3xymA-`we}0@8Z-tQm88=m!!K%c7G!j@|K{!YaSO z%umHDUN${xvKo1_X}o%87ldyW0=E2|EWpP;aolCY$D)SHFb{S)zESN~iS0eUE`|!2 zG`L%GE*C$0vS<xaV0|(4<oa9Kb|#F@)UZ9rg1f<pvD0T$xCWDta+QVOsr}E#PpBjG zsMJzHStMR0|J^lm&<vMxma5e#w(e^R>aU#c5ziRbzHTLo_M{VL3(|*Ro}Pa1nM2vQ z&u*;6LPWiGf*&CYs}{CQy(EVTw=Bd9;VI^>+Z7P8^C?2>6q_4JN01<{p!rnD`V6=> z5oCzu(T$BATPSoHbw6_!%Ca?ctVH>pWD`DLFoP@i6#B1WkwxZkE-?gJ3%VY06S8VX zP+2+EYgp2|yjf#~JI0ekNnwIZd|3Xl)=)es3QVpW&&{2tGEiB9AVp((5NX#G^9@1* z#x4VHUsyZav>N>>$h>Blk2qfeT^ST7!=wh`FdS(n%Ohol$08RpG#MY-!L8^)!un_f z774l)x=R|fj4Z}Llv@qgFrBrOs{P)BbqU8Dg}sCe_0YF4_XL#liQa);PzPWhY$+MV z?+0-;JyUtEl<-0jSrvx%v^82Q^=RkPGS6$qbpd_F4irn2-QO*BlmhN53J7D?9|rj! z^CK<j{q4Cr=pskh{(G3sPDIVm8~eOO*nM{r7{t<6Wk-C7WKAdX#g$zEgdtdB8g6m* zazl{UV_)O_i=loZ)R_2rWLE53co=?L@BMq2U;@kxyts7eC<%Qmo=P1<w;AR54be+3 zFDmHdqr;pev|Q*1O5+LaGc5zG6d6d7rN|Z2<v=vQl0%?2jiDhNm*80?S*bM($Fqb_ zVOFlngdxB|%{BfdZkD%np_G?DdCv{+hG43=zn32nHo@#)ql+uv`16lu@5v>^S}9c) zO<T`F%Q)8Kq!%tqA2S9lgoFv9VKN9o$#DyRTX1e*7-$f?NN^48BC9Ugz;|8-TMx(X z;0B&F9*(4lIDJM1xdza<Zcy2hVP%qU6Y*NF)qM#utIRGGD!UO34H6_2+ItBayBR6b z0~PqSl|)ns(rFwwUeD5|V00}A)<Ov3`#>}Y{)FexP#b#!EU{B!!RMv;nnbcmch$p8 z)15bZBA+<cSuv9q$BN%w%slb629dAKtc9gF(#(f+V2WlF68=qiSrW#nQ>tY=E^7(W zJ5DGe{F8~9*&dyPv6S!c-EYr4<9Q~$G8g(V4NJAY{oXlPxjxqppAiweS`SxVkMqdy zO^}S>Dj4#8K*6RUan{Pbjv|G0Ad_MTJ6iCfE%ztL*-X4mwfH7VF32R%uWE^JfBY>j zuwfMK$ve+O<H~S>l>(<bzs*s_?J8}h9k3Fy%>O)1Ns#o4q1JXzW5jsT5wN|j^8_1h zOLtp@dvs`s`<HN9a^>Jfh&#%{!*%q+WLBPI?}6hi3-@(dQT-<O!87ybBM}cbMOy(W zuXiJlwGtl%)5$_<?Q%6r`*<5A0ysr;_iIo9C=OX(*MBv(@kg}rwYn-u-N+gy^+F2H zTdu`ZLz4gEs@Pn$^Mt+oQ}g?)0M5DL)xX1!@bmnF-dXdn`D5ee$zEMjdy8v345B9w zBg;xboYd<G_~jGtO0)uA9q#6S1SM@;H7}yg#9D(@<@=0<!44TKVav;9V+f_35J83& zj<otZ$Xkkmc##3~A-25QAX)xoR4iqd*Nr8pA*P_{>b449s*mK+skCNS^*Y+N^ge5A zMUkl=AxRwXG3a0UDV^mSPo+8zYLxT5t`HS$#4QnT(e>!STXo5l^ULKQPq+Fc><PZr zir}Z6_Tg`zW`x8xh}b}Ij*<`LtH!-ue&s}zN@!%n?hUM_I)9I6Mss|K6#%yu%{n#W z_I}r}KVPJ0B(FD-aZGod(I91L_$}>HKK(+%hFt%*I}WiH!GxFw;ITEgFqi3hPJtnp zT|CKoxYy!-@VCjqj$HwpVAuybcP_*4=4dvh42%K}Q1f;ygiu+#XF}f3A#cO$&WqmZ zCDOd=b5l7LA~Iiio`QF!pL&;R5+l?v0mZY~8IH|N<+)!nIE^`&RWj0gnPoi|qi>Zc z@L4q%5X5hVbf38i)c1KQ1;6{Un)4yFdzfMwx_ljqe5}M?<J?avp2J=fA9$p%=Pzor z>g5uIYF?d8&W)4(t0Q;+)C`y;U}!?ICQM#xOL8}VwLpH7l4|lnZCu}xfqCz%OTF@P zD&8&3J^v5j;A3>5rdv3?m9}{1E;R-Izl3G7R)F2rFmfSb?&f6Z@5}qEfc!wYU4KKD z-;VImy&^nw1XVtC?)4km`}`O)4%KBKa1C9l;SOs+nN+f9gqwISu;u30jgd)3qa&c- za7bm;0xo!k^=)6|;C=3HIzx?3jYJ!mH>u>gt*dLVNI=ep5+UHlo9P$?e&XYPh1w&1 z>%De{FF2Jp+I^W?x6+xQieX+5oWGgc|NR|PZHqJ{7;5WQYb|G|sNaq)J+j#N@hKfJ z)>KfZ^Wn#4>a&Crv20}9yz+E`$CY}r_Z2#cVTi(~rLc*rc<Rt#e5%}Q!RlvJVJeU> z&%pqC9liTUs9_ja|Fa|M_eZ@6i@xyN-EB(b{tK57%AN@w+TsC_pHWL*NfpAOnh9M< zw-P3U)|(c--1yQs5j-e}5x4U2IQ#mpTy8-cbOler4t|Knhh&ks4S2Fw#-}3WjjTk1 zfzQpZ?1NG<XlZFlSQtWmpF7S5YfLYq&75MxRnescuxAt|>#vF_gkQZswc~#Kt{05i z7Q*ff9~+kP3NL*B^i*s%;EExicg&tvKD};2jl#id^YYjGE12?Zxlby<T#?}+CBY0) zudoST!dzoEkx+LUj%MIve1ZQwDh7h$t%Ne)fRZ*k^nM_&daLtJfWSzJTAB{QNyXnd z1cVOL7>G4tDq^`(A*WN+y)x2FL}cdIjE)bR7@J3itU1&W6qc~NhUnAKL$$eW+j>_o zOZx5=wx~OG^VI;Y>;S9Q!>2l!Yvh-7Uv&ws?cvQ?cp@H!P~!tx4`RZqkS2v3$2Pjr zI*wQiLp;*4zV*=wnVi-{9twXAOot>MS32$qRWx5J&mC<ba<1Tp$a?$U7sw{Y+m=mJ zP;49CFNAesy1kKOtKQyo!|!d7eCZ1Bkc~zNs6uxoXa<(P=ucV)tfo-V<x+JpK|6Qs z6xcQ|WV$Ntvd=qiSn?ZQ57eT3LviQC4!7G<S>bM5G%&=Kw2-ggH(q6OvcTs|#xJ5D zgONrV?1zMyiF(EBkGiBdt%~*R(k1S{1*2lNPr1%C??v-q@lJwxV`B>&Q;UV6Zz`;9 zZX8fm`m-FXoL=<dn#HG~J@`;z-?>i8kRD1`MvSE{bjm;Wm<gbN#1QPe9#UmWe}~=n z1%-3ga+(_om^p#w>a*STIb$JYlD`#mPTS)HUyq+}Y8zZmvD?-AcwG+@zU({Uxk}?x z@)Il_l56v5eM1aus6&=w{OmReTk4w<nQuUwd}fSLHHs@(`5QvHi;w0J6^=&RZTk!- zcv*3M?5Egz_N5}(ox0&W?`)HP+I1cJa8}{Nc|9gwmx}0*Iz1Litmj<h1J_^5r2-FC zcqnTrg-pi}y2E2%;>Cj=9)|Cr#!Viz>Iu9`__7prm=Zz@<Vv~e!ix;!_lq``(P)O# z!!?1^40@$}HVI_0V?v0e5xlC80^gOHa~a??;X*WtvNHb8@cjQEC_%{75%m^UHfZGs z+H4eTR1#Zx<QKee!7S@3vV4gAyH<{>uXr?al`~b7HvrqAH1Qv+S~zS{zi$64PIRWJ zbhn`JGmb!RL1aM2;0q#>Ek=z`dDPR|*9TjUPmQDL-%9-b_4Q)zbi-pWRlVYTjD5ux z{ku<V=f#T2fLYbqGM=&t6uV*V>}etjc|sh|L0n_}ZfwX$AeIt4IHocYjNtaB#S%)A zz@15H-{QY5brQwk4ShR&6E<AMU00Vekb30ERvi;)n+IpYjQEUpJDIR25<}MinS=nN zsGo)43`zfb-bGaCcHrW;QeZhxI99xll_Kw6`bi&efQQgV*7PRBT;Y{T@OvR+#<;(X zg|t<Szvq3Nl>!GJxwBE+g|6NApzHV+LKqb;bgWY0`}g_lUOZ}JlP*WWH-p_dj+V9R zY{R`wJ?@KXS8cJZ7YvC(*@V7Bnu_5Id%y7<xTkZyTw;En#h~XXw@H{OTxH*4=Z`{G zyCv@n+Z~T<V~EikY)0g({V!Y?H!;8a93<fPyPtkFVkQ`iIV<rT1yOs-(%F|Buy?P* zve0=H1JKOm(U4<Udd-puU7blqYb-<a@_g7d-0EM0TOyEp1m}5fchL8KoEZ{hQV65R z@lWSrD~gcFB%cV~sEnO$r8hgt?{ds(0@e{H?EO$hPss)MD7`<}KP?XqGG%rdhiHse z*5EZhe`>z0b|bjfthV#$#*zs~XvCg6+CIT~CTH+ugh?V$h&f%u7T%nS&%NASxk#my zJ5fIgL)@dn=N<4F@2z(})cqH^=6;nWI89{i)Q|cbS3r2xN?$Qt3PlxdiQU@0C6&wd zlBQAicmI2UO%aAOhSZTQahOTgXJZh-RXQ7!QO|3BC~X~Y-@V8ZV@~-Z(7NK@8KOn8 z@~wGgY>tSFvEh8vOjV%U?;QtqVWhzXMr237Y<>=58;e;K&XRN`5~kVbFXS0SO4}HE zMhcf3aa~IHOoqLk$6p(FRIcjjcyVN*`rk}t9JMf0pI&8j$&DmJv*6C_jX0s9daaFI zrqd$VjF-W#nB*BBlr~U}*zrtc;61|`YKlYHwTYz&NJFBEtv#H61EMDym^Sf|Yw_sS zqcX9S8+q9jRdj+(^?m%zdsQ)3W=f)WvW0G3k^(`bQWp3apn+LKw?vjxRME#5$8fqi zi*IVk6kJNDX<eLhsLa9t@r&+IQ{78YNy#A|)`{#A|GOd!V5&0aYheK<R!E5yz8FN< zcM6^QNNaK6yr`4_vX2!eV>gGbyY4+s9VNwebsE&>4JK%xAj)Yjj|36@J&#`Gx*Y0L z2sTU<@Xn4*esP3~P_#`fFtXI^lyAC$@1#t`Q6aI~aOQ8p*IE9O-@cRyC-lShjQWgS zqGo3Jo_3ZN62qzg1yjd-4vKqoaF*}Rt~VPrD_FgtZ29^JfB0JN@)hOvbToBEg?P)_ zCyKLQSjL0r-}?<EvVY%x$`uKgxEn*mICSX~QvLfoj9OrlOij{MfGVPLV1eMZ=($!h z<xkMI^YG4lRjF4Pj-vt<tc62)n{0e^f}l0y7T8}6&XGF;H!AjMZcd-Aqki_mbyB%n z?Q|(<`6!Q~B79XsAFA4@O~C1u-*6U=!=$6AK{~+pYMel1TKKw%bO1XGh$A88mBZhQ zBS@~=YBQJ;^;`xGxe$<sKOCyK#V2gM;ul!R=v|A{%Sd+bPc6jKl))#K;?ja1V|T{m zT@@X*9ShJ^QP6fXtLzJLmxnj;z&zGj2VL4``|7yz%^J?015)B~xsWmry}jLVZ;v*~ z^*!G2c=@h`yJLjxCz0c@|Evkizx}8uG{30V<zgp0q?QZOtgYm`QRZpI#J*8)FVtj< z891>qt1;MetrH_HiMcab*@RjnbcWOPg`k|?^L0|=)VkDq4HNv?Fn$wRit~1rUvDv| zh>}_V(=dkjq40|>A^{wSA;`FHc+H3|k@ot?d3PM=G*$kxv#}}pTus6BuhFfRdDHWs zAC-Kgt&_;!N00<N&2FpQUmVY=qG`Z|p12kt?&)*=%O1FyfWMz{6thjCsg<7mmm8V( zuJcY_vhEKOR2pU~AJ%Wp*lDuIg~D}m9$i^iO#u+3y{j(`S)FKxAZQCp%_LQTLUzeW zYx^se6YuYax^cY!OeGRaf7EnLBHT(p^MTQK#K_Y3pUZ0C(q&6;ZG3wnfxSe{Ykr() zGd(JVkJ!I3+sG;K!rd-p0a0_OhV=y<w_i>jH}!zi(zfd}wMSdB2DDh*TcuFwz;AMR zG>88=;~&{^QYH5C6s7A9VTTb5pA?K;D;$6~KGa{&g%PstYYDLBcab=}9ND#;)Qq>H z)!b|9rh-h^5^2yNWWC_T4ncClq8OFFUPL_-P=j7!I0Q;fnmaCk&Frs}-{~)1aeVC- zwmZaLYx|@N!)HccqlJFxmTcE}%plcr%bT*1%9=oh{6Z^j?Mv`)8Or{L1a}ct8;8i( zLdiJL#_pra@2evNED`uH6BBoI2&Eks#DF({PoHQKr7k_9j9BOoVxtj_t#??r&R<6V zLwXuti>lxxy1*!4wjLAip`(9z{$WU2X;S}3&FF~CSjm6#4z!{E%UyN?y4nz2D(d&D z<4=!>rp~cz`d2JFuyv$6s=4~!Hq5Y5kGVYUy;t)2wAoFPqvRAWW(c|NcCB5Z5SrVr z8hH%1R?M3xj=#jR$MVc}k@2}4PYH)|31-ddqc5UOHRnz{!b3C><HD5E3TALjC0-{{ zD1GRBhg=Y@DBHC}EA%P%1(%bDWY;$LT)%1f$+-QcL15-sb-`xj^9ziPzEqm{(4?=X z8>Td-7zyKu;g@hSR@Ls!isg~jCD<z~RZ1eGG9`$s!`cT}!Mk8-rXEE2f9(m6;Zj0s z3|;JRiQ;JO1f78W91+z}nvWO83nqA_xq)oe*T6;~HOe<T+kn)s(V{yMZRsuDQSYXb z0sP~1M7||{t#H+%?L=9Ky5vRjbbr7Iu0YWag0yW&EvN_V&={03T_tQQ4ZlFCe#VRK zp%qE*CBx+t+9W3Qbg8)x&)S20OCn3)Mo8>eWu8nS^?A$+Hw}#Wi}oiKt%b(wfJ$9u zcPpmBzIsNKtJC;}sKM9p(cg9JWlakG0esz?_S-s%)o~$mPa1x;$ChO;mc$xt=z#TM zni$d6;-_!bX4wg5PjKU2L9j97WP}DiIH4T3v`?c)KKGOuB*Cse_oz7ns&hkQO4lQa z4Xng?mOqb$n5B%n6YsKd>m_3gSrOl@*Rxh7_(q>k1&mu89P4MXc4I$qbt@|>zkT)Y z&QR(+gHzA{R=9VGGRxksr)}MzCrMVcv!{a%ONeAVh)6`LmD-3{8Y0DNSF%jhzJVfB zJXxdSM)d7GpvL8`29%ELE!By2mJ#pR#(ubb*rGn`98<Pkr3NQsGuci0)~OUiSUf`) zW|BYQPy>N9)7z6zr%W(QF34#Tt~Aph!40#-aTvu0FMS#hw2(hUa-a*B{sR@W+*3<D z4+vRuS^NpiOS7O$_*=3+N)W&J0x8vt!9q`j9n<4YH=?NZ8wf{V!jXGe@(7oBc)tyc zLXDhu<8yX`5IAKuGHQ9Z=T)f`TUd-KoPy8n2%mxrC1sL^fi}%u@2ZUFAc*VfzBf_K zqymkTHd1eaKGCk9*M4C@kVt(Jyf~9~_JVJfzJhFYp|Bm|o2Zu=t}$zK)M1J*ay4(- zLG5bU&M8bIQ-1S{H-0mSgyqw(>TPxk_*LwK-s!992l5Rrn`{e9I+`#Q%T>J%W%!^A z^pd_1Rm&Co$I~=FXfpQ=sQ`vd)=V6e5|x&u%=RS$2WI_}zdReS8>WI+K&|&)pczrX z{Xo%W!8S{-IA*tHe%nLPd~`5ou3`H?N<WPc!5-JxLNcl~CWho`s`M%;d>r?(83_X? z9rU){$qq%!X3BcsWVe#_EAW?aYiKxr<F^0pN=cF?eIopyx@KYabk!dzE(AHj!!g`v zNOmPRg1=0Trz7RDND6mXVoQ?0+YH=3b)-U_;f5lHtRw%Xz?tvxe+b-Jp2_Zgk!c>f zSE-cxL9pD;=PZ_4<0u`{{oTQWwU!-Xu4}z$i0_HtP}iVa{7K}eK5^9JD$Ie~6^A`u zm?U}c^Aa21Gq7O8rE`$~9Cz>PiIoc(jv&oX_x&4n{oaU(cwM>(vZ&5z?uj_ppiF~h z$T6~8kF7}G`1Rx@$(^i6_Nr`hSV(w`g>^(wiln@PAa5m7HO{SP<3~dpr_q1i{HJKl zfmgqLMnt!Ssg^=f88+E72qLlOA+*8GDANiwo>Vr{hqOeW0EkJh==l*J3Yl4?z}wjP z>KnUDESEVg2Zxb?W|B~>cYKe1M@?YtR{nn6Z0C_0E><nq;l4b3BGO?~J1A>$>wc-a z`7kMsQ>MPcrz+p$WiUU{R@HIRqIgdB=fcC?E)Ji^NK-Q8&&sG2I%!PAcQMPT_~V;T z_PdL{l0^tlL+7^L!_k|?MaO}{-3;(dqXydFL>auH!N>A8@F9RslOoD3wsutx0K#5r zFQvq$xkxs8`h-H)=gHm~Xj1w}qlm+^0E6~@5B@`60|?Ea2Oo?_w;6*6Rhp&;F4f4> zb`&v96dH82=Cav`MrJ)2`1Uis@(Iw>;A5~D0j8Xj4t`(IG&JY$Giydz1V4m}4PTlm z(0OBq!WA8na8>1vN4iAGFw5f$oM5ASFmEsw<UNGSthEFSbz}bJqrtKIhPD9^Kvbas z7u2&u8+J$WEV_^`Y|D@F`~O}5gv`94J${qWS{cp?&l?(eI!v*&{zKbwhK<PFZ{9St zN5hFeEsYWv@2=O6;SE0wSB8&|#4iAQSOH!|%8N_-xGK_NBW^?4hRph2Z)-g2&FPQL zU6s9Mcj;iDcLNvO;ovnZD2#@h=b=Jz$Kd%b015(B{rigoxvYd3hO}N`bgg+*>QeW- z0vk`wUqrkD$=&y#?Z1z{32XiAF-F#!NS{LMTFT4;`fS0CuP<LV-O9_IJLazL!3M;( zCL}-d?k(cJQGq5SXyN^q-)+b8{v?--_E(hUoEXX;@z`z14%C{oK@Uufu_vu%Z&|gN z{^j%5c7hUv2Y{VtjPgFb7!-v_PGrj-es<E@2qk_v#}Pm6!7ygivvD6{-0Jt`P`8s- zSIaPHmqAYFn_0LVqm|vCdENOQj5N}0;DY6(NUBpo5_go-3jgujlNh9I!K6&V=(DuU zbL#Y`Mk0@0E9kI5)798U(I>Z$eV9Uou5D<psATEQcELm;ThQ^4iqWkCSL$uQw?aa3 zhf?fa&g>`E0U}Pd&w-;dwEI&T9G)|L%YSb+F&O&-E`I+HQ`Z<)S=8^VIn~LSY#S3N z+qR8~6DQlYC)-V&Y<p_5G1X*E*4@4DeeQF=o)7!1z4pQ%KaBqqR5-z5ntNNeIE(>6 zns`k4BO(w9V=K+}#M*LjpV5MV6^G_bAA%-KvWpf^k&f$!63ym*Se!QUrC|n;_OtC^ z7?uV{n>lBw*v5vP+~%dH<D|5pkJ8`#wBD6CnjDqnbr1@%;1}3O2A~NI+85gOo*db` z9nxPK%@>)>U7Nm)V@1ZreXiVTgCX5_UMTUIDMjJ`U6Xy>Ccx!;MKQViYyTHPwgnp( z+ltE;#7n&cX+}&~t5FWMLUsYSEtIE!LqjIoW9Tye5y9>jp#+cPBl{`0yHZSCD}WSG z3oF#_Ww#&e@EAIBBG?C40TAT8z3QItqNKt-k>GHv49xS$329WYC<tKMi0R&&o)U7V zq17Wp#R)vpklND8P39G&$%v7Vk_HIu=Yh{3KT)nuN9gCezqkZS;|3#;clL@5qb|7a z8F%A)_w+BCbA|QT&D7>szyxfw+^RnMJUGz-$|m2Ep`q^kystOBzOiK*d7uZUGq-dr ze}GB1_7w;!Uk*o0&KwTvN$L3>_X8HWh}XVd-o@qe%G@*d?T+g87Jpr><x}<-w&_`h zA|C*>tj0Pj+>U1BPGq-{RdRWrkF&U3>%+@Ytr_S0MVOJ3^U@jO+NZ+LkH_nkZ(XZ4 z58ZXcdms1c=6=^Y%f3{;5Xh{R7^b)FUq`BYiCdzy@1fTH4zmAkr?brc)Mj&G^6Mi8 z;*$%+_Y)@)5JIl<v@kOW6iO#0`C`7%Y3z0d`vvf8QbP@GY2i9nU?&7DNiK|U>dq%U z-p~<XPM13nMEXGsmgb{b%N*^*7Lx_j-2kWLdGpN@pV<C8OVIbZa?$LkTNf+tyg}{p zz!wb8=9S*%U)#DLPH~R72#_@mKMe+=UgO?rwmX@tlEU<f3*KgWybWSo@!mBhg`~Iw ziEy@7%Pd}ku%RCS%kaNkO*5R`&tRKR=Wehlwa{8(K?h}f6iWWSBKFOL-t-oIb8rZP z1V|-xzvlR+`ZRyHxu`$6sFu<RgGKNIQ7lC=;frNv@Q@i;B14GeT`Dz88Qa||NuVFU zpY&q~d_CcH&3Hqp2>H|_eqTV!)=UR9__S`OMmj(NpG1Ww#;Sb!QG0Da`Kbwx%(UK~ zZ`z-Zmh_((eSvOE5Y@<>aX#;=6d3a}V$?8>1Nm<%<SG<vjnAAWNYdD(GE{V+P*ZjG z=#0Vvib#&OBAl;jhGpi35@XuLP>78@`l~;SZ=M}nO!{FK<3Gvc4^?g`UixTH3CM@% z?MoL@DHJ(n{cBw9M^XL4c`fY6>~dB$F;G(P3LVUIs?(SU1HdYr!7S4*fAc0bex9yZ z)@&>G>)I)cfMe~Adk>kgbf~7Yqg225h%7_DI)uPb8J?{j;klQ2UQe=nj$ydey!B9* zFie-5WUxM>W11r&v+J+ko;d0!1zm^>d7Jj#ZHWJ#_~<e^Gf;Oy2Q(c?B_V#kRk{!U z>^64c{1YS<RxSb2YF=_J>HgW(?i}Fdklx8PqO-%^Yfa~H*sy{_>mwEws~&KGtn}*T zq6(rn@9XapyR=JX!~K#>OKOCO_T?~&oF*o>&z4&n2RW<ung?YcPNBg!zGg8f1xYV% zX9U7e%5mg=!WT1*LFABq?A2GDcSo`{FA7Y&PCLO@+8Ka4JNw&@ZawqA_`y6YSx75b z<m<iPDJ-pwB10W*_1r%rm`emMz%Fv^nXt7%JzO2CE)tbk=2bI{sIg6KstY6c!2og~ zZ;pX}noZmgq46X}|4%X7I1r+d1&L2!rO)h0nR`;1p!`_=_JC__YNJ}WIaUTSug?Dg zN3K0@`&VPU=;W=r@s8DkFP||PS02%`=tS0Oes*h4|6BpVA8pt@lDV0=Hb+y<@V)7! zDw~IurZHyN5=&GZA=ey3#)Y!ZiT!{awfj(fHSQWwa_460mi@@k+E$O6_Ni(FOLx{f zKSU5ubP}~D6W-XqeNOTu$pvm)%*|Hmhq0+%^x!^x$UiY2!5p!`#X66M`kS?KlQbij z1QiAxuC0x;MiEDgY_=KkfnXb6M4nxRgEOsd0FF8)p}o=BkAESx=WP&QxSb^}#Lcv9 z+fUo3E{cQkZ!;I~tf-~%95Z<@H&HOp_h=D};P4;!T_@b%!aeJfx=J#+3m@C6byG@U zy116C>y56D3&g+W1i@qcF@o~Y7Fq^ONAzQT?9Nb}YpLNh{@Qaqo#McKu|xf~^&!}? z2NpzQD)`BR_b+8%MjdlBN(Jc(PKapBEBC~!Y5UT@dSjKXNQXVqeL})_pfcf|b_)<w zVFUnZ)o=+ac{GL0sa$ixus~a6P(IZP*q7L~UEk-~W&<rB5p4w3#fB>EQ|McSqi7@Z zFWD+w?xFm6Lo;JjW&}cT*xKd1<{=E{dQdqQnSbZ6I`WsHSLPt@0&KYkKw_{MMX=}# z0c91~M!93=w8?trdJU`O>!vd*<gX}#uP2C-XkIzwIlWa#BG6>XllRw(DZwg>8E<!+ z|KNBp$i-Mexusgip$-$!h<JH*C&#Qi!^9~)K%XdF<QAy;mpk&*yCU;-3<bdFWdxSv zlePQw1b}2P7%PtMA3q4gtWeqeN_MTtWVoQ-C-@Pc%Juo_T7<HG6yCyvGSP8GLg0P} zDUqOR4Yg7nT@GhBnsbNS6-mR8Isl%3%A~ID=QD+{0QZn#{R_b1T>geXZcFkfBH+Wz z5Fu}JKkreGBuJXotMjc(`y*!W2_gbQ%(DGfW%EYCqTrJpcW-|(qKBIA-o?m@+aHcw z;lSU5B2@k7l-QZ|O-92WYz>Uly<4^@hG}Wszl6S?HM^$J(RnV>pU4K0Kz)L(oc3ie z-1N<?*>#x6J6NuJW3@>~?X-p1lnl1&N_sKgxCS$?N6Ie?>?uwC@V$at@9|}_um2Pk zKSA=3!}PC`XX?|}QD^@=&5-lo2ewzCJrJ*!zo)Y42pYOQt+WloKd_l`r1uu@X(cS( zNn`z5bU@C3d-x?KLAKeadQWE5?s-#Dw2id+M9qc`VkLm2(5)IxNIYxVjYZ6jk|S{k zZLLM=u@!Q&+vo6k1fJ9Mq$)bjRWA?-S9&CV@b;e+(LsV^sjJRmm-cCd{mtD-?!j#d z0U~D;A&Cyk;%dM_hyw3r=>(bzK6^F3MO*lV<H0z$7F6BLv>JcQ#$Sp%J<Z`yc|+L; z9eCR#p#Fj&{l_%107ZKLL4B(zpPYLOmMq&GgTJqPd!x#@s{cDnelq!|;C}D4onI>e z|3S{f)QS{r1_6BKRLH#X_&^~FQiz87M=Io@gaK3bi@>-v#}gxfq2VU-8s1n^!SKwa z)5%jEcDXFrM=F}hnl0ZLNSUte0VF8Y!Eov3AtK?zeqiWR`EX(GMw-DW3~3<Ek^2~o z4c6EIwC;i=sFg@zZibzvaK-4xrAT2!cD<)s$wd&)q{a>jGSQj7I0LGvDzVv(BU}Ix zVwCH|+)S9{1Wjwy^`H9kKT25@_L5o)nlapwAsy(Y;W#7bvXB$<#8eRev)@u2IXRpN zOro6cP>F^}FF6E|bgv_h895h~cfTp{FHtA}7-I^1IfoYM%4l&Rxp7u3KU{rOunj2Q zsE+SP--gfKy76R~oB%ge5DZcQIzcu)qiC(~lmOMhgL2N~)~6rgLZM;_JbX@0>}0!R z0<jVyf4{8!pi<K#g+=m?K`kR8KNRMqe~Oid{@u^cO8y`hRku9pPqZJu`<v@N2x!MD zydNnFZnFDyU@d^!aX%D|E6SZj5X;EPpP@j9b{|S^l+EoNYH;CzqiEi2Hz{Z~{K$3t z<UU9t8L2A{ka)dG9EfceV1ObSE}>Z8Fb1RkKSIgc@d?rvN}Y^i`y{z35F`sD8>BeX zyIPwcXu)ZxYc|VIBpBySE5s&7fk97bcOkP-qZqSuj5IFJxv~1`K@?+@1n8d%p>a*2 z{p{LBov+sV^_ooH6oLAY#%c7pw}_pw>Bp#KGJT+s<+})%r9-j-D4<LfBHuI0`F=%% zpj?Wk9!+Z)4h^8rIT!m}?~&x-VeC7{rrK+@-w|hiCp7t3_M8<5$_LyiAr(Pm^`pBo z3QV4{NdW?b#x;h%ryRxd7gf{;0i+inGk5jdzD3kJ0Dz}p{CDo<Dq+$CJQS92-iL!~ zoH1_dE4;*0Z8K}u7<Qp8oUr553DP9AoYqekHH!2+<F8n-C`3iq1h1O!bZiK;t(t?{ zUs_UVGW<{g|5vbg#_;OAh)DL3^aU+GIQ=lLHJSz-%nt?7*foe7$OeL6su9xz0<)aW zuq;Z?XNwVT{^(Hv7y*=)uQGH7!&83&@~fQm$MSa^lrY&Rpw(_Y1*c4|9^Tt-4VaYK z{UzZc&tU_sr)5;-H_i#fQAm4_olI8NvTVkZc6N5a-!0=U2pI@SS!lr`PmBc7Vj~1V z$sm1~OcSVe^uR(H`aV!%Zh2#Ajed+200Ba(qRq}^;{bx6`Y|0l7CHbd0x|Uhuv1uH ze#L?~z5e|berJ!-(z}P{R{_etC41$aU`P<&mp?j{`t7duX8=7(JcKLvqb2u&U_}bX z%{j%BFKO>T0MSfi*yYXccBYrIPofR$Buq*&^?D`x&h5q^uLbDn6@ZpV<f?Sgh|pH& z-9{l`heem0F|sAqf(j}?2>>$IO0%IaD}hp?g@D=dzkfg@M^%x(*gF7K=M^L=>-}#= zlF@v~GvKA`tn<!*azM!D-|>&`NYvefVVV+iUx-8#08Qs0X#{%^)f0tMA|Wyp1u-%H z&KgIx&dUeV15m&P06}BP^}dpJm**K-)~M8wfB1H^UkWsY07Kk&`F<;Y*w_obbY9PY zfbC(E%>3n-K$r%Aj-mOHon8u2AE48(cODY0WPE@{E_+7^Q&tu_KU8VeiHLsywpfX* ztZ5aZGj~TAAZ<`sAk$|tA@vs)QpYGqqt(eg1w}3mn4k=R<yb6tgt})BVETezKlwFq zKj0ylE0n5ELPlk_I_;5dWobYfv!mpA&5XE{ksZ7Pjy?tyXYtkgzB~#XOBsJ7x!4Lq zq5Ig-IKlRjPzpXRQ0S2iViVE3qpyPw&+6`MeasKK)wx2gd?#?rF!npRVL8*TErQ5> z3$g;>81}*K8%qRA<JH>y%7bjO`W>2gc{pE*Ev)p3+)@XCHcN0=&new}s!1R~4udaY zwcp?iM41MFjwzebbyGlHrF5coJNN#VC`x(PJEI*vR>=*p?H$X7Ik%kX21J41?9cgn zifDdv>cLPPAfi*!L7`(PNe9wI(nK0yHiY9akp-e*U^bW_<FFVi5BQV82noR@IwuoQ z!bO&0zn!19cfNg^U}`w|!+qf4Y2jK@+WuHsb96l~T?6=8XPgE?%)oT@pVk1=xWTf_ z96uN8%rrtgd{jO6P@3fwc_0wRt5-D`E?0%te2ohw9Sb2w<$gmq$~^i#7Xjq{chjLC z$w1wKXn`u!pEUj+9Ph-P9cc(K5f%U;jEPmHF6DpH4ni}A3~=BD+?`h{#hepu>ayIB z5Xjk}_B4C>-o(Ky$uVm%q*#C{Kj5!?R?~jA^5eR0ZL2TzCX_23Py;wm)7y5P|Bc6K zn|`OuAdXsQ)O!_EiHlWWk>fEGb(+C$l_)8-v?w^;7lP1vY(#wf&JMq;lc+lfR^eJW z8hOX9$iVH{`myDQ#aM#BdK(J3eq67BUGs5@Nw{PI7Q!vc36hsR7zb*Z%pdo0Mw0l3 z>crNqk@+_=<Qmn8<t^}SJVI1{BSW+siIPa8_Ox99k`<*PvX-vBuUg_t8&5aK2Y}Yb zF2EYu)7EtFR%ohvULs}CjQ_A^Ri`5qSfY@9^llM5-IQt<7Oa%G1w8P^S2n(j_(U;J z@XQ1w9_9&fZ}xPPF{%|LerHf^Q^DjH$OETgcQdiyeYB|v7jOyfHU)%`Z}KC{B2HTz zwJ8^UzZI<k*vJP!x#s+I_|AEMV)UD4A(&1K(O4KBy`Nx;gXphbOEGX9PS`9~SL}WR zf<TYqh+*P}k%12`yrrb^Mc{N@?L=2S#P9M(bSTd77EZb8I8dQNCS?8``+*2dQWHbs z<?32GN@oBw`*<+iC`L#d(Cp5$Sl?>g9ge%3+5qUVa_IfA**q>+0@SIYj~ZGUP=W_O zVaSAhte}~6UbmXN&No=}p!k^R^h717@<1&kXpwgw_-A0a6Z#Q0+e=^&JZnhhcj7qU z0Dy0Hzce}bnhL1IeiC9ut?>f(^oG~#1@G<UpkLe#ZI<&x_b%pLn|mbm)9^<h-e%Hx zGcf9}VVQ&o(s)d$7Pi_%sl4*;m9+V;+g4H|@BkJh1KiwOA5XkOVABVL2xibvHo<LV z38k0G&zo7=04r)L`_*pa7+9|zivWws{gQqF0O~{C^_v9RVE6e3h?r#YxE#4JB($G* ztpYuG=Kw>mbE{!oi>z&c6m`(rE=)60E1|RBkFu$$9+r2{!WqvL>`u5gh|0hUkXse$ zEGXhfgj}U8w{)|gZQyFCljmDIK74sLMx)O$!OCzius`4gC4wO<s#SZU8dIo3g<Qc~ z)Um6eqM-rQPat$yNR5}@w47A*f0TD3GKHTJvj%>VXeogsC6z8BWQD(Tem!dSy^QoP z1~U4wK`$vWhb(Xeq=EZ3bewm`?TSr~SDKr@zu-VU2`bZCGa73g|Lv0L#uFe4WHB_~ zXt&P8rb|&{nH7il<<%p@>j~h~or)TYR<-;z>f1I<JFgjvAko{-8n;m!BF<HV_Qe{% zlQ1)h_?bP9DrEwy9jfIqH30_)^~`G;p7Nst#3<sav!8Lm#EaoV>J@6O-(hah<TB+W zD?ZBt7j8#P3wpB4EUYM~)|+IY9#s*F^pefXxV4WlYBMnfF@VCT)&1$Txik8U8S?RJ z^N%~ig}+8oh)lMqF#sc!r#{K}WmLhzy#FgAF1uwscCKozQSXE+Yx9mlsQ4jZSnUch z#cOuGR)n3Pun6i#KHZ6!y<ct2nc5fJj;7?lav3aJ`h^QDO}P%>`BKj}u<o@PHt`c8 zC|6vWL`fphYVXr2&N{d_2sI-brUiJfAqh&qELkBziVA__zeb&CAJP<w8+7}29w>+- z;jZ=9+8U`}?uynj08k3R80TLVHXwe{YLRfbR9A>?6xReDV1{b?VLAWPr)dej_E1FJ z9pG$hW7wwrh*J~_;XRd<Z4^Oz2ytd)Cf;v-1@M^NDI$4T-Yt0n+vo3+VzpO7R7)hb z6YZc|Q6~qFLV4e+Xg>uoBa>ZYh!~7kaT&tCmmYAQy*i?A&)26?iZE;8BL;vZ{oP@y z`aUOD%mOTCTBsJv`q|+RpY@M5{$Q}JZlZ=$Kn*B%-0wu$U2OFx@Y%``M?pLSLJ{U| z-$L57^rpk}xq-2W`Vgi}*Wy&`wa$)<!7Rg3#uJ9)rxpoaHlzxhk#L_#qJwS`%(dYT z^a6(ChQ0ppL@@(zX?$riO|1HjUe?O@N4s}`G6=Rj;=fz5iUkWliEW}z0R-C(OB{g5 zUPr-_McgO6T2IDu668XgK5*etTT46)FK)8*Im;QEm~G(=_&gjtt^%SYaA-Z>h|V#h zq`{dS+z^o7Gl0a0ynUf((|;Zg1IcRib)$pZEMah|v1iHzEQSI}&N|b-5NiuA0zut> zG%ZjVhL}nI+XHF`vtFTU!I!hJfDi-zlE=rom=?rYJLLpanW5?m((5`I40cs>k~J_B z-R1)029f%646Tg?3f1{JhX?@}dH!aw;G|^@@)to)-~kM1B(DKbVL*;H;Y9pI=7ZWp zh@{EI+FTrolOUav(|G&|R1oZMP9KF{YvsFyz})eR!)kUQDpT5-=Hv<~pp4XLA-zo4 z237$T6g@&AC8@=R7$7F%?{7P+uF{!4&v!MEi361&;ok<Usqd1~4t%$}avs9N>!nM? zfNx?(08i5n2tf_LCV%VjZQuU~_y+!Kk!zO`?Lh@&2LKJ7R>%)9?pC<c{MvxW*c}ky zooQlnt?e%RzMnLxUN->Cw+(@*rHB7!PSx0aW%SJbHXC^BXB0?yN}Uqcc6=2=$cB+3 zPn6XgTqtfa|K}?(W_Y=!+)IGndYgv~RWNW)g|kBevEJP*n;?MFw*#EFGr~%>)`9*& z)<bva`z&9e=5>wUU+#A9gm1s}-LirbKLQV>u;|vVe4m2&&-NQWu`yA&t^1$9u9#_L z_{2wS3St-)3R*8z;dV=iZBJmFv>)VnbsVi?m^}S0jN`i-n#$&F7e#4uX{Hgf3zH)S zflq!Y<p;WnGeDt+eK7$S@j%_0N!<|rJZ1wqg+qwJ*vRE2(<t{L;jz<Pkb4nJf~2y! zzy1Bi^^?T2G<0Mli~D!GfJVw_{q>(DMTzrZS2Ix(Xw-Ya>PjEz`{)BfAiwMcpa!+h zeUOy|rTHbz2y7PTkCt(AS!a(F&?pK))-O7P4UdbA%R2D3XaKwxS*>wD%xBXf#5>?2 zUI!0=?!r27qn`sJRXFq4E4@eoW)fR~wZr8QB1<gLJ8jWS$Z-UYQ()v0W-6Uwr;msM z1Y|w%Tz34n(C`=z1B_cbz>5t)L0V7TQw#~d+Yxo*FuwVDaR5{7gSOP>QhC=)8b+{V zo>dgj5?o@~?fZ0Q`GyNc8wv~*<5;PkK$HX>Y!njkYwNobAdV$PFdK^|3<Adr0+#~0 z`@jJA+`8N5DPTjkweCle<F8bh#i-ZvYZbk;cmZ9M#1Ykh<delyU%K(L9^ldQDtT)S zE~0=68?k*;lrhNAPX^Wv0Xq~B3ph1d&tGZ$Lw+Q=6%{4H2HS3St-l9lc7VKqK@6@n zUIP4cahk!{pz>*HsE{?;1pssNw3>J+!U2@G?o7skP&@HEwymZ)*o?$Td;CP>)B78u zk_hYrqBAal8o`l3?y6v@5Ca;GMpgRpN>gR?WH#?&ei>lCJSq!o(C&Jw#W^@-4yNFP z(lV><BN^Zp0PgM#+zIrq%LSqLA*BO0=3C2(NeGY{?xTNLcT02E0O2ORx&rZafO3*_ zOjgz8D86M$`zy2f(_ERsI-%E^qd^P|JkO90r~Sr>MvdMf5Uryht4f}DgQ!(K)=*UT zQy9$K)haE2jQrz_S>*S+ZQ+N%eFL`Knjhe-ov3u8F3~Bfyp$Y^i#^i+@4I3oz8wdR zgLo8%W7wgOam+aozP1BlU>m?6Yy*6^x$Dp0+kPgC@nf)$fZj}p7Uk>@ua~^8JmT95 z;w7c;9)nO@H$jhqT0D;sHbPzA4}8x)2*PxxnAJAqUKFN*0vAsKf3Ur$_IKl+C=XK) zCNg$!diCAb_Y!a(1GpqPlm3YNcUkxbSFu_Yj2q>b)U4*?1>q-yLs;5$pHde1nnmD< zTRy+eXp{r?3o=%%D5E%~DRF$WUl;1q*>DRbHLCvJ{xvYEVNffJ|E!2LLon7cI&=`h zBw>c$8CK<kCcIU{d*;9<ZZIa(dbrlsM4!KSrDAzPijU_0d!J;DIa8@eeb_vZ)g~rW zU_Iqm(5^3~Sh)Gbm?GIOxkR)Buvh#<Ocguv!PpteKMx`09M$U+2}w6!vOQ7LwgPC! zW%pRkvvN9W;xT@RsI<W!LVAG)#3EUqc8*buFF9!^P&2xf_y6JB|9%oI+VUupJa?0U zdy_}G$sY5*jl0!n+-Vj^dHbpYk8o@}*#$IOowZyekBtH8JM%CEWQ{yEY%o0T*1`|N zb6OFbg(^qg;AvOrF#Cjp;2pWmIe{aZyC{M2QeUCNa)u6Hv``}yM_O`~7G<AE^q5&S z8Asj9Z*q@YC}=q{6&9@esf|!a<02}dByT60Dn9yvfL@&1EUfK-cX4K3`r}Ja-vHLj zTa>lv9<~7jHuLSHahyYyVQH*xDNB-}Pfz+*YZD=~#x5l5=>2kqBc~?qvfm=VUjddD zXXWOr5t>jpbJ=C1L^m@7tD(k{C?`Lq`Z>K!1mjqP(ijsy#T<n%_}+OB2>RBoCo}Np z?I+TrDZz+7F=r}bFKX!?D!MF_78`oyUO`?IxsPd2eZ-|5cf)@TV$@%DU5_`6qFFUA zdzMNrUo+7wJ{gL*Ygqbj<*d-tXtmJ)U>WOlPF|ZF|C`d9$f%qMftwJq$!}v33kel< zBXsXJopXj!t#{XUzk`!CqmW<+_P*YYvkR}7c2m+*p2mS*prT9Yg+{DrHT)r)%H;BJ zqqHaNRWxFNL@SS)R<Ghu|M$&(%##)xzwv8|NQ+!$3M%17;k~k1;~C4aI|X&>A9g%< zpkC1Te*ZGg@U_UCFGg6_ChBVZi8-Y~gHO&&REb@Rlv=*Qpg<dwV!Uz>R`~-*X%+Pe z#$w|K<|}?NMV)$b!M|pVDj<_-TX8w|96(QmGnR2R=E!D&%rLA9#_ZVt$YMxuUPtgk zg03mCgpIvy>YY6M6o~5iwIh_?-up6yw^N(V-W&RDsNn>Bt*@fdT=}LQ9x(S-{^V1R zVGch0aa7npS2LDE*IGfA4O9ahBqJs?X5Xk1Bq1utygh;^viVmT0V`qZ0o6Mh^O(^w zvdTTSA(}hMIxZb?lHm<5Ozw@6x^}Zuoo1O}{9~j$^xc12lorMt(TklXVpLamrGzZP zs7BDcMf;QM(I^7pOQPJuOq1Ue{K0F0J`Ml@h^;vm#}@B&i4ZZ<Ql_#q&%oF_ABu5y zDGDd5^nb3<S+AzW4pou<vMtoBONKD$#9wtVn#k_=;I#PDTvk(`K7l&RL8Nb(0ML@0 z26&g_)k*by!l2CgCkbmWX<9-pHKoKI;8i(3)e=?Bb~dWW+_=)t{WjOvVt7_W2UvZc z$DeKg#wHb$;xfRXio>@}X$abW6n>RHx3Riu%O8O>D~=q3!5Zm7G}YctOq%xj+>rH5 z<!;DeS6N(C6S(&Gg&|?<l#EXv|Fi47nC*y>)umN$S;(7Ujh`yDqAMUG=~6T-APvgP z+8+VD?3>InW9N2+?-|F|bLg8s)oU#jPPR!>9r+HA{ccmI*VH{*m@M`hv?lEfmpgl$ z@y*TyKLb`#Xa8<b*KN)DSXn?OAw4lpES1G!zPY+~5l6Jz^Mj8w$^*0uA~9+9-7gFB zzdv|{Bv0-(1x{%wf~b~>24MS;_Z+(R6GsuSTFQ0ghqDXvtJ+UFnki(~dQk+Qx2Vc4 zNI@kX$*-Q)rarXUsM31ZiQ2xYO}4Sghurm4FHE1)$^XbW?~GEnCn@lZZV%_u1ZFM; z!+l&6OL!nR6CY!y*W^!l^dw`B2LwqOtYwR;rV>M35~N=7(5AlBKmS6J&*YTfmyMEM ztqb>M^K8^D7f<L8HfNhmL1Gus){^2dCbbM0Iuc_n`{Zh4iB_DQ2t(-ie5u^2M$W1n zvWQH?44RtT@^Ib~(v+$}2NF>Rv=(jgzf<iy@QXTiTICe?mwtKvJ56NFn2nK3Kb6Tf zMaP_YMxFSxy49=Jd8$iNf74knA~0Q;@R`IwcKs`qjB(1P_ZYQe)M>%f*|Vo|YcZo{ zg6(rmW#C{dR`M&vscTaK^SYd!;Mr`G7`*G;eO6ii7j9)qTdDl9b(w%%BzG(jVZ{tX zUE8ipGup`F26BwfQIYB`t|zWO9GRGNf-oN^Wr&Qa+FY5UgfiPKaBY8xCiPq8Sq8Gs zH{Ro_h61Mt39|yIN<#J=FyGZhzg?+D)zjTywCOKPt-mn>xSbl>4DPllT2YE3J8sA5 z?TV8}b{qTFl$kzIP5Dqp&3&t`*GS&+yx~|J0HK>0gm&KL=+>xjC>(`psKj2;)Q}RI zNA4hWrKzL4HOe+QIP64F4<W)F;dgR;E3>H(dfs;`?1f5=6{-3yMqAlG9OpJXj&$+t z_cA{2nG&mYzZl*03{ZOXp$C69rW`NL&??xz98aBLrVVb_<0&qZ4XCfTOL|w~C5$5b zJTcLtSu!>UG3WTMjDJf`#x{yk@C4a;?7%Gwb0|%5?M9dBjq40L#E%pJ>p+>i%TEa^ z#r|x5yov8tYf-$8jOvN}B0?{+D@k|J^vHRZPf~Wn=jrQby9pe(6b(*%y4r<5eqdDH z>pF}>(Z&^@Bt4!{2W&>tm(!#;{{1o9{j!E|)kF4)pQ)HXQODEFvUy5nu^0C`uIpDA ziv{15u>h5}=h<vp&5AZBM)qZF$SRL>8|2kW3aq!Yb+1ik=gZCf*8A$0IaDbwki<B4 zQ~0c=#I<p%1NK)W87>gs=-{7^=B!bv@jPi<DVFe7p67p87jMoNCZ*&!h1?eUMtfwL zT|U^gPs&<kP1eA^%m>|V97obh{J~9-A!m8&lR}hXrkyGs;hEL0p`X)^2#rsDgfZ{w zkg6lq%=IWqNGFMpOEW)Gj|t%ao@wB*9g=6S`nDNC_IuOkwe{HR^+Cd<{;yp<nL%Nl zsub<jru)r2!|m9^u!NaYY{m;&02)c=M;8-0!~l0PhYAgw#Zh~K40EHmSGz6qG!H9} zqz4|88CTYTb^5!KCm6&FYfiDNtWLX@-X;w`sdfq`Up#5PdY%OqC$H!G{IE3_r)%;Y z`b3e(KakBEM;k5I3Yz*>1M>9}TYBcz6?tZ`AJmfB^n-)(*~~Ley|zORokm;RrtBii z>6ph4e=Qn}&pSR7<0o7u)Gj6q83lNk&_cXt1|K13k>h59VnobFX>xsTGi)dZKg|aX zlP`ZBs37>kz;A?2>v6M}rjL-2K2(2^5gaB&TphmYb+^8KO0aKm7oNAlG+rd_4sJ}c zQHO5bM0&gEzwnL!;}w%YJ%Xz|HIecM^>p!>GxA)Q>F%$*3|@jR3t5sb==2UQ1VTN5 zzRWc+f)v#r#52Mgj-bUJ9JvOlScVVfcbY0_eyQnK-(IG$0Q)aDf~h=kO22jSUC8%I zZqcwFj=F?kJSU)9`_5^|*ub}c4~`SkU2onUibZ;Zm2RVU3?BdXBV!}s8u?lFrtk|* zrMBH*j#{huFW+1xFoQZ)ey4OBe2$Wkj;T{jkL^5Bet|1{QOE3Q4uj!~<tUYTV1a9} z^c1QV-2^AiJs)~tV57KI4uZAg5HOk(C1|He6$ud%R(y5p_aBU64Dt~T%9-Y-bbhdO zOPkIEzcKvwmaeW@i$u#<BeU%O3_^qyB^g95)5;Ij1hWn@(I2I;X(G^KlBxYghK>k( z?f`3J75HAlIR$pVzOE_{)aeC3<~K4tt2KWf#f3spKc>;vw{dnYo+7LuD>E#!i<Z6z z%c<Z%)Qk_3$3IMm%iXuE<^aWXBQOOBYVP9$kdGEDq>2`axv6m0tfu(T=qD$v1$qoe zF}JZL|AB-;r{&gzpQWzoHbNJGI`_Uwq1UW0;uQ-4lam(BG0~eJ`^?9(z!5{j5AVx< zwAnGnyB^&0ynmJ?4na}EJAb%IrnRezEZ2xp+KC>SY_VFSbv^vgzYm}X82&j59G7J; z4rX;+uYfohKWm?kiSj3QW?#jdHdRzY%8*SVYK>#XO3_FRlY$>}x5vco*vqh3lbHX+ zGD;M{D^8`THd&a6rnQH}r>frSw4ih{DQe$^dtcXy641js-*R+!(s0H(NtlqINUD(N zPwb+RU5%GQ=VA7xvz6@2vgq5xKK@-{6-zU%xla*=XG$U2ya;bBFg5R0YCTkv>I<b} zW1{ERC%aJA3HBeF)7cIaERI&P1=;^fLlD+cocCJ<-yX;NX;iTMw~mW$Mc(Lte4D2I zpcbN1ytb!ISC8ktVfOf@Tf9!p?@`aITkdD0QA3bY@>%(|rP_(*n4a>553``jv6gwl zORf7r-TY*=dQo`M;MiA_#I0?#d`wS6A`16IibhMu{^;-T9M@GrhoPbuHzxCMQW3-g z8dr&SGAe9B*V+_72Zds$(G(Zzy-i{Q&db|mu<18}KJzWBm@qZZE-#-yR8?CnvixrD zRKUt1b6wA2vqr44)YfQs8e5~mc*Jj@X10$!{|}e+bHlN^cFs;v_Mah*0?V1QTwY~G ze<zR!F4JB(S^kH1H@AmB&cyP<%9NP5UQTRYlC*Bq_Nxoob9ut-iWwKf2HoaSZiY2# zp+lT8`0dElSr^@$rKAblefV_HPf;(Atus!!ZBHqh9no&_-?8d3N-|NYIV6}uRa5Ru z<O?Ov<QXFgsoa*OqR=v=$gPLjK=r~#&4EhP(>bSDG&C&*bSlmBMTRtCF801-;6z@H zk6$!cJ$J-36;bZs&Bx4|qV66#ie=YrF}<GjPfgw)d8lZs<k9g1-UD#wY5vb8F;&GW zcFBhvV^43{L<%S)llw{u1R5oAi`Fg`CB*urJCS_h)YG}=ugVwv<?hUun;YspJxh@i z2U(1FDN|z_!(&wU*b}*%`Fe#T*8+rQB@98k$q9(ihYZG!KfrR!=#(a|hm_((_FU(D z%`zD;UFZp(o=cw;rGtJEc+DxZ*humzid_Y1d%Aj;L)<}m8nDmbpL<MkU)rklWoJ5$ z4z2a>z-nQpg=9XU*C$mhgAZhzAB1_vOw1|zn`*1EIy_wu(dS=T$UdsOp8}%bZ>S_s zzhZ)5TJO5<`^Lr83^E)vj)g}=FJoYd9KJrQSABwLe`-C@<8Dpws8|ZADw6$Sa3+o! za=cKkI->8p+PQj@Z20C-&5+0qqO=5h6*X-dE1Qdv=QAAE?%k{9n16MNeCF0E{1)2i z;w=+#z{x6a4TGYw$xwcWhoxSn26;svk`5-Ff5Z_SrjC8?$~j9gHOuFc5zgY+W6p`L zS6fKkYr|rVpVa<2(T*jxB3>aec{8Y68*dC}lG$VyAzJ6BAUxzER%=O&U&N+vRvk{^ zQ>Q~~M~bXfSR46IWqaIR#@t-|c%!wUy~<SbvqZqS7CkXW>i}6VPQYApghQ1A-;cBt zkL#uvib3z7ohGH(*4s86HI`94NGYOKJMFR(OCaPHRq?Nvk?Ms~5?`m+YLQk-uni3P z{@|2t66lT$WSx2y^v9H=CMHciMq*UUX9PC(v!wnA14wt9=b`i?r<e(0u9KPVWKN^n zWDPdCl0=S+M-8-Z4~lZ@--!HPT<$W6I*9pil5J?+AVwzS9-a-Uo#qZ7kJl5KyZ_ZQ z&kVT<ysfQ&8>N&EYv<s41m_LhM^13gpKS^M6hrxGpB(NFOHfqOn9N;;t$yP`&$u)7 z;VGQYqhPbec1kGHfT%0}E_V;TK}P0k|8#Yok9Gp&A_|YUSiU=0m`tNWkx4#yR|!); zP^(|klF{1d+a@(R4#?EStn)c-`^zmT6RTq!inl6!TP}F@Z(&HpC%>EaYk~+CM^tgh z8bOG3&guUs`*qtOw4=68h+XxMQ(M&<h(px%o=7~Ui%TUqy`@16yGp*&++&a<+_wix zw7qq)W^PC86oYK6?kMWn<GM|Q4t*?iD<^V3kbS^D4sVaq+GKk_r<}|l2}_X#DncmK zExgt?9$}mLJGu0JPXW|2?tu+ylvjHImO%U1B>J&sqX~4wap}N!w0fIz9}6%AEtD4j zMzRx&*_P0vI1_ExTyp)t4sVG}OLNDIU-@T5Xkfzy>w<K~;-XsbUHre0Q#P(wVH#e( z@$KTc^4`N{TB`M{epMLqV0GAs<)>LG`s`*KvLgW_f44Io75`M50&$vungo>W{qVJ> zi%g$Y>19Fgvt9L&3y^Db8kULsca4Wq@)KH~s9m}vM-@cLRbtyysfr)H^Bg9KZ+J(D zh7IcMvgs?rEX)0)l``4OrxNRX?Bjk{Xeg{-LrteepQnvgo6Z(V*JQGOpp3*w>)TX8 zvEnp*4kMD8QLC1?+S&~WSAos&r>M=hBxZuf95+I1)xlEgCL(o`pZE|-BE#J^U&L*` zOZwOx8KdHWVV-t3P*wX<9wW4y-TijvDqZ)7H+i?3OeemYk7t97))JLYN{ci*3Dyzw z>^Hdh!_5!D)E_rRCk>?0q{$SK->2rdaOq?}X-`+sK4Fb1b93tZd2A~Tdg>XNY=4)? zeR>gs6xO)4THJi}{=PCAC_^6Ou`;KDzz{D}wJ=I%<ZklvxFM1G`55oTw+Ay8gvMZ- zz~%px-gUv5;1ez)9TQ?XfC}0jZt-WYFF?+>kh^+=pB#_JNj~HuqfaSP`nU?|7-6P! zxpiB#`zCp~@r74^+BmpN3+~nP_Q$WCF?VxOGuwEhf^sZ%*Tdyeu00}BNGapVq>ihs ziOSZby4~rtqOW%cGH29$BI(qf^4FwrGOxde_<GZW%RU*_vtYeEB(^4TXYHmediVPD z_;^-Gd6}={ueC>z00qN*>?$PM7>F@%4q7SfkP*({#BM1W38P>C^c4k9b36_R&yA-B z`pj-RM#Lo_hQku$Fur+KOQFlWFr98n<GLVIxHPJ~UN&!fUh+`??LC&YsII6bWfN1Y z#UcGXR`9Lqx$j(s6K`tqxEGdRmHTz3C3l?)>f7qPrq|OH8YHi{wzMbR*As*>*FpFr zr<c%O;Y3SQTK*Qz$UL<fx|dR?6kfW1q(jkp{~o%5BE&<hjoN(O@`!&j4@<Wi-Y`&y z?V308JrkKl#W+TXJmR<}v*JH(%x&|N3&gPXCYA!B%~R()4U71MncU0lW<RV^6IaPy zW#9G8_jATKUVPg38le8X_7qPjg@WP*x-!&?q(spTM`zNc8MG}<l;XZ+q6+iF5!7OV zGUX)kp^>%`J5q{Qs?~Nhq&bH5@fSMkiQz6S&<K80>Gr3s9Hk(*mmR){ppf}Z<a16d z{YXF-$r|be$~Ut!+FMYKFXF;Z*$HVv-idSxKUa>GO6_TF%k}x`_DW!2Epi^qk%;89 ziy*U$x4rWbtnr5w&6cKm_())H<1R)0OrJGiPhHR<*IAK`sO63n+Z%bKt6t{q7Yb3i z=G$}OM|%}NQ3D)ansUjqA6o7`S0h>`(Oa=noSah8(kEzn-9f_))kbTq2F;eytP0JC zM9VcHVrRrdSi4CA<}68<;q!5IrytXFJb#*O|BFD<ehGJ;DBZkVwf2N@?!Ldn=Sady zidNlMQ5hRHNg)3mj-N1!<aw(rN`eR~ITBWvL@BMJh#%-+R=F;Et|C5PUDPju<^8xw zs3>q)17uluNr5u-(EMgtRw0>M*ixe1l{Htl-4EMg4Ls`m2X422w@a)-2>Kv1EXn;2 zGypT%TSgtTHUXo<d#xLK9Xf&i@iYS`mtvf4YZxmYMOqBlu|PGcr1L1JDEz#a@cc_@ zRO;^qbB(Gugh`B*l8x79)%7AgN}f{gzjg|u*63yEpN!5oHr`HtciL#W4Rb8I2Zz}_ zXp0S{*<*ZKHazeIJZSAmq_><O2;EvT`or22AvN&dKHq+K2j8crSN&F}Fvv{m6sPH; zVYJN$4PwjE?_v(Z<oYHX=B8gfR(4%uQUs<vY^cV6v&FDolhLXZ7dDwFq`4j&;v2SU zLkO-d+aEKncc=32Y!<1}e1uxlj=6gZfF^pqt4`2jCgtfWh;&&QCKibT^f){sD>j~9 zlFnY@85XBo1}7QRlgh;kh@(fupu~AYN#l{AS|UR5X#~~gR~-9~9DAWR(4^u0RRrZy zBkwSjkhI~Fe^AL?Xega`o$=U*#D*<x>Ds#(#u5{^toojhrgQGceC9hIqaoZ)I`49} zkH*=qP84u?Iz8({T_8sm;Z<@tLr0@4oL4Jq4o6PK21}D998xw@kD&C8@ql1tQ7K@y zc^-TV*YM&K#s-2jGHw3wEY<dTS7;F#8a!@&(3BlhMrqxB=xXt)Y3bt?#44#Y%^DrI z*JL-?OxUyfp`pn<YV1ABL|fRr>6+;!al-_p_6}%0eTmHd^R#Q5OTIvDwyzgRs&SyU zNr8ZaFUyu$bSn>y1Q{eN!4(*?x8~1<=PSw}kCYLVHpWN5;Ebs?=2o?GAK0$Lg(9NI z%fx-mVBgPZ+{VBj6pHA$o^pyfK2#2Q$@W@jMZEX{?Fw4N@jGj>5hGmk5$>6Z_Ygd3 zx0h_4h#+nIs0pD(Mh)yv-wo+N2fK2wxCWw+RaHxN&k~Nq8cC8Tzc~VYYO%2i!E?LV z^P9^&u@+TU#otF!X~yF74iOMjGAj91lpfL)-?iLbD>}zJqUSxE#DOZ01p0t#r`qi` z!NL&O)ui<lsA31Y<WTr-32Q|cKECpfG@yAjQFI1xMLp?Sci+G8sE8PPC>JS{GU4)v zr5NKcUbp{7pr5vFkRotu7{0g^L72|KiJLjicJ+9&w^TK@K0C#=2+h3~|LYvs*ZWl( zhy5&r&vpAoMtpznC(@2Qvybq)7LyY8hDAuPHAXG0QL@Du_DGDLq{zg3V_v<2W)173 z5$vz{b$K+bEjN0?EG!;~H2yRG_!IDRcWC^Hrpf7N5IP3rTJ?qC`Q5<BrzTW-m>N}( z0a;U}h6_dmMMDg;!I-Ev^T-w^1Cn^625moFD!O5p<|X5Ec>{WVNP_>C`QRD|LAc24 z4^OD9Zr;hHCCaI8etJsJbse#|fN0aS;81g$0)tGESJ&MTe&+zRmO*X7$vih&eaWOc zCzIes`ZN|z@yQoxo<|kk8mk73KN}($b!DCX3sM7Lis*9A6BRw`qZCc1zRuGvra{5| z8<nd;`2q{{Fd>1qJOJuJeQ$mnm1-raAV4y(#@!tsU=C0q{L&snRMqMFJ5bzFo}OMY zcJ#;M`G{P$KO@rZ@a31N0h-J#28v1^|0T`W;zro<jB4F>NkE6Ep_b3#w|oxAL4KMc z^2XQPw>tQB_sc<p{U3~D1THmDchbDFO8m>S;C+70!MFIpNL67#0<B7~90%i;6x`j- zn(j!bNVD)I2Mj)#mT{o(hdw<f_Pt<DcHH0<w<fU~SR>IaWBsRU-?_(g)Z&AIFY1c* zih^PCML?5bvKz4X`C=@NdVC6h8zzMmA3~c!i4pN2ZuhMrIh0LHBNAc?1NW4mZ}3c$ zGW;UKf+U0GbAu7aso#Gl0x`f$vL{@w{w}XckQP-)dU)lsd}Mr(jHAUb-^`>Mcdjx` z65JBMFh+D+Umv(G;9l#cuHE9$+y`w_G>asP7Z`G%fYQV)&XGy~0jEauGwpqj4#YAX z<jp{<kjTirI1B;n8he6E5Cw!UkJ;6lXdVe+Qjt<Oby_3Z7@kDrA6FijMgNpPosz^Z z)Ew5yd$i@W&Xu>+<g1<2g6GAc)AC@LW}2y>7ThTm<wuPo#KbvWd@I>=wSu5bu_N!# zO}py9w-{eGHBa-DsOvsckMUertcVJ{n58ID>{Hx4eN*@MRUDKJg6;?pfx98RR5*y- zdNeT`lI$}8Fh%bxK?Ml_sgy4iy7I$C3AxQ18<Y2Bb788`iV_8eL>t@Mh)K|h;AZs` zs(hp|d&Mh7)NC=rB$+Sy*|YOK;0dOyTdVGqZA-rK11XZT!@O<P=Q)pjjp$7c6E!M& zSzHaw<b9(+=*tz#yPrn+Fi2Lpjv>4%6l01?Tg@rceww!6Gm9=%=5LeyckV?)Zzm_J z3XZnNn7@!8xRvUZ-rOP{@gfI)X-FM)lKCZ|lCvcU&BPl2?`E_FYemZy+=Mq#@1glK z#9K{J!)WV&S{@FRmeY8;;tnRGetH@%AZ(;gAGqJVhS%<SM58g|M4(~`*bMWG^TG`t zrDh3un$RGfEUBOMu8h1Hu_xnj8~ch_k3O~{N}yPgm@cG8b|Vl!A$FARO7P#+0+iiy z-uw1Be^*<92_*R#5KEnV!la4zOI8h*`bT}^SIsI(WlB(wh^oeXwW@r5{eajt&xg_I z!3nXCR=hufGq8vbiek67Gfgvb7esX+J#VA%=a`i&X{6KiD;sDQ44Hglao#r0Uv3gR z_nnrsnFl~xgfOeZMY_l15O-T9;=<>~X0r|orrbr|AXVZmU)8co$T}?Q{kxtv`0m8I z6@s3qqLc^F90tA?SHz#9xXlQWP9Y-f&p3%A{j7mGOIy9$l+QUS{w|m={6UjlY;A>D z<$0tkJZrW}Tk6=>ZZ)2!JD1Be^ofxqTfiCjtXr#r7MXzYLH+P7!piI<z>%hx!!5RI zYXFN=)lgj2q0raRZlr~z@C3H&>J_6EtT|dne!Fau%bl2h|8&{eEm*Wo*SefKruEBH zCQ5g8Z{;Kj&d<=G_nvf6nJhlKi_X4>WqDCB=S`=}+ruq)hdzO_V$`+Sx(u3#{=$t> zr_rJLa5>n!a%a3Tpf&kAf#kqwVCb+n@8_F+^vFqEgwOl^=qTpjl4Z%Swd<~(QMw<{ zrfcV1T)Z|c>kfgv@Zd(rrnl`6Rn+=*`X?c;9ya9<FeTr`w3^Q0A}rF?5mm?v_2*ol z688OO+DmaSS^{Fhmv1AY{7+}?22K$MX?i=9%iva3c<=iizt>$#1UMJkxk686czki& zLell*i9ecya|!H4<<_du<NIG%S<e$<Dl}a_oz?aZ%gw#nt1*@is!WN()y4EDClad= zT0MA7;-Q!x#k~}Kd(AhC#v-Vh@qsveGmzFQAVq~73I1^*TIGKJmP_5EkDJ(cH}UcH z?zO7}3!KiZINK?!|Fh@(A=}UUO^53>M&|DP{D64%?Je9NXWp7Db()rly%y=!x8tgn z`E;ep*%mCVIi<=u9fXEo3LCWY+RZ}NAV_ulj#k5gQWo`x-Er}35vauMR92x<P%UZG z8V^x1+DKK&HLUEPNp${~$4py^SDvT+LRdYwnj1rSOr(m%i7q%G8k=who{=`Ukt3<0 z89i+zixU|=y64gssnyf3NZH#fl=K1|r(aAjiT1t=2q7m0rX$blVa1tR&s_{b)%h7# zoMBd8@}IR1`?&ajdzk4f4KzVKgausZD79XhRL7njiaf6Ow@>y@f44a9*4~Er`~ zIRryD?vO?*gk7($otf^kyMckO6rt~}jc8ATipM$x1d)ien6Mgfq#?rx;G4CE!)8Q4 zM?j*^epMA5N#jj!**<LEytRJ4&wcn|R+`Cb5qog;&9+*HR6lXat?=kbE>mz*Jw8*1 zi%A8;*nBWKMHAyE)9I?$;eg{QT(V0hXRCG;mIssIaK)r=+j);q;-!y|@x!0fW(&@2 zlZN?V$iq!Xdh3l^<9)XdIPA?qx}J8iR2V%$ls~T#nICa+(DV>!gL(gMzTLi}xH{_A zdbetFr8S-xxuEDl-}%ND`L&#Tz8)K9RKKaDmeq86wU;{|c-F|#tOcMWcWuYwzO~@9 zyZ#&_M1a(dr2UQZ@W%;<hIh`8jeUru`$j#O+r0{OW^+y+B*Z>Mi&cqBM--Ei#m&(& z;^Hu#<dKD!0Cy(;w23MO1#%zZd)x3jvo3a~I>rg1s?@AV)_$YBKo3V)su(@_6Rpk= zDR~wzo4`!-wJM7klPn1jrD>EDp7pU?JOeS>YfG=87`1>Ve}GjOmM)~o_En8ZXl|ax zk!it<N`r!`#onWA=PWS3YZd!>#y<XOvYlMg(<Svj%N@^}ULSL-MtFPfy7*ogC~Q*M z^hfE;l*B60;NH$_p2-aA)g39mkcUtBFn6`&+AIsrb9n06DK$cG2fTe>`4g9>G_0^N z{rAI>wlE2l^p}?VSBk+WkwF3@<LhGe-z8_10FnFv@{P9&#iC5<G#VO#kI(TzG+#hK zwERwg{HKk;du2zo_b!C}OjP}zbI$R_C-DZeAW(^cJANQI$iBsNh*x+hHzbnK=0^y0 z_{n5_{2G1K1$I3=bbD%y(#h~mB(Cv>r29|4yt*aPWeRzdFd@6)!^#T5p+2--l-}aA z?=||J9+**<wG>ZW9AqEY3U9kf<3ZrOqPoC|AQ)}{qSg(7bdMTApL5kOh%vtL09u3{ zKkSkWIq3s>RTKoq2zI|;PftHvXJ|AwM0yxeYk2)g?Hd(aQWfV8^LuGpq<Y>URh)0C zf9y&gDcsWtsBM#6bvzxFfxK~*D-_gDz}-O%E&EH2l5f;258k{Mf!{#JlCmSOF8Yb2 z^EHkmqV;)-SKpVPUAbdH`m%TmO@<j-*C&7JFq1AE-s5ngJW10OZ2ogG2*oqwD7-uX zBiH5ShDf;N6_q2kD{)gW_@N`y&Pf#$1-<GdHkWhKc@?sql<t#P8|zL8`q`kQdT`2j z{?D$*`<!;mI~Tg$Vl|r190rz}tO&w4)bvE)d;<)4#>fd&G$BD3kCuBAMs8jBPenm} ztr+J%x{xcoB?>Osv`k;3^<X|Q(!^`(Es?Mc7<=b=P$#2{ufmj#FuhFmNx9ML8Oo<` zNmxBKhfiEE>}22T*N?o%h6yf1R8dlbKlG31(Yz!E!+n5l=r9!RCudMs?~<?wjqa?T z#zYhW_q^fPE%LPu5GaWIYe2sv;)D#3bEjMfs70JBT`IiVpK`c?#hkKU55}20$z~^f zEGoA+DbaCCnSCw-`hCu}GwnA@>&ibV#EhK$F20R>r<~-)BYGZE{!rTM{h2Dqd42>L za+28BAyY#f*YQ}<{*WN4M=S5G&-y=Iop)3dYq!RgA_1g#5JK<0NKp&|kx-;5p%+6C z1VuVX4-o0S3DV1<cMy<{bWj8-0-+a`61ov0aL4nV^WD2{{+YEhD>IXK&0C)RKEJ&Q zST5EZUY734zmf@&1E<rf)ykD6nU5S1_GAh*{PZ+i+~MW(h~zg8+C0SWRJ$jX?Vr5W z*LolH$-|cd`m__s3|w*m0O}tD^;IZgtkle+JNu2=6=gM}z0@!y;vvFeR8^?6Cj{T? z$Z)T}BZ9j^O=jryEw^|{j!!`|{-N(x6r8?wkV~2_jIao;q>&7j9t!JMo-&ykf||_* zo@aYwcmmFmIgZ8j_2e&G`xEcShW|V($xb`7Jb25LVnF%>*tM=6-%R)0cWn&;1(Y(= zPND=FRFpVrQN0~%*OZXVIv@IRmWhtJeKAJYt%%UgTR%$Lez-x;Rn?1tDx^!_em{Y} zA*N%LsqOpKC*&R;BTro>{~+ez()LLa8uSawUD47_V~qB*kzH2^S)hx(niYJ+0qCQN zi&oghio_AnOP$1s!epNQK9<zT5~<g#Tm59{NMes%JnEwI54c#5SFR_&AD{!5M>aER zb1+Ny@c^^{!9>(99v1g#OBQte(5$;f;sZKa>3Mpj>@mA|7n10dk71yUhNVk>Z<-oL zOJpK75F68*L?RL{vVs|B>=W@ctIe{gOr2gb<mD1Wl|4F+1eP}8O5aSXM&3QDeHwn# zE+xPO-2srV<91<)bc&s+!@#J~A>Dp;n+`?X)FU-^aWM!v^r`SjKzVA)S=JET^xhKX zkSaYDt(spKJq?%4m!M+FmrSTv@GwHVA155jnDK4D$qxk%PfoiH*GF02TSeSd2if`c zp8QxcZt*2vO5P<&MEMYD*`x_tur$!E-A%xC{puy}-s;GG_l98`$uzL8T@TCuYSZ&p zWLD229Gf1^J|kV<jE=790=M8p7?^f$hM@yNywaW1OryJyODffNaqAAHFIDlxacVN6 z376a?!=y>sLn`ex3340YL|El4b)$Ksg8D~M>DN-muR|q{T@-8w!ZQv(gwTdSr>u4U zwEyWKfhNezp6EMX{ArmaBSDSa#FB<%7Ogzg;wG(vuI6@k0WppcyNq&f0J;w@R#Llh zsp%)6t-J4mYBRNFO59U|Ad?~HAZLhqkxwl_4qZM!0Tp8HB>~!~`=J}np=Qh#`(BW` zxH3PU`&x!6pEmK_7&q4;w^U<?Pa=2DC?jvH*K|3S!*q)p4i?z9TJ<9uC51n*uAS%X zb5&`%xbT6IM0HuC@@(vWa@n$KvCXj3uM(2Zjrp`RJfZkr<x1^8&hec&@|SX&0cJ}} zgpCgCU1V|ZjxCm1(K%~&!{v~|8v+iO20$sF$z_}nrO+*|v*2Z_UngIv8X<GU_1M$U z%0*=9+Wc=NX~iETT~w5^PYDjWZOx#q$V*B0zwr9iDI~O5x-P2Ph)EI@(Wy#T^S*nW zO3{E&G~IL9ihrv)To19HuyQ8-_d8kRv<HkYw?LH%hOcYfOO-@&{c+iCs;;CxY|y#h zRHf+(zqYj+c0nT4ngn}GtikFXYwVP;pB)3!9jMgp!7R@-rtXKouWRWD-hZ-si+)fB zu*Kw#iKb{jgdZg%z*s|0KQCH6TDo#el-6SwArA^<KF?Ev;$7QCYmoW{kEs)N5<#tS zP`juPNF=<4g{!4_btFudwA?6!e9SzmqxILAOb32Py!FmgK2I4ePK$mZne7IfCtP@R zUX#<Sob-@(dGrNkK2?It;VF*Clmu%tBwBTeU{DLeuR@9UAFVK|p1-yLeJfwF5iH}M zm4SHuWt7Zs1;+TEPFuB8Fe)b{6lL1*9qw<g1P)Oyy9~y}GgTb4;1M*c=OT=x#kW&_ z*<O#1hcGgS#*kPf>mb6rd*U83@?0Q=!7%^{hj=D8j;O4sb{u$|SJbN@H(TG$(S&=g z62cQa2PUO--jd-^L5VE*zdleDSy%;ClY^W=C~2*p>8U(W3i%x`j~BWMJ@^O)2-!+N zQ42wc2ba;Wj)wE&l;2CFpDF2oYCJ2Zy+dW696%F1)?T{yJRf26bJ6|k3MOY^G86Z$ zkt|TMYxydI1UH;90*AT?QR$}$Br$PT{ydUI7bV;$1^Z*NUG~i?6hbyfD*G?a^ZfBN z5v0{|%et8xamG3~#AA4Cf_c?)Ubg>2yo6lo)!oz{zm^TgL1DB~5?S&l&YxeJmJL|C zCu3hBI$`&MM^j$5f-WCQU&NC?ij86Bf+L%qXu^!?IUphyz<gWg`zVZvsp!2gRUMZ( zx2nEtZM}JxPJ|n(|6GM@veV77e^JuJ#k9lF*%<kx9q#Nh#?Vqmb8QiROn$6WKN=v> zzG63ED@twAVRX@rudg@r{8XEOj(!OY1%uIn;stM{C5fy-Q#N9-Mc0pVm*N>U7^?B1 zkl}kRL2J?x(E>%DMe{Fdm5I0&b?fq0S?=@0>R0ZoXRIfn>Z3@^k%mvQGNJswxWCxv zs3Q~zOr$_l<T)`4G)-in5vnCK$5a`Mp0!XSIzIS|EzKTsOh`2A<X_sz*jE~)gz&&) zpeicDy#RjnUPNx^d*1dWGFNq|07=V=aytFo`uk{9Ehyt-3A(7|ra@wdPZD#chL*MA zp5btI_tW#N6&EZO<nlUto68Qv!+ERzdL)<ov|u7z?n-@u-hc?fezc%A+cVq9y55J| zJH}SS(hbbdQX;S4KUco=d3x(L4p=ufq`+2B?}xTaLEnKnE|Fi@$`zzAbxD@BK9eL! z+t9uQvh1^!y>Rd+i!czAHEi8s(KvxTq<dI;APS;>#6UAqNwh=e&Ug`RC!eBX=IYe1 zgv)|ak&?MM`I`0(bQ!B&h@0LE?Zexm{Fs=e)(9c5ewxP1y0fpYv3+_&IzW6AJf)yk zTaNb(Aix#g{?hD*D|QiZ?xp6)A=DFemVQ%o<MPweHx3jq5i<JTyLI=gd#vSOXY{m$ z8LzBZe$;ChW{<6(P&1H^ic?D9l#v>eM2HBLg_)|wYpwE$XZ#H4)b-1eYw8%y6~fNa zl<3My37mfUboa^Vi=tbFCM#~#qWhYOp*{yHJc7B;)y0Z!Zq9ZF!pl~csY5^ORa9Lx z5pbBi4Y@G!tmI+JjnfKZdm}}%nO3Pjg@YMg?GzUK#2egF<*U-tFnc=0eb`zLJDvR? z#CnvsT`FtTX8zQmkdO6SUp9eL5||`mTCru>WccS|V*R{CNv?Y!eE__c3eNAy=MfU} z4DU3y2^ZXWl>;A!&`(4ahlNVq|6npMtT|yf4KeKXzyDe+#Su2yGjgHjjWPHtm&_;8 ze1sN7M{hpx%Z_6b`Lk>s)X`md_?rd(cx{0a?iJC#m`5COb(77jHXIhU%y*>FPn_P_ zN64L&<gk9SVjl%7KnzL)fJiO1L*6-O<?AHVNQQP8nK)5sBdxBSJxr%Xwc|QGz-oqY zr!4qA=r@NvV?7_SF~cWOkiE0rYH#~^;9#UpZE;dqh3qE3xwV%M+V65VE|F{*9V)>a zns*8#kCs`@f=qsU*tiaKxJIn$kiRo1pkOs4YLC=DE8*yFMdP>D9^MIHC6j?U`Riha zIzoyd;by(meIhqCq>K9jVP#Nq1z|<rz5_bfqp7B})=oD}0ff=Urh?_VAkAt3+@(bc zT|Q{NtU){p`6BuyaBQ(h{JElH&7RYLLT-Pe20vnf`27hssO_9rEx7xunNhh9OYi|& z*(9U*WMoQG<sfkop~^26<v`pPj&_d=6s|{~os;(WTZmWOB)tI>((D%0yaqa_pm&=q zLBLD9onp59Ff5k|#8t2dHh!|>c7p)X_D_tO0f<aS&dbaIU?e$FH5}@Wu)AEWGJlF{ zV833L5E}GzQW>{<Ow-40k*~i{o%h_%^dilp`8iVe!zslVZ$XIn(E^>xMFGN9hmsOC z7uC<4U+IDqN$dYZwf=n%m@$#4lj-}P?Uvq?+%8Qax|(|U$ZbX50-MNuT5myMbykMe zeNf&#@*&cPeTM(v5MVbo?D-#3&g;r#=O*fcnv);>I5RYya&f;+{VMqRU5WOqPQPDU zZ|OS9X72r;AGr{XHHr4z8OpQ=g{kHR0F~ezZI8w^lqg=59z(ZfFe5IWg>ckx4^M)1 zonbM<Eb>oi>+gM&A{l_knry}G%_p;>D!1SB9zhLwcpsxr)t%kB-|aE;8Q91GklTOS z4an>0E~fo$soCjopWvS$?>}df=z$j1oOhks{S10sV>ZoBdoF^XqK*(E@N}B-=#zn} zd6k(R1B^zYtHb`AoQ2FNy3CE6C^fX{9LQ*5sp4FFLhxyNgNW9rjs`7Rw_0PmDp&v7 zJ<Zq4QhxNJ`kXA8nH9G={_49kZA9C{`wc`dV)1nosKW3Ka`PNblfkL7yS(urSnJ41 z-dP75T{|P7OaS2a@p$K6B4dCZ0_?*8f3&DA(LtYT>;8FUy1bP_*x4~jhG?er^K(A2 zztmNafa=?C#se?*nqjxZtK#ZhM;VO)QY*s39Vq9H*~-4d<5uaRH1y6o$n%VTG<};O z@kx$#e!(QaqMA*`z_ft|dujt1{dFNi2c*ASxB*@HZX55}wYQYr&SWE|M@n>CGf`<T z<x$d4GhOA=$IZ`=pEetfL;!x&=U>!b)iE0ALM~1Y0;vbAUYIp|(R~{uwbfhb*RZL= znhR{!KM13Z%5v!?b)JY~z*0>QN|S<7n#bST(yGkRSE0O1$FkFUc<C`;Cr$qhniuCo z;2;l*OxZ}7fjy5j%^vF=lXe=1YDe6M&NKd{kG01oiUd4eS4&2Z#j=rUZToHG6!4F^ z#E`43v2>c`cM_>@mkU3<ZbeburJgvp^t(94?y$&Uv>4vVky4@TA!mJyZaY<gt9HOD ze8}=(X5poGd<Ccu_$xZe{XUoPJ+anIR?JION&YoAC|Fy*w0XD^S_df)*YMu{7zD87 zmMETh4@-#_PHq8NZaNy~J<dpAt^$JHM?-wJB?!<TA2OGuu}F)3c7WHEuC@ByGWU}t zh&~}W@V6@t1^@Kg|K}Xa>*2z_^~G;@+}77RY@$F;1lf)8&Vvv?&l3k+A-n~!8(*sj zww7>y$a=>x4E}cmyEn!at2B*$ue-j=<lcJng!JruIz<Q@zB#%W8h?-V2><w^7QXnt zdO>cs#c$bnH1^9{=<p?|cIEgN3AhEI{#q&Rv+AxWNzR73f-cX#K}gQb@flnR2_Wh_ zlKbIcjBUU<FA`sAP!`**uH44~5WMo|3ojpSsY2MLdSlb;tx6D3?`jN`%%P4mdlLt_ zbm19@_a>jM_WTGy5!qGKUmZqAxn-lbKDrxcd1Y2)Y|M1R5KWfi0pRaTnDR`sS={!f zR+=cw@P~c(N(Qq$AG5Hge8iAa#qzNRr~a#4I^IKwdzzSgP0Xzyy3gk(J=;zuc{`IZ zau?pyT7E#f{YBkYrsi_v@g{mwU2oGy$5cGb_C&t_x1~_Mv2l?^62RLeIn3A5-awHT zncUo-_+S)gSnJtqQ7Xr~O5O6HqE~*=HQWiPAky0;)=Y6ej^A%n<aX5fuCm7}@AYtd zYIu(;c^mhINzAQqle|k~_||q=P+Lw=;FR<iG{MuHh%pVb2~4YJ#T{*^s~&9KOqJZY z+QZgELf5pm+tT-Jvsx{jGM_Vg#GV?TZ&s<G7T{8lGVRD9tiE}7*yVF%gMC~4hL*Xd zl;$^}=tFtrMutK&P~`-dXi*9um1_<>-L9?zJ=}ELoV=0AeiYRseY!Z7QC$=z@4Y)U zym}6%09=a{kN4iVY6Kf4Y8~}h)EHf(Q{#k#epkG-_WlHAYSk8R6D=>nynFF>p|tx* z$(5THGPI@$6}Mjo)R7astJA5P;oZE}eY3xO4oj`PE2%m)6&f;s6@vQcU%WuZI?h*S z+|jH!JWNs4x^{3{YT#UOo}Y|)wlkvYo`sP!v`iZ%xNUog&{Gr^#f`|^@%DPL@Gf@2 zF5VFAU;QR4sBzN29`ukdHgtqWtBbuknryu<S&oshLgIBi%es=t=Z+WnpETOl)B`_~ z9n<&RpuZouwwXLiA9kF!h^9JCEvv{GdRX)&Y>@3uf2Ii2Xuz)DN4+4Yx{^*r_`Hy- zj;E!QEMV1@ooKQhWI!pq$(3?Ok8R+VWJYx?{J52M11DZIw=T`QR7Je|&0En8sv=vK z*wMR-HWNt5B=&{rWhqRs_kPe8X)qd1ag`G`a?==Q*mJQ<(7JI}lQLmv_$jT@bR8bS zyC!^WZAn#B{Eq%HobGsII9Wfv-kgPXaJ2tL7X3&rL%P039oMP4;Etw2c_zWkcd<KO zEpujb{9Di8GM0D=GMs91CyOCT%Zl(K2FTW0utY^4l~${j0WQ#>)rv!8l;}dgQa@k) z+1CAL+l+y}=ds~vLqmb_>>XW)`KhMoi}Ou}*0+31(PrV~yGnd2HrO2LXGw3x>~f)b zYXE+lnJM7pu-X(N$~4TqVCQODjz>gdBiY@W*kwG+D+;9MRLMnnA*De6qA_uxxyJI| z(e)Ev7nMo(-q{kS7u&w^bN9zlN$bm?yXTzK<BE?(>R*n?XX$8Wk1GoBr@s|c=l_M6 ztw`%7XgwD-oGIWlQ(P{1FZ@a!Sq>|!E)MskQEnV97uPT~b=Ddj_H8f~DPr5qL^q*X z#VSQ>9=Cambcmx>oPJ!lFi#V&v}8ycvV$E~=OoGc&4@N@&dONU^i8hJ7@8e}1A?14 z4UAE<&*wMbkt)6$P^^L=tO*)LK5}Ge?d&h8ktlqd)qnZH(N)v?+^3_9j%gT2FQI@l zVM~pNLO#@EhW$~mu+Ysy=E3B*PIFcLUDMlyv;vj0%ozd%t?CuK55tsC6IpLNZm^CP zPw`5F^r>Rb-0I$0MxACBhH<ofFis^mQ~Ve|mdfy$id5TsRe_%A;us);Z!D6GrL)l= zH#{O2INqFWHN$C$ZFf013k+X(-;}DCHNXpV07DH&>4KtdTsQf+^0#|N!gE2glH=)R z$x$CPw)1uQoPmyML1Yqu;oJmr(MJtx8*tTjWn+Y~Y_dZJ3KH<7Qg!~R)0&=MF0@0# zwqNDvkyGC)E$Qv8@2i%oQI@1e-qk{U`+7nK#0+;o61F9f{m6Iuc4rOb9ahQN4$=7< z+I6K>2NP8_F}ys-ziig9eQO5(eO19I>GdPOO<M%5@%@eV$w4uYTrBa;x$gmX`fOD5 zNoj$Rd&;`7QV@65L3poJN$!(GD^clI!`V)lNllXKdtIV%b&5@ongSi;#M9vZ?6vLC zB<`EkAGzx})qSmsBm$?&!~<0F)j=CdLJ!#F;smWStS>Iw=CvWW#F)fDNU_!thIqQU zk3utZd*cCJusWm40f_`TLc}9O+(k`qe%5(jC5*qX)8PR)MIS;BhtS8=8V<F5Epxr! z!E+n5p35pf>ZzAkN&BVSJ{9_Y=p6nij3b@%yr)up?$l+Y({@Z+m!)Cs?G7|bK0=oM z^W7GewBfb)@|+(qV5mosq_MQ~kKFc5fn&!mg3Nsw3wAs;+zDH>e^t?%be4qpC_2A8 zG2lOJV(%I*b@uiWVliks;$Mbuh`?%IN=P7#?&FBplj}@P$$6ok%lv<mM5N@@di4>Z z>Y_5v3_3548xgNV8Xrfc0w|3*!SreHaxrOFCcH52%QjMgSdwuD33lp9wVQ8zR$UR| zlx!M^qAyU_MD?com(W*Q8kW|I?@rnK6<xc^55J>sCqcCy6H6k;X4Etn_n9y&+{a3& zk9ZpVic8(sPK|}%QchjQaC#}3#T`+hA6J`$!Z`|wnXB{bSN@<foK+Kb{9(0Rea|Lm zOyI7o-~GAMNF*su&I?xZdl~99))K|+#H5D9jz<=)JbP33h&|ii8%7vpeGhl5|HP22 z`T%<3W|$<r9ds^bxgoJiSofq*;`T6WlhIcxMlwysY>~C$0BaNGS%$eK(VbNqAQgq$ zv(>>{L{@?o(6Bv98u}V=QE7<W#^)W9)sMnw!r9?hBA}hStOEV$0L`isV>p;;Qt>;< zZ8mVEABnVpv~FZP(zabX@M;D#6mT4lC5@oouniq(ff%Pvv@Dk5M{`9+9S_%xFXH@! zYCtq_han1gUbgk5*xexF(*WrRnhfN3KBvO}uqDA<)I3C-$%0xzLj`EJJ`pmy{eZO) z+SvLHk*NfYr9uTY(N}0#!L7O)fGoOZejeO;fDaB_aZj1=4pgBz#b3vmqDvrO$s(91 zsD=wtwr=t*EKs|AiilYx{>O{;zlQ`5&{t{ANzV+quYg*ZWWR1<0A$I^!b*NUW&1~g zDE?6(zi-rz0-^%7p%xtcdHfz;^o-S6?7xDII|9&#ppa+bfD-<X6#4gn`pO9y*lsH< znEw0Cf6sG<fJ|zGh<X~;{^nAs$%$ZHnyi*&i5dTBA3$^d@r-%wy=rs*UorK!rUCNh zNY2*tV(&V=|F_Nl#&Q6>r5Xgh*0VXyJO6#Ue>D*E9}R?gHRJLBFJ?*e3o;Zl<fX#k R^57cq(S+)%SF75D{0|49jWPfL diff --git a/site/img/favicon.ico b/site/img/favicon.ico deleted file mode 100644 index 46ecb3be94463c73eeece53179071c5aa3c872dd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4286 zcmcIn2~<>95-mZ4qe;++LZT6nMFiQAy=fK!k+ne)TSNqL0i@Ywr`Z=lK^A3EMA;ge zRb1kN%AmML<Gvqb=0s1<88w+0b+ms?{Z=_94g;PuzCQQ=_us2mx2j&%`@aFk&`(7L z*ekS50+<E>S_B#bTLL>SZ8SG!;v2jf82z=qwFM9nI>oKf-63}FYHzzt{pW-$onnzL zJ3d?!i<)5wEsfB3>KgR!KP=NwTQ=O1KaGpUBI){$HWBRY>C!4MExELMRWSN4^1+Z* z!M76Q<CL>9gi;yS&LOV%c#9hPVSn3>&)VzDCmiTI>2&=3U7cg+?)-^oxCd5WPp248 z4fIbgD#-8F(bh(il|o+r%WKP*@#miCKl(RuYqQMz++>R^J%8OUYBFvWiClNLbSCU> z+bcYF_EzHQ+yDA%@aB`@D2K*beFygLVPh{_S+*t5%yixmx3cm~Nnu{@Z?qTJ2*(L6 zgd*At0mUkS_%qr?%_rJKEkEvR?s`<*aQsotzUw^~9=tT$f2beCEloo_!*$p_4+#!} zy4sqhPLA_`A2Vjm5WkY5;wu}D^&egy9?yeU2L2Hr8}mFrCmSoalww_NwWM+PuGh7V zJy@{o7-sAo#N>wSkL!=zyVcp<8`C8gxsa{j49EL!eERh1;5gr5oQbh<vz(mVJMPmo zpG~MHbQ8{gHF@$gd3pKQ1ZJwj6jW26jSE*K;M(#t7`*KUuGo4Th3!9KcJ)3i$jiH$ zla&!CSRbxNYd>i3=g^Af{_Om8LLcELLMK6;au3Q%Hv{>qQCq#Ox@v31^G!vCn30x( zq2U6w^@~UAb#>@fco7$s-o}usn<&^ih?<t(jI6B8WzvyFFd}@=*ga=+eNR`%XFZ+m znyt-^F8R6Hv6;fOJc?^fZ+B-f#g0!h(PZlt0<!3A6JfKc9iu7-(5L7!@`|sbH(}An zi<sPWN8%K=ZE)Pg$*YM_n=t+Z;<p2&A1<ChGeOkUP`0C{>S0N7(Y>^kq_)`TsG7)# z^_8i~iQ=-7P0zMhZ^MTAIwbp~{LRfG6qFuD*POHHo_8KS3of#KW_|S~jA?vua?zGM z=6dci!_f|n8yOk^<u-SoiOFL%wOJB<J-wf;EG_RE=<6M~vb4Ctwc$Lm;cys^wG~ny z=iA#MkH^E{^agavIEgNqgXo%l1{Y+UVwM&TO5z*uz4R-;yLMUWW$-D!G{p1Q&b!)O z6|@p4&wXWNWbTh0JN6OTQ${}TsIA)ii2Qxkw7XvF_l}xs%Jq_0smVz<odS}7<;Lzq z$JBmwP9H$0v;oE?rB4!9e;t=s-fni=cw^$?Ew^Fm*86{@l|B<66mKkm{-Z~x)7~ah z>^J3RXBLtz`<5>8{l%N-^-sDpe7_~=zl@JTfh%xL$SU+-y5tJSH?C!V;^8ND35U@k zsgH1!;l>`2gjJqrbN`xK!4)lD@ip)z{{Pl?)olkAWo341jIohXyr+lzznz^Nzq7OD zZY9n`>%-Q)hzJkE$cS)^5=5|g79A-#T~v_!I4Wi1CG&MnPdPFBkQ;XZZQ~E3rC^W5 zKfj+@SAF$2_xy{#o`pYvOZo{|Tzc~Z`l`wb>3c;%VG2+_c?9?``v=)_ewM0g8J#;P zDYhu2ni8g_VB&`OCqCZ1-BYGa+5E*9pWiaFuz%nnXnJH3(T!Gu9<+|?MYHfOiEHwH zNqE%{j7Q%2%?`q|W8L$<2i``u<UVZd`NlRSB}H8mW1~BMzKaKmGlkC8hg5Tzo0W;_ zDaj9r`@)8}*i$s0JW2jbXIkQ<;V!H*StELF8Y)Kf@D8NAgd5q(gjSqGUf!9%JEjjR zEzCL%EdOwg;tbwio(eoK&qi}I({l?KIOmYxl^N-&FVj<#f1!8V0PXpac_zlKBpVqM z8KK1LG7M=BG+fnq&M2hmrOBEWG+xspF<mEOf{F)lQU2fqhm?LRr}P1!IslKt3&YWa zfxi9(cQ@B^s;^!nL&F$V6&1fZTC@Gh&%(t%J{6|(=GD*{pGmRNBPHVlHh&Mg{?%@} z0o$MG1=gWK&@MC#Zjg{Ij9=Cv#xMWGuXc%j{`N^n!7=qXxMU0t%gs(ueaDk;on%9R zgT0*>aq+0G12r|&!{~jpSY1tRHi^kQ*l}Ujf+W!PD^u6uS6-UCY#Zt>uR*;PI~e_y z^^9BM9)_Ra|AO)_krR7R#v!qf`1}7SH#_<H#EF)QiqmFJo-}FdH`ElTO`AhDInntu zi>AlwXwQ-A(B$QVmQT@yIg2+p&RJ55+J5CoHZT+i#xAmr;pZNGZ5z9<#9Yujo*VZa z*e4v8t{dqxUQP}s$;$)1+hD4K0_@ygBdrHBQ&Z_u9WEa<z4AeONf~>Oob6rsO3SAh z=lGT~I?JjUi#5BMC7A~$wlRBlnS^%ATSfJPP3-=WXh$_aS=9nGc?F>EmI)dj*&Ge` ztcMz&IXIhF$k0A8Mu9cxowA3qi|+0+Slv3!EW87(B71-v_y6J7^vVPB7qk`?e@!;@ zt1V1Jx-;azM3-NIu5n_tkLoz28`L<{WUUCy!`s11&^0RGMiNqY&jd}+T;kt2mh32* zwICT)os&?*GgqSRTY~c=8qhwnbwDRzrv~Ld&^Z8Rp=~3n9o0PfNxTK1;hqf?Z@#)) z#_!4wu}I?*iU)F5*Pwkw(-|HA+PMZn^<cEB0gOXN{+1gRISHvcC4!o(ki}ez_f4tq zP6-kXj~p}&C`Y@{`U{l%hDIy5gF|!&=&#&4TGb=UsXE5PESFTO^%PKZNsCuonEr}@ zYOWb*;8%n;tE(^R^2<$4R@T6@1qDE7z=%3OMnKs<hWt-pVL|(Sh3t@!J*dOWL5slh zt9pJV7Un_QK+!oDbe2|pjOtOz(mn&(FivHDe2a?X1|&OB(=826_(j+Be2c6tgDPOE zgAnK(8kN3}6`E-m1q3a!;b&>iIVYf=Pxf_PpFED`ic(OpO@i5UCVZ+k(^dck!<n{` zXO-=vQOh;?y0)i~W6CcC1)Dgk3HG!3)3q6#aG1dfoBOTJ`X5ytqOWQ!NU+xT&VZ?$ zXi!_2`ss+@YMx;o3IwHZIblr{2S*L(I8fpUK-F=>C*j6p%0d*<pIH{f@D4Ai&*JGD z$leE#J4WweA%TY2k45fVq|=?2vHuHY@jih8IeZQB_yUv(Tzm@Y1jMIf2(kn@f;>SP F@jq_qkShQH diff --git a/site/img/java-icon.svg b/site/img/java-icon.svg deleted file mode 100644 index 00a1b88acff..00000000000 --- a/site/img/java-icon.svg +++ /dev/null @@ -1 +0,0 @@ -<svg xmlns="http://www.w3.org/2000/svg" preserveAspectRatio="xMidYMid" width="300" height="300" viewBox="0 0 300 300"><defs><style>.cls-1{fill:#477ed0}.cls-1,.cls-2{fill-rule:evenodd}.cls-2{fill:#ec282c}</style></defs><path d="M200.71 212.986s30.992-16.92 30.992-31.11c0-15.418-21.695-21.78-21.695-21.78s31.506.98 30.993 21.78c-.513 20.8-40.29 31.11-40.29 31.11zm-133.27-31.11c-16.357-7.83 34.09-21.78 34.09-21.78-55.978 26.145 19.16 23.408 65.087 18.668 16.03-1.655 34.092-6.222 34.092-6.222-22.414 24.594-116.917 17.16-133.27 9.333zm86.78 56c12.818-1.21 27.893-6.223 27.893-6.223-11.398 16.208-66.415 15.21-80.582 9.333-14.165-5.877-12.395-18.667-12.395-18.667 9.222 12.373 41.632 17.763 65.085 15.554zm30.992-31.112s-67.173 16.486-89.88 6.222c-20.065-9.07-3.1-18.667-3.1-18.667-14.94 19.863 92.98 12.443 92.98 12.443zm-55.787 56c70.265 0 99.178-12.445 99.178-12.445-9.865 26.157-167.828 28.974-182.86 9.332-10.616-13.872 30.993-18.667 30.993-18.667-29.884 15-17.576 21.778 52.69 21.778zm108.476 0c-57.91 40.63-154.964 15.556-154.964 15.556C183.09 286.832 237.9 262.763 237.9 262.763z" class="cls-1"/><path d="M138.723 156.986s-34.712-17.284-30.993-46.667C112.07 76.03 155.655 62.75 165 47c8.356-14.086 4.716-33.125 4.716-33.125s9.12 15.25 6.198 28C171.748 60.055 155.407 70.533 136 83c-14.53 9.334-38.022 19.768 2.723 73.986z" class="cls-2"/><path d="M157.32 166.32s6.465-15.614-9.3-34.223c-15.764-18.608 5.5-52.324 49.59-62.222 0 0-52.903 23.227-40.61 51.125 4.55 10.325 11.537 19.297 8 30-4.915 14.873-7.68 15.32-7.68 15.32z" class="cls-2"/></svg> \ No newline at end of file diff --git a/site/img/kubernetes-logo.png b/site/img/kubernetes-logo.png deleted file mode 100644 index 77f507fe98649e97ff3b13b98f53953dfbcc6c3c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12338 zcmZ`<Wmg<c69t0n;+DYT7Thhc_~IVi-Ccw3;%>oxaZhmf0Kp|V2^JiJJG?wU;Qi3m zr{~N}SJkPhslIiiRF!41(8<u@;NY;pa#HGV^T^v6Kt+C&Kk|(X-zIo>by*3x+R66^ zZwo|gkP-+Et|0;A$rR~rjpib!>kbEpJ@DUv|KVJ20SCwa11ts7^f5Zgz3<Q*%KWj{ zlR-((t&E4Cn^Df4QXUm$#hV4*N**kC?fWo~+8SIW1+s7j;YLwMcTh)1mm|W{6cnLK z=){UkC3T!V20AMRRG!V;l(u&q<_PKkz6{8_1=T-Z33?s|9IgKj!X@RhV2q3fy~)u3 zFU@3BekU%q^{7;@%+lor7E6je>2>%xI7awbYNX%EYUPuVynitX*Gc4*Q}lVI?AmJH z(;FiFel*C5Up1KYSu@JnK#vK#tyTTKAGkzd>w@`wy{d9Zw`CzF{r3sp#0S!{!H`%G z_26_O;Oxs1O|8*9V|RdMn!sH@`Vc{c@mF)yjGdl{1G)QK=h2`DU6ZUirwaUp+EV~l z4Qa7d)Z3$QcC#yv>t<`AGDC>QJ>?RdL$6N_W058qugS59-=BUfHJQzfd8sXOaeTSP zG^I9A1k9|8!5m^RPVH37;A8v1h2{wO56<8m0fRj#x&7c%`pV!B-sgdyuT+>8n0N>) z9pQr)ak*PLt4Bko=I+dR%qdQDk5tM8wU4}`j6YOriMZ37BfGRni=l6pyhj+;`Btm( z6sX;md{r*RMk*=$`+d53>Y1S-t5e`N>Ek<?_ni}0tN1VwdwTv8W1df!WDZ~PDYrg6 zQ{C#c=}b#F>23zvg4r0&gxTdrteOGPJQ1$kjtqcPV@KT6-KJcrMi?zvWEGp_ac8Nu zWlyXfc@|rT_nF;J-RL{!OA==!jwT;wq(3a=$=#MQ=($L~M0sd}ga(a$d68~zi?`!x zDsC(8WW9*p{fjP*IRN8}<~sYS&M)Rbj}NhGRNYiqmg|2N@iuuMABNg=eoA}uM}Fq_ zj_Q6X++-M2MJI)j{DG)E4#t-058~8TX!nU?ny*_aa&3P6Bh~kK9r5NQ3}^4?_|uR3 zI&U`<^xXsmm>q6UFz>$WqL9>Sxu2Edh$hZ`JsB4|-!T?WO`(ENkV0(gZN2L<8e75d zc|l~I>S@d<!zAD(MUJt;bj?U(l`1W8GFDAy5*HpRWM{Jm$8Qjd;^wQ~d;qkiE4uXg z{jdLdOQmV+<@2hvwB#@&g^VcV>StHIlWa??0|=5_$^2o5>)DFgdTZHL=ug_?aw?hY zB1X+t%F0I}F*{ONhqK&*LAgkUtCoBy@@NV(@s?CyqO5A-Fqgk=)slvwAF1U$r=0av z;>90TYh0N;O+mFlvY+`B&P1y>aozh*<)(>$Z$J$eh7ot`<Ly(O0*Esn<TS<Xc+DiI zjzyO#6n1#ox3PKqUzWM$%vTG1d^|e@EZZ$GM+sh~5ln|kx>02|r8awhusaYqpB}P3 zCja?nnRxgY{=9e0OUuL~(gK}?#`Zs3(}t-@kwDz18!`=6_AMkIHDs8Ck(Z4Z>(HYZ z8PyMfJ78NQH<><gNrOSEf_dL^gPTG+Gad^=Di%alR~8>iX>3fh5G-#QTy`*M^4+)M zV$DBG#BPm+?}O9Ucjr3dpBsk+HMzU4FjyX_Q>5a|Sik0K+j%S<<{t&7=OfC-WKxV4 z@}y&Cqau~Ly@%m36Juf*zyAFtq(zf*<Mgq@60q`D(YI0eJ$I5w#h4~NK!T%>9hTep z@LW5EX*8dTCG0;>uRxw6&e5{R^t3zCRB2TzvjVl1IwXOLRM=BE;Qgk^x6hUgNj#-1 z2^l$mc;Q_zTYTx>EPJHB0XHR;-<UF42A&ZNDhsNuKyPbp@biC_#i*(G0p)&Os}~vh z8W-!`xs9ZlT^2;IJ)5om(>UZj$|Pb!oqeq~@_1pGmC1ShVsF>pK$`_}`LHGNUbsh% zl`DBaGL<`AcBAzE=y!WWzIMX3aQlMwPr8*e&av|pyTR%b)~z#?t1@f#L^<R_<CC$8 zHW8*>_w;s$tWKfQL3hLM0@u5W>sEK&DuKID1DD#I5e0L>n?pO!>jtay7HDY$Z8T`5 zJ$&%e*)v*fO}kD+SgQMH5+_g@>FRQC<M@4`pXlGNrON0gk3~k5i}r@Z;ZVmmTkmhP zI(U6R)(?=%@AuBm&z+9SZDo)zrDl%oHD>4cZiECBzB9hRb`xfQ889q{84bJAXd_oR zQmCdm^kivQ%}$tN63Mj$@5)Hsf<xLQ$*PCyrO>gJ=Vp?{!hwIJHm4L$Z2?uk&*FK7 zC@~;^@Lw-~I&Q%#n(R1Ld1WxnckrhIF0R^(I77)x0w>81x{W6hAtuG9RXN_)5S??U z^!<alJT+=d00mCe-FNbf3z!U*xexS3sVr3@FXttMYN0YBuITZ_2H5G%94CF6hVhp# zvW8OHeq44K=rP%Px=Ej=g>gV+-iXVG>iK6u=>Z>BW3QUSE4yO3?8iiyl9GzetpcVZ zC!{t_j6bE)x_n;1`1bAB7c*Lf1wY|WGx@+K)_2TW?__2Jf_U)kLwa>SIxaE-&>-W! zUu|3_B@6xcXXH^aG$UKYraSn=YL1A-atvxnz_-sc@=J7xvBvDU7M>x~q|j0M{S{jk zEHns!q_t_3Xxdc8GXHQQEw}wuF^4**)<<GMsvI-a|8!0GC=K+|lNIs)1QGIQCGaFb z6ymVDSf&SvBO<QDYVa~nVt(xZJfPJiVVOAL@3$}j6Wr!gG}|uLnDaG^P;z?~Fy&g% zRH<H9DTDe)L&tzTlo~T?69bm1?w9IQ>Wu5p#g%~zyPP#mY^Et?Mcp@ni~pkAWNk2K z`8jCWIy*r3^`1ECYRU;lSLrvQOoUI5UF=O2a(-UXHgdISq$Czd5n<CLh`(P{%H}O* zRq(ROc~kHQ+%;bIGSCS*99h+pSC&O-<Fhr34R*ey+gr#y?n$8`_RaSth8`Dkj*zeV zr~13`kY#WnW@qfPpE_c?e$z_G)~tDOSNqL%Ur2v`)s9{4C5U0<q=9v&oTFF83~1xS zl>CDU53TZP%;V71pY5Zs4qq+gL=~-&NePL{4fBvK*=cL|ur!Z3|HaR+EZS35Vs)20 z9~BL}Ixk8XV7BnpsX?`t{a}*=3ljm~>@1GYTj((+<@%&2B88q39a11X=j+U~L@DR4 zNzccq<z{~7NRUwNM7%0?s!>60T3qO;N=VMheO}~?wY87dcFI=FG=;?*bDC)iBtR}) zXOPtO0k@;Lfn9w^OfdPV@v@CnJAqat;UyT^W_vH6&I@>+Dr!LD4Zf-aW0|@D<<A?& zom_M55~hhC)cvSO`$bM0oK=~CS6}AI*Y^xGYU>%wBsfLC_UF-h;lmt!?}ynqFEf3@ z<NX*|6v>*4kwJ>H*6ErcLz=hUg*e6u96g@QOB#gkH|YuO3wd907#7QCHH~6Vg+WnA z-$rftJqC&=%oL)LuRrr##XQ0a6Bft0k3FaMqfKIqVZkEy%3T*6Gow)WBGcVes>Vo$ zuq5<1udF>;#8fgYiYlr~=>0(>6RZlgm1_}Qtu*TXi4|<uk2|T*TU%sms26gSyP-)d z?uOnz>0(l2((8U&PW<+VQR5?_DFe4#Tn#4df^Q~VAI^}e0*N=9EV>?feD*N=X99<h zA5@QVxC9dNRJAnKMP9wil8xB1Y6_bhk^&j7@3r*%V}8lR1<Y!<gY(*7Sr();fy6gx zr(bG4CyGi#Ph<`#MxDc4$AYx%a2g9dvGy<}keLJA2I|og@puwgkm%(;xcah{mv$&K zsb6nhSW>bkocR7+m9n?KaKI$)y?%Psy*|{1bh7h_di`m_;f*@Qb8^sR)8tt93<V&N z_8A=BI&S8G{>=o%V9s;E29XH2zLsT21XT!7=+D)980pKx&&FX%8n4As-Tz8KG-FwE zb#QYKUe6++>Hgv0<`D?7Ib-V*JgAH$EA7w*NEnLzt+d;Nm)yfFBFhEa7WqdOKsQws zNanmac)~}PaUPE1XdQ?h7#zxH_zd!`mNCiAF0(2P+ER+uxnzsa{U!psSyMk5m-}?? z^~k}W;3<<rNqTjdz)Pv^5vcMJ3J1zc7e9!$qmtrtP)jlp4}9nE>>7Nu?n-i&Y_CdB z(t@&rlP$xaB1BIeg4To4J-!*093#%FB5n>{l1UYZakSm#{kuUKpNJ(%cNqhEvc9V% z_s!r$^8+ij@C@4k;x_b0I)=h~2P7cJ9I6Wf1)V}Wno34%w90KI$~i*ClRhKkkCfMy zr~75=>Q&vZcSk3Oeo`}r5X}ryg{%cXLUk3je9@-Lh6#fb)B4HHSw2h_F&t!&cn1OP zn<?gGqxj*^n@OK7c+9~uv-wKr{|>R~t`m*ciW#WRr|>PPc?aoCqWAbbqU}f1le4oQ ztu6?Arv4Tg%wN6O9*TdXsYLsIi%{H?yx3}|mU2j{Rn8qvS<t_6E5B;JJW!2q|HEH; z1gOd}vdXG%yYWyU%Uk{@>;7zx&ed70r%=15i$%Z^&`v9pR50>)SX;ncIXCs;Z<2#h zMLun2Uk5hRp2N4UG^dPW6}!Ny7Ox5`=Ny<3t*)ocI6j$=0cug9UFIK3t>-P-lpY-+ zxPgXWJ~t+0MZ4~Kj;uOgc2Qtcy+Dz7v!rDxPG0J|<+q&hc00xPLaFdr?oeI76SS#Y zuZ|ckL36;@qe|&GJ1S59vn-L%AIC>VxaLOqe9hickyhEdX97=DPY~G`$xS`ln(gY< z!!*7AQcG@PJOT%Y=usO?>(H_AK-YdOUZfKCo`HQqWk%{TIq%Q#O3;nbIs2g#^_HOP zF19KgUVD+hlR5e=iGD|^J3>GK$|TGK^hpvZ@_ZS1T__gLlvBMd>I?i2#p>i{8T%dX zbDLK2oeZU^fQ<6=FQ?IB{A3J#{qwI3DxXpem`Mnzu5($c=E2P^%&l<sh8llmiXJve z&Y>BnMgu^Hc{CgztlwZ40dqjSSG=)CdZqo=oh&~?pjx>?q(WkwuqjnGuq<?mB<~9a zlQViH`oSg!{$XtyO@f+@qu*<l<K#OVA(KBEWfE${XZ_zVi)fWua)8yD7y{;rC9^<j zuA+y1ClTf?is|eM4Q!ULpd*E0rN?!|L3g=&9we;@0^<*d+a5Bk5UQ!KhxzuhQlC`2 zCN)^|sMX8R0ip2z))uvS#&7S^Pj4)rsOUH-kIG?%F+hAcTnypEJMN>_+*jl-lLot% z`N#?e+J0Y451DQ!!%x^)&52hrksv^CdYz>zlnJR3iP<cPUHXAO=94)oa|53I?@`W+ zx<yJQ;#}R~h3fM^RVh237F51tub=-nne=vKFgJl8wC(3D6<shp2C*QuUvC2=S8hJa z2(XkxT;UloC0^+dIk!agJO^E4W)$Cb0DYWQ5-zA4{;_GwZ2>V#3{Bo0P8sG*bMRvd zy}Zb%{U%dHI-vw?#V(`9S#)`4T#qkbAHIb4qd}_IS88G}Io?47BQ>BySu_izJ$Nbf zXyx?Ag1-$PHTf8&E%|z_T>F54ql*%EY(^Em#6liSJ!yNnP`xyjr2+wTW^zc?qB04h zCsh=LetDXtyeAEr?Ori3c-{52>ZH=Ee+b!<yP?8p!{W2?f9IJwd}xFBy!9O`@##K= zW{BP|e%pX_+<B2bdu%(O91hf`<yu&_f>~kY2URITg0s_7uOel~S|A_l?NyDdmYHWr z{Arl@Vse4~xUJvP60_?w%|yr?C&yJq+Wkt{sQ(!NNwe824?Bbu*6k_Idn%vx48N&m z3V>l45&-HY7y6pd?Nj~6n@r0-KtviV*ZCnXcAI*iPFL(2Ji5rwuz><!9;zRt&}DO& z$`e8O#P7IKrKQ(Y0TpD(rua#Jh-Ik6MnlAMcpMt_4}Y%_)AgO{3Smcf)#EzmA|k3T ze}pw3f4xbyf0C}`M3v1Jw<Q=9oq>g7UIJ0afeAGysxv=fq;?KQ?Fmq%EtYh4n?~%| z2Gaz~<1EYR)1f%SUzni*;{HYV>Y|tSkTe9mtSms5+|el&Td1k-QGuXs2nm#qA(tGj zq+_%dl`wKtQ5a+V#T`k=>!Z#LeSB8NPx#sOpK|>JZcjUQS{=^sFztqb$f1#QIXr>0 z8hpp-Jq|wW*qs@pc9j!Cg~)TeT&M_~a{7yjZwE&UBvdNpgd+;<vnvub%sUO|LW8C& zda81nMSs1`rC?+XdN!>bJO@lX!iLPEU?FK(O?smSV#iTsa8%5jTGbQ9C0B~cJL1VR zanRHoM$ok5g9fV|gb1HR>U#y)G_(e5gw-q;V<^kyy@<nnn8>Sknb3MMqGOWD3+*y> zPHUsdNn+?4I|Ap(H-6!?M)~CDkPW5OCFFTQk@L^|yoft=t`e&FMf$|QV*aA$bn^lA z^{}qyEfk8IO;NTQ+)X3ra-l0u*&=+EG%87DjJKQ@FI&_qI;W_>kMB59(Yzf-TDg5h zCH$y{j2ck;n|{U}R>%&OiY^Ch`aFmgx&bandSrjaupQI8gDyaZKfHH;hLc{@MWoV3 zf}^Tq^p3wyf)+p*##2HCZp{uZix`gbHKRQ}tx+1+CEiaR%>mq~733bgdq@XPAF(Bb z@<nR2dGujzVoWO7bqI-PQwFnJFjC%Y;vM}4-({Y1*(YR2x4a8iJ=ZSRTfS(>@Xq0k z+5&3$%=OHIL+scv{7|Qbv`ZZ<E(K<b{z*Yo6wJ(TFLp`#`hoaur!E)wF71>7pqzE~ zkoZJlEAsF^qmScUfu@@`naxY}Rr;3j?g4s4>*t-XTP0b4;k97Wx+hc4)r2i_XvUP& zQ)lr0cmbkfmLJY`CR8$ng&C>3-+&6efuu8fwypJKRlyw%-hUIoy4M>03LG%!7>5yB z|8rBXoaF~2zcK|@lkS?{%P(5;z6kcs;T}{TS3BWrn)4kMN~XsRKWDeRKON(WWvf>q zD!yb<Kb6Zh9%$d~9JRITsNWz>xjfP4_5EIXv;w$6rdK`r2~n0SKhHx_q6_9SMH8ko z-4Ex%x|2E`HTqvv@*HR~2WU3Vi|r@0Br>9IK9P*HA4x0OAo096*!_;V$`6@F-~r#D z9KtCrD2B*~j7d_Ou3rQBP;~1|VO6y7Bz3UuKVR8WlYYJy^CsKUQuIOiS;79>EX11S z4LE^+VnF}y^BFx&L}c2@qq7q4a)3K5nFw_WQ9r`?xW?ynXy_aTLuOZL>PAQ(hh`Vl z4DT@+;+TkP95k}ky}JZZvm-0u0z1ESlkBXocWte1r+CIDyIZV~W3Vsyx=*bBV?kr7 z>+yhO>wt6i)2S)0N!Hiq`y`Ndj;$whDV<N`wGmj1#K`XNM?d}>0=CKsN_d9U3q+>G z78aIvIia=h&a=29%SL}Iw1Iuqp8ooyO0f~wNkq7TdahFOJt~7z4Q>Ruuc*Ulevk>( z@p<G^<Y5e`Hl_Wg56x}&ynnQs_j-r3Hy@aX1t|^(Myc+e>TZzvGRvx}lKO^ea=UyY zkpIjiMbaj9?%1x1rvi@tuvPIVp}Khyu9_5)>FROWQ^xS6h#EnkZ@Z<|KLgVu=r)1n zep-Qfu(2#lQh~}F%Z}PbpYlscj|OErD>>#w9|7hVIJq$1B*S8+dJ4h|`Qj6dICMC; zpWf+WkXni?WY>Sd4Vh5ALWoW!TWo5Lk%fjldp=jv;Tv@WnK`{8m8M;Wgmc>pivQ}l z0>;N$1yh)G?(3}*cgcm**z!`v0ojB_?j2}yy+7{{t{1uK_{bTE%WT!n={(d2`P`Ez z!}y)Y*O1MHEq;pm;S<W;6w@QvBNq;R!V`|<aRWPesGstmasHI0ei!WT?TNr80cx5$ zU)DzLya`PAaZrj7LK59N1EF`LX3So1q}4>K!k`Nt*rT!w0b`<{xZ1dsB~%mJ9N)1Z zAEced5M_iF)w%-MgEbN6(7UW*A@QGw(=4`cmr-}X_wao}m9H<`o#^Mgmtti>!jHl7 z*|)1go}yZT=^a%Umm;9j(HO;^i;Fdn#TZEWL*q%S%@8mRs}0w_^yQK<azc4gQe(7K zZ=oQ;zIHbvvV@WN`%ohtjzjY{!vevY)EdL}Mqn~If8a=CoqUPjeyyWIPJ8PdQynj; zMMQ0?O_bkE?=3(hy@xP~$#pnU=W-G=z%wjVBv4$tQV`HjB8k4X<0}?brtkgLYwlk0 zD_$NQ*$UyRxP$!K%y(9D!*_?f=LSg>XVe96IOZ=+maMJhdy5<T2$0FYd;??iv@f<p zz*sFW(|>KWurqdNE*!jMws?zQhE*g~1OkHR;q?`^>jS+7ZS0NAi!E3Jxcf~pM{2kF z_lPujqqiE8ZUuqj%d!S2b>z44{u_MQ2z|h6Uh`!|*WKqCrT5%ad5<pl*p9Q|)g;h~ zG}_;Mg9^gFdb#^tB#~gY$&$)8@dN9Bn7%N2cGADSzerjNKw1>Zy-QXjsWg4!3fXbd z1fR6HibuKQ5z}Hw9><-j`mP2m1PMu7SZ{+<Al3`dPM^+6NeHQIqeo-1<DlZHQVEvG z=$ZW)AdYrb%O#F>%J>)ph+-Z0?3|i#<Ub-a03ZAT!&49<e2}&I#+A?a!;+s4?!<TX zMJl`TnPk&zII*H80(12wk>ztHAt5iAyZ4jO9H&W{a94Cr3_kLFnuMPO8kF|R^Joa0 zwSO(RKLlBi1Miv=zS>Ev;}M94f^0%fip-K=8C8U@PozsQ14xkhyXm}?OlmoKdEE+K zZ!GG6*5h3E>4U>i-My}TFY!-*q*nBo>sGRqzZ9rFS;$YC_R??hU;b|T0puy<9@1pe zNgf6+O$&Ig2u*BMaY96%hyO+LLbe6ux1@w^?P_MKF{WIkyG-0KE_}{&Vy|ZWv@59| z?H=a$`jKbT9;Zb$J~eDOtPq~HA64k1lGGh}-<01-r=po#yC!scV;VqFEfGpHF61sq z5JN@|9EFM|GC(#`UmGLBd|VOhT|nswqp@#biDvt4oA-Q72P{O%&TIOmCh*^#-ghV% zCJ>^!svHXf1qeC{XPWnWRBhMRsAaz6g9O?1q&A8U_|d$#&9q0LK?d>m1;x_NJ=^3A zCV|&^2t!NYAkz-yo=JtTuAyX}MqkHjS$k>aIv|r(fSyb>>vh?mbUOTpm&R*LOpBA9 z5Ogn%<h4!;S7v3-<5h=2pg$~6{#k1Jse~g^$zx|XO`cZ{|6lnG&z6I4av^<Dw~vf) z8S;!L3F?88M%dlb0Hxn!_&igt^BG;3Dk=C657>am3_#X!KATJcyh0_!I8z|_J~IF( zXuw0z%hwN)aU%BI`mYxv8Z?LQAfbm(mjnVxEq(UKn%Aa5ZZp@dO15dIjENDshC(~} zn{P=_&{NRoGZyqa=Ji%5vBA_U?Z{dQT$+E(;yAZ}mJuTw&}I4()nt(7vABGT0QJ5I zY8g=(aT($;?++TVwH*B{ay=6lEpd5w`CdK>{q`-?uhu<9_nM9rRvLwC-(&|l3>+uK z)w<cj&B63h^g(@zQx1*I9R8pX`vzI)Yi=JGxZiL|z03MyqLAw%Src7yZgk++FZ?%v zX2N%>{3l1Ebxaue=`=(5Na3q(x<w+t_chchs9yZznC`4gQOiQ^R0}iz+C`k<*voQf z%V$CK$iLQd?1%cuX%DSixol~3o{#P|l~0++$Xr2acU*4SUBglMrK-c62m=(1Gbi6@ zA&K6IKO#IoZl%yOO3$b!78MJKuOQQnz=HAPx~+Y)m);jYiz|&mE&|r?Y`#~b8Q*A^ zjXf4^CAG?^_jPSv_g87s@e;jfLsSo9*0Q<DbNY=)mjR$>TBvNMg%~uIa}uZ^8cO*1 z2n}XS^=%_A_`U^s+NX(8J_WM;^hswxPiLF<8#YFP2Y8&ibpIfkfs4nh%I|8J94mVa z&r4`l!_-;C$(z|_LCJlVXrVyDFdD}o2N8U1_Dg%<a(itBDp#Pu#F@g2FeT!UDt1f} zQPoz<v3`@ej1(zwAcoVUkkit9)Lkqbi;3!dm2qN2&lb2awM)<;?&P@_;_ipm-pche zy1ufsIfc_`nVvfC1V}!S-`=CN!INruJ$}>HI)dK@)<0QbN5IQ)O+OI46MJnl091;8 zAdV?ygaA$^kWklu=Wr$eUBO9qMOy1$`aIRMUg5EchCx1bR#GXeN#U7{1>M88jU|(d z4V{QUk-8P3@MRlR3g^A375_$sr{=9f6J`D9lc`fiSW39{F6jaA9%5g$U<(a!nNp}R z+x0gz>*aaoMHH$3W=ML~ZjB~Ut|QjdAXDiT#&4>ItoyB7qfK+5UQ8AI&*2W~L995H zbDvUYnf`&56ze)vBP4{+0YLH<zwfdrg98$;c^I>1Qf{HPrCcF|H0r;l>uZ(wLB^ET zgp6LPFe~liX6Kzgf<NA4Y_-j?PW$BD$;&tQR{YW<MeJJH-g+JPiy}<7IsA#j-B}lR z5uWWaMR<+swY8Gdqn!i6bgJ7QKSf?^k*}%Gcol#A4NK{>XZBQz3TF1oj&Zf&E6BxS zU)`mb;Bf1oJ{(2Hyea&J2#+{-5id7R>ok=5k<2>%f?^kv^0K<^K3>(K(gVv=K@a(1 z%<d3DU;&7Bcj*c<4f27`vuX85(Ne1`?@4*6X;OAatZY{ViZH(3ob6GlviHsJ|7OTc zq4%lPl%wUC^`7AC_{GHrH&9I9o^v|b@q^HAt|5%e*v_s#LO`j@{yg5tn*;OgRL3^R z?D3B!i?)$C`lrz>(-X;4_mz~eci~8+P-#;b2Tn;xJ;p`6fI$i(?d7@cda#>D9%qM9 zKzx|A^?O@rJncN)7DFlU)UCc)14<lJ+G*TF^D8t|lW-%Q1^hE>4rv%UpO<gZV2UOq z%=^%Hc_009Kg(+&_qPI&Rx6A`8;0lFm9*$0jrvRBFMsq40#^YL^2ZGFGbyqiql20# zI&SFLCEhsN;qJ0b!)>?A2%vUr_~E>l_8~LW_fU3pTX^5y?{H)bZvA&AznpHJ@pbH9 z+*%l`5sfm=a93v0cWYJB+fJ2rUeCdf-9S^KWFA-9?auSOfxmizkw@F3EIX%c<G?qR z>~Rq~1q5VcSoJbmKqQ}XgBAn2N%~yMm83S5KuCtuu5NNpv!*wX98#u{NGvOwH!QWz zIm8_SL~^+Gn0EZ%PRs2|rKKtM^$u>kK=L~BR0EIJ+YepwSm(lnTQ28Es)?ZKZQsp% zKMS%*Y0K}w+r~S?$vtaz-tps`X>5u7dlxH}gXDmXh{ts1LFZuCl2GzRZ?SHJ+*hgn zk%x-U#%V5t(z5l)HTj?6I&X?R>TUziAmg85wh>^!tQ><lkxVeXgODN@@30VvUW0=h z(bjAwnjv2me+1L#R`PKP<pY&UYr|iRO2h|Y<{zEAFJ!!kwqO5ExN4~{R1gV&YQXrD zY(8`=h$LggJvt~vOrEQj2!i`1nq0Gl*+Gj(j!D>O1)&ljo^-w&cWI=8{mp2*uH;!J znjbm}#81N_ye($-*NQpT&G_|oSunttXkJv`;MpWa14oZw@S%W;(B8%{6|}KBSIj1X zZ?TR;n>ihJ-mf*Eoli|N(g<x(Yg*Ejd$s<xx@SC-K}OHG!vu8|z;Xy#z4~IikLN#a z*aA5BxA>JTU;|M~rn>*s6)gLoo)n-t#$S!aj1Zs}=>hqCa9m8QGd*b8<gLBZWUA!I zPIONkREdk_cyl9BF>F`Pf;h5q%Nd?UB(ua*vbIIB8Djo@$x9_9ZRE2stz{XCNBtV5 zyD?*+9+4%Wmnk&Wf+YG>Hd)buV_tHVz{AR}*e=zLokdGI&}q{BW}@VvS|X@aT}XIX zj8)Q6T@+3iB`2Jt{<Qsm&^l5Ux1ky7x&5x2a+Ot?I#e0nr~=KDjudfPDS_CMIWAdh zf2f>X;DchHEI1tqs$Js753#nJ`F*ntHaw(s+07!El~?)NV=e?@p`33?f51Yd<lWY^ zSt%;yU5w}Z-Is_gA$b>1fa)Biss|+DM73q`pt(IFjY<)CR;7&mwF(ExW)z+>hy)8* zKFN8bhyKIPLVq6^(QVO;`(XI!Kg5aej#<#bW=A{w+Nbprn$0gdQ-oM{x&o$aMEEu5 zu~mZ$fupW0Q=FRUP{{rqqa<#&bBf*pr0_M7=P<aNVn)m+2U`fw#t31^$2dj@i=53S zdwQu;41Z7w{Q9>gGNl)2vGUNjQnM~Es+gjm?n&#P%~n`yNi>X!&<r+D(saV_;sYZ} ze(#fGDiE4Rh5<RgP->k-c1{HAh4k7S)OT#mVtv?By*D({ei8jjEn@%H+O-y89C=qV z{)PYb_D<N|jQf`$B_d5G4-suW844BSzL<9Pi#csI31XttBqyN^u2x2B3I*ZTFS;9K z-r2v+dv5MMqML97%RFw#6Umu@to&`1xw((BxeW(#7(zQ%!0Y!3M2nkbVy(?G)kzG& zlz$}X?FC{HO6yTgdpnA+?o#xGor`%PaO|yVe8su=aFBFUu0U}bYfWBtnC)K&eFACS zkG0mTa|1LPmp>M@$axPZQTUK6k+QQ)C3#A2Fzi-vv2Fcs%lkP-$-^%RFE?XA(-TbR zip4YWr%JjNlyv~{iKii@rpm8<!xl2Eg?;ZT-nTz;(fwMxAy{KCq0d}a{DM9^nrB78 zVVl*6()i&tb7ti~ECH_l-@$hq<(yhgf>T1mJ%Kjem^3}2ce#^d&EqR2%efdK{b+LI zp*#T^iL8oibaH7loOmK;pk!b%y-H2@Jvayq>*a?8pHSX1ji&OMV@WWGy1o_aOVEO) zb8tR0j#nBfR<UcS^nxtB73^KK^$%E>4nsS`ijevRZX^iKNjbg&#>>b@r7UK*YRd*7 za%oB)+2b;dxE{{Cz`t)Es(lV%rVbjRJ5Vn(wG!ejT0sVxAj4yPZjO=MwNl8Q!8UdI zmY5T6`ey=%WIum(zWw*D0#8JNN`q2(Fl&8FvQU2kNg;Cbz;Ntn!5f-Af2nL>!*?qj zB{hvrX!eZVf=5bx^akGyrca6%98<)unl-J^?~vxU+MkyurIH}Kn5p^@YJVQ6kBl|@ z_0Z7tf>1!S2?fDz;QAexDy{fHl_UxQ=EDXd^8mhoQnu_lehM{ixW&XYaykq~wAt%F zDYR)bM;@JDC~rR84&R!fi@12KG7y0zn{Z(0J7g-|8-<4hb5{D5I!PkBO2cFs{iKlK zb;5v3WSTe`;SE+kTD&X=qs6$!3SSvZ-8TY(U8#%;TQXym!qeRYnbw8ESA=V-r_tYP z&y#J%Mv6WKYTZ}byOl6I&gur7o%|U~ykVu3mNA+w5#qJ=f~?Q2p&H2Wu`H^jMd}EL z6x5@LDT(?Xk*V`<MP;L%-5yG-p{W%0G2*Mshq<a4Y-s9M@?Siy&tA~gp-VUJol6b7 zS&5u)OA5`Wx9TpFKsdBfiLx1+h|WfA^vK9OlqgEt53rd6WHaZ2;rqCYAK>!x@;cMl zT8uEzk}A(9t3FShR3Zs2X!&}9m_wWC9z($FS6lU|NWThw2n?T@lg)5d(OLuA4-CfB zY%hY<BnHu*BVd!ci(h9mGXU~IB}InzE0yq~Ta5#!1$CK@O7Tn*<^b{32ic4L;r%&L z6EakN^kUz-gi~rT%E4(4g{@{45XxTGp-}vVR^lx!N3s}+lhK?2ya*0a0z!Xc#b`H| zB!gT^O}5of^=AOJG;|)w@FXWoG`6B+J#W2^kBL*NP6gkTKRaf4NSMs1WKRAwrNKk) z(-QPuvcVw($7c!L$LFb*V6K=Ilo8eNlkNro<YC*PxAv@(A?MLz48In3@A8=2afHc| z_stV;#Q&8_c#EEs<Ad8Q`lul_<@xOR%b{K$JJ>ns0#=AmN9ex;7QNEzSn5v54BAAz zfos6q;*I^1K_90z7K|%_lAE6GuU6d}ea2Nw#k!0mviN!TG2?_o-3UF26BWpvIsuuP z{5!`gk&OE8x#P1|#eq^-{rQ<_K9<n6vh33ZVlD8Z?k#(<9?)MJIcP0U8Q_#2@?auf zpcSn-8cH%pjwGsKI*?6qIt7RDK7HzWBW0S=lIOqwZgw8B*bG>D-YX>l({dHDcb9=U zy)+|JL2#_jK1)1OE9c1^kDUOU$>DjHWMqtr_;U8fyYq{>rOq9}XBYK2fdH%ALlhBP z*Ob(o3A3B;td;46?!*-#`3*4XT*VzjPP5b01JBAA<YKY~)HK3U@m)p>8~36au5z72 z%<=3UDGeyo)ad1BX%SHYH)MFs%*~LLk4@!-RDFp&i4r;@A~6H=wPyI2;~Z$&WW%N( zRE3w19n?{I`yZV%2!^RikTqPW&jU0%K%C_wTH2#2MSu(>$uPRbMMD}Vb(wppU4w7k zqnt0(U_Vf5<$!-kW|~!QZ4g;XN#dCiF&XEwBaAk)(~EydIVqxK8huuk`Xe>}X?=s1 zom0U3$S@Eb@5xogw-a#+B&xPDkre|Tp_r3C>x4wSfSubEnH=i!#<vavzC-bM|KwSS ziMFyz`~G0q5((loUxbk~m}^DNHzVP*ojO~tbaG>+Jns6Ub3ke9Es8S<S{7KX37|P% z<zeV&0?{Ewtk!1z$OM>WS5Jl7h0Xi9Tc5)btdrWprEBII%5uDn&W_M2t7EcvWgiWa zv;>TdhP<P-QSwzCoCF5@z?Y$tYqsd90b)aHoo1=6CYs-C{P68*hGvBsYE{m>%f<h( zu?nbi4EuB6_r@X1wp@JTNJM?}c5f{AY&@r&QV5tFC`0q7>w&WEz++LKdB*VE3$j_- zn65Qd)+VzA{j-S}!~7PDQKkShbnEfj1f@DzJI>;c`AB#&&KzdPR30nF@8>?l>PRNA zG2q9g>h>e3bk0_1keN6bWJZF>v6@r6>H7YR1J#zSgnHH4;jB4Yk~T_!4ozL1+!*t* zWJ4uaggNt6$P|~861{bcXH1q_4HJRy_*gqvxG27n(${ygl=0Vf&RiHhx~+X!ii(2- z_3cnF201*SBrZ%X@S;&qP2Y@GZ(i;_Bv_io9rv(Yl<EE1A~9IeVhw(y0ERT@FcP!V zqi*s~rDrW)abio*^EVDepiAwbc@V(F9wx;#()%ZoEGV`y#*O=%t3cOjP?J&xDaAgA z#A?ucRNJg_%}CYU_^wjDkG{p*80#DMSt#@E=2cQnr`3C#ZtKPy7W@?~%(TfMIE`SE zNt~2@BEFBIV2<7j_{=NS6nP>k${$GYOU7{NaZ~AtV>Px0$?dg-#HQ}K{r-*Z#g27f z#9j!{?3N_zAjA8!JZiRkrT!2djDe8)_)59YCHfvm_$VvmvvYFU73W-+jC>r=x+sw2 zfdqRB`)kr{6+o=H<cVC>lTInEI5RP5^FVMzAXnKHZa2wt?<IlxCg_6NZ@M>Tpv6KF zO);{RZKL&0o|~_CnpxWb{iZ23Atr*JP=w}p>vr!WqEE#;_`_m1JNchsWs-_^Z>7`n z=u#sO0h8s`6ckzfp5xMDHdOO1D=k2Vl+FELW)w+w%O=Y4n<kjHu%qTVQGLERMpW)i z#@=`UWCYi@i%BvL$to-W9OgdNZdQ#9*^#N{sdJ}{adNDXMo;W$_p|cPel@2<76f9Z zsdMfoMoGWGU|GAjs`&I0Hg2QxF^QhX*ENMiE`9kN36=%wEq!Zml%wgwM8Q%ViEo60 z@B6?^t2dG+98J)9owbZV7iw`%CiAM8ve833>sTmA&a)T`hqNVghJxMKvfaLgkWEAZ z|Lr=QWmfvy9KhG_i%dTLkd6;gCW;0dBRMiWPm%Cu`2O5B_t;EdvfvvF8zwZ9jxiPu z>K%S#+s@!u(bPGTOhDC?VL960Jj$3DQa?a~OOAwxOqoSLnI|(wsw2i;;NZa6-#!`3 zQQ-T3<EtioGFFWd3OVyWICErg@%GIEAP#8}+gOBU(8h^Y&x&)P2)gbJr?HHE1&J%{ z6|~Be#rbH@&f5qb3|7P{eCWG;<Eac(8RjwCzv*Xgb#a_fTcu`16skizy_q~dUpG5Q z)`wcImP4|my(w?aM_q!l8~@E3fV1Pc{%n>__DRJqdhzD(&%MtUnou2@DH^$mOqNr7 zf~l@$rWKm^Fj58?EQrtkAt|1|*dZai<FO_avv<!@<pbBbwM>3W3w;9k(QH{i*K9rB z$+h6qkZ!~$&O;JfNn~;h%w#G_-2Wp%sOPU`V=`GxY4!MTJbyT_w6avKgh}ZC0O=NN A*8l(j diff --git a/site/img/logos/bk-logo-with-text.png b/site/img/logos/bk-logo-with-text.png deleted file mode 100644 index 80ca369bbad475400a352046746a4d4cbcd13fdb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 117674 zcmaHT1yr0p({_uyyIYY`SlnF~heDyaTX8S$zIbtWXmNL1+`YKFyX#N;Jm33&ea}ZZ z%V9Spxsyp|=DKE*4VIS`M})(Jd-duSq6ARn{i|1y6t7;r4u^qyE(y;^+<gA?+U~u$ z(5sRmf}Q6Nu+~6TyH~H^(SQBEewCDh^IYMlsnSRLk22C+`c@Y7x&~HyhV-8;te<PY zdd2gZ>-nvPp}j82XA5&nJFd^XWWQ^0J-`1|3?L)<UB%vvm+YgAJc+QCtsx03JqtYp z86O-82?>v_ff3hx5wU-pKY!vSGqJa~<^lkmoSf*LnCY!-jRB0DoSXm#CIAx?-E$2( zJ7-IK-OqHEcI5wb@<08E7~1LEnp)eNT3M3(>Q`6K%E6wOjO^Eg{`2$CIPFc1{{NFK z?f#wC^8^9E{sUm7X8`=C@8_mGze>4;tt_l<4ejin`{!fj`Capmvj6{_f9lJdSlL@W zOTpGuU&7Mf(Du2pz3#8k@iG0|`sEt`-ItuLsp0dif464*xAp%n`?o$1;Mc_emmvO; z^6%1TG4sLk0RFRNd~juJObf4G3A~aJ5mfs8dOsD`i$HYdvC;Ul&pis=)(VXDbsH+l zN{bQ%gdmD2r=J|^`nkZl;@K6Jyud>p1O%oM8JTTR$_kL9olr=qLIS29-PSlFap#d= z?!(l+!I-OzqptB)VNTw+MVGzeL+vBRv;Db+mZ{f$kby{dCD>3~P~PnCqK*&iS7wk~ z=3gU81(JY~z3?Io7sw(fP{1+3@)P{HbElh#?V{aq9GR(o1k;_XkMbJz|9*Lgi+w+$ zhCaU#4tT^=jm|&&?6x!){Z3~8qLC{8-8OZ1{@VZD@VP_paDfbA1K_EM{oZ@W;3KNZ z8zNTaR(1KU1xAP62_!o=!j({&?ia@UJdfURADvAf2X!QzN!Qe^K&s^m3nK}nZ(8@W z0r6N?ZkgEx+2O(vA_D*SD<oAQu<v>29lx#>3`t1^PScJ)jNHxmi-+N<?^8+K0bUlB zzrd(5P|Nyw<miaM!l~1{@k;yb;4b}JB1_d4Fh?9oAizL7C1*?o<?w9Bj=oX<_I#%& zSqb$e8NNon=77bx5VoM_HMm}$C21abEn>7DM}pA!CMH<LX6ABH3uiSh`a1m>aJO)N zRchxk8{!2M^uo)H2O{J)`ali99OpxxuD_;_F9f3KdZij=n7c8I$B3q{i*mnT(Jrpy z!4p*d^~G+2yvXGzd?=9gsMp1|qfp3CBKiz`z}`s8Xj&LbsLrU>=<eY8In)Z4Rr#`f zeQ7Mu<!<W`A!?Bwhzx-odIOO;9Hx@MC?<mU(b6qlExl>yO~L?Py$=6L!P1vD=5skQ zw09O31t~Yu5M0y<nMI?Ax}v5#ur#fCiGedZ!1B0X3YnO3b?sF#tLO`h{k#cyN(&$* z)<ux)sTdpH&pBjUd~C_hYhY82mR9#&qC0O_KTDaw{L2yr!C+LA?A)vfXcZhVkrP&$ z6RXA)&dxsE9kML<7SX|kH=OsZY;kH^{ACP$TwtAaz<|e$?UE=rYnf(-U{_!g+JcJh z#VKav#rDD;`0e-LmMIO#_36~&7_Gnc^!8UzVz|+Mh7>@)0m=r)Im==OT$dkZ6=Nk< zGgP)~tT3)OS_d-3m<Gk<FK%Tr9axNF63&Z>6T=bWyPxop7g-dFBr<Fi=fD``V13PI z+nJR{5YC+sZo3YOU$^I3!HV%Vyx{+>QNJ{8Ih$O*%4hxdr+NU2qDckWjL&&bfBE2> z89pY;L^=uXkOjvZ>w`XNXXhE^O&QyFFztJ-7r~?8pa`v(6!p1$Hbr0*E?PGd*9a2G zl(h;>jAF`p=pF#Z^TPfdgNaOLGv_e*ym2Tu%8f>U=f5*?H1gqZ+6DuIQA)CN*N#ZM z8;L?1_G(8f@iFLWgft9g$4??@$@Qw(#@CVPG7XpI_5FB@^F&aLwBoa;eNl=Qk{O>N zdl@=%Coho88Nn!S^W=j;9Y8iaDau%)50lk%Jk+t1y|Q(>yAtFsTepRDFZM(>atjfn z94UPr{K-f($yHR1bxbQ@a2=Q<#DpK6>tTlIrpa@(AWS8lbYIn*l0$k9Wi_+^ko5TW zZ=(~>FoBc+tPC^ZMx&6^aN(^)aYi`fMjhH(+#Ta{jZtlv8|Z$>GP@$$+hwL{FLm47 zRuHcx{3MF?vNlkw37+2OV>NHz1%D$Ew`5W=5e^!FixJB)*d>R9lIhAhSot~2uTgy; zxp0-nGj1q9yD~th;;A=msj9;ImxWe~6{z@-?cVW2OQUJ!1f2G6@z7Q9P23`#r*k}^ z>AnKo8^Bo&Sng4&Y6}Z<mhF7sg_#Os(7yZEW~Ex(4|_zt5xFfdo3!uY=+v_a5>DGw z{N>gg7vu46SY&HA@_1&og8_ajU)W5~$PRd^7UpMXcWpFX&fS*eD4E!7(42A*n6cfr zw)?<SBWnI4=wk7VZAgj0)!Ut;r4-<;xPa$<WNlOqL$9m!Y_ennzLc2UMtI3)-mp4g zm`A5?w3}B3y^`W2>A3maLQ#%TJ0Lvd<*^IUqWu!OdO8j-NqW13+&hmJiYOkkD4;pR zRd`V=Px2>y3JyJG`&J2V{(TkJEy7mUBff80e1uM=TG|f}_t(3jFF1^6BfXE~orOuE zSnUv@X?tDP4d8XTuG4^+;we9tq2Nmsem$m?&6h>U>oJ#b5`D*Vh>SJNfBv1EKP=zT zE<71W>!w)_l%`|u@)w;m!2}bf1qyKxK!J%n?;H<GMG!bH#EKtG4x`pz_I6&aobIWZ zQ&K>RVZek<;2m_%)mY?+&+_u$40s;mGg+oTXd4c@_~+sR>MV9A3OoCI2<l!o7#R;J z)N1935&1N^)<K?cbSay1xC%-d&I@$;pEje&1JMCpQ}XI8iIGhr_-r9j^Rsd8h%4V4 z9Qxy$T`mdh8gi;~!nMwY7g`Vf`8D<OwnbmIcRGQvAKfhj+uhDzTi0I(PT%m&6%Rn0 zx<0-&hnj?s<dME)@BsngB1<#@imTr2!Rx1Mvb=k#;z!MA)Y%4lc=NTC{e4vT=ic@3 zgA~nw@uL{h879mW=q;^LH&N31_Uh#HLMk1O8Kim+;w+VjEH^n8w9-j6Hd_=hv4Td& zO*2-X-JuX}y<`>A45l*Z9qfMiu`0Yg4HBteHiF=X8aWQ03{x}`4cU8h5YjMg#6XCF z&0G=m00wi|!oXSc%~6x%F)Edt2hu|rC>Yy7iRi3ZE{rdk4f6$j_zUK>{qixm0Y4=3 zdEQ&Cf0!PPomRyYGx3XxH)*WFd_%)JTLdIGd|wKh5WkLla+*zdJ+M1Q1&o=C%7jC= zKF%16%??P}fB2i(ko`6MCAoLmI@^-U9DYAS!ZD~bne@E$^D+b%MmH6cpl-`PW?%R_ zw>Plf9kG1wh^q)WEjCJXKDo0jJo8XrQjvODI2pOWY%;=wH7qh4En9=V#54%Sv@1g- zsY>E)^gxh7CWL7vd3kS0ylf>4{pXvbW=}llw)kPs9~YcRq-Rx6yd|g1LwUu2(d~Yi zX9*>qqleb;D!Hn%K}s3{jr=7qS9*$oR3Z&FYx*J*_`Qa*wmdPCVQxo*>`kwI1L>=9 z;|Fp?{79bNN_i}At)@EfU)EsrStA6;;2-=CUlo6cnTeW-ytb08av&pzE-DLB*z%<m zl`zXU$3;vik{ouu@Xx>GzJ%YRw~6<_-x`_YTnlVvGW(kvWQhKDTS|UsAe7+tT%813 zyd)5iI&8ioK^hoC4y$fZHI{uaE!C8b&~M^{O4DAp1@76%%ed2{pFZ5x_8G-b`|gL0 zK62)_znQ?ZUmMB!qCHeuKyx1GA4QfuXqz)&k}IZ_c7P#4$urgs7ct2mO_uFEx!Hxt zC!Lj3Z`-@-8=w-7+)vwJ&Zd{A3J)&0iQL*;P|<W#utD>(AASZg6mqG=P<Au|vl_t0 z>Nbk5<Kb&-{baE(EMiWkY}^<`#cHNCmMwN&kx!-EkEL4@TUw_fiAMIaPzX+%jU@Nl z_;P>4w1cGb^FBH032R6de8SOKAt1*N192`Y^Bom{vY2dC>X~_Ax}2iS#UyM$*3cC+ zG9fE}R6#m;DQzARyNeKu1pZ$ZoAMV_4eqa48ZcscF9lKp&Pq=u6nQ0F_=d&XHMT4y z!8h&sOXewwy_N>rr@I^zugP-k`LNF1$s|?vm$4k*tLMw6s!b3iU*nCIy|`_w@Saim zShUuv3^~CpHaV)8%v?=Yrn})+10@iQ=`AN)c>Q{}Nq-nRTRQ{J@sj6#q0uQL)c`~L z5ETOQu~x$#zJiIxUz8e;^qGJdNU$U0Mqe_ShXik)n#1`k<WX#N%u7G31s)wL-<)Q{ zK?PZ59|f!AngxpTptD_HLNn5~+GugN7(O@IULb$6OWU<>nDu1ji_-{t^-EeeXPy8J zSVWt`e!YRkCC0vhoLBXOF_no&3FOK~tzaoAy^J*nQY_{OV$%pd8RoMxQ#o>6S7TEZ zLT~c@be#Z2jg0tmh<{m=k6@y7z|$bKw5N)G-JEM6YV^bHK+kPWzZ1(<q`{#+%cway zX@y5xs2YEITp0eO%xsyXLuVvv>>0>1*xAwJc!!vn{nGw0=5GqZ<DG>~kqP7*7Pi9; zr6eCS4W?8a<6G|1&?97I$Rc-}f3s+57N{wfbHM36)>Z9I+_E?6oz8i5DuY3Gz@c*0 z{C!yCDOs`mDlBeP>}7*U5+QH`AFX8@z?lamum{Vm-%nygz*~u=bDU;>Dq1+_G#Dc| zh_i^XFm2kga?jq;;sS<!zVtRtM!sJ+PG{cJnK#f3c=Re}b!ZLGGyMxYiTs7AdM%r2 zwF1084%&;yboPLY%msx<^&|`?f#^%~KKB&$+(z4#_ELwVQrm4Z6XoNe<CPA(*&gvy z5s$WJN!22LAun#-GHg5Ee{#B?&feAk_U2`+UxvYGCgF6L-RI&z*UESgR2VU07OMw| z2^s`sKNv&VtkbWJ)&yleF2>M@gBy@3_*Xp@<5~oz@E$eaOnsBR-DTGvE(uk*rMf@f zTuR}p4-L)yi`n;jBY*Cb!wK?x!tfV>3%w{c8N~^X-i-dF&p<ya^<^f(OpjYKy=pws zLIS_7{<DH>Cp0{=k6G)FcG|6(*cE8vpxEr%iQ)&;{KCHghY9p;6rY?Us<dU&`u;q0 z3?zkm8}(#vQ$yjHQ5-|1_iij_SvN*+!4m3?8W2Soqq%Y=;;I18z}|D--od)>M#bEL zaUdE8nR$iX^Jb>pwY-$<WdZ8I+QSG|^nb|&tZ?nfP~xLyWkwMeU&82)N`wyBgA979 z!N+Jo=Q<l-=Ice){lsPqg=c-V)<0G{?4_0RKXpukhjZx$oP_Nk(@Equ^ZP00JHm7w z_Ef(}%)JRoBj=-f4-5~_cS3!9#V(V;1g+o2ijwmfb+)@GjKT!S!!1Oj;~&^ez{iPw z9^eu^Sr&Pk=Va5`rgL#}?4{ZA%%4+dPM6LZci6CW_nW?%`A|8<e{(pCJqu5J5cs0m za7%&UlD^a;OT4z?1;*qhU?Sgqu^=ZBON>$mR)vsjTU0-oBlu-7m=?1Utua7sO}Qph z*&};w53|$pJf?(dONr>8=f)5R@ZW3{t_2#deI<QSs;LPAfv6NxZ54r^iD+OpvB-(I z@M|$JR|EiaAzUdUuNhAr6hUu$NpXr0f@hRttL98@o7lB@%8ZX5$)OMk`O<AF?ob03 zd>bah5dN}wsm}p1R0_}A?RozZge3>63AoUx&n8yaL8Qqv5{%KE!2$c+q+<vP@jaod z`ZXenajK)ye4}+oD)$co+VAf6RazTo>Kh$M+&pfvslp$cD$QtLo>J_uXAQz4#2_=e zjs}H_0Yzl9QIRbyv)#jF9dfX2rzJxYx+la!;w(c2hlWF9DdSR)I=c38e2Ua7^-jn9 z&zc{Q_&toOLIxW|cr@DFAF8Qj_EW3B{Y8qA0)c204`@d%`<ywEK}djpNA+++R#BLs zne}e!-qFE9tv(rFyO5d`t2NO@OZqUJn=9hdBioUad_=&}m|-O<vu@}i6=zvjX$x0U zZrF=E!(I}MP%`rVrv1$L)<dsw6e_z(JpPAC4NgXhiHK-bF-Ky|Yl#yyG4?=h)Dnw* z7J09AjT#w*+c`e}ZY9lyl3_j;iyt<|n)e|NL&3OyTaTyfjDfaaU$*=9V!s~fX982t z&@`0>B*#^gL4g_%?dY+PW5Mhq7#%Sb5|brh6-4=7lkIKNZZ63U*=FGL&2JGtf}g=7 zKQo)^e97?Ihldm$c|msmel9eeZiy|#<wd4D!U<zfEjX4^VX#VMX>pm&8+@S}Zq$Jn zhAblTm|clZmK`h$%;`u-J(`UwGc|6k))7It4G_#*0<YEPzRM}qDj(G-8N@7BmPlWy zd8hgLjm*p8i;)=^dk(=dt(pTs#$&hMg=l;y5dMsy7zTG?4i!%~he5AjL9sxgA|4Y> zmsOC4Ypfl~ptBFyW1qEtXSbAC;jp&3=t_LqyLG_B?O!QhdjHdy3Z#$mGF{o0hTuve zf%Rq(X>O<BG4dCNi^Kp6BSpg)&1dZUv<`D;6k0P%UHLf5a~`I%(;>bF?akCZ+~LQ2 z+&1QFd3%%O8^N#tS;F*2Ibq+$96~U0E7#H+<rDaL-pgh))k`1{ox*uY5*@Qv2uE7g zCK8Q|wo}Aag7s(DtyE!gib*qQz6V82&wiE6y%-8NI6{dw`=b$?q3A45d`$M#;2wVp z%TXA1yytE|c8fu*c%w&-F_GKgR#&R={Ka3M>n9M1L6OhL;|eL#>Pn3=X$;sjpw^4b z%0P#8E}3y$CG!V{<G+q^Tr+7dwsN17h-Ayq9303epXhy>cd#p6X*^&pT*^qrwu{3Q zl7UZ*UimQPuG=Qi@Nz&{z?fu&6jCCs?meIht}UZil_fD;EX6+mi!(@BsR}I%oluu8 zI9%DtUf(fp(6Wjt*UT<0bWLv#XW!=C2Nji@{RgM!)5Pa36m7=kS!1K>bfjc)+mMp@ z>BWe+Kp<u0et|f<`{QkFVAu{f5=}6=#${G-f*<hNzi$UAOXaYcgj6OMSFcao^s<Y} zuoIIz&eo_VqzDiriRm_P909f+zB5u?ep_sMJYemQ+b2L&=O0eC9`F+c@Ldk=-WV~y zOxy*aDMxgfDF!)dY3iAxjt9XF6J|{Ogpj83d1s4JkrLrej~Gpd9h$+n<b&+>atKSW zR(T+5cC(X>#@-dnv9u09)N0*ruH>h1lc=hx8H=T;1<R7f)SqmY9?tz78u|9J7197B zWD}0g-%oTln<AI8B~MDPX(LtR6w+x4H*wqcc}=C&P%~qcujp*QXd>3x!F;l}h0l** zer-KS-*`~@)F9YSYpl|I(FgNQ-$?(w<eO|fFPHJ4K7HKFA+QltSo?hhiInp6d*dHR zq;QMoem&H_qamO}do_iT79&N1+QA^uA^(n8D#wn9Xk)QlQ&m*4iNv?Tl5@$!=98}u zjfZw4Ice^90AmD%k1OLLIq{mVyVt>%rByE@fN)?&JA5>cDZjhHkDHOF79`YTaM${p z7%a}+!`G&@V$^=Djk%!`?i~RZaW2JCw&eRw^K;>I&TM=SGfw4fRvN8K`|;_{TYhIP z&5?Qr?gHpl6zDltu=%nF7+ZegMK9^k$PJ7=-pL*nvdY329-ZwT5hfcvh&EiFpR&|N zR_dgq2J~Xu@KJAeIlV_6!=_D=GUGE$5)~`TX?U{gM!S#dtF?z)z$wF0%8vHLdRS)S zG3a*I<QfhST<Hg3v-4hFSZXj?sy%t=pDDgKCnvEn<C#~?K@s8!=Ax1TsEC%-<d)!p zA({Q3Ix^4+imfQP@X}4PPIA|AI&BfpsfcU`#sgEYb4hC-m(9rANi?GoE#jvq=oI)4 z_r@1L!m0d^Vcgz_AcF8(J0+18eyZY0Zul=}AS#qGZUEd&k09pbc)B6xXL$m`gt8|v z26;~)Xfj$DFSDVM(=e(nM3GSdro%gW_RZ(=V-leN><b+M4`H|wGSWXM&$+Mt00TPj z;!-Rk@JbJBC%>xv%b&K-AufYRkCj0(lz%cRZ!*#mQ%DIXk9(2pgnq}IFDB=L`A8KJ zY!)n>!R3o$He<@@04mxsBh6W7f=Zjw47u&AhNsJ=Cf>&keM-l@Q9DFdnaY5Z_6O?r z<MpK!wnvB|#ebug-m=me*x~LU2TocE^<yOdWIq`|7@X81;H5*L<5@S=VU2hLy@LbU zO0gHoHNDc0VtMpHh|DX@sylo^lvT3yAd3k>OP!7`3@Wh-Vbn1`&Sa!0XnzS%dHg~{ z7n|LPH}w6OvHO;BX!CdV%ZR0i<4AL9W^=t}?aen`+8h>_MoCKl@ZuWsW1e@=o2pjA zPfgE@&?&X7vwvI3*vJo6bHnJE_+R8X#7iK*H&WVotXr7=p5nZ{(ud(%blG`o&32EL z1o1NexTmeZlm?-~pw?uIuPjfCqY5lXfll+)(3-&)0!56_@<AA_YlFkG^>lXaO3Qpr zF#;BY^%6U3^UV_2bX^+$YewUOe~5;;__ET+(@r3l%>0mIfF31^Q~jkff&2$X;xCt) ze@Kpt+gp|#NZDphhbb8IBZ#hAFlOEaFUl)OkX%ws8<Xt@)lS-%j4WkE-%ps0hD@x* za<3(0*WFZ5iQZg=RlRGvX{G&PvxV#Ir(CMLGhSA{jcr!W31V}o=hF@E)r#~#`_=oM z_xmc3xC4v11TV|p8p?~1gzssLcYOG4_nHd-6}`>)2y0IyICf@aC@Rb*FSA7bg~vS- zC5#TwFUaJ!N)#+Fd6{Qho<CK2Pl}&b4-dYW63g`+$gHM}|F%}$B|a`Ld~PoY{n#tn zggDEF`b%$Bd;@0+SM_)bm6X`;fX@iE_ZefKl;fHtC#bO-u}Ap}#3~bo!&6TdPwcGe z9~-)WYQ)s9L}x5s+qa{xBgk6`RbhthCv_No=qM6X`=}u;o=sHvKu`x;lBE&uK_q|; z5=|dxu!{}ch_P$UD_8>iw*8B2?%Q<tV=HD#Lc7z1ic|VzQ?|r-9A+@X?W3zkr=uGv z%5VQH+Gj0ffonl@68jlPbgvMT1l-aaQMu_Ms;oaVCK$NCR!laPixqggGPrp!j}O8d zd6K{0tw1Ly9or2wnt<^&SNO6{`L#qFAdp0@Y(IcWk}aY9r7EV9an$`H@##?EIX`PN zfUE7<*}=9-MP)NA1b}HvU;MnO#jPItJFA(Yh<;5U-r|Ymtt30gdgX{{5t{VfdY?DF zB2}9LxQVWy{l_i}0SV-zQaEc#Hs01Q%Z9Pa3gpahiL?RDSY{0_ZOzCfH-T}=5mosd zc+9vFU3$z^#dW=$3T$sG-_ppc4UP}7*0?C*K)4gPoYj^d@felTxI-UQ^xSZbqE))) z?2ffo$6UPq#Z^+{zM~M_<T{IcLf_t8p?@Jsgf0X&+Qgfk$x{Hac3^*6{+UD}A1Q?B zJ(5Xo#UoqNR2Ip}GEovG$4R0sD3n}nb^XdFo(S1XN9+EvNFIi}YkuSB7wpu0eD;>n zP*k30Sr!p@R-f=#_1@1YPL`<EoGWf=w9LvS#Bv&hXi{<9T&1UbQThK~Jc0Z-$LIvk zQz_Y7avNp}Q|r^y^R%U<mNsZXt{I5K7k#Td1P9gs<uZN^PJ+ljSj8oEP{sftsqXh^ z3?Zz~<m5f@HA0oUOM@E|g4s;*hrRTiP%<h`u>o$U!FrKPZ-L_*4)IBIw{TJ1FS%Ds zr~Mx^T`yBvjE;P@Y8bQMNeyJlZ9}&{jmKZ@cX9qk?jyZ0pzf2jSf>HxOnztw)3j#x zTk<pWT`XenIs^$rajYfh38FtF_uUbKE0yHtA#k$^5%NSq(huVygG$YYp-;;%>S-_` z6B2SX#8;U^<kH|^H_YL#XhFm7dRL~J7R9Krxnr)A8MV?5r(JEjtXxK8d_!m+{l5J+ zdD8v!cb{4<kBjBuv~hmb_i9_4Hh_9LYupyb)SIZ}(%+F7donPUm5(vb-FM##S}BHy zN{GzS!E<}0(I`Uq{Y;(y$hc?JL!@l})ObQ#xq6bFKZw!ddH^=RuRG8+%q&CAWqFSR z>TBEB`y|$Y&@*OWYx9)0p~=Aa{U)hStY<*BR2$BQn!%Q@w#FK@@uR8gOcT6^jqAp| ziZRdGd~+?4rGHP)RNdW9&e4(U5A7un=A@$El+xZFEO6~rdLMUB{%C%xf4s%kl6rK! z(^I4`u^>%5o_HovSYJN~Izu_bQb^N_3;cq(pkH3S@(bDYV}YVP%_5@kL7-ZAlGuV* z-@bxT6kn)hfDQ6%ALl^$qodxnC1eMLT@ocNE$z?-)C>2CS_N&9jh2Z_Hsv0B_)ijP z<4+l$W@#2Ix9MWMD6tksJ(C$8LAS5pPTc|CUS-lrYC}9=?2;BVW*++WJ#5`*``x1L zw#sj1{x{(F@>1;JA_P4-xoxNeIa72hux!p{(O3$D_E%alNsHyTA%uaxM<pLsn$>fN z@$5uRPg7eKB*k@w{J|S!cyib*iED{&*udU^Nps!i7$e<J(6~N5eC@BbT&T0uIkX>= zybi{uzur?<jI8f9{Ov3=$kCT0e#R?Y4RZI8kE9dL(0{*t%Bq>;3F?9^QCz;<N^mk9 zn!hl)>fKSaZbZ1yQwoU;!-Vsz%{--90V$ef3OFI`nif!HUW!>*szP>aXPlCn-XMd0 zvIJ?gQgCUD@De*n9txmaz&VV|mkOoMc>#{VZ(Rm(AeU;%xTo$JccTgpr>t1276R^{ z{QAx$kct&3NqVm2<pgtI%_yzVLD7q+PJPMl`g4nAqXX^F!Ogd?Nj4v5HA7^Gr`Xa> zKprMXR|)Q`ju<c=*51P5T_2PC_FJmy&qCBg-)4q(?-i-qt~CtHsHSUpRvGg?mM)Ar zR4?WaHQe(uPq`1Y#UVGrHZOn2Z!PPd;`oPm5~h5|wsyPGTVE@a8Rk56vYRd89A$OX zN%rh5d+@ddm+dlQ+m%yDMkD9+6(aMrWEPH01kur92$c#H3%TQ5m`}t7MOm}o^_q`) zX4!f=<`8)I@_2-}(V>#p8zE1WW`rB$dkd{Fwifw%tHw3OpAyR(XAt@CdPrr>=&YRB z$_(KyW~iJ&QHv-}xP!-Prw^i%uWS+q-oD+H*#8$*{2Rm+Fs6J4dA$p7`wjz0>gJ6E zSEat^kiIvry7H-tS6Wk)UDsi6ir1H4F~4t<5RvOx{Futek$RGrJ?O?1vWmO3o&DaC zt`>@K)^=XwaWJAjiZ}_JsD3X%wHX?b*Q96L#;Q@OvC?J*kD^KYex(T(ygu!BY{gz4 z48z3(&(8cQ**Kfwi?*Y;cl^*Huos&`vuHT;gk=}T*yM-fJGUp_*<47w?Y^QO?JOii zW9a_ig!JLEdeWPg=b6i89nQ{6FnSDszXXC75?7}crPq&mhj@oMYpm5_pYG)0-Jh7@ z>}cL<E9oGF6b>NeMt=R1V5G(gWLouv5>1<q8%+r!-V;s;C{km+zh=%Nh7>mI{MPKL z&!)(9?rK9lMj94r3fVWJUYl!Vg+RB;0iRv(DvCE6#fFY^No_aAp|)irQ?l(Gy5xM} zg}A2-;64ROF_0r1u!w+^WcU$W^J@ZHdxunhpONDt^0r!z6x$@k0(9n)d{Ljws$`i$ zslH*Z9~nbO58RU3jMy4icnVQVVy?kFU{{D=pT;vg4F5sQwu!=gNo}5p!!6SUk>rVz zJ<`?R_T(cu8h-av{ss%mg@t9>s~N(Ph#SCdRWzX+xup;Jso}M_7T3bZV04Pas2h%5 zK4<j!>?fSvg#u3zL#Q2>i2%4(dbr;A1cA=$15(v%eOx&8mR@S;7UeS~k>tautzG{D z<bM%xh9cVbbHZ-eQR3<}`Cde?68)yg<RjpAv^&g3iVo$;^$kxtagBL7+c*+f>0HC0 zLvj==uDeN<ULHDYm;fy4SXXaXf5{KI1{mX8>%G%u;u2LQHDx}JQgS}#_5)~t>6+}K zKER@VS6W|1nX%9>m`4~YU_~%=b%qqoPw5$HY^Mq5b7QF2k_q|1Ay@?1RE*5d=s2Z+ zTluEG)HM@HcmwC*QaIOIy!>5>284ckoi~&@MuX8N$i|`~qppi1Fe44*LbMyNfGkW1 z>bqm<!Zg|?BxD~6I$#5(PJZWr9C;EX3{ea3L-<fK+@I{+`(RiwuHcMB^LC2}zG3b; z{aZOLHQ6oh*mVQ{mh%X73p>Fq-|FK+@X1q&Nj9@9cDM@{zuX2fq>U$U{(b1sMM1W_ z^P)3-bisgEu>dc$1M?>jT!{8JH)0om+NVmC?FpatSf|!%B(S9e*&u0onw*L52u^AR zY&y8mnbr6&pT89gNCY$X`S>wEn2rDNn{60iHY!Op^erEDNGaRS5|zRo-v3wyzNbu= zTw(sB(dx*3T|!^7|1Fyx@qPma{iyx`gWXy?Heo-q=Ras73dRi?TG*Sac0P1yuRb8< z5}(szYJQFtZn_ILx?a~(ERb(LUt^TkmydbFDDx36mTzFcAAkYDA8%!z<;mam6u^ZW zJLAGH`UbfKSB0&u=T6pEdkyZYxPIe3M;h*06Iu<rJhE^z<^}k~ybxPymR6t(#HTmA zq6J}?X;-9S97WwY`tG<6SYPixgc#K@FrQr&%pcW_MVE%5K_mo7MzPj1z@MWim6S6$ zk$gIvJ`&(Xja7}g@dqpUGf4X!2~JguT*n2-P?q|m1S8ui2eJ4N??1=)8i*%4=H<A1 zMQp<`8Ff4BP?^Y~;Ue~GbCBB3O1I`pso`Q7nDTA{p&-S(SlJpkI1W^`nW@g70c^vo zF7H6kM>igpZhE~uJEjJc2-POb-dLlS-{)a6^igm^+Iz5zskiS7qS`~zDX7?1mnP&j z0@9L&y^q|KLbPOJtF5oqz-0$^%_MGEQ(C=lg`_BdT+Kq=-?8Wnv!9`zX>a0>YI|`( zUi?TX50L(V^KoKC$6zTt$|j^pJcDl`NbYcictkbUENFH>-rO>igO77KG3#u!0I)T) z)*#n*J)H2`BbD8;h*8l1?SG#k^9~ox7>AEmdE8qNmA`n@^%n?<mcOCCEH4`jb1r6& zk6UH33JHxG>Cv6aK~(6Q{E+{NmmeWzq0{i)t%!@B`-q9mR(FC%OO4B3n$_TE^UWds ztEBmFPs>+3T?s~0Wnq^qjkPKDQy+&Eevx`_B!OjPj)0J+wL1I<r&II#1$H}bg=Btg z=U!{lT++;-nQ%nI1=gw|1cZ(SZQIrW%Bl)?`c;eI;9&kP)^GPnS~rQ;O;#KE3N#Eg zj;7KGEfAs2i$0Kvxz~Ci{z{@`#e`7_L8jIecyK_nmF0~qcsTLTDFN{Q{&#;QK8TUM zWdT4mS0+_k*5`1PJzN&+^gA&)&%9ymVcIivnl-2&Bh$q*w_*nkN(_9dY{-}lSX4q{ zgpDiT6sU2#;Zi0e6cXu7=bI9d7kOD?B%HfboNsJvb-?r|;a`K5^7Q0bl`Xpb3Xpkc zQI=s}K7yLabtqS#`YSh_Xc~U5<2nOcyE=+_t}Gf5VWXG4RUWzKq|Gzh-__qUAp}&o z>EmB~(_bwKF5gHF_h0F8BfrJi+3cKFZ=XQ9%SX8n2gCYE$V?@h5oW_2-?<f9{^Y;r zZZi}%y-Giqe-|XSX#)I%tZ}`24I9l--4mSx$$3l@kJ?))Qx53k8wd%&KsKeNft3mi zhoPck!P5pMO4Sb<Xf$YQa>EAX(x)Y_O_tl$VQrFbj2ejBqAC^(^w#g#yKIj?-H)Fw zxE$Gk?>V_^l}!FX%Nf%t!nfN6IEZ>8T?Fm^X80jN-#<)G-7m>FwP7tlW&a@8t<g@| zR_rmH!+>+`EIP+CNA8|Z)(`2AT(@Dns@T~q8jVEupIKz|ZCY<AJYB$n9%`iNT2#vm z5LvJ5?@F$ujNorA>S6CV@BZ$n26|9otjUu`3~sg2HkRoetNVj&Veoq2BfE=w3P2JI zv^6$pX@Ihh8@7QOIp<qUSm+)V5sZpnEK{=MOK`#Gc2{2zn6zr?914*^CNSESxn?z} zGVL0bHL6~FA}tz3E&IdHhPR79li7<CZBLzf(Syg%YqIhX6JVOPpHB9JZHT$*|49Z_ zIf12rN2=J2<FfaNHZ#c~beeILBGOtFV*irq)M!VoGZ#tnGuP`QU1|k0Wrt=K0oC$m z*CJu6upOep9hZ5s-P1rqp-j>>yoDPGn^}iAtem~ON(waY%2?wzF3Pa}*tN+~DU`04 z?Vp<|J$-*ohd}G}NBDwT77UFd`|R?;bMOwC&#`IHQdvI-511OoHom+C6%wd>zLKW} z%<%)RL=tOqbixJKnX1TG3*aojhJg9h#TnFRgWu0m;jZsrsv0M3`RrGx$UPe@9V@DG zY{$y;hsw}FD(9xbc5U7cg11|s%-=XkRLrY2VwGw5@zvd`5s}oquLT?`#TwH&*nWHS z;Cuu}Z2&}J4cevc6{+m<H-BNpcnwb)cHqQjQ={H~u_0ctz=FO{^*P$kJW3O?frqIi zh_2HQur4R`M9f&TnQ+UxT$TB{ao~?|=SU_jFmiyi*`uvBU^lJTx&!NJFccMECIT!2 zmB=;j3q_oG`}}m9t`ak{Oe115^0&PJO0;0QfxIfAH{u%AjD_4u1HgoZNr+B}t$Mf7 zdtW|{lgImVXo70vc*}(;T<>v?8O}Uh7wyK4{ooj~O1585mwyLA?XjV)HF!r<vmWuz z#%CR(BdF1!mXY2DH@Zn<?tb3NT#sBI(NL%+fTVM2gF7=^I2GhVv3pX`G=IHcpjRoA z=&Pluw;en;P?kznu8OU1MW>j8GcbLHT_+$nfKSBCPM^#+6^78u(#jKMWn$!M0`9R{ zLyrjR{nLCA#(6>XrYO#pdPX!hk7$27MK44b(yX#2;V$|DWih7b%v3%Ru7F@4t7jO_ z0c1pFje&J6DUoSa0+1vvgZy?kr|Y^5uvS9DKD4@gJRK=+y+L8VC2+hIwNgcHthwkk z+kVq#znSNH$XT6B{9Tjd_W}ClY|8IcpSm^U+{yR*)ykM{$k6@2FQ}pTmOM+Mk`5Gg z#7Dj4#RsXvSKRQ%pMNew=z(z;yg*7`vB1r`pSyrrfhNgj|L7S{hBYGV5zy8d!9%iW zJx7-wW2{UzJP2HEU2?xWtgbRDT{@Y7+wlMNN;O|7-H0<MHYd@tqbIb|`TFN3kZ*!` z4osyqmM$>C%MBxy`Ys#$iLe{^cet!Xri=DP2M@{y4cu-N1lO;U&n26ULI1Eqg#@Z+ z^*4_)su+sT#1P~`>nH2&&y&x(S@q{R+^ID4noWAG#6n*zIj_4`&D~6<HhOWZA6Bk- z*JF!e{XQ9$(TrY)!_D!|hzbx`{3dI7HUWoa!&r-L*=V4t%4G^CkPwqXyt_jKL*J}N z#QXTF0K<e>{NeT6N)jhQx^*XKMW6K_LHH_Z!OS&Tbg}{|{?{^wc5GGfHYaTD&bq?r zWOt}CpxVjfxnfM9%^%U^j06~rQ_{SI@Y2q^*=E-z^^Cg}G_H6;i66sSpAOE&w8k{w zCjs$D8Jr-eQ&CJIMM8Wpi7V*)**@eebL);uf)WtG^F#p`Kdxctr%gjdB(7ei=0AGa zu^%pG8f#Pp%Q`-1UyK-JVLeW_N)NfG?azN}_d&dX_Wo?FMU$&*EeT&Gjq}s@`%{;e zO#g?vJDed4F_7)`aqn8w68X~z%d2WuNkv?ryY6>w(w%vS%{a`(qIq>vHQJvP{47$} zR!^pH%xW^nFu|K9T8sPNzWe^)L@WGxUpFawoM-|p{U)6%8O8N7(Ql}U-V?0nC*&Mm zWnW_NcpCKte8pxej_jeiw3D;oO85fXhArbojkQ(Q+g7YjYsMU!uwqg?BK7m<{$*dx z)$sOMaW|j-CGWZ@d;H!BvoeMBWl>?_T8#Yf<Et6G=-1l&8b@r4KjL7bZEzi`r&~q2 z%OZGMq8QJXg333NJ||v{7c7Zw7#`SkY7X72AUtBlCdR~PrG?%)PaI`;#pW?lT!kcd zNVUNaDuG~YQ-iy4sOH%^O&bL`0|Fp=3E~fuFvw%(RMnR<_IoCczXttpQzj}T;A1cl zEUEl?>MmG>&|jE}l8c@EGINcpN#Qe$Yye->D-#Wk&1}@S<kd{G2_Nylj{A<5I-EU6 z8`x@P6F}<3*^5Qpj~^0cb88Nr=HsE_C}JEEzDv|J2xL0VS{NT(z)t;C{)Gl+abJhn z6L%iGp{!Z1TZ1TiJFul#15Y==)`7fz(APhwx{KI6;)ZwO<)T<q4M`Ug(Nxp`{g&9^ zQglK}-ZR97clV&+Dx-_n%ZPQKRvw>LUk@tT<*n1e7p%mtc5Yc1b4eZx^C%AQ`ohmP zS=3V*WrJtaY8TW{f8P7>9s>TFuxjh7m0kRepfxtXEr;%i`>pjM3K1;{1bxMt(i_gU zeGzl{#BV0aM&CO!ttfjCzR3hKhiR|-BF4(m8`Yd=qe$4qSm1lhR@q6_>Ku<>J+5?$ zka6lgt+lemfOidZeq;wA_&m8@XX!ilrvGLDRqSB=?MNqSVHTRVdfs)eN?pbHR62tZ zcIMs|k{~On8eB$4<6CAteN8IkJCa$90ct*5zBLx*)$n{8!4inqPT{Z(6kU@^d3ElF zVw)*o*tp@&y!xAL4)?<Pn<E`}PX8Tms{(t*T!;ea=o>2F;ZaZ2=b~I0n22Gok!vKb zX*m}{qCbBX$YHXbt}a9qGeY-oN&co7LqcpBpW5bJvObiFvUBO>sJt5vzxdho{YBmd zt@xqc^x}fcxmLS73mB!lbTU9|;}lV<H8k#@`B8!0=^4vL%w(2+H}deqtP4Z0u0yDt zOHzE!bWE`s>yh(Ob$|waEZjk<d2-oZgNaag!3A><&u9KJE3^j$6t5z3y!!1TwH?0* zdo<lY65T+ZC&u<lZNk&oY(?87+1=mlF$%{e=KlA=kYa)7^#n&Ltvfp4L_s9B*UQGd zKK{gnTHg5-@}4+ea>~=zF>`?6mU9CITnw1QtknK|i%w<AXmi{IQUt%lKy-{Tf1*@Q z8$qP6z#?#gxc;o6emY+(Fy%3flOQiuFV|)b{u{B!Rrs?_!lV10TrvSblyrrg-+>Rq zi8>uFBb^E|IxsRO+08f-4}|t3%%h-}*5|EQ&r@~XwBRH&pYTQA_<O$M#;$%7AI{9p zV0n<o1ktz!Ai99Zgu;iZP34poF#n`o3m>f?6x#i6IKWSby?TPxw&q@qwN=f|{J#$} zU;w@14GURBz9W~mbtA{*3h9o(1zZgEMjIdxOQ{Yb8*Q}@zsn<>89;CYF_P)AG;_uB z6-YVzuvf9mowBr%Z|Hsv9yPY7qEDhZ%tomxk+}Ps3y-YTY`@a<x)GF8F;1VoS^SXk z$^3z9bYn47C8|*7f#|S1>ErL)9NrP}-)%lV1|?`-Bbn2o&g%&+2Uaw9IF&a2*faok z8E!KUgzt~5I^X@E_Z&HU47V}l06}*l+!LEWySlK%@E<x%U70jdrI~9R!p5@|9Vb?w zwDITCIRq;>PgWr;9(OkmAXmHa<^m%Oi)^n8;E!Qi09Xwii@fu^o;E)b{fDM_{y>~2 zAzn-r_@w6xcbmM>Hb;=;SYr{y;>UVNM<!`iywQ348#4rAnFuE7ongZ%M&XP}j8{H5 zk9kQ-J=1J_ZaYMU!AAHpa_b#s<=JH^_%6ZXM@=5&_5<s>+I0!YTnu}24k+K<4*dmo zozoj026*|65B}j*5n-=9kM5a_F1mGHDY~?RXk_g_$p#ptcy(UfVO5|f2){mROQ?_d zp^T}YA55mFapROy4y!&ti-CVrwcjNvjGHNpmt5tm|G?;}BMqvv!(GENd#ki;z8!s? zRX^sXM{&sNE-zyt6H2>v5R`!HN@;-c2XW?k2HIy{>rUt%r*A_-mHn#vGSE>^@|vwA z+O+#PWr6mq6#nhz_~X+~ode=ju}D`iLVRDm+DN<W!_Sy_^@kX7iP3}G-lX7~^>;bk zeQ3UE>o?SFJmNKKZP!|VoMkP0`|wfK?Ww_)o}(#K>ZB;*pDdjPY>MwOB{PpP?qkB8 zqpYWRvJKg!HN?i7nXp)n*P)N(_{kGJ(~vm$D9n$^&zrxK_d~L(vrYH<W`_l`_eoHK z_m$4K@bays$xQV4uzIT!McP_?G~eWgC4J#WK}L?lOo74RsS#n&rIktfrAcY=x3jpC zkBh5)&VMTY$mBElViUYpclCID^ovi}W8v-dE0KvRFRFsL%3d`#jqx{e5bXl72rwH= z28jsChWm*O&D627g1t5~G2*G*(D1Ww%Z1;lnIX);$CO6k;XSz2>ggZYzbgbFy``Zk ztn8!Am00Ug`C`2__;W8WDereMWjUK_1XJpf!)S)T{n7>0EUzfMs;L-n!A=n((-nSh zzA}Um=s90L<oD1T)`5q`Eap9O&k=ze${clOVFO;q+CjPW@Y4<aw&_b#CY5}5U2P8r z6%@1L*#nknq0=Japb))^u^+a=+UN9c!T&jU@$YSEZ$c^(!3x-b$B9DkE$Auvck5f| zF*ZFBG*KuY<NOj#{6s+N*tBokI$=Us#Nj9dRi}+%hTz+Dw(KcCrv(7{s)&81Ik?+z zlM30erou&JDxp?$#;f(Uo%dUCwU{`cuePOgj+a<m%@z;CaID1NZ|u9T@m<eAp8w8+ zT8J+*`3<%1*Fx1iS!}#nG9l5@kGD93KD>t7jBO)wy0HU4)WSx%N32#)0|&&nPz=W{ z4W_rBEEY6*>e|6QJO*>+z24e1fU(_0XFQ`h)_f)(h|lC>UgLzt@}cgt^uwBPMAF-D zD*_5ktpo_|g*RPTm#)<#+K2xet&ok7J`*qVF4N*^=kxRjzt*-s*90AVp*dO_iE{?h z0=%qQgl-@ZBqAIV*yNT>KiM0Q0jTaAR3DE)v5UpT&c^8)hcc+rP#6sf19j?U+gfR* zKQ2$(f3@uI30Az^naxJ#mp2TZ0i64gypwR<!FNXP-_OANHH48%h|dw<k2>Dj@tXIr zSpBR(8&&QUndk1>lqMf3;~!2noI5}s;WbcKG>?ckInzoQpd0{9wH-^gFqXE`lnz^R z?wT@Qx{X`oP*Zp?d6MKWsV3bJ|Ic>;XhcQ$e0_k7Va6SdQJ|4zQz1D3Un<RJd$Zp8 zkXclJ7jiroTld*+Id*Y~dC}UeY32@nUxrZXaUxYm$8iI`Abtz^>z~)tQ=^`-tWnBE zsOMF>n%mpPMVP4j&s~y?sL44;VMg4tK}Fa&#h1pSf|7w3Y9#I``K~gWo0^9q@K$ll z5O3p*&<%~-+BNbBJIl(8<WL}d(;l`C0Ho5xH0iC!npAoXKW+U6d=KyYIx7E7mbi&P zcmz4PY!+8zJ!r4bsC~nyGGe8C_6QRjhwzm|G?x!B4%UYP6L}~%bQ4bU$YJL7$y=j0 zS|yp@j`JSl7f=&t3iXbUxjossjZhaBN=7HF%6K8^{DD_V_r@)ZyUhJ(4ii>crydy? zgf|{OKJ)<2Cycid7ZOTv?ekY%egZ>;5lPxf$%Bdf5f61|GxPde_x#yI`sAfb8J@^* zBMv;}yR(?~qFO&CE^xs(04AP~k0QiwrGniM%R76H5OPvVtNd|Y@i{j63@w<4^-Uu} zl6iP=l8HG}u1(H`G$d7`rQg-)Qj^;f$YP)=4AAoDm1VS(vQ_BmPAnz$5kT0S@H&QB z18WO#VA0W2pRW*$HhMf5uJ5ZfpRDHW%HS(s;4`geqy~Uki1X&vW~ACsu^Ugn-O;D^ z^uPLt6%@lN>3NrSzB|>QX6ty4yJ1lA9fW+x__8XgA+|8QaxE0j3|L4DX+UsbeA}6a zTL3SwlA#qgi_F@3j4Vr)Ik|z{agLAnn&xsLsIQvlP+@zGkzO=4Lei#;`)xnv#JD`o z+fA%l><zpm6Yb~2^@!IX-&C?~^#bB7Bl#UE`X|9$94ClRuy5fX@ZR3vW-KRbuMX+9 z$O%6@(@W=)Awi4nl#^*o!HqY|dkWwag9ZHxG?R;Ax4cS5eVZ*)vr+m&)PI_<Z5)Vs zX#a2<7K_gq@e3!nw`P2`b990_eCL9rl|I=Kto<>;xjzLt+>$pkk!%N=4E%x9I#r6K z6l%!X?{72X=ikTAj1+^?)y<#3^rTZ{%4pi;CIzQzjL6aRzGP{LYvrDpz;R>;+oI)( zJ$^hcOkYo*xJ>IQ41W;13;sa5IYi)&OrFzi0up7kb-Lve&NJ?~9izqf{dJ9GJzGoZ z`|Fa0AS*gJ^u3j;aEtk`EA0$18fufx=Dqtfl{Cuslc(?m9Gy+`LZ}g7>gur^D}k+N z)I2Q&l_n`95>Y2bJ*~!z1V&&LY#-_-VGA%&k`=Z5#`O!jSyl$5mQ~N-0iOIwv0x+s z)M2`T4Et6c+XKXE+&zp5XoY@!+fM=Rv~FMVPD1sEAJ!kS-b3k)c_jpg1!plHE=uD# zKPJ-6;2p((MGagnm!u*s%WmyH<~upO3H_F!JD(}FQJ`S#CwR?zZ{LS`1kYvEq~f9X znDvzylSMV=KLnodJy`Jx)E)W4N`zgxpSDMFd_gmgRnh>o8Jyn$*E_lk#h(t>`y$n- z?r>qME2D{n;o15kJucoOLuKM0)Z-UR%}jS_Vfo9`ZG_*cXh>gH>QPWNq7}Du3zukB z94|Zc$2UzGACI_;9k?G$eJJH^Y}K6P6jcZCJPm6vT-5-UnD#zki>aeNLuYES09|W& z3r%5IN#fzO)jJ{=@~KTMOgfl|%yi%}emZep!jO53EAuAn@h7(YrqOnB64wDw;>S=1 z%$bqkS~S?B;zRe_CT6qAcW`Wk)eOS>c8t(hn$skQHLB0q!}@8F2@XLf`s{F@b)E@* zg(L9=*y?(JT;Hi|CaIE|`WUiSyK;9p2YdJ}UWlhkg(;D<22u4|)1YAdh&eu_HH89C zQ9`_}5d?gHQ6m&KrB#Stf9u7G5kJa(n-h-Q$h9Z@O$F%|bO-DSA$ZToh45LNKp}lI zE$%mE<gML({e8!@?b~*QYLP_*oTqlWl-G(}sh8u4pqm4=o*fE63!uAn!3i$0l=DA+ zjhOeDsnO|p+*Gm0Y1#EzI;|iM&nR1FoG!Z>_8Mp}$yNI5Eg~cqY=)sO;?B_di@<mn zl*mzi9AQ7CoZOt19!}Flf9+OiIsLlpJHKpx>Dg9Bc`~a8LKi~J)5EycWU5DJ_2Oc; zN8wXjZJM0(j<l(xCbpMnu&lA=ceNPCE=>hgZ%0utz4bB7{<)ugy2kVDd2>cr5li~8 z1)vhjQF@8E&u`vHSU$%!zeo$XYM4sw<O>6U5RpZbrK<P>MH+rb8(SxYoP<jSa>IpR z`&uX18pTlAO2TKhuyyXlLCvf(_wpgc19OHo2ZWVCNOOI?eJOH$=8ezV-_tYGygxls z`&(#a_HZG_w)lH6X!+E~C?u|Y=K6$IUEcO^hq>E&meR0!p`Jv#oi{9??=%srFf49S z!oUgL;wjXOSzV4s1%=&EszF6hBswePnZOfVza|e5+BiWLwvf$)k#OVzcnUfQF($oD z^%ux&?RF$$nf^w-6_FRNE{06(?uQNK6g65AOY1fd-<;ulFk0Nmu9Kg7E#4;Xda>u4 zS^HU6(1ztdr$3jo-#o*x+$QwiZ_U!T@0xZ_D&vLj?n(C6gsGXR$VW#4s#mkmvOoUN z)4D;s;mV8(u%{#lz0w~hTiZohgXcdJFgG9kVzFu`MxmTU=T-(~q0Uw`_w>M7+RSA> zzj(KAH&7_odfBst^i4Z3%0jRX{LyxCaVu+T0$Pk+RQ)-l0|j&J${W+0FzW?BDB+PF z7aNxq=WJ-03u`X^;Y&Pn;1_2x!WK_b`4T^Y8=NuvBf?!ocZ1uk9oDc8vm@b6<C*TV zm;uRm7Ml$ibE_h)qMT>(neIrtwek_-8#aW&Trjb?_NAG8Aud)E-9~|Zp9UV$-ZH<7 z5L<wb#ZB5MP?xXg$GA@u()7bq*`8A~WCnX6xWF!$j~7;;IgDPQR(eYnDz7c*4s`)L z;a@OFL&e@*2+7y#4KOpqB6@bs@deNsrzz4x3&^oshMc?RN{fnPKjTL?>)$_n-yU*# zRn_{4cu9WwcuA`GQnf^CCD8t4E^TMet5#)g9~@_<5{_kJ|No!t5Ak}%N*{}bCJ;wa zLD+^)7K~EqZe=I4nc86Q;soO+#=)|hy_axZqc0+Qn-OJ^%*RYpT$k`p;@#0gZWZ<z zMdN$h9IA(z^*F3JbH|(=YaR91RaHgzsm%F{JV-WdJoX10x(Abk?jRir!{{>^1}K@L z;$zybG{?C!;-uHjK?2+)3@+C_Tc19eA$<z`e^h;QSmtfl^_6YACQr6)Yr<q>vTL$2 z*>=t3ns8-%vfY(!`)ls+eeUOdkE8!P&VCzfueJ6LFTqxHFd_*H3riozwBO}$nlea8 zp5ijQ9cts`emmi!Gskf+9Yradx1k0zfph5?8RLeYaG1zbs`okIZn&<L4qQUSP|DX= zGToC6n_l0;L50wUZ(Sxk>>-iOZuS?8Yr2CuRpMFD#^vxAUZV5G;06R=17u=mRO6y| zyM-J9WZ&aT;5-M5MDM@l<uqKlfap~2@<jHNCsw~OHu;f}PRdQ%n6bF19WCR$T$qL> zd%>p)RJ?+vMg>33gT!ITpbyplhYefDl2fSObknkHCBO<X9(TBr;DgjWdz-IA8SAQF zhw(jo;o>pB{~>si&)k7?SB_Mk{`gmsxnMy9-BU5?toZ*_sMKm7(W&2gr2u9&+8n09 z^-RE|g93!!FJsDB$R&08eP15VNM`KZX3e(GcpluO1hj}!|LSPox{z*bt_x(YA{QR0 zrcd-b4ZjUp67Hi2*$suE&plI<Tu+_q`-WnVqrU0=4;?_0^#ORn7k#XkPxkY+E_3u$ zHiK-@DHUON=If&urHEKm-_xQjGcf5Y$4n)(KM2h=Y1hxaf4ZWEh9y=7=qmlCAq2jj z%vb4cxR-Za9ki48km}9&9aO13Aubn>oDsa9JegZ@`kc3QzD$Vg`~EdiO(MfySoEe; z>a-_YYY;d(%`J0a(uYCiyZel#Fv!tKMLkUI3@5hM2kD){wxYUEh`=QfZH^w{{$)dv z0wTz%?i3eh1w+Hc&VGB(m6_@Vpj6WJGbll$5T<p%W-L4vQ%a4cvxNcWmH!2%G^`(U zKLWpzhfBYDe2itE)GDSOjo6fVv0$s~?|ck!Jt0kLmp5}?Oo%0Afg{y!KE=`c3^ILM zaY=;CG%Z^lmINQk-{|JJkgcOH3kEq-)h;7eIMubHC3^9`h!zbBLTJzt1U;o{w;Uqg zpFd5;z$$00)_d07cVC$a+P$3Hu4%79l$4gIF!-S;(FimUoqnr5lhqB~iQ)fJti@y5 z`PVCwrvQ6_Vwu@<i$eBbN2gh>2uh)cstgM2^%R3f9HmGcW*CCxHh+70M1Y4clKD4K zN%+7bl%R3~|CEfqaRmztDt9Ftc@)@j7EXl+qP{dW1m?H5WvJ$%p|f<!0l(6y^0u<n z#MU0QjEZ=mpNpEXb!bG@*JV*#)U~+ka-mbP+^knv+*MVsUl~cnu0Nl%+D(_6`|y!3 z3$=FaLp`rG+`Lt1kp11$sh0zv<KQ4B27n5`mwx%i%&KrE(zxR{5JU6_#`;%%7NGgE zoBdG&FX2&q7U!>~4h~D``>;t_o>G|>TACPtn!Bjl-r!H;aizsN>?LVVDqA9(FswA8 z*EG2r^DECVgW^Bp|3xiE9O+jOt^Y5a0{z}YMN)c(%ks3TgL*N(0Tnvj-?=)LyT34~ zj$cm*O*v9#OWKlex94-0RF@<add~t_R!Y`j7zaAeZ*$&&nd^q`ofs2mG1QOdPVNcM zHg`~bXJP9T=Wj<H=$x=}5|GO*0hM!e>j}Gm(5)A$O;WC@1D9JC=%l_7FUGhxx1VXR z?k=xt!X=#EMp<=!gk)qWV7=~%gkIa5V11hy=(fh2yYv=<shfku;v}rBuZ>OB-bd?< zL!YK1O*r;%Kw1iY5HV+6&5JxY@8Bv<rEp*@)%5AV@^P*D1&;h*aI6B&Nk<ZbrxqWL zWnX)(j_PnKmA@v+l7u7v;wt5Ki0Xf;Q6ONjv(Dh;)&#}isvR)sT-+rokW^_WNT<XQ zx%bJiowzP93PFSXT6#&|pFyRooxj=snzJW!;4We@j=5|uE|R9jKPTetgrr+I!riB? zTnhqweSWm+X5N!O*AShRyZzI>BO|-gR0^xQ&&${r>{mgx9}YTwSLOFNBhRlq+HYU4 zu{8yB|If<|{(0FWx9!LFcbiO03|Zd3NZieHS_&Gt)us>ORQ~c{4^Sok!Xj9h4Kuns z_Nv(RqxZu>Jg2w}Fd7E#gvVa|3?K4Td9Z!4Y-EX+dQ5cJ>rC@CLFWFh`&A-!%sfSp zf-m<u8MfW|>}~liF<RyCNvYEuVFg=;bEfo)s><$m!(KkgSVB+p){Vovx**5EqBD{u z-|^Tx@sBi<hWiqO>q_ZMtv6mFZ7PNBg6VC*`vG0r7_s!cg*FXt*0;)95&n0Io&FWL zi1;Z5TJM|j>{B?I)o5?pYvDG6q~XXWzraT+GS+i4N|D8h7GJad&OdbM`js>=nBmQo zh<uL-aV{}#%-)jbR<DQ|S7Nh?ue*X<n0Bnm$Y(hQqh5CzQ}y0Ko%I}<K8U9~?Fu3Q zN%xRC_X^x+W$e^^NCd*q)N#Y69`-W)G<n#-Af$IKo^P{wr+(Ed{lo)$zbpSVtn`;o zDSo6QCSAbDW?=_<X!nF;=a9o~m^Ff6RQA)|ZHxQq9wqu4%083sgC7*kwD^CB-~czN zyrQ*JmH5-iX5hr4%UWsc2CTq&>*8%>g6Y=j7pgQvWQZ+d36Y~<)X(H(Q*a@7!nCH# z#zax!xQF4AH0*nm<%&?Rt*NP?^;n~6M6p-)keMa%hLm3-UKHZ0j_f9|G4#V3GgVFN z1pIEX5G}W7K%3)GlrSx!@d-)LPDJS-jxSO4MqydFunOA2T;b1!b{!z+ip8_f4<S0x zbJQs*Y#U<Pnt7WGZ*-(Bm933!Be`clv!H~`>W$EWs*2<D!BpXfUX5XQ1wWVT@kd-c zZ6CKTr9ckT!~b&}E6)t_BCi31OmaPt+6u<I!0pv*Ev5f(;b-hZofitz=3}Rq{bo;Y zy-j1=ziz!{zK^-0(jin<;QgtdL_gGvDJ3<PVqQ4T1w3y|HA-A6u_pYs$<=<`2HS*! z%E>>2R2-SkRJ#FfV&alD-4Jt2C`f4{48MF1*_Iw7F+CBJxHL{2#terQAtOpYO2dA= z{=5LF?`_pb40Gbwwcl}hqy~KR2>&|ExSPVI)Aw{JG?D8xG0ZilizP8D2^kp(=$D3d zBDiFjI!ayNdPXCudSUfWOnNlZinPzJ@}EX%fH`vex-UYNSfwzRJUdt1meHaeix-7S zzfz{0q}%*m-8i*dEMb`WG1f;aI)l5stgME>y<twVF}JC(mX6==Q6kV(5bYnkVjdnO zsU&ge#t*-0xw%ag({gxha<BW7-P*0#UB7hTuOsw(-)6j4Qk;K7=o8rotjAKE4Vf{o zZaS-9JNJLP=is+${b3N67vE*UG#2wkGHs1=VJ|uv4fXg|td1^izfZcHXPI7ZU_~{P zc%@G6P-FDqTOA(W+E4_ugs;lRQo6L&c9YLVJKp`;l=TiV3}pKgO8T_@L7MwKqo=)l zp<V`2%`qFXY;}hG>D*!t7ShM%3kflRL61FyR@((A_FLE`44B_&8jC<e`Y34f-W?9Y zkETORibC#Xt(7X#%cb%_JEG$;9YKV(uAzjt+3Z-Ox}}8b876Xmd4ZjAL%-SF)oG|p zq;~<PTOs^8WIl4Od#i6vC_(7O+`VRdcJm|K!i6(2aM_GU`?}<W;uMQzQ>t~-cSwfd zX$4x`@8gIfa{J$UTcZelk=JoDxr#&jrV0Q3likS~w%X*}wU(cHP00RvvA(dQ^*`Z) zo@;X~fIJ8mv!i8KeK0XI+OVq}izdh<ViBauM6X&zmbC~>9Ysw6eWD6PkVQkAqql?V zxBO_#S9jB;%X%9gPXYThc_)4&kEJiKEXJm$eZ_kQZ^uxHB~7q@uUz(FfM7}<ulMb< zS{*i5J=N8f+JKhbMAta#3fY~On^xC2O!)iBRISvB5n>J~3v_aFa>Tu6yRJ?3%LKK5 zCK1o$)wQ+pNt%S|LKOL@l4NFn#u9NHkz<%6tggjcYiE&u4ax2%@%^6(BSS9l_ircm zz9A{LLze}!at*4-uv$@BV3l4w*sEJ$!r=riv*~2%%ul5c4>>&ilF0w+j2s_OO$Tl_ z#4lZdB(Set`NN+in4qp`Q^fgoBPL0<;xJg8%^XwwEZlmLN0CN8&!|{#qP*(QNzl0V zcZ+RFQHtjh^FbSaoG_gQ-`5B~P3ZJ5u3se<+9l=rAyGy}B2wI0K4IjPVY~@u@v0lp zK_}>o=P`q$(pi&z5rm1qO1;Km;N|?P--ywXs-#4fh$50Tib(L=ncK3<zor5!WZ0E} z^I|=vX<O`p;0>&xM&Nw1?nXEh+2^~f!$%(?feN@JA3%XuI~=?`E@3(|$&xQtg<|-j zQ|STme82A#yMC5^6YKZ(xL^K%dW?jK@Fz98oRXs4rpQfMj5k)hY(vX_DQz3ag1lk# z;Iy>cUg0X-Ax}3({X&_hSknm?-qCfOiP7jiGb{;UOtxO`1uk3+Cbs|++fK3H`>tn; zBKO`k#A^b2BJJT_e=zf(ElR}ux96*{aW2k(;2V5Ev-7^4c8%rKM!blpu>>p2$Tfzh z`|)&x`9WNE&`0a&#gK4+8DwKheRg<lq38&sj>h9SO43-mjKK<)e}G^C=FHY2t4RHG zcg9u<a8G-tzHRG+>34(QX#4@pZHdlxX0KHBHO@B60uYG}>wf(>_Y=JrwBr#Q`;R(f zOkT5{>+$0>e8s_!>1k!9`T_1e;<xH&IOX?Gweh@pkM(k!%)savEC6{is)>hKDi%$h zhPkk0ddf|;B8(cSKKSx6mW5}cq9=xqN^l_@aUee4X2-Ky2&klYn}UK7u@f#3`>;BC z!hS|gEx~8F_q?t|^02qb$3ybh9;F@Iby8M#k3(P9Qcz)?I`!b&TBjLHyv`T|ZSg@$ zxp~AZWO8liYc;tsA~LOR8p8kQb|qRsuN0wvtspE(*Lmt?lf7dg5T3T^V#L220>=De z-Ptw5z;Sj30jG+dB4+Pzz4+$`6X$Pl@$QJDe58AEbx&#@&FCTC;uDyRj8xor?kQZ; zHkyV@a+4#AbF-`D#-$AwQPxNuZ|khIwA6I8Zsw8vIm#o6!L&^NaOb<1lerGMsPREw ztE-R$-rDFm*7+Jt7SvA6GwAq^JAUxU0gYtL7UNmFO<B)$9jV;FiVAu*)5$z}C`O}| z_@49q=u9r^S{|F5gQ;XbbUEk$>45%_iUPN@J22~8o!hUQ-Q^2aG8bgFAxQ}d=;84} zti$5Qi3ZwEH?2R`^}hUX+(#q$H*X)ae739Yh4!dlu~6UEx$?<Fm^ESRQ{`7eqJM?& zaf*HlyGrVV%wgR2=IN|rOau44(zDpcjEu|LYef7OEEALNcT@1|uupkHr}-j#ICG&| zFYr?EImaUhjoMcJX=%&Y!`I`i9Y;Q6=j3lyt49j>Aaw0e5+lgt+S}Y<8I8(mm^Si5 zIBnNW_ny-)+KAwbhW}?&0)SVyzg&@Az*J9E=yi?{lC;Ac2YS(XiN)c4+^(H}^c}8k zTDUq11h-1~k0(`F1wD>*3Hm3}l0s)0jRLMnf+*?xiYgPN9XW+$q2$94NQoF{eA~tl zMRxO5Bk>f|8)Q^RJw)JG`rzXkt1SU)&=*FPpvm$y-+NaCp33){HLCknt4U)5@H$*_ zr`>skS8=Kf-&n4^lfQ?D>d6A25U@upq#qJ5zMQ!RJ)wYL8Ur=uif8l{7#NEYf-n$} z3nKn~O<`DU8@xT)#Me&8LLf1U5?#@S;Bb?Lsj`MPHpoqiddz_RJw$ZNGUngcT>kT# zeLK+b_FKICQ+agSppAHi5u^ZokR)srw!})zSD%SCbAt*Gs~4lfrGZ}?n=M~zNCphX zA~Pp(4=o|(8|%{Jbi*KiO=Z~dO*y}onZZ=+bv)>Wr<f%QKD%BE6SJ*+OPOcVCM<dV zsprw9$zAn#-fx8b0I(Y<6Jpphzx_IW_gY&fv49#253n@^xuM#Yh+`Q4X!HlGALzpu z!=9?M-Rh6I{b>sdLSXlO?t_?);aTE|m&K?-)5X;P{X8FY;m7AWub(L~ljT3(KvW=` zgbn>TGeTvF*=s8y$Ohe-j5J2f6F(IZDjAU%F3n_DsGSMdnsy_FL$FOnkPvetX)|M- zYkLMMX9~1sy1ZHITp4m(5PYRZ!L40caoNT{yK>^<$awwn%qY+XOfUaHJ^`GR{$8@{ zvGxOet!{@$?x)kmpoZ06=%bz^^8bFIj}6#GH1hEtf}wJthGy7D8Zm@JZ0I1bmijFg z7kmofN;e3@<*CdQ{_mKpeDg6mHBQl<OxaFup!*s#h6rTtTK0=GT8Ot-DC5GV#G&bJ z63&FgT4(U^2*yL?<Pb#lwiAtE{H7Im!g{(w%Dr{8C0NtKTZqyjFaX-=0^ebrH`4m= z)^mB7yh4E1i`!j!NRvX@%TFdxe~nM|0s-rA@8}i3+w9NRMe@t%=I5>~RH$x(f)Jwf zJ^z28Whed}|6(SB*KIi(8a7Af%6+ca8v&TF&v(}?uBVZ}{0KT6dR-4WFf#Uk7sT^| zP(MD5B=uDmU&(?n%?n$v+KWl?&Ox6jytf8sy=ajwU#4qcCI?&hm=R#Su3sy9uc?8P z4UukE+Y58TQ{wn4+T;wMQSr=epdFahOHI#k{(JDE9mT7=GV{2%CW*hds}+=sqm+)y zQ24RzDNKN#rmh~6>GB9nNlm66LGbQE@`El?rS%TKakK5!Q@8vFd<7s&8lL=8fq_Xk z)YkjFTdCE+iUv|JBr7(^?a+@sTN7AiyIS}FM6bW&a5MfrrZ(va`=OopI;S6KI6{2s z8j}=b4<v=^KM+jdoMT3Z%cha3oxpAl$rj`EzicS@)GzU5gt|g{5n2Lq?g<}a&fhBO zL(j)_v;u1EMLlq~YpF8tX-myX>ke3p<u<EHl`o3IxCiSW$gBA_uLPYwdVUiA*D|u4 zA^cTli@0j4Q0G1{wWrq>{lRUc5(LPg`G*vlEQLMc{5DIRPU=CEt(eMa(99a)CNjA3 zQ<JIMeIm!Z*K^QIU44U*CY&nbKh4L5=A%HbT)60a-s>D+9KK_>J+g)rko>^HsDzb9 z2obE~D6E&_T%L-^l+t8lx{@ho#mFb9;#xLJLi)-iDj8Xj4uOZEr!mmjYAPXllvzjo z_&l!6trlVG^bp*uls?>WyuHZd=EuTwgZw@i0+Tb~^LKf~QU}}_n;z7zx9@VQ8c*v& zX$}Ofcwc-Fd<dO*X8Av?bN(q$9|Pc`>w=zC32V1WlYlbtFJzoacM*BssFOmewm%p2 z-gvC`p;F*QIVZm?m;P_0)%`620|sEj;{6SAH>w)5+s{m1hwKf)kpY;{K-wA}u*vn# z_?)TTdAF8yDY|lmxXZxxbW-oN#pvlF5lx!Ul5fn-oc)qz7*ri-#Ob!_c#q55@7^&V zMjMos!uo<^=u9*#4YoHMvvY>SkOh9`+u*93zlyh|HF$)Vtek8jC8dXr1{mcN^I^4i z!?%6#66^oJyd?Y~I_@7#L9lW(#BRHh@UgI=17BzkhFU-7@y(8SvUM-Tds$h<Tuxi< zZoe;dmN5TSOY+wKR_cMDdy#$}k6`P;AB#I#la{IVvPBp$rASVz;lUb|?6z#<IM{Zd zgk~&Koh^HX_#M6A<<F%)7uL?i&O}|v3=*n16XeG$(mQzHKT5ex6diH)wx0_QX5VJ4 zQMZP|nQNHPydbpayOwJjSWzFV|0N*d9`ZdezI?T^68Qr;RqPfaUvP2UfuQ{#MKm%* z*v!mqqB4Ed4}yuXmM}fun?{?g*gBPQz3c7OGy48``08q6G*_szMmU7N>c3ju2RUP8 z#d-Qxv%G+ul9HmIT_%?eb&LFzul}|=|0{HfU4~k9`B@&HBrPjusF1a8bE5V%#{t)L z<YF|k*v#y$#%9PeA#)I*Xt)!HMrq~3Vs+CNz$ECfkD;dcmRtA~jVsjbmHS3wqu==* z=h&L)AtO8nYNqoQWCrto`Cy$}UPhw+ahEo)0gl4Q&l7*6;URs?b8e0S`P|q1AwvG+ zB*I`U$p)~b#Jq?ElKe8!jsiETFLo}#(!D=+hxnm+9!Y4VWedDVbrD$54s{p3Xa@zl zec03rA!$-K`c#|uNlk^rYw84x0lFHFfy)1SUO-JxfHnF$gqkH9u?=$EfV(e?Y2rNQ zdZ>ES?$_?G2lERcNY}RKd|+&~rxADGHUNhTCy6?+Ol^YyI;d3bC$`%)AEX*uz_=*D z*t9uSaU&KO;#H~JK2Tj%N_#L#cL8V=d4(~3xSnB>@pU>|ZGP$aV&xBDj3r4Ztt5ib zH@V=mMD$%p8ss^M*^m&pxUPa(w{{<_ey11!f-JVE>@KfSh0b3n;ofq3&S_q9%8usB z7rg4s$3D|%Hw5?`kD5nbgM_HkF9(8smGOf0Dc6N=V#Z}qvM}C0_PQJzGn{*!(YNm7 zNve9>M0%7!0^-VZz4!aeFqm%U8SlC6$AGwkmJKzx`|sOLe8}D9?6i?Ybm8&g34JuR z_X&Q+1(MMbl%aj-(GmECR4mxS(eR5HULqewKm=6;1mCW^`LwovHu}<eJ~r-{MF?aA zwgSCQg53FbZ@JFfH=ft4#jDN^NFMXHwjlCNjkzsbH*w{9r|chb*>I(tc%KGO_`XhG zRN;9oL&s2`#D5I1(65FSsV-*iI|&XkzlldB962!ZPuhEt`RJQV8ymmud;gT{^%qMj z7^`E+4X9#c<-RBDG`;ys@B|^RDv8ljk9TdP;LnJ2on2ODy;&yqwb4Uc0NB;!xUH&< zkB<z4&rbZB5_HK4FtGqU*#e%zb=mm6Z;XA}{mgoQJUXVPVPIKh7kJI*IuG*@FtqR^ zOW&>TwIe%juc~jn97ds4n|yu-+Z1*o96<MVh!{rcZ1Tsr_pobGoOp;qLF@94^f<U& zySzxcyy(zdpz?58fQKZs*z3mC@Fleu7e>?Kw}>>qgXF#(ilR`jdN{f0RT8LPPE2CG zm?U|1>Fr5(M?y;{kD{^2qxeMGD{XoINFuyL!Wz=_11Q`E)iVR0QDBqGuIE%bUSda- z#7@Lb3Ab~@$K|*szEGl&i7W{f+Z_zeM6@wKOW!y1)p-+1?A|(02uX~^MQ@(v(oqo0 z`E$~n@pV@-)-&%{O-!uD_KUE@!YUbC)w(a*%341T?E_3{5H8I(K@cdqoF7Y<7LjvL zdMMoGsLT4qD(uF=b`P=L1E*T`oX?e*r4f{|uLITClIa(4g&T-^o50m6d#$8Ng9EM* z7&G1{`7mPBJA$p}Voum_!PnW*l!U+nB}@rzL)@dDte30_Y$kmhk~ti+i1!Gf@ViaG z;mpr+1IDjcB=RX3<=?D}_OL-O$XSD@@mJ2*>`{ZKp=I&|p&cV4tgoE#`)w+elqQCx zmgsvx87*U~$CWm`f*xIaX3s}<%d^>Afqgy~#B{KVX2k*BeW%h$<ksCHq=Vx!Yi&6M z1fhOON3y-vkf2)#oU!Vxb42wspunV0za&l!@lN70l_=Icg-h>n=!C)Or04{o1==St zrdwxCxfc6UvqEAu5o(@{$j8(HPE}aQ659BaK9OiLCZ=zYAp)vXJ{_&m+eeN9T@4Z* z2>2LC55l*$9C5*^3m=sj8R&V@F8OYX3sYS>4cH)&(DGhgZgQJZ&Su>S)dGD<UjsTI zT*K;*m`nDyyS;C&U+n+<(Tr29SL=8cI9aSUsnf>SnWQ$>RT$uRMO}te_{LetQ^lRb zy$|*zwUJjR9g7fKD_+HThjifFAF@et;51e(gR5x<dx!SjF+Ks56Wd!I`P$wtrsp>e zZQ#@40`I2qvv4=oy2{@0z4QRmjchPEUu=*ao8gYOMK+3FzvkVa&p9eUd2mmrw`MEm zC_TN-n=&M1*RZh}ooR~^oxx$^D|u}E%hMo{Xa4SzdI=j0IUmBS6>^87{ev!ITMRei z1o9bXcxu}q9Z>G$_)KM#xvKYo(1V6}i#HQEiwCKY;Ku?=TI$w$-j)i9e9J{>;>GI2 z&0}ZfQiItV4P6YZbut|)%<)+g+uPePUv@&c!GKPeBLaY5Yj)3;&+U)f^UX3?&{c6? za9`{n&M$=fO#l^KH0nq@`mOcQ0m<}CiB?8h78U#~E@{;-yDnNhueoavQ@A;{(>2x$ zw4vr%(O204%`rdAbTM74feKdFjdFsWD2PJ7FUGYyvX;F@VEtJ~fk$k>2L<U1X&JZ( z-UIdgD@ZkZx09<sp%wgI2WVj(1Pig)y4wjL!mrxjqs|s4=FM)h$#z+(tTH&OabEXc z6+0b$5a%7EjXOp5%XXYviy#3M9~-y4Iq9!df9fx8H~|M1Gr!pkAf(WEjGv=dNcNPR zuJC+M$!@j}*4+q)vOXViAvQ=mR@{nNiA<TKiB(rru27zqRlCnt5V~sxao7+ZV=XzK zERQr;6H?BikSAj&#lVQo7ccf?|5&mkEtg(J)BP0*T+=Q8Ri0?AUUN;f&$~rE?~S8X z$S$qwP(@z69;LU*ulr*0@?rcUF(~YKHZo!y?j}C<7Mo`3_hyYDbE}rHO_<nET^5n5 zu@~0%<_fUO0XZNZ$I9@b7AeWfGo$mvtrpeU*T|$|Z|yDZ%+7k!X;!VyDX^-%H`V7o z?5A;!*QqIcr?yf6zYFfILi1g=gi_YT){T^>3DH10EP{#kL9mo~<EjxV9S9vM1KCO{ zF2$e7rV{--Rut}yzNqJ)o41;J&SzX*CF}jc?BvoBcEGV-UJ}M(lGq>&`MK2!<3nxY zM?EuH!m+WGKgKt+3rEa#?*+|$1@|3{v@o%=>$dK!C>Y0>=myBwc%k=HE6{wq^85E% zTH+E-%o{R^VB6xwBV%v(A1rCo*y~U8pKnN>C+SqO5uO>-YWo`^IZ^u-d)m@Km(o_W zi@#~FIMPtI7*#)V0FNBIU@*ocOe>G{8v7o%bjuNhBzD2Ww}?0qw`V@;9do}G;i)~s zMYM@Zy|n#=`)92-N@4*_mNg~$H7C_v7;SzO?$~nWaACETjgtIX*g$9IH_DsipNF*o zR32r{cp#p=>t0u%W1@Lqa|rJ;t?y<Nl(8-Oep3M_`4&Y@V{J37wMByuTu9Q6Y2>7g zVJ&}SMa2n5=2t0o2UOm$z@s+&naQs-xDsVIdnwq(#g*#lhS;##4!q!c5fN~zPR0qr zZj@x=qUF#p$?x%I_l1`V59>O9rkg{eohPqQF>1}8!(j_L+B#SsH#(UV^xcJOy}M10 zFnOb6f7zx2IApEJBe|;|Wpa<2zrvdwl7$vso}XWM&{iii>A2d09fhJHYiE=~gIPM^ zP?e8KHTd29peee+T)37m_nrozOIsQ_7nN*d9c<(R<B3ztkunP+Kon2Lv4H1IB#9Mm z0{p5;Bs>CM6T}SRQO~hT2K4^TW-^p869Z+N9LbOxNh_^Fnln8p;^Z*ZsHm`1O~x?U zgx3`$CK?{~)Q0k|Nr)<v-jG`u7xB6tUos-JQK}U?1gC0fWptl}zK*FH5tyL36S22z z{<81)N56kppO!2wY&Ng^L-M8MeP@pGM|aH^gEl)_Ml!R9#`x)s>xinS1_Su|EHwHR zvLz~I9=jROMVxyObB+>jnT%(N^b}^W^4Uo&o@9)W+0a~9J#a~ml9F<8$oJLaq2%Yb z3^>5Jvb6YX!$2~T@`ilvekY+O<ZCIA8>yb?DH24Xh_b^)nd;<Taprjv!!Y{Dy&Xra zYw*?6XQFc62lmFOw8HA%$Q$lZ^Co+Bah4*VM2wT^cQvU@x5$l;g_GY}s?eJ`2}9nm zt9H;98les$H$~p1NrB%>1LGqqlY3A+)s98K5jPr2=i<v1=LHT<_@IM+r$*Y2W5bq2 zLONQH^7rVxlA#4izI0_$Ybx?BZ3N~7S>2nSUM#h$r`Le9zS+wlDWV3`KSvWWhEL`h z8uE4C4}4`+1J4K)*Ll+PZ`AAHm)NkL2!?$vl`P-59CPMpnqAfey^7aHdoVkS@=Vd2 z-t9wi=k6zwjZGG2SLxAa3&N#}2%8yw6a*s>*u$;+DFRjNF1;E|>kW|5ay=O(I@o-_ z+uaQfZEe-Yr94GXSn|PGqux#rpKt{ZVfy_@+LmF2`}}mI*YKGOhdW<kAoP{L*IpM^ z*&K}X$`8Q&C{i?nIWee@@S!z{1^k*|!+oFm=60HJgrDVjg@%dCPZP_F3>4*p=#MNt zw(NB2g-56kaakQBOtev>jv?&@t@0BI2SGDm!QhHLbKvsSIv8)6TWwE}F1yh!v$d`* zR5`bI1$+{p?-AN(K^d*?DYTzWDl(;I;BY0~Yvao(dfG_B#u6xMT0=RBeOd^`!BK*y zapLMLjh(FpY_@6=OmcCrGDlEQLy!(}7j1YvNXFKd1hyi)CTtDxy12FGX9sZ{s1JE< zzyTz1SrA#51W_cJS+ydloMf;FnIY*52Ba}pu}QzGYF&{_!@pu(M#8z&ZZKDwIthoa zq_3t3MySeu$xh11_*DF}C<f}`7I(yD<)Wu&1x0Wx+~Q7?aR;rxKg_zt5r+atF#xt2 znOANz3+6jhS2J!0@mAB;CdF&j6L0dWdns+H?Dua+_gI~d_ycp^lWy*PpKL8GY_O2< z%4iu~<7QH*G|NwBd@U@W8otN{K*%=d*UCuK^C;A{dO$uL8eZ7hE>f?b!Pg22zTd9S zy+dFiDR$5+qgEOkdxFE7W~QO$!hP1xD|}1>gTsx??cxxYxSWJUe;oN9yALA_zrzMb z=q6q}&4Gc|Wem#3nxKzMCpMU~UGdgki;3R2`hG)eVIex3I06>D&19&Sg!c{(2im{4 zf4O?d5{`RNYJ=KhK<&W>l`;?LejaEn{8CGec3Gq2GVr!VvoI-+ukpnNToFE1xa$<c zrk*MY6vt_^RAE!SOxsl~9T=}?-8Z9ew__Bokpzdp?Lv5D@yPy4hbqDpOY<3jievoE zX`)38@y4~jAO2>lnRI6^TBngl$K58CAOoLFo8&Fp)7@49JS8igudKXiWGwSbWa$&f zEBA@xx>TnRZ`-efhqJAL+<OItyontP^YIrh--$g!5a!46_&QeZj{r^nJhz<_KK5SH zQL3`BudS`?E!Z^#!;Br3%`rzxpzwW0arl=Pmk^hxRIBK4xsxKj-P6geyMfVTbP6Vx zUi0v;imsd&Ux#I+E^lhv<ray}dwe+N-`0;!l3=jO+~?@*;O^$|O|$m%%PW7hG*X(a zv)(+Ex`MkPEq~oJPmsij(X!mDK_Lf3bvQs$m4EwI6rE>r;nVe&!-9k`g`>qq@^iht zgGR0C8ZvIkQxZZzPMDQOsuXw9KaPD?KU6lThe%z0dZ1G0OBF;sfV>qu!E;`u66#M4 zkr|l#De4zXDwhjiv94C@hd1w7i2cnXR}mowFa>wkXGr#BQ5i-jiMtT;aLu60XdiG( zK40>uODbE<8~Bc5G9Q%^=^ePW-Y>HUuyK84b1uwYNn&36NPvX3hej-7<peNt!k7f} zuGr-Xf%{b?M`2b%hE;_OOBV>jXurlTKIp{9B^RgG4OeJV1=1<_`ShRhBbDy008ltg z+5u%QBT-BZ--CWIeDJCKH|;4IKkbFBg-_D?8RnbUC<qteyryBvgg!@u@++TzLP=^* z)Wn%L$}Q`)exQ`dzhj%|95@35L&-R87%6g-h8CvD%flrLgtVyNurQ5)FTC>~sf;}m z5>jSNbX=d>+T&6t8B4X*c*D|$W<Y!kxc`wYWRh3phQWFwA8ckApYnY&;FD6ZAa5fa zm+$5$yYDk@t2K2|tC$IZZF4N@SCV8*?APO(E#sL*aeLPAK_nRjL?-$-0_nIvNBH;@ z<6w$}O1p59fhKK=$pVR8WEy7_*>9*}pUP`k_-2l8@d0~AZOv>UxVlI(D0{m*!eN@9 zzRc5Uib7uRf{gDvmA}XYH2ZcG5yTpaW&IKR4DgR(NDeCup^1!g8qNgTCQhA~(py`p zTuQw@g8oWY#5RrR;53)CT|?X793J|{+ywT}s5iX1pP&ZrAl+HPlAZ=6S-o}Vrx$M+ z4sz>y;|&ac!U1A+W>f@{AzsD&-z5Q*xxG1)kkkH<_hPiX%W8PvzbkDUeEcpfK-R`S zQ#aQgBVe~PXix5p){ef@;-l1J8&D%>ffZeMnQiYe1PW3sU?GhcogPWAz+K>*<v@RO zLD~x{r?o{xGD9@Ot6@^Di9~5kq*4a~lzjnVU!1#!a~sT_-O^!hS0{pJJo9p~FdXon zM9$rb(8fa%B$mUX|9Bx=u1g|Jib=sb#J0ukvAwuC5M~&X=jzlHVo3tx10C3VGtJog zyYSLBYvAPjDQZ&|bRE*68TqEs9l`6VbI&g&gH1}iL{vMid&VQ@{@@ne45`WL<@9Oj z&!q3N%$zo9M*T5(-3?xQqLLVT$n`+z)5!xe7<#mW#|4MS7ZyQ=7147HL~;+;^lZs- z*gx~ii{YqM<0%!uiHjrT*HtT$YRMY@FbiGGg7H5hLOlc<G(DVpj$9yF58wltI>1yy z7dsMgOL>lbZbB(U6FysBe`$nNiDp%!NBFUy?5ZUdhJ};6#9{Ly_?rO`_-L24`ks*0 zu$si{IEx3@s}CTbqsp^LytmM+X0BlsNvy|d?{M?fnNxIfU9nT0_UBoGC<P)=vF{pG zm(%-=xP5x102fXy&r2}I+I!mepn)P)Js60dKZ++uB~S`6<8XP!?KJWZ;nl^g+;!Ei zG79NMwzB^AhMsw<128TPjOp4;Ogvw+!aM2P=KUHS#0S1~dU^^W{LYW!aGVZ@LFJ+q z)tIPX9~$yd7l?tXeRMh>FXp19a#_AOxh~bfkBu}WU|Y|V5}`Yk*=Sq*ASCZV&<Pj= zg5OM{O=-+4Ct37`qe%eBar_Zpz#K%rFLD)mKC2#?8QgAp?2}P-pJ-Hb!CiOvDhY8M z6A@}4E^4-Ec~KwOkb5A@+-!psu#yj?4tdLjx<s&|WF<q(u?Q1<FZQa92nNB1Vg0Pw z9|DPxT@17Un|3u*^zdhJ$i`I-zWvUtu%j1zi4LUv;Fs;;<e$>+1@(lmf#nRKxT#9! zt<=NFU^IB9bcFYG<okeBl8NV^9>LU)FbaatZ0l*~O*E64j<|&1R!J8ag!-5raqI&; z5Q3|qyfaqmkYx_O<m+Pl7aX`zw2Amn4CId_f>>p!sbRPjrdk3PHLb6L<8@j2Z6WiS zSD9aacoei%bJsp0L-Hv8L9mB$@8bJ)KY7aME{j=;a!+f`z5HVXznRBV6;UxJfEzL` z4C7Ynj+$}7p}u^Ez1!A{PC6-*ARj-#IRHbd-3*d^t!%mE^A)W%-RBOc1Vz7ai1>!7 z&>cSr<1pe;rLvjcz>^Akd{k1R$Dh9=BRg-x8f$f6oPxwGE(rap-ofvU&;f8EFqTuW z87`Fse=Lda8X7#;e)ZBjjmYN;3Y_q6w4_iH-t*7)Wwb(GKT211V_v&HRfZLMKg%X& zRe)N0$!rANz7BY*>i~UpRaIGC3>=LQiRRVMi0fJ*f-MDFi)%0GybFJ-Wy4#-EiNAL zVKH?#R^}1qJsA@mkR$<>XKq7!;UoPBZC<osrN+Nkj<up2?y%^_mcoD1IN<knr>Pco zMG~f`+N?3A?QGDpyf++?R<v$HbPBo0%n9T0wSRVssx2(L7@L;*w)QEM0`a@dh;|Cw z2bnmlW8{&Wll{c)A~UaLNCGCHk1<-8bsIC$$I;y*oF?Fl239S;8pG{juD2}<mTGz* zse+sC@ySTKNGWCrd2sZRc*5J3VVkSs!IWiVH<tfovS@bJUj?yl67e%(;59xi(fzXP zU^amV{$En<qxVdKM+!y_Lt+!^&`yH@$@~NfSMinMXvp&M!0eSqpU~z5aP+Mm)<e2A z#rz6hS9!bT7WEJ;hY;!!RHz&{Wm2*v&l@vsOzVNYa}IhuKY-LPc}6hp_-NO`MB_-~ z03Gx&+kuG>UQI~2v;USts>a010m@}4`=JW)CnOsKvYI>rWDQDI{7>+x7ydx?@UjZY zkL3a3mrFgHteM8s#eCj1X>&7ESv;%5BI0+XA2~pAbxQItxZ_Zn#Ursb5qv+s&or^1 zq|Ti<XhduPxDI#Xx(R*PQgfL+VY1k)Hb>dAtO|up!^by<#l|%3GeMC%_=MImAVBl@ zFGVm((E%CXSGmzJseWyNI}Ls7tcR854)R8%Td?ir*JW@*JLJ8z0^Ve4FiJ1F@<fZ8 zCkB97HE^v`?n&2y<b7I(cMALGU-kl2;kR~t?Mm1ou7l_G8Ksl5%o>yh)8#?a(J(;@ z<@{ag(s<JcChVK_ah^{y4(~L|Jko~u*f~j;5BC0>bc??T0l3l!>_|Y`;Hl0PY^;Ym zqIf4323H_u_J_zYBo@m=fvgD+s3S0BSL-340eivNTG>M0d{_aaK*y!<$R`O@Qv#!c zpN!8V3%X=_M_^}(%k`My5yJRo`rDxG(gwRnwS?Mdv^2GHi*1I5qoh}8E7GrPtmyUe z7H{UM;SEt`>Te4XFM~offrqu3biBhtq^5#Bk%MMzCROck$V5R}5Z<xH^Us|Ena0V4 z58|V>RP!b(0~Ye+^RTd57cZ=SNJir)cYYL!3*^iz^l%O)sBmw5SLb-T1YPjg{!0Q1 zAO#Xi-gXRFT3uA1u3xDE`vQb+siqdRW)rPDD~Xn2(ctM6K=KV>xYJ`<<;?v@GP~_G z^oDVn#hs3ue3)UY4=tQ9E*&mGDaXCL80S4Vk=<Be{Gw-*oCU`_;BeCc*FKIG`C#Wc zD#Z=;nvmNRoNy@d#daOoWtND~u#3=0$bZn=c1iLP39_gO{u=ZS^QYy$6)?gOScGF= zyR<iYpPY9(dIo#=tk1wm>%)u)x*I56Lah&VfcBctBQ&x!%W<O+knIkr3Ig9yR8FO7 zrcZyc|FTEhKB^8a5(*Slfi>n?`f($%)cPlJ1G_oyl7XtLZ~D)#jry~mfe~lxs%tz+ zpg#qe-+^>)Y>m}}pgKMvvc6D+fOk<l8X_Uy?q&Rul0pt8NbdJ`lcK*Ttf8ovxiolm zFx=y~H`5vKStU0l6GPYU>uT4rFYPTXn8M<jc=Pf}t8tK&l?EMb?f#z~hh4^g$0<*F zpOSa#z1)2g#B&fn44yp^AxH~LM*)YIpNT^C5EQRoaZOAYeX;*L?M!B3=S>?$?GA&` zop_dRx<G)s1+|--STD60hyD<|{)k24HYCMp_9N!}o~yTmMR%|m82Q?#{L@NXl4wT4 zHHJ;pQ(IwX^vtUezzBjBO@IdVb9v#J=#ef&4SzVaeUsx+wQF?f_kis?T|&hdgiPol z7)-@I0jJRY{k=oDGe~rPi^}p}9$ffG^EhUtO+}tQ_C{N@-T49VK~yo3@$hJ;0T9ss zTZtQT(h7pP@P|Z{n7#3q07<T8oK4U!033OiXu*I`qRGfMyYD5qaW92yAetZ*oz9g! zJ2<EVr<Q~#jIbPY#*Nip+{6bCA2sjV>Wu>ThgU3iZ((g#cM7fH>Np@%4u%x2y>Yv{ z;Wi&CsjEw}W>~_On|S<+G~-B8_-zdlx%arRvF4d(afZ#}72^2vwx)wV%J?=_Y#-p> zbad+p7v2V|I6QMVg{bmA0!~p&bM9&Y_T#|hl?pd~yx|WoZn#T-un2)^8QlnQl#lT@ z(IGMH&i#)i>u-6N*aa)C2&9aaF!N02OyH3(EFyxd@Suq(i@*Ty0l*%cBT>^1N@7nS zzxd2<f0SHPyvQM6m-J;9Bo4$TgfO|9A-;JQ)_JE3sg{95mt7Nz;OSLmD*`z@&DHNn zyA-v~!_#A>11?^r(fRQAXrS<pj^Xh@w3G>{g>WsGRJm<xQrZyGW7%%<pX~IKhHR4y zG+0T|iJ45n&W$>#pRSu!qHwAnq#cS?Q$>9gw3Iw}AOB+8F`I@GgeL*VyEhLiH=?89 zA<i?A4qypL;4FPCWRZ~YY(x(a?3%Pt|J)_5H?aagrR1Gy#E^WzbrZd0AE65W6qWNj zU;*+^#HGrL`Lqh4>`k23{Ld|bQfH=Q0_q(0Ft*ND4qJzW7gW;wnJ)`~3;FJb(~<)K zc|IS`?AhZn-DiDZwzN=$u#c*(0z<XGIUgozw228Fvp)GQ(r7@2gQtwF<Z`S=r8d@c zc)*xo39%5Nv^QY9O1?^S69n4A?`!xVVAKmNy%j&~@Ip*VyoQsc;3xIp7|kq@;-+V& z`(AL4JY(~M0F+4EO?ywF3nn7&j66-8>KcO4_Al)C?Vkp}{UW-ZbZVE8t9_#MiuJ`y zN|dY7ACXr2nNJ<Dx|Ht%c{wx%$@0fky0bkDqI)V|UMRr|jRty5>MK0iZw37D@3tR= z+jtX@Zz14x;!Rdb0Yb_>C5eqxCyM9*y+qK+O4+U|i7(D1qQyZiM7z*o(zk+cE;>#U zJZ#e-6elHXra5cMS5H;To9T{36AGCwoHn?Ai17v^ooduEIpERGe*3e|NMF;ZF`Oeu z3Dj72dSSoGodG#YD9S`p?QU2q^U@np6d#zlqTpa9w)L)j-mW1cA85(nJpYUj6$YIh zk@ujch}Mp8VCpJo3y9m_K)s9sJGG=>hAjRlTVNuh$F;*_G;!jWp=k0olWkiDP8=DI zZbEF;-WhJ<1v+an5#ruO!MISryM+RV`(aJZi;c#3KEWok-1J~w8$21V%2;G4WhJv2 zb;TK#f8(kqV4!jsR|?>XyoW~B+=<(nyY;%<<ClzW{X5`veMJDVC;W7S<`gnsT$TxG zAqQJ@aG9vXU_v*dEvY=F!X2LB{rNQ7LPuA5L={M?0A34$syf&nFs8%Y3JHP0ci>CH z9kFb7ke~3&G$&Z@m#(!=IY37kGqn+KI&4ul%aUO{e}r83(7%Q@6zV4Afzg@tsg!%S z@u<q%m^?txWMc|9ad3AUGZFsu@>weV1(mWV!o3wy3<>?2F=+#<KdFh9kl#|dg`)I2 zAN4hO&dkifT{HE;*^2>CTdF23H)?h7S!~OfrQ-+9ZRoJBW_HLLZZ%P>8Tp2hAw0&$ zW+4!?Yr?9I!aY|!h(x~pr5B)^8kgXG6p2OmL{0E4*h1{=g3Ns(=r7nQ0)o{vgz_ZG z5Au`9NbiC}8B|S7cuhOLTg@G2t=zE@WhNOq@s5fOkZ!-`Eiv&)$OmegzlR!kkkdps z4Feph<}?=s{a<|1yrVynT%~`ldy8=cvoLM{0`uyS@5En(8D=m%>>In%F4bum3o)-{ zLe9x|y2!M<DF}l|kll+55}wx4(NTAr3KeBfI7SJ$APc%uQ_p>G5Q9hIp~pwb2(t34 zn#miM#JC8RI7*##Yv%WN>S^d_p|D&7w1+j5eCF*CJHDK)#}+9keL^-K&s&dh@#b3c zU{qui5wb+B)zHz_rabB%@L^UzWI+_+cRCSGW$6YkaffeobU3}f5R>0I0tk`cpr`q{ zE}x5ClztJWgR8P^7}-2Ju6d54@+~^+4qpk^=hAKy&;YtTD;pY+w#)G9J$0&om<1-R z%t1i!%H?tGM>8@Kcb2tzn^a@OIDpR+Lw%z30wfec7^=T`dS3?t&ntRxk6)LIhv0F! zbrTT&Bv%X(A%Zc3!&x6sX2El)6AD4fv<5D6yY#A3A)HBbP7#v`2d~(4PMj?_mvahU zB&xi3cfU(ulMcaQz4tS83+tN{u;*Ta#~9)Icb*suVoi+Fp=m&3GWB7>aA95j+!D8K zmQ%TiXH{u4p{i-M9^mTLh6{he3yzJ)5wvpzx?S?YL<??)DU|)m=5;$N?tqG28$mVy zzK69Ht`#u!)coH5796J3>DNsiuyFYgG0-K65Jg8jY&&&RuCuI-NY6>w+(@W&%(A{> zG<`zS_$ic!8ENt1rctxT(<ux+YG+tS`&(pEMnYM@VgsiwL)u`9vFhg_-YC|%Gy-)! zt68j%jSdDacJ+Q`RhNG0fs^Er$0}IIA4c)+9;%|#i-@$@$Hf5(xnMo=R(5vXfh%4P z3DkS2`zl?xtfF_;?@-h;ms8`e;!{dmRNh3HPJ!_;5<;!M`h5hn$dPJJL<9)6JYCCu zNDodGhQ~DlG`+H4Xv<aGtNx6YRUin9i*<IZDziRu5m_sd(bM3RN66B0CTw@(k-1<Q z1{HE}aKUYHU}n17r8U}Z<fA6p+{>N&x0<{xU}5pK%+vbXS3~6yoO(FqwvR-A=E2U2 zVwq-xBnP_ysXjg~k6eAdlbz}m6=HMV>m}VNNi>NXNYCQ7ySa7-ll(lQ(Z4N-=ZvY4 zbf(GJW%dM}5?~xi#h-Unxg8*9uH6!!kW^&!)>DOCmW=r5b8@}Tj#w=3HcY_6WcPHp zVt;?YZO3Tv{mJS8g5%1&#W&7oXa>+FA2KmAKFZouH9NmB+%7<~b)NtjcXlw=N5X=a zTTo9Cnp@!wBlop3x*Ut+eBk*b_hYo(btjmD3)>QwBuxr7_<*!cQ0l!c)b|Pc$IO5! zxrvx2DXJH%=ZBW@^;F>$K;7_3k5%#3WA!J*5IqIafGg}bg)e1Ahb<yNHS>=|C@=fs zuM-ljx>ol<T+$r|%~Piqrysh@3$Hj0as0utL=?r8hUS>JZo_Z1<q@bRs<B5U1KO$W zLn}j}v;Zl`p}�E%g)?6(6_yf82i_U@__mE-EW4Q~AAt3-h$)C4J2PfEbvlJXbKk z-3kx2ChP#6z{1*P3T3f9v2Ex+y4+x0?zI$U^aJ@Av@gQDNeo^$o4=N6pAuqvueA-Y z8xQ8M@&+HsP)&N8{Cj6TE-j_WXm=#p3(+`+?pMr^js3R~s5w3VqLQYyoEed06zWv& zC&^y2VA%%Vc3Sdo{xUX^cT*3J*W`#0jM#Jop3Zwqi^cB;t#H=_Vg!~CclQ8_+-#2H zSgls7nU*FsK<aBM;`l3tgC8~6#UQ^7uRxkWX#fubzR9xQ>8apo&}p2d(=kZr2Smcg zX-|iz@vo~~EU^|;0><gdhj94`YhEx4ctx@6b?;E-039@$;O$(2PB=$v%1BLlgw@Nz zGPse?y0PY;qU@2^1-2u-FNVM=#}Goh-ITbX8N9BS?vZodhKJAkcEqlvh-yt&jwq!r z5;1oraNn*v4`D;Up?u0MjLTJbIW~LY`YWUz<U?t{JZ0m0(sE~Gk}9Bd-+L&izykOU zwNfmkFz$&I`x1}8eKSg4l7<P`Tb#DuA4y;y8yRJ9GVF2ut^>|%o>Ml0fGl`Y!?Zu1 zeGDs7gk0HJQUV76Q5m5L!aQOIS&PHKM2HHgqw*c;BCaf~@}E^yX*j_k^G~nLUNNIL zeUNu(QU0vs;%<d*qSXfWTy6B6{TOwutdOV?I#AYw-?JJ7!~Q&q6_MrgQo;_qsu<|` zHBlT>2R@`D$5)DNj*WdM;ygx$Ig2J&X&~~a7anfY(8~#XQPgg6u{z%!{jT;4H-@jR z=>e8#OMJ^8{PbJ)%}OacjDUC~l8uv0@M=@XZ~ii5qmx-mOd~n?$hUBwJBw9i=PIWy zZb3&wa659>RH=CCYW-HXABir;0?2#qoq^CLU<72}moh)LZ@~z^fx;>LRbwaa%*0d; z<YbgQxp2p401x#Tj)$5(ox-mXnV&g{Vjs_>9^R_`m7wDcyQHI637A6&v}ADrpm*xN zInAWx<V{1S?}Xe<ijXi1L<Lo?-}u~!6=kKT+`*~R5ApDu{DcK@&jS-YpfLNby9v3W z?zkb34>fxsx>9`M7@t^Ble0oy4?b61hkSiQM0%a3<DgEeAyt5N+%BCM&GO;9N(UoR zi|P7Ih_BreW%!#&u5(9wY;<T+BsPFc?k>pAgT(NSaJaG!LfhSWA+)QmDwg*QLU=dT zLs-c9>E`h0H&aJils!n>8{ub9cq*7laNhlQ`_s&0d~V9&h}1&hGqIGf&=HL^rrw7? z)x2H^xk{giH)~329C)sq%J$J{hlXypmF4<RKS`T*Af~ivGUwVHl`aQ3O2j4Pkk0d6 za7BG)B;K1Tp3!5xH&ae#hIT0meUx?m4CT*G>Ga_#3W4V}elah-QsHljnlKsOBw#b` zp97Pg)<SW)+30c0cXi~8@u8k2E-KyxMZMtH6w4!Pe0@Sa-dso{(KIp9w!*EFTq5T( zWcw-l#D(x&OfR8Y2Z&ARF2uDS(yI7>0Kz~$zkB1PWN`SKj%y$1Na1=laa)PRe|y&g z5By~0YnVh~*@^v(Zf39GlE31KxZo8Wt~+;{^{V3;E*OFwiq<oJOq@s<mT$p5@^yks z8O<!r=T=C*XBN1(kVsJg@wDTaD1jo?{_uu}BzHpM0A2H~o&V3@mg01hA1SjCVLS4r zPA@b6OHO~_<5^5C?&2-bQwv}twFsv;$HJ0ZPeH%)o-FqeiTJo|L&b65c^P95Xw4TK zLtlbeGazmkIQ<R264aZ6McM#k{pfWY)^$w&!KV<OkR^qM&taN68>f*n4E6LX6K?JX zW(_bw2w85#MD!38CC>z*(>(>+iyWbNeFEs;D==?ggt=Bh-zelha>C8kxfnwC6s*@` zx&LWdMa5&C-$l!HX+hB~sPkQ{uTy}-s*4I~ssYg5l%ASOopjLCFFoZ_<bQu#UjBs@ zCX~O!3Wkl+W$hB#(62^b+^lm$CkHk~j<ceED(XDK`uFhq4Hdt1oKOfO#@hTzMe9CI zUjRZnjn-tHfc&I6*bg{>!9GX+9Sqpq8Nw!YlvH2#LrGE5FJRgA@8rfJh~{KN$pbB{ zr<%*{Zc!xH3F|7?EqBxqF&+sUy0dT?KbIUh<P>u1$YH|67XTdOX7l1Zy<hQo?(NC} z=`?*S9yqTK(A(Sp4AL7MPKH(k@QY=4>i8WD89Z2XDeBsU)RaS_g^QokBh7y8D*&!J zw3bUTK_`xL2j>lddp>}YZrs0$wdkfb<>hOgq}N>>i2NsjGvqpKly1d%Oruyl0bg=8 zId#-q31C~p-tI7{rufQfI=zYKbe<z8YA*DnQa2{8y9YBzky{({JftWRctF1cUH3pj zO3J^(#kEafPM3TbOU5*OXW9;cRc)}&+*?Q8umdtPqmteu$LP+j+rGeeIfCQL$$D(H z57kf%9FXlJM<mA8N09Hm9^f#6-;g;1$jg^3xdKSX9q6Z=797N2o`%qoBOU}%t`2%! zR!=RfEPE?tg|(TGld*+08xkT3pRTX0JO-To1H4VmYrQ^SCTS}>a}E2B2WQx0iGwr9 zWzSS)<Hf-R1&im;pYOL>6%kyNr@477m=1~)(^6isxM4Y%<F1ceKv7)x5Ci0b$Y0?C zzkFtbgM268K7se)p&6E_BE!N{eqTi6H6Jm&c*(GV_;>;0WDKHx%zVt}{N-K1fr7z% zW)X;Q-iWXw2wv#YfX`ojOiU<>>K<Nn=*gd0c`=B29rz`<$!C)Crw?aLOcgirKk*ib zw?L0AFk-}rG^pZu9ALi=@hjV4B!N1d_G3ks--sqK65>@q!~8;BPrU{sDXuXzf3OB_ z5KxHnooQ0WfdC?I3l|v4PFufW{bDTim#c}M;g~yTp$ytkY}QbQ9{Y4(0jQq`AUwqw z9tIG7<i5i8#XE}AVs#LlDBu?8RT0=$Y%_q7&_N(S0Un3avdW5|h6{{jVL1lMD$9T4 zN=ht1zuk+|UN65LenY)+-00JzZGKT&Tzr0S(%K*60*oB)(~b85CZtg}r_;?3HxL9B zAf|n9G=PzP*_Xm-f^Pf`TvUcrFp>ub`FVo?M!NhUFHhfZ!N}6m641O)fg@TcXyVin zNI5WnuwQ_Yvi!8{b`9rR6>2b&<%1aL7OYX{Lu1KtU`JX30LKJB92TA%`|6rn71aDF zZ*%j_4wDIKG;oP3&3^q^B$`W(GpArAxgx3Jfo%lw?3Iw5pR~TR;zywdBUyO>F<$h` zy@|3k3~SJnFj`~?z#qVzQ%Ak91Q|dpK0#o{X*zuokDR-h^Meux=Dw1~XNKet)}fB2 z346+`F@_4*p5X!`*(Pf%*RH|$@B>v+PQ$^>)&uee>ExpAts@Nu@pLHKGadv`J*2nk zg))BCwr!NuIf$HsPS<1h7maZdFo%x84U(*$fp(l2YA}-Zru5Ug%F2IbDDp61A%9It zPVpZA+KzLsQ^!Mh0??AA{fkmkPSfd&csSUlmc0X*H#qz6!Mr_yeHknm2~KVvMC$B2 zx5vE&W9{<X!TI^1pRdEA$3mxIB+J9GiZS~U)qhxEB%ensAeoD@odM59kL<Eg!YNT9 z`QPA4?B~Jv8+YyVUw;B|%0-<kWC406XCbz0on8{&v))_3;<btUiOb#3O)#?S`*y!x zif2$7YvP^63^8{E08JCTv@&JxB7gp2AmN~#ciq<$yiN76v@J<RM^7n&PDwAX%@9^N zK4CN&^qi63T(3n1A!C**{x{wN(c1!J$BxZ{(UA8rLJ!?pTjRn4W2>+nb2xd1RD|qR z6WrI>iX6tk<lMl#_Gc1ez2UGEn}}R&B49NU0K{T7!7~ONvA^r8(<zKqk+6}hz`}Z& zEV+&c=SwB6pAr<=J^$Lvo!kM?3q`{|3dAdBQD>RG6v7vpoE;FucpTp$0S>`k?<6<2 z4Xwknt=ML4J8%_V6-njcYu2o(LBHLM&CD2nJN$-F9vVAxQY8_GKm|pc3{8I|7A>Yb zu6QpaQjaWjTYso)+JJ2UBdJr&i35kUu%rrV<za|v|JF$w{Ur{K0`0a?Ugc=42Ig0t z8xvfg1>kH!e`z1v_3&#?qCJy1HwVMPwYT(XF&i5^KNQE+0^B_YhcB04W4?KBVMF=s z+p@BchZGeaiE;5PIkGTL9JAt))0Se<py+SKdN4Ya<#dwPvomKywTW>uI1C-@^CZVA zESLLuKn8Bro7PoU93BZnBNYF}8gw}}Oy>eL(TC*cYaZz8bmB-GQcM+XaoDoYmtN~6 zjcAVppcrd(t^x6BZZhYjsOqrw8_FLgz#FNuuia4oMq*0xAs}JiAq@l3d&jgM%gnh1 zmbXcCfNNH}D^m67DWBm5C8wfjm(ilOQ|TqKe&po(ao~(Lk{-w$IN%^~8NSj}-o1Ul zw7k52eMQBM(7!tc`rq3K_EX&0i6eL?q|yxOz?R0wKRQXHui^yui@B8M!a?~Jkm3qj zy%G-01n3X_#?7FQuTM-#;aGdJJN>Jns-tu2J5(k3ZaSE4q<_Kl)a!2epTVCEKOLT% z>Sy37!%2~@JK&FjkEs(Ol=r(%HvS}lKa5-p%Hzh(+SuX}&O{u|YcmP*GKzS(<#Q39 z=wgRsM5ZHt#8ZaBL^ue+mWVif#rK_~;U&{$`uxW<2#)g7EEC^JDWA;y=NdCn<$w_< z{&=xf-ax=WSP_CLFrVxHcf&~q=9Bo}cnic^U~eq|Vn3&~vFSsM)5EB{#fd%?iiJV9 z4h{>IUw~Vpd~f;m#WRSJJ&ZXenS>Y#GAE9k2q<&csY3{}%54{3dhv;USx0hAW)93A zhdJ{JF6Pu>w|g$MS3|M6fOs8RW+XUKq6b*<>$hOiA5H=-*i+W7HD{ZUh!@-ivNV%z z#<uGkE>ic{%#`ul0nv3cHl)U2a6V8p(TU@?9N1i>!D2cOPvfqK761qMV878ui^X}c zSgbm=Kufh0G#G);>q(rtN6q0&57iILYjRdYwD=Wp^U2W7i-QgO_!GxzUQUj%P>u!U z(skwK{|!rFV<1r)b->s@0ftgkr>Y&uwTtzN27$_437(_D$@5txt4?l~3ogC*A`Fnn zFisS30y<uwz$Ga|HHl*Q{C|&R9lGX)gUx|7w!wo<>K=6BC_w^lJ0C=Ub!<@11?YYs zzI|sziIV~#PFZE;3&{ISaFF@=jj&8k97Vrd+uAVL^*>T1(n%W88pi~U2`nX1FW9wc zvvZ-yl)6(R?#SJB!6lcR2D<Y;j%$jWJEiqlcpkt3_#-WRA@s8z7V!#o$&d%n;I)02 zoPo8w>bkrQ?lwT5`$1EIIa~P^fcNVoty{ZnL)nK|#wMe08ma#n9JRp}zDp?Tcc&F^ ztf|*x&MEHlI%;nNqUwo;q|L^j!U)FDR{%sd$W(#DnFU}MAC1G^2k_0eN1~$92uM;6 zjNFmK;BSN90N>RS+v33G`;Xx{F+B%=CcLxR91;7{UGu^>Z(`uAB<RP<&4L0umq|Ql z&Ed>2=5H4yagWInNtr+*<A{H}H{-!F-!aH<BO(Z_5w8Pr835tn%zXGl+~H3Ep2R&j zV7s=I9oJk}CBTlnQAZx^PDrk2dW>f#%0w8~bch2<1>Zf^ef&4x0`V5uvjt$-;bai| zpJO9A2G*B_HO4(d^n53byJTdNuzo3;tYYHi^$biKKLTf@;*=9g5T_6$gpN>6k(3L- z6?s=c!GF&U6UGoAxLUyJcLgTA+n{$v1`$NPh>mjX#EEhn^&UD(B!rPbq#lF`{brn& z8%|Hm*%lOwgXK!B56U00skoGF7Q$jWN{ZhOh^~78yquko;PDCZIftMm0W~VV$2xcR zu;OBDTH>y|79gPD_j(CN77&cIYgyGwmj^HOpfjGsM)68~XAuPyta3!duP{QO8*gDV z;@R2@qd+_~fMGBU3F%khHhDT4>ed%!pvn};Q{IoYN~2LACyweM(4GcXg3sKE3CEJF z-bp&Wjl+44Dlsq(KLDM#GC_PN&ae3;T8T#7NhJj(lboaxia5Gg>?6{vYik@GY!dhr zbG=K&8e~inz+fl}<{%A>vDX0roCey)OH1|y2Ant=xWVL?;%#ZU!AbmRjKj7dEk&bB z5c!4Lh{}ihx}0JxZK${ji_RUXDJeoMxi@8^p&Zcrl(~PPH|c~TuB>S{;TtL@$1MPV z+eupBG3aw03&=U3tsb<UNQffv;0JnDCxYJGMNrb|VRHb?lfg_u%qb?^ZhiJfDd&Zr z)LF1?cpy)mk=(=JaONbaz1kGX$~v3%dRIc`ISBr6_#eVw4gV*25<MTlQ-LryDHLV6 z6TSeRj!^yA@NdCC0sja1E8$OtKM|gslrO?NllMWyzI1|!R*ieD4J?1fKTt8a#CmC# zj+eq=S1(>z0469$(=irb5fsEP@2poQ5{b8g#0)6`I!CtgUJ;oDk_f~Xcg&~5SzxL8 z01GHn@_ahy5%JL?udIWnT(H~%B0K8RsSJFEB+q&C#uF!WBpbhow?MoFBpP)nEi8IL z)AV;i^bh4Gokln8I_gXrgvA%}cDR!gA<`5bjuwwKNq-2+%QX^UP8{bCfLahE^k5oM zez0ys`G1_G6V5mUNHL*a1u+o?oRVTLc9I`NfWe8PE4t)=%t`!^#_?+puq+@_J603B z=~FBkC+6rv<$4gQzqcDd+~Kh)oQKLyp}sthAE%qbU3K-ex(<5K_~*pJiOyt&!d=-u zPXCa?Loz|!zXt+h7>rse;Ouc!B639%Si!XL%Fx2Xi?~sAlu$p21n!!ly3zg~_O<ET zcI<$mOTVL9QS_aM1|S^oz=rN`juPo7ku<OO52#8mKrfXC@OB-!axm}loBS|maQ?|u z6=64xP=_bC5+>o<;8Le`Vn@jnoW*(o8bjkAE#j`Uu=HrKM$js=?1ZbDbOhFBKMeG= zdQ|l|w8=*iuSAy&*Or&R4;_Wm-7dE(44rg&AIz<{-An8526Pf$J96a6>|o+Vdl+(R zgQkVCbNQUUG%T{Wb^@(;J3gg*(f~?|L$SRD3gQ{1Vn7qC*c=~EhJ+xZge3^or$ANE zE&PU@96WPefn!wnxZH-SDyM>`niH0S+DMc}o)p>jJ&bj5{)xHRMr<=E*HT!2$H9km zQLJ!DlBS}kp3=r<p`EYWdeg8J4`+BFPY{`V81$w61+Uv2f_p)5klNf&hrbp6arjr^ z--MqBe-^wQNAAP|_+#NIW_>;U{qX!K$;r3@p86gDf=Nr@sZZeV@R!4nhG(n22hT<B z5_k^Gk?=Hx!sEQL@B|>QfS&=+@)DpVQ0a`f7!jkzjUAp#*8NJBhr|yyN2bGP!J}#c zF5-y=9`ZdW00N3;SQ9W5H(iJ)5yuKEe$Awz6YMBlEhXQhnSLM*#z&$eeg1ajWya&* zKoF_EYTI!QJ2$W4v+RPAM<!-wvwu6BNQ21*@?~CNf4(1eG>hF4GJX+nf&HrmC>{Y$ zsxwOp3;zcTvtMB|c~)IRgS2VuR@KQSof{0EazWot4fNwh#Rkag(Yo5&OE7B;!D*eN z%^lOBcns=@=}fsY+K5xDp8SP{?0;jr`#nz^9dn|92-*cPLA2R%C50v5@0@-luf^ui zFixp&sCNu<kXVQEey(HpQv@WE<+qa;H>QfAoQDmN>{RTYTm`gf%Wr*?yn%<Nao4E@ zAWoRtB-OnM`e7`=NQy)`akOj>@C>4-(p#{wdUkJ=tUJj+n&JrXLDSv^x9K1rWI<fp zQSmrek)sde^igiM9VOCFB4KSFA{p{8p)N&svK9-@uAE$U#Lg6#dTzM&Z`o<~R4L<H zgx>+Mr`@8<?ZhWDSAa|Mn9{<+DR$z9CanJ+ltB+h3~_JyQI96(co(cFT(-KbjQhu^ zaMV$V`RpmEeSx|JXJ=V3#6ct>1z`HGqM}m0C<G6j$f=Fp<8N^I{8BHH3R_fglHGza z^eW~~BUz{ra+Vd1;^C8~bnSJ{Qs{jwpX8l^R6I@(o~f<~ht43Oq?@6;mJDW5;e;_Z z*|-z*xI&IrC$6quKuj_?vu}11KhkliUWCS?Xee`EE+Ag70hDaV#_E*51d$0yu7H0X zo**EB!5_eL^K&NrbMS}4(~*yJIQ%5|3*p%*kH8aDq&~;(@B|wx;TOR_4}Tv#fyn}R zzGv85;LnFY5xxXI2|gE|`7rOF!&7(UEco}~zlCpx@Aq!fjQM^=lK(>7P7n$F<v;ui z^bbIuc@uaaXc3E`Fn<vSj@OJP(h^Vb2Ri`sX-6`6CPA034#QN6UqoP7k%Sr20v)Z> zVJokWBt>vTT|Dp6Y7y0tlE!`ilJ!hWlmP`0;3^9PXO@RAjicYXapF1V)A-+b3+z8F zz*9&V9k?7I<a35!tI{=n9{d!XWXT)0Y*uP^)e4J=c3Xssn+Zl@BeWeR?T&?hyj6D7 z2v_)t6DPVb5!{B2vgYLEmE|HKL614)o^_RFGs9KZP^AlS>yIexEO4TLP~|db-w93> z9-l)ml>OMDiD5|o+X1p^^#ID9IOyjhC`<A6*P)B<Mki@RGLE_Nk#8b5CkbpPj-1Uj za05=2TX^W;B#r(Uha98Oi~2g~ilbl{NpU(CP<NN92Txj%lb`?NNa;9=#y+9-Pk)>D z4bW}mHgzoe&dq;^==;SN|1uiu8p$#le*H!qovXFxM$~rJ<P^eY&jpLkeWvfOyF%0O z8$jouLT{nRDK{SVHj+yc+XaG~5p(7NnRFr!ZXH?GCJh159a|pxQEa~FMy%jLW|S&r zUXP&L^Ku6Iy6(4vNkVW*wzV3-jDITVu3#cXVHg_If<<j|7k_|5%=@FTh@wd7BLfB% zFT|dAZSKHqjR21mN01+&Bo8vDLRyMgOGMr1;^G5=WS>t?KqtC#381sTfE@gHeR=s` zBU<<%GRw%$z8ih@6|8>yob(YbjpJ|%DiMtHsmddWETcXM56!5z77}OpZXt2j$A;?& zNY30xtGV!wpkOULK|z9pM7?Lj{{j9q_<`{2;h%x$L^>V*82AI>Nr)c+PcA$HoLsmF zQc}!r0em?;LCm-dv_Nit`jarPUd3Btc;qG&*Vu)N#fb=EhwF45xMUo25yOK37|37z z4?ORPY+m3w7!^yg;ry6@r4=@SN{V<4Xr{@q<e=<`DxULQ%dYBUt2dQ>#W$%r2ON{0 zeLxw@ZiW|`i%5YiL{{LzzvsglagjE&HRFH#QwwmxB2htWrzJ&2SC$kM{TbreZ<iJn zR{M;$3UCNN4iIvBQ*%=qHjz^0=1nS3vIO>IvmrSC2%H3T7FGg`JQ}(`I44Jq+g!2n zB1{JdLY&nRwT6uoMc*`3^(zn-w?|FIdMHRrdfL^P{VK6J)l|iP)j5Ce0%sy7%p-;u z6i@9TyO6(z_|OsPw^QpI8Xax^_-#Uenua#mm8>dP;J(wM74qVSEJ1LG{S{?>XFyga zbyqn`cUN5<>8nA8rp37=(GXPGut}<X5ufylFpMPkBu3O-qm%0x-xL6D>2q*Ko*qq{ zXWf-KT+i92Ee(xmQ{#vo)m4tUP077XZc`AQZ)Rp^U%WqyzO%0c$El%T1;AW(vN_KH zmnpe%@QwZ<5*C}Cd|v?LUdPVhSy(MLoXm0L1cxCNtWDAsf*DTA;4F@IGdO{@23P>L z>)a4c1Ry~=Vwt3>+^<HRBUr4c%2l9w+)l-S(JxqXje(mgH=Q1J6%4XK1G2IpLMMA6 z>2q<k(+$uI+X(9|7m>!J^=+pvpr;mm3dS`~({LKEO*d}9j$NgQuU$tA=-{^YqrbK$ zq$WpYe+i>HhI*-rlL!E(cB?4DlZzEDNo#NaiK7IfEE4u52Ea(p6+eN>2udgSsUzS> zB%54`<Wk(5%fv%++WZcFbHH!G?+*BgZZ4jm`|>H+n4N%Zha=~~;Id0c0Tk$#*a>9p z3sC?6n?C)Fg_yi_lr1LRPaas@`pCIs{ssu}PNH@g-~s@&fIZYr0F|BonDxXb#dji{ zz!mX;4$tT;AW+=kkCiUFkpKLKd`N<-R6~CROolN)AbubM?>aiW<AUi1ZjkxsFaNV` z>NWQ<FLm^3msWo|`>qnRoMvPuX$EJV5Dx%FYm<Hh-rg0+CjJv|fsnNTzPGUe0vClW z!M!AeJjnUy!Pm^g7Z_}kFki~);8@|@+SwWH?LHk`Q^JyLtG7*R$92c}=^WC=HT6Z3 zQwUsW7=iMl1S$FKRb{KljUH7FN{w!zVE}NB*+=D26A}<O)zsapazUi6_4TG9SFBjk z4Bc)QpwbIjhn+Qok=$02<V49z&(I`Q{sSVuVJmVk1A_zRxA;cM9qFjwXiXcO4k`}W zuIekRm`<BO@whKDyzQ%Uv$NASZ`($2(U!BgBr!&Bftd1R`!g5c^y$;pWuJe+gXpQ` z0C#E?mz>7bA&Yr;B{+$`zkJp5CR?4-j{;*{$~f?v4LWKH^v#lc!%^)7zzc3ZtX-es zu;;8;nqM7NIEYC8wsrynNltVflj|fqGs_P;;L{5(x%f_iyFujGH$jqJ%JnUp{*#Q< zG&c?jY!{*t8+6R^idUB|L*yB@6R;Tp+~%WU@NXKxNa|O%85?@PGBaT~R?!Sq`AOOO z^-;?VL#y^Yz(&7J#y9A)ODB@^oZ`Q#DzjfGnHW=!+;%BR$uLSU-@k7C`pvOWc<3tp z9P{ym&?k9=Y4O~?DiZ1{gh4|EI(#q1Vbe{KGWtgYbDFpPO62YA00wj(jH0+O^_@p6 z<k8w+>o=@lR#I5>BqS`Z5QbhTd}PDX@E-bz;>{daPuPeWbwRT-GI?mmF?KsRBVUb% z0&>$#g6@y-5AUNuWFP(r{_A~&3G1slb5>!~zvf7QXJ-M_8o^V`)!(d=%FD~89Xqy< zuCK43)Y8)O>)-wEmiE!3$5bdT^^JAkR{p)Mt-d0x1q(ys$qBAUTD<MQ0mzX?5Rb?m z!A4#SK#$Mxl3<^JC&hC<5Rc4ojMMSNd@gPS@d!fF@d#HWryCi5Szm}#p5yu4jLR!r zcVui{b8!^!MchDwNUyD_;iw(uUoXik9&<!OYSvK$4jle<?Z&lZ%rwlhGbyV+tP60- z-+uV(iBC+rZ+_H)C@g8*#byfxHfQN5J@wg!(P1RvJLZ_+KZ;$@!!QlRnKUk5G;qOR z8DSiOI(sWT<c&aFXJ9js!2#HWL%&~vh<y@cmG@EQz-4zF+M}c%91V5}YgPm&cofxU z^ZqNE^gRXJ$lzA}1mgLRfeY!U5HpkQ>UU$C;nZ5i$@}5Mi;9jU;Tvq;y$MGm75Z|1 zKry1-7W;xvk*DVPK1uN{h*tei`>M^_wQIMO78c(Ou>Vnt`8ff6u$fc_<>chIx3*40 zUN7z|zyA6T->*EZsQ5_?;Ip9L4BxVS&L*}ERZ<8r5|TQK$8+-Xwae@JYnugAuVID7 z4<Xz+yC71`xz&jyZI?SR2i9w)DgeAwBeel`Qr1Yt!Qp|Yt*QB!tq^;K_-kiUIH)3+ zIye<>&Tox&PE#Hd0r+|)Ej8tgnz~&s(k^x!4jH93Z=0T!kaz{^`p2->*QZ^FdBng0 zneBd|Rc!zfsG#u#*5s&?AlX}+gVE+OMOCLzFJW_wy~7654B$XlAvrU3TuDi8tQ`h| zwj~`~2g@jb-do#5>$8jj18%RYuDKvPD>G;FwpPVSISFP0kW8+xt)&F&TqkMRjbr$H z|A#n9&<krfs>B1B%(Qf>e0vA9=|bBHL^^`1JO({=S5fVYU2afoTmj5G28SahP>WR- z>EeeXKLY+x)j=TV6>vt<fK4cp?R6RiB<UW5zYe~XH+ylUc1qy;43PZEPe!-nW4LSX z+_xXqm6lWf_TXP1TKdHoIeRH9<*P;n{I3S)<Wy&*XZ~;F#w`!|d@Wz@C4AI9x^(99 zx<}4C=wE;eZ!`<H8;s;<LkJP#&f=Hg9&S1e-3T-?5T@fB<J2y0aDfCm07g9H0OLK= z)pX-nsJ~c^5Tp2v@93;w@yUwte2P21XS$t!m%Q@0>-6g%WB_T@_b%{%^5pMv#dKH) z>(@*(FbGniRhxkLRq$~aZ-J1s05@n(H)$a$K$sFCQJoI)y0p|ZhyW`%Kal5wE%TbM zD=Kyb##5MbcAhM>OkILGM5q20C#eK<TzM1Z<e258#(DoA+N9=-L8vBf+qqLrrFPn) z5!_DzB&o3La>R=WAPJV<^D$$)8hWtM*Ihj_(lbb`s){W1O?MY8s-F)jJn-L}Dk^V) z4$MMuo!I7RZh<aIVU1Q697OviAUUk4=n;?z7m@fUc;>{>SQk`MK-5yMOpsGfTD5kS zeO6;98AoXx7!tW1bLuZBLPn6+sUvL%wIK=&1m(3zjgyeh%TCkkFY(%%+oqx}Spo4$ zTh-#`9Gg|@{#)GF`nZBEI7oOHU{j`(Q<h+eUDw#utYCwCz<*}VIvjDswl-1!>?Uc$ z+0D%@ir*$@6YB&i1L`r>U=r^hyLIe~@aY4yUazaE-I1P_nhikG+N%d$l&0nus`62r zb^ZH@G&|^seH@T9E)3alYxz$AKF&^B`lt>((1D7kyKsp45XX}f9mzDHCAeE<^(<5) zB7h{$Auh(@3njR0)96$e1vWbugvIXYqh`02Ee<1=V*OkOwHl*AlMpYm<v6a$T?kb? zG_d@zt;kUjnEOQPIz>^c(xlAGa8F<+Dx5XR`B0jZ;Hl?vz@R~c1`au}AoakZL(~x? zM|sDNJ-7+`ilcF2la`yC`TV3wlOT=|8aHRohpDPsn+xWl9B`QDp_OugU*(~ZAVO~N znxf=6z}#|QnEO%=2q=h1O45zL{;gCC5rfY@`&9ZI{@XQc@anr(o2#p;2Et#AH0U?L zzXDJ8R3iBys|V5wsY^^uc);7*ehp3oJ^X;^_#?nbG=O;42?(^{NvjZM4BYWk6E}en zyhc36CoyWJ&3O1M72S|wX~_f(#j`-(3@Zx2Uo*jubopE)i(g+`>-f#9KL2>g=p&DH zB_tn`H{#GGJHB0d1Pfy3DYD-qU+B}j?Co17J~ZX-`Cl{7xQn+yyajf*0LKkQ6{(jh zDKUuykc0Uh0Fw7dgWiwbCH?Ld#7>gyt831I_^M);`+})a=MxA!100}_tg47UQS|TD zc<)bERj%3&VCH^sJKRb!dk}tm57^C9>K3IaPg-*7d1%vLMS_zFAKIl9v-f(vYB$B| z_Axnl@x4NKr2OW(%8Jjr4H)GwXt^0;ShwIie1Tk&b~o%?Hv#yNQ)LSD$_|FEsjX37 z<&YL{SV7T!(D(K;f;&A~d+aV_8b!*7;&p6#wnE-=QZ%Zh=&tPHcn;3U|AYR%KVY+K zKvY{$cfHA{rgoPEKnaTNrKg|^aOtY@@{PSoFV^GAhJFDL5n%n@_Pj^S?~pojLk#YZ zvC<BtA;2YBiLwvH{ARn(JamCl6_D{VIl96INN$!lOoNfIG$)+uq$8*&6_nG=SoyxO z#{0mbKNH~GO)NhpZR|K2w=sb&WeVcWwiCB6!$M4**&A+I{#Rd?!)e~Ksy-&^{@+6X z=OE0{irsWbGf=e=IQXVbm@pwJ+&Yq}Cma$?r8royTU5avnU$XI#~$TF=*s=nb^?7C zLHRett{<h;1p#0?juK9w$2FJiItB4!4v-{+huZ};N{W2d@`~t^GO++j_Qxjp2cMra zZvn=`@qEzPfx$&!TibWmlKb#?o!@o7rf4;6)#NCOo~uc|TuD=N0CVLbOb(LBC9nwK z%lws$T+OIS(!@Nw=VWVwWm3H%%R3Q%C*qg;{Jx!AHf?HMziyodQEg?*mQ4vetE*ER z>l-r>ejLIS!tcu!;N>Ooli|OD{|EfX@KNV3e{$K@f1E#hmZZy9T7@%VKMM!!1SSSR zL;esi6cj4{2=Ee(7k~>*JQ0ZrU@-72q7vc|k-#OX=?BYh^<&7F9?Nh{P`nnGR-P1^ zFw<e#7}X5LUs?L{joFz)#~g)yh@lKS;jHwX-+swdfPgZN2rSC9JC3f(e)-RMGZjAW z;w`XWw*bc-K`XM=bHN06kVZ~3G)fI#J?4@UA6rworZN^931TNd#;{MZ3*=aJ11G4s zG!*5Z*0Wgsh4|6mWJCLPdRn@t%7HFPwjo4$QX6Y(rlSpJg|i(-6cvs0>&C%O`z<9U zS#U=JNq!`p<%~qy7hHVtyr-X<xd8wxH|(}I7_eF?I8`*S?;?~pCb)A#(XQY^_&tb# z8~F`UZ=@4PV;i&r34pswcBzxVF+)0azjXk|rvqeo1e++GMr=qY_tw!k$&RWjbx3}G zs#n)vm^g9b@dV;~n{F(|Kcw)G%qFRR8V-MKb3=387MpW`asc_>kCpnP+y-=QKMp$M zPI9N&HR3~VMI5kEk2L|w--oT<hI9b|HYY%FW=0109J-q5`BT`+7lwr0iu6RgCf$?@ z@lw0o<(wNZPpGNM$=}B{x*40nD0dt~IQY}hS#PhdsMrwY#ocT1uts^Y_C190aF3J2 zXC?|RV5%2Ls@}1Kd*ZN4Rv4ObA@b09;LW`4I5J6)J2I9IU|7ElMfy*jCjFIdY!{GZ z{Fb&>0>+bRRE`G+d=?81KqS>DQANuf8a0cyGIs_fMcFVREX3s10Z6t1J#K`~AN~SS zmImewyy=CxFH%M%g8(Dg>T&==36K$$1i-jDlL_tSsMqkRi=+ugwF8f;0xU&?IIDqC zeWWPHPRyG-HJ@I!@|&-GUoKhdTC?UG&(^J5)1h~5Fv1Oi9|{jHJ1HNY8}sg56JC&D z-5mJsdc6_P873y2Y`E_;6yN0l@^I*ima~XYT;r~j<2pl*M-%W9K%{^OJA;vd7=b{% zvy$wb7N23!F9Cku1Oqpz%opIKNPqWwhGzokG=WGnm))soe&6Ww&)@&3=!la}R9x;6 zL&u-^@uuY;9ZNt`I0PjaDQdxJc#bSiefy@#-<$I1_umbKivPr0V83bsig-}`0UHB~ z7Foo>7L1b*Bt`l!^wYc?ON$>n4{OO8&_Ae=Ft*#7oO>XiPK%g}*2jR(r=IEuLba;2 zplB|Xo}W{@tJb#FY7$P^{K2@5>jLC8tEc>W^Ik*0A-+K+=T6cAw+)CMdCP^DT=>q6 z88e(D5jAm8etlSB;iK4K+?xQ=VQ`GvDjSSfal@`7{`rWwUjUK-IBvk$pMmd3qOlBI z+iEigCO9$}V%p#YuahPBN$b|HE3=(QKZ*cOmdT*Wo+Wqn_NtvB0VA#EA=gKBZH)>G z#7A$c*l;(V-(<bppMD$V`sv`p^ElzcgY<#~oqrHrRQ?)kZRS#>*ZsP5XA<T&yMQF? z3Xo6%_;To&qM|&ij|p}CY4zOS>Ya*yaRd~nC8tow-Amw5uLyP7!kOOc^73`iKfM+b zbHk*1P=Pib`h^lE1EBBjP()p7GkU~>V~wQoLRFEJc@fV%gc&Bv^2|0vx)XGe%Zafl z0A5@(!KiJJ3omSdBo3ckn=2~M!*?Q+w6yC0lmk4Lzr+E|BD-<>GW^o=@_K;AA7Nab zjB>c_mWAUAfMjw*ZS4faU0^qUUxg=kB=%LBTb6#?S7q2&zTfGH9ox1%xOCa_>%UyO zG_kUxLTc=yEE`0k=d#PM{L92iC$VCrGIzm~gK;}<R$%iCi`T{u*-&;GiUg`j+S#hN z?|kNoCyaOBeK!N)EP#<9BRxBm&klh9PI>Uma^$P6geR!DIq*xs(E=U$+_^FxM2<H1 zfB#v&?w=QoeG%NL<cR4&5X~JTo1VZRekiFf9unds)Yln^Bq(6UFq2>$Q(W7;=U<VM znFf*t&^!pTcJqOPPUJ>{;gN_LkLaOrg5i-R5c#nn+;fPW@VoMZmj)Lfev%Jay9<w+ zI%La=g+9o1xJB7SdPveLXGih0`-~Yg4wbP13}h1jiMPOh(*iiTA{farREJ~7hZAr* zzK3`b=Hpm$w5<E(k|ig%wzk>cM6jN03lgU|+kY~a>b<YB8ZPBg*>9Xrjt9FsPB|CD zaGRQuo_^xEapO{G!FykM_2wP2ItAj|It~SG1IZLGL*GI3hw(VUyxx=}TH?wc&y%o5 zbO!)(PnAtTl68PsBRA}0MvWSkzk2oRZP8M%aFhe!?N7lO`4<rF#?I<$8uPN94Zoqm zd4pgjR)xACdGgwdb<l0S|GE*y#fSSf{WXYfD%*DMaHRgCr`97UZz4c79GKjQ&D#6x zD=XjbDNcWTj~&f49{M;rKubwZA_448&(3;FD2Ixn#>p=z_y)Ri+W-h9I^o%X0U+pA z%aFagd@QQ{QmCu0wYE>ee!;~94!cg!R6@s{Om$r`=b$QW35*66qu`3&!nb-&aa7P8 zM<H(11Cl(HhtokP{nu_?qbWQ=c^p2!nwpyOqiP&J+g$@09u}RIq@=_X!&$Fwn>HPV z)j8YAxZ)b^af5Y9l4DZuFmRDCz%e+5dVj%$WviKKZJQ3!$SAi!+(h{wa_iDKD3F{- zS67s8h{oc62LQb0nrjFeCUjR=o}NAR<m1x7?Cd|DeDX<m1oTM`M0g8ev;fEB0?&9) z4^i%Joeyuez*_)g*IxpV1U>gV2VUqu`3++_!8`sJ;2>JT1QppFMDy@MIP=#65Cuqx z=fwXkCqna+3`8^2V?6jkGJK|M`Z0>jRf$byp@@rTVp7F#r{oX?Y4Hi-?oN{&vc6$! z7QgYqh-0P#xmF4ak2-BZ`Qq14*d5mdI0MB%T#^34@3kzr82LRV@{Iq9x4?eX0z7rY zqQMi#0XP-7y-kyT3EfVMFftZFEaDSgQI=2~BvzVa*^*^PU|vY)X|EGUy_s;j>UO{3 zBz`o;fkJIs+};Sndnf?O-P2)^h2=K_E3WRA#>SHnB`hKr;HF5IeA<z%4s@uo8ng;Z z2a8KFxd9Y~@A3y0x2rK0-%3eJKCN1|zrkR?Q4zWy8*3yzvrF>*<`yEw7lNzf9{_Z8 zr(ArL#32~zR@F9uk*C0NC)Mlhw+{3FU`;dL0#25&ZO3-i2`5XijR!7C!J&_h+A{#p z$MtvcymPzo0BzalUreGnUgzh*UPGNmJD3CBi-dmPV9OM2IEv^4gkF}Im`F~#ZufuP zr5DhN3|TIN@k0(s8nQka3=JB90NZ_oIh%VFe~Og;W_RU}=I7x1S%UIk;6#rk4ivd> zgOrJJNHo{cDjR4%vgv(BX%g#Mtp3WX{G6gGXHxAF*MGYX*6U@&9RYeQCtU4;wPZ5O z1zOK;@uXxEJO&$?@|x|)Q4j%R;tN6{1nMY#WXEw_QGJA}s0Z7L*Oy_1@gZ6EC95(e zAfnRk?Zjn0^!Ds?L7@oJBai(3R)CRfP%D1PoIJjSr?Hqv;C~BG@uU;r3*l)I@iTa` zVLt$WJ^Xa|qv6RJiBmZd?s)v32mc%RhvAtX)7>w@NYq=pV)j?dFlgQ;=tt0wpkHU; zk4T_+9(Zp8l!(Z1<`4c7Gz`QGq=8#A9Scbm&&@pe4lg~>EmXy{O=srA>i}*k9<y8> z83kTg?^iGSUlEW^0iYak@bM$vp2TKmDpCymGOU#ce+@~y^NShN%pI-BA^s=c0{ce` zP)AV3rj6?6ZQCTAs?vx_7WC$xrW?||(Cxa=@M{$LeQZQY3AqrW-a#_pdn}~g#onqI zfGvowOG^j_S<m{zFXq3OKxi5eMYUZWay;M=h{w6#w;egy2!8)0a-%o~l9Q9jlfUKL zii#D%W*HOV4Ea@Z>{6`EuH%NC1gtJ;1SI>V8&+I=CP0Iy02&xh!ALZ(5Yq-I>4Vbs zsQ@EC@0X2bqf(%eiXgj}xpn6b)vjR+8`+4U<P_OawNo`DeSojE^*`Wnwm(VP%`{5F z^8gL=UaFJzigN-uZ9Ix>gyoPZ5={h7yf559y)1g%PM-X3@NMEOLI9F>q6QTfyN~ao zUu<)lG4YfX0L8MgXnAFH8E^=y;s~Ux>K6=4bs=^gnMknqKu}Q{Z#QnF!;@1D9CuI` zEYFX0-r>kkal7BaSn^p1w084oZ8Lcf6rVxnt}r=yG7T3xv04wHAA*!p1;El(VJci$ zlT|z(b6h!~=U6Z?u4E}DfPT_d;Ev?J;rSJ1Wl@W#eTPGm4GqBYCJdw>VKVFXG^(zy zj_-=$2}bgR+|-TVisyd#h`2d(-pw$K#xp@!Ou!;kiWAE`OvdE;BoJ0*D8_O{SKmlZ zcKvtSv}rUC7xm7ixbD#m|L%>9K=D`~)Cm8DthA_wxDn3<xW{mCoe2yx#O}Deqa!eO z|CST+b>j%bC4UYvmLmnEiPsd;N892b!Apc@9)WyBS#WEW%kcY)zWM02_YRtL#>se^ zTXMpW=YRYDa}%xLOhD+T#G)x2?r}A^uBGV)+}~k7jQ@?d!2Z($CIRWiV&{d*AmPBM zEJR?b3n&f#a>H+2SyE8A0o+4RU;>{Vt;)Z#N1@oS=0xn*g$0zvDO#Z4a7XU`id{Hp z^Iu0owXxZrN@H&P+S<H4IX?~QUA;Ta5MIghDNqxlQ)I|)s$4e+wgQoi*^PO(b#^PS zOFmtPXr==0g>HlG?FGj*H`ExvQwZ|FlQ`@*Zg@e-srWXZ1CcH}6)6U{fiPf$192KP z_~E*WiUs|qfgD!^v=%s=w(Z=hk`W*hj^KGsZLKnJK=#CEX3oAHdCrJLzI~ks4$w{} zDe44}6yn;b08~4~tf;B3NMFdhta{uYL4Vlo56~<sIBSov8?UGEB_$=LP@X*0Nxy)G zkU&d#Bm;AIjJnLs%;j~}H5x!~)roFX%o{Sq*2j<oc<$V}<Uou{N6~sXkP!0Xs4Sq( zLP1|?Bcv+krz9nvgcJ2{v6SwT<sJY|*JX{2S}%7O&gpZi{b^}voItw7ZW;uSLFeHN zLyp;T1xAN0kXG7(?=H`7{TUv5m*wQdq*0?sxoB|J002M$Nkl<ZkM^uuwJI`;vy*}% z4)k4eV4>S&GWtcW>u$qQ4kG9uL6o`~VUC199R3XWYvAvLe;)p0_>J%p10#_a)}8uE zVDG;TWZV@XE&zTRqwo~Qag4#ZK3LMFi@~MvLUXHk$MbVuoc+SQc~j=h=^l7xntkoA zd}cZIIeseOLJ&;c^qCGfs3h-&$HR_yA{_7G1OKczyTb@#*9;@lFvDBn_yR|sxyXdZ zh&pgjD=!hwDvAieKzwSYD@$uXe|ID{w*vUh8FuJ+Pf|)PgNibc`V$2VlpilxRzo&! z`gq1E$T#ldEf8-3iU*Jog(XyJ!=_CNm1F}fD!|zML%VKNfarO9cu`SQi4K^YHO658 zZfp}f>~V`504wMx{p|}Ff5iBOs8b6FI!9Fl=OU-jg5kr5Q@SAJ4*Zu#dcEy3!B4x6 z;xCxPL9QzO4onhaQ?k4i#w+YM3>q@Pahmn)Q_qgD+pdugk7XHq4JoM2>#(At6EsPC zi5vq?!ANd!^7Ha^f+6^Tro#f{JCUw)H0D=bTPtnZzFoBofcLg8^^J{Eb4v@h0s8Ms z3ri>VHoaJgkIGJlN-7?p*)A?bO?6Dl%ecw49XU!OxMg`6i{kOP9hJxg%#DHLN=iaX zyDOGhsRPw70ktoZTaDcVGO7T;x3$i3zd+3zT;j52%L*NJY=j~O07*&pc=`n($!5bA z_iZo~0I1(9@7F$Cmh=~mjUkP&(Dp424aZ^5b5sATT}K06aEhwLE+84G^OrcZ2#U_> z+D2fuAkyLTx3m;>eHU%_sVT{%S-*ktp(W8)w$A104v=J+JK%d_-$+H6#~3(T+}UTH z^~%}P&n&@EnFJNUvp`huwGJ5MsThnG>YDVHX1uiJ`8h8=`2346hWvCdWbLBLe-a=b zx1w<7HEvKrI)MX<tByzFp#b;-Z>>0t!Dze}&jRtpkH84cG(`f2RER*IV3WZAW=hP= zy6Jq)FoE)zKX_^ALsq^!XOWq<kumb5^Hv}bo{Nk`GU6ToFfHL^h|TzZZ{5Xz<1G+x z0kZ`lx+gVYv$$pZHfiJ5E%L7VdKp%%Cu_R#CB(TOr<KcSXshRE&n`tdlE`i2#1Yt{ zOGrtHdPVGZ>lffs_!I=VBSHvZp~w;(=ph%w3INGt$W>vtUJP#m$xuq)FGX#E(|ycu z6LMv?o2P@rP1d4b%m?c@SK}VpK!+6!8wD}V*8$?Vo$5CfU}SC{tdM&scyT_!$e7eg zW&48&A_C{n0H;h4q0Jzq!Kkm)8jC?k>=JNiO|^`E1)1!Bj(W_xyzYOuhJ3ga9ffQN z0O{lkr&_9*bskyX!NQYM*@?5%33v_TKqv9Li-U$e93P=Z0Kl#b9B?$yTD9G{Q59a6 zwnN2>BYMFC*kMLFEC-mOYA4ZGlDv7%nl&|{O0%!iak-RFSq7(bF}`^c8e_sy4~{FQ zi#gAB@(F=Mi!@1!YJe+<itSKcs-pvDYg|EBM^uk1O4xzBn&`}nL9|Do2H6*EPdW3y zz|ZKRghTP3yGLg`&mIyF<~2DSUwiGsQXFiSYKC5P=9$xn`u!RdfoplH%T=D7oHW|! zYZoFfP?iAP%S67g$O#Y(Fn=DbhhGZ=;@6&i&Usa>EzOH6DmJ{fa^-Rkwler03G-me z*_$v|&iv~2$1fVU3VaKrMWI3cn7AJ5fD4#3H6I5uAyI18(*h;IU)>SM0uC9RQ9Hu( zvlS2V*Gw8F&y2&d0o#rTgP5mq;vMo}8^VeB0%Sy}z&o=*hE)FLN8?7HbY_F<a;Iku z9d>k5TGqDK#$EYl`8(xj^$UJc)+;}{?RyWNc;~yZO-P8=iT{ao3s~Q`^?gP<|48IV z17o#yyJ)Rh9e@oejGbK8s!QjV78c(*L^|++`IYngY~I2)R-Ed#xBDD51$A;^s;OAH za^)^Z2}F4$h)yiVnlp)dBndLxaXdArgH-jQ`1!B63DKYDX3ZLgq!PV>JnSZ$XzJtH z1#UvFoBhvGoXjw;OGvP<2t6<sr~s6J<a5#7IFRn)N4ZbdiP43l2enBpZ$mR?3iTU0 z(L=>~dQeWT0dO9w!_>>yRhGRF&FvEHW#fF3my@f5`&!A+(zMOnwmRw;){`7MJF2VH z0|w>iN`C)rtff=YUnap9?G0hGe4~nr3LtHe>SW&Ih7aumYqQF?HnWYY;4QK&w8UU- za#SwtxkNmeP~o2N3rC6UEs`vaYxhDAVqmYaw;<g;!y4;HZNBO5eGEOXgC00Gux+&1 zuLU@M4LPcT6LP9yIan1#ebQr52ju2{S+#Y$AM2aTiMTy!e7M}hii<OrmY0k1*<+?b z20&Yn1s;NBTiX{7wAa+2nt>bl9icAMUNZ~pL!}-lhW*oJlnL!7N(TCUa9E4j%fz8F zfGNnbTYaUMNrgRnPaoo+Py&ByYU<SF<di~KU`v5M3r}`-w)=n~LtVp$57$dehvncz zZB}|((od&Oo!Stp5@YdEUVi!Ifz8eB`Cos%V${~{TShh1Hx@NDHVkcOY8uqs)HDD- zwXv}=;oNiG;OghP6B5UhV2Yp9(~$sM56_d1x8T{SLIfGV5pyI2|9#QH_eh5JG>Ld~ zaRvzl;-T|8&37y^|LtxN7{3$G&Ei^d%tr{-DFG3mH5Sjz6mSUz1jbi*I!L4-07qV# z;^HRoX<l2;_$#1BQ5=2H%D10?=aADbgl;k^am3{F*MIfO!}$!t*H(S@_>~Mdq-XBN z>*?@u7jFSuEs!@bht@)=Xx&ykr-3;h#W=t>4P#JZGsg`bJvZt+O|yQT5Z3!D7Z*l~ zaYM{b&d$oh`AgE>o21ICMwOIITU}Q6^}gbU>m8b49MU$ZXT?b#5KltK$uwxzKb_E% zWXL=@w`+ls{S7+O4>6$ep>7<ISiJ(0ss9i2c<45xp{`&r(=!xhF$}fw#M!p4InE2% zVCZP0sOOL*u!X6tuDj-f-lSA-Q}dgc%MO5kLnW|r>Mr<RpL4w@7)f1Jpk%HF7&&vV z0pkyYY5@Xc`MG%-_2vOkz8B_6ew8~gN9go)y5^93re@bJ1?$T9p%&!VD9E3KDol(8 z!nU9574+xZKR{!9(}4nBBQ9d}4-S#MQ2*iu$8v&`Dp<gmrszYQ#NS&Se77>nG{)=m zIie%Tb&Y<BX`q<xf;!YY;mT)R>!3)Nji?1AxfTFC#%V`vk0G{qXSaCq;?~lF!j+&G zxaQmbhApn(dR-ZX_=4NAbNaox#(w1x9KiI10qr#l&@d$ISMT*@3_V00>akd=7WQDL z=+?LbRuZ-2iuD0*$DbO^?@_;>>gi)7E5==qKRNSy%<2PJt1gbn@<4pp=QK37*5dl= zF7Nla#p0sOo%?3C?rO_}F_S@tF6T8jH4fgmVZ-np+qV~j@X2qes~cEfU!Oj5)TqRo zni{#brbcY)_ef|TH&{${PeMXF7NS;Fbx~Zq8OAM}aeCJTk*U`9b}wOV)|*_NS?~lP ziEwu}S@1czM(8huCy`EZ_WR)@>Pm~Vo-Zq}y#wu7$N@x$e!+M$iy;OTmo8R?&pO@( z0^pH#CE|<72#*8^rtE&k6wHv;GXy;roefY@J{US4JK>ec1phk{Hj85Z6=g6Bv0>%M zhmAeuC$*|OVL)o`&<V-ec^g`4b`+R-BRaxbWiYQr{%3t~^C?H2^rv?|>zr)-I^F_3 zwE(S(VT0CyIbdZ^VSD=?X9ckS$oWKu0zHFN6(kv?41D=1dr8|MC;>tM9NRvKOSB1* z38x#z&gqkL4m#W<>P^B%bZEO@`mCg|<jk_lG9IKv)L{%C?DN6urmovQ(dV4tf*b{u z*p)@J&=JfG=EGMoI}qXEL)#}K+FnZqQBHi`?!)%_vKJ3A>yG1$?i5I-kyfk)>v~ay zsE%4zR<;R>&>H|kq&q1n$GzL*2|1ALq%8Z213HRln0pnSqj7{Um;66_*8yNfasFpt zxylueD}dlNst6h@STTt)F|kojtXQMQ6is4cqDB&9Y>6>38oQt<9lJ69V?<*I1W}}3 zp&s0E$MyQ&?*I3j-8~MtgSYeE-Mu?@U~YG3=KE&m+qYAG^UXI)smXX$*{)rhihV<V z<C*E{Y^%7v<CcxZMUNh+JZ{{9B|tL*1S8S!wRnYR%KDO`H;3osl>qGhC%11?lU~(9 z1799gg;Q1U9-fo)-iDHrzBeAz#aPP6);@P4E2%960>AM0YCw{8M%!P5PoHxl$Ij9i z!U64KsAJ+z2u6+^nTx0dmG{CiAlex}I;39zULb>N)Wz{wielxWI4_uUM=fx|#S}O! zgNeTIu2`2nPSd``cwrm@9KSyRvOrD08<50l4iN<;VS)qN8o~T#g#{Q^ECzeg(N^?@ zcO@5H(dM;jJnngM2fkx_2h0K=mnW(<B(ZCWN&+7pxHi!ddoV!KDy8RXv}ar$?bPAL zTt8?2A|m)f4^Flg9wSS~7GIun4Y+Gyb8~anU3cG;xv8YYyKUQ6v2EKn_^pC~W2g?3 zx0Zy2gxzs*@n!xP|5mKx7s15M#$CHA)-*TQZw8<3Xnd=iFu|jD;1ZDbc=+k?SHMd& z4A+iaDgyrkPuQ6neBI{Gn6V^)ZQBnZ!$;Z*h#SrT3Fgr;9AkX|h8EChT{!_DipW$< zBZnTik-#IbCb^6?<7Fbr$GnqKabUn7H^2f5_++FcAOn-Tm6e$V9*`~53|RTbbN@c} z{A<s`qi@8wFWdIT{J&(&63KX3607WF3UcJ38v(A!pDqL3KduBENC3aPFAFynpW=Wd zqDeV%$Cl$%s9g*W3UN3G>iA6$!|LK#v}H$&1R;&T<68}4Qtzp2Xwd5G8?c2Jsl#y8 zxZ8%UdIK}F;$iCLjbXXDKZRYkN5Wv~m=xdj2n?ni55Sm_H8_QpKE23s#=V39tCd@! z`Oy^P^ULK+r^P0iofvNnoiJfSJXhhJ(sz~~-^?(q{9+f19^%+w4=27i^bJt0(>Cr2 z?SprQwjSRv!Ql2?Wk?m38dwQN1||f__zgsCM@dMThnUE+A;<T)t0R&XpDlX`;b&Lu zt`v0*^&KTSU>L_D83P80ehG=}@9#w0MAJa7Q<MM~prC0Q#8Fa}FaY?y^@WAY$!bGM z(PL<1&gh?#G8O%c=2R2gE)K_%v?n=q&|nSgMDvf%Ir^xTB`f7b&Q8<YujrS@>WbSJ zEe*-0X;)0!JY&X;-Y$wNRX~!z6POVRdOY5alvkAG5^M<)$iWbbx-rh2T<il0W-7fO zI5A8>l5HpD?8GrW`g0wo$vAh7N{5rs&2o9fmZ&UX7=^+s+kkJ%t74uf(29*haCqZ6 z@+Lxmw174m_>4~l=$GSjO~cA_?>12(MTzf<SNU!LFgkT;?{v-UNpVWoUFj{w94gH8 zibcebCLJCT_kgZ7-iU^cz+Gn!16boZMg}mU-m+=4SigRqSig3iC@L-#rQ5e3v>}DY zM-?XPTd+1?gbrvuJSU_C8l{J3Q4O>w1$~4aw_v|#`9ZjTHw69>+m;YQUTsOdp0sCz zM#b+4^vY&R#+cwB8a@;JLl`?QI$q6i6QCq?X+nRDglFG|U#p$ohAibXJ3fmK6C%|} zkH>fqyC_ZvmF38QZcjdlH$!B3_QY>lv;4%PzISPvFE&0iF*S2Sa^~UdYsxl{U^&b( zn#F0$(&TyCJGXuFoNwLz-`6Bl_m3-qULpbO@Di@L(C??IIePTyL4n5RiCDUx7$28# zHcqY^l94t*$LXe?s+wvgW7D>1xK$ko-)LDG8CpVI{6F$@az|Wt#q^soVUo+`cDk;+ zYJ=OhG&szp?L<ON>$89)+E*h#JEstnCfEda?(FSxWo2d5dWaYRh$jEAU@)KrB=31| zMxpZ>^&SdzSL^WJsO9KbPLQo#QH~$JwjZMDikXvH9L*@(w_N$Txj!*<<Ht31wW6v< z4Npz)pC*zL`=QhK+`qA;=nwnm;Xa2hfuw%@MCO2W9RsK$Uu?`->(;F+ZcpuvjkzY! z)O;*7=5wSyNktPT)CQqF$-!9zht_IUPa}`X?eppN_l7pSCD0nud}!agqAh?jzotF^ z?5y|mvvc>2?p4pwq8qHbHJBBZV3}sbN37C@o=;vXCLyrcXB9*LJ>JVNBUYxKV5Es5 zoU5s5a<RBL2LNZhN+p_}DE&e6Byd6plPIMJP29^8Xod&lv!L){H%!ps0W#A36BHI; znGN&(2s%$NA4O98uG&mg7}w*wni9cMbvr98L}l3Dl}hIPuJFrMQYui%&nb<0P!tEM zp3j}q_C)%SI^cqWv&-i#SokPp-)W=1ZR=LCYSkBF<%-W6)~;RS1DMDy(Ve;Nh@gNt zNvHn`9@>tr*jn9++a7V*$n_@p?;Ip|B5X8B|LcRK>yqdzXD(@a>hiIFMbmvRGmyX` zZ}4&mHSX&qxQ7z+%2*cObhwk@4BbOVe)Jqz><J}>`4luXtXLYA=-krwFk~F^mxtz< z8j)?~LLylPa?nKT=dV5U?-Qoncoz6;Bffn})v_09rqnE#ovwXJQ4FC4#XZK18E<Ia z2Byd&xqn;<bg2Z^tXads7u%9K{2oRP8+H;@(=P6pkT5+FMth;vqVKBQ?Qwd_s$FO- z-iaM$Wr6?z4!rp(CQMw(`JEl#Ro8Lj#wFHQ*Ce$AUmRtH?b<Ru+L+Cb2zyy#w2W&3 z;*CVHL+XdySC5CY(P=@Az!nC)k#YtPgxkC650SO~-Y`Hb36Z3CCew`dSW(A+=<u4` z;X<y54bodDo<=DkX-*DNsl5Qn5!rcXqkZ~oV^gECtFlt*H!{F%i#RHH1Yl&3jG;n) zSy7&lTd~tprP9+Uafu1%tX{pka^E5Ypsa=M$H@SHKZRXoZTrq0N{54xZ(~yvwtiP> z$a4~a=^qys7yo15{Ce-RahQE5i_<zZG~zfIO^XHH$xdmbF@1d_^)u955CeAf+YtjK zp>ID3n<8;;mJYVg98ah>%}H?1()%431)TObcC8KAM}r}ph`kq^Z=|mF*a6^vX{2~Y z3m?s6Y$W){1zYSCbzCgGcf;$U0#f80fHb+y*dC8K;mpMW_62D3ICiD))lS;>R$Qp# zybYC)+V5(Ql7P3kH@u{fCqjO&=YbZ(_z_-!)RM$itXsEMeERXnV%hSgVpn;&UEhwh zC`%CeFZlQ2KZ1{ji6%$g$G{VSJO};+_yY~HsSYp?{$Y56s6FN~e94bi1nX|aJ4lrX zBTC%{0PaEctS};a{7>CT6J%r(-nFl+Btuf7hIHiD7G?&32vnNUAG`A2$Ng#Mvjtuv zF@F;#C@ERUXS-a8lKe_Hesxl7b3>^=HerxIG3A8*hmZWSYRmeuq%h03Cx4R*pN%m~ zTK{($Zf2Qb?q62|eM<uPPWbhGNRKUXx7CT78{%T)ZW%IYP%00rp$#yV!6?5;9(V)z zgj*zUACaB^@%l}LY>T_hwbs=s4|Z560zGtw-%ribKI_o3ZHH!n<B1jpVrxhOkt84) zIfrvbIPp<&ZB>bh>2RyB3eTvd(KK%N5Y*~$`c1Zh9)>4s?Ri;SwqeF)kA@_+n2*t0 z1M_LqYDalFqS#zHa|F;*lao0PcxFR!(NFBLfio^{YRWA=ZGcFIb|gk=3x{agmn~Vn zq^UE>tS>J90%kjI#364_5x|74_Rf+XG@P3IuuG|9oA_Ub=M4W~L&=7f2PKN~1lTp; znf>gnbk%l`n!>8u!ci_7$Hg;&FRi0FpkrPnKsz&7hXoN_9ru%5$lbPMhf=<MjR^AN zL@H`ExhxGy{m3JaEW<*5pAx^}^OcHViySYgR1EEs7ieZg0ZDY=gV2K;;LYEKBgfrW zSlDtz(^nHm`*&rTzY+rm4cH_~qW>*~+{jOwG^q@T(LZr0jCkXX*ThXfy;1!B&O5|g zZ@t+bjBH=yO?db%{1NcE@IQi|3%?mYV(yh!UP+p_V9~vE=P#NIKY#xGMdx<FoF82t z`~~oL!M_d9)#T6M{|J8~d`BN9)69mS1kVQHhwwcLMk0T4-ZLN8;>BTgngAgJ+D#7Z zLB0$}Je>_a^2&I8M6$BH+D_h@L<lnDNiI{EnMhibzi|o@$zH?cVTz7%HVg8z^JN}( zI5;LPd*$gi84jB`&6@o(Pm`W$NpBWUmVjmc{puN0W5~;Ot_1qB1ZbZN+C%sF<Ku?o zi+Cm_Dakl&;GiH@kCm%k4qLrk*$j%{+)+8Ba+Dc`K-jA55|FgD_(CArD~Sbc8Aly; zl={qhFOE_A;1T6$Gl9Q`+zq!?q-|x}kQ$P0;o;nj9Xqx^8vNKc-VT{TX;4yN5Y)E; zNyCt<uQqV!L`*i2j8*yQ1thKQb6yd*lXV+4YSaLuHSjvLl;WUq?R5&m+Vbg_n24># zqA9`a{SAKiX2)(vy4R2-z_x-XlS?pC6AxW_<@5`elynY8+Du`m^J(-YFQg`?7zy#} zcoD~YILOF@Ho^=0-V1ozx}8UFy)K+JD+7z+KGk-wx7NEW%8lO#JF|#s4B<CBxHTAb z-a&Rn<$5tsycbT=BKBS)Eee=v<0P@(GIJDZ_C3lE<96lo0t-&i5Y4U}Q%9;Im~9V6 zKMsXvw;K*erq*{AwLrhmhl9O<bw$$7d^qwt7_Fm*{^7qrxbWAv{8FF&+$;gxRh{zA z&C4TvPF3D};3aU?Y2q*=&O7hCTI>;ct+}b`WE_$+b;ZgRubg+mr1yNjnESzc9{dXU zI`~5P#qhVop9!A~zptY{^%D3y;R(8B!c#NvCHPKPpb^*YE+*FZH`scqCwON9fhL$o z6puX*o~(cJ0e=wA2soy<uX|{%NHd0sq?ecWVe%g5VtPa?j^U*39GR5un%%hZDzj+M z(xj7RDl1xja!W(qRtCYI<k9K*M=iI@Wb$dt-mC+dAw0V*P<4aF#=W=_=<^a-yLN5G z#!bc7XddmnSbt385P*`<a#5-`L4+OUWz=ly*J?B^mZ$6~aXL7WM|f$67g~I3nsyS{ z(nM`v9GvbYJwr1Gs<uJ*y&@0%Ky4mOqIGyF>jS3_Ew-_-QSwsVYHtC_4cI!`5?YSE zGnL`>TJO!#@JUvd-yyz19bMzVXsx}i>6$nov25bRiM~Kn(?XQ>aI_!Z)-X)!U{M5i z#zjnhFgO!3=wNgZtrCfja`E5?*l?tFBF0^wn>Ut}II;<xkdkyA22>le($j;s=7dVk z2&$LCo+L(m`OS3=kE!s8CYmPQYc8NE_E*tVm!2*QhyL*9%2Wgj175Ic>;jVEiZasg zCEW7tOE2w~8V2X-ea>+_gwC$s?pK{w&^qKY!ztZ?(xW`E@g8tKfH@G;Lolv4d{=;G zYJ1D833<ylEA-)qA5xp}pg-c`q9THjoH-NVi<ILBnGJt<yQ{pvaPnk^r^DOsG~Cc1 z$mxlR{VFpuGqz+Op1p0<5u>*scl?CS$%*|M3{8h2OKl8Z<Zu9G0R8kDukcmDj0sTK z^c@8SCp8|3ad*UB{KWEIGp8T*1ZJc+oB5%%_-_Myb{L;cfN<~ezXS|1I+aAAGZ94l zVy7jz33~jKeDU?snfGEGrW7CGlqmqWCMd~kTfDs6^T#d+^Fyk{r7u76;mH^Od<awI zoqRz;MbQ^_5%v)A4vE{!uN$|$J7en8XUtg8Cp)gaEABqI5-=qI?UvU@=H?!UasGb> zWo8Y+b_p--BRO^4Zi&-UJ+LMA4Wl)98|Z)Mlt(D(vA;^H-~e|CCHa17c;X$JiK_h~ z`dR>#jaVht7afjs2Q@n^3?(`z?3e<oZwUcJ>F88coTcg6x3yX!m>6hC^6Rq=@jI3k zKMc<S4+wP3GLi(A=4a#b<;xp}W#|3_zlMo4`#`%<;W*lJ!**fMfd2izH!LUbx8QYm zIQ$MJTAt*Lwm<0e`vUmRF4<62w8&}wSFKuAkA<j79wV?UTzir{u^ZdBHE3L3g6-h% zQUlXzd7>pf4mQX_f1|vB7A^JYi{-%Zf1v8ZW?z&?U#*iIJ75vIuhtLE%P#b*N@F!H zJHx4IB6K29Oyvi;Kz}E~=|jO#W?^l54PN31kVM5o-HLSv(r$-;7oOX5H^X!LGl4vU zlpN3VE8`Yi?nC6qMvX;U=<aWMYHJK`=aoLl#Kc6AnVBWBvIdIGtV}WZu*1aAp@)kh zLxy-_Vqytm(tiV9I@nJ(OD2nFf?+JEE;v=A+b-tJUsMK|ZX-^*{t}?v(ipFJck<-P zMP_{W{f+f{{@i2)e}tmO5;(N*Kor0P5N#y{2_UvBRvWk|Nl+7BO@z`)Wdo5iLb8>> zBz9RL#LS06F~kULIIKZ~X=<A?OeSEWJ;``)gNnS;Rc_gEYI9xnrr3Td*`Ao#(V3&h ze^R#glkqlH`}(w%O%oZSW%n;Y{(H&9{o_jD5Rt&T;^K9q^YXsgqIo{R#B{)x((PVq z$vJg(_4PdA)<{T*zX_AmKX2T)QSGSQ17|F^Fz^IA6(^e_Z0%H`?Nu}$qn6!`1>`v{ zy499OY`X~f$t_)W8|}0}W;?C*_+p*Yayw1$H4OR9sGdg4gCEdMM(y@F6KK^cM&#s7 zM_F&IsjU_Ck#Mv@7#A07B*rKFept@14>y)<ToMVseVGSrHgcPG5bZ=Qv?PD9uCVY! z)%v2nUp+i0ANGl3p9L^1a5#}FD;{toVCPr|uZM?c5C3SxrVUg?=yeXK{3LMcOmzXR z*HyT86*$g)FTLx9sg%~7lWLLaWK;^!^)z)BIOI4dYaH#JI(4cyQoDvQ>rYf&cy?R? zm!QWafL*M_cKR$J$t|<gkemYF4tnZP-dqj%f#o|)5Q)H~q-2qiks;DE(s9ic>FK=B z5E-~a`%&Pq+WoS@JHn(%YsS0)TVFW+0@7v}ad>eFcsf|UIC9CvR2$p5jgtV%UDpDv za_;<vD>cLOXny|TGmks&I3<Ob{R(=JfQz14v3=$hM?H(eUoW*7t@Lc%Wv%4Q9yPU| z8O?k9D&Yb=bB_h;BE$JwWg0x1NlkFFEuMi^94?u-e-A;Yn@mj>1QS^v+sPb}$Se~> z8870YCAt2TOK-_$NcQ+2q?fJ#OxJYSps~~PC&sq}4I<Lg<Hz@lcW0dQ$Qd(U3l$tE zd)K)V==Bm<Q&hBJWM1C)HLvz@W_r2-dv3Dza$2$7)zuod{UwQj{v*)d>y-Xr>G7q8 z1#HQN$52xmJXqPHDIV<^7v|v1Hrc!n{jWUHR5N8AI#uT>(I!i9vwhp$dFuUR*zTjj zw(7Khd|wIC`t@r8LTB}l&TGYf3@&0ovhxxgn1TQ~dgyp8cRb$I+^p}~UCA+D#9aBV zU0Up*!NTVk^Ty<iIcjyu>Kzed-p3dvCM1Zgj0|k?*S7jR+F1Z2*M(cf4JCy$q4|1B zN>cJw(0bMC8`Mr!=I?mtu5!=NL4#w2r*+|k2@{U%Ew-xK%h2#*aR5Jd>$_|p<=*$8 z1PH)<{@GHW`(AqRC3I8P*1WV2FG6cg=J-fOpmMx`Lxk6H%uuhwVP6`K_1y1jU#axP zXW}kx+H_H0L8ls$+>TWMKOCM1#GD3C?LulYM%?woDT4j`r-}Y){mm;@bO#JbZvz|& zJjRA<Q%&=1@cOk&r=EZQGBkc%5Z%w=1Jz!C{q^|9Mh`UpS_bNxk%!lLG*&3S0|f2F zeF>Vdg{c99IS$}a)}vvU=ZTGlo36(}J7>eT<<5Q6cf~VKKfkxtFs>H5;kU3tnT^%J zn^Q#mA{l2-eC4!}PcjTm6K>i?OL{YaQT#!ZUbFV<Bc)}Jo>TxwP8LCZBP?SLGYN@T znhhxb%5s3-=E1LnDVXQ(ZH1>b)K(THB&Fqfyxw6$#-8@!wl9{POqI>H^0(zr?kEcm z#nA$dzs1AP8R9xu0*8_W))f^k%g@feH6bqk&p0GP)Yds){NmP0`dFOejY;I+x?mD% z0$cK1cm8&))+mzZKAab#TPS>}V}MPel4E}b!iZ_wp;eaeO<l89zPUdC-eO{6nhD|9 zeU#Ie$M=R87qR!|H2;I8hlXSjzl8xflq$Hhyc`N-?6V{1a5xT6>h=y9G$_4UtDghl z;oFhY{D@pBqk|K+L}T3L*&gQ+Cw^629I-KF#^8ZJDcfE;K660&sI4uf%C<oRIB^wb zK)lf4JfeL2j=w`{*G7lr9Fzje2u!Wo?m5WGP3k}b762pX4i3oNq^<;_P6C*@Hit1@ zkoCciDDD!C2p|bQE+83ozY@1VP60_<VjFOCzvUT5n_=1T&>ECmp!?xU)x=nUb-tn5 z*+T~o9GI7!lyZ2#q@*Fx&gst+V_^>mr`h|3KgKT-p~;wl2{!CH!Y9ZpP6(<7IJE_Q zHX9yeGjOo2&_XOVJX$G6$+ba4{~zK)bA~=#Z`IzNyI}E+Aeh+^5gH7RygApP-vPf8 zelh&-9(eF?=WSTOZsAv7uJUc(v<Zj$>=Z5S?6?t>95`r@7&rE)V@8Y`wE#2|;X4FA z5gsvj$<r$~&ARgFSpe#;Az&u~KTLOI<3}J6Ikj&H?YS>!MApP#reR9?ED0<c-oxAU zR+L$G=D|C|@Zo)6>mli7@;&J}k)dZ=ds4%U_k_nTUHo{_>C=CcM^=Nr@$Dg{U%elI zD!Gq&+7ls}54nTRs@Dyl%=q4e<7fP9E9qS4O5jkEz-3oX|LgP5&blHkwf|^r!Nv|! zewof3v?O7ZEH15V$BwH(`44B*2TdIwAj$SZi>UqKgBG|)5*TJ3?e^7a8Svg|AtsW* zR@m<i&2*m!B$JYoTB>$cwDof;?+r9k)=p`=BR$&`ZWFGo+U>PZ1&?&T%`MH?E?nVB zPa80C<+5eJ2mjk6okw5cht_>lB*93^g^N%laz;!Mo|fg%z>M3rgF|ZP{Rnwc!?L=z zMnL28nh`nq@2oE=oXhZD<icwAdQm-lRZ%g&lJCU@ihI>-a|_#%1TZebIFERM<J|^! zyNm<zl+4z3R2+oG+I%~ScY|FCbh`vp0m*#}N&t`cW26n4`SV{HW*EV<p<(zfOqfo? z4?k&Ng7(jteAWSKT8?Sxav)c0JchQ`=M!rtO`25EK3)6!d5adE=2Q_C<sfkS?A-Z_ zPEU-Fx$&HH&SAMjajC!h#Z6CxzEgSe^vvzswn_iiTW_8UrkmlPgP#Sz4!+Br=J(zu z0>)*KUo2!P@jjUuC-%5u7=c9&H|c3tfB>X9{Iozmq?D;i&lvo-dD-D6V>`Xc%jB<7 zOZH=sOu$t-Hg*<xh=lE#ft$?VOjo;O^Jxv$yEY`Gqz{LdWcH9_&ivozWpAHq)61e; z1LZvvcP*|t5a0yndXtO$*OkDbBY_z+X6Pfb^L~v}HQq?>mn0{HPHWFY32-ou9v2hC z1)7KglK4*j{C??G9Gf7t9=6e56&AgM#^==ot=K9k+t(;4v8Lm_$+lnZy|D^V{zHxW zwp+4fNgzKv2bE;+q~a)F*(<9McR9`O>MF6jy34acyHyK&l5wJ6LgH_S<>dSir$+y~ z>vMH;uqCj*Wc}LV*?HH-#>Bjkj-5z6xJcFEup-l*WK8UnuuZ%awkiu%d384(j+k-* z$?g_<2>D@7<-V5?<kVM^U{<Qyp+>z3FfLM7PO^v50O&i#ADjvRW6l^G0dPv|rgtUK zRT6NF)^}Er`SV{pe9pW@cfgk7>Y&lO0f)o<4YQN)V^Z1?7+Hf4;$>__xxw#^Ir74( zlT$7#m^^V>!K7bIn>zWaf`UmO90-hL8H}Jo5KPITOjp!5w0sKSbmYMpEL`}?adYP_ zy8Rcw{B`{3BgPyodHeXo@bDY>weTOq6FlRgJrQ@8Ke2o>TJp!qn2pyS;Le&d$@z~7 zMB39Jrr~}2w2~eZQafzVv<Y`MUz-6Ev-gl`k*UY$%?8lfU_ZC@EM_KT&BU^RR&l^m zHi<FIUwV9#nE*%X9eG;5#~0H~(3F*7T4sd*=DmEDaaa9s#`p91=sH&dhnfV|Zz_5d zU%{H>q<+c+9cZ?A@O_Pf9k+pzlphq$<+c)PjVn3xdsnWPQl33P)T(Jb)Xi)K74k=0 z%kvjQT~s@04j41b_hz^M2PNnr0p7jhOYeYLlm;Dp?6H0pO&M@EcVaY0Y4y6bSmhNJ zqBYP8i!b6u09ix2HFGyJv0+ouix^xzmYkGCU|3lq{1#|WQaAK?qcyN#%$PB<ZBu60 zjf5TzRwP{)brSHlwweHpoCHK&@!SGMPy%c}F!Ss+ZHD8jYdFByG_m8H)S7e0>w3iO zb3@v&&FZ))E`ck7E|oyoi2G}=y_R0z*z%w*nlDC4v3>9W<<+#8a6-u=1JhIAJL8Nq z!qR}jS_%)8QFizskJLr$vU&4f`r(BaUNDD@&?sv2UV3Rb3|yaV2>o=SYiG6wTcv%> zV~;yd(3h5$ivPazf8zc3mWY~~kPoOG5BBsA!M_4O6aH=Zh-s6=C3^nYTraM`;4+!u zokZ|>wQtzq%Za{>mFONPp0XQY8Y@^sB6%wtK?vAt3<(Z0k(JEkOnQWyAfy<K>$`Y9 z!-~WOG$xs(lzCbFWU!<cqN;4$=?%MgtxxEmJ^}_g2WKDuz4uB!dG+)?6JzAc{~$8U z47Qr^>%rDLk;!zKpZmv^z#%6A&r>n}n7=^7h1!meYu^Yvl60-2X(BFjc^?UfW)y}k z_bhBU=gC$EdMkb0X#uWMqo%TLbzWI4+8FqU1KJ|v03g=|gSzsm-&QI3^7^!EZEdwC zPJDaL9ePm`_6b0e{c&n?vamF+9SegyOQb!?frC;#fxv=^6DOWZ4OGXBTmoeY3>7&y zZ_!Fl%1BQi(_GpD97yd<M$*xqWLA2{H=61j9zcP9rd+7*C6pcEp7CbrnUe-v-i6oW zVw#j1$0Ct(evhZIMGu<u3k9aMzsf_QTo>8*(lzo#m#B%@uHhXOT;t3cxbw1DQ)b~a zQX)5IFt?B^$vX*WdvIr#>w3d?)w|wOSb!uBZ^?qD;Zl5PLl0JMP4hee4ai>|EM2Ja z7}6i9HXYcL1LlGK3=w>B?!1M+L9e%a-h##BFga|e#khYCgJ37}l1nca7hQD8`qy87 zZQbjyznWEBQ*%7HhB6Uj15jZ3Qe0=l&xHRPJ}g)8@QU*1t~~l~%_HuU^IO@zgQYAL zzyVqtqhmOMK#3HZ_cDx8R-8;^$D1VPpnxqS5=nZQiA9ezveDGElSI(igokX4nGD|} zMY9Ziwu)ua8HRu9%a3n9{mR=$AWEc<KK_U;%ipRGwzee7bXLAh+*WRKFvKMv-1Xf* zopQ%p>qzH1R|1EU1iW$nxq+tUza+#bC;^hq&CLYEjG$&rgxn$rNNPqMn-6M8!VtI0 zD?zE$x45We*dsI?fJ08GtWx$;i(H6^0g}4zfrg}xX4Y({ofk^B?5f1hv+E%W%ecJq zdxL!0jX7UI=%M;{maVj#osnt4-lPWmfx#`M+r28^J8f}eyRZibtc>4UQbOD0?tyik z3B3!#my|4N%FoZA>^1Zi1Jg4SwuWI(681+$d|W&xto?q9ZTSDiHheC?N43*6Eb3U} zTBil|d3_=g{Vy85B8-eVra$Bz7F=G8gb=XZN*O3-{y6l{h>MM_!>e5tiSqPz9+7%4 zT_Fc^2xLSK&F+dqhQ-r>%|<l__INCQP<*o}QRA^6l_8qImw1F@$Mz0=ehcce)G=`% zB#7qks<(+dEI_icxpgYOv4et<6beS*N6er9Zoh(pGn^j_+o4oFuVV6aM2Arh15t;u z9FTP8ILIz9FW**NSh)Jj)vLCyTDh{eqN0MpWe$86Jc*r7#0G)p7RxWgbte2A_)2&s z7neQ!q2|pSr(?bYxv)3{4|%bQhEm#%o4m5rGR(|Qf|is6h}u`C?H+M^(VJve7@29Y zKCfqIAgE`P0qHpNA~zFAGy|o6h(*i%>1xWMC0SLzHX$`*6wdR`$Qgg`lA;e5O_T+c zWit8Me@*6w*J#mafczqO*SQinq$Ge-D7WLp?Tr{5<iqA$+h-c85D0`dH8*Rp<My3U zX+lZo(Y1Q+!L$IgP)H8VG+Gebc+kRGZJ=Wzn2mA707<{ssB3|4n(9`2)&suJs|wl1 z<--D!H6wD`4vv2~;S>10UL1;*rQ^hLE_VLaCwxDJW}H!4wiD;qIsZY|*42sl*f;_0 zi<|Rv^4~8kDSSEnQo0;Ml|W%(;l^RPxmV$R&QDK+JxOe(S9KM;D`8JEHW=gg&m5DN zchu^lqRpzjx`PfKXA^(1_Sn`*p4g%;lj9qBY-4dTwL)DNAqk|Uq%?+eco^DgZsFQk zxT)ln2o=d?(?t@HeTLD;H6#`KMT*;Qm(DuttY#cW7O6MTgz<Tb3N!9%!27Cqzbh5# zxikloKv;mJ@M<4vL0EI_U=q^Q)F`&$keA(+m7)%VO`NLu{n*&poj2V0<Il#99ryNt z^t7)u-Md})i>i2R6~iQ`_6I-s!CsTA1q;6R3$dkNFs3O{)AfEn-IE*?flLwfWME5m zHO{Mg42Jl}qJkr^3o0Gua6Boea0ob7RO}MvW#s@K%djnbd+nCZn`^e0ZVy!L-kktV zqBJC<O-TAkIF9y;;5WgSz;nPx&wP{t$$}XRT3)#3n9ITALl_r}m9U~UH;|y9RLro2 zL{Z^hB9`X82@aZIq#U3jh3)J{Q38bR#N1w)#^QJaIDCc%>xA}O#7(Z?W;3>~vRvk@ z+`nwmW7|%<`i@b|v;VMf9PMBFMs;gbW6GYq83RIQDlZssXrkbQKb-Z)Q-1gF)r@tW zD}h5v0{9Z%#pEv!--r?r(Ad<()y~|JBS#J<z!?eI86IdzVx9)AcqMjdJ1|x<)D4;` z<=P`eQXAi%)H*EzCc)UTtwqj;WYCYr!k{vcEGHFRWGC6FY`QXiurVkH@;WVesyMUL z?6rU-fF^>GL7(3j@ED$pHxw5w9-fo43GK+=`lqDebnYq@sfb+wvM@#r0^^*8_ULG| zN5x8vTp}e208m>nJUjPcY{&Zv`vqz{syI$qcb1oXhYT8&+#J;BVjKQxQN9gd&O7!V zXNFdD)|C#I<j^SGRG~^>`SRthI6N$fZcb^D%X&Amj}0|XH(k$2z#2Qm_e}0xpDU)` z=|PwuP;qRR@Gro_ELG?`p`nIkCs79u=6Dt5arbN=SWIrLf&{_>B&SWD{Q10v3(v*m z^Pd0%53k1n=B<CdDL(t`69GVvDEhudqCt;|7<5jYe&to-j5EF^xN)h$GNp*qulG%F zK4a)kPOY5fN{1^0;7!dt^WKcx|8F1=0N_|5!Ny(X<)W;tOzhfaf{z3tD*>7sdz&od z#}>{9w;S{X7>nSG;Y;B8Txi`#NJ|vl_Hmay_4)D_uRZ2+?9hGz@|W+I%^4*n8z9OJ zk20*SJ+gsGq+&cA1_AQ@wQsc8e$g&(`!F-L%}*u}KCd>gHK;Y=DUR_nOXg+qwrOOX zl_7c7R_=f;$(?Hw`e%(s^3?2;CoL)b?~4=d^lkae3Rujbh-C!yJCTN4;a%rS;LwtQ zrhSbs5(m_K1=~5N4huz+B-q?M95)dGB)wQ*3SjW%vCTv{ER>u`7(!|F)?rpXojijD zP}$~6Ek?d~P3X!CSo>C9kI}Jms177s1Ma>*0n}PE*q5nHFxb!G-4Ur*L+$o`Y2Eyc z#SaNaq6&5R&VR7JsOSS0VM9sD-}7^FCiYKBJs)6}MnKD{Gl8r!09k_*VTxiA4!N0t z)0%rNaO<@G(U3l-U&1e&>KabM_W7f+mEPN;Xiu`GwMFd0HvEhM=@Yh=Z2lwac55`$ zucI>Hn{4Dff@5B%j&mG%9WAj7NIDf>Xz82remHM*><FBonRefcW65<tK@f;O-gE!% z&$;V!1d!-?{@IzSw9l>LJXqL=vG9m3JhKdF>C6LY6$-?D8m*phV$Q@kn)^^-;gYx# z=uiR@u<9@-#J~%uOnGzJvSq`HiZ=b=&O2`Z=eBKIxP|$Ej<(C5eEe~7_~F?iKY!Q( z(}ogh1CG1O%k<LH(pqe<_E%O`B;YDRNNQ<z0FH*jy0bLZh#<gN3||6I;L(2OeGPn% zxr?9teEtj9A6KFU^?Ol>Ggx;{grvj8yhB$FXNU8N;dZjU<z>f`g3hGnok<VMHeiu~ zp2Kmv(Yw$fIm1lDzouh)o3{yy+Pvhz&iC2kC$>+#`i{{|oqG5Y$N1wCcDFV)^k=!4 zg;_qkJSHvpO!;ue_m4bz#v5N**|-;10*8bIFjm=UVByv8^C<z6mKFg%Sx<VT1ZL;} zw%|CBQgN2XVBZHM4Gn-Kbo~_us@yUwou+M5k#R>fgIfmIZ9#Vy4v>uYxzE2hzz_1h zsRGH6J+P17XK#+Sqi}!+HI{au!THYmjT`r}eVrsyuc{SQ$6)LB5M_Ik?G8pU0)dAh zs~bA{t-4`dBZ1YcSGPbrb!wowd1Y2cX5zNeQm?88cMZ4U!=5B;`TYvp!T$>l%(uIy z)ZuZ34M{2xpxska#KZAx*c<F&o8Y=gO8~?EDv#Hr21pX*5?TrYp$Kc2MY@`YJU<-^ z*q9f_9DD4s3DAs<DEQXX!ks&JE`e4L>!{+WY=D{XP8HhDXn+V*c)cEbBHdZC4#No^ z({1r?VVk!_KDd8e2}D2wVFQvZ!*RzQ$7A_kf}ae33H(7FH4-2F$HUF{|M|gkug^zY zTKGnw0%=_VTj5?BHLk<yef8V7Z_~GJ-72<i-|DH_U7gw7(loLi;E1+N8cO60IP3{g zENmHN;N-ltw6rE{;eHb!=ZE0C3BK6!asY$-sCAb-w(K($?wfP2AD<(FEfb-aaf~kX zaX|ErMEHPx3oe1h_Q2&{3DB)uKq3J_0*P%cnY=LpPx<PY0AU`lZ+uX1*^0D7WMcAT zltdLR9#(z~ll+mezOwZ6`pTVOC#Gc`0gB|@QzkE2|IYIhNltp}53VK+LwIKxt#{&f zDxY2FO5jkDfX6U4b9g}G`YO&IT<C_eUz?!9k2Jd1yItr3;R8sj;<yzSTam-jP*=&e z`)M>I!_bW&Cj5?P!d~lrV|RPR07-ecMRra#-kVY%Dp$^NK-q)|6XId(XfN|C-7nlO z%u&-V4M}rg?j$E0NJbaA!wFqar{$@d83P8e!s|S-G2dCge*J#GyOkv?t48MKO+jD& z5kN_gsy#`0Fbc+C$w^7qkI2b?Z+%JO+@7xgp_D(j%EON5^ccT*iS{IMa<+2yaaa+z z;p={%|AjF*W5%v7Ssiu1`#8hcN;My!&v7#><<c4mcw@c3!<B{P5_T>DO>Dy0W)Kp{ z(SPR&_m9DQ!Q8;`&7M7bFk-jvpTZ4^775tn(5k8`jzjwtAPEyMnUu4th9m$<7(<QB z?xnW(tr=VVULO%s$AffxCl(0V=DC1mn^4`Ga3$dQ$9RwoXCS@+-hYsEG8!Ob?4=i9 z_%t5=1V0o$rxVW(<v{8-jyfw0ki>xWbdi~v3E<K886v}aPEKyq@{a?)b{+=UpI$h1 zawq%AQ8Kvs^9_t<UO7V7^ihDw4#SKZT75<=qOcuAD~7G5Wd#N2e=QSt<DZKj`=kWf z&9po&DC_kT4%3a+<Ir1A1dMwjP{Ki``k{=LpdkT5+u64a<GsWbabp9Cgajon**0cC zjfIHE#C9vRNQOuNlb|CQ?ny@&(WK;+&P*qKOJ91tbmG;&BOob~hmJZvHm={UmZrLN zStc?77v7t^tY<?^hAqi4t}RK*!*z#@1Y%-paWakEysJ`YTgwZ^2qOkaZrHG4+pwJ6 zCcobor(7n(Na~BL<$KA)5QhbVLD{116&}KaJ$d$fjQxP25Xd`EqG{^^9vt6!g3e=5 z7a0kYD73Y_vGbI@J4QFK$cEkz-;ULDg<c<1+W~`_UZf$3ZP+3$HI;qt2Pq!y+~w=n zbB9QWZe3B)vi#iKpTnNy!x6A2smI2|JUucucg4Em;&mNn=mvI)1bC<rv>G48AwxfJ zY;F=fbV$W<kz!}L+=idlWYoV1O|@@Fo3>iU;fHVEQe2{AvEV>+j83cH8VumLS|hL1 z1a3qa6417yuL}z}!EUW=6u<=}!*~l_O=|BF@dP~_hxRRy%Nxw2RXPIXDh|yO7Xl-y zAqjk9EA2eVc{C^V4x9~0>H+y(edvJHP3lVEfD(u_AjwI{lkh(}V6J;d-hoGYrNO}7 ztjs6BKSrdt2OKj2HWGLQ;7AZMIXQF%5PYlX&y4%#&R=+vN2E+IC^&Pk!T9|7Z)OCw zhP!oRQ32AXNR&(#10Y@7cW$6P(#4$li#8zs#TcJ>YVzdCJ=Q@MJoYgsvGlLJJ7eN+ zcJ8SA7s@qJ78`%qNYR^)l`!w^AoFShit?VysdjP_F%TY;uJPE;vK$&u$1;fWF!`FY zk@>X2ifzJzmgJ7p>nnDwP8=|B3}l~}eOkfN4R1d)kx5NH%uK?U%*}LQhU1lP$HPT- zkb89{(AOmp8wk{Mn4`Qs+T4DDk2!K0lKd25jJ^oKmRwYD`VQ;?Pi;jlS76sVrSC22 z(F%@g4YVrt!M3KIh>3}a-A`L)FuuOl@An^zXUF}q{oZf@9Vh1;3-n2cK4^Fl=qtb$ z$M4PtBz2FcG}?v3{+Swb)Q-gc0;pkMwmbxzj)ld={}`5?OM8-&8k?G7Ptv)vEEhX* zFp38Uqr?V;XE7Ehj_+$*`J(;(r=+FbT3u0baz@&K<C>dkPZ9>-6<l)*?MYU8@LM`# z_RN_aqy0gFO_!1Iz#a5gyP%0DHJO}dWVL%)_>KB`5Q|fKH~oPmU~I*Ep@|Er0CAv{ z`^E(V#4)jB!v#+E6;0GV2Z9{MVveW<IOba^ANtB4T_47(V63N9BQI<gU|beaKoShM zVk45q@rXTc3$<|ecZF$xcBnLNI#&XrNWk%cIs~R1Y)pp_^^{b{6Cdzy#+GW&z(Irk z+*)mdjTs^{BMYEohDc4N$%e=}jE~RLMa}ZLix*FxHfhoqHh06*FkaKebi~KlVVzzL zV>sgO#DOoj&7HUC70o9exp4C2cRNkr4H4zrYcqlK4P=LJyFK@4vG5P^CXh%((Y}{A z_+tj~Njj$If9a4OtDF5>TLT!$YB9hmtOx?0_C14mZxUIgzNN1`v18&@zZ=8Y<iU9- z#w4chY^kr#ViC-un)K}lDgb2)KDg_<GfuhVEmxa@f^%J;mq3f(9~2yT+uxi*jd3`K z?$vw|vmwbM;JaK6U~4pQofgR+oa&+_8?=2Akc5_yXZFlld1xD*ibtlmw!=5AUAwkI zi9shsF#EaQOKWO4cg5aD)k)I*f^7`|*w4t_FS1Vs7$cioTb1^AVsb4d5g#8PQ6M?e z;^hKIRz|v!5TC%#`tO%sG5uyInr$gw?dqzaacow4`p~VV+r75-tdnwb0*CFw-hmmJ zqpK<^9s%u@PUzjV;Yont+L*k&f+j=%Vqiu_($*cYCu!LabIQ58wg&bj<BY_FgnA>_ zk^lfe07*naRNoEH&0VsgxcGgiJi0c$wi#np+C+2QUs>&*r@FeD<NI#k1=Y23>0$1M zXDc8@*&jJ=Lo5J@w!g4z&2`Z%0ekEKL^3a$%c)W}Sd$wL6LmUvR5->uI1a;Dn#LX@ zYwE%SQ$Mg>&V_BKfFuv|Lq8CNex>_esr(>ZnocFq6@Vm>{0HHw^{DE&6ic=3V)&AF z-hM89_>sq+#47sLi0d{7jh1{ZV0;Ws#~)o(FnK03#-Tp1&3WmikpKm5#<s^RQKp01 zK0_MHeX<b{lc9O}HCmiU;$wWzpLgDQwdC0qZvKxZ_7k3l3{NXyP`{B9Xf?(H6g(RZ z-8sMpV@;6H9*ZGVj?3_D{>L-IWFlV8M>|NSZU<<#g)>ak@^B*O5=?F<O0ZIbkF251 z#Jn;o<bqgvBkC)5w%L+IaD2{bQ$O4A*3&pJ$u1kOrjr1wq=o6qp!hXHud?IZt1E%N zECCEA;#oh~B2=y>XbsIP5|kNqK#0$A9S1u+P}u1t)*hU~j%0QRtouaJoH=u{(cbxC zA6cnBG5#37@Q6=-!E`+f1Tb0P<aI}7vj(<!^`2ZuguN^=DmVg@U3OfKT?`EKp^Gd3 z^2;xy+75O4fHVOx5}McI*BeTT?*|y^sIJS4imLPT^QZV=Pckz-LxVj@n7eh}M%6bo zh^m?zEj2m$iecG#|J%5!=ov=^aTA9nfz?Gtn}_G+UFsM9*V5Alh_YSf%KXYJDzw<x z7#J<{%^fjf#5mX?RX0wlT#Akf@iCY9eEy>m>v&E?4N&QV!DDcZc$;i8KQ#YhT(XIW z_Tk-Nie+j%hAIh;JJ2WTpyhjUT@*`zrh_p1!`9@!x0s1u4?AH3=>DAde;kL=;gs#d zNWMNGqnjXXoK5G4#V2RuJeC7(o(Bs|$3?PCE-zOCjuHD1MfL#FsSGx*1Km(~PO^u= zp9z00{N3;`z<&f!%>v+7+H1IWyF;5259I9GISUp(@#dRv(nh4X=z<GME}A;!XC7EQ z#0jOpMA<flS}^cC0)W6Hjm?2=bLTI57#f(=DvGF^cinOS)B|-@!IofAKm^wZTJ@z6 z;WNm43T8*L4ie{+e<)hcwRn{m%%6BeN4U_s=SALHA#7`{23OMA(FntU;5ZnoC^Z{o zlaOg_fRa~|GG7@k85q80uRhVHMUs@6Gd?D%e<|3S<&sxf00!}tV0lFeFIjf?q(Sn* z{o_iYk4wO(>k>#*8SL1i;i<~_{2J`%NL;CD2R<54|90vKY;kZ3>_iX?sh0fPJ!V zEvkTIObiZ(jr`Uo&G5AWl1^pG-yjCw80YFoIi>9_>2U&aF3Oq^29ONHlUmeoN#jHX zzpqa=l$6}B`fdsf3zuUibu%U|9@>*sb(K}SIcdU@i~b1gN%q+`K~+9oO}C+_=nec9 z?n_Ec<UvEq*-^6w+wiGboE2<onT7UZ&j3!DYh(z~UwwfEH2in2_}pRwl2F7mT;MA* zucM6%Iyt!;4Yp+4!kpXyIKc}wT;Rls(0U*(#|{|x0e2I5J&=8uWhvV@%{bot9Xij( z1|+etx5hbj=M-sKS*sBnj8Xrt*mjTP0fwFD;>NfVa3+DS0wk&FcbBuG$kbQiFF`1R z_Xl)|asL0qAgS(viMldU7uVL*Hhu){$Q(Nx+Ly%PH-GVpN%;WC3NWJo*p55c6%AJs z{Ev-R{cC7ozB7ORq6^G}at=m$;P}^mF)<wqP}c+VlLl;=8gi4)yjlPx9!wz8j3h{i zlrqfJTQu)UW05l!X(W;;1EE=N#aSsVo={ishfRRRCVyrPjvymnlMsJFBEuL2$NReS z($gEO%H@hP(#GbUK6Qf`%k*|BBnL^rFw`}sHBkREJDq!VCD4~80L^1*pTemYVtbk3 zyXB`--##qddu-bc3)pIDf##srmnRaQ36QbT+~Rz7njkOUm=`Tzo9U2n-<M4vpB&FN z)*v9I;+mVAy=c}?P@(M&G<s7j-<y(F0Y9=Bzn|Uam%X8`z>5w#4m(&qpwaj!D!dT( zB#rnuwYdu?t2=j<d(g)D0e3F~D6Ec(9jw4^e1{Skl9O`>8srbt`={}sp$^l85=d>F ziru?CXd})ZmY4U7P-wa%jizg#Gar?@@%L%k*VK;fj<Oy~d2HOcu@LP?oyvQmoUHye zYxL;Ru98BO|8TI#=fx65pIXcum6taHZCx_PMM~1gP8iY!u|Im95`#{NKK}S)8g^cX z@5-8ScapG^VC%c`mY0_IsrS`Mb=}}@l0a7hk}UpX@Wl@6aU#;+916`j=bY1k#%ob1 zX}c;7K(b@B*5Jyy3m4xucfq1-=FD3(1GXWb42s3C0~z}r^p2DcbpRUo&7ff{){Vu* z*wTI5;>C+IRqJrhUzY3w;BX@*L95WQ%RLVS`-tapX2ic8IZmD!L(q`-`zB+o4CbFU zXSHou=GvDYMqp0zc(s*#Qd@kP2go?6Gx?d!BwIuHapLwin|(ss&<SzL8CxX*0%f^K zZzo_Fyl!}|TYKNR?xdxiT-@8fDFGk`Ijk8@`c$qUz9SDd%S05AOdm9875KKq$0{AP zEiiz?7Ef{1)TvXQtE?zXqB{wV@z9P0#^N0Fu~)Ux?nNAsT(xRdJ=)N<wD+bW4GvU6 zm^LbBRE`RL^wSvQ(I@I{+cX^%7+@q*!=AjSR{)X@s&?R{e(%-zO_jhVWsqhZR2;RY z0ZLNaDL>fU@}vq6mquLzOK|XuH`ae47DB4BGBSfS#i8nGPqLw@k#P4tBXV=2V)4S8 z5W9l)Q3bx5nwlw)3FF3%OH`GMOSiuSFowXyKzxq(;=H2-yG9ELYHe!z_Ws=6kZ6*C z>{AV6=%`Vn21HZ2!YK<z<-|0U7BET2F$4O}O#tTA5KS`a#Npdorutpk+q^X`=zdq7 z$kz>aE&=sX#liEUVHK>ZI(F2o%nyyq_e5yvx-Jza%_&Au|0A|tV~}jziOI?}0L;FN z>yeBLD`!olV$0JdEv?2=a~8ZbQI*%U$Co}j?XhJ?We!U0=kt4xhpHsEmDdn#B%Egh zfdm;b;M|i9ZYA(&{bd9x(2&VAk{W;b%n*5%ab`SkB|YQe{a#-`TmxqMc4?T7iOI0N zLs0Fm?GqcS%B8wG%InWLZAvi`%lu3ble|G>g_*pK)ZNX2YumGO?_CM>MG4@ipTl}_ z!mi@1i6t!_)Tq#RMq{qP8Xv{Ss{&i>K1AHgr7OpFmPc<5=V`2HyH03oZc;*L>`8Nc zQqrofkb$9j-j0inHB>-{re^uxu*IQud{>mMr+H#u=E*InBl{kwt_?``X<hQ(oR+7% z(ql4J17lTFaG*~R_9U?jlO48m*U;D~c2`$J`P#TJKR5Ts&N;Xg;Ya{BAGcxPbul!w zJsAVkF#~=(WxIB1IJ`{5!ux__^YX$QCx=rhi+0VLHKnLqx&4hgWy#+r>g8{!t^KA` zS~vZ{Bw&2b_oC8nP%{mSelSpIeD`3vxbZzJ0ekEaXm0K^n#Sm?CZb&kni|y3<qJSE zh#&q}&M8Bo&{klK=d}(<427UA4d)wJP{#%><9ls^?u{#fFeT8HfTWl+XW_=fv$LHR zk^6dn7z)uvQ_uehAM0nKq>Z*T7?anb>}vt1{RjS8C@K8`J14L8YM$?De$UvHq=b}d z1(TDfT{!s&_;W9sdcl%#>cB&7sw!FzgMsLH<lBNgfnX5K!{or4Tk%4WvVF&I&K=AI z1S{=0hI3jW<J(QnOnS4V3^vo7^r+u3r2jxBXSVI=WEmt+ay6aFU-K`2`N=JISVG$1 z3Gt~Jo5|5Giyg->nUiE;++@5v!?^)Fo0)rcCD1D*pdG^jH;sa;INQb}zMB#i`UBH= z-p1BnO=ZB-*wiFh8zR`}CK9!~cb|sJ#oKi}unhN)#nx9EUt7~v2crO-{y*gEQPFOd z8j=`*>q75y1JxsQbF(0qZ!|VHDUF+;b;>@oe#np^hZG>mUob$)&oO%VnQ2cll|f2w zcXgG(ccVjZ>cNqDd9H0ZB`K;CZ7eDIH%4B6?3a)zQj(ID8L_{T8n1>JS{*c=R%R4Q zLPPru3o%L^DcS^%ovFpw=lZKiidc!4rY#4Z$6knY!h>@Rv==lZ@?9LvrPOXa$<Ixz zE&*#BFb%f#iFO^oJ*xNq0NUW*z2>?hiiq}XF?PnHwIR((gh&>0GHli0cO}r~g-p{v zAr@h{pnl@fIY+B^=<ON0_lLX$y0n4zdSka<KK)9m5f`eY6B83#k#<q2>G4I}9ctQc zOxJ|L_j-VD?+A~W#Q_|^v+F!w@qMr68<&(Amo}|nN)k@$9u2VZO!%v&6->Hw+SJKU z6%<T*YucnqU!8sS*%8xNT72UP0}P{X2Uck|qG5brH}t=wkxP?cB87q*f_udA%)boc z69cR(hg(1`>?bqIe3G<QI?~I3<kv>abRfomoHBY0CcI_5m+`o^=Vd;bag1U^UR%C( zVq^6#OG{Gta!xJS+{T)Wm?N@aE_OKRhA#bZ*@ZSg_v%WZuStO0nejXS4z^ElV4wiw zKpem1+H3{t-a;h?2Oz?C{vSps0EltwaEh^jO(7`I7~gkF8|~?hpm83`s8g%PsiW2+ zwHMLYz2%fP)bug&@$aEU4M4j!RFbxIY_H`&74NODO#lIrpM#2OJo#D0CB(&3+u^_5 z{;ERX+i5nG6g`GMe?AVlF%sgr?(E$0_l1+5J<!nb1ay6oI~j5=!*D5%IpMh9W1RKg zfVBRwC#g1>Wf{021ArNmc4IP5Q2$Z5<qDS`6ZHRs7O&T`+vbQJO>dw+)ZU%acg>7A z5=q1o#~KM7N1)9}P~utN*uYsr+MMT~pQyyZCF&LlaFGIs;swzje5+eB?BYyD=I0-W z32a6~gY$77i)Jqf8;1A&t|)3!TG}U=GuNwrR}J{C;JkH4&7`g<pUd;mlR%ehkQEf1 zvkj+FKYPosesk@@h4VylQPJL7agaAO14U+LrpV79CXPD#XmR5B3I5o)xUm51_R@TO z_uY4Wm6eqPfmRL?x;7N{XNEvcWJplchhRighYqJ-w7fnHx{1j>Fm39TXHcdjc&;FD zTMDm3;q%}rdwLp{vadwfXe)yI-r9SH(Z*X9e6@8Ac~Ud!k<TjT|LDZqFb96f1Okae z?*Z~S$T#6Uz7L+bYyKm^XFG<sIo^XtQpxnBLo5R9bd04CtYG<`NssVkpWnBtwIyKM z#kAO(>1Dj6Vwub+$suO>Yfo-I<<j4bCXcw(tnu+FnVTA`%d=$;doq#bvg;7k1Ggag z3#4(KD}lZyfmNS>&TX-&wmlb>`r%uYWC=ba;(+AEmtVg8`DbU=K%+0Yrq22KWA#{p zP4)XHj2%0+e^+S(s>m!{8eJ3TH#RnDD)u9(We4Xmg!k=m%h4JA>eZ|3^0RZ6;=S?r zWwp?qrQScB1J%TYgaNZ>&Zd31PxepIOG9*V9#xT5PRyh&C}=$3AR<G8^u1I9VHevQ z8*@#tsrfi;-Ok>+eY+Rib{+E!SlV@ivjz^W)2e5I0Rev3b&~`ZELad6lQU*olh&|m zU`9sTmeTEBm9~t<#TI-lu*B&AsQ(Ikk{_-wE{?ce+)%}$d%bUnK=2`g<tke&YwPMX zEIMSaT)v#z;jf39ubb|`5?EcbddINroRxSlM^#qgKqQBb3rI$q5ex?Ot3di+hvaVZ zZj=BQGgFh3PaQL6Oh1l8x-sjBXN3b+&j(Wjb#gq0VqxOqVmY5~%@~yVN#Ul556{K& z<;z?1b8`NT@9I3{<5-K6^A0Qw2BF1yHHu5D!*#9%!jgbF8x4!aL5O|cKYinvW3GVr z)1F67O^v9ls}(%XzF)t7@JUol-kXUb?wvb-@mPRf03D4X_?n08D61HRL{?{Hf?*O7 z0|-W7tUDdSvfaCPi(TbqqN1`w04Noe71kBcyDN9c*45QLi84I_uhbYFq&}>2IXq7! zr{4&_9)2zS3iwL+z1-By?>)R=$D0=dIGjLY3EB~e1ggt>I$mrbQI;CfGRBHG(^%C( z82QLB`zq5(?$Q<|*^}v>lmHp=?9;p_!+Ovv$Z?!SYIDGqL>6cABNJR}cWi-{<nDFx zDg8$x#Fu^Q)GcdYe>7Vbn(1Y+O#ZyrxT#TyqdvL&>@&yT_U8Zb!F8?#`jP}N36$Gn zu_Z*gTJoR`O{-m1SXidaq!U7D$mtmH{0AT!0RzWcoCLNoIQ2Hw)>G5@u}-q-Rl!&_ zKN(}Vq4f=FX9!`!%QmznWkBltU9rjD7ABh9f8)0?o`;YS0CM60B#R7gyfT89g04>h zk|RcpNY`6hzXNR>#{+lD&OTw<@c9mDKyux>b+yBDhEMi-TR*`eH`?}{J8^iG)!#e# zkx{!2;DnZz+W*|)xw*G&C@y}$0lAwrObHMqLR&UX7{33@9FT78C@+%(E~jky?qN^T zGi2~!uh-*U2q5AcJq1J*78Y*ALQEkh=y_E&)sE{!;0t{+w4eonFBgz>`eQgidW`Jf zz?SW!Rr(`><#qM--lTs0Cc|FojmwLQsMO)QD3kzqjie+c`I{RW&Oq^Ai^3v?R3vIi zulQ`~73il7+R6=yXrDyTkOZb_EVWL=-X}IPj~Gk7>GgOf(MGG&lfc<_*Vfm0`=_Ly z#M8vr78MbLaGfgwWeMzq1S<3Cl<=YpE-1mt)3X3{T@wpi1sNF`(sxQABaC$X*nei; zw!u>eCW&xtBO1oCvYn!AXPGD~=M|pMWo4oq9@_~+$x*2x3V|{CQp1q`=nm3zIpt$` zYSF;vk0{ds4$S|_iPsnb9e|}4gD?}r1E+R+j518(fQ)ZfdIE`@Hnahj7QLB<3GGP* z0nRoY(Yi5t!K8-77fFi2MND&Z3un3#s36}q-jW^oSzsht8=4mT#p_QMjlblUk)RQA z{j(+{q-K>gR`1MVJj*9t`=YQ&K|w7^7mzfi<NobK62L)PXLE}S1`>|JK3fbJ^aSzA zd}KdI{HkehK{0F+HTwu6IB`smt=mBWW7mPMPXLn8(zpSATd)=rBqy@4(_*sBZR{Ul zb-Te1>xwIW6K|pUoRJ7yaXb~*iR0EQOwK(pzj8Sa@cC6&m(@wxRMQ$QEmxsCim9np zQ`$&Kh-YuK1GW^uRxLwcq1#ZhVde0goS#86?lDYe2rMd9tqP0V;KA+R_W&IGXkB69 zhf0iGA|(kx-v0#{j5Z?S4=KsXq8jf*#ZgNZ+wi@EGPClV>lz*fkIPl~M1)53ypF!= zCoqlSScBc1px1i=sGN%hj656yR1}dixlAxG@xEz@!0qwSUWF#AV~KEw1^~&p8eQjD z;(o^rT!L<q01o=Yv=6pe4Z&@|y%v!C;<M#vp~D}n@>`{#X+IKEWKL_}>q^0UV`Bal zXlfRT_^zA+k}Ma$EA-(;z{G2A?8>sby!*Zc9IL;ph{U7$|Aa5g`QDt+Gi$|xD_hZP z>7kQAjS!lRtH1h6{P&&zh`DoK5P$#MU&Zaey;c11nk&UsS6n7;{nanUgMYbS%zkc` zc>CXPiIpo>i0#|9g$hPGDN3Z%=7S-%Q=Wt00sjvC<pmFaT7*M$?rX#DtQ!J>L=oj9 z0X?`q03X9mFp^OwcxlF)X&B16mPL;{I-8zXd2eQcdq(2FM;itIK^qWhr<1&75;8GC zNraKCvb1DkOGEWW^2g-RJLHt9+hlnxp0XTfQ6#C%ONcW+yZh{pKwGkLohyM}DS`a_ z{A2M`KbkB5suiGSx@k09dry_e!RR!vcOKF-CPNcm#Q^|<Mw&R9^;MydCK_tlp&^?} zFj8fUBSBsHzC0Rb+NEwoNy(?kU|n)j5?fTqt_Hgx&`2lMR8$o>CXO-!Opr9)5I4Z+ zzTRed-$5Cv)dS5|tf`5)4#@g+QvALi$G0#yY<rRwJgqs1tz*Fj!-frmVy^4DQv#Pv zpAH-D+JDmfr|GJPBe1;GjG|T|#)Fp)&&j*8JL}LsFP~R?70(_X+N0vCYozj>7Hny` zLxtX@=~x1nUU})KXfMi=68kCjM+Cayp>ZaDJ#O4MwU^s*(cI7oN`T`cE@ooNSHNxC z5oF!1tZ?eub<~!o*0hRCH0`BoSQvRzg+5do>{!4KOKn-d#C}u`a_o58Hpa6A2(HB# zeW+swE`ck7b`t0oK(e6V++yT8t6jb!-vZ3Sp*UhQfTs^}KMywj|BUCG@TE@HeA)>y zK5Z~gqmH|<VDeC?(aq?HaV54<FJHDyEMBxgJp8wZ#BXo?mAK;aOT~>h{80Syt{LLV zCms{8y!?{*_@fU6c5R5-T1=+8&ap#f2IcVFCP~1M+pf1+&+7Xx!Jq?2xbB<qlz-6` z&n~Zp98FLU0l1gjtEDy~o~1*W{Y+vOm{;=;Y52fUZjm;5Foe#Gv#A&W-^PF`7=%EK z44@@>6z+D}&t^I^&SGSyHkpz%e#NU#wW)<C^v|3SpO#r<(laSDv-rz9{=pDeH-6dn z=-#*z=nE1E=nnxXF=`xYueGmwo6&4<*8!5y2&=*O`C?*X0!(x{$4;>&1WkpZYeD^2 z?aSU9?}N>)KSf=jWvAA^U^2#PH>IK<?2h_-H1R03Q#BQ!f!eLKFQgm6TYCH9HpBDs zCPUsi)wQ)s<0TCC`MRKUrOoP&_tsnT@Wsad5WRIF>`BrNl~S>(jk~kFj15Z`Yzxfp z?Y}i8A@m>-<gF+AJeQydWm$NS*mkX4n#!swYU$`Wbn%hl*~3RGGYW%n?Z%DoViZ}N zoYc>ea)X^Eb}T?^2<?O+F2e+Um>uq3!;=7iXLv79b5V}wzEm83U$DI_(i&=Neyl?8 z(sYjmEPLY#&2^2vaxrscZtggY^Ugs%ylBgmgog1lXU(PEqV?|)Px^71H@__vhl9yD zM)!(f@V6@TE{!VzTLNmMK%3gZ)$bL5Y7J`B@xyhNrzU=m!(ko{C98p%Y4;Rw-g?1? z_3OW}cFh`5R8%NRii<=w3}<(>OGrq-;XGL)Z9tkxOB)~t3`mC`Ao{1JNn4rR`WBBg zIMM2Lzpwvg1qEjws8Af$QKg8UtNw?<=fjVIKLLI$yotWy#(^7D+Zee2&u?wm^vkm_ zRw96BfrSJn>FjI!d&V#v?~cK6?XupB-n8M!WD+E_li_O9?+KT8IJ3g{zn0d(sWM*@ z$}D7@%umuWOT_WMa$C{qt&Md>{`h`*D39mxQ!XrB|JuWOl8pJvJo#j1&SyMbxa{6@ zetX=luWcqP*SQkt%@P=vn_B>o{S<1UIkoTCzTVhI%I~x;B{g+v`|wEL<0y~EjG%Tk z0X1mBIATx00RWx{TLP!T_S`+(?&*j@lq4RPpPyMD2>t?^KnAqz%*M$<*?78kzr;k^ zCwr;8Y+xmd)aPnN)&0<tjCFc&68nnE>MAen%N;R$)~t(B)!7cJ8#PI33+nTw%l<@g zRaF}qTLpQ#_SZdmU`Nylv|T}HPZ9^2Ou=@EPXT7Zp5zWCdy@QZ?%G}HftKEPX3v~? zJF@sgyKLOsE|vi84`b{0g#eZRo0*<&l$Pyu+`h4hY%_M^1oa_<2FD0b>w*arCLG_D z_R~5N3&xnnu$;UnaC-PXSTGkjz^bEU2MpsuJ?QH+PcX=N%l8hL*o_QX0-l(dr*NX! zt;slhpaM4hojNX(!RrZead$wQ`UNbOZgtATP2cqrpf)S}TeK%PBDZI{KF7%B5Y)vT zXv=hrb(L0UxNy%M73de|vptGz>GtR5i;0<oZQFmvE~W&2BTgO7*I{eA2jiS;@m)QM z@9J}>JlyoI1SkQe6@L}cO`bft2nW<W=a|+o?t0~wSF#R37lXJn;qN=+j5Gds)%458 z-gEa|;-$q4#TP4AL=24NY2X<dnPSY5W5qYW^=)y{#g~Ym-Sl&D_dWNCC!d}vo__W@ zanF4Z2%IP`uD<4n;`|FHiBnHIT^xz@IXSr^11E{|5Fd}%bB->mm(8F5GC{@89Mz|* z;J3i@<9-ePA^2<IkAkN>Zh-$9-m&|nurhn)mTI|lV*+><3p~h?qy_%jjG2a8s|g6w z(^(fA_e01Y5K!K?8<JXFEqX~V?@58x&Qoyo=vOki&7b+940a}rv-3qZi7Q@zW{t^3 zh}h(GXh{w#;<GH1%|n)lrG_(vFW9Qz#9P<766nnmn3yv$4xh#Y+z!HtkE$cUg>4}^ zPJf!u)Am&PghIEjsAw6!;V)FRPId#t<N!|>t#^l#Uo@s^4Cn+(;wx|vhf4Wir;Tl( z7k;J+y$a3Bl9g5H5awX}qOPL(1OSibO@aW=?}9B!HMJI%f=7^M#j<5r;>R}}?@j42 z8vq*uTBhjYv}Yq&tUkj^g0!0U6KLUj(o$2E>qYy8*dE7W!<{4Y^1h|a$R$*g01g#; zA7kX(Vb4=bO-@nbK_Y4xmR0QHM1D+J=?;#0yWPcU#<Tcd1FB|50H)ydaA@Nh;=5Qp zytLaZ&_^?*O-}TG{{h$=Y?}`_tn04I3J(kv#v84{K4FvEVfA}OayH_y56m{;7<@nH z^ol}Tg)~hm8I!S_!pp^U6=!YN_6V;wUxnT&4HpK{-o2o@i;4MRe8dC2gb@flJ8|Mf zpVKn9>0Jp>0^J5kGP4+ub~ieb0PjO}Xq!?RT3Y|SPkL$%PKRFrzYG3tcy8$%53l?& z#6e{_CpTA|c;a|*&bjAFVDUG%-60<O`y=A{7v_k+|Km|{`yF?RYkzpXIQP5@#P|s( ziM+gg2{a<#kR9qYOc%jV=FNNQqL7L9PFRjdcBlk3xgFWDUii!pH*;GFQAUa6v4iJK znE;;b>}9Y$cQ)fB;3wk=7@7beW8^;x1ey#?UI=X?U^ueEg-%}iYzAmCv2oZ+K|{ti zfM|iAW^!33>sAt1Zd!j@Yg7FuhN1KF3^{r74wgYW@{qjE{26N1=b9CNJzsssV{&)@ zx)SJ732ZesJ%Ath;pLST$^#x&fHt!5U9kOkzDT4X0mGv`>i79QD%(Ohu#>CJhB({W zKC4iIo+BBVmv<Z*w;y5qxz?hfal`ftlYEYDH(fe?`hU8!CYta)&fiJDeu<8=WjjnD z4}>|@6}t(@_9}N|Zf=&)jr*V_rECi-H6@vSR<rPVX7{SfcVl4-i;JJcPy9vv#(6-E za?Sv<G=M=yFlfvjJ$m#YWkxQck_0vs7v6{8`o9MB?+<&DYTN&qG3~BlLeIh75A$<! zE>tQ^7(`zc7w-i5%hYUEF*CxfdEgS-4eZ!7{_fgV3*%+%Lkhd|kK>(qRc(B>K$~lK zRW-&v#`k;DR_Q*~av%4RfVKN4h&JFZj3cbc-9Cw;?2)@Eux0590CUuiR4$!dxYx8T z&@}v5nTbO}>|%NXGiHzKA`HjqyDBTan5>W6TC%y<jrAOg%T3-z66jVyl3J3ehx38C zE-t<6Pk%ZawAa9MLRJbt6MhPO61*eFBkqTf7%9Ge=6A$ZS6?fBcjq6(KOcWeJUeT) zxaZyn#7#f{rMUQ#%VhkxaYu=)tSouzm7`1!k{FFF)RTfQ;so-)zWeUGsvs(X$iKkf zagglxi{4t<DE_miocpjPSZF7(0X(w@Fo8kFNl=eyrE~=Q%rHWT5=LYky%jDWEMSl% zlHN*U5hE#W;UdnX9pUwOK47?=KWS{YR~QJ8U(y$E&RS*1$0cWsPe{w+>M1j7%gc@@ z1!9s~>szkh6XxEz66jqL7?zXsAb$Hlrd>s9pM~Qv*@AYYwuPE4;qVESHZCFIMSO%c zJdrO{n)d0a4T{M`&<N^}_ol-usL>kG_1Qr^sI<+DG9XApEjt+dy_>eQ+RLS@?>Clg z`~*$v7X(B-62EqqBf$kzb4|aE+5^t0dp32GAs)w%HMJZXTLf2~I=nLi{~**J$K-;G zh_35I0<npS*P%VyNIQh8_9UsHu><xb4WXyCG&aqX6U0vPbAux<0gS&4e_Y&U7;BYg zWx$@Kg3VyNR291`C7`Dn+EZxn^X>5Nx*9&uy{L~zRoj#81=<aoAV|ikORqpfJfXYF z-k0+^`q*P%#yDa<w5gS~BncM5%mdCTGaehBlXGHUmRE1B8^0fD8S_BHqhQtpwuYnM ztz5d4wn>kH`4DAg0ov=vt_s+j77w#M?XFsL7Zb)Q?_dE{*$y!0CDvH)clkN_=XY1x z-F*6v1iBTFlw$LH{)i7P5Q+%dD6Lxch4|ZF9~3k0xPzzPJqiCUcxqjQ>{786=%kZR zk>KOK_x~Bd<MZMVcik;DAkX~HSz`1NM+m0r35QPcrk!P#?;K=X`oW4u4N=E#-e+4y zOV9!xC6;G{d29y}0B>P@CT|J$A(T;coNCE165~HwQT{VgMh3_yhM2DXGu|y7<H#>A zF7`;2L>h+as1@dKfqB`kqGa`{dUMlOJ|oU^*hy1L?NVE;+tS-~xc+qY{HfgH?>bik zy-ETSj5Nf}w1HG!Q4v<z<KyB5w@5-o`XBfvI{Mtg;@=VR>eZ`pI*s-RXh>?@e(coo z`!CxCU`UAmScQHXz+CtDT}}&)^w4PiGup$EcxTFk)U=dTfRNhO)U>pj3PlK?MALqS zlcK$_<Eg|8+PpZE2>3S+9T_=tq~irdrA)&l%Eu%Kzq#{x5Q2&wIc5*=oqbTNFE-}B zFpJaY$=N>mG%*FHBw7JV>UN*)wBmeM(9}TFW}LcW>C!(srFYYZCxNwV*P?UsP36h* zupf()<@b!)oN#7)PkWM3G355zc{p*mTgPpMg@wy8dR>%~l%)H-YWf2FEn0on(bW}| z&-Ru+%gKA{?z9UQEFchh8x}3RSX5QYUjDYq%RJDg#-StP6<32*NsL`i1iOSLaBPUb zjLI36)8!e4k-@0Eys`K#-OKSg%dP6H1(1eBBZXK;w>oG^{sxQAUb(A7F=y^lXlZT1 zhM?dJ0Fr55&ULN?LY6?cPwJ*!a6us^XiprdgiSE4`mZ<N5I6no$Ab1CKl$V%9N-jm z0zz_<{TckNIXOAwAA91dU;X4~H+>55u{Aq82ghAHVRfLAMkWrR<Y|q~t;^>wTEs2q zP8~-}w>l+a`rY*{;;pZCR!Le&j2*RwCR$g9G1eYb@&Wg_$blwqO~{dH;LOB`<NY40 zwm~W4?fZSNph%00@qI})9gSfbW|q&)8(RFSU;O*|6--YTmy$j{J~gA5_gr4;$bo)P z{l>;iI*N0{TnR*@1Za;XKPUHZ_;e%~2@N2ngCE-04cY|+xOT-SCqL3YysPiy`XxM! zJT?vJpQbcX#bGeq0?hiKgM(5Y=<4!@%OBboKSkSqBTv<)rl5)=cnghwp3r=ESKIpR ztb7}bi{C~=|5pE$6s5^IxAT;nVDiXdbMyRY*IdH^9cOB~cm&4vmA78;Bv>3m1Qb?0 z1shG<JCm&&zDEM<ii=laQS-;)+LPq>Mh$YbnZJgKhwtyn!M*Da32fZ7@ncPReigPo zNo=p>LJmfL$2QeB-rpT%Z=Z)p^WTASs)2T|eXyhZx`ujcdTHph3T8h$YgV+keU2*E zLy0#O6)i&6pJSm;No$|Q=3#)PWzeQJj0`-yAMJow4~e=*v&FliJy|SD7;&ve^X%?k zlnAx*$L8gw1cLg@SakKXyrI(Vn~pmJFy7NWLVH@;@7^~-7eB8rDq4#1%1bFp$w4kC zsW`by0pAsN1jPn*<K1D|!@i-y$E7(mC7}HAgd;AGHtJ^VhIkO2{a&*)+ShvZm6yd& ze|&>@Zq~D+Y$r{*IC2}2`Zo9+c&g&xS5i{)>9m3=4_s6*Wqe9fLXzgwCScHeGgm*6 zzX`@6VNW}>y@!dQ@!s48i>`A}mQ#@SL-@%KXxb+YytAh4YR%&*CqPI~7}13D_-ulI z9N9C}Oe2#>z|mx6f{bnSCNbHNSbBCWGM+$TTRyyHB9k|sgg?fA3WC-U%(R`EhG3*5 zmBHjky0Y~vC+MxM@<)Oo?~s!V<iyX;kBM!1d5=7RX8aQI%-D6V1fp63*s3;WOL57k z`0@Xg+aYjzsfQhYIF8$YIpGJ7dsnSm72dvYhXsKaR0|IE`i0-;({OULLuxa56}E13 z1sb-wehw|RpC~cu2BP8Fd6TeW{?~@4Mgv=TlyYR-1@IQOBektDiSf@UF^Le7$MF9g z<uOzb9;t6^;7Q1sn2YbO_-xrfqqPmjN$y9(n(^Y6K#R7cyj&Ul$BEDE*jf#b=iUgt zf<uB0!0)Fq(wJ@9lXR|0Z3_unxjaw_M}O(F$K>T5Zil<q?vcR8O+_5vzskig6}!Q9 zDcSEyjUz+<84kEO*ADNF>-v)QYtaNhhrUXunI}~TEo*?1IG9Zsm#$d4Y!3SMSXExp zMn^CPr=iDpXSvu9;l2Epwj{&R=Hdb;$5iNxh6aqL4@TzZj_l5QbY)&W@m{*3K0Lu0 z2N7{m;^gM~hUcPf+(3}DS~uolMw+{`yxhwJtyCR%RKOXA-}^UJ9^ukyKHu&5zO=N| z{>rR)FxmDUJGdh#+0X(@07_0!W)@|HedW7~GO@N5r$+$Ef`TzE(+Vd2BDr6Dn&0a` z9)Q?6ufF=qJ=gv4>Ya05oGof;(At@M2g$z&KLY+w@SEZHb7!A@cJs8!lRud@b@D^g zE}T3C`42?LwsOCOJrsb+S_~%IkIh-I@aZ?-e6zDb3zse~fPWYMfAA;3t2l0tK6h(H z<6@kNbJxBMppjz*nP8#>7>NT~4sDhR_A#9m-Yy#E&i0`volJmX8UC5{2y08nc*6tx z8Cdm{0}fIk$jV<nS$WEE#A#ak>i^97R7T+swIt(`GfT*OU&k<J4{k>+|I>MA?;Gzv zyAp^N31Gr9BtIwbG0_@a2>|jath#dRm;4sODI-tT&PW>&#MiMcH9hU0;qX$WHMJYG z52@Xta&j~CCD_mipbrO*{5?NAmkUr)<%VVFU8M_sKDO)`7^G4=MyY%ppaQ)0!gS-0 z2<9p=i4f6-O&iu=A@6Z)lQUu!C+9o~xw^K7?f<nnP~f>}J9&Fp&ajEP2)rK%2I5P1 zVwE3SG%607k+l2CUhEIp`ng4g$DvG<kdkyg#v2>5($mAVCs|Gns3cg6TM%u(wL{_k z#wYc=0`GAv?Sh75Pm&8d(tfaR%<As#wf^{c_Aj;Aey(grfbY5zr_d6Z#P4yc(Hi&| zrYmxL>u-R7D!}2X!*X(8tqvNy%1TRTcXwSjmX!P(L;9D|N9wBm5zCA%wqpBEfG4^> zFsO@Vz46dITlQ?@VA|(A_?LR>z1U@m<h2G1(9kmGw{zjJ-28t=i<U9v4UlvJ%5@gZ zglHI(Dp#Ao;be@hVE61P*qbg?W)vnN^8OlSc$nr865`bt1bKMWHhfosKoF;A2H!(_ zabuW8>oIcfQsZgw`8hd@`_gySV*))u#ytW^+T|fYIAzLt%cf18)PK$k&)?G2*qCL< zg?Ocg<N5I4hJOpb)0rO*k9lj(yhS_a&R@71_aDH2TvOY)0dvKpK^I+47gtr+HZ;wd zx6q)UJAaY>$Gh)s`0jVlu1`)$X+mKEB8rJ&6nY_O$H4!KJcf=L_ZapB%0po!m}lX6 z5;P<dY5td2rf&oMIMK#`;(sKO>Dz&UGE5Th0TSEO%Xzw_XS_*|`S+2~qWXYwc0OqE z&3j2;=R;1gp8oY`C+NYzF5V;BJM@&vTX}Cgn}>uVt+E-Kaf==AUR?=9g9Pv!*YdOT z&lr}SI~!YwVAevv7AFn*wv=x7hGUbaecia~N!v^KB(KKy-@UZc+o$X5dq8YA0UY(< zG*+do*dAcJl|}>F_^$tm0r;cQu$_lC!iMGK%>YvRG`9O{r5KQ@YP3=#hg+o4GA@MH z*lS9HiZGGK@4pibab0@a0OiR!2JUUzLa?j$8s1x4go~-fZdhI(OnC(V4ULQh4y0QH ztx9|dvO_zPNaoim>1k1KRzRteT}(vH?HI3ja=7*+Vatk(7X&3wsH&*^a~BueWxtOE zzWnmb-8igp3i=!!prn$S4zgq$3VV`#kICp8=W>F$PgY&_xMt0oQqAkR5rBawoUPj| zAVC$>gTTtTKy&j-EXrKeWd)BgW0YBgO}JArK6%0j1a?AkVm=mZzK5-uK4=Cwowr4# zba9%-Ln^B?v2d|xobQ}94!99pca(aYTbmQ0QN0j#e7LvRI%aLD)sF@2hpKl~ZpYB* zWz^vZIQ_53-iz}$aBvEO`PEcZGW6>{U%K>Fj2qMph#owvkW%pXQMIe`C43j>R_xv_ z7g|Fi>?9F(!f>otulYQ8bdsdof-?tY{RWv8W)2ttr2%z_h}x}NF@I@jZ1keN_y-m+ z=8enG&+N8}gqhLk{QM&UME-e8$);^6|EoAq^PIl+U4<#4h?A@T+9Fq!W5J5s(NqQ* z7lQ7UP>O<0x|o0)S!l42fQH7&@Z;f!hC<sDX=-p1+$SG@B%XZYF>%9n*J`)lc5D1w z|Igl+09H|*`=7Ztxmj)!NJ2IOgs>xuEP@Nl;!^9bEWui<wtcnQMXjxEU;Fy6ugh!u z+Sl#b+Go{jt%_{6+S+QhAfUJ*Ziqr4VF^nh34!dncjo{5oilTD!Jx^_b`!pn%$=FD zeCN#X%$zyrH{bcrKb}fGTw5CpPWlp3Q&V40NJyTGlSSC}YES%}_W65Sq0aFi=G-O! za00Br{uy5$L=%ph+ry8`@p+M$%8-Yi57TQwE|Exqnrt)?OnPQy9O^`+BqvS4S*<54 zl1D_AS1=6ymCApA^@<n0^htwr&x9?>9c>*Hlt%~Z#5m%)GiT$EE}P+{k-Z?$7YIxl zIb|?cqoXz?h^(C;ME(UM{#U~WT43j%3LiBiyD|kN`kFXx4hH(s3|jY6yC6EOw~x7^ zAD-!7@+)kY4AUPxV9{P2nwOiK>KZoCuy}7@i<jTHapOik^oIx6IF9{Y5~X|g@Aq*b zmslLzu3V6QXsGIvtkk%fmcC`n78ppfZsnB!>@G~{p|%i-Np$8fXsWB<1VZ`3Sa9l& zG^d<$N+!O1KZ4HYpD-{@sKCIK9qCvdNp@(1p$Zd7`o}IP)&dNWbep$rdj|&gzhLZ1 z#`fWYQ1D|8HEeLR-O``6pZaUH(k>ZZI$z~tW?dolXwFG5R;1KfyUWYJ*PD2hZr}be z+TzE%s*T4o)3zxG>A0RC-t~{5?OV-(Z*N{tNAq$?$Wt*3;d>aEm%%378fgB1roO%* z9k#4h`R-i_)P(9Q_fA>ae$>m&fy98{o$eZQQ4D#q*N2b57$G=c3N<xv#Wi0Wv%nrs zNDW)`8NV`mWbs?J)4CBv-Irk&B@;6;R2l1~rSIycr!E~+w-w1hRM16D^$o9&8#Rh1 z%X;078#!(QJ}A6~f?Qk$?QO2w-W5kX;b<T6x>MhShT?&)$m))?>({SuR0;m;*pJE1 z%I=a#&N87{1Z}dyaTeBx?NZoQzqL0%Ds|=pI6Do`{TI>Zz19*6zYQYtD-DeeS^SVR zP@k*L;M3D_V<xw{v0!B5V<`u<5T5O5IRbw5m6otI3mwzAW9Yev&#s+2Rr!uGRl0Sn zDlOfr_Ec1K^wK^u-qdN+)#aC8p(ac`X&OdKPohD84s)s>PfH0rdFiE>a_Y9a>=!RC zdE(YtcYv;&%5y{Lx#XfI6hQ*g^rFY>@sLa*!?35faps6&Bo}pv_A(@*mxOV>JSdd@ zh_3yx%fT1v=b_;+2c04#`{-~TS?7W9oo}t1GU42kLpUFdQ~%I2=I`46kN+$3<m-6M z(8~u$SX`W1k|p#NCkXTu0-W9iO`jonF&AO(ZZS99qo?4xnQ_E9G!pr8O0V4rjXHGn z6^!7=tW|fsYv2MK`8m04FL$d%|5uL}uQ&Y%WoLh*wyJ6trjVb8sm(qPI6LduHkj$# zO)4ildt%7)zeY`g<Y3BAFvU4G!;PIR1i#d`G&I~6Zf^N^lqZSP3qg{aMzE8zyyztR zl0f=dj_)g*c5K?``NzY4Yx(v+U=V#aR^0v!?%9|Z>*3UG*zoe@4#~;EwDiB@>-j_e zq`;k60H&-ru9F?&-8**N&~W$&&6VVFP{9?oc~8aaA_P08Je8K0z7(q*NoNqy_gYc$ zyule6*P@SP(GF=Wt_tLLa#A3iq>>)Ml;1ZojlMJ%Ig%ze1h$ox{V%l6=72Ps-`La~ z^VCL?Mf|V^lL+6$!c(iZmX-Z8wi0zdRnYIhsUZ}elb4e_h#xID(2muS#M`}huOA-> zRp#K#vmGb=2Jm@YDgV#8AW*DwyCWSxtc6=!&$AqL9yB=5Lmz$!<T_1F%?|B(f>7=D z{z+HLy{)|bF}yn#fb3p~4@wrugP6tX{zO%kCXI(=Weow*`OxU1k>B)L{+|Vc$&d7< z*^~(rCS-#II~T8~^U-0v0EB5VFO;SX_|c`VzRt&mdIq}Vdm7hlL3?i$zWl90U*va5 zD)_xk<vrg|;uP(})z!C!?Uw(9)^-3S74EmjE<sLH7Br&-A?1P8)L(T;ST`odJMzV` zMa4f)4+g)6**@&k#w?xB6+c4PH`M#^Tm_Spl79)?__u+W{6!!bydTe1?8}nJTq=_g z<1=X8hvtFXI3F`7FMv6X@eIKXnGm0=BlYz@D22p>;$9c8w0|V;(!i;kE^#yDP$MoD zw*ZMaeYrypt60xaVYM0$NbJTDI+W7~Bf+}5sybMu_V3@P_U+rJ_U_rE_Ux@td$5mT z@tzN6aEwnnIy>jgo})hYv5%{~{QP4=%_)ewW;}$;60L;07B0N<<-gu~=7nKpaq7D! z2gw8)s!=nAh8Sd+$0}ma8%Pt6IObT6wj(2hx~TMq9AsiTlS)g+A=6Q$CM2=&jY)5X z8|xcOZRMbWFvMotP|+niT~SOwYR36bjGKMs<*rv~+f%>Y+kE8U@U}dVhM9Omd{#K1 z0^=tC`-<&MBQE|Bz*nkucp1*Z0HD)_E6gw8;N>vxjki0UIEDn!Fb0{Sl5oETKrsez zpJhPIAZ#Bn_>RF0_^TE_HpQ1cI{Pq(7@IcTM-`P*NDB+{!-<K31}njThPDy&QHgpf zoL?i+0eBlmurr`F(xnA^68s4YogIf?o6#!O<6-}~<G1cb$Ertc_?=u_oY&+yH=v%s zjwU5bFJ1#XLS50WBkbwHG=(W>Y!B~;ChKQ=sVX{^4Hq$}+_U#ptbRF>_12YZ23fDT zP!3jh<dh{F@1uJ$UH3QCkn2>TIvAM*a&tq&Vmxr);%fsRz3c;3)jmFlU2!0gRbhTU zui>S^L$c?_gLY(B+fNJ}Uf8yytWUI1yt<e?d2+D+@Zk-ZhB$`iD`M3c=SP`g1w+Fy zO|xmZ8hQHa^3`3daIb(J*bjty=)h%uXcR66P#d;4hV1Z&-Fx=>VW;&&;~y`O?FH)* z^;w;)QYZHY5#g1ECRHqDKZ6f3KG+@X(#H!=CRei^j8(id($YEY*pL1LG_TZ>gyg`= zp8hyPTYT?q*f1&-Rx{9tKNoH9`FNidv25srw}Na*z<bSV1nI(sD%x}GXZ3YEp-F|- z&`Wq9jU!piepOc;#|r$wgvGVOwBw9UMI}ZLk6?VW@|x?fe*xxMdZxqX7JSmkN%_sH z{!~Z#&b1u-e7viuD8i4|VNJX?HTqaD$Gg_PsF$v^bDga(wgZ^vtnDtMu3B;b-k)Gu zzkI)}?ETKl`l$GPPBy;w+I(RDeLNrIVRzIyP*r8cYStw>zk=L62ll3I40~s7#R9z1 zVSRl}0v1XG&tP0Ln_7-tC11Vkp^a@kSG4_#2{-C6Y;8Y=J9`C`6#muI{U&^H84_*| zV<we74`a6TacwhEF0{KUOnb7}{Alx3Gt4-&=lbHgVmV@cw)(2P?P1puKhi*b?Vdpa zE{L%Zr0qY4?XeF_zjUeNxBV+mwL;d}j?cbI+14C1VBLQ0A=sU&tgO_8BZ<fTARave z@&Nq1@>TW2uFnf&r+doOX=>K2bJX<Hro&!bq8Id$d%S=D>Qf2dNP6L?cLA7rQJkSp zOR(@#G^8H)?m2e)SnnZ3lamo26Q2Ylg0#bc&6UGW55tYWPOrmU;t(Fd>&?c-rc<MH zV@HAM+|uZrn5W;LQ2Mc3f0++MMcTGVbM@Y}WzXJwj?RqH>5$fx+aols-%S736JIic z@*8gm#M%h+IzVtxlcHBIiT64G_)oi6+&FC)c9!rFj$5nl*weo^96N33@YtC4h{b-b zi+WR|q@&jdO_vL7+xZ8kIr}&btg8--VyUdm!A@#&3LkX<ZT$;_h)4V?u<E_?_sY8> zADx*(yR~&5xZ|r)$sdO-KL!UjzW(~yndz!y*-2`l-d@9$nb{zZ+wT>1HKT4w4^6W% zR><B+yIU1|_xZX?sAJFvZK1&=WZAm2?p{YH{f|yU!n0eqZhaT+KK)~Iv{Tt!UypZz zwGhuIKVqhUL=9Kh9`fV9b3cH^Bw8n;mcww6B-kHFwt`c8lO*}DQg@~OjHiw)2!~1d zaPcahwWO{PL20Q$m6wx)o6Gur>5j4+KMcF>3>zr*&{a&Z+Uis+aPSfqe@LK73%)O7 zacn1t6%_C#y82U}zTtwNCWoTS$OX%2BbeHPTxhPl9Bs8xi6A4GMzWOyWV%gQU}7EA zb6$qs<yQyi=5Eu|DNR_1fB6w6)Yv#7>^Kw9uuj1H3ko341oVf;p$_yQ0ecR}p}pFK zcUdE~&^pvE#!EXfVdR97txof+Ai6Uv_~D|Pv;{fd0AZxf-J~R^@P-A6Vqv8^^}OR# zFQcMf_9Z2~``&x+Rdu*R+=z3brN3|I&QT%M3&5TT;iw;r`-wQ`2W>hBGYRa5#wI`O z1#Mv5)=RAIToCOtwBT{iX|oI+x;_JqQqW@6gg!L+*dCDJdW!@XOr|_FWyX}Zmn~Zs zv)U-%J6Krq3e?BVsD@#<x58D`HGa%u^0M#Jo?{ezMCbeH+jt*wG2XayG-=atxb79( z2^UvDtDEgYcR6ai69W>mc+LztsjTNQetOC0w_Xl7&O6#HLnkw&E%!$seYD7FYMqFi zf^D7qY&kI3r#zeG4(_)wo~s1!x$4NctgqU;Sf8!FDsLyNEE1g^|Gsv#1G%J&h2Tor z4`TZ}w)gRJwN|WnVtUAKy%6WKF}^tye(5;wva7?ik%Q=nP^*!ej?u(p71aHaPnYxP zi~tN?j}#OPednBW<{bIhg&!|YNlh8sne@?d(I_-0R&KgGvGnQNa3Of_8uAZ2P3Gyi zyuJvrXGV@Do<}5_aFQ}i=SGjnf)9m#AS@CwszA)Pp{Z$IIAm)r4W8@Nk%OqnI2<?Y z(kF{g|M+F}MGmc{f4i%)rLMY=<-j@8GIiTA%H5FaB#fB&t;ajvnb3mdr$-TpwGr;o zLiH2jBuPj{a`iq;Ir{9@@@)^qYtb?7=DTPo?&P!(PI>N*HZ*{bGkOXw26({24Dc*# z7g~^Ymhx?dHpn(u`rD4dUajQaBuxMSKmbWZK~$ZPP#<V*tw&qxG$$kkF-L>F+d&|c zi&2=-AhOV=I%lEvC?<yR4`YgyUTo-i8Va4onje!2^YX*~1b<aRQqojt$L#5>jNXnv zwrJFR(C#ZRh;=|FTOH}peN-8-00v0OAc@Jyz#PqYHJa}=Nc$dM9NQe*x6SXftD){z zkHvrLK`w?8!{O9czdvY&LNJtSjX<4^Vcnt5PDkBw0XMB(LX!xbiqp}7-A9#|r2o*o ze3JelykTa?la@nQ+Ru3F$c`O3`UVV&{%{B@wbO1_td0Xf4uY^^{HL&3)SqIN)0uPw zsUA9W-R;H2G~k=?@#1dU6yr3$SRG78aPcUG4{hJUhqfQYDkqk7u-*4#SXuc-SFhoC zdGp;xk_H2H2k&nV);)rfNzhld%J6;WAlkMDkS~pB*_v_8SzjQrtaP+P>FDa}85&T# zNW(E^ZbTf+<%VG)r6m#Mg4S+thP^?MUNOCQ<Efnh;eDoKg;q5;w<PY>Hk@PTq9R~L zA8@uwYHErKV)bFP_ig<O6R8pJsT~*v?Z$WmtL-@rIB&#zC=u^2j6IzU3`^7XvEvLv zSh`zhI5ZCZo>tm^1X04eQJi@|&HY$xQTIi?#M;im+zvH_H3+n^N_%~sbzZ*#ADwLU z3A`z`ji`&4p=J6e>jot94m?S_oUp&nZ}}So{?-QcS*%8%FCEHC>6YIvgcbovD{B(! zW*#00%&=3`ma`8XIqd6>kC)zc;r`M62DD#muf5@hb9?inidTk??)mW<i;2C0*$EkD z<?Qygr8_F`*m&$ik@WYTE4}ch7s*m+m~_8=C)z<W!F;yxtbxF6d!{nl+B_=*;nYNl zey%V9I|O3CC9xHxH=ix`|6)-P>x<`#P{jId^;LOU=2%>$(qc*6jcHau*8J{}7f4EK zE*eo}4J|ATWWgY78r1HCX+da9rR!r5cDPgB%Z|GGmgW}PI8Y7Hm}tbteLcQ{R>S^K z6&9D^bZhQov31rR*X7B8CZNKX8wc0*&kY-1`%m~&%{%huU9xiJ$>?!>5xPO2g@4CY zB)tr<Pu;Y1r|)pZh=>;%J&&M-r-ltbn&cznNI%3e(Rgl-b%6e1ilYfm#&`Ka>`a)6 zWm1`Ngjm-8`Xlw6hnY^ILf4%a(c3c#j{UxbcQ5$LFGnF-GQAqBcCXp_+yk>IpH8Rc z9&t@6dE)!-v~T|9dkhyBUkJq72*=k|^nPjBIUAar7e>d73!_|%f%nqhFGa_5#B*(7 z!HK(IK>7<@fP%)$yMvTu3D}Ns#p;x#WNm*eIS>f>3_;1(qeBk-W2gyJ>}ki2nvf(R zyQ_RGKol8SpWySI$3=fSRIoUR1dX9DKx61G4xG61d3TVK-eqxM&addn+yY5q+}9jv zbfVrc^2VT>nm^h!0y?4HSw5GHG^Bygx+dP9v#zwC@z{|aJ#yq9@Z!Fn<TnS0u{gE| z)Ix@qU9**#a9VGeuo!6d5DV_nX;6=kq8=Ax8eo@fu;u6GIvB*-exEgOOIg|4=sbGz zSii4Ej(Q0F%1_c3E*I$PDW~6GXa}hk%J(|A!~{QPJorge!YT>-+L%a=E-l+O^!;HU z-DW$SK&^GmKEM_RxfRj_aZWSs&NeXK+6gXP1@jJ%q20IkVp@H7X-c+NzpM+qm+<~J z^}=(ImYCrHn%MF=G+)JW6KVx@gFUo1Q6~vF*6&Hy52wPDcqB;@3qjIex#-sy^%844 z#|4RKwgq?cU3jLx)K#LN{izfXeo5%_kvK*_f_(;$q&yyE5q~@%Fsan+t8Z-7?Gx*# zJ01Hqs2_Y!_3cYYPMX;tK0b2+p>WuK8qX7FDRgP!zmLL_W-Mp|ClNC|IdeuoSAINK zXz%bL5ziI5NIq9?n}_xd3vSW2t)8jFXN{&^*w*p6iq*_Zd*$ql=ZfWs_1WsH^0tRv zNBm-0Or|4pI~(U3BzOqhqRu8fHhv4H^=#d`MR6Mlj)dx%v18TPvE$YF@e|b8abq#S zPGST5F?2)@22%oWTz1)Ik>2^$^XJp{*3HY8KXF&cZuv8w+cBNRp=tC5BU0aZknd4W zPe5`*7*Y<;^bZ(4Xc_6wko(?c2%PC4F^zksk0NBC_C+WPro)k#y4vQ<laE~7a5O!6 zfk49QmX^?5EyE4c<uDnz5~;ZqI%(${YyUR<)N?MOXKMEF(-VUk`&#O2^Vw-*o@S4H zc#c2VXZg~upD+69@Y|O)Mg+-$Aka?{;Fm@$6cWZ1Fc?yDt|~3-+ZDAt0wbP9KK@_C z>UK>TX=%6f**t)cKVBQr5gEseLm9LO!_8d{5uBed($Qh(2Av#ek(&N02MHiXwM|dc zsR?la*JJO=C^E94zmbu|ymZI*--48Ef*rBnLnD_fDs~$w$<8FVT4PY1OIN$~mWY9$ zS8se1n6}-kJKCq?Ko0}FXw9PI-6AphGyei;MnZ#$o#eP9BV)R#I8QV<JNt{Z2dmD= z%^os#S4(X6B-tKdC0ZZs5JEvFv;sCNPkZaFx4LHzUYz3>FU3m^20zzS*Kk_ikeuSK z*abasu*ymr3X{z$VF?!fm>kQr*;p{)`|8d+@4W7z2Og}4w(X64cW`mF?s#7vTOR%X z!UwQpH1XN!^;p|>Y#I8~`PS{*SAdkfGGHaH7@A)I(*PhPk7NI`tJg)jbsv=7VdAT+ z`45!zVb`8)VuJ7gT0?dw7AX28vE?FGu{aWlnnc9MZ8+U;;5TU68Q11O(T^Zm%vr*8 z>^G34Cn4+-!#(zFkz6at%Oe?ChhFFf{eg@mMq9RTUpKC}cpl*VCqC$<Q3LsyHi^-? z#qlFT=z|{|<+<t>bmLn{yi_u6WpV2E4cNMFNDM;yqZ<U{){{>?Rb6)ZmFkw;zM_8l z>))u~-}iv}#y7vM<}X;NCY^jT$;im1p|f&LePi<rk3RnR$cU$VuyEn!n-cxj`8e%# zXWLHBRH2-c(PkSO(IW!AJa}%x3@u1D0a3N1zvuKZPPLckug|pvEh7>)au5-ui2{1# zE7vSAho#6?^PEPWd&~$ZfSXFazvl6L9BZ9&5ROR&Gp~4GmmkqguBJSijAUBN%BgB? zzMhBT1cCmF0H3w&%uG(xc0iVtV~{ori=sT;pRNZQW?u!l^Ix2@YZi9#p6c$s3U)r( znWB~zZGmyBILS(GJf<JR7>99t%5mC`J58IKk>~a&&B$m>KuX?k`F+>ndK^w)_GZXO z$LWq^4$e7X;7Y1(^>$d-8#R^mYNMFt5bJcDj+T*w3SOs>ksU>>d*Z?1eFGTj1sKe= z<mcp2Lo6o2B&%Wn&yQ}y_=egezl%wTB#IdUuBr-MXVzmYNXeM%k+zvZF0&r<TN|4n zMEeli4H3hYfRMDem6v@U@2mfzX%JiorjN(Va4Loi8bC=vO8!+j{*Obus)?Ft)Giel zzX*6*tgW^^G$}3hGicY>U>dhWvkcwg*l!@g$ypMd63%`CKXOL*`RLEtFWsr7dTiM> z8Ht%oAVy){CE=p2JGO2<HcmXBLks<tM8EHB%tWk4|0hi42XTTxcOno^Z&36rv>nJ% z*x!n+S60XEPB1@8RuBX2kt2)Mj2Sc4MHgS9uDkxz>Z@P-y87`?ex~mG!-MK~_uQxc z{afEr*L?C?b>^9~R9;@*M~S2&;S|+szxLRQ6(k>zatjtL*a5<gQ@T6tT85tiC3hcI zFLNCa(A|p+0^F{j9uq_&f%4P-Z5ovk)kkV0B2cGCd=pQL^m5RiMl9me*OiNIW+H62 z9QGoTSkw+X9YvoSUo1K?egD>t&w6oFbBkssCZ}nmBwpE>#yh3}{NL_4cgEAR;1yr? zg1|>2z$vZhc-vUH4o`;lNOD?mO6iW$jUSmX4kLh+{5GaM{XZ^1!KszhARLH}Hj`Lq zAny)X2?_K1D;eq4P3iXS4`Xn?7*p8nVHgBLW6}#6P<t$Fpck-#8GvSG`4`69H0=Qe zb~_iNr&#-}n?N%7vIl3j6Qg5Z^&qt|AcbnSbM5Hj;?H#~mBi{U1h5+JTWBx7gpU}$ ztS;0eKR#4pQ8D!5E`goVuXI=WIw}Xx|JR^t^{rrPstv7MZL_(fX(e9IA+T-7wpEz; za4txo>cWD2dk_{35*MEccqE`@TSM@%q9rvW{bF=vpQepieq<jAj%I^Mj$^uZEBXyr zqJO@=KVH{++qb_R@LOj=>0t+zAM{jvZ!q5<FG$K9R|xdjRJ1N!B{lkf?QRqCLb)O* zC+GS@hYpQvY;25c(umWmb8>Q3PHv7Gf-k1}IO;q#m3F6lz&TE~6ITCNvUK^^uP#~m zb1#RbOP?HJtCks>wDUYWxVHmoDm1wGrG)Bj=a9TJ8ig(qNEs2KBpKTXND!nEvM-X3 z=@G!xI+;r(8XnQeG`}WXQ;YE6-pdyQ_i291w;sy%XVQ=VhE$|P6OYB>(L2vPoPOT8 zX*Tl2N{%X};EXF?-ue8)bKOK--2m|zndO47qsG4Z-)~+CpOy3$CkXTh1o*|39X678 zxR<MNJKv50(x3W6**~m|Sft?ln7&;Oa_eraUYr7(l0Mp&90-nW78lkqM%=9zQ+#US zJEf)j2g0@C-dm0ZiVi?C<q?p{1vRyYEZT!&+dmK-34c6qRKwZ{O})!Nj%^wU*VW&y zVOv?*A90_}g$B(Hjm=Fl+moz5bO^IAlI`T=r2ic|a_p;H%eQiIL~*@@0NRWD#uOFL z!3w+A(mpRq;#gcQ7CM5hStppB`V-hXTZ`$1uf`%L)-<K%+kZ5+xcD`+dyf_t6lB7t zq>q~R;`$Z=E-*G}<fO^XR{fG8S=n>=;r4{RWn3M7vMrn-!7aGUMuNM0aCZpq?(S^d zbwjY=?(Q1g-QC@SJG{)fGjq?m^PK1X)W5wyb$8W&Rn@BMRi7{j&W6;HygmC~nO$wC z2v!b^u3^o-8=kK=C!Nu)*z>P9XKqJE1gOk;SU3aQzWel!P6T0-ji^P$oMBp1uG~oi z+Yl>9e>K`}gges|Xq0a~>%BV?uV>5db%7@q9v%0^fAlZaA-3aUBelTm-mVy_cO0>% z^s>K-Ra&$=4j`&1RBKk+cyIpxtP+JeN?bNChO5teWM<^d(6MV`Vtnk4@#QBeo};6n zKuew>6HZ2lgKvln{oBEm@krr5($L$gA2{p5qU?w_(S^YiOQ^kLZan)rMrnZ+I!y_} z{B9<9g>dqT+olI-u_u!lCaRO17#myv#?>&D<6G5T&$=;Gy>tPKJ!Mocv3FKk*iEr{ zvB>Sm-oZbJ9`TOf;z8|hkpvdz8O~K+*N#gfsN;sqvuz5@V3g)6AKwox84<R7H4mtH zjEQ%H2Ju6w0!crkVlF^Kd<74mQJ<diXSg|q+wuHenFjHGFBmIa7rOHV#wE`IuS$r+ z@AJY{*!)<Xc1p~wSY1N~g_>5dpo@=b)}@@s5b69p1^w#rFql(W@v*tvGl}NZ&`yth z^N|uzC|%-c0h}18QVU-L{>hqK3*Pd)W_oPfGcx5obh|dbxfk@=V8xY`vh=Kl2Q5rN z@1mi3RXa?q&*;lJyx1(p@3NeZbKS{R<xrnVkpJkIN5?Zl$YZOkb-NLHnH4-@XP+g= z_>78gXjo4%<k5@f1xHv}6<PsYx>3lLU90l;tmMml2)$GVc~<;6Nu3TjU_7V`9x zV~YrK^4?o~uwTRMXPR|pQ|^_WSIg-ScF(jRiXu!PzO-j(1QG;~KUO%2x=z?4eYd0c zYx4G#=S+aP?!hSB4M4B4%5>mS=WVU}-~j}8_nVyj>@8sjupgu;Db?s4a@vu&)j@$N zXulb8xbtf8F!`O3W$rZ%apLYr8WA!an7x!ZoqEGI8~4X<Bsxh;k*h7bZ{H?0Ju#R4 zgG&&Rz|{tV+4k<7Ek6p?#{;#W06O*W<AggI*YDpZRybOW<r3S2Rm}{@X|!I#fCJsI zkT0ni@CC&zMNdlC)f>|Ib#wL4)TMM{@yNBG5%r>u%;qU7S}s>?d2Z{M_*ENj4Q1et z35oJ0zO!)!ulC4##-Z)%1||pc$H($SlJ+$pI5EwZx^T&A(hO%``@6#{I()0^R&A_v z@WbU25>9<_gAGH2y`SG8tlM9^0eDA#VM=B{1F2E#XmKqd%;)9d9`<qWL}h9_-z8M? zcF1}T#t<QkL07#>^+Cl%-Z+IcXxb5Ux4($zOgs^H{^(awXf6N7@Sq*s)tiE=yY>ek z{?A(g8)oU!kzae}+w9R6=9`r?e~z_ulce(ASpJ?d&<~-;riUs6qFv#wpE$vEswoao zIUumjP6P|eradZLBe(s7BDcq@qu%tRPjAtA-xus6_Y2MC9j&xXC)`BFJr1Xi;^l=t z#C)ah28YdV6Xfv1m4}Y3tYTk>cQV4%qeZqgqnYai<7*if4&i<7BAjQ_AqgMsXA2=e zXq`<<g!V73S&R#SLT`VqyPk%n(B&#e`kEhTKV>TMY}VV{>DSc`Y`kvgDz~&>*JiVW zBtj4g@3PNM)XsFoes7<`N5<>T@S(5?sAvJnv>L3iNAA<7L~`GQ4BipX`kEpOoS;(6 zh$5@+Ha<CM{Y++)c@}DSt7`QIqai!ww(x0q&UpQTL8%dOn@EWD<;?Kr-OH_DyW@Qz zCi!7-`}a5tU5`c_nshtN%v5Q+CBZk(Z-_4@(o@Np`h52UUZI=_%)mcdD(WoxjImlk zi;E}bb9)JKz80rGgEFV5i1AM&eL6KYRmMn<YEyq^=wG+)-)m<|$(42-?%c04ga@4P zpBjSFhxjwLb2JH<v0(}fshw!~-EaLLJ(fo{v)wx@)@m;>HW?FMy;&-nl;uWZ)uOI* z`%MWXiUF2B@`Zv1h?pO8NC%ue@NuC+h~M0XImqczclU<8;^UPjtPtkTe%r@kgc8c= z;A|xn6_3gruC)Pva?j@-V2l;$T5L5Hcl72h+gV-{Wy~CTxn*4kp>e;kroj+PNcY}y zg0=B;t^sJ1OZ+K|wNhd<CPTJfS9%k6-}gQhO~5VnK-xi}x<(W5K4ELU@J=SCvA9j@ zweC+66V~Gy5EFYe4We9<l9;!`QdifUFE+e#QDVISwPuOGntvj+qNE2R8})Ft$-N`F zY>(q6Fb?u5YnB!=I~b3+Ig^rb7UA-I3;1&Q>c(7lLjh&87oNPYjaEs<Up-u(?=|8% zIoX7AM@pD;1d9yM_<3@2vB*s4(=rWQe>9#ub%Dnl6}cej8#zRl^)=n8+ce!_fOfGM zo#fX1PB$Y!Y%6!-uGNrA9`kGSMQLUvF8jIMR$^kN0oN17A7U!6!$JLb6tCFJf<hPK z2tKJ{S1kVE9BZcsdo;v$=i9FZ**={UuybLN=4uX4H%AnNB^4d4AM%~!TS6fPztJKq zF(Q_3O=VO&D*tq;cGjA;_}+ruYXFNeRf)wVX!_0SEoUMl&}2SdTV8OiH{laXQUT<Q z)z^3}H-`0(jvUJw(>FjmA!9YD;bOmhZ-p?d(ZR*k-K>(}^-oIG*0VWs)jNV_iu&O% zP10yjk9`YQoq_6_#f6?Gsr<r11#Dq{wa^*c<yOPWLjJUYbeVpYQq?%pGc;^$1g9MO zEow3{Q|Hvk8zZmnlG0&ssx3pa=~A9fRQ(v!N|gTQ&I2{bLY;O!!QL+u>W%c4%F!46 zOZUIwq5~{C&IS>V4GDg$J(~lAr}QTPPv+B~JY`$^JMR`09(rzW4hRe73@0cy9J$)> z^feN@x&{jJUdch`fI4Y%vWT##j=>p^ouTKI^&ip9-&~hPaNC*Oudp*6j$(WEY~PHg z9>kOJpuyTheHXS0<A7!!9DO}vi;xU3ew5Tvhlgc<9=+ilLom?vG)^03A()tOo~&0I z+M&z4mHx!)q$h7~jH@(dyPFmYo=kfB<nG+>F?JSA%Ni+cG5k_;k#~K+QV%Q4Ubo>_ z7!*eW#v-gVoPFaalsVPH`VI^W><R7u{)-%HyL?@qkGIdFw@RPw1ZkQ4;`gUlk$mo9 zjJU{~3W>YRfKMzeZ-MuB-bAmrSv>Qgsd-9hS{GcOFN#+;Kla317Ll{GC(ld7$_n7D zsh=@-A|faTR8uPomurlr`WmjTECp&tW&b?uZh=XQ^aQt_NxrKWMOm{9(N&P}IV}ER zxGx?m#==$Szt;2%TX9L4@E;pDZH@kMA^stwE<}A^pGj6w5ZRZ{>x&dzo@{e0Ae8DN z?pcVumH=lO`5PjV3Wsmh1PHxn`&;Vv{zsQP&XZ)}eWAyeFG5wFl=#<*#s=%sUe5=n zk&9LgxSrGTkQA2W<;mKh7MO9mF#U~mxVEZ^kQih{@on^vaTW<pS9mL?X>f$IK&2ML z=hvh8O-8JNcd&gFoa!cG%mSxHpI%Mf32IkQPdb6#H^Su)S<hKgz_YdI!uoArt?sJ1 zI01*<p)!jsWVJuWRZwf0hk~NRUaM3RpHH=QP^J)b3WlZnlMq*1f*^;+eRs)H3<gc| zMn%7hLyI$T2j!sjw*lTZZishY+*a#uvilO3^s{FmAcrVFDGln-x21a(`CQA?Xa+p| z#ijslKnVW9G$;4oVNsz_smMJk<%%IFF+5a8%wpWOtlW#$SpFP7T!xVCC;=Qdvt%I2 zlGhdF6KDCG(W6>??n(u3eURb5irn9xQM&T4@=m*a#9ft>&C~ltRY(fBaF3UFdowqD zKB|B*$AFrejPGI^@h1-&tLY-gYBj!)gck$nZP(+*K#xCNa$vRZy`PG)W^TKk5fI+E zr1uJ<h%0#y$07hJebH(RSb2BQBSUxA`I0y9(0QvT?dpkcqY({c3Ki-<UbeuD$zG7D z&Bi}n5(Fm(bQ0XoPX;a^<38xPc##+13K`()_AtI8ACo8Eg7HPXA{+a(Bt-z0`g{e( zCtv746z{RCC^)ktK14+EzXQ9kB;v`_xsVGN`##xLl7-FgAkXe(v{%6`yFW)*37J_U z5|)1b?(=>%6i*?ySNBJ!HPn!4Sy8nCL&r1<n{yw2)as2Y^t!a_v1lx;L5_Y`sP^;a z?jk!(j!fQ~NnMnFUD)JTwcnE(Vg2?)liMg0H?AuK=TK`C-Lki?-D6|A{mwq0HNTuU z+iCNzOC)<wt$x*=X%@6P!DgnU93qwIE~2=@ROg?_slBvQv^d^*y$vwMnkb)e;B!Fv zk}U0xrBNYei8Swpxa_t88J7n4+;tb%0^m!%Iyy5AD_CPPTSm~B^pcfFlGBv!Zj>64 zV(*_}Ueh$TAG`HJznCCoTVP#<@Uj_^lN^T$658_MQA?ZaJt(1n@)+px-C^{Jw9BN% zeE~ziHQV1ZkDNWn*Wh^>dRt$|zgXTG%M93Pa*$FjOWq1hOIFM7CbH|>20cbhayYp= zT_4sAe9TQf+q$`dwqf%S(Sl2+os&s&bjxJM<kM{w>rN*9qwZWZG&C%zUDV1x#_KTW z=Vxc_Bn0?;NgbO_<zh7o;?7JLq&W>01^el@3MF`QLk5{03EIUhh2(n54()=wn_2x^ z?B%*XT@CW`CuYsAOFNEg58q7w+4v0J@)(=4D`MZ*xEGO!%^4(@=5xkQZ_4K6yoalH zvGV7>`3WM7a)g$yO&&~&RWpoIq;U1_&<9n_=NZ;-N~Dh=U2h`1F?vgEt4kBs71YBP z`ip$l-nX*IYYN|)9Y7!okfm&6-+EhvXWc@0^9ACw&>;FK^HF#_tTsEMQVqK#E+%@@ z#&&$NRs>Uhsuh$r;F6ng@mjQrl@5;ck}=a~=xK<*ZAB7{A%F66MN(eLShL^I|G5iC ze85%Lu4a%KPN~WE<<J9%Z$qg<yZC(Y7@WxrDRr3)Yc@{UMMB&e0%_mD>%0XjAS{V5 zwusQfDgU)s59#9j2Hd0#Z!5nRSKU^Bh}D7fjkO|!0B=dune?N1K-7*`-N;DsM`O5d zAi|SmuIG~Bu;@KdA-BIi_$+EoC8lU_Ls02*KYD*-=2E~HJtS4rRx!FA%N<?cadExN z%D17lAIXAu>8{?@d-l-OGuhu28#<6--LfUqKLnZ$xvX>RgS^OQ-7cQnEB&=B#gF_V zpaq2+jo18H17;zIP!z#DLKt&HVzOyYNzT&~UEc-MxqXxOXDaA)l8$%;a^P}_YIQ|U zpwRJ=+t}EiQB|KCs?0>?)KaZ7$e6z(AkhNLg+8;Z*5lG1fRFmgfa%swWz4x@#|tB7 z${gC|K|g-tD-1BwK1O2dR;f~U@d6yie8>B9v<_n~IuXe`=1kIpD{qn?;B&80R({73 z&I$|=R*e<v7vV(Q8}isexV5fr{9u{Tovj%n+yr_w01ZrH50k;olZhDFjWolguXj1* z(Dt<{ncIf<c2IBF<*H+timeK}dfyN8P$`+@J|6@c88P^|m9P1c-l{Y>B<gSWRw{U( zi<7IW;46Qq_?0-k`#AsE6jV`CiYgby?^cBaU)EYJq#(G`1zM!y{7Nd2XuI>>BuYOe zxDT!pmT^Pb!sHCj#GN~Juh_Y4j3Uj$+&SVkY#P|d%d|TArP_5j!UGt|W3FED5&XhB z+Hu+0Kv7VTZZ*yM-1xfJ3AK`4MV^v7;#cNOAm8}DMd`}(Mrf_LTSSkHrNnX!*A!E6 z6w%fSqBy4nJ;~4Y^tbPAE|x%PJAa~vIfrtDj0b-k7VcDv<2UGad3>5juanVm@~TP2 z#i{T=(_g1=*$HSDT9L=vRX<GwE$(wddEvi)A8Ck8g~6-SceZLjAo~1^xF)&X7+;{y z*ervfho!h)yeCmu3ksRzdX!lS@pQ!%hA)V=zt#sERHg00faN4_>Y*Z|ZCk{>tb1bC z&S+_d*LoY=A|<UiphT9vo;5c7nO&FS_Tdrk*cPTvDkIoM;yy+%S%I**q9N1Gkl)|@ zN9S4Ywkww=jgK8n#yj^U<k8b8A-r_MPuq9VLbxeef7Eh<Wxp&Xot<z|eHYO6?Rk+{ zT^}%jc8Jiq<WoBuc7&jGz1J2bkupY3CByny;sp&9EMy0US!fNRZ3Dw{r(NMP|M1Rp z1fj6ESIAM6b&h+c-=;&;1I$yaiB(U`<*)~K6MsxI9<cLnW}zc;3HoKy<vZu%I}Lne z!?;*F4cMmX);25O<W-Ge)`NzIJiWdWy_=jlWXyP55)Z>8&i4Ks9L3<D^gg8YWVUc@ zW{%6W{PltR@i{&*UQ}@@cX9$&;#Oht?ia-Mu?hW>$ALDf^HG9^5{hgRzO%dc8+fT3 z%+rDb@*83ewEA*QO(BmBl<v$41b*pDaCCmXnu`#=%pQb2gLl6O&h~J=UP30X4l>*Z z+sZR(ce&msw~FA+<`&p?!v@y_ML%B--x-Dpbd2QfYc%ug-_A~Z%?7XT6N&7&2-XCw zL_?%@xzu8SN1lR|cV<emy^BjO>KtnC(6`x`YpOnFnY0%=^_DBB$;c-cweWy`FlLWS zXC!JZ&ae)Z%%-bh?dstk5^nZSMbDRZH?TTZ3`hh#S$Ah^rFYww@#{Ouo?qjBlg~>X zDYj6Gq#O^B#j_rThWy~4q6-<wkP~j;+m1nWLb4kb<y)>#|Bm55i4q=~?F34d-eHyG z8VJ=Azk0;6D78csEVVMLTQ!WS4UW;~^xmSq=6zqLr7!3FbJuC&S(DN@6ytej;ZT0F z_2|koYpUekRx_-Fj10jQ@(DicfGs%f8b6a60bkjNGc%Z$R}!{uDtvjeoaGle<r0D2 z&L>2Cv=!S&EBIXwEc;)Ow4b9uR6(NqIBuPX_nh=?x~jol{!JAmpA<REi#v|I%#dUT zR`j@V-mr^eiKiT7>Qg)bf=9H`LOpLbYfJJX^Lc=A)=R9P9bnFl7$~{|XB`aA`?HO! zBHway2D6EXpg*$zh~GnfeeUx7c_!;eZk*|1?33^}z!+A{pat{|t*Y3tYUsPwSYg*X zM%)17`C6ug%vxr>n>B0j+zs7GVKL8uUz4pqxcEn|za!c$p1mr&+o1V#!W~g2FY;On zuzj!LH@&`Wei1yvyOsT}d`I?!abO<K&eU5zd3|Sb0Q;j+ooHVRpR3alQ0ET!>^iyR zmfJa+d<07j;~C3V*R0w@e&kMv$aqm?jL>6E+P0r9InLYAF=-0DgT2!<?T`sb$2<<2 zfPV%eh8oOkIKBHxe!0k&&A8_7m<ux>ni!yVM`yNcPmU$bDZRVMj4pmM%_12WMrDIr zI>1*7l>Nw0!h-pkUfdR=<9d|XK&^ic+x;e8<?n4D!9R}+mzq&jxZR?FovCOMqUY1D z6lQo(6*pAa|Ljm$X=`;eIva>;g1HE?#%ek3K~Ogrlp4A#9u^)!LP(nLx8p$e2ZLk{ z7o09Mpd!<z=7QW}8GwDo?dR8J^wECb8;MAb^1SyogE(qZLyh)i`P>ZL@bO}`tG)10 zU8C9(BO)z~8Z|JCZdhvfUj684`aT?LhTD_8!B6Mq5@Q37Qlc-N7N`1H-}<9THPN`z z$D>x4SFa4b4N*G{8cv*0@Jshs>EG-6lVVxI<6>vdrw4%kN9$}IlWu#r!+Lj3(VE$0 zk0&lc;e{W5a6=729+}24?V?fGP4|c6hMAes2`dn8lMO2!m?i|+P$b54%o0V#s9HJ_ z2td4m%^#kWpcSI8r=g%Iw0eEIje0%aezw==l(%*AQkqhg18Yx)l#$dnZORjLt2EN* z$Fk<ALr?Q)`+9t7QxpQ^4aR`l(hYX?#umjk#_q67F=M>h!4(XrgHF?f7C50Euw4<6 z=l|x9*bV*7IO?}_gj1xxSsn+Z9ck`;IPVX-0hsJFxNHO~kxa}0sJ7OGoOuj3UD`;z zxlQml8{=o{GT(Yl>F!=1W*I_O3;FaNG{0<|FpqEF@zI(u^3mjb%aRh1W+T48nqBjh zpqN`^AGu*UsnH%|IOpCO^pDy=F`EP<5q30ojwc_H5KEl(uujIN;d+kMWrAu=HE?nI zeSvsUrl#;sgZhUD_Mh>!b$@3L6k(K(pK;HA=*{?cq#rt3_4TNvCjP8Xha;)a40v#X z0%I_Zv#D>0?sZgw$Z^Y?5N$1db}h(=9SGr7lPMB>mG%?oh~(O<tw*ZWr}A>qYve!N zV4gJj?F6-dNe+ICk!f=@4MeO-?}^FwYhvJ&TC7%N?JDRyUu&H^*KTFG#F}z}fLTIi zqvKL7Su~6qlTvflMuNoPN<xcV73L9u^B43A+&8Q<D}?D5FNz*L6yr?iA~`7{>Sc|S zsUjAzv^-(kE?rRmQZ1fgs3nv}^^@CW`D!T1%Oe|8wcY)!C3VblV*YBC#bycx(x0aA zB>^SOunc7U!bC3n0dEkW1qJ@cg$UM<4h8fGDXZvk)ndJ8T$?LA{13!Q6M(K;DhjQQ zprdTg!Z4Z^iJ{eXXa13Y5nz_XCcMh^Ro9b>O4;2#Wx3w6aLMId5eiYc-#W#Ra&i7u zZ1(yW^;}w5b8usBtQmQ%7D1Z;Dee`+cMY5k!)un89L)XmTg00z3QSKS;qHl_2M(bc zS{|Yd(28G|H=-MEw^8CEPx85|oB^}ll^Y>3Lyyu4tZ!xA%lA6HJaC7DEyl(T{fvaH z4l~inQOop~UdHqv2<h+pUqJbPFO$v?9Lqh))E?nZ3qj!!9}j-02Hi_bEP*CS9_RXW zru%l46q2g(o!#HA1qN>>*-fAcvUJOudaYm`_|97&w30~s!ziDQAbftu*iBD6h0F9i z;@NC+8%b{_G09OFW%LKybM4Zz2_d$4k&GYlz)qTRT_tRB^~R6!va=&;T=oK2khTGp z_I|<}*X6{Xh+$jbh&l4C+|*0#Mvl=(`o}blRL5@~BcP%(?xwS?E4zXr4G(`LL;}(S z{1w8v?Wh9tL`8i5!({(9;2{A7{Pi?%jfcqabbic4AmuXkZ)R=i8gnY9v-(OZZr5}9 zO54ROHk;K?Of|}t74y`XYeUjah(ac+iNm9#is~sT77Pt_bxv019#XRXiuV$$V8byg zy+tC>ns8g%+^+90?46Hul152^wAjB)O=(dF<inJ;+9Ov&H9wZ#Jxo&ybwRf5;=_m~ z4bHUYhKgj!k4)JG*?fKITHq#*J0wubk;n!JkCVK`fhY0II-CiGSVT_}IW>?I*V8m2 z+ca<1@3m<H$4-3L7<W-Dyem(`GG)xD*(8*O(kv8FVf0S++9W=bSNeWTioY_j9`fPv z$10u|urh=6m+l69V^g?{5A)j@kRUJMfX-ATB!*Nfe`l^VRzJvyT6<r^kH8qBtDK<O zNuUY{3I<FQ`^_6KEH9U=`<%$sU*0avoe&Y1mX@^8)V4xAyuwh?^M6RO!}qCX-8Z|t zQ_<7w8k?GS`P65$ucn;7hyWrkI84V+X);s)1c5*jw1kNdqxPIajIZw0RRw~2LNpEb z;YP4WmUk7M1Dv3y*<Qk)A#5g`L4NPUf}Sfs6n@b^4<cVqZsarebht@-pELsEl|b_> ztZWl5-rBNprlDEy<@VfcJgqrqPmgfP<C5ZOv5W@O!GP9{zVM10`yuh*BY1ngI?|w6 zQl9O<@sckPJm&?t!hJE&Uw<MXWN$U#<5*}?E!3#6DX#_V<+oVb{Rrsd&d;s;!K<l% zYxULQhwHdP*k&GBGTHgc%lUrbfm)@c`{U0fMI2ll4;k=UUWnS7+PTSWQvCL@*ci@c z&g$6cGviA^{t=C&1U=ehexT1D6!?l)#^Zk5#nzXnXyWV5nE2d5QTCF7vrA>v7=oq4 z^EI<z9a6@lr&eVHT<jxQiy4DF0V`5enf0yI<z>a$YquWJ)GPT8T#?5Y7~Jvpy@U8z zNLB%rru@sObh%fHa7@4L<Z$<70OZ+J;1H^<?p?!OEaCqHZvW?`kl}zFP(ScNKn?lO z6$gahycj{>@xZWc?a&xas-K3kSTS%%X=PY_n+o@3M!8E~hy~;|8e_(bQ6%77Tj;sO zIle?oe4rv7(+P%CY>P*KKRbGp$+j@N2CwKEs{aK$J~e`ad30Stg15Zqu$}J<_hO>2 zM!0q1a`?A=;h~w-xH!^CJed1JC?#AZ!ioj3Lu)gUTp0-v3hC|5>xHcwiR<dk2<1GX zcnT<${pzg$b+3@Y#>p$VmL?kg4F|FEpqb+@pX>kp=~qE4FupJBdI082Z`ca2S$M$d z`h{oN4o_XHEEoZ|A{b#nl|$=e{#5zsNO>`mAF8IR>xy%MR;S6X?^5#qv~j$3uKG+j z^=QUgDWm-XB}!uKbP6+pF#0yC@HLVyM@;&--Bs<lsG>hFW;BZ$J%k(<xOS6))P{&( z;z?;0GNEUM+JhXS^Awo3tK!)CXNGgKcXqjQtDr^A<#nebMWMQ0E1ketxS>2OvD5A4 zY=r8N;x8hRA1eOmsaT$=i%-oyf?1Z(@ZtzVhyvXwbY1-_358BXK83kI66K=sI2C2m z+(0<oK5aO>7InizdmdD;1AbLfDh>6g)yC9s8`~mqK9f2_aA4}L7nUV02%{U~G2})( zUn9~yWJD+P3D@+}h^6I<U04D31z|#qmPU$6wZa}HVm?Ie6fviyJ;W2IPKs|vd4(7R zTwYDsU?6cdruWdV<lDi&e(e*KeDgM$oSedB>fq<JHc{N4q#a}R>{lQFoG>vVBl3kc z7;Xd$t$)ag{l9Vc-=K{Q(c<en^2u57`@5th2}r5t9ekze{39D^Txa$iUoA7Ak(0_` zIWqbQ0!^=}-=y<odawY+?@af{el0jz5vzJ%x5+IhUQ<kL006w(3Yo2byU*}3;$uQW zFU@WFVSaWpL@mN3ysTn_0pGS6xniJ~2085Dd+N^Sek7RDfnw!Z_<6De#4I%Cbthg< z)01$JiUj6WI)7ZA@|qqo^)Q2E2Ta3ew|WD2w+wCFllMRM!u)4wwfhbr!U0h@-H!CZ zfoY3o$zfU@IUI3Q__Bd-bh+?1oRb1v1=i)N(=iZ3;5r|IG{;jI4fo?di->M-CzSJ5 zUkKb@Q>&JJ+cdBH^_*);_*wtHKx|(mBu?gLkQWuvA;xF+JLy=6K!zCGQ`Zp#UTV`m z&sVWRc6}}zCl9<C<2<NBA<=wK50;)#9^B&+daJT>jY7Z<MQnz%2NADllB-Q#lY8uC z;9%9-l{>xG-TzGr{!vT)&tKKVo0oLe`R9bi;R>T02G4Hn80--0!2UZ=tM}dlZ0ghc z4_;?4t;+BnoWg_5bivd2w2WNqlsZ|v9apIY$xz{nN;;)Na_wDG&~BsD)hs6tBHObb zB3Rg9CCwc5x19{zu~Kh$CnvkGaiI9@Rt8JI`3^?CR^UnTRs|3K8gKtcj5x+kH9hHW zx=LC86QPIY_|e0-9IB-MGtHu=R<_i7m%S7We>)t>;|y`8$w-{6ZRP!zp-LZh3i+R2 z#({!@j4*^+J-?R<>31%qmt`sL^F_Wrs=RejoMxlUC&-$mL1ZzbOLY9~o0=liyL+cG zZQ#Kt(uy>rL-z46Fm4ES<hBlY&npczK%FGlW&CpUpkS|?NxLwKuO0;MgG3iU=*4y{ zB&TDHa%1>|KO0GlF1+0o08$R8(n(1G5Z`71tMOw^lwF%NgDW|f(0)_BWPA9WO$Xib zhOe5DQb}pr9Dvv~AA!s&EscU-jJ&ie4?5m+tXx`uljtBr7<|{u`U=vbeYUZ>*N5v5 z9Gwa)LXn`A`WmOTYDCe1<V;2e_h$0wwLlDd%|?Ne-;4umuX4tvELsv`HvW}l4$XJv zvCb$l_^li1Z7MK#-vdRKpDUeUDy0{XFg_zki$sCK&I<Uk)B(k$=t%OEQVKOYvgEW* zM(+Jm8-IaRW{Br@=NHdXq(0s?qdD=RrH<DU-2dZ8|J|#c2>kpc#iwO=Wzs`(*U(P3 zxUp9-tIurN`|Wuuby$EfgXO<sP<b}nG?g4xuCo}QT6h#EVnf#R72D$=@&_1jfH8Wz z*fFerq2RPV*M?cLV*b+r*m{cGlcuLg3Omur{IB3JGu}S$rOQgDau(CVIWNcQzYe}p zv<ZdmP;dfolz~J?&=c4;cOIt01!eIY?|s`5kufi@e|OIqetzQOAdWl}1J|YyVk5+c zv1-`X7Nag+zRc_NA2aujPR>H6{zKR1#|+<bMN$0n9&YNaPTf!AeAb;kNOu|2a-$D| zyv;n4Y$A6Nx>B&sQwAwduOj~Fe{)!fEAyu<P3lA*(k&KDi63A4?U>k=cUi}xZZKJ_ zdWcBh@mIe5?{bAJMBw{D&Vcu6*S9UCmf{3kf$_p>=V3(By8ANYkqj(T&8^ymv$TAg z0~$9Or=?08Dvg(L7|1WhYgB3ojakyLqA|;%KVJH2tMnh-L>=N%K`(Q=SqG*$O~U!r zJBQ?K+z$SBse`13gFJOKY6VFdfw_scl${&=<cw4AUY4c2ArOFLp%4Yc7}w6%?F9P+ zx(d-7nkUGROrGqga$p083&(HA_z+D=T7fa@*SVY{LmTaaFe!Yozx8RZz~2G@(&kur ziJ^15cQ|5v5<n8&>j;)+y^jCG0lj*|l{E5?rBqWx%E3kvSvMXAZA;724-Ma>S=~&p z?{liEMPeN+Qu*ZMo5Ki{=59PSeoa0fd8IT5EBCQ<$mIYTj&^sPeX?-R(C#`4_$tHG zm3(}bbPPZJ`$z68TT;UHK^@o!lQRekTck+m%}StGR;nv<IN#0j2oAqQ+MAY_d)tn2 z@|gCDzkT}@(-WPOm=|5S^D$=gpBVZt-xml$#xe`sO~G$Djuu&VZ8MWfvVjdOrzI-w zPWsW{EU{>gzktyg#bkMnn=Dqh*JQ0(tkLL#&`@8}(Lv{s9aFwh%*X4AHps0Q7|cg1 z2py%6hV4HQNHj|J2(K_`CnU<GP1+tU1Y_gzWqQ*$tU}^Dsw5J-Hfhw|$3Aq#G7B@S zltMt0Lo$1imF9cWR!_m0O}9If_q5o=H=dU_jo?(8F_cQ*{DX<^n$}7Jb+2oXgIgHt ze^$l+j@E}nVBk@)r*voH-B5<1$COB7vLAyhFB6V5D;?vi7qq=jr%u5N;1|LUGBPs9 z?e}b@QI)pxmF-XAB^4Dm_g93T#ah~o-pRwVQea!+{)|*O(b@#^sL_1L5s^EBM{%*z zTX6|u`5{0jkMQVOaJfh*M6E02>>Jnus0k}8n{qp=)=A+UFH~wg*s&5ipsTUTl*Dy? z%mhDGx!Ki-*zg9|T9g4(Us$R0k*#&Lt=Ot2mTubL8ED|Y4CEKB?Gj?({khtw-Dq!^ zm@2QLNBPvUD;3>c_(upSysx-eG@e4fW@E|pH0|-4D|>OdW<!Y`$4#8t{Nh(tq>7~q zn@+bJ^4H~15t)mWn<wNO0)bm`@pA7&N!x4Y-W(|i53L;B5&1YSn*_Yv-o?Zu+>A5~ zX}Z!<O9)~Z1BDOb$kt<DPU2wo0brNAJeY^WVs9h)8r{eXrnu7WR1a__8K~i_Jy{oV zMSO2GKC7*<fieEe8t|XKI2rtpv!vUd@7-~B)#8-EY%<z3z5modDnL{$<^LqKZO2je z4^0A>i{*Tm^Hsl(?7TWxOXYlEn}xBZ)22*5VjL?w*c38`1fMb*B6}*0#gjh540udp zaZn<L!xT#uZX)SQ%DdyP&?W5gW}Ok7oR_R;F<ch>R4~@CBkk0O(~WlYT?PS@P4&W) zjg$ETY~PZMR7Q3j|Ix+Jc!@F<-WaSV`C(yCS+=UWi^TpFTwC@L<!>DCK^y{J={j!e zNd}?<)CQ+%;ri2UG`adXLbpR??*P*&Nn9=(;ZGwMpCxnw(9|=7x82slLKz($-C`mG zR;7C7^nTN8qoaM&3*Oc9&XIh-n2?4`<=TyJ18PW&UdS78H2}cSZip>>nk4&-y<sMK z1&u~FeaN94cJq~)l#4gF+uetjmKL|vGFsX>7w;!i^3VIlQOnNy#XR|88?KkD^@DTH z_NZMJ;+8gzq7rFoBn))mlOOqCCnZ>i(2;ahIt#O_ETv$EL*p&qBP&1jD0Dvr22M}M z3w}O);0FW4hiBq|(zra48dh3LJw7HJOSn6p(4F5P36(H)y^4p2#-dc5s1Ivzi6-VB zXf2}@Q%<BL_vd&YcjJYQaW&?BjecR3Zr)AL&CdF|t(J?k9rHqq6A(`98+tOe^jtiS zJ<oBsTFwpie6bu88NLbaGJ{o!4}TIn5fm*Fk$LIbm=m8BVM+xRI7u5|4K)<65U3x@ z44I&%XAs!=3FujT_kTA=t}49NGCeuRwWOklZDecTmy_m;(tk@3WClMHYG}mc<On~A zTov`7?iof{SkMgtUw8f6T7VDib4=@FsM#wO2Wq}IrlyZ;Qfl731ci(uvq?sUYzmJ$ z0)B}MijO$kyx^x7({Eq+ekLURIYn2~vd?QD{1_NfH!Iz*-y8{Ff#c7}WFK3OjSjZE z864>E@82Oiaeh|{MpAVJiaG*`ZdQ-xZ76wYMUXwk%&9DZV`x_xdpRuY<;OA$!R&2q z$^=6a)dXFGhB2X$h=rtjjjwvnXV3{0T}C^yx1x)&+wpI1SH_tXLq*z_)_cpt&Ojcq zkv~6VlU9u&GAA01Z<5FjU?0*OB#1lz6%GCyAqQ^-AuQyew@|5-PBb|K4bOrgv8@tt zhGQjXDAhY}9X;d1ZtWpvi<X*^ku4g9lOG&|l|3|+mWKE9SJFyC%FlWhWN~*6+rdrX z*e~`AfSNBQWpkN9UpV?fW8kdDq`6Q#*h)h=B4NdxM%0{&%JF7bfW>=<x5dU=d$K{h z9DY&RX`K1|$0Th%&~t(K_GMp^)s&=8s*6SRGIFpRIeIkW&B?l#a032w@qw4+7bdFq z=(5~oz57)FrMt(evgKI_A@F;5kwDJ-<9^~Ko{V#gFl7%?Vkf)Wk_&cnzd;7nm%TA* zaYWVNU&k?uF}LEfyNos~YZp4vML-iT`@eW2|1LuY!~`4}2awJeh)K!bL&r9t)C6s! zdcUc?R`j}HX|}gc%@g&sN9CErHYVsXG*;;ysv{uWhB-SshvP~N`E3T9Le2H)@@PJ0 z(@A2=pm{9El0HOw4>q)KQGZ77UaljB$q-XMCIUx+Z0<Fv^8#m8Jc*Q6rtPvvR3<2r z2$pNP6VW(Bw(U)5MvCS-7(R&#QKbF3qDts>HsJ`kLP2(Sv-`9}AN@FOSljkcgZZD3 zrAPFSS`fV1Wp5e6FGEa^$U@wsh}R!nHqHm@7jar!Y{o`>={6YmYQHN>YyXpn$giK^ zL&hd1B_EZpQsG!~@iEiMT-NeENuGWnp_98n`%Vn;1AI)Vqmc4fboNOy04s43P^*{c zdmIT#_u7s0u-%#Mu3lVvTEcFgN>r50bsYpdf{1Ec{au1WZDpCDD$D0Au;yIZ`j@a< zs#ToG*!0MvJ;gdl)uT1C+WytH8_55KPW)FW;R6FtiQPr#4~vS3=LxhXXS3&x^xe2n zGwhF6iq|*1o~2Y`%=dN*#4CeiFS!Ba58M7yKTRbxa@SH+xF`~o0L82mLEkE}E^01M zQ~{@q3`q$B@h38|u_~%*+xO}0xxVqf-W2Tbw{y@>29iN=h2kNG6sUAOU-dMj%Dt0b z`$mp-p-%X`O>&p+A9`sqO;hL2ufuD)66GL+aOnRgYGipseBmM<DqX+&(<629bl@oW zZr@jve>@oHvhdTK5dZR3970?GqbGVl5C(NlnC5}mep@2ha~%7S&`BQ{&IRHXLE!s? zvm`nhNQ81re_sJGJ<JehL1A?qN{;(slgIY;st7raL4vg1gd;AsjC7B=-sGFTLGg!J z^eloH;sRzeQ?mkchx6F*X<L|Soj=-MoySiuFG>5ef&*)@B|bIMcEGse-N(lmo@WmV zx(_qYC~}<tPV)bPGU0&r`z$AJP@SAJU-gb<a`G=Nfa?`F`Zn>g3|_U@>l9!S@a2_K zMM(%XW}7QQz{CWDg=nCJ(XdLN-{S`B11@Lv8AEF2RF7DH{p8oGgb|%EF+x)_GZRZ< zn0<?uA{DKTCS^uBjElx#p{}3LrGps~QzET(WTmAHAxOQk0tn401=Jp&;505q^NvRd zTrtlszf=RDTzTWzj23?;?EmOMF$WY@R8j+9rE_b{0~sqcWk+f>JA;cd4gdb+6AJ77 zMg^4<-x6+SPP-k=uC~5}m(G$U=~U==-=f#Mj9J|7<J#Ut{4C-B;mmswpOH~<b#<-c zqvN-ZiI(im_cA@*ViG-5{dG2bbw>Ga{ooezfu(p*DArRCO@hY4q&iA__L2lADzbP6 zOGPYQXMll!>Jq;B@bOuTZhBXcB%a^~lJXcgA<pGNzI-Z!tn1Jru|r0Vchc?2`5k45 znB%-G5@3w$gsnA1hdtyFcTy*=151o8D2@0Rw)&?XIdJ?w%ZSryw@~Vp-eXw0P#(8% zjy*&&kI&+&)vPz!2P{84ogAk(ZuBSE(1r5;(H8xXLhlDt@s){>A;KN&5P%HmB7n@Y znZ#l}oaCmeu3kn45?!(UEX3yh8k7g2T^NJlJ~$X`XSHIvJsGtsD(Zm~5i6K-j*wN) zc>BnB6sIXEIc+X>e9Ru$edkm#Npd-(rRl8R)NEhQPq~MeICup`p1&LEQF^4eWaM1| zXjw8tD%?5FiKPDvQu3eliXSTcKOXAID<8F6|3FCPX?l8kmh)hA3b%g$9|>iHmACFs zo28QBk&#)tG(*35A9qc79F`vnrHq6)(W{<;hJ50KJ4^1T>2USRx3M!S>lK}?y@Wq_ z)ePg5vqmmca+*>>%M)a%xg%z6Sdtc?@z}WFw%%gug+Cx0zR@n3oY%FT)2@JkpL26; zirBbusb|XOzuqo?&kw1smcdBu$J!4!N6D0)N7$tYqva!)+ZUh79HagXQV1YFd;$Yc zjfL0D?jmg3M!L)jUi*UrxN%u`$2?5VmKQ(U=+r$cE|LenuAtsqxnJ<s!~?3)4XA&! zLHt6}tZWq^%BrcUQ8rO+7#<sgJc8byA|sR#Ag5^TY?7|TtV@T-4G>Z>e6stg(Ktee zfPhl`C}!tb1~<XCTRp@YR)j4=-@6!aBQE-Me4|GeHBZU|-#myMKMk#~oX)>P2JwOl z2J_}jTfS9W&p+J7G<&-0z!%*>L~A8YwI_$o17HMriXDnOqgYFT!t*%#o$;u%W?J~i zOSn;IS1*EzHw>k71NMIWk4EReOS%t7^+QGche5d#RS;5(q<*Hx#%967!QHZ5-=w&! zCDF{nr3u531-ma^Rqq)9U{a54cnAWcM)hi1FOz3Hh78@usg0O<aMsFUp1)Y72}PeU ztaQGv$f!mAVmMEwiH+sut;cb{D=HmNDONnW{{+pgC`Ee5l2Vj1lv~T5mw8}H#2g4% z5?IJQx*RxKT1GJ@uy}7U(?gVD2x6LaR!<3txK~qU_cN1@4y~IFt-;J_YjpeI^I{<` z4@WqT{I8CalF%4a#*yNj)sgpUw2vDlCA;CNf%mT~<*xkd;XOZj{<>fGUjgoSSoCjS za#HW?OU-TzM_#IC%}5geo|f(E0acw&x62o(TpK(KY1Jmp1lI|?4@I7fME(}5M?pKc z#v5tGc8Ypdh9CIYEV9P49fcCS(?P*{EW0`^pO4}aQmcfmKtXvZL(~@hFJ`!L%SrkV zj_#uEO>8|VDLOg?Q)<UU_Y@Cs@$ulLzfv{#6`SI=inMLZ`0^i4N6c@SB96e+g9;zN zDp{n<qr#Zds*Q;GuUYWclS<Gq>+v-l*pUY2eG&d&8I?_8u`Ia)oO_HzGO~CL0jbwq z?rTSy=MFL{wbjLRCT{AJ;~+R2a6cD0)0sZQmsH}JgtOeGJ~k%MfD<Y15szYQ9`QR1 z_7NzGJd#qJty}3iKA%O=Neea`bH?y`7l%_cR*am<!bG(nO*luD3a^K9l&{@lWkt@6 z{)Nbg!TKdFs)eDVkb+-XHy5*E>3%>CGH%U0lzUW)!t;M`-^e;ad|@FT8g};TXl<zl z37R|#GQ4hp@<%eYI=mhO&lA=Z)QLLM`Xcr>AUw`u@MTadFT3B2RDOAvki)JFi*HYo zA=eiYBOHkp-qX#9e^q=xf`DeB$<0RDDvWP}S=}Y)-IK>k`%wE~e$ot>nNi%x=|nMO z@_cU=`_dJHQ@xxIpU$E4==HQZ7ABl8iNI?4y(XWS2Y>GpfKYf9@psh72Y_V6zVzml zss_+_w6t2{w$0?1>(`qP8B%AsSv_7+f5^Lw;KsG@i%UM0J!W>j#p#UHg)nikJQ&TT zCKu$#gRuw`$q7POC{Et?6lZ@yiFk)*ZXj9v#O^`Lt3sSY;5i{z7z2_3n-Vwn>1RC8 zij1s{9gs~&X-gqpU3D@Svfk5ecN;cE&qJ33s%N(D>1&kf<z56;<rcK+Ydwhn9S!mc z8EcY_v>iuQT0RrX{eirR!un$3`Xaa;DR9wP_J?-EyMPLQ9r=QPtcEq`yrpD*hl1jj zz3Z|q`|fOXDCjG7n^Uo4`qeup{C%-Y&J7&>z*X@vabfX`_r$L&4`GsVt@Fu;_?sOR zpccUt04Ivyr=H(uH4p^|=*n}AX^zNaN7*uj=s7ncicM2on_gt_#|AYCul9}s+dEB` zDLS#^V!1wC?bSuOQPU*c9Fg0@-7z!2U=mjrS^^>9<=?f#zv_zL0^f-!k>!IPA=5iZ z3sC>Tlu)pNQ9uwk>vj3#Pa0^IS)Ei8C3@_p3i2rQU62fR4#sLYA5RUa;z0<iggYlm zWluB-QF>;|sXuoo0)aWkbIT3Rq-1?uU!fszs)La52J0OisLsHlKwnW&?<wjVzGRtW z%2vOYr$nEPK+q;aY6c784dB2w2mOF)VJK6LuCg*!26jfBOoyu|O`H5&ZPSepMEMm< zdc)Ej3jSgV_8~I!ty^}x*Z3W4fDD<HW}Q>c=_&WlhAABhD2-PzF(#RLHO)P@rJq9S zBPaTrEr~ptokRod#^$QwXJCv~dQm;;su_|l@B0R%>LD_Ky)SaC=&mSvzvvfK&+IHr z9QdoU`)~qUkLE7a-$BAb(JCI&`RNVbH$nR&Otb`kLT{19YBX9Jek8gffK(vOG~BM5 zk55G^L1|q!5stSc-NGWqylp0ah37od4umngFu4mAjtMn_z52CFr%fjLWF{!~h-;$I zuA-V>1EwF?I@6ItXZvDkO5b8$EgP6=Cv+yodh%xXrBA+M;wqh&#PhV2yQFbeJzA2q zITcTO(6SgwmuGuljNs4o&_4zd|2oC;Lq+lXY%9dnqWF(CLcZMNICy>?|FUlM4!GZb z9&rhqcO+e;7?FjHL<}dX#39#N5FskkXCtt!wr_Daf9|8PD~iR04Lm^U6>`dw{RT}i z=~`*!l1MB~_SjjocQUw`gheIh(<qnm$apM0nW@byBY1=`0qAr2s0!qCgcUOlEQ|b~ zmCWm?+5`Wp7W`+B-GmU{3?v~{?<6C;UU)1|P!eD-=lFcWeNNx(G3aFc%o=k5@+ZZY zKu*nxw_{NzMigN4SdJ$+I=m)fB9IVi$4_5J6|IR!BafFSf6tj=_63SOst=1w$Jo(~ zOQ<06+UXkUfQ4_#WjfhVUK@Cv&h+qkGR<7+QaGpK+MTnTUQz!?m+#*vB`o4MjF@;$ ziIgNApWW>?B%HU;t$nPpZAQ}^AyAFrs<0?^bf`d2qm{86yqgDxl27QWJ}!!GX8?@| zoY#LEJ?l{*bZQ(x;ofa1BjjNFL-uqub37nU>s<jdU^a31ix6E)&1}b5Np*2bb)BWe zIOob88H)vIrU9zQ<WQMD(lb3hn>Dxn|0g>^1;yT2xL<Y$$<foL@S{^Fjza0z&0i;Z zZ;;3{KahR^=vK8`s`4zE1B|4FD8UKK611EUi-#vDDGPIm_n34tm<27LFXrLI44x4f znbXs>0F+E;qLO26T-M<0lc67$A5pD{L<6vrac-UUr=j%&EyZq?jkycWu@#IJ5wTSM zhO$W8gif0bBt0R3NK{g+*}N>3!Wfsmfs*}_eZ#-#lD>H_lBvUIB2YR^%sdF!-Q1~n zFV1F>FQU6*eS#8O@Z02xHKx8G?{c;i<xDc|h=q<-95|uSIt4%hBoyLm#bPV(EnDTy z`zaGut1MZTw&i*7dkoXG0~@y|F*U89X|4*ar(?pYDE?*9|Hd-?6BpB}prLiAW#<A$ zYZ7UnuOq7rb>V&KAWEHZUcN4eQR(o8{;u+#D6Z>=wcU5plr2KDAOSg%?hF`mfx?!h zGOs?mJv%83fN#qibIG%p^+gSBTKXto(WVjE8%ng%e^LRKDtG4CLzJgS<x1R58_QK{ z&*_nh;{8L^)4qj(RVDD9bZoJ|9c|=*?aKenNk;rkytG{G5G!vKt7G5Do$2#*q<6a& z%$#7I8J%^MRbAlQLq%@CBzx6#3pRl9g>gR8!Ep|&)<JqTQMGvZG=<9fKJ#bHP38bw zO<VBBj85vI5N)he9y5Bkt`HljjwO)n#i_mpWep!Sd6BDG0*HofJ{_MtDlVb?d@Qs` zA&`(e4MLrB1ZujV5o6=omnf=OMl#4TO<z?$iKFgk|4s1U>=pIZifqC{cDov)P&(1n zApb$UfwfmF7LbVp6J3h@<Q9Y+GKMp~5=a)NoUrw2unBgg+J&315Djltb;Q;>=-rHB z_l13UzEZrJnANr@ZMm_YjZ~rw8aE7(#NHdnNq!^-2Kp=}#(I>jgPOpFJQa&rUC&%| zv{m5xDns;2Ij=~oFV=qhCi^p<VDi8i>UgdUw<A{+9U&sm?O>>NZZbhtDLAxJQEI`q zvGeKo6V%o%uhpJ{pu@NS9T;E&K5=5(^ZNc)lH>iUyt1mbX{)l-kYCefW~|N@-P~B_ zG*F{NyCMlmod{b&4JiWcOLr|u81s~(H;Z>}^}@1M2D2se$GrP^41s74@i9Rq>Nier zg<uX<=f)B$qTHgUB2)36TT-pp<5e}W<|}sEio`$!s_h#eMP9foe>T{bN26ID@`{Zn zC@}a3OKN04BB>Wv6r;rVZMlJI6xLcZFzj)DQP(i>uORY|EtiC?J~}zh7pg@lM@64$ zHH$H0c1tROJHK?QG-{94ogE&u)#yfk6G_o}+y8nx4mfe24nO0w;{UDcC-XO%5d}P0 zDix=Gg({sgt#%95n5qg8Y`{eX=m%JA5{p`V*UD=m1w+_bY~UXri7MV{?j?QrE&g@q z;%ht|%F=p(-c>dOtU0Q{d%dlV8h|ODm9p8#Gpgm$vs0x@`D$2lj-90>mG!<vZ+q$} zv3#Sz1}K0E;v(=7XL(0liK#vt`0F9n{|YO9pJN189QwfD6!*Zw1SE#qTKXa|Mxlyb z)PRC{e8xpwczDGTQcUue%pjt?{Ty;wlwAkldh>dZTxHl<O%y3@6hD}oL*zby`H#4` zE#c4zf+3S6LIGqwya{(LhZ1kYw*40~F@qWwo>^W)_#H3uYS&?JriPUd<rq;#Cb(aJ zGlcbp1?;;x;^DFN`?f&C(nkgPz=TzA+eip_^SU=4lW)j;N#FVkfFi}dElBw;BycQl z{YD;m5P1k2N&<+&ERqb2_q}lwlB;k%ColNKYVA}(Rc<x8{emptp#PH&bkF=W%iN`z zYQC0mvbPmPLGd(KIFY`YUa-B1yEt667G{mi_BU`1FAfnHix)e$TpjV$yAAoir__Pl zm85@r_TZu)8~4mym>b*QpGyq4#v}xzVEKJpIUYzC0Bxgy>bEE|D4MHcKJKh13_;pN znomx*RXGzck%vbc4}#a12)7p&OoV0rx{BQcn(Lfz&x?6v_F9y$Z|0+Q7;p39gJrfk z9}=277|%RR_wc0SxY4#UUVOvHon`3zi;<##vabwP$jILv^yW+pD@`u9A3N0&D<J|k z9Sc$K9sGsNbCtD^?cz(fK6hPXkM*08fKVbuNszS8gwJ4flgf6@1I_Z--*~zw0VFcB z4J^SQ`hq+C=q4q!&$u^3n|e7KuM2xRvFj8_nO`|gseow2?&m{6C1&FC#`$C-+rs!p z+y7r*Ul|wIvqVXN07(cM5*&gJPH>k5cZc8-G`KTpfFK!sAP_vkB``P)8l1u19R`=+ z4m<zd{k`{gcizsY`{_=1byb~Hb-Hg~wU)auU+R{VOsbJb{P*hTXDcoBSBpSMn<L}n zWaK8qopqU>dS$oay<se<uvbX4=B7dbD$DvgV8jxTBiASQ*05CeXd#D?b_aju8rm0= zT6Gp98ZDPWc|tqefSJ&Ptv&d$A$`M2eWjdvxu&OIK{<_2`oLB$Dv@_CY$IL%>0F?F zTlmn>!{fms&`)dfTKw<U(kxgB3;n~4{D<}yl9$xf%!hOONpzAD2FVxJHB&YECE{KN zcBQp*T~jv>rP>Mv1bB5*PNgrHnF-J>am@f<(dDT7Ex5QZ1G3x>7hrh7=)Fo-Yg@Y+ zjTyc}&?B-3pz$yi@SWxMC4{KyAwpLe@{t<l36=y3D&7|q^nX1{D7~PjCdr%zj4fT! z2A_!<A8no2H9fGG6D>bC{IiE;oMV-eCBD^aq0YX-INMgZqa{pnY(QRJa0KmqGGY2t z)k@Y?FO(b4(wP&_EOATFdB#(qPQVFgI7#_$MFa;ayo+RgW+v4VZkvN`^2P&n5^^J8 zd;31NDNyTCidnx}O~>|Of(|>u;Tx)F$?-6oXbtAeoln?Xlwbm`-}6m7Go?S?ZpAh^ zuAciCmo}LP{m0P%^~lNe<%`8uGhh6mCmDq5)aUCoQBzCj-lrx|N+OVe1nM7LNAtNR zW%|AeBmpNjZuDd79lSTJVCEL^t}W;nKINsbsVo>7YT0sW8lgdp3+j{`(!f51nf}rX z#UdwtI=JW1BFUuEujo(~`TQ!eY56y+nhFNTb$Wz<rl{vT3YS}{fq1{~WW9?ELN}tm zC+;UU<RWla9nRaA9NaRvaRx3OKbHXKDZM-`$B-t+jdNz><D+E!^8b4&5c9z^K*h32 zf4C*U&+7p$!uGhERMu;9B<-Z}G|`!-Pj6@-&E5xNd8UU8(Vj0dI@&5v*bjQ2kvc_? z$q8!=0b0#pVqjawsYyK@q%Y?<Vdx6vV(y(DK*@<P0clgZ%}(8-c|=6lvpEkI>MT)v z0ad#>6Ebk-rneJ35Eo!2C?q$Vy@D_2N&y_~RpNVGJZURN$;;Rrw=-sLfO3(=8s<%G z-bd5>7RNQZ-uIwsLt`|GO;OhW!=L{5qEPoe#fuYZenY3RD7JQPY-~WQSfI02<2R3C z49J>0$nes2bUaiE*IxZe=RXRp@eELY>2h!DyGy@O)rraS=6$=V5*RH&=C67>+`WC} zDR-}pJ=#uuWmG)A?LNrFrImPz?KQ9!O9f*n&A`QB^Q*hi$jAqrw9qO4yVf-=XA%)l z&_ZmTo}i)z92*u?siN-1VQsg6GcRbHD9+36ZG!13WmWn^z94Ga$?K;G2+2~Q1hK!7 zkT{tz35t+y?zY?0>MJl#1{PpBxAgA`ZhkA98_E!_oBJSY?K-gEQMM~7JLfT<P$Y8B z_1QJTLKBC&kC$hDRHWa;f1{p8Jaz#@or|w;Ptn4r0D5jg!S$E%qEob62SfP!V;De^ znQ)pST}S_j|3MIbRyP8$w1YRV0a$1=YI*3?aEAiYMMS=XrC#USe`U-(;!d7T7`NT{ zlexJHv#)~w-D?Wy+mi;k(e{1#zhe8JwFr_!#5Ztvjr)2kU9<JR-V5}0g=R$XuAZL8 zy$#-QRjz=>0gqTn!~cAen2JvVp4Z^W-pzF(-{wa3v(m}Y`$fR@ET%7Exi^%bYsQ97 z0z-vlI;MFU%LMd6Cf|ZOpSV!bu-Wtj>4DHjeh}>VYzj=Hmvz0aNTPi2!!Xihn?gQ< zjV=i_awGO4x7NuN_r%L{O&R|v#k?8leg9>eq(B|lG4x-N{2#7lM);5QZ94?<mASkW zrK<INPBVSPRrgeq?NS>9mzSypS8D%OGe18ar0LZC&J$Ec&+!XmDCrfqW27`Xc{I8# zz6VbgO)QR7DoLC<i}gm*WrU3-^)@5(bL@G<QJ3VXn`?|5yrT$-0?{R6v8Bd5P%$n! zfma{B^_a!jZ<meKOxp#!B}%L?6^hxcDthhRlo2M#ouQ+<AjIi(4xWV)hjN898$Ee~ zPN#aW`hFmnVn`ahqgg!trF->Z3b*ymCUi3(ImEg3Xj{}sd&ogCA3~5nhN-Mq-Q$q3 z{P~rCe2_fO)4ne`vrq{&wa(WDULn6^fb4i2*yw!FM3NuaMUo3<Non27;j9qbTv&(C zXuEW&9$2R&w019K!`q338crsa)fAvS#pAXhF{%ji`Z$;>{K@y3?g<HmX5G`=S}Q{6 zUy1)eZ2o8Gx%=m{b$4OHPnVD=iNL^ahs2Y)oA>QlJafi`<RXNebp~&UzPrw4J*fFw z()g>4^USpdiXSZ1Bp~+Uyl?bwwA)fa?3cpvw3JhzF^U)ck#CVV)_JzHvGK*^-3DWJ z9L*o%E6Q9EY$*omay2PjX}wRem(rWL0^rBUDsz0+AXh&HE#7h`C$VXz$uwQ?av+)> zYxlbo^;u2%`RX**?AI{)jpG|>diSM|_;Ig1LM@?SJEDZ8h-c1}YaX=!1=*6ZeyD$E zw{0ThX$Z@DyO3gS;!I{fQ_7*=^!!coQhP;}^|f#${_U-;jm=NdMz7H!i4k>lN@<kJ zygYdBM-AWZ(>n5Xcqf-RIHk=$oreXGA%m?%;37jpkl5BM!H|pp1Dgk*k$Q+^Iq21v zwDTyIRIMYC&waum4<k$^v!cr8!txCUGqwKlO+Sy=P6W0iXdB)Rho5zp*x4*P8C4F1 zla(+YO`GR>%)E{VU#l8ZZPo<qB2TB;sJ(F_qK5_77lG-<M1Up(?8TPwX_NGFi?7dL zZEQWXI3I&0$-{4jG3kZRo#e*vw6wM9xhcI4XXG2b$n;_{amcq53rfWH87jiqS@?9v zOT?Q$$`D1f$}JJi##wy@zv!pV9msu!dDj}zMd`6vIErl=Gq|m~6h?b?INu-hT&^28 zLqcJ(M+QBC<*%*KATb+*E`Ji5Y|LRAu!XR0jr--(CY6<QKI7#rq8I<B7ytIMh#wLK z7zs2}d)u~h7O9K<d)x24cAk_4GC|1N7gWMaIC`@h%L{XJlX#d{1u{4j6vL1Bo=;6r zlXa9zPtjshNq4V14h5h2Gz0X;i{89>(s^X^jwWu6ID+#84fZEnFjya*QV=kP%?IM? zo3ig6sU5VJ&)~v5%eAD^Ro4lQ<<s5&8K1IKX?oH5tr~8UHA-C&|ChNqQ>-|!96UKM z4ERg>JFYPEa{8FL5;&;shx(_3<@NtVmk{&8b9qAY`I~piVo96cod85;CBAcmZrq#K z>isu~5DMGu!fN-1!vcN*&C2QHzT5kN;@=BocilNyBB1@Axw$F5!dS1PqVzgFtrTFd z|IOTdg15}x?PmIWw}RY$i=L5oE@AL6sbm#eQJkN~;nQLw%P4L?>lG8QQa1K}q|!jK z|73NxEsCf&M9!5r3HRZa4?a_S9Hc^@WuH^UbvsrbAU`p%oFKASMo=tx&I)5Lo4^(V zl@cGCu#>G(A@?p)bktu7HronHmu62d13sf?uCO~wViU#|=jB?=qC+-s`2z`E{;YO{ zqa_RwOO{Iibs5fF+&GoqB>YjukT=TlMrjRZ;Xdb97=*)HttRRAqUMTcO_MJ8yZo`I z!U2iwjz6Jgeop+c%|PNJsl|p`?!ebqk5W<la@;Mp=qz0<@5)GggI4me^+|8n?(DK0 ze^E?EyDw`7{`z_)-~#%*=N){(`vO+BtF^qJN9n?s;(uM|#XmZ1{GgR2*cSWanKcW9 zX2k}By5GBXe4GWUf^?QYH4&6#pL~eKaPVZPXxGjTpNcrWx1`r+Z?|iRg1D>X>h<c^ z`uh4R2fo{hn;TlA8<H=-L=eZB%Y@Tpa8;7Tta{&_;TmB#C(R~sc{4_m1i>UyKiTQf zg_B=*kNobDuw5dy1-F?k_;uS$8ghpDMSjVN)hV4<IW&7S%6qy+T6cOR>vWe;nXtQ2 z!N^icTd+{|^<t6_qaWR~^INTCnKWJ{)Zsn3Br+8Li1+RjW5vj{&C$BuLK_S12n-h* zed^6iV~%eVmxf0r_$o+^#1GrP%ZuBgNd^w?==cc{<Bwz)|K$8mfKizFaV-R`6wXU4 z0MpgeLZ^-%$;M_Ot?Nx-sAC*S<t#ek#s9<eX5I7Q<u~&dXk~Cq;n_&l@O>An%pRY8 zt{LVsE@kUo+iM>Sn2>_v@Xxa|_oe((PvV!v@0<y(J(l3E@D>u*C<>(E(t*kqK+tg? zn_m{YJ`@4_(8=1pYV5d*@qia5LvFZ5Y%IMpiY)<hcn;x&T3F|f5Wa9_3~3!DMqIJi z5xs_fgkD7+((~)M^N)OsfnYvrh*TKAB;;xC22%@c{d|<=$WNNP)%T6^(Ca{k?#hG3 z(1~SFsf#_dWvLT}mz*3FRdBwg3jN>Wu>Q1jIOiJ|)c>>o`CkdDGfqO-ERlHE|IFju zX(4ZqW`vD1N>{4ZGhN0Y<a%i&N*3RHV<*2zG2Ae#`;`20%DeG<0H=6(nTVZ@9mzw# z(_jJ*H+h#hrW>EbCzAzg)FiG|b;<}T+Hfx7Um_RlV3{~>XXAy(>zEGrTva~kW-Y_$ z)INl6E6K95MRnauBWE}9*5owD#fYQ$l~-&3IU8rtOXS(d4V7j$IPVl)3N-)PIC@r5 z2(onX*(Y4pcws4&Z)@$^n-yZ!okQ|<e=*;%PQ2+MIR30kk5FDa+H7i*D)8M3qbBv> zN-jwR4_acWhzw9{xFAZYe`+KjTLUA^4ige-06jc^Fmt~M)s0*HOgDOSN6H&`L7;ax zSUt@5b!xtoMd-L)o7b5tAa5Hj(-WHi?FjT=sxn&BZVU7Lj?tyjiHV6fB_$=0Xf<)C z6;ER<TbAg{!7n)^U&x2HXgH%pa>+&;eOIr7&IW468J8U6GZH!FmblF$WUxlD1wg3g z%Z3A|zqSVK%vmB`zmBm2Xp-90=#QptU=|dYY<;9N0B?iyDc*@s%Yc(jwpz~W{#5)X z&vpvw*$&G?mfoz}=XJBB@}jHJj7SP#6!Zp_pS*Kp9{$^WY;V3N8v1NFOYS(--Sy5+ zOuYKGCDb0>)V}9!EKOYrfP7S;e&=Z8bh8(|rJ(?(6}XH-6o~lEpSWqHJ!njKCSSQm zJUQrrec?Hwmkf?GoUpD)(6)<DmikjEa_yJus$<8n5g*1-Sqfyr>(sKT@#mtbOy2+q z6X{EX&WEnAtGd3*BCpc5UJ%=wt1ouVY#X{9+mWPh1};0+2}`foF6x>{XRn(Rq{=tt zACrDFTZ{M3FRxSgpHX&#EJ|)glSniZzT~6b<D?^I%1*eW+$&|$N_iK?3}#TbT>H2S zUsT2UoaFy%)Q{VMtDcDeX%gB+n--i{rn&hR75=L7Y8N&w9GQ;uDcCi9Q6?I*1UA>6 zF#osIzvlCtHHFke!foP~#8}PViBZ`z;ET=V@2`j^Q~m$EMiyjJX((YQo?~RA?nlvv z-_yTuRQ5xDZftC*6q8JTmHeqE|1+!AK+#~YvyM8NI-5l6>?45<#0|Stxh0+NOem{( zeGCSPK*|0frrr6)04<*8e6(qJ0kpd?m08Vu%&Krg@qCKoQ1OC_<}ot|(0B22AgN6? zKHHX-Bic^zrlB4}`vTxNL=Op+G&=?tB(odMmC#=P#74?hswL{u)CI@Bv|YUN_cI;f zm_5Mh>(7$qxPuS3_0Q~(J}H@02*JetRek0T%`dQ2^O3>m>WPE~MBp4N&W~5eb5mWy z?0fZxKL4((30UW>o@Tb4=<ppOPGJ+9?G2J88F+Q96t7P5@eY*1Cp5gN$$WmH-B(3( ztIld(UDsS3CDH+i?|6L`WBGy0pVUh%`tvmiTq8LlTk`8+&bU|ulY~E`lVjQmFm`Bv z!94wno!y^~h4x@rP*89wp%7L2EfPPM`1lmn3|wA&99`(A?y|!b!{^`Q_mQ$i+3Rrw zr}#Gys>UYJz5}r0&M-Do&7+`-i%vDpZSii*d?%fQ@a)n)FRHud4#i5va<7W`iEJnj zj)3DnXU495(L|jkN3^{@2`k;<*tK}P{0>O~{W%$!9RfA*t1ddhnj{yI8&6A4&^}g2 zqaS_VBiQDv>JdJ_(4GKVK0pA6XR*hM7&7D-NkNd_eOBd+PJww-2@P#Ft#^-~ZwALz zJ#}6b0;H(v>7}iN^1nsNl6)EX+2Vg2mbksJC`IX$F;)Fuhm^EyYlem+NP_lAo6f4= zG*|jkBeijG#7><rPTqF*o<yCRy2!QD*#?wwL2jSRR;$i4RTo2WPCKDkk|T6}CS&Y| z;{Mg7z%o=i@2CjmQ_#<A9lgzoG$)T}qhiT<f|~}-fgO97i&XlHxv&H#yDR58ePc== z(p$h=(~6I_BP;Le!t^7}rfIHzWX8XL_Uyzh1E4+*dqZR<Qviw45NRxIwH6MixNmaR zQb&Icd}i-G-0QPI$)8z#u(L*B$y=Hyu-M_7o#<k;bv!{Mtid|9`MJ1MSP9nb1D;>c z5&MRtu))-H=^ZvE@o21&uTT2(K=9y8u6<OXz&pWl$ZfyCyK4=<^z#S3Dyz31)17Td zqBJPtd3CoI&7A$EeN>33oxopi&a)jp#MSk&B6p_SO#4)_)NkLm5xY8IP;p1Son8}; zTh-?x>l4`isF+F<=5b1{$`!DkoX{^&&7k2Pty=TIjejh4Od*H=(f;MT-D*)YRBfS3 z3w}&Kr#!R4(yHHeFU>)8@-D5y=h(aA7vw?{E330D6j~LJSPnC8KdNZSY(DqPZs@G- z(B1+wuD7EX{wAl#K@CSSMb=!E-Mp;eg)fk$8h0sbzxd3U>ZQTH?dSJk&C%zQRCW{F zoQeLMdS_j`l_TRbfVn{|lfTSico5*(XwNW}nCnKY;KG~Hl;z7V>(PjtVF5<qjFMI2 zQTKpFBET|k42uR73HmY8J>a9I#jGk2`PrL`yn^obLR02ch{X9-Pe3I&)X!WOgHPJ~ zlx54Ebo%UB*W~X!tKr#1;-1ryzg+dP34Dj51Iw{WxeimTeJ2j9e{5b1H#!(5hYDSF z91md(_1u9Pjk}xKG=KM5pKcD!4h-jcls|tJ^6x6@->q9Sd>iHa=ZfdczZC>Osjx8{ zSH@J)$%cy|ZU%1L&dhp{vYH6Zd<@%<B+2AIW=4y}2}fK^<+ZrvzWKF^OYlqDppNxB zB|a0?U1nTPU(dix9ny)W(BD3aa$**Cu~8`Wmq|Iku33-pdNxrTw|2VE)n|!M34z|1 zU%O+2pY#eTQaxl^h$}cPQ6J%xe6N|((%(v#$yq2V%JSlea&r^}NOBh~9pq3R>*#B{ ze7Tr}LUBnm4vgOK{U$6lzV;!-_uIhnlW)-Ip8a9j5ZxzmRsW=|XX-Ls6iFt?mPH`} zXI!yMp%fy@Tv=c-T>94}4vA2rRk|-F+vCdp$oQhmBbIpL512H<FS)pC7p})|?fo@< zV&PZ%wPoA7Zsw~3Eso<?=tr_$*~e-JeOLHeM9DDjPe=J$!#KOE;&ivx7&Or_uJy51 zv>ud8&C~JnzH72B7`dDwUQJUH>kC=vA@SKmrO#&5IrRv!ok0ya2AUSyj=b;1T1rp2 z$HK8`A<J@x<a7M$7<3*jfi)fkz*jx$GF(@MEOs8+TPr#GR@%QUp-^2%don*(fn9cT zq&>%Ne2Fj%K<kTmdf0o|dAb5WbM-D7=I8C0dB?X|!^ACQK5R0Vm#(cLl4TrCf7|=0 z?pDH^i6Z+*VnMy$A|bRc9W%5HkKJo`%C1))xHybL_vd#4q(Cjeq-*!5Xn(Bh#3{3h zHG>2~(j?7v1~C|8aFUepTK5Qh4;JZ0(NCpeA+g~X)^Y(mBH_CNj44x<Dck^l*ns>| z=hF=(C7SR-8t!+?db;w1Y$8E1*%NLPCyMxf7USBlD-+>c!vt+TjL+btv7shNR*~A7 z>z^hDNhwYuQOaj&X+^5Z-pApQH6|wiow0uQ0{&%h(KC0}{aa{4+uLc8wt=}c$A+SS z8z}d<AR$khwSUUMbYO2qZOfWjXWXXg?kDW<!Bn<q)x+TTx8x*iK{1U2?)=eChHc!R z`TJM0y*D~Uw0!G6)i?F@nS4oPB+t{*`o3G*ya6OBAc|KsZu-6lYfG4~zB;pN)c%(C zW6Aos_(T$y_KFes7Y&I{!cQ0h+pn<+#5^$oM=ByNp?9>fF(D(}-I=0VR&w;x+}^R7 z<8P5BCU(t8Pj6UV4X4t$dAThf7H(k8l1fua3HW<TFQr+~Sj>Sl6hS_`92GT9YWb#9 zJpc7j`dxO<waCOAHR<Tx)0Z~0^br1{Z)d<Y#HYUHbo#?WLJVrHB3$M{*`L30J1z{* z=K0Qe5=>W3rto$}*z^e>BN;A&)yAT5(RKV~mrmNqUK^7Y<aIQ&Isf)e)Mjd5P^*5s zaQv4v4YW0u>;(>V2nL%4_AGh<iY$afWYJIK?Cw2V#K0lNuW`VzT-gVo>;=Sbp3ym- zrq-Ztet#tzMkYdvr!R-=*%+Xlh_E<vCHDG>f_wS^lC^Afb#vp_ZN9o$9Jppd)6(=0 zuXU)+!yd-#G1N}-o$qs74_7v(EFF#wc#o{k)g+P#+bHVGSZP_el=l&#d`Q}eEvl`( zK<9P8flt?W#{!Mq?$5<V+-EJ5vE!=@?cBp=PRiHXGM3EwOUnz6sG}Wjl!kg2Zz7&! z(hld|*S;++@ODyI4-{auwdmxV^%2kV7_8z4p_Z5gY_C5-d}=J^5oPi!w|R(Zne5Ls zIKXJy-tU1bogO%-{(Tk62%liNqdlo16Oq&s3+@GP#A1j|)w<z?hq#^^IjDx;Uq$V6 zj|<qHg4)->YmpqyS8g#rY@<B)PlkFL3EFyNP61c3BZz4TI#f$X)g<wM7TLSV_Y97I zof#r*Y@v1%Xl|=T2%kA<9~<Ld1i2Lp49~_j{jG%t-~rS6yid{<))T=6)AqY>FaOJ` z$D^Qv&HPe|#R}p=R1D_$INdg%@|gblQ(MsC)3g~mNyzW;F_1F0<&b15tN{q8%-pwf zT5uM!I8em|mbzPv?RKVUEi!Oxyz8`EOGxjgw#QM^%xE_GF0z_q;zF$<!=z1H$fU_A zI<I?z{sEL8mceaLn7WHQoB;p-4~RW^O?P{SRbIPBtu45uea7`xTl{frt1z!?{6RYX zAnr@n@9GZD)G6WN{PSm;Hg*f%1PPPXbpkw`r?n9xASJo)#glEBYisn$E;_r?GQg$v z-dt(zhhAw?|66`#8Y8=mcXFC#b`axqq-a^eijW9@EN5m97<Z}{ozxC0Wr~DWhRJ%; zz>ABSnVAhIYFPwu622&iA)tJczeHut>}32t_mQ(&5WjqbWPqQ)xOr)uw>8i9GWm9J zkGw2>%J5j);!$p(y8^j<K_DT)rBrTJL!$`3jJ;0Et#2`H^BOeApMTgZ<u7Ww`=P#p zZ}*rsfv2kGXd&l1QK1*TzLFb2_=ru2L1V3fz5Y5fUwC1U+*(D;JLxcAl&r~b>8hH} z<aCd18pMUXO7y}<{3)u_*ax>qSVcjpLmH{56}F{y*6fA*$5=r|nc>GB!4scmGlv2@ z7TwQ+bC>LxR{^DkyOTPTu1vAuRwep#eQE8(6)%_8V&>Q)X3?&WqBpLyA*<Vv?k6%N z@f#~SvM!X9QqDqThXamwH&3HMZgSCiRG5JK=)0IDT&6owvF_G!JiL7*QWV}i>eS3( zfc=@Vh5T!F;}haQs=s-)qOX_1hE<y{;zg$~G-hHh4;Ll#=p&0vAA-wk@QXl#nO+g| z<`Q{Pt<!q>^F_Z+htfeYwI=lzFq;$Gb*Nuf%+mO%t5{ceN}Ra{S*S?ydf~j*n*0p( z)ZDiGte0rMxz_zsKvgeP`j^+e(BP}BD~P%C`UThVAncK-2e}o0|D%Zi;_u~l<e0F5 zh1?na85m&*$5GC4vr1Q_Vt=GQbDz012|UnFb-kEPjDIVO7v9`FWFvt=<zdMSpiFX! zgAC;FBCsAh|6z}vl^y7esJFav&M7vXxGB};T~5zm8{el!Jz>_79wl%$lHYV*##$^< z-m)`;N!aVGS^m^Rr);*sJz&5jom}BP)aPJdFRIDg=&HGz5a2z}kST3Qdf`F^9KD}D z;Xnnqqatg!b8;a3{j2e>UN9@O{s}QA-Q!X%%un};f92YzlWRW*j?jaT>)Q)ik7u$s zUF)qhCA`lcvAv0T1Lcoja9<VfPHD4Gd(~!Pb&?!;mleUho?z0ty^6N7$yMF-4SpgU z|LrqfPMA1tjr8hoxvus>yN%@esij82k4&iFX=KOR)YT77%UC#$*m61}C+g2nb=}7t z@p}xdY6Hqe<afC5B3vk1|3mBh*CVy(Qz;R2#jogygA1a254)0Sb5`=8O_PV7<(*Rg z<p?!Z{WKV&5QhS<O92h$!&hPJpK*r2yDXWK#Fzf2ZR?3~_n57;yiZNL2c~BGXK~|H z3A?IVC#9?f#HQ<`Ik+n2D(sDzP9FOZDmw+}V#r?Vh3hEa9p<0~zXA4CmD&gEo|f_s zyO>@)3Wh6RQ=JkUAY(FkC5VDg=4$Y`jt(-;d*I}dRtkX>%sWC;77p`kliPjlD?aWd ztBS}=hSg&9x6e^CVR{qnOo~`EDAjCRrgmxY1rDWCtU8PrHkVhs6I_3*!Z`;bFz1YT z_nn7nQ$du*{jE)jKhHrmC9i%~zDqe*P->yhfn*7VW^#fu6pGOQJ*;#|S#-)aqK4#8 zKK{Zb<St0JGwwiA(LKTHEJk1get;W|VrB8>h?PpT@aB3ngq<xfyImUp5P!6MK*?u+ zD>k7|Q{C%ujMQ&qwp90N%XNK~+q}T_oYYkA{Q4t4=50W48!Vuw=vPXQ0%ZO7U=66u zuc$;pFtDF911)8kqQEb0T!GD+S1b`ed!?4@WtVkOt5jSbEmk~$b%ddC>Hwh9+YY%> z^fso9j15nHZ1K{F#bj?j8<U}|^+&^<4=`Tn>}d3e)P{i+)2a^&6`YYYF;LLWl~A6* z*uBl}*kF6SV-It+wy`~ub;~~7larG*&95%juV0kq9}T4X_Z|G*!m|^|p7Lx@VaF8< zSVSXw1F5s?+tedJOUtn#KhZ2o2lUVW+}%x<s^W`03J6;oMoiqrbC2cM*4(15ZG1=e z(&6cjHQ@GwBW+ucDcsJfT2-LfTeUQa!B?j@R!V0^_7T`<`H>w{<hBy7^ZmEmDF=CD zRZ*EI6{ZSdXSCk~zV&8ecx*8`=D!^LKNtmdH#$O(2!6xRO0VbU`~owJYmV=6+3P{_ z@ibToWarEDyN53=I@9}viB!OO<k>uan=EI)#~we`!QEgFi&w_qZelhyeSM*yU9vQ3 zCt?BU^T(7SqGGzg-KU!t7T_Y|m#7at=fT=`@&b@+hvG`?T#pOxSg1!sc(}vqCEecS z#<UJVskNd@wJTlJ7l%w&&^~E2S&95Cs%&REf(-iL@GK>vQmS@?lV}wkX_pKip>#KH zOS}950rW;=Q*t|xF2@(ep&=IsB~jkPGEwC6xT1H&xQjR{3T$I1#ZcN3;D;aDCuy-d zcC&Z0RbH}9Ze}*+^;gTy_X-_3B6Rbx;jQ%uD%Cz^KBGy8bWRcoOJe)4Rx^=jqj3<s zOtes{lINm3h?bF#;oXgzvJ)`r=vjW6$~h~5pTD9zAc@%LVAaMPt@Tzcd&sQCwQsFP zUHJ^+oGC75w>^Tmal}icOBDM3m?~YA>E6F})4;LTNu8B@joVF(D;7D5S1=&b=$qyW z#8o4+wL1F??Lj4HwMJL_<ElF;%VLuk?ou(<!?4A75YsmB;cGt;N@JH}-){}ZUQco7 zTYkLqC&hdsdy2P1_}HkH=uaNO$`JVw8@#YESxE^WQ5D=f2bf3jS_{-NHuwN!1&+vO zD;-VAp|+)>vgqEB%D8*lXc4&K6s7DSPl)xrjsO8dlBb8i_H9Qra0ZJxn3-Uc_kHQX zK&URimS1fs2kx8UKMpM`1F3!;$C;R0NV|a5FVY465*(=Fb=<QU8s4B$#@v4Jjs5%Q zu{F8Yk<IByqkj0CZA;JTO?0|QD4=O)_4umiTAr$-trgp76vM4HH>8RI*h1ogi#xx0 zB&;vy3bk^=;|LC4?l};5-j=~(U+UM3Em|YTq^)yHK~1(B>TAy)8-(c(Mg+AiRxbr? zCIQQ!Ih}EKLvf`WCumaF-<BpPOiHpRr^tm&m5nr3`ud5s2J?QI!-wnlIp7u96N2IN zm%53ol&3<qNW&=UpD=2#qpzgvuRGY7O*^lyT}0d}A}V^(E9syryL)BixZN(t`#SQV z>HcDVsG*^oj7Pe49-H;V%_euugkZlm3&h&9wnlUu(0uaFT6_-U=29cip_k(6HFP*) z3;tyT=d}594d#&VEuQzh|J~eY3Yg^P_r^{5mY72W&*4YjFLw`%<Y$TFKTqRs?!KBO zC=Be)AR#~~K_3aQu0NWxdNlf;@d5xnCA)nbbH1sZpSs6LW@itKK%LMR_3<S%qiVlv zDz+ze!(Xwn`)6M`-?L{j-Sb<$16MIE*Xr48Zf<QY(gnwNyJ=_i7?7!=bXM8p9vAuC zeK-YP&nJqxwW=uR>(i%z&oF=>v{+SE{kvW4(*|x)YpN}yhklXGx2wzCm;XP9Z><;@ z_!YRt*&;l@C)0LU(n-hoKSu|E#$U<tyi#;140x2IY#MxYOr()k&w@|8HJWS6p43rt z^!KOLFrVpN-Nr9Jr`$Vh>GzP11`e1MO$_)SKkysK#!GU_n7REKRcL692#>ww?q_Ux z;CIVv1prfwq0IE&|3yQTM4ckuwX54NM(`O{cK>v1<A9a}wEVoh@iRB85m_pBCtoIl z9yDltn)kgj9ckouzD@XMONT#^9p-C)sq#3)p}7TdpB}rqkqKI#lkmMnmlX0Zjsyv7 z5d(^KJ&kdf9II~ne);g|?&r~HsXiu|Sg=jUsTcLB;^am|G067JNSwW(mgIWA6t9r; zFqGoiIJXC~Tk@;aZ)A1!W4V+Ms4H)7SzVZZ@mK9Yrs}J9O-(fOd>3!Pkq3en-u`}l zf8tZK^<K<{Fe~9x9dP?1G_*^RlBW;u4SfAqci_g|sBvzg2RWL95ooUdx7HBxH3ISo zpQh%rvxE6$cIZas1-tZG`Vaf9J2{>Sig`66eU?ncf#Ssi)v@jo>kTJ43@eyU`kJ4y zu`x6>arW4=UqiaiueIj)!=LOfML4=7pV3R4`At)J;wFeAtS+g85^2(ZiAef68=^%w zg>Qi7iHipw)+&ya^b8EOUQ7gAWf4-eixn@USs(F-yE@9gr>34Z{~GR<JhQCLA<i^1 zyxaA58HA@+Y+*uCP@a*>OTE{+jsJmXz0@Swb?2|(R@hX${h-@<Nk^fVFy-RReU+gg z4veQK0fGFH@G$ca2rj?~R7R9oXy5|`Q8djhF%8f?cQ@DNCEq)Ey)k(6b)2@2Wcc`I zWMoTTsa$+evZ^l9jr95myP4tQ_FTCo{E;##(tjvye|S0JE*#7L#O$3S7*O|KLIcGN z$?h)dd_=N|%AU?+q<!pu@>}%4V<$V3{TJt_9>nw)w!j^W5?-Yy*poVPj~U)#i)$1F zH|e^@K+6Pk^_G51KXRKp1VD^&e}D|93y!~`vi3Osa-;xkRY^ufdNbgC4cnT8Rh5YB zO?f^AOF`&TII>w&;~o6NK1a#Gdf?~xmCfufE1-b09D_pHSjqz<oGoa`>hGq;=|zZB z*4Fjp!P-`0!pv7Oj^UYmPA0laTJ0gQAM07_%+~7st$fYKsiSFq-N{GW*`aum1>Yh) zol9E9v1()jUeF94qPpx`(gbrqKzjPY`w8F=bxQ#NJy!e-kK21jT2RrUbN(l8>ksVZ zOnu1*9!=EZmzTu~sxAy0h!f5VF)6uL0T25`<Rgia4T1?{yOVd1XFfF_s@N%q4#v*4 z`TiT(K0sYdbN`sF^s~?gHb`n4C}3PCcD?cG=Duq9r0!|WI?>MW(;CB3Ib%Dak`EK( zEhxA>ExG_p$@dQnmIvP%S=o(xpTM53rG8d7xxOF?86KBU?)XEbodGB3+o`!Y><~oa zTGV(B{}_3mZ1th(e99n(_N6jW`aR#a_oBi1wZhF})o|SOylWFpfp4ZP24P@FoPtUn z%|hS}s#EKIZ51MAD?FqT(|#7Vws_2geXA8T&*El$Bg|b8;T8H6nP4H+j1p!qyUNnA z24WFeC<U~X*$E~vjDHdjOvk_`zVhf&IZusnHSS>vjxS)Ifn6H;?BqpTXrI>4$*7;F z8iliC+pa0xn*ud@IX#i${={=cQJV*0VB%bT>J|oH_uym`&G?60{-bNUxVZGNcxr5g z#}dn*ZVk7lIQi`4I!djUzj!r}(HKAe)(P(=Z@mFHroK76OUzoin|E^Zn<+g~F%0h< zmAI0k3sUIpvcuMp|5!$AWN&Yud%1Ma;eWM?S#7s96th6Uy6%kFmU7G5@HacsX5s3Y z$Z=IACkwZOxr&O4sppod`o2^UN}%eA;6vtyvZN+~t+{^XY|bS?RlcDqrRj_Rw?B)5 zo~@0@1~&fO3y{ktUyick8?~ofe=;WO_fu0z^;-{F=4wlWnORthE=JzFHZG>i7bGk? z7G#S0sxMDyk#bouMiUEyD=WkLPh~X=#nZ%{4k0mO;W;*Vc+1413nORo+S+dH83J;; zg24lk`2ys&&q4r)-@?SQagni-6>o(5P^vO044p>J*PTSo4|~4%B88bzhz4HlFB*;T zrHTao$<9+8FOoJ`<cGzJcsfJ6DUbO8!>#_R=2N)<Rc97<q&tDwiT?1@H6}iGMi=BW zJcgbwUu4u<6eK${vptx2`IHuxYoc!sr&LL*PnO_J@E`dEN`7~{KWqqnZYwF2=c<K@ z%yjLHU$9~C);x74L%Znfj&1g=g-Tiyih8mO#|_J%S?L3!jAVL2V6dg2*kE9uLX0@n zaJd*{zi0CjnJ=fOBM=I(HA$Zr8L8W8rv;4;0pHUuJ(0e?^wT|r<2${1iHsc~I;)CL zuxDd}rgst2-f8(kS&FI5GzSY4(7_)2-LFVwW=28fq((4w>?XQVX>F>%P1$P6*gfaA zEq3N#pDlhxCY#|!xc&oRcTZCm3H89BGS5NmzJ>>fDk`Lm4PmSmcSOn;p=ich?&o%V z{1)zb^v$^{H$m*Cv~EcO<lbn5i_8WRKX0EQ0#5SCeRCjXaA6U%A-n(OOSX{4Y@_OD z$d-V5r1>9cskqdO6ThSnO@zlE)vm?*rP<&ke|_UWum9wvrT!{ofA@UhfHr<tsBY(N z-y9{v!Eulw_ZNvvB18hmSaxVj7bF4|KS*EzXh#Dxk{S#@8PJTrn)|o5_@AcUUP5?s z_x3IP*L>u+GvD?_f9bdU))q@thn{NUNdM;#za%0Oi35h8mNl6W7I}pq<MT+@8M1J3 zb<AKChap{+D~xB1%Rj9TYGd`aG`I|=6W@3k9Zi121$JgLPB$1r6@@EQw4Ne0+~1dP z5SkhP721#FWCYx}<XoqLIZpBR#xq_Ttm`$sU;}*;365V}iA9S=+EK_>La2-C7avsW zguA->D|t?qB^r!>CWz&BoHHp1V$vvTSKdv+Q)_BEE`L!YoiGZz%0Emb;SZ=Lo1PvL zeZI3)ME-*uvYps{e<a^vzFwO4s(91Bk3xvaWLE1wVPRn*qvFDEn*-_FB=(=(QB7J? V)fM|l)}tVP^3p0&Wgkp~{|Cx~kg)&& diff --git a/site/img/logos/bk-logo-with-text.svg b/site/img/logos/bk-logo-with-text.svg deleted file mode 100644 index 36ff7e7d565..00000000000 --- a/site/img/logos/bk-logo-with-text.svg +++ /dev/null @@ -1,79 +0,0 @@ -<?xml version="1.0" encoding="UTF-8" standalone="no"?> -<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd"> -<svg xmlns="http://www.w3.org/2000/svg" xmlns:xl="http://www.w3.org/1999/xlink" version="1.1" viewBox="-382 -377 1316 398" width="1316pt" height="398pt" xmlns:dc="http://purl.org/dc/elements/1.1/"> - <metadata> Produced by OmniGraffle 7.4.2 - <dc:date>2017-09-12 22:13:49 +0000</dc:date> - </metadata> - <defs> - <linearGradient x1="0" x2="1" id="Gradient" gradientUnits="userSpaceOnUse"> - <stop offset="0" stop-color="#683b1b"/> - <stop offset=".371" stop-color="#a1612f"/> - <stop offset="1" stop-color="#efa363"/> - </linearGradient> - <linearGradient id="Obj_Gradient" xl:href="#Gradient" gradientTransform="translate(-336.11597 -375.2114) rotate(60.63441) scale(386.4362)"/> - <linearGradient x1="0" x2="1" id="Gradient_2" gradientUnits="userSpaceOnUse"> - <stop offset="0" stop-color="#683b1b"/> - <stop offset=".371" stop-color="#a1612f"/> - <stop offset="1" stop-color="#efa363"/> - </linearGradient> - <linearGradient id="Obj_Gradient_2" xl:href="#Gradient_2" gradientTransform="translate(-307.3777 -289.38578) rotate(60.63439) scale(318.51095)"/> - <linearGradient x1="0" x2="1" id="Gradient_3" gradientUnits="userSpaceOnUse"> - <stop offset="0" stop-color="#683b1b"/> - <stop offset=".5161" stop-color="#a1612f"/> - <stop offset="1" stop-color="#bd753d"/> - </linearGradient> - <linearGradient id="Obj_Gradient_3" xl:href="#Gradient_3" gradientTransform="translate(-339.05287 -369.37777) rotate(60.634406) scale(370.37772)"/> - </defs> - <g stroke="none" stroke-opacity="1" stroke-dasharray="none" fill="none" fill-opacity="1"> - <title>Canvas 1 - - Layer 1 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/site/img/logos/bk-logo-with-tm.png b/site/img/logos/bk-logo-with-tm.png deleted file mode 100644 index b9b27682ede8ee375dfe2d9fc29460e8b7430496..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 118285 zcmaHT1yr0p({_uyyIYY`SlnF~heDyaTX8S$zIbtWXmNL1+`YKFyX#N;Jm33&ea}ZZ z%V9Spxsyp|=DKE*4VIS`M})(Jd-duSq6ARn{i|1y6t7;r4u^qyE(y;^+-nvPp}j82XA5&nJFd^XWWQ^0J-`1|3?L)v_ff3hx5wU-pKY!vSGqJa~<^lkmoSf*LnCY!-jRB0DoSXm#CIAx?-E$2( zJ7-IK-OqHEcI5wb@<08E7~1LEnp)eNT3M3(>Q`6K%E6wOjO^Eg{`2$CIPFc1{{NFK z?f#wC^8^9E{sUm7X8`=C@8_mGze>4;tt_l<4ejin`{!fj`Capmvj6{_f9lJdSlL@W zOTpGuU&7Mf(Du2pz3#8k@iG0|`sEt`-ItuLsp0dif464*xAp%n`?o$1;Mc_emmvO; z^6%1TG4sLk0RFRNd~juJObf4G3A~aJ5mfs8dOsD`i$HYdvC;Ul&pis=)(VXDbsH+l zN{bQ%gdmD2r=J|^`nkZl;@K6Jyud>p1O%oM8JTTR$_kL9olr=qLIS29-PSlFap#d= z?!(l+!I-OzqptB)VNTw+MVGzeL+vBRv;Db+mZ{f$kby{dCD>3~P~PnCqK*&iS7wk~ z=3gU81(JY~z3?Io7sw(fP{1+3@)P{HbElh#?V{aq9GR(o1k;_XkMbJz|9*Lgi+w+$ zhCaU#4tT^=jm|&&?6x!){Z3~8qLC{8-8OZ1{@VZD@VP_paDfbA1K_EM{oZ@W;3KNZ z8zNTaR(1KU1xAP62_!o=!j({&?ia@UJdfURADvAf2X!QzN!Qe^K&s^m3nK}nZ(8@W z0r6N?ZkgEx+2O(vA_D*SD29lx#>3`t1^PScJ)jNHxmi-+N7DM}pA!CMHaJO)N zRchxk8{!2M^uo)H2O{J)`ali99OpxxuD_;_F9f3KdZij=n7c8I$B3q{i*mnT(Jrpy z!4p*d^~G+2yvXGzd?=9gsMp1|qfp3CBKiz`z}`s8Xj&LbsLrU>=yO~L?Py$=6L!P1vD=5skQ zw09O31t~Yu5M0y@)0m=r)Im==OT$dkZ6=Nk< zGgP)~tT3)OS_d-3m6T=bWyPxop7g-dFBrQNJ{8Ih$O*%4hxdr+NU2qDckWjL&&bfBE2> z89pY;L^=uXkOjvZ>w`XNXXhE^O&QyFFztJ-7r~?8pa`v(6!p1$Hbr0*E?PGd*9a2G zl(h;>jAF`p=pF#Z^TPfdgNaOLGv_e*ym2Tu%8f>U=f5*?H1gqZ+6DuIQA)CN*N#ZM z8;L?1_G(8f@iFLWgft9g$4??@$@Qw(#@CVPG7XpI_5FB@^F&aLwBoa;eNl=Qk{O>N zdl@=%Coho88Nn!S^W=j;9Y8iaDau%)50lk%Jk+t1y|Q(>yAtFsTepRDFZM(>atjfn z94UPr{K-f($yHR1bxbQ@a2=Q<#DpK6>tTlIrpa@(AWS8lbYIn*l0$k9Wi_+^ko5TW zZ=(~>FoBc+tPC^ZMx&6^aN(^)aYi`fMjhH(+#Ta{jZtlv8|Z$>GP@$$+hwL{FLm47 zRuHcx{3MF?vNlkw37+2OV>NHz1%D$Ew`5W=5e^!FixJB)*d>R9lIhAhSot~2uTgy; zxp0-nGj1q9yD~th;;A=msj9;ImxWe~6{z@-?cVW2OQUJ!1f2G6@z7Q9P23`#r*k}^ z>AnKo8^Bo&Sng4&Y6}ZDGw z{N>gg7vu46SY&HA@_1&og8_ajU)W5~$PRd^7UpMXcWpFX&fS*eD4E!7(42A*n6cfr zw)?A3maLQ#%TJ0Lvd<*^IUqWu!OdO8j-NqW13+&hmJiYOkkD4;pR zRd`V=Px2>y3JyJG`&J2V{(TkJEy7mUBff80e1uM=TG|f}_t(3jFF1^6BfXE~orOuE zSnUv@X?tDP4d8XTuG4^+;we9tq2Nmsem$m?&6h>U>oJ#b5`D*Vh>SJNfBv1EKP=zT zE<71W>!w)_l%`|u@)w;m!2}bf1qyKxK!J%n?;HRVZek<;2m_%)mY?+&+_u$40s;mGg+oTXd4c@_~+sR>MV9A3OoCI2A45l*Z9qfMiu`0Yg4HBteHiF=X8aWQ03{x}`4cU8h5YjMg#6XCF z&0G=m00wi|!oXSc%~6x%F)Edt2hu|rC>Yy7iRi3ZE{rdk4f6$j_zUK>{qixm0Y4=3 zdEQ&Cf0!PPomRyYGx3XxH)*WFd_%)JTLdIGd|wKh5WkLla+*zdJ+M1Q1&o=C%7jC= zKF%16%??P}fB2i(ko`6MCAoLmI@^-U9DYAS!ZD~bne@E$^D+b%MmH6cpl-`PW?%R_ zw>Plf9kG1wh^q)WEjCJXKDo0jJo8XrQjvODI2pOWY%;=wH7qh4En9=V#54%Sv@1g- zsY>E)^gxh7CWL7vd3kS0ylf>4{pXvbW=}llw)kPs9~YcRq-Rx6yd|g1LwUu2(d~Yi zX9*>qqleb;D!Hn%K}s3{jr=7qS9*$oR3Z&FYx*J*_`Qa*wmdPCVQxo*>`kwI1L>=9 z;|Fp?{79bNN_i}At)@EfU)EsrStA6;;2-=CUlo6cnTeW-ytb08av&pzE-DLB*z%GzJ%YRw~6<_-x`_YTnlVvGW(kvWQhKDTS|UsAe7+tT%813 zyd)5iI&8ioK^hoC4y$fZHI{uaE!C8b&~M^{O4DAp1@76%%ed2{pFZ5x_8G-b`|gL0 zK62)_znQ?ZUmMB!qCHeuKyx1GA4QfuXqz)&k}IZ_c7P#4$urgs7ct2mO_uFEx!Hxt zC!Lj3Z`-@-8=w-7+)vwJ&Zd{A3J)&0iQL*;P|(AASZg6mqG=PNbk54w1cGb^FBH032R6de8SOKAt1*N192`Y^Bom{vY2dC>X~_Ax}2iS#UyM$*3cC+ zG9fE}R6#m;DQzARyNeKu1pZ$ZoAMV_4eqa48ZcscF9lKp&Pq=u6nQ0F_=d&XHMT4y z!8h&sOXewwy_N>rr@I^zugP-k`LNF1$s|?vm$4k*tLMw6s!b3iU*nCIy|`_w@Saim zShUuv3^~CpHaV)8%v?=Yrn})+10@iQ=`AN)c>Q{}Nq-nRTRQ{J@sj6#q0uQL)c`~L z5ETOQu~x$#zJiIxUz8e;^qGJdNU$U0Mqe_ShXik)n#1`knDu1ji_-{t^-EeeXPy8J zSVWt`e!YRkCC0vhoLBXOF_no&3FOK~tzaoAy^J*nQY_{OV$%pd8RoMxQ#o>6S7TEZ zLT~c@be#Z2jg0tmh<{m=k6@y7z|$bKw5N)G-JEM6YV^bHK+kPWzZ1(>0>1*xAwJc!!vn{nGw0=5GqZ~kqP7*7Pi9; zr6eCS4W?8a<6G|1&?97I$Rc-}f3s+57N{wfbHM36)>Z9I+_E?6oz8i5DuY3Gz@c*0 z{C!yCDOs`mDlBeP>}7*U5+QH`AFX8@z?lamum{Vm-%nygz*~u=bDU;>Dq1+_G#Dc| zh_i^XFm2kga?jq;;sSM@gBy@3_*Xp@<5~oz@E$eaOnsBR-DTGvE(uk*rMf@f zTuR}p4-L)yi`n;jBY*Cb!wK?x!tfV>3%w{c8N~^X-i-dF&pCp0{=k6G)FcG|6(*cE8vpxEr%iQ)&;{KCHghY9p;6rY?UsM#bEL zaUdE8nR$iX^Jb>pwY-$$mR)vsjTU0-oBlu-7m=?1Utua7sO}Qph z*&};w53|$pJf?(dONr>8=f)5R@ZW3{t_2#deIbah5dN}wsm}p1R0_}A?RozZge3>63AoUx&n8yaL8Qqv5{%KE!2$c+q+Kh$M+&pfvslp$cD$QtLo>J_uXAQz4#2_=e zjs}H_0Yzl9QIRbyv)#jF9dfX2rzJxYx+la!;w(c2hlWF9DdSR)I=c38e2Ua7^-jn9 z&zc{Q_&toOLIxW|cr@DFAF8Qj_EW3B{Y8qA0)c204`@d%`B*#^gL4g_%?dY+PW5Mhq7#%Sb5|brh6-4=7lkIKNZZ63U*=FGL&2JGtf}g=7 zKQo)^e97?Ihldm$c|msmel9eeZiy|#pm&8+@S}Zq$Jn zhAblTm|clZmK`h$%;`u-J(`UwGc|6k))7It4G_#*0 zmsOC4Ypfl~ptBFyW1qEtXSbAC;jp&3=t_LqyLG_B?O!QhdjHdy3Z#$mGF{o0hTuve zf%Rq(X>ObF?akCZ+~LQ2 z+&1QFd3%%O8^N#tS;F*2Ibq+$96~U0E7#H+n9M1L6OhL;|eL#>Pn3=X$;sjpw^4b z%0P#8E}3y$CG!V{cd#p6X*^&pT*^qrwu{3Q zl7UZ*UimQPuG=Qi@Nz&{z?fu&6jCCs?meIht}UZil_fD;EX6+mi!(@BsR}I%oluu8 zI9%DtUf(fp(6Wjt*UT<0bWLv#XW!=C2Nji@{RgM!)5Pa36m7=kS!1K>bfjc)+mMp@ z>BWe+KpatKSW zR(T+5cC(X>#@-dnv9u09)N0*ruH>h1lc=hx8H=T;13x!F;l}h0l** zer-KS-*`~@)F9YSYpl|I(FgNQ-$?(w%rByE@fN)?&JA5>cDZjhHkDHOF79`YTaM${p z7%a}+!`G&@V$^=Djk%!`?i~RZaW2JCw&eRw^K;>I&TM=SGfw4fRvN8K`|;_{TYhIP z&5?Qr?gHpl6zDltu=%nF7+ZegMK9^k$PJ7=-pL*nvdY329-ZwT5hfcvh&EiFpR&|N zR_dgq2J~Xu@KJAeIlV_6!=_D=GUGE$5)~`TX?U{gM!S#dtF?z)z$wF0%8vHLdRS)S zG3a*Ixv%b&K-AufYRkCj0(lz%cRZ!*#mQ%DIXk9(2pgnq}IFDB=L`A8KJ zY!)n>!R3o$He<@@04mxsBh6W7f=Zjw47u&AhNsJ=Cf>&keM-l@Q9DFdnaY5Z_6O?r zOP!7`3@Wh-Vbn1`&Sa!0XnzS%dHg~{ z7n|LPH}w6OvHO;BX!CdV%ZR0i<4AL9W^=t}?aen`+8h>_MoCKl@ZuWsW1e@=o2pjA zPfgE@&?&X7vwvI3*vJo6bHnJE_+R8X#7iK*H&WVotXr7=p5nZ{(ud(%blG`o&32EL z1o1NexTmeZlm?-~pw?uIuPjfCqY5lXfll+)(3-&)0!56_@lXaO3Qpr zF#;BY^%6U3^UV_2bX^+$YewUOe~5;;__ET+(@r3l%>0mIfF31^Q~jkff&2$X;xCt) ze@Kpt+gp|#NZDphhbb8IBZ#hAFlOEaFUl)OkX%ws8)2y0IyICf@aC@Rb*FSA7bg~vS- zC5#TwFUaJ!N)#+Fd6{Qhoiw*8B2?%Qn zP*k30Sr!p@R-f=#_1@1YPL`o$U!FrKPZ-L_*4)IBIw{TJ1FS%Ds zr~Mx^T`yBvjE;P@Y8bQMNeyJlZ9}&{jmKZ@cX9qk?jyZ0pzf2jSf>HxOnztw)3j#x zTkS-_` z6B2SX#8;U^TBEB`y|$Y&@*OWYx9)0p~=Aa{U)hStY<*BR2$BQn!%Q@w#FK@@uR8gOcT6^jqAp| ziZRdGd~+?4rGHP)RNdW9&e4(U5A7un=A@$El+xZFEO6~rdLMUB{%C%xf4s%kl6rK! z(^I4`u^>%5o_HovSYJN~Izu_bQb^N_3;cq(pkH3S@(bDYV}YVP%_5@kL7-ZAlGuV* z-@bxT6kn)hfDQ6%ALl^$qodxnC1eMLT@ocNE$z?-)C>2CS_N&9jh2Z_Hsv0B_)ijP z<4+l$W@#2Ix9MWMD6tksJ(C$8LAS5pPTc|CUS-lrYC}9=?2;BVW*++WJ#5`*``x1L zw#sj1{x{(F@>1;JA_P4-xoxNeIa72hux!p{(O3$D_E%alNsHyTA%uaxMfN z@$5uRPg7eKB*k@w{J|S!cyib*iED{&*udU^Nps!i7$e= zybi{uzur?;3F?9^QCz;fKSaZbZ1yQwoU;!-Vsz%{--90V$ef3OFI`nif!HUW!>*szP>aXPlCn-XMd0 zvIJ?gQgCUD@De*n9txmaz&VV|mkOoMc>#{VZ(Rm(AeU;%xTo$JccTgpr>t1276R^{ z{QAx$kct&3NqVm2 zKprMXR|)Q`ju#p8zE1WW`rB$dkd{Fwifw%tHw3OpAyR(XAt@CdPrr>=&YRB z$_(KyW~iJ&QHv-}xP!-Prw^i%uWS+q-oD+H*#8$*{2Rm+Fs6J4dA$p7`wjz0>gJ6E zSEat^kiIvry7H-tS6Wk)UDsi6ir1H4F~4t<5RvOx{Futek$RGrJ?O?1vWmO3o&DaC zt`>@K)^=XwaWJAjiZ}_JsD3X%wHX?b*Q96L#;Q@OvC?J*kD^KYex(T(ygu!BY{gz4 z48z3(&(8cQ**Kfwi?*Y;cl^*Huos&`vuHT;gk=}T*yM-fJGUp_*<47w?Y^QO?JOii zW9a_ig!JLEdeWPg=b6i89nQ{6FnSDszXXC75?7}crPq&mhj@oMYpm5_pYG)0-Jh7@ z>}cLNeMt=R1V5G(gWLouv5>1mI{MPKL z&!)(9?rK9lMj94r3fVWJUYl!Vg+RB;0iRv(DvCE6#fFY^No_aAp|)irQ?l(Gy5xM} zg}A2-;64ROF_0r1u!w+^WcU$W^J@ZHdxunhpONDt^0r!z6x$@k0(9n)d{Ljws$`i$ zslH*Z9~nbO58RU3jMy4icnVQVVy?kFU{{D=pT;vg4F5sQwu!=gNo}5p!!6SUk>rVz zJ<`?R_T(cu8h-av{ss%mg@t9>s~N(Ph#SCdRWzX+xup;Jso}M_7T3bZV04Pas2h%5 zK4?fSvg#u3zL#Q2>i2%4(dbr;A1cA=$15(v%eOx&8mR@S;7UeS~k>tautzG{D z0HC0 zLvj==uDeN%G%u;u2LQHDx}JQgS}#_5)~t>6+}K zKER@VS6W|1nX%9>m`4~YU_~%=b%qqoPw5$HY^Mq5b7QF2k_q|1Ay@?1RE*5d=s2Z+ zTluEG)HM@HcmwC*QaIOIy!>5>284ckoi~&@MuX8N$i|`~qppi1Fe44*LbMyNfGkW1 z>bqmFq-|FK+@X1q&Nj9@9cDM@{zuX2fq>U$U{(b1sMM1W_ z^P)3-bisgEu>dc$1M?>jT!{8JH)0om+NVmC?FpatSf|!%B(S9e*&u0onw*L52u^AR zY&y8mnbr6&pT89gNCY$X`S>wEn2rDNn{60iHY!Op^erEDNGaRS5|zRo-v3wyzNbu= zTw(sB(dx*3T|!^7|1Fyx@qPma{iyx`gWXy?Heo-q=Ras73dRi?TG*Sac0P1yuRb8< z5}(szYJQFtZn_ILx?a~(ERb(LUt^TkmydbFDDx36mTzFcAAkYDA8%!z<;mam6u^ZW zJLAGH`UbfKSB0&u=T6pEdkyZYxPIe3M;h*06IuigpZhE~uJEjJc2-POb-dLlS-{)a6^igm^+Iz5zskiS7qS`~zDX7?1mnP&j z0@9L&y^q|KLbPOJtF5oqz-0$^%_MGEQ(C=lg`_BdT+Kq=-?8Wnv!9`zX>a0>YI|`( zUi?TX50L(V^KoKC$6zTt$|j^pJcDl`NbYcictkbUENFH>-rO>igO77KG3#u!0I)T) z)*#n*J)H2`BbD8;h*8l1?SG#k^9~ox7>AEmdE8qNmA`n@^%n?Cv6aK~(6Q{E+{NmmeWzq0{i)t%!@B`-q9mR(FC%OO4B3n$_TE^UWds ztEBmFPs>+3T?s~0Wnq^qjkPKDQy+&Eevx`_B!OjPj)0J+wL1IEmB~(_bwKF5gHF_h0F8BfrJi+3cKFZ=XQ9%SX8n2gCYE$V?@h5oW_2-?EAX(x)Y_O_tl$VQrFbj2ejBqAC^(^w#g#yKIj?-H)Fw zxE$Gk?>V_^l}!FX%Nf%t!nfN6IEZ>8T?Fm^X80jN-#<)G-7m>FwP7tlW&a@8t+`EIP+CNA8|Z)(`2AT(@Dns@T~q8jVEupIKz|ZCYS6CV@BZ$n26|9otjUu`3~sg2HkRoetNVj&Veoq2BfE=w3P2JI zv^6$pX@Ihh8@7QOIp>yoDPGn^}iAtem~ON(waY%2?wzF3Pa}*tN+~DU`04 z?Vp<|J$-*ohd}G}NBDwT77UFd`|R?;bMOwC&#`IHQdvI-511OoHom+C6%wd>zLKW} z%<%)RL=tOqbixJKnX1TG3*aojhJg9h#TnFRgWu0m;jZsrsv0M3`RrGx$UPe@9V@DG zY{$y;hsw}FD(9xbc5U7cg11|s%-=XkRLrY2VwGw5@zvd`5s}oquLT?`#TwH&*nWHS z;Cuu}Z2&}J4cevc6{+mv?8O}Uh7wyK4{ooj~O1585mwyLA?XjV)HF!rj8GcbLHT_+$nfKSBCPM^#+6^78u(#jKMWn$!M0`9R{ zLyrjR{nLCA#(6>XrYO#pdPX!hk7$27MK44b(yX#2;V$|DWih7b%v3%Ru7F@4t7jO_ z0c1pFje&J6DUoSa0+1vvgZy?kr|Y^5uvS9DKD4@gJRK=+y+L8VC2+hIwNgcHthwkk z+kVq#znSNH$XT6B{9Tjd_W}ClY|8IcpSm^U+{yR*)ykM{$k6@2FQ}pTmOM+Mk`5Gg z#7Dj4#RsXvSKRQ%pMNew=z(z;yg*7`vB1r`pSyrrfhNgj|L7S{hBYGV5zy8d!9%iW zJx7-wW2{UzJP2HEU2?xWtgbRDT{@Y7+wlMNN;O|7-H0C%MBxy`Ys#$iLe{^cet!Xri=DP2M@{y4cu-N1lO;U&n26ULI1Eqg#@Z+ z^*4_)su+sT#1P~`>nH2&&y&x(S@q{R+^ID4noWAG#6n*zIj_4`&D~6{NeT6N)jhQx^*XKMW6K_LHH_Z!OS&Tbg}{|{?{^wc5GGfHYaTD&bq?r zWOt}CpxVjfxnfM9%^%U^j06~rQ_{SI@Y2q^*=E-z^^Cg}G_H6;i66sSpAOE&w8k{w zCjs$D8Jr-eQ&CJIMM8Wpi7V*)**@eebL);uf)WtG^F#p`Kdxctr%gjdB(7ei=0AGa zu^%pG8f#Pp%Q`-1UyK-JVLeW_N)NfG?azN}_d&dX_Wo?FMU$&*EeT&Gjq}s@`%{;e zO#g?vJDed4F_7)`aqn8w68X~z%d2WuNkv?ryY6>w(w%vS%{a`(qIq>vHQJvP{47$} zR!^pH%xW^nFu|K9T8sPNzWe^)L@WGxUpFawoM-|p{U)6%8O8N7(Ql}U-V?0nC*&Mm zWnW_NcpCKte8pxej_jeiw3D;oO85fXhArbojkQ(Q+g7YjYsMU!uwqg?BK7m<{$*dx z)$sOMaW|j-CGWZ@d;H!BvoeMBWl>?_T8#YfMmG>&|jE}l8c@EGINcpN#Qe$Yye->D-#Wk&1}@SLUk@tT<*n1e7p%mtc5Yc1b4eZx^C%AQ`ohmP zS=3V*WrJtaY8TW{f8P7>9s>TFuxjh7m0kRepfxtXEr;%i`>pjM3K1;{1bxMt(i_gU zeGzl{#BV0aM&CO!ttfjCzR3hKhiR|-BF4(m8`Yd=qe$4qSm1lhR@q6_>Ku<>J+5?$ zka6lgt+lemfOidZeq;wA_&m8@XX!ilrvGLDRqSB=?MNqSVHTRVdfs)eN?pbHR62tZ zcIMs|k{~On8eB$4<6CAteN8IkJCa$90ct*5zBLx*)$n{8!4inqPT{Z(6kU@^d3ElF zVw)*o*tp@&y!xAL4)?2F;ZaZ2=b~I0n22Gok!vKb zX*m}{qCbBX$YHXbt}a9qGeY-oN&co7LqcpBpW5bJvObiFvUBO>sJt5vzxdho{YBmd zt@xqc^x}fcxmLS73mB!lbTU9|;}lVyh(Ob$|waEZjk<&u9KJE3^j$6t5z3y!!1TwH?0* zdo~=zF>`?6mU9CITnw1QtknK|i%wRq zi8>uFBb^E|IxsRO+08f-4}|t3%%h-}*5|EQ&r@~XwBRH&pYTQA_f?6x#i6IKWSby?TPxw&q@qwN=f|{J#$} zU;w@14GURBz9W~mbtA{*3h9o(1zZgEMjIdxOQ{Yb8*Q}@zsn<>89;CYF_P)AG;_uB z6-YVzuvf9mowBr%Z|Hsv9yPY7qEDhZ%tomxk+}Ps3y-YTY`@aErL)9NrP}-)%lV1|?`-Bbn2o&g%&+2Uaw9IF&a2*faok z8E!KUgzt~5I^X@E_Z&HU47V}l06}*l+!LEWySlK%@EPgWr;9(OkmAXmHa<^m%Oi)^n8;E!Qi09Xwii@fu^o;E)b{fDM_{y>~2 zAzn-r_@w6xcbmM>Hb;=;SYr{y;>UVNM+I0!YTnu}24k+K<4*dmo zozoj026*|65B}j*5n-=9kM5a_F1mGHDY~?RXk_g_$p#ptcy(UfVO5|f2){mROQ?_d zp^T}YA55mFapROy4y!&ti-CVrwcjNvjGHNpmt5tm|G?;}BMqvv!(GENd#ki;z8!s? zRX^sXM{&sNE-zyt6H2>v5R`!HN@;-c2XW?k2HIy{>rUt%r*A_-mHn#vGSE>^@|vwA z+O+#PWr6mq6#nhz_~X+~ode=ju}D`iLVRDm+DN;bk zeQ3UE>o?SFJmNKKZP!|VoMkP0`|wfK?Ww_)o}(#K>ZB;*pDdjPY>MwOB{PpP?qkB8 zqpYWRvJKg!HN?i7nXp)n*P)N(_{kGJ(~vm$D9n$^&zrxK_d~L(vrYHggZYzbgbFy``Zk ztn8!Am00Ug`C`2__;W8WDereMWjUK_1XJpf!)S)T{n7>0EUzfMs;L-n!A=n((-nSh zzA}Um=s90Lt7jBO)wy0HU4)WSx%N32#)0|&&nPz=W{ z4W_rBEEY6*>e|6QJO*>+z24e1fU(_0XFQ`h)_f)(h|lC>UgLzt@}cgt^uwBPMAF-D zD*_5ktpo_|g*RPTm#)<#+K2xet&ok7J`*qVF4N*^=kxRjzt*-s*90AVp*dO_iE{?h z0=%qQgl-@ZBqAIV*yNT>KiM0Q0jTaAR3DE)v5UpT&c^8)hcc+rP#6sf19j?U+gfR* zKQ2$(f3@uI30Az^naxJ#mp2TZ0i64gypwRDj@tXIr zSpBR(8&&QUndk1>lqMf3;~!2noI5}s;WbcKG>?ckInzoQpd0{9wH-^gFqXE`lnz^R z?wT@Qx{X`oP*Zp?d6MKWsV3bJ|Ic>;XhcQ$e0_k7Va6SdQJ|4zQz1D3Unz~)tQ=^`-tWnBE zsOMF>n%mpPMVP4j&s~y?sL44;VMg4tK}Fa&#h1pSf|7w3Y9#I``K~gWo0^9q@K$ll z5O3p*&<%~-+BNbBJIl(8crydy? zgf|{OKJ)<2Cycid7ZOTv?ekY%egZ>;5lPxf$%Bdf5f61|GxPde_x#yI`sAfb8J@^* zBMv;}yR(?~qFO&CE^xs(04AP~k0QiwrGniM%R76H5OPvVtNd|Y@i{j63@w<4^-Uu} zl6iP=l8HG}u1(H`G$d7`rQg-)Qj^;f$YP)=4AAoDm1VS(vQ_BmPAnz$5kT0S@H&QB z18WO#VA0W2pRW*$HhMf5uJ5ZfpRDHW%HS(s;4`geqy~Uki1X&vW~ACsu^Ugn-O;D^ z^uPLt6%@lN>3NrSzB|>QX6ty4yJ1lA9fW+x__8XgA+|8QaxE0j3|L4DX+UsbeA}6a zTL3SwlA#qgi_F@3j4Vr)Ik|z{agLAnn&xsLsIQvlP+@zGkzO=4Lei#;`)xnv#JD`o z+fA%l>;xjzLt+>$pkk!%N=4E%x9I#r6K z6l%!X?{72X=ikTAj1+^?)y<#3^rTZ{%4pi;CIzQzjL6aRzGP{LYvrDpz;R>;+oI)( zJ$^hcOkYo*xJ>IQ41W;13;sa5IYi)&OrFzi0up7kb-Lve&NJ?~9izqf{dJ9GJzGoZ z`|Fa0AS*gJ^u3j;aEtk`EA0$18fufx=Dqtfl{Cuslc(?m9Gy+`LZ}g7>gur^D}k+N z)I2Q&l_n`95>Y2bJ*~!z1V&&LY#-_-VGA%&k`=Z5#`O!jSyl$5mQ~N-0iOIwv0x+s z)M2`T4Et6c+XKXE+&zp5XoY@!+fM=Rv~FMVPD1sEAJ!kS-b3k)c_jpg1!plHE=uD# zKPJ-6;2p((MGagnm!u*s%WmyH<~upO3H_F!JD(}FQJ`S#CwR?zZ{LS`1kYvEq~f9X znDvzylSMV=KLnodJy`Jx)E)W4N`zgxpSDMFd_gmgRnh>o8Jyn$*E_lk#h(t>`y$n- z?r>qME2D{n;o15kJucoOLuKM0)Z-UR%}jS_Vfo9`ZG_*cXh>gH>QPWNq7}Du3zukB z94|Zc$2UzGACI_;9k?G$eJJH^Y}K6P6jcZCJPm6vT-5-UnD#zki>aeNLuYES09|W& z3r%5IN#fzO)jJ{=@~KTMOgfl|%yi%}emZep!jO53EAuAn@h7(YrqOnB64wDw;>S=1 z%$bqkS~S?B;zRe_CT6qAcW`Wk)eOS>c8t(hn$skQHLB0q!}@8F2@XLf`s{F@b)E@* zg(L9=*y?(JT;Hi|CaIE|`WUiSyK;9p2YdJ}UWlhkg(;D<22u4|)1YAdh&eu_HH89C zQ9`_}5d?gHQ6m&KrB#Stf9u7G5kJa(n-h-Q$h9Z@O$F%|bO-DSA$ZToh45LNKp}lI zE$%mE_8Mp}$yNI5Eg~cqY=)sO;?B_di@Dg9Bc`~a8LKi~J)5EycWU5DJ_2Oc; zN8wXjZJM0(jhgZ%0utz4bB7{<)ugy2kVDd2>cr5li~8 z1)vhjQF@8E&u`vHSU$%!zeo$XYM4sw6U5RpZbrK

    MH+rb8(SxYoPIpR z`&uX18pTlAO2TKhuyyXlLCvf(_wpgc19OHo2ZWVCNOOI?eJOH$=8ezV-_tYGygxls z`&(#a_HZG_w)lH6X!+E~C?u|Y=K6$IUEcO^hq>E&meR0!p`Jv#oi{9??=%srFf49S z!oUgL;wjXOSzV4s1%=&EszF6hBswePnZOfVza|e5+BiWLwvf$)k#OVzcnUfQF($oD z^%ux&?RF$$nf^w-6_FRNE{06(?uQNK6g65AOY1fd-<;ulFk0Nmu9Kg7E#4;Xda>u4 zS^HU6(1ztdr$3jo-#o*x+$QwiZ_U!T@0xZ_D&vLj?n(C6gsGXR$VW#4s#mkmvOoUN z)4D;s;mV8(u%{#lz0w~hTiZohgXcdJFgG9kVzFu`MxmTU=T-(~q0Uw`_w>M7+RSA> zzj(KAH&7_odfBst^i4Z3%0jRX{LyxCaVu+T0$Pk+RQ)-l0|j&J${W+0FzW?BDB+PF z7aNxq=WJ-03u`X^;Y&Pn;1_2x!WK_b`4T^Y8=NuvBf?!ocZ1uk9oDc8vm@b6(neIrtwek_-8#aW&Trjb?_NAG8Aud)E-9~|Zp9UV$-ZH<7 z5L)$_n-yU*# zRn_{4cu9WwcuA`GQnf^CCD8t4E^TMet5#)g9~@_<5{_kJ|No!t5Ak}%N*{}bCJ;wa zLD+^)7K~EqZe=I4nc86Q;soO+#=)|hy_axZqc0+Qn-OJ^%*RYpT$k`p;@#0gZWZ^1}K@L z;$zybG{?C!;-uHjK?2+)3@+C_Tc19eA$vTL$2 z*>=t3ns8-%vfY(!`)ls+eeUOdkE8!P&VCzfueJ6LFTqxHFd_*H3riozwBO}$nlea8 zp5ijQ9cts`emmi!Gskf+9Yradx1k0zfph5?8RLeYaG1zbs`okIZn&>-iOZuS?8Yr2CuRpMFD#^vxAUZV5G;06R=17u=mRO6y| zyM-J9WZ&aT;5-M5MDM@l zd%>p)RJ?+vMg>33gT!ITpbyplhYefDl2fSObknkHCBOpB{~>si&)k7?SB_Mk{`gmsxnMy9-BU5?toZ*_sMKm7(W&2gr2u9&+8n09 z^-RE|g93!!FJsDB$R&08eP15VNM`KZX3e(GcpluO1hj}!|LSPox{z*bt_x(YA{QR0 zrcd-b4ZjUp67Hi2*$suE&plIoDsa9JegZ@`kc3QzD$Vg`~EdiO(MfySoEe; z>a-_YYY;d(%`J0a(uYCiyZel#Fv!tKMLkUI3@5hM2kD){wxYUEh`=QfZH^w{{$)dv z0wTz%?i3eh1w+Hc&VGB(m6_@Vpj6WJGbll$5T2uh)cstgM2^%R3f9HmGcW*CCxHh+70M1Y4clKD4K zN%+7bl%R3~|CEfqaRmztDt9Ftc@)@j7EXl+qP{dW1m?H5WvJ$%p|f~4h~D``>;t_o>G|>TACPtn!Bjl-r!H;aizsN>?LVVDqA9(FswA8 z*EG2r^DECVgW^Bp|3xiE9O+jOt^Y5a0{z}YMN)c(%ks3TgL*N(0Tnvj-?=)LyT34~ zj$cm*O*v9#OWKlex94-0RF@j}Gm(5)A$O;WC@1D9JC=%l_7FUGhxx1VXR z?k=xt!X=#EMp<=!gk)qWV7=~%gkIa5V11hy=(fh2yYv=OB-bd?< zL!YK1O*r;%Kw1iY5HV+6&5JxY@8BvBO|-gR0^xQ&&${r>{mgx9}YTwSLOFNBhRlq+HYU4 zu{8yB|If<|{(0FWx9!LFcbiO03|Zd3NZieHS_&Gt)us>ORQ~c{4^Sok!Xj9h4Kuns z_Nv(RqxZu>Jg2w}Fd7E#gvVa|3?K4Td9Z!4Y-EX+dQ5cJ>rC@CLFWFh`&A-!%sfSp zf-m}~liF<$m!(KkgSVB+p){Vovx**5EqBD{u z-|^Tx@sBiq_ZMtv6mFZ7PNBg6VC*`vG0r7_s!cg*FXt*0;)95&n0Io&FWL zi1;Z5TJM|j>{B?I)o5?pYvDG6q~XXWzraT+GS+i4N|D8h7GJad&OdbM`js>=nBmQo zh*8%>g6Y=j7pgQvWQZ+d36Y~<)X(H(Q*a@7!nCH# z#zax!xQF4AH0*nm<%&?Rt*NP?^;n~6M6p-)keMa%hLm3-UKHZ0j_f9|G4#V3GgVFN z1pIEX5G}W7K%3)GlrSx!@d-)LPDJS-jxSO4MqydFunOA2T;b1!b{!z+ip8_f4W$EWs*2>2R2-SkRJ#FfV&alD-4Jt2C`f4{48MF1*_Iw7F+CBJxHL{2#terQAtOpYO2dA= z{=5LF?`_pb40Gbwwcl}hqy~KR2>&|ExSPVI)Aw{JG?D8xG0ZilizP8D2^kp(=$D3d zBDiFjI!ayNdPXCudSUfWOnNlZinPzJ@}EX%fH`vex-UYNSfwzRJUdt1meHaeix-7S zzfz{0q}%*m-8i*dEMb`WG1f;aI)l5stgME>yD*!t7ShM%3kflRL61FyR@((A_FLE`44B_&8jCt~-cSwfd zX$4x`@8gIfa{J$UTcZelk=JoDxr#&jrV0Q3likS~w%X*}wU(cHP00RvvA(dQ^*`Z) zo@;X~fIJ8mv!i8KeK0XI+OVq}izdh9Ysw6eWD6PkVQkAqql?V zxBO_#S9jB;%X%9gPXYThc_)4&kEJiKEXJm$eZ_kQZ^uxHB~7q@uUz(FfM7}J~3v_aFa>Tu6yRJ?3%LKK5 zCK1o$)wQ+pNt%S|LKOL@l4NFn#u9NHkz<%6tggjcYiE&u4ax2%@%^6(BSS9l_ircm zz9A{LLze}!at*4-uv$@BV3l4w*sEJ$!r=riv*~2%%ul5c4>>&ilF0w+j2s_OO$Tl_ z#4lZdB(Set`NN+in4qp`Q^fgoBPL0<;xJg8%^XwwEZlmLN0CN8&!|{#qP*(QNzl0V zcZ+RFQHtjh^FbSaoG_gQ-`5B~P3ZJ5u3se<+9l=rAyGy}B2wI0K4IjPVY~@u@v0lp zK_}>o=P`q$(pi&z5rm1qO1;Km;N|?P--ywXs-#4fh$50Tib(L=ncK3&xM&Nw1?nXEh+2^~f!$%(?feN@JA3%XuI~=?`E@3(|$&xQtg<|-j zQ|STme82A#yMC5^6YKZ(xL^K%dW?jK@Fz98oRXs4rpQfMj5k)hY(vX_DQz3ag1lk# z;Iy>cUg0X-Ax}3({X&_hSknm?-qCfOiP7jiGb{;UOtxO`1uk3+Cbs|++fK3H`>tn; zBKO`k#A^b2BJJT_e=zf(ElR}ux96*{aW2k(;2V5Ev-7^4c8%rKM!blpu>>p2$Tfzh z`|)&x`9WNE&`0a&#gK4+8DwKheRgh9SO43-mjKK<)e}G^C=FHY2t4RHG zcg9u34(QX#4@pZHdlxX0KHBHO@B60uYG}>wf(>_Y=JrwBr#Q`;R(f zOkT5{>+$0>e8s_!>1k!9`T_1e;hKDi%$h zhPkk0ddf|;B8(cSKKSx6mW5}cq9=xqN^l_@aUee4X2-Ky2&klYn}UK7u@f#3`>;BC z!hS|gEx~8F_q?t|^02qb$3ybh9;F@Iby8M#k3(P9Qcz)?I`!b&TBjLHyv`T|ZSg@$ zxp~AZWO8liYc;tsA~LOR8p8kQb|qRsuN0wvtspE(*Lmt?lf7dg5T3T^V#L220>=De z-Ptw5z;Sj30jG+dB4+Pzz4+$`6X$Pl@$QJDe58AEbx&#@&FCTC;uDyRj8xor?kQZ; zHkyV@a+4#AbF-`D#-$AwQPxNuZ|khIwA6I8Zsw8vIm#o6!L&^NaOb<1lerGMsPREw ztE-R$-rDFm*7+Jt7SvA6GwAq^JAUxU0gYtL7UNmFO45%_iUPN@J22~8o!hUQ-Q^2aG8bgFAxQ}d=;84} zti$5Qi3ZwEH?2R`^}hUX+(#q$H*X)ae739Yh4!dlu~6UEx$?Aaw0e5+lgt+S}Y<8I8(mm^Si5 zIBnNW_ny-)+KAwbhW}?&0)SVyzg&@Az*J9E=yi?{lC;Ac2YS(XiN)c4+^(H}^c}8k zTDUq11h-1~k0(`F1wD>*3Hm3}l0s)0jRLMnf+*?xiYgPN9XW+$q2$94NQoF{eA~tl zMRxO5Bk>f|8)Q^RJw)JG`rzXkt1SU)&=*FPpvm$y-+NaCp33){HLCknt4U)5@H$*_ zr`>skS8=Kf-&n4^lfQ?D>d6A25U@upq#qJ5zMQ!RJ)wYL8Ur=uif8l{7#NEYf-n$} z3nKn~O<`DU8@xT)#Me&8LLf1U5?#@S;Bb?Lsj`MPHpoqiddz_RJw$ZNGUngcT>kT# zeLK+b_FKICQ+agSppAHi5u^ZokR)srw!})zSD%SCbAt*Gs~4lfrGZ}?n=M~zNCphX zA~Pp(4=o|(8|%{Jbi*KiO=Z~dO*y}onZZ=+bv)>WrsdLSXlO?t_?);aTE|m&K?-)5X;P{X8FY;m7AWub(L~ljT3(KvW=` zgbn>TGeTvF*=s8y$Ohe-j5J2f6F(IZDjAU%F3n_DsGSMdnsy_FL$FOnkPvetX)|M- zYkLMMX9~1sy1ZHITp4m(5PYRZ!L40caoNT{yK>^<$awwn%qY+XOfUaHJ^`GR{$8@{ zvGxOet!{@$?x)kmpoZ06=%bz^^8bFIj}6#GH1hEtf}wJthGy7D8Zm@JZ0I1bmijFg z7kmofN;e3@<*CdQ{_mKpeDg6mHBQl~RH$x(f)Jwf zJ^z28Whed}|6(SB*KIi(8a7Af%6+ca8v&TF&v(}?uBVZ}{0KT6dR-4WFf#Uk7sT^| zP(MD5B=uDmU&(?n%?n$v+KWl?&Ox6jytf8sy=ajwU#4qcCI?&hm=R#Su3sy9uc?8P z4UukE+Y58TQ{wn4+T;wMQSr=epdFahOHI#k{(JDE9mT7=GV{2%CW*hds}+=sqm+)y zQ24RzDNKN#rmh~6>GB9nNlm66LGbQE@`El?rS%TKakK5!Q@8vFd<7s&8lL=8fq_Xk z)YkjFTdCE+iUv|JBr7(^?a+@sTN7AiyIS}FM6bW&a5MfrrZ(va`=OopI;S6KI6{2s z8j}=b4ke3p{lRUc5(LPg`G*vlEQLMc{5DIRPU=CEt(eMa(99a)CNjA3 zQD+9KK_>J+g)rko>^HsDzb9 z2obE~D6E&_T%L-^l+t8lx{@ho#mFb9;#xLJLi)-iDj8Xj4uOZEr!mmjYAPXllvzjo z_&l!6trlVG^bp*uls?>WyuHZd=EuTwgZw@i0+Tb~^LKf~QU}}_n;z7zx9@VQ8c*v& zX$}Ofcwc-Fdw=zC32V1WlYlbtFJzoacM*BssFOmewm%p2 z-gvC`p;F*QIVZm?m;P_0)%`620|sEj;{6SAH>w)5+s{m1hwKf)kpY;{K-wA}u*vn# z_?)TTdAF8yDY|lmxXZxxbW-oN#pvlF5lx!Ul5fn-oc)qz7*ri-#Ob!_c#q55@7^&V zMjMos!uo<^=u9*#4YoHMvvY>SkOh9`+u*93zlyh|HF$)Vtek8jC8dXr1{mcN^I^4i z!?%6#66^oJyd?Y~I_@7#L9lW(#BRHh@UgI=17BzkhFU-7@y(8SvUM-Tds$hc3ju2RUP8 z#d-Qxv%G+ul9HmIT_%?eb&LFzul}|=|0{HfU4~k9`B@&HBrPjusF1a8bE5V%#{t)L zES?$_?G2lERcNY}RKd|+&~rxADGHUNhTCy6?+Ol^YyI;d3bC$`%)AEX*uz_=*D z*t9uSaU&KO;#H~JK2Tj%N_#L#cL8V=d4(~3xSnB>@pU>|ZGP$aV&xBDj3r4Ztt5ib zH@V=mMD$%p8ss^M*^m&pxUPa(w{{<_ey11!f-JVE>@KfSh0b3n;ofq3&S_q9%8usB z7rg4s$3D|%Hw5?`kD5nbgM_HkF9(8smGOf0Dc6N=V#Z}qvM}C0_PQJzGn{*!(YNm7 zNve9>M0%7!0^-VZz4!aeFqm%U8SlC6$AGwkmJKzx`|sOLe8}D9?6i?Ybm8&g34JuR z_X&Q+1(MMbl%aj-(GmECR4mxS(eR5HULqewKm=6;1mCW^`LwovHu}I(tc%KGO_`XhG zRN;9oL&s2`#D5I1(65FSsV-*iI|&XkzlldB962!ZPuhEt`RJQV8ymmud;gT{^%qMj z7^`E+4X9#c<-RBDG`;ys@B|^RDv8ljk9TdP;LnJ2on2ODy;&yqwb4Uc0NB;!xUH&< zkBTwIe%juc~jn97ds4n|yu-+Z1*o96?Kw}>>qgXF#(ilR`jdN{f0RT8LPPE2CG zm?U|1>Fr5(M?y;{kD{^2qxeMGD{XoINFuyL!Wz=_11Q`E)iVR0QDBqGuIE%bUSda- z#7@Lb3Ab~@$K|*szEGl&i7W{f+Z_zeM6@wKOW!y1)p-+1?A|(02uX~^MQ@(v(oqo0 z`E$~n@pV@-)-&%{O-!uD_KUE@!YUbC)w(a*%341T?E_3{5H8I(K@cdqoF7Y<7LjvL zdMMoGsLT4qD(uF=b`P=L1E*T`oX?e*r4f{|uLITClIa(4g&T-^o50m6d#$8Ng9EM* z7&G1{`7mPBJA$p}Voum_!PnW*l!U+nB}@rzL)@dDte30_Y$kmhk~ti+i1!Gf@ViaG z;mpr+1IDjcB=RX3<=?D}_OL-O$XSD@@mJ2*>`{ZKp=I&|p&cV4tgoE#`)w+elqQCx zmgsvx87*U~$CWm`f*xIaX3s}<%d^>Afqgy~#B{KVX2k*BeW%h$n=!C)Or04{o1==St zrdwxCxfc6UvqEAu5o(@{$j8(HPE}aQ659BaK9OiLCZ=zYAp)vXJ{_&m+eeN9T@4Z* z2>2LC55l*$9C5*^3m=sj8R&V@F8OYX3sYS>4cH)&(DGhgZgQJZ&Su>S)dGDSnWQ$>RT$uRMO}te_{LetQ^lRb zy$|*zwUJjR9g7fKD_+HThjifFAF@et;51e(gR5xe zZQ#@40`I2qvv4=oy2{@0z4QRmjchPEUu=*ao8gYOMK+3FzvkVa&p9eUd2mmrw`MEm zC_TN-n=&M1*RZh}ooR~^oxx$^D|u}E%hMo{Xa4SzdI=j0IUmBS6>^87{ev!ITMRei z1o9bXcxu}q9Z>G$_)KM#xvKYo(1V6}i#HQEiwCKY;Ku?=TI$w$-j)i9e9J{>;>GI2 z&0}ZfQiItV4P6YZbut|)%<)+g+uPePUv@&c!GKPeBLaY5Yj)3;&+U)f^UX3?&{c6? za9`{n&M$=fO#l^KH0nq@`mOcQ0m<}CiB?8h78U#~E@{;-yDnNhueoavQ@A;{(>2x$ zw4vr%(O204%`rdAbTM74feKdFjdFsWD2PJ7FUGYyvX;F@VEtJ~fk$k>2L3DH10EP{#kL9mo~&`MK2!<3nxY zM?EuH!m+WGKgKt+3rEa#?*+|$1@|3{v@o%=>$dK!C>Y0>=myBwc%k=HE6{wq^85E% zTH+E-%o{R^VB6xwBV%v(A1rCo*y~U8pKnN>C+SqO5uO>-YWo`^IZ^u-d)m@Km(o_W zi@#~FIMPtI7*#)V0FNBIU@*ocOe>G{8v7o%bjuNhBzD2Ww}?0qw`V@;9do}G;i)~s zMYM@Zy|n#=`)92-N@4*_mNg~$H7C_v7;SzO?$~nWaACETjgtIX*g$9IH_DsipNF*o zR32r{cp#p=>t0u%W1@Lqa|rJ;t?y7g zVJ&}SMa2n5=2t0o2UOm$z@s+&naQs-xDsVIdnwq(#g*#lhS;##4!q!c5fN~zPR0qr zZj@x=qUF#p$?x%I_l1`V59>O9rkg{eohPqQF>1}8!(j_L+B#SsH#(UV^xcJOy}M10 zFnOb6f7zx2IApEJBe|;|Wpa<2zrvdwl7$vso}XWM&{iii>A2d09fhJHYiE=~gIPM^ zP?e8KHTd29peee+T)37m_nrozOIsQ_7nN*d9c<(RCM6T}SRQO~hT2K4^TW-^p869Z+N9LbOxNh_^Fnln8p;^Z*ZsHm`1O~x?U zgx3`$CK?{~)Q0k|Nr)xinS1_Su|EHwHR zvLz~I9=jROMVxyObB+>jnT%(N^b}^W^4Uo&o@9)W+0a~9J#a~ml9F<8$oJLaq2%Yb z3^>5Jvb6YX!$2~T@`ilvekY+Oyb?DH24Xh_b^)nd;1LGqqlY3A+)s98K5jPr2=i2nSUM#h$r`Le9zS+wlDWV3`KSvWWhEL`h z8uE4C4}4`+1J4K)*Ll+PZ`AAHm)NkL2!?$vl`P-59CPMpnqAfey^7aHdoVkS@=Vd2 z-t9wi=k6zwjZGG2SLxAa3&N#}2%8yw6a*s>*u$;+DFRjNF1;E|>kW|5ay=O(I@o-_ z+uaQfZEe-Yr94GXSn|PGqux#rpKt{ZVfy_@+LmF2`}}mI*YKGOhdW4*p=#MNt zw(NB2g-56kaakQBOtev>jv?&@t@0BI2SGDm!QhHLbKvsSIv8)6TWwE}F1yh!v$d`* zR5`bI1$+{p?-AN(K^d*?DYTzWDl(;I;BY0~Yvao(dfG_B#u6xMT0=RBeOd^`!BK*y zapLMLjh(FpY_@6=OmcCrGDlEQLy!(}7j1YvNXFKd1hyi)CTtDxy12FGX9sZ{s1JE< zzyTz1SrA#51W_cJS+ydloMf;FnIY*52Ba}pu}QzGYF&{_!@pu(M#8z&ZZKDwIthoa zq_3t3MySeu$xh11_*DF}Cf?b!Pg22zTd9S zy+dFiDR$5+qgEOkdxFE7W~QO$!hP1xD|}1>gTsx??cxxYxSWJUe;oN9yALA_zrzMb z=q6q}&4Gc|Wem#3nxKzMCpMU~UGdgki;3R2`hG)eVIex3I06>D&19&Sg!c{(2im{4 zf4O?d5{`RNYJ=KhK<&W>l`;?LejaEn{8CGec3Gq2GVr!VvoI-+ukpnNToFE1xa$lnRI6^TBngl$K58CAOoLFo8&Fp)7@49JS8igudKXiWGwSbWa$&f zEBA@xx>TnRZ`-efhqJAL+r;nVe&!-9k`g`>qq@^iht zgGR0C8ZvIkQxZZzPMDQOsuXw9KaPD?KU6lThe%z0dZ1G0OBF;sfV>qu!E;`u66#M4 zkr|l#De4zXDwhjiv94C@hd1w7i2cnXR}mowFa>wkXGr#BQ5i-jiMtT;aLu60XdiG( zK40>uODbE<8~Bc5G9Q%^=^ePW-Y>HUuyK84b1uwYNn&36NPvX3hej-7jXurlTKIp{9B^RgG4OeJV1=1<_`ShRhBbDy008ltg z+5u%QBT-BZ--CWIeDJCKH|;4IKkbFBg-_D?8RnbUC~sf;}m z5>jSNbX=d>+T&6t8B4X*c*D|$W9*}pUP`k_-2l8@d0~AZOv>UxVlI(D0{m*!eN@9 zzRc5Uib7uRf{gDvmA}XYH2ZcG5yTpaW&IKR4DgR(NDeCup^1!g8qNgTCQhA~(py`p zTuQw@g8oWY#5RrR;53)CT|?X793J|{+ywT}s5iX1pP&ZrAl+HPlAZ=6S-o}Vrx$M+ z4sz>y;|&ac!U1A+W>f@{AzsD&-z5Q*xxG1)kkkH<_hPiX%W8PvzbkDUeEcpfK-R`S zQ#aQgBVe~PXix5p){ef@;-l1J8&D%>ffZeMnQiYe1PW3sU?GhcogPWAz+K>*1yy z7dsMgOL>lbZbB(U6FysBe`$nNiDp%!NBFUy?5ZUdhJ};6#9{Ly_?rO`_-L24`ks*0 zu$si{IEx3@s}CTbqsp^LytmM+X0BlsNvy|d?{M?fnNxIfU9nT0_UBoGCFXp19a#_AOxh~bfkBu}WU|Y|V5}`Yk*=Sq*ASCZV&+1@(lmf#nRKxT#9! zt<=NFU^IB9bcFYGLU)FbaatZ0l*~O*E64j<|&1R!J8ag!-5raqI&; z5Q3|qyfaqmkYx_O>p!sbRPjrdk3PHLb6L<8@j2Z6WiS zSD9aacoei%bJsp0L-Hv8L9mB$@8bJ)KY7aME{j=;a!+f`z5HVXznRBV6;UxJfEzL` z4C7Ynj+$}7p}u^Ez1!A{PC6-*ARj-#IRHbd-3*d^t!%mE^A)W%-RBOc1Vz7ai1>!7 z&>cSr<1pe;rLvjcz>^Akd{k1R$Dh9=BRg-x8f$f6oPxwGE(rap-ofvU&;f8EFqTuW z87`Fse=Lda8X7#;e)ZBjjmYN;3Y_q6w4_iH-t*7)Wwb(GKT211V_v&HRfZLMKg%X& zRe)N0$!rANz7BY*>i~UpRaIGC3>=LQiRRVMi0fJ*f-MDFi)%0GybFJ-Wy4#-EiNAL zVKH?#R^}1qJsA@mkR$<>XKq7!;UoPBZCPco zMG~f`+N?3A?QGDpyf++?RUQI~2v;USts>a010m@}4`=JW)CnOsKvYI>rWDQDI{7>+x7ydx?@UjZY zkL3a3mrFgHteM8s#eCj1X>&7ESv;%5BI0+XA2~pAbxQItxZ_Zn#Ursb5qv+s&or^1 zq|TidAtO|up!^by<#l|%3GeMC%_=MImAVBl@ zFGVm((E%CXSGmzJseWyNI}Ls7tcR854)R8%Td?ir*JW@*JLJ8z0^Ve4FiJ1F@yh)8#?a(J(;@ z<@{ag(sbc??T0l3l!>_|Y`;Hl0PY^;Ym zqIf4323H_u_J_zYBo@m=fvgD+s3S0BSL-340eivNTG>M0d{_aaK*y!<$R`O@Qv#!c zpN!8V3%X=_M_^}(%k`My5yJRo`rDxG(gwRnwS?Mdv^2GHi*1I5qoh}8E7GrPtmyUe z7H{UM;SEt`>Te4XFM~offrqu3biBhtq^5#Bk%MMzCROck$V5R}5ZRP!b(0~Ye+^RTd57cZ=SNJir)cYYL!3*^iz^l%O)sBmw5SLb-T1YPjg{!0Q1 zAO#Xi-gXRFT3uA1u3xDE`vQb+siqdRW)rPDD~Xn2(ctM6K=KV>xYJ`<<;?v@GP~_G z^oDVn#hs3ue3)UY4=tQ9E*&mGDaXCL80S4Vk=%)u)x*I56Lah&VfcBctBQ&x!%Wn?`f($%)cPlJ1G_oyl7XtLZ~D)#jry~mfe~lxs%tz+ zpg#qe-+^>)Y>m}}pgKMvvc6D+fOkuT4rFYPTXn8M?$?GA&` zop_dRxWu>ThgU3iZ((g#cM7fH>Np@%4u%x2y>Yv{ z;Wi&CsjEw}W>~_On|S<+G~-B8_-zdlx%arRvF4d(afZ#}72^2vwx)wV%J?=_Y#-p> zbad+p7v2V|I6QMVg{bmA0!~p&bM9&Y_T#|hl?pd~yx|WoZn#T-un2)^8QlnQl#lT@ z(IGMH&i#)i>u-6N*aa)C2&9aaF!N02OyH3(EFyxd@Suq(i@*Ty0l*%cBT>^1N@7nS zzxd211?^r(fRQAXrS{g>WsGRJm#pRSu!qHwAnq#cS?Q$>9gw3Iw}AOB+8F`I@GgeL*VyEhLiH=?89 zA`k23{Ld|bQfH=Q0_q(0Ft*ND4qJzW7gW;wnJ)`~3;FJb(~<)K zc|IS`?AhZn-DiDZwzN=$u#c*(0zKcO4_Al)C?Vkp}{UW-ZbZVE8t9_#MiuJ`y zN|dY7ACXr2nNJMK0iZw37D@3tR= z+jtX@Zz14x;!Rdb0Yb_>C5eqxCyM9*y+qK+O4+U|i7(D1qQyZiM7z*o(zk+cE;>#U zJZ#e-6elHXra5cMS5H;To9T{36AGCwoHn?Ai17v^ooduEIpERGe*3e|NMF;ZF`Oeu z3Dj72dSSoGodG#YD9S`p?QU2q^U@np6d#zlqTpa9w)L)j-mW1cA85(nJpYUj6$YIh zk@ujch}Mp8VCpJo3y9m_K)s9sJGG=>hAjRlTVNuh$F;*_G;!jWp=k0olWkiDP8=DI zZbEF;-WhJ<1v+an5#ruO!MISryM+RV`(aJZi;c#3KEWok-1J~w8$21V%2;G4WhJv2 zb;TK#f8(kqV4!jsR|?>XyoW~B+=<(nyY;%<CH z9kFb7ke~3&G$&Z@m#(!=IY37kGqn+KI&4ul%aUO{e}r83(7%Q@6zV4Afzg@tsg!%S z@uweV1(mWV!o3wy3<>?2F=+#CTdF23H)?h7S!~OfrQ-+9ZRoJBW_HLLZZ%P>8Tp2hAw0&$ zW+4!?Yr?9I!aY|!h(x~pr5B)^8kgXG6p2OmL{0E4*h1{=g3Ns(=r7nQ0)o{vgz_ZG z5Au`9NbiC}8B|S7cuhOLTg@G2t=zE@WhNOq@s5fOkZ!-`Eiv&)$OmegzlR!kkkdps z4Feph<}?=s{a<|1yrVynT%~`ldy8=cvoLM{0`uyS@5En(8D=m%>>In%F4bum3o)-{ zLe9x|y2!MG5Q9hIp~pwb2(t34 zn#miM#JC8RI7*##Yv%WN>S^d_p|D&7w1+j5eCF*CJHDK)#}+9keL^-K&s&dh@#b3c zU{qui5wb+B)zHz_rabB%@L^UzWI+_+cRCSGW$6YkaffeobU3}f5R>0I0tk`cpr`q{ zE}x5ClztJWgR8P^7}-2Ju6d54@+~^+4qpk^=hAKy&;YtTD;pY+w#)G9J$0&om<1-R z%t1i!%H?tGM>8@Kcb2tzn^a@OIDpR+Lw%z30wfec7^=T`dS3?t&ntRxk6)LIhv0F! zbrTT&Bv%X(A%Zc3!&x6sX2El)6AD4fv<5D6yY#A3A)HBbP7#v`2d~(4PMj?_mvahU zB&xi3cfU(ulMcaQz4tS83+tN{u;*Ta#~9)Icb*suVoi+Fp=m&3GWB7>aA95j+!D8K zmQ%TiXH{u4p{i-M9^mTLh6{he3yzJ)5wvpzx?S?YLDNsiuyFYgG0-K65Jg8jY&&&RuCuI-NY6>w+(@W&%(A{> zG<`zS_$ic!8ENt1rctxT(N&x0<{xU}5pK%+vbXS3~6yoO(FqwvR-A=E2U2 zVwq-xBnP_ysXjg~k6eAdlbz}m6=HMV>m}VNNi>NXNYCQ7ySa7-ll(lQ(Z4N-=ZvY4 zbf(GJW%dM}5?~xi#h-Unxg8*9uH6!!kW^&!)>DOCmW=r5b8@}Tj#w=3HcY_6WcPHp zVt;?YZO3Tv{mJS8g5%1&#W&7oXa>+FA2KmAKFZouH9NmB+%7<~b)NtjcXlw=N5X=a zTTo9Cnp@!wBlop3x*Ut+eBk*b_hYo(btjmD3)>QwBuxr7_<*!cQ0l!c)b|Pc$IO5! zxrvx2DXJH%=ZBW@^;F>$K;7_3k5%#3WA!J*5IqIafGg}bg)e1Ahb=|C@=fs zuM-ljx>olJZo_Z18apo&}p2d(=kZr2Smcg zX-|iz@vo~~EU^|;0>|%o>Ml0fGl`Y!?Zu1 zeGDs7gk0HJQUV76Q5m5L!aQOIS&PHKM2HHgqw*c;BCaf~@}E^yX*j_k^G~nLUNNIL zeUNu(QU0vs;%2R@`D$5)DNj*WdM;ygx$Ig2J&X&~~a7anfY(8~#XQPgg6u{z%!{jT;4H-@jR z=>e8#OMJ^8{PbJ)%}OacjDUC~l8uv0@M=@XZ~ii5qmx-mOd~n?$hUBwJBw9i=PIWy zZb3&wa659>RH=CCYW-HXABir;0?2#qoq^CLU<72}moh)LZ@~z^fx;>LRbwaa%*0d; z9^R_`m7wDcyQHI637A6&v}ADrpm*xN zInAWxfxsx>9`M7@t^Ble0oy4?b61hkSiQM0%a3pAgT(NSaJaG!LfhSWA+)QmDwg*QLU=dT zLs-c9>E`h0H&aJils!n>8{ub9cq*7laNhlQ`_s&0d~V9&h}1&hGqIGf&=HL^rrw7? z)x2H^xk{giH)~329C)sq%J$J{hlXypmF4Ga_#3W4V}elah-QsHljnlKsOBw#b` zp97Pg)0Kz~$zkB1PWN`SKj%y$1Na1=laa)PRe|y&g z5By~0YnVh~*@^v(Zf39GlE31KxZo8Wt~+;{^{V3;E*OFwiqf*n4E6LX6K?JX zW(_bw2w85#MD!38CC>z*(>(>+iyWbNeFEs;D==?ggt=Bh-zelha>C8kxfnwC6s*@` zx&LWdMa5&C-$l!HX+hB~sPkQ{uTy}-s*4I~ssYg5l%ASOopjLCFFoZ_!9GX+9Sqpq8Nw!YlvH2#LrGE5FJRgA@8rfJh~{KN$pbB{ zr<%*{Zc!xH3F|7?EqBxqF&+sUy0dT?KbIUh

    u1$YH|67XTdOX7l1Zyi8WD89Z2XDeBsU)RaS_g^QokBh7y8D*&!J zw3bUTK_`xL2j>lddp>}YZrs0$wdkfb<>hOgq}N>>i2NsjGvqpKly1d%Oruyl0bg=8 zId#-q31C~p-tI7{rufQfI=zYKbea}E2B2WQx0iGwr9 zWzSS)6%kyNr@477m=1~)(^6isxM4Y%;0WDKHx%zVt}{N-K1fr7z% zW)X;Q-iWXw2wv#YfX`ojOiU<>>K@q!~8;BPrU{sDXuXzf3OB_ z5KxHnooQ0WfdC?I3l|v4PFufW{bDTim#c}M;g~yTp$ytkY}QbQ9{Y4(0jQq`AUwqw z9tIG7ub`FVo?M!NhUFHhfZ!N}6m641O)fg@TcXyVin zNI5WnuwQ_Yvi!8{b`9rR6>2b&<%1aL7OYX{Lu1KtU`JX30LKJB92TA%`|6rn71aDF zZ*%j_4wDIKG;oP3&3^q^B$`W(GpArAxgx3Jfo%lw?3Iw5pR~TR;zywdBUyO>F<$h` zy@|3k3~SJnFj`~?z#qVzQ%Ak91Q|dpK0#o{X*zuokDR-h^Meux=Dw1~XNKet)}fB2 z346+`F@_4*p5X!`*(Pf%*RH|$@B>v+PQ$^>)&uee>ExpAts@Nu@pLHKGadv`J*2nk zg))BCwr!NuIf$HsPS<1h7maZdFo%x84U(*$fp(l2YA}-Zru5Ug%F2IbDDp61A%9It zPVpZA+KzLsQ^!Mh0??AA{fkmkPSfd&csSUlmc0X*H#qz6!Mr_yeHknm2~KVvMC$B2 zx5vE&W9{+nb2xd1RD|qR z6WrI>iX6tkRG6v7vpoE;FucpTp$0S>`k?<6<2 z4Xwknt=ML4J8%_V6-njcYu2o(LBHLM&CD2nJN$-F9vVAxQY8_GKm|pc3{8I|7A>Yb zu6QpaQjaWjTYso)+JJ2UBdJr&i35kUu%rrVkH!e`z1v_3&#?qCJy1HwVMPwYT(XF&i5^KNQE+0^B_YhcB04W4?KBVMF=s z+p@BchZGeaiE;5PIkGTL9JAt))0SeuI1C-@^CZVA zESLLuKn8Bro7PoU93BZnBNYF}8gw}}Oy>eL(TC*cYaZz8bmB-GQcM+XaoDoYmtN~6 zjcAVppcrd(t^x6BZZhYjsOqrw8_FLgz#FNuuia4oMq*0xAs}JiAq@l3d&jgM%gnh1 zmbXcCfNNH}D^m67DWBm5C8wfjm(ilOQ|TqKe&po(ao~(Lk{-w$IN%^~8NSj}-o1Ul zw7k52eMQBM(7!tc`rq3K_EX&0i6eL?q|yxOz?R0wKRQXHui^yui@B8M!a?~Jkm3qj zy%G-01n3X_#?7FQuTM-#;aGdJJN>Jns-tu2J5(k3ZaSE4q<_Kl)a!2epTVCEKOLT% z>Sy37!%2~@JK&FjkEs(Ol=r(%HvS}lKa5-p%Hzh(+SuX}&O{u|YcmP*GKzS(<#Q39 z=wgRsM5ZHt#8ZaBL^ue+mWVif#rK_~;U&{$`uxW<2#)g7EEC^JDWA;y=NdCn<$w_< z{&=xf-ax=WSP_CLFrVxHcf&~q=9Bo}cnic^U~eq|Vn3&~vFSsM)5EB{#fd%?iiJV9 z4h{>IUw~Vpd~f;m#WRSJJ&ZXenS>Y#GAE9k2q<&csY3{}%54{3dhv;USx0hAW)93A zhdJ{JF6Pu>w|g$MS3|M6fOs8RW+XUKq6b*<>$hOiA5H=-*i+W7HD{ZUh!@-ivNV%z z#ic{%#`ul0nv3cHl)U2a6V8p(TU@?9N1i>!D2cOPvfqK761qMV878ui^X}c zSgbm=Kufh0G#G);>q(rtN6q0&57iILYjRdYwD=Wp^U2W7i-QgO_!GxzUQUj%P>u!U z(skwK{|!rFV<1r)b->s@0ftgkr>Y&uwTtzN27$_437(_D$@5txt4?l~3ogC*A`Fnn zFisS30yK=6BC_w^lJ0C=Ub!<@11?YYs zzI|sziIV~#PFZE;3&{ISaFF@=jj&8k97Vrd+uAVL^*>T1(n%W88pi~U2`nX1FW9wc zvvZ-yl)6(R?#SJB!6lcR2DbkrQ?lwT5`$1EIIa~P^fcNVoty{ZnL)nK|#wMe08ma#n9JRp}zDp?Tcc&F^ ztf|*x&MEHlI%;nNqUwo;q|L^j!U)FDR{%sd$W(#DnFU}MAC1G^2k_0eN1~$92uM;6 zjNFmK;BSN90N>RS+v33G`;Xx{F+B%=CcLxR91;7{UGu^>Z(`uAB2=5H4yagWInNtr+*f#%0w8~bch2<1>Zf^ef&4x0`V5uvjt$-;bai| zpJO9A2G*B_HO4(d^n53byJTdNuzo3;tYYHi^$biKKLTf@;*=9g5T_6$gpN>6k(3L- z6?s=c!GF&U6UGoAxLUyJcLgTA+n{$v1`$NPh>mjX#EEhn^&UD(B!rPbq#lF`{brn& z8%|Hm*%lOwgXK!B56U00skoGF7Q$jWN{ZhOh^~78yquko;PDCZIftMm0W~VV$2xcR zu;OBDTH>y|79gPD_j(CN77&cIYgyGwmj^HOpfjGsM)68~XAuPyta3!duP{QO8*gDV z;@R2@qd+_~fMGBU3F%khHhDT4>ed%!pvn};Q{IoYN~2LACyweM(4GcXg3sKE3CEJF z-bp&Wjl+44Dlsq(KLDM#GC_PN&ae3;T8T#7NhJj(lboaxia5Gg>?6{vYik@GY!dhr zbG=K&8e~inz+fl}<{%A>vDX0roCey)OH1|y2Ant=xWVL?;%#ZU!AbmRjKj7dEk&bB z5c!4Lh{}ihx}0JxZK${ji_RUXDJeoMxi@8^p&Zcrl(~PPH|c~TuB>S{;TtL@$1MPV z+eupBG3aw03&=U3tsbz0469$(=irb5fsEP@2poQ5{b8g#0)6`I!CtgUJ;oDk_f~Xcg&~5SzxL8 z01GHn@_ahy5%JL?udIWnT(H~%B0K8RsSJFEB+q&C#uF!WBpbhow?MoFBpP)nEi8IL z)AV;i^bh4Gokln8I_gXrgvA%}cDR!gA<`5bjuwwKNq-2+%QX^UP8{bCfLahE^k5oM zez0ys`G1_G6V5mUNHL*a1u+o?oRVTLc9I`NfWe8PE4t)=%t`!^#_?+puq+@_J603B z=~FBkC+6rv<$4gQzqcDd+~Kh)oQKLyp}sthAE%qbU3K-ex(<5K_~*pJiOyt&!d=-u zPXCa?Loz|!zXt+h7>rse;Ouc!B639%Si!XL%Fx2Xi?~sAlu$p21n!!ly3zg~_Oo<;8Le`Vn@jnoW*(o8bjkAE#j`Uu=HrKM$js=?1ZbDbOhFBKMeG= zdQ|l|w8=*iuSAy&*Or&R4;_Wm-7dE(44rg&AIz<{-An8526Pf$J96a6>|o+Vdl+(R zgQkVCbNQUUG%T{Wb^@(;J3gg*(f~?|L$SRD3gQ{1Vn7qC*c=~EhJ+xZge3^or$ANE zE&PU@96WPefn!wnxZH-SDyM>`niH0S+DMc}o)p>jJ&bj5{)xHRMr<=E*HT!2$H9km zQLJ!DlBS}kp3=r;U{qX!K$;r3@p86gDf=Nr@sZZeV@R!4nhG(n22hTQfS&=+@)DpVQ0a`f7!jkzjUAp#*8NJBhr|yyN2bGP!J}#c zF5-y=9`ZdW00N3;SQ9W5H(iJ)5yuKEe$Awz6YMBlEhXQhnSLM*#z&$eeg1ajWya&* zKoF_EYTI!QJ2$W4v+RPAMhF4GJX+nf&HrmC>{Y$ zsxwOp3;zcTvtMB|c~)IRgS2VuR@KQSof{0EazWot4fNwh#Rkag(Yo5&OE7B;!D*eN z%^lOBcns=@=}fsY+K5xDp8SP{?0;jr`#nz^9dn|92-*cPLA2R%C50v5@0@-luf^ui zFixp&sCNuQfAoQDmN>{RTYTm`gf%Wr*?yn%@C>4-(p#{wdUkJ=tUJj+n&JrXLDSv^x9K1rWI+Mr`@81z`HGqM}m0C7P7n$F zVJokWBt>vTT|Dp6Y7y0tlE!`ilJ!hWlmP`0;3^9PXO@RAjicYXapF1V)A-+b3+z8F zz*9&V9k?7IyIexEO4TLP~|db-w93> z9-l)ml>OMDiD5|o+X1p^^#ID9IOyjhC`D z4bW}mHgzoe&dq;^==;SN|1uiu8p$#le*H!qovXFxM$~rJt?KqtC#381sTfE@gHeR=s` zBU<<%GRw%$z8ih@6|8>yob(YbjpJ|%DiMtHsmddWETcXM56!5z77}OpZXt2j$A;?& zNY30xtGV!wpkOULK|z9pM7?Lj{{j9q_<`{2;h%x$L^>V*82AI>Nr)c+PcA$HoLsmF zQc}!r0em?;LCm-dv_Nit`jarPUd3Btc;qG&*Vu)N#fb=EhwF45xMUo25yOK37|37z z4?ORPY+m3w7!^yg;ry6@r4=@SN{V<4Xr{@q#W$%r2ON{0 zeLxw@ZiW|`i%5YiL{{LzzvsglagjE&HRFH#QwwmxB2htWrzJ&2SC$kM{TbreZIvmrSC2%H3T7FGg`JQ}(`I44Jq+g!2n zB1{JdLY&nRwT6uoMc*`3^(zn-w?|FIdMHRrdfL^P{VK6J)l|iP)j5Ce0%sy7%p-;u z6i@9TyO6(z_|OsPw^QpI8Xax^_-#Uenua#mm8>dP;J(wM74qVSEJ1LG{S{?>XFyga zbyqn`cUN5<>8nA8rp37=(GXPGut}2q*Ko*qq{ zXWf-KT+i92Ee(xmQ{#vo)m4tUP077XZc`AQZ)Rp^U%WqyzO%0c$El%T1;AW(vN_KH zmnpe%@QwZ<5*C}Cd|v?LUdPVhSy(MLoXm0L1cxCNtWDAsf*DTA;4F@IGdO{@23P>L z>)a4c1Ry~=Vwt3>+^2q!J^=+pvpr;mm3dS`~({LKEO*d}9j$NgQuU$tA=-{^YqrbK$ zq$WpYe+i>HhI*-rlL!E(cB?4DlZzEDNo#NaiK7IfEE4u52Ea(p6+eN>2udgSsUzS> zB%54`H+n4N%Zha=~~;Id0c0Tk$#*a>9p z3sC?6n?C)Fg_yi_lr1LRPaas@`pCIs{ssu}PNH@g-~s@&fIZYr0F|BonDxXb#dji{ zz!mX;4$tT;AW+=kkCiUFkpKLKd`N<-R6~CROolN)AbubM?>aiWNWQqnRoMvPuX$EJV5Dx%FYm7bA&Yr;B{+$`zkJp5CR?4-j{;*{$~f?v4LWKH^v#lc!%^)7zzc3ZtX-es zu;;8;nqM7NIEYC8wsrynNltVflj|fqGs_P;;L{5(x%f_iyFujGH$jqJ%JnUp{*#Q< zG&c?jY!{*t8+6R^idUB|L*yB@6R;Tp+~%WU@NXKxNa|O%85?@PGBaT~R?!Sq`AOOO z^-;?VL#y^Yz(&7J#y9A)ODB@^oZ`Q#DzjfGnHW=!+;%BR$uLSU-@k7C`pvOWc<3tp z9P{ym&?k9=Y4O~?DiZ1{gh4|EI(#q1Vbe{KGWtgYbDFpPO62YA00wj(jH0+O^_@p6 zo=@lR#I5>BqS`Z5QbhTd}PDX@E-bz;>{daPuPeWbwRT-GI?mmF?KsRBVUb% z0&>$#g6@y-5AUNuWFP(r{_A~&3G1slb5>!~zvf7QXJ-M_8o^V`)!(d=%FD~89Xqy< zuCK43)Y8)O>)-wEmiE!3$5bdT^^JAkR{p)Mt-d0x1q(ys$qBAUTDUU$C;nZ5i$@}5Mi;9jU;Tvq;y$MGm75Z|1 zKry1-7W;xvk*DVPK1uN{h*tei`>M^_wQIMO78c(Ou>Vnt`8ff6u$fc_<>chIx3*40 zUN7z|zyA6T->*EZsQ5_?;Ip9L4BxVS&L*}ERZ<8r5|TQK$8+-Xwae@JYnugAuVID7 z4&Tox&PE#Hd0r+|)Ej8tgnz~&s(k^x!4jH93Z=0T!kaz{^`p2->*QZ^FdBng0 zneBd|Rc!zfsG#u#*5s&?AlX}+gVE+OMOCLzFJW_wy~7654B$XlAvrU3TuDi8tQ`h| zwj~`~2g@jb-do#5>$8jj18%RYuDKvPD>G;FwpPVSISFP0kW8+xt)&F&TqkMRjbr$H z|A#n9&B1B%(Qf>e0vA9=|bBHL^^`1JO({=S5fVYU2afoTmj5G28SahP>WR- z>EeeXKLY+x)j=TV6>vt-6g%WB_T@_b%{%^5pMv#dKH) z>(@*(FbGniRhxkLRq$~aZ-J1s05@n(H)$a$K$sFCQJoI)y0p|ZhyW`%Kal5wE%TbM zD=Kyb##5MbcAhM>OkILGM5q20C#eKR=WAPJV<^D$$)8hWtM*Ihj_(lbb`s){W1O?MY8s-F)jJn-L}Dk^V) z4$MMuo!I7RZh{>SQk`MK-5yMOpsGfTD5kS zeO6;98AoXx7!tW1bLuZBLPn6+sUvL%wIK=&1m(3zjgyeh%TCkkFY(%%+oqx}Spo4$ zTh-#`9Gg|@{#)GF`nZBEI7oOHU{j`(Q?Uc$ z+0D%@ir*$@6YB&i1L`r>U=r^hyLIe~@aY4yUazaE-I1P_nhikG+N%d$l&0nus`62r zb^ZH@G&|^seH@T9E)3alYxz$AKF&^B`lt>((1D7kyKsp45XX}f9mzDHCAeE<^(<5) zB7h{$Auh(@3njR0)96$e1vWbugvIXYqh`02Ee<1=V*OkOwHl*AlMpYm^c(xlAGa8F<+Dx5XR`B0jZ;Hl?vz@R~c1`au}AoakZL(~x? zM|sDNJ-7+`ilcF2la`yC`TV3wlOT=|8aHRohpDPsn+xWl9B`QDp_OugU*(~ZAVO~N znxf=6z}#|QnEO%=2q=h1O45zL{;gCC5rfY@`&9ZI{@XQc@anr(o2#p;2Et#AH0U?L zzXDJ8R3iBys|V5wsY^^uc);7*ehp3oJ^X;^_#?nbG=O;42?(^{NvjZM4BYWk6E}en zyhc36CoyWJ&3O1M72S|wX~_f(#j`-(3@Zx2Uo*jubopE)i(g+`>-f#9KL2>g=p&DH zB_tn`H{#GGJHB0d1Pfy3DYD-qU+B}j?Co17J~ZX-`Cl{7xQn+yyajf*0LKkQ6{(jh zDKUuykc0Uh0Fw7dgWiwbCH?Ld#7>gyt831I_^M);`+})a=MxA!100}_tg47UQS|TD zc<)bERj%3&VCH^sJKRb!dk}tm57^C9>K3IaPg-*7d1%vLMS_zFAKIl9v-f(vYB$B| z_Axnl@x4NKr2OW(%8Jjr4H)GwXt^0;ShwIie1Tk&b~o%?Hv#yNQ)LSD$_|FEsjX37 z<&YL{SV7T!(D(K;f;&A~d+aV_8b!*7;&p6#wnE-=QZ%Zh=&tPHcn;3U|AYR%KVY+K zKvY{$cfHA{rgoPEKnaTNrKg|^aOtY@@{PSoFV^GAhJFDL5n%n@_Pj^S?~pojLk#YZ zvCl-r>ejLIS!tcu!;N>Ooli|OD{|EfX@KNV3e{$K@f1E#hmZZy9T7@%VKMM!!1SSSR zL;esi6cj4{2=Ee(7k~>*JQ0ZrU@-72q7vc|k-#OX=?BYh^<&7F9?Nh{P`nnGR-P1^ zFw&C%O`z<9U zS#U=JNq!`p<%~qy7hHVtyr-XkCpnP+y-=QKMp$M zPI9N&HR3~VMI5kEk2L|w--oT0>+bRRE`G+d=?81KqS>DQANuf8a0cyGIs_fMcFVREX3s10Z6t1J#K`~AN~SS zmImewyy=CxFH%M%g8(Dg>T&==36K$$1i-jDlL_tSsMqkRi=+ugwF8f;0xU&?IIDqC zeWWPHPRyG-HJ@I!@|&-GUoKhdTC?UG&(^J5)1h~5Fv1Oi9|{jHJ1HNY8}sg56JC&D z-5mJsdc6_P873y2Y`E_;6yN0l@^I*ima~XYT;r~j<2pl*M-%W9K%{^OJA;vd7=b{% zvy$wb7N23!F9Cku1Oqpz%opIKNPqWwhGzokG=WGnm))soe&6Ww&)@&3=!la}R9x;6 zL&u-^@uuY;9ZNt`I0PjaDQdxJc#bSiefy@#-<$I1_umbKivPr0V83bsig-}`0UHB~ z7Foo>7L1b*Bt`l!^wYc?ON$>n4{OO8&_Ae=Ft*#7oO>XiPK%g}*2jR(r=IEuLba;2 zplB|Xo}W{@tJb#FY7$P^{K2@5>jLC8tEc>W^Ik*0A-+K+=T6cAw+)CMdCP^DT=>q6 z88e(D5jAm8etlSB;iK4K+?xQ=VQ`GvDjSSfal@`7{`rWwUjUK-IBvk$pMmd3qOlBI z+iEigCO9$}V%p#YuahPBN$b|HE3=(QKZ*cOmdT*Wo+Wqn_NtvB0VA#EA=gKBZH)>G z#7A$c*l;(V-(*ZsP5XAYa*yaRd~nC8tow-Amw5uLyP7!kOOc^73`iKfM+b zbHk*1P=Pib`h^lE1EBBjP()p7GkU~>V~wQoLRFEJc@fV%gc&Bv^2|0vx)XGe%Zafl z0A5@(!KiJJ3omSdBo3ckn=2~M!*?Q+w6yC0lmk4Lzr+E|BD-<>GW^o=@_K;AA7Nab zjB>c_mWAUAfMjw*ZS4faU0^qUUxg=kB=%LBTb6#?S7q2&zTfGH9ox1%xOCa_>%UyO zG_kUxLTc=yEE`0k=d#PM{L92iC$VCrGIzm~gK;}Uma^$P6geR!DIq*xs(E=U$+_^FxM2

    $Q(W7;=U{;gN_LkLaOrg5i-R5c#nn+;fPW@VoMZmj)Lfev%Jay9cM6jN03lgU|+kY~a>b9Xrjt9FsPB|CD zaGRQuo_^xEapO{G!FykM_2wP2ItAj|It~SG1IZLGL*GI3hw(VUyxx=}TH?wc&y%o5 zbO!)(PnAtTl68PsBRA}0MvWSkzk2oRZP8M%aFhe!?N7lO`4p=z_y)Ri+W-h9I^o%X0U+pA z%aFagd@QQ{QmCu0wYE>ee!;~94!cg!R6@s{Om$r`=b$QW35*66qu`3&!nb-&aa7P8 zMHPV z)j8YAxZ)b^af5Y9l4DZuFmRDCz%e+5dVj%$WviKKZJQ3!$SAi!+(h{wa_iDKD3F{- zS67s8h{oc62LQb0nrjFeCUjR=o}NARgV2VUqu`3++_!8`sJ;2>JT1QppFMDy@MIP=#65Cuqx z=fwXkCqna+3`8^2V?6jkGJK|M`Z0>jRf$byp@@rTVp7F#r{oX?Y4Hi-?oN{&vc6$! z7QgYqh-0P#xmF4ak2-BZ`Qq14*d5mdI0MB%T#^34@3kzr82LRV@{Iq9x4?eX0z7rY zqQMi#0XP-7y-kyT3EfVMFftZFEaDSgQI=2~BvzVa*^*^PU|vY)X|EGUy_s;j>UO{3 zBz`o;fkJIs+};Sndnf?O-P2)^h2=K_E3WRA#>SHnB`hKr;HF5IeA*<`yEw7lNzf9{_Z8 zr(ArL#32~zR@F9uk*C0NC)Mlhw+{3FU`;dL0#25&ZO3-i2`5XijR!7C!J&_h+A{#p z$MtvcymPzo0BzalUreGnUgzh*UPGNmJD3CBi-dmPV9OM2IEv^4gkF}Im`F~#ZufuP zr5DhN3|TIN@k0(s8nQka3=JB90NZ_oIh%VFe~Og;W_RU}=I7x1S%UIk;6#rk4ivd> zgOrJJNHo{cDjR4%vgv(BX%g#Mtp3WX{G6gGXHxAF*MGYX*6U@&9RYeQCtU4;wPZ5O z1zOK;@uXxEJO&$?@|x|)Q4j%R;tN6{1nMY#WXEw_QGJA}s0Z7L*Oy_1@gZ6EC95(e zAfnRk?Zjn0^!Ds?L7@oJBai(3R)CRfP%D1PoIJjSr?Hqv;C~BG@uU;r3*l)I@iTa` zVLt$WJ^Xa|qv6RJiBmZd?s)v32mc%RhvAtX)7>w@NYq=pV)j?dFlgQ;=tt0wpkHU; zk4T_+9(Zp8l!(Z1<`4c7Gz`QGq=8#A9Scbm&&@pe4lg~>EmXy{O=srA>i}*k9 z83kTg?^iGSUlEW^0iYak@bM$vp2TKmDpCymGOU#ce+@~y^NShN%pI-BA^s=c0{ce` zP)AV3rj6?6ZQCTAs?vx_7WC$xrW?||(Cxa=@M{$LeQZQY3AqrW-a#_pdn}~g#onqI zfGvowOG^j_S{6`EuH%NC1gtJ;1SI>V8&+I=CP0Iy02&xh!ALZ(5Yq-I>4Vbs zsQ@EC@0X2bqf(%eiXgj}xpn6b)vjR+8`+4UgyoPZ5={h7yf559y)1g%PM-X3@NMEOLI9F>q6QTfyN~ao zUu<)lG4YfX0L8MgXnAFH8E^=y;s~Ux>K6=4bs=^gnMknqKu}Q{Z#QnF!;@1D9CuI` zEYFX0-r>kkal7BaSn^p1w084oZ8Lcf6rVxnt}r=yG7T3xv04wHAA*!p1;El(VJci$ zlT|z(b6h!~=U6Z?u4E}DfPT_d;Ev?J;rSJ1Wl@W#eTPGm4GqBYCJdw>VKVFXG^(zy zj_-=$2}bgR+|-TVisyd#h`2d(-pw$K#xp@!Ou!;kiWAE`OvdE;BoJ0*D8_O{SKmlZ zcKvtSv}rUC7xm7ixbD#m|L%>9K=D`~)Cm8DthA_wxDn3j%bC4UYvmLmnEiPsd;N892b!Apc@9)WyBS#WEW%kcY)zWM02_YRtL#>se^ zTXMpW=YRYDa}%xLOhD+T#G)x2?r}A^uBGV)+}~k7jQ@?d!2Z($CIRWiV&{d*AmPBM zEJR?b3n&f#a>H+2SyE8A0o+4RU;>{Vt;)Z#N1@oS=0xn*g$0zvDO#Z4a7XU`id{Hp z^Iu0owXxZrN@H&P+SHlG?FGj*H`ExvQwZ|FlQ`@*Zg@e-srWXZ1CcH}6)6U{fiPf$192KP z_~E*WiUs|qfgD!^v=%s=w(Z=hk`W*hj^KGsZLKnJK=#CEX3oAHdCrJLzI~ks4$w{} zDe44}6yn;b08~4~tf;B3NMFdhta{uYL4Vlo56~S&GWtcW>u$qQ4kG9uL6o`~VUC199R3XWYvAvLe;)p0_>J%p10#_a)}8uE zVDG;TWZV@XE&zTRqwo~Qag4#ZK3LMFi@~MvLUXHk$MbVuoc+SQc~j=h=^l7xntkoA zd}cZIIeseOLJ&;c^qCGfs3h-&$HR_yA{_7G1OKczyTb@#*9;@lFvDBn_yR|sxyXdZ zh&pgjD=!hwDvAieKzwSYD@$uXe|ID{w*vUh8FuJ+Pf|)PgNibc`V$2VlpilxRzo&! z`gq1E$T#ldEf8-3iU*Jog(XyJ!=_CNm1F}fD!|zML%VKNfarO9cu`SQi4K^YHO658 zZfp}f>~V`504wMx{p|}Ff5iBOs8b6FI!9Fl=OU-jg5kr5Q@SAJ4*Zu#dcEy3!B4x6 z;xCxPL9QzO4onhaQ?k4i#w+YM3>q@Pahmn)Q_qgD+pdugk7XHq4JoM2>#(At6EsPC zi5vq?!ANd!^7Ha^f+6^Tro#f{JCUw)H0D=bTPtnZzFoBofcLg8^^J{Eb4v@h0s8Ms z3ri>VHoaJgkIGJlN-7?p*)A?bO?6Dl%ecw49XU!OxMg`6i{kOP9hJxg%#DHLN=iaX zyDOGhsRPw70ktoZTaDcVGO7T;x3$i3zd+3zT;j52%L*NJY=j~O07*&pc=`n($!5bA z_iZo~0I1(9@7F$Cmh=~mjUkP&(Dp424aZ^5b5sATT}K06aEhwLE+84G^OrcZ2#U_> z+D2fuAkyLTx3m;>eHU%_sVT{%S-*ktp(W8)w$A104v=J+JK%d_-$+H6#~3(T+}UTH z^~%}P&n&@EnFJNUvp`huwGJ5MsThnG>YDVHX1uiJ`8h8=`2346hWvCdWbLBLe-a=b zx1w<7HEvKrI)MX>0t!Dze}&jRtpkH84cG(`f2RER*IV3WZAW=hP= zy6Jq)FoE)zKX_^ALsq^!XOWqlffs_!I=VBSHvZp~w;(=ph%w3INGt$W>vtUJP#m$xuq)FGX#E(|ycu z6LMv?o2P@rP1d4b%m?c@SK}VpK!+6!8wD}V*8$?Vo$5CfU}SC{tdM&scyT_!$e7eg zW&48&A_C{n0H;h4q0Jzq!Kkm)8jC?k>=JNiO|^`E1)1!Bj(W_xyzYOuhJ3ga9ffQN z0O{lkr&_9*bskyX!NQYM*@?5%33v_TKqv9Li-U$e93P=Z0Kl#b9B?$yTD9G{Q59a6 zwnN2>BYMFC*kMLFEC-mOYA4ZGlDv7%nl&|{O0%!iak-RFSq7(bF}`^c8e_sy4~{FQ zi#gAB@(F=Mi!@1!YJe+EzOH6DmJ{fa^-Rkwler03G-me z*_$v|&iv~2$1fVU3VaKrMWI3cn7AJ5fD4#3H6I5uAyI18(*h;IU)>SM0uC9RQ9Hu( zvlS2V*Gw8F&y2&d0o#rTgP5mq;vMo}8^VeB0%Sy}z&o=*hE)FLN8?7HbY_Fk5TGqDK#$EYl`8(xj^$UJc)+;}{?RyWNc;~yZO-P8=iT{ao3s~Q`^?gP<|48IV z17o#yyJ)Rh9e@oejGbK8s!QjV78c(*L^|++`IYngY~I2)R-Ed#xBDD51$A;^s;OAH za^)^Z2}F4$h)yiVnlp)dBndLxaXdArgH-jQ`1!B63DKYDX3ZLgq!PV>JnSZ$XzJtH z1#UvFoBhvGoXjw;OGvP<2t6~dQeWT0dO9w!_>>yRhGRF&FvEHW#fF3my@f5`&!A+(zMOnwmRw;){`7MJF2VH z0|w>iN`C)rtff=YUnap9?G0hGe4~nr3LtHe>SW&Ih7aumYqQF?HnWYY;4QK&w8UU- za#SwtxkNmeP~o2N3rC6UEs`vaYxhDAVqmYaw;bl9icAMUNZ~pL!}-lhW*oJlnL!7N(TCUa9E4j%fz8F zfGNnbTYaUMNrgRnPaoo+Py&ByYULd~ zaRvzl;-T|8&37y^|LtxN7{3$G&Ei^d%tr{-DFG3mH5Sjz6mSUz1jbi*I!L4-07qV# z;^HRoX~Mdq-XBN z>*?@u7jFSuEs!@bht@)=Xx&ykr-3;h#W=t>4P#JZGsg`bJvZt+O|yQT5Z3!D7Z*l~ zaYM{b&d$oh`AgE>o21ICMwOIITU}Q6^}gbU>m8b49MU$ZXT?b#5KltK$uwxzKb_E% zWXL=@w`+ls{S7+O4>6$ep>7MrnG{)=m zIie%Tb&Y!3)Nji?1AxfTFC#%V`vk0G{qXSaCq;?~lF!j+&G zxaQmbhApn(dR-ZX_=4NAbNaox#(w1x9KiI10qr#l&@d$ISMT*@3_V00>akd=7WQDL z=+?LbRuZ-2iuD0*$DbO^?@_;>>gi)7E5==qKRNSy%<2PJt1gbn@<4pp=QK37*5dl= zF7Nla#p0sOo%?3C?rO_}F_S@tF6T8jH4fgmVZ-np+qV~j@X2qes~cEfU!Oj5)TqRo zni{#brbcY)_ef|TH&{${PeMXF7NS;Fbx~Zq8OAM}aeCJTk*U`9b}wOV)|*_NS?~lP ziEwu}S@1czM(8huCy`EZ_WR)@>Pm~Vo-Zq}y#wu7$N@x$e!+M$iy;OTmo8R?&pO@( z0^pH#CE|<72#*8^rtE&k6wHv;GXy;roefY@J{US4JK>ec1phk{Hj85Z6=g6Bv0>%M zhmAeuC$*|OVL)o`&zr)-I^F_3 zwE(S(VT0CyIbdZ^VSD=?X9ckS$oWKu0zHFN6(kv?41D=1dr8|MC;>tM9NRvKOSB1* z38x#z&gqkL4m#W<>P^B%bZEO@`mCg|yG1$?i5I-kyfk)>v~ay zsE%4zR<;R>&>H|kq&q1n$GzL*2|1ALq%8Z213HRln0pnSqj7{Um;66_*8yNfasFpt zxylueD}dlNst6h@STTt)F|kojtXQMQ6is4cqDB&9Y>6>38oQt<9lJ69V?<*I1W}}3 zp&s0E$MyQ&?*I3j-8~MtgSYeE-Mu?@U~YG3=KE&m+qYAG^UXI)smXX$*{)rhihVqGhC%11?lU~(9 z1799gg;Q1U9-fo)-iDHrzBeAz#aPP6);@P4E2%960>AM0YCw{8M%!P5PoHxl$Ij9i z!U64KsAJ+z2u6+^nTx0dmG{CiAlex}I;39zULb>N)Wz{wielxWI4_uUM=fx|#S}O! zgNeTIu2`2nPSd``cwrm@9KSyRvOrD08<50l4iN<;VS)qN8o~T#g#{Q^ECzeg(N^?@ zcO@5H(dM;jJnngM2fkx_2h0K=mnW(iA6$!|LK#v}H$&1R;&T<68}4Qtzp2Xwd5G8?c2Jsl#y8 zxZ8%UdIK}F;$iCLjbXXDKZRYkN5Wv~m=xdj2n?ni55Sm_H8_QpKE23s#=V39tCd@! z`Oy^P^ULK+r^P0iofvNnoiJfSJXhhJ(sz~~-^?(q{9+f19^%+w4=27i^bJt0(>Cr2 z?SprQwjSRv!Ql2?Wk?m38dwQN1||f__zgsCM@dMThnUE+A;L_D83P80ehG=}@9#w0MAJa7QWbSJ zEe*-0X;)0!JY&X;-Y$wNRX~!z6POVRdOY5alvkAG5^M<)$iWbbx-rh2Tl5HpD?8GrW`g0wo$vAh7N{5rs&2o9fmZ&UX7=^+s+kkJ%t74uf(29*haCqZ6 z@+Lxmw174m_>4~l=$GSjO~cA_?>12(MTzf}DY zM-?XPTd+1?gbrvuJSU_C8l{J3Q4O>w1$~4aw_v|#`9ZjTHw69>+m;YQUTsOdp0sCz zM#b+4^vY&R#+cwB8a@;JLl`?QI$q6i6QCq?X+nRDglFG|U#p$ohAibXJ3fmK6C%|} zkH>fqyC_ZvmF38QZcjdlH$!B3_QY>lv;4%PzISPvFE&0iF*S2Sa^~UdYsxl{U^&b( zn#F0$(&TyCJGXuFoNwLz-`6Bl_m3-qULpbO@Di@L(C??IIePTyL4n5RiCDUx7$28# zHcqY^l94t*$LXe?s+wvgW7D>1xK$ko-)LDG8CpVI{6F$@az|Wt#q^soVUo+`cDk;+ zYJ=OhG&szp?L$89)+E*h#JEstnCfEda?(FSxWo2d5dWaYRh$jEAU@)KrB=31| zMxpZ>^&SdzSL^WJsO9KbPLQo#QH~$JwjZMDikXvH9L*@(w_N$Txj!*<(;F+ZcpuvjkzY! z)O;*7=5wSyNktPT)CQqF$-!9zht_IUPa}`X?eppN_l7pSCD0nud}!agqAh?jzotF^ z?5y|mvvc>2?p4pwq8qHbHJBBZV3}sbN37C@o=;vXCLyrcXB9*LJ>JVNBUYxKV5Es5 zoU5s5atr*jn9++a7V*$n_@p?;Ip|B5X8B|LcRK>yqdzXD(@a>hiIFMbmvRGmyX` zZ}4&mHSX&qxQ7z+%2*cObhwk@4BbOVe)Jqz>`4luXtXLYA=-krwFk~F^mxtz< z8j)?~LLylPa?nKT=dV5U?-Qoncoz6;Bffn})v_09rqnE#ovwXJQ4FC4#XZK18E z$a4~a=^qys7yo15{Ce-RahQE5i_ID3n<8;;mJYVg98ah>%}H?1()%431)TObcC8KAM}r}ph`kq^Z=|mF*a6^vX{2~Y z3m?s6Y$W){1zYSCbzCgGcf;$U0#f80fHb+y*dC8K;mpMW_62D3ICiD))lS;>R$Qp# zybYC)+V5(Ql7P3kH@u{fCqjO&=YbZ(_z_-!)RM$itXsEMeERXnV%hSgVpn;&UEhwh zC`%CeFZlQ2KZ1{ji6%$g$G{VSJO};+_yY~HsSYp?{$Y56s6FN~e94bi1nX|aJ4lrX zBTC%{0PaEctS};a{7>CT6J%r(-nFl+Btuf7hIHiD7G?&32vnNUAG`A2$Ng#Mvjtuv zF@F;#C@ERUXS-a8lKe_Hesxl7b3>^=HerxIG3A8*hmZWSYRmeuq%h03Cx4R*pN%m~ zTK{($Zf2Qb?q62|eMT_hwbs=s4|Z560zGtw-%ribKI_o3ZHH!nbh>2RyB3eTvd(KK%N5Y*~$`c1Zh9)>4s?Ri;SwqeF)kA@_+n2*t0 z1M_LqYDalFqS#zHa|F;*lao0PcxFR!(NFBLfio^{YRWA=ZGcFIb|gk=3x{agmn~Vn zq^UE>tS>J90%kjI#364_5x|74_Rf+XG@P3IuuG|9oA_Ub=M4W~L&=7f2PKN~1lTp; znf>gnbk%l`n!>8u!ci_7$Hg;&FRi0FpkrPnKsz&7hXoN_9ru%5$lbPMhf=*~+{jOwG^q@T(LZr0jCkXX*ThXfy;1!B&O5|g zZ@t+bjBH=yO?db%{1NcE@IQi|3%?mYV(yh!UP+p_V9~vE=P#NIKY#xGMdxBTgngAgJ+D#7Z zLB0$}Je>_a^2&I8M6$BH+D_h@L# zqA9`a{SAKiX2)(vy4R2-z_x-XlS?pC6AxW_<@5`elynY8+Du`m^J(-YFQg`?7zy#} zcoD~YILOF@Ho^=0-V1ozx}8UFy)K+JD+7z+KGk-wx7NEW%8lO#JF|#s4B;ct+}b`WE_$+b;ZgRubg+mr1yNjnESzc9{dXU zI`~5P#qhVop9!A~zptY{^%D3y;R(8B!c#NvCHPKPpb^*YE+*FZH`scqCwON9fhL$o z6puX*o~(cJ0e=wA2soyjS3_Ew-_-QSwsVYHtC_4cI!`5?YSE zGnL`>TJO!#@JUvd-yyz19bMzVXsx}i>6$nov25bRiM~Kn(?XQ>aI_!Z)-X)!U{M5i z#zjnhFgO!3=wNgZtrCfja`E5?*l?tFBF0^wn>Ut}II;le($j;s=7dVk z2&$LCo+L(m`OS3=kE!s8CYmPQYc8NE_E*tVm!2*QhyL*9%2Wgj175Ic>;jVEiZasg zCEW7tOE2w~8V2X-ea>+_gwC$s?pK{w&^qKY!ztZ?(xW`E@g8tKfH@G;Lolv4d{=;G zYJ1D833*scl?CS$%*|M3{8h2OKl8Z zWo8Y+b_p--BRO^4Zi&-UJ+LMA4Wl)98|Z)Mlt(D(vA;^H-~e|CCHa17c;X$JiK_h~ z`dR>#jaVht7afjs2Q@n^3?(`z?3eF88coTcg6x3yX!m>6hC^6Rq=@jI3k zKMcpf4mQX_f1|vB7A^JYi{-%Zf1v8ZW?z&?U#*iIJ75vIuhtLE%P#b*N@F!H zJHx4IB6K29Oyvi;Kz}E~=|jO#W?^l54PN31kVM5o-HLSv(r$-;7oOX5H^X!LGl4vU zlpN3VE8`Yi?nC6qMvX;U=> zBz9RL#LS06F~kULIIKZ~X=`v{ zy499OY`X~f$t_)W8|}0}W;?C*_+p*Yayw1$H4OR9sGdg4gCEdMM(y@F6KK^cM&#s7 zM_F&IsjU_Ck#Mv@7#A07B*rKFept@14>y)rYf&cy?R? zm!QWafL*M_cKR$J$t|FK=B z5E-~a`%&Pq+WoS@JHn(%YsS0)TVFW+0@7v}ad>eFcsf|UIC9CvR2$p5jgtV%UDpDv za_;cLOXny|TGmks&I31QS^v+sPb}$Se~> z8870YCAt2TOK-_$NcQ+2q?fJ#OxJYSps~~PC&sq}4Iy-Xr>G7q8 z1#HQN$52xmJXqPHDIV<^7v|v1Hrc!n{jWUHR5N8AI#uT>(I!i9vwhp$dFuUR*zTjj zw(7Khd|wIC`t@r8LTB}l&TGYf3@&0ovhxxgn1TQ~dgyp8cRb$I+^p}~UCA+D#9aBV zU0Up*!NTVk^TyKzed-p3dvCM1Zgj0|k?*S7jR+F1Z2*M(cf4JCy$q4|1B zN>cJw(0bMC8`Mr!=I?mtu5!=NL4#w2r*+|k2@{U%Ew-xK%h2#*aR5Jd>$_|p<=*$8 z1PH)<{@GHW`(AqRC3I8P*1WV2FG6cg=J-fOpmMx`Lxk6H%uuhwVP6`K_1y1jU#axP zXW}kx+H_H0L8ls$+>TWMKOCM1#GD3C?LulYM%?woDT4j`r-}Y){mm;@bO#JbZvz|& zJjRAVdg{c99IS$}a)}vvU=ZTGlo36(}J7>eT<<5Q6cf~VKKfkxtFs>H5;kU3tnT^%J zn^Q#mA{l2-eC4!}PcjTm6K>i?OL{YaQT#!ZUbFVTxwP8LCZBP?SLGYN@T znhhxb%5s3-=E1LnDVXQ(ZH1>b)K(THB&Fqfyxw6$#-8@!wl9{POqI>H^0(zr?kEcm z#nA$dzs1AP8R9xu0*8_W))f^k%g@feH6bqk&p0GP)Yds){NmP0`dFOejY;I+x?mD% z0$cK1cm8&))+mzZKAab#TPS>}V}MPel4E}b!iZ_wp;eaeOh=y9G$_4UtDghl z;oFhY{D@pBqk|K+L}T3L*&gQ+Cw^629I-KF#^8ZJDcfE;K660&sI4uf%CECmp!?xU)x=nUb-tn5 z*+T~o9GI7!lyZ2#q@*Fx&gst+V_^>mr`h|3KgKT-p~;wl2{!CH!Y9ZpP6(<7IJE_Q zHX9yeGjOo2&_XOVJX$G6$+ba4{~zK)bA~=#Z`IzNyI}E+Aeh+^5gH7RygApP-vPf8 zelh&-9(eF?=WSTOZsAv7uJUc(v=Z5S?6?t>95`r@7&rE)V@8Y`wE#2|;X4FA z5gsvj$!MApP#reR9?ED0mli7@;&J}k)dZ=ds4%U_k_nTUHo{_>C=CcM^=Nr@$Dg{U%elI zD!Gq&+7ls}54nTRs@Dyl%=q4e<7fP9E9qS4O5jkEz-3oX|LgP5&blHkwf|^r!Nv|! zewof3v?O7ZEH15V$BwH(`44B*2TdIwAj$SZi>UqKgBG|)5*TJ3?e^7a8Svg|AtsW* zR@m$cwDof;?+r9k)=p`=BR$&`ZWFGo+U>PZ1&?&T%`MH?E?nVB zPa80C<+5eJ2mjk6okw5cht_>lB*93^g^N%laz;!Mo|fg%z>M3rgF|ZP{Rnwc!?L=z zMnL28nh`nq@2oE=oXhZD-a|_#%1TZebIFERM)*KUo2!P@jjUuC-%5u7=c9&H|c3tfB>X9{Iozmq?D;i&lvo-dD-D6V>`Xc%jB<7 zOZH=sOu$t-Hg*mn0{HPHWFY32-ou9v2hC z1)7KglK4*j{C??G9Gf7t9=6e56&AgM#^==ot=K9k+t(;4v8Lm_$+lnZy|D^V{zHxW zwp+4fNgzKv2bE;+q~a)F*(<9McR9`O>MF6jy34acyHyK&l5wJ6LgH_S<>dSir$+y~ z>vMH;uqCj*Wc}LV*?HH-#>Bjkj-5z6xJcFEup-l*WK8UnuuZ%awkiu%d384(j+k-* z$?g_<2>D@7<-V5?z3FfLM7PO^v50O&i#ADjvRW6l^G0dPv|rgtUK zRT6NF)^}Er`SV{pe9pW@cfgk7>Y&lO0f)o<4YQN)V^Z1?7+Hf4;$>__xxw#^Ir74( zlT$7#m^^V>!K7bIn>zWaf`UmO90-hL8H}Jo5KPITOjp!5w0sKSbmYMpEL`}?adYP_ zy8Rcw{B`{3BgPyodHeXo@bDY>weTOq6FlRgJrQ@8Ke2o>TJp!qn2pyS;Le&d$@z~7 zMB39Jrr~}2w2~eZQafzVvq<+c+9cZ?A@O_Pf9k+pzlphq$<+c)PjVn3xdsnWPQl33P)T(Jb)Xi)K74k=0 z%kvjQT~s@04j41b_hz^M2PNnr0p7jhOYeYLlm;Dp?6H0pO&M@EcVaY0Y4y6bSmhNJ zqBYP8i!b6u09ix2HFGyJv0+ouix^xzmYkGCU|3lq{1#|WQaAK?qcyN#%$PBw3iO zb3@v&&FZ))E`ck7E|oyoi2G}=y_R0z*z%w*nlDC4v3>9W<<+#8a6-u=1JhIAJL8Nq z!qR}jS_%)8QFizskJLr$vU&4f`r(BaUNDD@&?sv2UV3Rb3|yaV2>o=SYiG6wTcv%> zV~;yd(3h5$ivPazf8zc3mWY~~kPoOG5BBsA!M_4O6aH=Zh-s6=C3^nYTraM`;4+!u zokZ|>wQtzq%Za{>mFONPp0XQY8Y@^sB6%wtK?vAt3<(Z0k(JEkOnQWyAfy$`Y9 z!-~WOG$xs(lzCbFWU!%rDLk;!zKpZmv^z#%6A&r>n}n7=^7h1!meYu^Yvl60-2X(BFjc^?UfW)y}k z_bhBU=gC$EdMkb0X#uWMqo%TLbzWI4+8FqU1KJ|v03g=|gSzsm-&QI3^7^!EZEdwC zPJDaL9ePm`_6b0e{c&n?vamF+9SegyOQb!?frC;#fxv=^6DOWZ4OGXBTmoeY3>7&y zZ_!Fl%1BQi(_GpD97yd8*(lzo#m#B%@uHhXOT;t3cxbw1DQ)b~a zQX)5IFt?B^$vX*WdvIr#>w3d?)w|wOSb!uBZ^?qD;Zl5PLl0JMP4hee4ai>|EM2Ja z7}6i9HXYcL1LlGK3=w>B?!1M+L9e%a-h##BFga|e#khYCgJ37}l1nca7hQD8`qy87 zZQbjyznWEBQ*%7HhB6Uj15jZ3Qe0=l&xHRPJ}g)8@QU*1t~~l~%_HuU^IO@zgQYAL zzyVqtqhmOMK#3HZ_cDx8R-8;^$D1VPpnxqS5=nZQiA9ezveDGElSI(igokX4nGD|} zMY9Ziwu)ua8HRu9%a3n9{mR=$AWEcqzH1R|1EU1iW$nxq+tUza+#bC;^hq&CLYEjG$&rgxn$rNNPqMn-6M8!VtI0 zD?zE$x45We*dsI?fJ08GtWx$;i(H6^0g}4zfrg}xX4Y({ofk^B?5f1hv+E%W%ecJq zdxL!0jX7UI=%M;{maVj#osnt4-lPWmfx#`M+r28^J8f}eyRZibtc>4UQbOD0?tyik z3B3!#my|4N%FoZA>^1Zi1Jg4SwuWI(681+$d|W&xto?q9ZTSDiHheC?N43*6Eb3U} zTBil|d3_=g{Vy85B8-eVra$Bz7F=G8gb=XZN*O3-{y6l{h>MM_!>e5tiSqPz9+7%4 zT_Fc^2xLSK&F+dqhQ-r>%|1xWMC0SLzHX$`*6wdR`$Qgg`lA;e5O_T+c zWit8Me@*6w*J#mafczqO*SQinq$Ge-D7WLp?Tr{510UL1;*rQ^hLE_VLaCwxDJW}H!4wiD;qIsZY|*42sl*f;_0 zi<|Rv^4~8kDSSEnQo0;Ml|W%(;l^RPxmV$R&QDK+JxOe(S9KM;D`8JEHW=gg&m5DN zchu^lqRpzjx`PfKXA^(1_Sn`*p4g%;lj9qBY-4dTwL)DNAqk|Uq%?+eco^DgZsFQk zxT)ln2o=d?(?t@HeTLD;H6#`KMT*;Qm(DuttY#cW7O6MTgzi2R6~iQ`_6I-s!CsTA1q;6R3$dkNFs3O{)AfEn-IE*?flLwfWME5m zHO{Mg42Jl}qJkr^3o0Gua6Boea0ob7RO}MvW#s@K%djnbd+nCZn`^e0ZVy!L-kktV zqBJCxA}O#7(Z?W;3>~vRvk@ z+`nwmW7|%<`i@b|v;VMf9PMBFMs;gbW6GYq83RIQDlZssXrkbQKb-Z)Q-1gF)r@tW zD}h5v0{9Z%#pEv!--r?r(Ad<()y~|JBS#JGL7(3j@ED$pHxw5w9-fo43GK+=`lqDebnYq@sfb+wvM@#r0^^*8_ULG| zN5x8vTp}e208m>nJUjPcY{&Zv`vqz{syI$qcb1oXhYT8&+#J;BVjKQxQN9gd&O7!V zXNFdD)|C#I`SRthI6N$fZcb^D%X&Amj}0|XH(k$2z#2Qm_e}0xpDU)` z=|PwuP;qRR@Gro_ELG?`p`nIkCs79u=6Dt5arbN=SWIrLf&{_>B&SWD{Q10v3(v*m z^Pd0%53k1n=B7sdz&od z#}>{9w;S{X7>nSG;Y;B8Txi`#NJ|vl_Hmay_4)D_uRZ2+?9hGz@|W+I%^4*n8z9OJ zk20*SJ+gsGq+&cA1_AQ@wQsc8e$g&(`!F-L%}*u}KCd>gHK;Y=DUR_nOXg+qwrOOX zl_7c7R_=f;$(?Hw`e%(s^3?2;CoL)b?~4=d^lkae3Rujbh-C!yJCTN4;a%rS;LwtQ zrhSbs5(m_K1=~5N4huz+B-q?M95)dGB)wQ*3SjW%vCTv{ER>u`7(!|F)?rpXojijD zP}$~6Ek?d~P3X!CSo>C9kI}Jms177s1Ma>*0n}PE*q5nHFxb!G-4Ur*L+$o`Y2Eyc z#SaNaq6&5R&VR7JsOSS0VM9sD-}7^FCiYKBJs)6}MnKD{Gl8r!09k_*VTxiA4!N0t z)0%rNaO<@G(U3l-U&1e&>KabM_W7f+mEPN;Xiu`GwMFd0HvEhM=@Yh=Z2lwac55`$ zucI>Hn{4Dff@5B%j&mG%9WAj7NIDf>Xz82remHM*>LJXqL=vG9m3JhKdF>C6LY6$-?D8m*phV$Q@kn)^^-;gYx# z=uiR@u<9@-#J~%uOnGzJvSq`HiZ=b=&O2`Z=eBKIxP|$Ej<(C5eEe~7_~F?iKY!Q( z(}ogh1CG1O%kGgx;{grvj8yhB$FXNU8N;dZjUf`g3hGnok+#`i{{|oqG5Y$N1wCcDFV)^k=!4 zg;_qkJSHvpO!;ue_m4bz#v5N**|-;10*8bIFjm=UVByv8^CfgIfmIZ9#Vy4v>uYxzE2hzz_1h zsRGH6J+P17XK#+Sqi}!+HI{au!THYmjT`r}eVrsyuc{SQ$6)LB5M_Ik?G8pU0)dAh zs~bA{t-4`dBZ1YcSGPbrb!wowd1Y2cX5zNeQm?88cMZ4U!=5B;`TYvp!T$>l%(uIy z)ZuZ34M{2xpxska#KZAx*c!{+WY=D{XP8HhDXn+V*c)cEbBHdZC4#No^ z({1r?VVk!_KDd8e2}D2wVFQvZ!*RzQ$7A_kf}ae33H(7FH4-2F$HUF{|M|gkug^zY zTKGnw0%=_VTj5?BHLkI!_bW&Cj5?P!d~lrV|RPR07-ecMRra#-kVY%Dp$^NK-q)|6XId(XfN|C-7nlO z%u&-V4M}rg?j$E0NJbaA!wFqar{$@d83P8e!s|S-G2dCge*J#GyOkv?t48MKO+jD& z5kN_gsy#`0Fbc+C$w^7qkI2b?Z+%JO+@7xgp_D(j%EON5^ccT*iS{IMa<+2yaaa+z z;p={%|AjF*W5%v7Ssiu1`#8hcN;My!&v7#><DO>Dy0W)Kp{ z(SPR&_m9DQ!Q8;`&7M7bFk-jvpTZ4^775tn(5k8`jzjwtAPEyMnUu4th9m$<7(xWbdi~v3EG48AwxfJ zY;F=fbV$WiU;fHVEQe2{AvEV>+j83cH8VumLS|hL1 z1a3qa6417yuL}z}!EUW=6u<=}!*~l_O=|BF@dP~_hxRRy%Nxw2RXPIXDh|yO7Xl-y zAqjk9EA2eVc{C^V4x9~0>H+y(edvJHP3lVEfD(u_AjwI{lkh(}V6J;d-hoGYrNO}7 ztjs6BKSrdt2OKj2HWGLQ;7AZMIXQF%5PYlX&y4%#&R=+vN2E+IC^&Pk!T9|7Z)OCw zhP!oRQ32AXNR&(#10Y@7cW$6P(#4$li#8zs#TcJ>YVzdCJ=Q@MJoYgsvGlLJJ7eN+ zcJ8SA7s@qJ78`%qNYR^)l`!w^AoFShit?VysdjP_F%TY;uJPE;vK$&u$1;fWF!`FY zk@>X2ifzJzmgJ7p>nnDwP8=|B3}l~}eOkfN4R1d)kx5NH%uK?U%*}LQhU1lP$HPT- zkb89{(AOmp8wk{Mn4`Qs+T4DDk2!K0lKd25jJ^oKmRwYD`VQ;?Pi;jlS76sVrSC22 z(F%@g4YVrt!M3KIh>3}a-A`L)FuuOl@An^zXUF}q{oZf@9Vh1;3-n2cK4^Fl=qtb$ z$M4PtBz2FcG}?v3{+Swb)Q-gc0;pkMwmbxzj)ld={}`5?OM8-&8k?G7Ptv)vEEhX* zFp38Uqr?V;XE7Ehj_+$*`J(;(r=+FbT3u0baz@&KdkPZ9>-6KB`5Q|fKH~oPmU~I*Ep@|Er0CAv{ z`^E(V#4)jB!v#+E6;0GV2Z9{MVveWsgO#DOoj&7HUC70o9exp4C2cRNkr4H4zrYcqlK4P=LJyFK@4vG5P^CXh%((Y}{A z_+tj~Njj$If9a4OtDF5>TLT!$YB9hmtOx?0_C14mZxUIgzNN1`v18&@zZ=8Y_ zk^lfe07*naRNoEH&0VsgxcGgiJi0c$wi#np+C+2QUs>&*r@FeD0$1M zXDc8@*&jJ=Lo5J@w!g4z&2`Z%0ekEKL^3a$%c)W}Sd$wL6LmUvR5->uI1a;Dn#LX@ zYwE%SQ$Mg>&V_BKfFuv|Lq8CNex>_esr(>ZnocFq6@Vm>{0HHw^{DE&6ic=3V)&AF z-hM89_>sq+#47sLi0d{7jh1{ZV0;Ws#~)o(FnK03#-Tp1&3WmikpKm5#L-IWFlV8M>|NSZU<<#g)>ak@^B*O5=?FXbsIP5|kNqK#0$A9S1u+P}u1t)*hU~j%0QRtouaJoH=u{(cbxC zA6cnBG5#37@Q6=-!E`+f1Tb0Pm>v&E?4N&QV!DDcZc$;i8KQ#YhT(XIW z_Tk-Nie+j%hAIh;JJ2WTpyhjUT@*`zrh_p1!`9@!x0s1u4?AH3=>DAde;kL=;gs#d zNWMNGqnjXXoK5G4#V2RuJeC7(o(Bs|$3?PCE-zOCjuHD1MfL#FsSGx*1Km(~PO^u= zp9z00{N3;`z<&f!%>v+7+H1IWyF;5259I9GISUp(@#dRv(nh4X=zk>#*8SL1i;i<~_{2J`%NL;CD2R<54|90vKY;kZ3>_iX?sh0fPJ!V zEvkTIObiZ(jr`Uo&G5AWl1^pG-yjCw80YFoIi>9_>2U&aF3Oq^29ONHlUmeoN#jHX zzpqa=l$6}B`fdsf3zuUibu%U|9@>*sb(K}SIcdU@i~b1gN%q+`K~+9oO}C+_=nec9 z?n_EcNfVa3+DS0wk&FcbBuG$kbQiFF`1R z_Xl)|asL0qAgS(viMldU7uVL*Hhu){$Q(Nx+Ly%PH-GVpN%;WC3NWJo*p55c6%AJs z{Ev-R{cC7ozB7ORq6^G}at=m$;P}^mF)5w#4m(&qpwaj!D!dT( zB#rnuwYdu?t2=j8srbt`={}sp$^l85=d>F ziru?CXd})ZmY4U7P-wa%jizg#Gar?@@%L%k*VK;fjC+IRqJrhUzY3w;BX@*L95WQ%RLVS`-tapX2ic8IZmD!L(q`-`zB+o4CbFU zXSHou=GvDYMqp0zc(s*#Qd@kP2go?6Gx?d!BwIuHapLwin|(ss&fU@}vq6mquLzOK|XuH`ae47DB4BGBSfS#i8nGPqLw@k#P4tBXV=2V)4S8 z5W9l)Q3bx5nwlw)3FF3%OH`GMOSiuSFowXyKzxq(;=H2-yG9ELYHe!z_Ws=6kZ6*C z>{AV6=%`Vn21HZ2!YKaE&=sX#liEUVHK>ZI(F2o%nyyq_e5yvx-Jza%_&Au|0A|tV~}jziOI?}0L;FN z>yeBLD`!olV$0JdEv?2=a~8ZbQI*%U$Co}j?XhJ?We!U0=kt4xhpHsEmDdn#B%Egh zfdm;b;M|i9ZYA(&{bd9x(2&VAk{W;b%n*5%ab`SkB|YQe{a#-`TmxqMc4?T7iOI0N zLs0Fm?GqcS%B8wG%InWLZAvi`%lu3ble|G>g_*pK)ZNX2YumGO?_CM>MG4@ipTl}_ z!mi@1i6t!_)Tq#RMq{qP8Xv{Ss{&i>K1AHgr7OpFmPc<5=V`2HyH03oZc;*L>`8Nc zQqrofkb$9j-j0inHB>-{re^uxu*IQud{>mMr+H#u=E*InBl{kwt_?``Xg8{!t^KA` zS~vZ{Bw&2b_oC8nP%{mSelSpIeD`3vxbZzJ0ekEaXm0K^n#Sm?CZb&kni|y3<9ls^?u{#fFeT8HfTWl+XW_=fv$LHR zk^6dn7z)uvQ_uehAM0nKq>Z*T7?anb>}vt1{RjS8C@K8`J14L8YM$?De$UvHq=b}d z1(TDfT{!s&_;W9sdcl%#>cB&7sw!FzgMsLHnUiE;++@5v!?^)Fo0)rcCD1D*pdG^jH;sa;INQb}zMB#i`UBH= z-p1BnO=ZB-*wiFh8zR`}CK9!~cb|sJ#oKi}unhN)#nx9EUt7~v2crO-{y*gEQPFOd z8j=`*>q75y1JxsQbF(0qZ!|VHDUF+;b;>@oe#np^hZG>mUob$)&oO%VnQ2cll|f2w zcXgG(ccVjZ>cNqDd9H0ZB`K;CZ7eDIH%4B6?3a)zQj(ID8L_{T8n1>JS{*c=R%R4Q zLPPru3o%L^DcS^%ovFpw=lZKiidc!4rY#4Z$6knY!h>@Rv==lZ@?9LvrPOXa$?hiiq}XF?PnHwIR((gh&>0GHli0cO}r~g-p{v zAr@h{pnl@fIY+B^=G4I}9ctQc zOxJ|L_j-VD?+A~W#Q_|^v+F!w@qMr68<&(Amo}|nN)k@$9u2VZO!%v&6->Hw+SJKU z6%?bqIe3GabRfomoHBY0CcI_5m+`o^=Vd;bag1U^UR%C( zVq^6#OG{Gta!xJS+{T)Wm?N@aE_OKRhA#bZ*@ZSg_v%WZuStO0nejXS4z^ElV4wid zKrg@K+H3{t-a;h?2Oz?C{vSps0EltwaEh^jO(7`I7~gkF8|~?hpm83`s8g%PsiW2+ zwHMLYz2%fP)bug&@$aEU4M4j!RFbxIY_H`&74NODO#lIrpM#2OJo#D0CB(&3+u^_5 z{;ERX+i5nG6g`GMe?AVlF%sgr?(E$0_l1+5JWf{021ArNmc4IP5Q2$Z5dw+)ZU%acg>7A z5=q1o#~KM7N1)9}P~utN*uYsr+MMT~pQyyZCF&LlaFGIs;swzje5+eB?BYyD=I0-W z32a6~gY$77i)Jqf8;1A&t|)3!TG}U=GuNwrR}J{C;JkH4&7`gFm39TXHcdjc&;FD zTMDm3;q%}rdwLp{vadwfXe)yI-r9SH(Z*X9e6@8Ac~Ud!k1Dj6Vwub+$suO>Yfo-I<&TX-&wmlb>`r%uYWC=ba;(+AEmtVg8`DbU=K%+0Yrq22KWA#{p zP4)XHj2%0+e^+S(s>m!{8eJ3TH#RnDD)u9(We4Xmg!k=m%h4JA>eZ|3^0RZ6;=S?r zWwp?qrQScB1J%TYgaNZ>&Zd31PxepIOG9*V9#xT5PRyh&C}=$3AR+eY+Rib{+E!SlV@ivjz^W)2e5I0Rev3b&~`ZELad6lQU*olh&|m zU`9sTmeTEBm9~t<#TI-lu*B&AsQ(Ikk{_-wE{?ce+)%}$d%bUnK=2`g2IXq7! zr{4&_9)2zS3iwL+z1-By?>)R=$D0=dIGjLY3EB~e1ggt>I$mrbQI;CfGRBHG(^%C( z82QLB`zq5(?$Q<|*^}v>lmHp=?9;p_!+Ovv$Z?!SYIDGqL>6cABNJR}cWi-{7Vbn(1Y+O#ZyrxT#TyqdvL&>@&yT_U8Zb!F8?#`jP}N36$Gn zu_Z*gTJoR`O{-m1SXidaq!U7D$mtmH{0AT!0RzWcoCLNoIQ2Hw)>G5@u}-q-Rl!&_ zKN(}Vq4f=FX9!`!%QmznWkBltU9rjD7ABh9f8)0?o`;YS0CM60B#R7gyfT89g04>h zk|RcpNY`6hzXNR>#{+lD&OTw<@c9mDKyux>b+yBDhEMi-TR*`eH`?}{J8^iG)!#e# zkx{!2;DnZz+W*|)xw*G&C@y}$0lAwrObHMqLR&UX7{33@9FT78C@+%(E~jky?qN^T zGi2~!uh-*U2q5AcJq1J*78Y*ALQEkh=y_E&)sE{!;0t{+w4eonFBgz>`eQgidW`Jf zz?SW!Rr(`><#qM--lTs0Cc|FojmwLQsMO)QD3kzqjie+c`I{RW&Oq^Ai^3v?R3vIi zulQ`~73il7+R6=yXrDyTkOZb_EVWL=-X}IPj~Gk7>GgOf(MGG&lfc<_*Vfm0`=_Ly z#M8vr78MbLaGfgwWeMzq1S<3Cl<=YpE-1mt)3X3{T@wpi1sNF`(sxQABaC$X*nei; zw!u>eCW&xtBO1oCvYn!AXPGD~=M|pMWo4oq9@_~+$x*2x3V|{CQp1q`=nm3zIpt$` zYSF;vk0{ds4$S|_iPsnb9e|}4gD?}r1E+R+j518(fQ)ZfdIE`@Hnahj7QLB<3GGP* z0nRoY(Yi5t!K8-77fFi2MND&Z3un3#s36}q-jW^oSzsht8=4mT#p_QMjlblUk)RQA z{j(+{q-K>gR`1MVJj*9t`=YQ&K|w7^7mzfi|JK3fbJ^aSzA zd}KdI{HkehK{0F+HTwu6IB`smt=mBWW7mPMPXLn8(zpSATd)=rBqy@4(_*sBZR{Ul zb-Te1>xwIW6K|pUoRJ7yaXb~*iR0EQOwK(pzj8Sa@cC6&m(@wxRMQ$QEmxsCim9np zQ`$&Kh-YuK1GW^uRxLwcq1#ZhVde0goS#86?lDYe2rMd9tqP0V;KA+R_W&IGXkB69 zhf0iGA|(kx-v0#{j5Z?S4=KsXq8jf*#ZgNZ+wi@EGPClV>lz*fkIPl~M1)53ypF!= zCoqlSScBc1px1i=sGN%hj656yR1}dixlAxG@xEz@!0qwSUWF#AV~KEw1^~&p8eQjD z;(o^rT!L`{#X+IKEWKL_}>q^0UV`Bal zXlfRT_^zA+k}Ma$EA-(;z{G2A?8>sby!*Zc9IL;ph{U7$|Aa5g`QDt+Gi$|xD_hZP z>7kQAjS!lRtH1h6{P&&zh`DoK5P$#MU&Zaey;c11nk&UsS6n7;{nanUgMYbS%zkc` zc>CXPiIpo>i0#|9g$hPGDN3Z%=7S-%Q=Wt00sjvCL=oj9 z0X?`q03X9mFp^OwcxlF)X&B16mPL;{I-8zXd2eQcdq(2FM;itIK^qWhr<1&75;8GC zNraKCvb1DkOGEWW^2g-RJLHt9+hlnxp0XTfQ6#C%ONcW+yZh{pKwGkLohyM}DS`a_ z{A2M`KbkB5suiGSx@k09dry_e!RR!vcOKF-CPNcm#Q^|CXO-!Opr9)5I4Z+ zzTRed-$5Cv)dS5|tf`5)4#@g+QvALi$G0#yY7Hny` zLxtX@=~x1nUU})KXfMi=68kCjM+Cayp>ZaDJ#O4MwU^s*(cI7oN`T`cE@ooNSHNxC z5oF!1tZ?eub<~!o*0hRCH0`BoSQvRzg+5do>{!4KOKn-d#C}u`a_o58Hpa6A2(HB# zeW+swE`ck7b`t0oK(e6V++yT8t6jb!-vZ3Sp*UhQfTs^}KMywj|BUCG@TE@HeA)>y zK5Z~gqmH|h{80Syt{LLV zCms{8y!?{*_@fU6c5R5-T1=+8&ap#f2IcVFCP~1M+pf1+&+7Xx!Jq?2xbB6z+D}&t^I^&SGSyHkpz%e#NU#wW)&1WkpZYeD^2 z?aSU9?}N>)KSf=jWvAA^U^2#PH>IK`BrNl~S>(jk~kFj15Z`Yzxfp z?Y}i8A@m>-ea)X^Eb}T?^2uq3!;=7iXLv79b5V}wzEm83U$DI_(i&=Neyl?8 z(sYjmEPLY#&2^2vaxrscZtggY^Ugs%ylBgmgog1lXU(PEqV?|)Px^71H@__vhl9yD zM)!(f@V6@TE{!VzTLNmMK%3gZ)$bL5Y7J`B@xyhNrzU=m!(ko{C98p%Y4;Rw-g?1? z_3OW}cFh`5R8%NRii<=w3}<(>OGrq-;XGL)Z9tkxOB)~t3`mC`Ao{1JNn4rR`WBBg zIMM2Lzpwvg1qEjws8Af$QKg8UtNw?<=fjVIKLLI$yotWy#(^7D+Zee2&u?wm^vkm_ zRw96BfrSJn>FjI!d&V#v?~cK6?XupB-n8M!WD+E_li_O9?+KT8IJ3g{zn0d(sWM*@ z$}D7@%umuWOT_WMa$C{qt&Md>{`h`*D39mxQ!XrB|JuWOl8pJvJo#j1&SyMbxa{6@ zetX=luWcqP*SQkt%@P=vn_B>o{S<1UIkoTCzTVhI%I~x;B{g+v`|wEL<0y~EjG%Tk z0X1mBIATx00RWx{TLP!T_S`+(?&*j@lq4RPpPyMD2>t?^KnAqz%*M$<*?78kzr;k^ zCwr;8Y+xmd)aPnN)&0MAen%N;R$)~t(B)!7cJ8#PI33+nTw%l<@g zRaF}qTLpQ#_SZdmU`Nylv|T}HPZ9^2Ou=@EPXT7Zp5zWCdy@QZ?%G}HftKEPX3v~? zJF@sgyKLOsE|vi84`b{0g#eZRo0*<&l$Pyu+`h4hY%_M^1oa_<2FD0b>w*arCLG_D z_R~5N3&xnnu$;UnaC-PXSTGkjz^bEU2MpsuJ?QH+PcX=N%l8hL*o_QX0-l(dr*NX! zt;slhpaM4hojNX(!RrZead$wQ`UNbOZgtATP2cqrpf)S}TeK%PBDZI{KF7%B5Y)vT zXv=hrb(L0UxNy%M73de|vptGz>GtR5i;0JlyoI1SkQe6@L}cO`bft2nWmm(8F5GC{@89Mz|* z;J3i@<9-ePA^2Zh-$9-m&|nurhn)mTI|lV*+><3p~h?qy_%jjG2a8s|g6w z(^(fA_e01Y5K!K?84}^ zPJf!u)Am&PghIEjsAw6!;V)FRPId#tt#^l#Uo@s^4Cn+(;wx|vhf4Wir;Tl( z7k;J+y$a3Bl9g5H5awX}qOPL(1OSibO@aW=?}9B!HMJI%f=7^M#j<5r;>R}}?@j42 z8vq*uTBhjYv}Yq&tUkj^g0!0U6KLUj(o$2E>qYy8*dE7W!<{4Y^1h|a$R$*g01g#; zA7kX(Vb4=bO-@nbK_Y4xmR0QHM1D+J=?;#0yWPcU#0Jp>0^J5kGP4+ub~ieb0PjO}Xq!?RT3Y|SPkL$%PKRFrzYG3tcy8$%53l?& z#6e{_CpTA|c;a|*&bjAFVDUG%-60}9Y$cQ)fB;3wk=7@7beW8^;x1ey#?UI=X?U^ueEg-%}iYzAmCv2oZ+K|{ti zfM|iAW^!33>sAt1Zd!j@Yg7FuhN1KF3^{r74wgYW@{qjE{26N1=b9CNJzsssV{&)@ zx)SJ732ZesJ%Ath;pLST$^#x&fHt!5U9kOkzDT4X0mGv`>i79QD%(Ohu#>CJhB({W zKC4iIo+BBVmv|@6}t(@_9}N|Zf=&)jr*V_rECi-H6@vSRtQ^7(`zc7w-i5%hYUEF*CxfdEgS-4eZ!7{_fgV3*%+%Lkhd|kK>(qRc(B>K$~lK zRW-&v#`k;DR_Q*~av%4RfVKN4h&JFZj3cbc-9Cw;?2)@Eux0590CUuiR4$!dxYx8T z&@}v5nTbO}>|%NXGiHzKA`HjqyDBTan5>W6TC%yM1j7%gc@@ z1!9s~>szkh6XxEz66jqL7?zXsAb$Hlrd>s9pM~Qv*@AYYwuPE4;qVESHZCFIMSO%c zJdrO{n)d0a4T{M`&kG_1Qr^sI<+DG9XApEjt+dy_>eQ+RLS@?>Clg z`~*$v7X(B-62EqqBf$kzb4|aE+5^t0dp32GAs)w%HMJZXTLf2~I=nLi{~**J$K-;G zh_35I05Nx*9&uy{L~zRoj#81=kH`4DAg0ov=vt_s+j77w#M?XFsL7Zb)Q?_dE{*$y!0CDvH)clkN_=XY1x z-F*6v1iBTFlw$LH{)i7P5Q+%dD6Lxch4|ZF9~3k0xPzzPJqiCUcxqjQ>{786=%kZR zk>KOK_x~BdP@CT|J$A(T;coNCE165~HwQT{VgMh3_yhM2DXGu|y7A zF7`;2L>h+as1@dKfqB`kqGa`{dUMlOJ|oU^*hy1L?NVE;+tS-~xc+qY{HfgH?>bik zy-ETSj5Nf}w1HG!Q4veZ`pI*s-RXh>?@e(coo z`!CxCU`UAmScQHXz+CtDT}}&)^w4PiGup$EcxTFk)U=dTfRNhO)U>pj3PlK?MALqS zlcK$_3S+9T_=tq~irdrA)&l%Eu%Kzq#{x5Q2&wIc5*=oqbTNFE-}B zFpJaY$=N>mG%*FHBw7JV>UN*)wBmeM(9}TFW}LcW>C!(srFYYZCxNwV*P?UsP36h* zupf()<@b!)oN#7)PkWM3G355zc{p*mTgPpMg@wy8dR>%~l%)H-YWf2FEn0on(bW}| z&-Ru+%gKA{?z9UQEFchh8x}3RSX5QYUjDYq%RJDg#-StP6<32*NsL`i1iOSLaBPUb zjLI36)8!e4k-@0Eys`K#-OKSg%dP6H1(1eBBZXK;w>oG^{sxQAUb(A7F=y^lXlZT1 zhM?dJ0Fr55&ULN?LY6?cPwJ*!a6us^XiprdgiSE4`mZ55u{Aq82ghAHVRfLAMkWrRwTEs2q zP8~-}w>l+a`rY*{;;pZCR!Le&j2*RwCR$g9G1eYb@&Wg_$blwqO~{dH;LOB`;iI*N0{TnR*@1Za;XKPUHZ_;e%~2@N2ngCE-04cY|+xOT-SCqL3YysPiy`XxM! zJT?vJpQbcX#bGeq0?hiKgM(5Y=<4!@%OBboKSkSqBTv<)rl5)=cnghwp3r=ESKIpR ztb7}bi{C~=|5pE$6s5^IxAT;nVDiXdbMyRY*IdH^9cOB~cm&4vmA78;Bv>3m1Qb?0 z1shGMh$YbnZJgKhwtyn!M*Da32fZ7@ncPReigPo zNo=p>LJmfL$2QeB-rpT%Z=Z)p^WTASs)2T|eXyhZx`ujcdTHph3T8h$YgV+keU2*E zLy0#O6)i&6pJSm;No$|Q=3#)PWzeQJj0`-yAMJow4~e=*v&FliJy|SD7;&ve^X%?k zlnAx*$L8gw1cLg@SakKXyrI(Vn~pmJFy7NWLVH@;@7^~-7eB8rDq4#1%1bFp$w4kC zsW`by0pAsN1jPn*@Zq~D+Y$r{*IC2}2`Zo9+c&g&xS5i{)>9m3=4_s6*Wqe9fLXzgwCScHeGgm*6 zzX`@6VNW}>y@!dQ@!s48i>`A}mQ#@SL-@%KXxb+YytAh4YR%&*CqPI~7}13D_-ulI z9N9C}Oe2#>z|mx6f{bnSCNbHNSbBCWGM+$TTRyyHB9k|sgg?fA3WC-U%(R`EhG3*5 zmBHjky0Y~vC+MxM@<)Oo?~s!VT5Zil-v)QYtaNhhrUXunI}~TEo*?1IG9Zsm#$d4Y!3SMSXExp zMn^CPr=iDpXSvu9;l2Epwj{&R=Hdb;$5iNxh6aqL4@TzZj_l5QbY)&W@m{*3K0Lu0 z2N7{m;^gM~hUcPf+(3}DS~uolMw+{`yxhwJtyCR%RKOXA-}^UJ9^ukyKHu&5zO=N| z{>rR)FxmDUJGdh#+0X(@07_0!W)@|HedW7~GO@N5r$+$Ef`TzE(+Vd2BDr6Dn&0a` z9)Q?6ufF=qJ=gv4>Ya05oGof;(At@M2g$z&KLY+w@SEZHb7!A@cJs8!lRud@b@D^g zE}T3C`42?LwsOCOJrsb+S_~%IkIh-I@aZ?-e6zDb3zse~fPWYMfAA;3t2l0tK6h(H z<6@kNbJxBMppjz*nP8#>7>NT~4sDhR_A#9m-Yy#E&i0`volJmX8UC5{2y08nc*6tx z8Cdm{0}fIk$jVi^97R7T+swIt(`GfT*OU&k+|I>MA?;Gzv zyAp^N31Gr9BtIwbG0_@a2>|jath#dRm;4sODI-tT&PW>&#MiMcH9hU0;qX$WHMJYG z52@Xta&j~CCD_mipbrO*{5?NAmkUr)<%VVFU8M_sKDO)`7^G4=MyY%ppaQ)0!gS-0 z2<9p=i4f6-O&iu=A@6Z)lQUu!C+9o~xw^K7?f}J9&Fp&ajEP2)rK%2I5P1 zVwE3SG%607k+l2CUhEIp`ng4g$DvG5($mAVCs|Gns3cg6TM%u(wL{_k z#wYc=0`GAv?Sh75Pm&8d(tfaR%u-R7D!}2X!*X(8tqvNy%1TRTcXwSjmX!P(L;9D|N9wBm5zCA%wqpBEfG4^> zFsO@Vz46dITlQ?@VA|(A_?LR>z1U@moIcfQsZgw`8hd@`_gySV*))u#ytW^+T|fYIAzLt%cf18)PK$k&)?G2*qCL< zg?Ocg$Gh)s`0jVlu1`)$X+mKEB8rJ&6nY_O$H4!KJcf=L_ZapB%0po!m}lX6 z5;PDz&UGE5Th0TSEO%Xzw_XS_*|`S+2~qWXYwc0OqE z&3j2;=R;1gp8oY`C+NYzF5V;BJM@&vTX}Cgn}>uVt+E-Kaf==AUR?=9g9Pv!*YdOT z&lr}SI~!YwVAevv7AFn*wv=x7hGUbaecia~N!v^KB(KKy-@UZc+o$X5dq8YA0UY(< zG*+do*dAcJl|}>F_^$tm0r;cQu$_lC!iMGK%>YvRG`9O{r5KQ@YP3=#hg+o4GA@MH z*lS9HiZGGK@4pibab0@a0OiR!2JUUzLa?j$8s1x4go~-fZdhI(OnC(V4ULQh4y0QH ztx9|dvO_zPNaoim>1k1KRzRteT}(vH?HI3ja=7*+Vatk(7X&3wsH&*^a~BueWxtOE zzWnmb-8igp3i=!!prn$S4zgq$3VV`#kICp8=W>F$PgY&_xMt0oQqAkR5rBawoUPj| zAVC$>gTTtTKy&j-EXrKeWd)BgW0YBgO}JArK6%0j1a?AkVm=mZzK5-uK4=Cwowr4# zba9%-Ln^B?v2d|xobQ}94!99pca(aYTbmQ0QN0j#e7LvRI%aLD)sF@2hpKl~ZpYB* zWz^vZIQ_53-iz}$aBvEO`PEcZGW6>{U%K>Fj2qMph#owvkW%pXQMIe`C43j>R_xv_ z7g|Fi>?9F(!f>otulYQ8bdsdof-?tY{RWv8W)2ttr2%z_h}x}NF@I@jZ1keN_y-m+ z=8enG&+N8}gqhLk{QM&UME-e8$);^6|EoAq^PIl+U4<#4h?A@T+9Fq!W5J5s(NqQ* z7lQ7UP>O<0x|o0)S!l42fQH7&@Z;f!hC%9n*J`)lc5D1w z|Igl+09H|*`=7Ztx!D6rNJt2fgdI@?5nNCf*J@o^g0)s{``T(3wOZT0F0Xd`w|(vF z_H6C5YPD|2W~;5ORto}(8=@$p5J*_U5=bCq&%Jl%|NEUYb92Ff$xTRZ!grE$XXc#m zEb}`vXU_cQJKy=|GpUDaYh%GlpF(PC>YE7($+K`UAAYwE#LwwpxHkxOj_;Ubm;A#4 zumbyMLb(%7IBISWKQ71T2{Dx+54{kk*BQA)A~V!vqY*IaS&(sP6P1#jG!93#US5$h zB6)cg!>hki`5&xU`m&ckX<*)2uqC;zZGeLM=t!Lydpve#uKUU5Q@u2@6$E+%fl0-a z268nzYD0p^+73eG-!S5T4Q!wVw(qL+Q8ThDQ$V86iPPp_pda0!bsx10qT_mcpDX&| zp8hqz!gk3p{r-Ix?Zu&cxw*N%aV-st_w>1V<&7CLX2ipPdT6EN*xw^jx@+%V9~W|o z#j)?o1L=p3sy@j|jhkudn>KBNfh6lzPWjL2!jv9r3z3+_VE)49`i2c4lrN42r|w8| z$|BAqPo7Ul%W_G`Q!xwS2Y4|rhfTPZ(ER^QLqlT* zY+0*{ojVe!3DsNf-SYCiXqTGK$kQ(G`=}z#0Z)! z>v1=xc+5C_P5%8<8wuY_g7?{Q#L(fHgc5L6SDz=rYvdx=S zS=nZ_tFp4Ar}mL@lTVwXuDIe#HE#S#r{PWMX>{l>U{3Xu=_!GyFT3nAPTf|Q|LWz1 zPu)8GPSACed29$hmt53@B1k}*Ui5f9?vg2F9JUNL&g?Oc&2G0NqXUb_hzbr|R? z7{QNOtL|v$zy&t)bMx3=?o^5XJB}vTli>q$a=uYpT|FIB$WO=AW*=WTJL}jtnCaWi zDmN!*e8}>@K}~_=w3MG=igRp+8#`GEeyMM5Y`iVp()ypMPZFmWf+RJaU?=nP#7Xug zf%LN+-&Z$m+pxzAkB9B%imiXbOY}Kdar-m4W@BEghf}vgMu(T>%T#M`-hw;vw} zRo1|)a~vo97VvpeDgQ6IAW*FNc1JpXSPKV(=Ua|C9~zwJV+=nC`8v(bEe`E@f>7=D z@kv+ey``e!2|PRIg6y7$4@wrugP60^pnCa+{Oi%Ie8(>X^dqIX2^tiU)41<_@ERL4~lzTJk!3Byh|@m-E@i@kqRNCUE>5%(?w&XAs#pJZ>sc3vmBp#= znj9n(XsAZb6dGcXaUQFPEiWKVJYt{ka{t&=ds?~THPR9#Crz=;OU%;1_L%25H>2%^45Y;dkjl=-p9T>sRgw{xx7VJszCnyYd9DZ#^uhf8>{g;m4x(@@Z<6^__44A_i5vcE66*FUPapx^m7S z>lGKu!OD)DvSj0VbRVYc{*D@QovKs^g-IZ{G&U~44ficRHt^BQ-dA1Y<9*l_2Lf3Y z=I8SqUKuzjXI4CDM|QRU#EZjATeg+=iWZ937855ow{?HB`IL%{ms|!u4Sjv75A7Xs4JJ_X<7haxR&2}JG@y<-o;Iv~u#t+cEQVSE3 z1Iv#0#~IqH5uvb}iZT3o=zA}~^Q?$(l z6@sBx@H`qrvY6wlt~$Oe@BEb?z2SzJV4mgp4A|U?Pbi*{ z-=Z2$b(HTs%ds!Ovx+4f>TfN9R!?jqVMi0k)( z1k3vM2j%4-bT+S#lIMN0?u|F*0Q(=}{um3pqt3qSYAaT=F45%`xA#fFLTTVRyw}X2mSb1RSMPjiV;lDsZNFl|jXDHd+fU%iUJ50Je;@C06F#^M z3b%wYlggfrceV>~ZqraN^t&odd$RC;^m(cUW*pjcy>VZ$9W1{t zG#h1w>~*xeRk?eQud9SQUizRdG?0WWdv~_on;4}3*-1!ve)Hzd@1fskcubCVDq9*F z@GP+A;r`@D%oLEQ;hNfmeq49%2e6p3)`_U)FbpIKw)>K;w8=e5k{qknUFkpLX&?*2 zVFEr}ypDS)oSTb_%lc#4w(`#(3%l-&>o4ulRZOtj>SQc%@Cp`xNT5jz zJ}+Z&>?em76!0Os;!~f#@xtRx4n^0I3zpGFFtr7_&|G&p+G?W`L1tPy$yUA~(`~>4 z6RV+~^D68vzdkT8Z;PHzY2rHk%a1UjrlxUW#~Fu?bsU~wPylhpVLUtrZJ=Kgu;mLm zv{#$)ENh|`T8H|@cj=SXwx~ENnkfN zHT&5v=mX=nU1II$($FtM3m(^;Hp|eV^E1#W1ua%h=tGl_{Q(KC^(44pGUb^`QzyN% zc=6(x)kgW;!NQVPqCIX#GYrMG6|Sy1;KwW`ulO!)`Hq5*=zJc18_y#y#v6BzCT$uI z)xTyt;gU*db+cdSE=O&5Vn9L`_n9FlmGvUtpI-6#tycq%^R70_(8&U6%l+}kA1`v6 zgX3{gu&;CPEeGcMlxMTt!Sxo#eU;$dR~>mT>#hDS)_beB>f6aCi->dJ-`kFMAQyJA z5?m?!A^3m5e}IRpwRGuIQ$lv|VjR!F`^{Mh%fNn@T|K6a?8iWaT8-3nj3yqdq3(}z zx?DzQGQi7gT|vQ+ch5a{X5B>>f2<@WHDz>X(nrgpQ)o%7+Hg-|*|WFdMDW}-~e;pgkag<@2+aCuPJ0baE!D}-L{WNrX!^eN?$xaU@bVl-XToH)1 z6F#n$>LZzxBq15e)%!5z=(C$EwmcZGRmZfO@1dW#i_=0l<+(fh&;UNp=qa>#!2=fN z1<$f}pa)rPDc@#jgKUANzpdCj9h?9FKmbWZK~#9ztF;pn8Un#!1Nu^@B_SaVb2K=* z9RM=91eF;HA`5-0b2fU95@Lw(Fs4Z9#fFZiq0m{Z`7x<*aDLdI;IB?dN}3Gqm|dOC zqo?IZ7mb($+I=ZrVjYmlR!0VOA5}*zfB{l6jl^W+#T?yt4Z81@Nc%n>99taQx5e+X zYoPAdfW?0rKrV(7!{OAR-=AiMLNJtS4M&@eV%wq3&OqC70XMB(LX!xbiqp{-yN{|b zN&g{(^GW)L@PwHWPg)LL=|AIbAUnEv#(xTnMLiO0 zKAlO|pW2}_=iQzxrUBoCj~DmQrWmL3#p+-(f{RBfd}#YFKD7NP)_h_~2itu=g_V_W zboCsLRyLnqBx&%X?%?^&mvxVzWD<;3t#W*y*^j=h5#&n~dbSqqbJiC~EGq;3PzHv& zdWHtnF4D1&nHv!YbGczyNNG(3xuCV%TVQVxq*qMO-FWIJKzN_!SfS-DEv<>WwGHQ( zxu^(O(L0=NlA4;L(y;n4`unzVg%N7Pb7~vjf_CD41FP*hjW}+?b0`tdF1&j>nRqSD z(EE-v0C5>^n_<^DjC+E#{RpCjZKF8zfSUWU*rM)@c8Rs0gSj1Q2x|~%W0m&$I_tb~ z13o(07!!C?Y@5&)uR_c8ZMF?a^yal}-VG+Hc)yD`mDtAcvzVUs8Q_VZ_eLeQYMv&M~r3dZK!MDJj@Id6}s&_L2t_>*!TMq-n;Otza9axWCk@=?_9b4g$HLa ze>$C(TgaMP^1%1K)4uuFA242Ad?65PCmdgA(et(8;A}|V;4lVeTo~m#ym&9_`C4?m zMBLXF7M!>n2Bg2p1t@5|ygNuq)`0y8SFBD+O4jzrk^_N|&k&UCJv!vTKZcqy#h!NT zs0m3Dvb*ZX3Ph2S4GBKq`CRm;LluhyA!rPJ5gJ2x^TmlPpLYi->75o|%=r~PnLdyd z#&yjXjZU;1-n{YBP0b%|8UcgQ?yR3nMjFz zRdgB0b6-ENibp(*apfmz3zrLY9WSS!U+4#^70Ty2ePV(iGamdTDq)p`V{J?%$B>qN z8^->yk8ZOcPN3F0W*@+ZL2iZgi#Vs5c4r?LZ~X)pu7Y`oC(!R(yD_c4yEG-+YhShn zo=bRsn|9$bNK4Fc08MQ99Gb7rWRY;(A26xZ;%jJX z()|;`FB~RTQvRnUx`*u`kacCZS;qcVA z9^!M9(-V-~5QfZ$NBReh9<+?~V90&$G6v2JkeJ3j(t8myP=_KG71QxZOkHhr zZ7kDlQ4Wvs2m35v-to)%KOc7c;-*MOvLgueQ3Uv<5etQcF$D~Ul$@)}%6oT3?T)~R zdy%*Qm$15Bb7p$_?YuYl;p30jM|32QqscH2+Jm8YS3?Bn=Zg$~3SHd2yJ^79H&pi7A}r7eF%-8cZA{#~m3N(@Dj7qJcR% zU#i_-eP&+HpwT;8W3wm8{s1e{`e26;3NoRkuu*yX`t|F(XAWMR;}4OL5mUP7~=)V2cR?-ldY*q;ivFOLdSf)Vd3Lti@HymjkRkdjvgti+{5@(W-Z0Howm>|b{EyqItO zhx6_*@zvGx`%C(;Yfm;Y!S{c`ke!7EiatqfxrkLPjs&765%F;wPLCV-4VrewwK-6X zBS;o=mM|U14J7GF2)o4aal5xjt`!U(Ofs?_qtFZc0vSn+Hf`OydQ3^lY{2<1e9%j$ z2J#Vo62-d3@gqX$!yg>wzUmfq<2#dhX=K{U;?(UM;k#}~44L#rHvq=1C!Tt$y8Mc( z)GfDtRsH(6zf*s_|3USQZ+=_NnLAHSIQe9fk&#owVCC9|rk0l;fAY!VNT9njZ{8If z68+W%IPCOb+fL3>q1=VfEl4&2qS`Usb9x_#I>-yx$2td{M?^NV z6Uj&u1q{YluIXS7%TTQ5IZb}h3)Y4n8@UqLvkHfpMxh$x3?eGmLSRq3rQ;oVMdm)23$Rd3{MUG8z+*k`GvZ-}N{j zhtrq68S>FG-LcP?bG|TeCDpceJEYr&TZ1o~MwJ9Yw5rS_EAH#;RjkR@uh$$0E6f*)`RTa9QXQ6VJ!cLVqpM?>h%G5o<912~+t&oFLGh2*lG9 z6ypkQ2XX|qx5D?x>bTtr=11j~h8OMP;u1A=>NIu9r5{(<-|%U5$Jf5De)_XtsQdr) zkov>D_pAT>)_2vlpS(_;b=Gt>c<|tl%94tNlT^@tR04>V$ z&x^Nd+WiXbc1}i5vG!Rvfn@OI49sdrqGLe~Ahq#A3e{}qx{)O%pX*pEk?Jl4uo~`q z^cP>oM+{$f7ut~@AF8pa7)Eg)hn>-{c31s6nh);(uR+u5TWP7OHneWF&E}4#6}jVw zz?N-WmSf_>c_4vm3Jdb>0a!3dTzn$nk${$M4Z_EY*3`_5OEHjrmNsJfk-a}Snhhd3 zj_KM#j2o`P_XB+MPZQ&T(>_u=?l1 zMN7VZP3gQ}dL=Ac^z?9BwNBNfofp`_tsO{Lp@Ai@B-CsEx&h=U%z7czQKR4f@3*f)$T9|t69oDK z0{r620UOCXT+7wCoNvbq(j$GL?#HSl7Ag24rf*k(+`0#=7pK6cq>r{G`-5Yj#f3GD z5qIn56rY;+Zduvh{%~%%_Lg9QqJ7Xzc?@K7!GYR?7VSZ?@9z(egg@>#s$p%1rrzZs z$2Rnb^XhBou%*2G&$!NJL4)STrk3WI?Mc=gJc!vB$#!yb(*KSw9{u{}ip^XcQCtrp zfd1nCQAH&)vBK_kw9iYDI2Kong^plr)=5iF{TXbXt-|!e*JH^i)-+`mTYoaTq~s0s zdyf|u6lB4KeN$j%+tO|a-LX5iyJOpSI<`7y2P?L1+qP}4*tTu+=A3)Z!~Org_kCI8 zWj)L}M~xa^ef3q<=kuoAZ@Zv_(9p4DJN5LTmFjI}ou(zfTDKy=MB+`TA+);=xYq*y zOyR8@>|Vl}2|$oG!zJj^H#*^qNjjnu8hMszL9ubI zuET{;IgZ_EwH9ViMW)ex`BwG8jxh17(CmX}WCn1zYd{GJK=XDYAt65aQr_t{I{b*U zZc_CtI9H{9y~8rKq-?cVrE$ISPi9Fs*gk>sNp5)6&XI|MJweB=k)iRCC5twNINSc7 zw--Qs&;Y>IzV`*;O#5|WCp1xN96#=++5H7R&78CR!M82+Xog@tH8rhU->Moujy6!Z z-aF;hZi5o(faAQ$o8}QLX*{}&yhAxeY1QLF7{Q$Jb|aVDm0Cc z5|xsbKIRawHbT&5Od<67aZ0V%L+;9OI?s{$rkKwVKAB2!GL1;FbnS2lwfr)d0 zgAur`%ciEzM6AY>vx;RFz(gpSnUm58m^xhwd5nRUw?jV<&a$S%2-MFhZ5J{A?Xd8p z;B9QhPSbuah4+y-Ww!Z??Jo}+YnB)t?_nwqQc zCl!+pF>p({#T$qVHZ>ld{DlE;p@m@>Hpibj0N301KG8bQaD3kPI9-LCp4HPd%*iA` ziMbh7N)qCNE2KryVwLgx>(fE!(=xW>R_|G(SVorC>mA)0Aq@}WCFJiPAqK9@+f9%U zcW}89BIn~`k{=w#+PXL(t^R`#+#8ggoDFWByFT1i!Q!pFsEe-5^I z$PWZ@6-JFG1<|M zVks=D6q)K>sxO#odvY94`Ej`&IzmYxnGkg|*tGY`yMN%^v1Us$A}5HLk_2^w=C;jS zT_ph0S&*NfUkWICR=BC2kixB-vv(~mspLyUtObGB4L>rOC$D&aZQO9#H)r%Hu{bGA zL7U~}lZf>@1;923hQky69O{Im54I*FbjA=5(VsXnFi|9QuTK)N5 z{YF2`x}%9*b$~D4;>(ewFP_KNME5Q6@_=8dfWo?_vfR7MBEbnu+4j%eMv8TgpyyG1 z1x+_FZ4wWN90JVc7>{MkYi*wG;ttVL81zyM$kuVU?5>+4B*`{+7=@XOf%XG#k%$Dv zo=8Se^Z7DUECnv3itG9gb~aYlV%rJ4(SSp@+Q|ltI?mT54`tBsNQhr&&V$XzG49_% zX9GZyxE9c1+d*2h=o^U(-F-CgdmEN6m#ggXLz3{7q#{qPR-0^CGgJ=BUEl61x76O+ zX9@kqArNq{voB5*-n2!#wohT($h!;Ni7dQIn%*UvoUVS)U$hd3io9dby@Nf6(kG8$ z!ZWGyW9XlCdh9k)&|9Y5`#W6eIu3%XN=!N~zFIu6zraz+*Fm39^Rj(k9>n-~vXpsAu{~eMcrGT6s0z+-Z@`cG)T?oU0x1wCZ|AxDOb8?RHnPoQf0i zv~@4;JQ-;kBqe-$%*ltTnm(%J0HZ>81&#+yVBTKj=oxF?@8ElM&XkfV?AQ)G)n*QK z0hk9G{L_E(W*E>@MJ9)V&s1Z!BxJT3o#Swy8E!^(cb2c!AE0g0`|Y>`D%g}{N1}|w zcY4<+xMJjC%sqY;^8fCjNxC%}(FeyMLTYu(-Xt`XQX?CtjNKz6FU&xNo%_@45j~E` zo6^RV%qS{t7?5&oyr5*MksPhU`Me8=yscA=Ply?ge=7Rw4%yu)D<%G})A8lbg*4Lc>uO?ft4IXS6OPQgwuz(+ z2Agh{`{pJ;em%}_Lc$2Uc;cnlpu97RvcBO4{f+yTyt=xN?HqWdY>&3u;+$>5D7exq zt~cJrU`|iI&#aQJWqHQ#g}eGlrQ1Z(jog_v&I-40XA3VNyksANenSU^n_n5Qy+8(V;%VM zk0m&5<)9mJ55g(%yd!xuE`o~%w8VCKDm_??Mw3K)%FB5N0k^W7f{Ks2YlNP-JVQ=1 z3a!6{DoOVk!|(;CSUFzVpaZ8`T)xiy@?12JKNjk)qhvQ6j*CT@pIuE!NpoAMA{3O@ zg&t4|?LU34A*0MxdDJb~A_xY!><)xZyJba60tH>t$ixiqcXdND<;`he3& zUcyM#Ac=^47@a%kiX^_-Rzoo_(|x2a9cm-bDglP{yQ}RXcm|FgAiM|I5p?dV>RLXvpNS z(PHVEp$=NMTiQkXw~r^K%2vRBdOCc$;Kh4*!o`@ohjf1WP)<)aOW|qO+51^7vaD$9 znnzAlG~PP_c7w%|vIj`Zq8gpwJ2X__mcKuXG<=^%Yqp0=$lPVNpk)Joo%Fgt$g|{d zL;M6e=23dA#SR3Bi3yj_!TlS-+nuCy8FOyA0-`z&%PVN!E@t&U@zHNVF9-hCQrzkk zn!55+op9=($ru`zEg5Dr1=8^v6ciO-feggm7oOoTbf|E+`}dUY#@Q_QQzWfTAx>A5 zdU{q0u?kmK25&uoz7G6ZQl`gc;mo|?_{|;LyIgPX+cfVHos`0WRd`RpgYk`iZPq)0 z=`8AAinOKp`#6R-6owL$MbX$5;_*gM;4yx+&y6VtW%^X&vOlebSH6}&jCJ`&O+NI* z)(Cay1qT&uEhE0sw62-p1llnfTWcKL;JN0ezdnzfpEVyc?H&17ndP=}&)8@@lSA^f z8cPlQp{j2YJ+7~nx_H@8q1d*15`k}`ZQ*xqE{%7!WTSH#GBje$rvHTG%n50Iik1o??+oETm$$<6euP2-=^ z&>z>W{tTSw+y0(d-&`sKaP7Y_GEXol;1^6b3sg0qcYT$vSE?+89ZDUxU<{C9Gu&t~ z#=6{7pGY@Q-NmVjl%tZbU-w>{V3hfHq|fg_|*bIMKmN7 z#Cw(SNynQT$fzf4tBHOlj>YNs?njD3CdF3ID^nc`vW_Q-Y&ytqGN{$6hhBGckG>Q* zIgi7mOzQXgPWmZhrGb#(&^04u~5mjNjW2?p^Py5-;<;f9GD3=6K5xRhrDowrTQ0r!*Iv`9@q z5HbZk|AdpAyTWs9-O}Dk8V_Z~$hufWYs!oGUx|JU$!9H^Z+G<<+jpxj_kK0$wn`WMGDjxZN1lO5q;q#e zU!ANRyfH4WP}cgFfH+v!mdXc5QO*Y^n1hi&n$7-_i8A&AGNYxxwn(jye5ToNrq}^b zXZu(EvKfEa?>B62NcvI5aEoA-@=xf6g?Lp{HL^I?%1jorQ8Vrx)HO8BDIJw7-1g!V zm2uguy(B@ezmX)5P>cG^GK4?sXmTEnLVZo{6iFhx3d19c9JRScYlj1atIgTLggIz{ zTpr*C2OHvkcog%IhhQ8-Qu#R}*z1W6d$>b{uxn;rul;yR=fZ z@%O~C=jb$q*l648F%cwlm11aol00D8GV+3%;d(^_GK`AC`v~0Cr40_27aBoZz9|Oh zp8q!G*C2dH&NaW|;v6h{TR58`e0kE`rsq3LC4Kr4_^a@uw8Qj7+S_t&ZQ1l#5<>)2}fDa`m-h61zctE z4vZ(w$c_gJ3k7<^kpY;NjDU%QICGT}&XDI{cy$ocXUJ`{(LxU5VoHJ7c2GQyYoUq3 zDm-yT2;TttAAKWu8U7uKS!*sbI&AhPz2R_6DuPFhNFr=mQ)3bw~^sXv@XyuVOF)r0?$k7)m5h}zA0 zdZEk0x1zNJQNOeFQ19q5duZ&L2b(tn1E$(2F72I=gwm+XQR`Q2%dn@D&8p`B%`34tUp{NO8_OTSNoPtdE5l#ceyo&2*bJOLS4 zIo#O7vdp6bTWRv8$ExVFAkziqy*sxwQ7s&MM>advs>)*IFG!Mjot}}E87S{A{VOI) zvUz$T9qhq=E^AKhM%V%i@{8gUBZyQ?b4({M9^ap~IyBVnIb8ExZGccT>h@*j@!xH1 zSLTY9oH4<3b1$DFoA{FDR+Y1jP90pfUtpemxPF!XAeJ=ec3f818y?3@%r4d~WLL{z zKD0*$4^7N|r0P8=LaGM%dKE=W*Qz5|jo~B)hW(9c}j24G^k`n{Jaad zLz^Nh&cbV2<;g03#ytO;^F5fit@^|4Rk0ix5B}iO>;nYxRLa>X9>wQzD|K@}fnFw~ z{>HOJ9UiXIdrw!1&VAWEpKbZ0ZC4E^EoO0_UgweH*+0P@lNsf4I$*B)E`b*6GGOTU zRYE<&@6ai3Zf{fh(x?FL;3jwi>v=Wf{L_`%gPq6k4PPFf_r!WUmbyo>d|75n&5j)e zSEk<8MOVUw1$P!LS?jcZgF<9bK%ITFRPM08v6=SD?96#X6KLI4Gf4z|V8g;3lP=7? zqsX2P(!rfPv@5>JpojJ&@I<_P$iVWq@Z3Pql4(#hc;wE+n~--0&%>YbO%`X-3lr0E z0-szQ>xiGXJ9`s$WjU0*@sJ08BgRteIMG1HJ7Z9W*@x~ij(7`rSE6F&;0$7a5(((% ziXVfokFVPha`>5g%eAsh6O4$O@aVkVP77@-m-=lr+F#FUf71-|R=nKWSN>)J<*iHq z1%J;A)7!m?QwMbW;=v-Ly#q!~iwz!9S&w>#%yQ;)5T|^3IBEw3i;X07snlZQ5163x zIP9yF8%^r=KA^e->2R#LWIt}^a~Za~DuP-f6+qs1ytMEDQ5d7WmT6|xSS-o#3V5vJ ze4k;YqSvlUO--c_Fj1Oqfd3}wvVzo;`2fc&bp?jXt8052NS`Sf>0!t-&9#KaG3PA=B!9S0KdsJ>hAxwz=>D*nadhitGRN~W4(QE zOhhnK6i$o*>K1@{NTSonbmK{ypZ7|vZ7d8hgq=zpu+h#R7t=xSZFxc#t4O zQWbos*b-+YFPHFadL$&xu}wDbSU0j|vx@@MxVRD(ZQ)!rCdWTphL1+ip+gKY=v@^z zV+jAOTR|p%A|lVMtZvxv;x~zh3m$(MdOOw2O_WY{9KoS&bC{u|d}zP9z|>x+T8SGh zcYwGq3)6dY*5**~4Ir1%GUQQ;VVaOlif6s(E;atYm;tm<5GU{Q=_J_^#8<`T)T6j~|w*_1R{ zOC*y`ZCls!Ba#qu_zJ?$hj^$4E<^JZ6ZTHfY(L#Qa7N7uRJ#B1LCj9hN%IQ$6(-pU zy*mL<{FV>9mR{AEn^N$JofAxCV~vOI`AL|NV;OtZYwC^@L#z`<`_z8rz!i62_z_7d zE!4|{Y>rj-Ynn0W0V4=2q{Km$F|rVQf^IL)5=~e7S@-BI+6&KklBr`un1%tySq*C5 zn2W_mzwSF)3LOdTn57lJ`ls-z%b4yW_=osX5-y z(j~N(s-qtsmq<#7eeV6qCXd9uk*Q}vZcalowh&hBjfgKa%O35#h?`P%!wv?dMTv+e zm0K>w6qfxwH=~T=^>@-EcmB(0tNq;Ov^0HiPBa$Xr6YKnQA%iaaTb{+5_PYW)=(_| zz21o5e-z2V#_*sj{5d$^zl>zb@EJ$uRUwA+72T`EH=27V!fIVHPr5U zzg_CB;kFQZJdfi6DsWx5ZQ-zgn7TGHxF(8@5QNZ~_X7pWIF1Y#4u~N>0f?Ki2~=6| z!{&@Rl{*A*sIVxH=Y?$c>8z}Oco+ysBnGG8}NzMLwAB5NLF*Q$M8bQ!Vl`^jmVDvQ;c~3 zlp}(IR<(4*=LW(e1N8wzT{4?u)hIQ_lMCBUx@`#upcQ?3B2c zF>491OHJT_1HrSk2_+kx6A9Di(xlhrw>I@tQ786{yFI6o9}~C!Bf_EC2s)7R_WYaR zf?w2inmtx%e4hD{0EhvCKh4>U)D1Rq14Kd;zKAl2R4)4L*`^^rcOX7@5e&3BId|1c z{xq}felgSAJ^aG6d7;?QISsAK*-s*MiaryEMEq@*Qw9m6&+*j!L7sFCZskk{^ zDba<@I>DjN;K6mriF%>Gh1%$#djFugUpjoaeH^bt8D^Vzp9@2qefS>aCupLuuHj0D zT`+A7;qW~3+WNc#n{?jAwSLXl+Fz(qcc=qqaWIK;NI(-Yf#p!9JXU2|ZFeA3Z#M}4 zdC2Q<`33M0u+ec#l}_bzA&iS!|9FAIq)S4J&=8$xL3B&H`R(Vb*3E$Hkto9`ywVja zh+cD0LDz^Z3SooRH8gnOcDP6zeJ|Z*AI92NEN+`Bq)vXklAXBOHe;z!ch}u1u>*Km zVKp8j@{1%W*bHCx$tfTG_P|6ThlB+Ev>}53XF>GmON0czf|CDacfLyqxyF=J8?N~L zPZ}R&-BM9dZR{t!sX{ZDCY7a9b6`}>$M-SGVG&y8K-T%}+@d5b+`L?^-+Hy;Smgdv zvfVIVt!Qzx&T+RrK*4P1ma%n~B+i5)MuWgl5Ci9oK}aoq7;T84^ zI$us1${ev*v<U&)@$=jUx+3 ziSMR+d@OXn-)-&BPag2It$|0FDsmC5ezF`5ah7uUZkHX)LY<~4XlligfhD*P99&H; zn(3T0i4JB`YDn7?h|#cp-$l7}0NB^)>ruWmnL+v-k)TW|5nKjs5m(py3G39p%3N0) z_h8O(%w$oAuJjl+#xWQX!{Du>*!&3;t98GRw&`mRQsWuj-(x_CWnK3+6IgY7eUQ~d z*!5)-vvlLOWpJPgi{+y6KfOO9IQYLVf&aO*K!EhN0c5qgV@7g>Wo3MSE>m_hYQff+ zGC-eIS5kZqW>ol7p;bCxt^vbPBUM>3PfWfB6lj7MRCZ_>9v)U+Qd2V{aC33_=GJl# z1gakM1oLFQstSjx`&HE=u2;Js(0V?!Zcqu@h+hIPGt%p+P6bKjZSEMJw;DB>3U+J6 z1-rhs?BYU+CJxP{_67;#{Ti9F_P0XzBb?95lXmU{<~8J`Aj{hJ7J0W0S0vs2vBTqR-J`-Q;l}aH zprHf0&<`!wH@nR)H{If6W8q%rT<&d+LvGMf6IY$jM=_Upn9oD=^HJ0z;~&fDn+56K zoEfVM_;m%T8f-!gpfSxFteOVdW143B@OOZi^02#^>-IBxPhDadU%5P@r#QIdjGO1+ zp{hIPyLRL9;>V<^8CQQLD(oVG-*z$^tt=lTqDeRF(#)tq1@ABf%BpHK-f)*44kIRp z8Klu5a*9aGLyGqh%lt3f_^-iuTk&=6ShWzAJ_3D&4E6O*k}LxQ788bRE#kqH8u?X= zX&`Xs`7UKJi>Q?+lvTM?{jlcK?G%aFioX~Izp!*P(r@*njwoompT)-&3=H7SbXg#D zpQhn38$gPSPGDctv(mDnO;@)Q@!%M#mBu2MHQViv7C&1XXbjnHq`iS@@@+61;Pu#8 zV=NtpxUAwKFkc-;ho8@*GTssei?gogBd-jJvF5T+ix@%5GEBwc^8C`!2ynnNUldco z-@gd?k;$uf=G^#d3mTI3tq8i2rM>} z6yr7;X~Kw*wF~Lhc#4>O>fo9>x{vwneQx5b-xoOrQs=!WUCh7Gh~?jGhu3!Czz`C_~G;P!v_@-KZrz!&yZ zl(lq-FCCzs>~4hz?2g~KmaVC1e#!|3!Yc>{cB6m(Lc=JDd_!f~nee8vsB;^?gV`U` zaH(77{wTlsV{h@YVDt=Fi8Gev$)PV!<__?=?(OKf1SCxf-v)`d-&IIZADc-%55cxH z|6T~XT#&x-Cpg^fepcymc$AXBh7zQ5IYNnvNt}4zo-h_bQ84ceNIlV6xteG#6c#^m zvv-+A>i>p9?Fn35L04grO8X0EP0j(DL+G5N17iE`!?%EVn1~gF>*aT#1wEk@M{dv8 zF+-HFr_kTuZ}$2aJ9}8IKR1L#tYn3^pm4+q@26E#Q{FF2rTwe}vp?}B=gLeAX|PdS z*!;{w=h=@ZEyj)5ZlT$!@jEKv>8NDL|e+))AqS8uZ%EN9sU%n z_w`5F$&eVEAIWlrJGO&!dl1TBd+UGi;vpy~4cM$d&a<*Sa&%hE8}1|0nwEW;RlE7X zsM=DYG&{F*k6)~kgoK16`jqR_)9lWZuPqOw>;36IF=h2@>VfBULz-xtb!6CWkf{5q zNX!Kz7ZEc8+||K#f(YnU3LhX?=$lzY6sKIDJ^&M`CYr%ItheqARW;58Ehg$06E9bm zAfK7b{l@t7sa9h0mm_IAH|(-2RV!ZZMy@6$)TSim;?J->8t0*5SzE)FgW1ADi6Xx*<8>n61miUuZ}Xqihp7kUJs|>jf{ZcJ zB6=&LnFowkwK47#N zl^4xEdHI@Ty}@^K_*;o_D{I~*)iJiE1-e!j@&ZNto}AseBnGYYmWO(uI_}Di;$RVE zzw;$z4CG)-y8>zO9pFtxu*lj`);$l@=xz z$=Q@U`xCPPI^exYoPeWQSr7M$#De?++r4uogb6x>A{FDf;;#Ucy{5V}5~CfPP9VWuONKF}z%WHx+ro#maMV^x&VG{Bs<6UX5$4QGzrbfK79H9vqFxdcKkMA#9 z*k)&|GHL)oc7FvhAm+<0$VvVU0LCaQA9~|YQHS#2NaSC*g#SWSJ}Jms6@i#i!woP` zUN?2AA)JgLtmxSq5+q>1bVD!xOnk!lTkGD976Rc*lF!?ZnhHKzi!~8wv-?S@rF~N@ zpUAe1Mpx`3{f{EB#2$gw|)B*_NfN zzh$Frb%J*`Y6a(lYcqO|3q)zK;*+*)e1%ZBQ5^A-dv4>BpNI-vFP3kQ2G1X)ORs}_71{L4@4gjOOF807yW+B7h^WSa3@&c4ZJZurW3#xLGsRfysfjaXugQ{Sa)_I z*+ojrO}N>PNM6s!CN?DADI2||iu>;HBHO+>N7G^bTb1R*%jS?PK=m?t-^4h08->~>>n?blh173j=zivtTo zc4K0bwh<3|g}H6k)L*(@%f;8TbTGqty0=B)LH&PLp|JxZMM=<$e*$%+;a&7Z`ba6l zaYb3Hm!eSoW`xKv1rxGwY;SI-W36YOG=?u8A|<45J27CF8O7;xxB)mfTkqqJ+8Ipy zZC{xYCyZ2=t6~S|%05&|^#j4USPYx$XEBSG3_Fu~4QI-42(O*@P}JBXiX0)P_5jUdr38NwqDYx)+~ngVV=ayN`mNAQ++eC!*@ zhhc06iXxQvQ<9vs_QJ2$%uKVQGPl9ZD|3a$VPBj@LnbE!NuaaoLx=@0lQH_5 zNiqqcG5-vS6@=z3FpPFfPl#dR&~2xXFjd1^&j?Q+MyYhC@R=#GKhe%p!t9ldI{MQlv&(N6w^41uf zrXQH!#cIDza)2-0(1{eYmX9mmrVA^_y#q2>yanvaoXe=A0Cf-pyvVnJ!k* z+^p9A@h(z-K30DmzM!MP%*=e2@s3idSVUO}tHgdJPJkm$Clai;u?NDubqH21hKV2% z>1P1_S;Ila-IF`Rcp}Dm*#)jUNQ!_ol2sR0SDT{l6CYuTjzzftC$A@a@-`*pA;Aze zok=wC@s)Twl&nKtrTQIKrSE$Pn99jCzD50?z*FkKiK_l@>`Y6TRUDcdhbyg#D|!9{ zNZf6@xI@%ALPR*Y-tB^S+?E@B%PL^IF(tzka7jxv_6MCtgGJ1L_AV*J9apuMC=xQ1 zu#+b_ucAUG-g3Qcc$lU5VSOcw<muxm(TLaz(x9`=|8Ut$n!A>KI zs&MWy&gRo+8aO4yY+D?1QqMCNJx13SQr|_yI(%wBb9+z@dc2OaMf8T$aMEhteDdfu zN7&-MTUE=gj*-jF`hm+G?%1ym^_Cj`CCj1xroRE68Y;#6ne3WyBQl&}00RdS$ngEV zB^P0#Km~L;P8IIyC~pGHZW56ZQ;BJmuP8@*FWI+ZDG4bI?bX@Nxo2!U+~ty1$&N+M zSK9?C+!N%+a2-*5>QLe+u{R+brX>`-!m-)kNkW4?wK{C4KgZ`HFYr#^#r*?m@Qe;R zu?;Y4Ed*dTkO0+Jcl0fD3e7*@eBM%`+@Q%9Av0a5ggCGu%Dc+&Jw9me@+((!u5VyC zQp$2|Sq{s`J~=JFqYX{_B2i5{yV#PlGR0O0rIO)vqgI%~Pna68uivC`f>>i|f6+NI z8@ZoZ;>;B(WI$+{mreNK^;L~u`q3sg@=7v84keHVDoD_T;#fx-8cRMJkRvmyN`2vf za{E0{|7lg%J9R$=%$`i=TUTqIQO|Sdh7RLd%O5$J*0Nx9W|mIt%8E*OI_D&W9}ZNau^|Vaeb#=r?ul)^7Z3uAYaedSsN%|*kk_$lPqiWLkifFWS-Cw@T60+eVv39_}ExuB(A9&YlVXHr+Kkt zMa47YzY3+&iUMG#3Lqop!s_wAeDoU^8^^>TXLMcJmlVqEwH2L2IO+aP>J+v!Z%j#n5G z%zBu4?5765#s~(KV0*9`7gYAH^qG zE~u#lVFm%kfwY#NLMR#InUJdTOZ>J*_8GDfN47@`=Z>8^diCk7unmd~*Yp-dwdl`@LJ?;-zGTV33tR#7l zY9=GP%eRCSO(;aL$LV3ZG8P~f$}%Fzon~B?#@m9hIgO6vU?$1|O2iT~;iDjnfFr-r zHq-?eF~B4o0M9C&JQHfpZ0W*D!=Pa?xTp!)%PCe@t&?2J$RkgFk0!XT6TjfQ+tC-Z z{RKPuA9VT50DAK4Dg2Uzl)M|k>tGq%Gw{!ZA<-ofD=4GiT*STKGch6XMibtX_%(Cu z?Q*bxZVp%ePv=|0z1&w4{SSpp1E1z&N7Prs$8QTW^90ST6R}Lv$()W>5)$gm`#nB{ z&Ees2!FP9eAsKlFp4OgDda)dinih-(w1Ij8y^JR09p?eU*qYUp|Y0{u59B&*c^yoX2{?Z>B+9Z2I$rO+f+= zkCr3NjvT~*?-F9yZcyKWWEouHR`+?t!trO%)wtA6#o7E*MuwiYYHDi7bw3MBX9{H$ z?$qGS_|9HZ9*LhYwr+>edyc}l;F;Nr!9KeiOUU|6Vll+Hstd+wa3CjE+|;2v7tJsZ zld?#QqkShR{{%jHd(zTlwe}2+0;7$#ZzT#MqfN(OLUN$KpblZ%lkD3!|{EDElwc0SGnCQb89M}_`qpZ`1sUZ<<&8ULvolb7rFv=n$yq6W5<0C*fp0%cXpCd9nl9jK zg0i6aiH>!wv_!`zxr0oIT2jJOZf|z$eqlplicjpa>Bfk;j9C-aN6=v&x(8@bFmrR; zTdrjXfGUtDBKV)~$B)>4i{xbt`nh88rgU5-PFjzxU59JP^zpImSbQhoA z`q6cAJ~hGl?{}-bbG#o{VI0<*MI@FlV&X+21eBE6SW&riMRJ#m5jgA;q@<+w80muS zaxjXEMR56goK<0UI!!)EJ_;O?YTEe7%Q<&&CzRqf@pym4ko;4|G)?C+^|Y#sH-F3hU5IVZ?1 z8JvsEmlql?bypHz+WjM{#>1UxeLRc<7u1#K4;aagjqgvck1yZIQx7aE?r;t=eSp{Uz(t zL`4~vYur-OW<$43uI0@|DFTQ_SqT{5CcD*hxy^$l@@(S%^fDr_q}#kX5|QSGvvbx~ zxRqDl8tlrBR z7q$O{dzy77f?diNv|l6vNjpqCj1ydvt`!FDR%_~?u?g#3>qVRI&oxcN){oZFQK_i# zuO(l!Vw1&*xVrPuEu2sABovP)LWa#p9uFR!WP|$W(q! zo}V_tMdhh0AaNdaH}*0@eymTusQlR^2Ui8%@Y`Q8lK4O>U=#q1=9D;D`R|tuZV5OqrqKRJt>^usn zDk#%JxM;aMaGgWef*cGexC5tvD&WE?G2cO8d8$xg>kB&=M1j)Wto7Xwec~83bqe=h zUi?+AtKn2;LZASnEPG|K^V3_jZC%oM5LCI)@INrfS@=X|bJMYo8jlk}^flmluTxE0+)V zn%MqJ|I4qJigI_|4^1%OpcI^(8is}kpB?hKq9O?Jw63GU4Qy*fr5>E-cxt)|>Ih4I z>L*E*Z*0Xx&-#UJ!%U%NxDOFCGOpt1a~q$sxsVVGE+tH1MW|2_`sj7CVkWPb~#1s_2X3cnvpa`$p3&vs+S##!rkI%1{h(2Xk=! zgX|JE;Hfj9}U2!tN_x`=*xFaPX(g3M1 zJQNIsg#4&zXlA@;VWz&7Khq1**@7iRVIg`CdTR;Ad~BcWT+R`^$Qg%}0uGxeOxsiN zdK1P=4Pfq8?BP#F!q4^0Alp)vvp?rAELAY#TU7zGohBUms%&bSWP^I)Zrysf zPCioA8um{~p~D?Wu7Ky*sJvVwlK4A$=PwV@Gxa}bE1|@7Oil}`mKVKlPSJ9MHmfS! zwu22>sZ{9>2z?GX%%tRT!)@f&LFX7wX;Ny053?g?pcDr4;jf5Qn`6!1TyZIZEZ@v;j6c zT7T(*R2ZeRYK|Yi(r-aP`AechYqDo;1hI(tzaQNRxB`@~7A|oq$M8 zxL;^ggj6K&`^oJh1NyTV;!LmW(=i(>n6z@si`r6hEP+tDA}cdv?L^V~^|%5VY+yue z0*}K!M`MY|^ce4~^(PHodoYnHAvk&TImIYH_8(|tcN~y$X!jm&h!n=#Q&FDjshi^k zKN5axw~kg$Xc5s}KBFgoy0?wbis20H8UtWcMBb4VW|Y~u&y$@MX#4%a`FLyL4a`fd zlI3`D$`1?5)ILI>h%`NsszL}y7(>`il8D41Z+aID>k-k>8aNp`EBB3e;f$^N0Jrjd zSJ>_B9W3itO!XUIbUbV?M6*7#LeHbX6@?AN!^YGgNDKOZG+* zXLt-E*`o|2CQ@!NEjU*fxs{t`&;Zt(K#HWK3WaD;CLsa@8|qe}Z1~>n>~Dnqo_&}a z(N=zxc;d23Vg-ZqFf&KF*Lg;H!1z5((It@NWaHehvlaqyM)9Wnr<#G~l1^vaVPn+1 zL@8$_{MI`?^)lVuO8=_df+m}~e1d;So4@;fd5WTzBfmkuZxqJGw`SH4WeA+$mkBF@pE#r|d-G*M@wENX$w*JtFuYhvj7v&tV*QB$%lA1= z;P;EKTEy81D0KpyMu0EQAeN6COf@nSFlQfIgVP#LjZHe*c4qm*55F zVg4BK7az@s4`1ZV#qY?UKN3mE`&ra}6<+in&?=#WKO=(9rib*~ldO3X>OV&`^_eXr}7 zHyvn^R5@{-&d6nad$MB2GN%zCLDHO(mO$53Pv~}ZnTqc^DM_yJ_Yt~`5cqRD9Pm)2 z-RM2dkY|q*KW=*c(0VBn0bD)iTk&$3Tnhu9cM&rw_kI*f_6Y#dBMgUV2)xIcDEb!i z;U2THSQL_;8z{p{LP463@#i;r!fsKGdw2yEaolo?-kOj597IYzfd}&RH>tZqqqAG+ zLD8PN;gp)IB9ugg1yc>|vrO+|CiQr2fhFkwtx)?lhGuk<}GE=PMkj`f&@}}PMi&^1d)>foBK>W5^(G66GvQJ^tN@>J}TxA z(_?_V59~a=4cpX_^Ju6cJVJ^fZ7x3Eg1CqrmE>c7#Y7pW)K`;=v=mu1mNig+Kgs)S z$QtpsFZK@^G&UO~wk=3;7JJj9{Fe$fI>(HON}g{qhi52_PdgW*g74q$3Cwefljg0$ zF1_ev1wtmG+$;@pM;__F4)H~10T?IU2x_LriOHS2+@%F=%~fSAMl)%A1*{%}d_c?; zlyV8t*V<+?!asWg`^zualO1H}zaXbD0ZmuH^cVp(=!4}nfMvog$4nvjP_o7qZ^B-w&L}H`6BtexLy;bFsrZ~oG1}9}Ejs&Dw2xSps zwNnEnCqsAf?2%0bu|nkWMSJQo^hfc90XU7%vc(D_u*%f@8V3`n1>X%z`bQFkOQKU^ z(k^8btf2Q6{*)${SsVb;MHSQIz(kb(;ocx;10`o71U5i;?|pKr?$;^P&Wy-!-m&~Y z-Vk0RP+nUO=Vk_BO_Z2TD4hA(9?(zmfFk8}hvuyo+@RBni&CBY9rJNa-p{N#TEWEW z1Nq-D&)LmbT}XcWT?2AqYNi$aS+I-k2~fo>l->tyX^Yn*(wMaVY?N|6=4V-EO0U;Z zKlBRi{cnZce{<~KC}<#ClFOzpQEKzs{hrBK4yTeJll{8!GNP&ioH@t<93+gEgRk-H zibrEU(y=t&6Gdx`0eGF36;r*vq8)9Ct5rg5AdHNiy)U!|2EUM z=_kl1*5L5jL-{*q-h!s*LhS@ks@IPAd2jH5}E|}NXqweAu61N8cr{mZmp~$ zt5pw_xI8xXHZVGxj5`=O8Yfg=Na$w?T4Sr|Vw>Z#brFg{OfnOYS)zC+5u2RIIDK7N zP=R?f*$?&i=leG?AB~76EF{<4?MyLTdHGIR)StNPCE{*bRNoy6KOSwu8sElmcT9TX zM5y+Q5DXR5wq@xN6FH3BDWsG3| zo;Z!YB?bWo^$JUK6fA(}NBCh(^Q&{lvVyJNS~Ue5u+`QpnlH+o@Ge%^GJ-xgx#{?( zc;L1tr+iVJMZN}BSIQKg6t z+(d5EAc+f1=h5dMy0eZ-);+gNFmr)XXWb|t@3nB<-TiT!300;{yKa;h-Cq z57l1JId?LFL-4tL=>34~K-T`JlMM@+Lcy*#t}LTStzO5qraTwvrQcdp*%dusCkeOl zB9PZY@_+jJ%CI)OtlbuGODPn0cPPc(iaUV<#X^g_TX4!-pt!rcQ`~|T4ek`T;1I01 zopjFmX5RA+Gx@cDBv-cF`(A6^Ydz~hc)jtfUYMmne#-G3`wNlxG2MT)bmu8Z*1J(B zIVKTuS`7NFl`b!r1e(!WsW}eMAnt9E-vRUUW3)6&va94sKY=pk6@-MA*12`34&PXQ zP#atek~||{^?lin|KH|zPv&308FG7HJn-VJ|HQ-`yew`s9Z~HN*7j9aVgS>Hq5P9p zqH)%YDUG=7u#^%G)xhQh*r-)135dO5LM*3~VlR2TiZ(2Rw;CO_%%he-+-E$*%_^+4 z-Dy}DZ(K~DAOb-lw62*R_JujtiHndBFcoH0MV6AbYXDW%CFX ztDl>V8@Yc;P-oY`hUJ^>SzXAqpz<(c|2>*CCT!lSU<@-#r-*g2`#9;VEegOm?*)R! zclWnZWOh57`=l-3AALP9$;hQZM{n)~YZ_?a@2$s_rGO6H6TPOOV6tqjBcn*9&9bR0 z?3=kTpSCQ&pR+ul(y{Wa6>j~cVr7$0L-`!A&!3?Kf1@=`@Br!0bj-IDl$XP>k-N50 zPQg@KBsq$fx$w;KxhCN=p;0jX-I=#{CP)(58Q{usSZotNt=;(5JKoA9aVYa(*NM;8 z0(~FvJjEty*2o&K`F!BT))$ha#nF8cAo-Td`^rxg&z_@+Jww3sdxrGSVTT`%n3$tj z+f0V=)C4VZWO3ed?+HhSw_r6sIVyD|EpRBEtL?-b>D9gW7L8=`3rG;h)pjI>qcs2p z0fddlL!=$wTLmU4`f7EI%4o70Gdg%HMFW_k-pAo4e6E0HND|8t=aqxWmKYY1&ASXT zfQ4q(ju+Rq(u&x#%&TbtlB@LCY2zOb_r7msH{oJq2zURP?rQwx`>Ovv)jzZL_owyq zgLZgiw)&*dDx7B0O1U_63@`1cc6muic;RvPVt!ieK#TO#TP0w|S)S1;ni;0YZKCubuqYrUtt{N21BSe|;hT z7#e!2wHtCHh0(!%Degj9zTQ8^53M?eLQOvddQtDkAz2hAS=bL{GH1Wg=}}G&Q*KSA^T3(=Z@BhXk`rR(Z8Mtnka#B#!)qT^>*jFuoyXBjZ0nG3i0)oP2I&j? z&y3VXoSa=bv@R5YrSA85m9D_2_pb;~OpJr8QYEkTL87$Jj?|XW~{HW>0Y)@FZ z;U1o3Vllq%8za>elWR;*RiM7)+->GCwzigtg{q3^FOp0OF~J=Xi|dr~{R@z>-SK61 zZE+pUFQm)U*+rc>3^ydZXC6$5A`~S{*-; zcHu4Zf#yc0uJkE|Jh3VIwz9%Yq8XcI=Pk{ls54@q+sO?#upDgY|F2E`7ia;DetyO) ztvu1g7o@bFj$hel9p9IS%xeu*o~Efy%NjcGd_SBDP5by!B>$$8e=u#dOB+ z?%KE_ct%KSUPnLfzn16(9-~5xdgzuNQA0m~e9qSXhDJh!JiM>pCc&sYsGwgF`tm9c zyizt~pp3$Hofhm3nqQTP(Mdhtdzdd1PAm|LA`@m+I;RP{pOGPnS*P*K@ zyY)uVSYYK!2H0@sfn9lZIf(Yu?F)YQ9Et1Qo%i?P2_rDs>Nz8D`*eOXyN)WD5g}d| z%Am|v;n`)$H;X8o~RWI}R%SL}2Qk2znbt$!+ccmhY ze4Rr08RtZyYQX5-0;s(mqYA6xx{`2ufEfDj2*PKYGbt{k3$Ctc(hev#9A=gVJSYXb zOchde`~ z(tGwC^2$@3YWk2AetYXUtDSvZmVY+bY`Dx^FD<*d;o{(s*nh&ske`t8*0Nfeog(3k zxu7+W04i8#IbzYEchm5%njw+*Xe-mB>%9*M>z&CYV;m$PjP9+t!o3V#TQZEk3|-B8 z1UKQ%;wcVhZ48q_e`uQzwmi}+BPWclAfMy4p2{*K&V*p~#lga=R~!VL5hMLjor9gEn}tL-@6i4R8_J}gJLqWsB( zwWa(KC=M4JWmS+kifRfIYRM|E%qEtUc<~K&)YPEXxh==FM38spY^kRc^}vV!#pM5> zNBdV46rhLoS64IOv-bNkw-CxzsxkhHGTNWcotrm58ROL|H-i^fx~8BYZfXviOF9Fk zx)F)?KE~iGRq-E$SDw zUN9gu;Mm(H4*LM6Mo7Q-WOQ{k7C}@Z?4!2GmCvI5pSRDxPW2?WgUWJeM{ zTG;Dlf=>q<$6df}y;kkM2x17Us87M>3K=s>=us4zJ=T>0pB??OQYzc&db5-PYI|PwJg!?oKGWq9cPU3d!)ub--7u`d>}NXxsx% z5ANBbKOp%-9mcAp>lihQdJiH{STkNqJojG-l{-T*wK}Y|D&(|%o`cUAS14|vCl=KO z$m(iRY>YEsxUFEJhunG)RB%Yx!mc-fa_?WH`8T7F&oovz9{K7}F>a}y68-*Oqyf-r zwT=3sYs}nD$^KZedl9Aw(CM`By^&aoe{spLG`aJu%E)jWu@C$pgbr->>{TQ)!v?M^hkA>GWuIA)S%Bf^CatC6=U*)`NV0lf=#m=gl z!fV0MFSoZAwzdUbK^POyea8SuWKz$nb8~aGSResL*SZl^j)sKnkfEI~$~sAj;u~t% zl4*cIB$M|OnJ+Am=59iu=KbPqSJN10e$S~!{vX(dY*pJabD7Rco{s? zey5d^GR6>)O6AeywPfpY*E3wZDXL#Rw6I`Rbi8et=hBZC9dpeK_sQx)v{?RjF8>@- zlp*v-f#1Unjt_mCelU%I8VH)bMJq-5%^>e_8Qw8nxj+Uq==m`&%`oIIc2}Ybu8Rr^ zq)mm$e);-gkiwhsMv{vz)2nECmV|c`fy-9371(@^UU8I|8;hS-^apZL$LW1Sp){e( z=UaW0V!5c@mku|5=H$2~d&1aHPN z@0l05ZI2mI2q?XG-_6}S_CGAa8o`@L@uhAhbs=T6#l^EqY>CBua>x5p&=TIdtB0Pq z1K{Cw;`G#Gq9|>{xmWR>aN7dIuc;|x^aecbND?m`!pI=LR}&j2zpD$**xn%oKYDEP z`G(|*_V&M_K@8nOv&3?Trpa>tVwhkMtf358;HjZ?8Biuvv|29DPUB-cESp!^sNpp3 zT-=?sn0@pT%T1I1Sz6?={8zNUJkQ~NYKK+Uha9@)m7k^kTijYFFP`!K2VDI^HZGz) zD90#6!9U#q69meC@*(8?Nn9?%YA>iqil|jL_s1tznQqwWO4_Z-l|hx=PjOT zF%J-!T`51d(v3M(bt(|na$WRXCoZo@-CF7HpsGChuNxY5GnUw`zL@7_qi-ouS8oYI>G@@R zv6;F2OMK;VAFlZF`*bVvdAj31m*IS-4Y~Es076`z_7c6B{&JM<8ovLZwP=NnDP>rC{O z4D?g%7Dw8Tz^A+eO6F00Ic5D3Y2J@a0y-;i88%k1=$kQ`@WlHsDv%w}pR$HI;vHex zk`e~gt&waFsCMWQ^ebCD@3yCdgTG~q+kWtZsv>@#2js`+T*Hph>zrr~Tb*gX?&tmQD8WFvWrX{#6?kyBY>~=?SWt%ksV> z?wznIf#5&+>Tiw+MF_I-9ZkqBxtxCH&vdNVZ^7DO60M7!ukW(Iwo&H=MBLIWA{6vlc4rt-_u{)Zo&)W-V!dOw{0cH6kgFa z^eM1`9~fD$PFuN{weFG$2A6pgCS`SV4`y|jccF5!oNVb;n>@eiJT^uKixdb69BTlwU~=jp4$N$!vyB+(s7YVo z;^W$b`V=2aqTY47T&+uA38cv?yzJ;c33)0~5+R%~$+>SNafqh7Hs`3&_*5ce4NB@9vsAk(i?sGVB^85bV zq?|_hmzgbcUy4-*b&BEDJi=fu#JF@JX^_ZhVVL6J$XEfo+D(uZYDK7S6RfH?!P4d`V5yK>&T|qn(rT|Uycm?p=rO>iSJAZIlJhjFOra&QCl~G7 z=ecq5^Itd3&(G`roElwdOg9Udm*4j0Ts23b=sRt)N2h+XCk=%Gxw+#rnoZvI%(GU< za-cvwrP0UHC{v!x0K^VbLqQ@}X&@zQ6F`J&%BEyD2WAQMUVt^aOubE(zqkvbVH!54 z7!c7$Z?|lcDv4U2>~FQb2#D*_+d}P;x@NOaO^=>(S`L0{L&dTlkpyyC?`MOaPdZwF zuIs`JQ<&SF`Wn*8%gfi?E2M3q$iTu=CWPn!P>&;qKci9L1bI7%kd~GY!D%$OBO7a6 zkds>NBRxU_nG$6X!I)11x-7K;2Ll>JsufRAyAA^+hs5GW4-~IqgAM&QvueO=(wlsl$Xd?A##+vM>&N<}2Ez3m zee*DvH)l7)&QZfrGiJY{-Ty5%_HB{rN9>pWuCx&k8{r=FlN4>p7D!mbSvd=DWn|J~ zcQ(5BEf%7fi+Jb)yh6|@n4lU`=#8A1jB4b=w?pMc6~h8a7u^PmFP3BdZ+g7Hx5YvQ zy>2AumZ#wFp~qoN;W-_PiIbz; zpck2H7DWv@8Mt$ihFH0+hurZ{GQn69o<}#9jkmk^NzCRS`U>NU((}IPkcLl1sD=v+ zG#2Bx>$RuLwiGVqdes@5^h7#wO!{UfCT{Oa;W9iOPcf~&iT$fKd{khY{2?DiM(T| z_`nN~dbKF3_j%u8z!o5YH963YB>SYC7~xyDzQ7|r2jh~FT^^PN;i-;pP@?NcUNv}4 zJXfVRsWvVfw>m<(n;uc)rI%adu)a|7mG?WXPK9konjHbCS9a<52q zp*`g0(AJVGJ;Jcs9Wr^leU9Hcapz`gIvCz!&^NLrBhxZ1>ja zK=apAS-@ZYRv3N}XS=lpd9{Y+B}JEwdZE0bWJXWQPxo*I(wH2wz>=UPBytqcIX#@l z9qQN{kq>XQgqqpw!-m9y#~e;19MY|7#eSfJM9vAv%R6PCiW+?G^R^;1PJ!0faZnLi z>ZM!0gPo1|+z_fC{h_{Hp3*ZLZArx+twu2u84AyhLdKWC^zwwWE@UdK)zm$PHkJ7| zf$DL5#_#KzTDe%3#ZQF3U4FSFkUuBQmMkg6IXp&2r7+**ni!5nD{2|*$MrCi_ z=`~^&x6onKKVh!_(#Ya0cspZ?YHCDsFgN6%x!bAIV06oKbJ8XHQsmox4O(|k51~ZG z@TyS(J5AswL_Yd2Q4Y}So-j0rhK{N6p6(_|bzI0R>3~h*v2%+iE zrPD8ZlyWU)A`VO~Z^;HMkJ3z9tuWPTkgc@Y=&;#*ql? z0)EmX)zr5NN59yP24(El0!7#U8gdEAI5kSNveh-vjA;I+Zv8h1iK!rcz~9}K+ea)5 zs~A(^$1SJ1s-ja&lWfA*S0uM7K^F=4&XEW^#;P<#JK2&U1094^ZS*|~2HpM?IJ+JH zMM5@*KujYwS50f2=)pQIfbBHyy{u0S9IU54fRvQgYzn-mI3 zPgP`@F=geicz-3V<+~~V$b#B78Fm+P^_JVWA(3k!b9X}FA-RRfwDlvI-btVp#-_WQ z!{^$7?IoUy!4JZBYwl*8+eSVE`jyaSO(xCF0NPOeBk1;o$0y_v@xQrK|1Ath3AoF ztNDD-97)HpNhu+E?BFc+&7j0khH$mbn{dk&Da}@D2J-3}8EL;5&%trwrqM^od6ock zvqeSAD)a>84Uqd~ZGs!Z^L~`U_YWCRv;p%Kx+5&Y`12X#`Smmf?0KGXv{6dS!L|(P zFi1fG58LaIx{2PB4d>eI@Y}3$+Klts9)a+bxX}WvP8nDSeI9dsTDW%cLQzaWL^&}noa4H@(@2ZYj$zah)&eAg{+txygkYmO)0Bw|AO!*A-XeD5AYrVF|FgoP z{Wxwp&FO+(NMD17ub77p9>`02eLZRIgoC2^>^`-O6H?Bn%o)=uqksKL%LPNEBwn*e zM_bi?^ZAoTlH|Fu?8Qk7rgjWT+~Pp7Gxpa~5g%{_>|ulydQQ~k7Emd* z`+x;=``mx7p{1_@?@I1#RAdO?-vD3{Mso7KP3$29T=U{>SjBf$qB@9;?^><*jrW%e z6lZXf?H^b`H3WhHp>|5jRzMl{lq;HCG@ZAHk;#&%q#NdOEg{uQx>rJMX_VLB&(the zvy46ZltVh1LMFNagDR58(Mr5J{c74DYgLiR zscGV@gz$VQpGYi;+5I_2;&QS$++<65gM0Gv{>z5dbve1N-TD2B!uIx4$Kze|UzZN_ zh!Pc`ugQ_{$5Ji|K#8pVtBPZK8RsT=29GFx^MRp3SQ06S#gPCK9;=4#aex9Hsf%$4 zs=)sG*n+v`phpIvMIQyfhdOgTdp?N7Xj5480U=LvjUhj$WXhP7YIKmVJXr5^y?9w3 z?#mQ(l{PGt%Mbky`@+l%6S;o>H8KWO^a*A}fA3#31PuM{M$t~&RBcF%y4{OtVrw(tjdb}$;&)X)oL++TKlmtFp3nEvO&I?QH?kab}Rl?GaoV>bTrz)PjpTK)-be@6-7od&kn;nFtsx$&lvCxqC!H97YPk8t1S>?-HszG9V=0I0lIeMS z-BIW?nzP53@f|wbN87{5H+vBzD$Y(%F&4d7yk9WQnH~p!CUY!b>6z_wQl0)O&XN4P z-#{g9!QPcaZ6V~Gdva>VO%93HBquj(jj*AIhS&7H+HZY~jSKwMFUy&yMb=v4;+-QY z+rYL+eqU2R%aTjza(aDURq}dL>43#T%UboXGVOh;^XqZ8f6F|=8Qi`NE_4YQG&wE0 z)w9SoEDW#2+_{xhR*K5J_V&4^tvJyK^7l=VVo?#`JFd}7Qb z2g2g9x}s5>j&343KRf$Fcam7k&v1^0ZLd*LX~V;QX)cVJIkw&W=9ThS^7Y4 zQ!5cxm%wW&w0{v{`!yObBS28oSoPG`&;qcr`RgzyBqVemXPd+!qlOZ+X5%@@a9XQ* z3gw^fWz1KtTws@lZ9J(yqf@{$jS^QJ8r}{sP}5yqYz#ZUGBeWB*6r`YbJP#!lfOw9 z>VYGV6GMiLwcke^1g9KeykY1$og*}Zg2DVN905)R zH4WB};DZ9p`bF>Q(YuF+sqr<~1J%fdY=?~I{85suZhGR_OygWFy(!aoh-I@bNlFq( zSCUMME>FI0&)iLWu4JE&`P?#B=*Yo%B8RDcjf!>INYFHCR@Et-N=3=gGnr#kiqx>0SWq!2N&YLC}3^I=5$$pZ=Pai$wemDj(Vo_Xax z(lzp9{eTh8*RB@XpPuZ-Y|fDX-qeMJ(+jGd_h4$NcE{kCz=*= zYv!e(re{!pFlbu!!{%{&x_uJ1WEMvCM)x+eFwp(0Beg_KZp5J_Ix7A7!Zp1DL2#NJ zdzdau3e0?Az0th+XN_|M+49!fS#Kfjb&ASSqf}hylW`(S2s~x=#XS~p;E}j5+Y*0K`QxADS4@Lsn`ZFIbim#^c}NlL@6z9hj46nE#sRNTN& z;>g<`imIUj`lzuhN2c7GHkg4X<;#uneju0zm3%4Wtd+SDCk-Klourx{+zieOb;n^m)Ae1JEqrws2s*C$!XK-Um{mfek5D z+C1&@@K8}11mO~!eF%}W!Er2&n6y>@PXA?lBs71Hqe#p;@)yT^K_hc1%gffaNeh3z zK408i-3yHnJLQK=zTPKyF-ISsyL{g&p5frxBwgy$y@EW1Ve2@QGKm&3B?{f4TV0*jH4X` zH??G&e$dPo9&IR7!R1W50Vo{6m(!_TqnJn`TR}p78|3FF$%fH zQuFxcO``WUf&%*p(YfL~iC`1}gUS=o4{(iarlH4%>iOlHvYs{E^%K7AMmk2U-2&zuqL`zkZ9E(h$O zZ6r*lf7O-7b|JnHIEBIy?9TlMv|p}E}DD|LQerv>grexLU*+%|xnuzjMCj zI5T%>zT@tC&#lBl0l-sEsz>n@%ZzkSfluM|H~SRjUdKl_TH4!)39?&BfzQG=wIait zpG`>W)LI3?c=)%HM?yZ-BJsT8UUU{Gb{ zaKVj@aJGtytce)hFEA{CrYn!|4S8hrfI8svaDvbaI#MJ$!hJ zhkD!A*$KfiE6z^uleb$d8;-=)zn0XBU%F%ML&q?FxM10ANz>fpFFna7xx5IP5cUi_ ztT9oKw^_F^=>^pTn%&4jkPmh}z^=2?Phg>`yIBcrl>6g(t4f=-FAEJPKMu{hm4v{$ z4WHuEl71(9s+1z3K8cV4B+RG-c()$7Ymg#6Pu{|F=-RSdA=s|;`t@slwr_XtyVa?? z26*oTj$E&tf?Y@e`vPlT`L9Y18arq@?B? z{ew>?ZK)q(M%SEUr2NcXPAMI^xlNJifd3?)19j$O6}*{PSE0sN#HYL+OcMk&*^91M zu`4lUf>ogbD#`k(kK6l?S&u%NbUiMuI1@&x!_twWCp7s55dvm{R96h9O|ZMI+;sTi zY;-y;6XYW^F)^tLfN823eLEsQ3oaP_=nD#ohv}=S{)jNU0royNX4F}dIwUrk zkwmkCq^0wW0uYU+ixiY0)%QX%geUlPzLuY4w>s5SBgv+30xx1t)XTtu@ll(B3sa_@PdIOd~ zU<-fBuc0HKq$1A2mQ+V~8^DJ5e!7<8Fd?VuD9rmd6PmFIxf0dk;@nWm+|oCzw*#=8 z8P%LEvqx5a`8~FD(y$8stJV9<-Qq{D>iyjxBy2i* zKWyYqsx=Tm6Wb4N*$guLnA*~K_iR1L$f?G{ z1CS$!nSHk&Osf(516Aj9vIzT!&t82AQv{Z|k%u6v4$s(c2S?`6TJ6n{h9^C$QJjtpWl@7H#d=+RkZcMd8=DnC2w3$HmlYN8;8x(`D<(i7tU6No{TY_ z{hd&ORQE4(8hvS*ncGIX95N4KSUy*eWLd(Vu5NCOQ`_BKE9#(oJ>`s_*vsgSE#B&0 zc`ECx1_c%7x%|2lo!YTND`6fzwGzfVkq+%h$T|rs z^`%3yVWQPZV9Z2F`gfhB#*aso^Oo)Avv0K?kl3LqoW`Y1ULg#kOEur6ExBtUDKdTK zkkSzSZ^-N5CN3?TE0M(eVj;~q6f@0rSt-JOo=T%9{0cY><^I6XOP5N)@p2ha$g?J6 zxsG_D-X>qh_fBtlh4(*Bm14)pe|+lDe|#4H@MmNRv*<3&f=($# zk<-zsr3EL#Nz0L%_gikZxs_2*C|f~%MVUMJ;p3!u2IGCA>HC-(MuQq{>zYJ7+~s#) zqkEmE^+t4wRi1U%SQmmcgk77i!FM+Dl}#&IbZV_JvM)SpgnIn@`n1Zt9NIuj$i_R3pMQvBl|@V8eT9I*RtvA3Z%?@OOdq;}IVEa|`O zYoc$A{$VA zQV25)d;5L|CH*lhjHT{jWQ(0JA2H}%Cm4P zuMpq+T*;eHn!eUOulj5h3?{& zGA#Da_KTZ$SJNooYW6&<>5Ed!eIkweP^-IT;+HR9M(D8KoGb0EVtcK-iqQAjo{b%r z{>f4FGbC=WX?GaB-=B0jG<71^eR409;3h=6W5KbB&dr`;uL{=8;dN!-yp^)w{0B^i z)#=PX$H&C1B{X)7PV@{l55IK1tw$B5_LZ0rRj6Ob*?#rvm7A`tHjY(8nWpn#yKBO< zZ=>O+C4Xm>k``iDJ^v^Eme#%8s`Skts83n|3Ha4R1F4jVtnp_MY^u5~Y3r%Vo}8Tg z10lpe$NS;`O+6^u?R4_PD!&5uQstQQ6w^ETkRi}4F}{^gL(i${bOj50okhF4F|1w1 z0wZ1vcj@x$RYo4Q*%ls#ElQjEF7=ygngBr6n}45x*he&Ym_=DBXDsI^R=vO?hZW|G z%XV0qp?p*snA^Iv{`1?Bg9jA2!4D+Ez7slexG*>#JAgmbCEe!vwetHtRu}fsc6*SE zVP*}b%;_b~BNalb7eRjv6k>xJg(LPgFr*D{vLN zCjOhNiiFF3XQkbNklT%q3okK@>j$0k%<5n(?|H984kZ3Z^8MTD{bljt;q2GtsG*pT zJB3Rw-x0Nbm)k9g2cW#(gE=ycrlXl`BDTV?x`KqeDq$3&g=yV%S0GR?%uS9mnzYz4 z0h-%6+b!~DMMSzJKiKWQt2JJRndwc~_S$mkaCP$=?U(}Or_uwGL2$Ph>muWAT9vPm zZUXd29!>LEg=fFvi~X_%J81&IN0%T@(gl8O$7Cgd$Zo|(j}&sdriMohCuRF5Ake3m zhFc*NuYB9DfMjAgIZiWj>o&D7H$KVb#NIZs!mc*C@9AJ#79VpZuj}m>1b8gdRvvv} zV75QwdDN=V?sH@DP@m;mPvz~>mcdLkYNcs3(YpHj?g(A`E9s()j4a}t)AAcgln*(y2-4_?UzT;(=n{wT~ ziY~?8unFxEF;lH@HXX$~diivXxchVTxrh!e=dRbhjZM0dzM8YArXx8rj~ndra&q@& zhTEP>lKgg?xSjoHxCa}z6|EQZc-1A}^lULdQ@eKh9YZSoG0 zBe{nJxHY&<%5zW2$S}V!xA7V|U>-)aesYC*e^yv*7kFFedN`kyG9RmvhB#a5Fz4dp zQtL8USnFnotD&W(ehD(EKLh6{w@k6Wr`rW#al;r^A{;dFsJjahuN**5={)olv7aG&FCYvj3<1;Gf8s4qN zeK|!=`W1JV9!_rt0=2ThUP>8SXG=vf8*cAr@vVL?2 zNl$z7Gw^d9kgCQ#>hCc%iX0r~h(Be^p7}p#^rKt=9P8*;G}#DQxA@W}ji2ui3>u`6 zAS>=rJ>|^$iQ)XQ0LW}|&VXt43;@nv;#wnc_;#A%%Lm@4K=S`e)6smzO)n)A@^VV= zIRmOId?5#@PZ8c9WxWv-PWWt+Y(tl)VGHdW*$Y8ki!8f&DqiQ)TG&6A-pxO`2bcuz ze=PI3p#G1~;(~6MLHFaETU!z1_v7#8|9a|JF$q5X`N7;R%ew(zAf<`^RZIER&lNhI z|L|?|e|(!*14oL!<%S#PYSh>~V$=NH@#0&la4q8BjK#l(@9p8DP}om5U9h`Dd`^i$ zVkgA@w?X0byIE+2J=Zs$r*I6{yX)bk54iCG?Oe!H-9M~;lN*D?&S!s`U~M9Q!=o^rk5KOI4iqU3+}@GJobudPUJNb;coY5)p8|27g> ze!{X$i`T;rC(Y-_ef}~}mOo?B!NHH_Nr^w7UR}-2Pjqcdt&!M z{jlT6x;VLUtBU#Lg5i+IghL+x)3|0WyuSzpxWpNR7^{w(KGF0TAg5&wwxRxJ$)D=j xt~^l4qv2bbm9;8XD@=a#WHdv8>5cG69<%F*lezei_YD4#lTwzf`0)Ab{{gyyq#pnP diff --git a/site/img/logos/bk-logo-with-tm.svg b/site/img/logos/bk-logo-with-tm.svg deleted file mode 100644 index ac91efb7b7f..00000000000 --- a/site/img/logos/bk-logo-with-tm.svg +++ /dev/null @@ -1,81 +0,0 @@ - - - - Produced by OmniGraffle 7.4.2 - 2017-09-12 22:15:33 +0000 - - - - - - - - - - - - - - - - - - - - - - - Canvas 1 - - Layer 1 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/site/img/logos/bk-logo.png b/site/img/logos/bk-logo.png deleted file mode 100644 index cd532883f04409572011e165e7d5cee121f6a994..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 130048 zcmZ_018}9!(*~MsoQ-YU*2dV_wryu)Y;13A+qP}n8{4|s|8Mc#TXj=)>YVdt*fZ1p z^wT{jSVl@18WIB%00015R76k?0N|7A$4dnq^y7(a?)CP^8=$?MFh4-~IM(6E7YG{> zReJz{&xpTXfB?yPKfZAhm^e7taM02^J3G@jGtyYw8Pn3Uv$NCEG0-wFP=A!5w*O}3pzA_yWl#8D zBY)ZvG_=>ZGqrIrwYI|h)vm6dwW9+U0l}}1{{H?LvwzRP`w736gpPQNUe`Z z(ET+!Zie5r|Nl$;f7_C_Gd28}_5W(q|E~RipZzY+N&9Q!|3?u2N%?OiP&M=`MGG6I`udfMp$^Q43|qx>yPq1YF(6HRHTCT+v)nX_C~`U$H8i%j4yuc zef8Z4m)m*W!qfcnhS0?Q%0c_HU37y}PiJ5O3@-LwvJrC5c2g5uGTJ~>vIh4N%E0B8 zMw~kQUQWmR)XRbAf)&dNrY&eHpm8U<4X7GDkYQOP3+*WyI-8qG!N?BTn<$d7@z%P5@uI`}_e*0KPS>!>vok0*pmA@;&5yT%LGRE~eWFt4mQWWp zyR*sk(7b+gK@G*qc?}wbD&qHDVKpqF#B}+YobAi8=Y!h)HW$m^RmXYNjd`s))|WAX zaFetc+f+=N<#AA*;H&yh`3#20QTtG!uKM3cU{N`rW#=?c!vQxU|JUV5^@}IQO@iZF zFIcTuNNy+D+egX1eV*XPBS<0SS|zr*X;bdR=RKj<3AYLLv&Ni@WTdUkKb>}pj(&{T z>1AKQ1bKVkZSSR69BD)m2V0C38kLhuBhe_Z_#?(e!<*g%iUsQa>( z{ywyHPNJ#~&IDY-fARc^(R=Nr5lCPoN*u&>liJV`1~yv1=ywOmZjv!1P!v}StTTb=(5 zJ=I5fz}t)Dc+vMBBU}V<&Ur~2QM)#){aTZEZpQ?-mb40NPK%6IFXkx)(VgA`I$Z> z!LhKo^`Yh7a&g?SSjQkuKTt2@?6?uQrYYj<*q%^GyY@0_aS2PLP9O`LT zJeSqp?x~Sju9+e70!J2Zk7YKr!EIq?QrxyTedN=;t^B|9@xK*eLjlz=!2YaWHhM3B zssxMJm79^VJ;VK~U6nmL0;ov3zaXk)`dF(cOVdd0m3K#P)ST_)xn<78mQUZeMPGt~oB0nRGE}3$hVFE?N>5-hbW^Q= z`E#ttJH>yUu)Igpxss+rw&Vg88DTy=K4Rj?cR^}CI*>A<7KtUY-0sFj4K!J29c}q@ z?$>#J?6-QWH$Ajvm9Tvez4?nN{LwG}#OMO?A;vaf_A)PW)(~IFVaRT9kP#svWE|`S zND^CeQq!Fcip`CR+Bkk75vO5o9v{tuz4o z$F@m$7Z?;GEb{HHiiIF4EIC0_-#tXShcwND{xtlSj^BS34`a+cyI>tH9H_3-z#J|! zMGOy=&9CrWB>q(boNk@{-=t+fwxJ`E+_LYXd}8XnDsZHGo9|%GO`~YPtFJ#-ZZ~AM zxYnx5gQWmEn57bez0Q04^%K**WNJQSc@UIfAc3Z z>=aTW$F($?cX;Ro4W`P2*)#7ZCGSy^$PpsMO zxZ+qVPtQd>avN;#&z_M=?_igLKla)2&k2>n`&e`-yrls~MS5i{QCVWtRAya?D#tjY zN&%HDKD;D0fv99dYJl%^Kbyv+J9IZUbt4^57Qbci=acFL5aYK&+TL`QF$SjPaC`o3 zG!;<4Qz(Ics^9fd%s(1Qk&j7twVF6%#1~+MO3o^&P_9qt5(HI*h<>`)j4?va`&HqbW-XKRO6so&O1b}tmo{M7%jX!LC6U` z{`YvusvALyTmM>LFyI8wCh{s`&&zC(#Pn}(H{Xzv6>sCrz;Q z9u)c?`~Gcw0#sGYJ04#bi~5`0Sqii)f3ixlS_g?}B7$KY=%XTJ1H~slo8D?ZQRJO! zfBUl}$1Y;!@u9?c<}RxOqn~q$2xaC~lkO7*8#0WNey>M=$GiG#yd~#e13)&WQQBoo z#Is~tpAl<{jpNFnM*j?%?eUkk1KqfWoxoU8SCGoEIE4Ee%;pOW$uH({(+V*yvKKE( znKna0gHa(TUYF!bs8+i6=JM>7=C}mS%7u5$Rsr)}Sc9V3E=@4*^->-cwu4PQBQotj zo1W?u8xHs(k_+q#vRP@tC6Y9U&jfxCCU6KMRBTPzaB`11pTr31>T$!N7bIHs$dQzg zix=<9wO{pK=2$&9J-@IM>ILb;?1tN6@9=MpcQiHbPkW!F{ZZ96)AXAJCD84AlD1nC zK{&(cMcN<=aQrNZj`0eMndut3Uke);g7QgnS{YrT$-K}uQQF4EqwueX#t zj{RUK1#gj2rr`Ob>}3P@$(~+Gjg=O^9%19UM3JgN0UJF%M5~pcDF-h; zbx2QOc!KD){XOWUh##kBdAlWTA~LTqn@&zwh*;2(C}wyz<^=kacnSm++8n4eRGvOm z&x~Q%ZafR3$xm2p4yLUxo|oe1xpS9utXsH^O)FA^9|zlo6vN0~&q=IMEfhjm(SM{g z`SwdQfo|+*Xq>?43c|;h@%+R}R~gb%I+;YLfb@Y%5Qh^2jh-^<5)_7{YrL8bxisUc zv_4mbUGq{N5U+4M;Pg(;WQ;bx=wmCVLfYPN-;YUVRi!!qyU2IrA26o=0p`H2MPjfp zis4&MN)@x3BDQ}IW!0dDHb&13RX`F^x2qFYZ~USOG{An1$8iGrNQOHV>rTiq$|RuF zq|S6rEDi}o2^mWE3XjL-g?Z9A?H`!L`=PmolrY`>gjz?(Jl{n3B#a2NQ_0%NTzb`! zV1Of;!)Ap2Z1}~wVm7b@vAXB*HTnQ@TJc)&>eE#_p0ZCTcsAVEN0%ZCtNA7140%bI z%!`AzyWc&ZpnJ+`eQtvP3JT^wLCqFx44elye@oyOJ7Q(mFA%TH#WjO1L9auSO6VRh z@@7M6i>pNvmq1eACOsDPBeZytU8yF;NrE3}3YeM5cw^&jM9G}fG>b8zk3R_Mdw;wk zW@aTCa{q-<|2L;v0np*)t@P#B1e~SX1TwrN&2^HxYo0gBU{pG}g-0qIU>LYQA1epb32e4c1<5n1zm zDaZ<-NRc>QP6hy?dBLOd?Zcq>aG?VXmv03rcle!Mc2mkb9&;jTJuI{upeYljHobH@ zrMtF@s8NyanqIWM4%!ReS0w%!?o)q2o*^@^^WB`3yE6}qnRYN^*yCHj05VsS@x8CX z+}Vo^N#8Av21qKaR{~KOy7I;;B7T1Md+A|AR^b)&i`&jF6*i>Ia4lUxwr=}&oUZu= z=4_=#{%BLb8)Y$QcxdNnJ0(Bj=k1Mv^tq-O*+0~E`u#)cobHDAvY4@+T6$Vsx0BMk zEtK(Ct-j3KJrYdP6L^6xqTu^02p0KgQ!LKn^dbl%zHjl1g7D?CK$37Rb*tG>^@V2- z=S9+Zu^^14b(Z)s?wBZ8K+^%1Qi7FHvG+;~lUW_g{i{0EGe8Gzna z{4F2X+>(~}g2>)HPxH0!PZk_CR8^M_c3ELycNWRuRkq-@?;L`o2;z3N2`Ux9iTg7r z3v0-t@M2E3_ZshrUfi zl8{dx_mj-b1gL1XF{m&`wOXuMeEB~UC-|i~7h9Z0Zn7G%Lom~@qN!kiuM(S$=_w7A}i*81>?M0pYW); zzJI^#Axu6x{&K88LKXA1M|*p_+WpL9qt$t*!;tHr%Td`8BFQW!y(hD+dJzQ{)16Kj7EdSX2Zw?(>+aI6`TXh&>jqn;$$DGcwqBM7ArmP6 zJRHT<$?#|4yLEy{m<*waTzDl;9KZ*);K`c{Ui z8e_zoX%N?9d_zl7%jsZ)>*9t{c;{?BX-+#9w!q7WU?>hQ%a$BjBnWo?!%7-_D}#MR zjE*Ox0s#NCyYNtxKbqq48jkIG;U6hUgOoC2iF=_#jh2~30vhkJxpHs$fKKw)gZ8QK z{$;SPq(svwsy(&5WkhkViAjZuHm9kJGPejjH9Uje!+Tz@Fau!scCHN5+>|Z9R90Bo zR24m7M$_;3xiTON`Q-W@^$m7;u^{mx36WHmHoU-@L72d)1h{malpIK>Gt``rYPUFs zm=tkt)+(*Qal%iOxa{&gl<`UX}(ieFk9cY;3^*#^&2V^qm7&kX!=Z-eAr#}geo&;2i;*<>0!yWK(cCE~h zj?2?d(ITl&(xb^E2h%`yLmHXuin@h{_#k5KY=^~Map!|Zi`+YsP2N+aQ((AnQ1Y!+~YEI>6P=#?OUnQ#k_ig@>psV%Y zLat+odZqB5SIb$5*WFw_!Z8k7Z;XNjDF)&Q(d_(G{pdcrqBr3ho^; zP;lwxRpEP((NV>7YZL&@%6qISFbM(-@XxXuRrNx0cGcVR1Ecj?8GGe$IIDKKZbqKt zo#sRSJ;6YXKp2-w=@?cR~Ba z;n*n5OyWRzhy1x-=n|C}l(%Bh<^<>k0Q_NgnCSW-vO^qyHtv}smk~c|adx;*C^rlo zY&6_1r&t%!pd_POFV$KFLC}|U*#$P?9nhq(6c>ChyJ-J*@E^39Z2kq7o7W=~%Z!FA z*W>MyZ?sHMk(r}Jp5kFB%nXF2f?a$lst`-UAgXBR$#iyqfWx7Yk}3~=r4lq2LXOgz zL8OAr%GoP`{&5se#(qxo&>MHgL`3TJbAw)BynN5Ofs7T5DJVJ@-l^|c@(ok{gg z0*>*Y&+hXQAv}_*>)$#+{ofyj2sp z96n>Wu-F$3UM4M5zrt-;x2nLm%{LE$1`E%psRoZ5dqUQ7m!l@X=li>_NdEX$a@uDC zK6_KT6PDKtkAYiV@vQ&x3qA{pAH%4ieXpQ+j??7|2p)Fx`=TE`V}DuU8ix?e;(eUrR%>lxap|VDKvk~`=p(0H(Z&H zOdz!m;xE7lq)V7NEAf*Ng3O2k<*7l#7=V{5h9N|Rt_za9f36n{kHFR!Xuc6iB1r6& zo!|Q*S3jjh^u-t-^L0x=6FpHG>();wMQ3wrYi~yQ z-c@(KnM?Y|+FzDsQ}7pSCCKQBbtTE}wtA=W+9e&@N!B9@lPnfdt6ZX01qztsE7%8h z10@8VanJ=@0w=dz0sLJ(1hzR z(rC8i-3|oC<8cVh8c+MpKq{*!D+_VHj3um$#(&cqyLpm;{*RGyY5O>px32W>sJk8x zd-h{jgBJ|&NCIRDWU6wVrJ15-aLnk~en0Tc6s|fMk$JF!L)GwH*?g)+17&!FVaEVu zFXIkPcR6#v5{>-_o2QC^zEEOG5v>f+CvsA&{|JipHn)FHjm`5*Ge;82=E951 zE%N#i1m6sHYI3Q-!>3@lLyci8ygRYmrN(CHTf#!Z1nhwotQy0ENV`A&{urr^+30pq zUkvNM_nbmWSpt*CoMjWOccYv|GS`_98geP!&g9nXX1G1}*PqY&pi(%p9B?}fZ0S4T zy7^P?;7FrTNSrEB91!C^qx`hm9>!_F$C&Qz4ewK@nwqikizXX0cB~B(w(H^b>`?^B zBuI}07Qj7fC?xt(-gxu0VErm-s4w8L0qg#lp`@swz;cwJg0I_yf@m=)?(=+TsT|R& z8ukx-AohRQC&aI}my!zc6|N$1JA#d9ttOBmWs@mdS)aCKVRPeI3L_0PC5EtIs^fx4 zB-j_&G!UVv3^0z!hpf#Zc!LZU2w2o%6I86#Q1kuybthpvUM`z8zV!v&yT5&r-X9)@ zO!(BcmmhwCK(t)8sPVFtNQ3*BURv>wmRX1ae3zIvA38G+YO~oH?>T@yx2+2b@tH&* zsn0ZYupkvypycD_`zkBuPk9rVPEw@gdll^%g6K=KEoN&e5TIv25^RDQ_yS%lPlc|Y zJ0}kEovbl_*6!w|lNM!LD-<`q<}RDEoX2tW@PNwIYzs(0hMDe?0$D9&tdFeOuK(xy z#C-Wm8(Z0`@E?TOY2Nb@CTL@6O@~u}hq;$mh%@k0+d7o2EVh}3=!aw6qA33sglB5z zCo*M&R{6=z#z)N$I+?Qnvcuy8L$DHybN>sG5Po3iV`hRo!A>o@Y|NY4 za0-X+A$SC^OT8|o^bq7bIR#1P&^wSo{v;vNr8o~BJ!IXk!Qy^G z-kbHo@=Q!xPFN;8PwiPJtrwX`ev_`y(8wu|fxo3-!mHJd4m7z`WE{9zW^z7FC$8UZ z`$Lu3aDGrFj+{%JT`F~y&Pe09Q`Z(#&^rp*yjh#t6c#Ztg`9LXfPz7zC&ZLUMD?kc z`bb!e;fz(X7dFrd(=TeGpvsBC-5Br$wDaC3D7=WMOvH@zkr8O+bN(+a^_FEtJPsyr z{)ugm1ge@3qfd%+7tl7KKY&3m{kJR9>GK!9vQMcMhyJ69BXB=JW(00x#ph{|9#;nh zdRt~B$X+0!NQD?qoZJieA;3O-RAeCv3o^4_Ncz}x48uGqy~=V@6&9s1xZHkVfNhAe zD^gM=e4g0g&^-%ynyCsr9(OUbkn982a!j?kqLG{fZM(T|SKrEnXgOFg@~*BkLs zy(cNk)7k6qXY3amCm}8V;2@jS4~^@X8sy%llMju1VL`v}A_@<6NK0Sno%!86F4HP& zQbsj82#RVdf*XnflOFTwLPn1E(A0wHWs<1HNfpHjb&?kS3SC#QClL z!kLy)Q`_Gt+MC?RW_rzoY9GIRp$OFF$`8Ik18vLHWpzRk={O;2oJWRwU8qpC2`uJD15wcX0fPzEq_9+`!M23|tH@B>7wwcUucu zFv(3B0Ed~vn_yeQp-|W;#QlIoAWWUgRpukSK4D{*o(7&23>b+eX%vt>WOz1!QUN)L zcNL>O6dQzv+D~kPp!yoOMf)K3#I|-IZdpPwd7( zYBGdls1Y`@-T1Jf-Qv3ISkw{u*R%PEfPQF5j&(J~*>1<@BU-d|{p!MqDymUnI5&}K z3(B;TOy3KHq_10q?IhkY4@~`1$z)EO7@nCf(+^aFL3EcO12Os67M|OgI1j4SSmxZ& z%1d{8k3w9I+O3;D#uG{OEeq%~G?rSo2mWAtZgtIO7HX6`(clD9_WOI0$Zg-|f2>m; zl&-Z6zx>-$5%C`^ABR%ET9H?3s3*L`C(2zF;VeBwRwubU{vK{?5Yh;8GeH{wUW14< z7-Bt8C7p`L|EG~F(#u|dWfC4kq$?9~sdyr`B7aAdvcZsmR&WH2*#wt@B5E4(>LhEMa3c1Et`8SC#^@JMVq(x zF7&%rpD*+kQiK2rW}74-5RkU7ZD6KH;3tlOBnXG`1ufCzWz(3^YO@Iu#FV8A&|IJj z45oynR%PAlgZqTXk2u)?EKVv4<1b~O?OtW-fHb!-2mQ@yZTK;k=Et}B-fQD>PqQwM zwNU^4klcLlu}a`j=J+9qt(3iE?DUulXBEuV)_vb4O@zoi*bw0O zLef`7gJUiFMTy2icqpb7Hb$2;9T7cHA0&9kyEL;sPJODh7jYRr$BJm z5i8Ph!1Ll;dsC6~q-Zl1%$={*xxY2-Of+2-6`-tdNOK=*C6vMGmjn_> zLS0ozwELbkvxc3%?)sTKmDRNVMY-ZYQqkT z1)n>mgym^C8U|eEN#8(=`S><@>55Y2+K^DjP|fipAD83^OrY>DX9j|nUWBvFEuUDB{ z=8da7{$@6`qWuzASQ3j_21mAWY}`1kg#aV3&SDOmS&(2jun_(cVE6ZZgHm%ODzxMv z{PYmmTWOfuN&;F-5;M9lW}XmI?%kKMU)8B;V>2%CRpnB84F+`CQ*iT*=gT?rj5pq- zVX(R0R+Fm=s8!wuSIy#Jc%G+XWc&c^v0=->02{gk11IodfqIW=-0DOpoaJ|41Ci&FW`il(Eq!LHzp;v=lMkvUR zbahBvn=MflZzo((+>Y0}fk|upYg7n|;IzF00)!+XMyIshC!K%DPQ$O=MX-l(oj#6dq zDdZ|*cAS9hp8#YpKG;?{0hb}%qhtdat$(dRU zrY>-AmonnID!Lw2MxltCQY?>tW&?O@H1TVoPjxDuw?m3l+O?JLCrJlZ*i7!D=$dXf z#cP!>Ll`>CT5y2@i%g`KBYifnrSu*j7}aQjgyUcMmJRY3h~O7fXB|UZF&4;Dc_4!^ zL|{hf{UmMHyIMVvj?X|QM`wax?+{J^lx`De_kxtlVSf>7vX^9U}8M2i_=<& znGv6-K@RTMR0gH6A4Aq8{U5ZGSx4aR)Rqxi>)&7@^{)U`OtceSMu+D;&ZEUVdv&}l z1-!(hjy;8?3v!z`w+SQ}mS!Nfn(uLCRu?bWxQT2RAHCi8%4lZIdA5ChqWxs$OMw^w z>)YDVic2^VDC5Y~-WVJnies6UQ4;}zONed?wwfHSkQQn!m-qRad`&r7MyKtd9xzxc z*R3RIzuldELwF8B1`OS zt?=p#yZOt}RC3_EMMk?|B;iM~irA^T^&)gfn-=BVc)cavy}xo|=E)?3mpmj^=@;6U zL6gl-bAo^7S$a`0h7=hDD7^?N&_k>WnOdEt)4AOi#^JGF&Jz%p{?JMD;pPixD@|<~ zbA8wlQsk`QDBs!2kvbYQ7}_^h!IVzj$B%A)Q-~j#aM5!S@0e^}E)_sc8t!FQQ$)TH zXg6>W)sS`Yt|G8A0*Vp%b(|2<0 zm5_ym9>bI}z;HF{%*FFDaDOAB)Ly8Eo$ilBHRpDnpV--qH`n|GFqyE0@PepCO5Wgd zmjJc`@PNA6K)NIzL<5cwgCjW5F&u1QEd=yoD!jS05;rrQxmA=wnC*fP_&9m9;+_`i_tO3XCYPfu}N!SjyC2$<)R{ z%+f4MM=uYsf<87x;}gZAqavk34?2E#z&g^?vGs&8bu4S0?1<*cpdn*aqtAcL@Z zOpWQ6cN3Qa;~UNrWs@lKWeXFbdGaF4^v$^-b0rHe0LcwTCZ)dt;|2f{_Kuiw)GtTT ze9n9f_I~kA3g?Rezyz%n`AqI@Xp_t`*0DLhGNJkS(Bn0p7_YeT&~$!wy4WMyJe)x1uT5Q<`?iKP@`5_P%D)C=A z0zv8t`=EQlg}zgKu13!G`=*+T5?LhM5f2FC zWlAOklJD>f@~-qntF6Yn+nIKoeM|ZC@9eTo1N{#fO8Gjs3aBo@?gCxB$bVsLBabJf zz;IzeZXEnbiH3xpRz#QYgcAb(ag*9ViWC6v@dj35-5hU54#PbpRNNBr~(XL zB0Le-?;Bs+ZogIn5nCUGNS%pkHFMSQHgC+9T zSQU>U9|8N4<#0=e%@{1g_v>b20&&=EmLC~0zBnJ#i?y%oPE8BrViA1=%sbL*i}ZGk z(1WNLVkhj7D?EC7xt+Nx?H04>EeGBG#gUPl>jr(}TpC`y=L5l&m z4a_oxFaVUDZ!&&gj?069{wEnGmGk5L&Wm1YEtIb z;7;KQrxC5#mXIJRQHcaLde+Nony4NyOUz(Ct!~$Jy~9~Qk4s*56vrup*dxnK=%+Ua z$cN4@G^eL&f4sUZczIh_TzOnn^7|9vpG-&pp!x4!&gvJ{=RlZwCibo=GkM7XGWo0d zLHYUA2-v#8X=He71)+>0J>bW{4G>nX8MhK z*|&joKYH`0R05ni1i(v;YFm@9(N{^)n=@JUYN0`xtbi$ zlms$BB~E(lFRo%9#4ty=iaeEUu$gY2pQ*QdbUeD9{|=aN|1x&m8w;pwbH2B3ELv>E z3xc+6ihW<<)bwOzv$K$BYyHJ#Fl$3Y=;-x*Kgpul*V2CvTM`Eb_r8cp=r=a+{*<^x z%v~u9o@ZMhkVkkQGZd+4#S2I^ex&122$6IsHcGbKVy`aQv*mXk5T2sIXs}yUFnhZ3 z{19=#0v&*>=9?!!jy!yabRYJ)uf#|i>t)4F+3E}2U!TY%@K@qpp!Ly+rRj2GJFJlZ z_80-guJsw8JFqChQ7^Z79vWX?VQ2_+^6vKn2l8AfvmiTgK2yJk&Jn5bQfcpA#hl2O zRb}5#fl(X4D0L}|%=1T(5u$jY6?7c{)l&WWb#OeLRx7Pp%>o7Ekuas`s$A|b=pOYv z9k%{K6@XCixR7O&lo`nzjS20_RVt0!F~LBd_V2^kVCV;^zm<;DFW<=NKB4_30Ht)z zakE>t?S61$B^tI1%!gPvN+^|2V$=V#+u;ox3FCsJW!#Pq^RS-2 zKBm7o!g{Xy&e*iV1CH;VklM%#*7;o-O`uUHtw6aem1>w1la~*_%1O?U%kSF>uva{7 zf-1a?889G0V$Pm$=}QlXb{3nz1W=T=xgvi2Sfp*T;7lfspnk$GEb(O6XLAJF?}0CR zWltSmug|(G=_`#&{cGjV7pQLchdeIw6ouJjum|2!?h?9F+4NYbbIjZiYYmcTe!+jj zS9g6M(!9I%Y~NlBLJP8&o%?d#^dc1m9^#h?Q3wqjEJp?o&R8M$2 z=!#3NZ93b1_xJh1`4w%&S?^PB3bbBtF*E0mu*%l+ZuBOD^U%=);P37QOQ_S$-3Dqm zfn(uJi^lons}CfH3gW*R)uj80q^k~)P5=^;q<1it8J@CiC=gAD@+oSPKW^G-g-Qq&qWB&`-7pJoXPt*ir_)!D2 z7+q--TN>6P8M4*)$a9XiK()VG69?^L!k{kD`I2_A=LOnre)ymUk^sPnALIv|H%!-$ z?ErlZoh;@-<|jS&2QYvqBH1Dt0-`k^Ou6qk=Axb)-C(qZh^NS`vjCW*D#%`~ zP^&ylodZ&J%?|gDRDe2 z?X#MiVwwPYN!}Rh$z4m~h!Zi7)rNf z6$%*|M|1?llb(zZrT@rheMwRY7i@GGk*uvbD^0gp7+n202PK{i))JY`uP^_u=%w{= zFf?P7LBL<~-dIyWt$BCh;gJ`>kRX9Jo1YCyA5bdZ3PXW{nXk#!{(kCQu{s5gYyBtx zC=t+0W6^1&Wk>CsqvM40P(#-{A0K%Nn?jqe9UiizJ7*$$o^MSErU*77974f+kPv$e zzpN!(1~<0vUJ$?*M-W$Y=~$RSGo8HNZ)KdNg#{& zX5*@%4v!Y+^!twYmpYJ(qs6)vRqM9fl*oZcjVYTL95R+F+d_4OG_OXLhaKq+)+R)o zKMriI<}ZAC+U`71ch!=iS%4upy&IuV)jO3<Cpg~6YdxVbBm$r*3)K-A?z zx2m^~`k(sAt+9BXBHWa@7c9#Ea8I?=AC!4r+5ItdS-sOltBpiUk1zX2)x*PqV9=P1wc?Rlb8TQyAN+-b zEc6v3Jz~$I;AB^inw1KT8Qb%fs_};HMD{I6jq4wS>hwz?IIWi;%9&bPT5@{5o`L0u zpS1}hNX05I2#WmZWUvLv!>yY(zl)b)nzC^vz4DN-AW-Mdz6DnJu9PSCvI9H3)UgtUx81*5S z&P8YXE7+XF&q>pD#^Z!JG#HkMDhOu7aO7+;RYY;Da5Ah_xBju6=l6XW_)g-y$KL-! zvcEEu2ukKUges(ct{2W#sUu;%)GRFeHx@9-o!uXI`ip+jk;jAM!-HmH78?3u3IUf0 zqX~f+Q_zei!js4#b@htXOZS0)QuL{un2j5o5Zlvh+be+GLGBlfywwGxn!PF+GZWMp zguRK{aNPwWjVZ8rofgt89*|g_trI@y1~U5!ZE0oVVsM=AD1a6z^Pt2iup);f?p{sh5dhLe@j%EM zek#eOc|py?;B!k#w&~1*RRGd7D1GYwgapz~Qn=WI^*w0m5M3&YisR6*li0SJAt)Ff z%B1U&_&PM!^E>V#Jb(tg8n{ub6&|lGHfoA`#>7E=$Mf}u<;l8nr%teC^Sd&STC>$9 zVQe^JIB%|EsQX}^huQcVJrV1crFGK|4A0aj&Hp0KCO}Y|TAO3&Y$TvoEp~PV`}tfm zLQEQJqkYz+eN#ULHNg>t_<3^|Gj-^eqF<$9x<5%q^=eS)ne3YtvpJZ7o7O%hH;S3T z3qKU=$WG-}2@|Ni)uM!h@CG-|Ac&O<=4fNYB0KEu?MZi>*Lxk3<9E;m`t~L@XA$Pz&1cH>8FXR+#SwV}B&0q)`A}U3FuXKFm9n?&0Zo)PN zJkteV`KHcRa=+9dI1)`!V-0T!It-_!SVUZJ6t;evR9BecBEw^>TJL<8eu3fn^HJp2 zghXWrq#tV?&j+-pOI3z5;A4q;T~!##pff`W(EkRb)H3uIS5pU7>ZhM|##NU{+^ zpxTMri6~OP;u9Q8EFsitJS9;gazJo{^66lsdmxGYFaUC29c(M?JYFw3*Jt%Q%os-Z)pvGBj(`4r-H>n2;Ll5;~DrP;*gZXCgJfm){{R+7Jy8Rmtn4w?5wl^#{ z;Kn;kfJ0ckZ@x&8lmhu@mLrV0Hf2 zCC&MV@>;)Svjr>5WcK`pE4FbD@D*9USgWuYknB&}U9*C4%oI#$f>T+S7@^GO7$sN|x7FOp3B3+rY+GL$NSuxT^>9mU5{fELd zG5iJk4Xs7o$PYd)EILP5cs=-%Mb_X!+3$6t?=r;38-%(0mp^(i# zl4^ftm^Y!Itf0u{K3q8^Ffu&y9KmJ0T)h{$xnSsRcRF}IX zmDPqF|3I%->5MP56iC+!GU&2m>qJSnfK;}Wp;y{(i`bOyW=`k7`3N;Ipe>duj*v?V zHBun$RIY6+c$S(X+FU_1*z|m?B7TX*ADx|81^g7r4DU_;u>^}XV{nMljGg3Rk_U9C z-$SY3GkBL&DQcOp%%^{LLszmC$9rc^xjyAGhrWj*C7m$G#`s3K>E{dn^hf_vmQi== zbxf`8q1!pRl~Yyunsj^pwh`IRGR7i!hNj!{#TB;IJ?{Nvz)Jhl+2Q_ojGDJZ2XBL- z_<}%<8ORQ{Viwc$Y4bDdox*aECUh`f7ZWmml4C%y3LIE0ly^4Id^PmvE(nzucubgY zn~_3`lQA!;hkaM(sSjO>TAUE_-bCXz{6t?G@e(Cffav%=)2XUx#v+NC{QF0x)_}1+ z^|~q9*Fv+c8s0Z4*I)({+40IZCZlI)Jf8#2V_p053!DO=XvYk>>_xfy3Lj3rT#p~r z4>$iAJF+E;tRS6ouS^*nv8xmRIUoIC;5P$eo9HvpV)4z z_m1tQR<&~})BcBquOR#tIupKH5j!&7ab+Ebe$gzJ^_7F@OVP=+f!OwCvCR>obH?E* zqF!rA7n}ll+aIZtLeL1pG)<5;uC@8`&;`cR8HM|@6DlSdAd2w4ZnT2EO0SQMhPtbi zyYVO(qXt+bbm0YDWG6ZlR3loAR!+J~<)m)As>ysj^{zkt3NvsIC?FhXTE+RI>U4sO z3r#B-e>VNX8`(4g!3lk2>c;R&J#Zp)(lzGzZyMbi+Qoo>1?H_KsqNCPGZkD_AbA_{Thlcsu|Er8|aWtXQxE%m&k<5D}wAi)0|dYVEtVRM-WqFYO5qEywhGc_43GO2r58@u<`$+^%Yn7Are7WHdT<*)D%&>0PEa4250KpR3YaI!L=8AbTOgusXJ$+<>=v zRKyk$tX~*5zyS{}f>wgr&Qwbag{LzeHiIV9%A!{k2oq<#4;M3_6ZTK5aqhF!3ytF! z^4GAt3W(_G4DBi#=BTQC=B0;z6sI%Pdo0>8$o8OFZZLNn+iGt9*3&Op4Bt>Z6-yh& zLZeyx5jOF`5ixcIh`jh&h4&3g0IG5?wC^E#3Gs<0UO3Qv^ZYmWw%N<@ao1a#YQvI4 zU9tTI$nJeTG=)PD`Rs^-sdiz--OLZt8;;ixKtvIcWD*39SLHL;z8+Y9pDHZvzL*_O z5-vXwb&)7(tu9ED47i)hRz{U7R~T$CqBd|qF_Lz%(d)K|#7&6t<##v$!E6wb{Q~Gq z6&+63aP1E_5tYX6r;l(IQQ3J4&)b}PA`VxBE`K>4FQP@yE)oe8FzHF22B+0~7c`=A z&&mH8FR3bN!Z}Nt<@mBDn9-oVz~k%$^I}kNl7qbKOrUq?*!oj~UT2&SBw9Q~W4?W; zp!GK>_(lA$KtB<&#IN|HCR_Z`L_Y!dSE8ZqlcY;HnChB(3TDwO1t4*k`U4#HlP5Fz z37O|Abyl6?Q+dk|rL5Gtie~uqd@Va!Z}IYCk2~T%TTN0|;=E^odK{DD23inGEW+XTr`e zkWMDtI~M(szRO$lnj81*{l_rpaSjKt6n+plevpc*nUQY<7af)8*FhNzqPp*#zQVG4 z7x)QV=$75sgQ>UvLStEVD%>|ku$A(%JrcT_(+CndKs4c9R=1PV=6iIrq$+#n>F%|5 zv$-`5PAAi4#_j(PQSad1XV-KM$F^rn_xZ|;fB(;AA}UPK(nlQT)QyrRDr%C<`l1dqV9Q^SVe$@PC($J#v9pkx*<*m9e*4%Fc?X+x0*rHv*Fb2a1Fx zgws@yq9`Q5VM?%p(F)Nt>m!PQP5~1vBv1E?wZc&B6=>O6X&_<%WHJ^Ifu2XfVDPn+ z@)Cl0Su{O0eA4wIMGZq+v5Ma0+rI(V6CPL`hUECsL62aKO@Kn}XM37FuR zvGQON#VwAZ7F(4Z*WQK*JMSAE14h35mw&U0-(u?5bi+Lg+9u|zsp4_&Yf5gJwYIwn z^9(@gcR$1C^s&TJ8V)@+0}(?0wm%$qUFK$1MifhpOW&5wGxuG6b4hs(FG)OO?DkoH zXA1is3DbFa%z!Or$JBB%8ntdnJrv7m-bC~i(B$l{!P0;V+ADb~f2LIJ;zQ>7Q&CdH8#%k~zI@RDzLzgH?={Il)A%c@m+=078RVuI8p{Ep<)k5_O z?KH%6v|?N}SrUi)lE511ba;Jk*TGK&r4Nc|d$s0$_o~fnlSNn~C@-Z#twVQ(NQTJ- z!lJ2tJC>NyjDIVRjMu#lzFTX~^9R8-%>9WzdxIO9oLk~%jvuZ`$M&f6+TTlFqUR?4 zDb-oksML}@-=To5`OAJeA7@v|r$MQKdNCald^$V=V~!XTvnGSzyqo`#ZnabA_T_(* zuqOR4IZ-D@Wj0+ZzXM^)r!U7fgAJ$sGH-Ag)wGspEe><*YF~)jOFGcDe>!*t zsu?7f64CVE-50?0G`eZ8^NDpNwMm$ap_mb#89^<#bx-dsoYY{?FlVm-%<5vMYNu z0;cxM%C+xK(F2Gxm)TUQ!*&t2>5E3_fPQ|= z7X-5r@LU~D9&#cGYqBl;5nw%lVJT18k<5x{JDN~axcGeim29<%s{KDA!4>_tibWe` zM}BWQb+OGBC{{CV^V9RyY7K`7MX!x}z?$s^@0h6A`c8-n2~31e8b$@2`Sh)X8;!Ye z8wca`1or->1;|4m&K6xgbqbZ}rUhgdI0Pp`F=Vv9WRH1Of5FfB*_0yd+#9kCa+9Fil!q-&~O#8@l4W*^mB`wh14m!4a7F`eGS6m{PaLXh+Dk@WjO zv(x1RJ*aiW8LM;PRbtB))x9%4o1|^~S!2(q>tj9UL(iX#e=Yp(Ys0_#H9-ai-NI=6 zPO1!_Zt{nJ_3XR3MP#(QE_Z zBJ7Bf)Q&AdWVm5cLzzAGNAKc1d^{#!iehdNg=A#+$6I{d_Iv@PYLwog(Dbw?sR+^m z@q5_?Ze{`&>9@PMl<@Uf|05gGh<^n>h%uDxqt>pKX5)vno5U@b?nrq%z%5R{m6e4& zyQ6i*!4{klIMGGf0KLO(=@cDCk->(oD~GsoEIqJ>s(x$cFwqbAHrbt*-)B*uJ#7dR zX3CoYr6OWiOmQ+4UP!b((R*pZ8ug1z3rA?eP4W+96=9QFjC+cSG3)YWRS>P;rB0S5rplRVUd zK46IYCHMyE+Yp5k9WX|T6GVhmM9CB!&z*A$K3OaVc(`0wU0$DjrtzyAUzH zleY|NHSQ>KL7-ZqF${1Nt5fk^*Pa4A7xIo-IngCLl@87gf0@fMtXrKyv*%c2#9iT}WyGA*h>Frr5MR!w+jXGfPj zFSjgDEK(_@2}6U76fJ^@{8^gAH5@|dOD?xNXY|0vxYOhvC62Wlg@R)yYXQ)KPVC2o z0sTC^6!gd~%kvgo+t=EzsOh+0<*ndx;_E(J#xb_K&r=wz#GUphkVM5rBy7Dbxy;bM zfdy8-|4&|GclP%=yG-)!h-1a#x~?(nxg@OhxuyZ-VzAra^-UoA*(;K3llT|+4{?Mn)j%-ari_60;JUW64U zBN<{11DbLn`gaI%9_hF&zJraGU9SPPM0qv_se{K>wAb>H7~eRC)%HlY9}K=sf8JLq zRL8)IKU>!o47_c!IcOV&P@Tr1bQVLE24cou|ufo;`-|eC4Z$~K+wC-iFH+OWj9^AN1o4t;u$xQB~W_fnSNfJ!{#84+R z#l*QTSqMnRNbbaNG4_X{E{gyrzCf+*QFYay`DmW^HJpCeqd8}(E`Z@djgJE$>R`?b zf+v+^Af;^YP**=O#qNjW(f7W+u1=R1eAl_^xvr$)|F^;5_g|HM$B*K*%T%mZ@0o7M zQ`iCAy}>NcSe_%WwZ8jc1}}p#EVhf#?Pg}5MWZGboGWNP3j7;@87P^+2F5}8F?k7 z{HYC1PzoM)!u(S?Lsci|gT*V)mp?qC_B5E>_R8kFFIE~9{hCSKkGeV37#eL|B=O>yMjbav*?eBc zXE=`-ns01@$8^Yn2n2yCbcykhaeAQBNha&N@D~B+zqtSw`=SN{NLtIqeoBSgkqyGP8H= z8*FPtqow{Fa6P+N4;fnfc^=~Zn6#o<(VhQ4;rrJLsc$ykjN6AYUpu~7uY!0N1=C}8 zYF6KIA00Pl_bCW2cEf=3V`H7OCCE8{j)G_^vFDP6?SRwfgPPX|7%K6py*pu^fD(-%`x{$apKT*xHpYfR0V=1DHA(t= zL6|hdo8mhfhZ*0JHa`Ugu^Z0Q;4Nc+7&w|vwjcafG}XRpz2IOfTa~g9C3xNT$n9B+ zNOb5^SbTpt?Dk*&Uqz?=TCTJ}yF1dMOAcztlYQ;n=0oYTl*+{QEzak6ov4-2cQA5) zTz;DyHpTd3stkhs01+V(xj>`EiqTt-2LHiYYW;p>YG#ump~x8tMbT;Cj7n)}4+Qq{ zQd@>`6~?*39fW68Pr4N}4>3_nj@0S+eWnGsv_SVcVZ#w{J31~Qno&OH(Fu*6w;7b2Tb8SM&NaAF3QwrcsrmW#)+6J)9@Y9!6|^|`k1O} zDumFI%ZeOY*(tt@J7Ycmhf6#pq_`W4?=P>x*1j=_vI9KIBBgF5iM(y)5)= zfJL)aiP0VE2*xTrjA4KmGW|{15AzW}&hU3X{r>;nlEz=rETc>u?^_WZU#rDpHumld zALEbR^gXhHe&9LCAh7x^G#PPm8X09EA+9YGTTbuOfKef)BFKZA7T8_U`=E=`qqhYBbqw_m43rFGw?o8NW$nNn|-IX1xt=1CaG} z(CFuq=L-1(=++6#XSFM-N9EdX?!3`m>DL89oL~R*Vr+^3E%*6zPrcVh`WNXsGA7aI z&gJ^iYS%zcYr{YH)ol+J6U$=`7)&~s8Hg5~- zaH^l7GW9JH`i9c?$6ut$a@$ll)$vp{bU9jN1>h9)=+VozV&?+<=&?H|68NqFkuXvX zTAg6RoW5W;1|)^?Y4b%UCM{$;Q)_cvcC(|D1lWSc$(~pEZ8z(K*{r+IO%Vsm7%&qm z2jT1>5OKqFNR@dU3?b{oE8=Q79J$AnMXMN0|8E_Ur14)AoP@#Ii2>&{;`-A^AjL!V z9>jK+7+x#@8nWYjH7xR8eK1u{$BC|)P16|!qnu`RQ4}?`6?*cFj&DF!`^CaOtTmUu z(3qgD+xRkX5c{UY?SO(^F>^TGC^zShLlv$nL4pnn%5VUiRyAV2Vk2B=yT3#94oEEs zT^3lSt~MBnZLX}qY;nP~WaCxulaEuj!1L#RDyP@mD(>j$C=NhWzGB+FK?@ED9``4# z{WY$j*I{5x(W!vLzoiUjBVRtECZqnt8f?(VYVW*{9i>-(jmQYjAGBmFzV*a=^RuYm zpn1wDwZBXzFgTOBNM`X#jU$D0qg8@y%B9Q&VVL8@7|QB)#i>2WW=IU?3}9fNNMlsM z$Zr6ZedF8F+cun?5&xr@#_aF7@FtQDT!2ZDEk;4C25Jx!3q$f$dvuP9m~CQ)D8%d? zgp9n4+o4f zKA**SYlZR@z5k{N^^%r zT_H`+777j#y-#9TL3})4d7na5x;1CFOBQ9jJ-WC2Rw>G%f81iF&RY8DE3EKvCK$Sz zr!rr58|&RSH0k4hEy`X>PTOdO35^Z{XN4l7k3pX}p!WOt?&I{c*65Xm&f`CDEcNwR zk{q@RQ2V>#XDts1%PrICDM{aMITJY@I7@~{HpcBkT9jz3Pop+2uBlQuz3;+srsRep zJ3R0$tOUrmliG{@$s>l`#wq}1!9cZSAXXsw@p+&WB`juU76fZ~^>2#PdZHkaRY5Nk zdIn?+=&Tf(y&DPZK;*l)gWAc3=6vg^KORG=tSc3R2hmTHVk)fFAKX&%b}6hc)~ZZx z-CU0SY=bV9ZmO=<9Q^}Vw#y8nEG_g|l&O7!k4`G&30vPL$yZhTX8)@$zF)?^u}>MU zZMe1j{)L@^l83C6hyvFL{ZcVtG(r++BFuLDMznAl z*q|r0ml^1#C&(Us5XpivTwly?kj7%@d=QQm<`z@aqkZ3s&N8lCL*wNfWAj>Q0y?`) z{ovm){whD21z1(>$sx>zu};YBAt^|pQ5URaJxVcUmNiA%%jaYM*z(<;gnVWzFB29Kuo(C7<)(x!NTJ;64up>DHSC?yvr>Q={bk+j+cr z{B&OJ;Nw_hx3s!E|Iw|I2lm*xUbIk6OPKNcX0F6oHky*8*28D_R0fK8<_);+ zJpYs9FBNRpz%i|!R<|sWDxNjg|Gr{L0k8S=TEu9@A@~nEbESRVH)ocv*P7dn!^-9R zy)R3Qd;-g$qAb}rN!UJkK98J3czhXQHZl>SR5m_0Y5kuJdU}iBj}p)saTo-Ek|83k zR>(paloCss6yyAy`wV|$FsLCjwza<5Db+a+h1bK^FR1OxvX#Zgn-Ll53QeQZdcc(y z1u0bu0X@e|q){j9vgq0*SwhxO^)9NKi$7!pOh zMhy07Kgh53A(Aw_-7MdtO*ysME;&Cr)aZXdbliaIMe(>Ew<@8Ru3@QN z1;Eosd!U66ynftgnMNxG{7cVV+}K}pGs?r)jdGXwuWh@yrctM>8a4-*LNH4aC*Vot z0$cFeAVW5H25QrQ9bnF{RWEogXMJX=4f;$C_lJv)a=8%k{BzY-i7VQ9_XM_dRK$`Eq} zZ5ep09duQLC(cJYXO6(r8;WyN+K%*k8GS+Qzx+??uI=eZ5%RyM*OmYy$yv z<613r#guGFQ0{*Yw?ZRepq|tc_a%g;a?>!FsePU2;e`u(fEm>T^lO#y20r z*i}l8kVE`RvwyTEg%<_oLCH_7y+Q^U)C$^KvcLM#PCr zvErdW6q$wk^^xUlPe+vLnuB#e;1j!8TM@F$6p&PU%%q&A*y{69ApPM4pF$F7p3-B1 z2poZM)t4bCkx8S1%-%XhXg~nj6 z^O4rW$HXOgi*e23Cqg|D{l9sIJj@r1&O)mJ4Dcr2qSi+4%aDB zh9m0-<@-_Ssk$%Z^~uFhLY1rKhQ^Z!dlPLuWSH<@|#A_r0D1e88bBC88 zAN@U-0|yNLYUyB!VMl3Eb?QmCfJOxfiGz?#5<8j+O%`|P!Th+!CNWxJ@04Lpn z7+?WQr4TF}*Nlu~=lp9^d_bq6{`FGsgN6PBb3=**~g8Smb0oZ5Qb{KG(x6{d&OuuB4&D{TWoErec_FJq*fcIfAuhCObI(9IuB2$evEz0W)0rRS#6 zvpCQ{{cWSqUym3wI@jAjzx# zsGPpm-%8@k-}HMn)1%oqxPG8^gP(waj5i_}#A|~InL~rnwcGaC)=su8A&Ij1D6#M=LMheBtX!nsE67jCFq<5SdR|>F zQ*VZHKE13%{ZmJC9QS!C;c>VD)IXtM$NL)7agp|fd@DI8MWf4jannC9A6BambCg&- z|Dgl3n7@v{ho|c10REsF=WaDZ?Oa#yj<8dBEiu2fx&);}B4lMS{I;KKYmU!mj-rN& zo%uwWEIrBCp#J{A{ENzNQy4^k3dBQuVTuspBX3|NLO>wt%}k^7oC76Jn(Wu=Np?nA+QVSjEDf11IfTPRce59*WmGgFW;Eyl_uFGUX%ChmW%^9G}*rys!M zuy<0lCL{8nuZ_}MV%U}mwsxs$a?z^MEyQ$e@xu%+XRn4zvR$n6UoTh=zCWZiW$dNk z{&N^j3H+aP=ee^vLg)TsaUb@EUrpi2#ICQI0)+$$VPp7b%%U?C_7ypVT&0POl z9e;Vqto{kUa5u2LuDAEH-0xeuOy*_pLQB~39VHGWQi9D3Vd}p6Y~5axb&v5vQ~1p(KE?+*9dgW43yoZ6Nlll^F@dTc zcxl~%xx0tv0U)b(>DrA4A%i~JoP@9lHf|D~raf^18eD>$xoTW-56W^|*%FaKm(As7 zEV|WaZ|VBiuZdJp({cB`4RFU`cAWP04#6e7+Dj z$Gye9jpxhjT%ai5J9W9lucx?NlX5(59+cqvl`&%(+5dI`Sc3cd{H_;Sa*Rc>P?j088~1!@n?LVcgA!2GCvji5=*G?8Y5 zd7Nam&{uSSFBhWF5Ivs9%Qt;2J5x!t);J~o*00Z+EyR9?7)HKxLhvgJ)xiWROfI=uP%^dAs>T+k9mNW4 zjG>XpVJyPTbST*I@IUZL)iH46#03gzhNEX`Z<#jt^hPC3?2J8EZZ=lU2*P4qThVi1 zx+XHZ2ZGSPQF0gr@e`-93h%g4XWQ+Q73~)`!Wx1~nJNo_8DPOn^-hy}b!Z`W?&S#D zbEXn|&qqhqdC2nYH^K_AZH6Xgg+<4~RzY+R*HxBy6{xfw`Fw1Z#^6|V9>VN>1zZ%& z9;Jrpt~1_E$I8Z*rY++blEK!G8FMlxd#_@mctE`;M=9sC}QdhSE)kc6K%Wp}$+ z{N8`cbk_T7yx%@}i^b>-IbeUW;)R%W!?e*=DF6BSUY9`dK0+U}Zk7C;O&j|aOs%xB z(hFPO?=EF$HSL!BqNeyf0Ic*|^Ro$ynbd%zQoUUaG7>3h_$ak4FO8LzzH&o+CaT1wo%?A8;1?erV2DVAXiEUQ8T_z_7Q5A<5uEWf; zW<7PUUs8YD^~Q%{c3Ogs6pz0BQM(Dwhb8#oIxf-Ed0QU$nGV61bHkPl=l^}94H%<% zb~n9WIxM#e9q*wPDT2-0c#X$i;%MqYVkku9gjWEuUL1jo*ygLA%lR;VuPDk|Lh?}} z5Q(hiH1x$pF5X=fQYk6i={#-C+Z1Fty5>LK#9_HvB^D-kv_ZKDIZ%Q{6$LlAeD)dR z9Z(pUP6%4#YeUavJE0vF)Kq)>O{+5k*<1s3kPtJqxs3(C?2bX#%h3_#umvo}Z6B6g zSN51TUX3N?)Zd5SU9Gj>3#BT@X2syfkf9)gmo54o9jJaZy}WSZwJcf-c0SkrN^sQu zd-uL>UoXtwqY->BrnQ>3@2WsTjBTElo8}zC|p#fNaSl$rt9>HZ`SX{ABIBi zHU(LpZMAo8-%NkNNyGVxjoc+mkA{d)vgUFAKqZS!ZDe@8^{hXb_;&6aPx|N7`=+XZ z&||B5fQ2bzyCnN_+iU2&t)}a8;wIm#Tm>(Xg$KfpEvSRwBr)AUQPQx$j@Ly8s zQTLOrhuE!XjyQZf6jb-rtI&KU8bcxzD-@1eg+^c#=Be~)$Vd>|YAEBeG(55-D-sLs zXO`m2MN9T*%@n{BmYSI*q+~LI@L5R&J~GM+T9@aNl}_7_s4SnqxYv4p^xG1)fHV;2 zi7XIfe+?f=up<90BJVjL_5i{Rwh2YsHXFz#Ko!Fp2zPr$`+1u~VWWQEUmr`nxIXEA zspkw&$bSJ%YxD#hVP3}&UGc)I@M*}iq=nKa)aZS8?n>z8pnu5yum05jtv@{v;rpH{ zeqB$~S(_q^AG%0cbGg&CO$T%)`f@rl=2sGT?)#z9n$AsUxf~wd@lr<;gh6##V_}q9 zv0zp3%7wlZCc(TPqgkA4Q*$@OHdL|_s0e+WDJDLSy)X1nlAiM?7IDbfsgnj{;8Jm= z1#UJHjc`irC<5;jHu?-MXt}`ETo{$+7sN9fp*!2@6NCL@olk07ugx|Zme@UAs+*7V zm!GRDSvq|jqe#u+hXVZof;Yo9s1sXB@)*h~$H)i+dE zmNNLr*4he*(rT?bCrOF_H6zum-H)t2d_>XwSUS!KDDcc}Z+ z;<_y=^9qkkaD#XwQ!%qLuL=Td_PEgcAz}Q`@_E%7aQwfDs{gm51{}|iwl%A4#V~3# zRu}00l&sPmxFi$nleg-Y`ep~}ke&<9mgnha>Xo`Mm)A2JoMIlqPso?8g+#T`kaB*%)FyGKLvbjkW-b>auxD*RHBL)vq9EyG{ zLC~xKMX>S~k4TL7O}*d9Y3Ig7W?u^E<=#ru=S-o~Wg7m+OO?mn(ys1vNY3vb?Ja0t z{N6f{2?OEKE@U+q;023w#j=Ob6W&X1lW(Cnng7x@7ESD@^R~WQl0&qDhTpP`Dg&?O z3mDtc3ARc910GyYXbRcUhQ(I3MBipXNmo|>Z9}QBNEuohEM?V!cBo>%5 z_6BMk?>(8i#K)U?#-!=d??)?bLF4yy$^-c<0YgoxUy@N-EG|0&x5Uey zu$C)M;uQo-Rc$Q}Tz;{;z`%;eBc{dKgPq!GkWlp7VQS(=3sd5Pv*edR_1odO zWq54;*dnbxBG~4%wjQ_Oz)>3$6S>9AW)kuwQs`u&TQKH64>cH>rC1ieki1z7@4&vM z!xQ*bHwlY0*mZeuYD(V*cyMO9#r3|QtUNC1x=0s-^=N11c+a*O)`h47ZF#u+)A_{D zIwY9owXpwxiTR8UeVEMB^e4FkL6;-=hWiM4)I~IccV(X@yHOk}B?cLSJVd5W>7o)_ zWyONIa1`4Vr|dDBk-okGxGt|W)$15$bmqBaQnZN3HqbR`Pc9bp;(IIO7>AX4cBGzQ zL##B5Qamat6^GV17qDWR3BJF&h8zlmEJ~04?eMQFhJhJmyc{C==s>?FPy1ij;g=I` zd!6JjP%v#aSPA`*@nr;E{~}bVcsU7wI$qFl4cDG#kc%&Yyv=-53ySwentPz9gTcAw`7#rZWWdOzP@TebdX6yJzQ$^5v#NP+jo|B(TC)vr3~QjbYia{ETZl&|}V zB(ShekcZ|e3Kl%UCuXr}cCwyzUu|{+F0JqJ_;nNnwsA7bJ(M}H2;;Wl1j=?6=Pl>}<1v+vDKM5t}fIyZ0`W?@; zBnB$N2_RO|id91w63%USV$FuQy0+*_%klJEKAzcMz1uEN_5&O*ifB)-k6L+PL`fEu ziLgcoi7o&@ao5l){x*#Lxnp-5Z0PgS^rzYDYtP3<-RCmq$Al=pPL@{0triv{g?4p> zwrtei8p{Y~z8teBwvhQApEeAmhV;Uu{d%3sH2uu(!KaxX#f_EyjHr!U{Tf1O zZ#KMbG~|NZ@WY%u?!Ef1P=yO)s4fgIm&VGG*12)(__x7L@5L;ii2RP+bJgGtY4wNNNkpBEI zdUJSpN>12xS}Pnh<`i^k(!cqyJwRD1^kKg(e?_u~=HdxHb*6>iS8>_vvqd`}y`06( z1)yiya_rC@>SC#^MiA*vmiy>dm4|$nk_jYvlTCpFcUpMJJEVCY&4LOanmRCSv$JNV zDydqQrofj1@Bb~$=mTQ~wosh?kr21r+l^wj(v9&k+LP=8nQQDv-%}v4Vo>vx2_piX zWYM!6Ogyy?VGmbLck?u#QT;R-IdswW`cQkoXTNec8zXRPeQRzjAYb@~)O#QYyD)TC z=^*}Y7ESs#;kw~i>22@EEus65hyGSZAO3O$m5&~_GP;3#|6Co7GL_rhWk3qLt&(m~ z^PnA~0|ABrlCDy*X6RSSqqbU=kXnG5;+f4MU$~<;DO582(tdo< zV0vjSO(`x!Jh$jjueSuDU}4cL3U5J0FZdp8kTSO5Bs19Z!w<7fJ>Ks;G;m8J>j_W+ z5C}&4;34%N6u3$P;vj|;8hY>Q$aCqQrmQ01@Z3`msf-+&t)kCZSZ_2L5Pe0Far;f& zTpPZJs`^{P+8wC(T*Wq>`PiFp+Tt~{IyC7&UVz!~g*!V^=O^oQ?_ji@4@Zj*@tR)= z%$6v{8V;@GaO9TDq$E2GNT4#SGdUs~ZgCjj z4_gzZb0{pxcd}f9I^06wU}4RrgWH8sI1x`v?|Y34rPncWNlBLwiR6+qkqe2GlQT%& zJSB6P_KhspfaM8=VC5O0ity;c(_hNcNiQ{=?onq=v*-OdyN3?7RUQS7hM_A`Ha#AT zn<^+Y>z!(EQ6;&BjSiu##J4DP8v8dFJBPhh4102& zV%f08U&)1=hRo0rXzHjwGwNv_XVACU$I8lc9L~=CEaCR{k~Nfj!3-M%pTz{-jqDho zu@q_KtJPn7oA`LJAun-UgtZ;b?OYn3d0CV8^mFfKhw!)UAok3@*1rNJqx@B%bn|-;YZ-OH zH{LgI-FbM+57jKm2(1RphEEo>>&`_X2CgPwetc_HQKHNs61Iy+a$G`^=7Nw`(gBG@ z4hyY>BQ;(BCH=b$RmfedEQF8@t3rS5rz8^|mY@N^KO5GYu%Y@w5}2QUO#E+*`u z_IP@(OgcO~NJiwuQbxC+&>XfF=MPfFWPQL9f0-jK`Gqmb{hpmsWByt@6J5O>(zVm! z<3&p_z#b-tSrL)PdYM>p*SIAmuGHCVE!RyMKGFFUKqf_9$S&)+d38y4S2Rb_i*9Zu1@wN*{;87W=g_`Zi{a9G}bMy?=W zcLDH#vu&m+sTXd5{`xQUEy3!h6+^I1e*MGXgfl=Q51zv#4wl2as}vd*Ftkw@idx`h z9o`+rZ3UkwRtzLXONSQ0`pJaph&OH4oPBmux+rARZj6d^4ExQ%8%czW8t9uR)qJQ; zBl`7pd*O~>WEC@U6}D1;c=LD!rf{gQP_b>aZoB>m>62NAjc221%3N#Irs?Nw&uv#o0o0s0WwEpBs<0r1Hroa#nk&|RK$-Da##Ye!_j|iX)CBo$UFYNcY(9u zvJf4M*}K2w+OQ9NIQ!3Nf@KC0!%COxLVlH`oxlhfjQARK65&ytBT7-e<>fk01+iH@wgifzY<^ zp`w4z)c0N#{`jcZ5U@ijfxM>301{SiYYj_o5R5 zV5~dVH&rI`l#^WRj!whT%7EwpkxHbeEDUgo6CJ0ZEIU58C0JZsoRg$bhcKoSLel2q zpp}N|P3A!`AlD!hHHG|6pe<@zIPGa={WDE)Bo=^EUBy|Wt>k|+tHtfE_%}@>Hr%L(v0>8N)wE{=B#bLw)W{eCZ zW#VOrLLKOz3ewqTwBx3gh?Ba3c0lLGs>knbOBEwu9KG?lbQ`ce)aLkO^&b6aif5NU z#|93U*}{3r!2};W0P_1wCiNk_fky{k0Kl-3#@6iZuKWws>(j9< zr&P8`1%Kaa9ZhZ;OGXonL#(eW-d3@nbEUR8SjlDmURp9oBf99;y-Y7;gCiA_5E)+z z6~^v{iU!Vrzr5qjH*7E>7`mYbNa;S=oLFW;%z^h7Mi&Dv9Fa(DvMQ(uup%Pq(Jp?7 zK(uLl$$u;j;Y1e!*x_BN!FDsy ze5D$->+PcH+5%CnF@*X74V>`xzzOY}xp_J4rWl`qi+=&WwskLK1@Ux;F!RrI=xEzSxeUwBdooH59!BvWZ`Yn1H8i4RGo&w zeE8BAWPk%1->H#imE+9E^%}g^hil0TSiZ?8xGGZ9j#nkl-Ju5R*qtW;8iEk$p9^gA z(y86~_hCc^zK@!C62`xb_N&pL56&{O;x{nqH)S_9K-*Wa% z=!Gy|*i2zS_^U}1yUFgt2VkkviB&e5681uBi>ml{2z5vO|8@aHUb;nw$Q<6qJ}ArU zLL)-pP(z;Wm18$orywt0T-@WjJ})wsKiaG)k2d_kX7}D1PuRHWd3G>CxFQv>2hE40 zwHjzp*+Y45POs`}d-B2e0W9tR*V>j;A?mtzlvH8ag*$jBNjjv>#51g*$ zwQP#w?}!SsdYew2jht=nSYFTQu(A(;g}7XJNFYoDVkZG_ULa^8f(ojFK1N~bE0iQw zVhaKRv6s^cm*0&?l0izSbK{-q$htq_a&rQs&P#JBX!`>E5RZcobcB^-hg1%Q&R}~h zma(eE91i_WtHmQ2e9{%mcrACy6)6N9x0&l#w5~k*Rm=`8{KX5ucxp7Bx7L-Yuz>Q8 z;uQAG`jeZZ3FW~`RrKx;b{}+K`hsE@4me`|)mr?op(L55q%yN1To{b!-c^eaLQW(% zXF4;_J>bc$4APCO09u_3c?(a$zEUm0?z>!o+l|urLu-%}mb`N4NE}gHcM&BW5Gx9b zW`lg4ky*1bkfvmH`q+g!tss>Y+Y4?KsmCiWY$~W^Y>bmPYDT7CmU5f=O>NY&Q?})Z z-o?_8Y|>6zS$y+~J+=tw(Q5tk5pCy3mu;htz2H&5qx{BUE$S6B9);ztKa<1Gc*OM# zwhVJ3s=!Zzd|REjkKI4?Bzr3V62k4@to^6Pd)Wg@CD(mgd}6x_aM_PquF@kHyt{a2 zHz7gCvYWNm-0HYZM~TBlZgFDV22|S9t@&|&5sU7C64g5}415irg5?s%g@j7FLrZ?) zJiX{7#<)8}(1rA>>>0hFeLQ?}wCe{dIfK%ZFA(~@-Z`hf3k50vNrNXw7Je@Nu7QSg zaFLK64y4(VKCqZ(iK5624=@)H(HR zWWg&A+w%eb1HM!-0=?=E;_j7Gizvuy992MP`35=P!3W**|WJJUDc!}{p z%^IgaU2s>>v3L#s2v*F~m>pDob5-j|Vas!?-=*s{w%9T|BY_jf@QkLf*jEnlTH{DX z9g)&hYEA%o6oZS7|F!$wU)Winq&DLhNE{ham_PL*u$UfS&vIiEGlC$E4sjN*BYyuV z$Z+a|W+9*T=~A9Es{&)RSrUsjGCs|kAaEhmcQNER*cFL2xMH^44puQ6#I|?;E@6H0-XY7ySiPO8+s{VL1de--MODlBiIGW1B7&+J zRRWj$5ny$ew&-eSsWw&fD+ghvTmTlYClWl9&a;u(f+qRZ%^QnGqpYQKaxL)^lo%@`( zPTj`)BT{waPaEnsusQ!TdMfzu3Tz3ipoGg;{C0uPHc*d}8y)iv?SF?_R{J=VnO>Q7 zbEL{)ZBc}2`>#18g&6A7FNLpby&4HzbLm}|J!;i_M#U5>n@HDW*v z6hl~XB3CD5#kjNtAaouf@Tce(e)3l8WypGD`ccI=v08`2h4lfTv-DG1=(#h>hBLdr z1YvHqf1D#woynwt-(7bf<6z%|haRci;Dd%$r<|c0l`8bLn#@i0-J1mw)ZUI3{Kl(2 z*qx05j1!Ns)y{O|C`>Ubjgd4VN;V%Q(WRi*(~4XFwr&TRVYj;m8SH!gSKMkV{}#6n z)n!NQ@-A_JJz=4-EX;I&CNx2H z6*lxy5JmNfQ+)TLp~?1w5A?8U07nrgqH9>)|6zP= zklSf{1XD){mlCCKWM)`S;xd=MUak_XJ|mZ}xm@SEH^#H^wCKSoF}r(2{S(jyXQnIX zzh^d)4N+BXrtNWg4i#1Qg|}YYN?C6Gx^E|y5%m1RE0XX(Y|BjP11iauO|BVd)bW45 zRJZ_###vuh&?6}L4y_>0=vj@|q9z5^@98mJHV*e-e)=PffZE$!EV8Mo(Z2#kYuQ2) zOJ99XxTQVSDIl=N4~ct4g0gIT){G=O82( z&#}|?BkYSX@8=Htonm6HuD8>}vdCDM-1Z)a!(6O*^$5<|5(*ehUT4nnQ4uUbNNHlP2KRNXy@nrrW$Q73^Q2_8J%WK>ij8l+F^3nOjSbUc0? z1DzU#?ZEKizJMSj#_QVH4ppf}@jb$@`WbGIDF~#jd9@xv z_A?EH8z6>FM9J(;E0FnYZ6R1vBVN9nyR3=JJZg)`DUy?ST zJ}DFkzD3b}%`Aa!LGHVy!Pt*W#%~C_c$>tD5B$@3&(RRj1gz~Nm2se_p6nCY;irdZ zWW~)h9A`M#mT52};KNrV5G{y09~)lq7nV3q3QM0c?b*|NlF9FEG-Kdkv8;cdn|;4z z8|7=^ZIq(Yb3zZX1=xmp0Qwb_BP$CB9ilqlwqSmMzV;3MZxB-YTgHn^Wl|2{5lgyo zvBys?&ME=u#12)YXh^Q!*Rb_w6KfJ(vI%0bw$vDu(nd&>ZoRP8T_Caa>>vlM9eo05 zx=agM4>Y!GsU&%Bb_y!Ek40%|R|Ii8i{}JOvx`kIrmuB{GRA#p<7*N{$iE6m8J6Pq z;1*fC(XmXT@(rARF0Z903Qs5a%2p;)av8EezaWE-QiVKF03Tb8Emow=9xln{o?hkX z143CieaGEDfm^y7{ebWN8b!&ZI8`8*`VxCYRvVOP^{ z{hZe7Jio$*LiEnOa)xiwMPyC!NtTD>W8oJZk5_}h71}Qh6wZ^THYfa;m+(HE%s6~t zB*Wq@qoFxN_|{BQ6&Ii3NKA`5LZf>NMRR%~&Mi34N~>y_PT(X$=7>*JP=WjTVOpI* zHslSnp7QsgVbn1y(2<$y>T~8#(J&DgKcM{P+~R8OBnHa5fbYA}k1SIx*TGmc>ewu9 zkGGF7r;L&O*-6sf(M5?bg-wik>b=xjboP5Lc0vXl_`X$}@h{_K|6^lUK7?kfUIN#Q zYMZ!p%m?T5jh)+@S1`Zeh)voT@SM48w(#p~OtaU?zeBP=m@izDW!k%0R26*@i-W-n zE;gJAW#vLnO-mHo*+QQt4?uLs@H^^qg%Af&qB4psnzI)#oYXgSPqGic{@jNQ#~A7g z$v3`@m2M!Sut!+-t7Vgz6T>luW5+QZAB8b-7?3?tl|8xm!|OH!9x96t}rnIeSl>M`*Tdkll$CB_z` z{ODUR323KOfWBZkP`-LiCBpeZcFgt^7%IEgE@1@hIv$-%nGbhYA|3!h1uIugiZ&&*n zo}P3z5rciWruyXxI=16)8>La!oSABGr-5uJ{uckM4K^RGT^W$TBa!a%l=d<_XT3dl zU7^47&+pscpYvJD9BL7qWN{LT&Y6Fsdw5HOo= z3qHUhfWakN;guqbl_KW2ee_;g%T7UAM|2p^n&Zo(^{X0SGhfqkL=JuN@V*wT`jwgPr2x4~}2U{Kbs3qLmr0{y`}hd43!xWUv8bKPi(&J?|a6V;_OcM>I@}E~C?7 zZ0~Zb$DwLDR#807%r0FyA?Pb-j|pe}pS!Rgg{;jhr|CY+df7a=6ceL=XIGuJ50v<9 z`yG6DN2n9t`?~nU^|Vq7%7|_^2W`W;l9z36juk$0tg>-YP&TA=%1GjjA-gqC&y^$1 zzy{Opol`)niftqMaW@pNEhNF^Y~nZ`SPvu&e|6XqV{lG}M zjUY@L2so*$;SiiQ2K%&=;#*-zO`_@LOdvD6%n}@>$d*Zo`F>>-m!`T^0 z2sG#R+RyMvwUmJ3kX)O!3SUTp$fkIkE-3uc)KKQQJ&^74^gxh;khRzh(Q5w}6bjY@ zdEXK$`OE3J9zPCd3ydd->=Se_}03}ztd2pJm9rocBULKdq*Y->jKNkSRP zQw&0V4Hd08%o9GHfwc-&tkBdzrear&_eLZU&KOb)MciA!?*0Qf{E3FKMPbEM)N|!EM(vJ9(sZ?_pB&Hu7RJ+-Hp3b=7D; zic6b&qwNbNBV1w;vsLEVT|Uj^P*_8QWaSCZDAvZC&`@`B&41K{P8GxxIe}|8wg9om zpxWl58&LasM8s|IC8?D9lS9>~@}fvT>$TJ7g|2k)2UGlLLN5*VQ8_|Q!i`f461tr6 zK#JZfV|YTzkp6t){Pa$y&*EIP(m^I(xL+nOQRsPZm>9j)kCHswOJiN-Ng65X#c6_U z)+Xz0yCmV#l}56}c)?DF3eTyO)?IDc@?*FSih<2W&Cl^L_xNlMGdPfn&GA-uU}$6t`&)}sD};G7LrHM+}=T%JHGa_H94t9doCZtMOTH&UkGT? zv;D?GuM0jZ!@Fw`+&2^LCu|?qcue#Go8=m38E$4;md3#Pq6q`st|^SHMtC$Dddp|y zQfm>R7o{dve?LiMFxFZQdP*|7f%`eBEl3jYutDv{d(MdxWvxX~cD}Ns!+i1?d3bs< zG-!Ix5OD#g`Ydn$bY-o{R&T0r3VMiIN_|s9m4H{2&a0YwIl&&eN;`a*Z0B>yL|xah z@w(aIOvizBg(M|XiV}fv3i-Lr6C8K(r5wf`irGFw+{lzlBj)zmv!3pY#&+Z1a zm(5%R&gQi$IzgtfdJ+V^9bfUhCyKgkB;>mSE3}^^bp@NnQy=> z@Zq>mNZ{zc-_@gq1e9S?Fxo3pWrN+^ID3gZ&tn^T(TfzX?V?tfRI@M(N+uiD?ZLtM zNwLfb+Ba}{S#FD_gbs=`-g&DOvR(Lr+fydj7P{TuP zauNpMUnQDI6%r834Zj^ebU6R}b~f32=x+ujcf)_Tqi23M!|6e(T$ zX(NXV>=E%YVu~UEh;e5AwC2FKToupchd$FqfUk z9>PRu`>~FwRv*>BNS!Mq+CmMKBPSH*>#!uf`q8;pQ*Y%Hwur}QNNVu>@UU}{*0pgp zA+Es1xPNzD4*db}*@{>D+j6%`{eAx`x<}p<*gy#1r=8Xg|8>uF2PnZ{mX2RgRr~so z3#_K_7;IAmaN#V0c^-*PxVi~wVWv3!=n!}Ai#N`FNuBp? z;oX}s#=wy!hUhm1R3K^vB82GjAy$EuWj4EGG+vh!Z)#@NQ|e7%ao`GpS=^M4t<|DimL)x+ZQ#BIrF{j+~m$5~=VZ-nmT`a6l&&l5P z<A{MsL{eC?XLGfTtC<*d?D5VBq$Yk2o+NjCCi-z@BCHn?X`m$JPkkTJ^@Uj{= zIbS1z@`d8^HU6PwZ4CCL9*j7kLpBMA^StL*A8&jO913PQN^^!UBw&mftQyM|9=rZ* zwom;JzChWdh3R>pzTpcHleY|TSf??Hp`f%FBsTIh{K=ajdSF=oDG#)Qv zZm4+x)@t_1HbQ>J^lNQ#Du;Kk6cnj<6o$KVBq_{Mx*uwllSmFj)ZQ;XTcD|x9Y1U< zQKy7ggc8!z(NFdWQHWJZwOxAFlmo$}QV!HJv(pb?M{I5%U z?o11hHRD5n?tjA1SuOHA=19c13kE~8@SDII5vHqWkzq^mY_fjxXg_W@>o|ScXu0=i zpc?;Y7|PZBTb4^YYM!+DzC$CI5MOZ}mM_)}i}f-L1qu)v@=<*!v5>bK_}Gn!a!7Gek1(16n?OA+=a8X|n{W<4y>p*0;>6mFfy1P-NeWWz4k zOWixUmwk}JVpfwll#Wy5MuPW&5HC2yw*Aqu$C|xn=F`Ve@C_$J_-!cbn0{@cGN5yJ z;j4FXwp_!z-RC_urj}DR_!jzS#HX*p#eev#un z{Ak`F&-+k9JGidtOtjzFKV|zHz1kNFX;XE3!VrjEl(8n3FHIco0$;wZ011Eksfvdah>_aij~HOE4Ei}8oV zmUoQqzats`JyJxarQqxj6@L^x$HCq^&AZq%Wy_I5j!Z86+6>M`)&g*B0YC zphxx5te7S#G|Y0*)$GK^ob7%KT|rezs1lhP4Q_29=N>vb8tn5CiWzc3j;e{3-U%kX z-F5~@oHrh`T%b8^Z-2c3D~N&@#SV-lCg{k%&J>sw^qXRfA&LbXj@dn8qt>DK%IWRk z=JD(NJ$T|3_4`{67hkJ5kcRy>GHYIkltk)sEQr%Jf8aV<0*2deXQc!5q){UNF$Rg$ zLp%v+h}IRLYOY_HdKj5-Isz_7Z#V_D^H~JNG!RXzbg3%je61F1gd!Ordg`Q1QRe_Q;^rZ!FY4 zcAgmojuX+HIRvKPDjU+Vq+A(G4a5+V7#O9AL0H>3G&K~nn@WOM8@8rm{geC_#U=u+NXbI&5Y%pSt>&9dKFURkZlGRyjD7x<}t+yPWy+p}`$B5C046dN;=s}Tu?v$Tf> zZ{!}6k0rtuwg-#7FJ>i{Q*EV?UdS2@lq#UiPquj2#5~lqs2n&QE3>c8rdK1q%92L zPK5@3wEWAWo3tfxIm2~K8G&WYSQt_Fd=g3hhdyOzQJ9UCL85l> zI-!N65Gc%=peU5rlH={9h0(CVAhaEnJLk}xOZvVoH=Ns+D)~5ylfcci%UiiUolu&G zg}y1~78bhUd&5D8B6yX2r`ev*=lpX}<|lul-OT|0qR`c42QSqr1F_+C#NwdSo%e4A zDOW2t71Q~VlXVuS5u}aR@!p47_J4z2x87gfWA{7WJ_ce9Jm|^9%?DTZ&P--?rFx}u zfp^3euTx`&SHfSQFr3N`(1fonv()Mr0*FV&+zm)$p~vra7AhV2n7roFC|%Tf=yHEo z34Cm@d4%{1kiX^9ob|$wrku9~<^a${?z`Aga7Ki&hC?*p42aNTT}Qb0dU(-N;VrBQ z<*~lLF-l?RaA%Bn1Ondbdb5+8fjrlzsO8*W=Co?eXM0y?LIxHT`3K?50Pg+TVUS&zoA-P(a{V_DsjmKhq3S z){h6vP4^XYl6wq?aQ7iu=SV79;|R~JfYBq*6;RjZsegopz@3jUg^B9p=KYZ*8j~R% zpY})0boGfI7{!l1Ub~D3Zfe^J!q&f75UNR>2NFp=r$|9eLu$wE(p- zWYX^wspqkHCzP{jS|;le78D}$6Kw(vCPhqS$Bd*z`WAfze~KxxOO8&(%zP9}@Tg{`|DB(_3~Moem~#Ou*V`Fr_Awkm1lD<)`LO*K$iRI$%Zdd_pY3L6hv7YjN&eXgD~fK@tM{&Tr5y4 zOX&0ZNb1Fhxv&PpMsGUs8j9yg+h17QpN%nNQK)>uMYw)N)6UvI)PNlMtveX#B4Y+; zy{VbN@F;&nMT#@k2N#Fe;b9;IhG8JV6J|=#jX}a`^ql}@@DQYg2o9^ZB_fRTt;y#4 z5~`rLW!`e(>i4BM2OV&=Cxvl%9nE@DJv+zIC}JP_x7f(y1EZ|pFd@8jKRJyD0)1bS zU)Vx+CuqfNVN31O+A^@;6toK|C$u@Irbq7a^>_BQ65#6j3)iGt_f}uqI8anO9>Mj% zMtkucX>0%Aul85j*EiTu;fI91pioU{K*#P?{lSlqMM`*QnH~)!u)m-9a`dT-Vvy2B zWYfj8Yb~s7RbGAzg4! zfI9(&&bZinpLi;L%T*V~#F`wL#c9K80p$I9lV!h6<^sQ9IQmwKqpVkt!%j1X+UjL@ zIe0ERaDnB5+r#p;m5ZOBOZgwxjYIIaK-jna;29;=_19z6Xns8Hb`{RXcb+~Nefcg_$0n6DXtcFl0qE)pJVYm$fk!hTwNCgSdAqz$ zJ0ZSja;Obes}pdD zA7!7?XS012XQH?(*=AZ>k0Q8Y04cM$lJ2)@kJZQ^-$k9z&8E{ca~%ZbD=UY@lt@F& zt8ZskM6rOGsYBqsoPe))wFz9*5s$qF+it?Od#{xmtU|N;EYYusnFj6Xjw#Ack?}dQ zI=yUJe%(^F({t}bSPHcMXLQk*|I1hkU$O2{xCO(vovb$693Pfs#ZIiy&Ga%xl0x{I zX2QZ`cCtW^Pq)a5@{yw(KssH))nXWLO)?P7C^0{2ZtgPG7^Gc)$_{PB-4~WP{LRfQ z@kWBAji4erU;`z__sCZSa)6xn)X?+2>DH{DF{(xt=60NkAAw3#jE$>8T)%8KMMT)g z$Cy&+pwYt1+1h*e*?k^N{bSLxty?x zIiNtbtTAu6C!*fS#}T6{sV^q{*)D)x$Xbu*EKC$+-Y`8ktY5`w4G+5|(k|P}1^87Z zZ2c{%E&QS>#HJdz0w;>DR#f!c;OzG7Wb6QS??!;gVQt?C4|-8sF)=xmU@yLFA5zU8 zJfuC7=WNbh0R$a{JFcE=gHswU+?96QQ^lPw@W$HU@|iOr6*ix;FsK@v)G$Sldy(Q1p#Nl z>BOu}_!ZSnVq17IWf<6RaVa-vWi3+QFcE>Y2wn8xWw1xV=q*DUA z>jLzde>kW@9+fQa?_f*iOz7;YhsbrHY98L#)`-zi3ZWWNlrzUZ_X+NEHv3CL>kkld z8qW1BdOsqFt6Flr5)C-o(xt31UsN;;{Y=*VLQ9VOAm@=p*jr(o&%QfxnEVpz;+6#( z>3qN2S=){O`vIL)zv=<`%q#;1%ha}^F8VkivEM75GVQwJ1Xlfh*Y%o_7J+~(5j5oX}Qgwf(_PsO#JYcp~H#F7JZ}<+yRS6_lP7qB}V(1s>deR6DUQTy2!kQ~8A^=P3 z7-ASaH|B01J&a*0%bl`VP+pZ~FQsTMx_(cn!L@jd4&hxhY8KiXdyEUq>N0vkf`drz zYGNtt@9=t4@aZ^M9Ap`?{w)*f+91HH(e+y=;HmX(d#onP>XGvTboYQI*s@!M5%!I~ zJ-Z0A#CR=LFLEgB8Ygbq7jjHhJ(9PyS5D{vp|Z9t z)brma-2eo97A|b0>u8gsBN<<}BV1wyyybCAoa-P;&+MrhRzf9>g9N}zfyc?cgLAr* zlUfLo{C|yTr~IFn&KI9164&EKg6j7%fJPv)tBi0fadf6G71b^Hp3gVvL!Q6KjO!H~nC2C-;<))Fe)>^w`)XTv5(tMyQ3lPqm@J?d1PCI>4aL~Q%iJ_@+cf=v z56~|xmWz?xRw80I1KiUOTt;!KLuh)%ByfXWRogi)SRqt%$EVE|YmfQxd98fD8k@ZT zR!8E$Z!gAu*EhU9100L3BX);WzMwhqJ!}wo5c>IUG%u(*zPI@PU>163yIv(eH#7lX z3(hgqO|p~D024UlA@X}UCerQfb+GKs{U}qGc<~AqtO9)!o0_#PRsm|r$N}r=mEmp)dEGb zx|s0^qlPsw87Cy6^LVt>&eZkQ*D-QEKwy_u6$bJ}lpw;&!`qok=753iofs25LU+S$ zW1X97{+aos&{?cvLp02;4F3&HF1Dr#Uh%WSsf7k_$?4WNGa-cUCW@5WrHyQgBa}LG zh&PD`K9|ItF0r^m^#uzx%dL0!CvPWgOec1?NRgO5McxY}1|Jogrn{=4eb1T+`i*i& zFXtur$ycdn=pV(H^$P;BbHObjNw)?wgw!z|)h3qWqhATQ^FA3Oeg*d4DO1qE&VGHB zdfX+(_ysHC;r+xQ{=-W3*|yu4!sw1TqlNCr>HGX=WwP)Yr8vc#P?2cGz>4w(S{SMQ-I1}#UQ7ov z_f3a}rk{b@@dY&47?$IEN=Wn3W)9z^`9_Wjb)Ux`da1NDby0F95t<4uAQ7VSNl zL{k2g=SJM492ZVS<41ajo#6zzntmp{_tRNu2XMOFyR&=>o2jKd+gsk>#amcf^}MTy zDmD)3-3X!H*q+|wL};K3UgUhQm?stfj91 z`KpfSx+*spcUr&}ubPUsYbq`2vwJrmg5E zUL98OP7Yl<8|&$W150r^!1X&_{_*EhV z4&v?%v&1X9J`mB$PP>6H;E5Mjea5&*5)L0@3lJ6?>UVcB6CnVuie)wi!Df~>9(@s? zCo$0e5AW8Xpef46`s42?ZrijkV1(RKJeeftDHK3@r^HZve_8RoAqC*(O|5O^q7@97 zlc@0iuVhkren~X)7O1LB`CID{(6F^s@9{AUiQ4OLIk?6QHr<=^lVSp%o4JI{l*>QP zHzs@{~TD z#+!?K6xv){+p(~2ed_S2Z;?C+RB!HAPsneU5hUU)so~RIE=%_NURfU(2Z#0Zs^1zEzo=W1?=&@BIK*BBD#fmrx&$5=8wl2PAR1=ADK z5l8wWo}BvNR3evtQw`OrX!!0E`#evxU_5Xb(xY^8;M3VYcKT{v{An^U49ou%G;Ou> z^trw!txm2o3wmE+=ER>Ld+{WzIQyLPJ*ted}0tzfq+1 zZ1;8niTB$6=dH*6&p7>jcL8eYqhiB67z~58QW%dHf%8!F;OiCFEgxng`#-IxZXG

    D7m#`bZALZ30MU5HQw5TA)m!|)3Kk@Uk_saZD8+D>^e80hkXLFcw%s@Oqk zv8jaF5?6@JTgX7JUbww_(xZt?%(6dlBM1sMy5eXaA8fGJedxPG=(u^hO^F^#ccLlH zD@b#&HKlA-RBw6e>f{F9H|ONMRQyN7Z}tQId_9VO<@6n2%$z?lb-;(AY}Z_*O%mG; zk%KNnh4@P^$eN93a3rggsj4oN>*{E90(?w*Ly(eXGdWco ztk)HfmRFRHPEN|$@sR-Jwp%$Jl_ImA1)ZG^#JgN;ZW{t>HtTIqKQ?b@jEDkDAl?sj zsAbG;{sif@&XR#Mq50XMbTc`Uga1&^W9e0p#rLdbQDjCH9v4&>9kazOV0gk|W9Fv{ z>Bi{~li@EWYnD(f60T^HWC_dfr3&B1T=LF?&Qtk4@og$e#0ym8QjL7AC_!(B#s@EL z#$1D)vYHKK7*y=Qco5nxRa$>5HVZcM09>zY?K7mmW@%tiF#`dmelh3u2rgg}GP%pE z!_BomK;4&BCqb@@S%d6-|IsZ ztVz8E6Ts$hiDB1%w{R7SMPIV!wIPtaH=3g2>6Nn~2R*H=X*3X}!{)dVpi-Zn_= z3!?37#$6oHDVhqr{iP@H9ya@;_3A_mq+LHb4VCY;5sH%t!632>kWV`3$tV0;HQg!j zDbt<*s7B-0`VO!rrL?~JP zH{>L2c*@mrAN?Q%x(sAXWzrKg9ZeZu^?J<~Nh)TCzQU`m@p`;m<+3|YuXWgIvC>Zu zUsOstDOCE~x@`FxYUXfa$?taWPZVW-;;o@rFG*~xli)Yl^i>J+mkAPnTNC+Z%M{Gv zIJ8z5q)FfV{c_D6(9_?-Pp8pB`^j`^)QIniT00bK@3c1-*)Djtp>sVXtq|=aV8_DC zv43eK(SlhhY6zb;YyKgH52+j?1c8`RoMz9PUT4*DrB%@NA;<@XUo&6^sUg$$3GKaq zO9Uo!$pIT(2k#1eO!%w#8W{z&IE>75pzROhfAQ8=ocK(%nyPX7h@Z6nq zay5QYX6H4EFBoh`V!guG-^Ei}Yc@$#?XWYz68$NgVE$#WE~B2M-qJ|BPys*Kw;5&| z^4spN#EWS%6r5bz*PrM-dE-tMAG0@jhuO7R5Weu(!bdFU-6LZ}Sxrl0quFR)CYI}l z`wo&9qfU!+Cx?})-HR-UN8C#@)W0thieBi-b0y)-eNS!O=g}BsHwk?FKGS-HkRdyt zSsMm^pM+8XJB-+(PVK9K-Aon4oD^G*0D&3Z{;`FIxtbu(&p`1MD&j3hdKL$Z(zTz! zc-r28@*AlXnG6Wg#RaIUgE0OUoC@5=>A8=dVFQmM(=ZODlnoD=QWemW4LtM>#=S~-5Cb7*+)`1;=eGOe8yEF!nCo>t}B#k@=8gbx)>ll~dN ztH)FpbIN;k8d>3~eE$zAy9`j(_u9Z{fOEgz<0&jEslOi$qjh}V8cxWjorib` z&X~r0zKTY+6NF)b5QgB{X~XikvY6uCwW+amc`b_0es8;aApau=eojkJ!tS z6KwWeTB$D!xqEoH8Qyl+2GxsK?^hyi16nuINH-J$`z#qM=9?aw#CP;!2Rtc_xi))Z zt!{k?m57*}p*!yHdpD*^*qIO-Fy(|toZ>}3f<)O=#gQRPMgd}oGRusi)?n3A(S(sC z)Du3KVNl?Db|Dw<9+>RVbWh0=*igbUY5q5)FaKmGea1nOkK*Qxm(| zyGP^{$8ab8ls}x^Ur6BCeW~=#Tr>{N_hBU`3lJPi=vvP>_7-c`Ux(RbmjN4X`&TDe zKJN&{1vegw4vdvn}f$ulhN{9F8dpZ?e$d!J+cR~V; z9cU`P_QQojJr6mHLhXCCi`!ar2Yh_AdXzkt{sdRU3T5*IN;|W8GqH04IN%=i9f^LT z%dzv9e}aT;@%z`Zedacn(R0|8bV47BuE|jsCmAafk z^|49N6&elB3JucM7}07>5=$)Yh=^*eH&2NO?aS7RVVYG4L<1<*~Is>_lKyeoA# zq8Ao^fpL|n;!+76ZG2mE_nYb5=(Tc0c0~DV^yl}@^aE{<*C;OA2R#Bo{|yKCho2Xd z@G`sw#j5eKg{1WBWTNGT67@Y3uk!wMlf1UuC#81n#uA-AA_tR=A9Gq21SrE>b!R)W z5^uGw_u?0RN8@ckp#P)t!cJy{fB>0fj~50- zVB2yK@-FN~o^V3x$<_FTTwWSXLVjGTS==Chh!k|k6HILjmyORgmA3-1!q4H>wnjY;>-W-F zLD_9#M}tZGrIJ5>7}4BICDn`)u)*I$-8h0-=X;D&#t^ld>f|?r$|P3%%tainVd7f% zYfY&?o#osuTeFV64|Ug`~ZC$Dj0!u zoU66l*AjFe&4wWWeADRXa31&C*Q?R*Pr!dl_rUYaLe;7i8DUGZ4U}x&^aBZwUt07b z^fzn1uJGh!zPuHTM*ba_33dG&Dm%@-b#4Z@WryY%zCKRyU7UI!cEe;p48job8Hw%n z5Rp*aWu<5T@0VxK{c-`@WZIDn99cRF`+TZmZC{_%JLesHe z20Nko%YqzK9Pl~Ulwe#TtM|&(COAv`yGNri7t@;Zm1?oV+FndFSVwJDsH6%pXw;Oa zE|!&nCOs?N$iC52ODnVNFw%v}pU6X`;Ca2(FEKs>N>Q|022VM^*UTW9-WH650C+dEy2)f=2O}rd1PyU0DV?Kt7LP4vkG6Rp5 z+6Dkt$Sew9`E@bz>!X@jEb&WaeTTt`cywv`gQTWCQj;{@FlA*z~VnT(p zRl2t*H;{IF=5F}*6Bl2FYouce{NaV2^?dw*wWQI=y;Ut#{M$Hyh=iAK~>}(h}cgZiTB&Ql^pNYF>7vL+ zHx$mL9v)a#g@V0-+#n8+@@d&5@3TKwsI!ZZ>Kk(iL^c1!Mzf9kmByQ|gDsKleF#_L z6BT;5NtELotAD`Nm%$63pOJrXQ@8~Zf?Khn5;)OPT}R@NQpA@9oRb{%<#PIj(;JMT6#^LK;Ey)_OEScm-A~4T5{A2K^7-A) zkC=J`7R%M%b}F%(H2%v~l!xB7x@hidhaV8?y)#7pd62NkwtQXN9sremP^<|PK60Vu zSu|q$`a|s>F10jJw5~-9+-Il6{`0vu?3>qt9DP@YBWl6Dk%!(r)fwMbNC;9}9ab(_ z2$zq6(RedIJj&#_69@e@COqQm5I)ys!qVm1cUOm6?n`Djr|yi3J4+WLtl9!)-fW&| z^3EAeDlk>=!|&Vn3k^*&G!q9bN<6>FLxfv8Pf48#v1elhZSEB+JqMQ^Lay76!uQlR zuJf7N`Hg$n+(wsgkUW+gkZ}*#rM`h2&jw2g=MyJDtvo1{KL1qW{1@)NR{etA#%&Q7!$v%l?ZAq|=f`Kh zSpMo|b6rToO*x^{(LZ7E3u~Wfre@NF`QRMUAywj1^s%m>#2gR9M-9BRird-GxE#|- z2yb28AiAbrSqhsHlqd}f?Di=S6B-NP7(s;ia&;BmBKfoMUXdDce0{1^KM*66DBKEh zSrSfYQi&gsC)h-N%tyBBe$bu&9();imUG`BW=gxQF?tl9T8;Ybrp@*Kb^`G9ocrpe ze&xNl0w%}8AKF7Nk$tf1v2C$Hm>+_|_f!xG$lho`?H_Tf=(;Hf>jVK7uUxo0F5FxI zpE{X@eNhfR~A zWclElP-c)QxZMQK3W~jWRDXkr%-pF#A%v@FNaPbTg3Zfb{&8mn;>l*g5*`A9{7lfLg4W>Ns zb-o_wIyfsPm+{peAmYa4?a5I7RO;(-0gdc0k+WhVYT4i^P|#u+aK$8!Eu_YIEtzVJCZ6SXxNg~O=L--I zHqHPtnzqZCu4Ekj7lpe@X-{52A@S3SFQKZu`WzpQ<7av0j;AgrItTqDw1R~K0J4^) zl71TQmxc_T(>f^0ZKhKz?8na5%fZR>2ZL!U9!4V+f)S!H!03HHO4}3Y^RB{6PaA%= z8}kn^Y?#A8+?IMF7hZ?CTe?oy{MzvRD6Lvelf6Rz;2wI$XE89~o|NXgeF5)zYKKfs z&m(lHBod)l25u1BSFhJydV*}8@;HX?A2ZdSj_)|jhE(yW81JgHLvQ~{dz6JfR2C*!GUWUY4fV~ZQoPgu zM)3AYYhv=QU4~Lit=;t@_PLRrD@Eb6(&nuBA4Eq{8<%_-BM>*#{{hTEGrvh`zj5*H zbMFZH;!5m+*aQ3I0sDUL6SHGF4&}?pJQ|_B(^3c=jT~=Xx3_1r&ei|pj z!|mbM-U2LpBuEECs5l^IW<&$DVgUm`hLyB!^j1bihDbm)MrDQ|D3m9VBoJT{*=zRd zC9q%~GB*QBiX5mTObSIpdL0497SJ_1Ell1CnSdT4!p^-#EO5PoQOU~6lte`_i=Ae~J1O{J4%XqP2Zt-V@XQazK&mL7OC zyc<^55(MEAga-aTVnGt8^`rEX$yO~}wGG<9e+(FsBJU6bQ#9eEZBH^z{Vyyml+*u< zii^CWqGIGC4+qCWWNLF$bMyNRb@fX(Y*_c^x^?STKnCx_<0WJ6X2uULzzO@4e)p-- zKgP)-{60kz-zx-R7nlQF6$!o^*RNotQwY&^B}Z~4>33`*21E?`1~jx}CuLEB`|P-= z9|eRQWr_j6=)?k0N8-V{HGb0crit!T2*GW;oByEZ= zn;-ifyiuB{uhRl-6DI$7&Kn{qeu+I0dtk3Upx?s1X3NRIMc`Y=e8jjc zCq-5#0&lukuU_fCefy;c3>e_!=XO%Vh7D^y@~EThMjmm*wnU;}Jr=zsxVPu2sZ*ys zfkh*X?pd-9mS%bRl48$W1OK&@ty=ZokoB834zJ#^V_0on?U1I%#z9R@P5ql%n)@_2 zHT7<3ZtjV8cjVH>rp6NdP5vNj?7ESynk;C za-4SKfw?zNDqQ~Fw)u#qX9q!*JVAiva)43k(xt&bG=sD9Xe8d%!XaZupyEUaWe5Pr zSQzZ3ArW4{h3FO&+Ps*}91s9u2#XVd-6G$WuFJ^?ci0SYxPgp9Wcul4_>*7==3Qoh zN{Ax!a4wQ=u?aPu53QgbOlp&sx%!33(Q!W7);~}n4Q&HG7~Ou+4L{h?qwk<&j4N8M zdF`ne*T4R(fH3u8mM@~ju5Na7F2dt{neo1GE`DNtNl9Y-gq!DV$RQ@~iaii}U^gCc zU+vvc?cK0{%Pl|oLds9wm`&wm+@?{L`fn-MHxVlP7B+afj=p%7tpdy*v4(zdvJ-`c?_T<~JBH#$_%?sQ zf~@iF$N#Xhdi~0i3`7QH;18Le85!uO+Ay)bqc`@*u)9_Np{`Oz-B_J(taYq1Bq zmIpGvV(n$x{nKZ>ghlAA_L66Eupm60@J_gF+O(6Jt*^bxY=DHt28&s`Eam%saG)cf z@F~M>;k@sxtfsQ2|Bf9y22|Hn^{uV0DXXrj=~Z7}+pVU$rl_X2#uJcaCBRZ$U2S`r zx!-cIdziu{yJqAjKaps{2cQvutv0~7^|)l$fLxDDgv@oQ^Ikaq!Z)|Y^q0OhTJUqj z%*(?(gNY?};Ji{~Tz14cTe_P`fAvGo%jXPob0yk37x@n4$8!-J>V1Fs%$O;uRC0y{ zb{FI`Km;SNMR!;74YcrU{xdSRtuAjg8%}OOD<6TVgb+-TRQa{fc&t>JPNXz+L~UkW zft3L_1uE!vW{5e;lJQD9&Qq*HU4aSN(C>S%V0X5Xs;!?5bIZ)zqJ-1VB=%4~{QzZLzHkD+b!>GAvxB)z#HrMa6ax01COn z+fjkC0wwF{tE#H1hbR8ks(bQ^W(m|MbcK zgU|jKi2x*+Vk`{I*6Dyy1Zb%Uj1y_N6x5`G7trJLDDZ+Z6=vj+Pku*hvaE~-cu~)R zhHf3i1Du?J!g?3oAj8?|S&dqq`VKI7xqQg{f9 zNBisW{rB+q6xRR&NH8q2sN^=2H*r^N-|lVQx&?Ww%U0haV2U&yoKgh^h1CE_m00L` zqyJWbq|GfYExgg&4qs)tEvwzRC=bT8Ln7Gy0`d=$xf_q!z zmFr%?_j61TWDRU+bd^ZEvoCLyRO*qSZ9WlRK@#XOK_XiLhS6Cmz|3?TiwO?}2)Woy znWPsT2JBQ7@qwun2G?dkQwXe#(g2W-3yr=!hhmaqB~gfG7I5UC>r-E%sSIkLSOOa7 z3pH1Hoaq^Wi`;FJGZXTYV=w>bmkv8&?1VNn=xC_i`r;c8{2YJ4z*c82by#v*(KLIU zU?DIcII0Mr@#9+RwHMwx=kA~#S7Hyu9@wP^a(#_<$*?zr_m|JS4L4x_$KLSl4=pZ% z06cB;Hf`MKZQQs)@}^CwZ`kN<+qMnsF|Q=$EfJv!P(sNk;s}gb<^oC9$DJmwXoqr| zkIA2xCkN2C+{t<%Px23Fv48yR=;Kl?-ojKm)k8ps12`c7(dpSPi%r&OB@_pN+yM`C zX7bz4juJ-4qH+yvP)7&~z+<4xtyCenBBdx38p@?Tq>t)gS2U??CrmiB&R5|_sr+oA zSLHp2$|NxbT|(yD%u{_LtDZ*1hexKP1xPL@Pgas@E*uG#U6+~up1BXTa>p{UY;&UdNZHK zLUloVDrs$P@z$?f@2y+E7Uen*kNbP;*RS(%b2&~Z!0qPko46wJ>9r4#*JtHTXmi(z zB>`1lQ5@tSEQvc;7a@PcmHC+0%MeO{KQAA`4;Gn!yl(Wh$z=L239Ouyf(nFO1YL>F zI#UhMT9>%(;;#hqfYt#ji_#=_f}{K@MPH3n#`#TS3Oc2MmrPp(pa@MMK4K|(!sN`{ zDewwZxm82whK7L?CmD3ObcplgGBia1q$qL|FrlP%{R(*0UbBuWWNxHq4IQN0UE=%V z7ZonWjqt;;6V4ssj072}mCsFIxaGaq&u|7{Tnx!5)y%d!)dwb0`P6=+O>gy!i^osC z`N`Er6_>FGVh`l@!0x}%x%u|E7l&L9YHm%vjxX+Ed(0Fe(hYFX~B2@cbJB09l zLZu}Q=_%OJho$PGZuY4?3R0vC63iC0%j^R=lZn(;&evXj2S0b==YKd6Is&T$Lf@}i zJmbGA>MAxK=`09@sx71p?FroC1n9h#hOgoKpmRq1?V_^o=S=w4BfQr$PO%4K5A4JP zZNHp5A>5zt^75HyC)25Ukg>~qZ|dvoycH`}c<(J=?yX$8!h3(!O0TlAUGG`0h7E7K zycp%XE^~*9Yf+r`&jb8CjYK{Xc{K7E%9SvX$)>6a zWQz`%VZcE7l}L4hTgHVl5MV9U;Kb(8uQPOWis;8jxDnPa#cc;;wom6z*75>dPx9eZLx#kl0#U40NJ-}~Nhot)-m~kV1JB>FX z2MLF0dhaY*;w@hMrj#FiutpZ#fMEYB+?{$A`4!~XUFPl-my?4oz4qE`MQhfq8IC*R z565$tgHk+h0#8i(=|n4@lEVFA#jj4Ab{Q{u?r-8}W(e|;$b4Fmc@#2%)bYsOk@Jzb zoA?$o-%sY3kokVbsU1ANQJo|+o1_k<3gqB<2o~C)kF*$v@#43L4jg~@(@D2J!$YX!6nh}{Kn@Ss*Ez>f z`*-2|to!g&F4w%iaG^JU{yguEMT@+ami9aH3THr7<1*r(VEIgB2H}?WpGDV{c>jzU zXW$nezlDYJN?4d{*C>S*^5Fu{|Jk%@Q|OyqiF^Z)38ID}A0ElWQ9lBC1ac{IJ`;Bw z38;9A>))U58Qd}m>aN5^E(N?jOIuZ)Ln-HEB3iv9<<%y__7t==&kV)^_0v1A@3KoPCFWLGPWS~1tZk4q8 z!}hKCgRu@`kFdE2d6xcQ3>CAHLajek(6%P7MR!orVPG_~#3a{;gPyltc*8B#-OKtP z)mD<{Rj*tA+|sB1G-*%t8b9sL1*$k6*yM;4o`2oMAOGzy#v7Nh2VxI|9+2;9*vxOO ztgLKWQE~BKwr$;7(%zbm88vD(9px)zQah|F`W2&`BZB6n1x{$i6DP104Z*NNaK4GSfMt`)D&1nfl9O< zSVdZ~U3*M#FfPODclHIogAJ4?eRwG-E?)eRum19gL_uLM>CAdr`|9ik8x}vWyIt8j zon!62&9Kvu8A<_O5xJp16HNeG7fihM=>?JDxEgyP_5eH(`FiKwA76%sU*}QFosj)6 zd6eJTqIsxiKjZ-}_s>xhFbNG7#6egr`Q%YYzQ<74Ce>nx@dF&7`abe%{1`NHj}=3P z9JcJX+kWstadB}H9IcZkO-jOMjT&u4#2p>wfqQN|vv*@vJ$DX|wt$}jfCx?yT|-!* zmSqHHXKWaR*)t;k#9k;9Ukf2d0)?K} zYrxQFF8Jr~Ux0CAah&3VhIbzQ{p#w?@1I~ALiq~f*bY)eZqSE{P?!y*H@b9sbHXn; z13-%p3dJe*K>1G8b;??%WIH-5At)!OOU(j<)v=0jUb7Dh}r790#IDY zJM#U4x?QKDq9XjbcVEA6zy4moe*KX9d;R-s9yoB2H*nBEuV>Gke^~EKFn+~I-ZKC% zpYI*YbPLYqX~canJfEi#;iDcQp&nzkXxHLSlN#K2R*BOyst|d1_^HIURKnZt7hutE zE!{kI>PNQ4;B9B_&fWii|1|dWRH}Ia?msHgh>!5@0;&XNvN8CxFGms@8RuX|AQm11 z8aJZjY(o~Mg)v)I3Ss2MWV1`Oiwy!ll%M_RQ_$zAB0^=1-3|ps*rk^*ZNXKp`3s?k zdy?Ktrf^?kI)!@?!70;K-H6dy@V!Mn3%gB-f8$Wx z62HVA5D)0Lt7Dw;C8+sL$EXr5ydP^d@*3og$UMht2Qm);t#g?_F~gg}SdyN*DtuDL zGVf7hrk}Wz#7k+#$hb@(B`X*{#FeaPZ zDT$QEE21SrQ#OORjfADq7~t4j{m$?VJJ1k)s*na~QmAJwXPBc6?XYo_==8nR-&1M- zRQQJLLI$8xu$MI{n{S2~2Mo!NEF6%nz=s{7j6kM=ov0kFXJ=2l@$~^mo_JQIJE*oc zH@x-6y|$r?}$T z7T8&mg7iq%tu1{G!Ikw!ggcE27@$9Dr-)YB!7dF&v}G3?ITTzrWP`F{SR3+O_Iuuo z$#n8;Q1g#6O*%DeWt1va^<@m8Ve@6b`BiA5dZhy|;y)qOI_>rSjTe96$1SD3`tTu@ zHmPpgy7_NE{6FUhvMHA-%U=U%inGlnej{PfkWH_?%UO& zUZBp&$X6j>j2r_lIAKpxK|z7nTL6=Sq!2iTV2QxVXh^?535z8!+iio4by$CEQ8q~S z(~bC{ZSMSW7qA(ty6pnuzN$3OiRDfp7keZe;fUuN(Ja8Fy&^P=ur3#&U5s9Ag{5^D zFzxpMWiO2PMxf&*RPG3Nbm9}3&tu;A^e8R9_p-|_<7LT1Hu3(frs~FpxD<8_-bm*M zjl^sUpkXhEq&uic7l#yfkpyEh9N~AYq{~jK548H9pCu)6C@NdOc4z9RCdU-fj~+3F zJ1+XP&RP#ZT+Ek!%V1F+`I;izbZD2&carP33nL)lOb^|-E#LU<{|%}?b^hjXwh zYSV909G{t^bMn37;_k11_^UrX0lzhvbGv!N;`z_7`}>pUnHVc+hv}ksM6T!vTIe;v zEN>5NccxMZ8aOT@?H3eWdHxTddYD~tial^Bd%(V%o#_MIAK;BeJ_VWQEb-i>Banw7 zcjS|9P}ebu3t9K>ye!v4F3t7miL!g7E|=@Nd%bW!lL9FPRlI!H842bBU?qOrt)$== zmtTJQ=8mvo?%cVh6%`eO@heHgaOT%YEd9qIke-a?d@M?SM)o=BuLY#NL}FJU)Zd0D zrhYVS+Lgb-`z4@ppA{a!jCX(bv@xxz$K^yQ3qw*Kg>NPffV?TbBf+kqYM9 zvh3+UFRfg+Ozx9)UJCSzeWss$sEhlelLfW*a@+|wTfB5_p$cA@!(%;3f`UXbUW)Ma zTTuZj#l=O5!lI(WVr2X#82%V5@R%{=p(C>_EW}@n#a>BCiC0=u>J^u;Uh0*UmUtz_ z$W})B1Xux3b?eq$7H8PkCSfh(ePLBtzAAAaQ6(;8Rbolj5;u5bXJ`i&z)COWe;qqN zN3{`<#}3%mfSb+lEGqPVdev1|t;?gTU5cJDW5(gFDer3d;p-i8h9`P-G= z<}I7OitXFwmjbG)s=T_|S~IHB+Zth2X{TaY&{={o|O=B7*-hh_B`e24RL}{FmS^1dO+B zmlcu*{5iZB@MDbi?Bp~t){8zF59l=56Bv*ngNB6Qp-r+|6_o6EdX!CokMN7|tQXQk zzS!x89JjXII2*8I9`Zy-ma_=$!LZ|n86sa8%vmU#1g$3 z4#y4nK5}X++FP-ivfWQ6um}}Y04r8ZoqAC<7Oc$GD$C09^4SAZUc+WAVVNT7^ zd{;rb__k@&KK_1|c^%c9fE(|p4jw$@^J~|xElwue|3OmT*niZ~$9Ov`w*Pj+h7B_? zv6qk!$xDd;ciou(MJT^sf+@Z{jQ5iGNp*D%_v}R{2^h2^Wxyi*))FZy!U-iL)+si++_QuXxE)>u6L)z;8yItvZZe2gYDa)D9U&pJ+Av98a`xcBhl6)B^OM z%Q1sNYO0ef1EG$nE}?Vd!DQHWdPz&aiY>5TY_>7dsZ(SebPeZy`kU|c8FnPE7PLvJ zWb-??5&o!TOG~NQM*IkH-e^fX^y#xco$``*x3}EAm4|@RibeR|gkSjMY11xWwzvHI zLH}3q{=?}7fP6^&m0o`H)Z+Tp%b!i9)91>fksah2>?HuhBa%WO zU_IGo0C>O`fQ&36igcJkMg(85)<(aJ^&AtlzzX^tTu^Yp9u6V}{G>4!TMiwKkij;V z$V}0?%0v&1bRb8mMOUtxMbEH=j!BiE}XD{H&EN$14eepwCS5rJI9rWfJ|HR zTHycu{2x61P3ekXVhn7wkf~dEAnr> zX3CVe^XZKHXU;s~&4r8p;l=0Yf9~xiZx7|A!A?wk?>S!sT`V_rqCI(4eeXY?Ik-95 zfFF|f57*!t3914#s9wQV5PId?;Q*Mk&cGF$Ru{0r4hL3*A75%>3#MS9LdAl==t9eA zSqbXGV<15JWsKaEOQCqNI%Tp$`ISJ%0s}bi?9c&iqaFoE(K_x?sBcXsS3t>VPDFG% z8KOr7QkNN$tBlOo=*Wk<`OIXsEkVP1;eTU(2tE!|+yDKwlJ{+m52Tsowm?E*jXDcqlRua3p9u?G$v5Agfc?nTQkz4X$ic2%?=!U!;{ zYU;N@L_T-u_Q^u{k73{;`0YoZ_{8Ol_DM*D372|D;&!Vsc;pY4I#S#b7zYIdBP8DX zb?dyhmn`;{y#2Pf^qnQR|B6>H}14_vW4@*y^xVWYAcBg z{!y@DCP+Z$Cj?0YGV8`D-ST1>Mn-~RgF#9c>DEAQ9b!@xT)6=ocWT&F?O{!a3|xqy zOzX@r61uw1Lpkj=N|s1?m94Ewd|&-z12AQT@P*hw1}6FjRnCCaXF@B*kL1?JaD%!o@nIxg5 zHZ{ztp|HT8aQ4kly)LxzOYDI|!~^cz-8E0SXz2KEoutnrpCdE0vjdFj%3y!!f1zlCZe<`D&5 z-1$17pH{s4bEjU>no7;UaU0mIF;K#HS?jD>iUO7+KX zpTq*LXs1j@OE*sW6h=M+KWeE1SH>peYB#EylqslVWlN9#$fuqiF?zyZX3@MIA`)EFsbL*P!1*PL~BK$V(jbCFA9Q+>O*Rac_ z^72QHOnJ?}!WZ;o`7#cR!=Dm<>9?m%`zUWpKVT9-*27qgKYpMXj1!>$=6eNq0#|Is z`#BsSih*VX?qNF;mz|DAIUM~3`IrxnYFNGceF0q9=}H51rJ97bl^?Ix&Hz=gn|cEILai^+d4}Vv2fkX9YE5>- zoCXmtiS%HO}rSu#1_QLNd|ZhRlBCVlRE>w5Gb!uwU*B?;NqNze525#$KR6%jZJR>`^P%fN zm5KMa7NKA<-rctV;@-qA<)TD#=$q4~ox>-bEm z)oDAE*=bJd?S3+HF5o*_JDnNyUZl-pFZPP_!8fkZ-cO;-Px_Q z@H-uKxMq9J=l&TpPRCCum%~R#x0lQVRSp8y4jixW{-!Nk)~)?u^``e%uBu$IVrBiN zO&f3-OYYqokX7=2E{=8O#9b}kw=y4@psPzi!T6WYj^U1T#`jeMvY`%4fNe_>=-J1P zpy9^JgDT;MhH)3nC4xtoGUybDP@Di36(CI~1p2iZRD?eSK{iZnR(BjyMD5S)SCzrI z%{fp=YGXarA?y*?flqC8@Cfam zeFx0F;D%c!0yO6QK5Tn$(Q~U`cyLlMHxW$(pfswN{jS?cbozV`6_f?^B*Y-uk*}HX z!>6YUPy7;l;2`z@Kkr>C-T%PMsrc)-yd#Xl0)j{K(+FaT`Fgs}qhKADS=4KvBi<_?xs>HkYw?;DWUEvA)bCI7x?vj@k zxuXT}fE_I?Cl0{C)}1B^!icvLnWy*cZ{bzPsqlz=TFs$= zmda%uy*d1jzPiG~!n4o$-t5Jhn&VdNfrHBf{2Fw*bpHc0zJ_n??FhVk`~)F_uD5yf zCU48;E#B6xTX5p{4sXYf9Ug+RS65%>)z{T~jg1XB@w?T-4v2j3d767sQIXdjKX1dI zu;Gc{!-gG}9(C-oJB~f}xOF%MehYqYuL3vt;yxO`3ij0F3S*t0PSoS?zk0NJgOB`v zU&>D~044#}Qhw4;;Ya-B=4PCil}Phf`%-w$h)**6Nu2(G$EcG9c$7NL$ElgSff%RO zr}@_{?%VLXqpv5Hr=Gav)l2I5T|d}RhVe4!@Og1#s!&SNes_qgK+Iu5-X>vPq1+oDr0ww|{_Os4@w(<0t zlwA@;!M5F|yFWMP1{`nl-w3iLcpvfovZcj^UPB#}Btv{LAi+)x(X2Tzp(H{d25-`0 zATq`t$DY+45{Y^tbm}uGsaq56&TT(bI5M4DC|i}2pM6X=XXPQi*{|c&k$N$eOq~Nt z6{iC>4I>LhyYawmYqJ#jj?@XHtuhz!r4k2y^8Ro<(R#*}pMQDKF{ez3inBUIRKEMr ze^=ogt&xG=h%U~_3u{`qxMLR+C>}2 zH(m6FTN_Gx_Bk>t$Li^H>+(16z3s4MOH&Wwgl?6qB@H&KjfXv)9~DYXtKo2@hVe(w z{lQb0!;Z{}L}=n_?12N>1N;tj>4YB&F5I-Oa)OsieF)#!^I-7lf4%4K?z10%%*)45 zsbo82!g=R=U-;secJpJx*<4+_-P)?VOaw>V>0`;?`KzrZSV9}SO|t8R%XX|2T(u{F z?IPZ(fgRjIB0wT2iq_fA-gV-oyg2Q~1L>P@PTaHZ;j6Ka`)_!X^mLL-aAyc*#YJc? zJk})@5;D0JiQ!qH47(h4gkOmRa%)Bf1Smk*!V#f0@R6$|fvK{(FwqV}7}^ogWz0v$ ztVbk7dKBE)lyJ##E`EY%vJ)qT_{DV?7!rCTioy}9K{9`dR*chr_8Q<~-IDeRrO*_W zlrFsFD?dNM^Ap@bZ~p*?-}qJevFMJ}yQjM9>gogt z5fJGmxXK1xDn+hnsob$6$=#(Y+{=XBsN%+khHkiATDqsQ4-h(Dn&Z#gaA%1CDHmvp zf+*H`nKw>(^uRqgoOWV5>3!NfKU7#r0vS=*J#RPwoW zDm^I!P|=L+WPIqic^O!V$}|w9wnUhsM1gEo)bFsPo;c@I-~33>omHt=vFN$?Uzjn8 zZe>z0nQW9&l?i9O1>t0sFZyclLn4t#;Y9ckpY{FObE4yMJ@&xC-~soQIpC&p7c9QE zBds2P=9%7&U->#7f95S!`Q#!u_Ima1<>5z~>u_J!TkF?v ze2r&q!Si;hj&%c7+>MfZ;P&sGiMjLXftjB_XHY}B;dtD$cr*aTkn}Of9S(D5-njzYSsMmMLSl#sXsJriYXU$1$&r@Dv7c> z9Z@>!Q$n?uyO!RzZhrAt98~&27IWN;J#avKAm>-&0GJVPh@Xpmc1PNZ;Qz$UH{bj( z#~wR|Y73IZ$#iiNr+Oue(`=!#-Qm=P!fF=MZDN&EMgFXRPy}j<;+g`Q6n!Ezb z^sPe%_Pb~DDoncGI&b;Ha-FyAm0r-LS z;a(Dl*7~Ut_`_o)@W)XI-J{YeZxsF*0SFUNg4Zn6fe#25GYi^Ls|^HXlo_BysTubT zq)EwclPmf}g98;Ij}6fR1BnXFLNH-8EJcs9gF|Jax1}Z7x(Ei07h2Jc7oD)s36KGK zs7LJ;(!h3=r}GG8J8f;e=M;zp*6Yr@?%T`D1`j{YF=P~&(~hOHepTC8y=}M-gp=W% zm@@q5`gBBn5xhTO6<--6h){jCkJtOWlWu+>-eoqY#pMC-fvm5=0W}{USMVzGhz_=T zHuT+uOhA+iBxQpqeN4N9jm(!i2CAMtdjcetc|Cje60k&&6a`JaBJHxW-aP!U{mk+G zRf)8J>!3k>|I!`+hd=AHdi925TT(42BSapFpY|DvTX2U0&<#SH?fj#Gl8F8@E?SA3Xf ztZOnF8;-CGc}RyEgvf4ARF^Y={MfClY;JDd3>C)-i2xanVBg2ph2JUDx|1n#jE-8B zFEj%4ar)Uu@@#(U=+?7r-bG)!Z35~zp{8wWs@b++$;|&gNBL>9(wKd;HPjRGM}1Bj zv&*a`C20~~`fB`;ba}`b*J2MGxE^p{j{|oCu>9l5vyeMK@^=9@+NC+}H1)zxlkF;5 zTfq`{o^(9drMWIJjXgKP_t#)uzAceH{!i1UowVD_4CUpI9+vW&-a|m>w&zOPhZ1%! z#wD<){etwI?jkw7SgEgTX3Eq&p;WeAV|9v zq^NM!s~Q5M%s6&u)U{2Gs)zDaj;YjA!4atsJ8}G@r+)MwFJnDd+OcBcbE_9TG%0&b z-UH`YVgx`Al<=4MB zWYn06N}u@(bCav*PJe61+9jtcU<=1-Z|D;@$#6KMX0+_23!IZ+R{Cu$YU9uO#sdf4 zfqNMah#LpJ2l%l*Skj*`k$>#slWBr|_C#Ft={VMNwF$5?0`GkfJamqiYUL%;IOXzy z@2|r>TetV>ReI;{eje$u#~$lj+t7RoBEqEz1uS#khmuQ8yZ8D2c7ViZ5Q66vCA>LT zU3Jx(cGqzL2`vBm+1(QLsl%K7^zcMFeGI+^V*m^Api6oT0&%cwZ9!Ur2myob#^4i4);BU+{%nYD;_c zIXaXTsinQ`?>zL&#-^GbhnfF5nk^^75me1LO3QBP(Q$o)=)2#Z`~4@r7%;_^*aHWk z2O^*B1JL?Rllk*B&miagiHJX}Sce=?QycrVon@bw@Ba~|E$f(Hj$9YE-gnA?j;hrydM z*J7{*XYJIP2#q2Jp^wf_`4#Yp7!?4LP^jS?^06Kafh3U4K_me*LL-AhEBrcaW~!jT z{*aagQ57l_Ju*qt7o|xr#*Wre=;;vx5~nQCW^;*_PL^|?&XInAKZRX=%PL9C2f_R~ z0JDmUOBa6V#-AUL8!>yZD?6oIo0q;-e%p~LJW3t-T7+a%hHcVOsaR_YwE-fj=*;M- zMB=h@zB~J|kUOr$9yqW(z>o34l=!qCpEx|S6DIXKbY6_i-KV@JT<*FALF&o8DmiF` z`x_FT{}Vio`1?HpTQ+U(a@EzfT%7h(AHlNEU9O=v*3l(`5{|4$zX}{UCsF9l8Pu=e zEA8=QWk=8B{x6L`Hr3MlEaK>47T6idWjBS!AmG}9q<}&Jf%Rn|0SJ&_4K}0Ao)7sl z0=Y=nARZ0o))Pi{N(y=do5D&ste{0foY+TJrf`B0HnT`c__`;+#80PQYie#C4MF%r z(qu$SN(${&x!NVYAf=DUYnUcO3>mVL&5PZAhaNTioKOGTrCB6Fv!QbH3-3Pin+dci z!v_I{&Xdg$C}!C&{t^piCdP^*J+>waicUG}KOf-}ym5*>aKL#$z843a)w?$MBn&S= z?!9ZWyo|mDsTU!07b~yn{_+Q3#JY4&UX>h7!hL@uLiSIvv-Q7w1GdZs<>j+Z6~LA9 zK8yf$4uVCe+{;CK5SVKaGUsAGxSy+N&a`Ql9}qVP|MmK@-}O@IA2PrzoQmvbI1nJg zpK}z)TOgK3#>?&i7Kx3907POq5r$R*uwfu13!{jxfFDEx1f%@0P^}N=q6RvfqkvHd zWKIfo)HdNqN1(^iSz-MICOB|-L2EL37VO8RQuB+I<29L+Cp6QmIsQsDl6iKmbWZ zK~!}*g+|LVdwTH06K8+_iJgKxkrlth9_XSTkk51%wQYaQ;urD}<4hM$_j;~IF^T2?-L*Spp0SDeBw z;}bRF00J{s@ANW2$^16hQ- z&xRZ<^1>hiFPNQyh^URSkXbNq16!0b5E+pflEMP?X8@9@0_sOr9HP5eflkL1j%P7i z(On9WPH8~eMT9Z{**dJJi0h+_M%A?<;iJmOqwPlBfaEFdk zn5-{RP_XQhul}+>F5mICj2tPw`JI`+^qcFdaF8z&6x)tjhGVn+CB$mCytgv$^rG(k z(pZ=%8aMuXkMfV$IK>_~U_9VH-Up1mdu!opyx>#fOjc4|*Zb^LeRin=-)H0B76%)a>q;SCO zlK0T_-Wt&NnDL{&@@s5}#wqr|f#HGRV|`$lyLVRp^rt@^^1I*tZt1F3tGH9q8EGk$ za(jKZhh*-nanq;I`ee#W-n*~-F<+PWKntR1HJ-d9V5-8!;qCa+Rv`LqLCC7WP5o5} zixmiY+Y>@lP*qk|vgMLXE~)GCGuw@UGro4-(3Zx=CHUdwzOtKvNhc7r2_!7JT@Kmp z&^`rA1W<$v22R2<4B(^>g1CSIbSPU;2IHU%c7jAdUch70q6?|n6pX`Q>tmUboB-=L z;AaMecB4mBibd(>=H?f0PS|jt3Pf$*=7Bf59Td@>duLrlUimrMlCgq!7V-kO-N?S1AdhR+Jt|{+25W0jclE9 zJN7_V^+5D%*HvxYXS=3PpZWRf+N$4v=i4`Vn>TOTXF2=HcntVg?x&1hvg5MhpK;mn zx?K{*gPrOD1ROpgSdH+9%aiE}eDNv(m;^WhC{-YkZO2$eB7xHPx8l+DD%|k3{mLt^ ztn5_7^Prq(V5RDsPw)#8xaZC9l}e|2A)5A1r_z1#lT8DF8xBX@Jr?~ZBDl8MF_A@! zfmu-pEVSqh58|J+Y?$^7gH@E4T^XQBJ0#v4g#sSqb8?(cM;?kM8Iu482PPu4IvBo& z<|~LHAR(ABux0WCkr|WX9Q|Z-bK~Dp;m<;8Qo}>Hpf^({`9S38$YgrdZ}eYuSn1hk z^I{c3o6o%ZD}V1l;`no=o#SQG8_&GB`R)1VnlUO@6^n}^HmE(JX3^(l>U)4585aNG zrSQXNefQC)bEu8GVh?mV4@5uDUCy}uHHeqf>YG~M+OU4zpdWnyyIys5_5O<7V}1mS zYCIA2A7J?H9y9GD;iHc}+P$v6`4uckxW^?<`Sbt+L=w*$RO5?7kOaWw31C8?07&w- z0ulfe`8dV{fC+v0)4QT)&!Q~{%Q;u=;oJMKAB+32JOYHiCQm>H8v-;~2|)T1RnvrTP?$}5RjP6P=9E@^;v1O zZEYk)Ci)<4+oTrYL5HTsChYX3)29bA&E9NLSwdu9=!?$TNJJ<6qL$cfIcjxiPWWpt z_|om^;?kZYj4xWk?1mLj{_%sltsfj`Gm_GY&^j9OK8CUl>`9N!Jwt$4v9Yjw;plPy z_Gp)Uz$5C1xE_0eANhkiO`kDiayprwvu)eo4bfBcEw znA1-4#*Q6ZRot!Qf@`k2>h-)RgHSB4;gfK!tvm_)F8}~J4-L$h4F|hifF@j;OWa*t z=-r9AZp6+~1%eQl{W$H12Oj+L_;Ibx$vHqbaw$K$<3h0pKum?kTLUf(v;;+(g3upS z*l$alQW;nhJsh)PCqp?R0&HB^Wd}tl*_5HEfvzq<$Hs*^@&YEY*?|hN$H5q*GLbe{ z1zZk?pi`|=3j*4NU(?jk^Z_6dkFHS`vmH&*Nj=VRl_8Y!Vr`X9GTqGz-T*a*YfYPZ z7nSx{fP+fMf+zP=w&~Od@67mF@7AXJKBiwv+6B2Zi5deNUBA)Du=Ifn#4=~siTm~* zKJ$BzUtu(H8GE3sdLZ(3?5Za2zg_oDpZP8PEXVDXR8djkEqZ;Sw`|$F-nszrKo7sQ zYrSpTwtBU-wX%rpN)vXbydFJz$d4|U_37*N>(|d4FkpZ;c<>N!=&)hlh!G>a?%lKT zjlaz=^3T6!$`sz4wbN8yo-XLt4ToRy*Z>?EkVty{lQ=2>%Mfm8Paw1O`eB`sQv1

    7&vU%#F=_4ag`Z+bW6(nyZB8GuO7 zbG2AT>+ECZ_i$$!L-Nx~_(({}zYv(M&64;0F}LibY(Re_GD~LOuf<&Jay3fa+a)<~ z_YXpTM#7F3c3={7a9OaQ?4HP|Lj#UdV>~|b4Du3XU9@$PSOc^=j~=vOY>EUtNWTVi z>8GceL==O)0}%2Sn0}TF>`EN%HFg^hY(t023Mn$}H~ToQ^3`&co@076z*!YqpngyZ zA|IEU8=IOdA@XP;reQ|TI+6hywJe--XodBuZNhE_QK*?5sZ#FXqfUBc?36F=^!}`5 zW9{GHnemIU_`?=g#lb9Z5IK=lNno<-KxaTldeBj~6!tmLHkQ5{&idZtzqeLg#vbU3 z9&jJuu4r2O*@Q*VpZ>s%S@?cmo=Yk&>v{F{bqJ)5d_>yABhqq%ZDC=dS6Ec!b?a8@ zm6kGY?tQ{~u^x-!9WVixMnbRz#|PjEt4MF!xcAayjy?523P+pHE7pJ zo&9xnbzXg4y;t9$rL^&xVANg{|);NoJkd^6g9Js686xnOMpJOmUG;6U0)z$RY?d@-HuXI;kI zu#hcC0(Q_cU}dF(g@8;zg6Wcm8#4P97>09JnoJ8wn++gCItax1Z%qvixEJ0(ES$gE zAF(8;Mri_2Bco6xzJh*JYFv@Fy+>FmQ>R{f-Lu0^7=MwGWtTPUmOc0Wyn82^Q`Hxt z-^9wm8ID`bh`fK|WgQcKj;FnPudw8_Grv7+dA82D9ebdwcz|E0Ln4)z&nZiJRj*;8 zI_8j=@IgLF0%L-s>gp=F*RHzCm1tL?UW5DZsyK#vRaG^>5;tw))XpB@&e>Yz)h@3< z$)igc^q2|QcAM_MVf+|efqe!jqCW$CCg|W2MZm#?@G9fsu9QHLU=`EwEno+P3e?D; z+>}iU190pWehx619CV5v*QGQ<;RYN5yI`z=Ek_}RfeT3vP?EQ*v8kySK&4K7n|3wlfe6ZmzV44iZz?ho8Qg z^v}0cbSvw79K~iQ=uCgGVAg9pRxLV1{4W}4H}v8kZPG54OMaE@B)JJnVK&K-^eikK zeE8YpZumVvi*brQ(B(YfKE7SfuDlrZ;DZm3ZcVk|lyteglNTMa(0%s+cM}PSvH?>R zND(*@KviQWDqr{3?fVR3&WHzi8FLl#+sKPtUWk%AVQrG^Xr+Dpu3;*;8D7J%!xV0% zlK4(;jku^FNVf^rPJCo_vnl}ulRyM^I>2B6LI&9_G<*S((QwciApC4ck^mTTIwfRv zhDNAVvW}T()uH?(RRMvG5pFZg^czSC0cC1wV^a%$8#3LSG&1faDv#>7ezgyMIp;~c zz|47I-e#XRl}G9rE-Wm5{o=3u{8&Fxz{}`u5~!IFs(Cn#{`T?TdF&IPd^3 zzY-v2?ko`?afeBFnqa!5T_+sUj9I@St4m4|hWhbY$`?rS&PGU`v5QqzJl%!+?i0^hI`EB5kV*03iTF z%B0agosClKI4pJQT`!6VKdU;ma9C>!c1)KMMNk`9-)L%VJ{}r)pPVCUB8WI|PNH zDD$(HoEyjd6pjPAVEnfqoflHZwb%n)!vpdW?HZ z$W&m7!_fiwNtiT`3&1e~IE)gDojW#weTj;~!cfKzh; z9-vrT)T40pXj#Z@Cf@qX%jfn?dD|a_+)38A ztFon~#oM}di??m-R&OhwyxhKR8=k;)8GQs-huDqayK+~~v4Vm^uehW*m27RT!O7e^ zK*W2KEH|UhEjwOLvyydYejl0n#HiKzBj?2~^T|Z3^GOUo*~r`-xifwY9ctS()j-;# z*G(At`bSS0dkBPSE$5Pj|J7#!0R?HpY>Ad+61!wlV!Ht<`c-Vz8PNn+lYy%9|LHsL_vkU<7~e;kdoMq5$a0>aQ1$C6QiW%~H<%)Tb0Gj7Bl=wuIMe6Bm$mJU@v zW5$e;%}H+|R^Z^Nrj9fKi|34lKjrFa(|iWkyr%N<1>KU~#+jJwhpDKcp~2gmz0#qQ60T{_4a!8?uB}9IlhBOc<`8xuXo>XPo-MI!vBGvPTX?z(IfsmZrr$h-ke(kodjGw)`OY%Hu1=hLC8ap z2O|$h9_ccHS(i;`9@TI2H$HUq0Qe8!QI{#cT0l3D<)Xht>nN4dx#cZVg#4k@0DM51 zBUNCAc1B_|B0z^7h;y_oh*d0!gg{J|0BBSCD7bPorU!xotFqXg%D|)mG*P(&O#Ge0 zme%B?0BD+lR&vH7*MU7}IJ=n44DwUH_3LQvSNwqVl8e9ciy`6~jTq%^Od`~^*toua?IeCCIf z=@c(haOa@I6BdX+Up;loXHjzP$VVCmp(i1qm`D_yeAr>bK3Y>#GnAL~4y9CtV>o*E zE<^6^^)4&(%2+S!?Uj}F$?7W;a8+E)&7$^A2qvp>NXIQh2KKva^5n@~;pVf$F~j4L z`S}=&%w1i!yUdy}@wfrrjTan+hvoEm69(iAa0O!n6u>%CHVr2efF5ZGR?rt42mlAW zBRW)|>R>&RNEZ%-F0)=44qVyx>n(a*x6($0yQ=D-{QyAhBVh#WnAA{vTN<)8H#N^~ zO{OM`5XWzF0_Q-s*iK!ji)>e?3`^`6I#4KwjRXVI(X;>HC(r-v_b-9%8Ri)^G;CZp z^Nzx%#_EA0M}>-TwIiG^6-I_lL|8i8rNj4Y{6zZHGj4jE4_(A5_CTk5fS=t%I^q1R z@267fn>))1SYUpA&DB?ZWk2=z0{>{_F)p8slEG7MAVzsmB?f_>J$nHpDOe(al6pp| zfQn4m zcC=Js&~}Wbk^Nd%xzy6Lf!V;RY*cjkse z=%>!3$A9SIQRjX9@^B!tmTYQRv~8O5#G18qIPPwyd}@Fh*)iNnY7<(5lno#71Jw0Y|_)24i9 z4|Q`3o;we=`@nl2jzm77-%DO@3wN2q-KMsEy4%DP_z&cSII;qJ0Y5D8rrg^RmyGwp zS45_tLH;-7PCZ-h3zLuXCLcdQu|!tHx9}@rIcx-B*}6D?s|#4l03>a9WrBbWCPxbW zfdq;}a=Sh`!9bLjMiV;#`~RXF@4H>c12XAUOteCfE-d`t@ z-h5+a=8VnPv`CZb()Y!BgRvjrvf<5{y>Tn{K*xK4-^)Wjm6t!+FX`32j&J$M&T<*dr*B`cPoF-L%UG6ayQ~i{ zsZrcSvZ!D7EoPtVhQd4t(Ek$;4&I}UkLv@@{<3E%tAj~(Zo za(Hhc(M5}|Ap{5jfOMlDj?JJzTcS86xG@ptUIP9|XYL@bI!zo)rVtpgfsB4y5Xg0O zq-BbZkw*JCKp>`oLOM;BDU?oEC{g$A!cx_CTk3Ao~64G$Xo#VjNUDj!)xYf#VItosyQnyX@b;|NZ}5 z1Lgc{^BiPe{^Jv6T{0CF6?=mQ4bnV#us3kvAg_P_{>THo{sRE40H$y+SeLZDs~LrG zv=n!*`~YC9oPnpS*#~<*hkR#e+0~<@(7ScYiQeIT`FDrH5?Q=lL}x%}Qh;Fq1RZYN zO45mQvxBs)gcuZ>jB%Ap={hE=nvAGum->+IR`!|)1JGPA^Uo@FK>$7&Ww6W|GvR3sfEI9S}Z#`0zJr=iP z4|J3VvOdxsrM7G8xxakI^;n4hx1+4vjHmhDdFO3!={xUu?=D>`M<;ZY?mg90P*8v$ z!5-`lJM1uT=+MKwVfZoZp+kpxgNF9G}y{7_Wx?OqzLGf*D-27nb)G0F%NElbU zO5z_MJcgpv(%`aE@3yN>^m>=#?+%9xB$!7m3KldOQ1?XxGJyu$L2`1eoiQfxh9gRE zBBShqA_p{8)J3GB3j0K;Qz0nIv?=Tn`l#X@;EOata=Hv$(JZye30Iv}4GoPO zaWCreS*D2?qYPxS;^YXtC4oz!`3B}3z7xS=-^f!ZK6Ju`)2=YS>@t;XS-yPc&ks+w zHg{J<3JT5@$=;JT@12b5OVmYy2h~@9!DYkiBBXIO_CSYvAlG-SLk;L^%IyBMXBG?23v5=8^J#%+u_u%o5p$2W);1bUnpwzZ$*N^5|@O4FtU zQSHKzp^vtj9xDZyI8Z{hX1>kKY1Sb%e@jDsV?CbSJc47PjTj7!5%QcbCRhBQMWJJ` z43o7wOl~@J+SmWy=kQ}DWRV9=9#s0??B53CQ5~$CuL6chrOkW9F?4b&xNG5>u{S+7 zJwhH=V-Iwg2lyF`Q^W&2sIU<{i^8f^E4|kjzUIBLXpz)+$pzBss*?uRPC7Cr`&Rbqbcuqw!%m3RB|QOL6Kb5BT09+}rj&TtPvbZ!c`jmftcx|SXxD|UK{~pNw3C_RX zF0bLf@)=j-V$Hu|(K=&qjOAjoY}vcs%P%ePUVY^iIU#*-`13@|N15M1ejfP+mp7x_ z>ojZDtUfpt{%B0|=#-Z_25Z_VtfixHKhg16N&D}$z?fmrJmCAU7I^;mr%jvk58 zMlYfi9YI*o8wdyjyb?xng%FE<;1lp+fuFQa4MheT(XSvy7?j`1HSiSB%BaqdJQOXv zQK;*jLj_?!?NA=oD#};X*EjZq{9XR>6wZ^1%}$jkJyF{+qf9-VrKXaeeV)DGhFi{u z8R3b&Hi#0oHm`p6t`)VL-#;s7VDcsyHl--uLH?yJ_qNzJGI9e7MA> z6PE{}2ZGP&K{&T|nG#QiZ%O$R(_ZRS{0u1{Wgp`CX*?<8^M<5uSm0!_*t%tlH+Swc z-u(IV()cNwuHe^qVc#6&r;+C&=l$l-nKNe&YEJs6Vxc=1i{YnH9tFK|YDW(smMuv8 z1>gJR)GME9N3j&ed#)xS^L#AcJHvx)yHesG3V%et2^l{>mG)QSnT0rJb zMy@X(v5LlD00jI~8l#NxGt66e#_ZWiKJse{=q&ELQfZWyJOT=oJ&F&U4UPiB1YnVx zb*F_vXD5@5|AB?>W0+uP9^S+t^c?a`wMKG+kKOn8I-wsrveo zR7pvF3a8w+Y~4DkCC6D@{O+6$0Bgggnk2Xa?SEimYQYYNT=iKJF`#cgTAISKJnF8`i(gDd{X5~$%eYumOt_f{J=F&j8}en zb?=AS68e_!BPn5vl&YDpL0`ge!LKx(J@%%DcxyzQVh?nP2kg_|A<7OiCHzbOSIAd& z^1OC9JN|%jeS6!&CoZv2;hZFY02Vf`Sk<_U%3ZNqEPpjv((3W6C+JV(-VVPz+B`M< zNYHlrY2oedivw96e$+et{;E|i?=D-Kes}3R{_uWG_sBc_ zT+g_1(&4zL`c(wup}7__f)Wk4Hmbo`!ZEruSX*hdU~x%74X7Qv&~wbh86mq!B;vDl zM971Y41wgsx5xF7CJ-+&b<98&RT?epmZBorr)^4#N>mF1{>V^kOY-H$rsgy7P-}tu z#lTaxe_Vf{#0fEHvOfZDq#0?`BIsXz;g^4wC@3x+q2p~|Yd(1AxwQ)(oFu-G$v}^& z%AS+)IzQ14cu@E_qykVuzwa&g`VBj6+zr34=TMwt59Hee!RIjFiViaQyrgnBa_>%_ z7lY8hB3I_i4m@r7dpvFVFZq%m%WZ!TwBV+>wQE1{)~sIbty;O#TlM}*)Yo{)WNZ5? zYhQ`A5ce=LzgvGp=I-on(~Pg4bB3Qv&jV1Ad&e9u5g@`{Cnf_R5`3}hxo(NSLEHwN z;sh>>2-x^1z=r&;uHZz+(Vf{3W|ft-tTMJ!r_l3-4YtBM%X)RDeJaR*`hBtk5K#Yjbn!^Ucl8=Vh9xyh2Y#Gy~p1htkMkFjF;NWJ@)) zmrA?!D0_C&=Wja?Uc;FjZ4>xX>tC9&sB-n&V{KGrDVU-RdtY>1W|wK7nqb31557N) z{q|${<1tRL2lDR$_jSv^qJv1oCoz|MkUM)9;J;wQ*YoXH+>>^Aa}qbr$2Ya-+tqt5 zzqPf++q7wuw_*KyZ{51J-r5h40j@S~+`#*6_F7(NFccORH4Pa$WYN&WhCe>x>@)v# z#~pXaHy}WTVuCRmE!(WKkk_o22%X4sU?Pi3nnvoxITA`Wbm8yls*8wy&7| ztEQIPDqJ}*!?iTolp39X2mnJ1XcgyXWkEsVm8X5{p@-TkjC*1aw66#3TTY<$=keTv$ti_ zW^d!Bjozk>8@(-f*dmRaXb!YA2#4FAcG~IQ>0`%vC!TncS6C>|j<{*~n-K|@-~#j_ z-%GraE=n)F`pPTc1=wI@-DP^{#~e zwFT;+G=akI_)%BUPEg1WKo9~Kg$t2Et5OUGLy%?my<^d4rx*Ws>}MK+pBV{y}A5htiKi^!l_CL2JhRMet_W{dQb&hxi-D=j?b#IMiX$X1+U z59G@O@^QdN}CT8;V~I*qem7T7{7O zDnj-P2`@3PXV21?FTM2ArfeAi0sg~ZJ+~a?Cmae85&<3(|GC_`h#a&@B8><@L63k7 zT^@q~4MEm#r6_QSO{t~FNz}RtHtkyKu8O0OLI95I2<5A7?B+ZS99qf#ux`+TGg1%} zj)??Pvm(i;4g3g5u|Y64HZ;v|NhTv_R&j7=v()*Rbo04bFLvuZtUmxml$c1Ynf!&@ zy`s|YBTapF*|2rpyfx4MX`)I|Gjwk1&0t!A0?>ds0IAZ?J|_Wee%haN+D#8%0vdM4 zDfU2KJP`fHWi}yoS zS5}H!Flg3f75c*v7-R2PWDbZ=_K!%QO9QOR_(Dn za}pw6HXx+WOQiiLe82F~Y12Obz7g_?!`6ze^YJjo*o?Re0=6|UqmF+XStDSCLR`#d z83iFaq&qXxZ!0H+h+GmNS+~?NAUY(llP&~lfgWR0V?t1%21GCs37qu6Hozm%&weMs z(Fmm@rwSR4v<1+C#lTqhI9ULGQnj^pFQ-$fa{}E)XpWS&P&@xd!IsuJDhc|HEdY_u z?2?|n=T5%iHXi*_Y~3b`d*?Qyjia9CB8BC zribqkqWC5DKps3GAJ06D2^Shz-d?SMUNgmykm|Tue%&1WSRXOiv-j$er9zy_q9pSvZz%ROI+O*5oKl=3x4r@&|{v8p0D1|bvYO)Fg zFSyXOpwDrsvtnedBV`0=b~q-Ii6kdTrINu3CM}ZgrnC{O{g5Sq2Xq7;)UEo2mA#Hm z<)}s*QwySii6hpQc?3oozv>YkZUSPii6lS%3HU-CPY^dZ)YZS0OnKv!gaVmB<)%q7 zOaY+b0Nb37(S$@YCNI|KdWM}ae#R*uxpr4iYuBuK>VLO2RBb&ne6y$s*<`tTI0p=H zEHpc43JV@hO&9p%$A0sXx11Sq5qqFrJzyWrb`^RM3+sisEk!=8i_MW>>lx&7+#mz$xx>n)@hxP!8kxUJLD_40UyNy36E%Qk8%v+jUa# zWPpm7_2Q%+zzxIR@DankV~!r>9fRGfQP`#84~831=wv^o5Y(T-Z`J*#+w;HPnohUO zN8pd#Kgo};?yhLebpR(Cq}>EKAK7gQLNW%j5d-+7i$mm#jH@Q?ZzJ9Yg~jP1ZzM%T z37+6kV3W}544h_B9bqRmveD>B2EPp9C8Af}LqkTwLZbiA-gf}TRb6S{88yq2EK9On zF<=9x7#mCvU~CfzDI^5j2nb|DLb3!1+4S-!*|3`}*$p9^O(WU#2FREm0tpE{A=J>j zv5k$pB&)ZX`M>X+d)|998W|U1kED0)_wK!?-!n7!etPrfK7nH$ZEYQ|A|HpEyjnN= z=1YMW-6}{USo1E77V#H^7pnIidFCbePMC4XTQ%Z==S(K`(o4&)n3hiQLDJyEuEZ@( z1El&ID_;a4=Xo?KL2+c-OePc0jGtJ0)Qk`QyqoN}#2FaM4DfvxmmSN%LWtjvBiyua z$MkoH^a?n!=Mp}H_6s;x;vg84f?o|z=}RZ)WYU?rxFumOhyYdqRfV}Br3Sr?GkOq2}UFmbq9=_uXyL?0_RI_s^BQ@~1x)t$+MuR8xSV24vF9o%wPWUX{VwgAVp7 z!jj7iP>?VO>SW1#16fyEKr>K6wt7;)5@Me02=an3s+yQO^|o~U(E((M1s-W2EKyK~ zIGXX*;bw3TV61eMWy5!b6J_|kidAc~W}%!fmJgRqtbXf9zusRwredb?a_iof4G*mN z!;eldouHPaKwA8JXWmCNBrkvpR3?}TiNtq~_`*#e&-E4e;|%O-2K-xaSIvw%{dZu* z*Weg7_a$M%?60h>YM(G+LT1X8Ddn|wbz`ck#yRW=?pTiwMZ4{V zO{G$fU(1C@m+L{lqoWgF!F9Oyj&|3Ht53WFWe0BxZv(I*sOst(nfEvCju)V`v@BCq zRkZ>6ya!k99)r)P{qcrZMENKs>~nfT+9ehM!@mn_*8aOA=I!-Sy7KM~MQb0O6a@?c z*q{+V1Cmd=fE`4@JHV)Y9MTDOPty`0~yFtudKZC&kHa9`ibx{=>1t+p1c3PO;6r6pSqHJ zxVJdf1qc|ExVI1`9L;~2BN8DcGRb!y@r4_IZz0BYoPnXp0N;ag*|`iH0JWdsn7?!C zg|AgoQsVH47CE$C=2(}MmbjuM9#@P7Cz-_Wy&?zaTy86XD`c`)QyK*JQ@E+4A2)^b z#&9`s<*Bc?&+!UWPfri(g5M~!@L#ZPM97=b>nGN`Ns}hy>%A%R6(7Imn=*BZtF5gw z%~pJMa%ELT`M2=vy`B15<8{~FJUf-?y&#=VT!44N1T(xR>#`SaaYdV7;tK8;Q#So# zOVEM$Xtd8*qlvv9;E&cGhYz?d;(9Ij5fy1EHC>Rf&O zL^pAw9`zF^x$)!2hl}$YM2JNR4`5FIckTF!@1AzrX&rAMo|i3Ker#>K49qV9MC{o| zET#M1m?y7s$Y}&??Be1jjv7TliiRyG5m*O}D2m5h} zGqB4U;2u*1NOn}CT~kxzw5}CEMNl<%Y{f478ekxi*ogJ#KPOG7y=u{-MLTsrhLLUSZ1~}NZV8fOd_Oim) z03!Z1mj3?q6CGWhv!GiZ3E51lA&J$;8Q4(~5fHwqNQ5v&z$O7O#1$uh?24kIlCpi& z;J|ZlN6UjNZ~N|X;Y+Fhvd83ml62aLB z<9e~i;2Gf0#?Qua296VPM7Og>H!h<+1N?NNJ?N0nA~M_Mt9g7Al>W~SwBmSWofH^PG-`D?8XIJ+;s9+~{Gwy3tQCg_0 z3=EegrYum*pRUN_}7x$ ztddN26=R>ZIiJ6Qza)=KoPk}+K=kvpONL_Qycrk|>LMHqac~ki6vytnn;g#KvU@VX z#R@l%D{!i=mMcvPo@xXz@ye5dDf|Fo_XIV>Ua(NE#3mm9Qsfr@s-Yo~+7a{DG+ws` zYuX;)`lNXCbFS}o^zIWJD`HCzP0!M=7shRD>%P~ zg#l_x#H|bvT)tKVipVlJEs*(p$&AutXE!X1_iwSJ#Pv>PAkVvPr|iT^+m``;r*kBZ zqi}HB1Ri`IU?Fb@f$&CeDaWrY4ICLzs;&+JlhSbY30J0)_%Xm}C|HkPNMwrsxuM~` z%di-QE7Kr@zq)bx1z6yIHVAvSlktngmXdXk^ADh45e|Z|#n#sXo;)%bY_b$l7*KfQ zjiM*UrB>^dqf~h1O6jBt$2$nH5Hss(fh3HmJ{aH&m{G%O17}gok|PBLG@1+a*(Mo> z6lu_|zglNCq=K=D0Z5w?4D|N&+|%9Dw;;$>)F|4d)02`z07fGU#(YgV_zcy(GhA7E z%ekL=sA|%_3pCI?=S*hpE4O}YY^uAfM!2F%ODNv1#-XIpN61%EDry02ykg$xZu+eC z<2ug3&SW5bN9~Muth3V@;Jqq%=<)@)x8b}^n>Nl|x9)YvW8;PmZquesI5xV?s7K=t z;yazx?NsCr~`ZJA1#1mGk3R*b8W`Nq(W~!Ee3=>-iZ27F!Mk3@ILzXeaw@3$_6k0}I(wA;2J@ z(Bf?qwty7qgUo(z$pSZEiadoP!c>!Ss6leIh&?rGrVYXbELD^-+n8ug$c&kS+EJTq zqp16k+1=HBS5I%>qHGldQD!2P*L;a3oXml*Ma+O6o$)5u#EU0ic11<9xOC8~q5WO0 z55IcbcaFj@n`r?t(!UOY{oEpazSfbvg*5o&9}tbZg0@s+$0YX-ipV$aAQqww8_1pxKC6^JNA%jZ^zzI zJUX$@RF_P|yB=v_TS#DAZ|?0?QCZ+p5wIrYyj zDsOA)e*nucHdNIG*}zo@HZ(s92>euuioi!;h^)^BBy1}6i#BcIh~PD8({{L~8<5bN zh(KdezkwgK#hS`i3?LXCebrYP^-qcjXl3mnFc2Uj;KBzG_z(l47M#?5l%Z|vw0Csg zlj=_|(0uu^gUxZ!iNxkD1??ai-lp}fhP=jxJM72R-FDK4FFy^T4|>VCZS|9PuYc&4 zg}LB0@E43k8)Dn%LdUI=ItB@TZAo$I+}WR6)|8_fcjF9fR|fdbj>~Syz_rVlFUs`0 zEAZhx;>~o&$)%^K8=G_T=#)LEdV6|ZPj3$|>-)Nw8+vY8^V;giUwGlgm)5U;eJf(( zB^%yc$~s@0H}LaUlYGzc$kzth@m?M7EYCXM!40u`%FD}>^X46KVohz`z9l8a*ng{3 z_TVZ;yABhSK&u5>t zt^BUm}i}32mtz<(o|!pcl(Gb2{VLQ zIq+LoT=bQ5&pdNwuEcGhn14b#nfWu;+EOmU1Rc@BJ{Ngi?ZE-tK#&2G==v>f5mP<2 zr{Dq0w{l=ZOa`vlhyVaedB37>!rqKYhw5lihJ^vJ!Xe;B>}X#Y!u0YgD&rwJ1ViYk zfk+%0p{jVM2&qeRkNt`JTH4$Ho=Ihn4)Y{VBlfdn)z|}v!B{K@9FuV21VFz1xbr{v z_p186_{;6Q1({#>>aG7<(%0QSA)i2C7fi{o!0T*n#M z-V9{Fv$j_&mfNljG&cTjayrv}J0`EgwoBp76-r~Hz0Wl)U3?`bmh2h*&Gq|+)~~t# z`dOL&^e-{j9s7pDu|S-G?aM$1z}2rx67I^SOP6B1_rc5WJ~jUW0I8pegsiUkc<>YA zXGa55`7BV}Fi@Z%f}sIhu(R|7Hl46z3%)dAGY{ZXmYhTNFaRHJHlIDu2tiNwSDXR3 zf}Gh1$CPEjLIp95FB&U-q!X^Qb!+>}7zCef83Hvi1rIu0u^}Gm)Q?wKI9bjt4h6dU zN+grT&%E_x->7v(#oR6@Z%KD|KJv=%zI7zsQ$WEK-9+LRW-wy0qd5vq+S?(KOtxd| zm3jEu@a2#jx8e-EaR%%i_QsO2#13VkvGJMGbmsZLVv>1NzaHG)pfFi{^_+&K|GK^M zyIZcYF;kRw%fE!_@GF=I!MCg2_Nn>bz$2hO z6BP~?W}*UF_(O2Q#g_np#Q=f!?2IV2OuYGo5B~G%m@9bOLf+E$+B0{r|J#iV$)p_vSgegE4dH7# zWyF36mT(!O1m_-_d(7b{Bo7WK{Jp}JbzK404xczIJs=l zU<-i3eU}IVi<8A~!HoEP0fZEXFibLl_3veTjr8wDiQ?JkoW118v(H}qy&VBq9OgU5 zz4h~0=6}bwEbLfl343H=7670iz!v>r;iZDC#HCfdiA4y~xA&=PURi0h ztwqO$-=&PNuCAw3G{c@AfSC89! z_WW^s9r$@A#2MJ`48$O7yHmA;f|(3= zd@r1m%A8O*!3-YB(xqoSQskx_fvNX9nB1cC>)=W;XY^$t;hx0T-(O3VBo7BTKj55( z#b3t0UC-`pA`+Kho~|ybITu*r8QLHOkbkx^n_;fCT%S=JfYGIybTYyPon56xltn^i z0(}oIL>5YBMQRl_1A+k5ra(Z6(LN-ko4DFi|0=0cYzJtABq$iTiRud@h$PE>psrNC zLd3S{OAgqP{-kkLm1!_G3S!|nH7QbwU(gtaV^7}D;6b!Jzgwax*EPDIWrSJXQ4i9DzDKs`-e`?yLe};)|+8zoac3U_Z z=*0E__w@Dm{(jxs)pz{A|M&l1LumXJE0;mm*0yC4q#ZAJeCDJXeVNQdfcFywW8lq4 zM4<2mpJ?sEaPHJewRidfQ8+UJXkPE;EMLZ{uw(x|`Q`9Hyia(>% z2HKHGMIm%=f{;Md{8KGjsL%0;ZTOS68m1D1XHPD2Qrt(~}kD#I&VVpAfZeaKMG zN^aligVjaFC66w;^okj{XR3BEnN)AbV=Hg@&fyuC!bOG9o5iBzWXKk=)wHPcO2K&f zFPT_1wyOBB{VrOz-Csu!7F$do`5Dl6(8wR>ZXe;b%dY!II___-GU! zeK~cM#x_3_oeK`Eul(#g4>}c}m7?eZ90CrSx1!6hF3_L}N&roO2AAj%j0F0S2>^;W zV-vBehu})s9w-I+=7vml?*RhcD)7k$JyAIgk#P#9FauK4px5_yNQo93kI_P)Vj~Db zQj2iHn9`}#6RquCv(PEmoQ8fR5XeZjBz4L&AA{BAyJp&c%Z_UJ#GuYVDT8N>r8qBkZ2hO~#*^;GSfPqN(Py}cY7~x+4th{Zj^2M5C z&5Ldu++_Li{$&8>2z3xW8j=_Cd;ooYslRo0bsdTKU9rX;k&oC2uug=gW)!eR$zmFS zdd{iuziIO9W6q2!eg}^L06+jqL_t*YqHV~f*8laor`p!Oc({P7FgS>rN3Cg-B=&e# z5QY{A!jn--=lr>!x#2e=7N6n_43dH1GcgE1ChtN9u35JH!+6BQ|LuZ_H(eDI=Sn!a z1J987OK~Fo*Ck7q;3ne2ycg=Gn{Jxg+0*|M%yw^o(-SjH9q_6FcmHp}(*atVo8<`s z(r#%M8lM`_j5fhkTU$F)$qPPl$(MnTES%}iJd)p3`&HN10ucOyD_4$4x0N}^kN~Wx zdJf;Z!NMOBWK|Cqc+i8%I%ca1($m>r=@i z53^`m)~AT+vDMSldmkRbb0Wi`EW@Xbzy}q!qXc=Of0l$>x9}hTV{^&asyRv+c*ZE! zue$Aji+j5|WdAI&A@QpLjm9E}Ngs1~dcP z_l}OUwbL26cG>dJ;Y0d$Ovb|^*O*t@G37lAnDq>%ua}c9`D&&({c18*`dWQ`<%UIz z7VYt$FOS9zO!?ItAE~s}gCzw{c+8RqO8HF%tIqWem+SK@p%f{Th=#p<72_F^NR>B_l&surg{~SrrO4Zm1P8ncm z&+LY}(U!iNLV4iOeTo+(8kUXx%e&D8#O>|QfPbTHw|p$P6B%f1Tz+OM<9>vXZQdP- zcf#ZzQNe`AIdTi$H0$wm=FOO%n$d0oxjCIi8#V3CT^q@<^>xVX4?%9JU6M;&z(?~&PNfy3nQfB*ZE_V#i-@~5Y?tgNS`Gm|MT z?#s;T&!n%!3_Ew51`3!-Fr>gq0aD9WTzLXWG61EZNv}W=WW{CoXP~02*nR7~d2X+o z@~j0|BJqp6F3_?>W2x3fJAeXMwK>UV0Ruk_S=OVgx;IJ}C^dViJ%B^JG1pJs1Ya-ze4PyScn zX|?zzHJSM5LqC1}*VvCsoPpd7=sPTz8uy1h1Nf?IQfF`9f8ci93t)UlZ{!{FcnZZJ zCeMCMjy9DAVS!FW$2<+YT(!5i1wbhTP2tyd2BP?Y@3<6x2BwZLcmH$#JXZz4#l_Z& z0vtjW11kgpvKpI6P}{;xupp?099hNLP`v=4(MR+XAtu8Go*Ggja)DVdn{9xuLNH~Y zw!l4%S$wpsF$z)5(2p+>mnz56BUZJqz(;MF25qC5V#HQkYv+CaseWvSU`*AC+Ok+f zYXfBheda8g=$&`wC3n{Ceb75|b*ztZr`A7k-81d$Uz}^>mN8Sn@*WAAGRjh>^d)2& zICL`o$&#WIXMbY(u=(Y-3@ASBmJH~-XScYDoiGE9jW-^W&ZItrD?#UB0^x(0<1$P# z(AnAPwrtt#nsKjFb90kxZt}J7H#IfkO4L?-UAM=-Q5_~39X?^soj%_E$60e+QIeS7 zR$`6c7Gb{@X8{vgi~TTVvNbeh2nUmZfC|WDO~zi)00)`2xbtt4z$a~cMXjP>jB$EH zUP#@mPpW1YScSOQN5g85qD0i<@h&A30YHI-deitq@EL>{0b`8QrM9-TJ%Znn%Hy=m zw#5`gF1Z>A=4jPov|5v&d}Yz4|CK2!8MD9H8c?TG>sJ5%yJdYn9kuFLCB#)!*is0Z z<)4Au*eo}=z2cR`*y6)yU%IS4svfuF4A^HS&ctrc!1dQ(Ke4wzeLmb?fH`x}n>BZH zSfkoqD}H6SVZ(a2dDAAhdGltix1hZV?aiAUK~`KwZwB5mcZ&P?+x#yM2`BJBSllCE z$ucC9tD+Xj`l%LjECx($4VT(vBL=d-l+e9~y0- z4I+e-57#V8yihjl7QgyaDT~!dNJ|Q;0w1KP1(U@eo82wVZO`M8Mu(Hj67?zh1x;*9 z;A-9IIvDRUGPv@}akriL;eS5`5fA!@Y^uBMVF0b8psk}-Cq79e`n-^BK`|0}oq|eY z)>GnVb3SwZha!Zy8D~J=I&m8Ia0d8*WY?cwjOp%O_$(iX2_$}Y-<$NBwBW|^*VnFf z>({RrK(%4R2DibtySj$O)?sf_W@5A3lYx((yuW)de{qPlHUK8FID2is21_p&G?55e zuwJt(g1$jJY(hqCgKMxR)Iy8=0IYcc;;l+3;@5)!v8h+C41+}VH3H>ljfrvK@F6iQ zDH1R3tFSsjTLB40LR$((Y*4Bq;4Cl{1W>>i!Bg=n&?CUo|+O>iP8op z(L&Sr_de{T-_BaJ^nDN?w6w2&;_i))+_q2z=oCfAVmag3G!OC`NGFKfhc9X(mnce{ zHT#p-U#FD#9A`k@p>Y}tDFe87tE#IvwZNq_^YMGc<1t|zh4{u4QjCQj3pcdfuzrJE zyLOFx{q?nO?V7b(16*}=#aE*W{q-9xKKx+96%`fpr}GdY*uYs9e3!u{r`>&hz4%Rp z{E8Ppl5qS(#p=OK?l7IdIQ-YebKH?L_=`iGKnWul5sS1DQ}&gXkgrmavw#dP+MXus zcL=fqNsbjA+UH4!&5lt+0|r#c3+1A+!hsnNh{%c*vq+Lz%{Ig+loRkH`XVbpC$x}P zj#j{+H2|y)c%pPK@;2S#7aa&lE=4eF;M7uO@F+_;8kDx$?l|^+pL?jXey>I7<}Hx! zd;R|3zSzBK)oc?mfGd3o!Z^NWz@UqtAS5Fsn#)Qva}W6V^=opp;_fKTK)$!oC{0CS z4*QWu9w}L|;B%6&1O$V=G)mWreG(sB{$-XtPG2?J;8j#6Ua0%5jX5qs-T1%F3Ly z%f`6kk`h-^QtSw-ii=Bp{l*t}vA%A_0)55!ah2aX?X=T6B4ekrfi7+ioO|#m|D;vn z7{85UB6t(HuKL7$7aV_c`+RW&DAUh(*h1GI`hRPtG|dk+?kGgf5bv(b}}P(r6w3upKw< z>8k7Rce7G1vp+uc`(xtS4}Zh&gK24R%wAJ5A-(acG|WcN8_Qo^vBJIj>ML#q>lLra zulR=9FotEg<>g~tP0e^5wXV8)yc<_N4t2E~Hx38tYG02VH_nxpm+#>*CN|)=sd#+U z!9Qwfm_5{Jn{#H}`{(a_f_SksNTrzAV^~6)+Q(kwsjm zXTeG2Zre1g4`@gstHU|Bra(_q5 zgRB4WgX5y0QmlJ_QG43V(cfR4?BiD^naXq`AvKmJeuYw*Nu*IH0md?DXT}1o;Xbj<7>62H z(8geDl{S7YGzQZg`(?O2rWi9(G2UcFkmZ%NBFuqOqn-6vC2v4~@rCCFRK57p3;62q zHLMMQhjH2A4DhSC03cNZIN=JE04dhKPasuQRV5434m%iT3Rw7_N+jKv&R)9omZ1*) zkDp)g5xmcTM7siJ@Q&5R7k%$c7G7C^_3cbK0)e1dlfZ%6=!>p5pt2DjcJ*{Tq)p?&I4q|H7}WEvXoHgz67G!&CR`cl~m8cgyRu zWMoj(&~==F28q#_^0ZYW@qBG%-H}r-`cZr%Xpp?o*Wi00E^!9-L)u-qcC|;4m)u%Gt-xQaD8Nk~37vTOcoO{-L z?;FV7dCDE1Tlg(Zq@T#L5G=%?a+OCBypRJ*40sSJ=)!EEA=>i>yoBKnwJE0ML|a)` zk*F3m6N(0{AWSwQC8}m-0=6+h$Q_f(gmx8=Fb-1-;gJ{+0outf2|SRzOCP@@JVYUZ zsGQo0(mdGnPAm_B3L44MvU~8@9fB9`WwV0H3sbia!>j-V4X5FZ>$Lt~wcTssT_U zfQt4Bu#!n!5RA(%X8?=*ZLTEo|IS&w_=#OM1_PPKvx@%s$VOZxOT2^j!bKL-t6b&a za-BWZ+InrZ!~qdXg%?Jo2MV;&^rlouC55LR#R4UBfEJ>Gve79;tD!;fBMdqSAoNag zUQnVfDzl=Wpqt}U8KST5X+Z5#ncBW9HAyDNc2Mvj)%K3NdV2d7DM^780Z-^T+R{wW zuM{>Y8YvU^KkV0YPQQ@*6AoJXwr;$C&0Rk|kph7&X?ee5SFgZXBvubF3fZD$(HRGQ zV%g0DcrjsAWFUMGjf&yMgWQc7;N8y)aGd6kgLk9vf^fx~#p~&^6hI*1v%>3MksueBrcOB!AFo~#|whz$v%F1XGk89gJy5c4!zp1tRRmKXnef;SjQSZr34&v*ta zltbb+myao$Gvm@_>j#sI=_5A-+j@VD+)>9Pd$Tey6NWi?y%Wbm93y&vFMfqrUq8W3 zo-)NvoH)r%nlxFC`ib?RPn7Knkb}4sb_Q^Ra^i=j#fh&iUc7kYu7rB$7fzX($)x@U z8~9O)0s+>+GOXx87XGklElRTJl&ovBrzlA~Av!`o8+;iEQo-uZp+iS1sXisxfrc8B{CQp)moJQP$N8y052)5&{oh!@Bimo{0j5FGHHU)Qn@vZZ* z&(ob)N=pq47yz-Fi>?9)1t{ox1VXO?M3A;E*yLCV^}QWM5Xv5iVq}pwz#@o<4k2fD zhyWx8xQ==kgAg@vqVNI|q7N?P_6SSk#E3mxgW*|in}6fy)rkzdvCWl4qm0=b_a18`56Fy*p$d5|1FU?@T#5NW9v^}IQar> zm+~{oR%C$}AOKV#fJPAtL@))*!fc?2i*x=w2f)L0a>T@pS`3XS3X#6LQ52NrB2Yxb zqaWkT)flk}im(;X7|vn{Spdginu>=Y$Pdb!_fcwcf^msajgNK*l#F{FSG4dj4Cgl% z)mTI>3|Q6J7gKum&45-AH+VCSd;70v9{0{|-k;UIW%WH9?!S5gBQ+Q54?Uk-i=Xrm zSh1I_1>ZfcIC=EJmo0mW+_;R63}n4;M#qrjQSL+r_&%AB;~X4k;gGLGP!IRQdnEYv z&ciq!!tp4MCV8|`Pye^@*Y}0!$7K{|0Kj1-?g;wI&hD{w=jRvw7na!1DYnSHa>b7m z7hqkKd6h??ieI-?N7xa%fiSIYfsZ0Gvl^|*mwte56qu-Y`iBd3FTQA3K!e3c?Whc7 zspIiQAWM+}f>6}4gyR>nz#wfIPc7ILFO^e^G@^9pQhlixn_Jr^A-rmnkvb>FTv`YO z$$>rm!`Ft*$6fHx&y-i!$xZH}ljpSQvEM!1`P$P*lNF>YN6x^N#cy>~Lv!Jt9y7k` z=ougUd3TP9xI02KpzrJvI=FaPyPSdjVc`NC{A+I?9K*dliviq=<9-}}!SO1NLCZDE zu0I_QRr^0IC?o1>-Jk=G83Qwra8KeQ<`=QY)@=isgHtk>U!J_P=iVED4&E~mfKbna zCAK2K#XyNJ&Y}HWFcNaK74X>=BL)Ywj$)8N*W|>%0vX~T0Sr`&7Qa&X%7cwakG2GP zDX=o{Q4`vxjv6;$63F;Om?@JKFBU5T?CDue&wm~mTPzw-zEeVwiPC^Hx~{UGlWpN%ANB11Knqz59{v+I9x zf7U;lbXPXv4#C<^Tl&t*F1VZ~QaA z`HJiBnPCfUbWzqtHw0UnT7?GLa=iM4Vn(QFZIV?{&!mH&r5}Yv%2sv7I)MQK2NquC ziRz0w8DK}jg@GkAU?Ev}9%@iCC|Pzv4X^m92qO?Z7z=osD7euG8DCFV@BQtaohNXp z61Jqo14Zx6r$WY}qw?`{d?KS@tZw=Nzdn5NW!wDPFxA`n_}brp?=ajPmSkKk*<(Re zJ_hA+Y$6t%G(QM8!Kjn(Jm{a6-5xOG#)!>;zSl-f(W(rN4)%vt~#P#zsa(28w#yUCoQvxnzH*Uv#=r8^GF%n179fSKkN}--+Y9IED;h!InGw>@%+(JEr(h z%%bsMpG-(xwkrdv(n{Ad;~jYV6n`(Gi?HDfk2l0lxHy}PEX3*~A_%H1Y%aPvIDwSE6uz-Bzwyw)`S&I{8gFWDesJXW$Jp zP}cI2t6urX0dfKguvi;#2>}ll?O?Iy@{JOJ4&N4#Ly)9=TkNx& zGr+#M_2399s^jfj^+}MH6I0WUaX;U86Bct;5_1 zxXdK4eBnFi9+WK;_eN3%^c^yiMih@_TN&VY7Q6y8oPOED*{~7Ef8lr>N5K?KlF4hA zE&nto)32dD=wH7D6L1{J9?!tIm3O%EmKD1Adn6=_ZHS737;T9TK#0gFI1u21zRFasZ5|53~{OgQRvy7yG8~fubLa zG0hM4bX=?yG%3O2WB8EHWHxMRYVFTt($iE|k|8|P)AZmhh$x7{o6`X$GKuE5T=3;5 z$5c;PAif7VwZ8P=eJ#)3J733$2vXV!djVP8xsK~Uf)E8FR96G0!YtA< ztJlyEeb9}@O>f?%r@MJCz3%?awBveYWx&51M%JL>(G17{U+3#^EFQoqTmo-i|0It7 zLJ1L*WCB|{OhZ63GwH;Bm^k;#B+@fc&qRXv!K6KjGKCUv97kbhAlcXH>R!Sx4*NPJ zKm`B>e~1C_7Lrw1)?n>J(%x8Hq@~RhI+QmMMT0?~9TbB*VwMDNfOG*OSq!gF87Waf z;~jdA`0>DjBmog@&|aj7c^{{cIW`d0nzs*(uV-zUuXNZgU zUhR=`>#euebawV0luo#VunZlD#p59SJZu&w_ZbL?R|Mk{XJDr?(AU@Jo_YEy_u8`$ zyYpxDxiO{u24C0VaMcee!n2cKt3@nCB;d;GEwBMiUcJdbt%sn(gEFY9tQrXU<}rV5 z;EG(f1rX+-j38ETLOf*2l43ElX2)zPCm5I?YyezDjxs0_M>dwOuAci^+dJl4Kd>wE z^kJ?v1~VdK5u-R|GJS`?_u`vs_BnW|k@D5un^xVk>A}VY%JWyM2s{~>4M%DZCMig_ zpjpv4rMGr$rJFnb!^@hC7uUl-1Cf_{_zxx?LEa3E1NAi=tcOG4F9)aLXnwPUz2Sx% z#`pDQ4oxRAbFnzh!N!uaF%=$&33lR}?K`$VdNQzK{d)Jr6Mx6?H~`kuuBS(Tt8v1t zdiS-ZbAv?4WDEcySA+;8r09w)UBd*RN3ZS>SlPnO1v>y*G~pL?qtm4MF`m@EBlrb6 zfqN>T3;G_Bes@oA-$K&E zqCkiFjT(uLHEJi3Oul&1NB=vWEG`-J#&nD=v*oc{AMIH4%uy02ogp~$8?_yb*9T;; zNl-%=|CZ1Cr^ampJFgyB!z2Upc^f9N#$oPU1}=o!k9SUWk7zvtukxI*q4JJNHL+~j z&HJTX|B;x)4#y10l?`7hWTp?5kJwI}0m}eE)$=br=bnA$839#WwhVrI&)LWA<1RWG zPjv}aV+A1soVeur1v{)GD>xhq_%YyO0dQ6ia6|~&Lhu$r9^P>1-;8K2_JJ6pHEm}B zIf&XqB5q+*ebI&nL--Krfa=3DHV5J*!|@9xG8QUHxHRhx1Z*;d9>&_-wDtb}boxXE zRb;A+*ocfN5`@|hc(`m`Rdd}5ANrT~AofzRpXZcu=}jB{@I%+v(^gMg8k5G6D;~{* z=7V8rT#_?r;6$9d=ck|{g&!p0Bo`mJ; z3II92aJ+#E8c{mi`Y<1kH5*dEeQLndRXKn|GDG7eE5E7^yczzq5(UaLz#F?KYf;5y!rb4B&Zt@=U^Qh!h-aH6r3y zh8uGuyXBT!CUkW7o#fJ)g_%U=M7)9LVxrW$hjWGFew=}w$N<6A>i|z{*1qOmU!yg_ zRZGi`U7^|$*Hz^u?gtkf?WWa^6@U=va^Ypcs>{h*K?qvf(VBe735cL5`~Iv*fnBdJ z$V1yA7CGew8sOWNA9G2SP*!>r5+CfbXbjpG>>vc3M2>t9qP(O5DvpCBinA#tFl{=Z zMg}XgfsEIQSis|J)@*KSDTD8NjoW1Me5L38n}o$#R8n&HNtawP7eUn;FSmwl&&CIS zy{f1A^;uC~m@AIoa~L#>kDEER*GL&SC*8l#x@_5(gMQo?mKm^@dRPV@2fbYxI2TgC z-7d+(5_%Hhyd9vbud}CT0k%w7hzV&S=FZuekmC1kVZBzvE*b)+jT<+(jT>;FUbk+Y zTfZI$>h|{bJu;rX0IL%sqk14=;+f!q&K zE*LQ2V=m05;2Zixk@&IzmJdeRf`!DB2UiLzB3E+|MKC6FWVP8BTV~e}koG)3%e_#; zljX0sx9^$emev{I4E}@*b3yfJ8(n6xZok<-ne(oXZ1dKOOgi=Q`a8ZmEtTpmH-N_B zNI~K(|I&{HZZhK5%Ow-ZbdgIeocZx?6B~#`|SET5K$#MbAwnx{h4>iQC6b+4sz-d~Un9dELFZqIHt^^CIj(bfnjZVsai=(}$i1ssREQyKUIwElCaGz&oKsH2Z| z4QHS0rcd9y0HTRQ*i#wc9#bvN&91q*Nx)Nc3))TEZzdg}igi~PcL5o`g*U-}{`5nz z2pcE~S87{}J#NSW3buOd*I;(lBpXa{%BQ`s3;+Wmi+CWAmc;^&&8~<=j4R(Om}0aB zD+Dkit->-W1zNPJqm5kQibSB}*YSxgge@2B8(QO-EKaM*?r874yS<}pp$v=!D`Llh zq7sA9)u}|0EGm9t!6n}q2b?y;RP)s>Pu>1d`>LmoA&+C0QNfPcRJx6oJ>T}eY#hQ! zCa#`!StFm-8kb?10lwyO8TlFD&yV?tns8H|VL0Z)=go7co_d-)?C^Q6sA$;UhJ`&& z6RYuy$5-&B=L+o2wZf$ntymO3ly>POhvCbz%OUso_q(n5`mPD!Nx%|n1yjuerr72c zsg904^EKWs4{#?8eDst9+_}f?%N1IRP1NDVkbnX{+a6d^+G8lL*I`i^>kF_#CjcNY zEWpGBe64Z{dxRvQimebZnB`CwY_FnHqA8~UI6_JE1l%wV^d(>_T7fo%yeT1u<7YSU zjCKgT$O*>9PEKiRZoLTZ;gkTJ0)8XIr843FvgI_B8&2y56j)9DnpGiU`s+5*tj5?q05;T0$XDt{HK zwY7EFj`7V3Jc-5jU(Y(s&BunNvc$^*kA+ueY+a@md}v!2XGW6^HU`G0z(pkSMw5j; zM@W1qs?Y@={3uOAA)j;{UQ`>TLBPP2euYT6KvHapI!_o=oiR#B3f$!y2$p3<3LF!Q=2z#c?^4K9c@vCe2YC8j@U*bWSo>kTOJ^xpe4?6k-rkAfy z_jW$A{*T{3l3^R){DlOx2&z~-k+}-)XMv+yr%0;4EUvMW$g8Ufa>Zf{VS z8Q48hjVUQ|-~GVR?tm#(SazdRH5X;}6(n(uCLm!kj`qTH6nMzh8LyyLLOHe!u!u=B zOu84#@*#Xu!V(1u0Ts$oUuG+$8pXz>gEFex$+eIVBwBP<`?$qWmVW!jja$}YYlj1A z+r}JitBPJ(lKe#yjbX1m@uL4~Dk>>ELSxH+?!vYX%}?LCKwcmZrX(_p$J>;yHxYe( z%&Pi#f7x&Hl9Lh*4Jo6?b>U|q`q~zLv~h$(kpW&NzY)if zV-DVJm+3gU^OtpY-%|jbBKJMGWXfkLg30feBS*?y3uqbrVqsuz|ov7Hplq z*=6zTwRW;V7&WyBSfhMII4}pYH$~JTG6FsV&l-WrXUS3q|4lvsS|aS7vw{-!BJKlU zDy0G<0hU z{XTZ>zeY%Lv%oW8uW*5f8HYE-8Q`nUYP%SG4(x!$?tyrn))v#$>d)aFJ8Q5cMY4)#EPu>L1dzlEH+OhxB*4C}K|7ol2`NW;cw{AtBHE6BirBd-9$vd0tH%o8-$yM%q z7aZfp;$hOVzWc@3)@RbRjdv<|iUD*p#z) zhecQyc*6j>L*p_nS3?F9@CtU7hFt|nuq&qhst!A5f;dCuv=Xdjew1#@h{k5ZQL5>i zg!(5{0h+kLi9St+9f`&Q5W|L<%(^EUPb}ujh;?wpn$$8jkjd%l>3Mi-OZyR^6$N0* z?4Xt+3TS!IWi>1|9e2T39xbb?JJn0(t$Lf+-n02HS1-^3YCMtyj$h)6sD)u_fMC(R zx<&&(n5izWIK2MCpK?x$OMz#AFK1jvcLv^t;S+4_H@edhk3P;oVQ1idC+_bqK6$3C z?+8{`<0xR^2oyA-Vc^6A1g-R|GHdH9?w0{s#c5Tc9Br|9b+)|69AIFpi1Z@F-;xVZT46EC@fdv3@t2~{rp44awezu)p$=h_$WkoxQ*=CVl=a+ppX zpNgQZ-Cyp1*|i_YQH#3;n1S5ay8uIsL)r})D1(nr;rJ|$5qq!JZpdToBF?~&WT2|5 z%I&-F47cx$8E(dmnW$&F$&)9$nstA1mCeuO@qtC(*5W~C^?*&m5Rt%^F7pAO)}tM= zS-f7tI}$CSk(aHK-3tsr&|kR{l5eZJKdUT#$W^7F23vw+9Tss971mLodBcYUt&6O> z2|9#KTbf$#>Fw)Zz}%=DtQf$`%9nY9ls8zr&%r;M`<{!o`L!WNv|+=auPW*9>ZoN9 z60(KO8n(i5D?_W|Xv{!aS(%$QZ7=*lV=uS&-h1QN$4$pkQQ>+3%G=ijAcT3d#aMFa zfdMCXU5?Q(G(rJIBy7n-PZ^WUs{^7z0{JSET3LdDx>!=Mz$UMV=~b^m+^hT`=MZqH zca2kp1?U+7qQh{YOa@$sIt(=!zjPQ`7@I;W+*F=PrZ;Zh@-Vh`IM!@rg$BD8Q?7m3 z>z=pdvYV==&OAFyC#RF@-1@hTcmMcUAzKc;O!%@Wz+{fIM-dECG?EF|QkF=}op$Lp zujROjy9Ja1|CTGDh~iKRAp`ui%y~FIhGST7U|I;tjAMv1P>UZ~Oqnv(O_@4Xe(lPd z;A+Bz39@kSNQM&WJ~#2zn_WrwmSE8r(BX|35Rl0oE4KkKt3@mqh}cRX_#}IgH=rc7 zpe0)H3qJL{j#5J)LVzIR5pZI0diPY364E33A?h$OZ!hR-?y`(U^cAH9p=!uJ;2?Yt zw#-*1)3IS=(`sz(a4>5^|&B@)T~MR1T)$18ukul0rd z=NnsmFt{KlG$T0;7*NoBhBbiJy)!O7cu^v8dGx+dGac6jkOBK>6+kF)2!)h^#>TrU z)2`{KFFgO;S@+$0ulws?AHa>&9fcHP97~*mg2_Nx*%1ZqzF}@rUx%B#@&(hQFri~L7End500_T=`<2{7jXW&1pnU2OLU^lzl8_@j zl~Za|R(%RE5Ia%0;H53jNUfZKXR!m5tQplnYXCaAv6`r8ctB+`#?vqZbD4B{1AtZ< zezJe&o=k9Z`p_O8u&HfrUvb- zUUckX+qnMK`O?gfUGpCyEp8Q32K*WGP+^Z>u8U(Zi#?;IJee#8f zI^u_I+RZH%tJUP2oB@ugj>(BB83t_XrJkTR-TVa$ z_2Q;2&D=k$O6ZI+WK&X#@*=ifGBx|O5Bze%tiwNOGI{I%zNa_*>3avKQ>h|Nk&i9# z0xdAarv;!S89HrB&h!QBcZuTT_q+5rd~{k|3M>PB4dW7LAjrTCH{3A3r#F2cCZj_G zVormPOL_je=iFnDJ?5Tz>It`c)v6qcxEp6+cw~S#nAg_UDS#q)s+#~1RpV-FK&#XK z*s+7|HMz$JIi_uuo4DqW!PSred_*I|LM#h1W_tq_!CEdW08de%fWDHv6|C~n1z*@v z0c|T7649vlplx7@1c@i$ut6k2J6aTYFy3e%d=IX&yocn784pktI>9o8L{7&UaV1Ul z9sxR##a1bgG&i@;gV7>_GZxzMA))7ODMNizCXsmYq>KN%!xfc4ckt4^{-wK{AHR8_ z*o?eJ5eK5oT=-!}jDaXcUt7fYa+Rx_H}TT%59!9p5r=VeCo^EL)K03$Y9lQJS2r%d z37_cq?8vydq2<$0Kk1%(_F4DB3(vc?YxJhv9kCJ1#u*sO4DbQs;KBtJ+c~ypnxbs7HJGqzyxP~j}#2}80e8JJyeha zp$NDV^Poq#3YKUbRCRBQzFH7ji%UtOL$Jppeg`55FX{R01u_b3eE8m{kIBYm0n$FB z#echU71;3xvd0twSw2B+?H%{Fw|DU=89P+Ut7?9G><7QP7-0|kmv=>1No&E(_K`-knS5DFfTBR$0RR_)p0s0rvK=eNTyFI!jJ$!ffw_nc{Cs5ATeU%N62N-&lMHhOGTS0+lK0iLnt_n~_h8EF(1bP>Jn6Wz!OMi2n)w?yH9gdy>OV%u zr8<-i1LOG7g+azmvoAex(|hUg?`YzdOU?WVCtHCbw?= zI=6n^I=5l{dTDRiuwLjf0Nb6f#+&P_sHhk>jgQwWO}yVJ&)0YaQ6u5&yEiwJgIiCe z``zSKx42Sl?Z9POCiegd_^Do2ZSv43$pmFFSKHx|P9qSC$Y_UTZ%z_ z803vUC`5%wEln|-aLKarn@;@4|6GWHR#SYi()RQp9_o1QX%FHEprVp`?Mmaa@xo=6 z3`0nKamJ-re?6)hxA%Mo;wxEu{xv9w2zq)_2kq`aIH~cc?mXtsJ#2vc*49?HdGjV5 zn+0?=H8tVE#?AQZ5jDPk9Bb-YYFy$?mOetBU5zoP&=)=oiZ4~u_lD9WZnfx5k`9}s!3-lHs3KErp)LC8tBge` zF66`uSu9e~2n8+3z>d;17D}T~Myra+^>yRcZ`iyZq{%{zo>WX@6MIz6xKwxN87uBx z{+*fg7he_?&uv#8ddiI6%`3L_cX!q?JS}WUpmHWn#w+$CIrhF?(v_=Td@iI6#2~AX zUX48)%b@$Z?5VK0F!Pu>bMVJ1_H=h$cfkkV|7pZnpL5iq&uz2E`-6W;TAOwJofDD} zExg)PSy|yKDl1(@MWw66*S>65qRqN$TvZ67JeVpkkLUGh4)=6BRXo-Mty{2zIlhX= zik)Z8R_@*2 zBN$XOa&5}IOBg;{ScVM#WehlaDX}FMeMs;tuIhgn=PkJARX_f?_x)}2 z-9N#+Wx1mb#1eMBu0*Q#J&*oj;|)KItG@rfnPrtl8z%q5ui{M)^S%;8PmDp<(C2hm zT1=NDR=Ht#dsNw&@}>AiHO>I(Mh!&$u~>S{-e-PoF-$?A>RadEtb*2}f2CR1ri0P;oz~xD5Xc z^p(}P&3m1KtsQO$X7LL@fdkqCM8c+!1W3qA?m-FUba7XpKs9Qz=<=>@IeS?G5gHRc z$|=H#2zvLP7XmHSVU$`3YE+-KI5bueqX#iy8mNh=OVhAPXaGjM9T~4W@xGK5jxBKL zW73K-mzIBjFw!t}|Xl5dc1pGjJqr#8Ug{Z~v=&{v}te18+)}(MYFg z%(!DK_dE9f_E#P}QPZckDI$I_00C2BpgKJ8B@da@HbK_P|9k(8Oke8Bo{kiM-1HB$ z8<#gf1JRf3&G#02DCi7abM5lg_^?hd=pge1he@+1>0CoY!;+ixunU1?;EVMemFNu5 zwgD=FtRXGeEL;9z++*@h7#%mHhGSy|k%7vsFS!Zp?hSB;0S`|49%G%(-1uv|ztB=q- z>`C0d0z zhd-8OXke#x%Y8p>>+fipM%DnZniG!jK@*VW5#jn)I!mC$D@CGx;CckITr)KzsE8uBD!PY3jJ7k_9=sc> zk-!Q}8C1dIZlJ^XX2}-+5R51%JOg?Vw0M9czQLqz7Sm^NB1d8%L_QAxPos+Pqxz9p zRb9`FTYQLr(v)s>=%0Rk3C3>#i*(~=>yNLlI%!O4>3uZI99WUuNe58}bhX#C^4SYI zHokVX$qrJFt3CGIO-X!*DlyWunP+X`_@?^sIZOQCfa@ z_3Tsb5!XzOk6mp?`y98AkhyjV-rkV6q0EHX)o<&=M{{JL!N;cWDR*rhp)kR zzx&;r5&aj3?+9cfI}}g-x&7+K>mJ0tT4y&lKEwN^;_?O=IHzIBkCLwRP|WD@RjoJ3 z)o^2Po^+z?s+fwfL(rmQP(&r*00aRGT(za>0v>cdm9nQR{%Ox*lwNX8002M$NklST_qdX1!!%v3S)c<#aS!LqDB9;fWi*a~phZ^XhcjUUT14 zI)2j28inS^#$xf3C)Ooz9BXD!kd;ZK&zAg(5_6Rt{MSV4G(Hbe24avkM2Q?JX3l9? z`U89dZy5ID$+%-NBVL22EWVC=&AxZ_vKtmQHf9R$7nFsR=isVVGVu|_#jh;mG7>V7 zNV<(vPjS6vwI1F?KnTZ&McQyqI9x#!z8iuSTj&WN6qH0Xyo%bAJ`Uf8nsJPnMXZ2Zv;{jR8}&s)T6#QZRZaLmCt zgqPqCcLtOV|M%fZNtZZ55u)nj7ZEcC2*!jrB?B?YdQ;Lmyp2|tm!6ML^CQE16h8V) zrVbO!!!e# zWQp6b*V|mGL@#y-sPF)ULkbq_NZS^4107Ky5>b_m$inXt$Us{r%_}G(4wFVhA5GE6 zq(gY5`pS>m5;>8fir4q{M2^P2E$_u8aG5Du%wE=52vCB+2Ij(h@)(8o=$%yMz{k{y z^);IST(D`Mj)ArEt){GkS-2bX4KB(rf(;rX)ABBt&ZL7hQ5u%SV zC;%}HTjPa?H%GAOlYzpshXultjuBDIVh`eg+UtuE z8W36OMM;4_`VZY_gQZYQTPpiTGN$Fh2xAUL6AmC8KiEFbj*ia1wY0P!13(jrTA5kH zBFMqNiu!3kIP&bzd;}7M{#XZksn$nte71Y@$~gj#qJ{=W_8J}#y*DF{mQ_DE_a=`ra4+c3M;}ZofGhV7}uLXCQ9|@Ti@tuI~QpF=?EVS72lk`>-m!f}y>L zKlfh2l=Nyc<5oGBTwPvXyyo=NPv3&cnltC9DUFS{*QOJlS70@}026gQXAJXKzP$5w zH)+iuu|YVVR*JFk1vLO-L$f~X+!FGDFJM4LOwKG)OTmOVFtW%4z<8e45Ty~ySfM$Q zCBb(B9%Vh#*OXwP32O_1eb^2GO0Iag&T9`tbhh>sy{GbtwQM zQbQf~qJ_Hn-*?EnKJ?Stedk;h<>j^$ne?j7cU?6h-Pcp0d7^XvXC9^LRT3@xUh>=g_%ZK6ah|v)ie%szbg&%1g;U38V?_!t}NQf3R-Ai^5|)0NDmCDVvhXObdjz7G;WCAi6b`8rO2xS*Nr? zZ<{x|?v59iuVrQB?qqKdvmVtgi*tSQQ*YBHH*O(vPH!AgV)I$5i=?3Q1TaZSRg zC*ie^cgWuzFVb%EQQ5Y_O(dfULjzzWrJWff2bi@vQaB(R__pg=&8ww0kg zq6NA)>?8^TAuJ*>5D3tG;47z1yE+z%YXH_at-zNEo7vE+waBK9T>k;lhp)l<_$*eD z0qswZuW|=YDi`18HyVczN$n<*PtO17H&VFwYM!_nQmal>wdfGOYk*yqTO2TkOIP(b)eg!9B3LFeP?pl9{eV zTKXx>=xNN=oY~WmOXL0tTt8xsYicZ+ObMDLpqGGN%o@ZJ*hgP>oX5`3nF`SHmv5#5 zhVTdw`Q=f(rMo0BJ~0F1o1SuYTOJtz1`HTbMivOahzGzVXqbd9;Q^USw#3UXCeTamLE(tbD=@5G#WakXuB)V?_M1IPkb1KAo& zM3x`hRLZ#rpIPblts9G;j+aTWA&BcpIOXFf{NK^%fB7FEt6MBYhGI?5A?;7z_E_iZ z&mSq{xA;wfH3m~rI%Zt`g+C3pVp(<7x%{gC6X7^~JP=P7C6`YB$6qg}>gub`|Hj@Q z{q<+rj?2(yfN#gR#2MJG3}Bsy$e;Nw$g1uc?i-qD06mAA4Y0H%^3W73E}kR$vxkB2w1GqAb-Sl3axj~}E5Gm%AG7H=-*9*pSX z?12L4a;1hu(Ls$#A-HE_EgQ6AeifA|hi5E2T$@1cXQ;Cms835x7b0MIMdQhw$Tk0lGsBs-Vyv37o zDP7}Z1l~4%;iA$rcP>Bw(Z4^%E8ByYL}F^qF=ss`0umfr)Q54&5FDp7_}br>*MI!J zeUKLqu&AK2tQjLNj-9ves`C!%Xm{TR&Ku`CZo?JvxD0g$Vvse|865@omMmGa=A5&a zd?e{g4#M>DeSFCI$e6gq8Hh75>@tvW8zwJsUFDNxu?~?8zynr)76A#uFu()`c~W^H z>4t4t^hKFs6ck|r1)d@%^h7}vbI7)$0AZ_t619jI$db+?_LY^>rwsXOnr#JR6ccK| z7Ya!zQV9t0JrFDh(Q#2(WTSdsg^*_YgesRRE_eNDXa)8(7J?%wUUypU7)U(mnL1sbj&wj#Nl@+9&MS3#~Wv0_+}uJD01tuS!qwn zcn=1+U~|Rh0W<)FpqN0f+6YU@fe06ImGUx#02HL00LVZ$zn-K`lR{U^lp1UY{}Hpq$kr6i9_8MGq&MVMsWusK{#E5))-eBb#wb1AP?I`HwV{4Gem_ zA}Q*sNoR-dQ|H#T<2Fm!lVPYiSQ=Os?O)}-{jDnpGIA(m4_-90sI0un;)%wVP9}q` zR&b^29ij?n=3Db+8DJC~EbELbt|^Z3fFx{aPjOO*HlL%?T0_Izo3K;-*Kr(>EKV%I z&s=_iM8rFWkIHvLJm}$=fpoIet=sEVTzRqmEOp@(xB!GekRaf~)@12>AVM07hZeeS zbK&-DIQ3LtSp)@vH0_8sNo;A45-dXT%4#P9s=T7arQ)I?X3#QFrDhp}*(X?H@p1Vc zD;)zG625{eFF9cRCLK}t0gLerZv(Hy5I&On@j!p=Z{QH5JO4ZFf8%ed@n zu(s?HbZ|YEH5o+K(p4Z5y_yopfG>8aE3(w^G!+t~0suq+wsPDmFoF`DM`!_Wj}Nso%|8<;AP768T&ueS(i3_krbO5*eKxw$%r@X3I<|P3bfd0xbg)K0XF61vUS(%h^cUkn#Trde~MRLMCOk zj;M=_03@0KUuCFgemx!h6Xxve)-?Njt{}n^_(5Czp-u=zJ_ua=?1krERqRiv^CO5h?Nj~#={2qvVod5BwHX4iv zqz4KMEq)5|WfgzICy0|(dq#+?v++F4S@>qp!ad2nyxIK07!ULU4>Y9Y(cYn-pyB(T z!4ej}y6Ptp&SaB$q#6(ikierf^Ki8urjcqMuHb6trxgkeMuZ5y_yU;dels{xLDLy5 zaFAurGhl#LWJQVFNwvXG9Hj4H{e>9ItJKF>NXJdl#K#zK0 z-MV$@6-Vn&#%n(2;Y&Xkw}QF23ykYgqfFz*1O40snKe6T=)UKj0bIkD2oMq;x1I5v z0fyv}Bn?P3@X!Oh<6e!J6hX){ldG)r?o^9uy#k>y16R1fh13@YV4{kF1^x*;g(8+< zkfr3q%?ov0jO@SW3j-mMjH zL%!6TO^X#aJn|4tOFba%gp$WL?WD_Q=d1p}9;kP@(6D(G^VPpq#cIRQ zI_`mZe-9nkXe5mXI>7@)MNefBRZqhy^V9GHbUIE$=X4702)Ui0-l#AhFdk@w2ePX+ zQ{EwQVt@b{Zcrk`T_cEu7ZQ#Lq~QV(%qzi;nnB^RP!Rrb*N3Ssb8gTefP&Mc26xcR z$~9`yDm2XzlE6uVE0+xO#Ry?>MTr1WyRyqx92%F9=+hjmi*f*V-6W6sf6&hW} z16}EX)vH(MHU>SD@x~cMdou2I<8YH4i(A!L1Hih{^WCc(1Ati3=-yUMYq30#TlqHS zR=yJkEqeS0vmQVc&WR9rgNld>E`&$;Fklg6b>X>Q1O+!xbmJP?va40Dj3Xn-tdIFw z5_w%zX)K`SW^wEl3L>D4(5aW-L_vLo`!9qPHEi?q@9m;7*}SCZvWIPS`DTAr17*%% zMhnh$UR$X2L?)Xyy-i~(o_7&dJE{{dl`bCLLrF3q%jKF*%(9nFFdj$<53F6gHnXwO z9uu-c`M9Buz#sPrJ47R~8Ds=*Ie6;E8VXttZiwdZzK5;<}iJ zZ+hrjw5Qp@hhN9*cRm!G-+A~Jcm>}g{vvUi0fodcdd5ViE>V?Dk}1VLC=to zMT4yn4Yoa@LD&>K5H~k|B}ES6K)^80%d(`KS5CyfxV0D9?JXV{df+7* zP>u73F={wKM33TXVAmi-4D$+`#Ij!x?2=!Ck1z;=ekc%F1Hu9zxNwJP%H1Bf13tmR znW*5{3NEJ&B_P^y@rT8=MpP8!oL^s`lhkYt3|h^6Gj60Uh1v`x{Fq4Ng94Ga{kSP7V$G%95rzyOAe5)_DGTny?hhZw=D>hvs+ z;Ai=$5Df}swdM||k`J}Q2&CaEWL@Bj6}akjH<*Wob+4FB6DFcShV8>Y=!N*7&luA?Vb=B>Bhru@Tl z*-(nQ!4*r0-Uv~B9hkygk#BxLv+UWuIxCzDs#sfX*0Sq+O z6Hh$B04zBe4EjT%kiW6f;ailEY9b%ST>xQcEDAud!tinGc)Mx!Rd_aS5ZYx ztM5+a8LnbP23)0xCbYo=@-L?ijE2Q{z<9uTpgTNJTwFXQ5VC%W5nxewv}f9Ap9lEt ztdaZHQ(FDrD1gyYj{{B=V90OG-9&H2I zv22)uWy4#P{@6+MvZ3a3Y*qG&esdct%(%Xo4p+AdvNS8FJj2C+YZc-SL=)n8Kwr){ z)rM?5U_4+v(7hg5vSi7Ys|pJii}zq!^j7z_ZdyB*2l$23yN7&&>QjeGRy9yF2;iVW z0iziPAa0sd=jKy+W|gWi@N>qcSIg;?aY97i0XCA(;7Qagqm(MGQgCZ|h8dDq8lcO% zp6%D_-83i90tjl(%xQy#!T?e-IWpk4A$lc)b^pr#N z;F#0k^3HSO(?=Y|*TVZ>Av>T_w*CppkFaTfh zK#l7Zp4kC72oLTG?FgUAJOdDp5D^&xE`%1%3wgya_(WdH$Sf<+Ip&pMLMjDaw9l!- zChZd<(9BA?l5J)hQeldycgb*BFsKS7(pCmPvR$r`?Mt6KqCqN~oF7_;6*JQ?|4B7TI(v(P_6oPyYIg8^Cn_R^b< z8xQmy4-^(&P_^vJf@?kGoq@q?ec!P>u`LaGl4#cu23kV|01@Dd;erA%viA%TWZ*Gf z*M)%?BMdd01Qfuc8hilzNWjG63Ta6B@GO#;4Pqg|E9Du}a9d$<&Pke(M^U%$$P0c{ zupk2S2~fqn{7(T#4Thw4&CB{^6#9h@ZGn`R{K_|q*D(^T+y?(={pZXWLpv+#rE<0f z5mh2^{>0=1^we`NDoX4mBR!33Mw}`tML#&l!YZ&nZcFQJZ6@oQOHLx5$@=bP(=p@K zX4o(jP5c6JGvYglHzM{sz}04N!8s-v4;T*^5A>)93Jb6Ja9QD!^F3Z#f-!Jsj~Zhd z7arglEe2Y3slx?8l5WvMH6XN975!6s^?9>LB0N-xR|sDh06CyY^yA^{aDq!y$)!*#j?qL%q| zRCO#TWW|TlhJ`n(>RHenp;X7D1bbRDu zW|{z)pmE+82R!9wrqHJ_gTa=`iEDy?> z4rexnhifrKr3QH}f{CM)VmVo0a3h15OOl{Kfdf$mAnMoXT!+Mpnx@HcDEcd3C)=xk{-NAV)DVN%U8U<22+9KMl$oNcBzf za8Xptpkx=EFpzG){dbxgH=T9PWpua^C!#1h-8T&)g&;cOlc&VA(DUiDCQ?e07vfUE zf*B)Q@U)Y_;Ob|H47M0_{S@&e#Dok6WF_QfHJvgZNPG_z7M@mr<26+&)_rZNT%HLEMSHPEC3R6aF^giqadcq!Cv7KjYqF>K1~~G<5(U)QHdccucwH*F z5ByI8K4~NKiKy((ADl%^UO$DDKZFj|J`dg}`1aEJ=U-8j*vSAqZCIH%PujsO&@F7$ z?8+BBdlnsdY4G!B<9cfQ!EnZu4$WjmtJY;+KD-(6J;W`Dw;=MYSHgsrUQL*;nT{9_ z7!ULZ4-^y>?7)7l%e|I$DaMQK{h`mjwU0KQrP`Z&3LVVB{%3$LIkGcCP$)sA8Z1aG zp^%4QQG9ApXFffLB}Gw>{|XlflFkdCXdNIjU9^Kn7|3vtpU7w-CUEnJVIJ2^7T2ku zR4_)7dIen+bWSuNRVqO(52Ba};z|VBf7vtb3t6K+h5Zb{h-Qr&LVK$5yax&zkTMr{ zVRTL^JzexS;2T zur)J|RW}(7KSAXETU!x-jyMCcPY1q+CKwMG547%qL8u;sI2w`jxhm>>I?DWA;>dR@ z{k`HlUp3+pM1FHC$8y9{L_WZ;^*bXNOlY469)JAtf32A)*F@z1F1|q#BKabfpVGNCtdL z)k;jyqRIBd07jOzj8LZl2Gb(>!hlQw7q;HjhHOY`lKqlB6M4<2n^TU_WGN+0=KRx} zcF_3T6irrui2r-(1ll1bIeG$A}Sm1l*N<=>VbYmB`(Z=Jzt7_7;eG7W&V z9NVa{hfKLbH|(?|0SCiDHxQIMv^82|6t4n+0+Bi3CCn@7sL+OR6;y6`vxM?5a z4_vk5U>%KK@I^ZNR6Zj#et7lOSLlQ~;1tmpX`}A@vOAup=Re%*2#*~yfL{9fLdOQ} zE505#E`NjWS^2! zBYqx{!B$rQET|Q?xv@wuL%ap?X~YV|b%@_W%tJJxgB~a>ylf|y5SMr@&jO57A9YZT z5j#!~G<#BL&!E$3CpN`5`S6}}t&hQj_%A6Kw=s6}r%E}?RVrm5z(vW;83hdl0UrP@ zj)(>@VACNQ1aVo@q$b(6=8phaqG6|H!6Icr6aN{sfvi=C6u?VaE5)U}Llga1C0P|E z4kgT^4AJe)pEic}RpH(4peq%kAL9q7(|;cLGu72M=q|)e=g&Wt-ml37hsa*kk22K;o|Jp>{zt+J;c`$OA&vI*gCQVLr(?+vU<|{rtO~c zzz{UdhvbzYZbG~n@pQx_#GVX(G`;}w4~TmZ*C3vcXhH`)P*}M5sR`r8Ov6)%H(+e2 z>7W`TcAOrlNf}Nh!!M-+*)u6*VK5h{B&?JNZV4o?s|$IJSVUH^DF~(s5%LlQXeoYO z4TA;|G`~}nTyhpjb18YQLXdVu=2ewb=blAkEuxtg@*12-T>_*io(S_*o6&8sl(5BN zwkZ`rXng)qI@*9eS^zw<3Yw9*=>!ij zxH_115^W!T868a@BS%6Ehk8y$AR`R;612#1Qshjux#{uLi!;7fUVj1oxPW(FbX(y4vezNLf>@4t2jXdnZug?t z{lS2&?)SNAv1dH62+i`DEY~20caQdrE$wZU?^bIOHz3YHG@%0?z!Kt~WrYRH@Z#xD z;MV&=2UHl5PVhj$n@+n1ok1l-FQV#Xo>W#NArFTVSLj5JkZu&Cc^T|5I|Bp`(L<>w zN6`g{anT%cw872jvSSDoM>em7SwClWJIp7FOi)DRm?$J7NrFT0s|E!R8j}4{vvyiW zRI)M|xDQiC=1^r558rYYi)&cb)ZrOa^pF3grlx>oj`Mli8E4W5)z}HaZ72f=rY8$- z*8k-TP5>GI#QO}TA<{;$pqQh0W2yfl6Fu&Mufl>=h<$yZ)7j_)Z}$8dVwcawqm6_Nc=7bJL-TTHVl25H z14?y5cCrt2s@|VV<%7?s(jn(j4ZxNj#5v`Go}&ni9=Po4ldcmfR|po#p=C7>R|>!& zs=+AL34PoI^GR^SZE_=mTYPd!%;pG2E~y|HnN{!!SOl6B7evQ?Wxi8p4ca#5*X?Ur z*?O3V?d70lM1Rm0zf8KcG5~sH{UuWe&>esLvn00iIrfx`sV?Y|{X`psvr+~3C9_A< zgdtgIR{9Dp>js4HxR8;`I~!UCKD6{Zmrh58MS}(;Q+^(RT+fCPX#Rf?d9UqmL|#ul z6S3Q{Pq#IMPMKgl&=Vdw5slu3sO}U!VM3?2$>8cH#1|3y7n=zk@W8p}p4*Jq^ZXuT z!ercTALxJzBhpD8s7)S9r2tzy246tcsXQhkFf!5LictjTIYfVpxvZ*0z50-IL0rMW zNeW37TvqA@uT-Fc3X>#-HH4Q2bsJpAQ8&tfNbo7Ds0vpLss&VyarefQqm1+ zMPSwT>o(Z}**_wFnRMx4z1dy%H^paB#=&>#oex@n-ec&HJlg7;EbWH_#ta^Sv$DMO zgGIAkeF{5Q_8W#FlaU(Uzokv(lwTm7C%_uypp(zk>P3OiqTq*$o=4n+_(#NZ5W5ry zb*XX~Z-VhaPkZ3YFd+8NuAa8+_}b?uYIxH=|0*+~gC4*^rhBhiy5zIu@tuXiqm3L62O%63_tywuBJQQ11dC5{SqdCusuNN^~+5C>hirBk8OWem&6)>XTLK@m}- z<(yR82bQyDj-x$CG|-ZE%^ICW_x$m8YHkj)rnup(bIzk})tJG80rMv0(I3Bb28|z* z$=YR~V58_JiUJoZa;*0p;^7kd+5((5P0mD#oF<_{U;x1Y zBJ(m$d`du}!GumlB`e{;?TSPg09`0KHc1<0z2=jtYE-n!c@382ESS(N>x5cWFH}q0 z!rGXZTfnDCY;jWi^ob*>vX-AXWI9*abm`i6u|)% zD6>5OlK_U9B%{#FvIveIS(Ig#Q}UdVpiD6cnXr^wmzz)8srk5LS|{s~^lWTI=a)%0 z1n|V6pkc`nps9Js%d~l0DKo|mWAjJR`^{s7oNk5vC+(Ikgl&`UH#dh`==;+7>h~VfNE_wV|s z?d>s}?|NTC{AYWF3~zi7EMK*1J~h+b7&1p}f1sy(gkPaO>gm@6; z!zdr3rrIMkDaS)|#}A@Yu*8+0$Hxiip)5R0B<#XqFJJ*zs|*kRVzLJW1`iBq1c=~z z4oVe+6}|T`#Y{f-4fEmvzGBU4%b zR>=X%^fU<=swE$Kzn1R8MtFDIj(U~hWyO!y-9w|(n;f&5U;4&vt8C}s(MFnfZTK{l zDI>D!xmy=1TeWQ#{r%|=>Ba~7+|WKB4xn}!=OlmxkP#)l4 zCc6<6abH&Sh2s11*AV{}U17={^8f}JPf^jDuh>@T7Z}WPI;O=)xjhgH1*xK{z0zZF>IQois92 zt#qNfqBcOga_7<4u5Nv@d&@`LX!yP-iRW*_?Wu-EGtxSST-*0m(-}WrqfMgQJ~N-b z@QKMzS!e$GeHpbnMLCnD*4mo}O1R?oVnO{_OO8^x%a^ThypIs-I|cr+ z^+gqrAk~2DDdfYBL4iZqbFyqcjRz9Z1N;t%_Yw0Gv2$_ybS>z2#;G&p4tfByTC+oT z=#Ll@=XX#=pB4)Q0?rJS1Wj1RJA@gj!z#tHUSnfppY0!Vnb2ZZ2Fe37!ZwV22u&=7B60=ZZ*$%*|LIEf) zGnlHX;vepb8=9~;iD$)fFzfZ3&&(45B@7wD{T6_V(GW+0KzIifzz%~Zu!(w&kdy+h z=it;MCuB5&;X1|3e5^}y>t;|Xg!Swd(-pwda;i-aX5&sdX$-yl(N4_f;g$#bIm4Uj z!BvmarwT4(^)W+gN-|Xsn@>ZJ@U1f3KIe5?Y-6KqhU#KAe}*?S->u z%%B$@`;ca3V(SsNE!DCmA}88awCVY|G!6UVB*2nH4aTbKnrLGQ@774Dz_U~3G2&Lx zc%7B&B0|`I5oA*^9!OXZ@RNquC2a3vckHJi|15Tup&qXXii#ed7Np=GFw&fTyt+Cn z9}EWRNM)r9KuOTVU`ozH9irOW+K#I4H*%Nal`=OinMK*Ljvioi!2nCA@CpMT0fg8? zJeoR%mU+nwu;egmAt~FEu9b1B+ESoT;5;khj#B-`2WdbOzl1v6ZOy8e3xD(3`E>OsrUbK~QN8iyR zeI@0o+VU82Cukob@{(Y?0$&Ofj0gIY2YANeQN)Y-Q>W2ap3VAkKkIY1c6Y^!HCNeI z@V6KMM|5_>H8s_8?~^kN6$zp$D=IPmciv|5&bGfd>tpaUJwJ!ObMc9E+GLzdsj!W} zDKMN<0gx23;e2$^aL7&9GO_rhfeWjSR@(wJvGQ0Axf*rb>_$hgez=j{j*>&sI zrB_zf-bfU>4g+VK_W$u_a^6H9-asyoG0;!r@seI&-b~)nOTcF@eg5o;G-=oXF_2d` z1n5w8106itKu2mD>1cg`>KdD=z9}g72sPn=%;sQ-LI5HmIG>{zuX^#}aX}wa95O4O zD9pgpnRuEIPaKZOO{YmivqYKg6c`3zLCnT5L7^cxU}7GPcANoJXHXgDWj@#J612-Y zrputz3%bB0aOVnf39s9e9Pghprk1?{0X57Mo7{Fw%1 zqzkclA8$PNFPb{QJp(FRWZypY;&%Ep-c_hUr1n{DV~{Smc01~0kC|XR&>uZ87M{VDJhhu8L}4);V~i;IgVHV3Uc@g(BK zyjfgt9G82c!XSzP6mK>+p`{1zym~%;a)x_SP!8|{h9IkOghCmRSHOccX`rI>qB4wV zm@GnAqY`)Xa048e#tEXc949DTB>|a6Ga7JkwdCcTl;@lXC<~wpNyZFQh}In%M8EuL>yz8N_w1##tq&6pj~Dg` z-Kx({U!4mfjr`)>&(dQX%VnW+KkJ~56~~LaJGZ&c(dPKSbJQ6v#shuI182foK1{An zGY);KXRRB{XJ~yJQH>(4Gn?90J&=joQHY~mDd$BQ1JP&z$SA3Jsm^Ul3fvD1RPyd(Z_<0gY+o^1f23x5A!{hJMtt&SQ-VS^mhD>>! z$V+*B@)c?9eiMub`m+bVfX?634;{cyS$+*s-ZA^31KQAKe)@Viq63;-c>$tO9*Af{ z7kYq!!qk!3IKOHz%^8I5QIaYc^=NRSQm)m( znZ1e-kuz8<%8fZZN=U2JIvhUe*LnHXQ+7dv07MRI(vnCOE!;oQbOTbEu%G1}ahI)v zrtIW?R@O97Y587pBL00F57V80`Wq$t^HPeBq6x}iC%mOvE^McA-Z?WPVjT=8bE2#7;h4QlFb3XdHGAA3 zy5!{1bkVGl*smog3d0~ogC2Aoz(arl2YpEV#YGMh*r<{ME)uLrnvj+#LJ9~OiOIBl zB*1btqBU%tbyf1F?oi9>@H$#Ca38W;T zLLR`1fmW-*vR;Jd`!mv#S6+JQr3rJumiAVweK(cH0}1W{{`NnNcv*t?GgkK=0P$+X zZr>0Z%a~52@%qw8L^o*C%X*rY)`{&IRXyMVenrkDvqsV9ut9$E@GJ>191svTNB|8K zBL+Opr$CMdA`v`Vg;F2^2E!`Wr8#8@P)1Tgc3~QUZDh-ukCldJt^`%cNE+*vEij+5 znQhfdB{{NnT`dG$1=Xb18Mc{d({{{eSnL8#mF3lg2B}v6ENVKStC2yF?{g zP`#Du&ZKd)Eva*!BSKf3p#CurS-3Y%`A=ctiCpfpfqi9y@qqCFQ3jmw65{lJ=#1s) z^JfubuLtx#hwy!CC}K2da+ee4B=+9t{@!WP=~IT#Hx^9AlHwpeX(Ll*t{ChhzzPru z{)^cp@I(xTTu=Z-q^tup0ag?!V^Ww2Sy2y)&>2nzoB9xKu{<*)h*>ZV_7o2%Zcrp= z%9to+oCcUN!WJRON+oZEAyTIh0fTP;`FG1`ScaGW{eBbOg;(x`XQ`MoGVqg|wVR45 zse!jng#(-Qmj4tw?9B@kJ1u8bLjOwJ@;l+DIiLlu7$Nl#mE+V4>h@L5;a;LNL8V{#+G23y6z z13`lc31HM-CLRr=hH%LzP>Yar01v|y@JHYn0fHhGNU$W$UafQF(+? z-+xH7B77t4Au3IpNF~0>F=aZ72z$%Wtp~i8?}5U?g?un*p9t#m_6b{!k;Vhv;ek`p z$g_ya-O*StYhx|!DMakb>3QBv&d=q$K@;cQ<<{@?U^c;c;8-3wd+IQ}==mh#^R}FU z4xokej6wL*fgxd{16DK$Va_nt0TsNAEi_%>9#NKgsZ7YLmKf5A_QHy!5llvK^Gfg{ zh>;BmUEIDjfK5>1%0N)?U(~Zw1O|m1SicTjFX)07`vj0~-&aeGS>x%H3zy(5vDUXC z9^CQ@WgL84Xx6r{?i~PHJCeEkabI{eG{tV)5d|2QBDHIO>nF1Ga-L8!_X7C=W0QSU6`iU3b}Z z90QRmYW3`uDDVi#I7FwMWe}ofXx!jL@T(7na}=PFKqg!tRwm?I8D+yDMKTH7!pa?$ zEmWu`gsrLrk|cp+L?2lTD%lnRu0&BO5FN*_s*puH_8cPLtSf2W%+~i-H8eF-^QQaA z+uS7l#O6xvI8Gf)hVLiMqGqdmPAx(Io8kR+NFVUnL)R7-o>tG+^rj%()0+%3wiple zvnqPtoA%zdF3kLyy0My)RM5AF=3Dm$V zvsDoX>V9SvpN*$6@62-u>x&lO_PUqRRw75g0Io=+}6Lw zhxYEG^zDyHTU>2(FB+^gdN+AC9ksHYX1gPUe(*V3tFU4I{-sM7Z9qBH9qk^g4Fj@{ z|EP|>fbu3py_CmkSL|$!YSt8u z2fEG!xtS?+?IkDSfYY($0RqtggTV;Hk8qHHN|XgK3D=06Q_V86;8wviVXWI71h`>P z<47O_9yhP9i_oc+us-Nz8J!A#cUxfvT1v2U0h`BhYYQT$nUG2ag2rkO)qCh;|5!Th zyo)FauOo^V*ybbeJVB{7I|WPR3>iPF?4ghRC(#}+{|@W^fWGWOhyU%thWWz6E8gpV z3&Jh3i%l>d==UB_uR(bNH;eJw8T`JupWbvMm6erhl1YsRj0Y0H1H4b`M~i0CsgnjP zFvLKKgFqc3WzfNaaSaC-w&4<@TZTFGtdQhoN|Pszjwa<&jc*XuBxO?oWe+huN@N4ndSPh0#`VP$;47Po;;s(^j( zSlGXX+02I)E?oF=C#h|-8g_>X#smG@1CKuX=%AXqrgw0g;r&7p;qc+ZbmNb1pd&|) zM9@vac))m|?|b09X~XFUSInfbgZY701)2n!3inN89f=;8E@6;h12WMwRy@1Y;LW0H ze;!pQ4WeoQt46Qf2Nu>A&Yf7bfijy93rwQ5a@t~076$MPn5^MhBb^u8v_Q1^C^vij zv?XIc9fd8vP!r8N`h=X#f=(9Gz>4*R^A_!D%BB%#EG3@#Yb_ky@jhjI{DNbT^b90~ za+y|o2GKkIS$J%@#~%pB&NJ&7Okv*du?Ma$EIhZSwZ3&$oxVJsrO)Uw9_Vu(SiWM_ zUvT64YCOYBwr{7O{q!aZ1iJE}yLhb|y77SVKv#HxPax0E&ZZ$l22;V@d|EOyi!zch z496W6C30YR!a#?iPQ5Qj0IH+@Tmh<@J#Bv|`J^MyP-d_adK9qIz=S(15u7V!9@#FZ z9E1gekNMs8Tx(ICHxKm(`>G9)K4?lGPJ`wy#!2n6 zJt-0S8Nk_Djhl+d*MuLT(5aVSBki%M#*|WOpINBBm ztSVkf#Vc2|h1KvF4;T-0u?JF9Q^h;+!Y>_`Ad26Umj^(Vo!zl_cUyLdvYYl&dUJ&U zDGk0%s`F)2BW7bdE1YojdCCeLbRdocq=4$8!IhjdQ?pol=1Lh7p77E`xlOxMr%-vy z3E69-L-Y2M8Bht#+Cdx`nqSv<_qJz~9 zqHcWiRvOp54XvYnHh~*cw$x)*?bW2y1z?R()JqDmc{8ki01u=7yP%+;yfe%SUxm)l zW^@=2^f3?MW@4>axvCcTf%uyzRG;|9!?RhhzVdP(>!mT*c%bijAS)}&0a1en4RZ4Q z((8N#aR2~0a7jc#RN)~*hfqq&F~73CveTpLKBSTLo26|IN46dY{1L&aKS$+(Qid{bV<-o>fdhWx$6jFyKj1HF3l)#6OkpN3}Q*@9HdWO(jNwZt%>rDjsh28N2 zA9$MV^As;yw5Y?Ia@o@+7!UMo4`5H$Gx$cG+ZHD@H8qO;SvyKf+QM#lj0cPdVtByo z_0qtB1LYE)nt9?GC_O8cH#m=czV12DGKP6wMjBDSg+|u{dy9!78627;c! zk|p`cd`(jeC~wFxH<@YytkRRi%cWcPRMED5;YEQ_!E!pG`2*QU)d!6!Hzm!bLtXN& z!w3^QrGWl7;xavo^nnRuM?XGi&YX6=dQ7>tQ*1Fxj0gIp2bQl~{RKM|x~HwiS5{Vv zCk`trD%vVy_>2eol?RfOld&Hv*8x!yKn)_Eg<>F;laq`6P(Alpe|KMuv|tsD#*q!# zP5UIC(P#sy#2*S!X~Xo$tU#E=L19p(Q4ve+#_F< zEH|$vnIto}qr_z*8Fce@QaYv0{~RSJg+UeLL4Ij;l22a85D3qHu#f5hzH}JfTt+7Z zKh)S!lc~naq?i0>I_18Sf!}b`w^qpgTZ><5i`~ktZDBP$#si7zfsGqCCT%Y%$69>& zjI3kzw6t`GI76$c$vkm*EMJ-W#PdLUdV0k2o!C?^02F`}K-7Q%tv*MRcn*o`kq4(! zr3H>qR`Wp`7}!ghfx}@A4X{8Fz(>F%Jrujpj;Lp`Fu0Pn5&+eC2GI8O+1P;Ja_?C- z_IpjPeT|Z+Szsrz6v%qGEws-wRzkBtTOzCYB-WC((mqiO{`z5aY3K>_CA0cGT-}Ic z9>V*z4jiqg*RVg!9rA;vG`V>*^oRSw^_!CBVCitMo3&5n(Vr6dNx!gqdpzUym5o<# z=*9#6!UHRci!TcW>5=wy_Vw3Zr8|CqdwWC-ukk>y_W%#Ve6)dHz7r|$i2``ynW(`- zh7iwGB~tL&0m{N=_oPq&2Zb{ju>g}iWaIb(+vb@XNkbxPxa7eGd5_&p-cU37a0*ls zo=Eg#SuQ11N6EojA>sIl!3=3pYf!k#O#zF8f2vat5%2He#n)w+l^JX)f2=#c6VG3rq+-f(L z9HPBPBHze0Hn@W(HE&T;(1!m9t-%0Vy?bw#GWeJYJoG9KAp7@#jI{eMzWCzsnkgh> zd;FLx4TJGO;&}jjv;K}7&6nEKv$YSeq2&+#yFDU?*La{uJ>c{Cuw18(HUMbinI{dX z7)0d)P%%$`3Te$pXeyrg^WwlxfhH8H5IDo-hzPJEFo^;q;L(FAlOy>AqhJtVMu9C> zt7n)v&moL!0D4@YOzOs3m47&urk;eeuUbCo$VX6&uYZ?@G?hiP@61}Uh4`=a$`-?) zq74|J>w^H7UViVPJU0Tt!sJ>H@=hF)BU^ua2LJ!;`8khF}%hDUFiWn$+|o$?&R_pzxb8p3l?1P zJ&eK^b)^A)Q8#Z6A5r%aw(sp zEn-TP0@0R$U15H1ha0K$Wzmk*X;kHl{rb>UoV#^G?HhPy4v+E5MsAnoV+2iYVZY$D zX^(##m849g(T&?_Oyh^b3|7nh4P(!vA!GQ62pMV{0+gN-`H;ePY=Ga4jquT7d~iEW z2yTNSIji@U?_}DKrww~cfWMky+5z~d)b`k=8bnzhyDZ7F-d((Sv1{G8ZB}bvw~gCy z84o152bQm1J;@G)-at18w6&{&K!9%j+0C?V+t#*<7(U~H3JZoqUSKCp#0-`P zOPw6t@)8&UG$l`>8ocsnO8rJz4Gh2BanPzfsC9bXi!>0g|B?LM zfR{XTuBMFi@DUnEY6Fy=9tK*BMK^r7k7@wGqQkgg2~7%aMRPWl_$JbJ@5E@9E-S+~ z-0Wdo0?b5F89G&pYf_3SOn@p+>6MEXZDXF!G(i7tbf!+D%XpwqdjL-xF2>V_tJ>3v zs;Vmb@egmH!-pd`eYQu3;WZwB2ZD$nBbG-+F%tz^8Df1=(SLJ1p2qwj1@aj@VI$EP z22dCt@-cXf!$=a|I0FF_I_3dB0wJeqHx0yotxOzFsz+aqlN6qk$`z484v*vvtmJ%` zYFi6&(Zwr}m0*$kqFDZ@H;48nPoP6s9+b?2A>xC<;P|?CXmDVc;E^ij3=y}_dN}M) z%E$2zvna`KqWMQ37c^OetkVI9LObyy>9XW0f{#D)0()p`!$xLCM8ELAF?#~#&AO1Q zMX0V1P-bcvWHIp|HpRcX^?+oE`WzA3O()`bhn=2Lw8b|ys`mJb=yx?BB)}7Zlx+zR z1>96dH*Y*2KGd$E(%I8xQnx4=gWQ{Ttg3eXl*;C@(Lgn}2d6H8zIN zq-l@5;q4b5;2puc5zFJmT^Jzk)vH$z!5lz7w%O!k`%6B?%rO|;FcXEDDYC|bCGtR| ze&M>F?POA@5vR28M?63S@YG;RuudApK!rgS2V`ZU0Fx$2vaD0=T1gTr15}No z{mG-L*4sKp9ss~MsqPK2$)4r$_n>qu15|s5l19?zG~Qg!a4$q>R6QcV7xRmDRSHm| zBrMEgUEsX$kcsJ7f}B&kLDfT(YOi6+Vj4UwJRw>Ipp_K`v^MQHOnWMopBxP`lVR83 zD2I$F>amMxpUq?jH;Y_7&^QKPV#L5nksrAApUOS-~j zmWS%S1E|tB81EV!LPvdrgj8#t=FRP+8@ABMMt*G#0zMqTrt=mm#OVbr<8f0rDVK=hC@{yKo(xH z;|>kH2_7fGHwV#1+d~7)*$WTK*(iW0vOHzHxje;Ze|S8nueaTV&f`>G?mUG?lkq^J zc%Z20KSP7mun9Mz$fJ&r(ZRQGS4; zzCJW6WQD|jC|m4_18$?hbE)NBcOf)ZUM!yfXLZat2j63#06+MT^QDg$a-eUx5VZjOl{$Kxcd4p_LDx zY1zSN@N2PM&l&&a&v(;v8=mcKbBw;O^#JeE-GdklMCqBR#-gI9vOJzTXFn9ito)E| zjp3On1*isht(iu9fAWBsy=vTy*{c#cm!y~aloJ%*aa^(+o7_uNTJCeo0!W)t_X7AB zUSVDef^rf|X#n|^j89UNC&kgl89w)Ge(lp?yvE`PT0S`Qvp82xJ||RJ+eib`Tb!X5 ze5565*t;ee4;T-$;(_Hw#n<6x^t)DAT2wVR2k7Uw-a;R3 z+1#Se5PHl5yjMemrE=t>;~s!%d(E0Pc}=#R@4c%Ny7SU09?V`%qkaCdF?}=yPa#gO-^B1rz$|8R001&Io*a5JJ#wj# zd6Ml0I<;nqHje?SKh@(Pm)-QWR!nPhpK$l&T=QIYKp`qvKk zHiex1P&^Zb-}GZppO3)-d5Zig4}uBC1KsHX9}YzwU;jRy>?;?bD2#@r6+CeBgpy91 zv4!s=ydb(FDgK#L;~L(lkxe`s#)9~i(^xVEsWjyT+UcLfwY)SaHkSjAm8OO2_Rp(* z`f6qu0iq1l+!mxvU9iEIH}M=%Z(Glk)p9Q@R!a-h#+|ZK@bse(;qQLbU(;3Nfev|K z?b@}O^^JiyaHE;r9tFF1@1mP-{1MgH^Fw7O9ES&Zd0`)7c`LEz@y8!euBi!(3SvK$ z2OG-8Oce6i{~?z1FgA?D3_!bA^Mfv=GX#skI!ZSX*G@gpY&f=#c1HR9%REYY{$+C;zk^=-IW_M2@8$8t%m z`X&(hz@PFsk?R5sb^`%xbdZ|FG9Cs8*K!`ti6R<`!N}a#;4BmRmIrugaD3x?cuEnQ z#PJub!BwnYFeu{s{mJ<_nSDCe=)AGmTr```sCitxXHkPA21^W*Wa>FI^#9#E7<%0h zOvrC}=%>T8>UQkmFO|Yp$jOqngJ;J4gT`(0PrdMDA!N}DU~aMYXp!d^=@aJOD0uqG zhq)Q`lM8LS**P9~sA$!vaCiH6d$m3B#DD0%d;i&9u|($IfX@9Dv0TNyNcnuIT)iKP zk2k<$S#mQumh*74%E!2p13nXs2NK2u8No`*$FU3paR!%!F>;1TfFFRD=)l0Hk^Hnh zZ6+P|we7*@jNlQPUH6<&FMyE?V6pz7H}yZGzVNq8quat!HYpVc_x<;@>L<<)Tf#XF zw`8i#Or87bmprLir(?$LqmXU4nDqy!YxMd(C#H;^Qxe@)qL$6Hd7@rf)2}Y^z*U6> z4=yiSbt-;EeycqVTy)VTw0qYsdS*SI(lLQOaOT2=i$A{V)1R)w{W>3eIK}`#iTJa>TLNh$e*zsCll|( zYHJu6JOpiW)=JM{v8-;R&YSsWv@YD^tQ9L)-|Mk004!w}uNkr&Y~d`Y9GJ?vhIb#H zls5nBDh5~{%d;h9ho*474rn_$7zo{o@@1Nq=&2{sJ81f6JkZ%5@X(a&anpIVy?Vd! z#V^s+6HjcfSfcWKA9>{APd)$KhOgtsdoga<{LFc~mSGdsF{WSr#{+40Z5UvQZb$x< zVBo?)rpZd7UH%EQG4lc{Nj<4u0E^XT17zvo23rCdq2l9|aUbe>ovv89=BEH!SIRQ% zQl>Gk0v%*Y5MWCQ*^RY%&AZ=uhl?Q!@o^|E0%R|(E!lWUi;4s#NEn)+oip7s9_T6$ ztXj2dWFTm7!VM;`J#Ey~RO5-mAJYE)yocKa9zeeG)vwbz=YGOC!+4-ydLS7m$MbTY z->xSgHoNnn%`EYot9~;^;vALbkq3ZZ>eR|D*)oJu$9Re&tYXhaxYXcS(E?c^I zC5j?oTf~(>Qp>sE#g4DDY9ALyu?zWYYJ4@Iy*V zOY17r`jlS&Mf}&l{E5~-{ZyZ_+Zby+&{-a6^rX_^q+zrdC#~iXYj@OKtBZ-88jzL9SJ(8dOFeL9K>;6w z{FC-JGJN<5`p&iAB@YgcF+qCZAAkQVty}kmafk7M@jxegfZwMm_D(Sh5|d40hm}J+ z;v0wxOKegvE?zsGLczyCNO7Ct?7d=qrqC9rSnA)A^M!lg1K9eWMqy0%RP7wrd}QxO zczM)rw;p#M&oSOuTeh)nJNMm!eJpS8@_p=r8FPD=2bL8s{uS;id{b&ym^OVneg5h% zwo4|_neV;l|LF1mwA+=5?iSNON7>HVm?ayvHNz4XZ1hr09Af9|kmyZ~DVySX7Xw14YrU5=H)w$Id+y>@D>xnsmg0sPQ8f+?B3fD*fq?gpYb z01QxrC{6kRr3rO z4qCpBpMd-*Sw20jH8z#L@-!INx6~>r0Bn{WwC>_G9#3DVn{%@I`V};O;41( z_Enr7hFT*fiO5`c^S;5lW+}zT0&pj6}@#~)O(3&-4tftVf@Eu;(JuP-mD`(aG zM;yhVTa&XIacl~$im%q3bGdO0IL%VJ5!)vIyIY^)R?B_Je zqSkY z-Vjld00r&41n7$Cg7HA_@IX<~BV&Vh0I#WtdP=7a=B;1<6#e6Ge{BP2|6;lDqKnPb zn7#Ls*8({0gn0l@JQHP=dc2`hY-%nAI@*5m#TOrKujzPo)mJbe-{$@fVSwiutr`FX z9Dr#zpq)mf0brs4=={rs-#Wwp87J{BAM81{Jvd$fZ-%jbNRvS0o!O}v_;~q{Px$N? z`;wOG0m0qlpwi0+|>dd+)B`VYMihq}HEN%6xeOS0Y0B*mt z>|sAEEIh6L@V~xzUGC@a`Mp+MTl&UAI~a=lR15xwn!L&06H`Xb-mYa5AvGW?5w4=? zL$C8dQPHYPa7yW;xT&@KJ9huS?xn{cf2`N_sizG(|AGbd#sBx^h~zO z#$l{a?TR)NRp;oy4w}&P0mDx*gQffVdg|F!<6*4XdN{pdJ*D7edJf`-#?xR>cEC5B z-u0i%I^QOr=jvsP7r)b*rrMI%9)s8=@pzzhZTf^$u83FP7wN)ZebI1Zv++Q0_rQvk zt8NK}?4PvP&QK^wzx~Z^^!8hCwO2IZ`Op2tdGzJ4e9Zu@PQ20>AjRf#0iuYP^%zLW z&E-T%rAwDyx(l;NYVM;G8}6Vg{BXfC-y`FQnrC8$>XeXe&Bs7?GU(t2iiSj_rb+BsBf9eeZpBmlHg&81Qve+m4@z~G_IlR>Yfocrj2%+#frv5b2r7PrNr zzU-Z`p6FW#JajV zy7?zRrro=DwO2IZ`OiB09Qx|lzF|(xZ^xAeka9McGl;@4i4oIN))Sk{+hKN_S@9I1 zZ+}I_ESxrQF1~l?K=4cql$fQNpYHm06IM?H~ekH+eL zMyQJBH@@JS36t{t)Zq)M7yKb$&G>F%VcXtASiAl88v){Pjn$5zS=M{$hqV-bf7nT;L%4P9aLM_vd9|-_aRp@sqmh;Noo6CKcuQU`Iw1fA^^e4YC=-PLvY93w@vj8|?0YJt) zfUbW1$1=U)bs7}h4|g!+!U8bH-G+V^sWE7hP>W>F?Kqe%Nay@cs2@qqIh$; zCpe&_u0dapmgFwSXmQCCJQ*ngD_%`)9FSw6pG;S2@gc7KUEFJU?CY<)}=s?}8 zi1(zVc$Y6+xG?4~#F`x&7XSdSbA(!Ctb^&6XQDNJPM`m~HXy6T#We)ufzJ29@}lCe zv!?X04Ki}OE(S(`vlV|&fAvu6gFatlz!y0B z(J8gh=4yb&`>OUNjirzLC$&YL;F;g}9FBshf$1#GlUaa_==96U=U{(Ui!E`rcy{x_ zBrEC9d|0y(aD8ACd=c)6OI@UhEc|$T+Jt%Eiy-#Ff&p25@S+)$`=$qQQ0W7>Id$k{ z%?JN|KRx=$+P>NGF1GsQ*|X`|>%I%1mDI&89hdqLMx>81B$a9qg_(^qKDk_NF5is- z3lF54(A^&3Q{nd=I&wM=eK;5Ii8&nuKc8_Y9=QXCsK@of8k_wwAjKYRwBPLsd3KRE zw9^WCcO@r#$}hkCa=gZ;?GFYN7Z*wZ_d6r)#>Oq!Ll^U~E`Y=)uirB@W%Qg9Ez=iM?WVqH zyRq4Lpig-K`?J!pKkGFBt+xMkR6WFNK-Wg-Z}*C9 z3tb&1;GKDpEyGO}z=^?=TGk_P>9}zt%ID0P!{<4f&<8xgvrVa~4S~xpyR5FW%*COq z$<@^jU$w2!F91X`I!iC>0U+57fb&nxTHWtHQlqG7^%)`C{u}N&Cotc*z(+K;X)6Te z(0>G*-rr0)ovYd$W;ebLjxH^6Oyn@)ZeS>O1R^PZz#>xco!1C3rC)t6}8|col z52uOhb3;Qto;bXbcJ3^Xsxd|O0KaJayWhV7K#R|t?EHXnNo+31mf=zi7Gh5ngDKuz zo}6rzcIJTG`PLhaJ>h{>t5yxe3mi@kTK0LE5nG0}wYKdtZ;P=OpIv4ZVOj0L!oo#r zdPxdCv~tZkWQPD?>E_4vHrZtE&3^e$b9||ClmU zPdt&n_x&GIN=o~;*kG`z!&t!xR=oa&HyOyp_hgy^vve+V_twh$iRo5+IHh* zm|;q@@Kgc1z()t*^mrRfycDwV&-MT!p%lWg@xT;V*tK{QpdahIevija0cR$7GeOHD zUIw(S;V9<=gfOl1f@N*NQ+$uX&ecnoE_xNl==mfZVEp)iqcwHB9f9|MPK(#*+{Tw_ zKoB!g2#uapdNJj^c&u^hCkEc7;mz1W2n4|fFyN}U(n2q!oO|_Eg$4I(LAI%?y8asQ zT?b!eYdXp=U%9m4OxPc4*zv{%&7qL9jHr2Z3f-I1#?PJO?yIir3#kEFec>t@8xzh0 zD^?c&7At_??MTPAZQV+@{Ol&Y(@&py(-AF+D>He@3G{;>{s=&;#nzAjZrD4~a{;0- ze3tRawiJe*$~5coY0-zzefLYq5*FExNjZ=O~cO2D}&OiICs= z?^TNymATu1oj90v%{e#`|7y?+V%ZIF(g-AU0JBZv7@6VCyJ>3RZGny$WW5jtvhW@G zr^m7$0?3>Luy$iCo&2hu)0>{7Bs=mRw9$VG9r5MfUbv*-1_n^tR$hJ$QPnjlU!!NX zJmmfCl~*o$fO)V9ekJHjG=GcKK|VxC2)BJ zXE-%a!%_GrV36dctV!7OG+qOQ<7*+VbOQR* zvzmj=pN37#VABXqk4pJR#XVS>`&n{wazk}(gP6YZP>9Z`e;Vh>G*JWgsK1!%n#uCe zT~{q#!eB^*)@I#D1b5Pjf%oM8E*QaKuV*B^=byndTu*o?>#FG4-=d;t@y^J~FXFz3 zNRMB2Wx$yYwG*&EskRD#MI)-LLQXY66=_e777>^habcP4SjEt{(3>FkID`bxr%YwFz zr9nFn`9Vkr0d*i!763!U_{z&&2-N^S72x|AwY#zGw+(>f!z7Pyb10OzrL{w2v7hrK z9N4}T{Drt@Oo`VY0GuY+aIc4wZYwOjyp;JiZrqr(eMfntwrN~oD~)U3hGQ2par}b( zod^5goG>o`RQCfK*i$yLsaYIP5wp43f!8TFcu)W>?rad}y*}$dpMsW`XTE zt({)~EMBQIfZj?zg;@hhUjM8mOD_MwEw_60>fFXa=#ZP{&ZpS*bXwpAOcrC=5K~hC zc=S%vEZXPMzZ-D~tH-nSs-=t8yX9i#)h2kH1aD7@RS(rQw%3}NF?Qa*SoOV?WaqlEL&P|%)^+E zbrL*4{!n;E z>s-X6t?-yCb#Mpp~aS5k)M$JWD%&F|7kfGwWR3ShU_Gs*m$ zCJJ77=x(oFaLuu1^J+_8FUCw@%zGhh4f<@)%;X7kTkUh})^=}gloz_UanqXdfboFz z0G}&U-xzoUzXo}eZM#EB$#(kL%{NgXaLoH*yLBg!z;oye@6p0*P)sl$=vfb7cIhV= zfO!-6m}6Oj*`PX16ycqK_H%eD??2J!Vznl}eC4Wd+aWvdQ!wrOS6+FE?z;1iW3l!3`hUX#pXu*T zF#Yd&4`9#Mz}m*fnV}#J19-~9hNfhkza_?G9M^>(0#-dvK{(_k|Bf+ZhHrImNN=n4 z04kmpD_51`$J^*wGH7k*ki6XSc()=pRmBbfS$u97uPjCc$kyW=IlQjNUc?prwBo#` zS8$Yl9lhzDEnXZQA)pJ`5byWWCD=E%LzAM@+8uBF2xkfYBATwtK96TX%DA~t>2hyQ z4an-vSI5}a?>z8O(W+13ro}*@Lt$0%N-AEtqC>L%Na%X>_4oa#zox7G)B|`ra2%cv zEQ!~B0HyCND_ruw@#>i#4`cxpK81rv7w|m`An{KCa#=}s6HN?mqf*~ky!0{VM3iU> z?H;r(-xXIay#nuWY#D5OTXJ1V#rv2-o6xdW?EQKtZQQ(*F~b*%Ro|OvEzoIiGS1jy zJkZ}ffScBXxN-ckqx~*evXo9et$ptz>u6hjSnf91KD!UwZ%j8HXp;wm!B*dz-hidM zds|b)GhwKYi4(AF_a4jgJuAs;%|Q93faRwTeJ2Wlh4OCzhbm*~2O!Jir>a-1SaVgZ zddr&FNS60IvFexx`5D-|_1Sp!y@?*vdA-R%V~g=XfAav&s<{p~s8>7M@2`FRn>2oW z%pZUqRhejVN$AkMi0wW$BhegT`jglm$jC@4Lk|P7dWj!%kG8fqie*-=+z@v<3INP` z(gsU^4A$czPJ(hk>_Le9!O|iEP5oZE+euWE&%{#hf75K)F6(5_%WV(ya)hj z>xBIL_xK>v0hy`uVMAQ7gjuPdJ+!j;etrtlt&j8GB+qxC3;WYr20Sn@sy1}MIi0>G zNAI8*kkvb`gR#PRAi@LqWw_h&TAu>3LwgPe4jf3|yP;z*N{%oju?s&$_mUI4i>7z| z-2>R1eorhPc**)G7TrxNEWBhNKu#QxX3ul;>im^UmaLmFKK~@WQ6E6>D8Sc3%su@| z^R!4q^qG*oWI$H$x*o#qMkdA(*;ERO4jfPO`sIzu)dNC*#jY4HloB((rkwkLGV^SIan z9xLf;Y;r%`!Wp)G20szqf)0WPSh^0gULk%@q2=*h1zYMh-JN1)>a6Oz#+QqV9vSPV zr=?*DlB`{Bnv=Kf?Ak^v;*eFRu9svC$m%5*!kA$^5X%G03K##XwW;=4GH%+m>C@@! z-!!XYZl6TuFOMplqVYgNc|aWgX?cFsqEogtzXdG@f!}}VCC|mEYH^Tnb1-lpp9&vC zEGB8$(#6ZkYn{;wXqpnFz^ltwt(vP9@Jpew1j$bX#|+p3_Ko_ocOrmUOvWA}8j#gP zjyKI3546Pt_yuSuCwV`MUy?i8!XJz0^wZBI-VAS9=Ib8AKQm*{49R#Pp*#R^vl_tW zVRxrOHnsUNWE_6_y4UhvfSL7(_ZVXK>Ew#4n&M}leYVZh<;w~RHl?Qc=HiC*irXG1 zZ`*_Mrs)mKSFXkqqYP=|PFaba?$7A5JB1DYV6gF*Zdz~3o4amrx*En7%m+By7Qt`Z7w$h;ad!mm7=?uk7w*r>NIe&aL;NciE1o<|!(pVW@#NvR zbbXS~`X3zZ8S~Io(66j5d2Np7?M|wH z*Rkq)ng#%9g7JXyKwt3yZa9C(^!S&K$KE}=chl{^y^Z$m+jl%_5=EA8%s)t!&X|4} z4;;$_ScY4SS)%2bkt)GKq9+`SKUO`1nxf*>KL!Z88F{f(2;kM^^CZp155jk2$+Si* zDq3|((6%20Pt2KREOq`HPb_|6_Sme{+9MTP@p?plXr*QF(uEmgPg&QpqUR`PCadRs zZ(24UXtxK349@vBZaACTE$9*$F>)l``it9W_Uv9cyY8~I%d@p;;g4P3ifO=jpci?7 z_h)%6&)FU?eWRmh@m^6J%kXpZ(rEx|TUZYOmp>2+{2v3b_UbJxEcg!&EA&>PY+TwYNR|?RG|clp9{-0po$r_W(cQ9JB*(;>HxB6~w8j19&EJ?(+VAAdZ#s=EIB zz+K-5(E9)Fooj3q*A>Ul9k18fhgpMd6ZX^{G%%7;kF$D|}pW%gME4Wd-UX%aP|rN9@0Q37!&4)vmd;s=y{%=F9( zBYSsEvfg>_{zn>iXXeg1=l-r)ojG&vxn9X3Hgfryf!Wv{?h33^qhglW;AB~17+?el6h}Y}uX7T^AuXQT5{VJo_sYw3 z>QqSqvf?^mZfn)SdWv^MvCJI@XfGoIY9Y~yiPWP)I0k~j&f(d1?Zk-_)gOL5^b%)j z?JgvPBl$xQENgtG*>_i@(C;5#zu*o?D}-n(xSUWSnR=dslcgtJ$OZu&34|iC-^#Kw z>tdz3rk!Rd>$O++(dEmZ=3XkN-{bvYIduws0fBNO5DLYAB9p9Ed(TU2;)vFj zZ-1rvJ*Pw3TBzTWL{D+qhk{A+h1%lzz(lMQvz73J8FJo&-YCXosAkKGrcOSC7l-nAo(&&s8ZXzZj~Z zgs0XU5c5}$=iX=ZF`O);i9;UFCkF(%BEr zXertZS?X+~-|_x;Gn7Lo2!O!s38+i=q~#>1M$qD1DNCDb(<__n3Kl};XxFozz`8xT z_ZWQ)C(CH!&<6q<6NpCp*QMq3+v@BdjYVh0jc}=lfq^&ay??w*i9|vV6`ISetE;20 zuYZUhY;TY5eRj_d%@sf(2!McL1j6CNjTyQT=cxU|Q4nrP3wcS;D1D|DlurK9Nn5KmXB3luB75 zw=`>XOUq(X*J^BS{VJ_owTdK8%uqK@gnfS3&hBTj2lxjBK){v+!r?PcSs7I81gbfe zx3|VQchT3DHQqj+_C<&nyn(e(PdjJu0h}y@fkPb#=tY1FmAs^VO`i9t^VZPNK=&+KL`#+|r9dD+fu+l+wzl>TTE%aAcXmH_ zhX%L=0w7=m0)3&_uQ-CWck0Z-_CNzQ)w%Ml=Lf8tl929o1=>u@!-SKSH!L^-0`?@p z`B`qx&pOVI)*7>VkWAjB;o)IQj3oHg;z)MCcI_ICj*imU=;t&xHco1rnId3Lr;|J$ z54qhQ@_0QockVozH&5N`Np-CJci&BZzn`4WnxebNKg>_>3iiBE^o_Uy1VF%21h|Av zk4(yIoFb9+tlTvY+OVpLqu2Q^4Hr(`=LxL+zF8f}x5z9@pce!{pkxC39Fi zls6lvWZ@bL;HS{6qb2jyA&E;*s|#1_y?C`+rJ5^BKgkuZ|Da)4APNLPzyt!(=z+GB zoIc2*bDm;Iw4mNi%l$r59j>VxKU`%TRgU{=S{^(*^}JEv;ba*_8hSv$AOg46CR585 zV9Oz({@_X^cJVSE(VR;-1OX7R8G%?VwjhZF$4w% z1{@=atH<~W?;FNww4IjG>MG~@o}R6?F9K#e@-P?#bRy8#-@j0#;;Sl0T3uipm)c2-r+ojpBKD{lZ-}r zbbmPdL!bZw5C8!^2q2QB2UW-f0gDmn3&$etQ1n>LEK8Wf39)0Y8pn#RuC7llVGxV} z0T3t~0*GXl%}}BvATTolo@w(0=OCS%xkcq(;a91(*OMt!gRk5_0rVUMbR&RBmTqVv z7zC_EKwZr~l}R3Ft31}S%zSG&nw6Hw83+bDOmUFEIuO9?LLHz&BnViUfLc2y6plx@P^l@)Ojwl-=dAIPJ1gD6;OeARZG>GQ z00Qu=2U)jr0K7T3|jHc7zYaEwoW$0rDY00@*y08Un^1aUtI*p$HL&6_7A zs_#HV>(&4Y@gdjB8nfwe!fX%#0Sgj6oFuH!#L+5ZRS^qe>6&`RrLr` zUbU)iunPo0z@7w<_-@Z}gVi9QIf1@V?AN@`%pMk8aSvaD#QC{(-c~ibr#V;%1OX7J zSOOLIV_LBzjgEuB7ZB*(+5H?zRo~**hr?gcw5%?3f3Hxr=!An+R>O+UfPf_kz{#@2 zFu({9um*wN9oyb=I;2*9eHgNaMLJs{#3hGRy=m9>?Pqn?1o0pM0u@LAuL~=1(9vZO z_%Z}SvDhXhqrAwa4VHhI_O@#y#NVpO^>{GYam9AF!(b2q0b3A&lVyuxf{`F#4uNp^ zj8jq0?Use|3sRH?=2)SzRv|8Pr0!>2{r*oHONKZQ009t~MgUIMG#oep0T3ue;Fh!X z1Up+lW@l?fp*B`O$I0zOB$Vepu9{amJ3Fsi-Co!S0w7?20&ueIKZfW52pCGBFVg>r zP%^uDCgL`JmFP88d6CLEsr?2i;(#RRb)xM@fx$nl(Dy2m+N& z;PBzYHKP+ZH!76bDrB*R*FtQu0ui+w0|`3HvwDvBeBNW2HE4xVfej!~nFQcuRpz*( z*C3!dfmkfIASH{3`E}w$3eo*s-19z;bon$_bW5PR1|#da$a+5DHLp(#NgQ`MtKR3T z?IU{Wgj^5+0T3vL0GzC1XrV0#fIv|M*zsDFlBMOO$Sp)-F~4HGhaICvp-{adNL`HC z%T7`ae>(YkHD6Eh_hfcggjyK%s=9OA&p1NG^}FPYLZVNETs7qLxd#!YVmjDB00b;c Z;Qu7SIg1gvy~+Rp002ovPDHLkV1jjl3{U_7 diff --git a/site/img/logos/bk-logo.svg b/site/img/logos/bk-logo.svg deleted file mode 100644 index e42c2d46593..00000000000 --- a/site/img/logos/bk-logo.svg +++ /dev/null @@ -1,53 +0,0 @@ - - - - Produced by OmniGraffle 7.4.2 - 2017-09-12 22:10:18 +0000 - - - - - - - - - - - - - - - - - - - - - - - Canvas 1 - - Layer 1 - - - - - - - - - - - - - - - - - - - - - - - diff --git a/site/img/logs.png b/site/img/logs.png deleted file mode 100644 index be80a879f1933bb214d51ec4f41dd49118dc1212..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 51935 zcmb@uRahQPvxW-<2oPL@y9U?b?(XgmZ*Yg;?(Pl=8r=-wxy6BgeoP;R)aV(g!Kyud zL2-p3Mo;iiT}UUQ*mkHsxsWP(-#LLr*Dfm||4hGQBms7P)YNgvng|B`9!#+6yjy?Y zTw^J3rBaBYJDlL;WXyE3M?WGe$zN39QxgwRNup^7*yM20U9nQC=VQ|IGm4DNjI726 zU_sm>H+uqhZp`6Xf}zA_iC@SL>Y*U_Loe9JnIDc_PShCz-@`Zf1hSTMk6UZ< zr*pl{w#{o{%D-A3RW<#3DZXKen5|L#TPgErl1fJV+mcpYKc==tY~SiSC&%5|N}*(W zou%hYn>?ZADeqn8oqNt+Gl%=$_g4fo4KOGY0mx7=6e0)_Ul?TYanvTM+VnW1e}4JT z84_^j#eK2=x#Inx?*MFU_snx$pN^;5ODZfkJ}+G!EQ4WKiJw z{+$0bjYBB_oiTtLI!pTR^Zz-H4RS*I-%CUUwt?Gb+&;v4{GZl|0-{d;+X`^pA`=W7 zgkwvj6!U*tH>sN91zz{RUl)Z&6d=2eAaaQEe{O|mZ~6GYmyqPzLzq-qw)si${^uQm z)?>TH?*7{faGX`n7oI(e1Gz-}|FjMmiGb*TKjZ&TBjwICc>{L*tN`IpbnWTh8;o$J zIq>@ObTRU!(tzmtyaZA=)B-|LQE_N!sOzM0onQ*D_)H9|X$rvavk4=czhe{t}YHC>h`yQ-5rxk^xXZ4aoT|Hp5AIe-s! zFA%avk@Uq?ex6g_fL@+c*Qb*s+OQOl{tO<_?W5D{@@{j@d)kDoW!&PV`m&S3BT(%Nkne^ZHSFY zHT3=Axzz8TU&<+wW=E2HTNRYBR`})LTLTHfae6Vmzlq^k`}$^tME#l*GYmzwV>Ma` zCj-%eJCt!%dArC6V4Q$>{NlG3&TsYK{G@pXhul{9-mE1wo8iO!!eEN=q(mS?1$F>q z(K-A{x56=kxQtXdL_}cHpbM+b8e}pnzp#U4T61_RYf*D)O*A;>1%suUq#@d`L2wb0Ey)4GNx!7Hxc(XAkV3vKA|LWZEJJ-if}a{D zP~xAP@{tgK{d072yN;uwDsD8t5t6&UoE=4Xob!-h|sTP_h_Y>B)6 z{`%4sp3wV)8GQ?`G?q-z8gB7~86K|#NxSefr~yf@H=aBL8x`#`0?XKZ#B5f|*GA(7jn;TDi!H<;y&7&;sqL~M z3m==suEm6Mwzn}o=5*jb2shCr`(rbPtVXB6xBI&MjW}ClXSrr`rM>21|CtBvL}rMo z=GQ3Q)H<&+b$5zdlxbx4%7s;uCAUU2ROGA6rF}E&L)$)77KI_&hCd_FiyD7zELR|> zVc*@dOBCn~)FS42nFGzCdaVtdj-)8d zpXS7_&@S0aXJ+NtW7t)HcS*jCQzd->X*8kRvW~vQv*T%DRWi@RON`d^SPlEe7npqc zIlk;?rYVHP)tKWp5BSqhQzDLqaCo!g3k4;OiFF9L=wymnNow01&w>;lrqp&{O%P9F zj%b7qojLpq<@;Q4PYz6Q`aDi?F!vNh@?4?lNEwu{!dmpD8Gj7STlvc6=QlSR(1*7N z%!OQX@6%+;*Xy95idqyXV%BNaBUcJmOpf~U=&Ui~=4+rcX%@p|t>;M{CsA0FGz+ck z7oCf>cpB!O5zB6+rh?&BTr+#J$7C1lOev!mGh$OH5RNNXql*ydzSPWpJmuoH+ZbFi z-iKHFJI-BWPB~w>9+(9s69QM}MV;%q{9x9L{8RAeT} zr2ry==uY7WW%}1PbKZC~qkemh=)aBC3C=ecv5Hz7)u14Ph{iUsGR3yYlaDm^%?~iy z$&WZuniz|9U#Hv^A9JeQ@Gobwut(msddIjwuC2Y*InDWQEw~bCuI3a=ci|jK$3CkM zaVm6Qx}3L`;$1 zJL4$0!NVOS&suYxq^k%VQKVJM;q1|K5aGs=lj4lTt)`k96;I*A<~M>?+cYzSO;*_w zUdAcrrf5Cz+H9l6TXP08t6Ty5kU)7c z4c65?gO_3p_qn>5sa(8NO-div|b{`viMFBOZ^X4E_I10xR#c@#or-M~(Tz`d z4L~)OJelsXJ<7kQKj%VSgGGrciRhw0W+)`U8yp4rc^$nXRG?FCgo~zBZO0wey!^@Q z)Ru*<*y#x$oVo|qagc*RqdDGbqOcK zB@(encoxN+2Ruw;1A)E>`C2NO7G|GTr097x@jPwc2ALAeCSC{*^AmMZ5DECAWMZ5b zyZ#Oma8pU*Iy-|=l&(($lLQe2ikL42zKBMEva_&o*NnlZsM+(R{2W$7%lJoJZ}+MU z*C^aC+=G#Wb(RUplLLVVwk;T9)GxPVDZfuOoweBz8I7Yt5^}=v^P9FaE))GhJBdy? zP=A1=o|;o>Z+Ddi<&1h}o5$=De}pXPSo|(;YMYB?!H#kH=@GOVe+lzqvO+0XJnoG- z$om)HDccQJoBU?FgLggRZM;k+oa{YNguds-xPCC#@xD9QBZF?*aW%och6Zm05_hgB#$7%TC_X&T4{aA5Viu6v-d?r9(Ap`-SA}qtc#yRn zqQY3@vXwG8s(p~1TqdQfl#(Pw%_7JDrTQ&gXZ<6c|J;73oS1ab*>Ql^?l5Hz`UBtn zMnFJ0EC%4FzLemXRZu4LN+cw|vwV*YBNBrgok<6!kiEw2X30rvTcG+en;lig$9%6> zlgQ_G|xkX+g)PDB!BTT8s_OFQquv zWYKQMOr4OW8oA7`?T&2Q@+w)L^$ncr#c1w8+NwGjr`R!nsF=yn#=M`9Q~zh4<%5CW z#>|YdMwbQN4hsS?wR`}wghUqX8!xBWel$4Z^u`y|C z2!i9@1OinI1sW&F;+v&-O4c}76pkO$g?*@^wqCXrH<#w49Qk)Iq0GCeE!79StP=}! z-F)5hP?Unjv6)3*bL`^6V#IG*7B=SG9E$~vMvw_u3*zpjT%u8*gBF_>362V<%@h!G zJPxTDCz*i9F|?icBwU0K zQHg3r9>8k}Ktjg=Cj$nvr4StwFXdGAiha`7~lQ8Es)#?1O zta59%MEOrCqb17@tct2C!ULBQRG2^$Dv+A$Xh&ZYI6+*}Dz1^^l-ejZXE+Z=;|J*R zD-k)s4&52XzL(^S5a|G=BqlwbsE1XwFHbO?${L$qH=Ff*D8smFGCIloXCnIv zk_#jtLSyh_5L7@mg~+lm-hXaT!1=a05_nDV?-j4_1Dk#cf2BmGfSBSGfz1L{H|Zte`8w(f!gJdxaW z;M3;MLQ8z}qy*Z_LqHZmr4`!)QNl_}O2vI~cqGWkev6BXzSD{%?~arWCd>8N?XSuf zGBFV@sLt*b*;1ht`rSqXy^uq_KQ{-6=jx<^hCX70{(PgHj)V!s2M<9wIXT(iuOYSS zqq3_k%I+xXt-yW~HoUC5`KTJtxdVHCVOPGz0npEe!HM1Rvx4h7}mHxn8It z_)|h`DFg(z2wz0?7>8_4nskFk*SxZbb}p{p(+W`}(&=v1JmzczZ-tQI?&ugNsGEDm zH~V}b`aRglp-L-%yF(Z_I5<0z{hkA*G8Xeqtz-Vp%S7FiNQ}3=LQv0?i)x@#vE0lH zO@AORP}lbS>X*qG!VX(s(SWYad*D zoC)7n;Zh5gJGJC0A6gl8iwdHqdtk6?>T82vK3~`FSDflEK~aK6#z8J+ZMfYQ_C{}{pcQ2}xQV7V((aw2 zIxW)N`^gpb3=0oP%nvc8{kBq>t4TJn&sh@DP4WC~!7Qug^!TTLl!BkzypA>86GG5G z&^|OXUM#kkozHHX#T+?tB?2*nJy9#+kU1C90)E+diy*THv}Z4a7YM0i@9YFs5%CMU zQzxT$fH!F)L3*uo{{xm7b-gSRqBMK>PqbFeW5E)x-Dq2`nH1Z>I+U4uZp@+Mxs0nS*bD~NqG*`cOLF>Nc zSLO!KsfwQm+G>}xc>`H~ts%V1H6*O|%~Fz`=@ECNl;8++)SZiYeH$`}xYzatyG{mk zU-XcskGyn$i@w$b*M1=`q}2)(oe43Nnf5YCd6JRBdK3WCi-=mb_3wAOZFkv~_PSkr zIX?7=Yd*J2<`<*-uhX`F_=?fu%ZIme!DHFi#txx2mUE{LLC)YlEBFDp)N1 zz`*iJWYQLofQ}mVrg&Vk+i@`Kz#}~a2en^v$QLDd_Z}La4TCii#ttb%ZqLh$q`&vC zMdjeKkF~-0SJYa}Nnx2I?6%j#9J{u^q_oeI@U$79w?Rh#-zEOC?C+(>0`cgL#bJ+R zf~MO)Q}h|gmB(sE8euu{v zwtdSX>q4q0Pxt7(_QKC1D-E-O?qQ_75x1VFO$ssbmkynlJ)2t(S9^+@nqgsRlq7t7 zYh@!C7#R1<>#xCI)GG`wcZQqJI-c5omX((FD9iJbPn7QPJGh*#P*<4#O}p<2`cyuS zm6w+%nCL#qd$$2#xLX;D>??8xM0~D7oiFBNXDvtcyE=ULy@anD4Ph@CwjE*9(@GBx zL+dB%^tX~f=_$Z#u`1E^wOvjH&8q)R5KJO$SSjHKYTiwG{}FXNRgVA)d{01 zgLPz2F}Qlhmq2mp8Qh+s#HYR@LN(B5M+b5w7=Pd1O=K-Ctw7f`cX5N5imphG&FvU| z%tg=?QK6fBdTJ`w$WBc?CRb^BzKyTs5JLmp7&uXuaFtqp*rAIG7Oa>m&t5@WBzDgM z;++P|~V*^Ja603=Sa12KvDSNFMNhPTwjh`1J#w7f=G(sWISov#&!P7>WEAkA`N z=H}*(owbVT9fuLgIy0&SziPI7)Z<1RRd(v#?eFl9PbYf7uq!X8-3-RHLGHNkXMk&5^C&yt3?cN8UHNF4s$^vky~3?7^B zs&NSZ{X?c1L8bJnb%(##6li&EDe{!iccgQm{hZ6g;boIah&jfI2%hFORblY0vVF0* zv%a?}0TWplKloq61oN;*`LBT$G}cl{2F~HEsQRUPPKk)o1E^ye)!|WH_Rz{6n91vI zY|(t37(qNWJh(G59J#wg6n)h*6D&_m-kZ?WYzBOO(|pm9uIWU8fb%PT(NnG%`M2-53Z$EPrJCd&sk z(T~Pj(5E@hA0`N1KGb{~RqWC0hMW@4*GulUhcc^r{77noJTsEf$dx`&zC9-^i_T*)bu*a3=V5dUQ_3c+FyUmm(Rd&ddrM~da|8SUN5=(%?eO^z zhVQUr79Z~F?9*UjjXrtEW~8OTz8+FX;&WqmK8)LXNMY_cpXp~%x|oxQCz|xUL{PSj zCeGVA#w1RPNoMZ}wYeNq|J6Oxvt7L>8#5W+HJ6@Pdf`RkubdFRjjTUb)Y#RjYxE=u zhgFA2=b)ApB*|rh$PbBnao~5qEZjo)>Uz5J`>twu4dMW+jhIfo*v1O&*yza#_Q7nA z!L#AMNMYx%)Dh|kEk%&3n&_v|%}eB}Nx5#V`HlGzS(?ejDXm@FJTCb-aN%HuUoM)9 z8G$^*8C|>1S9Wf5&9YTErder-1Z!L9S1o~F^qYQ4+pSj4OL@QjfntxAt8#UQr9)5^vS~SbWj9f!VzYN& zEFa?-bGYDH)nm`29i>`b79*~le$hm3(F)9mfADBgpyaco`A&A$phU%8Ts3s?WOr5y zHC;^sbAD4m^`6~9g4k0wxGhQ|Equ4o}pRZ zpP}p7<_ZZ*{xdEZ2wb$n!h@u4jGxfUz#0Zs{%WIbPHaK~DS`Mce~Mopf|fpmut&$T zo_YpQOMUE_%o!z9=1SA>J!lB0SWD5`o(8<#dp6LCU`05{u9xCP~g$_p3X~ z!R2;McJI?(QrUkg(}JUVB8im*{`1C1{?AG`2Qed`7dvuT&h&zx&V0arx`(w>8F6hH z6>X(eK##c}!j`X1JU&~GE;0H$q|h4UglE4(snM*Sc#z3*)I7D}9Zww+IHAR9CPid& zdJXx3FGS(3@gnpCc_~AVhoSmMrBoRBui!UapfFM%CMwA1uq@w^hD#KR^#ZuWcJ zI@N#474v5%Qyn75B_{*4{XT^HT#hPvF|0b2unv1-j0dnOMgY%bbmcSsOP@gy=g zcWBLbsk~2Gz%pY-vQ~f->=%f*V?*U-^X#bPWW#Q{g-dFPQ@^wJp@8v$Tm9MLFCt0X z`B9Z{>sR7&HAX!E=NeJgbG%c3M$~ z0Od?K1&Y`k$?1zKzUGMnglqP9DcM{w%=Jg``tAW}o@;RwLsDg|lbS@c?`MkTIe%UX z`lr?+TVlR7(c6;*FYuMk{%T13rOP%o6R^>RyR2s5?gBfO%>3*!`Yl$r?awhB>xoYA zX_h=9>ujde2;0~IB%vJ34};xYObp8OYEarvkMfq^tq-1X^I;}p7UbpS<#`eR3#^xJ zt-78&6fh%F<^k(sxjC@Ev@XoE-%_v+ndIGJfSGjUjLO@ua~D&tL!tiIRW!wej=Hg)IBJ68zE-bxBIJ zu5A|E$Jlt%DE3xQiY?ugHiCYcjsUvmP57mlQ(@yqm6NZjCt8;U%eyd<`}_M{z_b{> zvZ6DV$*HV!YdoHAzzs)=jHj9g6X*-n2E@J+BqR}Fgq;KW>ecndJIIy^h&vu%&RR;` zLJ{tP`1>vAnxo!n3@-V!d-lQ(#|@;OysRg(+<@htLU>FAkhyo{>A zx2WU~$444doTp!IGD=Qmao-h_ASa?UMQu3Y51*IAJu>D`b|vKrjYQQZRs8nGAJnMtIKg}lNhePc@zXt z_aOG@p-%RL8&Vbkt|lK^rL7@dIwkcdY1ww zrcM>&r~uI& z+hMTK^RfI{N~OBm>Tt>qHli{2#EF_&M$Fl0uiB69yTV)`ueI5i1&EnuSW!(qC-&4h z^-Kom44hcT7A$GfeR1QK9f?T8R>43|je0ALWO`kiQ~$2P3dMJlumNzbfsHNRC$5i5y@iEjtnBRmC*nO=`E#3`lMOA2G;L$5j%+e+18I$``a%^!e=R{>weTuRDj^MW ztc{J`HZeV+C^C9Fgu5joA*00Wg)hKxq5hX&`K#AIxTronJZu;*04c#X8-n&Yq4`86 zN@MwDx-SV}R+w~vZ2is+QiLW!mW1rH2dV3l)0yk}qylliGcHJB4OlYteeen7rXN<$-+nQ6d^LuxbxR?mg9ZwmmC z8xF&G{z6zg23({aL)__cnWRtgU5y<#ba;GTpI2}fW@kkv#bckTI8n~UfX3Su^S+Y& z{s8XY<9BRG8u06*ffMHz46Q~L7@^m>-@dMLrRHaQqT(JAA`yUPV3Nd=Q82>}5wIVk zMvdJTX>r50T~AAYbyxahDg|;VFB@SVz^vw{IR0- zv?bJRC1b?ra+uQKD1exg6C3oaYArN$p9e@|*8?iv4rYq_hoOuIqx<<^FOYIr!~-y8 zuz~s04ggpgxQJZD5Pm4Jyw;bV#VsPHwvg)@8A4~UcBKBmPXM(Mq5hOfBcWgK#*D#q z<#Pj62A*ST+S(Dnv>UfV_@9Y8ZtE*=QnOKk{GzLIzd!(3U9N-yNJ}0;ubP5_Z%qwz zy6Qq1rlGb{^-Z^xmDTyUtv8zMioH0)4-pr3zSb-5^#{2wtKTgb$- zUyW-$Z_kr&8}$o%y+E}=kK+MuS_r5L`~i-H*&QXXdd3aDWbkS3GkZrKyN}{NKDbT! z#X|etS_C05JGY|=HTzT8?Pa-c4aJdyyN2*x{}%r0e2DIHwuaDQwj7S}rHj}3pCm#S z5Rt5d_Tf;@B?V65k`NjcoWmYzW9cD9Hum`6IDg7!0F!$sf9`c6l@sJ?%Ta}IkUc~8 zand$7iA*{x^+sD_EQ|;Z_@8tdCGG6PRcnZqueD2{`{UY4yQ^`I?dOLpDmprh!L6ZS zoys6(F@SCXrYC~XgZW)UpqHzFj&d&Ui)(r*eOtwehv=iZj{}_ihwSkIbn}<|J(Xi2 zS}d>CEIH1BE5scCEaZKQjjpb49;WnVsKDYoECppmqZ7(N)PzLAH*|xbXtYNmqw%?e z2MIjk)hl#hwPgAHHs`f%`^zMm?^z zut%s+H;iGA4i7_vp<&9k8|7w!i99zikC-ZIAcimojZ$u-O#h|yYj;s?ElcrpB9PgL zzQ04i+7B*-&wy7Wd_MGr+bY|W&*s6dQx*|udSsg_RV}LmEivnKcr~l9rhAo@0hL@t zZ55z-S*~n}w+*fq zln5{?6AMTA@^GtoQzB)sYq<@_lW%P2{2k9ILI{$G@FwoU`o*3xJ~T1{!Rz_rpBiWt z3+5bVRi!BLjiGT!9cJ5%Ef7ce=uf$K$5JI)8sj%+UGFTzpi= zFOeWaheCw(pS)_tz}nIh#rgFr=M7J-C80)yC=-^9oP5)@^9kQAC&T73I+^#Sa1^J2S~kVVc@mv5Xsqo zfe+`3c*87ORYkub6Es+S#Mxba)j}c->X*&oTivdQp)QFrxXbIch(^vxWB0jpJo`;f zvJvG=1XHzqvSID5>+^Ece_X0-^y;0-Nb0Ib&W;!RIR@-wCuy zqggyG*E7_pp)wC_YqLT3P971X;a+wTocL0vD#)FsJcp8%(D&%%>jRXMqq z#P^Gca>qof4Xo0iGK}7(o>*85I*X!Ai(Kz&a8nYHhil7C@YJnYPIv#dk}n+x11Nu3 zd^da7MoCi@*?ACy7kI1d*#H1|E-nTM@2v5G^|253OZa?izb5mpXrt#Gs3~~bSukKd zQbfKVtMIfLQConnQiZBi;-Wgbn%HCkJ z>Nf?(6`(OXdoXVqaV>pqW*3PH7Z)lgXt*`2b6QMtOhLxQRbGJboTVb8X!5*&PGQ)? z?Y->Jz35X^;m{Rg#l=l!JisIa%5Bt9$GUBqoAs$#57#OgAO3dD)abkS6e|`qSBO{~0&EAkt3?29Rjbz>Z&ML4J zx0zSwyao48cWt+rdF534Lktzje-g{^h3u0h$9k`t_000hhbs!VKFVS=LpY+BOrIMU z#AchJZ^I``F?~;(d(lCeR;2#y?j69^bh+y#fmW{P#HKj{S^twx^13kpiJQX@lL7q>)Q47XiPcu2P+4Gu#kYzO zQa+gP28K#fWEsYH2{+mM?Wk1aEu@tuJd^&sBO!Ln4hC}+?PeOOk{Yj*^KFz zR|PsHE*C|LBAUX317C^W0BeMTiWaTLjBc=*Vt}%ur*erjrEz+!x(Y4&u4(huba>W) zhIWwAETH5O@>YzC&De}j(NifZUOf?*qWl^ls{n76pl~o+dZeatFAB`t@8Y$PSg^bD zYN)SJ?DtDIdBc2d8tO%MN52}w*BT4NV_dE}+;Kpl0UV%K0i~>kZ=G~?syw|yt!g7s z<*31&6Dj~f4@-li{JalM{#y4}0}XNSKJy=w?WoR-=~P1{d#wwC=1ANE1zKz#IVhY! zT1hbv`wtN{pMGFN@0rJNs_)_(tEc*znAC_q2;zXiQz}18lq0GYVNI44=~78K9-97g zc!VFVA^VVs9YjZyjz8x3e%Lo*nmutuuL|C8VZ~bE8g|;?Dt@x; znK7nN^5d~(E^;&EY1@??Iy!pjQ3*c3H(pv=+6J!dRkbAmd+7nZ8wT|W)7Mty``gRi zb`X{x*wKWd@38UW&Mt4_CnrWK$en)W-8(@=f4DhF@K@057FK6h&?l^}VBTxpqQ#{} z_rB<#GpbZ0wM644DkjOt;{%65GsyKe{C_k-drNmji0-{Ik>aEinVj2RY!I&W##v>w znLOWG|_ zKQ%5T$Zkv38;!}*ux z!MA;xuWQP|+h$+|m-lAY_^_=%b%en+Tfh>Q-+)SOyjyQCZ}80l?@62+b4)p-h(OYx z*Ll%ial{yj-S+bj$^hAdsHE&{#3ha2#o;Jrm>d&w**w%8CNN%U5O4wV24ksA5|WbG zUZN5KxiH6z)gOxFvU+TNp0umNAOv^uL;J=G>~UTgDi6M$lkA6`W~+9@(bPqFaXaTixq_isq;6WoS<;g7_s(rfuSt2DFQwsNY| zV?KSuz_tQqdCY0F*E*|uSiuk;QKG;T(7pLVUadvf7-&WB%|lwf_k=t^PtzmL-|aWq2da>a{VolUneCRMhjc#rRu@ z5IF|2-GiHa(&%KKHyq|?zn)9f#4M{p5XYGYuQ#(1s)I&|c@iBVN5oA@e|h4KM%q68 zRbSrA{ab#Kv6&G=d8y)QO=9is_G0qZwGj^8o$;1JgViDyITQk_0d!!OjfCIJ;cTvp zusr!Z9gT{b8XO2dqidilECKPW&jSW7?w1K7Q{zR*4l-~gFHH!7$r-P0-1S_^kHTjS zxi|>OekAM(vHard`rqr%D}7$9eWeiEJ}(e?!vtS@`O#$_VQQ5|B*H>idSMna!>l=C z$6^vM96#$Y?Qy4kj5GJHITgzjs$UToPE&n&jx!HxP?fU(%d-FpD7G9g0;WB)kaM;p#g|CU9!u4hWGPE`#D>5j zOF|ZF-4&h3rsSKd@XLpIy63sMDH4^Wzop4YOmG;7OUMqQ!N0lh0u_D}z3ZEQD&E5@ zG$7?IXy(_NjiY+s*5*9u6ht{YQbFV5a)?4Ncd=q8F^j;KYg7yAXzk^^Jr_|}yK(XB zGUYbe4>$Zoot~bCOz6JpMHGu5l858X%K0m0%lhfyz%pEMFsjySY5$0euCT-#FMWC% zsutucc>$~kBZxnUlfhFc{#<_i27)eXL(~!2ebPkioiRZBL!&wnC&*Z24r3{%zA7r5 z)R-y$eXN|%(ySoAk+brT>yrl)5k`P)zl33KA9mogRrBAD{*Wh(cSrXJsuY>t;swL) zHD-1+6$Q_!XUlE`c6KwVXCk8~()jMbNSL~$5a7rg*=Tej!6RPX<;+U6!NTc3BosJ@ z`m8eP8%iTti|maV&0#SRYU(NG8h&Y26-VMI(rX{lE*!F=ng&D_ReY(Z8i=X&3NRRE zaG>YR*9wU$I%)Q57=z*XiU2A>K)SB!P6oCajP5LD&;-FF8OJl6ThmpjC9`hs!j zSx)I=kp{#j=~KJS>c~_)E5oxlfK}{dU45&<7AaVY*QT8<;c*WyCw3Fa=&Ws(j35S- z{~!!+dm{RrYKc4_m`r{j0!~g&jSeqPptAYc;VhTy_q}=JeW}bm-5W?XK5Cn;e=`H+ zI#B8E!<=h&j#<-8pd$RF6D|Um42OhhxK*pGbM}p4oj(u<+C~H|kP&thM-z8AQ-X;Hgjm4dQqUo z*6SkL-Vj{Mnnh3zE*}El=A{bjjl_F%oup9|gFju%{(Y{sbIWpb<&{L*fi&EAk9Ik< zv@MwC>7Z!B6nV!Lp+u|{*>3}Ou`<-13nc9BT)j~(Vug5FC2BRQ+%7C6a9eDTgp2g7 z8~sTVoQ#YLg4y$dw|ESwfGr|@ZH}Ex3V#^S$A{s}M9yNv15^q=#pg~O<}Fxghf<4PO@lla{oSE=tvZE zuYcbd^Q#g0pvV%~jxqs7Yw6p{o*p(nkAcP0aWQ~fpNDKYWeow)DDcChqiVxm=<{L0 z|5t6F1LO5Ds*iakpT;bDjChj3I3fd7PC{KEcH_TeXs2 z3rlKB0ig#wmd5fC0Rcg;QjrWLlohZ-L>*?)V{oD`4{jWML#z=wAR&7fPi+VSUxq~F zBn9bD3~UxKuuufnBHn=tK*pE>xkB_M3b>d6`=SMW2D?oNAVJ!Q(0?X*1(It^Vd823 z_a=}@0PYY3LlzbWAd_wiTUm@W^g87ufK#!uv2EU8Z2wUVX9bu(pmqB%F;D4I?F|k~ za{YP_J@^id_T?d-Llhkxm~Gl`)ZXy~e+0>rh=Ak!P9L3wSfNc3gH!-W9~&0@;Dv6y zp^!->q9N);s$f^~D26@Xi#S8`yxfsIUTJ2T-I63g`OONvJnJe?VdgySkZwMpdIMr; zV?QnCr!}9q?<+vk6Ho?|w8j8=e|0;?G8tNv<8>+NPj%%2r@kp%C~t1c0V-s=l_R0YfA`4vx&Y|z=V}A-c#kij z{oJk<*-gtg%)O0Oy#TNsT8bwC2ex8=k*bby`_0d*onZm`MtGEWNE8=bXa$c5E*}up zBKENee@BVX^O))R*qpx?STx$D@`wS0@ZwClzjp%)D1HblY1K#-2v%v<|8YxTYM{gq zBx!p(ZT;$X&p32`gg>61j>EKFR@o8W(9p038+NV(?Ap4YaP0^HmQVOV67Ln+?9Cwm zD^tfsl&t}fn;8~xQ%gZYypxsOLg=rnbd4+ipRni+?$0+r=rCho!2>Yzi9_3u7hP-~ zkkct@YvV@3CC-2H1CVRX1_N9>X%`mZ>rJK4YU@})>=!yZ1kc-J@VD-(=c~yGy?H@} z06=l~A_Zvqy!)3uH*EpGI%83u04D{uE8M$=@_>YG8(-aS2<+wikj8mTTpQtb+~e zNlK(L*uNbbJD=RAO2U9fn6b%fPR?gR^<|z2vBIQ z^ol_*BU>m;(!?r{CrIpM9hq;t>f-P39|~dlEVU|FI@|b|)~JN;8S%&~-!K}5e9bEp zzkt&V)K!&#fh4_jysGlt9WPx3#?RcAsF*1CCw8r=SyR^rZ0pA8222%obYfYj#AM@> zOt(bV;Drxz1Py-eMa6Z*Vnp%DH+FRk+b5lDRhW2Tnh|@%KX2gmd~W2w>Rq8=zM4ZP z%O-O9{tQ+B_6(E4fHYXo;RWj$so$?V9H!t?3{_eSEd1FLdVf-_ zKigJ@A4Ir7vtL$1`%50}q&)W)_Tgsm%aq*#T)Y|GKwqjF-uD+(JkAHcnYg*5;9;4; zN<(`Ge;+0%RY%Bck2_p@k-8n%xfox)!BeY)pVqcEB3HX^WPOKIOJ2}tDr`hHuc%~$ zoM?-S-)~<(?)~=ga}UZFC!x22fLBJ_Ij*YbDEQGE&9se%Vr<>FNjg_0E^fTD4=Ixs z7x^{jPa}5N&@>CcH4pd3>6Ylc+M)CqN`vDvHBp4MI_Y&JpcMq z+1Z+c7751EEl%8B(=lW@*CzF!B_s>Zx>;&m6Td7RCW-1`9uCCT-DNNv-}l8;%x)=X z4jZ3SaQcIKjupbKrvB`!hXTvhJi&|81_p4}T-D#)y)1>mA;=ZmV zu}Q+FwTHSi93r4=L77sEyff2zi$C3$z|(nEuk>@sQU6 zFw!(Fh(@i^4z>+fd?v4~K-R*_0vQJ2yv6ZQmLKM@&1w;?n?`la38UCta(m6+1#ri({n;4Q zhol_6Ejc@9;e|>r$>+$?e{gn!cC0Nm)w->Q?cVa4Okiu$ioD#OemBc931vA%Sz-&> zRHzOqU|U_LSQ9~OXG{JfpZAaAv)a0HoxG+djiK5mc={qmN|+j*z3)coHJ9OmTO?(p=aOGA z5mM>rYKFQ9-|nnZr^I{kd@!(WODOXgZS@fG zhiIpo!NXQi5-qo5&E(n@k7IH1Q7(VVenyrf40?HYH04R9{3kDn06ODe$yPEh#uNE3e(ca%58($W^9ll`0QJ5sedQ9)uFh{o(R+noE!{;yZIr^o5P zz2J?&{zGSA;7L8UxMKd*(qUTgXv!Rg-yGvMxmL~$Gm?U3jH19T^0Hkb4JGDPn?U?t=H`~J<^->|#|bzonLN(Z0!tKCLF8Fb$ESG@?3({m7Z&uppC z-+Gy=8q92o?1>)u^lUjB=;mx`KImI_?X9)9zgZrXg7;UR3DmT}=t9hE=tErDg%a-x zK#gvr@HkiSM~?4Ku;wFvLS11dqg@Q7v9xPm4)T?oj%)4RX=oFUG$fw>(I&peiyMnB zh3)|nfyICsWM*QLTVKz14aFGd&T!JlGv`}Ny-wGCFoSUa|FHK~QFT1QyLKQ1La-pg zgS)$H2o~I3H|{PQhXBEXyGw9)cXxLQ!8h)~&ye3a|Lbpkx96<2Z^%q~W_o(6yWgsM z>Zxbw$e_wE(z%3N*Z;{UiU1mgK0F~pJu9WNP%7z9Ys1nY__?)7EYo^GO|fBfa&ybf z%6cG8rmwO_5>+L2_YL!mg2O!G{ zGRgS%6Ip3@LOFegm@-Q^pzM?TA!k?_NP5Dwh7Sgxnn6hT9M!2VJt!Utans;|9vKQUOzt%hf1kn8jwqwdiyaK(yir3seH zM;B@d?#AG@X5x>=oaODwTOi8t)1x=5<^((K!D-7LCL?Yi$#pVmE8H9Z#&Q)(AeWUK zo6|6QrNK8&Co`OI2v!w{F}(Ww1eRvB<5nw0mP^QXz_T*4#( z?54lbf&m-Sd4GM#Y&lO?+t9G}8t?QVjDQ|P*)oCrLsFtQKB%G6xX%eO47ZEo6PZE( zBgS@|2cvhK!0?toSe7D7PFB=B;nJj-ezd=s{4yFt5LTfFllZceS=%JwIJqyV_ptnT znU@w*I{hP`;Vd?pOp0F|68A~9HQ{eF?noBUNd!bsm`^sc=n@mubreEF8a+_M4L{y+ z^75H*tH*)B+613(2P#A_gK?MIXA5K2Ceh|X06q44LNGXe%+VRvwfad(yD$&zo%j8l z&FPL)Mj!LJb2N=d^zRXf_jO3%RS{U^O z>_o0NBx)Po&EHW;9DISyo;+@|X?nQ=t`B=uoUZRJCdEF%HJT-Y)(8!tfNkr`;oMe? zQymQk9Tvl3E?c=GQ`ygkWN=9aQ%)5V>=UTu6YC&aQ|G3!#oG$UX>~%LUJBor@x%xP zxiZi6R2x~y&AUBX`aTpoD1n-X+m080Ai1#8urG8H%SJmql}}*!=ZR)$pw>uVwX`{o zx_}R6KIO)b=Dj`Nvb*K5j{(NXqp9^(sPW=2;iTg0{atBoM*{TXH<~xa1`8~pDmHK8 zuuVVrO1cqd%VnokY^`8!goV(mlNTfOe*XEc8-uzUIBiB;zmtW>O!vfstIA3}dPpSF zOZ69zy7NKdp%FKwhUA89dg|KgTGLAEYuYYa+l(%&8D;Vy!hN(;F%1O)fzS-!)cA=- z*|;P1h`8g76Jyrl6z{M;cn5Z^DxdUdexeh?4zXCM9df$yq_Cf)hp00e5wm%BM`D~k zG4`lhHETPX7hz`y?J9!|u1ZK&h=?52Ycx>%d2z9@mh%IPN7;g@!~tscbYjN@q`Ow} z&gFf@IFhCMZ)tdCi6KHXyH+Nl*rjH?bwgu^o8>x{rcf7Z_vz5m6{)LDr-?k6N5fqelLEa(`$f5 zkPbPD4;?ivK$6sHEO#iv*;J|UDOJZ$ggG-q~{ z=n8i^5~LM6Gh$Y|8W~~}vBMs0LQ=Otj?4OPV>@E(i_0*Zw46Ik3(99P8823zJ4O>7 ze9Q`I(PDEyhqnY9#xddSZ&^z3sD>wu^D$8Z)mfe?3$c96dC&Se5~~u%r$1rs`Hdyn z<681Ht#&q34cSp)M6fG0$Nw5j#9Vz_U+#iyX`a9r=YXS_IkY>9_A8ZxC_T>MCbS=BLGT0}fAU7WJP;CmDi7wC zb;1XRty`k{w=5d|P_|gks`dO)=U621+SJym9Z^iVZWFIEAHlRbcjJFti*I~5@u+l} z!CiHm`}E7pGSyg0I>Zp}(Om}h_Dp)sS)Fat2)%-%1zHA@H(d1=iJdatG-xR)>oH3S z?JrlMXdnEkiN=HA&k#MZ0E;3U>VC5CF|HKNUOqFj49G=-d&yCL7yH#_kSVRh19QJoG6PMgIF-BZ@f6hw=%bJMO2czXv z`Idb+(k^Rv(l)88-+vQ1@-!7$`(sB`pB2H2q1}MP9_QH&xnS3(*n(Vx35S5!rfR|4 z?RH+qp(OP=|bv=^<`M+ERq~aoG&`yg0tuDK&N&(1w@KR*vGYJ7F zYZwdCGxca0`PF)Mu?x!fPmvf=FDy;NSbrM6c%v`nH8_GWSTt8NfQVyk=;WMXX&!!M zxJM^X&d+N%W(ET1@g@q&pODrEsWdF<>&%rKV=pQa6;o=Wtx#+iCmigX*W=CzCEw9Z zs4E>VTsq&*2H0c5*L)S}nsYSCp;CJ0Gi9Y;qa-3fuovBOnvLMU{0(<6R@z zu&F3uw6S1MR+ci6{;+%G`ibTJ-Q1dh^K75xg}Mg;hZuHMP1WU(&oTui@^PlZDSRSv zIy6hE5pxIy#1?Db_ML`62teLBIV*F<7$=|OZun(iKEZm^=s^*W`#s(A(zBsqfuSa9 z`yodg9n@hpeXBBK*Y}8`8lCfLH%x&fhra2rYLD#W_6!#;pZy@)Xded=aTY2tu8-0d zdyPg@IU^%_F0qxi(Fh(DK0biS^t5+vn1VOVQ3ON}PCI^-_K_sBUYQW&f}=b9bYRx!-G%;K*{xLaMCb8idQ z&bDcNYtQmFeGM$vvotJR`x1 zTRSOsOx0XJn5kP+J7P~&cAJu}x3iWsu@4Me5UjJAof&G%1Bs%&#XRS^G!UoD(~z3# zZ#EN!a~oA3-4(^n2c<<4kVYb#C#L>Mk-ec@CIo|fI#~U803!RCaw57{* zcS5qrv`5Pgeo?uFH6@gO9QVchL>@hJnCslKNvVX()A{U5wlYr<7P=gIqm*SuGNJ| zi%}tGk~yR}Sw%fOgiv{q%*ia;fibf5;{|2$*$CO3k5r5NGai<7XFP3r74y^-^A7_P zPfCi-6j~qW56#o$Qp)Sj3;nbM*<5xKG*@}CmfFHp-;g50|Lr0Q5|}Tev{L+Hc8(I) zime-4{<@qe7F=>BPiO~tcg^(l3rNB06g(>maVkE~`9$23IE4(B+H{S++~X+LpIl3T zguo|01}i>vj%Ak^tau)a6=!!#$?B;(i}YTq&$ef=v+gz-)&EEsVAzc1YJ?^7ofCIN zb1LvTmO8iAy}Z8KTOfHLD7~Yz_?UnPZ**|vnP*s0mDQGT#lzl7POg+daweh_FAIO6 z_iNK3MS9=hKKKgOZA>=WKvswl9U;gy{4m z;&PpKOGcd2T{vSu$=g079$68V+!bvytqYTN`2dUl?o==*8ON$TQnzWyNh0NUXhfz% zja#Fy4fF5Rfy!mU3Ix4AH6=UGy$p+Z(Ak8;ES$Z$?5i4b7WwAR~0h`%>D7Ia6 z;e8R++3d<;6Y_{lb={T=xNVRG6s1X2nOJd%@gJN=<~PkoBJ5tM_bP^IrU0H&^;I-9 zTT75l$cMUl{BBrtld)^~TvS%~a!0ZKb(KX`2eDsHv@Qh`13_bWQLe4<00PMQr;mZd zj4Po^0swt!$GiNcLCaUevMEMRz1Sn<{6V#xwe8_3ZFeiU(sF^}$Z*S@Pu?Btl-W@% zLzuJ!ZbWeLYCS#%w1lN8o1A!#df|NBHUhgMe^YuL%JZN?m>Bh@-SZ0xYkMoMrA60h zK)LKGh%1aIc!rfYA^P{6x}y~SAp9jXS>-!fboAub_C`Kv>agj5-avn{EbaiA0a&H> zn2FuQO8L|p+%0j$o}4&kAPyXvdM23IyPqcRGE5A?`$mT2qoN?5U}Amq?-5@_t+-He z0*zqNoEI% zo^}NTPh6cg!P7{yV^tJGRlnNBV;@%Rc#5w-)6!sgICC1Rl;okDfOPiL)HyOV^x{I( zuQ=)VU$`}0ZDAkzEliMJbWM71b0tr3_m7ikDMoZe5S@Ee;x$oESt1cN9xC8FdZmT| z4;Ep`%MQfVD)07KIYD%F6@RpO+iBsFy=r^b?Y|S zRAxI8slIa5iPMUPvnN4B*BOD8z>&ro92s!{UoV2$4GHZdS=F}vsAQkCUv$o)4rW2z zzLr7Zw@ci^!zZq~1)7!vzZC4mXZln!1NI2DJq`vaxZEH1PtqD$(3?yoFNZ)P@)f_S zU{X@>EHAJ*YzohxzEc0^nFV~)D9}BNSP}W+l9$%Wn#(+28N*}+Kb%|9g`a+AS&VsO z<-=7kOW@Q~A#4@i4C?F8f9?p`jb^R1(j{5(S*TkRJH%W0I04&IldMCvo~PMQO4j>y zfDr?IuOeahxI}X!T&LDiynfoaaO&@bO;Qau$p!$;ti7 z_JV-)I5+^4b6kf%ECB-+dG#-L%lX_Z@c~32x6ME-Fsc6qaNCn|Fl?L$oi!Oia7Mj& z!R-=VhWXtMM^*dua&eWrWawhL1+_qgjMWIO&GpdQBW@o@`1^UjyR2A>G}N!a&RUjEgm zmZz$$Wl%iEN$%rP2 z@O-JMsWnR(4lsyewjY?j6idyKK^q7Y9xAqS6nDk0ABX{NTKgIR!?v-Wa1Q1b(H=U#tRFb_07;e z3jYQ!(5C<@ZpfH__QNzY>Ld$PU9gO9n9Rt1j14fXfukQYUcIFgE~I7bUxWyH`sd(9 z6AQ(PMJ<41fDhldAAnutSHTiHcZc~F1k<-C?>b^L*pq=*zADoxM_~+cQPqeA;9+Fu|l0aQ&xGspUyWx(n%TKnQR(3 z_iMPyyxCzwkhl|U(||Fhw(vFF;>nExeWA{&MDABKe%hLFlZ<|WIgj^LOrmT8d z4C|-$nGEcL=qQZ3=M9pDwAb=Himo9S%vP_#9RIouNGN4lY*v`tv@nm9k`hHHNGC+_ z;N2kujxr>mXkNkL}w0B0_KVVnPxed=T%3jA@YFs;t(rB zm2mxcuEIMIMetD`Q}XnyzWjS2z!~77P;D2mx1su#18dP5>Z+yrJQZ3VY#$#Kp z#>8Dg2i-_fmVfAmeq1>Xk>iBs^vJ^!4)pU|W+#!;jJWM4n=%;>$1&vz(*(2nz2@z@ zySo!NCH5r0J&=W|1@MX-f@Ov+lm6o_a10(AADy{Yw$mm){|GhFWx%{)0Uow8e!V|h>##_!lC5i@Ng3nSk19RXj24!%cO?vTSR+da^d!pof!Je9dDa5M4ex4^L z*(Z>SkyE?QRp^~8b;%_RS@2{r0 z^r~ z9S!d@={cIrrFp%9J7n^UjG>jA~u-SvY(TK z_$-PXno^4?Mi~k5w~&bGk@Vn;>ysL$9oh_P_f~V^>gnSH>4Jejb*l3V%{bg0)P2Y^ z;Qhl$ecpytX~F(<_%EwikFuqUaY@`p*1gpLeMa{A2H(G;6q7Z@#N*9J_Te! zFK3mhb<&v8xcOaC=)ZL1SvDza39o$VGxX}CRY#&vq-hnC55OoDZu*hs>yORQ1@k^? z|8Byzy2it$re$gE$jj&Qj?9CB^>WnHbeDS~d8NpdaenCPyaVE>q2TYq?QnB&?e28I zslGNt@SA!kO@kVDoOmiOU=T6(4Sdua_`l?nv!`nq9q!tvl9-?0zxBvak&|x`=$Vyf z-@)W3x1^bktM03?5RC2DK~L*4j?^eqM!HAeF+Xsq{t3#(zd>ui0J`fK1DB#w?Gk)yr~yn@Ko_2#u%x;m{r1AXV*){9|;i~hsK`+6apW3`6K zE^+B+M&4kvClAM#RQr7GBt2K-vHLZf2D|F#jlS#hU(@ec$-V{9$A3$ACx=2K^4mwt z>e`KT9aC^tvyLfI4FNw~V$I&RedAW&`IR~>?z?&@CY905>3SrQte7*Ojk$Jm6aRg! z1Zy}Pht;URrlgl$YFyl?QQKYHi$H2;SUjV(fKKTi$#0X2#dxy9c+MuV;xlbV95L$} z=lvvs(l#M@K^g@w_^VRhWU>qz5fs`FC@5Sj@2tqv6@{`Jsl+QMMjthPQj0i6FUH?d zKZQ?2EF-0s*qmLpu_Ddo$XWJhoj#pT7z1afE04TiyvsX;3eHq;1P!t^w{g=SuB>5Qw-Q`C+j3 z>oV)ynLzPx2lL`$59UU^{%t4@_}V5Gb)uNQcx$E1<7RKV?SaYNz_GP}t(}|2nJh{g z#;h_>Gv#M?Uo6krD8MtAi`!l}Jb2&|{JAIr!-Pony;S2BoUM9^lWV)qWU@0lZMx4~;C!pqMu{7bR5sj# zA(LteUlL(i=VdbFAME)JT4J5=u-#X1 zK`lAR1Fb{}_m_JvHuqv13{=eu_$#g4ZF{gE2{Kq62MV3QgD(AwFZ#0xp?;>-%R9s0 zvDOpp9ci{F73KoXR?J*SxjbYK2NFlsjvN*0f=6zpoS`tR5^fJCSR`jqOpRWWz2f|{MZ#Vs^W7_+ zYPWJqKg`;lEawVhjj%J{@S*pEkukA~C z5;-kn8oktse-!SsSHqy$OVrjIrnX*gy1dJ;)8?v-G&)(MlX||+FV*c3zuyp)gmo=i zv8^y%;n6ss_o%ktBVZS={*~q+E7x@2T#}~+cbn}_xmmnc4LNSud5WwbYOrvYTB)ZT zG~d%9dJC(~pmE)BHq&)OgL7J{%+jvR^0Pj!z95aE?Xo!dDmHpN*i2s>J9{q}sl`@kdnsVFUWMytHZ zCClipQy*qlRYgM7I^{CfcDMPd;pR7ohO(kzM=$90ODYuih1KFGJ9o&gyBeN|9UkVG zz749nIJEcrLvA1)3P+A~>?Pd*&u?|sH$^uj1 z3e;q-86s|TC`(DdNTy_awCiGfTg+X4I@Y{a#GBmakQm;y+D<;E>%O_iIWm|`uIpJM zbvI)?^9VmZX0XqmvFthRVPa}3dEn77tKXr#Z%xIg*6_nSsjBd9il1L32Gw7xU z73^maPa96qBy$~$>Wt3V@W{zh(i7YHi0s(w)z#L1he|#B|?w$ZA+feGaZl&{HQ98Rr)k@U&AT=)N+>v9*F#}{m(A_FIr7&cZ63`)B zWRT>YH2)>V`a|E@!ax>RHa!v%{C>^#Jj0?@dAitvh-i2vy(rR^#FC{rXPZZzOQQ;b z(wIPWG2Yy?I?om>H-qAEJa#b(!KeBoO@TFg+uTug_58?^g$_DS91y)}Oq_~MQ~_mV ztUY@E@Ctj2k**(zHZ?+NyDD%c&3*;t$h>`LP|&EfiuclVq&%g;xcFO5i0zG?$Dn@X zWuCg1VB_DAAr_NSE_ZMb)YO9pX$t4a)r~+`|NerHOvJ`Yt0Xj8R0?1|f0;C~E*OyL zpBhNz5i2wt(Z1bVD;SWlQRSUeQl>q9Je_S`8IA%QShTO0PO?SgM`$%hpzowx z&ndP%Si7$p(CvIGi?e@Qb9lo0Y>M=lJjysaV@gg~s^*bJjWZ!|(wIuxCq;nTFx6d4 zhr@nIJ~yJgXCly1{oHPDV_u%{>2lQD*S-4j6x4gH)I!tF?D#D`D4%rM*XP5oAnDyt zZZBuQ<)L`aS!-ibyp2R_D69DSRZ~Lk&Y>)EVDiME>?EGTsCYB#y#=4dZ3WLxW-)Se zrT6r8nR}wd_yKl6e!t)RVl<6)`}@{Fy6ZK52|sPxUJD9AJS3nc{c^*m@y2tHe?gcR z=e&0XGND}*bxg=@w9QZSwz{n_J6!v&uhi4szO@<#D}ESjE_YiRUHhrIph@+{YpXHY zHB9?xIGpyYi7gH`#Yn$jB;^Jk0(g`vQ(^1d*khmJYef;z|GJv}XR$ zNjl*{%|+XeVj0J<0HigOBKerbydOvX5m*bDMfqaMEa_u|0aIf4(YI>8RD;zkA&KAW z&Oub}cJ1jXjJhEF^I=1&p3kWxPP}RBwj?koCHUhq-mBrz{-oC8e%st~{d=jc=|<*3 z>}TT<(N3RSiiZp4DFWwAUXA8v=;;OQ9(XwU^I^%5=3ax4c2)e?*a8BgwwkuMQcxPb zl6Jwp+5GZ1oCVsCx&%<}-+*xv0|T+liRq6gGRW z$F}-<4r-3^u}44BJ?r~n#({_n!#8uoC)T`qJY^A1LnlNkhnTzy)~jWRX$%=&+I1Fl z2XeDtf<#$7?;YH2y8FhTiy~%Jhs%osem!S_%B=MCIMb-O+qT1Dd2+&|o9vfAEb*%x zNA5~Ta~DRjkBk<>IMTOvthe)2dyl_AV7?JeuDkJt$Xs%$;;pkWaR8%lo4ivLZb++s zR3q*^Sr3f@RWh z&2xO^K5OMVH(hM2mN~1;av;s7>SY%q+}+oeA+XEUI4wi%iO%aY|<*2!p+D><83G3J(A~sgO?I_00do$^ z-!R;j=0(!O6Ez;?yyMhS?<`%1W^%#Yr|)8Rbu6(4?ZizpS+jVsb=l;0COD(PyR*RLw4i2q+erQ~_LorU+W{2}-u>hw>3J4WOcfXnv-)Vn1 zQ6l8CJ;bL#;;2aHbxrup>oSQk#_)l|?dzkO8W0*XOzNMJuWG&g@D$Y~>x&z- zs-;UVUJ3F6(;g(vbq%QEm2X2L^Gb-PNNFTWIFgY@?A~32m47Xd`;m&z-g0XdJ&p57 zn7m1M(k1;?6&A^Vlk}>*It+Yv9o%?%zQq{fuaSa2m+`HzJ=YPC)=5J*D{fvys|u#a zF2x!CJ@nqh50t{vSdFVbe-Q0kJL)nm{1iB1>>@9w*+>H7114{>k8m(tWVmpFon&Mj ze)%$o9OK5O&W4GkN^;P^;d5zV-Nu)QJcIcHD+`${Gh zK#wXFVg3^V8v8?E0)fojZj&9;;^>)~Hd#~%V6TCn1TJ)1PXe3hQ26&Sp}PBRG}S!< zfa-#0w9kHd#Lc?!%1mXibH)>V#V0Tdn5{L}&2+$oWoP55t*htV8jl`=Eb9=C@ATnE!e=z#Jr!TJ-4`m*Da2OlxKMTT26 z42i^>eoX@d0~m593!qQr-`j1i$`^_fBXW*MKpPQ~0MYdn*qHp_t3sCYxUPIedfRbN zky-BvZPtz(4=#={G)=0Wa%Q|EaQ}$o>pKE$Hbo~v{WgQ%E&~iSQ@%cv`1BpiJ_&WK z{eL0?8N;i2(sGe@hk>8W3MJAOlX)E>WFOg}PI@reSYZEq*Z-P&-MM4xtjCG{I|DdL z2wM2nT>QVY{%`IHiLmms5ae3Z5L+w`W906U(HqogeHi zR)H?oTsQ7oH)_q5%$KTvO*}OEd8#M;dQw(XJ9PB${W=`Xzm_pD8XP833yGI9e9G;Q z6E9IMgNP5(LZO zII+c5RTHj(nsVTXttHdUx;Wu^ef(Uphznc-tmH@@HaZlwPmnT^pUH*7^P|_Qmk<>VJpSfM$l-U4y&;Ln=Nxey6Inn!1!VOu z0i@PfL=>PX^Krh9z-eYdWuN_oU!JW)Iwqb z3LGY4SNgm5US5YAc!s5V$odCMzZViP=n+2rc+7s{6Kf?tr9!c5k07L0{_#^$JngRuqKW4FHd>`4|Vk z@Q3!F8PGv&N{89Zn$Bx_B(@Y;yzZS2vx5mNw8>~{i?ikd09_dX2VdejuAPA#%j8k0 zYmthDS7I_~n-zu5nYsUX^U!RNNq*$$1VGytR?rXPlw_;QJD$OZ$)*|SO^J$B@EUa% z@ziIl5Q%w!XBms*Ub%5Vd15i@w*tyr~ zGZ#}&{Sg={Up-)&F92w(JHe!`d*L2nnpx=psXGapz_fw>U;$L-7;6TSEK30%_lRP@ zgZg%V&PfpYCN=)h(NawrkiMy55eUwu62cOu6y_8bFM{)1A#}oV!ga!fNOIvr=pl>{ zW(X^!PKN=)4-s63S{7-)rkFNLKPC^PChCBz;5ePiN zWOcUcIPs1QJ@oMT;nWp3>^%ZG&X0fhpO~^yFJ(vupWG8i@?ah45g7f)vy>ilF zNzji8jyJvs>b|ezq>8HLm`AxPmZZI+zp^2^`%e7M%Lc5Jy_bFZ*51uK>-Y;p2CZ9r z5|*n^^?7Tz;|O+`TgGM+=D!eY?T<{QN$ z3!**W-I^%kFDV78a%pfzu}<(Q;kn=Zh2~zbw@zg@nnSHMHYO9x z`L6BycmAzw*PrX}XDq|OQ$Re7`mB!a$0*8JSKC|nF+o3hM2>yzeO$ZX6Zl!*ss+J) z9?)p#-!EPld-8+|jKt{I7yxWVk7&RQ%ZxC;J!|c+qnT@d0pc<9WFnD==H#=_d6*uT z@fX3~+yDV~?JU!5+vB_U+t}bW?DM9kCZu+&KTQTfJm-8}j^rP?T?CohrQ13kk?j9E zRu#6iPlMb@1w;PC>`K8w%4E*56wcn>kA7DanyxXGJAMJyL73Os%Ta142IK|cOLR{; z1r=R|53`3fQl2N0)RGE4AMYKNfDI0Dg@vMRD2w-_n}W_P*m*XX(3`55R*UuQu^=w5 z&;U(gZSevCD@l#>9MZ`#LS;^*uuCapwfWux7KFMu8xEginn2I{&R9VNMl&1&!DQ$A zSS@i~PX$nm{Z$k#-U|WvlPz`zPaf&964cUgJXVwN!Um&?XXvIv0tObReR&0b@|+-? zuSfxII$gM7&4;v4e{q8$Wm66D)0aCD3tk`JwCJ~+`N#WPw8s<@AeTu^K8viup*X7| znL=E`oeG1C3JvDK)PdzqfGHHkan1o7e7)8V0dp-}V(kYG0fi24@^(Rz=H|e8Iu*iG z8ME^FR3MqzsPe)@*?;#DWVHi;YnyJ0!SUMw_7#atUHOq}ORc3&$Sd} z20+NObSr@w2OLtV^M2l>@?FG|ff+eQ&32GnC4^w1KP6>gQ<)&UA2|>{K(4HJI#fC0 z#UtWdnf5er=!*8$oDTNml8A5J1650i0E=2><0Sa7Gyx<%SL*w5%xB@x;aTT~v%mL^ z{sD`;lS9$}t+xL2c?$Q6C{-cA?~v0|xYWTlE+t0e!`IKtR@lluzge+e$(#H9XQc*W zt&V4DL7rH6$8-G%;|PT!Jd41Wb!MmBgRUDekaGf-dCw@8{>BL*oP8g6KTSS@oeZA` z${C;MBV%)++B!aj7(xmmhnRwZ?R}(+{|$720@4ora&&=+he(Uag9t7jsFc{IwN;q0 zQ&>(&uQwpzSs9_ZRv=I}(Azq^XQc7Kvr~%as)I9l?_E7d_vPD(Xy6*X5y$27jFofm zc}UOFE_Z>cqU{%Hbj7c{uKa|+uoS|qkz*hvF7_nb!iqTO_QNMaSR2-PyW9 zTP^)e$dkH6c*k&UdKu%joV9|rJ$79C@Al#L;rGMb>?SuRbN4q9)l#zdBJ4jrMeg~o zWd~zSAYdOwMea#}K4+Ll3fvd|gq`5#J3V5+bY5T()cC11t)I&6n9m^Sgi})WTp7RP z3-vb7=j9n(GhC1X9;ROhpx&C5(M2haB-lBKiAp8y+rVCBcFF*I@?=(*d)ZF;68G!z z0;prI5Y##+2nb$0$W*$B$-UElCGJsA9WS>p-Obr#1o#$G&`(54B}J>%<`C4yoCP** zgu>bU>XA(~GRB6lRFxHkiKGZ%&EjwTrQ|!3+JMnB9g~d%Fi6MB`DAymaQLkCF zk&pZK$>iDsU0{1qQ=f2q=#eFNHk*MIyH(fH!iO`I8VKz}_7xA^6>N50#! z^90eBkMj)RN%S}(O>o-vU5=js1LOE2c8ALxkz?jpnApKY5&0kRkB{E{KR+K%SW4VV zrzOc5XU+t;quV8+8268hp#3KQxC`zq0}yH`*4T_pQ%>FqLcL;pWTEt5hMwfs~R5c z+m|NjT8dbo{qEYtEZXosHwg$*$Z2kl=*e?#Bo+xuk}`M*%wb1B4u^iPwJj`Wbb$k3 zFA=(FI#Iyg4lGF87?|K$5UyMClI0=0P=I;XVl-&|s0yu=FFoZrz@)r4AzW74YX5&& z06s#ILJdL#-&cgOzEI|rXJ>S6F1Cgz-($-*K}ul!Z{C_NS3Yr}v7mNLq30uQY0{-d zqwwR}fUxhF?`muME}7aX|vCIikcsK(%Azd(8PSd;_aJo(581WRrwCed$fBX%FHe*j-!DLgk#_%DeI&D*pOLKR*oHcEi!)o(jsvtj;TzFt@Q0%s z7gZ(KANOu(oD3O|of<5N zK1g@_qbI{1Kz|+<&iy|ctOPr>q;VyAi*vd~eC!>$pE8j?m`IVhVw4m<3UF7(^y~L< z@|=f!vd!d%|BNE?8q6iM{r+)ZTyQ!}iiVb!l=3?>p!<@&^UGsi1l+B;4T6P*g%Y9u z1{8p>6O-8qs9;3uO~FXY2d_d;C*kXPkH#^#z+WUHZ+hIH?yqBkwJie+G&b$4IP=x_ zjT0t^jcgt@Jat2vILvUmLX6Nb+G2-Khok9ZTwrJT>*esFJg0#L)JMQs|NrYGB4BU>YSlL|82nWbk}>=#(47(r?Y$t> zKD{TXN+y+KGct`C=oYe1etAptAOP3waJ8>szSiul_~|372;l#1g{iFT%;5LfuAPyU zT9;-q1#Z%SIEoEc%f`k=-Ioas(9}EOYps?PQ#hiTELNdi z8;X@^rUhx6RAbk6uhW{#( z32noZMRe5zYX^-Q6PfK01?bmSzK3)IduABg)qk%uix^sxHMk6zvmLevQHu%-f6o&j z#q*p3t@OV010ejWK-W?5x}I#$6WA+|NW;MQGWz8m7(4)LxnKGDVtCf)VX4%le%qvH zfO0Sd_#$E$K?;MYR|&=%iW(X?@60iTp~>{$3jv=jd5lyFLH$?%@!A-n>W`h(e}OKA z=pF+Kl+h68C2(5QHVi}aJ_)zy-%5a&FUP{df=1H7wuqLKlq7-nLk{{?vTIvT`Y9y- zIx0fG=%22)@f1xOgcw$m10LTTWi+7E{-;a)$pPn)y5ZG7_tt;IN`V6Sr{PZOf1Uo{ ziT_{E0^asYAC?}SPK!%CAQ8SB@qLyAdY*^^hM{R2ZD@G-Pks|~wp=z@lsZg|iB=?v zz+WL-p`|>@*j^f_{Djv?Gx$e9OpALJCsR0NQe5}q=%q~3U`S+u0m|WHaI( znJ%z0={m%-OH#uhTlf=gyvghW*;68}wKzXChGE z;~T(k^+y@~`1+h5N&hLUo|V` zIe`Gk$}=9F@=v2(65BuT&_6%k8^8lp3in^9{cCa$cqj=cVDk#RRO*h0wbMWN?ib{? z8|_{ff43q>KB;LFW9c|WFCcmYoc?N{5JX$dEkON_<2`BYIczNe3YrYnObF1bsHlK< zR)HDkr>&%x^QBXSgtrhsEJ>pASGuG)hxIa<51{h2)dzFQd4Hii1(r(} zi=asOdewuCj*f*^tCf10bzqy_7Q@NPsw5VHOTHd7&4dMM+RKQ2dbmC8vkiblNw}O4 z+^|f0)iPn45x}uO0S@KsTNFXz&%>Jo%HBzt1GmqFxX<|&I@9O530Xr|m!SjT{_Ozu zKoTYd9!4##UN<~>wY9VEr@*Hl3#fR@?l#DP97_c}5PLzj#T-o;Q#zIv9I)kFmf%dh zIa#9xw9#_z9ox9~zsZYAOQoe=MU}^0&W>{$_iM-;3%~{8pW-AX6$uRu%>(4D>qi1X zZ|#MGijHT^jh35b%Q`>3iQMM1`@@Qn5yp;EJ($Up^Edqgb2HgLWmDLO?nvx__c{Uq zQ~25rE`e3{w*}s`-^NgPm-R7@JwZq;&JbmQp&EU4<>ZG90m#j5TvgU^NxwHyT<>-g zN8Y0FWkq)BPh{{r01lAq6AO~)7dU<>*S}{5?q^+a*EE2e6$Sh$d{5HezUq$sgmia^bT>$P=DzRezW>iV z&im<{FXxQ$G8hcltiAS{bImp9HLqV>Q=jmtQ#Ln@x;{U3oYr?sRJ#IBiXEL_FGn%P|h0SYi0l)|=i5|D(Q=u@4em0O$e zh~PQ+IQWj*@gveR75Cm6+u6#rHUKnOt@bT^135l3P>5_};wpqJ4|pw(!LUtmg)5g9 z$nTF(VW6QYb@oliQe}ZNC+q#lCt>1V$7bb}A{VIqp-i^+yW1*<@G9b)>+pmQydu$N zy@2nE07%X|y#Vfx@#98Z*L}Bw!p9VEkwVi%m1jXdlP*EuHBnh7zkfuFf&7`xi1HH2 z*({s)F~(vi%IbY?vi-@1qG2aS+gOgg3#7gFu|>+eKzK_85Pfdum(QGX%gfmv#X3lw z#-7<4Z**)SQW(`b@}Ext$GzNE3igd#B4;}0SZWV`#CPslUCY0^LXe?@3qfqS)+^@^ z0|sqxKRb%!#PFG5sTvQQ?M(L1cDe#%eh)=VkL?k0V~?3zVHFNfY?FaX=EbU44nIZ9 z-u-&;IjEG#nq1I1U(4s)Z%{ZBGF7TybeQfkrDG7rvmjB%X`risGY_~O#hvc=;x}y% z=gwp<4;JUgF^02b+O~qhbIF}wH$wcmHBb-GB#_x~{P-vEZ;b^}F?qA3AEL{PtWdL? z4?HUh0`c6Y}Y~g-nWMnMZ_@&qR0Me5$X<**!P(tW_o-%4| zOo;1;Rxihms6s^hA5aLsyu|<@-(k#D@4qIm2Z-Pj5aQnaO=SOTY2-&AF~Xo%?3Vl| zz7Pe4%fRfAJWC(UWKX3c{|uZ{R08dpmzg&;)rTl}d;lUO9oo-W%*Qzo?URz9oJS{8 z_{CBHxyqBfoQJl8ihr$2_h?|vqA?aN09; zQ48ZgQdyl})BS(a(uK7Xcuo=d1H7vOseF{<)oCmPu`(}jD<)XEB1w>ro?>{8A_+3T zA^ESk00>C_D=zRo#HLS@Pm)9pMoHU-GwT2T_7jD-GWH3ax(aVKJSGT5qls$|ME?Mf zO=u!zkd8<-1>whAei8*EF+tBU7{1U9pz0I%@x!BlNYP?+c(52ft}LPtm3^{Xt?~-I z=!q}$Jiu-+iKo)6p8$jnpP;`KK+_P=`O!SVJW~1o(*6o00((EeSDIVE3kev6vckaX zOhJXjo-$o#C{hpVGf_Pjq5ew%AcN~W)BrYpv{AV<4v+|$38ZdWL`E^AKlgX4y1GeJ z^=7X;lee0yF8GW&Dai>J4v z;C4wQAV{r_{O_a2PilDDqg&K5@CQ%W!AsoQ@B2<{ zp(_gFG^^NjscA55I@@w@VKd2SKZ}L5$^gap?DtuWOkbQo_Cx!KY;$+ZAzuZa;-$9;;+K874Yk<`lEWaDBej?m*uuR)7hA2 zN25t?zvke?GNc~Y!{Kcay-mJ-rsMZPRzif)qC6GAq75ma^crc!2`AXYZ(PEwRDK%( zk-asM)CTbIq+SL_*Ai@PCPncsNhtXY z=`r0ik&&B`KMwFF#47Z1npyvr63_h+5%tlTCQBadfa&R_6HIh;bR_!l7u(5wZY#p) zdqJ?%_^O94G0LOAt-IYpYHB+#omJ#_JI)#~3a^BnntQP={J|8D8FuI)xAdeNrW5uH z#F$7At^-lnknp`i8rG%g>Z1wrCoK0PT|U4dW5-A;S5`UW9rDgFWD$#-`A#z4m+}1; z!7yc>gDmfPU$0oR_#KxEho0l!v3s8_V3A~`wkN*7rV0I{&?E)@z_F9@l`}=r!~YS9 z?7NVgOAP%r0IsQKa~x=e?UtVy}P=W*9qjg6MobZP4H zTlkEa1NLqpoeI+N3uVhH_N9RC0bPO<59)vVV>myVot^a;*~7mCvD1ggCCltLmxl=e zqPG&+2&e@g`P7yU^Kv5@y0h$X-zjbal*)o~)LbdYsQ3v{e*oZsjL&S;8kimx;0t7y zf5?WV=L*!a1f{1O2YIF~?*U?t z@m}C|AL=vyZpon&>#cIXBHvdBouypViP zcwJ6Gzqww2)>w3*^F=kLKN|MAS8KUFk+%!I3L$~_0PADH?-P9R#;+>eb>2A;Kegnu z6xn(s{+@HFx(d_<(HOb!*gm=vgJ;-{I|Dy*?PRR_FMYWG?m%{C#ZM}wnQ^&-sZ2sV zazNG`ME{9S1l%)&+WgjqK$2S4Qd)O&B zaR)XeyFriWM#7)-^fBV2<{C&=9W>Ckts>!EDy0g_*4Zx#9UO71AdX=8@vALom<&{h zh<5O-p7X6g5*kS!Np~2@8!1|GgqOywSqi%6ld21zkL8&8b%n=jk!F`a3^rjdA-n*KV0mDm zlu?z@xW~p|(9|`V)5a~SM_Yd!OvGvWNzZlGtYP;eq4m~G`}w1WOE~hm^&MW|&K?^2 zZ6-&aupXq|&+pLj=|1eMaeu;qW`3bJ3MF8Xxa)k8x(v2%*YJJs<5J3>^l9#5K4TA` z-f6X7$Kerc*S>SN`jpiZF(0jG_v{ERIQJySOoxx7@@q_kBNGNg-}OcIlp zkyu7GXO)ROz@7NPk_6D)IwbcLZs?L;t^MK)I1#FcG%YY0!PPv--cNiHtw%*D6lqHo z)8sY%SMOQUo%B(g{Dl1U6$eYUPZ;|>rz_#E=5zRShTk?I?F^DQ#F(^yfjH?SS-&M7 zpIP&hoHcO-X8swpR>bE^!-tPAzfEa=l2;|>zdD=_imQek|Cf%-=zQD-V6MQ9D?q+@ z0a#}f&H7<>P0wC1Ff&iYm@4zHAi==KzB&F+wbl6AgqreXApTPE2+QKLOE47Qd=3ez zq2zEo(5s`=#%Ei~WmiVQ+5s6N=vf~V=E^fO1Y{FU#pke!>yQ5Ia`%An6z#U`Rrf}= zy%VYFNaefu9;o?PL`Y621NNm5!ya}GllL5>yAFW-yoE}%bRSx=&&UsMGZ)r{lMwFP z#eQpLWlnN&knT77D(I|a62zE!Gs1DHjK14-|XujluCVne3COLFR4GCUiW z=KBN3u?@Y><1k71%MIA%#I4%udckr+rUUpj*o_V=9Ujd#bl8|IXZ!P72JS1!cC)K? zhagyXJ(xDX zb`MDsf>r9P4@BIY@8*%467$(ld`4S;zXaPvPA$bE=IvGXKEW5RLlVvcRunGm6ba9L|bsc5T#Dv$JlH z{MtD$on7tp6vX`VwwE9f(*t2p6VKHLi&VU)Geb0OR2;pt{T0%@%5RzFp6pu?{cdmK zI%9e89D6MpEdtD$l5fd9>!9T2daT;nTQ0-SmKjL|7jzGl1`H?k3JOYX)@QZ%+sZyW z+}BawTM|$>E(sBQe4XSWE^n3BeSCcN;BGog8b_YUApZK3i>OZ@TBZ8>gz+RKBnPv? zSDO{^SX80af+@Q8b_Kyn4}OR@XclW5GiD7Znwxz?rcuLU(k{f?b=bjLOnT+jkVDsq zLZek8R~7r!sZSl+8t+l`NXEWexS-BQe$ek-u12$+u62vD=#@GSioCD_^Z8UxXHlJl zd@~P9EHaODC!?AoUF{O!E%}Blc55~?7ib>xXIL3jJJ{^KTL z$0{svQ9Czr$EHp&~7uXkFi&6Npy~a>!HLhUZf{g*{Bq zqT)RsR!!(3T98w51Le{;~1(qX6@@V z%R*LrBHT9cKO3B*S)1rJ|7cj(z^c)~1@Mi`}7jQ`%3c;iCe^Vg=tS;x)=S`98w_17jo8uc=6@BiQ@f zQZGN2N;gX0)NDT8Cq`kMYlI#9>uo*U`snTopHxPwWVp^)F_pHAp;9l-c+dFGzA@r# zxjEo;uL$^W1_wWcGrf)$Dyq6T(v;nm3dltBe1v+AgfP0n9D@?m;(neCG7@jltf7gK z+F6-d@4j@ueZltR>l}g#6C=-)_2TSrIO|z~;w4@;7awWV2AQfRJ{Zb}1zu2r9`z|)kuJVA*MRSe9FDhMPFu#e5mW=k<% z^1iX->w4RV`3U<;XfwrWAy)8UbzgmOy<)<2FEaY8=D3dax85+8o!BJ360^d3QF}Jj zNc0Q$ajf%w5io#g_%YQZe7DavlOfZMguTdHWLp&D^X79*&QCF!wpbhA1l{CEjJV|U zW%5s`O=wL(sBv1^5#F?p)~YNEyD-9OjttoQc>7~VoQ}q^nWW-$b2ap+&j>~y2Co_2 ziY_M!{UZE}Jtq-SNdJDzY0G8nQiUXq{^HFOYT{w?-Ves%o6+^*hC+2mw;fK*PvAFP zPH>e`0VFqasa9ArIFw4JDkc^lm24@bTo^J?o&~N9qCkaL9NYuci}+b;uAsu=KYFF8 zHJp^rjtuxs3!jPs>TAxj-r!L&z&0n5Q~}m?Fb1t$i6koGpAf>j^X2Rer}Np+QuLVW z%y%EhoU}X{m*^j731y#x^`GX6Fx{}wse+7^ z(z)mcE&B|xYOo)X9FZMSObd)IaARM}m{(EmSW$t#@p%&=j73Y0-^^x-YKa44fON~e zXG;=|JdS&;)K~8mL|RRZtK*6ACr1jz$Et??D_};sngWl z4Uk_Vt}uCl$wmglS+l&kx#a}6WLJ?w=BZT2%baXyveb18#WsUTl;#vWp(A(f);&UGN-2HG(Rph zKo)-YJjOPzV;uh`O6$BWSUU-1#*4I6GG7@Z)_G)XdF4{oS90pQT!KV}#{-PtnPj8L z(g-dO&c%srJ&Hd56nlRe)(~b#Ro+7|@scjh#E@(9;fAp2vr@8`t_r(yTZR)4dpqP5 zm?MK@J9i{_Zu1CHjd==oTppWI?8JSJiY6*@&rFWevhDJ-W5`H8*7JQ894mg;?)1Qp z1X5zeft2Q>EkwvjIM{&UnlJ9&vMP{_22c{J*vI2C03|`_MhV6KMgzMHd9f|?%=XP7 z!pA64|Hq#s=#{hIWbeJQjY8@F4F`Yz{-)3aS*MR7G=E8!{*!n^!vh)^%JNWv(fo5< zha|{EeUsrN`zJdGGDo;@EOo-sB&_%J;UeCzZcF%CGe(g@TR2<8!XxoO4vB`vuXo=J z*US_kLX+f8=L`3UA1T`h@+p$9r7E%Xg#@;h)3FCa?6=waHisj#bL^3S={= z_d4STl)O~tcXI;@KP{Ak48ffNhf$W)4ZV_3=j?FBXeM9YE3`wpezI=wP|_b>0vlQ3 znEjm^;=eKtjO!dHjBTgn2wx$NNV_in+U%f= zA;_cttZ>Pu38pRPYIHbTQ+B*Fx9z!Mz5*k2fh3> zUYaz-y+CWHO4mcp=&!4*dT0L?!9wuAMvFG0Y}FRUOz61CKRH=&lm8Kf(*N@s0L(Mm z<0~bH;b1vDar^(HgJk4qQ=|n}HRd(@TeYD00u{-6FTW+2O6Yk1{AF!6U&yxx9Ts8B zIkbDaJ@W?Hj#~HzJ9gK;`3=g3IWpN(uLk`5Il2&nlK52?iKKMb#I?&GuC0gsMRT=s z1SR)Uy1g7$OJ=?&(i;Y^kKcYrrwLzrTu?GY2Nn9m2fII4eUl_#z0tSB_%O(Tp6}j0 z!?dexEX-XXkU7J(tk`2!aTcaPQ}saK^empCdD5&cA_vDWD0I<6G=%j9($r92l{%$% zG*`bM?$cmOw1Qdt!RMu8pPa9+iC^{48Q_a8$P%x8(x`2kNfPn#t8tyi`%}+sXur;1 zzo9Mf2791WHTlB%8qelFSMJZd6IOJINKvsZmbgFq{T5eKI^h@GHdhyD(Q?)K4<{hf zeZ?zEUa=M>XIDpa_KJ?z_(e0x_VU~k@4gsd2VZlWb-t%=uFp!yHgO6bwjb2h;c&?h z^%;#iycJ7qby^Z5Of5nUoe^mdnJva0rdq8^WB6TaL^ z@l|DIwYnxNTfELk*w8LGWt>@KK|Ob*=((m;>1}~?e1LKAnU86d)?(|`Y>;VE=kyWG zEIR#Zy0Ut33&~)>IjltSqmBVYjx=hQ*fw3YsW_O(Ym6J(n79L9{ zC1c+PB<@V5Z{-H*Hi@_PDp5RUoZp6PV?4r?u84JXqN}L1tu0ER@NvDp_%5h65MWA= zd+8znRCIRFQ@Zg-oo4*>Va-yDXG4|#`k-#|o+N?xz5f`5NAsO=c+AQkC(gQ$QBItN z$F$?;Vtv0H-xe_nQ7-;)7CSEb)j?s4-lutcv9yNOu@HMAO7R=#<2iXPNQF*({uK(j z7feOBdW%6A1uLq(xzg=~{g6uS=2WfF50$9#&yfnR}xjO7kp5 z{pxbV`z*8j>XXHDluS#kF-fkAW-7}Us@Hu?mCl_?v=uoi<9~GPHHMLkRUDU997SS2 z?|4PO>zi~O{zgnY-|rb$NLpmJ0@BIX!Y@7&v8;z;Ox~8;uqH6Sr;pd)Sl>4*p|(Fu ztr5;@l-rD;ogNs>Ut(c0xG(ym^aDqppos5xOwM#he@?(~d>kd;y>SU<`aC4){kt zzw*4OPJ$$<=%?%UZqAAsPLr1TxMLTqm-h))=*8gz3!@6G>JXZqW=-|4d`tf0iLyJ>&M zhdAk_*7lAu)0fRvmg?Mv7y2y=x-Cr;m94!~u?f1!t~wX1{xHpY;xhfIJ(P_fifkpx z3~%bEc-Uy^djkBTqqsbyH6peCH1yKg#6T>x!ZZZSMy>+$iRQ-m8z8D0i=u8C_=q#B% z-Gq$WdG3jmLkMR=YnsGt?#I@r#*oMDLiI5ac76<`n+~!WSPwEeVp@qW9bM@j@4sNF zHJ>fTN$2iQ5`OQuUcMzFFP&PZsm-))O-3*-E;7B|$7$4AqP z+RlB{Hm4|W{`VThhV%!v6ciSNuHRJcp61kt;=C?&PDk9ZBxp3~x?%H;GrSHmp4#xO zZIxDlpoZtl49Fo>iBPkC#}lp#V6qSVD5o#));-t@es_Mx?Dpz+5F~u4ph5H^tqcLP zOTV_BEDhQXyF%{6)ia8N6@OUZUS`N7u}Pa>K*?!Jy;2DymNKmG#~n=9upa)RAvphi zbgKWin7l zU(61Nx@+t_jckvE7g9SZ-q~!SdXD=W);%M35{dLxjN)0d*{tU4O=Y)3yMi}tndafl z9v_-)!o3`=gS0_Y%Wk zg-vhuSkk^qY^eeF_g@aua;e7yn(@6CIl1y)(KVA!_T?Uupriv&NTa>ci2-(ypA43S=?mGv)+uQQ1?my#BOS#-*wzJ~c=C!i=?bbL?Kt}a( zicXmoOInPa2A2cz!7fXj|9)34{rs{2M%0VVUkHod9YuAq8>;iW19gvrk`wfkM|0oT zI~Z&|;|v22Ik7Tqe9=%rUr4yttuoBPNY$s@qo0+lb*bY~oT2pa7d2@fF{r?)(KiBS z2d+TjIFG%7wQ`5rIEyTD5Q`f7pFTc(w3m!4*oW;@3qfuI1r8ue1MDER8eG@x_3 zKD9z4yRm;KY1cc`uPU@ykBDOh{Z31x0@Cu^ImWYTLl|~mrdF+hl>V}z?ra~znAK1~ zdt$dkwZLx{>1>!lPrAR0CnuK@^CqIG>*r1FOVuQwKEzU(H#gi;j!gzBl--Me z`)1vpI>aKh<3AjmuFxXYmy3PT$WMdx22%_XkG>pQC+@Uv$g1DWd@WGm#1@CbW7piI zSJ+WpC!v@(Ov*2o*0$`VQTe`yu+plt=VYnXAdbb;Z+t|jzJ`;(-+2`@J|7(Lb{6IJ~gg5+RhAOa?m1^RG`xEg z%! z!+06}NHmL+sqg6HEz1OuB~R2~l5|6h48(a@Q6iX(z=p*Qjpq3^rQbKoNPBKAGkr*5 z0(-(vbP_wzJis36)Lq~UsE%t69VRn*;92YDn!5a$^m%wH-7C&D)lYt~M~RPozBYl{V;e0?@d`r$RLcC%T;JgQe?f)ts>k3kXAQp{RH$&<+t!GtQlXp5ii_3!L| z7Jpq;)4>V~r5RqL)+@W$of5DOR}-w(YLsfYecw{GZD~D_oD@89bGRAOBM*2VIhTu2 zw9uk!R>g+(Ke`j9y8F&Oy5?FngOZCdgHgwVVgV{3*tYjU#U{bMU7XKK^=vjyWpP-{ z+->D#@nYmKoYwdAB1?sh;`RvFXmpp#3H*v`>FaAaBXxIEKdT+hH((T;?`=^Bf6ZQRWjc$$;z%vc^n|}Nxs>vzx&-dPKFfc)>pV2-kJ8atSZo0czZSs z#AG+FZtRAL6mFJWSN-9a#^Qn$eTntKjOu8vT)+&UN)1u(#fbRkiomDU{Kpz!A$tu= zXSe>Yg-f*Fvwjk$>h_a8SFWWnGPD{Kz_ZSGr25n%_R9uyMXi#MqHHy-vvS5d%L32r z3!2KtLYK1Mna4MA4&({FX*=9M%JY;-%V58H1ikhT4c9tVu;@s=<)`r8)G&K&@h0x_ zCs&`z){{W<6TQ}c{E`hXVv{>GWW^NsAQ?RlLV7NrLD`)1OQfJm+=<0Qa%y6nQloOn zxu+-$C;6iyG$&bysIuCe>~yH}3p^iUcNX{3Z2x)? zsF|)keBeGiq#Aj0_r(n@@zIQTX5F7jhn`Fw=+n9-4{Ds6|2SV{e?tR7x#d>AVN9IS zGH0GI4!W3FfG$_45+H{yzL)Kr#YRlezgB&c$w8Pu6gx^j|>Gb((?&OO|&q@ z@@UJX=V)VkEa?z-nXZnoMQEt&)>j=IObt60ArED{at* z&?YAcGiqVI^WXdQ5JFa^Ie z=WZWAF@Tzd=@_pb7ikF>D%wsu%~#Z*@_u5eri0m5Z`FS6_EY8iQ>ZD>DHuUK7o|%6Z`TfTPA4fl!{oD8 zlXQNpxoFU}&GBCGcw<57E#~Ev>J+WxLRXNHHCu$01xia^d-i?MKHX-3EX*)W)X38P zb}zQ%LGLV^yPz0Ogsu{8mB|n$yc!l4hMSpbj@aQWzKd)zbp-}I)flrS1@}IG(jtaL@Vd0 zafo(>4pbvI%bNd+f%L8FS13%UhA4FDgdYELpv8gJi|E7BiE4@0MrX*fdxJUzglqzi z5_NoAtXma_EjO%b(OY~hvK1xLz#yas6m2Ln>yK~nV+Fov*8sV@H5Z!WOPyEbC!fJy z?O9!T_LiqnQ}-im>2?i*ks{XqUyN z%`#_3J?a+o1fJnqtCD{0{wi~pWqVHnpE}m`O?&q3Fxp9!1~U0!rv8md#sv?nW&5XA z0Zcu*<&$1ohIv%SXjSrF0v%?|^(S#^?DLh4DlPRR#*RrzDr!$xcE4krF!^Jl`S&Nx>nWSULN^M&u0M54x2M~&C$1aQfe)-03 z5!IDs*MZa{m5;Je9aP?eu&%}Pba|ZMI4u>^l7$`ar@N0B#XRJRWx~tq2QGe}`Hw0K zq(O{lFnQm95j~F=NpuYuBy-E~=!b;yUf~XVx}TCrPfvJS+w9NPxC}qdX@u*o?Ap5C zJiepLK|d(`oI0Y^oYNPc`f@#JGo7vPEkVxjwGIrn*i!vnHmFi@=v3blDa35EB|z0l zVz9|FnRd~sG2c(dO{?O4UD9DJYa;2({P&Zp8YSJ^V`DU2;$CI;UDRXVBTu55W*vX- zi?%B3u)O4j#-=u67j~@=0>cvLZ3ijBA43N}-Db+I4|!)R?aC&y^ZL>0ouk#Ns%XYv z34d)6b4@6Ut?i4&CRLX!D%{aDFQoj8Im;B`e3&1kHwnoI-F@npBW>+&I^WGwsa2r> zb@6xJaV|71aQ@Iasi}b`1%cG6JN@R|OT?DmJ0|P_GK{rO9Ff?0`$k2ty4sJPw?F8k zy}F4COSvOjzjs@G4qa;48;;3Y+J?(f0mayA9z=qvyvj(r_~ zvOVT=c&BQ{=RAE{x~i`Djh4wOjakulw&UG+?hkfhHbp^GH#MKyfytM}@w6t&aZz8$ zqeYZogh&_kZ62vNDv}ZBh4QSQ*l!Vie<8!unGIn(-~2@1mOg;3pPg`hx?bqkM46Yy zr@O3~EZ+A;FGl6OVog$+f1_u;ze}ImsQ7iE?!hu`UBYjr%(7TMPhpm-)a)@S5=+B0 zyHaDhvlZF4A>7MoacE|NOQe1My{-h!S^QjBiGJU-9IWbj;QOc%!4`SatoGtdp` zRrSc!v>!+%@_J`~%i;EEu=NRTyydHX#V&+SgZN;lM1p}fmm>}Bd#9REnmdi9?%a(#Kq?pLqPuuQ66>i=#RpY%3>)O8>}E<@J4dLv z&b(dA%_+&PPO*Tp##%N*F}*KE=}dUYy@uF26iIoV`!s z`*lXah{?Nm2Pty3;C_#w<$Y7)c{<1!J;lseNtmY*^h3@J5xxL8r0uiC)g#5YDcgg! zZY0a_F{e|1t@b>o(d{42&F9P+8eYao4zVRIV5L;ue4 zVEhMukF2x2t}2o3_6~9UPCy^rVnE>dJAMvO-B>quv$P5UQ#(uM2a}zFbrV3Q(hoFW zNc-We&WMRS5JyxTvRJ_2lGD^AIM>`_Ac_XGX@1w+FX{zQ?N*=*nKS(A^@tU8n^F*FN=C62mH5beU8cP?Jk^xGdUhEN4}Rjdw@d zmwSRzmtb&F5se>o(*I#K*XqcBb>CFk9%UaD2ht%fcegp&!s zAO*f4&=lgNXg}4HNEg4F1vxYNI8nE`2(GjBy~h|EA} z>_468i=_HV6sQ$VfxyfWxFYrDzsWLP>y2^*PLe^OjzVp@ktxl;x28a;4a_p)F?5P+ zNj9ZXKr$x|_w<4r;?+MrW5cS=h~oGM1O)g08=5D8e}NAsEzrD6wkW_(#FQ_0N0VX&r#ZbG8Xqr3YJ&JApgYotw0}18Q1ds@oD{ z-LiLMipNgD1>{vk{Zm=tcqv9E71`3Q63u!_Y<$`K|*=IDL--1ke@XWMGS z@&PEs%-yZG>ue{$wrm1p0OQ||0l%!!jsQBe-M725JP{Y{L|%TH&p(`i!_kT&x(V<| z{`U%xdhYO?5fISjWF=m!%OQvt*VNS5Uq0V6o(CSUWLj|cYR<2=2UcU*uZf2h5tRRh zDN>tzCMXWfk;cG=XI?&{|GEuqsEE z3%&pM{i=SVowVtO^_+V~0b#)G zM(K{nt^)^zJ}jx{7Z&0(z+P{sE44*4nb)$s z223z)8eIp?EmY_<@6LcK`JZ?HkV5Dta2;!VWy{{+DR9uMsa@cCv#+0-lcjtR-uglKJy*yJ;fjkjgEdw@`Yh51QR<0rp~%r16(_qkh0z zM|n;__$5>A7G>A699cf}FQx9^(M%i017I81AI~x;+PP2@7%0#zv2yx2NW$yu3fBLO zCbSfUK*`Y<6mhI>V%KR=s4l}z6->?y=l`~>7~w!to}$J4b31~SGXAdbaFJ1m^wL{M ze`t4Fx^j%k^!z1xpRpCIg`sD?sRf;2RCivwux*${@3Vi$ku6f;)Lf?Z1raIO zS`k^KrBz4RbFOUXrFz5K*htH<|DA9of_+Om71zFrWKLGv&%T{)GR*Ebo0cK(DGX@d z)#^H#lh*=cxjovXhAp~!Dtk5@ug?XEMhfc;oFloK)p-@RvO0_3JNllH z8Ga4NFx4F8;wzY+=@hB{a?Yd>L?QUE%MK`n7LED>6S-qg7)-_T!LJO|A(A z_eVPIydg3D?9!eP`Rfj^#1QNb#n|J2Kb<>CG@d%Jifb(OT@D|g)}~-s&8@#IXLG)a zV@eCb4@F*+5~DHah#O+FnQsX>bHB?L>2PW)h;3l1+Uhhhk50pIq9j*yPMg_e;&$aG zA}q@#Nd=JqRW)*_iKab)7RLU{%j^5_L%BClzrV};ZNZ)B=j=UST4m)_l|T1WY&jiK zzAnSJVz~n0PLBv>>d$M1IAE0uUSGAX`{Z**wEdl=`wTUcRl=0L$~l01F#g%yzme4U z6w%QWE!x&(JL$z@Z^!1rx5 zWYw@eG*qOv`-Aq^86O}LVHB(I*QB}LlzkDtaXye|yJsO5{&LMxs~ne1j!xDic6>B1 z%2{z46==6?7g(~NEl2~dQ(y9&|8CF&L_55#8;`?AK6#rmQoY7Y?ou+= zsrT~n7Def68n--7toPp6f>T(b-+vrmCpp;3{^SyTCx6CO&Q1IGMYK@(wUBZwcvd-E zbDJNOmfK3Z@M0?{bg!)?^1Bq>BY&_*W0+~kJV^QL!hA(h5buu1x3@VXf`yZRP51M? zwfCk>6ES3X6~Tj_H80)2+DbaVT%)WG3oIc)c>r1o+H;hH)^zp#BX`6l4r$o@xZAIJ z{Bz`2Afv%bIHo_<#isSvQ!9bYi0Ls$XRR%_?SJ|;^>O}L^`o>f;UZ7Ts=PQw<>kQc z?(V4}niml`2jMIwYH=)@uMC1SSG870(-YnnUuguK5Y`ycN|E0NA)z!sdfEFfqm^)Kj(`r-qT8$qWQb_rqgV0PFe#Tn{22j!d zeK&kXUt}1`^L+oi2N3))k)PNSVSkSOX8;uV`m;>p;={)mXfNjzm2xwF234xR|3u41 zd{nD0uJG6Y&~STIMBC$6GGv0`hk_GCK4O?r#p3w;4^ay72gs2BZ+!3(!B78{H;T9Y R6A0i-R#Hi#RLsc#{{rIourUAt diff --git a/site/index.html b/site/index.html deleted file mode 100644 index ab55e2b2daf..00000000000 --- a/site/index.html +++ /dev/null @@ -1,57 +0,0 @@ ---- -title: Home ---- - - - - - {% include head.html %} - - -

    -
    -
    - {% include navbar.html %} -
    - -
    -
    -
    - - {% include footer.html %} - {% include javascript.html %} - - diff --git a/site/js/app.js b/site/js/app.js deleted file mode 100644 index 1d28a40d6db..00000000000 --- a/site/js/app.js +++ /dev/null @@ -1,32 +0,0 @@ -document.addEventListener('DOMContentLoaded', function () { - - // Get all "navbar-burger" elements - var $navbarBurgers = Array.prototype.slice.call(document.querySelectorAll('.navbar-burger'), 0); - - // Check if there are any nav burgers - if ($navbarBurgers.length > 0) { - - // Add a click event on each of them - $navbarBurgers.forEach(function ($el) { - $el.addEventListener('click', () => { - - // Get the target from the "data-target" attribute - var target = $el.dataset.target; - var $target = document.getElementById(target); - - // Toggle the class on both the "navbar-burger" and the "navbar-menu" - $el.classList.toggle('is-active'); - $target.classList.toggle('is-active'); - - }); - }); - } - - // Enable popovers for terms like "bookie" and "ledger" - tippy('.popover', { - position: 'top', - duration: 100, - arrow: true, - html: true - }); -}); \ No newline at end of file diff --git a/site/js/jquery-3.2.1.min.js b/site/js/jquery-3.2.1.min.js deleted file mode 100644 index 644d35e274f..00000000000 --- a/site/js/jquery-3.2.1.min.js +++ /dev/null @@ -1,4 +0,0 @@ -/*! jQuery v3.2.1 | (c) JS Foundation and other contributors | jquery.org/license */ -!function(a,b){"use strict";"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){"use strict";var c=[],d=a.document,e=Object.getPrototypeOf,f=c.slice,g=c.concat,h=c.push,i=c.indexOf,j={},k=j.toString,l=j.hasOwnProperty,m=l.toString,n=m.call(Object),o={};function p(a,b){b=b||d;var c=b.createElement("script");c.text=a,b.head.appendChild(c).parentNode.removeChild(c)}var q="3.2.1",r=function(a,b){return new r.fn.init(a,b)},s=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,t=/^-ms-/,u=/-([a-z])/g,v=function(a,b){return b.toUpperCase()};r.fn=r.prototype={jquery:q,constructor:r,length:0,toArray:function(){return f.call(this)},get:function(a){return null==a?f.call(this):a<0?this[a+this.length]:this[a]},pushStack:function(a){var b=r.merge(this.constructor(),a);return b.prevObject=this,b},each:function(a){return r.each(this,a)},map:function(a){return this.pushStack(r.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(f.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(a<0?b:0);return this.pushStack(c>=0&&c0&&b-1 in a)}var x=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+1*new Date,v=a.document,w=0,x=0,y=ha(),z=ha(),A=ha(),B=function(a,b){return a===b&&(l=!0),0},C={}.hasOwnProperty,D=[],E=D.pop,F=D.push,G=D.push,H=D.slice,I=function(a,b){for(var c=0,d=a.length;c+~]|"+K+")"+K+"*"),S=new RegExp("="+K+"*([^\\]'\"]*?)"+K+"*\\]","g"),T=new RegExp(N),U=new RegExp("^"+L+"$"),V={ID:new RegExp("^#("+L+")"),CLASS:new RegExp("^\\.("+L+")"),TAG:new RegExp("^("+L+"|[*])"),ATTR:new RegExp("^"+M),PSEUDO:new RegExp("^"+N),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+K+"*(even|odd|(([+-]|)(\\d*)n|)"+K+"*(?:([+-]|)"+K+"*(\\d+)|))"+K+"*\\)|)","i"),bool:new RegExp("^(?:"+J+")$","i"),needsContext:new RegExp("^"+K+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+K+"*((?:-\\d)?\\d*)"+K+"*\\)|)(?=[^-]|$)","i")},W=/^(?:input|select|textarea|button)$/i,X=/^h\d$/i,Y=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,$=/[+~]/,_=new RegExp("\\\\([\\da-f]{1,6}"+K+"?|("+K+")|.)","ig"),aa=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:d<0?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)},ba=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ca=function(a,b){return b?"\0"===a?"\ufffd":a.slice(0,-1)+"\\"+a.charCodeAt(a.length-1).toString(16)+" ":"\\"+a},da=function(){m()},ea=ta(function(a){return a.disabled===!0&&("form"in a||"label"in a)},{dir:"parentNode",next:"legend"});try{G.apply(D=H.call(v.childNodes),v.childNodes),D[v.childNodes.length].nodeType}catch(fa){G={apply:D.length?function(a,b){F.apply(a,H.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function ga(a,b,d,e){var f,h,j,k,l,o,r,s=b&&b.ownerDocument,w=b?b.nodeType:9;if(d=d||[],"string"!=typeof a||!a||1!==w&&9!==w&&11!==w)return d;if(!e&&((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,p)){if(11!==w&&(l=Z.exec(a)))if(f=l[1]){if(9===w){if(!(j=b.getElementById(f)))return d;if(j.id===f)return d.push(j),d}else if(s&&(j=s.getElementById(f))&&t(b,j)&&j.id===f)return d.push(j),d}else{if(l[2])return G.apply(d,b.getElementsByTagName(a)),d;if((f=l[3])&&c.getElementsByClassName&&b.getElementsByClassName)return G.apply(d,b.getElementsByClassName(f)),d}if(c.qsa&&!A[a+" "]&&(!q||!q.test(a))){if(1!==w)s=b,r=a;else if("object"!==b.nodeName.toLowerCase()){(k=b.getAttribute("id"))?k=k.replace(ba,ca):b.setAttribute("id",k=u),o=g(a),h=o.length;while(h--)o[h]="#"+k+" "+sa(o[h]);r=o.join(","),s=$.test(a)&&qa(b.parentNode)||b}if(r)try{return G.apply(d,s.querySelectorAll(r)),d}catch(x){}finally{k===u&&b.removeAttribute("id")}}}return i(a.replace(P,"$1"),b,d,e)}function ha(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function ia(a){return a[u]=!0,a}function ja(a){var b=n.createElement("fieldset");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function ka(a,b){var c=a.split("|"),e=c.length;while(e--)d.attrHandle[c[e]]=b}function la(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&a.sourceIndex-b.sourceIndex;if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function ma(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function na(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function oa(a){return function(b){return"form"in b?b.parentNode&&b.disabled===!1?"label"in b?"label"in b.parentNode?b.parentNode.disabled===a:b.disabled===a:b.isDisabled===a||b.isDisabled!==!a&&ea(b)===a:b.disabled===a:"label"in b&&b.disabled===a}}function pa(a){return ia(function(b){return b=+b,ia(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function qa(a){return a&&"undefined"!=typeof a.getElementsByTagName&&a}c=ga.support={},f=ga.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return!!b&&"HTML"!==b.nodeName},m=ga.setDocument=function(a){var b,e,g=a?a.ownerDocument||a:v;return g!==n&&9===g.nodeType&&g.documentElement?(n=g,o=n.documentElement,p=!f(n),v!==n&&(e=n.defaultView)&&e.top!==e&&(e.addEventListener?e.addEventListener("unload",da,!1):e.attachEvent&&e.attachEvent("onunload",da)),c.attributes=ja(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ja(function(a){return a.appendChild(n.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=Y.test(n.getElementsByClassName),c.getById=ja(function(a){return o.appendChild(a).id=u,!n.getElementsByName||!n.getElementsByName(u).length}),c.getById?(d.filter.ID=function(a){var b=a.replace(_,aa);return function(a){return a.getAttribute("id")===b}},d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c=b.getElementById(a);return c?[c]:[]}}):(d.filter.ID=function(a){var b=a.replace(_,aa);return function(a){var c="undefined"!=typeof a.getAttributeNode&&a.getAttributeNode("id");return c&&c.value===b}},d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c,d,e,f=b.getElementById(a);if(f){if(c=f.getAttributeNode("id"),c&&c.value===a)return[f];e=b.getElementsByName(a),d=0;while(f=e[d++])if(c=f.getAttributeNode("id"),c&&c.value===a)return[f]}return[]}}),d.find.TAG=c.getElementsByTagName?function(a,b){return"undefined"!=typeof b.getElementsByTagName?b.getElementsByTagName(a):c.qsa?b.querySelectorAll(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){if("undefined"!=typeof b.getElementsByClassName&&p)return b.getElementsByClassName(a)},r=[],q=[],(c.qsa=Y.test(n.querySelectorAll))&&(ja(function(a){o.appendChild(a).innerHTML="",a.querySelectorAll("[msallowcapture^='']").length&&q.push("[*^$]="+K+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+K+"*(?:value|"+J+")"),a.querySelectorAll("[id~="+u+"-]").length||q.push("~="),a.querySelectorAll(":checked").length||q.push(":checked"),a.querySelectorAll("a#"+u+"+*").length||q.push(".#.+[+~]")}),ja(function(a){a.innerHTML="";var b=n.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+K+"*[*^$|!~]?="),2!==a.querySelectorAll(":enabled").length&&q.push(":enabled",":disabled"),o.appendChild(a).disabled=!0,2!==a.querySelectorAll(":disabled").length&&q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=Y.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ja(function(a){c.disconnectedMatch=s.call(a,"*"),s.call(a,"[s!='']:x"),r.push("!=",N)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=Y.test(o.compareDocumentPosition),t=b||Y.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===n||a.ownerDocument===v&&t(v,a)?-1:b===n||b.ownerDocument===v&&t(v,b)?1:k?I(k,a)-I(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,e=a.parentNode,f=b.parentNode,g=[a],h=[b];if(!e||!f)return a===n?-1:b===n?1:e?-1:f?1:k?I(k,a)-I(k,b):0;if(e===f)return la(a,b);c=a;while(c=c.parentNode)g.unshift(c);c=b;while(c=c.parentNode)h.unshift(c);while(g[d]===h[d])d++;return d?la(g[d],h[d]):g[d]===v?-1:h[d]===v?1:0},n):n},ga.matches=function(a,b){return ga(a,null,null,b)},ga.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(S,"='$1']"),c.matchesSelector&&p&&!A[b+" "]&&(!r||!r.test(b))&&(!q||!q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return ga(b,n,null,[a]).length>0},ga.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},ga.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&C.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},ga.escape=function(a){return(a+"").replace(ba,ca)},ga.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},ga.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=ga.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=ga.selectors={cacheLength:50,createPseudo:ia,match:V,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(_,aa),a[3]=(a[3]||a[4]||a[5]||"").replace(_,aa),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||ga.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&ga.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return V.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&T.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(_,aa).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+K+")"+a+"("+K+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||"undefined"!=typeof a.getAttribute&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=ga.attr(d,a);return null==e?"!="===b:!b||(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e.replace(O," ")+" ").indexOf(c)>-1:"|="===b&&(e===c||e.slice(0,c.length+1)===c+"-"))}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h,t=!1;if(q){if(f){while(p){m=b;while(m=m[p])if(h?m.nodeName.toLowerCase()===r:1===m.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){m=q,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n&&j[2],m=n&&q.childNodes[n];while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if(1===m.nodeType&&++t&&m===b){k[a]=[w,n,t];break}}else if(s&&(m=b,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n),t===!1)while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if((h?m.nodeName.toLowerCase()===r:1===m.nodeType)&&++t&&(s&&(l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),k[a]=[w,t]),m===b))break;return t-=e,t===d||t%d===0&&t/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||ga.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?ia(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=I(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:ia(function(a){var b=[],c=[],d=h(a.replace(P,"$1"));return d[u]?ia(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),b[0]=null,!c.pop()}}),has:ia(function(a){return function(b){return ga(a,b).length>0}}),contains:ia(function(a){return a=a.replace(_,aa),function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:ia(function(a){return U.test(a||"")||ga.error("unsupported lang: "+a),a=a.replace(_,aa).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:oa(!1),disabled:oa(!0),checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return X.test(a.nodeName)},input:function(a){return W.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:pa(function(){return[0]}),last:pa(function(a,b){return[b-1]}),eq:pa(function(a,b,c){return[c<0?c+b:c]}),even:pa(function(a,b){for(var c=0;c=0;)a.push(d);return a}),gt:pa(function(a,b,c){for(var d=c<0?c+b:c;++d1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function va(a,b,c){for(var d=0,e=b.length;d-1&&(f[j]=!(g[j]=l))}}else r=wa(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):G.apply(g,r)})}function ya(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=ta(function(a){return a===b},h,!0),l=ta(function(a){return I(b,a)>-1},h,!0),m=[function(a,c,d){var e=!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d));return b=null,e}];i1&&ua(m),i>1&&sa(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(P,"$1"),c,i0,e=a.length>0,f=function(f,g,h,i,k){var l,o,q,r=0,s="0",t=f&&[],u=[],v=j,x=f||e&&d.find.TAG("*",k),y=w+=null==v?1:Math.random()||.1,z=x.length;for(k&&(j=g===n||g||k);s!==z&&null!=(l=x[s]);s++){if(e&&l){o=0,g||l.ownerDocument===n||(m(l),h=!p);while(q=a[o++])if(q(l,g||n,h)){i.push(l);break}k&&(w=y)}c&&((l=!q&&l)&&r--,f&&t.push(l))}if(r+=s,c&&s!==r){o=0;while(q=b[o++])q(t,u,g,h);if(f){if(r>0)while(s--)t[s]||u[s]||(u[s]=E.call(i));u=wa(u)}G.apply(i,u),k&&!f&&u.length>0&&r+b.length>1&&ga.uniqueSort(i)}return k&&(w=y,j=v),t};return c?ia(f):f}return h=ga.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=ya(b[c]),f[u]?d.push(f):e.push(f);f=A(a,za(e,d)),f.selector=a}return f},i=ga.select=function(a,b,c,e){var f,i,j,k,l,m="function"==typeof a&&a,n=!e&&g(a=m.selector||a);if(c=c||[],1===n.length){if(i=n[0]=n[0].slice(0),i.length>2&&"ID"===(j=i[0]).type&&9===b.nodeType&&p&&d.relative[i[1].type]){if(b=(d.find.ID(j.matches[0].replace(_,aa),b)||[])[0],!b)return c;m&&(b=b.parentNode),a=a.slice(i.shift().value.length)}f=V.needsContext.test(a)?0:i.length;while(f--){if(j=i[f],d.relative[k=j.type])break;if((l=d.find[k])&&(e=l(j.matches[0].replace(_,aa),$.test(i[0].type)&&qa(b.parentNode)||b))){if(i.splice(f,1),a=e.length&&sa(i),!a)return G.apply(c,e),c;break}}}return(m||h(a,n))(e,b,!p,c,!b||$.test(a)&&qa(b.parentNode)||b),c},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ja(function(a){return 1&a.compareDocumentPosition(n.createElement("fieldset"))}),ja(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||ka("type|href|height|width",function(a,b,c){if(!c)return a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ja(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||ka("value",function(a,b,c){if(!c&&"input"===a.nodeName.toLowerCase())return a.defaultValue}),ja(function(a){return null==a.getAttribute("disabled")})||ka(J,function(a,b,c){var d;if(!c)return a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),ga}(a);r.find=x,r.expr=x.selectors,r.expr[":"]=r.expr.pseudos,r.uniqueSort=r.unique=x.uniqueSort,r.text=x.getText,r.isXMLDoc=x.isXML,r.contains=x.contains,r.escapeSelector=x.escape;var y=function(a,b,c){var d=[],e=void 0!==c;while((a=a[b])&&9!==a.nodeType)if(1===a.nodeType){if(e&&r(a).is(c))break;d.push(a)}return d},z=function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c},A=r.expr.match.needsContext;function B(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()}var C=/^<([a-z][^\/\0>:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i,D=/^.[^:#\[\.,]*$/;function E(a,b,c){return r.isFunction(b)?r.grep(a,function(a,d){return!!b.call(a,d,a)!==c}):b.nodeType?r.grep(a,function(a){return a===b!==c}):"string"!=typeof b?r.grep(a,function(a){return i.call(b,a)>-1!==c}):D.test(b)?r.filter(b,a,c):(b=r.filter(b,a),r.grep(a,function(a){return i.call(b,a)>-1!==c&&1===a.nodeType}))}r.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?r.find.matchesSelector(d,a)?[d]:[]:r.find.matches(a,r.grep(b,function(a){return 1===a.nodeType}))},r.fn.extend({find:function(a){var b,c,d=this.length,e=this;if("string"!=typeof a)return this.pushStack(r(a).filter(function(){for(b=0;b1?r.uniqueSort(c):c},filter:function(a){return this.pushStack(E(this,a||[],!1))},not:function(a){return this.pushStack(E(this,a||[],!0))},is:function(a){return!!E(this,"string"==typeof a&&A.test(a)?r(a):a||[],!1).length}});var F,G=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]+))$/,H=r.fn.init=function(a,b,c){var e,f;if(!a)return this;if(c=c||F,"string"==typeof a){if(e="<"===a[0]&&">"===a[a.length-1]&&a.length>=3?[null,a,null]:G.exec(a),!e||!e[1]&&b)return!b||b.jquery?(b||c).find(a):this.constructor(b).find(a);if(e[1]){if(b=b instanceof r?b[0]:b,r.merge(this,r.parseHTML(e[1],b&&b.nodeType?b.ownerDocument||b:d,!0)),C.test(e[1])&&r.isPlainObject(b))for(e in b)r.isFunction(this[e])?this[e](b[e]):this.attr(e,b[e]);return this}return f=d.getElementById(e[2]),f&&(this[0]=f,this.length=1),this}return a.nodeType?(this[0]=a,this.length=1,this):r.isFunction(a)?void 0!==c.ready?c.ready(a):a(r):r.makeArray(a,this)};H.prototype=r.fn,F=r(d);var I=/^(?:parents|prev(?:Until|All))/,J={children:!0,contents:!0,next:!0,prev:!0};r.fn.extend({has:function(a){var b=r(a,this),c=b.length;return this.filter(function(){for(var a=0;a-1:1===c.nodeType&&r.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?r.uniqueSort(f):f)},index:function(a){return a?"string"==typeof a?i.call(r(a),this[0]):i.call(this,a.jquery?a[0]:a):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(r.uniqueSort(r.merge(this.get(),r(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function K(a,b){while((a=a[b])&&1!==a.nodeType);return a}r.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return y(a,"parentNode")},parentsUntil:function(a,b,c){return y(a,"parentNode",c)},next:function(a){return K(a,"nextSibling")},prev:function(a){return K(a,"previousSibling")},nextAll:function(a){return y(a,"nextSibling")},prevAll:function(a){return y(a,"previousSibling")},nextUntil:function(a,b,c){return y(a,"nextSibling",c)},prevUntil:function(a,b,c){return y(a,"previousSibling",c)},siblings:function(a){return z((a.parentNode||{}).firstChild,a)},children:function(a){return z(a.firstChild)},contents:function(a){return B(a,"iframe")?a.contentDocument:(B(a,"template")&&(a=a.content||a),r.merge([],a.childNodes))}},function(a,b){r.fn[a]=function(c,d){var e=r.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=r.filter(d,e)),this.length>1&&(J[a]||r.uniqueSort(e),I.test(a)&&e.reverse()),this.pushStack(e)}});var L=/[^\x20\t\r\n\f]+/g;function M(a){var b={};return r.each(a.match(L)||[],function(a,c){b[c]=!0}),b}r.Callbacks=function(a){a="string"==typeof a?M(a):r.extend({},a);var b,c,d,e,f=[],g=[],h=-1,i=function(){for(e=e||a.once,d=b=!0;g.length;h=-1){c=g.shift();while(++h-1)f.splice(c,1),c<=h&&h--}),this},has:function(a){return a?r.inArray(a,f)>-1:f.length>0},empty:function(){return f&&(f=[]),this},disable:function(){return e=g=[],f=c="",this},disabled:function(){return!f},lock:function(){return e=g=[],c||b||(f=c=""),this},locked:function(){return!!e},fireWith:function(a,c){return e||(c=c||[],c=[a,c.slice?c.slice():c],g.push(c),b||i()),this},fire:function(){return j.fireWith(this,arguments),this},fired:function(){return!!d}};return j};function N(a){return a}function O(a){throw a}function P(a,b,c,d){var e;try{a&&r.isFunction(e=a.promise)?e.call(a).done(b).fail(c):a&&r.isFunction(e=a.then)?e.call(a,b,c):b.apply(void 0,[a].slice(d))}catch(a){c.apply(void 0,[a])}}r.extend({Deferred:function(b){var c=[["notify","progress",r.Callbacks("memory"),r.Callbacks("memory"),2],["resolve","done",r.Callbacks("once memory"),r.Callbacks("once memory"),0,"resolved"],["reject","fail",r.Callbacks("once memory"),r.Callbacks("once memory"),1,"rejected"]],d="pending",e={state:function(){return d},always:function(){return f.done(arguments).fail(arguments),this},"catch":function(a){return e.then(null,a)},pipe:function(){var a=arguments;return r.Deferred(function(b){r.each(c,function(c,d){var e=r.isFunction(a[d[4]])&&a[d[4]];f[d[1]](function(){var a=e&&e.apply(this,arguments);a&&r.isFunction(a.promise)?a.promise().progress(b.notify).done(b.resolve).fail(b.reject):b[d[0]+"With"](this,e?[a]:arguments)})}),a=null}).promise()},then:function(b,d,e){var f=0;function g(b,c,d,e){return function(){var h=this,i=arguments,j=function(){var a,j;if(!(b=f&&(d!==O&&(h=void 0,i=[a]),c.rejectWith(h,i))}};b?k():(r.Deferred.getStackHook&&(k.stackTrace=r.Deferred.getStackHook()),a.setTimeout(k))}}return r.Deferred(function(a){c[0][3].add(g(0,a,r.isFunction(e)?e:N,a.notifyWith)),c[1][3].add(g(0,a,r.isFunction(b)?b:N)),c[2][3].add(g(0,a,r.isFunction(d)?d:O))}).promise()},promise:function(a){return null!=a?r.extend(a,e):e}},f={};return r.each(c,function(a,b){var g=b[2],h=b[5];e[b[1]]=g.add,h&&g.add(function(){d=h},c[3-a][2].disable,c[0][2].lock),g.add(b[3].fire),f[b[0]]=function(){return f[b[0]+"With"](this===f?void 0:this,arguments),this},f[b[0]+"With"]=g.fireWith}),e.promise(f),b&&b.call(f,f),f},when:function(a){var b=arguments.length,c=b,d=Array(c),e=f.call(arguments),g=r.Deferred(),h=function(a){return function(c){d[a]=this,e[a]=arguments.length>1?f.call(arguments):c,--b||g.resolveWith(d,e)}};if(b<=1&&(P(a,g.done(h(c)).resolve,g.reject,!b),"pending"===g.state()||r.isFunction(e[c]&&e[c].then)))return g.then();while(c--)P(e[c],h(c),g.reject);return g.promise()}});var Q=/^(Eval|Internal|Range|Reference|Syntax|Type|URI)Error$/;r.Deferred.exceptionHook=function(b,c){a.console&&a.console.warn&&b&&Q.test(b.name)&&a.console.warn("jQuery.Deferred exception: "+b.message,b.stack,c)},r.readyException=function(b){a.setTimeout(function(){throw b})};var R=r.Deferred();r.fn.ready=function(a){return R.then(a)["catch"](function(a){r.readyException(a)}),this},r.extend({isReady:!1,readyWait:1,ready:function(a){(a===!0?--r.readyWait:r.isReady)||(r.isReady=!0,a!==!0&&--r.readyWait>0||R.resolveWith(d,[r]))}}),r.ready.then=R.then;function S(){d.removeEventListener("DOMContentLoaded",S), -a.removeEventListener("load",S),r.ready()}"complete"===d.readyState||"loading"!==d.readyState&&!d.documentElement.doScroll?a.setTimeout(r.ready):(d.addEventListener("DOMContentLoaded",S),a.addEventListener("load",S));var T=function(a,b,c,d,e,f,g){var h=0,i=a.length,j=null==c;if("object"===r.type(c)){e=!0;for(h in c)T(a,b,h,c[h],!0,f,g)}else if(void 0!==d&&(e=!0,r.isFunction(d)||(g=!0),j&&(g?(b.call(a,d),b=null):(j=b,b=function(a,b,c){return j.call(r(a),c)})),b))for(;h1,null,!0)},removeData:function(a){return this.each(function(){X.remove(this,a)})}}),r.extend({queue:function(a,b,c){var d;if(a)return b=(b||"fx")+"queue",d=W.get(a,b),c&&(!d||Array.isArray(c)?d=W.access(a,b,r.makeArray(c)):d.push(c)),d||[]},dequeue:function(a,b){b=b||"fx";var c=r.queue(a,b),d=c.length,e=c.shift(),f=r._queueHooks(a,b),g=function(){r.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return W.get(a,c)||W.access(a,c,{empty:r.Callbacks("once memory").add(function(){W.remove(a,[b+"queue",c])})})}}),r.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.length\x20\t\r\n\f]+)/i,la=/^$|\/(?:java|ecma)script/i,ma={option:[1,""],thead:[1,"","
    "],col:[2,"","
    "],tr:[2,"","
    "],td:[3,"","
    "],_default:[0,"",""]};ma.optgroup=ma.option,ma.tbody=ma.tfoot=ma.colgroup=ma.caption=ma.thead,ma.th=ma.td;function na(a,b){var c;return c="undefined"!=typeof a.getElementsByTagName?a.getElementsByTagName(b||"*"):"undefined"!=typeof a.querySelectorAll?a.querySelectorAll(b||"*"):[],void 0===b||b&&B(a,b)?r.merge([a],c):c}function oa(a,b){for(var c=0,d=a.length;c-1)e&&e.push(f);else if(j=r.contains(f.ownerDocument,f),g=na(l.appendChild(f),"script"),j&&oa(g),c){k=0;while(f=g[k++])la.test(f.type||"")&&c.push(f)}return l}!function(){var a=d.createDocumentFragment(),b=a.appendChild(d.createElement("div")),c=d.createElement("input");c.setAttribute("type","radio"),c.setAttribute("checked","checked"),c.setAttribute("name","t"),b.appendChild(c),o.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,b.innerHTML="",o.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue}();var ra=d.documentElement,sa=/^key/,ta=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,ua=/^([^.]*)(?:\.(.+)|)/;function va(){return!0}function wa(){return!1}function xa(){try{return d.activeElement}catch(a){}}function ya(a,b,c,d,e,f){var g,h;if("object"==typeof b){"string"!=typeof c&&(d=d||c,c=void 0);for(h in b)ya(a,h,c,d,b[h],f);return a}if(null==d&&null==e?(e=c,d=c=void 0):null==e&&("string"==typeof c?(e=d,d=void 0):(e=d,d=c,c=void 0)),e===!1)e=wa;else if(!e)return a;return 1===f&&(g=e,e=function(a){return r().off(a),g.apply(this,arguments)},e.guid=g.guid||(g.guid=r.guid++)),a.each(function(){r.event.add(this,b,e,d,c)})}r.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,n,o,p,q=W.get(a);if(q){c.handler&&(f=c,c=f.handler,e=f.selector),e&&r.find.matchesSelector(ra,e),c.guid||(c.guid=r.guid++),(i=q.events)||(i=q.events={}),(g=q.handle)||(g=q.handle=function(b){return"undefined"!=typeof r&&r.event.triggered!==b.type?r.event.dispatch.apply(a,arguments):void 0}),b=(b||"").match(L)||[""],j=b.length;while(j--)h=ua.exec(b[j])||[],n=p=h[1],o=(h[2]||"").split(".").sort(),n&&(l=r.event.special[n]||{},n=(e?l.delegateType:l.bindType)||n,l=r.event.special[n]||{},k=r.extend({type:n,origType:p,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&r.expr.match.needsContext.test(e),namespace:o.join(".")},f),(m=i[n])||(m=i[n]=[],m.delegateCount=0,l.setup&&l.setup.call(a,d,o,g)!==!1||a.addEventListener&&a.addEventListener(n,g)),l.add&&(l.add.call(a,k),k.handler.guid||(k.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,k):m.push(k),r.event.global[n]=!0)}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,n,o,p,q=W.hasData(a)&&W.get(a);if(q&&(i=q.events)){b=(b||"").match(L)||[""],j=b.length;while(j--)if(h=ua.exec(b[j])||[],n=p=h[1],o=(h[2]||"").split(".").sort(),n){l=r.event.special[n]||{},n=(d?l.delegateType:l.bindType)||n,m=i[n]||[],h=h[2]&&new RegExp("(^|\\.)"+o.join("\\.(?:.*\\.|)")+"(\\.|$)"),g=f=m.length;while(f--)k=m[f],!e&&p!==k.origType||c&&c.guid!==k.guid||h&&!h.test(k.namespace)||d&&d!==k.selector&&("**"!==d||!k.selector)||(m.splice(f,1),k.selector&&m.delegateCount--,l.remove&&l.remove.call(a,k));g&&!m.length&&(l.teardown&&l.teardown.call(a,o,q.handle)!==!1||r.removeEvent(a,n,q.handle),delete i[n])}else for(n in i)r.event.remove(a,n+b[j],c,d,!0);r.isEmptyObject(i)&&W.remove(a,"handle events")}},dispatch:function(a){var b=r.event.fix(a),c,d,e,f,g,h,i=new Array(arguments.length),j=(W.get(this,"events")||{})[b.type]||[],k=r.event.special[b.type]||{};for(i[0]=b,c=1;c=1))for(;j!==this;j=j.parentNode||this)if(1===j.nodeType&&("click"!==a.type||j.disabled!==!0)){for(f=[],g={},c=0;c-1:r.find(e,this,null,[j]).length),g[e]&&f.push(d);f.length&&h.push({elem:j,handlers:f})}return j=this,i\x20\t\r\n\f]*)[^>]*)\/>/gi,Aa=/\s*$/g;function Ea(a,b){return B(a,"table")&&B(11!==b.nodeType?b:b.firstChild,"tr")?r(">tbody",a)[0]||a:a}function Fa(a){return a.type=(null!==a.getAttribute("type"))+"/"+a.type,a}function Ga(a){var b=Ca.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function Ha(a,b){var c,d,e,f,g,h,i,j;if(1===b.nodeType){if(W.hasData(a)&&(f=W.access(a),g=W.set(b,f),j=f.events)){delete g.handle,g.events={};for(e in j)for(c=0,d=j[e].length;c1&&"string"==typeof q&&!o.checkClone&&Ba.test(q))return a.each(function(e){var f=a.eq(e);s&&(b[0]=q.call(this,e,f.html())),Ja(f,b,c,d)});if(m&&(e=qa(b,a[0].ownerDocument,!1,a,d),f=e.firstChild,1===e.childNodes.length&&(e=f),f||d)){for(h=r.map(na(e,"script"),Fa),i=h.length;l")},clone:function(a,b,c){var d,e,f,g,h=a.cloneNode(!0),i=r.contains(a.ownerDocument,a);if(!(o.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||r.isXMLDoc(a)))for(g=na(h),f=na(a),d=0,e=f.length;d0&&oa(g,!i&&na(a,"script")),h},cleanData:function(a){for(var b,c,d,e=r.event.special,f=0;void 0!==(c=a[f]);f++)if(U(c)){if(b=c[W.expando]){if(b.events)for(d in b.events)e[d]?r.event.remove(c,d):r.removeEvent(c,d,b.handle);c[W.expando]=void 0}c[X.expando]&&(c[X.expando]=void 0)}}}),r.fn.extend({detach:function(a){return Ka(this,a,!0)},remove:function(a){return Ka(this,a)},text:function(a){return T(this,function(a){return void 0===a?r.text(this):this.empty().each(function(){1!==this.nodeType&&11!==this.nodeType&&9!==this.nodeType||(this.textContent=a)})},null,a,arguments.length)},append:function(){return Ja(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ea(this,a);b.appendChild(a)}})},prepend:function(){return Ja(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ea(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return Ja(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return Ja(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},empty:function(){for(var a,b=0;null!=(a=this[b]);b++)1===a.nodeType&&(r.cleanData(na(a,!1)),a.textContent="");return this},clone:function(a,b){return a=null!=a&&a,b=null==b?a:b,this.map(function(){return r.clone(this,a,b)})},html:function(a){return T(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a&&1===b.nodeType)return b.innerHTML;if("string"==typeof a&&!Aa.test(a)&&!ma[(ka.exec(a)||["",""])[1].toLowerCase()]){a=r.htmlPrefilter(a);try{for(;c1)}});function _a(a,b,c,d,e){return new _a.prototype.init(a,b,c,d,e)}r.Tween=_a,_a.prototype={constructor:_a,init:function(a,b,c,d,e,f){this.elem=a,this.prop=c,this.easing=e||r.easing._default,this.options=b,this.start=this.now=this.cur(),this.end=d,this.unit=f||(r.cssNumber[c]?"":"px")},cur:function(){var a=_a.propHooks[this.prop];return a&&a.get?a.get(this):_a.propHooks._default.get(this)},run:function(a){var b,c=_a.propHooks[this.prop];return this.options.duration?this.pos=b=r.easing[this.easing](a,this.options.duration*a,0,1,this.options.duration):this.pos=b=a,this.now=(this.end-this.start)*b+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),c&&c.set?c.set(this):_a.propHooks._default.set(this),this}},_a.prototype.init.prototype=_a.prototype,_a.propHooks={_default:{get:function(a){var b;return 1!==a.elem.nodeType||null!=a.elem[a.prop]&&null==a.elem.style[a.prop]?a.elem[a.prop]:(b=r.css(a.elem,a.prop,""),b&&"auto"!==b?b:0)},set:function(a){r.fx.step[a.prop]?r.fx.step[a.prop](a):1!==a.elem.nodeType||null==a.elem.style[r.cssProps[a.prop]]&&!r.cssHooks[a.prop]?a.elem[a.prop]=a.now:r.style(a.elem,a.prop,a.now+a.unit)}}},_a.propHooks.scrollTop=_a.propHooks.scrollLeft={set:function(a){a.elem.nodeType&&a.elem.parentNode&&(a.elem[a.prop]=a.now)}},r.easing={linear:function(a){return a},swing:function(a){return.5-Math.cos(a*Math.PI)/2},_default:"swing"},r.fx=_a.prototype.init,r.fx.step={};var ab,bb,cb=/^(?:toggle|show|hide)$/,db=/queueHooks$/;function eb(){bb&&(d.hidden===!1&&a.requestAnimationFrame?a.requestAnimationFrame(eb):a.setTimeout(eb,r.fx.interval),r.fx.tick())}function fb(){return a.setTimeout(function(){ab=void 0}),ab=r.now()}function gb(a,b){var c,d=0,e={height:a};for(b=b?1:0;d<4;d+=2-b)c=ca[d],e["margin"+c]=e["padding"+c]=a;return b&&(e.opacity=e.width=a),e}function hb(a,b,c){for(var d,e=(kb.tweeners[b]||[]).concat(kb.tweeners["*"]),f=0,g=e.length;f1)},removeAttr:function(a){return this.each(function(){r.removeAttr(this,a)})}}),r.extend({attr:function(a,b,c){var d,e,f=a.nodeType;if(3!==f&&8!==f&&2!==f)return"undefined"==typeof a.getAttribute?r.prop(a,b,c):(1===f&&r.isXMLDoc(a)||(e=r.attrHooks[b.toLowerCase()]||(r.expr.match.bool.test(b)?lb:void 0)),void 0!==c?null===c?void r.removeAttr(a,b):e&&"set"in e&&void 0!==(d=e.set(a,c,b))?d:(a.setAttribute(b,c+""),c):e&&"get"in e&&null!==(d=e.get(a,b))?d:(d=r.find.attr(a,b), -null==d?void 0:d))},attrHooks:{type:{set:function(a,b){if(!o.radioValue&&"radio"===b&&B(a,"input")){var c=a.value;return a.setAttribute("type",b),c&&(a.value=c),b}}}},removeAttr:function(a,b){var c,d=0,e=b&&b.match(L);if(e&&1===a.nodeType)while(c=e[d++])a.removeAttribute(c)}}),lb={set:function(a,b,c){return b===!1?r.removeAttr(a,c):a.setAttribute(c,c),c}},r.each(r.expr.match.bool.source.match(/\w+/g),function(a,b){var c=mb[b]||r.find.attr;mb[b]=function(a,b,d){var e,f,g=b.toLowerCase();return d||(f=mb[g],mb[g]=e,e=null!=c(a,b,d)?g:null,mb[g]=f),e}});var nb=/^(?:input|select|textarea|button)$/i,ob=/^(?:a|area)$/i;r.fn.extend({prop:function(a,b){return T(this,r.prop,a,b,arguments.length>1)},removeProp:function(a){return this.each(function(){delete this[r.propFix[a]||a]})}}),r.extend({prop:function(a,b,c){var d,e,f=a.nodeType;if(3!==f&&8!==f&&2!==f)return 1===f&&r.isXMLDoc(a)||(b=r.propFix[b]||b,e=r.propHooks[b]),void 0!==c?e&&"set"in e&&void 0!==(d=e.set(a,c,b))?d:a[b]=c:e&&"get"in e&&null!==(d=e.get(a,b))?d:a[b]},propHooks:{tabIndex:{get:function(a){var b=r.find.attr(a,"tabindex");return b?parseInt(b,10):nb.test(a.nodeName)||ob.test(a.nodeName)&&a.href?0:-1}}},propFix:{"for":"htmlFor","class":"className"}}),o.optSelected||(r.propHooks.selected={get:function(a){var b=a.parentNode;return b&&b.parentNode&&b.parentNode.selectedIndex,null},set:function(a){var b=a.parentNode;b&&(b.selectedIndex,b.parentNode&&b.parentNode.selectedIndex)}}),r.each(["tabIndex","readOnly","maxLength","cellSpacing","cellPadding","rowSpan","colSpan","useMap","frameBorder","contentEditable"],function(){r.propFix[this.toLowerCase()]=this});function pb(a){var b=a.match(L)||[];return b.join(" ")}function qb(a){return a.getAttribute&&a.getAttribute("class")||""}r.fn.extend({addClass:function(a){var b,c,d,e,f,g,h,i=0;if(r.isFunction(a))return this.each(function(b){r(this).addClass(a.call(this,b,qb(this)))});if("string"==typeof a&&a){b=a.match(L)||[];while(c=this[i++])if(e=qb(c),d=1===c.nodeType&&" "+pb(e)+" "){g=0;while(f=b[g++])d.indexOf(" "+f+" ")<0&&(d+=f+" ");h=pb(d),e!==h&&c.setAttribute("class",h)}}return this},removeClass:function(a){var b,c,d,e,f,g,h,i=0;if(r.isFunction(a))return this.each(function(b){r(this).removeClass(a.call(this,b,qb(this)))});if(!arguments.length)return this.attr("class","");if("string"==typeof a&&a){b=a.match(L)||[];while(c=this[i++])if(e=qb(c),d=1===c.nodeType&&" "+pb(e)+" "){g=0;while(f=b[g++])while(d.indexOf(" "+f+" ")>-1)d=d.replace(" "+f+" "," ");h=pb(d),e!==h&&c.setAttribute("class",h)}}return this},toggleClass:function(a,b){var c=typeof a;return"boolean"==typeof b&&"string"===c?b?this.addClass(a):this.removeClass(a):r.isFunction(a)?this.each(function(c){r(this).toggleClass(a.call(this,c,qb(this),b),b)}):this.each(function(){var b,d,e,f;if("string"===c){d=0,e=r(this),f=a.match(L)||[];while(b=f[d++])e.hasClass(b)?e.removeClass(b):e.addClass(b)}else void 0!==a&&"boolean"!==c||(b=qb(this),b&&W.set(this,"__className__",b),this.setAttribute&&this.setAttribute("class",b||a===!1?"":W.get(this,"__className__")||""))})},hasClass:function(a){var b,c,d=0;b=" "+a+" ";while(c=this[d++])if(1===c.nodeType&&(" "+pb(qb(c))+" ").indexOf(b)>-1)return!0;return!1}});var rb=/\r/g;r.fn.extend({val:function(a){var b,c,d,e=this[0];{if(arguments.length)return d=r.isFunction(a),this.each(function(c){var e;1===this.nodeType&&(e=d?a.call(this,c,r(this).val()):a,null==e?e="":"number"==typeof e?e+="":Array.isArray(e)&&(e=r.map(e,function(a){return null==a?"":a+""})),b=r.valHooks[this.type]||r.valHooks[this.nodeName.toLowerCase()],b&&"set"in b&&void 0!==b.set(this,e,"value")||(this.value=e))});if(e)return b=r.valHooks[e.type]||r.valHooks[e.nodeName.toLowerCase()],b&&"get"in b&&void 0!==(c=b.get(e,"value"))?c:(c=e.value,"string"==typeof c?c.replace(rb,""):null==c?"":c)}}}),r.extend({valHooks:{option:{get:function(a){var b=r.find.attr(a,"value");return null!=b?b:pb(r.text(a))}},select:{get:function(a){var b,c,d,e=a.options,f=a.selectedIndex,g="select-one"===a.type,h=g?null:[],i=g?f+1:e.length;for(d=f<0?i:g?f:0;d-1)&&(c=!0);return c||(a.selectedIndex=-1),f}}}}),r.each(["radio","checkbox"],function(){r.valHooks[this]={set:function(a,b){if(Array.isArray(b))return a.checked=r.inArray(r(a).val(),b)>-1}},o.checkOn||(r.valHooks[this].get=function(a){return null===a.getAttribute("value")?"on":a.value})});var sb=/^(?:focusinfocus|focusoutblur)$/;r.extend(r.event,{trigger:function(b,c,e,f){var g,h,i,j,k,m,n,o=[e||d],p=l.call(b,"type")?b.type:b,q=l.call(b,"namespace")?b.namespace.split("."):[];if(h=i=e=e||d,3!==e.nodeType&&8!==e.nodeType&&!sb.test(p+r.event.triggered)&&(p.indexOf(".")>-1&&(q=p.split("."),p=q.shift(),q.sort()),k=p.indexOf(":")<0&&"on"+p,b=b[r.expando]?b:new r.Event(p,"object"==typeof b&&b),b.isTrigger=f?2:3,b.namespace=q.join("."),b.rnamespace=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=e),c=null==c?[b]:r.makeArray(c,[b]),n=r.event.special[p]||{},f||!n.trigger||n.trigger.apply(e,c)!==!1)){if(!f&&!n.noBubble&&!r.isWindow(e)){for(j=n.delegateType||p,sb.test(j+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),i=h;i===(e.ownerDocument||d)&&o.push(i.defaultView||i.parentWindow||a)}g=0;while((h=o[g++])&&!b.isPropagationStopped())b.type=g>1?j:n.bindType||p,m=(W.get(h,"events")||{})[b.type]&&W.get(h,"handle"),m&&m.apply(h,c),m=k&&h[k],m&&m.apply&&U(h)&&(b.result=m.apply(h,c),b.result===!1&&b.preventDefault());return b.type=p,f||b.isDefaultPrevented()||n._default&&n._default.apply(o.pop(),c)!==!1||!U(e)||k&&r.isFunction(e[p])&&!r.isWindow(e)&&(i=e[k],i&&(e[k]=null),r.event.triggered=p,e[p](),r.event.triggered=void 0,i&&(e[k]=i)),b.result}},simulate:function(a,b,c){var d=r.extend(new r.Event,c,{type:a,isSimulated:!0});r.event.trigger(d,null,b)}}),r.fn.extend({trigger:function(a,b){return this.each(function(){r.event.trigger(a,b,this)})},triggerHandler:function(a,b){var c=this[0];if(c)return r.event.trigger(a,b,c,!0)}}),r.each("blur focus focusin focusout resize scroll click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup contextmenu".split(" "),function(a,b){r.fn[b]=function(a,c){return arguments.length>0?this.on(b,null,a,c):this.trigger(b)}}),r.fn.extend({hover:function(a,b){return this.mouseenter(a).mouseleave(b||a)}}),o.focusin="onfocusin"in a,o.focusin||r.each({focus:"focusin",blur:"focusout"},function(a,b){var c=function(a){r.event.simulate(b,a.target,r.event.fix(a))};r.event.special[b]={setup:function(){var d=this.ownerDocument||this,e=W.access(d,b);e||d.addEventListener(a,c,!0),W.access(d,b,(e||0)+1)},teardown:function(){var d=this.ownerDocument||this,e=W.access(d,b)-1;e?W.access(d,b,e):(d.removeEventListener(a,c,!0),W.remove(d,b))}}});var tb=a.location,ub=r.now(),vb=/\?/;r.parseXML=function(b){var c;if(!b||"string"!=typeof b)return null;try{c=(new a.DOMParser).parseFromString(b,"text/xml")}catch(d){c=void 0}return c&&!c.getElementsByTagName("parsererror").length||r.error("Invalid XML: "+b),c};var wb=/\[\]$/,xb=/\r?\n/g,yb=/^(?:submit|button|image|reset|file)$/i,zb=/^(?:input|select|textarea|keygen)/i;function Ab(a,b,c,d){var e;if(Array.isArray(b))r.each(b,function(b,e){c||wb.test(a)?d(a,e):Ab(a+"["+("object"==typeof e&&null!=e?b:"")+"]",e,c,d)});else if(c||"object"!==r.type(b))d(a,b);else for(e in b)Ab(a+"["+e+"]",b[e],c,d)}r.param=function(a,b){var c,d=[],e=function(a,b){var c=r.isFunction(b)?b():b;d[d.length]=encodeURIComponent(a)+"="+encodeURIComponent(null==c?"":c)};if(Array.isArray(a)||a.jquery&&!r.isPlainObject(a))r.each(a,function(){e(this.name,this.value)});else for(c in a)Ab(c,a[c],b,e);return d.join("&")},r.fn.extend({serialize:function(){return r.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var a=r.prop(this,"elements");return a?r.makeArray(a):this}).filter(function(){var a=this.type;return this.name&&!r(this).is(":disabled")&&zb.test(this.nodeName)&&!yb.test(a)&&(this.checked||!ja.test(a))}).map(function(a,b){var c=r(this).val();return null==c?null:Array.isArray(c)?r.map(c,function(a){return{name:b.name,value:a.replace(xb,"\r\n")}}):{name:b.name,value:c.replace(xb,"\r\n")}}).get()}});var Bb=/%20/g,Cb=/#.*$/,Db=/([?&])_=[^&]*/,Eb=/^(.*?):[ \t]*([^\r\n]*)$/gm,Fb=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Gb=/^(?:GET|HEAD)$/,Hb=/^\/\//,Ib={},Jb={},Kb="*/".concat("*"),Lb=d.createElement("a");Lb.href=tb.href;function Mb(a){return function(b,c){"string"!=typeof b&&(c=b,b="*");var d,e=0,f=b.toLowerCase().match(L)||[];if(r.isFunction(c))while(d=f[e++])"+"===d[0]?(d=d.slice(1)||"*",(a[d]=a[d]||[]).unshift(c)):(a[d]=a[d]||[]).push(c)}}function Nb(a,b,c,d){var e={},f=a===Jb;function g(h){var i;return e[h]=!0,r.each(a[h]||[],function(a,h){var j=h(b,c,d);return"string"!=typeof j||f||e[j]?f?!(i=j):void 0:(b.dataTypes.unshift(j),g(j),!1)}),i}return g(b.dataTypes[0])||!e["*"]&&g("*")}function Ob(a,b){var c,d,e=r.ajaxSettings.flatOptions||{};for(c in b)void 0!==b[c]&&((e[c]?a:d||(d={}))[c]=b[c]);return d&&r.extend(!0,a,d),a}function Pb(a,b,c){var d,e,f,g,h=a.contents,i=a.dataTypes;while("*"===i[0])i.shift(),void 0===d&&(d=a.mimeType||b.getResponseHeader("Content-Type"));if(d)for(e in h)if(h[e]&&h[e].test(d)){i.unshift(e);break}if(i[0]in c)f=i[0];else{for(e in c){if(!i[0]||a.converters[e+" "+i[0]]){f=e;break}g||(g=e)}f=f||g}if(f)return f!==i[0]&&i.unshift(f),c[f]}function Qb(a,b,c,d){var e,f,g,h,i,j={},k=a.dataTypes.slice();if(k[1])for(g in a.converters)j[g.toLowerCase()]=a.converters[g];f=k.shift();while(f)if(a.responseFields[f]&&(c[a.responseFields[f]]=b),!i&&d&&a.dataFilter&&(b=a.dataFilter(b,a.dataType)),i=f,f=k.shift())if("*"===f)f=i;else if("*"!==i&&i!==f){if(g=j[i+" "+f]||j["* "+f],!g)for(e in j)if(h=e.split(" "),h[1]===f&&(g=j[i+" "+h[0]]||j["* "+h[0]])){g===!0?g=j[e]:j[e]!==!0&&(f=h[0],k.unshift(h[1]));break}if(g!==!0)if(g&&a["throws"])b=g(b);else try{b=g(b)}catch(l){return{state:"parsererror",error:g?l:"No conversion from "+i+" to "+f}}}return{state:"success",data:b}}r.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:tb.href,type:"GET",isLocal:Fb.test(tb.protocol),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Kb,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents:{xml:/\bxml\b/,html:/\bhtml/,json:/\bjson\b/},responseFields:{xml:"responseXML",text:"responseText",json:"responseJSON"},converters:{"* text":String,"text html":!0,"text json":JSON.parse,"text xml":r.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(a,b){return b?Ob(Ob(a,r.ajaxSettings),b):Ob(r.ajaxSettings,a)},ajaxPrefilter:Mb(Ib),ajaxTransport:Mb(Jb),ajax:function(b,c){"object"==typeof b&&(c=b,b=void 0),c=c||{};var e,f,g,h,i,j,k,l,m,n,o=r.ajaxSetup({},c),p=o.context||o,q=o.context&&(p.nodeType||p.jquery)?r(p):r.event,s=r.Deferred(),t=r.Callbacks("once memory"),u=o.statusCode||{},v={},w={},x="canceled",y={readyState:0,getResponseHeader:function(a){var b;if(k){if(!h){h={};while(b=Eb.exec(g))h[b[1].toLowerCase()]=b[2]}b=h[a.toLowerCase()]}return null==b?null:b},getAllResponseHeaders:function(){return k?g:null},setRequestHeader:function(a,b){return null==k&&(a=w[a.toLowerCase()]=w[a.toLowerCase()]||a,v[a]=b),this},overrideMimeType:function(a){return null==k&&(o.mimeType=a),this},statusCode:function(a){var b;if(a)if(k)y.always(a[y.status]);else for(b in a)u[b]=[u[b],a[b]];return this},abort:function(a){var b=a||x;return e&&e.abort(b),A(0,b),this}};if(s.promise(y),o.url=((b||o.url||tb.href)+"").replace(Hb,tb.protocol+"//"),o.type=c.method||c.type||o.method||o.type,o.dataTypes=(o.dataType||"*").toLowerCase().match(L)||[""],null==o.crossDomain){j=d.createElement("a");try{j.href=o.url,j.href=j.href,o.crossDomain=Lb.protocol+"//"+Lb.host!=j.protocol+"//"+j.host}catch(z){o.crossDomain=!0}}if(o.data&&o.processData&&"string"!=typeof o.data&&(o.data=r.param(o.data,o.traditional)),Nb(Ib,o,c,y),k)return y;l=r.event&&o.global,l&&0===r.active++&&r.event.trigger("ajaxStart"),o.type=o.type.toUpperCase(),o.hasContent=!Gb.test(o.type),f=o.url.replace(Cb,""),o.hasContent?o.data&&o.processData&&0===(o.contentType||"").indexOf("application/x-www-form-urlencoded")&&(o.data=o.data.replace(Bb,"+")):(n=o.url.slice(f.length),o.data&&(f+=(vb.test(f)?"&":"?")+o.data,delete o.data),o.cache===!1&&(f=f.replace(Db,"$1"),n=(vb.test(f)?"&":"?")+"_="+ub++ +n),o.url=f+n),o.ifModified&&(r.lastModified[f]&&y.setRequestHeader("If-Modified-Since",r.lastModified[f]),r.etag[f]&&y.setRequestHeader("If-None-Match",r.etag[f])),(o.data&&o.hasContent&&o.contentType!==!1||c.contentType)&&y.setRequestHeader("Content-Type",o.contentType),y.setRequestHeader("Accept",o.dataTypes[0]&&o.accepts[o.dataTypes[0]]?o.accepts[o.dataTypes[0]]+("*"!==o.dataTypes[0]?", "+Kb+"; q=0.01":""):o.accepts["*"]);for(m in o.headers)y.setRequestHeader(m,o.headers[m]);if(o.beforeSend&&(o.beforeSend.call(p,y,o)===!1||k))return y.abort();if(x="abort",t.add(o.complete),y.done(o.success),y.fail(o.error),e=Nb(Jb,o,c,y)){if(y.readyState=1,l&&q.trigger("ajaxSend",[y,o]),k)return y;o.async&&o.timeout>0&&(i=a.setTimeout(function(){y.abort("timeout")},o.timeout));try{k=!1,e.send(v,A)}catch(z){if(k)throw z;A(-1,z)}}else A(-1,"No Transport");function A(b,c,d,h){var j,m,n,v,w,x=c;k||(k=!0,i&&a.clearTimeout(i),e=void 0,g=h||"",y.readyState=b>0?4:0,j=b>=200&&b<300||304===b,d&&(v=Pb(o,y,d)),v=Qb(o,v,y,j),j?(o.ifModified&&(w=y.getResponseHeader("Last-Modified"),w&&(r.lastModified[f]=w),w=y.getResponseHeader("etag"),w&&(r.etag[f]=w)),204===b||"HEAD"===o.type?x="nocontent":304===b?x="notmodified":(x=v.state,m=v.data,n=v.error,j=!n)):(n=x,!b&&x||(x="error",b<0&&(b=0))),y.status=b,y.statusText=(c||x)+"",j?s.resolveWith(p,[m,x,y]):s.rejectWith(p,[y,x,n]),y.statusCode(u),u=void 0,l&&q.trigger(j?"ajaxSuccess":"ajaxError",[y,o,j?m:n]),t.fireWith(p,[y,x]),l&&(q.trigger("ajaxComplete",[y,o]),--r.active||r.event.trigger("ajaxStop")))}return y},getJSON:function(a,b,c){return r.get(a,b,c,"json")},getScript:function(a,b){return r.get(a,void 0,b,"script")}}),r.each(["get","post"],function(a,b){r[b]=function(a,c,d,e){return r.isFunction(c)&&(e=e||d,d=c,c=void 0),r.ajax(r.extend({url:a,type:b,dataType:e,data:c,success:d},r.isPlainObject(a)&&a))}}),r._evalUrl=function(a){return r.ajax({url:a,type:"GET",dataType:"script",cache:!0,async:!1,global:!1,"throws":!0})},r.fn.extend({wrapAll:function(a){var b;return this[0]&&(r.isFunction(a)&&(a=a.call(this[0])),b=r(a,this[0].ownerDocument).eq(0).clone(!0),this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstElementChild)a=a.firstElementChild;return a}).append(this)),this},wrapInner:function(a){return r.isFunction(a)?this.each(function(b){r(this).wrapInner(a.call(this,b))}):this.each(function(){var b=r(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){var b=r.isFunction(a);return this.each(function(c){r(this).wrapAll(b?a.call(this,c):a)})},unwrap:function(a){return this.parent(a).not("body").each(function(){r(this).replaceWith(this.childNodes)}),this}}),r.expr.pseudos.hidden=function(a){return!r.expr.pseudos.visible(a)},r.expr.pseudos.visible=function(a){return!!(a.offsetWidth||a.offsetHeight||a.getClientRects().length)},r.ajaxSettings.xhr=function(){try{return new a.XMLHttpRequest}catch(b){}};var Rb={0:200,1223:204},Sb=r.ajaxSettings.xhr();o.cors=!!Sb&&"withCredentials"in Sb,o.ajax=Sb=!!Sb,r.ajaxTransport(function(b){var c,d;if(o.cors||Sb&&!b.crossDomain)return{send:function(e,f){var g,h=b.xhr();if(h.open(b.type,b.url,b.async,b.username,b.password),b.xhrFields)for(g in b.xhrFields)h[g]=b.xhrFields[g];b.mimeType&&h.overrideMimeType&&h.overrideMimeType(b.mimeType),b.crossDomain||e["X-Requested-With"]||(e["X-Requested-With"]="XMLHttpRequest");for(g in e)h.setRequestHeader(g,e[g]);c=function(a){return function(){c&&(c=d=h.onload=h.onerror=h.onabort=h.onreadystatechange=null,"abort"===a?h.abort():"error"===a?"number"!=typeof h.status?f(0,"error"):f(h.status,h.statusText):f(Rb[h.status]||h.status,h.statusText,"text"!==(h.responseType||"text")||"string"!=typeof h.responseText?{binary:h.response}:{text:h.responseText},h.getAllResponseHeaders()))}},h.onload=c(),d=h.onerror=c("error"),void 0!==h.onabort?h.onabort=d:h.onreadystatechange=function(){4===h.readyState&&a.setTimeout(function(){c&&d()})},c=c("abort");try{h.send(b.hasContent&&b.data||null)}catch(i){if(c)throw i}},abort:function(){c&&c()}}}),r.ajaxPrefilter(function(a){a.crossDomain&&(a.contents.script=!1)}),r.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/\b(?:java|ecma)script\b/},converters:{"text script":function(a){return r.globalEval(a),a}}}),r.ajaxPrefilter("script",function(a){void 0===a.cache&&(a.cache=!1),a.crossDomain&&(a.type="GET")}),r.ajaxTransport("script",function(a){if(a.crossDomain){var b,c;return{send:function(e,f){b=r("